From e6e696cd6f4b9e7a104e4625fadc91e608c8a416 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Tue, 19 Jun 2018 15:51:54 +0200 Subject: [PATCH 001/808] Bump version --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index d11d76608dfdf..1bcaa55bb9507 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '2.0.0dev0+incubating' +version = '1.10.0beta2+incubating' From 859ec64aee98c0294229b3c29f57f3c89b6ad533 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 27 Jun 2018 22:07:31 +0200 Subject: [PATCH 002/808] [AIRFLOW-1840] Support back-compat on old celery config The new names are in-line with Celery 4, but if anyone upgrades Airflow without following the UPDATING.md instructions (which we probably assume most people won't, not until something stops working) their workers would suddenly just start failing. That's bad. This will issue a warning but carry on working as expected. We can remove the deprecation settings (but leave the code in config) after this release has been made. Closes #3549 from ashb/AIRFLOW-1840-back-compat (cherry picked from commit a4592f91eb125d7b463f1762fbfe0f64f7bc6b3a) Signed-off-by: Bolke de Bruin --- UPDATING.md | 3 +++ airflow/configuration.py | 53 +++++++++++++++++++++++++++++++++++++++- setup.py | 2 +- tests/configuration.py | 40 +++++++++++++++++++++++++++++- 4 files changed, 95 insertions(+), 3 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 3a66e735c3ac4..c341b38f0bc22 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -56,6 +56,9 @@ To make the config of Airflow compatible with Celery, some properties have been ``` celeryd_concurrency -> worker_concurrency celery_result_backend -> result_backend +celery_ssl_active -> ssl_active +celery_ssl_cert -> ssl_cert +celery_ssl_key -> ssl_key ``` Resulting in the same config parameters as Celery 4, with more transparency. diff --git a/airflow/configuration.py b/airflow/configuration.py index 2ee453fd7d93f..e2089e5b5b94b 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -121,9 +121,29 @@ class AirflowConfigParser(ConfigParser): ('core', 'sql_alchemy_conn'), ('core', 'fernet_key'), ('celery', 'broker_url'), - ('celery', 'result_backend') + ('celery', 'result_backend'), + # Todo: remove this in Airflow 1.11 + ('celery', 'celery_result_backend'), } + # A two-level mapping of (section -> new_name -> old_name). When reading + # new_name, the old_name will be checked to see if it exists. If it does a + # DeprecationWarning will be issued and the old name will be used instead + deprecated_options = { + 'celery': { + # Remove these keys in Airflow 1.11 + 'worker_concurrency': 'celeryd_concurrency', + 'broker_url': 'celery_broker_url', + 'ssl_active': 'celery_ssl_active', + 'ssl_cert': 'celery_ssl_cert', + 'ssl_key': 'celery_ssl_key', + } + } + deprecation_format_string = ( + 'The {old} option in [{section}] has been renamed to {new} - the old ' + 'setting has been used, but please update your config.' + ) + def __init__(self, default_config=None, *args, **kwargs): super(AirflowConfigParser, self).__init__(*args, **kwargs) @@ -181,10 +201,17 @@ def get(self, section, key, **kwargs): section = str(section).lower() key = str(key).lower() + deprecated_name = self.deprecated_options.get(section, {}).get(key, None) + # first check environment variables option = self._get_env_var_option(section, key) if option is not None: return option + if deprecated_name: + option = self._get_env_var_option(section, deprecated_name) + if option is not None: + self._warn_deprecate(section, key, deprecated_name) + return option # ...then the config file if super(AirflowConfigParser, self).has_option(section, key): @@ -192,11 +219,24 @@ def get(self, section, key, **kwargs): # separate the config from default config. return expand_env_var( super(AirflowConfigParser, self).get(section, key, **kwargs)) + if deprecated_name: + if super(AirflowConfigParser, self).has_option(section, deprecated_name): + self._warn_deprecate(section, key, deprecated_name) + return expand_env_var(super(AirflowConfigParser, self).get( + section, + deprecated_name, + **kwargs + )) # ...then commands option = self._get_cmd_option(section, key) if option: return option + if deprecated_name: + option = self._get_cmd_option(section, deprecated_name) + if option: + self._warn_deprecate(section, key, deprecated_name) + return option # ...then the default config if self.defaults.has_option(section, key): @@ -352,6 +392,17 @@ def load_test_config(self): # then read any "custom" test settings self.read(TEST_CONFIG_FILE) + def _warn_deprecate(self, section, key, deprecated_name): + warnings.warn( + self.deprecation_format_string.format( + old=deprecated_name, + new=key, + section=section, + ), + DeprecationWarning, + stacklevel=3, + ) + def mkdir_p(path): try: diff --git a/setup.py b/setup.py index 0b012eec0724f..a1b3b1dc6ab6c 100644 --- a/setup.py +++ b/setup.py @@ -230,7 +230,7 @@ def write_version(filename=os.path.join(*['airflow', devel_ci = [package for package in devel_all if package not in ['snakebite>=2.7.8', 'snakebite[kerberos]>=2.7.8']] else: - devel_ci = devel_all + devel_ci = devel_all + ['unittest2'] def do_setup(): diff --git a/tests/configuration.py b/tests/configuration.py index ca1d1d7a37df6..ac6f7b7db7746 100644 --- a/tests/configuration.py +++ b/tests/configuration.py @@ -20,7 +20,7 @@ from __future__ import print_function from __future__ import unicode_literals -import unittest +import os from collections import OrderedDict import six @@ -28,6 +28,12 @@ from airflow import configuration from airflow.configuration import conf, AirflowConfigParser, parameterized_config +if six.PY2: + # Need `assertWarns` back-ported from unittest2 + import unittest2 as unittest +else: + import unittest + class ConfTest(unittest.TestCase): @@ -154,3 +160,35 @@ def test_broker_transport_options(self): self.assertTrue(isinstance(section_dict['_test_only_float'], float)) self.assertTrue(isinstance(section_dict['_test_only_string'], six.string_types)) + + def test_deprecated_options(self): + # Guarantee we have a deprecated setting, so we test the deprecation + # lookup even if we remove this explicit fallback + conf.deprecated_options['celery'] = { + 'worker_concurrency': 'celeryd_concurrency', + } + + # Remove it so we are sure we use the right setting + conf.remove_option('celery', 'worker_concurrency') + + with self.assertWarns(DeprecationWarning): + os.environ['AIRFLOW__CELERY__CELERYD_CONCURRENCY'] = '99' + self.assertEquals(conf.getint('celery', 'worker_concurrency'), 99) + os.environ.pop('AIRFLOW__CELERY__CELERYD_CONCURRENCY') + + with self.assertWarns(DeprecationWarning): + conf.set('celery', 'celeryd_concurrency', '99') + self.assertEquals(conf.getint('celery', 'worker_concurrency'), 99) + conf.remove_option('celery', 'celeryd_concurrency') + + def test_deprecated_options_cmd(self): + # Guarantee we have a deprecated setting, so we test the deprecation + # lookup even if we remove this explicit fallback + conf.deprecated_options['celery'] = {'result_backend': 'celery_result_backend'} + conf.as_command_stdout.add(('celery', 'celery_result_backend')) + + conf.remove_option('celery', 'result_backend') + conf.set('celery', 'celery_result_backend_cmd', '/bin/echo 99') + + with self.assertWarns(DeprecationWarning): + self.assertEquals(conf.getint('celery', 'result_backend'), 99) From 683f46688bc9228c70b3b6c96486280b29542dc4 Mon Sep 17 00:00:00 2001 From: David Hatch Date: Tue, 26 Jun 2018 15:26:13 -0700 Subject: [PATCH 003/808] [AIRFLOW-2681] Include last dag run of externally triggered DAGs in UI. Closes #3551 from dhatch/AIRFLOW-2681 (cherry picked from commit 78f3d33388c772eafbed8fff81b0e50188297fc6) Signed-off-by: Bolke de Bruin --- airflow/www/templates/airflow/dags.html | 2 +- airflow/www_rbac/templates/airflow/dags.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/www/templates/airflow/dags.html b/airflow/www/templates/airflow/dags.html index 23978905780ea..217b39dcbb4b7 100644 --- a/airflow/www/templates/airflow/dags.html +++ b/airflow/www/templates/airflow/dags.html @@ -119,7 +119,7 @@

DAGs

{% if dag %} - {% set last_run = dag.get_last_dagrun() %} + {% set last_run = dag.get_last_dagrun(include_externally_triggered=True) %} {% if last_run and last_run.execution_date %} {{ last_run.execution_date.strftime("%Y-%m-%d %H:%M") }} diff --git a/airflow/www_rbac/templates/airflow/dags.html b/airflow/www_rbac/templates/airflow/dags.html index a712e5a9ee4bc..a7a7467de1f68 100644 --- a/airflow/www_rbac/templates/airflow/dags.html +++ b/airflow/www_rbac/templates/airflow/dags.html @@ -120,7 +120,7 @@

DAGs

{% if dag %} - {% set last_run = dag.get_last_dagrun() %} + {% set last_run = dag.get_last_dagrun(include_externally_triggered=True) %} {% if last_run and last_run.execution_date %}
{{ last_run.execution_date.strftime("%Y-%m-%d %H:%M") }} From bd4937b56efcffb117f266d1a8c523c82e023e18 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 27 Jun 2018 22:11:43 +0200 Subject: [PATCH 004/808] [AIRFLOW-2668] Handle missing optional cryptography dependency cryptography is a recommended, but optional dependency. It was mistakenly made a hard dependency by a refactor. This restores that behaviour (though without tests, as it's hard to test that in a unittest) In testing this I found that running `airflow initdb` would end up printing the "crypto is missing" message 15 times, making it hard to see what was actually going on. So I have re-worked `get_fernet()` to only compute (and warn) once. This also makes the consuming code easier as in the case of the dep not being installed we still have a class that presents the same interface as Fernet. Closes #3550 from ashb/AIRFLOW-2668-optional- cryptography (cherry picked from commit fa6c35743a5129057d870752554b0827dd7d462f) Signed-off-by: Bolke de Bruin --- airflow/models.py | 111 ++++++++++++++++++++++++++-------------------- 1 file changed, 63 insertions(+), 48 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 4706c2d50b6e1..4b7e123b66aa1 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -24,11 +24,9 @@ from future.standard_library import install_aliases -from builtins import str -from builtins import object, bytes +from builtins import str, object, bytes, ImportError as BuiltinImportError import copy from collections import namedtuple, defaultdict -import cryptography from datetime import timedelta import dill @@ -104,6 +102,33 @@ Stats = settings.Stats +class InvalidFernetToken(Exception): + # If Fernet isn't loaded we need a valid exception class to catch. If it is + # loaded this will get reset to the actual class once get_fernet() is called + pass + + +class NullFernet(object): + """ + A "Null" encryptor class that doesn't encrypt or decrypt but that presents + a similar interface to Fernet. + + The purpose of this is to make the rest of the code not have to know the + difference, and to only display the message once, not 20 times when + `airflow initdb` is ran. + """ + is_encrypted = False + + def decrpyt(self, b): + return b + + def encrypt(self, b): + return b + + +_fernet = None + + def get_fernet(): """ Deferred load of Fernet key. @@ -114,12 +139,25 @@ def get_fernet(): :return: Fernet object :raises: AirflowException if there's a problem trying to load Fernet """ + global _fernet + if _fernet: + return _fernet try: - from cryptography.fernet import Fernet - except ImportError: - raise AirflowException('Failed to import Fernet, it may not be installed') + from cryptography.fernet import Fernet, InvalidToken + global InvalidFernetToken + InvalidFernetToken = InvalidToken + + except BuiltinImportError: + LoggingMixin().log.warn("cryptography not found - values will not be stored " + "encrypted.", + exc_info=1) + _fernet = NullFernet() + return _fernet + try: - return Fernet(configuration.conf.get('core', 'FERNET_KEY').encode('utf-8')) + _fernet = Fernet(configuration.conf.get('core', 'FERNET_KEY').encode('utf-8')) + _fernet.is_encrypted = True + return _fernet except (ValueError, TypeError) as ve: raise AirflowException("Could not create Fernet object: {}".format(ve)) @@ -673,9 +711,8 @@ def parse_from_uri(self, uri): def get_password(self): if self._password and self.is_encrypted: - try: - fernet = get_fernet() - except AirflowException: + fernet = get_fernet() + if not fernet.is_encrypted: raise AirflowException( "Can't decrypt encrypted password for login={}, \ FERNET_KEY configuration is missing".format(self.login)) @@ -685,15 +722,9 @@ def get_password(self): def set_password(self, value): if value: - try: - fernet = get_fernet() - self._password = fernet.encrypt(bytes(value, 'utf-8')).decode() - self.is_encrypted = True - except AirflowException: - self.log.exception("Failed to load fernet while encrypting value, " - "using non-encrypted value.") - self._password = value - self.is_encrypted = False + fernet = get_fernet() + self._password = fernet.encrypt(bytes(value, 'utf-8')).decode() + self.is_encrypted = fernet.is_encrypted @declared_attr def password(cls): @@ -702,9 +733,8 @@ def password(cls): def get_extra(self): if self._extra and self.is_extra_encrypted: - try: - fernet = get_fernet() - except AirflowException: + fernet = get_fernet() + if not fernet.is_encrypted: raise AirflowException( "Can't decrypt `extra` params for login={},\ FERNET_KEY configuration is missing".format(self.login)) @@ -714,15 +744,9 @@ def get_extra(self): def set_extra(self, value): if value: - try: - fernet = get_fernet() - self._extra = fernet.encrypt(bytes(value, 'utf-8')).decode() - self.is_extra_encrypted = True - except AirflowException: - self.log.exception("Failed to load fernet while encrypting value, " - "using non-encrypted value.") - self._extra = value - self.is_extra_encrypted = False + fernet = get_fernet() + self._extra = fernet.encrypt(bytes(value, 'utf-8')).decode() + self.is_extra_encrypted = fernet.is_encrypted else: self._extra = value self.is_extra_encrypted = False @@ -4366,32 +4390,23 @@ def get_val(self): if self._val and self.is_encrypted: try: fernet = get_fernet() - except Exception: - log.error("Can't decrypt _val for key={}, FERNET_KEY " - "configuration missing".format(self.key)) - return None - try: return fernet.decrypt(bytes(self._val, 'utf-8')).decode() - except cryptography.fernet.InvalidToken: + except InvalidFernetToken: log.error("Can't decrypt _val for key={}, invalid token " "or value".format(self.key)) return None + except Exception: + log.error("Can't decrypt _val for key={}, FERNET_KEY " + "configuration missing".format(self.key)) + return None else: return self._val def set_val(self, value): if value: - try: - fernet = get_fernet() - self._val = fernet.encrypt(bytes(value, 'utf-8')).decode() - self.is_encrypted = True - except AirflowException: - self.log.exception( - "Failed to load fernet while encrypting value, " - "using non-encrypted value." - ) - self._val = value - self.is_encrypted = False + fernet = get_fernet() + self._val = fernet.encrypt(bytes(value, 'utf-8')).decode() + self.is_encrypted = fernet.is_encrypted @declared_attr def val(cls): From c01a12eda27609c7150ec05d91aed4c32368ac5c Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 20 Jun 2018 23:40:04 +0100 Subject: [PATCH 005/808] [AIRFLOW-2654] Fix incorret URL on refresh in Graph View of FAB UI - Fix incorrect URL requested on pressing the refresh button in the graph view. (cherry picked from commit 8dc54e6e0034eec1511453e168499931348f684c) Signed-off-by: Bolke de Bruin --- airflow/www_rbac/templates/airflow/graph.html | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/www_rbac/templates/airflow/graph.html b/airflow/www_rbac/templates/airflow/graph.html index 35f79b6d71edb..d56c5b760336e 100644 --- a/airflow/www_rbac/templates/airflow/graph.html +++ b/airflow/www_rbac/templates/airflow/graph.html @@ -1,13 +1,13 @@ -{# +{# Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at - + http://www.apache.org/licenses/LICENSE-2.0 - + Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -350,7 +350,7 @@ $("#loading").css("display", "block"); $("div#svg_container").css("opacity", "0.2"); $.get( - "/airflow/object/task_instances", + "{{ url_for('Airflow.task_instances') }}", {dag_id : "{{ dag.dag_id }}", execution_date : "{{ execution_date }}"}) .done( function(task_instances) { From 7c5157a9366dfb118bccb0c900313ea46e5eb80f Mon Sep 17 00:00:00 2001 From: Kevin Yang Date: Thu, 14 Jun 2018 18:20:46 -0700 Subject: [PATCH 006/808] [AIRFLOW-2624] Fix webserver login as anonymous (cherry picked from commit 2fd9328b412841429acc288b1441c8351ee15e98) Signed-off-by: Bolke de Bruin --- airflow/www/utils.py | 6 +-- tests/www/test_utils.py | 85 ++++++++++++++++++++++++++++++++++++++++- 2 files changed, 86 insertions(+), 5 deletions(-) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 7d9c8a07501c7..44fa5c4dcd6fb 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -246,8 +246,8 @@ def action_logging(f): """ @functools.wraps(f) def wrapper(*args, **kwargs): - # Only AnonymousUserMixin() does not have user attribute - if current_user and hasattr(current_user, 'user'): + # AnonymousUserMixin() has user attribute but its value is None. + if current_user and hasattr(current_user, 'user') and current_user.user: user = current_user.user.username else: user = 'anonymous' @@ -286,7 +286,7 @@ def wrapper(*args, **kwargs): dag = dagbag.get_dag(dag_id) task = dag.get_task(task_id) - if current_user and hasattr(current_user, 'username'): + if current_user and hasattr(current_user, 'user') and current_user.user: user = current_user.username else: user = 'anonymous' diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py index d69041a368af4..9d788e88f1ed9 100644 --- a/tests/www/test_utils.py +++ b/tests/www/test_utils.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -17,9 +17,12 @@ # specific language governing permissions and limitations # under the License. +import mock import unittest from xml.dom import minidom +from airflow.www import app as application + from airflow.www import utils @@ -109,6 +112,84 @@ def test_params_all(self): self.assertEqual('page=3&search=bash_&showPaused=False', utils.get_params(showPaused=False, page=3, search='bash_')) + # flask_login is loaded by calling flask_login._get_user. + @mock.patch("flask_login._get_user") + @mock.patch("airflow.settings.Session") + def test_action_logging_with_login_user(self, mocked_session, mocked_get_user): + fake_username = 'someone' + mocked_current_user = mock.MagicMock() + mocked_get_user.return_value = mocked_current_user + mocked_current_user.user.username = fake_username + mocked_session_instance = mock.MagicMock() + mocked_session.return_value = mocked_session_instance + + app = application.create_app(testing=True) + # Patching here to avoid errors in applicant.create_app + with mock.patch("airflow.models.Log") as mocked_log: + with app.test_request_context(): + @utils.action_logging + def some_func(): + pass + + some_func() + mocked_log.assert_called_once() + (args, kwargs) = mocked_log.call_args_list[0] + self.assertEqual('some_func', kwargs['event']) + self.assertEqual(fake_username, kwargs['owner']) + mocked_session_instance.add.assert_called_once() + + @mock.patch("flask_login._get_user") + @mock.patch("airflow.settings.Session") + def test_action_logging_with_invalid_user(self, mocked_session, mocked_get_user): + anonymous_username = 'anonymous' + + # When the user returned by flask login_manager._load_user + # is invalid. + mocked_current_user = mock.MagicMock() + mocked_get_user.return_value = mocked_current_user + mocked_current_user.user = None + mocked_session_instance = mock.MagicMock() + mocked_session.return_value = mocked_session_instance + + app = application.create_app(testing=True) + # Patching here to avoid errors in applicant.create_app + with mock.patch("airflow.models.Log") as mocked_log: + with app.test_request_context(): + @utils.action_logging + def some_func(): + pass + + some_func() + mocked_log.assert_called_once() + (args, kwargs) = mocked_log.call_args_list[0] + self.assertEqual('some_func', kwargs['event']) + self.assertEqual(anonymous_username, kwargs['owner']) + mocked_session_instance.add.assert_called_once() + + # flask_login.current_user would be AnonymousUserMixin + # when there's no user_id in the flask session. + @mock.patch("airflow.settings.Session") + def test_action_logging_with_anonymous_user(self, mocked_session): + anonymous_username = 'anonymous' + + mocked_session_instance = mock.MagicMock() + mocked_session.return_value = mocked_session_instance + + app = application.create_app(testing=True) + # Patching here to avoid errors in applicant.create_app + with mock.patch("airflow.models.Log") as mocked_log: + with app.test_request_context(): + @utils.action_logging + def some_func(): + pass + + some_func() + mocked_log.assert_called_once() + (args, kwargs) = mocked_log.call_args_list[0] + self.assertEqual('some_func', kwargs['event']) + self.assertEqual(anonymous_username, kwargs['owner']) + mocked_session_instance.add.assert_called_once() + if __name__ == '__main__': unittest.main() From 90f05cf3351aa3bfd1201b2906062b762417d9f9 Mon Sep 17 00:00:00 2001 From: Tao feng Date: Tue, 26 Jun 2018 00:23:11 -0700 Subject: [PATCH 007/808] [AIRFLOW-2678] Fix db schema unit test to remove checking fab models (cherry picked from commit 0c2206c7d617fe4925ece6478dd5b6caf5b179ba) Signed-off-by: Bolke de Bruin --- tests/utils/test_db.py | 83 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 tests/utils/test_db.py diff --git a/tests/utils/test_db.py b/tests/utils/test_db.py new file mode 100644 index 0000000000000..8ddd3ef715368 --- /dev/null +++ b/tests/utils/test_db.py @@ -0,0 +1,83 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from airflow.models import Base as airflow_base + +from airflow.settings import engine +from alembic.autogenerate import compare_metadata +from alembic.migration import MigrationContext +from sqlalchemy import MetaData + + +class DbTest(unittest.TestCase): + + def test_database_schema_and_sqlalchemy_model_are_in_sync(self): + all_meta_data = MetaData() + for (table_name, table) in airflow_base.metadata.tables.items(): + all_meta_data._add_table(table_name, table.schema, table) + + # create diff between database schema and SQLAlchemy model + mc = MigrationContext.configure(engine.connect()) + diff = compare_metadata(mc, all_meta_data) + + # known diffs to ignore + ignores = [ + # users.password is not part of User model, + # otherwise it would show up in (old) UI + lambda t: (t[0] == 'remove_column' and + t[2] == 'users' and + t[3].name == 'password'), + # ignore tables created by other tests + lambda t: (t[0] == 'remove_table' and + t[1].name == 't'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'test_airflow'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'test_postgres_to_postgres'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'test_mysql_to_mysql'), + # ignore tables created by celery + lambda t: (t[0] == 'remove_table' and + t[1].name == 'celery_taskmeta'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'celery_tasksetmeta'), + # Ignore all the fab tables + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_permission'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_register_user'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_role'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_permission_view'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_permission_view_role'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_user_role'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_user'), + lambda t: (t[0] == 'remove_table' and + t[1].name == 'ab_view_menu'), + ] + for ignore in ignores: + diff = [d for d in diff if not ignore(d)] + + self.assertFalse(diff, 'Database schema and SQLAlchemy model are not in sync') From acfc72e01a5dd24a86513ab93e33b8856ad0e284 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 27 Jun 2018 22:34:16 +0200 Subject: [PATCH 008/808] [AIRFLOW-2650] Mark SchedulerJob as succeed when hitting Ctrl-c Without this fix it turns out that the job would remain in the running state. This also sets things to failed in case of any other exception. Closes #3525 from ashb/scheduler-job-status (cherry picked from commit b0061f1369636c86df87829b93d5ece582591e2a) Signed-off-by: Bolke de Bruin --- airflow/jobs.py | 22 ++++++++++++++-------- tests/jobs.py | 42 +++++++++++++++++++++++++++++++++++++++++- 2 files changed, 55 insertions(+), 9 deletions(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index ad114abda3d6f..5143e8e73e23e 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -197,14 +197,20 @@ def run(self): make_transient(self) self.id = id_ - # Run - self._execute() - - # Marking the success in the DB - self.end_date = timezone.utcnow() - self.state = State.SUCCESS - session.merge(self) - session.commit() + try: + self._execute() + # In case of max runs or max duration + self.state = State.SUCCESS + except SystemExit as e: + # In case of ^C or SIGTERM + self.state = State.SUCCESS + except Exception as e: + self.state = State.FAILED + raise + finally: + self.end_date = timezone.utcnow() + session.merge(self) + session.commit() Stats.incr(self.__class__.__name__.lower() + '_end', 1, 1) diff --git a/tests/jobs.py b/tests/jobs.py index f534b6540aa41..5dd6ff3efda9e 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -39,7 +39,7 @@ from airflow import AirflowException, settings, models from airflow.bin import cli from airflow.executors import BaseExecutor, SequentialExecutor -from airflow.jobs import BackfillJob, SchedulerJob, LocalTaskJob +from airflow.jobs import BaseJob, BackfillJob, SchedulerJob, LocalTaskJob from airflow.models import DAG, DagModel, DagBag, DagRun, Pool, TaskInstance as TI from airflow.operators.dummy_operator import DummyOperator from airflow.operators.bash_operator import BashOperator @@ -86,6 +86,46 @@ os.path.dirname(os.path.realpath(__file__)), 'dags') +class BaseJobTest(unittest.TestCase): + class TestJob(BaseJob): + __mapper_args__ = { + 'polymorphic_identity': 'TestJob' + } + + def __init__(self, cb): + self.cb = cb + super(BaseJobTest.TestJob, self).__init__() + + def _execute(self): + return self.cb() + + def test_state_success(self): + job = self.TestJob(lambda: True) + job.run() + + self.assertEquals(job.state, State.SUCCESS) + self.assertIsNotNone(job.end_date) + + def test_state_sysexit(self): + import sys + job = self.TestJob(lambda: sys.exit(0)) + job.run() + + self.assertEquals(job.state, State.SUCCESS) + self.assertIsNotNone(job.end_date) + + def test_state_failed(self): + def abort(): + raise RuntimeError("fail") + + job = self.TestJob(abort) + with self.assertRaises(RuntimeError): + job.run() + + self.assertEquals(job.state, State.FAILED) + self.assertIsNotNone(job.end_date) + + class BackfillJobTest(unittest.TestCase): def setUp(self): From 2c336f0f11ff4b6c2cf80881ef54c5f65b6730e4 Mon Sep 17 00:00:00 2001 From: Stefan Seelmann Date: Sun, 17 Jun 2018 22:17:11 +0200 Subject: [PATCH 009/808] [AIRFLOW-2604] Add index to task_fail (cherry picked from commit d00762cb914803341c5d019d14385d249346d601) Signed-off-by: Bolke de Bruin --- .../versions/9635ae0956e7_index_faskfail.py | 44 +++++++++++++++++++ airflow/models.py | 5 +++ 2 files changed, 49 insertions(+) create mode 100644 airflow/migrations/versions/9635ae0956e7_index_faskfail.py diff --git a/airflow/migrations/versions/9635ae0956e7_index_faskfail.py b/airflow/migrations/versions/9635ae0956e7_index_faskfail.py new file mode 100644 index 0000000000000..da69846233e87 --- /dev/null +++ b/airflow/migrations/versions/9635ae0956e7_index_faskfail.py @@ -0,0 +1,44 @@ +# flake8: noqa +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""index-faskfail + +Revision ID: 9635ae0956e7 +Revises: 856955da8476 +Create Date: 2018-06-17 21:40:01.963540 + +""" + +# revision identifiers, used by Alembic. +revision = '9635ae0956e7' +down_revision = '856955da8476' +branch_labels = None +depends_on = None + +from alembic import op +import sqlalchemy as sa + + +def upgrade(): + op.create_index('idx_task_fail_dag_task_date', 'task_fail', ['dag_id', 'task_id', 'execution_date'], unique=False) + + +def downgrade(): + op.drop_index('idx_task_fail_dag_task_date', table_name='task_fail') + diff --git a/airflow/models.py b/airflow/models.py index 4b7e123b66aa1..b0841e6369726 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -2079,6 +2079,11 @@ class TaskFail(Base): end_date = Column(UtcDateTime) duration = Column(Float) + __table_args__ = ( + Index('idx_task_fail_dag_task_date', dag_id, task_id, execution_date, + unique=False), + ) + def __init__(self, task, execution_date, start_date, end_date): self.dag_id = task.dag_id self.task_id = task.task_id From f45a5c8ed47605974775cc89799433f5a793ee3b Mon Sep 17 00:00:00 2001 From: Tim Swast Date: Tue, 12 Jun 2018 23:53:21 +0100 Subject: [PATCH 010/808] [AIRFLOW-2512][AIRFLOW-2522] Use google-auth instead of oauth2client * Updates the GCP hooks to use the google-auth library and removes dependencies on the deprecated oauth2client package. * Removes inconsistent handling of the scope parameter for different auth methods. Note: using google-auth for credentials requires a newer version of the google-api-python-client package, so this commit also updates the minimum version for that. To avoid some annoying warnings about the discovery cache not being supported, so disable the discovery cache explicitly as recommend here: https://stackoverflow.com/a/44518587/101923 Tested by running: nosetests tests/contrib/operators/test_dataflow_operator.py \ tests/contrib/operators/test_gcs*.py \ tests/contrib/operators/test_mlengine_*.py \ tests/contrib/operators/test_pubsub_operator.py \ tests/contrib/hooks/test_gcp*.py \ tests/contrib/hooks/test_gcs_hook.py \ tests/contrib/hooks/test_bigquery_hook.py and also tested by running some GCP-related DAGs locally, such as the Dataproc DAG example at https://cloud.google.com/composer/docs/quickstart Closes #3488 from tswast/google-auth (cherry picked from commit 0f4d681f6f6e15acd1399dede146e75cb688d536) Signed-off-by: Bolke de Bruin From 4add288ccbf296bc64e3c8cdc72a7d9d071293b6 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Wed, 27 Jun 2018 22:56:42 +0200 Subject: [PATCH 011/808] Bump beta3 --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index 1bcaa55bb9507..aa28ee9f952f9 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.0beta2+incubating' +version = '1.10.0beta3+incubating' From 80fd3125044a62759b6ab19a046078c7f568c6a8 Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Tue, 19 Jun 2018 17:09:00 -0700 Subject: [PATCH 012/808] [AIRFLOW-2646] Fix setup.py not to install snakebite on Python3 setup.py has a logic to avoid installing snakebite on Python3, but it doesn't work. This is because the variable devel_all is now a tuple of lists. This PR fixes that variable to be a flat list and makes the logic work as expected. Closes #3522 from sekikn/AIRFLOW-2646 (cherry picked from commit 105ac150580af0a1a64bb5e0417fc57b2a0e49bb) Signed-off-by: Bolke de Bruin --- setup.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a1b3b1dc6ab6c..c91750057cef0 100644 --- a/setup.py +++ b/setup.py @@ -223,7 +223,8 @@ def write_version(filename=os.path.join(*['airflow', devel_all = (sendgrid + devel + all_dbs + doc + samba + s3 + slack + crypto + oracle + docker + ssh + kubernetes + celery + azure_blob_storage + redis + gcp_api + datadog + zendesk + jdbc + ldap + kerberos + password + webhdfs + jenkins + - druid + pinot + segment + snowflake + elasticsearch + azure_data_lake, atlas) + druid + pinot + segment + snowflake + elasticsearch + azure_data_lake + + atlas) # Snakebite & Google Cloud Dataflow are not Python 3 compatible :'( if PY3: From 54da3eaf175b0224a7d6e09f57dc6278293937d0 Mon Sep 17 00:00:00 2001 From: Stefan Seelmann Date: Sun, 17 Jun 2018 09:52:45 +0200 Subject: [PATCH 013/808] [AIRFLOW-2606] Fix DB schema and SQLAlchemy model * Add test that verifies that database schema and SQLAlchemy model are in sync * Add exception for users.password that doesn't exist in model and tables created by other tests * Add migration script to merge the two heads * Add migration script to fix not-null constrains for MySQL that were lost by 0e2a74e0fc9f_add_time_zone_awareness * Add migration script to fix FK constraint for existing SQLite DBs * Enable ForeignKey support for SQLite, otherwise 2e82aab8ef20_rename_user_table won't change FK in chart and known_event tables (cherry picked from commit 680651f0ae2a314f8e9882a6bc38f4fa3795cdbe) Signed-off-by: Bolke de Bruin --- airflow/jobs.py | 1 + .../versions/05f30312d566_merge_heads.py | 43 +++++++++ .../856955da8476_fix_sqlite_foreign_key.py | 90 +++++++++++++++++++ ...433877c24_fix_mysql_not_null_constraint.py | 54 +++++++++++ airflow/models.py | 22 +++-- airflow/utils/sqlalchemy.py | 6 ++ 6 files changed, 207 insertions(+), 9 deletions(-) create mode 100644 airflow/migrations/versions/05f30312d566_merge_heads.py create mode 100644 airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py create mode 100644 airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py diff --git a/airflow/jobs.py b/airflow/jobs.py index 5143e8e73e23e..70891ab4c370b 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -96,6 +96,7 @@ class BaseJob(Base, LoggingMixin): __table_args__ = ( Index('job_type_heart', job_type, latest_heartbeat), + Index('idx_job_state_heartbeat', state, latest_heartbeat), ) def __init__( diff --git a/airflow/migrations/versions/05f30312d566_merge_heads.py b/airflow/migrations/versions/05f30312d566_merge_heads.py new file mode 100644 index 0000000000000..78d5652679ca9 --- /dev/null +++ b/airflow/migrations/versions/05f30312d566_merge_heads.py @@ -0,0 +1,43 @@ +# flake8: noqa +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""merge heads + +Revision ID: 05f30312d566 +Revises: 86770d1215c0, 0e2a74e0fc9f +Create Date: 2018-06-17 10:47:23.339972 + +""" + +# revision identifiers, used by Alembic. +revision = '05f30312d566' +down_revision = ('86770d1215c0', '0e2a74e0fc9f') +branch_labels = None +depends_on = None + +from alembic import op +import sqlalchemy as sa + + +def upgrade(): + pass + + +def downgrade(): + pass diff --git a/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py b/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py new file mode 100644 index 0000000000000..5b11dc7860ea9 --- /dev/null +++ b/airflow/migrations/versions/856955da8476_fix_sqlite_foreign_key.py @@ -0,0 +1,90 @@ +# flake8: noqa +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""fix sqlite foreign key + +Revision ID: 856955da8476 +Revises: f23433877c24 +Create Date: 2018-06-17 15:54:53.844230 + +""" + +# revision identifiers, used by Alembic. +revision = '856955da8476' +down_revision = 'f23433877c24' +branch_labels = None +depends_on = None + +from alembic import op +import sqlalchemy as sa + + +def upgrade(): + conn = op.get_bind() + if conn.dialect.name == 'sqlite': + # Fix broken foreign-key constraint for existing SQLite DBs. + # + # Re-define tables and use copy_from to avoid reflection + # which would fail because referenced user table doesn't exist. + # + # Use batch_alter_table to support SQLite workaround. + chart_table = sa.Table('chart', + sa.MetaData(), + sa.Column('id', sa.Integer(), nullable=False), + sa.Column('label', sa.String(length=200), nullable=True), + sa.Column('conn_id', sa.String(length=250), nullable=False), + sa.Column('user_id', sa.Integer(), nullable=True), + sa.Column('chart_type', sa.String(length=100), nullable=True), + sa.Column('sql_layout', sa.String(length=50), nullable=True), + sa.Column('sql', sa.Text(), nullable=True), + sa.Column('y_log_scale', sa.Boolean(), nullable=True), + sa.Column('show_datatable', sa.Boolean(), nullable=True), + sa.Column('show_sql', sa.Boolean(), nullable=True), + sa.Column('height', sa.Integer(), nullable=True), + sa.Column('default_params', sa.String(length=5000), nullable=True), + sa.Column('x_is_date', sa.Boolean(), nullable=True), + sa.Column('iteration_no', sa.Integer(), nullable=True), + sa.Column('last_modified', sa.DateTime(), nullable=True), + sa.PrimaryKeyConstraint('id') + ) + with op.batch_alter_table('chart', copy_from=chart_table) as batch_op: + batch_op.create_foreign_key('chart_user_id_fkey', 'users', + ['user_id'], ['id']) + + known_event_table = sa.Table('known_event', + sa.MetaData(), + sa.Column('id', sa.Integer(), nullable=False), + sa.Column('label', sa.String(length=200), nullable=True), + sa.Column('start_date', sa.DateTime(), nullable=True), + sa.Column('end_date', sa.DateTime(), nullable=True), + sa.Column('user_id', sa.Integer(), nullable=True), + sa.Column('known_event_type_id', sa.Integer(), nullable=True), + sa.Column('description', sa.Text(), nullable=True), + sa.ForeignKeyConstraint(['known_event_type_id'], + ['known_event_type.id'], ), + sa.PrimaryKeyConstraint('id') + ) + with op.batch_alter_table('chart', copy_from=known_event_table) as batch_op: + batch_op.create_foreign_key('known_event_user_id_fkey', 'users', + ['user_id'], ['id']) + + +def downgrade(): + # Downgrade would fail because the broken FK constraint can't be re-created. + pass diff --git a/airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py b/airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py new file mode 100644 index 0000000000000..44edeef0695a7 --- /dev/null +++ b/airflow/migrations/versions/f23433877c24_fix_mysql_not_null_constraint.py @@ -0,0 +1,54 @@ +# flake8: noqa +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +"""fix mysql not null constraint + +Revision ID: f23433877c24 +Revises: 05f30312d566 +Create Date: 2018-06-17 10:16:31.412131 + +""" + +# revision identifiers, used by Alembic. +revision = 'f23433877c24' +down_revision = '05f30312d566' +branch_labels = None +depends_on = None + +from alembic import op +import sqlalchemy as sa +from sqlalchemy.dialects import mysql + +def upgrade(): + conn = op.get_bind() + if conn.dialect.name == 'mysql': + conn.execute("SET time_zone = '+00:00'") + op.alter_column('task_fail', 'execution_date', existing_type=mysql.TIMESTAMP(fsp=6), nullable=False) + op.alter_column('xcom', 'execution_date', existing_type=mysql.TIMESTAMP(fsp=6), nullable=False) + op.alter_column('xcom', 'timestamp', existing_type=mysql.TIMESTAMP(fsp=6), nullable=False) + + +def downgrade(): + conn = op.get_bind() + if conn.dialect.name == 'mysql': + conn.execute("SET time_zone = '+00:00'") + op.alter_column('xcom', 'timestamp', existing_type=mysql.TIMESTAMP(fsp=6), nullable=True) + op.alter_column('xcom', 'execution_date', existing_type=mysql.TIMESTAMP(fsp=6), nullable=True) + op.alter_column('task_fail', 'execution_date', existing_type=mysql.TIMESTAMP(fsp=6), nullable=True) + diff --git a/airflow/models.py b/airflow/models.py index b0841e6369726..bde6137022013 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -56,7 +56,7 @@ from sqlalchemy import ( Column, Integer, String, DateTime, Text, Boolean, ForeignKey, PickleType, - Index, Float, LargeBinary) + Index, Float, LargeBinary, UniqueConstraint) from sqlalchemy import func, or_, and_, true as sqltrue from sqlalchemy.ext.declarative import declarative_base, declared_attr from sqlalchemy.orm import reconstructor, relationship, synonym @@ -885,7 +885,7 @@ class TaskInstance(Base, LoggingMixin): max_tries = Column(Integer) hostname = Column(String(1000)) unixname = Column(String(1000)) - job_id = Column(Integer, index=True) + job_id = Column(Integer) pool = Column(String(50)) queue = Column(String(50)) priority_weight = Column(Integer) @@ -899,6 +899,7 @@ class TaskInstance(Base, LoggingMixin): Index('ti_state', state), Index('ti_state_lkp', dag_id, task_id, execution_date, state), Index('ti_pool', pool, state, priority_weight), + Index('ti_job_id', job_id), ) def __init__(self, task, execution_date, state=None): @@ -2072,12 +2073,13 @@ class TaskFail(Base): __tablename__ = "task_fail" - task_id = Column(String(ID_LEN), primary_key=True) - dag_id = Column(String(ID_LEN), primary_key=True) - execution_date = Column(UtcDateTime, primary_key=True) + id = Column(Integer, primary_key=True) + task_id = Column(String(ID_LEN), nullable=False) + dag_id = Column(String(ID_LEN), nullable=False) + execution_date = Column(UtcDateTime, nullable=False) start_date = Column(UtcDateTime) end_date = Column(UtcDateTime) - duration = Column(Float) + duration = Column(Integer) __table_args__ = ( Index('idx_task_fail_dag_task_date', dag_id, task_id, execution_date, @@ -4670,8 +4672,8 @@ class DagStat(Base): dag_id = Column(String(ID_LEN), primary_key=True) state = Column(String(50), primary_key=True) - count = Column(Integer, default=0) - dirty = Column(Boolean, default=False) + count = Column(Integer, default=0, nullable=False) + dirty = Column(Boolean, default=False, nullable=False) def __init__(self, dag_id, state, count=0, dirty=False): self.dag_id = dag_id @@ -4804,7 +4806,9 @@ class DagRun(Base, LoggingMixin): dag = None __table_args__ = ( - Index('dr_run_id', dag_id, run_id, unique=True), + Index('dag_id_state', dag_id, _state), + UniqueConstraint('dag_id', 'execution_date'), + UniqueConstraint('dag_id', 'run_id'), ) def __repr__(self): diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index a00fe17c18a48..4dab322d4a106 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -101,6 +101,12 @@ def ping_connection(connection, branch): def connect(dbapi_connection, connection_record): connection_record.info['pid'] = os.getpid() + @event.listens_for(engine, "connect") + def set_sqlite_pragma(dbapi_connection, connection_record): + if 'sqlite3.Connection' in str(type(dbapi_connection)): + cursor = dbapi_connection.cursor() + cursor.execute("PRAGMA foreign_keys=ON") + cursor.close() @event.listens_for(engine, "checkout") def checkout(dbapi_connection, connection_record, connection_proxy): From 9b0136772c5bc64e912872ec14c8eb65d378668e Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sun, 1 Jul 2018 11:54:56 +0200 Subject: [PATCH 014/808] Bump version --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index aa28ee9f952f9..b8170040b4a8f 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.0beta3+incubating' +version = '1.10.0beta4+incubating' From f65507fcd4632c9050d10e0911e00e8f5c6557b1 Mon Sep 17 00:00:00 2001 From: Matthew Thorley Date: Sun, 8 Jul 2018 20:52:51 +0200 Subject: [PATCH 015/808] [AIRFLOW-2710] Clarify fernet key value in documentation Closes #3574 from padwasabimasala/AIRFLOW-2710 (cherry picked from commit 6b7645261b5df69b01936838f38f7f701a667518) Signed-off-by: Bolke de Bruin --- docs/howto/secure-connections.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/howto/secure-connections.rst b/docs/howto/secure-connections.rst index b9c1fa10048f5..f9e252c4c365a 100644 --- a/docs/howto/secure-connections.rst +++ b/docs/howto/secure-connections.rst @@ -16,7 +16,7 @@ connections by following steps below: from cryptography.fernet import Fernet fernet_key= Fernet.generate_key() - print(fernet_key) # your fernet_key, keep it in secured place! + print(fernet_key.decode()) # your fernet_key, keep it in secured place! 3. Replace ``airflow.cfg`` fernet_key value with the one from step 2. Alternatively, you can store your fernet_key in OS environment variable. You From 3dedeeab0bfabcdafc0ef104cf540bdf959115e7 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sun, 8 Jul 2018 21:15:22 +0200 Subject: [PATCH 016/808] Bump version --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index b8170040b4a8f..02346d56fdef0 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.0beta4+incubating' +version = '1.10.0+incubating' From 26ea12a1622d9a42701c3fe49afa80465244063e Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sun, 8 Jul 2018 21:49:35 +0200 Subject: [PATCH 017/808] Update changelog --- CHANGELOG.txt | 775 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 775 insertions(+) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index fa4e6547a7e1d..00ee4a615ed0a 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -1,3 +1,778 @@ +AIRFLOW 1.10.0, 2018-07-08 +-------------------------- + +[AIRFLOW-2710] Clarify fernet key value in documentation +[AIRFLOW-2606] Fix DB schema and SQLAlchemy model +[AIRFLOW-2646] Fix setup.py not to install snakebite on Python3 +[AIRFLOW-2512][AIRFLOW-2522] Use google-auth instead of oauth2client +[AIRFLOW-2604] Add index to task_fail +[AIRFLOW-2650] Mark SchedulerJob as succeed when hitting Ctrl-c +[AIRFLOW-2678] Fix db schema unit test to remove checking fab models +[AIRFLOW-2624] Fix webserver login as anonymous +[AIRFLOW-2654] Fix incorret URL on refresh in Graph View of FAB UI +[AIRFLOW-2668] Handle missing optional cryptography dependency +[AIRFLOW-2681] Include last dag run of externally triggered DAGs in UI. +[AIRFLOW-1840] Support back-compat on old celery config +[AIRFLOW-2612][AIRFLOW-2534] Clean up Hive-related tests +[AIRFLOW-2608] Implements/Standardize custom exceptions for experimental APIs +[AIRFLOW-2607] Fix failing TestLocalClient +[AIRFLOW-2638] dbapi_hook: support REPLACE INTO +[AIRFLOW-2542][AIRFLOW-1790] Rename AWS Batch Operator queue to job_queue +[AIRFLOW-2567] Extract result from the kubernetes pod as Xcom +[AIRFLOW-XXX] Adding REA Group to readme +[AIRFLOW-2601] Allow user to specify k8s config +[AIRFLOW-2559] Azure Fileshare hook +[AIRFLOW-1786] Enforce correct behavior for soft-fail sensors +[AIRFLOW-2355] Airflow trigger tag parameters in subdag +[AIRFLOW-2613] Fix Airflow searching .zip bug +[AIRFLOW-2627] Add a sensor for Cassandra +[AIRFLOW-2634][AIRFLOW-2534] Remove dependency for impyla +[AIRFLOW-2611] Fix wrong dag volume mount path for kubernetes executor +[AIRFLOW-2562] Add Google Kubernetes Engine Operators +[AIRFLOW-2630] Fix classname in test_sql_sensor.py +[AIRFLOW-2534] Fix bug in HiveServer2Hook +[AIRFLOW-2586] Stop getting AIRFLOW_HOME value from config file in bash operator +[AIRFLOW-2605] Fix autocommit for MySqlHook +[AIRFLOW-2539][AIRFLOW-2359] Move remaing log config to configuration file +[AIRFLOW-1656] Tree view dags query changed +[AIRFLOW-2617] add imagePullPolicy config for kubernetes executor +[AIRFLOW-2429] Fix security/task/sensors/ti_deps folders flake8 error +[AIRFLOW-2550] Implements API endpoint to list DAG runs +[AIRFLOW-2512][AIRFLOW-2522] Use google-auth instead of oauth2client +[AIRFLOW-2429] Fix operators folder flake8 error +[AIRFLOW-2585] Fix several bugs in CassandraHook and CassandraToGCSOperator +[AIRFLOW-2597] Restore original dbapi.run() behavior +[AIRFLOW-2590] Fix commit in DbApiHook.run() for no-autocommit DB +[AIRFLOW-1115] fix github oauth api URL +[AIRFLOW-2587] Add TIMESTAMP type mapping to MySqlToHiveTransfer +[AIRFLOW-2591][AIRFLOW-2581] Set default value of autocommit to False in DbApiHook.run() +[AIRFLOW-59] Implement bulk_dump and bulk_load for the Postgres hook +[AIRFLOW-2533] Fix path to DAG's on kubernetes executor workers +[AIRFLOW-2581] RFLOW-2581] Fix DbApiHook autocommit +[AIRFLOW-2578] Add option to use proxies in JiraHook +[AIRFLOW-2575] Make gcs to gcs operator work with large files +[AIRFLOW-437] Send TI context in kill zombies +[AIRFLOW-2566] Change backfill to rerun failed tasks +[AIRFLOW-1021] Fix double login for new users with LDAP +[AIRFLOW-XXX] Typo fix +[AIRFLOW-2561] Fix typo in EmailOperator +[AIRFLOW-2573] Cast BigQuery TIMESTAMP field to float +[AIRFLOW-2560] Adding support for internalIpOnly to DataprocClusterCreateOperator +[AIRFLOW-2565] templatize cluster_label +[AIRFLOW-83] add mongo hook and operator +[AIRFLOW-2558] Clear task/dag is clearing all executions +[AIRFLOW-XXX] Fix doc typos +[AIRFLOW-2513] Change `bql` to `sql` for BigQuery Hooks & Ops +[AIRFLOW-2557] Fix pagination for s3 +[AIRFLOW-2545] Eliminate DeprecationWarning +[AIRFLOW-2500] Fix MySqlToHiveTransfer to transfer unsigned type properly +[AIRFLOW-2462] Change PasswordUser setter to correct syntax +[AIRFLOW-2525] Fix a bug introduced by commit dabf1b9 +[AIRFLOW-2553] Add webserver.pid to .gitignore +[AIRFLOW-1863][AIRFLOW-2529] Add dag run selection widgets to gantt view +[AIRFLOW-2504] Log username correctly and add extra to search columns +[AIRFLOW-2551] Encode binary data with base64 standard rather than base64 url +[AIRFLOW-2537] Add reset-dagrun option to backfill command +[AIRFLOW-2526] dag_run.conf can override params +[AIRFLOW-2544][AIRFLOW-1967] Guard against next major release of Celery, Flower +[AIRFLOW-XXX] Add Yieldr to who is using airflow +[AIRFLOW-2547] Describe how to run tests using Docker +[AIRFLOW-2538] Update faq doc on how to reduce airflow scheduler latency +[AIRFLOW-2529] Improve graph view performance and usability +[AIRFLOW-2517] backfill support passing key values through CLI +[AIRFLOW-2532] Support logs_volume_subpath for KubernetesExecutor +[AIRFLOW-2466] consider task_id in _change_state_for_tis_without_dagrun +[AIRFLOW-2519] Fix CeleryExecutor with SQLAlchemy +[AIRFLOW-2402] Fix RBAC task log +[AIRFLOW-XXX] Add M4U to user list +[AIRFLOW-2536] docs about how to deal with airflow initdb failure +[AIRFLOW-2530] KubernetesOperator supports multiple clusters +[AIRFLOW-1499] Eliminate duplicate and unneeded code +[AIRFLOW-2521] backfill - make variable name and logging messages more acurate +[AIRFLOW-2429] Fix hook, macros folder flake8 error +[Airflow-XXX] add Prime to company list +[AIRFLOW-2525] Fix PostgresHook.copy_expert to work with "COPY FROM" +[AIRFLOW-2515] Add dependency on thrift_sasl to hive extra +[AIRFLOW-2523] Add how-to for managing GCP connections +[AIRFLOW-2510] Introduce new macros: prev_ds and next_ds +[AIRFLOW-1730] Unpickle value of XCom queried from DB +[AIRFLOW-2518] Fix broken ToC links in integration.rst +[AIRFLOW-1472] Fix SLA misses triggering on skipped tasks. +[AIRFLOW-2520] CLI - make backfill less verbose +[AIRFLOW-2107] add time_partitioning to run_query on BigQueryBaseCursor +[AIRFLOW-1057][AIRFLOW-1380][AIRFLOW-2362][2362] AIRFLOW Update DockerOperator to new API +[AIRFLOW-2415] Make airflow DAG templating render numbers +[AIRFLOW-2473] Fix wrong skip condition for TransferTests +[AIRFLOW-2472] Implement MySqlHook.bulk_dump +[AIRFLOW-2419] Use default view for subdag operator +[AIRFLOW-2498] Fix Unexpected argument in SFTP Sensor +[AIRFLOW-2509] Separate config docs into how-to guides +[AIRFLOW-2429] Add BaseExecutor back +[AIRFLOW-2429] Fix dag, example_dags, executors flake8 error +[AIRFLOW-2502] Change Single triple quotes to double for docstrings +[AIRFLOW-2503] Fix broken links in CONTRIBUTING.md +[AIRFLOW-2501] Refer to devel instructions in docs contrib guide +[AIRFLOW-2429] Fix contrib folder's flake8 errors +[AIRFLOW-2471] Fix HiveCliHook.load_df to use unused parameters +[AIRFLOW-2495] Update celery to 4.1.1 +[AIRFLOW-2429] Fix api, bin, config_templates folders flake8 error +[AIRFLOW-2493] Mark template_fields of all Operators in the API document as "templated" +[AIRFLOW-2489] Update FlaskAppBuilder to 1.11.1 +[AIRFLOW-2448] Enhance HiveCliHook.load_df to work with datetime +[AIRFLOW-2487] Enhance druid ingestion hook +[AIRFLOW-2397] Support affinity policies for Kubernetes executor/operator +[AIRFLOW-2482] Add test for rewrite method in GCS Hook +[AIRFLOW-2481] Fix flaky Kubernetes test +[AIRFLOW-2479] Improve doc FAQ section +[AIRFLOW-2485] Fix Incorrect logging for Qubole Sensor +[AIRFLOW-2486] Remove unnecessary slash after port +[AIRFLOW-2429] Make Airflow flake8 compliant +[AIRFLOW-2491] Resolve flask version conflict +[AIRFLOW-2484] Remove duplicate key in MySQL to GCS Op +[ARIFLOW-2458] Add cassandra-to-gcs operator +[AIRFLOW-2477] Improve time units for task duration and landing times charts for RBAC UI +[AIRFLOW-2474] Only import snakebite if using py2 +[AIRFLOW-48] Parse connection uri querystring +[AIRFLOW-2467][AIRFLOW-2] Update import direct warn message to use the module name +[AIRFLOW-XXX] Fix order of companies +[AIRFLOW-2452] Document field_dict must be OrderedDict +[AIRFLOW-2420] Azure Data Lake Hook +[AIRFLOW-2213] Add Quoble check operator +[AIRFLOW-2465] Fix wrong module names in the doc +[AIRFLOW-1929] Modifying TriggerDagRunOperator to accept execution_date +[AIRFLOW-2460] Users can now use volume mounts and volumes +[AIRFLOW-2110][AIRFLOW-2122] Enhance Http Hook +[AIRFLOW-XXX] Updated contributors list +[AIRFLOW-2435] Add launch_type to ECSOperator to allow FARGATE +[AIRFLOW-2451] Remove extra slash ('/') char when using wildcard in gcs_to_gcs operator +[AIRFLOW-2461] Add support for cluster scaling on dataproc operator +[AIRFLOW-2376] Fix no hive section error +[AIRFLOW-2425] Add lineage support +[AIRFLOW-2430] Extend query batching to additional slow queries +[AIRFLOW-2453] Add default nil value for kubernetes/git_subpath +[AIRFLOW-2396] Add support for resources in kubernetes operator +[AIRFLOW-2169] Encode binary data with base64 before importing to BigQuery +[AIRFLOW-XXX] Add spotahome in user list +[AIRFLOW-2457] Update FAB version requirement +[AIRFLOW-2454][Airflow 2454] Support imagePullPolicy for k8s +[AIRFLOW-2450] update supported k8s versions to 1.9 and 1.10 +[AIRFLOW-2333] Add Segment Hook and TrackEventOperator +[AIRFLOW-2442][AIRFLOW-2] Airflow run command leaves database connections open +[AIRFLOW-2016] assign template_fields for Dataproc Workflow Template sub-classes, not base class +[AIRFLOW-2446] Add S3ToRedshiftTransfer into the "Integration" doc +[AIRFLOW-2449] Fix operators.py to run all test cases +[AIRFLOW-2424] Add dagrun status endpoint and increased k8s test coverage +[AIRFLOW-2441] Fix bugs in HiveCliHook.load_df +[AIRFLOW-2358][AIRFLOW-201804] Make the Kubernetes example optional +[AIRFLOW-2436] Remove cli_logger in initdb +[AIRFLOW-2444] Remove unused option(include_adhoc) in cli backfill command +[AIRFLOW-2447] Fix TestHiveMetastoreHook to run all cases +[AIRFLOW-2445] Allow templating in kubernetes operator +[AIRFLOW-2086][AIRFLOW-2393] Customize default dagrun number in tree view +[AIRFLOW-2437] Add PubNub to list of current airflow users +[AIRFLOW-XXX] Add Quantopian to list of Airflow users +[AIRFLOW-1978] Add WinRM windows operator and hook +[AIRFLOW-2427] Add tests to named hive sensor +[AIRFLOW-2412] Fix HiveCliHook.load_file to address HIVE-10541 +[AIRFLOW-2431] Add the navigation bar color parameter for RBAC UI +[AIRFLOW-2407] Resolve Python undefined names +[AIRFLOW-1952] Add the navigation bar color parameter +[AIRFLOW-2222] Implement GoogleCloudStorageHook.rewrite +[AIRFLOW-2426] Add Google Cloud Storage Hook tests +[AIRFLOW-2418] Bump Flask-WTF +[AIRFLOW-2417] Wait for pod is not running to end task +[AIRFLOW-1914] Add other charset support to email utils +[AIRFLOW-XXX] Update README.md with Craig@Work +[AIRFLOW-1899] Fix Kubernetes tests +[AIRFLOW-1812] Update logging example +[AIRFLOW-2313] Add TTL parameters for Dataproc +[AIRFLOW-2411] add dataproc_jars to templated_fields +[AIRFLOW-XXX] Add Reddit to Airflow users +[AIRFLOW-XXX] Fix wrong table header in scheduler.rst +[AIRFLOW-2409] Supply password as a parameter +[AIRFLOW-2410][AIRFLOW-75] Set the timezone in the RBAC Web UI +[AIRFLOW-2394] default cmds and arguments in kubernetes operator +[AIRFLOW-2406] Add Apache2 License Shield to Readme +[AIRFLOW-2404] Add additional documentation for unqueued task +[AIRFLOW-2400] Add Ability to set Environment Variables for K8s +[AIRFLOW-XXX] Add Twine Labs as an Airflow user +[AIRFLOW-1853] Show only the desired number of runs in tree view +[AIRFLOW-2401] Document the use of variables in Jinja template +[AIRFLOW-2403] Fix License Headers +[AIRFLOW-1313] Fix license header +[AIRFLOW-2398] Add BounceX to list of current airflow users +[AIRFLOW-2363] Fix return type bug in TaskHandler +[AIRFLOW-2389] Create a pinot db api hook +[AIRFLOW-2390] Resolve FlaskWTFDeprecationWarning +[AIRFLOW-1933] Fix some typos +[AIRFLOW-1960] Add support for secrets in kubernetes operator +[AIRFLOW-1313] Add vertica_to_mysql operator +[AIRFLOW-1575] Add AWS Kinesis Firehose Hook for inserting batch records +[AIRFLOW-2266][AIRFLOW-2343] Remove google-cloud-dataflow dependency +[AIRFLOW-2370] Implement --use_random_password in create_user +[AIRFLOW-2348] Strip path prefix from the destination_object when source_object contains a wildcard[] +[AIRFLOW-2391] Fix to Flask 0.12.2 +[AIRFLOW-2381] Fix the flaky ApiPasswordTests test +[AIRFLOW-2378] Add Groupon to list of current users +[AIRFLOW-2382] Fix wrong description for delimiter +[AIRFLOW-2380] Add support for environment variables in Spark submit operator. +[AIRFLOW-2377] Improve Sendgrid sender support +[AIRFLOW-2331] Support init action timeout on dataproc cluster create +[AIRFLOW-1835] Update docs: Variable file is json +[AIRFLOW-1781] Make search case-insensitive in LDAP group +[AIRFLOW-2042] Fix browser menu appearing over the autocomplete menu +[AIRFLOW-XXX] Remove wheelhouse files from travis not owned by travis +[AIRFLOW-2336] Use hmsclient in hive_hook +[AIRFLOW-2041] Correct Syntax in python examples +[AIRFLOW-74] SubdagOperators can consume all celeryd worker processes +[AIRFLOW-2369] Fix gcs tests +[AIRFLOW-2365] Fix autocommit attribute check +[AIRFLOW-2068] MesosExecutor allows optional Docker image +[AIRFLOW-1652] Push DatabricksRunSubmitOperator metadata into XCOM +[AIRFLOW-2234] Enable insert_rows for PrestoHook +[AIRFLOW-2208][Airflow-22208] Link to same DagRun graph from TaskInstance view +[AIRFLOW-1153] Allow HiveOperators to take hiveconfs +[AIRFLOW-775] Fix autocommit settings with Jdbc hook +[AIRFLOW-2364] Warn when setting autocommit on a connection which does not support it +[AIRFLOW-2357] Add persistent volume for the logs +[AIRFLOW-766] Skip conn.commit() when in Auto-commit +[AIRFLOW-2351] Check for valid default_args start_date +[AIRFLOW-1433] Set default rbac to initdb +[AIRFLOW-2270] Handle removed tasks in backfill +[AIRFLOW-2344] Fix `connections -l` to work with pipe/redirect +[AIRFLOW-2300] Add S3 Select functionarity to S3ToHiveTransfer +[AIRFLOW-1314] Cleanup the config +[AIRFLOW-1314] Polish some of the Kubernetes docs/config +[AIRFLOW-1314] Improve error handling +[AIRFLOW-1999] Add per-task GCP service account support +[AIRFLOW-1314] Rebasing against master +[AIRFLOW-1314] Small cleanup to address PR comments (#24) +[AIRFLOW-1314] Add executor_config and tests +[AIRFLOW-1314] Improve k8s support +[AIRFLOW-1314] Use VolumeClaim for transporting DAGs +[AIRFLOW-1314] Create integration testing environment +[AIRFLOW-1314] Git Mode to pull in DAGs for Kubernetes Executor +[AIRFLOW-1314] Add support for volume mounts & Secrets in Kubernetes Executor +[AIRFLOW=1314] Basic Kubernetes Mode +[AIRFLOW-2326][AIRFLOW-2222] remove contrib.gcs_copy_operator +[AIRFLOW-2328] Fix empty GCS blob in S3ToGoogleCloudStorageOperator +[AIRFLOW-2350] Fix grammar in UPDATING.md +[AIRFLOW-2302] Fix documentation +[AIRFLOW-2345] pip is not used in this setup.py +[AIRFLOW-2347] Add Banco de Formaturas to Readme +[AIRFLOW-2346] Add Investorise as official user of Airflow +[AIRFLOW-2330] Do not append destination prefix if not given +[AIRFLOW-2240][DASK] Added TLS/SSL support for the dask-distributed scheduler. +[AIRFLOW-2309] Fix duration calculation on TaskFail +[AIRFLOW-2335] fix issue with jdk8 download for ci +[AIRFLOW-2184] Add druid_checker_operator +[AIRFLOW-2299] Add S3 Select functionarity to S3FileTransformOperator +[AIRFLOW-2254] Put header as first row in unload +[AIRFLOW-610] Respect _cmd option in config before defaults +[AIRFLOW-2287] Fix incorrect ASF headers +[AIRFLOW-XXX] Add Zego as an Apache Airflow user +[AIRFLOW-952] fix save empty extra field in UI +[AIRFLOW-1325] Add ElasticSearch log handler and reader +[AIRFLOW-2301] Sync files of an S3 key with a GCS path +[AIRFLOW-2293] Fix S3FileTransformOperator to work with boto3 +[AIRFLOW-3212][AIRFLOW-2314] Remove only leading slash in GCS path +[AIRFLOW-1509][AIRFLOW-442] SFTP Sensor +[AIRFLOW-2291] Add optional params to ML Engine +[AIRFLOW-1774] Allow consistent templating of arguments in MLEngineBatchPredictionOperator +[AIRFLOW-2302] Add missing operators and hooks +[AIRFLOW-2312] Docs Typo Correction: Corresponding +[AIRFLOW-1623] Trigger on_kill method in operators +[AIRFLOW-2162] When impersonating another user, pass env variables to sudo +[AIRFLOW-2304] Update quickstart doc to mention scheduler part +[AIRFLOW-1633] docker_operator needs a way to set shm_size +[AIRFLOW-1340] Add S3 to Redshift transfer operator +[AIRFLOW-2303] Lists the keys inside an S3 bucket +[AIRFLOW-2209] restore flask_login imports +[AIRFLOW-2306] Add Bonnier Broadcasting to list of current users +[AIRFLOW-2305][AIRFLOW-2027] Fix CI failure caused by [] +[AIRFLOW-2281] Add support for Sendgrid categories +[AIRFLOW-2027] Only trigger sleep in scheduler after all files have parsed +[AIRFLOW-2256] SparkOperator: Add Client Standalone mode and retry mechanism +[AIRFLOW-2284] GCS to S3 operator +[AIRFLOW-2287] Update license notices +[AIRFLOW-2296] Add Cinimex DataLab to Readme +[AIRFLOW-2298] Add Kalibrr to who uses airflow +[AIRFLOW-2292] Fix docstring for S3Hook.get_wildcard_key +[AIRFLOW-XXX] Update PR template +[AIRFLOW-XXX] Remove outdated migrations.sql +[AIRFLOW-2287] Add license header to docs/Makefile +[AIRFLOW-2286] Add tokopedia to the readme +[AIRFLOW-2273] Add Discord webhook operator/hook +[AIRFLOW-2282] Fix grammar in UPDATING.md +[AIRFLOW-2200] Add snowflake operator with tests +[AIRFLOW-2178] Add handling on SLA miss errors +[AIRFLOW-2169] Fix type 'bytes' is not JSON serializable in python3 +[AIRFLOW-2215] Pass environment to subproces.Popen in base_task_runner +[AIRFLOW-2253] Add Airflow CLI instrumentation +[AIRFLOW-2274] Fix Dataflow tests +[AIRFLOW-2269] Add Custom Ink as an Airflow user +[AIRFLOW-2259] Dataflow Hook Index out of range +[AIRFLOW-2233] Update updating.md to include the info of hdfs_sensors renaming +[AIRFLOW-2217] Add Slack webhook operator +[AIRFLOW-1729] improve dagBag time +[AIRFLOW-2264] Improve create_user cli help message +[AIRFLOW-2260] [AIRFLOW-2260] SSHOperator add command template .sh files +[AIRFLOW-2261] Check config/env for remote base log folder +[AIRFLOW-2258] Allow import of Parquet-format files into BigQuery +[AIRFLOW-1430] Include INSTALL instructions to avoid GPL +[AIRFLOW-1430] Solve GPL dependency +[AIRFLOW-2251] Add Thinknear as an Airflow user +[AIRFLOW-2244] bugfix: remove legacy LongText code from models.py +[AIRFLOW-2247] Fix RedshiftToS3Transfer not to fail with ValueError +[AIRFLOW-2249] Add side-loading support for Zendesk Hook +[AIRFLOW-XXX] Add Qplum to Airflow users +[AIRFLOW-2228] Enhancements in ValueCheckOperator +[AIRFLOW-1206] Typos +[AIRFLOW-2060] Update pendulum version to 1.4.4 +[AIRFLOW-2248] Fix wrong param name in RedshiftToS3Transfer doc +[AIRFLOW-1433][AIRFLOW-85] New Airflow Webserver UI with RBAC support +[AIRFLOW-1235] Fix webserver's odd behaviour +[AIRFLOW-1460] Allow restoration of REMOVED TI's +[airflow-2235] Fix wrong docstrings in two operators +[AIRFLOW-XXX] Fix chronological order for companies using Airflow +[AIRFLOW-2124] Upload Python file to a bucket for Dataproc +[AIRFLOW-2212] Fix ungenerated sensor API reference +[AIRFLOW-2226] Rename google_cloud_storage_default to google_cloud_default +[AIRFLOW-2211] Rename hdfs_sensors.py to hdfs_sensor.py for consistency +[AIRFLOW-2225] Update document to include DruidDbApiHook +[Airflow-2202] Add filter support in HiveMetastoreHook().max_partition() +[AIRFLOW-2220] Remove duplicate numeric list entry in security.rst +[AIRFLOW-XXX] Update tutorial documentation +[AIRFLOW-2215] Update celery task to preserve environment variables and improve logging on exception +[AIRFLOW-2185] Use state instead of query param +[AIRFLOW-2183] Refactor DruidHook to enable sql +[AIRFLOW-2203] Defer cycle detection +[AIRFLOW-2203] Remove Useless Commands. +[AIRFLOW-2203] Cache signature in apply_defaults +[AIRFLOW-2203] Speed up Operator Resources +[AIRFLOW-2203] Cache static rules (trigger/weight) +[AIRFLOW-2203] Store task ids as sets not lists +[AIRFLOW-2205] Remove unsupported args from JdbcHook doc +[AIRFLOW-2207] Fix flaky test that uses app.cached_app() +[AIRFLOW-2206] Remove unsupported args from JdbcOperator doc +[AIRFLOW-2140] Add Kubernetes scheduler to SparkSubmitOperator +[AIRFLOW-XXX] Add Xero to list of users +[AIRFLOW-2204] Fix webserver debug mode +[AIRFLOW-102] Fix test_complex_template always succeeds +[AIRFLOW-442] Add SFTPHook +[AIRFLOW-2169] Add schema to MySqlToGoogleCloudStorageOperator +[AIRFLOW-2184][AIRFLOW-2138] Google Cloud Storage allow wildcards +[AIRFLOW-1588] Cast Variable value to string +[AIRFLOW-2199] Fix invalid reference to logger +[AIRFLOW-2191] Change scheduler heartbeat logs from info to debug +[AIRFLOW-2106] SalesForce hook sandbox option +[AIRFLOW-2197] Silence hostname_callable config error message +[AIRFLOW-2150] Use lighter call in HiveMetastoreHook().max_partition() +[AIRFLOW-2186] Change the way logging is carried out in few ops +[AIRFLOW-2181] Convert password_auth and test_password_endpoints from DOS to UNIX +[AIRFLOW-2187] Fix Broken Travis CI due to AIRFLOW-2123 +[AIRFLOW-2175] Check that filepath is not None +[AIRFLOW-2173] Don't check task IDs for concurrency reached check +[AIRFLOW-2168] Remote logging for Azure Blob Storage +[AIRFLOW-XXX] Add DocuTAP to list of users +[AIRFLOW-2176] Change the way logging is carried out in BQ Get Data Operator +[AIRFLOW-2177] Add mock test for GCS Download op +[AIRFLOW-2123] Install CI dependencies from setup.py +[AIRFLOW-2129] Presto hook calls _parse_exception_message but defines _get_pretty_exception_message +[AIRFLOW-2174] Fix typos and wrongly rendered documents +[AIRFLOW-2171] Store delegated credentials +[AIRFLOW-2166] Restore BQ run_query dialect param +[AIRFLOW-2163] Add HBC Digital to users of airflow +[AIRFLOW-2065] Fix race-conditions when creating loggers +[AIRFLOW-2147] Plugin manager: added 'sensors' attribute +[AIRFLOW-2059] taskinstance query is awful, un-indexed, and does not scale +[AIRFLOW-2159] Fix a few typos in salesforce_hook +[AIRFLOW-2132] Add step to initialize database +[AIRFLOW-2160] Fix bad rowid deserialization +[AIRFLOW-2161] Add Vevo to list of companies using Airflow +[AIRFLOW-2149] Add link to apache Beam documentation to create self executing Jar +[AIRFLOW-2151] Allow getting the session from AwsHook +[AIRFLOW-2097] tz referenced before assignment +[AIRFLOW-2152] Add Multiply to list of companies using Airflow +[AIRFLOW-1551] Add operator to trigger Jenkins job +[AIRFLOW-2034] Fix mixup between %s and {} when using str.format Convention is to use .format for string formating oustide logging, else use lazy format See comment in related issue https://github.com/apache/incubator-airflow/pull/2823/files Identified problematic case using following command line .git/COMMIT_EDITMSG:`grep -r '%s'./* | grep '\.format('` +[AIRFLOW-2102] Add custom_args to Sendgrid personalizations +[AIRFLOW-1035][AIRFLOW-1053] import unicode_literals to parse Unicode in HQL +[AIRFLOW-2127] Keep loggers during DB migrations +[AIRFLOW-2146] Resolve issues with BQ using DbApiHook methods +[AIRFLOW-2087] Scheduler Report shows incorrect Total task number +[AIRFLOW-2139] Remove unncecessary boilerplate to get DataFrame using pandas_gbq +[AIRFLOW-2125] Using binary package psycopg2-binary +[AIRFLOW-2142] Include message on mkdir failure +[AIRFLOW-1615] SSHHook: use port specified by Connection +[AIRFLOW-2122] Handle boolean values in sshHook +[AIRFLOW-XXX] Add Tile to the list of users +[AIRFLOW-2130] Add missing Operators to API Reference docs +[AIRFLOW-XXX] Add timeout units (seconds) +[AIRFLOW-2134] Add Alan to the list of companies that use Airflow +[AIRFLOW-2133] Remove references to GitHub issues in CONTRIBUTING +[AIRFLOW-2131] Remove confusing AirflowImport docs +[AIRFLOW-1852] Allow hostname to be overridable. +[AIRFLOW-2126] Add Bluecore to active users +[AIRFLOW-1618] Add feature to create GCS bucket +[AIRFLOW-2108] Fix log indentation in BashOperator +[AIRFLOW-2115] Fix doc links to PythonHosted +[AIRFLOW-XXX] Add contributor from Easy company +[AIRFLOW-1882] Add ignoreUnknownValues option to gcs_to_bq operator +[AIRFLOW-2089] Add on kill for SparkSubmit in Standalone Cluster +[AIRFLOW-2113] Address missing DagRun callbacks Given that the handle_callback method belongs to the DAG object, we are able to get the list of task directly with get_task and reduce the communication with the database, making airflow more lightweight. +[AIRFLOW-2112] Fix svg width for Recent Tasks on UI. +[AIRFLOW-2116] Set CI Cloudant version to <2.0 +[AIRFLOW-XXX] Add PMC to list of companies using Airflow +[AIRFLOW-2100] Fix Broken Documentation Links +[AIRFLOW-1404] Add 'flatten_results' & 'maximum_bytes_billed' to BQ Operator +[AIRFLOW-800] Initialize valid Google BigQuery Connection +[AIRFLOW-1319] Fix misleading SparkSubmitOperator and SparkSubmitHook docstring +[AIRFLOW-1983] Parse environment parameter as template +[AIRFLOW-2095] Add operator to create External BigQuery Table +[AIRFLOW-2085] Add SparkJdbc operator +[AIRFLOW-1002] Add ability to clean all dependencies of removed DAG +[AIRFLOW-2094] Jinjafied project_id, region & zone in DataProc{*} Operators +[AIRFLOW-2092] Fixed incorrect parameter in docstring for FTPHook +[AIRFLOW-XXX] Add SocialCops to Airflow users +[AIRFLOW-2088] Fix duplicate keys in MySQL to GCS Helper function +[AIRFLOW-2091] Fix incorrect docstring parameter in BigQuery Hook +[AIRFLOW-2090] Fix typo in DataStore Hook +[AIRFLOW-1157] Fix missing pools crashing the scheduler +[AIRFLOW-713] Jinjafy {EmrCreateJobFlow,EmrAddSteps}Operator attributes +[AIRFLOW-2083] Docs: Use "its" instead of "it's" where appropriate +[AIRFLOW-2066] Add operator to create empty BQ table +[AIRFLOW-XXX] add Karmic to list of companies +[AIRFLOW-2073] Make FileSensor fail when the file doesn't exist +[AIRFLOW-2078] Improve task_stats and dag_stats performance +[AIRFLOW-2080] Use a log-out icon instead of a power button +[AIRFLOW-2077] Fetch all pages of list_objects_v2 response +[AIRFLOW-XXX] Add TM to list of companies +[AIRFLOW-1985] Impersonation fixes for using `run_as_user` +[AIRFLOW-2018][AIRFLOW-2] Make Sensors backward compatible +[AIRFLOW-XXX] Fix typo in concepts doc (dag_md) +[AIRFLOW-2069] Allow Bytes to be uploaded to S3 +[AIRFLOW-2074] Fix log var name in GHE auth +[AIRFLOW-1927] Convert naive datetimes for TaskInstances +[AIRFLOW-1760] Password auth for experimental API +[AIRFLOW-2038] Add missing kubernetes dependency for dev +[AIRFLOW-2040] Escape special chars in task instance logs URL +[AIRFLOW-1968][AIRFLOW-1520] Add role_arn and aws_account_id/aws_iam_role support back to aws hook +[AIRFLOW-2048] Fix task instance failure string formatting +[AIRFLOW-2046] Fix kerberos error to work with python 3.x +[AIRFLOW-2063] Add missing docs for GCP +[AIRFLOW-XXX] Fix typo in docs +[AIRFLOW-1793] Use docker_url instead of invalid base_url +[AIRFLOW-2055] Elaborate on slightly ambiguous documentation +[AIRFLOW-2039] BigQueryOperator supports priority property +[AIRFLOW-2053] Fix quote character bug in BQ hook +[AIRFLOW-2057] Add Overstock to list of companies +[AIRFLOW-XXX] Add Plaid to Airflow users +[AIRFLOW-2044] Add SparkSubmitOperator to documentation +[AIRFLOW-2037] Add methods to get Hash values of a GCS object +[AIRFLOW-2050] Fix Travis permission problem +[AIRFLOW-2043] Add Intercom to list of companies +[AIRFLOW-2023] Add debug logging around number of queued files +[AIRFLOW-XXX] Add Pernod-ricard as a airflow user +[AIRFLOW-1453] Add 'steps' into template_fields in EmrAddSteps +[AIRFLOW-2015] Add flag for interactive runs +[AIRFLOW-1895] Fix primary key integrity for mysql +[AIRFLOW-2030] Fix KeyError:`i` in DbApiHook for insert +[AIRFLOW-1943] Add External BigQuery Table feature +[AIRFLOW-2033] Add Google Cloud Storage List Operator +[AIRFLOW-2006] Add local log catching to kubernetes operator +[AIRFLOW-2031] Add missing gcp_conn_id in the example in DataFlow docstrings +[AIRFLOW-2029] Fix AttributeError in BigQueryPandasConnector +[AIRFLOW-2028] Add JobTeaser to official users list +[AIRFLOW-2016] Add support for Dataproc Workflow Templates +[AIRFLOW-2025] Reduced Logging verbosity +[AIRFLOW-1267][AIRFLOW-1874] Add dialect parameter to BigQueryHook +[AIRFLOW-XXX] Fixed a typo +[AIRFLOW-XXX] Typo node to nodes +[AIRFLOW-2019] Update DataflowHook for updating Streaming type job +[AIRFLOW-2017][Airflow 2017] adding query output to PostgresOperator +[AIRFLOW-1889] Split sensors into separate files +[AIRFLOW-1950] Optionally pass xcom_pull task_ids +[AIRFLOW-1755] Allow mount below root +[AIRFLOW-511][Airflow 511] add success/failure callbacks on dag level +[AIRFLOW-192] Add weight_rule param to BaseOperator +[AIRFLOW-2008] Use callable for python column defaults +[AIRFLOW-1984] Fix to AWS Batch operator +[AIRFLOW-2000] Support non-main dataflow job class +[AIRFLOW-2003] Use flask-caching instead of flask-cache +[AIRFLOW-2002] Do not swallow exception on logging import +[AIRFLOW-2004] Import flash from flask not flask.login +[AIRFLOW-1997] Fix GCP operator doc strings +[AIRFLOW-1996] Update DataflowHook waitfordone for Streaming type job[] +[AIRFLOW-1995][Airflow 1995] add on_kill method to SqoopOperator +[AIRFLOW-1770] Allow HiveOperator to take in a file +[AIRFLOW-1994] Change background color of Scheduled state Task Instances +[AIRFLOW-1436][AIRFLOW-1475] EmrJobFlowSensor considers Cancelled step as Successful +[AIRFLOW-1517] Kubernetes operator PR fixes +[AIRFLOW-1517] addressed PR comments +[AIRFLOW-1517] started documentation of k8s operator +[AIRFLOW-1517] Restore authorship of resources +[AIRFLOW-1517] Remove authorship of resources +[AIRFLOW-1517] Add minikube for kubernetes integration tests +[AIRFLOW-1517] Restore authorship of resources +[AIRFLOW-1517] fixed license issues +[AIRFLOW-1517] Created more accurate failures for kube cluster issues +[AIRFLOW-1517] Remove authorship of resources +[AIRFLOW-1517] Add minikube for kubernetes integration tests +[AIRFLOW-1988] Change BG color of None state TIs +[AIRFLOW-790] Clean up TaskInstances without DagRuns +[AIRFLOW-1949] Fix var upload, str() produces "b'...'" which is not json +[AIRFLOW-1930] Convert func.now() to timezone.utcnow() +[AIRFLOW-1688] Support load.time_partitioning in bigquery_hook +[AIRFLOW-1975] Make TriggerDagRunOperator callback optional +[AIRFLOW-1480] Render template attributes for ExternalTaskSensor fields +[AIRFLOW-1958] Add **kwargs to send_email +[AIRFLOW-1976] Fix for missing log/logger attribute FileProcessHandler +[AIRFLOW-1982] Fix Executor event log formatting +[AIRFLOW-1971] Propagate hive config on impersonation +[AIRFLOW-1969] Always use HTTPS URIs for Google OAuth2 +[AIRFLOW-1954] Add DataFlowTemplateOperator +[AIRFLOW-1963] Add config for HiveOperator mapred_queue +[AIRFLOW-1946][AIRFLOW-1855] Create a BigQuery Get Data Operator +[AIRFLOW-1953] Add labels to dataflow operators +[AIRFLOW-1967] Update Celery to 4.0.2 +[AIRFLOW-1964] Add Upsight to list of Airflow users +[AIRFLOW-XXX] Changelog for 1.9.0 +[AIRFLOW-1470] Implement BashSensor operator +[AIRFLOW-XXX] Pin sqlalchemy dependency +[AIRFLOW-1955] Do not reference unassigned variable +[AIRFLOW-1957] Add contributor to BalanceHero in Readme +[AIRFLOW-1517] Restore authorship of secrets and init container +[AIRFLOW-1517] Remove authorship of secrets and init container +[AIRFLOW-1935] Add BalanceHero to readme +[AIRFLOW-1939] add astronomer contributors +[AIRFLOW-1517] Kubernetes Operator +[AIRFLOW-1928] Fix @once with catchup=False +[AIRFLOW-1937] Speed up scheduling by committing in batch +[AIRFLOW-1821] Enhance default logging config by removing extra loggers +[AIRFLOW-1904] Correct DAG fileloc to the right filepath +[AIRFLOW-1909] Update docs with supported versions of MySQL server +[AIRFLOW-1915] Relax flask-wtf dependency specification +[AIRFLOW-1920] Update CONTRIBUTING.md to reflect enforced linting rules +[AIRFLOW-1942] Update Sphinx docs to remove deprecated import structure +[AIRFLOW-1846][AIRFLOW-1697] Hide Ad Hoc Query behind secure_mode config +[AIRFLOW-1948] Include details for on_kill failure +[AIRFLOW-1938] Clean up unused exception +[AIRFLOW-1932] Add GCP Pub/Sub Pull and Ack +[AIRFLOW-XXX] Purge coveralls +[AIRFLOW-XXX] Remove unused coveralls token +[AIRFLOW-1938] Remove tag version check in setup.py +[AIRFLOW-1916] Don't upload logs to remote from `run --raw` +[AIRFLOW-XXX] Fix failing PubSub tests on Python3 +[AIRFLOW-XXX] Upgrade to python 3.5 and disable dask tests +[AIRFLOW-1913] Add new GCP PubSub operators +[AIRFLOW-1525] Fix minor LICENSE and NOTICE issues +[AIRFLOW-1687] fix fernet error without encryption +[AIRFLOW-1912] airflow.processor should not propagate logging +[AIRFLOW-1911] Rename celeryd_concurrency +[AIRFLOW-1885] Fix IndexError in ready_prefix_on_cmdline +[AIRFLOW-1854] Improve Spark Submit operator for standalone cluster mode +[AIRFLOW-1908] Fix celery broker options config load +[AIRFLOW-1907] Pass max_ingestion_time to Druid hook +[AIRFLOW-1909] Add away to list of users +[AIRFLOW-1893][AIRFLOW-1901] Propagate PYTHONPATH when using impersonation +[AIRFLOW-1892] Modify BQ hook to extract data filtered by column +[AIRFLOW-1829] Support for schema updates in query jobs +[AIRFLOW-1840] Make celery configuration congruent with Celery 4 +[AIRFLOW-1878] Fix stderr/stdout redirection for tasks +[AIRFLOW-1897][AIRFLOW-1873] Task Logs for running instance not visible in WebUI +[AIRFLOW-1896] FIX bleach <> html5lib incompatibility +[AIRFLOW-1884][AIRFLOW-1059] Reset orphaned task state for external dagruns +[AIRFLOW-XXX] Fix typo in comment +[AIRFLOW-1869] Do not emit spurious warning on missing logs +[AIRFLOW-1888] Add AWS Redshift Cluster Sensor +[AIRFLOW-1887] Renamed endpoint url variable +[AIRFLOW-1873] Set TI.try_number to right value depending TI state +[AIRFLOW-1891] Fix non-ascii typo in default configuration template +[AIRFLOW-1879] Handle ti log entirely within ti +[AIRFLOW-1869] Write more error messages into gcs and file logs +[AIRFLOW-1876] Write subtask id to task log header +[AIRFLOW-1554] Fix wrong DagFileProcessor termination method call +[AIRFLOW-342] Do not use amqp, rpc as result backend +[AIRFLOW-966] Make celery broker_transport_options configurable +[AIRFLOW-1881] Make operator log in task log +[AIRFLOW-XXX] Added DataReply to the list of Airflow Users +[AIRFLOW-1883] Get File Size for objects in Google Cloud Storage +[AIRFLOW-1872] Set context for all handlers including parents +[AIRFLOW-1855][AIRFLOW-1866] Add GCS Copy Operator to copy multiple files +[AIRFLOW-1870] Enable flake8 tests +[AIRFLOW-1785] Enable Python 3 tests +[AIRFLOW-1850] Copy cmd before masking +[AIRFLOW-1665] Reconnect on database errors +[AIRFLOW-1559] Dispose SQLAlchemy engines on exit +[AIRFLOW-1559] Close file handles in subprocesses +[AIRFLOW-1559] Make database pooling optional +[AIRFLOW-1848][Airflow-1848] Fix DataFlowPythonOperator py_file extension doc comment +[AIRFLOW-1843] Add Google Cloud Storage Sensor with prefix +[AIRFLOW-1803] Time zone documentation +[AIRFLOW-1826] Update views to use timezone aware objects +[AIRFLOW-1827] Fix api endpoint date parsing +[AIRFLOW-1806] Use naive datetime when using cron +[AIRFLOW-1809] Update tests to use timezone aware objects +[AIRFLOW-1806] Use naive datetime for cron scheduling +[AIRFLOW-1807] Force use of time zone aware db fields +[AIRFLOW-1808] Convert all utcnow() to time zone aware +[AIRFLOW-1804] Add time zone configuration options +[AIRFLOW-1802] Convert database fields to timezone aware +[AIRFLOW-XXX] Add dask lock files to excludes +[AIRFLOW-1790] Add support for AWS Batch operator +[AIRFLOW-XXX] Update README.md +[AIRFLOW-1820] Remove timestamp from metric name +[AIRFLOW-1810] Remove unused mysql import in migrations. +[AIRFLOW-1838] Properly log collect_dags exception +[AIRFLOW-1842] Fixed Super class name for the gcs to gcs copy operator +[AIRFLOW-1845] Modal background now covers long or tall pages +[AIRFLOW-1229] Add link to Run Id, incl execution_date +[AIRFLOW-1842] Add gcs to gcs copy operator with renaming if required +[AIRFLOW-1841] change False to None in operator and hook +[AIRFLOW-1839] Fix more bugs in S3Hook boto -> boto3 migration +[AIRFLOW-1830] Support multiple domains in Google authentication backend +[AIRFLOW-1831] Add driver-classpath spark submit +[AIRFLOW-1795] Correctly call S3Hook after migration to boto3 +[AIRFLOW-1811] Fix render Druid operator +[AIRFLOW-1819] Fix slack operator unittest bug +[AIRFLOW-1805] Allow Slack token to be passed through connection +[AIRFLOW-1816] Add region param to Dataproc operators +[AIRFLOW-868] Add postgres_to_gcs operator and unittests +[AIRFLOW-1613] make mysql_to_gcs_operator py3 compatible +[AIRFLOW-1817] use boto3 for s3 dependency +[AIRFLOW-1813] Bug SSH Operator empty buffer +[AIRFLOW-1801][AIRFLOW-288] Url encode execution dates +[AIRFLOW-1563] Catch OSError while symlinking the latest log directory +[AIRFLOW-1794] Remove uses of Exception.message for Python 3 +[AIRFLOW-1799] Fix logging line which raises errors +[AIRFLOW-1102] Upgrade Gunicorn >=19.4.0 +[AIRFLOW-1756] Fix S3TaskHandler to work with Boto3-based S3Hook +[AIRFLOW-1797] S3Hook.load_string didn't work on Python3 +[AIRFLOW-646] Add docutils to setup_requires +[AIRFLOW-1792] Missing intervals DruidOperator +[AIRFLOW-1789][AIRFLOW-1712] Log SSHOperator stderr to log.warning +[AIRFLOW-1787] Fix task instance batch clear and set state bugs +[AIRFLOW-1780] Fix long output lines with unicode from hanging parent +[AIRFLOW-387] Close SQLAlchemy sessions properly +[AIRFLOW-1779] Add keepalive packets to ssh hook +[AIRFLOW-1669] Fix Docker and pin Moto to 1.1.19 +[AIRFLOW-71] Add support for private Docker images +[AIRFLOW-XXX] Give a clue what the 'ds' variable is +[AIRFLOW-XXX] Correct typos in the faq docs page +[AIRFLOW-1571] Add AWS Lambda Hook +[AIRFLOW-1675] Fix docstrings for API docs +[AIRFLOW-1712][AIRFLOW-756][AIRFLOW-751] Log SSHOperator output +[AIRFLOW-1776] Capture stdout and stderr for logging +[AIRFLOW-1765] Make experimental API securable without needing Kerberos. +[AIRFLOW-1764] The web interface should not use the experimental API +[AIRFLOW-1771] Rename heartbeat to avoid confusion +[AIRFLOW-1769] Add support for templates in VirtualenvOperator +[AIRFLOW-1763] Fix S3TaskHandler unit tests +[AIRFLOW-1315] Add Qubole File & Partition Sensors +[AIRFLOW-1018] Make processor use logging framework +[AIRFLOW-1695] Add RedshiftHook using boto3 +[AIRFLOW-1706] Fix query error for MSSQL backend +[AIRFLOW-1711] Use ldap3 dict for group membership +[AIRFLOW-1723] Make sendgrid a plugin +[AIRFLOW-1757] Add missing options to SparkSubmitOperator +[AIRFLOW-1734][Airflow 1734] Sqoop hook/operator enhancements +[AIRFLOW-1761] Fix type in scheduler.rst +[AIRFLOW-1731] Set pythonpath for logging +[AIRFLOW-1641] Handle executor events in the scheduler +[AIRFLOW-1744] Make sure max_tries can be set +[AIRFLOW-1732] Improve dataflow hook logging +[AIRFLOW-1736] Add HotelQuickly to Who Uses Airflow +[AIRFLOW-1657] Handle failing qubole operator +[AIRFLOW-1677] Fix typo in example_qubole_operator +[AIRFLOW-926] Fix JDBC Hook +[AIRFLOW-1520] Boto3 S3Hook, S3Log +[AIRFLOW-1716] Fix multiple __init__ def in SimpleDag +[AIRFLOW-XXX] Fix DateTime in Tree View +[AIRFLOW-1719] Fix small typo +[AIRFLOW-1432] Charts label for Y axis not visible +[AIRFLOW-1743] Verify ldap filters correctly +[AIRFLOW-1745] Restore default signal disposition +[AIRFLOW-1741] Correctly hide second chart on task duration page +[AIRFLOW-1728] Add networkUri, subnet, tags to Dataproc operator +[AIRFLOW-1726] Add copy_expert psycopg2 method to PostgresHook +[AIRFLOW-1330] Add conn_type argument to CLI when adding connection +[AIRFLOW-1698] Remove SCHEDULER_RUNS env var in systemd +[AIRFLOW-1694] Stop using itertools.izip +[AIRFLOW-1692] Change test_views filename to support Windows +[AIRFLOW-1722] Fix typo in scheduler autorestart output filename +[AIRFLOW-1723] Support sendgrid in email backend +[AIRFLOW-1718] Set num_retries on Dataproc job request execution +[AIRFLOW-1727] Add unit tests for DataProcHook +[AIRFLOW-1631] Fix timing issue in unit test +[AIRFLOW-1631] Fix local executor unbound parallelism +[AIRFLOW-1724] Add Fundera to Who uses Airflow? +[AIRFLOW-1683] Cancel BigQuery job on timeout. +[AIRFLOW-1714] Fix misspelling: s/seperate/separate/ +[AIRFLOW-1681] Add batch clear in task instance view +[AIRFLOW-1696] Fix dataproc version label error +[AIRFLOW-1613] Handle binary field in MySqlToGoogleCloudStorageOperator +[AIRFLOW-1697] Mode to disable charts endpoint +[AIRFLOW-1691] Add better Google cloud logging documentation +[AIRFLOW-1690] Add detail to gcs error messages +[AIRFLOW-1682] Make S3TaskHandler write to S3 on close +[AIRFLOW-1634] Adds task_concurrency feature +[AIRFLOW-1676] Make GCSTaskHandler write to GCS on close +[AIRFLOW-1678] Fix erroneously repeated word in function docstrings +[AIRFLOW-1323] Made Dataproc operator parameter names consistent +[AIRFLOW-1590] fix unused module and variable +[AIRFLOW-1671] Add @apply_defaults back to gcs download operator +[AIRFLOW-988] Fix repeating SLA miss callbacks +[AIRFLOW-1611] Customize logging +[AIRFLOW-1668] Expose keepalives_idle for Postgres connections +[AIRFLOW-1658] Kill Druid task on timeout +[AIRFLOW-1669][AIRFLOW-1368] Fix Docker import +[AIRFLOW-891] Make webserver clock include date +[AIRFLOW-1560] Add AWS DynamoDB hook and operator for inserting batch items +[AIRFLOW-1654] Show tooltips for link icons in DAGs view +[AIRFLOW-1660] Change webpage width to full-width +[AIRFLOW-1664] write file as binary instead of str +[AIRFLOW-1659] Fix invalid obj attribute bug in file_task_handler.py +[AIRFLOW-1635] Allow creating GCP connection without requiring a JSON file +[AIRFLOW-1650] Fix custom celery config loading +[AIRFLOW-1647] Fix Spark-sql hook +[AIRFLOW-1587] Fix CeleryExecutor import error +[Airflow-1640][AIRFLOW-1640] Add qubole default connection +[AIRFLOW-1576] Added region param to Dataproc{*}Operators +[AIRFLOW-1643] Add healthjump to officially using list +[AIRFLOW-1626] Add Azri Solutions to Airflow users +[AIRFLOW-1636] Add AWS and EMR connection type +[AIRFLOW-1527] Refactor celery config +[AIRFLOW-1639] Fix Fernet error handling +[AIRFLOW-1637] Fix Travis CI build status link +[AIRFLOW-1628] Fix docstring of sqlsensor +[AIRFLOW-1331] add SparkSubmitOperator option +[AIRFLOW-1627] Only query pool in SubDAG init when necessary +[AIRFLOW-1629] Make extra a textarea in edit connections form +[AIRFLOW-1368] Automatically remove Docker container on exit +[AIRFLOW-289] Make airflow timezone independent +[AIRFLOW-1356] Add `--celery_hostname` to `airflow worker` +[AIRFLOW-1247] Fix ignore_all_dependencies argument ignored +[AIRFLOW-1621] Add tests for server side paging +[AIRFLOW-1591] Avoid attribute error when rendering logging filename +[AIRFLOW-1031] Replace hard-code to DagRun.ID_PREFIX +[AIRFLOW-1604] Rename logger to log +[AIRFLOW-1512] Add PythonVirtualenvOperator +[AIRFLOW-1617] Fix XSS vulnerability in Variable endpoint +[AIRFLOW-1497] Reset hidden fields when changing connection type +[AIRFLOW-1619] Add poll_sleep parameter to GCP dataflow operator +[AIRFLOW-XXX] Remove landscape.io config +[AIRFLOW-XXX] Remove non working service badges +[AIRFLOW-1177] Fix Variable.setdefault w/existing JSON +[AIRFLOW-1600] Fix exception handling in get_fernet +[AIRFLOW-1614] Replace inspect.stack() with sys._getframe() +[AIRFLOW-1519] Add server side paging in DAGs list +[AIRFLOW-1309] Allow hive_to_druid to take tblproperties +[AIRFLOW-1613] Make MySqlToGoogleCloudStorageOperator compaitible with python3 +[AIRFLOW-1603] add PAYMILL to companies list +[AIRFLOW-1609] Fix gitignore to ignore all venvs +[AIRFLOW-1601] Add configurable task cleanup time + AIRFLOW 1.9.0, 2018-01-02 ------------------------- From f1083cbada337731ed0b7e27b09eee7a26c8189a Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sun, 8 Jul 2018 21:58:00 +0200 Subject: [PATCH 018/808] no incubating --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index 02346d56fdef0..be4038b4af24d 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.0+incubating' +version = '1.10.0' From 8e54a2917ef86d5cbdd8533a4fd4f38eaf6cc144 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Fri, 13 Jul 2018 11:46:03 +0200 Subject: [PATCH 019/808] [AIRFLOW-XXX] Wrap DISCLAIMER text (cherry picked from commit 39adc77f6fefc54fcfa893362b576070a4c3e660) Signed-off-by: Bolke de Bruin --- DISCLAIMER | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/DISCLAIMER b/DISCLAIMER index 8fe69887c3638..2758508789843 100644 --- a/DISCLAIMER +++ b/DISCLAIMER @@ -1 +1,6 @@ -Apache Airflow is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF. +Apache Airflow is an effort undergoing incubation at The Apache Software Foundation (ASF), +sponsored by the Apache Incubator. Incubation is required of all newly accepted projects +until a further review indicates that the infrastructure, communications, and decision +making process have stabilized in a manner consistent with other successful ASF projects. +While incubation status is not necessarily a reflection of the completeness or stability +of the code, it does indicate that the project has yet to be fully endorsed by the ASF. From f123cf5f8de71152d1b2b39c5c994bfc00a67d72 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Fri, 13 Jul 2018 11:42:41 +0200 Subject: [PATCH 020/808] [AIRFLOW-XXX] Use .sha512 for signature As requested by the IPMC (cherry picked from commit a54f74821e161fd9e5423f5cfebaeab43a021422) Signed-off-by: Bolke de Bruin --- dev/sign.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/sign.sh b/dev/sign.sh index da0f7a94576b2..66427a7e87ce8 100755 --- a/dev/sign.sh +++ b/dev/sign.sh @@ -25,5 +25,5 @@ NAME=${1} gpg --armor --output ${NAME}.asc --detach-sig ${NAME} -gpg --print-md SHA512 ${NAME} > ${NAME}.sha +gpg --print-md SHA512 ${NAME} > ${NAME}.sha512 gpg --print-md MD5 ${NAME} > ${NAME}.md5 From f4bcc33564d68b2146fb4fbf6a08f20326069306 Mon Sep 17 00:00:00 2001 From: Carl Johan Gustavsson Date: Fri, 13 Jul 2018 11:58:39 +0200 Subject: [PATCH 021/808] [AIRFLOW-2739] Always read default configuration files as utf-8 Closes #3593 from cjgu/airflow-2739 (cherry picked from commit 78da52fee477b0f7b2ee3481d32a06fca7a9841c) Signed-off-by: Bolke de Bruin --- airflow/configuration.py | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index e2089e5b5b94b..2e05fde0cdea2 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -101,15 +101,20 @@ def run_command(command): return output -_templates_dir = os.path.join(os.path.dirname(__file__), 'config_templates') -with open(os.path.join(_templates_dir, 'default_airflow.cfg')) as f: - DEFAULT_CONFIG = f.read() +def _read_default_config_file(file_name): + templates_dir = os.path.join(os.path.dirname(__file__), 'config_templates') + file_path = os.path.join(templates_dir, file_name) if six.PY2: - DEFAULT_CONFIG = DEFAULT_CONFIG.decode('utf-8') -with open(os.path.join(_templates_dir, 'default_test.cfg')) as f: - TEST_CONFIG = f.read() - if six.PY2: - TEST_CONFIG = TEST_CONFIG.decode('utf-8') + with open(file_path) as f: + config = f.read() + return config.decode('utf-8') + else: + with open(file_path, encoding='utf-8') as f: + return f.read() + + +DEFAULT_CONFIG = _read_default_config_file('default_airflow.cfg') +TEST_CONFIG = _read_default_config_file('default_test.cfg') class AirflowConfigParser(ConfigParser): @@ -502,8 +507,7 @@ def parameterized_config(template): if conf.getboolean('webserver', 'rbac'): - with open(os.path.join(_templates_dir, 'default_webserver_config.py')) as f: - DEFAULT_WEBSERVER_CONFIG = f.read() + DEFAULT_WEBSERVER_CONFIG = _read_default_config_file('default_webserver_config.py') WEBSERVER_CONFIG = AIRFLOW_HOME + '/webserver_config.py' From 23191605e463c85f0935cbf5c47f31c357d1596b Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Fri, 13 Jul 2018 13:18:47 +0200 Subject: [PATCH 022/808] [AIRFLOW-1729][AIRFLOW-2797][AIRFLOW-2729] Ignore whole directories in .airflowignore We can ignore whole directories by removing them from the `dirs` array that `os.walk()` returns. Doing this means that we fewer disk ops if someone has a set of modules in their dag folder that they want to ignore. Also fixes [AIRFLOW-2797] - we weren't honoring .airflowignore from a parent dir as of #3717 -- that (expected) behaviour is now back again. De-duplicate the walking code as well - we had two versions that had gotten out of sync as of #3171. So that doesn't happen again we now only have one version. Closes #3602 from ashb/ignore-whole-dirs- airflowignore (cherry picked from commit 6b2fdbef0ab4bd1ed91e6338bcf6440e782b7035) Signed-off-by: Bolke de Bruin --- airflow/models.py | 58 ++++++++++++--------------------- airflow/utils/dag_processing.py | 29 +++++++++++++---- 2 files changed, 44 insertions(+), 43 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index bde6137022013..5f0713f636b40 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -510,7 +510,8 @@ def collect_dags( Note that if a .airflowignore file is found while processing, the directory, it will behaves much like a .gitignore does, ignoring files that match any of the regex patterns specified - in the file. + in the file. **Note**: The patterns in .airflowignore are treated as + un-anchored regexes, not shell-like glob patterns. """ start_dttm = timezone.utcnow() dag_folder = dag_folder or self.dag_folder @@ -519,42 +520,25 @@ def collect_dags( stats = [] FileLoadStat = namedtuple( 'FileLoadStat', "file duration dag_num task_num dags") - if os.path.isfile(dag_folder): - self.process_file(dag_folder, only_if_updated=only_if_updated) - elif os.path.isdir(dag_folder): - for root, dirs, files in os.walk(dag_folder, followlinks=True): - patterns = [] - ignore_file = os.path.join(root, '.airflowignore') - if os.path.isfile(ignore_file): - with open(ignore_file, 'r') as f: - patterns += [p for p in f.read().split('\n') if p] - for f in files: - try: - filepath = os.path.join(root, f) - if not os.path.isfile(filepath): - continue - mod_name, file_ext = os.path.splitext( - os.path.split(filepath)[-1]) - if file_ext != '.py' and not zipfile.is_zipfile(filepath): - continue - if not any( - [re.findall(p, filepath) for p in patterns]): - ts = timezone.utcnow() - found_dags = self.process_file( - filepath, only_if_updated=only_if_updated) - - td = timezone.utcnow() - ts - td = td.total_seconds() + ( - float(td.microseconds) / 1000000) - stats.append(FileLoadStat( - filepath.replace(dag_folder, ''), - td, - len(found_dags), - sum([len(dag.tasks) for dag in found_dags]), - str([dag.dag_id for dag in found_dags]), - )) - except Exception as e: - self.log.exception(e) + for filepath in utils.dag_processing.list_py_file_paths(dag_folder): + self.log.info(filepath) + try: + ts = timezone.utcnow() + found_dags = self.process_file( + filepath, only_if_updated=only_if_updated) + + td = timezone.utcnow() - ts + td = td.total_seconds() + ( + float(td.microseconds) / 1000000) + stats.append(FileLoadStat( + filepath.replace(dag_folder, ''), + td, + len(found_dags), + sum([len(dag.tasks) for dag in found_dags]), + str([dag.dag_id for dag in found_dags]), + )) + except Exception as e: + self.log.exception(e) Stats.gauge( 'collect_dags', (timezone.utcnow() - start_dttm).total_seconds(), 1) Stats.gauge( diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index 543eb41692e03..e236397da0284 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -177,13 +177,30 @@ def list_py_file_paths(directory, safe_mode=True): elif os.path.isfile(directory): return [directory] elif os.path.isdir(directory): - patterns = [] + patterns_by_dir = {} for root, dirs, files in os.walk(directory, followlinks=True): - ignore_file = [f for f in files if f == '.airflowignore'] - if ignore_file: - f = open(os.path.join(root, ignore_file[0]), 'r') - patterns += [p for p in f.read().split('\n') if p] - f.close() + patterns = patterns_by_dir.get(root, []) + ignore_file = os.path.join(root, '.airflowignore') + if os.path.isfile(ignore_file): + with open(ignore_file, 'r') as f: + # If we have new patterns create a copy so we don't change + # the previous list (which would affect other subdirs) + patterns = patterns + [p for p in f.read().split('\n') if p] + + # If we can ignore any subdirs entirely we should - fewer paths + # to walk is better. We have to modify the ``dirs`` array in + # place for this to affect os.walk + dirs[:] = [ + d + for d in dirs + if not any(re.search(p, os.path.join(root, d)) for p in patterns) + ] + + # We want patterns defined in a parent folder's .airflowignore to + # apply to subdirs too + for d in dirs: + patterns_by_dir[os.path.join(root, d)] = patterns + for f in files: try: file_path = os.path.join(root, f) From 48847d737e5e4565f77501ef52d61060ecf072f4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 15 Jul 2018 17:56:18 +0200 Subject: [PATCH 023/808] [AIRFLOW-2752] Log using logging instead of stdout Logging should be done using the Python log module instead of writing directly to stdout Closes #3604 from Fokko/patch-2 (cherry picked from commit 846e2746c0be3d9e55edd3351ae9f43228ea48a5) Signed-off-by: Bolke de Bruin --- airflow/executors/base_executor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 04c90884012f7..701ac66f8b72d 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -150,7 +150,7 @@ def heartbeat(self): self.sync() def change_state(self, key, state): - print("popping: {}".format(key)) + self.log.debug("Changing state: {}".format(key)) self.running.pop(key) self.event_buffer[key] = state From fc00875ed00555e5da5661324479e8703787bbd5 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Sun, 15 Jul 2018 17:59:32 +0200 Subject: [PATCH 024/808] [AIRFLOW-1729][AIRFLOW-XXX] Remove extra debug log at info level I left an extra log call, at info level in #3602 that was being used for debugging. Closes #3603 from ashb/remove-extra-log (cherry picked from commit 7c19ed02bc30f71fd73e38234b3b210072274f3b) Signed-off-by: Bolke de Bruin --- airflow/models.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/models.py b/airflow/models.py index 5f0713f636b40..cb0165a2d0125 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -521,7 +521,6 @@ def collect_dags( FileLoadStat = namedtuple( 'FileLoadStat', "file duration dag_num task_num dags") for filepath in utils.dag_processing.list_py_file_paths(dag_folder): - self.log.info(filepath) try: ts = timezone.utcnow() found_dags = self.process_file( From 64dacb1ca319443f06e4860b817888946943c8f1 Mon Sep 17 00:00:00 2001 From: Kevin Yuen Date: Sat, 21 Jul 2018 12:38:21 -0400 Subject: [PATCH 025/808] [AIRFLOW-2778] Explicit import for dag_processing.list_py_file_paths The use of utils.dag_processing.list_py_file_paths causes a failure if utils.dag_processing is not already loaded indirectly. (cherry picked from commit ba5b67b6c99e0bfb0f827518cdb31b5eb5a5a626) Signed-off-by: Ash Berlin-Taylor --- airflow/models.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/models.py b/airflow/models.py index cb0165a2d0125..6b244cd22640a 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -79,6 +79,7 @@ from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, RUN_DEPS from airflow.utils import timezone +from airflow.utils.dag_processing import list_py_file_paths from airflow.utils.dates import cron_presets, date_range as utils_date_range from airflow.utils.db import provide_session from airflow.utils.decorators import apply_defaults @@ -520,7 +521,7 @@ def collect_dags( stats = [] FileLoadStat = namedtuple( 'FileLoadStat', "file duration dag_num task_num dags") - for filepath in utils.dag_processing.list_py_file_paths(dag_folder): + for filepath in list_py_file_paths(dag_folder): try: ts = timezone.utcnow() found_dags = self.process_file( From 3b8c4ff7a02bb7914ffab8edc810a69e77cca0e9 Mon Sep 17 00:00:00 2001 From: Nick Hughes Date: Sat, 28 Jul 2018 12:55:47 +0200 Subject: [PATCH 026/808] [AIRFLOW-2812] Fix error in Updating.md for upgrading to 1.10 Closes #3654 from nrhvyc/AIRFLOW-2812 --- UPDATING.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index c341b38f0bc22..80e5858192ff2 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -86,8 +86,8 @@ With Airflow 1.9 or lower, `FILENAME_TEMPLATE`, `PROCESSOR_FILENAME_TEMPLATE`, ` ``` [core] fab_logging_level = WARN -log_filename_template = {{{{ ti.dag_id }}}}/{{{{ ti.task_id }}}}/{{{{ ts }}}}/{{{{ try_number }}}}.log -log_processor_filename_template = {{{{ filename }}}}.log +log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log +log_processor_filename_template = {{ filename }}.log [elasticsearch] elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_number}} From 3b7e71f820bd312f2013d41eb7b2f9486a532036 Mon Sep 17 00:00:00 2001 From: Andy Wilcox Date: Fri, 27 Jul 2018 13:03:30 -0400 Subject: [PATCH 027/808] [AIRFLOW-2810] Fix typo in Xcom model timestamp Fix typo in Xcom model timestamp field No new testing - the field is already represented in migrations Closes #3652 from andywilco/fix_datetime_typo (cherry picked from commit a338f3276835af45765d24a6e6d43ad4ba4d66ba) Signed-off-by: Bolke de Bruin --- airflow/models.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models.py b/airflow/models.py index 6b244cd22640a..b789fd17323e2 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -4470,7 +4470,7 @@ class XCom(Base, LoggingMixin): key = Column(String(512)) value = Column(LargeBinary) timestamp = Column( - DateTime, default=timezone.utcnow, nullable=False) + UtcDateTime, default=timezone.utcnow, nullable=False) execution_date = Column(UtcDateTime, nullable=False) # source information From 90e89f0d89e8448748a31fefea8e4e97db884048 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sat, 21 Jul 2018 19:41:43 +0200 Subject: [PATCH 028/808] [AIRFLOW-XXX] Include license folder for binary (cherry picked from commit 27fde3840c47fbf5e4d7c429ed4a94de22f9f80c) Signed-off-by: Bolke de Bruin --- MANIFEST.in | 1 + 1 file changed, 1 insertion(+) diff --git a/MANIFEST.in b/MANIFEST.in index c04b80617d18d..b069194984511 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -21,6 +21,7 @@ include LICENSE include DISCLAIMER include CHANGELOG.txt include README.md +graft licenses/ graft airflow/www/templates graft airflow/www/static graft airflow/www_rbac/static From f0549211f5b026a5b2d5c3bc6b3b74d0bf0a6536 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sat, 28 Jul 2018 13:10:43 +0200 Subject: [PATCH 029/808] [AIRFLOW-2815] Use correct copyright period "onwards" is not specific enough (cherry picked from commit 4f562ba8c7f12b8de74e40fe67cac7b4f38413b1) Signed-off-by: Bolke de Bruin --- NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/NOTICE b/NOTICE index b1e78ad9c3380..a4727302be245 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Airflow -Copyright 2016 and onwards The Apache Software Foundation +Copyright 2016-2018 The Apache Software Foundation This product includes software developed at The Apache Software From eb34cd596b5995474aeaa6a13829df0d52673cde Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sat, 28 Jul 2018 14:25:05 +0200 Subject: [PATCH 030/808] [AIRFLOW-2779] Update licenses and remove copyright headers * ssh_hook is not a port anymore * auth backends should never had a copyright clause * minihivecluster isnt used (cherry picked from commit a5fa1bf8886ee3ced1306f8c2ff58541a72db96c) Signed-off-by: Bolke de Bruin --- LICENSE | 17 +- NOTICE | 17 +- .../auth/backends/github_enterprise_auth.py | 2 +- airflow/contrib/auth/backends/google_auth.py | 2 +- airflow/contrib/hooks/ssh_hook.py | 3 - airflow/minihivecluster.py | 47 ---- licenses/LICENSE-bootstrap3-typeahead.txt | 20 ++ licenses/LICENSE-hue.txt | 202 ++++++++++++++++++ licenses/LICENSE-jqclock.txt | 201 +++++++++++++++++ licenses/LICENSE-scikit-learn.txt | 31 +++ 10 files changed, 487 insertions(+), 55 deletions(-) delete mode 100644 airflow/minihivecluster.py create mode 100644 licenses/LICENSE-bootstrap3-typeahead.txt create mode 100644 licenses/LICENSE-hue.txt create mode 100644 licenses/LICENSE-jqclock.txt create mode 100644 licenses/LICENSE-scikit-learn.txt diff --git a/LICENSE b/LICENSE index 405540c64fb87..2288b870b4239 100644 --- a/LICENSE +++ b/LICENSE @@ -209,6 +209,18 @@ limitations under the License. licenses. +======================================================================== +Third party Apache 2.0 licenses +======================================================================== + +The following components are provided under the Apache 2.0 License. +See project link for details. The text of each license is also included +at licenses/LICENSE-[project].txt. + + (ALv2 License) hue (https://github.com/cloudera/hue/) + (ALv2 License) jqclock (https://github.com/JohnRDOrazio/jQuery-Clock-Plugin) + (ALv2 License) bootstrap3-typeahead (https://github.com/bassjobsen/Bootstrap-3-Typeahead) + ======================================================================== MIT licenses ======================================================================== @@ -239,10 +251,11 @@ The text of each license is also included at licenses/LICENSE-[project].txt. ======================================================================== BSD 3-Clause licenses ======================================================================== -The following components are provided under the BSD 2-Clause license. -See file headers and project links for details. +The following components are provided under the BSD 3-Clause license. See project links for details. The text of each license is also included at licenses/LICENSE-[project].txt. (BSD 3 License) Ace (https://github.com/ajaxorg/ace) (BSD 3 License) d3js (https://d3js.org) (BSD 3 License) parallel-coordinates (http://syntagmatic.github.com/parallel-coordinates/) + (BSD 3 License) scikit-learn (https://github.com/scikit-learn/scikit-learn) + diff --git a/NOTICE b/NOTICE index a4727302be245..544c0807667fb 100644 --- a/NOTICE +++ b/NOTICE @@ -1,6 +1,21 @@ Apache Airflow Copyright 2016-2018 The Apache Software Foundation - This product includes software developed at The Apache Software Foundation (http://www.apache.org/). + +======================================================================= + +Apache Airflow contains subcomponents with separate copyright notices and +license terms. Your use of the source code for the these subcomponents +is subject to the terms and conditions of their respective licenses. + +See the LICENSE file for a list of subcomponents and dependencies and +their respective licenses. + +hue: +----- +This product contains a modified portion of 'Hue' developed by Cloudera, Inc. +(https://github.com/cloudera/hue/). + +* Copyright 2009-2017 Cloudera Inc. diff --git a/airflow/contrib/auth/backends/github_enterprise_auth.py b/airflow/contrib/auth/backends/github_enterprise_auth.py index 5196725156393..641b81e46da63 100644 --- a/airflow/contrib/auth/backends/github_enterprise_auth.py +++ b/airflow/contrib/auth/backends/github_enterprise_auth.py @@ -1,4 +1,4 @@ -# Copyright 2015 Matthew Pelland (matt@pelland.io) +# -*- coding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file diff --git a/airflow/contrib/auth/backends/google_auth.py b/airflow/contrib/auth/backends/google_auth.py index d1a35791db888..e41934b926c31 100644 --- a/airflow/contrib/auth/backends/google_auth.py +++ b/airflow/contrib/auth/backends/google_auth.py @@ -1,4 +1,4 @@ -# Copyright 2016 Ananya Mishra (am747@cornell.edu) +# -*- coding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file diff --git a/airflow/contrib/hooks/ssh_hook.py b/airflow/contrib/hooks/ssh_hook.py index f51f0fbd11948..b4de5e08eef00 100755 --- a/airflow/contrib/hooks/ssh_hook.py +++ b/airflow/contrib/hooks/ssh_hook.py @@ -1,8 +1,5 @@ # -*- coding: utf-8 -*- # -# Copyright 2012-2015 Spotify AB -# Ported to Airflow by Bolke de Bruin -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information diff --git a/airflow/minihivecluster.py b/airflow/minihivecluster.py deleted file mode 100644 index c5441c67a1b61..0000000000000 --- a/airflow/minihivecluster.py +++ /dev/null @@ -1,47 +0,0 @@ -# -*- coding: utf-8 -*- -# Copyright (c) 2016 Bolke de Bruin -# -# Licensed under the Apache License, Version 2.0 (the "License"); you may not -# use this file except in compliance with the License. You may obtain a copy of -# the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT -# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the -# License for the specific language governing permissions and limitations under -# the License. -import os -import subprocess -import select -import re - - -class MiniHiveCluster(object): - def __init__(self): - self._minicluster_home = os.environ['MINICLUSTER_HOME'] - self._minicluster_class = "com.ing.minicluster.MiniCluster" - self._start_mini_cluster() - self._is_started() - - def _start_mini_cluster(self): - classpath = os.path.join(self._minicluster_home, "*") - cmd = ["java", "-cp", classpath, self._minicluster_class] - - self.hive = subprocess.Popen(cmd, bufsize=0, stdout=subprocess.PIPE, - stderr=subprocess.PIPE, universal_newlines=True, - close_fds=True) - - def terminate(self): - self.hive.terminate() - - def _is_started(self): - while self.hive.poll() is None: - rlist, wlist, xlist = select.select([self.hive.stderr, self.hive.stdout], [], []) - for f in rlist: - line = f.readline() - print(line,) - m = re.match(".*Starting ThriftBinaryCLIService", line) - if m: - return True diff --git a/licenses/LICENSE-bootstrap3-typeahead.txt b/licenses/LICENSE-bootstrap3-typeahead.txt new file mode 100644 index 0000000000000..33c48448eb596 --- /dev/null +++ b/licenses/LICENSE-bootstrap3-typeahead.txt @@ -0,0 +1,20 @@ +/* ============================================================= + * bootstrap3-typeahead.js v4.0.2 + * https://github.com/bassjobsen/Bootstrap-3-Typeahead + * ============================================================= + * Original written by @mdo and @fat + * ============================================================= + * Copyright 2014 Bass Jobsen @bassjobsen + * + * Licensed under the Apache License, Version 2.0 (the 'License'); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an 'AS IS' BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ============================================================ */ diff --git a/licenses/LICENSE-hue.txt b/licenses/LICENSE-hue.txt new file mode 100644 index 0000000000000..8be4f5ecb89a2 --- /dev/null +++ b/licenses/LICENSE-hue.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2009-2017 Cloudera Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/licenses/LICENSE-jqclock.txt b/licenses/LICENSE-jqclock.txt new file mode 100644 index 0000000000000..8dada3edaf50d --- /dev/null +++ b/licenses/LICENSE-jqclock.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/licenses/LICENSE-scikit-learn.txt b/licenses/LICENSE-scikit-learn.txt new file mode 100644 index 0000000000000..ecd7256f20d2d --- /dev/null +++ b/licenses/LICENSE-scikit-learn.txt @@ -0,0 +1,31 @@ +New BSD License + +Copyright (c) 2007–2018 The scikit-learn developers. +All rights reserved. + + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + a. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + b. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + c. Neither the name of the Scikit-learn Developers nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH +DAMAGE. From 5f3a4cf3037a6caa2f464f3c9c43cb02513478aa Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Sat, 28 Jul 2018 13:20:26 +0200 Subject: [PATCH 031/808] [AIRFLOW-2816] Fix license text in docs/license.rst (cherry picked from commit af15f1150d3c2e02df9a8d330ac3dcc18c9d3e97) Signed-off-by: Bolke de Bruin --- docs/license.rst | 40 +++------------------------------------- 1 file changed, 3 insertions(+), 37 deletions(-) diff --git a/docs/license.rst b/docs/license.rst index f920a01658bb7..3c53035713267 100644 --- a/docs/license.rst +++ b/docs/license.rst @@ -6,9 +6,9 @@ License :: - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION @@ -180,37 +180,3 @@ License defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright 2015 Apache Software Foundation - - Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. - Status API Training Shop Blog About - © 2016 GitHub, Inc. Terms Privacy Security Contact Help From c720c352f0762a483fa1202e1477e740631a8184 Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Sun, 29 Jul 2018 11:56:41 +0200 Subject: [PATCH 032/808] [AIRFLOW-2716] Replace async and await py3.7 keywords Closes #3578 from JacobHayes/py37-keywords (cherry picked from commit fcd51f362c6804cb69dee8e2dd054304a0564dde) Signed-off-by: Bolke de Bruin (cherry picked from commit 0423060bb20c54035fafecda8cc0a50e65953f27) Signed-off-by: Bolke de Bruin --- UPDATING.md | 14 ++++++++++++++ airflow/contrib/hooks/gcp_dataproc_hook.py | 12 +++++++++++- airflow/contrib/operators/dataproc_operator.py | 2 +- airflow/executors/celery_executor.py | 12 ++++++------ tests/contrib/operators/test_dataproc_operator.py | 8 ++++---- 5 files changed, 36 insertions(+), 12 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 80e5858192ff2..fe0e2356abbc5 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -5,6 +5,20 @@ assists users migrating to a new version. ## Airflow Master +### Replace DataProcHook.await calls to DataProcHook.wait + +The method name was changed to be compatible with the Python 3.7 async/await keywords + +### DAG level Access Control for new RBAC UI + +Extend and enhance new Airflow RBAC UI to support DAG level ACL. Each dag now has two permissions(one for write, one for read) associated('can_dag_edit', 'can_dag_read'). +The admin will create new role, associate the dag permission with the target dag and assign that role to users. That user can only access / view the certain dags on the UI +that he has permissions on. If a new role wants to access all the dags, the admin could associate dag permissions on an artificial view(``all_dags``) with that role. + +We also provide a new cli command(``sync_perm``) to allow admin to auto sync permissions. + +### Setting UTF-8 as default mime_charset in email utils + ### Add a configuration variable(default_dag_run_display_number) to control numbers of dag run for display Add a configuration variable(default_dag_run_display_number) under webserver section to control num of dag run to show in UI. diff --git a/airflow/contrib/hooks/gcp_dataproc_hook.py b/airflow/contrib/hooks/gcp_dataproc_hook.py index fc15137cbfa91..8e4f32b13790c 100644 --- a/airflow/contrib/hooks/gcp_dataproc_hook.py +++ b/airflow/contrib/hooks/gcp_dataproc_hook.py @@ -21,6 +21,7 @@ import uuid from apiclient.discovery import build +from zope.deprecation import deprecation from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook from airflow.utils.log.logging_mixin import LoggingMixin @@ -224,7 +225,16 @@ def create_job_template(self, task_id, cluster_name, job_type, properties): return _DataProcJobBuilder(self.project_id, task_id, cluster_name, job_type, properties) - def await(self, operation): + def wait(self, operation): """Awaits for Google Cloud Dataproc Operation to complete.""" submitted = _DataProcOperation(self.get_conn(), operation) submitted.wait_for_done() + + +setattr( + DataProcHook, + "await", + deprecation.deprecated( + DataProcHook.wait, "renamed to 'wait' for Python3.7 compatability" + ), +) diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index 5d59f7fb6eb58..e72a66569d5b6 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -1216,7 +1216,7 @@ def __init__(self, ) def execute(self, context): - self.hook.await(self.start()) + self.hook.wait(self.start()) def start(self, context): raise AirflowException('plese start a workflow operation') diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py index 6cfd2d3769893..481daa5826686 100644 --- a/airflow/executors/celery_executor.py +++ b/airflow/executors/celery_executor.py @@ -89,9 +89,9 @@ def execute_async(self, key, command, def sync(self): self.log.debug("Inquiring about %s celery task(s)", len(self.tasks)) - for key, async in list(self.tasks.items()): + for key, task in list(self.tasks.items()): try: - state = async.state + state = task.state if self.last_state[key] != state: if state == celery_states.SUCCESS: self.success(key) @@ -106,8 +106,8 @@ def sync(self): del self.tasks[key] del self.last_state[key] else: - self.log.info("Unexpected state: %s", async.state) - self.last_state[key] = async.state + self.log.info("Unexpected state: %s", task.state) + self.last_state[key] = task.state except Exception as e: self.log.error("Error syncing the celery executor, ignoring it:") self.log.exception(e) @@ -115,7 +115,7 @@ def sync(self): def end(self, synchronous=False): if synchronous: while any([ - async.state not in celery_states.READY_STATES - for async in self.tasks.values()]): + task.state not in celery_states.READY_STATES + for task in self.tasks.values()]): time.sleep(5) self.sync() diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py index 65ff5cd756c89..aea130349bdea 100644 --- a/tests/contrib/operators/test_dataproc_operator.py +++ b/tests/contrib/operators/test_dataproc_operator.py @@ -508,7 +508,7 @@ def test_workflow(self): with patch(HOOK) as MockHook: hook = MockHook() hook.get_conn.return_value = self.mock_conn - hook.await.return_value = None + hook.wait.return_value = None dataproc_task = DataprocWorkflowTemplateInstantiateOperator( task_id=TASK_ID, @@ -525,7 +525,7 @@ def test_workflow(self): self.mock_workflows.instantiate.assert_called_once_with( name=template_name, body=mock.ANY) - hook.await.assert_called_once_with(self.operation) + hook.wait.assert_called_once_with(self.operation) class DataprocWorkflowTemplateInstantiateInlineOperatorTest(unittest.TestCase): @@ -556,7 +556,7 @@ def test_iniline_workflow(self): with patch(HOOK) as MockHook: hook = MockHook() hook.get_conn.return_value = self.mock_conn - hook.await.return_value = None + hook.wait.return_value = None template = { "placement": { @@ -591,4 +591,4 @@ def test_iniline_workflow(self): parent='projects/test-project-id/regions/test-region', instanceId=mock.ANY, body=template) - hook.await.assert_called_once_with(self.operation) + hook.wait.assert_called_once_with(self.operation) From cefcf4c61f64be3792cbfed509b82a9eb4cc47be Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Wed, 1 Aug 2018 11:25:31 +0200 Subject: [PATCH 033/808] [AIRFLOW-2817] Force explicit choice on GPL dependency (#3660) By default one of Apache Airflow's dependencies pulls in a GPL library. Airflow should not install (and upgrade) without an explicit choice. This is part of the Apache requirements as we cannot depend on Category X software. (cherry picked from commit c37fc0b6ba19e3fe5656ae37cef9b59cef3c29e8) Signed-off-by: Bolke de Bruin (cherry picked from commit b39e4532d9d1086c60b31553d08972bcc68df641) Signed-off-by: Bolke de Bruin --- .travis.yml | 1 + INSTALL | 31 +++++++++++++++++++------ UPDATING.md | 6 +++++ scripts/ci/kubernetes/docker/Dockerfile | 3 ++- setup.py | 14 +++++++++++ tox.ini | 7 +++--- 6 files changed, 51 insertions(+), 11 deletions(-) diff --git a/.travis.yml b/.travis.yml index 01c08d9537f64..ebc1893b1b551 100644 --- a/.travis.yml +++ b/.travis.yml @@ -46,6 +46,7 @@ python: - "3.5" env: global: + - SLUGIFY_USES_TEXT_UNIDECODE=yes - TRAVIS_CACHE=$HOME/.travis_cache/ - KRB5_CONFIG=/etc/krb5.conf - KRB5_KTNAME=/etc/airflow.keytab diff --git a/INSTALL b/INSTALL index 5c8f03eb663e6..596ce25814088 100644 --- a/INSTALL +++ b/INSTALL @@ -1,13 +1,30 @@ -# INSTALL / BUILD instruction for Apache Airflow (incubating) -# fetch the tarball and untar the source +# INSTALL / BUILD instructions for Apache Airflow (incubating) + +# [required] fetch the tarball and untar the source +# change into the directory that was untarred. # [optional] run Apache RAT (release audit tool) to validate license headers -# RAT docs here: https://creadur.apache.org/rat/ +# RAT docs here: https://creadur.apache.org/rat/. Requires Java and Apache Rat java -jar apache-rat.jar -E ./.rat-excludes -d . -# [optional] by default one of Apache Airflow's dependencies pulls in a GPL -# library. If this is a concern issue (also every upgrade): -# export SLUGIFY_USES_TEXT_UNIDECODE=yes +# [optional] Airflow pulls in quite a lot of dependencies in order +# to connect to other services. You might want to test or run Airflow +# from a virtual env to make sure those dependencies are separated +# from your system wide versions +python -m my_env +source my_env/bin/activate + +# [required] by default one of Apache Airflow's dependencies pulls in a GPL +# library. Airflow will not install (and upgrade) without an explicit choice. +# +# To make sure not to install the GPL dependency: +# export SLUGIFY_USES_TEXT_UNIDECODE=yes +# In case you do not mind: +# export GPL_UNIDECODE=yes + +# [required] building and installing +# by pip (preferred) +pip install . -# install the release +# or directly python setup.py install diff --git a/UPDATING.md b/UPDATING.md index fe0e2356abbc5..3b47882212225 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -5,6 +5,12 @@ assists users migrating to a new version. ## Airflow Master +## Airflow 1.10 + +Installation and upgrading requires setting `SLUGIFY_USES_TEXT_UNIDECODE=yes` in your environment or +`AIRFLOW_GPL_UNIDECODE=yes`. In case of the latter a GPL runtime dependency will be installed due to a +dependency (python-nvd3 -> python-slugify -> unidecode). + ### Replace DataProcHook.await calls to DataProcHook.wait The method name was changed to be compatible with the Python 3.7 async/await keywords diff --git a/scripts/ci/kubernetes/docker/Dockerfile b/scripts/ci/kubernetes/docker/Dockerfile index 498c47b21a027..93b20dbcd22fa 100644 --- a/scripts/ci/kubernetes/docker/Dockerfile +++ b/scripts/ci/kubernetes/docker/Dockerfile @@ -17,6 +17,8 @@ FROM ubuntu:16.04 +ENV SLUGIFY_USES_TEXT_UNIDECODE=yes + # install deps RUN apt-get update -y && apt-get install -y \ wget \ @@ -33,7 +35,6 @@ RUN apt-get update -y && apt-get install -y \ unzip \ && apt-get clean - RUN pip install --upgrade pip # Since we install vanilla Airflow, we also want to have support for Postgres and Kubernetes diff --git a/setup.py b/setup.py index c91750057cef0..2f5f730af9b25 100644 --- a/setup.py +++ b/setup.py @@ -34,6 +34,17 @@ PY3 = sys.version_info[0] == 3 +# See LEGAL-362 +def verify_gpl_dependency(): + if (not os.getenv("AIRFLOW_GPL_UNIDECODE") + and not os.getenv("SLUGIFY_USES_TEXT_UNIDECODE") == "yes"): + raise RuntimeError("By default one of Airflow's dependencies installs a GPL " + "dependency (unidecode). To avoid this dependency set " + "SLUGIFY_USES_TEXT_UNIDECODE=yes in your environment when you " + "install or upgrade Airflow. To force installing the GPL " + "version set AIRFLOW_GPL_UNIDECODE") + + class Tox(TestCommand): user_options = [('tox-args=', None, "Arguments to pass to tox")] @@ -235,6 +246,7 @@ def write_version(filename=os.path.join(*['airflow', def do_setup(): + verify_gpl_dependency() write_version() setup( name='apache-airflow', @@ -353,6 +365,7 @@ def do_setup(): 'License :: OSI Approved :: Apache Software License', 'Programming Language :: Python :: 2.7', 'Programming Language :: Python :: 3.4', + 'Programming Language :: Python :: 3.5', 'Topic :: System :: Monitoring', ], author='Apache Software Foundation', @@ -364,6 +377,7 @@ def do_setup(): 'test': Tox, 'extra_clean': CleanCommand, }, + python_requires='>=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*', ) diff --git a/tox.ini b/tox.ini index 88fcd426183bd..a20d2c5546040 100644 --- a/tox.ini +++ b/tox.ini @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -28,7 +28,7 @@ find_links = [flake8] max-line-length = 90 -ignore = E731 +ignore = E731,W503 [testenv] deps = @@ -64,6 +64,7 @@ passenv = BOTO_CONFIG KRB5_CONFIG KRB5_KTNAME + SLUGIFY_USES_TEXT_UNIDECODE commands = pip wheel -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci] From 862ad8b9c2eb9af206d369bc661b6a2decc22148 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Fri, 3 Aug 2018 09:11:55 +0200 Subject: [PATCH 034/808] [AIRFLOW-XXX] Update changelog for 1.10 --- CHANGELOG.txt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index 00ee4a615ed0a..bfcef0f2d27d2 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -1,6 +1,8 @@ -AIRFLOW 1.10.0, 2018-07-08 +AIRFLOW 1.10.0, 2018-08-03 -------------------------- - +[AIRFLOW-2817] Force explicit choice on GPL dependency +[AIRFLOW-2716] Replace async and await py3.7 keywords +[AIRFLOW-2810] Fix typo in Xcom model timestamp [AIRFLOW-2710] Clarify fernet key value in documentation [AIRFLOW-2606] Fix DB schema and SQLAlchemy model [AIRFLOW-2646] Fix setup.py not to install snakebite on Python3 From c764c1114a9831443ea8ce6af89a8dd10f346ded Mon Sep 17 00:00:00 2001 From: "Taylor D. Edmiston" Date: Tue, 7 Aug 2018 10:20:30 -0400 Subject: [PATCH 035/808] [AIRFLOW-2857] Fix Read the Docs env (#3703) The Read the Docs build process was broken due to #3660. This PR fixes this. (cherry picked from commit 8af0aa96bfe3caa51d67ab393db069d37b0c4169) Signed-off-by: Bolke de Bruin --- .readthedocs.yml | 6 +++--- setup.py | 9 +++++++++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/.readthedocs.yml b/.readthedocs.yml index 643548c03e07a..c6a4da8d690c8 100644 --- a/.readthedocs.yml +++ b/.readthedocs.yml @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -22,4 +22,4 @@ python: - doc - docker - gcp_api - - emr + - emr diff --git a/setup.py b/setup.py index 2f5f730af9b25..25b3c8c5ca77c 100644 --- a/setup.py +++ b/setup.py @@ -36,6 +36,14 @@ # See LEGAL-362 def verify_gpl_dependency(): + # The Read the Docs build environment [1] does a pip install of Airflow which cannot + # be overridden with custom environment variables, so we detect the READTHEDOCS env + # var they provide to set the env var that avoids the GPL dependency on install when + # building the docs site. + # [1]: http://docs.readthedocs.io/en/latest/builds.html#build-environment + if os.getenv("READTHEDOCS") == "True": + os.environ["SLUGIFY_USES_TEXT_UNIDECODE"] = "yes" + if (not os.getenv("AIRFLOW_GPL_UNIDECODE") and not os.getenv("SLUGIFY_USES_TEXT_UNIDECODE") == "yes"): raise RuntimeError("By default one of Airflow's dependencies installs a GPL " @@ -142,6 +150,7 @@ def write_version(filename=os.path.join(*['airflow', databricks = ['requests>=2.5.1, <3'] datadog = ['datadog>=0.14.0'] doc = [ + 'mock', 'sphinx>=1.2.3', 'sphinx-argparse>=0.1.13', 'sphinx-rtd-theme>=0.1.6', From 26f6faed690846c963f90fe06458b3711d841e25 Mon Sep 17 00:00:00 2001 From: William Horton Date: Tue, 7 Aug 2018 15:48:28 -0700 Subject: [PATCH 036/808] [AIRFLOW-2869] Remove smart quote from default config Closes #3716 from wdhorton/remove-smart-quote- from-cfg (cherry picked from commit 67e2bb96cdc5ea37226d11332362d3bd3778cea0) Signed-off-by: Bolke de Bruin (cherry picked from commit 700f5f088dbead866170c9a3fe7e021e86ab30bb) Signed-off-by: Bolke de Bruin --- airflow/config_templates/default_airflow.cfg | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 2f6130c8975f7..9a15dc0d117e8 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -606,7 +606,7 @@ image_pull_secrets = gcp_service_account_keys = # Use the service account kubernetes gives to pods to connect to kubernetes cluster. -# It’s intended for clients that expect to be running inside a pod running on kubernetes. +# It's intended for clients that expect to be running inside a pod running on kubernetes. # It will raise an exception if called from a process not running in a kubernetes environment. in_cluster = True From 3a981389beefb1f45dff44a8748b2a2601963136 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 7 Aug 2018 19:43:41 +0100 Subject: [PATCH 037/808] [AIRFLOW-2140] Don't require kubernetes for the SparkSubmit hook (#3700) This extra dep is a quasi-breaking change when upgrading - previously there were no deps outside of Airflow itself for this hook. Importing the k8s libs breaks installs that aren't also using Kubernetes. This makes the dep optional for anyone who doesn't explicitly use the functionality (cherry picked from commit 0be002eebb182b607109a0390d7f6fb8795c668b) Signed-off-by: Bolke de Bruin (cherry picked from commit f58246d2ef265eb762c179a12c40e011ce62cea1) Signed-off-by: Bolke de Bruin --- airflow/contrib/hooks/spark_submit_hook.py | 7 +++++-- airflow/contrib/kubernetes/kube_client.py | 14 +++++++++++++- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/airflow/contrib/hooks/spark_submit_hook.py b/airflow/contrib/hooks/spark_submit_hook.py index 0185cab283345..65bb6134e6ac4 100644 --- a/airflow/contrib/hooks/spark_submit_hook.py +++ b/airflow/contrib/hooks/spark_submit_hook.py @@ -26,7 +26,6 @@ from airflow.exceptions import AirflowException from airflow.utils.log.logging_mixin import LoggingMixin from airflow.contrib.kubernetes import kube_client -from kubernetes.client.rest import ApiException class SparkSubmitHook(BaseHook, LoggingMixin): @@ -136,6 +135,10 @@ def __init__(self, self._connection = self._resolve_connection() self._is_yarn = 'yarn' in self._connection['master'] self._is_kubernetes = 'k8s' in self._connection['master'] + if self._is_kubernetes and kube_client is None: + raise RuntimeError( + "{master} specified by kubernetes dependencies are not installed!".format( + self._connection['master'])) self._should_track_driver_status = self._resolve_should_track_driver_status() self._driver_id = None @@ -559,6 +562,6 @@ def on_kill(self): self.log.info("Spark on K8s killed with response: %s", api_response) - except ApiException as e: + except kube_client.ApiException as e: self.log.info("Exception when attempting to kill Spark on K8s:") self.log.exception(e) diff --git a/airflow/contrib/kubernetes/kube_client.py b/airflow/contrib/kubernetes/kube_client.py index 8b71f41242329..4b8fa17155eed 100644 --- a/airflow/contrib/kubernetes/kube_client.py +++ b/airflow/contrib/kubernetes/kube_client.py @@ -17,9 +17,21 @@ from airflow.configuration import conf from six import PY2 +try: + from kubernetes import config, client + from kubernetes.client.rest import ApiException + has_kubernetes = True +except ImportError as e: + # We need an exception class to be able to use it in ``except`` elsewhere + # in the code base + ApiException = BaseException + has_kubernetes = False + _import_err = e + def _load_kube_config(in_cluster, cluster_context, config_file): - from kubernetes import config, client + if not has_kubernetes: + raise _import_err if in_cluster: config.load_incluster_config() else: From a6d5ee9ce57bcdd22dde3fdf00e3b1cd26274ede Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Wed, 8 Aug 2018 08:07:08 +0200 Subject: [PATCH 038/808] [AIRFLOW-2859] Implement own UtcDateTime (#3708) The different UtcDateTime implementations all have issues. Either they replace tzinfo directly without converting or they do not convert to UTC at all. We also ensure all mysql connections are in UTC in order to keep sanity, as mysql will ignore the timezone of a field when inserting/updating. (cherry picked from commit 6fd4e6055e36e9867923b0b402363fcd8c30e297) Signed-off-by: Bolke de Bruin (cherry picked from commit 8fc8c7ae5483c002f5264b087b26a20fd8ae7b67) Signed-off-by: Bolke de Bruin --- airflow/bin/cli.py | 1 - airflow/jobs.py | 6 +- airflow/models.py | 2 +- airflow/utils/sqlalchemy.py | 61 +++++++++++++++++- run_unit_tests.sh | 5 +- setup.py | 1 - tests/core.py | 10 +-- tests/test_utils/fake_datetime.py | 4 +- tests/utils/test_sqlalchemy.py | 101 ++++++++++++++++++++++++++++++ 9 files changed, 174 insertions(+), 17 deletions(-) create mode 100644 tests/utils/test_sqlalchemy.py diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index b56e3253276bc..ac52a42664caf 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -1003,7 +1003,6 @@ def initdb(args): # noqa print("Done.") -@cli_utils.action_logging def resetdb(args): print("DB: " + repr(settings.engine.url)) if args.yes or input("This will drop existing tables " diff --git a/airflow/jobs.py b/airflow/jobs.py index 70891ab4c370b..e2cf830aca296 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -40,7 +40,6 @@ Column, Integer, String, func, Index, or_, and_, not_) from sqlalchemy.exc import OperationalError from sqlalchemy.orm.session import make_transient -from sqlalchemy_utc import UtcDateTime from tabulate import tabulate from time import sleep @@ -52,6 +51,7 @@ from airflow.task.task_runner import get_task_runner from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, RUN_DEPS from airflow.utils import asciiart, helpers, timezone +from airflow.utils.configuration import tmp_configuration_copy from airflow.utils.dag_processing import (AbstractDagFileProcessor, DagFileProcessorManager, SimpleDag, @@ -60,9 +60,9 @@ from airflow.utils.db import create_session, provide_session from airflow.utils.email import send_email from airflow.utils.log.logging_mixin import LoggingMixin, set_context, StreamLogWriter -from airflow.utils.state import State -from airflow.utils.configuration import tmp_configuration_copy from airflow.utils.net import get_hostname +from airflow.utils.state import State +from airflow.utils.sqlalchemy import UtcDateTime Base = models.Base ID_LEN = models.ID_LEN diff --git a/airflow/models.py b/airflow/models.py index b789fd17323e2..2096785b41c57 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -60,7 +60,6 @@ from sqlalchemy import func, or_, and_, true as sqltrue from sqlalchemy.ext.declarative import declarative_base, declared_attr from sqlalchemy.orm import reconstructor, relationship, synonym -from sqlalchemy_utc import UtcDateTime from croniter import croniter import six @@ -88,6 +87,7 @@ as_tuple, is_container, validate_key, pprinttable) from airflow.utils.operator_resources import Resources from airflow.utils.state import State +from airflow.utils.sqlalchemy import UtcDateTime from airflow.utils.timeout import timeout from airflow.utils.trigger_rule import TriggerRule from airflow.utils.weight_rule import WeightRule diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 4dab322d4a106..46f4f86eface8 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -22,15 +22,19 @@ from __future__ import print_function from __future__ import unicode_literals +import datetime import os +import pendulum import time import random from sqlalchemy import event, exc, select +from sqlalchemy.types import DateTime, TypeDecorator from airflow.utils.log.logging_mixin import LoggingMixin log = LoggingMixin().log +utc = pendulum.timezone('UTC') def setup_event_handlers( @@ -101,13 +105,21 @@ def ping_connection(connection, branch): def connect(dbapi_connection, connection_record): connection_record.info['pid'] = os.getpid() - @event.listens_for(engine, "connect") - def set_sqlite_pragma(dbapi_connection, connection_record): - if 'sqlite3.Connection' in str(type(dbapi_connection)): + if engine.dialect.name == "sqlite": + @event.listens_for(engine, "connect") + def set_sqlite_pragma(dbapi_connection, connection_record): cursor = dbapi_connection.cursor() cursor.execute("PRAGMA foreign_keys=ON") cursor.close() + # this ensures sanity in mysql when storing datetimes (not required for postgres) + if engine.dialect.name == "mysql": + @event.listens_for(engine, "connect") + def set_mysql_timezone(dbapi_connection, connection_record): + cursor = dbapi_connection.cursor() + cursor.execute("SET time_zone = '+00:00'") + cursor.close() + @event.listens_for(engine, "checkout") def checkout(dbapi_connection, connection_record, connection_proxy): pid = os.getpid() @@ -117,3 +129,46 @@ def checkout(dbapi_connection, connection_record, connection_proxy): "Connection record belongs to pid {}, " "attempting to check out in pid {}".format(connection_record.info['pid'], pid) ) + + +class UtcDateTime(TypeDecorator): + """ + Almost equivalent to :class:`~sqlalchemy.types.DateTime` with + ``timezone=True`` option, but it differs from that by: + - Never silently take naive :class:`~datetime.datetime`, instead it + always raise :exc:`ValueError` unless time zone aware value. + - :class:`~datetime.datetime` value's :attr:`~datetime.datetime.tzinfo` + is always converted to UTC. + - Unlike SQLAlchemy's built-in :class:`~sqlalchemy.types.DateTime`, + it never return naive :class:`~datetime.datetime`, but time zone + aware value, even with SQLite or MySQL. + - Always returns DateTime in UTC + """ + + impl = DateTime(timezone=True) + + def process_bind_param(self, value, dialect): + if value is not None: + if not isinstance(value, datetime.datetime): + raise TypeError('expected datetime.datetime, not ' + + repr(value)) + elif value.tzinfo is None: + raise ValueError('naive datetime is disallowed') + + return value.astimezone(utc) + + def process_result_value(self, value, dialect): + """ + Processes DateTimes from the DB making sure it is always + returning UTC. Not using timezone.convert_to_utc as that + converts to configured TIMEZONE while the DB might be + running with some other setting. We assume UTC datetimes + in the database. + """ + if value is not None: + if value.tzinfo is None: + value = value.replace(tzinfo=utc) + else: + value = value.astimezone(utc) + + return value diff --git a/run_unit_tests.sh b/run_unit_tests.sh index 42c78f3439e7b..27e4d08af1416 100755 --- a/run_unit_tests.sh +++ b/run_unit_tests.sh @@ -8,9 +8,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -91,3 +91,4 @@ nosetests $nose_args # To run individual tests: # nosetests tests.core:CoreTest.test_scheduler_job + diff --git a/setup.py b/setup.py index 25b3c8c5ca77c..af8b1958f7dc8 100644 --- a/setup.py +++ b/setup.py @@ -298,7 +298,6 @@ def do_setup(): 'requests>=2.5.1, <3', 'setproctitle>=1.1.8, <2', 'sqlalchemy>=1.1.15, <1.2.0', - 'sqlalchemy-utc>=0.9.0', 'tabulate>=0.7.5, <0.8.0', 'tenacity==4.8.0', 'thrift>=0.9.2', diff --git a/tests/core.py b/tests/core.py index c3adb3c5005aa..0c178b4f715d3 100644 --- a/tests/core.py +++ b/tests/core.py @@ -38,7 +38,6 @@ from email.mime.application import MIMEApplication from email.mime.multipart import MIMEMultipart from email.mime.text import MIMEText -from freezegun import freeze_time from numpy.testing import assert_array_almost_equal from six.moves.urllib.parse import urlencode from time import sleep @@ -70,6 +69,7 @@ from airflow.configuration import AirflowConfigException, run_command from jinja2.sandbox import SecurityError from jinja2 import UndefinedError +from pendulum import utcnow import six @@ -261,7 +261,6 @@ def test_schedule_dag_start_end_dates(self): self.assertIsNone(additional_dag_run) - @freeze_time('2016-01-01') def test_schedule_dag_no_end_date_up_to_today_only(self): """ Tests that a Dag created without an end_date can only be scheduled up @@ -273,8 +272,11 @@ def test_schedule_dag_no_end_date_up_to_today_only(self): """ session = settings.Session() delta = timedelta(days=1) - start_date = DEFAULT_DATE - runs = 365 + now = utcnow() + start_date = now.subtract(weeks=1) + + runs = (now - start_date).days + dag = DAG(TEST_DAG_ID + 'test_schedule_dag_no_end_date_up_to_today_only', start_date=start_date, schedule_interval=delta) diff --git a/tests/test_utils/fake_datetime.py b/tests/test_utils/fake_datetime.py index 42bb01df01811..8182d83e9a086 100644 --- a/tests/test_utils/fake_datetime.py +++ b/tests/test_utils/fake_datetime.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/tests/utils/test_sqlalchemy.py b/tests/utils/test_sqlalchemy.py new file mode 100644 index 0000000000000..66f00f9427c6b --- /dev/null +++ b/tests/utils/test_sqlalchemy.py @@ -0,0 +1,101 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +import datetime +import unittest + +from airflow import settings +from airflow.models import DAG +from airflow.settings import Session +from airflow.utils.state import State +from airflow.utils.timezone import utcnow + +from sqlalchemy.exc import StatementError + + +class TestSqlAlchemyUtils(unittest.TestCase): + def setUp(self): + session = Session() + + # make sure NOT to run in UTC. Only postgres supports storing + # timezone information in the datetime field + if session.bind.dialect.name == "postgresql": + session.execute("SET timezone='Europe/Amsterdam'") + + self.session = session + + def test_utc_transformations(self): + """ + Test whether what we are storing is what we are retrieving + for datetimes + """ + dag_id = 'test_utc_transformations' + start_date = utcnow() + iso_date = start_date.isoformat() + execution_date = start_date + datetime.timedelta(hours=1, days=1) + + dag = DAG( + dag_id=dag_id, + start_date=start_date, + ) + dag.clear() + + run = dag.create_dagrun( + run_id=iso_date, + state=State.NONE, + execution_date=execution_date, + start_date=start_date, + session=self.session, + ) + + self.assertEqual(execution_date, run.execution_date) + self.assertEqual(start_date, run.start_date) + + self.assertEqual(execution_date.utcoffset().total_seconds(), 0.0) + self.assertEqual(start_date.utcoffset().total_seconds(), 0.0) + + self.assertEqual(iso_date, run.run_id) + self.assertEqual(run.start_date.isoformat(), run.run_id) + + dag.clear() + + def test_process_bind_param_naive(self): + """ + Check if naive datetimes are prevented from saving to the db + """ + dag_id = 'test_process_bind_param_naive' + + # naive + start_date = datetime.datetime.now() + dag = DAG(dag_id=dag_id, start_date=start_date) + dag.clear() + + with self.assertRaises((ValueError, StatementError)): + dag.create_dagrun( + run_id=start_date.isoformat, + state=State.NONE, + execution_date=start_date, + start_date=start_date, + session=self.session + ) + dag.clear() + + def tearDown(self): + self.session.close() + settings.engine.dispose() From 95dc7f04a866026c039786110078e1095b9eec70 Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Wed, 8 Aug 2018 19:12:34 +0200 Subject: [PATCH 039/808] [AIRFLOW-2870] Use abstract TaskInstance for migration (#3720) If we use the full model for migration it can have columns added that are not available yet in the database. Using an abstraction ensures only the columns that are required for data migration are present. (cherry picked from commit 546f1cdb5208ba8e1cf3bde36bbdbb639fa20b22) Signed-off-by: Bolke de Bruin (cherry picked from commit 95aa49a71dcc69d2e9a8e32b69a2a61cacec2b1b) Signed-off-by: Bolke de Bruin --- ...24_add_executor_config_to_task_instance.py | 24 ++++++++++++------- ...4_add_kubernetes_resource_checkpointing.py | 24 ++++++++++++------- ...5c0_add_kubernetes_scheduler_uniqueness.py | 24 ++++++++++++------- ...7_add_max_tries_column_to_task_instance.py | 20 +++++++++++++++- 4 files changed, 64 insertions(+), 28 deletions(-) diff --git a/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py b/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py index b7213a3031ddd..27a9f593b54df 100644 --- a/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py +++ b/airflow/migrations/versions/27c6a30d7c24_add_executor_config_to_task_instance.py @@ -1,16 +1,22 @@ # flake8: noqa # -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + """kubernetes_resource_checkpointing diff --git a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py index 4347bae92af1e..c489c05f7ea39 100644 --- a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py +++ b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py @@ -1,16 +1,22 @@ # flake8: noqa # -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + """kubernetes_resource_checkpointing diff --git a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py index 6bc48f1105639..5c921c6a98aaf 100644 --- a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py +++ b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py @@ -1,16 +1,22 @@ # flake8: noqa # -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + """add kubernetes scheduler uniqueness diff --git a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py index 0503d93df5bec..3c56f0707b57b 100644 --- a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py +++ b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py @@ -1,3 +1,4 @@ +# flake8: noqa # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -33,10 +34,27 @@ from alembic import op import sqlalchemy as sa from airflow import settings -from airflow.models import DagBag, TaskInstance +from airflow.models import DagBag +from airflow.utils.sqlalchemy import UtcDateTime + +from sqlalchemy import ( + Column, Integer, String) from sqlalchemy.engine.reflection import Inspector +from sqlalchemy.ext.declarative import declarative_base +Base = declarative_base() BATCH_SIZE = 5000 +ID_LEN = 250 + + +class TaskInstance(Base): + __tablename__ = "task_instance" + + task_id = Column(String(ID_LEN), primary_key=True) + dag_id = Column(String(ID_LEN), primary_key=True) + execution_date = Column(UtcDateTime, primary_key=True) + max_tries = Column(Integer) + try_number = Column(Integer, default=0) def upgrade(): From 55200b48bbc0bf86db72c2b7eab01109bb98102f Mon Sep 17 00:00:00 2001 From: Andy Cooper Date: Wed, 8 Aug 2018 07:49:43 -0400 Subject: [PATCH 040/808] [AIRFLOW-2856] Pass in SLUGIFY_USES_TEXT_UNIDECODE=yes ENV to docker run (#3701) (cherry picked from commit 8687ab9271b7b93473584a720f225f20fa9a7aa4) Signed-off-by: Bolke de Bruin (cherry picked from commit 3670d49a4c5da4769be1df5b4f8ef3504b74a184) Signed-off-by: Bolke de Bruin --- CONTRIBUTING.md | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e45060bfe75c0..21cbd5010c4ee 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -85,10 +85,7 @@ Go to your Airflow directory and start a new docker container. You can choose be ``` # Start docker in your Airflow directory -docker run -t -i -v `pwd`:/airflow/ python:2 bash - -# Go to the Airflow directory -cd /airflow/ +docker run -t -i -v `pwd`:/airflow/ -w /airflow/ -e SLUGIFY_USES_TEXT_UNIDECODE=yes python:2 bash # Install Airflow with all the required dependencies, # including the devel which will provide the development tools From 26e0d449737e8671000f671d820a9537f23f345a Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Wed, 8 Aug 2018 20:10:41 +0200 Subject: [PATCH 041/808] [AIRFLOW-XXX] Update changelog for 1.10 (cherry picked from commit 8b04e20709ebeb41aeefc0c5e3f12d35108ea504) Signed-off-by: Bolke de Bruin --- CHANGELOG.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index bfcef0f2d27d2..b4ee1755b467a 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -1,5 +1,10 @@ AIRFLOW 1.10.0, 2018-08-03 -------------------------- +[AIRFLOW-2870] Use abstract TaskInstance for migration +[AIRFLOW-2859] Implement own UtcDateTime (#3708) +[AIRFLOW-2140] Don't require kubernetes for the SparkSubmit hook +[AIRFLOW-2869] Remove smart quote from default config +[AIRFLOW-2857] Fix Read the Docs env [AIRFLOW-2817] Force explicit choice on GPL dependency [AIRFLOW-2716] Replace async and await py3.7 keywords [AIRFLOW-2810] Fix typo in Xcom model timestamp From 948b09da20be7e027bb315618fa0901ef8b48b80 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Wed, 22 Aug 2018 17:06:09 +0800 Subject: [PATCH 042/808] [AIRFLOW-2921][AIRFLOW-2922] Fix bugs in CeleryExecutor (#3773) Bug-1: if a task state becomes either SUCCESS or FAILURE or REVOKED, it will be removed from self.tasks() and self.last_state(). However, because line 108 is not indented properly, this task will be added back to self.last_state() again. Bug-2: When the state is updated, it's referring to the latest state `task.state` rather than variable `state`. This may result in dead-lock if the state changed from `STARTED` to `SUCCESS` after the if-elif-else block started. Test case is updated for fix to bug-1. --- airflow/executors/celery_executor.py | 4 ++-- tests/executors/test_celery_executor.py | 6 +++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py index 481daa5826686..6ada005ab77d2 100644 --- a/airflow/executors/celery_executor.py +++ b/airflow/executors/celery_executor.py @@ -106,8 +106,8 @@ def sync(self): del self.tasks[key] del self.last_state[key] else: - self.log.info("Unexpected state: %s", task.state) - self.last_state[key] = task.state + self.log.info("Unexpected state: %s", state) + self.last_state[key] = state except Exception as e: self.log.error("Error syncing the celery executor, ignoring it:") self.log.exception(e) diff --git a/tests/executors/test_celery_executor.py b/tests/executors/test_celery_executor.py index 1169c8d11b95c..d5fbbbcf1c78c 100644 --- a/tests/executors/test_celery_executor.py +++ b/tests/executors/test_celery_executor.py @@ -25,7 +25,8 @@ from airflow.utils.state import State # leave this it is used by the test worker -import celery.contrib.testing.tasks +import celery.contrib.testing.tasks # noqa: F401 + class CeleryExecutorTest(unittest.TestCase): def test_celery_integration(self): @@ -53,6 +54,9 @@ def test_celery_integration(self): self.assertNotIn('success', executor.tasks) self.assertNotIn('fail', executor.tasks) + self.assertNotIn('success', executor.last_state) + self.assertNotIn('fail', executor.last_state) + if __name__ == '__main__': unittest.main() From 4040ce2bf4d2d51ca226c9b7b169570b224f0990 Mon Sep 17 00:00:00 2001 From: Dan Davydov Date: Mon, 20 Aug 2018 09:14:22 -0400 Subject: [PATCH 043/808] [AIRFLOW-2895] Prevent scheduler from spamming heartbeats/logs Reverts most of AIRFLOW-2027 until the issues with it can be fixed. Closes #3747 from aoen/revert_min_file_parsing_time_commit --- README.md | 1 + UPDATING.md | 6 +++ airflow/config_templates/default_airflow.cfg | 4 +- airflow/jobs.py | 42 +++++++++++--------- airflow/utils/dag_processing.py | 25 ++---------- scripts/ci/kubernetes/kube/configmaps.yaml | 3 -- tests/utils/test_dag_processing.py | 2 - 7 files changed, 35 insertions(+), 48 deletions(-) diff --git a/README.md b/README.md index e68d26cc8cfa2..f5fe2e1bdcd4c 100644 --- a/README.md +++ b/README.md @@ -233,6 +233,7 @@ Currently **officially** using Airflow: 1. [Tile](https://tile.com/) [[@ranjanmanish](https://github.com/ranjanmanish)] 1. [Tokopedia](https://www.tokopedia.com/) [@topedmaria](https://github.com/topedmaria) 1. [Twine Labs](https://www.twinelabs.com/) [[@ivorpeles](https://github.com/ivorpeles)] +1. [Twitter](https://www.twitter.com/) [[@aoen](https://github.com/aoen)] 1. [T2 Systems](http://t2systems.com) [[@unclaimedpants](https://github.com/unclaimedpants)] 1. [Ubisoft](https://www.ubisoft.com/) [[@Walkoss](https://github.com/Walkoss)] 1. [United Airlines](https://www.united.com/) [[@ilopezfr](https://github.com/ilopezfr)] diff --git a/UPDATING.md b/UPDATING.md index 3b47882212225..d00e4f8742f26 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -5,6 +5,11 @@ assists users migrating to a new version. ## Airflow Master +### min_file_parsing_loop_time config option temporarily disabled + +The scheduler.min_file_parsing_loop_time config option has been temporarily removed due to +some bugs. + ## Airflow 1.10 Installation and upgrading requires setting `SLUGIFY_USES_TEXT_UNIDECODE=yes` in your environment or @@ -386,6 +391,7 @@ indefinitely. This is only available on the command line. After how much time should an updated DAG be picked up from the filesystem. #### min_file_parsing_loop_time +CURRENTLY DISABLED DUE TO A BUG How many seconds to wait between file-parsing loops to prevent the logs from being spammed. #### dag_dir_list_interval diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 9a15dc0d117e8..deb67bb96ca25 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -428,9 +428,7 @@ run_duration = -1 # after how much time a new DAGs should be picked up from the filesystem min_file_process_interval = 0 -# How many seconds to wait between file-parsing loops to prevent the logs from being spammed. -min_file_parsing_loop_time = 1 - +# How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes. dag_dir_list_interval = 300 # How often should stats be printed to the logs diff --git a/airflow/jobs.py b/airflow/jobs.py index e2cf830aca296..88aa643c50152 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -532,8 +532,7 @@ def __init__( num_runs=-1, file_process_interval=conf.getint('scheduler', 'min_file_process_interval'), - min_file_parsing_loop_time=conf.getint('scheduler', - 'min_file_parsing_loop_time'), + processor_poll_interval=1.0, run_duration=None, do_pickle=False, log=None, @@ -548,6 +547,8 @@ def __init__( :type subdir: unicode :param num_runs: The number of times to try to schedule each DAG file. -1 for unlimited within the run_duration. + :param processor_poll_interval: The number of seconds to wait between + polls of running processors :param run_duration: how long to run (in seconds) before exiting :type run_duration: int :param do_pickle: once a DAG object is obtained by executing the Python @@ -564,6 +565,7 @@ def __init__( self.num_runs = num_runs self.run_duration = run_duration + self._processor_poll_interval = processor_poll_interval self.do_pickle = do_pickle super(SchedulerJob, self).__init__(*args, **kwargs) @@ -592,10 +594,7 @@ def __init__( # to 3 minutes. self.file_process_interval = file_process_interval - # Wait until at least this many seconds have passed before parsing files once all - # files have finished parsing. - self.min_file_parsing_loop_time = min_file_parsing_loop_time - + self.max_tis_per_query = conf.getint('scheduler', 'max_tis_per_query') if run_duration is None: self.run_duration = conf.getint('scheduler', 'run_duration') @@ -1550,16 +1549,18 @@ def _execute(self): # DAGs in parallel. By processing them in separate processes, # we can get parallelism and isolation from potentially harmful # user code. - self.log.info("Processing files using up to %s processes at a time", - self.max_threads) + self.log.info( + "Processing files using up to %s processes at a time", + self.max_threads) self.log.info("Running execute loop for %s seconds", self.run_duration) self.log.info("Processing each file at most %s times", self.num_runs) - self.log.info("Process each file at most once every %s seconds", - self.file_process_interval) - self.log.info("Wait until at least %s seconds have passed between file parsing " - "loops", self.min_file_parsing_loop_time) - self.log.info("Checking for new files in %s every %s seconds", - self.subdir, self.dag_dir_list_interval) + self.log.info( + "Process each file at most once every %s seconds", + self.file_process_interval) + self.log.info( + "Checking for new files in %s every %s seconds", + self.subdir, + self.dag_dir_list_interval) # Build up a list of Python files that could contain DAGs self.log.info("Searching for files in %s", self.subdir) @@ -1575,7 +1576,6 @@ def processor_factory(file_path): known_file_paths, self.max_threads, self.file_process_interval, - self.min_file_parsing_loop_time, self.num_runs, processor_factory) @@ -1722,13 +1722,17 @@ def _execute_helper(self, processor_manager): last_stat_print_time = timezone.utcnow() loop_end_time = time.time() - self.log.debug("Ran scheduling loop in %.2f seconds", - loop_end_time - loop_start_time) + self.log.debug( + "Ran scheduling loop in %.2f seconds", + loop_end_time - loop_start_time) + self.log.debug("Sleeping for %.2f seconds", self._processor_poll_interval) + time.sleep(self._processor_poll_interval) # Exit early for a test mode if processor_manager.max_runs_reached(): - self.log.info("Exiting loop as all files have been processed %s times", - self.num_runs) + self.log.info( + "Exiting loop as all files have been processed %s times", + self.num_runs) break # Stop any processors diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index e236397da0284..43948837065e3 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -326,7 +326,6 @@ def __init__(self, file_paths, parallelism, process_file_interval, - min_file_parsing_loop_time, max_runs, processor_factory): """ @@ -340,9 +339,6 @@ def __init__(self, :param process_file_interval: process a file at most once every this many seconds :type process_file_interval: float - :param min_file_parsing_loop_time: wait until at least this many seconds have - passed before parsing files once all files have finished parsing. - :type min_file_parsing_loop_time: float :param max_runs: The number of times to parse and schedule each file. -1 for unlimited. :type max_runs: int @@ -358,7 +354,6 @@ def __init__(self, self._dag_directory = dag_directory self._max_runs = max_runs self._process_file_interval = process_file_interval - self._min_file_parsing_loop_time = min_file_parsing_loop_time self._processor_factory = processor_factory # Map from file path to the processor self._processors = {} @@ -529,24 +524,12 @@ def heartbeat(self): file_paths_in_progress = self._processors.keys() now = timezone.utcnow() file_paths_recently_processed = [] - - longest_parse_duration = 0 for file_path in self._file_paths: last_finish_time = self.get_last_finish_time(file_path) - if last_finish_time is not None: - duration = now - last_finish_time - longest_parse_duration = max(duration.total_seconds(), - longest_parse_duration) - if duration.total_seconds() < self._process_file_interval: - file_paths_recently_processed.append(file_path) - - sleep_length = max(self._min_file_parsing_loop_time - longest_parse_duration, - 0) - if sleep_length > 0: - self.log.debug("Sleeping for %.2f seconds to prevent excessive " - "logging", - sleep_length) - time.sleep(sleep_length) + if (last_finish_time is not None and + (now - last_finish_time).total_seconds() < + self._process_file_interval): + file_paths_recently_processed.append(file_path) files_paths_at_run_limit = [file_path for file_path, num_runs in self._run_count.items() diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index 7b91aa2e8724c..1673c3e54b577 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -57,9 +57,6 @@ data: statsd_port = 8125 statsd_prefix = airflow - # How many seconds to wait between file-parsing loops to prevent the logs from being spammed. - min_file_parsing_loop_time = 1 - print_stats_interval = 30 scheduler_zombie_task_threshold = 300 max_tis_per_query = 0 diff --git a/tests/utils/test_dag_processing.py b/tests/utils/test_dag_processing.py index 7abe7efe9b35c..f29e384b8c657 100644 --- a/tests/utils/test_dag_processing.py +++ b/tests/utils/test_dag_processing.py @@ -32,7 +32,6 @@ def test_set_file_paths_when_processor_file_path_not_in_new_file_paths(self): parallelism=1, process_file_interval=1, max_runs=1, - min_file_parsing_loop_time=0, processor_factory=MagicMock().return_value) mock_processor = MagicMock() @@ -52,7 +51,6 @@ def test_set_file_paths_when_processor_file_path_is_in_new_file_paths(self): parallelism=1, process_file_interval=1, max_runs=1, - min_file_parsing_loop_time=0, processor_factory=MagicMock().return_value) mock_processor = MagicMock() From 392ad4af83dccc0176e5ad12764d287227b9c48b Mon Sep 17 00:00:00 2001 From: Diego Mora Cespedes Date: Wed, 4 Jul 2018 23:51:29 +0100 Subject: [PATCH 044/808] [AIRFLOW-2713] Rename async variable in setup.py for Python 3.7.0 compatibility Closes #3561 from Perados/rename-async-to- async_packages-in-setup --- setup.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index af8b1958f7dc8..b48087084def9 100644 --- a/setup.py +++ b/setup.py @@ -121,7 +121,8 @@ def write_version(filename=os.path.join(*['airflow', with open(filename, 'w') as a: a.write(text) -async = [ + +async_packages = [ 'greenlet>=0.4.9', 'eventlet>= 0.9.7', 'gevent>=0.13' @@ -314,7 +315,7 @@ def do_setup(): 'devel_ci': devel_ci, 'all_dbs': all_dbs, 'atlas': atlas, - 'async': async, + 'async': async_packages, 'azure_blob_storage': azure_blob_storage, 'azure_data_lake': azure_data_lake, 'cassandra': cassandra, From 152214ea6dd08533aa40e455415c2741259482e9 Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Thu, 30 Aug 2018 14:26:11 +0200 Subject: [PATCH 045/808] [AIRFLOW-2984] Convert operator dates to UTC (#3822) Tasks can have start_dates or end_dates separately from the DAG. These need to be converted to UTC otherwise we cannot use them for calculation the next execution date. --- airflow/models.py | 7 +++++ docs/timezone.rst | 67 +++++++++++++++++++++++++---------------------- tests/core.py | 2 +- tests/models.py | 10 +++++++ 4 files changed, 53 insertions(+), 33 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 2096785b41c57..7437a19521b5f 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -2387,10 +2387,17 @@ def __init__( self.email = email self.email_on_retry = email_on_retry self.email_on_failure = email_on_failure + self.start_date = start_date if start_date and not isinstance(start_date, datetime): self.log.warning("start_date for %s isn't datetime.datetime", self) + elif start_date: + self.start_date = timezone.convert_to_utc(start_date) + self.end_date = end_date + if end_date: + self.end_date = timezone.convert_to_utc(end_date) + if not TriggerRule.is_valid(trigger_rule): raise AirflowException( "The trigger_rule must be one of {all_triggers}," diff --git a/docs/timezone.rst b/docs/timezone.rst index 9e8598e2ed464..fe44ecfbb9f62 100644 --- a/docs/timezone.rst +++ b/docs/timezone.rst @@ -2,23 +2,23 @@ Time zones ========== Support for time zones is enabled by default. Airflow stores datetime information in UTC internally and in the database. -It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the -end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators +It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the +end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators are not converted. Time zone information is exposed and it is up to the writer of DAG what do with it. -This is handy if your users live in more than one time zone and you want to display datetime information according to +This is handy if your users live in more than one time zone and you want to display datetime information according to each user’s wall clock. -Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database -(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is -Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward -in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions -happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter -for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day -deadlines to meet. +Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database +(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is +Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward +in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions +happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter +for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day +deadlines to meet. -The time zone is set in `airflow.cfg`. By default it is set to utc, but you change it to use the system’s settings or -an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`. +The time zone is set in `airflow.cfg`. By default it is set to utc, but you change it to use the system’s settings or +an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`. Pendulum is installed when you install Airflow. Please note that the Web UI currently only runs in UTC. @@ -28,8 +28,8 @@ Concepts Naïve and aware datetime objects '''''''''''''''''''''''''''''''' -Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information, -represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset, +Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information, +represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset, a datetime object is aware. Otherwise, it’s naive. You can use timezone.is_aware() and timezone.is_naive() to determine whether datetimes are aware or naive. @@ -39,7 +39,7 @@ Because Airflow uses time-zone-aware datetime objects. If your code creates date .. code:: python from airflow.utils import timezone - + now = timezone.utcnow() a_date = timezone.datetime(2017,1,1) @@ -49,9 +49,9 @@ Interpretation of naive datetime objects Although Airflow operates fully time zone aware, it still accepts naive date time objects for `start_dates` and `end_dates` in your DAG definitions. This is mostly in order to preserve backwards compatibility. In -case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied +case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied in such a way that it is assumed that the naive date time is already in the default time zone. In other -words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of +words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of `datetime(2017,1,1)` it is assumed to be a `start_date` of Jan 1, 2017 Amsterdam time. .. code:: python @@ -65,16 +65,16 @@ words if you have a default time zone setting of `Europe/Amsterdam` and create a op = DummyOperator(task_id='dummy', dag=dag) print(op.owner) # Airflow -Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous. -In such situations, pendulum raises an exception. That’s why you should always create aware +Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous. +In such situations, pendulum raises an exception. That’s why you should always create aware datetime objects when time zone support is enabled. -In practice, this is rarely an issue. Airflow gives you aware datetime objects in the models and DAGs, and most often, -new datetime objects are created from existing ones through timedelta arithmetic. The only datetime that’s often +In practice, this is rarely an issue. Airflow gives you aware datetime objects in the models and DAGs, and most often, +new datetime objects are created from existing ones through timedelta arithmetic. The only datetime that’s often created in application code is the current time, and timezone.utcnow() automatically does the right thing. -Default time zone +Default time zone ''''''''''''''''' The default time zone is the time zone defined by the `default_timezone` setting under `[core]`. If @@ -92,15 +92,15 @@ it is therefore important to make sure this setting is equal on all Airflow node Time zone aware DAGs -------------------- -Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is +Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is recommended to use `pendulum` for this, but `pytz` (to be installed manually) can also be used for this. .. code:: python import pendulum - + local_tz = pendulum.timezone("Europe/Amsterdam") - + default_args=dict( start_date=datetime(2016, 1, 1, tzinfo=local_tz), owner='Airflow' @@ -110,18 +110,21 @@ recommended to use `pendulum` for this, but `pytz` (to be installed manually) ca op = DummyOperator(task_id='dummy', dag=dag) print(dag.timezone) # - +Please note that while it is possible to set a `start_date` and `end_date` for Tasks always the DAG timezone +or global timezone (in that order) will be used to calculate the next execution date. Upon first encounter +the start date or end date will be converted to UTC using the timezone associated with start_date or end_date, +then for calculations this timezone information will be disregarded. Templates ''''''''' -Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC. +Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC. It is left up to the DAG to handle this. .. code:: python import pendulum - + local_tz = pendulum.timezone("Europe/Amsterdam") local_tz.convert(execution_date) @@ -129,10 +132,10 @@ It is left up to the DAG to handle this. Cron schedules '''''''''''''' -In case you set a cron schedule, Airflow assumes you will always want to run at the exact same time. It will -then ignore day light savings time. Thus, if you have a schedule that says -run at end of interval every day at 08:00 GMT+1 it will always run end of interval 08:00 GMT+1, -regardless if day light savings time is in place. +In case you set a cron schedule, Airflow assumes you will always want to run at the exact same time. It will +then ignore day light savings time. Thus, if you have a schedule that says +run at end of interval every day at 08:00 GMT+1 it will always run end of interval 08:00 GMT+1, +regardless if day light savings time is in place. Time deltas diff --git a/tests/core.py b/tests/core.py index 0c178b4f715d3..db17008c6cc51 100644 --- a/tests/core.py +++ b/tests/core.py @@ -46,7 +46,7 @@ from airflow.executors import SequentialExecutor from airflow.models import Variable -configuration.conf.load_test_config() +configuration.conf.load_test_config() # NOQA: E402 from airflow import jobs, models, DAG, utils, macros, settings, exceptions from airflow.models import BaseOperator from airflow.operators.bash_operator import BashOperator diff --git a/tests/models.py b/tests/models.py index d38681741daa1..a8cc005ba3d8a 100644 --- a/tests/models.py +++ b/tests/models.py @@ -1499,6 +1499,16 @@ def test_timezone_awareness(self): ti = TI(task=op1, execution_date=execution_date) self.assertEquals(ti.execution_date, utc_date) + def test_task_naive_datetime(self): + NAIVE_DATETIME = DEFAULT_DATE.replace(tzinfo=None) + + op_no_dag = DummyOperator(task_id='test_task_naive_datetime', + start_date=NAIVE_DATETIME, + end_date=NAIVE_DATETIME) + + self.assertTrue(op_no_dag.start_date.tzinfo) + self.assertTrue(op_no_dag.end_date.tzinfo) + def test_set_dag(self): """ Test assigning Operators to Dags, including deferred assignment From 6b62af7e373c5519cbeeca144365afa21856fc0f Mon Sep 17 00:00:00 2001 From: abdul-stripe <34078811+abdul-stripe@users.noreply.github.com> Date: Fri, 31 Aug 2018 16:52:02 -0700 Subject: [PATCH 046/808] [AIRFLOW-2145] fix deadlock on clearing running TI (#3657) a `shutdown` task is not considered be `unfinished`, so a dag run can deadlock when all `unfinished` downstreams are all waiting on a task that's in the `shutdown` state. fix this by considering `shutdown` to be `unfinished`, since it's not truly a terminal state --- airflow/utils/state.py | 2 +- tests/models.py | 21 ++++++++++++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/airflow/utils/state.py b/airflow/utils/state.py index 9da98510eb03d..70dbc5ca048e8 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -101,7 +101,6 @@ def finished(cls): """ return [ cls.SUCCESS, - cls.SHUTDOWN, cls.FAILED, cls.SKIPPED, ] @@ -117,5 +116,6 @@ def unfinished(cls): cls.SCHEDULED, cls.QUEUED, cls.RUNNING, + cls.SHUTDOWN, cls.UP_FOR_RETRY ] diff --git a/tests/models.py b/tests/models.py index a8cc005ba3d8a..60aee3c84fcb1 100644 --- a/tests/models.py +++ b/tests/models.py @@ -801,7 +801,26 @@ def test_dagrun_deadlock(self): dr.update_state() self.assertEqual(dr.state, State.FAILED) - def test_dagrun_no_deadlock(self): + def test_dagrun_no_deadlock_with_shutdown(self): + session = settings.Session() + dag = DAG('test_dagrun_no_deadlock_with_shutdown', + start_date=DEFAULT_DATE) + with dag: + op1 = DummyOperator(task_id='upstream_task') + op2 = DummyOperator(task_id='downstream_task') + op2.set_upstream(op1) + + dr = dag.create_dagrun(run_id='test_dagrun_no_deadlock_with_shutdown', + state=State.RUNNING, + execution_date=DEFAULT_DATE, + start_date=DEFAULT_DATE) + upstream_ti = dr.get_task_instance(task_id='upstream_task') + upstream_ti.set_state(State.SHUTDOWN, session=session) + + dr.update_state() + self.assertEqual(dr.state, State.RUNNING) + + def test_dagrun_no_deadlock_with_depends_on_past(self): session = settings.Session() dag = DAG('test_dagrun_no_deadlock', start_date=DEFAULT_DATE) From da183afc2814ecf99b494301487e06fa4f222a80 Mon Sep 17 00:00:00 2001 From: tzulberti-jampp Date: Wed, 29 Aug 2018 13:27:52 -0300 Subject: [PATCH 047/808] [AIRFLOW-2949] Add syntax highlight for single quote strings (#3795) * AIRFLOW-2949: Add syntax highlight for single quote strings * AIRFLOW-2949: Also updated new UI main.css --- airflow/www/static/main.css | 1 + airflow/www_rbac/static/main.css | 1 + 2 files changed, 2 insertions(+) diff --git a/airflow/www/static/main.css b/airflow/www/static/main.css index 57164b94e5ccf..147695c4a9591 100644 --- a/airflow/www/static/main.css +++ b/airflow/www/static/main.css @@ -262,3 +262,4 @@ div.square { .sc { color: #BA2121 } /* Literal.String.Char */ .sd { color: #BA2121; font-style: italic } /* Literal.String.Doc */ .s2 { color: #BA2121 } /* Literal.String.Double */ +.s1 { color: #BA2121 } /* Literal.String.Single */ diff --git a/airflow/www_rbac/static/main.css b/airflow/www_rbac/static/main.css index 5a10bfd34dc62..ab39d44d9e1a7 100644 --- a/airflow/www_rbac/static/main.css +++ b/airflow/www_rbac/static/main.css @@ -265,3 +265,4 @@ div.square { .sc { color: #BA2121 } /* Literal.String.Char */ .sd { color: #BA2121; font-style: italic } /* Literal.String.Doc */ .s2 { color: #BA2121 } /* Literal.String.Double */ +.s1 { color: #BA2121 } /* Literal.String.Single */ From f04872ed0f8170da48385882f5e04b3c27c46134 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 3 Sep 2018 12:17:51 +0100 Subject: [PATCH 048/808] [AIRFLOW-2979] Make celery_result_backend conf Backwards compatible (#3832) (#2806) Renamed `celery_result_backend` to `result_backend` and broke backwards compatibility. --- airflow/configuration.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/configuration.py b/airflow/configuration.py index 2e05fde0cdea2..4f88a6fee3c43 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -138,6 +138,7 @@ class AirflowConfigParser(ConfigParser): 'celery': { # Remove these keys in Airflow 1.11 'worker_concurrency': 'celeryd_concurrency', + 'result_backend': 'celery_result_backend', 'broker_url': 'celery_broker_url', 'ssl_active': 'celery_ssl_active', 'ssl_cert': 'celery_ssl_cert', From 2b596f0c136f8cda7f86f1bcd5271a488259f0b2 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 26 Aug 2018 13:57:26 +0200 Subject: [PATCH 049/808] [AIRFLOW-2960] Pin boto3 to <1.8 (#3810) Boto 1.8 has been released a few days ago and they break our tests. --- setup.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index b48087084def9..03edaaced6cd4 100644 --- a/setup.py +++ b/setup.py @@ -163,7 +163,7 @@ def write_version(filename=os.path.join(*['airflow', 'elasticsearch>=5.0.0,<6.0.0', 'elasticsearch-dsl>=5.0.0,<6.0.0' ] -emr = ['boto3>=1.0.0'] +emr = ['boto3>=1.0.0, <1.8.0'] gcp_api = [ 'httplib2>=0.9.2', 'google-api-python-client>=1.6.0, <2.0.0dev', @@ -201,7 +201,7 @@ def write_version(filename=os.path.join(*['airflow', qds = ['qds-sdk>=1.9.6'] rabbitmq = ['librabbitmq>=1.6.1'] redis = ['redis>=2.10.5'] -s3 = ['boto3>=1.7.0'] +s3 = ['boto3>=1.7.0, <1.8.0'] salesforce = ['simple-salesforce>=0.72'] samba = ['pysmbclient>=0.1.3'] segment = ['analytics-python>=1.2.9'] From ad951705c4f1e70eecb04a2e018759d882e3fb49 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 3 Sep 2018 00:19:51 +0100 Subject: [PATCH 050/808] [AIRFLOW-2476] Allow tabulate up to 0.8.2 (#3835) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 03edaaced6cd4..481c94c8f27e4 100644 --- a/setup.py +++ b/setup.py @@ -299,7 +299,7 @@ def do_setup(): 'requests>=2.5.1, <3', 'setproctitle>=1.1.8, <2', 'sqlalchemy>=1.1.15, <1.2.0', - 'tabulate>=0.7.5, <0.8.0', + 'tabulate>=0.7.5, <=0.8.2', 'tenacity==4.8.0', 'thrift>=0.9.2', 'tzlocal>=1.4', From 1c19190cae1eacff9e9489ef31dfe5ea5b637775 Mon Sep 17 00:00:00 2001 From: Jake Biesinger Date: Thu, 30 Aug 2018 13:12:46 -0700 Subject: [PATCH 051/808] [AIRFLOW-2900] Show code for packaged DAGs (#3749) --- airflow/models.py | 5 +++-- airflow/www/utils.py | 25 ++++++++++++++++++++++--- airflow/www/views.py | 2 +- airflow/www_rbac/utils.py | 20 ++++++++++++++++++++ airflow/www_rbac/views.py | 2 +- tests/www/test_utils.py | 34 ++++++++++++++++++++++++++++++++++ tests/www_rbac/test_utils.py | 35 +++++++++++++++++++++++++++++++++++ 7 files changed, 116 insertions(+), 7 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 7437a19521b5f..f586139bb1145 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -335,7 +335,8 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): return found_dags mods = [] - if not zipfile.is_zipfile(filepath): + is_zipfile = zipfile.is_zipfile(filepath) + if not is_zipfile: if safe_mode and os.path.isfile(filepath): with open(filepath, 'rb') as f: content = f.read() @@ -407,7 +408,7 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): if isinstance(dag, DAG): if not dag.full_filepath: dag.full_filepath = filepath - if dag.fileloc != filepath: + if dag.fileloc != filepath and not is_zipfile: dag.fileloc = filepath try: dag.is_subdag = False diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 44fa5c4dcd6fb..09ac465c52d1b 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -18,16 +18,19 @@ # under the License. # from future import standard_library -standard_library.install_aliases() -from builtins import str -from builtins import object +standard_library.install_aliases() # noqa: E402 +from builtins import str, object from cgi import escape from io import BytesIO as IO import functools import gzip +import io import json +import os +import re import time +import zipfile from flask import after_this_request, request, Response from flask_admin.contrib.sqla.filters import FilterConverter @@ -366,6 +369,22 @@ def zipper(response): return view_func +def open_maybe_zipped(f, mode='r'): + """ + Opens the given file. If the path contains a folder with a .zip suffix, then + the folder is treated as a zip archive, opening the file inside the archive. + + :return: a file object, as in `open`, or as in `ZipFile.open`. + """ + + _, archive, filename = re.search( + r'((.*\.zip){})?(.*)'.format(re.escape(os.sep)), f).groups() + if archive and zipfile.is_zipfile(archive): + return zipfile.ZipFile(archive, mode=mode).open(filename) + else: + return io.open(f, mode=mode) + + def make_cache_key(*args, **kwargs): """ Used by cache to get a unique key per URL diff --git a/airflow/www/views.py b/airflow/www/views.py index 8f6725ef59b44..851ad90cb2770 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -670,7 +670,7 @@ def code(self): dag = dagbag.get_dag(dag_id) title = dag_id try: - with open(dag.fileloc, 'r') as f: + with wwwutils.open_maybe_zipped(dag.fileloc, 'r') as f: code = f.read() html_code = highlight( code, lexers.PythonLexer(), HtmlFormatter(linenos=True)) diff --git a/airflow/www_rbac/utils.py b/airflow/www_rbac/utils.py index 7bbdada5556f2..14e18f8bc30e7 100644 --- a/airflow/www_rbac/utils.py +++ b/airflow/www_rbac/utils.py @@ -26,6 +26,10 @@ import wtforms import bleach import markdown +import re +import zipfile +import os +import io from builtins import str from past.builtins import basestring @@ -197,6 +201,22 @@ def json_response(obj): mimetype="application/json") +def open_maybe_zipped(f, mode='r'): + """ + Opens the given file. If the path contains a folder with a .zip suffix, then + the folder is treated as a zip archive, opening the file inside the archive. + + :return: a file object, as in `open`, or as in `ZipFile.open`. + """ + + _, archive, filename = re.search( + r'((.*\.zip){})?(.*)'.format(re.escape(os.sep)), f).groups() + if archive and zipfile.is_zipfile(archive): + return zipfile.ZipFile(archive, mode=mode).open(filename) + else: + return io.open(f, mode=mode) + + def make_cache_key(*args, **kwargs): """ Used by cache to get a unique key per URL diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 43e481ed0278c..a65e1d26e3995 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -379,7 +379,7 @@ def code(self): dag = dagbag.get_dag(dag_id) title = dag_id try: - with open(dag.fileloc, 'r') as f: + with wwwutils.open_maybe_zipped(dag.fileloc, 'r') as f: code = f.read() html_code = highlight( code, lexers.PythonLexer(), HtmlFormatter(linenos=True)) diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py index 9d788e88f1ed9..f5c3f01fb916d 100644 --- a/tests/www/test_utils.py +++ b/tests/www/test_utils.py @@ -190,6 +190,40 @@ def some_func(): self.assertEqual(anonymous_username, kwargs['owner']) mocked_session_instance.add.assert_called_once() + def test_open_maybe_zipped_normal_file(self): + with mock.patch( + 'io.open', mock.mock_open(read_data="data")) as mock_file: + utils.open_maybe_zipped('/path/to/some/file.txt') + mock_file.assert_called_with('/path/to/some/file.txt', mode='r') + + def test_open_maybe_zipped_normal_file_with_zip_in_name(self): + path = '/path/to/fakearchive.zip.other/file.txt' + with mock.patch( + 'io.open', mock.mock_open(read_data="data")) as mock_file: + utils.open_maybe_zipped(path) + mock_file.assert_called_with(path, mode='r') + + @mock.patch("zipfile.is_zipfile") + @mock.patch("zipfile.ZipFile") + def test_open_maybe_zipped_archive(self, mocked_ZipFile, mocked_is_zipfile): + mocked_is_zipfile.return_value = True + instance = mocked_ZipFile.return_value + instance.open.return_value = mock.mock_open(read_data="data") + + utils.open_maybe_zipped('/path/to/archive.zip/deep/path/to/file.txt') + + mocked_is_zipfile.assert_called_once() + (args, kwargs) = mocked_is_zipfile.call_args_list[0] + self.assertEqual('/path/to/archive.zip', args[0]) + + mocked_ZipFile.assert_called_once() + (args, kwargs) = mocked_ZipFile.call_args_list[0] + self.assertEqual('/path/to/archive.zip', args[0]) + + instance.open.assert_called_once() + (args, kwargs) = instance.open.call_args_list[0] + self.assertEqual('deep/path/to/file.txt', args[0]) + if __name__ == '__main__': unittest.main() diff --git a/tests/www_rbac/test_utils.py b/tests/www_rbac/test_utils.py index 05114881ddecf..c7179fa3a421d 100644 --- a/tests/www_rbac/test_utils.py +++ b/tests/www_rbac/test_utils.py @@ -18,6 +18,7 @@ # under the License. import unittest +import mock from xml.dom import minidom from airflow.www_rbac import utils @@ -109,6 +110,40 @@ def test_params_all(self): self.assertEqual('page=3&search=bash_&showPaused=False', utils.get_params(showPaused=False, page=3, search='bash_')) + def test_open_maybe_zipped_normal_file(self): + with mock.patch( + 'io.open', mock.mock_open(read_data="data")) as mock_file: + utils.open_maybe_zipped('/path/to/some/file.txt') + mock_file.assert_called_with('/path/to/some/file.txt', mode='r') + + def test_open_maybe_zipped_normal_file_with_zip_in_name(self): + path = '/path/to/fakearchive.zip.other/file.txt' + with mock.patch( + 'io.open', mock.mock_open(read_data="data")) as mock_file: + utils.open_maybe_zipped(path) + mock_file.assert_called_with(path, mode='r') + + @mock.patch("zipfile.is_zipfile") + @mock.patch("zipfile.ZipFile") + def test_open_maybe_zipped_archive(self, mocked_ZipFile, mocked_is_zipfile): + mocked_is_zipfile.return_value = True + instance = mocked_ZipFile.return_value + instance.open.return_value = mock.mock_open(read_data="data") + + utils.open_maybe_zipped('/path/to/archive.zip/deep/path/to/file.txt') + + mocked_is_zipfile.assert_called_once() + (args, kwargs) = mocked_is_zipfile.call_args_list[0] + self.assertEqual('/path/to/archive.zip', args[0]) + + mocked_ZipFile.assert_called_once() + (args, kwargs) = mocked_ZipFile.call_args_list[0] + self.assertEqual('/path/to/archive.zip', args[0]) + + instance.open.assert_called_once() + (args, kwargs) = instance.open.call_args_list[0] + self.assertEqual('deep/path/to/file.txt', args[0]) + if __name__ == '__main__': unittest.main() From 036de74f6f685cc95b81cfa76eac1885bba0f3e7 Mon Sep 17 00:00:00 2001 From: Keliang Chen Date: Sun, 12 Aug 2018 11:03:43 -0700 Subject: [PATCH 052/808] [AIRFLOW-2524] Add Amazon SageMaker Training (#3658) Add SageMaker Hook, Training Operator & Sensor Co-authored-by: srrajeev-aws --- airflow/contrib/hooks/sagemaker_hook.py | 241 ++++++++++ .../sagemaker_create_training_job_operator.py | 119 +++++ .../contrib/sensors/sagemaker_base_sensor.py | 76 ++++ .../sensors/sagemaker_training_sensor.py | 66 +++ tests/contrib/hooks/test_sagemaker_hook.py | 415 ++++++++++++++++++ ..._sagemaker_create_training_job_operator.py | 141 ++++++ .../sensors/test_sagemaker_base_sensor.py | 149 +++++++ .../sensors/test_sagemaker_training_sensor.py | 118 +++++ 8 files changed, 1325 insertions(+) create mode 100644 airflow/contrib/hooks/sagemaker_hook.py create mode 100644 airflow/contrib/operators/sagemaker_create_training_job_operator.py create mode 100644 airflow/contrib/sensors/sagemaker_base_sensor.py create mode 100644 airflow/contrib/sensors/sagemaker_training_sensor.py create mode 100644 tests/contrib/hooks/test_sagemaker_hook.py create mode 100644 tests/contrib/operators/test_sagemaker_create_training_job_operator.py create mode 100644 tests/contrib/sensors/test_sagemaker_base_sensor.py create mode 100644 tests/contrib/sensors/test_sagemaker_training_sensor.py diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py new file mode 100644 index 0000000000000..8b8e2e41e7678 --- /dev/null +++ b/airflow/contrib/hooks/sagemaker_hook.py @@ -0,0 +1,241 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +import copy +import time +from botocore.exceptions import ClientError + +from airflow.exceptions import AirflowException +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.hooks.S3_hook import S3Hook + + +class SageMakerHook(AwsHook): + """ + Interact with Amazon SageMaker. + sagemaker_conn_id is required for using + the config stored in db for training/tuning + """ + + def __init__(self, + sagemaker_conn_id=None, + use_db_config=False, + region_name=None, + check_interval=5, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerHook, self).__init__(*args, **kwargs) + self.sagemaker_conn_id = sagemaker_conn_id + self.use_db_config = use_db_config + self.region_name = region_name + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + self.conn = self.get_conn() + + def check_for_url(self, s3url): + """ + check if the s3url exists + :param s3url: S3 url + :type s3url:str + :return: bool + """ + bucket, key = S3Hook.parse_s3_url(s3url) + s3hook = S3Hook(aws_conn_id=self.aws_conn_id) + if not s3hook.check_for_bucket(bucket_name=bucket): + raise AirflowException( + "The input S3 Bucket {} does not exist ".format(bucket)) + if not s3hook.check_for_key(key=key, bucket_name=bucket): + raise AirflowException("The input S3 Key {} does not exist in the Bucket" + .format(s3url, bucket)) + return True + + def check_valid_training_input(self, training_config): + """ + Run checks before a training starts + :param training_config: training_config + :type training_config: dict + :return: None + """ + for channel in training_config['InputDataConfig']: + self.check_for_url(channel['DataSource'] + ['S3DataSource']['S3Uri']) + + def check_valid_tuning_input(self, tuning_config): + """ + Run checks before a tuning job starts + :param tuning_config: tuning_config + :type tuning_config: dict + :return: None + """ + for channel in tuning_config['TrainingJobDefinition']['InputDataConfig']: + self.check_for_url(channel['DataSource'] + ['S3DataSource']['S3Uri']) + + def check_status(self, non_terminal_states, + failed_state, key, + describe_function, *args): + """ + :param non_terminal_states: the set of non_terminal states + :type non_terminal_states: dict + :param failed_state: the set of failed states + :type failed_state: dict + :param key: the key of the response dict + that points to the state + :type key: string + :param describe_function: the function used to retrieve the status + :type describe_function: python callable + :param args: the arguments for the function + :return: None + """ + sec = 0 + running = True + + while running: + + sec = sec + self.check_interval + + if self.max_ingestion_time and sec > self.max_ingestion_time: + # ensure that the job gets killed if the max ingestion time is exceeded + raise AirflowException("SageMaker job took more than " + "%s seconds", self.max_ingestion_time) + + time.sleep(self.check_interval) + try: + response = describe_function(*args) + status = response[key] + self.log.info("Job still running for %s seconds... " + "current status is %s" % (sec, status)) + except KeyError: + raise AirflowException("Could not get status of the SageMaker job") + except ClientError: + raise AirflowException("AWS request failed, check log for more info") + + if status in non_terminal_states: + running = True + elif status in failed_state: + raise AirflowException("SageMaker job failed because %s" + % response['FailureReason']) + else: + running = False + + self.log.info('SageMaker Job Compeleted') + + def get_conn(self): + """ + Establish an AWS connection + :return: a boto3 SageMaker client + """ + return self.get_client_type('sagemaker', region_name=self.region_name) + + def list_training_job(self, name_contains=None, status_equals=None): + """ + List the training jobs associated with the given input + :param name_contains: A string in the training job name + :type name_contains: str + :param status_equals: 'InProgress'|'Completed' + |'Failed'|'Stopping'|'Stopped' + :return:dict + """ + return self.conn.list_training_jobs( + NameContains=name_contains, StatusEquals=status_equals) + + def list_tuning_job(self, name_contains=None, status_equals=None): + """ + List the tuning jobs associated with the given input + :param name_contains: A string in the training job name + :type name_contains: str + :param status_equals: 'InProgress'|'Completed' + |'Failed'|'Stopping'|'Stopped' + :return:dict + """ + return self.conn.list_hyper_parameter_tuning_job( + NameContains=name_contains, StatusEquals=status_equals) + + def create_training_job(self, training_job_config, wait_for_completion=True): + """ + Create a training job + :param training_job_config: the config for training + :type training_job_config: dict + :param wait_for_completion: if the program should keep running until job finishes + :param wait_for_completion: bool + :return: A dict that contains ARN of the training job. + """ + if self.use_db_config: + if not self.sagemaker_conn_id: + raise AirflowException("SageMaker connection id must be present to read \ + SageMaker training jobs configuration.") + sagemaker_conn = self.get_connection(self.sagemaker_conn_id) + + config = copy.deepcopy(sagemaker_conn.extra_dejson) + training_job_config.update(config) + + self.check_valid_training_input(training_job_config) + + response = self.conn.create_training_job( + **training_job_config) + if wait_for_completion: + self.check_status(['InProgress', 'Stopping', 'Stopped'], + ['Failed'], + 'TrainingJobStatus', + self.describe_training_job, + training_job_config['TrainingJobName']) + return response + + def create_tuning_job(self, tuning_job_config): + """ + Create a tuning job + :param tuning_job_config: the config for tuning + :type tuning_job_config: dict + :return: A dict that contains ARN of the tuning job. + """ + if self.use_db_config: + if not self.sagemaker_conn_id: + raise AirflowException( + "sagemaker connection id must be present to \ + read sagemaker tunning job configuration.") + + sagemaker_conn = self.get_connection(self.sagemaker_conn_id) + + config = sagemaker_conn.extra_dejson.copy() + tuning_job_config.update(config) + + self.check_valid_tuning_input(tuning_job_config) + + return self.conn.create_hyper_parameter_tuning_job( + **tuning_job_config) + + def describe_training_job(self, training_job_name): + """ + :param training_job_name: the name of the training job + :type train_job_name: string + Return the training job info associated with the current job_name + :return: A dict contains all the training job info + """ + return self.conn\ + .describe_training_job(TrainingJobName=training_job_name) + + def describe_tuning_job(self, tuning_job_name): + """ + :param tuning_job_name: the name of the training job + :type tuning_job_name: string + Return the tuning job info associated with the current job_name + :return: A dict contains all the tuning job info + """ + return self.conn\ + .describe_hyper_parameter_tuning_job( + HyperParameterTuningJobName=tuning_job_name) diff --git a/airflow/contrib/operators/sagemaker_create_training_job_operator.py b/airflow/contrib/operators/sagemaker_create_training_job_operator.py new file mode 100644 index 0000000000000..409c5f6aa936a --- /dev/null +++ b/airflow/contrib/operators/sagemaker_create_training_job_operator.py @@ -0,0 +1,119 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerCreateTrainingJobOperator(BaseOperator): + + """ + Initiate a SageMaker training + + This operator returns The ARN of the model created in Amazon SageMaker + + :param training_job_config: + The configuration necessary to start a training job (templated) + :type training_job_config: dict + :param region_name: The AWS region_name + :type region_name: string + :param sagemaker_conn_id: The SageMaker connection ID to use. + :type sagemaker_conn_id: string + :param use_db_config: Whether or not to use db config + associated with sagemaker_conn_id. + If set to true, will automatically update the training config + with what's in db, so the db config doesn't need to + included everything, but what's there does replace the ones + in the training_job_config, so be careful + :type use_db_config: bool + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: string + :param wait_for_completion: if the operator should block + until training job finishes + :type wait_for_completion: bool + :param check_interval: if wait is set to be true, this is the time interval + which the operator will check the status of the training job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the training job hasn't finish within the max_ingestion_time + (Caution: be careful to set this parameters because training can take very long) + :type max_ingestion_time: int + + **Example**: + The following operator would start a training job when executed + + sagemaker_training = + SageMakerCreateTrainingJobOperator( + task_id='sagemaker_training', + training_job_config=config, + region_name='us-west-2' + sagemaker_conn_id='sagemaker_customers_conn', + use_db_config=True, + aws_conn_id='aws_customers_conn' + ) + """ + + template_fields = ['training_job_config'] + template_ext = () + ui_color = '#ededed' + + @apply_defaults + def __init__(self, + training_job_config=None, + region_name=None, + sagemaker_conn_id=None, + use_db_config=False, + wait_for_completion=True, + check_interval=5, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerCreateTrainingJobOperator, self).__init__(*args, **kwargs) + + self.sagemaker_conn_id = sagemaker_conn_id + self.training_job_config = training_job_config + self.use_db_config = use_db_config + self.region_name = region_name + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + + def execute(self, context): + sagemaker = SageMakerHook( + sagemaker_conn_id=self.sagemaker_conn_id, + use_db_config=self.use_db_config, + region_name=self.region_name, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + + self.log.info( + "Creating SageMaker Training Job %s." + % self.training_job_config['TrainingJobName'] + ) + response = sagemaker.create_training_job( + self.training_job_config, + wait_for_completion=self.wait_for_completion) + if not response['ResponseMetadata']['HTTPStatusCode'] \ + == 200: + raise AirflowException( + 'Sagemaker Training Job creation failed: %s' % response) + else: + return response diff --git a/airflow/contrib/sensors/sagemaker_base_sensor.py b/airflow/contrib/sensors/sagemaker_base_sensor.py new file mode 100644 index 0000000000000..149c2a1aab124 --- /dev/null +++ b/airflow/contrib/sensors/sagemaker_base_sensor.py @@ -0,0 +1,76 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerBaseSensor(BaseSensorOperator): + """ + Contains general sensor behavior for SageMaker. + Subclasses should implement get_sagemaker_response() + and state_from_response() methods. + Subclasses should also implement NON_TERMINAL_STATES and FAILED_STATE methods. + """ + ui_color = '#66c3ff' + + @apply_defaults + def __init__( + self, + aws_conn_id='aws_default', + *args, **kwargs): + super(SageMakerBaseSensor, self).__init__(*args, **kwargs) + self.aws_conn_id = aws_conn_id + + def poke(self, context): + response = self.get_sagemaker_response() + + if not response['ResponseMetadata']['HTTPStatusCode'] == 200: + self.log.info('Bad HTTP response: %s', response) + return False + + state = self.state_from_response(response) + + self.log.info('Job currently %s', state) + + if state in self.non_terminal_states(): + return False + + if state in self.failed_states(): + failed_reason = self.get_failed_reason_from_response(response) + raise AirflowException("Sagemaker job failed for the following reason: %s" + % failed_reason) + return True + + def non_terminal_states(self): + raise AirflowException("Non Terminal States need to be specified in subclass") + + def failed_states(self): + raise AirflowException("Failed States need to be specified in subclass") + + def get_sagemaker_response(self): + raise AirflowException( + "Method get_sagemaker_response()not implemented.") + + def get_failed_reason_from_response(self, response): + return 'Unknown' + + def state_from_response(self, response): + raise AirflowException( + "Method state_from_response()not implemented.") diff --git a/airflow/contrib/sensors/sagemaker_training_sensor.py b/airflow/contrib/sensors/sagemaker_training_sensor.py new file mode 100644 index 0000000000000..90c62ce988fbf --- /dev/null +++ b/airflow/contrib/sensors/sagemaker_training_sensor.py @@ -0,0 +1,66 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor +from airflow.utils.decorators import apply_defaults + + +class SageMakerTrainingSensor(SageMakerBaseSensor): + """ + Asks for the state of the training state until it reaches a terminal state. + If it fails the sensor errors, failing the task. + + :param job_name: job_name of the training instance to check the state of + :type job_name: string + """ + + template_fields = ['job_name'] + template_ext = () + + @apply_defaults + def __init__(self, + job_name, + region_name=None, + *args, + **kwargs): + super(SageMakerTrainingSensor, self).__init__(*args, **kwargs) + self.job_name = job_name + self.region_name = region_name + + def non_terminal_states(self): + return ['InProgress', 'Stopping', 'Stopped'] + + def failed_states(self): + return ['Failed'] + + def get_sagemaker_response(self): + sagemaker = SageMakerHook( + aws_conn_id=self.aws_conn_id, + region_name=self.region_name + ) + + self.log.info('Poking Sagemaker Training Job %s', self.job_name) + return sagemaker.describe_training_job(self.job_name) + + def get_failed_reason_from_response(self, response): + return response['FailureReason'] + + def state_from_response(self, response): + return response['TrainingJobStatus'] diff --git a/tests/contrib/hooks/test_sagemaker_hook.py b/tests/contrib/hooks/test_sagemaker_hook.py new file mode 100644 index 0000000000000..6887a5b484bed --- /dev/null +++ b/tests/contrib/hooks/test_sagemaker_hook.py @@ -0,0 +1,415 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + + +import json +import unittest +import copy +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow import models +from airflow.utils import db +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.hooks.S3_hook import S3Hook +from airflow.exceptions import AirflowException + + +role = 'test-role' + +bucket = 'test-bucket' + +key = 'test/data' +data_url = 's3://{}/{}'.format(bucket, key) + +job_name = 'test-job-name' + +image = 'test-image' + +test_arn_return = {'TrainingJobArn': 'testarn'} + +test_list_training_job_return = { + 'TrainingJobSummaries': [ + { + 'TrainingJobName': job_name, + 'TrainingJobStatus': 'InProgress' + }, + ], + 'NextToken': 'test-token' +} + +test_list_tuning_job_return = { + 'TrainingJobSummaries': [ + { + 'TrainingJobName': job_name, + 'TrainingJobArn': 'testarn', + 'TunedHyperParameters': { + 'k': '3' + }, + 'TrainingJobStatus': 'InProgress' + }, + ], + 'NextToken': 'test-token' +} + +output_url = 's3://{}/test/output'.format(bucket) +create_training_params = \ + { + 'AlgorithmSpecification': { + 'TrainingImage': image, + 'TrainingInputMode': 'File' + }, + 'RoleArn': role, + 'OutputDataConfig': { + 'S3OutputPath': output_url + }, + 'ResourceConfig': { + 'InstanceCount': 2, + 'InstanceType': 'ml.c4.8xlarge', + 'VolumeSizeInGB': 50 + }, + 'TrainingJobName': job_name, + 'HyperParameters': { + 'k': '10', + 'feature_dim': '784', + 'mini_batch_size': '500', + 'force_dense': 'True' + }, + 'StoppingCondition': { + 'MaxRuntimeInSeconds': 60 * 60 + }, + 'InputDataConfig': [ + { + 'ChannelName': 'train', + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url, + 'S3DataDistributionType': 'FullyReplicated' + } + }, + 'CompressionType': 'None', + 'RecordWrapperType': 'None' + } + ] + } + +create_tuning_params = \ + { + 'HyperParameterTuningJobName': job_name, + 'HyperParameterTuningJobConfig': { + 'Strategy': 'Bayesian', + 'HyperParameterTuningJobObjective': { + 'Type': 'Maximize', + 'MetricName': 'test_metric' + }, + 'ResourceLimits': { + 'MaxNumberOfTrainingJobs': 123, + 'MaxParallelTrainingJobs': 123 + }, + 'ParameterRanges': { + 'IntegerParameterRanges': [ + { + 'Name': 'k', + 'MinValue': '2', + 'MaxValue': '10' + }, + + ] + } + }, + 'TrainingJobDefinition': { + 'StaticHyperParameters': create_training_params['HyperParameters'], + 'AlgorithmSpecification': create_training_params['AlgorithmSpecification'], + 'RoleArn': 'string', + 'InputDataConfig': create_training_params['InputDataConfig'], + 'OutputDataConfig': create_training_params['OutputDataConfig'], + 'ResourceConfig': create_training_params['ResourceConfig'], + 'StoppingCondition': dict(MaxRuntimeInSeconds=60 * 60) + } + } + +db_config = { + 'Tags': [ + { + 'Key': 'test-db-key', + 'Value': 'test-db-value', + + }, + ] +} + +DESCRIBE_TRAINING_INPROGRESS_RETURN = { + 'TrainingJobStatus': 'InProgress', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_COMPELETED_RETURN = { + 'TrainingJobStatus': 'Compeleted', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_FAILED_RETURN = { + 'TrainingJobStatus': 'Failed', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + }, + 'FailureReason': 'Unknown' +} +DESCRIBE_TRAINING_STOPPING_RETURN = { + 'TrainingJobStatus': 'Stopping', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_STOPPED_RETURN = { + 'TrainingJobStatus': 'Stopped', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + + +class TestSageMakerHook(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + db.merge_conn( + models.Connection( + conn_id='sagemaker_test_conn_id', + conn_type='sagemaker', + login='access_id', + password='access_key', + extra=json.dumps(db_config) + ) + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(S3Hook, 'check_for_key') + @mock.patch.object(S3Hook, 'check_for_bucket') + def test_check_for_url(self, + mock_check_bucket, mock_check_key, mock_client): + mock_client.return_value = None + hook = SageMakerHook() + mock_check_bucket.side_effect = [False, True, True] + mock_check_key.side_effect = [False, True] + self.assertRaises(AirflowException, + hook.check_for_url, data_url) + self.assertRaises(AirflowException, + hook.check_for_url, data_url) + self.assertEqual(hook.check_for_url(data_url), True) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'check_for_url') + def test_check_valid_training(self, mock_check_url, mock_client): + mock_client.return_value = None + hook = SageMakerHook() + hook.check_valid_training_input(create_training_params) + mock_check_url.assert_called_once_with(data_url) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'check_for_url') + def test_check_valid_tuning(self, mock_check_url, mock_client): + mock_client.return_value = None + hook = SageMakerHook() + hook.check_valid_tuning_input(create_tuning_params) + mock_check_url.assert_called_once_with(data_url) + + @mock.patch.object(SageMakerHook, 'get_client_type') + def test_conn(self, mock_get_client): + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', + region_name='us-east-1' + ) + self.assertEqual(hook.sagemaker_conn_id, 'sagemaker_test_conn_id') + mock_get_client.assert_called_once_with('sagemaker', + region_name='us-east-1' + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_list_training_job(self, mock_client): + mock_session = mock.Mock() + attrs = {'list_training_jobs.return_value': + test_list_training_job_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.list_training_job(name_contains=job_name, + status_equals='InProgress') + mock_session.list_training_jobs. \ + assert_called_once_with(NameContains=job_name, + StatusEquals='InProgress') + self.assertEqual(response, test_list_training_job_return) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_list_tuning_job(self, mock_client): + mock_session = mock.Mock() + attrs = {'list_hyper_parameter_tuning_job.return_value': + test_list_tuning_job_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.list_tuning_job(name_contains=job_name, + status_equals='InProgress') + mock_session.list_hyper_parameter_tuning_job. \ + assert_called_once_with(NameContains=job_name, + StatusEquals='InProgress') + self.assertEqual(response, test_list_tuning_job_return) + + @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_training_job(self, mock_client, mock_check_training): + mock_check_training.return_value = True + mock_session = mock.Mock() + attrs = {'create_training_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.create_training_job(create_training_params, + wait_for_completion=False) + mock_session.create_training_job.assert_called_once_with(**create_training_params) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_training_job_db_config(self, mock_client, mock_check_training): + mock_check_training.return_value = True + mock_session = mock.Mock() + attrs = {'create_training_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook_use_db_config = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', + use_db_config=True) + response = hook_use_db_config.create_training_job(create_training_params, + wait_for_completion=False) + updated_config = copy.deepcopy(create_training_params) + updated_config.update(db_config) + mock_session.create_training_job.assert_called_once_with(**updated_config) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_training_ends_with_wait_on(self, mock_client, mock_check_training): + mock_check_training.return_value = True + mock_session = mock.Mock() + attrs = {'create_training_job.return_value': + test_arn_return, + 'describe_training_job.side_effect': + [DESCRIBE_TRAINING_INPROGRESS_RETURN, + DESCRIBE_TRAINING_STOPPING_RETURN, + DESCRIBE_TRAINING_STOPPED_RETURN, + DESCRIBE_TRAINING_COMPELETED_RETURN] + } + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id_1') + hook.create_training_job(create_training_params, wait_for_completion=True) + self.assertEqual(mock_session.describe_training_job.call_count, 4) + + @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_training_throws_error_when_failed_with_wait_on( + self, mock_client, mock_check_training): + mock_check_training.return_value = True + mock_session = mock.Mock() + attrs = {'create_training_job.return_value': + test_arn_return, + 'describe_training_job.side_effect': + [DESCRIBE_TRAINING_INPROGRESS_RETURN, + DESCRIBE_TRAINING_STOPPING_RETURN, + DESCRIBE_TRAINING_STOPPED_RETURN, + DESCRIBE_TRAINING_FAILED_RETURN] + } + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id_1') + self.assertRaises(AirflowException, hook.create_training_job, + create_training_params, wait_for_completion=True) + self.assertEqual(mock_session.describe_training_job.call_count, 4) + + @mock.patch.object(SageMakerHook, 'check_valid_tuning_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_tuning_job(self, mock_client, mock_check_tuning): + mock_session = mock.Mock() + attrs = {'create_hyper_parameter_tuning_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.create_tuning_job(create_tuning_params) + mock_session.create_hyper_parameter_tuning_job.\ + assert_called_once_with(**create_tuning_params) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'check_valid_tuning_input') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_tuning_job_db_config(self, mock_client, mock_check_tuning): + mock_check_tuning.return_value = True + mock_session = mock.Mock() + attrs = {'create_hyper_parameter_tuning_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', + use_db_config=True) + response = hook.create_tuning_job(create_tuning_params) + updated_config = copy.deepcopy(create_tuning_params) + updated_config.update(db_config) + mock_session.create_hyper_parameter_tuning_job. \ + assert_called_once_with(**updated_config) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_training_job(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_training_job.return_value': 'InProgress'} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.describe_training_job(job_name) + mock_session.describe_training_job.\ + assert_called_once_with(TrainingJobName=job_name) + self.assertEqual(response, 'InProgress') + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_tuning_job(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_hyper_parameter_tuning_job.return_value': + 'InProgress'} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.describe_tuning_job(job_name) + mock_session.describe_hyper_parameter_tuning_job.\ + assert_called_once_with(HyperParameterTuningJobName=job_name) + self.assertEqual(response, 'InProgress') + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_create_training_job_operator.py b/tests/contrib/operators/test_sagemaker_create_training_job_operator.py new file mode 100644 index 0000000000000..156c9d74c79ec --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_create_training_job_operator.py @@ -0,0 +1,141 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_create_training_job_operator \ + import SageMakerCreateTrainingJobOperator +from airflow.exceptions import AirflowException + +role = "test-role" + +bucket = "test-bucket" + +key = "test/data" +data_url = "s3://{}/{}".format(bucket, key) + +job_name = "test-job-name" + +image = "test-image" + +output_url = "s3://{}/test/output".format(bucket) +create_training_params = \ + { + "AlgorithmSpecification": { + "TrainingImage": image, + "TrainingInputMode": "File" + }, + "RoleArn": role, + "OutputDataConfig": { + "S3OutputPath": output_url + }, + "ResourceConfig": { + "InstanceCount": 2, + "InstanceType": "ml.c4.8xlarge", + "VolumeSizeInGB": 50 + }, + "TrainingJobName": job_name, + "HyperParameters": { + "k": "10", + "feature_dim": "784", + "mini_batch_size": "500", + "force_dense": "True" + }, + "StoppingCondition": { + "MaxRuntimeInSeconds": 60 * 60 + }, + "InputDataConfig": [ + { + "ChannelName": "train", + "DataSource": { + "S3DataSource": { + "S3DataType": "S3Prefix", + "S3Uri": data_url, + "S3DataDistributionType": "FullyReplicated" + } + }, + "CompressionType": "None", + "RecordWrapperType": "None" + } + ] + } + + +class TestSageMakerTrainingOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerCreateTrainingJobOperator( + task_id='test_sagemaker_operator', + sagemaker_conn_id='sagemaker_test_id', + training_job_config=create_training_params, + region_name='us-west-2', + use_db_config=True, + wait_for_completion=False, + check_interval=5 + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_training_job') + @mock.patch.object(SageMakerHook, '__init__') + def test_hook_init(self, hook_init, mock_training, mock_client): + mock_training.return_value = {"TrainingJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 200}} + hook_init.return_value = None + self.sagemaker.execute(None) + hook_init.assert_called_once_with( + sagemaker_conn_id='sagemaker_test_id', + region_name='us-west-2', + use_db_config=True, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_training_job') + def test_execute_without_failure(self, mock_training, mock_client): + mock_training.return_value = {"TrainingJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 200}} + self.sagemaker.execute(None) + mock_training.assert_called_once_with(create_training_params, + wait_for_completion=False + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_training_job') + def test_execute_with_failure(self, mock_training, mock_client): + mock_training.return_value = {"TrainingJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_base_sensor.py b/tests/contrib/sensors/test_sagemaker_base_sensor.py new file mode 100644 index 0000000000000..bc8cbe349858f --- /dev/null +++ b/tests/contrib/sensors/test_sagemaker_base_sensor.py @@ -0,0 +1,149 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from airflow import configuration +from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor +from airflow.exceptions import AirflowException + + +class TestSagemakerBaseSensor(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + def test_subclasses_succeed_when_response_is_good(self): + class SageMakerBaseSensorSubclass(SageMakerBaseSensor): + def non_terminal_states(self): + return ['PENDING', 'RUNNING', 'CONTINUE'] + + def failed_states(self): + return ['FAILED'] + + def get_sagemaker_response(self): + return { + 'SomeKey': {'State': 'COMPLETED'}, + 'ResponseMetadata': {'HTTPStatusCode': 200} + } + + def state_from_response(self, response): + return response['SomeKey']['State'] + + sensor = SageMakerBaseSensorSubclass( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test' + ) + + sensor.execute(None) + + def test_poke_returns_false_when_state_is_a_non_terminal_state(self): + class SageMakerBaseSensorSubclass(SageMakerBaseSensor): + def non_terminal_states(self): + return ['PENDING', 'RUNNING', 'CONTINUE'] + + def failed_states(self): + return ['FAILED'] + + def get_sagemaker_response(self): + return { + 'SomeKey': {'State': 'PENDING'}, + 'ResponseMetadata': {'HTTPStatusCode': 200} + } + + def state_from_response(self, response): + return response['SomeKey']['State'] + + sensor = SageMakerBaseSensorSubclass( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test' + ) + + self.assertEqual(sensor.poke(None), False) + + def test_poke_raise_exception_when_method_not_implemented(self): + class SageMakerBaseSensorSubclass(SageMakerBaseSensor): + def non_terminal_states(self): + return ['PENDING', 'RUNNING', 'CONTINUE'] + + def failed_states(self): + return ['FAILED'] + + sensor = SageMakerBaseSensorSubclass( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test' + ) + + self.assertRaises(AirflowException, sensor.poke, None) + + def test_poke_returns_false_when_http_response_is_bad(self): + class SageMakerBaseSensorSubclass(SageMakerBaseSensor): + def non_terminal_states(self): + return ['PENDING', 'RUNNING', 'CONTINUE'] + + def failed_states(self): + return ['FAILED'] + + def get_sagemaker_response(self): + return { + 'SomeKey': {'State': 'COMPLETED'}, + 'ResponseMetadata': {'HTTPStatusCode': 400} + } + + def state_from_response(self, response): + return response['SomeKey']['State'] + + sensor = SageMakerBaseSensorSubclass( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test' + ) + + self.assertEqual(sensor.poke(None), False) + + def test_poke_raises_error_when_job_has_failed(self): + class SageMakerBaseSensorSubclass(SageMakerBaseSensor): + def non_terminal_states(self): + return ['PENDING', 'RUNNING', 'CONTINUE'] + + def failed_states(self): + return ['FAILED'] + + def get_sagemaker_response(self): + return { + 'SomeKey': {'State': 'FAILED'}, + 'ResponseMetadata': {'HTTPStatusCode': 200} + } + + def state_from_response(self, response): + return response['SomeKey']['State'] + + sensor = SageMakerBaseSensorSubclass( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test' + ) + + self.assertRaises(AirflowException, sensor.poke, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_training_sensor.py b/tests/contrib/sensors/test_sagemaker_training_sensor.py new file mode 100644 index 0000000000000..fb966f60afbf0 --- /dev/null +++ b/tests/contrib/sensors/test_sagemaker_training_sensor.py @@ -0,0 +1,118 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.sensors.sagemaker_training_sensor \ + import SageMakerTrainingSensor +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.exceptions import AirflowException + +DESCRIBE_TRAINING_INPROGRESS_RETURN = { + 'TrainingJobStatus': 'InProgress', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_COMPELETED_RETURN = { + 'TrainingJobStatus': 'Compeleted', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_FAILED_RETURN = { + 'TrainingJobStatus': 'Failed', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + }, + 'FailureReason': 'Unknown' +} +DESCRIBE_TRAINING_STOPPING_RETURN = { + 'TrainingJobStatus': 'Stopping', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRAINING_STOPPED_RETURN = { + 'TrainingJobStatus': 'Stopped', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + + +class TestSageMakerTrainingSensor(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'describe_training_job') + def test_raises_errors_failed_state(self, mock_describe_job, mock_client): + mock_describe_job.side_effect = [DESCRIBE_TRAINING_FAILED_RETURN] + sensor = SageMakerTrainingSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name' + ) + self.assertRaises(AirflowException, sensor.execute, None) + mock_describe_job.assert_called_once_with('test_job_name') + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, '__init__') + @mock.patch.object(SageMakerHook, 'describe_training_job') + def test_calls_until_a_terminal_state(self, + mock_describe_job, hook_init, mock_client): + hook_init.return_value = None + + mock_describe_job.side_effect = [ + DESCRIBE_TRAINING_INPROGRESS_RETURN, + DESCRIBE_TRAINING_STOPPING_RETURN, + DESCRIBE_TRAINING_STOPPED_RETURN, + DESCRIBE_TRAINING_COMPELETED_RETURN + ] + sensor = SageMakerTrainingSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name', + region_name='us-east-1' + ) + + sensor.execute(None) + + # make sure we called 4 times(terminated when its compeleted) + self.assertEqual(mock_describe_job.call_count, 4) + + # make sure the hook was initialized with the specific params + hook_init.assert_called_with(aws_conn_id='aws_test', + region_name='us-east-1') + + +if __name__ == '__main__': + unittest.main() From 1d5a6ce9e1543b80add438eb08fd7adbda826966 Mon Sep 17 00:00:00 2001 From: Keliang Chen Date: Sun, 19 Aug 2018 07:31:41 -0700 Subject: [PATCH 053/808] [AIRFLOW-2524] Add Amazon SageMaker Tuning (#3751) Add SageMaker tuning Operator and sensor Co-authored-by: srrajeev-aws --- airflow/contrib/hooks/sagemaker_hook.py | 24 ++- .../sagemaker_create_tuning_job_operator.py | 121 +++++++++++++ .../sensors/sagemaker_tuning_sensor.py | 69 ++++++++ tests/contrib/hooks/test_sagemaker_hook.py | 18 +- ...st_sagemaker_create_tuning_job_operator.py | 165 ++++++++++++++++++ .../sensors/test_sagemaker_tuning_sensor.py | 118 +++++++++++++ 6 files changed, 505 insertions(+), 10 deletions(-) create mode 100644 airflow/contrib/operators/sagemaker_create_tuning_job_operator.py create mode 100644 airflow/contrib/sensors/sagemaker_tuning_sensor.py create mode 100644 tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py create mode 100644 tests/contrib/sensors/test_sagemaker_tuning_sensor.py diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py index 8b8e2e41e7678..09993f96d8738 100644 --- a/airflow/contrib/hooks/sagemaker_hook.py +++ b/airflow/contrib/hooks/sagemaker_hook.py @@ -59,8 +59,13 @@ def check_for_url(self, s3url): if not s3hook.check_for_bucket(bucket_name=bucket): raise AirflowException( "The input S3 Bucket {} does not exist ".format(bucket)) - if not s3hook.check_for_key(key=key, bucket_name=bucket): - raise AirflowException("The input S3 Key {} does not exist in the Bucket" + if key and not s3hook.check_for_key(key=key, bucket_name=bucket)\ + and not s3hook.check_for_prefix( + prefix=key, bucket_name=bucket, delimiter='/'): + # check if s3 key exists in the case user provides a single file + # or if s3 prefix exists in the case user provides a prefix for files + raise AirflowException("The input S3 Key " + "or Prefix {} does not exist in the Bucket {}" .format(s3url, bucket)) return True @@ -196,11 +201,13 @@ def create_training_job(self, training_job_config, wait_for_completion=True): training_job_config['TrainingJobName']) return response - def create_tuning_job(self, tuning_job_config): + def create_tuning_job(self, tuning_job_config, wait_for_completion=True): """ Create a tuning job :param tuning_job_config: the config for tuning :type tuning_job_config: dict + :param wait_for_completion: if the program should keep running until job finishes + :param wait_for_completion: bool :return: A dict that contains ARN of the tuning job. """ if self.use_db_config: @@ -216,13 +223,20 @@ def create_tuning_job(self, tuning_job_config): self.check_valid_tuning_input(tuning_job_config) - return self.conn.create_hyper_parameter_tuning_job( + response = self.conn.create_hyper_parameter_tuning_job( **tuning_job_config) + if wait_for_completion: + self.check_status(['InProgress', 'Stopping', 'Stopped'], + ['Failed'], + 'HyperParameterTuningJobStatus', + self.describe_tuning_job, + tuning_job_config['HyperParameterTuningJobName']) + return response def describe_training_job(self, training_job_name): """ :param training_job_name: the name of the training job - :type train_job_name: string + :type training_job_name: string Return the training job info associated with the current job_name :return: A dict contains all the training job info """ diff --git a/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py b/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py new file mode 100644 index 0000000000000..0c40a9adc93f4 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py @@ -0,0 +1,121 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerCreateTuningJobOperator(BaseOperator): + + """ + Initiate a SageMaker HyperParameter Tuning Job + + This operator returns The ARN of the model created in Amazon SageMaker + + :param sagemaker_conn_id: The SageMaker connection ID to use. + :type sagemaker_conn_id: string + :param region_name: The AWS region_name + :type region_name: string + :param tuning_job_config: + The configuration necessary to start a tuning job (templated) + :type tuning_job_config: dict + :param use_db_config: Whether or not to use db config + associated with sagemaker_conn_id. + If set to true, will automatically update the tuning config + with what's in db, so the db config doesn't need to + included everything, but what's there does replace the ones + in the tuning_job_config, so be careful + :type use_db_config: bool + :param wait_for_completion: if the operator should block + until tuning job finishes + :type wait_for_completion: bool + :param check_interval: if wait is set to be true, this is the time interval + which the operator will check the status of the tuning job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the tuning job hasn't finish within the max_ingestion_time + (Caution: be careful to set this parameters because tuning can take very long) + :type max_ingestion_time: int + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: string + + **Example**: + The following operator would start a tuning job when executed + + sagemaker_tuning = + SageMakerCreateTuningJobOperator( + task_id='sagemaker_tuning', + sagemaker_conn_id='sagemaker_customers_conn', + tuning_job_config=config, + check_interval=2, + max_ingestion_time=3600, + aws_conn_id='aws_customers_conn', + ) + """ + + template_fields = ['tuning_job_config'] + template_ext = () + ui_color = '#ededed' + + @apply_defaults + def __init__(self, + sagemaker_conn_id=None, + region_name=None, + tuning_job_config=None, + use_db_config=False, + wait_for_completion=True, + check_interval=5, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerCreateTuningJobOperator, self)\ + .__init__(*args, **kwargs) + + self.sagemaker_conn_id = sagemaker_conn_id + self.region_name = region_name + self.tuning_job_config = tuning_job_config + self.use_db_config = use_db_config + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + + def execute(self, context): + sagemaker = SageMakerHook(sagemaker_conn_id=self.sagemaker_conn_id, + region_name=self.region_name, + use_db_config=self.use_db_config, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + + self.log.info( + "Creating SageMaker Hyper Parameter Tunning Job %s" + % self.tuning_job_config['HyperParameterTuningJobName'] + ) + + response = sagemaker.create_tuning_job( + self.tuning_job_config, + wait_for_completion=self.wait_for_completion + ) + if not response['ResponseMetadata']['HTTPStatusCode'] \ + == 200: + raise AirflowException( + "Sagemaker Tuning Job creation failed: %s" % response) + else: + return response diff --git a/airflow/contrib/sensors/sagemaker_tuning_sensor.py b/airflow/contrib/sensors/sagemaker_tuning_sensor.py new file mode 100644 index 0000000000000..bc74e3a5c5461 --- /dev/null +++ b/airflow/contrib/sensors/sagemaker_tuning_sensor.py @@ -0,0 +1,69 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor +from airflow.utils.decorators import apply_defaults + + +class SageMakerTuningSensor(SageMakerBaseSensor): + """ + Asks for the state of the tuning state until it reaches a terminal state. + The sensor will error if the job errors, throwing a AirflowException + containing the failure reason. + + :param job_name: job_name of the tuning instance to check the state of + :type job_name: string + :param region_name: The AWS region_name + :type region_name: string + """ + + template_fields = ['job_name'] + template_ext = () + + @apply_defaults + def __init__(self, + job_name, + region_name=None, + *args, + **kwargs): + super(SageMakerTuningSensor, self).__init__(*args, **kwargs) + self.job_name = job_name + self.region_name = region_name + + def non_terminal_states(self): + return ['InProgress', 'Stopping', 'Stopped'] + + def failed_states(self): + return ['Failed'] + + def get_sagemaker_response(self): + sagemaker = SageMakerHook( + aws_conn_id=self.aws_conn_id, + region_name=self.region_name + ) + + self.log.info('Poking Sagemaker Tuning Job %s', self.job_name) + return sagemaker.describe_tuning_job(self.job_name) + + def get_failed_reason_from_response(self, response): + return response['FailureReason'] + + def state_from_response(self, response): + return response['HyperParameterTuningJobStatus'] diff --git a/tests/contrib/hooks/test_sagemaker_hook.py b/tests/contrib/hooks/test_sagemaker_hook.py index 6887a5b484bed..8bb56cc8e7d12 100644 --- a/tests/contrib/hooks/test_sagemaker_hook.py +++ b/tests/contrib/hooks/test_sagemaker_hook.py @@ -212,17 +212,23 @@ def setUp(self): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(S3Hook, 'check_for_key') @mock.patch.object(S3Hook, 'check_for_bucket') + @mock.patch.object(S3Hook, 'check_for_prefix') def test_check_for_url(self, - mock_check_bucket, mock_check_key, mock_client): + mock_check_prefix, + mock_check_bucket, + mock_check_key, + mock_client): mock_client.return_value = None hook = SageMakerHook() - mock_check_bucket.side_effect = [False, True, True] - mock_check_key.side_effect = [False, True] + mock_check_bucket.side_effect = [False, True, True, True] + mock_check_key.side_effect = [False, True, False] + mock_check_prefix.side_effect = [False, True, True] self.assertRaises(AirflowException, hook.check_for_url, data_url) self.assertRaises(AirflowException, hook.check_for_url, data_url) self.assertEqual(hook.check_for_url(data_url), True) + self.assertEqual(hook.check_for_url(data_url), True) @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, 'check_for_url') @@ -362,7 +368,8 @@ def test_create_tuning_job(self, mock_client, mock_check_tuning): mock_session.configure_mock(**attrs) mock_client.return_value = mock_session hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') - response = hook.create_tuning_job(create_tuning_params) + response = hook.create_tuning_job(create_tuning_params, + wait_for_completion=False) mock_session.create_hyper_parameter_tuning_job.\ assert_called_once_with(**create_tuning_params) self.assertEqual(response, test_arn_return) @@ -378,7 +385,8 @@ def test_create_tuning_job_db_config(self, mock_client, mock_check_tuning): mock_client.return_value = mock_session hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', use_db_config=True) - response = hook.create_tuning_job(create_tuning_params) + response = hook.create_tuning_job(create_tuning_params, + wait_for_completion=False) updated_config = copy.deepcopy(create_tuning_params) updated_config.update(db_config) mock_session.create_hyper_parameter_tuning_job. \ diff --git a/tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py b/tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py new file mode 100644 index 0000000000000..d317cff6f2289 --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py @@ -0,0 +1,165 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_create_tuning_job_operator \ + import SageMakerCreateTuningJobOperator +from airflow.exceptions import AirflowException + +role = 'test-role' + +bucket = 'test-bucket' + +key = 'test/data' +data_url = 's3://{}/{}'.format(bucket, key) + +job_name = 'test-job-name' + +image = 'test-image' + +output_url = 's3://{}/test/output'.format(bucket) + +create_tuning_params = {'HyperParameterTuningJobName': job_name, + 'HyperParameterTuningJobConfig': { + 'Strategy': 'Bayesian', + 'HyperParameterTuningJobObjective': { + 'Type': 'Maximize', + 'MetricName': 'test_metric' + }, + 'ResourceLimits': { + 'MaxNumberOfTrainingJobs': 123, + 'MaxParallelTrainingJobs': 123 + }, + 'ParameterRanges': { + 'IntegerParameterRanges': [ + { + 'Name': 'k', + 'MinValue': '2', + 'MaxValue': '10' + }, + ] + } + }, + 'TrainingJobDefinition': { + 'StaticHyperParameters': + { + 'k': '10', + 'feature_dim': '784', + 'mini_batch_size': '500', + 'force_dense': 'True' + }, + 'AlgorithmSpecification': + { + 'TrainingImage': image, + 'TrainingInputMode': 'File' + }, + 'RoleArn': 'string', + 'InputDataConfig': + [ + { + 'ChannelName': 'train', + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url, + 'S3DataDistributionType': + 'FullyReplicated' + } + }, + 'CompressionType': 'None', + 'RecordWrapperType': 'None' + } + ], + 'OutputDataConfig': + { + 'S3OutputPath': output_url + }, + 'ResourceConfig': + { + 'InstanceCount': 2, + 'InstanceType': 'ml.c4.8xlarge', + 'VolumeSizeInGB': 50 + }, + 'StoppingCondition': dict(MaxRuntimeInSeconds=60 * 60) + } + } + + +class TestSageMakerTrainingOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerCreateTuningJobOperator( + task_id='test_sagemaker_operator', + sagemaker_conn_id='sagemaker_test_conn', + tuning_job_config=create_tuning_params, + region_name='us-east-1', + use_db_config=False, + wait_for_completion=False, + check_interval=5 + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_tuning_job') + @mock.patch.object(SageMakerHook, '__init__') + def test_hook_init(self, hook_init, mock_tuning, mock_client): + mock_tuning.return_value = {'TrainingJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + hook_init.return_value = None + self.sagemaker.execute(None) + hook_init.assert_called_once_with( + sagemaker_conn_id='sagemaker_test_conn', + region_name='us-east-1', + use_db_config=False, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_tuning_job') + def test_execute_without_failure(self, mock_tuning, mock_client): + mock_tuning.return_value = {'TrainingJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + self.sagemaker.execute(None) + mock_tuning.assert_called_once_with(create_tuning_params, + wait_for_completion=False) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_tuning_job') + def test_execute_with_failure(self, mock_tuning, mock_client): + mock_tuning.return_value = {'TrainingJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py new file mode 100644 index 0000000000000..49f9b41b07c89 --- /dev/null +++ b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py @@ -0,0 +1,118 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.sensors.sagemaker_tuning_sensor \ + import SageMakerTuningSensor +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.exceptions import AirflowException + +DESCRIBE_TUNING_INPROGRESS_RETURN = { + 'HyperParameterTuningJobStatus': 'InProgress', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TUNING_COMPELETED_RETURN = { + 'HyperParameterTuningJobStatus': 'Compeleted', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TUNING_FAILED_RETURN = { + 'HyperParameterTuningJobStatus': 'Failed', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + }, + 'FailureReason': 'Unknown' +} +DESCRIBE_TUNING_STOPPING_RETURN = { + 'HyperParameterTuningJobStatus': 'Stopping', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TUNING_STOPPED_RETURN = { + 'HyperParameterTuningJobStatus': 'Stopped', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + + +class TestSageMakerTuningSensor(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'describe_tuning_job') + def test_raises_errors_failed_state(self, mock_describe_job, mock_client): + mock_describe_job.side_effect = [DESCRIBE_TUNING_FAILED_RETURN] + sensor = SageMakerTuningSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name' + ) + self.assertRaises(AirflowException, sensor.execute, None) + mock_describe_job.assert_called_once_with('test_job_name') + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, '__init__') + @mock.patch.object(SageMakerHook, 'describe_tuning_job') + def test_calls_until_a_terminal_state(self, + mock_describe_job, hook_init, mock_client): + hook_init.return_value = None + + mock_describe_job.side_effect = [ + DESCRIBE_TUNING_INPROGRESS_RETURN, + DESCRIBE_TUNING_STOPPING_RETURN, + DESCRIBE_TUNING_STOPPED_RETURN, + DESCRIBE_TUNING_COMPELETED_RETURN + ] + sensor = SageMakerTuningSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name', + region_name='us-east-1' + ) + + sensor.execute(None) + + # make sure we called 4 times(terminated when its compeleted) + self.assertEqual(mock_describe_job.call_count, 4) + + # make sure the hook was initialized with the specific params + hook_init.assert_called_with(aws_conn_id='aws_test', + region_name='us-east-1') + + +if __name__ == '__main__': + unittest.main() From 9a347253771100c955dc3b911c971777679b7a15 Mon Sep 17 00:00:00 2001 From: Keliang Chen Date: Fri, 14 Sep 2018 02:19:25 -0500 Subject: [PATCH 054/808] [AIRFLOW-2524] Add SageMaker Batch Inference (#3767) * Fix for comments * Fix sensor test * Update non_terminal_states and failed_states to static variables of SageMakerHook Add SageMaker Transform Operator & Sensor Co-authored-by: srrajeev-aws --- airflow/contrib/hooks/sagemaker_hook.py | 79 ++++++++-- .../sagemaker_create_training_job_operator.py | 2 +- ...sagemaker_create_transform_job_operator.py | 132 +++++++++++++++++ .../sagemaker_create_tuning_job_operator.py | 2 +- .../sensors/sagemaker_training_sensor.py | 4 +- .../sensors/sagemaker_transform_sensor.py | 69 +++++++++ .../sensors/sagemaker_tuning_sensor.py | 4 +- tests/contrib/hooks/test_sagemaker_hook.py | 93 ++++++++++++ .../test_sagemaker_transform_sensor.py | 118 +++++++++++++++ ...sagemaker_create_transform_job_operator.py | 140 ++++++++++++++++++ 10 files changed, 627 insertions(+), 16 deletions(-) create mode 100644 airflow/contrib/operators/sagemaker_create_transform_job_operator.py create mode 100644 airflow/contrib/sensors/sagemaker_transform_sensor.py create mode 100644 tests/contrib/sensors/test_sagemaker_transform_sensor.py create mode 100644 tests/operators/test_sagemaker_create_transform_job_operator.py diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py index 09993f96d8738..ebab5d83e4099 100644 --- a/airflow/contrib/hooks/sagemaker_hook.py +++ b/airflow/contrib/hooks/sagemaker_hook.py @@ -31,6 +31,8 @@ class SageMakerHook(AwsHook): sagemaker_conn_id is required for using the config stored in db for training/tuning """ + non_terminal_states = {'InProgress', 'Stopping', 'Stopped'} + failed_states = {'Failed'} def __init__(self, sagemaker_conn_id=None, @@ -96,9 +98,9 @@ def check_status(self, non_terminal_states, describe_function, *args): """ :param non_terminal_states: the set of non_terminal states - :type non_terminal_states: dict + :type non_terminal_states: set :param failed_state: the set of failed states - :type failed_state: dict + :type failed_state: set :param key: the key of the response dict that points to the state :type key: string @@ -177,7 +179,7 @@ def create_training_job(self, training_job_config, wait_for_completion=True): :param training_job_config: the config for training :type training_job_config: dict :param wait_for_completion: if the program should keep running until job finishes - :param wait_for_completion: bool + :type wait_for_completion: bool :return: A dict that contains ARN of the training job. """ if self.use_db_config: @@ -194,8 +196,8 @@ def create_training_job(self, training_job_config, wait_for_completion=True): response = self.conn.create_training_job( **training_job_config) if wait_for_completion: - self.check_status(['InProgress', 'Stopping', 'Stopped'], - ['Failed'], + self.check_status(SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, 'TrainingJobStatus', self.describe_training_job, training_job_config['TrainingJobName']) @@ -213,8 +215,8 @@ def create_tuning_job(self, tuning_job_config, wait_for_completion=True): if self.use_db_config: if not self.sagemaker_conn_id: raise AirflowException( - "sagemaker connection id must be present to \ - read sagemaker tunning job configuration.") + "SageMaker connection id must be present to \ + read SageMaker tunning job configuration.") sagemaker_conn = self.get_connection(self.sagemaker_conn_id) @@ -226,13 +228,59 @@ def create_tuning_job(self, tuning_job_config, wait_for_completion=True): response = self.conn.create_hyper_parameter_tuning_job( **tuning_job_config) if wait_for_completion: - self.check_status(['InProgress', 'Stopping', 'Stopped'], - ['Failed'], + self.check_status(SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, 'HyperParameterTuningJobStatus', self.describe_tuning_job, tuning_job_config['HyperParameterTuningJobName']) return response + def create_transform_job(self, transform_job_config, wait_for_completion=True): + """ + Create a transform job + :param transform_job_config: the config for transform job + :type transform_job_config: dict + :param wait_for_completion: + if the program should keep running until job finishes + :type wait_for_completion: bool + :return: A dict that contains ARN of the transform job. + """ + if self.use_db_config: + if not self.sagemaker_conn_id: + raise AirflowException( + "SageMaker connection id must be present to \ + read SageMaker transform job configuration.") + + sagemaker_conn = self.get_connection(self.sagemaker_conn_id) + + config = sagemaker_conn.extra_dejson.copy() + transform_job_config.update(config) + + self.check_for_url(transform_job_config + ['TransformInput']['DataSource'] + ['S3DataSource']['S3Uri']) + + response = self.conn.create_transform_job( + **transform_job_config) + if wait_for_completion: + self.check_status(SageMakerHook.non_terminal_states, + SageMakerHook.failed_states, + 'TransformJobStatus', + self.describe_transform_job, + transform_job_config['TransformJobName']) + return response + + def create_model(self, model_config): + """ + Create a model job + :param model_config: the config for model + :type model_config: dict + :return: A dict that contains ARN of the model. + """ + + return self.conn.create_model( + **model_config) + def describe_training_job(self, training_job_name): """ :param training_job_name: the name of the training job @@ -245,7 +293,7 @@ def describe_training_job(self, training_job_name): def describe_tuning_job(self, tuning_job_name): """ - :param tuning_job_name: the name of the training job + :param tuning_job_name: the name of the tuning job :type tuning_job_name: string Return the tuning job info associated with the current job_name :return: A dict contains all the tuning job info @@ -253,3 +301,14 @@ def describe_tuning_job(self, tuning_job_name): return self.conn\ .describe_hyper_parameter_tuning_job( HyperParameterTuningJobName=tuning_job_name) + + def describe_transform_job(self, transform_job_name): + """ + :param transform_job_name: the name of the transform job + :type transform_job_name: string + Return the transform job info associated with the current job_name + :return: A dict contains all the transform job info + """ + return self.conn\ + .describe_transform_job( + TransformJobName=transform_job_name) diff --git a/airflow/contrib/operators/sagemaker_create_training_job_operator.py b/airflow/contrib/operators/sagemaker_create_training_job_operator.py index 409c5f6aa936a..fdd935fc2931b 100644 --- a/airflow/contrib/operators/sagemaker_create_training_job_operator.py +++ b/airflow/contrib/operators/sagemaker_create_training_job_operator.py @@ -50,7 +50,7 @@ class SageMakerCreateTrainingJobOperator(BaseOperator): until training job finishes :type wait_for_completion: bool :param check_interval: if wait is set to be true, this is the time interval - which the operator will check the status of the training job + in seconds which the operator will check the status of the training job :type check_interval: int :param max_ingestion_time: if wait is set to be true, the operator will fail if the training job hasn't finish within the max_ingestion_time diff --git a/airflow/contrib/operators/sagemaker_create_transform_job_operator.py b/airflow/contrib/operators/sagemaker_create_transform_job_operator.py new file mode 100644 index 0000000000000..22c8c2b4ba297 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_create_transform_job_operator.py @@ -0,0 +1,132 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerCreateTransformJobOperator(BaseOperator): + """ + Initiate a SageMaker transform + + This operator returns The ARN of the model created in Amazon SageMaker + + :param sagemaker_conn_id: The SageMaker connection ID to use. + :type sagemaker_conn_id: string + :param transform_job_config: + The configuration necessary to start a transform job (templated) + :type transform_job_config: dict + :param model_config: + The configuration necessary to create a model, the default is none + which means that user should provide a created model in transform_job_config + If given, will be used to create a model before creating transform job + :type model_config: dict + :param use_db_config: Whether or not to use db config + associated with sagemaker_conn_id. + If set to true, will automatically update the transform config + with what's in db, so the db config doesn't need to + included everything, but what's there does replace the ones + in the transform_job_config, so be careful + :type use_db_config: bool + :param region_name: The AWS region_name + :type region_name: string + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: if wait is set to be true, this is the time interval + in seconds which the operator will check the status of the transform job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the transform job hasn't finish within the max_ingestion_time + (Caution: be careful to set this parameters because transform can take very long) + :type max_ingestion_time: int + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: string + + **Example**: + The following operator would start a transform job when executed + + sagemaker_transform = + SageMakerCreateTransformJobOperator( + task_id='sagemaker_transform', + transform_job_config=config_transform, + model_config=config_model, + region_name='us-west-2' + sagemaker_conn_id='sagemaker_customers_conn', + use_db_config=True, + aws_conn_id='aws_customers_conn' + ) + """ + + template_fields = ['transform_job_config'] + template_ext = () + ui_color = '#ededed' + + @apply_defaults + def __init__(self, + sagemaker_conn_id=None, + transform_job_config=None, + model_config=None, + use_db_config=False, + region_name=None, + wait_for_completion=True, + check_interval=2, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerCreateTransformJobOperator, self).__init__(*args, **kwargs) + + self.sagemaker_conn_id = sagemaker_conn_id + self.transform_job_config = transform_job_config + self.model_config = model_config + self.use_db_config = use_db_config + self.region_name = region_name + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + + def execute(self, context): + sagemaker = SageMakerHook( + sagemaker_conn_id=self.sagemaker_conn_id, + use_db_config=self.use_db_config, + region_name=self.region_name, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + + if self.model_config: + self.log.info( + "Creating SageMaker Model %s for transform job" + % self.model_config['ModelName'] + ) + sagemaker.create_model(self.model_config) + + self.log.info( + "Creating SageMaker transform Job %s." + % self.transform_job_config['TransformJobName'] + ) + response = sagemaker.create_transform_job( + self.transform_job_config, + wait_for_completion=self.wait_for_completion) + if not response['ResponseMetadata']['HTTPStatusCode'] \ + == 200: + raise AirflowException( + 'Sagemaker transform Job creation failed: %s' % response) + else: + return response diff --git a/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py b/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py index 0c40a9adc93f4..46ccb2a201144 100644 --- a/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py +++ b/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py @@ -48,7 +48,7 @@ class SageMakerCreateTuningJobOperator(BaseOperator): until tuning job finishes :type wait_for_completion: bool :param check_interval: if wait is set to be true, this is the time interval - which the operator will check the status of the tuning job + in seconds which the operator will check the status of the tuning job :type check_interval: int :param max_ingestion_time: if wait is set to be true, the operator will fail if the tuning job hasn't finish within the max_ingestion_time diff --git a/airflow/contrib/sensors/sagemaker_training_sensor.py b/airflow/contrib/sensors/sagemaker_training_sensor.py index 90c62ce988fbf..449de44c0819c 100644 --- a/airflow/contrib/sensors/sagemaker_training_sensor.py +++ b/airflow/contrib/sensors/sagemaker_training_sensor.py @@ -45,10 +45,10 @@ def __init__(self, self.region_name = region_name def non_terminal_states(self): - return ['InProgress', 'Stopping', 'Stopped'] + return SageMakerHook.non_terminal_states def failed_states(self): - return ['Failed'] + return SageMakerHook.failed_states def get_sagemaker_response(self): sagemaker = SageMakerHook( diff --git a/airflow/contrib/sensors/sagemaker_transform_sensor.py b/airflow/contrib/sensors/sagemaker_transform_sensor.py new file mode 100644 index 0000000000000..68ef1d8dd7b05 --- /dev/null +++ b/airflow/contrib/sensors/sagemaker_transform_sensor.py @@ -0,0 +1,69 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor +from airflow.utils.decorators import apply_defaults + + +class SageMakerTransformSensor(SageMakerBaseSensor): + """ + Asks for the state of the transform state until it reaches a terminal state. + The sensor will error if the job errors, throwing a AirflowException + containing the failure reason. + + :param job_name: job_name of the transform job instance to check the state of + :type job_name: string + :param region_name: The AWS region_name + :type region_name: string + """ + + template_fields = ['job_name'] + template_ext = () + + @apply_defaults + def __init__(self, + job_name, + region_name=None, + *args, + **kwargs): + super(SageMakerTransformSensor, self).__init__(*args, **kwargs) + self.job_name = job_name + self.region_name = region_name + + def non_terminal_states(self): + return SageMakerHook.non_terminal_states + + def failed_states(self): + return SageMakerHook.failed_states + + def get_sagemaker_response(self): + sagemaker = SageMakerHook( + aws_conn_id=self.aws_conn_id, + region_name=self.region_name + ) + + self.log.info('Poking Sagemaker Transform Job %s', self.job_name) + return sagemaker.describe_transform_job(self.job_name) + + def get_failed_reason_from_response(self, response): + return response['FailureReason'] + + def state_from_response(self, response): + return response['TransformJobStatus'] diff --git a/airflow/contrib/sensors/sagemaker_tuning_sensor.py b/airflow/contrib/sensors/sagemaker_tuning_sensor.py index bc74e3a5c5461..1f081100e2c69 100644 --- a/airflow/contrib/sensors/sagemaker_tuning_sensor.py +++ b/airflow/contrib/sensors/sagemaker_tuning_sensor.py @@ -48,10 +48,10 @@ def __init__(self, self.region_name = region_name def non_terminal_states(self): - return ['InProgress', 'Stopping', 'Stopped'] + return SageMakerHook.non_terminal_states def failed_states(self): - return ['Failed'] + return SageMakerHook.failed_states def get_sagemaker_response(self): sagemaker = SageMakerHook( diff --git a/tests/contrib/hooks/test_sagemaker_hook.py b/tests/contrib/hooks/test_sagemaker_hook.py index 8bb56cc8e7d12..3a863b3cb0dc7 100644 --- a/tests/contrib/hooks/test_sagemaker_hook.py +++ b/tests/contrib/hooks/test_sagemaker_hook.py @@ -47,6 +47,8 @@ job_name = 'test-job-name' +model_name = 'test-model-name' + image = 'test-image' test_arn_return = {'TrainingJobArn': 'testarn'} @@ -152,6 +154,38 @@ } } +create_transform_params = \ + { + 'TransformJobName': job_name, + 'ModelName': model_name, + 'BatchStrategy': 'MultiRecord', + 'TransformInput': { + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url + } + } + }, + 'TransformOutput': { + 'S3OutputPath': output_url, + }, + 'TransformResources': { + 'InstanceType': 'ml.m4.xlarge', + 'InstanceCount': 123 + } + } + +create_model_params = \ + { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role + } + db_config = { 'Tags': [ { @@ -393,6 +427,52 @@ def test_create_tuning_job_db_config(self, mock_client, mock_check_tuning): assert_called_once_with(**updated_config) self.assertEqual(response, test_arn_return) + @mock.patch.object(SageMakerHook, 'check_for_url') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_transform_job(self, mock_client, mock_check_url): + mock_check_url.return_value = True + mock_session = mock.Mock() + attrs = {'create_transform_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.create_transform_job(create_transform_params, + wait_for_completion=False) + mock_session.create_transform_job.assert_called_once_with( + **create_transform_params) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'check_for_url') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_transform_job_db_config(self, mock_client, mock_check_url): + mock_check_url.return_value = True + mock_session = mock.Mock() + attrs = {'create_transform_job.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook_use_db_config = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', + use_db_config=True) + response = hook_use_db_config.create_transform_job( + create_transform_params, wait_for_completion=False) + updated_config = copy.deepcopy(create_transform_params) + updated_config.update(db_config) + mock_session.create_transform_job.assert_called_once_with(**updated_config) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_model(self, mock_client): + mock_session = mock.Mock() + attrs = {'create_model.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.create_model(create_model_params) + mock_session.create_model.assert_called_once_with(**create_model_params) + self.assertEqual(response, test_arn_return) + @mock.patch.object(SageMakerHook, 'get_conn') def test_describe_training_job(self, mock_client): mock_session = mock.Mock() @@ -418,6 +498,19 @@ def test_describe_tuning_job(self, mock_client): assert_called_once_with(HyperParameterTuningJobName=job_name) self.assertEqual(response, 'InProgress') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_transform_job(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_transform_job.return_value': + 'InProgress'} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + response = hook.describe_transform_job(job_name) + mock_session.describe_transform_job.\ + assert_called_once_with(TransformJobName=job_name) + self.assertEqual(response, 'InProgress') + if __name__ == '__main__': unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_transform_sensor.py b/tests/contrib/sensors/test_sagemaker_transform_sensor.py new file mode 100644 index 0000000000000..bb4a184bb2797 --- /dev/null +++ b/tests/contrib/sensors/test_sagemaker_transform_sensor.py @@ -0,0 +1,118 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.sensors.sagemaker_transform_sensor \ + import SageMakerTransformSensor +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.exceptions import AirflowException + +DESCRIBE_TRANSFORM_INPROGRESS_RETURN = { + 'TransformJobStatus': 'InProgress', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRANSFORM_COMPELETED_RETURN = { + 'TransformJobStatus': 'Compeleted', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRANSFORM_FAILED_RETURN = { + 'TransformJobStatus': 'Failed', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + }, + 'FailureReason': 'Unknown' +} +DESCRIBE_TRANSFORM_STOPPING_RETURN = { + 'TransformJobStatus': 'Stopping', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_TRANSFORM_STOPPED_RETURN = { + 'TransformJobStatus': 'Stopped', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + + +class TestSageMakerTransformSensor(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'describe_transform_job') + def test_raises_errors_failed_state(self, mock_describe_job, mock_client): + mock_describe_job.side_effect = [DESCRIBE_TRANSFORM_FAILED_RETURN] + sensor = SageMakerTransformSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name' + ) + self.assertRaises(AirflowException, sensor.execute, None) + mock_describe_job.assert_called_once_with('test_job_name') + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, '__init__') + @mock.patch.object(SageMakerHook, 'describe_transform_job') + def test_calls_until_a_terminal_state(self, + mock_describe_job, hook_init, mock_client): + hook_init.return_value = None + + mock_describe_job.side_effect = [ + DESCRIBE_TRANSFORM_INPROGRESS_RETURN, + DESCRIBE_TRANSFORM_STOPPING_RETURN, + DESCRIBE_TRANSFORM_STOPPED_RETURN, + DESCRIBE_TRANSFORM_COMPELETED_RETURN + ] + sensor = SageMakerTransformSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name', + region_name='us-east-1' + ) + + sensor.execute(None) + + # make sure we called 4 times(terminated when its compeleted) + self.assertEqual(mock_describe_job.call_count, 4) + + # make sure the hook was initialized with the specific params + hook_init.assert_called_with(aws_conn_id='aws_test', + region_name='us-east-1') + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/operators/test_sagemaker_create_transform_job_operator.py b/tests/operators/test_sagemaker_create_transform_job_operator.py new file mode 100644 index 0000000000000..a8701530d9daa --- /dev/null +++ b/tests/operators/test_sagemaker_create_transform_job_operator.py @@ -0,0 +1,140 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_create_transform_job_operator \ + import SageMakerCreateTransformJobOperator +from airflow.exceptions import AirflowException + +role = 'test-role' + +bucket = 'test-bucket' + +key = 'test/data' +data_url = 's3://{}/{}'.format(bucket, key) + +job_name = 'test-job-name' + +model_name = 'test-model-name' + +image = 'test-image' + +output_url = 's3://{}/test/output'.format(bucket) + +create_transform_params = \ + { + 'TransformJobName': job_name, + 'ModelName': model_name, + 'BatchStrategy': 'MultiRecord', + 'TransformInput': { + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url + } + } + }, + 'TransformOutput': { + 'S3OutputPath': output_url, + }, + 'TransformResources': { + 'InstanceType': 'ml.m4.xlarge', + 'InstanceCount': 123 + } + } + +create_model_params = \ + { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role + } + + +class TestSageMakertransformOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerCreateTransformJobOperator( + task_id='test_sagemaker_operator', + sagemaker_conn_id='sagemaker_test_id', + transform_job_config=create_transform_params, + model_config=create_model_params, + region_name='us-west-2', + use_db_config=True, + wait_for_completion=False, + check_interval=5 + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_transform_job') + @mock.patch.object(SageMakerHook, '__init__') + def test_hook_init(self, hook_init, mock_transform, mock_model, mock_client): + mock_transform.return_value = {"TransformJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 200}} + hook_init.return_value = None + self.sagemaker.execute(None) + hook_init.assert_called_once_with( + sagemaker_conn_id='sagemaker_test_id', + region_name='us-west-2', + use_db_config=True, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_transform_job') + def test_execute_without_failure(self, mock_transform, mock_model, mock_client): + mock_transform.return_value = {"TransformJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 200}} + self.sagemaker.execute(None) + mock_model.assert_called_once_with(create_model_params) + mock_transform.assert_called_once_with(create_transform_params, + wait_for_completion=False + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_transform_job') + def test_execute_with_failure(self, mock_transform, mock_model, mock_client): + mock_transform.return_value = {"TransformJobArn": "testarn", + "ResponseMetadata": + {"HTTPStatusCode": 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() From dde50e4413c2c8accce75609cfd4cb1fe51dfab0 Mon Sep 17 00:00:00 2001 From: Aishwarya Mohan Date: Thu, 9 Aug 2018 14:42:35 -0700 Subject: [PATCH 055/808] [AIRFLOW-2763] Add check to validate worker connectivity to metadata Database --- airflow/bin/cli.py | 5 ++ airflow/config_templates/default_airflow.cfg | 3 + airflow/config_templates/default_test.cfg | 1 + airflow/settings.py | 22 +++++- tests/cli/test_worker_initialisation.py | 73 ++++++++++++++++++++ 5 files changed, 103 insertions(+), 1 deletion(-) create mode 100644 tests/cli/test_worker_initialisation.py diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index ac52a42664caf..a84ede16fcb5f 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -951,6 +951,11 @@ def worker(args): env = os.environ.copy() env['AIRFLOW_HOME'] = settings.AIRFLOW_HOME + if not settings.validate_session(): + log = LoggingMixin().log + log.error("Worker exiting... database connection precheck failed! ") + sys.exit(1) + # Celery worker from airflow.executors.celery_executor import app as celery_app from celery.bin import worker diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index deb67bb96ca25..7ea81487aa99a 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -173,6 +173,9 @@ killed_task_cleanup_time = 60 # `airflow trigger_dag -c`, the key-value pairs will override the existing ones in params. dag_run_conf_overrides_params = False +# Worker initialisation check to validate Metadata Database connection +worker_precheck = False + [cli] # In what way should the cli access the API. The LocalClient will use the # database directly, while the json_client will use the api running on the diff --git a/airflow/config_templates/default_test.cfg b/airflow/config_templates/default_test.cfg index cd4bd32e68fe9..9dc1ef94ec044 100644 --- a/airflow/config_templates/default_test.cfg +++ b/airflow/config_templates/default_test.cfg @@ -51,6 +51,7 @@ enable_xcom_pickling = False killed_task_cleanup_time = 5 secure_mode = False hostname_callable = socket:getfqdn +worker_precheck = False [cli] api_client = airflow.api.client.local_client diff --git a/airflow/settings.py b/airflow/settings.py index 7c0376d12f9b5..984c52ea5d206 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -27,7 +27,7 @@ import os import pendulum -from sqlalchemy import create_engine +from sqlalchemy import create_engine, exc from sqlalchemy.orm import scoped_session, sessionmaker from sqlalchemy.pool import NullPool @@ -211,6 +211,26 @@ def configure_adapters(): pass +def validate_session(): + try: + worker_precheck = conf.getboolean('core', 'worker_precheck') + except conf.AirflowConfigException: + worker_precheck = False + if not worker_precheck: + return True + else: + check_session = sessionmaker(bind=engine) + session = check_session() + try: + session.execute("select 1") + conn_status = True + except exc.DBAPIError as err: + log.error(err) + conn_status = False + session.close() + return conn_status + + def configure_action_logging(): """ Any additional configuration (register callback) for airflow.utils.action_loggers diff --git a/tests/cli/test_worker_initialisation.py b/tests/cli/test_worker_initialisation.py new file mode 100644 index 0000000000000..477221693abe1 --- /dev/null +++ b/tests/cli/test_worker_initialisation.py @@ -0,0 +1,73 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +import sqlalchemy +import airflow +from argparse import Namespace + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None +from mock import patch + +patch('airflow.utils.cli.action_logging', lambda x: x).start() +from airflow.bin import cli # noqa +mock_args = Namespace(queues=1, concurrency=1) + + +class TestWorkerPrecheck(unittest.TestCase): + + def setUp(self): + airflow.configuration.load_test_config() + + @mock.patch('airflow.settings.validate_session') + def test_error(self, mock_validate_session): + """ + Test to verify the exit mechanism of airflow-worker cli + by mocking validate_session method + """ + mock_validate_session.return_value = False + with self.assertRaises(SystemExit) as cm: + # airflow.bin.cli.worker(mock_args) + cli.worker(mock_args) + self.assertEqual(cm.exception.code, 1) + + @mock.patch('airflow.configuration.getboolean') + def test_worker_precheck_exception(self, mock_getboolean): + """ + Test to check the behaviour of validate_session method + when worker_precheck is absent in airflow configuration + """ + mock_getboolean.side_effect = airflow.configuration.AirflowConfigException + self.assertEqual(airflow.settings.validate_session(), True) + + @mock.patch('sqlalchemy.orm.session.Session.execute') + @mock.patch('airflow.configuration.getboolean') + def test_validate_session_dbapi_exception(self, mock_getboolean, mock_session): + """ + Test to validate connection failure scenario on SELECT 1 query + """ + mock_getboolean.return_value = True + mock_session.side_effect = sqlalchemy.exc.OperationalError("m1", "m2", "m3", "m4") + self.assertEquals(airflow.settings.validate_session(), False) From 97b63bc6845014fbecc1062d9b3852fc8de06e2f Mon Sep 17 00:00:00 2001 From: Yuliya Volkova Date: Fri, 10 Aug 2018 05:43:50 -0500 Subject: [PATCH 056/808] [AIRFLOW-1874] use_legacy_sql added to BigQueryCheck operators (#3717) --- .../operators/bigquery_check_operator.py | 50 ++++++++++++------- 1 file changed, 32 insertions(+), 18 deletions(-) diff --git a/airflow/contrib/operators/bigquery_check_operator.py b/airflow/contrib/operators/bigquery_check_operator.py index 59ef5d377d182..a9c493f4fd418 100644 --- a/airflow/contrib/operators/bigquery_check_operator.py +++ b/airflow/contrib/operators/bigquery_check_operator.py @@ -55,21 +55,25 @@ class BigQueryCheckOperator(CheckOperator): :type sql: string :param bigquery_conn_id: reference to the BigQuery database :type bigquery_conn_id: string + :param use_legacy_sql: Whether to use legacy SQL (true) + or standard SQL (false). + :type use_legacy_sql: boolean """ @apply_defaults - def __init__( - self, - sql, - bigquery_conn_id='bigquery_default', - *args, - **kwargs): + def __init__(self, + sql, + bigquery_conn_id='bigquery_default', + use_legacy_sql=True, + *args, **kwargs): super(BigQueryCheckOperator, self).__init__(sql=sql, *args, **kwargs) self.bigquery_conn_id = bigquery_conn_id self.sql = sql + self.use_legacy_sql = use_legacy_sql def get_db_hook(self): - return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id) + return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, + use_legacy_sql=self.use_legacy_sql) class BigQueryValueCheckOperator(ValueCheckOperator): @@ -78,20 +82,27 @@ class BigQueryValueCheckOperator(ValueCheckOperator): :param sql: the sql to be executed :type sql: string + :param use_legacy_sql: Whether to use legacy SQL (true) + or standard SQL (false). + :type use_legacy_sql: boolean """ @apply_defaults - def __init__( - self, sql, pass_value, tolerance=None, - bigquery_conn_id='bigquery_default', - *args, **kwargs): + def __init__(self, sql, + pass_value, + tolerance=None, + bigquery_conn_id='bigquery_default', + use_legacy_sql=True, + *args, **kwargs): super(BigQueryValueCheckOperator, self).__init__( sql=sql, pass_value=pass_value, tolerance=tolerance, *args, **kwargs) self.bigquery_conn_id = bigquery_conn_id + self.use_legacy_sql = use_legacy_sql def get_db_hook(self): - return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id) + return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, + use_legacy_sql=self.use_legacy_sql) class BigQueryIntervalCheckOperator(IntervalCheckOperator): @@ -113,19 +124,22 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator): example 'COUNT(*)': 1.5 would require a 50 percent or less difference between the current day, and the prior days_back. :type metrics_threshold: dict + :param use_legacy_sql: Whether to use legacy SQL (true) + or standard SQL (false). + :type use_legacy_sql: boolean """ @apply_defaults - def __init__( - self, table, metrics_thresholds, - date_filter_column='ds', days_back=-7, - bigquery_conn_id='bigquery_default', - *args, **kwargs): + def __init__(self, table, metrics_thresholds, date_filter_column='ds', + days_back=-7, bigquery_conn_id='bigquery_default', + use_legacy_sql=True, *args, **kwargs): super(BigQueryIntervalCheckOperator, self).__init__( table=table, metrics_thresholds=metrics_thresholds, date_filter_column=date_filter_column, days_back=days_back, *args, **kwargs) self.bigquery_conn_id = bigquery_conn_id + self.use_legacy_sql = use_legacy_sql def get_db_hook(self): - return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id) + return BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, + use_legacy_sql=self.use_legacy_sql) From c4b9ba97eabcf813786535cc4de1bda6d0760375 Mon Sep 17 00:00:00 2001 From: Cameron Moberg Date: Fri, 10 Aug 2018 09:25:33 -0700 Subject: [PATCH 057/808] [AIRFLOW-2786] Gracefully handle Variable import errors (#3648) Variables that are added through a file are not checked as explicity as creating a Variable in the web UI. This handles exceptions that could be caused by improper keys or values. --- airflow/www/utils.py | 9 +++++-- airflow/www/views.py | 15 +++++++++-- airflow/www_rbac/utils.py | 9 +++++-- airflow/www_rbac/views.py | 13 +++++++-- tests/www/test_utils.py | 4 +++ tests/www/test_views.py | 52 +++++++++++++++++++++++++++--------- tests/www_rbac/test_utils.py | 4 +++ tests/www_rbac/test_views.py | 20 ++++++++++++++ 8 files changed, 105 insertions(+), 21 deletions(-) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 09ac465c52d1b..03ba1f775b001 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -58,8 +58,13 @@ def should_hide_value_for_key(key_name): - return any(s in key_name.lower() for s in DEFAULT_SENSITIVE_VARIABLE_FIELDS) \ - and configuration.conf.getboolean('admin', 'hide_sensitive_variable_fields') + # It is possible via importing variables from file that a key is empty. + if key_name: + config_set = configuration.conf.getboolean('admin', + 'hide_sensitive_variable_fields') + field_comp = any(s in key_name.lower() for s in DEFAULT_SENSITIVE_VARIABLE_FIELDS) + return config_set and field_comp + return False class LoginMixin(object): diff --git a/airflow/www/views.py b/airflow/www/views.py index 851ad90cb2770..ccbaafbbeb5a5 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1903,9 +1903,20 @@ def varimport(self): except Exception as e: flash("Missing file or syntax error: {}.".format(e)) else: + suc_count = fail_count = 0 for k, v in d.items(): - models.Variable.set(k, v, serialize_json=isinstance(v, dict)) - flash("{} variable(s) successfully updated.".format(len(d))) + try: + models.Variable.set(k, v, serialize_json=isinstance(v, dict)) + except Exception as e: + logging.info('Variable import failed: {}'.format(repr(e))) + fail_count += 1 + else: + suc_count += 1 + flash("{} variable(s) successfully updated.".format(suc_count), 'info') + if fail_count: + flash( + "{} variables(s) failed to be updated.".format(fail_count), 'error') + return redirect('/admin/variable') diff --git a/airflow/www_rbac/utils.py b/airflow/www_rbac/utils.py index 14e18f8bc30e7..0176a5312c373 100644 --- a/airflow/www_rbac/utils.py +++ b/airflow/www_rbac/utils.py @@ -58,8 +58,13 @@ def should_hide_value_for_key(key_name): - return any(s in key_name.lower() for s in DEFAULT_SENSITIVE_VARIABLE_FIELDS) \ - and configuration.getboolean('admin', 'hide_sensitive_variable_fields') + # It is possible via importing variables from file that a key is empty. + if key_name: + config_set = configuration.conf.getboolean('admin', + 'hide_sensitive_variable_fields') + field_comp = any(s in key_name.lower() for s in DEFAULT_SENSITIVE_VARIABLE_FIELDS) + return config_set and field_comp + return False def get_params(**kwargs): diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index a65e1d26e3995..34bdbc6ecb09a 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -1866,9 +1866,18 @@ def varimport(self): except Exception: flash("Missing file or syntax error.") else: + suc_count = fail_count = 0 for k, v in d.items(): - models.Variable.set(k, v, serialize_json=isinstance(v, dict)) - flash("{} variable(s) successfully updated.".format(len(d))) + try: + models.Variable.set(k, v, serialize_json=isinstance(v, dict)) + except Exception as e: + logging.info('Variable import failed: {}'.format(repr(e))) + fail_count += 1 + else: + suc_count += 1 + flash("{} variable(s) successfully updated.".format(suc_count), 'info') + if fail_count: + flash("{} variables(s) failed to be updated.".format(fail_count), 'error') self.update_redirect() return redirect(self.get_redirect()) diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py index f5c3f01fb916d..904853bac156d 100644 --- a/tests/www/test_utils.py +++ b/tests/www/test_utils.py @@ -31,6 +31,10 @@ class UtilsTest(unittest.TestCase): def setUp(self): super(UtilsTest, self).setUp() + def test_empty_variable_should_not_be_hidden(self): + self.assertFalse(utils.should_hide_value_for_key("")) + self.assertFalse(utils.should_hide_value_for_key(None)) + def test_normal_variable_should_not_be_hidden(self): self.assertFalse(utils.should_hide_value_for_key("key")) diff --git a/tests/www/test_views.py b/tests/www/test_views.py index f59470ea3de11..bb183d4a649b7 100644 --- a/tests/www/test_views.py +++ b/tests/www/test_views.py @@ -20,6 +20,7 @@ import io import copy import logging.config +import mock import os import shutil import tempfile @@ -450,6 +451,28 @@ def tearDownClass(cls): session.close() super(TestVarImportView, cls).tearDownClass() + def test_import_variable_fail(self): + with mock.patch('airflow.models.Variable.set') as set_mock: + set_mock.side_effect = UnicodeEncodeError + content = '{"fail_key": "fail_val"}' + + try: + # python 3+ + bytes_content = io.BytesIO(bytes(content, encoding='utf-8')) + except TypeError: + # python 2.7 + bytes_content = io.BytesIO(bytes(content)) + response = self.app.post( + self.IMPORT_ENDPOINT, + data={'file': (bytes_content, 'test.json')}, + follow_redirects=True + ) + self.assertEqual(response.status_code, 200) + session = Session() + db_dict = {x.key: x.get_val() for x in session.query(models.Variable).all()} + session.close() + self.assertNotIn('fail_key', db_dict) + def test_import_variables(self): content = ('{"str_key": "str_value", "int_key": 60,' '"list_key": [1, 2], "dict_key": {"k_a": 2, "k_b": 3}}') @@ -465,21 +488,24 @@ def test_import_variables(self): follow_redirects=True ) self.assertEqual(response.status_code, 200) - body = response.data.decode('utf-8') - self.assertIn('str_key', body) - self.assertIn('int_key', body) - self.assertIn('list_key', body) - self.assertIn('dict_key', body) - self.assertIn('str_value', body) - self.assertIn('60', body) - self.assertIn('[1, 2]', body) - # As dicts are not ordered, we may get any of the following cases. - case_a_dict = '{"k_a": 2, "k_b": 3}' - case_b_dict = '{"k_b": 3, "k_a": 2}' + session = Session() + # Extract values from Variable + db_dict = {x.key: x.get_val() for x in session.query(models.Variable).all()} + session.close() + self.assertIn('str_key', db_dict) + self.assertIn('int_key', db_dict) + self.assertIn('list_key', db_dict) + self.assertIn('dict_key', db_dict) + self.assertEquals('str_value', db_dict['str_key']) + self.assertEquals('60', db_dict['int_key']) + self.assertEquals('[1, 2]', db_dict['list_key']) + + case_a_dict = '{"k_a": 2, "k_b": 3}' + case_b_dict = '{"k_b": 3, "k_a": 2}' try: - self.assertIn(case_a_dict, body) + self.assertEquals(case_a_dict, db_dict['dict_key']) except AssertionError: - self.assertIn(case_b_dict, body) + self.assertEquals(case_b_dict, db_dict['dict_key']) class TestMountPoint(unittest.TestCase): diff --git a/tests/www_rbac/test_utils.py b/tests/www_rbac/test_utils.py index c7179fa3a421d..68d1744ab8fa8 100644 --- a/tests/www_rbac/test_utils.py +++ b/tests/www_rbac/test_utils.py @@ -29,6 +29,10 @@ class UtilsTest(unittest.TestCase): def setUp(self): super(UtilsTest, self).setUp() + def test_empty_variable_should_not_be_hidden(self): + self.assertFalse(utils.should_hide_value_for_key("")) + self.assertFalse(utils.should_hide_value_for_key(None)) + def test_normal_variable_should_not_be_hidden(self): self.assertFalse(utils.should_hide_value_for_key("key")) diff --git a/tests/www_rbac/test_views.py b/tests/www_rbac/test_views.py index 71dcab0d073a6..0a4eb7c03d16c 100644 --- a/tests/www_rbac/test_views.py +++ b/tests/www_rbac/test_views.py @@ -21,6 +21,7 @@ import io import json import logging.config +import mock import os import shutil import sys @@ -168,6 +169,25 @@ def test_xss_prevention(self): self.assertNotIn("", resp.data.decode("utf-8")) + def test_import_variables(self): + content = '{"str_key": "str_value"}' + + with mock.patch('airflow.models.Variable.set') as set_mock: + set_mock.side_effect = UnicodeEncodeError + self.assertEqual(self.session.query(models.Variable).count(), 0) + + try: + # python 3+ + bytes_content = io.BytesIO(bytes(content, encoding='utf-8')) + except TypeError: + # python 2.7 + bytes_content = io.BytesIO(bytes(content)) + + resp = self.client.post('/variable/varimport', + data={'file': (bytes_content, 'test.json')}, + follow_redirects=True) + self.check_content_in_response('1 variable(s) failed to be updated.', resp) + def test_import_variables(self): self.assertEqual(self.session.query(models.Variable).count(), 0) From 26f97c230e87b58dc3616086187228a6d83015ca Mon Sep 17 00:00:00 2001 From: awelsh93 <32643586+awelsh93@users.noreply.github.com> Date: Mon, 13 Aug 2018 17:42:04 +0100 Subject: [PATCH 058/808] [AIRFLOW-2860] DruidHook: time check is wrong (#3745) --- airflow/hooks/druid_hook.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py index ef4f2338d29dc..50cee8a2259b1 100644 --- a/airflow/hooks/druid_hook.py +++ b/airflow/hooks/druid_hook.py @@ -81,8 +81,6 @@ def submit_indexing_job(self, json_index_spec): self.log.info("Job still running for %s seconds...", sec) - sec = sec + 1 - if self.max_ingestion_time and sec > self.max_ingestion_time: # ensure that the job gets killed if the max ingestion time is exceeded requests.post("{0}/{1}/shutdown".format(url, druid_task_id)) @@ -91,6 +89,8 @@ def submit_indexing_job(self, json_index_spec): time.sleep(self.timeout) + sec = sec + self.timeout + status = req_status.json()['status']['status'] if status == 'RUNNING': running = True From 7bbde14655506d5041271db704b78fd7131b2bb6 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 23 Jul 2018 13:09:14 +0200 Subject: [PATCH 059/808] [AIRFLOW-2773] Validates Dataflow Job Name Closes #3623 from kaxil/AIRFLOW-2773 --- airflow/contrib/hooks/gcp_dataflow_hook.py | 35 +++++---- tests/contrib/hooks/test_gcp_dataflow_hook.py | 71 ++++++++++++++++++- 2 files changed, 91 insertions(+), 15 deletions(-) diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index 7abb413a1afcf..f24bdce5986bc 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -17,6 +17,7 @@ # specific language governing permissions and limitations # under the License. import json +import re import select import subprocess import time @@ -166,7 +167,7 @@ def __init__(self, def get_conn(self): """ - Returns a Google Cloud Storage service object. + Returns a Google Cloud Dataflow service object. """ http_authorized = self._authorize() return build( @@ -191,10 +192,7 @@ def _set_variables(variables): def start_java_dataflow(self, task_id, variables, dataflow, job_class=None, append_job_name=True): - if append_job_name: - name = task_id + "-" + str(uuid.uuid1())[:8] - else: - name = task_id + name = self._build_dataflow_job_name(task_id, append_job_name) variables['jobName'] = name def label_formatter(labels_dict): @@ -207,19 +205,13 @@ def label_formatter(labels_dict): def start_template_dataflow(self, task_id, variables, parameters, dataflow_template, append_job_name=True): - if append_job_name: - name = task_id + "-" + str(uuid.uuid1())[:8] - else: - name = task_id + name = self._build_dataflow_job_name(task_id, append_job_name) self._start_template_dataflow( name, variables, parameters, dataflow_template) def start_python_dataflow(self, task_id, variables, dataflow, py_options, append_job_name=True): - if append_job_name: - name = task_id + "-" + str(uuid.uuid1())[:8] - else: - name = task_id + name = self._build_dataflow_job_name(task_id, append_job_name) variables['job_name'] = name def label_formatter(labels_dict): @@ -229,6 +221,23 @@ def label_formatter(labels_dict): ["python"] + py_options + [dataflow], label_formatter) + @staticmethod + def _build_dataflow_job_name(task_id, append_job_name=True): + task_id = str(task_id).replace('_', '-') + + assert re.match(r"^[a-z]([-a-z0-9]*[a-z0-9])?$", task_id), \ + 'Invalid job_name ({}); the name must consist of ' \ + 'only the characters [-a-z0-9], starting with a ' \ + 'letter and ending with a letter or number '.format( + task_id) + + if append_job_name: + job_name = task_id + "-" + str(uuid.uuid1())[:8] + else: + job_name = task_id + + return job_name + def _build_cmd(self, task_id, variables, label_formatter): command = ["--runner=DataflowRunner"] if variables is not None: diff --git a/tests/contrib/hooks/test_gcp_dataflow_hook.py b/tests/contrib/hooks/test_gcp_dataflow_hook.py index f16dcdfcbc1e1..90714c6ee4f62 100644 --- a/tests/contrib/hooks/test_gcp_dataflow_hook.py +++ b/tests/contrib/hooks/test_gcp_dataflow_hook.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -172,6 +172,73 @@ def poll_resp_error(): self.assertRaises(Exception, dataflow.wait_for_done) mock_logging.warning.assert_has_calls([call('test'), call('error')]) + def test_valid_dataflow_job_name(self): + job_name = self.dataflow_hook._build_dataflow_job_name( + task_id=TASK_ID, append_job_name=False + ) + + self.assertEquals(job_name, TASK_ID) + + def test_fix_underscore_in_task_id(self): + task_id_with_underscore = 'test_example' + fixed_job_name = task_id_with_underscore.replace( + '_', '-' + ) + job_name = self.dataflow_hook._build_dataflow_job_name( + task_id=task_id_with_underscore, append_job_name=False + ) + + self.assertEquals(job_name, fixed_job_name) + + def test_invalid_dataflow_job_name(self): + invalid_job_name = '9test_invalid_name' + fixed_name = invalid_job_name.replace( + '_', '-') + + with self.assertRaises(AssertionError) as e: + self.dataflow_hook._build_dataflow_job_name( + task_id=invalid_job_name, append_job_name=False + ) + # Test whether the job_name is present in the Error msg + self.assertIn('Invalid job_name ({})'.format(fixed_name), + str(e.exception)) + + def test_dataflow_job_regex_check(self): + + self.assertEquals(self.dataflow_hook._build_dataflow_job_name( + task_id='df-job-1', append_job_name=False + ), 'df-job-1') + + self.assertEquals(self.dataflow_hook._build_dataflow_job_name( + task_id='df-job', append_job_name=False + ), 'df-job') + + self.assertEquals(self.dataflow_hook._build_dataflow_job_name( + task_id='dfjob', append_job_name=False + ), 'dfjob') + + self.assertEquals(self.dataflow_hook._build_dataflow_job_name( + task_id='dfjob1', append_job_name=False + ), 'dfjob1') + + self.assertRaises( + AssertionError, + self.dataflow_hook._build_dataflow_job_name, + task_id='1dfjob', append_job_name=False + ) + + self.assertRaises( + AssertionError, + self.dataflow_hook._build_dataflow_job_name, + task_id='dfjob@', append_job_name=False + ) + + self.assertRaises( + AssertionError, + self.dataflow_hook._build_dataflow_job_name, + task_id='df^jo', append_job_name=False + ) + class DataFlowTemplateHookTest(unittest.TestCase): From b1041cf7d93db64345bc86228afa368f27f93a17 Mon Sep 17 00:00:00 2001 From: Paul Woods Date: Sat, 21 Jul 2018 13:04:23 +0100 Subject: [PATCH 060/808] [AIRFLOW-2769] Increase num_retries polling value on Dataflow hook Closes #3617 from pwoods25443/2769-dataflow-num- retries --- airflow/contrib/hooks/gcp_dataflow_hook.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index f24bdce5986bc..279b9dd21a862 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -47,7 +47,7 @@ def _get_job_id_from_name(self): jobs = self._dataflow.projects().locations().jobs().list( projectId=self._project_number, location=self._job_location - ).execute() + ).execute(num_retries=5) for job in jobs['jobs']: if job['name'] == self._job_name: self._job_id = job['id'] @@ -61,7 +61,7 @@ def _get_job(self): job = self._dataflow.projects().jobs().get( projectId=self._project_number, jobId=self._job_id - ).execute() + ).execute(num_retries=5) if job and 'currentState' in job: self.log.info( From 133c02cbd571e36f0c89f150c377ba23f13a5870 Mon Sep 17 00:00:00 2001 From: Ivan Arozamena Date: Tue, 17 Jul 2018 13:56:05 +0100 Subject: [PATCH 061/808] [AIRFLOW-2749] Add feature to delete BQ Dataset Closes #3598 from MENA1717/Add-bq-op --- airflow/contrib/hooks/bigquery_hook.py | 29 +++++++++++ .../contrib/operators/bigquery_operator.py | 52 +++++++++++++++++++ docs/code.rst | 1 + docs/integration.rst | 14 +++-- .../operators/test_bigquery_operator.py | 22 +++++++- 5 files changed, 114 insertions(+), 4 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index b452f51c22c69..90513d81fa25d 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -1255,6 +1255,35 @@ def run_grant_dataset_view_access(self, source_dataset) return source_dataset_resource + def delete_dataset(self, + project_id, + dataset_id + ): + """ + Delete a dataset of Big query in your project. + :param project_id: The name of the project where we have the dataset . + :type project_id: str + :param dataset_id: The dataset to be delete. + :type dataset_id: str + :return: + """ + project_id = project_id if project_id is not None else self.project_id + self.log.info('Deleting from project: %s Dataset:%s', + project_id, dataset_id) + + try: + self.service.datasets().delete( + projectId=project_id, + datasetId=dataset_id).execute() + + self.log.info('Dataset deleted successfully: In project %s Dataset %s', + project_id, dataset_id) + + except HttpError as err: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(err.content) + ) + class BigQueryCursor(BigQueryBaseCursor): """ diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index b36efbd6bfd43..ca2efb55abad9 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -481,3 +481,55 @@ def execute(self, context): allow_jagged_rows=self.allow_jagged_rows, src_fmt_configs=self.src_fmt_configs ) + + +class BigQueryDeleteDatasetOperator(BaseOperator): + """" + This operator deletes an existing dataset from your Project in Big query. + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/delete + :param project_id: The project id of the dataset. + :type project_id: string + :param dataset_id: The dataset to be deleted. + :type dataset_id: string + + **Example**: :: + + delete_temp_data = BigQueryDeleteDatasetOperator( + dataset_id = 'temp-dataset', + project_id = 'temp-project', + bigquery_conn_id='_my_gcp_conn_', + task_id='Deletetemp', + dag=dag) + """ + + template_fields = ('dataset_id', 'project_id') + ui_color = '#f00004' + + @apply_defaults + def __init__(self, + dataset_id, + project_id=None, + bigquery_conn_id='bigquery_default', + delegate_to=None, + *args, **kwargs): + self.dataset_id = dataset_id + self.project_id = project_id + self.bigquery_conn_id = bigquery_conn_id + self.delegate_to = delegate_to + + self.log.info('Dataset id: %s', self.dataset_id) + self.log.info('Project id: %s', self.project_id) + + super(BigQueryDeleteDatasetOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, + delegate_to=self.delegate_to) + + conn = bq_hook.get_conn() + cursor = conn.cursor() + + cursor.delete_dataset( + project_id=self.project_id, + dataset_id=self.dataset_id + ) diff --git a/docs/code.rst b/docs/code.rst index f055fc60cffe6..521b55f9d41ce 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -117,6 +117,7 @@ Operators .. autoclass:: airflow.contrib.operators.bigquery_get_data.BigQueryGetDataOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyTableOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryOperator .. autoclass:: airflow.contrib.operators.bigquery_table_delete_operator.BigQueryTableDeleteOperator .. autoclass:: airflow.contrib.operators.bigquery_to_bigquery.BigQueryToBigQueryOperator diff --git a/docs/integration.rst b/docs/integration.rst index 660b2163d9849..99dbafbd2bc58 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -118,9 +118,9 @@ WasbHook Azure File Share '''''''''''''''' -Cloud variant of a SMB file share. Make sure that a Airflow connection of -type `wasb` exists. Authorization can be done by supplying a login (=Storage account name) -and password (=Storage account key), or login and SAS token in the extra field +Cloud variant of a SMB file share. Make sure that a Airflow connection of +type `wasb` exists. Authorization can be done by supplying a login (=Storage account name) +and password (=Storage account key), or login and SAS token in the extra field (see connection `wasb_default` for an example). AzureFileShareHook @@ -349,6 +349,7 @@ BigQuery Operators - :ref:`BigQueryIntervalCheckOperator` : Checks that the values of metrics given as SQL expressions are within a certain tolerance of the ones from days_back before. - :ref:`BigQueryCreateEmptyTableOperator` : Creates a new, empty table in the specified BigQuery dataset optionally with schema. - :ref:`BigQueryCreateExternalTableOperator` : Creates a new, external table in the dataset with the data in Google Cloud Storage. +- :ref:`BigQueryDeleteDatasetOperator` : Deletes an existing BigQuery dataset. - :ref:`BigQueryOperator` : Executes BigQuery SQL queries in a specific BigQuery database. - :ref:`BigQueryToBigQueryOperator` : Copy a BigQuery table to another BigQuery table. - :ref:`BigQueryToCloudStorageOperator` : Transfers a BigQuery table to a Google Cloud Storage bucket @@ -396,6 +397,13 @@ BigQueryCreateExternalTableOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator +.. _BigQueryDeleteDatasetOperator: + +BigQueryDeleteDatasetOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator + .. _BigQueryOperator: BigQueryOperator diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 6a51d0c46d294..21a42d3ef8f46 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -23,7 +23,7 @@ from airflow.contrib.operators.bigquery_operator import \ BigQueryCreateExternalTableOperator, \ BigQueryOperator, \ - BigQueryCreateEmptyTableOperator + BigQueryCreateEmptyTableOperator, BigQueryDeleteDatasetOperator try: from unittest import mock @@ -114,3 +114,23 @@ def test_execute(self, mock_hook): allow_jagged_rows=False, src_fmt_configs={} ) + + +class BigQueryDeleteDatasetOperatorTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') + def test_execute(self, mock_hook): + operator = BigQueryDeleteDatasetOperator( + task_id=TASK_ID, + dataset_id=TEST_DATASET, + project_id=TEST_PROJECT_ID + ) + + operator.execute(None) + mock_hook.return_value \ + .get_conn() \ + .cursor() \ + .delete_dataset \ + .assert_called_once_with( + dataset_id=TEST_DATASET, + project_id=TEST_PROJECT_ID + ) From 77aa239f982a38f4b5e1fa883868d99bb394cd0f Mon Sep 17 00:00:00 2001 From: Tomas Jansson Date: Tue, 17 Jul 2018 13:59:41 +0100 Subject: [PATCH 062/808] [AIRFLOW-2704] Add support for labels in the bigquery_operator [AIRFLOW-2704]Add support for labels in the bigquery_operator Adds support for bigquery labels in the bigquery operator and hook. Make labels template fields Closes #3573 from mastoj/AIRFLOW-2704 --- airflow/contrib/hooks/bigquery_hook.py | 41 +++++++++++++++++-- .../contrib/operators/bigquery_operator.py | 27 +++++++++--- .../contrib/operators/bigquery_to_bigquery.py | 10 ++++- airflow/contrib/operators/bigquery_to_gcs.py | 11 ++++- tests/contrib/hooks/test_bigquery_hook.py | 27 +++++++++++- .../operators/test_bigquery_operator.py | 6 ++- 6 files changed, 105 insertions(+), 17 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 90513d81fa25d..f4c1a3b520217 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -206,7 +206,8 @@ def create_empty_table(self, dataset_id, table_id, schema_fields=None, - time_partitioning={} + time_partitioning={}, + labels=None ): """ Creates a new, empty table in the dataset. @@ -219,6 +220,8 @@ def create_empty_table(self, :type table_id: str :param schema_fields: If set, the schema field list as defined here: https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema + :param labels: a dictionary containing labels for the table, passed to BigQuery + :type labels: dict **Example**: :: @@ -249,6 +252,9 @@ def create_empty_table(self, if time_partitioning: table_resource['timePartitioning'] = time_partitioning + if labels: + table_resource['labels'] = labels + self.log.info('Creating Table %s:%s.%s', project_id, dataset_id, table_id) @@ -280,7 +286,8 @@ def create_external_table(self, quote_character=None, allow_quoted_newlines=False, allow_jagged_rows=False, - src_fmt_configs={} + src_fmt_configs={}, + labels=None ): """ Creates a new external table in the dataset with the data in Google @@ -341,6 +348,8 @@ def create_external_table(self, :type allow_jagged_rows: bool :param src_fmt_configs: configure optional fields specific to the source format :type src_fmt_configs: dict + :param labels: a dictionary containing labels for the table, passed to BigQuery + :type labels: dict """ project_id, dataset_id, external_table_id = \ @@ -439,6 +448,9 @@ def create_external_table(self, table_resource['externalDataConfiguration'][src_fmt_to_param_mapping[ source_format]] = src_fmt_configs + if labels: + table_resource['labels'] = labels + try: self.service.tables().insert( projectId=project_id, @@ -467,6 +479,7 @@ def run_query(self, maximum_bytes_billed=None, create_disposition='CREATE_IF_NEEDED', query_params=None, + labels=None, schema_update_options=(), priority='INTERACTIVE', time_partitioning={}): @@ -516,6 +529,9 @@ def run_query(self, :param query_params a dictionary containing query parameter types and values, passed to BigQuery :type query_params: dict + :param labels a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict :param schema_update_options: Allows the schema of the desitination table to be updated as a side effect of the query job. :type schema_update_options: tuple @@ -606,6 +622,9 @@ def run_query(self, else: configuration['query']['queryParameters'] = query_params + if labels: + configuration['labels'] = labels + time_partitioning = _cleanse_time_partitioning( destination_dataset_table, time_partitioning @@ -636,7 +655,8 @@ def run_extract( # noqa compression='NONE', export_format='CSV', field_delimiter=',', - print_header=True): + print_header=True, + labels=None): """ Executes a BigQuery extract command to copy data from BigQuery to Google Cloud Storage. See here: @@ -661,6 +681,9 @@ def run_extract( # noqa :type field_delimiter: string :param print_header: Whether to print a header for a CSV file extract. :type print_header: boolean + :param labels: a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict """ source_project, source_dataset, source_table = \ @@ -681,6 +704,9 @@ def run_extract( # noqa } } + if labels: + configuration['labels'] = labels + if export_format == 'CSV': # Only set fieldDelimiter and printHeader fields if using CSV. # Google does not like it if you set these fields for other export @@ -694,7 +720,8 @@ def run_copy(self, source_project_dataset_tables, destination_project_dataset_table, write_disposition='WRITE_EMPTY', - create_disposition='CREATE_IF_NEEDED'): + create_disposition='CREATE_IF_NEEDED', + labels=None): """ Executes a BigQuery copy command to copy data from one BigQuery table to another. See here: @@ -717,6 +744,9 @@ def run_copy(self, :type write_disposition: string :param create_disposition: The create disposition if the table doesn't exist. :type create_disposition: string + :param labels a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict """ source_project_dataset_tables = ([ source_project_dataset_tables @@ -754,6 +784,9 @@ def run_copy(self, } } + if labels: + configuration['labels'] = labels + return self.run_with_configuration(configuration) def run_load(self, diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index ca2efb55abad9..09f442ae7b1a6 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -81,6 +81,9 @@ class BigQueryOperator(BaseOperator): :param query_params: a dictionary containing query parameter types and values, passed to BigQuery. :type query_params: dict + :param labels: a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict :param priority: Specifies a priority for the query. Possible values include INTERACTIVE and BATCH. The default value is INTERACTIVE. @@ -92,7 +95,7 @@ class BigQueryOperator(BaseOperator): :type time_partitioning: dict """ - template_fields = ('bql', 'sql', 'destination_dataset_table') + template_fields = ('bql', 'sql', 'destination_dataset_table', 'labels') template_ext = ('.sql', ) ui_color = '#e4f0e8' @@ -113,6 +116,7 @@ def __init__(self, create_disposition='CREATE_IF_NEEDED', schema_update_options=(), query_params=None, + labels=None, priority='INTERACTIVE', time_partitioning={}, *args, @@ -133,6 +137,7 @@ def __init__(self, self.maximum_bytes_billed = maximum_bytes_billed self.schema_update_options = schema_update_options self.query_params = query_params + self.labels = labels self.bq_cursor = None self.priority = priority self.time_partitioning = time_partitioning @@ -171,6 +176,7 @@ def execute(self, context): maximum_bytes_billed=self.maximum_bytes_billed, create_disposition=self.create_disposition, query_params=self.query_params, + labels=self.labels, schema_update_options=self.schema_update_options, priority=self.priority, time_partitioning=self.time_partitioning @@ -228,6 +234,8 @@ class BigQueryCreateEmptyTableOperator(BaseOperator): work, the service account making the request must have domain-wide delegation enabled. :type delegate_to: string + :param labels a dictionary containing labels for the table, passed to BigQuery + :type labels: dict **Example (with schema JSON in GCS)**: :: @@ -270,7 +278,8 @@ class BigQueryCreateEmptyTableOperator(BaseOperator): ) """ - template_fields = ('dataset_id', 'table_id', 'project_id', 'gcs_schema_object') + template_fields = ('dataset_id', 'table_id', 'project_id', + 'gcs_schema_object', 'labels') ui_color = '#f0eee4' @apply_defaults @@ -284,6 +293,7 @@ def __init__(self, bigquery_conn_id='bigquery_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, + labels=None, *args, **kwargs): super(BigQueryCreateEmptyTableOperator, self).__init__(*args, **kwargs) @@ -297,6 +307,7 @@ def __init__(self, self.google_cloud_storage_conn_id = google_cloud_storage_conn_id self.delegate_to = delegate_to self.time_partitioning = time_partitioning + self.labels = labels def execute(self, context): bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, @@ -323,7 +334,8 @@ def execute(self, context): dataset_id=self.dataset_id, table_id=self.table_id, schema_fields=schema_fields, - time_partitioning=self.time_partitioning + time_partitioning=self.time_partitioning, + labels=self.labels ) @@ -396,9 +408,11 @@ class BigQueryCreateExternalTableOperator(BaseOperator): :type delegate_to: string :param src_fmt_configs: configure optional fields specific to the source format :type src_fmt_configs: dict + :param labels a dictionary containing labels for the table, passed to BigQuery + :type labels: dict """ template_fields = ('bucket', 'source_objects', - 'schema_object', 'destination_project_dataset_table') + 'schema_object', 'destination_project_dataset_table', 'labels') ui_color = '#f0eee4' @apply_defaults @@ -420,6 +434,7 @@ def __init__(self, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, src_fmt_configs={}, + labels=None, *args, **kwargs): super(BigQueryCreateExternalTableOperator, self).__init__(*args, **kwargs) @@ -446,6 +461,7 @@ def __init__(self, self.delegate_to = delegate_to self.src_fmt_configs = src_fmt_configs + self.labels = labels def execute(self, context): bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, @@ -479,7 +495,8 @@ def execute(self, context): quote_character=self.quote_character, allow_quoted_newlines=self.allow_quoted_newlines, allow_jagged_rows=self.allow_jagged_rows, - src_fmt_configs=self.src_fmt_configs + src_fmt_configs=self.src_fmt_configs, + labels=self.labels ) diff --git a/airflow/contrib/operators/bigquery_to_bigquery.py b/airflow/contrib/operators/bigquery_to_bigquery.py index 93a52b31020ce..3a0b44db60580 100644 --- a/airflow/contrib/operators/bigquery_to_bigquery.py +++ b/airflow/contrib/operators/bigquery_to_bigquery.py @@ -49,9 +49,12 @@ class BigQueryToBigQueryOperator(BaseOperator): For this to work, the service account making the request must have domain-wide delegation enabled. :type delegate_to: string + :param labels: a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict """ template_fields = ('source_project_dataset_tables', - 'destination_project_dataset_table') + 'destination_project_dataset_table', 'labels') template_ext = ('.sql',) ui_color = '#e6f0e4' @@ -63,6 +66,7 @@ def __init__(self, create_disposition='CREATE_IF_NEEDED', bigquery_conn_id='bigquery_default', delegate_to=None, + labels=None, *args, **kwargs): super(BigQueryToBigQueryOperator, self).__init__(*args, **kwargs) @@ -72,6 +76,7 @@ def __init__(self, self.create_disposition = create_disposition self.bigquery_conn_id = bigquery_conn_id self.delegate_to = delegate_to + self.labels = labels def execute(self, context): self.log.info( @@ -86,4 +91,5 @@ def execute(self, context): self.source_project_dataset_tables, self.destination_project_dataset_table, self.write_disposition, - self.create_disposition) + self.create_disposition, + self.labels) diff --git a/airflow/contrib/operators/bigquery_to_gcs.py b/airflow/contrib/operators/bigquery_to_gcs.py index e2ce93068f433..278b20804747c 100644 --- a/airflow/contrib/operators/bigquery_to_gcs.py +++ b/airflow/contrib/operators/bigquery_to_gcs.py @@ -54,8 +54,12 @@ class BigQueryToCloudStorageOperator(BaseOperator): For this to work, the service account making the request must have domain-wide delegation enabled. :type delegate_to: string + :param labels: a dictionary containing labels for the job/query, + passed to BigQuery + :type labels: dict """ - template_fields = ('source_project_dataset_table', 'destination_cloud_storage_uris') + template_fields = ('source_project_dataset_table', + 'destination_cloud_storage_uris', 'labels') template_ext = ('.sql',) ui_color = '#e4e6f0' @@ -69,6 +73,7 @@ def __init__(self, print_header=True, bigquery_conn_id='bigquery_default', delegate_to=None, + labels=None, *args, **kwargs): super(BigQueryToCloudStorageOperator, self).__init__(*args, **kwargs) @@ -80,6 +85,7 @@ def __init__(self, self.print_header = print_header self.bigquery_conn_id = bigquery_conn_id self.delegate_to = delegate_to + self.labels = labels def execute(self, context): self.log.info('Executing extract of %s into: %s', @@ -95,4 +101,5 @@ def execute(self, context): self.compression, self.export_format, self.field_delimiter, - self.print_header) + self.print_header, + self.labels) diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 37ad32d7ec036..2f39bd9bce24a 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -58,7 +58,8 @@ def test_suceeds_with_explicit_legacy_query(self): @unittest.skipIf(not bq_available, 'BQ is not available to run tests') def test_suceeds_with_explicit_std_query(self): - df = self.instance.get_pandas_df('select * except(b) from (select 1 a, 2 b)', dialect='standard') + df = self.instance.get_pandas_df( + 'select * except(b) from (select 1 a, 2 b)', dialect='standard') self.assertEqual(df.iloc(0)[0][0], 1) @unittest.skipIf(not bq_available, 'BQ is not available to run tests') @@ -281,6 +282,27 @@ def test_run_query_sql_dialect_override(self, run_with_config): self.assertIs(args[0]['query']['useLegacySql'], bool_val) +class TestLabelsInRunJob(unittest.TestCase): + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_query_with_arg(self, mocked_rwc): + project_id = 12345 + + def run_with_config(config): + self.assertEqual( + config['labels'], {'label1': 'test1', 'label2': 'test2'} + ) + mocked_rwc.side_effect = run_with_config + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + bq_hook.run_query( + sql='select 1', + destination_dataset_table='my_dataset.my_table', + labels={'label1': 'test1', 'label2': 'test2'} + ) + + mocked_rwc.assert_called_once() + + class TestTimePartitioningInRunJob(unittest.TestCase): @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") @@ -365,7 +387,8 @@ def run_with_config(config): bq_hook.run_query( sql='select 1', destination_dataset_table='my_dataset.my_table', - time_partitioning={'type': 'DAY', 'field': 'test_field', 'expirationMs': 1000} + time_partitioning={'type': 'DAY', + 'field': 'test_field', 'expirationMs': 1000} ) mocked_rwc.assert_called_once() diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 21a42d3ef8f46..7c76ab73fec81 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -73,7 +73,8 @@ def test_execute(self, mock_hook): project_id=TEST_PROJECT_ID, table_id=TEST_TABLE_ID, schema_fields=None, - time_partitioning={} + time_partitioning={}, + labels=None ) @@ -112,7 +113,8 @@ def test_execute(self, mock_hook): quote_character=None, allow_quoted_newlines=False, allow_jagged_rows=False, - src_fmt_configs={} + src_fmt_configs={}, + labels=None ) From bde5bffd497bc1035b52f109dd16198bef86af4c Mon Sep 17 00:00:00 2001 From: Yuliya Volkova Date: Mon, 6 Aug 2018 04:23:17 -0500 Subject: [PATCH 063/808] [AIRFLOW-2845] Asserts in contrib package code are changed on raise ValueError and TypeError (#3690) --- airflow/contrib/hooks/bigquery_hook.py | 55 ++++++++++--------- airflow/contrib/hooks/databricks_hook.py | 3 +- airflow/contrib/hooks/gcp_dataflow_hook.py | 24 ++++---- airflow/contrib/hooks/gcp_mlengine_hook.py | 11 +++- airflow/contrib/hooks/gcs_hook.py | 15 ++--- .../contrib/operators/mlengine_operator.py | 4 +- tests/contrib/hooks/test_bigquery_hook.py | 2 +- tests/contrib/hooks/test_databricks_hook.py | 6 +- tests/contrib/hooks/test_gcp_dataflow_hook.py | 8 +-- tests/contrib/hooks/test_gcs_hook.py | 4 +- 10 files changed, 74 insertions(+), 58 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index f4c1a3b520217..aa8fc382a6a67 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -592,9 +592,11 @@ def run_query(self, } if destination_dataset_table: - assert '.' in destination_dataset_table, ( - 'Expected destination_dataset_table in the format of ' - '.. Got: {}').format(destination_dataset_table) + if '.' not in destination_dataset_table: + raise ValueError( + 'Expected destination_dataset_table name in the format of ' + '.
. Got: {}'.format( + destination_dataset_table)) destination_project, destination_dataset, destination_table = \ _split_tablename(table_input=destination_dataset_table, default_project_id=self.project_id) @@ -610,7 +612,9 @@ def run_query(self, } }) if udf_config: - assert isinstance(udf_config, list) + if not isinstance(udf_config, list): + raise TypeError("udf_config argument must have a type 'list'" + " not {}".format(type(udf_config))) configuration['query'].update({ 'userDefinedFunctionResources': udf_config }) @@ -1153,10 +1157,10 @@ def run_table_delete(self, deletion_dataset_table, :type ignore_if_missing: boolean :return: """ - - assert '.' in deletion_dataset_table, ( - 'Expected deletion_dataset_table in the format of ' - '.
. Got: {}').format(deletion_dataset_table) + if '.' not in deletion_dataset_table: + raise ValueError( + 'Expected deletion_dataset_table name in the format of ' + '.
. Got: {}'.format(deletion_dataset_table)) deletion_project, deletion_dataset, deletion_table = \ _split_tablename(table_input=deletion_dataset_table, default_project_id=self.project_id) @@ -1284,14 +1288,10 @@ def run_grant_dataset_view_access(self, # if view is already in access, do nothing. self.log.info( 'Table %s:%s.%s already has authorized view access to %s:%s dataset.', - view_project, view_dataset, view_table, source_project, - source_dataset) + view_project, view_dataset, view_table, source_project, source_dataset) return source_dataset_resource - def delete_dataset(self, - project_id, - dataset_id - ): + def delete_dataset(self, project_id, dataset_id): """ Delete a dataset of Big query in your project. :param project_id: The name of the project where we have the dataset . @@ -1308,9 +1308,8 @@ def delete_dataset(self, self.service.datasets().delete( projectId=project_id, datasetId=dataset_id).execute() - - self.log.info('Dataset deleted successfully: In project %s Dataset %s', - project_id, dataset_id) + self.log.info('Dataset deleted successfully: In project %s ' + 'Dataset %s', project_id, dataset_id) except HttpError as err: raise AirflowException( @@ -1518,14 +1517,17 @@ def _bq_cast(string_field, bq_type): elif bq_type == 'FLOAT' or bq_type == 'TIMESTAMP': return float(string_field) elif bq_type == 'BOOLEAN': - assert string_field in set(['true', 'false']) + if string_field not in ['true', 'false']: + raise ValueError("{} must have value 'true' or 'false'".format( + string_field)) return string_field == 'true' else: return string_field def _split_tablename(table_input, default_project_id, var_name=None): - assert default_project_id is not None, "INTERNAL: No default project is specified" + if not default_project_id: + raise ValueError("INTERNAL: No default project is specified") def var_print(var_name): if var_name is None: @@ -1537,7 +1539,6 @@ def var_print(var_name): raise Exception(('{var}Use either : or . to specify project ' 'got {input}').format( var=var_print(var_name), input=table_input)) - cmpt = table_input.rsplit(':', 1) project_id = None rest = table_input @@ -1555,8 +1556,10 @@ def var_print(var_name): cmpt = rest.split('.') if len(cmpt) == 3: - assert project_id is None, ("{var}Use either : or . to specify project" - ).format(var=var_print(var_name)) + if project_id: + raise ValueError( + "{var}Use either : or . to specify project".format( + var=var_print(var_name))) project_id = cmpt[0] dataset_id = cmpt[1] table_id = cmpt[2] @@ -1586,10 +1589,10 @@ def _cleanse_time_partitioning(destination_dataset_table, time_partitioning_in): # if it is a partitioned table ($ is in the table name) add partition load option time_partitioning_out = {} if destination_dataset_table and '$' in destination_dataset_table: - assert not time_partitioning_in.get('field'), ( - "Cannot specify field partition and partition name " - "(dataset.table$partition) at the same time" - ) + if time_partitioning_in.get('field'): + raise ValueError( + "Cannot specify field partition and partition name" + "(dataset.table$partition) at the same time") time_partitioning_out['type'] = 'DAY' time_partitioning_out.update(time_partitioning_in) diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py index 1443ff4740b94..2e5f1399b4765 100644 --- a/airflow/contrib/hooks/databricks_hook.py +++ b/airflow/contrib/hooks/databricks_hook.py @@ -61,7 +61,8 @@ def __init__( self.databricks_conn_id = databricks_conn_id self.databricks_conn = self.get_connection(databricks_conn_id) self.timeout_seconds = timeout_seconds - assert retry_limit >= 1, 'Retry limit must be greater than equal to 1' + if retry_limit < 1: + raise ValueError('Retry limit must be greater than equal to 1') self.retry_limit = retry_limit def _parse_host(self, host): diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index 279b9dd21a862..ee3b510ed706a 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -225,11 +225,11 @@ def label_formatter(labels_dict): def _build_dataflow_job_name(task_id, append_job_name=True): task_id = str(task_id).replace('_', '-') - assert re.match(r"^[a-z]([-a-z0-9]*[a-z0-9])?$", task_id), \ - 'Invalid job_name ({}); the name must consist of ' \ - 'only the characters [-a-z0-9], starting with a ' \ - 'letter and ending with a letter or number '.format( - task_id) + if not re.match(r"^[a-z]([-a-z0-9]*[a-z0-9])?$", task_id): + raise ValueError( + 'Invalid job_name ({}); the name must consist of' + 'only the characters [-a-z0-9], starting with a ' + 'letter and ending with a letter or number '.format(task_id)) if append_job_name: job_name = task_id + "-" + str(uuid.uuid1())[:8] @@ -238,7 +238,8 @@ def _build_dataflow_job_name(task_id, append_job_name=True): return job_name - def _build_cmd(self, task_id, variables, label_formatter): + @staticmethod + def _build_cmd(task_id, variables, label_formatter): command = ["--runner=DataflowRunner"] if variables is not None: for attr, value in variables.items(): @@ -250,7 +251,8 @@ def _build_cmd(self, task_id, variables, label_formatter): command.append("--" + attr + "=" + value) return command - def _start_template_dataflow(self, name, variables, parameters, dataflow_template): + def _start_template_dataflow(self, name, variables, parameters, + dataflow_template): # Builds RuntimeEnvironment from variables dictionary # https://cloud.google.com/dataflow/docs/reference/rest/v1b3/RuntimeEnvironment environment = {} @@ -262,9 +264,11 @@ def _start_template_dataflow(self, name, variables, parameters, dataflow_templat "parameters": parameters, "environment": environment} service = self.get_conn() - request = service.projects().templates().launch(projectId=variables['project'], - gcsPath=dataflow_template, - body=body) + request = service.projects().templates().launch( + projectId=variables['project'], + gcsPath=dataflow_template, + body=body + ) response = request.execute() variables = self._set_variables(variables) _DataflowJob(self.get_conn(), variables['project'], name, variables['region'], diff --git a/airflow/contrib/hooks/gcp_mlengine_hook.py b/airflow/contrib/hooks/gcp_mlengine_hook.py index 66f392b1564c9..b9f1008fa7fde 100644 --- a/airflow/contrib/hooks/gcp_mlengine_hook.py +++ b/airflow/contrib/hooks/gcp_mlengine_hook.py @@ -152,7 +152,8 @@ def _wait_for_job_done(self, project_id, job_id, interval=30): apiclient.errors.HttpError: if HTTP error is returned when getting the job """ - assert interval > 0 + if interval <= 0: + raise ValueError("Interval must be > 0") while True: job = self._get_job(project_id, job_id) if job['state'] in ['SUCCEEDED', 'FAILED', 'CANCELLED']: @@ -242,7 +243,9 @@ def create_model(self, project_id, model): """ Create a Model. Blocks until finished. """ - assert model['name'] is not None and model['name'] is not '' + if not model['name']: + raise ValueError("Model name must be provided and " + "could not be an empty string") project = 'projects/{}'.format(project_id) request = self._mlengine.projects().models().create( @@ -253,7 +256,9 @@ def get_model(self, project_id, model_name): """ Gets a Model. Blocks until finished. """ - assert model_name is not None and model_name is not '' + if not model_name: + raise ValueError("Model name must be provided and " + "it could not be an empty string") full_model_name = 'projects/{}/models/{}'.format( project_id, model_name) request = self._mlengine.projects().models().get(name=full_model_name) diff --git a/airflow/contrib/hooks/gcs_hook.py b/airflow/contrib/hooks/gcs_hook.py index c5e356f41cfc0..08d44ce7faf92 100644 --- a/airflow/contrib/hooks/gcs_hook.py +++ b/airflow/contrib/hooks/gcs_hook.py @@ -477,15 +477,16 @@ def create_bucket(self, self.log.info('Creating Bucket: %s; Location: %s; Storage Class: %s', bucket_name, location, storage_class) - assert storage_class in storage_classes, \ - 'Invalid value ({}) passed to storage_class. Value should be ' \ - 'one of {}'.format(storage_class, storage_classes) + if storage_class not in storage_classes: + raise ValueError( + 'Invalid value ({}) passed to storage_class. Value should be ' + 'one of {}'.format(storage_class, storage_classes)) - assert re.match('[a-zA-Z0-9]+', bucket_name[0]), \ - 'Bucket names must start with a number or letter.' + if not re.match('[a-zA-Z0-9]+', bucket_name[0]): + raise ValueError('Bucket names must start with a number or letter.') - assert re.match('[a-zA-Z0-9]+', bucket_name[-1]), \ - 'Bucket names must end with a number or letter.' + if not re.match('[a-zA-Z0-9]+', bucket_name[-1]): + raise ValueError('Bucket names must end with a number or letter.') service = self.get_conn() bucket_resource = { diff --git a/airflow/contrib/operators/mlengine_operator.py b/airflow/contrib/operators/mlengine_operator.py index 9fe966d387ab1..8e75b3c608238 100644 --- a/airflow/contrib/operators/mlengine_operator.py +++ b/airflow/contrib/operators/mlengine_operator.py @@ -427,7 +427,9 @@ def execute(self, context): gcp_conn_id=self._gcp_conn_id, delegate_to=self._delegate_to) if self._operation == 'create': - assert self._version is not None + if not self._version: + raise ValueError("version attribute of {} could not " + "be empty".format(self.__class__.__name__)) return hook.create_version(self._project_id, self._model_name, self._version) elif self._operation == 'set_default': diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 2f39bd9bce24a..39435f0c4ec63 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -414,7 +414,7 @@ def test_extra_time_partitioning_options(self): self.assertEqual(tp_out, expect) def test_cant_add_dollar_and_field_name(self): - with self.assertRaises(AssertionError): + with self.assertRaises(ValueError): _cleanse_time_partitioning( 'test.teast$20170101', {'type': 'DAY', 'field': 'test_field', 'expirationMs': 1000} diff --git a/tests/contrib/hooks/test_databricks_hook.py b/tests/contrib/hooks/test_databricks_hook.py index 6052a6d54f1f8..aca8dd96004b4 100644 --- a/tests/contrib/hooks/test_databricks_hook.py +++ b/tests/contrib/hooks/test_databricks_hook.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -110,7 +110,7 @@ def test_parse_host_with_scheme(self): self.assertEquals(host, HOST) def test_init_bad_retry_limit(self): - with self.assertRaises(AssertionError): + with self.assertRaises(ValueError): DatabricksHook(retry_limit = 0) @mock.patch('airflow.contrib.hooks.databricks_hook.requests') diff --git a/tests/contrib/hooks/test_gcp_dataflow_hook.py b/tests/contrib/hooks/test_gcp_dataflow_hook.py index 90714c6ee4f62..bc7c587135b9b 100644 --- a/tests/contrib/hooks/test_gcp_dataflow_hook.py +++ b/tests/contrib/hooks/test_gcp_dataflow_hook.py @@ -195,7 +195,7 @@ def test_invalid_dataflow_job_name(self): fixed_name = invalid_job_name.replace( '_', '-') - with self.assertRaises(AssertionError) as e: + with self.assertRaises(ValueError) as e: self.dataflow_hook._build_dataflow_job_name( task_id=invalid_job_name, append_job_name=False ) @@ -222,19 +222,19 @@ def test_dataflow_job_regex_check(self): ), 'dfjob1') self.assertRaises( - AssertionError, + ValueError, self.dataflow_hook._build_dataflow_job_name, task_id='1dfjob', append_job_name=False ) self.assertRaises( - AssertionError, + ValueError, self.dataflow_hook._build_dataflow_job_name, task_id='dfjob@', append_job_name=False ) self.assertRaises( - AssertionError, + ValueError, self.dataflow_hook._build_dataflow_job_name, task_id='df^jo', append_job_name=False ) diff --git a/tests/contrib/hooks/test_gcs_hook.py b/tests/contrib/hooks/test_gcs_hook.py index fb65938fd9afe..eedceff1f7790 100644 --- a/tests/contrib/hooks/test_gcs_hook.py +++ b/tests/contrib/hooks/test_gcs_hook.py @@ -66,14 +66,14 @@ class TestGCSBucket(unittest.TestCase): def test_bucket_name_value(self): bad_start_bucket_name = '/testing123' - with self.assertRaises(AssertionError): + with self.assertRaises(ValueError): gcs_hook.GoogleCloudStorageHook().create_bucket( bucket_name=bad_start_bucket_name ) bad_end_bucket_name = 'testing123/' - with self.assertRaises(AssertionError): + with self.assertRaises(ValueError): gcs_hook.GoogleCloudStorageHook().create_bucket( bucket_name=bad_end_bucket_name ) From e7f73d20462964bf104c7508d3f3d669e7797e49 Mon Sep 17 00:00:00 2001 From: Chris Riccomini Date: Fri, 31 Aug 2018 12:42:43 -0700 Subject: [PATCH 064/808] [AIRFLOW-2994] Fix flatten_results for BigQueryOperator (#3829) --- airflow/contrib/hooks/bigquery_hook.py | 2 +- airflow/contrib/operators/bigquery_operator.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index aa8fc382a6a67..6db0c00bdd9d2 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -472,7 +472,7 @@ def run_query(self, destination_dataset_table=False, write_disposition='WRITE_EMPTY', allow_large_results=False, - flatten_results=False, + flatten_results=None, udf_config=False, use_legacy_sql=None, maximum_billing_tier=None, diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 09f442ae7b1a6..9ec1f992c11d8 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -106,7 +106,7 @@ def __init__(self, destination_dataset_table=False, write_disposition='WRITE_EMPTY', allow_large_results=False, - flatten_results=False, + flatten_results=None, bigquery_conn_id='bigquery_default', delegate_to=None, udf_config=False, From 7603042361d3aef18bf774c34d0fabed4279d0c0 Mon Sep 17 00:00:00 2001 From: Iuliia Volkova Date: Fri, 14 Sep 2018 12:53:52 +0300 Subject: [PATCH 065/808] [AIRFLOW-2772] Fix Bug in BigQuery hook for Partitioned Table (#3901) --- airflow/contrib/hooks/bigquery_hook.py | 13 ++----------- airflow/contrib/operators/bigquery_operator.py | 4 +--- tests/contrib/hooks/test_bigquery_hook.py | 7 ------- 3 files changed, 3 insertions(+), 21 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 6db0c00bdd9d2..f9abdbaaacdab 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -540,9 +540,7 @@ def run_query(self, The default value is INTERACTIVE. :type priority: string :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and - expiration as per API specifications. Note that 'field' is not available in - conjunction with dataset.table$partition. + partition by field, type and expiration as per API specifications. :type time_partitioning: dict """ @@ -869,9 +867,7 @@ def run_load(self, :param src_fmt_configs: configure optional fields specific to the source format :type src_fmt_configs: dict :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and - expiration as per API specifications. Note that 'field' is not available in - conjunction with dataset.table$partition. + partition by field, type and expiration as per API specifications. :type time_partitioning: dict """ @@ -1589,11 +1585,6 @@ def _cleanse_time_partitioning(destination_dataset_table, time_partitioning_in): # if it is a partitioned table ($ is in the table name) add partition load option time_partitioning_out = {} if destination_dataset_table and '$' in destination_dataset_table: - if time_partitioning_in.get('field'): - raise ValueError( - "Cannot specify field partition and partition name" - "(dataset.table$partition) at the same time") time_partitioning_out['type'] = 'DAY' - time_partitioning_out.update(time_partitioning_in) return time_partitioning_out diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 9ec1f992c11d8..2eadd04f003d9 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -89,9 +89,7 @@ class BigQueryOperator(BaseOperator): The default value is INTERACTIVE. :type priority: string :param time_partitioning: configure optional time partitioning fields i.e. - partition by field, type and - expiration as per API specifications. Note that 'field' is not available in - conjunction with dataset.table$partition. + partition by field, type and expiration as per API specifications. :type time_partitioning: dict """ diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 39435f0c4ec63..74e3af452bf6f 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -413,13 +413,6 @@ def test_extra_time_partitioning_options(self): } self.assertEqual(tp_out, expect) - def test_cant_add_dollar_and_field_name(self): - with self.assertRaises(ValueError): - _cleanse_time_partitioning( - 'test.teast$20170101', - {'type': 'DAY', 'field': 'test_field', 'expirationMs': 1000} - ) - class TestBigQueryHookLegacySql(unittest.TestCase): """Ensure `use_legacy_sql` param in `BigQueryHook` propagates properly.""" From 06e09689ff6179f3597fe99983d8cde8a8e816c7 Mon Sep 17 00:00:00 2001 From: fenglu-g Date: Wed, 15 Aug 2018 01:42:57 -0700 Subject: [PATCH 066/808] [AIRFLOW-2893] fix stuck dataflow job due to name mismatch (#3744) --- airflow/contrib/hooks/gcp_dataflow_hook.py | 52 ++++++++---- tests/contrib/hooks/test_gcp_dataflow_hook.py | 83 ++++++++++++++----- 2 files changed, 97 insertions(+), 38 deletions(-) diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index ee3b510ed706a..3a8eb28cee77b 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -34,12 +34,13 @@ class _DataflowJob(LoggingMixin): - def __init__(self, dataflow, project_number, name, location, poll_sleep=10): + def __init__(self, dataflow, project_number, name, location, poll_sleep=10, + job_id=None): self._dataflow = dataflow self._project_number = project_number self._job_name = name self._job_location = location - self._job_id = None + self._job_id = job_id self._job = self._get_job() self._poll_sleep = poll_sleep @@ -55,13 +56,15 @@ def _get_job_id_from_name(self): return None def _get_job(self): - if self._job_name: - job = self._get_job_id_from_name() - else: + if self._job_id: job = self._dataflow.projects().jobs().get( projectId=self._project_number, jobId=self._job_id ).execute(num_retries=5) + elif self._job_name: + job = self._get_job_id_from_name() + else: + raise Exception('Missing both dataflow job ID and name.') if job and 'currentState' in job: self.log.info( @@ -124,36 +127,50 @@ def __init__(self, cmd): def _line(self, fd): if fd == self._proc.stderr.fileno(): - lines = self._proc.stderr.readlines() - for line in lines: + line = b''.join(self._proc.stderr.readlines()) + if line: self.log.warning(line[:-1]) - if lines: - return lines[-1] + return line if fd == self._proc.stdout.fileno(): - line = self._proc.stdout.readline() + line = b''.join(self._proc.stdout.readlines()) + if line: + self.log.info(line[:-1]) return line @staticmethod def _extract_job(line): - if line is not None: - if line.startswith("Submitted job: "): - return line[15:-1] + # Job id info: https://goo.gl/SE29y9. + job_id_pattern = re.compile( + b'.*console.cloud.google.com/dataflow.*/jobs/([a-z|0-9|A-Z|\-|\_]+).*') + matched_job = job_id_pattern.search(line or '') + if matched_job: + return matched_job.group(1).decode() def wait_for_done(self): reads = [self._proc.stderr.fileno(), self._proc.stdout.fileno()] self.log.info("Start waiting for DataFlow process to complete.") - while self._proc.poll() is None: + job_id = None + # Make sure logs are processed regardless whether the subprocess is + # terminated. + process_ends = False + while True: ret = select.select(reads, [], [], 5) if ret is not None: for fd in ret[0]: line = self._line(fd) if line: - self.log.debug(line[:-1]) + job_id = job_id or self._extract_job(line) else: self.log.info("Waiting for DataFlow process to complete.") + if process_ends: + break + if self._proc.poll() is not None: + # Mark process completion but allows its outputs to be consumed. + process_ends = True if self._proc.returncode is not 0: raise Exception("DataFlow failed with return code {}".format( self._proc.returncode)) + return job_id class DataFlowHook(GoogleCloudBaseHook): @@ -178,9 +195,10 @@ def _start_dataflow(self, task_id, variables, name, variables = self._set_variables(variables) cmd = command_prefix + self._build_cmd(task_id, variables, label_formatter) - _Dataflow(cmd).wait_for_done() + job_id = _Dataflow(cmd).wait_for_done() _DataflowJob(self.get_conn(), variables['project'], name, - variables['region'], self.poll_sleep).wait_for_done() + variables['region'], + self.poll_sleep, job_id).wait_for_done() @staticmethod def _set_variables(variables): diff --git a/tests/contrib/hooks/test_gcp_dataflow_hook.py b/tests/contrib/hooks/test_gcp_dataflow_hook.py index bc7c587135b9b..686db10e5c305 100644 --- a/tests/contrib/hooks/test_gcp_dataflow_hook.py +++ b/tests/contrib/hooks/test_gcp_dataflow_hook.py @@ -22,8 +22,8 @@ from mock import call from mock import MagicMock -from airflow.contrib.hooks.gcp_dataflow_hook import DataFlowHook -from airflow.contrib.hooks.gcp_dataflow_hook import _Dataflow +from airflow.contrib.hooks.gcp_dataflow_hook import DataFlowHook,\ + _Dataflow, _DataflowJob try: from unittest import mock @@ -62,6 +62,10 @@ BASE_STRING = 'airflow.contrib.hooks.gcp_api_base_hook.{}' DATAFLOW_STRING = 'airflow.contrib.hooks.gcp_dataflow_hook.{}' MOCK_UUID = '12345678' +TEST_PROJECT = 'test-project' +TEST_JOB_NAME = 'test-job-name' +TEST_JOB_ID = 'test-job-id' +TEST_LOCATION = 'us-central1' def mock_init(self, gcp_conn_id, delegate_to=None): @@ -152,25 +156,25 @@ def test_start_java_dataflow_with_job_class( @mock.patch('subprocess.Popen') @mock.patch('select.select') def test_dataflow_wait_for_done_logging(self, mock_select, mock_popen, mock_logging): - mock_logging.info = MagicMock() - mock_logging.warning = MagicMock() - mock_proc = MagicMock() - mock_proc.stderr = MagicMock() - mock_proc.stderr.readlines = MagicMock(return_value=['test\n','error\n']) - mock_stderr_fd = MagicMock() - mock_proc.stderr.fileno = MagicMock(return_value=mock_stderr_fd) - mock_proc_poll = MagicMock() - mock_select.return_value = [[mock_stderr_fd]] - def poll_resp_error(): - mock_proc.return_code = 1 - return True - mock_proc_poll.side_effect=[None, poll_resp_error] - mock_proc.poll = mock_proc_poll - mock_popen.return_value = mock_proc - dataflow = _Dataflow(['test', 'cmd']) - mock_logging.info.assert_called_with('Running command: %s', 'test cmd') - self.assertRaises(Exception, dataflow.wait_for_done) - mock_logging.warning.assert_has_calls([call('test'), call('error')]) + mock_logging.info = MagicMock() + mock_logging.warning = MagicMock() + mock_proc = MagicMock() + mock_proc.stderr = MagicMock() + mock_proc.stderr.readlines = MagicMock(return_value=['test\n', 'error\n']) + mock_stderr_fd = MagicMock() + mock_proc.stderr.fileno = MagicMock(return_value=mock_stderr_fd) + mock_proc_poll = MagicMock() + mock_select.return_value = [[mock_stderr_fd]] + + def poll_resp_error(): + mock_proc.return_code = 1 + return True + mock_proc_poll.side_effect = [None, poll_resp_error] + mock_proc.poll = mock_proc_poll + mock_popen.return_value = mock_proc + dataflow = _Dataflow(['test', 'cmd']) + mock_logging.info.assert_called_with('Running command: %s', 'test cmd') + self.assertRaises(Exception, dataflow.wait_for_done) def test_valid_dataflow_job_name(self): job_name = self.dataflow_hook._build_dataflow_job_name( @@ -254,3 +258,40 @@ def test_start_template_dataflow(self, internal_dataflow_mock): dataflow_template=TEMPLATE) internal_dataflow_mock.assert_called_once_with( mock.ANY, DATAFLOW_OPTIONS_TEMPLATE, PARAMETERS, TEMPLATE) + + +class DataFlowJobTest(unittest.TestCase): + + def setUp(self): + self.mock_dataflow = MagicMock() + + def test_dataflow_job_init_with_job_id(self): + mock_jobs = MagicMock() + self.mock_dataflow.projects.return_value.\ + jobs.return_value = mock_jobs + _DataflowJob(self.mock_dataflow, TEST_PROJECT, TEST_JOB_NAME, + TEST_LOCATION, 10, TEST_JOB_ID) + mock_jobs.get.assert_called_with(projectId=TEST_PROJECT, jobId=TEST_JOB_ID) + + def test_dataflow_job_init_without_job_id(self): + mock_jobs = MagicMock() + self.mock_dataflow.projects.return_value.locations.return_value.\ + jobs.return_value = mock_jobs + _DataflowJob(self.mock_dataflow, TEST_PROJECT, TEST_JOB_NAME, + TEST_LOCATION, 10) + mock_jobs.list.assert_called_with(projectId=TEST_PROJECT, + location=TEST_LOCATION) + + +class DataflowTest(unittest.TestCase): + + def test_data_flow_valid_job_id(self): + cmd = ['echo', 'additional unit test lines.\n' + + 'INFO: the Dataflow monitoring console, please navigate to' + + 'https://console.cloud.google.com/dataflow/jobsDetail/locations/' + + '{}/jobs/{}?project={}'.format(TEST_LOCATION, TEST_JOB_ID, TEST_PROJECT)] + self.assertEqual(_Dataflow(cmd).wait_for_done(), TEST_JOB_ID) + + def test_data_flow_missing_job_id(self): + cmd = ['echo', 'unit testing'] + self.assertEqual(_Dataflow(cmd).wait_for_done(), None) From c9767f893ca0839259d4bb0197dc2788d42caaa0 Mon Sep 17 00:00:00 2001 From: fenglu-g Date: Mon, 20 Aug 2018 16:55:44 -0700 Subject: [PATCH 067/808] [AIRFLOW-2905] Fix get job API endpoint (#3755) --- airflow/contrib/hooks/gcp_dataflow_hook.py | 6 +++--- tests/contrib/hooks/test_gcp_dataflow_hook.py | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py index 3a8eb28cee77b..8c9b7423e0e6d 100644 --- a/airflow/contrib/hooks/gcp_dataflow_hook.py +++ b/airflow/contrib/hooks/gcp_dataflow_hook.py @@ -57,10 +57,10 @@ def _get_job_id_from_name(self): def _get_job(self): if self._job_id: - job = self._dataflow.projects().jobs().get( + job = self._dataflow.projects().locations().jobs().get( projectId=self._project_number, - jobId=self._job_id - ).execute(num_retries=5) + location=self._job_location, + jobId=self._job_id).execute(num_retries=5) elif self._job_name: job = self._get_job_id_from_name() else: diff --git a/tests/contrib/hooks/test_gcp_dataflow_hook.py b/tests/contrib/hooks/test_gcp_dataflow_hook.py index 686db10e5c305..7811e4aabd59f 100644 --- a/tests/contrib/hooks/test_gcp_dataflow_hook.py +++ b/tests/contrib/hooks/test_gcp_dataflow_hook.py @@ -267,11 +267,12 @@ def setUp(self): def test_dataflow_job_init_with_job_id(self): mock_jobs = MagicMock() - self.mock_dataflow.projects.return_value.\ + self.mock_dataflow.projects.return_value.locations.return_value.\ jobs.return_value = mock_jobs _DataflowJob(self.mock_dataflow, TEST_PROJECT, TEST_JOB_NAME, TEST_LOCATION, 10, TEST_JOB_ID) - mock_jobs.get.assert_called_with(projectId=TEST_PROJECT, jobId=TEST_JOB_ID) + mock_jobs.get.assert_called_with(projectId=TEST_PROJECT, location=TEST_LOCATION, + jobId=TEST_JOB_ID) def test_dataflow_job_init_without_job_id(self): mock_jobs = MagicMock() From bcd1b42c2dfe602e1ff25af0aca5778a324fcddf Mon Sep 17 00:00:00 2001 From: Leo Gallucci Date: Mon, 20 Aug 2018 11:41:27 +0200 Subject: [PATCH 068/808] [AIRFLOW-2907] Fix Sendgrid attachments bytes err (#3757) [AIRFLOW-2907] Fix Sendgrid Attachments Object of type 'bytes' is not JSON serializable Attempting to attach files via Sendgrid operator always gives: Sendgrid - Attachments - ERROR - Object of type 'bytes' is not JSON serializable --- airflow/contrib/utils/sendgrid.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/contrib/utils/sendgrid.py b/airflow/contrib/utils/sendgrid.py index 9055c97879e17..d628097159904 100644 --- a/airflow/contrib/utils/sendgrid.py +++ b/airflow/contrib/utils/sendgrid.py @@ -88,7 +88,7 @@ def send_email(to, subject, html_content, files=None, basename = os.path.basename(fname) attachment = Attachment() with open(fname, "rb") as f: - attachment.content = base64.b64encode(f.read()) + attachment.content = str(base64.b64encode(f.read()), 'utf-8') attachment.type = mimetypes.guess_type(basename)[0] attachment.filename = basename attachment.disposition = "attachment" From ff454a7b1648a519fa8108f83c634281f0aceb2a Mon Sep 17 00:00:00 2001 From: "Taylor D. Edmiston" Date: Tue, 21 Aug 2018 18:01:25 -0400 Subject: [PATCH 069/808] [AIRFLOW-XXX] Fix some operator names in the docs (#3778) --- docs/concepts.rst | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/concepts.rst b/docs/concepts.rst index 9b10224a817d8..fcaaf99214ae0 100644 --- a/docs/concepts.rst +++ b/docs/concepts.rst @@ -113,13 +113,12 @@ Airflow provides operators for many common tasks, including: - ``BashOperator`` - executes a bash command - ``PythonOperator`` - calls an arbitrary Python function - ``EmailOperator`` - sends an email -- ``HTTPOperator`` - sends an HTTP request +- ``SimpleHttpOperator`` - sends an HTTP request - ``MySqlOperator``, ``SqliteOperator``, ``PostgresOperator``, ``MsSqlOperator``, ``OracleOperator``, ``JdbcOperator``, etc. - executes a SQL command - ``Sensor`` - waits for a certain time, file, database row, S3 key, etc... - In addition to these basic building blocks, there are many more specific -operators: ``DockerOperator``, ``HiveOperator``, ``S3FileTransferOperator``, +operators: ``DockerOperator``, ``HiveOperator``, ``S3FileTransformOperator``, ``PrestoToMysqlOperator``, ``SlackOperator``... you get the idea! The ``airflow/contrib/`` directory contains yet more operators built by the From cdf29af564ce99b5a3ab4680196da857dfd3380c Mon Sep 17 00:00:00 2001 From: PointyShinyBurning Date: Thu, 23 Aug 2018 13:05:36 +0100 Subject: [PATCH 070/808] [AIRFLOW-XXX] Replaces incorrect env var name in INSTALL (#3788) INSTALL mis-named AIRFLOW_GPL_UNIDECODE environment variable. --- INSTALL | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/INSTALL b/INSTALL index 596ce25814088..b018839ab1a5c 100644 --- a/INSTALL +++ b/INSTALL @@ -20,7 +20,7 @@ source my_env/bin/activate # To make sure not to install the GPL dependency: # export SLUGIFY_USES_TEXT_UNIDECODE=yes # In case you do not mind: -# export GPL_UNIDECODE=yes +# export AIRFLOW_GPL_UNIDECODE=yes # [required] building and installing # by pip (preferred) From 1357641ea1c150f063f29ba166c5ca474975310a Mon Sep 17 00:00:00 2001 From: William Pursell Date: Thu, 23 Aug 2018 15:44:09 -0700 Subject: [PATCH 071/808] [AIRFLOW-2938] Handle improperly formatted extra field in connection gracefully (#3785) --- airflow/hooks/http_hook.py | 6 +++++- airflow/www_rbac/views.py | 5 +++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/airflow/hooks/http_hook.py b/airflow/hooks/http_hook.py index c449fe0c15ffb..caa89d3142e9b 100644 --- a/airflow/hooks/http_hook.py +++ b/airflow/hooks/http_hook.py @@ -70,7 +70,11 @@ def get_conn(self, headers=None): if conn.login: session.auth = (conn.login, conn.password) if conn.extra: - session.headers.update(conn.extra_dejson) + try: + session.headers.update(conn.extra_dejson) + except TypeError: + self.log.warn('Connection to {} has invalid extra field.'.format( + conn.host)) if headers: session.headers.update(headers) diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 34bdbc6ecb09a..d42cc6fbab33e 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -1725,6 +1725,11 @@ def prefill_form(self, form, pk): except Exception: d = {} + if not hasattr(d, 'get'): + logging.warning('extra field for {} is not iterable'.format( + form.data.get('conn_id', ''))) + return + for field in self.extra_fields: value = d.get(field, '') if value: From ebf2260472c67edb8c4b95b0fbc21e3351fd365b Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 10 Sep 2018 10:19:22 +0100 Subject: [PATCH 072/808] [AIRFLOW-3030] Fix CLI docs (#3872) --- airflow/bin/cli.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index a84ede16fcb5f..5a45a4595dd67 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -80,6 +80,11 @@ log = LoggingMixin().log +DAGS_FOLDER = settings.DAGS_FOLDER + +if "BUILDING_AIRFLOW_DOCS" in os.environ: + DAGS_FOLDER = '[AIRFLOW_HOME]/dags' + def sigint_handler(sig, frame): sys.exit(0) @@ -133,7 +138,7 @@ def setup_locations(process, pid=None, stdout=None, stderr=None, log=None): def process_subdir(subdir): if subdir: - subdir = subdir.replace('DAGS_FOLDER', settings.DAGS_FOLDER) + subdir = subdir.replace('DAGS_FOLDER', DAGS_FOLDER) subdir = os.path.abspath(os.path.expanduser(subdir)) return subdir @@ -1310,8 +1315,10 @@ class CLIFactory(object): "The regex to filter specific task_ids to backfill (optional)"), 'subdir': Arg( ("-sd", "--subdir"), - "File location or directory from which to look for the dag", - default=settings.DAGS_FOLDER), + "File location or directory from which to look for the dag. " + "Defaults to '[AIRFLOW_HOME]/dags' where [AIRFLOW_HOME] is the " + "value you set for 'AIRFLOW_HOME' config you set in 'airflow.cfg' ", + default=DAGS_FOLDER), 'start_date': Arg( ("-s", "--start_date"), "Override start_date YYYY-MM-DD", type=parsedate), @@ -1711,7 +1718,7 @@ class CLIFactory(object): "If reset_dag_run option is used," " backfill will first prompt users whether airflow " "should clear all the previous dag_run and task_instances " - "within the backfill date range." + "within the backfill date range. " "If rerun_failed_tasks is used, backfill " "will auto re-run the previous failed task instances" " within the backfill date range.", From 98fd9b545d8fa0854843f6fce75de13d769f3fbd Mon Sep 17 00:00:00 2001 From: Juhwi Lee Date: Tue, 24 Jul 2018 11:24:31 +0200 Subject: [PATCH 073/808] [AIRFLOW-2792] change parameter in post requests. Closes #3633 from happyjulie/AIRFLOW-2792 --- airflow/hooks/druid_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py index 50cee8a2259b1..e5fb6a7a26d91 100644 --- a/airflow/hooks/druid_hook.py +++ b/airflow/hooks/druid_hook.py @@ -64,7 +64,7 @@ def get_conn_url(self): def submit_indexing_job(self, json_index_spec): url = self.get_conn_url() - req_index = requests.post(url, data=json_index_spec, headers=self.header) + req_index = requests.post(url, json=json_index_spec, headers=self.header) if (req_index.status_code != 200): raise AirflowException('Did not get 200 when ' 'submitting the Druid job to {}'.format(url)) From 96a6c296c1c65ed4a42723522a9fe065045d4536 Mon Sep 17 00:00:00 2001 From: awelsh93 <32643586+awelsh93@users.noreply.github.com> Date: Tue, 7 Aug 2018 15:43:54 +0100 Subject: [PATCH 074/808] [AIRFLOW-2860] Raise ValueError if timeout < 1 in druid hook --- airflow/hooks/druid_hook.py | 6 +++++- tests/hooks/test_druid_hook.py | 4 ++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py index e5fb6a7a26d91..fbd74e8e47d39 100644 --- a/airflow/hooks/druid_hook.py +++ b/airflow/hooks/druid_hook.py @@ -37,7 +37,8 @@ class DruidHook(BaseHook): which accepts index jobs :type druid_ingest_conn_id: string :param timeout: The interval between polling - the Druid job for the status of the ingestion job + the Druid job for the status of the ingestion job. + Must be greater than or equal to 1 :type timeout: int :param max_ingestion_time: The maximum ingestion time before assuming the job failed :type max_ingestion_time: int @@ -53,6 +54,9 @@ def __init__( self.max_ingestion_time = max_ingestion_time self.header = {'content-type': 'application/json'} + if self.timeout < 1: + raise ValueError("Druid timeout should be equal or greater than 1") + def get_conn_url(self): conn = self.get_connection(self.druid_ingest_conn_id) host = conn.host diff --git a/tests/hooks/test_druid_hook.py b/tests/hooks/test_druid_hook.py index 6fd7b3cc76552..4243343b88227 100644 --- a/tests/hooks/test_druid_hook.py +++ b/tests/hooks/test_druid_hook.py @@ -98,7 +98,7 @@ def test_submit_unknown_response(self, m): @requests_mock.mock() def test_submit_timeout(self, m): - self.db_hook.timeout = 0 + self.db_hook.timeout = 1 self.db_hook.max_ingestion_time = 5 task_post = m.post( 'http://druid-overlord:8081/druid/indexer/v1/task', @@ -131,7 +131,7 @@ def test_get_conn_url(self, mock_get_connection): get_conn_value.port = '1' get_conn_value.extra_dejson = {'endpoint': 'ingest'} mock_get_connection.return_value = get_conn_value - hook = DruidHook(timeout=0, max_ingestion_time=5) + hook = DruidHook(timeout=1, max_ingestion_time=5) self.assertEquals(hook.get_conn_url(), 'https://test_host:1/ingest') From d47c15cfd2c1ed393dfd9d5d35b29f3794831d33 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Sat, 15 Sep 2018 18:54:26 +0100 Subject: [PATCH 075/808] [AIRFLOW-2707] Validate task_log_reader on upgrade from <=1.9 (#3881) We changed the default logging config and config from 1.9 to 1.10, but anyone who upgrades and has an existing airflow.cfg won't know they need to change this value - instead they will get nothing displayed in the UI (ajax request fails) and see "'NoneType' object has no attribute 'read'" in the error log. This validates that config section at start up, and seamlessly upgrades the old previous value. --- airflow/logging_config.py | 33 +++++++++++++++++++++++++++++ tests/test_logging_config.py | 41 ++++++++++++++++++++++++++++++------ 2 files changed, 68 insertions(+), 6 deletions(-) diff --git a/airflow/logging_config.py b/airflow/logging_config.py index 33c2dc82e1be5..84850b4f8244b 100644 --- a/airflow/logging_config.py +++ b/airflow/logging_config.py @@ -18,6 +18,7 @@ # under the License. # import logging +import warnings from logging.config import dictConfig from airflow import configuration as conf @@ -70,4 +71,36 @@ def configure_logging(): # otherwise Airflow would silently fall back on the default config raise e + validate_logging_config(logging_config) + return logging_config + + +def validate_logging_config(logging_config): + # Now lets validate the other logging-related settings + task_log_reader = conf.get('core', 'task_log_reader') + + logger = logging.getLogger('airflow.task') + + def _get_handler(name): + return next((h for h in logger.handlers if h.name == name), None) + + if _get_handler(task_log_reader) is None: + # Check for pre 1.10 setting that might be in deployed airflow.cfg files + if task_log_reader == "file.task" and _get_handler("task"): + warnings.warn( + "task_log_reader setting in [core] has a deprecated value of " + "{!r}, but no handler with this name was found. Please update " + "your config to use {!r}. Running config has been adjusted to " + "match".format( + task_log_reader, + "task", + ), + DeprecationWarning, + ) + conf.set('core', 'task_log_reader', 'task') + else: + raise AirflowConfigException( + "Configured task_log_reader {!r} was not a handler of the 'airflow.task' " + "logger.".format(task_log_reader) + ) diff --git a/tests/test_logging_config.py b/tests/test_logging_config.py index d6f1194f75d67..b568a88cb2422 100644 --- a/tests/test_logging_config.py +++ b/tests/test_logging_config.py @@ -17,16 +17,22 @@ # specific language governing permissions and limitations # under the License. import os -import shutil +import six import sys import tempfile -import unittest from mock import patch, mock from airflow import configuration as conf from airflow.configuration import mkdir_p from airflow.exceptions import AirflowConfigException + +if six.PY2: + # Need `assertWarns` back-ported from unittest2 + import unittest2 as unittest +else: + import unittest + SETTINGS_FILE_VALID = """ LOGGING_CONFIG = { 'version': 1, @@ -41,14 +47,24 @@ 'class': 'logging.StreamHandler', 'formatter': 'airflow.task', 'stream': 'ext://sys.stdout' - } + }, + 'task': { + 'class': 'logging.StreamHandler', + 'formatter': 'airflow.task', + 'stream': 'ext://sys.stdout' + }, }, 'loggers': { 'airflow': { 'handlers': ['console'], 'level': 'INFO', 'propagate': False - } + }, + 'airflow.task': { + 'handlers': ['task'], + 'level': 'INFO', + 'propagate': False, + }, } } """ @@ -200,18 +216,21 @@ def test_loading_no_local_settings(self): # When the key is not available in the configuration def test_when_the_config_key_does_not_exists(self): from airflow import logging_config + conf_get = conf.get def side_effect(*args): if args[1] == 'logging_config_class': raise AirflowConfigException else: - return "bla_bla_from_test" + return conf_get(*args) logging_config.conf.get = mock.Mock(side_effect=side_effect) with patch.object(logging_config.log, 'debug') as mock_debug: logging_config.configure_logging() - mock_debug.assert_any_call('Could not find key logging_config_class in config') + mock_debug.assert_any_call( + 'Could not find key logging_config_class in config' + ) # Just default def test_loading_local_settings_without_logging_config(self): @@ -222,6 +241,16 @@ def test_loading_local_settings_without_logging_config(self): 'Unable to load custom logging, using default config instead' ) + def test_1_9_config(self): + from airflow.logging_config import configure_logging + conf.set('core', 'task_log_reader', 'file.task') + try: + with self.assertWarnsRegex(DeprecationWarning, r'file.task'): + configure_logging() + self.assertEqual(conf.get('core', 'task_log_reader'), 'task') + finally: + conf.remove_option('core', 'task_log_reader', remove_default=False) + if __name__ == '__main__': unittest.main() From af776e034ee6e9279f7e808776b73fae29ddac64 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 17 Sep 2018 20:10:04 +0100 Subject: [PATCH 076/808] [AIRFLOW-1195] Add feature to clear tasks in Parent Dag (#3907) --- airflow/bin/cli.py | 10 ++++++-- airflow/models.py | 26 +++++++++++++++++++ airflow/www/views.py | 8 ++++-- airflow/www_rbac/views.py | 8 ++++-- tests/core.py | 40 ++++++++++++++++++++++++++++- tests/jobs.py | 53 +++++++++++++++++++++++++++++++++++++++ 6 files changed, 138 insertions(+), 7 deletions(-) diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index 5a45a4595dd67..c1c74af136ce9 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -634,7 +634,9 @@ def clear(args): only_failed=args.only_failed, only_running=args.only_running, confirm_prompt=not args.no_confirm, - include_subdags=not args.exclude_subdags) + include_subdags=not args.exclude_subdags, + include_parentdag=not args.exclude_parentdag, + ) def get_num_ready_workers_running(gunicorn_master_proc): @@ -1420,6 +1422,10 @@ class CLIFactory(object): 'exclude_subdags': Arg( ("-x", "--exclude_subdags"), "Exclude subdags", "store_true"), + 'exclude_parentdag': Arg( + ("-xp", "--exclude_parentdag"), + "Exclude ParentDAGS if the task cleared is a part of a SubDAG", + "store_true"), 'dag_regex': Arg( ("-dx", "--dag_regex"), "Search dag_id as regex instead of exact string", "store_true"), @@ -1739,7 +1745,7 @@ class CLIFactory(object): 'args': ( 'dag_id', 'task_regex', 'start_date', 'end_date', 'subdir', 'upstream', 'downstream', 'no_confirm', 'only_failed', - 'only_running', 'exclude_subdags', 'dag_regex'), + 'only_running', 'exclude_subdags', 'exclude_parentdag', 'dag_regex'), }, { 'func': pause, 'help': "Pause a DAG", diff --git a/airflow/models.py b/airflow/models.py index f586139bb1145..8fc259d1b5a29 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -3762,9 +3762,11 @@ def clear( only_running=False, confirm_prompt=False, include_subdags=True, + include_parentdag=True, reset_dag_runs=True, dry_run=False, session=None, + get_tis=False, ): """ Clears a set of task instances associated with the current dag for @@ -3785,6 +3787,25 @@ def clear( tis = session.query(TI).filter(TI.dag_id == self.dag_id) tis = tis.filter(TI.task_id.in_(self.task_ids)) + if include_parentdag and self.is_subdag: + + p_dag = self.parent_dag.sub_dag( + task_regex=self.dag_id.split('.')[1], + include_upstream=False, + include_downstream=True) + + tis = tis.union(p_dag.clear( + start_date=start_date, end_date=end_date, + only_failed=only_failed, + only_running=only_running, + confirm_prompt=confirm_prompt, + include_subdags=include_subdags, + include_parentdag=False, + reset_dag_runs=reset_dag_runs, + get_tis=True, + session=session, + )) + if start_date: tis = tis.filter(TI.execution_date >= start_date) if end_date: @@ -3794,6 +3815,9 @@ def clear( if only_running: tis = tis.filter(TI.state == State.RUNNING) + if get_tis: + return tis + if dry_run: tis = tis.all() session.expunge_all() @@ -3837,6 +3861,7 @@ def clear_dags( only_running=False, confirm_prompt=False, include_subdags=True, + include_parentdag=False, reset_dag_runs=True, dry_run=False, ): @@ -3849,6 +3874,7 @@ def clear_dags( only_running=only_running, confirm_prompt=False, include_subdags=include_subdags, + include_parentdag=include_parentdag, reset_dag_runs=reset_dag_runs, dry_run=True) all_tis.extend(tis) diff --git a/airflow/www/views.py b/airflow/www/views.py index ccbaafbbeb5a5..bdfaa525366c7 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1085,7 +1085,9 @@ def _clear_dag_tis(self, dag, start_date, end_date, origin, count = dag.clear( start_date=start_date, end_date=end_date, - include_subdags=recursive) + include_subdags=recursive, + include_parentdag=recursive, + ) flash("{0} task instances have been cleared".format(count)) return redirect(origin) @@ -1094,7 +1096,9 @@ def _clear_dag_tis(self, dag, start_date, end_date, origin, start_date=start_date, end_date=end_date, include_subdags=recursive, - dry_run=True) + dry_run=True, + include_parentdag=recursive, + ) if not tis: flash("No task instances to clear", 'error') response = redirect(origin) diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index d42cc6fbab33e..3f5ccab5cbdd9 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -768,7 +768,9 @@ def _clear_dag_tis(self, dag, start_date, end_date, origin, count = dag.clear( start_date=start_date, end_date=end_date, - include_subdags=recursive) + include_subdags=recursive, + include_parentdag=recursive, + ) flash("{0} task instances have been cleared".format(count)) return redirect(origin) @@ -777,7 +779,9 @@ def _clear_dag_tis(self, dag, start_date, end_date, origin, start_date=start_date, end_date=end_date, include_subdags=recursive, - dry_run=True) + include_parentdag=recursive, + dry_run=True, + ) if not tis: flash("No task instances to clear", 'error') response = redirect(origin) diff --git a/tests/core.py b/tests/core.py index db17008c6cc51..f00c9aeead2cd 100644 --- a/tests/core.py +++ b/tests/core.py @@ -1316,8 +1316,18 @@ def test_subdag_clear(self): 'clear', 'example_subdag_operator', '--no_confirm', '--exclude_subdags']) cli.clear(args) + def test_parentdag_downstream_clear(self): + args = self.parser.parse_args([ + 'clear', 'example_subdag_operator.section-1', '--no_confirm']) + cli.clear(args) + args = self.parser.parse_args([ + 'clear', 'example_subdag_operator.section-1', '--no_confirm', + '--exclude_parentdag']) + cli.clear(args) + def test_get_dags(self): - dags = cli.get_dags(self.parser.parse_args(['clear', 'example_subdag_operator', '-c'])) + dags = cli.get_dags(self.parser.parse_args(['clear', 'example_subdag_operator', + '-c'])) self.assertEqual(len(dags), 1) dags = cli.get_dags(self.parser.parse_args(['clear', 'subdag', '-dx', '-c'])) @@ -1793,6 +1803,34 @@ def test_dag_views(self): response = self.app.get(url) self.assertIn("Wait a minute", response.data.decode('utf-8')) response = self.app.get(url + "&confirmed=true") + url = ( + "/admin/airflow/clear?task_id=section-1-task-1&" + "dag_id=example_subdag_operator.section-1&future=false&past=false&" + "upstream=false&downstream=true&recursive=true&" + "execution_date={}&" + "origin=/admin".format(DEFAULT_DATE_DS)) + response = self.app.get(url) + self.assertIn("Wait a minute", response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.end", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-1.section-1-task-1", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-1", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2.section-2-task-1", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2.section-2-task-2", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2.section-2-task-3", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2.section-2-task-4", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.section-2.section-2-task-5", + response.data.decode('utf-8')) + self.assertIn("example_subdag_operator.some-other-task", + response.data.decode('utf-8')) url = ( "/admin/airflow/run?task_id=runme_0&" "dag_id=example_bash_operator&ignore_all_deps=false&ignore_ti_state=true&" diff --git a/tests/jobs.py b/tests/jobs.py index 5dd6ff3efda9e..cb82ad1800603 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -899,6 +899,59 @@ def test_backfill_execute_subdag(self): subdag.clear() dag.clear() + def test_subdag_clear_parentdag_downstream_clear(self): + dag = self.dagbag.get_dag('example_subdag_operator') + subdag_op_task = dag.get_task('section-1') + + subdag = subdag_op_task.subdag + subdag.schedule_interval = '@daily' + + executor = TestExecutor(do_update=True) + job = BackfillJob(dag=subdag, + start_date=DEFAULT_DATE, + end_date=DEFAULT_DATE, + executor=executor, + donot_pickle=True) + + with timeout(seconds=30): + job.run() + + ti0 = TI( + task=subdag.get_task('section-1-task-1'), + execution_date=DEFAULT_DATE) + ti0.refresh_from_db() + self.assertEqual(ti0.state, State.SUCCESS) + + sdag = subdag.sub_dag( + task_regex='section-1-task-1', + include_downstream=True, + include_upstream=False) + + sdag.clear( + start_date=DEFAULT_DATE, + end_date=DEFAULT_DATE, + include_parentdag=True) + + ti0.refresh_from_db() + self.assertEquals(State.NONE, ti0.state) + + ti1 = TI( + task=dag.get_task('some-other-task'), + execution_date=DEFAULT_DATE) + self.assertEquals(State.NONE, ti1.state) + + # Checks that all the Downstream tasks for Parent DAG + # have been cleared + for task in subdag_op_task.downstream_list: + ti = TI( + task=dag.get_task(task.task_id), + execution_date=DEFAULT_DATE + ) + self.assertEquals(State.NONE, ti.state) + + subdag.clear() + dag.clear() + def test_backfill_execute_subdag_with_removed_task(self): """ Ensure that subdag operators execute properly in the case where From 25bb0dff64687948f73b3fb86fae4e8476f4f9ce Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 21 Sep 2018 23:30:04 +0100 Subject: [PATCH 077/808] [AIRFLOW-3099] Stop Missing Section Errors for optional sections (#3934) --- airflow/bin/cli.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index c1c74af136ce9..4226d7e6df859 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -464,9 +464,18 @@ def run(args, dag=None): try: conf.set(section, option, value) except NoSectionError: - log.error('Section {section} Option {option} ' - 'does not exist in the config!'.format(section=section, - option=option)) + optional_sections = [ + 'atlas', 'mesos', 'elasticsearch', 'kubernetes', + 'lineage', 'hive' + ] + if section in optional_sections: + log.debug('Section {section} Option {option} ' + 'does not exist in the config!'.format(section=section, + option=option)) + else: + log.error('Section {section} Option {option} ' + 'does not exist in the config!'.format(section=section, + option=option)) settings.configure_vars() From 79d2406dab638be758060a0b076590d00f2fc044 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 4 Sep 2018 10:26:17 +0100 Subject: [PATCH 078/808] [AIRFLOW-XXX] Fix python3 and flake8 errors in dev/airflow-jira This is a script that checks if the Jira's marked as fixed in a release are actually merged in - getting this working is helpful to me in preparing 1.10.1 --- dev/airflow-jira | 61 ++++++++++++++++++++++++------------------------ 1 file changed, 31 insertions(+), 30 deletions(-) diff --git a/dev/airflow-jira b/dev/airflow-jira index 4c9b947a0d49c..27ec99191426f 100755 --- a/dev/airflow-jira +++ b/dev/airflow-jira @@ -39,10 +39,6 @@ TMP_CREDENTIALS = {} PROJECT = "AIRFLOW" # Python 3 compatibility -try: - import urllib2 as urllib -except ImportError: - import urllib.request as urllib if sys.version_info[0] == 3: raw_input = input @@ -52,12 +48,6 @@ except ImportError: print("Could not find the click library. Run 'sudo pip install click' to install.") sys.exit(-1) -try: - import keyring -except ImportError: - print("Could not find the keyring library. Run 'sudo pip install keyring' to install.") - sys.exit(-1) - try: import git except ImportError: @@ -73,11 +63,11 @@ GIT_LOG_FORMAT = '%x1f'.join(GIT_LOG_FORMAT) + '%x1e' def get_jiras_for_version(version): - asf_jira = jira.client.JIRA( - {'server': JIRA_API_BASE}) + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}) - issues = asf_jira.search_issues('PROJECT={} and fixVersion={}'.format(PROJECT, version)) - return issues + return asf_jira.search_issues( + 'PROJECT={} and fixVersion={}'.format(PROJECT, version) + ) def get_merged_issues(version): @@ -97,7 +87,7 @@ def get_merged_issues(version): match = issue_re.match(log_item['subject']) if match: issue_id = match.group(1) - if log_item.has_key('body'): + if 'body' in log_item: match = pr_re.match(log_item['body']) if match: log_item['pull_request'] = match.group(2) @@ -111,6 +101,7 @@ def get_merged_issues(version): return merges + @click.group() def cli(): r""" @@ -127,22 +118,34 @@ def compare(target_version): merges = get_merged_issues(target_version) issues = get_jiras_for_version(target_version) - print("{:<18}|{:<12}||{:<10}||{:<10}|{:<50}|{:<6}|{:<6}|{:<40}" - .format("ISSUE ID", "TYPE", "PRIORITY", - "STATUS", "DESCRIPTION", "MERGED", - "PR", "COMMIT")) + # :<18 says left align, pad to 18 + # :<50.50 truncates after 50 chars + # !s forces as string - some of the Jira objects have a string method, but + # Py3 doesn't call by default + formatstr = "{id:<18}|{typ!s:<12}||{priority!s:<10}||{status!s:<10}|" \ + "{description:<50.50}|{merged:<6}|{pr:<6}|{commit:<40}" + + print(formatstr.format( + id="ISSUE ID", + typ="TYPE", + priority="PRIORITY", + status="STATUS", + description="DESCRIPTION", + merged="MERGED", + pr="PR", + commit="COMMIT")) for issue in issues: is_merged = issue.key in merges - print("{:<18}|{:<12}||{:<10}||{:<10}|{:<50}|{:<6}|{:<6}|{:<40}" - .format(issue.key, - issue.fields.issuetype, - issue.fields.priority, - issue.fields.status, - issue.fields.summary[:50], - is_merged, - merges[issue.key]['pull_request'] if is_merged else "-", - merges[issue.key]['id'] if is_merged else "-")) + print(formatstr.format( + id=issue.key, + typ=issue.fields.issuetype, + priority=issue.fields.priority, + status=issue.fields.status, + description=issue.fields.summary, + merged=is_merged, + pr=merges[issue.key]['pull_request'] if is_merged else "-", + commit=merges[issue.key]['id'] if is_merged else "-")) if __name__ == "__main__": @@ -154,5 +157,3 @@ if __name__ == "__main__": cli() except: raise - - From 17381aeaa29769dc8a2bcd92f442d5fb70034efb Mon Sep 17 00:00:00 2001 From: Trevor Edwards Date: Wed, 19 Sep 2018 11:46:59 -0700 Subject: [PATCH 079/808] [AIRFLOW-1441] Fix inconsistent tutorial code (#2466) --- docs/tutorial.rst | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/tutorial.rst b/docs/tutorial.rst index 1c2dfd69b7c41..0ea58d2784e65 100644 --- a/docs/tutorial.rst +++ b/docs/tutorial.rst @@ -37,7 +37,7 @@ complicated, a line by line explanation follows below. # 'end_date': datetime(2016, 1, 1), } - dag = DAG('tutorial', default_args=default_args) + dag = DAG('tutorial', default_args=default_args, schedule_interval=timedelta(days=1)) # t1, t2 and t3 are examples of tasks created by instantiating operators t1 = BashOperator( @@ -147,7 +147,7 @@ define a ``schedule_interval`` of 1 day for the DAG. .. code:: python dag = DAG( - 'tutorial', default_args=default_args, schedule_interval=timedelta(1)) + 'tutorial', default_args=default_args, schedule_interval=timedelta(days=1)) Tasks ----- @@ -277,8 +277,8 @@ something like this: .. code:: python """ - Code that goes along with the Airflow located at: - http://airflow.readthedocs.org/en/latest/tutorial.html + Code that goes along with the Airflow tutorial located at: + https://github.com/apache/incubator-airflow/blob/master/airflow/example_dags/tutorial.py """ from airflow import DAG from airflow.operators.bash_operator import BashOperator @@ -301,7 +301,7 @@ something like this: } dag = DAG( - 'tutorial', default_args=default_args, schedule_interval=timedelta(1)) + 'tutorial', default_args=default_args, schedule_interval=timedelta(days=1)) # t1, t2 and t3 are examples of tasks created by instantiating operators t1 = BashOperator( From 85c6f300cd4500b44c13ead3d7b35137e28fc893 Mon Sep 17 00:00:00 2001 From: John Cheng Date: Mon, 10 Sep 2018 01:00:54 +0800 Subject: [PATCH 080/808] [AIRFLOW-1917] Trim extra newline and trailing whitespace from log (#3862) --- airflow/utils/log/logging_mixin.py | 2 +- tests/utils/test_logging_mixin.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index 3f696931c955b..85c03f77abe81 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -92,7 +92,7 @@ def write(self, message): self._buffer += message else: self._buffer += message - self.logger.log(self.level, self._buffer) + self.logger.log(self.level, self._buffer.rstrip()) self._buffer = str() def flush(self): diff --git a/tests/utils/test_logging_mixin.py b/tests/utils/test_logging_mixin.py index 74da742891c3c..6d795584d9bfd 100644 --- a/tests/utils/test_logging_mixin.py +++ b/tests/utils/test_logging_mixin.py @@ -86,8 +86,8 @@ def test_write(self): self.assertEqual(log._buffer, msg) - log.write("\n") - logger.log.assert_called_once_with(1, msg + "\n") + log.write(" \n") + logger.log.assert_called_once_with(1, msg) self.assertEqual(log._buffer, "") From 08dc5b201601d75bf77e432ec14599d0580f8184 Mon Sep 17 00:00:00 2001 From: XD-DENG Date: Tue, 31 Jul 2018 10:30:36 +0800 Subject: [PATCH 081/808] [AIRFLOW-2825]Fix S3ToHiveTransfer bug due to case Because upper/lower case was not considered in the file extension check, S3ToHiveTransfer operator may mistakenly think a GZIP file with uppercase ext ".GZ" is not a GZIP file and raise exception. --- airflow/operators/s3_to_hive_operator.py | 2 +- tests/operators/s3_to_hive_operator.py | 20 +++++++++++++++----- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py index 09eb8363c0dc5..5faaf916b7417 100644 --- a/airflow/operators/s3_to_hive_operator.py +++ b/airflow/operators/s3_to_hive_operator.py @@ -153,7 +153,7 @@ def execute(self, context): root, file_ext = os.path.splitext(s3_key_object.key) if (self.select_expression and self.input_compressed and - file_ext != '.gz'): + file_ext.lower() != '.gz'): raise AirflowException("GZIP is the only compression " + "format Amazon S3 Select supports") diff --git a/tests/operators/s3_to_hive_operator.py b/tests/operators/s3_to_hive_operator.py index 482e7fefc82ff..6ca6274a2c173 100644 --- a/tests/operators/s3_to_hive_operator.py +++ b/tests/operators/s3_to_hive_operator.py @@ -89,6 +89,11 @@ def setUp(self): mode="wb") as f_gz_h: self._set_fn(fn_gz, '.gz', True) f_gz_h.writelines([header, line1, line2]) + fn_gz_upper = self._get_fn('.txt', True) + ".GZ" + with gzip.GzipFile(filename=fn_gz_upper, + mode="wb") as f_gz_upper_h: + self._set_fn(fn_gz_upper, '.GZ', True) + f_gz_upper_h.writelines([header, line1, line2]) fn_bz2 = self._get_fn('.txt', True) + '.bz2' with bz2.BZ2File(filename=fn_bz2, mode="wb") as f_bz2_h: @@ -105,6 +110,11 @@ def setUp(self): mode="wb") as f_gz_nh: self._set_fn(fn_gz, '.gz', False) f_gz_nh.writelines([line1, line2]) + fn_gz_upper = self._get_fn('.txt', False) + ".GZ" + with gzip.GzipFile(filename=fn_gz_upper, + mode="wb") as f_gz_upper_nh: + self._set_fn(fn_gz_upper, '.GZ', False) + f_gz_upper_nh.writelines([line1, line2]) fn_bz2 = self._get_fn('.txt', False) + '.bz2' with bz2.BZ2File(filename=fn_bz2, mode="wb") as f_bz2_nh: @@ -143,7 +153,7 @@ def _check_file_equality(self, fn_1, fn_2, ext): # gz files contain mtime and filename in the header that # causes filecmp to return False even if contents are identical # Hence decompress to test for equality - if(ext == '.gz'): + if(ext.lower() == '.gz'): with gzip.GzipFile(fn_1, 'rb') as f_1,\ NamedTemporaryFile(mode='wb') as f_txt_1,\ gzip.GzipFile(fn_2, 'rb') as f_2,\ @@ -220,14 +230,14 @@ def test_execute(self, mock_hiveclihook): conn.create_bucket(Bucket='bucket') # Testing txt, zip, bz2 files with and without header row - for (ext, has_header) in product(['.txt', '.gz', '.bz2'], [True, False]): + for (ext, has_header) in product(['.txt', '.gz', '.bz2', '.GZ'], [True, False]): self.kwargs['headers'] = has_header self.kwargs['check_headers'] = has_header logging.info("Testing {0} format {1} header". format(ext, ('with' if has_header else 'without')) ) - self.kwargs['input_compressed'] = ext != '.txt' + self.kwargs['input_compressed'] = ext.lower() != '.txt' self.kwargs['s3_key'] = 's3://bucket/' + self.s3_key + ext ip_fn = self._get_fn(ext, self.kwargs['headers']) op_fn = self._get_fn(ext, False) @@ -260,8 +270,8 @@ def test_execute_with_select_expression(self, mock_hiveclihook): # Only testing S3ToHiveTransfer calls S3Hook.select_key with # the right parameters and its execute method succeeds here, # since Moto doesn't support select_object_content as of 1.3.2. - for (ext, has_header) in product(['.txt', '.gz'], [True, False]): - input_compressed = ext != '.txt' + for (ext, has_header) in product(['.txt', '.gz', '.GZ'], [True, False]): + input_compressed = ext.lower() != '.txt' key = self.s3_key + ext self.kwargs['check_headers'] = False From b2ac815fcbc9fa8e14aa9aa6b59141ee34ae01a1 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Sun, 5 Aug 2018 11:54:14 +0800 Subject: [PATCH 082/808] [AIRFLOW-2848] Ensure dag_id in metadata "job" for LocalTaskJob (#3693) dag_id is missing for all entries in metadata table "job" with job_type "LocalTaskJob". This is due to that dag_id was not specified within class LocalTaskJob. A test is added to check if essential attributes of LocalTaskJob can be assigned with proper values without intervention. --- airflow/jobs.py | 1 + tests/jobs.py | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/airflow/jobs.py b/airflow/jobs.py index 88aa643c50152..27d6d120d23f6 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -2568,6 +2568,7 @@ def __init__( pool=None, *args, **kwargs): self.task_instance = task_instance + self.dag_id = task_instance.dag_id self.ignore_all_deps = ignore_all_deps self.ignore_depends_on_past = ignore_depends_on_past self.ignore_task_deps = ignore_task_deps diff --git a/tests/jobs.py b/tests/jobs.py index cb82ad1800603..67717acc2b654 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -1099,6 +1099,39 @@ class LocalTaskJobTest(unittest.TestCase): def setUp(self): pass + def test_localtaskjob_essential_attr(self): + """ + Check whether essential attributes + of LocalTaskJob can be assigned with + proper values without intervention + """ + dag = DAG( + 'test_localtaskjob_essential_attr', + start_date=DEFAULT_DATE, + default_args={'owner': 'owner1'}) + + with dag: + op1 = DummyOperator(task_id='op1') + + dag.clear() + dr = dag.create_dagrun(run_id="test", + state=State.SUCCESS, + execution_date=DEFAULT_DATE, + start_date=DEFAULT_DATE) + ti = dr.get_task_instance(task_id=op1.task_id) + + job1 = LocalTaskJob(task_instance=ti, + ignore_ti_state=True, + executor=SequentialExecutor()) + + essential_attr = ["dag_id", "job_type", "start_date", "hostname"] + + check_result_1 = [hasattr(job1, attr) for attr in essential_attr] + self.assertTrue(all(check_result_1)) + + check_result_2 = [getattr(job1, attr) is not None for attr in essential_attr] + self.assertTrue(all(check_result_2)) + @patch('os.getpid') def test_localtaskjob_heartbeat(self, mock_pid): session = settings.Session() From cf067267827ce534463475f721f650c20409cd93 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Thu, 27 Sep 2018 13:10:03 +0100 Subject: [PATCH 083/808] [AIRFLOW-XXX] Pin Click to 6.7 to Fix CI (#3962) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 481c94c8f27e4..f92d3f6463906 100644 --- a/setup.py +++ b/setup.py @@ -221,7 +221,7 @@ def write_version(filename=os.path.join(*['airflow', + cassandra + mongo devel = [ - 'click', + 'click==6.7', 'freezegun', 'jira', 'lxml>=3.3.4', From 8807071d2b0c09ab816d3008aba042b14b8b755e Mon Sep 17 00:00:00 2001 From: Christian Trebing Date: Wed, 20 Jun 2018 20:19:21 +0200 Subject: [PATCH 084/808] [AIRFLOW-2592] Bump bleach dependency Bleach dependency is updated to 2.1.3 to address CVE-2018-7753 Closes #3524 from ctrebing/AIRFLOW-2592-bump- bleach-dependency --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index f92d3f6463906..ab6d06e8ff64b 100644 --- a/setup.py +++ b/setup.py @@ -270,7 +270,7 @@ def do_setup(): scripts=['airflow/bin/airflow'], install_requires=[ 'alembic>=0.8.3, <0.9', - 'bleach==2.1.2', + 'bleach~=2.1.3', 'configparser>=3.5.0, <3.6.0', 'croniter>=0.3.17, <0.4', 'dill>=0.2.2, <0.3', From 13a30ac215e248a25bf0373c548c8fe9f2c285d3 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Fri, 21 Sep 2018 18:24:13 +0800 Subject: [PATCH 085/808] [AIRFLOW-XXX] Fix a wrong sample bash command, a display issue & a few typos (#3924) --- docs/concepts.rst | 6 +++--- docs/howto/executor/use-celery.rst | 2 +- docs/howto/manage-connections.rst | 6 +++--- docs/howto/secure-connections.rst | 2 +- docs/kubernetes.rst | 3 ++- docs/ui.rst | 2 +- 6 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docs/concepts.rst b/docs/concepts.rst index fcaaf99214ae0..f8f5d742f5fcf 100644 --- a/docs/concepts.rst +++ b/docs/concepts.rst @@ -316,7 +316,7 @@ Connections =========== The connection information to external systems is stored in the Airflow -metadata database and managed in the UI (``Menu -> Admin -> Connections``) +metadata database and managed in the UI (``Menu -> Admin -> Connections``). A ``conn_id`` is defined there and hostname / login / password / schema information attached to it. Airflow pipelines can simply refer to the centrally managed ``conn_id`` without having to hard code any of this @@ -338,7 +338,7 @@ See :doc:`howto/manage-connections` for how to create and manage connections. Queues ====== -When using the CeleryExecutor, the celery queues that tasks are sent to +When using the CeleryExecutor, the Celery queues that tasks are sent to can be specified. ``queue`` is an attribute of BaseOperator, so any task can be assigned to any queue. The default queue for the environment is defined in the ``airflow.cfg``'s ``celery -> default_queue``. This defines @@ -346,7 +346,7 @@ the queue that tasks get assigned to when not specified, as well as which queue Airflow workers listen to when started. Workers can listen to one or multiple queues of tasks. When a worker is -started (using the command ``airflow worker``), a set of comma delimited +started (using the command ``airflow worker``), a set of comma-delimited queue names can be specified (e.g. ``airflow worker -q spark``). This worker will then only pick up tasks wired to the specified queue(s). diff --git a/docs/howto/executor/use-celery.rst b/docs/howto/executor/use-celery.rst index fd6db96a95872..0e1bab060b0b2 100644 --- a/docs/howto/executor/use-celery.rst +++ b/docs/howto/executor/use-celery.rst @@ -44,4 +44,4 @@ Some caveats: - Make sure to use a database backed result backend - Make sure to set a visibility timeout in [celery_broker_transport_options] that exceeds the ETA of your longest running task -- Tasks can and consume resources, make sure your worker as enough resources to run `worker_concurrency` tasks +- Tasks can consume resources. Make sure your worker has enough resources to run `worker_concurrency` tasks diff --git a/docs/howto/manage-connections.rst b/docs/howto/manage-connections.rst index f5203157aca8d..f869a08b3c061 100644 --- a/docs/howto/manage-connections.rst +++ b/docs/howto/manage-connections.rst @@ -3,7 +3,7 @@ Managing Connections Airflow needs to know how to connect to your environment. Information such as hostname, port, login and passwords to other systems and services is -handled in the ``Admin->Connection`` section of the UI. The pipeline code you +handled in the ``Admin->Connections`` section of the UI. The pipeline code you will author will reference the 'conn_id' of the Connection objects. .. image:: ../img/connections.png @@ -17,7 +17,7 @@ more information. Creating a Connection with the UI --------------------------------- -Open the ``Admin->Connection`` section of the UI. Click the ``Create`` link +Open the ``Admin->Connections`` section of the UI. Click the ``Create`` link to create a new connection. .. image:: ../img/connection_create.png @@ -34,7 +34,7 @@ to create a new connection. Editing a Connection with the UI -------------------------------- -Open the ``Admin->Connection`` section of the UI. Click the pencil icon next +Open the ``Admin->Connections`` section of the UI. Click the pencil icon next to the connection you wish to edit in the connection list. .. image:: ../img/connection_edit.png diff --git a/docs/howto/secure-connections.rst b/docs/howto/secure-connections.rst index f9e252c4c365a..bb13b1bb08bba 100644 --- a/docs/howto/secure-connections.rst +++ b/docs/howto/secure-connections.rst @@ -26,7 +26,7 @@ variable over the value in ``airflow.cfg``: .. code-block:: bash # Note the double underscores - EXPORT AIRFLOW__CORE__FERNET_KEY = your_fernet_key + export AIRFLOW__CORE__FERNET_KEY=your_fernet_key 4. Restart Airflow webserver. 5. For existing connections (the ones that you had defined before installing ``airflow[crypto]`` and creating a Fernet key), you need to open each connection in the connection admin UI, re-type the password, and save it. diff --git a/docs/kubernetes.rst b/docs/kubernetes.rst index a4916858fc15e..3fe9bd38f690d 100644 --- a/docs/kubernetes.rst +++ b/docs/kubernetes.rst @@ -4,7 +4,8 @@ Kubernetes Executor The kubernetes executor is introduced in Apache Airflow 1.10.0. The Kubernetes executor will create a new pod for every task instance. Example helm charts are available at `scripts/ci/kubernetes/kube/{airflow,volumes,postgres}.yaml` in the source distribution. The volumes are optional and depend on your configuration. There are two volumes available: -- Dags: by storing all the dags onto the persistent disks, all the workers can read the dags from there. Another option is using git-sync, before starting the container, a git pull of the dags repository will be performed and used throughout the lifecycle of the pod/ + +- Dags: by storing all the dags onto the persistent disks, all the workers can read the dags from there. Another option is using git-sync, before starting the container, a git pull of the dags repository will be performed and used throughout the lifecycle of the pod. - Logs: by storing the logs onto a persistent disk, all the logs will be available for all the workers and the webserver itself. If you don't configure this, the logs will be lost after the worker pods shuts down. Another option is to use S3/GCS/etc to store the logs. diff --git a/docs/ui.rst b/docs/ui.rst index 4b232fa1aea18..5a09ce5aa0a92 100644 --- a/docs/ui.rst +++ b/docs/ui.rst @@ -1,6 +1,6 @@ UI / Screenshots ================= -The Airflow UI make it easy to monitor and troubleshoot your data pipelines. +The Airflow UI makes it easy to monitor and troubleshoot your data pipelines. Here's a quick overview of some of the features and visualizations you can find in the Airflow UI. From 9c1030eaa34033e041136e6f9029f274a156a1bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Szymon=20Bili=C5=84ski?= Date: Wed, 19 Sep 2018 16:25:45 +0200 Subject: [PATCH 086/808] [AIRFLOW-XXX] Fix SlackWebhookOperator docs (#3915) The docs refer to `conn_id` while the actual argument is `http_conn_id`. --- airflow/contrib/operators/slack_webhook_operator.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/operators/slack_webhook_operator.py b/airflow/contrib/operators/slack_webhook_operator.py index bab3e90856d34..5263b5c0f2b3f 100644 --- a/airflow/contrib/operators/slack_webhook_operator.py +++ b/airflow/contrib/operators/slack_webhook_operator.py @@ -31,8 +31,8 @@ class SlackWebhookOperator(SimpleHttpOperator): Each Slack webhook token can be pre-configured to use a specific channel, username and icon. You can override these defaults in this hook. - :param conn_id: connection that has Slack webhook token in the extra field - :type conn_id: str + :param http_conn_id: connection that has Slack webhook token in the extra field + :type http_conn_id: str :param webhook_token: Slack webhook token :type webhook_token: str :param message: The message you want to send on Slack From 0f6755361f8cafa2caba5b0a49e9aa3a2ce475ae Mon Sep 17 00:00:00 2001 From: Andy Cooper Date: Tue, 24 Jul 2018 15:57:36 -0400 Subject: [PATCH 087/808] [AIRFLOW-2800] Remove low-hanging linting errors --- airflow/__init__.py | 13 ++--- airflow/contrib/auth/backends/ldap_auth.py | 4 +- airflow/contrib/hooks/aws_hook.py | 2 +- .../contrib/operators/awsbatch_operator.py | 2 +- .../operators/mlengine_prediction_summary.py | 14 +++--- airflow/jobs.py | 50 ++++++++++++------- airflow/migrations/__init__.py | 5 +- airflow/migrations/env.py | 4 +- ...add_dag_id_state_index_on_dag_run_table.py | 5 +- .../13eb55f81627_for_compatibility.py | 4 +- .../1507a7289a2f_create_is_encrypted.py | 4 +- ...e3_add_is_encrypted_column_to_variable_.py | 4 +- .../versions/1b38cef5b76e_add_dagrun.py | 4 +- .../211e584da130_add_ti_state_index.py | 4 +- .../versions/2e541a1dcfed_task_duration.py | 4 +- .../2e82aab8ef20_rename_user_table.py | 5 +- ...0f54d61_more_logging_into_task_isntance.py | 4 +- .../versions/40e67319e3a9_dagrun_config.py | 4 +- .../versions/4446e08588_dagrun_start_end.py | 4 +- ..._add_fractional_seconds_to_mysql_tables.py | 4 +- .../502898887f84_adding_extra_to_log.py | 4 +- .../versions/52d714495f0_job_id_indices.py | 4 +- ...61833c1c74b_add_password_column_to_user.py | 4 +- ...d17757c7a_add_pid_field_to_taskinstance.py | 4 +- ...de9cddf6c9_add_task_fails_journal_table.py | 4 +- .../8504051e801b_xcom_dag_task_indices.py | 4 +- .../947454bf1dff_add_ti_job_id_index.py | 4 +- airflow/migrations/versions/__init__.py | 5 +- ...6_add_a_column_to_track_the_encryption_.py | 4 +- ...dd_notification_sent_column_to_sla_miss.py | 4 +- ...6_make_xcom_value_column_a_large_binary.py | 6 +-- ...7_add_max_tries_column_to_task_instance.py | 4 +- ...ae31099d61_increase_text_size_for_mysql.py | 4 +- .../versions/e3a246e0dc1_current_schema.py | 4 +- .../f2ca10b85618_add_dag_stats_table.py | 4 +- airflow/plugins_manager.py | 6 ++- airflow/settings.py | 6 +-- airflow/utils/__init__.py | 4 +- airflow/utils/asciiart.py | 4 +- airflow/utils/cli_action_loggers.py | 4 +- airflow/utils/compression.py | 4 +- airflow/utils/dag_processing.py | 4 +- airflow/utils/dates.py | 20 ++++---- airflow/utils/db.py | 2 +- airflow/utils/decorators.py | 4 +- airflow/utils/file.py | 4 +- airflow/utils/helpers.py | 4 +- airflow/utils/json.py | 4 +- airflow/utils/log/__init__.py | 4 +- airflow/utils/log/es_task_handler.py | 4 +- airflow/utils/log/file_processor_handler.py | 4 +- airflow/utils/log/file_task_handler.py | 4 +- airflow/utils/log/logging_mixin.py | 4 +- airflow/utils/log/s3_task_handler.py | 6 +-- airflow/utils/net.py | 4 +- airflow/utils/operator_resources.py | 4 +- airflow/utils/sqlalchemy.py | 4 +- airflow/utils/state.py | 4 +- airflow/utils/tests.py | 4 +- airflow/utils/timeout.py | 4 +- airflow/utils/trigger_rule.py | 4 +- airflow/utils/weight_rule.py | 4 +- airflow/www/__init__.py | 5 +- airflow/www/api/__init__.py | 4 +- airflow/www/api/experimental/__init__.py | 4 +- airflow/www/blueprints.py | 4 +- airflow/www/gunicorn_config.py | 4 +- airflow/www/validators.py | 11 ++-- airflow/www/views.py | 44 ++++++++-------- airflow/www_rbac/__init__.py | 4 +- airflow/www_rbac/api/__init__.py | 4 +- airflow/www_rbac/api/experimental/__init__.py | 4 +- airflow/www_rbac/blueprints.py | 4 +- airflow/www_rbac/validators.py | 4 +- airflow/www_rbac/widgets.py | 4 +- 75 files changed, 226 insertions(+), 209 deletions(-) diff --git a/airflow/__init__.py b/airflow/__init__.py index f40b08aab5e77..bc6a7bbe19f36 100644 --- a/airflow/__init__.py +++ b/airflow/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -80,11 +80,12 @@ class AirflowMacroPlugin(object): def __init__(self, namespace): self.namespace = namespace -from airflow import operators + +from airflow import operators # noqa: E402 from airflow import sensors # noqa: E402 -from airflow import hooks -from airflow import executors -from airflow import macros +from airflow import hooks # noqa: E402 +from airflow import executors # noqa: E402 +from airflow import macros # noqa: E402 operators._integrate_plugins() sensors._integrate_plugins() # noqa: E402 diff --git a/airflow/contrib/auth/backends/ldap_auth.py b/airflow/contrib/auth/backends/ldap_auth.py index eefaa1263b250..516e121c9b26d 100644 --- a/airflow/contrib/auth/backends/ldap_auth.py +++ b/airflow/contrib/auth/backends/ldap_auth.py @@ -62,7 +62,7 @@ def get_ldap_connection(dn=None, password=None): cacert = configuration.conf.get("ldap", "cacert") tls_configuration = Tls(validate=ssl.CERT_REQUIRED, ca_certs_file=cacert) use_ssl = True - except: + except Exception: pass server = Server(configuration.conf.get("ldap", "uri"), use_ssl, tls_configuration) @@ -94,7 +94,7 @@ def groups_user(conn, search_base, user_filter, user_name_att, username): search_filter = "(&({0})({1}={2}))".format(user_filter, user_name_att, username) try: memberof_attr = configuration.conf.get("ldap", "group_member_attr") - except: + except Exception: memberof_attr = "memberOf" res = conn.search(native(search_base), native(search_filter), attributes=[native(memberof_attr)]) diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py index c712d2de7910b..5d515d91bbab2 100644 --- a/airflow/contrib/hooks/aws_hook.py +++ b/airflow/contrib/hooks/aws_hook.py @@ -72,7 +72,7 @@ def _parse_s3_config(config_file_name, config_format='boto', profile=None): try: access_key = config.get(cred_section, key_id_option) secret_key = config.get(cred_section, secret_key_option) - except: + except Exception: logging.warning("Option Error in parsing s3 config file") raise return access_key, secret_key diff --git a/airflow/contrib/operators/awsbatch_operator.py b/airflow/contrib/operators/awsbatch_operator.py index d23b44e0d4471..ed7f1ebf36cce 100644 --- a/airflow/contrib/operators/awsbatch_operator.py +++ b/airflow/contrib/operators/awsbatch_operator.py @@ -139,7 +139,7 @@ def _wait_for_task_ended(self): if response['jobs'][-1]['status'] in ['SUCCEEDED', 'FAILED']: retry = False - sleep( 1 + pow(retries * 0.1, 2)) + sleep(1 + pow(retries * 0.1, 2)) retries += 1 def _check_success_task(self): diff --git a/airflow/contrib/operators/mlengine_prediction_summary.py b/airflow/contrib/operators/mlengine_prediction_summary.py index 17fc2c090379e..4efe81e64151f 100644 --- a/airflow/contrib/operators/mlengine_prediction_summary.py +++ b/airflow/contrib/operators/mlengine_prediction_summary.py @@ -112,14 +112,14 @@ def decode(self, x): @beam.ptransform_fn def MakeSummary(pcoll, metric_fn, metric_keys): # pylint: disable=invalid-name return ( - pcoll - | "ApplyMetricFnPerInstance" >> beam.Map(metric_fn) - | "PairWith1" >> beam.Map(lambda tup: tup + (1,)) - | "SumTuple" >> beam.CombineGlobally(beam.combiners.TupleCombineFn( - *([sum] * (len(metric_keys) + 1)))) - | "AverageAndMakeDict" >> beam.Map( + pcoll | + "ApplyMetricFnPerInstance" >> beam.Map(metric_fn) | + "PairWith1" >> beam.Map(lambda tup: tup + (1,)) | + "SumTuple" >> beam.CombineGlobally(beam.combiners.TupleCombineFn( + *([sum] * (len(metric_keys) + 1)))) | + "AverageAndMakeDict" >> beam.Map( lambda tup: dict( - [(name, tup[i]/tup[-1]) for i, name in enumerate(metric_keys)] + + [(name, tup[i] / tup[-1]) for i, name in enumerate(metric_keys)] + [("count", tup[-1])]))) diff --git a/airflow/jobs.py b/airflow/jobs.py index 27d6d120d23f6..69471fd64f890 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -173,7 +173,8 @@ def heartbeat(self): if job.latest_heartbeat: sleep_for = max( 0, - self.heartrate - (timezone.utcnow() - job.latest_heartbeat).total_seconds()) + self.heartrate - ( + timezone.utcnow() - job.latest_heartbeat).total_seconds()) sleep(sleep_for) @@ -391,7 +392,7 @@ def helper(): log.info( "Processing %s took %.3f seconds", file_path, end_time - start_time ) - except: + except Exception: # Log exceptions through the logging framework. log.exception("Got an exception! Propagating...") raise @@ -1087,17 +1088,18 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): session .query(TI) .filter(TI.dag_id.in_(simple_dag_bag.dag_ids)) - .outerjoin(DR, - and_(DR.dag_id == TI.dag_id, - DR.execution_date == TI.execution_date)) - .filter(or_(DR.run_id == None, + .outerjoin( + DR, + and_(DR.dag_id == TI.dag_id, DR.execution_date == TI.execution_date) + ) + .filter(or_(DR.run_id == None, # noqa E711 not_(DR.run_id.like(BackfillJob.ID_PREFIX + '%')))) - .outerjoin(DM, DM.dag_id==TI.dag_id) - .filter(or_(DM.dag_id == None, + .outerjoin(DM, DM.dag_id == TI.dag_id) + .filter(or_(DM.dag_id == None, # noqa E711 not_(DM.is_paused))) ) if None in states: - ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(states))) + ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(states))) # noqa E711 else: ti_query = ti_query.filter(TI.state.in_(states)) @@ -1119,7 +1121,8 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): for task_instance in task_instances_to_examine: pool_to_task_instances[task_instance.pool].append(task_instance) - task_concurrency_map = self.__get_task_concurrency_map(states=states_to_count_as_running, session=session) + task_concurrency_map = self.__get_task_concurrency_map( + states=states_to_count_as_running, session=session) # Go through each pool, and queue up a task for execution if there are # any open slots in the pool. @@ -1190,9 +1193,14 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): ) continue - task_concurrency = simple_dag.get_task_special_arg(task_instance.task_id, 'task_concurrency') + task_concurrency = simple_dag.get_task_special_arg( + task_instance.task_id, + 'task_concurrency') if task_concurrency is not None: - num_running = task_concurrency_map[((task_instance.dag_id, task_instance.task_id))] + num_running = task_concurrency_map[ + ((task_instance.dag_id, task_instance.task_id)) + ] + if num_running >= task_concurrency: self.log.info("Not executing %s since the task concurrency for" " this task has been reached.", task_instance) @@ -1212,7 +1220,8 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): task_instance_str = "\n\t".join( ["{}".format(x) for x in executable_tis]) - self.log.info("Setting the follow tasks to queued state:\n\t%s", task_instance_str) + self.log.info( + "Setting the follow tasks to queued state:\n\t%s", task_instance_str) # so these dont expire on commit for ti in executable_tis: copy_dag_id = ti.dag_id @@ -1254,7 +1263,9 @@ def _change_state_for_executable_task_instances(self, task_instances, .filter(or_(*filter_for_ti_state_change))) if None in acceptable_states: - ti_query = ti_query.filter(or_(TI.state == None, TI.state.in_(acceptable_states))) + ti_query = ti_query.filter( + or_(TI.state == None, TI.state.in_(acceptable_states)) # noqa E711 + ) else: ti_query = ti_query.filter(TI.state.in_(acceptable_states)) @@ -1600,7 +1611,8 @@ def processor_factory(file_path): child.terminate() # TODO: Remove magic number timeout = 5 - self.log.info("Waiting up to %s seconds for processes to exit...", timeout) + self.log.info( + "Waiting up to %s seconds for processes to exit...", timeout) try: psutil.wait_procs( child_processes, timeout=timeout, @@ -1657,7 +1669,9 @@ def _execute_helper(self, processor_manager): self.log.info("Searching for files in %s", self.subdir) known_file_paths = list_py_file_paths(self.subdir) last_dag_dir_refresh_time = timezone.utcnow() - self.log.info("There are %s files in %s", len(known_file_paths), self.subdir) + self.log.info( + "There are %s files in %s", len(known_file_paths), self.subdir) + processor_manager.set_file_paths(known_file_paths) self.log.debug("Removing old import errors") @@ -1670,7 +1684,9 @@ def _execute_helper(self, processor_manager): if self.using_sqlite: # For the sqlite case w/ 1 thread, wait until the processor # is finished to avoid concurrent access to the DB. - self.log.debug("Waiting for processors to finish since we're using sqlite") + self.log.debug( + "Waiting for processors to finish since we're using sqlite") + processor_manager.wait_until_finished() # Send tasks for execution if available diff --git a/airflow/migrations/__init__.py b/airflow/migrations/__init__.py index f0f8b68337da6..114d189da14ab 100644 --- a/airflow/migrations/__init__.py +++ b/airflow/migrations/__init__.py @@ -7,13 +7,12 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - diff --git a/airflow/migrations/env.py b/airflow/migrations/env.py index 828cacf0e0fd6..97ebe4257f2c4 100644 --- a/airflow/migrations/env.py +++ b/airflow/migrations/env.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py b/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py index 58517256949ab..6ee50aa94da81 100644 --- a/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py +++ b/airflow/migrations/versions/127d2bf2dfa7_add_dag_id_state_index_on_dag_run_table.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -39,4 +39,3 @@ def upgrade(): def downgrade(): op.drop_index('dag_id_state', table_name='dag_run') - diff --git a/airflow/migrations/versions/13eb55f81627_for_compatibility.py b/airflow/migrations/versions/13eb55f81627_for_compatibility.py index a14e195bbefe1..b9e88bd01becd 100644 --- a/airflow/migrations/versions/13eb55f81627_for_compatibility.py +++ b/airflow/migrations/versions/13eb55f81627_for_compatibility.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py b/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py index 47bdf13d672d3..3db0d41190408 100644 --- a/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py +++ b/airflow/migrations/versions/1507a7289a2f_create_is_encrypted.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py b/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py index 57ee27aeb4bfb..aaf938a0355ee 100644 --- a/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py +++ b/airflow/migrations/versions/1968acfc09e3_add_is_encrypted_column_to_variable_.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py b/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py index a0af7fa7ad6c2..79fcff454114a 100644 --- a/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py +++ b/airflow/migrations/versions/1b38cef5b76e_add_dagrun.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/211e584da130_add_ti_state_index.py b/airflow/migrations/versions/211e584da130_add_ti_state_index.py index 93347a2649214..afc600d58aca1 100644 --- a/airflow/migrations/versions/211e584da130_add_ti_state_index.py +++ b/airflow/migrations/versions/211e584da130_add_ti_state_index.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/2e541a1dcfed_task_duration.py b/airflow/migrations/versions/2e541a1dcfed_task_duration.py index 7b540430a5dd0..6b24ef66e4ab2 100644 --- a/airflow/migrations/versions/2e541a1dcfed_task_duration.py +++ b/airflow/migrations/versions/2e541a1dcfed_task_duration.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py b/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py index 03266679279f9..75db27cdb3a8c 100644 --- a/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py +++ b/airflow/migrations/versions/2e82aab8ef20_rename_user_table.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -41,4 +41,3 @@ def upgrade(): def downgrade(): op.rename_table('users', 'user') - diff --git a/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py b/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py index c101e4a477230..4f1364b971a57 100644 --- a/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py +++ b/airflow/migrations/versions/338e90f54d61_more_logging_into_task_isntance.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/40e67319e3a9_dagrun_config.py b/airflow/migrations/versions/40e67319e3a9_dagrun_config.py index ecfe58cfc3d1b..02ea51501c49f 100644 --- a/airflow/migrations/versions/40e67319e3a9_dagrun_config.py +++ b/airflow/migrations/versions/40e67319e3a9_dagrun_config.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/4446e08588_dagrun_start_end.py b/airflow/migrations/versions/4446e08588_dagrun_start_end.py index dc546f8c77750..101f2ad9055bf 100644 --- a/airflow/migrations/versions/4446e08588_dagrun_start_end.py +++ b/airflow/migrations/versions/4446e08588_dagrun_start_end.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py index 80af57836771a..c3898e95969b2 100644 --- a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py +++ b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/502898887f84_adding_extra_to_log.py b/airflow/migrations/versions/502898887f84_adding_extra_to_log.py index f5cded7ea5776..333b18e8d200d 100644 --- a/airflow/migrations/versions/502898887f84_adding_extra_to_log.py +++ b/airflow/migrations/versions/502898887f84_adding_extra_to_log.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/52d714495f0_job_id_indices.py b/airflow/migrations/versions/52d714495f0_job_id_indices.py index 43893dba8a081..13f561cffb77f 100644 --- a/airflow/migrations/versions/52d714495f0_job_id_indices.py +++ b/airflow/migrations/versions/52d714495f0_job_id_indices.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py b/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py index ad82c9378b2f1..d92db9f891b6b 100644 --- a/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py +++ b/airflow/migrations/versions/561833c1c74b_add_password_column_to_user.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py b/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py index 5e2963f060665..e23a0a8b2f742 100644 --- a/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py +++ b/airflow/migrations/versions/5e7d17757c7a_add_pid_field_to_taskinstance.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py b/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py index be747383289a9..6dda5df43b146 100644 --- a/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py +++ b/airflow/migrations/versions/64de9cddf6c9_add_task_fails_journal_table.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py b/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py index 47473e318d625..d6a4514ae2883 100644 --- a/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py +++ b/airflow/migrations/versions/8504051e801b_xcom_dag_task_indices.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py b/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py index 4c8bce9b471e7..b821cacedc400 100644 --- a/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py +++ b/airflow/migrations/versions/947454bf1dff_add_ti_job_id_index.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/__init__.py b/airflow/migrations/versions/__init__.py index f0f8b68337da6..114d189da14ab 100644 --- a/airflow/migrations/versions/__init__.py +++ b/airflow/migrations/versions/__init__.py @@ -7,13 +7,12 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - diff --git a/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py b/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py index c780e5830dce6..a19eb58eb5c8d 100644 --- a/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py +++ b/airflow/migrations/versions/bba5a7cfc896_add_a_column_to_track_the_encryption_.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py b/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py index 84c54fad7157a..04cdc22d6cdab 100644 --- a/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py +++ b/airflow/migrations/versions/bbc73705a13e_add_notification_sent_column_to_sla_miss.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py b/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py index 5e06766182f6d..a02eea5519cc6 100644 --- a/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py +++ b/airflow/migrations/versions/bdaa763e6c56_make_xcom_value_column_a_large_binary.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -36,7 +36,7 @@ def upgrade(): - # There can be data truncation here as LargeBinary can be smaller than the pickle + # There can be data truncation here as LargeBinary can be smaller than the pickle # type. # use batch_alter_table to support SQLite workaround diff --git a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py index 3c56f0707b57b..68228f7219a87 100644 --- a/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py +++ b/airflow/migrations/versions/cc1e65623dc7_add_max_tries_column_to_task_instance.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py b/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py index ff67b4fb16d3a..5ecb0d5c72357 100644 --- a/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py +++ b/airflow/migrations/versions/d2ae31099d61_increase_text_size_for_mysql.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/e3a246e0dc1_current_schema.py b/airflow/migrations/versions/e3a246e0dc1_current_schema.py index 6c63d0a9dd337..cfa4147dd925d 100644 --- a/airflow/migrations/versions/e3a246e0dc1_current_schema.py +++ b/airflow/migrations/versions/e3a246e0dc1_current_schema.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py b/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py index 7c23d507127a2..3e3d40abc4384 100644 --- a/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py +++ b/airflow/migrations/versions/f2ca10b85618_add_dag_stats_table.py @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py index 735f2de1e8c79..ad630701dec90 100644 --- a/airflow/plugins_manager.py +++ b/airflow/plugins_manager.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -34,6 +34,7 @@ log = LoggingMixin().log + class AirflowPluginException(Exception): pass @@ -107,6 +108,7 @@ def make_module(name, objects): module.__dict__.update((o.__name__, o) for o in objects) return module + # Plugin components to integrate as modules operators_modules = [] sensors_modules = [] diff --git a/airflow/settings.py b/airflow/settings.py index 984c52ea5d206..9646f035a26be 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -46,7 +46,7 @@ TIMEZONE = pendulum.local_timezone() else: TIMEZONE = pendulum.timezone(tz) -except: +except Exception: pass log.info("Configured default timezone %s" % TIMEZONE) @@ -241,9 +241,9 @@ def configure_action_logging(): try: - from airflow_local_settings import * + from airflow_local_settings import * # noqa F403 F401 log.info("Loaded airflow_local_settings.") -except: +except Exception: pass configure_logging() diff --git a/airflow/utils/__init__.py b/airflow/utils/__init__.py index cc4e12be8033a..e8e889c59b7a5 100644 --- a/airflow/utils/__init__.py +++ b/airflow/utils/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/asciiart.py b/airflow/utils/asciiart.py index faf796c18d4fd..952ebfe1e00c9 100644 --- a/airflow/utils/asciiart.py +++ b/airflow/utils/asciiart.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/cli_action_loggers.py b/airflow/utils/cli_action_loggers.py index e61642d30012c..8b8cda538fdd6 100644 --- a/airflow/utils/cli_action_loggers.py +++ b/airflow/utils/cli_action_loggers.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/compression.py b/airflow/utils/compression.py index f6e923a15619f..2565299175bb0 100644 --- a/airflow/utils/compression.py +++ b/airflow/utils/compression.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index 43948837065e3..89e5701cf15bf 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py index c147a65430da2..216b2ed62667f 100644 --- a/airflow/utils/dates.py +++ b/airflow/utils/dates.py @@ -160,7 +160,7 @@ def round_time(dt, delta, start_date=timezone.make_aware(datetime.min)): # We first search an upper limit for i for which start_date + upper * delta # exceeds dt. upper = 1 - while start_date + upper*delta < dt: + while start_date + upper * delta < dt: # To speed up finding an upper limit we grow this exponentially by a # factor of 2 upper *= 2 @@ -177,20 +177,20 @@ def round_time(dt, delta, start_date=timezone.make_aware(datetime.min)): # Invariant: start + lower * delta < dt <= start + upper * delta # If start_date + (lower + 1)*delta exceeds dt, then either lower or # lower+1 has to be the solution we are searching for - if start_date + (lower + 1)*delta >= dt: + if start_date + (lower + 1) * delta >= dt: # Check if start_date + (lower + 1)*delta or # start_date + lower*delta is closer to dt and return the solution if ( (start_date + (lower + 1) * delta) - dt <= dt - (start_date + lower * delta)): - return start_date + (lower + 1)*delta + return start_date + (lower + 1) * delta else: return start_date + lower * delta # We intersect the interval and either replace the lower or upper # limit with the candidate candidate = lower + (upper - lower) // 2 - if start_date + candidate*delta >= dt: + if start_date + candidate * delta >= dt: upper = candidate else: lower = candidate @@ -209,11 +209,11 @@ def infer_time_unit(time_seconds_arr): if len(time_seconds_arr) == 0: return 'hours' max_time_seconds = max(time_seconds_arr) - if max_time_seconds <= 60*2: + if max_time_seconds <= 60 * 2: return 'seconds' - elif max_time_seconds <= 60*60*2: + elif max_time_seconds <= 60 * 60 * 2: return 'minutes' - elif max_time_seconds <= 24*60*60*2: + elif max_time_seconds <= 24 * 60 * 60 * 2: return 'hours' else: return 'days' @@ -224,11 +224,11 @@ def scale_time_units(time_seconds_arr, unit): Convert an array of time durations in seconds to the specified time unit. """ if unit == 'minutes': - return list(map(lambda x: x*1.0/60, time_seconds_arr)) + return list(map(lambda x: x * 1.0 / 60, time_seconds_arr)) elif unit == 'hours': - return list(map(lambda x: x*1.0/(60*60), time_seconds_arr)) + return list(map(lambda x: x * 1.0 / (60 * 60), time_seconds_arr)) elif unit == 'days': - return list(map(lambda x: x*1.0/(24*60*60), time_seconds_arr)) + return list(map(lambda x: x * 1.0 / (24 * 60 * 60), time_seconds_arr)) return time_seconds_arr diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b5e0c49c61d33..93458c6f61a7b 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -43,7 +43,7 @@ def create_session(): yield session session.expunge_all() session.commit() - except: + except Exception: session.rollback() raise finally: diff --git a/airflow/utils/decorators.py b/airflow/utils/decorators.py index 966d27e30aeae..f1f0ea9a425e9 100644 --- a/airflow/utils/decorators.py +++ b/airflow/utils/decorators.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/file.py b/airflow/utils/file.py index 88938d422b3af..918375fe5e5b5 100644 --- a/airflow/utils/file.py +++ b/airflow/utils/file.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index 911890dc30ab5..db58e650d8ce9 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/json.py b/airflow/utils/json.py index 434926f9d2be8..1767c523d6a3f 100644 --- a/airflow/utils/json.py +++ b/airflow/utils/json.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/__init__.py b/airflow/utils/log/__init__.py index 4067cc78ee9a2..114d189da14ab 100644 --- a/airflow/utils/log/__init__.py +++ b/airflow/utils/log/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/es_task_handler.py b/airflow/utils/log/es_task_handler.py index 04682318655b1..d74aabfbaead6 100644 --- a/airflow/utils/log/es_task_handler.py +++ b/airflow/utils/log/es_task_handler.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/file_processor_handler.py b/airflow/utils/log/file_processor_handler.py index 5e637440a1fda..f39dffe0c9333 100644 --- a/airflow/utils/log/file_processor_handler.py +++ b/airflow/utils/log/file_processor_handler.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 627c8755707ef..113bd254ad0b2 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index 85c03f77abe81..442fecd0363c8 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/log/s3_task_handler.py b/airflow/utils/log/s3_task_handler.py index 07b9b3ec11e8b..bdbf94b6bbbbd 100644 --- a/airflow/utils/log/s3_task_handler.py +++ b/airflow/utils/log/s3_task_handler.py @@ -42,7 +42,7 @@ def _build_hook(self): try: from airflow.hooks.S3_hook import S3Hook return S3Hook(remote_conn_id) - except: + except Exception: self.log.error( 'Could not create an S3Hook with connection id "%s". ' 'Please make sure that airflow[s3] is installed and ' @@ -139,7 +139,7 @@ def s3_read(self, remote_log_location, return_error=False): """ try: return self.hook.read_key(remote_log_location) - except: + except Exception: msg = 'Could not read logs from {}'.format(remote_log_location) self.log.exception(msg) # return error if needed @@ -169,5 +169,5 @@ def s3_write(self, log, remote_log_location, append=True): replace=True, encrypt=configuration.conf.getboolean('core', 'ENCRYPT_S3_LOGS'), ) - except: + except Exception: self.log.exception('Could not write logs to %s', remote_log_location) diff --git a/airflow/utils/net.py b/airflow/utils/net.py index 03e8fde9c90cb..c435496ffc76a 100644 --- a/airflow/utils/net.py +++ b/airflow/utils/net.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/operator_resources.py b/airflow/utils/operator_resources.py index 47ff336991b95..649961340eb42 100644 --- a/airflow/utils/operator_resources.py +++ b/airflow/utils/operator_resources.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 46f4f86eface8..76c112785f852 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/state.py b/airflow/utils/state.py index 70dbc5ca048e8..a351df07b9654 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/tests.py b/airflow/utils/tests.py index 6f29ffc25b2eb..954ac35f786c6 100644 --- a/airflow/utils/tests.py +++ b/airflow/utils/tests.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/timeout.py b/airflow/utils/timeout.py index c5ca5e2f1724f..a86b9d357b5c6 100644 --- a/airflow/utils/timeout.py +++ b/airflow/utils/timeout.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/trigger_rule.py b/airflow/utils/trigger_rule.py index 7bad8ecb78ab5..ae51d6a30134d 100644 --- a/airflow/utils/trigger_rule.py +++ b/airflow/utils/trigger_rule.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/utils/weight_rule.py b/airflow/utils/weight_rule.py index 745cc76970cea..b920ef40229a1 100644 --- a/airflow/utils/weight_rule.py +++ b/airflow/utils/weight_rule.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www/__init__.py b/airflow/www/__init__.py index f0f8b68337da6..114d189da14ab 100644 --- a/airflow/www/__init__.py +++ b/airflow/www/__init__.py @@ -7,13 +7,12 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - diff --git a/airflow/www/api/__init__.py b/airflow/www/api/__init__.py index db5ba598d7c23..b7f8352944d3f 100644 --- a/airflow/www/api/__init__.py +++ b/airflow/www/api/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www/api/experimental/__init__.py b/airflow/www/api/experimental/__init__.py index db5ba598d7c23..b7f8352944d3f 100644 --- a/airflow/www/api/experimental/__init__.py +++ b/airflow/www/api/experimental/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www/blueprints.py b/airflow/www/blueprints.py index ec719b96d2f6a..eb92837fe3b7e 100644 --- a/airflow/www/blueprints.py +++ b/airflow/www/blueprints.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www/gunicorn_config.py b/airflow/www/gunicorn_config.py index 7cf12c56c778e..db76b6de0c5af 100644 --- a/airflow/www/gunicorn_config.py +++ b/airflow/www/gunicorn_config.py @@ -8,9 +8,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www/validators.py b/airflow/www/validators.py index 6eee76a32a8ad..3db8b2850e0ad 100644 --- a/airflow/www/validators.py +++ b/airflow/www/validators.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -45,8 +45,11 @@ def __call__(self, form, field): if field.data < other.data: d = { - 'other_label': hasattr(other, 'label') and other.label.text - or self.fieldname, + 'other_label': ( + hasattr(other, 'label') and + other.label.text or + self.fieldname + ), 'other_name': self.fieldname, } message = self.message diff --git a/airflow/www/views.py b/airflow/www/views.py index bdfaa525366c7..cfd46b871aa7e 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -206,7 +206,7 @@ def nobr_f(v, c, m, p): def label_link(v, c, m, p): try: default_params = ast.literal_eval(m.default_params) - except: + except Exception: default_params = {} url = url_for( 'airflow.chart', chart_id=m.id, iteration_no=m.iteration_no, @@ -404,9 +404,9 @@ def chart_data(self): # Processing templated fields try: args = ast.literal_eval(chart.default_params) - if type(args) is not type(dict()): + if not isinstance(args, dict): raise AirflowException('Not a dict') - except: + except Exception: args = {} payload['error'] += ( "Default params is not valid, string has to evaluate as " @@ -448,15 +448,15 @@ def chart_data(self): if not payload['error'] and len(df) == 0: payload['error'] += "Empty result set. " elif ( - not payload['error'] and - chart.sql_layout == 'series' and - chart.chart_type != "datatable" and - len(df.columns) < 3): + not payload['error'] and + chart.sql_layout == 'series' and + chart.chart_type != "datatable" and + len(df.columns) < 3): payload['error'] += "SQL needs to return at least 3 columns. " elif ( - not payload['error'] and - chart.sql_layout == 'columns' and - len(df.columns) < 2): + not payload['error'] and + chart.sql_layout == 'columns' and + len(df.columns) < 2): payload['error'] += "SQL needs to return at least 2 columns. " elif not payload['error']: import numpy as np @@ -622,13 +622,13 @@ def task_stats(self, session=None): # If no dag_run is active, return task instances from most recent dag_run. LastTI = ( session.query(TI.dag_id.label('dag_id'), TI.state.label('state')) - .join(LastDagRun, and_( + .join(LastDagRun, and_( LastDagRun.c.dag_id == TI.dag_id, LastDagRun.c.execution_date == TI.execution_date)) ) RunningTI = ( session.query(TI.dag_id.label('dag_id'), TI.state.label('state')) - .join(RunningDagRun, and_( + .join(RunningDagRun, and_( RunningDagRun.c.dag_id == TI.dag_id, RunningDagRun.c.execution_date == TI.execution_date)) ) @@ -636,7 +636,7 @@ def task_stats(self, session=None): UnionTI = union_all(LastTI, RunningTI).alias('union_ti') qry = ( session.query(UnionTI.c.dag_id, UnionTI.c.state, sqla.func.count()) - .group_by(UnionTI.c.dag_id, UnionTI.c.state) + .group_by(UnionTI.c.dag_id, UnionTI.c.state) ) data = {} @@ -652,7 +652,7 @@ def task_stats(self, session=None): for state in State.task_states: try: count = data[dag.dag_id][state] - except: + except Exception: count = 0 d = { 'state': state, @@ -1540,13 +1540,13 @@ def duration(self, session=None): TF = models.TaskFail ti_fails = ( session - .query(TF) - .filter( + .query(TF) + .filter( TF.dag_id == dag.dag_id, TF.execution_date >= min_date, TF.execution_date <= base_date, TF.task_id.in_([t.task_id for t in dag.tasks])) - .all() + .all() ) fails_totals = defaultdict(int) @@ -1892,7 +1892,7 @@ def variables(self, form): return self.render( 'airflow/variables/{}.html'.format(form) ) - except: + except Exception: # prevent XSS form = escape(form) return ("Error: form airflow/variables/{}.html " @@ -2463,7 +2463,7 @@ def action_varexport(self, ids, session=None): val = None try: val = d.decode(var.val) - except: + except Exception: val = var.val var_dict[var.key] = val @@ -2802,7 +2802,7 @@ def alert_fernet_key(cls): fk = None try: fk = conf.get('core', 'fernet_key') - except: + except Exception: pass return fk is None @@ -2814,10 +2814,10 @@ def is_secure(cls): """ is_secure = False try: - import cryptography + import cryptography # noqa F401 conf.get('core', 'fernet_key') is_secure = True - except: + except Exception: pass return is_secure diff --git a/airflow/www_rbac/__init__.py b/airflow/www_rbac/__init__.py index 4067cc78ee9a2..114d189da14ab 100644 --- a/airflow/www_rbac/__init__.py +++ b/airflow/www_rbac/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www_rbac/api/__init__.py b/airflow/www_rbac/api/__init__.py index db5ba598d7c23..b7f8352944d3f 100644 --- a/airflow/www_rbac/api/__init__.py +++ b/airflow/www_rbac/api/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www_rbac/api/experimental/__init__.py b/airflow/www_rbac/api/experimental/__init__.py index db5ba598d7c23..b7f8352944d3f 100644 --- a/airflow/www_rbac/api/experimental/__init__.py +++ b/airflow/www_rbac/api/experimental/__init__.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www_rbac/blueprints.py b/airflow/www_rbac/blueprints.py index 35a5badb6ad1e..75bbb93aeadc9 100644 --- a/airflow/www_rbac/blueprints.py +++ b/airflow/www_rbac/blueprints.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www_rbac/validators.py b/airflow/www_rbac/validators.py index 7746e74338023..62b7db82523bc 100644 --- a/airflow/www_rbac/validators.py +++ b/airflow/www_rbac/validators.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/airflow/www_rbac/widgets.py b/airflow/www_rbac/widgets.py index 3bca3869ce7ee..ea61d3ff94004 100644 --- a/airflow/www_rbac/widgets.py +++ b/airflow/www_rbac/widgets.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY From 9541e7c48fc602f0acc5ffe3b1170eb0d7081f58 Mon Sep 17 00:00:00 2001 From: Dan Fowler Date: Tue, 31 Jul 2018 13:25:04 -0700 Subject: [PATCH 088/808] [AIRFLOW-1104] Update jobs.py so Airflow does not over schedule tasks (#3568) This change will prevent tasks from getting scheduled and queued over the concurrency limits set for the dag --- airflow/jobs.py | 5 +---- tests/jobs.py | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index 69471fd64f890..f2caeda137721 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1073,9 +1073,6 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): :type states: Tuple[State] :return: List[TaskInstance] """ - # TODO(saguziel): Change this to include QUEUED, for concurrency - # purposes we may want to count queued tasks - states_to_count_as_running = [State.RUNNING] executable_tis = [] # Get all the queued task instances from associated with scheduled @@ -1121,6 +1118,7 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): for task_instance in task_instances_to_examine: pool_to_task_instances[task_instance.pool].append(task_instance) + states_to_count_as_running = [State.RUNNING, State.QUEUED] task_concurrency_map = self.__get_task_concurrency_map( states=states_to_count_as_running, session=session) @@ -1171,7 +1169,6 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): simple_dag = simple_dag_bag.get_dag(dag_id) if dag_id not in dag_id_to_possibly_running_task_count: - # TODO(saguziel): also check against QUEUED state, see AIRFLOW-1104 dag_id_to_possibly_running_task_count[dag_id] = \ DAG.get_num_task_instances( dag_id, diff --git a/tests/jobs.py b/tests/jobs.py index 67717acc2b654..c6297af2459f8 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -1575,6 +1575,39 @@ def test_find_executable_task_instances_concurrency(self): self.assertEqual(0, len(res)) + def test_find_executable_task_instances_concurrency_queued(self): + dag_id = 'SchedulerJobTest.test_find_executable_task_instances_concurrency_queued' + dag = DAG(dag_id=dag_id, start_date=DEFAULT_DATE, concurrency=3) + task1 = DummyOperator(dag=dag, task_id='dummy1') + task2 = DummyOperator(dag=dag, task_id='dummy2') + task3 = DummyOperator(dag=dag, task_id='dummy3') + dagbag = self._make_simple_dag_bag([dag]) + + scheduler = SchedulerJob() + session = settings.Session() + dag_run = scheduler.create_dag_run(dag) + + ti1 = TI(task1, dag_run.execution_date) + ti2 = TI(task2, dag_run.execution_date) + ti3 = TI(task3, dag_run.execution_date) + ti1.state = State.RUNNING + ti2.state = State.QUEUED + ti3.state = State.SCHEDULED + + session.merge(ti1) + session.merge(ti2) + session.merge(ti3) + + session.commit() + + res = scheduler._find_executable_task_instances( + dagbag, + states=[State.SCHEDULED], + session=session) + + self.assertEqual(1, len(res)) + self.assertEqual(res[0].key, ti3.key) + def test_find_executable_task_instances_task_concurrency(self): dag_id = 'SchedulerJobTest.test_find_executable_task_instances_task_concurrency' task_id_1 = 'dummy' From 630b1c13f6363a2199165eaece9fae11d9f9d559 Mon Sep 17 00:00:00 2001 From: Yingbo Wang Date: Wed, 3 Oct 2018 06:32:55 -0700 Subject: [PATCH 089/808] [AIRFLOW-2951] Update dag_run table end_date when state change (#3990) The existing airflow only change dag_run table end_date value when a user teminate a dag in web UI. The end_date will not be updated if airflow detected a dag finished and updated its state. This commit add end_date update in DagRun's set_state function to make up tho problem mentioned above. --- airflow/models.py | 10 ++-- tests/models.py | 118 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 124 insertions(+), 4 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 8fc259d1b5a29..428923ff9e73f 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -4845,6 +4845,8 @@ def get_state(self): def set_state(self, state): if self._state != state: self._state = state + self.end_date = timezone.utcnow() if self._state in State.finished() else None + if self.dag_id is not None: # FIXME: Due to the scoped_session factor we we don't get a clean # session here, so something really weird goes on: @@ -5068,7 +5070,7 @@ def update_state(self, session=None): if (not unfinished_tasks and any(r.state in (State.FAILED, State.UPSTREAM_FAILED) for r in roots)): self.log.info('Marking run %s failed', self) - self.state = State.FAILED + self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='task_failure', session=session) @@ -5076,20 +5078,20 @@ def update_state(self, session=None): elif not unfinished_tasks and all(r.state in (State.SUCCESS, State.SKIPPED) for r in roots): self.log.info('Marking run %s successful', self) - self.state = State.SUCCESS + self.set_state(State.SUCCESS) dag.handle_callback(self, success=True, reason='success', session=session) # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and no_dependencies_met): self.log.info('Deadlock; marking run %s failed', self) - self.state = State.FAILED + self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) # finally, if the roots aren't done, the dag is still running else: - self.state = State.RUNNING + self.set_state(State.RUNNING) # todo: determine we want to use with_for_update to make sure to lock the run session.merge(self) diff --git a/tests/models.py b/tests/models.py index 60aee3c84fcb1..55fa41bd90bab 100644 --- a/tests/models.py +++ b/tests/models.py @@ -915,6 +915,124 @@ def on_failure_callable(context): updated_dag_state = dag_run.update_state() self.assertEqual(State.FAILED, updated_dag_state) + def test_dagrun_set_state_end_date(self): + session = settings.Session() + + dag = DAG( + 'test_dagrun_set_state_end_date', + start_date=DEFAULT_DATE, + default_args={'owner': 'owner1'}) + + dag.clear() + + now = timezone.utcnow() + dr = dag.create_dagrun(run_id='test_dagrun_set_state_end_date', + state=State.RUNNING, + execution_date=now, + start_date=now) + + # Initial end_date should be NULL + # State.SUCCESS and State.FAILED are all ending state and should set end_date + # State.RUNNING set end_date back to NULL + session.add(dr) + session.commit() + self.assertIsNone(dr.end_date) + + dr.set_state(State.SUCCESS) + session.merge(dr) + session.commit() + + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_set_state_end_date' + ).one() + self.assertIsNotNone(dr_database.end_date) + self.assertEqual(dr.end_date, dr_database.end_date) + + dr.set_state(State.RUNNING) + session.merge(dr) + session.commit() + + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_set_state_end_date' + ).one() + + self.assertIsNone(dr_database.end_date) + + dr.set_state(State.FAILED) + session.merge(dr) + session.commit() + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_set_state_end_date' + ).one() + + self.assertIsNotNone(dr_database.end_date) + self.assertEqual(dr.end_date, dr_database.end_date) + + def test_dagrun_update_state_end_date(self): + session = settings.Session() + + dag = DAG( + 'test_dagrun_update_state_end_date', + start_date=DEFAULT_DATE, + default_args={'owner': 'owner1'}) + + # A -> B + with dag: + op1 = DummyOperator(task_id='A') + op2 = DummyOperator(task_id='B') + op1.set_upstream(op2) + + dag.clear() + + now = timezone.utcnow() + dr = dag.create_dagrun(run_id='test_dagrun_update_state_end_date', + state=State.RUNNING, + execution_date=now, + start_date=now) + + # Initial end_date should be NULL + # State.SUCCESS and State.FAILED are all ending state and should set end_date + # State.RUNNING set end_date back to NULL + session.merge(dr) + session.commit() + self.assertIsNone(dr.end_date) + + ti_op1 = dr.get_task_instance(task_id=op1.task_id) + ti_op1.set_state(state=State.SUCCESS, session=session) + ti_op2 = dr.get_task_instance(task_id=op2.task_id) + ti_op2.set_state(state=State.SUCCESS, session=session) + + dr.update_state() + + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_update_state_end_date' + ).one() + self.assertIsNotNone(dr_database.end_date) + self.assertEqual(dr.end_date, dr_database.end_date) + + ti_op1.set_state(state=State.RUNNING, session=session) + ti_op2.set_state(state=State.RUNNING, session=session) + dr.update_state() + + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_update_state_end_date' + ).one() + + self.assertEqual(dr._state, State.RUNNING) + self.assertIsNone(dr.end_date) + self.assertIsNone(dr_database.end_date) + + ti_op1.set_state(state=State.FAILED, session=session) + ti_op2.set_state(state=State.FAILED, session=session) + dr.update_state() + + dr_database = session.query(DagRun).filter( + DagRun.run_id == 'test_dagrun_update_state_end_date' + ).one() + + self.assertIsNotNone(dr_database.end_date) + self.assertEqual(dr.end_date, dr_database.end_date) + def test_get_task_instance_on_empty_dagrun(self): """ Make sure that a proper value is returned when a dagrun has no task instances From 3dfb5b8c2dfe63576b2d2b848d7a4aef2d58efab Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Wed, 8 Aug 2018 09:07:43 +0800 Subject: [PATCH 090/808] [AIRFLOW-2855] Check Cron Expression Validity in DagBag.process_file() (#3698) A DAG can be imported as a .py script properly, but the Cron expression inside as "schedule_interval" may be invalid, like "0 100 * * *". This commit helps check the validity of Cron expression in DAG files (.py) and packaged DAG files (.zip), and help show exception messages in web UI by add these exceptions into metadata "import_error". --- airflow/models.py | 14 ++++++++++- tests/dags/test_invalid_cron.py | 35 +++++++++++++++++++++++++++ tests/dags/test_zip_invalid_cron.zip | Bin 0 -> 1389 bytes tests/models.py | 15 +++++++++++- 4 files changed, 62 insertions(+), 2 deletions(-) create mode 100755 tests/dags/test_invalid_cron.py create mode 100644 tests/dags/test_zip_invalid_cron.zip diff --git a/airflow/models.py b/airflow/models.py index 428923ff9e73f..0207ac25252dc 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -61,7 +61,9 @@ from sqlalchemy.ext.declarative import declarative_base, declared_attr from sqlalchemy.orm import reconstructor, relationship, synonym -from croniter import croniter +from croniter import ( + croniter, CroniterBadCronError, CroniterBadDateError, CroniterNotAlphaError +) import six from airflow import settings, utils @@ -413,8 +415,18 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): try: dag.is_subdag = False self.bag_dag(dag, parent_dag=dag, root_dag=dag) + if isinstance(dag._schedule_interval, six.string_types): + croniter(dag._schedule_interval) found_dags.append(dag) found_dags += dag.subdags + except (CroniterBadCronError, + CroniterBadDateError, + CroniterNotAlphaError) as cron_e: + self.log.exception("Failed to bag_dag: %s", dag.full_filepath) + self.import_errors[dag.full_filepath] = \ + "Invalid Cron expression: " + str(cron_e) + self.file_last_changed[dag.full_filepath] = \ + file_last_changed_on_disk except AirflowDagCycleException as cycle_exception: self.log.exception("Failed to bag_dag: %s", dag.full_filepath) self.import_errors[dag.full_filepath] = str(cycle_exception) diff --git a/tests/dags/test_invalid_cron.py b/tests/dags/test_invalid_cron.py new file mode 100755 index 0000000000000..51a0e43cb500a --- /dev/null +++ b/tests/dags/test_invalid_cron.py @@ -0,0 +1,35 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.models import DAG +from airflow.operators.dummy_operator import DummyOperator +from airflow.utils.timezone import datetime + +# The schedule_interval specified here is an INVALID +# Cron expression. This invalid DAG will be used to +# test whether dagbag.process_file() can identify +# invalid Cron expression. +dag1 = DAG( + dag_id='test_invalid_cron', + start_date=datetime(2015, 1, 1), + schedule_interval="0 100 * * *") +dag1_task1 = DummyOperator( + task_id='task1', + dag=dag1, + owner='airflow') diff --git a/tests/dags/test_zip_invalid_cron.zip b/tests/dags/test_zip_invalid_cron.zip new file mode 100644 index 0000000000000000000000000000000000000000..fe45153abe85a1b22026bc3f3c36b09d6b06d585 GIT binary patch literal 1389 zcmWIWW@Zs#-~hr?ovgkLNI--^fT1L{xFkL^uPiYqGbKK`C_hiHpfWUqhk@llTJ-Dv zY0));&!?w`#OB{}5UBaSK4K!f0u%E@tv}}5CSEy1-neSBJRwQlaBuA z49Ut~X1O+0RXDA6k_p$MuQLjs%$o9qLEE?0DQL}-tB(RASw3x3*yvLnSCHB3FH(3f z1!?U_emC8T3%n)n)wsnL15)S9Bm-MGB67m*$D7AS`i_3zH42I>E$F>MJAA9QA z{U*cn>ZI&q5!GF9=k#UV4K~cZyH2xYUdYvx7QuHU4Hvp!DJ)w5$Fg?$JWU;kG92zVYxJmn5qVdaCtvnO zCnjS++=ETcir}llomX=e@UTb#Tso zw(1-A)DL$o#Jr-TSnciNw5O!ssCuj(8aU_M-=FdaT(`%5P~Ngh$$XzE>;LBCSC~Is z+0k7ocVnlmnM8Te(;EFHPk)K#FAV0soEU1%p{9AqhOPV|*8=I@$PzZErTtnSf?y|v#md<BdUw+e@^-z`HP-iiUwk=Q?A@xR4~uqPW6+h|U~5{ha5~X^-oqQexdXh}Ij+|} zRJ+c^z+lG8zz_h;C|uy|!P|wJQ8Dos`-UeEi*H>bM|EwW8-4w(<<4|dlvn<6OygYY%Cn^?(7`P zt!|tgD_oo|9?Pw4>>eBLZqCgeY_80mt}e_53^R6)=^4?5kRO Date: Sun, 19 Aug 2018 22:44:56 +0800 Subject: [PATCH 091/808] [AIRFLOW-2904] Remove unnecessary line in celery_executor (#3753) Clean an unnecessary line in celery_executor.py --- airflow/executors/celery_executor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py index 6ada005ab77d2..7916eda122c57 100644 --- a/airflow/executors/celery_executor.py +++ b/airflow/executors/celery_executor.py @@ -31,8 +31,6 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.module_loading import import_string -PARALLELISM = configuration.conf.get('core', 'PARALLELISM') - ''' To start the celery worker, run the command: airflow worker From e098dec8050b5f82e8980cb60640fc16eac77f37 Mon Sep 17 00:00:00 2001 From: Victor Jimenez Date: Wed, 29 Aug 2018 09:26:29 +0200 Subject: [PATCH 092/808] [AIRFLOW-2709] Improve error handling in Databricks hook (#3570) * Use float for default value * Use status code to determine whether an error is retryable * Fix wrong type in assertion * Fix style to prevent lines from exceeding 90 characters * Fix wrong way of checking exception type --- airflow/contrib/hooks/databricks_hook.py | 49 ++++-- .../contrib/operators/databricks_operator.py | 8 +- tests/contrib/hooks/test_databricks_hook.py | 144 ++++++++++++++---- .../operators/test_databricks_operator.py | 10 +- 4 files changed, 162 insertions(+), 49 deletions(-) diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py index 2e5f1399b4765..802b8efb87842 100644 --- a/airflow/contrib/hooks/databricks_hook.py +++ b/airflow/contrib/hooks/databricks_hook.py @@ -24,6 +24,7 @@ from airflow.hooks.base_hook import BaseHook from requests import exceptions as requests_exceptions from requests.auth import AuthBase +from time import sleep from airflow.utils.log.logging_mixin import LoggingMixin @@ -47,7 +48,8 @@ def __init__( self, databricks_conn_id='databricks_default', timeout_seconds=180, - retry_limit=3): + retry_limit=3, + retry_delay=1.0): """ :param databricks_conn_id: The name of the databricks connection to use. :type databricks_conn_id: string @@ -57,6 +59,9 @@ def __init__( :param retry_limit: The number of times to retry the connection in case of service outages. :type retry_limit: int + :param retry_delay: The number of seconds to wait between retries (it + might be a floating point number). + :type retry_delay: float """ self.databricks_conn_id = databricks_conn_id self.databricks_conn = self.get_connection(databricks_conn_id) @@ -64,6 +69,7 @@ def __init__( if retry_limit < 1: raise ValueError('Retry limit must be greater than equal to 1') self.retry_limit = retry_limit + self.retry_delay = retry_delay def _parse_host(self, host): """ @@ -118,7 +124,8 @@ def _do_api_call(self, endpoint_info, json): else: raise AirflowException('Unexpected HTTP Method: ' + method) - for attempt_num in range(1, self.retry_limit + 1): + attempt_num = 1 + while True: try: response = request_func( url, @@ -126,21 +133,29 @@ def _do_api_call(self, endpoint_info, json): auth=auth, headers=USER_AGENT_HEADER, timeout=self.timeout_seconds) - if response.status_code == requests.codes.ok: - return response.json() - else: + response.raise_for_status() + return response.json() + except requests_exceptions.RequestException as e: + if not _retryable_error(e): # In this case, the user probably made a mistake. # Don't retry. raise AirflowException('Response: {0}, Status Code: {1}'.format( - response.content, response.status_code)) - except (requests_exceptions.ConnectionError, - requests_exceptions.Timeout) as e: - self.log.error( - 'Attempt %s API Request to Databricks failed with reason: %s', - attempt_num, e - ) - raise AirflowException(('API requests to Databricks failed {} times. ' + - 'Giving up.').format(self.retry_limit)) + e.response.content, e.response.status_code)) + + self._log_request_error(attempt_num, e) + + if attempt_num == self.retry_limit: + raise AirflowException(('API requests to Databricks failed {} times. ' + + 'Giving up.').format(self.retry_limit)) + + attempt_num += 1 + sleep(self.retry_delay) + + def _log_request_error(self, attempt_num, error): + self.log.error( + 'Attempt %s API Request to Databricks failed with reason: %s', + attempt_num, error + ) def submit_run(self, json): """ @@ -174,6 +189,12 @@ def cancel_run(self, run_id): self._do_api_call(CANCEL_RUN_ENDPOINT, json) +def _retryable_error(exception): + return isinstance(exception, requests_exceptions.ConnectionError) \ + or isinstance(exception, requests_exceptions.Timeout) \ + or exception.response is not None and exception.response.status_code >= 500 + + RUN_LIFE_CYCLE_STATES = [ 'PENDING', 'RUNNING', diff --git a/airflow/contrib/operators/databricks_operator.py b/airflow/contrib/operators/databricks_operator.py index 7b8d522dba85b..3245a99256502 100644 --- a/airflow/contrib/operators/databricks_operator.py +++ b/airflow/contrib/operators/databricks_operator.py @@ -146,6 +146,9 @@ class DatabricksSubmitRunOperator(BaseOperator): :param databricks_retry_limit: Amount of times retry if the Databricks backend is unreachable. Its value must be greater than or equal to 1. :type databricks_retry_limit: int + :param databricks_retry_delay: Number of seconds to wait between retries (it + might be a floating point number). + :type databricks_retry_delay: float :param do_xcom_push: Whether we should push run_id and run_page_url to xcom. :type do_xcom_push: boolean """ @@ -168,6 +171,7 @@ def __init__( databricks_conn_id='databricks_default', polling_period_seconds=30, databricks_retry_limit=3, + databricks_retry_delay=1, do_xcom_push=False, **kwargs): """ @@ -178,6 +182,7 @@ def __init__( self.databricks_conn_id = databricks_conn_id self.polling_period_seconds = polling_period_seconds self.databricks_retry_limit = databricks_retry_limit + self.databricks_retry_delay = databricks_retry_delay if spark_jar_task is not None: self.json['spark_jar_task'] = spark_jar_task if notebook_task is not None: @@ -232,7 +237,8 @@ def _log_run_page_url(self, url): def get_hook(self): return DatabricksHook( self.databricks_conn_id, - retry_limit=self.databricks_retry_limit) + retry_limit=self.databricks_retry_limit, + retry_delay=self.databricks_retry_delay) def execute(self, context): hook = self.get_hook() diff --git a/tests/contrib/hooks/test_databricks_hook.py b/tests/contrib/hooks/test_databricks_hook.py index aca8dd96004b4..a022431899b4d 100644 --- a/tests/contrib/hooks/test_databricks_hook.py +++ b/tests/contrib/hooks/test_databricks_hook.py @@ -18,15 +18,21 @@ # under the License. # +import itertools import json import unittest +from requests import exceptions as requests_exceptions + from airflow import __version__ -from airflow.contrib.hooks.databricks_hook import DatabricksHook, RunState, SUBMIT_RUN_ENDPOINT, _TokenAuth +from airflow.contrib.hooks.databricks_hook import ( + DatabricksHook, + RunState, + SUBMIT_RUN_ENDPOINT +) from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.utils import db -from requests import exceptions as requests_exceptions try: from unittest import mock @@ -79,12 +85,48 @@ def get_run_endpoint(host): """ return 'https://{}/api/2.0/jobs/runs/get'.format(host) + def cancel_run_endpoint(host): """ Utility function to generate the get run endpoint given the host. """ return 'https://{}/api/2.0/jobs/runs/cancel'.format(host) + +def create_valid_response_mock(content): + response = mock.MagicMock() + response.json.return_value = content + return response + + +def create_post_side_effect(exception, status_code=500): + if exception != requests_exceptions.HTTPError: + return exception() + else: + response = mock.MagicMock() + response.status_code = status_code + response.raise_for_status.side_effect = exception(response=response) + return response + + +def setup_mock_requests( + mock_requests, + exception, + status_code=500, + error_count=None, + response_content=None): + + side_effect = create_post_side_effect(exception, status_code) + + if error_count is None: + # POST requests will fail indefinitely + mock_requests.post.side_effect = itertools.repeat(side_effect) + else: + # POST requests will fail 'error_count' times, and then they will succeed (once) + mock_requests.post.side_effect = \ + [side_effect] * error_count + [create_valid_response_mock(response_content)] + + class DatabricksHookTest(unittest.TestCase): """ Tests for DatabricksHook. @@ -99,7 +141,7 @@ def setUp(self, session=None): conn.password = PASSWORD session.commit() - self.hook = DatabricksHook() + self.hook = DatabricksHook(retry_delay=0) def test_parse_host_with_proper_host(self): host = self.hook._parse_host(HOST) @@ -111,34 +153,85 @@ def test_parse_host_with_scheme(self): def test_init_bad_retry_limit(self): with self.assertRaises(ValueError): - DatabricksHook(retry_limit = 0) - - @mock.patch('airflow.contrib.hooks.databricks_hook.requests') - def test_do_api_call_with_error_retry(self, mock_requests): - for exception in [requests_exceptions.ConnectionError, requests_exceptions.Timeout]: - with mock.patch.object(self.hook.log, 'error') as mock_errors: - mock_requests.reset_mock() - mock_requests.post.side_effect = exception() + DatabricksHook(retry_limit=0) + + def test_do_api_call_retries_with_retryable_error(self): + for exception in [ + requests_exceptions.ConnectionError, + requests_exceptions.SSLError, + requests_exceptions.Timeout, + requests_exceptions.ConnectTimeout, + requests_exceptions.HTTPError]: + with mock.patch( + 'airflow.contrib.hooks.databricks_hook.requests') as mock_requests, \ + mock.patch.object(self.hook.log, 'error') as mock_errors: + setup_mock_requests(mock_requests, exception) with self.assertRaises(AirflowException): self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {}) - self.assertEquals(len(mock_errors.mock_calls), self.hook.retry_limit) + self.assertEquals(mock_errors.call_count, self.hook.retry_limit) @mock.patch('airflow.contrib.hooks.databricks_hook.requests') - def test_do_api_call_with_bad_status_code(self, mock_requests): - mock_requests.codes.ok = 200 - status_code_mock = mock.PropertyMock(return_value=500) - type(mock_requests.post.return_value).status_code = status_code_mock - with self.assertRaises(AirflowException): - self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {}) + def test_do_api_call_does_not_retry_with_non_retryable_error(self, mock_requests): + setup_mock_requests( + mock_requests, requests_exceptions.HTTPError, status_code=400 + ) + + with mock.patch.object(self.hook.log, 'error') as mock_errors: + with self.assertRaises(AirflowException): + self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {}) + + mock_errors.assert_not_called() + + def test_do_api_call_succeeds_after_retrying(self): + for exception in [ + requests_exceptions.ConnectionError, + requests_exceptions.SSLError, + requests_exceptions.Timeout, + requests_exceptions.ConnectTimeout, + requests_exceptions.HTTPError]: + with mock.patch( + 'airflow.contrib.hooks.databricks_hook.requests') as mock_requests, \ + mock.patch.object(self.hook.log, 'error') as mock_errors: + setup_mock_requests( + mock_requests, + exception, + error_count=2, + response_content={'run_id': '1'} + ) + + response = self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {}) + + self.assertEquals(mock_errors.call_count, 2) + self.assertEquals(response, {'run_id': '1'}) + + @mock.patch('airflow.contrib.hooks.databricks_hook.sleep') + def test_do_api_call_waits_between_retries(self, mock_sleep): + retry_delay = 5 + self.hook = DatabricksHook(retry_delay=retry_delay) + + for exception in [ + requests_exceptions.ConnectionError, + requests_exceptions.SSLError, + requests_exceptions.Timeout, + requests_exceptions.ConnectTimeout, + requests_exceptions.HTTPError]: + with mock.patch( + 'airflow.contrib.hooks.databricks_hook.requests') as mock_requests, \ + mock.patch.object(self.hook.log, 'error'): + mock_sleep.reset_mock() + setup_mock_requests(mock_requests, exception) + + with self.assertRaises(AirflowException): + self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {}) + + self.assertEquals(len(mock_sleep.mock_calls), self.hook.retry_limit - 1) + mock_sleep.assert_called_with(retry_delay) @mock.patch('airflow.contrib.hooks.databricks_hook.requests') def test_submit_run(self, mock_requests): - mock_requests.codes.ok = 200 mock_requests.post.return_value.json.return_value = {'run_id': '1'} - status_code_mock = mock.PropertyMock(return_value=200) - type(mock_requests.post.return_value).status_code = status_code_mock json = { 'notebook_task': NOTEBOOK_TASK, 'new_cluster': NEW_CLUSTER @@ -158,10 +251,7 @@ def test_submit_run(self, mock_requests): @mock.patch('airflow.contrib.hooks.databricks_hook.requests') def test_get_run_page_url(self, mock_requests): - mock_requests.codes.ok = 200 mock_requests.get.return_value.json.return_value = GET_RUN_RESPONSE - status_code_mock = mock.PropertyMock(return_value=200) - type(mock_requests.get.return_value).status_code = status_code_mock run_page_url = self.hook.get_run_page_url(RUN_ID) @@ -175,10 +265,7 @@ def test_get_run_page_url(self, mock_requests): @mock.patch('airflow.contrib.hooks.databricks_hook.requests') def test_get_run_state(self, mock_requests): - mock_requests.codes.ok = 200 mock_requests.get.return_value.json.return_value = GET_RUN_RESPONSE - status_code_mock = mock.PropertyMock(return_value=200) - type(mock_requests.get.return_value).status_code = status_code_mock run_state = self.hook.get_run_state(RUN_ID) @@ -195,10 +282,7 @@ def test_get_run_state(self, mock_requests): @mock.patch('airflow.contrib.hooks.databricks_hook.requests') def test_cancel_run(self, mock_requests): - mock_requests.codes.ok = 200 mock_requests.post.return_value.json.return_value = GET_RUN_RESPONSE - status_code_mock = mock.PropertyMock(return_value=200) - type(mock_requests.post.return_value).status_code = status_code_mock self.hook.cancel_run(RUN_ID) diff --git a/tests/contrib/operators/test_databricks_operator.py b/tests/contrib/operators/test_databricks_operator.py index f77da2ec18eda..afe1a92f28d9e 100644 --- a/tests/contrib/operators/test_databricks_operator.py +++ b/tests/contrib/operators/test_databricks_operator.py @@ -190,8 +190,9 @@ def test_exec_success(self, db_mock_class): 'run_name': TASK_ID }) db_mock_class.assert_called_once_with( - DEFAULT_CONN_ID, - retry_limit=op.databricks_retry_limit) + DEFAULT_CONN_ID, + retry_limit=op.databricks_retry_limit, + retry_delay=op.databricks_retry_delay) db_mock.submit_run.assert_called_once_with(expected) db_mock.get_run_page_url.assert_called_once_with(RUN_ID) db_mock.get_run_state.assert_called_once_with(RUN_ID) @@ -220,8 +221,9 @@ def test_exec_failure(self, db_mock_class): 'run_name': TASK_ID, }) db_mock_class.assert_called_once_with( - DEFAULT_CONN_ID, - retry_limit=op.databricks_retry_limit) + DEFAULT_CONN_ID, + retry_limit=op.databricks_retry_limit, + retry_delay=op.databricks_retry_delay) db_mock.submit_run.assert_called_once_with(expected) db_mock.get_run_page_url.assert_called_once_with(RUN_ID) db_mock.get_run_state.assert_called_once_with(RUN_ID) From a3ecc0ac3cafc9440e875a61d9ac670079b9d2fa Mon Sep 17 00:00:00 2001 From: wmorris75 Date: Fri, 31 Aug 2018 03:33:22 -0400 Subject: [PATCH 093/808] [AIRFLOW-2974] Extended Databricks hook with clusters operation (#3817) Add hooks for: - cluster start, - restart, - terminate. Add unit tests for the added hooks. Add hooks for cluster start, restart and terminate. Add unit tests for the added hooks. Add cluster_id variable for performing cluster operation tests. --- airflow/contrib/hooks/databricks_hook.py | 12 ++++ setup.cfg | 2 +- tests/contrib/hooks/test_databricks_hook.py | 70 +++++++++++++++++++++ 3 files changed, 83 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py index 802b8efb87842..4e05ddb0b1af4 100644 --- a/airflow/contrib/hooks/databricks_hook.py +++ b/airflow/contrib/hooks/databricks_hook.py @@ -33,6 +33,9 @@ except ImportError: import urlparse +RESTART_CLUSTER_ENDPOINT = ("POST", "api/2.0/clusters/restart") +START_CLUSTER_ENDPOINT = ("POST", "api/2.0/clusters/start") +TERMINATE_CLUSTER_ENDPOINT = ("POST", "api/2.0/clusters/delete") SUBMIT_RUN_ENDPOINT = ('POST', 'api/2.0/jobs/runs/submit') GET_RUN_ENDPOINT = ('GET', 'api/2.0/jobs/runs/get') @@ -188,6 +191,15 @@ def cancel_run(self, run_id): json = {'run_id': run_id} self._do_api_call(CANCEL_RUN_ENDPOINT, json) + def restart_cluster(self, json): + self._do_api_call(RESTART_CLUSTER_ENDPOINT, json) + + def start_cluster(self, json): + self._do_api_call(START_CLUSTER_ENDPOINT, json) + + def terminate_cluster(self, json): + self._do_api_call(TERMINATE_CLUSTER_ENDPOINT, json) + def _retryable_error(exception): return isinstance(exception, requests_exceptions.ConnectionError) \ diff --git a/setup.cfg b/setup.cfg index 622cc1303a173..881fe0107d9b2 100644 --- a/setup.cfg +++ b/setup.cfg @@ -14,6 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + [metadata] name = Airflow summary = Airflow is a system to programmatically author, schedule and monitor data pipelines. @@ -34,4 +35,3 @@ all_files = 1 upload-dir = docs/_build/html [easy_install] - diff --git a/tests/contrib/hooks/test_databricks_hook.py b/tests/contrib/hooks/test_databricks_hook.py index a022431899b4d..5c1d7876d2f32 100644 --- a/tests/contrib/hooks/test_databricks_hook.py +++ b/tests/contrib/hooks/test_databricks_hook.py @@ -52,6 +52,7 @@ 'node_type_id': 'r3.xlarge', 'num_workers': 1 } +CLUSTER_ID = 'cluster_id' RUN_ID = 1 HOST = 'xx.cloud.databricks.com' HOST_WITH_SCHEME = 'https://xx.cloud.databricks.com' @@ -93,6 +94,27 @@ def cancel_run_endpoint(host): return 'https://{}/api/2.0/jobs/runs/cancel'.format(host) +def start_cluster_endpoint(host): + """ + Utility function to generate the get run endpoint given the host. + """ + return 'https://{}/api/2.0/clusters/start'.format(host) + + +def restart_cluster_endpoint(host): + """ + Utility function to generate the get run endpoint given the host. + """ + return 'https://{}/api/2.0/clusters/restart'.format(host) + + +def terminate_cluster_endpoint(host): + """ + Utility function to generate the get run endpoint given the host. + """ + return 'https://{}/api/2.0/clusters/delete'.format(host) + + def create_valid_response_mock(content): response = mock.MagicMock() response.json.return_value = content @@ -293,6 +315,54 @@ def test_cancel_run(self, mock_requests): headers=USER_AGENT_HEADER, timeout=self.hook.timeout_seconds) + @mock.patch('airflow.contrib.hooks.databricks_hook.requests') + def test_start_cluster(self, mock_requests): + mock_requests.codes.ok = 200 + mock_requests.post.return_value.json.return_value = {} + status_code_mock = mock.PropertyMock(return_value=200) + type(mock_requests.post.return_value).status_code = status_code_mock + + self.hook.start_cluster({"cluster_id": CLUSTER_ID}) + + mock_requests.post.assert_called_once_with( + start_cluster_endpoint(HOST), + json={'cluster_id': CLUSTER_ID}, + auth=(LOGIN, PASSWORD), + headers=USER_AGENT_HEADER, + timeout=self.hook.timeout_seconds) + + @mock.patch('airflow.contrib.hooks.databricks_hook.requests') + def test_restart_cluster(self, mock_requests): + mock_requests.codes.ok = 200 + mock_requests.post.return_value.json.return_value = {} + status_code_mock = mock.PropertyMock(return_value=200) + type(mock_requests.post.return_value).status_code = status_code_mock + + self.hook.restart_cluster({"cluster_id": CLUSTER_ID}) + + mock_requests.post.assert_called_once_with( + restart_cluster_endpoint(HOST), + json={'cluster_id': CLUSTER_ID}, + auth=(LOGIN, PASSWORD), + headers=USER_AGENT_HEADER, + timeout=self.hook.timeout_seconds) + + @mock.patch('airflow.contrib.hooks.databricks_hook.requests') + def test_terminate_cluster(self, mock_requests): + mock_requests.codes.ok = 200 + mock_requests.post.return_value.json.return_value = {} + status_code_mock = mock.PropertyMock(return_value=200) + type(mock_requests.post.return_value).status_code = status_code_mock + + self.hook.terminate_cluster({"cluster_id": CLUSTER_ID}) + + mock_requests.post.assert_called_once_with( + terminate_cluster_endpoint(HOST), + json={'cluster_id': CLUSTER_ID}, + auth=(LOGIN, PASSWORD), + headers=USER_AGENT_HEADER, + timeout=self.hook.timeout_seconds) + class DatabricksHookTokenTest(unittest.TestCase): """ From abb72481557ff0c8a3b20a6bc1d36f970ae9a343 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 1 Sep 2018 11:33:48 +0100 Subject: [PATCH 094/808] [AIRFLOW-2981] Fix TypeError in dataflow operators (#3831) - Fix TypeError in dataflow operators when using GCS jar or py_file --- .../contrib/operators/dataflow_operator.py | 6 ++-- .../operators/test_dataflow_operator.py | 29 +++++++++++++++++-- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py index e3c8c1fff1572..99d10199d9703 100644 --- a/airflow/contrib/operators/dataflow_operator.py +++ b/airflow/contrib/operators/dataflow_operator.py @@ -16,7 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - +import os import re import uuid import copy @@ -358,7 +358,7 @@ def google_cloud_to_local(self, file_name): # Extracts bucket_id and object_id by first removing 'gs://' prefix and # then split the remaining by path delimiter '/'. path_components = file_name[self.GCS_PREFIX_LENGTH:].split('/') - if path_components < 2: + if len(path_components) < 2: raise Exception( 'Invalid Google Cloud Storage (GCS) object path: {}.' .format(file_name)) @@ -369,7 +369,7 @@ def google_cloud_to_local(self, file_name): path_components[-1]) file_size = self._gcs_hook.download(bucket_id, object_id, local_file) - if file_size > 0: + if os.stat(file_size).st_size > 0: return local_file raise Exception( 'Failed to download Google Cloud Storage GCS object: {}' diff --git a/tests/contrib/operators/test_dataflow_operator.py b/tests/contrib/operators/test_dataflow_operator.py index 4ea5f65698f03..a373126b24e4b 100644 --- a/tests/contrib/operators/test_dataflow_operator.py +++ b/tests/contrib/operators/test_dataflow_operator.py @@ -20,9 +20,10 @@ import unittest -from airflow.contrib.operators.dataflow_operator import DataFlowPythonOperator, \ - DataFlowJavaOperator, DataflowTemplateOperator -from airflow.contrib.operators.dataflow_operator import DataFlowPythonOperator +from airflow.contrib.operators.dataflow_operator import \ + DataFlowPythonOperator, DataFlowJavaOperator, \ + DataflowTemplateOperator, GoogleCloudBucketHelper + from airflow.version import version try: @@ -186,3 +187,25 @@ def test_exec(self, dataflow_mock): } start_template_hook.assert_called_once_with(TASK_ID, expected_options, PARAMETERS, TEMPLATE) + + +class GoogleCloudBucketHelperTest(unittest.TestCase): + + @mock.patch( + 'airflow.contrib.operators.dataflow_operator.GoogleCloudBucketHelper.__init__' + ) + def test_invalid_object_path(self, mock_parent_init): + + # This is just the path of a bucket hence invalid filename + file_name = 'gs://test-bucket' + mock_parent_init.return_value = None + + gcs_bucket_helper = GoogleCloudBucketHelper() + gcs_bucket_helper._gcs_hook = mock.Mock() + + with self.assertRaises(Exception) as context: + gcs_bucket_helper.google_cloud_to_local(file_name) + + self.assertEquals( + 'Invalid Google Cloud Storage (GCS) object path: {}.'.format(file_name), + str(context.exception)) From d9af846e8e07874318d85f3c82a0216c3956249e Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 31 Aug 2018 08:25:05 +0100 Subject: [PATCH 095/808] [AIRFLOW-2989] Add param to set bootDiskType in Dataproc Op (#3825) Add param to set bootDiskType for master and worker nodes in `DataprocClusterCreateOperator` --- .../contrib/operators/dataproc_operator.py | 19 ++++++++++++++++++- .../operators/test_dataproc_operator.py | 6 ++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index e72a66569d5b6..6954133b51ac1 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -75,10 +75,20 @@ class DataprocClusterCreateOperator(BaseOperator): :type properties: dict :param master_machine_type: Compute engine machine type to use for the master node :type master_machine_type: string + :param master_disk_type: Type of the boot disk for the master node + (default is ``pd-standard``). + Valid values: ``pd-ssd`` (Persistent Disk Solid State Drive) or + ``pd-standard`` (Persistent Disk Hard Disk Drive). + :type master_disk_type: string :param master_disk_size: Disk size for the master node :type master_disk_size: int :param worker_machine_type: Compute engine machine type to use for the worker nodes :type worker_machine_type: string + :param worker_disk_type: Type of the boot disk for the worker node + (default is ``pd-standard``). + Valid values: ``pd-ssd`` (Persistent Disk Solid State Drive) or + ``pd-standard`` (Persistent Disk Hard Disk Drive). + :type worker_disk_type: string :param worker_disk_size: Disk size for the worker nodes :type worker_disk_size: int :param num_preemptible_workers: The # of preemptible worker nodes to spin up @@ -141,8 +151,10 @@ def __init__(self, image_version=None, properties=None, master_machine_type='n1-standard-4', + master_disk_type='pd-standard', master_disk_size=500, worker_machine_type='n1-standard-4', + worker_disk_type='pd-standard', worker_disk_size=500, num_preemptible_workers=0, labels=None, @@ -171,8 +183,10 @@ def __init__(self, self.image_version = image_version self.properties = properties self.master_machine_type = master_machine_type + self.master_disk_type = master_disk_type self.master_disk_size = master_disk_size self.worker_machine_type = worker_machine_type + self.worker_disk_type = worker_disk_type self.worker_disk_size = worker_disk_size self.labels = labels self.zone = zone @@ -272,6 +286,7 @@ def _build_cluster_data(self): 'numInstances': 1, 'machineTypeUri': master_type_uri, 'diskConfig': { + 'bootDiskType': self.master_disk_type, 'bootDiskSizeGb': self.master_disk_size } }, @@ -279,6 +294,7 @@ def _build_cluster_data(self): 'numInstances': self.num_workers, 'machineTypeUri': worker_type_uri, 'diskConfig': { + 'bootDiskType': self.worker_disk_type, 'bootDiskSizeGb': self.worker_disk_size } }, @@ -292,6 +308,7 @@ def _build_cluster_data(self): 'numInstances': self.num_preemptible_workers, 'machineTypeUri': worker_type_uri, 'diskConfig': { + 'bootDiskType': self.worker_disk_type, 'bootDiskSizeGb': self.worker_disk_size }, 'isPreemptible': True @@ -401,7 +418,7 @@ class DataprocClusterScaleOperator(BaseOperator): cluster_name='cluster-1', num_workers=10, num_preemptible_workers=10, - graceful_decommission_timeout='1h' + graceful_decommission_timeout='1h', dag=dag) .. seealso:: diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py index aea130349bdea..ab191f3c32760 100644 --- a/tests/contrib/operators/test_dataproc_operator.py +++ b/tests/contrib/operators/test_dataproc_operator.py @@ -61,8 +61,10 @@ IMAGE_VERSION = '1.1' MASTER_MACHINE_TYPE = 'n1-standard-2' MASTER_DISK_SIZE = 100 +MASTER_DISK_TYPE = 'pd-standard' WORKER_MACHINE_TYPE = 'n1-standard-2' WORKER_DISK_SIZE = 100 +WORKER_DISK_TYPE = 'pd-standard' NUM_PREEMPTIBLE_WORKERS = 2 GET_INIT_ACTION_TIMEOUT = "600s" # 10m LABEL1 = {} @@ -104,8 +106,10 @@ def setUp(self): storage_bucket=STORAGE_BUCKET, image_version=IMAGE_VERSION, master_machine_type=MASTER_MACHINE_TYPE, + master_disk_type=MASTER_DISK_TYPE, master_disk_size=MASTER_DISK_SIZE, worker_machine_type=WORKER_MACHINE_TYPE, + worker_disk_type=WORKER_DISK_TYPE, worker_disk_size=WORKER_DISK_SIZE, num_preemptible_workers=NUM_PREEMPTIBLE_WORKERS, labels=deepcopy(labels), @@ -138,8 +142,10 @@ def test_init(self): self.assertEqual(dataproc_operator.image_version, IMAGE_VERSION) self.assertEqual(dataproc_operator.master_machine_type, MASTER_MACHINE_TYPE) self.assertEqual(dataproc_operator.master_disk_size, MASTER_DISK_SIZE) + self.assertEqual(dataproc_operator.master_disk_type, MASTER_DISK_TYPE) self.assertEqual(dataproc_operator.worker_machine_type, WORKER_MACHINE_TYPE) self.assertEqual(dataproc_operator.worker_disk_size, WORKER_DISK_SIZE) + self.assertEqual(dataproc_operator.worker_disk_type, WORKER_DISK_TYPE) self.assertEqual(dataproc_operator.num_preemptible_workers, NUM_PREEMPTIBLE_WORKERS) self.assertEqual(dataproc_operator.labels, self.labels[suffix]) From 4f6f496b763f202fcca0de5fdc71f5655d9474f8 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 31 Aug 2018 09:36:24 +0100 Subject: [PATCH 096/808] [AIRFLOW-2990] Fix Docstrings for Operators (#3820) --- airflow/contrib/hooks/gcp_container_hook.py | 4 + .../contrib/operators/awsbatch_operator.py | 16 ++-- .../operators/bigquery_check_operator.py | 6 +- .../contrib/operators/bigquery_operator.py | 82 +++++++++---------- .../contrib/operators/dataflow_operator.py | 63 +++++++------- .../contrib/operators/dataproc_operator.py | 16 ++-- airflow/contrib/operators/ecs_operator.py | 13 +-- airflow/models.py | 19 +++-- airflow/sensors/s3_prefix_sensor.py | 2 + 9 files changed, 117 insertions(+), 104 deletions(-) diff --git a/airflow/contrib/hooks/gcp_container_hook.py b/airflow/contrib/hooks/gcp_container_hook.py index d36d796d764a5..3cc3744c03686 100644 --- a/airflow/contrib/hooks/gcp_container_hook.py +++ b/airflow/contrib/hooks/gcp_container_hook.py @@ -47,6 +47,7 @@ def __init__(self, project_id, location): def _dict_to_proto(self, py_dict, proto): """ Converts a python dictionary to the proto supplied + :param py_dict: The dictionary to convert :type py_dict: dict :param proto: The proto object to merge with dictionary @@ -62,6 +63,7 @@ def wait_for_operation(self, operation): """ Given an operation, continuously fetches the status from Google Cloud until either completion or an error occurring + :param operation: The Operation to wait for :type operation: A google.cloud.container_V1.gapic.enums.Operator :return: A new, updated operation fetched from Google Cloud @@ -82,6 +84,7 @@ def wait_for_operation(self, operation): def get_operation(self, operation_name): """ Fetches the operation from Google Cloud + :param operation_name: Name of operation to fetch :type operation_name: str :return: The new, updated operation from Google Cloud @@ -194,6 +197,7 @@ def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): def get_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): """ Gets details of specified cluster + :param name: The name of the cluster to retrieve :type name: str :param retry: A retry object used to retry requests. If None is specified, diff --git a/airflow/contrib/operators/awsbatch_operator.py b/airflow/contrib/operators/awsbatch_operator.py index ed7f1ebf36cce..89dd4940b51e9 100644 --- a/airflow/contrib/operators/awsbatch_operator.py +++ b/airflow/contrib/operators/awsbatch_operator.py @@ -42,18 +42,20 @@ class AWSBatchOperator(BaseOperator): :type job_definition: str :param job_queue: the queue name on AWS Batch :type job_queue: str - :param: overrides: the same parameter that boto3 will receive on - containerOverrides (templated): - http://boto3.readthedocs.io/en/latest/reference/services/batch.html#submit_job - :type: overrides: dict - :param max_retries: exponential backoff retries while waiter is not merged, 4200 = 48 hours + :param overrides: the same parameter that boto3 will receive on + containerOverrides (templated). + http://boto3.readthedocs.io/en/latest/reference/services/batch.html#submit_job + :type overrides: dict + :param max_retries: exponential backoff retries while waiter is not merged, + 4200 = 48 hours :type max_retries: int :param aws_conn_id: connection id of AWS credentials / region name. If None, - credential boto3 strategy will be used - (http://boto3.readthedocs.io/en/latest/guide/configuration.html). + credential boto3 strategy will be used + (http://boto3.readthedocs.io/en/latest/guide/configuration.html). :type aws_conn_id: str :param region_name: region name to use in AWS Hook. Override the region_name in connection (if provided) + :type region_name: str """ ui_color = '#c3dae0' diff --git a/airflow/contrib/operators/bigquery_check_operator.py b/airflow/contrib/operators/bigquery_check_operator.py index a9c493f4fd418..3eba0771db91b 100644 --- a/airflow/contrib/operators/bigquery_check_operator.py +++ b/airflow/contrib/operators/bigquery_check_operator.py @@ -56,7 +56,7 @@ class BigQueryCheckOperator(CheckOperator): :param bigquery_conn_id: reference to the BigQuery database :type bigquery_conn_id: string :param use_legacy_sql: Whether to use legacy SQL (true) - or standard SQL (false). + or standard SQL (false). :type use_legacy_sql: boolean """ @@ -83,7 +83,7 @@ class BigQueryValueCheckOperator(ValueCheckOperator): :param sql: the sql to be executed :type sql: string :param use_legacy_sql: Whether to use legacy SQL (true) - or standard SQL (false). + or standard SQL (false). :type use_legacy_sql: boolean """ @@ -125,7 +125,7 @@ class BigQueryIntervalCheckOperator(IntervalCheckOperator): between the current day, and the prior days_back. :type metrics_threshold: dict :param use_legacy_sql: Whether to use legacy SQL (true) - or standard SQL (false). + or standard SQL (false). :type use_legacy_sql: boolean """ diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 2eadd04f003d9..037b9fc43e631 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -232,49 +232,49 @@ class BigQueryCreateEmptyTableOperator(BaseOperator): work, the service account making the request must have domain-wide delegation enabled. :type delegate_to: string - :param labels a dictionary containing labels for the table, passed to BigQuery + :param labels: a dictionary containing labels for the table, passed to BigQuery + + **Example (with schema JSON in GCS)**: :: + + CreateTable = BigQueryCreateEmptyTableOperator( + task_id='BigQueryCreateEmptyTableOperator_task', + dataset_id='ODS', + table_id='Employees', + project_id='internal-gcp-project', + gcs_schema_object='gs://schema-bucket/employee_schema.json', + bigquery_conn_id='airflow-service-account', + google_cloud_storage_conn_id='airflow-service-account' + ) + + **Corresponding Schema file** (``employee_schema.json``): :: + + [ + { + "mode": "NULLABLE", + "name": "emp_name", + "type": "STRING" + }, + { + "mode": "REQUIRED", + "name": "salary", + "type": "INTEGER" + } + ] + + **Example (with schema in the DAG)**: :: + + CreateTable = BigQueryCreateEmptyTableOperator( + task_id='BigQueryCreateEmptyTableOperator_task', + dataset_id='ODS', + table_id='Employees', + project_id='internal-gcp-project', + schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"}, + {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}], + bigquery_conn_id='airflow-service-account', + google_cloud_storage_conn_id='airflow-service-account' + ) :type labels: dict - **Example (with schema JSON in GCS)**: :: - - CreateTable = BigQueryCreateEmptyTableOperator( - task_id='BigQueryCreateEmptyTableOperator_task', - dataset_id='ODS', - table_id='Employees', - project_id='internal-gcp-project', - gcs_schema_object='gs://schema-bucket/employee_schema.json', - bigquery_conn_id='airflow-service-account', - google_cloud_storage_conn_id='airflow-service-account' - ) - - **Corresponding Schema file** (``employee_schema.json``): :: - - [ - { - "mode": "NULLABLE", - "name": "emp_name", - "type": "STRING" - }, - { - "mode": "REQUIRED", - "name": "salary", - "type": "INTEGER" - } - ] - - **Example (with schema in the DAG)**: :: - - CreateTable = BigQueryCreateEmptyTableOperator( - task_id='BigQueryCreateEmptyTableOperator_task', - dataset_id='ODS', - table_id='Employees', - project_id='internal-gcp-project', - schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"}, - {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}], - bigquery_conn_id='airflow-service-account', - google_cloud_storage_conn_id='airflow-service-account' - ) - """ template_fields = ('dataset_id', 'table_id', 'project_id', 'gcs_schema_object', 'labels') diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py index 99d10199d9703..cd289f546ad3c 100644 --- a/airflow/contrib/operators/dataflow_operator.py +++ b/airflow/contrib/operators/dataflow_operator.py @@ -252,6 +252,38 @@ def execute(self, context): class DataFlowPythonOperator(BaseOperator): + """ + Create a new DataFlowPythonOperator. Note that both + dataflow_default_options and options will be merged to specify pipeline + execution parameter, and dataflow_default_options is expected to save + high-level options, for instances, project and zone information, which + apply to all dataflow operators in the DAG. + + .. seealso:: + For more detail on job submission have a look at the reference: + https://cloud.google.com/dataflow/pipelines/specifying-exec-params + + :param py_file: Reference to the python dataflow pipleline file.py, e.g., + /some/local/file/path/to/your/python/pipeline/file. + :type py_file: string + :param py_options: Additional python options. + :type pyt_options: list of strings, e.g., ["-m", "-v"]. + :param dataflow_default_options: Map of default job options. + :type dataflow_default_options: dict + :param options: Map of job specific options. + :type options: dict + :param gcp_conn_id: The connection ID to use connecting to Google Cloud + Platform. + :type gcp_conn_id: string + :param delegate_to: The account to impersonate, if any. + For this to work, the service account making the request must have + domain-wide delegation enabled. + :type delegate_to: string + :param poll_sleep: The time in seconds to sleep between polling Google + Cloud Platform for the dataflow job status while the job is in the + JOB_STATE_RUNNING state. + :type poll_sleep: int + """ template_fields = ['options', 'dataflow_default_options'] @@ -267,38 +299,7 @@ def __init__( poll_sleep=10, *args, **kwargs): - """ - Create a new DataFlowPythonOperator. Note that both - dataflow_default_options and options will be merged to specify pipeline - execution parameter, and dataflow_default_options is expected to save - high-level options, for instances, project and zone information, which - apply to all dataflow operators in the DAG. - - .. seealso:: - For more detail on job submission have a look at the reference: - https://cloud.google.com/dataflow/pipelines/specifying-exec-params - :param py_file: Reference to the python dataflow pipleline file.py, e.g., - /some/local/file/path/to/your/python/pipeline/file. - :type py_file: string - :param py_options: Additional python options. - :type pyt_options: list of strings, e.g., ["-m", "-v"]. - :param dataflow_default_options: Map of default job options. - :type dataflow_default_options: dict - :param options: Map of job specific options. - :type options: dict - :param gcp_conn_id: The connection ID to use connecting to Google Cloud - Platform. - :type gcp_conn_id: string - :param delegate_to: The account to impersonate, if any. - For this to work, the service account making the request must have - domain-wide delegation enabled. - :type delegate_to: string - :param poll_sleep: The time in seconds to sleep between polling Google - Cloud Platform for the dataflow job status while the job is in the - JOB_STATE_RUNNING state. - :type poll_sleep: int - """ super(DataFlowPythonOperator, self).__init__(*args, **kwargs) self.py_file = py_file diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index 6954133b51ac1..957a1b89e6b07 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -412,14 +412,14 @@ class DataprocClusterScaleOperator(BaseOperator): **Example**: :: - t1 = DataprocClusterScaleOperator( - task_id='dataproc_scale', - project_id='my-project', - cluster_name='cluster-1', - num_workers=10, - num_preemptible_workers=10, - graceful_decommission_timeout='1h', - dag=dag) + t1 = DataprocClusterScaleOperator( + task_id='dataproc_scale', + project_id='my-project', + cluster_name='cluster-1', + num_workers=10, + num_preemptible_workers=10, + graceful_decommission_timeout='1h', + dag=dag) .. seealso:: For more detail on about scaling clusters have a look at the reference: diff --git a/airflow/contrib/operators/ecs_operator.py b/airflow/contrib/operators/ecs_operator.py index 60540f5e289d6..84bc2a3c91a2f 100644 --- a/airflow/contrib/operators/ecs_operator.py +++ b/airflow/contrib/operators/ecs_operator.py @@ -33,17 +33,18 @@ class ECSOperator(BaseOperator): :type task_definition: str :param cluster: the cluster name on EC2 Container Service :type cluster: str - :param: overrides: the same parameter that boto3 will receive (templated): - http://boto3.readthedocs.org/en/latest/reference/services/ecs.html#ECS.Client.run_task - :type: overrides: dict + :param overrides: the same parameter that boto3 will receive (templated): + http://boto3.readthedocs.org/en/latest/reference/services/ecs.html#ECS.Client.run_task + :type overrides: dict :param aws_conn_id: connection id of AWS credentials / region name. If None, - credential boto3 strategy will be used - (http://boto3.readthedocs.io/en/latest/guide/configuration.html). + credential boto3 strategy will be used + (http://boto3.readthedocs.io/en/latest/guide/configuration.html). :type aws_conn_id: str :param region_name: region name to use in AWS Hook. Override the region_name in connection (if provided) + :type region_name: str :param launch_type: the launch type on which to run your task ('EC2' or 'FARGATE') - :type: launch_type: str + :type launch_type: str """ ui_color = '#f0ede4' diff --git a/airflow/models.py b/airflow/models.py index 0207ac25252dc..6597dac4f0d35 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -2324,14 +2324,17 @@ class derived from this one results in the creation of a task object, :param executor_config: Additional task-level configuration parameters that are interpreted by a specific executor. Parameters are namespaced by the name of executor. - ``example: to run this task in a specific docker container through - the KubernetesExecutor - MyOperator(..., - executor_config={ - "KubernetesExecutor": - {"image": "myCustomDockerImage"} - } - )`` + + **Example**: to run this task in a specific docker container through + the KubernetesExecutor :: + + MyOperator(..., + executor_config={ + "KubernetesExecutor": + {"image": "myCustomDockerImage"} + } + ) + :type executor_config: dict """ diff --git a/airflow/sensors/s3_prefix_sensor.py b/airflow/sensors/s3_prefix_sensor.py index 917dd46e26c28..42e574c5ee2b8 100644 --- a/airflow/sensors/s3_prefix_sensor.py +++ b/airflow/sensors/s3_prefix_sensor.py @@ -38,6 +38,8 @@ class S3PrefixSensor(BaseSensorOperator): :param delimiter: The delimiter intended to show hierarchy. Defaults to '/'. :type delimiter: str + :param aws_conn_id: a reference to the s3 connection + :type aws_conn_id: str """ template_fields = ('prefix', 'bucket_name') From bc27aace7d23baad145c75171d105e8cd902ece6 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Tue, 4 Sep 2018 23:47:41 +0100 Subject: [PATCH 097/808] [AIRFLOW-3006] Add note on using None for schedule_interval --- airflow/models.py | 1 + docs/scheduler.rst | 2 ++ tests/models.py | 7 +++++-- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 6597dac4f0d35..14c08e7092532 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -2302,6 +2302,7 @@ class derived from this one results in the creation of a task object, :type on_failure_callback: callable :param on_retry_callback: much like the ``on_failure_callback`` except that it is executed when retries occur. + :type on_retry_callback: callable :param on_success_callback: much like the ``on_failure_callback`` except that it is executed when the task succeeds. :type on_success_callback: callable diff --git a/docs/scheduler.rst b/docs/scheduler.rst index dfa0a4275770a..84dcc4a7f9db4 100644 --- a/docs/scheduler.rst +++ b/docs/scheduler.rst @@ -63,6 +63,8 @@ use one of these cron "preset": | ``@yearly`` | Run once a year at midnight of January 1 | ``0 0 1 1 *`` | +--------------+----------------------------------------------------------------+---------------+ +**Note**: Use ``schedule_interval=None`` and not ``schedule_interval='None'`` when +you don't want to schedule your DAG. Your DAG will be instantiated for each schedule, while creating a ``DAG Run`` entry for each schedule. diff --git a/tests/models.py b/tests/models.py index 5b31a634f4a7f..abb726f7a9e6a 100644 --- a/tests/models.py +++ b/tests/models.py @@ -65,7 +65,7 @@ class DagTest(unittest.TestCase): - def test_parms_not_passed_is_empty_dict(self): + def test_params_not_passed_is_empty_dict(self): """ Test that when 'params' is _not_ passed to a new Dag, that the params attribute is set to an empty dictionary. @@ -311,7 +311,6 @@ def test_dag_task_priority_weight_total(self): with self.assertRaises(AirflowException): DummyOperator(task_id='should_fail', weight_rule='no rule') - def test_get_num_task_instances(self): test_dag_id = 'test_get_num_task_instances_dag' test_task_id = 'task_1' @@ -595,6 +594,7 @@ def test_dagstats_crud(self): for stat in res: self.assertFalse(stat.dirty) + class DagRunTest(unittest.TestCase): def create_dag_run(self, dag, @@ -1113,6 +1113,7 @@ def with_all_tasks_removed(dag): flaky_ti.refresh_from_db() self.assertEquals(State.NONE, flaky_ti.state) + class DagBagTest(unittest.TestCase): def test_get_existing_dag(self): @@ -1585,6 +1586,7 @@ def test_kill_zombies(self, mock_ti): configuration.getboolean('core', 'unit_test_mode'), ANY) + class TaskInstanceTest(unittest.TestCase): def test_set_task_dates(self): @@ -2602,6 +2604,7 @@ def test_xcom_get_many(self): for result in results: self.assertEqual(result.value, json_obj) + class ConnectionTest(unittest.TestCase): @patch.object(configuration, 'get') def test_connection_extra_no_encryption(self, mock_get): From bc97e6ee0b6136368d6aebc5c30df848420bb6b6 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 5 Sep 2018 00:46:41 +0100 Subject: [PATCH 098/808] [AIRFLOW-3008] Move Kubernetes example DAGs to contrib --- .../{ => contrib}/example_dags/example_kubernetes_executor.py | 0 .../{ => contrib}/example_dags/example_kubernetes_operator.py | 0 tests/core.py | 2 +- 3 files changed, 1 insertion(+), 1 deletion(-) rename airflow/{ => contrib}/example_dags/example_kubernetes_executor.py (100%) rename airflow/{ => contrib}/example_dags/example_kubernetes_operator.py (100%) diff --git a/airflow/example_dags/example_kubernetes_executor.py b/airflow/contrib/example_dags/example_kubernetes_executor.py similarity index 100% rename from airflow/example_dags/example_kubernetes_executor.py rename to airflow/contrib/example_dags/example_kubernetes_executor.py diff --git a/airflow/example_dags/example_kubernetes_operator.py b/airflow/contrib/example_dags/example_kubernetes_operator.py similarity index 100% rename from airflow/example_dags/example_kubernetes_operator.py rename to airflow/contrib/example_dags/example_kubernetes_operator.py diff --git a/tests/core.py b/tests/core.py index f00c9aeead2cd..8cd69dc4ee1fb 100644 --- a/tests/core.py +++ b/tests/core.py @@ -73,7 +73,7 @@ import six -NUM_EXAMPLE_DAGS = 20 +NUM_EXAMPLE_DAGS = 18 DEV_NULL = '/dev/null' TEST_DAG_FOLDER = os.path.join( os.path.dirname(os.path.realpath(__file__)), 'dags') From 35f996b65a2e68f4e168e3f466f3a939e4fb904a Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 4 Oct 2018 03:20:24 -0400 Subject: [PATCH 099/808] [AIRFLOW-3103][AIRFLOW-3147] Update flask-appbuilder (#3937) --- UPDATING.md | 23 +++++++++++++------ .../auth/backends/github_enterprise_auth.py | 3 +++ airflow/contrib/auth/backends/google_auth.py | 3 +++ .../contrib/auth/backends/kerberos_auth.py | 5 +++- airflow/contrib/auth/backends/ldap_auth.py | 5 +++- .../contrib/auth/backends/password_auth.py | 5 +++- airflow/default_login.py | 3 +++ airflow/www/utils.py | 8 +++---- airflow/www/views.py | 2 +- airflow/www_rbac/decorators.py | 2 +- airflow/www_rbac/security.py | 2 +- .../templates/appbuilder/navbar_right.html | 2 +- setup.py | 4 ++-- tests/www/test_utils.py | 6 ++--- 14 files changed, 50 insertions(+), 23 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index d00e4f8742f26..130df60530c4c 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -3,13 +3,6 @@ This file documents any backwards-incompatible changes in Airflow and assists users migrating to a new version. -## Airflow Master - -### min_file_parsing_loop_time config option temporarily disabled - -The scheduler.min_file_parsing_loop_time config option has been temporarily removed due to -some bugs. - ## Airflow 1.10 Installation and upgrading requires setting `SLUGIFY_USES_TEXT_UNIDECODE=yes` in your environment or @@ -119,6 +112,22 @@ elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_ elasticsearch_end_of_log_mark = end_of_log ``` +### Custom auth backends interface change + +We have updated the version of flask-login we depend upon, and as a result any +custom auth backends might need a small change: `is_active`, +`is_authenticated`, and `is_anonymous` should now be properties. What this means is if +previously you had this in your user class + + def is_active(self): + return self.active + +then you need to change it like this + + @property + def is_active(self): + return self.active + ## Airflow 1.9 ### SSH Hook updates, along with new SSH Operator & SFTP Operator diff --git a/airflow/contrib/auth/backends/github_enterprise_auth.py b/airflow/contrib/auth/backends/github_enterprise_auth.py index 641b81e46da63..78afee46078dc 100644 --- a/airflow/contrib/auth/backends/github_enterprise_auth.py +++ b/airflow/contrib/auth/backends/github_enterprise_auth.py @@ -47,14 +47,17 @@ class GHEUser(models.User): def __init__(self, user): self.user = user + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False diff --git a/airflow/contrib/auth/backends/google_auth.py b/airflow/contrib/auth/backends/google_auth.py index e41934b926c31..32ad6962a0a82 100644 --- a/airflow/contrib/auth/backends/google_auth.py +++ b/airflow/contrib/auth/backends/google_auth.py @@ -46,14 +46,17 @@ class GoogleUser(models.User): def __init__(self, user): self.user = user + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False diff --git a/airflow/contrib/auth/backends/kerberos_auth.py b/airflow/contrib/auth/backends/kerberos_auth.py index 08be299a197dd..4b5bb295d9685 100644 --- a/airflow/contrib/auth/backends/kerberos_auth.py +++ b/airflow/contrib/auth/backends/kerberos_auth.py @@ -73,14 +73,17 @@ def authenticate(username, password): return + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False @@ -110,7 +113,7 @@ def load_user(userid, session=None): @provide_session def login(self, request, session=None): - if current_user.is_authenticated(): + if current_user.is_authenticated: flash("You are already logged in") return redirect(url_for('index')) diff --git a/airflow/contrib/auth/backends/ldap_auth.py b/airflow/contrib/auth/backends/ldap_auth.py index 516e121c9b26d..cb4675757b75a 100644 --- a/airflow/contrib/auth/backends/ldap_auth.py +++ b/airflow/contrib/auth/backends/ldap_auth.py @@ -236,14 +236,17 @@ def try_login(username, password): log.info("Password incorrect for user %s", username) raise AuthenticationError("Invalid username or password") + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False @@ -274,7 +277,7 @@ def load_user(userid, session=None): @provide_session def login(self, request, session=None): - if current_user.is_authenticated(): + if current_user.is_authenticated: flash("You are already logged in") return redirect(url_for('admin.index')) diff --git a/airflow/contrib/auth/backends/password_auth.py b/airflow/contrib/auth/backends/password_auth.py index 879aaa142a4db..55f5daf8fdf76 100644 --- a/airflow/contrib/auth/backends/password_auth.py +++ b/airflow/contrib/auth/backends/password_auth.py @@ -71,14 +71,17 @@ def password(self, plaintext): def authenticate(self, plaintext): return check_password_hash(self._password, plaintext) + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False @@ -137,7 +140,7 @@ def authenticate(session, username, password): @provide_session def login(self, request, session=None): - if current_user.is_authenticated(): + if current_user.is_authenticated: flash("You are already logged in") return redirect(url_for('admin.index')) diff --git a/airflow/default_login.py b/airflow/default_login.py index d44dbf39ea9b3..b6453c55d0c69 100644 --- a/airflow/default_login.py +++ b/airflow/default_login.py @@ -44,14 +44,17 @@ class DefaultUser(object): def __init__(self, user): self.user = user + @property def is_active(self): """Required by flask_login""" return True + @property def is_authenticated(self): """Required by flask_login""" return True + @property def is_anonymous(self): """Required by flask_login""" return False diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 03ba1f775b001..1566faa91ac1e 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -71,8 +71,8 @@ class LoginMixin(object): def is_accessible(self): return ( not AUTHENTICATE or ( - not current_user.is_anonymous() and - current_user.is_authenticated() + not current_user.is_anonymous and + current_user.is_authenticated ) ) @@ -81,7 +81,7 @@ class SuperUserMixin(object): def is_accessible(self): return ( not AUTHENTICATE or - (not current_user.is_anonymous() and current_user.is_superuser()) + (not current_user.is_anonymous and current_user.is_superuser()) ) @@ -89,7 +89,7 @@ class DataProfilingMixin(object): def is_accessible(self): return ( not AUTHENTICATE or - (not current_user.is_anonymous() and current_user.data_profiling()) + (not current_user.is_anonymous and current_user.data_profiling()) ) diff --git a/airflow/www/views.py b/airflow/www/views.py index cfd46b871aa7e..540980baa602c 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -266,7 +266,7 @@ def data_profiling_required(f): def decorated_function(*args, **kwargs): if ( current_app.config['LOGIN_DISABLED'] or - (not current_user.is_anonymous() and current_user.data_profiling()) + (not current_user.is_anonymous and current_user.data_profiling()) ): return f(*args, **kwargs) else: diff --git a/airflow/www_rbac/decorators.py b/airflow/www_rbac/decorators.py index 2dd1af45df09d..41be4eb4be597 100644 --- a/airflow/www_rbac/decorators.py +++ b/airflow/www_rbac/decorators.py @@ -32,7 +32,7 @@ def action_logging(f): @functools.wraps(f) def wrapper(*args, **kwargs): session = settings.Session() - if g.user.is_anonymous(): + if g.user.is_anonymous: user = 'anonymous' else: user = g.user.username diff --git a/airflow/www_rbac/security.py b/airflow/www_rbac/security.py index d2271f822a47e..ae21029dd182b 100644 --- a/airflow/www_rbac/security.py +++ b/airflow/www_rbac/security.py @@ -171,7 +171,7 @@ def init_roles(appbuilder): def is_view_only(user, appbuilder): - if user.is_anonymous(): + if user.is_anonymous: anonymous_role = appbuilder.sm.auth_role_public return anonymous_role == 'Viewer' diff --git a/airflow/www_rbac/templates/appbuilder/navbar_right.html b/airflow/www_rbac/templates/appbuilder/navbar_right.html index bf5aa43221884..d42f8e2e8a82c 100644 --- a/airflow/www_rbac/templates/appbuilder/navbar_right.html +++ b/airflow/www_rbac/templates/appbuilder/navbar_right.html @@ -47,7 +47,7 @@
  • -{% if not current_user.is_anonymous() %} +{% if not current_user.is_anonymous %} +
  • + + + Delete + +

  • @@ -302,6 +309,12 @@

    DAGs

    + + + + + {% endfor %} @@ -244,6 +250,12 @@

    DAGs

    function confirmTriggerDag(dag_id){ return confirm("Are you sure you want to run '"+dag_id+"' now?"); } + + function confirmDeleteDag(dag_id){ + return confirm("Are you sure you want to delete '"+dag_id+"' now?\n\ + This option will delete ALL metadata, DAG runs, etc.\n\ + This cannot be undone."); + } all_dags = $("[id^=toggle]"); $.each(all_dags, function(i,v) { $(v).change (function() { diff --git a/airflow/www/views.py b/airflow/www/views.py index 3112c081acc47..40fc2d6838406 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1043,6 +1043,32 @@ def run(self): "it should start any moment now.".format(ti)) return redirect(origin) + @expose('/delete') + @login_required + @wwwutils.action_logging + @wwwutils.notify_owner + def delete(self): + from airflow.api.common.experimental import delete_dag + from airflow.exceptions import DagNotFound, DagFileExists + + dag_id = request.args.get('dag_id') + origin = request.args.get('origin') or "/admin/" + + try: + delete_dag.delete_dag(dag_id) + except DagNotFound: + flash("DAG with id {} not found. Cannot delete".format(dag_id)) + return redirect(request.referrer) + except DagFileExists: + flash("Dag id {} is still in DagBag. " + "Remove the DAG file first.".format(dag_id)) + return redirect(request.referrer) + + flash("Deleting DAG with id {}. May take a couple minutes to fully" + " disappear.".format(dag_id)) + # Upon successful delete return to origin + return redirect(origin) + @expose('/trigger') @login_required @wwwutils.action_logging @@ -1286,6 +1312,10 @@ def tree(self, session=None): dag_id = request.args.get('dag_id') blur = conf.getboolean('webserver', 'demo_mode') dag = dagbag.get_dag(dag_id) + if dag_id not in dagbag.dags: + flash('DAG "{0}" seems to be missing.'.format(dag_id), "error") + return redirect('/admin/') + root = request.args.get('root') if root: dag = dag.sub_dag( diff --git a/airflow/www_rbac/templates/airflow/dag.html b/airflow/www_rbac/templates/airflow/dag.html index eb449085023b9..e6495fe4be9da 100644 --- a/airflow/www_rbac/templates/airflow/dag.html +++ b/airflow/www_rbac/templates/airflow/dag.html @@ -99,6 +99,13 @@

    Refresh +
  • + + + Delete + +

  • @@ -300,6 +307,12 @@

    DAGs

    + + + + {% endfor %} @@ -242,6 +247,12 @@

    DAGs

    window.location = DAGS_INDEX + "?page_size=" + p_size; }); + function confirmDeleteDag(dag_id){ + return confirm("Are you sure you want to delete '"+dag_id+"' now?\n\ + This option will delete ALL metadata, DAG runs, etc.\n\ + This cannot be undone."); + } + function confirmTriggerDag(dag_id){ return confirm("Are you sure you want to run '"+dag_id+"' now?"); } diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 3f5ccab5cbdd9..69e2079059889 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -727,6 +727,32 @@ def run(self): "it should start any moment now.".format(ti)) return redirect(origin) + @expose('/delete') + @action_logging + @has_access + def delete(self): + from airflow.api.common.experimental import delete_dag + from airflow.exceptions import DagNotFound, DagFileExists + + dag_id = request.args.get('dag_id') + origin = request.args.get('origin') or "/" + + try: + delete_dag.delete_dag(dag_id) + except DagNotFound: + flash("DAG with id {} not found. Cannot delete".format(dag_id)) + return redirect(request.referrer) + except DagFileExists: + flash("Dag id {} is still in DagBag. " + "Remove the DAG file first.".format(dag_id)) + return redirect(request.referrer) + + flash("Deleting DAG with id {}. May take a couple minutes to fully" + " disappear.".format(dag_id)) + + # Upon success return to origin. + return redirect(origin) + @expose('/trigger') @has_access @action_logging @@ -964,6 +990,10 @@ def tree(self, session=None): dag_id = request.args.get('dag_id') blur = conf.getboolean('webserver', 'demo_mode') dag = dagbag.get_dag(dag_id) + if dag_id not in dagbag.dags: + flash('DAG "{0}" seems to be missing.'.format(dag_id), "error") + return redirect('/') + root = request.args.get('root') if root: dag = dag.sub_dag( From 3be8ce724f2eb59929a5bfdacd66c761c96e2858 Mon Sep 17 00:00:00 2001 From: Szymon Przedwojski Date: Thu, 1 Nov 2018 00:00:46 +0100 Subject: [PATCH 140/808] [AIRFLOW-3231] Basic operators for Google Cloud SQL (#4097) Add CloudSqlInstanceInsertOperator, CloudSqlInstancePatchOperator and CloudSqlInstanceDeleteOperator. Each operator includes: - core logic - input params validation - unit tests - presence in the example DAG - docstrings - How-to and Integration documentation Additionally, small improvements to GcpBodyFieldValidator were made: - add simple list validation capability (type="list") - introduced parameter allow_empty, which can be set to False to test for non-emptiness of a string instead of specifying a regexp. Co-authored-by: sprzedwojski Co-authored-by: potiuk --- .../contrib/example_dags/example_gcp_sql.py | 134 ++++++++ airflow/contrib/hooks/gcp_sql_hook.py | 173 ++++++++++ airflow/contrib/operators/gcp_sql_operator.py | 297 +++++++++++++++++ airflow/contrib/utils/gcp_field_validator.py | 41 ++- docs/howto/operator.rst | 152 +++++++++ docs/integration.rst | 37 +++ .../operators/test_gcp_function_operator.py | 6 +- .../operators/test_gcp_sql_operator.py | 314 ++++++++++++++++++ 8 files changed, 1145 insertions(+), 9 deletions(-) create mode 100644 airflow/contrib/example_dags/example_gcp_sql.py create mode 100644 airflow/contrib/hooks/gcp_sql_hook.py create mode 100644 airflow/contrib/operators/gcp_sql_operator.py create mode 100644 tests/contrib/operators/test_gcp_sql_operator.py diff --git a/airflow/contrib/example_dags/example_gcp_sql.py b/airflow/contrib/example_dags/example_gcp_sql.py new file mode 100644 index 0000000000000..a484456f6e6d9 --- /dev/null +++ b/airflow/contrib/example_dags/example_gcp_sql.py @@ -0,0 +1,134 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Example Airflow DAG that deploys, updates, patches and deletes a Cloud SQL instance +in Google Cloud Platform. + +This DAG relies on the following Airflow variables +https://airflow.apache.org/concepts.html#variables +* PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance. +* INSTANCE_NAME - Name of the Cloud SQL instance. +""" + +import datetime + +import airflow +from airflow import models + +from airflow.contrib.operators.gcp_sql_operator import CloudSqlInstanceCreateOperator, \ + CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator + +# [START howto_operator_cloudsql_arguments] +PROJECT_ID = models.Variable.get('PROJECT_ID', '') +INSTANCE_NAME = models.Variable.get('INSTANCE_NAME', '') +# [END howto_operator_cloudsql_arguments] + +# Bodies below represent Cloud SQL instance resources: +# https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances + +# [START howto_operator_cloudsql_create_body] +body = { + "name": INSTANCE_NAME, + "settings": { + "tier": "db-n1-standard-1", + "backupConfiguration": { + "binaryLogEnabled": True, + "enabled": True, + "startTime": "05:00" + }, + "activationPolicy": "ALWAYS", + "dataDiskSizeGb": 30, + "dataDiskType": "PD_SSD", + "databaseFlags": [], + "ipConfiguration": { + "ipv4Enabled": True, + "requireSsl": True, + }, + "locationPreference": { + "zone": "europe-west4-a" + }, + "maintenanceWindow": { + "hour": 5, + "day": 7, + "updateTrack": "canary" + }, + "pricingPlan": "PER_USE", + "replicationType": "ASYNCHRONOUS", + "storageAutoResize": False, + "storageAutoResizeLimit": 0, + "userLabels": { + "my-key": "my-value" + } + }, + "databaseVersion": "MYSQL_5_7", + "region": "europe-west4", +} +# [END howto_operator_cloudsql_create_body] +# [START howto_operator_cloudsql_patch_body] +patch_body = { + "name": INSTANCE_NAME, + "settings": { + "dataDiskSizeGb": 35, + "maintenanceWindow": { + "hour": 3, + "day": 6, + "updateTrack": "canary" + }, + "userLabels": { + "my-key-patch": "my-value-patch" + } + } +} +# [END howto_operator_cloudsql_patch_body] + +default_args = { + 'start_date': airflow.utils.dates.days_ago(1) +} + +with models.DAG( + 'example_gcp_sql', + default_args=default_args, + schedule_interval=datetime.timedelta(days=1) +) as dag: + # [START howto_operator_cloudsql_create] + sql_instance_create_task = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + body=body, + instance=INSTANCE_NAME, + task_id='sql_instance_create_task' + ) + # [END howto_operator_cloudsql_create] + # [START howto_operator_cloudsql_patch] + sql_instance_patch_task = CloudSqlInstancePatchOperator( + project_id=PROJECT_ID, + body=patch_body, + instance=INSTANCE_NAME, + task_id='sql_instance_patch_task' + ) + # [END howto_operator_cloudsql_patch] + # [START howto_operator_cloudsql_delete] + sql_instance_delete_task = CloudSqlInstanceDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + task_id='sql_instance_delete_task' + ) + # [END howto_operator_cloudsql_delete] + + sql_instance_create_task >> sql_instance_patch_task >> sql_instance_delete_task diff --git a/airflow/contrib/hooks/gcp_sql_hook.py b/airflow/contrib/hooks/gcp_sql_hook.py new file mode 100644 index 0000000000000..e0b3f92d8f20e --- /dev/null +++ b/airflow/contrib/hooks/gcp_sql_hook.py @@ -0,0 +1,173 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import time +from googleapiclient.discovery import build + +from airflow import AirflowException +from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook + +# Number of retries - used by googleapiclient method calls to perform retries +# For requests that are "retriable" +NUM_RETRIES = 5 + +# Time to sleep between active checks of the operation results +TIME_TO_SLEEP_IN_SECONDS = 1 + + +class CloudSqlOperationStatus: + PENDING = "PENDING" + RUNNING = "RUNNING" + DONE = "DONE" + UNKNOWN = "UNKNOWN" + + +# noinspection PyAbstractClass +class CloudSqlHook(GoogleCloudBaseHook): + """ + Hook for Google Cloud SQL APIs. + """ + _conn = None + + def __init__(self, + api_version, + gcp_conn_id='google_cloud_default', + delegate_to=None): + super(CloudSqlHook, self).__init__(gcp_conn_id, delegate_to) + self.api_version = api_version + + def get_conn(self): + """ + Retrieves connection to Cloud SQL. + + :return: Google Cloud SQL services object. + :rtype: dict + """ + if not self._conn: + http_authorized = self._authorize() + self._conn = build('sqladmin', self.api_version, + http=http_authorized, cache_discovery=False) + return self._conn + + def get_instance(self, project_id, instance): + """ + Retrieves a resource containing information about a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :return: A Cloud SQL instance resource. + :rtype: dict + """ + return self.get_conn().instances().get( + project=project_id, + instance=instance + ).execute(num_retries=NUM_RETRIES) + + def create_instance(self, project_id, body): + """ + Creates a new Cloud SQL instance. + + :param project_id: Project ID of the project to which the newly created + Cloud SQL instances should belong. + :type project_id: str + :param body: Body required by the Cloud SQL insert API, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/insert#request-body + :type body: dict + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().instances().insert( + project=project_id, + body=body + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project_id, operation_name) + + def patch_instance(self, project_id, body, instance): + """ + Updates settings of a Cloud SQL instance. + + Caution: This is not a partial update, so you must include values for + all the settings that you want to retain. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param body: Body required by the Cloud SQL patch API, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/patch#request-body + :type body: dict + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().instances().patch( + project=project_id, + instance=instance, + body=body + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project_id, operation_name) + + def delete_instance(self, project_id, instance): + """ + Deletes a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().instances().delete( + project=project_id, + instance=instance, + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project_id, operation_name) + + def _wait_for_operation_to_complete(self, project_id, operation_name): + """ + Waits for the named operation to complete - checks status of the + asynchronous call. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param operation_name: name of the operation + :type operation_name: str + :return: response returned by the operation + :rtype: dict + """ + service = self.get_conn() + while True: + operation_response = service.operations().get( + project=project_id, + operation=operation_name, + ).execute(num_retries=NUM_RETRIES) + if operation_response.get("status") == CloudSqlOperationStatus.DONE: + error = operation_response.get("error") + if error: + # Extracting the errors list as string and trimming square braces + error_msg = str(error.get("errors"))[1:-1] + raise AirflowException(error_msg) + # No meaningful info to return from the response in case of success + return True + time.sleep(TIME_TO_SLEEP_IN_SECONDS) diff --git a/airflow/contrib/operators/gcp_sql_operator.py b/airflow/contrib/operators/gcp_sql_operator.py new file mode 100644 index 0000000000000..0ba7a300c9e41 --- /dev/null +++ b/airflow/contrib/operators/gcp_sql_operator.py @@ -0,0 +1,297 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from googleapiclient.errors import HttpError + +from airflow import AirflowException +from airflow.contrib.hooks.gcp_sql_hook import CloudSqlHook +from airflow.contrib.utils.gcp_field_validator import GcpBodyFieldValidator +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + +SETTINGS = 'settings' +SETTINGS_VERSION = 'settingsVersion' + +CLOUD_SQL_VALIDATION = [ + dict(name="name", allow_empty=False), + dict(name="settings", type="dict", fields=[ + dict(name="tier", allow_empty=False), + dict(name="backupConfiguration", type="dict", fields=[ + dict(name="binaryLogEnabled", optional=True), + dict(name="enabled", optional=True), + dict(name="replicationLogArchivingEnabled", optional=True), + dict(name="startTime", allow_empty=False, optional=True) + ], optional=True), + dict(name="activationPolicy", allow_empty=False, optional=True), + dict(name="authorizedGaeApplications", type="list", optional=True), + dict(name="crashSafeReplicationEnabled", optional=True), + dict(name="dataDiskSizeGb", optional=True), + dict(name="dataDiskType", allow_empty=False, optional=True), + dict(name="databaseFlags", type="list", optional=True), + dict(name="ipConfiguration", type="dict", fields=[ + dict(name="authorizedNetworks", type="list", fields=[ + dict(name="expirationTime", optional=True), + dict(name="name", allow_empty=False, optional=True), + dict(name="value", allow_empty=False, optional=True) + ], optional=True), + dict(name="ipv4Enabled", optional=True), + dict(name="privateNetwork", allow_empty=False, optional=True), + dict(name="requireSsl", optional=True), + ], optional=True), + dict(name="locationPreference", type="dict", fields=[ + dict(name="followGaeApplication", allow_empty=False, optional=True), + dict(name="zone", allow_empty=False, optional=True), + ], optional=True), + dict(name="maintenanceWindow", type="dict", fields=[ + dict(name="hour", optional=True), + dict(name="day", optional=True), + dict(name="updateTrack", allow_empty=False, optional=True), + ], optional=True), + dict(name="pricingPlan", allow_empty=False, optional=True), + dict(name="replicationType", allow_empty=False, optional=True), + dict(name="storageAutoResize", optional=True), + dict(name="storageAutoResizeLimit", optional=True), + dict(name="userLabels", type="dict", optional=True), + ]), + dict(name="databaseVersion", allow_empty=False, optional=True), + dict(name="failoverReplica", type="dict", fields=[ + dict(name="name", allow_empty=False) + ], optional=True), + dict(name="masterInstanceName", allow_empty=False, optional=True), + dict(name="onPremisesConfiguration", type="dict", optional=True), + dict(name="region", allow_empty=False, optional=True), + dict(name="replicaConfiguration", type="dict", fields=[ + dict(name="failoverTarget", optional=True), + dict(name="mysqlReplicaConfiguration", type="dict", fields=[ + dict(name="caCertificate", allow_empty=False, optional=True), + dict(name="clientCertificate", allow_empty=False, optional=True), + dict(name="clientKey", allow_empty=False, optional=True), + dict(name="connectRetryInterval", optional=True), + dict(name="dumpFilePath", allow_empty=False, optional=True), + dict(name="masterHeartbeatPeriod", optional=True), + dict(name="password", allow_empty=False, optional=True), + dict(name="sslCipher", allow_empty=False, optional=True), + dict(name="username", allow_empty=False, optional=True), + dict(name="verifyServerCertificate", optional=True) + ], optional=True), + ], optional=True) +] + + +class CloudSqlBaseOperator(BaseOperator): + """ + Abstract base operator for Google Cloud SQL operators to inherit from. + + :param project_id: Project ID of the Google Cloud Platform project to operate it. + :type project_id: str + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1). + :type api_version: str + """ + @apply_defaults + def __init__(self, + project_id, + instance, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + *args, **kwargs): + self.project_id = project_id + self.instance = instance + self.gcp_conn_id = gcp_conn_id + self.api_version = api_version + self._validate_inputs() + self._hook = CloudSqlHook(gcp_conn_id=self.gcp_conn_id, + api_version=self.api_version) + super(CloudSqlBaseOperator, self).__init__(*args, **kwargs) + + def _validate_inputs(self): + if not self.project_id: + raise AirflowException("The required parameter 'project_id' is empty") + if not self.instance: + raise AirflowException("The required parameter 'instance' is empty") + + def _check_if_instance_exists(self, instance): + try: + return self._hook.get_instance(self.project_id, instance) + except HttpError as e: + status = e.resp.status + if status == 404: + return False + raise e + + def execute(self, context): + pass + + @staticmethod + def _get_settings_version(instance): + return instance.get(SETTINGS).get(SETTINGS_VERSION) + + +class CloudSqlInstanceCreateOperator(CloudSqlBaseOperator): + """ + Creates a new Cloud SQL instance. + If an instance with the same name exists, no action will be taken and + the operator will succeed. + + :param project_id: Project ID of the project to which the newly created Cloud SQL + instances should belong. + :type project_id: str + :param body: Body required by the Cloud SQL insert API, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/insert + #request-body + :type body: dict + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1). + :type api_version: str + :param validate_body: True if body should be validated, False otherwise. + :type validate_body: bool + """ + # [START gcp_sql_create_template_fields] + template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version') + # [END gcp_sql_create_template_fields] + + @apply_defaults + def __init__(self, + project_id, + body, + instance, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + validate_body=True, + *args, **kwargs): + self.body = body + self.validate_body = validate_body + super(CloudSqlInstanceCreateOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def _validate_inputs(self): + super(CloudSqlInstanceCreateOperator, self)._validate_inputs() + if not self.body: + raise AirflowException("The required parameter 'body' is empty") + + def _validate_body_fields(self): + if self.validate_body: + GcpBodyFieldValidator(CLOUD_SQL_VALIDATION, + api_version=self.api_version).validate(self.body) + + def execute(self, context): + self._validate_body_fields() + if not self._check_if_instance_exists(self.instance): + return self._hook.create_instance(self.project_id, self.body) + else: + self.log.info("Cloud SQL instance with ID {} already exists. " + "Aborting create.".format(self.instance)) + return True + + +class CloudSqlInstancePatchOperator(CloudSqlBaseOperator): + """ + Updates settings of a Cloud SQL instance. + + Caution: This is a partial update, so only included values for the settings will be + updated. + + In the request body, supply the relevant portions of an instance resource, according + to the rules of patch semantics. + https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param body: Body required by the Cloud SQL patch API, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/instances/patch#request-body + :type body: dict + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1). + :type api_version: str + """ + # [START gcp_sql_patch_template_fields] + template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version') + # [END gcp_sql_patch_template_fields] + + @apply_defaults + def __init__(self, + project_id, + body, + instance, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + *args, **kwargs): + self.body = body + super(CloudSqlInstancePatchOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def _validate_inputs(self): + super(CloudSqlInstancePatchOperator, self)._validate_inputs() + if not self.body: + raise AirflowException("The required parameter 'body' is empty") + + def execute(self, context): + if not self._check_if_instance_exists(self.instance): + raise AirflowException('Cloud SQL instance with ID {} does not exist. ' + 'Please specify another instance to patch.' + .format(self.instance)) + else: + return self._hook.patch_instance(self.project_id, self.body, self.instance) + + +class CloudSqlInstanceDeleteOperator(CloudSqlBaseOperator): + """ + Deletes a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance to be deleted. + :type project_id: str + :param instance: Cloud SQL instance ID. This does not include the project ID. + :type instance: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1). + :type api_version: str + """ + # [START gcp_sql_delete_template_fields] + template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version') + # [END gcp_sql_delete_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + *args, **kwargs): + super(CloudSqlInstanceDeleteOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def execute(self, context): + if not self._check_if_instance_exists(self.instance): + print("Cloud SQL instance with ID {} does not exist. Aborting delete." + .format(self.instance)) + return True + else: + return self._hook.delete_instance(self.project_id, self.instance) diff --git a/airflow/contrib/utils/gcp_field_validator.py b/airflow/contrib/utils/gcp_field_validator.py index 20f72d94b813a..e8b59c855e27c 100644 --- a/airflow/contrib/utils/gcp_field_validator.py +++ b/airflow/contrib/utils/gcp_field_validator.py @@ -69,17 +69,23 @@ Each of the fields in the array is then expected (unless marked as optional) and validated recursively. If an extra field is present in the dictionary, warning is printed in log file (but the validation succeeds - see the Forward-compatibility notes) +* List fields: (key = "type", value="list"): + Field of this type should be a list. Only the type correctness is validated. + The contents of a list are not subject to validation. * Union fields (key = "type", value="union"): field of this type should contain nested fields in form of an array of dicts. One of the fields (and only one) should be present (unless the union is marked as optional). If more than one union field is present, FieldValidationException is raised. If none of the union fields is present - warning is printed in the log (see below Forward-compatibility notes). +* Fields validated for non-emptiness: (key = "allow_empty") - this applies only to + fields the value of which is a string, and it allows to check for non-emptiness of + the field (allow_empty=False). * Regexp-validated fields: (key = "regexp") - fields of this type are assumed to be strings and they are validated with the regexp specified. Remember that the regexps should ideally contain ^ at the beginning and $ at the end to make sure that the whole field content is validated. Typically such regexp validations should be used carefully and sparingly (see Forward-compatibility - notes below). Most of regexp validation should be at most r'^.+$'. + notes below). * Custom-validated fields: (key = "custom_validation") - fields of this type are validated using method specified via custom_validation field. Any exception thrown in the custom validation will be turned into FieldValidationException and will cause validation to @@ -130,7 +136,7 @@ from airflow import LoggingMixin, AirflowException -COMPOSITE_FIELD_TYPES = ['union', 'dict'] +COMPOSITE_FIELD_TYPES = ['union', 'dict', 'list'] class GcpFieldValidationException(AirflowException): @@ -158,8 +164,8 @@ def _int_greater_than_zero(value): EXAMPLE_VALIDATION_SPECIFICATION = [ - dict(name="name", regexp="^.+$"), - dict(name="description", regexp="^.+$", optional=True), + dict(name="name", allow_empty=False), + dict(name="description", allow_empty=False, optional=True), dict(name="availableMemoryMb", custom_validation=_int_greater_than_zero, optional=True), dict(name="labels", optional=True, type="dict"), @@ -202,7 +208,7 @@ def _get_field_name_with_parent(field_name, parent): @staticmethod def _sanity_checks(children_validation_specs, field_type, full_field_path, - regexp, custom_validation, value): + regexp, allow_empty, custom_validation, value): # type: (dict, str, str, str, function, object) -> None if value is None and field_type != 'union': raise GcpFieldValidationException( @@ -213,6 +219,11 @@ def _sanity_checks(children_validation_specs, field_type, full_field_path, "The validation specification entry '{}' has both type and regexp. " "The regexp is only allowed without type (i.e. assume type is 'str' " "that can be validated with regexp)".format(full_field_path)) + if allow_empty is not None and field_type: + raise GcpValidationSpecificationException( + "The validation specification entry '{}' has both type and allow_empty. " + "The allow_empty is only allowed without type (i.e. assume type is 'str' " + "that can be validated with allow_empty)".format(full_field_path)) if children_validation_specs and field_type not in COMPOSITE_FIELD_TYPES: raise GcpValidationSpecificationException( "Nested fields are specified in field '{}' of type '{}'. " @@ -234,6 +245,14 @@ def _validate_regexp(full_field_path, regexp, value): "specification regexp: '{}'.". format(full_field_path, value, regexp)) + @staticmethod + def _validate_is_empty(full_field_path, value): + # type: (str, str) -> None + if not value: + raise GcpFieldValidationException( + "The body field '{}' can't be empty. Please provide a value." + .format(full_field_path, value)) + def _validate_dict(self, children_validation_specs, full_field_path, value): # type: (dict, str, dict) -> None for child_validation_spec in children_validation_specs: @@ -306,6 +325,7 @@ def _validate_field(self, validation_spec, dictionary_to_validate, parent=None, field_type = validation_spec.get('type') optional = validation_spec.get('optional') regexp = validation_spec.get('regexp') + allow_empty = validation_spec.get('allow_empty') children_validation_specs = validation_spec.get('fields') required_api_version = validation_spec.get('api_version') custom_validation = validation_spec.get('custom_validation') @@ -332,15 +352,18 @@ def _validate_field(self, validation_spec, dictionary_to_validate, parent=None, field_type=field_type, full_field_path=full_field_path, regexp=regexp, + allow_empty=allow_empty, custom_validation=custom_validation, value=value) + if allow_empty is False: + self._validate_is_empty(full_field_path, value) if regexp: self._validate_regexp(full_field_path, regexp, value) elif field_type == 'dict': if not isinstance(value, dict): raise GcpFieldValidationException( - "The field '{}' should be dictionary type according to " + "The field '{}' should be of dictionary type according to the " "specification '{}' but it is '{}'". format(full_field_path, validation_spec, value)) if children_validation_specs is None: @@ -359,6 +382,12 @@ def _validate_field(self, validation_spec, dictionary_to_validate, parent=None, "nested field defined.".format(full_field_path, validation_spec)) self._validate_union(children_validation_specs, full_field_path, dictionary_to_validate) + elif field_type == 'list': + if not isinstance(value, list): + raise GcpFieldValidationException( + "The field '{}' should be of list type according to the " + "specification '{}' but it is '{}'". + format(full_field_path, validation_spec, value)) elif custom_validation: try: custom_validation(value) diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index 6138d5eca71e1..025274a5a41ae 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -283,3 +283,155 @@ See `Adding the IAM service agent user role to the runtime service `_. + +.. _CloudSqlInstanceCreateOperator: + +CloudSqlInstanceCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Creates a new Cloud SQL instance in Google Cloud Platform. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceCreateOperator`. + +If an instance with the same name exists, no action will be taken and the operator +will succeed. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from Airflow variables: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_arguments] + :end-before: [END howto_operator_cloudsql_arguments] + +Example body defining the instance: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_create_body] + :end-before: [END howto_operator_cloudsql_create_body] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_cloudsql_create] + :end-before: [END howto_operator_cloudsql_create] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_sql_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_sql_create_template_fields] + :end-before: [END gcp_sql_create_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud SQL API documentation for insert `_. + + +.. _CloudSqlInstancePatchOperator: + +CloudSqlInstancePatchOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Updates settings of a Cloud SQL instance in Google Cloud Platform (partial update). + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_sql_operator.CloudSqlInstancePatchOperator`. + +This is a partial update, so only values for the settings specified in the body +will be set / updated. The rest of the existing instance's configuration will remain +unchanged. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from Airflow variables: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_arguments] + :end-before: [END howto_operator_cloudsql_arguments] + +Example body defining the instance: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_patch_body] + :end-before: [END howto_operator_cloudsql_patch_body] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_cloudsql_patch] + :end-before: [END howto_operator_cloudsql_patch] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_sql_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_sql_patch_template_fields] + :end-before: [END gcp_sql_patch_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud SQL API documentation for patch `_. diff --git a/docs/integration.rst b/docs/integration.rst index cd63d60b64c95..e72f687151de5 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -439,6 +439,43 @@ BigQueryHook .. autoclass:: airflow.contrib.hooks.bigquery_hook.BigQueryHook :members: +Cloud SQL +''''''''' + +Cloud SQL Operators +""""""""""""""""""" + +- :ref:`CloudSqlInstanceDeleteOperator` : delete a Cloud SQL instance. +- :ref:`CloudSqlInstanceCreateOperator` : create a new Cloud SQL instance. +- :ref:`CloudSqlInstancePatchOperator` : patch a Cloud SQL instance. + +.. CloudSqlInstanceDeleteOperator: + +CloudSqlInstanceDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDeleteOperator + +.. CloudSqlInstanceCreateOperator: + +CloudSqlInstanceCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceCreateOperator + +.. CloudSqlInstancePatchOperator: + +CloudSqlInstancePatchOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstancePatchOperator + +Cloud SQL Hook +"""""""""""""""""""" + +.. autoclass:: airflow.contrib.hooks.gcp_sql_hook.CloudSqlHook + :members: + Compute Engine '''''''''''''' diff --git a/tests/contrib/operators/test_gcp_function_operator.py b/tests/contrib/operators/test_gcp_function_operator.py index 4192560dd984c..46d599bf7d35f 100644 --- a/tests/contrib/operators/test_gcp_function_operator.py +++ b/tests/contrib/operators/test_gcp_function_operator.py @@ -344,7 +344,7 @@ def test_invalid_field_values(self, key, value, message, mock_hook): "Parameter 'sourceUploadUrl' is empty in the body and argument " "'zip_path' is missing or empty."), ({'sourceArchiveUrl': 'gs://adasda', 'sourceRepository': ''}, - "The field 'source_code.sourceRepository' should be dictionary type"), + "The field 'source_code.sourceRepository' should be of dictionary type"), ({'sourceUploadUrl': '', 'sourceRepository': ''}, "Parameter 'sourceUploadUrl' is empty in the body and argument 'zip_path' " "is missing or empty."), @@ -360,7 +360,7 @@ def test_invalid_field_values(self, key, value, message, mock_hook): ({'sourceUploadUrl': ''}, "Parameter 'sourceUploadUrl' is empty in the body " "and argument 'zip_path' is missing or empty."), ({'sourceRepository': ''}, "The field 'source_code.sourceRepository' " - "should be dictionary type"), + "should be of dictionary type"), ({'sourceRepository': {}}, "The required body field " "'source_code.sourceRepository.url' is missing"), ({'sourceRepository': {'url': ''}}, @@ -452,7 +452,7 @@ def test_valid_source_code_union_field(self, source_code, mock_hook): 'service': 'service_name', 'failurePolicy': {'retry': ''}}}, "The field 'trigger.eventTrigger.failurePolicy.retry' " - "should be dictionary type") + "should be of dictionary type") ] ) @mock.patch('airflow.contrib.operators.gcp_function_operator.GcfHook') diff --git a/tests/contrib/operators/test_gcp_sql_operator.py b/tests/contrib/operators/test_gcp_sql_operator.py new file mode 100644 index 0000000000000..245631808a223 --- /dev/null +++ b/tests/contrib/operators/test_gcp_sql_operator.py @@ -0,0 +1,314 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from airflow import AirflowException +from airflow.contrib.operators.gcp_sql_operator import CloudSqlInstanceCreateOperator, \ + CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator + +try: + # noinspection PyProtectedMember + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +PROJECT_ID = "project-id" +INSTANCE_NAME = "test-name" +CREATE_BODY = { + "name": INSTANCE_NAME, + "settings": { + "tier": "db-n1-standard-1", + "backupConfiguration": { + "binaryLogEnabled": True, + "enabled": True, + "replicationLogArchivingEnabled": True, + "startTime": "05:00" + }, + "activationPolicy": "ALWAYS", + "authorizedGaeApplications": [], + "crashSafeReplicationEnabled": True, + "dataDiskSizeGb": 30, + "dataDiskType": "PD_SSD", + "databaseFlags": [], + "ipConfiguration": { + "ipv4Enabled": True, + "authorizedNetworks": [ + { + "value": "192.168.100.0/24", + "name": "network1", + "expirationTime": "2012-11-15T16:19:00.094Z" + }, + ], + "privateNetwork": "/vpc/resource/link", + "requireSsl": True + }, + "locationPreference": { + "zone": "europe-west4-a", + "followGaeApplication": "/app/engine/application/to/follow" + }, + "maintenanceWindow": { + "hour": 5, + "day": 7, + "updateTrack": "canary" + }, + "pricingPlan": "PER_USE", + "replicationType": "ASYNCHRONOUS", + "storageAutoResize": False, + "storageAutoResizeLimit": 0, + "userLabels": { + "my-key": "my-value" + } + }, + "databaseVersion": "MYSQL_5_7", + "failoverReplica": { + "name": "replica-1" + }, + "masterInstanceName": "master-instance-1", + "onPremisesConfiguration": {}, + "region": "europe-west4", + "replicaConfiguration": { + "mysqlReplicaConfiguration": { + "caCertificate": "cert-pem", + "clientCertificate": "cert-pem", + "clientKey": "cert-pem", + "connectRetryInterval": 30, + "dumpFilePath": "/path/to/dump", + "masterHeartbeatPeriod": 100, + "password": "secret_pass", + "sslCipher": "list-of-ciphers", + "username": "user", + "verifyServerCertificate": True + }, + } +} +PATCH_BODY = { + "name": INSTANCE_NAME, + "settings": { + "tier": "db-n1-standard-2", + "dataDiskType": "PD_HDD" + }, + "region": "europe-west4" +} + + +class CloudSqlTest(unittest.TestCase): + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceCreateOperator._check_if_instance_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_create(self, mock_hook, _check_if_instance_exists): + _check_if_instance_exists.return_value = False + mock_hook.return_value.create_instance.return_value = True + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + body=CREATE_BODY, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_instance.assert_called_once_with( + PROJECT_ID, CREATE_BODY + ) + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceCreateOperator._check_if_instance_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_create_idempotent(self, mock_hook, _check_if_instance_exists): + _check_if_instance_exists.return_value = True + mock_hook.return_value.create_instance.return_value = True + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + body=CREATE_BODY, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_instance.assert_not_called() + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_create_should_throw_ex_when_empty_project_id(self, mock_hook): + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceCreateOperator( + project_id="", + body=CREATE_BODY, + instance=INSTANCE_NAME, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The required parameter 'project_id' is empty", str(err)) + mock_hook.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_create_should_throw_ex_when_empty_body(self, mock_hook): + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + body={}, + instance=INSTANCE_NAME, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The required parameter 'body' is empty", str(err)) + mock_hook.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_create_should_throw_ex_when_empty_instance(self, mock_hook): + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + body=CREATE_BODY, + instance="", + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The required parameter 'instance' is empty", str(err)) + mock_hook.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_create_should_validate_list_type(self, mock_hook): + wrong_list_type_body = { + "name": INSTANCE_NAME, + "settings": { + "tier": "db-n1-standard-1", + "ipConfiguration": { + "authorizedNetworks": {} # Should be a list, not a dict. + # Testing if the validation catches this. + } + } + } + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + body=wrong_list_type_body, + instance=INSTANCE_NAME, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The field 'settings.ipConfiguration.authorizedNetworks' " + "should be of list type according to the specification", str(err)) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_create_should_validate_non_empty_fields(self, mock_hook): + empty_tier_body = { + "name": INSTANCE_NAME, + "settings": { + "tier": "", # Field can't be empty (defined in CLOUD_SQL_VALIDATION). + # Testing if the validation catches this. + } + } + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceCreateOperator( + project_id=PROJECT_ID, + body=empty_tier_body, + instance=INSTANCE_NAME, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The body field 'settings.tier' can't be empty. " + "Please provide a value.", str(err)) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_patch(self, mock_hook): + mock_hook.return_value.patch_instance.return_value = True + op = CloudSqlInstancePatchOperator( + project_id=PROJECT_ID, + body=PATCH_BODY, + instance=INSTANCE_NAME, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.patch_instance.assert_called_once_with( + PROJECT_ID, PATCH_BODY, INSTANCE_NAME + ) + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstancePatchOperator._check_if_instance_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_patch_should_bubble_up_ex_if_not_exists(self, mock_hook, + _check_if_instance_exists): + _check_if_instance_exists.return_value = False + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstancePatchOperator( + project_id=PROJECT_ID, + body=PATCH_BODY, + instance=INSTANCE_NAME, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn('specify another instance to patch', str(err)) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.patch_instance.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDeleteOperator._check_if_instance_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_delete(self, mock_hook, _check_if_instance_exists): + _check_if_instance_exists.return_value = True + op = CloudSqlInstanceDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + task_id="id" + ) + result = op.execute(None) + self.assertTrue(result) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_instance.assert_called_once_with( + PROJECT_ID, INSTANCE_NAME + ) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDeleteOperator._check_if_instance_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_delete_should_abort_and_succeed_if_not_exists( + self, mock_hook, _check_if_instance_exists): + _check_if_instance_exists.return_value = False + op = CloudSqlInstanceDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + task_id="id" + ) + result = op.execute(None) + self.assertTrue(result) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_instance.assert_not_called() From ae39df5101e4e6f2c22568d87c1a42a87658eba8 Mon Sep 17 00:00:00 2001 From: yangaws <31293788+yangaws@users.noreply.github.com> Date: Thu, 1 Nov 2018 13:31:59 -0700 Subject: [PATCH 141/808] [AIRFLOW-2524] Update SageMaker hook and operators (#4091) This re-works the SageMaker functionality in Airflow to be more complete, and more useful for the kinds of operations that SageMaker supports. We removed some files and operators here, but these were only added after the last release so we don't need to worry about any sort of back-compat. --- airflow/contrib/hooks/aws_hook.py | 24 +- airflow/contrib/hooks/sagemaker_hook.py | 825 ++++++++++++++---- .../operators/sagemaker_base_operator.py | 100 +++ .../sagemaker_create_training_job_operator.py | 119 --- ...sagemaker_create_transform_job_operator.py | 132 --- .../sagemaker_create_tuning_job_operator.py | 121 --- .../operators/sagemaker_training_operator.py | 97 ++ .../operators/sagemaker_transform_operator.py | 112 +++ .../operators/sagemaker_tuning_operator.py | 97 ++ .../contrib/sensors/sagemaker_base_sensor.py | 14 +- .../sensors/sagemaker_training_sensor.py | 58 +- .../sensors/sagemaker_transform_sensor.py | 9 +- .../sensors/sagemaker_tuning_sensor.py | 11 +- airflow/hooks/S3_hook.py | 141 +++ docs/code.rst | 9 + tests/contrib/hooks/test_aws_hook.py | 32 +- tests/contrib/hooks/test_sagemaker_hook.py | 748 +++++++++------- .../operators/test_sagemaker_base_operator.py | 84 ++ ..._sagemaker_create_training_job_operator.py | 141 --- .../test_sagemaker_training_operator.py | 134 +++ .../test_sagemaker_transform_operator.py | 136 +++ ...r.py => test_sagemaker_tuning_operator.py} | 67 +- .../sensors/test_sagemaker_base_sensor.py | 12 +- .../sensors/test_sagemaker_training_sensor.py | 104 ++- .../test_sagemaker_transform_sensor.py | 36 +- .../sensors/test_sagemaker_tuning_sensor.py | 39 +- tests/hooks/test_s3_hook.py | 41 + ...sagemaker_create_transform_job_operator.py | 140 --- 28 files changed, 2279 insertions(+), 1304 deletions(-) create mode 100644 airflow/contrib/operators/sagemaker_base_operator.py delete mode 100644 airflow/contrib/operators/sagemaker_create_training_job_operator.py delete mode 100644 airflow/contrib/operators/sagemaker_create_transform_job_operator.py delete mode 100644 airflow/contrib/operators/sagemaker_create_tuning_job_operator.py create mode 100644 airflow/contrib/operators/sagemaker_training_operator.py create mode 100644 airflow/contrib/operators/sagemaker_transform_operator.py create mode 100644 airflow/contrib/operators/sagemaker_tuning_operator.py create mode 100644 tests/contrib/operators/test_sagemaker_base_operator.py delete mode 100644 tests/contrib/operators/test_sagemaker_create_training_job_operator.py create mode 100644 tests/contrib/operators/test_sagemaker_training_operator.py create mode 100644 tests/contrib/operators/test_sagemaker_transform_operator.py rename tests/contrib/operators/{test_sagemaker_create_tuning_job_operator.py => test_sagemaker_tuning_operator.py} (71%) delete mode 100644 tests/operators/test_sagemaker_create_transform_job_operator.py diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py index dc945e3ada091..44f33d49ab9e4 100644 --- a/airflow/contrib/hooks/aws_hook.py +++ b/airflow/contrib/hooks/aws_hook.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. - import boto3 import configparser import logging @@ -163,15 +162,17 @@ def _get_credentials(self, region_name): aws_session_token=aws_session_token, region_name=region_name), endpoint_url - def get_client_type(self, client_type, region_name=None): + def get_client_type(self, client_type, region_name=None, config=None): session, endpoint_url = self._get_credentials(region_name) - return session.client(client_type, endpoint_url=endpoint_url) + return session.client(client_type, endpoint_url=endpoint_url, + config=config) - def get_resource_type(self, resource_type, region_name=None): + def get_resource_type(self, resource_type, region_name=None, config=None): session, endpoint_url = self._get_credentials(region_name) - return session.resource(resource_type, endpoint_url=endpoint_url) + return session.resource(resource_type, endpoint_url=endpoint_url, + config=config) def get_session(self, region_name=None): """Get the underlying boto3.session.""" @@ -188,3 +189,16 @@ def get_credentials(self, region_name=None): # secret key separately can lead to a race condition. # See https://stackoverflow.com/a/36291428/8283373 return session.get_credentials().get_frozen_credentials() + + def expand_role(self, role): + """ + Expand an IAM role name to an IAM role ARN. If role is already an IAM ARN, + no change is made. + + :param role: IAM role name or ARN + :return: IAM role ARN + """ + if '/' in role: + return role + else: + return self.get_client_type('iam').get_role(RoleName=role)['Role']['Arn'] diff --git a/airflow/contrib/hooks/sagemaker_hook.py b/airflow/contrib/hooks/sagemaker_hook.py index ebab5d83e4099..823f430db994f 100644 --- a/airflow/contrib/hooks/sagemaker_hook.py +++ b/airflow/contrib/hooks/sagemaker_hook.py @@ -16,299 +16,746 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import copy +import tarfile +import tempfile import time +import os +import collections + +import botocore.config from botocore.exceptions import ClientError from airflow.exceptions import AirflowException from airflow.contrib.hooks.aws_hook import AwsHook from airflow.hooks.S3_hook import S3Hook +from airflow.utils import timezone + + +class LogState(object): + STARTING = 1 + WAIT_IN_PROGRESS = 2 + TAILING = 3 + JOB_COMPLETE = 4 + COMPLETE = 5 + + +# Position is a tuple that includes the last read timestamp and the number of items that were read +# at that time. This is used to figure out which event to start with on the next read. +Position = collections.namedtuple('Position', ['timestamp', 'skip']) + + +def argmin(arr, f): + """Return the index, i, in arr that minimizes f(arr[i])""" + m = None + i = None + for idx, item in enumerate(arr): + if item is not None: + if m is None or f(item) < m: + m = f(item) + i = idx + return i + + +def secondary_training_status_changed(current_job_description, prev_job_description): + """ + Returns true if training job's secondary status message has changed. + + :param current_job_description: Current job description, returned from DescribeTrainingJob call. + :type current_job_description: dict + :param prev_job_description: Previous job description, returned from DescribeTrainingJob call. + :type prev_job_description: dict + + :return: Whether the secondary status message of a training job changed or not. + """ + current_secondary_status_transitions = current_job_description.get('SecondaryStatusTransitions') + if current_secondary_status_transitions is None or len(current_secondary_status_transitions) == 0: + return False + + prev_job_secondary_status_transitions = prev_job_description.get('SecondaryStatusTransitions') \ + if prev_job_description is not None else None + + last_message = prev_job_secondary_status_transitions[-1]['StatusMessage'] \ + if prev_job_secondary_status_transitions is not None \ + and len(prev_job_secondary_status_transitions) > 0 else '' + + message = current_job_description['SecondaryStatusTransitions'][-1]['StatusMessage'] + + return message != last_message + + +def secondary_training_status_message(job_description, prev_description): + """ + Returns a string contains start time and the secondary training job status message. + + :param job_description: Returned response from DescribeTrainingJob call + :type job_description: dict + :param prev_description: Previous job description from DescribeTrainingJob call + :type prev_description: dict + + :return: Job status string to be printed. + """ + + if job_description is None or job_description.get('SecondaryStatusTransitions') is None\ + or len(job_description.get('SecondaryStatusTransitions')) == 0: + return '' + + prev_description_secondary_transitions = prev_description.get('SecondaryStatusTransitions')\ + if prev_description is not None else None + prev_transitions_num = len(prev_description['SecondaryStatusTransitions'])\ + if prev_description_secondary_transitions is not None else 0 + current_transitions = job_description['SecondaryStatusTransitions'] + + transitions_to_print = current_transitions[-1:] if len(current_transitions) == prev_transitions_num else \ + current_transitions[prev_transitions_num - len(current_transitions):] + + status_strs = [] + for transition in transitions_to_print: + message = transition['StatusMessage'] + time_str = timezone.convert_to_utc(job_description['LastModifiedTime']).strftime('%Y-%m-%d %H:%M:%S') + status_strs.append('{} {} - {}'.format(time_str, transition['Status'], message)) + + return '\n'.join(status_strs) class SageMakerHook(AwsHook): """ Interact with Amazon SageMaker. - sagemaker_conn_id is required for using - the config stored in db for training/tuning """ - non_terminal_states = {'InProgress', 'Stopping', 'Stopped'} + non_terminal_states = {'InProgress', 'Stopping'} + endpoint_non_terminal_states = {'Creating', 'Updating', 'SystemUpdating', + 'RollingBack', 'Deleting'} failed_states = {'Failed'} def __init__(self, - sagemaker_conn_id=None, - use_db_config=False, - region_name=None, - check_interval=5, - max_ingestion_time=None, *args, **kwargs): super(SageMakerHook, self).__init__(*args, **kwargs) - self.sagemaker_conn_id = sagemaker_conn_id - self.use_db_config = use_db_config - self.region_name = region_name - self.check_interval = check_interval - self.max_ingestion_time = max_ingestion_time - self.conn = self.get_conn() + self.s3_hook = S3Hook(aws_conn_id=self.aws_conn_id) + + def tar_and_s3_upload(self, path, key, bucket): + """ + Tar the local file or directory and upload to s3 + + :param path: local file or directory + :type path: str + :param key: s3 key + :type key: str + :param bucket: s3 bucket + :type bucket: str + :return: None + """ + with tempfile.TemporaryFile() as temp_file: + if os.path.isdir(path): + files = [os.path.join(path, name) for name in os.listdir(path)] + else: + files = [path] + with tarfile.open(mode='w:gz', fileobj=temp_file) as tar_file: + for f in files: + tar_file.add(f, arcname=os.path.basename(f)) + temp_file.seek(0) + self.s3_hook.load_file_obj(temp_file, key, bucket, replace=True) + + def configure_s3_resources(self, config): + """ + Extract the S3 operations from the configuration and execute them. - def check_for_url(self, s3url): + :param config: config of SageMaker operation + :type config: dict + :return: dict + """ + s3_operations = config.pop('S3Operations', None) + + if s3_operations is not None: + create_bucket_ops = s3_operations.get('S3CreateBucket', []) + upload_ops = s3_operations.get('S3Upload', []) + for op in create_bucket_ops: + self.s3_hook.create_bucket(bucket_name=op['Bucket']) + for op in upload_ops: + if op['Tar']: + self.tar_and_s3_upload(op['Path'], op['Key'], + op['Bucket']) + else: + self.s3_hook.load_file(op['Path'], op['Key'], + op['Bucket']) + + def check_s3_url(self, s3url): """ - check if the s3url exists + Check if an S3 URL exists + :param s3url: S3 url :type s3url:str :return: bool """ bucket, key = S3Hook.parse_s3_url(s3url) - s3hook = S3Hook(aws_conn_id=self.aws_conn_id) - if not s3hook.check_for_bucket(bucket_name=bucket): + if not self.s3_hook.check_for_bucket(bucket_name=bucket): raise AirflowException( "The input S3 Bucket {} does not exist ".format(bucket)) - if key and not s3hook.check_for_key(key=key, bucket_name=bucket)\ - and not s3hook.check_for_prefix( + if key and not self.s3_hook.check_for_key(key=key, bucket_name=bucket)\ + and not self.s3_hook.check_for_prefix( prefix=key, bucket_name=bucket, delimiter='/'): # check if s3 key exists in the case user provides a single file - # or if s3 prefix exists in the case user provides a prefix for files + # or if s3 prefix exists in the case user provides multiple files in + # a prefix raise AirflowException("The input S3 Key " "or Prefix {} does not exist in the Bucket {}" .format(s3url, bucket)) return True - def check_valid_training_input(self, training_config): + def check_training_config(self, training_config): """ - Run checks before a training starts + Check if a training configuration is valid + :param training_config: training_config :type training_config: dict :return: None """ for channel in training_config['InputDataConfig']: - self.check_for_url(channel['DataSource'] - ['S3DataSource']['S3Uri']) + self.check_s3_url(channel['DataSource']['S3DataSource']['S3Uri']) - def check_valid_tuning_input(self, tuning_config): + def check_tuning_config(self, tuning_config): """ - Run checks before a tuning job starts + Check if a tuning configuration is valid + :param tuning_config: tuning_config :type tuning_config: dict :return: None """ for channel in tuning_config['TrainingJobDefinition']['InputDataConfig']: - self.check_for_url(channel['DataSource'] - ['S3DataSource']['S3Uri']) + self.check_s3_url(channel['DataSource']['S3DataSource']['S3Uri']) - def check_status(self, non_terminal_states, - failed_state, key, - describe_function, *args): - """ - :param non_terminal_states: the set of non_terminal states - :type non_terminal_states: set - :param failed_state: the set of failed states - :type failed_state: set - :param key: the key of the response dict - that points to the state - :type key: string - :param describe_function: the function used to retrieve the status - :type describe_function: python callable - :param args: the arguments for the function - :return: None + def get_conn(self): """ - sec = 0 - running = True - - while running: - - sec = sec + self.check_interval - - if self.max_ingestion_time and sec > self.max_ingestion_time: - # ensure that the job gets killed if the max ingestion time is exceeded - raise AirflowException("SageMaker job took more than " - "%s seconds", self.max_ingestion_time) - - time.sleep(self.check_interval) - try: - response = describe_function(*args) - status = response[key] - self.log.info("Job still running for %s seconds... " - "current status is %s" % (sec, status)) - except KeyError: - raise AirflowException("Could not get status of the SageMaker job") - except ClientError: - raise AirflowException("AWS request failed, check log for more info") + Establish an AWS connection for SageMaker - if status in non_terminal_states: - running = True - elif status in failed_state: - raise AirflowException("SageMaker job failed because %s" - % response['FailureReason']) - else: - running = False - - self.log.info('SageMaker Job Compeleted') + :return: a boto3 SageMaker client + """ + return self.get_client_type('sagemaker') - def get_conn(self): + def get_log_conn(self): """ - Establish an AWS connection - :return: a boto3 SageMaker client + Establish an AWS connection for retrieving logs during training + + :return: a boto3 CloudWatchLog client """ - return self.get_client_type('sagemaker', region_name=self.region_name) + config = botocore.config.Config(retries={'max_attempts': 15}) + return self.get_client_type('logs', config=config) - def list_training_job(self, name_contains=None, status_equals=None): + def log_stream(self, log_group, stream_name, start_time=0, skip=0): """ - List the training jobs associated with the given input - :param name_contains: A string in the training job name - :type name_contains: str - :param status_equals: 'InProgress'|'Completed' - |'Failed'|'Stopping'|'Stopped' - :return:dict + A generator for log items in a single stream. This will yield all the + items that are available at the current moment. + + :param log_group: The name of the log group. + :type log_group: str + :param stream_name: The name of the specific stream. + :type stream_name: str + :param start_time: The time stamp value to start reading the logs from (default: 0). + :type start_time: int + :param skip: The number of log entries to skip at the start (default: 0). + This is for when there are multiple entries at the same timestamp. + :type skip: int + :return:A CloudWatch log event with the following key-value pairs: + 'timestamp' (int): The time in milliseconds of the event. + 'message' (str): The log event data. + 'ingestionTime' (int): The time in milliseconds the event was ingested. """ - return self.conn.list_training_jobs( - NameContains=name_contains, StatusEquals=status_equals) - def list_tuning_job(self, name_contains=None, status_equals=None): + next_token = None + + event_count = 1 + while event_count > 0: + if next_token is not None: + token_arg = {'nextToken': next_token} + else: + token_arg = {} + + response = self.get_log_conn().get_log_events(logGroupName=log_group, + logStreamName=stream_name, + startTime=start_time, + startFromHead=True, + **token_arg) + next_token = response['nextForwardToken'] + events = response['events'] + event_count = len(events) + if event_count > skip: + events = events[skip:] + skip = 0 + else: + skip = skip - event_count + events = [] + for ev in events: + yield ev + + def multi_stream_iter(self, log_group, streams, positions=None): """ - List the tuning jobs associated with the given input - :param name_contains: A string in the training job name - :type name_contains: str - :param status_equals: 'InProgress'|'Completed' - |'Failed'|'Stopping'|'Stopped' - :return:dict + Iterate over the available events coming from a set of log streams in a single log group + interleaving the events from each stream so they're yielded in timestamp order. + + :param log_group: The name of the log group. + :type log_group: str + :param streams: A list of the log stream names. The position of the stream in this list is + the stream number. + :type streams: list + :param positions: A list of pairs of (timestamp, skip) which represents the last record + read from each stream. + :type positions: list + :return: A tuple of (stream number, cloudwatch log event). """ - return self.conn.list_hyper_parameter_tuning_job( - NameContains=name_contains, StatusEquals=status_equals) + positions = positions or {s: Position(timestamp=0, skip=0) for s in streams} + event_iters = [self.log_stream(log_group, s, positions[s].timestamp, positions[s].skip) + for s in streams] + events = [next(s) if s else None for s in event_iters] + + while any(events): + i = argmin(events, lambda x: x['timestamp'] if x else 9999999999) + yield (i, events[i]) + try: + events[i] = next(event_iters[i]) + except StopIteration: + events[i] = None - def create_training_job(self, training_job_config, wait_for_completion=True): + def create_training_job(self, config, wait_for_completion=True, print_log=True, + check_interval=30, max_ingestion_time=None): """ Create a training job - :param training_job_config: the config for training - :type training_job_config: dict + + :param config: the config for training + :type config: dict :param wait_for_completion: if the program should keep running until job finishes :type wait_for_completion: bool - :return: A dict that contains ARN of the training job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to training job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException("SageMaker connection id must be present to read \ - SageMaker training jobs configuration.") - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) - - config = copy.deepcopy(sagemaker_conn.extra_dejson) - training_job_config.update(config) - self.check_valid_training_input(training_job_config) + self.check_training_config(config) + + response = self.get_conn().create_training_job(**config) + if print_log: + self.check_training_status_with_log(config['TrainingJobName'], + self.non_terminal_states, + self.failed_states, + wait_for_completion, + check_interval, max_ingestion_time + ) + elif wait_for_completion: + describe_response = self.check_status(config['TrainingJobName'], + 'TrainingJobStatus', + self.describe_training_job, + check_interval, max_ingestion_time + ) + + billable_time = \ + (describe_response['TrainingEndTime'] - describe_response['TrainingStartTime']) * \ + describe_response['ResourceConfig']['InstanceCount'] + self.log.info('Billable seconds:{}'.format(int(billable_time.total_seconds()) + 1)) - response = self.conn.create_training_job( - **training_job_config) - if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, - SageMakerHook.failed_states, - 'TrainingJobStatus', - self.describe_training_job, - training_job_config['TrainingJobName']) return response - def create_tuning_job(self, tuning_job_config, wait_for_completion=True): + def create_tuning_job(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): """ Create a tuning job - :param tuning_job_config: the config for tuning - :type tuning_job_config: dict + + :param config: the config for tuning + :type config: dict :param wait_for_completion: if the program should keep running until job finishes :param wait_for_completion: bool - :return: A dict that contains ARN of the tuning job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to tuning job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException( - "SageMaker connection id must be present to \ - read SageMaker tunning job configuration.") - - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) - config = sagemaker_conn.extra_dejson.copy() - tuning_job_config.update(config) + self.check_tuning_config(config) - self.check_valid_tuning_input(tuning_job_config) - - response = self.conn.create_hyper_parameter_tuning_job( - **tuning_job_config) + response = self.get_conn().create_hyper_parameter_tuning_job(**config) if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, - SageMakerHook.failed_states, + self.check_status(config['HyperParameterTuningJobName'], 'HyperParameterTuningJobStatus', self.describe_tuning_job, - tuning_job_config['HyperParameterTuningJobName']) + check_interval, max_ingestion_time + ) return response - def create_transform_job(self, transform_job_config, wait_for_completion=True): + def create_transform_job(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): """ Create a transform job - :param transform_job_config: the config for transform job - :type transform_job_config: dict - :param wait_for_completion: - if the program should keep running until job finishes + + :param config: the config for transform job + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes :type wait_for_completion: bool - :return: A dict that contains ARN of the transform job. + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to transform job creation """ - if self.use_db_config: - if not self.sagemaker_conn_id: - raise AirflowException( - "SageMaker connection id must be present to \ - read SageMaker transform job configuration.") - sagemaker_conn = self.get_connection(self.sagemaker_conn_id) + self.check_s3_url(config['TransformInput']['DataSource']['S3DataSource']['S3Uri']) - config = sagemaker_conn.extra_dejson.copy() - transform_job_config.update(config) - - self.check_for_url(transform_job_config - ['TransformInput']['DataSource'] - ['S3DataSource']['S3Uri']) - - response = self.conn.create_transform_job( - **transform_job_config) + response = self.get_conn().create_transform_job(**config) if wait_for_completion: - self.check_status(SageMakerHook.non_terminal_states, - SageMakerHook.failed_states, + self.check_status(config['TransformJobName'], 'TransformJobStatus', self.describe_transform_job, - transform_job_config['TransformJobName']) + check_interval, max_ingestion_time + ) return response - def create_model(self, model_config): + def create_model(self, config): """ Create a model job - :param model_config: the config for model - :type model_config: dict - :return: A dict that contains ARN of the model. + + :param config: the config for model + :type config: dict + :return: A response to model creation """ - return self.conn.create_model( - **model_config) + return self.get_conn().create_model(**config) + + def create_endpoint_config(self, config): + """ + Create an endpoint config - def describe_training_job(self, training_job_name): + :param config: the config for endpoint-config + :type config: dict + :return: A response to endpoint config creation """ - :param training_job_name: the name of the training job - :type training_job_name: string - Return the training job info associated with the current job_name + + return self.get_conn().create_endpoint_config(**config) + + def create_endpoint(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): + """ + Create an endpoint + + :param config: the config for endpoint + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to endpoint creation + """ + + response = self.get_conn().create_endpoint(**config) + if wait_for_completion: + self.check_status(config['EndpointName'], + 'EndpointStatus', + self.describe_endpoint, + check_interval, max_ingestion_time, + non_terminal_states=self.endpoint_non_terminal_states + ) + return response + + def update_endpoint(self, config, wait_for_completion=True, + check_interval=30, max_ingestion_time=None): + """ + Update an endpoint + + :param config: the config for endpoint + :type config: dict + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: A response to endpoint update + """ + + response = self.get_conn().update_endpoint(**config) + if wait_for_completion: + self.check_status(config['EndpointName'], + 'EndpointStatus', + self.describe_endpoint, + check_interval, max_ingestion_time, + non_terminal_states=self.endpoint_non_terminal_states + ) + return response + + def describe_training_job(self, name): + """ + Return the training job info associated with the name + + :param name: the name of the training job + :type name: str :return: A dict contains all the training job info """ - return self.conn\ - .describe_training_job(TrainingJobName=training_job_name) - def describe_tuning_job(self, tuning_job_name): + return self.get_conn().describe_training_job(TrainingJobName=name) + + def describe_training_job_with_log(self, job_name, positions, stream_names, + instance_count, state, last_description, + last_describe_job_call): + """ + Return the training job info associated with job_name and print CloudWatch logs + """ + log_group = '/aws/sagemaker/TrainingJobs' + + if len(stream_names) < instance_count: + # Log streams are created whenever a container starts writing to stdout/err, so this list + # may be dynamic until we have a stream for every instance. + logs_conn = self.get_log_conn() + try: + streams = logs_conn.describe_log_streams( + logGroupName=log_group, + logStreamNamePrefix=job_name + '/', + orderBy='LogStreamName', + limit=instance_count + ) + stream_names = [s['logStreamName'] for s in streams['logStreams']] + positions.update([(s, Position(timestamp=0, skip=0)) + for s in stream_names if s not in positions]) + except logs_conn.exceptions.ResourceNotFoundException: + # On the very first training job run on an account, there's no log group until + # the container starts logging, so ignore any errors thrown about that + pass + + if len(stream_names) > 0: + for idx, event in self.multi_stream_iter(log_group, stream_names, positions): + self.log.info(event['message']) + ts, count = positions[stream_names[idx]] + if event['timestamp'] == ts: + positions[stream_names[idx]] = Position(timestamp=ts, skip=count + 1) + else: + positions[stream_names[idx]] = Position(timestamp=event['timestamp'], skip=1) + + if state == LogState.COMPLETE: + return state, last_description, last_describe_job_call + + if state == LogState.JOB_COMPLETE: + state = LogState.COMPLETE + elif time.time() - last_describe_job_call >= 30: + description = self.describe_training_job(job_name) + last_describe_job_call = time.time() + + if secondary_training_status_changed(description, last_description): + self.log.info(secondary_training_status_message(description, last_description)) + last_description = description + + status = description['TrainingJobStatus'] + + if status not in self.non_terminal_states: + state = LogState.JOB_COMPLETE + return state, last_description, last_describe_job_call + + def describe_tuning_job(self, name): """ - :param tuning_job_name: the name of the tuning job - :type tuning_job_name: string - Return the tuning job info associated with the current job_name + Return the tuning job info associated with the name + + :param name: the name of the tuning job + :type name: string :return: A dict contains all the tuning job info """ - return self.conn\ - .describe_hyper_parameter_tuning_job( - HyperParameterTuningJobName=tuning_job_name) - def describe_transform_job(self, transform_job_name): + return self.get_conn().describe_hyper_parameter_tuning_job(HyperParameterTuningJobName=name) + + def describe_model(self, name): + """ + Return the SageMaker model info associated with the name + + :param name: the name of the SageMaker model + :type name: string + :return: A dict contains all the model info + """ + + return self.get_conn().describe_model(ModelName=name) + + def describe_transform_job(self, name): """ - :param transform_job_name: the name of the transform job - :type transform_job_name: string - Return the transform job info associated with the current job_name + Return the transform job info associated with the name + + :param name: the name of the transform job + :type name: string :return: A dict contains all the transform job info """ - return self.conn\ - .describe_transform_job( - TransformJobName=transform_job_name) + + return self.get_conn().describe_transform_job(TransformJobName=name) + + def describe_endpoint_config(self, name): + """ + Return the endpoint config info associated with the name + + :param name: the name of the endpoint config + :type name: string + :return: A dict contains all the endpoint config info + """ + + return self.get_conn().describe_endpoint_config(EndpointConfigName=name) + + def describe_endpoint(self, name): + """ + :param name: the name of the endpoint + :type name: string + :return: A dict contains all the endpoint info + """ + + return self.get_conn().describe_endpoint(EndpointName=name) + + def check_status(self, job_name, key, + describe_function, check_interval, + max_ingestion_time, + non_terminal_states=None): + """ + Check status of a SageMaker job + + :param job_name: name of the job to check status + :type job_name: str + :param key: the key of the response dict + that points to the state + :type key: str + :param describe_function: the function used to retrieve the status + :type describe_function: python callable + :param args: the arguments for the function + :param check_interval: the time interval in seconds which the operator + will check the status of any SageMaker job + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :param non_terminal_states: the set of nonterminal states + :type non_terminal_states: set + :return: response of describe call after job is done + """ + if not non_terminal_states: + non_terminal_states = self.non_terminal_states + + sec = 0 + running = True + + while running: + time.sleep(check_interval) + sec = sec + check_interval + + try: + response = describe_function(job_name) + status = response[key] + self.log.info('Job still running for %s seconds... ' + 'current status is %s' % (sec, status)) + except KeyError: + raise AirflowException('Could not get status of the SageMaker job') + except ClientError: + raise AirflowException('AWS request failed, check logs for more info') + + if status in non_terminal_states: + running = True + elif status in self.failed_states: + raise AirflowException('SageMaker job failed because %s' % response['FailureReason']) + else: + running = False + + if max_ingestion_time and sec > max_ingestion_time: + # ensure that the job gets killed if the max ingestion time is exceeded + raise AirflowException('SageMaker job took more than %s seconds', max_ingestion_time) + + self.log.info('SageMaker Job Compeleted') + response = describe_function(job_name) + return response + + def check_training_status_with_log(self, job_name, non_terminal_states, failed_states, + wait_for_completion, check_interval, max_ingestion_time): + """ + Display the logs for a given training job, optionally tailing them until the + job is complete. + + :param job_name: name of the training job to check status and display logs for + :type job_name: str + :param non_terminal_states: the set of non_terminal states + :type non_terminal_states: set + :param failed_states: the set of failed states + :type failed_states: set + :param wait_for_completion: Whether to keep looking for new log entries + until the job completes + :type wait_for_completion: bool + :param check_interval: The interval in seconds between polling for new log entries and job completion + :type check_interval: int + :param max_ingestion_time: the maximum ingestion time in seconds. Any + SageMaker jobs that run longer than this will fail. Setting this to + None implies no timeout for any SageMaker job. + :type max_ingestion_time: int + :return: None + """ + + sec = 0 + description = self.describe_training_job(job_name) + self.log.info(secondary_training_status_message(description, None)) + instance_count = description['ResourceConfig']['InstanceCount'] + status = description['TrainingJobStatus'] + + stream_names = [] # The list of log streams + positions = {} # The current position in each stream, map of stream name -> position + + job_already_completed = status not in non_terminal_states + + state = LogState.TAILING if wait_for_completion and not job_already_completed else LogState.COMPLETE + + # The loop below implements a state machine that alternates between checking the job status and + # reading whatever is available in the logs at this point. Note, that if we were called with + # wait_for_completion == False, we never check the job status. + # + # If wait_for_completion == TRUE and job is not completed, the initial state is TAILING + # If wait_for_completion == FALSE, the initial state is COMPLETE + # (doesn't matter if the job really is complete). + # + # The state table: + # + # STATE ACTIONS CONDITION NEW STATE + # ---------------- ---------------- ----------------- ---------------- + # TAILING Read logs, Pause, Get status Job complete JOB_COMPLETE + # Else TAILING + # JOB_COMPLETE Read logs, Pause Any COMPLETE + # COMPLETE Read logs, Exit N/A + # + # Notes: + # - The JOB_COMPLETE state forces us to do an extra pause and read any items that + # got to Cloudwatch after the job was marked complete. + last_describe_job_call = time.time() + last_description = description + + while True: + time.sleep(check_interval) + sec = sec + check_interval + + state, last_description, last_describe_job_call = \ + self.describe_training_job_with_log(job_name, positions, stream_names, + instance_count, state, last_description, + last_describe_job_call) + if state == LogState.COMPLETE: + break + + if max_ingestion_time and sec > max_ingestion_time: + # ensure that the job gets killed if the max ingestion time is exceeded + raise AirflowException('SageMaker job took more than %s seconds', max_ingestion_time) + + if wait_for_completion: + status = last_description['TrainingJobStatus'] + if status in failed_states: + reason = last_description.get('FailureReason', '(No reason provided)') + raise AirflowException('Error training {}: {} Reason: {}'.format(job_name, status, reason)) + billable_time = (last_description['TrainingEndTime'] - last_description['TrainingStartTime']) \ + * instance_count + self.log.info('Billable seconds:{}'.format(int(billable_time.total_seconds()) + 1)) diff --git a/airflow/contrib/operators/sagemaker_base_operator.py b/airflow/contrib/operators/sagemaker_base_operator.py new file mode 100644 index 0000000000000..cf1e59387a784 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_base_operator.py @@ -0,0 +1,100 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import json + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + + +class SageMakerBaseOperator(BaseOperator): + """ + This is the base operator for all SageMaker operators. + + :param config: The configuration necessary to start a training job (templated) + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + """ + + template_fields = ['config'] + template_ext = () + ui_color = '#ededed' + + integer_fields = [] + + @apply_defaults + def __init__(self, + config, + aws_conn_id='aws_default', + *args, **kwargs): + super(SageMakerBaseOperator, self).__init__(*args, **kwargs) + + self.aws_conn_id = aws_conn_id + self.config = config + self.hook = None + + def parse_integer(self, config, field): + if len(field) == 1: + if isinstance(config, list): + for sub_config in config: + self.parse_integer(sub_config, field) + return + head = field[0] + if head in config: + config[head] = int(config[head]) + return + + if isinstance(config, list): + for sub_config in config: + self.parse_integer(sub_config, field) + return + + head, tail = field[0], field[1:] + if head in config: + self.parse_integer(config[head], tail) + return + + def parse_config_integers(self): + # Parse the integer fields of training config to integers + # in case the config is rendered by Jinja and all fields are str + for field in self.integer_fields: + self.parse_integer(self.config, field) + + def expand_role(self): + raise NotImplementedError('Please implement expand_role() in sub class!') + + def preprocess_config(self): + self.log.info( + 'Preprocessing the config and doing required s3_operations' + ) + self.hook = SageMakerHook(aws_conn_id=self.aws_conn_id) + + self.hook.configure_s3_resources(self.config) + self.parse_config_integers() + self.expand_role() + + self.log.info( + 'After preprocessing the config is:\n {}'.format( + json.dumps(self.config, sort_keys=True, indent=4, separators=(',', ': '))) + ) + + def execute(self, context): + raise NotImplementedError('Please implement execute() in sub class!') diff --git a/airflow/contrib/operators/sagemaker_create_training_job_operator.py b/airflow/contrib/operators/sagemaker_create_training_job_operator.py deleted file mode 100644 index fdd935fc2931b..0000000000000 --- a/airflow/contrib/operators/sagemaker_create_training_job_operator.py +++ /dev/null @@ -1,119 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException - - -class SageMakerCreateTrainingJobOperator(BaseOperator): - - """ - Initiate a SageMaker training - - This operator returns The ARN of the model created in Amazon SageMaker - - :param training_job_config: - The configuration necessary to start a training job (templated) - :type training_job_config: dict - :param region_name: The AWS region_name - :type region_name: string - :param sagemaker_conn_id: The SageMaker connection ID to use. - :type sagemaker_conn_id: string - :param use_db_config: Whether or not to use db config - associated with sagemaker_conn_id. - If set to true, will automatically update the training config - with what's in db, so the db config doesn't need to - included everything, but what's there does replace the ones - in the training_job_config, so be careful - :type use_db_config: bool - :param aws_conn_id: The AWS connection ID to use. - :type aws_conn_id: string - :param wait_for_completion: if the operator should block - until training job finishes - :type wait_for_completion: bool - :param check_interval: if wait is set to be true, this is the time interval - in seconds which the operator will check the status of the training job - :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the training job hasn't finish within the max_ingestion_time - (Caution: be careful to set this parameters because training can take very long) - :type max_ingestion_time: int - - **Example**: - The following operator would start a training job when executed - - sagemaker_training = - SageMakerCreateTrainingJobOperator( - task_id='sagemaker_training', - training_job_config=config, - region_name='us-west-2' - sagemaker_conn_id='sagemaker_customers_conn', - use_db_config=True, - aws_conn_id='aws_customers_conn' - ) - """ - - template_fields = ['training_job_config'] - template_ext = () - ui_color = '#ededed' - - @apply_defaults - def __init__(self, - training_job_config=None, - region_name=None, - sagemaker_conn_id=None, - use_db_config=False, - wait_for_completion=True, - check_interval=5, - max_ingestion_time=None, - *args, **kwargs): - super(SageMakerCreateTrainingJobOperator, self).__init__(*args, **kwargs) - - self.sagemaker_conn_id = sagemaker_conn_id - self.training_job_config = training_job_config - self.use_db_config = use_db_config - self.region_name = region_name - self.wait_for_completion = wait_for_completion - self.check_interval = check_interval - self.max_ingestion_time = max_ingestion_time - - def execute(self, context): - sagemaker = SageMakerHook( - sagemaker_conn_id=self.sagemaker_conn_id, - use_db_config=self.use_db_config, - region_name=self.region_name, - check_interval=self.check_interval, - max_ingestion_time=self.max_ingestion_time - ) - - self.log.info( - "Creating SageMaker Training Job %s." - % self.training_job_config['TrainingJobName'] - ) - response = sagemaker.create_training_job( - self.training_job_config, - wait_for_completion=self.wait_for_completion) - if not response['ResponseMetadata']['HTTPStatusCode'] \ - == 200: - raise AirflowException( - 'Sagemaker Training Job creation failed: %s' % response) - else: - return response diff --git a/airflow/contrib/operators/sagemaker_create_transform_job_operator.py b/airflow/contrib/operators/sagemaker_create_transform_job_operator.py deleted file mode 100644 index 22c8c2b4ba297..0000000000000 --- a/airflow/contrib/operators/sagemaker_create_transform_job_operator.py +++ /dev/null @@ -1,132 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException - - -class SageMakerCreateTransformJobOperator(BaseOperator): - """ - Initiate a SageMaker transform - - This operator returns The ARN of the model created in Amazon SageMaker - - :param sagemaker_conn_id: The SageMaker connection ID to use. - :type sagemaker_conn_id: string - :param transform_job_config: - The configuration necessary to start a transform job (templated) - :type transform_job_config: dict - :param model_config: - The configuration necessary to create a model, the default is none - which means that user should provide a created model in transform_job_config - If given, will be used to create a model before creating transform job - :type model_config: dict - :param use_db_config: Whether or not to use db config - associated with sagemaker_conn_id. - If set to true, will automatically update the transform config - with what's in db, so the db config doesn't need to - included everything, but what's there does replace the ones - in the transform_job_config, so be careful - :type use_db_config: bool - :param region_name: The AWS region_name - :type region_name: string - :param wait_for_completion: if the program should keep running until job finishes - :type wait_for_completion: bool - :param check_interval: if wait is set to be true, this is the time interval - in seconds which the operator will check the status of the transform job - :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the transform job hasn't finish within the max_ingestion_time - (Caution: be careful to set this parameters because transform can take very long) - :type max_ingestion_time: int - :param aws_conn_id: The AWS connection ID to use. - :type aws_conn_id: string - - **Example**: - The following operator would start a transform job when executed - - sagemaker_transform = - SageMakerCreateTransformJobOperator( - task_id='sagemaker_transform', - transform_job_config=config_transform, - model_config=config_model, - region_name='us-west-2' - sagemaker_conn_id='sagemaker_customers_conn', - use_db_config=True, - aws_conn_id='aws_customers_conn' - ) - """ - - template_fields = ['transform_job_config'] - template_ext = () - ui_color = '#ededed' - - @apply_defaults - def __init__(self, - sagemaker_conn_id=None, - transform_job_config=None, - model_config=None, - use_db_config=False, - region_name=None, - wait_for_completion=True, - check_interval=2, - max_ingestion_time=None, - *args, **kwargs): - super(SageMakerCreateTransformJobOperator, self).__init__(*args, **kwargs) - - self.sagemaker_conn_id = sagemaker_conn_id - self.transform_job_config = transform_job_config - self.model_config = model_config - self.use_db_config = use_db_config - self.region_name = region_name - self.wait_for_completion = wait_for_completion - self.check_interval = check_interval - self.max_ingestion_time = max_ingestion_time - - def execute(self, context): - sagemaker = SageMakerHook( - sagemaker_conn_id=self.sagemaker_conn_id, - use_db_config=self.use_db_config, - region_name=self.region_name, - check_interval=self.check_interval, - max_ingestion_time=self.max_ingestion_time - ) - - if self.model_config: - self.log.info( - "Creating SageMaker Model %s for transform job" - % self.model_config['ModelName'] - ) - sagemaker.create_model(self.model_config) - - self.log.info( - "Creating SageMaker transform Job %s." - % self.transform_job_config['TransformJobName'] - ) - response = sagemaker.create_transform_job( - self.transform_job_config, - wait_for_completion=self.wait_for_completion) - if not response['ResponseMetadata']['HTTPStatusCode'] \ - == 200: - raise AirflowException( - 'Sagemaker transform Job creation failed: %s' % response) - else: - return response diff --git a/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py b/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py deleted file mode 100644 index 46ccb2a201144..0000000000000 --- a/airflow/contrib/operators/sagemaker_create_tuning_job_operator.py +++ /dev/null @@ -1,121 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults -from airflow.exceptions import AirflowException - - -class SageMakerCreateTuningJobOperator(BaseOperator): - - """ - Initiate a SageMaker HyperParameter Tuning Job - - This operator returns The ARN of the model created in Amazon SageMaker - - :param sagemaker_conn_id: The SageMaker connection ID to use. - :type sagemaker_conn_id: string - :param region_name: The AWS region_name - :type region_name: string - :param tuning_job_config: - The configuration necessary to start a tuning job (templated) - :type tuning_job_config: dict - :param use_db_config: Whether or not to use db config - associated with sagemaker_conn_id. - If set to true, will automatically update the tuning config - with what's in db, so the db config doesn't need to - included everything, but what's there does replace the ones - in the tuning_job_config, so be careful - :type use_db_config: bool - :param wait_for_completion: if the operator should block - until tuning job finishes - :type wait_for_completion: bool - :param check_interval: if wait is set to be true, this is the time interval - in seconds which the operator will check the status of the tuning job - :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the tuning job hasn't finish within the max_ingestion_time - (Caution: be careful to set this parameters because tuning can take very long) - :type max_ingestion_time: int - :param aws_conn_id: The AWS connection ID to use. - :type aws_conn_id: string - - **Example**: - The following operator would start a tuning job when executed - - sagemaker_tuning = - SageMakerCreateTuningJobOperator( - task_id='sagemaker_tuning', - sagemaker_conn_id='sagemaker_customers_conn', - tuning_job_config=config, - check_interval=2, - max_ingestion_time=3600, - aws_conn_id='aws_customers_conn', - ) - """ - - template_fields = ['tuning_job_config'] - template_ext = () - ui_color = '#ededed' - - @apply_defaults - def __init__(self, - sagemaker_conn_id=None, - region_name=None, - tuning_job_config=None, - use_db_config=False, - wait_for_completion=True, - check_interval=5, - max_ingestion_time=None, - *args, **kwargs): - super(SageMakerCreateTuningJobOperator, self)\ - .__init__(*args, **kwargs) - - self.sagemaker_conn_id = sagemaker_conn_id - self.region_name = region_name - self.tuning_job_config = tuning_job_config - self.use_db_config = use_db_config - self.wait_for_completion = wait_for_completion - self.check_interval = check_interval - self.max_ingestion_time = max_ingestion_time - - def execute(self, context): - sagemaker = SageMakerHook(sagemaker_conn_id=self.sagemaker_conn_id, - region_name=self.region_name, - use_db_config=self.use_db_config, - check_interval=self.check_interval, - max_ingestion_time=self.max_ingestion_time - ) - - self.log.info( - "Creating SageMaker Hyper Parameter Tunning Job %s" - % self.tuning_job_config['HyperParameterTuningJobName'] - ) - - response = sagemaker.create_tuning_job( - self.tuning_job_config, - wait_for_completion=self.wait_for_completion - ) - if not response['ResponseMetadata']['HTTPStatusCode'] \ - == 200: - raise AirflowException( - "Sagemaker Tuning Job creation failed: %s" % response) - else: - return response diff --git a/airflow/contrib/operators/sagemaker_training_operator.py b/airflow/contrib/operators/sagemaker_training_operator.py new file mode 100644 index 0000000000000..69036925f34f7 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_training_operator.py @@ -0,0 +1,97 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerTrainingOperator(SageMakerBaseOperator): + """ + Initiate a SageMaker training job. + + This operator returns The ARN of the training job created in Amazon SageMaker. + + :param config: The configuration necessary to start a training job (templated) + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + :param wait_for_completion: if the operator should block until training job finishes + :type wait_for_completion: bool + :param print_log: if the operator should print the cloudwatch log during training + :type print_log: bool + :param check_interval: if wait is set to be true, this is the time interval + in seconds which the operator will check the status of the training job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the training job hasn't finish within the max_ingestion_time in seconds + (Caution: be careful to set this parameters because training can take very long) + Setting it to None implies no timeout. + :type max_ingestion_time: int + """ + + integer_fields = [ + ['ResourceConfig', 'InstanceCount'], + ['ResourceConfig', 'VolumeSizeInGB'], + ['StoppingCondition', 'MaxRuntimeInSeconds'] + ] + + @apply_defaults + def __init__(self, + config, + wait_for_completion=True, + print_log=True, + check_interval=30, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerTrainingOperator, self).__init__(config=config, + *args, **kwargs) + + self.wait_for_completion = wait_for_completion + self.print_log = print_log + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + + def expand_role(self): + if 'RoleArn' in self.config: + hook = AwsHook(self.aws_conn_id) + self.config['RoleArn'] = hook.expand_role(self.config['RoleArn']) + + def execute(self, context): + self.preprocess_config() + + self.log.info('Creating SageMaker Training Job %s.', self.config['TrainingJobName']) + + response = self.hook.create_training_job( + self.config, + wait_for_completion=self.wait_for_completion, + print_log=self.print_log, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException( + 'Sagemaker Training Job creation failed: %s' % response) + else: + return { + 'Training': self.hook.describe_training_job( + self.config['TrainingJobName'] + ) + } diff --git a/airflow/contrib/operators/sagemaker_transform_operator.py b/airflow/contrib/operators/sagemaker_transform_operator.py new file mode 100644 index 0000000000000..7be570cdacd6f --- /dev/null +++ b/airflow/contrib/operators/sagemaker_transform_operator.py @@ -0,0 +1,112 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerTransformOperator(SageMakerBaseOperator): + """ + Initiate a SageMaker transform job. + + This operator returns The ARN of the model created in Amazon SageMaker. + + :param config: The configuration necessary to start a transform job (templated) + :type config: dict + :param model_config: + The configuration necessary to create a SageMaker model, the default is none + which means the SageMaker model used for the SageMaker transform job already exists. + If given, it will be used to create a SageMaker model before creating + the SageMaker transform job + :type model_config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: string + :param wait_for_completion: if the program should keep running until job finishes + :type wait_for_completion: bool + :param check_interval: if wait is set to be true, this is the time interval + in seconds which the operator will check the status of the transform job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the transform job hasn't finish within the max_ingestion_time in seconds + (Caution: be careful to set this parameters because transform can take very long) + :type max_ingestion_time: int + """ + + @apply_defaults + def __init__(self, + config, + wait_for_completion=True, + check_interval=30, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerTransformOperator, self).__init__(config=config, + *args, **kwargs) + self.config = config + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + self.create_integer_fields() + + def create_integer_fields(self): + self.integer_fields = [ + ['Transform', 'TransformResources', 'InstanceCount'], + ['Transform', 'MaxConcurrentTransforms'], + ['Transform', 'MaxPayloadInMB'] + ] + if 'Transform' not in self.config: + for field in self.integer_fields: + field.pop(0) + + def expand_role(self): + if 'Model' not in self.config: + return + config = self.config['Model'] + if 'ExecutionRoleArn' in config: + hook = AwsHook(self.aws_conn_id) + config['ExecutionRoleArn'] = hook.expand_role(config['ExecutionRoleArn']) + + def execute(self, context): + self.preprocess_config() + + model_config = self.config.get('Model') + transform_config = self.config.get('Transform', self.config) + + if model_config: + self.log.info('Creating SageMaker Model %s for transform job', model_config['ModelName']) + self.hook.create_model(model_config) + + self.log.info('Creating SageMaker transform Job %s.', transform_config['TransformJobName']) + response = self.hook.create_transform_job( + transform_config, + wait_for_completion=self.wait_for_completion, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException('Sagemaker transform Job creation failed: %s' % response) + else: + return { + 'Model': self.hook.describe_model( + transform_config['ModelName'] + ), + 'Transform': self.hook.describe_transform_job( + transform_config['TransformJobName'] + ) + } diff --git a/airflow/contrib/operators/sagemaker_tuning_operator.py b/airflow/contrib/operators/sagemaker_tuning_operator.py new file mode 100644 index 0000000000000..94c995072a8da --- /dev/null +++ b/airflow/contrib/operators/sagemaker_tuning_operator.py @@ -0,0 +1,97 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerTuningOperator(SageMakerBaseOperator): + """ + Initiate a SageMaker hyper-parameter tuning job. + + This operator returns The ARN of the tuning job created in Amazon SageMaker. + + :param config: The configuration necessary to start a tuning job (templated) + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + :param wait_for_completion: if the operator should block until tuning job finishes + :type wait_for_completion: bool + :param check_interval: if wait is set to be true, this is the time interval + in seconds which the operator will check the status of the tuning job + :type check_interval: int + :param max_ingestion_time: if wait is set to be true, the operator will fail + if the tuning job hasn't finish within the max_ingestion_time in seconds + (Caution: be careful to set this parameters because tuning can take very long) + :type max_ingestion_time: int + """ + + integer_fields = [ + ['HyperParameterTuningJobConfig', 'ResourceLimits', 'MaxNumberOfTrainingJobs'], + ['HyperParameterTuningJobConfig', 'ResourceLimits', 'MaxParallelTrainingJobs'], + ['TrainingJobDefinition', 'ResourceConfig', 'InstanceCount'], + ['TrainingJobDefinition', 'ResourceConfig', 'VolumeSizeInGB'], + ['TrainingJobDefinition', 'StoppingCondition', 'MaxRuntimeInSeconds'] + ] + + @apply_defaults + def __init__(self, + config, + wait_for_completion=True, + check_interval=30, + max_ingestion_time=None, + *args, **kwargs): + super(SageMakerTuningOperator, self).__init__(config=config, + *args, **kwargs) + self.config = config + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + + def expand_role(self): + if 'TrainingJobDefinition' in self.config: + config = self.config['TrainingJobDefinition'] + if 'RoleArn' in config: + hook = AwsHook(self.aws_conn_id) + config['RoleArn'] = hook.expand_role(config['RoleArn']) + + def execute(self, context): + self.preprocess_config() + + self.log.info( + 'Creating SageMaker Hyper-Parameter Tuning Job %s', self.config['HyperParameterTuningJobName'] + ) + + response = self.hook.create_tuning_job( + self.config, + wait_for_completion=self.wait_for_completion, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException( + 'Sagemaker Tuning Job creation failed: %s' % response) + else: + return { + 'Tuning': self.hook.describe_tuning_job( + self.config['HyperParameterTuningJobName'] + ) + } diff --git a/airflow/contrib/sensors/sagemaker_base_sensor.py b/airflow/contrib/sensors/sagemaker_base_sensor.py index 149c2a1aab124..10dd6b2357a66 100644 --- a/airflow/contrib/sensors/sagemaker_base_sensor.py +++ b/airflow/contrib/sensors/sagemaker_base_sensor.py @@ -28,7 +28,7 @@ class SageMakerBaseSensor(BaseSensorOperator): and state_from_response() methods. Subclasses should also implement NON_TERMINAL_STATES and FAILED_STATE methods. """ - ui_color = '#66c3ff' + ui_color = '#ededed' @apply_defaults def __init__( @@ -54,23 +54,21 @@ def poke(self, context): if state in self.failed_states(): failed_reason = self.get_failed_reason_from_response(response) - raise AirflowException("Sagemaker job failed for the following reason: %s" + raise AirflowException('Sagemaker job failed for the following reason: %s' % failed_reason) return True def non_terminal_states(self): - raise AirflowException("Non Terminal States need to be specified in subclass") + raise NotImplementedError('Please implement non_terminal_states() in subclass') def failed_states(self): - raise AirflowException("Failed States need to be specified in subclass") + raise NotImplementedError('Please implement failed_states() in subclass') def get_sagemaker_response(self): - raise AirflowException( - "Method get_sagemaker_response()not implemented.") + raise NotImplementedError('Please implement get_sagemaker_response() in subclass') def get_failed_reason_from_response(self, response): return 'Unknown' def state_from_response(self, response): - raise AirflowException( - "Method state_from_response()not implemented.") + raise NotImplementedError('Please implement state_from_response() in subclass') diff --git a/airflow/contrib/sensors/sagemaker_training_sensor.py b/airflow/contrib/sensors/sagemaker_training_sensor.py index 449de44c0819c..2d820111a08c8 100644 --- a/airflow/contrib/sensors/sagemaker_training_sensor.py +++ b/airflow/contrib/sensors/sagemaker_training_sensor.py @@ -17,7 +17,9 @@ # specific language governing permissions and limitations # under the License. -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +import time + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook, LogState from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor from airflow.utils.decorators import apply_defaults @@ -27,8 +29,10 @@ class SageMakerTrainingSensor(SageMakerBaseSensor): Asks for the state of the training state until it reaches a terminal state. If it fails the sensor errors, failing the task. - :param job_name: job_name of the training instance to check the state of - :type job_name: string + :param job_name: name of the SageMaker training job to check the state of + :type job_name: str + :param print_log: if the operator should print the cloudwatch log + :type print_log: bool """ template_fields = ['job_name'] @@ -37,12 +41,30 @@ class SageMakerTrainingSensor(SageMakerBaseSensor): @apply_defaults def __init__(self, job_name, - region_name=None, + print_log=True, *args, **kwargs): super(SageMakerTrainingSensor, self).__init__(*args, **kwargs) self.job_name = job_name - self.region_name = region_name + self.print_log = print_log + self.positions = {} + self.stream_names = [] + self.instance_count = None + self.state = None + self.last_description = None + self.last_describe_job_call = None + self.log_resource_inited = False + + def init_log_resource(self, hook): + description = hook.describe_training_job(self.job_name) + self.instance_count = description['ResourceConfig']['InstanceCount'] + + status = description['TrainingJobStatus'] + job_already_completed = status not in self.non_terminal_states() + self.state = LogState.TAILING if not job_already_completed else LogState.COMPLETE + self.last_description = description + self.last_describe_job_call = time.time() + self.log_resource_inited = True def non_terminal_states(self): return SageMakerHook.non_terminal_states @@ -51,13 +73,27 @@ def failed_states(self): return SageMakerHook.failed_states def get_sagemaker_response(self): - sagemaker = SageMakerHook( - aws_conn_id=self.aws_conn_id, - region_name=self.region_name - ) + sagemaker_hook = SageMakerHook(aws_conn_id=self.aws_conn_id) + if self.print_log: + if not self.log_resource_inited: + self.init_log_resource(sagemaker_hook) + self.state, self.last_description, self.last_describe_job_call = \ + sagemaker_hook.describe_training_job_with_log(self.job_name, + self.positions, self.stream_names, + self.instance_count, self.state, + self.last_description, + self.last_describe_job_call) + else: + self.last_description = sagemaker_hook.describe_training_job(self.job_name) + + status = self.state_from_response(self.last_description) + if status not in self.non_terminal_states() and status not in self.failed_states(): + billable_time = \ + (self.last_description['TrainingEndTime'] - self.last_description['TrainingStartTime']) * \ + self.last_description['ResourceConfig']['InstanceCount'] + self.log.info('Billable seconds:{}'.format(int(billable_time.total_seconds()) + 1)) - self.log.info('Poking Sagemaker Training Job %s', self.job_name) - return sagemaker.describe_training_job(self.job_name) + return self.last_description def get_failed_reason_from_response(self, response): return response['FailureReason'] diff --git a/airflow/contrib/sensors/sagemaker_transform_sensor.py b/airflow/contrib/sensors/sagemaker_transform_sensor.py index 68ef1d8dd7b05..f64724bde9b24 100644 --- a/airflow/contrib/sensors/sagemaker_transform_sensor.py +++ b/airflow/contrib/sensors/sagemaker_transform_sensor.py @@ -30,8 +30,6 @@ class SageMakerTransformSensor(SageMakerBaseSensor): :param job_name: job_name of the transform job instance to check the state of :type job_name: string - :param region_name: The AWS region_name - :type region_name: string """ template_fields = ['job_name'] @@ -40,12 +38,10 @@ class SageMakerTransformSensor(SageMakerBaseSensor): @apply_defaults def __init__(self, job_name, - region_name=None, *args, **kwargs): super(SageMakerTransformSensor, self).__init__(*args, **kwargs) self.job_name = job_name - self.region_name = region_name def non_terminal_states(self): return SageMakerHook.non_terminal_states @@ -54,10 +50,7 @@ def failed_states(self): return SageMakerHook.failed_states def get_sagemaker_response(self): - sagemaker = SageMakerHook( - aws_conn_id=self.aws_conn_id, - region_name=self.region_name - ) + sagemaker = SageMakerHook(aws_conn_id=self.aws_conn_id) self.log.info('Poking Sagemaker Transform Job %s', self.job_name) return sagemaker.describe_transform_job(self.job_name) diff --git a/airflow/contrib/sensors/sagemaker_tuning_sensor.py b/airflow/contrib/sensors/sagemaker_tuning_sensor.py index 1f081100e2c69..8c835216d6bb3 100644 --- a/airflow/contrib/sensors/sagemaker_tuning_sensor.py +++ b/airflow/contrib/sensors/sagemaker_tuning_sensor.py @@ -29,9 +29,7 @@ class SageMakerTuningSensor(SageMakerBaseSensor): containing the failure reason. :param job_name: job_name of the tuning instance to check the state of - :type job_name: string - :param region_name: The AWS region_name - :type region_name: string + :type job_name: str """ template_fields = ['job_name'] @@ -40,12 +38,10 @@ class SageMakerTuningSensor(SageMakerBaseSensor): @apply_defaults def __init__(self, job_name, - region_name=None, *args, **kwargs): super(SageMakerTuningSensor, self).__init__(*args, **kwargs) self.job_name = job_name - self.region_name = region_name def non_terminal_states(self): return SageMakerHook.non_terminal_states @@ -54,10 +50,7 @@ def failed_states(self): return SageMakerHook.failed_states def get_sagemaker_response(self): - sagemaker = SageMakerHook( - aws_conn_id=self.aws_conn_id, - region_name=self.region_name - ) + sagemaker = SageMakerHook(aws_conn_id=self.aws_conn_id) self.log.info('Poking Sagemaker Tuning Job %s', self.job_name) return sagemaker.describe_tuning_job(self.job_name) diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py index b4f3ac3b030e8..a64c331cdd09e 100644 --- a/airflow/hooks/S3_hook.py +++ b/airflow/hooks/S3_hook.py @@ -67,6 +67,26 @@ def get_bucket(self, bucket_name): s3 = self.get_resource_type('s3') return s3.Bucket(bucket_name) + def create_bucket(self, bucket_name, region_name=None): + """ + Creates a boto3.S3.Bucket object + + :param bucket_name: the name of the bucket + :type bucket_name: str + :param region__name: the name of the aws region + :type region_name: str + """ + s3_conn = self.get_conn() + if not region_name: + region_name = s3_conn.meta.region_name + if region_name == 'us-east-1': + self.get_conn().create_bucket(Bucket=bucket_name) + else: + self.get_conn().create_bucket(Bucket=bucket_name, + CreateBucketConfiguration={ + 'LocationConstraint': region_name + }) + def check_for_prefix(self, bucket_name, prefix, delimiter): """ Checks that a prefix exists in a bucket @@ -377,3 +397,124 @@ def load_bytes(self, client = self.get_conn() client.upload_fileobj(filelike_buffer, bucket_name, key, ExtraArgs=extra_args) + + def load_file_obj(self, + file_obj, + key, + bucket_name=None, + replace=False, + encrypt=False): + """ + Loads file object to S3 + + :param file_obj: file-like object to set as content for the key. + :type file_obj: file-like object + :param key: S3 key that will point to the file + :type key: str + :param bucket_name: Name of the bucket in which to store the file + :type bucket_name: str + :param replace: A flag to decide whether or not to overwrite the key + if it already exists + :type replace: bool + :param encrypt: If True, the file will be encrypted on the server-side + by S3 and will be stored in an encrypted form while at rest in S3. + :type encrypt: bool + """ + if not bucket_name: + (bucket_name, key) = self.parse_s3_url(key) + + if not replace and self.check_for_key(key, bucket_name): + raise ValueError("The key {key} already exists.".format(key=key)) + + extra_args = {} + if encrypt: + extra_args['ServerSideEncryption'] = "AES256" + + client = self.get_conn() + client.upload_fileobj(file_obj, bucket_name, key, ExtraArgs=extra_args) + + def copy_object(self, + source_bucket_key, + dest_bucket_key, + source_bucket_name=None, + dest_bucket_name=None, + source_version_id=None): + """ + Creates a copy of an object that is already stored in S3. + + Note: the S3 connection used here needs to have access to both + source and destination bucket/key. + + :param source_bucket_key: The key of the source object. + + It can be either full s3:// style url or relative path from root level. + + When it's specified as a full s3:// url, please omit source_bucket_name. + :type source_bucket_key: str + :param dest_bucket_key: The key of the object to copy to. + + The convention to specify `dest_bucket_key` is the same + as `source_bucket_key`. + :type dest_bucket_key: str + :param source_bucket_name: Name of the S3 bucket where the source object is in. + + It should be omitted when `source_bucket_key` is provided as a full s3:// url. + :type source_bucket_name: str + :param dest_bucket_name: Name of the S3 bucket to where the object is copied. + + It should be omitted when `dest_bucket_key` is provided as a full s3:// url. + :type dest_bucket_name: str + :param source_version_id: Version ID of the source object (OPTIONAL) + :type source_version_id: str + """ + + if dest_bucket_name is None: + dest_bucket_name, dest_bucket_key = self.parse_s3_url(dest_bucket_key) + else: + parsed_url = urlparse(dest_bucket_key) + if parsed_url.scheme != '' or parsed_url.netloc != '': + raise AirflowException('If dest_bucket_name is provided, ' + + 'dest_bucket_key should be relative path ' + + 'from root level, rather than a full s3:// url') + + if source_bucket_name is None: + source_bucket_name, source_bucket_key = self.parse_s3_url(source_bucket_key) + else: + parsed_url = urlparse(source_bucket_key) + if parsed_url.scheme != '' or parsed_url.netloc != '': + raise AirflowException('If source_bucket_name is provided, ' + + 'source_bucket_key should be relative path ' + + 'from root level, rather than a full s3:// url') + + CopySource = {'Bucket': source_bucket_name, + 'Key': source_bucket_key, + 'VersionId': source_version_id} + response = self.get_conn().copy_object(Bucket=dest_bucket_name, + Key=dest_bucket_key, + CopySource=CopySource) + return response + + def delete_objects(self, + bucket, + keys): + """ + :param bucket: Name of the bucket in which you are going to delete object(s) + :type bucket: str + :param keys: The key(s) to delete from S3 bucket. + + When ``keys`` is a string, it's supposed to be the key name of + the single object to delete. + + When ``keys`` is a list, it's supposed to be the list of the + keys to delete. + :type keys: str or list + """ + if isinstance(keys, list): + keys = keys + else: + keys = [keys] + + delete_dict = {"Objects": [{"Key": k} for k in keys]} + response = self.get_conn().delete_objects(Bucket=bucket, + Delete=delete_dict) + return response diff --git a/docs/code.rst b/docs/code.rst index f4a91e00ba24e..d577b768eb79c 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -182,6 +182,10 @@ Operators .. autoclass:: airflow.contrib.operators.qubole_operator.QuboleOperator .. autoclass:: airflow.contrib.operators.s3_list_operator.S3ListOperator .. autoclass:: airflow.contrib.operators.s3_to_gcs_operator.S3ToGoogleCloudStorageOperator +.. autoclass:: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator +.. autoclass:: airflow.contrib.operators.sagemaker_training_operator.SageMakerTrainingOperator +.. autoclass:: airflow.contrib.operators.sagemaker_transform_operator.SageMakerTransformOperator +.. autoclass:: airflow.contrib.operators.sagemaker_tuning_operator.SagemakerTuningOperator .. autoclass:: airflow.contrib.operators.segment_track_event_operator.SegmentTrackEventOperator .. autoclass:: airflow.contrib.operators.sftp_operator.SFTPOperator .. autoclass:: airflow.contrib.operators.slack_webhook_operator.SlackWebhookOperator @@ -218,6 +222,10 @@ Sensors .. autoclass:: airflow.contrib.sensors.pubsub_sensor.PubSubPullSensor .. autoclass:: airflow.contrib.sensors.qubole_sensor.QuboleSensor .. autoclass:: airflow.contrib.sensors.redis_key_sensor.RedisKeySensor +.. autoclass:: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor +.. autoclass:: airflow.contrib.sensors.sagemaker_training_sensor.SageMakerTrainingSensor +.. autoclass:: airflow.contrib.sensors.sagemaker_transform_sensor.SageMakerTransformSensor +.. autoclass:: airflow.contrib.sensors.sagemaker_tuning_sensor.SageMakerTuningSensor .. autoclass:: airflow.contrib.sensors.sftp_sensor.SFTPSensor .. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbBlobSensor @@ -396,6 +404,7 @@ Community contributed hooks .. autoclass:: airflow.contrib.hooks.qubole_hook.QuboleHook .. autoclass:: airflow.contrib.hooks.redis_hook.RedisHook .. autoclass:: airflow.contrib.hooks.redshift_hook.RedshiftHook +.. autoclass:: airflow.contrib.hooks.sagemaker_hook.SageMakerHook .. autoclass:: airflow.contrib.hooks.salesforce_hook.SalesforceHook .. autoclass:: airflow.contrib.hooks.segment_hook.SegmentHook .. autoclass:: airflow.contrib.hooks.sftp_hook.SFTPHook diff --git a/tests/contrib/hooks/test_aws_hook.py b/tests/contrib/hooks/test_aws_hook.py index b92e594c10369..57814b43d9038 100644 --- a/tests/contrib/hooks/test_aws_hook.py +++ b/tests/contrib/hooks/test_aws_hook.py @@ -35,11 +35,12 @@ mock = None try: - from moto import mock_emr, mock_dynamodb2, mock_sts + from moto import mock_emr, mock_dynamodb2, mock_sts, mock_iam except ImportError: mock_emr = None mock_dynamodb2 = None mock_sts = None + mock_iam = None class TestAwsHook(unittest.TestCase): @@ -185,6 +186,35 @@ def test_get_credentials_from_role_arn(self, mock_get_connection): 'gRmpRV3zrkuWJOgQs8IZZaIv2BXIa2R4OlgkBN9bkUDNCJiBeb/AXlzBBko7b15' 'fjrBs2+cTQtpZ3CYWFXG8C5zqx37wnOE49mRl/+OtkIKGO7fAE') + @unittest.skipIf(mock_sts is None, 'mock_sts package not present') + @mock.patch.object(AwsHook, 'get_connection') + @mock_sts + def test_get_credentials_from_role_arn_with_external_id(self, mock_get_connection): + mock_connection = Connection( + extra='{"role_arn":"arn:aws:iam::123456:role/role_arn",' + ' "external_id":"external_id"}') + mock_get_connection.return_value = mock_connection + hook = AwsHook() + credentials_from_hook = hook.get_credentials() + self.assertEqual(credentials_from_hook.access_key, 'AKIAIOSFODNN7EXAMPLE') + self.assertEqual(credentials_from_hook.secret_key, + 'aJalrXUtnFEMI/K7MDENG/bPxRfiCYzEXAMPLEKEY') + self.assertEqual(credentials_from_hook.token, + 'BQoEXAMPLEH4aoAH0gNCAPyJxz4BlCFFxWNE1OPTgk5TthT+FvwqnKwRcOIfrRh' + '3c/LTo6UDdyJwOOvEVPvLXCrrrUtdnniCEXAMPLE/IvU1dYUg2RVAJBanLiHb4I' + 'gRmpRV3zrkuWJOgQs8IZZaIv2BXIa2R4OlgkBN9bkUDNCJiBeb/AXlzBBko7b15' + 'fjrBs2+cTQtpZ3CYWFXG8C5zqx37wnOE49mRl/+OtkIKGO7fAE') + + @unittest.skipIf(mock_iam is None, 'mock_iam package not present') + @mock_iam + def test_expand_role(self): + conn = boto3.client('iam', region_name='us-east-1') + conn.create_role(RoleName='test-role', AssumeRolePolicyDocument='some policy') + hook = AwsHook() + arn = hook.expand_role('test-role') + expect_arn = conn.get_role(RoleName='test-role').get('Role').get('Arn') + self.assertEqual(arn, expect_arn) + if __name__ == '__main__': unittest.main() diff --git a/tests/contrib/hooks/test_sagemaker_hook.py b/tests/contrib/hooks/test_sagemaker_hook.py index 3a863b3cb0dc7..bec00bf601a8f 100644 --- a/tests/contrib/hooks/test_sagemaker_hook.py +++ b/tests/contrib/hooks/test_sagemaker_hook.py @@ -18,10 +18,11 @@ # under the License. # - -import json import unittest -import copy +import time +from datetime import datetime +from tzlocal import get_localzone + try: from unittest import mock except ImportError: @@ -31,200 +32,221 @@ mock = None from airflow import configuration -from airflow import models -from airflow.utils import db -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.hooks.sagemaker_hook import (SageMakerHook, secondary_training_status_changed, + secondary_training_status_message, LogState) from airflow.hooks.S3_hook import S3Hook from airflow.exceptions import AirflowException -role = 'test-role' +role = 'arn:aws:iam:role/test-role' +path = 'local/data' bucket = 'test-bucket' - key = 'test/data' data_url = 's3://{}/{}'.format(bucket, key) -job_name = 'test-job-name' - -model_name = 'test-model-name' +job_name = 'test-job' +model_name = 'test-model' +config_name = 'test-endpoint-config' +endpoint_name = 'test-endpoint' image = 'test-image' +test_arn_return = {'Arn': 'testarn'} +output_url = 's3://{}/test/output'.format(bucket) -test_arn_return = {'TrainingJobArn': 'testarn'} - -test_list_training_job_return = { - 'TrainingJobSummaries': [ - { - 'TrainingJobName': job_name, - 'TrainingJobStatus': 'InProgress' - }, - ], - 'NextToken': 'test-token' -} - -test_list_tuning_job_return = { - 'TrainingJobSummaries': [ +create_training_params = { + 'AlgorithmSpecification': { + 'TrainingImage': image, + 'TrainingInputMode': 'File' + }, + 'RoleArn': role, + 'OutputDataConfig': { + 'S3OutputPath': output_url + }, + 'ResourceConfig': { + 'InstanceCount': 2, + 'InstanceType': 'ml.c4.8xlarge', + 'VolumeSizeInGB': 50 + }, + 'TrainingJobName': job_name, + 'HyperParameters': { + 'k': '10', + 'feature_dim': '784', + 'mini_batch_size': '500', + 'force_dense': 'True' + }, + 'StoppingCondition': { + 'MaxRuntimeInSeconds': 60 * 60 + }, + 'InputDataConfig': [ { - 'TrainingJobName': job_name, - 'TrainingJobArn': 'testarn', - 'TunedHyperParameters': { - 'k': '3' + 'ChannelName': 'train', + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url, + 'S3DataDistributionType': 'FullyReplicated' + } }, - 'TrainingJobStatus': 'InProgress' - }, - ], - 'NextToken': 'test-token' + 'CompressionType': 'None', + 'RecordWrapperType': 'None' + } + ] } -output_url = 's3://{}/test/output'.format(bucket) -create_training_params = \ - { - 'AlgorithmSpecification': { - 'TrainingImage': image, - 'TrainingInputMode': 'File' +create_tuning_params = { + 'HyperParameterTuningJobName': job_name, + 'HyperParameterTuningJobConfig': { + 'Strategy': 'Bayesian', + 'HyperParameterTuningJobObjective': { + 'Type': 'Maximize', + 'MetricName': 'test_metric' }, - 'RoleArn': role, - 'OutputDataConfig': { - 'S3OutputPath': output_url + 'ResourceLimits': { + 'MaxNumberOfTrainingJobs': 123, + 'MaxParallelTrainingJobs': 123 }, - 'ResourceConfig': { - 'InstanceCount': 2, - 'InstanceType': 'ml.c4.8xlarge', - 'VolumeSizeInGB': 50 - }, - 'TrainingJobName': job_name, - 'HyperParameters': { - 'k': '10', - 'feature_dim': '784', - 'mini_batch_size': '500', - 'force_dense': 'True' - }, - 'StoppingCondition': { - 'MaxRuntimeInSeconds': 60 * 60 - }, - 'InputDataConfig': [ - { - 'ChannelName': 'train', - 'DataSource': { - 'S3DataSource': { - 'S3DataType': 'S3Prefix', - 'S3Uri': data_url, - 'S3DataDistributionType': 'FullyReplicated' - } + 'ParameterRanges': { + 'IntegerParameterRanges': [ + { + 'Name': 'k', + 'MinValue': '2', + 'MaxValue': '10' }, - 'CompressionType': 'None', - 'RecordWrapperType': 'None' - } - ] - } -create_tuning_params = \ - { - 'HyperParameterTuningJobName': job_name, - 'HyperParameterTuningJobConfig': { - 'Strategy': 'Bayesian', - 'HyperParameterTuningJobObjective': { - 'Type': 'Maximize', - 'MetricName': 'test_metric' - }, - 'ResourceLimits': { - 'MaxNumberOfTrainingJobs': 123, - 'MaxParallelTrainingJobs': 123 - }, - 'ParameterRanges': { - 'IntegerParameterRanges': [ - { - 'Name': 'k', - 'MinValue': '2', - 'MaxValue': '10' - }, - - ] - } - }, - 'TrainingJobDefinition': { - 'StaticHyperParameters': create_training_params['HyperParameters'], - 'AlgorithmSpecification': create_training_params['AlgorithmSpecification'], - 'RoleArn': 'string', - 'InputDataConfig': create_training_params['InputDataConfig'], - 'OutputDataConfig': create_training_params['OutputDataConfig'], - 'ResourceConfig': create_training_params['ResourceConfig'], - 'StoppingCondition': dict(MaxRuntimeInSeconds=60 * 60) + ] } + }, + 'TrainingJobDefinition': { + 'StaticHyperParameters': create_training_params['HyperParameters'], + 'AlgorithmSpecification': create_training_params['AlgorithmSpecification'], + 'RoleArn': 'string', + 'InputDataConfig': create_training_params['InputDataConfig'], + 'OutputDataConfig': create_training_params['OutputDataConfig'], + 'ResourceConfig': create_training_params['ResourceConfig'], + 'StoppingCondition': dict(MaxRuntimeInSeconds=60 * 60) } +} -create_transform_params = \ - { - 'TransformJobName': job_name, - 'ModelName': model_name, - 'BatchStrategy': 'MultiRecord', - 'TransformInput': { - 'DataSource': { - 'S3DataSource': { - 'S3DataType': 'S3Prefix', - 'S3Uri': data_url - } +create_transform_params = { + 'TransformJobName': job_name, + 'ModelName': model_name, + 'BatchStrategy': 'MultiRecord', + 'TransformInput': { + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url } - }, - 'TransformOutput': { - 'S3OutputPath': output_url, - }, - 'TransformResources': { - 'InstanceType': 'ml.m4.xlarge', - 'InstanceCount': 123 } + }, + 'TransformOutput': { + 'S3OutputPath': output_url, + }, + 'TransformResources': { + 'InstanceType': 'ml.m4.xlarge', + 'InstanceCount': 123 } +} -create_model_params = \ - { - 'ModelName': model_name, - 'PrimaryContainer': { - 'Image': image, - 'ModelDataUrl': output_url, - }, - 'ExecutionRoleArn': role - } +create_model_params = { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role +} -db_config = { - 'Tags': [ +create_endpoint_config_params = { + 'EndpointConfigName': config_name, + 'ProductionVariants': [ { - 'Key': 'test-db-key', - 'Value': 'test-db-value', - - }, + 'VariantName': 'AllTraffic', + 'ModelName': model_name, + 'InitialInstanceCount': 1, + 'InstanceType': 'ml.c4.xlarge' + } ] } -DESCRIBE_TRAINING_INPROGRESS_RETURN = { - 'TrainingJobStatus': 'InProgress', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } +create_endpoint_params = { + 'EndpointName': endpoint_name, + 'EndpointConfigName': config_name } + +update_endpoint_params = create_endpoint_params + DESCRIBE_TRAINING_COMPELETED_RETURN = { - 'TrainingJobStatus': 'Compeleted', + 'TrainingJobStatus': 'Completed', + 'ResourceConfig': { + 'InstanceCount': 1, + 'InstanceType': 'ml.c4.xlarge', + 'VolumeSizeInGB': 10 + }, + 'TrainingStartTime': datetime(2018, 2, 17, 7, 15, 0, 103000), + 'TrainingEndTime': datetime(2018, 2, 17, 7, 19, 34, 953000), 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TRAINING_FAILED_RETURN = { - 'TrainingJobStatus': 'Failed', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - }, - 'FailureReason': 'Unknown' + +DESCRIBE_TRAINING_INPROGRESS_RETURN = dict(DESCRIBE_TRAINING_COMPELETED_RETURN) +DESCRIBE_TRAINING_INPROGRESS_RETURN.update({'TrainingJobStatus': 'InProgress'}) + +DESCRIBE_TRAINING_FAILED_RETURN = dict(DESCRIBE_TRAINING_COMPELETED_RETURN) +DESCRIBE_TRAINING_FAILED_RETURN.update({'TrainingJobStatus': 'Failed', + 'FailureReason': 'Unknown'}) + +DESCRIBE_TRAINING_STOPPING_RETURN = dict(DESCRIBE_TRAINING_COMPELETED_RETURN) +DESCRIBE_TRAINING_STOPPING_RETURN.update({'TrainingJobStatus': 'Stopping'}) + +message = 'message' +status = 'status' +SECONDARY_STATUS_DESCRIPTION_1 = { + 'SecondaryStatusTransitions': [{'StatusMessage': message, 'Status': status}] } -DESCRIBE_TRAINING_STOPPING_RETURN = { - 'TrainingJobStatus': 'Stopping', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } +SECONDARY_STATUS_DESCRIPTION_2 = { + 'SecondaryStatusTransitions': [{'StatusMessage': 'different message', 'Status': status}] } -DESCRIBE_TRAINING_STOPPED_RETURN = { - 'TrainingJobStatus': 'Stopped', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, + +DEFAULT_LOG_STREAMS = {'logStreams': [{'logStreamName': job_name + '/xxxxxxxxx'}]} +LIFECYCLE_LOG_STREAMS = [DEFAULT_LOG_STREAMS, + DEFAULT_LOG_STREAMS, + DEFAULT_LOG_STREAMS, + DEFAULT_LOG_STREAMS, + DEFAULT_LOG_STREAMS, + DEFAULT_LOG_STREAMS] + +DEFAULT_LOG_EVENTS = [{'nextForwardToken': None, 'events': [{'timestamp': 1, 'message': 'hi there #1'}]}, + {'nextForwardToken': None, 'events': []}] +STREAM_LOG_EVENTS = [{'nextForwardToken': None, 'events': [{'timestamp': 1, 'message': 'hi there #1'}]}, + {'nextForwardToken': None, 'events': []}, + {'nextForwardToken': None, 'events': [{'timestamp': 1, 'message': 'hi there #1'}, + {'timestamp': 2, 'message': 'hi there #2'}]}, + {'nextForwardToken': None, 'events': []}, + {'nextForwardToken': None, 'events': [{'timestamp': 2, 'message': 'hi there #2'}, + {'timestamp': 2, 'message': 'hi there #2a'}, + {'timestamp': 3, 'message': 'hi there #3'}]}, + {'nextForwardToken': None, 'events': []}] + +test_evaluation_config = { + 'Image': image, + 'Role': role, + 'S3Operations': { + 'S3CreateBucket': [ + { + 'Bucket': bucket + } + ], + 'S3Upload': [ + { + 'Path': path, + 'Bucket': bucket, + 'Key': key, + 'Tar': False + } + ] } } @@ -233,94 +255,63 @@ class TestSageMakerHook(unittest.TestCase): def setUp(self): configuration.load_test_config() - db.merge_conn( - models.Connection( - conn_id='sagemaker_test_conn_id', - conn_type='sagemaker', - login='access_id', - password='access_key', - extra=json.dumps(db_config) - ) - ) + + @mock.patch.object(S3Hook, 'create_bucket') + @mock.patch.object(S3Hook, 'load_file') + def test_configure_s3_resources(self, mock_load_file, mock_create_bucket): + hook = SageMakerHook() + evaluation_result = { + 'Image': image, + 'Role': role + } + hook.configure_s3_resources(test_evaluation_config) + self.assertEqual(test_evaluation_config, evaluation_result) + mock_create_bucket.assert_called_once_with(bucket_name=bucket) + mock_load_file.assert_called_once_with(path, key, bucket) @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(S3Hook, 'check_for_key') @mock.patch.object(S3Hook, 'check_for_bucket') @mock.patch.object(S3Hook, 'check_for_prefix') - def test_check_for_url(self, - mock_check_prefix, - mock_check_bucket, - mock_check_key, - mock_client): + def test_check_s3_url(self, + mock_check_prefix, + mock_check_bucket, + mock_check_key, + mock_client): mock_client.return_value = None hook = SageMakerHook() mock_check_bucket.side_effect = [False, True, True, True] mock_check_key.side_effect = [False, True, False] mock_check_prefix.side_effect = [False, True, True] self.assertRaises(AirflowException, - hook.check_for_url, data_url) + hook.check_s3_url, data_url) self.assertRaises(AirflowException, - hook.check_for_url, data_url) - self.assertEqual(hook.check_for_url(data_url), True) - self.assertEqual(hook.check_for_url(data_url), True) + hook.check_s3_url, data_url) + self.assertEqual(hook.check_s3_url(data_url), True) + self.assertEqual(hook.check_s3_url(data_url), True) @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'check_for_url') + @mock.patch.object(SageMakerHook, 'check_s3_url') def test_check_valid_training(self, mock_check_url, mock_client): mock_client.return_value = None hook = SageMakerHook() - hook.check_valid_training_input(create_training_params) + hook.check_training_config(create_training_params) mock_check_url.assert_called_once_with(data_url) @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'check_for_url') + @mock.patch.object(SageMakerHook, 'check_s3_url') def test_check_valid_tuning(self, mock_check_url, mock_client): mock_client.return_value = None hook = SageMakerHook() - hook.check_valid_tuning_input(create_tuning_params) + hook.check_tuning_config(create_tuning_params) mock_check_url.assert_called_once_with(data_url) @mock.patch.object(SageMakerHook, 'get_client_type') def test_conn(self, mock_get_client): - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', - region_name='us-east-1' - ) - self.assertEqual(hook.sagemaker_conn_id, 'sagemaker_test_conn_id') - mock_get_client.assert_called_once_with('sagemaker', - region_name='us-east-1' - ) + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + self.assertEqual(hook.aws_conn_id, 'sagemaker_test_conn_id') - @mock.patch.object(SageMakerHook, 'get_conn') - def test_list_training_job(self, mock_client): - mock_session = mock.Mock() - attrs = {'list_training_jobs.return_value': - test_list_training_job_return} - mock_session.configure_mock(**attrs) - mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') - response = hook.list_training_job(name_contains=job_name, - status_equals='InProgress') - mock_session.list_training_jobs. \ - assert_called_once_with(NameContains=job_name, - StatusEquals='InProgress') - self.assertEqual(response, test_list_training_job_return) - - @mock.patch.object(SageMakerHook, 'get_conn') - def test_list_tuning_job(self, mock_client): - mock_session = mock.Mock() - attrs = {'list_hyper_parameter_tuning_job.return_value': - test_list_tuning_job_return} - mock_session.configure_mock(**attrs) - mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') - response = hook.list_tuning_job(name_contains=job_name, - status_equals='InProgress') - mock_session.list_hyper_parameter_tuning_job. \ - assert_called_once_with(NameContains=job_name, - StatusEquals='InProgress') - self.assertEqual(response, test_list_tuning_job_return) - - @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'check_training_config') @mock.patch.object(SageMakerHook, 'get_conn') def test_create_training_job(self, mock_client, mock_check_training): mock_check_training.return_value = True @@ -329,71 +320,56 @@ def test_create_training_job(self, mock_client, mock_check_training): test_arn_return} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.create_training_job(create_training_params, - wait_for_completion=False) + wait_for_completion=False, + print_log=False) mock_session.create_training_job.assert_called_once_with(**create_training_params) self.assertEqual(response, test_arn_return) - @mock.patch.object(SageMakerHook, 'check_valid_training_input') - @mock.patch.object(SageMakerHook, 'get_conn') - def test_create_training_job_db_config(self, mock_client, mock_check_training): - mock_check_training.return_value = True - mock_session = mock.Mock() - attrs = {'create_training_job.return_value': - test_arn_return} - mock_session.configure_mock(**attrs) - mock_client.return_value = mock_session - hook_use_db_config = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', - use_db_config=True) - response = hook_use_db_config.create_training_job(create_training_params, - wait_for_completion=False) - updated_config = copy.deepcopy(create_training_params) - updated_config.update(db_config) - mock_session.create_training_job.assert_called_once_with(**updated_config) - self.assertEqual(response, test_arn_return) - - @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'check_training_config') @mock.patch.object(SageMakerHook, 'get_conn') - def test_training_ends_with_wait_on(self, mock_client, mock_check_training): + def test_training_ends_with_wait(self, mock_client, mock_check_training): mock_check_training.return_value = True mock_session = mock.Mock() attrs = {'create_training_job.return_value': test_arn_return, 'describe_training_job.side_effect': - [DESCRIBE_TRAINING_INPROGRESS_RETURN, - DESCRIBE_TRAINING_STOPPING_RETURN, - DESCRIBE_TRAINING_STOPPED_RETURN, - DESCRIBE_TRAINING_COMPELETED_RETURN] + [DESCRIBE_TRAINING_INPROGRESS_RETURN, + DESCRIBE_TRAINING_STOPPING_RETURN, + DESCRIBE_TRAINING_COMPELETED_RETURN, + DESCRIBE_TRAINING_COMPELETED_RETURN] } mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id_1') - hook.create_training_job(create_training_params, wait_for_completion=True) + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id_1') + hook.create_training_job(create_training_params, wait_for_completion=True, + print_log=False, check_interval=1) self.assertEqual(mock_session.describe_training_job.call_count, 4) - @mock.patch.object(SageMakerHook, 'check_valid_training_input') + @mock.patch.object(SageMakerHook, 'check_training_config') @mock.patch.object(SageMakerHook, 'get_conn') - def test_training_throws_error_when_failed_with_wait_on( + def test_training_throws_error_when_failed_with_wait( self, mock_client, mock_check_training): mock_check_training.return_value = True mock_session = mock.Mock() attrs = {'create_training_job.return_value': test_arn_return, 'describe_training_job.side_effect': - [DESCRIBE_TRAINING_INPROGRESS_RETURN, - DESCRIBE_TRAINING_STOPPING_RETURN, - DESCRIBE_TRAINING_STOPPED_RETURN, - DESCRIBE_TRAINING_FAILED_RETURN] + [DESCRIBE_TRAINING_INPROGRESS_RETURN, + DESCRIBE_TRAINING_STOPPING_RETURN, + DESCRIBE_TRAINING_FAILED_RETURN, + DESCRIBE_TRAINING_COMPELETED_RETURN] } mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id_1') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id_1') self.assertRaises(AirflowException, hook.create_training_job, - create_training_params, wait_for_completion=True) - self.assertEqual(mock_session.describe_training_job.call_count, 4) + create_training_params, wait_for_completion=True, + print_log=False, check_interval=1) + self.assertEqual(mock_session.describe_training_job.call_count, 3) - @mock.patch.object(SageMakerHook, 'check_valid_tuning_input') + @mock.patch.object(SageMakerHook, 'check_tuning_config') @mock.patch.object(SageMakerHook, 'get_conn') def test_create_tuning_job(self, mock_client, mock_check_tuning): mock_session = mock.Mock() @@ -401,33 +377,14 @@ def test_create_tuning_job(self, mock_client, mock_check_tuning): test_arn_return} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.create_tuning_job(create_tuning_params, wait_for_completion=False) mock_session.create_hyper_parameter_tuning_job.\ assert_called_once_with(**create_tuning_params) self.assertEqual(response, test_arn_return) - @mock.patch.object(SageMakerHook, 'check_valid_tuning_input') - @mock.patch.object(SageMakerHook, 'get_conn') - def test_create_tuning_job_db_config(self, mock_client, mock_check_tuning): - mock_check_tuning.return_value = True - mock_session = mock.Mock() - attrs = {'create_hyper_parameter_tuning_job.return_value': - test_arn_return} - mock_session.configure_mock(**attrs) - mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', - use_db_config=True) - response = hook.create_tuning_job(create_tuning_params, - wait_for_completion=False) - updated_config = copy.deepcopy(create_tuning_params) - updated_config.update(db_config) - mock_session.create_hyper_parameter_tuning_job. \ - assert_called_once_with(**updated_config) - self.assertEqual(response, test_arn_return) - - @mock.patch.object(SageMakerHook, 'check_for_url') + @mock.patch.object(SageMakerHook, 'check_s3_url') @mock.patch.object(SageMakerHook, 'get_conn') def test_create_transform_job(self, mock_client, mock_check_url): mock_check_url.return_value = True @@ -436,41 +393,64 @@ def test_create_transform_job(self, mock_client, mock_check_url): test_arn_return} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.create_transform_job(create_transform_params, wait_for_completion=False) mock_session.create_transform_job.assert_called_once_with( **create_transform_params) self.assertEqual(response, test_arn_return) - @mock.patch.object(SageMakerHook, 'check_for_url') @mock.patch.object(SageMakerHook, 'get_conn') - def test_create_transform_job_db_config(self, mock_client, mock_check_url): - mock_check_url.return_value = True + def test_create_model(self, mock_client): mock_session = mock.Mock() - attrs = {'create_transform_job.return_value': + attrs = {'create_model.return_value': test_arn_return} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook_use_db_config = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id', - use_db_config=True) - response = hook_use_db_config.create_transform_job( - create_transform_params, wait_for_completion=False) - updated_config = copy.deepcopy(create_transform_params) - updated_config.update(db_config) - mock_session.create_transform_job.assert_called_once_with(**updated_config) + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.create_model(create_model_params) + mock_session.create_model.assert_called_once_with(**create_model_params) self.assertEqual(response, test_arn_return) @mock.patch.object(SageMakerHook, 'get_conn') - def test_create_model(self, mock_client): + def test_create_endpoint_config(self, mock_client): mock_session = mock.Mock() - attrs = {'create_model.return_value': + attrs = {'create_endpoint_config.return_value': test_arn_return} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') - response = hook.create_model(create_model_params) - mock_session.create_model.assert_called_once_with(**create_model_params) + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.create_endpoint_config(create_endpoint_config_params) + mock_session.create_endpoint_config\ + .assert_called_once_with(**create_endpoint_config_params) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_create_endpoint(self, mock_client): + mock_session = mock.Mock() + attrs = {'create_endpoint.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.create_endpoint(create_endpoint_params, + wait_for_completion=False) + mock_session.create_endpoint\ + .assert_called_once_with(**create_endpoint_params) + self.assertEqual(response, test_arn_return) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_update_endpoint(self, mock_client): + mock_session = mock.Mock() + attrs = {'update_endpoint.return_value': + test_arn_return} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.update_endpoint(update_endpoint_params, + wait_for_completion=False) + mock_session.update_endpoint\ + .assert_called_once_with(**update_endpoint_params) self.assertEqual(response, test_arn_return) @mock.patch.object(SageMakerHook, 'get_conn') @@ -479,7 +459,7 @@ def test_describe_training_job(self, mock_client): attrs = {'describe_training_job.return_value': 'InProgress'} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.describe_training_job(job_name) mock_session.describe_training_job.\ assert_called_once_with(TrainingJobName=job_name) @@ -492,7 +472,7 @@ def test_describe_tuning_job(self, mock_client): 'InProgress'} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.describe_tuning_job(job_name) mock_session.describe_hyper_parameter_tuning_job.\ assert_called_once_with(HyperParameterTuningJobName=job_name) @@ -505,12 +485,188 @@ def test_describe_transform_job(self, mock_client): 'InProgress'} mock_session.configure_mock(**attrs) mock_client.return_value = mock_session - hook = SageMakerHook(sagemaker_conn_id='sagemaker_test_conn_id') + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') response = hook.describe_transform_job(job_name) mock_session.describe_transform_job.\ assert_called_once_with(TransformJobName=job_name) self.assertEqual(response, 'InProgress') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_model(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_model.return_value': + model_name} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_model(model_name) + mock_session.describe_model.\ + assert_called_once_with(ModelName=model_name) + self.assertEqual(response, model_name) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_endpoint_config(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_endpoint_config.return_value': + config_name} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_endpoint_config(config_name) + mock_session.describe_endpoint_config.\ + assert_called_once_with(EndpointConfigName=config_name) + self.assertEqual(response, config_name) + + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_endpoint(self, mock_client): + mock_session = mock.Mock() + attrs = {'describe_endpoint.return_value': + 'InProgress'} + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_endpoint(endpoint_name) + mock_session.describe_endpoint.\ + assert_called_once_with(EndpointName=endpoint_name) + self.assertEqual(response, 'InProgress') + + def test_secondary_training_status_changed_true(self): + changed = secondary_training_status_changed(SECONDARY_STATUS_DESCRIPTION_1, + SECONDARY_STATUS_DESCRIPTION_2) + self.assertTrue(changed) + + def test_secondary_training_status_changed_false(self): + changed = secondary_training_status_changed(SECONDARY_STATUS_DESCRIPTION_1, + SECONDARY_STATUS_DESCRIPTION_1) + self.assertFalse(changed) + + def test_secondary_training_status_message_status_changed(self): + now = datetime.now(get_localzone()) + SECONDARY_STATUS_DESCRIPTION_1['LastModifiedTime'] = now + expected = '{} {} - {}'.format( + datetime.utcfromtimestamp(time.mktime(now.timetuple())).strftime('%Y-%m-%d %H:%M:%S'), + status, + message + ) + self.assertEqual( + secondary_training_status_message(SECONDARY_STATUS_DESCRIPTION_1, SECONDARY_STATUS_DESCRIPTION_2), + expected) + + @mock.patch.object(SageMakerHook, 'get_log_conn') + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(time, 'time') + def test_describe_training_job_with_logs_in_progress(self, mock_time, mock_client, mock_log_client): + mock_session = mock.Mock() + mock_log_session = mock.Mock() + attrs = {'describe_training_job.return_value': + DESCRIBE_TRAINING_COMPELETED_RETURN + } + log_attrs = {'describe_log_streams.side_effect': + LIFECYCLE_LOG_STREAMS, + 'get_log_events.side_effect': + STREAM_LOG_EVENTS + } + mock_time.return_value = 50 + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + mock_log_session.configure_mock(**log_attrs) + mock_log_client.return_value = mock_log_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_training_job_with_log(job_name=job_name, + positions={}, + stream_names=[], + instance_count=1, + state=LogState.WAIT_IN_PROGRESS, + last_description={}, + last_describe_job_call=0) + self.assertEqual(response, (LogState.JOB_COMPLETE, {}, 50)) + + @mock.patch.object(SageMakerHook, 'get_log_conn') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_training_job_with_logs_job_complete(self, mock_client, mock_log_client): + mock_session = mock.Mock() + mock_log_session = mock.Mock() + attrs = {'describe_training_job.return_value': + DESCRIBE_TRAINING_COMPELETED_RETURN + } + log_attrs = {'describe_log_streams.side_effect': + LIFECYCLE_LOG_STREAMS, + 'get_log_events.side_effect': + STREAM_LOG_EVENTS + } + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + mock_log_session.configure_mock(**log_attrs) + mock_log_client.return_value = mock_log_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_training_job_with_log(job_name=job_name, + positions={}, + stream_names=[], + instance_count=1, + state=LogState.JOB_COMPLETE, + last_description={}, + last_describe_job_call=0) + self.assertEqual(response, (LogState.COMPLETE, {}, 0)) + + @mock.patch.object(SageMakerHook, 'get_log_conn') + @mock.patch.object(SageMakerHook, 'get_conn') + def test_describe_training_job_with_logs_complete(self, mock_client, mock_log_client): + mock_session = mock.Mock() + mock_log_session = mock.Mock() + attrs = {'describe_training_job.return_value': + DESCRIBE_TRAINING_COMPELETED_RETURN + } + log_attrs = {'describe_log_streams.side_effect': + LIFECYCLE_LOG_STREAMS, + 'get_log_events.side_effect': + STREAM_LOG_EVENTS + } + mock_session.configure_mock(**attrs) + mock_client.return_value = mock_session + mock_log_session.configure_mock(**log_attrs) + mock_log_client.return_value = mock_log_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id') + response = hook.describe_training_job_with_log(job_name=job_name, + positions={}, + stream_names=[], + instance_count=1, + state=LogState.COMPLETE, + last_description={}, + last_describe_job_call=0) + self.assertEqual(response, (LogState.COMPLETE, {}, 0)) + + @mock.patch.object(SageMakerHook, 'check_training_config') + @mock.patch.object(SageMakerHook, 'get_log_conn') + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'describe_training_job_with_log') + def test_training_with_logs(self, mock_describe, mock_client, mock_log_client, mock_check_training): + mock_check_training.return_value = True + mock_describe.side_effect = \ + [(LogState.WAIT_IN_PROGRESS, DESCRIBE_TRAINING_INPROGRESS_RETURN, 0), + (LogState.JOB_COMPLETE, DESCRIBE_TRAINING_STOPPING_RETURN, 0), + (LogState.COMPLETE, DESCRIBE_TRAINING_COMPELETED_RETURN, 0)] + mock_session = mock.Mock() + mock_log_session = mock.Mock() + attrs = {'create_training_job.return_value': + test_arn_return, + 'describe_training_job.return_value': + DESCRIBE_TRAINING_COMPELETED_RETURN + } + log_attrs = {'describe_log_streams.side_effect': + LIFECYCLE_LOG_STREAMS, + 'get_log_events.side_effect': + STREAM_LOG_EVENTS + } + mock_session.configure_mock(**attrs) + mock_log_session.configure_mock(**log_attrs) + mock_client.return_value = mock_session + mock_log_client.return_value = mock_log_session + hook = SageMakerHook(aws_conn_id='sagemaker_test_conn_id_1') + hook.create_training_job(create_training_params, wait_for_completion=True, + print_log=True, check_interval=1) + self.assertEqual(mock_describe.call_count, 3) + self.assertEqual(mock_session.describe_training_job.call_count, 1) + if __name__ == '__main__': unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_base_operator.py b/tests/contrib/operators/test_sagemaker_base_operator.py new file mode 100644 index 0000000000000..996e61e20f2d1 --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_base_operator.py @@ -0,0 +1,84 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# 'License'); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# 'AS IS' BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator + +config = { + 'key1': '1', + 'key2': { + 'key3': '3', + 'key4': '4' + }, + 'key5': [ + { + 'key6': '6' + }, + { + 'key6': '7' + } + ] +} + +parsed_config = { + 'key1': 1, + 'key2': { + 'key3': 3, + 'key4': 4 + }, + 'key5': [ + { + 'key6': 6 + }, + { + 'key6': 7 + } + ] +} + + +class TestSageMakerBaseOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerBaseOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=config + ) + + def test_parse_integer(self): + self.sagemaker.integer_fields = [ + ['key1'], ['key2', 'key3'], ['key2', 'key4'], ['key5', 'key6'] + ] + self.sagemaker.parse_config_integers() + self.assertEqual(self.sagemaker.config, parsed_config) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_create_training_job_operator.py b/tests/contrib/operators/test_sagemaker_create_training_job_operator.py deleted file mode 100644 index 156c9d74c79ec..0000000000000 --- a/tests/contrib/operators/test_sagemaker_create_training_job_operator.py +++ /dev/null @@ -1,141 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -import unittest -try: - from unittest import mock -except ImportError: - try: - import mock - except ImportError: - mock = None - -from airflow import configuration -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_create_training_job_operator \ - import SageMakerCreateTrainingJobOperator -from airflow.exceptions import AirflowException - -role = "test-role" - -bucket = "test-bucket" - -key = "test/data" -data_url = "s3://{}/{}".format(bucket, key) - -job_name = "test-job-name" - -image = "test-image" - -output_url = "s3://{}/test/output".format(bucket) -create_training_params = \ - { - "AlgorithmSpecification": { - "TrainingImage": image, - "TrainingInputMode": "File" - }, - "RoleArn": role, - "OutputDataConfig": { - "S3OutputPath": output_url - }, - "ResourceConfig": { - "InstanceCount": 2, - "InstanceType": "ml.c4.8xlarge", - "VolumeSizeInGB": 50 - }, - "TrainingJobName": job_name, - "HyperParameters": { - "k": "10", - "feature_dim": "784", - "mini_batch_size": "500", - "force_dense": "True" - }, - "StoppingCondition": { - "MaxRuntimeInSeconds": 60 * 60 - }, - "InputDataConfig": [ - { - "ChannelName": "train", - "DataSource": { - "S3DataSource": { - "S3DataType": "S3Prefix", - "S3Uri": data_url, - "S3DataDistributionType": "FullyReplicated" - } - }, - "CompressionType": "None", - "RecordWrapperType": "None" - } - ] - } - - -class TestSageMakerTrainingOperator(unittest.TestCase): - - def setUp(self): - configuration.load_test_config() - self.sagemaker = SageMakerCreateTrainingJobOperator( - task_id='test_sagemaker_operator', - sagemaker_conn_id='sagemaker_test_id', - training_job_config=create_training_params, - region_name='us-west-2', - use_db_config=True, - wait_for_completion=False, - check_interval=5 - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_training_job') - @mock.patch.object(SageMakerHook, '__init__') - def test_hook_init(self, hook_init, mock_training, mock_client): - mock_training.return_value = {"TrainingJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 200}} - hook_init.return_value = None - self.sagemaker.execute(None) - hook_init.assert_called_once_with( - sagemaker_conn_id='sagemaker_test_id', - region_name='us-west-2', - use_db_config=True, - check_interval=5, - max_ingestion_time=None - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_training_job') - def test_execute_without_failure(self, mock_training, mock_client): - mock_training.return_value = {"TrainingJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 200}} - self.sagemaker.execute(None) - mock_training.assert_called_once_with(create_training_params, - wait_for_completion=False - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_training_job') - def test_execute_with_failure(self, mock_training, mock_client): - mock_training.return_value = {"TrainingJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 404}} - self.assertRaises(AirflowException, self.sagemaker.execute, None) - - -if __name__ == '__main__': - unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_training_operator.py b/tests/contrib/operators/test_sagemaker_training_operator.py new file mode 100644 index 0000000000000..147b7f1bb7fac --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_training_operator.py @@ -0,0 +1,134 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# 'License'); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# 'AS IS' BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_training_operator \ + import SageMakerTrainingOperator +from airflow.exceptions import AirflowException + +role = 'arn:aws:iam:role/test-role' + +bucket = 'test-bucket' + +key = 'test/data' +data_url = 's3://{}/{}'.format(bucket, key) + +job_name = 'test-job-name' + +image = 'test-image' + +output_url = 's3://{}/test/output'.format(bucket) +create_training_params = \ + { + 'AlgorithmSpecification': { + 'TrainingImage': image, + 'TrainingInputMode': 'File' + }, + 'RoleArn': role, + 'OutputDataConfig': { + 'S3OutputPath': output_url + }, + 'ResourceConfig': { + 'InstanceCount': '2', + 'InstanceType': 'ml.c4.8xlarge', + 'VolumeSizeInGB': '50' + }, + 'TrainingJobName': job_name, + 'HyperParameters': { + 'k': '10', + 'feature_dim': '784', + 'mini_batch_size': '500', + 'force_dense': 'True' + }, + 'StoppingCondition': { + 'MaxRuntimeInSeconds': '3600' + }, + 'InputDataConfig': [ + { + 'ChannelName': 'train', + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url, + 'S3DataDistributionType': 'FullyReplicated' + } + }, + 'CompressionType': 'None', + 'RecordWrapperType': 'None' + } + ] + } + + +class TestSageMakerTrainingOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerTrainingOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=create_training_params, + wait_for_completion=False, + check_interval=5 + ) + + def test_parse_config_integers(self): + self.sagemaker.parse_config_integers() + self.assertEqual(self.sagemaker.config['ResourceConfig']['InstanceCount'], + int(self.sagemaker.config['ResourceConfig']['InstanceCount'])) + self.assertEqual(self.sagemaker.config['ResourceConfig']['VolumeSizeInGB'], + int(self.sagemaker.config['ResourceConfig']['VolumeSizeInGB'])) + self.assertEqual(self.sagemaker.config['StoppingCondition']['MaxRuntimeInSeconds'], + int(self.sagemaker.config['StoppingCondition']['MaxRuntimeInSeconds'])) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_training_job') + def test_execute(self, mock_training, mock_client): + mock_training.return_value = {'TrainingJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + self.sagemaker.execute(None) + mock_training.assert_called_once_with(create_training_params, + wait_for_completion=False, + print_log=True, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_training_job') + def test_execute_with_failure(self, mock_training, mock_client): + mock_training.return_value = {'TrainingJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_transform_operator.py b/tests/contrib/operators/test_sagemaker_transform_operator.py new file mode 100644 index 0000000000000..e6bc4272b1cb0 --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_transform_operator.py @@ -0,0 +1,136 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_transform_operator \ + import SageMakerTransformOperator +from airflow.exceptions import AirflowException + +role = 'arn:aws:iam:role/test-role' + +bucket = 'test-bucket' + +key = 'test/data' +data_url = 's3://{}/{}'.format(bucket, key) + +job_name = 'test-job-name' + +model_name = 'test-model-name' + +image = 'test-image' + +output_url = 's3://{}/test/output'.format(bucket) + +create_transform_params = { + 'TransformJobName': job_name, + 'ModelName': model_name, + 'MaxConcurrentTransforms': '12', + 'MaxPayloadInMB': '6', + 'BatchStrategy': 'MultiRecord', + 'TransformInput': { + 'DataSource': { + 'S3DataSource': { + 'S3DataType': 'S3Prefix', + 'S3Uri': data_url + } + } + }, + 'TransformOutput': { + 'S3OutputPath': output_url, + }, + 'TransformResources': { + 'InstanceType': 'ml.m4.xlarge', + 'InstanceCount': '3' + } +} + +create_model_params = { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role +} + +config = { + 'Model': create_model_params, + 'Transform': create_transform_params +} + + +class TestSageMakerTransformOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerTransformOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=config, + wait_for_completion=False, + check_interval=5 + ) + + def test_parse_config_integers(self): + self.sagemaker.parse_config_integers() + test_config = self.sagemaker.config['Transform'] + self.assertEqual(test_config['TransformResources']['InstanceCount'], + int(test_config['TransformResources']['InstanceCount'])) + self.assertEqual(test_config['MaxConcurrentTransforms'], + int(test_config['MaxConcurrentTransforms'])) + self.assertEqual(test_config['MaxPayloadInMB'], + int(test_config['MaxPayloadInMB'])) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_transform_job') + def test_execute(self, mock_transform, mock_model, mock_client): + mock_transform.return_value = {'TransformJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + self.sagemaker.execute(None) + mock_model.assert_called_once_with(create_model_params) + mock_transform.assert_called_once_with(create_transform_params, + wait_for_completion=False, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_transform_job') + def test_execute_with_failure(self, mock_transform, mock_model, mock_client): + mock_transform.return_value = {'TransformJobArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py b/tests/contrib/operators/test_sagemaker_tuning_operator.py similarity index 71% rename from tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py rename to tests/contrib/operators/test_sagemaker_tuning_operator.py index d317cff6f2289..9ec533fa869b4 100644 --- a/tests/contrib/operators/test_sagemaker_create_tuning_job_operator.py +++ b/tests/contrib/operators/test_sagemaker_tuning_operator.py @@ -28,11 +28,11 @@ from airflow import configuration from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_create_tuning_job_operator \ - import SageMakerCreateTuningJobOperator +from airflow.contrib.operators.sagemaker_tuning_operator \ + import SageMakerTuningOperator from airflow.exceptions import AirflowException -role = 'test-role' +role = 'arn:aws:iam:role/test-role' bucket = 'test-bucket' @@ -53,8 +53,8 @@ 'MetricName': 'test_metric' }, 'ResourceLimits': { - 'MaxNumberOfTrainingJobs': 123, - 'MaxParallelTrainingJobs': 123 + 'MaxNumberOfTrainingJobs': '123', + 'MaxParallelTrainingJobs': '123' }, 'ParameterRanges': { 'IntegerParameterRanges': [ @@ -79,7 +79,7 @@ 'TrainingImage': image, 'TrainingInputMode': 'File' }, - 'RoleArn': 'string', + 'RoleArn': role, 'InputDataConfig': [ { @@ -102,55 +102,58 @@ }, 'ResourceConfig': { - 'InstanceCount': 2, + 'InstanceCount': '2', 'InstanceType': 'ml.c4.8xlarge', - 'VolumeSizeInGB': 50 + 'VolumeSizeInGB': '50' }, 'StoppingCondition': dict(MaxRuntimeInSeconds=60 * 60) } } -class TestSageMakerTrainingOperator(unittest.TestCase): +class TestSageMakerTuningOperator(unittest.TestCase): def setUp(self): configuration.load_test_config() - self.sagemaker = SageMakerCreateTuningJobOperator( + self.sagemaker = SageMakerTuningOperator( task_id='test_sagemaker_operator', - sagemaker_conn_id='sagemaker_test_conn', - tuning_job_config=create_tuning_params, - region_name='us-east-1', - use_db_config=False, + aws_conn_id='sagemaker_test_conn', + config=create_tuning_params, wait_for_completion=False, check_interval=5 ) - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_tuning_job') - @mock.patch.object(SageMakerHook, '__init__') - def test_hook_init(self, hook_init, mock_tuning, mock_client): - mock_tuning.return_value = {'TrainingJobArn': 'testarn', - 'ResponseMetadata': - {'HTTPStatusCode': 200}} - hook_init.return_value = None - self.sagemaker.execute(None) - hook_init.assert_called_once_with( - sagemaker_conn_id='sagemaker_test_conn', - region_name='us-east-1', - use_db_config=False, - check_interval=5, - max_ingestion_time=None - ) + def test_parse_config_integers(self): + self.sagemaker.parse_config_integers() + self.assertEqual(self.sagemaker.config['TrainingJobDefinition']['ResourceConfig'] + ['InstanceCount'], + int(self.sagemaker.config['TrainingJobDefinition']['ResourceConfig'] + ['InstanceCount'])) + self.assertEqual(self.sagemaker.config['TrainingJobDefinition']['ResourceConfig'] + ['VolumeSizeInGB'], + int(self.sagemaker.config['TrainingJobDefinition']['ResourceConfig'] + ['VolumeSizeInGB'])) + self.assertEqual(self.sagemaker.config['HyperParameterTuningJobConfig']['ResourceLimits'] + ['MaxNumberOfTrainingJobs'], + int(self.sagemaker.config['HyperParameterTuningJobConfig']['ResourceLimits'] + ['MaxNumberOfTrainingJobs'])) + self.assertEqual(self.sagemaker.config['HyperParameterTuningJobConfig']['ResourceLimits'] + ['MaxParallelTrainingJobs'], + int(self.sagemaker.config['HyperParameterTuningJobConfig']['ResourceLimits'] + ['MaxParallelTrainingJobs'])) @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, 'create_tuning_job') - def test_execute_without_failure(self, mock_tuning, mock_client): + def test_execute(self, mock_tuning, mock_client): mock_tuning.return_value = {'TrainingJobArn': 'testarn', 'ResponseMetadata': {'HTTPStatusCode': 200}} self.sagemaker.execute(None) mock_tuning.assert_called_once_with(create_tuning_params, - wait_for_completion=False) + wait_for_completion=False, + check_interval=5, + max_ingestion_time=None + ) @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, 'create_tuning_job') diff --git a/tests/contrib/sensors/test_sagemaker_base_sensor.py b/tests/contrib/sensors/test_sagemaker_base_sensor.py index bc8cbe349858f..5870544838033 100644 --- a/tests/contrib/sensors/test_sagemaker_base_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_base_sensor.py @@ -28,7 +28,7 @@ class TestSagemakerBaseSensor(unittest.TestCase): def setUp(self): configuration.load_test_config() - def test_subclasses_succeed_when_response_is_good(self): + def test_execute(self): class SageMakerBaseSensorSubclass(SageMakerBaseSensor): def non_terminal_states(self): return ['PENDING', 'RUNNING', 'CONTINUE'] @@ -53,7 +53,7 @@ def state_from_response(self, response): sensor.execute(None) - def test_poke_returns_false_when_state_is_a_non_terminal_state(self): + def test_poke_with_unfinished_job(self): class SageMakerBaseSensorSubclass(SageMakerBaseSensor): def non_terminal_states(self): return ['PENDING', 'RUNNING', 'CONTINUE'] @@ -78,7 +78,7 @@ def state_from_response(self, response): self.assertEqual(sensor.poke(None), False) - def test_poke_raise_exception_when_method_not_implemented(self): + def test_poke_with_not_implemented_method(self): class SageMakerBaseSensorSubclass(SageMakerBaseSensor): def non_terminal_states(self): return ['PENDING', 'RUNNING', 'CONTINUE'] @@ -92,9 +92,9 @@ def failed_states(self): aws_conn_id='aws_test' ) - self.assertRaises(AirflowException, sensor.poke, None) + self.assertRaises(NotImplementedError, sensor.poke, None) - def test_poke_returns_false_when_http_response_is_bad(self): + def test_poke_with_bad_response(self): class SageMakerBaseSensorSubclass(SageMakerBaseSensor): def non_terminal_states(self): return ['PENDING', 'RUNNING', 'CONTINUE'] @@ -119,7 +119,7 @@ def state_from_response(self, response): self.assertEqual(sensor.poke(None), False) - def test_poke_raises_error_when_job_has_failed(self): + def test_poke_with_job_failure(self): class SageMakerBaseSensorSubclass(SageMakerBaseSensor): def non_terminal_states(self): return ['PENDING', 'RUNNING', 'CONTINUE'] diff --git a/tests/contrib/sensors/test_sagemaker_training_sensor.py b/tests/contrib/sensors/test_sagemaker_training_sensor.py index fb966f60afbf0..5861d7a6fdb15 100644 --- a/tests/contrib/sensors/test_sagemaker_training_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_training_sensor.py @@ -18,6 +18,7 @@ # under the License. import unittest +from datetime import datetime try: from unittest import mock @@ -30,55 +31,51 @@ from airflow import configuration from airflow.contrib.sensors.sagemaker_training_sensor \ import SageMakerTrainingSensor -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook, LogState from airflow.exceptions import AirflowException -DESCRIBE_TRAINING_INPROGRESS_RETURN = { - 'TrainingJobStatus': 'InProgress', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } -} -DESCRIBE_TRAINING_COMPELETED_RETURN = { - 'TrainingJobStatus': 'Compeleted', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } -} -DESCRIBE_TRAINING_FAILED_RETURN = { - 'TrainingJobStatus': 'Failed', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, +DESCRIBE_TRAINING_COMPELETED_RESPONSE = { + 'TrainingJobStatus': 'Completed', + 'ResourceConfig': { + 'InstanceCount': 1, + 'InstanceType': 'ml.c4.xlarge', + 'VolumeSizeInGB': 10 }, - 'FailureReason': 'Unknown' -} -DESCRIBE_TRAINING_STOPPING_RETURN = { - 'TrainingJobStatus': 'Stopping', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } -} -DESCRIBE_TRAINING_STOPPED_RETURN = { - 'TrainingJobStatus': 'Stopped', + 'TrainingStartTime': datetime(2018, 2, 17, 7, 15, 0, 103000), + 'TrainingEndTime': datetime(2018, 2, 17, 7, 19, 34, 953000), 'ResponseMetadata': { 'HTTPStatusCode': 200, } } +DESCRIBE_TRAINING_INPROGRESS_RESPONSE = dict(DESCRIBE_TRAINING_COMPELETED_RESPONSE) +DESCRIBE_TRAINING_INPROGRESS_RESPONSE.update({'TrainingJobStatus': 'InProgress'}) + +DESCRIBE_TRAINING_FAILED_RESPONSE = dict(DESCRIBE_TRAINING_COMPELETED_RESPONSE) +DESCRIBE_TRAINING_FAILED_RESPONSE.update({'TrainingJobStatus': 'Failed', + 'FailureReason': 'Unknown'}) + +DESCRIBE_TRAINING_STOPPING_RESPONSE = dict(DESCRIBE_TRAINING_COMPELETED_RESPONSE) +DESCRIBE_TRAINING_STOPPING_RESPONSE.update({'TrainingJobStatus': 'Stopping'}) + class TestSageMakerTrainingSensor(unittest.TestCase): def setUp(self): configuration.load_test_config() @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, '__init__') @mock.patch.object(SageMakerHook, 'describe_training_job') - def test_raises_errors_failed_state(self, mock_describe_job, mock_client): - mock_describe_job.side_effect = [DESCRIBE_TRAINING_FAILED_RETURN] + def test_sensor_with_failure(self, mock_describe_job, hook_init, mock_client): + hook_init.return_value = None + + mock_describe_job.side_effect = [DESCRIBE_TRAINING_FAILED_RESPONSE] sensor = SageMakerTrainingSensor( task_id='test_task', poke_interval=2, aws_conn_id='aws_test', - job_name='test_job_name' + job_name='test_job_name', + print_log=False ) self.assertRaises(AirflowException, sensor.execute, None) mock_describe_job.assert_called_once_with('test_job_name') @@ -86,32 +83,59 @@ def test_raises_errors_failed_state(self, mock_describe_job, mock_client): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, '__init__') @mock.patch.object(SageMakerHook, 'describe_training_job') - def test_calls_until_a_terminal_state(self, - mock_describe_job, hook_init, mock_client): + def test_sensor(self, mock_describe_job, hook_init, mock_client): hook_init.return_value = None mock_describe_job.side_effect = [ - DESCRIBE_TRAINING_INPROGRESS_RETURN, - DESCRIBE_TRAINING_STOPPING_RETURN, - DESCRIBE_TRAINING_STOPPED_RETURN, - DESCRIBE_TRAINING_COMPELETED_RETURN + DESCRIBE_TRAINING_INPROGRESS_RESPONSE, + DESCRIBE_TRAINING_STOPPING_RESPONSE, + DESCRIBE_TRAINING_COMPELETED_RESPONSE ] sensor = SageMakerTrainingSensor( task_id='test_task', poke_interval=2, aws_conn_id='aws_test', job_name='test_job_name', - region_name='us-east-1' + print_log=False ) sensor.execute(None) # make sure we called 4 times(terminated when its compeleted) - self.assertEqual(mock_describe_job.call_count, 4) + self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params - hook_init.assert_called_with(aws_conn_id='aws_test', - region_name='us-east-1') + hook_init.assert_called_with(aws_conn_id='aws_test') + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'get_log_conn') + @mock.patch.object(SageMakerHook, '__init__') + @mock.patch.object(SageMakerHook, 'describe_training_job_with_log') + @mock.patch.object(SageMakerHook, 'describe_training_job') + def test_sensor_with_log(self, mock_describe_job, mock_describe_job_with_log, + hook_init, mock_log_client, mock_client): + hook_init.return_value = None + + mock_describe_job.return_value = DESCRIBE_TRAINING_COMPELETED_RESPONSE + mock_describe_job_with_log.side_effect = [ + (LogState.WAIT_IN_PROGRESS, DESCRIBE_TRAINING_INPROGRESS_RESPONSE, 0), + (LogState.JOB_COMPLETE, DESCRIBE_TRAINING_STOPPING_RESPONSE, 0), + (LogState.COMPLETE, DESCRIBE_TRAINING_COMPELETED_RESPONSE, 0) + ] + sensor = SageMakerTrainingSensor( + task_id='test_task', + poke_interval=2, + aws_conn_id='aws_test', + job_name='test_job_name', + print_log=True + ) + + sensor.execute(None) + + self.assertEqual(mock_describe_job_with_log.call_count, 3) + self.assertEqual(mock_describe_job.call_count, 1) + + hook_init.assert_called_with(aws_conn_id='aws_test') if __name__ == '__main__': diff --git a/tests/contrib/sensors/test_sagemaker_transform_sensor.py b/tests/contrib/sensors/test_sagemaker_transform_sensor.py index bb4a184bb2797..1394920d5dc3d 100644 --- a/tests/contrib/sensors/test_sagemaker_transform_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_transform_sensor.py @@ -33,37 +33,31 @@ from airflow.contrib.hooks.sagemaker_hook import SageMakerHook from airflow.exceptions import AirflowException -DESCRIBE_TRANSFORM_INPROGRESS_RETURN = { +DESCRIBE_TRANSFORM_INPROGRESS_RESPONSE = { 'TransformJobStatus': 'InProgress', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TRANSFORM_COMPELETED_RETURN = { +DESCRIBE_TRANSFORM_COMPELETED_RESPONSE = { 'TransformJobStatus': 'Compeleted', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TRANSFORM_FAILED_RETURN = { +DESCRIBE_TRANSFORM_FAILED_RESPONSE = { 'TransformJobStatus': 'Failed', 'ResponseMetadata': { 'HTTPStatusCode': 200, }, 'FailureReason': 'Unknown' } -DESCRIBE_TRANSFORM_STOPPING_RETURN = { +DESCRIBE_TRANSFORM_STOPPING_RESPONSE = { 'TransformJobStatus': 'Stopping', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TRANSFORM_STOPPED_RETURN = { - 'TransformJobStatus': 'Stopped', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } -} class TestSageMakerTransformSensor(unittest.TestCase): @@ -72,8 +66,8 @@ def setUp(self): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, 'describe_transform_job') - def test_raises_errors_failed_state(self, mock_describe_job, mock_client): - mock_describe_job.side_effect = [DESCRIBE_TRANSFORM_FAILED_RETURN] + def test_sensor_with_failure(self, mock_describe_job, mock_client): + mock_describe_job.side_effect = [DESCRIBE_TRANSFORM_FAILED_RESPONSE] sensor = SageMakerTransformSensor( task_id='test_task', poke_interval=2, @@ -86,32 +80,28 @@ def test_raises_errors_failed_state(self, mock_describe_job, mock_client): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, '__init__') @mock.patch.object(SageMakerHook, 'describe_transform_job') - def test_calls_until_a_terminal_state(self, - mock_describe_job, hook_init, mock_client): + def test_sensor(self, mock_describe_job, hook_init, mock_client): hook_init.return_value = None mock_describe_job.side_effect = [ - DESCRIBE_TRANSFORM_INPROGRESS_RETURN, - DESCRIBE_TRANSFORM_STOPPING_RETURN, - DESCRIBE_TRANSFORM_STOPPED_RETURN, - DESCRIBE_TRANSFORM_COMPELETED_RETURN + DESCRIBE_TRANSFORM_INPROGRESS_RESPONSE, + DESCRIBE_TRANSFORM_STOPPING_RESPONSE, + DESCRIBE_TRANSFORM_COMPELETED_RESPONSE ] sensor = SageMakerTransformSensor( task_id='test_task', poke_interval=2, aws_conn_id='aws_test', - job_name='test_job_name', - region_name='us-east-1' + job_name='test_job_name' ) sensor.execute(None) # make sure we called 4 times(terminated when its compeleted) - self.assertEqual(mock_describe_job.call_count, 4) + self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params - hook_init.assert_called_with(aws_conn_id='aws_test', - region_name='us-east-1') + hook_init.assert_called_with(aws_conn_id='aws_test') if __name__ == '__main__': diff --git a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py index 49f9b41b07c89..8c0ba11380c1a 100644 --- a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py @@ -33,37 +33,34 @@ from airflow.contrib.hooks.sagemaker_hook import SageMakerHook from airflow.exceptions import AirflowException -DESCRIBE_TUNING_INPROGRESS_RETURN = { +DESCRIBE_TUNING_INPROGRESS_RESPONSE = { 'HyperParameterTuningJobStatus': 'InProgress', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TUNING_COMPELETED_RETURN = { + +DESCRIBE_TUNING_COMPELETED_RESPONSE = { 'HyperParameterTuningJobStatus': 'Compeleted', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TUNING_FAILED_RETURN = { + +DESCRIBE_TUNING_FAILED_RESPONSE = { 'HyperParameterTuningJobStatus': 'Failed', 'ResponseMetadata': { 'HTTPStatusCode': 200, }, 'FailureReason': 'Unknown' } -DESCRIBE_TUNING_STOPPING_RETURN = { + +DESCRIBE_TUNING_STOPPING_RESPONSE = { 'HyperParameterTuningJobStatus': 'Stopping', 'ResponseMetadata': { 'HTTPStatusCode': 200, } } -DESCRIBE_TUNING_STOPPED_RETURN = { - 'HyperParameterTuningJobStatus': 'Stopped', - 'ResponseMetadata': { - 'HTTPStatusCode': 200, - } -} class TestSageMakerTuningSensor(unittest.TestCase): @@ -72,8 +69,8 @@ def setUp(self): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, 'describe_tuning_job') - def test_raises_errors_failed_state(self, mock_describe_job, mock_client): - mock_describe_job.side_effect = [DESCRIBE_TUNING_FAILED_RETURN] + def test_sensor_with_failure(self, mock_describe_job, mock_client): + mock_describe_job.side_effect = [DESCRIBE_TUNING_FAILED_RESPONSE] sensor = SageMakerTuningSensor( task_id='test_task', poke_interval=2, @@ -86,32 +83,28 @@ def test_raises_errors_failed_state(self, mock_describe_job, mock_client): @mock.patch.object(SageMakerHook, 'get_conn') @mock.patch.object(SageMakerHook, '__init__') @mock.patch.object(SageMakerHook, 'describe_tuning_job') - def test_calls_until_a_terminal_state(self, - mock_describe_job, hook_init, mock_client): + def test_sensor(self, mock_describe_job, hook_init, mock_client): hook_init.return_value = None mock_describe_job.side_effect = [ - DESCRIBE_TUNING_INPROGRESS_RETURN, - DESCRIBE_TUNING_STOPPING_RETURN, - DESCRIBE_TUNING_STOPPED_RETURN, - DESCRIBE_TUNING_COMPELETED_RETURN + DESCRIBE_TUNING_INPROGRESS_RESPONSE, + DESCRIBE_TUNING_STOPPING_RESPONSE, + DESCRIBE_TUNING_COMPELETED_RESPONSE ] sensor = SageMakerTuningSensor( task_id='test_task', poke_interval=2, aws_conn_id='aws_test', - job_name='test_job_name', - region_name='us-east-1' + job_name='test_job_name' ) sensor.execute(None) # make sure we called 4 times(terminated when its compeleted) - self.assertEqual(mock_describe_job.call_count, 4) + self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params - hook_init.assert_called_with(aws_conn_id='aws_test', - region_name='us-east-1') + hook_init.assert_called_with(aws_conn_id='aws_test') if __name__ == '__main__': diff --git a/tests/hooks/test_s3_hook.py b/tests/hooks/test_s3_hook.py index baac20386feb9..8553e3b6471e9 100644 --- a/tests/hooks/test_s3_hook.py +++ b/tests/hooks/test_s3_hook.py @@ -19,6 +19,7 @@ # import mock +import tempfile import unittest from airflow import configuration @@ -66,6 +67,31 @@ def test_get_bucket(self): b = hook.get_bucket('bucket') self.assertIsNotNone(b) + @mock_s3 + def test_create_bucket_default_region(self): + hook = S3Hook(aws_conn_id=None) + hook.create_bucket(bucket_name='new_bucket') + b = hook.get_bucket('new_bucket') + self.assertIsNotNone(b) + + @mock_s3 + def test_create_bucket_us_standard_region(self): + hook = S3Hook(aws_conn_id=None) + hook.create_bucket(bucket_name='new_bucket', region_name='us-east-1') + b = hook.get_bucket('new_bucket') + self.assertIsNotNone(b) + region = b.meta.client.get_bucket_location(Bucket=b.name).get('LocationConstraint', None) + self.assertEqual(region, 'us-east-1') + + @mock_s3 + def test_create_bucket_other_region(self): + hook = S3Hook(aws_conn_id=None) + hook.create_bucket(bucket_name='new_bucket', region_name='us-east-2') + b = hook.get_bucket('new_bucket') + self.assertIsNotNone(b) + region = b.meta.client.get_bucket_location(Bucket=b.name).get('LocationConstraint', None) + self.assertEqual(region, 'us-east-2') + @mock_s3 def test_check_for_prefix(self): hook = S3Hook(aws_conn_id=None) @@ -241,6 +267,21 @@ def test_load_bytes(self): self.assertEqual(body, b'Content') + @mock_s3 + def test_load_fileobj(self): + hook = S3Hook(aws_conn_id=None) + conn = hook.get_conn() + # We need to create the bucket since this is all in Moto's 'virtual' + # AWS account + conn.create_bucket(Bucket="mybucket") + with tempfile.TemporaryFile() as temp_file: + temp_file.write(b"Content") + temp_file.seek(0) + hook.load_file_obj(temp_file, "my_key", "mybucket") + body = boto3.resource('s3').Object('mybucket', 'my_key').get()['Body'].read() + + self.assertEqual(body, b'Content') + if __name__ == '__main__': unittest.main() diff --git a/tests/operators/test_sagemaker_create_transform_job_operator.py b/tests/operators/test_sagemaker_create_transform_job_operator.py deleted file mode 100644 index a8701530d9daa..0000000000000 --- a/tests/operators/test_sagemaker_create_transform_job_operator.py +++ /dev/null @@ -1,140 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -import unittest -try: - from unittest import mock -except ImportError: - try: - import mock - except ImportError: - mock = None - -from airflow import configuration -from airflow.contrib.hooks.sagemaker_hook import SageMakerHook -from airflow.contrib.operators.sagemaker_create_transform_job_operator \ - import SageMakerCreateTransformJobOperator -from airflow.exceptions import AirflowException - -role = 'test-role' - -bucket = 'test-bucket' - -key = 'test/data' -data_url = 's3://{}/{}'.format(bucket, key) - -job_name = 'test-job-name' - -model_name = 'test-model-name' - -image = 'test-image' - -output_url = 's3://{}/test/output'.format(bucket) - -create_transform_params = \ - { - 'TransformJobName': job_name, - 'ModelName': model_name, - 'BatchStrategy': 'MultiRecord', - 'TransformInput': { - 'DataSource': { - 'S3DataSource': { - 'S3DataType': 'S3Prefix', - 'S3Uri': data_url - } - } - }, - 'TransformOutput': { - 'S3OutputPath': output_url, - }, - 'TransformResources': { - 'InstanceType': 'ml.m4.xlarge', - 'InstanceCount': 123 - } - } - -create_model_params = \ - { - 'ModelName': model_name, - 'PrimaryContainer': { - 'Image': image, - 'ModelDataUrl': output_url, - }, - 'ExecutionRoleArn': role - } - - -class TestSageMakertransformOperator(unittest.TestCase): - - def setUp(self): - configuration.load_test_config() - self.sagemaker = SageMakerCreateTransformJobOperator( - task_id='test_sagemaker_operator', - sagemaker_conn_id='sagemaker_test_id', - transform_job_config=create_transform_params, - model_config=create_model_params, - region_name='us-west-2', - use_db_config=True, - wait_for_completion=False, - check_interval=5 - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_model') - @mock.patch.object(SageMakerHook, 'create_transform_job') - @mock.patch.object(SageMakerHook, '__init__') - def test_hook_init(self, hook_init, mock_transform, mock_model, mock_client): - mock_transform.return_value = {"TransformJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 200}} - hook_init.return_value = None - self.sagemaker.execute(None) - hook_init.assert_called_once_with( - sagemaker_conn_id='sagemaker_test_id', - region_name='us-west-2', - use_db_config=True, - check_interval=5, - max_ingestion_time=None - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_model') - @mock.patch.object(SageMakerHook, 'create_transform_job') - def test_execute_without_failure(self, mock_transform, mock_model, mock_client): - mock_transform.return_value = {"TransformJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 200}} - self.sagemaker.execute(None) - mock_model.assert_called_once_with(create_model_params) - mock_transform.assert_called_once_with(create_transform_params, - wait_for_completion=False - ) - - @mock.patch.object(SageMakerHook, 'get_conn') - @mock.patch.object(SageMakerHook, 'create_model') - @mock.patch.object(SageMakerHook, 'create_transform_job') - def test_execute_with_failure(self, mock_transform, mock_model, mock_client): - mock_transform.return_value = {"TransformJobArn": "testarn", - "ResponseMetadata": - {"HTTPStatusCode": 404}} - self.assertRaises(AirflowException, self.sagemaker.execute, None) - - -if __name__ == '__main__': - unittest.main() From 805a35b150b8f0b2fdc194e37c415d947cf44815 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 2 Nov 2018 13:33:09 +0000 Subject: [PATCH 142/808] [AIRFLOW-3262] Add param to log response when using SimpleHttpOperator (#4102) * [AIRFLOW-3262] Add param to log response when using SimpleHttpOperator --- airflow/operators/http_operator.py | 11 ++++- tests/operators/test_http_operator.py | 63 +++++++++++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 tests/operators/test_http_operator.py diff --git a/airflow/operators/http_operator.py b/airflow/operators/http_operator.py index 2cfc9c0057c7c..40fba453db9d1 100644 --- a/airflow/operators/http_operator.py +++ b/airflow/operators/http_operator.py @@ -46,6 +46,10 @@ class SimpleHttpOperator(BaseOperator): 'requests' documentation (options to modify timeout, ssl, etc.) :type extra_options: A dictionary of options, where key is string and value depends on the option that's being modified. + :param xcom_push: Push the response to Xcom (default: False) + :type xcom_push: bool + :param log_response: Log the response (default: False) + :type log_response: bool """ template_fields = ('endpoint', 'data',) @@ -61,7 +65,9 @@ def __init__(self, response_check=None, extra_options=None, xcom_push=False, - http_conn_id='http_default', *args, **kwargs): + http_conn_id='http_default', + log_response=False, + *args, **kwargs): """ If xcom_push is True, response of an HTTP request will also be pushed to an XCom. @@ -75,6 +81,7 @@ def __init__(self, self.response_check = response_check self.extra_options = extra_options or {} self.xcom_push_flag = xcom_push + self.log_response = log_response def execute(self, context): http = HttpHook(self.method, http_conn_id=self.http_conn_id) @@ -90,3 +97,5 @@ def execute(self, context): raise AirflowException("Response check returned False.") if self.xcom_push_flag: return response.text + if self.log_response: + self.log.info(response.text) diff --git a/tests/operators/test_http_operator.py b/tests/operators/test_http_operator.py new file mode 100644 index 0000000000000..6ab2c03bb5f47 --- /dev/null +++ b/tests/operators/test_http_operator.py @@ -0,0 +1,63 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import os +import unittest + +from mock import patch +from airflow.operators.http_operator import SimpleHttpOperator + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + + +class AnyStringWith(str): + """ + Helper class to check if a substring is a part of a string + """ + def __eq__(self, other): + return self in other + + +class SimpleHttpOpTests(unittest.TestCase): + def setUp(self): + # Creating a local Http connection to Airflow Webserver + os.environ['AIRFLOW_CONN_HTTP_GOOGLE'] = 'http://www.google.com' + + def test_response_in_logs(self): + """ + Test that when using SimpleHttpOperator with 'GET' on localhost:8080, + the log contains 'Google' in it + """ + operator = SimpleHttpOperator( + task_id='test_HTTP_op', + method='GET', + endpoint='/', + http_conn_id='HTTP_GOOGLE', + log_response=True, + ) + + with patch.object(operator.log, 'info') as mock_info: + operator.execute(None) + mock_info.assert_called_with(AnyStringWith('Google')) From e77cd5fb77e324eb01391a7419fd1eca658b4ce0 Mon Sep 17 00:00:00 2001 From: Szymon Przedwojski Date: Fri, 2 Nov 2018 14:38:31 +0100 Subject: [PATCH 143/808] [AIRFLOW-3276] Cloud SQL: database create / patch / delete operators (#4124) --- .../contrib/example_dags/example_gcp_sql.py | 60 ++++- airflow/contrib/hooks/gcp_sql_hook.py | 90 +++++++ airflow/contrib/operators/gcp_sql_operator.py | 235 +++++++++++++++++- docs/howto/operator.rst | 148 ++++++++++- docs/integration.rst | 35 ++- .../operators/test_gcp_sql_operator.py | 152 ++++++++++- 6 files changed, 701 insertions(+), 19 deletions(-) diff --git a/airflow/contrib/example_dags/example_gcp_sql.py b/airflow/contrib/example_dags/example_gcp_sql.py index a484456f6e6d9..136c88c843014 100644 --- a/airflow/contrib/example_dags/example_gcp_sql.py +++ b/airflow/contrib/example_dags/example_gcp_sql.py @@ -18,26 +18,30 @@ # under the License. """ -Example Airflow DAG that deploys, updates, patches and deletes a Cloud SQL instance -in Google Cloud Platform. +Example Airflow DAG that creates, patches and deletes a Cloud SQL instance, and also +creates, patches and deletes a database inside the instance, in Google Cloud Platform. -This DAG relies on the following Airflow variables -https://airflow.apache.org/concepts.html#variables +This DAG relies on the following environment variables * PROJECT_ID - Google Cloud Platform project for the Cloud SQL instance. * INSTANCE_NAME - Name of the Cloud SQL instance. +* DB_NAME - Name of the database inside a Cloud SQL instance. """ +import os import datetime import airflow from airflow import models from airflow.contrib.operators.gcp_sql_operator import CloudSqlInstanceCreateOperator, \ - CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator + CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator, \ + CloudSqlInstanceDatabaseCreateOperator, CloudSqlInstanceDatabasePatchOperator, \ + CloudSqlInstanceDatabaseDeleteOperator # [START howto_operator_cloudsql_arguments] -PROJECT_ID = models.Variable.get('PROJECT_ID', '') -INSTANCE_NAME = models.Variable.get('INSTANCE_NAME', '') +PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project') +INSTANCE_NAME = os.environ.get('INSTANCE_NAME', 'testinstance') +DB_NAME = os.environ.get('DB_NAME', 'testdb') # [END howto_operator_cloudsql_arguments] # Bodies below represent Cloud SQL instance resources: @@ -97,6 +101,19 @@ } } # [END howto_operator_cloudsql_patch_body] +# [START howto_operator_cloudsql_db_create_body] +db_create_body = { + "instance": INSTANCE_NAME, + "name": DB_NAME, + "project": PROJECT_ID +} +# [END howto_operator_cloudsql_db_create_body] +# [START howto_operator_cloudsql_db_patch_body] +db_patch_body = { + "charset": "utf16", + "collation": "utf16_general_ci" +} +# [END howto_operator_cloudsql_db_patch_body] default_args = { 'start_date': airflow.utils.dates.days_ago(1) @@ -123,6 +140,31 @@ task_id='sql_instance_patch_task' ) # [END howto_operator_cloudsql_patch] + # [START howto_operator_cloudsql_db_create] + sql_db_create_task = CloudSqlInstanceDatabaseCreateOperator( + project_id=PROJECT_ID, + body=db_create_body, + instance=INSTANCE_NAME, + task_id='sql_db_create_task' + ) + # [END howto_operator_cloudsql_db_create] + # [START howto_operator_cloudsql_db_patch] + sql_db_patch_task = CloudSqlInstanceDatabasePatchOperator( + project_id=PROJECT_ID, + body=db_patch_body, + instance=INSTANCE_NAME, + database=DB_NAME, + task_id='sql_db_patch_task' + ) + # [END howto_operator_cloudsql_db_patch] + # [START howto_operator_cloudsql_db_delete] + sql_db_delete_task = CloudSqlInstanceDatabaseDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database=DB_NAME, + task_id='sql_db_delete_task' + ) + # [END howto_operator_cloudsql_db_delete] # [START howto_operator_cloudsql_delete] sql_instance_delete_task = CloudSqlInstanceDeleteOperator( project_id=PROJECT_ID, @@ -131,4 +173,6 @@ ) # [END howto_operator_cloudsql_delete] - sql_instance_create_task >> sql_instance_patch_task >> sql_instance_delete_task + sql_instance_create_task >> sql_instance_patch_task \ + >> sql_db_create_task >> sql_db_patch_task \ + >> sql_db_delete_task >> sql_instance_delete_task diff --git a/airflow/contrib/hooks/gcp_sql_hook.py b/airflow/contrib/hooks/gcp_sql_hook.py index e0b3f92d8f20e..549ceaf49cec3 100644 --- a/airflow/contrib/hooks/gcp_sql_hook.py +++ b/airflow/contrib/hooks/gcp_sql_hook.py @@ -144,6 +144,96 @@ def delete_instance(self, project_id, instance): operation_name = response["name"] return self._wait_for_operation_to_complete(project_id, operation_name) + def get_database(self, project_id, instance, database): + """ + Retrieves a database resource from a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param database: Name of the database in the instance. + :type database: str + :return: A Cloud SQL database resource, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases#resource + :rtype: dict + """ + return self.get_conn().databases().get( + project=project_id, + instance=instance, + database=database + ).execute(num_retries=NUM_RETRIES) + + def create_database(self, project, instance, body): + """ + Creates a new database inside a Cloud SQL instance. + + :param project: Project ID of the project that contains the instance. + :type project: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param body: The request body, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/insert#request-body + :type body: dict + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().databases().insert( + project=project, + instance=instance, + body=body + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project, operation_name) + + def patch_database(self, project, instance, database, body): + """ + Updates a database resource inside a Cloud SQL instance. + This method supports patch semantics. + See: https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch + + :param project: Project ID of the project that contains the instance. + :type project: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param database: Name of the database to be updated in the instance. + :type database: str + :param body: The request body, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/insert#request-body + :type body: dict + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().databases().patch( + project=project, + instance=instance, + database=database, + body=body + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project, operation_name) + + def delete_database(self, project, instance, database): + """ + Deletes a database from a Cloud SQL instance. + + :param project: Project ID of the project that contains the instance. + :type project: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param database: Name of the database to be deleted in the instance. + :type database: str + :return: True if the operation succeeded, raises an error otherwise + :rtype: bool + """ + response = self.get_conn().databases().delete( + project=project, + instance=instance, + database=database + ).execute(num_retries=NUM_RETRIES) + operation_name = response["name"] + return self._wait_for_operation_to_complete(project, operation_name) + def _wait_for_operation_to_complete(self, project_id, operation_name): """ Waits for the named operation to complete - checks status of the diff --git a/airflow/contrib/operators/gcp_sql_operator.py b/airflow/contrib/operators/gcp_sql_operator.py index 0ba7a300c9e41..aca2e8ad40a34 100644 --- a/airflow/contrib/operators/gcp_sql_operator.py +++ b/airflow/contrib/operators/gcp_sql_operator.py @@ -91,6 +91,45 @@ ], optional=True), ], optional=True) ] +CLOUD_SQL_EXPORT_VALIDATION = [ + dict(name="exportContext", type="dict", fields=[ + dict(name="fileType", allow_empty=False), + dict(name="uri", allow_empty=False), + dict(name="databases", type="list"), + dict(name="sqlExportOptions", type="dict", optional=True, fields=[ + dict(name="tables", type="list"), + dict(name="schemaOnly") + ]), + dict(name="csvExportOptions", type="dict", optional=True, fields=[ + dict(name="selectQuery") + ]) + ]) +] +CLOUD_SQL_IMPORT_VALIDATION = [ + dict(name="importContext", type="dict", fields=[ + dict(name="fileType", allow_empty=False), + dict(name="uri", allow_empty=False), + dict(name="database", optional=True, allow_empty=False), + dict(name="importUser", optional=True), + dict(name="csvImportOptions", type="dict", optional=True, fields=[ + dict(name="table"), + dict(name="columns", type="list", optional=True) + ]) + ]) +] +CLOUD_SQL_DATABASE_INSERT_VALIDATION = [ + dict(name="instance", allow_empty=False), + dict(name="name", allow_empty=False), + dict(name="project", allow_empty=False), +] +CLOUD_SQL_DATABASE_PATCH_VALIDATION = [ + dict(name="instance", optional=True), + dict(name="name", optional=True), + dict(name="project", optional=True), + dict(name="etag", optional=True), + dict(name="charset", optional=True), + dict(name="collation", optional=True), +] class CloudSqlBaseOperator(BaseOperator): @@ -137,6 +176,15 @@ def _check_if_instance_exists(self, instance): return False raise e + def _check_if_db_exists(self, db_name): + try: + return self._hook.get_database(self.project_id, self.instance, db_name) + except HttpError as e: + status = e.resp.status + if status == 404: + return False + raise e + def execute(self, context): pass @@ -162,7 +210,7 @@ class CloudSqlInstanceCreateOperator(CloudSqlBaseOperator): :type instance: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (e.g. v1beta4). :type api_version: str :param validate_body: True if body should be validated, False otherwise. :type validate_body: bool @@ -226,7 +274,7 @@ class CloudSqlInstancePatchOperator(CloudSqlBaseOperator): :type instance: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (e.g. v1beta4). :type api_version: str """ # [START gcp_sql_patch_template_fields] @@ -270,7 +318,7 @@ class CloudSqlInstanceDeleteOperator(CloudSqlBaseOperator): :type instance: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (e.g. v1beta4). :type api_version: str """ # [START gcp_sql_delete_template_fields] @@ -295,3 +343,184 @@ def execute(self, context): return True else: return self._hook.delete_instance(self.project_id, self.instance) + + +class CloudSqlInstanceDatabaseCreateOperator(CloudSqlBaseOperator): + """ + Creates a new database inside a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param body: The request body, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/insert#request-body + :type body: dict + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1beta4). + :type api_version: str + :param validate_body: Whether the body should be validated. Defaults to True. + :type validate_body: bool + """ + # [START gcp_sql_db_create_template_fields] + template_fields = ('project_id', 'instance', 'gcp_conn_id', 'api_version') + # [END gcp_sql_db_create_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance, + body, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + validate_body=True, + *args, **kwargs): + self.body = body + self.validate_body = validate_body + super(CloudSqlInstanceDatabaseCreateOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def _validate_inputs(self): + super(CloudSqlInstanceDatabaseCreateOperator, self)._validate_inputs() + if not self.body: + raise AirflowException("The required parameter 'body' is empty") + + def _validate_body_fields(self): + if self.validate_body: + GcpBodyFieldValidator(CLOUD_SQL_DATABASE_INSERT_VALIDATION, + api_version=self.api_version).validate(self.body) + + def execute(self, context): + self._validate_body_fields() + database = self.body.get("name") + if not database: + self.log.error("Body doesn't contain 'name'. Cannot check if the" + " database already exists in the instance {}." + .format(self.instance)) + return False + if self._check_if_db_exists(database): + self.log.info("Cloud SQL instance with ID {} already contains database" + " '{}'. Aborting database insert." + .format(self.instance, database)) + return True + else: + return self._hook.create_database(self.project_id, self.instance, self.body) + + +class CloudSqlInstanceDatabasePatchOperator(CloudSqlBaseOperator): + """ + Updates a resource containing information about a database inside a Cloud SQL + instance using patch semantics. + See: https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param database: Name of the database to be updated in the instance. + :type database: str + :param body: The request body, as described in + https://cloud.google.com/sql/docs/mysql/admin-api/v1beta4/databases/patch#request-body + :type body: dict + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1beta4). + :type api_version: str + :param validate_body: Whether the body should be validated. Defaults to True. + :type validate_body: bool + """ + # [START gcp_sql_db_patch_template_fields] + template_fields = ('project_id', 'instance', 'database', 'gcp_conn_id', + 'api_version') + # [END gcp_sql_db_patch_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance, + database, + body, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + validate_body=True, + *args, **kwargs): + self.database = database + self.body = body + self.validate_body = validate_body + super(CloudSqlInstanceDatabasePatchOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def _validate_inputs(self): + super(CloudSqlInstanceDatabasePatchOperator, self)._validate_inputs() + if not self.body: + raise AirflowException("The required parameter 'body' is empty") + if not self.database: + raise AirflowException("The required parameter 'database' is empty") + + def _validate_body_fields(self): + if self.validate_body: + GcpBodyFieldValidator(CLOUD_SQL_DATABASE_PATCH_VALIDATION, + api_version=self.api_version).validate(self.body) + + def execute(self, context): + self._validate_body_fields() + if not self._check_if_db_exists(self.database): + raise AirflowException("Cloud SQL instance with ID {} does not contain " + "database '{}'. " + "Please specify another database to patch." + .format(self.instance, self.database)) + else: + return self._hook.patch_database(self.project_id, self.instance, + self.database, self.body) + + +class CloudSqlInstanceDatabaseDeleteOperator(CloudSqlBaseOperator): + """ + Deletes a database from a Cloud SQL instance. + + :param project_id: Project ID of the project that contains the instance. + :type project_id: str + :param instance: Database instance ID. This does not include the project ID. + :type instance: str + :param database: Name of the database to be deleted in the instance. + :type database: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (e.g. v1beta4). + :type api_version: str + """ + # [START gcp_sql_db_delete_template_fields] + template_fields = ('project_id', 'instance', 'database', 'gcp_conn_id', + 'api_version') + # [END gcp_sql_db_delete_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance, + database, + gcp_conn_id='google_cloud_default', + api_version='v1beta4', + *args, **kwargs): + self.database = database + super(CloudSqlInstanceDatabaseDeleteOperator, self).__init__( + project_id=project_id, instance=instance, gcp_conn_id=gcp_conn_id, + api_version=api_version, *args, **kwargs) + + def _validate_inputs(self): + super(CloudSqlInstanceDatabaseDeleteOperator, self)._validate_inputs() + if not self.database: + raise AirflowException("The required parameter 'database' is empty") + + def execute(self, context): + if not self._check_if_db_exists(self.database): + print("Cloud SQL instance with ID {} does not contain database '{}'. " + "Aborting database delete." + .format(self.instance, self.database)) + return True + else: + return self._hook.delete_database(self.project_id, self.instance, + self.database) diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index 025274a5a41ae..6333e32dd7d67 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -284,6 +284,148 @@ If the source code for your function is in Google Source Repository, make sure t your service account has the Source Repository Viewer role so that the source code can be downloaded if necessary. +CloudSqlInstanceDatabaseCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Creates a new database inside a Cloud SQL instance. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseCreateOperator`. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from environment variables: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_arguments] + :end-before: [END howto_operator_cloudsql_arguments] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_cloudsql_db_create] + :end-before: [END howto_operator_cloudsql_db_create] + +Example request body: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_db_create_body] + :end-before: [END howto_operator_cloudsql_db_create_body] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_sql_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_sql_db_create_template_fields] + :end-before: [END gcp_sql_db_create_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud SQL API documentation for database insert +`_. + +CloudSqlInstanceDatabaseDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Deletes a database from a Cloud SQL instance. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseDeleteOperator`. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from environment variables: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_arguments] + :end-before: [END howto_operator_cloudsql_arguments] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_cloudsql_db_delete] + :end-before: [END howto_operator_cloudsql_db_delete] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_sql_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_sql_db_delete_template_fields] + :end-before: [END gcp_sql_db_delete_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud SQL API documentation for database delete +`_. + +CloudSqlInstanceDatabasePatchOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Updates a resource containing information about a database inside a Cloud SQL instance +using patch semantics. +See: https://cloud.google.com/sql/docs/mysql/admin-api/how-tos/performance#patch + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabasePatchOperator`. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from environment variables: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_arguments] + :end-before: [END howto_operator_cloudsql_arguments] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_cloudsql_db_patch] + :end-before: [END howto_operator_cloudsql_db_patch] + +Example request body: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py + :language: python + :start-after: [START howto_operator_cloudsql_db_patch_body] + :end-before: [END howto_operator_cloudsql_db_patch_body] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_sql_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_sql_db_patch_template_fields] + :end-before: [END gcp_sql_db_patch_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud SQL API documentation for database patch +`_. + CloudSqlInstanceDeleteOperator ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -295,7 +437,7 @@ For parameter definition take a look at Arguments """"""""" -Some arguments in the example DAG are taken from Airflow variables: +Some arguments in the example DAG are taken from environment variables: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py :language: python @@ -342,7 +484,7 @@ will succeed. Arguments """"""""" -Some arguments in the example DAG are taken from Airflow variables: +Some arguments in the example DAG are taken from environment variables: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py :language: python @@ -398,7 +540,7 @@ unchanged. Arguments """"""""" -Some arguments in the example DAG are taken from Airflow variables: +Some arguments in the example DAG are taken from environment variables: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_sql.py :language: python diff --git a/docs/integration.rst b/docs/integration.rst index e72f687151de5..49c383541f191 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -445,10 +445,37 @@ Cloud SQL Cloud SQL Operators """"""""""""""""""" +- :ref:`CloudSqlInstanceDatabaseDeleteOperator` : deletes a database from a Cloud SQL +instance. +- :ref:`CloudSqlInstanceDatabaseCreateOperator` : creates a new database inside a Cloud +SQL instance. +- :ref:`CloudSqlInstanceDatabasePatchOperator` : updates a database inside a Cloud +SQL instance. - :ref:`CloudSqlInstanceDeleteOperator` : delete a Cloud SQL instance. - :ref:`CloudSqlInstanceCreateOperator` : create a new Cloud SQL instance. - :ref:`CloudSqlInstancePatchOperator` : patch a Cloud SQL instance. +.. CloudSqlInstanceDatabaseDeleteOperator: + +CloudSqlInstanceDatabaseDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseDeleteOperator + +.. CloudSqlInstanceDatabaseCreateOperator: + +CloudSqlInstanceDatabaseCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabaseCreateOperator + +.. CloudSqlInstanceDatabasePatchOperator: + +CloudSqlInstanceDatabasePatchOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstanceDatabasePatchOperator + .. CloudSqlInstanceDeleteOperator: CloudSqlInstanceDeleteOperator @@ -466,12 +493,12 @@ CloudSqlInstanceCreateOperator .. CloudSqlInstancePatchOperator: CloudSqlInstancePatchOperator -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ .. autoclass:: airflow.contrib.operators.gcp_sql_operator.CloudSqlInstancePatchOperator Cloud SQL Hook -"""""""""""""""""""" +"""""""""""""" .. autoclass:: airflow.contrib.hooks.gcp_sql_hook.CloudSqlHook :members: @@ -496,14 +523,14 @@ GceInstanceStartOperator .. _GceInstanceStopOperator: GceInstanceStopOperator -^^^^^^^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^^^^^^ .. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceStopOperator .. _GceSetMachineTypeOperator: GceSetMachineTypeOperator -^^^^^^^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^^^^^^^^ .. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceSetMachineTypeOperator diff --git a/tests/contrib/operators/test_gcp_sql_operator.py b/tests/contrib/operators/test_gcp_sql_operator.py index 245631808a223..31ed3d37c3c52 100644 --- a/tests/contrib/operators/test_gcp_sql_operator.py +++ b/tests/contrib/operators/test_gcp_sql_operator.py @@ -21,7 +21,9 @@ from airflow import AirflowException from airflow.contrib.operators.gcp_sql_operator import CloudSqlInstanceCreateOperator, \ - CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator + CloudSqlInstancePatchOperator, CloudSqlInstanceDeleteOperator, \ + CloudSqlInstanceDatabaseCreateOperator, CloudSqlInstanceDatabasePatchOperator, \ + CloudSqlInstanceDatabaseDeleteOperator try: # noinspection PyProtectedMember @@ -34,6 +36,7 @@ PROJECT_ID = "project-id" INSTANCE_NAME = "test-name" +DB_NAME = "db1" CREATE_BODY = { "name": INSTANCE_NAME, "settings": { @@ -109,6 +112,21 @@ }, "region": "europe-west4" } +DATABASE_INSERT_BODY = { + "name": DB_NAME, # The name of the database in the Cloud SQL instance. + # This does not include the project ID or instance name. + + "project": PROJECT_ID, # The project ID of the project containing the Cloud SQL + # database. The Google apps domain is prefixed if + # applicable. + + "instance": INSTANCE_NAME, # The name of the Cloud SQL instance. + # This does not include the project ID. +} +DATABASE_PATCH_BODY = { + "charset": "utf16", + "collation": "utf16_general_ci" +} class CloudSqlTest(unittest.TestCase): @@ -312,3 +330,135 @@ def test_instance_delete_should_abort_and_succeed_if_not_exists( mock_hook.assert_called_once_with(api_version="v1beta4", gcp_conn_id="google_cloud_default") mock_hook.return_value.delete_instance.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabaseCreateOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_create(self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = False + op = CloudSqlInstanceDatabaseCreateOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + body=DATABASE_INSERT_BODY, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_database.assert_called_once_with( + PROJECT_ID, INSTANCE_NAME, DATABASE_INSERT_BODY + ) + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabaseCreateOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_create_should_abort_and_succeed_if_exists( + self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = True + op = CloudSqlInstanceDatabaseCreateOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + body=DATABASE_INSERT_BODY, + task_id="id" + ) + result = op.execute(None) + self.assertTrue(result) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_database.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabasePatchOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_patch(self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = True + op = CloudSqlInstanceDatabasePatchOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database=DB_NAME, + body=DATABASE_PATCH_BODY, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.patch_database.assert_called_once_with( + PROJECT_ID, INSTANCE_NAME, DB_NAME, DATABASE_PATCH_BODY + ) + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabasePatchOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_patch_should_throw_ex_if_not_exists( + self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = False + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceDatabasePatchOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database=DB_NAME, + body=DATABASE_PATCH_BODY, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("Cloud SQL instance with ID", str(err)) + self.assertIn("does not contain database", str(err)) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.patch_database.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_patch_should_throw_ex_when_empty_database(self, mock_hook): + with self.assertRaises(AirflowException) as cm: + op = CloudSqlInstanceDatabasePatchOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database="", + body=DATABASE_INSERT_BODY, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The required parameter 'database' is empty", str(err)) + mock_hook.assert_not_called() + mock_hook.return_value.patch_database.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabaseDeleteOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_delete(self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = True + op = CloudSqlInstanceDatabaseDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database=DB_NAME, + task_id="id" + ) + result = op.execute(None) + self.assertTrue(result) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_database.assert_called_once_with( + PROJECT_ID, INSTANCE_NAME, DB_NAME + ) + + @mock.patch("airflow.contrib.operators.gcp_sql_operator" + ".CloudSqlInstanceDatabaseDeleteOperator._check_if_db_exists") + @mock.patch("airflow.contrib.operators.gcp_sql_operator.CloudSqlHook") + def test_instance_db_delete_should_abort_and_succeed_if_not_exists( + self, mock_hook, _check_if_db_exists): + _check_if_db_exists.return_value = False + op = CloudSqlInstanceDatabaseDeleteOperator( + project_id=PROJECT_ID, + instance=INSTANCE_NAME, + database=DB_NAME, + task_id="id" + ) + result = op.execute(None) + self.assertTrue(result) + mock_hook.assert_called_once_with(api_version="v1beta4", + gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_database.assert_not_called() From cb65b1998b9290d71a20a35151c416847cc9a699 Mon Sep 17 00:00:00 2001 From: bolkedebruin Date: Mon, 5 Nov 2018 12:16:20 +0100 Subject: [PATCH 144/808] [AIRFLOW-3277] Correctly observe DST transitions for cron (#4117) `following_schedule` converts to naive time by using the local time zone. In case of a DST transition, say 3AM -> 2AM ("summer time to winter time") we incorrectly re-applied the timezone information which meant that a "CEST -> CEST" could happen instead of a "CEST -> CET". This resulted in infinite loops. --- airflow/models.py | 64 +++++++++++++++++++++++++++------ tests/models.py | 90 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 144 insertions(+), 10 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 9ec7a472aa840..93114cce2f438 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -41,6 +41,7 @@ import logging import numbers import os +import pendulum import pickle import re import signal @@ -3340,34 +3341,77 @@ def date_range(self, start_date, num=None, end_date=timezone.utcnow()): start_date=start_date, end_date=end_date, num=num, delta=self._schedule_interval) + def is_fixed_time_schedule(self): + """ + Figures out if the DAG schedule has a fixed time (e.g. 3 AM). + + :return: True if the schedule has a fixed time, False if not. + """ + now = datetime.now() + cron = croniter(self._schedule_interval, now) + + start = cron.get_next(datetime) + cron_next = cron.get_next(datetime) + + if cron_next.minute == start.minute and cron_next.hour == start.hour: + return True + + return False + def following_schedule(self, dttm): """ - Calculates the following schedule for this dag in local time + Calculates the following schedule for this dag in UTC. :param dttm: utc datetime :return: utc datetime """ if isinstance(self._schedule_interval, six.string_types): - dttm = timezone.make_naive(dttm, self.timezone) - cron = croniter(self._schedule_interval, dttm) - following = timezone.make_aware(cron.get_next(datetime), self.timezone) + # we don't want to rely on the transitions created by + # croniter as they are not always correct + dttm = pendulum.instance(dttm) + naive = timezone.make_naive(dttm, self.timezone) + cron = croniter(self._schedule_interval, naive) + + # We assume that DST transitions happen on the minute/hour + if not self.is_fixed_time_schedule(): + # relative offset (eg. every 5 minutes) + delta = cron.get_next(datetime) - naive + following = dttm.in_timezone(self.timezone).add_timedelta(delta) + else: + # absolute (e.g. 3 AM) + naive = cron.get_next(datetime) + tz = pendulum.timezone(self.timezone.name) + following = timezone.make_aware(naive, tz) return timezone.convert_to_utc(following) elif isinstance(self._schedule_interval, timedelta): return dttm + self._schedule_interval def previous_schedule(self, dttm): """ - Calculates the previous schedule for this dag in local time + Calculates the previous schedule for this dag in UTC :param dttm: utc datetime :return: utc datetime """ if isinstance(self._schedule_interval, six.string_types): - dttm = timezone.make_naive(dttm, self.timezone) - cron = croniter(self._schedule_interval, dttm) - prev = timezone.make_aware(cron.get_prev(datetime), self.timezone) - return timezone.convert_to_utc(prev) - elif isinstance(self._schedule_interval, timedelta): + # we don't want to rely on the transitions created by + # croniter as they are not always correct + dttm = pendulum.instance(dttm) + naive = timezone.make_naive(dttm, self.timezone) + cron = croniter(self._schedule_interval, naive) + + # We assume that DST transitions happen on the minute/hour + if not self.is_fixed_time_schedule(): + # relative offset (eg. every 5 minutes) + delta = naive - cron.get_prev(datetime) + previous = dttm.in_timezone(self.timezone).subtract_timedelta(delta) + else: + # absolute (e.g. 3 AM) + naive = cron.get_prev(datetime) + tz = pendulum.timezone(self.timezone.name) + previous = timezone.make_aware(naive, tz) + return timezone.convert_to_utc(previous) + elif self._schedule_interval is not None: return dttm - self._schedule_interval def get_run_dates(self, start_date, end_date=None): diff --git a/tests/models.py b/tests/models.py index 1e6787927b339..16a8d6a3e1b76 100644 --- a/tests/models.py +++ b/tests/models.py @@ -544,6 +544,96 @@ def test_cycle(self): with self.assertRaises(AirflowDagCycleException): dag.test_cycle() + def test_following_previous_schedule(self): + """ + Make sure DST transitions are properly observed + """ + local_tz = pendulum.timezone('Europe/Zurich') + start = local_tz.convert(datetime.datetime(2018, 10, 28, 2, 55), + dst_rule=pendulum.PRE_TRANSITION) + self.assertEqual(start.isoformat(), "2018-10-28T02:55:00+02:00", + "Pre-condition: start date is in DST") + + utc = timezone.convert_to_utc(start) + + dag = DAG('tz_dag', start_date=start, schedule_interval='*/5 * * * *') + _next = dag.following_schedule(utc) + next_local = local_tz.convert(_next) + + self.assertEqual(_next.isoformat(), "2018-10-28T01:00:00+00:00") + self.assertEqual(next_local.isoformat(), "2018-10-28T02:00:00+01:00") + + prev = dag.previous_schedule(utc) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-10-28T02:50:00+02:00") + + prev = dag.previous_schedule(_next) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-10-28T02:55:00+02:00") + self.assertEqual(prev, utc) + + def test_following_previous_schedule_daily_dag_CEST_to_CET(self): + """ + Make sure DST transitions are properly observed + """ + local_tz = pendulum.timezone('Europe/Zurich') + start = local_tz.convert(datetime.datetime(2018, 10, 27, 3), + dst_rule=pendulum.PRE_TRANSITION) + + utc = timezone.convert_to_utc(start) + + dag = DAG('tz_dag', start_date=start, schedule_interval='0 3 * * *') + + prev = dag.previous_schedule(utc) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-10-26T03:00:00+02:00") + self.assertEqual(prev.isoformat(), "2018-10-26T01:00:00+00:00") + + _next = dag.following_schedule(utc) + next_local = local_tz.convert(_next) + + self.assertEqual(next_local.isoformat(), "2018-10-28T03:00:00+01:00") + self.assertEqual(_next.isoformat(), "2018-10-28T02:00:00+00:00") + + prev = dag.previous_schedule(_next) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-10-27T03:00:00+02:00") + self.assertEqual(prev.isoformat(), "2018-10-27T01:00:00+00:00") + + def test_following_previous_schedule_daily_dag_CET_to_CEST(self): + """ + Make sure DST transitions are properly observed + """ + local_tz = pendulum.timezone('Europe/Zurich') + start = local_tz.convert(datetime.datetime(2018, 3, 25, 2), + dst_rule=pendulum.PRE_TRANSITION) + + utc = timezone.convert_to_utc(start) + + dag = DAG('tz_dag', start_date=start, schedule_interval='0 3 * * *') + + prev = dag.previous_schedule(utc) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-03-24T03:00:00+01:00") + self.assertEqual(prev.isoformat(), "2018-03-24T02:00:00+00:00") + + _next = dag.following_schedule(utc) + next_local = local_tz.convert(_next) + + self.assertEqual(next_local.isoformat(), "2018-03-25T03:00:00+02:00") + self.assertEqual(_next.isoformat(), "2018-03-25T01:00:00+00:00") + + prev = dag.previous_schedule(_next) + prev_local = local_tz.convert(prev) + + self.assertEqual(prev_local.isoformat(), "2018-03-24T03:00:00+01:00") + self.assertEqual(prev.isoformat(), "2018-03-24T02:00:00+00:00") + class DagStatTest(unittest.TestCase): def test_dagstats_crud(self): From 8910def43be0ab3d5fa001aa91b9c39140f7153d Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Fri, 28 Sep 2018 13:09:01 -0400 Subject: [PATCH 145/808] [AIRFLOW-3129] Backfill mysql hook unit tests. (#3970) --- tests/hooks/test_mysql_hook.py | 86 ++++++++++++++++++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/tests/hooks/test_mysql_hook.py b/tests/hooks/test_mysql_hook.py index d112f880e9e3a..415d430fc4b77 100644 --- a/tests/hooks/test_mysql_hook.py +++ b/tests/hooks/test_mysql_hook.py @@ -18,12 +18,81 @@ # under the License. # +import json import mock import unittest +import MySQLdb.cursors + +from airflow import models from airflow.hooks.mysql_hook import MySqlHook +class TestMySqlHookConn(unittest.TestCase): + + def setUp(self): + super(TestMySqlHookConn, self).setUp() + + self.connection = models.Connection( + login='login', + password='password', + host='host', + schema='schema', + ) + + self.db_hook = MySqlHook() + self.db_hook.get_connection = mock.Mock() + self.db_hook.get_connection.return_value = self.connection + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn(self, mock_connect): + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['user'], 'login') + self.assertEqual(kwargs['passwd'], 'password') + self.assertEqual(kwargs['host'], 'host') + self.assertEqual(kwargs['db'], 'schema') + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_port(self, mock_connect): + self.connection.port = 3307 + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['port'], 3307) + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_charset(self, mock_connect): + self.connection.extra = json.dumps({'charset': 'utf-8'}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['charset'], 'utf-8') + self.assertEqual(kwargs['use_unicode'], True) + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_cursor(self, mock_connect): + self.connection.extra = json.dumps({'cursor': 'sscursor'}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['cursorclass'], MySQLdb.cursors.SSCursor) + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_local_infile(self, mock_connect): + self.connection.extra = json.dumps({'local_infile': True}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['local_infile'], 1) + + class TestMySqlHook(unittest.TestCase): def setUp(self): @@ -85,3 +154,20 @@ def test_run_multi_queries(self): self.assertEqual(kwargs, {}) self.cur.execute.assert_called_with(sql[1]) self.conn.commit.assert_not_called() + + def test_bulk_load(self): + self.db_hook.bulk_load('table', '/tmp/file') + self.cur.execute.assert_called_once_with(""" + LOAD DATA LOCAL INFILE '/tmp/file' + INTO TABLE table + """) + + def test_bulk_dump(self): + self.db_hook.bulk_dump('table', '/tmp/file') + self.cur.execute.assert_called_once_with(""" + SELECT * INTO OUTFILE '/tmp/file' + FROM table + """) + + def test_serialize_cell(self): + self.assertEqual('foo', self.db_hook._serialize_cell('foo', None)) From 6cedc453e0ccb41d4c8737c88132da9a7f07cf11 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 28 Oct 2018 19:27:21 +0100 Subject: [PATCH 146/808] [AIRFLOW-3265] Support for "unix_socket" extra for MySQL hook (#4110) MySQL hook does not support "unix_socket" extra - which allows to specify a different location of Linux socket than the default one. This is a blocker for tools like cloud-sql-proxy that creates sockets in an arbitrary place: https://mysqlclient.readthedocs.io/user_guide.html --- airflow/hooks/mysql_hook.py | 2 ++ tests/hooks/test_mysql_hook.py | 9 +++++++++ 2 files changed, 11 insertions(+) diff --git a/airflow/hooks/mysql_hook.py b/airflow/hooks/mysql_hook.py index c02c0f43b5e37..9fe162626bafc 100644 --- a/airflow/hooks/mysql_hook.py +++ b/airflow/hooks/mysql_hook.py @@ -88,6 +88,8 @@ def get_conn(self): local_infile = conn.extra_dejson.get('local_infile', False) if conn.extra_dejson.get('ssl', False): conn_config['ssl'] = conn.extra_dejson['ssl'] + if conn.extra_dejson.get('unix_socket'): + conn_config['unix_socket'] = conn.extra_dejson['unix_socket'] if local_infile: conn_config["local_infile"] = 1 conn = MySQLdb.connect(**conn_config) diff --git a/tests/hooks/test_mysql_hook.py b/tests/hooks/test_mysql_hook.py index 415d430fc4b77..e6c64f7c5ca4f 100644 --- a/tests/hooks/test_mysql_hook.py +++ b/tests/hooks/test_mysql_hook.py @@ -92,6 +92,15 @@ def test_get_conn_local_infile(self, mock_connect): self.assertEqual(args, ()) self.assertEqual(kwargs['local_infile'], 1) + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_con_unix_socket(self, mock_connect): + self.connection.extra = json.dumps({'unix_socket': "/tmp/socket"}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['unix_socket'], '/tmp/socket') + class TestMySqlHook(unittest.TestCase): From 317cbdf72c0f7d3201366e36d4dc98760e69aa6c Mon Sep 17 00:00:00 2001 From: ron819 <40163787+ron819@users.noreply.github.com> Date: Mon, 8 Oct 2018 18:59:59 +0300 Subject: [PATCH 147/808] [AIRFLOW-XXX] Update manage-connections.rst (#4020) Explain how to connect with MySQL --- docs/howto/manage-connections.rst | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/howto/manage-connections.rst b/docs/howto/manage-connections.rst index f869a08b3c061..a79fe0bc30b1e 100644 --- a/docs/howto/manage-connections.rst +++ b/docs/howto/manage-connections.rst @@ -133,3 +133,26 @@ Scopes (comma separated) Scopes are ignored when using application default credentials. See issue `AIRFLOW-2522 `_. +MySQL +~~~~~~~~~~~~~~~~~~~~~ +The MySQL connect type allows to connect with MySQL database. + +Configuring the Connection +'''''''''''''''''''''''''' +Host (required) + The host to connect to. + +Schema (optional) + Specify the schema name to be used in the database. + +Login (required) + Specify the user name to connect. + +Password (required) + Specify the password to connect. + +Extra (optional) + Specify the charset. Example: {"charset": "utf8"} + + .. note:: + If encounter UnicodeDecodeError while working with MySQL connection check the charset defined is matched to the database charset. From f53fd2dcc0a9b1e28d92a73a4a1de3e9d02691a6 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Mon, 5 Nov 2018 17:22:17 +0100 Subject: [PATCH 148/808] [AIRFLOW-3268] Better handling of extras field in MySQL connection (#4113) --- airflow/hooks/mysql_hook.py | 10 ++++- docs/howto/manage-connections.rst | 63 +++++++++++++++++++++++++++++-- tests/hooks/test_mysql_hook.py | 24 ++++++++++++ 3 files changed, 92 insertions(+), 5 deletions(-) diff --git a/airflow/hooks/mysql_hook.py b/airflow/hooks/mysql_hook.py index 9fe162626bafc..57f577c261353 100644 --- a/airflow/hooks/mysql_hook.py +++ b/airflow/hooks/mysql_hook.py @@ -19,6 +19,8 @@ import MySQLdb import MySQLdb.cursors +import json +import six from airflow.hooks.dbapi_hook import DbApiHook @@ -87,7 +89,13 @@ def get_conn(self): conn_config["cursorclass"] = MySQLdb.cursors.SSDictCursor local_infile = conn.extra_dejson.get('local_infile', False) if conn.extra_dejson.get('ssl', False): - conn_config['ssl'] = conn.extra_dejson['ssl'] + # SSL parameter for MySQL has to be a dictionary and in case + # of extra/dejson we can get string if extra is passed via + # URL parameters + dejson_ssl = conn.extra_dejson['ssl'] + if isinstance(dejson_ssl, six.string_types): + dejson_ssl = json.loads(dejson_ssl) + conn_config['ssl'] = dejson_ssl if conn.extra_dejson.get('unix_socket'): conn_config['unix_socket'] = conn.extra_dejson['unix_socket'] if local_infile: diff --git a/docs/howto/manage-connections.rst b/docs/howto/manage-connections.rst index a79fe0bc30b1e..4d0adfb1009dc 100644 --- a/docs/howto/manage-connections.rst +++ b/docs/howto/manage-connections.rst @@ -133,8 +133,9 @@ Scopes (comma separated) Scopes are ignored when using application default credentials. See issue `AIRFLOW-2522 `_. + MySQL -~~~~~~~~~~~~~~~~~~~~~ +~~~~~ The MySQL connect type allows to connect with MySQL database. Configuring the Connection @@ -152,7 +153,61 @@ Password (required) Specify the password to connect. Extra (optional) - Specify the charset. Example: {"charset": "utf8"} - + Specify the extra parameters (as json dictionary) that can be used in mysql + connection. The following parameters are supported: + + * **charset**: specify charset of the connection + * **cursor**: one of "sscursor", "dictcursor, "ssdictcursor" - specifies cursor class to be + used + * **local_infile**: controls MySQL's LOCAL capability (permitting local data loading by + clients). See `MySQLdb docs `_ + for details. + * **unix_socket**: UNIX socket used instead of the default socket + * **ssl**: Dictionary of SSL parameters that control connecting using SSL (those + parameters are server specific and should contain "ca", "cert", "key", "capath", + "cipher" parameters. See + `MySQLdb docs `_ for details. + Note that in order to be useful in URL notation, this parameter might also be + a string where the SSL dictionary is a string-encoded JSON dictionary. + + Example "extras" field: + + .. code-block:: json + + { + "charset": "utf8", + "cursorclass": "sscursor", + "local_infile": true, + "unix_socket": "/var/socket", + "ssl": { + "cert": "/tmp/client-cert.pem", + "ca": "/tmp/server-ca.pem'", + "key": "/tmp/client-key.pem" + } + } + + or + + .. code-block:: json + + { + "charset": "utf8", + "cursorclass": "sscursor", + "local_infile": true, + "unix_socket": "/var/socket", + "ssl": "{\"cert\": \"/tmp/client-cert.pem\", \"ca\": \"/tmp/server-ca.pem\", \"key\": \"/tmp/client-key.pem\"}" + } + + When specifying the connection as URI (in AIRFLOW_CONN_* variable) you should specify it + following the standard syntax of DB connections, where extras as passed as parameters + of the URI (note that all components of the URI should be URL-encoded). + + For example: + + .. code-block:: bash + + mysql://mysql_user:XXXXXXXXXXXX@1.1.1.1:3306/mysqldb?ssl=%7B%22cert%22%3A+%22%2Ftmp%2Fclient-cert.pem%22%2C+%22ca%22%3A+%22%2Ftmp%2Fserver-ca.pem%22%2C+%22key%22%3A+%22%2Ftmp%2Fclient-key.pem%22%7D + .. note:: - If encounter UnicodeDecodeError while working with MySQL connection check the charset defined is matched to the database charset. + If encounter UnicodeDecodeError while working with MySQL connection check + the charset defined is matched to the database charset. diff --git a/tests/hooks/test_mysql_hook.py b/tests/hooks/test_mysql_hook.py index e6c64f7c5ca4f..22e1874ed2f36 100644 --- a/tests/hooks/test_mysql_hook.py +++ b/tests/hooks/test_mysql_hook.py @@ -27,6 +27,12 @@ from airflow import models from airflow.hooks.mysql_hook import MySqlHook +SSL_DICT = { + 'cert': '/tmp/client-cert.pem', + 'ca': '/tmp/server-ca.pem', + 'key': '/tmp/client-key.pem' +} + class TestMySqlHookConn(unittest.TestCase): @@ -101,6 +107,24 @@ def test_get_con_unix_socket(self, mock_connect): self.assertEqual(args, ()) self.assertEqual(kwargs['unix_socket'], '/tmp/socket') + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_ssl_as_dictionary(self, mock_connect): + self.connection.extra = json.dumps({'ssl': SSL_DICT}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['ssl'], SSL_DICT) + + @mock.patch('airflow.hooks.mysql_hook.MySQLdb.connect') + def test_get_conn_ssl_as_string(self, mock_connect): + self.connection.extra = json.dumps({'ssl': json.dumps(SSL_DICT)}) + self.db_hook.get_conn() + mock_connect.assert_called_once() + args, kwargs = mock_connect.call_args + self.assertEqual(args, ()) + self.assertEqual(kwargs['ssl'], SSL_DICT) + class TestMySqlHook(unittest.TestCase): From f3d5a70abccc57fe1f77b83d53d83015574511d5 Mon Sep 17 00:00:00 2001 From: Gwildor Sok Date: Mon, 29 Oct 2018 15:21:37 +0100 Subject: [PATCH 149/808] [AIRFLOW-3112] Fix SFTPHook not validating hosts by default (#4085) --- airflow/contrib/hooks/sftp_hook.py | 16 +++++-- airflow/utils/db.py | 2 +- tests/contrib/hooks/test_sftp_hook.py | 60 ++++++++++++++++++++++++++- 3 files changed, 72 insertions(+), 6 deletions(-) diff --git a/airflow/contrib/hooks/sftp_hook.py b/airflow/contrib/hooks/sftp_hook.py index 3cb32ab8ba93b..c23bd3012ba32 100644 --- a/airflow/contrib/hooks/sftp_hook.py +++ b/airflow/contrib/hooks/sftp_hook.py @@ -49,6 +49,9 @@ def __init__(self, ftp_conn_id='sftp_default', *args, **kwargs): self.conn = None self.private_key_pass = None + # Fail for unverified hosts, unless this is explicitly allowed + self.no_host_key_check = False + if self.ssh_conn_id is not None: conn = self.get_connection(self.ssh_conn_id) if conn.extra is not None: @@ -59,9 +62,7 @@ def __init__(self, ftp_conn_id='sftp_default', *args, **kwargs): # For backward compatibility # TODO: remove in Airflow 2.1 import warnings - if 'ignore_hostkey_verification' in extra_options \ - and str(extra_options["ignore_hostkey_verification"])\ - .lower() == 'false': + if 'ignore_hostkey_verification' in extra_options: warnings.warn( 'Extra option `ignore_hostkey_verification` is deprecated.' 'Please use `no_host_key_check` instead.' @@ -69,7 +70,14 @@ def __init__(self, ftp_conn_id='sftp_default', *args, **kwargs): DeprecationWarning, stacklevel=2, ) - self.no_host_key_check = False + self.no_host_key_check = str( + extra_options['ignore_hostkey_verification'] + ).lower() == 'true' + + if 'no_host_key_check' in extra_options: + self.no_host_key_check = str( + extra_options['no_host_key_check']).lower() == 'true' + if 'private_key' in extra_options: warnings.warn( 'Extra option `private_key` is deprecated.' diff --git a/airflow/utils/db.py b/airflow/utils/db.py index c09aea4eef602..ef2dcf181762d 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -186,7 +186,7 @@ def initdb(rbac=False): conn_id='sftp_default', conn_type='sftp', host='localhost', port=22, login='travis', extra=''' - {"private_key": "~/.ssh/id_rsa", "ignore_hostkey_verification": true} + {"key_file": "~/.ssh/id_rsa", "no_host_key_check": true} ''')) merge_conn( models.Connection( diff --git a/tests/contrib/hooks/test_sftp_hook.py b/tests/contrib/hooks/test_sftp_hook.py index 105b6b81d8478..7715161adb08a 100644 --- a/tests/contrib/hooks/test_sftp_hook.py +++ b/tests/contrib/hooks/test_sftp_hook.py @@ -19,12 +19,13 @@ from __future__ import print_function +import mock import unittest import shutil import os import pysftp -from airflow import configuration +from airflow import configuration, models from airflow.contrib.hooks.sftp_hook import SFTPHook TMP_PATH = '/tmp' @@ -105,6 +106,63 @@ def test_get_mod_time(self): TMP_PATH, TMP_DIR_FOR_TESTS, TMP_FILE_FOR_TESTS)) self.assertEqual(len(output), 14) + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_default(self, get_connection): + connection = models.Connection(login='login', host='host') + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, False) + + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_enabled(self, get_connection): + connection = models.Connection( + login='login', host='host', + extra='{"no_host_key_check": true}') + + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, True) + + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_disabled(self, get_connection): + connection = models.Connection( + login='login', host='host', + extra='{"no_host_key_check": false}') + + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, False) + + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_disabled_for_all_but_true(self, get_connection): + connection = models.Connection( + login='login', host='host', + extra='{"no_host_key_check": "foo"}') + + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, False) + + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_ignore(self, get_connection): + connection = models.Connection( + login='login', host='host', + extra='{"ignore_hostkey_verification": true}') + + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, True) + + @mock.patch('airflow.contrib.hooks.sftp_hook.SFTPHook.get_connection') + def test_no_host_key_check_no_ignore(self, get_connection): + connection = models.Connection( + login='login', host='host', + extra='{"ignore_hostkey_verification": false}') + + get_connection.return_value = connection + hook = SFTPHook() + self.assertEqual(hook.no_host_key_check, False) + def tearDown(self): shutil.rmtree(os.path.join(TMP_PATH, TMP_DIR_FOR_TESTS)) os.remove(os.path.join(TMP_PATH, TMP_FILE_FOR_TESTS)) From d5be5e8d2c8908417460114fe05c23b26d4997ec Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 5 Nov 2018 10:55:31 -0800 Subject: [PATCH 150/808] [AIRFLOW-2192] Allow non-latin1 usernames with MySQL backend by adding a SQL_ENGINE_ENCODING param and default to UTF-8 (#4087) Compromised of: Since we have unicode_literals importred and the engine arguments must be strings in Python2 explicitly make 'utf-8' a string. replace bare exception with conf.AirflowConfigException for missing value. It's just got for strings apparently. Add utf-8 to default_airflow.cfg - question do I still need the try try/except block or can we depend on defaults (I note some have both). Get rid of try/except block and depend on default_airflow.cfg Use __str__ since calling str just gives us back a newstr as well. Test that a panda user can be saved. --- airflow/config_templates/default_airflow.cfg | 3 +++ airflow/settings.py | 12 +++++++++++- tests/core.py | 5 +++++ 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 33257261c3aad..3d0b2444ed505 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -87,6 +87,9 @@ executor = SequentialExecutor # their website sql_alchemy_conn = sqlite:///{AIRFLOW_HOME}/airflow.db +# The encoding for the databases +sql_engine_encoding = utf-8 + # If SqlAlchemy should pool database connections. sql_alchemy_pool_enabled = True diff --git a/airflow/settings.py b/airflow/settings.py index 9646f035a26be..26d0ba983f09b 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -153,7 +153,7 @@ def configure_orm(disable_connection_pool=False): engine_args['poolclass'] = NullPool log.debug("settings.configure_orm(): Using NullPool") elif 'sqlite' not in SQL_ALCHEMY_CONN: - # Engine args not supported by sqlite. + # Pool size engine args not supported by sqlite. # If no config value is defined for the pool size, select a reasonable value. # 0 means no limit, which could lead to exceeding the Database connection limit. try: @@ -175,6 +175,16 @@ def configure_orm(disable_connection_pool=False): engine_args['pool_size'] = pool_size engine_args['pool_recycle'] = pool_recycle + try: + # Allow the user to specify an encoding for their DB otherwise default + # to utf-8 so jobs & users with non-latin1 characters can still use + # us. + engine_args['encoding'] = conf.get('core', 'SQL_ENGINE_ENCODING') + except conf.AirflowConfigException: + engine_args['encoding'] = 'utf-8' + # For Python2 we get back a newstr and need a str + engine_args['encoding'] = engine_args['encoding'].__str__() + engine = create_engine(SQL_ALCHEMY_CONN, **engine_args) reconnect_timeout = conf.getint('core', 'SQL_ALCHEMY_RECONNECT_TIMEOUT') setup_event_handlers(engine, reconnect_timeout) diff --git a/tests/core.py b/tests/core.py index 0448058b3b3fb..6585d1742ca36 100644 --- a/tests/core.py +++ b/tests/core.py @@ -1930,6 +1930,11 @@ def test_password_user_authenticate(self): self.password_user.password = "secure_password" self.assertTrue(self.password_user.authenticate("secure_password")) + def test_password_unicode_user_authenticate(self): + self.password_user.username = u"🐼" # This is a panda + self.password_user.password = "secure_password" + self.assertTrue(self.password_user.authenticate("secure_password")) + def test_password_authenticate_session(self): from airflow.contrib.auth.backends.password_auth import PasswordUser self.password_user.password = 'test_password' From a632dade29b63467ed14852a3223195674933440 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 5 Nov 2018 19:36:48 +0000 Subject: [PATCH 151/808] [AIRFLOW-3262] Update SimpleHttpOpTests to check Example.com (#4135) Update SimpleHttpOpTests to check Example.com instead of Google.com so that the ip doesn't get banned. --- tests/operators/test_http_operator.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/operators/test_http_operator.py b/tests/operators/test_http_operator.py index 6ab2c03bb5f47..5e87d06f28c80 100644 --- a/tests/operators/test_http_operator.py +++ b/tests/operators/test_http_operator.py @@ -42,22 +42,21 @@ def __eq__(self, other): class SimpleHttpOpTests(unittest.TestCase): def setUp(self): - # Creating a local Http connection to Airflow Webserver - os.environ['AIRFLOW_CONN_HTTP_GOOGLE'] = 'http://www.google.com' + os.environ['AIRFLOW_CONN_HTTP_EXAMPLE'] = 'http://www.example.com' def test_response_in_logs(self): """ - Test that when using SimpleHttpOperator with 'GET' on localhost:8080, - the log contains 'Google' in it + Test that when using SimpleHttpOperator with 'GET', + the log contains 'Example Domain' in it """ operator = SimpleHttpOperator( task_id='test_HTTP_op', method='GET', endpoint='/', - http_conn_id='HTTP_GOOGLE', + http_conn_id='HTTP_EXAMPLE', log_response=True, ) with patch.object(operator.log, 'info') as mock_info: operator.execute(None) - mock_info.assert_called_with(AnyStringWith('Google')) + mock_info.assert_called_with(AnyStringWith('Example Domain')) From 40649867abe9775c5000d441a9a174df59102a9d Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 6 Nov 2018 14:54:26 +0000 Subject: [PATCH 152/808] [AIRFLOW-XXX] Use mocking in SimpleHttpOperator tests (#4144) This changes the tests in [AIRFLOW-3262]/(#4135) to use requests_mock rather than making actual HTTP requests as we have had this test fail on Travis with connection refused. --- tests/operators/test_http_operator.py | 25 +++++++++---------------- 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/tests/operators/test_http_operator.py b/tests/operators/test_http_operator.py index 5e87d06f28c80..4252867669dd1 100644 --- a/tests/operators/test_http_operator.py +++ b/tests/operators/test_http_operator.py @@ -20,35 +20,28 @@ import os import unittest -from mock import patch +import requests +import requests_mock from airflow.operators.http_operator import SimpleHttpOperator try: from unittest import mock except ImportError: - try: - import mock - except ImportError: - mock = None - - -class AnyStringWith(str): - """ - Helper class to check if a substring is a part of a string - """ - def __eq__(self, other): - return self in other + import mock class SimpleHttpOpTests(unittest.TestCase): def setUp(self): os.environ['AIRFLOW_CONN_HTTP_EXAMPLE'] = 'http://www.example.com' - def test_response_in_logs(self): + @requests_mock.mock() + def test_response_in_logs(self, m): """ Test that when using SimpleHttpOperator with 'GET', the log contains 'Example Domain' in it """ + + m.get('http://www.example.com', text='Example.com fake response') operator = SimpleHttpOperator( task_id='test_HTTP_op', method='GET', @@ -57,6 +50,6 @@ def test_response_in_logs(self): log_response=True, ) - with patch.object(operator.log, 'info') as mock_info: + with mock.patch.object(operator.log, 'info') as mock_info: operator.execute(None) - mock_info.assert_called_with(AnyStringWith('Example Domain')) + mock_info.assert_called_with('Example.com fake response') From ec26c13c2d0f8f5f6ec115e75d4b71f093ec7d7b Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Sat, 3 Nov 2018 15:36:25 +0800 Subject: [PATCH 153/808] [AIRFLOW-3295] Fix potential security issue in DaskExecutor (#4128) When user decides to use TLS/SSL encryption for DaskExecutor communications, `Distributed.Security` object will be created. However, argument `require_encryption` is missed to be set to `True` (its default value is `False`). This may fail the TLS/SSL encryption setting-up. --- airflow/executors/dask_executor.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py index a6ba677f8bd7c..afdf3ad2d198f 100644 --- a/airflow/executors/dask_executor.py +++ b/airflow/executors/dask_executor.py @@ -49,6 +49,7 @@ def start(self): tls_client_key=self.tls_key, tls_client_cert=self.tls_cert, tls_ca_file=self.tls_ca, + require_encryption=True, ) else: security = None From 04f5745fa490c8f618373f925b9811fa83697d20 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 6 Nov 2018 15:33:32 +0000 Subject: [PATCH 154/808] [AIRFLOW-XXX] Fix flake8 errors from #4144 --- tests/operators/test_http_operator.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/operators/test_http_operator.py b/tests/operators/test_http_operator.py index 4252867669dd1..e6033155e9b46 100644 --- a/tests/operators/test_http_operator.py +++ b/tests/operators/test_http_operator.py @@ -20,7 +20,6 @@ import os import unittest -import requests import requests_mock from airflow.operators.http_operator import SimpleHttpOperator From 0990b84653e2d85a14a25ecee25a9a1bbbb74170 Mon Sep 17 00:00:00 2001 From: Eric Chang Date: Thu, 4 Oct 2018 17:36:40 -0700 Subject: [PATCH 155/808] [AIRFLOW-3161] fix TaskInstance log link in RBAC UI --- airflow/models.py | 8 ++++---- tests/models.py | 14 ++++++++++++++ 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 93114cce2f438..a4a6a165cf91a 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1121,10 +1121,10 @@ def log_url(self): BASE_URL = configuration.conf.get('webserver', 'BASE_URL') if settings.RBAC: return BASE_URL + ( - "/log/list/" - "?_flt_3_dag_id={self.dag_id}" - "&_flt_3_task_id={self.task_id}" - "&_flt_3_execution_date={iso}" + "/log?" + "execution_date={iso}" + "&task_id={self.task_id}" + "&dag_id={self.dag_id}" ).format(**locals()) else: return BASE_URL + ( diff --git a/tests/models.py b/tests/models.py index 16a8d6a3e1b76..aee32c1646a67 100644 --- a/tests/models.py +++ b/tests/models.py @@ -2369,6 +2369,20 @@ def test_log_url(self): self.assertEqual(d['task_id'][0], 'op') self.assertEqual(pendulum.parse(d['execution_date'][0]), now) + @patch('airflow.settings.RBAC', True) + def test_log_url_rbac(self): + dag = DAG('dag', start_date=DEFAULT_DATE) + task = DummyOperator(task_id='op', dag=dag) + ti = TI(task=task, execution_date=datetime.datetime(2018, 1, 1)) + + expected_url = ( + 'http://localhost:8080/log?' + 'execution_date=2018-01-01T00%3A00%3A00%2B00%3A00' + '&task_id=op' + '&dag_id=dag' + ) + self.assertEqual(ti.log_url, expected_url) + def test_mark_success_url(self): now = pendulum.now('Europe/Brussels') dag = DAG('dag', start_date=DEFAULT_DATE) From 3e06f035955884d382cd9f36b558fc365c1cd4e4 Mon Sep 17 00:00:00 2001 From: Scott Kruger Date: Tue, 16 Oct 2018 04:16:28 -0500 Subject: [PATCH 156/808] [AIRFLOW-843] Exception availabe in context for on_failure_callback (#2135) Store exceptions encountered executing a task in the context dict, making it available for on_failure_callback handlers. --- airflow/models.py | 3 +++ tests/core.py | 20 ++++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/airflow/models.py b/airflow/models.py index a4a6a165cf91a..f9e3c6a7e96c2 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1759,6 +1759,9 @@ def handle_failure(self, error, test_mode=False, context=None, session=None): # Log failure duration session.add(TaskFail(task, self.execution_date, self.start_date, self.end_date)) + if context is not None: + context['exception'] = error + # Let's go deeper try: # Since this function is called only when the TI state is running, diff --git a/tests/core.py b/tests/core.py index 6585d1742ca36..f466c4c4f94db 100644 --- a/tests/core.py +++ b/tests/core.py @@ -442,6 +442,26 @@ def test_bash_operator_kill(self): os.kill(pid, signal.SIGTERM) self.fail("BashOperator's subprocess still running after stopping on timeout!") + def test_on_failure_callback(self): + # Annoying workaround for nonlocal not existing in python 2 + data = {'called': False} + + def check_failure(context, test_case=self): + data['called'] = True + error = context.get('exception') + test_case.assertIsInstance(error, AirflowException) + + t = BashOperator( + task_id='check_on_failure_callback', + bash_command="exit 1", + dag=self.dag, + on_failure_callback=check_failure) + self.assertRaises( + exceptions.AirflowException, + t.run, + start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) + self.assertTrue(data['called']) + def test_trigger_dagrun(self): def trigga(context, obj): if True: From aa9244ff21c64b4443aea2ae3b6f65598f92ff78 Mon Sep 17 00:00:00 2001 From: Camilo Aguilar Date: Fri, 6 Jul 2018 22:14:20 +0200 Subject: [PATCH 157/808] [AIRFLOW-1163][AIRFLOW-XXX] Add support for x-forwarded-* headers When running Airflow behind a L7 proxy that sends x-fowarded-* headers, the Flask app miscontructs redirect URIs. Closes #3580 from c4milo/master --- airflow/www/app.py | 6 ++++-- airflow/www_rbac/app.py | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/airflow/www/app.py b/airflow/www/app.py index e9b101db37689..6eea5d2d0ffd5 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -25,6 +25,7 @@ from flask_wtf.csrf import CSRFProtect from six.moves.urllib.parse import urlparse from werkzeug.wsgi import DispatcherMiddleware +from werkzeug.contrib.fixers import ProxyFix import airflow from airflow import configuration as conf @@ -43,6 +44,7 @@ def create_app(config=None, testing=False): app = Flask(__name__) + app.wsgi_app = ProxyFix(app.wsgi_app) app.secret_key = configuration.conf.get('webserver', 'SECRET_KEY') app.config['LOGIN_DISABLED'] = not configuration.conf.getboolean( 'webserver', 'AUTHENTICATE') diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index 85125c6eb06b8..1004764459e39 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -26,6 +26,7 @@ from flask_wtf.csrf import CSRFProtect from six.moves.urllib.parse import urlparse from werkzeug.wsgi import DispatcherMiddleware +from werkzeug.contrib.fixers import ProxyFix from airflow import settings from airflow import configuration as conf @@ -40,6 +41,7 @@ def create_app(config=None, testing=False, app_name="Airflow"): global app, appbuilder app = Flask(__name__) + app.wsgi_app = ProxyFix(app.wsgi_app) app.secret_key = conf.get('webserver', 'SECRET_KEY') airflow_home_path = conf.get('core', 'AIRFLOW_HOME') From c0f6cd1231df4d68bd0d6012791446ffb58e8d85 Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Tue, 2 Oct 2018 11:16:55 -0400 Subject: [PATCH 158/808] [AIRFLOW-3137] Make ProxyFix middleware optional. (#3983) The ProxyFix middleware should only be used when airflow is running behind a trusted proxy. This patch adds a `USE_PROXY_FIX` flag that defaults to `False`. --- airflow/config_templates/default_airflow.cfg | 3 +++ airflow/www/app.py | 3 ++- airflow/www_rbac/app.py | 3 ++- docs/integration.rst | 9 +++++++++ 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 3d0b2444ed505..1a3768ff74603 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -312,6 +312,9 @@ navbar_color = #007A87 # Default dagrun to show in UI default_dag_run_display_number = 25 +# Enable werkzeug `ProxyFix` middleware +enable_proxy_fix = False + [email] email_backend = airflow.utils.email.send_email_smtp diff --git a/airflow/www/app.py b/airflow/www/app.py index 6eea5d2d0ffd5..1140be9aa06e3 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -44,7 +44,8 @@ def create_app(config=None, testing=False): app = Flask(__name__) - app.wsgi_app = ProxyFix(app.wsgi_app) + if configuration.conf.getboolean('webserver', 'ENABLE_PROXY_FIX'): + app.wsgi_app = ProxyFix(app.wsgi_app) app.secret_key = configuration.conf.get('webserver', 'SECRET_KEY') app.config['LOGIN_DISABLED'] = not configuration.conf.getboolean( 'webserver', 'AUTHENTICATE') diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index 1004764459e39..bdf500d590add 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -41,7 +41,8 @@ def create_app(config=None, testing=False, app_name="Airflow"): global app, appbuilder app = Flask(__name__) - app.wsgi_app = ProxyFix(app.wsgi_app) + if conf.getboolean('webserver', 'ENABLE_PROXY_FIX'): + app.wsgi_app = ProxyFix(app.wsgi_app) app.secret_key = conf.get('webserver', 'SECRET_KEY') airflow_home_path = conf.get('core', 'AIRFLOW_HOME') diff --git a/docs/integration.rst b/docs/integration.rst index 49c383541f191..2d9bc84c35928 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -64,6 +64,15 @@ Your reverse proxy (ex: nginx) should be configured as follow: } } +To ensure that Airflow generates URLs with the correct scheme when +running behind a TLS-terminating proxy, you should configure the proxy +to set the `X-Forwarded-Proto` header, and enable the `ProxyFix` +middleware in your `airflow.cfg`:: + + enable_proxy_fix = True + +Note: you should only enable the `ProxyFix` middleware when running +Airflow behind a trusted proxy (AWS ELB, nginx, etc.). .. _Azure: From e68544e42679c2ee04c38f66d124eaa1632bfdcf Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 25 Oct 2018 06:33:21 -0400 Subject: [PATCH 159/808] [AIRFLOW-1508] Add SKIPPED to task states. (#4059) --- airflow/models.py | 2 +- airflow/utils/state.py | 9 +++------ airflow/www/views.py | 20 ++++++-------------- airflow/www_rbac/views.py | 20 ++++++-------------- 4 files changed, 16 insertions(+), 35 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index f9e3c6a7e96c2..f5052231c7d24 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -4852,7 +4852,7 @@ def create(dag_id, session=None): """ # unfortunately sqlalchemy does not know upsert qry = session.query(DagStat).filter(DagStat.dag_id == dag_id).all() - states = [dag_stat.state for dag_stat in qry] + states = {dag_stat.state for dag_stat in qry} for state in State.dag_states: if state not in states: try: diff --git a/airflow/utils/state.py b/airflow/utils/state.py index a351df07b9654..b73aaa12a087a 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -51,6 +51,7 @@ class State(object): RUNNING, FAILED, UPSTREAM_FAILED, + SKIPPED, UP_FOR_RETRY, QUEUED, NONE, @@ -79,18 +80,14 @@ class State(object): @classmethod def color(cls, state): - if state in cls.state_color: - return cls.state_color[state] - else: - return 'white' + return cls.state_color.get(state, 'white') @classmethod def color_fg(cls, state): color = cls.color(state) if color in ['green', 'red']: return 'white' - else: - return 'black' + return 'black' @classmethod def finished(cls): diff --git a/airflow/www/views.py b/airflow/www/views.py index 40fc2d6838406..40bb8e577e88e 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -579,17 +579,13 @@ def dag_stats(self, session=None): for dag in dagbag.dags.values(): payload[dag.safe_dag_id] = [] for state in State.dag_states: - try: - count = data[dag.dag_id][state] - except Exception: - count = 0 - d = { + count = data.get(dag.dag_id, {}).get(state, 0) + payload[dag.safe_dag_id].append({ 'state': state, 'count': count, 'dag_id': dag.dag_id, 'color': State.color(state) - } - payload[dag.safe_dag_id].append(d) + }) return wwwutils.json_response(payload) @expose('/task_stats') @@ -650,17 +646,13 @@ def task_stats(self, session=None): for dag in dagbag.dags.values(): payload[dag.safe_dag_id] = [] for state in State.task_states: - try: - count = data[dag.dag_id][state] - except Exception: - count = 0 - d = { + count = data.get(dag.dag_id, {}).get(state, 0) + payload[dag.safe_dag_id].append({ 'state': state, 'count': count, 'dag_id': dag.dag_id, 'color': State.color(state) - } - payload[dag.safe_dag_id].append(d) + }) return wwwutils.json_response(payload) @expose('/code') diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 69e2079059889..199904ce5419a 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -288,17 +288,13 @@ def dag_stats(self, session=None): for dag in dagbag.dags.values(): payload[dag.safe_dag_id] = [] for state in State.dag_states: - try: - count = data[dag.dag_id][state] - except Exception: - count = 0 - d = { + count = data.get(dag.dag_id, {}).get(state, 0) + payload[dag.safe_dag_id].append({ 'state': state, 'count': count, 'dag_id': dag.dag_id, 'color': State.color(state) - } - payload[dag.safe_dag_id].append(d) + }) return wwwutils.json_response(payload) @expose('/task_stats') @@ -359,17 +355,13 @@ def task_stats(self, session=None): for dag in dagbag.dags.values(): payload[dag.safe_dag_id] = [] for state in State.task_states: - try: - count = data[dag.dag_id][state] - except Exception: - count = 0 - d = { + count = data.get(dag.dag_id, {}).get(state, 0) + payload[dag.safe_dag_id].append({ 'state': state, 'count': count, 'dag_id': dag.dag_id, 'color': State.color(state) - } - payload[dag.safe_dag_id].append(d) + }) return wwwutils.json_response(payload) @expose('/code') From a59bbbffcb19b429ac03bff5618f2de5ebc8726b Mon Sep 17 00:00:00 2001 From: Jarek Date: Fri, 26 Oct 2018 14:29:00 +0200 Subject: [PATCH 160/808] [AIRFLOW-1970] Let empty Fernet key or special `no encryption` phrase. (#4038) Once the user has installed Fernet package then the application enforces setting valid Fernet key. This change will alter this behavior into letting empty Fernet key or special `no encryption` phrase and interpreting those two cases as no encryption desirable. --- airflow/configuration.py | 9 +++------ airflow/models.py | 22 ++++++++++++++++------ docs/howto/secure-connections.rst | 9 +++++---- tests/models.py | 1 - 4 files changed, 24 insertions(+), 17 deletions(-) diff --git a/airflow/configuration.py b/airflow/configuration.py index b92e93f3feaa5..1caea93a6cbbb 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -56,12 +56,9 @@ def generate_fernet_key(): try: from cryptography.fernet import Fernet except ImportError: - pass - try: - key = Fernet.generate_key().decode() - except NameError: - key = "cryptography_not_found_storing_passwords_in_plain_text" - return key + return '' + else: + return Fernet.generate_key().decode() def expand_env_var(env_var): diff --git a/airflow/models.py b/airflow/models.py index f5052231c7d24..d9c63adb09d10 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -144,6 +144,8 @@ def get_fernet(): :raises: AirflowException if there's a problem trying to load Fernet """ global _fernet + log = LoggingMixin().log + if _fernet: return _fernet try: @@ -152,19 +154,27 @@ def get_fernet(): InvalidFernetToken = InvalidToken except BuiltinImportError: - LoggingMixin().log.warn("cryptography not found - values will not be stored " - "encrypted.", - exc_info=1) + log.warning( + "cryptography not found - values will not be stored encrypted." + ) _fernet = NullFernet() return _fernet try: - _fernet = Fernet(configuration.conf.get('core', 'FERNET_KEY').encode('utf-8')) - _fernet.is_encrypted = True - return _fernet + fernet_key = configuration.conf.get('core', 'FERNET_KEY') + if not fernet_key: + log.warning( + "empty cryptography key - values will not be stored encrypted." + ) + _fernet = NullFernet() + else: + _fernet = Fernet(fernet_key.encode('utf-8')) + _fernet.is_encrypted = True except (ValueError, TypeError) as ve: raise AirflowException("Could not create Fernet object: {}".format(ve)) + return _fernet + # Used by DAG context_managers _CONTEXT_MANAGER_DAG = None diff --git a/docs/howto/secure-connections.rst b/docs/howto/secure-connections.rst index bb13b1bb08bba..b3b9ba193d295 100644 --- a/docs/howto/secure-connections.rst +++ b/docs/howto/secure-connections.rst @@ -4,13 +4,14 @@ Securing Connections By default, Airflow will save the passwords for the connection in plain text within the metadata database. The ``crypto`` package is highly recommended during installation. The ``crypto`` package does require that your operating -system have libffi-dev installed. +system has ``libffi-dev`` installed. -If ``crypto`` package was not installed initially, you can still enable encryption for -connections by following steps below: +If ``crypto`` package was not installed initially, it means that your Fernet key in ``airflow.cfg`` is empty. + +You can still enable encryption for passwords within connections by following below steps: 1. Install crypto package ``pip install apache-airflow[crypto]`` -2. Generate fernet_key, using this code snippet below. fernet_key must be a base64-encoded 32-byte key. +2. Generate fernet_key, using this code snippet below. ``fernet_key`` must be a base64-encoded 32-byte key. .. code:: python diff --git a/tests/models.py b/tests/models.py index aee32c1646a67..51b8363454054 100644 --- a/tests/models.py +++ b/tests/models.py @@ -2742,7 +2742,6 @@ def test_connection_extra_no_encryption(self, mock_get): is set to a non-base64-encoded string and the extra is stored without encryption. """ - mock_get.return_value = 'cryptography_not_found_storing_passwords_in_plain_text' test_connection = Connection(extra='testextra') self.assertEqual(test_connection.extra, 'testextra') From 50cdcdbeda0abd3bbe4c877e28ad30e86ad93093 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 1 Oct 2018 21:44:27 +0100 Subject: [PATCH 161/808] [AIRFLOW-2574] Cope with '%' in SQLA DSN when running migrations (#3787) Alembic uses a ConfigParser like Airflow does, and "%% is a special value in there, so we need to escape it. As per the Alembic docs: > Note that this value is passed to ConfigParser.set, which supports > variable interpolation using pyformat (e.g. `%(some_value)s`). A raw > percent sign not part of an interpolation symbol must therefore be > escaped, e.g. `%%` --- airflow/utils/db.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index ef2dcf181762d..4697605815682 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -341,8 +341,8 @@ def upgradedb(): package_dir = os.path.normpath(os.path.join(current_dir, '..')) directory = os.path.join(package_dir, 'migrations') config = Config(os.path.join(package_dir, 'alembic.ini')) - config.set_main_option('script_location', directory) - config.set_main_option('sqlalchemy.url', settings.SQL_ALCHEMY_CONN) + config.set_main_option('script_location', directory.replace('%', '%%')) + config.set_main_option('sqlalchemy.url', settings.SQL_ALCHEMY_CONN.replace('%', '%%')) command.upgrade(config, 'heads') From e207eb0e25f12860029c8f74a9d512c317c17ac8 Mon Sep 17 00:00:00 2001 From: Olivier Morissette Date: Tue, 23 Oct 2018 12:21:30 -0400 Subject: [PATCH 162/808] [AIRFLOW-2744] Allow RBAC to accept plugins for views and links. (#4036) Airflow Users that wish to create plugins for the new www_rbac UI can not add plugin views or links. This PR fixes that by letting a user specify their plugins for www_rbac and maintains backwards compatibility with the existing plugins system. --- airflow/plugins_manager.py | 6 ++++++ airflow/www_rbac/app.py | 20 ++++++++++++++++++ docs/plugins.rst | 35 ++++++++++++++++++++++++++++-- tests/plugins/test_plugin.py | 30 ++++++++++++++++++++++++-- tests/plugins_manager.py | 41 ++++++++++++++++++++++++++++++++++++ 5 files changed, 128 insertions(+), 4 deletions(-) diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py index ad630701dec90..c589e9b5a91ea 100644 --- a/airflow/plugins_manager.py +++ b/airflow/plugins_manager.py @@ -49,6 +49,8 @@ class AirflowPlugin(object): admin_views = [] flask_blueprints = [] menu_links = [] + appbuilder_views = [] + appbuilder_menu_items = [] @classmethod def validate(cls): @@ -120,6 +122,8 @@ def make_module(name, objects): admin_views = [] flask_blueprints = [] menu_links = [] +flask_appbuilder_views = [] +flask_appbuilder_menu_links = [] for p in plugins: operators_modules.append( @@ -135,3 +139,5 @@ def make_module(name, objects): admin_views.extend(p.admin_views) flask_blueprints.extend(p.flask_blueprints) menu_links.extend(p.menu_links) + flask_appbuilder_views.extend(p.appbuilder_views) + flask_appbuilder_menu_links.extend(p.appbuilder_menu_items) diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index bdf500d590add..82f1b37c7c423 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -17,6 +17,7 @@ # specific language governing permissions and limitations # under the License. # +import logging import socket import six @@ -37,6 +38,7 @@ appbuilder = None csrf = CSRFProtect() +log = logging.getLogger(__name__) def create_app(config=None, testing=False, app_name="Airflow"): global app, appbuilder @@ -123,6 +125,24 @@ def init_views(appbuilder): category='About', category_icon='fa-th') + def integrate_plugins(): + """Integrate plugins to the context""" + from airflow.plugins_manager import ( + flask_appbuilder_views, flask_appbuilder_menu_links) + + for v in flask_appbuilder_views: + log.debug("Adding view %s", v["name"]) + appbuilder.add_view(v["view"], + v["name"], + category=v["category"]) + for ml in sorted(flask_appbuilder_menu_links, key=lambda x: x["name"]): + log.debug("Adding menu link %s", ml["name"]) + appbuilder.add_link(ml["name"], + href=ml["href"], + category=ml["category"], + category_icon=ml["category_icon"]) + + integrate_plugins() # Garbage collect old permissions/views after they have been modified. # Otherwise, when the name of a view or menu is changed, the framework # will add the new Views and Menus names to the backend, but will not diff --git a/docs/plugins.rst b/docs/plugins.rst index a21f1007ff686..8173e632e5f3f 100644 --- a/docs/plugins.rst +++ b/docs/plugins.rst @@ -72,10 +72,15 @@ looks like: # A list of objects created from a class derived # from flask_admin.BaseView admin_views = [] - # A list of Blueprint object created from flask.Blueprint + # A list of Blueprint object created from flask.Blueprint. For use with the flask_admin based GUI flask_blueprints = [] - # A list of menu links (flask_admin.base.MenuLink) + # A list of menu links (flask_admin.base.MenuLink). For use with the flask_admin based GUI menu_links = [] + # A list of dictionaries containing FlaskAppBuilder BaseView object and some metadata. See example below + appbuilder_views = [] + # A list of dictionaries containing FlaskAppBuilder BaseView object and some metadata. See example below + appbuilder_menu_items = [] + Example @@ -139,6 +144,22 @@ definitions in Airflow. name='Test Menu Link', url='https://airflow.incubator.apache.org/') + # Creating a flask appbuilder BaseView + class TestAppBuilderBaseView(AppBuilderBaseView): + @expose("/") + def test(self): + return self.render("test_plugin/test.html", content="Hello galaxy!") + v_appbuilder_view = TestAppBuilderBaseView() + v_appbuilder_package = {"name": "Test View", + "category": "Test Plugin", + "view": v_appbuilder_view} + + # Creating a flask appbuilder Menu Item + appbuilder_mitem = {"name": "Google", + "category": "Search", + "category_icon": "fa-th", + "href": "https://www.google.com"} + # Defining the plugin class class AirflowTestPlugin(AirflowPlugin): name = "test_plugin" @@ -150,3 +171,13 @@ definitions in Airflow. admin_views = [v] flask_blueprints = [bp] menu_links = [ml] + appbuilder_views = [v_appbuilder_package] + appbuilder_menu_items = [appbuilder_mitem] + + +Note on role based views +------------------------ + +Airflow 1.10 introduced role based views using FlaskAppBuilder. You can configure which UI is used by setting +rbac = True. To support plugin views and links for both versions of the UI and maintain backwards compatibility, +the fields appbuilder_views and appbuilder_menu_items were added to the AirflowTestPlugin class. diff --git a/tests/plugins/test_plugin.py b/tests/plugins/test_plugin.py index d4c7f27b1c573..0770da120bc4c 100644 --- a/tests/plugins/test_plugin.py +++ b/tests/plugins/test_plugin.py @@ -23,6 +23,7 @@ from flask import Blueprint from flask_admin import BaseView, expose from flask_admin.base import MenuLink +from flask_appbuilder import BaseView as AppBuilderBaseView # Importing base classes that we need to derive from airflow.hooks.base_hook import BaseHook @@ -60,6 +61,28 @@ def test(self): return self.render("test_plugin/test.html", content="Hello galaxy!") v = TestView(category="Test Plugin", name="Test View") + +# Creating a flask appbuilder BaseView +class TestAppBuilderBaseView(AppBuilderBaseView): + default_view = "test" + + @expose("/") + def test(self): + return self.render("test_plugin/test.html", content="Hello galaxy!") + + +v_appbuilder_view = TestAppBuilderBaseView() +v_appbuilder_package = {"name": "Test View", + "category": "Test Plugin", + "view": v_appbuilder_view} + +# Creating a flask appbuilder Menu Item +appbuilder_mitem = {"name": "Google", + "category": "Search", + "category_icon": "fa-th", + "href": "https://www.google.com"} + + # Creating a flask blueprint to intergrate the templates and static folder bp = Blueprint( "test_plugin", __name__, @@ -67,10 +90,11 @@ def test(self): static_folder='static', static_url_path='/static/test_plugin') + ml = MenuLink( category='Test Plugin', - name='Test Menu Link', - url='https://airflow.incubator.apache.org/') + name="Test Menu Link", + url="https://airflow.incubator.apache.org/") # Defining the plugin class @@ -84,3 +108,5 @@ class AirflowTestPlugin(AirflowPlugin): admin_views = [v] flask_blueprints = [bp] menu_links = [ml] + appbuilder_views = [v_appbuilder_package] + appbuilder_menu_items = [appbuilder_mitem] diff --git a/tests/plugins_manager.py b/tests/plugins_manager.py index f9d093f16284e..0013d9f641f3c 100644 --- a/tests/plugins_manager.py +++ b/tests/plugins_manager.py @@ -30,11 +30,13 @@ from flask_admin import BaseView from flask_admin.menu import MenuLink, MenuView +from airflow.configuration import conf from airflow.hooks.base_hook import BaseHook from airflow.models import BaseOperator from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.executors.base_executor import BaseExecutor from airflow.www.app import cached_app +from airflow.www_rbac import app as application class PluginsTest(unittest.TestCase): @@ -86,3 +88,42 @@ def test_menu_links(self): [menu_link] = [ml for ml in category.get_children() if isinstance(ml, MenuLink)] self.assertEqual('Test Menu Link', menu_link.name) + + +class PluginsTestRBAC(unittest.TestCase): + def setUp(self): + conf.load_test_config() + self.app, self.appbuilder = application.create_app(testing=True) + + def test_flaskappbuilder_views(self): + from tests.plugins.test_plugin import v_appbuilder_package + appbuilder_class_name = str(v_appbuilder_package['view'].__class__.__name__) + plugin_views = [view for view in self.appbuilder.baseviews + if view.blueprint.name == appbuilder_class_name] + + self.assertTrue(len(plugin_views) == 1) + + # view should have a menu item matching category of v_appbuilder_package + links = [menu_item for menu_item in self.appbuilder.menu.menu + if menu_item.name == v_appbuilder_package['category']] + + self.assertTrue(len(links) == 1) + + # menu link should also have a link matching the name of the package. + link = links[0] + self.assertEqual(link.name, v_appbuilder_package['category']) + self.assertEqual(link.childs[0].name, v_appbuilder_package['name']) + + def test_flaskappbuilder_menu_links(self): + from tests.plugins.test_plugin import appbuilder_mitem + + # menu item should exist matching appbuilder_mitem + links = [menu_item for menu_item in self.appbuilder.menu.menu + if menu_item.name == appbuilder_mitem['category']] + + self.assertTrue(len(links) == 1) + + # menu link should also have a link matching the name of the package. + link = links[0] + self.assertEqual(link.name, appbuilder_mitem['category']) + self.assertEqual(link.childs[0].name, appbuilder_mitem['name']) From 24d83fa5dd268f90aec39483f630e147b6b08cb2 Mon Sep 17 00:00:00 2001 From: Ethan Vizitei Date: Mon, 5 Nov 2018 11:24:55 -0600 Subject: [PATCH 163/808] [AIRFLOW-2865] Call success_callback before updating task state (#4082) In cases where the success callback takes variable time, it's possible for it to interrupted by the heartbeat process. This is because the heartbeat process looks for tasks that are no longer in the "running" state but are still executing and reaps them. This commit reverses the order of callback invocation and state updating so that the "SUCCESS" state for the task isn't committed to the database until after the success callback has finished. --- airflow/models.py | 14 ++++++-------- tests/models.py | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index d9c63adb09d10..c397e3f5d39bb 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1697,14 +1697,6 @@ def signal_handler(signum, frame): self.handle_failure(e, test_mode, context) raise - # Recording SUCCESS - self.end_date = timezone.utcnow() - self.set_duration() - if not test_mode: - session.add(Log(self.state, self)) - session.merge(self) - session.commit() - # Success callback try: if task.on_success_callback: @@ -1713,6 +1705,12 @@ def signal_handler(signum, frame): self.log.error("Failed when executing success callback") self.log.exception(e3) + # Recording SUCCESS + self.end_date = timezone.utcnow() + self.set_duration() + if not test_mode: + session.add(Log(self.state, self)) + session.merge(self) session.commit() @provide_session diff --git a/tests/models.py b/tests/models.py index 51b8363454054..0c34878ee5696 100644 --- a/tests/models.py +++ b/tests/models.py @@ -2425,6 +2425,40 @@ def test_overwrite_params_with_dag_run_conf_none(self): self.assertEqual(False, params["override"]) + def test_success_callbak_no_race_condition(self): + class CallbackWrapper(object): + def wrap_task_instance(self, ti): + self.task_id = ti.task_id + self.dag_id = ti.dag_id + self.execution_date = ti.execution_date + self.task_state_in_callback = "" + self.callback_ran = False + + def success_handler(self, context): + self.callback_ran = True + session = settings.Session() + temp_instance = session.query(TI).filter( + TI.task_id == self.task_id).filter( + TI.dag_id == self.dag_id).filter( + TI.execution_date == self.execution_date).one() + self.task_state_in_callback = temp_instance.state + cw = CallbackWrapper() + dag = DAG('test_success_callbak_no_race_condition', start_date=DEFAULT_DATE, + end_date=DEFAULT_DATE + datetime.timedelta(days=10)) + task = DummyOperator(task_id='op', email='test@test.test', + on_success_callback=cw.success_handler, dag=dag) + ti = TI(task=task, execution_date=datetime.datetime.now()) + ti.state = State.RUNNING + session = settings.Session() + session.merge(ti) + session.commit() + cw.wrap_task_instance(ti) + ti._run_raw_task() + self.assertTrue(cw.callback_ran) + self.assertEqual(cw.task_state_in_callback, State.RUNNING) + ti.refresh_from_db() + self.assertEqual(ti.state, State.SUCCESS) + class ClearTasksTest(unittest.TestCase): From 5895c7d2578eba74cddc689b24a4be8bd882ab5d Mon Sep 17 00:00:00 2001 From: Jeff Payne Date: Wed, 5 Sep 2018 01:45:44 -0700 Subject: [PATCH 164/808] [AIRFLOW-3002] Fix variable & tests in GoogleCloudBucketHelper (#3843) --- .../contrib/operators/dataflow_operator.py | 8 +-- .../operators/test_dataflow_operator.py | 54 +++++++++++++++++-- 2 files changed, 55 insertions(+), 7 deletions(-) diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py index cd289f546ad3c..792488156dcf3 100644 --- a/airflow/contrib/operators/dataflow_operator.py +++ b/airflow/contrib/operators/dataflow_operator.py @@ -361,17 +361,17 @@ def google_cloud_to_local(self, file_name): path_components = file_name[self.GCS_PREFIX_LENGTH:].split('/') if len(path_components) < 2: raise Exception( - 'Invalid Google Cloud Storage (GCS) object path: {}.' + 'Invalid Google Cloud Storage (GCS) object path: {}' .format(file_name)) bucket_id = path_components[0] object_id = '/'.join(path_components[1:]) local_file = '/tmp/dataflow{}-{}'.format(str(uuid.uuid1())[:8], path_components[-1]) - file_size = self._gcs_hook.download(bucket_id, object_id, local_file) + self._gcs_hook.download(bucket_id, object_id, local_file) - if os.stat(file_size).st_size > 0: + if os.stat(local_file).st_size > 0: return local_file raise Exception( - 'Failed to download Google Cloud Storage GCS object: {}' + 'Failed to download Google Cloud Storage (GCS) object: {}' .format(file_name)) diff --git a/tests/contrib/operators/test_dataflow_operator.py b/tests/contrib/operators/test_dataflow_operator.py index a373126b24e4b..8ed4583255733 100644 --- a/tests/contrib/operators/test_dataflow_operator.py +++ b/tests/contrib/operators/test_dataflow_operator.py @@ -7,9 +7,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -207,5 +207,53 @@ def test_invalid_object_path(self, mock_parent_init): gcs_bucket_helper.google_cloud_to_local(file_name) self.assertEquals( - 'Invalid Google Cloud Storage (GCS) object path: {}.'.format(file_name), + 'Invalid Google Cloud Storage (GCS) object path: {}'.format(file_name), + str(context.exception)) + + @mock.patch( + 'airflow.contrib.operators.dataflow_operator.GoogleCloudBucketHelper.__init__' + ) + def test_valid_object(self, mock_parent_init): + + file_name = 'gs://test-bucket/path/to/obj.jar' + mock_parent_init.return_value = None + + gcs_bucket_helper = GoogleCloudBucketHelper() + gcs_bucket_helper._gcs_hook = mock.Mock() + + def _mock_download(bucket, object, filename=None): + text_file_contents = 'text file contents' + with open(filename, 'w') as text_file: + text_file.write(text_file_contents) + return text_file_contents + + gcs_bucket_helper._gcs_hook.download.side_effect = _mock_download + + local_file = gcs_bucket_helper.google_cloud_to_local(file_name) + self.assertIn('obj.jar', local_file) + + @mock.patch( + 'airflow.contrib.operators.dataflow_operator.GoogleCloudBucketHelper.__init__' + ) + def test_empty_object(self, mock_parent_init): + + file_name = 'gs://test-bucket/path/to/obj.jar' + mock_parent_init.return_value = None + + gcs_bucket_helper = GoogleCloudBucketHelper() + gcs_bucket_helper._gcs_hook = mock.Mock() + + def _mock_download(bucket, object, filename=None): + text_file_contents = '' + with open(filename, 'w') as text_file: + text_file.write(text_file_contents) + return text_file_contents + + gcs_bucket_helper._gcs_hook.download.side_effect = _mock_download + + with self.assertRaises(Exception) as context: + gcs_bucket_helper.google_cloud_to_local(file_name) + + self.assertEquals( + 'Failed to download Google Cloud Storage (GCS) object: {}'.format(file_name), str(context.exception)) From d44013abb081a28ebb1ccb4bd2a71a56c24f8409 Mon Sep 17 00:00:00 2001 From: Dan MacTough Date: Wed, 17 Oct 2018 14:42:01 -0400 Subject: [PATCH 165/808] [AIRFLOW-3046] Report fail from ECS Operator when host terminates (#4039) Add check for host termination to ECS Operator If an ECS task fails to complete because the host it's running on is terminated, we need to raise an exception so it can be retried. --- airflow/contrib/operators/ecs_operator.py | 10 ++++ tests/contrib/operators/test_ecs_operator.py | 52 +++++++++++++++++--- 2 files changed, 56 insertions(+), 6 deletions(-) diff --git a/airflow/contrib/operators/ecs_operator.py b/airflow/contrib/operators/ecs_operator.py index 84bc2a3c91a2f..34a5e0b7694f5 100644 --- a/airflow/contrib/operators/ecs_operator.py +++ b/airflow/contrib/operators/ecs_operator.py @@ -17,6 +17,7 @@ # specific language governing permissions and limitations # under the License. import sys +import re from airflow.exceptions import AirflowException from airflow.models import BaseOperator @@ -116,6 +117,15 @@ def _check_success_task(self): raise AirflowException(response) for task in response['tasks']: + # This is a `stoppedReason` that indicates a task has not + # successfully finished, but there is no other indication of failure + # in the response. + # See, https://docs.aws.amazon.com/AmazonECS/latest/developerguide/stopped-task-errors.html # noqa E501 + if re.match(r'Host EC2 \(instance .+?\) (stopped|terminated)\.', + task.get('stoppedReason', '')): + raise AirflowException( + 'The task was stopped because the host instance terminated: {}'. + format(task.get('stoppedReason', ''))) containers = task['containers'] for container in containers: if container.get('lastStatus') == 'STOPPED' and \ diff --git a/tests/contrib/operators/test_ecs_operator.py b/tests/contrib/operators/test_ecs_operator.py index b5d3141ec86ce..1467fff0b43eb 100644 --- a/tests/contrib/operators/test_ecs_operator.py +++ b/tests/contrib/operators/test_ecs_operator.py @@ -134,8 +134,10 @@ def test_wait_end_tasks(self): self.ecs._wait_for_task_ended() client_mock.get_waiter.assert_called_once_with('tasks_stopped') - client_mock.get_waiter.return_value.wait.assert_called_once_with(cluster='c', tasks=['arn']) - self.assertEquals(sys.maxsize, client_mock.get_waiter.return_value.config.max_attempts) + client_mock.get_waiter.return_value.wait.assert_called_once_with( + cluster='c', tasks=['arn']) + self.assertEquals( + sys.maxsize, client_mock.get_waiter.return_value.config.max_attempts) def test_check_success_tasks_raises(self): client_mock = mock.Mock() @@ -159,7 +161,8 @@ def test_check_success_tasks_raises(self): self.assertIn("'name': 'foo'", str(e.exception)) self.assertIn("'lastStatus': 'STOPPED'", str(e.exception)) self.assertIn("'exitCode': 1", str(e.exception)) - client_mock.describe_tasks.assert_called_once_with(cluster='c', tasks=['arn']) + client_mock.describe_tasks.assert_called_once_with( + cluster='c', tasks=['arn']) def test_check_success_tasks_raises_pending(self): client_mock = mock.Mock() @@ -179,7 +182,8 @@ def test_check_success_tasks_raises_pending(self): self.assertIn("This task is still pending ", str(e.exception)) self.assertIn("'name': 'container-name'", str(e.exception)) self.assertIn("'lastStatus': 'PENDING'", str(e.exception)) - client_mock.describe_tasks.assert_called_once_with(cluster='c', tasks=['arn']) + client_mock.describe_tasks.assert_called_once_with( + cluster='c', tasks=['arn']) def test_check_success_tasks_raises_mutliple(self): client_mock = mock.Mock() @@ -198,7 +202,42 @@ def test_check_success_tasks_raises_mutliple(self): }] } self.ecs._check_success_task() - client_mock.describe_tasks.assert_called_once_with(cluster='c', tasks=['arn']) + client_mock.describe_tasks.assert_called_once_with( + cluster='c', tasks=['arn']) + + def test_host_terminated_raises(self): + client_mock = mock.Mock() + self.ecs.client = client_mock + self.ecs.arn = 'arn' + client_mock.describe_tasks.return_value = { + 'tasks': [{ + 'stoppedReason': 'Host EC2 (instance i-1234567890abcdef) terminated.', + "containers": [ + { + "containerArn": "arn:aws:ecs:us-east-1:012345678910:container/e1ed7aac-d9b2-4315-8726-d2432bf11868", # noqa: E501 + "lastStatus": "RUNNING", + "name": "wordpress", + "taskArn": "arn:aws:ecs:us-east-1:012345678910:task/d8c67b3c-ac87-4ffe-a847-4785bc3a8b55" # noqa: E501 + } + ], + "desiredStatus": "STOPPED", + "lastStatus": "STOPPED", + "taskArn": "arn:aws:ecs:us-east-1:012345678910:task/d8c67b3c-ac87-4ffe-a847-4785bc3a8b55", # noqa: E501 + "taskDefinitionArn": "arn:aws:ecs:us-east-1:012345678910:task-definition/hello_world:11" # noqa: E501 + + }] + } + + with self.assertRaises(AirflowException) as e: + self.ecs._check_success_task() + + self.assertIn( + "The task was stopped because the host instance terminated:", + str(e.exception)) + self.assertIn("Host EC2 (", str(e.exception)) + self.assertIn(") terminated", str(e.exception)) + client_mock.describe_tasks.assert_called_once_with( + cluster='c', tasks=['arn']) def test_check_success_task_not_raises(self): client_mock = mock.Mock() @@ -214,7 +253,8 @@ def test_check_success_task_not_raises(self): }] } self.ecs._check_success_task() - client_mock.describe_tasks.assert_called_once_with(cluster='c', tasks=['arn']) + client_mock.describe_tasks.assert_called_once_with( + cluster='c', tasks=['arn']) if __name__ == '__main__': From d13e943484a8bb28e68f26f5ffc9de6c6f20bb57 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Sun, 14 Oct 2018 17:24:45 +0100 Subject: [PATCH 166/808] [AIRFLOW-3064] Show logs/output from operators in `airflow test` command (#4051) The logging rejig we did a for 1.10 ened with us the output from operators/tasks when running `airflow test` not going anywhere (because we have call `ti.init_run_context()` the FileTaskHandler doens't have a filename, so logs don't go anywhere) --- airflow/bin/cli.py | 5 +++++ tests/cli/test_cli.py | 26 ++++++++++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index 74b304e403826..2940d2e1232d1 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -570,6 +570,11 @@ def list_tasks(args, dag=None): @cli_utils.action_logging def test(args, dag=None): + # We want log outout from operators etc to show up here. Normally + # airflow.task would redirect to a file, but here we want it to propagate + # up to the normal airflow handler. + logging.getLogger('airflow.task').propagate = True + dag = dag or get_dag(args) task = dag.get_task(task_id=args.task_id) diff --git a/tests/cli/test_cli.py b/tests/cli/test_cli.py index c3eee0c1674ab..944ee16fad167 100644 --- a/tests/cli/test_cli.py +++ b/tests/cli/test_cli.py @@ -18,6 +18,8 @@ # under the License. # +from six import StringIO +import sys import unittest from mock import patch, Mock, MagicMock @@ -25,6 +27,7 @@ import psutil from argparse import Namespace from airflow import settings +import airflow.bin.cli as cli from airflow.bin.cli import get_num_ready_workers_running, run, get_dag from airflow.models import TaskInstance from airflow.utils import timezone @@ -177,3 +180,26 @@ def test_local_run(self): ti.refresh_from_db() state = ti.current_state() self.assertEqual(state, State.SUCCESS) + + def test_test(self): + """Test the `airflow test` command""" + args = create_mock_args( + task_id='print_the_context', + dag_id='example_python_operator', + subdir=None, + execution_date=timezone.parse('2018-01-01') + ) + + saved_stdout = sys.stdout + try: + sys.stdout = out = StringIO() + cli.test(args) + + output = out.getvalue() + # Check that prints, and log messages, are shown + self.assertIn('Done. Returned value was: Whatever you return gets printed in the logs', + output) + self.assertIn("'example_python_operator__print_the_context__20180101'", + output) + finally: + sys.stdout = saved_stdout From c28b0cf9e165ca12fda96641b443866edec0bc37 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 25 Sep 2018 02:20:17 +0800 Subject: [PATCH 167/808] [AIRFLOW-3067] Display www_rbac Flask flash msg properly (#3903) The Flask flash messages are not displayed properly. When we don't give a category for a flash message, defautl value will be 'message'. In some cases, we specify 'error' category. Using Flask-AppBuilder, the flash message will be given a CSS class 'alert-[category]'. But We don't have 'alert-message' or 'alert-error' in the current 'bootstrap-theme.css' file. This makes the the flash messages in www_rbac UI come with no background color. This commit addresses this issue by adding 'alert-message' (using specs of existing CSS class 'alert-info') and 'alert-error' (using specs of existing CSS class 'alert-danger') into 'bootstrap-theme.css'. --- airflow/www_rbac/static/bootstrap-theme.css | 22 +++++++++++++++++++++ airflow/www_rbac/views.py | 9 +++++---- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/airflow/www_rbac/static/bootstrap-theme.css b/airflow/www_rbac/static/bootstrap-theme.css index 734f940feed61..3adaec1d3b80d 100644 --- a/airflow/www_rbac/static/bootstrap-theme.css +++ b/airflow/www_rbac/static/bootstrap-theme.css @@ -4949,6 +4949,28 @@ a.thumbnail.active { .alert-danger .alert-link { color: #843534; } +.alert-message { + background-color: #d9edf7; + border-color: #bce8f1; + color: #31708f; +} +.alert-message hr { + border-top-color: #a6e1ec; +} +.alert-message .alert-link { + color: #245269; +} +.alert-error { + background-color: #f2dede; + border-color: #ebccd1; + color: #a94442; +} +.alert-error hr { + border-top-color: #e4b9c0; +} +.alert-error .alert-link { + color: #843534; +} @-webkit-keyframes progress-bar-stripes { from { background-position: 40px 0; diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 199904ce5419a..50f30649039bf 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -732,11 +732,12 @@ def delete(self): try: delete_dag.delete_dag(dag_id) except DagNotFound: - flash("DAG with id {} not found. Cannot delete".format(dag_id)) + flash("DAG with id {} not found. Cannot delete".format(dag_id), 'error') return redirect(request.referrer) except DagFileExists: flash("Dag id {} is still in DagBag. " - "Remove the DAG file first.".format(dag_id)) + "Remove the DAG file first.".format(dag_id), + 'error') return redirect(request.referrer) flash("Deleting DAG with id {}. May take a couple minutes to fully" @@ -1895,7 +1896,7 @@ def varimport(self): else: d = json.loads(out) except Exception: - flash("Missing file or syntax error.") + flash("Missing file or syntax error.", 'error') else: suc_count = fail_count = 0 for k, v in d.items(): @@ -1906,7 +1907,7 @@ def varimport(self): fail_count += 1 else: suc_count += 1 - flash("{} variable(s) successfully updated.".format(suc_count), 'info') + flash("{} variable(s) successfully updated.".format(suc_count)) if fail_count: flash("{} variables(s) failed to be updated.".format(fail_count), 'error') self.update_redirect() From 47d5447c323b3811705ff7817a316deece423371 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Szymon=20Bili=C5=84ski?= Date: Mon, 15 Oct 2018 07:41:35 +0200 Subject: [PATCH 168/808] [AIRFLOW-3069] Log all output of the S3 file transform script (#3914) The output of the process spawned by S3FileTransformOperator is not properly decoded, which makes reading logs rather difficult. Additonally, the stderr stream is only shown when process exit code is not equal to 0. - Send both output streams (stdout & stderr) to the logger. - Decode the output, so that new lines can be displayed correctly. - Include process exit code in the exception message, if the process fails. - Remove a potential deadlock, caused by `stderr=subprocess.PIPE`. - Don't load all output into memory. --- .../operators/s3_file_transform_operator.py | 24 +++++++--- .../test_s3_file_transform_operator.py | 48 +++++++++++++++++-- 2 files changed, 61 insertions(+), 11 deletions(-) diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py index 84a6eda0c8101..5111d455a832c 100644 --- a/airflow/operators/s3_file_transform_operator.py +++ b/airflow/operators/s3_file_transform_operator.py @@ -19,6 +19,7 @@ from tempfile import NamedTemporaryFile import subprocess +import sys from airflow.exceptions import AirflowException from airflow.hooks.S3_hook import S3Hook @@ -82,6 +83,7 @@ def __init__( self.replace = replace self.transform_script = transform_script self.select_expression = select_expression + self.output_encoding = sys.getdefaultencoding() def execute(self, context): if self.transform_script is None and self.select_expression is None: @@ -114,15 +116,23 @@ def execute(self, context): f_source.flush() if self.transform_script is not None: - transform_script_process = subprocess.Popen( + process = subprocess.Popen( [self.transform_script, f_source.name, f_dest.name], - stdout=subprocess.PIPE, stderr=subprocess.PIPE, close_fds=True) - (transform_script_stdoutdata, transform_script_stderrdata) = \ - transform_script_process.communicate() - self.log.info("Transform script stdout %s", transform_script_stdoutdata) - if transform_script_process.returncode > 0: + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT, + close_fds=True + ) + + self.log.info("Output:") + for line in iter(process.stdout.readline, b''): + self.log.info(line.decode(self.output_encoding).rstrip()) + + process.wait() + + if process.returncode > 0: raise AirflowException( - "Transform script failed %s", transform_script_stderrdata) + "Transform script failed: {0}".format(process.returncode) + ) else: self.log.info( "Transform script successful. Output temporarily located at %s", diff --git a/tests/operators/test_s3_file_transform_operator.py b/tests/operators/test_s3_file_transform_operator.py index 6b72c58f7857f..af00a4bcf5f75 100644 --- a/tests/operators/test_s3_file_transform_operator.py +++ b/tests/operators/test_s3_file_transform_operator.py @@ -22,6 +22,7 @@ import io import os import shutil +import sys import unittest from tempfile import mkdtemp @@ -29,6 +30,7 @@ import mock from moto import mock_s3 +from airflow.exceptions import AirflowException from airflow.operators.s3_file_transform_operator import S3FileTransformOperator @@ -48,11 +50,15 @@ def tearDown(self): raise e @mock.patch('subprocess.Popen') + @mock.patch.object(S3FileTransformOperator, 'log') @mock_s3 - def test_execute_with_transform_script(self, mock_Popen): - transform_script_process = mock_Popen.return_value - transform_script_process.communicate.return_value = [None, None] - transform_script_process.returncode = 0 + def test_execute_with_transform_script(self, mock_log, mock_Popen): + process_output = [b"Foo", b"Bar", b"Baz"] + + process = mock_Popen.return_value + process.stdout.readline.side_effect = process_output + process.wait.return_value = None + process.returncode = 0 bucket = "bucket" input_key = "foo" @@ -72,6 +78,40 @@ def test_execute_with_transform_script(self, mock_Popen): task_id="task_id") t.execute(None) + mock_log.info.assert_has_calls([ + mock.call(line.decode(sys.getdefaultencoding())) for line in process_output + ]) + + @mock.patch('subprocess.Popen') + @mock_s3 + def test_execute_with_failing_transform_script(self, mock_Popen): + process = mock_Popen.return_value + process.stdout.readline.side_effect = [] + process.wait.return_value = None + process.returncode = 42 + + bucket = "bucket" + input_key = "foo" + output_key = "bar" + bio = io.BytesIO(b"input") + + conn = boto3.client('s3') + conn.create_bucket(Bucket=bucket) + conn.upload_fileobj(Bucket=bucket, Key=input_key, Fileobj=bio) + + s3_url = "s3://{0}/{1}" + t = S3FileTransformOperator( + source_s3_key=s3_url.format(bucket, input_key), + dest_s3_key=s3_url.format(bucket, output_key), + transform_script=self.transform_script, + replace=True, + task_id="task_id") + + with self.assertRaises(AirflowException) as e: + t.execute(None) + + self.assertEqual('Transform script failed: 42', str(e.exception)) + @mock.patch('airflow.hooks.S3_hook.S3Hook.select_key', return_value="input") @mock_s3 def test_execute_with_select_expression(self, mock_select_key): From 6b1859849f2fa760eb67b12083046418f584724b Mon Sep 17 00:00:00 2001 From: Brad Holmes Date: Sat, 29 Sep 2018 13:17:43 -0400 Subject: [PATCH 169/808] [AIRFLOW-3079] Improve migration scripts to support MSSQL Server (#3964) There were two problems for MSSQL. First, 'timestamp' data type in MSSQL Server is essentially a row-id, and not a timezone enabled date/time stamp. Second, alembic creates invalid SQL when applying the 0/1 constraint to boolean values. MSSQL should enforce this constraint by simply asserting a boolean value. --- .../0e2a74e0fc9f_add_time_zone_awareness.py | 6 +++--- ...ff4_add_kubernetes_resource_checkpointing.py | 17 ++++++++++++++--- ...215c0_add_kubernetes_scheduler_uniqueness.py | 17 ++++++++++++++--- 3 files changed, 31 insertions(+), 9 deletions(-) diff --git a/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py b/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py index 64ee41c44d9e2..ad10c76ab8bc6 100644 --- a/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py +++ b/airflow/migrations/versions/0e2a74e0fc9f_add_time_zone_awareness.py @@ -84,8 +84,8 @@ def upgrade(): op.alter_column(table_name='xcom', column_name='timestamp', type_=mysql.TIMESTAMP(fsp=6)) op.alter_column(table_name='xcom', column_name='execution_date', type_=mysql.TIMESTAMP(fsp=6)) else: - # sqlite datetime is fine as is not converting - if conn.dialect.name == 'sqlite': + # sqlite and mssql datetime are fine as is. Therefore, not converting + if conn.dialect.name in ('sqlite', 'mssql'): return # we try to be database agnostic, but not every db (e.g. sqlserver) @@ -176,7 +176,7 @@ def downgrade(): op.alter_column(table_name='xcom', column_name='DATETIME', type_=mysql.DATETIME(fsp=6)) op.alter_column(table_name='xcom', column_name='execution_date', type_=mysql.DATETIME(fsp=6)) else: - if conn.dialect.name == 'sqlite': + if conn.dialect.name in ('sqlite', 'mssql'): return # we try to be database agnostic, but not every db (e.g. sqlserver) diff --git a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py index c489c05f7ea39..4eb056892ac5f 100644 --- a/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py +++ b/airflow/migrations/versions/33ae817a1ff4_add_kubernetes_resource_checkpointing.py @@ -40,12 +40,23 @@ def upgrade(): + + columns_and_constraints = [ + sa.Column("one_row_id", sa.Boolean, server_default=sa.true(), primary_key=True), + sa.Column("resource_version", sa.String(255)) + ] + + conn = op.get_bind() + + # alembic creates an invalid SQL for mssql dialect + if conn.dialect.name not in ('mssql'): + columns_and_constraints.append(sa.CheckConstraint("one_row_id", name="kube_resource_version_one_row_id")) + table = op.create_table( RESOURCE_TABLE, - sa.Column("one_row_id", sa.Boolean, server_default=sa.true(), primary_key=True), - sa.Column("resource_version", sa.String(255)), - sa.CheckConstraint("one_row_id", name="kube_resource_version_one_row_id") + *columns_and_constraints ) + op.bulk_insert(table, [ {"resource_version": ""} ]) diff --git a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py index 5c921c6a98aaf..ea1e9ac4bf837 100644 --- a/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py +++ b/airflow/migrations/versions/86770d1215c0_add_kubernetes_scheduler_uniqueness.py @@ -40,12 +40,23 @@ def upgrade(): + + columns_and_constraints = [ + sa.Column("one_row_id", sa.Boolean, server_default=sa.true(), primary_key=True), + sa.Column("worker_uuid", sa.String(255)) + ] + + conn = op.get_bind() + + # alembic creates an invalid SQL for mssql dialect + if conn.dialect.name not in ('mssql'): + columns_and_constraints.append(sa.CheckConstraint("one_row_id", name="kube_worker_one_row_id")) + table = op.create_table( RESOURCE_TABLE, - sa.Column("one_row_id", sa.Boolean, server_default=sa.true(), primary_key=True), - sa.Column("worker_uuid", sa.String(255)), - sa.CheckConstraint("one_row_id", name="kube_worker_one_row_id") + *columns_and_constraints ) + op.bulk_insert(table, [ {"worker_uuid": ""} ]) From 4f6698146bfd00fac4180dcd179daa463ddfb54b Mon Sep 17 00:00:00 2001 From: William Pursell Date: Thu, 20 Sep 2018 13:35:03 -0600 Subject: [PATCH 170/808] [AIRFLOW-3090] Demote dag start/stop log messages to debug (#3920) --- airflow/utils/dag_processing.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index d30c6aeacb5be..8b77c796d2030 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -487,7 +487,7 @@ def heartbeat(self): for file_path, processor in self._processors.items(): if processor.done: - self.log.info("Processor for %s finished", file_path) + self.log.debug("Processor for %s finished", file_path) now = timezone.utcnow() finished_processors[file_path] = processor self._last_runtime[file_path] = (now - @@ -560,7 +560,7 @@ def heartbeat(self): processor = self._processor_factory(file_path) processor.start() - self.log.info( + self.log.debug( "Started a process (PID: %s) to generate tasks for %s", processor.pid, file_path ) From d77b43c3236294db25d8a14e79cea157f3969513 Mon Sep 17 00:00:00 2001 From: William Pursell Date: Fri, 21 Sep 2018 05:34:27 -0700 Subject: [PATCH 171/808] [AIRFLOW-3090] Make No tasks to consider for execution debug (#3923) During normal operation, it is not necessary to see the message. This can only be useful when debugging an issue. --- airflow/jobs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index d17ab262e6d76..1e5478b9d7955 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1100,7 +1100,7 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): task_instances_to_examine = ti_query.all() if len(task_instances_to_examine) == 0: - self.log.info("No tasks to consider for execution.") + self.log.debug("No tasks to consider for execution.") return executable_tis # Put one task instance on each line From 7b24b00f0d0d673949352086dabc4b8983d00d6e Mon Sep 17 00:00:00 2001 From: William Pursell Date: Sat, 22 Sep 2018 06:33:50 -0700 Subject: [PATCH 172/808] [AIRFLOW-3090] Specify path of key file in log message (#3921) --- airflow/contrib/hooks/gcp_api_base_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/contrib/hooks/gcp_api_base_hook.py b/airflow/contrib/hooks/gcp_api_base_hook.py index 053494743f064..70523ffadcc92 100644 --- a/airflow/contrib/hooks/gcp_api_base_hook.py +++ b/airflow/contrib/hooks/gcp_api_base_hook.py @@ -87,7 +87,7 @@ def _get_credentials(self): elif key_path: # Get credentials from a JSON file. if key_path.endswith('.json'): - self.log.info('Getting connection using a JSON key file.') + self.log.debug('Getting connection using JSON key file %s' % key_path) credentials = ( google.oauth2.service_account.Credentials.from_service_account_file( key_path, scopes=scopes) From e8b2feb3da885bfd3571206dcbef550d9d13c938 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Fri, 28 Sep 2018 20:43:55 +0800 Subject: [PATCH 173/808] [AIRFLOW-3104] Add .airflowignore info into doc (#3939) .airflowignore is a nice feature, but it was not mentioned at all in the documentation. --- airflow/models.py | 8 +++++--- docs/concepts.rst | 29 +++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index c397e3f5d39bb..de9b06d124623 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -532,10 +532,12 @@ def collect_dags( Given a file path or a folder, this method looks for python modules, imports them and adds them to the dagbag collection. - Note that if a .airflowignore file is found while processing, - the directory, it will behaves much like a .gitignore does, + Note that if a ``.airflowignore`` file is found while processing + the directory, it will behave much like a ``.gitignore``, ignoring files that match any of the regex patterns specified - in the file. **Note**: The patterns in .airflowignore are treated as + in the file. + + **Note**: The patterns in .airflowignore are treated as un-anchored regexes, not shell-like glob patterns. """ start_dttm = timezone.utcnow() diff --git a/docs/concepts.rst b/docs/concepts.rst index d28921134f3d6..f1fd14f5b187b 100644 --- a/docs/concepts.rst +++ b/docs/concepts.rst @@ -852,3 +852,32 @@ do the same, but then it is more to use a virtualenv and pip. to be available on the system if a module needs those. In other words only pure python modules can be packaged. + +.airflowignore +'''''''''''''' + +A ``.airflowignore`` file specifies the directories or files in ``DAG_FOLDER`` +that Airflow should intentionally ignore. Each line in ``.airflowignore`` +specifies a regular expression pattern, and directories or files whose names +(not DAG id) match any of the patterns would be ignored (under the hood, +``re.findall()`` is used to match the pattern). Overall it works like a +``.gitignore`` file. + +``.airflowignore`` file should be put in your ``DAG_FOLDER``. +For example, you can prepare a ``.airflowignore`` file with contents + +.. code:: + + project_a + tenant_[\d] + + +Then files like "project_a_dag_1.py", "TESTING_project_a.py", "tenant_1.py", +"project_a/dag_1.py", and "tenant_1/dag_1.py" in your ``DAG_FOLDER`` would be ignored +(If a directory's name matches any of the patterns, this directory and all its subfolders +would not be scanned by Airflow at all. This improves efficiency of DAG finding). + +The scope of a ``.airflowignore`` file is the directory it is in plus all its subfolders. +You can also prepare ``.airflowignore`` file for a subfolder in ``DAG_FOLDER`` and it +would only be applicable for that subfolder. + From 1329b987f82b359e46b0049f06fce109b6fea688 Mon Sep 17 00:00:00 2001 From: Joy Gao Date: Mon, 24 Sep 2018 15:27:29 -0700 Subject: [PATCH 174/808] [AIRFLOW-3109] Bugfix to allow user/op roles to clear task intance via UI by default --- airflow/www_rbac/security.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/www_rbac/security.py b/airflow/www_rbac/security.py index ae21029dd182b..2405aa8f803de 100644 --- a/airflow/www_rbac/security.py +++ b/airflow/www_rbac/security.py @@ -105,6 +105,7 @@ 'set_running', 'set_success', 'clear', + 'can_clear', ] op_perms = [ From be28af62c5989cfcaa183a32637daa2bd36a0005 Mon Sep 17 00:00:00 2001 From: Eric Chang Date: Tue, 25 Sep 2018 09:26:41 -0700 Subject: [PATCH 175/808] [AIRFLOW-3111] Fix instructions in UPDATING.md and remove comment (#3944) artifacts in default_airflow.cfg - fixed incorrect instructions in UPDATING.md regarding core.log_filename_template and elasticsearch.elasticsearch_log_id_template - removed comments referencing "additional curly braces" from default_airflow.cfg since they're irrelevant to the rendered airflow.cfg --- UPDATING.md | 2 +- airflow/config_templates/default_airflow.cfg | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 27fbc7444d732..6fde229a2038d 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -180,7 +180,7 @@ log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number log_processor_filename_template = {{ filename }}.log [elasticsearch] -elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_number}} +elasticsearch_log_id_template = {dag_id}-{task_id}-{execution_date}-{try_number} elasticsearch_end_of_log_mark = end_of_log ``` diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 1a3768ff74603..07a9458073dae 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -62,12 +62,10 @@ fab_logging_level = WARN logging_config_class = # Log format -# we need to escape the curly braces by adding an additional curly brace log_format = [%%(asctime)s] {{%%(filename)s:%%(lineno)d}} %%(levelname)s - %%(message)s simple_log_format = %%(asctime)s %%(levelname)s - %%(message)s # Log filename format -# we need to escape the curly braces by adding an additional curly brace log_filename_template = {{{{ ti.dag_id }}}}/{{{{ ti.task_id }}}}/{{{{ ts }}}}/{{{{ try_number }}}}.log log_processor_filename_template = {{{{ filename }}}}.log @@ -556,7 +554,6 @@ hide_sensitive_variable_fields = True [elasticsearch] elasticsearch_host = -# we need to escape the curly braces by adding an additional curly brace elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_number}} elasticsearch_end_of_log_mark = end_of_log From 38680e2dde47543d46511e436d7d42b33d908a19 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Fri, 28 Sep 2018 16:56:43 +0800 Subject: [PATCH 176/808] [AIRFLOW-3127] Fix out-dated doc for Celery SSL (#3967) Now in `airflow.cfg`, for Celery-SSL, the item names are "ssl_active", "ssl_key", "ssl_cert", and "ssl_cacert". (since PR https://github.com/apache/incubator-airflow/pull/2806/files) But in the documentation https://airflow.incubator.apache.org/security.html?highlight=celery or https://github.com/apache/incubator-airflow/blob/master/docs/security.rst, it's "CELERY_SSL_ACTIVE", "CELERY_SSL_KEY", "CELERY_SSL_CERT", and "CELERY_SSL_CACERT", which is out-dated and may confuse readers. --- docs/security.rst | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/security.rst b/docs/security.rst index f8b747eb0f752..b47a58f7418f0 100644 --- a/docs/security.rst +++ b/docs/security.rst @@ -344,10 +344,10 @@ certs and keys. .. code-block:: bash [celery] - CELERY_SSL_ACTIVE = True - CELERY_SSL_KEY = - CELERY_SSL_CERT = - CELERY_SSL_CACERT = + ssl_active = True + ssl_key = + ssl_cert = + ssl_cacert = Impersonation ------------- From 32e9cd8f1d74f4b947a22a1750c7559be804324e Mon Sep 17 00:00:00 2001 From: Vardan Gupta Date: Thu, 1 Nov 2018 02:10:31 +0530 Subject: [PATCH 177/808] [AIRFLOW-3136] Add retry_number to TaskInstance.key to avoid race condition (#3994) We were seeing an intermittent issue where executor reports task instance finished while task says it's in queue state, it was due to a race condition between scheduler which was clearing event_buffer in _process_executor_events method in jobs.py executor was about to put next_retry task's status as running which was failed in previous try. So, we thought to add retry_number as the member of TaskInstance key property. --- airflow/contrib/executors/kubernetes_executor.py | 7 ++++--- airflow/executors/base_executor.py | 2 +- airflow/jobs.py | 8 ++++---- airflow/models.py | 2 +- tests/executors/test_base_executor.py | 8 ++++---- tests/jobs.py | 13 +++++++++---- 6 files changed, 23 insertions(+), 17 deletions(-) diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 4ea52c42ff604..5b02c4d40923b 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -326,7 +326,7 @@ def run_next(self, next_job): """ self.log.info('Kubernetes job is %s', str(next_job)) key, command, kube_executor_config = next_job - dag_id, task_id, execution_date = key + dag_id, task_id, execution_date, try_number = key self.log.debug("Kubernetes running for command %s", command) self.log.debug("Kubernetes launching image %s", self.kube_config.kube_image) pod = self.worker_configuration.make_pod( @@ -447,7 +447,8 @@ def _labels_to_key(self, labels): try: return ( labels['dag_id'], labels['task_id'], - self._label_safe_datestring_to_datetime(labels['execution_date'])) + self._label_safe_datestring_to_datetime(labels['execution_date']), + labels['try_number']) except Exception as e: self.log.warn( 'Error while converting labels to key; labels: %s; exception: %s', @@ -606,7 +607,7 @@ def _change_state(self, key, state, pod_id): self.log.debug('Could not find key: %s', str(key)) pass self.event_buffer[key] = state - (dag_id, task_id, ex_time) = key + (dag_id, task_id, ex_time, try_number) = key item = self._session.query(TaskInstance).filter_by( dag_id=dag_id, task_id=task_id, diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 701ac66f8b72d..3522f7c449790 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -175,7 +175,7 @@ def get_event_buffer(self, dag_ids=None): self.event_buffer = dict() else: for key in list(self.event_buffer.keys()): - dag_id, _, _ = key + dag_id, _, _, _ = key if dag_id in dag_ids: cleared_events[key] = self.event_buffer.pop(key) diff --git a/airflow/jobs.py b/airflow/jobs.py index 1e5478b9d7955..03610bd983607 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1441,10 +1441,10 @@ def _process_executor_events(self, simple_dag_bag, session=None): TI = models.TaskInstance for key, state in list(self.executor.get_event_buffer(simple_dag_bag.dag_ids) .items()): - dag_id, task_id, execution_date = key + dag_id, task_id, execution_date, try_number = key self.log.info( - "Executor reports %s.%s execution_date=%s as %s", - dag_id, task_id, execution_date, state + "Executor reports %s.%s execution_date=%s as %s for try_number %s", + dag_id, task_id, execution_date, state, try_number ) if state == State.FAILED or state == State.SUCCESS: qry = session.query(TI).filter(TI.dag_id == dag_id, @@ -1456,7 +1456,7 @@ def _process_executor_events(self, simple_dag_bag, session=None): continue # TODO: should we fail RUNNING as well, as we do in Backfills? - if ti.state == State.QUEUED: + if ti.try_number == try_number and ti.state == State.QUEUED: msg = ("Executor reports task instance {} finished ({}) " "although the task says its {}. Was the task " "killed externally?".format(ti, state, ti.state)) diff --git a/airflow/models.py b/airflow/models.py index de9b06d124623..1536b37df906f 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1249,7 +1249,7 @@ def key(self): """ Returns a tuple that identifies the task instance uniquely """ - return self.dag_id, self.task_id, self.execution_date + return self.dag_id, self.task_id, self.execution_date, self.try_number @provide_session def set_state(self, state, session=None): diff --git a/tests/executors/test_base_executor.py b/tests/executors/test_base_executor.py index f349355d32efa..06a47541e404c 100644 --- a/tests/executors/test_base_executor.py +++ b/tests/executors/test_base_executor.py @@ -30,10 +30,10 @@ def test_get_event_buffer(self): executor = BaseExecutor() date = datetime.utcnow() - - key1 = ("my_dag1", "my_task1", date) - key2 = ("my_dag2", "my_task1", date) - key3 = ("my_dag2", "my_task2", date) + try_number = 1 + key1 = ("my_dag1", "my_task1", date, try_number) + key2 = ("my_dag2", "my_task1", date, try_number) + key3 = ("my_dag2", "my_task2", date, try_number) state = State.SUCCESS executor.event_buffer[key1] = state executor.event_buffer[key2] = state diff --git a/tests/jobs.py b/tests/jobs.py index 5bd3a00451a1b..db2374c13d8c5 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -72,7 +72,7 @@ DEV_NULL = '/dev/null' DEFAULT_DATE = timezone.datetime(2016, 1, 1) - +TRY_NUMBER = 1 # Include the words "airflow" and "dag" in the file contents, # tricking airflow into thinking these # files contain a DAG (otherwise Airflow will skip them) @@ -2329,7 +2329,7 @@ def test_scheduler_process_task_instances(self): scheduler._process_task_instances(dag, queue=queue) queue.append.assert_called_with( - (dag.dag_id, dag_task1.task_id, DEFAULT_DATE) + (dag.dag_id, dag_task1.task_id, DEFAULT_DATE, TRY_NUMBER) ) def test_scheduler_do_not_schedule_removed_task(self): @@ -2597,7 +2597,7 @@ def test_scheduler_max_active_runs_respected_after_clear(self): scheduler._process_task_instances(dag, queue=queue) queue.append.assert_called_with( - (dag.dag_id, dag_task1.task_id, DEFAULT_DATE) + (dag.dag_id, dag_task1.task_id, DEFAULT_DATE, TRY_NUMBER) ) @patch.object(TI, 'pool_full') @@ -2950,13 +2950,18 @@ def run_with_error(task): do_schedule() self.assertTrue(executor.has_task(ti)) ti.refresh_from_db() - self.assertEqual(ti.state, State.SCHEDULED) + # removing self.assertEqual(ti.state, State.SCHEDULED) + # as scheduler will move state from SCHEDULED to QUEUED # now the executor has cleared and it should be allowed the re-queue executor.queued_tasks.clear() do_schedule() ti.refresh_from_db() self.assertEqual(ti.state, State.QUEUED) + # calling below again in order to ensure with try_number 2, + # scheduler doesn't put task in queue + do_schedule() + self.assertEquals(1, len(executor.queued_tasks)) @unittest.skipUnless("INTEGRATION" in os.environ, "Can only run end to end") def test_retry_handling_job(self): From ca33a6e2abdacd6c0b7619a462f307a59ea8be76 Mon Sep 17 00:00:00 2001 From: akshayi1 Date: Fri, 5 Oct 2018 04:47:48 -0400 Subject: [PATCH 178/808] [AIRFLOW-3159] Update GCS logging docs for latest code (#3952) --- docs/howto/write-logs.rst | 67 ++++++++------------------------------- 1 file changed, 14 insertions(+), 53 deletions(-) diff --git a/docs/howto/write-logs.rst b/docs/howto/write-logs.rst index a29f6069ca089..44fefc64809f3 100644 --- a/docs/howto/write-logs.rst +++ b/docs/howto/write-logs.rst @@ -89,54 +89,22 @@ Writing Logs to Google Cloud Storage Follow the steps below to enable Google Cloud Storage logging. -#. Airflow's logging system requires a custom .py file to be located in the ``PYTHONPATH``, so that it's importable from Airflow. Start by creating a directory to store the config file. ``$AIRFLOW_HOME/config`` is recommended. -#. Create empty files called ``$AIRFLOW_HOME/config/log_config.py`` and ``$AIRFLOW_HOME/config/__init__.py``. -#. Copy the contents of ``airflow/config_templates/airflow_local_settings.py`` into the ``log_config.py`` file that was just created in the step above. -#. Customize the following portions of the template: - - .. code-block:: bash - - # Add this variable to the top of the file. Note the trailing slash. - GCS_LOG_FOLDER = 'gs:///' - - # Rename DEFAULT_LOGGING_CONFIG to LOGGING CONFIG - LOGGING_CONFIG = ... - - # Add a GCSTaskHandler to the 'handlers' block of the LOGGING_CONFIG variable - 'gcs.task': { - 'class': 'airflow.utils.log.gcs_task_handler.GCSTaskHandler', - 'formatter': 'airflow.task', - 'base_log_folder': os.path.expanduser(BASE_LOG_FOLDER), - 'gcs_log_folder': GCS_LOG_FOLDER, - 'filename_template': FILENAME_TEMPLATE, - }, - - # Update the airflow.task and airflow.tas_runner blocks to be 'gcs.task' instead of 'file.task'. - 'loggers': { - 'airflow.task': { - 'handlers': ['gcs.task'], - ... - }, - 'airflow.task_runner': { - 'handlers': ['gcs.task'], - ... - }, - 'airflow': { - 'handlers': ['console'], - ... - }, - } - -#. Make sure a Google Cloud Platform connection hook has been defined in Airflow. The hook should have read and write access to the Google Cloud Storage bucket defined above in ``GCS_LOG_FOLDER``. - -#. Update ``$AIRFLOW_HOME/airflow.cfg`` to contain: - - .. code-block:: bash +To enable this feature, ``airflow.cfg`` must be configured as in this +example: - task_log_reader = gcs.task - logging_config_class = log_config.LOGGING_CONFIG - remote_log_conn_id = +.. code-block:: bash + [core] + # Airflow can store logs remotely in AWS S3, Google Cloud Storage or Elastic Search. + # Users must supply an Airflow connection id that provides access to the storage + # location. If remote_logging is set to true, see UPDATING.md for additional + # configuration requirements. + remote_logging = True + remote_base_log_folder = gs://my-bucket/path/to/logs + remote_log_conn_id = MyGCSConn + +#. Install the ``gcp_api`` package first, like so: ``pip install apache-airflow[gcp_api]``. +#. Make sure a Google Cloud Platform connection hook has been defined in Airflow. The hook should have read and write access to the Google Cloud Storage bucket defined above in ``remote_base_log_folder``. #. Restart the Airflow webserver and scheduler, and trigger (or wait for) a new task execution. #. Verify that logs are showing up for newly executed tasks in the bucket you've defined. #. Verify that the Google Cloud Storage viewer is working in the UI. Pull up a newly executed task, and verify that you see something like: @@ -150,10 +118,3 @@ Follow the steps below to enable Google Cloud Storage logging. [2017-10-03 21:57:51,306] {base_task_runner.py:98} INFO - Subtask: [2017-10-03 21:57:51,306] {models.py:186} INFO - Filling up the DagBag from /airflow/dags/example_dags/example_bash_operator.py Note the top line that says it's reading from the remote log file. - -Please be aware that if you were persisting logs to Google Cloud Storage -using the old-style airflow.cfg configuration method, the old logs will no -longer be visible in the Airflow UI, though they'll still exist in Google -Cloud Storage. This is a backwards incompatbile change. If you are unhappy -with it, you can change the ``FILENAME_TEMPLATE`` to reflect the old-style -log filename format. From dd68a28e8b904017738d7fe5f667655e500a39dd Mon Sep 17 00:00:00 2001 From: PaulVelthuis93 Date: Fri, 12 Oct 2018 14:13:58 +0200 Subject: [PATCH 179/808] [AIRFLOW-3187] Update airflow.gif file with a slower version (#4033) --- docs/img/airflow.gif | Bin 622963 -> 622832 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/docs/img/airflow.gif b/docs/img/airflow.gif index 1889b868f708aaaa9a29a3ef2613c8436b8de384..7646e37f35ed5fbe0ad3f65d612ff3aa1a1e531f 100644 GIT binary patch delta 580462 zcmV)NK)1j1f+p~RCJaYOMmRZPt_8pY_W-dB_yP(3|NH;|00000lMMqG67&H904x9i z006E9zykmX{{Z`!ZIc0alaQ>10*n8bYm@;SlTQTV1F**blQaeV1m1l2|C2NZ`vpJ@ zq7eU+Gzal*oG!^WVAr=z> z|C2Np>yscC6aoK}G#BfWAQ%(@|C2NrFnM0Mx;2= zVnvJ>HD=_v(PKxDA4PKPSJLE3lqprNWZBZ?OPDcb&P-Xap1gO^poQc)&|bYxk?x4d zkb%Teq)Cxx5OKu?000FXOf{k7$d)5nwQh|OW5tLwwf0?n7Q`$OooS(cD`Pf@8-Ik2 z-0mF6#qBeO?}(gz*4OXfXWt0cvl|48yt-!vKOE*Z4?#h51->$SIEh(N1JCUZlBwtC zkq7mfrOU9fVVn1$DLzQatPyzvZ<$TNh zjTy!m6jM}jMHW-!B7_jicqYYXQh-sA6y1PP#uc9_V@DM8fYFT^Qxr0UMt_P75=Kf; z6a`3Qo(us7C6`2!ML}THa?22)Xz@!hNfdJ!5FI-0Fo}|TEIBtELJ+RVluPKFZ?xmi zJMbFn(8Jxz^z%GG{}eP(LI*XpP(&M||7lT18+G(iNK3K_C!G?V=%-xlAnK?YI`wo0 zq-MzAh&}kIq!K2eYT%49T7TW))vdDXYK0YQIN}UWI2lKyIQi6#i0=$a4!;QttIeVN zpa4%oX6s7~HTf#r&>;cMqEDb|I~yoY`l>6BGSVzP)IQGgYY5yVHte-MD7fG(h+78o zkH3QiDdS#ztsQ8#@yZb8opHrgNX>-gfg;?5)HE33fvv4)G-YjSOMf>gaCA1IJ;@7r zP&Jz+4w{MKefE`upxGL6o1pCjT3K&>Z>PYw<>-)yr4H0*l@%TS3GgX7r!V{$Rn40@|Pyf zNjRG_*L-u%JNMisUv@BxgilNNbPQ8ZRhpa=me%Czt=Ar-mKHhE;z6_gMex;rc(|@hU?hKoEeth!Fcm6)i zIR`&Jd4%^F`+Dv(W^<2CaabT9`6SbP{`>cz;LW|`J@ez7ngnPT0&b3A*ioKM8sR?p zh>vp_yPy9oc)tO<~e!|BRDKFmoz%pjjI6w+6{0ImQ@`hEnqm@Qp`7_wgBk zCODqzVeoe%8h?|C)^@bBp^ZzTK+)4ekhK*_sSH*^qLs?VL??j_N?M~@i-M>nwf)d& zL>rL~e|IJkX0B(Vs7K52#yK#Wk$eyIkQ>67ISTg4i*ux79qo8WJmyi4d*q`9DfdS} z2C`9>tJKD9f(;ss&UB>^Cnd_%}-(p4QYTBMub z4Cgo-+Bl$^Q>Z~D>QIe}5J4tYsY^WxA=9_V*niZaD5P8D45)AgJERVhr{cjRFZoqU z>f()^3mGMYJY1zj-qR(>(a+$=0 zxxxFKupYz0i|43!lXC=1|3dHR)j|P_M*wcFFqYNa#K^c=vPOr2v5D+Cnl+5~c<*x^ z%75Wy?U>mN%F%`((8+&{r` zZj)TF$<53%3=~(#d;9;X&3r$^uj)x?VmJPA|6rRr@cGy#*}BE|ex^At$1j9sjDHz7 z??F#3cTN8JiJQSN&#a&L||p!5TPPGb89Raf*-TI+WMdvRRK@ zY%hzxON8oL2pYH0(t{c?-4zW(X%-EyM@{PK@=B4U=~XX#8=YyQ?NPp*>s4O&^wj;F z@XjQLqggc6GQofc%>?G2qTBpyV1EZ&*u%!4gpGY{Mp3wrKk{4M780|VIcc^DjLSFIJvEtg^9q7(ZBiXB6lQxi`8E_LhG=3tmNU7| z95fgdg#9H)wVdM@|G;fR*d(|+et>&Swi0*~HRk(wZ>|SD-y*(m2v2zycYkB%GX66_ zPpr`NrVXzpUOtM;?8q&CDuz0vxq{~~TYYZcu<)9>{S*(&4zsw<Y&C}E1-yM^?` zck^%J{r#4ih4i!Cocp){dmg&VP$WaK?sh9y-x49fA}v zn*R#{Q+S2}41qkLggk(OEt$Gc1F5acy7OWYtK+)p^14OyKG*w|hszj2cm+y$7ewoo zB*+A1atpGnzU#|A@qgn!7koh&j6w71I~uIPe)zjS!UDky1H+3p4okdKS-cYnF~{SC z-Z7toBR9^mzY^NBaeE`q8z0ScxeOs7c`yZ>QMiY}86WGJl5;^C;u9i3GMl)or`nu2 zsjGWnJ`WLv|GnUi_8SkyazTnSEIy%x39&;D!NYPJGr02*BY!wM8FU^tXzza+@Jt{a~;ewsf9Ch*)&;zL4kiI`0k3bwmVH`$cv^yI-Mr6amJ$eTrP$a_Z z!NUW>AoKtasDFYY%#~W`9|oDjFUk$P$ULy4Ex3xYa0IA68;?xLL(Ta&eDgB5Xh*!0 z#PNWFJm|7KOpfjAoG=@Xi)y2tXbZ(s4PNAew|T+jfJeE>1lZ#OGR&NTP)OZi$UFqJ zgM&T1vW@>@9IWBtf|bcbyFd%h;ip1$gInC3Y)PMcqkkV58V@2+5QkhWoOz3q{IyIZ zI-?5#A_9|V_^zm_E=pj)25dzX*{D;j!0<8!C`p=8^R%3V#aJAZJ&HpVUtO?gIWjJ6J=HXiVR6BxoG z6f6EwM1TGmAK$w<7Ly0f6Gz&hH7TqN#B@hr;mXeWHnZ?Wo7g0SgTadkN%FakmrN#w zf)Iv~n|J676j820v`Wl`24naJV+fYck(bn5&B1udJTwT^L`~r^&9vMIT>P8~xh@vM z&6nCu|J7R5yr<0ksqpwa|R8EqEc^cWXKP8-$FWtq_$#Ze(`%@;K@ z&neO&WtL%c&QnW^=v0wNgU?W$(&>aw6v0v{6;B!w&%OZ9FZDz%CC`X5&pcvL9Cb(@ zP1650T~a0mQaFuMIeoDHtkXaGPdxGk5q}VY0yR9;K~M!<(5?b2M_>fs=^xD;pG0L& zgn&^-Rn$i%h<7lCTQG%J@XR!A)YgpDIqg(W{Zvo|to|d@_VNo-Jyld)DKQCER&7;R z%{x1dRms8AJJQoW?ZH0o8wE-Sb zf@`a!DzE{3B?My~SAOkRfBn}u9o23fSc2_VSp-;wP1tcYSB6cto4B0I%7t}h*TmaZ z|9C}#dF25)2(f$R0UpTL#Z*|49e-JpEm;N2)`Cr0m4(G`HCdN^S+s0enUx82og;Of z*dCOGNl*e0Q-XJe*AwtnCGgmd6$6hw+K**}?=U%-ZCa;&T9Osmm7Q9uRjGr8TCB}l zW1Ly8^}C1FiHOBn!;1u)@C2TPB%f`8i$wt?XxpJ3TBJQ%8(;%|(^|W|TYtQzR+Oz; zzU^E0g37!NT)|DSt}R@Zs97(H4T%+7Nw|sFNQSdjyq`q@wG~>oz1+4%S|-2?rX5_* z{anyJIjQwq((T);4PDetUH&m#)-?&l)gi?dTR(V*pYR=WkON3k+ZNbec*R`bggV#f2fxVJME`e|@f}t6?kFR=|~FEe_Wm?pmjc4KVhdFdkzuE@Ltd9mKHvVHj4rD+cWI--uLOx_ePGm%0WJPXd zMt)>Rj$}xlWJ#`MO1@-E&SXs9WKHg5PX1(14&~UG;VUlXrz8}mIquI){A}DcTM5ARc2x}R$G4OZjNVoo@aTk zXL`P8d(LNk-e-O8=Qbu~Y7XcxAsT)zXn#IvgHC9KUTB3@Xl<5R3ajNePGkSW1LvQh zIGE7vmu_jBu4$aU zX`RmLGk)lXmOafS!GRtMRhZQ7P?dr${?P>0oihj(yo|KI*?;0|u# z9&X|;ZkHN71Q>tP25ZXBZnQEHRJLyK{%-INZ}A>)@-A=lPHxXG*SD_a#ckY+W@VfR zZOxu<&aQ9zzHj@^Z+(6)?e1^1A`$!!aQz-|0l(_>mQ~Q6Zi&TS+&*sye{cwoa0#Dq z3a@Yrzi$aT%|1 z8ozNF=Wh=0Z<@AoAIEVZ|8Y@9aXU?L>6UHg=4}&SawczbCx3D%k8&wDZ5`k7>=u(L z&vGr_axSlMBCk^;?{YC8b22Y;Ge2|xG}mzDv~vIM?lgaMIFIu#|MLC}b2-0rJkN7I z-*Y~%?xKHTbM1EXJ|A>KFLdgrbN;k*Ltk`8Z*)h0^d(<&Ko9f}hjdH7bVx^Zwp4UX z?{rW9bWjg<*Z%WKpKL)FbyQDvF4y$6~YL^G)l;&d>=wz>UZtr%{c6J*a z2ugtlZ!dRqKlfBO_H3tSEJt^DpLTGsL2N7pL(jVdZiC~OV9be+xg4Ec&z_=un+stP9}XP`UxHrnos+h zS9`W!d$(_UxPSY%7yClj`mJ}|bI=K|r+a_C?|Xzt`k^=b_vIq1FMPv4e8f+D#jkq5 z&vUy+HmGJP-3?|jeye9#Yl(I5TMO_9#Og~d;O)n9$qZ~cP5 zd^CUjW7~P0$a_h9a_s2wBgl{b-jPeQ+y-wQjIY}(keXVb22`!?>}x_9&L?fWAA)pSrJM&A%zuMcp-)vYPcbX9eVg7 zh#`tNB8erMcp{1^s<X;p5x6Sw?kU)P6sDx?3IVh%?YRVm+Xh}FJ zsG*AL=cc8adMc`^s=6wxt-3mup%A+F&ZV!~daIzFz7ye+N#gn|u)zv{J1nuq8hb3V z$r2i?ShLa^r?SyXTbZuA%G0Z_(`vgdx7~XCEx6%|OXqsc&iYDZz27EBW2`juX!wqv=uCq2eJn?`%5*+V66l=UO z#~pk8G02wnTb5b<{TniW$_1Ko&cz0&{Qok{G0QwN&BH=Wa>)|cymM9twk$8tK?^-J z(M21r+0D}RC(6-9i?TAyW|4()(^Xr2HP%^o?AoksiNdtjG=I(3&taQ=Hri>cjV#wV zdkyy5A&T%_H{N;cO<>X`5A^lk9Dn@{-BK3}bl{0AzBuE5jaLP+(tpG4c*28| zEj87YYrZ+>ogdw{(%*aoxaR;5PI%PTx?DQzt-JpE!l3&tI_a?2dwSuj*ORgBz5D(< z@Y2f87VVP<4=(P7Prdu`%{%`*^!%FZ_duf?AN{n+N4-4n*n9sy_~B#6b>zj1FRk|P zu6{oI?YsZ}*-qcc!-`L1n~ky=s^U6=mzW5;{rg`020Z#L?gj~0dtIs9H$7w1X}T8i0dLC z11ZRjebH@Uyx@7D0K^;^Ln^OuV+XI-nP+rR1Mlbu6rped3Ft9Ld6bGKIr*_FF3@nh z%iXR`e<^HbqM^+NEn8w_eD|d*< z?U0cHIY4DphT*|RP6Zmbc*QNCQ5E(;=K!f9jaV2wuX;BSo~irwlI8g9sgPobdPOZpO|cFcn@Opyd2J{Br} z4X?gzA!6Z*b!a0zC@>IHfWpDL(Y5*QGpaU&vfQAK(2i}flS=tr4 z@^UIR=4CUXRbu4wF3T~VuygNBl}M9$DrVL$ldVh8sK^5vP<``+-ONPi?l~%~h9ak> zV&?jOtIri3G_G?E=pw^1JNZ>JRqkOHsN`4DQekyK?MTf@U)j>C?gvoGSrxi}j=0Pw zMl+?WVvI%@;mJWnHHp{%JjZAQ`?_xAtg=0Q?OOLYn5Wb=yyM+kUdt6bbWK85pvqv& z8r#(!tnZ7heP(QH`IEDq_Kn3Ya8OG-&&S;>xI6jDB{y8%6Ark=Tg@$7zgylO|2RhJ zjo{T#Jls@CbY2ra6=W+sQ=ux*0Zklt#iHWUjI7SyZ(%C{L|^I75R3|%z!9c zMa~QmDWj`mV+`Ls=}^&m%blwMygGQyLyrnb8LnpMMqt^UK76NNO_jiZ!&>N#9XOxc zP0@~p%B@g8R}(>W>> z_d8T;4;82i-qA&eN9kCF9o3CeHR1JS^HUEvM?P3a^JXIo-w4~#Jt1QfgS`FllN)I?IBkqI8h!{mH$b_ z?L~kf+yW&)(%L$P29RrmA!7Ye_WjPb?EFgc$(Bw5;29ApbPN2tZAq8e3D{UYLepUx10bU`X zO*L9S3{?vb+&C4S(COc2N!r8h75_;^9VQkpq2bZ#k!IOc3X~liGTQ?livxn;y4gygKh+i~W79^#B@d%eFwIRNNSyjPdCxRIlh?z6xmdd?@fogh3JvWRsXN7$%hgB#9EnQQ0bbkRW6cCkosrvYQn?-y_nC)P14GSm5?0 zV;B~iAsyO()Wb8F!UdVaB38va_>(b|)^(T#F4dhkGNV?1bfX3C)E44ZSIEONoWs~H z%ppSKx=dXmCjZPmOdUP0QY3E6iQjW;sfNt(4Z}z5d z{w8n+r*IA@aTcd>9w%}pr*bYQb2g`QJ|}b{XKdc4pxCCmSSNKZPHsjgcXp?DekXW_ zr+AKkCwVgGbY>@=V5dRWBziv2c9ti6#;1JFCwL zD1jEJfzoGw{-=)k=eIDZg04+~9;k#)D1}z2h3;m8KIn}&s7kV@hWbu~UZ{wUD2bLR zcV_5^5(tMjrH7^{?1ZR^#;ANY+eQ#S=oyGlW10Af;yPs6BMT z5%A^_)PXW=%szBNBDiT)$Y~;ULRDZx9n?WC^n;$(0XAesF6?Qb0?0lj!krF^o+g5S zIk+a1b`%dusiaOSr8*~-R_O?}=w-Y^3XrKI;*qC*g%gDA&u9YjUdlC5rf?0~3yBw?@Uc zKL0LNRBk_rE_$e~$a041o-DY=t)a2(-M%jDj%eQMZEEoC&t`=|sS`Fm14Ctu&khtj z%37zw06U&)2-NJ+uHo^T<7GG~!{UIhPK7!YEZ6pfJrt|5K5JBHYc42%!|6%|Hf(G6 zMsKY$ES}m!=Weg~dT*&{ZuR!V^fs&z=%zYoEe@o?_G0hoN~`sDYt`C=GHfgRa)mmG zZ!SddRO~|lZ|mu%?o@>Dwj%IUEbsxRul1UPuwE=Ee=epJw$GADr{6pF>h``qsFgqqHPuXCKh8#({2H_erqRKG1_YG zw0>(McyVv`X~#bAZjNzPn1j@^>EuGiq538qV5vce^1W(2F@^P*h3ndu^d&tTWf~{t74NQbjcX=H%K#9`~lh1~MJ@rmpUT6-%?m{{QCV zwlilOsG#O4A;cXUO2G)RB6NQX2@kF-fsv?!1AnF4HO;49C6o-)6ZX%L9QFhHT<4ug5+ zK~aW)dgXyR*27L)*O-QY5nNX;3vUPzZws^Z+gj`QUWMlF<_>H@F4)7GR_v`_^%O%z zHE%&D=RzJ=^`2g}5Y#~;)T&kQW+G%YIA=-{M=VsNHL+p?vKDLDIxNQ8!&qCZS(h)H zcJlKwvQ=z9H%EH!3uQo&(8=l^7caykm)m$GE-*;!@6x& zL^TeetrSPEz?v+>p0*mB?RiT@dPD1EX9{C4F;vj?{aWu;7ixQZ?$nCz^G@@{);CqK z@xu;ltyVMU&Ufm*HGa=_!Jcoyn(_8twPUw!`a<~oV#9=6Z^5QP`bPDyDl%}MHrn(v z4XikSi?_Iozc`G?xQh!x)>>~Tu(%LZwdcw>kN3Eb|2UBMxDeFuZnwC^-v4-zC%KX@ zIg>ZJlRtToA2)Izad=CHOPext8}64G0~A2PvYj$k7z3FS!Bn7EnO1kcQU!FM`Qldj zWiTv(YeiJ+YFfv4*IxCTl5XWzZk>;Fo$s-Kpp3-|M1~R9guBS_J0~c(#ns{%bc#5a?fk8Qw zD=an$!HchRM$5Xc?>deXc^&XNum`)a4?C|*`IJ+Ja$m(M2d&|j!OB<#m|umM_pnrd zG&laC`F5wWB__9dCYC5x92dCc@FPxXj-?uJZh?$Gok3^|q=hj9YfiC&J5*JlA)< zk{kQ6OGdIod%LD`4!eQw^+U8<#k6l2yXr8uQ-!uK^>vhbR;YtIbnBsmZ*gXSEW4+H zBl|mL3%;NBy{168_s*}w#_zS>^%(~^;fM1W2mDo(^X4+D$rER?rvGj{3#Xn&zEuQr z=LaV?-1j&OGRU*KtEY{~d;N;1yo@`n%F}?3cdajMy>2H00^38-zdl!Yf(7@3GHm^d zzcDb+{_B6QGMs!O1b^oCye}kwa*M-(1gk^uulSJz`aW&a=+_`zxcN}*oVDW zjD63->n&inD(ApXt-ZF9>4p_;8AzWo+`{l013N`UECYn8egg>>G_L>c#ooRx60sGGC=p~xkZ}6h+bA+*E`o_9svP(crAU$wY1XuVdD9>qiJh=X ztV2*E$4(Lxd6YP_A4?oA2P$PcFer|W4ejZS*{Pqp9Wg};v1f1p>yeB{xQfI^b}3D; zIK)ClTb3i*rvyJD9XM1bPQ82i_VxQ0FyMRl>dBM$4)M8j-@ZW;m5D}V$&)EpwtShg zvVr!FV2->+$Yjk4&2^@KAw-?SeRU#Zb*OBzYlOKb&n?JVcEQy&M$?X*s3O9Bog|NL zNSwxz)CqlicK#fCbm`NnW6mTB8e_$X-x;nqIBdpiVz{b zJv1`Ee!+9E^5mF5KL~>EqwgVO=5vFfApTpB!4O0tsFC>uD(^ji_LKoBw}X`0VxOp{ z8pxcjcIbq}5>3oypEN)^B9UCMvB#;ES|TXKiA=n)6WiJ&5C1PX@OnzAPON&WE*aA@ zaxM?mQYMj;Ktd6%L~t~5pS9YmyyL}VWT0pKnH~ij^Fm#ESFAv32vW6 zS_4WM5)Wk(8)bqsM;UA)?d>x{69tMj-fpW@nMtz>O*U*uycE(sb={TMUN5^YyKlBb z%scSnJQmqxm0gzEW|^(9y!i05Z`wKy`UZ;s@*6O}hK8Acj|+mbZLq-y2g*Z2^Pp94 zAGF#+$Q~hx>PJa}w8FuwgZ5FUqI+@3Fvg~49O&MB4cZs~OlSG(iLH|k?dL)qwm>Uk zHIo`x;3I)L>BEE6;a4{{;i%W3k2m}B;(rnPcv7Y8B3K9^Fe(Ycy3(16-YB$vA;)DKe&-V^r2cjVp9hs%h|r;K{_ki|ND_U;J0>v^$5p zV(%H6^2#m09P`X=T6a8Y33Q0tegsL^AQ-0YCv=B@zL75ng5*;~~<2N&y!WQV-S{q#maG9C@;cp6vPLf5S_I z^Ei?}i5LfBAPE&n2ADC81*srx0%XLba>apIWnR?(3rU77hDanGtbFh)V1~p*5=J(v zdsyM$9Y+Qf<&nf+X#kTtx);2)%#W0^qSq-)(!KPVWF(H%WwRJqr*|L?PfJtcWDMqi zi|LFehdS(5ZfGBZOb zGATn&8Zc!awzAtW=~++cFi}{LBi0j*wM&2g6QBWQPaOl}xfBoqp$SE34g`U^X+dK_ zf_Mdg$_N87Iy3|+C<7PRc#v^D!y(dtYMp*q7ezYyu04$ONiK4ch_~QifX1ukNLUKV zJ+5UZm3rwGa#4}tZ6OURx#O5>3X(nQl1;=rNdHm+*^f$kFR0_gWFHd($t~ol6HaoC zRdY1FeDRNhAQ1vq-!dAnb~Pl+!RZm;u+6YaWUM*`$!L<=qE2)VXadzMdU|<(v^y!y zo|2hLGAYAWyP423(LBNgD;Sx>h0rzT2+cI#S(8fj={fGan^S4PLTM6nIkXv#F+oc& z)+py7_UKe#S=(BC4y!x-H0fSxTU*=SmY{&~9J&63pwxYlw;>n?gc#JVg8-KjAuVY^ zUU8uF*AU;%p!=Oqm#jR>Tw5Yz#z&kh8QF#>0HAK4S}i#RK0M$2m?Q zbo~;?;Ql8M812?A{KFtFpphW;a6=jAc#w0*hX~3ILAVN1hWQM)cW{CcSAe6NOd5ns zB5^BJlv2PQ0LPYNOT=`I6XuYNS(o)SU~19gvxp%jQ*3@K`6lu|FmqQTBa$V(hG*15 zg7qaYWs7a@Dv~X9tZPnxN?$G%eFR3!m#BHw@um@y;9vgrF)^Tls7XC)QFb}LTG)MmV)Yi_umz)+MB1c|fjPeN>Y3|s@18fXD^#>uZF|eQEet~ zrW@@yS9{vozV>mid!Ox&`P}JVcPyg+w6+EcK|^sOiuok>xp=qx;0a%N!-r?)poWaw z+rH`#b%NEo7M!b9JvG>1Th+YIFg8-1bs(ZTU#pP~aZ1g9!rfq=)TqMYstKoSRG~WT zq0cs{T^aSpGamQ1*S*`}UgLfq9{9mO@W6MOtrY0HCOycgJkUdY;Xfbx(a&+>AGrA4 z@1EzJE%l@6BI87dF<1GeyD|ILO zHtN@(I_#!@PU+gun{v+pukQd8aMXHFa-`1zAFzs?&ygT11OTrlZsG6JLjoa?13S!nym=IPS*g%7g*{;I1d4*a0ID;@cI}~wmwh`w{TfD@QTnS1UjlA zd@Kh7?xeU74bxB!mn8&iPUzkc4&zV`=a3HR(0c5}>EN)#=%?3Wpawn<4rTEC;_%so z?q=u*4qFc)yzLDEso4q<+F*`eAW!HZ@eKp8)~Lej>W~vV(Gx!r6hl!IMG*_-ZVgWn z72QLB@M=pQ8i5j+k0#`S5*k4aDP$E>5f^h&7i*#oLv0j)u?`af(kO)v&8{Sf5e_?| z*#fa&gzhK0Mh@u;4g-PYB2EtfjS+h&@eLmhQ(nyyA8pXOW)t(z+Tw6I;NT5g5Gb%A z4}sAg-w__;(GyGY_jb`9@39}ekPGz@AN$dNAEU1qGeI8X@d$E+*aE>5eZ)t`aUef& z3j%>e7%~pwAQ0-H6C~0d6XBWyp&jATB0sVrLsBG1avmKgpV+M)|4}8;kRP?MC07zA zV=~?Xk|b-=CT|iab5bXFk|#Nl9?eiDgHrll(gTT7D31~;^@=8Yk|~?gDW4K5qw*bp ze=;enQuvIL1F_O7w~{NbXDOu;EW=VP$C51Lkt(|qE$2=vA8;+x(k*CA3)3(U6ER0IF9nk^1FA2h?=c%wGAENJ2s1G+ z6EidOE*FzBJ9CO6^Z7v2Ge>hWEmJdpPZKp$lPNiqG+WbhL^Jqc(=})FEdz1|V&FD& z6E}BLH+z#ef73UE6F7%cIE#}wkJC7l6FHYtIh&I?pVK*`6FR3;I;)d9uhTeLlQy^W zS^iQNWs^I@Gbw4(I?EF~&(l296Ft{cJ=>E#-_t$g6Fs$4JnPd#j(L6iK1)Mt_t^o77346iRnANRyOG z3F1iA(mShEOYd<>qtr{k6imZZO#gFAw-iaQH0IEM)J)eD-nw*5-xN;cR8B3lOxcu1 z)imDl)K2&G#sZR053f%DR73Uj-3V1s4>h3t6jALiP#3j64VBX%)lnyviWHSnF|JWB zl{O_+$2e6}KXrkM#+PXBF+dgGGSyT^(^L5}OHTUSjnSTk5!<6RaxD#SZ(WBn^ju(B3b{Zb(fgcTDKD4RPr1a z!dt%;T*Fmd$CX^m)m+aNUDH)v*Ogt{)m`5eUgK3>=apXT)n4xvU-MO8_cdN+Cw6Y< zb|&f-OhFU`A;_{7pz=TvM8OmU7GPn=U-?ykVkeejE7oE!HeF@X-~O>%G1g;07Gy(K zWJi`{OV(sh7G;gWUx(pg1(sk97GV>1VHo3ZNnsh*VFC7Lk=AOjR%m+`YqyqbyVh&J7Hq>-Z0%KP0oG-I z3)X3$){&x?W?#W(BX(@#R&ME4Ybz&OS+Z5t_C>2zZ>0!Z|NFLXDd%qYF>e7^L;2Ql zHRo>=S0!(CaJTSr8CMq(S8^?9aVr-e9k*~jGjcH(6)6`zC_xY!0VoIJ7V;nuK+1Co z;&NB_J2y8@I=3ic*L7VJbk8FP7=QtPcqbtQL3hibcKe}qgBKMwcRyv9cx_Zv{i^!n zBnWhucM-yO82~|QcX*Rm4UN}6kr#VA@OI7Pd3U!Vrnh=s_j2*B5^U6Mo46^6mM4gwkw_!bs_xF7BT z7W?5HZow4v?Hh){ZH?g_?DT&7H!Jy9-b9!&+ZTkN?|=WtBY=B%fD0lIbQgvh-~!C0 zATTh78Gs17r5-eZdi$XabT;ml*g`SUgr(dRurP zjG%~NI1>tDdW%>Bkn9~UfO@%q0EUBj7_wM*H30djREeP&`kFXApV*BHPm05Xis{#i zZvlq60Cp%rj2S=zI=CN90fvp>7L1{bkB@@?7=~fsh>2K+DZvlDM;S0mnTxGB6Li<2{|bTXvDkf}sEkws=d0S(#-tnVWB&;qsW%c_x=v!IZ{%p;v%=xF1BJjPqa^1o@d``A!() z8=gUTB>-~%cz*jK1YkIS2r>bJ&)Fnls{!G87vY#Y-QQKyJuKoEG@ z4bV7-gSeFq+91yOh9iIrL}88DIHGB6qB9y7EqXgK+NO1Eqy3_z2jZjqp%GyDhPhyJ za@VAR_#0aJn;AF>vN!_Tc{b5`r}I>*ZSi=ox~jQUo{>dLV`+;GzG8l&ZJd-EewSwHmM|x2s9joZlpT z)7Buuw|cq5APhuz>w0>{nUw{(mFv17?g23jLLR!1lBAIMmIF?ps@vUFEFgw;2zmD?>_ zn=fR0vNyILp5c#Gx*sZ;0f4t4wpp_m8lMS*xPkY#3xc2-AOd)KAdottMfxl*;|J5di?@s5SKb(afX!5D6#zZswr2tuEwnGyznw!f+Dz!{(dhanSE0Ep43 zybA)nBft$BHbN#GqJu@Evk<@Yx4r}8z6-C!|3iG2pqnr10gIEGthGCfHGqxQI*S3X z!mTu^3%tXd2)>gW#Y1z%Rk68$yed_^FNiFQbKn*!0j<$H4<>jZXy=BNoR!76sSN@e zo_fl$Y{yT3vd4*hzK7<-)fvpkQ@_7DS-#;Gt|Jp(p|1(z9PG^)-n<_=78?BdCgcGc z_WT*}yvsk_6w4f$nR|OZ*@#|>MOf}~l;N5*at{q67p|!e?oiPE!pP-&v@89e1ba65 z5tqv11~X^C4$&a;E4?&bPR=}-Q2iz{oz)%53+v&39R~Vb?d=_6nn;7ZD)q5O-~tEI zARH^F&?w!I(&yDJUDfIN*dM~xmz|0>P|yEwL3H!L4XWMRuU%m&;TE6)%I|K-abl=^ zJ=85{*lA*TjNRE0EZIvn-4mkO+kMAqw;mj$5ncg_4PrkU0XuNLA;#S9yb#dFDpUpx zzJxu0CCq(ABoW>n;?kY_-4{M{B(Q=TLA`0>85-dh{JcPPJt;S^yd*-Ex=kc*CgMsY zR#NLLM#bPjF6D#6R#pY!pX4AqB}bm-+!Y?;J8j(nKS0306FcXHHxu^U%irN+QobaJ z9%4wuMLN#ldaCDjNdmti!S~`7D1puo!rzT_eFyB+f0Jqp3c?(eMmpFjsT2(d;Oy=d zNCKZB5ANL|&LMSi;_B&60^4Ph;AIPd>Z%ArrBLGg1fMHhg1xwc@C)MY-wH0!J@E~q z;guT$fAroDy+NNo9u7u7COU@TLqhde|8gYI7=+s<+<R2dCho z>($HG&psjE{^QGH_jx~p?Vj$j0QO_R7xXEqo-q2^A~*^yCcX#zF~|40doRjh_&FXa zC9wIgA0fEw2$Vq`k}55le)NGh zf6UlRV`cjBtOq!-;KBF$mC@@m@7=|W8!L8cFYUsSE75kPtSqFx$O^Gd+{d+`XRtU# zPUJip<;8*-QRDDRF*8TXjceP!jXO8ZojrdB9a{9L?%~9X8<(1Cgw+@|)V;%|Fws5I zuqTq&NdHBkJ$PS+8g7UiZ$1I_j!;Ayg%pP-o`@n& znP9OT4ZggCAVJtnx1fa&O%`BBx$Kw=f&8TiWROCdnBPk2^#o#)N+u~!eDft%f8UT& z-c+PYmJw84N!T>h&_1MjR*`l%VB=jyUn8NQW=5oCq-JVU&Pk_~9{Ls{a3t2r zXP@C^qDPB$ypy9nG^QEnYxV?KR#{p432CH~+Eisq>EXF#ra;LfAAOU4N>QaK*&{?- zCb_o5SDlfxk&IbT7Z&tgNu-H%rnO`e z7_W7!%4*S?wMNKqzjCfiaKS7A``d`Z8O(6EnTXN}voX{`LqY0H8=$hgK@ML78>9dTCdHvTs2dqa?35h z>}lKfz1`i71tl`l#$~y6f2E?aDP-W#J&%_0L`c&{lC7j_+jG=*Pd;ST!tP0V<{NU& z;@4jjlndF1Vo4ck--+oKMRmZddH?IKmxOmeam#Hk-Riz>TyW5e{fP9^OOdw5DsjHOpFT!htxH}=CEUnIy5Ix)TD)lmqY*hk)ew?l<+hiS$W z7!H1T5?jT_cLf5A9b>BHnYLCyJdsoG|;u-RACqZv_(Tp~% zo*I>6BdWrzufgFR05w`iPyY%Y-JQc25iJ83pa9dD%9IObP^dB5X;GA(B%V6$X^J@N z(+g4pq~F7VH%Q7!-#97oaNL38kRHNB(PfH7Dq};7+N5w+wS4vIYEuha zrGPfJe_9)@P(TDw+UhnYuf<7kOZnT~3fCpG9d5~NTiV}q$rjessnUXh3YM(C9;2^ED(u)E#thIbfUU5O~NE8dlyqY*h^fPEPNgBQ9E zxwy?Db^j~f%z7840silDts9m4-gm(ZhQWYIe^TEFe8Vmgo-j%j%;5V*AP*Hq$vZA^ zUznYw5gA@^65gRX{w{VW2A*z#dHdjwzPQD^MKDx?P~r?nAjT8Hv4rj7V+)@I$2r#U zD^Mbe1@{FdTv%`fh|mKGELa057Og0t(Zc?omd3-KaZbhRWiG?k#!n$KC7^+3XgmWN zf5vFCeRDA0mIN8d4EQlh%$#O6Q-a9$HN}t@ECHOBBoFrufrz1zg!={|b1F71i^VM5 zFy9Zbl7_Tr$?Oy&AOG1Tgs!uYbTF zMLNv@5woq)%C$6YD{USgXB61JHY}!_f1>G=oI}c^rU*1{@rqk~y2qg=$vLF@3S+Rk zB=vZ9RvC(tYiHHMbH1~EO-+(v+_w>fwyz*wVP6>7x*t%muLiaaVf(s)B=vX(w1+Wm zgYnwG#f~kpV<_y49=zaiJvLL4ZITaHdm^_80fUL~>2gox8GN>H1l)k{dfmpCwBvmJgDeS#% zu;*YRZr*oAfX)C3bU55Xp@o?u9SwBse)c(gD6p3#4PMRTkZR&r5^k1)e>fIsBr#_+XJ70_5~#;=Hh69xhI2xOdPabN_O)pJ_k*qW zf(-U!nGkQ=MhQ^B1!K1^&1ZNZ2xc4j6dzbUT!@A8B!W0Wf;t8>oF@n?LlmKxckMK1 zX(w~A(|KLR6BQv3(9mf=27m#Vdp&4-Nf>cKND&qXgA}2L5#WAAe`a(Op?^9yVM*q3 z4c_mT*VJYE%6VZyYb6*z#1HT9Zt!H0(KoMo2fEDo!Nx%imxQsPtXMe|FFdzsM zcytwbTZN~IaAk>Fe+X9Gn2nbvSa@Pa zrhjkPa%-25@plqncW-oeiVX0N{lIxpuoCqrgt53_TyTZQGKt@qg~XAGt}~GkX-S-D z6S-)LcGqcna1_Y`l2nxm6_tv|$cjPu4zUOYnE-J8@OM9!e|xo=K4_C9k#-Egd;NfSp2v18*blHLd?s;!69yt`DTw&UeAbv; z1oxED<&8{Hf03F~n1LxqQArcI=!S)Hk0uvyg)wI?=!X=+eshO+@YoMHXlKG`lofH8 zxLBEGnS2sZW%Ou%C*hM8hI&1>Q=>Rx5k`fsW_m|Sc#hVPh{ zNu0%LoX3fr$*G*n$(+sUoX-iJ(J7tNNuAYco!5z-f7z*>+sU2XS)JVnp0^;L(Dn+i z0E&X}nQ@VeIvI0%C5W2oW=mFXZYBXy(wdWpcR>(m^jT$&$N+vu5yZEDqB)5F*9c3f zgI`JimhtFr6M=&{SQ4+0YJkXob=eP>$CvAp2ZA69rcj>IHlEqcv)yH;SX&C!*s?p62;p=~<4W`F|i*gZ<#2AwUVkkO{*0 zljTO4_EiSohhcEm4{MnNqHqg>u#;@GnS#gwP*9%6D2oId0Xbl%IpC!xX^Ka95^gq- z=-7*bAfAmNj70Zi{s{w=V4iNq0B2_~8`_~Cf6AjGIt)2#sE3NE*}0;L%BYR%sE-P% zkt(T^N~t7Ds66_i=Lw{5k&Cbgp#3m(3)YtXpmB?sVJYx?sHt{K1`RvOhy+@XqY8Uh zdH-r5Mw+Xtb#}&nB_?(Z1E?Nao`Wi)l}fC|Dx-^PtjVgZ%gU_H>a5R7td}~Tnfjx2 zf8l26MiM<>gcb1$b^2yeLW7_74eXbE5g-V0_6}~Ad@yhePkMg3$fvJ14E08UDG{#= z76M!#3fq=;rkN5C3Z#^vkez2%7kky&HSGe|`J6yL%$N>$T|AU(w6G&}+PO%e>wDyx}Xpzxm6*{p-IrtG;OFx1QU+y!*b{3;(?T ztH6+JzjC|33!J$R48apj!4>Qt0nEMyydl^-x?pR;AI!E5{I(Jd!ir16ea+{WOM#R!&nas&$?8uKi9)Uc=o!rBgOtza$%4%!Jsm#ix{K=pk9-=(|$*o+% zraZQ&Ov@IF%elpbI$GuF!wk)>0+`8vL35~$a%`6JRe|x&fOw7}~z`qR4 z#UadFOw9{y%wa3drYj0Omth*P%F$fT+lGbi_c@M&vjdG8bDEDD`gz! z9`yzT0$tG&Ez%#Xe1JIT{9TmJ?<;MHH9 zY9a8|UtMz&FxFlz2$y@+Q_a?yYt*=_X2+*%OkEpt4b@s)f6s00y*3@vBiz?-I}BRg z&o3*^0a;`pZMkc$*NY9gZ_Tr6#!^a6*SDd}nP3ce4ZP6I*w0(lHLKVl>ju!^6M?;8 zx4_kBjoPV=+CNRXpS{_yZMTlSzCC+Oa@z448RS+vVGFeU<^r}q+EsDbMaKir>B>V*`%x4);+w! zJ+s7}v3z>mqkREoki;Y`paJc+W=UnuTi@udz_Pu@+Rc5-+{^=>UIlLd;0KQ239jG^ z&fpF1;13Sr5ia2h?gnl!;TMkK8Lj~aU;rDM-~&G3e@U7by*(~?ect>{yT<$uMWLgF zN~l8n4JV$l?0vEK`rdU|v%6~83&z@&OTzjm-)_sEFd*Zy3*;#7zy2-2ew!D=@O=VK z;2AFEQ%>boZsk{Q;1!PLTR!C>z~LT#;2_=zFMSJzO5UO?)3c1^DbCj-YqUw72Nf>o zb1vr+fA!RZy}W-tw)gsK$erUmo?$)S+Is#TM=s61?B zm2T;mj_H}M>6^~!o$l$M4(gV^wV_VxrC#c<00RlobDIvaBM#M3t^ekX9=oRe4k-O@ zw~c$Rz@ZEkvNY}EUVW;V$muPVVJy?w3C5=dSMIp6aWf>8t)W{Og z@NVDG>(Xwsz^-pN8`Sau?T#qg1rP0*`{Pim%jVgxIXeudUExlt5WK_e6N)wR+ieQBkkJCWj6YYN(x2|W5<%Z{a(3Zj$lu;Lj@*+mI0U+Gh?O4IuP^zGjCm;U0s>9^%A z)i*D@rkqNj>Jp|POaJb+0?z<$5Vi{Mf79}S@bVz{f*!dI&j3Nr9)hrh@%9QHAF4TU z&cooXuV&gQkLVrW9^U|BFaQmznuGFipiV{!J^^;u-X3Dg(V=<)Bk3M~?_}GTYC)du zWDmrfJ1*Jnn=UT&uMhjNAM`}O>0FQi2_Oh=aQnL-0#lXhyPp6sVD%|*`*|?we@Uk5 zSV{>Y0Q|S#16Dttneh9(PYFhl{Nl;=UjOxA|LP`=`eond%bN$d9&T*J_H66+a3A;h zQUkT6t20!-<5Af`<=yD5))z zK<-?+5+cx@n_?>zxiG%EksIVzBfNFHQYuV00s~Q7VI*MuHS6T6RHZ(He+tbh({o9Y z8U-C%^k~whO`q=BPxWfntzEx{9b5Kn+O=)p#x0ww&tYY9uP!sP=Wf(Fxwv#w{9;QO z21DkNCrThRdCZ+NfUf*B@t!gn(3E#vU=C5k1cekRiegLufksRjJ1vQ6<1`jMVmZ(hcl<1ifXFLjF~J)9e3ohM<0I#GDsnZB(g{&fpm+=xaP{C zkw$XB3kpgac}y-!hRH-O#wJTau10PNK`sg0phS=!<_d0@SMHLFe~T%Tpli!v9+T6@ z8;!Xvvl%&q(M3Q11T@eVeFg_=|+6AdzEpi2$sqGTqAAPSJD0VSaBpn;6o zmD7kOx<$TZ!&6C~e?|mqkfLwyYiK|&EMo7wn0y^ITye)Gmoyk-Tx!P5ptEeF!F ze+T}imWL<4_(Yqw=@u^P0jeG@A{}mR6@gYL>=cuR&S<1Ry)HZ8k+msExuElCh9^b%D}2bmNO zBTR6^k?Z=BhxgoQh6K42!+>ZEI=v!>v`? zrJ0b4tcjh3W;9JxMQT>Fcy6SK^6JAyZiWYpJHcjX6lkeGu~B=T0@bF17tU{aD~IVl2A`feQKA-gC)Px2QWX`?oG33w7ldk5gZhr=^=w7wU}*IKWzIFCQJvpo zYEIzjDq_`Sp6@IxPV9!CJSgZKPZMN6f7>Rvxdt?w)Kb>0F7gaYZ~>wIkw*h9S&<3_ zZh=NcY+{dsxWrwwMm|L%NF_U2%2u{2A_ZYeO&Y_>XqJd_QBas9(6~(aQ88P&OA3ey z8J!w~r?`Xc$AB7Cp?XHKx4o@Wb-P>Ju>aGkO_6GFJ5$x@r3iXh)#_HgDynm`e{r2* zbsD(PcdLs)GJQc1NZj72fe=84p=Ed$T>Yt&xZ0Jj0DT8S=hG5BvMQ)=e49iRC_n~+ z3!=EiZ+>6ISWr3^qp<~WfCW5Y0vFi82S#v$6}(^oqY%L)%_xMCC}F7(CIMV{MVs1n zm5dr=NwFNp>8|F}Tij*D&&cV3f0wDm-7(R&`h~GHc{^hox5l>$1#WPoI^1vSgq=fy zMg&%Zk*u1LMO-}^I=k0K=-%XMSu12ktQINu7~vuf%tJ};IY9Z9m%QWkvX>R;28dXh zB6w^sMPfBIqs@mT_c82=Vm#;DZjm$pIri_0|I1)L_u0>X26UhaYzk96f7nV9KJwfcRmVG~ zYL9dBW1)CR$PU4yk&nDwH741)VExI{tOj0d{k4#oWPn~{a-dt-Cyxcm=7BEd<(z~W z%;N>gec;9tO)5(w>77vff2^Cy!=l>UM748c)g@|nx7*$ChIhQ>J#Tu~+urwn_bXmO zZ`n~|nf?B^m6mDCy!mgU(Cq%930nFbt?bY2xzUacy6GZjSqI)gM=? zZ^4V_S8wJWJS(nt7M2S57_2SPMyAj;7^ov6S%@k+1 zj@8b^vseCPAjf<)f7#g^bd`K}Cl^i_n=G0j6L84lg1M5@$mE}35f@bwse?R)2F}3scAH~e~Uy7S| zgJ-ZP&s&OTFc_;+j(l3A(QA>G2mvPvi6m%*tPzAK!8r(I06}0qNm-1r5xa$;me6Y- zo=b=VLLh?>g^)Oq0y;VQ2r5FVpBKTSC}SHMoH`q<4}@ZfnrH;u%Of{9K@{}CGAJxm z>A?L%Lg3-1fBTcZ?sLK?gu*D4!YRCDRW`kOu`?4~4?Ll^PCIwT%;dImud1$E1~r4R){cm@P4j+JYY;}e(M(>;xd zfIxh%(SQbHc*G#A2^)z)B>zel9AU&jiIp6PL3wz@e^11TN9?qTxUV@h#lhhwbYa3( zWW`o=#aD#ISd_(Cq{UjaMW)z_TfD_@NP_fWLTJbeX!u1n(W$1OKV5XO{5!>ER1Q0I z#+6|wXb=U~QAJjKhhb=|Et@q4G>UJKfE*A6LoAmv(6&Sr2o2D{4RoScaFTX(M|T`7 zqu4vse<(#}w8wEt#cD*xedNb}^v8b$$UDo$fNVu$SPbI}$njwaWX#8VbjZ?p#)w=h zpEw6&XoNh-glEvlQhSC>$b(ms#BEd#rzo6!94eY~qPI!J3+Tk2pbkvJpV6qthonhA zF-fH$$eh&4o#aVf9LS#kWD3pbMeHC*g*1j*f4s?-tjVQ3Dv5N;-(aVA_=aI1JTBk@ zD8S0B)Jm(&gDd$4rL+iIW6Bi~OOg=7gI2TApdxt@hp>Y^iA&Me@^0*&(|o+!X%aEgwN7|&iv%h{>%yU1keB# z&;d=#@+8m$txf-Q&iQ1})!0wecu)mRISHlE3LOdr#n24Z(AG524fRmUv(Vsd&<2fA z(K$ zRZ|#4(>484F&)z^RgpH$PdateJH=CJveP_eQaD}AJ;ez?bxuL$(?T^=-yzgPl~F%c z%tXBiN9|HMRn$rUrBsQ6)JhFee@1=EOI0&TWl~PX)KC@G;sDiA?N3d0%2F-Wxm;CK zb=6m`Qdo6WRFz0o_0%P8Raw>5U47JBNm8 z$A#R;U0TPL+{(3FsHI%p%G}Fsna$_bx_|G-r_ah;n`iyBi`e+vE^mn=Y@aX$W7kp zrQYft-m10U-p$_aYW z;0e~?4YuG2=HL(Z+6+Dv5k7xl4+h~AM&aWz;S^?JrX}G*d129w;TERh4W8i|#^DjY zVV7B9MwN$5P)kY>1sdJq95&(rF5)9r;#%$D|7&6~H3jt02n~1!5`E$&#^MQ1;w0tV$>=?&35);w@I=Hr_2ZF3vC}9tZq}Y6=NKa1DRvKw~%NV-$wt zJ_h6|`s2Zs`_p2!c-b66H?D<4rAON>=6ImE=`+WgfQVl`ThIX&EUtN0)$sMor~c=4IGzKe7=oo8Hf~MXq8E523Tm-s0V$!1%2*gf*u}&*64lyje38O%4bNXudD~Cd1{&CL^3v>N$pKjfiGcp6NX5>Oyh0pT>=6I7c!*X`luc zp(Z4qUTP&iYLbY*<)U5|;Y=l>6>Df33b`%9@XzSieY1`Q9n4TgBbc+b21PY`C zJjyvrSY)*w>zbD6x6bOBkfO7O>ratuyw2geUYWZV?Ey~RcNmN+zKv)25;DGxK~8M! zs3JY?heZaUI=1b~*5%xo8F^3*3VDTFfCkq1hJ_ghrS^X!*q|BUUX7_7ZexJ$*LbT} zNbc2GZq+dD-++dNk?wx*Y}a@Oef#d!|Iit*=?9u&jlD*0jckpYfsO1A?|J|b<~i*p z-re60Zj3JO=SFVjF6{TV@8>4Q@}_Pzxo*|S?ta+r`_A9g{RW-;jaQ(AvL@sin{9Zm z?MA-s4D5f0-0p|n_HfI_4Fw9jjffHKN(dtOYla%)*7%0S*pP4jqlAEhzg`V!Q1K6W zhN!`81}EzYS4j)0m55;IM(8<#aD#JJKqa^6+bpAe_UQF?m7n%(08$7eaBw5uT@zm` z(L9G87YG(t4HtKf7+<88CL)cv@z&7sjp*?p2l9U(-}3uS-Dk+M-#CXnU~R+ZtqiXw z5yxW?7j&5}aoxKBTONoU(3u7z0@wer8!qSv@8}2(C~T|ghzt1f2Cof=b_ulZhjH78 zlWuYbaDy^;2}##(IbV&S2Jg2nb)v5G7VdIHSM>2{^f8C@aHjMrzH~(2bXK2r_f1`6 zICXzQaRd2fV?Ymd4##bQ&~0ZQ@!mf4W4Vf<^bZ^vLk1XHXjllPW+-L;qYIz}V|ayb zIb~-HpAAZvfO3BYY&UY%NIrv6o|l+(H~1q-a)TlE^ye`KQHY-zpL3-S zZ_j>(sFt>>9(Ee8b!#66d&drKABb-M_SSz0_kI|6dPn!*zK;lC_nVV*c#rt-Ep~y< z4HklBrDFDG&*csubZ8H8X%A~@|E~6F?}wFB=GBOwS6Fq5VGVDmXpz2;KJN#I*7AN( zAckNKN)L8AuMHKK2-fiQpEmjk$mrDopgB?vwKjFuQ1MJ?4LN#uVSn{|5AXCQq?CWJ z;fd#YpJ$DrA9_kC`g25jflzv;SB07^o2H#y^5P_h#t;lhRwBTlS%@lvo)hn1-( z?;YjJmM_1PSFg;lr;ZbIxhpzj9>)EkXeC z|8qmO;IoWD3=SKukc%jpfBm3E&{(u$j=y}Fipu!XB2b@lm5=t>a(hE#jf`j*jk6!% zhTS>7Z%ec`{rdLr zn4yOE`L`5glVKL3W%6+5Sy+gj1zu^U!6qAPuAP=zKdWtMREJ9uf9Ts<0UA_XP%@om zN<+{j0bT@QB^T6m4Yfxhj>BPPKp0c zAsU!G)hV|ki!##Ke;Na?ae82yl^zwRopZv8YCo8Ul>!42$YYT_!wAxYXcBzm*G3Uq zw;ofkXlGP>5;h`ACdVEtin2oXgJoOkWdubg!w_UduX>FM-+b8`+O4RMs7ulGWw{%eX}W1_ zGal$Es1MUPD#HJ%dMa-`8k8Lac|a3vLHAHrkvu7i;uc+5LNplZK%Hue*rx3z>!Yc)Y(^k6G0{gEliqyEkgX_r!i72Dw=OcEuhAN#|cyj!bR|m zf-YGU+QzPaY|&XR->a0@8R=HhL|~p(hDB7fWHlvL7gSGr>)FIW<_4uc^tM3lTDI)7 z(~dOYa!Nfo$6p?xKkc>?aMRYfBa&%r>*v*Y#Z!0(_W1wH{I18 z@34#gNX+;57K@efxw#vdQSANrm^gJiddvW`1$lH6*iShw!Obc96mmv$9|Uq~;Thu! zGCI60DSufz3lthsKQIvuC{}VF1uJMl4Bbw5Pa|B$(gz_rxvwFZVoUty*N^`PChvkO zTpc2S5MO!10F zEb%S)kUZQ%6r1ByK^(^bBZz`l5Mh7@oU^l*Fdzq_xP>4R za+f@W<$*!C#VgDR!PnU_2V4+d3kzw;Lr!Igf96}H5k-?kgG8~3PmCg5sCbtvW^s%6 zLnJ3V$-LVc!wriHm<)3jDK}gXDH)W?IihnD0OkjXdYjnz{+3Fl5OH^Y%7_HcQ4u(T zrx6B$h8?NIMW@h=N@D8AyY#d;;cZEop6f?6mE*c)j&Yfu+$J{-smR>r|59_j^ksRJ zf5`x07KE7QJZ3lB=}vKhG7rx<2PtJp&u8GVhJ#5FFK@}1(1d^zlz>PQD51sKP@*~k z9cU2EltE7wF`^bZAQy~6B<19wdTsF|141wi)VK^+QF%r-89)SMBxHU4Xv7-t!Ub}s z6C%QUhvV!BgIhdERPR6y1MCQebJ!{ge_vW3G>G6zmA*5nN+sJv87a|(0xy+TAX9S>o2fIJb>nk#2X(YhJC~P9D#o zL>@8`*xRC2vYE&OC7uzhgAsJKHfBWm-|3a6!iFrpocuJeDNH=xJ#p`(+>|js+ zmMR1`aA3&$;0t5;C>cD5G3HwaENCTGiLg`}aVyhaMY|$H5P=JGmWo;%abG$E zI?%S;Ggj`LXF@}z&wmy)quUAEm;gA^ajtTsqax-?V>;8Fy|kt~t!Ykk1=L>_wWmvM zYHuR?)T+ibsk72*BENdof3lu+*JAX=6Lvin+G7yInzU`xe@Q2Di7%ZEo9$``qeAB)RJh>~^m^ z-txXky60_g3c_2J^`>*a@9pn$_xs-h4^+VE9qfV+JmJMgxWfM%f1V+R>uld?hPcBk zu4;-~JmasGcvmoPY>sO@DN4d~lUUjKseaKPIdd{hS@T+q@ z>=)j;*jw)PcZ0p`7HaP<+1D=ev;Vs7Zl8Ot(@yuh|5CA&=87_Z`Okws^r9a<=}T|= z)1yA~4^BPnTkrbU!#?)1pFQnsZ~NQhKKHucJ@0$(``<@?^~R6~@r!T&{Np1(`O05D z^W_1S=0iXF(w{!{t8e}5V?X=Mf4=s+@BQzCKm6h!Kl#gV{_~?h{pw#o``hpS_n)8p z@Si{Z>u>-2<3Im@;upUDkuU%L17HA#-~S0<0UF=|B47e4-~uvW13F;y5#R$#-~>`& z1qvYl6<`Ef;0Csz26EsBf?x=W;0Tgn2?n4CnqUg5;0h`r1_~ewvS17<9}LQ14cg!h z;$RNyVENJD4*K8^0^#ww;0X?25ZYi58etMD;Sw@o6XqX(5;|cNN}&V}Apk<*6h7b; zV&N8YVHbK~5^CWWis2ZlUlsOW7?L3orr{d0VH>((1)|{_%HbR)pZ^)^UmVgQ{NZ69 z@?jtP;U5~`9s=SZa-khQVG$DI86sjLGGZe-A{z?gBTC{A8Y2Eh;v{z8C1TiemJc;wh@)Dzah+E@CUfVhoa^{iR|o2I4E)VlL|9F2)}&^5QR6 z;4Ic(E$SjM0^>0v<0%?rGBP9j2_yY2V=P8vGg4zU{^2xQV>V{rGtM72vZ6O?V>pVV z7lPwBnxpb@qaOO=ITl|zvg13#qYk=bJep%VzTZ55h9W-F<394^29y=52)o@ zMn?#&>UwGG=3b z-(EUq9{Oc33ZNT61Mvx_Xxzg1rR7|D=4Z}-rBHTcWRhlSKIUSYW;0Hv`9UQ>UFNL# z)Eh!(1HOYG%og@RgB#=k5kSE=l%FW%fgof?Y7!?jqGoX(%;Bl^>8HK=W3T7F| z;cQBuF(}Jx!r$d+z;IGulH3G#il2nM<~xXj6Cvk$QYHslrzn6Ydg@?BzQZ>V)-ce2 zXMNfyeHsHad;@n*-%fJWa}r;Bj-ON>rf7(!Vt!!ihywkcLrU?d^jS-Cy+!%i4Do5_ zgSuyiW*>D%pMq-W4hrNsJi{xnmm8SqiK3{9($_1@Q}x*-MKt=^!~94FlweM*nZd>v54;J2ulzJnZ$E0<^eP zS^B8)y#mS_2Xummuug`8i3SW6pLj;AtH`W8*i8J0h7r_j(1u}>H0&0CmF#4shZ~T{ zcM{(@SnN$?tnqEEa(HZ>g2#?J-^enNp9ZY~+M_Y#0hTVsIp9&h+8@9wt-xaCr4p>c zBJ7PiX>zy{c;vuvTCE$H(S%wpc%-d7Jdkp1=iZLSj(Q4sXuzXVpN}ci&YldFNPvC) z?Okl=JG7*7yd?1%DXKbuDCW)u%}OrUb}f1itwJbM-CAfu;O)g;rr?f7-$rhg#3#tQ zEAbUB;3975w&0<5=QKqHXP7Php{@J9t=q0Grz-6K^m%7bQNl2YLQ7t5kBSBiB*A_{ zP6P-7^R{FrOa|m!KqH8%R0^jXB=Kpx~lX24Md%$6ubs78Q)uJJvP1bDA0wC~Cm z>|GcEsVbPT=4MT)Xl=R_lu-iY=nv;o?5gT1&(4J@xIr1XQW=Qv`{FKfj_zGt!16Y) zOEoRXN-y=^#Pw$I|7>qYbno|y={txo`I;~KUhMj|Z<0=M?&cbE-U9DJMHy@+qyn!p z^{v__Cff=n@&2%X@j74yyRY#XNa7r6G35Yu9#8=npMeCf9WCNn04?$5hVk*#;~L)q zp#}AAP6*&C|71qQ&W7=+ECWO>>~QD1L~X6Q3jf7s?grbhk-|<4YpW5%a1`H!eL-&= zIWaGRrxeF-MpSV|Tyf*7ju!J0Rq9a4;_(66V+~(Tq*-!*J#YjL|L*$*@B0mJMrdmB z5^S8tZO`N^SuC;oaOXQfP7XsXrcjF=X)mfmK|vKCp+18vnF8`Tj_IP}UAS`bMX1br zj4v0T$6Rr+w(+a9r;*NaD(@_xCU7J(X$2q2DW5U)?(qmO^0LD6-p;ax*76(UvM%$o zN=Ps@zuzQ(bAe=3Up;U^Fhid{cCsgnau9>^!QM06R$r0?LGm3C6r}POBk)xI)j@v) zPn|Ey20?u(AECw!hDNmXCCN4ypQp&|SqQ@aK_7HW;0iRavblmS(<<-*({VJ<#Yv~L zI!9&&|1!(7SFfh-gMUZ!-JUvpo-S z+vfAzl5#&&D)L2ikA8IVL4#p}hjhMf1V{^i5~{~YC-U{ENi&~VC!f{I?9&?4Wh%4A z3Ickl$7??GTbJ}}+I3K8Xa)Ch@l|y~Ty=nIwQ_Lvf_$~{g*C$_Us=njUeg~tgGf<_ zh&&*FwGOu*KecZCeKG@ta#JU45ts5A`=3wuDf}Mh3OZPAco@$3>`N-)4!A0J~& zU+R3eO4rRgx%D(d1II21U8l71*|jv=um59^{V=ZjU zws0qeSsV9b>+ba30vE(kDVap1RCaaxYxbRgL*?E@Uo$oFVd_)wb5t*EY$xAe7x%}q zQ6^j$B^(PS1oJu2(a1t^V$(NSXLn;GpQo@le#>m?xPg9OSAPR*0-poQLVRcV^q)W_PxGk8-CXuW?}L0U5!fOf!COXIigheBSPnD%g(pbVp%mOE7Np z)z6pHE>NGv8yjCqbog*4$u%3Fk=ly?S$p|#61SMwcxk$HlOr;epLjw{Ir8PoWT3fl zXL)NA^sFRTmjiX3bN38WUyj#PDUr8-k8dB4Qy+9h?T}A(k?*q*Bl&GdWN5SbE88fD z*Ud~5pW`^IVn?NGm$Pd#avVu;R%gU6AK&K4Y?wc&kHD~9`&eH`EXAe{truVG$a!#= zwxXw|HNPU|j(U`r`Z>!_{m3{^X}PQCva1&#PZ=|^yB|C2xOhi;QlDS;TzYkXEI9lb zxjuU~KhHNC-EQh!GpRrEA{s++`D|@~l(e7CSN-Bdw|zpW{4l z0{d7GtaAQ5Dg&|XU3fTN^Z811JO5G|@!=XDybqte_fsI+`!nGCA~HKcpM}3~alp$@ z#>3wv!_cGKlcY%c@`2+GUAmut2cj=GWEcy z`ccKxaU2a2#=AU88u1t1g3xO)+Ak~0mI2aR?9xZQ)0+a+_c5VUz17cuZ5d?!*5iEu zoIqp0-fN#d0*uVB4fTk7`Q?;CnEB3cI()CYd@uRjV$^Vwx_ZfOauH_oDu>P+x@hdE zbF;qn)o#pkr{{_W?5p)~vNCcly>DAn|FIu%HqkldYvt&g{%H7e>Zh@Ch;iC06YRVG z>?>3BBR~6Dz8lOF<_pX{xxsWdf2sOmwggc6jCRzUqN$paMVqGonI`w~{`|+>g3DSj zcOLs}v$gv2CqO()!hm4G9Jk~pOxQ>R!3f3_K7&H=fJc$%7+C96Nabw4iAxG}ok8=NI%a<`{*1VZ>XV0HOhZfB_E!Jm(M4eN8?ptfu zuUDTFCFmK}(vEXD%+ctZe`Kqo{s#XWp|p5fJsD}kVY&+HQ&w5|=?mfM{Z{+Q=FK?WFef#(E=hwfVe`DlUm19mG=b+rO z7wgc=FTo5`v?`(Uu8}-4d8Wh@|5tnGnN0F%WIn6N6sxr--^o+YJ^5S`$~rl^l1q8= z$Wf&h2TfGbMHy|>e^Idj-AK?!DXr8}{lv^v(@i^!)TQ?*$%6|npeWVUQ%B8%l3~8_ zlOs4OZA;iPehMglsKV*=3dG)Y)gD)$CY#-U06x zQ$)!`+ikfO#gtpV`*qgBm>qW7TL==c+;!P)*WGt3JU3l z01jB-)>IRBTzf5@78*f#L)hVmA&wYgg&D3<;)^l1P~VL?-jLsf`V3i5u0GC~N^(t3 zS>=^krnKaiVUF3Vj%ltLw3q)K!&zqfbl%zLpMehAHk5@f+USIAPFm^9dS0yQqs@96 z>Zz%&+Fq%xf6kiOrMXTTYqGx9daJO-F5B$02PGTrwHfW&?T*z3D{i)-nw##s@y^>f zyY=o{$hQH{mv54Z_8aWN4NqM0#U&e@@yF>4T=HifHx_Zo$G#l%%{lKps>(eNU98DT z*A(=jMJGJ<)mg{<^wwdgs`S~xe4S3sW6#L;+^1n}CeF@QD-*55Z@2q`!<~O>2 z_wmmkvitR4JO2MOGe76>5%e?eHm4~9@^8*E?)zc)e>t}uTpWZ}+C2tX8a&xJ3v;r42HLzczxe>21(26Y%j zA-V{NL-ZUELU+no8^DHfQ%uaE_Cl>wdqa!K~kLVl&7Y2X-_YtQ=kr&sQDCXQIU$&qjGAgHEn8Ap&C_eK2@rJRb?qv z#e~(BYSpS;^=d7-`c<(`G^`_~X;I5MR<*8`hiPT&Tk+{upQ=@^aJB#IT}xM2yzX_J zdR69H{mNIt4t8UKCG265Y1lEHm8gnE>|-HYJjO8(rx>FuBvUZV#)QDcpW$A|d!jzIH1J%uECZ(7o+gRt%+U*Hn8R~L=j4o0;1Ik+OuIGIi3w6(3kSHvEtctq|6Ls8*s|EHfXl;o zb6VhGAb7$(_HmCZEIk;@n8-dk96}Oo4r7h!UKznA;g~l|dAzF`WX@`sGc}G2h zTIbQ=dA>M4hqG#e;=crW#iRK%176{oqcj?D?;weyM>CK-^Eym_zK#j7A06pW&so#B z)-?l&y=-Prc0jktavt1}ZEbHm4}ze?EtE2v4YL{~2Zpt*gG2voAy4+!x$X=UE+7F< zcg@%-0Cs5V5dvtR64j2bh`lLeY#7{I+DUG z_Yh;>af?RyHZ;#@#4XIM@uYDaBzcGyc`0LGQxN34Q16F*5y@_s_oE)qxJ6DfbLz^- zBjcALd)fPOj$u&x${d2m*Ns79n_K4Xm7aG!=)!;$)PwJTmU;Nsp_6n;_dPRY5BR?o zjrV`oLmoaSGS9mX_poQ&@P#LQWz^1g#}`+i-=M_HLGubq|IC}tOV5$hpN`+FuYBqZ zsJiO04oa;vK7zQ52b^bHa&Km+lXxkd5|LIfNe zBRBxBN1^_I;)Z*7`ZD!M3BY6CdX*Sq^Db_7?HPUh%H)3c$v1vwfG>RF&-eZ1f2}~D zk%x(k<{Wt-y3%Qe=?Vh%QcuOCj_Ufs>SC|fMB~NuppGp{xGcB9ugO8|L7Ulfq_5U>e7?v-YbAaege6pXGZ$d58uFatH9?4W@l zy09Ohp(#Q@6pVoqZXyEO4m3(o0w}>4UP0Ih;seFc2*AYw_b$xv>;kY&ygCrdz(o{N z0x2ecjs-1)=3?#@vaKoHpxc5l!)$ORtgsQV5C@^{ySUH`!4TEPFwe{o4IN?)ACa?$ zkk2rr43JROHbV&+fB^9Y3j3k;Ue5qmF6&y3Ac{~q!f*vIk?U^a{r(X3@GJ@LtsXvt z0aWlah%W^M?+^73*xrKHm~Yei;T}lBGKg<~%Pb=$c+C(i0}-3g7gcW*Yj6s4aTB$$ z7wh2{Eh88yV;GC^7?Y74DXSE#k?-&^AN7&%6b%&@|EptI(G_12_MFi55|ALHkan7F z7b}DNc#+qzuLuY2$~sX31g{@r1SuNtGrq9*Fhke?uJv*NGg>bT=k6$U?=lS08xQY) z8DG#PU-AR#alaN4?Htk&BhoS^QVe&(A}=E&C14|GGO`3}7|MY0vZ5Z!p#Dm4V*;{c zU{Mwik`@y(oMx{G)3FOA!G3gLDLo+kRT`lGgkj{EBm1bDT5MBlQP85P(W`pfMWu~ zuKC;m#TsqyD1%6h;4R>yFjcc9!7m(nb26{1*-ld>x6&I~vmacO0hTj1b7VGuk&~~A zk~uIFGv#43$43<{BLDkB4pz!bPeTSOsSM3FOI0T=NT;IMB+ zYBUpWG!q(+K!FnqGJ#e4!3b28FToKe3v@z>{|iB}u}KS!MUmp+b^=BzLq=(|Mu#*< zrSz^UG%hjpLN$~_XAEC_??Y8WAUS44XEEw(QTA?eHd8a&gv}T-Ve2S=LpkP=GN=XZ zI72yPvNCqmC2vhazcJ}-jxir1L9=q+=CU}Uv_Hp`^JWiEt<*C1v`^KO1fi_(#4b~@ zvog!TLeCUK(=;vh2|T|c0<_UhmoRDMbSmkzD)VYnQ#3#k6&G{QQ7eKk`$6AQ@-s4$ z_EOX(4OQkkQY>vQIDog*=PScZ9eYGVE!W1h5&q~5FqVrHY1J6bf z{tDs)lMgHL%oL+FBnXaS&Ft2o!1*3l(Fn96W{_Qhi%KmqU|({7U@ya91utQ9PGrZ* zUEeiBtxGPpi~q`yn?AqW(G(+5}-3OhFE{5j2i=FK$5; z<_{)o)4*0G69!QYi4)iuQxb1M5DM(e`XK`uAPByW+_n|jZXyS`U@%=4v`BVR<<>HY z)+6Z-E|PX>3!-U%-!`+x#0}OYW?iFTj*wTc1dxvls8407bQur zxuQ2SrdK7b*M8YbprGO6V&h$7Llhc8Oi4o(ZMFv*;Ubmha?#BpNKXnUXF$>-gGXaJ ztivrv+mN2^^ugD@?{ZOm1(&{Yndiz*_OR3kZ+ljdAXONDv*C!o`xBij|X0YNc8F z(_lKL{|Tb0tD0$=oo70y&B>*$YNvVH;A}dm8~UXCnW%MosF50>e7dTDdZ}aVsiC@> zI;pCy`l_)ytF?Noxw@;p`m4b@ti^h)?Wm&2`mE79t<`$1*}ARW`mNzQuH{;+r@F3x z?fS0qIyuc0oz!5yb6@0-Nyuls(!67`tC49muyuvN~!ZAFNxZ~O{KQc_#Z`R8S-izv{Ka8B#$|lQX}rd5{Kj!S$8~(idA!Gc{KtVj$c22! ziM+^-{K%0!$(4M`nY_uJ|NP0JJj$hf%Bj4{t^CTdJj=Cw%elPEz5L6;Jj}&>%*njW z&HT*KJk8a7&Dp%o-TckrJkI5Re9q~-&h7lp@jTD?vuK|Rz(ebh<4)J^@=Q9adF zebrgL)m{D7VLjGmeb#Bc)@}XPaXr^{eb;%t*M0rhfj!uTeb|Y;*p2;v*pWTim3`Tn zz1f}p*`Yn!rG46|z1pq)+Oa*`wSC*Uz1zM0+rd5D#eLk#z1+?H+|fPV)qUOBz1`jY z-Qhjn<$d1iz25Ep-toOX-}Qap`Mux${oesT;01o*3BKSB{@@Wl;T3-28NT5i{^21$ z;w66KDZb(@{^Bt{<28PN<2kZT{wQKIe6Q=Xt*8eg5ZxKInyh=!w4QjsED7KIxTy>6yOio&M>eKI)}@>Z!i! zt^Vq$$$`z5eULKJ3MQ?8(0D&Hn7sKJC?h?b*KV-Tv(&;Xdx=e(pOyAOHX% z`2+y;0sjCj00000t_8pY00{p8jR==+lL2;ri07q;{a zkRiE_C|Q#>d6PK*lNYz-9N7=OS(ym=mp_Ra##x-&`I=F=naYVV%o&~H|C1NE@(zSr zsDWpvznPF;$ES?{lNY)151E4Su`Jt$FT1iWt8*~_lNY-15X*#-TeDD?y-66mAIrS` z|C1NH?-5Qo!clj}N{GViOUOL;!ZH7o7rY$~{g4D8YXm{t55oYqk=&C)yd?qslW4r1 z5(ZpQ26-Ey26qx;cie5XD+g3aJf(alNY}p0o#){zswb~ zDZt$O-N*l2xx;?D+PmSh8wBeAlNZ2X0o;>#z)%6-lgYq17C*Kq?CQ9Gx{8mISL&OI zYL90-@pk`{7r`)p3lPQ_3?x|4poV(=6!sHk&>=xl3;PLtXwZm6iUeOtlR?_Af}1|1v@mc%#ft|@NSJbQ+J1%~LMiCCW68->^^WFfOhUn7r2E}k z9ON(H$ROL2o*TAw|3Si!2cqn#d$3Zggq7wX7)0|x7{*jf^v#>2Z{f9fz9lGp`E~5s zwQuL%-TQZci5D)UNCCq5s!)!~62(~Es<{fAnVQVL=F0k+&WAZR4HQ1_C=o`gysBARfB%F(& zJfL}J7`}4a=|vcBIp>^VIFss~b3_q^lG6qX!W2<|kg|#aD24%Y7~)K<%Aq8T!7vGf zkSdX&3q-lllo}^{N1=K&C@2Rfp?Q&kMiyF9%dA?=;~5MaDRLk|RujdPyCR@O$ek$6 z>B9^+T&SUfps@cE8WCqg>o4gRI?m4vMKP+T*UTJr$6G)v2u|#TB8UhlziH%B27*|q z%z~1C=!aE;oWstTJzsPKN#*3SaUoF$BJ~|pQC0OUOo^S$EHX762u%#vY?D?~Y{hlU zqUNmgP6qPSuEP&Q98n?Y3VU>`3NLKbpghib=N830qt-|(X>=%rQqLXemW2cbi%<=B zH5M`=G;kx#vN(FCATAz!=M_qfFbE3Od@HJdkqd`8t}$>U3*iw+RvKyd4oq;t z1~1^bqX?&a|4y)}IC=#&g%BMGRC|RB_Z-DsTdu|F)`Synh59W@w1qsac~zMua+`sF zJUR+_*on~P$j7f2dQnA=)*{GeN^s+-SIR8=bTnu#J()Fv-0jEh2;inRYl*fdh{S%v z%$6g{$JG_A=`_pBO_u)hngPxia`^G>G|O8evZaE1Nm1-gVa4sNvRQC{Gb`x!g>nzg zSes^_%<6s;my*|l8lUoTi6Wm~q0$+Du>88$!B%`^af>}2tXviHIHHBLH(%m}lUM9_ zk7)NTc7;422=qy@BAy^^{cnE{TGMV`qnEp_ZXx!vp2Tu+KOx~T4qCdQGotvpC92`SVI z+sL(>j=Tsd5sbj_xaS5ghH+&YWTWQJm=>!bgmmNUhq5HJknRa5Oqf~7-H>vQJh(xT zWk_FEzHuia9IzgOh$EaLwJK$Qsq7q9blkF%@*>0(Pc!V3NXTqMGZ1EscG-bO>pEgD zH{Q)41@n$oKy}Ghnoc6HG?rhS2Rl24PM3vTr7b1$$Wcj>l3WDUaKi|J`Po#(N*u>^Mqo2JTgv?8qiHA}O{svLEh4p+rc?06l`IfP*1_qf!W_&VRA5 zMaTlzod}toyNE2BoGf3efWt?QM9@g0Fvcwc+D_FVQ=C*Q&S)O$Mi*vOqZ`#I3}raO zq1ESzBPD4`O?pz4rc|XX)nO71grKLH=ZP(hqEhbh4x9o{eY9Ir4nz^mk`+g8ae0|w zT$Gb9N~C*8SrZqWRK~i0Nkkqvl}85!gU5nk40m<23N*nMAVCU>eJp>a!?MTc+ z(EDWZ6mrv?zLIt?@}Cwjh0UU*jv(>e<*t15PldQ)mDo`zMil}LIC2qszbuH$YB|{R z*{1)nNkMEJcVd_e01lWP8S6wPVpf8n)tGbSl7(>A)1PXknlCYbU2XVDK5Lq;Icaoj z-^khzxV}TD{jisoY*)|-q_4ACBp_YJq!)G)K`Ffwn_wWOx60L%G|+hGK#<2y!}hYC z!C6W{0?0js*z6g`Aa8lid)P~5mz&;`?mDYWT!oC+yz}K~eeHW+)%Xl(>JeHH=)+O~ z2Ux%ZCUAj~I3Fc{E>S&Da}Z5yYtwHGBNKUm0;-OZtcP5H3zT^6yA1N(h#56)Qeo?8 zhg*;klp(b-(;!FW0gYEQq73iF>TOB;G2yAkDjRM@k9~YMz%EC#IU#}|@Oj2Ra^xNn z|DJG#vxH=}G7nOsj4o|a2iuPHs+p>(4N#%#5VStl#*w{$QBJ&EP=~-EDLbabkA3V) z+j_PiLk3HckBpruLT1AqUM)J!cgY*83`^Z?F`%y}=eIJM!WQ;zPCCpGKGr6w-;swg zh9Qa}^DQgAA|*$r=qfC;)_mBJB^7SWrwCBM)DvCSd;N?!jYRpQgiCzw3wI% zIa#g0gFcC($`QUt#grV4UmgIQQprO2ZZG^>b~JZKOP z1+^lxDQfWy&1x-`+8{Z)ny!t(1!YyF#)yVl_fnY^Q%D9m?PU?ma^i;L3NXq2&^G_K zAA%IY4ur3<{WFzEE%1?8#j)YMuy)RczvXlfc2xy`&uAgO)8(n=I0n(`Lh`L|cX?cu zvH^H^z73g@_gTtNZg&YKGMBW-{M^2QQ^9@G=!f4po?-eP&Yp26a%UySO1Y*>WWL@Y zYvmgTpLnPPeNbJK#++ze@4>CU@>ZXS>J$aZmW6$GwCjx6=;1F7=ksj1$6fAoCt|dn zwqTimT6=}q?jW{I0w-wbt0FL{jKmxgb)}e*%pDx8yi7tenV--k^RBqKKQ=WXgC{}hwg zZeHJ%ZzkyRGHF%{{vm`1`>I>d`>S&V=YP?Ed2M;_Ccl^6&+P5)=U`uQoiP`Ij?Ri< zJoEJDt0of;(E#$l{(ji6TIAyz`Qx50^YtIZ{ats6+0*`i0NjqXGd5&{54v-JjjHGn!O-_w~B}#!z%-90kJQC zyE2-ixMqTehT*({aI8Knpd2WJMtFs9c)s0(t&=N?{Ywh{iGU=40whdAB}76l;5XHf zhg)!i_u8u!F*N1j5)c%@r8=X`Q;3jRJ-rIL#S@{V(6a3ay@gW^b|SHkgRK8cLc`rE zKZ}r?9MGDaIKm~&!zM(xY>UDvRJb^Qyp9r#H-u?E)tfRGjJ}(bt*H7q%R8jV$gB_? z!P`ndibIKui!#-?g%FS_pnAFu^1s$YzirZr0?LDWst2MYBdcIRR+GQqD;9$*h=@={ z-(wvBl0Jt(#asi4P76R_494Xcz_u$uADTd9Oh#o~MrLeAXMDy8tiZaHj|7{4wkFCu zsd2DG)J1{_0d@L8<`{<1yE2K%lh;YQelxcR1ILB<22&U`M%tIKzG6sTtGKCSxSsDpaY`AUwy6H1SZhXB* zJR=53$WYuq#Cs)%D9W8^h&P5%7fU-v|LNJ6h^$vORXTr^*AhgZ-@@au&$GOv%1L`v2uk2b zK`<=S14N!=HLvQ2FZ_!i#K%ODy=-blrMLxC$jzDBO4g;-!F&<=yPK6Lo?WoKiyUffaKbiY0MD$5T6rh}|O6UVSq$ti{noOu`FE(<6OYuL_ z_{ydPP%XL>+v|rQ{Fs|;POVsyGH}Al;s|)`!uU&yhD^>Ak(}#)=unCH(A_bJ4b@8* zebMjW%N%e!z~rIByipv@Q5}WA#5}P0sK%JmK*!uW$XqWq>xgkg3Qv&$_>3N|sirkV zPpRb3a~sWfxD!o-3f);v*fXq#B)%yX4(W7?jyR#z7}K|;IBf|M|Mg_h1&Yh;@h$e5GB#Gd!F)sCpHUu_>=-5^wDJYBngpI~|sebfq2Wq=1_R(Fz9 z7~NKGy$%_rQ3DLj9UWJ4Emvpk(I8sPAWeY{oSF_?2vbU>thAZ1>4-kD$wn;CA!OCi z_-+ zvzt|V5fF-h*Mtf$EzJqX8ivSN)p~SJpRCqY*@<@;M?_l=%}7{59SEY57*x{?Lv2ZI zb;Pd2**7H!qDsGw)hdexo`EGlO;xJS&?!T*ID}G&d-XzE%%1t#&m^lpLbExc6;P}N z3wbr!mQ4zHtlI1t)~P(8@_aOoI7c_ChhHjKu(is6(dZrx_y%a`Tfg<&XP6F}L7s(e zETvOhg%CU#u$tGgp5++a3;PkP?6S58LUElG4oy~(ChzV}O^j+WfWed2~i`or=+r5R{ zg)_-xq+4i|tMcE6bv?UnRahk}`hDHlo!tY~irO8B-E7cV5`^A~j4;y*(P5HTh%vv@ zk_-Jz0bUH}*w`FMBve5>?xIbEDa~5rPSwz2E{3x!ZVLYh)?79YK+ZKraUESbo?{$; zE!`hF-3(;cA?3g#)wrgJ2<`<5o9VM9Nmgp<*MS=?!>b7VF}Sd(Kd!h3>)oCrUcLjW zWQRyo^DW@ZS&R$v-vo+W@WyU(-JB(9Vaj4o2lC+~Z`WXvlFfG*@;A zlCeaI;Tud233{~UVo|q;Xie1!XJLEezYJF! zrDK2&XlS%!AHw5nOkHjCSBkXejd%xd&RQ;wPqx@s|2zd=g?89}cm{uy3gjz)<*>NI zzL4bE>oZZV9B?k_n$Vm#QH^=7+B_?vckZ@@cm*2FQ^0X%KQ**^ptGqo!|s?T%o#9!(^ThvL3-1L{~|H~_{ytwQO;2&SZVmzK`Dh}Vut9x~|0RNd#%3R8%U?7fFsUWiN}M6f6Wt9b`y@hbRfL3j!{ zw^5*fJzseUVP+PDdJ=_r^Wdbm*3=N4u9)Pz#mxWP4!2sCynz7j4LV(7?bCVpxA*&+rzvPo3Y_eg6bXAS*br50HJ`oR#pwW z>ktG~@eJ+`CSySFdjlNyPSJwkt8)}>CLLaQ*(0|#yI;cLB^BwS4b_q$Nv#Uy?6Dk5=+AfV`{ z&CvSmlJ9tNct*aLUI}APY#-;S#jepEf@~r$@&%Oa$)-kWTi13yJ7^dZN_Yi+dh$PU zzi+sOOrV5Jc!g)sRTqT@Q=kMbzlEco?j6ys|L1t!Kd};}((++{fNSiiv@CA~5hRUz zpcOGUg%$RW8Od@=SWj>+r*BY~j-c`=pYkCQ1sN};Thu6*#Ex|l^F|PbJ=Z8;A@r9~ z^GUCC?znR(*Ygk>^g!?PJZE#9x$-RE@-Ej4V`#G|4|L5nbSclCL|^ozc%3YL5ZV>< zOd#{`0P{=;_NRh>(N@p%GUsy%%X3a|NiqNPWB+kz*UKRH=OH(8YkzDc-yzDz#%S+$ zZ~u1GZsNm2TzPU@j_8wR2X}Q}cW(zLc7Jzxk9T>Wb|U!PYrl6o&UR?~(IrP+J(hQX z4|u)Q%v6#G!x;zw|BebBqquFIwhjS^k9hxypZE~afF93(nLM}vir@H%k0gO#j)&*? zjkoxZzb`Cu+y&Bjk*9c%FL{@Ld6>r;dbf9culdl`_Z-@GgpPTh?|Gh~**qAAXLtrX z*>5lJj&q1_hJuD$;NHe~2XO|8HP#t>28p`x`R9oGikNz=_l~J6gCT3ps;7&4s#JQn?b+7EtqPv8urgx2p4Xea|3Bo*Or zewg3b3rKzEpML5G#@p8+-Ov8QJR^Je}`un2Hf=Z`uc{3x`knYhNBmM z_)qLRH~(8m_xQhm{GXcZ&mrvxh$I3B5-e!&Ai{(S7cx|6>GCDam@;S5td~C%1tx#$&XXxf!VsC`W5 z_AJ`8YS*$&g7GcfxE93{>G-jSSFR>eV(shqFW|s}2NN!A_%PzciWmPgZtVCmP_10O zf(;^;Y}?G5H*@Z6P%h}u80p#tDN^rA$f{SfZteOt?AWqr)2?m%_F%~*DfbAw5wqvu z!iN(VDEfam@`t27nhuGYFYf5lr&F(P{W|vS+P8E6sk^Jl%9ke=FK_<*+2ra$FTb0& zWa{qn=hLrm|33cw`uDf~&fB*}z@axFf#|Jw;9KoAHxhi%@%JEv5lT2Ag%w(O;bZ=V z*PCU(kyjvyAqs~ei4`e`*J%x6xFU-!y7(fDG0J}^pN1ic=b=O*>bPTDCHk0=Sf!!p zoQg9Vc_flaD!C++P2N=FR~(L)Bac;D$lfuDR;E>rARn zDd?-go);^Evp&ZwvdJpDEVE_$8Yo1@C9$JN&Rx z;9}Y;xfSiqtt}Fz_+RTo^3VI~#Fv4@>Vz05Z!hyZryLC@j5_Ldzk<9MjA- z->g&4EZZzo&mjN2Q_vp|4O7t!91StkO*{RxOA-rcG1X{coH3&A&AG8rx4aDU&M<$8 z0&)r{uk_9ZU#kuBD=QhHHVpE-^!C?r=S26*cDsbP%X;6$cgudW1USorLwz{ni8Cv8 zrd2!M(ACIkTxU_nto`vVF&Ps9*=DPR_T_4~oe~t6=is?YrLTSZO{%NSI!ms94SP+p zzf61TrlYPn@4fpzD&rJCK72xvA2)w_qeF$zau9$^aBoZJMZ{sm}oXAfvC+{Pll)|Nrm(`~Ux)f+sHG5%4p{ z3k~v)LJ5{>02(Rz2F*a02bDm>4P1yI^>)_}d3>XT9y~`5Ko$XGoWp}2d>4P{Kqd&4 zG=gOkILGI%_pueGq=g`RAx%;^Ll(LahAGh@18CR}8-5IjGXbIyiO2vW_5+DLyrB;P zD8(sK5lI4!Rspj}kOLwxYYRG-Q7{;RXRPEKD=Xr=`Z0{PRgWe?5aGyLVhWZ)fhBHW z86{LQN08NVC3gf_9%r)0kM)01C4c-_Ap0>$1{5+Shh#t^W3tFb_T!O>oTMG`SjA0p zvXd6N;zP73O1E%PE(t6OG&uJ$F0^ENAR9(XET^_M&gB1%AB#X7S;7SoI&dZDI3ojy zppsqQvX?6PCCG$16JjzT0>(@UGC!uwelXLSEQw|?X;RH+_5+*K+~$8ZfjLc2lCzxV z><%c6FiI_!(v)6Q6c~x3zRa;imitS|8fU4&m)NqIxm*bbVW7bz+z@&J9cV#Q0*zJW z0G25^XiErMM1z7%pb<@IKNh-?hN=XjERpCEDf%&seiWnUJSj?3TAOsPGmGtv%PEO+ zNL;o=hkcX>G)!m=8Pb1JCjF$TOUwu|H@HL)A|#_wKStD+81ffHqg-r3hC4%6bf31}r%Q6{lE>1M zCC~t7=Gg0!_kw?no{43ya#_~imCzS+`K|AK;fq=P%C{x`Wk7uO8(;umx4{m6a89zT zU8-^SB6StYU8(w#t1|IyJVkFy)Z3C*+)%JOTp|C7i)dmQ(olMCcySqwk&+ZIam6a^ z2aP3Q;~jQ04=;8@jAvZp+OinOL#}C&A*|#jGdU#^o^XG~Dm)kOf(o`ksmJn^y5S>R z@2_rz<4eZZVZUlw%u)34P-S9fBBy!HNKSK1+}!0dLy^r|ZZe+pOx-6#d0bL95tV5L z-k|g|2rqFZ14ue&2HcjPU|uiF9C}R|w(YT!_9dldVrhJ3dehO%wB{nsY2%9e)6Ug1 zs#Bd(KKFlFT7RaSjF7TxAnH*%rP zYv#IJCB&XKzVpqyYFpb?*p`U4f5OZ$_oE&%G_-%68I5aikK4VfWWWSlvcOkn^x+X_ z$;3Z)>G@jRB^hS`EK?~#kCz%IBLCRMSMu?YcU>e94&LcN}J zmRx@wdNMyd+8IXknh)yc5DBMT@lY*a2e)+CVUsX z-#=D(($(4Mgg1QSRg!oHEWRIyXUXI%pZ5RFbADNCXM3C6uAD%}lPE~<@Z_<6_l2uO z#4aDX%2_7FkJJ6*LOh2kOriU7(;nlpS4n^GV}E04=6?6WN4f8Z-;dw}|M#7LKJ;_- zdC(V1^l*s$lMV|l;AgN3r^;L=J8Nv1S1Q1<79^^rm1t9o+nE3fqOk5Qq0ygeoM3;;oD~w{U9MSROq_!ZeIYkN(;c>q1aQ-GQK22G zO&^Y$6Z&BZ>Y*U!VK?oe3n^k5J|ZMq(FJV9ZfwY)}uOrYW{!r~vcge|%ZEp}Ziav&^5qA-6B8HvP(miC)nCZNEkq+Fyh1Rxgf(KLHu^+2M&mcmL^x*SHrhlv zj-yPZ<2e$eJHF$D7~?SxNHTK7GJgM8Gs2@j?jt`+g*?vVuxz3-YE6~#Tt6NpLMEh8 z{3Ae?M?KcXJr-m>DkMc#q(y(m1Vc6?aXh5X;p0VqBuIv&lTG4eSmLTo%m8Vmm2hN4 z3fe@9BuvKSFlMAm1_w(Lq)W=APVVGPl4K>?2%v-u7S-fR3Zy;aWa;swQZA(c(j-xG z#!Y?^8!{zTR^{0_6JPUM6NLPGm}^Lr`W*CS_)Z|E6fFolk-XP+nr=aHe@IW=@W#YrdwoY^G`+1Z#pO zY~Chr+KOz>W?Usxgl~4Da3-g6_6czoCn=RBCtBHZPA7GWNplwG zaY|=(ZYOt2A|;w;l%S^YIOm9XreGpxcc!O$!pL>IHhOWkf zCMJe%D2RqAVsxlqeyE6^D2k$%gqq~945fvZCv*;^RHA2!)~Jm}1&P+BiQXuW_UKXM zC|mBRj}9r3_5_fQrH~e>k}m&gudL{4QfO+vC{&u~jA|#7UMZHU#F1uYl4hxwD(RL! zWtV;_nf7Ry66GhDmZ_Si=$Y1Jnyx9Ff@qtnC z<2?&HYMwUao~|ma#;TV+BnKIPPHL+HWS!nAvI=Oi&f~ErtFz|kvL0izJ}b4VXS8PG zv{tLOR_C>%VYY57xF%<}0^zrYtGVLlxE^4+o-4bKX1Y?}y0)vlUgo=k|6aV#E4~(H zy|UfC=BvNT<-XQjzy2%1o@K!H+rSpA!dB(MT3f;{EX49;!?xPPMy$nu!sNszL<4Lr z$9Am8ek{m_tjLZm$$o4K^nh_vDOb|y#kOo(W-LOKtjx|V&DLzkrofxT=uuWFe7da9 zGOUw^CzOuo@Yt-;9xc*t>~r3nS^lik$|TGlMAA+z)sAcmRP57cEj&i8K~$~Re(lEg zfXcq5)|RbOa&19?t=dk1Z3v7lUYaf3#w6Mj#M;*F(Ymc=#;x9hQe6N_N?pHZuG%!>?-c;;-~HQ?&jftZteo^?)onA4j}Lb zuiXxB@h)%Y9WU~(t@0iPHef*wV1V^rFZO1y_HHltcCYt-FZhP9_>M36maqAqZw+7p zHr&MYR`2<~FZ{-@{LU}^W^ekguT3Ze_0}){_OJi`F8~Ly{N6A7?ymqJFajs=_!4k@ zI`8v>ZS)pJ9Vjq=1y`^IUoZyLK}}3B26wOrC$I+71QvWS374=5lW+*n1P7lm3%4-$ zrf_;X@B@2o1P_J#jzA3@fe!C55BIPS|1b~-u@Db25f`x$A2AXq|FIG;aSe=s_13^l z+;9#rF%?&_6<;wHXK@cV@e|L46Jr1lYq1!QF&US!8K3cg6>~8LK(S0nF&e)y9LF&d zt1%nLaL}d+O1dZk&2aMSY+&*%@9M-BJh2=XvLPSxA-nGu#Ka)4F(N-QBp0$GGqUwY zvL#yOfpO~@)T<_D2K8Sb8>geFdt8C4Fg3d5V8>9fGfW;EXT4e&oV97vMt{- zF6Xi??=mlc_p&elvMUS0CR=YOyu>Oyaw`L~GA}bTH?uQ8voO1FF}uVu=WsGVGd5?l zHg7XGcQZ3bGcg-8D|<6Jm$NzhGB{(vG;^mZpEA;>@=qKA7$5(^IoGp2-!nerz!5m{ z5xj&vj{rUAvp^5DIqx$D^s`I+b3hX`LpQWT541slBXmnB^g~y)McZ;j+wtbID71KP z7P+(9!gEjDgA>=lMV~ZE+jI4ffCiED8mBZ&$Mj0KbVuQf2Abtw!*XA9V_q*dWWRD_3-v}nY3Hib=hAi6+VxH#|8``5wmBEJV+S^9m$oa5 z_Fq4?X|HxvqxNB|Hf!&-YjZ1H53O7mt!7hy?Mvl=25c-06m`?q#618u4d6g=AGdKQ zH*znxayK_~Keuy7H*`<8bXPZZU$=E-BX?n=&Tw1rcYC*amp6K!w|b{H zd#`tMhxba1HwLsfeY>w}AKefEPG{NB4izc4Z6gWs6dO zZtHd_!NA96TXfPcP6Rj&WVnWJIEQz*hkrPThq#E3IEk0IiJv%%r?`rPc!Ku>hOan` z$GD8oIE~l1hqw5P+qjPJIFI+ZkN>!g<2Z2xIguB+k;DIZi%(}eKX}Qu6Ua6X&h6~V zvTjb~_>o^ZmS_2q3psgfxtD)AmUH=ke1kcem${jrxtO21ny-0@C;4*1Zj>~@$@V}B zF!|z^63B{el>aeaXuwJQ`SlEim9sga7kZJWxuGYzqSFAPFS?^Yx}rBaq)+;mV}Lxc z9=g_cgMTj03Is_AL=f2d$U6DQ>iOHb7RVZa0SNk#Q@X3adWTCotk3$7%Q~%p-#V_( zxUJ_puba4=D<|safN?Yc35PB zm!yEmhrk7>I;)d+!>7E%t31oE{KBVs%g4OS%RJ3@JIv2K&ey!o=RD8T{LSw?(D%I1 zyF0`SXXJKz64d#|-#HiMdB%sp#)d!-NI92ifXPccV8?*gZ#~y{z1M$#J=ll6*pEHg zm%Z7aJ=&+e0Bk^$zuK?8*uy*9zdhW?z1+_|-GjZ`*FE0nz25IV-}k-T-@V@lzTgl3 z+82FrMy{|Y0TIaTcQU!McdXbh{<8CV)n9#3OgrIczUFWL{^kQd=YKxv55DJzKIxaf z>4*O4pT6p^zTI0orr&1dT6_}Ze}Hdae8=Vh#oIoy3xwoztn7z+$I8BhLvP0~yAde; z%~JlHSbel#e%7--^hdwur@r)8KkHM!^=H5KM?dy&zxVsS;R9#n8i&#sdlG2;#>&1y zNPedqL2rxhuyd>isFM-&anpM|@^fqmC_ktGga!i!`V};I5Me@v_vB%~e-Hy=M2QnA zRDIM-7jIs@dy^1pfAAMZ+#Y`$^pM*lY!JCY{t}4`gK!GDMp6(CoP;r9&6_!Q z-uxDHXwjodhwj{iGUpIP(3Xj0-Me|`#<@FWZ{9{P0*9!ZIMRZ* zT?+3_+#+)58l|&T-jDf8=i75u_h@{2c<~vpXBRoVxbX22y{`{>f1PW5`Sa=5w|^gh ze*9j+iY03nt+m(+jjsX?JP^UTAOlRWza)E1v&&M;p~1f>lxx8T6iXsQ&piCFfzm`A zam3S7+ibPg%z{le+Tt@WxDGIok;WNqtZ{|kW*i~^#v6U?V1yjcYmcK9WaKdhBWFww zgOY$s$)yo~{4vP&f28a{NcSYVGE0@p3ktU>YaBDD5jwyUBqh<@2|O*`jOa@!RZ_Fd ziRfHmM=j07pw2fpf=|Cd1s#;oLJd7sEB<7I4ZyV45-q_myef?FLKya#q*G@v} z1lV3X!NQ?Fi*yLdj{@OUScrU4#up}?q$$^yu1%I#Ujk`%ylZoDc0F^^^(aoER6-V9 zV0}3PCuHXZXlivI2pg`&P7)iYycw6rvAM3J zlTHFryXzIc?gyJBt?tL3BjDaU?yTG88*Y1=vL}!#e*y3N@Eir-r;fxCf1K~d8;AVy zzbPME8L0AheC!w`zg+6M8Rz`+wwDE+^wd#b9d650zq=n_#16!uw!u~%cZdY<2kYE@ zUmehlg&&^y;*GazV;C4c%TcvJ9vS-RIgGIX!cp@&wbM@@qi{k9bN-%Zo;ib|tyfi( zl~$!~e`OlilvIb^sZM@>dhWU%&%7yL`c1rk*1f(x23rtZ-u%ZofaNX*>UhfA<_Ee2 z_JeT+lwju+7(v_}aDw=AialyDL9NxGf|EO(=p5F-r`14mCiI{R4H&T}Jp^}U!&?ou zwgO)iByas=;pISBkaMjNh=~i{@rYPNBOZ}rf5}^3fF4C4={=EBo4L%U;G&mPJ;*QI z3&DdRYapX-R0zt<_)Io;!NEjUh!9Y3w zkvH`NWN+|?jfB-P5V?>c*n%`j-3Zc=y8ofwK9rFg0pd+>k<{dF{|-E0 zf3e8Tc@mYm>EY;7*-GK8vT?5LWGnZG$WH7o7l{z%7F_w95N?u$y>!k4St(3o9y67* zOr_}jxEmDSZjzNmrY0ZAyJ=q2IYUI^Ho4hNZzAt2i~0}qKITLyp7TItoFdAM5F6RN zGc2W|nPN@_7skNFWnjda7+JNc&zMm&e?&W*|6`8PMc;S z4*rUnn~XwqH#AbxLs?qViFN>%yqT&0LuzW&nmTkhOHFBSq>9vE%F>10n(8qxf7w%| zj?|?_J*qRWsvCzkwKu_RX-gj}oSy;}PZ+2mv(PHltea4j#pI@~pxpT&-0TmN}k;oPm5&^29sSD*t2r?g0XDnO@-N274ya+6qfFYql$qLcjCV;O{!TPtOH&; zN3At@u#MGAW3gUW#3Xib1q0z=vvya#O%7|1AzWd>#+bf$Rj`$V3grj`7?mvcqzuj5 zn;T-%zF1)Cme=NMRyAkK1FTZ#qK<+!_Z7N#~yTWCWcI*H-r zSXs!~M9{Dn(s>ylrffFp+rBm~Y?)pIxaY+X@OGcR9T9NLXuenxv{naIZdRC!%<;nb z%m`5j7BCu!j~c`qYSu2@FscLp1r|hp;@z&93&Kh%`C=ak!G;ckf3(PhC>8^;KFF*e zSrBRFSEL#=h&}W-XaTpnz`xdMZxEYev))qHb*O2%33uy$P}{75`1Q5};SF<(RNe?7 zoJM)ehepDJvmw~+c+VS$^}hFtzfE9cy4bB)KzOlz(Nqj*8;1tpF`ni&Lm(GB zmiX(0v?17uibmVrfBjI#oEeJ{UdDOi1z-07Z7pkBV^rb^$#qkYo$P}wTfSJZx4r`* z*f2L7%)l&mNS_(IT}ulv$O(4x4V89_&ww`;T!Z_`Clb)O=67=3DP2 zp%LA7x4(TprtMgxQ~T&h-~XL1mfqBZHZ9M-hp381__X%qf09p7gGkh)?u>BHINQG% zca82;b<>jG%2$_k?AyM1#}nf3U2i?kA078jhs~Qh_?&^hPKv*qu56iR9*3KEF1A6b z5XBBwVgs&xB=KxiEP&jnBsT~vFR2u)kD%oWF*fHB;)@{lP#x0F#fDQ3wwfZJDPHxY zm|Y2RaxX=(f3;6Y_j<9r(&rqr9ZZ9`ZiU2=PH&jm{JkJ69kN!AFH|Vebc%%28tLBuiaP}$? zd@d=}4)@>=1yfK3vqJ9w7!9=s#P0A;2KA0GHl#1aed+G8T@ceS26LMkR4X1vs6TnIWPbt>s z5Lvcuf6nfYZrWzv7)Suou#^shl}-F%s?TqyH#zk;-fY)DRUJX$82ke-f|myaMb11u>P#Y#h%F1LelKw($(R zk#0ty>m+IT{OF>-F5u!w2d)tw_u=LQP#GZc=RE5GSI{625+M(w1wF>m-a-Z+k_JWP zE{F@$DgC!CkZe!fg~*daQl|hm>du@6RXUL5A?XeV9r7XZjx$m~W;P>e=pqO#g)^$CLoUSdG*VSKe{wWD zlJNqHA{zkmmQeCYa=9E%m{3yW&cvb4>|E$ie;jJy=J4qRP!+|{1aULqWJnoCN`@5c z9z||X90ES?uqXlWtb9Qo3`r@=Q9V7$6O+R0cv2lk?4gXZ28wU=7Bg(*li18{FZpkx z*sU3Rkt%_0qW|`>Af}5odE+bdf0HgzbDHkbk_M;-GKxhvhbR4QK^wGjRtXDY%__HJ z{bJ9VQiz!zO5Q+`D~k!Ha@3)AR2s{WGjFHqigX_R6F|EXl;G>Hu%SVBaTT?SDVy>| zuQJ(+36i*Uq5{-2@n+7%Z|j=QG`TJ}$?zuCu@0K9Lz`|t$&omZlTZsaf3%YG1)rxm zAq|YSC zN=8ZVC-ISYKxtVYR$Ua>zH~RNm2jN3kA%+JtX2E6wOCoPMpLLah78Mo zjYofOSZfND=FJ<7t}t2Y@{(#V3j!K^fmxUePp=hTp-f)&W?EITT2V9u>mW!8kz8+8 z98F1FFqFhj&`ggBCd2E@5EDx!)>em99GmWK%1un=G)~najyjMxf2OqnThdzlAwNZR zMhntVb5>`K2vM77Io)DW?@m&OHc}~7G%OV)i%X5nQ#LyF0zUPcP*O&O5gSr&C>6pR ze8I@%FLw0RCS#QYajEr`Vit#OYJH(<6XI&eFiIU$--0D$1(jW0=}U9LZtoTwNRWYG zFBM_x5B~*eM9DNoe@7=+mm(KvbvNKgDX<}MKgnGg4nj@RlNMGuYz$e{fdp|#Zzj|p z;P(0E&FA>9uL7wT@rMB&lV3Xz4T*K*@I(hLH*-DCZ%LQeJPOS$tNQfq2y9o@1U7EV zw6G-Ck^VJEtrP;scJ=hebmKG^&g^Xt)UxvQz;K8#-8L8Qe`X!86l~>oXS>&Xi>GJH zDF%afwTRYy56@_gRzQ}PQ$5uQqxL2nZ0M3=+|Eu}gCsE#Vp-$JW?@zU$yN`CO$X*z zA=Y;5+V&Pr6(PJq&>+`Xh4+sTcd-mYc92*3>HvXTGi(yGdXtK858`kyjB67D4XGC& z=TejZ>N1sVe^jQ>ZS}(IV5*OS-FAUNb=OkYnqHAjt4}deSH(uag(r}PG8h6AQ*bBi zf0K6w1}=8H(rY70g*9mb|Mcdh)L{f}h;dHJ7eMJ%xiW{Hqvo==V0njw zCwO~-_j}(Mju9n%8xm;Am$l9pkHIK?DRq5Ck_nv_e`=d+duw2RwJ;D4CgAub&Mp@r zwDk-xurW6(evKqX7qEGK0h4{DB`EW*^vw2gzk~ zLx#34mM=MzYZ(iOI9IW^fw@jeua1^6nU)Xv!fcn@ez}%$3;-SXTHV+jdx>&+gR!2G z3jd|me}pS5k<#`I-If4@wc_OHNXz(18)vfW0F|eWhBhvm`H-4rx0JOvY(}(tB}yI> z%a}L02AJvygb{NO;x$|KIXXYnA48YDUVA!Rrc6BfBSe-4GNIoSC9jBW*KmxPZbN8rJ}Gv zgU)4JX{h52PLj7*O6ZSeL1}+|s-_$1qGCF#px0MRxu&rItv)cOnE6Ujok?Iid%He=+BY+0UG#h1ioudf9=KCEt1jes9SQZcV~0Ijt)w>59zyZg zidx{jn!~s*7kqPFH};qiEa&>+HVgW*DeI_p!!`vcZ2E0(I#gD1J6!*A%~+HG<(iX* zE~8i3ql??P<4)0(=d^rQQAhfTOd7g9e*~pDQl*be@>;qi0a}xy@Qj&l1o&m>Os;-* ztqs0y8FAq8`$vIxxU&?DeAjB;H1!4 zqvtl<5}S50Ffm7LzcHY|YrwMIIYm{Dyk&$fmw0#RYQTNFPuV(2yYj*@02;ove=7e+ zJi|3Mc(pr{$eKS{8%Xvf!UKH237mSxxB-368-7=NX%;9I+-(~tKn2`PJlt)e+Nyvp z#;2Bgsn6pmTs|+HMddq|>sv7CMqG6u#?9noPkY6Y=}QObZ=I{kxBSZ0Zd>!wD3@7Q zE6%;&{7e89#iKP#cZA6&nYfQzf6w>4DnR;Y+rqg?I=TlvxTZVtsyp(U)_t=Zez&*4 z2i8Dq2f__PRUN7flRB{9n!V|pRHY5;7;8RbtXDbR#_8dW+e;sPr|GgXm zugS%o$xTHahbYR`C+gUfomrjONTW43pjWZK-A5g&=0=C89l|Le>H;as*Y8k+Djn5n z4bC<=xkd<1FEbojE!3wIlnc1By)V+)Xi2J{!fwR13S~)FEqSz7G$==Hu^noScJ3AXv4~7kE7$ zT^{JY-sRin>&G7Ky=*Tt-c0WUXYx;AhI?bqcuoce=b-Ul!E_*+~r#@ zVCz0?Q_GQCy|u65b4j4?PG5x2Y=+|4fGM8>=obl8Gq&f zHw06j=gsiyZr{{Xf8LbdUc)IE@oSIhr=RF+098Sm>uEMf1TgyY_UD2h@!9U;$N#u9 zUbQx!<2`=RL0;5EzI~t83HMw6=b!%TfBs>f{y&&{_3zmLV#J6}{rX`Oh!9)Bet{4= z6sT;W#EIByJglK^;=+9m1!A!$@t{JD6YbR?XwV}_lOt-3eJ%YV&HIuHe?};jBuA#62gj|sR_8*VK2O8dTQIZcf>FCB9sP7+?a;V^hQ7C~%Rzm7e-_U+ueFT8Le1&r}I%IE0OBS;V;?1>n858p_V z`Sk1CzmGq^{{8&iiU$M)s)V%VH(wsGE-hbu!#R zCdL+6z7ea4NROOmA!g=PLUOL3wYHkhs(~3fpMOA10s1rzzk40%_l~*eE zP>vnVR0oxdX=T|(FJ3v2nFeh-f)3a;id9*TLJDd>oz$S_LntkUP#u8oi-3KGY;YdUxgGD0tptlVTKbBnB&ZMBv% zf1)e5yz$2@TRW4`EMv*!&4~swbkBt9Y$(r8$2Hkxq%s}Inl^)s^VG_HY_blnRSooC zEO(6-*k=RvGs>WC?KYz9cIh>swys{xZmU z_~QE)dimxfSbF;E8*zFN7FImo#ToaGeEs&{KLYQ@Wt}xDJ6dtYHBW*N++6aEG}wAP;{C#M${Ui2b8XQ?T)(qx6h|8f>Byp9n?9Z4U;N z`{4H$M8YhN4~1Mb-wMANp%zl?e}x;0p>}jg#54+Uh-z%3|MYN(MmNfFj;Mj7?dT}H zvS`F`ZzNt6{|LxH3i3KrtYW_|2R&$d##(nrJZe&RY}}+L zv*gDdX7ZDy1Y##kiMw1N(H^W+k=K|w$XLp9mVYdy2M-y-L@sid0du4;e-ReRV8UmT z@GIjbWjIPyDszH%q~9_%dCF%>(|^&NrYUhq%WP_Mo7l5u_pFFWEb?-kA+NumEkQ168^J_>DVeHseW6h0J^ z5zWv<=eN(0-V>xcrKUc63e;ErG^hj|=}C`@)T5@06_-s#=u;CKRhUi{j5W0+P6>r7f3Z+XvrPRfqMEaohd8HFjB z&zaMgW{PS@e;ztOF`ggYft%<+9vWe^qah7t3{3yJ(vAKtra#SBNmF_vmi9EL<*eyZ z_j%QYy$XPp%Q3gdF2753R{Rj_{msS>ZiyxX*q3 za27_Kf07eNvUbvajykM-=}d3B)1MCYs7rn7|5UHK)vu0q4AkKq2FG{OvkvyKi+${5 zAN$t34)AxJ0}EwuyW8Im_qfX)>}UVC*Ofl^yz71MRj<3w|ITNgzvt(K-*3;^& zWct<1e)bq|{pbOY`z#AyK85eSOAp_$#4G+CYCnhZkB|K1D}VXSZ@%-N5B=y%fBMv~ zzV)wveBxjJF84@x@v#s7@QZ)^HB5C8aMU;D2w{rcsP|NQHJ zfBN*#z5j2#d%h=t2?cx%BYcV{55%W{3&{U~4d{Rm2!RnOffGo96=;DMh=CcXfea`Q z+gBOgmw+26f+I+RC1`>tNPHbwf0-A8DCmMO2!k;wgEN?cD%cz#NP{`3gFASEHu!%( zXkP(1fJ3-Q1sFy{cU5ZU9Uv$VP3VMAe+Y$9D1}o4zLhHdDEZwQBRD2Hjdf+Qh^bBKp|sE1mphCt|t(hroj|hp8D2bCuiIr%Hmxzg(sEM1XiAlzZp9qSfD2k&qw9FXpi@Ze~5UCJOyk&F73q;536dcxk_p#kRCG(T1!x%gjvDzM4>^)A z36n7?lQXGJ7paoF$dc^CkrYRhJ?WD_36w$EI5v5cgP4;pDU?a6luOB!e@*FaC22?~ z32Q}}kt~UnPKlLSsg+xakVHw9LwNs`J85)WNtR`4mS?$*Uip;*7?$l}Rfwo)XepO- zNtbn*duq9sz1Nl$GGIhUj&=!{fhm}S34l;}lGoL9dC81>DKUMSgmFojktvyzNtrci znBN3uQ>mDO$e3>ljF4HGf1xRwqe+_8gO{7>bDim!a9NtI>6)(zo9=R&sL61uX=Ks} znz5;yyUCloIh(X8Zni0Lf7zSGX`IIyn!gF0u_l~;*^$TzozW?sXsMjbDQ3<2nA549 z+sU0vS)JDDVc5x<-ASJ1X`UhZo#2^W;`y29>7MTip97hm>iJmge~Fv%iJ$qYpJ{oS zUnZ5{MxTUdpTxPJ1!|xN3WW0+|DY{3pwEe*4+^0X%4!O_paf`@VmXZxs-YXop|3@u z6*^80%99;Rq9tmgB=w;nx=SKDmM7|>FAAejl%guCNGzJ6F^Z!(s-r{MpDM*+iAkg5 zW}~^eqeW_@N9r9je;TADG^9xCq)!T^gw&%_ zriY5CI~u2Isv#y>MRp3PkqW3P*`$eTsh66gO1h-uQ>mEhIH{l7p_;m>;nS(y@eQYn zs;R1~tIDda>Z-2_tFbDpvr7M~wQ8%kimSP*tGmjpz3QvK3ar5@td~Cy0UHpl+Ag9x zs=RoiZn>K6=^fRV*?0pRe@d>`I;EKjpxGLp+sc=>NuZ$Wum8HA*ovp-Kum^jd z0z0tIsIQI5uMUc^5i7CLsjv$RFa&wGTU@Tx+&xE09#nv_^}zZR@t> zsJ2AQwr@+fb&HH~tFv=!w|mRCeu%d+tG9hCxPzN>e~Yq#OSp-vxPj-jKMJ;CYoShy zp+vg4m20^$#&#x+vPXJodPd8=_-7oDf^OuM4{+1-c6> zy0L4!w=1KkTcxU7qpa(kWsAGPE4;O&yPC={C8W{u~gdr1`rfW*MPVB`W ztiC|%juAt~WlRJ;Fdjo(n+OEXt!y z%B5_|r;N&}tjep*%B}3muMEqvEdR^1Ov|-w%b)xT#&8aG>}Pizz5bYMM35nB9Gw4{ zAvvI>WDC68fp?AU%v$#x&v47rOwHA7&DV_0*{sdm%+1~G&EGuB&%nzX0mbArOlX_~ z4fl_Ne+&bRwX*%X9h}z^M?lZ@Y|r$`(Ge}t6HU<-ZP6Ev(HX7L8_m%j?a?0%(i`0fmV682ED_~wq3o>A@p#3^ z{It}u1p^JxGfmSqZPPcs&z!dfcVP?-?bAOEf7C%O)IJ^3MQzkajnql4)Jy%*l#mI= zVA4Am<#_#O9*imgYa0Eht17&U2XN}frt=4PJ)@|+9Zw=RR zjRQh(1e}Kw8gULot=D_a*M05Rs?5}ZE!cxi*oBSMmb}X*jnV|D(t!Mo<1t(N+Pcep ze;r5g7Dpi0nXTEI&Doue)<=*(N01af{nw>!+NX`$KW*5n&DyQ)+DkpsP_5X$45I9l z)sEeZ<59+5P1)6P5BXCAo-N$NP29rWcq8z7m7Lnk&D_oH+_n7L(JkH6P2CWk39k^< zi*3%0Eu`_N+g**d@O&Nha1&%*+~U>VNg zKMv$^y%r!&;ze%cN6yd>?cz($PrD zF+R@P9imwtZvV*TlZ#U{!0Dat>7Nejp)Ts9PU@v@>ZgwCqh9BE&g!l1>Z`o}-;OTp zvrg-^ZtJ&>>yNJAf*|RCEZfJFe-zmFmZi?8&a|%dYCJ_3F<)i7?7JP_U+vpp&g|)~?(44Zt1j*D4)3f^?b=T7^=|L? zZtMBo?R5O@MHueaIqnrc+3`CaAHwbjkMIdE>h2!z4e#(Hj^Fq$@e@z+6;BEJKIJlw z#a3R`Sbp2xz2(IIz5{>Z&ha5JKm#bR@+r^qEAR3x5A!cC^D$5JGjD(MHIMT*uk$R= z@DK0vKi}LDZ}CG<^hICm_`UC6-0wB|??4yuU2gJUzVlO`^HopvS8w%MKmYYT5AA}`s$OYqWh@-cw-d9U|-&-Z=r_kR!g zfiL)jPxyY%^<$6tiLZaqVc+M9kJ^E5_Ef&{K8m_lebr5$0|Y<-24MNC`%rxx1f`Sd zBTx4we}isO_^Ge@tIzta|M!RQ_^~hh*Ub1%J^OzR=#j7SUPk%5yPRu}ADK@84AA-0 zJ5wQ``7j{*a__LMdjS%_09_m%cklYuZ~fO_{jh)g+t2;4eBWn*-u--?`}dvnUcB_k zi}{+L0L8ze$DjPmPwo$!0GiLlS%m@EkN^3v|Az1E0Fg=HK!ODg9z>Y`P~k#`4IMs& z7*XOxiWMzh#F$azMvem;HKG(!WJi)EO`b$KP^8F@0{gv$8B>>`)dLoPjUGjsbmxT& zDPTO!5j97T9zlW-!Aiu)R<1^pd<7d;>{zm8&7MV@R_$0LM4p<>L4ZI4uWRj^?V4Bb zUcOvI$c53j@6)V>ty1kFHOEs1aw8tyF(%8qH#6mI+O=(e-^QI=_wI)uSBB&rT=;Nsj#vFr8GRj(G&9e?dqj16wIyj9p3nApN!wz4IO(x=hMkKLB6Hi1jINyFt zu|*dvG7h=wW~8x38*juhM;&>(?kW7VtLmz(yc@4bBadXuEg0aEi>|bCXiP3C6UgDK z|1MxF0l9MUOG+sj;F2r!G|&qMz=$xwODO|0B254XgTcT9gFs9{7@~X(smLbt5VaL1 z3`Dm4YR?Z`8sTcCqC1bbdx@)h${uWnDaxo3nZ zxsw4mPP#pJ;VViSxQK}FU1SYzzMAO^YJa}C-(&=N8oQn01GXuyFzh@6}#m#;f9NKXzj zO$If8xIqrCpmQ+6MjcGE1!e5xCHn|OXH3_RiFMG0FN9$XWw?UZ!6bIp6GRSmScaX6 z%m^hxp7UIWLmd_|humAp%ZUHbJ&RD!cn1lh@|bt0Co+*{AYsw;ikF8X=79?r7+D0c zcQopS0Et%2MAnjHJ~p<|jc*Je`hdipA)W1ieRqTw+p6WZw%so*WBN)Eqy(<7peF)e z=^FtUpceo#@PPhFpb-ow7<$D?f^c@*nmkcIG6~Ss6&OPqgVzf1R4yC zW`mz$WhYhHO3#4ClP;{`OK!*jGZrUk4A>q?f>=coy{wlY6WJ2cSgYdcP9@1(o=Jv( z>C9y&Bzk95h!d&Fg^|Gko1ZG6N7G!Cf$zb7sq@4&vAP|*79bjPupTR;We1V0GZZMbq*rf)L zCxolo-V+EVlqfw~VO1q9)2bVM7FIXp>Qv1< z9yGtItD`2BngU^}Q@3PN|Zs##OV*a)|FJ+_X7tb6@gU#)hWoe7nzQ4>T{t@hZmu1d2%lU`_>>NC|=jUwbsXKZCV zTiPzADLzVH+3Lu?c=rD`Jo6k&J+DO;w}4Swe(aWGymGg1*)2-?DV(|tC|v|DRFaW& zC|4kwl?hU`g8RB$qI$YPU%&)^4lb;N3Hu?!k*1P{oe14a)KLcvbtYj8lgU1OQQr?@ z0|TJ!$38?jUY242Hk8W9%Y+I=w5Bl$F(ZQ2cv!LsKtWf6=po4@AkNz;_?tjWY*sbE z0HHn@W?aa`Vx{=hAUKZ_P!cK^A|SM^5`n@)Bat@KDza!&F9j1O*~toj0^~4SI1?C# z*@jz9ja1hD=x z4+J49&&K$&6rhU7|J;;vG_$H=4bRLCG4p1e#hSce9$C#Ke)CgJT(Ssfm98x&Sr4>o z=8+|V!<1#ouWGw#PIvl$oa>y5Zr#aS-)E(2zh&z}CFUvoRYNEf3&L8@NCg4YNX@JIM+kS)DZ^W{4F0AW=@Y zK>pmc3%sfaB7^?|LHdnj6QkJUFiUh4W2UN`15@2Hqx6CyUU8NDGG^sYo1s&@^4)au)9S4_S2w4nNrsx}L&7Z(B;^ng9C2Fri`naC_Kv9 z<0HSQ%XdTb|KZxrvby2R(LDF@^U(WBBlXLTg&r2WTJi=T#NBcBJ(c;gURt?h3Zk6} z9ubqX6eF2`6euehGZ|fx9ntfm9@;QBXbUF;7YiG@6ksz!a54=j13>^XhRcM|8@=Sy zH#IW?l?gcI3oSF-u%}xI@Ts~R#K9b_k*gCm0oe+!1H!WKIX!!9k9obFnT@3N?(z$iQXplusGR9b;tkS|1GrB0Bq8W^b>SS1Y50hhEq zlCr~Y|HDHB!-%0ZffWF-o7gs;R6KQ1FG2}bH>|^-+&uh$J3T-DVD9jm7-)77lOs8iF3mDn3()2!o)H-*#q*`rol z6K~;I0w_-W*%GmEPLB0J?fO{jl2gI?$Sj=H>*TI3)K0mOfJ?Q*#~U3N%G7=!!<+xy zSyHWo5r8k7u!mJWolvFE`piQc;yi-2Ce~XxlyL)a?bpnFN~*el*V`-BtUXqIO;!SJ zSb@z?UhGhH&B{~UnRU!PA)4Bl>9Tz-O`jpkdP^hviy6I4R&piR{4);33f6%@%K)uR zch#|B$`vpQf*{(SIIBc*EmKxux^@JsfPEPX&Bp3cl}3PC3R752L`pJa2-up~(IwqY ztJqOjln-tZTz=mA#h3(~D?oH1V%HB?>gXslTkviI>|5aL{ zya?ONCQ8eJGRTARBeR-G0;(0cu*KRSJy8ZV)(1t_=+jDnV$z;6Am9QvV6S{Wvvo_f z&Dy8xQTnxCwvxrC9AE>s;HvS%6ySogVwK?2+rKQ=V?5VOTid@)BQyJ3sf`D_n2DQ>A#ZoP^-~v9}k$KnpwGEUT&jF zO{FCb>Hx#zsShgJ6)4a8GL3|>DL$4Mo-z&c-D5pQ+N6Eep&ZgA+RC_1Bjo?^h~AS* z6Aljcg(B6Q#H#dPul*S?s$`$hp2z};5;?tOrB()i22COT#E_6=$D-uUgF z)1$Jy^kfLlTM*{kEaE>TKEbWT1 zDbcupsR<$9!yC`?o#W?irJKTlKwv!l>I8oV6p5(;&N*N7UF7v`WQD*p%e7bIaa>TA z>+*CxU{sh2tDBd|3(>a4n%{% zzOo9~y7l0ZHfo*z<^AJPB$geKGfY(OJ&|*NQVGDFY9_~>p-^b|wzErHy(+4?5qS^)FU}2JC%2WniU^vWHeeWc0m5`_w~N)nA10M!SmTVV+ih zB&+1GY`2R>2o*~?E9F{|A={T3#c)(KCjejkm$X{ zC5V-?YM%@1P96yCBUn$2u?SR&(*EXu5X0Yae%OAD>i`$<9GUBPs%yK}Sa@1ljU*4h zc5q4p?1>U=!oGm;8jTD0j10%{4cG7tuYplLJYNU|UjT7sxNt-s4a8<45%&epdBe$r z24Cm|5bp;ONAV8l@EphS46kU52x%V;IrxPFg{wXQtgw|yf*uN{ak~Jj&_~>V`0nj7J6US6;M=BqgU@~$jKyoE#t6ysJ z9*|!KrL-Ubb0L0Lo>{&#Cz%{znGe?Kz0K|sMl4{WRi?u+03`w~|1&zNb}TN}vwoYw z|5fYC% z+h&ZJdvsELAN7L>ZkUm@G=le%NnB0dZ+6EtSBBVHxA~h_id+9r-CT#KU4NDvPB?05 z@L(tUH6C_*F80C(LvLq)dT#Fo{E`WE2n1!{_7TX1_+p_>NcJ^6A^8df74L+qZ~CV1 z@rt;{m_CR`=)E2a0WNYfr^^GX6`33$QzGagnYWFz=dfn%p&V$0iHn-HJ?oridrN6$ zWrd=K_dOEM4OX#uJ73!r5QIa6{AHATFsb{yNBG32FeBTu;wwIXTQ##fTe*lg{gFfT z>GAgnK$l-F2&M%6#Xkt-wktDB*3t*yONU$vvott^0Vgw~%H^v=y8s+h``B-L!;Co_ zbM>{x`=@eioR|LT4+@>XSf1DQ1ORhDjK6==qUZhyx0AtEdhR^vvHvIko64zN&|dK^ z|7bs{K(HxhuY*8{|M$Okf0&Bs^SY?5B1u+qQ1s z!i_6;F5S9z@8Zp?cQ4<*e*XdvEO;bFQEYlOYtWcq!;UR`Hf`97DdUd(xJTn0ieaSxI86ASJPa5{AWyD* zIrHYupF@u>eLD5$f5zqAb4zwR_wL@mgAXqrDkYgT$D>b=a;e#->fghU|GqsXw3!Oy z&#!+!|Nj2}0~nxyZXK4`VvRi(8E&z))}VtALZ}&Pszq4gXSHFNp@y&BhTCMiJ=WWA z!Cf$1amQ7cqKYfB*rIdSW!IK`GtyY2jW?2KURF5r2o-znfAtuokTcR}-+)IVnWU0S zGTEe)cNO?sZwX398G{yLnI(i1!k}fA8-f{TY#n+CB5#e2DBNJg8Rw##Z^8*DjM@!Z zr=54=c~N-`gs@|te_F*9kbn|es6|DF^rWJTGTNx4k2)FUf&YwAxgcy_Vw$OFTxuGj zn4gMCW^O_3e=zM`A1x|vYI1s8&> z8|1D}#al0t?HPHjzWegqufKvNyR3p%UP`UOSWbJee`(h;Ok1`ek{TkKr+OP0xa4A- zadqadWv|B{gB&ta^SURp$=v~a-@hxf+_KC128L#-~!enwq&MJd0Wwbom6Env+5l+H|W%{r4^TFzyk*0b71 zJWRy4e?y~cw5m$etufR6Se>`td-FIo-+z0AZ=zijUbx|h8-|#aG7s!*!DvH%T54TJ zp7z>2xBWI_X%_wM(RG7P?%jZsUb^X3`F;A{SR>xL>#uM9wPRsBK6d4|pN;$Fn8VPl zwh^DXImMk@d}0U3BcHtT%QN4+^Up&cz4X&Ne?PkF*JGc3Sc3AZeblVO9=`bF6DxbA zVqP^JqFm($$J7EVBS(kK$C%wdVq8nBImw_1wmQz10J{H<~if^FN7i-;RvmF!2c7X@M8u{p~n)~z!${kuZr$Tw?P~$igQ=kya~=qPkot!z*Gj zi*b424WIQxvKaw>UkoC{e3OGRaS(150}QyB=*Bme@P9En7ZvMh$2(2&j=8d;7W?SO z7;@2fUi_lXczByJlF?d|i{I~xNWUY>f9-y7oFpZu=R`eXGL!Miqb3c-$3KEHl;H~` z&C*9mon;UPJs@Nvua(L(22El{^kDHoSjk)BQhAr`Brki(9Zvf4S7uqt;6!Q6W44Zz z=)24*6^0u6`H+>Zv{ovW#Vkj14vD$k<}L5`ieQ5OGn_mj=Agv+C&3Nm706sCe>=IR z%-WfjrXU0Y4S@yoRI9IcS2M`jM2`Ig7jbS#9=(;*_wNzbTRRh=ROtbHr@aSpaUhSK@mzAd31uI z4&A8;je~(sNYpLo&?r$WO3rc?e-%gcsZU7G5z?nZwR|*G7UKjdL6c$?S}g??8nIbF z!qnp(3zZ~K%ep^^zM~#+p=eUu%FClJHLh}BGyY={ZvJ-T9`IncTT<^LxHaGn@2I>A$->+ttnWAng)3~~3u8FL z8s0F6JM7^PgSf))NNarwY~mB6IK?VnvFi-X%ml-jH4EO5c-2U};nYJOBi=EOd+g&M z13Ab-9x{=OY-AqGgT5?YGLxI^vh`UpT}68}9IjLpf@{^-H zuRGxjZ}`I}6EOrQyUg_N_s2ut?|>hg;E|5_%VR$C znunR|NTfB$mayItR@|2^}Y?|jtR&(mlMepIo4x$3GAGeToDO5-$AV>Mdi zHDY5nYM0A30S|v;Ihx}+qGLL$<2tfqJG$dL!eczj<2=%1J=)_l;s-Gn<0T%WUfkn9 z0%Sl6S>n%H&LDoJ+o>7{-4jP3q)MdYet&rUvF_?(L>;8s~8`3jc5hC+Wo| zaysX8=7n-DrwI;c@}VGfVrO>dhjdOSIZK53N- zW0XoM95N|-T4|SZC6;FCAu=fe2tWdO>6w}(n1*Q|GN}OwfB^8vedSmGe|^V)1sGk( zDV>VPD-6Rspea?V>6$j&1}4Cp?#DCeX?lM=16?3$qNc|%{Ap7Ls-Rw9hK9fe6e^tN zhi5q|dazbpaB8QbhdM>-P);hPCfwu&lmtkC0l4X#!fJj%m#B(|b+yH;(rT@is!*OP zs%9c@kVpa;009synZhaoxT>KR#!%%ddfe$+Fl)0q>#hbVfwG~m(q0Y(hp-lFw-SFq z0x*CCT!01ul(G^Au1c$S*s5BfYr3i{wd!QAV(Yr4fCMCfw|;8`NI(dr0BZh|xemsh zx~q4PDq9Hbz!EIH(xkl3Yh;9g1bAx#?5iqO9C4-qvEm29hDQKt?EiNt>`XGOufhPt z7OTXjfSDAXbus`D?nTFXY^=DfcYJ@WOoptWhQPfNYXWFM4!Gi$E+}5aY zBB`~KX2VWh4qQO78h`}wtoivY%jyNuZpYACEmRooO4jUZ}9k}Xec ztx9&Si+(M!GQi2g+t{9MQkd=Bo;cj--Fy8Ym1f zfZO`*-%V}C5-v&%Zsb;k;ff^Ul4lN(tO0~Tr6TAi&L@}h1?Ao?%#v;*vF z>>6)G6tD3j@0B8L9=4&E|+LiUvmXQpEHQZ}lec^;)F%Hk{@XE7VFH z^pdYanD6bPFEA=^UgiMKcCMC!ufH}aV9c*X)GzJcuP)|qT3&4TPHFHG#sEvN=|V95 z9xyH@a8`0}1jMfYrmTWeZC-?jAS8hXFhB`MKng%XSI~$cKmiow0ZM2wASCe|kAzGxzzoMR1E_ML)Ulp)a=3mn7>DwBP(lbOaaSbr z1&A^wAaN2)0B}SA7@IOd?8$kc2Pva5Bu8>8PV!0aFR^;>C3k;t=>F_opv*8cKn^@I zyX3(I7;=q}ax$NUFrV>6xPb`|1Q$d=1jI6-%<|QEGdL?mQIK;9AF_B*g5pL%SL{ij z=*bS5v#{J z^TbDotDm^>H%EW51Xl%0GXNBX^e{^RNpHtL_i|9k1VLK_DKkJ2>Ifg9Y!NSK5{IC4ZF zwOEt&A~(V@Gr$YSgdFFypA>~7XGK@nwO+FeUu&&PlLvoRQ%xenazvmtSD1A|d;}o- zu{uWpLMV13(}Z8=^Dw6X)sXUI7xhfk#8@B2M6mTi5OG(p1VWrMWz(%vpCVJ&qEkb4 zRA=*lKJa$5b52i$7e6#XpfW5sg5oN&RTy*0&*wue?vqW>^BU+M1KeLIKOcr z_jA#z_EJ;vep+$5!9dQwHf(FO2SaXE;5Gv=cu#*26LXhv8SFD>6LujJw=lzXAs04_ z+c`9Ja_IKy_n8&gI1e8II6Zw)ud4_u;Ym;IQfcT4U zv0#7Pv~QDlM|gP&r+^XQv=X~;9&iCG6ZD~%a1R*8D=YCMd-(;tiyN@=byEakYj>J2 zcaSS{5JvzPjI}q2Z!>?uBL_GPxWP(D`y5NU5c|*_Lwm8;xt&iU zo@1hGPo4ZCg;3LV;4Br6KQx+~cQ||XF`tK{2fBW*^RB-wPN2HHkHj4_IVq2MCV+Y- z@OVu>1R(?Sq0h0SYk94wdf7&KN~rsd2Sr@rxc`|Od~WOenwR&aD+E9b^iGrUiHCo8 zS+KH}vkRaT#49s-CIrE{OZsr*#BQH_xQqKDlKUf?`?>e|_%gD2t1yq_F;V+8L=Utd zU%~b-^nHN^5&6`#r}CZO)JThVNj92SLyK zyi{-W2Lo-;gZx8mdvYg4W;ZcV7{nddyKi$nyO6VHld{GW{NCR-E6eqssQl>{_MTjT zKoHYX+_6l*^>6Ps7m)Ka^Z2Y&MLlbEvba|x7?So=@&dsNBn;>m;5ka zIIXMwLm)M~1Vtk_xXizCR$M>MBfcQ+d?7miKx7gKg#iOM0ud%e*rOm3fq4pQpp=lH!Ay-L6x6s;qeDt57(@`1 ziA%^vFc@5*L@9?!g)d>o6xe@{W=)wlV^X?M@L<9+Ie-R@z(69xMrDG8;K;yXCWs7j zOr*$DASIL$8YQHZvFJ@B4RKOxlr_`Vlu{8sZ7NVunI0OJ_N)lyrCOYKVb;{Umv3Lc ze*p&;JeY7{!-o+kR=k*TW5o7@d!$w! zgmr7zuVKfQJ)3rI+qZG&*1h|71Of(f@D?6hwd&JAOT!SI;xlEucs~~8g0s=kqEC&~ z&A2q;M5_!Wkk#7KBKDURd)|DiG801v)2Ei_zBGuylJ7IMlvq-v*%WA>!-=l|?}`q; zo2ny9s{7_^$~=Y8tLlF$@yg-~K$t3m;X;LyD2OhzLb%YbvdEiHE~*q7k-)n;Yth9Q zVT@768ELH1#v5_WQO6w-`)s-7j?1CA)P^h4$Rm+VQc2e&AbPn_dM}^72 z=}Hw5EK1yCAO$WqrR2U7ElN?$M`PmA*I$7RR@h;QE!Nm$k=4vc(SST|$|#|YR@!NC zyTC~(skLp&DocO6@&f00RY*|=%8<#x>vkQepb)iM^G!h&sxtyXKUFtZoW5cpiKqV4 z(;_}4)v6J+C{ap+Jcc^REF$K>iPG@GjMLxr)LWOlisMuGVz?KYlb{Gd z3i&$;;9?i+nfCfy)Q}v0scVVkrO;9_>&+xokSPRl!XPMz>=I;6Qpv-W z?T2cXIOu=VQBPg<)md-d^~;7Ht+XJIhT8VqZBN0;5O9Z0>eHwW&FaVmA9vn(@3U^A zugNo6phk9LfZaD2y=c3#(_RbIu+5D=q>$GZ2-U6G8jAdJ`Cf1%=l_lvYvV7kDtU}| zIzFwGv&&x2TtP0p!Y*-wGOdmO4r~Rf*yhp#7Da!d4IvB?9nBa}AHm)1brrPW1u>XG z4Q^0Iu`3#ChSWO|ju2WfaMA@x*qYx3C3wvFR+a=use;TZ4{m_McC>~rwQa8;J#d2? zh_I0eaDh{qBa|ER0ELuot#P5cOGOB^p~rbI2uKWKL9ka9xa`V!{==I4uqH!UaiNAc z6e54(E*1ez{bVB{C_|@e_PC2(AT?Pri>!KJgvlYKYtkd54RNTUs(dbiak}FW^H7G1 zVbKkgaAF}D0D~UT?O`0W+Fsskk8{OVnfKz@sJxfOLE0 zL+N(FCkS7zYmZ}!)kZErK#=&YknnM2LAE0SE;#c)>lBC^wm8)G*^{hgHS1Z?nn`?` z29(9|=Ue;PBmx9Sl|$nH7HPZ%u9$yPLJ)2!0}`7`9YHjLQxnSrryy4rK^#_bG<_6b z1rgc6I<`@Pb1Yx|3fKTfwyScDZJkzl(~N!Zh6rQP2{GRz3p``LZ%yE`OcTV&#kV}_L*J%W-A8I| z8(tUc^}Sj;Z-EhvmuM!K!3~~ZeIXoS2~QZ4_XVMC_nTp90W^XD{x0wYe9*@@I6%>H zDTz@mQ=6oi#VsZfh5un3V;O(Xm}eFSO6@e9<0M&Yu5b-7UR_CG7aNXNLniTpjhy5b z)0oLkcJh;j1{!9462BPM@s)?8I|4k-#~)^4h{ro(C69T@>m~D<9SmhPubItlW{i!6 zlwT~*88c3!2i* z1$3n`z1l->n$w-8aH8eRXi>X{$8?3Xmsts9@d`Q9t)}dyT^;MOdYaa?wl!=)ZRJt# zTFYF(9Z*fxW8v5~E%TQ8g0&6a1bueT+k)*xL|o!@K4k+AihJj{-H#nI*c3cg$&>se>|*TGJ6r$0Tzr%YvzsZJSVbRFEr;LFz;VUi~H_)zirg7oOZPzzSwT>3*0GBY|Ag6 z^PTT>;~oE2v;Ti%S@M**Mdn?73&5Nn^{My#>tT=Ro69br)x9G_g&BLt_tFUZ=d@MSO8|uVGOPtU&*g9 z2H^)2{_&H){KI(1^5GxsSTV$Nqo+xi0_@!29H{01fZ}5pcSi z5C29;1Ab%g_RIhN4*&&l`v@=rF)#x)a054K0T~c?D6YFwP68(o`zp`^zmEe|a0OYg z1%Ji^KhSo31Lbs#1QnnHR3He5U5a0rR82x9~WV^C?d3<9%D1+cFI z@@NO8um^ur{{zH;u$3VARHd@$Ait_$HX z4&^WozYq+k#aaYP$36i2&JYdNkO-)dN`CMS=dchB@enz%4(+gnuI0)k@B`4$57jUb zt&k8A@e(mH6Jbvg7ja4SFY@>>64P)J*{})+F$jM(F%?yD6~&JJa?bvAtrL+Z;I7OG zt*-(^aTKSp6a~=@QIQpaF&Kq$+&EDdjfC1nFvo2126OQbDe)Asa2Ta=8mZCHiqRN{ z!&=x)8Ltl)MbQ~~u@Zl=8p*L7&2j#I=ftsUQ9EHkOdi1acmC(ID{(AsI3wHF6`( z%@u2}%69G{mqZK~Z6d#s9!=2~qmd(7vL#)zXB<)_X9Euz%_IpDC412xE%7CJvL}5~ zSYlEpVFUVVQXWsTB2!W)G4dywvMHT%GlG9oD4B!+Xs{%W@+OfoCl7KZpRy~x@+)mF z9Y0d%Zci$UgyNWx|E^LJvr-(n5-i~|F6FW>qS7n@%G!$Z7AbNKE%F}c@-G21F#Dk{ zg)$;V%_iBBFOzaB1+y_7vnvTxCP{MC5K|WwQ!6j>Egy3;IddN)^C6qY0Hm!l3vz!k zGqW>M|1&ig@-y2p1V?i%EfWyK5gJvqHf^&USrZ$zG2KWLC*ATkfipOPF*jwAC|^@A zkMc5qQ#hHkIS-LIJF!Wsa@TsZEosv^u`@fj5IXG;E&r`LmGe5ab3DnD1-TOpn`Hlv zlNpm!Hv4iX%QHUZGycvq2EFq;VY3A^G4l$am$HfkM-d;iK0oj-OAb8U(=_>$MQ!w# z3X239f0RO96#onpKamqF!&4>a|8q*IborLl`Cv3g^V2s^Q39&;OTl#Ut~B(Tl+jQr z1n5depE1=gKmY(BOyM+6nQlxS@09RT)K1AsA|M3RbT(N@0stUT<#bR9HRR}&>_Ri$ ze8WkcWC6C+I*}s<+!O!^pinLKQuFOlpU%IMe^J!ricuZ)NfUEMwK4<(6#z67Q&n|U z-Aq$UE>RaP1wvpa@{~~<^--g=piDJTy_8jb^;fOYBiRZh%hD`ogHx{z1~fndYSmW# z%T_~`J)x9Tfk#qL6eXJQmV{cuNs1O) z(v~y`_HF(4Z;vHxuWW2{i&3NRYH^iX#S?HDw{dkuaC1y|M3a={DgsCrZ4-A*f0Z&- z9k+8m7k1*-_WaFWT?3%b|F(DTw&Jp7Z~gLfUH5gBC3J~(bV)Y|%2b44;9233T3GjD zVfS}|S1}+r!?Lt+1L|@SS5#wDa~U&uo%eaYPN{P%z9_dU5+fF1a43HYt}7k(SDcL4Z-G1zq@SY{2Ff_F=VE;xflxN$d_ zpLVl%nSb6nxIY;<~{kDXk&uPeacTy;XVK|35upZRU6nyxHfjEeTe|U(AxQLDT zh>Kk6c6wp{Q1bKu`f9Z{-h2jpGgv#*x`fya4_cRxIlwa>1eE1vgp&wGYtWxFn*loM>LHTl!JQ47oe}z=6*{3AdZAO%oWRL6#|6f6=hfo{?s|%=flz5C?TISDRO)nL6Z1+7xa&s&jg(rMjxA`l_w^ zrFq(?#d27uS0=4xI+5mAJ4%`mXUhul0Jb z`MR(D`mX_-r{y^;<<+aDn0nh$qm{;)p;?sBdIHy)t>1dGfA=k}OB%2-JF_)=vpM^( z1)Gw8`u;kxY4ZQ@o;x_h`Y)}mFS0L?vR!-JEZY=1d$wu2wr$(4KYOCPT5hLU7TH)@ zdhN=l@3C3CtzY}Nmo2vIK^~U7xtaU9ojbard%C5&x~coRtvkE1d%Lx}ySF>H=Q*_5 zaSnpb3A5N;}5uG1X|C%e}nMrwh#8JkSMw&-9pQ1l zf1YAn<9WX4eg5Zxz7a6R6!gLvfa*PXUQr|&=$XFho&M>e9_mq^FbF{9t=<6m#;{-B zx3`SH6r13)+zR(ReRIC-EotYY9_pWAJ5nH)_kvO)U>G!B>C?XM?f&kWe(DF~0wf?K z48ZCIfB;frSb_QjMcot6-R29P&&|H^e=i8_@&4zZLIRMPhk3rI-ywPgVeSiJ>3jYg zZb1}8|KSy&q3Oj>^lzc`nI7~}Kk}Vk?*$`vE+7J))k&~E1PY%4ZJk zVLveNi8%mjTrWV9U>^Ur%j;D)-EN-v=^ulP|M*`)YoMX|e;y#-^{ZF!-nnyv7%&iZ z@SHIT3L{30V5*@hi5N9}YxwK}#)xGaiX2I@Du5Qu=atJkkhdx#yovuuu> zFl_(!yb!aKuLCN7{(DazN|TsP-cp#NUgnkt5h`SO?wf{)T!!6p6H$V((kT>Ppk(Y7 zl!$o_ev8c@LL+0Qazvz*xwuG_Dg)0{i`Hz}5D|U`O`xGD*z4G{8_faKy7r|Zx_=)} zKFC)+46fBwps8E0I7BMRm4j(`m%w9r8WDrAaF z#zde{NF6Ekj7tp?u~CB(oa0g+3TcF3X2`7s*m5>C=iGTPTo6G}1SB8<4CU2WqXt{q zcocU8grMV(U7aVEb7%kMv>sdjC8^|+Og8D{lTbz}<&;!bDIb6pjfk0oZfGgmg5QLo zkQ*$?BM(b|lqe*@GthXnP-XS>^9~eHWCEK)!}J3uOTsa5i+in+2Q2Ww1pE7H zw+MeD3TFsE^GvM0|9XJdC$03-OdEah$-8pQz?$4H%s>vj zjwZ23DU>$rITU=T9KkE!yh8FvCPS@j%6_eUQ_JMO>{JfW(OjL+ zFW>9}ZPuql@7wg^j5qH13{avz0~$Q*(mMiw zb?gspst^dPLKn1~p( zOyR#-y~An@*jBA>g)H=Kgg0%Im4YOounCP5?qGF}dhjrhY8oW0ZXuxv3}c8y1mK{U zgT$Qx?>cyU|4~q!h{dJM%u+S_iOxD{BNc4Xi>MT%Di1(LHoEeau#BZFXIU73H~LQ= z(C}k#%3wP|@Mdb~c!vwW(T_L=ffAXxjhYVf4rN8449Ltw`6BQzIHD#KqR=4&xF8Qm zexrzcIbs1_cAoKMvWcL?A`F1?qrfGAlr}mgDpR@2Ft(DG_{^t1_sP#QVhI`m4X8i| zO3;B4vqEdbA2h}W&|U#l9_Mg>Q^$;P4`{pta|vw(E^#SQjHG~n1r4c4!KoK<>V%wQ zsnI!uGNVo+PkBIL9z4g>19{HVi}kc;8T-l8p8E8sKrPrmMM~6y3S@pGlZbCta*lvH zRD1);LnHqc5&lE*uon2u!!}~P|tc+fv^M`udtmcjL{OW1~g8o z3#f-4`>SVQ*0L5YQ$U~L*m`J|B^B*#OPb19qNWuuYelJ8CItfpLdtkJ;);#rWktJw zM+IK2Ko5-2*W&(lNYE31Yz7m{+~zv>xzP2?Vq06(Z(tTC!x|`NaT!o&#e}s1bu3Ik z3(#`}a3H^;?nosGE=rEGZf(_6-hlF`D{87zI^vmcwbO&(2Eho#EiMs`+k?X?S8>rz zu!0xN|KP+-&scp zQ98BU+Ztht%h>6EZx?SIX5;Rd%!6ICq8H8RMjLE|agMa4Cymw-$F0Pb%)uoImjm0{ zH#>owAbI8rXs`g6ixmKKq5XR39w*w-u#UB?XT790Q~K7pX6&5pQs*Sbw}By;nYuzr z0Ga=mM}nok&Z+aN>Hr%Wuth2}t*1@xYFk?}w$8P-w=K|rUDu`8C;l}BsZ8IvV&`4E zomaBGvg}l=I>%#n^^dPj?|R$&-gChcoAynue)Id^{th_62QKh}6a3%?PdLICu5f60 zn=atiG>W09VwOViIytk^%(A?0c6ZU;V1{?A11oKOoBZS`x3|6>#`2cC{N*r@xy)xy z^P1cI<~V9_+84$786Se zAB3WhddrJv{Kwb+Y~+&s?Uj$c>}QY0%WwYnxVL-f-}3o8cl_rM^e1Qg+*|+pHQ&8kdY`DpcRDBx2>Q^!Gk)YpMft1NG2}}={r0>6 z{Y<-uE|H-P_0 zfqu@$YU>Ak4!DCn*n?0~8N*--!*DG`ScFD@c!Wrpgi5%COxT1@_=Ls~gu~EemzICB z)qONrdeUJOrZ*m1!Esp#6*j1J&Bt_R=7VUMhH7{p^6(6%a0^fvhjKWFbXbQ+cnhZR z45-syg_45dwt_S$CPqPg;~|4#C>3MKdjyQ5AWcHLYRrTn2WkdgquiO z*X4;UzMe85@HVw@`_@n2p-_gsMS_|L;JB zRrrbeCyJv8gXD2`$>mt7j!LnP>er6!7mpTsk&t)~^Y|;ZA&?&Vksuk8B59AW zU<~3@kmrSe;&_GWNQmEoju6Qd5*cp^=!_V7lQ_A68p%I^(vv>f4n3E#xr%k!iZ-d6$eEnVX&=0Kh%*>G za2cFQF_Ue_bP}+f*qNQ$2_MZVol*gr-kAW80zuTdeitAB0I;3vxt{EQc^KX4olgOq z-l-MzB?0BxjuHR>`q`fR*`NMNXDFzaJqMpp0iNEeKm@P=v#E%OlLF`o00i)#5;~z2 z$`bEMpi3d2!C4dZ859aSoKIPuA)ucCaGVu-q9}@+7mA@tVW8=l0wLfc{~Zb=9y*z; zc3g)uqWXEADcYkxnwu+s%A%xLY^WGB=t85@(Hk`Sq0H87H;ALqZ0Z+Jn2F_5pV$^kODT-L^gP%_F`c9xt~_LrfeFISc;(<%7O~?C>n5aOc4~!Q>IUP zraguu>Di`$8mO@NrUa^`MhAB3fUBaIom`dz z(s4CEBCQi?V8W_@pTv}{?Aorh7E$5Ct>cJ=Lh3HSiK>l~0IOOa=1P+};irw@t^zx- zi}niJYGP}-kU|QR`z5YG;hQ?ruhoeYF-j%`d$AZBWNvT)8t{qTN{kr#kYAUti(0O~ z%8Ut^1R1-sEDK#k0Hz*0j`do2jyYp6AZ!o|Bodp0GeKE@E&H=T3sl3PtT8FFom52$ zP_mLDv3yE$Fc70T5VTS|wffW|9lMftMz8hCu<+CZ-Mw7He^~YWuf<3{`_oy*&&RK&-&TlK{ODtB-ufUVOwztQH6vx=b9Lvs}-GtCqoAuca(k!q(32oL8&tq_i0{LPO8>9LoXS(k}hdFdfq}J<}~c(5&=- z&^VpbI=$08-P1n()7Bix+&s~h?7$XX$F{uE<&3bwJIc%ImUJ7^q||ODZPJF;cB{*` zu)NY--PKnVjZ2*~U#JAklpd88^ zjVMkXm(1!jlT84T%{P_}&mxD_+^W^TT@$B0-PB#()_vXBo!!>G0QN-N-u>Nw;2qxL zJ>KM9-rxPa3Sj^SP|50@$y^-Pzx~|6+tl?6uEx_S$;~%{#zjPM*_cgWi9M!w;!#N~h<=z>1zgkI=|e&~pQzS6d>-au>s z3Gk&Mz|>7W+(j#6rcBv0w%>V01aO{XKMrkV_S~Y*=cHchrhe+Ep6U$G0$wtnllp6jF@qY#1H3+(8LEv8FcuUZks^ioBrT-m`jT+H3dI~L$$E9%ld z?bKfF)_(2(*q-g$zU|z9-tFH0?cg5n;y&)=Uhd_d0<3-jy1wr0-tO-H?(iP(@;>jn z{;$s~Y)GoxU@Pt=-MQHzh6z4JDL2m90-|!Cq@DLyI5iPBoi zkbb%Y_|Y#96vvxvcvbUezw~II_G-WOY~S{7|MqYn_el@j7GL*vfA@Hw_j2aCPoc0rAziy^v_Lkp8a-aE{zxkZs`JVsz zpnm`#K=`Cz`lf&SsGs_(zxu4-`mVqFhHvIl_OutU^+3`1`JJgqDcLiF<6Ejlmv8x@ zKm5dB{KkL$$glJtpaQSo{LcUU&>#KMKmF8S{m~!$Sxbz6i0>j@di#))`^^f~K0_4= z;PvY!+@OyA3vd1G-~R6Z{_r3F@<0FdU;p-h|Lw2-7JRr#0T4M-T!>(Rz(IrzE`-Pt z#Nk7T5hYHfSkdCeAUO`Ez}V5_#0v*ud?Z=Yq{$vBRj%xj!{tkuF=fuAS<~iCoH=#w z8t{o2*< zU%-I{4<=mL@L|M>6)$Go*zse?MESDBn8o zmFRm2xPs8r;4TA$Na3J{8Vo`L2qm;|A_6B9a6=9||Mc)f5JMDk#Hljm>%xaD6HYUQ z7+jCS3oNWlEfi~X$bUo}bJTH19((lhM<9a~a!4YJH1bF!lT>m^CYyBfNhqU~a!M+z zwDL+Uv&7OyiBwELK^jxjh=2$UER6v7xU^AAHrsUbO*rF}b51(zwDV3p^VD-sKKrCo zO^duNtGF^fqwu)~bNCWbKmqmhQAi_|bW%zywe(Ue_0*W*CfpZ_K10xBPOQi2Nv>sHbpaKQD_x^kE!xw*i^2;~>eDu>-e|`4bcmI9( zBw-0nctRAWP=za;pnnQF_(B-Q&_ZVlm#J8ILmcK%hdbn9 z4}Dm^56Do6LnNYJ45$|og71GxU}6)U_(Uj1QHoQfVim1;MJ#4fi(BMk7rppJFosc# zVS-zgv3nJGWGPLl!OX-hFW;*G zC|&tVSjJM8v!rD$p-9Q@G4hnVcHn3?nW{2~BW@Q+7uq-hXKFXi09SQ=RK%XFFd>gzt${p7S(a>rit} zcji-{`{ZXo#TZX|22`My84qKZxkBcXAs&@jAm4$8+GMF zCHhg2stsnkJjq2ldQz08RHbS3Xh>c9|I({{sdD$MiYE@)lPVN3g3*cSD) zs-T;wYBx58zvYR5F$;}|!&)1|IMz1Ljq zX4fI0rH^#0`(5x_bGh0jFI`!iThae+F$Ez=0Y`hF0S)wki|cJ~Z;cRwqJQ->1!=kD zdP87@la9B%1J0^lbC*^HtaXb-5C8xO0APRi00tv*Vh#WRfD0_R!4D>54kirYn-XEd zIdCF_9Sq`pG5`Pq2m&Hi%;ErFm%uaTY)CTJAn!_%2m}B?0#Lk94O|!o6~1tWwYcLR z|M-guSaFMSYGDkQxWNWcGJiWG&}2kj8O9tHFpa(ZQm0BN$5BKwl@l5QAv>`L0uX=( z7R+Nak5S7`zA2QA%)}kTrONA^bC$8pWf^~2&}KcFD!ZHBES7oC?L;#GhAc)!ld;cF zE^s_kX*mXm2`yJKfioqQdvh*g$t$<35!sib73nIHR~U7bdw~qL_jP zlNm|4jlkNXJQYo zta7?FJ>@DFz{PFNV+|PL0vG?@G7+{;_4NdgH_IWqWo&bN=&6GjZlU z|GxGC5a%RNzWUZDf106;c6dL$!NDOpjFC+K^}oO896oyc*NjOOY zYd-;aHs7l>6N@t=h&%Gj|GxWcKLpgGHB&%3%RurIfPcaRzxLxgI=epd)442SI}3!c z+UvQhYa$Y4!BeY1^1HyF^FElnGZ|F5{0l+;%Oc?;J|KJ~+v54%VT_ZBwyGSS+IyXZy9}7z-BE}XB z%?1+y6)Zq{d^?*HKh~4QMZB;a;5id~JPA<5)&#b(BQZ;y&C(n}t=!2cVn{Uv#DDVx zu&-OqCi+PLAhG-7z3bygGGx8%6TjRnM4n{Jw%f(&WFo3;G0_}1(R4L6OF3CIu@-!~ zjcg)fG&3l>PKIpGCwfkDyfEG?#0)D$*UU}XEHMeF&AU_p%^;2=ynuyB0R!zz1RWyo zq7S!02zjimE{gxOMuSf5%&Ib?%@vAZ+j z?8zi3(J11?428ER-MQWrwICQK^F&5kTtM@Tzc2+s#(UAXlf@}2OD(N6DSsu-)m+jh zO;Ia#z|zDw4mB|X0MdsL(xux)x$Ds{iYFMjj0lLB06kO#Fv0;PPy#hjN8QT=jnv@z zv-L{k;?r&nU7g z7%)&qEmR{sRDccGK~YqKl>kOP*b8`6g@rCjZCJiB2)r_o20e%b_mog&3{!~4Zoc2Ks2o);S&f)y^Mn;SR)h_fz4aJ4bTLzTf0TrgIy4X zHBfbQSi=PwOI22}8*L7ZqJTWpGCSNnV-Ne%>qc!eKVf6*P*BwX{6VBvCy0z3=pDQ-? zg(HZH|tIIzVT9FnTU!+6gc7L51{Dd1s7;5EU?^=$c+*bC9JjBt zVJJGt9~)LC`nm_R#J8;?6J|&PRLLj$Iu=ecSf$Tj{Kqe4)0t&i9PU=2ZK5Q-QhXG$ z2qgdFO@B3EW?NMhq`cgUVn@6>0%urN)PE-akU<5AQfHvep1Dt|3XiSCWIFsrS z*w;Xm!ur#I5QsfXh9XO}!&T%uE`92z+tn?i%Kqfp@G}C>LtSf>y`qj~p36n$Y%>q@ zys#6)3jn?uRcT!eL}Aeu%l6aQehajw?SBcQyp6nRCh}EMZXyXx3mDMZEd;Rliva%| zHOjL?;FvmYH0lm^ys5KA7-&ZE+tJ*vzgqKCXq0VXEbXGH?uW!8Wn_p4*+#Aw=Q5_g z2_x=%J!gjW)0Vk_!2?Iw&Tg&5?Lqa&@0I7b#vHj_FGi(n0XD)2AW-&V*u6GfN`Dpq z7t^SSgMQty#jwDO)y4)korO7G6T=E8VpP*`X3jxjgkmPFuWwa38HULVN8^MG#MY~| zo^9$NaB--sHQpBKm5a}{1%MWl!S_8^iVVT9d%!f7;xx5E&@N`T9Zw7Q&GwUFEMmf* zgTxn8Z|6O6^4n7CeO~pX@uTf+FMmfp7?0JYgxmjCoz$gUeBRr>J#c^4>jfXQ%$TX@ zuq!%`hzLhzv2#h(3s-I2)gWlyDLU`UJ2i_2wllnHk@QENoj;-TKA|pVZsqXKgE;~) zQLXdG`~65gpFFf9z2wYrxGPWHYcmkL@ktg|NY^^aHb5vmWo#w%7yiK4G=JM6-T$Hq zLA#_ZLrBNB(*3*~*gRRoJe9k2FA_bKTb17pTORvu0lYz0&rT+@^`#TKK!ohD#_hfbCm@T@VG2b1zE{IA@FDxd=zzx!7DdDeB?tcI_+*f|Q6& zPr6Z;P;|#@33~@5Be>C&*nhKwR|%^-cz!21G(vcY|G9$KB7V;Zk2Ir$cZvRPx`%hQ zjjwo7Blt~EG=ML8ERuK-a|x8+IXS**hfnx>Ux|#**E2_+eZ7n|7gz>3aQ1q4c26w> zOXO9N_rOl0r?ucB#bezy(0Go}rXSpz?53lDp#-SM*bq(a11Q(deW z2k}^9z7TBrOlq_|oi$27q_a03fzWwF)r>?%3rK}~=h}$*xE`a2&_UWegM~s4t9sr2 zWG`}%uiss+-z3x@Tz}QyE7t#d7pv+D?76@<`uEw`{nPczmyxe_z1fXqaFx> z2uOermVnv6e)`WTZD9cTXMhl>el!9I1p*ZWDQNH@!h{MJGDB?W@FB#A68|SstZ4Bf zMuRCrT!@f>0LYL85=6KVB1guQDp#^>>GCDam@;S5tZDNm&YU$P5Ri~F1W1$-D-2YT zh&wwuwuuOEvt1PL7i&XvTgek2FC~nM22J_Ap{J!diV0}>-R6< zz=8+6Oc4|(43H@%5*29FEab?NCsVF$`7-9rnm1oYOZYSB&=WUaU@(_%0tp&Y2o>%6 zHSE~3XVZSofWQTuIX?PiT$D5L;KGL!FK+xea?O99Ycp@oHb;t%AL$xUz1TVS?Ao_; z@6Hz}fz~x8jI7vL!QK7auVI3R(d%@qPp z;E6ZXZ}K_#AcPT0I3b1f6?mb5a7i$oNDP>?Aci4|I3kH976g<5x`BkCc@0wdB8)N0 zI3s_JTUlr#j@acV7j^#S_#==(3Q1TFE8=up1Q(r`UX4vU`6QH4s(0g%Rk{XUX><_~ z!D?1|`6ZZPQgi_U4B(^z0S17`pkq?H`6irk%Gni_V%qr^3>xs*;RV;-`6r-(im2F4 zi4ovui*3%iD5H%!3T2&wO1f4KA#6yOhlqbpx+$lfy0#@wyk)fHqota9Dyr;7>M5%; zReGsNT>jT8t+m=(Q+PL#nLtXUrur+e!3zId77?r7daO#XatIQaYa;tBw9yvSTTO-$ zFzZDh3{=6e-Fo{ixJ{|Bt+eH?^y+CKRmY;a?Yg_Evo7T@S8gfgfEE$q>boz$rJjHH z!Mg!lR4GrfY7no$2`h{&d1kJJ7PtLMJTb*Jjyn*-3u~;=1>CY#e6f)Ko&`W1}>eazz6xVJT%coV|KDYCFi^_3?0X6^U_gE z9UxB*x8yQfALL*(*Ij$PRSzosjNX6LQRkqiNC_NVHrs92rXWmMqcu0(b=!S6-g)c2 zH{X5x{Wsu&3qCmEg&Tf2;)yH1IOB~w|9&{YZZA!+NY!C0Ip&%3wR6ug7|=~I_jybzB=ootDw2mmR2so<*?g+yG&Krq;2cH`~Exd!3#e;@x_;3fzp4tFPwq| z=^C)M^3h9w5!^P34Z88!Yrj4B-FyE%@fB2{Lh{oyOo2nI`}_Yt0Oc2gf-#F-0X!hcs)xV@GO&SCqg+U6=fDX{>wOfwAO5^F;qUL~=KPJH5E1{nVm1AGvQRuqi2YInsgaX-r}I3~0waI>bw%j20N-EQt(zK=!dtFQ02c9hS5;shBTWJwWvk~ zUj=&LQ}bmrpG!^H5)&5Fqf)i1R&8BUpE|vU0#mEeGL#mjHPrF-;A>`e6cNNWw+eW5 zdNAP6JKI{WMZQaJ=s7E1OVxv)MNY2D+bKc)daQ#A>}OlcD`Fc(gr6x@u(%T_MjiW8 zBLz%uOiCZB(k;$}DQvopAn9D)IL~Gt3@l#{q6)$`xMBeqVxV|8lm#uuH*dyy@TpEfc{4Sulp(1KYe_ZrRCtKJ-`kEa;o%fIl%_^j-)RGdv@@(p6D3 zq+?b{D`%Q7J;52JEiG!OV7k*HgTZO+Y-*k4>SlkT9yP3&QtDNMOrUY5wOYt*VFkpx z*HW1^t~a*RKnJ^huXaigG;pv~Z6pO(Nwx-r7AGTwVA(_icCi<>n-*vLfX8l14gdgv z3ot4NFo1y(lEMH02p|Yn(SW#N@D(GRTi)#brj+DLgy9c9he7f&`*tsX(DYEA|>-CQMq)^`Sh3Ea}Nk4xT zk&E43)-u43iyyV)v&ihH7dOl2egSb)9{@=T{Pe4Tt%!5nDn@{J^#SnCqf{RHN&&wB zOwW`a1o5CFo2C`cY7(woWn&0H*oVbY*AyFS2M%*dpKq^vS_fZ7|MgZG|z#8U@ z5q#kL;h_XjMGss6nw+At<;#BGn> z2+ZLGirb;MVhTE966oN;P@}lv;5brWIqD!f3SBIgq4?Dz1`47nid#9dA^;o~7$ToJ zQeT|CA^B ze&k1fV#0vOhl%97d?ZJ3WJfRnOWu%{fP^Tb9XwWD4rC(|A|z7OVJu2T58&k7wI2+Y z9{}_s1A?4Y|G1pn6=PAl9r%G9<8fgENShIuqCg(S62jnA>fBW*+)gGz8L}KJUZTPY zK)QwH+3{bxxnt~^V)TC{K@gZCJ@TL29fesMqxEH+4pwCnWFO0wAseP6#V-*3g`L=Ay(sVR&X-8B*acT~xKSz4Nhgv5X^Xm>(Ea6m-lX_-ToQ^YxqVz?8N`1AirX^MTzB@U%sHu7s^Y;2 z=&$)8epqOJOsGmmz=KXEg9fUg4yuCwsi9UTpC;;v>M3Vps9e2P!EtDnXb5W7MMUy>^34G!zvooe>+AaTYeG&&tAj-a!u=kqO`P(tFu(VUJJqnsK6 zKK`Z#tSBY|O>IC2qk3dYDym8v=%Kb`x)7?nzAL<{i=m1rW@74QCf`39N=ZtpknBi< zdWdGe#0m*Tr>Y?U+$N2}0CJL^oI=II?x;e}=u&^Ys>HczRM1=rk|%yzt9c%*#6BV< zMqdk-piho00m7hKZmhDxX|4`kw6a{8hG})$rL~e<8bVxtHmq0Hgl$T={UMFB`01R;ExupPjdaPBzBhN6T=h^M#L5BpC zVAZJ|^B}H%`fc{k{xanvank`d^s>-EF zZ`Q58MIu(Xt-C24s2)Z05P-X3YeLFrLh9%fF5>4-#k!3fnUW$As2&~KuF&c^BIo^RhBpx=h8CYBMof~15}rt1c30w8}X zN0ulfy%yKzu6;m|y4*$WPDH`R=bOgnUW(gu4&;?eECaUcQDmPuX5~YzTbU2r6(;;R*r>!zCH2G%Ivgur%cM_9#^dY%J6?ClP3dq zrwt;nteI{zp{^O1M(YM@>@KR-LQ!a5iyUu<{S;6fV+0G z>KxyuU?=4*J~|sgr+`5tq?3QbqX0~#uXgO@KI{Efw0ctW1z0on(dNzKvOen^Kf@zA zY4j;(WF`XCM_%IvoaVBmCQC-BX4)|cHJWHPGi?x2x-fw63I-lGZXWVpw<6%j{{}1n zd1?DCvI~>#Q@EkT4d>d1Usc|04hzjBR^KIdYd4oC#|kOWhMxrVtrUNLb>r?NQ@^26 z3vv|0a$3hSQoN-UMlvN&Yz$*{x&?IjH7iw&u{x1!O8RM~O6D44YBWpFhT61jw2`{x zP-w(tzQyhLk!*l61wMOYv~lsrPTn?$-v#hnQ}&%AR|N{frMvMc{Y9V?b~GX9WFm?% zV+KKy_OQgZuVw?X{5^kRW^3Oi62Wb&pcnq|>2<45j3Itjufhpr`H?QP5@y`Cp!;d3 z|J^d2N}Fr99~Bx#b6>VzCzQEzq@Ol$tLI=MKDhuh4@Rbw^k%K%)MsNc4k`aI`pU z2#0SYrptOucKcOnt~3L*G^T{NiHl6ss<(*S1Y#RebD(%ujAhlKUw?D#Mu$aJOJRj; zoRpVzaNwbHr>B1?f6I=`7P)dHf(~k+%gTg+yR2xuI3Y#_AhKVjz6d zxKgKhi=&%^7=fM-M;uyaBJQuSgt>Y3_EC2ui0ImhV0{{sB z0Q;A1lL7n!i~pBvlmYz%u*UzFYn1_85J^Aw)a&oR00+#~y&i>VaF^DV0g(hcWe@+C zYnA~g0ri)YmI2cRJ`5rd|Cei*0XPBCm!y{gYytY05tso~1~3B>CSd=UYnTD~2W@NP zVh_3hmur~;i2+2H^_c-~2msG~^$STw33&Jlr^bf}|Cei>0rU(8&Ieq3=fLcR0V1qoJbe% z@Unnd7ZEt<=Kq&#rUB~}$Iy@oL`GFrjo}qP|Cej10Wg0K5cLE$3KGPWK!gbeiYk~#;X{ZKB~GMBaachp@*gQ@1Q}8s?b{u5+r7fB|DbN6*7MKRASrKtqp%ioaD}>Tet4V ziqHxMA@XkKUzzndHbfXQ2v>nzFy>TAWHC_1K{5)=vsaWvH+v5vlxL)1Ba{d$^V~VP z>(g_N4?}#lmy5t}s)qrdnK=mm&U)`uC&Bx0Fwc6o2n(D!c=PF}p&yPmjnLTMTlJVT z7Wlki^wg3yLMix-xXgcNk0pxgi5#l(hw``suNN6H*7fBDmgMVwU_ti+ITGP+I=kta z=yOOAlOEbhkc!AN5312#qwOBO3c743BJ4vbJN804?mmMQ(omp_*b7lUkP6yRClVJ_ zD#YdzBCsF?>)C0wgmn9?9{T!HXeXD}vu~#Gw3|pl6^k_TNF;xgRB}lsn?%kkD5=6q ztM{U^rygH22&*i!&?18jG29Z9trNts>rA`c`N^;p6EbBl9ZM|ey~tX;P&GRzsxzk< zd4njzf}T;xHZCH21)(=38~-duCxODyLV=*Mt)O8z#0?_hD4lPwBm_<9oZo=6E(l{1 zQqROjg+ec(AVq&->>JO7Y}JidW1Z|jjRqUZ&6}u8m8n$``sTMv+nZCNO!mqop-kDU z4o=ey3xZJ_gIG49(Hv^^T7g!Q@K>iWyey)~Uh53M-sFiaLsS#`(A{=N1aUZcI|SD$ z7JW5lT_VoG%2t1|8mz*4Ff4}h13`9$@ z$RfeZFZUQzOeH+X0!@}VndYZ+3o3P@YV9?QPVpMcxuQK0idCeCye+7Vhng*j;7C>5 zYi5kQ-K|hAp!iwBd+Is=HHd{Jignj}MjKj`TSS?3FNb0#&6Gxd_OxjZQ@m%=Bq(|1 z#p{TJTW){4gNUwR$0kb_BT7sWdoQBd9?CGNpjP}hlQtwf5J|V7_>tY6|X7hYVbPj1WR>+{?33cSknd^!-GkKUoX|H=o%>Zb%<6Vkv7|L7) zi7=aa9ZMoFLfSp@R{uc>IVL5e0wLDA7p#PYs)CfL8bVZ*mb;twgr%Qk8X=f_r4+t}%Mud!wQIuJp@ZO=sy+9!k z(HUY$m>7{!?M)`eprISM!jY^&=!IOk1t^;ql2In4YMD?X7chvBVHWe4e0yN+Kqv^m z=n+R}fUmX;BKf$2+_RCrSttBnuSDH}Qy}Vqpe98o7Um zD3rL$;p~Kqy7S_**aHm_j!{#3Bw^-=zy(1Vxl3Q*pIm`i+y8l71|X!=r` z8|7$6J?bPrUd3uuG7mIF05Xyhk_>A(BwKjkffLM+k(|)Q03lLE|54;;2&!ZgCDWTv z?Z&1GQP(BUS;BF#CUbEVNNe^IPs4vzO{!O<>QvRK(T8aC9_JW_D7Y#5*mK3R^?!`8=$i7L4RTELKW-Y?>y)`sG`gCUJ=c!rQ6%5jnmPfe2Kp(lM~KrO8z;51MNiM=-)_LAk6?*4pD9UsWhUuIf3A z@d_k^m9%32U95?n{A4IcS;|wUa+R%oWhkS#A1Xc#i(mZW7#}yeD2RWl4U}->xj5nm zC}0FUhMZQQ{CKbwAu?v+{Mpme`Ob63a-aSDXFvy9(1Rv)p-HS|E<@!?x%hIU!*s|o zli3D5P=cB2QUx1inhYhZr|TiC-UcCm?q>a6_MO7&oMt5qtANhslDN;rY6C9MP{lo-esF{*T;bzxx3Zz45qLXW5?0elGVG1WYRfv`X6Ao)$NBATCJ0L03nzKW zO@4Ber(ET!Yq+YJ0`Yi%$q(ATteCIv?dC-Sm^h`1$8IM1{@a6H6FzGwrWY&NC>;Y~APr&2(#^-+Xz2ANC ze_#CJC;#}(Uw-tTU;XK4|N7nEe)zv%{_&^({Owy`RD)s{oj88{67Hr$E4`;!Jmq^lpU?<(sD@b6x7q8r>&w30 zOF_a~MLqC5EL_i!wK`cZ`9&+OR+3VvOG((OiQ#}OMkU&OSUw{pM*$wR7icq zO1r#Eyv$3z+)KXfOTYX}zzj^m98AJ&%D5~{#7s=ZTujDnOvijo$c#+MoJ`8p%fqZp z%*;&9+)U2wOwasG&h+PkHzTXgCJ|Jx~NqPz7C32Bl2)w3>5(hHq$43V*Fo3%yVb%}}Cz(C{=+4gF9M z4N(ytQ4+mDE$avHe2J~_&=PG?7kyC}jnVNuhZAj~tRPSc1xp#-Q6BA4AN^6+e1~%& z&>F>xtccL9NKz$TQYLLuCw)>VjZ!I{QYx)dE4@-I%~CDhQZDUMFa1(54O1~4Q!*`6 zGd)u@O@C80T~q%yZBsXWQ#g%NIh|8Fty4R_Q#{R6J>64iI8hWeQhgj!J{?p-EmT82 zR76cwMO{=zZB$2nR7j0fNu5+mtyD|B)F%A~u=LYEr8s^3R8S36m>5-3Emc!JRa8w? zRb5q9ZB^B?OVV7Tfhxm z!5v(}EnLGrT*OUW#a&#+ZCuBFT*!@F$(>xvtz65!T+Gc}&D~ti?Of0OT+j_&(H&jV zEnU++UDQoo)m>fIZC%%WUD%CX*_|R?+O1vNyf=6EiMKge`G=~WJ5k=L{4NyUSvjYWJi8vNRDJlo@7d{WJ|u}=!KAB z5FE+5g=csNOb%sHX5M#@98);uQx=79=-P8IWmLX}rhNufMrBoYSXrKB!C7U_{RT?t z0S5SGU=C&&5QIJ*Wn(_(-xY*l4(2lG+F@8`28aM%23lo?W>Ik1e`;RlYu4OhFal`) zW?+T^Q$S{NF6Y_}0dST8P~KW&fM#iC+H4NySHR|Po@dNm32@HmaDE1J?q`2a-Dl3{ zZPwa$UgmdR+IR+Ld-l_W{^fJb3D=kcUQ)XmF+ku)b@&uG^c|XIQrBvqtE(f7Y_8mfU*K0C3iTnkJuj z@a1o&>AZey$X;7FfaV;?0ccJFvo>hBCh27c>W~`j$lYlMNP%@mv911QD2Qy;UhS}b zhYPUgss`!Jp4z>R=Gj*3lICoAmR#5VX8#GGQ9re8cjjl-E^gztT2pZ6A*JhJw&~XZWNOTXb2mxdIhk0+Ir}2uQCPjE^Vn5YJ(mIhdqbahUTI^ zTL|%PQ&=sjeTPk!9Ah|c{?6kdfMz0?ivM2b6u1YKJqJNB>~BtjzV3(WzU}M|Y@e2J zjX4KO5bkdd0b;IMd0>cq4gohHZm30PXm*2|jcwV6f09>lgFI;QJaB{k4k>7GaTgZ^ z1Lv()e)V3b%@PD1)KSCNEd%9EkD%roDy6PHaD|hxcA)B&Ujo?rQRBYn+Dgsd$Aq z*Jm>4eCh2KaM8$MXs2^a$5>C{b@}m-Hxs zhC`=z%x2nU59TGeW#0zwSzl%o4=GzOA8S`>K_DMq|K;Dl_hmkJU{3Uq@@8mHc7ab~ zJ6GtcD0XYsGE=woV83=QU*-lsQU7l@c#WBOYHxOp5qD}2=2WlQZy0xAcJ!s?_g}8^ ze|8V%cju#cw~Bcm>QSc(eV2fZ&gT?RaFAwmD1r242KRyQc_5zoU?%IT=xs#Lc5sFP zH=u+@D1%Vfbf|{rRYEeUpL(iCvSIjjPmhvZKd8z<f3SIO7jTfK=oEN$I1ed=U*@&0jsNnoXMiMN^K-ELn~xHU|K%JI@f0h42GD@AzV2_P z`=@tztC;&*clxPld#Gpk+i!UP22f`x(f7vY0yijw9LNJ`_~uyOb!bk3)=zxrXW!u| zFL@RJ=8o4G(Ff-0w~AqCdWQ$>%};5dj{b_Sb8wFDq38YVuZm~j{=?sP*=O3V_x#zX zb$GY^e!zO1j`*u+`LG9weggXiVPNneLJhYDGAxLYL4;=vCw`Ogkb$6y8aHz6lfW$* zf6}aJ^Cr%mI(PEy>GLPhph73+xTv9?#&;JkN51!z4+SW--Ce}CTZ?B>cWOrgFM~{wf61B z7m2TK@BTgf`10q|uW$c8{`{rgST@E{_(eu^)SE$cfpm^V4{gWWZw&&+5Pug9e-qMq z2*FSuG!a}h8AuT#bU}F}HS%Ce&uCa%hy-5OQFaC`*bjmVVg?dp2SK5pWxH_%-9!Fq z^v#I~8K@tTMH+b|l1VDLB$G`#IaEDMp~z7^8sx}PjRr~Zj!0ZwsS<$>{#O4OLq$y) z5|0N-pqxOjST^HF(A+W%ox|KBf6+506uhKbq}q`QHxhGMB72^ffSjVYzma6 zL4$5oCbfhhT$EwvLu3k6;X$m1B%ISVHLu z1gwpvntLv~>8iUfyY0H$Pj`vYIuSI}g*dFM2T?Lop)b}7o=GD{#}K7_f<&xAWJ2nk zv#zmpYPEs#Wvy=AJ`_X_8Ed?;#?0+$@j=siB=AGZ4wT`?@2b2q%PqV7|1!)mrv#*H zdAM;i&N*9U@<0ePAnZp7fBS1vm_UaBYCx60RP;ps&fAgCrVIqBr8kj`&{SDDEb&26 z6a3|)VvY&2K~;l9^Fv|_glV`ktGzbcZM*$8-0_9!HPx*SOkuu;f=QxCNN>&UNaRNP zbi_PKd#_25GB6~yf>Jyy*mU1bP~jlOi_n&3ZWuS`oqPT{=%I_w5x9~w#eXG3K4&tL z&`Xo#_v?HsO%l|Em+h0!5vi0ro&#k|71xxTZk4dLiv)47CkN?b@1r~aJoM2^KRuGj z4v*04sptJO>v<1d66_;|-%p`yc0ZTx6Z<5%L1TgR4736vk2&hTS59c~ieJzFIHXRC zFyH}>L6hs7>@35xplwfK_tQz3j&h)- zm`!*Offd#UN5P7v?;t^d#xr6l!x_@BhV?VyL5$`RRcVP$?AsRugDAuy60wK}^w|R; z0;hgdX>%e$l#5&^KE54|Z$4vOoR*ic78(SFL4(FO!uSnn+^{6|0)LoC>ftd41i>lW zLf=FP6hQ0*&j=q(38Zw$qNvd2j4U)FAOk7LK@yT(j5-Jm+)#!^GP044bc2nHq51mK)O5F_P!UpO@d^6{lH`eKTHE`vM`l=(;yszO;j$BCiTVfA_W{!@T@1ziPXaw_PnR2aJWU1EVGB~(_=9MD$s!v zw4gM3BQr@-5fWKXDj^wQ6s<p8GsvQAKqvc_iT|Lb#A1HNn!LKK&_Bg9_AAvU4Pg^P(pKvq6oZER#f| zVl}ClJ0CG^dkom%m`QA6E}wS)dr16rSrWE# zrG!GgD_--e*MA{KDJn}{lv#8{R5g7yb&+%`_@GKrsPa%tg&jyej`>b+nx%OYIix{U zX%f0hMqI-hDOXmSw5Z_nDTl3PKN<=H&oX44N?c_QO%m3U?X|YHy)AA*m(hoa_J3B5 zDcKbCS7Ve_Zx*!aG?B}YrYb0z{eZ?Al{*m70>-%z6@M#Bkm#a36qZteq3T=9hB)Y2 zWoX9?!DHXyS~hA#t|UpSRWzI1`O>$(_AOOb4|2|ubk&q0YHXrJ0gZEf;}xzfuM{yj zQQRnYun5V4fW!C-&LX6=8fmCOAS^~vY!i{QGzm0zYLyERAy+*x>HkBpsFD00QU1qy2Vn-dPjLOKKqrk^Hi{OIT`DTQ1~}tE{^2l_Y!&3dUz24D06+IftO}; zA`bq|eqp;$lVJ8BDke3wqb==eGo+y?jh0oim%3h)G)u3RHK2q4JnM;M0>xIz!-qBU zihnP21qMup$tybSn+u?r(aU8u0ngDokF4847jP~Yfou{as+!scKRCkQd|eBt#MB1K zbG2Fg5H8FpzXdSBL- z65R0OpE&%gNk2T|6R-HaQr*}UPWO{2)!W5WzVep8d_XZ}oBvXUX~>xWJm^C&dVd7o z;?|_0C99=K_+0(xENFK&Fka7+l^WFBo|2^=7PfYJtQuxL{KJt?v zMB*c4=E{FQ^rJ6*m@(fVv7bKnv#fqP`iUJhY=bz9T%wqdd+dJ=UW= z-v1*$=A%CDBR}?|KmH>?(k|mMo`Wr#0&Bg47JfrsKwgsM#W$G3EA-SkPGko1VjRAM zJk&!y+yh5;q(^=vm$AqJ7k}hPYNR~8!xawWMm(f3FeH=Y1uu>qMgAlN%7ZD~g1*Uv zPEy`I%;Y!7;v6!>J@h0;_QOI-B~?zPRaPZdUZqxMC0A~xS9T>>e&tlw15O?#P@W|M zR-!RrB<;;4FXmG{WT8=pC0&YTUEZZ#<|SToB|^ReBcdf>79cM|!+%`PUQ$AXO)`W~ z8st4}q+>QFWIm>3MkZxWre#(pW?rUdW+rEDrevn0DK!6OV2&mOW+5Ni9{xe(L@Z`! zwx(ygCTzZ@Y{q71YN2S7CT<>}7QVw@eqKEwC@CvqmIaxN!xHm7qw zr$*YQS>`5nuAg+;o_}x#=VC%9cXp?DekXW>W?@z*dH&yZ)?Rix<#vWAd$y;0Lg#sw zr+m_1d>ZCrs^@TKq<+@Je)cDS{-=KiD1Z*AfEFl$9;ksPD1riLea@$Ys$YY;UV08h zb_S${UMPm%V}w2^hl1aRvfhLWM1^K3iI(VverSiL==7ne>VJi(R4%58#;A<$Ba5o2 zjV51>ik^#BC`R_^O#Y~k1}TsZsgM>akshg$CMl9G|EZD|X^!3~lmg$Ag5HiUrZ!$F zmS!n5PHB{O>FjYS=2hu#YAKnPDVBn%m!_%ep{eDGsVSH#oW?0NvZsh=(*o`2e@pekOVk{UFGfUfQ7eNw2E{;8w-X`v3Pq#j+QGF2XMmq>hr8xTPf zaKSq~!=-{FXh1_A2m+gsY96GVrx-y9m>x**X_gwprOqm?UTP+g(Vw2frJ8D%R_dhw zs=@hcwS_uWq~v5HE`5r6CTDXJFY=bL8fZgE8(tfQQU zSb_9vXig`vo@>zw>p)1W)-VJVAS(=nfU**+s*eALF+i(8+-nH5E0df<3Lt9;#Gi;R z>cH-%z8x%0CM?1(tisY{jXgvXXkx=MEX7W&#S)m5h(g6~?8T}px_<1OdF*rjYqHf{ zyoP{(Ab-WOqFid|>ub1dsFVS~h5!tRYQbvfEV|+*0!Ru#gU|jf*zJ_Y>TD?P88{KG zFpek4F74ro?4A8<4wT2r)~jg1EcxK;r`*EK5^Kq7gqtF*juDV08Ld=+?JG*jHxjMW zGA-NM7n?%EFucMr#GyvCBSUE9$-KhU`D+N|0Dp1D61~R(&>n*$ktG4Ra_U%1PZR*JE%tFC##GBjE z8h^2tP!9-iNqpSQHUhgw?!M;1)^=@?c<=FsfER7V*CH*?g5o;}j2md8o(wG~dPxMt zp}4}pFccaQ3|Q%+Ez;KR?f$P`ajEzo@Ao}!v=VD3T<+xp?1d<=KtR{{hJdmhDzQ4s zJM3=MZq&b~z-@vA7ep}D>T3kIQIb$_1b-(gqh@Og&#um z_JCdl?xi3r@LJfG2}jO-=@a^qaCQ6ujd*B%5kA6G{WWh zttJX`>=rWYlI@{&OC#`bkNvRHR(~=yBiSW`F+*VQCWkPx4$1}hZ78#BKYVcrlx(%$ zTJX}WQzUU23q-u8fJAOYCJ-wJN6_xF|1q%^7C3tzspxVG|ArB0)*2f&ldn4yYf6ubn~D)l*R)OF^iuu?7vNtQ39?Na^B66~S;Uwa0Z19Bu2$46 zPA@f1`>#Vkbs>Q%G*AK;fHS~aG(Cf|Jae<}1nxILEk_iu|E%o3l=CZ8D`r${vu=bG z*X#Bg@UbFqy390F*ELNKhJOaMu3n1*B{0TO_jG~$bdLpfLK7HG1Be@lkyfODT{rea zPBK(Sc5ZR0EFUjcbFZXmbT>N<1b_e9B#E-V-bl4}<0`+atA<1zdvnsXG(MNJXVbAA z*D4-!F3y^QW83v$z`%}Z$uj`<;s|y`5Vlg^b);+<132$(GqqDoHh*&u(Ui(^5H!Lq zV1_kgwsmjyMte0_Z-iJ+SF6gy2S@ETPsP7p>qm%hy=v`6uQb6BYX%#M36Jk=({*y= z^kA^_arOhv%&%|1SWN@9qlH;b4@P2#k#MwceK$2^H#dXd)s*h8wh0huw{&K2_p$_T zM;F!=*YiM#?=8gZYk$0J4ygB9r&Kym1Q)NjOP`gsCdmMAZTV`h3g2vk+jnN+v>FBW zH@t66J6doDxJ?syQA>cx)woSB|968YxdA;mS2ILgU$@o1a+3qDH-odJjI*QYYib(^ z_7VZH`lP8KYd&=}J%@mgRc{F3vwkcvz-I15(6^C`(S76eIe$2We+xNHyTy+$1!E6* zafcC?NOYXjG?FX1p+nD!1@i1N1CT^LS1tG#G^uaWM~J!@8>%_%rAVjAAc{oA1|mYNyS_5t8Q*~0xim? zyvhTuysx~T^eh>fSw#3OpQpS*|7^ytyw0zDp(ni0zYUkJ?C!d;Je({M^g}t*v!zsT zq;FL{bbPUjSx0YGwH|#+m;#%-fqK(3va`9GA4#=_fO(s*$uH*4pFPSOgNDF>&eMi~ zDTiE0iGRBp$Dx5yQLsJA560Qie1V|7-}n5_2fo^XX+6L$E<%<)*=$Sv>@)P{OmrhA zGU63tVyCd`X?uj5`n})R|IZky{B3l8UKBn-(n|h0O6UJ9>fD3RI*L4K{@DY*;K#np zz&>c)LL*QD?&rSlQ$iz@c)J|xY`=NgYoY6(J%8t~eEqz`&w3Vb3Pk?x#_9Vk>eR#P z4@B!1KhH0?>~DVp(f((!`S{A0%J}}OuQ2wXzu)H=O`<>hYrgk$Km4c5`*+4GWP<(Q zKmOl;@5N2#0|YdI0|^#1co1Pig$o%rbZE+yynFp9RxlaKY<4rfAsmYCq#)ri54|_l;b^l5G^W&=n!gDsZ*&Q6gm-UR;^pPcJ=xd|7=*X zW672^dlqe4j91mRb*s~bPQD%BxvZf3<8Hc5K5>Xu^pPH(nh1aplRGFL&M?`g7^glke_*9eZ}|+qrl5?$Y{q@xS{8e|8>ydiCqs zw|D;@eth}!!;80nAAf%R`}v=(-=8<~%$twE0u4M6LG}IvkiiBWd=SD2`&&>#*aW02 z!3;Ire=tD`EBp||5Jeo3#J4;wQL_sn;*iA_T}4o_9=nKryfdbA)-)5(w(c48f1E0fd1lZ;2|d)%L=jz7(MB14)X_*G zomA39#~iKCOf}t<(@ry5>XvsrP3xUoBARO$Q-(2y)mB}771mf~ot4&FZN2r@REHUA z)Lwo471$fcWHVT?hQ&ypIB&s~*=C)67TQ-$eMX*Qt-Ti8Z1JlHqHna_N=;MhK{nh~ zf1^DY-E^5vmCS0teHY$%<@HR@HOHN|r+O|86r+0HiBnZ|1s+&ncAt6Y|6Y6*UYOyA z&8qj7XU<93;g{+;=NVH?;TI#N-eLC^a|s@qWLk&Oc$jy1t(fJOU4GdneIc5e=9+E3 z8RwjJ-kIl~ef}BfpoJcq=%Rr(hT?wTe>vxyC5?U>>Zoge<(qR5h8gRuwcc7JdCoD$ zX{p5?o9wd9J{#>?&OQGe^w32go%GU8Kb>@W>d`y%)?I%cS@vu< zmxuP+ZNDA&+;!ib_uhT~9rx8^AD;N)jqmiHX;?!W-Vld5)Zvq%n;2o3#8(XC8Qw}sshU_tD_#+cS=8bdx!6T7ei4je6yq4l zSVl7{F;pi3n8-@j##@;Kj&Xz|0O$w+Io=VEdDP<``PfH4{t=LY6yzWYSx7@35|N2i zXG%D?QfHj;565ENNLwTiz0vxzyz@dD%-} z{t}qM6y`9ASeScG^trlYhDwZ+0^DXx!Fx`eiNLgEQzU{ zSx#9&laAq3=Q`QhPItZ&p7E6DJn30ad&ZF@YGR`~`H4;dG(!~u1%E~}WC>7#UJ@I= z@Wm_*stSTyQxAGrXd|&9g$wB6o*6ZyGJv5_K4PPzz)%JsIqFf3o)o1iRq0Au3QCg1 ziJ!3&L=Iq(2RHh206Ep^3*-ol7XSgMK?TAKrXWWuyx<9a+{FJ+g$mRc3<8h90D?|+ z3e`a{wW&>g=paQQ0e=Zl;*i+LYEWOG3OFuRs6UMZF!DGEst(`$cg+Cbzn5Tzxp3kWMm(W*88r!DMkX-nYv2xdVs;9^;TxvWV zYF*=-Q!dxFyC1!P1SBef3n)VmW)?7cbsA+PKS8byK7aC5bvoru=yVMrg@Bjg7+)td z*}jD2tA8ztjsE`Ck$RAVC@?u_L|^m3Q_iC*kgQ&!BAKn~?XITHF^DAV^a)A6?3bzf zmHjN-JF_rkJ#Ar0D$mEX&*ApUKRJa3mnMu!ZpD;oibi?I_(L| zH;(5EfL-*&B#$$XBzH`rq73p5R z0MpHlZ#||x-ku+L>CGW;cIOlV7=JhD1zGlN>j4HK2muVFAcbR{=ATuHT;vYYbz9{}ek50!FI6scbUw?k{@96tIW+Di}Pl5E;Tzx5E0~>l^03D@3 z;y7j^1$AUT_RXOO4G8xA3Mt_J2==&+s_JbD+RnLhs^QEb?g&X40)epXuKfItOxqN&2^;VS4T%DK zYUAt;wm@vv>TT6PaPClU?mlbVDRR4?2d}-5~J<(^e6{kZwDuB z3Qh0fz)c9rZIGt!2;rz1w(X(bPVJ!Z3H3_5|Mtx5_9(FKa17xP02NID_=2x$QQr~v1Rs_yIR-fiI+u=9>_kwP(! z<}Ss|!2&I=@K%kmEDj4y4u2R}GLBx7vfp3%5P+=6w$2=+5h~B?xqx8X1Z>Zq zV5*)nk9;8nQs5!e5C&2}FE7CP0ucr?#(5=BqC>MEz$ z>IsfZyGnBCZ15@%NhL{(*^uPCzr z(vYwc5&#&j3>|a+K9rB{atrB*|BO-e|8(#YWAnP$Yd!}KH-9~GH$m~_9*nd=%ojH9 zI9rVbN9yqQE;(^>M{l%8sS`<)R7qjcI(Z8_osGEas2OTgr>b%(k1Wa7V6(&%>bwj~ zmCQVc%#S{ikhXnlz@KH0Po;oo;parm{wz(h~o%jTm_kRnN0Q-G7sin2|--%0BP&EM?UIsqq*S z?ila1LKPK`AW;$J$WP&j66i1h`7Tri3qk*L9z_!!36VqNXkKAS0O~bC_f=2nXbmsb zUp3Sm9uhLQ@l@pqLj%%aPjW?tu|=Va&I0W=V^v)7h*k&5R_Cq`X!1{YwKy%1ut@MG zNvmXmwSUA$5m{%JW@~n~l$D>Db@QAxmf&b(A1Y+WRW3oY7h{iHPcl6}$p~5Wu{M$0 z7H|NR(j4fF-HIw)p-^b`HDU)%Lg7^$B~}U;Qm{ZY9bqX&^UYwlv0pb-LzN+H1(s|> zG-4|f00_YT6qar6&rczikG9b=|Me1Z?ec7OvwvvEGgUDWxvp+LG4W&3@>-MDM(^%c zJ#5X2)D&UM<7BcGORI8W)@DB!bVFC3aMql3b_RL&j!4qsAeUO3QIFmYs?d{^mKI~t zOL5sWr(&1ypz*nis>pILYteRV`>SjJG#o$mjws6G zQh(RcZtn;mJ(FSg*84uHuZkCuT+?+y61n6pY8iJ%k(QBc^*Bo_@tBkDNKj_um)Brx zeo3$;Mc04-7l5;gbj!(fj}mp|NOc`>S^;fn8TVq#wSmo(bvcQ44HxJ(cFhjV!OWAV zdKZqiZR^zaquABo-c>cPuy6k%Q6SUtuYWvrZH0F(@9U$Zw2!EX<}S8u=8 z-vFQ;;|L!SHIdAhY1Qt&sMNta7HVl!TM@}tgEatn5{`0pbL-9GC@vMLI9R1NfWH`w z!+4hl*qLfB7on8LqKWSkIB1`1gE7`x^GM!kZ&nw0lPb7Nfll7K^W(Nn(&7uMJb(CB zH{r+LG=c}UzaoGL&B0L1O&$TDYcl~s6Os(eZVyS+kfc|Jewx0UMM-6oO!d1Boi?u1I7zkN$2LY=eH>^6rt2^N(zZ#icadMftnFGKAhSlUQP*-8L6+Nz+<65rg znv$-$WRz{$^7;bA`CR{S3V#C7wcft-t(vO|mkW2{^Ny(0zS<0!pIB*&(VwAqgE#iA zu$-(i6t$bx=o-AkTf7m;u1O{j@;a|mfUlzkDR+tq1v@0=h_c^m$^_cDl2+)biazZv zr}$Wuo)O*lXy8Oo3awSK>nx6@YQGaazzK;GR{J6^lpyOCtU7a|#E)KGYyQFx8&o(1 z;x7wBT*R3U8%DH_|9{AK8{>9E5f#G)c5WYBV9N~~7*Vh=Y8wlz4T<;+0bsWO^pfi+ z1W3#mVjBP$VCe=4zSjL3s#(6Bx|5Poe^1Hscl z{mfHc)zN$;kWI~*Yqg~pk-Pl?k=9nFgcTtV0o!h(J+rPcrmmS-i#o58#+|M1|(|x?f zecWAX*wr20<6Yk8y=K|H-EF1Z>0RIVo!|R?1z)Z;0-h>Zh~Nv}R|JA(AVc66z99_W z;UB(F5WZp*p5ZGVAs`;(GagJN-eD--;y->MG+yLK-hW6qzF|Bbb0>aRoT z=Y{E;o-(2y>$83~re0pE-s&Yo>%(5`J%j7x#p}O5FUDT&*M2d|9$wBK?eBu^=br8X z!|mS%?tkOHD*x)9@B2P2?*3cyUhk*E?+f4X!2&JfhG8b|M-ia`G41b`F$n%mB0C?KlGo!SE66|sbBjq z-}*-d`*VN$!~gKR-&4LH_QfCl`=0ze#r#h{{ojA?*Pm0`-}B+${>NVaH--Ky|Na4@ zpTL0x3mQC#FrmVQ3>!Lp2r;6>i4-eZyofQQ#*G|1di)47q{xvZOPYiT4cf_-EL*x9 zXn)F7%9%9(YudbtGpEj-JZrxE2{fqCp+t)sJ&H7`(xptBI(-T?s#J|Br&_(LFlNuK zT)TR`i8ZX)v1H4dJ&QK2+O=%kx*b{dt=yGbzuLWvcjnx^eEa(S3plXg!GsH&jcYjZ z!n%1IJKm`{vgFB>D_g#dIkRSzDmNQu{C_yK=+2%?n?8*?wd&QZTMtC+S}y3(wDHEi zjXSsQ-MoAIrb_y^+1a&=cNLC2x$@=An>$Af`?*x&#;Ze?jy=2f?cBS2e;xdLQtH;r z+Z2yJz54a++xv~~9w>S9^ux!$k3YZu{row??_bh<^_2%;fd(FkV1nQMr(j0{4u4o( zgAz_iVgH2|V#eTw6+H-HaT8ibi9O zI_}7$hBNMHo{1(~V8$rsm|mp`XPk1*nSa!pFt+I?S9I>lXPY<{NPD<&SjJ7z8Dc?X^Tcw_U3aXKpviQuSoQ@W1s;aI^;;1KnW6Ch6qRMKm zw%&T6t0&%K$}_WuhHJ3G4!fSNB)&7~uQUd3#?y=VWS%Mepz$Q*u>e9Jo+ee=vP zj-Cv2%rehRbIms2jC0O9@62=0KK~4K&_WMQG{Ub8^UOEuF3oh)PCpHG)KX7Pb=6j1 zjh7Sa0bGBgk4}2&rW+{FImWzP^#8uE7<10Fr_X*XG%mM}d+sG4Z69Iv7^8c@#?(W5 z@uAiOZ}7@5-|;YtofFe7#z6DUIo4l~efHXK&wcmae-A$Qn;Jv!ImVw)X{N6v9e(@n zzYl->^3P9y{r2CFfBxOG>}X=pn*zJ*y<>m`ETDe@4~W17DsX`eY+wSf_YL5g!hGnf zprN2)idR(c91iTD2R{hH5Q=bwBrKr`Pl&=4D$si0c!go=5tCV+1hVhMj?4uw52*^MRa*%{9 zq#+N9$TikOjuae>JWO#5J>DZQ_mJZ^cqO|#YO+i*0i)OCSQzLT!;^%OBk&rz$y92o zlSSlOJyb>*;6XzklO#+fF=z}`>T*gkp{0MYz79OgQ2HA|0yDVW(}VEl;<`dnNENz^g;r~8bJZmP{8 zwTw2T5g#q8Q6nGQR%*lpKImX&ia# z)P%YaN*MJ5IS{K+joA-leAEq2UFm;X=<1NDl4Pk2Xlhk2z74nV?uywG61t?`ddfCiw_M@F$N@F8H){TrdYLZ2WDToV8r51nn76P$t zUB&4T*oI)c+YRnOjKSUi@P?oVEICICLXd*q8APumv2JhQ%aG?XKmySn2wxdMU8Nj1 zxf)q6)S7z`LCh5)N%$xwsypD4-r{xjEy%lG@w#COB#Z_TqlEhr#tRF?zoy_W42Zzp z6g0BK;@xfzcpDNoa2E;#TJd7D5Z@`(Pus}+c%X>8VH8GxX zW}mU(9K+Xw0tPLC4SbMiKzNiib(!>xEXLQoBELjFnvwV0k9Vx@7QL%+d?7nfpE9J$ zK|ly4&}+=kq7<-3&4WRn(b%K9p_Oh$ZCq~}oLhV}2hK$^Lh1p7?H+%)ALjsZ2<$8n zL4+U&H7uZ-Lvjo8g8x+^E+$B(y-I}S=GL;%fPq(&AiF$x^`o*?YXrm`0;bjju!Qz> ztc@D#f~eZ77ufWYX~$+a!?_@hsBBT)pjBg}VAUZ&?K3`_vOH&1Aep%8T7BJE2|%L^ zUJXHT3@g|(V>h1?5k-Gd^MGQ4e0J4E6$C+E!S0W0fP!53D5M2qSO)ld*NZmDa={w{ zzY63X=dE==j1cgk^gz5YaBV?49qD*$d*F1Acdbpm>ZS@<0-`Xs8%7=Oh_id*0x?3o z8IXxz7th@GPIo~}&TbgA65jESKxg=xiCYJDlH88>z4I*bMmB#L1}@*Qp|1vDgZ%cU zWzb+k*3EzmPy%@Wbf)i*iyDDE0I$Hnog~ePK9B+fR^a_uw#^GYZiN7x%O#&QjT7B( zm*kNrdpAbk zQ$RdHf0PTi7zTeOsy<|ui*D&4ZMu;O;)Y`n;}(hUD~;(abz=-8_4Y=vS93tue8WH# z#(-^6QwnqjaNZcSXMk))KJCRE9ix|C2+euTg$bs%s5xlmsL$RHC=mDCiVZVhu|Mmg zV&29c1^Yq%z1V1i{a{!`KoEe3S)zA3&$m|5M^eymQ*0++XZKm-HxMj` zS5UWSulG?bMi4!~R0CL43Dpl>#RUWL3U1&A5ttRr7gWu+5YV7~@x}$w^;y_A5W5yH zOUHHsk$8W#HiTgHY*^PBEXRX2;)4}d5ZZNvaYc8N=5Qa82Sz}KWmtxG_i(i3Z-cgm zddGk$R}h*PTJ9xs3_ydD*AMb#a4_*vM$iwL09?aF2~fZVVW@$iBZMbcgabi)2PhD< zCtGxAdntHq1i^9=pii1tYkAg%1Mv-46><^bUr&FNbO|93!;l9UXH+Ctf?CyNF4T$! zQG&!HUtOsGRuTY+`X&(ChlKu?g$j{r+eKB4z+FU;2T)LqP0?vhHd5c*6JKxmeLm3-?(XRsKPWBHIyW)NrD4=mRRAZUj*I95cZmOa>!3!!2r z=99Q}UVTXvowjRvAPSk_oX;r=%_&DUnQRbwjUVNYc(|O+`JB!fotc15(2$gm$pDTx znVi#-D8`o9*$-kB0gb?&&>5Zal5c+(APBu8W=t8IJ$I5)=~#)mkq?2DO{0|z7y*XZ zkA-Rf5DqzuclJ>xcn()(1b|3_A0-L}2A9AonFHZyArN;1VLcK?a3NrUBr%3&Xokqy z56ftwW(iBC`CZm|nCuCllhkCswv{hBK&f?W0eYUAbA1TPp2yWYJKCNxsBVAyNt*-F zWMnsR=S2jQ1ZFil5d+FJ1zHae=%icunhIJF4!LwuIfQ%ISQlVw;Wn68rJ{L=n*C4) zCgyM>dWGL`oF!3zUr3efrG^E;R{^S(oCQnz3beJ5M7&G z>$atF>ktU{g$WmCiAHz5IugOiT}^m*dukB7#%_z2vk;k?PWFGZ3>b=(yK4OgwyQR8 zP3t*!tC@}ngY5R3v#FWL<*@zGdaK4|?pd0MTBvY~g#!D76+sE)SyVwlm7hqc5x2Sm z@eM;rXAvL>yk%Gb8GiUBZX1e>hZ|b*Fr3_#1nMOc*UONcCbyIz%bbW6IDGry|_ya&+;2)TKvWC~YRSqEDWJtvK?`~Qc& zo4u%Kj}Zx2Nx-CoMweE)GzD6>nIZ}drV;WWLrF6d6dbeA;49=v67E;53;}Tjp*qt; z!sVq}&_KdN6hWtVzn$|97_7kx5kaa&j?@Di&tMERd=P)+0y`%G!s%kcYH`F(0bA(n zz0O+0Q5+jfd@f0x7F5g>%$k4SE5%)m8d$t7R*V*4Ocp8Z#b=BeWSlKxd=_df7G{jb zaXcAr>?~|N9Buj*a4g4r{26$hmG;{fbc_{x%*TaXP`XPsc3c*Md=-Rj$dAkyj7%(w zOcs(n6^?%l$(igHmMkokJQkcB6_~8aqkP4FOrU_w7N86jqD;!IT*;<lQ+mLAXrJfegzPEX!&EJl_D!ZE+8?%VF_IDU%rU5>DT&&S8I{%e6AjVc|uI5zp&P&r{*c+FZdc z%oi}EDTM(mrl4WO%xb(vR?BJq8S+fa|<%W z&}rc=NcPYYEz(P&KHt#MP*V>r)SrXl(9FXx!!yz`jTFe^(lu?<6T`TLkvgkm)2|ad zG7S#YNMSpybJIm_)B(dgL6>3q0UQDUmwovGIu3(P*oAG_hmF{Y9T=>|m&N%3K7Vc7 zw~gDmt=qfJ+mbC2y$#&KE!@LR+{JC&$Bo>{t=!Aa+|BLW&kfztE#1>i-I)E`(JVQt zg&o>0Dcp@H-hC+GogU)t7v!DY*sUDq{U_?pC+xi^?#&(W{TA}gIP~2c_AMv)ohJH? zCj9Lj{*4v@UN{0C8w6heCI-$W2!FmM3N9QB4$tTfIS!s05#>gt}cMy;Z9B` zg1#Pxt{#YP7K*+QjIJ$?z7&uy9+KW2lpf!No*kB+T9~dlnr;-Hz8#>x-JHJPr2ZVD z9#^J*IH)cZtS%j{-v1o1&VL-T&J?Il>#K6>Es^W#t?Fj7>kZ!PvmWe=KJ0_@>$O$v zUUKZdvF!Jq?8ss3&%QSM01xo64}UK0s?qGsp6yby?Yr^qt={cd5$@eR?x#`i8-DH? zzV1kp?mrRjhjZ-@@$Mc`@3(R9t?-tM^(?}B3w?a&VPfbFqCMt|3BCI?^eU83+1 zvG4QL%nv^i4Nvd=4iyq_Bn00N?LZ5(&<+AI?+k(QG!~)@-zDy{k0!4rD1XwjXz?1t z@+$uuG4Jp%?-V0XBm@r+z%U4d&2 z_FMXWG5f1k`wp@De9`+atoyMq7{fn3#n1P9Gw=K04XO|af-m@SQ20Y{{2sl+t0hAl z8vOg5{hiPK*KZ*1!r0pn{Ni6AZ~eld{@Q;I;lKPxAqs>D1wzaa_0I)RV2A+m zED0AVl=>Y+n1BCJ;X;ND9X^B@QQ}036)j%Gm~kPzgX(Av3PVLCl`w;VsXJ)zp)q3` z844t5&?QDuPzauL_>RlYGWT5GI|t3!GnOo&z4IB7A0d{E$dEY^7n?jBh8AlbigB%^g zZ7Mo*>BpTJe9n0xbBeAQ>j`Dc^4T|efK!{U8`wAS=LdC@t{DC7@#M;xJ2y+&@>833 zXI|$$F@Mia|0p=CKjz+jdGp5a8KQTIIrA%u;)~l)ExzRD1J5b^^zzTT(b6+bK(+|1 zFT9;9`wyZ7&BARm;TRz9veEd~T!U%9IaH5{uMmO~T;hFsbXxK#fiz3k-`-F5leJN2fCE zFHsNo(@2mq%5X!`e)NJB}@^Mlu}C#wKSr4zNvDp z?|+Eg^vM4Y5o)zk`&O06%|dB4)0kVa4A0RncU2Y4iGuKK#9W)2wN|2zO;bf++hkTE zSf|CS+G4T2)2eOJ`nDfvD&yi7)Wns~So)9^D_un6#1KzeCBQeKpji5*9)|uk22Ek5 zddH|sodvgHhbyA79aTUIb)-R5X(y#}U4Kfrt%Y+;=pE#aY6v+yU8+Z`d+Jg4qI%8= zvd&m9)t91?e?m^$h2+sn=Zf;^_Mvl%ddEWLGU^zbkDo+HWWN*w8d8Ps$vLB!V-|Ae zR4amcsG>vWx6i4GJrg3KK#Mxbi;`ZNO@aefNKAyN2KeEJuCCQet!tj>=7&0drhhPn z&xMFs|Iiu<`W~{;d?)Nf5t65c(_O2Qs30$&fhC4}IAYlK&3O$lBQiJ-_# zf)VB6BKQ!}W|MO7z2C>47GJD5Mt@*9Qi-9hCBj~N?h(2rh2|aG zB!A08?l96dXgmWN2SlI&y@o4|h(ZIwi$Ga`#~Wx6&s_(}Lj<%p428&}49mMfS@g3% zOnmQkgy-|TOJXVb`U-_B5l7Z3;VJ+n(a9YhRds8KTbG57=I)sgB!dE z0S*6%zym5Ub_~QwBZ!xo5b)=BDCCbk+@QhWg%BaoVgMJQ0fWpC0fj}>q5GUQyyY3s zCvHGtdXC6L4t@_IN@QXavm(2VWyFCHEK3b<7{P?JkB9H;8}Jxl1Qx2TA6n$%XuNpE z`GM#hI~+|A?HCa}=Fy3KRDWOZ9AZc+zK>4~GD2q12u3lIkpX~$BnS;c$>qJ`84Qym z1l=$%{iNU(${S4%CfAToGVzm8yhtv!XGKNsDwbWGWizFANg(=+BJ1PP{5JDJ6fz(P zK7n30dE=#51P@RW0V8ngA-`@iZIkcJRi{1`k|~sL3R4(WsSa`~gnwZ3ZtN82K{C-! z{|UG!obwY7bs&WKE$ER;|(Q($3fr) zfrbEG-ewMR4*V^T3kT7ULZkG*(WIar425YoBXB`DMRIu;tQ<<`$D4t6v!FEX(L01X z8kN3DqV=GtMJtL?jnc~^aO{7kDEO4pcK{QG((4BpmnTiJL1dlne9}jk7X(_GvmZ*$ z>L#5wQK?R~9vS_{MibIeho}`yJ?-WN&)UxJmCUD=e5+UICy1>EZ!^!B;%Fo=LwGJ^ zu28M(IjEXft5#&N0X=2Y93|G^xuL7=WNb^Bx~f#7u%J0T>}b&Ft*3uRRjQlSYd=NF z5UisAP^+rd%xSxsTHp2qvaY3UMfe&z7$`ObDG*BVpwI}h0ta|G5JV#*pfn`i!bz{B z2u^Q!0WwCiu$4vciR3x4IK*KPgJ^Gi-z#5InF>|AUC4OLYoe{fKoFQ@fGOVEQW$7( zLkfKePHzE0S8@@e6On(%H=6hi3Rl>GOFYNCLYWbf@(`seO#%;l``UhlShmF5?m-j@ zgDLVSrvCd7B|`APn&u&_(Rv|+gZJ9TM!=^~P)`^4o1^~zH+ZIyF^z{~V-u?q#mR+8 zgePoa3K#Hz8Sd=8GQ$Ny^k6g^I7f4}2VEo?PY_=15VTYqgc*MaF9kM^h<<-#qVG<( zy8qwcLI@%m0S#=X$tirY9i?1ADpwi98on=w$L!bvL-5BK6!M%w77BgGlFC55^EQN`TZOYR*qvT_%V{6#z(zCly>EW=8ymLiqlQKk?|D0kOI{jc3j?>p z;#rd+I2CISnsW|-mO3qWtp~<4uJM&ch}j3eU{O=(lg59JM!@~Jjs_ZPq6K^Af@uT) z@{64(HDTej(opcpHvnqmty-qH77rQz2u|mkCdEx|O8LqseesoPJmXF8I5Mv(*`&Dg zwW2O7@Ib_;Rc1ywA)o{$Ky$Fa3H>1QHj%gy!lZP%C+P=qXuH9+GN^By3l)27q1ci;uPAm8GX|5U+VCk7TB+*OF&o`bT%)_~xZP5hqEY>ea)^@=c`-P@+Ykvjo#HFJ07N_!kq7y^ z1^R!pKM*;((V##Dga#L!hylDi15~cyfq_O?4vS(ud@%<3t0Q?31qbs9vHGTjAiL1u zLHLtC>a#v$_`e(bjS)eT9JIjEp&TEqJ0P68%t*imY`~*P!4+)0dvHI5fWL@PJA`O5 z{z$@GSi&ZZlXRI5V~~dh6uEO!KQhcbGn{{jIDA3So3zo(j3_)iB|Hcwbi!Ew;}Sh- zId4h`#K{pP$-gPQo4U)rL)^W$5JDofts^X}Jlw-d^h3`8L_r)4LcFHu6RpuPhDI;~ z#p=R-0I*)QRqB5V zd#MO{2ns7hwIw1KR*NFElA3<#Ee^rEKSQC7JD_shD_cAWB`Y2xXew{Kg*@m1ixiu5k_=E} z2!T91XUWA~Y@-puMukkt?*T_W8pnT#T(yZjM|6CJe@qmTY#WJ48z~W%jsNV(<>`=b zDY^LpIze+tW1}*e%)5xtNj(WIRZKHjTnM1_$e~=lv>S~~c*ZS5NJsigDS3#O6i0}J z28q=z6v~TWM(8Um%0sALtcPEeESx~9-)J?4LK1(lTB)!> zKV?)*PO%4gaLmVq%*f=YMB|psFwA}!L6GsQ;Hk#`NeFCQsA#zk-%AK!I?9}&6}qY` zsv?_gyM9u4p|%AwEd zw9kyf&u_Dc0L4u83_Z6T&-OeB1U1U`S_$vdCERDcq$MA|4U_r*kq*6}V zhbzU>EX7jJJ41KUlf+A*JTQeC^ql$W&O7QigGe zgNc$`^^9EI)f^R3(jZks&A)nc*0xyH1d`Kf-PB;^uQ!y+4(&&T5ZB)z*8ofxJwXx^ zElIDW#}>7$XZ56bZOVy=)mg2j9^Kan=+_0M*xg9iJzZDlLmGQs5|{g;Xn_Vd$b&vT zh;`9Q%iO`yxXgdh#K$kgyE&}Vm~EK5R1j!;SrbW{42)FS8@Yr3h*arZh&BBU$~wV} z5R^LcRSONmr8@`eyM?46EzcOXgLww(u}pgu*Vx63VxGtpd{yok|lT%Rsvv5i6B?2PMNpA#{UO+Cr8EjwXWvoPaAa{6<>A8i1L}&9#U> zjewJbh7>FsFm*G58O>pX&Du=32)GM}0u&@kU?G3v%n#&(@53p8;sWdisM1JV2DU8x z(cgLNUm5~n71pc{K8O#thzMJYDnU!!wFsE)$ygiTrgg!@>$KD7oJXo*W<^3IAZ$O(N+!fn{#a=qFJu)#BAd znd$7RX&==LUh5)X^NE6GUWoYVLD}YrKxo<&goSQs)q3b)QfkSvY1P8%VA|=Pmf(wU z$5)8dZsL#q6iR_{lY?S_SAID1Tq=LhDCT44>tET|Y+sZ+PHP3+jnj=nd{zHWfLB#!LnkzkVp+;ZjAn9$4ABDe{C*k6r0|oru8Jput8* z-XQEE+C~(jMT&^9$cA1cwusZt3_^5AQ((!&rs)5O?FVIT>}YE&T0|3>?fHMHZI^`X zDv|7~mWax(DS=^QE;;VREZN9<*x)A5wa&4)Qq0@`mhRjp6TZB;O)--ZTO}R@yP9#_IW+dibaaXasi3z`lPL@#;YE;qCAbFXsoGq8@t0gm?x)5Ct(UF}BU{7LuJt z@Tv7k=m&KOze5h?Xp7dF?1@k&&kzKk$V3P12m7ndM^dW`XsSt6s}QifXHcvhc!jW0 zl1xxCo6=3$c?F@8p?$pP?X8>>2k{?XaXv-}E)PHt0--3kTc@erAb)?*68|GW@)rf~ z%kXo9>ENK&Lfkg*@(1N`2tsiNfU}rNJ_kpLj{klVIPW4kw_G|0@?^X7zwU=3|BWM8 z#WL?%6G!wkPbBSOBQqV|9SS`_Pw*60m#EEwI#Ns@e-Jw#@-rXx6z_*YY6NaVBS9~Y zK7VJ@V0Fw?a3@(;PM3f1dErDCGT6kh^tHhBTZf1|FPKz{PB!B%63sK9xNoS>Ldr?Fkah@7Dq@pwm(5}(AQafq`#OZAa}=gL5Uv0?cYC-$4SD!+?5xGPw~D>zd$#v`y8-;gsQ9{n z4#PkEw~u?dU;M3T{K=0CQGkKSU;7@Yc*4Jk%6}S?2Ysm!eZeRF`gMHBc!t=5=hBCL zzkUzv7VFlxNz@=|%o+UHcM02fPRhxBAy55>@qPZ}{Q-Z{3f&)mBhldaJbtSPejY#R z=f`Hf*?s$6ed^bKzRrG>AyU}_`RWj2GylH+?H}>+|Kzkb|6@k~s#n_eH|F+NF3gX8 z`7eK&?SA;z|76U6faoW1Ai;tL`$41RhKrj<#tuS^C~+c1gYTe0dsmU5v4Mh+Wb}ZR`UB8Act9C8YvtV;FWh-~?Ot)$2;)Q4zZr+($ z`vMLucreGfgsa}WC-U#%#(@(rHtcvZUC4JSW0rqwnQ!K`n~6&9ERnJ3(xy*?M$MMA z>ej16yN*4Uwd~qeY1=M~8};tozJKfgh7J5^@8QO47C){$W%A|DNj86OG;*}DfHf4)HY^nKgEZ#%#K(ZXBv*Pnos1=wDJ z1FC;^AcCbSxL{`uA_(DJ5;|C+S`>PRp@ofUxS?PjdI%VZ^Nl#$e<7lHSc%WAnBs&d zx>#F_F$PB?bTFPaqm4T3*LMCMu}nnMxq3 zr&75pt5>@Es+O@<27-p1ygJ;#u*#RF;x~5TyIC0I~4*k zf6_}c-L%tBLmjo$Q&U}a$&p!|wbom6-L=>MUxOXC*khAjw%KQ+ownL*v)%T~Si2p! z+;h`ix7~N+owwe5^WC@Ke*=E?Qh^g*xZ#H*p19(RGv2u4k3*hxWb#m6x#gE*p1J0m zbKbeivT_T7UYe)HdpU%vUn8*{$;>+4QF z`|oq^zWUR{U%&nLV=q7c+3UZ*|M&Ore*#or00(Hm@D=cY;7cI*=%>I3LNJ1ve`BBo ze@DRzVz77@%peEbm%a{y(0LmSp$0#QKmP}kFoi0--UwHyx)!=HhNFX_3~Lxa6WTC` ztfQe0VaLPhp|FQS9HRGdNJQEF@Q5``q7qlg#3rWjiBJrk6pC6$w_kWl9;rfCO1jFPQsCp?2BX$zflGaXaEZD zAfhO8Z~+b60(Xr_LL>GE$_{St7J|rwAVjfB%o(E*dB_7Lp24|SxWSm5e|yI+&p|q5 zDpPaMC_^kocMehPQV`X|oGy9kOVD-yGY_J0Cg#48iD5Qk7@#vJI@O7}cbFoZulQx> z;3-dfqEVQ{jOO$B*v($zvvR>4W-*VM%z8HSnFrn6G`9)5Yi84-m3s#^c{xvTX7ihv zO9(neiN;}oG^E4er#$Rwf6;(u4jLj2DIwpHhgxnfml;K=JO;W?gOc=}2^}3YIoHt9 zL6nygr5r`?iBWITl%p^8s6bZ=(wtWA8x8TCN}cLag+R5Y{+wy&uv$(^v@?viR2?ak zmj_x#?;RRo09!RcQyEf%t!xzq?sC9@GQ=(uNhrfxdB@aAF3%Vje~>F%BS6=Qa)GdH z<*VlGidW7FVy>ifEa&pT0JnxN6aR{3KnQRsSj8H)bKiggXen{IMr<|*k#?J6jySBe9BQY(FpeS;B@Ev76;B<|qqU$&SvrnzJlsL8n>8b{4j>1@0@9 zH(1~vwsLORtPxh5f4O&DcedZCoNWn9+s*}Hvuv#{HyIECXvEZrzm;oniK<(|68Cb- z)m&s5kloPj7jw+*E9Ra{*yxgXy7XmlOv78)|H@;#itX+)9V}M~j5l*Va4&p0mtOTU z_;R;cY;VzM;rh1suBY8B3_3i;v8Jx9$O|y?M$CX5G(r#re_o+oBfv{{+Ah71eO+Wz zu;bam7=y%nMg$6*gC?)Ha%GVJD-2Q~*F0p-vMS!(Fysnh&uMwcm@@@iVQ^&1i9)U+ zuq#_JOgSby8OjU@@#WqT0=ed3tx-m^n%T?%GjA@AUF4^_SVje?B&0zbub)TvyC!#No5ijX8W< zo6m#xb8{W}SM$mEctVa|m;WOG#M3PoR{0JI0~*c|sOG9= z{SC0{dCT8TxxK%dhm`C49hne>pN;%EwXzka-(a&FD93S+gZ!<64nc>t3}~D)0H(@@ zcAfRCfAs6JchVidHxVqoIjOJm990i>rrk^OaAR5?KQC#}XWitHQ?BG0KKaY#s&C67 z-M^n>JIw1m^O~O-={n~*q2|mX9(v+iHCzD7f5Z7n z-vk}uDq4;p5+W3$pCk6)BDS1}4cFy-LnMCABzmDGMjj^q;3jq=1AHQf$wMfPVj@Ce z!OdGNW=<=5+Z4)T2GU|6I-}*hn#!4)s3G6w8Qcs;AnKSP4;sS>%EJfN;5+Uh3PFPr zDAongSsut)1W>}1k)TTf*WXc2f0xl#=&4!y0GxTbBR59CED_cP>>*KUSz!reO+^5P z_16W+S?>{*8*F6%5SWlLT;w``#WmA4uO=4Hr@!>{>fJ7DLfgz=peWf!3Bz;vR=orI>(U%)|9QipEUH0Pf^bYlr z<1;#=JvwFTJ(e+~Kn@_LG|B^+8P;0ur9d5KVm{(IASPlq*%n@#QeIAanVDZ+PGk)x zH=f1M^#rRF=B8VUBDGXh+sl_LYl=I=3^U@jV*ZC_sw)?b1q zeDS8UHCbUsCjXmlA7hGwV}6cdR;C+LW@TQcB(jx6appS&BiBuom31312IprwV`Rx1 zQ4Q8(ttRO~L1yOWlrdXv5+$S&SQ%6%`MD!-awin-<~Md7azPDn+IEs-QhsMK<|iMHC!_%-cx_pF`WyqSr{`>!2Beu&!lzcsCsoN7=m9AA z?Ou4A;$T9g*;t6ZC~jiW@(2R3U4ST3PIgA4ZKj=0@*_?PDNk-CVf|zw>Zk)j zgM}3qkO}24mXx0%f9XJ~SDSgoi7g}xIw`dg6_v_Whdy2sT-au@;W^M-VX@bk%9Tl8 z>6IQQ>ZN7`n6g!e4&#*qe_Crvz%TVz4-}#A{GoGN z&UP_ddTk-jNnD+sXXa3!T@Kb`zSFL5j;&fJkP?-Z8QD3=s&?_GZWGkvQiGM!WX)trb-4Yfc7ee z0;{g3Xpb^uf3YrFQX1JiP(ndf&Z=#nvJPm3IU{OCXpLs;mc4^r5kVsmK@LCxCD0SO zT8_E4-v6={-DOb%6y!h>2!cCKE2beVgu!d0_G`M*tG)`}z5eRxU}*obY=T}R0>T}# zzJtIL?CPax%3jXOGAuqhEQLa>#8RxqVr+`N0^&KPf4F|E288U$maK-c8pW|I3%aYq z4(5iLnbtx>BT#~ONm#xPQunFXVaY4ySeCrysbY~}+!EHFs+R_kQ%cH}14$hS8UvX+ zA?2v*IQ=c{OpY%fGGSQ;9BeGa+Wwr;aQ%8fiZv}a4yGvDi`=I;F^}^HbU;gq49+p zG@Km&l&O=TUBKfS1DjFeGMU0Fke20QZWBfT?sl$R#Zp4Lmm8F==TKbV7NGC)Ywtb7 zIkc9M-NG%{l?IrCDZrQVGOzR2XIrj}3Zrc$?rRi>PC&`SUktG!u94gC$Do>Cplp)^U zs0nT71+Z@Rq8=7cEAi%m@lpcv*3^{&voOnV0n+a+=dxuv5NQ=$&6bpvnPDEQ7&J6l zCKjrZZ7>7+X64vpZgNgSI#d&`r#0UgJyD-Fd#RPmn>ll^Zq-ubirNOR7CdX>f0j9O zGOAcYG8n+F(xknls{x;tmFGJM;^(B7?Pwq^pJw#pUxoKh}qlqkDE(WOZBr zQWo?WKJ+i~b9cV5K$%cdGU|pZuqa^Hr%2$b{xAGWJ|U{T{g@9re#|&?d-PJ+Ob@zpZ^T? zj;RrWnD5xG&ow_bCc#kTj)fMA#y9X0~rG)Jlu2PmGSADjp0=jN)%{5!#Fx? z3k!GW0NlDBE2NzRd3*JFe}i8f%A24~b&Z;tsePxZ1+-#Q+;(?2i2>Y8^Q)PqBHRg+ zC>UQ~hc)N~c&XNzT}9f6M)Z~mxaD{^zrJpBQz-5=coWZ+h1aivvn&*==rD6AFz!-E zNI2Y4I5d#Be7C48Qcf8fktn>>FEy??a*`;C0+exLG-q&T2h?VFf8C|c_3ptRMnHoL zmUdmE^%j0ZlVb#wkJ_MWPLQ+JkY~=XC9Dpo7dCsiU6VOkhiF-!8gz%I6Ip^;ob#(@*EO6e`OGr8lRtUkCwR%%7YYYyxWDmXi|Jy_fEVs-05&hIDD9wv`Q-}Ju zlQm^)tXHeKe;qwqqj-vY;no|yl$$)P!}vLyCqq}g4G+4)Th86%Q}XS01IC+;54~J* z{pDnWrU5%qVMGbamXVP?%}3R{iEtl>y;}3C+h;M-e@FcaER<6neqksv`1F)&JgkHOb=8EZn5rPdQZJ@LN(i}}6>8l(*q z5%FMfVI=`2Fd#2;n8i=uU|oFWsC>?o*z&tq^J{kn|6pj>}YV~ON%QVa`_k%;m(r>UBZlL z%vi#OqDacSm@%SEk03=J+=!DQ(xe6{!Vr4(e-^_aSHXrA8{In#jN=!X9k5 z8uW><(ZW2pUcJh+u*@xJ@}@S_5Le#Zq(Nh@9DL9XLl{NPqE&m<@c+SYAPYK;n1|zE zjh#1lzG(GeR|cdJj*e&=DPp&9N0yCvc46AJZ4bW1Ef+IgyLk2T^$S=r;mRXj(=5)| ze=%gRoKIh+&2e(XZxk0oIX70K?suvYvgS@a!n4maNy({FA4#g&q?NI@BVotvZCRV@SL4;Csi2qw=~1x<+0h3@n+RF=nA2 zhRF~l7;bUyAoUJn??Dk6AkjnyP*gFZe|O$OkHQ^!?9s>ZSOiRhl?1`+J>@zQs3MY- z{7IsVsCtN|Asgz)$}6$V@=En~Vjzb`hJog?ioiSQnFuQ4ECNdZOlh;tSMY+SO!=0S z$0Y7{%0wbHp;41fH%)=_6*-TJNFnq%lfX0gUYc`G&48+?%P+$mv!LVbgbP!7f4=$V z9Mc4O1yED)vhlDb#cQgwO~JZzQZU7wPe4d(5(-pN0WI}7Oa(MZy*Ul)^qo`_>*zQ? zM-_C@q`ZshoK6qwav`REtoEuPp(AL!h?2y+}Ue7hxARKF> zgrscC*!C(*4P8i2NFyz<*kv;ve~efpO_gV0@`~z~Hz{MYkzRXqJCaUt|EqIcs>*HG zR8}L6Ph-g7g_+QA7gyy4g=;Q~#eZ0f011F&e1ErRNPEo+R5K&dQL@8`_)t;u(4dMg^0lF;XoVy>A_tYYFeg22B_Slq2=zyj z6VeEXXW1YJkB5*Zj^rIGc}NK*0s|Vw(Xc+tV3B9EcI6FmIo2%VVSpauIbZhbO#(jSm^1rxr~scR&2#Bu{mcGBR$V z>>T8H?j%raRxK{@Gi5`IC>5dMFLL@*&pSZ!MS~o1qH$Z#LfUzpcYf17;tUIIn6|z0 z<>#BUiO(!|e{|A;NWeTK=^fJ$3egdA3M8Kd3nG6Sn}r0Sgi0dncn+jcd7yN6?nD!A z>a$Bu5ydm_1gl-PsyhYxY9TbWn=xS0(6yqZtuQ(NDngQ)9_LuKs3W=SLGl`|$Vp^O zPNmHBZga^MO|^sKvj}Bc8HOES8GW^8_SLav&QfYMK*ho&c16k9HH!J z2V`2*ZWVWp8?3Kbn={JZsJd_Y$yAQF6FdTyyyZ2Tb9%?ZX9SWsAw4Yw|AJ7h#&B9a z)YnTfe*)5FP30Ep1zt2kC%T1b;1=h2#(~aAUyd}w7@~mLlJc2@0zV_45Fuxs-l73f znBulwia;YeRUZbZGr|+Da6cJD&x7Ps0;V{JG5?al$L%c7pQ;6@!3I2G1Y6ReHZcwN zIyIGh(nbQJs9AlYikv#p=EF71*36c=u8Nq3f4~ObF-&F!8uB#46n`PIk)eU)`-!W^ z9%7M@T?~zUYLdwy?h`|xJiQVd_{3q^W*MHrJD8~j8W1dM)(mOhYdOa>L>R`XNEIxY zxuIrTlz@kpx)Z~q^ST|otvqyj3q%*C(1*@kQZBNl7q6{+naXXR>+?nEPz*akKy_^( ze=`*VzvC4!W6CknBkNnx+SZB=WTW|5LIE?t(1;$|HW>{KM?)I1f}Qjgn{8bIyV6a2_93Iq(P`U9f77EXijuHB&TXd_$Nm1cJi}I<7_CW6!wC0)a*%SE&E&^| z2r?g0R`jFi3}s(F@3IEQ>~e=0wL<4V2;afg_4Oz2U5LwpH)%CnxJbQ6m7t0Mb? z)1E??9|`VBlC{!T9PSXUK28R+O{!R>_<*I#{GRU^p)}t!B6f%o!pA4-Uu-!FUWrq^ z58-NzW^ON$!cX@M;>R2!xjw`7-a-9vg8kUXzHq{{Ms88)&miotA@DDvfAp&LtS{z3 z&>`+=^-OS?L5Qp8e)cEkRbqX{eB52s!mH-Awu944x}w80qcUG z)>f`CI*$v=N&*;=PQJz%-mU}>ivcMA!SB|@S(4B*4n^el$-TBJ4}>qboMQ_SqR*Oe zBfQX@41xz6;s=AUjksqme|#wuzU~J}hDzFs3tOdw3}Oh|kTiAzlJ+nv*pMOI5bEBc zF?^}LJmVZ4z7S;)E74DJ~>Ro)9}2 z!U#sM8<%g)*v2ZDkuD?=1_b9Eu>bKKnXWW?FlGet+#Koy&(9mT0vrud903v+4UR$@ z3L@n}9WSjNsp1{)e{UWWvL2-+0u3S`>0=)SCtuvF8U?Z;E}$SI1sByr7=2O6qKF}D z!wo3T6>*Y+Y@`^$u3i%2>$u<*4pJWIFdQSIC>!z)8zKipjwjI~2SdkHz9B>wB}BHv zz+mE8U;;)M+VYbuS)e@wt~N1!3INMbJEvT*FO z?5xEtt1>PJCog3QUDl8&tWj*}k~`MYFL$K4%rYk_L@>t!FdcI;edH^{k}<~vGq0#D zFHJ5uGbtpK8+#-)Dbp1V%ov{kp)#NK65OJOE1R)GR1;4OVb2TGc`+-H76)G zt%WuRM>B78e>k~AFz50a^vy2Q(l*QRFeQRHkq0&Z0xGctIF-y_+R^+9#|?6zJH7Ke zbD#_gi#W-XaGVi5!4n4b&_^->2GA2c)3Q8gQ-V&AJKxhgr%^s>i9YqSKmAibt@A$x zbUk4gJeNS!oFiT@KyrL;`z zHI-Fi)l*j$QfD<*m$X)GHAitZS8a4xS=Cp|f9q9wbyxuvSc`R2i4{+cHCa& zmlaf_RhESHS*=w|r&U><^;)rYTe($Av-Mk_)LX^%lyJ0K%QanbG+gPFUDcI5-Su5d zGhXG@OUYGT>D5}$wO;xEH9_;WTl+On|21H@^j-<}MF%!g4VF0f^cXs(!b!qo@arbg}%{P4`cYWFSeN#7l;dg#B*L~^tehJrp@wamIw|V<_e*rjv4flTq zn0V#4fDt%<4>*At*nk`Of&Z3(e<65qC3tu%c!DuFfiF0Nt!{xi_=AxbghjZ6N4SI~ z)P&=gem59}S-5;XxP>|Qg<<%CS9pfMw}x$advQ32Pk4TJn1_M*c7K?Ihd79zw}_4S zc#$}XEm($`I6{~I_;sWBiK)1Lo%o5TxQclnHlLrFo)@^G@fn&jL7=nbo!e!E^VoAw8o=@Zkwdp0uucOmx`AO&6c!g}+@Nqp z0Sd|>rc)S>ZTA@-%t5l`_ih?=aHb48c_B<8OpiL6HUg%pvv=C?8LCqfIf9r$=N!}s z>@p!sunSMdT)6D8o>722mAVS{u;n8TQcA}CrVpO#2QOpAq+Vpf1y9;u^WPz>j9w};;=*5 zvfVl@py?uz8cTE=&O|t+HJ7>NIwV5(B65Hd1Yv@4CITRNk;BxS3b!U=yK$WxepUCK zI3fpxkC;0N272(Bv;;beI=~p>6@b*ep-CudIN%*acy zj+mu71d5`4ig`o9C5>7Yr%7e{9b+cDOgqp`%om0PnloFUjdOX!=P<9ogX zslE}U!Lvla89>2{^c<}2#4CJp0Q|hK1i{psA-Eu#9^5g$JJlLHnuwIf`=P+glp(r0 zxc%G4C;vQdG8{oRyuDN0#Hk6zp&NAUo57*d#bI2=e`fs3YrHm|oS1dI2YVcvetgA) z6fGvaBZ!}KRCQ~SGy^fyYcvsGMFNa0CEw1 zaJ`#<4Lyfdx5ECo9u9=d-zfr0*}nU`ADAf*uCk14@AEtuKk9=DE8)mSYz>U2h#sQN9>%@y~<_1)n7fGlYZ%$e$Xdf ze~~}Oos8??vE&}+a=_QU>_1vd-hnW+>Mim2-mZ^c@x{KFje*|Xp76^)@;e+g(_Yxw;=2`}@3Uks zdH=fb+1i7T6Z5SE@&QcrWqt1Vb}gY{fAe$qbh#=8(w?#Vfi{-hN>;lcR_V{R#PC$U zxZ@g2-odJ4zUpA)xUGb%7@Vu9PS!yp2%yP_;QUIEE0Y~Q&D$K$vt;|LlP|)3AT;2k zYyZ1Vz5J0LBw}9ov84Bzy!LOOBO2TNucY^X|L8&d)QKM;7zh;Trk_EB?<5Qqf0*Ll z!G{q0853CM;kyeAU^skOX5p)d2QM;6QSV_81`)pXYb0>kLzWC&f_x^B%fpEbDnd*# zFeS)?6C*(T*)ItL3GW^nm2fViF=HVyk~E1@B_)BG9tsj@%HXht1c`PW_>t(ruV4%R zT2#2Op;@35En?INajd|yAVp3+e|XX-OqmA%{-nvkCfu9`Pf93>P@!RvL5Kdtne|{E zl#*{kMJSbV;;IluEmUhT^4rs>Q>$Lhx;4b7AkRH}mejSuZ^oXf{E1sP(cZNq%BJla z_QP*R7hbppQBZ`QncOfS-S2YGJa8pEgR(Hhvwk8MO4x1oVgyRu2VSWBf6%(X2-pom zi@+4-h3BIU)^mejV362ZTcm`#{mOvrkRS!?Y+@g?AZ=FOMa$qgT7Xx{`vLcTLxkuJPApb5j zqJA*Sh|)6^Uie;yPHpy5e}5wZmg0{s9`s^NO42AKe5^rpQcrF;R1b#{SUDzMwzX-N zaTEgi5@<$6(9}WmO;=h9IrJG8NUxyuiUtg@Nt7{s`l+WyJ@lEwb8el6la>9Pq)|e0 z9>fz}5sAd9L82A9(p?a7=+=d5!DOhZ_uxcbKOOlb5{j0G*sXRx zl&4yVI{IfsfkrT>sFEg{!?6f8+9#wNy)`LBmR8$OvdbDOm#?Gxv=vhinL=tmsQ$|A zL9DjNfCkBuH|uU!Ek+a!89^i+imyp2QhW!oPIx{2{1e`83!-J*V9otie*`A}7}q;IxgEiTgPT}j z-H@M~cHI#`#Wz@h!Q?h+(z&exiF;ECnq6S|9jA#jnq}i{YWK9+n05P&FIZCd(>Yk5 zpWYLbkyBcllm@l!(LtmyWV%SGe*<^jbEAfn+jLzuv(x@ryfoAPHmombC;vsNAE8M^ zR1Ze4!kouFe;7Gz*7yoX)V=kPUHuTg4b_tgC7Ep-aj&lu)(EDLWC3V3& z$~$mksH;>iPCj8EL>vf}_dTQ@;G0bLQYIarSWrtDgb4e}7pL>3k2NDyUB#Ryzk|@P zTK_^{$}FP2hjdUS#-K#)4)QD-o$P>m8(?tIa*LqIe+wgCdRHc(&;uw4;%BeG-*Kd3 zJvhCih6lM}rB(<+282(9v~dv{nlZ@AZT*9Lxc2iH$M#E zL4?>3?kw&gp^A@4lBj_tHt~rMAq){VvLyocDgP!<-O39Y9z_>le zjJLQ2zMv`1P(`2-=lF(uT!OamQU@A>FCLKM(*3=}fJlU@We@s4v?$6gl2s743E(Xe?Gq}v>3G5=A6L9x|x7+w2c ze^KIs6LNVnX-(_L(ek8$l01uQTl5~xNN5z2h@^~r3nKtjK&!tRxk%N})FdL`xW#4e z1H6aq6kH-H$Ys2mO25RVIwOOKz`jwk2_5DmgPf^A99tSvoz-Hb1WI4W`kb=%s&4Y=lh*2bQlr zL>|MSgan?W)kj_sB4jnI#zJec)JCMOraj?mtoxJ9$`+7!JqUJVYewkO)+x6YTPhKQ zSi*)3hxjQ&Cepc^$bP17#tmf%8qtUlna;UdGVi-QBe0>2^uBI0?pGJg|6tWvQ6U%L z<$qGAOG+U2qo8>>s3;P8;elirp#o8;Kp1M%yJ9NB?^tRq=l8B^oslO(p^Bcq@thn~ z>#q&zYOW4u5F@m&JR!1@^j=0|2H5x@IBu1PZ-T4aJjcDE{NM8c*45xdSxa)omj+xM z#_&*8p18L!o9ync5=U9-8Giw%L73)T)uLrCFKSF6i`8ye0)vN`xnO7x zPn==KfG^WkvwlVi@IvVlHb2D8aAr)Ak8F@6pQdEZGnhbfK)&UX!7O;{l^IR^ZX zyE_sss&lF6IKh>gBuq8@P-C@Le~SSaX>LT&VyM)DRx6;X$87HOYqI}U#llQx)_*si zAsJ*pcGYQx->9c3kW*^&Wh8QicU-zBHU8Z=9~M+@$3|Crn7Mh&#FJ?qnqTS^GmZfXWTLGiSE z*mjJ!v3-~$e;$X~gpRy{M;0JuD&#LdG;wRnKaP{}Oo+1%EW``5LaI zW~7A4bjF!7tKQbA?obcgJ(e3>BazFVuY}H-+L9Mlz50>GY`2@FGQZj z_|NC6?DEdvPy8tRSBUd41KGUIo38SMm;ygR*a%OU9&L!MI#0)wcrU-f{BJL)W#m@Du1NcVLSK|m!s0bmi)fDc$^u7!W!@+t~QUjD~?2DpN%r7iz2Xn%bmfq;1O9LYCB z^9Ov^6M-W!fuB)0zntcQ#V8?5xr*+CRi;z!D(OD7)scIB&Bh? z_k~~eU?`CXl5r0i^%-gfM14my^#FLHu!a}phMBi{IKfLZLWTy>aEGFD&_D@XFgszv zYx!3YPO^Q5NESFjS%2JhV5cDxx(7ssXoz9+3Y1ZJ9LFyVSBZx~S zt|)H@g%J-GiHCSDswHP;w`pL=8q;#e?#t=5@1`+abAu(1no!5<-)rr!FjP@u&5U58u_X^wSg@`5`*H{zUXpY<%j6C#> z`Vo%k*d664fcMyt13`{A!Hy5{ER@DRltmQQh>d!PDEotk=T{YNM`eixN1JpDg@zg(Nsw=nII&ibRdEh% zkq1fPQv%^gosyEEaUmBV6F!oCJ0p-DX&EQ!i6LQ?>NkyMi9HYDV?B`sqL3>-2a6bC ziuos(55bF)7&Zl=mhZuqD~2asvpMV6jX>x&qM(m!wSOc3s~CVA=}N~IIRa6b&RAJA zB0V}IYlaz^Y=$C;%^7aL6A{Y!9?ThT z&Z%wCDV@}5ohY*q+KG@LK}y~^ptcj9mD!&W^`B`*B+F@(R0I;~Sry4)3K2C77eWcQ z0F6JPbwp8X|{k7iH)*`g1cA`yBT zmX@0ASrxS=m&x&JuQ(KFl0P^hS|r+|20<7>UuW$=7f*X;!h^rZZ#fVrbF{gG~ zr+;{Q3woL|_BJk#_db_-nVq;g;jyM|np(Gc572fu?|>G9a0`R_3Nd1(8Pkf9xt`I- znvu$;ZYo!>Nj;IGshm2fpK6=m=3TjIc*ge*_3)mgs%?TgsD-+t5or+ea5ud&5WT6T zT?v*1!G*zEiUCL+z_}1Q@f6gW8_zm}{eM!eI@1IH)f5qwkOC>tTR71(hnZ|rV~;1{ zcdw{7>6)vf3Zkc&G9XGy&I(iz;T~?ltSH(mborZQI<3`ettlFR*-EY3+O5$TIi(pS zs;IB;00SwYdv%g#=sIA4136G~TyA=REny5HP#|CwZefM5`dWPadadH9u)m|ND1XbH z789@n+Yj7|61^F%1F^8!HysWOu@cLVnMoC81+Ql@s=~Ul_u3L+N3hAtu*Z+aksluYQ43MT)CxbYlcv9C7q0(31KnWjl znS7d<`DdJ(LUtiWYI2L2a50&v>3>&qp=2H7sQ>)9I%#W@l`0bI3LXkk1`7Chnc$(J z`lPCNkt0*8rr8gPTdu1LK$xe4B18$3E3558t5Sj!Oh=5R=Z&qWFmKDZM%oWP7LN%z zprHf>GlHd7+ccpf0ied1w##|s>aUJzw*;!Rr}2T;=Ro5bcOY>l19qcD8GkBu#6D;R zDvxlbin%)2{OFn+Oj&SN?>P>!nnLmczre-l5RVd zpvFFmw?&Ohbk{q$w*xs2GcKTW87V;+3Whv4r@coJyzOhg;No=>SvSaQlRc@29C5!C zTBfn~yZwt1|J$x0VZGd|F@FLxp1MjsRLQXaswsAu%cOS~!SUmm8iBw^Tb5Sa!w>up zlt2kwSP#~L9Y#SlH^jI!gDJI%5sk~lk*go1Yg$1(#OP}+sXJM7vM6W*7~xS6lcp_{ zN4M9;h;QL#lBa|~Q)6PRK)Pvfc>-nC=C^eGsCtr!TU-#8+DF?B?$l4ahRPn+;Aw8y; zCvy`joF+Yc;loncMF%o}hm}B7JFo$mu?k1O!AcaFcY^ZxMSDcFlGMjy^?6BAWUUOH zl{Oo@fy|zv%qVEPIDdhK1_KQ`17n#n&Hq$5%Y0nR4_sBT{1YW(is|7yi7XPh+=#lo zuW!-I9Q-Zm49mRx8RUj@ZZRCq3@Oi?eHJq~`2!8stS(&_$9H4QdHl_U63!qo&H{un zww!i_sq;rR$ALecxa(`Gk+@+01Y58G0;zz6g>ny zwWfENoX{JsEe>&DD?M8ZQbZMah^(=G9y^su`G0{@73L6q$yL zK!_{Vy;Ri}jDMoMh*9rwA>{JAyfh&H4DrrcxW6=;QBKlSeX7U>fffal*VxF;MQDMI zAXA8~PJ~@Cru!kKYhVbYW4}^bL!>*J|C>z4X;(ofEaxO-0#u@i--dann3w5(Lm7 z4?+bX7czp`z1`i7js;Q=&J?(q!g2314CWmoTN56@eGX$`FZSvcpUm51``f`iJrDgY zvzxppN)#6ooxMGty;4THjo+{e!PR{Zlz<;v_1!z-fa3ig1}>TMLevI>A8v3^vaMC2 zRNhqm9)BvA+xOkuG6CQFP2T{XfB*lj!7hs$1HRmYdf^L5oF|^&h~3~1KF0+F-u_YH zOM+8LKpQ2tLg}p%c`_Xy&T{I~Q)r#eCa#1EZl}R3mK#nGP4448-Q}-rxiouFM=mum zkpr>KQ!3HdZVK2g)USolhS2_A%^I79KCeEJ^1 zN@y4iNt81bZ>&d%Lpc+2neS=GdK~DGlLMD5Bn!bFe!dV9?bsb0JLRexkZS2r_*sXB zY^PK^ojwp<(b)n1OHa15K%I2V;FJMTy4jLS_&umJD~Lh7UsFm z{D0?d6HT9rtg@U7IcZ`Rwtf`^xd5mHV`?Yp7r1Hm3Tk4syPu^w1AX=*T`7y^1#*eUXtL~!0;BM-&H1ED- zkT~uYSN>GM9&!*)kMg?_6n!-jlT5@^@*jT-JV8_K#x}_(5g*V{^haM>&$4+nDSry4 zZuE-vBRK%7tOyaAaQFR}YyYg#_VNMu*3l4`k&E*44knU0dcPSWp>A_8dJk+FR6+!L z^Y`5m_@)*SUB3`eFKQR-A%T)}tu^sk3MbyV5TCLv6SZuZz?}{Q)UWzDTrJWM5DY@x z>Q~U9!4wSK9AsGV-311(8ct-Ga3RHhZWzG0hw)$%1|da`yx8zzM2w9_u7BLvv0x(! zBQH=oC@N&0gV1t}j6n1M;IkkrlYBdvP{KKb!xD;|*$}2opAC%}GpLfKNRAYPmMo|g zsz!t@Sw@~gVn9Idy?*cBVSk1eQsTjTkDpOKd-kNndXMo=4E=0$Y0#oc!yfAxvcbt` z&$(XR@SE$%v>E=sE!?B+;JJ+#Bp!VELBlx*1eA>kJ0k8 z%VlL_wB0*wd-IRek3Bzs;_l)e!JEZTz5IUw%WJMI_{PJ}z1Yy3ZGWQu%IYsb2qWxn zK(Y#)s6NcbI?$sBG5>1NK;9%2aYP0UV`w*M{A13w1smFsq8BHk&_(oA4C_OWg3=(v zf{X}h7zp#oVMrp2%%P0aNK|r3uyiugNEl39P81lPEV73dnfwn&CWW+;$TGB)kV`Sk zH1o_d$wYHaHrwn@O@BAzlygoGrKIyt65+&iIXw?5Bcu>Sfo7PX7|0<)F0L90h9viN z6s%_uQxv7{D48^*2oQV}tUe`7^wO+0C96|WOEr}|QcqQNRkKoM^;K9)jCEE{?WA>7 zTfur~sg)$)?#y>$`s`L>i-pitW6eZ1S!VHEwpnP~d^TEYm4D5(+ES;D6;rG}nc9Uhm|k_g>-b#WzBIRqc0QfCGN+UxCLWxL|}C zIyhm37iL&shaWC2Vn->K_~L*qzF1?7=jFKLbUy|eTaibW%3W$pHhE=cRW8_NmRm+S zW{_c~`Bs}v&VM=HnQ!(P;+}u5RcN1gCfeVij|N)lW0OwSXr`lH6KZ;?mikevuf8*D zt>@&L;jfhpyKAxm9GhgP%YOH4wXJPCVYV|)yV$tjwtKO;qpthzCHLlAM8E&$H}Ag# zZ&z@{kxslI!xul?ad;bt{7u67mVCU+Cx1J0S~=G|-+#(KS6pZ`Z@ zdhD~;etYg&$Gdy*!xw*i^2;~>eDu>-e|`4bcmI9(v2R+Ea41Q3ABP3x7 zKNvz2rci||WMKeN*g_b_(0?zK;R5ehLmcK%hdbmU18LYpATIESK_p@kV>m=3CQ*qe zG{zF0_(TsP5sFg;pcI{WLn>xbi(BNO6}i|$FMobfj9U!j7|rO!GM-V5MNH!wiP%OW zV$qFrq+=cb`v*rmju4M|bLmS}_EMO?6s8@CSxh<}6Pfx`W(jw>OlV57nWmg(G|gB|YjP2r+0^1ThZ#&N zK9if{%%eA72~Ki$(wymB;yT%xM0e6Mo);A7JMAgPc{(zm_XOfT`Kd#H{u74*r6xL| zSkF#=qYN6*02JPVN>SwC0vfml2OE)uMt^*fpn?41Ed-GVL5Sj^{4+))@{k8gJOhBQ zaDz$#`2UWNo&$j_b*cZJQHF{d@EoG}s33CsKbYo0rULZnNkuW#{e2@7CvC(q2&h!1 zI+cI#Fhx&aG1351)v8!!rb$s+(;0%Zs3GNQ{z`h%l&-X;S&ivT*ZRMk{*-__?SH9U z_4f@*oKz47-Ct4jaMb=mH4jZCAT;ti*kBH77>i{rU-|b9H)M3GJUyvhnJLzuiu8_N zP3!+?%2uJe6|DS=>jCKsQp@5GuX=TC{`?Bq$1ZiSQWc*P3fcE`g9<B;`AGLjq+w|@^N3~`4MlaWZku@~+YC*#W!`rc8M>}4>5KfICyN@A5i$UYJQ5@8Z_Ocr z!8Zbk<|!+3!IR2CS=WnYOn)Kx97q;P9`?uK47XODl1}1uSvcku*2Peb5QLd4xswQh zl>fFD49a*Xh~E@=dV}fhBMKvhsT@d3$@u#dpLykI{`r>yrtaS`zI#Aj^Ll@#;Ipdr zXA?l}BqVp~^{;^{7dmHm*788MK$N`^Pggqx*Jc#{UftwaFT2X5MSn~L!flr-*Lowj z{-3MMeeQ_lPu(zJcdkPs>_!M!Em|-R%K3eA#{`I99HW zaR0pn+J;&<{$&1d|9?T8FeA+|3o)VNF@6WW#e3bjn6Ltkxga%>AKsLmw)Ut+B6WDQT%uLCu^SWqW3oG?}(|KQa>c7P%T6<@s4w(Km7zWMq;X5^~WpQ zvMAJj^LZ~1-+$L0`0_Y@%bQz#0qVb9Gd$%JiN(t*?&Ck`!#}o=fa$9~31c`w$^X7$ zYrX{ZJkOXR@WUVRV*v7lhS{67XMna$$hicBypX^-$AAVFWWjF;iU1q{l+6`kskDPmkBlNU)P=7*W@U(B3igCjqCi^?%n~Kqz zJSX(DbC94{K)0!K!rSYgE>xvKF^1cF2P_0IBSb>~DZ?zxA83%m|CvK$h(H0_3CYtR zJDft&dO`tGyZmWGKa9GmC@Z$>A3MZDtOLYB#KKLosi_#l{prI1QbMe1!b4P`C|p7+ z3ZZj=h<^i=2PWe|?}{`02{U=nwE6lUbz3>20M)Hun4I{XPmcg&w}yvC3~MR^!USq!>!93XWh$o%<+cCC z#fO^5f4s*g%f|@VM`L70f2>D<)SrQL3j=#d24sLB05Fp*KcM5EaI`XR9L0((MX(C9 zX@A5ji6qHW`XM`{!~?QK|Dj4noC-#a%C{iGO7sz#kik5Vl%Dt#$$JM~!~c{#7%5-V zu%-*NYV4Il+@Da4fQy0(!#m3+OG^xjfV(iT3y_o)oH#cC%pg!CW8lkgFffX;3uD+a zBY20kNDQ5X2Crm_S(C_DShEOlF;mE{CV#`oYeY0Ckjmz2OOe>RYox_Mfv<$Pg;$6P zg#;i$q|BALfN2YgzoJE4!%J-pO&y~-igQiaB(cnNudw1DK>*Bw*dIzzP5lba*K9K8 zGE1J&OI%{I#BhV{YcjdQPIP3U5c&p)2*Qk%PI>T=*xbqSGKLf&l9|-M&D@LcWPhgi z91?##hazDMr{o`vv#X%sMR_2$_=H6=M9E{TJXM^yy+}F(!?^@?##Gb)xIL7IKjA@% z+&T%}p9*Cz^c=TWOGZT!&(*^*^X$;5V~h2yPi3=00sV{lG%FRo&-|Ro0P4^E`A^N* z2>>m~TPQ?{1W;rHuLo^B1`W*e9DlfpgNX666bazM>YGB1&;Xl=O^rZO47!D23eWK* zQ64l=tV2-@icydpwkC7Y0b)=098=Vr(f!Of2Jla8dj}mYt6}KT0u4%PL(txY(1&W! zRJ4mRJPDjMPyJys>$_0qvQhoP(eH~%+yhGT#4jFXGKrJGmCI25+0a{hA%8)j#R~{X zd0gbk$b| zSXXEu)|D{UcT|*avx+ADQwG2ifKsG$kk9?m2p$ZIQEWW4Gdv{?EQrufh%Smt6{z+-@?g)fv%s*ybjlLc8dy-^xXIZ~bfxP(YGiA>j9XoN<{ zgoBL8t;Hb6D^GR>+VUhwl>ks^h>7u%F-m}f97qB|7&oIGH%)tm_KQtQl?P*>ghrr* zvV+K|m05Y9T6yhQr++=!ur=72{o2WkTmG@y8ZF!AJ6rUF2(@Kfw*?@Y{jIt6S%9?* znc~@+(_6kh(kTQr<|^ERBwd%~E{e0hf1O+xv)rbeP=5n2q}3myjohcC!K-*L(Uk}P zQb=z#QhB&ug#g!)$X9vTGLeuY*SkYwNY*%m*rO_6Z>gQ$+@_J>H)_&QUzQ+_DnUL5^gJ^xTu&Tp}L-+pKD2e$==;*tr_4%8`}J zkmY19WiCyIFaD{uIKxhsK*dq6GKsunLY?Mm<+DOu;F}XB_OqfYNsBrLJ2}mEl-#W z-mR8SvL30aX3cXBp@bf_Pq~Gv13m8=>-H*ZpN?s33?O7ISc=p-BEEXp zjDLdJ(ADMEO6Jt_L}Tr)@@)-v z(l{a@Z<(rSbDi)!9no6TNo31vK!vYwL)|Cf!PvqNY(D_^@LpgiSf5AYvhj8y;yI^{HtOt zSlKpff-FYFZoZxH6+|^4wR`a@Z@HilUh96cMC52NzwM)XQf|a@dEHUQHO!DG^Tf=9 zsrYllJM$F5bNnIld4Ajc!N*1(EPu6J3Gy5$gAxVANN|dSZv5tPVS6=P(`?m3j}~-8 zPM6KG{f!osj~EjPF!yxh{?2EJiPH|*p!n%H@Bg1#M?~qI(kWbCUS?OJ&2;!y9Qdnc*(8aQ4zcKzYaXJke!d&pb$e29=ihMx5S4CD*%`eVLu zUyt(FFXzr)_aFKO%RfCquIwFM_SGM$&)@oREvwN7ZshM?uIfMrcu0-o=lrp9{*-QU z*X7yYpG+t>xeDMyCx6sRj0wqGOPcFdP~Aye?ET*#Dc#TI;g8$mPttg6evuG;fTw-X zPyS;Bhzkq`Hj-y>*uV(J3>wTdunfb92>%&~iYV`#K!`yYqplv zq6V^TDPbf@i2@16e5ZkdgmW<&8e2&68xfXVJd(`A5(Z3pAb&+#oLF%u)TmOYQmtzB zYQ&xqjkJuAgwi*L8#_i^3hkRfE>O`jP`mRM%L{VlvV_Q$rB_P*_VVrPm#Og_RDYmBY%MlhL`Io*UQ!0R2+eB- z&DS1ju|;&FJQxNDUU>Q$#9v7Mi3Q}3172hyLm-OC;6o-x$iRl(F}ahEN@*BhhXcW& z-+WLYcBPfTMWmvZFUrVbOy6uY5EMF1wbU~)-6hI`w;07hBd@%}mIM^aBcPz-?T43< zlp$#lp?_nVX&{0MLg^rcTRLi4Oj%A@r953S@a2whUV0*#@}Oy!MKiATCIfMjG)xY7 z338yCBzX#IO!HwHlc7T*8q+W{G5;V)Q#vxlBYw#$yDYP|2`PvKjS<`9qGtWJYed(U zr(IhHK}*0`0%cMZXYtZo z@4f7n29r@QkYAx3i`Nm=@y7#;jH1O^I-4VT7Jc*0J0@p{9+?6UZ0{>E?$l^^KSw4kpUp}gWU6(^ zxqsTCi52(sqgrz7R!?E~sxP+x6)JXM@30mUB_vHr*wzf8DbYaxK71snC-X*S(D|l0 zvOGfnJ!sZx?bPd@id~%~g<6yUgyP#RfsOXaDIcjM%{YpzAmJk(~X6WPtC zqIpLI%}f`hgxFaCGw`}av2rWQJ1Klh;eT(K3z>Ksi!)kyx1(nGwX*WS>L?uAk9D~vaAsBB;vYYKg6S}G8slx5r5$i zL{`FpVR)_)6{%5y_%)Tp)F)fI%ZUg{7?Ed~a6C9Vn6IL?I)T8aVfnfnwIH%Vu0^bC z7kfuy^7f5rap7~*QXeHKA^#|nO~x1=R9~!KVmT`2uWR0cnRS5VM2oy5iby0=&K9*Z z*c9(rqOoGD$e2Zxktv2H#L-T;5`Td@MI=^L+DxmWrI7zg%}^qg0Q@4dnL+Z*ImS~U zx5~DYEgrEdW$+UIyr!M9{U&c9$_#(n2R$B6@iYSw%D5X4SQ(?&_<-{HypO>Boa8MK>%40ANEXvYN}09 zk4m=YdBl|r_}8KMl)G0FQ=^$O(?JjT!L82qDsUZ0Rw3y~Fv0{RFn@*K3uDBXTi~oz z$P`aA8JX6Q7$dN=8!Q93iXyK<^+*<#3V*itIWftjK@a7Rl1h3N%-%!-e2j?*w`SK~ zQqoic!Q)lyc{zw$1pkgfATIXW8j~*Jb(A>KiB|#Sq!jf8Cee-0boaVPo(!b4Ve^P~ zZwp7L<~Aa`ji35zdw*M!l$RmredJEq0^Fz^Q;mnC4{`09*iM&*xDW- z)@D8$a5J3GXn#cN%x73XW6W6vDgjZD4@9xW(xc>jXlNuZCB0o3a6Aoap?HL`Zf%4q zM#7&}w|XELL!)KwYbaDx8XACvsjSIY>%#i7yN#CRT4qv)XPm>%*6gYv`M5SoLfNFM znGX?$0kV*#XeMqTpE3glPjXji#48l8qFyb#1g)EYiGNA*c-U8I+p(5f3wXPuhBON^GoLy7#j9#;W zjpR4ML>a1K0(E(u-R3;AaE{MCkgVHm>WBo{$A~g9dhZyksBoNwQShzzrkm^& z=Ap3~(8JdM^Az#7q#`Kde!iz@sCU>d_^L5~OQ2t=d;|L}SyhNoW2goH^;b0c4dx}@ z@qY|Ql@Q7JB^SNvi`ZFUR3y*_BG1@~9RRMAco`63t zV+mAaB$;@~6HU=z1^NuwVIWrEA4zZ^RIJT#6i4FhU`&J{PK@AZnBeFz6?;I+XBYud zykOL!4u+{9uc!tNPGA&jAbtJT4}zT#E`N;&65bc?gc8~oPmo|sI3aLA;b%O>D-1_- zSs~$Q$Bup5kVW8UOo(uN1`HP93_hSC_K%?`AxA)0vvimD|88VPy+A;LFKjBIRGFRSeFDSRhi_HF+g9c1=m1*ANwE%D#Ao*$bV3d zlo|Bofg1*aYUCn6v0{esVpVMvCo+I1g5m-;7`4n`CMcgLT0|o>Vt45xRUG37N#kfF zMvGO2Hf~}|ETgf#Lx&V1mchh1TpEbULp8QU4#~B@8@eOb9|$X2n#_SydKTxul^%hEIERWn5}uUge~Q z;b0GOOPSFmVZdZVMrI*BiyKBj@Q{Y`aX}rHSu~<$Rr+K^ydz7*V_8%tCicbHe8V#w z7(=Y4PMm}JTm;bk*oeKuYJVET+Y!>s*`_fVS+?9}Ze9g%;^vd7CTngKabg8D+}KVm z=W$Ana~dDB7-v$rCUi2V{{&FG^aNUwM0BzxaN@@|^n`Cp=6z@eJGGJi`?-jkJjbXWC9EjDUX_HkXD6;Vjhkvsg6QslP1fP zRwKak%qb91BvS|kY zsp(H7BTKNMdFIqmS}LiQ>K2-6mSQTZt}1}4DyzQgA-XE8&VOpT$ttZ%rmY64rn+h) zzVym}) z>$8Gut#+%pmaDm$E4rR1xu$EnUMss=>AGeqx4tXA)@!5M>ty1qzUt(@?yI}@E3N)3 zz@F=)&g;M)tbex(ti38M!mcXAHY~EnE5sV>!%l3)Hl)E$tj6Bz#kQ-&ZfwAQEXcBI z$BwL`mh8czEaqjb$+oPes;s!etjm(A%+4&Iimc62YR&5F&dL_c<}A?eY0p;c(EkQ) zlNPPfdMMH+ZL8v}%`WZ#{4CQ>EtW!Uvsx|H;;PeTEq|I~E!RGv*OqJ8HjC7Lt=XFA z*oN%c&T87OElIVl+Y%DoV(Z-EN7=?L-ac8~I_%cwt(NYs-!2Q_1}?l3u0uj9w`Pvw zE-vFXuH!x~ZY#ht}g4g zuIs)o?0?3t?9MLj)~@Z|uI45#?(Qz{_O9>#F7O7g@D4BW7O(LhFY+d@@-8p)Hm~zO zFXVEorcN*QRM364wW&KFZ!mh`mQhgW@7vQzAya7 zul!HZ~f*k|MKtt_OJT>F8~iP0e=^;0UvOio-YC~F!c&B13xeXzpn#F zFa=le_fD_{XRroWF9vV02jiXsfAI8nun3p137_!zk}wJnunMoR3s-OpzpxA=a176| z4YP0v->?PO@C);>4*xI^XRr?karzE15g%~?=P(j4ar_!F6F)KcIx!TtuN0SX6;Cl1 zXMeHxUa=N)uo8DM7)S9JhcOv@aT%ZS^>Q&9Z?76}a2u~N9Eb56$MKqtF&*DA2;;FH z*YO_rag6$L5;O4}12Q4^upJjN{T?zRpYb3s@(nk#BhN4-NAe#p@g!3+CjUS1CDZXH zXYvVmvL}BqD2K8eD>5m==qMj>DxWedH-E7ypYkfVG6l!7EHf}I*D?*iGA^61AnWoj z|1tsJG8hXpFbA_Q7qbf!GcxC|GUIVG*7PCNgwDk8wS`0zeDGEnwnASM)`%b34B?Jzr2cpL05Mv^&GI zJm4sv>+(-UqG}&-yTr^f3bX0j+$jO zI&bt}JVQGRf+(B=NRKg3zk)jd1%Er2bV~n4KG*X-v*}ig%RaN|Ob2vK_r*=;G$!)2 zPy5ACKeQ+`^ccT`DA2SoBeg~Q#ce9?JLK_SOm$UXP$sza1)W14gtH!x^ZX(<2IT=b z=Wjg#p9bhO8B?GrjBl*CLHCNnLP>TJA9E%<@H0S7k@!UjVz#)%N*UC%HGgR;YGV*c zMD}SSAWMY6GxSBGogW6B0}KRTCin%Qew9%D_Cuj&Q{sdv8$)qRfD)HB27$I{M}R4i zwhCTQYrFPpvuSmU%WHEs2FZ3O);3?UjMKPVJP*Qjdx$X zLr~y0ANNIQ2VZpe1#_RaVt=SM1MD}9W_P%3cWgHXZTE$E6NYjNRe5(sZ}-J|2RC}# zwpl`TbIbQ$?Dp>QHDBa+R|t4tKLiGCLy_g zUreHvVGwSZ3-Bp50%G}5RCzy)dH=zHK?ivR3_0pu5SzO>>+yw=BY9N!MVu2kogamr zD|wfLw39>mpNGqot2riKd6r|ia9lwDiG-JrW|;E@nwR+nO}dx==(MK>I#?S}n;Xzs z@dYRPDoe8 zQ@XfVI+gE*XTSi5bHra;HK)t@r}u+hNg!tfd9&;JUo?}fi;Kdc_+j7pCwu$)<~THS zRvd4H5umTQZ|{y%F}ORj0Q1>3*F%>Wx>rR2KTo-q_k&O60e|o@xlIG#ldIXj_q$)z zgDCVWOQe9Ao3-wR^Es>ylee~SO#HI@g$ShphBJnt_XBP;!d`&S>|9XCfBZ~rv@%(9 zKO|*{T+qp1v&tuVz27^g@iJe)dq2oKt;hUc;5&v7JWKd{!T$riP`5L|oAtr>pTghX zW1rGqJp7+SpMS@vJRqg^up3*((>li!1j+xpxQx81m;7EdlPKK7#QO#YwS3CoMUr1U zhKo=HU^r^#u+8IqvFm(4@O;w~{m);}(!;vI>;0hbg~21du`N7b^u5DF{9Zu)UdY4L zSGv_3JJx$W$ip|UYyH>%`Zt_}Qhhx1RkpaG{mbvgJ%9X+=>v_4TYmV~JaNSPymNQ& z0sV*L1-CnMyNmM}ch!Nxy$5Q!em@+4_wPRYKSRZ@ z`(I!`dw;KU`}alo14KW80|^>4Mvx#hW6#bZB=~J%Lx%~C-5Xd>A+&lEIU;=b%$UZG zBT1Gtd9t6uf$uIbNU?EX5C##x^`pdqK_-HNaGDZ03};XVZaS6``pP6wp+h}NSen9O zNn<$65M;>!hQ+I*Q1*i|6B2zojp0K6qxsJzp*s~>m10FfzN(Qg%agxG$_)Nl!IDc=~Sp*scKiY%nCON!VnB3 zNObs>tXQ!9*t!)6x9Y^&Y2)6WeUh);(k?BI4f`0U=Kq2R54U_No2E^ijs-%N%o(R= zhkuxpCtb?aD(Il1N#eF@`W|Z4tX(67eO#b%+Gly^cDVbkTF8wOMqm(xqu~59nRhmX=$V>0BGJT&NEC6R4@1OI zL4m$WVyO#mK}Zn)aEc%kH^37p51}H+BY!^$p7|=M5T03ZDG5^Sh@}W9K@z76s2eCq z3W*d5nh08Jq)Htl?57?##QbkbF1@Rs?B%%Sxqm?Jw@hwnaSrL-yG{Odxc_M;&KDcCS2V&SFhr1=3 zAf~00ATf_x%9!4PI97~-93E6!0%?$U_Oj-B3go#p%_@UcWHLZ+;iD!$+60 ztR9<}g2K1&Absa5Ax|&*_Lp*+)D+|gd2sKhvqd`d>y+Yxh`CB5yz;5)I~-%|R=qypB;eLi!@ zu9TM-E&Ll?^UA2^07$y?LiF#vz z?)X;3t57V9Uo>DBMSobi4!eyZXlVoBr*j;NuKm_AoIFdOGXLE zHckYBUc}rS0qM$qaV=y@xy4irC_{6eGyg*|l9Ti_gGc042!DUe8%R7WQ%C7lupjGO z3h}In87@HMBS>-#3r+D3Xi&*Am%@ho(d*ArM3(Zeg-aykbe~lt4kBp(ih_(NF}Hgf9#LVnCh0|9}EOhfyg0d`o{VaA^%XueGbHjY~N5 znjNW>_{fRS=p|Pv;z(@U5qAdiYi$wcckJVk!!DiGO^#PdP~Z^rZ$0 zl*($p2buT)_w95uXiclfk4=b>-^QK3DW9U!=%u49>|>??2x~nkV`o68eub0 zzCb&GkqPr`#n+bFDB-%Wbj{zZch*A`0+w%@4I`sgN zME?EGZ(Q=a@O#E_5Z>Jozp~omeu$#@|MPJ9K74=OA4bW;GhA(x$QzB8oAJMNBszYx zxZo}f_`n5{a!E!Nv1M?Bc=+{=g{%B@Yn(`*8Q|&-PI%lDH#d4&%*J^$X}5p&Ga)@* z5|AH6LU_0LWlLk&Mk5&df;nyCp<^vWkQ*kVt}4j4f0$ zxaoghcU4}O2n3On7p?3qQ3FoWe-Arydkr0k;3V$@(JBh9_>4?g5kf>^3&oT|6bOBL zaS92l)jf0Ml3c~-#CF`}aRe5)%y#bmfDn*9m$%aw2$(bumuxKKc*v*gDq!!|Jp`0T zO5{NlBG-5Ha;3}K1J5DzY7NGuOOpS$+8%%K;E@TXU+zND2%o``Vg7J*FFOY-h%ML_ zh)DCW5ac67`RCZaKu8a8_;lnwE?en!gED-(yA9|AF@=s`D3A#)9^#5_AjTzx*bOhB z;Tsb08=#^2IF9M80s$3}0VhIL^lh);BNHHH$)Le0(2gl&LfQas$v`lMwy3HMNaueL z;skT9JO1z73hC}RV*g$!DqN80IIqI;txuH0pcHTc8_?ZekY{>O0-H~hHcue*4a3C;f^`3v}dfw<6 zd4sze&hwz*7ITp)LhOyMqPP%(GMMpq2I<@^V;8fqB>>R*T1Fdjv7WeQwl0Y$L}MJ8 zu`{khtGp53VsJ8WB6cPs9TRaKhX%0lk(BH()=E#?JP{=%?i`39D7YXhEUxR)4gCxa zR&0fDQYyWIkPUm$BMahpE)9SGFGvU>_NVxWZwle9;T})~Us4PWF8f}=BERr9@-HGM zvLZneXg>00<|PD=$hsI2Cqe)y%`pHwGD&#G7)mlB6euXT>$-~1w&d$2O|c4V;vKF> znwCr^Nl8^;zzF@3Aa?RBdJ>6#vJU$}D5HWXner%;QkU|@C@aM&p>lr!rE#5HRXvPp@J{W5vPBohyOGWh+J?2$X?C?Wz8~>(|s%hG)<%)qvAP<3%-`~`#NMM zd80bF2)=kTC<0Q}aB|;*BJnU&5*;D~cSK?2>*CxkFa$wUZUI8dU>ItGp6Y@wWi4L{ zbkkZXC}2Q14Jk6NC~o-ZC9|^nL~tetvb)5wJM^<3%~Db*BtL)QZ9iM4KmRj82lQ}+ zVnKNkL3u_&3uJ;E)VLmn3vR(b94QEGt3n0yLTQpCGS4poX&0Ze{^}D^L^LFq(nSU0 zM7t_Q36y^hR5}xMN>dCg9`qb+6i0_C#CX)~EVM@}1gj`S5YA}HG8Al-B09~GRg`oL zU&t7`Y8Xso7!H3zPfUUTP9cL5Zebzm6Ht*c?jUIOiScP?1 zi*-DMaKsb@E>@Cy>On)5wPiMSSU)vX-3s31fqI4`D5^71a}ps`#SOTy@7NwN8WRYJ3mD_&A2)DAJL#V!U?NaGWbgfzTFbM20B zDCiX?`{aLK-Hkm-L;qm=wU(CkV*{371*Tv>Y9!=g0Y#)Rb@XN~^+!nsVu4g*>p>pW z&mpqp+mKaZDz-5HW69i)dLu0m%`$w) zS2y3!e4AD~M)o!#xGtM+V{n2ji-rrzKw*FO&IsujVgutw_?Ocza`5sOJ)X&bP;~?S z1Aqr?>fAMN*$+|}*nk~a(&|ux)wfCQ(1IrigN^VVHn@Xj6;@Uwgzpz1`*(krxP-If zNXkGs>0;X|$At;_BIsfGNDY9aM+$ZVq_ItI35-5RQ>mf^sghpn|j;? z;G`q|s<@COX#bT->^5^Jc*G_`2n2ucI>Mbx zIG*t(pOMi2o@E52?N+~Ogq(>LbMILv;lTn$EvToc&2|K>#pwMmzUHm zSKB7PVgH@{g|^EIP%!IeD*LhxdYt0|pge2XS}($yg@-#)L6pJ>jGwS%Lr>Ox23o8UN8XTjjQ`mpf`y4Q%g`P~j$-8hbT)kO?y>}N5+l`##n<44DT+?KolHeJ* z3N(@qE*=XS{<~WKL&0~%zzsUV30%iBbts}ryH6{^y;HA;!NDyYV>W77g|(x?8z;uQ zyw6)vFkHh&e1ehgy-3`rhepMRmBn$w#bG=luZzZQoUR1i$7g?H$8EaD7rY62gm&gjs+Kj=^V56!`E9!7^AiEK(CQw`bR=Xh}UA6N} zxLv!r*+NPXy^0@ll@@i|^yxM6Le7^WHqy~HcjRcTZacZje8M8LJB57YXH>*69Y48- zK%G3@_B*IU)P;Xyr)?LA9Z0cH;#Nytdc0;{a2cL@ld!+TCyu*b6B>KXM_>dQ$a>{L4oRhxK`<6nu6IET{8%3-A4n;vJ0h0o7F^ml*Mp> zd?}QLepN^Rpe;6?kOq|-hRIz&xG{eOZXruVBNJSc2r8VB@83=-xNbC@l_? zjn8DE@`LbY#O?!48f9ZkpaJdG9=1!!{@9ZJkUj1Hi&ApHSY;bgG@l1s!X$G$@+W`o z?;%L$@sQquO75rgnd2%-e_k%%gh6z4&V&d~Clh}}e>h6N9{^t{1pgts2T!zCouiNc z;vG^!3REa&CuWi)H#xUd-TNdaL@9j)J{_Xoq-W^6KLW}CZYLr~gdzw|r^fw$@VF)Y zdxs$8;n&N6X9`as;D0IP-(@s^`om^L0fK?Rfec{W>xb}_27)1OA$$li1p_w_DJK1@O|<-~{QxDbpW$mByo5eNqPRuNi`my{ri5|{^~PM#Hq zNgQ~gXG8`P&OMapl3$lg&@e~7E`{W77lD!u{YU@jU*7v;Tj?Am#aBf zdAD0#}nc9Do_C;A`nP#485t((V$fJW@D{qKU!~qJ5&kASt3fX!+=gdv;oA zriYf%X_=pvifXE=rh4kCuD%MZhpom+Ypu4nc5AMk&Z=vJyjE0(Kp}_%%`ktVF_1%* zQOyMdqrE;W(=)Nn6<&jwc@{yF(BkUqvR}cq?PdK1ly6(QK+PLt# zEAPB%wwvHlDM_$ms_z(At-S&dY%0D4-&*g%3WuBU!VV*B@WT@K+VI4+RxEJEl=R@+ z1%i~cY92;JY;nmCV{CGjD5rnS@|i2Q4D-1$&+O{UG^1%Vy(HhvbI+yb?Ef=sKnG2< zOhXrqG(<-yo%GU7L%FlVPCrd`(@|F~-_%(5to7D2ckMOIV25q=)e4VIcG?r8O*PwU zTkQ7RQkyL|(s9>)_r-MA&3DXs7yWnN8SgDP&w&@NGvZVp{wm{&KOTRp&%A%fx{f?aECnXp$~ru#2^ZBh(s)+5s!$(Br0)C0hqQkKYUrZe|=%x5|=n$oQ15vgg-Y*td4+UzD2vFS}Jg0qgy45vBIDM)Uj(~9GN zTxT}h>CR`uGoHTtEZ2;9?_^(1=eGHJE?A zg&^`E2vIzhhsJ0`9`ZnmX8`dPZcr-_-|^LRAd#(Z_2DzhP*o$ILlj>XME_oS_zg;& zRS-tqp;yl;3b_I{6K8G2Fo^gK#^%AXJ$#2L`ud8o2GO#Z)ht8Jiq^W$5u}HI4XbB! zSk|+qb**jPY+UEs+8^HauSDEyUum1ecf6vrL_BOB5(`ALD%Oa@aDyp&_zciC;u}M> zEm(1zL*I&mxufl2X~lX=(607}baibWhFiq$$`-Fjbgo}3yIZhkwzpsXEqHSX4Z%*f zusV!vV;{T2Vhv-z{H3pc`MckLfTlLIKSbng-#S^@Dwn!A1h9kmd)j)^cacbKV;-^+tCN2g866Fa|3*q%x)k;)Wl-<2C|+$-#!@jc|p7h13~m zOtWyKhg`Sz&&1 zcuSq;U~zfAX}oiPe;@JZJq-HChY0rF;1KCb|4Y*|O>tyhobC12R}MJ-8;_6m9r{Yi zzjNDof|$AHZnrc_?GEH;WaQX9+WkM}FovR%KNN(;|UA1Y*@kboEAlb`}H=*nSMCeKE)YL7-Sq27?Txd-7+1 z1;>5b632Xpri5^pX&5*VKloNR$bl0PdPfn0yZ`q?gAoEJNQ6}(a(Oij zFXna0W`*yk5e%3SneYzh0EcjxWNr9{aX4HF2!IQJIDok&g#tl^^6(0c;CUiPeg`Lm zI7EXubPmwKh>Zvh;8%$9WO|9TPyJyGD)NBOaEYY$4w`5Tr1lM45s5g|1N(+Vp?HZB z=ZU0t4pQ_AOfia~=ZONyL#?Pz#bOK$_6}Q79zA4Inia!JmTQO@vv=iHx zU&RQ2M4D)DvzJ4^7>hcziaS($J#>u5kc=Sri8>^V+hkW;L5n)Hjn!z4-*{%%m_(@9 zi<1;a=a3OpSc;r?EoO&9pC%7Tw}d{#auGO0{0N0PL})4h@rORdb0o-yS*LGjC=W)4 zQd<`R+!Pv2Cl6w%kn+HD%7;CrKgSL$VBI8jLsOE+~`Bw$Xv_>4OoX1>;IPr(*hX5!i?__l_Hl1W7TgG zU<`wpaE+9bSAmY;cmWY`RRoa~PIsGs-z-5o3JRKKshXs#{jpG zYMd8D(rBK@DQ}_iTR@3#w`q0rNf7pVW)cCQ#xao@v7Ud?d~~E2zkyqnz@KJ+_JI>h zh@rQfxoJekP>O%y2EDfs|A&@wX-vw5X!qBWBN`|I+JF0I3@LCZh1s4WCmcM9MLcRK zK6(y_QWFQ+Lp+CDq0ubwuoHKeL&I+sCD2cWyk#j4L;y zDOqx4x-zDWe&GfpcuxrX!j|a5|(tnv*{o0YN%MJIbfZcch1sq;~nA zJLIJAz;lMir7)Rwh8UTYfGIi9YL420PPY?K;1w_k4aQNY^hc#ng{H{Hq>7q#ONOPv zf|bICm!Xvj6mmR?&Jnl%G;@W{mNmi1`&w5uBJQ zL=meJI0vOx0R?*}a&p0c6e>Y_=MZ`B=4-~rvE1nyDiLD^s}cuGu_^IaR3@EpNe~*) zjRfgzBl?fawvdB3t3EV)D$8W}#cPu$50@!-@uw0rYkV2OvzKNS&?Tokl(1Hpvjf3w zOpBQps}dPYL_DVwANOQi8LgWFRLXP?yZUM)8X8nMc1W3N8S$2XGrFP~!K-iEr>Lr= zJhYpU_<`q^l8tr?jnD|0u$ai&r(&2@u{w%d7k)$AkC&3XD-c=b6=&jAPA9o zxnQUP{rkTG9J`*o5ZoKJK5M=_guhcND8`Gt-H5J1V0-I-7KBz2W7&&CJ|>_rOAyQ` z!Vo931EI8(p%Su`f81yca$$C1n^-U0kM!09xo8S6CTL!(0jAK3mwJ(}kP$|33#MRj z!D*b|01Z)zUp9=xI;@clF$M5@TS(Us=U{4rpu}8=v|QlA zFieu;S;lvNDTagr4S^LKk2M&M(TazWkq>AJudrl9Y{b_B6G|MFRTZ%j;0BaXv-DbG zE!>J~EDyNWR_EY@?{EvZuoFFO3dmN+KD-znS;&X1vPldLW-OU%>28!D$SL|8&JTFw?J{&hjpaKb7EP4M{jgm?bcxzi=~hJz)K6eff535 zNVi8cqj5^FvYTu>G#J1P6oDs%sR_P2^uGHm%=l|$BpAH3sJMgSkWmbfDam>2Ji8%= z%%|nSK0X60o5IL7whhh^Lj!wHUY(P!r^7&zXkLHF3=QjJK!d&pZUsDe1ZkNzKY^ z&?t@2Be;*Zj6)2qjo(b1`xasJW_IQL!F*|_$d;lifsYrF!8nxB z8y&wsB$1QYOJP)y7cgC}oOSh#zi*reaZCw+b9|9Er`Bw}$#?9LVLjG9CQC~;lH~h= zpZV0G1`RcFczAice5t266etf3M8Qd0*OIY+jVziqql2BUQ7oX2EtXM{RbGp8!8}}; z{kkQF*P2DX!6|Qj`&9I1(u0wk692K3?~s;56dKcc6jOICu@{_VOGL95+Mi9Ylg*}o zF@V zW}-Q*+sl?PXk<_~E%HS0TO++*PSRc+{ zAr8CPW@zE;X{vnU04@&#p43SHTn1&ZXMYvrhAVPcp5@Gp*yxQ&M4r?&dK@o*uD28o znhr|@TBpBejk8-VsnCYnIlbC^>Dq+Ib8U{WvSbXDuxfTT8)B$@*fty2f?Kir3_Mq- zPOi)~P2nnDyODwD*!Yw@ho!7XeuH7s*kK&STCY53(gi+ye;(*?my)S>+u|cx^B3tuFT2Yz-`R{+s%Mq zwH3FXX_|@brktq#UAU3U*F-m2x|tH8I#fm_3bBE~S*X(+F64Py6ZqEN45%WF2#h^m zzEl_>jyUh^Yqbu`VZ&7Jng4-Yg%;_v5gOMm?C|~AmVRtMq2RikrI2fX;2((JM2<-R zzJt9d5fi`6j-I3Lj^V}^?~Iu5yIb#`b?@?CjZrDshspYA*ksmj-G9xtsYFY>X-@#p=(P9Gg9zwd_K7@6#$ zHy7|Eez)JivEi-E2A|Y_1u1KU$Lz*u9A|H)LMQQ3|A%9~%>5mSIluEM_g**Z)n6^6 z6q*Oa1Rc=uRB(Q5C+X_HW%!x)>c#$%Ay$yz2KePvR$O4Tv%%y851C1K0h>>sqQG+w z>FClpu^A!j!rtPXzxn?E#&hnf=||nF)vUe@5TBn<_kSsHlVACNYy8liJEQoW-$K=}&0?4A|D7tMF5Ux6u^ktwAuSq%c=8*OgbcWZ%*mjlMV>whZd4laT@D60LcT-zs$ia?MFncqd9q+ir(C-- z_{tEW!idk-o=s>oQ!5K?f_s5I&IJ(Llr6C`_n6@jO~8rp4qd z2`V?t19HIzWm}5>I>1VlXC#$KSs~t5kXx(JFWtY!8SP%)|5`<2tT*6UW zsd7Y(R0@5R`78v7K2SlZs%m7I=5!kIna3*nloC^>p8AzOp|P;WB*%j)laoG$CTgmy zzIw!lHswl!;=M5-?4kzZ_ZIukXObA_Z5-z+WORt#SAz+xYv?@ zTWM;CwNA>A`u=-|ZhGZW(93VWdB^ZDEfsrgt6y0=FOYBhJX*zGhCQ!$Q@;q zAuol+(^AM>=$Q;|uu=pxAObHXt?Lm48f&@02P=j#j|gOd@3Dwkgm|0gd@Uh=y@4M> zH9x{I2~28Etd?BdNgJcs-$H*P?(wc#u&SISx{Sf zDU3K5c%)!0ib6dTVB+>So-!aSb;?6X51ayxnZYw>hQgIx;$%Z?LB~~p2_Yy#dSZ;n zOmSIi6AMFk7mFs&yR;cS?O7=VdvXp;x(j5h2+~n>k_4j(5$P{U`cCqo^lmC8 zDob7eY7v;q)Hpc;C|}-xu_VigXP_YbWZFt74@Y&+WZ7D4TP^BTQbm<>gG*_3I@QxW z{1h@hHNrRrJ6Q1;jS`J;pSboU5A_&S9`!5|LRQ&IRXT;0Mk!`2XL*jpCN>XxBpNdr zpcOJ5DQ6K7A_OY}jmKDJWRqN^hCE`|cqW9jOHD{%Sqqxip2KW^P=a1eJ;Z>8gfeBF zN>WU2OROU)QzZ#&potXH#cZuDFdnU?@|Lv3I0e?KiK>WlNy<*h3GtP-bL5!O5k%9#WTYvE3DIxx2GZgM8W zrYLwM&uhPHcoJE8lMEn1bOfaO`1T2OxQPnM=tP%8BpdWE6T}-0kLtW zOtCoi)jn93hl⪚=sU?n2({djEUuFc?36y6NR&o3sT0sip63H_A*r>0A_id)xf7l zwSp|T#AiVDR}dv;I86fG`VvIfGA91W;~Ik5ud}=XbA#sX;T}Mcj?}CzI6^`Bq1UEHEzk^ zQxP?Ju-L~&rnG>gCI$%NbAb_VoFsq~Xy7W(;LWwSiBr4Yd^a`p9`?N}Gm!FbiY1LO zg-HZXaD(UDkL@k+y8v#A0dD6n$^LfOiqi0ak-OM`<_?c)3)pPF@RKA9X5O4p?Hge; zfC$3?9JqGK05=e4s1l&%ccEhE+UnfGPR!LUf*zEZD4RIPLUXk=$Zo1IAgK`xLIH6l zPP3#B>|%FFPy*w@cf4XPi@B`Dhu!wK554HnIJ%j8{_~)J5Q2BlyPPRpXCuoqhX3I) z3!kQcKJMI#`4uw@>P4Ez^=MXkG??a;K?8FEk{cspu8q4%5-)YY1!X&IH?~7ay=m#F ziSKl@yV2jBbi+rB=_jrIRS~}@(vQvbjc+^TnO=2DP~PhV>AEN?H@26Q9X@+TJ6)ki z^k;m7=C+=etSh}6H&p-na9<1HXTMRxA-Q~isoQ+#Kkt+4?%e%?RZveg*qmV`pIABY z{#ua3Z-0LS;P2tTe|xw0*f`?oIQgRl`pW@eLlC3M5ejjw?O>A2c)+vRCh*ak04b`v zn2`jk!1dUe-|C)o@FKW~0MfD$@EIi<`9QeH0R_@MbUBL+lEAaLJ*Lo+p@IwlLvuKP z9ArTru?ngf81pKMnW(|8$Oub;!OeLWow7jb(806dL5FEUhiEh3upqdo51kqkn~S{n zlO5?;A(|?+5SfhJBc2I#htjLeWIxX`GafeYwJ zicJB;voNSvvBASokTFaPL`1?fWJBG5QA3M>LELdeJNX$OdJFpj4?29eD|ij#oIADWQZ43<9^qz6+s4E=Vy6KrKAcC}Su= z;Tk*!padDzAdMKvmpGf%BN13ro;<+AAm|d3SUr9^$m_sJRAHxk?1~W3$Akb0s-QGf zF{n)VK$NtIMu0pRvqq%+qHym^Y^>7IE&D41)uS(1MgqfAV!w6^} zhWJ1(U>A&pN#iuk^jtK*d`gkXPkJLzGfj9@yu`y_Q2M5KRom)E3 za8OV*4h>x^Hpx(+7}0TR& zV^!8=O|?vA)@LP0X!VUtC6Hu|h@lDrQFw-1U>^M%tRoD6L{pVk|7Gn%3QyIb=}u~UDkgUR%+cNXB7&etivgNzsovT zfOXi1B@cmx*tC?`iM3c%t=NmzSY$QWjrCYV&Df7+(vU?}f)&_C$bkpEfa;7>H~2Y` zh1riK*_ai7qnV}Io2}W7#o3)TFq`GspN&?a9af!HR+A-Kp+(xH9W0+!+WTzUr`^$` zh1#j@QmVCDmSNhfb(5@RQlr({uLWDD4cnX@+p?`%vqf9AOxv>M+O-YCe)U?nm0PH7 zTd=L$xxLuC#oLF~+r0(YzMWgZ1;e=Y+rq_J!JS%v#5LS{Rowr@9oNQnTxf+{vz=T6 zBizWvT#Kz-q}|-iJ=V_k++YRW(CyXH)mqaPqst}T)umU|JzS|}-Mod}*yY>VrQN@^ zUDkEo*iqfv^<87#U78Kv-=)~xCEnIG-sAPxF4&E7VF-tHw|_X}T*J>T*rr|(r?U`^ll#ai}_-1j99qwSxU@el_^e*?DP z5=P+^R^b(9;TCq`7lz>&mf;zu;TpE#{~N~P9M<6-=HdP|T7m`QAQs{wCgLJC;v+`l zBv#@jX5xZv;wOgUC=M-Skm4$~;w#4DEUrf_=Hf2);xDF0paA1BCgU>3Vlg)3G)`kQ zR^uyX<2HulIF{o%rsHa*;yT9Te;|J2Jm%v*=Hfl}<3JWHsn=y zWh;i|Sf*t+HsxBzWiOuPT;}B@-sN7_;$N0zVFu=6Hs&QJ=3`FeTUO>~fA(c&cIIea zW@)BoB2MONM&fHmWNpUgZgysF_U8X!gXVB1=SD8)a~5ZGR%c9IXIsu?aBk;#R^@S) z=P;({dZuQ0#%EI2=Y0m{e)eZ}#$|vG=z`{Bf(~bdHfTy#=!ItFhIZ&}zUPR(REVZy zi>By|*5!$&XpGkAK=$a5f6n8O7U@vt=#rk|cRp#9X6ZO4>1KZEmUiitmg!B7>6$j< zn=WUaE@Pe^0U16VQ7Rr@SA!V8fc(|JjjDg zfCfFfg*@nL&=Q22A=s}DYodV$H%RJBl?PK0ggi)^dca5=C|FceM1gYz)PHe5_YOe-sU>j?BDC^43qqJUY1#RoM zj%x&|hg&%8p;l1NCK^ft?UnB9y^icWC_V->Y^p|VJ<0^lHfP74YRE7HsgAzKn*Z$X+}g*fMTj}10@y(C@6#bcITSj~Er>iZT%ExFFaD*Q1fJZw1Z5q6mR!Adq*!Lz>B>e{+BV3(SNAaYWDntrEA2 z1@+5pnP_8JaS6C(1=phj2S)#SNpL^RBMF~y2&YyZCmISDaL~GN?9lK6xrG;}X&^@s ze}Mt?)*}>uhXVoEMgT1rM@4#$2?tMS12OObL2v_+aRyh42WNmUzf>O|8YQR4As37y zCy*n*as^d$e*Dxq5G1d1J|~(iZ|nl;ayc;&5QhsYH;`#e7dt2C_ul9^ z=5I5Oi5#$QY!)@DrfDFaEhUzcAg^LaujM!1Et1HAG^%YKgaLf~ECex&1rM#Fcm-;0 z^*myibn$ee0V=^b#YyT1`Yx$FT9T@$z}n8j5I98`f4Fr5DfVMG?NN__QXjz0-i>G9 zjhHGBR9E#p67^|^_9wdb|3}GoR)?w?hjm#WccQ6vWT(emH}*Po@|a+N3pmA+V8C<_ z@>h}fqDX;c?}uf7l>1iA9Kgrg?rmQaQ1Fjqi;u+a6gb)*YQ zq8M=knGhHVag{9ZM$hz02KAxY^qzJqZ}tcysN$qY;!Y3dq7UddrnaH3hhtIq-5NCl zsdZlO2hPfa3UqbRuE3QCEwLwi>=p%-HF?>7f54_b2j*FDz4@HINeZw>5DQ8HCl3sD z?}sHpbWeJCq7i%oA$k2gcfyJX-vrcKlpH{Kx3}e)xJuUr^^%2Ed6sw`L;jBeTRF5B7M3ikP<|6dDspuADVh_{kJa=_|osN4|;+> ze~`mx__aW6plWhu_YTPK5y=OTRhM$pU--_?`jKaN%NPB3bMn=%d7|-k+fx0kUi;?m zeU&_Ul#lz_KX=qn_n0~gK;^zSpf85!?YnxpJvf51Yg zDiliABSgbhgHCO}odOU3ROG;S z7fkae8$oPL2ALq1W?*XIFpeBKe>BKZ;Lz&UDHD7r;rr>K#txQ&Xq){KAZRy{m=Xe_cM@QJsL2ef5B+$l^4-~1s-_ZY0W)H+CkHO1RH|M-Nf8M5e^g` zh2B|*T}Ij+#GU_8AYK=dc;vD68h_y(r_y;0BoR>vblDc(c-AG@&wA+*G?#Pz=||9v zKu#vXY-CBmTW$mz2%~8&_QO(&6%yy8gBUK?4}{Q7NZohfjcA>TL7{k%e~3Qr$Wf0# z9jRW7_sm!$L^mq=(~3N{_#;F@a+pzcWs+d$u7YTX4nL1)uJ z5oCgrhz9=w?f6uN?h~Cer?lth%taU4ORsZiS< zWNkm0w8=mSCf38#Ft?1?rDovq`dz>Sk(I|Q8Dj)4p#wQ=UBruee+=?X&m2c^vIj?` z>p=VRI84JCZ=74rA*uDDAl8zajW(G+_HcF%HoYn7K~-DLo}yK5_|=>cSt*dY3}AiZY0+_Zc;Zve zxTZl}u4irJ5lw#Se@$ZBw5E|+uE%sgUTYpUsGWlp_hfS;%(mM^m2G$0;njUM-hXE_ zBi|_dJtP0m{Y3Y6akby=sFZ?0Dnr|r0nQ=T)P=<*mdj8~P5bvZh)w^LY*u z=*NjpdOu)uPPXc^N3?bXZ6gl*`g6ZS{Oz#s9T8bFh6!f;e;?J)ZudK0@(!{*#u0=P zFgP0wN(3OuWk^eO$sYUq!99X}!+YS9)cCjt5%V<=1`$DtwMrwhzDYo0Sz93snc_1+ zWb6Vy;az)zG(m(ZY*@>~VX}4zvxgmIW~7N>u2S_4ASNqI6#EWm>an`-KmjCU;z$aq z6|Pxj#EN4=f8L^=C^IodU|SjE&pUWBCY*qATl1PC8y(n>BTg{_BGKY^_;$E4VFx8+ z(trv9|G2IQlIS~JT+g;n2$aq23W?z&&eNudyas+}hI;%CA0=2woV9UxpNq&6w?mLX zT4#{QDdabtIK?OyWkzFaPji~op_bsTg{cJH>mY)Xe=d=wM+>r3p^g*2(B0!W!rO{3 zgObZw?#E&TX<*(m$gZHQXHEY3lnG%qpA<5WGS%c{re=h-n8;CrUGio`z$ubr7K%W; z>=`ZpN03{J1)Wc#QAX-f%q)^GnC>i!Jy};N;|Zl%uwo{i3>cB%5k;PhI!>M>!hotF zka>pef5$b!au8$G;}wl)#49AWo^iS{fj3Fj{_NvVX0;P*@HD9|)mhJbswkhrR7yX? z*SG(?Rp^EgA?S>Jn9X7851{?%XG1NRzift&oJI;MOhzhB3hvWOPVJir5s9Vn)bgDX zd<7-y1INYblpvFts4UBB*4(A)Uf*afAaD(lR`(bgwQ{70+nWe9&H}9n zwbB&tfX22sqn&oW;}s8kjw&^9i@`$DC(pRWags2MS2ziP56jj$&M_$B99A!i4V)@9 zCWuDd!fTm$MO;WC0l_|_SJ*6V2n1mYRfb>~?|>8o1kqXT07*Ms=R5(qw9QQz}LCBbZ7-J=OzM zJ;C%NZ54`>taROs%qIfw`pJMjC?$^|%fzFL;NgIjWZRH#zrb2*{64DRgG^b9@oO=a zb}T*xMP$WZ;c`&l6yuD{xW+R*%9(%SWuWv}Ki=_{9&h~Oatbu71Z*N=6ApPG4TR5$ zo84>+jpqV_m9XDf+M;bTHxJGg1$6t7?SE}|d%yjyu6|Y|?RQ96ugfeDYic?Wo-n&_ zWvK}Ra=@A*&!ohN85?#QVpGi?wW%F`qfP8h5PTQZA^R~M!2?YYl6*>)%PO&^AfB6h zHc$iYE+)0I%#+jT(T7%g2v1CGsiHvIj9G4u5da*7%%Ykz8Q(Z%p2oBbnoEQPDS!Er zGDNMH3v6Hz`M9EV1@E-xPvsC{xy#)`Z-Wm6;X~OkDs^&@90WZRX4Is!Q56(8GT>8f zzW=kFSZ7WN+8LiGe!%C^sW~1;P^i0WoKQ{01;(I=!sqTm9twZ{~=X9Ti z*D`lpPxuH1-rENeF{Ih~b=*g!R|QY-s6@??-*r9xrh?6kh%k~&SsP4U3V%7_6i$Eh zbB*W!Tu%E(qV9n_Lnfq(m3V}ItciDcgq3KL2~EX8aFM`ZNB0TEdbC_bl-zCwoZmJ7 zV6`ok$XJO_ITQbIAhQ^hTy4%jaYg_RV5F=^`BBw9FiN3NLLP{M@jalHp`V*khJRF@ zSm8+JG~qP*EA`O@FLmpW%JPGG-r7IODECV<^U#te#rL@o+NG9iEnmO?V*|BVYJZ~-Jm3f%!>Hu2X5pd=8z!i;2D zn@J!v9S0&-)1agzN^ZxDc*}W+Olc_7i{T`IWZCp}A2fiZNPm_E5Kcg)a7kmN!6+;MpSB29+n zsj(x>vE|hrB<#3hKb!**a0ErWqJhdm zxL-i^&nfQM^nWpC>dhndeS~Iu6YuTM-zc75^5sNihG>o;1ajddE|k!{(szV_Vy<2M zNr-GdW@LhjB3Va4_F{{?T_$+NGeD9H)}j1;Lv+yC`$?r+G2@f7%w|c#B1GcmQ|RWn@Fqw2COZP>un4EO6@MrGNMC1?U~__|13~9ABG?sD zr(b@=J-mXjaL3xPQBU4wfudNdSj4a-fhZtGTUC=%0vwR2CBVdGStRI9EdQuSLPHM4 z6(+JKVn&jOx@2wIp>MuNa<)-Vcqj>qOmigLLwcx(3MFN1C>wbMQclT&{99>Fk%X?7 zmig3FE`Mgs=>&&rrGsLhS87=smBHBIXx^}&TvAz(K4^}CsCA?zss-kRQl^2vh8t!= zxM0nW?Ou*Ghah~+Em%nz41-wA6}y4wZ+_4ovBo69z>D5y4#w$+%BN<+9$WI?sjVe5 z7Et_<>GGjeMuh3GjA^QsDVdyUnyzV^E*T|Zr+*Ok$eU_hU9ktAo&&{j!7bQC+S#T< z@l34=N}py(bokVYdS;=rR!bOzb`3*}41*gg-lDS2ac<_=MXGj9s+-dP>7H6+rVi^s z*r}(2s;D++v9^eIoa%9qRqim=iVkY_5d_`&M-a^FEz~M|+-jsXjF`IJ?2y~A2m&}^ zk$;!6Ykqdf7=_(KBBzeR8G-C*iU!ELe#ie&OAb(iF?eAy%}ACaUs;&YYP~|jzJj;j z!e21eh(2Z65o|vo%ik%O!X_-kI&4(wWD*{qPgW~a&FGCjjK2bGj6LZ+@ZLMHtak{) zEnw^`d`w;<0?lI+#}Yrq;|kAHNf@whC^V(iR1X{MQFmiSH*af~e$lH+xNEn`CHE0!%A0g2nD?Kfg7m486) z+YUtB-o(}hn;-6lfRbukC5e;xDc<&tfqY;lN~zz{2ag`B;R2ePBre+uOyQ2*m2$-6 zLay2lX{nwq+721!_6tzR1LvBdlsfBR(uL^CXY`$k>c(3{IPV6w(WVZEAtG+iguuKg zNTR3*A9}?~wQKmoNnS!jCU60|2!9QdF2~BUEc?F0(8lb{f{OXt#Qz~6SbC)RYV!1+{>iQt*YXlEk-(*2%d%C3jru1FbV zqYSpERaW7^wWgA+tqytw23x24dh8O~#2VHEB?#xy&P~!%SyiGX4`xIcj(?p3(+UF* zMMFIA>A|T+1X0MqBfIK~l}G^B4a9rY$Bgh|?c!_1GOl_Sg?7L|5RS`1R3MMth~73w zPFP8n_S2T`uNlwD*i_{;1tA-AQ=ULEfUIMp;$IWHhZs`X6kBN(1K;kxS$3pE5WWK! zlmXUkgp$cAK{SLb?QKSsaepuQjeo46kO&VPLxdZ%?so*{5?7298}SfJvE+8Ajs9_M z26CCO2O%3WB7banLPR4g@3a!?yOiQUxbbb+L>uGCCR;}*um2!jII(~D6d`#V6qLb> z5=cIZZ!=eKaZmy!fNed1%R^ww6vER6^F{}2vwDn&2fwg_$Z(BC^M5qg@t0CrmK_df zOby>qi$^?Ni!g~r?GwbFGHecnm9$(u|5FOB=mQt5iCJ8XimU~f1xwE9Q6jCQKueaM zZpk^u;efI=Vu!-)&Z4&53D@j=s&m?~@GI%u9QLF{({NcXA8ho~5Wh!USTsanv|#Q^ z!O8Ox21RrD=-8l=m49Geb1Y{wqcnj8V<^5FMP(q_;+S-F8Y&}dY>XmMSc$g)g&w6P zQSAl2^|Y#S(XFV3RgcD*&f)XUNIpUXGz=NM#Yb2V#D4kzD71j`Pt!#br}a>?H7OUc zo)$1Twq2Mt_4Wc(|0y5PmSEH|)Lv+HOe05vAtlW@He6SZ#H_RSZMMKkrZYP;Y$)Jl8tm~w1JV#R130unHFOa&3`8gNFeLZ5*sr|k z7Bo!Kk2OcOK!1ZMIOP*xU3UXl1LH66Xn-hS*0m6r7d8iMl0*)mH;J7ZKpQb#bX0M{ zA3?{bAzd(c57u|Xw7h~yc)bHMS_Kme);Y}AXz%v^a{m^J=8r~GG^r7|clV|2;;jjB zlvp@8gGG2r^Q!A2V@G`hNwmjz6_|xT)Pe7}eGgW8!+$q=^wnqyoioMT(^#?kouwBW zL(Ix-(B;8zRZM1S^yHngXcI+rped0jTXak`tP#s#txnbP88pa4c)Lqwod4p|)k%i-5Vj1p$|Pd2IIAGn}R~s=3D&i1oBN18DAqgJPZEIgwY*oA3FY zUopV_`Kq5dPz<`1n>URl`i}g0TS=UA={S%?I<-dsRdmeDnY+1jXH&3$dZ<@+klQ() z7wxPEJBLpdfi#yq91ejRxA?+_e`JeuovTkAPJa&M_-2Vkf_ou@*Lm2LJLm21YmvB2 z+=I4L0=F~sEMZlOIwM|b zW`CpZ6#dxhAeu8R`a!~m?G`a_y^LquVXJ%mtLvcI+9E9r@|Xm5!H&tYM3mEi$FkIe zk)}jyN~>B_{S>(v(0kH8Hs-N&r|Q%H)q-JJis}5YJac&dAfa|S3G&leFa4&*6ckeV?Fjw{xA3a zN%ee+ZYmvh{6GlFm`v=ohp(=@ylK2d^stf0#63rFNbA4Xevb)-yu5}QWeD5?xRggG zkzD$_X8jhx@6+G^DO~&K^&7yF_oc}5wM{%b%MnD7j>;63;3s>sYJ8TO;4C^IS-7n3k?cASVX0;Pek4ie!g>VII0jX9AfRl2m}$9|1y($pC7RZ4^*ZXr#QLu zESXx|+7+prj7OCwi8_+(Kqw3(oC`ZpkVu3c6Q7D`woUs>-ZLJw}AO_Tbc?Ru5v;%GIkwuyY%eE_;@#TF$EbolDHK;(uJTo-MxG z$}2< z$}U_xEP0r^3-^xr8z(e_ry)N-7=1b1=KqmL;xDJ^a^yr%zl<)6$Qa=+Fa^8` zDXh>!3k@X5uD;5U$H5Rr;(y{6xbhMOGzLWb>%(GA!wdy4##(5 zf~KB+-g)OoXb!@!pL=k8hQ|xT`mP=v^MVPKz<)#1gf0sEm@MZ>Vf8 z%YMkr@ywjIRCAym*G$vQHT$tquQHL66TB;zoUl!3=nND!JLdw^O@BcVO_a$b`-G`Y zF};EkDS2@8aV|=qu{0+`=fsK7usBszR6#KXRM1CJLp7%-Gi_9H>4)z&Oa zMO0NfVZF4{STS^J&q`gj$WTacO;*|eI)#l0OlUeCD9vJx4HKzj^TZRUXQN$e)&UvN z0N8$vI7lE-^TCx3sim!M3cPoA zqe%i&8-C}$tjR80VW`d480)mz{&?-S;imZQxan?q?z$zuTWYK+>A|WC1Sv1!Jd8ja z@59^H8*xMxU)*s;8-HB#Gb5ke^8X4ezg*PJk>*_U&p~I>^U%FDob=NzHXZfV!5f`* z&|QDMM%5FiU4Qo5amUQ|+-K+A_qk&K9r56W$D8=#VLIMp<&kebYvxyn-ubkppZ@vX ztq;BW;H?Cja;D%}*Nr^p9SjclRBJpZ)nE9-n;q=@%LQ{MELff4cYg zx&Y2=fd4a~())=sjumyZ;~nvsM?LP5kA3vxAE7uqKn{|ZJ|zekf3)N!vp73>Y?70m^yDW&8A?%( zl9Z)1#3sm92E;D`6Q+S+de2v$W+cahc0a)>4?G`o$XYn zG29tXe|ZLTo%6KkDC-$dZQhfg{q*NI^Z8G322`L0^(R3Mn$UP6RG|$;Cqo^2PKciW zbDt5l=tVIK%86#wm>cEjN15qSkdBm|@FeL;t0_{HhSa40tmsN%8dGnoRHm_{X-#pe z%A4ktr}%7XPk}niod)%mL&fGxiJDZUmh-1ef6b*)oyt?FMpdR$r7BFVdR0_r^_fW3 z>Q|}C)vz`dtYsBzSu=W8v|d!LYwcuPk?B>p&b6j+O{!h#nozvv^`Cp?Yd`&JR!5$W^MF9*=~kbwA&=@T^Bn~ zf6M~)8)eXd2B7ecrHUd47tp{he%Xj5G(w=KC8;=XA&5K(LKL$7q%j(ihdfZ?8ULev zg&UeX%6AC&9IJe8bVK=!GT=6q=MaT(1;O1<23NVF_{J&8n+H);_mkhqgmN1(j8nQ7 zzViL#J4^xIR~VO+_|-3e-HYTYuyKnQoG_krFX$= zVNSl|75`0Tg+oc-JjC~u(4Yi-2kYPA?l+a!wWfeG0S!?C7re#QsDK+RN*7Bv!m8YH zcDJia3=dbso8+*EQ3+yDikQTvG%<>oeBu$?Hyl!7?Op;O6dQRX3BbPi=Eq7s6iJ_AHF!Llg70Rx)!#FLvDNejfm)$$mI zix47#)Nb@L<^4pW!Mo70w32ualyOXQ#jJHqOgW*8j)trIZm)~7zf1V(llOu`% z#W(qEmCTx`fqo^MpQ&$Wjwm6G5QL?1I*|wrSKzqBh`nd&=oG-*mnKK-rvL2{fgEh; z6`V9#M!MGSPg1(xqa4PaRmpUmd(sr0;dv(kgCf32z{}^*I1E6;@eOx6A)qPxCbJ#_ zzFa(%XYu1;cR~zJ=e~kNZ<_;H#9~}lspE1zoPIRLm-0w@@kJF)^N~w#o>KVB@ zC*2-vFSDEh7vM_-2+Vk^ml`3KN65}a$@i}10PaE0``1jl_%V+jG)jLr;PEc_O^O~N zuJ5GkL-}-kEB^4IY&_db8To2YzVnyQyze*P@Zx_S=CnTu>Ct`_f4(%^l*axz);RSPae*J2oA#bJjrBkFb8)~21Q~3 zH>m*2Z~Hij|NIXCWlI1FaQF@|0ii>bRB-o5=>a2f%qY+T@2?6qkOP5jHv}Qw_>T&= zkoh{v>xLr}HVGO;jR8!-n)-sfhM~~nP$q0pwtz4&KurpJe~**w%mhQOFY0dp@(|dV zP#&DH2%%6Ouy6`PsS1zH0sXM)w6Frvu#~*;0Kt%J4q^;(unZ&d4F8J)z|=4T*pO1* z@D7h_4r$E}fh{E>;LzOA3h{|2D2~!n;2e0d7h6rieo6@uG5KDh5#E9j8E^$T@gO=- zt9-7Nyb4T=e<4#M@foHO=H9^?jREGq!6WKU;udheJmUB^OdDbD9BK(DP)`E2@d!T& z9h)j;jG_GAVI1j98mZCD#!-}@!5jI}BYx48CZZWVsUMA@?mWpFACLYx2_F%XlgzQ; z7~lds$shyL?zS-v_wnzt%Ol*eBK>ic6tWyW;{KM*e;)6NnADIWCGPs{j1oG@*ba^I zAQ6;~ZW2q0CjamNscjlfJ zrIM4VG8QYXAsvDuH!LNRawTQ!C1cVS->Rj=vXfd;#>kSBWO9>q(h_ZQB*bpKa?O*z zp(njEfA_BP0EzM)bWi@il0-sGFHO-Qf|8Sff&pL>l#DGEH)%0DL-Rl z>@ATH2Q5=ACr%#P(gSx7GeeIqP02Kc(lAX4D24Jb12Y*7aLHJcD{Dg94x%wX=`nG$ z>2%Nbep3^>Cj2%DBNDTeRPr-9X)QPD4RLWQfBVuA|I!%;%OF28l??JBDJ3L9$s$9^ zIz0r53bI^A;sRc9lR|#a0G>!qF;5zSXKIv|gD1q8S0Mb11766VoMTsGyVM7nXJUIy?)Qj4b?mrVxG*2`| zjchAu)F7DBGz@MQ-L2qe1S>r0MO8vZk+eo-%sqL-BzH;Tc)|_*PZllAOY8EPh-oPK zuq>~yTx>M%_RJVkU|n`HM2Evfa59?We^g!O^c>hlB=!=NbWg)Dq9_l7z(g%6({MT= z58pV`$Wj9IjI9|PRV#N+_Xx5EuTKq8&r*9ZG})Bm-1JHRr87<4(o;hdB1Pg&^K>Vw zvg&3rCCc=zR&`JN)F)f>AOzL%bPw4kH4ST(BGnKS;ZhmpAyP?mHjT_7I)qa@e`zK% z)gjgqPUTcCH%a24Z&B|Q7&!?|dC<|;^i+k=5~Wp7sg+J&HCtzFR?C%7*TqkN5?4KG zPyy2{d$l2d^)M--lUNSRjI~$^bHkKX9$4-??Sm`Q4O&6YEHr>9h*JeK)Tg%9A-eTc z%V1q(k>Ub1ULB7*3y#_@05atje>c}r9z?V-`jccw^khvGIgU^eJhq~s;Y4|Y%2-q} zj7&HX0t#sDHR8fPpOch)HY!~29dPy_bXH)8<2-uuHvi9|YYOA+JnlRqbZ2OS7G<_C z>1jYb#0-6XxHLsZ*?s$ZZjmnxQy*zMhh)r)sxKP`Y?hbRZuX; zEMVKqA;8ojVilBJqAKO~Pd_O>Db5o?&(4I^7z|7p>jLva%F6JlaX@)wp{&^D_ml9@5^)B)>cbN zbyv?Pb?;=Ej(e3Ce3Mo029E)_%h>h|Dn_tim(?u)btZ0$x9&>UliuP|N!Bd?wjrUH zlcYC&H8&-!cYVVVO+y!5HPL%PiF(7AeK%|?Dm6Fv4S*w9eSr;Of7|zcdry7=jD9Jl z5h$V6j^ohsS6_=HDv7gTm5T_7fD6i0B?owu3Rog-RD3I#gPqC{*W%D57(Gw5YnhRg zjM!jnFe=1$lU#-YD#{0q0UceVF&%fkjNv@?%w&3C9!;UpU{d$O#T3x-4e`tvUcn)Z z;1*2b@$6GSzX2Nlf78CW*o(oq;;v%E@@qIK!4%+WHYC8jvbfr+ON>!Oy(kYW&OziT zB928-im4cb-?$MR`6_ESFrWbe?Lxi)BSO$|C;yI<3Qge^L~V`R*v=wA5aReham!~T zzzt5gl;{tV2bnSDmKk$oGj{G4ZUHGx!4zbylgao&q|%jPfB9{PgOZ^^6gyKi`+^cy zx!T6EFH~8M5u%kf?v+otj=$lHF<29=0t%-17CH^f$TKK)4_QsCsS+&i<_vLVcZjn$ z9LpsH-a!q~_cgJ%ljOO-NboA3_aK5GxrXpOD-v`)New%M($x8OWz5(TlcC+QcL9SH zcXkp_F_;g+f1>fYxF#Z=eR!AHIUz;joj*Fk`Wa2>8Aa|{bX7^EN2#Bs>z{9v@cQQC)L`jhIpr5*SR zA^4MInhI$;C~ewuHJGb?c&C@kOMe=oH|&#!`nXJCe<_>TGr?62(HUF~ncJ>Yn%cRf zwGCX?TKqyes?O9PF2KC3d1K|dcdc>tR5BPL1L#LenNZr&APYt^}ePM<&nK!va zp>~Cp%1;T|jaC9D;%hV&Kgs${OTCj$U2b)^RKw5ycIgxGe7!vQU>Ts{UBWE8Y<=-sy$@%qg+!PdQ+GJ_HsRq_w;_(wB7haijpKM8?HzOGGdS-Z-4T^E4@_!H4j1Ch z{cyoD+SLo%$4uI(ecds8<~kp#kc$iIwnILA?zim%Y~S|pO!q7r%Na84e;uOQUB32j zU*klfe2qH%7`pD3*Z5%^4pAJ)aqjhFU#|UkdC~OYJFfGFza~kE{$_u~=~|U~U-|j{BTAfT%-F+kL=-aMf70;V1qNjt24MiBVZn1BLl%6Og8@T=?>@dNm`9^V4JHYq z1SxT(K!Y`HGQeqY*hhtp47OwtkF8nnH)>G^#Oa z!DmM!GcYxg;A4WX6h@#l&=?GbLAW~16hZVSb%W3( zFmO|_s)TPRq2V_ue@Y?Wf_3$F*GA`c6`O7jVaI@Foi*s-gAhh2VMGBEWQ0u-Btgl3 z@=Qn8TU60d2n6MIv=ziwkt!L3f1_q0*;b@L?`UA;Y(Q@0 z;7BT=#h{l5?nEDia1tbDj%Vh?Rzg3?gBFln*g52y)9GpDmI-DzR-GT+30s>yI!0V) zC89RrrW4vo5S5}O+2=w3SO(gok&P&n$1ShCl8Jw}fYqdX(ET@@sRjyVof{g#V5L0I zw1#V{YwAQ~e?h{1iP(V;DNEf~jLMT~rS)}L(6k+9$yKT6c{*i_qGpS$JltY+tUSc7 z+Ulzy#hR;Pwf3Z~L99Xo7tg==Yucf%wyKAq%f9l0A!~a73-M|DDEVHr9LiEkT z3^d%?J6I-c5R0O;dP`7^yi(x7#bykyM9va~FG($PYW3AvTkWYJ@(FdZxPBg(X0a=& zY3sv;BGk1;G`2lsh5tLb)W|UU85eD^Jk>OsDSY$&3aijCYFnNT?v#l`dPAc(C4Ku1 zc(heXe_4|R2o|&zql_w>XWTi=T{lk@Z$)u%-<*@FAg?rT3aeEMtdhj^LQ1y0WggWz z-NJz#yOe-7bb9KIvu@hEss58bo2+ksRePW1;s-+B)K5GTIR`YJ;jsSP zOkTKZi3GS3HG)9@<0uz+jX@Zwz#|pQ6dLIP^(J_b3OWRW<#ULcI`|t!%}DmYrX5-&DmAK8r6q@8m=mmW zC2aF|k2V43P?nn8Xf|vP~JxKZ&e3 zLUS!JUH@bp*pHYBEwHpIu8;$x3Zq@ zY1wFrWvEdWo~ik;tY?ieE4eVC&mgLS01PaFg5|sh@nuW_7;L2cw{lUr#jUkt+qQAi zJpz8l)bawV2sFYNf1=2YN&uRJXah7v$K?x_G$0C7EbUaWX$1C&h2MWJ#M^k$CAF-p z-L#tG9L7jOS{>HiN_|&a&_0K>e{;@s&MQ+45Crz3QdLk%KosAA2C-l~9KKXr-HOx5 zRO(y*aexmc+S4A$etDN}Q-HVhPzc(}he>#_oM?5hb_aC?6T~jX~ zr3@_0c6Wd&P<~qR(6rvowG={xVW@6(dO3<4B5t#+8hT=h1~a3XI+4)c;_19$Q3&q+ z44WaDD)z(oQZk$^rm&D$3=N6|7uJ+>z@0Zy(ZDU<@rt8ZrkN!FIN$vqb)r%|tp$2E z1DgKy*8q3wf6%D9p1004MJsjdbVZ!1zOFjn0MrP1enrAK1L-uvz-zz@+(Z$W7E%fGIr*gGjPzGW*|{OZfK`E=AfFHY=-*Vl zLI0ut8>;sY>&@qS;57~`@D%Sg42K^GxNvEOA1D`b6W3Dq^M5(eY%~HlY62uL z0Z?S37aJ%PNX9&a(;}C3D0~8fnt^1HBSz;iOOX))hruY!^Ka)6f|`*79R*HPBNQa% zfkJ_Ne_&z|i6s=A#&J?qCO8-w4ABxP!(TR`7C;z<GdG7%^xiMQ9U8=s(}E zgrd=Eg2o*oqJ<3NSi7WuN;fIou^1W?g#BZDl@}p9ctm1`hiMZWnc;&H(S>RCWQipI z6p|7ZeBn}>At_@M87K7?QnH9bL0WN0g;i!He@x_0lc*70bcay1h7-Yw7esLvw1;FA z889Ug5JVlC5dsZihhmC}=o~h|ijxu&z}1WIA{5!>h?k{)Pa%e};am19L_$cE(DJvIV}E7xOd zeeU2_YB+w8tRl(1lzuWD?+8haxG) z0E4HtT$Ip*f5aFDsT+}FN^QYw%8>_y$P~vD0T^jR37Jp{p^?c&5c$}Q{kU?1asOl4 zLRy)CgZy|9jj)b65Rz8KQvlhFJhBtNe>5|9l9C$1kN)@$yh2*LrW@y=Z(LTBHgS_U zGbb%6MM$YAh|`flKs`G-ls)+oKUt9KbAnXyM(IWnK*SU|(P7J{f87Wqq$LV6sat_D zOUiX9Lbwui`G=R1Pi-P8-!O$R;9g^amDFPlWNB>%!7iMpCx`h<=P-q3U=sh8e?5@N z6>zDNwFXh#@s@E35$jc#r*fAtk(U>>mlMGv+#xCKfgyshmIPy(1tALh7#nV25{k(* zaC8!tiI;i>X{5;|K%$XW>4vgs5|=rYa><*Zd3+J_nR;235P_PBcxf*a8?E{OAO!=R zvWX$JSrE6WJ(|HI$0-_a@L99;e~=kjIp^SxL_mhaIgN~{8vUV>(is`cS&sO5RxKk4 z9T@@PQy)0U1);%fF^QRp*_j66lo|n*UecfTC?nqT4bNZ<2r{7#BX9$84$4Cw$ny>p zY7BdXSvzqR9%>Bk2@@Y0q7qV~Ao^hvN}-q0q7ecOn)VPh3Zva~qhfhAe=l00GKw`i zI!J{CEETa=ja8!#!lEkbAaG`)`H3|{+8{ytq~-VyH)^C4k)k3}5htoMT#7{#`V6kA z6JfeEQ<|BZa;8+OfB0siWBMRXDyD!WaBj+mQQ8@5x~56jrg92Mar&kW691=xRH9l$ zA+4jQh>D+uM5E7eSvR_*f0u@&M~b61Dyfm$sPgcjHFBty@*XTAnB3zAIWVfETBYO0XBstJXX=BhmVr3=jgM@C>(LMZ_gYQ^<&^e_E}@iiohtJ(a6|xugDs(!+Ncol&`%pn;H>l`PodO zDXVdOR$ZQ0~>h%0oO^TZqRfi z8?zKEvN8)uG+VPcf6F~Lo3lLIDJ9#pKwD5h8?JNvX! z`<_!uNwjbNJZacB}3bt`u zw@(YVVT-qRJF{iGw=%1@e*3S0ySIaze{>tTh+DITJGG0Of4KJAxQ<(`kQ=$uI=O7y zx2t-%l)JeSYq?76xty!4pc}dsi@BsLu%j!ssXIuTd%CXMuB*$lvHQBlYPz(0ud{o* zPnx?`%e&muy1E;@=W4rd>$}2Bj>dbu%Syb;`>e^!w9jibz}vjkOaG<3 z{`oWKgazzp2L4*b9n9KjMi!4zD<7JR`NoWUBr!5rMd9{j-| z9Ks?z!X$ja_FKXzoWd%+!YtgvF8snU9K$j^!!%sOe>Qx>IGn>eyu&=)zesDaKpezE zJj6s?#72C?NSwq}=2+&g9(4<$TWKjLz!3sq1XZ zxeU(j9M7se&hi|_^jy!>4A1yX%KE&|pxn>?%+9zB&;ec0d`!^cjL-&6$qK#DjNH%; zz0LO=(V7a;bZpTSozc2H(G-o*8Xd?U{n2?G(jpzo99_~={Qu4;jnXU~$0Xg%FWu5E zf4$N&y~#0M(`J0r<*d_X%+rHhkQ$%?TyPKAYzjGW0U9t2P}~Sm@FI`w)4g2C!_WwM zfI9WiHqbx`d5{O001cXSG+u36f^abg3)Y$h4Q`;+nko;bAP9LdH*Hf7x8T--a0^6@ zfm2=8=GHd0wAI&UkA3~t&|D9%K-afme~*N1*lv^6Tiw;RCDl}I)mW|7TpiYKW7c9# z*4K*JG?Len{h*0m)rXx)Xsy z+|eD|wsqU%{0v$x2%=EiDec=;{oLv;2=^A=dEnIN4Ao#v-M^Y5P}+T*U}t^ zq5#D$;RZ@93Q$l6{_V~+y~o>uA*L`RfJoqO(-LJM*4<&S6s}1L(ccg1Le2OLGI9&+ zNJ|ra4yS=3nJ^=n$Pl)*;=ElWe?#M(%|3{#z~IcL&4;H?d75Ye=zV3GV%@s zq7k)~&SLK1GUDVyq2-x05)okN^G(y3Y~MaUA7+deIY0@5fXyut0acCZ2Af<;3@CIi z#hPBrZyaHTPqD+xkX;J}4LPMqi;x zW8xAa7~38m0&g_znKbRye}31n9s#ocfTW#B+@47HQRcmJ>zVZG;NI<0LGJlt?lc1I z=ickTPE@L$?z7%(+pg~8J|n_T>@xD~DywlZqJ|=9>CMKJ)*d4-lLPt)-ZO&jHxb;` zSshG{?xLaY3_$DUuIOz8@4U`j^UmqmZSMkaTlp^k?gk$t{oZT>a@9k~_@$M1a{h*HlZ)1!OBR9Y9Fw*iGf#)*cRZ<{`|(@M%49yFV-GmAPFPk0qC-o2O+rYG_v3F-fU^V_A^osqL8v0kpliM zBW^!ZFmeu7B;hb9f9dYV=56CbDS+Nlk?><8>80rK|Be(we-xu_Geg}E^1vdvP9v00 z)R(U+Lf;x?-w$Vx-n|VYU*8X5pD#pB_G&%)eXrGW?>2jH7`6@jr#B;~pZX+`_x-T> zmXG;)UiWW=_qJvDh_CuEVqN#}4$MCg_NXz$5C0=)2?OZue?mc!NjPHUF!Bv2?(6+< z57-m_%0uan@AY^A_F}*Kpx^pb-Vbqq`?4Q`-XGq!Pw!ZM{-~e+w&nYs4-gFU^y^no zRHGaW6sEXG(BVUZq86SbC`ciUhW&;yhyd;0JCN_>L4>%msPR0 z#S#@;*6dld56P002hA8Xp(O9Qg~qiW-FjQ!5zPuQ2m?20-#s(dm*(BKd9}8E*6$Vr z35FfcHTLWsSH=*(RgO!!F3q-YL)~20lI$(}VeGEQ2_lB}yX@e^X|-=xTa@6t3k*_N{~IBzR0f$I zT4aDz`vnzS(_*Nc%L{NR%h zBKmmxs>k>UWYI?R>=KbghfHjUKmTf@3BryZqL83QR6Gls2(CIYp$4sl%sLNc%8&s; zwsPvlp}gy-MlqAj5ytj-%uhe9f@HwHuK$Qse@H$!9V#z16qj64yeIpE??kPrl(3&G z`^l2YuqGwbp%f{7lSL^BLU1Ne6-pD*2zA_ZMmGbU@Wvz{E3Thb!-Ob>tCHmIR$Oxh z(9MU=dB!sBP9xMIVNF|2D_?;%Yq^|Sh3|q}AQGfYg(ApgDf#k%F9J{ll;9Z%uL_~D ze*)o)%q#FBpd?&{E@0CkZEFRU%LcP#-~wukEXa*~pK6RCde5;}gN-DB=BTTZxTleP z)qT#{tDyZZyj$3-Rz`wev8h%{Aj%}XtDtz5D|Qt+x1UTDRVV~L>!Ed+Tl%w=tJ{c$ zxZP-1&Epl71tKV3`1t%7K9Q$;IcARkf1aV(;+wQ$7R!E97{=qCS;q8enfn5oBT&JD z=C4}aNXn)NrWRC4U2ml^ysjl>HK>QU#M77*aY_bQtbLf)M ztq?T(>;rc+)$b<5!{{NWda@UCn}lQUwaXVo6u}=^O0vLnK?m~f2u73pM-NkFe;cAK zlWaBhxVwCf?!xh=N$&{wJ{3T~6B2w=pC5{S*fNLjNA#@TA>8i6J2le4+A1gLC@8qv zoTP-H$nw8)`fFPrV}{9#^sMm9Ij87tT`%6D<6(u{q18PNUU!ms_Z@Pl?QeJm zYFqsh$G?@_$a#f}9#;x9Ifq!Ue;@zY(;ojQ=rjZ_Pl4e%m-?R5zE$M_A`9A|7P0shE#_x(=#hY8*b@Zl)i5kH>KhAh z^cO~@NI@tHkNXICkMIDne=GGU|0c~41xv`JW+x0mkPzvl%mV$$LXGH$MH<-;2ua{F z+w10yM(HX9K~5@oBwmuz36XUc1(>xFX8fAe%ZDr_N%MS2J&CEji3tRmzY7a92Z&FH z=<|#3d`LV=!cV<4vnmSVW6MpSv* zB*;ZGy3vlB3O82OC`W(8P=`2_rCD()Odk?2`#|VNdTXkuQkB%5zJd~uvLmC^7oczI zQm=cx>Gi^BABa}Of1G`sBK`lu_{S}Ba)MiA;zQ`e1!xQ=TrZ-fDc%7McD;r^L9om# zK&BooYTy=^wd4+;af|RIVHmHt6iLdZu5+BjV&2(oXFr=o_=G?ZjkrbZGVzMWAWQ<5 zeTK7a8cW@pLW?07#ygxMs@T$JKpMPIWWl8n^}Lo6u|?Q)e=Q`CZ#b7)MKJ}=_9DCv z+2t0u*okUc>y!vI;vC-q+SIxj0XJkKrjFEXORPKDRsAWl_F~8|hLI4_z9K8-z1L`s zfnMTOE5GLe8EntdU79cpJ=S{1IpS5ze!y40^u3CGkBDFW*7hBq#qL*9!~iHnb{HG0 zsIvxyAX)L%e}=p&2t?$@|D=RK@N9_7CQVv3o&ptBs_q+dE)O$8_~GYWN^L7aD11WV z1x8yrMq!fdHQg=C=6DQVU#Lzs)_#zA0F6L$zbHZqvXlZVN=*5TY2w{@sXFGij%n1N3s{!Kb$+uqv+~_MgjW>f z?FYKm&29!m>wmoPgHpKPLE^^$I%Fprpo&3X!|X?aEim8cQVytcWonu%F8L!!Oqfha zkcARG*&{du4DJww-$RRC!>dp#&SivgbK}Xgsb3j@X@3)B-~Dc8xyno{;Q%t!P8kTT zxI8GZ)Qli(EHiqH5aA%SRx3JRIi@5EwyrS`g)4N3&3{KTsB#58pbU7qty}jHX7dVR zQ{#C<74q|_E2iQXw=$=^!r)i-0~!huVT3cm#!z+}mOR!B$hhh1CJPmyVoheB0V*cM zh2=3EDazU*vv6v;4?mDAvZ-~Hq6k0bz%qj^r`rQ^Ed+X%^2Hn>7C@GhgLj zdXF}ohf{jj8`|}JuX0yhUw4XjrSr5tedrxx`&{F8Wx8jyP$K^fq2zw|VcI^ibF%pv z#G3*M0;s}>OtFr7poBaS1;ZPa zM2G+yDs+|$c!hHi1(j(Vfawp$Yo@;-yMM|XmR6J!b1N^|o2iGO#ngBwTG2qzlb)x! zMMcbrU3A1u1T;zH6Hcr|Pc)$xG=*UN4rs8%OQZ-~#JpPbzEo_hBEt_VdMo%@Ocp~6?Rn50C?`hP%>DJ#ixa-k-}$}=gm_IpaT14^OPkI$Q@0trcy z=#i>Cq81dB*C5JQED&Dw3bBL+vhqK;>|pqGZX`=t0!W7=>7$EMYow9FGL7KFmazXk(B;az2bR&I=Jp zKl{w+gb3!G4vKkB*R;ptq<>EST$y{oPRP{G1FDno^G?m^kI+-l4?+|IRnJ3XPci`- z81TYh?1$+*POZ3ySD==j0MBmONXslyv|yI{tCl1Xg}5j;tVm0#Qp=Kvn8(zQ6y=B& z^+~nc7(Y75Ogzm?nmwmnO^Bm9k~oUqoFfJhh2PXl^q9Y1oW0o#OMg6p1|BV}vqY2> zdAL~GA{qTc@&9W_bMr?qwbH|KN#HD}|LD>?DFeLA(!evo0!a6 zyb267(Y|tsOsEyeF*NBrD)bNpn7IX~D1%|h3c^wde)CUaG^|ac3Qd9mDAmxC;euP} z5^Tb=BGXOXR7@H)kAL4xO!s>T+cVVg6g`p{)M_bI)H~GSNz_Gc)JJujOQlqTkk!3` zoj}q!$W&EBqf|0AP@sa0MKy+)kbqa18ur-1?Bt3b*f#w5QsNX;Laj%@yU(x^m0T5& zU5(Tc0auUdR^u7gR4veB9ZckOBy6OKd1_Y)rHN~u8Q3($#DAeXVC|i+7`(u67E@TS zVVKrim`p(bUD$2yrZhS^G*!$k-HM52syUi29Lf)$ESDUh1Yj{VqFbI~)5s-}Hfm@$SoP1^93Qv@ZDcc_P| zwbSA8rjt$CoE1Q2B?x%{2`~aRhUG|^`jh>N*lcsbw^dRGm@c9?+{zH#7Bn=(C5q*{ zR*q;#kJ8YD33N7~E}^Uy(vU@&D0N+^&L^e8hxiaWlY z+4x{x(+!U=#XH#bwZ=_cU$tG-&E5APUEbYIN(Ek!&BLFyvh>Bz!`t0vg9zqz-tiRA zcoe{=0e@e$2mylmCx#%SEwaMB9pI~Dj!fW!e3=@rT}*%q3D;v24Jxs;zVP_drjtR10AWfHVaRk+z3mmrA=ai6+D59G3@%DI z{fO|0H7F^ANJJ4z$f~ctRi0x-TV6EGBYueC z#XG0}U|}VDh?t1qEu7TFf-#vvK89r$r;q^Y^(!*|6vd^?GX;xeeOH>Gj|(LvE+_-Y zIg0fy2%Ctt=k;JcH7Gtc)%LJMvPwsX*;!phUEje3 zAHfzZh&7=kLDDdWv)~q7bVO)^kP?%?ZFXK`H<>~F!5g1lxU7NfB^AStNq=ei_Gzwk zY4rve43^a9f(A|MkPVwFQE*F>%YpuuE*}kLQqcfWU@dg1uFIG4RVm@Fra< z;ZmvVQb3SyDv=&vg1M~!Zffo95&3$DbEJqiwk&~}ikuT1_}Z>-uox2~(#T|D{wDFw z1l+#lh?L5T7Vj_u3=xnm2*_7Z$OyIneq9uaTBlb1AlC(D%{9oF(U=fX8#ko zbzTnqFal$E+Gw4+JcurnxfCU4J+sDfXEqf@CG&qlm?ZEGq>)er;V@_ro>u~!AbFI9X zzVtB{Td!b`M2~dIeRJ{Cb3V`Q$c?Q&@(%XUR>2Z7_K>${FwVAX^sF#EfdE-8|0(g4 z%DxKoQlH>S@ANbu^&%B@|3vmoKXv|AbdXu~S3MI~$HRh*^%q}Z?4B+Lv#?o5bV$GU z?|5%uKl2p>83iGo)qlEKr@a&~|GH#PS+ikJ^YQO$ky@=hZ+4c$YN3SidNAbw=G7d? z@(N3zt4Q$`AK#M$*@qwUZ)XS#QI~=*?#sj}C2hOi>rte-qq>mGA*~j=-8Fp)`8KMH z44;`bx}%(#J-H0=cMy>|%7Gp3+dE;CkB1N6USh`zBS{2hxPMgnFcP7JV3f`yEKKV8 ztH24N_Ez|W<)kqsp^w*l5_zy-usi7x;lVWUyjPQ4j_oz$=gN=EeM>fn%CoLpN4nUu z^U|(3CCHEvaLx}<`KY^x4^XLxE4>N`GB&^8A|X5bpy}5D?NtxHleDjFXn$QLHDikX zd#0G$c}<`@X@BgnrX)IfxN+RV%>N8yGgpO+X4|y%vd;*^FC}Le+{ah&*8d#VWupAy zd(DUMJE6jlBBfatWWuMtzA!XOVp zXfP-;2yrXV#1!>%Fc1W?pGg?^6pGTo(&f!Y5?JQYc=M=2lWuNi@<}noD~0baFldDI zDnXMd`+uqY`7-7`m^wjYSo)GZI)y(_NFnT3L~=rB;rtE)1OfZ z!nJx+UKy4Q1Y$*Mj3&hhlnB0>NaUs{u#Go!?(EoTBgKmE_Iu@YBG9WT7%XiJcd%!I zt6N`nV|OR#B%)c;KF5V+1iG6ANu(I$Z%W7$C4Ym=d=exyORD zarT?0y*AFPF!b(;q{lzKg?0Ak*Rd7CKc8N`cC6pA=LczPMwFKf(^6c-K4E z9a-K;KDIa}nPr+8TbAb)w_F(=QW@n$T-*{SPbQ{TCwU;XMNv_Du6fXyX-WmnGfxuw zPLj~nbJjbHLNgI;1l>dEGl-t)nSAvSDu1RzlRi`FrJjO%DQE8xYUrr*o%1QD1by@A zlce_ZYN4`TWooS{rFto?vD#XYUeLjcRj)0IYU!A`Lh~!L^9kDGs<}2hEwu#6L+G;e zQHtzQxVC!hrut#KsAtzcqbOF<`udrf>Q?*ieCDFd?6$-X%dDp7a=Y0%;a>Wzs(<&w zJ8)z2lKX4DR}~xE!blk$TffhIi)OH~c?!+G1Z%u8#Yn9R&AP7tewr~z(SB>Pz0%(F zambtjyqQTF9Ndo)6RlNk9y#c|GtW7c!6L^&3yoEDJ@*Vl#b$~EgV8+qK=RPdz8rPV zP48Sr)C5nRHP>Bx-SyUAi#;~kzkh~ZHri>c9pc(;M}4;2vci2+Vl9aR%`hf0ki)=U z$_RAafn)W|pSwkRDJ3HwG_m09itYDqGd8~2+>={=IklBzzB%V)YTh~Mp=0&A=$w;Y zI+jVhwwh=XWZU&SLmiGf?X|;uy6s-$o;&Y>@7_D`X#XBO@m~*L{Mg5bj(cRmbeu!Fqg zAOb|*b@xIZy+jErDBWBtAe zLo=drXl!gF`qT)=(!CLWb-bM%=cvb^-4S?w++(%=2uNlLa*(zvq%QanNyt1JvXN3_ zqz4u0NCHA~lD?~?C7FmxBues-dHEzLLn+EplCqShJS8gsmwrV9FA-%jvzg9(CN!fN z%Kk~Sn%2CRzC{BW1-Z)p$)lItMFSvz)Sr1NC_)pe(1o`1pbUK|L?bHEiAL0+6ul@$ zGpbRFT6Cix{U}I7YR`^}w4^4TCrM8VPQT7LwXQ}Ls$KJXQo8DOu6T{=S^Fy3!NQcUgdM11 z535*yCU&uoRq9_KE7^iJcCrbrtV#oG+0AnHrjhOJLNNT|Ur z4F~`s`2+#<0RaFk00000t_8pb0{{sB0Q;AS76V2B{g=-c1GNH+|Cfdr1G)iDmq8c< zA|qx$@WAuaQI%STC?vHk+GnJsmTKy$sJb_)r^XQD>Z`EE|Cfds1K22AKY@Ae^7#;&<2`|3z zQhPO`@c);F9s~9Y-+G~JSoqt|zW022R{Foh9Cp#AJ-3paC#yJ zlG!H@Zjg@&d3wzT15p3)3?B)TJSlwrmyaO>&jHz&Um^n<4Uhyx8Fgd{nt9N9g8!F> zA_G?e*O%TR17Zr*4^^53f>4q9*Oy8o11=7&*ALqHY+q_!WLXb+|Cfd%10VtYm!~5G zK>_`j=_3Op0sWT^Bm-&>nn7R$xA3mYDUnbBgPi}Dh9m=@0o#`*B?C|pixkIN4~)6B z&|tgCYMGh;mxd(+?-5>A23)YSJc@x%7zBk0mFNGLh9(0<0sWWDCId17{g?hG19la} z7+405X}iu0s27G}7-pSX|E!Pwmx(6>Edkq?w=Z`uctEIxw)_gaYhf_(s>6VRum6{ZDFf>j!5I&*20XM4Z@)<{@)u9?C|>c> z|Cfd;1L~J?D+3h)|Cfd<12BII5dF@TfdS=JKT1T12PUVN!u^Fh0vfqzA)>_r7rnlEj!XBRTSXt9ULC3^&FE<$$8G<*R=(WBVzE<|fC? znNW-g4Vu(wU!q8p9u+#cu;HSC3+qfg7_eWYgc~!C{8%z&#EKJZ=FGS8-_MsFPZr!6 zG-=dB7h47#n04sKpi93No!R#PU)rlT>#n@}XztOaZ;Sp7JN04Jy_E|m-a0sM&$T&s zEb)0&^#U z$_RpsZ-BB$B4d1n;+Cu&GXG{rN-QG6h`kW(0g61@eDj^f`gtZIkzykAO9HD~4LBs} z%p(XQ;&NlvGPr;G6U=kMa^szf2+9MRx$e=>MPdf~i^4TC2uezFP2Jx(CW@t{ znqp|PbEGQ6n2X4LYA#cHI93#eaOCqGXwG?uk{o(?Hri?B$<)G}XPOk?Inm|K=yU5# zP0piz3(sjfDf2iw=#ozVdb#JWX8P*rrdIN`ua|BbYuU!$`fTc0OB=PY$v%6uvbC;S z&bOOeo5_E+braj|wm-hx@4SoJuDiK*GW>AF6IYzidlC`hJQ8M*eDcZ15JL_=Dv@IW z3C{3*4A76jVFl9v!_RY0IB}=&#QQPEDpZp&LU!53m8-%FKiaTEF63cdR}rylY53s{ zXVFEFV&vkW*8RMZRv5=M#w8U?`(J~Ld7 z6f%rVz<>+JVBJHm<^L#1xh_uwEMOTLu_wdn%1UK2)5%)GKp4i5CwcnPJI--Kn!zLw z@;HCTjzpmpc@RVp6cLI%Ug0uCi6VYq;fg2_l(DtAp(Wn{76}a~2o9Q}U3f9mlNwPt z6-uR37y(lUWq3yK0WW?=cwhS>-+(w{a1Z4dQS37}74t^M>gJmRP9s!BY zGQ_7_L~JEK5lBMD_&hVBQIdz#6;qsoI5B^X=s`Is!ybEL3<(K>2uT=5Vu)v<9$2vq zf+%AOHI*lsOlla6nPe?(c}ra8QkT2rWiNgCOJD|5n8PGyF^zf5!F?!hnn|H$`cVfF zpoenP6oD8rR|#zYR-y`yNL@EM0SQQW&U29PCMU*0AXDD#E;)cfJKZ@zG9>{~xRZa8 zhB)-6I86*e^GnoC{8^aey-0aP^Ng!RL8}9~EE8M7NRveJMX4C!7PJhUb?hXZ#2DjA z#t@7JufoG-rShHdgaWSoM}$WBZYSHJmOL)f7G24U5-fs&%ZkLDzBH3glq8h{L8X*) zKtY0sIZ6C(K?xXi5{w>n*F75XC=P!b=%_#<;RbWXr*Dx7BT1C~^$?a#li&310206nSFK+?&-NfAux9Sz|PQwe{@pe_bNQ+udQ&>@|xi`MC zfvt{qRLJBR5US zQ6UZ2%qBMFz|C@|xSZplXdO32KtYHi6a3>}yK>M7mH-V3y1QrZBCD%?jx3FAZIM9h ziMV^j5mmPTtp_C{ghip|!6jCiBNRB9z+--oe0!;5$?9p0c9mg4*&2U_8?5q?M2^Ul zx*A@)+Oi#%aK{*#h!G*P{5_wdFoI>EV+H-U!A!6PV1`>%4;uB<5W-1A&?r=lE=ft( znYER7UYtB=$iXh#<*c#Ffm&N)D`nn89;P6mTl{p`AkYtHCT!mx6)lf%7~>X~ZG-9TVc(WN`t@rsWAj5VqQdo#`MZ zf$d5<*hrUALZ!n_Z^#~v)jVj=6z}*m3U6{mmB36THc<*(8u1Nz_zu>Alo-esF{*T;U66c*7kI!(3eROt+|am>XdPBYdPDQsIIU=SM_@laH(| zIY%RQMkWyNg3EY{VfBW74e)z{<{`0SNhG~ArdqBg((+tBE$Y8mSD=`!RxsdZF zU7(=FsSvvuiK>%5D>Jl~<1v`~oqm`(k|Q$Yk%zT-Ae?9jGn0q}(iEvk0o;TFppmq4kbI2sz)aWrhfw{Limu$Yv12?Fr58EDuRp+h*c z;|oxd2vK;G0FobE`-bmJf<`cgcX$R$*pZ2_yWrBGS9lX>h&~2XoVl=>`=c~V3zAGY zHvj#(5|pY3#;dkON~$M(hX-kqkr4>1AQKL%ixz*OK?AX}Brt_H`36z2vLU(%BFFz^G70vw4RD4>LCQHvJZjyh`uXpjeJfRL*QJq0wo$r&!l13@SjB4mFNG;uwWG)cc=xl5!W6yk|yVGG2H2u_Ns zVoMc4cu73c6-478|B(onw6Y)=v~rV3WIB}~u}G)f5l#w}ISdNAqm;apA1?5uemEYb z3XvAEs$Iz$N;%1PxxE_X$zsWwMp+y-TR4l5A5)+xmq03Av^JX*gyR{-gDFK1@}qxh zORYi5Hqk1+8!4F9QA%UFiFh;ur{sbpNTr6M2o0%<97qB;z)Sh5myk)j%X~?zvc0i% zgR*=DdP~Y}JWbS0P1RgY)@;psh&XQy1dGEs{~O2u#!{*ZdOI?CGQ8}?&RxMB+ytiX1XvhazK%L@}K zvatbbClutU$Wp)ok|@>@k;^kogloD+2vt#SL%D;*TGKFB5`|M`vu{|glN5wnV~H7~ zBLh)YL4Z|Ch$`kw&{NGh>}dp8Z3Idn3kY%@Q56JA027lK21*!Krn7%lU`5V7()c27+>r_hV>Qs%Rg)tRsUhV*B7dw zdq`MtZP+?uR$xU|w}hl75^-86qv+{X1^5-wpAK4BD2VHI9s z7H;9hdEHyG2X*iR1Fm5**aJJ=*#$-c9#Dd9LNPJ00U^!;33g#4K4K(JVkKT;CT?OU zMq(Jgr5U#28ph!O)Zqp0VIJUtZ4!em-eMbIf?b&4CoW?%K4UaaWB)Z?V>WK%Ns3}y znqn%p%}9TMhh&h0+*IHeK!Fxe0zdv@K;GgZ2nKl|V>dozL{4NyUSvjYWJjjPWQtii zp5sVRhkJkra=_zr)Z;zwVL$%lPyS*XAY=-DWK%w6R8D18US(EZWH^4~Nw#4{z=pzs z2X$bB60-pm;A0f90Z{&BB@ks>c!yGMWn(^OWKMr(WnN}xR?SzIBzcHs1D@r6u!ohJ z;7L#dY~q1kZh=k)W>6MoPk0A=XxnB!XLL?ybzWz7Zs+iMW;BXs8kS{Rwr0eU2VBS; zCD>zc-epe)=U@ioafSzTc4vb=XoOB^gZk5rr;cf+hUutYYL~|9td?r7 z2J5XBYp|~Br9Nt^#%i-ZYqeJDu~uuHPU?TK4(q9Q>bjomx5n$VK5Md`>$R5avX1Mo zhHIq0Yru|a!yatCzG}Yq>coC(s(x(2HtW4cz{fW1%U*1@Zfw79YtIhstj_Dow(7FR zY`->b(2i`!KJCvYY_wMF${uaeChgWP?9S%w)VAx|-s{FrZU5OmY^~00sP5{`mTG^= z#%fd{diaDJ`01biXK)r}8(;%+&=9NEYObbl z1t0EvRvZV1YzWuym1IZ+H@CL8%!-;SYPw)-5@C(;)`!;b5x9|{G@DHbO z7RT@wfAJNM@Dg9}&{lC7pJ^N~Xc@Qh8u#%P&vE^(aT<4V9)E2ZXK)7J@k%rDCSURo zC-Nb8@)rj296xd@hw>y>@)$Q~DQ|EqKk_h#a4hfe&>r#sE`M?~PjfBbavOi|-Z8Io z4&U)Ir*jTpb2rcPIKSsTkMcS1@;V3fJ7@DeNAW%<^f#~YKTq*RpKdN^^g@60LwEBZ z4|GV!^AnHp5ohux=X4aO^bn6^=pOabrs;8%=$w9UndWa#U;+=&UESs3jt*o|-s1>H zaNy2$(ynXh{%T(T^~GI&I2ZQrHh1<$cWe)L_-^+2UTZ}s_25=_Z6E7;&-T@pcX6Ng zd1v-<=l8#+cW!rgYH#;{CwRrSckBjux5jtMcKFq1_y2cqcwb+5ho^sdeaHAog$MbI@Ar+@c=EPwgdg^CFZgX2d4U)Aj3@b%_ji8(c#n_vXlHnt&-q=S`3&oM zf+zOZjOAQVhj`z3cz6eE@B}u%Q#?HZ8=!SS7UCArT{Z~bj#qez4|kx?dYTt{9anev z_WBzq`(saemjC+nj`@GE*Y~lncCpubw7>C+&-syWd$O#n<`QE_`Osd$nJ8zi0W7zxlm?{L4RgfPZ_0Pj_8^dCS*%uBUs)Kl+&$ zZfCFj>mK~cPw&mId&oz9lyCXaA9;yye6I)lmH&MI-Y@xq&wPKn7yZFEeaGkhwfB9( z2mZJ}Y*9~j-A8NZH}y_1dg!lwpRr|Yumw+G1VSJLHeiD+2!HVx|L@O&?`MO7e*UC} zZ06T}y~q6ZA8da=d*_#bYG-QoxBvXd>G%f-dGhKF6iBe3!FmT5B4k)lp}~j_2|8?O z@nOV)6f-iM2$6rI#EcpRTAWC7V@Z$+O|nb~vSmY*Fc&H`i4h^rloxMyRLL`>&W}BV z4xQ=Jrp$^#Mal$PvFT5qMx{D^=`w0jrA3)ugnAHb*O*p`RwWCTYR;`A!P;ayR%qFz zahffWMXTQ|FcCsJaAEtlQzIgnY zr-QCOlQn;I5Y6YDditppA8z68N1j;-`gfFr4ED7iX^&xN9efWqs3C#?b~qA+)mgY4 zh6!$0(s&V47a)hyQ5d0$2aZS}iv3C0;)x%gwW5p?!Z@Rc5`~!GbM%?UrZxgn4F|@Tj>an{Bqq8=7&-IVYWU z+Ic6JsX_xQe~KtUsjcQ;V@!S-nT(p(-Sr zxw@JqldmoIC9Sq*=_|3XDz&SYXZm{Tg)Y`gtgx_FIc$&14m;kgqC!g|x5hqLEVI{I z3nZ8yihYOL?cu|7O?uT zXDxQue_Bh7?btpeh4$JJ>x*-Z$c7Dc*(84L_G2Y4E%n$66U}ekYG+2a*KzY*x8J|A zn>O2nyWF!DsZ%?~7&7AwIx!@VkYH;YPmyUSm_Qp=| z{lP zD8UI*u!0u6AO`=Z!0u(RgC6`K2tz2s5t6WkCOjbtH;BR&vap3Nd?5^DD8m`ju!9Ow zS`Blk!yWRlhd%rv5Q7NA8{SWdMm!=Clc>ZcGO>wH+!JCE7x=^}Qn89wydoB}sKxye z5gOmyA{fIc#xatyjAk4oM7{xybDXh_ZhRvg<0!{DQV=5NPz@dPsK-6>v5$WIqoTN1 zk9U0I8Q=IKAQP#`MKZFHj>MuMRi#GNoTHMKyq9K011SOjmybmQ8x%v?5AF%%9ln9a zGobm5X+oo#*1VU#MFT_|x49c;s>+$kd?-XCD$$8jw4xThC@8<7(T#GK9YzBo7j&c; zy(vy}s?(jya-j@uDAj^0)S(i!msUmtIT8PTD_r9$*SRuvHw&E)UGu7!pGE^48&|dK zU2F2l$x^nmmc1-yJqTI5-lMZM`7CHdm(@lCK?+y9S@|@v$(}BChv)3EaR7+M+3E&R7e9m0sohe zNCT07Pp9SiS1g>BE0bdH_fCXM)4#0p0kl+0MUj1=F^+6cz836Pp zfCQZ1gn8fmVIBjd02c^CCOlmuxIqs{00~AP6hK1+?jR42-UKp00#L#;7y}U+!w}vA zC6qxBpk4>M|DNnc*fXTS4FX~C5kMY)$b*C#!}q!01(d-pU?F7H12hOi4s>As2|^F1 zp&DA<4<>-|F&HJ}fB*vD1w2E9Wx^CjfCfZ>@TGtQMi}-*-vxx8gFQnII3EEN-x@9= zBMRLQG5`uX*gKTL96BKwHW)M*!4yhB9)tiN{+&>{n9z&k(zEhaz?KA1e@fG$P=^|9XtxE~bY-;6!OExdv# zWI_zy!Z6(8ggGNLN~0*eLN!8IJz&x$p2ILiVl%qJ(k5P0F1Fq(CNY)-%)}OJZIMyd#Fa10`Gl0W{$N=D-&^*f}(U22>{W z0bwG(|H52C)-4bK{4qd(BfP^cFd-<0KrHq{NQQs{j$iqSW{jPKAh2c$5`Ym5!(2W? z5SV8C5y3z1hT~Nx{9OPguqAUgXOg+)DQe%61PDTL?#45J9HeDBUjlL>b=uW1 zY+@!*f(^Q#YueR2Sm%GbUj*o3wRz`JJ78)+iU`0Tk$er3dEch}DA;EGZELC5I|# zS~6%J*k=laU?Y?%5W>KrzA3Nvs)xntgUuVnk+B}`%c zk*9|hYVe)ug^5BXP9Hk%=|w_jKQJVKx`AT$fG*bPazfUoLZ1defgnHu7wDidgn;u& z|3D^uXg`#H!MR>Qi*{HZ;A-cQtBa!7o1Sg53kv3ZF!3 zD!^WV0|M=P3hcy=E!kEWusWDAylRCuLb2`!g3h1Uf@Ic$A`vucJv4%};vOctK@vcL z-~KJ&{w-e0r~#&c;Tm9fKG-XmY66Jube+R>{++gYJ+3qgyI@sAZSlWi? z{Y@7a}JJdHnA>*QR4F+YCS-Ka2fys z4DWnKUkE&y2DR{w{+BFP}ETXug8lN`L{=CA|t?)Y{eEN-GRRFJwvZm)0NbdKlKy zq5+)jf<432z3?+s-RP<=^D>wbbYK#wD?%E;F&=OpfNbe9uw5-{@FnYqCF-fN|1e#D zzVOqDa6eq%gNi5rIxrPaECer@<7sRrv@Ku;AC*p55d7jth9Tan?4hRZmzwDbcNoKd zq6rSF*~+E|YQR=jXPJ8BC9Z3YDr5@>!4yhl5e9M*Lc^>Ep9VcQ17#9`DzCC6 z?%gpAM3^baafPNp9!o3qGFTRyX$TmuFb{JIXyDB@DTY16)0W@qR@mKIX8+Z3Bu}%O zUh#CjLgZE`c~Zh8cOEedGYlZ0^AW*4-a-c&z#=~`B?tmLx3k!7@LsMlhmrq(?Dt|A zH_x9#LfA0eX=_5J9uoo6I%x=CX&N)Y1xT}nk?j8}qKCbM{ze}hQ?o=@b2nGAU7cMM zPUSu5>-e#rWx{Vdv!|)@Z~!-$x4ND@x1H>kpG7a&Jp*44Kq10bs0PI1KL_+L3!g{t zv;-h@g();B_Oge4gX{{Q|4lT1Qn#r!Lzc%%YY0&1!4h-{W3G43G!TYBGc#Ds8sjUk zv@BcLt$4QLW>4xd7f5qXyPgIZpj*E+T)VX`dKgews8DlQFMc#%9%)kdbwDol zQqG|Pl)*WMwfn^%V#{Cj!N57nLm32Sg`LAdyRwMUGbny_g>hy_)-r^Dz5fC(-(Cn3 z?T%vcBtI+@pEhbgohk73g=yjcVq}MrvV9h9MEkXEw<2Ko!#fmj^oat#(kGavKwQIh z+zBGh?uNETUk^BOgL&VzrnH7dp~?=fFN38U*rq7pa*YNwWEpm@{;6mqGgtES?}Y$& z#~3s$a|-ZrgHeKY%WuYi>U8uG0dwEBefJ<>-vbf%;Z&}#nf8FwrS=QIf*as!0*p0? z7Gn)dA~8Pqh`mFDgQ5~w*FmOU>g9pn?X&7m_+|n3UP6Hm2j6HvZW#b^=TY~wc2r3_ zGwGTjhuf81TKEb-ZW81z4E$o`-S2(x_yazo5PN5CHle~omcDj>Nza)hIfONu?KeWblrm}1a56> z7}(l1m_GwNo1mE1!-_gzCd4-=DkqQsIS;mBGeg5GlmWmRv{K3gc>`e%oHu82w3(`T zJ*aK2W&&&PhGZ6hbv$pDm42NY2m+Y5cQ%VGg9$7N;DKpw>Kbk)N!r26x5wI)+9OLks=vS0M{YX2?&Ub65d!7b`+ z2M!|Zf->$#fY63G@X7jSeFJYhxvev(|I(q3Ul=r;G`cPrnd)eZ5Rs2ilr_dw+f0wuQc@4~S5gn_(UlE4DKL$_u@Lhd@ZvJ@K<) zHy3o420@d?eDJ9`geCp&J!ZLoeF^$r^p!yp>|f9GXSxTy&r3cKlCNpnE;3Vm^oc*_ z1EKv%ZKatPqE&+;`ZrR+YD9GYB7=Il?@%b5aXwjodmo|MGb!yeC zS+{?7{TlXZpb47d9T)+3gb-ORj-_4J0Fief^Ekv^HpsJlKB2lVqDuSz` ziVeX86*Q0#xA zdIa$h2G1-!6wyQ#U6j#A9eotiNE^MUo_j346w^#Q%9KZuDAiOWd7cqP5I5Y&Ly%02 z>2#t?E6UW;l-_yfl}s9Ggpy&-3D2W?psAG-N<_Kk99A)^)YXqHr8QVeOfkmHnr1bn z*<9b@wN+Tz(^k@Mz5N#4aK#;$+;V@-JvXI#)KwQ-(J(bBS)AhS>0NZ~y%*nn_1%}> ze*OIy;07sOm*9d8Cb<9IfE55tl1)DOQDZJ3faR85ei>$#0YD1nnr*%r=bUxknP-%%gaYWGg&vw{ppB5M=cJW? zUYhBqoqihX)V_g+>Z+~28tbZ!jhgGOz5W{Pu*Dvm?6S>18|}2!UYqT<-F_SHxaFRk z?z-*18}GdJ-ka~f{r(&9zy%+i@WKs09Pz{zU!3vA9e*71$R(eg^2#m09P`XI-<VU4I>a_Sj{go%Y&oza977b>E%$-hKZa_~3;fp7`R8 zKOXtym0zCu=AC~Y`sk&fp8xvlt-l`o?6u#X`|cgKCznLF7@z#|%|9Rg^wnRV{r25| zAO85|pP&Bv$*-dmbG+Z5|NbFV1`a~l0SyRPKm#5SfeBRL0vXsq2R;yj5tQJ61SwcS z3lktO+*YkV7LL5s67u;u4vNL>e3c8$J}`6scInk#s@`C2Zmrx!6T7esPO5 zNQ6DCSVl9Rks@^f;~Lr6MmMs5kp^4P9~$XcMTq7U}*+@rb&yb4$l;k8Sc|%4%5|f#HUL-5oNl$*#gqPIhC`nm7PJ$AZsq~{L zOW8_S?(USTl;teJ$jVpV5|^=yB`tZ`%Ou(|m%$X~>2~Q$V;*ycz#Qg(GMU-AVj>fo z(bQoen^{e3X6~8N)Fv{Oc};JAQ*qhc<~VuTO>mwQoq!W3IoX*?bEXrX@wD4I+gZ;} zzEht0)F<2MX-|JD5}*4N=-0533tx~iZU{|?JrwG-e*P1peGKSHh`5DLG<2XMF$Xr( zkCnhK(JhA&|K^0vt~2P?$1t2Z_iL)fWunZ+EzH!frBTrM4)BLZs3HU#XXP48 zYX}E(kkKB>I6@2~_{*!lkb#KA;3R1}lDXzl2)@W%USF69lyzbUM-c5@A(aaj#xWN~ zt%y3dT3g%NR}4Bj6_UX1O+?-TxlZMEi5@9od{`)i(H2zY{AO(C9i1Y zD?{$8*M&RiAw1x*51-N*9{nhA4tG!oJ;V8rJk+y2=RD`+focFif- za|Tzt5jY%wAq^z8iBN@L*W(JI4qz4uQghtn9qV|hlHG!)C1s9F?-FybM?4|W9qbo%Y7TfL z`v_;1ETnat2+&qKBlf`K5uiO2nWMNLuGV*?OM+Bb4-f~j)~X0Z!1G1H!@Cr=Ex$KX zaT4x-U<>q4cBe(%5p!Ib%xRr!%|p2E!+ON&BG4ec?af0ek_30$%%xq!m}p(vA1Jdx=Zv4_Pwt`O9U zF|E4hI-4(!M1}A5%x*_ve&I^S@VVE_Wi{Y`3RnMMKMR$zf==su>y8k0Y}|$nC*fvM zxYIP?TC0f={qA+&?#|==ZjT1w;QDOuu@pibIBLP{!5k{^BGM}Ui0s$=Ey5_`0w)kQfGwr= zYw)&!#Q(mFBKG0`sA?jP%iU&+1NCeJ)d9ma@CFg$@s0qpHW0=hkFd&&ghsH)u%QQe z0^wlj&9=`Ml%du-5A#rJ_)h4|#s|uOQYZ_3%=I*`;ap0u1kD^0&EWb0sY*zIRA>j> z&h5@@3$yUaK+E_3YYkI~e75kv?5hizDEUBQ`H)cZaEj(Mf(J2dA2v!I&Q2j7t^%bm zBNEXZt}hWe$|L5g;0n<8lu)4#QSyY)BGdsCOXIlcM+Pl|*eWl~NGQ~bY!oYhLK686 z0)cGC&}}u$!K*Te*)Bq`o@fLM49I@%0?mODU6G^?P^?N2{1VUulj+Me!qpo4 zsw(`_(wu+G_ z(n<)jtN{sVgRX6|QtsNe0P6aGA;Id-98T}73Q7{Li?<|>w}2}fd8?>?u%Hsby4VrC z(ypP@Esi?M8`H5O->MT1O3%0|9pjOjP_X|_q6uH<38zckTBr&it>KnU{ea8Nun?}y zuhs^u&%n@xSg^hHkc7xE(WFt&22j^>@`QwPuyk#Fc2fHmjkpr4tHv>Z5rnN1#0S6@ z&xC{^4eCJs3hL5Mh`rtlu*i}qbLbC0V)7bp7mJJ#p${7wavQlS3cr!Wl;Q0DYAeC< zrvFgPhv<=^;!z42@*VX|3Xw~zfU6TsFex)}1{<;;3aaj)YaNw=wz9Fs65=2&a>Qb5 zp%(KZ9gH*r&VqJP`*g~GA_Q{Z=5Z_S($G*+FFWBHz0t6E%Ns{%$ZRt;)WPCHX@kmd z$u`c|u)(ETu_SAA8)4HTNemlCNbS^78<#Q0nsJ8e4xfv&_82ePaV)-3NDFChkSJP`s8L?F{j@(6A%Jhwo!)FB^#>wu&LOY9U&sKiqc z?-8kJ2)~%^^4_fxw{rTnpx|6iJc-NV^f5f^iXfFC%j}JW#4{17>G5*Szrs@pnzIrA z6G7~z%NGdfqHa#@e1X4E^rEn#J-<$>M1VEzFUvHMBU;jh-b@ws@8V{1`1(txfARR0J-K|2(b^^z8!oQ-jzu zJWmlGcGNuMXx!p5G7%6Va)Idd$K~QEi*}C%^D#v0Dm^WKG(^Qnf52xy%~M1n6DLrU zg6cCM6ZQJ4>%lCQ_zX_n#50nJ&aw0lP@N1=MPMaIg9cFu)95V?Sdi!JA&cH~w*R&= zuI@A+VU_BB?N4F#$($^(l8S)zO&$|aBNFTiUrR|df;+3q)iUTyF|D8kj>J;$zzWQw zsEr1mZMNutD&r7~5UHtAB@hj>h`!XyU;|0}01~B&61~)6vf#ivXTnNls5>cQxo99k zi>!!za;lUeIPRrn9=A)_^6K@pk6^WK3^}L_^=hZC z%&v?yr#Q$SChHpYEV1whZA}ughD;ge2=VrdO(&}rfo-d(Y2|k9ATd%M5)JwP>hvS- z6zEV0w(LKWU)OD?h|O~0?@KrK z*BX_7Q_;|b^aoF$h{PZ_W&iIrVGjq=U@Mc#v`}+rudLWc!yL>k)_ynEdhB$WbWy8p z$}Cr3X%-iSGo?Hi#1wU^MyxEKib_p`{fv)*su#SnC|TpKWtR$JO9;I*?5R>JlKu^J z@n>Y8NK?)A*H9~loOUDHRUzKhgm$fnDv#KIi1uW0Kw@o=@0d*)W)`7(k<12G11T@J zTDBARLCh9)A6(9aXg0DYt9w7AVq-`^FJew}YacEU8`2>i#*d*kj{nHWNdHeUtujdE zPN=Vl&EU{9=yJ>AO4cIcQ{o1wZgDn{TyEAn^(j|a{+7#xT5*7FH6@Ieg@|;z#Fq7c zSXT7;Hgt38)9+=36Hs`$RfBV4~rp%S4G!dlev+(aUz{kG-UQ2r|;__;_ zen&31U>KNPuFsr#Oc=DO<`}0 zE7QzkR0GizAWyam6T)BZ>hb=yN=azVKI?~-q1Ygchuhe(e2T78PFhbG_L}spJoPnI zLWxt0$4rQcZ)|Cym|643wYCj^?v5<3#B8$C0Jw1K^~kJc2RV$rHkivelFry{J9U6y z>+0e!wchxC$!LW5Sg^VnHBZQh(UXjG%CshpEdiG!#yY1`h`HEz2mcM3B9JN#$W{Xt zIZSFbg*O3 zs;AocJ;xCP(@HE!1BA;Dwh?a$E5e6il0V0_r^m;%fld2}c6sH~K4&+OqOty-`JsJc zu7Vd2LAhb&43{@MrWImNZ8PnvYUSie!^WtmXs{67H|Ep`hSYgG*|~o6cblgyzuJp8 zEi&lf3;r&4p@{H}C;Fs+5I7h1YQ)|v*?@T}u_2kaYztEKEy-~-h^>Y$`bkTh8wW7k zpa}LLdzQH>d{fkuJyI~E8yrWqa;N-MrhY|OHt!bI}u_nISfmGb!ex=ENr`)Y`;28 z7fypl%s2n+totFY1x>JM77f`mC|h-`&(Y~4KAF#~L(i+Q0JiidMf`;^nTQ$zd^Z}O&@ zkga)uYT2hs6x!hVl8JUA6d8*E?7dc#x!w1#WSp?7j($BefBzNg(G&>OOgMqydzLX7 z*xIvP`vHN8aDjSQm$wS)6fd9qmj9kxC+s%Fj$E%oRhkqS^kFpZIDdGU;6iio|Yz_B*PLQ)=uH+l0)S zCV*4Mo%Xq}IlFrIiSXSP(NMkUi@4C#-*vH}0@KfakkvT618uQzkc?VY0@an$i;Z{WaAH0lhr z*Gb*LL^~RKq;VuyKXY3=;-Ilb;J!phI`vbwg$BbNF8}Qv!GWzGwwovGXe6ktQyr1^ zO5*4gaUn>NxrD$WNYUe#oFqH#Q&&eO4j~DD?IqG`GOXB-IVGAsi#F{*Y;a_zB>1(V z5P|(V9ihQB;@P4&j+j&1X@}cRxv(wVqA@0%vT#viQ-)Wh*`gv`s{4ubk!29BVy{=Tnztq5`sgr&bFKfj#g(c4a!8F5c2Ls_@cU8a5|n$!`QU)<;>H6 z?i?GI#*T!`a~%-@=ORL}*3s(~<=^J?>)T%qo1shk z*o4qvkCtH|VFc2A66F$8BAWe&)?w0xN__ zFg3J8PB!7B6Hhek7uZRm@pV*EC!yLDXQMTg#{XYzT9y?Ia#$t5T!WKshFaFm|=sBO#4a1W@Y_chLSp(t#_7ZzTEXLR5F#OmOZG| zdthsI!6qA93?eKOZZ$%RC6ek$deM|TVN7OyUL8Sm%{Jfs7ISTdhE{GT#i(Ag{pw4D zdFG%e(W30p$m&IyT^KWn#qnuhNjttNlay4hdzM|(F1Q6elih?^Q`h=`2oyH^M9p^F zZodt8+;Yz?Vni&Slt`fUF%95AHy&Zu!#OXEnL&n9H;{GNn)aMgze*G(iZP{;WLx2G zWN2T?ws{oYCvHu1SVA1(-{5N1(NP*Oi`&J~~Qp$T{(B2K9?7%#J!BN26A7hihkoa7m(aSZ%_kH8#*zt%ws2MS|Y zW;PQsMo}hL2z%LbG?p4du?7dO0hvTZR-2cUtREbSO^$9d6zO0rFx~N7#;%0LEG9`g zI_h515)mq8kfb8o(a}?m_OJ9Xsy#0HBL7OZXhr6b1aDAd4vU5rxR?m-cTzf{Zjxm| zAsnnO9od38isC|lH{vUfjBKPM9|_4wlIU(O(%XzqCXv6L1$F;Y5qU}%t*z8SjGZt_ zsd^R{8A1s=K8gw)a$yc%sHZY|(NWR<^02M23zC-^3O1T(9F*iDC=yx@{$z557nwsD zmO4r67DcCx_$Lj8Ax=jGS&)WoK}E#z4|{m$yFn>uN?`GS#o8`%yt)kLF-_b;E%Wyl zY?RB3)dPzLg=LYXZLc%*RFznoCcateBoVgCgk z#G{vowQ- zyWEgH4k^Na`Z1(p1OckO3?)`K_{^x%NAJ%GY-RHXVwU& z{rN%?!Gc-OHYP-mxhiC|N!1eBv_DCevP!H$iHcI>LrA9RqI~TuUr+f(h8*FQ=15T* z1oJbWafO{cF(ewtWV|&pZJR1u9#=dnNwhR2k@s1Dlqg{t6r)5`cf>19DFySMehe|0 z1iRJ0astMylBXkZ5L;$%i`(4lcDGTZWZp8wHya_!UdW`GV4!LnCheeRG$d-zhPWQR zMM)z9>dWwo#u#|C1y&=>8XO^M5C3%lwJ8qKk?M+SNwlcLQ_D53p$>x?HJ8uR@=ctQGL>)>gVPoD{qVMhWRr%E3)%aN|e}b=7 zXChu%!b{-Z{KZFyjL484N;O2<&lfE-QsRt)-F^TGH%S9&DDf5%AJI6l$BKxhm~xjR zu$UmY(Muf+MU--FC8zeBaHx6;qB-hMD~}t0r%`r*r9LzxC56u(~09m>uQ+fnvSql1m+L8 z&9V>8jIPQXsYpb!&57yNb^11=(=sbpF!9Tl2CJPZZ6VHa ze&xQCg^jJ0lG;0dv_l()~)Iq^t0VB<$qu zS?rW7O%=#rr64ADm{*g+(ukT6SBbN@nOBB}Ee5{=}TEiK|7>z0fnB|G_5Q1(!KEDnc%w41Uy zigxs(6$GJb^Ttt>Q?!G%x1Ra@(oq#hXy?4*6d^6!ilK~^V_pvJ> zXv8{Rl&3>d73ZJmTg%)0_Ad#(?Q46kfMe(OWO8v>0k`Cr%{mR=mpk{@Mm&icTdZvi zk8qJ-#icu7JS6ewrVV|@Q66D`Ew@%jMOjwg{^;>bzElpFESp6=N1m7AxwIIxFQyn z3%-CP1)&n*M_`ZP3&Vu}BmuZG2J{%`M}RV7fCM%k)#o?)2Y(be9_Yt^F7BrtE5kAo z)?^d5Q3rBi5}`hU!)jweGh5JMGqD9SRR?h3D4KFB%VjGFVS~nDXvJ4E#sP#k!(3u9 zgCt~wXU1znXecvS2Q^4G2!VAkC_Bt1R0vmt1a*a8<`F+Qc4R>&UXd#VLxwC7Y3er~ zX{d(Il7_>C8!)01k|HC2`>;&$XMX61BI%ch1xS8@BZn`+fNA&`(ouEc0WbB2t#fP^tj5%Cy$VSmFEBLy*vjkp$) zn1|!>6ZUtBx5Xy8fd6qCqKXzp7|M`d?uUNq)q!q!h%hyOBdC3U$B2x{sEqb89263N z%Lt7ek{lrBQ}%F+9|oV(UFcJSBM;vhg)P0u*8is)He%) zLJ=7^Q!`1c?{$hL&%C372sxmvc#%b!nF)@=MDn zmtR?yed(8f37BPhmUmenDN+Y%U}+syl!VEcjp>+=37L^8nUAz%bIF&1iJ6(HnVU(J zlPO2;x0s$OnxjdYrD>X{iJCNdnViX*t?8O!Ihd&_o3lxqwP~BTiJQ3@HLLlWz3H33 znJ%%Zo5M+eoW*IJ$BCTDd6m2goXzQ+zbTx`DV@_voz-cb*NL5E!ko{^o!yz4(V3m$ zDW2m=p57wi%%m>Y@6Xpc^WpBTAwr zYN9CFp&zQE>KURZ>Y^_SqcJL@jFh4)YNO8CqBE+aJIbRy>Z8k1qcZM-_ro|aMSxTnd>1|<(rfI6C zYf73+%TlIq`kO)k5o}7Qb!w+~ik061r+dnp#)hYV3aEi9sL1#rFjkL;il~XIsEf*| zjq0e63aODQsgruBIK`!dim92ZshdhS3uS4S2@U}pE}&{FrpN!OwQ8%kimM;;ifYKK zz3QvK3ar5@tiwvI#cHg_imb`1tjqeV9hs|_z<2{C710x2}E4{EQwwFtK15E+{mza73eSgfwT@1nX z@D0WQ4QS!Hr27u0ArGWm4_?g1I(rYsKnZS;1VkVRXK@Z?U<7%r$4Srwg0RMEVG2ZG z1Vli{L=XhG(8h^uv+s}xDG&l0kOFRS7SPZGAz;auoB|iX1!s&F_s|F$5CSkD$}o@u zub{}MEVJ*B33<>6T#y2Ja6K0CVE+o4K+BmR3WAUWAyCI@;mIMu1)}f@!E6fWfXc=E zvYadkAppzEAqXh|1;(Hjo~!}8Od{uS3&Vhy(R%|&AC1rnz0V&a4|$->%V7#R00W|6 z8q%Bxe>@`JV9TPwmpFU_NFvlt?b8gM%*(76-v9+6;0DAT&5aNOP|*Jfq5#bz(z%)| zm!W(E8h=gKb-lSBA`elm9D)D?IWP>$(aR|y*hWAJoNUa8&9Zs@&}v}}L=XavAP>uN z3tUhJT+jn4paGDL*qfcQiY?V>;RYc<1kYd}qL2hB00W53*{e;mo~;(900SXF(f5%F zxl9SHjoTsX+G$bATrkG>VGKs_*pZ#v#l5k+eSa32K+26^*b*TRW=q^?Aqth;2pDbL z*FCXi3k{*|2Avz-{a^$s;MZtT52nBcTyWb1@eSut4`A%drHtM2EwS&=2uWZBIUwFi z00o&K5tN()N$}fdLEnO12Au2v21ZZ@Wk3X(oCjDf-w)2P@8AZG90Gz};nX|~xqQ`V zAvzC@AlS=X4_r{;j+_F84dExwu=OAhE6(Eba1Um&;?`{z_pstGJ`nFP45lCo!_eF) z?&BwW4>tbem-BrCDwiI919}1fmzaJ7C4b4T?7UnL?*I(}E)itg;_}er%Wmy%^9+>m z-k;qM_dp4te94b&3fE5VZo>r^0Lm~x%>xnGInV>|o&%~~?(=RUudw4~kOJ%8?OL4& z&+zQe{@rb356u7!p-Jzw{^Amz?`L7rGX6H$FbGs2@l>z|n_yc9|LY2`?+srf&=}AN zlmPM}AM&E03TrS0Cy(+}5C?jY2_lzOe*;5*|MgqX^*&EG8qe@rO$2Tr2&OC|=P(S$ z(9Om`3t@2b6EE>opz^+;_6lwHcaQgZulIY;_kHj8e-HS9FZhE`_=RuyhmZJ)ztD37 z_A}lH=pM>Fzy;p`*6nTimah&GFZXaC_iLaE_i*{4FZ!cT`lWCBr;qxnullRc`mOH; z`mYcBv6mcx11nnm&ky@wKjHe`571x=uh0lV;K*)3`BBphYry&8KL{;}@weXijQ!qb zA2YyU^5H-6Q?LdXU;ep%{;yrh=^Zn)Kn0uc{s1v$4ZMB@4IV_8|4`vVh7BD)gqJ^a z1{Hruk|j-^M43|MN|r5MzJwW5=1iJ3ZQe}y&P@tI4u=s5A?Te+U~8sOL7Ft_lrRKw zK7|@p>Qt&#tzN~NRqIx+UA=w<8&+)9dOLeI%#mmG@&x69c$*zoj->jUHWwD)vbGV zO(Y4FB-(A~$&mv_4k(%GsWh{sh!6kv?2D)wWbXQW`t|MK$Dd#Se*RhS85`ymV_1Lc z#tI*bKrq1r)u5$62OoqmLJ23NutE#@dm}*&5%j@|Kj^wJL=i_Mu|yM3L@`B;ma$?( z4G&zR85L)wu|^wj#4$%5^&+K37ZqG$kR694vPdJ3L^8=FN%`@`C8z*KNhznKvPvtj z#L_e-o8<7xII;vYOfknKvrIEfVv&DK1fN`^!86~4Gfp|@q;tm%n>-LXD&knKPCx$y zG*CeYZ7$6w-`IqTqoO3VQAZzzG*U^Y>e0Oo-?%1>DW-_SsYyQtHB?bYB~_t&mMP?o z7On8n3Q|;gLQ!D$DYaHxZ^boN|1st1W|?JredEA4+KFS+FLgyWS!I`HR>Xf?B7rs% zNzic55@0r+LKRP&<+fXIzXi86Y$WlcTyqsM!$2$A8D!Hn`V2Q-dFQ3KUaQO@*Ia%5 zFk`(!fJsVQdj}@CV1p0tD3^Wry+e%-muZ(%ZxE)qVv8@f*PKWe-nZW(3Q;BCZ80Xf zWRp++R2fA$*7ptstyrd5fv|r;xn`Sh#`#TUSZ>+Je-+Jk2s9Kp!Ah6w0*k z#yfAl^~yzSjw6w;zyqyl#;E@(Hi36<#TRG1@tgW)xf6r7EW#4a1Y&>Gam_d9ymOG0 z*?3_|6ehdz3%74}0BnhRP+Zc9Sm)HCV72S5Chvo1FhgWxHI!)s)YLu&Eghd=&! z_sQjsa&s@J8x9dQii&@+>Nd3r81~SI zKdccp%+(1H!EHT0u$;So{|7`SHqnU>lA*bZn7s9%E)Gwa!d!ozxJ52@aX#iS*V_!j zBK8y_Q3C0p7uC2%HmYrgekfu;($}Fs$jo_d#A6=y=oc`SD;Fm0N18I&yEN|6kcUL1 zR-#xv(w*>SbuYwwZRKlZKN5JLKSb9nv76^a=N2=02V5d=mDiL@9Md^ZdiF41);owe zJG9KBu&|#01SkY;*-ro6Ey!?j&}Kjtx={F?(hrfSVd;OG#>=AwjiDFC==UIZ%Q%v; z2v%qpQY4vCl9tqS#f+#t5{RHb818>2g=tJ3H&BF#^P^xm${L!HP?`4Br)ERxLFz%d zKOp6B>iB6=m0C5GY4RYZ8(hSw;LoL2)v6|QC`4<@AvLT)QG-D0RmD2ii#3&_LapX{ z%z!Ogkd1$=Z-r~PvKr9~zRatfqHA3By4PlHRE)^mk`IciD1q_Su!nV!JciPjxgx}L zF9U<3bc$HYR<=3vFoq~7ahZesCli9mLmnD2jPo=E8kERHYE}Dbt27)Bn95P}et{{bP;07?i_kb(>JKos&|$UT1=p#k1DR|BTFN@Edny2xVAw%5h( zU(T@;@uVOR2r)(uxR3|X1|bAFctvs>ab762mpjA2idiMM)%A3h3e6j7cJ;g8uGHfe zqIg9aLh#*#oWm53DF`KUz!T(}YQ2kr#xo?`V5>|JcL`ak50KM6{&v{IUb%-L2JDb? zBq4t&i%p1fHE@=$6wiP{!=M%J+P)9h*v3?mhlmNxkXyh;C<{^UfX^Y1dw`@K=Rk6j zmAqsoH`&SW@J7BOp^SGd8MZD}N0pttWiEHw%U=d_n8iG1GMCxRXGU|H)x2glx7p2a zhI5?dJZC!B+0OrW#&e$ayk|c5+0TClbf7*3J!nD~ddVEg@rWD3T~Vkv1VL~^BX02< zk3d5ZlxRezHN9z0ciPjQrbHmLmWWPMqLWtGa2NtX2~2<5)tAuy2RVQCrV*{525+uiSmH#EKhjTtJ!7xP~CH(-Im*nSlo_>OnL1wL?s z7u?_nM|i>&zHo*&+~E(0c*G?>af(;m;uptw#x=fij(6PS9|w8JMLu$pm)zthM|sLs zF7RzDS|MrC{{R;d%#eR0pW_xrNI?i>z7_I4R3UTJuw>17W1<(mC5hfKLK>0tGR!<8 zZh%1upu3g&-7g{ah_P3j7gnQp-Rmbox@{<#9#lqG5>p@PvVc*f+@Q1$$w-54--IjsU34woxDPH09gLpd=U1_^K z1Bt;Ucr`h0MSSQ(4~=2mLcr!8Mz|?4Tq2M+w0XEik>#P_9Js+Cw;0Ad?s4|YaMthC z^MeHsvdSV5e(0h{KHaxF+z>D~1u$qtKSF!~=8}8BLwScFG@t?B_Cp?t2~TWta9ppr z_@SDAtPlS>FPDG(Cg1%=m&YqWD+v0Nm$Sp5fBy4-i}UR@F z!<1MjfxO#D)Mq z;WG%8|N4JHJ$a=_)Wl63Aw^6GsItPOc&SY!MMIoK%?qS1k*q%~#a2ASPVB*gfQPgS zx-4=+SHwj#1irv~s$fA7q<8{gK*d}n#wQFa8cdMbnwer`MlB>Cc0xt;aheUAt!A`F zBD^O`G>8IOk1O;Yp1Q_xd_T}BmrjU9>5H57*sy;=m>6+%M+&SrZIs1bDgu#t$9qgZ z^Yg?{R7HWvLwp2CuX`h2JSGmI#`X)ygtReOTnJt2Mo?17i1fO?u^)mQBYsm98k5M4 zw6Uy`$AXw6AoQO>6L`avk$-N>+5*kK1xtLd|8@v z#2z-h!Z=_>t~AS-GOGhx2xNK^|2fOI{3&yY$A(zJtSmgZ#7mg!IZ4>b8bpdUxW&8_ zOojTp29(Jv>^C79%*9+NbI_U;w92}qf;N9)%*q5PY+%6oS_r>Pj=;3c&wQpu@&m9e z2s^Ywp9Ia++$Ky)#GRbQOPtJqRL$8eCDBv{aKs*8T*6MF&E5ni$26dwG@>;@ieK{0 z<0K{eF_&p#h;FE+Jc&)@lul8yhi@Xy(ow>%ipuHaPBmI5B5H_bIHIEe;^HhJM zH~+d#nxjR#EKBr+&nE&-61vXELXaldy7<&jFH%k<+)U0RtnJ*-0R^G9drws|NPi^I z1!baL6i#_y!+JcNnq<%leV}Y~2>p4;2oz8Y^-%Z$&O6x0g~%|wl+6!C(fFy#hNy=- zVZlK-&=i%??`gY_yvT>Ns2SB!71e(l@N9?@olGIz(IFik40Q;g%Ssp}(j}E17ZcIz z^gaw?LE~i7Dh(ds(@cgSqc)@hH?h($eI1yzQ6Icd1_IMF4U-Fej+`7(g-Aaj1S>R^ zQyq~9QE-DYxV(h;1~+goN?1G95K4=zDY2Z)ES%FrEfLpPFc@GlgE$8$h`N8_VpJ3V zOpT?4Qn(>hjYQN9PL{^A2EP=#gKhIQD7h1iId z*omdsitW~y&eh1ru01anD* zOmJD71-d5*13j?WnE&LS&VH~N9_UcT04#nipRr?cv(;Efg#op7RGj;<=IA-5G*)_Sg3egh3517O&6$jC)N`oZ zJXl-IrBpy&QHR)4ojuXeH9B~>22!vDOCW`1xQ4Sa-Hn9})xBJvXkFD9AwP)Gdoax> zfmPZ)ymlZ3^>CW^xCYWanco#boO1&T1KAX4gyJ<`bK%?!b$`_yb>6T4Q-?qJK_`KQ zvccYhc-Cm0)*Pr-=O|1%Y|aQpj#)Y1uOq%6S(mGM--3V!;hMVdO93btkmx``Y%tFo z?T|f*(f-x1YjDvf`2*{{n8OeSC`bV=kUU|%jo~tecgTavrGz4j4%Q4#h)BoN97PJQ ztkJch(N%|bz<*&J?uKrFhaOQADE3@ zYrx_2IT%xTz1I_j^fCx&$OA4QQrSQhG>PmveZBk)@6!JU@sK&nl8&N4BbcXa-vN12s?; zAK7FM@#IhDd&)!i0paDJEG;DYKL6*5jXz9;z(mk zUWRrc2!9{OllM4fd7xxc$YnK_Ub*RI7qNmR_yb3Drs`ap3BHH+ zf|e!)uht_(tmlY`c;t2Ze|*WRK4k(#y7_eYjmq;Ox$}0yT(j*)EA^;h2vm33X6ybS{L& z9vrkL?n6q$G3~`m)KuLjCFu2SslMnX(0^UtZHK<*h>be#$NvTiem-eSSnQX9?nFw< z3EsSJY=XGPZcFNCp>_{B4r0D$35Vht>NW|JM(NJ}C{gy(J)&(F=Dhh<>-yfJeK2X( zWfS|(eXxfNw+C~e9lv3jDdTW+zy@>Ba1GxF@y3XDfCUNnWf;+2aMq*Ic7Gd* zSS9sagH14R2X`Wgu4D{y57-XklE`oo#|CAn9oy*yNst5&=NUza7HLs(N$7-5IC5pM z@)0NT9G3{*#@*I#@g*n(qs}8I)hdX%=kWwx9d9FbFk>&b6zKL$jsIqEGU-3S z?~h7m3lE7oH}X!11SW6tJ{Mn8?{g=o@?0=<66Xl$b@Pi>gZ*A3(G*k~?4;r&!%KG} z+co2x9-OmIYmgZ7A{TXY0raJD@H?Ra)K=g;$aBQfZB~SK1 zmzGt}g>cVsgkT$%dCz_?@@db7B)@ZQFP~Lc1`gkMgJ=d8KcXVQ_it|I0ZL2LE={p) z0?VFv5VBq02JTtE26jJ05r5BxB`+61$MbJ*b@Mhqj{oTdm+yxm7x7RZb%}TKZx{E9 z$aL5KC}oE6CxQpyQ%e`_k~BYgl)nZ|-;m&5hKpwi45yvYSr{iTZLRNzoY$902MKd9 z@=@<|Y0vVCfCsBbcLqV?CK~Nql87^3lgXNT2!aRYR_`_r?gn=Vu796fj!|_0j|jO8 zda&ntitu+k$MX*__$){9YyRyYRD*ARBJ;zii0B4aT?6>u`{-fwKZsBS!FNyJh}rS? zNJw^RDfp2PagjfJWq|mFv7bnwa%>oPiGX#BPWN_DVIZ=l@l|Bc&BW1%AA*1dH^+QP z?})Dd`c9~LvB!#jkAHb+_x9562l2LDO$Yq0{&?~Kx!;KZMJsGx<$oVqP=gnFj|Lh4 zf~b3anX=v&3=D?{YA_j|C`2(*X!-ftYLX0SJBE^apFJjE7aU;i$ z9zTK%DN>}lNPbRY)7Xum!6us)j*KaDCe4~QZ{p0Ub0^Q9K7W4#4Jvdf(V|9=>V$Vq zO_72J)hz1=Q=~m?l1^gvqo|X)e*2IXwYSe)E|XcYMp9#WmtQ>5*gj74QcJJcN zt5+g6MOag2Tn4wLs;_$wBTlS%G2_OLA485Tc{0v+q|^-d;lqj=Sn8&Rv{&}5B1y{3 zv87ydEm=t>?|)>C)Jg3{c$VKZHRxlBrE98x1OE?h)1ET4sW0AJcF+nKRmX)xk1l;W z_3GBIW6vIP-=8;`cmG+|4m3#7X0a;Dg>9{Nk880h>vUGDY`_(jUe>()kMH*W|LY|h zR-HUo(LJO%RRkEo7zUt&4?-BBgcDL&p z#Ag<4r5!yHJj-;K>C|kOt+KScf?W~UG;dOP;EOZfxZ{ZVoVFE(EF1$&3D>Mgy-+$-XeMjfg2sJnHrmHet){`v(sMtMz$@(6pfrM6EjI4he~kOU#@sZ zy9LvNHmz3lcCPK*+9R;=7%kL#5o)NS`19Ly-~H+M{UK-(OB4c2>=+RzpQKu^oN?xh zPK4pC`fX~i_pr7MXD%FB|N8m&12BLB^cH@iHmJ(cWFOd&7kccqxYn#|W7L{hrhiCr z4bu$}PYlc&|GM@%3}!HdA{-%~@Dru~+#o~^H<}4?R`fF8l?_W-Nl&jPLJQpJW_*7b z4+;0gK;yA5BGy|1?0RU#BO(z=wt*6jQn*4~nL~YWTHWkaq$}8H35NZ!hnaM<8>%Id zOM~-+B(xWi>fvyF&IuzM+vrB?z<(oIZ)hUvQnea+(aSW?+ef()E^c z!8abs9#WLV|N1dRGc*#BlAI(Zfl@s)C5SgqoQYLtIKAz4Ze3eh&qM@r!rY(?gq8eH zCr`4!4i;oHV2I=@YiY|{-mV|CU`kxZM~W@pL_mq6W#Gs%pE7E(A5WT5Tz}GpO9{zv zU>&j6LCgTSXJRv(+LXvXmf1^RO8*5X`9sSqGZ@8V)-WP&*^oh&aL8@8XPoJoW}Obw zgmS<$pZdh*rB10C&ZsFT=R^;QNcbx-oh5D(Q3@uXV!wS_M;Y_dN;x$Gk2*O;4R(Ym zM?2a>3S;1AN zp|;b4G7ZTFcZJe69@LLq(c4*jnaZ+8MysOo2-i-vL%aeuunz0lWq-btLRh>rC#}>U zSC8l@jVV-}qRSM@t|{2PC=xCIjg`nX=2)?2^z^c#9j%VaK?+M)f)rtuEYu8Ju4{-D zBPNY(B_F#qZnlGOGdh_CNqd-qvTL&u5hM!Lz*FBMH@Ovhl6T;uAr(4Akj?et_0$^L zm{=uD#`~l#of<(0N`EA`Z-FU)lzSI*2#k>)xhO#j0#?hKH@@2F-?+vr!tsqSZk%^0 zV@PuPoq3D++yh@Z!7a7#CYHMhRqDxDq%&<-wrNO3#(z zPeSI9P-_hiG3s7bt@~UepK;B!I3`1Apo*O`&?W`Krc|fs*pVITV89|B?^3Zfl~f-fXO0}Wywiozch zAJ8mElj*@Nyh0*UVkKJQC1PSGYT_nxVkdgyCxT)qisC2^;w!v@Bu*kJs^TiLVk^4h zD^h>rDgFWT5!D~`K_<*0EcPKnWknzuoh|}nFbd-^wqnu+!VDf$E+$NsjWI`(BLNa7SI^;t_WJF5jL{el$ zT4Xym!YrzR8lZt5T;%^pf@DaFVZApqf5SIP1@v5 z;^a%}!55+jO_C!@noN!SK_eLCNcIFJaN{dj#5+90GYo?pz(5Ej0XT-lJ-owJVr73; zYUNgPWmkITSAu0&ise|6Wm$6NIjF-^y^K7Vr8=|@m|YKSdBZjE&HPkVc)->@u%%h@ zWncQ`Ujk-e3g%!EW?>rUVIpQ?D&}GWraE-det5%PGUjAbW@TFDWnyMo%7Z-^9Cc|5 zU3f!ghNV2nL+E9nGN^-Ss^)64W@~@CW>#VYPlgIMY~@@wR1xrtH(2Fm^29d~!3Dx0 zMw~+)L}mK9-`NGv|Nkh3C7|9CoSp?W$UcZ0$rQqFp-I@00%$oQwB!#m#638KAml&@q+$0cse0~&%gv&OEEz() zK_S>m2yUp4E=m7Wse`y(!=2Klfz}#>{Xw5TW~n8FCG=@(k%EEZX*-NZoQhUEzygW= z6Uy`sW|`)$*l6;Z12V!%<_O?JP|%b#i)6e*5Zq^e_Jcf7LJs6W9&~?cCJ?14G(rzt z01-UH{iJ8Hz+Xgo2{te(fhuTs%-FJMN`n<@islH2x|wv|DJ(2Qt)bQ*6v9TTL4!7E zj;ZMf%2|#{YlB)Fp(-n!J{VyI3b{-TY++X2B;dQPo4qldlqo}eXxCP3>Z9e0pzupw zoTpK&s0MJs5ITfB2*Q8el>tM@0~e$Ks-{2)7{M)g>HP>>1I6n@v_>RIY$x1mZHU`x zRp*-4$cUvD|1A(}gpkFf<=s>`z6G-{fGq@?!A#nG-IMYe=l3jEUQb-o1>>czLSIAqIam2!b1kf{ZrAJq$x0 zWev{&!Ufmuj-e!|&{S0rRY}+i5 zD7CGRdWkaZ>K1>QRJ*8C7Zn6l-0Ne!!#6-faE4H=!iKT*iijqZp6x?3oL00}tIUp% z8W5@>g`odBXs&9|D7_qB@~R4YUD=Xah;2;gvSKXiEg4+mEVaG?v~nzJ5o)c`!uakI zb*Ut7s@C}KDQdZFv^E!pf|+v?Z>`;FJ2+A&g>6a7fXeAb5Kp4;tvy`m znG{v>Ezo&j8O6qnY*tiETr6`=SN3j(rWC>-q?VzE;D1OI^Tq}Yz6$ixpn5&XtljCd z3Tn2gsp8g%hIFsBX>hG6tMMZ0Qc*;Z46mPJZxL7miXdv9GV3h9Z{osQ_coXz`JXnK z;C@XG|DJzWyhIASgiHW;#M*hNdiV}=2@2)B?wi%>lV%?_m_r$x@foA>8Iy4um$6uk z1vZGA|3-4#w&mDzI45UV8!h1Jox(92`|%&IMIa0EAQN&S8}cC|@>8XJkoP`;T*vHyK98D(?Q!dQ{ zlp*!(8~KbAm(F4E8G1z9=O&-t=Gk!Z(v(2)))|`^XR0D7^D;AYGc$7}NJ1w_^E6X) zHA}N3Y{DRz!8UVqH+wTTgTWwdLLx-7HJkG}OY_L8^E$J0JG=8c!*e{3?9kF+ddTxV zgqo8b3gm@KLd0?3-mw}bV2{IA2S>RmeD~obVIW-Btx_y??Y1+@7M$pj6~_cU zbH$c;XA$s+Bpcb$MgpcH^HC#pQY$r24oiPiJM~ka#Zyc5R8w_T4~rOEbyjP&Q~QA( zc=cCGoAp_vHCk)MQuAbKNWxpgbzIB!T+?-3!!E#Mz#di>#B76m#R*!-{_(V3s*|emnO(qjP`zH#O&iDrECF6L^6W_*M_g zGfV7(GkAkL_=7{Z#6D?+Q+S0xxKRsthHLnSb9jg2wlf=dQM+eU5%PGWu(H4%xABLE zgb_or69^s!)*zrVi%EC0H$`Bd@I3?+o>1#b?aEbOB|k@4mwWcIQ(%( zW+X%{PZ}S!`J*2ODSHTLs6#m<>oUN?o}O0Adaqy~9H(>gei`wUjE{e(F99mk;4J=l zx3r9@8+u--gYT+)k8iSerxs#)#6%-}!Ylm3D?B0l^M0!{hDWt*LbC%C8>230{|=Fl zM|)52KBjviDOgB10v7GG`vfq%l@JAKT;5;J?VM|4N1TVdLwG6m2Js%_X{hsQpF%6U zzq;@)uDzcH_{lIWEMR}btSx$YJ5Ja`GxVuRZ~AUn8)9*{RE+m_Fnq!XvZ@~w886VI zFmOE^_mqLeLbO9SKnV&hhjDpNGtlDroY>W~)su>_$^VYgtS$=81^xwc8#lXZgga;i zew`v};h%jd^GzljIpRL-2#m3<-JjG2c38vkh??JNaa%>|a87uVl+-P?U=JUdq80u22@5+=mo=l=h6ggfCE z{xx(%H^j)KKYh%(@J7$EVHfWn7ruwcJ$2Xvlnl0Dlfv~w#6|Zi>b*G7Yf49lvM5i7 zlOkZ|J4ERN3#5Oz+YG^2NYrUziO&I*gZDp+TZR-s_$X534XJkU`XyAjkYPiI4uHWJ!}JQKnS6l4VPm4<~lY7ztZLc+1oj5m-dcwUzrdAzjMU5p|b~6VT{_+|rHH~%p z;7#fp=HY(}?WJCw(KG9evY%(yzMXq_;hU>R%4PG~AHM`i0r$S1eS7!s;TvsNsBc)E zmaZ++534PyKWDYd?Xj-xv#&A67<)s(1>Y#-j|cxajFh;VsZhQ6O4_SFv6e|`p9&BA zqYwrevv0tx{%C8NYnoFBo(nNt$sTM{>!g!hun~XEBzt%?henI+(8x8|bUX>3{ESOZ ztT+B>CdMa)^5&1lxPr`*6`j+LB1sg{NIYdEsgAW=V1&}lGto>lq8a}!YE6?e$AO%l;M!luoPLc4R>Z@~>$D05`Oh$fqGI`0o<#ckK! zci{~Q9t|H`0v2{XvZozy3_TPvOCUV!xK@7)bkyKS4V+Zru_~=p;D;f8*uhLGuGnHs zksGznfX|w8L50COSW@0Dos8Bp1 zlvcZmDr)(n*J5Mnv`!>x8b_3Mj#}z)H#3)`hOWtJr=F(n+Uu{Mj0(51A|qumr`mrh z$I$%z{TITt(ppTx09`%UZo5a#TkpNY%9!th@78UgtQ1V^54QAU5h}3T^HDk>uT>^y zkxnIr=0^Wwk~yduKWd-iM#TJ$pt-XLH7+p%?ZzA-D$TUXJC9v89JZ5}Y@-j$i+!eU?G9#ytP3H*?_*MXEKET*L-; z{&V01GefS7NTMbTiOL(SQm33jkb@ocAcl@qC{#`FU)7_YWUN=ch^_Ep+H-&53t<=_ zzm1A~5=-G^;KHI0LdkRqoS4c2bS4?Kg^}Aeq#e6UPC*d)Ol@wHn|R3@qqxG8V|EimrOAIqEI;E1MG)y$ z#W_xu1hO1sR*`4M$z?om*30ckQi|#f79PnNP}xZkBNS}NKK8YPjJlexB&5{4B9&?&HKg={&Eh)|r;rW*o z8B-k}Z5M5b_a9>o?uZS|Oft)*Kq}sqvf_#+My|$?Y7Vfno%Mh0XF;2gswvPOh6Rbf zdPY@8UTQAt(-~sznM;;l6{<%uXH56{$I$K;PtnOpC%iQ$-VT?z#SLf{z2s3W(&C+f zij}9B(vJVNb?9rs{68k1*rH_ggTRhbgd+2T}je(Y3|9^6%!;B4Kx;&yjz2jkPt(C() z7S2Q>QJrr2n8-yoGPshL)|qfoz=TpM!urOW6e-vwegc1#6-K#|cF^KKN3B~CmcShT zO4mEo+Nd9J6yhVV86^b0NM_j_XF1Q=J4xQ?xrhP}dBDTK;fxf?nlzGY{Hb6|=#VYB z++6N~`DO>X2bs%^+it0s&XsnFJtE6dHCvj~o%VD{i1uOKHn|(H@R>-V46BZbS#eNl zpI%nU6-$3OXCK+6OQE{q9{|meh`W(;YrPws)8KZCef{*YOM>2uNP@h`cJ{NO-4An4 zQW1PL3T+6*pAj{ZKm{U8tkhsQLIX-UY*~wluqXdavXT2MOK1?{()S+J80`*7gITok zoi8QJkEZ$d_rEQjOryC5&P8cT2&JcPNBvD>dRu>ZtOiRfrfmA98ckij(`;IRAA3eE zQ$)Wv9&(ZUm}vd1_GFP#Ys9imvD_XI-@3XD8*fJM*T=HQ3B)jl@TPc2bnB!<#6nKU@D}S4SnZs_5;_!9`?^>Jv{8(POWcU zs5yU)3No{zx#FCnarV$Cfp_R5j+!m@Xv^%A>AmjX1gT_kM();k)vt; zyU3CIc*##5qBI(~wb>J@4OQh0kmS>rHo8t8|5HBXnk4<{QU6S8wM=zM^Uh6tiZCn( z`$>|7+yWDy`Z%Lp_r3SM7%iVV(dRulf=qu9vS=Tl87Y&QtNZ=)p&xzezbp9Cq@Uak z9=$Q^X(Ef~mUu=zeesW<{N;ZSxndfu+^;Y2+yb%r;U9nb&mS+V@2C3L-#H%fN+v;) z=4*J5{seFU39tZ1!t(fN`0{URdW2={&Sike03~n&DX;=-hc(pCgL$oKX#f1p1qo5=Q3CHja;V=#>jYoeht^h9u)zjBu@-Id7K^J4e`F2S$u(YMfy@DAuz?qs3Fty1nJRx#5?nEs zF46t)4rT}u7o~9;sqwg0Ml~2uNPLkLb76tNgcx(HM|edUP@@}}hG~{j5t=c6Mo|`L zaRI9_9_4W!Nr@7rV>;~a0ohR?V-w_W- zlKh4t4{Y)#-QXE!awj=$SE`Q}*Qp!5kt4AoM^ca{2k;&m;Q}-u1e`JiG(Z%Paw;t? z7pjmt$b>3kP$@&e2=YJ>1i>bCQY*=F&3tYRLBj{jQUaAy0|cQS)-oY3*$f*l@#CUV zE~Bq4^T6hSq8|M6F99V7voHMVJGin8h5;JR;V0f<7>ps8Sc(KVG1EJpVHi3g5ZJR5Zs9!PGd|^WKIyYQ z?ejkIGe7YYHya@YV4w$*AP$$MPIZ$O@LmSFFc8SeF7r-2)|VIK4$9^wIEHMYp&fgQpj9Fn0_r@>am6;VePV(}G! zU(2;z-JxYaa%0bA7&M_?d%+MCwPA7AVSnKlm;oG0H5!sZ8$OmD;(;Gv_Go!49>Red zv_Tt6b!T^$W~KIFA+~BImSxcuY1@MsY}R3~l?zxv3dEK@B_R|-p%xM~XB!r3ch*#K zp=pKoW5aL?;dBSqS`SwffERc-Ra`Z}5RCVD4Iy}icXSOGar0GiYd1!J`alW{ zA!lnA3A&&QkiY~k*K#$t3BVu>#5WA$!0yyB5kOaT57ls?_H-v<8{l^vMqz&E_ZX<* z80?oCT6b>i7H^~Xe>+EM^>-AuVG>MtU~2(Ut+jX!!40@T3VHwukbruO|uq&{Yrw%KzR*83RnOQ^o$rXcze4Sb7cU0 zselTm;D>vd3dGlZ(KmD@cW_O(T1$5lCV>@3ffZ8!A&O%`it#sp^>=7xcXnBLi)}@K zk0BLUVG??GbbFzA-9QSEfCVbo1Xuuj`v48}V0bxI7c>D4q~RQh0Ty6?0fV~$5Tf9Z zO@NKn*mB!=f~BAgNO)$~S8Y4>dEEiLxOh04A&ps}3(#1N`=Eb_L3`7fdoR}nHWv&; zc@5T}hky79Q27ajxO^1>ebcvKPq-FjIf0j$6>K?*Qy~^&0TpyP7OL2aUzct%wrKs= zJ3Ll@TQ_4PnRYw&ey5m!6@H-?5}00V!E9r=az9yu`+yj}wI4*G3>LXzdqE16iXr$R z9F8FtiUAeqpbY@wouhz~vH6oL`H+owd6W2Xng0Q6lSlr*t7ZH7vk<}_U;zyzxC^!! zBJlWzS-_Jm*94FN4bZ?02BD${Ar45{lv5d%r+^CLU=Yl=+CtZVZ08tfXIYl*v~)kg zmRVYgcbS)UIhTJKn6a20@*yAM0UvVuZjS;VoVk~MIUI_4nKibV(>WHx_!rRDTB}(S z>c9+SpnKaGp^3pEoB<7zms4Yyh9SbI)A-O5i$XT4Vsd3*mAK~5A+~;tM-V+ z8DjbQWqIyQA+{HP?sXhKb+>2(s^r0jwYQ-jAE*JT#dv3% zI2OX;tOA;-|4nJrnU|qinh!S=I3oHNO|Je8+l4L(}W_gpCb+-ghq zi0|~IXIY7t7{YH^ilunM6&f&hlxE%g<>Sm5o}IpReA>^U9;QnjyrCI@A=?$3+X38% zz5Uxenv^BGqBmg^_?#TeHDwdryaU}%QChRvR;8J^iCNl~C%sueAr_MS(0jSje;Jv7 zGyHWmoWuXU!-Z#F3$_e%6+VGptM$+CU=2FDhh6=XyMV~G z`K z_v_O(`x#sp#(uL;m$SpaiRrie=hunp7njw4{n?-WXZro&KmHwE?&+WY`M2MNIVfti zf^iif#@Xvfu%N+%2oow?$grWqhY%x5Y`Bk^u3i{3_S!eGqsNaRLyA;rkC{e)jk=h* z8_BYz%a<@?%AC27T`E+lPNn*Vve&dtRDx2O0!D_34M>p?qY08-ESxxx;d=;A9W690 zkZRpvf(4qT2=|HIH_oK4izhYzG8@N{soS@3;kt7-Zrn6xc+=|L%eSvyv?tfPYa6$) z;lnWN5$nRj#L=xu`#6(VZ{k>gvF)0vQZ*>fnl(a&P6ji=T1tGV;%(4+Ij_{$`-Y1pyv{#CcTcklb` z-5XDyyuI`2)7!huo-pF~5G(dp>mI)S`wGYTorb@^|Nj6&#Lh7}p%Tk~Bxz04OdO*L zWkyy=vCzjk0UkskRH?8NUq9{8A_EH$ZYAM4l`TY1Sv{R}(RJ>eSK*5QhS$$P`_PqJ zM#o_^A79DkNZfMDsff`uE&sx3gH1EEsIti* zop~l2YGj7iAemEq1}1BNo2Wv|Yt6)o&7A3^Gmf3$edpa=oOU``dt$AJa4WZ%Fuv0oe~QxJxzlRP@|v_}l2};I){ZxEzD7%HV2U}#6DXtEU}>a% zrdk{@uL)*rt!-w1Gce5nlS{xnhLX$AKnFeV(DGJlG@y(|s`S!KH$AAJh%zd*qDNP4 zwWe5i=jqm5bFJT`;&v_e*kfmHHrDD|t@he#PwjR_M&l;#!VTALw@d>*wJa;s1u8Bu zaSV0xAkB19tYowfc*?S|P}5OW*T8~?4AmlWEwl`USL2O;$fboBZsYc+`Q2e{Zdcmt zLQN<}c;zY1G`S>l$L#&W409N`hcV`9C_gdUXST;7iNF^rGSm}P;2y^venZ+Wy!paK`jz}XSZUeEJOEr@0?9WAXE7{uV)HpsyadhmlF z450{#BN)0-?}R8c2tJOXw^Q&9M?RrPJAfiMDgXl;NO+71CS)soP@_PoIEF{As%d$E=j8j>m3b4HUwq!q`2NJ`#= zi#zH)>N7+yO6qnO;szD}Yh41xqwD8k^$QkwFVs7$3QSBc8^x$>1l3B&MsBnjP> z26xGS10Wd85C}sU!WNzwg)m3Kieehmn3G86C#2Jy5&p$#&|KYaqDjqaT2oR`>>Q*z zD9w|A?vUUl9W`2bpun68a5anrHlz_E77#;YwrWPYcW!!8n$GF%o2weDq8j1=l_*w6RS%o74DaL&bfl z!VcLqjwA*FjDl1}OPU$PMw533Nd&B2(-cPbHiNW&Sh=tCDKwiQQOZc5U)$9pEJ z4)J-Te3!)JX8&mRC7A^eX7{5)90Zx1Pz27S_zSC8$4b_+n)R$`O{-ehdRCU*;VgfD z=>;j**t@#cwWGNMgI>*G20aKu5Q_+GDq6TWs2J8ju}G|96Pt?0I@Ynp9HwMhk<7|& z^sQqBymXuZg`qhac_m=cF-(SeR>_`(;4poAV~D}~zH_O`gqt!`(_ zTRmt3u7AS0}tf)){B>T9AGthtp%896ZnCs`c>Y|$-d2XU}8B2UUa!}yHkUw#`$GJF_u7~cW zi)y(}ro#6BseLFFA(=5_XVOR}-+43)s{v^P2c%&VJkY6Zj(TjN9VGWyq*snq0 zk(rqK5qV*_Lm;!s+=W8-#uK?7F@ic&?Qu^^*m@C`#21^V&S^_~j4Cbs7>7WTI9`e% zWgJxTnkrlQ%2>{_mbc91E_?aQU=FjGxvYvN``3c~vYDExx+dGD(XMcgWf=6jOEl;q z4}bs!HK@VDSRCWegIz36=$Wg3#TnYrhlR>w7rj`={`nYt4o5P$L5|wYrA}Tuo2I?v zn@xNA(+nn?*jmLZ^{D4E{n&>xe4&Rv2=!thigm0VQ!K5pKnA!*0vcS+>psBZ*RUXl zGlm^8L;uiE7onIEELuHOiy@)5#x+Dr0rNS5;P~3_$y=^ivCKzFuNi}Xs3GCtL5vV} zA(C|qM=I!fjChPh9TA&c<)pMkkNC4Yd5Ne*KPR`jJ=h@#71BzqG?q>k7(10xoxTaF z;7v6+JT_K{H_UiO3iFPQ8K&}v>*(Q@ltT}J!(|xEAc}TS1!P`Y1Im(Thd}y>Mc9t{ zLwrgMCNajx&h~Yc8V0_9O&~9M{t}?zR6!6@8~x}=PrA~V&UB{t10Fz8L`r*c2P)8_ zV_0YLk&&D;V7w`lVLIOW)_0oFtO3I_Q$;hN5s!SZ#|(pbyRW zz9+i>(T}D^IN0HCnOu=C0G2sHq=kkqG+Wjb6H%_w0PJAbeAwrI$b3_qnsqFRB7!Ok zg&yEh4|VAH*L{Ex_bexcKybBPU(~7MXFm}il5-N))di6Vg3hvcw;!+wQ6FSAJ)5vw zLnE<=IwB`;TU!hb=pJH29lEY=>czDE2BUuHM@SoG4L9Jo8W*F+@QH%)Qv(szvm~rh z)1ZcpgP=0b+JPm1ny_#YfXIXX7cp38MTEsMbuvK_V^WtW%piNThi%y7YEDvXNYND+ z<7>`9A?4Bx+r>U{&FLD?r<=FDsmABNIkQ04~#cdPQpk~ z(ipcU18A^ozeWt#a1GeN66&A_&x0Jo6@3M94m);pwuXgfpa{;RTyO<*8#sp&fh&cT z3ZL*Oy7pC%LVhAdO?|;{ z4hL}~F&lh;)EOLAc0sWk8ux{T5e~V6ScR4$9)S%bbAei=2;G7fIYozbC=U_CfDKrM zM>Qo>0#BpThjWD{fY^k#$Q1AZ3@SGTwsbOpD1$698%RbAT|zQw|DL0DLbl?p^CcyCvD5}_X&gb^5jR7g9ud=Vo#jyHvo7i$-jg}!Ea z*Fb_Y;S74iMuib_F;XIdg$N_bxCt7i z8hQkcSy>om892bBN5%+-Erx>D2pT7Imt}W$8K-dqITHL}gK?*WJosnjc#e#dSpVxN zc=2#vnZpgZ03F-F4d8SQT$zQi=9Lj*F|mU<#B#35_tCuy73#fs3%v zV_FfB1Nbj*=p)~>mI=rY(v^zb_9$zyP{0swCxc>3Bo8OKLnUHJD;Y%j5EJ&meC_pK zuEvlxIbh$B9Y@1KfP#J52bB3G7ULmF=3|z2sXn8Tar5_?Zh;T+kSsrFSgF8{gy)Wb z5Rr_WaXhhTm4ZQ=33v|>!(+P$8Qar~!Ul{10uNn>Gv5b@Aw#)O#gX*ci|RDHmR)Iu@;KbD2q z;td@ZmomX(?T~pF8WHNS80{njU@#Pa16WfQkt1Iuln4r4dE=4W@Lz<{dmeao{UDqW zLYzBwqKLSnE};(1kcIB`fJyX-?-45KQ6uP*F1ryfS+g}Wk{)AGbok+==VNhs83%3B zjS+Vk={cBsx1J%|SnD`=$mbCDfDL7a8d{YIvv`#_x}=9B4gA@kyy$8*CkbhqH{|69~N<7duTUTf(@8|WR{^A6dwf}_kj-^%4dYtlz@7U1~Q_9_pJwUN8vdI zgU|_$suFv;tP02^uSTiw6_&x~445jQ2s^Qd!LDG}exbN>7}h4ETB;*aujA;0AL@~= zx|Hkbf1{!f-k@H%8m36(Y+*?YC5dhK01HHAEE8#$zB90)PF-uM>uTCaqusbByH<18P;Uw74G+l`XkZ3uD}XGO4gi%fMDnPY+aJw3UCVN`O1Kt;;SInt8k_L7 z-xVQ!P_8@W6a`|naIuIVQY}&?dGa1JlLmL1$quyDX6Nf!YXk{T?+0m6|$6bss5K|bLg*x;LK zq6#(^y<8!+F=xHY@|50b1|JfWNPsOSj2I|FAGBh&F2=bmJY@_jyf8tp<5Xyh25E)b z563VEjk^I+KnG%65bsO0)G%0vg=nbIo`jbWP#2z5unE|Ia7A%gBu?zW5;KL2`^R7D zYr-ZC(%=mwQk)Y}vymrWm&_81)J}b{b13ABW*G)549B6$8aD*GqzVsoED_Cm$bT9O zbQ{cicM1dSuQD-7q?g0P_M$H9pA`Ijw9vlznRmfF=}i3gsG(b9=9PqM*==DWtK(>uWJy0 z()=F$kPk(xXnwZG22pO2XDs~05IQKbd$-8cu$Tw|ZtjtEZxM;fSE^u1kef65aG~Vu@$InNM6c+exI7L7DkG2(v2Nfe>0(;bN~pr zwgI>X2y`%LK~&5IJQCp3R1kBIl9SZ?xIHppId(R)LJJMT+6B_Uw9V*&umKNitc1Z! z)A#|!LO2R~s1hr-AU|iV>dek2&;@CHV|T18C~FwlUUv5GqqCvoi7e zy>P*Q4@8@aIN@jEfDczv9^@epUGUH-DG|1bcY0e`{TmMj+=r073GI;97h!E|O?zAE zq8G!-{<(UN=W1IBv6;8h*3$=^y#Zd_+7e;Y{%I_(yi@SNt|K#HD?`gINS?sJaUHYE z2H_NKaHYk<0(7t}h{nh_>=5B$tLh*RD|;1xmg&0tcxxCFYyp|d_i&OB-4DOwM?^Yq z{qRrha+NR)+%us#+MNox07K6e8>)xmQi@M#!(=A

    x8f!4F1x7*tfh`+L?nKygN*aG7Ldg4@G@dTKIUGI;SeF5H?2LsZP>sHfFt7>%e{hTH_#x?pjj8E5Wx>JUM-7o zSoD+%;Skj>H5>H+4_6*)c3vpfNsf8j*}SK|apcRpf6$ zUW7&sC_&@}VGq4g z@tN5rZsHM>i@jz!`Pdlv%jC1x>`boY&wNzxL>UfIy#I+D6?<&j!LkNKCm$Mr5#XL~ zi{7BH5!~!bp6=?d?#^~G?*8uZ9`EknYxG|4_I~g9p6~k3Yf|3t{{HXT!`VoXRJmR7 z27mAdZ+cd~@C<+DT>kJ}9ujmaQ?RtcX_8@^@Fv0Dd059Go6Xs1un+3?gUuojn`e4P zl!cRn)#*;bU4ezPVBq6^t*pL(LmD&SqOk_e&=%Uv8>Bw-E^(rSCdJ-#u(Uv=#)=Sq z@EAG0flJwv{Snw@jG5hI?K5!?N6ymkfhCB#yKxS$@u%t_QS2v9!a}vMIkfI-zxHZB zrYo)Mcbv$qSP%=cGJ^mu+gu#A2=ulHEzrWjT#5Go&(8LQU-*W9_=ul>_=>;yCf($Y z|M*R=?2b42lwbLlAC~)``L33`bk_Nv|M_%A1w$GXozGW%MUAMB6}`;@mM2ts1_)9( zh1*N`{HSZaN~KT7_XuI7TG0jdHbVqX1;2MuU-K;?8R@#8EQEFpsgn|+<5EpqcAHQi zu<#h}l#yB*6}oO1%^=u+EYH0byw@H9q~X;Ezgk5wC1J%{tSvmpvWNBO;0-?o^pf#vL6KZp?}hC3Du)x}e%Vuiy8v7IU^Q$ogR z)^8uPUb<5D8Yj`^OPDcb&ZJq>=1rVAb?)TZ)Boqrd-deiJLk=RnKNR{+yF)vnbl!z~gtu;dcE=1m^FdiL!7%9Q?%8k%=8@aLetYSg47oA9#TOiGwjJ)=&k1vdMI*j5ichRB=TXe+sG| zq2!Uqz^%FPRG$RB}lsn{;wXB&?EhN-C?QiYxfo z(sD~KyY$j6FwsDR7#{2LMxCJGfd{e{+jR3UVrbkTI+H#Msj)RSfhRJ_C^L;2Hs2I< zP@KZCXf#xRjseFsmE0jOB6v0t$-Bo~nsJ~$>LGMfm~P80Lrx=uM;&jjp~Z|Qij&GJ z4PhF{fhBjiYvDGVv5bvcw>$`_V{CvLk8I!l1n!E zWRz1T`J6a6c=jN}bd-5!nsrnPK%8^NIjNf~Vf7Sb(Ls>43>~ z*EP1&4h$;5P`!!B!JJaQq+K@a-d2_WBQwW;l{KcgT$l{SDAA5=om_T-y7(HVd*FqJ z7IoMH=eifs7>Z4$bq>-)-cupdJ zt678EM07lpxZ_|Zs?0d%RzVbYid(r!1uiTDw{h@JBG?#2YvKS55JCilUBFxlQA7$=Wd`E7Wq1dfPgUBJ9HyXQi3mYL&x9m3sXCi@CazhZHW#d%VdJ&}g2G8i$4 zr~?>(`&FK>;gWlZkTR8#+bSt|x6u@16u)@kW!w=H8rrZKIqXLtu!4k3`cgB0_~2He z!e@^nnu->}NuuPs;w&$6rHfTz;+6;^M&yvOKW025?#x3m+C1=gsN9S+_PC(ryox9A zLJGYmN0lzXLQ^iGP$@r!O2v3k6?=M=G#&a-oY+%Y3n2#jASym*VDKPaxZ_2`cMZZ zVDlV&>9i*gywJbI=Nol6t{K5O735s!rL#l}owf7NTnOY095wJ96dNew4y6$dbw(cE z$WbSK5-;Y}>p^{}l}iL^p@Vv(hPB*;I+Xg?lp5tIeg*9JB>F^HxV5l<3LBeL6v&gi z7BW$(T;Q=U@aH=5_^mTB_zaYW#7aOLLzFIFKm=d6r)U-kk16}Gwn@0 zl{M=@vWQdNCO47TO(jkdAzS1k3uN#_BnfF}1G1LJ)c3|yOpt=|I2IDJf+ZlXR$fU6t1#1axV3>GFZG$kXW-&*7^! z1c{fr8At;AU~!zkP7bCBFykBQ%76c)26mfpqr?CRC`cT#!A^<-Cq%f z7bfl)E!;4}XG;rGvZ>gHgU8WOm%tiWY>P>3GDurT-xd{r9lmHg(9r9I$UtxzX6k4* z0#R=ax-j~bh?onR3Nuha+Y_gF#T)H#I`iA&DRN~}hic{TRv0D%_S3ni&D2k^;up0r zx2s>hNbQN+Qt2|mOC!xMjcEL#?clW*(J=4>PcIcZ_ocA_lR#|0ql8&kxfRtu)x*Y} z{&c8U7RITjhMlACf0>TsyC0kX?d%k=4DX19w%jT)d8F^!uT*%Vm;^?5L?qmp?ZxD> z#j9(yfjF9D)YqfH*`^t`IcxhW_#$~%YAb6j5Q-A?jj_0T;~oEaPs$Tfb%s1=+Wbtd z_XEw|`K(ODKHyfjUC4gUn#x!5;FZ!~S`b^td}s|nMrImnx{Qy@IBTs*`q zGAUyL+JFW(5Cv%X252aQ#-s#_gow)w0WP3~Ovp}3z)H~UPVaPdS`%gPfrjWEl5^c@|OTAP~%~Va@)JE-^H)=zCPzO#ORZ=ZgOT{n5BZhh~RC!3I@4^O0e;q4%P=|4t zhOw&*W|)R?aQ}yQ*qB&72YcAEaoAONFppn7hj-XATlEuOatC#&pGFl^uWN=xb)`Ol z26-@5ZQWK&RflsJP-=JwQRP;0Emv~|)ULS=M4^Un7}s-sS58%tXMlnbNP=Oo%tp9? z5P$(9C<98^Kfd^eOb`Wvll6Kbf8AJ)?O2cfSda}_k-gYOu#i)Fg^^8Jm0j76HHA!| zIV7M2lxhC@Yv4hCIjvE=Ym9rGyCei*vYzVK`jGO(&$$BkVy zqk=HlgkiAV%#B@Rn1*HTe=^hPSxrD%!}ZM=VW)0MT!gsVnvWi!IZeE+Z{w$kkzWe}}f)htm_p){==6 z9oDENt(5qMNf-rc@X?(RLYa!Yo47E&O};rx6gSmGepn#{MnnA#fglhCI+Y1H)qqSW zU?KKQHof6A)TAk;32$J)DcA%u?F(g1#R)by3%+1?z+lOcOAeMBd$_$T9@bi|;GeY& z7o7xFNQG1|;+?RQe>-Wtn}7v_fV2M-&Eo9B@H9b`L_uhP0u3O7uiS}Y2;fEl*dZ=t zj6C4xi_-i`(EU=RU$Idf+6>7EV=12EKdIo+62$TIGFug6?&~t~`-M+ng)>G49?RpJ zh$+u{ONkIh#2d=wyRCiHM2$q{0F;M3NdJKx_~Vz5hi@3Gf0%fMAP9j*cws|6W_v{B zOmvuMTEv-lefC0`#W`sWE_oF;aY_m8< z&6nWMM|NTvf8`7JwB~HiXpO#PUDXHR1jE1B2WSqFEW?R=fMdF&L2BxkI;KH(*u8P) z!Fv#d5ZG5DxalAug8qF6nzl}es0TsF15q#qH;_y&5Xpp2YIgLU!GqE>BpgDf2`g(1 zNM>ct@L~|g=#9o?%-B^n*_Z9g8X4>Viw^ma#vlouf9OYN{y%-QWwth3NzBB1paiNk z=&FPPBb5g@&4E1lX)e%!n!ejI079iMY+jt6mZm{76$n^}>Y2#eHpYgE?hIzo;!Tc4 zyP^hz5^J}niH}$V5TP)gARqR#L0NWcmxgNs)x>wuXV>26hyY30hKOgNghmhqN~lc3 z-tA&Ue{4CD)y9y{u8EQMOOt**#B=NdjEB^M zKRfotrVeicFL2{i*SN++=qrfAA^+!6oK`{Ee?vyG$>l~lf62aduh=i?*K%X1%~M9^R8%KvK4WoQJ92J5M?t4H0c>oW$aMESmG6a zVw!v{g56&9|&RIrrCx7y43x{81aR6NGb>Z)sNQ|`64$-dh5sw8h#|UcBE;dA~ ze-Q5p4^eM7-U*bR6HL70)mEQ%s0LxE^KF=iRbuk`hH^gdbNiC=vdqLB^>CQLp0?bC z6=8>8WJ5+54YcZvXpCl`=tnHK!F)KBCI=sKs0LZ+bWeAMZHPRRrE)(nbyKgccm{Ax z6iS~kzv{!2lCF)FZuCY6!xHcRi#lu#e+rxIRt8K$N5OPd33VuEkq{Q_3-?8*^({1bG~bnwsOq0kNhBa_RoaF+uk&iycV8+>(b#j|;0Jt| z`I+yBbr1%8&-rbjhZO2}-uw4roCloG`IQgAfFJmQ7lvxkhf-Hcm`^^^KJuD?2cG*e z5`FiaQ1o$U!)y161uPxwlmG0Wf9P;&5_(t2_fVInc)*)m*N1s{d$?Z*ZJ2wypZjgN zd%VwkluvoS?|Z-h`?JsaeE)mDuLixp`?}YLw|9GnSCJK3Mxg&heHePdU;6*Md08;_ zPB(U8V0kF##GJ!K1oF6^Xa*+!4q%{hzeshC&-A{KrRux*Nd5_LIM1@Le+JFZd2LWW zn#Tun&aTw5zkchF{+!4D>yQ5D zPx`{I2Jy#xZ7BYE0Df|qd7jUS#e4cbp9gIq|Hl8nZODDW-+lQLdi$4tVE_nx{RR>& zXz(DygbEijZ0PVI#E26Aey?5Xk8F@YlOwOkW(N|;T}K-TMHVuOh;=SETuxowuLtXi|qaz!pA zK5}Bk>YHcQ>^f~~*KTX;_HEm0a^(uEYu6Rtyn6A{yoK$db42#cNmCT)1o3nH4K=AGMvQVr8ppI^sNSZMCfvdipi&*r(}j6Xr^*t7_V0 z^X^UWH*nzn@SQDg{P^1B%7twmyj7U;=+dWCuWtQ%^yAvMb2lq6IQa0tJzjkbd3W#h z>esVBjqez$Q>Zw$f3N>RAgZ)e5j#1Fc8YvH{S-?_BDE9- z6){mU2r%k|1dB;1Idk7@dxme9*byZfGd0a_`SWkf!=2(oE zz4#-GLB1tjHMi9Wose(+2;*f&-gcv5j8$fri>!!-3M<-}f29vSAc9oeUe(OwC7EST zv<+mgu(75qU7=^@MPtdyU3E!rv*e3C>KWsVdr`I~8%Ac;C!vKJdMKiX?zt$VG2;2C zi*~Z8TASw0A`EBR=@XWmuEF^!sGqU}AA`q47%Hltdeg;8QOPsXe{ob8(@d&(6H$Vx z#K#~ivHazre@3m0|M`(U&1_=SQEM0|k~2u0!~!f-aVjUZ#AQmSb#}VQT#bFnIAgcn zmS`iC&RII8qjItGQ!A`!7TvYSQA@2z_;739n+wr1Z&_!hm7J1uHOW_Fk!|%WDy_KU zV>Lsv*4bC7qFFGesaou%Hh2NL@xsG;#1B3p<3rE1e`9I44$3M2D`{LyE=nk5<;pxW zVOg<3h!;8MOvsZo`}{M|K?@CZU*TrO=#rDVrIyOZ=`)td`0#VFeW@+&G1Uu=8XqdA zX6+C?>%i68*A+GUY#&t_yT+_jT&PMjHYvnUuDkYn3^k+3vdXUwwi4Jj3`HICZ~W-P zk2=Q2e-YCYRK)+GiZc2<-Cc1iTh1D*PywQvot(1_slbieR%pyn4yq={TG4EDUy7z1 zX_BlZcihM%lZpy8a!U-^W5ReEBFX-;1y|_b=A{+Ft(1OP>r~36S;}WUO-|HZ#&ENnR81R)#e%tq_( zC%^#`uz&?*-TZuJG@(gpFRy#vklM!@lxZqz;JXNz2D312{O*GN*qVI2=D~vKV;gNE zj4Mzy!iNyUSsP%1N~i)4U~EEGPk_mVHUSK5kRyHbU`MVP)Sy_Tf;H8^$}ITSiofY7 ze}$@%&ifjM{|jH#a0)k^0vHVOi9CsJbbWHdQ#N1(L5YigW}J+aw70R-^=VIdk`%n0 zCOOGPEF|%`9W^e-4-*PTb(dNPsPG{<;dyQwtUy>Qj7P9?;l?s`f!)$1m=+3lXH$b4 z+(5*KIyD9=aC5sPC`IN&%2jTHDN~CCe^+>#dGzmusxu#EwzaUk(5Q@OT#RE1)kQsl zj%Dv!pUSi$3?~3^M}%;~Buoj+Wiqpw&U_{`3+YQ=Hf)-AV$v?}Vl*YJU+am6f!sw46(0)E-kdldIe>=)V z*%NVLh&%th1cx)}#4c(h3|6FKLBU$ZEbk?ZPZ*-3pWs;|A!N1|U6{ zk@#?A9tdp_T?Qs7nNg;V4-{vRe^AL&mty6o5`>~PLH9C4^^+>=0HApYhQPUY^`>s@ zkxt*0o$cu3Q}Xb{OuNYjPqekIJzNKgECn4Ev8apOL|~NI**oXuRhe>>T=m#v!nj8D zok`_dP*$joZh10QAiLxoQ+qOPxFM#}yymugRH<$X#ub{l14=EF#YtGne{Km|MRNxA ziNzfS|0$m!W49JY2t$OSxC06VA!_=~0t&;0CSXKHaf&owUWb$r!N(-BAX%=k0T+dB zNN6;240>LS9G>g!1;g>6Q!S)lsEmtj{bWl<#*8t}+1_K+WG?@Xw?^L7%6UT)4CFQ@ zPy*C$T?jFW2~&6yp1=i$e=}^0U~ss@#&E_pLM-ADlh_$1KJkf%q2d*@xWz7hF^pp@ zV;39a7K|vcRwf||W6Ze6GJXa$f-K}A6B!!ccn6Y`tmGv#S;_KvGL)nI`FBoi4qta{3T6^~DPp)0v=q?Z^$Z!?zTPHN-)v{2GRe;@aHOf^EO=RNbe z&wl^b^FG3HrJ#KI&xSrUqC<9P9?gSfa(*9 z9v&8H)g}ZU&VDwuf1@pJdDKJ7QLeVN?m`UCrsEgjA&DTC?d+fBIVv$&+J1z3Ntfi~Sk~x~)6* zHsGRPT!YZ*FBF`0gIO*Xz%Xko-Y^ViFvK1h!L#FCDh!RIHLV-h(nrQ}8@SL!CyvE- z3x%N$n)t+Of9eEvL39WTp1AtbST6a=PhK0ipgha5JMLK!!V``7ye(`&iP4ij6R$7@ z>P2x2WAJzw-hhU>&CP9g+e3FCif4Rck;jCKY9|^AM|G|1T^!s& z9ca-c5P~M$9p2?#Al!y9l$0R^!X1!WN?8aMfPp6P3<1spOts2PnGplB&H=$+UZ@fC zSsW4ce*hGe0T+}(BM3qxyuvNq0_>?9x7mX|+=F@~6@$bA@NplR#6v7J$nKejv#A5R zotuyq;Vn?Y=jA~exPcpR0T*}yBjk+SEuUSG&s~s@O*z=jL|o3~!HyZ(k;wx+@B<%; z3_3|6nT*y94k95Iq9Gn4A||3DE+Qi~q9Z;ceX-0Xu$WQ2UA0(${seZ)g5G{`YbgxuH!JX9N!C1Wx|!!3wH zeN2{L&jfSW@JXXL8qC3M#|PF_5=~>g~+AdDV`)s z&Pqy#0T#?%c6}tHWX=Y70ZeMuL*kE3f1Q~DB8JAS$^Lu*FRDNe^uP}a;g0_q1CSY6 z62ik0_JciK#8h=n);Qz$+z~c5n^QVl?bU-iyn{D<1CSx5DP)2mxIq~t0WW?4J33fm zFkqi-Q>guy6L^6g0@*jn0~6*Wh>*jgNlBMEM4|=epYSH2 zBm-ndres29=vAg=UM6N*rYB@(=y?LkVO&tEfENUoiLgOvo+fIhrfRMxYqq9qz9wwO zrfkaQXm*Qfh9<>rrY$5xyf`LfT9Tjf133tq6$a)Xsz+ZkhA{M%Ugk%qA;E!!L^d2l zF*Hc8v;h}LK?pDacXDS0NWcZSe}H(pfZC#f73BENGi}Pnt)+u4&xM+ZPvr%}T;DiNC_(}iR%w%xnhzl`f-PjiEvQ3Y z>ScROOn||cmtIVHn8#m^=|{|-ny#r9#-5v=+ln~@CES>$c!8FBA)e-`p5`ef+^3)Z zDWC?bpq7CY6sn;fDxxN;f1<)b2sEmrJ}RVMKzB|mrB-UC!j*Y=!3FORET*}fl@RBHef?LC@6anLre)2n$|;^b`v+OC}5fle{xJIUog)=`cKXX zB@CQv56CAaWCFS&1abBXzLAMDNCBzlz`5!{Jx~os@I);1gEu&XDD-Ke_COFA!OlFN zgB?#?8ir+v)58?Urg+A2KEyr6)i&53$@EZ)V46!A9L|)19(aSaqA6@7XJ4Et$%cxZ znMgG-XUg)%$QlHL z(>~d@vFYr2r78I5el~&>sKCuK1{-KV1bpYGer>0QZKw8t9^68z*c(rM`n!Bj)a zo`VZ*Kp(scMF?6?$VVH5E!YlgCOCsSOc|9CL_2JP+o0qWe+4A( zLO3)CDCjCLdI39fZrS-1X0!#$)XsX$1IS$pKQ#n#)CNA~63&F|%gV-P;gc)m%k!ej zKKd8;cu)1hf5|Y^W`l9Tt!!_5Tp9V^!#P|n!I`EMY}+}g*nIz%0TD222!QU_f@-Hq z01T9Y5uQVrdI>u;$SJ68sw|zkpsZdxgX2a^MAQTB{ls-@0I*&_qmr#CK*Kq_170#j zGoT_D zYemIS#BY#;*f`bl4hqgxFuRGbNbCbL0D`J50w6qt53_NZ%mbLtP9h(U}+=3Ah z(+xTSe-xl6#>|JttvmulH+h?mG$+LjF- zUy}+|FEMlo3_XM~)atFC0x0CI1f;;|Ho`D`Ls#~1L%@S7q@*eoF?+~^O!mQ4Y%d;r zk#3x@nG~AOP3h<$sTCU>!q9=|%CHIq0Te*Nf1#p5eTssXnwabPEZs^3Jb1xuH3lJ= zEk*b&JpAj7j$$%mX&nMJazo z@Tdliyc!x~sVQ8;AyY&*6oM5L0Y+oA6K!F~Jf{9|VM=XOHw2iVh0V$YCe#BJximxD@1u|#N9_pLF zun)(Q)sAFENTLEp?FA>%;V$OPQ074Vf1WE0D5{_4rzjAE9@Juhc>#aEv!5cW3&(zcfR@ z!y9BYZWqBu)3%pbbZ%QgS@SlCRJ0WocX3-nI16`sG|R4nEq_$RJ!~5^7=us4f4~L& zswRsn*&=m&L?!Ng4LmFZDJ+98d+9xt#C||7LF|Jl|44xm(1GtU2q-v#8*sHZJY`69 z!$mK_5;TE-^S5p<1DX62DqE{^7uJ_)Y>F_njFir1_j7U;3i-_8@>!g@lCy2*Kn~n< z#W5cnj1QXx+`{(JX17AaXvtZ4e@0uY5-D7CZWqBD2>5J-nex6iNqK=DMCXAIge4S# z54^aKTY-%4cyGW%A^3Q1|F|XKGLgr|GTioyzX3@%d66Hv6|ex7WBElF0+Un3JJguf zGODKnD|iw$5i~+QvIjKEhXBLLM;mu>7s8M~FEDoq5(v~g+=3qTfC|93f8E5wC|H9r zcrsO3!jJpFk7x9uFS(br(6gvFmGd543y&dj4J!ZWq#xbLD^1s9k)NSpul=>39nxG{Q5aHE(wVlmoevN4ji~0+3_04*<8X$A+#W zI*`9Xc^A7!w0NQe`J9IYe@~l&Ae2E5U@`-IX9x_o9+<)~_r{ukH>y0jk0Uy=qqg|I z1A1>j7x*;2IzbzN!mf-)DIfzZECashyS|eGzxTVp|GU3if+diGC0N0;o4cZKG{Fx% zz&HGO>wCU;gTzn#viHN;ypRGPayR6A#d~R;MP)2Z(0#$yK`U&if6vSU1x-RymcwY@ zJZ#E7;9dx=IQJcSun&81^TrAniF4pZ=LN$p|8)Gzs(P?zv=tNr(kH#rFFn&YeIW=u z)JMJ4Pd(L7y~AHU);B!FZ#~y{z1QRWHBej8-ovpMeXEnS(Pl8SON5~py$|HP_~x|I z&OO}^Z9nL`*~54@f7X4{cIi$(!!0xd7vzAiMt}r!J38}*-Hrw|Orx3L{j&?Z55)L8 zE$zlbJitqRAuK_wumB7=0UgM@5P`xRoc0@>KI*6b>BBkex4!GIdfvaj>|25A*FNeq zz15R~bs@nrbVD~L_s|zY>K6hjboukbn|kSJg3nQ0|I6_7e_NSp6%9HP(q)FjkW;)o z&EbePLV!@$aQ{f~y*yn0*O$Nf|2ym_d%ypiKl_`%Ggw0z*UZ9L7$uOm&rUS3=e_># zKmP+n5rG2<7Bo1}Nl{A(10VmNa=1rASMp zShjTe5@t-9e=?WVw0RR}PMte>-Xvwk$HpQ)-jM2f6zNQ*OIun>`SdABsZ*tXv?5es zN2*&TsuZb(1&a$8B$U0N0fv-5me{VOiL_f>xpJ>{%NFa@DOAVe_4E4|aA3iM2^Su0 z32Tu;h8Z^|Ja?{6m~2f-eMrzs*bSRa(8-$@)SMp$e}f+MaT;}M)vHk_7RVa5YSO1& zW6o^C#O8`5*|xkX_nu?n!-*G%x9rX9|I$_#7kB;~di1~fVxh85P5NJY`p9+HHxGRH z@awd#+vZAEdiCpN2_wHwe0*~H=~VU3?M7(h`f2Mi)=K z@xh05+>yr~bLO6;mUtv#}~YiTS( ze+xiZxe!BFz5qlERn0y3uVorFNUI2UbJsFP<(-$_A?+nd+kExiH^~|K{deDc1>T5M zf(Q1O(nv30fC00v+@RWhe|Q!YL}Ig--i3Y)(nNqCb}-A$+&xG9gSylJg~;Ad_p?xK7Q^5}aQv zVi|O!gQuHaz5e>?u#<|Z6j1Vt{{@QMW!?ni`^)n63tbQ$O**~Tlbpl_I`+6l9&%uo?S<(S#Q+2gHbcK5!lMvY zBT6@hNDL%2vJZ}l%g$QDw$f3=Ar)lNK!UQGgLQ<3AoJ0|hLj>wijqcicoy>Z$I7gP zQXqI3#kXAH3RWzFgLZ?Ee;qxNEIfe8I>$&`OqKz^o={SZBDzN*5);N~0_Ilw*ai+| zaZPUqrX62YPLr(pO>82Kbv zOl^nV5a&SNfX^LT5|gRo+9s=Kri-*|T1C4T|CH1`Bigcmxa83rN@1N-gs@$9CsfSLwu+fV9bqC6dTaGxAQ241<9iL;;O0W``XB`e;|hSR$;H z)31(^hcPH21r2EQWS;-wTP%)&9lzA04<%&*EFSB)xZd#%mbgYGFZriFoFkg@xW{e_ z^g_y%q)rf;*NI%`x~V|Lb4ytUF^*wWWnSZb_1Nunsaucie|%9kL2>C0*`Zava5b~& zk|HTfBUZPzHoRS2#(`GC*Yv)3IQJMYW=K(2_}+Ig>M~jp_KIKs5=I_BDM(Syd(qyJ zg-Rg^f*y>aju3S>K&}eMSyxI*^ZxfQ?-+(2day8)wSkVw6scfHib>oGai(2UB1|{a zg%jVmYf)P$e*#I7R`kVoO$^HDWEW;?Z*0ncpA%!$bsY+6~ zg*vPo9ez+l&wF0A66=xHg_Ie?_t3B&J}ZH-Bsxrto^WG`QBo810oGftbfmf^z zze)!4f4K`2*)Sbr*;=FWv)#*Gyr`Pp`PTQo%?gipGy~x6SU0~3UT|8QvBP6o*ae8t zh-XOW*vUSIKE(fc>zaf+y!BAWJ{M-~jVk)ds37gO9a9fTZ{W799(RkKVQ|3&cqpc> zx6AwCZUK#2YpyPK5cv^~d+&wjK^OYaqq7fee+(iHYfyR|RMA$VKOO4nRY$MMA|*r% zTRP!DWTT_*7-jNwAC9!2%NZrTG&H(Cj$ZUa z6NWhcG$OzE|DS{RT;pKI@u|;@!8@+-$woZU_u&rDpK}_$&;|9!KaTIK%II;p{Q9Xd zg+XBB{N}O;;HNPTcQil#sj`Rhbus>4e~jb${7W4g;lMoZ0$@N89LzcJAu7Xt9q;R8p6|N5{0sL%Vf2G2rn`+|<~#4iZVFaLzd9;VL*ZEzZze^5l= zK^3L|^QHg`FHaSyP(=1m{GyNvy)Z=FK?%w4_=eEG>fs$sK?+jfcN8uJ%D@x`ia4^b z?;H(Xz|j8&%MSKi%>*gmd07A+O1K@UMB1X+L>ROQ%G;S+$c|9*iLsNn%cqy<~- z%An5pLO~Mz02KW|5o(Yb60f_Se~jx;h5ve?7HBXOYN7v15nX)Fs9xpiWH1VKff>}0 z49_L}{4W=Kfft)l9g5Kyjq%@r0pOOA8G#`Vv=9r!AP&Ny{je|&lu;R(Q5%~97zFMZ zixD3F@cUx$43E$M#;_Nm!yT9*2>Y-XtPji1VHg@A2N=fVLZAoq;Kdw+e;@d62Q5)+ z!Y>TXarwkio{XUkVBqy6pi0_+4hk{Cjv*X+FhroC3nGPn-p)DLffas17d%lCBq0=Z zff5r@|3p$NQc|w#rXdtU(IrLk{}QhVKk5iUgdIIm5kgTBBmoxD5ml&xj-YSo%+MSq zQT*UB9nlX8CDACSaNpVie;c9_8=8R@s?sVq0V}f->EeJZyOJxz04&2&ER~Kd%aSbH zFAVJ94npw`?4bRqz%1j^EX7hRz0xbQGAmV~Dpf%ly&)={j^BQeDTB};lhQGMF$!(a zC;yAkdW}jvP6JXv3Wk9=@GRT>&Ron87~OF(*{RoljUht-#LVs#e~tm|(B&NZAW}@A z3%p@Oj_nsh!4Bd;EoajXMp7FnGCJg;3xrQ4;cFj)Q#gl{IAbvIjuSbPQ#p}S99r@v zT~hxNm|;1mlRB%@IGZg(}5fEOhCw?INy^v znQYRgum3=jCztS?f8KK#^G!2#p%x@T5$<3V&(QG-)IbjuK@(I#6;uKnZ$TgQKpnL4 za-kDCfkGuA5;8Or?qCo)6zRrNF3pntMs)q4(L_%aMNgCp!eA|*(+LDX64=*uQ-a&pM1qm8a zAK1ZDKb0NAe}NiCl~hTU_fGXxSwS|{k}d7P4kTd`P*qlCwNz_$RKvkjv!ORRqYnm( zx_a- zx|K~oGAFCm4QabIGShX`* z7j#<-ZXCu>5vH*&(RDN5RS5?ZVvVvEYM~cgkp@FDB(+pa?|@@D)>XffF2k}$vvB`L zQS=E;7G+a5Ww)?KL)K+mv|3+uMO`*zM|KKS7H6NJWKFb2%Q6hsfGfAMW3@C9Lb5_F zHWU!_e;1ZEUL$l_nYC#dwDFEl64Z}h;{aWIp+TdTYrED#yB2H<)M3X~TOpQg&lYXl z)bWxp2R@4dfF}fCfC~&EYPTU3K4UL%A#54c@dOfDm340e7eO0T9MEAJ4519#7PK6a zd$wv6a3LC?R&bSd8lu5+FV}K$L38uK(vTnze{f+NKH(E^VRS!X8$1DZQx|o&VG>%` zbz4^x+A?PCKoefqc5PR6b9Z&OVG%ml5SA8rgBN(4Ra}j7T%mPhlUI3_7ipW<|9PJm zdYji=U2+to*LtrPd$U)2x3_th*L#<@7d+wTI+9|YHDZ~v7pQf5NAi5z7h-FH5*rs1 ze;?L;>(_qM7Jsu<5nxnBv9@9JH!4cG9)dQ{2KUd6#*@!e~}ueAPlk=fV-7}pBRdxSc=`3AOTcM8J2#h zSc|t9ijPkd_5cb}fMI|K(Q?2L_5cX8*#EwDt0uue)mVVdcQBjSiFIKQ3?csrQozGR zlMc#`QcZ9Yeu0h^HH|ev3jD+bpgdJ_~JmKhuzZ(WV*6jDJOz@Z$uSc|_`dHoZcw>f#?Sc^k}e=Tnd zwJ;7w(wmL4|F~J5*Lh;u)L{2le}$iuD?>pX>e-&{nVH3*@uq=7D^wB^ArS&}g4@z7 z#d2iDd1r6 zkJ)-lTBSu=ct=`yL)s8xTBZ#l5bD6D>wrVQat)GLp?8)o%QX!C!ubsa(7`DG*|y~mm77NTXR)+ z5>Al1t9u)+8@sbxySMwgKcSJo8@#{!4&HJ^u`m(B`@2n7bl01`f7koEs~f)KySh_> zzU!O53l1A3+g$Bn6P6Je{@X`YVH0+=E6;rb9uyr*m+}`qzxeyRN50t8g(0i#v4Hp06`E0 zK@ajE56l1!0>KOxf81qB)MQUKM%NFhxl*WymV{3jX%Te1Thx_bn+gx z_RGN`8k#xzdZ5}qZc;-|<@9k|W?f1juP+G{xwMgg!#ArM}>-A`MERXg4d;k5^v-zV5ZkKHakRH6r-;0u1D zD;nV^T;V6&!E@Z62dR zm;QrendzS%>Sy?eiCW=7ToY0^`82^6WctA)-sc}2r|Tfd8+sZQ_MsiRh7lSXZ&+v7 z&n%C9M>|xq1KQg^aW?t#zw`dTw-Ml!K`NyZ8nEFR$iW&4-|(%$Q^P^=7vEN?VODuB z1tVYbe7NHGPAdMKNO47m&G9efifAK@L0rTPzJvAPVANi9X|M}TLAMnBX>B0HQ0T;&e zIg^$hl+^k0;qbS=@Ie(+fuHef^;Ab;5xf8jf5J`!LaXdNU*u%pzUiC%O}@q-UdJgK z>>b>u{e8hf_J9E*j6i{4WU$zPPzIPVaT+?b12Lk+h<@x)nKFfnMvPSK`1uGjq{xwe z?wC2a!GsJfeiV^>2@~Xr|0KFTi&YeLs2gMl( zII!Tr4htJT+>oK5RH#m!N@WWUVZ??##{76Qvt~brJf;6~d@7cwL=Z2Rw5i!n;lLVO zU&II-T$)%o7YGo{nj!J5tkTU6~1QKYV zLK}268fPj>E&@ociwJ_NAbA8CWFThbr58_Sk%E27Gc>DNWTg%d?X z%OZLRS8>ZN;}?@1KFwSQUN+y=8Do&5l8PsWBm)1|S7c7qEllf8bS{V@!Ui(P96vop zZmlsC(O<1KjrDA>@pfKAf2oZWUf5!fO?KJ26peP;YF~C>fo{7kVccKeBWJINQjY=(0e^D1xdy6^sOAHM&-d+@sVe{pZEeS`(GOf(T+ zeDlsffA1X9PfvaI)>~gnn{0;4J*;u|slo-SG9bYPth(wwo{!>rCn|<)D$`i8u)n93 z^;c<@l1w@gKB2ABF(=UMX%x1|3$dyehs_af|7~f5gF(c_0yDq?ZE=`F7&uoEZG|Ci zci6%M5fZ`aWF~1!e-jaLpoYQ}l1OVRe3}VgXu>eup$?woiN;`v!;Q6Yg-+Wc)C98} z+DS}exhf*Al(fE-9qBArSwtN!69^_^WrFMv!uqtVBjOUJ1)s zmV%X;poCFuX+%ig>_WW6oO=ifOhJ;SC4>L;Ln1r}Of)KRnW+(7w&)1WXx7n84=Uz# z0F%ISA!c^8f9s|<(eS%)igSp0FrqmHK_pu!B`sb^f(}A(0r5#7P#5^$k(yG)oyjQ` zYU$^6s#!)mC_)qc90idY;YCBx%qMraf)Iq@0vDK11x{!~`gW!jpH(E7BAM2H*0!LU zn8^$ube0)FXC}?aaf8Gxqa3r@Ad@ufSzs{JeR875f1Y&cCwXa^uUw`(Z`Q7M%TZkp zH9HF*rc;W;^DZUC;?TIx@Zvu_>=n#lg{?bfo z8mp+{ejZ@+bwUe2i)KacevF<1S00(2GNrCBW{?3ai0s_=t_6G(mjNAuZ!L6YInQa zJuda2XS`iOY>16q=pl8QA@}ssm%ePtXp30df2Xo{zVsC=A?u6Z{OUJ8r%{9cJ)2VlvcNNQEXCSzHRa;cje4RCfhWoW6_R^t=m*>NyV&UFZXBeHT2f zV$3;SwyKleW6rcDLdkZ%R#C&iu5Rj1wy?#6^Q32H;s!Sgcxbev9}Ve9N7_g_lM^V8 z3kXR&`gq0+ajxFy2Q+9J(rHEnD9XyGe_Bb}w^2~E^=dVT;{9${y7Y;Tb)r&dvXM1? z=1m+xOxhW1%Y{eXp@;~A9Nm3(+4XqFD@-wrZ!EhzfrJAf5Ru(E2%;otkb$?q4eoGz zn+4=9Hw#RF?sTKu+$JzLxzp|L|9G^PgOCcgU zk=5re2-3W;$VYzG(x}Qb+w}hit1N5TYh^oXNV{3)!i$%bo%L5!{~!3r7-Pg>l;m)9 z2sr7~jqZGdv^0nkO6SJt?hfe&5s{FP5RidR7FUvLq^ShqS_z zdA*-`q9Iwrphuys04lmKdj>FvsPdCsgPR2}v?X01@O4C9z2)$-82!LMuMcwTSG&s+ zI=SACyVj&q-9CKXL<1|Bxgzoy0A!lTf_#QJxIveSH-u)#lg=pwqjllIn5CQ39fI%Y zW4$C{>(iB)lT+AcPN$^gP|22}*U4TO|HiBC{i1 zGptQ@x~ZA_{bxH-i91o*K6dN+GLfrPe(o0Ey)3*{8GGK@!%AUEyfGyV(WBPXLM2gE zyhu?Gda-rYgtP#`OI&0Uam-2#YBjV?QlvVHC~`aY6K2p$+n15m8d9aygNF!>4_>`4~D08&dL zWC9R@vl6D0$=W3>ge19tIw|Wjh-@=N1Q!wu`6fmc5-;7U+Wh0C`dgXAVNFb+W-*;^ zM8=m)gW>afEs<2HoXp7Q^}LXfuaJP!Ap$l4;6LI|&^MB^h47uQMxY{1RDB*~`oLY8oWx&(ya4c3c zeJERx9+G&K-%g?bFYhJDPC3Y z;dd*zYZ6e-oh&?>T+qbgX6Q8~X%U8szlcV?hz?)d`-n}74rlmU1pi%hpR^cOU9`?n z%;j9n6I1-XuTUtvSnzi-7r(jU7mGDWwh+Wzn3PoHtHl>X(ucoG9!i@(n8PlKnLlwZ zRry;KB~`5YwNy*G=xK|ElyosPic4m$L_4O;)cDmBtTVIam0^RQOd_1bnS9z+b0HD?|7zL+2`j)hi=oDx<3_ zy__p!e^(}wR)tp=C97AZJ6Fw#l?nS-<$kS-$*IaGtu7+1jOVW|bFP+Fuda|Tv1SrK zgn{^yh=Wbw+g2bp1mv*WWCnwhNKPfoMwDLvBJ1F|FaLrgo^hcG$nR3sI3H zNnG4r`$N5M%DL{Rb6qj2WQwkK=6BsPY5i(UZShgfx-Vprzy4Qs{jPJ}ymaZr*ZO1r zx2H@}bJA6jfVXGWZ?CKC2ajskqaZt9-xA(wH2jdxBs6ItscBgMT|b7&c}pf!Tg$|{ zokZBcUvrD9;d8IaE2HQ-l6+-aLU1j~q0z|oqLC-Ik*}ul!EmF$2>D%%i^mbEpvHT# z7w;uv-}CRjMFv3nlHNZdYeGM0;=CsPOH!*4+oV?0Bx+XwNano;*$2G`AB5(cgf5#5 zVn3KLH~o{YDqDGPO4e*WjA=x?$gp|Q>}b~bR`-KtY%|w`cPp4z$M-sH_LC5fLN3|JzwxK<;0aBLRGWw$PCt2Hfverlx@l&kkS{QE%M-);|6@I5#+(%XG8J< z@~_x3-2r>qegl?P?T&$Psd9T~xDzR%^WQhJwN*k<KsgY92y_e|bm3JH(^D3O052~&Aa9Qk_?q@H%TF|V&9j{F0T{Mencm!-eoe5AW} z1Qkd5c4PS4%}C#k`e%GIocZW<9yBj)w5MZ~orQAoX1KneqLk&^y}g!=h2giLZ@qOb`T)KZO6V zwk1WvfabbjDHu>0Nq#9aMr84Wj-^&!Zjz*fawdsP#iB>uWl}S!`)d-}3}A|+cCx^1 zinx}d%?d+d64b4(HDw9yR$rX5>4e88fdVrL&E3hU>$=b`(|2Z*ui`(Q7f$+W{Jhwm z^b7i=51$SceD@SOZI}*z2(Yr&-68&FnEn){hE^NhDySQ zTK$H`w+*d>4IPS2J)unl?M=hGf8BrFDKG=>WCN&)Sv|@_02H|oKS6BS+-?pa$@>tL zt}gIDM^Mv$87ZM%~d9sK^8yb}|&_5AVgc>-ue!g#FZ_PZd;;)clA4+@Ku;L8Y7yBy$0 z0r92^oE>;CbzfN(@eoB%?Ir=0Wwh~~cSBUiwwOAxm117OS- zYYz%tcY=82%rdI{^c&IiT>-og0|QDS5?5bQDvs@^vi>|AgUo;1N9!Kl0-*LQN3WSd zfJ69p5;0p6s9I?HWDFwNxi1Gho+5zmcI~t2f=QR*X#o6`6KSX*X#n5B-+#Y#b&ux{ zeyqMC<3k;V-jb`W9I+v@W(c6>g#`F=C*vESkL(Q3_k)E3VvH*}?_12iD^k1<(dUPn zKSUiXb`zQFzT-oYxOOHm749RCe%N^6pMQXyPiC^+3FKhs!Ysf;OZXqaC#&y7a${#Z zu0Mc~%fNjw;r;WYT%f%>+z&vg20N!?2L7R7UB(`YfB%`ba;n=+^n5f^W0`WfAyTU0 z9Q-LWzWc=H7Aki)hPgBc5CuR8C;pM!krDz2eH2%&=8KN`07Okf)245L2@lUtD2@hn zYhH~0mC^;Xeu4`4Vu1>I*Qk18w$E5Eu*OrS%p5KYk*dMOgakSfI z&wWtew=+AP-!`|T^j0@Vxj*h?$!E1MB=K1A^=sq%^8mf?nE%S%PX0o7sc^&ZgQ0IW zsW^FwDCHO)(`mfR3;5a@Gi-Wibgy43x2l#M>z&~1^a`R7Jkv+lRs z&v|}(Et;U8q<@A!+^2^syH3?yxXv^L@bdNSu&JNEoa0ln$CdwJc+P$(utZH~PWuDF z_7xy@MXF4Z^CADq7vJe$oo;cVS&h8>N{MC{8;f75dJixgZE2_2v z@uwwjgTe=sNj4h_Tqo{`VH}e}{2ZXaqW&tZwz$H|%6Mt#1B5!0)P8YhV2xw%F>nLP zG4z*;2+Xkd!l|43P=j4QB@7qB8(TnCMx`G3UbVoDHb8|IZTYL=ORVNgh%AFI=oef< zelS!ywF3j(y({*xDOLRJmK0D%r6+Xy?n{8;E^#@`yotB3En~icvqOGtxY(`GZcuH* zHTk$hK!$U#m{VWU^&RY{W{CG|U~l7_L+<~MS)2+|sM@5YKof)et-yywvnl?9&9Sc8 zNvg5rVmr0G^ku2Tn$o16!vWg+70H{Tf@)G*Lc$madA-LS4R2V8tf7;DmJZc4me*Nl zaI`*RP1Y~Q;8m%|*HoGTWys&-fe4$t|DFP##QjH`U+@SxN^i>_I-Go|u*dFHNZ$0u zzchWD4qg30d?}Ve1}P057ER?yWD?d?q>5*FNF%Fo)PG(*bvxiG>R!i2en z>#$>lPNLNA6nP)ZGIW|L+D$ER<|sxFwM2y2tg)O(erMo_*Z@Yfr6As|pT()I4e#hf zLMlG$ON)#;+7!&hpLP`U+GoG6oOL++xCv}4G8p8jdLmfEml%Dlm{6BalA+v@^7}i#kpn+U@GrejG`p$Bld>R=ue=v9wi zW}hyx$KdA|$n3F8-xYZtXA`lJAjj~-E9alOL~%kDF9xMPZoXkvJ>o=*3?k!JP>{84 z&#@gNUmT1k{3szbh=4MsK~#)4YV96IJWtbJ$beF413>E9Y5>u8m*{kRD#gLFsDTuD&w(*BV?6`7wZW@UOUM-$0^Bv3aP{i0nwwf1%$N^5wlYe?z zY7v~Vcg%LmspCC`ri>FPioGnVWhDWSVuKw7H^$vJ5^hZ2Ps3Yekod6-^@PzxHx%n2 zH@OI8ityLz{Hsi_hvLx3%@hhHrjU@!({QGda6+_zpI~cS%4u0-uc;&NqfxdDagT7K zL)7e?O2nj!uxHdqO4O-JBea--%K<6ZN11AZQI4+20jaI`>Rtt5JYG#vvz8=j)ZYv^ zdH23ZScZJ}IjeEDh*1L{^U<^WzANOFQqk0OihJwS9X&_RUWgNovrYN$L6>#0PTW-v zt<`pkDw_==mnMuB=_DmA0_;!+4)`pr(Om_CX_J~VKQivnZ5NjFxy4{;n_Ia!^`Is- z8Fvb))(odCl*!L>n>c6|7K$aO^Q!g8`#Lq^e<7JP)i})v(gGYn2q~$gr*|#jy9Rshj{E zn8H8F%pJz0-_<<$o(+@TilU6-q&!Y%%PHz~rV!Ny_ZW#0IR)MCHy#ps6~n7VP6l+< z(N7$iMoOC_aNyDuVnjk|0ppeV6F+{b;IeKAtD%sxYqE%@wvi#V0GlrwsCRr`s6;i) z=*C=J%_=F{@gya-RH!&-zN3P}|H05y!+(zlHdci-!*CV_k1$k#WqRaQIK7V8Sl``y zPR^eCka%6@UIvM=9XVyh1bffM%8M<^`Wd*jmkLhB%OZ^WZRwElvmQF$jt9Gdl5gEq21-*gsae+0GiW>?dH z%5mgjoAJLLeK$)dW1eq;W*yN}}! zZOBuGpXMPZUm8oHeyMh6sY$$)mbE(g640GLx%_xu&9pBy=0kZyaYL3jj#pGiMq2Pn zt$bib+^5=ecs;Dz`F-)ukaxs>DLbX~l93$5V)~qwU39kR#ioQI!w&;e{u@q{X1b!H zB*lmHjm4OdJln3fCWRg0EMAqJWRG^oqY5^ay7?r`S|I?ZL1Phz#L7L|3|5}R38SF^jL&m z@?z1=`IkGwbg8IGEITPq$0BWpHV`h}PEn`9sDihRIjwG#Ss_=iIsGm*0@$9noeGYP%#JfG|C3TXGS-`DGY>ok zCx@hY9!|;_8%F&y-x*%XwT_w?*dVZYp_nW@rH34letwE6DYpksM2=UfO@{*XjG$SF`1HH7bcJy zytr1MBuQ**RRs}E_}3m~M9l_O1?nNeo0m<&g93>(PfpCJvWJ@Ee8CfX$v)T!N7YW^ z+01J!uC0nZCrLyo>r;+L{}`!~?4!2e=e?Q8h`IH0GgYqKxjb54AwqL$c@{RAktna+ zG=D5OkUH9bty_+ys12ZO_YTs4>1(f&CT>F^;u?~f+o+rv?mI~`-31D_6;?%h?1YeA za{U*8{ukZzvZHE&@lC4lI~K`aH{XZ_lp z%B5tVM609nYVk(3?{+0LtGLQvDyS;r#=eRVUujhIWZ4$F*Hkf_9W04%x+Su|M6TGV z()ikk%_t1AJ|>u^cAk_(_$J23MGJI1d|Mb>zVp?b-~;U#w)(v$FK=jVBP4AhK~pmF zscto*fefXu5L+mB&3sRE_3Kjxus{#v)m2o$N}}9xyxKZYy`j&CH!+M5k4BZ$l7D>0 z`Bhn=Z24rgLO&Jb$WVOj%&<`y_rDVx$#XjrC}2w&0e-rRDULlE`bxk0;N~(t+P91T z6`sk-&Mv9Vlq}p=EwrUCxY-IY&Vdv#>K}%Dj!23;S^<`peoiJSMcmTmTr#AZCOiuv zV4aU5YsEgWAdNCcAXhc_d&Uxv$*yyow|YpI$I! zH~4-+%!_{9ffP&1K`*7*L|T9(jR?Bjq{>JXQ=*S54szL^1W8mz#owGXcYe(k8h@uI z#wN@RCRura3@6|^s}+V!?))Y|CPy;uHYV+epoT>Lh5?JS$$qCqO4>9ZS$#ip4kK&* z52*zl#{hESFF?uEVbfOi5Cltd;lq>T!i37)K2+S@Fd{x)6IiF(eRQcW@>wmu@(}|} z038L$Cy9pnrN3*(vaA5#IFoCpDRQX+Np_GQN3y;0A?yu&*E0{;qafy;1+>NnGJjca zeIs|I4JKBzJDDR;rmYE9-Ay>Kkh{_2e8>rAmHwu~@Tw{!$YgHV}4gtGy(X*ZuP2UQ}E2?~V%0u}CS)?oj7N=Z! zpM8w7XeNew703aoXZ|sQK;quS%Rm>Pjbiia&NO*!K>gd~Q^~iz$&s3sbF5CF>P3uz zA48XLVRTGR*P>_W0SE5Qen_c;&S$AAl;5X2RwM%f3ANbYvZuW4Gwy`y$c_!tS(8SH z!5}NG-83f>*Bjqn;?f&{#Q9Uvj-HyiDZ&eeK>35x7k_gyXZ0_5wTCrLF5iAYvAzl^GN?w1;FbT1%%3 z14h6ysyj{PKIT1z;6hv=5`)Y_Z~?zT!Z%{(yklu@4VKwOWI~d?u)ukY=W)W%6p~z< zxo8LuCCFB?davyr1f6x?u3zvPVRkohJA2?OSinRhWJR|i_Z=yfSjkRW^k8liGMOnQ z0N4alubQ!Afdxb)?xkyiDXE^dHqWzbnIF|zqhjX+%p;L~!;+DPr%-0KOpk_|+~GCm zRZ^hyI*MRe=$uU*3B;)VN9)K0{#+SZuTL&|508qj7=N*zr&_7r5r4lIA{m??gCqY^ zoh>ZRy@)K&(Y2LAujhLq(XQ)r9=1F~JVFG`PiG!yGf=9zraD&I^<0BBn{7e`Hr@tC zgbad|xSN^z-ZY;THP6&CRmE8BQHzK|(Em8a3OtPB(20JVbwDc=SHex6D zi5@8|UxGzmLdA@zB_D4!^{(9s1jB_avzS-291B63JqSWGK|=~&v`M@O>xP!edU2d> zLQzd_mZrd_LmU=N`iSV51Se|=eOyoaxL{kb+k{nmE5%Xp_2T4T) z!fQkp`Q8c5QP0!BzX8ebv+A3RDPYgyex>U<1BPk~4mK;M(*qbja(3%nm;RoEplLv> zb=rx)kv(f=Rt>=%9G{)PiN2UyS$X%k|3f|WQmkh>+;(@4;bHB+Zy(E{dB?xGCtq~6 zh6^Kci??N;4qyryGUM>7{P1t{;q}1b&Gg~z z*5SYN!#nU1fc}Vp=LjfqM5uBEdU{0Ucm(!2B91(QWFC=J9Fevhk$pKL|9J%6KBBld zq9i_s(PNIOc#f$hj%iemX`dd`IUe8hIi`<1X2?8dtT?`7YB^^9a?JAc7`}bXdU1T8 z_yj?J!p3vLE^)%4a>Duagv;@S+vkKQ@`N|@gsoo{Bl1iu;^OM4n1!o=R1mO1EH6Wxkxs{ydf2K7D*~`h@rlMSq6o zIg^(-Q&2fme0rwjc&6-grV@Fknt7&Hai-pKrt#%W^XHk?_L=s@nGW$EUHU(IJYJmz zdyVHVdRABnmf*)$p%_U*|8xRKm7q?1H&G1>yWATP&xIsmkw!cWRraYc5Y!6G$DQ3JrC&U_V<8ogUf{|DSc(iOQfDnPRd&AEVg#}60?g%Lo z&P>lafe2Lr(82TY!*eRiA&?c|M~5rMZ_}3~3Hvd?C7>H@KS*ScBrZkynEH6-vJHLH zrPYf909&!hI61$c-?^-}@_mx53qf~^(LR+aPcOG-D=z}R`y|U`DmPGvMUn}v_()9f zpj(NC+(SVy00WW)CjqdQiSfrKdv+DkA^&(5!AQytYOw%Wz1mJ_a z%->Hyj|k2)Sx_KFg4;}M6i5ML_v9MISQc@cS&ITO5zJgA8$|t36XObGVl;=A@U4}AL0{gpi?L~{zX|} z4ppucH~lO=q>lds&Sr{efT}1c*@8J&gB_anvcSB__>>0dm@+y#Je?{eRqj=2UY1xf zKBx&gg1t!nMN9Y6;e&Cj-z(0L9(+^_bV$`74OJ!ZG>p?CMAK!Mh=k+A+MomKzFXRU zaLPx5^9iIJ^}{v3NBFo7XrBgLWFlMCGZJ;UX<4{U1nd~oHT1*D1AOYX8=NCh+$u;;XNU?D)qs)uStqt^;^nE-A;^kkMpg4jyZKiR|C3&YMi)&amVmjd9UpH<0u~Of*Ae=vJ_TV#~ zjzrQHEI!A_k+yMN+ZA?DVjU^SFsvldD;=`^IP|N*0R4=<@U{d@VPiB;TV3>65pVv| zL%(S-*@t}S`byLe_nhlxooY>KNGM6rRfc<2%8;`MNc$n*24m9LDnBZki;z&jJE^t% zi(cFORu_0b`C6lQ#kX>Bt(8^;I{$m(VX}0NfXBlsgudedccf-5;vr3=N};Gnws10& zOc9xYD9yM?k$D33>DYtkkh`R>%hT(FD#{$Dyj=Ha)>0#+%Ww7|OqX-=4cZTIszbVo zmuW>fZ)WHFXZJ8lnuB2@RUH4ZJ>F~@7Ty(3X)gHt3siQ_PUk(d8r&3D1P6u3e28km zgKrAc)IyOuPwJA{B_+fA-pd>O7AtC(ZcI0wBe#7&Dayi8Tm1EP)i_%mH+6c8Hmd@e zDZL8+^EYZW_NvRV!DTM|FJqX~jKdVnW2)?~O&Iu;uKoBi)o^pIZ~UPVoUflY4M3+5 zk~8vRT#00Tze{m{uV2n`2pGA2P?Xsh-B^{Z_lnKrov5o<Te=W`yM^J+0nY&JI-zSEitdk;>Q6oi)4wCwG53coiC|97GHH7jCJ=ICW=MoI3M zHD0;MSr5uCjCx{(#D{kEQAWP>$iHruAhOrrQIe!zLXOw6IjH^PE-|HN&5nzCMptVbOR~HwZ6EX1 zYm}>-P%Mh73l$_)>DglYuX5BJvdyMhDu+G!jT!X5e2fT-4>@x4@~X_~ukdxf{6 zLYA%sjJa@}YYO4TJvWsbchp4ZZ)r@Em%$_)VJmzkZZ~~sCVy^`XO#||?paa5_KBKs zQIP5IYfQEd_8vc_GAAdyWP!c&VDGe(I-FXRT#-d78Ekokql}I3BR9mmrUe5OtJEu} zw~@)b?mV*82%~SdPqGca$Pw-Sw-Wg0ZKt)}LpA|A7Ud|aAzq{)jntCwOX0F=)aQs9 zwz!+eIA&2BuDI%x&;P(}{JPbkuqO!ut! zXZB|ape&OA3Y{WKd*9!s3uCfIn8Y2Fx&*~aBenY}_{_xFs8C0*oc7YlxLPIc^*6kk ziZG?s<*CZCd+U+H^IKOD#M;<C7@;c;Yy!NjUVZ0^EpuwwHVkI z50weQi==yjlX)3*cS!S2TbXxmqQizcx&0r3`o6aAo~rFc|r^YoMDiO|O(rB|^51=@e6cyj*KI(}%GtDpEU zToBHye_wr)FL|fpc`sO9WS4%v`+1PbEmP4yoJHH2L-Vsqi0>2I#i*4J&L%_~&bxD* zBhn)kv31Vwk(O-X5T+~QSB;L1a1~uNs4_|%91lZNxyq^#d8h8Xde|s z8YYvs3jIzJ&Iq8fr)UD6cB2;-rQ7<<*Qn|t#sk+s0 zk`+WvgB~`SfU2`Ar%^G;b4ZW(S&J(73($%`V6^@_q(qe19Lh_*`=j@@Wv&u4i=NKk zQD!##Iu|+Al5e))DsxN`bo7@dX812Vg0(Tro%FrUPdnW66Ai%<8(TsdGY!f)oTkdv zqbNPI>O)QOjF>nzP&v9>$fc%XG|1I+UTvf&!>emD&($~AOl-}ys#S2U_!TjjZS|Xv z-ukT@mbfGL>*bp}=>^`SQuUaHgOPB&hYA(#)4hDV_kt&I6-xOoo7nO=R%gc2VAyw1 z#quQesk0&Qhgy!F<*Dxdn^n0xzY3n<(-NMbUs@Rd$qLIqH9mgZ=sW)z;@17zilBWL zOu)jnz8t>`zWnm+-)Y6C13bz2hGyyZ&o<`nf9@}G zR7v}%n+M(H^G^sgQAFk_@Jkf&8P6XP6v^b1@cvSC6Hy z7XdSjQ!>f>Y%o$}^sbo|-CNBNAx8R@Xok-#C@b zJzlUB<}PkTHGV-gVNW&jMm33CEgA3e5>QK(RZG)UOE*``c&U~dsFsCqp3PIssa4DE zP|F)p%U@6{*i(CRqgF_+Uc{naETCQ@t6r+9US_Ud{!+anP`xrvy(&+=x>miWL%nuH zy>3Cheoy`F&HprB?^rY%1vK8vYBXtTd@$F*G{4kn!FMXgX|&~Oe5}=I@6hNN(db;z z=-SixbfeKtuGzz)*(;#gC#%`7srlJlbKs@smq5+IIL)Cv&9AkZ!yTFwKp`iH_f%TUTSZLjk-l?|H{+;U8}v@p}jYvy}zJ+fcJRaXdjX5 z9JA=02YQon{4v)#f2ng3sB;;obCsv_w^rx6L+55h=XOEo-=5ChjSc{+O90me zihcKx)CFnj5?SbiU37_qbRqG&B>B3em^xjuPF?a*UFf1N#l9}(tu73zM+MiT7SyAW z)1%eWqqESv=b}d+q{k4i$C$6jRHw(>smC&^2j3hU+0(m!tA~K`}z-W^^s76M{omSK?4yv15quEftZEC zow$pEM38}Gyn$4{fpndLOs9eDsDa#~!Q=h^FOS!M+WY_M-2SHvHPrjR9xnqq!~e7l z4J{0fTnwKE85+kMn&car))|_08k&z9S}Yn`?qdv}-5Od!jjZ8DHiAaBaz=JqM)np) z4lYKHn?sHaVB0LC=W>ybokrFn5ziNmY%jqdufeYQ(~e2Ra{&>otpKR%Qz(Q$Xs5^D zH^P12=w-NukSamIho3Mc&I~Iffei`J3V{uVXcl(LCgDpDGohUT^iGcl-u89*lNJKR zyrygRXeGKcXf-x~h5KYZ%_0YaQ8RctE{tKOWgQ0rm=Nk3&xI0zon|w%u(Sa=dff)J)8PER97Ln^sU zp<}aZ0mMQLCP{SR0f4y-+PN|@Q%tF$X{NpD^g6I9Xcp#6bYGUZhRYN@Xlgwi25X(m zhzA(J=HJ2Rn#AT?H0H`WvE_zxeh|Ev0m}-Rub!IE_%N>)K*WGG2S;H`{>-Oy&UcTR zw{R>pi7iaggg4(BlMII5Pcs`%H9?GtH8cbnWt?mWYR5s7qs|*F_hx~EM$%n z+}WxwwtY6O%Ww3+d&ThOn!@?ce2Y0TwRu6)ZMe)>kjc1Kr&}vvt8A_nZkcIs;aVL@ zcnkKW3oo^>+;Umk)*9LgpZ&G?@%Qb_u96)3XfY8=ppKNEDlJ5~~71BhKd%sg!+@|()>XWCZvPV)be>{>~$6 z`l*vG+klm_i46n*7(JV;0Jb0u5H%5 zp|B)?X}vuO9&61~7#4jS?c)S`_Rs#=y|6dX7#|oB8sPA(JdF1riQgcIw-sQPutkO? zKqKvJ#KUd>*`rn0d|+X;Q4XflSXtGzD9K2?Le=yj#M%LWH*ipQU+{1OF`%~1ghE%i zNCktJ-@u623mr+uo>~WhCB^`r1NQb25K-@yLM#}A9wfkI;5#~Tn(p>@MXlkSSO@oe z+nnWLFDbS(XExuohWiKYuoi-1z%hGeW`rxaRP88#BoRlqlY0aZt^6Flvt#8NX3aqA za+xLx**5a04TvuH+A=`3{(E?IK%lv@b97h@gud|ljrl!8FQ*r6C14nw+7 zMC=HwLLw6!7>@R1Tk)Suu(-7X8hZ&{0rpeG1dLgIOb3L6N!|V(P;_BwAxE*8EJx1> zf|!5c=>!KbEKEBB%!}Ar&p04+CyaSa+*og)9u$d7-^+x$O@>GLK!9gLR#98Hac#&1 zkz;F6__@%^br+WW$*-pR`HaVr-?@*@I_(*BUE>Bpghz9^A7%}vx2>fjSs|{b1DnCE zk)-smsVh#%#cF`5DNU*qo zO7l)FEG+3y7&1cb!gH(hkY(m1Rx*;)*qr_7PpoeQLf6z9V=@;3bU2KxusDTa@lB4) zVuu6{Z;8=kfTtCnQT5N`B#HO6%mCYSiWRT&3|E7FBVeuWnQ|Ue0}yD$0*C>uXEp1D zF(KI@@Oa{LX!rtCggo?aaRW!~7G6etR(9z77IMegg2)LvyXM9PxbI-@W*}n#P>c7B z;d6!iF>aN!W$D-(s0Y9G$*T%a=+0g&5=`~!@XvX)TavHTwnv;3sPglvwS>7w0)+VI zg>usGc!@=2N#azwzr~#wQNF|~5-YTqwr_}iC1wu82G1}-5g{25J-~w@`R6qOg<%Bs ze=Eg+kw#u`j@H92u0rFziX@Ml6IUUFp%BYC%P$byoe)#ZA3-pwEy|jZ_f=slQM99f z!uODnNJv5ABH5Uuq)(tU@o~Z1zfDuvuG@W2A2YOjXnn;o47DcdbImg2_K|x|*vjO+ zBY;8ogavcB9P1P~Ea5SJZ_mFqXyMas@Sw}xaOZBA+AevL{^m{-P@Hb87pqaU`$FZ+t0YhB77jded|tp&<$2 zWh!f54<}uQ@AA}XJcC;r&~o`D92G6kbSnAlg%jUIjnOzhf2J^H&OHtEh4fO1C&2+m zt&$PC!LW6_NR&=5zc8Bn-X@Gzs`u`<7@xDNVkYfF)q63*yo3y9as_7QdVMlU@ua&Q zfg-9-)o|uy#T#9*m){EL8%5Qd9*vXXHqe3g!a%MWHGYvK0GIg5#@9>{w}Tt0v;Qag zl0M(VOTG>d;8!{wB;=~taSjy>gwPin=DV0o_9|Se3;i;!x%j`tW!#Fr6*hc@A6un)Q2Jbr4`r04WQi4SDmUrc9XkFsIx#ja_ojj1K^9$S|= zKqQ=x?3JWY9L(mJ!XUfMWYVgi&Xgs+e^Z#N*uF5CQ?>e7INs*lbrWqBZ7*xlv&<*v z2c)y@sVocR%s*K6yB2I+jgb**+sp|N`uMBbOjj-~tDlt?*I^U)D9!zN-IDp2#BV+7 zH0v=D8n_pl`#y<$j=Ex6GD&A&7CVHNZ^P8wgjD#_5MnQiu^|_2X{F+I;M#0JcDA6* zxm|vopHf6+F(%!~cbqbDsI$uHw$hW+DQv8%At#hTuh>1%L&$-w5u#q&SF8InewusF zAjeB5BO=^{bpo2gUbKq3OsM2YKCT~{iW8)N-1*ThqH5#A=AyzQB7HOSp^fCxCP}hD z8hdlrVp>6r52IXRV5vSLyV$_}{uJS)Ui5SGGr@TGcjp>Z0;FGx(+Ff76!Tn19{8;|O%Meu-8&&dwJAG^%Ci)B^UAIA=p3c3F9&S93&-r zGe3-aY4-~*jfgOPi1)F|@1ARjK4lYa4f%b$*&htT@`AzblxB*-(*5@zX$4e=IDVsF6z-lSQf45S}5qb=`5#vzen2YbCJD45#RFEcB&1= z&~_~I2{Dxu%G#aEib}P0FpHMu9ij!2143jeOtxE~DB(5LQfBX|r!Z-R{|9#GM z2ep`^o=VyT=J9W!>V6xMkj?}|kuI!& z@b1T)N_P?kHS3~IV9d6ESW4QUJpBgCj?Rq9Ye)M1yJ;&H<`#4)O{rBlc z?fW9R8Jz+SA~mz4uZ^TN!iR~xhlFBsRVAqmzy_D6Tz^5)yc2}?Fwn{^#yHt% zIl%~elK&<+O6{XS3oz=>)tp2xm3Dpz!3^YyMw{T#yp%HRF+EhPt*k>VmQ1;DTjK`y zBExuuso7jNHR!RGb%k?zZgS9c-}6XX7xb1twJ=UMg@*)a$nF<%dm{0+hR;Z&h@pwo zNkT6md`KvgV1%MTl*8c`GpN#&0uVB8(f&7Ls?z08`}eccThiq2*H5>W{FT4g(G7*( z(Xk1~EeoRv8O3Tb!`CAp{llmUDUO}N*Tet90Ppz2e(Q-QN#YjP-I=HP75APd=Xo~| zX$RvaUjmz6V!fP3`tf|O&SQxT!m6D6&MKP8g#S)5A=Ou4dWO__jHzWDuLmhfpg8%m zrl$C#>F}a-gN0;vYA)jYVF!w2z$SXCX~ct3RgPe1c&$TeKtuAO?{W!&@G}IF0+_^f zwNb|wmc|=x6#t0Hx#a@Q@f=zu(9M)gjBYT6`3;5|lk_SNfQq`GdiHhulch+0F=%a( z-o|KB>nE|qO-$%xno!Iii^*ei2t+Nv;^fnphbR;0OyfhA8s$H(1YmAKM^{B+7OK zwqRPy)nBT*qqom-{hNwNF+r^Dw@EHR)hTZ6SLqLBA79&1n{#%1L>;|I5u+2+cRi+% zd!W*(*d`2Bdx3q?qpa*sLeMyGCQtva5> zzr&EB>erdp3E=hXdU&ewn@KTp7JQsqLMaard8IJY6oz53oI!_X);d9i;^HW142T<6 z_)?Z407y30ReKaKwUQIQzBc;vA|D*-wEz44zC_`~UA?DT6c!CF$ZS3%;ywKP^yN4$ z#TZ3sS_ZprLrf&4x9i$6nekw2z&dlg*i82m$u-Yh%qm7s5+OWe- zUAzSt244;3DJ3jk=<|D>i&m~SEQU`y5wQsSS>OIZz>97Em4(Zib zd$w%{=wa8dj~GKYgQGS1k(M_j`M(n2@RX?4`hks^l09w(=Ic|_pKweq+(M`rv-MZB~Cf%mj%_Oy=bMsZ~0 z)IE_96vn*giHYv&r#$?g78I>-t(@#?>=+EA8dnn{ zC|c^M3OzItx!X}!+;{nRA^EP?P06}Z7i&Gb*OmIA{5q9dElQYJQYko=wnmAuGVW`BbZKN%fy}(}*>5z17Ea@=+h5?87Kb}`+qszZ{n^gtYWL{8*{;9bj|Yf*Nz#-(iEY!nZ;?xe`*n;sf2|Fnny*uI~ux6 zw;yGWkzR}SE?zYBq5!~@It%UMVg?Vlk-kY$Ws5SR!ZZt^``pl1W`PB#P zZbbjl>Gro8C!N-be&OhqwgV?-)?QGP@p!imH;AylFoHW9i|Nao=*P2^XcP0-*f#WS)Wzp)Rqg&CL^+hlB@r5Tol;@J%E%`JDbRa=kvlL8gJG;CUP2Co`bldHRh=XJGr#Hf$J6DQ! zUPK~Mu~&h`^-K$up?0E9{@c2K749~PCd{s=UlP=|E!67o9{sBdB?gQ?qp@z>&DkCh z!^c;u-7F{d4S)XUP$5)1Bq_4XH&#-wrRpZGz2JWPH`9}a$>B_ELJo(l5xb^V%m*-HbRM_?vvB|--Y1faKP{q*QuhxC|AV1QXlwZ#c#*v^E1 zMFVona>1YP-0fDIzr?>oo~8Ox6Si%0P@u)s%s-NWZpR;sh()>|>^!uSS7xVX@c(%~ zV*N;cRhnNR_(tdtjc~I&^G@-Nx=aTvG5{$RLJ`0x4UKmn?J=6U$uY8j-5b@MIk71yCi94SB z8C5G{>Pr70SQ|NjKBZdaOMKcCywv8LHOfo)NTO0!p+W8ppp#MVNC7BsYSA67sNG<~ zFi!h1v;4nBbg7u^x8qB8NI_3juw|KgdiJVm(iwX~fSqG-`Y}F)x^F87vTn|mbA>8e zOwTSv^-Vy|%hvNNzTBrg-B_B={6{12+0ZNrV8IEM(|CN=?ciFO53@S5e-l;~KHD{b zNvtcHU|%lv)8{T1&tJ5do1+eJ&-~XdGzL!mZPZ1sU?s0`j71%mm?Ufp8y?eH%8CjL z(MmQSRb0RjHDuja4!*4>n57{JxWACc?U3Pp_3we^?N}|cjV8C;f3ID%{T#ka4?i&< zq0cnmSc&Jt#|F}O?w*O=@l3D)qn)}WNPf$CU6Nkvg`ka39~HF3b5T?@ofbwF+_xIN zqncWcoxcnBN%^VC+HnC4#*)U_FZVAGuD7o_C1=(%JS}AhJ6*I+0iTI{qec)9V-qQ2 z(_E1cWQiPIvAl}GDJA~~M;)idfZl-B0k76Vw#Ys`o{C@)w8+QLGTyAC+(AB$eRVE}z$aO_h@#5=|x}pFGADX$J8bb6=@N-tU3p1NszmufZBZv+2OW1WMbQRcH1?sVok z7}uqdb(rFKY<}D6x>?Oi(jB+_ug!5mUjE74btB@FRL$E-SqighHj12Xq|ETpO7Gr7 zqZ!+|SXrK-w%LkM94fH;K*9GVVb)Hmo_I40q4l6H;ayPYp__g5{9{`lG24K4+qsB; zKQ6S1rI-RAitUdG^~TPA3hzUKl@7NN8l&+L0YM zf2~qu+cV-4-Ornd+hr{~Cw&U3RtdbOsXho^QM==HRwyMfFf0&*e}vPFIkYyuoz7kF z#cOeEQqw}gqI*O6dASEOaT_970@O^-*aag}*oWTc5hmqPKjjOg@t$g$Pg)reFio9a ztaU&5uG+6UrMl!M>gVBh2^Q2ZB@0ygM7%{Gl^CV77tKpmlV9A}A~`*zp8cgLFOwjj z{Hc4BJ@1G1Z%1_SSm#S%TpcKyJpQU&_BG)3)8Qjj+z^d_F&cT*I-m-cok;xqoSqZ zN$`-D{P3t*5O?2E*DI27kL+91{O}1zTFqg#VJ5h(99}b-e4oZX+Dy>R?HSqDVW(J? zsLn1*zv_DLSsoBkM#VhZ7a-FRu3dsxA*q|W>zLW82jV2gT+p?XIz*BhAVGwI7Rf4G zSm~mXc%Fm2jU0G0m|zA~7Kd&YYe_=HrS-@=D{85yQ*vq3q}FmUf%$K5H&KefSs-3< zy9ej@S;JhIM+kmw^2sERngtv&+O{TxRE_fK8aZ^zOoPY)^1~&^t&%ZcTmLJdxu`bH zR30B-Oi@4u;t z(Z&`@T%sW1ONd^4QTi_7nd;%5E}5F)C5DO4yjzgz-|W|c!mv`qalyiR{sy7@{(lT> zOYhMckS0q7x|kj+x&$>#BE8b~n(Doz4WH4kAU_w+C5iHD76vQQNnM*s+i&SuGr&B3 zk3Drm|JGcvvZTPv3^}J;Y^~+F#BjliZBk?~<4dR|Dx~J?v)Rn<#2%`vTAm*S z=o}Nj$U^%bfkjy*g6yjv&xfqwOjd$sZkcuRf3PmFcXU_2fpy1wF7hE8Pf{-T??qh0 zT%7D>D#hH7pJ%no`fl5H5bc@I%xME3rZV4Uh_k`o${kT$$qhCWd0jeLJyDZ2(N!du?9G*Gt#^a*rWEaA*$;a(Y*gng8`^ z??2SHmJxLEw>EoJl1-e1bj;ZLG~=(`_xOtU%3Fhb<2okc3Uo_qq1@m!M34rG3s+Q$>5=(A6=g9_k^R`&rxMNu`_v_2W_Jq4m z2%#*zx=3Whb8h`GAzCNLHgW3Rmx$ZiyWBUk=L{NmYr@&peQwVwP%;iD;- zkbtb~09rdd75YGW2)VooBZ5B*Zn4uGjko`sg>OHX#Qqn>O{hthG|?85o%_RVPp43_ zm*04A#m#B)hY;PH)Nw$(@7pMG-Ga!%#Kh_Nlz+-c72HYQ@Ml*wo( z3RIU8jUGbk96*m@6zUrW0<`W)1$zVc>&vr~!LJWN&1a+V`(5N433E{pGJu2mct~&xjM}LXbn%{uk&gCD z)2b`c>WkGBn3cBkmq~m+9!}~RS=nT_D5?fp;Aw^Sn&Pzto@o3UeE^l!$e1Pz4k!+q z?o52nl$hA0Wpf+2hQ1{f^Q=e%@Gd3EnOLZXjoB8VeXc6kCO9mGzfjHyr2i#o5VEgn zB$V2we-vs#bvXV-V2DVgM8w^`OA2O?ho$k~ zZ2$+8lfPf_xI=edmMX~#+7eo7#RkTU7V0IOQiy$NoF;I-3!LNo;AaM=w^Hvi7y?r4 zjNwU%VnA84g06fe%_+5{(|66(ou19qjP2UrYm@fw=hYdcq!Xrx0!A^GYcQd>u4E;_DA@`wbU)r=fW?K6gztWUs$aZ7 z6%MZsSu8qJIG8maaoZ{$e=RzTf7pdew3x;cU{HSwpr-U~|8~FlB1Bm^iNE*J9qR%PnT*=~5!*eom&2@8XDeO#H*W z)o-|ch-^J8>%WOsKms9q1;<^~u|H$BW1-~5P^K>OxUc2lC+pG)32~EDOUGzc(}g-~ zJ08+^66xjaB`c-oaQBOmNVw2`mv%ql>0xNRGE?5ciK|6>(A^y{3fCDfG}=8L2WUdC z`u}NlC4!xAJPPDH(szEha;LMO=SPsj2c>jZBLj~M4c2Dd=z=|UhGPmJOWye3wFwY| z<;rzdR*lF;H#MFclbf6*rf8w#sbA>Ps6$d^7q#r#`4un_{WIp_YnAk$X|F$i;>?gu z68Z5otWWGScl7ora_s3P%(e)m_qv@!RJFbn{b!uOzvm#%0(;YSw)z-|L|7`F4 zJ9+wEN%IzO_`v->s|gylHBl>kb5xG)e#(EpNOKXxc@^qu1%Lm%`*`oPpZ6b)U$pNG znsfnTw1@wG+s_47$+h)@rpwXYee$;6B{WABE|fz%Hy?U@F!c$%&mPTy`uPm}Rn znw&og*bp#+jhVp&DccqkiOL|-pJu^pf>FVjXVNUc5KeD>AT6SgPT!KyF=V{#fV_ai zMGqi?0O&*#eJ1X%wAV|kiR5{5S>FcFRt4HC{YT}Nd946^CXKOqno(?%ha|GWmNbB5 z-b$3jpGzayEX~<$Tk|2U@UbL%Nj%6ao85PYJpjRR<0ohemoM!a9*JQB6AnZ+;Bq6- zPh_??UMx*5ETTiK1@+C5{IogQsD>HTMe|Qzr|f(iQ`VO=T>S{{A#?8Tnae{Pv$jz^ zSlK#TmDGaBFKDGL4w5E0#}eJfc=i#zN9Me`&GbQmyfT~7e6$((170d9I^PT}7?0N0 za;KZc@-p-NH0R_qmrmRh$?;`ugRn?Tvs?JonDguA2pCG!e;k1J`99@aXEP4Q zZVezs;u>O73M^v2I!XJ1wT7L=>HC2tpXtWMsRU$dV6T~kvq;RMxxwcJCKty{ocXia zNd$O0zIPbjc6iwz#eBuQy5gY)yk1M}M$2>$bG7&;=PNS3(Jis19ElGWO+PkxK3PbX z1xxe}BL%XU4=oNf5U4M+5^|d8i_g-ML&1=*1RVcQ={3Ud#;aFZ-f~|sOY>Vw3+GCU z%`tJjm6oxTQ9$O(sLaWzv&d>%%IfCI8qUd@vdCFl%Go}s@1RmdR9Pzj|HZD(+y{w9 zKrO^=0fr;lz5xO+s|LT7hH#$7gZD;2ZW61urIq%DZJxH{ytWIg&ZVw(BTvV7UMGN6 zH^fReEKfITUYEqG7oVp|b)=g__&tiy&$ZHjkf&cfuV40mxYm&*?E5pt_ z!|r**{{O?Z#`69T*J3rEvoc=FGyX9Df4G*F$xfch(lO7qx2(<^?7v8Prhn&6FIdf} ztj%ch&FB}*nApr&tj#&{&AArLdD$%Ztu2J}EyNZquCiImSX(OOTdFKrs+4JT z*FP*=|H$UNY3;m|@BDSad7sVY$lB%f(vq=0!nC?j*$`<8i1dp@CU#dA8&{43SFS}@ zUUs+C69v8kH?c*xtL*MFHtq@q?kbD!>g*oxPS^qRBoD(y57SFr%f{2Tz|(Qj)5XR; z8R&JRz{_{hD}en*h|P_#f*V|mNKpHYcpLBJ0`L2a-dXH7bJ;yeHaCkGZ5JfDoY{i9Je()LzHZ98!p{8bMl@!g0r4!Q0;Uj_=Z)0FJPb z2W~fQ!=jeLNF3quw$~$U!|yMJXK_U2ast~5Hc0|z$90M{1%N~1c zAu1>qz6?msE%e}Ihe9Z+Wt6yVHr^Olh8Rj(a~J0tf#+%A{a7OW0PE9q8cAcrQwX3o zSqCpuQ0uWSFFK<#@_!@|GIzp}r6gD8205WzJp50v?cWV9ItirAh6t$R(I0EYFLtM0 zRLNgY(8wQRTpR2fl`g#^DPr$zlDlj+23V2+87jZQW)u{K&cL}2o11|SJ=5qlb9l7pbuXZ z#S_=h3?E!%pKu$+7QFX*_z{@iMIrLxK^7QV)BxGlAL(P~^40Z9Jf$cjhJNZt^3jQy zg~@G649!Z+yI&{V-cv*3MP=K}xzv_CeX+FH-aVK>=4spKS11;TvGM9~7SNRE_X0ua zNI(s-px6c-vq3F{1c^J4h_AYG7YZ|uPNF$0s2u>*egF)$hjZ4TO9}8$w0w77x&ZrduP42%d^VxZ= zVyP~yIH8Bih3_a0$wcPyipH#y7T?guY&JBsBG(bXk@g(jCJN-EOj!WhT`CJuu{DCf zYRX(6?o99tV4#Npx|O(e4|x~+?c3qj#dbdxn1k%PL-X%#w%vAk>_Kd06_D9)$FYf-Sf(tE**m1<2M0Ia+Q8tTt z&y*{E5A;z-_y3NH;p(yBTIr8x>wZ)8vZDK?{hz_=sMZ})@XO7%p(>T3g{A1uFy?E& z!50{=yXrl*!rd8<265XJC9>)CZ&P%mfO^~PKtq*W|?@Hws*o1mb$n0mL#H+1RE>dFky z8fld)?(XucTFe4rHSzA^+spj3;*tX4eJp-8TW$56XPqr-Y3O_{cXj1H&k(kB(YNIrEEaecE1+sL^bJAqv=T}7$#%x?y7nhv2X&ga%-u)de zN!zHDzA9pboEmkuTjMVMqSBG0W*#M@4nCVO4elk*yKo! zeWy$$$)bCE>gtU_7wk1=AlTz?RNJ`Sse^{KpQq^4{bSCW(ldIs(AGcPZA55cIW_dp za1jwYSmv?e$~<)eS*EZV(qZ0<#3uEGo#Jf~PbXj~DXryI(C;nMRO>bgZw zWNuyMK1leem9qQ^A47{Z84EkBSq4|qeVjHHpke4m%@o()$CUv>&dk;pvniz%@T^g# zvY#)JF{VYR>3zODO>Dw%4gYpUr))-tKja_gJ^Y*D_fu|7zwHznNV+9Y65OKizUwZp z`Fo1Xx)UA~I866vRmoH#sZjoIJY1<|uTQEi$Q_f$A|izFF+WHW4&U#SQu=;#mLX9g zQ0d+doP&@k(k4(0%|hgIwcc8_S$3OM+MW-pI?McYJ1BHav%lfS>l7eEC*R_$#{&SE z`6>Du%X~Z`?)>X3Bv)i6Nd)($lb1?#>!*=cv)1MOqXs@e1i|1xc`vlyI+d4Uex&jN zshFRgW~!P!CTsCKX^rCoJ6+iz)@F~oCtOODgGrTBGiGo;7D>iKJMk4|KqpmxQ?)z_ew+I5 z_HuQWXSC&88UF-wya>wz|9H7)icm@dGrmfYHT`qUI|VfgMO)~`AXGa}PTtKU3nkm; zp>?Q=0+q&rRsMRSBl?bv_;s8pU+!@PN*T3&%*)nb64jfaTu_@b?m?;%1pbs7wL$zu zJ2R(DE#~7E7D(dQG8Zp?@MsT+Bx#Kj7xxw9rysz3HQ8A{8wcWU!dRH>hCi{tSazs>^oYk4*gc8>YO4-wMgR5o>oPk5I>k4Z<7j=RiBc zW#z^02To5S?u^}9k-8XXOE$_B7IJKSA-whdere>l7c|t5l)P?MC9KC4-NdJ3-56EH0OpB7toKeELCt|Br^shyF$9GtJwnF<$H}~{R3o> z)JuW>khY?^Z;SQTqILQ%ww5+^BJocf1AQW`O~*{85_xt2V*0AfxyMF{#C_1B{R)4lO@A=n6V1glUPz>WxO-p7b0}qw?5@zM3iMW+;kq%E^vuF7 z!9V3DYH^-VZb~+uS4V1Ff*%A<#}irlWwTWMZRCfg1oJ(qGzzw|J1l}J-T@(k7Tyy^ zWokgaMJvybYp#lqzNCvLif!Y54={PiGa_LgmRqW%0dDd{y4)6pTZmt&_$QhF_8Isc zLush8-sl{rG(`_D`~<$7EmT@)mGXU(<>AVNDeK2@h^(`@E~M*{mc-%H4)L^n3u3^V z3Cm&+FKDO;DU}GbE=mg>vR5bx!xVSs4EjM`>pE{pMT*laNk9S>$U)Jt!b*{q>1(uh z%_LrbiWx|joLnVzLhc=KHLsqhQV5K7IDJ)CBWaeT;}Pdr3vt!dpls4|Ir%C4*Y?h_riIpeI<82X zkgDtrOb|Az5Np&*UsIN$#^B4nb-5DKL(_9bRSBcP*fa^@xGcHhTjRm)lR^+GaNW53 zq<{AWeP!j39bbGisR-@EO3aYuzUl- zMkpSlzU0;w3}y=(Fo^vRIh^wZqcEhJ5Ew(HkvTNV6pHf)#9vBK@>1?&Cl_MD9pj)_ zC55*o3%T-A-6qLWH{co+_p(Qt1BqgOa2@g(YUsBBRx%aD^@xfjPP6SqkKvZ~6>z8c z6|RQCAFUIQ#lmnoG3_O*eF~^)*5$ZSrKfi@jl}mraWT6}BG_ty*U%56@I?Qe$sX`A zKx=oxl?gCXQTz}TpGoDMVbwtC7j z6&-^7M@hr(wj}G%j-RvxFY@hd45qzJVJ~AuaQrEeL5`%YX|iiqV_4}nd+24&&nWH? zAZB=ZS>fydCGn{Kj@tGFvh5{BOwn1q`JVo_rF)Rep?iWW?x6S2Pt$Q@0&UMl*zx15 zy}PDaf9CsrTmJlkDD`08X!Z`Jk^c=h+@?W%q)9t?lGCDol6{^{~lC1Ffx!c|Cs zilg||ppqg<$IQGR8g1PnfPl1$P|0d{AZ5 z1zqN`m1qB(77Q6U7xsP3`E?8mpEp-jfdKaoyb1;)ZLv%;@}yOc3pvGDMb|sVD>q2K z6sBk8@}?M$?DtVzJlU zNPu+EWjhFd2OySj!E_SHbWQ}je1fb&{9!RyuZIMGso^b(;XH+hjkF77P%u|C0^S0w zehV>CxdC*Du(z^TnP;gYKH1M#@|~NdoM4#fEB%x60$toLu0T5lc8EaA13 z@Qolh1-sT|EP@=44Q8}`;{W@QQSy*vm51|$SZUF^hYW->)?ic<2Sc(#8ME^Nx%rX^ zGe{dXe+_e*-EI2PP|MX{4A%z%&2SAI9$;z9M6+d%T2}`MtAhoY0Jc_fM8^!ggjc8I zpAAT00=kV4Xcoh~V-|=CX6g-hMoc?LglXFyGhE-4)aBr7E)|5*KFb1YE|yij3(!hprBHpUV3_|zye0wCv|{uDiqD4P~g!hd{uYpTdD#-kEO4p&zSmM@(m)9%NeZWliI=EEPP5 z;2>>kK6M{J|4|l|M)frZdu+Lm_ZT361JT3`#Kn-9_wxb%L^|OsbXt`alldS4*4*-- zlxaX@(e#rg&p_?ttc>Iw<{(BVB7pcQy zF6M6LvGkKGcej8{t-8tr>`J3y7g7`D`P2jX69@VG6sUVqeFc_jjp)593z$1rgaaR_ zvykrjuoTfi5phZYD>I|#vZOhmSjP~hZ{S=FkCC`eH#BzewC7F!CmWE{+)?I!ZECNdtlT zPk%8m#+KG)*H4J%g)W1_UcOP@V*CO@3e8oQI0t(9kbxx;hcUDwK))@RY5-J+*vGJ0XzcCBa|Ql}${NP?!ltO( z=9AP?X?O&g*1A6P+xV@M7Kzt$)h<4g(>O;MAni9+^evc7B}?`UyMwF*ius7-QB<*H z;bnfoAhFI9W}uFlcq&&d6DgAI^ni|kw$t#d)xXb{Lc8InM=*Vidd#>=;3o+4PiJg- zRRvw&hFIUGVPAz*{U3|^7qRu414QZp@CGeUjaOa|Lg}?_Lnq3wyQRTRpUb1}YLbxQ zs4`-L?jE5^-WPvS#tdwl-Fwi%iI}-9=NipE4&Ec8Mz`@UU&cZktmM&aKoDdwuguP@4@s=tS4@o|!}j_k7T`q81aMBVo6sI7#x zPerd8&_0VIchXCpl4&1S0((DTT|kp>btahc;1OSK_1Q5Ng1zW;C^*paZ4K`<7yInr zTKJ3_zwFEN-`Wr}!jA{pU!iBrcZNy*7ADd~D27jDT$s zP!R(`J5u-+bO%DE@>6p`jrgfw6Wn4y|&?sRGQJhfYp9 zkq##j8+^>T5BIcQ{rOD5B)qzInt|_pRs8oB?(oGf@^w`d<8buLv2@M?VL`i`w+g3> z^}gc{OEu450U)%9Vn2Zo(P}3ks;=|*Pt|meou;%BVgNh*sCAn>Jnf2HPWqL%nlqD2 zuP*Q>_BmG$B_hV*XJw$l%psub`7L*ZA8Hl^T;qU35p5f^g27tOV&F9Jhm5NSZ z-4B{?TC(DwYN-vrK5iCr(Spl#nG840x_?^{Z`kA~6R!;3|U!Z=zP-(iu4Hu2)t9v~C0cuA&v z>FV5)9dH>k&<@yaFQ%(-5*D>Am_)!qxzEebDb%dDpW|=S=b1sY$|W1p{b|deH;r68 zFFl6`TUm{1zFnib<|>83-F!bTz!Z`xMO<71B5=r5Y^ zaK3?h6P>F00IYkxtjWE(8zf5{wJ-U>xYSK$mWO^OgO+floQU4%1@M9< zR`%}$#U77^<42$E)gPc{P96^zD|WR6Vu$s!Ow3ql_UT|({n1AgkF1UKpQbH+oY|2& z+A-$-qU56gV7_EOrga2c+J!Iu{2Ts-bJzL5`ej7vQBm^C_}#OrT_+NT>782LcCDZn z>$_*yndnhkfjJgHxVWn529$Xx7 zr$cGAdcO*P(Ev-HPnjE#;Mh5(2pscP*TPlPIQqD~l2=rPdnT^XwE-c@cTCDA#rW4|IcLPw&42*{gfpO9 zu0@Kj`yAs9X%dx=HboP!aFcnXajbzXp$`eI>ej!a2y6jc**hb~rGgEsh>=U{DlGQJ-G~@%%D0Y3tt4rSauvRV05rUAq%y}y1-+FPle&SB|529a}UG( zX4)0^rj+1l4R4j$RFB5{Z&=J|@3KsVh&PNO`C-#UWan1h4%%Hq4+FfA6neecEZt*J z%WEL3N5X>?{#5Kf3R$VFE3+l8PdrO-z6qr(Vty&i;-qPpUynxd4I&fvevf8U$l5jX z4-RUlELB97H}ERli5p<`7^tUtIO2VUW3l(_rEXpuaa~W`+X9nL?>(rj%L+l_K5-Gh zzTB>Zid5`A76^Xh`Y|CVWj*dGrh&$R3K~OjAidt(&;U`DpM6GmCT+OYLX$Qgr1Tti z*`kNJcak~!Xto;3Pfq|3g#I;I-(Mq#HXK7U6AQnmW0VMOKzh z^ZNcTK9}D2f4L_#BsoEdc_%P&IA!Tcw_P_*ju)*pyGQ>0tD8yseDrIx)V-MTK)`m{ z4!KO0+Q~gboCfJJ`CTRffV6(en=p~^@--=kiZSDpZvIgjQa~Rg;ADQP*aF$Ci{o8J z&V)n4E93wKRrmpdoB+lEjbqvC*PW9JwyFer0g;x0Gb+QU6^A7Onny>o01QB=?_iUb zDtrJ1X-zdgW-_4QoIIvn={1fIEJIKruIO=Sut%EdJ-0VO;EPa@1}B|)h!2p&<)vB2 zj?>`j3NiWCww@~fjh0$r`N0#sVErPepq#x?B~$Le729n=meH6^oE9OWNe{y$j7R|T zk>gwgx44_}NUHSE<|Nur?9*S^Wmn39rFt0-v@JH9&o1vZGmjCD`bOqcSg4yefR4xs z(|>)3^_H3uyrJ#BE)tM45=|OHR*DRXwF$u$&hASM@-0%MT9NK~1i@&>Wn2HBI9f9e zmyc&Uoemc54dWJ=N8&b|KeSoMsp84eb-GOGU)eLQ^_~#Spji6l)*^Vo2ZkB!`!kT( zMFHH{M?w}p9gLW|Pw&3kd5^7&si}Tom?gS)MXW!ndL4^{oZPu;1 zKkY2}CY8h+zifJ{kx(x1j!lb}vjX%{YWce#pZKxm`gwXKWABepfbW4RKuraWnNDC& zT@|C@Q$(|Wu4%ub)s(ZPn-6<`LTl;v&D*+5khSCDMxqAu2FdrkGWCjy4$nto0?p{& z$o8L6l+L%gvOL9F4ITr_H=*jo zb~@#lH0@H-R~IW+J*Qy4G}U!FVZp@9ifd4VYDye)98LoSBhzG^7*g^6OS>*_)5!jS zrJth8AiJi+EV(r>-dUI?;-7Qbs01p=`a6y@aZe!I&r2PS56FiQttVHiJV)Yy2#c7H z&;-B;{HHer}jSKMqSJ3c>DNw?#&C}T@;B>4aw^q8G-&ipHz}=-4eRN_X<^- zW@(iJV0hLAyAVY7OLGJH%`{j=B@Dq+ew_tLyX@}h-Rg?x#>Mxl5kdCV*{kH#ipv#8 zxmQpQRJ1o_%eCO(@uRWYm%h%Y@0a=^cUm-uwkPu>8v@S_!hA!xMk(h%KaVX&G&w$k7!J5K9+vl(f^7*S&Bq`3fU1FQhtg6;dwm7q2PYc+n zC$f0X2Qli%E^Te!{b5&CJd8JAN`0ttVMPjgo^~KU9R0#YJdbmNtKM8`6fMyIBlv@g zV-Bb0ji3p(B0+e9V|cZozuhw+497~l;z7p8M$T6I%aHXHC0UbFI_PrbB6=#{N{==} zj6epoTIJupc_*oHHXLjUTBDYo_H~Fy9NZ-BfPns}Rw2+9{^ItGv==3oEhkxUh;)7> zqfp4W*3Wo6S~H*bUAt#b`o!bza}W-B#m?#Zn7*Vjr8Nyl~S;?SxPgAWhVPrXcDONyraS(K@ao$imH(+Pb2IYK>@7GSdPCZz-C zBZhIF;yq8hv>9^(WF%GjlZVVEEJzoLz4v}~emvh#zSvGN<_wQ{uGlYH^~B$d+b`@d z&jA#&Jj2S0VLi2=35MhS=po+xd11ta^KPKiK|c7`x;n=+>NPgx5tn=S%K`p_6@nOleH0+uH-l6;cL#aZA_ z4Z>!Lb>$caS^}E@GnWN6m8;7OQ$Fkv|0#zipb#`43F5w;14F^rr4cmm8<<tOtwkb zzq0BR$WVu1U;`Uq!`mUc#~2*9Gk?9R{|f;wc!jTNlAvh2?TY{nxU>+s0G?3@Q=p@& z^TZ1%1M@SATWC9>S}BHzw2T=BhVaBaDxA9T2nEwVPt*XZK#O{~g>Ui1s?z{BFor-g zmk@ZT#t;Qtbh`*Z#xWBrA-M%8xByfPfks%Uckr)HREZ!MhHgxVmPvtExPPV`GX)c4 zfWXL%ddLF}pv3CSfjmH}U>v!q{!lSA~vVll| zBnT#>7&MYV$HowvVk|$yA)64$1LX6>9Jt5BS)^hNnIM?EjCq#)f($_jfm>7&5)}Lddtu_OSiRk#R}h6)7>4G{jB_vsQvgob+>CR$1ylIV+I&Zkin4Uf9OC@VWMnGDX^8SGOJjHi z*F?^Z#2M}sr^h&lVSg~rTQCOI|Fn#6Fi+rA&&=pe=DdaXM9TWK&-=vB{2UFX)X&S{ z%KQvXruxr;n(HCVuxr9*^jZo5@QDCdk3xB=Q%W#`qIV$NqQR2+9 zRdUhDAWRsWIhDURrPDjD3>n4K_Ok{@@c{AKQ>O|6s}Y3Ao6~Q| zghrqQD8)}#gMR`7kU9f^#m@hDh6wlzBn{L|)znSp)S9r3fAXs=gjLvuWms+H*Txu4 z7;Oe%aG$+wSok}KV6X*Sz=iF)*pCI-kQLdH-PeH44TC+0WKtHZWKjZofdRq>lSK<~ z^@f`D27mcM51Xyo16|Ub|K(ZZ(%GFoSjz#xEEs~L#f3olh@xuKjDDc9q~l)=SX0KR z2cgK7n`w+;comz8I{m`i>1E&sc3`QkT<)?%FVYlhdr|Ji+a%;Gc^hBba^Dw4U()*E z#=r*8S~t(BzeAK?qdXAaP^$?fncahm?SFe1sNe$C{Dz#G;XJ^mXDKiVFoJK1iywX9 zA~xb9j@}6VKdh>aXF!hJT~TakhKV&0&!DU*X$H+&4}KdD^;Oak-j33>j4j@e6YiXm zvxZcp*C^PN|1DT`AXsJjzuBT>ct}}JG1z1)SlBoRgGE_Nrd_`94t|)4n)MFG z_}5nc*AR4N$f}0}Q4z5C4h=5fR?e{zbdM3HiX%~4RIZCxc8pdwSAVUCkn@W3PzQcU zM}T!NfDIF7J`(*A;T&V;1Ci#LRev{MmIqY6*)x&ZO#Wnojl^+H0s`R+EM#iE)DEJa0<;|j zAb0}fnS!OhQol;y5e^=rhJfQ~k~Me&ARrSVQ3a;ngvk(9te!#$n1W^y>d(rtW_SXz z=79JS!X|l;q5dpu4r9}$6@SDM>lZK&{OIP!pwbr*YY2FOIOuAT`-VxdX^tl8q-7G^ zAp|CnqAbAa=Nf|gB?O+wvBh- zB9fjCCi3i%_9mBsf{1(vS2U$t%c%>1g2cI)Swf0jNdd+|3ro)E?0?qo?S|H>{RS1j zt^u@Q?&8BQ|Ek}cI}S=t20pyGAj(Q>xL%tRL{K4)MJZ(}iG$IS>+X$|DjhEV7C{Js zH+@qF3v{f|6~V7IoXD-@W7P@aSV90K~TZ=dF_ zbLi-3BfvVbj**kYFBhW_M2~zrxZXjyU-a8$OKUsiX_;VdU%$< za7XTD^EP*LS@mxE9>AN^US&HumP0Exh@BpfUqbi>vFLcr z2RXTOCL)A)Ab*6LTQ6yYV#yHb<9-WDc$g4S8DSaJ6o3ZwJF#Zjn-CC%GJW%Rhxd4I z(yBd&EMS5xumSP5_j?bcDGF~UU;^4OD`nFn-sp`M7?rd_fqVBN_O=0JlZ-=$p(!Yz zb_0UV)}I$hlFt1K_O-W{-Hszv2P)9`^MP!QNA-@c^nVV~K^ST*%c{51{pKSn!S>() zkh5@6{|ACBc>ysT03g5yexQZ((fIBm^)6(eNEc%)Y#yafEOeVJT6b{>p?I?n5M1|9 zHyi>cF!&V)b|xz6J+T3<%I#ujt~s2J-pGPcQ3qV`uC9U|D#|&UesV{NX*sNRm*)CC z|000D_kT7H#&7cWWN{drK6Nrh$)Ar7h+prd8ho!xBOtMTHU~fT_6PK zXbnP02C6h8e+Tr|!2B%N_8Be0jsP0@aT(%~)IDrC%r*kIX z!GFWW~wiwAg!N%GUoWI%~Mzaz$~1$=SEIQsT{6kBbBYP@u_^curIT1`KXRln2d-feUW>eRE^D z%(saB1YIxy!&u;V=e~gkeJ(HQ(xy+NPJgX>HS5-{U&D?qdp7Obwr}Ikt$R1`-oAeW z{|_#FIPv1IIlqRd99Pkvo|G%k2#-xno>`JXhxaX*I!|1AvN@;R+Krh#t&!V#B8xR! zQBkl_M<`~C=d0D>Z#HNR6)J09IK+V}_(+5gKKuaT&=c5nmeEEIK`_Wi_3$H4Gk@45 zcAz&64FJI!AQ^}fHXIJ{1V#>W^a3^?VUwUi5CEeNhbnHu5jM>z*i3_^@iRz8K=kO4 zD*Vid4K`0e*o=V)Liphz4h^uzJ0H1d5RmF@0_7$G$-~bSP&ShhMGjq*(t`$ZFi4mm z;p5F~>OhxFTr}AfQBLYH0go*~J%5*lTt_9f6be~DM;%X1^GYvtGa_3|b)DZvn zg=Jn--(+V^GWbEas8EhV*V3dYVFV?W8CR^a$0D1ovdc2tth3KT8?Cg{Qd=x?u)*U=b2r$B(sb3eKz|54Dk;av zcT6zu_vcYguKH@Of-$FnmVdX@d8IsJ~SXoRW7(=L}!+WO+6VI{LYvb zohTxSCZa@;Lmo+#rI7ppIKT_~K?w{n4y|$1X+8dkk188?tYR}&0#TqU69?%KFcG2b z5Rl%i0VNPqJn_U7Btk%on139VDdN8a0P(~c2@x2Zo^#f8)15pq@lG)2d7=&*|9>K- zlw2*~V&^W+!I#r}C`@vWJvhzB(+!iPdyZIg-GG!tobIiTr|jkH+Ngd3meDN|q-t5N z3FL52J@q`J0RsuB`VOvr@#>i<1`uEX=UfzCt>~kZUb^Y0qn^6ztADfJdbHOjr>)#{ zr}V8+-AWg?b9LjwZZhdvXU#5gvGEQ}GU0xjJTn8_yns<>L@nStBBC@e~`>wv_*>VQ^bi zc}Jz>l7(OdE+u{o%5`GMn!%MyFoa=%1Q6pHBP1XJJ@|$(iU@}gM87h~W+zW1TTjL#rLT0=oR^uUgw4}s7# z4g7#0Kb9TDe)ubyMo8q5C!y(o0n`~k7GgjHCQu`eWaPnsur!_hFF6%F;RUZYmkr9L zYhUY0*x*PG5q}1Qgj3_CNphmXy4~#rgdm5>WcWAizy^n*)0|;Kz?phH|1pRf-~zBv z#eg1g<}#Uq!7w^QL}=t8GgS;DJ?m-Dd*U;n`rIc!c{9e=lyP=8v{M>iI8fW|E+Xgf z1RR3_y!FsgCdE4*zEXn^THw!Pn`nkQD2b*`Y%(I!YkycFt*AhV&E zUhs3xfq$w-n+^JgAi9~(corrBp7UH`7!ZM1$fF;W2!R1K(1?3TL>~Y3$b>w%+0V~{ zHngH0Eon<@+TQ##tg<7^8Bc;xgL=z5JP8*(@I#KP$R&;sWzX;=YDeS&7BBoL>O(3t zr3Z2Vq$iQbNU3L#_L&BeS$&5}nZ`)$c`_od6n|1u0S1IBSxHA3@dhn6p$hX>6R7hW zs>g0rr59KaA3jHm8Vnzsz=;rY!7DV)Gfl9nWQH!ZMs4vUD!ve|GxB)G5gAab zdVd(fFcP=`k9&-`1T?~N#t4D|B4C7tWtI}1HR)-a?BpjyIm%L=|1z~wyPCVs4Y03W zqidN%Lu;61uI6FSyMotS`24n_s8JdIUa&WgBv&B#ak2CstUr!G_j*X1Zgn{jra4ap z&Z>}y`0&S6d2C3kaj0)W){sj8*7QXaynk=X)>}Vpe90jRiHM=Mw7wGknMBsz7nB^` zw`Oc2&j(AN-UQXlLj8jUe7LiEjSqG@EzZ1L}N{Hjv2$C$YtmUq1(-c2!AAi z3$~I67lIgqVK7X|RP-#B`|a<413cgYANV`0tQsu0RyqacGD`oZEf*%(!V_yl>pRK} z(KAobzCye>Ol1-N}eM1l{}Wz#WRxwy}-QSBcl)~dCA@6Kqv`F6$r)< zdtaoAl|g9`)x-xdNW_6hs~40WYkx-no(l}{s)!#JyBICNYh8!^3ka&Xkt#gOfgRyP z)XAm!P2|zXH2_1_`!eONPos`+;KCDf7co=JB6m-?#jL?jtA&d_6DHtdn16cCIe@|x z8@$EY>$!5Tz32f#I(|KA>4Px+Wz+A_Be6k+ z`9GC~(tjlocI2h+55WJ(gMX+QP)Xc}HAKqrDWG{U!mphb-w~UjETCMl!B@b8yiDNW zbDVG$bP5vIiMp$3*MA5Q$948g+((v^9%K)1EU9Dg-f#sQaVG~FwS z-2Qo?Mg-mUNf&`(Lq>r~Vz|^R8Q_%QS8+9iq0s`^(SrZcv0tTa%#4WF{Y>8!n9S5o zACPF*{mtLkF-&yVyvhZ{-6TuC4m(hcL;jfr(2 zUW^bXj@YrW#y#+bVSlI~s)*AqjE10xIyL#E-}Al#Zd@L_vDgXn9sDoy1W@w70(d6T1j$~iaf`4h6h8_WfXsSZnp;{cQ!EA06 zN|c{!j@)L(h;6p!&+Vpzq^2r_321KJJEZ0&{O13`q}pt*!63v)!vKS2I;Z`pW`J#i zY|f_o2xr?R26c+2{s2R&S!Q!?T@V7MV%}wXs%LtV1A4mEIkm`_GX<%J4QXo-nnHL(yZsmR6xc^N>61=KMt}U*bfLtOn&*@{DM}EUIv^>brG%9x zhLc)RlOowk{8wjW={rz}aWM=&aH&FgX-Wi!|3V;@Jb>wP$-|qHkmVShp5{RCj z13s816lTYv<|s~tilWvQqt@1iwSc31MrF8~Ypq0RzzJR?Drx{}cDROS-NWOEs{5o~ znSZ84>NQMY^k_X4YK*$-tHNrmw&;u&%bZoJtl}zQLh3OA8fn<-t>$X53hS^EYq5?F ztr}~x5(k{zC$cH)vqEdMO6#-&2eMM@wax~d0L7t%5Vm^jw}NZ9N~^VsYq|RBon%3} zGNrk)YrDGZyB;CA!fU&}T`<^dGR$kf>VNCL@+*|dYrlePrvmK25^TX5tQP(2!7A*+ zGHk;-?8CN3!a^)>*y_Yu?8Rbi#=51%POJYn7zv$L%cOq8h-z%fn(WDLA(!y?;Q= zF6%qc9k*?qlR?8UbSM^`!!rznG4Slyf^FF9lh3NdZQVc%pe@>>A{K?6&WP8iE(z4; z5r|x$H#n`_3WrIEn04$z)s{}D>ZokULnBB|8SvX`XqKtg2CAY4I0XR%q(D1~?c`E! z<+{$w!pSfxWw%~Y+rl5-zJtpG!G8;^!Q0A)I`qrvu0hM$t|h4OUzu z3OK9l4B$YTE^E9OPtitoJuUNjLzNh!wCF8S`h?$(PEwS|vxy7u>f1d1E`JdqjwslM zJY)hBg&S#r6ATc520ZWd8t?%luyCwwKn>pAk!)Z2YJ%#E1lLCQe(r0$m%s>apIu+q z#Q|oDj$q#1WK!c!2CfJJD_~M^BYF!?veo?NhCI~o;ikqp^Z>kN!v21qU&x?oK!Ygc z!LTIo5EF3`*M`I@M?ZKOYk&2_qW)}-BB~A9D3Fp%6eH>hGas8cDTxqdfk5e*GK`Zt zEz_mxNjT|}u&EdGLp^XYluQVH zXzf5TqS^^O^n+}%72SC1Yed#PXjVT+mhY-5CHKSQP_p7)awZ!Aynnp{CilbWp~NQ3 z!#!kjDO0j3FcBA=gD5u)DvL5HXM`+o@+e1gWZ?xRgK8=p@h=1O&Pps{ss*@Yfq*hX zFzQKR@+W+*A2oGa0CP_+J3a*@(e;k z+q3#C*L61Mf4smGz?wUkZi5KK|AE0XJUd!XQKmHrsDSck2d+dl52#u^b4yq=HnUhY zax)<$Lol?oOG`01cr!HfCr-H5OYCP&gflebYvC2oLB=Rh{Z{^fLY8LIrvEnfcBxB_H2<= zY82W*+miPN1~xFnEkVEw48s4yV98WLP#^7-j5UKn@8cjqk3kFfRKXX{P!d)3QU6J) zEg||9{*11&hWR3U^qKrMdYg4%+1dDPHbiI@%4wtscB_Pp4{H!uNh4@wqPheYX( zodjM~oa7KVMGTEL+M2d{EZaA{3w~qodZR!xV=+q9!|`SY0d!6d1VLZS(_rKP6Ir;0 zOF;5M!@|tiyYX0KML>^506H~@Jlp~lXg~sdcm%*e9*E%K@R(uf7>`MS2IRqxJVOrf zm%Gos{4W`U!8gYIOn|J>9>RUkMfqP7~juU^RC{1K!cU6S+gRLiy_j4~Q^m2ZI%pcf7Zn@!)$ML0E5bDGaZwSU=(RRjf{zxfcb7&&}Ln#=B2uuz1J zSf{hWD55#0uLn6aTlQY!HBN%52bM7;fdLc-3<$ZdGXPabM(12W3gkeCE7k=F!e}Ul z8-zf!o6`j}dkAO%6u{0c&l-w( z$OLG+kfQ`&P<-~BoQMCMF9pI2N;Kuoofil@O4ZWrxj=RZ2cZ1Fn8MQ zCD>vDxI4iEfAE`Y|H5|1=)Qc9*6`+s3@r=q@jr>&}Dgh$d%QShXzuQnaRLJV=COTm6TdGv%9CV>#1 zy3$K71Oqh41U80JDC{tkyiN8SN}@~Rzyc9Y zSmA{kZrI_6A&yw$i7Bqw;)^kU=iv9?@ub0`Vzg^QpjtTUBy|X7MAnjRQsK=ynMeOoAmklJvI_aUjR2j(By0fnacEVZC=;S3fk9Bu0bZcR%9FgAV(B$h*+lE7 z2f*eeY?MxjJb!Q6g!;{qI8X%}JzsbS7(8u+V34dgQN_j=fSG#GcP5>wrkirw2?cB3 zk(Cg&9=)N~q?Q^Ym|I7+DlMgEdi))DaIpa>6xeu2s*z_knIV*Mg=dnLMQ^X4MwUyV zQ-2VGTd+@SP3jcY1(S-lIIy1w12&zciqd#BM1hVdmS%xxmDLL zcHIpw&Ej&nWwy!2#aFs>j_)^xJak`vvjm9PIOF%>k6-@z>961Z`|;0T|BKykuj7vu zCnJ=A|L#F4fd?Ue5)_}QWm7Fk202VL6Km`OXPuiw&jeT=ph09448cd(ek6}(*l1}> z1`p{AYlnh=rx9btw+{F3E66R5;zSYZEHh_+cM%p7!GlT?U72~5CynJ z4elJWiqr)u#W!wt|YjkdH!nczu_S_!b0U7nExJPs0%oPW`PF~~)|+K~r)@7i7hy4Rb%gfCyn zDh?qB$(a<~;#d3gPrIl%+J~DdE>Y_WTbh0TkI7neev(BG5r0Q^z@Q(ZB~r z&_n+e+=&G-h@K4|L=_B?2MF->`TNr%bVmmkS0_IOn;`!BUoY zF&*m4sGc<@ppE3BL>z}@JT4eOTM-BW3}BE03A6@U!t3K7QxZsRT@<4w#b^p(fJkF1 z5}PM!r1s(kN$>dvIN=c=c%X5~3*^A0C4Wr;5#WVNp$?U(MK$VCk(yMc21u0&@>nZt z^~wZh!Eys+!&3@)%Uo(@X1ttW1>XV1QYBP8!-POF7xKX#HG?0jP#V70VB(b*F) z!5R>BFIeM?aY=APm(3Ptw|mx!P3&UROw2uNlq!b8>K5v zu27c^2%zrLguiLw!v1$ppnvugJ~jai|0zfvlQ`7lFtkL;zO%%YipixEWD65nrBf~* zt4$a|Dn^{@#^Vt$%dChi)ly&cX`@*>BB+(&Y`xi+ojJ91DOE{xfjYxeiQ0UmIH51ioYX)}7>9Dmg~H(Mp&;LtCv zA#B0X^Bgpj;Cq3&6Zum4OAz6&DgdSEjqJBetoBHN-=SZR%Ly33@P_B^bTw~SsTphF z00h>65`M&O3N3FrLyFY$LVLvLz-UH2Q2j$yYoxvv0SP{cWb*-(L_IFU1ukr1i%$h* z63MUyE@q}{cfOdiQGc{l%qD@0wg_$rQlZL%>`Zo7`*>nW*90p zCT$E5NzS{m(!B{nbd3R+P-lC&UAFi5`r(F+9;Y9~a*QdKRex~d51;tOH~#TMdGPZj zoUEXmv&d*!6(*eBfjIFh#={y+^vEMi6$!b>Y4KJLOyPx)X2>Q=2m$ruO(D~)2nXP= z{)6QFBqqO+`XTUwm-Ax^H<$m?@DBo}V4MOg)iNTPNC*Pp&;RDHrR)M_W^4!~P(*(1 z1|;w-a6#-ytbejh?ANRUCK||O97S;YjtOw)mm~!51PvaN;ft!m0wvHB>}l}apfp^g zQ!*h0v_+2I==1X68|uL~2*X)AN?P(DI+W)YhT#@!iy_MAI`ZHd-a#?)|A40&p_3NE z7(yTc3@;OmVHk!X_?*xfjNutF0n9Xj5}tt!rI7aMt$(K8;iiDl7{JV;+&~JyVHgAf z1~kAFknaxxF%SiD5D5`Kl+QfgY~z@%yvRx&$l^m63)T;YQ`1-P5WjBVgCD8*^g$<1I1lmb+JRnjGMJbPBp0dCxj1dqi7J%RdUH}My02l`037!B5#0f93B!58{ z6bR~O4H$F{_KG3+0UHKk3LrE>Ikf(;K|>W(K^b%k)aJ=VC`3o}3F073HbF#V^hP0c z6V%~BfnW`q!5genBHp1HAQTADs!KrBLNByORpI5T?w{T>N~Qml5i;aGJLNr1B^ln+ z7Mk${7{NX5Gd~@3bl`zN*&s~WV1E)~OhVjKJ>wH5-=R&T)E4N1J$RD~jDQG?017hU z7TkafM8U@t!W8gSPocmKputbw;1+D?9`vvbxFAxi^CWsB51@bu43!J=;1-aKQlTMt zB9#jk^(4L_4?^`+L$wU5QxLeI5gHE~8bMPR;u#>7Q(?y#EEQAtR8wtrJb!)lSAjKH zg|%VIlRRuCLJZ1QV_uwhO3pu zBV||9ATI+No&g%pfj3Upm{e9f3bJN#Ry!6VWf_8JceeLLwq=v$Ha=rmjrM4fHffbM zJ&LtJsP&!*Wc8>l)bK;owcXn;}c5yd%b$@qv=c9Ic_jiFec!hU(N7r|W_jr*vd6jp0fz^1K_j#c=dZl-I z*UWjT_j<86d$o6aJ;{2x_j|!Ne8qQs0mOUB_k7Veebsk;vru)}_kH0ve&zRi&3At7 z_kQs=e>t~)^|ych_kRI+ZTUBV3Alg__<#v7}iJ7>GofvbM|M-cec#5eQZ=<-1u{evhxOrQ6 zi@o@Z!FXw}IDd@ExQxv>;Kulj)p(8BIJ?rgjo~exR3o9 zlSA$gK1xsN5ek})}x+xU_-xsyHli#d6z zc*ABtxs**=ctaVf`r#W)K@jpl5U%Z%WqFouHd8n$aHH?4~UcvO5`I@nLZ=YGH>LCx}h+BGZkF$B4$$31t z8I|rqb_~D`#JQZ^`JGwvoKLA9xL|hJ8J_VupZou|ouvg?Toz`@!?r|2Svr*={`qx; z>n@PxWq)C|pe2T$?U|qTd7>$r;C%NRG64!MAObj=0W{zat*0I+fe3nl61r`pIT{8` z!N-EI@;JjajDZV?08z;W8U#THIXVI^0195gp&tgK*IA+~dZ>vS&3qS*IHOQS-itFxL`7ppz%biHHe@Qpn#&Z<&aF`9Jrug z3{Mamp$xz*2+kTk>fse+Ite`cvq3uvh(HkF|DzwEz&qSPu2p-r{mpkpfeR?X7`g>n z8h>F6CEz3})tlw948jl^jG+;x1FEeJC0*!!8^hwe5j09JHFu-7W@@n z;qfFOwR1a;V8D9-shGt~G#a5ak~=S6FY=&4JB;N$py3rvVa92^#%bJ|&!bdBAOx&~ z0l1)7Cp^f7{FHpRABKSo;4uZ1JO!2`y+8ll0E|F9o}rjwKopdjA%?+_Hs%}_yMHxu z01EOz6pR6A+haG@TCHDPJ-W?V44^cCN~niC&gI;LC0Z1kg)>{Dt6w?~)H{xn0Gp;f z^+Jyx2_v};QWT2$I4>XtdO#oBBQSCnXCZt%&fykbnF};PTY7+#=DgEAoqlSUI}(6b zf}nZwAPO`A%BcP%MnBri~t5gV0PO4Tr&GSj3*N)A=rg|*n_z|E6|hD*@E)SL&O?vHcx{I|AHbWq;ox&`aDp zJk)c#zzZ`$ z9JQsxRX*ykCEuhuR_|!$|H2f4B?ydt80@^F{E%WGn%{|D?G@kSeOGocsiW!s!&hEB zHv5tai8J`V^7o#QXxi}W*#P2w@kRfC^zFiT&mj*sDvvH84>EzH`G5SwxBbbC;k0-A z78Xx7@q&()gOISpzavJZYa<1GCGknW_=nth&%qc(ff6#|6`sMihBhVq8No?)4PW-c zwS`-x#kXG}6TJTx06J}lVGB8W(cNIujX(YE-Fg$6K0el%PG4Kf!1t|Fpy3COcAWu; zfaTR6Al?ZaNU)&6gO_xZ0VRJ!8k3i@VZBCWM7`_R@1+C=2}GT1De~JbN_on-$;6VU z(W6L{DqYI7sne%Wqe`7hwW`&tShH%~I&~$~et9%#ND%}I3=Ihq{{(@?l_AS-(4c*5 zOSi7wyLj{J-OIPH-@kwZ3m&Y|>(hR2A`mc;fPn$VNrnk7O1QG+%b0&NYu?Pcv**vC zLyI)*RNk#TT#WR{kt0bS(xPL_o=v;9?c2C>>)y@#BkZR2>U}dd?AEPk^(gmV&b+zv z=g^}|pH5x(@2B#DC$G-EyZ7(l!;2qJzWep^=+moT&%V9;_wWOsk59k8{rmXy>)%g) zKL7s!1}NZw1QuxEQvQDtcFuyKEyy5J4(`-Zgdautpg8M$Cz@KzJf1Y;{BoEU-8;AvCbOh)IhBNpm6|K%QvNQRn~UhzT(gim2q0Og8DH zR3rjs4=fW@Nz7;V;NgG+B_!DrF;zOi<3{%UFo6Sj45`tSKRAC-WtCkDWDh1KxEbaT zUZP}=GbL1sN1jgxD(Iku78+5MfPrZxmUzGk=t<7JIp&&LoQVgS#I*S)oCD2S=cafb z#b=e6JZk8vsHUo_hlu{A=mb}KmZhVjO4`vrW}>MkM*B3%B_6~CYvVxYKqJjLzJ_Yl zs1uaB>a);BD{X&$t@`z9tewp|L7dClDr1hrX0%VG6KKKEK6E;uEKkfjJMFvh#w%}h z)%JDmLi^Zrjy?9|OHjZ5CIqmW`}mu$I{WS;aKQcUYEZ#L|2F)N#06=~Z9?_X)XzN~ zMOx*g6+;|xzy<*udlFTy7DeL@ByQa=7^w2~X zZCSm1-HXtkTKwR&)1|5jixyg7BJk8zze5bwut?MN)0ZkI&_1v{?X(ueC|oftcvQ`% z*|6kkanSET(+C%Clp#pR-WpUS9%e(mua`w?0VI%GfJ_jwhM%*w*?6R=4klD<(dPs| zv`zJ;52t@?b3yxTU4q$|sLeRAJP)KfEDi&0^y{$4E_+=_@1?ZgnB=;pnG!_3Ff3ZRJ7?VT<7hI47-ya`z zj`h25xh>ibBfF`e37ykt=F4-*qnY#1|NYpWGgW_iR+UCh#Y8(dr=H(1M#TmAo@yPIn4)Sf#c!IuY{j zf&WTZ2|hKtPjT;GTH;cQ5K@m}5P>lafC1mO)Io0q@rBTHlNNr6Av%pNOq^3+pLTer z9=d-8Di^NB(Wz=Rat43px`;yz>5@iptl#ChgiFf(fJZX|W|b00 zBSp?Efa+`0uY#33IHfI<2b5DJ1(Gf;{Ei^dSRLK|NX~MaGj@T@3%^n~rORnyosRSq z)4nJWY0!v!p5oH6T6n2Ql52Pv;@|dS=14b9ij-U`UC!PSx0`K~h{NasV-ldmj;w!^ zfh#gu!Jwfj>iw%NMqGKOwrloDOGt&Eq$>%a8Mzz9~hQ+FNgOF#FHaq)kZaLN0@ z?>bsg5}v~wTfA*%x69q`K54Unc`D}q>bbOia9h_UY1~v8!_<~*wa_e5|K_`{d`kDY z6p@F1d9pR>^-s8+BCZy<)k}p0)~xt~X2nPw|G(Et*SV){A)e-#Q~Y}O!Whmle!%;e zaEg+=HA`Ejs^pBJ>ha@E9v3!F&p`%QI1)=oQoT5y#+F#(TsTiO*^ zauZzG1@qN;4t54>rJoShxWj4!?qe2K$T@VY+j0Lc`q;=$HeMS2=Br|>xw+aNzfAnG^TLUG zn>KT&U)8~zxtSwHBtm}zA`pQTpt{L3o+N@xX6MD-FUU)i_0@J=%ds6+yWUuARv`>0 zCW8vp6FG+{7(i|Y$VAx)Pq@Ol!tA}YbD0FQ-b^PpVrw?=PIk;T0E>Ka%mJKVy7ZE$ zI}$!|6FinC(eij#&1zfgDs}e59K3HP@1>RPS8A4%#4|k*6FYyaSC<tq@iOL|vA*oVx}M}J7c`jK)E{cRxeE65wtVM}MQ%&VBL}xJTuJbX zo&*ziLaaK+WSPj|~a|JE&6$RXeNBHn81y*?_XO6fj!dq`uL5)Am! zlkW7&TmJGRX}EvGZw@Cejb<=L(V}7q~jzLn9siUw?7fh z@1=0;4m+_oF^i%xYl~GBEk*3nt+i`bQB_p6etEtB!a3)WKRnFX~OrC;U(rejpmd1kL9CP0IN_ zkU=HDB=tFmq1&gvAp&K8vAOt=};gSk(HxdmQR35VJ!zNz?4Uz;#h@PhqCAjn{&C+VLmZk zmcRoBs2|!xs8#Z)1$ZvmwP;S(v{3kGAVUSF@XjAb3K>P4+EbVwc3VuKfQ%_@SuK=) z^M7JAL7Y=uoK65o00_WybUBNk1?L$BsUZfHeM8tn-YQU^GQZ^)w=|UUS~ukFfyZ<{ zx{=5>hk;~d({}#JArojY7|1KX`-TK?7e4K14$6W8U;zxCv@9nC8s-6j9^k>WFBnY# z1#H@A3{dNs!!M99uQ|{XEl39gp<(7BuR;vr;KOgYCy;Df6~NO|zo#nrDA8=$q17@S zz|#zPp@=}iLl8BE6O1Lm2kamaeqb29w1Tr-K&#l$t(fCYkft&V%2ozLdyRGno@-O` zlWc2W#MY}4aAe4Qa!7}S1D}pFub@gP%qZFk!<-q~Qt5IVv@3uMMH{XHC!@gP$f~}ZFyIMf+lV6sH3ep#8_JVL|Ri~HZ2nG zXgP?~Kk4HHimNjJcBLx~118bNYZd`;3cGNt32JBiqS!zngaC4sfa~Q^VNir30P9i_ z*rI^Oq!?6YM2np(eJa7U@wrNG*M`m>engY}C3FO*ZTMnY=jcIrh%*DpIe7#!jaQU$~i z@j_B%y$Me(>-iBI7Ak*w+ciBL`?342>=!h4oav#Wt=xQj&&*ZX)ht3XS09*Zo733o*5%Hc8x_t`uC}HM27_jNLsx&mKwT zihWyqs>ieyC|Yv(x-JN=lNsfj4gU zv&yu$MK}%X%7RVttkEV@rj%khwgD(WDBv__s&ECTq5+pC7?wYPAR*xG2lhpi(*%He zq`DuZ3Y9>h(;@;kkdKuD#=>)s_yaNgKnRKq2?SzY8@Dg91UQwiTNHM2MB|VyJgf*O zfEjmvkx12u6+@Hma;sC#oXNPD zN^g>7)|K6?xMG%{t_8(93kSpY8W=^*z!g>cNcFha3f+AZRy5@HH7vp?T2A$frB6 ziVR1#Zzg`h);)_D(z(wH@55hYCNCBajtrdvkMWR3MCTK;mt_6tH{|1pdb;TmO~=hbe^(ypvo4 z)gZ@19BwjW-ptnVbOpYD*#Oj^Mr_92N1{0M`0KfOsbY=Ig{9z^!e;E$^Q4?t;uALF zs}SrtL&jl^I6t)S3J=v!SUa{8^t0h(dQp*my%2Ld_RAE9@hgF~kE%}7R~w0xLFIrq zXVb~kSJ$RoUg*yRUQ^e{y;iia4)+fcJUA9~w@HkYQ%%Ve^0TT)pX56dn@Un*hM;`8a;U>PNsgJ3YW z2&C*r9^$7{S$*dx1wR2$bKoZ{kTB11pmH`S)37juK04GmtV5_*J%X`^q}LBSk(hLzd7Ie4_O5|a71y#mQH6m{Q#RkLwZDKU&6BLRH>c)G zSl;_j!_d>~WE^#9>AZTh*Tg;tK2@!+QtqA2Eshb*Mr}*062-#}OOqS3Xj`p|wwtm! z@_Y5(NxK`FH|{>kv9k;owu^o;++B||W1xsr5E;elPFG#p(9Codt`FV?Ap==&!D z0i?FT{WrH8d*!ov2F+Iu9?^%C%Cgeug+#W22Z^x57@VwRf%Se-1u1zjdyN0d&I0fz zXD)4p>grSkt41KBlk}L^;b_1>WqdRvHt;0F_PG)zP!Y3Lu$F{_wX3jTun{AS_@-3xW2h3Dn9y1FTo)XgcrMQAa=$E2O zB3U^~ZPxEzf~qiWai$tf>AQ;@DK~*RIN0DfU4$PUTLUoEiAl(LEcS4eF_>wp6cA<# zznOABIDOjs?wiRw^(=7Uu*$BW3)6PsE*ENN78iJlNB#J52!rhs7oLC=zC*fa;Ipfc z!TO??|4p3)cD|K>+U`P(@4&@PCn1+G_8bVa3JG%s|8e|ij$<*@To;qw|5TEYH-``p zto!ov^a8zIFlc5_ApjfJ{Mvle!MCdbWJ!&7qhu!SWiro@?odKS9&+t4WJM?oOKh*it+2IJu`}g`PEYs( zZ)G6QtDdfUn;ZqW9!(S-mCYQ%04tTvf8@-5#lJjME>|LHo};b<+cn|X1)|mu_MkbjI>QCe@kt30xXkuJenv;h5c z0VVgZ++#13*>CTZ?~1kUh{jiOj#oa~b4wNmfD~v&;;ks|0HdV4^K+7nOCr#=owjb`$(6lI=dPhcPR||T* z|7>ETWdI_%M;fl88ypkVhQ_2dq!XqxCRT5I$|Hzl`wTuDiFl53IfVtO zWaC6t8anlBdOMx-RUf+37r@hJy;?YW#(tfw^*ZsY#XuD+VjO$8Z$+Boxo=gjD3?|B zD>Z#Cch^B};TRm$=-Y=2K|ya}yBUFxWl^_zk>ympIeND7piCAEMkUtWaJwPY@+tILZ*16S&jpo!(aCVUfq*x&Vy%M8KIY{a^x z;5l@X?|3xs)VLY{Y%0w)uHE#BIuLTV zPGy>JP-#=E=%&0o<)^-e;4W{oroa2oBUu~L&28HX){=MsNZJ!n{5!vHHM5HWp~1{d z7y1sv6}TDsa6%=#~ zlUI!%uf0~AmNQ`EBO)c>f9nTA&q^>bEnLUb`(&;h=z5qc7M{SO0WXtn3(Z#-W1=H+ zk-aSjn&Z`5%rw)X6BY!T{H0Ar6&Brx6w0Wc7aMV5-y= zW~0(PJ%xDx7#QdsMegT%CKK1vqar=89=1aDcDL~t-~)cg^|8;eHpA?*A2!}9oJ<~c#PX?E|*zbBtAH&nY<(h5lu45+XCW&igSJz6n zWk_qiTA!t)^*Y<_G8meuyZ7Io0E0BC3Y;ljqG0slqi^D-JGSt1;EdZ4cz~bvL@X0h zbu=A;igB9^DUqEx%$pRKkz*c<5laukhK_5)QGV&-&PN$;8|5@uM56U zK;AoqF=f+DhA|G|07)E9P}{$E(j!}Q^>+;3RQ6BF909o11q`@jUocwEEQVBkf}I4wA2%ZFcvZL^$N}{ z{pTsBZ*O^9q+#XXZ;*6hy#TAh70CJ)AnhM&Dl`ycskQJ)YSc%R!Y$H_ugeAhXEFsc z_tznoCrEqr2p$QH`TX8{r4ZL+&yzaW+AQU5W&>@zGjqgLdm8ut>wMO`9(_LiYy3^o zwd>1yZ?qT)PGY#gataMI5^v{Qn4_Mj3KZW#GXyK*JR zijfu@DmpRlS1h;Ln-7$pY_P?%GjDP#y)3kB_ysGp?ptb>8Iu2KndLZ-p-R}8Mrt9i zkc{lNg4jd&f1#k03i0uNBnxAh)%=u-XNDp4nW(;U?) z6=)l`#Qj;sIZ~b%zkkfQ#ZqEK;?4O!YjX5_GH+>*GN`Sv?G6B;Vgg@PivpnpfNZ`4 z+r1#8rTrQFs`)!M5N{&0a{!>1b%5cpo@VTBa%k_66BYmTyb2f?$0B+YtWm-&!eR#K z{>yue-Wf&8=plRHL|Cji2q;5ZeuUWk`E(Q~%8VsbgA9l;q=JEq6-X2*%Op@id_*^# zCnQGv#YasOU6}ZV$^0MRnW1ZjwND<+hyl^_L)Zw<6Q;O#I}3@J7_B=A@3?O&UZoQ? zOXerAfPcM`f2e3AJhP9({o8h-XQeu)+=uzs2kchw{ciDWEO8&S&2)IPv_zd^z z`2+*x9|dTPPqfhzg~A4gV>3hMY21BLQgPIljT*bhJAv9tPCHs!5L42@K z87(yg@-1f6xZQcRJ+pkZZrXs#Xe^!&O2q|w_J~M&==jPJ8q4q6Gm2ppLuk1#^zwxz z8S^WqeUI7CDTwY)kb}kwOaYZ#am9`ZMN#!3`8}YJZ#?-{EPo(u7Jp4*rFqSE*!N zb>iDoKb|M@kg`v{Yc3%Z4XcJDLgI{*AIpuUS+A?_o+av`Mzs9F`E^M+x3St!9(j6a z6a`2(UT{&wBAW8w6Jt3GrT(`_BzSu(( z7YXqtjGeVVU!>QZoioTNDOyEB4sKr)4>nATyjJq|ncKXfFTgN`AW%oZs|rdy*&B=R zPQv8TeSE=~c;7Jkz9wV`_&p;1UN+rR6}`qOqozfp=DcK2^#*&~L?%g?$!-ALu1>kt zVLa(Mp3(X~vfeYBYD#uKs$$D*(v3fwh+>=5H@*BA7w7t!9bi7L$B(Klv+qMSxEN3UGTKn zsc3n83x^jjG?HSll;yRQlYV#2!(w#snY}z<6lr1sgHyvkZmRt6bXhNbT%?!H$mP1CBjGFh@R-M3=u zv|94BG~>0lkhZp*p4D@D$y;2is$p##Wo?&Xjk~yGb@fecP>wZr+S+M}@}cO?`$=T1 zMMMifkGiw8jfaMfr}b5r3=O01R%kb#9v|abYID2E=FTRYOMjBv87(mt@T$)ykij;H z_tUYjb;_KOMK8dbje1_l_ED5=XvU{_;s-~hk}zj?1i&(E$u@f5*4gBP-eqUJf)^)k zhl|sgx9hZ0@6?FDYnK!?Z)j~7EUuZ{WS7=IuaZBnK4O=-Z?SE!mgt+gmZ%$H5aggIhR`>?<`Ks-#Ut zon(04X`C86)Mhx;^=pdWNYsOa(QxV?xMGLqeTSA2O$Cjld&rU&X^LaJ#)7D$O^1`? z^Sh4v7LHvRjy^`codc# zg&jL_9L>N^G-10+v6Iu->3xT(CG6}4_US%$j@Rj}wEY`tr}rjK$r?@{?mB(KZQ6c} za=Hw71!g#X>33SZVY4vpw7kD$yX3UO;JmhEwaV+fuHmdB?Y!aSyp?acdDnR-V;LFc z{H@9PyN1PXzw`dmGW)di!G-hT1?C5X%aQa7Ew9Tj6PFWd%(0Wp@2HhO{w`;wF6Sbq zf0|tWO|MM#xm@hKP&D-ZyKn*W;auylV5x4pf@ds1rmm2H)g~Cll_t}bHd+{3=1Tw4 z^}iukhGkceMpwrFT;Zo|OpI%)v-MrOTK08!b>T@rE}`kQaoA1` z6iO8}!tc>dowmz%!jstafnnzy`JIs17Aa|1k>9b1I(79h#wW=ADh)Ek^I+U14%Q_K zzA}Cjec)A<%0dFj%Zz@e^+-md5gw^*)JnHm9p8Tn4^7}f;^E20%Xamb)7#`_o*PM8 zuvTWWc7r_@sBofs9wv+{t4zu59#|4%Z@<4@o7>RKMZ8C#Jj-x5);!q!L0SkgxIr5j zryF}sX2)|wXNI1Dm;Ji==vX>xH3${t+C_t?-?n-8i|?Iy2e5F4=J=B?8WZSoOfUFz#4@H~zxC-5*}{ zAj@`Jc0!>sdVyPy41oO8GIvvx-3Y$7a{!1?weaE!?69Z#*_Hy`2o$kxUCbm0^6 zw72AlMlq>npV)AzTOYW2(=!2=4!K)*f-yeU`bm+RG^JdFJ=2HZFSmxxo-LmL9YbEP zr&Y9{>5Fas^I|FsR8V|YwGi{8#N^0m0?6I8YAmKw=5~dteaZRQBjip)twinl*CLj& zm<9b?hC5JYhCzZ(upvM-Ci!XxAW@b|@CFo(NtZh;3CE4udceU8dvHNJ!{n{8VRms} z)OYgo=zG)p;zr)b=R8XVj2Zv7iel@-AAI|U_j_^rJpij00Mm~@BX2vCTLwq};zzwL zl4Ao8=z{b~;!x7(lwQ$}kqO}kI|q4ZZN1^u8_@wCK>t$w%d2(9LuEg9Q17CVkjriE zf`M^va9n)V%dG#lsJtAX%fS=GU#A0;s6H2xjK?VM$E3YDK4mrWa`+h7WfC~IEF_s~ z`o}0{E*jFnMfc}n$yXa=aq4SfXScm7Z+h@k1=-}-27-hThr(l^>n~b-0znrK3w^pB zp5$cSOGW}GzepXV2_p`TNuKKkj2j2vw{21&?{Ts9f1%8@vj`rMpTY_jgK=WGO%|}3 zObePBJ%FgnKU8Rf41=le7aeA*`ayqjbZ|=~F^S`{{saY|o_+)EX5>rxpPzb&I$gRz zH+qGd*}1kJGGCs<(uSX0n<>m7zvCSq+7O1ujCRay`%Y9~{Q|vrQP8@gu##scdo4z17&yLU=ed_EqwoQSpiN5Zx!mE!<7q3ebvQSM5-0 z_d@d>EjmJ!dP(xLbUlP>?xt!r0{d_~j$|oMqE}7v4T)q&M{_j2`3HR`h_d?XmvZ(! ziqH5pzl8$yij|{Y!|LYn`tLKMmAz*ka&8uj`n*fEcz5y$*Vfw0 zYM~O=3wPzfPKi~WT`q5^c?WU0^vj+hDL&sF>4^@Vy3a(3so%9S!=671Grm6f@Z*)s zTcW60`Pr{k1*8GLqLlgug2SIImG3=QyU`I)={H8zuLh;q{02RMYWfRpYw1w@<)+ad zx%xFZcAc7gIQ>3~Uui83`j7>WD>xfs$1c6Dk>!EN@N;C`<0xbvVru_Vh3#3sGaVM* z=_A%%HwOTZ(1rg}T3tzp6N;8dduIB{6|V}zUu3`R zrz3-J^2<80Z*g;#aLl12VO*WC)9vvp9C>ZrAol2Q)pz3rrTF-(;00?c-=k;J#~MdQMk9<6ng#LWh4LPiM0P2 z^6PLahkU@^ne4_`&Na(s|37kDQ^iUl2YY|ycjhYeb7k(HE9|zDH_(crF~B(W9|tFD zwvgvwCHBpt@RMejSK%k;8n=Lsd6VH7L7u)3X3iQOT&{QQrn}4PaKg-8x1>CrTKjyk zEIMNC?d}>f5I6O9=>F@U;j_VrZVG>`%ScFAp>`Pgd`SgJ7h0gMBg~KHJ1ey|9fpI|E+N3Kox)zve)>%_{-Q8 z4YQopU)jh8fIi?_&)n!}dGhA87Vka==>~rudTcj5rVzuvz2`BTZ`9;fpljjx;Dg0> z|IiiLzt|KJVaWSRTqcjBBnI>@3C*g^ozZ2;1`sHZQ*O<%jpMoLs0wKww20cc^I`L3r8_?q4C)F!iM_@b zwiWf>jI+(*(TpT1WlWF_#Wg5|k3HOW(d~lDweV+P=RFv^CH|Ugs32YOUF<1TRSEx= zf6XYpI;UH~QFo2KGZW%M5Msq>9NAvKlCx1X&g`(Ua(~GT4R+StO zZFcOofxNuJ@Tv~XU?#bvfIUKWeM4<-O8u7!W8i6P|+*UyP+&>OJ zHpqtPN2MXmSG`URScGYcxG|+9rERc6(FF>oYbR81n zbtZjIT{RTJxN^~2PzLPe9zb!rqd{@2gmtITEhpev)ZfnbdUMFovUvvnlY1S*a-ZXJmG9jQ(!AR1_5O&F zTS3ZG)z9n( zxE$SrVVptb_|3j@5h_)N+ww8GR6^B*jTxR8cpj@rJC#xElviHCH=?YS*rfg2wu0w< zA~GR$ljfYVvXPS@*y9`8X@Kig%yTHl;7cJz7jL%~!~-*qAwPmu@_(pXbO%>W-(^ge zH2^Q^AGo!t+Ef)o3}d;E_@QE8Jt{0t8NtrV-68c9P-KH9sZ#Abv?NXuP? zAX@mk#Dvfygji1si?BhkKd*{r4vqTlTXGS5>jD3X#n>?(u^-C{D`|7_AKR0_q8j2x z>l{W_&2llahQdaV-NzN(EOI5+HcHi1uC*d3@~?*Rn{CFwEiV8_1T%X%$%NpT%zaVF z!9@|PN{J737M=Ar{kQAFQwm)mmYPF(G$K?&W@tfUcv{33Q`}DdmfwjCB!LQ+BmSvJ z&Ps1yeM$-B&5V32tC@ipa{aWK4=ZSOGPqo>q;KSXj)Kf|2%Sts92uqf9@>qy-?4FXm!TilY^KGYLGqw0= z8dpWfH(1esEtHxkJeI%*Uo6UY#OvIFyZVC4K=;)Gf6n^>Ub{bSFIk@#b{8_CLz2rs zbGrJ({Ttl6UvU47%%@khS_&YVC$@791zD$;h>1PoHAj>S7zG>9^@ZSUhc-^sA-ZC0 zWgrH@0kQ!j*^Nr;@X`&g;D`CLBpwmxSJGu@4pq~B$NMmUWj=OZA`&DrcSKjs>Akwr`Z2l^MoEC?-gY!v5!~`h`!^WoV9)WuuF6-x^4k-CRi}-I zNBBYmqHwEPfyG-yk0j2I)+0|{d~$54ks00;_O5a3Mtje6K(xH~oFn4p=77gOrRCYU34-Le^Mgp7dcD zFzOHgIx zV$J3;KC{iBIDP9?d$Qa!cji^2S{CoB%p$oXLd&WZY&(XvZxy1c{Q92w_Dgs+FoU1@ zP9gOeR$QMPfK#T)+AV)dN9|C;VgMK^^O*a@7BrBgGAggrbd=8x;#!uTQu>9djE)4+ zwaB6va;|BIK; z-+Q;mHwU$<)ytn2Q@yXrueFNFp`#CZDrX7s4qh79aHZY4o7IsH#yt9?hMnaCM=;ntI#jn{9S61;#gtCz>Aw-OqCBpTK+EmdindAeW*+90MB3W7{k@4-DR55 zKu+D!ejc3JjASxz3>(xPkuiX%=&w87Z&;FDF-G%oWyiCzRm~TOPS**}IQ}Iq#t%j4&4JIH7FnSKws|`LLP4_kN6hwyRYIws&Vwo& zb!gQyOpAuMqzN0PzKxUwCFm7CS1rt$P02hzABIC=XO`(7%0z*a z&$Im8&W*j9?~4J?n?GfUOaabx6; z;s&Uz@WYFy<1utB%lKs@&N4ZLWD) z-dY~;gPHlbp<-D^_I`?C6D>YD=Y8mUp540IA-}}uRR^iZzo)-`jQPs>pAZUCg<`Cd zsTPP)(8}f)1mdxbms`X3AR_ZzKx1JL9LNU812@XmBjfnl#&!5~7=j+(4ok&tyL#?R zRY8tXoRM=lPpy3Qr?TnLe3fh<`a4FLZN3Buc;{hPp(V3vXetbhcblu#w{=D_`Jx25ggsrs=E?xa z=bN@Aaa0-Le5(<&jCfE-JkHu6!SQW;v7N>2#_&y)_N+3VqA>kogElZs*U@6U?}S=m zc-Z-&qlQF{V?DnMA;Hn6=B6TK*ua?XJ|{Y{#y3;cuVAa5SZ#M4Czgk1K>*qEViNOW zxEGgG`(n9KG0eh1whbVS>7+p$`L%?C3*^CK$QPaP%?VSKAan<^3<@Q&qe9}#aT~8pdc1Xoj zK))Nz(8d%`l`xN!5_r6mmB3}hUYsBb)U~!NN(un;xth4mgT7hsq_pwLcGUh|jahRN zy6l&;|7Yx30=l`WhbgYf<`d5mt;;DWkdaA@MGq;drzg!hBnhWaQjZR)Cz(;vJciKB zuFY76r6%3?3xq=-vy%%`M~03k*W4-#2eJ8$pJ#0N}uOkQL>f& zPEN%Z;s#DUMod!ZWh@S$uK|mDb!fj*3a=0(m!i>(Y(g>O4QAorRTg8#B8LKbV2~rX z5m?-%uGss$q4$MkNMhY-lu@;3VuMX$uv7e=p3bD+KbAMAf?h~MXk#SaAKoL|Uj^8yM7zk8IHU$?}?U(G0msNMYw0F)K z_Wo+}o40XlMr!)wUZ_Z;k5nVWlIi!uy)i@`x!4ojw z3*Bs(HkJNh)AV6R>U}U-QJ*|3_~5H*@bt|MiRI z>;G-F{FnIn;-H1{-*o^DiT0!gz(E3DMN&Pb171Udlt>`WLvJZ0*bGT+OQKFo0K1SN zxO+$%P$1+H5}HJUKKntFMWTI*q|?4f`wU6nMWX+8K=+CSn?*7_4uE|^GH#O?zwa~r zATd#pa3+5y8d+wJR_3HF_|;aH>$0rhomrG*+4Ni4o_%IDYrWzk%dTyG#YdLoQ7cF8 z96PQRktNI7Zi09$%k`|4>(K;fODp#)oGi~BJ?>Fi-cPN(*T_68t$aUPdADV+Qe^on zTd&f{32?}=owhJ$DS(bYAg>7{C9e%$;aGHm=ejkD}?IYn69&3-KHNrRbvb zy^h+hB(=$#)-cpL-Dv!AYa7Z%0jHHwc zz2|{Sr1nIoU%+M(9;N++jF}j}(-KIWd~nPkr=XeE;YOKjlX%4Hmv?;AOu?vK!BXi) z!`WR6Dq3^`{<>}aa~2>hvIGK+6?b*X^vYx2Ab?SWk^@QlPv|hCwV2-txEU!HeN3A- zf2YoGT}|5#%at3>zuxNonv@hH#xYwV%0Pcgof6;=fdQ$)iiknGdHV>~GzFP7jGn{0 z6Nh&T8lrFC=OtpFb~tT!`ruA~?04!eYSm|(_6MRZW2%D?C?+I$Z~))C!{y~B$cE8k z0l4#vev!^Y!x1s}D2jo&Yv{{7v9Im0=L)r8>#%_bRH6v{7YWDukiCDk=MjcA;}jqI zDEX}VYryJV_k7sUL(<*A;{LcGEI?#zs4QVXni44Nk9l3HIQB^Yjr>1@QxUSs>8Gp* zH1CbH*O=p^gdTZ)FX(P+o9bBANFv{7dayPaJL#tsuix#p>Ti!T<0nTM&ymctFY)pF zTWJ2(2%6+q2owvULF#lMN=Q$mj|w}&?lg$!PWJKCc@(VFVTYjm$5WmTQTEuW&kEU9 zzrKvUrJT(X2IlykCX6aIVNJBv$=*p$N^*;i;+jyY9?0=Y71CSlqk8j>WhIKU$>^t#Np z)bOSJ^QS{f_1ZJ~>e^@}vOF|E2&?wo=WpjLs-<8u-lIs5O}EK*++)#K?rEpy zS}%>-%*VNt`@Olp-qgzfRWt5sS7U^0JiEau{16@^eOfWV6iFA4$EPH!o`n<*JtgDxUa|h6k^M>PFIU9-B;8@= zR34SyJq2c4d43-nXr)5qN`o^pxz5z6Y(K@3KefjzoTZ9qX{>MGn3>QP6<(mu8tH%o^E{pXP#NG>CQ(<+`bxB( z_Wnnl5fQyQckDE^#T3=v3qCg1|G_$CW!c`f5A$yLue|DQZxLB&ExzxA`hm-5Gp zK=HEXMLI=tr~8X)2`dPR$lnZKloGZPqW90d7{5rycERQHLYX}cE^GG%3Slf8D)%I9 z9U>o@ipbg(`m)5Ou=vcRA??F^>sWm;<=QFiH^SNcEUIr6DLy!7yKPlxS!dH3e&vo$ zlVi`rlk+S7cC7{(jV_;A?APCvs{;Vu2BwZq{O+to*Bnb=e+YOjHq8f7LU>Ilot`!@ zGsBt4Vos!$5He36+$}NYQu(Q4e$&AO4Pvf@79*Hd4+@|xP`6?F2aDOGe2*tcvR!&P zqX;1w0mVk#@YA89>c*2b>eGApV+C&w1D`L!IJ1mwF5JUp+Q4i#h%T zVX;4lBh>w|5IP6~K4N_JWHEPEBL=(2W3FXTo-L3O^BPa@Y@U-t4+{;xRaZtU094~N z03>P_1@w-OY{>4gZARlS_9{P@e??x1rk#1LnLJb(arm}ob@wyjLO_T=DWJ#9xNC6 zd>3RiEQ>yuZ77SLkXD_jNP>v(M5A7e?mQyei0xDqmpxcjC$fL>t7oub^Gi4g`@`G( zw};}hmy`O?z^^T)e$*{_+~x%~jQCJ(Ct}&pY)AUZ!D!yy2xP>!RvGWrEA6-$*+hC; z76K90f5n1p=OynK)!p!2+AodGBU|UbFV!CH>}GeD?JR;7lCAlCU6Thc`PiVWBhnig zzuo>>(UhaMq~{C2`nyEKAJz1*51WfYm7MNPSMt^Qjwjgop67ANDng?omdpMgMEvEf za$Ll}&hw8m0ABFJqxA`Li^;eLg68}@WC4%GKvpiaD{952VwZU&3X4ZqD_|g?7qiF- zqxVSXpqFg%0JY_3v=@$7Ld&jv7BH)dg|QG`W-mViU@yBuED8N?_s zx+tXBkff9#)6bxCm7XSJ2BF`0#Ze|5{)ec)Y>P8bqeAySY$=YTg1QAxuc#26Y=#jKp3%}s{Jp+1_yc{W@a!9UYiZ@h^T zzhs1$eNi$SjsCbRWVxn%u=couwmGB=b{uD*%RW%=EvOqe0}VYUpbfN}x1!T<*`RfucM9$%Cui~&jp~PPDJJdb6c=jD=V5P&%_!tsNtqWLUXoT z7$Vo@os-NwEABmEncE(-7y#JyAMsYq0lJ*uFd1PciduLQA%~qR|pnSQY0J*v^NBXRVT7>;aqy$$BIa7@BR%mf6yNx5Vvr2Pv!Zyg z@$En`0c>O~5jQp)!SHZX}&EvwvrA*#*c`c6Hx&_*KXi12K9X8SRpup2M zQ(_Qlfp@MbJU4R;ee)IHmoiD}16nckdIp07un;CAc+B&BBkuq=g0KjuHauW#DDs=* zih6f>HHuSwfrmWZ^)e1hTHnz*LXsT&H=f!FQdu`t#i)LME|vS@vT}f`mZ%~@+SVP#)J*`*>!P~( znOni}hIsm8*BV%$c27VyfXaZMX4R`^G?;`m2Mf?>*l|#lLev(nnjTz{ziGi#1D)j( z+Tq6f?!;$L0jL_Y?^hr`Q2XOl?A2B{XEW;dmgNRloi^95bP;tzSp}I8^A9^1#hY3& z5&CHBajPz*o)Dwcy7RXUPPe|19zhg1gFz_$s<}-=I)AE9-)ksKZGpJ+pA1{@3s}9L zjHb`jWrm82iyI_tQN_(07AOD>7z0nFXVVH=Nr0a>(y+-Gr}&^M=W%`=Ud=Pk8_)QAJMqao^Z3n*KqQi z9wo~HC9|+4R^Gri)m(Xk;1+3ns5h+<{2pd-I{&qBJ-7XG-*r*o`$pa)SmM2I4P7<; z(hH8RZ?PL!eh8MxrgFxJ+}xN){pU27IDd6J#obZmtamHkBFmOvWwDjQ_UYPL(VLF;Bla5i&sIu#zXC9IUtg5WO=Yg$cF@A5_ok{& z--_$pLZ(^&-VNMgGZC#l1vaT~mLbrVFG+vY)Q9)pbxgd6*x#FThL&|hI>R>x?ZWx} zQpvq}J|{A-5y7yr88_bp0pRs&!kOVTIfWj9`Jue`b~%QHEKV~p1JZ_<94%m1!(waHt&p#}dU4$d{_d>kY(+&l{#fC?MX zSd#V;O~uXxXzWj3o_n^KZ>T`H{r$r`d<{6h@Bwf_;Lp@Uf!otIVgkQzRzZR)K8=wP zIZ(Pw@{%~vNg~{3%zScbRvNn@#F)Mh!(XI-tz>}9B0N{)y(BwgqAPMqu-~;l7q#&nYJ>sG zFG`;70^FdijL}#+kQu{LEXPtPoYBOfKpjbk8Wqj$uCZgXF-k55M~dPGjU^m`W*isG z91jm2i7--baLeE+1c<;5RKy8Ms5?YKhqj4SfB2DlD1pV`>BzX?O~PvL9ZdsB!=iMRIcD=HBYz7ZU(2 z&Eg)qF9akG{3>qzmZj7$u8#Di-1TPa_N#z`_DDyzo%q*`Cu_@;z7ojKC ze+F?&xbb8pFi~8|Bqk=A$dWzV(>)O|9!p~E{E{Tras}Jc(cV%UF9s`#qAqg~FPDWJ z%~1mKfQ2k89j`?m1R=9Na0BeNF!mBBW2SR{>e5&G681t9XKrr>cMp` zrP0dfvQA9m-htv`@?b!Mr(m!jaLHtdAVae>L&;1f9fJO>lZ}+iSS|nturvt{4J1j* zJD>7t8b>@4a8&wJ7^5mZwI$?e?>+NW|4;ACEDb>2>hp0_5F_vt?hF7fW$-F1f9pT* zl46! zX-ITaYen|JZQR<>PROxSxG4|ZBSt0i4uj7|Su;nIa7PKTM~&(>gVZ6M3bTyi7VJb$ zOYFpe(&=6cN|}XpI;KQVf#w|)?^0yPZ(vn~x@}fzwE}Gwe{mK|Oasm# z>P<#{?^jVXSaH-fH*&mq)JKzbCRIhvAXK@OY(<=kQcmubZh@-UPJ#A{ZGvgls+3hC zAVP71-7bLDMxsVAB{de518;FnhvXTfaZ7SV@KWRH^wlI7>j$Y;VC&XycZp!}22F!- zozT<;^)oIVHaznw$oj-}fYK>=UcpXEwIoW! zMpU*SIZ#5OfC!M%bWay4iC}AU!rT~3G(#>ZB{5r>jYe~ja@O zBOp16#7H8bNJ$4fF~wX*Edro`|D-lD;c zCkIj&S00AY{{C`NqQoo^rFj=uCt9a?BljXg7G8!(hd6XYL=hyi4|Lf^&JZ>OhoVJJ zxJyjpaJKJ^rUlaSf8dc!s$C@L9vZ<$YBx1k(`R#6M}yW_hZabM6mAB{QnV-}%0M!& z&j{DWF9S~kmZ)`djC+}oZ6lx#@8}$Ep-a6Kd(ZV8*!E=LMig=kVH@EaOre;(nYWm2hg4)aFU6UZz%cp=GI~zdZkxYSMI5% zJJuL3<(@topROd~3ThZ0W}IpowQR`5Y|E!{$}wB#VbJNLLHbhWG9;$gx)64yB_O}V z;&AH8w}6cD&f%XPhov2-QfQP?E>c(%;x&miXg!H{la&oc7@+?(k*Wk?Gs{j4;8fmN zoyj`0e{AGY`okDpbWDAERN94wM9gc?^(EZkQlJ{7tt6l?!bnt_rHdB*ssyUHECIMn zj@#FSs5cMpIHOM+waH`8W~E8Wpa!4um^OXlZHk@ zkOm02V~(q9DxlD+3^E}t2RgRXWdgT01PI_F9fkl>z%`F4384YElfYQ1iU`c&dTiTN z8>`cjC4V}Gp@hnob`T0oETFcMD&fjIw>y`W_#%!?0mqDj52D+uzy#K0Hpj3Ezl zf2_8xOGscn$eC#vQ2;K0xE3l)TEve9V>n%N8dS%#4{}5Cp%qiiU1fmh{f9uF8fk3jD%v+f)pnT9pLEB!2Q}W=?jUmsi z%%IK4)dF$_Peeijr4iDubwpuI6yh0JnerMY?&cb3&cPJw+zsyhnWe^p@f z$E)4i!DG*;r*Pu-ArNA7nx-iia#@RqGt0f)p=Ws(t9sOZ+qGRGg{B9Gh9VjwZAi!5 z?;UT_{c;wuAv|u~sb+UPf&t@EX}BiHt0E!4kR|9wapKIO~B|I|j`v5q$*0_1szA}lf_hMC&-s$u4N%X{SUOX z9`FOdHppHpZW)F;2=UP;1Ykg&^dbQhAM#_!t|`P5-Atg$;O}t)_m1R=1mE*NUrz~N zD{i46AldX!-}DQj4*(_j79|Hsjv{ofz{lmXwSU(bD zii|h2=FR_{JA3{Ne>$}2(WFb8J`L3>>YcBMKoRBiTMh;ZUXi_(gFuL3T{Fsi1|tE3 zVLIiOa4^B{>W721{J@0qc=CN72O}@B)VhQ16DIs<>=c%BJR>ts{^6hm9?Z+zyMGTq zzWn+0>)XGN|Dfvp4$n1WAb}fqDfJZt4A?eQJ-5I>fCjv+e>GbK#L;A2cgo4Pj}{X+ zpapvb*;Cnc)nT$7c1uZvNs4%M_nn7E4R#!3E*5lNdh79rV~#rR$YYN_{s?4{B<;uJ zIf57vK^X%kRp5a{{nkhsf~2$&TMS58+*}qe>0X8#Y5@_57E0O)$ecCI zy6UdWZoBTj3$I3i<|ofElw?8)6bVF-!@c-sLS;h6Wa35{21`lYs07Ia4J8E|>_#KT z42Wq#@_h3OB^u9kjwzXl0&*s%eDjbzujBzSCC>zLrK&Pj*eXOQVgk!#XWmRqNwB1t zlR^JGe`As}LEfYY%R8`~sRbT(q5F_TDPmHY1J6B;Nzbt!>&(q(T1${KR1?jd(MaEn zi8HQ}G%cGRI<2+R#LSA&KEz zZn$-UfwXBr`2`_}0%=087!iU21bj|_1U)qJe~rSoaG`qi#q*63DMR^b?#xYIH8eiExA@ETIWcSVG3#ryh9- z%LOE0pa@jXa+o8?H{3vg1W15)DY!raSl0;s27_&WOe|qR|uxG!Am6hY)HxFoPTV0WtRB z3@yAdeCV+tnW$C3_-&{Jmr=<%(kDm`{;y~s3K4e_sf7XUBYlb4P-3|BCPivcYFsm< z9JywXJU&v8>~oAn)R)0cey=anQp^Vb7ec@tCzdFzr7drX%UtRbK2x-hy)w}Ve|<&& zAO)Gg*S)A@BnhOT45ApuF!GRs2pAv-x5(T%3dMldnZk^DfI$vK0X-Y`z(B8XV(KE0 z1ZZS#CFjV5ASh7=F>3K3&`5!BG~fnfG$I6&!_qz60D~03V4wsoC=5c7iE9z#FW)G{l zL{uyh$SYZ5&J5`q}JMxe{&e)8UF@p z(pXw(00uze8|##bsfU zqSH&MM&H3v7D0nnd;8AI^s*vhaSLwtvkbQY3BGITg<3SFku%Uk$l2~g8pLpDM|#@H zmRv@-;UZ*Uyzx;?sH0mpe{mnQ`WK^dWlJMh^{RKj3*PXGH(p@9kALcFTu7bc9N$=P zJ;o`4JjnID5pV;^Fyhx13Wp1&>q%L9px9B>V>SlhZ&$7<94ytN5nO=>jyQPNw7M^oTy#+L*3g_01=ZIe_Z{ReM3S2u8UZC z4sUql9MAx6A>RRwmbPyrZ`$d2kBsCbD|yN4DzAO$t0au@*c5_rLmu)Vh!g-wzBUv@ zf(2oW_PVM`(WDI*&Y>`x{5QY{86E?;T##YFPojEy1ACQ)ro-I84V{baXNN=m36nenLec$3E#He`t(*gB;{EjQ`1vhqt{r zMx<7?z3+|heCyjMtfr5v9m23O2qysv&_D_fE&(0FIU%A*xNxqNJYh3&oCbP01Kc22 zo)CMn3maYn7XA(^(HYMqx3ivE6m1QcHbm4GG>r%?Z*3o%+hZySqv1%Sa66=X$YD2# zjmwcs_c=tGe{899?}k*Lqxf3kGV;Ek4)v%@{ptOl&%a4UiG)Xh3%7U%G%P}dY})#6 z&w%G_9-gF(Hz@**$Uy`WK+KHmWaFcf$FUos?BAs79rX}|5rRO3zzcrxMCgGjW9U&5EHM#(zm)HB+8%cKhhRQ?{>|L-$g=sjU%YvY zc!Rj&f9Z~}tjkV`XD!^LVHflvJyuF0B@G}!N#3y}qZWU-CxH`4ffdLnx)&ctvM+h? z5I4j+g0M~uV^8rVcLp&j@9;#^v>$%Oca3mPP>^OTL3bwTac*!BrjTI7aSwUW13AzG zKL~_D*n_Yk6N#sO8+HzT^mx|xa@aO`Dsf^re|9Wn@(!7Ye=){g@Bi>y8-fSsrV`Ge zHZ_F@$iYYifn5psAs=K9hg3%o*mT?D6Zen_Ij}o0&cvpf8p64#sc`wP=gCXp67lg&bjo5CIKi=pAq~Nz$=|k(X_h z7h6974YTz@Q)quKMkBN$N7>>S-e5Mh5-Y*dg*zb)phu0dVo`!)59$C_oK{@e_zsiQ zhD>)HB*u(7(GQK_MGA+9eQ1yOh>!Wme-nQQAEF=}5>N)Wzz`GvQxJJ@Pl9j`_wWpj zXdsUWWP*?bZg30zB?>(t0R|8RcL6*1U27gWD0vYWd+DK6#86xY;f$Dfe}5(6 z48(8}NL2z#8I=|>8RJo0+(StYfO9M14K4K?F0nnocpXdAf9>{MKY%{4FcJ*}QV1~@ z5EK$hxr-gq50v0m*^~$MsE>aMn1LCX{OBI?U<^Z~Lph+BLGTKYQV*s;eGH%hNdN^w zAWa1Khz!F8h?oOL00l`9O&_Rif3iVN1tEu>69GMNmly#JQ2+3QL_j+_1OtT^1cOBo zWgtxv00XXhk!0jDM!}O0Q9T^QmeK)e4+VKdd5kMDV-UoaBvugW&{D8xg(CKxe<45D z**<4@KKO&3=`mFt17qp&7bMmfYjPdl>3XUs4bs7u&Y?;_b`VjM7|$u6e=E@sjrEUx zDVY7~pZ{r(gy|mVAPQ}vIs^~_$!0_QWB>=s06j1~5ddf3kOvnDIs`yOA&>{q&>OPx zLx*P&f*=6|zyNlo62>s1phH9lN&puie9xeYs>1*gc!3AVJYCztx zX={n4kF;XALR4Y8bcR(16?Xxd;Fka@r*lfD_9mc(X$*NFctOAgi**&55Cn>;n66_A z4pu`6@k~L`gMtbLlyDA(Wh8+IJoWGlc>s82uoBfs;=t_ zUUyoUiIOM^Ln$nIe-2!+DTO5{Kvt69z$ph2uLr>>CI5(*@Jg_W;;#zv4E*XT+2Ji2 zu@5$Z6MO+IYgiD#5;w)NtHjc<=?EDEqAewnD;le@jv*`7sFde`jy@5x9UF~SiL%T> zu^+-0CMy!OwJjs7X1GCckEN6B>a#x!w7TT3Ibw1hQHV70e+|Z@F-yy|Oq&VliV;Uk z8a6u<%Jnr@OEr9k7u!`+KZCVb3kzB+7*xw$M#&TTDMk#C13}BSZR@rsG_MQM5@>a{UPnAakSf z>%adCz*I56r@>2r38}Y{NHZ9%9>oi zCV0vZp&*FNIGcP9P>_;JlxO35wD${ey&)=f2FeSA$|Zx!UL3W%%*#>R8^63LzO2l- z?927|8z&);B1+icH$ zH4LKQ(CoFZzShv>EYJ5f6ZH(w4t`>p( zrf;Uv)NIo?Exp!^7tqjHMD$h+_Qw%1e+>FUa6e=Ow{RD~0R<_L17-G+is?-!oDi7+ z1wOQ*{6Yye{SI>&$4OnFIe@7Mq0fm7uu$MlAs}l2BB(Hs1X|q@4U+`(NN^Eg1ft;9 z@30^+FePqqnICu;utpm{6atMPsze~i1_5hU{X@EmkTdZNg3tqRM{tP9bq;Y4e{E-n z3wfCz`LqRL3|t^hL_|!Ynil#|26DYGApd+e?PMUOD9ktw+p!I_Iz1Jd8CVb(*f8|T zB|!;q2XV6Y)0tB*{FFTXRNM?u!~|gsX`MX9y)RI35V2+ed7#mK7s+Wh){N6^X&pn& ztpSsa5c!#Kzn#{eyd?5q3K2IOf4^-2zjGDO5R1)C8?=#jC9P%TElrnL5~0T5{atAU zA)*N;-)|MdlUSh_kOTns+kz#0#(Ync1mvhCp?p1vL4 z9&V6em(xYSw%c69+-@KWEkqkB08R^PWw>b|P;g~eR!+(dLmE&9lpqR%f8bWNrfgvR zan=1F{DR%MvCnDN2v=4HZpGMF7BLn7Jc;sHaBkq^721%8x-hT6-3$-{Wgx0iG{_YC za4>)`HQJD$_zvG72o=|;qRMce z#$W{d0s|`@avo6!vO835FFm;uln^?dH)b<-(U(^T@RwLm$Uv3!|>=%vTXlc!~A^J@MC|0(KF(GL~+k+;wr4M!WFykKGMe4z-LtJs*mSECnT3TM2)cNULg1Os<5 z>_&bi$1c`NPSya&f9%dKk{r`sc(z~{`KcD;T?QM!B!So9M8;O+yrci>$RUiq_*%y!Z zd5@17{}ic68#z!|-@s4bEIYrKO`=dt#q{Hy1LHq`4lCdCeiu_^>kJ7qzzwUak{o8MWeD4(Z z(571_ck&R~e}pg7=ITP}|8@X9_mF?a=pIJte*td?1DRjs#eVaizw-_e`T)_W-aCQ? z4IV^ju--5zAv9co2r+^IH{T4(dli8}igO9yoy%iDf*XWH5{&Fe@LVGU3?gVq0Y%WO zb15&%!N<@*M1!=YoMG9^c8!CUH`bO@h#1FktY?Ae5F7Lcri6 z=+-@XPTzg&!5~4Sj%MAyh5s9u`lkgYf3UH#k`v!6gEa`%r%)s$5cKq$o0bfO3Y8E- z(7l7_G?+Fu5TUUYpADRZNHhTqgMuO?$YYUUy+$r`R91K^hXELroqkJVK!d=&{V1iF zXuv>33BM~|%orhuaD*U59=#akq#LTwxM{bl7fN`V0be+ZB- zx0BH8Nhzj~JE)!>q~h$G9I(=gAbD=-0Y2PP$iW!1pqY>XQ7j{|L=#U$F+~+uWU)mT zUxYD68E2&NMCWR>3p1CJ;H(}lJ_}8r(L{nmluV99G80PwGV!pr*Di<+p>N7a;H0pY z6vd4YcI)l8;0hwnI0lDPj=38Xe}Yar>hhB386&E^pbSBf%tHc-pg|6!2qNI3t@1)L zszH-bGU%B+pvXak1a8|+l!4C4(hW%`rBu>9JcK3$0bwYk5hSS`2`Qx_DC0ha98BPX zsSJ{k!h?o6G>4@a?QqP3o_URdTX5vHS6_bxHdtYYCAL^&k41K|Gm{;Tf654;;0&tY zf}Bz$_W0|syeFeP$QVr{$OD@FL`i|UE`cL2Oz~{3=M@;tw6&l#r;`n@dngG(IzejE zPZT@dfCjuhN%F`%3Cd{gK|%|X=bmS7p~Q_GK6`ec(f?AwIAe`7hGE1^eoKLpcl5i5 zrBDq@YAE(Rf+_)UIm^^le}&q7rI9kC5|AyAj3MfRVVDIvXrYHDx@e=1MmlMw?OK*u z%o1QAhl9R(4Ya4uAuR$J>b+%u4{vK%mhIbOSzK78$1Uw1qXYMM~CWw;X zpm=7Wex8YLT{NRpf3+W;rsiKB{3kfO{LKKE8;&tvbEm_VA#oSU1W_W8iF)7!Gt-Nl zMJkw}&q(bfrpSXJL^whcj_?`pkRQ4R(lAkMX?5%y)$0(_k22UP1)}g@JH;j>q zP9?-O=9|q0up)@{8RJLqGYHS5Vv%RKVS4&24_6ofgIhr38P9m&JKhtuVT=zNzLQJz zG@vHaor57-G#tai<3N^G&Sf_oA0-r3l-GSjg#UfKTIJM2gvv|~Tc;!l=@O8FOf=FI z8RDdSRN9;n)VL%sO;OJcl<b)_6M~3}e>@{;p-Bd^Am;!l^~z8~sBtnS za$95yIt9;(s0w^?d4?L9Tk%)u~T~Q8ka%Cc;%oJ13FR&I*(#>nO=(YGO#melC$}H{20j-c-(2n?J-^CTr2xy5DZ*+|3wcsE z@fnIo3G$2_6!xkvfdW$FA`d&pRXe6pqw7S~9a`p5i?@-FFDq0J#cghNyIbD&*0%(a>QujR&8%is5N^kE)@cB`3@rx z*{V~>m763O0}7@QAVI#d&7nLu1^c=iT>t7Ji0neQ5wFmI2DDd@!8&ABZUInMF8~E& ze;A=CKdVYmwOid0-41vc(ClwzJYyQy*v2<5R&bN`ja%Ge7#+c#Sw(Y;SNOPW5e~9j zYXtR*yvJdCn!56Wh#N0zf}mLO+2!f+{zF^b&e;apiELWW&i>hXTUXoK*T#0Xm1b*gfA6KI zRRVXo#chB914eGVV`~qRJCPfj+C}v+#Z@GrfgIFEEg?8zxAnbme)rqo%;NUHs$$G#I&+>$8g@Kn_IrD|m;10E}8x0*HWOfp^^F9|w8Jm1Sp)zF z>z8xfq7iQ30u++Kg+}~1Sl@WXe{o|Dg)&fr&?N_Y(1kv9p$DAk4BJW4b%Gd+J;NBr zsPnMk?8m1EjO0hhde*hRbsQU=>nB?JRR8yNvX|ZLXK$+4(Y|)Jx83b;hx=dD9{0GJ zneKJL-QD%RcfQAZ?tFJTnfo3WX#YGN+Ji@Y;uX)-znkS8k5AU)SKb=PfB#DIllRCo zpg@F)5>4`W!Teypama6s{_rJ|hbg$w1D!42^{4K#rBx*d75zajjX2uJ}?tDuZf!VdHcGc*7+v@>~t0+-=}7R16vbi_xD3@tPW zgXuzI`N9PIkTGP$f6dB-GfYFom_aslLxX5Q@zB8;p+h^28$2Av)_A&na|}SFA3^L3 zdFaGK{6vH(nNeFpNc6>D1V+LbK_U5uM&iP_DT`t}hdxV2AbSe#!#g;WJZMx1WyGDR z0K>Q`z3o8@17JbBz=?hkh5tq3f^Q%TYRtQHC`P+$kh6e1e_IJfK8cpGvk>=i8Anqx zbEM!KjRDBuDO&Fu8@Ek^orrh+!~- zB;cDyz)2X;De5VJ)EysDS#li&zQ#Vyh4ks8e){i;Nd&padhxjhIYKT@(db@kt}ltAg0d zuKdcdlsU5Wh}sIC%w);vlulo4yna9~&jPLW;4m&|9kp^92}ps!(w!dojf`Wf?ie&$ z+@4<2e+aMw&s(S$)NzByVw}jzM$t$D`b@0^$V$052jI!60DUPpI30XBt=lMy6d0)C z;7-eOnF|0+x}=I4nil{1n2ngc)XOS2fP%CFHzRlkhgm}jn7H9c0@!*Tv4O4R$~dSo zg1NgU0ToaQpuUAH5dQ(`sOpg=mw+v>gq>l?e^1XsEiTB#y1-3#nokFXEg~J04t3C| zagP729T^?acu9)SfC1Z_&MoEA4{SV2P${)6gGCxp4od-#o7nZ#COjxsRSatVRGdO$Y<3SM}`DfhAbPYrLyCpzPC4Zi=PhAk>+7hSD1bH_(U(P^Rcv3iC1sF`^Qz zuv7J9KqHumX%nM@k^sKho?%#y2|+Pyehs=`jE!e@L{a z+gO80);ur;k~Ibja~Y9gpEvrVLxP53uo>}OLwWp$H!`A&0u)trvC~QdCK0m`svmo# zQH6Mug;7z{cs^)&g^04&qO7_%qMyJMyrfaD$hULotFnL2!v!wU9U64ldZ( zUAl!lXaxN;(*OA>*LQ#hL4blre@L{^yM+#A4)|1wygjy8aN1iCAy~{mp7mL|jWO0* zju4QqtsHM_sN1)?}voT#9+c_F9^qW$=WHxZGxNDB&lhLorW%JSH`E4-26k02-} z|4=3GSs{6NqihURsQK9KIR|rURFvehAPUKHJjXF9UjBH8Ja8E|XbX9Wf64@4j}^)? zXh@tHT#s|eCkDs^iyD$Z;W7ZdA}{NQb6{VpItgB_hq6sk-5`kyQY3f*$AVl6gsdQi zMA#lM2LHJSW4MY45C!*|98uU4zUV**cB}Oy@vy&jf8pi2?dTAFI5+Fu%8TK0)L0F`mRj6B>MW@J*IP0-tfR!>< zv-7>8d+3LISfPW6$FP*tN_@jagsHdz9*d2e4yHi>7GPlQ8A5B9e|KpPA0C++s>2_| ziad}FGM*rdvc(OKn`fvZ2`I<>s7Q-s+<3X&r;Xz>$;xkl(*z*g{5VIJVdKU1CQ@PUdOBiE1$l24S-U z@gK3}RsUOh7d5Uge>N`CcbMQq2?EdQDc5O>D?N!EkP}h(F|Y|m9=#+5q^k2!GT;dT z#`52>1(QV%8=?qLzFdw-#tZAT6E0XZAk4$6cw$c8Lk>QR=s1aGyRZ6sQeV1Bkqe>_GD$prGeIG52im$=|} zc!qMxC9dI_I#oN2K;EnRX|LP+Xz}#9Q=a$Ch5FkF9Hto|U zI-4d+mB?wwH8#dJf{o&WsMus?W{y&eqiBAfu&X5xRzKJ_INq_E%n=DL;F?S9=0u(v ztS-aQHly>sfE0-Bjq+7WVxP)Q0T#U~2sVg%ehF?>e>F1-iD%j8Juw1*zUzR_>vs?v z22dY^M&Q4m&%kCGRxF5Nwk?VFV!1Z=f|mr0Ay$Ac_5` z6aNEvf9hNw6KtevXYuT+cV+fKCznOB$|1}I05 zXas}G(C`4{i%9ReOoH`p@AoE?gRai`X6X86NUFf^O+M`X{@1f=gqu_&7~Tshjcis~ zoI1^f+szNXQScxd1|*jiY%*@pX6X#ye!|niS0r{`J2{Hefrdm> z^PyG~sz|RYW|c+=0=yL2Q{kZ=h!*4>?k*N(qZZ@jzDI>17jIPXH7<07=&Cw|hI$#U zBCjP=!Vr%o9|;XqEs{$xX*CGF-#xMJPuB6_7?c5aZ@w0DNlwLvKJ)uF?6^T}E}IsPef3iIOiT81@A9sp+=-7Q?^m@1VS^GOm_>2(1?d@qH zNv{*o$P*}ciQAek0fB;nymU_w+?IlYhhO$)u@cTOhO`Qj@ReRaAr3_WL-~9gKe-1x zy56j=<)ZqRhqNWyUiLt`kn{<58bnCce}U7YQ0u2q+5b&;#9^5<$H%9jU~6IIXoqjV zE+9(Q+KaGnh|X`=+4k~?8;ZT!y9l(6u-E%`gOgae^-N<&4W#J7(sw^YTLvJJD3yCR z`?IICzk`OV`3}b&q?4(K6Rks7bR&1VE$f^f4GTz0$kAni28W0E2MT$$fC2KTe_sDB z(w=)5N2zFwZwL@>CI}4%>b9T_oK^RMD>egQC235Yu1bI$NX@Dsjm0Vmd8T2N;QPI9 za(M)#-|Ygc+TpTwuwLl3Gv z$9;t#^OJA5si?P$sb33=ct?ArfBniI`HwJtKjAF2A>@47eBx)zDU4~fSO4{AJV|C% zSbn-LFVm+0(<<>7>v;9*{cvL6NI32f6pS(|Dr~r zrAPoYk0?r+j5RcNfkA>CqCOf?xssyE90_1F8urYX8<7#XGQ7uSB^R3S&WRG4zzsB_ z@?MF&@Xe@6yk{%&eJAQ5kz2-~NH8eDjoV>~L)2wh?^X^2Fcb#P+&5{06oN82ja(B3 z0||}XIxMp>6X18xOo@uEf11IC*rJ}Ey9Y8tlBYb092BK_IKy*LL^6OPNDw83=NhRz zYwQ^%jZ}jUb(n%dkFd~&agmTf0+;L-ktz+r)G5@W{oIJp(V95@`u6YR&#!+!|Nj2} z0~nxy0}@!Efd?X(pn?lB*r0H^8_*Yla{fL|d?MaqW1Z_2RP8rBGV%IZJY$S?|JDRc~T1MIA)r}oC_Lyk(fMl74 z`pH9tRzF@~ibXJte|MUTszp%ZJN=Y#L6!&o$Uq+7^aIjHS<1zQX#Gi5Bm*gAa*s`f z&iNB|3_(L0rXm@T!k!H&$t0jp7WGXK23#NrY83L?tFONT8?3Oy5?idX$0D1ovdc2d z(5v`8V?+c7Bw#=UL~vnkNe!Jjtx^Wa)CjeTGIR@Y23X65e<)MAO4K)b|Adf2L~mMz zl2Z4QB!~nA}tmPLDL9nf9P8K%AmM5VjeVZP^;!ef?NPKzEJb~O+C*vYO+#JZtCA*6}Ku%cE)5C za8NaBfaF_wX!JFCDHA0z#ZX_o^rDo60P{0~H=DTPi!_eNb`yD_R8 zv*t5FtaM9Y-(*kRGnpJ8ef`C;Ki@gRx9>mJ!=CEqv%cjK1$5Md#`*~8!0^3=fv?B} z=|q9Re*uAxebjqR1>5(X3o>MU6Kn(tf2TW(43L1j6Ive3Q@IzyForUmAq{J2!yDpo zhL^j~Le7D@n!KYPGqXq!&ry*U36UuE=tn>5ArE;#Ob`Ys1NCooEvOKv9TEEQ&m|*qSvGag1aXeJ!2^1t^%wr-G20a*tdfB+Aze^>%~AhpF2GY2xADNX-tN(5$-v_2={LQQ*0 z(;mo=Sv=yxHGSGsD7@m4IbAAKo9fi3LN%)CH0gvUS_UHI00t4s0T+g$RA&9}7GfPM zS<4DMsq!VPW?idTqlql~!H%tC9R}>&iq^a0HLrTzD_{F6AgWI29`&H#D@0)me`5&t zaiWwYW50^m#WpsQFpX?u5$e~=Vm7my-7IIPYS{@rQH`JNENM$?+S8&owW?ih3<1m9 z*TOcovYjn$YirwPx%RfY-7Rl>>)YP~ceS|2UND0j?BEBpQo#_OFoi2@;R_2(!WiB#hdb=y z58Fq>ARaM^OKf5Wi}=JUUNMVXtlkv6IL0!bF^!Q6;~L{Q$2#6IYH!Tre;)%m$U<(a zkB4mJBO^JZ085h`ObRYGoSIB=RN~E(1ITCp9gK|LnGR`g-$f18|~=Z zF8a}uo;0ON8|g}8I@6k#f9j<-?dea0`kkE)HK|K&>Inaj_SC9gHLKY<>Q=)#*0MI% zt7k1BP}(}zx(-aOcfB8*^bV)K9(JyKZLDuR;~8rr&Nsd{xyV8Y*~+%h5}DY z1#=;>yUl82Lx`8xhSM|L5CUx8M-({-fgtRfIG&gy-tzX`xDzCge}{ws1}_~>J)Btv zC~!fSq@i0h_BO*S4ekelrwseH3)~1G0YTYE9*x+SrWQ`Deh5OV@)~&x za&S|R=VzMMD(G-t(Sr~qCIS~oL02^@&Swas40%}ggL1H;MxeamLT~iL8}yvg>>N&q z!PUfxqj8QyBj~|;e~$#vrBaom>no<o=|8e|m_jCV~}WKQcB+BHn}B zOI*kjojA&j=UyI^u$CL-A=t$By+(iM|B!vb9e^L2i5&89AX5^k^s9!&D8>&o2ZG`_`>uF%3Nm%QM!r|P{ax}tKG{i8x z&ZbyTyotv4SkV9U4(9}wei*~+m;$0C!Rxrr@YICgc?1P5hbVjl;%rY@O`Fo;j|iI3 zupnO7Btgb3L<&%eAhZVkl+SW}#xQW*FuWk=wBU}|e~ScoP8ABmRbBm)Ji}Mn#1V4ND+m(tyuvFy!#fzm3yz>@SkEhL;RxQL_}qdi z41*wOL~V?fDHP!u4I<~jAkSrvDFopqa$)Uc0+krS-ECkg3Ymf(Tn^;QEg%L%fr}d) z5jR=Ie-N2r2!z1a)x;~PqH+X5wNMKT2-AJk{{y_FKnNg5XUI#wJO-2uBKnld&O3a+R2QedUFO-l~!2k!7? zU*1k2-Um50p^E^IKybf^f>t&{MSxt@m4>zflfby-8KR-KL0-S$V^n&KTJ8+Q*u-|| zg#Vz)R}_aHeT&RFO0}c_U^c|Y9VK3F1kWHIQhx@4QI(6CW@EZR6*-eigvUmJr2!B@--JX0Py*Kh<#p;ADrQ37tV9arfgm78 z3haeEkcLwp%4$@^MEQh4ct%7bM>^uiEnuRbL_i3T5a|FWF=@r;nSvW=z#av`+;K}D ze1E6U$z{~J3viT%8@vuBoMUw6$2SzEHyINVod2FtEMZl=&MWk1PNaasElMN6oM)KA zEr_UDWWvngNm4|B1_*-d2m%ytOHJ5|VsL>e;0;y8j1!67N(AGgxB*qb0ALV_LFB;i z7(q@L!4MVZMkoyLP)wl^MTV+`1bBuZB!2;U+-8<)02EL{VJ(S4^Z;N|2j}o4(%oks zU}0LE2pZ;rDR50ryderv!cUY&5G;q!K|umdMFe0Rb^WV^< z$b2Y?kPeAl5P>n6k!p}CmrUoi4#i-(fl7qHd{C)<*4`5h0}$3kBNX1uEJXXo4-p|C zsdB0~zC*Z@swrsZGyG?&hDn_&hJSE8(LLP4k3z##gaC*J>cVEH$jWeJw#k!4PR&+`)h;n8$J&#EVGyuHY!G|Mg<`@!amZ)LJaG8l)>SM(HDB3WMl=|-i1g= zz@^qiN>oL)R>hutLm7Ze87zb#Acq@J!sc)eynLNSkYf)ZsQ*o)^lnZLpzZgmEvd5Y zFyy8NfNJw-gn1SRz~;b(o`ciwZRq;#{5l!n(heSxr=k`v#-?Yp5`RJc31VWj%i}^v zKk6b}KFVC+Og$I_ z5!i@WE}kGTlM{X+*wsT;42?WEZ3O)2w0;GVQVA3Q@H^zeM&v;AhUKbIMFh;~qFkQ_ zXT~|e3s^o112kPdgnw_b8pS=Fhyf(5;1J5T{cFe$$1@---_q|GkD2{Guw4-Dc_vE# z7S8`>4C5B<#}+W(F(cDe?szaT8>KAgo(GzA3|v}8*%F7f5pp5-lIbR~;4}i2Xh2ed zTxn8^Erx)jIHpDDV#i3Z1fZR}zVP?SFk@&FXOask1?kkj1Aks501w+p&YZ}uCb%GF#y9cix99EOD+Q6@yY7317p$sTvgmrOQk;ZaEM9c!E-#@6VBQ) z6BY&IlIkn;gMTsfBB^!`_NeCUhRHBA0#8+)Qk(^XqMs*^2PpSwHtz`W4k^{{FaVFt zN(AvcIB&$l&I8farYxZ{PizpG(nb)0D0K8KB*FLYYA|DHMiBGT81teoX9QH|aD2m5 z5L7htw3h*H@%f+kaxtoeYc?aU8%KrS$}u<}upRHJ<$sd1UF`80?}+VaP7rvW#Nx*{ zh-wNXZX?x0>IrRC4-!p0YN60XUGr{3*B&%PrF_5& zW#EcqpR!2ot-Ff%D`qN{Tm~L8wlWK{JeX`ATYn%+O9ttY>avw2uHg4O3_~Q^4Kg>X zW}HJ=L@ZUKc7vz3!^*>U3jd5b)dLYAcJrA+%)vHva`S8xu(c4u9YG&4<#rJ5_O4tJ zE_DqR9aOzgAQCZ(wGcOcz(pq5_>JTEjpL$ykRufI67|stm4tvk!?b*;=r91&b!!YX zsDGbi)PrYtcPHC5c)vv^RH-SX_>usHzEU(*ltid*cWo>L92@b#N#Os?H+r*?SFnUz zKujJSL=?Y5Qe1$fj`%b1bp+^Ti}XWt?r0Z7v}SB=RS2kq8#rIL!#6a7bbsDRYXIodkRU1zk(|cQdWC{3ind z<6A(YQjm9-cSa)wp(*?fWHiKabVeSu2mOF27o@o$-$P`7g>Z~b;>EcV{mXFQLVvP> zkUSi=?bgM+K;AL~`Y7YFD9i7m6Fi9}di1SJdXzyKq`*T(t2T4EY*YHL83SirfDter z7d#}VPn>;^$ER--7l{0HE}qD9!JHa{X+nV;xB(YvM60y3JlX067!yiTOB)KTYhPu+ zG5k8u1%lRtAPf{&vAnp;r9VD<5P#iuaY#|Mz`(-C-2$VAUr+_jxxv^ihI#ZTxDWx$ zb3qSujRMv)eLr@}e*cOaHBP$EwM)VPu3MuuflgiJx?mb`_~@`T7?Ort!IJu=ms_)28WnlSRn{MIN) z3V}b3yz1AHXi=j_i`Jvm!{$LQncm5qSnm~)pCM8-su)7hw~5BwSbw}40-{uj&;*5m zF-&7UHz{&ZGAl6;jH_`?YJ3$T0hByBex`7)gH!7 z!AQZDqr7v1EQSn0N#WFIT*eX^mRI9BQwLv_MtUzv1Os`H=GMKNcW>Xnfd>~poOp5L z$B`#jzMOe;=g*->mw!H;dUfm9v1ivF+!$-%^7{3YOtc@gVaA}{t5+{PYw+PYh96q| zz5IUO@4??F-~W4k`~UBmYA9%oDexTewoB?i`^Jl^9`lTOX1#a9lPtV*o++@q`8Hzj zzWpkUubvFit0%yX_DX<@Z{&IAyaS6G%9!(>Iq<^{Lp(~x^MAg%?>(bDly5wF3_EWa zXm0GRJ>HB7!GIiIX)wVlsjSkHGJ!`SjWkjT8sRX}OP7dmNPvh$+;Pb*_nl>#JxASj z*=6@rP|Y0_nnoH)DBgSV%~#)j`R&(V;?Vt9VByvs*ni-I5l&d)g&96G;D#ZNSmKE( zuGr#>8-5t$jXCbv!G?9@*rRQBGOqm06a!WPyD>)Si})vd89|$(;GRm}&Nn zXW(=O*XEvyE*j96`#r}~=nCddX|ISo#~W#+f!9=<&yfZiXsYh>9B04|+Z(I3JcsM9 zse77i>VG_4``@Rb9?op4-DXZ|X}qx#XQZ;$x^ANZ4;)LR5AElkpPl=R3oaO2uEj>U z$RnfNzW>RC3y6r51`jQ?faUK??U4qP%{kWs6RYjnC+IEDfu(fLf50SfFYSS4b3BMi zTNEhDFh$WaxM(D9gIg3N^2vcF=5|;n(`D?#*}GqVJiPIgAwhZgqt17gVVdqp&2biy~Nb5KG87ZAjCzR?3&y`nkp z%K-uZLNGt1JR=7Th`{}lSAq#lz<;O0-U~~R#uDHGUfuv=2{1v#DYfngbrhomw)dqo zI-ms1o6a}fV1Ntc;e^fU6~gKWDOgxkfooDRujx;r{YSJ4^_$k8BI%ARU=0zRW3`=1Sxu8`(lwL{gcQoTdXVnSUKR z3UiG6(*gsF0!nm>vXu3-r*TdRD9FdWOfdDux~O_i@F>qMUkQo<5UuzzC;c*fH+28I^dn3 z1mZj^dQYA1lq31{+NZuDhy;8PsDJO;$kRwfP`4?KM9)y_MgS_%1A3~Uy-`}bjAFI| z)##Q9Z6Psu*-(y1ZEWw5nlp&u3}}=_U1Nin)Ql;SwyukMX^m@e+6uO`x^=ATat1l^ zU{<&uaBRqw-D*6E^WZ5VRxJ8l8alKKXcC?!fUp+Q>E7DZjqAVyj_ zI}w&%a!Y~j7@}fEshH&9AMhMPfh+qU62Z0OEAPO|fAc0s# zAP1%(0xlH_R))H5oIhY=4cocKbJzobCHR5oCSngf)G>|{`D+QZK!1#whBBu_5#1)+ z+27o4L<@FIavgu5~Rm6?t~{%+oK4-0vOONAuCq(E;R8L!PZ%Kkq!=smE&$A7RJ40ZC<;c;IDy zfl4ie5|n?utYmK%9Oqo*qb`G`o}s5o^M2;@3;OYezJHN-Ptx6izPDJ{?EEm}AtIFo zyTaE@cdX+Yn2Sh>nz)F~)Pa}>_=_Y`4ENO@e4FzT6>BcDP-eKz6NecpqowOjLe&eFB z3=c3N=}I8Zqzra=3Cnn>G)O1SWQufjFwT%E$?#yx_6&JkD9q5Q%-F8Wh~n+WDyGn> z3SX!NeefNmVWy01%Tnv^lx~d}ZJ3~JBQ7X|`hN~O0LcInj}r2L(h}eT4p6QH!2m2R z{~Dn-9)$>6 z5PD$8%D^j54^||s@n)s{bgT!40Z+1Jkf0#;(jr^@NggPH<;p-1Acrq3%@j@q$7Ia_ zdVj!pVl4qyA}_w7ix|KFpnwvfK*u_5HJT*|La-Y>kO&+w`(B9oG~$dhD0Yx+jOOqi zp2^LsaFg(;`kYS(x2}vJ$sG+16V9+Vx^K+@O#C>4{Jik|2-0ZM?@Y|5wuEYx2&%S@ z>l`wH3tqvv+Di~FzyJ(zKAI&~k`1V!;eQn zV6Aq7#1g6>+yDk}GADI1CsP0_jtUll(ttW*1>K1lsKyyE0TUP~2LGGR4PnR!aZrPx zL9*cS2QV);vS)!nOW=4(?<8VssPeF=25sH}EMri~#&WGB;+cf$oyNu)kgSh_CVvYL z4Hm|7bo|Kepy|&hV(Ms$7}Vv?gsB9^>>SdO`lLqBen4kvFqG6q&-|++paCe?(eDDu zi;SQ~zTp*`k(K1i6rf-t-(@G9WY;wE7a2e`JQ5^Xh*w0e$3Ep2fXm7NX)c3}7vw!3~D02M7uVh)oo1F#@3A9fkqi z7=ScP!AUT}=Xx^}&;zJmAp~Nj45R`XdC?e@h7&OoS6nk&PE2Y4@}LRRX@%@jo; z_Ip(HO*V3s>Lxrsl7D>}K z0%Ko+d`uy{kP}tq%J3K<1YE8{d}R`0U{U%Z6J9a}DhuKyzyOj!LOdcbZ1N(F02)~1 z9LhixktA8+wV%$R9-cK5@FW4IwI7~=Iuq(us1b90wI9CW7BsduFY;T@BI5Q$HtvB7 zO!fB;N>)f?B7gdUR>SETa298E_IhG!jg~ekm-cL(ZormI`Q!_=h;}0G710>9P3Q6i zpHiV5He$uMz$O+Uanv(A!X!*#1bspvQh?H&h(km{*8ftV2g;xkVk1`oiqfW0W?wET zjMQb{0oN$SQi#>7VITva?0yycVi!1C(YC-jcO2T z?M^bMw)wX1gF*O0+m@HEH#xjkBBrn@B?1qGX)zI~oyu=}HOT4uv>fkj$f_g{CE%kD zh(S6yt$SC?D6jkJ?L_xPmGJTZw z9sbW^MSnK}B!G0Km0B}!b+OfT&!r5M7%TKdMe+g}BOnuWmv>L~KDr8w?m-mNC!7+P zffac|dZXZE3WaCNfB3X5<&+N3DV_S#l9=b+?vPLaYp{|VIef==lREDMO(6;n} zb6CrSNp1UdF4s8)vm+0ks6UK&D9*u$cDHv0 z8KJ<)eZXab21sa5tuXHIC7>u5UVkwec4JgmJIwyAf;*LH5)J`c&L5ol8;)G zQ%9*2ZJO~%l%+bsy@#sB1eM7~9)AWg0tDATi~u={c_C2O6a)bZazMvQNHzc}2%f3=yQt~UbLB2mQ-m@RBlGn&FC%ozN1JDZ{jyi#D;bIX52Y3D3cqoXJw3 zmLm+SvNwXVy*c_U$r*?|%AM)#(vCUOSsK7htfZ`lw{+?VvwIsZZ%d$gF@sZ1AAL|KXSu6{84g1HX7QvZ8Zq^qKlIP5zqTG$v9c1JLISv zq~92J-C(<0H*sTah;1Z1Nc_C{SiRq2cmKVpy$881fRx=RW5#E^UY+`;AWF)kTxTZP z4FB7xH3F$O`Ci*lBGkdafqw?UhXd#oyw@j&!NcS;m^HHksm{%cfRETCJmeV&@JZ+| ztxG~4La}b&dN)*DBdC)B_+%m&5y)&Z2Vel)`LjQ5yajPQnn9&Ocl@xk&&SQwFrmpC z7;|iod?S*)$u;8Msg~CDF*mZX1egPZXsS^SwFZR}oXM6Vpm%HMnSZeH=ue5RGOg+s zxLO#^{Nn;C0+PTihSf>p7voC&_ckIGjeSQw1RB72GxDO((et=}V|UG)jyu989~TNh z1>-mux)c4QBfz>tI=dfTjw8JhEYDK^zSuS}J-z?9(`(v`Fim(Z@?9}*+;@@$eHxvh z+TCL62aaOC0vw$-S${C$XmvunO`BRbp88=QD-*g}5KP_I;XYx29ZXW`^N8XwN<0rd zPAOipi4W);yrUls+1a5nt*NLUjKKm^;8Ta=+9hI(Z6k7J0$z8FE#x5++#vHcfAclJ z404G#RxmQJXKcOFh0NZti?GLW%Blq3PZ?AD>OJ2}xF|@!l7IWVE&nyj)PXF;ahR9` zsfj^Oy&*2?iG%TUW*Gb7F_g3e)eXTuXfSAu&UrTug8@3c}IgYTk_x=kQE?O7)bI?-m3@^ z%ABhQCBlVF{eQ|N(9o#gH#ZC-414!XAQWi*8fENJ@14nd#!@6mz(t-(Xc8l2I%o3T zJuVr!^m_*lCj%(oT)O14GQ==>=R}op!?dZ>rp9VYojTPXrVU*8?BSV^ zb*of+(!Rdich$iUuqKIN{Q=9KJz_r`O!&doq*$z4`xjCjOx5l_Eo8MwvrD)!Re2o= z7$5+{v6H~88h4tmGje!U`o|!y1ECd4v67uYP_svuzW8eS4 ze#>*gK=9b#dK`acvLB@owsXMke6IYvTRjXIEpNUaB@GEKIOcz8$^hN{u$o*0>IjWA(9kC!S$H_} z&N(VNkX2n%B32|O^kI_MUVYi~U}6`Zqv3#Wh1Qz^$6Wx8Xcw_5K^bqplM(|4d{oal zg8T-9F@N+K1H*9=xUt+huRIr>N#6*ffdPNUc2ELPJQLeJnSd99fto=BLju;B{{q%I z#t6~~P4Wo3QV+uMl!t8*a958h0uf{oLJBeD5JVDDWD!RAb(AMajYJUNE%ZH;gaiz* z6dOzl$OP9-Iqj6xaSTxMjY*Awmw-`1`6n)ao_~4spj)uiNF_0NVB(Em8Y-CXls&df z6I*kQCl|V0!8oA1QzGer7S6c%p)}4o;~-gdA$(zIp4yf`53^Nl@x>TttntPickJ=U zAQJ{#$QR3V%LT`UkcS|qj_jTU(dHn^Fpa=WsB#7Mq+TAG++qqBHgi!Dqd9C>Q$21N zTYo5ko1K$MBaKkv26z{gK?&3|P5EhjWzY;>M)+0yWPy%7x#gJ#j7Q93nshXIGzMW92FeC#kjE{F=0NGT@AT79c&3z7%peQ-!6Y4WZfskmTN>FJ zgI1_rNg9kdR`fW>Km(u^k`jibVb;Lu=qy9pOKT2?II^x4l3I|RRDbfi!?uQHOgJoL zB|~!$QwT5uP#DdcF5m_z#8d);Fwk^X+9pakDmjP3DQ5}A1(Tk~|5#jrMr?ZbsT2ef zGeXVjeTc9TsaynQ*n+I=gbIT^$>UZL5JW+wsR2=Fj{zab#66}{Ri3ERCFiKwQU!rK z18_l}ZitpBriGJXN`GY-8dzxpV2}qQ7$5~!bxno{iMSC_1#KkckY5z44yiPhtcYZ< zR$hA<&B1b^p1Fr0h8G{3p7g*7POyR(%wUh1w!t>k&2C=wjN8p@Tjg1lRL%hnV_5hW zO&ZKObR*&kduGHEPO*eFI}A~X0>l|6pT*dN88cx*aXVrUZ+~dKVI6~pN{hp>j?G2fGlDq>jiycQX&(sLm#J^4 zpUoL*SNq#u0kvhgEn;#X7D~fu_Gi|I90jHZ+t(I%x_{+;Y-KOI+@Lo1jOpENezQi$ z+8#H&ldbL{o8%EYzdKQ?08n}TR*nuASfgoss3K)VU*irCT zf+(1RD!76y*nuP1f-uN2=kN^Auxl}RgE*LjI=F)q_kzGyaHuzAGp1tx$AgWLDrLY0 z!exX^*o02_givTiKIm(=kO`D9dwwPglz*^=nQ#=caSoovGAxq`mKTMZQ4hn=TM}>! zX4r;q_=a#8hnO*iyjBm|VE}nh8k7JtWCR7v5pi?0{}Sirg^ht?MHqv%5dlGPh;rD7 zj`)a>Sb=m1Ve$Y4FtdcBVG7fPEfRoFtpN=aBu0@^h}+WxEd__ELR4%RiLBU)u7CK7 z5IBhqR*9EL8cAR*GV=_>SB6lwEe8ZCPUlNBM-g+^hS3E3`UCy zfQg|I1PGA{7&sgEa0@vAjM*p;-!P1gP>gLj49p?{$mfjexQ^`Dj!l=1zNR1f(GT}v ziPJb3_3(%Eh>!bm55|ET|1lN$=zmiCF&O&M4@Y&7H!h_v5=Y} z5BE@z1+kBfu^%pxj~|JUQ=uQIf{^-g53Q#ktL6WZqJfe9Ff;>sge;klCn1tSwI_JQ zI~*yH2uTz9Sa^g14OGPd4djkSd6Y<*lt=`PBV`ah5CTvc1jZmu20)E?^nVV5kOEK% zm3i)IiGjwz1zccUnUDipNd$0I6Y}6#TL}eh zwiLI(H2q)-ZMl^&PzI);CV#Ks2AH{(IdCfE*gHuGM~wgjP?-WrFbuu)4!4<`m68c( zIeR6M35^MxZor#_5nac@DoXjB&>5Z58FBCk|6$P;UQ$Px8c>Uz301?BcNfqDudp0X zWgk*>AGicr%kc~_Z~+lebqOE=FraW)>7IfhWA#uo7eEB9R}XwSY=1Ctp1FlfA;625 zaSxQxp6^L@1C#=5G!?HP0hCn+L^pLmVJS#)LD{L565u_5cn(|;0t-5I(UJr_1(D<8 zAF)9RDF8E9cL7p)RQ-?#8lVA7@tsqb5ae?XWxzc-ngZLohDS4%?g^iBVxZD<96nc_ z)7hj>`lL?Elpy6EkAD)MTwqyLbdULQ3tKZWTtEbP!T?EN3?(s3InV<+fNLob1Zdg= zf>3yM_Zv9?1z#!wHJ1XW5ItswhxjQH43Pf={rR7*#|1rrrnt4Ag*vA;m_23|cRfI- z*aRvtU`Bpv3SDGz1|R`v`Y1U-rb$6y7jUKTNkwH4S}vLbbbl(C=QT1(@f5Dg6nVf+ zdB6aI*8`L}C~UHbQ!%6NsR7jy0fqV~MkfzQ!-t_t1i92K5Bd&C!zL*}nMCj?LBLZ~ zVGPU?0jmh5+PbaWs+3adQNARRx6ugD@C<+_MoZ{R>_%B2wwS{c0eLVqq;dWvhxf%pnrsd)?ry0El{AR#*&yPArin5sNL9?Qeh1F`U>e$H4a;^h}IdHVgyhi z2)FPI(6E@yqNSX95Y1v88hZ4`0Vl>^_wr=~ju=uSV3Fl}(d|B!@X!Ni!s&@fY6aK>evrXK9NBC7#J#=~#3W`F!C$??P0K?K+%4@C7vFEPMs#6~U4HH}~j z!>~@4A;tHJ#k#!9yqtkryhPEI5q{{Gw+P0xvs44B0rWW~ZVbneEC?Y$qDwViQlZC3 zlevA&uv7ua7~GP^;078X8SDd0t?QTbBOc4?$kBT=L?Fpi@(S=Fbr;aDSI9I0_P(&CWz`01Z9v3i`r9ZRFu6u(G*?L=J(4=6wH4* zm(nO);OH9>paEoD#w48tK>+^+r?_FwOpah!a}JRLM34kf00WX+%>}#13A?!q%Xr*u zk(@yk(V_vmdC05_!oa1gBBstG*?$A>Y!aqmaeX9+_ar+V=6%M8VPG1c^Gwp>Qz)iP zDHb`{~}(QqBta_xIes$jv~iC-xX#q81Fus$`l$td|E zD@_&bc^o-#3y=#yy@khEdDGX-#}7GFemodGEgHNPGtbZ;uTUKyY|i0Sh<{4$m+efm z`C$-(00rBrG{!R>K#ROWn*uq&9Y}l}bkZM<(7UD#T7)6i^a%}Vw5&EE!{sFb`$WQn zVXYaFt#iHH%-!7FSJw$v{|{l*K3{nb#$n92#Gg$w*i&JS$T53mcE>htxi~GsD7@3( zSlOWw5}XDo@W@zBf3 z-QUG5FkpE=&DfP2-!&_z7-+#EkkXGKlo#yNnelc45d=pSpdKt(z<;F>jd9xpv|HcN z)PtctFe5~b5(FiEBjVt&d6 zEDqvyUgvgBaUyj^c!PxrrEy+|q zq5vK_J(AGSvg%og5`Ver>kYUJD_sd-;~`8xyPd+2l^@?AKs0*}AV!*HU@St!YW9X? zm20A9+8Zk=~JQ65)cGe z1GBq2-D8_RrhqJ+aoG%jOfx7=Vnpgmj=tXxteKDrZe2V(q3&axvj^|Ap12va{S&-7 z3`1Ew3jXUWyIxH0Skrvg;Gy5z#{Qc%sdkaSqxs_ddTzVIH+(F7kevhHT;x zMNd3ylt&VZab1)r)Y?<|J0Cb7`I0~Ro1ybdbPGWs&VN$}1=dIadB8N!kd=1_tj29y z)$zs)*(f7A895*bFhBsV1@A^+cQt3LvJQ{K05nx6(eK(9plS%t=hnTj(>we!wfZyGgNTM&R9oU*5lNor` zUj|u+5`V_Q=#(G-@<0D}LhZsP2(bDbJ4*>a2N368HF8rhPa{#rzP)4EkfA6_qM$jX z_)U}`gN=rr8%F6BMR~>ux;bd;geEGFK}yEX zK@>_6TBc0Kb~^0Y$mOkbJq(1PbdTXVnvDegbbli#)3c>v8OnP`iEEy(-;gS$XbdIW zrDQWXrF08g3I-wIo^z|UU{{Gc&oxRAflE%Yi4`wq+}QDB$dM&ard-+bWz3m1Z|2#1hao2{3>`0}{}{4a7Rtbyr?{_4QX^gB5nmO^2=Y z7E_W{c3Eac0jo2zxDYf14KxTLg*?7-ltp=l;Q~}@t<9m4ZH+beTy)b_cU^Yd1+!Rp z$Hc83dh50K-q8Bt$dpVdG3A`&w133yn_>9%$j5jSR(N5C8+Q2Nh2e!5V%g}ua?6VQ zn|Nc6JNEcvkVEEBVv$QW`DBz+R(WNXTP}@cmt&TBW}0iZ`DUCkg*j)Qd-nNfpo12A z=)`s=|2o8TC=sNQg;IV-5J7Z2dTOexw)*N8jfM=GVH6VcnXa2&3aeqB$$u3_u7`5V!urelY$Sh1D0uAuS`ftDk7ku!_w07(qK_t*X&M=9P-~vICTMC*a zB7mW)$*Mc;o-!UUL-NGPf`I`B7VVO{e#&hqhYK`l1oY5Fx!{5vEc-@~S-Bv5ciwyV zop8hJ&SO?L!mKBUL?e$2ntvP!peo?Uyv;!nu}8ZHio!l$?3_$UPojE(GvUIBMwWsh z(bpN1r+Wr!udG|VH2i&j`s=s9=HMASXLt#U2eY0YBoB5nfreC`7n${izyNT&m}l<6 zg+%{pFLci^LSaO4HZAqT4Hsa5?sOuL(On>Yi>U`CRuzKFd?ODDV1Ga!FnB*0#!!Yc zOj!Q5$3Opl34n>p!vJ|=K+Hi#9+bck6kJ7`2Qt7gKHP~Z7!(4(VW}UTGk^zW0zw8z zV1$b?MnTGVnOi(DiZmo+8O?acO4;y(smo1ra7du)v{7|R!cKCEGsn<%M~*wm4Q{f7 zJPGs=I=gA(H-HGe#eZbS6c{>8c6I|tHXg`8gyfE+lDNbQWoIQSoK+dz^GG>rjv;OA zO@Wj&oZ?JKI|$6q1`Y5zM>+(HTGZnsS5n6VA##v}%uWO4!93Fa3ys4hW-*P)B{i-@ z|3P`+LNf(n4BYjvEtzmbGr3TuTU@Cg+$%#f%Wy%62oo~T$bSSgy_tts$mW*AsAe;n z)`KhxvLP1gW-}@A3?fV=h>YP8B{C7D1i-*6Hrdcg@=#AMWtHU%gSYT$(wc}2C*a;P>%pbXHc2Q)Sq zf{)CT3mQm)^4?*L5L}?OFwpC&G$4rKCPomm(*T2%=-Jjq_d(~o02GoS0a+QKgMm#? zBX}DE{D0c%ir;92Pu-M&(t@ucLm{1BbShQ^Cs@G?Rt#AiVvO@83RDrORTgvDkF^>= z0@q5x1rWsmK~N+Qvt@vGF;pv~lC>UgkW~aYtbqi`H(L&Q#eA~p*BG-fkdhmU|A&p5 z0vMzCfQ`9OwIo2g2%xvRype|>(8@PZeOA6@$bVHxxGP~$%{BoJ%I!l^5aSf6w;_bF zTv+5`3ex33fqV;rkaA#*hC*RhQXA!3l`FY0I;mFFJ+c8buusO|wOZyC?P$YH9vIL{ z!$4+X0;ueee&BU>5@2BpuGk0?UjSS}B2oj5abyXkl)mw zZK&qpr$I%^P!TPGC%WMlcqQEhTB_}Y7X}70`Gyf@g{`iMqOV*y;!2<)Y%!Z!=tC#E z8BzTV4kiFs#rZ}Vblg@^T#${q!7k}8$R93HIt8X6;r%Br9P^cz9O3d(|^_@ z6ENlzA79!idV50{OCB0S{A(5oM!U?kb;MUjqt!#KC;2jvRGxOd#6a*j@&Y=i?9 zhdeNQdxvpe5=)VQ2a@3Cy#OWZA)J0Z!!8VWIK=BAaW_yL);SP-Zp!de{}_xhEP8k4 z=jU;5ME9jYnu(-hp| zIh1FsEMEhl+C0Mzg~>!tzVT%SNKiR>7&fZ&^P7ca-2ifuN5Jz^AtJ{8m*bDXz#Ag( zZrd!ySf3*5x!(9b_%k8pLJWDx0Vm-CBQX-mlYnpXFc%ZB2WdDB%b^b{IDaMjqFZXX zL@_Kj5L zEW)WlH79el%L@tYktW45D3bUIJ1f7Cm=CI$EKz8`u-l1qfCf{*h&=xYL!Uys2WdJD zD+3C7hA-Q(XekU3O}D$b^1Vq7Z-r zlv)Wy6vhz1z+c?J&9fYP+(r^)D-(=8P2@yRbORxQzz<@teJVxr|3k$bBtDQR17WnB zGN1(6Qx13vLRh>-jep!oj&zt@On@01us0eDpOQxHiZUglLSw8C=otnEX~w#7I?&Rz z6o7#vo5pXbMhSodKiMHRoJ5HGr9(oXI%J?L+8~n@6c~W31)QfwXh$WA017;tc^pP4 z2(QfZq(^ic)|152t3-di#3O^EfLy^X+=)*#$SV@XEmFu+#D76mq&bKT#;7F5$r&96 zfWD60OTO&Obot1TyhkU5Egwq)wQ!?Fu#a7BVN zfDofbYow3{)IXp!ilG#uqP#>Hynl{@ivV>D!LOVrOQfo?j1mhG$g|Xmv<$^8YD-hR!CPv`;amxhNUHg` z&1}*M)r54N^;a9;)5o#kgy7Bx2wL1NNFaD{cXyZK5F}U$?k!NDMOv%{THGn_-WDkC zP>K|aK79XxXMegov%6>aoV|B<=FWY;#JhR4FCg01=eXnCyKvZJe&w)HvigPjKm>#}T2Y`fpe~EH!Kyrjq?UQ0N?0CFrZE zUBoJO&y%U~I^aX2qUj~{sdtF=Yd8_FB-$q-32se=QNFq@wV9$$ZVe1KkbjNv8p zkY4GTa~+S;5Dvp(`uCy7fah;C9cPN6Y0&}?9xfX5KP>c1bTl*`wCAl3z}tg~hWmu} zy8yqpqy&pEo0Q!nJAojBoD^Rj4*l?q?ft!Cp{jGH08XY-e07}X*96eO7>>;AZUQ7> zFh1p}e*rvTx~V@mz5?#CJfA1V^32MD46$ctzP+r#X|g8jnVa=%l45&j?7bzZo3tx+?`* z-~1TZZE5a&`02qB-Bx9u9`IKBkA8(L1rYvxg$}@5jHJlCqYi)}eouruSn}2f_xRb~ z3AiF~e@@@LG${LPP4#T(0@%x7x0HTDDc%g9?FVtsHp*-pWm~w4{!#eH4(?N<#!Df zE3hZcH{5u(_5*&C*v7Rw=e{9yCoYz9<_(LJeKrpOPDukOEB(N1(Jb8Uoq}D$I|IJY z_HO_u^yR$I-vIGsoMS8tInMFzieg`!TEm?@I^R9}2LZ?AXC*Z!$BFM>Bf*h#ABv0& zO>}6UJwt#a3QU8@*G6@1;I|)UJf@JU)KL*6CC{#5z#%1j3}JVk;_Kr3qy@3!FpJe4 z2;U#)XWs}w06_#&@cXV2z?#t+`8ONydrYmhoTE=_Kwqvhu963h|I5*G9xAc0Wj_k? zOqXAZ*<6No@MXLqjeJ4JQIY(yUClyqGQ0vjf4gBRh;6QciU7&5mTT($_{;ZN$`brR zta_CmV+L{NR*SmEpZAsuL`VzlO&4p_V<2{|(C@K*fgia%@WgRKxF7t&|AQ0+ac*CtHc&s~?R^OC=Uf z(i?|LNhx22s=GtXS2aPWQ*8=-VtlJcme2udJE8XsZ+2V}weBH5^@FGrhr0~sh0_a@{L?E- zE{&e|)h}YB9EkarVPaG!|Gz@ zmq&Gp+4{IZ_wIme%iyh?pl3SvmustVenrhM!zN^N(C&_-J*oETV_0=%_Giw|TKJn( zw#&(LRA~pZQ-AE6U%tGr)q8ZbksHowVE`n;!2)M$3%UX=d1`VS3&H= z;6d9LFJU4ga|yHWgzT^M##;7KY6_CdH=7Nge;}mJa}%ABHBus5B7^L+stM=xLiV^p z!t-3O%E(Q*ylbaDf!_b_w`lL|06 zZAV;pg(W+@7-5!GIT?^$QwDJDXoW+EN8i0-c=N*X!;5+*iQ}J%{m1s1e}(&Sj|D3a zbr0QgYcDO2Jq`3S)2S)5=Lz3ye>gJAtI6Xhbkd2X+qS`xxp{&DPc@=aqZ7I<8p+Vj z4kTwsh`eoJ)iJ$rHu2E;e$GKkiodG3hz|%53`0e#JnkEyZY*u3^$%Z#>W8{)$h@=3Te;-ha z(J3WxbWvou)k&Om7xN0+GX47;yH9)rmx&B16JnD#pM9{U*Bt`PWHzK#jj%iVPb&)z z=C`)g=x|n7b>7TZq}#_uXQ<+Wc35JM6M_RO{~#UeVlY|Ln*PvBaLNv7!}O!fmNj7D_9=9k}bt z&FYqiBHwfG)bO8Xh)vlw)tzo3+#JjY>HHIO!3kWZ<6`Ia$YmrC^_4|PsDyDV)8BHM zm!-~?p)z^nS*h9oD&R2DpITMUN~{`|0dMJENF3L~eQ^I%JWO9*NYQE^eXw@E)Snnh zd~druR^=}Bb4`O{0c_})rXu`xDRI}h`)Zxs*1IXNEE~-;*ALGxBQblpD$OVtVoS#Tgm++65QwS z`zeMFyg}T4B=(EJ(hzVDalBFJp}zYRJ<~9OTplbGd=g`g*YMaPQ>0-ous>Jyf$xxb zlMT7Cq{VmE*w!)QM2tlve$TcQtdj-NFe*o$TbQ71!Htsrpbv_)c5`(1J!E|zD2SR|+j}M#6#f}I0OnNU~%Ql%DxmvYpVfQkB zO-pRE(1#+v_dPkjthRh`GZ`{V>gpRzbc{GUtU2&&*(Ho|O_7yAI1Y$)PYTwj5bY{R zi?B3kAVZHSkK7q=XBx`HYLK(89?o=TJ3H`(lm}0r1oM?G_Mp~@h|>Q2tCb_Cp!QzY z3=_7p@w(KnUr!>u!?UjEPIrU4A0E7;a4go}ZyY>z;6(Z$r43qK@I`Bmio6q5EH+oP zg1_-b`l6gIwupEUe+OkAeP@^omWcC$M>N-5U&UJNa65;LSw8yZ4_WLUvHu(KjP(EX zK73btFJvm>(Z2-8a$lxiax%p$porM=K*Kq7uIMqKN+qIUL?(2hx;Ws2v*nTbUg%Qi z<5e!z>Y;st#A0t`U{j^#iMw;yPdU!Oh9S$JPC;R-Cy_xNh&{{SUJ30h4|Tu8Pc6@4 zIl?!H5I1Sp55JRt{p})o3LaLmI?t+vASkAS$FGs6IeX!IGEX7X39>>LBjE>otsyhd zc+hI+h@+p8A#-_*S9O&UC!UC~m2UdKpQk#u{GXn})^V(F`n|>Hf)V!x5Y)|>bL2(Q z(|!C$qJMJ;gjz}oBI4ND`tIk`IQxgGyDu*b@3uG)|0cTc*^0Gtj+_y9t51!-Utxub$B2-|ee-6tPzz_vixB{Dn0=tI-N4NqfN`b3T;YE!CcZUMcxB~Br0^hL$ z|Gk0$p`swYq7aXwFho%Vt|)4uDCVK~Bp$9Pfl`z#RFtYwlefM9BcIWN4ve(J`*k`M+3Oqx*i0wj;qm7~7s2yB`7{fWyC8zz03xL*ejY z6nvx*K3W4G>wu4s!|j9Pz0(sK(-XXz#>eR=2Djl;SnzofxsYJ&Sr5&naLr|u=J&#h z#SVbIL_&i^LZ1ls+Og)Zd(CwOq1Mv|z1Aju{7<;%_DWZ1RJ=KLymM5%ccRuoq1IuI z)=`Jn@wnE>iq`3|*6(|*GeYe@^xEf0Y!?CSpWz8P)0*~C@&3U$Hz@6Yh1$0@+IJn= zcMu#G0j(9d)>C+wzreV6ly*9g4rbhxH|Mys1aEd#LLV}IZk{P*c?y7d&?aE(3Js1o zkBSefN(gPi0TJnvFzAx<>XOOml56TxSn5)G>QY7MQm5$B6wQQH;Y^FbuV6`N5!hD_ zIz*v3i{mo{46`d)IG2!w(BPi1;Do;8sRSD>mm3}0qFHjwN%I?QmWkP)!n3v@J)D4P zTc6oTn@R689bp-L5lw`?sHOgsn5VwTLp;%yE^m?kg8i&*;xxabCXW&x5wE`ZgT6e` zyl9-hOS-<4%>2}r9>lXJY+66tWuBYDK&{9?oXtS-`6VugC|&g^GsMfB;HakPB{a>$ zc^5{18sHw!YZEObuo~(|7!E5Mvb~&7ura*vFtnI3w0LP~a)Mas%QD0`KT)J^6m ztCqf^>1~B$BLq)y5?9Bck|J=yDYMzZI2Xqz`xc8TnmSv^>1VV@hG@LdOBCMfgty7! zMTbew^hDy-Y@~!K@L~KyWHMVRDQCen9bovT+_b<``!z#K7}0`ToW42;E6@{Zysm?& z)sCVt`KZ}KpNPg1Z2U~B0VPH;wg6^)mK$8=mDe$zmlYZV(dCTop+MTI&JI!_ zh0uOB_~<}!SAnaXEnwq{JG0K)qu#@LwKad2XJi8==QB?UgmgwTw2v~z99-^0SvC%1VH;W5qpj0} z;z9-Da04;7Aios561+ixo}6EgJNkUX=p&r~O|f|PIvg~kWoH2Zrz$$9z!X+IhJG-% zt}4dk$xr9?7|?o*F=&mH5-TJhVL4{FsVJ7L9EA>7)}~;`uuo68dDt=6*F_#zi2&5`?u70;+5?^Jt5P}ErhV_@;kf8{su=qF zYyK?&PEM1l&cOr75I@j9Lbh4ZC<(3_4LY;nOSc-0z-Cd6hDXJ-R&62(Busb%F9{l5EyF!3h^T821nHx?V`{L~BmYz1NyHu43_6b-zp==WTm7#R3UkqA)xLCqh90 z^hF1BQW#^h7n$f002EQj#Vb%;pA7)zGL>9W3EJkP*gNVWTI_}jc!ytRv;!% zXF`?OMkwyIz)6DnK01O-mNx{=hA%kY826C%n+rAPQRMVAtIfsje%h+fn<((%0fV zK~(rl(V)~i&x6>@QwyNd`ljd_ zf&7kl;-fZPoJzr#@RZpKK_?MTn24905dtypBXPKbX3YgKScZ|x zE{@p(fU7U=Tk-q0`{OYKQbxfFx>-n@{P0T zxYKt3wg+e8Wc1@zf`AL7b__v}tG)52gL9IjwYIM6I%5eIQ&AvibKqB??dz`dce^nN z$nx}b-1E?hwK|SPf{?~Hp(;t(?^Y zP}N!0_yc;M*d?zcq3M?zdlq`~(qGH&fTwD4awH^LgQs^Cy>I?aEv& z9nvm@>LB$Py=*R*Goy6I3Lq{0qT;h)Xo#LjC*Y; z!N1-UgG0vWaU-xffaCvU;c_XYk5An^>6^h+RSX?w7J_Q{LCJR``~5slIezb(q@JUQ zvMRq?&C^S+tl&S?5~3ibcU;@&$2Ky$k}d17W#KQ-XfITfo!M?{mGSd*ku)w*(7VuI zSb%8s@)_^gX)GB8-+oS?5E0R9e1&bh=YFlGGb*Bt?1E-+P5#n$Ju9MP#@!@U$mZFe z<%WTn)xG%&G8KvII~?G|H&_1V_}PT^Fkk_a6$Fh~2+-G(4tcpLUgk1QEn>j{&;(O$ z3!n_ludpL5`%%AMpZ- zep=v$9a}=be_WxH(t8dViliHVc8di6Oziy!pO#0DWvaWvoOqKB8E|v?h{$ZRdfP{$ z`w^PGjrNVVtVbSMmC|D~P_jsaj>(22lQod@vElGV4aVljr9}Tlhs{)ZA~G13T8AHb zWwkF!FUo7!<(1(cgdCk}Fo3`y?YV%Nzw6&&JAe1X?Hzv)Isrr=OTSCxHFlnESu_^Z zXC@Y#FzSHDg&|{iqfhC?2|?>#S8SNZBoJ@|9AW+CpY|D=$TB6}41zvoYDAJcrwgiW z|J9V??Owr+#)-IaqREf6_aLqzL64=I+O{}{Q}_1x%XsOD?8Q|+1KRL=*8Ed_1W@LK z3jH2MNgnksL1O{{qKosWD+uP_;^xb~dghm&79OF1&cFZu8rPdjL+yf`42-)}IZi74L>fIA#|2iv+TmifJrV!TK zd!l%=^N;!!nFMDCLz0Q#)xZXDnAi+up4~Exo$(vBBW^SnV-R|okF8X5FwTF?t(th3 z@>&{&6-00M!F&ykMW^8$kS(sMkXe6KAGpRV+R~{A#Nh>veaMhNaUKpdT-Y#Mz&w0f zT<3rR$j(JNP~h;AB=S5fL(tcp80Q84$0S|yLgq0^D|B5(T6^(50l;NpLPsPuW?1+~ zJhn*(IZrmWa=HiNP_K=T3CiB(!{i4F5kLBzXIWqGdz2Z%GjR9>M2VZD;yGc0twF4g zTwWj;!f-OgfcB&CtE>ALCjBoKKWBcaaavfCne$5v{q&{QeZ7jqpvHPMgHQ_tAEs+tAEMx zht5^6T{SFOuX*zK^d+00&n`7h3GjzsRR4N7Z|;6{91D92BSS16k! zYO`vO9fjM!PezJ0F=LtZnxJ*@Z~wXWDAv;d#Cz(S)xNSTE8dpHpxC-g zn1JZ|#=Xh=J7lhX?QfTO*O8{OIXC6c$4Bv=Cr{;3nbvh|If*{9y0bX2Mf_YBXBlVS zVbW0n$6or7BWB&76#tH`y1E2Q4fgYl|FSx^+2tfhHR65K8asCQ5R!#UKb8vSI(Egn zB`5ib@+NixYI#(900uAhBE$&jpsK<>R+pH@}{^l+gnJI|4D?bWW1F+8l6} z{7Z_#v17vO)4J0{F!c_+C3h@&c**ownsP;if}ogyxh2+x>uC5h993Dg`48Lt84tqYdsNBgr|l z-Hk0ihTTn(Eg9uJ8Ex4SQyE>uBukqXIDQlSRg1L`NOfpSVrFM$vw4_p%jCDo6uL>f zW_yrmhXw5C#vbST3M5`EC2p`|)Z1k2v$GktyWlEi9Kn=nN@Z#TGKc=hiEq!+#Y`bo zMsLMT3miwh=+fe5+2Vg;Zz%xAEF+QxVbC>!`2uXa;r2%c_V`xdqqCU4Fw=NexW=8Y#VW?vwdjdaACWRnfm-hb}HxK7q|93 ziL6KI4!;YwckuvxUjYUMeSk86LUF2skOO?uK^-5Z`4OlkRH9V@l+`NbAKcJ?Jt?4I zC-`bh5Xh?MW)(w-YvD0lJgj8!deT7bJ$&~4p@ZY0hoeFu8__xIr7!DIu)}Ti_92Mv z3{mc=#tz;t=)=KdgLhz%4>($kJ08qC8g4os?z0`9l^fjd9E3lWv*7O@f_Cp|Dh`=; zabE1|3bD_9UlUHJtn0wsF)@4VgYT)(|D|wnI~WxxVZrN*uUWIbE@4s9*@v?_O%xYz zr8_Bp!>-5UEEk|gGr+D*{vK+=-fOdKiLi3)@o~c7D;GUv=}oK9I;?U! zq%J=My`PW3WXndUYUwRdRn^2*(2v^@kkAH;P@IP=vM)Q+I4CF*X3`1dFa-dKO_74Q z7|ttwF@ETrBS^LEXSPS>_mzHu?dWJd3@sPExZPfF4^k?TzL?qgpiyKXkTA#$;i&n8 zMf8=kOy*us{+@E#p6o~GkOq#!mOaV_r|>$*IW`!OIyz$XDc)Xw+Q%ol=w6w9T$7j; z{ilyQm^wO>J$e+TNqUJ!mEBuDPxun0luJtPg>;=B$MWlw&soEvx?y31JJGPZs#^@P z$*6n?NP0sWJwdKx-ci*vQ00_~_%CJhU5XWJDn2UpFRSavRM$|p^y1X?%?;L%D@ZgQ!60Y`99qO-MBfoiY~{#%+1_p2{mBylPfUlqgH@E;N>J~D|$@>IV@j6+h&k=8AW$rkkh7XhUy=q$Yl_;_!mG z(tK1A9IkWXFr7dPZJgKBD5_&>8h2RaZNzXg-9G6)_OxPjQ(|}1;yc{H=Q{*>xvm|n ze68s|_Bgoo*g`xW&&t3)mQ6(*yY*ubjO_2qIPlG@*S?85^eIjO%s;dMvfOvpTynQjo>qlb~!i&R(FRV5wP$Zzu@l? zc`;$<+zoqyz01?9!{2N2`#_>@&eQ|sa-brido5O~jd^jb?{Z!gr(P($xTM^ryhz3&Cd7S{iYv_!!KVaHuVI=-G1c&r!?a|U1=vu4 zRO*vB9fU%C-AUB+@DEzu;NgBQ8g-SHXol)T>zTv|1t@8D^miR~X8wgBLQHV|N%^OF zc?zs@PNB8pZ{_+*w zN|B7L)wwPYv=t1B*CYVG!S~4Xzi;Y9Z;VQ|KoOF)37?fN;?fJeUhL&fB>jXnHn{!F zs~vJ}LEiR(Rdjs2vzaBE8bx9D+)Ml1#82Kl{PpC?f8PD@qj|ea^IL}*= z-GU0XXd{&ES|Gb>oWh)_*x@x=L#k%SL_e&JOH>i@FV!xruK%WLGzTiO z9rYGR5C=(7Uc+LptzLDZ4An4x$f#|4lDW_{zhppsTg^uzy}Mo zdczzetmbFz>-6r*iC7|(7#<+}!VVLGia8>Py-p67YcZW~j_78LOZ4eS^2`xRSn~mI??!w@% z_Y&PTEg9W_psnPf#z|jHRF<(w_SDyG9C%t~vn`HTu6Dg$w=;j^=U7CmITG=&6BPmh zAHI1PlKN{bwNWw$b1Q8xFr5x;Pvx4i6;cd-#y(0GECmNYwrW>O`qxYLG_@xF*A~Db zWyaSwCD!&%Q4GKCYa}tep#3<-JD@~mq{KV46cd$W>{EE1oV!Jk_d7W+ugz>fB>SwD z^4gQIODcQdw%aE(Crm0yxD}DPecO2~WwzQ{@67$)y5jxyzdvSDCibBr?x77+x83t? zy1%4or~akCy1mK_{i3+@#k-Q-?5Ur1T)K0qy;D1eBOSB5rmeg1Upd`f7E^mSXM3rC zSk#+#xp(d5z}8p#VeN`H?fq5kn;2wGQQxc~gU;cUCjDLdQY~L^t($ET{8E8k>36o$ zVVUw_*{We(m$!q0Ke(xte9^-yUCTHu1S$7ACV$@#20=!_N!>xxJ<>@%*95(DQjOOC zrv6JBRm29z{)3pyCI;@mE8S2{m+4X^g`NlJ1nc#7WOm--%b4cY=FCzKh=27q#-NTmi z+0VSYZ8EhT@sS-xwOx0TcN=Qat%GjC^DQ^y+zx%Fl=*`BeKhe1b{#2n?b{A9ziQ7YcLC~8_y&DQ0gv*Ew`cR7`O~rR6AN*P2)h!F<^Z1}+rSz7kl!nfY z=RiMS_-zrriOzm?CXf5;(65b8Wj)W)q2GOoScQ!B@^R_QJ4!E$R{sO+lq%3tIvMB9 z->b18Z%X$7JSws9Me4r(WE$~!>Em4Op)4+8Sq^n)$B8!^k%+LDdHyal_(w!^*!}@- z6V0za2Fm<;MeD_W;zLrkP1{OFw&w4qN_vuuLJOns3~kL3?!JT zwo@wvi^1Z$prEDV(+Tw;c1r)xQaFh|mZ#0GICHE+t>6 z081h(t=c}t({OvejQTLR>?r{f?>c}h-bEXq{CnDuxQlVSeUD;4Jan*YGnX!{2d%Vf zR^H`uO99HV;sa8^yK}|DKg~vQ_7mA(ej))~L~T-TA$FfBS=-0=E<8!+KaT2UG3kyc zM@irCPH!<~)#z;Vjq&QPLojP~*VUzYXV*;>oAg%gz#?-?UiM9MOQCe}13H#;brSl9 zDI){)1L5Tz&O+nN#HC%M(}@t6FSswLB$@>P#-bS`GLE~gn9oNY2$1P}jkr*QTU7Vy z_)r1}B}R1{&<7Geo8eMIs0I0Uv9)cKLOLW-l{pcDb%6e=-pkd2Q#=#|D|WMGQZRq{FPRSF)NHL3Jcdg@6IRUFSYO8zwPoRIZ2Z`93-3llV$sGw^# zvQ80Iq+SuIert|mD$6j(LO-K-nSRIm&Y5tspIXbFmc3KC*p~l?>z(9~nZ613r^kNDiL-LJT(RA#>0xTvrxSqYuSdt*Vkq(Z5TvE`uV)e;3ZKrgL;KQrG{s zT52u)UF7Z3+?-M3pLZ$=JzM#WMFK&z2oQOvMAKTm7ZdHi=ZeN3F~g$j7scmu(xrl+ zmPK6h8(w}AaK&V{T^8$W?U+FNXPDz}*@;umDtpUDQ&91#-~j~pxH-mpxp9pS-$ zESGQKE8NV6PvO?VsKqqz8|UQfciP~U^?26f83YV>o5mHu#`0-K05`mp>0&@$RuUv& zh-rYq(W2gDGM%e(YsNR#2Sr$!jf>1|mE$64vPdcYIa;7jaHE#-$d>7XoS8T3K)eBO zhd!n2(4mpg_4oih7P79xv{{~{2}bx#)FsJ4P6^YiS=D4St!z6mc_!Y%$X58hf+bA= z1`$?il7=`5>|dY6pxX(Y`8qKl)n=O62s)#`zF9pWx5M{Y#l{Z_MkF)-8y;8gVKTwG zQ#+~4#?mFkalFh}Wbss;iY(-D5G!=Z5b{%VUn8asNjDnj+~{VFst6Ht-?uer9S8ii z*Zy=&@Pq|68FSp6EucYF@E85-bs?QA7_wl;fA@XdQ9<~1mg~YvHV!OAU_0nfC?MPc z+>YX$R86a;7Uq-FlYKf9A?zO*ge)Sewv>k#wf@taX4g;@@q zV>OZNHPu8;CZF5jwJcfHlb0`qdTFf&k=_XvAS8*Nyy7l&AXxgsS=_rfOQP4%^*!4p znB*yx(#64!n(>HRLoW}{BUtlBBkXY;RkzWJcWS)Mt7;qM=K;t&L ze1;Xx&QBW0g}sgaU`im;ER#pFqi5j2DEdKN2|e)7u7QK!_1Kyye{jE>p>r-(!5*(u zXkP1@-Pfhg9z`DJ~}Fnk0h|m=R0B$@{wVA2q2K?9WqIST<8`=2>?iNhrmY6rB3h zp!qSR1;K-GPG2f}oftjw|7Ij1U9F`K4sTHUh)QuK^udg$6|l}>FYk7!-_Nxzl?*() zZ>#AE85#m)V6)mePl7CWGx}~kb2WCXUXr>!{vAW>)?mO!ymChjX_@@u1g6@OZPgQO z?|Qj{GrS?Fgo@!)JXng_nD14!^NJGOdb}Nh=fu8(cAk-Uzx`ccOz8LB2R-6ygKq9w zgdLE+0PwWa8&Ys)$5)s~6x4$A-Tw2wX?pBU zdF)*vU!M<4%TwjgyHCqQo6g1ZK?BFJk=C%qq+^lo&|@hGoknxS2;d6F379D0Ai!wcK_md!8AWvtyz#u)(2?~;6qu1vX6e!pC{bD`QYMd38skwF=~7uIQaur8=$)tG z#rk4eyAXNek(%PtN`U;_Ck7jr!zAA!gQ*BpZpJ zCsBW@6YX&)beV7(1m@A{fTYZ1=S+=0Z4hGGc7ew(fsfmQar+s0$AWp2Ao8ovmp8f_ z_ZgjNnM^8~tW?Bw;plAodThEvREIDEh6V;^KLA@pdl`tF=rd!4FZLhbONn!Ane$8e z^GR3(Yw|27u_{BR9#@_of}tatx!Z%_NrxRXgMDtCJ&1&ZUyoxVox^>aV*$kZq|Hf_ z%K7}snQ9m-CS5DjBdg3LtDgN;H%r?nlqWEoE8<7h?U&Q#o6{#4H%P+!gxSctAQ-ot zk++f&xBgjxYZl4Vz}I}v*ev*8dq&*NdEEUh5=~z~T%QSV4oN7C$O8$-kzWYY_#^4( za*zFT&Sq(^X6gEcaE64gMl*%oFF0lVk@9o7usNivu(&1}QpZ2X;3Ce%pV6{W$To|} zQ5em2PVB8et1lTcz&|J09~tg172Oz%yogK8ibFNBWC+Lc2oo3h;}nre%{9`OH^x?7 zFsz5) zWYFjCJ_1H~|3jc2LmxNMwUENyEMyfVLtp^o`XkTJ3DqB_>p*9G>U*JNEK;)$RkEI^ z6B6mrYf`ossS&#rJ!ph*?@}uUz+wZ?G_qqTaT$l1X%LIrYI)geZ|2qBlB>TnP_N8Z zubx-0C)a2+&}h!qXrI^UCWrSMz=yKoWApGSk%!D^#%%?atSlAuOGPk=YFv}(^$-Gc6a6nbDoJ&qhb?gc%53VmTi zeeoQ9X~crQ9EE|Rp@B+{fyRP?_!agt6U}myoXWnOCZ{~qMgG7*oUn|vw8~iMnm)F9P{=C z^KJ@@enX3)9E-69izy1rIZ?|38^f(K&7np`ei31U_F~lsFRt!}JgPTFIijHC3R!aiDWvp!-swH&u`* zrHA=n-nAT$U*tx6SNunYo}!A3@lX(HU{aSX-a+IkI<}7P<>@)d|2vW~@ z%!UhL#a-VC_G7}!=c{}lry+t;#d4AcvX`|1pzCo5*m)5|d8(uuWv>R5N_ym{%U)Sw z;R{tu9&T7JZbNCIjPxpIqiB9Bo0P%t5sAFcCAshn+zB)~2j$|#)m&Vu5@s93;M=Y^ z$-+%kH>-oJNm~i2h;n`$M?og{_e_48EMZf3%VkUJud#M5u}(KJZp-m_?_^ z4q+|wrzIOc6}2z)mcT}NUS`*}qTk;>(Uf4CmEh3ENq%=6S~MJ^)QY(LZ|-Zher)#Z zIBEF`Wz=qFC>r}tG{R#n5P0N+aM<$kh=<$s__e@7l3@Omw)vkw|EBl7l8a0Yt*?65?k5FJ7?Ww22P`y!Q z#J;m{DKUjTldAgAD2U5xYE*RPu`ji5c+C6VMJY0(2+BP0BW8_@Z4aXk5 z(pivK&OekOX2pEN)k{8n#|K2!qmI6c9F|ifKdOlP)P-`i1tn15B23z8$+|;Z6fFi+ z3I{YS9G{Zuc*Dg@qm7;7#D$ZVTIsgZU3n{Yuo5@-+Dw(&pW501=8ecI$SGfa_%|gZ zhnTV05FZTqH?HEE&$(EHx;u7}52uh*9lB1jdlP$55;Er*3H+WuK`VV{&|z2tOv{6?wag{{xLc za=*o!g=N5l_`Q?aJ}A37Zbl?>Tq3Ysv!jE*YX+RWMM4ax5!ky&th~&B72?^-Bkm+l zh}@v|&KPLcieLZuDg~lLwtbpkhT$0~f!WFAl<-+0{Jd=d-O$IK+{@kE{~5)5>9{E< zxhZJ56R5c(-2!v?E9AVL%;-%rT3nb;h~)X1bOc+Lh*!?ZxSUO0OkF(OAb->aUi?Ew z9LLJ!p<5D`(eMQ%cx1_c&e*;^fT(6HFOpy*8U@e zIq!pr>x}~2&&oj)%-bab+|@?hncnnIAN5mzWS!oa);)sSJ%ZjHZ{q#x86()l1NVz0 zPHwo37O+&#_9sezh39e~_gfn+g4Vd^mP8DK_{C#JVCxP~3;1)-Bkp1P-D335=BTV; z61v~}zaRX=U;M|P{LA0`&maBMU;Wpg{oCLD-yi zz<~q{8a#+Fp~8g>8xr(HuAjt-6eoTYb*`eujT}3A{0K6CB*-BR97|${W_Ss3@pO zyHd303!6Q!WOtssg;e4_dGy-4eG50P+_`k?+P!P{-M44&*y8;QII!Tr-@eiFJ2>J|BoI5uz>^ZdPxR1Z}9gX^LUy&CxmVGUgq)FJcRknOt zJ9kK%ICcB}4czOw!oX|)G;HcXv&;NZoN z|E=zpxAVTe4O`D%H$3q0;@5^Z%vpW=wOjM2E!%&8wA}`ifMoHOlYt5@$Y6sGUKiF_ z5B9{}|Aasd^Gb9U>NLzP&TXjEIik!4B2C}C(p`x&)zcwzD$exEFdx3Sl7BSD6yS|9 z5vZd|2=)kMkU|bA5q1(1=~0D9hGa~JOMcYhj8AGLB8gLKbYhBER>a~k32s@DOHzW_ zkBwx1PL$)B6Y;2KKR>n!XPk1PWF(wQ#@VEXXGSS!m2qY%XO~~9DW;res+ne+Y^wR@ zoQ^*FXq|1|31^;h@)@U}Z~_{qpm7%ZCq{85s^+3+HVSE~s)8x0nv}L_DV&(vIpUuv z20DzFpUP;csBn^crm1A2x@xh;PH2xh$}Y=)tg;c>iPoI4wmIvhwQhQAr@0P#Ca9bG z+9v<7WC{x=vB#c^u6_1k<4e2lzT0jt*-aX)oYSgVt*h6z$tj$k!s#u);38U*oaAn4 zZn_FD9Gp7XJPdKf4?jzA@_=BpPwDXYera(t%8YJyBCuc7Kmrpa5PJf+GlFU>S;G?QubnHZCqZ_PZD>GPRD zcPsR_M1yJcl}JCSbkk;^-P6-xMos3_U{($F)nH~#=GJ72+%w5V7aZr5 z_Lgje>GqazuPyhMbc1O(n0RBYH`siC!|6AYfDg`j=NlD%<>89sOs&qeTG?~9{%#v* z|BPeKUHO(_PnkK9n|qFX=b%q1`jqqCVqTl&#R-o(rg*cTn%<}OQGyfak2uCmeP(}Bs z2Le48L?jKdh>z%_AO8r*Knik@ge;^X4~fV`DsqvGY@{O}3CT!GmyQhv7=NTJr72HI z$vJ}P9Ip_?E51>pJFcW2w3Ove@<^mTUILZ8?4>V%3Cv&$bC|>|rZJC+%w#HanamWX zDlK#lK?I=(P!L7?9P|zh)ej^snTi`4@eEte1eZh7Wi!u-&UC7Co$PFTwGqK!FmOFornogMS&WJi`rKu$V}O!3834AqhF)!Z3V7P9d3-o)oR9MK6lc zjB0eF9POw_CuvUy)k73H$OHfS|3?v8PNW{!oQOPZsnU(0G^K;0L>X?86IKq4r5za* zG$?o6n25F)nZK_kB3e~7eb*faYDn&!;owGcpAAj#KMXjzP3Q@QL z23%-_F9m2du{6Yi6Nx8RXQX5haCXH{*H>w1z~fKw_#@ zUklsV%67K2t*vc0ihot@KqC)Oc&IVBU<6TAHV<47L=sX!O-g(Xp7%#Vyhz@xui`&5V(M% zJd^<`yDcwp%kYYRMAHZhH9`-{@X(YBA_s!V$So4eg|cqbBY&*zk!x>@;S6hd!yN9g zhd=Dc+*;=wd4K^W?m>whq@cl##qU_TK#8+%Xl7r*g&s6Q6i^7lRwxk!@VfG) zjUYl9_^QwVmN*D`5Cwaoz+d1F!x)sHgemgy*t0m+4Vi$3G0I>Be^Mg3rf|c7A>#)7 zelrO{$OJ|1aeoUoi$c5$=0aFgyOIj)=)xfGv!DMA=s*j4(7Y6Ja^yi$H+;j)FhDRR zjL?XBWP-t}mB=lE9N>PC|I`SXctsgtAc${dLZ2go$vKGXhF4I6%`(Drb<5<2AW-5R zqUgbP?{N+<7nuuC$wLoB0h4ExAqmEKhZ}M*>wXNQ$A5l3c6I9^2qH9Mr9-u~N!~$P z{GyId46K<0`AFz+i+kMUF1NY+Z0O_|BMDG21r&x2W8Qjc((3IT1fCdy`pwJ%o_8q={MNaBL2_qY$;q?%89the4Uf09ddHr${hVcz91mPAi zS+uOr5r6Bd^n<gN#+8=Ns|)+&u5O&wmc|pr13{!GXrH&729| zHn$AtkTin@cxj{N8{z+P6+~u{gOwz}ptlBpY}F}Y8eij7A_#>!fuS+l9#A(A3VlaC z%-r!|4+9zrB{{5Ry=Wq&h2{OY2ST4tk+i{_ zYncm8M>_E*$Icn&h+muHC6E2=Yk&LPA2j8E zgMZH+q@WC5=0XaRP?gUEo+D9r&2M3l2ejPeM3lINr*n{rdraC0F}_7f7(Q}eVhTZ! z0?2m`Tu@SCl>_Jp|7|%y74;x&#D-!GB?`BIWn6$}$hUIyRS%}n1Mrs#A&3(q(g^m4 zfTl209g%Z1vUA<%f-eYzF(`u<^?knq4S%232##@dCZT>gXm60_8gGSwJ9=73N!T0xKlL68Lg#s$#u3JjG5K?Pq(L1NPcg#mUB zc|dH*w@xdjn!z4*NBbTsEymmjomnng@5=Pcf>6Z#}sZ*6y`X5;&gheAra!`b`^0Cj`5D| zC{tY07ck)%?dTW@g^?2{$&zp(ku3?65*s36mQM zk;4HMH`$VIQW!A7lPsAT^k9=Q36A!04`NjYOUabUb~ebUBI&av^AR3#GCXinJnG>+ zSK=Pp(g;GSt3%oB-(T- zrKm64HaDvCF}YG{SK>OoGJj!DGCNOFJ41qKy#bezi6nTrN()&jq=PShX*Gc9ID+{z ztOGDzBQS}%B#XHujVUCzBblp-ED6awmI*7EnK1$sI*oHBkFzC_vn7)QJEVyurimn| z86>L7n#H*wl(`{#i6n`0C5vM?wy8dzsWqS(GNL&$zDXp&NhHDvB!9$doZrbG$O%ob zc{A8KB+glzoOzqlSvS-fG_KUB|1v&1? zHSk%W462xyqbM$6mw$hA69-A5HA)ui0xvo03-WR#-e#hR1EOyuqKreL1}5UL8>@I`kY0|q(_>YD0(;6nK$MsBn}!R z5PBn1TBUKCXBtYU{P`gMIV1r3Bm#;)1UjZadZ1=1IcQ2bYJVCeYzic9N+WP8r;W-J zcIqH_DkOTkmtdNnwgRYAf}m5Ppts4QrTL`65~XdzsE%r?Nu{NF#HB!Ls6fJ}Lgc4N zVxXET|ERl3sJ%HVp*k+3$|y5>s=rDbklG-U8YGm8B(JJRu}UPf`XrnBB%KZ)GaoR&JEVk)a-8mLLyCWkpRxk|3V$*ZU0tJ*5BKf$dF z;;lM@AHZ`dRf(1AFd}Hlq_=>IVfj3}5* zn@FC6Xk}1b1A9+(I1{iqWjD(zshUfD$hocSx~~hnF4#Hs@Crnb1lpCU?14bWV0DFA z6Yu3*1NRC`3MY0ee;_RwyUDA(%gelh);aQ!VuEnG4ANCw)ejRV2)^49w8jN)AP7X| zQ0HR2|AD;CYrf};zUezvD^qbnXdEzsYwLq$T~=xD5E))ZS>Ip`#*l{sw!imv0q!vJi<8>|@#ti7M3y5+>W z>Wjolti(%9Pb$sSbzKb1_uzrNcD)LX zVzveqDP~grFluff4|E4*l{IVu7IMf3i?5Igxz`ByBXXH`ga3AHd8LPvn|NtaxtvYl z1#bV~xy-(=aA48KhS~)Md0@ZCM!_2qe{I0V)@6WP(sjF4k=Qw~b+dM92xrzwP+huR z2B?T_K`_V!4NIbsP-e|wqe#xk&2*Un10!u|(xtQSCd25Z-=Ai46w%=gWeSz1e+Tzi z5B1%4EoRat5w+%n5vSgCR|c^34WMg=2_++$09;To+9wurZRjRI#s#K8V4`~xe<=-6 zn0?@F?&fdax-3)AJwRZu;08p%%x zTh;`PY!K+>(%T3yC*#N64-dw4Bvs{4es=ZDbOPq%{=C+2wpX9_g*{~qPDqDKp6LqJ z|Ku$P<>fta>wR8Uj^!VndBebFf6^;W_Z{Zl@mmd5RK!~mY91de4d=_w?9IM?EE7#I zkOJE^W+~9#dv1(=o^OwZ6b956(13O!+yle#3?e6W_Ip2#J`qNj0x%F}uMP#0=xQ?F zTQr_cTv!#$=LTXO>e20RqQHf69mEC4Kl!A0907}cv0#d(;9vMucg1>@_UR2$#j&2H@5w;Na%$DX;P?|9C8eXfR-6SPg$rf$bf^cGGvg ztM}lJJa?F^UkwG!qjq2y7i!;d<}Nj9Gk?8~6=5G31g4qf$Z#|>@3gtjqms%)-nSX#+CKG!j;N4XN;l{f6vFe5!SsI zObMQ!YB^Qt5mp99wT7Wa5#4oDp2+3ZEC>WOYj*zN``f#eFZ!>5e;m+Ybx?tANAV6^ zeqLnka7XC*Sd3Bw<@t~t{7W|#F-N<3U<#IY%}`;1hkVmgcMf-D65F2#kP-T@Pf(L3 z2(LB~!*2PJ+oJ(axhVhm?+^d+pJ6Rias1Q4x#k?X=6s|$!2oef;4o zW5bL=lXvgjGh+|se<^$it)a1R_wIfBR!?ESdfy-_WN7T!IgK4Ns(dH!+c}gQPm-L8 z^4mq0(0uwtSaM@Xh60V=8urrRxqAuUJsY-irpR{(okBy{uPV)tO#4X;`PCp*s4H8R z44bgtt0YjKjf5LFu03Da?8(KOSMOfFef{D+cgt8pcs{U#`fpo_`r+WnqhJ>37mgA>x5 z7hnE-`t|MKf5)F+|9<}5xbX)tKmi9Na322(L@+@G0sL=42Om`MKnN$Ku)hW?#4yA7 zB-F6O3NQ39#05bVvBUy7G%-a6NmTK^^HD?jo|u;@VRP9w(>V3m$KVQHM$`_43A_Ugghx#|=cE#zG1a*f&nB;2(@#2`DO1oX>nzkzMHf9zo_8LF^c@`;)$$%k zBaMktxa9F^QsmBKQqvcq6m`d8UKz5~^@h16RaMnHN0jbfRqvZulBD&!|9S)}FIVqg zNy^vsf4&jLRbi)l#*|r)t?nJ9JY_by8I9>RT5GSxHrw<-)%G}2yFJbrRKYcl)vU<< zE7oPv<;zxF+1;z2VTKV2UcQhz*IvDdHCEral3mwdxtz^WV7%lp#h7{xX1HO8o#gi6 zg#?orE^+&9*xZUyl6B*L-1V5Gc^8%#-;nwBe;DA935K>|gcS}sW|?P(D`JVi-S}dU zHTGCtk3sfWUV1$?`R8I&R(9oQH-4F9nxBT6VVfVuxni9+=6PeEHwJoRp`CWPXs(am zSY?k{W;$W8qh`D9SNE|eZn@`?1~ewcyWmvhdg%K=d?#1 zY;MOr_uKE@{9TDXe;9PCMF*VpphqS=Ux`sJJL$8RW?b#rr>9;^y04F$_nSG-d0eb- zQGWP<9S&S#(l5BcxqjrSVzPFJ|G9nN6w zd)WIN7C-tm(19?rU&HYCFyJ{%c=sdT|9t1L^OcN%sWYI!5a=)kx=eZxL}3c86G4Ve z&|wwK82@^f!H1RaVI6!}`f9kohmEjdC1e-_RX9W<`h|rCb798x|Mxi=YK?~Ee*>W4 z1XwX(g^zqZbeInr21J7iv4~#;<4BGuup~0f8MpY7)jsGxQ+&f?-XK%R>NK>9!2>DP zdKf&;VTus?%Nws49|@uNx)X-ckYOC7zse}EdqCqER%Dn-@R&ej^if|=aihmLb(7U` z(r?CqBobGmw1PzsW`{&&5fiDee{a#l8p(iVEM+-MTGrB*x5Q;Gb-7Dk_R^QX1ZFUW zIZR>})0oFZW-^tzOlCIIna_k~G^IIBYF3k%*0@KXRK-Psc`=pWq~9v-^(|av;RfeK zXFAonPIk7_o$rKaJmoo0de+mP_rzyD^|?=e_S2vL1ZY47I#B zI#Zg~)TTGZX-;*zQ=azJr#}U1P=z|wlxD#i+ANhs^+i#O{_GZ}SYwRp;gdm9H z>LbsDQFdvRqa7t`Sj9S4f3lX>aUuRI9iEysZd_uvv9H~6(lTt$~x!3tN(R@SnY#cXCZyIIb5*0XJutFY<{ zue;*4VfPS354a!*N*tC*&v1@o@d6Dl^tCT&QHUwZK#8^W{{tFje@MbD{4!Oqcn7lN zlI&-dyIkfr*SXJyZgi#l+0goGwCpOaX$|&{G6X?LnQBza)FPLQ{FV#2ib!1Qu`t@6 zw=aNgL{^N^h*!X6y$ShkaPeZPkvt<0LfVvgji^1Uac-lNoo<2^ykG`5*uf8mFk7pe ztLvgGyV|u@V4fk8e{Y!V9rH3p5^|7-VPtYs#xO;SU-61txUrFs zOs;zjW3_G+1r*Bg7Gtb74>^DV5!_JUxSWF$&w+vwpzw+|=K%_zZtTb0P=+#afe1zr z?2{Yu=}+f?3vco7r;Xr-VSH@WKejDx`)VXEGy-GBC<6>&00lwVYPa{0mm4&#=%{%4 z9h4xp8(&cde<-v?5RT*m404bJ!y3a4ZRxco9(V5TZMt8c^y>525`=b-4RfWlQ z;XxnE7({>qN8FHt9*9C7l*mCNC_#iA>~@3BYQ&L}!0%Q)y4Pjsfgq*|1#Ooh4|}bz zDfY@*F1Y%U;@K_41;LPxvz8Lw(Y28^eGBmdHU|+GfB(fhGC_nqG|~;j7Iy(7F^)>wSfyty^5kQw}8TU-eC+Vco!O(Faj4q((yci#0^AQ%2uIgZdp%7A>Ey*2dCv;o1@Vqo1fky}`7Nr8E{djzf(TqlSZebiFUEK0 z9pY(h)Z2tip)Na?yJdz-ix>x*e?eRSD86W zv60J*`{RQAGlI=qzCi@NGC;MsgThP*0xlSaOF;-RTo##Vz#K3IILnKv%a*F6!(ueX zV?@Sev_nk+yKWJ?JyaEWNIDqsHU2w8nmC8&%Zv7lxQgJ2Z@4)wcm=2cpY0pNR~_^> zK}Z28$OLGEfmids6hOUaK#GaTgj)&3P}H?s5CsvV1W2O0Adr=N(7f_XfieIVS?RJQ zFgsZ!xgv~;XEBB#NP?sL22-E}af!QMJPu*R7Rfq+m;7D?E`R?7%Agd=p(M(pG|H7k zfn}tWX4IBvgvL_I1Syk_VGyzUI|o=a#JB*q6o|AWXp0*h28bhqw&=TWs4X27gd}jg zMw`FADmX?H%Os#VXfU*~JT;)xHBh32AOO6zX}`JjF^OA?xH~l{xCKNs%o!m9ql*klO(7N} zd@@tWgjrdOj=6`JfE0P)#Uta*j6^nLpoAKsFWoe|pdhpEke45!FHsnmdw7OSu#s}4 z5ka6Yns^7Z!$you3f;`jdbo%A5`<#OvRc!MVfYc&!hZ^W|8yhHqXc^Su=8xsgaFR) z)Cl+t!Q$}9Yxx#y@PtY5&;JC_02R;yCC~yj&;v!#17!sNT+jq{&>@pE&1gY3 zg}5&rrGE-Q1QHy1hxHOtl97jVXuf*c(ec8F=2OyyDAMNBk$QLsDAiF7b)e0k(J*ktx7UG_IkOw!_%{Z0QIi=G&wbMJr(>&EvJ9Sf# z2+}?Qo9wOVc*dEnHnKaZ{B1 z%rM2uqbo@xiRqGKm6$OJl!^fqIf4@z^_Mz{laSeyH&PQ)60mNv(OM;~XIQatT`>q0 zm4Cz%*A=@@S`pT0StNlu6<}qT3zDEGY8V^hAb|izf2pFA!J>xIqV1wlZcSHjCD?9R zS7vEfe}N%}k=GWY*GP&Y^O+(K+EsgDp4D+4#R=Gn>ehnwSY|ocV@X(jSy+Q*S$Ex9@%fpr<)E;2*@}gii)Ej)O`x~}^_QYGn4@Kwq=ng~C6}i4m#4Lss5RWsoes&loYM7$%n23Gj2Xw(7syp4$<-Ii z6&TAk7|bPG%>@_Ey(GcKF2WVv-!(4V(H-4U2E`pG#x)q$wHMcACD^rCyk%LvMcT1d z*|HTOz!juYBBX%@-tJ8gxItXGv45M+tUcLE-mL$PA-?S@=j~hQHCpMtT-87i zwHMEg7SQ$H{T*H7%_!s*80Cc*<{c;YWf$4?7uppV+pS*Q?H2om7yM-w{pH^Z?nwXb zDgahk^kvw(#h|X8T=%V9_^n;}om~3W7YJq-2|gAIw%`};i;xwTk+l~Aj(=AVo>vf# zT@jvL5?){vj$IVC7ZpYq78Vv4hT$Z}3mNtm8XjN{Hd!2YSRH;|^G#qM_7?`8UkBC} zA|@9j_7x;n;xSeTCe9TnW*1`!vz(D)l%?YKwPJly;4B7Wixpyc@nUcR<6IGAG9F_y zrWG_MG>TB)%>||YF<^8-U>-TfVtvu#6XxP<|HPpxo3RdXGS^ZQt_(N)W91UXK4it za|Rc5h8K0Vmq!lbNX8am&K6;gMtZL27=ecNtSY^zLa5`Bb9e=^fz-c%2IeHMk@jZ{ zjw}j1%9eKNmxk$>9!V4+e`s6MesyYV zk!Wn8=u#o(qc#pxpuVN%D-(l}E;E9`XtW!ou@=({(<_6xc((V;e~WXNmoV{aO#$n$ z*1fSV=QloJwBBj87TZETXQ0kqp$1s(4K(hh>q;}y(moeXd4{yO%^-E8j?hs@D(Q$D z64ZuuZT?QJL%RwKVEXI6*kNCliG{aDzPHpKoZ#S0HdffEmtPZ~!#LJzDTrxCnXJYBz9$ z8`SMkT#H7yh2DD36Z|jU7P7Z!vo-sQ`6_Tv%eN=XNK5m>e~iS75aWUYAG*Ub2I{^} z5f@L&%!B4OPYf3XkDd`8%Y(v-Pm(Y)$}VH`o-_1L@7`PQCuwhOd2e)yZ+4mQUaoIu zxo>O9Z%x^6;9W?$%z+e0f}|L@ws1A!8V17jye+JUeoQkFOM#L5H@U1k!(7C;1GVnB zJU6hFOrr$nfBv+h>o=J&HR(9B9vHme|B|x0OTV{Bz%_d`7*H)riwlj*0V5!S9-uyX z5CkHignppP9{99tJ3C?X!Vc5KPwN4-EQyZe%MzqPgrF@W_;BNBY)X0TDBrp%PnRms zmMbThEJv3uZ|5#QmM@Q%FsGC;2VRhXhDIR5k)ykxe=~+W2!hR{%6EJPjXSR{b99Zk zxy@@dZ5NNt69v_3u}li|TO|UeKE%VUcDc(&%F_xoi!U_mtBgLiw-^RT|1c;Zb(G%p zAb`PJh{{D1btB*{{>$!y5VL+fuYfPo;$ z14fhm6wvj(xCL1JG#J>td}nli-@YF3`?WK5f;aohoAD$l1Cf9RBUrS0aQjj}hmDW4 zf2*&1AUrkl{J$JPLN4fQr--sY|7;7_!q38|y2D5Ol`rVYzFU}wZ<*I}nonrXeqhgb zTFY-r%*SMa_?9OJms|b%$rDBJ)v#N`pxqk=h0D8s@3?R=W{LX9@ zJ-dhSRvscf8r?HyBngxpZj2qwsS{e6e;#gLWy&G)92X*{`kfP$qtUmTqU!Z~RwfC~ zVK3i3<6`8f&aS4CxKRl$NttK$1R0@p@ZGCCL4t_tcWdIes78nwjM~%GH!f6jO3YD4 z=((|E95k(eZWQ;^TWb%$N zWxybkL_tte(K~g01I;_o4CID9e=?OJ3Kup#gV-=hz;zBKIppyUQ6*+##8T(Dfg&{D zG!jG-AzIZ4R$RdKm?&V0MV47;sdWxC^6<2jJacj35m8IMbB`!QP}SF8!U+OH4nf@F z9&U;;HlLV>Syr58oXM7^nrpJzrkiiVxf*S@`KDWVW8&G*aK+_`=W@;Uf0^fWf##Xr zc7)FN-JOWWSKct^F$&*$|LsLusipE6LqrHcxRFO5f*5m79z7WGsT=-D1cii9K=IH( zoYoYAAOH%|0}*Zrl826>l;J59FV&N28PL4)YaSsQX3U3NxM2hgB@P5d8P39>VPxZ6piqyf76s@h($zM@(4l%B?)oyDG@y?3&W`<^4Dn+C{9>ZarKze zLkeXa+8daSU5Z|rIh}ce!w*9ovBVQoT(QL$W1O+Z8*|*T#~*_nvd9vnFz0Q(*_m*9 zd#aq?pI)|n-JmgxXDH3xDSC5vjzX$4bd$pDv(Q85wBSS-F*{6De}b@zQG&*JDxs&w zMAYm=Zg7#{(p(&^<17DV91UtzZs?d$8E)j&)jOGR&FoY<-ZGW6SesJMH%w1d#@QY% z=wClygOzqy!;r1gH@HGmPbNRr^9s^j2;w9qtj$}JEnm{LFYJo8LN^*r;6DGwR*jdSM=BZ~D9q5p~~*RN$v;uaa>&@jf} zpZ_=0udh*i8=B%3&wxhxgma7cbv;r8o)^z3n_EECQe3oP?SvXrU)6}S@AK$%i?2<$3?^>?~7s_V^6%0y2<-93&wNX~;t&GLed0BqJN? z$VWnwkilUh9&_@eT;(o8tWDEW6h;-|!NTzAWW1UQw`0HiSB*MCLP(Vut(C>{k`(2s(Yiw7NJ zLNU3}e|O^1p{a!EMI|aqisI3tW5nnfHA+T~lF_3f-6>CrH&QW{w2~)n=q==VOxl(5 zl9+@kCx>BFPs*~CHXS2Q$EeeuVl}Hi`{@^hipQa{v#1Y6swkCONv4ibredV27^f=6 zs*3TdTAeFhE5%hVdX#Ukgf2CT#$ks2q^^0)*qFn3h*vE?Lu360M z825_Fz8-Z^{}Bx=9?>es!gBGjTudw%7t6)QLN>Lk4V^l$;o8^2*0qa47$zwzM$5ue ztSkL%JvmE8&wdfKSrly+Nt;E}W)ZckT`qGUV-IZjBD&I@u5`n6Og*L&w<*=FUp3oO zf8UM~xLg#j6^Uy_<6059R#a|t+w0!Ms3X4eoo{PrJ5TJEQoCftY;AeF*<}v4l;oYF zd8dfpDXMph?0qkT8=Mo#TG6&&)bAMi+e`m`7r=lLuv1l8U??6KiV23|f*ow)_Ck1y z66PX>#b{x5VtA%5U21@v+2K@vxQQWdf8vOnSmF}{S+yyK;)=7_;xE1!p)p?Tj0L-4 z8$b1#Q5FB?j#=1a7XCQMTCVGmn^XbnV9#vj>d0E~v zow3?w7Jhk(?L_H*xtm!WpZT|GuIQS}+U97!IhAo%-kj^~Xpr7{d3kOkJe)BMX?3L1 z(wHtKJk)WEVwTby!|;YG;lZ5zh|<)-OeH+LaSL@A6OXPYbt=^{j8&)l7Px*iuN@m{ z<@qw!RE?6NiTzM=axt8!E;gK_T*-T!*V^Xkua8$cc&uX|!4~*~@PBv!gxjYF|6s#SUb7 zOpYTWiBx$GtrIyyfhKDFjWI+4UR7eQ+Y4{_!y`WNieEhA8}InXLq7692XaNvE=<9@=&8L3x<$?Zp5T8y2-}eEa za#Y*Wsf6^o9%8^A?#Q0+L0|+*-~>{CU1!5rWsaE~e1^BQ>=Uqmv7(wY}n}TFQ zBQ%06IUNd$f(ISOhMXWJ7()mRLkxDwJ4VFmmwVH4gMu{;A4Hp2F4QTJ`31`!8TyaRI-#-4OvCa?#8s+CDS zoWm=ipgV2ID}={FS&08JSm6R1Mg!K)17=_!`r#h}Vjv3QAQIy40iPg5A(CW>|1pgl zBmoS-K>2aj+UyH|K!F0)N)H496ug3C1jf9~NiW@YD3wDJl1OXSYhXUcCPiSM_8H7T>fc6!O5yXW+ z5iF!L zKFj+AK_0XrU;5TP&rlIOR8} z1i+w(2TcexcnA?V)nK^a_J|+(m4OHGq#U7$1`S3dn22DQ1wkr*rB2XdI?mwse2W`Q zWfD{+ZU&zq1SbFbu*x$GM2D~qgd_>(A!p>|BovexWW43j#ARM`XLow%cYc=y8FTfMtGACPpl#Jjg?{h=N$4WH;ug zPZ&hw%%(+%0@nP0iV+M$U-ZCJoKX2~2a#|kB{+vtxSun^#NW`-Zsr7rP^E=Y-X2>v;jeO(u3z=Oq!=aC|5k}Bzv zGU?)})>&+#w6qATT;kuj=V8=`O*~b0q$f#2U$8I@L6lE_OO!z))QV*Ws8$lF5roJX z=E1&zo+&J-|0(1h>1l;Ya>0apiEF}UTHH+zXy`$3D86I@jE1OVKm#Rwh*plDRQAJ& zw5UP^s-C#Pt!PV8F-VynT@pSj5tIrYc?6i&1Y)9-kk*zS6)E90>8i47tGeo|!fMK` z)Vq{z3t=AT9? zpe`FFlp?2iQcc{8#7gYM-sNg#MG6>! zI|+g!?Jj#Iz9ssnnH{a43@rvAaIaNIE1k%Dik`EQj`H-NCgpi&kwpG zl^BGarYpB{0VUK*wPc4&#} z07JC^`;Da7_@1BaPezt7M}N>*3*1>G*VODmJi{Y~B8}j#U*v&z)#JjdQ^Rgu;EM0~l5hE% zZ`dtY-={Jf*??T z!Y!=q_O&4ph6?pW3H}O#ARv`#u5Wpu;P_+$B{+#2k{(g6sL+~1^L*ea^>2p-3#YPg zsK_q>pMw#asSKi;9TIRmy+U1>Ce`FbOtde3FysUyRYl@O1<^18EAE$gFG8W}I`G0J zIPnuhaTH7O6jN~(Tk#cRaTaUw7ISe47klv+gK-#(@t06T1{Ht*9Lw<>({UZ!@g3uF z8Pi#!PH-O+F!z?xF!*sFzXFgRk1?3(Am37Y$U{(YPIY9b(D)KQXu~#G@+D((CTsF0 zb8;tp@+X6GD2wtalX5AW@+qToDy#A;vvMoD@+-q~EX(pN({e4_@-5?XF6;6x^Kviy z@-G8(D{omufDM0@y6)f6M>kHtHZ8g5;1KWr=>GJhjYZm2a-U;KJ)W8T=QD+b3oIh zl~qVU?{iz`&Y0Oz5|b2u@fAabRbv9xbq-gch1aR25<`DTwEtn*T}CCEwULr2H5xvX zG@;ys)gebD`^=;@QJ7_sLU$EKmzPD0k$7d%fI(4?u~3yQ&!QdANSAa_vye*b3`?_+ zOIs04zSYR7a*G%GAG-h z#Z;j|)j)qE)<=sHRtJw(b9G&hj#twRSSycMLy=NX7E|Z6Q>XQlso6${R!7$qTu)O- zSJNrk^<#e#UdN1HA2LMq6&$ccf-*37b31o* z;jK1b6JIxxUq={VPc>&-k!M2@XfKaw9}j60k7@4?Y9se~qsKnj!+NuKd#|@Tz;;*3 zwouV_Zu$0g={wrw}j_+ zQ}4Hh_qP~nw1)xrTkko7wYgQjd7LZyn9O;l)cNSN+Bp9bJY$lr(V8f|^QEU+J*S#8 z+;gR0S}Lu1?Myk*RQaN#y23{H_wornxcaNZdaTR(tkZg}+xo5JdamzJuJd}Y`}%*c z13NuTNU#%ou^T(Df5ST*`?52;u_t@8LwmF%`!`5?wOc!^@A|cCd$m*hwtKs+FW{(; zPN@&gsi*q6ckikzadB8f3?0$C!+X5T`@GY8z1#b}<9ojA`@Zvgzx(^Y1AM>>{J{Ir zB#=X7h&$1cyU>_Bx;y+~Gb9~2p{KtcQ$cy~QlYGgW zd~BQm8zck55BNR!7A^xBmv~^Fk(FZ`Bt9R}hGe)5Tf^>g z6+4!!M0uu+)hoM}ZCkf*;ktb$NYJ-_(DLMY*6$moJdKQ`Fj7X8!-)4x^`P|bohUb4 zpb%C&Oi=$LL6F@HlP4-jiX?;;HiFDW)WK(s@^W!wEK?pzZlG{!WH;m7(W2BgE`Jld z(bayhU_;3>tlvA}P@uTk3(fM?bD4H0H+8PxIlq*4bIC;go9Nh!w^Fnim;76iBoUfK z`ILQo_wV7ymp`9=ef#(E=hxpd>Qs(ZeJfAEwY&mrzyrlPD=h>WY|z06$9kp=B8I6) zlpJ~xrJq~4;DVAQ1S`W7!yrOTF@Jf!xy+3+dVqo?QHaQa2;_3e;~N-ZNWzU*pb>-! z)rc_SmVSnT0)`;C5X6`~V8|i19*ocsqjwq!LO3^Y>xYscbmSt4M(Tma%QD;$gg9f$ z$iWRa|MViV!)NNDB*-=c zGIZ4k7fr8GO)0L};)^lPSmTX3R*KXAJbm~il2FqW*lv+W z5E%a&8I{+>!29NwOaw`yhhl?Jt{-DMv@4GsGEs}O9LmUp3m61Br+-K;(0ixZ()#gH zkc_CrFppOp8penmxRBb(ebH{@nXn{;!Na%nHO6cx_WEr`V+0POw0iQ0Ql3nL=)sK< zxpG)#0VD25Jx4o^T=K~&uiWy>G1uwijR0KyK$1Nt@8r-)FI~VQjcBBYJYr+L40vyD zWY?WfOffb$k|4Gp7=HzE2+eixS(*}g;>-c* zl8}j4ltFjA^Ih`30XU|qg>Xj5#4vDpl_I&JdE0^>LhSc7jeKKfIY`1Y%=RT54#OA+ zsfRIaRU_)i1Bx;09n!FNx4hlL2=0U5UgnnsDScyuP`bsbu%b8$ip75uJK!7P7)LqI zk&YEhU~Lw-Mt`w5P>))PU>^awMk>KTZqP6V5j4^OZV`)^$uBm0IVfqWCR@#V?6HJqu)qk|+HL;mZZElmB-Sp-+!5L0* zj+30_H0L?dnND@Clb!8!=R4sUPkGLhp7pfnJ+P6g4%Bn^xJ3}C zmP#n3pbUFy70yUlB2g&j955oniV)h+Oa9I-goA>Ty2GVB2qF)K*NnIP0<7;L(Xv#9u{IJHPnGqO_|ig`?zk~9fPFiQ%9tRjqDSbNU1sKc%{ne}>g8Xh9=c(Yh+UZ~-h-Dnlk5(iqa5N@obU{|8P@ z5hg*vENUIP1#k*;7rIafFUZLRibT^?D8O||jejr?s1Wu+~j6Kxy^O%bDfaex0Ud}0#s zx4+sth(bmz7B%L#as432kAWOyArG0zMSnK(k&&EaB`=xDO?L8=p&Vr?PnpV9w(^y+ zoMkOW=Z-<2Elu9dQ;hn&T`iZy;LD>Tm{$+YAqHc;l@2g~u}%9p~a0qtFCpv;x7J-+!^j z_r97owS#V98M@f?jH)uus#o2J3WXKW^%b<5)7M`$591bj=CLS3U0*Vy2hR1tVWsi= z6<_1`M!L>#=Mv54HkbO;)gI72>XB`2*Mq*G!uGbajn-Pn_tyCx#w(nS|6g7&+t$F2 z^RP8->{-KE*_v*4oS_ZpQRkN0)_?Z*zsWl8+?E@^xsLO?HC@qQ#~0plmN&BNTkm&^ zBHi`PcboT})_wz=;~np#z|%_b`WhV02#>SE-~Fw4A3M$xhx5JRd~r2r9IYGo_|0)% zI*_APGo!N%$+W3sOMMeRd@T_A$0YyW<9N2H+R=h{`EFDyx2!?v&;8>_BP90 z?N5#Ro8g{##gj)J*qHR=A@2=rIJ3=kN2}e{jd!8zUCn$KyWiCe_|E?eUd@EZFU`r? z_QkKB^&7Fr@-NW9PuTiT z_&_heL=X4|5WmzfzSvKy5^w=YQ1KdYpdOH*A`q=6u-*1At%5JVhOhjJ4+cBX;tCK1 z50KOluLONi0Z$O1QgEPFkgQse+$xaaE>H$DFxdhy@X#-}J}|dJ|B#?Y@CUIF=Yp`G zh7h36Eea)2^L+0G(SIri@oNUv>=mAH^uQ3UrZBClaGx&Vc!4dJx8*bqb)r=AqG1?4|?Ru~dJrVBi zFp#>C3lZ$(N-@q*qz1_v>aUG@66Uae=bg`d!(HH5lzJRfh{3i=qfUEeiAN~LF9|1BTZ=waVz#8`9 zf#UI*=Ft;{<$o4#feg)K!L%ZkAPp<9O{@wc{_ata+6bRK@*_bqBt>#0NwOqO@+46* zB~@}IS@I-NpdcmDa}qL;7LpF{K@<$;R$7rQZedV%@*#SXYqH`V>?R6(%CX>YBiD$k z1hOfe@+qM*Dgg*45poCltrP39BgCW!UMjE{Bw6y{Cw~#*k~)GF5kiFAU`R5;6m+Bw z9|A>UQGyOkEImRgDTpbhGB5RVFZr@B?Sm>AXeI%vCbQ5Olm#pz>LbDpBNh`P!c7zc zg+vC;xJ@Eb z^C24Z8Grr)(pEEZun+f$?SK68H-R%ag>yJ<0xbl;zk<56oRIA5aJm!;XM0`h$tZw(BqPzVHom8 zA}mNvc;ty>%dU(;6dK`%&O$V+VkeA@*wOU0!@fuOM?JO zWPeH&prC_3h&qa7H-f;3hUaKlXtThlcS_S>h~QZ40;fDEP^iO&a0&{fBqBDW2U0*r z`b7}RU`VV(XlSaTLX(G{0e6}vlETzVwj&dA;7^T!Q1}!lgJ4V9YH-3dcNWXR)J15} z<}tv>Tlk_w3`Rzfr*P!-E;wjQp+J!UCx14CrU%&SWeCY|>}E~J^tSegHyLF;igZ|s zwOEanfR40ul5~xhbn%KOEQ=%smPKHQ;1#IlR{B%7L_rh;0ik%PNN5x%i9kbcAqV89 z5!`?)8-iL0BZ_)QLWtlzxxffaVa|TU6x7gZ;=#3jl6d7N)?~_mM>BwHj7ek%A>@Q%8J7~m;!i;6CLj1@jYhOc!h(ZQfLDTqSMk)R2NVOR@}vyP zMrME5G3~c59zr_HAb;sP%%G~JZp_B1>Oq7|=|*hT7#IV>>fyn_6b4Ge6b2(*^)wd4 z286u0*VxrJ4&rZL6|S_VMr6vy{yZWrj~EyZ1VrAPzt#eBcE(=J}9T=iH_vQRl*lM&wVD07lOZ8tOq(2NnIUB`yC78nm}k zjUi@}U_|D4oPPy@+a&dz2V*bPnW6uXaFS>0_JBtnosr_s(!V0 zgSDy%x~}c|uAkDNQwE_KNTK0waG~aHlGmvaViZs4g2tawK7z0e51ZG5`I{-y!um&-1|K(TGWRWF;l7DFQp$UdBy48rb+I!BKa2G6Q ztz=JL;jW;dMgA9+Ojm3VvsE%RgHi{<{-kMkr?hXz;uUyz(1VXo?tP zza5MzDB&u+f~5Y!7%U7%58_6ct}*h0L~m#_DI+gB^NCQSz}NG^?;=5UnZFsM zmaWx%MdIt;0X;{G;v%%w@Vhhji^N^yDl(I#z>~unqd2~Mq%y%SOq8Sqf$5$IMz`~Z z;AT8)Ilrsp6+XK-MB&CER21?8c?JK%6x@v{Z+~bzQA4exTEYz+lCy7Uv)~52{L8^S z%*A}n$-KAcSE{Lb+_&-HxI`Ml5l{LcYB&;@hHI{dv(h7Oq%s;I(vg=8d)b5`f_O9n(;d@<8za&^y)x8tHfd8H9#l3p zoqsl^9MU} zJ=v9g*_plBo&DLNJ=&#x+Nr(Tt^L}uJ=?W?+qu2lz5Uz4J>11T*{$Io5dEMPJ@pv9 z(K%vCAe+xxIdmSH&+d#89At&w|DCZB8-KrWoqG(=72dPl#t&r%A{_6t{=?6~f+pGw~>Iere3HdMT>kAEa-s%DV2GdXL z;gIX)(CfiI?*R(sO-JPc$?RR;=_zo$rOFK3e(K|E1DQ?krI7B?>b*@TzV&|cRfX?M z$M5?H@Cgd-Lr(2sa0#V8=bir$?tcRb>j{wXr3&&(NAfAZ^##NpU_bU{ANIvQVAb{`2* zf1nIMzTAHF&2IPys`x2>6PYjhmp}bM<@IQ<{cCUflfL?W?)tfo^n=m!%YP8`5C8WQ zA0URk>qoGl!Gj1BDqP60p~Hs-zlk~rv7*I`7#}_}1udh;k02esyEW`y$df2N$}?rG zUdop+W6GRKv!>0PIBV{MXRoKvpZ3)G!e%e0(TnG989U0fInHnVDw5?;yo=qEN zVWPE>D$SkIF>i~Ke_OOH9HQpp5Iw_2zECvthNYh`L@j;c>gow!V}IYyy?f5vhrKKI ze$Y4he!OXZs?;Fj=an4o3+F zx!{oT3A7)Q+$qT&liN8N<(qIaHYImelIM*#@ThlZp6bEl%wtE2N#2Rx8Oa@*+o`!7 zo7lM-=cACyHRpD9_9KrqxMZs7rkr-_>8GHED(a}DmTKy$sDGxa>Z+`^>MEzf5P7Je z-Vs`)nF-Q&o}$<>TAibj1}kh+lVVq?HEgtS!?Mga>+G}8Ml0>K)K+Wlwb*8>?Y7)@ z>+QGTHp>P%VQy9{nGNDPAE0T1sh+RW0Xyux7|L_bWKtDdU8Ur(al*d<2Q2Ww1Q%@Z z!3Za;@WKo?>@)Dg5dTLk@x&Adj6xRVl4x#swsL3ac8FrfYjyBGCojDy=O~CMvYrDC zzBB179l!pfV3XPf8-F&RaB;>!Zp>ZB+kK2($kmN3UCGRyobuP*Wnx4g@*GqTC2qWe zlg!W2>@?GG$1V5Vbk}Y7-EcdNak(aKdWD_mRX}g)GSm_TpiWUA95>WRk}% zYa13Ijt2GnHr||f?)m4Sht9O#P^(BQ$9@+IxQc?aNce_RnI)$&pXDLa!XwIJadau^1K7hH?ZkmPxHPT19L%Q|K~BhSZNt^%gehL zB>gx2+yjm8&wRu-b8d?+|NQjVZ~xMy^DX3O>242l>q&Z@i(;3b#MSPA8F_{vpuj*b za6uFU8HO96kbeX&P+}m)C_^45NQ4Kv@G|e$0XQ z?qSDF+=7Aadxsk)d5bcPO%O(aLMIy%#o|m6i@40CE_eAeEk=!tio=f8s)IG@Xze*% zL(c5dh<`?7Kp_Z}IL9{{kpnr3A_xy`Wi{{diX7yi5zi1s5}@!5XmH`1#-L^yAoCw=z$wt=7t`;;u%Ef0XWYQ%zju>pBn*WN9-tyZ{#5dnJ|kH$}kK<`tTO#bRtG- zDV$sK@}d~csQr9tx>;6-YQkLIk;0fL0WOJvh<`gHGgpd`SM+c!x9EW&&LM~#5Wx+3 z_@f}|p~o_`FOsq}1`%?g!7>ol7|?J-4%|45vfW~wuXsfV|G)7KqcZi2I1FSzp3%pF zCINz$bi*`*@JKscgR&gP9-$D~7|Mf)-sHk9?(q#b1Ocbt zG)XIakOMB5VjvI6h9Kj-ou);CyVHfvQC0SmIGDm3rQGD2M57+1L% z&M%|Q=_Do1$s0J3E$ z10{Hx-pls5AFoiWWVt{>h)UK7-+!bCO<@oOQ4l1OlFg?a2~0Kv(>B32CWtvq%@}!rpn;o>K6%_C4}Mg%?gWu&MaRXVU4aIU&4SuHY9k7vy@$MnL-97Fd(iCJw>e&c`yo0LekVieBA&Q>+Tol^Vc|lZ8 zka`Hc=qNvky{7k$!$EQ2RDXhN(Qz?}s$2c)SkJoFx6bvhd;RNR54+gMPWG~!{p@H@ zyV|ji3t2Z8m}L_2Uny=(i}woSZPNJ0(RdF^h>-saNg!(ba1!Yx;5#Aj05k&^4~0gc z?<=&XO{GpwkIGyA@|e%O<~Psz&U^mznU9|C-XRaroBs5uPd(~~{C~32tN!({kNxMP zCq22_+2MPIc-18SKQouu-6t+%N&~+4uff|5WzeKbZIaGRe6b$k{JAJH(NwZTem0Zm z&X&4Fp6%~P9j|!&FZV}$2TVD0&Bf6qq6Z7aIYH9Usyi21q9em{$tu zMGQ!Qbq9PGCwwWkd=7>^1mC)v0!+qfO~ zSRDB%Mf-Srqr~zC>!s%kR&-3^EevyIEp)R9usLx6)9}}*oObu zS8zCDj!2M@*n^TNfs|O1KB*Tc2^uMh91$rTEm=`7iGDGO92uD#8(AA2NgE$I8zC7R zBl(k9X@3(z$r(dg97XvXM_F7+`HD-qiZdBqH92=TNo)mqT?g4U37M60i4|Lk8C@wH zU&$L`X<=i@jAe<8XGv#iIb8$EmN{vK|2z4EJvo<*$rN=-8F%>`d6^q~IT*>nDGiZ` zAh?Jk*oZrLmy88ZzzGY_eOD_J0iNE-&{G_=VOnt6jxSbnY8n*#Zpu^61j zHk^f3obd<^xq}$j!w|#6opwQ<;s+Y2$r!4c8>~qit_c{?i2~EP5Y@ScAqa&!xGUbt zet+PZUE*1d^3VtmG#Fd)oJyAoP;d*_X&3K+pq^$L?}-@kSsU{?8}<1Y_nDsxv7f2f zpCR_2g9DmwDVhjLn(zouL_i68p+5i5a23zM1^AU6<7EovCx0R)4@iX*@{k8c0HYR~ zoNcq5G1#0S!l1J8p|yDsB5I5z8gT)Met(4toG2=sDhih@T8%DBqy!N?ZdVUU2N7bL z5j5on`1TH_^h9Ik5(nA?I+_qZNLq$U%7IK88%|1{x5<&YiIKbchE@8CSK6If`kh-!oLx$dUw=vv z-(U)Y07Hi;55v$1FeD0`tPC+ zuoPxNtAYTbM}}!H27E7cR)PQ(|Kaqi(|Q#W@v4-N2?ezf_h1Z?14G6@Q%0~d$~q9y zT1SmA7CiSnv`PtsXBMy82;@2z=YOE9_-Ya(;jOoTs}6CY3OcFwSgA93sjZQz`njo4 z*{QkFl(q4cv=NoFF_p4Wm924=EV!!EG){mA1vl0Yt<*+<*8^;o5Flg(Jw$270BBs` zUy+8VE_+ZgU<5T(qxdC3L{LFo&*{#dPgFJO@}wNWuUSTL{mLbK}pa?5K2=H1O;eC5s|idK|lnBmRm6!w3KiU zN4t3K+7S>HvqVc_kt#(BtCIifNtq6t8W0$d8yg;*8X&uZ zUwRLwkWY>93N-}k<_Q7Vp8x_b?bj-%>~0XmBV%4Y?ts)8^{0h|XPWK@|D zX+~f}qVP@I6QnSpWCn2$|5UqFmqdqEho1cB>J-*gL7=3}cA zwg{nFWpEWYN1n=?NPpCXUk24*3NZ|7MM*;kVeON`qb5x}r?u^)M`3nZBJ>QJ)oT6l z!81h(tfo9=z_QZ>S>Xq%2V8^+ET0Q}hKXw$4{V#Iiko*#8<9I3lPepQ8ylC~8kuVv zn_Gb{szl05r111j`o#rZ5mUEob7cT>vKBcokOD~%z6oJYNq_KKTcub@5lfilRj`6v$ATEhw6USG(V>3P$57h1l>4}X z{HfVl!5n$P>VKHQ(73_K*1@>=$fm1LVOmp-P{|!p4=J3<=0n2xB3?O=0;J0jq-@H) zdpoLZ5C_U>tY$;EfN727NKlbfL_o|1fy;n|K1l^xuc}Yypj2#Sx+km zoRws=WKKp1WemKvRG5|m3SAH# zon){JUsM|e%JaO@TM*~)#w|oO*Oqx(5s>V6$VvJ+Ec|r zUp=;D7X*+cW|y@5A zpaGk*xi=B}$DCT@t+Cv#(cG!=+@=xTqcPo_QQh#zo)bO{(+70Glf&#~RzwSP=Me4D zJ`gkJbL%QT|4}hx;{)x|elP2B3;!)hN|6@5({n7LKFf2%zw_eF)q0S(S77T~@VE<=B>AW2O%@6rB2_019LQ=Z@65wdFU8&Ob6ap%XgLM|aa^p`+(k~GunroRxha|1!Shepdsv-LG1A7 zbf5mHMMuQr%0@&GRU706L{Ie56n~~FXA)#y55^z}MPKwWg!KPqHiw!%FMK$;{Y0Ps zuTM|(dEk7O!t>EHUpjZR{ED?ll?h#sQ?FnWMt5{pU(zK_be8fs{bv%NwMmjA>5v_! zqz4;sAEEyU0o8#a)w6Nqt&!t?LFTotKi8Rqo(Z6~zR#x`o~b&Q2#rGHM1Qp3R&I&r z27wC^P4D@MMnXq{(8%ZcFBXLcrTM^hw4|>=ZEq87x_A|I^rxxn6~*da>FVBC^7=UW zj*IK5;qs;d^P?g2ph5GTVe^!M^OneSj?@UdwI1ws{p9%9 zkO_|TtZ*&<+Q#ouUX8@swSVBSTSvW)?RQR; zO0r(Ry|S9tt3!xxyJi$CS87PEcB7(1t2gS)VKMy@wTW}^VZ@0QFHYO^DPx^f+d|f< zH816sVC!OLX|^xtm1`%1UI}+8>5_C&v&6gFBwyDi0Sk`ZvGD2IxpnX6ebF)0-V`MV zzi62_Ma{>BpH(hVbboV+rlBuXJzZgI>j+_IM>w0i!Q0=(k0-zLcloka(GN6^KKXjj z>z}Wej=nm2?B%zk_g+5ydH@3y5TWzR8xFws%6kvJ`0!&3!1~I&ue|)o+poO-$op?V z4m72fal9PG+i|=e#rsi6GRsuVL~PPjbImkS zWY9?#p>!`S02{1w#_?(#Z%gsI{O(Kf!u)PbGYd7;Ef5n`G|>~8R1r?}S{zTx@$AIW zO7!&9QX>HUaS3HBWL)bcnp<@_2hns_VVwPL(r(%vT{w`zdHny(ggg-8L zT?yO8?_KmxZZ2izR(AR5dtts=X6|U7&Suicv1V#($XN(yl6B@TWa~zruH@!~R*vZ7 zj7IuwVSkmLZpBFbbrIm*$RT71x|tvZABC{~TI=k%p3ZCNz*bJ|LH*(lO0fiG1ILB2ug96^i_rJ~`-`X-M?J|o`ECV$V)Y3M+WGtGP6=R&8y zX8CVA*K1Dpl(RkJbT2sH<4yR)N0N6?A_tT}pIq*7i%ht$AM4A52n}+-9?UN~uMnMD z@PCKE!xT?B$LrT+EXTOm5U_IwlwJ$br9kK~P&p1f4g`ZE!QN1CHx|60N93VF4hrNP z#;D3=-tmlK7~>nkyP7c`0u6bb0~%s^#~2TC4`a9i3TL_9EnabydO+hL#&`$(|AL^x zZ^?8iH}D3C42g;3TPNH>V$9yZB?F=B$yfnGDA@DvI(Hp>5nGEjn$ z6yu+&0tCS32oN)|KRtjR@5NSNNY1G5dmJ3FB1zGz6jWYDtea6rxH0TS1JUH?k zx0`1pe4Et+Q`L8r;A|ebkObNqc(Qr0EDS!u2$O!Gg%aj>Vf0=mtl+O{1>Htbgkoo^F?%Kh0TC zl`~Z06g4O1nV*d8-k5U&UV7uxa)tCID85XKEO@&E%EhX#k z;I|Q6!0Hr5+uepHL4U6G;BXKGaf^}UxCca_U66eXT;UeB5im#s`xMJguHqNh9f@*p zru@Gu*X+ux$?_~^^xfL^?8$*6ub0e=A@us^CWKxyONbdlutv3z8l+}l=Za@Hkk!vK zK8PuZkb)%q?i>leMNAmXg=gF%2T(XA`~VFDBMameNq~W9p??5{!e%-No0hDnOPqve z4x6F(|!y?$@zQ6GoW#gb7)#2ohI=;UV+=W zhGgShfCj3e@QS`;n#CkXh{@$Dke9rf2n;RDD>j=1jmSL5O?GJxkQ@0&KI%JT}U0TLII8N^k)OjOy-B3L@R(B27vww~dsM0#piJq@~-H?g+iIuGK z5XG)VQAaSs&#R4qEpW+6)$1pk@ej_bJSJICw{keehCFLw3#rw>o6o@g;HsT9PV6xn zV~nDZ`!kGT1h<+n+(Dz*@;QRIv1hmkXmBJ3%Yj>v2mOKxW-2lGa5Wdg1T01~Ot1^HzH^`k#NP-;TFCri;1A~UM;;d*g2CjmFTX?j4Hr+*~qffO(TXaFQAD8eQ?2YKMWIt!#m zn4-8cz=41UuX+af*+cn)0xg0jKD0AhgMq5zxfH0Z`{A#z`YU-#t41^_6HE~ld=f*n zvTfTb>?$BFqbn|3I&5RRuE`}X>9#-lHXaE#4H>r|oQZn4{{_#ghhxeEZCa+b*g*E7 z1b<6&gM>gMLC~gcG6rM>uJ!pvgs>+rD5+-xGpMp}Ra1=*M2&8DbK0m`d>G7s-pgv`C15+@Hd+bJ2hzNPorhCeR z^U)}L^ek@7$5Si_pxX_hn~vEdj@n}k7k^|$7^EH<?o-MC2TK+N7y%*Ch&dKgZ6n14Shl1z}KOm>;bBEn2v(ab>MOfm7y&x{Jsb4G8x zB5`6L?sU%8q6F^b&U7S-*$Gefd83&yhV2YbcH#`(%njab4&USr;0!wC)Sg-jBJi>m z%fuD$To~vK73mxl>a0$pC?hJONr8|D5~&9UeJTbW|B(lkiU^g^ooG-I>3<-~c+c91 zPvw};-KbBCyw3;9Podh+gXzy{`A?GxP+l3(Q7KRZol)rE(Ae-$*;By)V!@js(IPU@ zFGEq0QPG59(XesRYk^T;kx?3L(%`UB*T7NY08!l#(cSb>qXN>V6Vg~@PNOo?!9h}a zQPON-QYSsr+qvQSgh zVAI}k)7pU34wciWqto5v(!U|n@hZh?5z~etQ+hE|LETij*oQ^=R1Jpbq!RFQe-trW!()s%?&)Q z4Lx-YK3xqz1r0#OR(jo`4MROl9e>R!9$lM&W!Hhtje?DhgLRFBMU90ejfTaHhmBa3Wwv~cnSHH| zenpLcE!2%g*p97@k9CcZU5$|?jgke8lckK5U0I@~h=~P_iY?WPExL@AoSGHcn~jZ} zU5%YZjh+RKpXH38g^ZysTCaVsmR*{cjg6Qk|BacY*r(;$sDE{hsYQ*dC5@})jI70s zt!0d^{aU=G$go`-rhztS#aLr?wzRd{wQU?`9U^Dd9$AebTAkJ`6gR`jTgf$#y_KB4 z9lE8Zy{0vu!G+txMUA#4jkg7jxW$aQrHr~=jJus&yggdZNLp~sT#ek^vAM$BQ%-H` zl*K)w#;x7Q&40znWnJN23D?C8*yUH*O-$OQ44`-i=Y3x2jovyMPN9H@(ZSu*3Ebb? zU8L&W!lBlpvDV=o-=C{o=u;7mDOqDtHcTH@eq;v{iVuCRW^_I3`y<3{D{Y z4KELd7eQ7kLIxj0ju;iD1YG_{{>yGlieZtl%#YCWbEJ`OWKZcd?Q+QoYuOMiP=G{Qj+GyBjs3*d&_y#!o!>t0lh2si|Iuko4J4hRQ@ZTN?@g}Vq zE5-k_Gft*66613MBc+Az>yFz9Q@||9V=@w(G$?4MKZJoYAW{?Gr$m-)76)nwWVHF2 zK=ZjGGq!Qgf{08IajS4~0Dr6hqhS!OAr~%2ce5Ge9ntz7Gbtyjhkx&N$N%$A69j*gwRsoriyE?n$oC#7b8&Qo6i5MiR|tXMN=JVO zgMZ<3uy1ohuC3fJTl?mUc8G^_bA;l#_v)dI6RZC+^%fT^2#l=-Yp@X_`QUA3Vdac>=hYZ-km;wLC}y(N>5F6tplmh+bxlUv`bM1%HjRB@3ZnYNGe9=w42# zUY#*_Vz(#IGe;VE;3MBE12M9CtJwRLr;3N83QUKRC_Btb$$ zFc@HML(hb&G6F%HsbMezMmIeClE8pw@l9IjMtE7MfikiHe?Wl05tM@UO!W{{ zuV15<&jexoW-JMpXZHjdX_T;0jvh4;5{2fbM;?zvmoi0Y&lfg(O{Y?=YV|7CtmkeS z%j)$j*sxTMy*elMEZVe8huwOo_AT6g^SbNVQwE&R{Kqq*~*&3J~K6PGia~xBHs&bx~e==#u}e)?HZx6Mxt)H$x-Iq zw~K;?Sf0?SNnT~@K}3Jl$TNAs zP=+Wv27wt*J{>fS5h?sgl_){@MhTO*$P?C6QN1>0KU;qpM&*=bWrk&IY{kc=T65L4 z7nynW^%t0+5q4#op((~#o2EtfWt^H_wq>1_b@o}FnvFJ0X?;?b+G>ItD%LGr5Mcxn zP>9lxJd^~{Lm9&~qNIyP_y!Gai-v?$JKx|Mcsvc^GqxbZ#Y@1Fl@amj@Kaypu^1yq+@*7eN@2gp9Ze z(nAn#Sh^??;R<^YG`ljIY(JuCs%^P~Xp}@!AC*zYNY8|H5*I3&!s|Ix z5j$F%(s#j?aV|oV!r)%6cxDx)REHvFrN)$L*)e}&UV@C3n0A#}=9weMwWi8~z3E-c zYRwrlV0Y3C*Pb`S^(U=5i$y5RK5KlLAc9bs88nm#LJ2e)7A=Y?f@E^eJ7m4`3a~D{ zV@g>SiDK9}v$=Kc)uLSV$~*6zv-aBWJo!u}MK^LwXQC*jp(r4-&GwVVD4hv3mjMc? zrA2>Jvf3?}_4Cdvl%W5uSR--|)3?%0$7Z-O^2nq1-fG`cwA7ywEr@xU3{$1pRbD+d z)NUE`wLuyhW{m0ejBei2(7ev3Jhe6ARy}*Stx+b87;n6jP(qcl&|z`hJX#+^AC}0| z%aui4*=xT&_d}p*{Z%Z3|JBRmdo{E9S=oP_{#7}%e-+Q~qo!KW@?&|==1(HD`DFK` zZIIjgkvOh3qFSa16s>x85XNPKYu`$OF`@=K(7obG^+=okf+ml`O^|eExuE_Oc)@qD zrf2n_75}`GKZA_$C1LAe(N;4;6aKFs3Y98B#Vv9%i)_duA*?Y)gotm9L@A#c5rV!n9%OxN ztcYjcxIog#5omIZ86~*j97YjLD}rzi7pm2WbL253hUCKUvhoV#9FhxtoDwwt*hnsz zqJD8|m2^Dv4p&WbYH`7s9G@shDYAdDiqj*9HBza{RkAXb!AK=JqaE+4ve@Jy52RBHR^-v4Vv@%!&$tCNqe(XltxL{ zajRavD_%ziR;G$|lwFgfS)qtlj6!UTY=xp*p$Jzf%5{o#g`!>aD%r`N<{p;4>}Bn7 z4|Oo{p@khIS(_NxCJvUCYPIEJ7An8QHgT~{Y%CNX+r-FHwzjqvj6Hv_p^V+~wztag zMK0L6(!Rd)|F3Txt^G=y)6?cNwRdE#{akB5*fvqN_M`1>v#VXP*h3xI_%3+EtJ}^F z_p?(Bt^JTIKjl)Dxlnbk9HA>e>B^6~_Oq`1u)E#;^0zAX_%DD1{9n!r*Qn-&B6{Vg z-uJQht?o7BVVfx5_tAg%efE_fe&46x{&Kj(SJ~@XiQ7`+mJ*3~Y_MVQZmA< zF#9h2z6`T(!ydjdjt>H2!HPJI34Y&#*@xoRs(6YKw%>%=2jld~n0+)(pN(@YWe)F{ zv^_TCkJ%UG^bI+>MTX*$SL@tjI%a70O^RxK?Jw8E?kI>*V zG@?U&S4A_J(Fk_*OCh~dNjvq+O|*3SFg-p^gOAhT^R)FqE$UqhmDEBubrZ9kzAbl* z9^^PSvZ>PL`MQ64&n({On4MZ?UdHi`xV9qwc&+W*`kKqYwx6)gN9;q<1B*;JcNQk0 zY*we0((i+{`D6_~T3gT7*1I(P69_MRTTo2PHdzT+EoqFKj3UYUhh3^ce=Em40$t}}m+58zl9cv%VVGlL_1 zObd4+|GNpR(Pwis+Pa+fKdbFaIs!A>Ij+a&C}OC37y}yHT*~K$7?-L+1EJ4o1SLcc zmva~coB)hksj^ZJQ5x||JN5RdTvTO=ByV5+rsn7cf^qmSlphO?2 z(Qzz~0t$brO}+}$H}cSkh93kag-_@?s7amQXx$*`DTCMJBF^8y>>NCiI#%A1ha_N# zW-JGv%jI11R9Bv6ZbyN_89wQq!zsRb_vq^5+INxuouq-k@!+L%#-^ZLhhiBG(+raM z#V4`xGQ#2hUcvk*mLU^#>e=NzqD0G{1?fEMydZyuV-@a0Di21$&<)AY6eU{n9`!4J zQ%?VHz0@~4_c3pM%w8XRoHO3R4^LmjfzQNEPsKsc#X(QT3C+ggRqzRk?Xbl#d;@EA zQ`13%{SlE(M8-2LgnbyqJ)8qHh@3c;!G(Z`Vn~ZY$OAM$;5Xa@;VEASF3zz;pbZ&A z84!O2h&07$q?7GfAg2sNW?Y^z919Aj;4^W>WV}Pu=#LV%U^Nj33$5T{m>=jdg%Uy& zM7%?g{GbFH#5o*c5jJ547U2i7MFnQY1JX@=a9w0LAy~-Y^w8g_U0`~#-OcD9+{s?6 z{h#>+p!EzO^b}z89ANV#pwBGeT{R%2(1w3{KtUdyPCtADi8x0Y^oQ0V9kjTJ8_Fkje|fs3RBcsyT@l))_oPae65v3On%kYc6Ok1@D`^+{qG z2tpZbOCKdlDIyFM3{sV30&+M3(c-AUE$<<#C;)$a9F@BP&8d6*)Cgl`A}7X-^YP(lt6K`{jZ z3q zMbZ#KO&Ws|-NJY%VHu=A4kW=v1i=tJND7GKMNCAbfQuVI%W`yF9@vI4U<(nL0`ZB$ zOR8Y3{33P`K{pZwAt{F&^yDqrA}%HY45*{$)#XLJh$-A+59pC?e8aC~M;?FF#&Xny zzW4^PaKYqILJy#d)c7M)z#;U=VX&!P9a`qdAfz6)8|}egyhS8cO{DJipJQF5R%K*| z4W5kD110Q;iew8VG-f7fPE4R=HPVC-5t07rBvl?vToy$h1i>qmMsv8qfdoe$L|#4E zh$8~a!$hKmWJ1B1#5o*A1x3aQ8$9dcRcS0@DR`>l5Di`rc&UGOctkxIs2e1x)D=f^ zegqdtWva~QAYDYP#_60M1b!I7McwK7*$}g$Jp$@;4r*rTTGtOu!K5HJt z;yGkOtPI5)W@w0QDEM@!^?Ycbgs9x*T%L{QbCqgqo$7R@>S_*7(G^OL#i$mZ! zWCSCi_K^(|>Mi6zv<&M}43bDzO4EcO0QpV1yaH(&#H4=;O{K~Y#7>XI<_X0rq{TWM z#@e1|DjCPtm&eA|s?Juc@|Pk$C=|Q`G!V#Hj%Nxuk{c*xt}aD;yreNa0}zhqcp&O| zHbuI;sRRqj;C)l z9Z-lws|0^Rztje)ple7%=xmIo*k*{1Z~<}f#;?q4+J+@fR0_590O0?!M7_jq(_qIh zUc{S@1`+IR;B+0?a0+&G$HEp^K&BlXVrt|T+osxFXzCn^)?SMpaHdVJg;j2KUG8;d z?taCn-3Eb_lB5I*1h{NVSjsN25W%QyqQH;}%l?0=kBTI{tS(OW1GY?MB@Pl$Zjwt9 zOrn_Zy$FKS0mTa+@4zT6)5e6hW=ACk0jb=BKwz&b8bNM?1QA%@vYam0CP}X}ri<*P zoD$zF#ENz}izt#Fy|!#aV6UQ(r74VuGa`r6Q46kENR5bM4^ZX3crHrZiwSS&;QCDA zzR!Q*LQmq>$>MryPLCf>4;7n3Sr|heP=Y3zg5sS+GM^CB z9g$@P^QagNju6o}FU~8tfho5|)Ieg604^){a3egc?lQAmFa$E*hw&)mz$hmrRHCy2 zp(&X1w-%|SGAxfJ^7I69^9*vF)PpCm0Y+oAx{2r>KJEj*m;}cd1>cti=NAU=m;Zlk zQ^y5PW88x^tuD5_r8SJw|Ns;e+$hbLr)_M%_I9vB>T)H-%KUD>Nl`7zxByVccEqs zH_a6HKwc`4Wh!Y6xR({U&m4bv^CY;>EV#}zxXwH{%|v*G$2j|Fc+7A(hkCe{fp~I> zIJ1$s&X~B+pt#Pc_|35R&A52Xz_^Srx%JR^%h>qC;`p%X_InB6vjl(&p%6ZGs`OVmQ%;0&;=y}WVxyt|eIiP2H#|Zk!5PC#Qxwug| zgDHBZg(?7!s)S+od`0)BO}CSYxtfu=rq8;dbeDMDIyRvBqA_~Q7<$Yg`jeCTteJWr zqB@ePI)$-%t+_gy!Fqq8$vUk^dz<8zFI2m=UwgF&7xSPUQh|Cwin@>q`@9kR_&7Sv zKzhqay2?aJK z2)=8`Lv(6RReWlx^E=46J;=cQ(3rjfqW-O^KDJ4|>o0%5X~2G-$bQVwzQ)+TX~=^o z5JEKY;@Y7BqMeVM2uq3qCUijbTKI6Dd|a`0m!Qdlor%^e9oDDP#3MmhAX$ z*hrHr71n=~w-MpDVb56Bw2AQEyPGAy-J-ej*)51Wi5|4)3!A;7OKYCHWsGT5md0M4 zlX|t|uvP+!y z($`p>zMYMgD&X5uwMyPS@oS3{C{U0!i}rWa|F*W17p*rpb@QEhr7!iXySmN6*@+$= z&3)zK=+ccDaf7|^J7eDjo)g67RYFlo@=QGUj6L^v&xXDe1?86U?2<>690ZZaBzf{6 zf)ao52#QT92Un_1LYs1{aHZZbJSe!~iYqIjvk2Reqvc2}={dSi^oY6@JG#zAj<$=@ zBJXIVNIdY2N#eYL-Z|$S1Svb`9eL^zu%LV70rH)DT#C*}ghD#as&`&##I|~Ve1{}c zps25(CkK+}Ma*kB#$Mi#bpzM(OHCMjP|08zN3Fu zpd7i|vB!Rl$wUxAOwsKeQv_*5lvlP&NS zP~>60`4(jMpm8GyC@7)JUcj7UxRHNbgrv%tksEo1PJ=L_wJPvDYO%$9_=olBMS;nDdd|JkBWL(?qxP9}#9*iKO2RDCcf+mk1 zf-oY8h$kNN6)18b!U!WoF@=jD8hK}CA~GrZ(@cU$AqU$Hg~GLOO7}c;s=ln}mL#As zrJipD34#dF1^I@O9#VjT2tm*jDBm26=m832hN+#1n&UHGd?L6QW@$=LHzEf&Ao`E` z9$<&!r~mSJ#SNZIU1UsQgzbM3ArH{V1QA|w59tL#fDBrdAYhk1@tNXTg6M(xf=~uy zKx2BylO2L+=ehL3AXD#X1nNL%KQ{jjqYT0OoSHNkyBxf(f#-0;={DG-R}diy2haDIE7x0Loc!mY?~!9stCh_<-6$0civ zkrUkH1Xnr1T~2VBs}8sJQ#(Nn!!$WqA|+}jieXT&eA3ca8Qd}qB4r|W7^`36`jQ72 zxZxGwke=-7K?&|r!hHTJ*!lFR6GU<$Wn3VF%SuPcH5J4dTu6bGc3U{>nrTUQrL<5f5rW z9L75Y0fQXmK@>DQ&_M_UIxMB+{~VMc0yY~_vk#Ub2zeL=?NG=(BM@&GJIutj1VXxf ziNs~x^d>mn0E63EX_>a9$t!vQgVs=x3($ZD7YsxdE?}x^Nq~QkX_7E54JObS&rsGv zu=qDEJ}z*A`{Ieh2%<3#OLB5MqvIZ@#=o)gZ*Ke>9RDUq^6>a3`P-sM84TPj&K*5&? zqM9IbPzGrlNGI?3hP$|6goTnXK<9%3G=@PwsK|qY-s69x%`zd^Z^Ra+dP7QtvUsa4 zauKf4S}BQIinub$lyPXRDPPN2GRi&VyFBHaPn$zWKDw|U!?@@9h+vA1%xtL#v1Y`+ zcRt`PF(H)%ZdPSC(H!`fAU}iw43hAZZxj@owj_cm?$L<1CM1eI=3!3)+Ls z%%G!z20Sy7t+R@P5*?&T|2-}hnQqpsxZOk2D^7=*Pc4WuQ3S4k%D@Ft8aIk#Wlz5N zXNunDXN4L2@`+(b3H8LV2dxP$LrTiGlQJ%)e;a@AhRQ1}Mn=nw>7^;-Y|1yB@=d3F zlP|XDn;iT0b}0HSU+rAutuh4IAY3NjDf=`Z;6*0ESJjCtW%rF3a+n|mq+SjjwzPus zD-)SOj)$>|kPyS~9?k7uS~IaOzmx%VnUJ1|0V`~p%m>zU8 z8e5yDR;PbMM=}E_$z@=l8&Wd8KxXEMZJmD`g7RgAGJH6nypupCVCWPbE^7!9CjxCW_h~x($W6RN- z@<|J>(u#9$8{xY}|CvXN=DR5mVy7HvQ8XgOs3{;ckFN12pY^Yqz*6MnDa)E6Ce;P9 z8GaVznjd~s{A@EKS$QivE2S8mScZSfG+Bu+8GoId$Om00VO*Y}YXrqWUPi211Iyq? z6sSlqzCo2T%T>;bUudQWDB&!s2TRf^2dZg4y63KH#$R5k%FZO7Ld)Rjrwo{FYd)u8 zEReA%YBK^T1u`X-LPr!tq1pIpY1#>vuJ0CHi9tpTxOj^T8sU}nNMYm*^_+i+W;Dum z^r%oM$_;i9ll-D#GGWH5NxNi&adwYydXHLuPeM9mIK)W!vg@Xb?{8EN`B+YJmM^E8 z?{S{bS)%VYo`L@hV9hjuD0Tz^BGRe2-0Ysr>`wB@bDYN%M8VCV;u((Vc8m?4)~^xZ z!?ivq2@ohKHlr^nNOXEE#kzmZ+e|G8WDE1a&J8>$^A_Zu=!3DKzPD5EDZ>o|&+Ah&GlFeo?qIPmI2ter;WXpdlB2VY?%MF64 z3{p%G@NFJX3@sV~h??meJ@NbfjT3n7= zV2)bwa9Q+l|2CLLMegBS#>N!Zgt;t+61ojPB#9;^=_Np8Y|z3O?xjyaLnvyJCUpX8 z8o`jLqCbS@W`s*_wg!JWDg$HCgVp9wYuUNnv*&k0TFM+P^i;6GaipHB|e>HK98k7kEK7?NF%W&K&fRwmjxuFMI@tzB#(c^I3?sUv5XQTsZS)TCY@=} z#z;yf$(d4dDay{7XwsQ(??kkQDreLr&Y??dQcG>3CoC?PuA)YH)VsDQM~P)eiKR!| z@IRd;|45@{NROpRo#jZKB}s{8NhQQdRs~GIOHq@^!NPAV&ezm89l~}vg zTVupnapgvrWlnd+QpMF-PYzo3R9Xd;S}o050Tq8+V@_M=i(9`HU-Pvy?15kV)nEG+ zQoBo2$yHfHj#;xPU5RvEH?>`b^j(#YOo>HQkA+lm#Z>hbV>7lW>_Hu{!DBxbWIZ-u zO$z^6d9Pr7?_f;W5m0ce9(Xn{5t zz)*i+6}4P(n8Qx&IWNwi`4^kF|TUPE3z%hn;!79!JBHP=>V+ZI>e zmRRDpR_1nBYIav{_C|7+Z!6bwjbdy&4sd^2Q*d1~Z9C3zXSfftYw*m||f#4`rB!pIBdQm_%@xMhe(P z4tO^10T(8Li@Vs1zjzXGVKH;_yBat~9{5Be*hMDTMJkv?E*OeO^-bb5N@Ca0$Gp;d5~{_5XeD=SyqS>mxy83h*x-v*myU1{iL`2I zR*7wviA&OnX_!ws!aXwPHHuj!^Prfk$2lxofR?X0h|4&a7k7V`{j`_q*5m{=n3Z&xGZ-V@`N_&abULG4u#$QDaL{zf z2u!#wR9Qz?Q4nY*2v8U{Oj$Ki8AlCTX}_5}!kL`M`2@_lmdSXF%-BWHm_*cg|3xJE zL?{_UELmq?Wm!wFZp_M2`r=j6TZts5{?jxkm!A(*F zNR*{{K*>l@0yCn=7(S>12YMssBwKpAE_Nv*+hwR1Vn04eNPaq7f}}kzkFnT8Yd#20 zh(xG^>#23asn=snh~#B5xm8NVAUZ{B&dLkFkh`=QX(bwWDLOkW`l5*=qtDrv)tRAj z6`kc4pH=slBX^z&wVr=t*Ae}(l^lt}+#sa5%^$rG8ZOZY!m0(MCl3S>;7*&{%tJ+n zN2AK14A3wAPVJt0Mi5$KaNM8~5+`3KED2)CL+8q5M0>QWLaU(S7Th3~K2iS`ZRQ+$ z+r&iYx|$oQKr9brQ5s!E37^Xp++cSiFq;hBHw!bX$elCvPgkjxhv}T||#hgpWIeunYUJH`+fHTbfO`oj>)llQ^DVShly7 zF*10fI_R`piMRIUP=+B7%HYH6CrGA;M~2**HxDxM6#ERFDq%>GCJV7G} z9xrz^@cVA)G=hJ^-2za>PJ1hk`xZp$b{+`Cjo|T&;jDUVw=d;OzzMEctPk_x7J~ZJ z;={8r${5mz2>$VgM%xP6!^3LkA7Q|Ape+O2<+g8okyc}aM3jO8tn&~||M)CpLGX*O z84{uiwwu{ku&blMFZ#e6+j_Sdd)Jx44IOnK+&D>zjgR zogku}bSHm1DJ(Q3FQ=X97V1%?{EBTLh7vLz+7lwVEhxE(MEXQXnnX;xLQooGcU`Ok zEk1uP9&$cZpI42UhQbu$N#r5xx4|a~n(6DK{=x7C8^5mSQy>MJNg-|?7Dvun!E|N+f1{J{w=2abx_;b7NlnU>>Q3RJ`}9mg>25;3?g!D|GD}B`x+z|VTmNR z?zyj9pU1u)1i=XMeMdq?#!4;0)Z?tUj=z5$8b15ncK;pF`TjQYUYrY_qi1Tdr&awS znBgVZ;lUT;aaQ6j_cH*35|kWn-a+FM#8|&~d3uz2B`OJ&bFX@EQTVM<4ng&xLE-YO zAEhLM`t>uGqzp7l4x#mfmM4d(dM0a{e8xp4&6@Xy4a||`)lQo=hZ(up@7}8*dES4S z%E4ur&2tUQ^dLg%T|ZGdjEFJ@_FKPOP@pI@dG1z2BBF>Ag=Wf44k*vgl`-#`9PO8x4Y>cO#JBgoJub9wdZ=(Br*j5PnM$#Aa2kZwzJYDa^VY9b7w(QxoYui?x_U!H5ynBE9u4(Mmx!=T#8!viG>+$8r>9J{X0)_PH)TN(j z!RDEA=G?oN{`N|@_wnBA4I4I;y!zv}hHYODpZGUX!{giU{mhih`}}XeYkyCF0?HwAa~ZcTWhZ^2x>pMg-U90oRT`= zFv+3HoO94s2P+EKW%nMd{DCK)t^ApXo~{D6SF5k_&3CG>@a4CkvG)BpOn}P5SKxuq zPV3t(Ww?b#a_C%c{z~5f^2IlnF^JLg9bnIfAH51QF`a zq}nYca^Z#>;8GGqU^uOt>|1G;!R0MPG$l_X>0axGDO4rVU`$A!Gg%VYx{=8{&s@j_ zCC@a%MILSt0YzS=J(F!7We|Z{J@+P|t)$!<8Q4jL8HWFuWPv&s1fi5^euU;DmHnl23xjju0|Itb+c-hws*M3UEX;J z(p?;|cymYW-pe8D_j1fS8~AV0-d#Ag#<*bw%Z4Uptr0_7%QCi5xbdYZDhrZF(e>;l zh)b{Jkx76sH8R{v*kYZ-*7v*)ds3#9kp#<)+#<-3k6vpG?7n}SB#O3ffaL}y^5}(f zzL&kzh#Min%mwP_P@*l%S=AG~op<~DY?9O5+2K(Nk;+?_lzKd zhy_y~VrgzlW5bleuP9|?wKyXf0b3p$hUVH*lX~za|25(pALWQ}6yy1(Yl-_!*eduN zvSF|{XWO7j)^>k_5Q=bwB18cj$bmKw%4TjV+zqb~*TU84Erzz~8x3g#xEs=Da5^NA zfneA}PTWC+#*-8bc+US>6Q+9kRF7%K?{Bf%wPud2892B8!p|#!ebKUg~}w#44YXK8$z>B z`k5wZ0Azp7!&wt=)VtO|kWw3axvLCN5K?8jr@kZ{Nf0SC4Ht}%PIcxXVF`kX`O5MVLHHZHNBj!bc2sk6Tcp5j20LDJ5>f#GDf49(Gj{|6cWKpL*OCukwKFwB7;81oqXhReb8F zrip*hX)3gtScRrj=*iXal=Q363~MvT+RU9I- zk7?doqSwFby{Kll>0VsIx0dofCVg>fUtHpMzZ9;pu=-oe|Kc*hu^e!Q3oOa4U`JSj(C=nl>}16FTN; zg8GJ{HgK5RYwFyhTC9(rFRMq}tZc%1*351;YjzCTTT3?A3tiPVdmZ9mL68+Ug-bAY!}7C=a5_b;xoWx?G1azoE=& zNOPM@J+(NGq0WQIa~k?wY`A}z#3UF2c1C!D9=I_%gB`x@U{iP4pQU(;kNsPW-&f;P zue*1v-omT*FzYegI<@dYh!#{Q3)U#R#ZQd(daM1tY~L-yn+$jD%3Z>Aw>xnFD>lqG z8}BOAyAJu@LcdQ7gds!$6AEu5vm^h$=si684U-dLN$y&>m4==mFhUWKB6*4n6Zdcz}{^*VGt4PoCx+0&5rF|>UPai9CxjD%9p z8Z!Cag1sP|pH%X=MI#RH!Q*c*`7~U<2b-^~=T9g4sfYi!6Q6R6r(lg|U5^)Tkyl@n z*J70ye!I~KM9>HlHz0oA`3>Y{b?=T`br7Kxt4EuB(@`pk6 z7en^9LHK7Z`X?Ry=N3aZZ<{56B}Rbp<#v$gfBmL_QO1A`Xd4g6fmtLaZoyU(2pUqN zNqbc(GUGB?atkOZAzD!n8c2RX|7S;>@(qH(1qc)yYz2a7aYlb|b%iEz54Thg=TIMn z<1B7y6V8HI%CUsYl!SBG7;N*$E~A4b z3n2>NQhf4IC*OafNTNU~^{`Z?2r}oOh26qaw~!RWkUI<`Nts|LTp$Q5MM|(}itlF< zD6NXZT(PRq4kSPQqC32xs4I(Nz5uRef5Iw*s zP+=e%@h3K;Gp4{724Ms^;)_I(0^j3LMzEXr;syqE6wn|KW$=@N;6EGq4wL{Dv-Kc3 zFeHDi#5K?cW>giKg)^D9Qkl{>YML2CoT);dxj~?*LZazGq`5(+d76@e717xgCJ`i! zfS6)p5)M-kY@#VWV-r3ymFtq7pg}wN()HAwZNe7z9BH5MmZn5(Hvc z8uzdxxZ|B72o+osAn*Sm7hwnuS=tb{@C|=fu_k`9mJouMkI@v+pgwJZ6Fra#dJ&kz zU?B|3Q*bg20SX!CI2k1|3@xz`ld%}m@}Pf)Tp$NvBF9`LmpB*Nf76G95XWeXsdSjQ zZwPpC$yQo|dZM@SfF{!rj)I_qkO@aZBt|i#HQ1uH(oZ>%0_YhVc5x*)0i;G^1jv7w zKpM!Z;A9jq@DcCiJm263Pr4fS5I^q11u%e@){+APNrEle1I&64=P9S!fMd8{F$Qz9rIFGqO6B;-b z$|{Wdh?=N$lGu8b|9EefIB|mba+ZI(Vw_lRp17%{(Wx*28O~BDq98!u2q~vZ6e0)% zf>10RlLIi&1N3Smw1Yq+lLV_&J3eX>MM@$@V2(pM5Ckd;qL3UGD6F`_o{hktE&+^! zP);gg5}A?&<|M5Lp?*VqBxQ;fG0LrK`YtCyI~WnC2jh=5D;)Kpu2$5pW8r_NNpUjA zY8F;Gh!6^B8rp)0nsAc(HU4^;02_M(TYUt(at6C(l$x+`+kFiCZw~vgC5aW&BaqLq zL>H0?XaX6*A)p#-6YR>X$+1R)Ww9Ou6(m@)SV0pg`<(+q#8alDewQ_Pwx^*ZfJiB`f`F&;=1{=n_ucJLsB(!!L-@xFlp+oQj0O-sz^U# zKlRWpakRDUx-DLd6m57$Xd)9RSR3~0H~5M{`sz3QiYodS9TPbfn3->Ki;;ATbawk~ zc*}GO`(%7ObAJ1`s0o2>k|rS%6BluyG*UY-fUGHm6F4$clmHf1!6kn&QJt!Bo?L1n zDZ3TvQaqFZK3bR&J>V6@_^bpf!Dj>^{IjDLh^#0B14&>YOL~heb1Cv8B1fVfol}(- zJC!iWtH0~C!W*?_S-c)Gu3JGPA(9D{0HDp=wMMcn%1H^NBM?DgxwnD6d!xM-#Jzjt zy` zJd7cNG~x#Cmo66Uv4a{_XDgvOdT zGLOb!p2q)VDRh5jMb9wK0NNPP&>r?|&uKK9A8628btdg`g4d`jw?Me3Bz)as&WU0X zE@3H{6eTwiN(sG|*O<{(q|ogld_ zAk;$*&C#5pkE(eZY0Z<0u6c+Ic`w zwXqVX|80LY;mFqA(G#p~H6gi4(nh&>b6jJ!OA;t=ovL(~8}Y>-}l%Eo9`|a+2Lb z9O^+H3g7qbK_a?AB#J>M>fak4D(4+HyMCOTO=7odiYEJ03!RG2<4~eemie6&`?qBJ>a|qsZ3T|d~{+V3v zL3v&{d+s-Vo;ZJAIDsBGgI?&R9vfH2WvPGOWjX!mcMippu62}dXqFz?u3k8r9ypx7 zH=bTNpdL7)?l+`v>ch^MGnQk=j_f(+=otp-SO@EUC+nd`>+p@~DNfm)dED7Y;YkkT zAi3YA+26!2?x()!i{1~3zU-T3>mmN^p%?AEHtj85?L&U;9h&Wz9p2ge-b?=NB^rP3 zwW|5dn4={PxI$K@ay*Ra31n6IP!T{@_c^sxBl=e z9`PT#?PK2UP%YzgN8>f`^a9`U>Rx}$6YuWzJ%2yn=|LYjDZe)>Pd6>^H!r_8F^@Mh z|MURg=3n#XcoXMhGv{iL^}DY1e#7;6R+Y)k%JX`^wV!Y4iEh68ii>dN)w~m2&!cpZdx_^hI9u z6HfK`ef*(*@lBrbcZNGGt1N#~iqHVjPvAg-xAMHIM^NEHh7BDOlm`u(yoU_GJ$qMC zBg1;4^4!`OQsl^r_IzQpM^fcVmMYKPGS*V&OqvRfy*gJ@=T4EsZe`>dR3$w&El!|7 zabwcMbE2RLg&OtStC&%(@~k&(Si-AaXWkkH)a%%l-$dmkn^xpAQ%`@@zSZ~+?EgKu zbs5StWsIv`zJ2>qg3`z@!+R6}kU($01%7jr=b1xFNg9c*cn|W-XL;N#7Aa3qnO6ek z8K$W@38ld;O`(=08!O|`+^=xvl zY~3=yDt8!N=uz?Nt)87EV8O9}<-KZD^_(C<&lgfM@_6HSLBM4~YgB$@JxYRx^@C3d zv`l$?dkf+rOgz7Z}d75p(hi*G6sihoh$~*^U`t3rUhTBWSmX7m~CbM7* zaV6(Ulu0`3P$cQP4q2>@yMds&MU-2K!QAGO|H5jWI6+V%$dLLGLkR^5E>AZyL!Liad;LB!_s*z?LB<-JsTg2rei&=bmB80NDSr zVFbzJDfkNJq7fXw*-YFrjG-itVVFz-26Q*eBqKn}Fr^`Fw>?V`BDg3ci6l<$=N*E# zr6i+^ErUX0Q4%snTqw9LLzI4|#W|yYw^dK2RiRbLRjwC`_3MG;WYJa$cT1J)Ud0{= zShfoa>upb$lRH^|xtV?YS-JU=k<2nS>*3^gDCxlnj-h}dk3?hAAfX9Xt;pEh%%mZBMDKSaRY`N@~BVrn8@sGp>rsI2?BMFsT3rK9Jmk{{|5cs zt=Ekppm>V++RJFXjC1H6pW`5M5JVm(ThC+Xu(%Hl>CRM=lMsk)UXxqc*hV%KnN2qo z;*H$a_BOY@ZCDbd+uQ8+HoUP-Z+oi?cjS?Y@!d&sLC6G?VDOmj^+0hANf;DN@eX+a zHQK^yDEALEu76zA?FBP(o0~IENc@ zkimrvqX$Hw#6`52hj+Xp2SgABjwA;KBQ?%j?&<*-o^d?`vaN$+`=Bh< z!kzIrk~)nDLLNln$Epm&2rpd79jEs#%_uAgS!-7wdXS03bnY2k=z-J-Vnrh4APF_= z$1{k3a72KuFD!Y;0X!Qrj3Bt-NQ9H08S{XaH5$VhMt}llhLN;e950~)sRthWaf|ik zq(AukR}cUsI~)Q>Goqvm?;aUXQPd}CmPFlCn3T|4jPFQ|Fr2$!K!n6VL8DDcLf|ry zwvD_}YzTZCEx%^V5aAMSd+VjxekrzKvdvh3b!!U8fjjf`=Rf_p$Lt<| z*|hbQ!3ajMq$t%$Ew+g6NIDalAYc%Lzm~xbuaJyabAURshQy~|11j5u$~B^T$fywH zX|t00wWWTosbYJo2BW&SsdBBVR`m*Z-l5x(YK91cIAsrX2S;owq#pQmgN2xMKT$^a z8*eRV6roWDAS$FW+(KPUppk?e?DQLd->4Y<2C?ExF}q&`7xZce>Xf;e(~%rg z=@ou-N>(SPGe;ITu|2?Kk z`9e_EWptG_U9MRfT3A-wFu%s-uX(vyU=DLsWOUygAQaB&8V2%)OJG5`xl)4Yfjgge z3rdJ?B38^@Lk1^4&TuY@yoN!JlR!o*J|^DN`$oKH`qh*1Yq&dfwihRVe!t7yL!=ZA z-sgT@EyvaDoF4wP^Ww<`Q@?Pbg|_$9ZUKdOB`u7wW^qd&nTdQGp`Cg)1!Hdvje2Av zbZ3OKAnq-`aXk;;BtQ6(QS&xwbB)@{vbKY;tvxAYm&$wdcC)|@EiDssyLX$5+|ZqF zp$wxDAig^C{WCCJt2#+Z2!=?HdxrhIaz*ws(5yq(*qhxMa zn%i?N4}&xU8uxJg-|3E@MkHn!8Hb`}mKT$IJVOwlJWM7SFBgFQjaJ($nm|4m9yA#Q z@Hu%#88lj0g~($V0P}nq8?jf@=lUDM27D%*XwcJV#4HpGS`{gOVG1{tIIg1jdwW># z2G2YD*+`~(EJ1knV`Y6o^dLkFkb;6Cc;cuM{JPx2uB@?3SKM49H`mOqthKWpD>OV5 zAZrv)A(UG%D@vdfO2~u-TtHLEiA0kmNLr%3SfWT#Kn9#X97!}*3L3GvhfO&!iclgv zSv0PIz(%2t4cw7`3uHi3iY7G?ixC_M$#@6qsE18~hA24{|Hzm?h&l%Z6NpzpsQd6e z23$Z&cplXHK?F;P8sZAcFpLePp%9e8Ov#ASg9&I5g(N_sfdC~*(m{DxLLFQQXoHrj zgBJ257V}$>WblMH^aNZWhxg;9_@f}LFtIC(8>G5FXvsf+XxYDG=|BHVi|^o;duIaigvqN9P!@a4$ZR^8k`NL)jL}D34LL3XsTNmydyiCdj zQGiBK$ON2!5Q%emg=dV$OgOu-ur+LyMjLsCMtFq*ijM!htBN9w#&2{Fp~wo+3pUU3 z3RYAWS5y{Q^c7hQ7F~hAb)>Fdl$&2Xma-eSZ8WB|%bUE&Lu7o58!QqinL)a^hlp&6 zh@{B6kVuIfh&khn5>yC^gh=7I2rnT=o{-05p~qu?vBzD(M_<7C%O1umzy?h(K zWEH3q)pibKUf0HdeqDXqLsbz zOtJ&bek6{6Bu%FZMr9dBw^U5;V$9cc&gX=GO>FRm>7>r;oX%XBI^2|{uVfX@gpuEr zuHb~p;grdd49Md=mgG#$<@~bdgwFST&)BTZ+LX@h{3*;-uHCGq-b}Jw{LI`W7Qyrt z!rT?ZL{7v6MD{#4_k_;|b?LWP&*VQ(k^{WBkfEiO+zKEHYP&)cjEs6+|v&)nlF1+q_R3ja5H?Ek8hA z5OTnUX_do}0M937R#^d6RuNSgA=O1S)nP?d|D9A-WHnc|bJY@o)m@QQ6rt7F@Pr{i ziY&kdYsJ+9)zuE=Rk8C`vQyMuVboo5R91o1b2ZpwN!JirS6OM-5_#9wh}RQ10Vb%| zjkwk{&DK}hRv7747Wr0y4c9q;C01iJ)`P7HKZ!ZGc!?w;2^}fdsCWlu1B!%fh=_a1 zTWpPl^^k>qm4*$GhfR%$o!E-ih>IQ6jAfOLWs#0Wk&l&?fR&Yjg^_|a*^n477;BB0 zYXo3*n9t(cnou<)iYrR^ypVW>zbPz>NulJ>SqYg5vP3>sC^ZwWf7{aT8+4)x@rxP`da^+FakGdT*~N~m{_RTle34|LN_QZq2U5` zx|~Aci?+Rxw}p|o)sVRzjJhS-iZI$v)ms+f+Y<5H5CL3P5nNUwTof_f!_AdB>mPmk zz{bVdOgRT<oI}|UgIs^g&5w45M2i$T^2Fj3qf7KP~E#ls(iiCeWkyCwLj^7+K_G3 zk%h~W>c8 z2PENo(b9S_hDLBAZ<(d&?T0EF$A)-@|4iT*X?nmqc?uRzA-aHulsTLW8(S>m9??O- zhA@Sn*jSdks;4{@s<>4UyUPklKZj+l7(bEs@<#U?gdbo%uEO!jJyZAp#TI9&!V2 z@uL#zh=QRh%B7@J?4-RrB|)*N6i8oD;DRJbnuE(dghCgUir$NXg4kn-b9ovfIJF_7 zmd+TQ{}4m}0OimDs~JvZ)9W31nM8#!1tWkWVZNyc>s(%clbFDO28nZ=dTSblN}7QA z2GZqLy4EiTGWR`9+cXb&&u2?F%?AS~=cPI@VVIZAt(JMgb070v1>UUQGlJ zta_-+sS%H$o%lK)Cpnipv!)bCHFsuS~N1ik>?yEH`G@{d) zAR?vCWfPLY9kQjL&V2`oHjYI%E)a5NU2`5)bkTRrf?=4V90(%EN+(4ECsje?Xya4e1nW$t z>Hx8SYC25HD&y+G^y)+nZWiHV7V+Z{0ptWm={)d^-`y`X%CE=HGslr-N|2sg7Nv7o zC@#o}bKn*_*%`M8x}BpPP|m3cQ#JA70vJB1UbZWTK|E@4B+5SNgb|rvGX=WHJtBT- zH!$6@6ISW%2M3UFL>wdNjG3$OP=mYN)9_DbjhX8w7Yc4}Mi6f<5pZ4*apnsZ zk21W)*A@?v7uS#&zmOS6k>^H{==PB5=G_OgEfz*6G0~#@c?UgeYL4JJB8VA?+N{rm zLS5zpD379BNCF}NE|TeQTz}=l$OuG#s$w_LpT}*4oZ35vSg1czt4Eq5G)k?;*;=#O z+THo8#W?Uc_uN&s9U!jr|9Zf=sevRwe+)h!8Fb2ogg7%IXd@6;HRo>!pc$e<%X%-zZX$a7nb7*jI^U@FMTR;>ejfKLWu!tcAY#e{uaMl2n-a)xC zr1yEW_e;h1qSW{5@Pr$XSi6mXV}n0uO7{>;zYt7!5KS);PA?Hp*AP(W-GON*g*&x) z5~tz%HH%jWTo<1p=pO&9g`Q6OqOFZL`3ai8x{ouPH&m+mf9a$kXq|VMCOG4*`;eT0 z5*mpC?R&eQouRj;__g)1xmwOQLD3<~9gU2brjk=)bV-QgNgaX_1wbNyg6`=j0mC=q zQZ>e(WPyPn;OHZ*VhKbq5Je9WMn@1ww+l~Tf);22dez^3{C9a@XMrzFf`4bWw{e7D zOodN}wr!WA(n>ifHQD#e*LEof~$`*teE0zrE9#spBwF!|I*8H4x#g zZ~gi~Gc^<8FnIKmkmbvk@4g}YSr4I6Xv2JddnMB1!l0RMRoyuAnZkO;hB^G% zZx|FJ#?-=%D_5;OU)b!;&8v4W-@bnT=FRyQFyX?659>Ad>Kx*K#*QD)Rhbqt<;Uc3 z;qruw+~0Gepj8enx^LFOqfeujI_zfZ)}n8rbuF9nH&Kjd<39Y%6zJT(3BQA_@Hg?j z@;Y8Du6#N3d-9UUt@j*S5-y?ymC3H%%}Kq^>kaQKJv?0J^2pZ%zt^+&_O|<-mpc6W zTk8PCgT5ZurX*NZna&gqHMc)=;Ca_4HKY@s2$hcjXUy4S{ZJbg57@H*=XWB zrtFv9d#=f&Uy}VP_l+Q2nBtg&c=d>yU|G6mA(yCW_~mJT9f~|8jYZZ8SdMGv+2?Qa!1m2E#t>SlF)6xu&o_n^`VF3=hjw*N1VpiIP zrJsduxhY~^dRkbRp%O+Wseqk%>R)QA`j?xn`V}Xve9<{8Uw7K7*PglZ+N-b3UHL0q zoeB#VsKu^-bqk`unu=_y$g+xTKd(wFE3&mpiz~9bO53fs-}V(OvcyVzY_X$``&qZo z78@h;Mzhh5^kv8^kw!}NB$Hm_~N z9l7Lxs8NlL<(Fe_xz&4v4e+JX0?w=8LKEJu*xi=h>)E}g?W^OtLO!|dvs*c}?K5}H zce$S<8+!k)qnlm2us@THc(sbB%y_oKNgF%u&qEIx?t$y>d5aw@>n-rA3;!$ey{69V z>b$nz>+7~UAHDkP|24hn)o;xEx!HHSz4+aK`}#ev;paMjuI2OUe6va)`wD14xVZ0k z@XMO~1V+EB?QC$J8{X-p#Xnp5&sG4;l>lKyzypGCeF?M`)_&!;T=|ZK5hUFNy;s4* zEz4}kBbxFc#=LDICxklO;Rt6{LSLQGRw?x13WL|e{Jl_N#*5eW&{o5h-SA#H+#wZz z4cJ3i{jgU-oYfGqSVZm}u~+)b)dgpj!C7t4Rvm2B2dn5tw_UMSS>)9gVbw)B7K|R& z*drhN|Hwzqc#2&VG@KG+sJa5m*B>~8E2?}$_cA; zk}91IT_-WyiA-fOGf&Q(Cp3#^&;H?4gZeZdKiTHbf3i;yLC6F_4f@e6K|>U;7@KGq z>M8$+whE%5lITq-YSD|rb5k4T)fjD6Mp&H@R%x8o8dcRsl)eorE6Kx3`|%A^yuv8w zc!oTn5vYv7Mv-#Kh)RY5jhH-t<2Ltz##!@{k(R8+A(6VpEx4+dcTfZ=@1O(}%7B!I z1nMXE*y>ynx)+;n3a6^lsi%0FSfBnB1@FvgorapLqR#55tx9UEmI|w;p6aPkUD`9I z#Dy|Ip$7wfMk6Tkj(WVp4VlOT7q}n@QP7qPdGJYI-Vxi{a)A=YpoDCH9b$|;G(xOD zfrde+ln0hHq71bKl0&{h3F&g75qK>F402G0Mx28^wH3r$-XRJ%M3)O4J?!TEwLlX# z{}6jGC1P)cS<#vxCS10uZQ8+qtK9+XQ^C|no{-b@e?8VrS3+~XBQ z@a}h0!iDbIHl{{kFcg9yh^M$A1wCM}j@sAWjJ+3#@x|E#(^X$~MmE1fl_!7KyFY0P zEx<}0Fb4|^y8|N_ZV?W`E8g*pGW39@!#FL46C}`Qzzqe-i!orFhiBk52|X}{Jig(E9PCyUIp6|I9CD9L7=a61J;$XR@eEOf+8F?cfWL;^46+Muq)r!qFOE`moM92k9v!?9x zZG-ynambI-g3oFg4O4z3&|F)`( z9V%pZTwi-t5QtSOZG2N3<=2M7%6HyvxqO2QImiTv1v}>)lv@;rY;?}o-R_^fy9@=X z2W#goiYDEEJKx~`3|k$g;-(Y)$u#3SFKPWZTYn7Kk(I30dChrPdA(I&Kb6=?MfOvf z-BV~!JGky)3L+5UGoXHO$>DzTbI-l*EuYIf4y`P0GKOdCTU*MACCoqiao=Sk|JqI#cyte$?XyR-Do>~+^A{^kE}R?M9x zvl`8eJ~Y!_%wS)P1p&FSl+RI^Amjk>VVkzeLngq$-N}V9{1_2%fs74y% zFZ%}VO2%Ic9y>xIhekxJ}s%IpPT#N}2VEZ;Y=mTTqMb16hORmoJi zK~}ldFwj%GR1XT#4|Yw#Si>X`!Z(6rI1<7n;DR(Bp*5b05}pcw z6NZWtdI}VViWK?@6^UgTjuju&tB2qMafB5n#J)`=qyB*;idfiPr4 z7RW83iDlKFmt`XUg`NHl8~-tt{|yiT)?)~DQ*$*J@co4viljlwA(sf`n-HX*7^Iva zBujqL{h%YMs3WPcV<2JV*lFZvaij--dE`fe&^-`kQ9|Tg7^P8u$x5JoX7aGG}?>hI3|r33Q@~baDxGrl)nX9(HbucJ>K(R+e}A33zUbc;*Rt zn&*F>A2uk%fD&kdDg)h6Y=I4#ZV~(EPj`nDlRt9=LXq0J@kzNv#{*IDD z+{jR<>RG56VQ8LcXrFMXoOq~~%BhnLsfiY;YWk9xst{{l=ZVhenJ!_P0vMk1B%kJ` zYxWat&gr6VX^Fm%pB5y4nC7WP0+XS-D5N$ipsu5!<{zPUrKRF1qMDSVGHR)2UY$Pb zopy_Z@(6^!C8p+ylgbH{vWb*tD3yk%mC7ELnrg4=1(&v}m(Ks1r|y)V($~5$X{>50 zPIBr+5^HgSYL$wrfsv}OT5GsCssh1kT1qN98LJmX>()gaMK&gXrILy_H0!6^)~ zuU_lB0_%>bXPGc)kFct|lBu|wDY>4ext2*ikVC)B5cAc?7|YPJitS7 zv}vBa>6^sqyDnri%ew5#t}GjP z!pNpZ#Ma5gvPs2%TCAL6tc+;vmvHQkc&yKY?EG!3oXF~%(5i~i10jsw(?V_3O07HH z03qz-Y}l-v;4GTx?9OIs&$`IZa!JtM$k2vu(VnBoqE5P&|4GwE;@iS)+{*3T(rqNR zfYm}QuI?wh&ZE0}t*L^oiimBNkgbfEt>B_EC2ui z0ImhV0{{sB0Q;A1lL2>=kgOY*F6{)|CgHX2CErz zKQU*O4kVEDV^7XR7j5*>9_`%oM;SKVmtOA%I0W8&_y3og?*`2Qua{`>1}y~-z{EoT zmzwYf`V?&CNHTQ)j%6})htKq4H0MVQ;`NuP@doP$t!!gUTW|lDn(_wa8;E`|SB~|7 zLJ+VUgjc*V!bCWSb)TDrZ@BsYmzwhi_X!XnYzG_|xd{K4n)C+t3NU@#8$%RHUwZ$S zn)L?t3x_v>M?irqpab6jmzwql^bC;(Ns83A9V0oA1^<_t_XhI~^$kmCe>J!vx0weY z|CgHh2J#NhCWodFrP--nSQUxg|CgHi2JsI#lbV?t$XWY(5B2wM<^PwO`Udb2N4mi} zyxCQlDmuAEdk@S1mzw(q?-BAQ3jSLLix$W6Setfwu5$mEn*0Xt5_+JV(srnJnOlA> zxyPey%^3fen*9dt6ENL{OVNqv#7;F1*XXL9s%D!1mzw?t>=b9v=ON0(BTxfJFa~{o zHBQA0-2a!F{|4(7c)g&FM;gJwo8WG(xr0iFH)rFy|CO2t>K2RXjzQ^`ap^({DUYFc z{Auu$m3#RAmzn_wFn6k1e?f(#OQS8t_Lka<#?HE3$%N`JQ+;R+nKw(ixpZA)5h zxz+FAtalrVEEvQ{*|Q-j8Fad`@4$0Ol5mj+F=En+7c++B3)u5#(4j?-CSBU}Y1FAz zuV&qPAT((2GWMfgTj&2f*skk&i$Q|K4B)|mQ@KLM51&fOl`ps9h|_cE(WOf-YZ4k$ zvS26HD~y*d%zuF|8E%%jaJ*LWgX^gUI?@f(xH@qLY2mEHbNKBAkVTucZ zzA`fqx1jf+zJ@%xIz^KN71ckgO8K zBvFFA@iX=a)b6_WtRkXF$jobLF1hqdGl;nEYROHI(5!LG(gZtEpecj6aw+gMdeERo zgd9skfvlp;#Uf>-|8b-&>2&l_NF$YWQc5eO6gJres?D}~wL6OF`uEPur%GrCpJf&L5vVvX>tM_-POvgn|K zN%IdOk;EfVp82?tP#*E{lJnbl*KE@+c?X)g zU4N1^uDRFn7Tw6dVM!`hp!ul$d*mR>07=}j$b`ncySFw9LT9;vqHv&;ayQVC=0ocq z_Ldgq7E!om>La^762+883O{%2g9lRZSV~^WQ^gp$O``VONmP0;hd-pml+u|}Tyno| zt-Nv_#kS6 zpNK|op#<|Y)zn#kns3Gn&b-uYeSsMe^O!Og?(xYWf_McbAXlwCB_R{T$dl0O0Y4}V zuXxTIUNa(#Qk88K}6)4e-3y>A|Qw^ULnbptap#wT@fe? zX^Hm4$T{T2P9y2MA||kPsWLVksJu)`uL4P-s56doTMG& z=sQ=6(Fl1YWi4%aOI+r%9we;C*TB}PPr2qDLRcZ%TVTEvzy-1jM(9VAdJybm4DvCMGz9jIPZbbE0*CIg`DUWD|%5~hE=R%CF`YPR8 z_=Yhe65D>#*0vY5Y9Ng<+n98gl$v!daXC8@r91;R#&AeydFxt`C}p;*9a?YuIgDZK zHmtBENpEL6jAt|#x~KK5NO}uX^Ny9c#*OVcYXJ$JgpsIF&g3*7ORRlob?Z+}t4 zOJAzBHOgpf2HtwiWPi@J5sEjG!Gb?oF9GugjmCVz98&8!gsn-G{{V~sCF zAOg6;bqsQyD}x)%0|-yY5sc{6R=A55*#7y?$Mti5846lsyh72X74D&NduB*STGErI zbfqnQX-sFD(rHfXngNL#H^({7b5<~eQE*oqC?U_#al{Q!zzBHwWzZ!Fv}YSV>+Tx* zxVW}8a&@g~V1EZ&*uy4vv5kFfWc!!XobnW#QzNQeh+5jLQl_a-Z37-CAy(30UN%1k9RfOIy^`W}CH9fP!rEKn@V@HV=64ZNsix-~%Uk z|G^D@aD*pZ;h$;uFPr`8(m-3@qL#!Yl)yG6oZ#NqR)2yL{Jn26_*>-vX2a08@9pzc2VLwzA2}0ha&oArUF~aUd)wXq_Vr2q;Z(1O z)#dGncYlbg&*YFpt9;%9z~6i9gZH?|nZPcz!(H)z=xgiWEc4mtdV)hr(X4|XMO8kPjJaQ8#R@mxF0aD_v>}a;>pQ43PAtw z^K6fv^!~mVf->I0P3v|2yb?FMP3&+y=Wi zzVpq0e)Okb{p&vs^yzN*W7+{5<wCZll!wY-m7nvuQwu)+D>@tS1bDze3xvQDJV6soK@?m; z6@P3&7JNY$j6oQjK^d$;8oWUp%t0L7K^^Qt9{fQc3_>6rLLn?dB0NGPOhP1FLM3cM zCVWCCj6x`!LMg05D!f7~%t9>OLM`kU-Wx_`5QSBteU1Vcw`L`Zx@ zNsL5FoJ32kL`=LyP0U12+(b|8L{R)hQ4B>=97R(sMN~Y+K6Ae;#6bA#Ls9X;UC4q4 zt3`Z6L|jb7Q)7fw>_t`dMPU3zVGKrM97ba-Mr1rjWlTn9Tt;VXMreFSX>36~)PF*| z>jXdi!E*=(MgV~mu&cYe#askIpSvqUD2r%J$7x(gb!~+{gcm%t(yf zNR8}Bj{HcE3`vk2Ns%l`9z;MuOn*t0TuGK}$$wN8NstG7a7mi1NkD`Lf>Z~c+)19) zNqbNSc~FO$dFN~e5EsEkUfoJy*!N~^p|tjtQS+)A$OO0WD% zunbGF980n+OS3#nv`kC2TuZiW%c$(fEQ~j-i$x~%l$0b%yfjI@%uBx9OMk!YOThd~ z!3<2o9L$hJ$*qhw+XG6sY)r>|OvsE($(&5etW3+iOw7zo&D>1R>`c%6Owja9xTL~d zusxYH!gI(J!z@hJTus+(P1t-**^EuvoK4%TO?*U5t<*`XB+AhIP2dbp|KS`?;w(<% zJWk|HPUT!q=4?*P)J^A%PJii~PU@^q>%30v%uemxPVVf^;)G7`3{UYKPx35J^E^-V zOi%S(Pxh1t@N7@`j8FNTPx`D+`@B#5%unWgPyOsq|NKt?4Nw6cPy#Jb@8nMdO;810 zPzG&K2YpZojZi>DPzkM23%yVb%}@>9P!5$$3hhu34N(ytQ4%dt6MtpUhSQu+IZ+mE zQ5Stt7>!XG-OsbiEMHPl*qBir-BBLxQ6K$LAbraky@wn1w4w}BBu!E!T~a1(QV0E% zOd-;*(HwdB252}3C(TkV-BK>?QZK#EbAX0#AVDeJ6so8UG5u0CT~q%yZBsXW)3mfw zERB&O^)xJnQ#{R6J%8O(KF!lGtx+2-QVjglJMB|LJyb+ZR7E{eLM2mQqKz|s2P+lR z)Lc|cy;MxiR876lcQ}Vh{f0~l)H=n$PK6CqJyldqRaIS8R&7;ReN|YERau=?TCG)E zy;WSzRbAaxUhP$1{Z(KMR$(1hVl7r^b3IpdO;>f7AdUkRJ5>FK)GSIm?N)EiI-(3%fh9`I zC|H9%ScFYjgUf~^H;w@g|JznHZmjSi{LWJMH-48bE(RVYWI`@vLq23gPGm)1 zWJYdeM}A~Tj$}!mWJ<1NOTJ`G&SXvAWS4L*1|NTpWm%qOTCQbVzGYm_WnJE7UhZXI z{$*eeW?>#?VlHN5K4xT2W@TPxW^QI@er9NnW@(;gYOZE$zGiIBW^LYPZtiAp{$_9v zXK@~9axQ0cK4)}JXLVj@c5Y{PerI@&XL+7ydah@CzGr;SXMNshe(q;~{%3#=Xn`JR zNv?k_;xPqNxP{UpXohZRUVa9Ij%bOV=>Lg+1=|wdSD zXmDhkzUiQbUXCyVqfUZXplPc9=A|}jtR{cy2*`tX*j+9#>S5sBtIq12W*SBwYmtTl zp$?tXMXu`W* zZft6Dx6135aa0Hif~)>B45-clc}0W&kd5Y}9^c$S&=hP6B_! zhTR;<>b$mHV_y7=Qui&S~5hZt@mp$QFdzVlCEMEzxoV-!^JNpxrzm z>j+@!+1+cSCIXJ`)?t7Fqc#HV2JL^{7T)B}X&CU9+Aszv_~{@hZwC)%$mZ$CJqJqQ z>z}S~*{uf)&uZY7-R{om2=HvpZ3Ls1@Iv11Jwjbg^}eSsnP=UfdXlgSO~z@ZFDjh0&63&HV<_ zF@;PJ1$N=xZ=m$ix`mpET@-(Z-BKr>(UED`Epw9AfGF?Q`M&AUwsT!~WjvSNTleu= z_tz;WX$k1+x{iQLFWsWXX~6gUE|BvJCjx|cTxbyYZHI9M2XQbDutq@lFIV$!T@QEXYT9Ubd%u5abszBBSarF* zCnwixDERgO`*)F^?8n`5txop;U7z?=?)BJh@|-qtxeWoFc6dX_X+(cq{KjcYuw6fA z^G(_73s`xU-|F9eg$>8*kLuQs$LV61+i4%^2*CHR5rvq~>Ih(TfA#5`cK8p+YA7hv zS2%dd4))r32c}2%emH;nyPj=-CHO~$21U>65YTvEf`&h508!A~+NO4ir+Bm{Ws8qp z6)$?S7w_8G`jLim&DC+6CwJQA0wUOZz7GPtr;U-vb%CEjBn=fv$ekvL|WA_HVmB?qMfu4Ve6|34D>3fN#k8tak0INBMt8WqiQq^GzXot>*b| zt$dw7`?Rn9PhNZ2wfmc^b&^*6;syPY=6&3)2f0`5kpCLr zCutG?8nDmlJm}Wvk8I{oa4MhvtR@}KKJL%|`3Okt$j$+$rwy#%6ysNHBY17u7ip<~ z+-Hw?+t2^oH}ikm1&DVS7!WKd;atCj3Kt42SkTQvXb~I|S83^5jB|8&|GuCD^cH$C52;_AJ`8YS*%D>-Li!E)st);J{#=ZtQ3_2v(yY zMgL(nb`defnIMrCjDYYi<$@*G>SP%tPa`)Pb{f9;jAfFjD~np?nM6>dJX}W0)Oryp z(K3`8sT7(6be@_l^Bf%*^mptYJOz0q>~J$i#w=mDiQRi6@r&esVx@BV*1{P^n-Zv$cRa$>}C6-xgxh0ofdimvBLUMHf zmK!VWcM&uzF?Z295^Qu6NC`ei5xV?V`+rLDNiOGNjB!c-Wtg*^EE3LKK zdMmEEqO^bNnPRQFsYj<)rsP6`I+SRfE9HRaLifD$EVR)|3$3dO*%|6s5f&B_Og>g? zZKezm(GOQ7eM_T(Cv{4ZqY718P>L8qMp0HE^;j8@;CgtHum)8NDv=Fl2T~bWDT)F1?f~oYAe+?rnb`qjc3&1pTKS7b&aE1r%j48~>KE z1zB{`y9@w(5y?rh_EFCG(%h4zZ@#>0d7NUEZ$EaXhEkI^_XF@n8am|FNezNLmedNq zTrt;Od;K-oVT(O>mlvD0s-Yge($b-sw)zn?ZnBF~%|Z1#SDW+_3(~v=75(W{Eq-*4 zD20C;esV}z6LfXW@nY07ya*nxUARwABjnIS!J%aH`%3|emd%@tG+tx zW|y^U=LyA3(9Rx}OVQo^Ku%EFl9$Xt?r~}BP!3@wY>m}h(8nSjvfjQ+ zPr2q`#Y9oZ2|=w;-VRB2Qkok%jnX%IvSfcb>+QS$KK${^KRfcB(?>SzuQ(wM8`I=48wm7i%g0bxwo}<)$5EMOk<{e__OM{G5>qs`(#kA z=8@xdrGbR}B`||2%wZDKErtY3fLd8VTz2JK^#CL7gxIWUMGG1#%!?kqaZiG^qc?72>%qKTl3cQEzlpwJqP*A!jQK3Wt23&B1 zDCChtjSi`t3{a*=Scx=;hUI@7BSls%O&Zps_HKtffdUz6s@Ao#wXJUdrIJlSVn>HC zl$R(SpSxDW&a!BdlWaj`0Chl$zmlFI*ufIEu!c>mX@)YekF>BLL}-LD5Zez#QuR(& z#cK3k3DPNwWp_V1DLKb#5W|wTw9oLRPmkr&x3adiu6-?R;}X}bjt(PDg6P73XtI*B zlEs1~t;CRLKR5Xxq@s;)HWWw)BC+=8Wq&=sw76O!5!!X*PVcBi?Src$A#uq_X?J|KBPG;nVDFBtgD~A80mv3 zY2k#~6t4d?mI1L7Q+{=O$0J+zxmVq?R21cisOO7tm#d2 zy3-C5bA@)oP&pS`EU9JHx9r3K-iG9VEa}xdR<<&~g5-9{t6HWs_86)t#FPBJrC34I z<3Z;&*h|)Ia}TX(dKp)L&jxFXEfdn%HC~z`<64L&Q*bU%bGzH#{`QrA{}mE>7()SFm7KinW#940)2{Zlvpt>%rD9|AOqTsh^R|`A0T;@^1tK`z zOfL>CS~N+JB)sB(8}B$rDfg&m7VrsL4*^<`Yo!B*{=KD^WM{L*Jke4YkRSTk0R{u zho4X?R9J`W^I;JsGlxI))hpLdtWQWbwW$G|cs}YOeYmoJ@BMZ+8(Sg6W~PF(|H<6< zzWwfhe|dlZ$)$T#d|N(=Jqh~wgC0ak%#UNforH{`5=NXKiuli)5!yj@O!q-p{XyK$ z$&;?No`YqGHu0Nn;okkh9|v}z2f~#6l}s6g*GEhr0|XFS7+^s(n?jsgQosO07~ua! zTUykcP%M^z*yWt#mEgX#;3u(JXf>drv0uYMVA5nt1^o^Vf>s5>|Jx@8L8^rlE)n4e zR-qMM;fO)h1?0h&*-3}BQ@99OL|~K>;)J^m)v~#sTD(HevBU*Xksd_`+C|{%5sP=} z%)$&H1JWSQgdX@&-8F?^K@1-gA{js2*Rbj!+LFB z5wh3TAl!vm)n|zx8bZSW7DO4`Ukjb0LcD_dT@@})h7``<97duwUL!W9-93~W6s#fx zRAL!_yh5g}2Q-L+8yEo$z(5Wp0aR^ZUTjAm7(oueKo4+19+>|^1HA$lq$3fOfhf3R zUrbp)$ipq<0Tj$53=n}Eh~r(H!z&mhLiXb&W+O#bq(xq&BMKx&ZX`!`q(^?_SZE|j zjwDHzq)9@eNS>rht|UvgB*>+tOU9&3&LmBLju%YUBu?g}PVQte-6T)`Bv1yWP`-~( z4y92ZB~m8k!xW`bHlwEC0^#GUhXAd_N8C`C13`oU=Ai>7N%hyCSoR+ ze>T{eb%Ra-Y0(2 zCmh5lfA*)XjAwoZsDKVAf#Roo`X_=WD3>Thfi|dvJ}88LKBo|Pr-D|fh2BRZM5u;t zD2L`}B3vklhUj~2L5G&8iJs_nZh?rdD2tkhaiS=U#;ARsg+)-LQtueZYgwPsh2(} zmxigB9_N>TmZ_1BDVl1jnXW00rm34UDVxSAi@qtH4yl~pDTvmop6V!`_NjvIDWJlr zpAKq&2CAW!D4{0mdmbvIYN(<20Koqlq$m}~D5=z(GcfO>0~`b93_z!um;LX4-1a+wCq zE5kN_tiwJm#73;dKC1?(q&g%59AvD-byWm+O0C6EOSoi-NL~Z+^x?pXuIzd(5u7f6 z)SkyabV42UYc8DX?cPQ1p04l?Z@JdN(!?yfKiXdQ%wI`D0Cx@}09u7=iu<}zq=>T4&!LCo4i^eSmyyexG# zFG}cd;C}C3sKapz!S8~FIUuLLVg&qu;y^CEtjV^A1kA1kPcQ{n@VZI>_NK=^&~2=u ztoqW$22*FY4sZdRjxyY@bAB*cEH86z!O_-*IXv$7D#PPuZM;z`E6sK~35^-PH zgQ^}u38O?B948rn#Mu(C1HXr@S}+{Pu^i7at_pyv0x|AJF1a2j7y~Z~J7*n;vGkVl zKJCL9C+8O2F9F*{8!ve=@3U0{PUV1qVe!@u6`HY-Cc@4yZO^FINu5M-`GCkjC0IqsPmV zv;15v$IdiObL=NCa6)4Q;94_Wyz?G2aF>MgK9j`(ldwndElF!~OaHTddNfGm_d{VfFf=QK)WUQ> zoNX%uHb(3NV?Y1xMa1&w{q>)#I&`*wMSr$tM|MSnggF?t ztM>D_j<#Ya#BED<>2@~Mj&}e0bz+w`X)Cl^->y9{HaH_KZhu5`B|=bw9i|ZaemDNB2Uw?9}eV6kqmnx+ijDi87!yPm9I!+ORs@aY+AmV}}HO=k`)x zEqTk3eWvYM6!mm}N_9URv=ET4{i5@M53n+rHpnXHghDu8uXjSMGIF*u{3bLNPxwVd zI1U7IKPYi>mUVK*>O$9bU0io{iYrJsv3D=T0`vDE|LcKX_ZE=uR8MU!i+Es<@d+zu zPJ_fSe>UBk|Le;pFoq+i6z}*TD?~r<^AE>2O02e(Upbb4|7vT81UAsMg(pOPOK*@P zasjI|Tbnr$1E{tBureGnateXu=C0$O?PM#&OPBCthq<`k?}zTN`sM;!Z^6j!LoMI) zl&~;)LNkGDF@SD2WP5qrDukYo`CCJ6E#p8UTyk|lzi>!g@o|RqLP%`_ zqj_>R`9-LIIvS(-$O5{M*E#Flr>F}vK^ydPO7&|m#K0cGt^f5p1G{uXdWM&>a+>QZ zH@OoN`$v;#PU~-iU-!~!R^LYRX#$E-AS=cf0=r5kmFKWAod zJ6roQsbl|ot7q`KKk(W%XBscCj`ssbJ9<&?fKy|CM6u?8X$?ajys^aQ042P_El`5% z?kWS^<1L6n9uR=9M!+jj0>a~}5!9D_5dpCpK_=Wn!}q|iE_?(RtH_f)C73*}=77i3 zR}Nf&X%QCI-D<>}!o*iRulnOHyn?w6tIq@d!tQDcGy=nm!psKi(I@@V=W5RHyv2L% zmVZQl(x&i_v-|B*wQ^E3ACI`PBj>)8cyp>Mt*^Uf>-iMBdvZc`A0zsS2X@$h@_N8M zc83Ij0{r@F^S)nw`ewazlJO7oFx%7pdkC#`BJ)WU`KbRc*VE{}ulsY3tKDZb40*Cy zs6*HX|E{h}_^sQ6I*jMIa`Rv-gx`;T4V%7yEeCe=nm7(HI|rM7>R+wp`+Y*ktp2($ zzi)n(-o<-gC#w^=x`#zSBWI}|_FH#CwXQP}`|mx|aO0XRMjxl~=E6FIdqSWrIb$}| zzBAuCZ7#U^emXH2FZ8$qXpDnJv9j7l$U_c9tN@&kHRUS%KRx{?#QW!J1Taeq00aPk zfddH&Bw(yxLWRQ!6ciXDp&*A74I)&SkfFd^7da;6(h+38901CIRVlOs)nB|Dz9`4ec+i92xMz!oy$zAZ&@)X9}A5z`~~`q?AO)M>7zMU^E~ zceVfNQ#eeE9_<$uN3MQK$)?GrY;4hgL<-U3IP$h`S)?3u6`k7As$HsqnI=-!i7XBw zos`wxvM(XxqFbKUVJd`b-=c1zf~C3_bZF6|6)HQD7o+I8;b89+yl|A@PoKjV`h}^f5i51mfwPZ^Zlt znsXF-Cjl?j{ArjoRZ8d_0|e^fA~O|&!J#1TNeEAZ=nQG3pDgOi3~0t&G@&rZ{3eAz;k>9%1LRTYoO%?>qa{&EH8mkr1rkJmA$jcVC(f2& zO-NQ!`qOfpLx(rEH3I3V-T z$k8+ll3T%wWGXuSwm{Cs!BWJ*z<#i~szJ22xTrfEqBW75uwo_FJ&*IA2Qei`P3 zRGfJSVndsDME)R5OlJbgGHc3)`>_sa)QWy?JN*)Y4C56yWHx9fty|Y;Dx2IYyqQOg zIltRf+@imbuoRd$)U1__Sc)%FTfcpslaFpU&|JTiIaSI7G?B7qB3R~;x% zD-T_v6a<2Tk~S5RC@9wj$|H7xa`+wi5JXAH6o-b-+@Z}o|NPZV@f5%mICpjE|JHY& zkDgXJRqZDefr4OvwefPn{=r1HWP+reL8Byp* zLi=?He|rm9Le$p5m*FiwAZ$%h{x%xC$u;?vC17a{1) z9$mo@)tZCE_;3wDNIZhg)0YC&yRgVY^-BKbl$hH`|AO!D7WKo~)^#H(D)LZG=g5<$X*?OX+1AXP4E4og;YO6EY2 zQxs^#^{68eaFFRtXXsfk%G;EKc@m=PpEyjmIt2zKArEwrokX(U%18mz>3aG0Gva?1w}|Ne}`os-6JN1NLs$O+HZs zd$Joy1ol+VkMyIS1hL5TjByUBmhiBmshfs)H|GOr)E;by1yOBpORaU z2#m^~z=Z8Uj$2;z@d=>sm>vLbB@ux315fdtZ+!u9yzr??zWvp%e+mBrU<0(bAj>6x z=z;-&)xFrGJcSi1Q8?R35q)eQ#4*c6xzbS+_Jd0tK@uNBGA;#r^ur2?po%~2R1wxO zF-NF_62UqevyPQC0ON=^V$xvof{~EO?|FcAjSr$ zd*PW7Q2>C~_{BAH3v$nQA1_et45tPJOz(c@8vyF;hX#n9ZjUgSpbjqadx}i1Mw+-I zmex^%B!1d7fIeDusBV|I0qW9`>c8JT(JShY;oiSTj25>t<&vkRxNQMCkvlIcmm#^we=cL^j9g z=^ViuS;lX;VfnSi}Bm8{|??}*qlM8cn$L4tIH6jYbW^UpR&%?|mF34)AIF6%4E&wgg>ORao zfaCKnkVOg!4sPSgq7Eb$ghX5pDXe5KB8o`(uM_+a02#*N|8Nd}@><9P)gv$NYKfle zWKgAg|Oj^W@S5sqMm9xJBgOzS95iq^vG)C>gqW-N3D^Aba* z?4u5trVc(MkKXPGtZ)jm5JZqAv{q{$Iw8=A5H(zAhj1)3kc6{Fi{h{bU+9H@sD$Vg zsDenZ1v76i@Wvc}L~FEOqvG%dq|751Qp6BBNiztKG+s?sy33bv2lj65z8c2>RD#-a zAPH8_H36el|n2{Eb2L}ItfYEfe4gb_c|MIYeChP%I zC|~#pv{=aSG)|++p7KI`NFr^>JjlPnt7Qf3HO~(|lEd=bOo+N;O2>*a52ZCTEdI_I$(Jg}!-2hWC z36qypg1Xw0Q0D8nB9r)RaUgD`6h-Mznkn`iqBDsRx!h^_oUs%G&YVhf{`f>Q6%#YJ zF<83MVb0MV!SN6Chd4TL;;7{u4>9D#qty~3V*-z-;_>60GUOzX@;EEYh$A?4k_Jce zW$FTd5`)IkD2mf;b0Gu9Dq&DMLt{!dMm_#RY@(7UWoRoTFE?3EFETQKI_^8aQz+l# zB*iQxxef{w!$775vFdDZb|ffcs46B4HE?o2>x$;ys1Sc@EwYa9T&N12f(bj%4&zYq zd~U~5q=C?pSiCA*pz=I|b4a?PDs!+!Z~!cSbQ8({rUZp{s6$dFT!w}$$PD#_f+xPg z7;-d6&1p|w2`$CX{FrYtfu#{H;&lK3dvvehR#V;vA^~7P_k=B=c!DEx02irmNVCy& zs`N^Qv>5@wB985vezE;rQxKN)GtE>qhhf^B4;3AvP1Os(l2lD$6LPNNIWGc9p3EG7 z-X#KyN<2{LWE_$ym6I@D1Tkb2=)zGQ6*3_ob%(qKMsmnown9=3$jIW;4J}KqMr&C1 z^dkCnp^%gExbp?=QBy%nL6L(f=$3qkFMS^sk(S)a8G9Yir&%>?JOgY1E&gfl`c z%Pg>g^G2hz;NU3|EhZ0fSyHeQ;EFOJEHSV{HUOwZlM*RPEhh_+(wJfkQm`&WDr2_e zUD@N+f=^Q5q#=GZ{IK*(fg%OWNuFxr++Gs^WD76tlG_4G6)z$O;&dPgp!&3bl)XSA zWJ}f{RF>DgZA>wCNi{Z2AEFr7v>3*hh2ko9;0TTuX_2;y zn1VB0tP>iKRFUir$-~7qtT!uEIio@B6^d3e#&QZq($;kSBWGHzv;g#J>>_p7 z;tN0Hi;krzi4dtI!{??W=2~^{q(U+ZRa|G1UK#E)R!{&hMmrQJ=USJ4EcQh|0w!b- zHuW&}F%zOD9Cl`PQ7Cc%bL!zpFSePEhX^hcyK2v3N7hbc#|XH6Q7J*rUPK7kR2;zYy)0UQ$e1|qR zO%kwFqZ5$VU%{p12z4!g3@~eV)5}Cd>Hi+pgTbn)jW=3 zgHxk}qgIfXrSfpbhT~H+bTd7a!E1Pik)A`OSjOoT!i8;+b34~_3$#a2a*9yH3Zt0q zVh$Vru8RdPiUFxKQUfRD!;4*5hG9rU?(A2QsxRCMF^)1l2n#iTcAyIbLyoXkTZ{HM z{IqzT0`1@|ivV}ikWvJfWQDGfY4emY97wT1$wr0NB1Yo4!beJh)V;VDD7MEK-Vy+) z$$tgzH1o2ZRN@@i(jd$vViPPOdhuj~)P4o!mAOxr!Svv4Vt_??^}-1T&TE^7X&`Vy zVP^@wg!v}Ur$`fjIGapGH3wpu6?m5aoY??)nVftXgMkI@QbS?@az+aYMBe5P<3$}@ zObCL?YN;eHY{v8+EvPmF8~AwWzBc5?DVIqi6@94HO9`BgP_3R^A?J7Jp_Lq^fV zn*kP_4bPr=Dqi%t@^%9(22xYUf=1cH@l=D51#c_BVxy0L)tu4UWD4sprdT?w*oyxZ zrALJGI?qAn!t-AGS6jLhWO}9F1F4)OZA4Hx%)weyuqKrwj?6(Bg6t{u2FZxJ(Rz31 zlI|`15oTU(+wyW|ZAX(KR!#)q6O%xd3!<0;m?8|bAA%s3Q`vllbS??vR>I9z5~6+u zjxJ}}CT`__t-~pCVFAE)6j+-EF5tGBEd#sS5`y2P^c|2HCyECE zQXm@pgs~l)c&>M>%et$xIVh5YsEHamlFnI|BaYsJINM`jsBH3LP^qyYcKd;-yDF$t zHK37*r$Hj=I40@H-gJ*<_X2l-xg%V5Aa)kJ1g;~0-VG?0nVBYHNo|78Il`0>>?d~iyVRT` zs7(M|@gmR(-o{V5Lc-8PIV3;_d&-NNIs$Y6{UX>L$bVwXudFH9MT#@s(?1>5LtWHs z%*spM)K8raN4;XkTh%2wDDJ%70*aSxhsWK3bm~d^Z73*O)l{*DG- zJy>1gLnE6(JPAVf-faRfKk>mK#}lQ0F<4+=vh9@Pli)PBvEqr~;&ZVFqLJT(LI6%a z2~u9cqfz4F66UeZAx6IBCmtw3-Hlgcx*Oi-e;(+Ae&rCJ=!^c~g`PCbH4*X@={o}F zpC0O?Uh1cw>Z@MV&2AT+RzaH%S#puVL?5RBL&)&7FUhUVO?c3ge?b{yh z<6iFPp6*4x?C&1$QM&H`_g*31-tYe&@B?4)gPiXR-|!C~@rOR|7oYK`F7f4l@FQRH zC!g}C{_!s#^D|%b)uZt{-}7f?^S{3GN1yad-}ECV^iyB;SO4-q-}PT#HCSKhPoMT{ z-}X;m_H$qNcfadl-}il=_Z$9y_J^PNi~sF|ANiAC`9b~noB#8dAK;B&`lp}zFCF@? zAN#XkQ=Omt8DINFz52sn{KwyYzTf=MAN{|o``7>O(?8S6AO7QC{vE>o>)-zGzdhJr z|BW910ivJ4fdmU0Jcux%!i5R}ID7~(qQr?5D_XpWF{8$f96Nga2r{I9$dM#VnvBRW zrOK5oTe^G+Gp5X$H2-VbyoocX&Ye7a`uqtrXhIxBiyA$OG^x_1Oq)7=3N@kh3pcLZxpeE=9eOpd-o1SL`u%(LuHeCh z35L87B7}gnj2kDDbuJuJGpMMUC-GR&Ko;bK4|gFL2giONGjB5vKheQ%sz z{d#taJ%+h~SO(g&_3Ya}gO=$bl;E!#!I+q$(JSj*H0GeRA0Q5gsIC*O?)<%p4z1~DnoEB(}yz(ZCtMC1kE z81%|w4`pJ|ITu)e&a3ho$^+1rL0>S1Gj{?2(&Ok^;B*2!IMl=jUt6Icq zq69q{t3aOi11hYY&MHwJqekaYqu+c3O{oG6i>xRH7~9W(Ijsp~%0nayB#)=(06S5u zqcVG~L$x+bQ2(wHeS0gY%O?74Khbt%XuIyd3vav@4oYvm_J$^JzCm$xi$JG7)W|>) zI3(qRA_?&2t0H0AkER!iYD{zt^xMxn(bZ}j0KN*uA^<4>n6APA1UV!D7o2?Zzz+-b zOqLfw<8P>c0!0kK#J{fGPs;$j9COW$h~lZm6h(03IX?mraK#TM-Oqgn$RiMpK|Cui zL`DphF&9pP3bDi#3q4WSHGdsl((f!ikii=1%XZstzYX_JLflJt-F8DIMBI9(rV+vd zO$w0!F!uwvq{kb0a)X1B-zvkdZ}6g-0<^>7#$9(ay2v?fQY!=6Ku z*N=ZWM0GF_-1r_6t5sd_94!2v)dT_sk8r_-4-pj%Kqx{0kkBGC?1#u~xU+At&{4;8 zo${~+0KXXxBGaOg>csW5ibW>}M+AWOI_406qWKU2IE=^+8AiG;rmlBN{81CxrN1<) zagA&&O#dS62gf)LDiLkWA4fzb4=PDSdjug86%o=OzkS0=2$>WS5CS6rcrPIwq@n9t zQ#Ebnz;DZg2=Jg`IRI!;WE-P|F^Dw`Jyt|Y4l`XpxNreTvg&UZA!S5NDN0Hr5h9;| zL!tmCn8S@Zk(JGxU_Uef04WKLOhwGW_fmdKjY-$$ow+Mh?j$%tK*189=gbHSbH6ga`;l#>avf<7of; zR786Jn4^k%>!KE!R@G37Dvr)lPF0MEN&#@Lehi3WSR5fMrGzDi+!O$!;vN8Q8moGQ z4vc_1YDAK%yPM{8KX05jhybO33ThT_ zWGQ~OGl?+!*R;v(8x zoGT?_*hf+Jgppd`Mi)!InpT7WgTo}rim1c!NkFho-Qz&KgW1DABnL?#LM9)Q*?vH3 zBBT`vCaoFP;0kxR?}LL^kBeM?sK9}^`tk?^1R<(|OqO+|a|jgt*}SC1b#&tL+p9zc z%>U1VZg6OIh$$Vmk!khg8_{8F&J>GUhAB({LaN7HnRcHL5FrQr&4JtJ=TLKqFw__WWob~A#6sNi{cck z_+6dwW^!NrVj+Na#qSbH4xmdA?iDN|z%jTYV?`E%9MsMs+M>~x2qF&yXs@-w9B`BeGGzdKc_3jPGlK`5M43_~!n~!J z(u#aLFLIV5OzTI+E+^%G4+9#_Go8&q&ITbGD|*q4mXI}{yc+znc!WI%i&PEsi ziX;AI6D@t&tsOhW2nKDPECUz0I4@+$prz@PH`3?sG8ss-rX^gHG(7?)FaZtDt7l`a z=Yn9V*NTMmsYBaph-hsfrm&pq$h;*oL*&dX^E5<&oe^xGl#tDT28pJ>4eoG@d)(xH z%SK=-5d*ZQlA=atvocYJ8{YcZcwQCB)Q=w$itjai*Aj4c^I(8b4MY3BO+ZH zA_;&&3E!-)Btn4;6sLF+1VC)nwR7MFAM(U2&V`F-Tp)8b_=o8X5wdF+N#qJYINn!F;brFdyvwn2k4gZwj zf;9W#ziY#}rC6hgsLI>J9NJD{QsS^fs#%e%O|+NeQ}4_(qIQK3*_Vr>H)By-E7VOS zX-3s#Ycv{>C=ThvZclI5Cz5xP2z~L4Z@l9lkIjvMvF~Spw^eqew7D`T^C2B{x-GwC z*Pgw#Sp=9CG*0C19Hl_8zRTjXibIiU$^GhPM|hP1(8F(>D5s)~hRDq zM}D6^+*=;LCl_rJMTOqkj!MW1#j6o&YhHm&cOt|$zIJMxgzR*mA_9PZNg$8@^s9gU z?AH@V6gs4TGK2sINwBltmp)IRmf-?K&{oa}V0V5~LA^nIXLSKw;3zage9!>@1^-u;808po+c@$WIdEf>m z7(MBff-Be$dWUMsW^2iYf&KP^2jV>65Cj(he=($gd{>eHB-1bh^@8^yB`#QjF~~&& zF)2^zgHb2|J@|tM!gKBCgs&9o0h!kt5pU zK~|&?Zq;W55q*iJYzR>gCMFRPP-+BGNg-trZdDP4cn}9?WfK8HMTAa;G-^lES+|3M zbclz42@zBnI4OuIY`nG*J7*DNXo{zZim9lIE8!7&@Ia%|U0)V0LGVEG0RK-F!vJmg z5Jtc(9m8q_@d_$s1V(lxjj%xuF)d=}QwecAd5|vsV2c!SF4)rp2sAStR}mKwT{FWx zx)z6mCMVlCBtykzI`|OhQbcoz5C&rex6%)PiKKbgqJ{^tOUxJ@ImII0Xgmg>jt9|t zrMQX#DUbt6kmv^zDZmA8z;A*`5&D$`1``BH5MCaEPqjr-AH;fSL4OaqFoG2X7kLp9 zKm@hbfhHl6BpDkI5dy3wgDE){1qqWeDU&m)XfA1!H;I!usgpWMlRfE^KM9m;w39=B zNt8utlt&3%L8+8W$&^jWCrAmEQ7M&E85>S%l~;+CS-BZh$(3E{m0ubE5?U#iV@Z}} z*%e@kmT9S$U}=_Z>6UK^moBlEb4izVxs!2;mwBm|ZfTc&>6d?*7JDg}gGrc537Ch8 zn29M8g~^zW>6ojin2{-&lPM98X_=RQiJ9p~nVZR(hnbn537VlfZk>t6_FqNAqw5uoX^>k_@N*CVI0WOJhu>koWWTe zLzWu`f{g$G9Kt~y8o?kAG77z8E*SxTiDGSg2#ptd5g}?T)}|vuGJEc*iE#2HKq8Jl z1d!~IxiYj(C5NiT|5irV!U!kByYNSVc8Vp)7$ik%G@C?L>5k`Vq9wBNO zpZp%u7fQOQCLyJNP63|F_p(Ld^Rd*Gd^nPXKA%H6v3=n^Q=TQGhTBu=Lf0Ts;%3a6-pX? z6tP;BGZ6_81YGcusfAj9^v4DN2mp;k1ZRS&tz>j6c|=!tq0R$)YX)aMlzc>2J(p-H zQX+wo2#!VfUI0s71vE;I1U^E>JEUVuIG9?d>W2NQJv5>{eWzCxk+Bf5vE{QnqrxEAqjf#XZosY#0~5li<-n-{3jRBNu+ z511sjNrX$(G(E(GL{7FU!W12}3lX-vGpYm;l4KHgE4;&dvPxsM%2sPRot5s ztceVxdk)q^eKob~W&ptR4VEKc$fQ&eL3bZP0I~?S4%5QcW2(gi4cSH&HV`t5 zj0I+?Yh%TsEXt~>q>V6t4u{GxFfa@wLb60`0MIHHga3i|a1Tj#Xn%w~3)T@_kS>Wt zY>P}?4wGe;ql7t7JWoPNH^sxeJP?U=%zT*4e|)|ZVY(DSZG(5JSZj9@VQ9&^cLecv z713vZ;Y_Q9Mx>*R&gp!aDw~a@>&j76BvQn@2O&T=@(mKPxf)T&hi0O{?1ymtttKcC zJuuI?#!FhlOZz-3{hYG~ZNCWp&^=5M)hrR(=5#8>I}xgE+;&^n{0uybA4c}k->j%% zq0T3b(u2v)bLP%6MQkP{%Qx~2Ot^2%kplRCyTNg55q?Vvne@-pMR}U1DHm`<5I`#Y=YW>9H%5rSvhW!tPT>DI07 z+E*#5E8Egi^m{YM!O`Kte~6>hs$05EIsyQGAgOlWK#t|Op{)B$jBKi^gI54hE20|_ zP8L26QGN7zEXDnw(}&m*H_gf|GUFG2d-6+rxhhz|RDN-teY}^@eE;Ei|of zMu<+0fKQHM*NP1hwQ>tR5CVeWB)e4+K{$j(sAd!XgBnaAqa%bwI1ntZ*%QrwuvYkk zQiy@Kpn)8iM8-lRTKIz!poD@z;raF;J%lAgUgUY;guZo!78tzsE#*^AkXT$uKYeOu z_>I(khFm_lR2I>0h!6^Q-Us1*91amE`lC?FtObE2>T3{S{Tr5uJUET8Gsua|L!#^R zx|4Vi#@x^@D@+NHiS)RMfQa3HBunLs&gfrg<%#$>LPFx6H)P&84E@rK4^%Iae4R`cr8SSneaLcxX*&G?MbSVJb& zj}SDEaUv!1_>69%kB0ck7}4m>?(FQBl6j1h-pN`OL0g)076Q-%Q z>K_sAlVp<{S=oyD?CZ|%jE9@=4(}=1?(^OfO=|D=j_>)d@B7a0{qFDo4)6gl@B>fq z1#j>NkMIev@C(oI4Zra54)GD677kDG6>squkMS9=@f*+a9q;iU5Aq=|@)fTVBX9C2 zkMb$6@+;5sEk7(05A!j94;C&@^EGetH;?l&-bON_kR!gN!s^=Pnm&#Z}^9A8Jy;x&mal{ z`WS)$ACmE%=t=mTiTIa~`E~K=1OW}-{uM3~`MXmok)h^hQu&3S`Khn^YoVm?z@*Qp zx1T@yLslwsjuvQ2sHZQPtIzwr4-~Nt5lP@F>E0r#mR4?1a%mxWu$tJq@0q?2{n2j| zD{Vfr4(?FUR0+_31DTTpKf*iK?+`fv1yC@u;d;V|asGE^y?zB7R_s`^WzC*Nn^x^ron;1oEJ@(pz#Ia;wR|b@)xeP* z<;@*vOfJz(Ar;oW3lV?-V?<4XOh~j$#1N53#yr_C2g!y5H2?)2TJ&hrrA_x-n_Bg1 z)~#K?h8V%Zw})3R5k~%#_niJmpIx#fTm5Ci+Usu)bBN zrMB9fY{fQPZMWsNTW`Mw3)hm27^ofwc$El$OJ5gLubx-jAcBBK$6Tyf6d}}1Su%?WdjytR=W$g8=|HVTBiFxM7DMMp9gfAOZqil82h={&e*n}vipXPtNExo4k$ma0K7BT50p3hR`phlP-CFVu*C zI2G1u5(!|bA2&$)jzxV#Ud+ZM+KefO=0J`plYt()Ic>Fl1iEdv--bJGxesmdnMNp~ zL=>Y3q9omc&Iuv}K_cV$X#W5_?x&FuaxlAG`Gl)SknP6&uAfZA%h2kBQr>sC5D5CW z^M1Yw;)2HefnsgdS2t~L*I$P{cG+ithpoY)9)!WDM|&hwy`oYO4)ie@bBr?1xkfy6 zlE@v(Y1W^Io+sL;r@ngYug6{}+ZCz@8cIGZiHoWOW3F)CBZ|QHfie*UbO4MvaUkID za=sCn3A$+FxeVeuM)_$n0$>PotmqZcfHuiq0u{JG1~w2}`>9b9z=x9y1R)83b%_WJ zl3=HtM395`2|*Grh@%6R(1av$UpaBi}L?0UGwwq&t4 zEtqXK!>O-G;GPOWCQNcbSXWK9D;9%E2<``sWb!Dhr$y~rbYfL`aA79EQ3!iBOB61s z)jdQY471`IlMD3!RUtjNtIBkgm(x{5F`XF-TsBsbdVHq0L;>z|hXP#4Mz%Jd?TS2x z5rn5zAMkY*u<E&YC1HRY;M1klS@ju4xIhw&);H^5M>`UrntO8qgVS|2cv6xGr-B*+C>TvdTVvLQ zxY>_az#s&bbFGR7qBt19ryrIzWD=NL z#?WPRWF`+|3`2d%EeT~?;-U8b%|j!86?O^o3?n1CBJWsolb>7>{{8eF1CxNvUY5K3 zw%N`4)k*m(au21F1gZa*F;T!w1%L<~Z=oTQh%wQ|UiT7Bg!V*M;v`}KnVo|GAYR!^ zHrBU5|2WV6aIHkFf9i%yuash%$;``O)Mx^PdeNi~vbBaxSy8}A0CL7i2!>{WF<&Uk zKpmKiHN<9EX}Q&$f%V~LUF%N8Od%Md7)GT0W@b0LDm-awV=I6@uUVgnf4qSOXCOqMMa_JCd}e^+B{=)+E}RS(3qQcj5m67rGE zDMr^rkvnr^g49@H_x1|O{q7^u`-uFWDB?65dIgXJL>IU5*N=Hl)E*N?Px-RtzTY`A zQ3ezHncIVrM;`ywzkW9?$Fez*02lP85DhRLFOa4T!lNGA5JZ4erqGV{%Y!m-18j@G z`6Gx4e>%W{da2o?p@NgLd-w)p5QRKwlh27O!dUy^ZSgy>19LbS@e_K>48UqZB5PpS zR!e~pI1c%Mtr^P!tIC+l5rlipj1GawJdnu9V4Q)7NEnz%4LQaKY>knitl(3O03rz2 zibg|3M??RVGWke>ig3q5vB^vM!C}!!+at%gGRF-hie`d>%wsvg0xkQB$E++Uf1N-s zqF5HPnB`v$k9h0f+#; zG=`0u$vb(C%?JXdyr*yQki-h9ofhomhwE$&wR|GpgU1`1ycY}0F@3(f9nCjfW0qN z%>-p9pGW~NaD(<+DVqoZBG@2_u%HIyEfIyB=}J7bVJuzmIMGB|5-NxNPvU`iF3dR-P_Mz3z!J3%NDU>sX8Q(!Wk^sgw{f0M1Oy}!S?wgBucq{4+Hhi@3Xtayi45CwXJ6-u>KOx09Q^;EAoe}`Kz1q5`|SgoZ< zrB&9E)mz2YPO8;iRhwM()nC0NUKQ4Z0oGzQ);rqHVO3UC5rAWL)@LOnBV5*L-4h9b z)@#MqD9S}@Ks5wb*>(*M)T; zfxRA%mDrC(i&roL?zq^JC6zM7(|6rii0xRG1=*KNTPza9t?k>u z72LtK71#q@!bRM~Ra`-FgTr0i$A#R;mE6e%re;_JAb^0(g#aK}gJz)I&jsDkjg3Ko zfB`Vw(?wkfe;5SOW!=_wU9eDv(na0bOTwy-|&s!@}1xJmEZfNU;MS-@@*vNe;oh_u-&7G-P1i@*?VG=&! z^kpPfI9&*c-lV_=*&ToYUJWMmos^IO&eA-%iwcM^OQ|@AVYo~Pej(C8fhA_*CU#;b z4%diyf8r^2;wT0HD#l_Iu;MJH;w|RlC$?fof>*ZnUhoxTGj577PUA5~TZK?#rg-Bc zV&gL|<26QOIG$rWu46pDV>#|)rHErd&SO5_V?QorLH^^UAmcWYT^VMI8a7=B|7eZ( zvJ^V>$`bihr=aAj;9(?olG6BMFlOUWuHsQ9e`P2(XvZJ8XS0%Pf2I3+%vP}PehX1^U zB;#SeEawF|=Xs6^bzWugMQV4BXQftWf2KBSh}LMSW)O?6XsSkIjMnOn&T6i%hW8FmV6I9<=Jh{?({Q5e$L!elL3$2}7O>+8TZ zH93s906{bWqhXIvlM>og3T}P}y6ZUK`Gy!Rwe`5Qp?+7Ai07hiXLosy-4v>oC9RTaS z>-ivOzvecR;zmR4xYTo{G-K{sB$E&T8|p?1ZmtgkfPp7%344-&gyXr)|JLk<D)#j+!gh-a~#c;8`{$Xy@xEX; zvEOa0KY6kb?lM~N0ch~3c<=#;@OLLUKqs-j7H18Cb)m3%(A?xzwTPnslBC#V?wU@5 zV3R|%h;JCJ@9FELS9$|sdQp!2As2FnKlIVQ{L{|-B8T+Oe}DAPzxJ|6`$;eNtS|ko zKmD#x{jXpBum}5zR;1xBZqr5XZ^S(S&~6QB^`N8qU1w~V*K{d-e*6QO{3dpKh_CwVe|m=>eM#s3NcVov4}JT7{j>ic|FJKB(LaCjZ~dz$ zWJQ8&)0OL{f1qpAy#sZSdxD^}_Da7HmwABr)^A|H901A?OsH_+FaQLM4HN`GL&A3n z0GR66kia*K0D8z9XzU`wg$p+dgaN<+w0`#}0f?DVq5x3@-8d8|@xnQiK7RrYDs(8( zqDGG*O{#P$)22?JLX9eQD%GDcuL3}!bt_k{Sh02ue=9cTz?fpsZYA5#EZVbb*|rV) zHfvnD0LtDyIv43)y?NCd?Hl-S-obeJ5=MO3?_$Gi3l|j}IWgnMiY-?LJh|~^%#Jk^ zl^hy!)zYRC{%($EeKSDR#EmDX$VHx8E6!8Hd=-ldS>B-3Rb9Lhaz4G z;)W)U$YF^LhRC6OFTxmOQ-jE+3R0@b7KS9Wf7v!r5&)o=kSO2*5P>Kc_-4#XJrUFc zkVMtvwCT17~hB z;+dzOd-B<*pA>>s-+grbH&=m%3W#W-{w;bUi;gZX{Qq-m?1rfTY_s=oTCt+%en+5iw>V-z;JLZD73JJuG|J;p%u zjA`F=q|`T}Kr7}_-@NjQvI8+&4?)9F0&STIeRGbsxRP70x#yx=?x5?kODKPezRPI5 z0VXPMey%QruR{7J#4kbqE=Fsz!h^Wb<|Wdym8hYSB*8q|5|^|_10EHU3SJ}dyO&Je`<>z zHrZ#po%Y&7(;ed)vO!>@$hrB^^mx$}75e=XY+y_$qgY_hGP5H>qXqRjK(gCD;5;}0FY`M;lU z{8j3czI^)3uMU6f^1n`h?Do%&f9?9;&VTOy?+$>y6CnD`$3EdD?|lk1AOVZlz~hlG zf~ldAYzDxADQqHxo0tLzD%dxZD1&=Fk{}67Xu|&!M(}(PL?H`X=)xDme=vqIR0ZEW zXhR5W;wQtR4a;Qc!yf`Mh^A5D`h;l2BO)=0N?hVo22n#B4nRlLGl@NRmp#V>*}j6Q3|8h`+SGeQ6eYiLFYC2_@{Y;Q+X6eAt$XvaI^F^_uOBOgr|$4&$# z7c1){Aq#29Ln1Pfid^I(f9=7&pj2^3Z)7ATD{09~VltDO+@!g7ILGRVkdvYuB`Hg3 z%2T4Ul7j4>P<9B&RKhZrvYaI?YiUd8wQ-U^0i5@4>C0aNGnm30CNY)r$Dy>bSjAi> zGn?tmXF~Ikj@%JZTp7)4Vl$iC+$J|mxWf?w|3w|#`(!uEY0h(^e>0uxWRE9_vdT{M zaGmm;Cq3(F&vmY{mqLl88}I4Qe*!e10+nShixNjkkkcn*G$R_-*o=WfG@=rnC`EmE z%%Z$Ao<4CR6!GS_ih?wxA{{A7KlaF?@bi@>plrX|3N{@7LD+y>)-@yJ|j>;>r>7G;cy-X={ecCuoR*W(twVErxN` zk;b)uiT&SV1K8LBKDK~l9jjL-YuU$vr_3dtx7F^!8jcsq6+tm~%x8`Q% zxn^VTbdx(Z=VCXy*_F_CJ#?9?Ui2t{s|~PJTRmXm$vMXf&rj5wTZ$HUX-;bk(~LV@ z;l}rA^F>R2f8|SExG7h<0KTqv3(Q>udl$hZ67YW;Y+dLcIKl~b%vHr1xTFA%3Me}Y zYI7JP^&-c%gEd7)-doZ9(s#dZQSn$-exI#qhAC2OE-^Z(TBg;d1g+s0- z5#yZW7Pqj@PY7!YQ7F#D_muI9VeH@gBA3oBwzGcuoMJ!E*uh5*GLZ@WDMObE(Xmo= z!iIckM<*K6i>}OJZ>kh^+(9&FpADe;VSEmiVM8?(82++v3*N__nXzZI5s5 zuytJu5Lz<@rc9wVKv;-D9MXtq)T7kv_D#D5p{+BOSSVyBfSF?wrhdqS2r)A(#FF3HFJ|L?bh{rkRHrluzH?Id6>;&fc$I0GxkbfQRVW;$H ze|Mx)r0B}SO4$UILCtDu$($iWe-f83l852oIwsO0j+UHbkn#+q9yv&XV3m$etoIk; zkeB+^i@j{iOCIwIe>lb?-ua1tzT%<3IJ8H}gm1+9}-jel8{MOriU(SPlT= z5G3!Y7RW|gLJ-^lzDqmF@;NTj{~V&nf8+v!2(#k>u6c$}p6UXZe$%7x^r}DK;n`OE z*im10@1LFZ-7h`)GYR1%e(1ik=3Np80X$>3v}8f307S zh2Zyz$YD*BN+m`7&0XCEhy6{-%<&#>4FvGYtVFyZK7E+-WS|Jx=;TCpEZY^0-l-vNE+)||6 z0IZzd!C(j^#LmrNm(<|?T?hXTe_{y$ItQ~1gggub6vRZE3?Tw4;UEH66ILD}64w$| zp4gG#6@s86ieL$b-w19U_cfvyJ|ZSYA|!H5ZdqDVh#R@JhPnM79rnX)`P&@Q91ZH< zod8Au<>3DvA4fbxOvKp*m|`GyQxX;;FD4=);+G<3ouF->_+26xb|Er;e_=8*<1#v9 zCMqK|Hls8?qsLU594$pNXqqv}#vOoDD<(vCpocun*3iizINsnxm`5!3;0~gMM}UM# zkc2LtlP?A%FcKR+8e11Wk34l zF;1gg#^p`UrCp9DUY4d_h9+I^*BCCvQ%#RgOqFbiXBs3z_b`=Eu4jc}On?$7hQd{We{QIT$`yxxsD~;Rh>oa; zG8T!RsEIljims@NLKce-(1B7@6M2wNX%Gi_5D6SX8q~pf!V^<{qlF6TkP>N;`V6== z1z)vP1!ceyY{3@vC_~z&ky2@uTIrSI36eSmN3|3L=;#g{fs;aL9qV$sgrI293(=7zEVa7 zYNI;pqY`OfriU_YL70Z=7NmhB=&ASc)0sl*r-EvznpkV9hB~ByrQ&IXUf4kesi?Z@ ztHLTtk*aFI0jgr^c&ciTSW~R>YOnh0Ky7Dxbb_s>e?g|I>KpOWuPW=ZGAlYEsA}Xw zrCMsSCMx&zP%^b@vuf+Ma_cWyD|=wWsZJ}8at|vF#IAO0x~l8CqLP-ThC0-$r5>t} zzTZGtSW~fUzUu3~GLpNdMjhBH9As*_ju4~r>%k&y!q$)W^Br;?8^Sk#;%7lh^wK(fscM-KN##! zaA`+DfM2m}&-(1o=ExqHWq9Mb)B4emDxdCa&%-F7Gz3?>;W@ zf5xbZWo~M8?Ej@g>A)t_LYP)Ryk?(D72l9<(3}GoL_m>1fqJ5@j9_NwwWMTzZ*eN& zBam+h)WFE)ZJ)*foL=ECPLM22mcQFK`403j3%Lpd6NVm5G$Jh1pqa9K)lS_b0uoQzZ{!+0CXQzqzD{80%z>aI@c0$#vjFUm63nqk=QJIzXf1sH4 z;Fh(qNPwCDn8^dLeHojb!!Qhk?8A`dZ3@?#QT za1>i{B#W>lpD-n-@D)SyBx`adZ}KHuGTX=*!>WhdezBh3YqnzAHYP=r7zFMS#E-D!GRMauL-QhgFGNFde+5%B2Gb;G zuCFy`bN@MyvpJLVMuYT6i?m0RbVol}Or>+fzU{}t@bdaZ8J?k1q~RK#TdKj_{q0>Y z-<(X~@Ry^Gj#bx^8`AxL<_MI6I(@F zG+TD2Mvt^fgLO%dwOC&?e{y2|?!}vy z^51#HR_Ze$uj4&@LsA3cK-@x12w_lo#8b24RJ(^mOS4uJF$EX#5p#9=o;5;t_Dp(p zMt^o&CMOh+_E$ewOtI~0Cd+eCXk)$RBZ6AegzhFY(G)8`lZ|@*qe~WOlOJc{{Jj%CJKKFY>H+5S!BNuo? zU-xEzH+PRWgF7*Kf19?0NBCxmUW)M75*Ii62< zaL)Q6R`)|Ee>hhY`l3I$pocVBFS(QNdW9qTzYVbFO8UG?f~sNzhAV_Nwg&XBmZyO^ zL5L&Jm7}PaB0`XQaL1rZ0KhGegi;U1H*iM;D9cbscC61wo(p-OzdM5adFBE7uoruw z3wy9fxxNoNzyJD^H~I?WE60j!)*7pnvamwv%0nW>fA!E`wd=;9XuFx;cd1W=av(SV z0N@Nk7e%fCz(B`@Ah0`+*ZO6;JD+@V{7=Tz7Ndq+ zU#h@XYd^`bLQv2EOz+al#)v1hUpFO@7zvUHgpx${R6a>l1AwY?$uXZ9K|lj8_J&Ou zMLAx@e?O1>dJH(N-+E=w`;Tumk-xm66FbcF`h>@Pzu!E--&n87c51Y?$0BN$r_6aP z{9hh$i3`QCB+GgjgDAX$0T#tO+(KZz`PipM+1t90XLZ_N{@P=A+oN{fbN+TGdgq5e z=+FH*ueG_hu%v%1ndWLg`0MkUjP$09&ESk~ZobUUNIr~EzR8z7aauR#Yah#de*ewe zecT&*>C^o1EC1++x6-9_dT2O1&uqE&O3*u%ckUJ6oGz{~j^m(b;Y3dEzliE0knqZG z?V|tf+OF%aKkSNy=r>Vxlca zj%?Wi2TsO}%KE9x*eT(f0WSwU!G;jq6PN?maMZbzXHTC$fd&;ilxR_-N09UY(j%E7g8ow|)idbu8JkO}U=$dkKO~ z-Yd|+0})J6!37y?(7^{GjBq{aAXBEk%hG_O4wEE7$+MFhYGw@}ASe+6Al7I`!WCI; z(Zv^Gj8VoJX{@nE3LjGj4v<31u%yp45(uLpZ7kBrBauu}$t9U=(#ab$iVPb#n%g3c zI)3a-rHt%(f6~h@!3Zo#yIyB>OARs+F)6P5b%u~-j`Rvor ziGU2vF?-sy56+c96SBWQ8Ew?jM@i7ANo77&s`OFQ>IO>Hx>ISb)z({a%~jX-hOE=EPA;_af65Jkly6H(y4+RSWtnZ(*=M1h zsMpAVu1(}eOBGFS+;`!PSKfK)MKHd|bhH+_ zI8*g?(|Q38Sm1#PE*PnM&HYHEH%rn8)`5nU&ma*=G!aD=4bE8OjXCbvS%lXtv_~vC zOI2cffBq%tzW@bv&EuD0j#=iJ@7z-2#{bw<*h&pK8&Z-_`-zw!>{uqPJDQPBTIr>k zju6MkEE6fiM2KaV(|=vM$+Xl|W9=rFn+{v-vB@sGx2GM8T3B_HD;MIG?8*8inP{>J zrku>q8@9pR;(IN>6+`PSwE}0VaKZ+cD{;COf48e~ydKvpa=s?_t8&0D7mV-2``$cr z!#+n0uFw}FU2wonH(YO?&~8kUU^A1XVUV|#WD4%QTZrhFz+T<=qfigtbKw(99dY7E zN1kxwJ1<^&#h+(9ddH`Syn4y6r#yShf43-2YJ*L8_C*Dn%nt4ZT?iWh2gv!e> ze|u(?Lc#v~@z4MM&h_s7YYC*pLu1WP}STp+ZiWkQ5?hg$HS&L0(vp7!qWL1F2zEaHzqx%#R{g z`&Ia`1)A>_Bo_dJK>8T6J^>j>0strje-D*aKL_H^fK#lZ1FyKiEFN%+1ni;!6Iekp zQt*ai$sq?5h`}`K@QfZ@BO2QX!Z*4xjv}0+9P3EJJK8aiCfuVQ`-s9n`Z16y9Hbzj z*uz9YCKu-tRvqflvwj&Q4lr@wLsS710o|wp7NpXty`=b9&$-t_Ve{z2f zWF-nx`9N2)5`ts2Wf^gKNJAEqh5x$jB`;m*%U}W%hQcf+F=c4XWFnJ>%4{YxZRpHs zLNkYMfukAOD9VWVurWH)8P7^Kq*)CFKu@5E-5_v~d%Qy+&$z`o`tgo>6yzQ97)U(@ zVvJ!JW1a(%2Rr+rPh$+^9K#p}e>4vA4TC;IBJg8MD79%TQkF872wdeWD@w&#QgowM z>?jt)Xv;C?lA74GCLOVP&6DCWr6^q~A6v@OmjW`TFr6tOYf96bA~L5q-RUlS%G00z zGN?d}=ta1dBPuy3Q<|b(Kbip(Bt8TiFu@o>a^OTRC`AykWH|T?If02x8W;mQ^YO7^qvp zl83fNKq&%Xz*5-ikafxf8U}D;ZfQkWkV3YwVtFMj9gE8Ns?xm}ov$nFcG1DcB+2=bO;ThNODsrM1#pXjxOgDajj>YPaL=@A z7rQWMv{D+eXaH#79^Wu87YK3-rMw~{PQG)gyxdqUf0@fwPW5A3o#j`{ch#yEOPI+R zXIs>KnhYW;MSp>Tg>E zT%)yywNh{`=3Vz%&dN@S~i9>wZoQ*ieA8zrA(-hwYc{XDbQ6H&EyE?}rh_x+} z?QJtRu#6Unf1}k+)^eX0yEuM1jK59YseszvX?=3XBSr958l2VwKez(zOZ0ss{mMo! zI>Hkk!G>de;u_DmXD@E`s$+fQpv^khukQ7(^AzX<`8ZOl)g$~QLQ^05S)d1SZHcHn z#plg=xF5Z4ftWkTH^zI;Wz2vO44&{8NVh}oU3N(cf8CV#4LIZBtMq=XuR}(f@yb^j{Zy*hOpl)Mr!i{BalfFr=Z)xC_wz5b*#=ToDtanAr^*?fBDf5_N||O{i{bkX-OY{)x)36(r+DOI^jqk0XIr{HttK1t0I)E?gA?h06+rz zsUNaR+;EQg`tJCKuOD0t8XygIm`Xul1zVF{YaNt7l=-#XH8V}48j{MZ` z{8lji*pCHY&;{ME{pfG(YViFIZ2pMK{-Pu@e;P?)_Rr63W+0;KC8o+Ds_G@K$^i$0 z5-Lz03{dzQaPMgU&iMY0-8M}iU_dFt%NWvX$uQ9FG|*8vF#9~P1h3EX%5V)w@ae)Y z^HeYfTW}8LunuPs2JcV?Z;%Fa&<6Vu2NSIRTJQdl!kd_?B%%W)hNvIP>6{F$_XvUl zf1iQ@^-ck+(C=Nze{LuOf{gM1cV0 zN)#?&y28o;;AtRyEYjYAyGSjc#Ht%5f3VvI0vaxWDS&Ju!Vv(bkrYeO6}9gaSy2t? z@eEf{6`Ky~qW>-q^NG<~i7oGKO#AL4(jYvteT<_XF5nXxpewzy zD+wS7RFYC&lKW=zEccNnBkvw-(jQYW7K_r?dJ-v*(kKlwA@6b_^RglL@*(>YD3en0 zCh}nT3_oU1xZDnCmy^3W9(LD6=vvvj^gFEH`j?IMaDLGkHF9dA0|0 zMpJV<^E5&K6E#IMG@*wzrN=d?2R5xoHYG>;Y7c5eZfEl2?QXLl5SHCvN8U(-2b6FO&8It5hk`KHCCfFiZedtb3W-aJj4_K!iOZ114NFcxTKPOibmU-u|5g3Kn?Ujb0n!A zBkisQOE|K`rz`F*HMU;_Qwh?X1K>i)A_<1|cZML{20{G&Dt3bVXUT zA2`$~@=s?DMNV9ld|LEIe{nQNHS|S`!WiEqPKJmWb#zFHv`96RM~T81U1dKdAvAi^ zNTD=JrT-MlI;96Iq)ntIQ<&64kaSAD^h?21YzXlvawba+6G&#oQ^Isj*|bfSW=xF& z8$4p4MvV?m=rN<^iKOUQ5AJjf4fCU8N(hzu1A8X z6Bf}?8TC>zHB(6?LXRR+*QQ~tMpiU+R7tf|AB9tm;&mi71^S zi=tXSwN;5kAX0`1N5n)ghE|1jScw%v@)Yd^)k+xVBo-w-1!6)21OgtjSgEyIt(83P z6YZ!3Ga#vCo~BBhf3zQVCqM=CTFv!b(e*d7^(cN2GkB&$h|y$zvoub_J-r4drZrvl zbzk}awGb7w7`Ih7t7a)(&u-=>I1Lsj`ju3;Q*|2lc)ZgzQ*&aMlVY3GVxJRZqf=w2 zlVhusIiyQQ2YS$AO%ux2@)?6QZMh8FZ0$f_ZBegQZE0Nz#=MY zgMv+xHFm}ae`_}qB0O^C)Fb$uA_Nl9Y;yx`!B1^r5_8QmE#Vd}=h80wmM;HNbVC<( z=l|AjPZw`fS8rFBZ#ho-2p1><)gnQIxCUZ9y;gTLX+5yZiHso=bSyI|_ckmS1T{Bp z)AD%1uXAy7C{4F+RTp|$S9)ESdi{2EpLZa&mu|Ore+Q*6Dj{QbopN@prV%Fsozw}S z+zFoKiJt7Kt=?ClCikE6X`cwnpvbKt6lxeq(ji1)#_ox`5O8=amu=IqEe#kIWiojg zcx{;%^dPwO;xcYg*Lq2}b+5N{0T+8WScCa%4F8lUAeB5xu0sH=2nV1l)yJv`(8!z% z+{lpLt}Br2 z9hDCh0pRcy!sP^_u8P=*Wdn&xkcl^UZIxIAe;-(aKevN1_<}7sd%5?KD|m~iSCXrj zk~4UbOX{e;cqqa+Ln6tkI^*_ok0N%j?pCbDvapSpg2wv6#&E26%YesxOss-z$cjuU zkZc%~?D-S|p9aDhtPzI&_=ppjE#0=50oe@;`REQgim~{LJs6rXS&N5&Cj} zdWjYKAEo)Ci#nSjTBEsGsWW=1ow})^nxdIns=HYOZIviu)LjqL9wv#TvCX8Z(D#HH z<|44A9}uS7%_{VnA>;uHmVy8jZKnqsnd#b~3p%Ki*old{iW3>5sXD4H8mav{f3PbW zn+sc;pZc#|YYeY?D6;xK4ig*l%&X@Ptj&6zk*}=zuIA7>3j>ev3J;GH;vBgD>#XFO zct-(+r8Txu;W{?-&?*jIiimBD5iE{gj-dlql^Xcj0un+3~+#{@VS$lv!^?yB{4JS z&7T#*3F*1Jaigd8k-QUnyfwVM7uuP1`?mY~s1G}-;Ty5-o5W8Xy-j?ve>HGjjbdsK zS2LnxBLW45RahaCZ~&Gtx|{F`qY%LzP{ENerTZbLyYLIII~uv_6-=SYoBuqnDLgVP zyd_)vt}z^_UmMp{9L4E-zU#ZoSNzMx+{?*4%*&k1$K1@-yw{G(#d{P^b_O~w5>G+I zd?f;X2Rt360?4CV$RltQe*9!mQuxG~WoW6=@#yD$9Fu^cAN8@DIjwrBgpqutVXf7__5U9cAp)KQkV zLM}@=BRX79ITn|yj0hw}vVV=jBvF#s!^7B*8QH14(epaX!)(*9eZ=km{n@3x(yd+J zsomdkY7ArdCuX-iHzYdt<9nv^h`830vT`fGGUCIM-BBCf8C}^OeULHUkURXmIX%Tc z9?U&G(?dSZNB-pVf1UFL-bP)AJnQs4-OfW6Gk1r^F~1`+C(|-56Ei8Er%9G+eb#A# zR_KM6=!te{jTY&V_GoXG>2ucTlOF1qUg?{j>Yx5;p_b(t<8{mw8G~j-f0J&UEn(5P z=gZ__o5$&uR_%d)=!5?4-9GM%9`5H}?vGyTs~+#K-tMEGf9~(z?>)q$_q0lH#-pVML_5QF|C3w=gytpx^Fg0TH~(2E9=pGlVu&uTNL(zLiP_|KMu9>dB6CLUq^uiRvRN1 zuyje;MNN-CMEa$lMSnXeV0>rJlw0z9`nkXR<+J*M;`+sTOWS3fy#M^se=NcOCzPSv z85!0XZ8}U@l3Ll(oc`D9x5MX%fMa)Cs4g zfj1@Ps>zqZlK~cgRF&b>$&#l}phAiN7L}w0#^ZQH<*BTJr4xw7TUm@{kM%o*fhvMePX?uj!g=%hcg zW6PdRd+XrTp+_f1J@75jg3Z8%|Fc$iWnwl(@+h92mHsgeXTgzw$bY%4#TZf{p7+F$2LE9bqw00K>E<&;%ksil)y zZh2*wUSio#m|u=brkH1C z8Re2Pmj0w;rk#8?kSd-Mxezu54ghLiqQKy!06`3c?Yvq!iNd}3=BqEh%I@p0zWx3y zaKHk8`%7@a0J91J!UikM@V?6bJ|gkN6jwa4vJzn&k;V{l{E){Ef!vVD43WH$$qJ#I zkjex`UFQB*)0}eKLtWyE<@;=2Z3I0OqoZ| zR8M8^!}%VJwbl=F&9K)9gI%!L11oIt*%ixwOpw|Gv7P1HH^ZH?+&j}9vd(wgJu=>V z>s>P6fBSth;DZaEGU10CzB1yASKZgoEGg4dhD0YO>3HX@`w+S2ZGx9EP7=__Fi`^_ zfH97z&eYXd-%IwtuQ!}!>-;)QI|{KwY&-1{qwTxfW-8t?n(;1d)D`Lsdzyd%&v zx17_@JM|RwjywbTpFoT;3}f_fAo8$ZKM06Aft+I)!+^#?z5&5!M5GJ=Abdk8kplbnL&003p+ zf|Eoc2mok+AfO|_H~KM*3^pPGPZB^T1o93%3L*qgioi2U=cFFo1A$3^QUpYQ@en}v zfFSk2;|Hxs$&`%Hg(u|y-3!0A$*^^DY@aM!CRvyz8O~6LJ>*#sRXIdf7SWYQgk=(C zxkOq%(Uw!hWfiH|JSpx{m%ZfW^n`gsUj`GH!$f8=D=8N&mPLz*8H;H;H2@A+Bt-*A zk{P4KMgW9>Aa#@@9j$b_{oyfxOjFxm1m4AfC+)Ezlz_rJIp{%V+LI)ggyASLX{OuZ zE|jnXsO{B9+E8U~)T0=!9!DuU zQjp&BDb8Gpbw-n^oO}id=9z*;reGc*803t0d`3Om_{M>RbB=X%k|6qjSGo`O(Me^f zM?HEFIz~PNk0}s{8vg;1&S4k;00vOtLyU2YJUA(XV?FG;3MWN=ml! zldPUZXhJDQQj^v-qa~dvDm#i-yYBU_BK50Y2dmML{!p-D#P=5w(fU zEk@u9O0~w-t!L$}D1p0H;Re*Vy*1NZ)ArZq5;m`evjB09oDv+O!lK>JN$QjFjU`_zc)4l@PY)~5k zjRHdhbO2!FL+Szlg9|_@v*3;J&HS0d+97wi#vN#e4@%tMZrFs%UGBx48(qL^_r%^s zaVxvK;_bFr#jj*>j9uJ430Fm2Ey<2~CGnrQ7-hYbwd^+eJ73O<6Tkx5Z+|Cw-#G@@ zjpMwMYb7F&HUgl3v^LK2;3#ZR5Pw*O7d~!>llx^0j~T-N5b=ngi&*Hoxy^8vaV+IL zON6~+aAr-^_j|>*ZQGgHwkFQRwt2;NGO=w>oJ?$UVw)3lt~dAd)O)_2b9Qy@)!nta zzjg20wbuIm`BoQZ+J;=2AN3`fSRg^=X9A-d_Py>*7(F}B$Ip8>pqn;sp?2x-_kBo)PNClo_tGt2 z2FQB7qfC3w^MCuCyz6^q{-G5%qRCQqyEDD-b0ojxk4%OEzo#qJzmoTOP2qBqfrKCh zyK-CT!r1o(<9z{E7cuvQ;?=*5;;g|s@wJJ5q;E*a6m;%OEQ52wb zZyJ=ExJ2+${p=}?J^@Ihs~V`rh$jz}qE3Mq<$6=AHjzLWx25m46zz_rV_vK2{(Eh~ z2QUFlv;jva!ECT0pcBgg9ULDu9CwuEAeQ-n69~n>&je*joCiNq6%P#2oKZ#qNM~B; zBc$;4b0X+$h()CEWd$w*r$JPwj?=4QH0@zg&tWSOZgyoM;b8$qmLX^5;nuVvT&LkY zk>TI0++BfQf;#OO()S@Ejgl7_I1p;g$j+hXMOcqcV11*2$c)HH8Gxu$#N<=ZD|8e9 zCY<~y%58S?Fvjhxzr{0}ERws$gZW-@(8vU*SwO<~gJyc$sl9e865j0g z)(Oq=&Z@}Fs;b1vi$X6LCyHvsHbcv-*2!-5&TjWM^|vNVDrPk^Om58pJ(nak~O6G|!dO-)706Q?W4o6r_3-!@e z__bM>dQ=)f*#w~>LY6H;#Sn&qDT<*XiX|#D?o;oVR6pNV&$UrsN?!}8c3YOKTZOMX zkZbtLVieDA*nDeVmurzG5JI0UNL|Q2SvC!sH-z|+bs{FU#;v&}VT6F`>5&@bGb9}V zDFenUqhcZxSt0`ujRtI>0;o87vd7e?TDfua`@8G;^CEaJ;x{avSBZ7C$W1l`T{MIs zG=fkNT6VHpq&DOg-;-YFG4O-wB%DczEfukX6y>taHkYI}Ej*LBsgB$G-6BjjZ}4hD05~U4FIdgr;5gBn3UYG2Hv5D3YDgkn&$T-PsDx= zG!+z=lTI(NE-(46Cg0|V>gMF~u8xhiy@{^hdMyCN_CEgZ5xwq#>UPYC%n57sGg+&v zNQ_8sv}g|IXOXup?^rR z6$#OlV}ZVs+{1BGE`T4_o)Ol70_&Vs>pUq}JYFj7sd3l6as7;udxD8sxxq=SNstiX zki^vVw!qL%&FB{5%pv2<9^+Tm+jPhI=v-992iX`i72GUA4<=g=dAe@y2Fmuxmsb3k zd-`w}kJ9O}@I@NxRT-56^t21vv~l~$s@-(+25_WQprKyBZ#{POz;5O!b_Q>H{-kCG zpJ<`_B?DIueWj2LJ`2MUjWSIhEyH&fT-1d+!sR2@Hvw@DN8fk!&o97SAVUA}+$~aO zppItE9D>2fqsNHF)Z9;*X_k-rSN_ok{dt(!c@q0&0;C1v++}U11>MhO1HpxOodpW# z1)!f7(U;ibvGFf>XAcO}0xbnp`||$JTaU-+Mb~uSM8dhx0?(V!AZ^V#WHcoZq1BR6 z|B}_^9Gc)njN*jq^aLi;GPdHfr{8><;@TPE%8zMPXW12U;}yPOLuXY?v7$y9*hLI0 zsp4zwx|7GGyQbJ(73HbMXQJD2SWV?22sL8@Jd5@1;SM*2@0Y*Lk0mzgD_#K zARAiMBQ?Y9H}vN>y8L&<(l=Ny9UTbQ5_`HP{g=l+ci!c9s}(lUVlo%L(e=Rp5(WN2 z2_dU7N#&Gn4^@;wSH=x{=)rr+-RbSwK|-3JU)=%ZZDtDY!V0ZX&unIG?Sp9JwX>qi zD3j_w(L&EzJp#2UGUXz}DP8{a`^0?1qd#LW*988J>6eKevWgY0!||eLvQ=V=n>cp7K915Y_0c`? zvpETfJ_)Qk37$9!JwFMDKaGTEVk53hK(N=lPC6$|c<@4%!FRD%VzCckN+FJORys>} zICGt;$iA-d4mc_JcUIJUQmAxZynRv{e_sBNA*{D3bpb=C!)KX^_!>W%QVm#4H6D}H zd(pjp5q;Ex3v&@gM!aF2LLME{n|C?Ud#T9QNI4HT@Gr-CF9c;G6_vCeG?jO?-g`BR zd`ahwaYGoo8x;Qumj;-)I?cO2??sytLVu*oiqy%v5W0CVyb(FOI;F$E{l~Fqee>_% z4ZsNX*X8xv9!XShE?E98yb>i(gf5AOS%s6yV+=z^Uu5*}$#3V`4DXqo?HQy``8YsVyzF*C zWm0@fb{w$t5?`MjNP-?bTZT1oDotOU`TmlrdbZ|O5xsuqg#6}m{pyx?4s>`6ihm1X zKFwEt%ej8bb2x9RKL>U-WhUL=y(ztd?;T~yp}JJP=Jdb9Q@mv;ePkiO7bbiZ&%TxZ zKCis_sIGsjo&Bg+{%ll!{{wvMXv&mMc>ff=BU(U7n^=J$i|j$cp8El*-|+7e1?nE@ zL+*1=>2o~R{_s)QpVDwwy>3$x8TAtq9ug2(&>sYcMbGHvvdGTQMz0xIY>$rmOGZgD?W-Qvox`&)}gjcq&E$| zKJ{)nOD|Hd55>cKJXoz;ohdWe;I=PQx5Ws9piy1WS2z$MDzGV|-_Lf)#)1?9M?0PK z&CxU5dUL;A`?ctxhg)ZKvx^XeqwufZyZ7`l+-x&|?F<-MWjvWa?c;hm{=#tadrFw& z^8>J{-1iUmW43qp<6^203QZuce~wV!o*$l}C9!7HVX%yL=Nyz+lA<3#ejSyS7YH95 zaqTT#VChhJo$G_?Bb^syfjO4(r8DkK>aBGNBW()JXPIGK;C)I!o}Wnv2e@m$m8C6C zb(v`Hya&!OJzh?@G=$XU#IUlorOGP|YP;S{N$>K@_QUFADG%^$)hN#Z_Hvqk@tqOQ z%?SM6RG$TZ{aBn80TY~?gGRGYnS-cwdzh!?Xt5vUMy+K$ppL~W1W#RKwmp6?M8XD(@`W{7b?shOo{dg#E{ zb|+8W^!aJ6-&PRKuHQ8XCJ1tFsHaeJ%ZI4wiz1kF&b#7(dK@=jCImSRT(IJBwmpt& z)3>?SrdJKzR*!hZEFiF77{9I>m1H@zf7ap{FuS|d$OXC(OLYKNtUy$Bm!F#78JJ}P zNrCo{UWe4Sal59_?M3dCdT?IDsZ#hI=0CD*TVIbqyNA>^9=r_e<#P)^(N9_ZSCTgz zb`YMwsOVd()=rOcZ#&-EBc3)DniKf_wZE6&&g*Nj`9&)SsR=Aw7P%{bm^-LTCrd2Q zo|m^71=>Vsmj%MZMjQE;<*9QSV%5Zhp#mvK48eG{%v3Kb%VFvx{i# zO7pL`Y(Rx9yu?+QQri ze0kt@G%?YuCYDhE70D!{TACt3p};D{Pt>&uT`pd-qAJt4Kq;)kLi(6b#{lq3X*~Fa z5WH~JSX0O-2o_wXjYq~f;p`vtD_$);a;zcvcCoOaVjMAnI;T()H)9f2uX5ohy2KFt zk`DMaT6XFtF{nwXr~NY4f>TubooZPW)dz7^szX0;*&PfWE>tRZHm%WpN+9Mm&k#*A zBH-47(=DE&$ZCicGdW_K?y*F@)W(C*rk~g3XCYiz1(2YvbFSK?p}K>T znuzIKI+@)h3!uHot7v z3o|P-Mii)ZZ{FrN!hxJ^6kAtaoIV4c4lQjqVGOr2zWPd^5~Xc_qZ|Vkrxira)!EK4 zHH3a@2dRKJ-v!buE1u0z4Kz1BV8SB8H?f<7hUn8s(6oq69^zHN0t3cM9sNDQH&!~* zexWVjSpP-nUw-qf>4NODv({5JBX4j0ZEoyLq_Dcz`q)nt*8tR~iQRx(`#y-!7~ztR zL0&BIy)+kBHEJemD?s1FEKyO0c35x1WBo>2Ushyd#unWQv4Iq4R}MDtmoX{KH_MOD z8;Cf+kw4gYk+TuZsXoRXB7fCeBG8(-iU9xU=_ zvSmgagZ_(i29v!t6%6k<#6PO?h_kbh;7(9s(Q#E!e$EjzFmu2lt)YtT(uRI;eWGdP zE-fs3Pkz;N%A7emqIDIeNL!MA39D7%+8EJ_5Q}L9br{GX#e-qBIe=LoX;)WUyQlE% zxmY{x(I$(u;wZed1lL8N|G_Y}M|=;vFV%zJFs6TDJ4;$Pa> zerV$myjFeo`E#Pw)TQ2YZ8Y08`o{d!XiIqur0D`KNHSEe+tH#q?RsvEbVKhW?!qk+ zMgE{j&>pljyvK&`-W^r)YVag}$ZLDwo_6qQDK@;SvgE(my7p-wG`wyq_dOA3_gX&? zeD-Pk%HssRZoLb>1djUOL>RqHD`Wj_Q585C-E7@NB)>z>9-yxI9Yu$m#w(&6@Z1CP zzl{JIeH_pTE2ar&A1B~7e8ML6`w02JK=_?}6)AsS0f8SUMxZDGkX-2#8T4 z;t~(V6}>QY8vo8O=%dULqpuNT=oVw_j-tvzBKbfiW5h+85@RD2XJ-`W&=Bo4siS@n z3mNa@@Dt~a75|naJ}oc40h{D+3J-4v?_MA-{2?xaC?N_-%fE>f`WLm#ijtNyLlkHy zA?+t26Dwg(C$UZ%?PM}|gfAkyDWP;Jq5L7y6eY! zZY)q1*-r`?t<^25KP71pJNR7z)SaI=jHfRlc_@McDT+qfZ=fJ$t}kWb)@NweYox(U zb{UaM-}eH+du=OazbWN#DK+RjvIcDb)?qwYdCJPq+Gi_}^m;GlVJGeBCtY>^`&Xn? zU-Upw0p0#?;CWZ?Wp=OKS5-fvOc0^Wd6M+6g3%{+i7^7S2@Kdt3gjt@F(po+Xw04i zgr5E8vFIt8c*x$k2bnm;@p#6b1WMT?$nm7Zt{6(epNz6U31vbQ5PTp91GIrMx$lwB zA<@888CkdS#ryFkh>2y4i4}^8RgQ^2;uCA?6YKo4DGKBHZWC##6BVHoOCjUdEpg0t zzm>e2E3C#MaCs}VrE}0iQ@}s~FsBf?gK^d_Uh>Iydtdq3?8&x6UZ}1~7mN-)ijE}P zj%2-##DmG6hsj>Zsg#YBpd4V|PcisXTn=Lh`AF1BdAV%N+YAIN##Ti3wie!Yv!)qH zR{MK}&O?Q+DfxbZwnYW50YZgAx3-lJ`JXY0X};3~snf~eV@&!8`9G#g6GOQOCJ$ud zTh8h*S)ol6TV5v=raz`A7CGlr_WUrYRYY_1Wj_Kq}zbGg0I`8w->8VpahiI2v(s2LML009UXfUW2Z3;=pTtD-`c`=~+ttDMBD zNFyoAL<6b0o3YZK1VL5v3FXeJR(H0A`KE*j6}JnqAiMzhq@l1L4=%KV&@ zt5u`_Im1vp7Y}S!Qsz=;V_Ier)MUq6<)B)12V-hk4P9C4c z>WjO2j$49}`5g80g^ubqy6Am8|8u@3!AT1s1Kq@ZKw3>8T1{kHO%nW`jv(WyiQuqKTpq~bdIu!aMPicZn%CdL1#@rIb}q`x zTGJ>ZXBa4wHeDen$~~583IA0ef%Gd&nxAlCQKAw{(+^ zbdx1DgA~?UcFTSzgT52CZI0#=n_o8MYsY8XF1u7wwm`Au6s`H$ z>*bWzv}=~fca-}Ll>6gHsWdJI&D8fScik@WkB^lA9@V-@wCa`kKCHizBy z_kV8g6B!;T8XlJFn?p*2LQ2v*v%A*_qE+=IT(^J$f9SB~Hw5F#b942IlhG=RBD+i1 zsFwBCrWw|ucs9(9HhT=VripfzwYRx82af*iLT$fJuYU+`|I;)SG?6;Vl|KDc9H@dD zjFy?zWIVB|8C3)21Mv-RKI{5a*-En60HmDpR-wV#8bN`C5Dl{rwN;Obwa@NGc8bO@ zOp3tD;hj&cZIHn>Zr&cw%Fdhn9zh*OFyU^;f{f+ssn0ptZd6I9NI<14rBbTn@(6FM_5I( z@jOSvZnO9W2u9?75s?^HlPpl&7Fp7?Wh@C!_c)rc!bC=>5@n)L;o#r-;|bLR1fmLA9Q z9=g_L+U()Fi$i-3#7EpK76iyvZ(OIg)TjPnmU`WBPw&6_9yPYHKn45h_{ES zw})4sAlV+v5t}9*pCtX`w2{&eBL-T9ZXc%x9Hjj-c%s1VyznbdnR;Gaa$eGMUOIGM)-q9Xd|vr< zUIl&ehXN$?pb!&W`ORTKdK_XhB2~R4T)Ygsx1CxjwdSI{<)RYvqGsiyb_94)^>9&s zd{Me`-j97*#%i8hY0hZ-b;wStCEWq% zZW_`vLiS5a_R9L|MjFDW*;2L&(vTYV>vi@U|HK- z#Ey@Kj!yv{cS`2gEs`q^j&F62@7PVxLI`DJiC4Xjpnr~l{L&9@gt`{#F=Qu5Whbcb zB@f&n1XhK4Xd@@Md?$FzG+3G#7^B~ksRQsgPADKJ>5kjRvD>3_Md%mLgeN`7<3_qVz-+jgMkwx5i6D*>kZ7+85w5Psl$e&C0B6u@~DJb4&uefUP>F8bYF4CY}${o#CNBHv1QoZB*} z8FacA3eNn;-Eha9LfcY~1|e~!-51$9S;!-W`2cu)_>=iG11NQ7L33pF{mk;l;G4yW zBnpDyN7Sf*WkwI4EJbDk(o+-eQ~Q)<|CXhHo`t!iu|>Vddzfcn+Owz8lS-aPy2Yg3 zY;@>@Vz4X6-cTlcAXqTV%gchg3lD-TP|0o9z+KtXlx2I3Jm+$j&`tfn}sFxqg zt_9(3cD$HuAJ8;&?jS62R=iXhHwo>dTN$*J6nZ*K@~Rw!n@s*KnQ}ZT7aTyN=oJRq zH=%h7kJyPI`IXZ86u4T)B-uylfQygXj~4 zOWgBQpH7YRDB1`vt&VjetB(W_2%cY$UO?i5$?Q|a0z6aUfAdOs3sriH1^NC)@heaF zsrtQXug09k_ajSY_Z!pC-t$0-xbJ7+}}R^-SY)MSM#v6+v+z0cJmFPBLl!;&>3l2!2!6?IP7+(w`4<+=pY)k z8mqtLBeD2AE|;f&DaL{UfC5_UJ4(3}3aEI6Ga&lWL;#b~LhQGcGWl#u=VQpeFOo@y zkU2O~mCNavv|Kv7eWymbWCd2KS_$8Ll|&8NFR{b7xoYtSq+xc-uDKe~7V;$I+6nIS zjY{XO@EY4^=Iw5uXXHytS#<7R6p&mn>yJL_xNKx`)(i*{_~2lG6)7(wAjt8P86!~Y z7U-NAaDxsQ<4kO}2ufivf}VrLx3e0NimuAAK4j|e^t`{mdgr;_b&7-kl>I1~IgpNH zkWT7;bv>F-xNx)SsniW#gK+?z8C-jIO6-OI=3Di9tvOjP-|5`;f3Lea8U=y)b^<;d zo*xlzZa~6;*soC9z)-yK43pSz&>~h4z*-ADi>$&YypWF1>7kM;YReO=&^taKpm_;<2VpUFPa|&hNeT^-5K@T3{#QkAPGrdzN>%&AD?D6d+ z1?scIYAS9F6vvMK{2)&o!FBX zCFAr61?<*!ud4_ViiPu-5nzo!fFZ%*B6AG*(DbPJINO=J!YGE+eH>6d_ezo~fFFI| zCMWn=dL9GZ)XOD?VQu0*woe`70XMWxigs~6TM2M&Ra*m_TpwGFy;6vmLjzh0cIjWi zzl&kSo%6p9q@I_zqm&VLcVko=UF?%+TntfJ14!*7#)MWw{l>OGc5r5|f&G{*O&B;Z--Kc~2LQzkk3mKJ}U5FrN1-(bcdg z(zW$84mqhawhg~YoY&+EFK5qnp6nR&oR^MHw`U^pHQS=c=sDC8? z$ZSu|K&TG53Ew?1w&d+<0j(a|3A9#F5Gm+v5Ursb80HsKX@&T&em$2zuIgJ>`S!L) zOuw)6CLsA;%a+D%4TgQuV0MFPc0Tsz2!uZGW}2D&AHSHz?N?j;(bGm952Nje%0C`L zgEJ9YI3p?cUhReV-gG((@HZD{7Xg%gfe~?1oSA^wy&f?1hCo6zpy68NGe2aWV$h)J zuDsil2<#K2{iyH6Q(d~~vHhG6;SwpVFs&F$vWhPmBiZASX&;sxykGL^(942E^I*jQ zc@lqYHheN9fHRJ?h%_*QoHQC?*P8p5Tn5F97C%%%k+hjiq#F_ol=*0|D-KzKBl1TnO34#2h1!T5;UX`O zEu=hhk?@`_y}1zX*x|u{O|3Znlui-fHn#zo$5VLsrAB-wklx!8vWQm z8>G4H)GoeKS%t6~6>Bf_p5fPqrCRG#-0d{hs&u05Gdl@0VZQ2U4=vr9_#=24xyi0NmX zE8LqHFbJ_IvoB5^IkS4}%B8Jq`;*yg@Pgpv>BKpRELISDA(jgO27Vm|KwSy^_IIJJ z_o;^^;J(^uHTgSl?1eP&nPC?UqrDGWh&A}^(FnwM;T$K=8cM;q2dCaXfbB6B9HTQA z(5>Ky^N%%BoN*s5#j6z$I-adGW(>W#eV91_$QEmA_X`=tZSYg1KGwD7fatz`RA?rK zt%Dve2=ZlAN{Br)?y`E0yzzsz{fh6`~PVc4Gn z$wA>GpH5_0H3<5!4O;VQy_{tnxn@%D)MHN#Atm+MDvBwcOJzcjVc)h0un;I)xF~9^ zRo^${iaS?Yd%5bm_$_Hr)|O4oWpp%SR`hM(nntu3Ey{f^yQtr*CLJ!DtJg2PrS$5S z%NUyKnbz!Fv)9&J?b-Vms7!6PEE@wHt z==EQhqLiP*9IxO8uM-{%_qlIw--K`Q?)_%`^6@WdK%;<4*PQq016<^M;M~v{Fp}SF z-??-Bcs(5^pThI-gz;!m{p=wzT|Y-7f=|^DZW8%g|Cs05XU1EIr*Spz(i6gWb83d? zS9a`KUuz?7X5}_F5KV(b%!|#kc z;?*N^_?OT0PXOr&?g*=ZX`&-wCpiXxeBOZ#eJ!lFHPpy&UmEYz$>Zu=Ex&h6b4IDI znaC+mnE#lEAh?i)QqOh6)%GePi^=iSiqCXfELI;qpu;r~p?ZPGR;$;1RFWQBl*hNS- zMl3Q$$SQp8ARSZKi*wfm0)kppVtz4>fds_`FG&hur)6Nl=V*679^gWj#04qQES5$% z5=StPNC2dJtcST55<~o?kbpxG3fwORDl|gSfEHFBA+}2awxJP_*5^zAaOc%KpPW9FRwAF#AfHx(q%-_% z6Jk3IR+N@xiL>xupbVskiLJlM#Lprui$b2fO}adl)!`zB6%&G){cqbQctsS9RmC{_ zMYvBn%qu!9`*SS73Aiu?G&XoL3)uWqOeAa0iMEVMnQkdN0Yz?ZNKQyn2-F}d_&!#v zVq53py)aXr;9{xhecpenZ^NeiO{VzKhxnZoOvxooGY2wNX0pfRNLBcbFZjm0s$Y|r7z+OIxcgF-{(mJZelN#^XVA*OKk<5TL!r7O(31P zs#&-NA8ETB1^Ji`BS9InlB7$zO-FBDOP(u~XYCfBo67$#l|eERobeJ^QC4y5v;hJ7 z%YRSR&#jEJik!f$O()JweasSxE0g6c^Mopbug?;bsgQ)M9T5jMHA6qvaz!o^?f;=d zq9owY0Ck{Xikmy%};SmcE5csakL?7A~Ay9@1lDldB)?Rq;ddx;3g;WwwybEdZK z##ihH`pFz#VMH^R>zP7_EojC-hHJlp<}35`1XJKSGx3Zw3zv)Qb~92}tz1{*y7m($ zR}(h&lP*`2?0Pf|;%7uPQs3H0)_M~GxH2o$1v$^d(7-xq>WnyJvOB8jp*_Ol4A9SV>|5T72PJ&OqZqqL_{m0|2h<$ zXF=O#8!`~Kp|{~*CQvgDe(53|v(;TY*Pc6*-2bheDd09?NkzVpz`R%{q>7X&!GDG( z*s=UocK%zpnh@i10H3Y{2r_9Rvc02vw;2?xA%SV4{awf%TyKa2 z85qTRCGkbn2xCn6CD{S@{q_}F==C!3?x*1`6XahcaXYEdF4iUyU?e!kLeq~wF2IR< z;2i^SrCCGwu7A6crJR9-q?sMmD3ZX=j>5@C;3kUE(iy)2+7AmO4!^|SI{Fbq@vk6P z-$n_%AOew;JvkhVynr+w5&}6?Kwy_Tz(!mEIu3aVObkq5v{~@IJWK43Qoe~)ir}t~!KJ^-_1wMzbuetmD~^I;`&jcTgy~hK zagNw6A~`&)IpIr1R7`!?AxF>vyL5Mo0Z#_0tqL*RLaS~B_jga@t@<0kJV!ljnz_9a zp6w#q>`z#ML742ppeaf&7$jlvHd_~|$_W(F_;S%sDZ4Rfdx@dNb{A^YBT4l(1MRk; z6z<>?4ja!qy+seP)u*o?VHNS0PY+g7V}GmSyzD3`M+WMgT% zXK`AW_dc4S@1DW7Jc`A?6tmqE&RvN(+HJC*J^8(WVW5{V2=DM`&H|4jx}C6y?~sup z<&p0c`gdlw*xYRB69)UGAb}wc5mER)z!z2+-Fz_y$18U6#p8EnWiRM?hj-=R$8?*=pFBeu;emJ`nEP?xMva8b6I}+f8FvWPlF}Ydn1BC z)iR~lK2IBRkQ(HLpA4h|cRlQi9)7w!m_F_wp*;4;QZp0(K6t zk4CkHp_AdJF)MiuXnY@7~#-t<}$-3x972YO)(> zTw{j5i-4PZAdFI2Xozf`zoZusOQ;-af7SnD?F2mD{CfliJfVP|aD@M=1F=1YUrgD6uX@0j zeBdh#_74zWmt)OGbzyByW!jmr@aHP<{_i^u!fi=h@8|PxP*?U!e$zkGRnXVNv9>ytNdJN$=4atK3URWG0ji39O~lEKe$S;1pq_mTR)Q9Yrf z|2n-SbVqY&wv+}E`E7`v$j|e*|JNV*|F%H&@auyhPIxYf`ZV#-Q9uAm9rg*jd0YQ2 z*!9)&bCyj1pWXkb^NuD74jBptk@vvig=#zysthfr5_!8;t2c5^_9YJnG2mFQkoopB zmqDvKNSr3{qGTxSJsA54?0Qz(>xzapuj1ss%p3*XYqlPB;C2FVSO0P1@ui&z)NTa7Bkay-G!kSU8159cx&(IxFDk-^ z_(7_!0{83!pfyD+?vEk!g6BzGtaLv%4+M9@FlcuT5#1#Cz!u|sTz`Lkeg={B0iYl! znfn%^r~3Ln=aJpri#A8f3-021-;CbX78R40bd!OcXUV zj%DU0JI+-@&RQ;fONm-3LPs|?aXgq0v@z1KA)B}$-oEINpcR!<>^-qS6oBXuLVZ5v z5II1&v7J>$5FjQa+AdXqAKsDIevjOZTE?he1N%d)O1=^J2SJV(I<$DG;JVkB2E36n zkUfPAiYRqX3`y{z3LK;4R#Xbl%}o`OA+<#nSN+~uT;BDNO`X8^p+%iY7>i3oiGRsW zT0t4gT~keu>rq@oI*dy~M_!XlOUu>W{kw$!Czf`7Ag&dBO&KmMY?v2$U`a{vdQ<+v zn{_?!PoH*RT@m9W2Q@83yi2q~qYEx{RsiS$5~U*sK>DM?^N03xakwJu*r;5a7u z94C$6%C-&7|tITCl?1z3ikFUHXn$G@};I7YlA^o)kxO?CNK zyv*X|AH2RNDN}o!rzlH&GfR}$dNoh8_4p>4VVTweG|zI?!aXer!mZ2@N-Kdg}OOKBDB5S_Ty3$vSN$gdrRi}9=}?Q>6Ls~=1{bKbDbm^ z^>dvDvVf*sM-|h%UFM`wYTcGNU6=vV?f?E!!Qxpf~f*cR*90--i}T8XIdPj}P8!XEtuJbwN|8{hrZ zo}W$p0#?NxbMN0ZK{*ef%CkUsw(}i;&WSMa9qLvM67vTbmKXkmAZoBm;nsX|87fah zrJaUYp4)^I=Aj@H*^l5n*)a*g-CqQsHPa2*U>Cu1vwA`1SYmi27p7wg5!9uaU1aY+ zgCv#p0e4*ppW$P@qyvs7GBp{eFQXzf%&5_DenptY?Q{^}4N;m3nh>>P!%RwS(n6R( z3P>8OF8BdOMx3!A(#{lS`bP+S^FYvodv6jP3zeC-GTk2BC^r=kdASe!5*(slWL*cF zBd!W;ywf(fl(Z0fvM!hjT9^zDg_>FvPId_u!&w@L?QhBp#1UAtjHJ2;TCCC1eoY;( z9AkLX&mv57O6#>YW26G@@84Dc$zh2=C%iqqPwg2JgS$AM+(FDOFp-Nhyv3aCheN7wnd3v1DJ;3M<@`G|liJ&h zF*Q>swNI~&0cD8!l}qI!ud*sY{Fd*h8deR_oPiVuRI(7-$x^{CJRuSh8L55171;;# zXs5k1(jrv(<}6FGc2ThGRm2i*P|&Di6WNKr7Wp61r*@U>ZO-6Z2?Vr_N_D9D`$!>s zRQ}ahIkkbid^Y*>nKZtRva;*ST!d441i9r9U9LjjN@#L^_$yz&QP?LnOlqRr1l9ax&N+UF@wM9X&)<*qeX}PoY7f;Nk z9fU+nutpdeM3Mx}n9Wz60Tq}lq=<0>1eXhVAeiVW27p`zEh(A`(i#i+C0o9R($+)= zslvd_4W)HYJ}_ozR{L1YllIY$1!Ej@_x>|S1Y!8*k|~4`wQ-K?QNCM-#(t4 zJ~fO#^+v`(bMD0HVAz_iCeNI89$fQx;^?m(g4?)gqKVmNL)?>kF zqK5;2A%F{}n{xViIG@LWTPk;JpRNf>df%w|IvV(-=JVR2wCeEV(Q}pTgmp5+-IDhD zQqZ%ZpKd{&e}C4z+-v0eLL4_hpLvG)4nTd8w|*;+5#eACxa~&tAMB-D-P-Jk37=^7 z>nI|>Fy=pyFUss4S2Y+-}ACq(#%O~ z7BPOPVPbbz?!l7&<#)ZESGQmPI9gRTe&6@CZ~N`x(Ha(Z*i(4vPg%{<`t2Td|Gl)W zdj6m5Wb?mY4g~{W!vr@;e=ar&r|>6Swd2bC)s7NLDvXrT;kE4Fm{$80LhdIE?v(Hx zFMfOO-TCm}4wCjqd%Dq@lr6aTD(`sJ^;X!AU(L!fV=BwO=IYsFk9Heq#~XKVh0h2H z9nfn+&yNBSxP}lIYZ2pJ9KFkO<>tY+umAQMentEy{|ax|zp&qZD{}jx(D9P_i5Z1# zH&rF{Z!_=YXzA9;fb}iT!aCf^zk^@olM^98`04INVSr@-U`K+uh@iOHUvIrYLP#dC zC=o0v{~8PigbyH~9S{gO3(!9=8EFgv`~*1spUzcKc1C3sG$?(yHTfh@dm47kP~AcQ zG6S;$4doTjz)H9#n17yR2bi_xZ}m;Q=-w{JY3$`)n}w}1R&ofZ(>I-iE$|-trV;At zrrh$n+TXj4yo+B5O$j)wO)b*H^RNS$uG`wPC_(y<1(+RpI5Mw%0VV|g9+}s<0K3DS zZ8`v#Hdi@mJI>Qxgq4Fl^HvvOSD7|=A*qKd&&c$>mSAF_!#tNIm;mTFFK!7|%QKPQ z>PB(gG{T)^GZiCUdd+?wQ`i3-<`I`+r4CoHeOQKtoVl3A zWWobheI5GsWFd3`dyR<%(W>`=PI?`a5Na7 zS9ULw2ZJ!PJpD{oVc+Qi!8I6~l_B$2xoT?so%?tV0eWN&w#6o*izGOwE8_9H1MnmI zx^-9`{ose|Fg12$=|CaFCaXpWi*#~8G(Gkkwt;jfJE1Gu?$#KpT;oVYzM!w&fZYWB z$Sc`^+0z%0O_&^qD_?f>SMt+c<^`bQYh4pNeWOhnk$w_6n=s!?#~Z9x`vac$0$8n) z(#koVkcN8vs&fZgw*@Tf1T|pbs)%T)@c45ohe|{VNLrzUpav+Q{=EjHiZKC4$ zA@tZDOr8uw+W5KDXFU0Z6XEj4(G0qr6X)@J(@l1z%{Wop@qOPJ#5P(VDXhze=FEp>Qm1T8(Y@NU@Jt$zNe*kv zOg9#66!hF#8mV&rHE|razd6_c=e)#?4b5Q<9 z>Z?qWi+tj!TPVKuo?Iw232Lft{5DLAsRXA|Z`jfTEy>VRc1U!g8)11(e-dpdR+#>X z=2%H-LGm3`Kg7sc=xaBl2~(%;lPA8tszO2z26gYr@!_?iV?9<%A3bX{$~XlZfBNKD zCldpTXD(ZM0lEDNuKVTt#ZFe)bI92T$>WKD)|+Q#gXMyL$g3~$V)eb{)?yhoLMmeU zCrecCo(jb{eEk5)gvo4WfjhfCR3f1ndTOI6R%;U6KH&zws=9SeHm%S6HTnFc8G{yb zu7HK43cJ72v#~w`-wM_fG&4Q!&^VubVb9_}lF+FzV9(O!QEuS(WH!S24^zccNlih; z3KH5o%OOtr)$8h=`F1@;zsK+RF7J&C-F7hld4G2B#<`v5gAR14>+7Ij!Hh?K)`yYe zB2fZaxTyD9k^Nrm?%8Ki=;;@tz&e@=LiK_XEnR{Vg?^fO(dXXEq@434)9q#eZ zGHj|lw@c*4{q+tJTid*6#}`tOP}DNAwDK$3KWug7KJ$%RnddIEt$)zwiMT^236JkQ zI>1K+6lZpL=UYqb>}`BV%^Z0B!{6t-OS;jUasSDfhrD-2lU4o&&t{uX1k>BUOO{{P z%8!5hOBsOU7vA1~A5={y^ZX0jNPe%2N_?nt623EJPCh=l?|c$*R9Y1ly8mRZ+0Uf) z>&fBfrz(0HLWvXx#Dhb4Fc!(U?zN+h*^z=N{$4YVwqLo$ARCNBI1yu<6epn^QYV++ zDGbDb!$_AY4$LkZ+Ho?3DQ3(6`2cV5%6bSR+!tHIl2tkZ+x=Q6Hsjjut z=%4#|)ea8ln>jO^G_;DssEOunr78KodOQ~Dm+Cp}AWRoK(_wxg`PI;Ia~=JrZoewC z?5~B;iIhHp>~t+Yw#ZRZ^PpS%3r9|5SlM8TT~b!+HRWB(gWqyb4J_=lvY(aRmi1i1 znM}O)ngR{|?6XUXT;f!y%NS=$EopTKy;AT=!%6W$H^DLVdBNcI`SZ^WKD(Z1ZS&Tk zoK%_)jJBH?;wE?am56ui!t(8dCXVpu$lHhj!y)u`8QEELEyOr{-t^(&@Pdq&uz?@+ zI+bCqP{YfY1<$GE_D59;Ys5tG8J(e~AkxoxwHT*3U4O*?;K{|)&xjfE*UQwyIx6ve zZ?byUz5eCo%l;ZHyf-m(I#P^B&ogu=rFr_4BVFq;;_r|vV<-A4#I4U(n)$`j+UbiC zP!qFnZ#T(|+Zd;?I9=aRv&ueyw3(-yR`Or7>iB@mg4+wY&!IPJ%gtW6-*&hD`S(VB zhuKU2$L=@RLd_fJ-@K?o#wj$q)rIchpM;8g*s+FLv`^$#)%i#pEpMhp6tte8EvW^0u&L6~Y)|G@{Ega2kNX>*Nm~w=-+1%q^W zZ`8f_T$4!|{1VNUT-PS<<(nF2Gu~YKv?j#EG{x6yRxk)XpxRmW&(Lbs$^89~xtHuh zhiScW=8ZGr-XWjEYhr$o5l0Rfm}Qr?gkNTAxTBb)W%rd6C%^0m zM~+f@mOZBBez|$x2@HaE%3DVCZWp-Svd_QV|G`?gv&MpDf7rbL!{Z}oEX-;kM&eEh zYlMrAq}5=u)161Bk6rZjtiEKI-+3$>;Yx6}8Y-N>^F;I5m6%{P{753;;m;y~|B2Pe zbEjQN5DCcky_&b4z#vYpd2sCEqG$bms66n^iwI8-XX_t7<^$`Rjy-)6tjA~8-xS&z zdEI$p{d2`B=>5-Qui#GWiS6>BrnLy~@CEB%hx32vl9VfvFq=sR$>5U~)<~aNNt-Da z=is)}|9ld+|Ni2568up%(l^D~W`=(uxKs0=Z$?7cw6J7Iw^5{D4*3b)XT>?>(@nw6 z%nvs6=bwap_Kfr|TCiDA=Lzl&`R7lC-CWd`3>{33ypwz=uyn;abm+mqI~962m#>=t z9exoRQ0aVg#d;z1>#v&uRS7p&9VElPeU1#Qh&jFH?i}{x=fA-E&ZbGXys)2ZkwJ|M zH-Cryx$yJgUl0vuyAdO)LH=@V8{95wyP519PQM+3JN0b0vY&*{$$EQs-Ll;-TnJy# zynN70w%vIo8L?z!yf^UBcK5k+#L7+b_V5SWy_zQxYo7fZW4~D_de?QNY2Ll2kw}iaJx0^+FrOdi5e#JEMN+k?A~OtqO$>^l9h^%IL@1 z=*J%^$9d={rgXW5>EEr?PqtA$Nov$j9q%H2(oZ|m&uF}mjxeCtwP&6+$Qkd-CK%ky z>B{vm$hYarOEI`lMs*cb8Wi<*7WNtx=X5?;HJ}nYA0iA(A)O^>4a?d)9uW-7!#f^( z7*;5CJV`NpcJ%RSrD0|5$LGC`L?Ml@CW(Xb3`!0HO6Pp-qF(jvy1P;qmW0dea&^V6jjw zY!nw3^r?~1+e`R7PUu@D^dAuh5JrOnMtvGl!W*#-cL*cBZ{CkLc8kS2CK`P!GWuR= z^rO*eyw~XGxY5L_(XVzR8#MR?RUJQUV>DCrX0%bm8w*ml;D;E-^OG0?RIe`QyqT@k z@TSFaZNwsa$Y2yDVHwfCQl{a(0YTB?5cV-#K>tQr-)2$cmPS89BzD)s_)jc>F~xYN z9rM@G_|WF_pB#yRO3OW^&e!YXc z(DH)d?00b)k?5f;HJ+1~E<@Y!%G?6s#Hm4gy{XXY#*NT;3ULfQbTy z0u)GMo}E4@Q;I<(I8KKVz%9b063;VLB%l^S;6=Xs@)SF%7;|BYwM(qAI8YY-CNhId zkIo58#e~5@TCAW#>d-~qPwM2k_sfd1R4E2IeGr2Mz`sBA3m#>y4|dmtLAw}CmS9#l zpNQ$_`!7a$8d6v&V699bcrD5sNvZW=5JkocOy<1h05UTuiu>_WOG(1r475Z#lMlhk z$DcruOstd9m_!Qy1&Z|m}fPoAIO z2UXCgCK#kB(TYAX8ZHo%$&5=E%rlsx%o#^D(1|h)lz`&JJ+kU;Lm>yZ(0g7Q!> zl?i7t{7Ie>G?t4ZS#&GJC&YnWy4axkmFDLua0=V=1~|haE2})fI$46kj@Y50IdLFd zm-ReyR56kM&VLK=el{cs z$iqXVVt)O--lm!Td@afjpk(TUaf=`qBqc`ysNsEBn_Fbp1$sF7&8|L8LmxOXO{A;@ z%b}}L4dQup4|8-dW5_;Kq>I%lH+$G6i9mtQpC&^=z-TgdynU|cW3IYOR}97{M$9wG zitCGO1fwAapkgiL^Z}DD5dOv8Ul-zf_T%CDU>6L4)V$81Um%c?PsS>#Hd1Va<2BdM_veTr_$UH%TcXTx@Ht)=Bi$(X)3 z!W~R<6p+JYG*W7$N-DF6I9;`I>VTfr%CdEl9oyPTm1_xp~37`%%uNyhlHIPnCr@JwK#`qOQb3P%@ZcmPJuP;7_R{TnBHioLR^CWlEuMEPJ92%_KkDqF?7c{K|M1V9#^%u?=+P?Y z(RRtB{i?@DIgj^VE?znxc4S71UFd=jdR1>PiSOn1On79pxmSe23<35&K~LGn)g|== zYpnGbThG*Ok0&xC)+Enw&8lC$-VrRuzV)sB4O=~2X+B|{;<2~(_#Y4`=QVR_eP+}% zTgh|bJxru4HuE3(Lcm5mX#?^8A7w0vf6V=) zSsyre(VuPITNNZhO;BxRM0J59iZ|jbH%j=SWH}1d-V}z%Munw`{vLPZNZ03Ne@C83 zLP^GQ*So0QhoI=s$28xVqRofeo>IZyOC7#`6{3QFd@%|7!V3*Sl&A?=b_)e9+&V;qR z59|`_x=ns-I;VNrePYlg?*5`=BfDY=Wc}F!c#4xsK>`INN{P1)U>JZ$b&p;9JjR9R zO%y~Xi!H)xUfkVHr4W#e6;wznM(A?MbirZ@QdJyEyrI!dc~-%6_C{iPw1AF5!CjXu z8akJ-2+F3$CiuiWKqp>Rh8lJ`o>-{-x$z-UATc!(0J^6(9VrP1udBA00|Ur9I|c8N z_=PviZiT6K6y|!6#)ri8Q*j!TzmNm77=4f_km9t<nt>>$t7-cU8<=5s;I+kc&P;o ze#1m7lBOrYA{6LoocRfu_C2S`%w_);+9LCD6Su?UULT|kJKp$;PDlhm zs_X^w@x01VTq3Jx@k@$AG^AqUZB$r8!kE&V!5pJClMpkv&Ld}dR0}&Oo)T^1dcUv> z0!sW5X|eSJoJdH8%=wzev+zja6KC~7qPmGJJfs~L{WAM3j7=8i?95+05Y4wQ!kW)Kdztc&eYp#K!G9S@h$n3Si9)N9rdpce_oKk zhQQ&ITW1wlqiqzqLYxn{bK1v7%H`+rlvs#YJmah4GhH&!DrXkjQ7C5;X`*kiioAaM z-_gMweR!-lpHSCmpCna4wC6MsIpv7tfeGh8PR+MH?@^wAwnRg@MxpGFyu4Rg`a zgqgcFg8?ta$q~z8fV6|446txUvepIVMH603oEWEp_=B7;hNbyioCJlBV%Y?xGX_KF zjVA77U)?hZ>9aPo3K73&=|+FL-wZtv&$mmmJQ)skd3r(OzN^<6eNlrZZ62Oz>^573 z+5?94TNFkog8i{r#$Aq~o0vk3TG5d0Wi>Rm;C8;NGQnqy2E@2z-OMn!Ope|mrbE>9 zFmm0gZ&%GY} z()c!;rO#$scBOpr)3qpQDftd2+GERth*KSfGp2HIf^?BOXFYVNiXti1G0J?igVD^d zT{*>%nw7E@RDYVU*Sp;hSL`|SyixI!?E9mWEsLrGrt_aKaP1;^+AzCUC2SnG2xQHt z*$eIiHb@*qxJXpr|F#&Wg|Uyh6${}(V+`5qdiHECd!aEnh~>h4GVrA!wjv)GfZTwB zq8)?!7@5R?Hg)+`H=Qa@`3x-EAEF3WR44p1?WNO&tviq**9cVEB=~@J$K6lYX*=Yz+Ui!`tqA2Smo;}stjU^t-#(Z*S$z*9D~j6?U5C{|a_oBd zFR>nf4PEHUxv$X?Zrb1U3oV+C=?@<|(TBJ_o%^nt{}o<{aYSqLWhnf$OR?6_zAzEm zL9y9Na&GSr_;~kjK=ai7po%pCv>k*oLSwe=Zg_NYPLK1d_#0%tXehpoFP@0!5+3gi*sP#og$Cqk%T1*Q!K!j__ zKNWQBWQ|@NlP^|KTs*Ux=%gXaH&oGa;kAkMxVkC-jMP1U$Q1Vye~%iLQ;Aw>Y`k*t z$WnwxQ3Np7pJug@H&f}lOXn!s(i&SD|>e`8sOYV2!xag z=0bRuXS-gHeEZE^iiW>R8OS*6Mm^C{+7b~ba41SVd!5&OBrYFwhVZu0H@_piN3GA4 z{Sme;c?)eQq|Vf}q4_*3lR#Cfp(B5O=L-w;*;Gu(A%whddY6}@;}vW(nW6>CVC*;( zs^5<8y+#oOk(z$h?Wg?_TQv{mWy&uz^knI8>|=fjr$^9+7O_&e!?q2&D*1KoW4VYkgaq(F{`|H;>SU`{%odvp zu~mx9`dEq>BRyPVI8>_iqF|Z2DFiy@f|*Yya$bcno=Xxrw@X|ysd+Z?g04h(er@?& zev>9yj|bZ{zIlDG$wT}b+2>t*q_N>|dA!6H5GO#xK-8#45^na{ z9e)3x`MWJ$v;J`}MO@ZUq6e@n32x()A~86{$c)!n%|KNsdYb8i);zx%B@+04I1GyFl14em-#D{OYGWFkWU{HQYg&&u7d|8N>wr%Dk^c|* zJHBcUm#o&6Zd%d&IwcvmB6cH{J~@uRz8m6t>VL?e-S2>)*4eUHe!<@CAZ!bYPq61a zJPM4eCACXjQaDL6t(-pfAM-~?{?5+u_U}VfHgqVml{V6{BdAp53*9XYvx`l@+=Tm7*3I)KG`$D+2d|fj6|1M-1DM&wV{SSGrQLh zTJEq_YMz(sMzktEoKD4Yy&L$C@q4kTd;Z;ETBK5$sO*x!!a)i|xnt(qq?7PaY>3}W zr3!6UH;rVgE5>hUf?3U$&7KbiY*FH+wgyU}Ob6V*s6$UIU##48$Qy3-`lMz%$VV>0 zK0=i4UwEDSsl|NkLYdCS^zmZOs;g;XP4mmQ<`Z_Y4lHx0ylBexFHZEt@34u6qo$(I z*r3+~Y6>5J_4;EL4`MuKVr0G;qtcw|mhwFoV9p_VHZ#a=s5Xa95=&w+v0G`XYon8F zLZ;i)!Nfr=98&rS|5E7XX6o~&lLBNtW~F-C+QU^fccIG-`Rm)PCYb$7U%Xnc!$0x# zPa0AklxQMNX5#+#k>O5Mhpaq8RBf|*0e?Bv+$HUfdfaT=6?ZN(3EADJrcKK?KD;Z< zv(Ps{yj7;0J%{GdLu{qhgFX}jrn zO4TlYRR8o@ufozw^&pH6pI?J&sl zErqmwP(g#2O{@Nn)HLtDBx|lZoICtB)V%k$?wIw<^Mj=a+Ob|;EzzEue`iBQ+qE}< zt?a6!x$_?m`pB1d9-TX0G8JMZkVz0Fsis3U|B1y>Eurpiu3h0;66dVy<-bE!|8{FW z9IulvAI+RQIT-rzZ_gMS%3G4Q_Jc5&>`5%76bFT9a~EN`kzlX{6)cBgP^L0m!a(p; z$W;s@Czz`U5Z<^|93a%cX30sh)kL<)AZ`(ZV`PhHVu(EEml&b~j)hziv*$`$E{ah| z)Z*QtNHAdnpT6Ym8Yqjbl;csxk;NdA50uOL0vV9a}?J`UKKv&m*vEp z$zf=Gz6Hx0`!ld5hUG>H`X7fJ6-c{nmn~e<+ilIU*RDpxnjHrIp~NfR*1b= zlCUtJhENU6=MR2$>iq8BD{0)Zkj%_Cy*)1Q2ZUQGo&&DE{kg*7la^cQAb8){X`UBPRhxVMMe6!1zab;IHJ2x<(O64 zUq}QRp) zH8OCZ`a&JS*8)p5Y$y}s9d#b}2!`y;Kd$-cy0%}ENZ>Da&GB^t^X%O7b~nl`;acFv7%A*Yf@O-M^-F{j9E@;`W^)=*51w_6vHg1dGEVt~6aq-acaGSl=F&|oe+tuoahTkI|S2WIo z@nQ==X1Gpc-RHO!|Gy#}dk;xnPFmpA1StaZW+4Fv9a`_N1zNF z_YB69@q`SppCqJ72D!1I#AsAiDNvbS z$d`|Sa!%tOwI=fkD5Z=T z$Bs9tG3tfn+{}&QaKQ%HF1Y7wL5SUq!!h_p@oUOj>|G3#w2sp2@#SamV5~N{5$r)s zvJ7uyq$D{8CP6eLEZd<^6LlOLV?YRPcDNRVMq$srni#1q2*F);TxO)0TG9aD@m!~j zIM3kuLL}{_K24^Y^?M?DkWqsHF%C`VV*n)@+kfu}t(Nq?K9^*n&(_|RLLj7IdRxZz zC2~P6tBD;u`UW0|PM?%0)JY=hn|oBgmIN{}qgLm`Ye*C#u{l2jVnECkSZxxx+l1-P zQu3G?S-e`C);?`o+@w8ugfM60$~}v$VTy*JDKO$A%W*BR5_uqYnmI=+28vAR+>3$a zw`z`k_V9_S5+gzqA*i!bIoB-N65U=`^vI$q-}Pyt5%B|;5<>U)c#vmQ%`|yG^nrUhLeK6KdC$`*x)i>Yp zgMyQ#V>HwT%Z@s6v=TFHsep2n#b1Vko0}H;l?M>M@aT~d&I{2%H;PP+Nkr-+=P#)+_D!MAs+A| zO4mI`aYtD@K8ov(HY5>@NPMQSlMWGsg_KdCXUCo*=SQLVg^ai}_1Vzz_<^nTa$W#M zCb>G?kN-)d*)HL~8YRs1J%tb)*_&D!Df$*!x?e+I*hIacWwsl4G2~qMXe$PK62cdO zj!|h`bN+b3$1o25+05MfnIFjbD6{JM$g#s|^4aL#=gD%4Ymrkq3cAiF<1_%}$zuH6 z3h=bAW!;N13@^lx`Uo5Sck<(mh+0T;88#+M7p@NrC_5AA(g{ER>AVfVQ|rk)@GKw$ z&nvcG9c>Cm$+}*x6f=IEQ}n`k+=WZmw#`d%4d`kWJlou;(>$KG!P#Y4q0U1VqOEC(Ve@=eK7&0FHhLD9*inc0mXXL-?jBSg(c-tObjVYro2VEw2%9ePcx1IRq4>a*X#COz#e<| zZ{v$c@q`LK95TF3aMY-5i~QL?R{F_B<~lSjSRUZcV;uKsQ5o?2T-@dEGc_b}U8(>K z0FR`GwZSy$N?w}!Y*aoc&C`YkFJ#p1O@f9q06F{s;>s(5E6hi)UPJtXkU%UaAb5q% z1HiIo>)!i5Y-BFZ-OMfFpg7~Zx^Z#jV|!PUh7SDBx~=PA{TDc0FrVp1De9PcTX!U0 z6FRKs;Un0UY+OI@;sI5X$4byBpqR8bq#9+GXayTuh62q2^+g4at(?Q6$#`d-lef%F zz`Vs9h1yOvwS*0S+a4WgQi%ESBg&}6c^46AY!GJ^Z^%snk+5J;Te93By)oc_kK7gScV9u zBxu~Jw#r{{O$m~K1N!3~@ozi*lBLCMC>q$PxSkGKct9eU0^KPN{@Rww+2ORqAZ#!f z^%RP-(8C$=#9W1)eo!6<1%ZgI(b;jB>pqlp6$IHrsyMCOY?ODU_Z+iq2I}c9en3%2 zzgoM#OEy)fvmzr)ej*#44O4T{O&gNJMdAKI?DBL4S33lrFO`ZJ`k_x^pe~+a6gGD; znu?=VaMm9$b>rg>P;mlSp?!uqj zZ|G;lRzOE8X)B-)r?NhExVEl@CD8|5?>pySc~{S(SFn%bjn%S%w@to2NQ;fCj6GcL zk67HLM41bshc7WI@kr$mkAiQvJTGz~4;gQ>IdhJ%&NSRu3VH9lDK6PU6=Y;Kx~6mJ zDA|CE&zTDyZWA$wTs-Ar4vW__7^v49)Qg6`;+am(MoisOr>u-Q zS^)M`F;GfuET-DCYmt$+z_aIp_I_F6r8pg1L|pZU+;@M7k|QxpkAK{jRj&~u=hh_r z_@ZHFJVr`8EKm@rGv*CdEq{c0gU1Nj zGUR>28pA;>NWC1x;TUbAq45cJs>3;);HgpH#9(m=ZOr&L1YdiVboix4qt{eq;00Id zJTSNAqV-d&D{mG@(&S^bW~;qPt`?mgtuY%?KfZ#Cc*AIo)EN6bB5snA_kAIDv@=30 zP*dCqMIN4?qw5_6tj-&S%Y^btgGLO|1oYbNDZDzX>}1 zi^qY*f??jFIWhGD+t-B1GGa}G>dl#~GbpheaZD}}QBvr}&y5W651P_1`;(W?AA5O6 zvZ+W;O{M&B5U9;&|B*>m*ros6CbF(dA8>fK$b%R-q9;(bht5?jxYXVeBTu}?(B0o>XS3+bfX$h|ABU{jOE)q2b@lXg7dDUwIN*2R}&B=n&+KX z%^oS*TyxQ%#-nP3exsM~SRwE8%WW0kc*D1SQxK;-!f{dI?unztD~E2k9XF#VDgo}_ z-Ub|Ud)YMjFO9wl>D3e8LMtFG`|)?LQ`SskjZ-QX9!EV7uZ^pa!RE zZb>i1<`M55m%&E{dCC!d0d6DDEo<<;(RWokYFxj*jd;JT_H#h7VTnV85rwOx;@W<0 z*c5AY7I2l#&X=$9OTjarOR|RruKj2T`t$E}c<|;-+n*~=S*gB76Vb0+j|C&c54M+{ zIB`{mL>?d7aVq!zj~*jyNZqw&yQqg$k$3NAHTB!-;QO;&n9<5CTa?bE<;b7W#-<6` zD0)1hP|l&^M*$@%@`c>acjmV8dF$Wf!@!5W9KI}ILbdKc;ghgc2hr&!FCGe0Z)Z~c~6 znJY4VVi!%};1i91TABIcOEhc%`+xKpYEb9Y&748}=?D}tV6-gzi=nKiL0!;n)O;?a zVZRoE(+R-~mb+!^A>~qDZC=Zp;OIH?O~WpB)XKBz)K{B;pSfRc|3{DEeKwuG6pU#{ z-F57-y7!Ij^dMkQ&?daLeC)=HCXPP$H%&Vah<4$R5xerI#K%29Yx4ZOGa{>B=TGJB zKpAdHi6&sSf*%W-@`mZ5q;A}-D=~L<9I>#c!(%5?v8QbFr{YgXqNfwtM-5IZTM_rZ zrCgZEcSx?pMWqG;g1mJe2A3Kd!o;Y(7!|&dB>@NWx=z9Teb45CSH)fm3UOjmc1ffU zUQ?~Q_l-(87Sw+5M&C_)5XrY(ZB-8{E_)$}c+@)GczC@-y{Jsf_|;qW(}wld_x`S&JXrW=#&ds6*10*b-2X%{{B_{M zKeu1qzxi;Nza57j#?N+7{oS5>_%Je`SFrW+?3qXRqW{qLMH%ja9*EGjxF#DI9y`Mo^uUHgh70_Pl6D?Q{{BxFSsJLlJ}E zgu^7l)q(@|PW~oG`_#Mm7L~x;VCI@a8{Y3K$h)UG?iyU5<}4V~k-TJLr=DYPe`ouR z0K8O}D1FC_E?)G)A0g{q=%D|;%L0Es%UArP$qZleHbUi+C>KvgG{Sv9HHcCr_$vdZ zU%g$+f7QhLBBZQA@l{cbPYX^~0nZLEXLAn*XJww`k%o@QtKtUxnfV^@{$UQ3Ctq4N&14j!4DL??2-u8^0;eLTAcMj9A-ZUj$b^3 z3jKTL5!>UU^Y=4P`0w#u^^r(S{xR9{Qh~nsue*l0Msn2-Qs(fTHM)d0*)PNNBC6Mr^nX@kL}DqWH42|LHqcW01rPk$c{ zsrnM3+ZA5ye+L<$a2in8g$pn#vV&H$sUF5u0Xk8ck%8RSmWS*aGI7)%-q`3W+I6JVkge|yTEDY?mbjF#s)WGq<_nR~Aw z_~UqQ@XJN3c2?nxPFXphbhr%<6dbI_+%u3aHp~U23)9Z)WJVuYIiev_caYcU(^9<` zz&0)rV2hL@kR$y5c0~eI4qyexV#2@_yl8NWodc5e@(l8SLQZb1J9mLwN;H*gwCNL! z{-nq+3{5sgn9`|FVObzsR9ngVP1}pc0z+|88sk8W$s<(=ZZUyF=PU&k&H$Z0)gJF@ zVJ=(4RJ9;DpiauLiV1#~2nm77&K##^6M26fjG{Fwb2tM|eni-;<3vfW+5U|pa688# zo{|9pMXedhc2wMx`hgfh8V07v0TG|C?6C!V_WHf^*PRt+CYQKO@&nYw>XH)Bgw_-| zl?3R@VVI4{Ga^+;-^;}u(bfP^h$B%jSy&Dccenyli)?#U{w``q?lQ*yZgzMjtdPn4hRO!=)t3-yU9zsI)84BAOX;EMBBR53BhQS?a2)G$WDJ_F?(T&6u7OlzCa9}l&1Ztpz7C3;2fO68BMyrmgm~!MA z@82nK>utqrDNiIk(43P3N@Oe>|5P)u!7E~fniYn$uL278pl#}VDG;Dx1U98^5LdxT zFF6KQ)|lB>H8w;dSE1<+5c5Ih=G3jL8(Tdo;upQ4-W)tKrO=xSkcXh+acsFuX{~KK z%UOMS#W{W*VfGXfXI$HSafxcq!lW1lq%paFBHfCD1Ar0@xJT9o0~iSB?+?Iai%t~6 zl;y#|2Do_;uq_dK%>jsEq^T5|Y$JDfst z_&t=12Ij9I_gBEsQqF~|V{c6WB@F!HCbQZa|HWRw@d-+M62*1y6i+%&!J|{AS@7;T z3>DXIVNdq3*hN@&4CdJ(Hit=3wR`5zDC< zk4=e{Yrp)i$)2p8?to@;gC%Um6HmM#dx_=e?!=JGXT|T}CH)L<@73c~yzF3^FA<5t z(icSFoB%VL!l?x=eEpQ8-J@qYiu2-m_J0|oDL9#nHD^r}&pC_ATGyN-B}j0TVkZ@Y zNzrm^*Hj)O&e@rMx)Z}0--@1njhQz>CT6SI>cRQ5Z{lP901R;VKzTh*#@1trPph&!o`CLtliWI8J>Yd<`m=`h;@Tv`UxOi%nKE9IvJuF;Ai z&TD{4u`-meYfV&N(G7~yy<3^<=PKS6r+J#qzer0(M;NZ7*Nt6FP9cr*fpv86s2J25 zJebz`lp~Qsi{(b;-LUs4-@(id9cj@{EA-=T^nHlu5}jztZ!QsxpIRlXh}e=ppDvja zT)cwXW4baoCAc$;_P8p1j;b$Xza`K9KFQ#4q9i|`wajZuFn^jiab{Iw_&uEhyB96> zeeT+lzmEI`QLGz!&!r#pMrhBY&uLvSKI+C}cT*PjWP-4%x83#XWK$j7y5O#))U-*$ z6jA-u<-;h0L8U6CQ_vGB&{Y_@)Ee%FrOjF8$z0#YsOo%*E4Jf(+$r38ov@Zgc)Tmo zmvi%0E1HDy@Qa1t=oj|s7guGhpWP8$+!2t&fJ87tB~c)evx9Q2ETUMj+$6Wy8ZVr3 zR;E>vZp0TatarWz5TgM03-9VFpgZ!Q0Rt46l|vRhu7m*x8_`R0$gFl)J6>Q~40r&Q zhKOJo!(R*W;TWV^067|<3~!dgy_}c=qp74vB7$604%umLtx=$smAQy})sF&HY&MT$ zz)5dA&rZ`{fXAXSf~?dvTouvj=g0OnOq|L^ zmlpNril0zR$h8&Q(j?d==V;;L&MT=?Tc`O-obn&)2)&mis-lpWswd z9L^z8ON?$tvnXL1tT6a9BMW@OV73deP#WCu?0e-Qku7ZhH4k)(FbGW@C|kjoL>-8b zUaq#ILX89ualqPSx#TKyrqtE9@3xh>@V5*>*#R`( zWlO*aOk7$NDj~&^c4Q{0;dy&I`70_tO=O$|T2cniMS~{u$e)uIUb+aei>{&0 zwh!lteCZc?mcO8c6}(Phz~r+KR|P8NK>4yAY##4tGg)e{M9XUJ6bX~-;h|X+_*u3Y zM;Rs^x4QN-7pjstGnneVg1lCx_Al-;MdUl#<1-W(3sKh3>8LI5i$<7^e%!y+x z%ga7-41_VB#^*IA7d8u_#^wiy*G+=jk@N)~BaWE0MNQPZ44iP9o**!X0P6 z%)so|Ld&@G=j@+Ii3C#ZnxOgatRz#SY)AF;c{BlBdu^b4#rdAf*fr9_aUdz?Y3BqE zOnakWv$d-9ot3f%o?kw-I7m)sO|KaB&Nx$=Au9svdMzl;J;y9xjF?Fir&bOx^Ao0s z3qwnO+%_uPyc>T?&XK&C*Rp-W3HjOMQ*uMiG{F~ zZ7y@0`z6X<QXB$;_yaJ;}lIDohNVH}IdYkR>OKiZ6xXc7F=+!AvC&2G5FBSo9+ z1YVOsXzjLQY}bz=jDOm>k2^jV8!_j40czs0Qva?>7}B`Ks<3EDui|TFgJnRoQ5CWkXe}9_L(<_ZQI)0);ynbJIAWo=mB%P~ z3&hbEf$~E(i8b}dvfnoI(AB}1U)}Eu1)zleX}91EfR7x2S6l`ten~r{xBhK28~|`? z9@|{-{s0l6ME@YUB7TP73%&O{B|0@h)WfkkLFSq-#)~LKQTfSY->H*;EV5$X9ur)? z+F9|<3@q9zN;7qgWxF1nZXoS&McFiJ4Ycw&nH8@mn`E%V{Q4xKh$b4O;pmd#qlt*R zfPqWCCH-O)N3yD&wa_&Q#~+Xr>uu5t%gAJXFsJouJlFnQowT?*w{`O1{bo4IB;%_L za=NqlQ}jP^XVmyOLQjxa2f#ZX+P)E(5ttz>uHN>6rV2J8Oy9>($k?H-9(&f8**?nN ztdIeH2gKRRO!SVgRY*Lm{x+h!to2$S3^9=L;IEIsf?3Pev%wR=e4jB}`1dNvg@vlF z2I)l~)v8s-f6&8c6%WM@rGo{>I~Aa>X$t*)=g+&ESdi2S<;|@qBqnv#T8-OWJmz=! zv>Tv#Cdy@`}gpLuZ_I_cIXYz;mMJOe$T^Bo&{LV5u5Y#?!KJs zn50g?NQI}u@_tyCw{5A1T&v3!%(6soLrJ%`++L}9p&4rLZCd{wdfw*5uy0|mwKFH4 z4^ud8F6;KQpH&1}l{NDEO;RCQN16mU#h<3lzkF;6CQnDn#wa57-H?G#h^RSaA;}}c zd~HD1X|137Hrmx(r~&H*q)M7tYpxf=`xbRqwNn9cC-2|UU42)y{w_`;tFTP%>PN}k z6Kd~mBjv^9731vUV7SD1Xs8HrU2*KW)*xFR-z~m#KgL$K9x6C)0&j0) z>pTv1jo%4o2T4rxn|Uqi!PP%|bm6p@J>MmgWqz+uFlhI!o`#fgEl=SU?_-84$oQW&L0kU~o( zT#pVn-hdoT1jKKLRzBC#;VUZwYt>CwlG4SE({$=z)Y#lQc70Q440T1NG1Mqo_-2bs z@iu`0teJXn)%e~`<8-}yuW$BOz*SYVb8kIdS`=fnPudo{NB;S38*>^ZWFK=Neu8Dt zvO35T2U7r9XrOa*OT)aiWth$u&2mMiW4@nzvr?8&{DwV&VC zK6Rs~zaJ5`9-n}jOpvqd63>RZCS;s^`fvYl@#3w>c%{Ys8vBA#OVXxjx0ZTjEyX+tIaZ$2 zap`jh$Gi6rk4Y;6RY1oB9+6%CNtZX&`9Gajud)J2UgLw`U&q3p{*z5N5$R1759?Jd zxIpO>JUrS6n>TY&qSusnG9{>q!P)u~;0LFMrD*VQj%=1q_=sXTD|}R?=~?)gM!F@ndclP{ciB_w-8<%A7m{qGehUx`T~+jx)%1OD2Q<>FJ2C*OwsgC$8+?MXg;Nmg2ULFfms z^d#4PBGKr1WI5M@rzsyk2M75$UvJ~-g?5`u*?60Rghih4ef=or5o4AVN=$qz#udhA zf8&czN}s(?049Ff1oc&vEwueXa>!NGcDkRC+1aFKqS~)I_ezcnb~(kBnvBj$8IV^& zr*7k`jLM*9xVU#lLmw;zoA&tCd^OYULRJNyrqBlBcIX1g{0gxNju5XKnYlEIJ&-VS zGUa_N2Sc1XCpLN`JEF4^t$D3l`ssJp$nJ774g5;|wu=m+L_Ra?;_IjsR}x75rppa2FnpDc}Q3jeFj8(XM_EIR74Mz1RyH_&s~Qc3f>%3P#IfhhVaS)1pvK>N$5C5 zz$_z4vOr?YN~-h-3{rXWdqme9!yMtauPtuGDF1{6v z$wb*5u_7Ha#f1);OGFvigSMKZriU1MK5recA(Y5_4aOylAr_J7waQ%wblda{E#!ab z#6N_2=xGO8K^@^F+b|Vg#}Cs&_pc?LrtwZ!e2uI#czoXWytQL-Ut0}4g|0scbFRyE zEH^o(v)Y|t3|NYFbI3KWnOmkL%upLj>T*AMA#75eYZY$$5 z#euNp_qiRNgNi3Vs#skbD|nj7(2M}sYJ3RD*QiOfJ$ISU55sk|JJwf3{gauMo!05> z&v%0y2f2xq9}jM;e+l|CbxX_q%EQ2t3+~eXGp{mnJlrFUFXL!%)cUu~kb-+*``ZFn zyH-TL|NRmfT;zVUO{1cU(`)>Cm{tigYc=RVncXcq!_0Hb<6~u$POf5T- zTQ!Pjt-7t>ZwR;-`(VTEU7S+;wkIpLlP-3v^S)x7BS#gLiV0);vB4V63?yqAtQyk` z4{c76hWkKY)jkofT(LEncIQ3dcHkC<>Wtq&NAyN`wT26dX2_(82*|)_=9VD6@5@&9 zgec4qG|cmuyVPYg11~nq(SH3%oa(=dV}M8S%og#H_V*%_bZcYr7I7v-;feSx|MkWY z43$l|?^u#W{Cb%o@cO9cBHmoYjs9Mzk@`F&^uetq_=dq4bv*L^_mW3ENHup7%9xXQ zb@v&j>`7v|_hZgC$TlrAWVh4ARTJ_!9FQP|Ye#p~`I|i5&C~)P_}pdw`ljz0!4H!> z4BpobTg0^AJnX9@lc__NPA~hET~L!}3U9>(-@}O4R+Q>as~<{+qtw~0i8J4r`DFYp zHC*icv(WYj<%loAr($1E-*11|q<%?pVHs??2YF6;o_z~l38$4nI{ZiJJ>9q26-Z^Y z?ZSf-EVUhBFDoK6H=8FVL(iw5PL%rEjX9kqn78qPM`v*=D_Vy-TJ)D$x8wtVvhz-g z5u#OJzS0}FRsokciRlZ=@yfc!wYL3X_*N(}m(y?z=SCgWHVUWsEtqN#;ktT@Js_CA z1v&J&z2li4wC$pk34m8_okrqc(Ux5UYbW)6{rwOEDz{>_4qVE*&(cei%OrValyOes z$ajd~g=I)*Ow+uhDO3ox`sU1O%c_2p`gZL=D~%D!mW#gdmKP+HXAS2TXujaWdl|vW zx?%Z?5M3ge(V2nXBZm_J5Ih^hyl27djTMq3{R7bi4wY+p8YALRenA81$~YMTNaW&S zW`YjX*vbq#{J@|OG%Zilu|3t*FuXZ9_$UyUuqFmT5x`Ko6wt#<=x`zuc$JdnB2U;p z)c#Qw*%V-k)X!@@04{iTLR_#8s2>*x22VoAl(4{=lf&GzM=xL!*?^`=_Kp~t207bg z|24P%*nx{L_(>H12Z-p^L;w_mLU<4aWB>@M8RYg9v@+SPX_8BH87)e&F8awY znjIE8iOj$Py_te!RNPl3sM-xc)g&?>3!DT19B{y2L2xi0o)SP=3j+UALP7utG~T}( z$gS*6fq28Eg<}EOE2&@t0EFdLCcyTWFG%MA z7|<+I#bruuRFhmOOviNpXa6w%pKx%?&OQEsN56q)BT&)(3p|%f58+{Je&Z_#qs27D_>K_!;QU}8_l`sNq@O~E*KSA zg+=O|&m&n1fY{tt%jk$)o~l_S0mfw_7PpU8{{0it(i-d!L)&~TMFSC0@xf*S;#SK< zn2ac{PNXcMzztSz+FL;L5-Ssda!nBnk5!dx2!J6KJXg1_9+Tznz+(>#u?AZ3tJnkX zSRhhGR!;GoVI8`>x>_-DD?wPZNcuAWGb2A-Wewp$03sZddB zDBhk7A^`wu>y`QRYoY8EJbc6>7hoSjZ40E$sy7#T@r z1CGf73{3&G6+i%m|Hcg%ES!(iVY@^x8Mpy5vQC1KA#X^qqHyOydS zpL8=og;oLL+XAWx!f!5fOIE|k^!k7bke4@KFS|kUd%9>^C5{bAKX!0TpAG)}5heB; z-NB}Q{ApKg3El!DwkK0WmmzaExJ0*3N#@@5Ojh3~BM+@k^?WV?a-K__y(t7CN?^(m zU`l%>q7}eUfF{2mTz~)`+J>F>~Vu557IJ)5OxMA{wTVB7JfHIp~ zluq3z(}pgX&A34%Ztz$}F~#WyuP)pX@s~+@Ux(|xBm7DJk8^wiShP+!TDLd)Z9O{G zy7Ak59j5V|Xn&e`KjgY$)KR_%nyL@9PdzyLMpd7!s(bu_l=p*Ust+FZJvbrp;K2$F z)_e>-K>-E0wC)q3`gnZ&uT0Av^*J5om3&(#Ro)j?jgB=MPnZWS3YS~Xx$A5Y1uxkP z9)L-OGOcp41*iDRmC{?M5*phA5jGT%@BoC`*bzI{as5*VjjxlT+L>^wGd?}=qEeC7 zr}n=quqT`-fyze99G6;%9aYy%s)JVNIHTW>YnMQ`m);T+DHc0jd;+i*<(5=;=~3q+ zXveuvg>`#|$j7tA_>*J4_r#erkKeB)HC~ebi+uOb3Q&&T9k6r{Hkxqzl zyDuy&N+li5y&xj5Dnx3g{Q1VakB6IMh4LJQcCpFzEwO*UnOaPBpZmk#kqx3H_R`L1 zH}?J!pQRk@Z0y|rB>wnMCmP+g$5(!TUCO?=N90>q=^uVFz0s=%>kwcl^ z__r#$O^5*5dg)y@V3%TPA_GOEo&YMu)q15N>}Io^C-?s#<_2i9bP!?X>0KiK+q}+h ziT?V^r{q89^^Gc>M8FZp*)(2zw(p~wFM`Lxns7iAs-=}-vc?J}_Y#RIyC~4Zx=++}Q+Ie!>@r?at02s{pY>TX7>85P+OYB*H#zJRb6wnz08 zpnP|6kX%7`8QzQC*>EyYi6Eq^Ow)}7La0DXS%5lB;Kna_00FR#0?mW_u=dE}Lnob( zypQD+0*`dltN=f&fomcl^DSN}>n_=#s~yq&cjaq2`QRmjjp7VoEPe%4#as|zh&z@i7yIw*OrSPf3?#-wTzIlVJ%}xaa07ZY$ z|4mZ!DZmU@3i%~e)`z)q6PE0d+9oEX6Igv#7?{5W9}k|2=}W9-+XZ#1z$g7PDuBgo zTArG{*&!zPf&%Y}o1iyijr)YSp=j?tBHO;s^}L#6t2*d#s5zE7IU4Hv>720|xpaP|hRj=Y@`?5pt^Jp$>|g2{ z38_^!VeLf&^93!5I!!Gvg?s52^xvX65Spizc5Coh2dn3k`pl6t@sUm<+t{UR?_M-^ z3M}E_wb|;5SEg(V&zcL>kpGsM5D#d|*j#bc9X^W`Vh*6a>+$6BwOP2p1`4>1eI=Jj zF=Sj7v@*>R1&R`M;_g-c;Nb#qDTZQ!yw{oI7GTpLUSsbI#gkmy*#LEAs0?yTzo}&3K}p zGG3aXo8R#Xh%aZO*C5-$Lfd@JQ@=iSimf2dHY?sZY6X>|{RuL9ySj+Jd@!BaP$)Hf zg>FhE2+{3J#ZWxQaSGDrtzlnGdbu?QL)DVd14eMkQ+>~jj;%-n+&r|e((#9rkuoQgLWzJTnp3C zInMDG{@8cl1YU2$yYCZe{2$EiUz)US*?^DN(m@R&ybVE2kL6SEMr>35p`p1ffpBE} zi?y%}tVnU{RvA~=9{fl__IxRbK#YBirW$jD?_-6lMo`VTcLAtb7H9qbK&eHSTK#S@r1rJ+y0{k>gR?^&VCOZFEyr_ z-^!H%aP1Sy%}d>OFU;PnEI?oRA;}855O%pZgq7ZM=ob`^=E(mD3QIIf3B+ajJPzBy z=<)txvGwg|%-(J)A;m{OkCmULi3T*BGhW}tl>Hfcd+p@G*AE3*v}nG+`qYn0hO`?e zsNdH%z@?4D2wb;&s1ft4A&U6EM7x8%Tzs zI&r2IMw#0UqvrPq1Ws9}cw<3_f5)d^b-->P`R0ux>2P_CWLnyTi|llePZ=PH{e{ip z3@N4z8xzc$JhUdmMpjCV!cq5FH-2jlVqg_kGSB0}mt}z%=C4|5OhwEP27t4or;921 z2j6WCU3jJ*wqPkkOA+o020Nb(GW&EyBeYW2<-jxgG1~Bj%{yIJpXA5SFmf0E9HK^F z9!qW*=`m?~802fSTJVRU`n6Nx&q^O%3ygbL%oq*6ZRyARy4`{*p86s~i~0ITDWAjP zthVgSL6)&prYJG9`!us&;U_nWVv{CS(yNshJnREJ)H$XqxZOHSHr#YjBORxag0hW<77aA zS#=@+q|uz0vccMN0s!SXz2k9EU7@M`1IrxY+`95vbtblpR?4f6pfy68OzY zYVg&v-Xxd0B!K{wmvIWJXNDsuNTwv(((kM&4AQVspl);mgf@W* zw(E&dNXioTC@LOxdrcBXH$g(TG9~jbV#N0#0F7)g*A5;;-O7i-vbk_jZ%map9cd-P zfP#AyST6txyd1Deg(Ab?B2)-Mf^m7!bNA`-j~#MT!IoH1hS7-S=wJ>l(rXzsemCFH zuueE4O_sa3QyjiHm2&X01}y%bMSw206!?$`AQ(XiEpRv#@xWJdRxNEu@VTq;=qlF*@MN$yPRvTI1^Y;=e)NreiL8Uc9+gF@SmhEBj%hDz6_Fcea)=V(8U- zVv4Tg0*p16N?|A2?A6ztn-N}?O!oTmRi*YzQW&$LV==-e)pv9Eg4NO+&9s&@<0IRCta`robf zp5p%2Z;}VZx{{D^dP|vWWX1vZbYvei$O0Q-$6@L{~U(cnPd)rV>q zA|xb!eR$#=V+i>mP3}|xg21mTq~fNb`^i;6O#*5Hu&q9-$~Y`WFzWEW>zxXvo3CMQ zGiaS`{m8pOsYkt-6JL3<<>(zZO60)X-h_`8+3`Xs%B{DO_dvp>xmOL)UT^FiiqB8W z`oZOpR&*0Z^61C^!h3~^1EYHkM{>@Myji?h9~D8G;(E`iy(aLF8H)c|yxBV%A->I2 zC$k|G{e=}y7f=uq7uL&>(`@t{rH`RQTYHj#vWj#C{WS^D z@_`BK&r|~6WDh#&vxSsiaRMKu9*rY~=-QGRqFoz!U@=3a_{s4b+H0VR1QRhP(Eo@w zfP|)`0tn9C(QFSL+kemUq)$XCq)M0q6PUe2iYY$cx@%n*z?`KMKdJh+7b9sKI?I!L z2;CXTTLvYFMONdUC<$WI#)kurF=&f>SMAGxn`&%%}JatID}-k2~PcW$5Gc2Qw3L z4<5Xi+{SoLOE4Pmc0V!i$j;gS??}-8>CT}5qhueP&=N*@BnBc!FbeE#S$u>Ji-9N_ zFhsK1u-k0jHa6b?8$QiOtg-ob*vQWWAp@WZ^zlE5Lc6XmK1xCm&9wtQk|-5yEcgsH zr+ott%QM1gA2%t(Gvdq8%?}!Cn%*gfy=dRn?&~xxSR`bX9g_dH1jR_gZ5Qj}PSM7aya0utZaWT7+ zeya|^=T*iwNxBdE$dSsaMd)CI$~Fu$U`jcvK5KkZQlrSg=^Nwk_#;G*fq!=xVViJ{bL3w#k4`Rbu$PLE3~SQ>t^Ac320mtU?vy z#F{>`i5DKFePvzkS)`lL)==kf}pJ)2~2!l{%HN}(T29s#(~kh)1&v+tnY;|(0?KR`9pgYuIcJTcXnH~ zKLSG@v$!zQzEBio880R#=V8d_{;}bqt6A?$I4$@Cu@n3ZmcVr0(Y3S5cn&ludzNEG zw%(OX7Gbwfv$^#?jE=TF8ym10+W{aA7G7b_-5Pqr-AX$)8k*(#=q{2f7)T5%{|t!3`d?H(!yIU9l%whtZa7g7R^SUcBA5P zAOQ<&WH`D6@EPbDBDt74aqwAgD?9wuT9KJU4>GDU&6XBYFVaP_MxnRUWC2%NuSHB{FeWhf(7|-zPEp2) zCa66o8M|!zyiRJvkSwm?a2D8f#HoXeTR=a?tkXZSsfU3pc%fYLLM3Tbrul_R#;Dqi zgT}6d^5Bc3!Y?KEM%CuHTi|9pF-bPz33;2!j4tjq2JL>+pR{ffMoG%Gi=+=1D*6kSfGh_qRq^-?fSHBDfnm| zop&qVZrw=<>a2uywo^#tU7oh`p0T|Lu=(OdE|_t8=)Dng%6#Qby=hcH42y^7s@!c#*sMPKzLvMd}Qj%G& zGIy3bo4C#js%wv6i?_L&wNtp>J!9EiaFC$#xp&TF?#^X#&u0tI({dE%bNA=Gq4r_B zkYFg&Y2V3Sci!&$Oi|7J&4=^1Y-habOwU~w_<%3|{)Yng+)aB4Ok%Q^Z~$3>d~GQL;f%&H!7KD{NtozNx?dHLE**S+P+nW@g>uWU0S z>I&zr<430^{@$Il;mcmn6Aoi@3t>PWmJoj3#dl_lYtD=7w1CBTgGtC;x0Ab&LvEu< z%;fvQ#m_T~U)H_&2b0oR&t76N=<{HrYGK#GGgdS%XTne}u=nYO>yC~TQT>e5>5fxt z=M&A*n6JtSaRr{E(2@gP$vHDg;0eTNW#+%fr$nxt13zP3wM4ryAbOp^aPM(GJDrWS z8(~-2f?MWr8pMax$aBa){_q&SW6-fO&AK|L`aC9CKG^4>0S^vf)<$vt?wsD1fKPNR zkovJA4OzvCtjZ{^%IdAkDXxBe23F1n&7pklv*V@oyhl3`2c^D~&^M}Q_#8i*Mu%kj zpm3$#`FaSQ}HkWUNk^h<3USi(?V>Y^I(~f z=RQded?FA}u@m}FoDdhc63wI5ESPJS2Zd`^wQJVxYo~_RY|8wPZt)%EtRaK_jb>LL z5|-`HWT@!{aDHVSeODi7|9l4{AGnLKhE{k`nVU85Jj{=TW8&48&x$+q2tbrP72g-Y zye8k#SW6?kkCjj3V%7z}-o0hR_ca03Tb~?R4QIz!D;pugpb#hth!Wbkthf=bw;|FA zlsGirJ%Z5wT*fx*uTX1FL%twA>#O%!7n9~LoJjZKJRM1eKXVm5Q7arOpXh7;A#x=^ zZ*Lug2sE_&kYabkFZ9DBn`T4X)8j=><2YX0pJ`>+oaVQste?lLjVhKVg8eu~YkfV}~AQ`ryVmVcnj|+z%B<9I>HBFRsiRa{ez1H-h5W3 zEs0-z@fdDS4{pjFNFxP0MwM$}Hm};H&h1|u!F5JwV=q1~2wBe;41MFR8Qk;Z!d;EI zd(6+HUwN9_;Z#(jl%;mOHh0_)R>JR~_TZ3qgCbS|yn`RK_l2d#i8p-^LXzygtKrJQ zgiPQzRpWRRR8J5ebIWcu{0_luye}B0WqwI1{2W&E`Q6tG&&pVBeDM@3&XNphdJ!&B zgBYFDu7aage<1wqy~jI&dK}{W1F1*fYQKc3%a}d6G{*U&+Fn&^l6CN?$p;TQ`!8cK zDx^X0KVQ4hFwd|5GJ`h)3+L3N15bEQE~0R#w4Au4BkNzIm0w>GyEr?<`lb;Er15k} z0D#8L9_o5RS)u=p#e@OR9Nqf_9LaE9;sE(>u%c61PqqjA)2j<}Y;fmL99;+GzX?vU z^9#C?eXs=!Du*}Ydf=b1V8qtw3C~8wt@rr61m!cw-?FrdHYO+u_c(#9b^Fw8x%2*M z=DxjrN@0kTVJ6M3$nsT+*&KS9#u7L(Eq>)J``Ycid*D9eqnCVT)wdF==55*d za38?y3((-P=6n?Y6qFIW0u?`S6MLey194DYD?(&BG{}d|Dhyw=&jw72Q-tO|y{D9$ zs5Hq0Xx{n9s@>}wNikf35&jOaz^6c!ZE(2$_hgB%D_cQl1LvYSiuyEKM6Ae%rIa+zNToe}Mt z)qT0!QdkXAoP8wu=f+6gi_!aE6e})8cTr=@b|*9CQnQb|iZox}ks675cP#rmK7$kn zy{?~|7zF*-nf1^iGr=N@20|AxJCnEH-(1IOaM?#sb?zl^{Kz+mnd;h~= zK3Y$&GEpLf-ba(pHmCgkxD4_gWe8>83_6u+lr0Y2W1iDRR9XT%{$rpW6~s>c>k|c2 zvDbhf8U-~0Fjpb!K~hVfv`ibQ#y=sA&8yV4@dx-3_$+KM76e!pNsF8N9E3&uXAwuo zdb0!oAdrbADtoyMmjK$nCYUY94e;A1iG8?ZFQ*|yV`;6g_{wLsL~lH)w5bnQ7%N;7 z@1kymE85x%uu5+KvL!3t+%-AzvVQxms$3|Oc#E2l(UW0UZjzyLMP;VpOq1{PhMA8J zVN2uoqj^ra`PRURD%-{=b@$Q@a6!&|l*ZfcsQ*?UN(d8%dg2ZmcF%<3SNfPpIlH?t zn(u~^X*DR-I~Eh(z7FtndEpt~bak3v8=5bSf9}c~@(Q>h&#^7m^6={S&Rzyr){#3? ze7EkNOOuM%8a>b^zdzp}w(p&CELz4Q{wmpV?c?zBEeXW_qGx9S{ROxFensL3eK`9# zCZmGMA&mgq;sjVVos?2KPJesCisN{c$d4GJm6`~G;yiZpV8DY*PUhfOR(KYqe?6yP ze|yA~eLTPtJE1R5FeO9=Kq+?6_?^6Dh$zZlHq|ai8!X`k6scxu|E}Se!7YPk4DoPB zZ&b#iLZoTfBL`GiI*&~ORQXrEIZkfd$OnTJb6`S?*SY$&@wX?DqGFHx2*#o$+HgRs zC_ImJa`N_;EfR*|$^dL1PT)q^tye0-FD&M=@d6}zg?UHf+yr+9Mi3|jEBQXH4yX|7 z(?3vlolZfA-+VD)%GAq+RfjKDeZ&X&zsQm>?%^V!_Q6x8Ck;s(LK^BZmC4wu9>3gk zyX#j@@yc+_@-#g#rg=vB1R~AK&ZIO#O_Z7fdfuw6J~MsBp;U0)Y5v&mv*WK$I16+- z&s2Zubh?t1Yt%S3cmL6Us{uBsmwoO{kAH&NUC!-orx)sf3F$wjz$NI>+{+_Z(%i?V zz0lliZQubLK%AER*4vlQRefrshbsdobNMiMn?aUMQ|?&MVrj-FX7ljU9ulwI!g~Jj zv*tI3ze0|mE&vXi|J;#-0*Xzzxl&(xcG*)v&}MgDh$R*XY+9o%I(`fQC73H0RWWb} z;^;$^sDk*^geriAsJIOyiN$yI+?$)YZJd^yP%m&Bb^GG0-mNr%xbt2H7*1l-AsLB> z&{X5r4^>0^CQ^p(dbHI#KYruD7ybH%q_-SHFbNP04XFENCxG@KLniI)LBYJIeT)0F zl}jvO+BLJ4^q_M>1H0TiECn<|g4<3+{)l9YakQL0E>$h-P zmB=~;8`h%rCSfU+=nnjlcCG*+n9{HYK}~oBG`L6(1Nc;4Pd&*2 zzgzdg>}bZ%I*^$=RDQ@z1n+EcVY~|~_+n^hhB>L#oD1+t8TgwCBw!dFzwG6VF_0|Y zX)f!%kCLlz+^NuAbw3oAP>dhKYkf3lD0Zw+Q|mH=cmb^3Uj!AZ(pJ_hy@LjKbAP?M#te}<$g~)un-u!)rh&0M zDx60a(B&9OYdk$p0-Xrfkxh})Tn6o4<5jC9!hrrJTq0;N`iba?jqKAx?_Z_>Luo)R zeRv|Y{7nj^f{++CGAiddFm^l~FVbVw4;AR-=5q<>*I$!y)%X^Kw5fFHFnT9qEC|*q zTVaOT;8mhW({R@y26q;ek&PfXW1Z$(GyBuZhoSJ&n^i`QYQpsrZHsTJEDCBg3RK&j zU4oX%)?iJ7kHXZ<;LC))e0F(6Iqp`HD*I_nIC%`Hb z+IIPM*mZ1?2eg-4hll%~ZfN8AxA$3*Fk$L#Koz*o?c%=PBP^ks2f15($HrbM_=g9grjCD~NaGfdJmaoLf~0JsK+l5Cuf! z*DPeA^2K<68YDxij7q+d&Z7SOn$Z^k(4ji_;$hGwbuF3W#;@g1|BxF4 zG`kn6-)7jPf*}9l2g~IsSw0xT*tytIn6m@ZouWVeb92+x?9!%q)$X+RHpKH4`jt!n zvDvfr0}5>|Yz9RJrZeMw?toV~|DPT(XC{T=jRF!EaT48QWQ zH{Q$2`Hy?R@(=YjAa{C?$rYKyT*-kB;n4@9)Ypc10Gd4u?W)~&%u zuv0FMqGEG_dBX@+m zcVHj=+|j6i|4SxM&a6HP80BL&Og#tgPUH;EsA9brSWDd=0Lix?>o0Zy^FjB8atVBq z6SQxC-~Lx>GMZDGaX$ZCNk385MyFjOtB=7@WR(n@}(R)=-{^ylrP}W0B67WXro%v1pM?3J~+XdT*txzR-L9awwCzyAJ=Fx&H=mFYN;ElH7maJ`FSH z#?(ygBuyT#v-!;_s)y8ca2{x$Jx26 z@2i}qx$yj^XOhxY`MJzIsJ%J)8RySg6_k8^#pc$-to)xB(}rHH$=i{wO*F;%@?VMFcMKn-fRUwEfX$L%(}gnf~UW)V=nhz>XMQ}fU~a|u9cy^i9j=G;~w=lV3y>W)??2iP4a3{IDlZLalP)xs(o~**(m9qq{_7z^Y zg=%BifT2PSAz;pt#baVVtFDI0pf2VZS&Hbb(fY0@FeJS}MhPJb^j6oEI7~7c&pzG> z6g)PFPvq>7GbTEFv(C@r@z&NR+^0_TC(k4R+A^cp=${6Yq`vYnP~t*hqVY9(wI2!O z`-ulym)6f-STdwA__cx}a|a!*gHBlnxclC5X8P*V^P2)TqT@H5iUO`M-Q1WP)G8&K zMNfq%oP4gV+O#Iws|qEGJB5ntzci3~7G7%}Id3{07`xW{YoVVG%9zBF#5kvkYyg~H zR`l3xe!%7EF1?V-!(%p|gG&YM-5TBJF+0>vC50JMJb+-r#oj=oD_E3N9KFMSHOcG1A&Bj8qpLilxQ6a4(DIzaFnRIGlpz%u9osCa!{T%TB&v; zzk4PsetL|2zOZ3E#zgsS@twkWW#`cvPtJ5W$o$$`Z_cJWyk_t?;3ZNc+wODFfpZ-b zJ9pmMdHwiweQE2ubAEMcS;gfc;hcs6V*=4Ts=oehUyr;>-ih;hUO`1P>rLkqpy4U}e8B`5w<7#fUa>sZQB&*D>7uCD>Bwi$gnz{x` z?95ncCQXVwQFb?*(m);#X2wv}`z-0o`vb(*%{$iPna>&?NUc|Eoxh#A`S3?V!6LVg zKTy#7Q-?#gZjnlZb>mtL%@lt~U8aBPzM8F6Lf4&3GJ^X)xSwRSLkQDplql+R(?N-% zh>FXb)*noY23TtG~cTP)tYqLe9Ph2yRVya7^J-pWL=+o{`Q~hj>%FXM+>G zxqlEL&*JX%HYDC{`0S#5-a}2@(L{}E9pYReK#KIeJ!K_tKX@EP0O9GS{>q%$Bz>bp z&NMx}nfSWl%j?!lZ@Ry{>ASQr`eot8rMIuYyj}hBx}7~{bcvsupi}|iUHh^u5V|7% zbwyTVv5df1a=|m7LBmb+6-~+1RVI~%6US=@j)Tjl21Z(i?oIhBy$IB}?i;!^&iUL~ z{_*O$=`$PCKTM5xNY4|k-&}`5e!IK@-o5ukV|$J0$l?`Ew3b`0aP=6z$Io{V%1Oxj zIQ+^3DpZne#2$FCV=BgZo6MMh5G;4Du&_msSI0jdYQj*~F`?zm^aHzx@Yso00~N@! z0aaWc4uIGRXuEl9)`;1N^2Sa&=@swv$F$$bPcE^J%nSyDRxEhRE+!a#{h(Whrm<5b z7!j@%yRDQEj!WOhO_PG&E1^#Y%P*@jbCb(L9QIn>KJ4(ir5PW&Rwot&whStmi#zrLV z`=$x^x#h__-Cv{sR+36EI+T}9IW4&6x9=&v1IDJJFi>)dY0QyrGc`*oOm7RT%qRua zfa$MB6C8C&^hE3L=LhsjgfvudEZv?PNC?k%nhofaDa|@C`Y_ZOS#X6j@+nTr0pkHb zKf6Wr1fB@50Sz5}v~#LI^3c$-c}a`3eY0cxF4FGPj@|b*o8<3ss@ZA2UAs?=#(_4R zCF2-;*FpP=ZUZGjU}w9UAdY^1|gW zl*?Ifi{kTvi0k`eP)LT?n54P1gYk$dRL6`z()k9|?xbZ=uGVw0<0C2T^(_h?_9EE(*@IDh)|v51B{F%jjfPvtagkWt-Am@~=j@3(J4b#YKD zr#)c&z^Qc9=&)O>ty67^Asg1A_~(`Irnd4NxJKY+t=OMh$(wZwf9lk3KGXU0%%Dkj zu0w+w>c9sXPL6Pkq-8(3q$88Q)0+1zN59)*7vkvC1)LX6e2hr^<$+sb8kgIcx7==e zYsF=!pHpsgS)5B+ok4!B>`MBf5Xqx+zk}=huuDhm&!<_hddtNc+5^Pr_!!e={?7Ay z5b=Hp3wy&4yqt;GjWPdeeI_!ZK3lh3(&mcdVCj(ykZ0PpfixYLW+U#=^DY?|&0jpX zP&A5D?wA7;WMUd$GeYNj+;dwI?;#`|roSZE1eH2+K8i7HlrYV)5hd=WO47WAXM4tjNAnqpI}#;qg#gI^7X=heIy6izyxJb7N~@cT~Gk3X#z zga6*J{;}*jA=0HKxW9EoS2}wFFviE)`Kqq=abo|~Xs(|oO`|iSvo)f6rZc)%8$C1D zm#Dui%=6_^>otzq^}#5+8&R8`Jl~E!{PsEO>`IhNX`K@M=D)A3e>Ngjo6k(N%6AhW3Cby4L;xSrTUj1nTp_DU?<8ukHqkbQ@7sh!k z?rQ|`&m13Hs;7nY40uJn&n`FdF^q`CIhf)Dqx3Ay$k9FK8 znB?l_2eZ5f@2XsyR1E)Aa{0!)aM5wbHQ%?7Y#%wa+`E2-_P)lwH}}XN<6J66aCNdp z$5ZFD0SAr&J2ULspgI=j<+R=ifhP8-69WM>;O{@u?)QRmbDsp1Ic; z+78%rB%`rr4T&v#R=DZs3tI_83O>G3ES8RTg2D!%I7)7nP(##04n2oz-{4biI=JL= zo<18{+m1(LXwLZ9G(HN#whL{hIMyX#YiZjp=;W8#EfD`SzFow(t558-Gr7I{NchHB z@6l-2Oaqrd0bd<^-q@ZrjDCallMuuC;@6@IjGlo@0YzO%VlbT$i)XSEfu6I3+hqCc zI*&NZS)+aePu+tx-@eiMV!27u7ThVzGGBQ-I&um`Lr$o=;XagAUy8@-<0vBj6JsW@ z&Sj|nX62dsOL+~^;-*^PChXtz)w;udj2Xs?TC71xy!e5=#W9OpPu43h8KO&5@evJp zPRSXRVUByM%8H1c`@^py%E?wQ%Oj{ZlTN@l*Ub-&&5u{mZYyIZAN?D@O@6%EjyyLL z)UWiOHWTvZfk~c;JVCbeymj01+wd<{chiKD3mh&KGm`3OX`uLP{21N-(B^e#Z<`G9 zwt_mMXG7SElT%H^C9G|OxdbP1PC=57LPRg;x-)B&n{QEmW0&u!L~te>+ju)&WbenE zl=NB0KG?552>|GOqm5o-w3v6}s5mchy7OgjF0E+a`?b|@xAI5d zr>_k4X%)@gj^ZAe2g^klb7I$Rzltl3+{kTyHmfp%F&A~Zol-r$lb3a_@bN)weo?eX zUgh2`vE&ByRoBvcM>LWOF3X$!&Kgu1aou^N^2X|F@Eh#wTpMZOyTESt-(bPE4w?KRDzGok^7*(q=i)V_;WNvUy|bJ4SG@L~C8Tw2Nndd*wAM>kkOr(dRO;q@>J{2}#8ZhHteJ=i+gS z0m`l@BO;nu5kh_J@Y2pDGRLI##Bp%W+g(0&26d!z&+fl#+9ooIKn$~-i(WIw&`{#P zgR=n_(Oim%3&ooBbx1HwK>8mOTi~WJrxJixbGQM`$H%24#|m1)&eb zXXQ{566nsrEL+VVT(Wo5H>PM;2D6Nls3T=tQdi%;rh7}ao34g?nAGO|n@tj6GdOYT^av#bjKL6k_E zgCWK&16x=u3+D=?&6fv=8Kd%iPjAl@r;7GbaMTZ@tb#>w((x-UUDup$ZP`Cl8=!DE zfRV_V5;+^qM<>$eOj;nosU@%%H6x*acN3S7Hsm9+_ct>P;J%8! zTYf{+aN(V-JK->h;tnrYpgC%EnIVSn1Z*wK5N-ZUU0Q%U6twm(%vL2HHi-XlHCP(v z8U{1_(&$=R92ai91`>vDac4EE=-4nAk%w6tfNaV$$S_|{A_}TJ1I{>$mM~t*JV7a# zP`+4<{X}w=S#s)UE3_Dx=1uj!i5QUWbqFGYifAW;I1K6fF07*=3#?m`cMnPW=Q%^t zr2tx6VE%!;@Y?&^c$;?^zgxz4RSA4pDS-R7O4-eSH*c%Ia({A&GP=8T`H5;Cqgo5q zHn=X&M+v(_I*4ueZ$vS0OLYFjK#^n%hj@CfyRF^^g+E}c^G8eZ(fT%3uYt?dS>|<8 z&H;|wo(wc{gE>f81#$Pbiu|7j7N0+U*P($g$9~l3=rcaxa`{7vy@ZkxAlU3Gy}XkA z!*&nsDHt1YVQh`*GAo8XQN}BPQ{NUb}Tz*lM++#pGt6^O^pa5&RB0? zlbe`RRCQKnNFz8fdO?3By1p;AY6hQd;J|&?jHEBK!Oy}_?7bcC{Y|bJ(s$2_qFK&< z{h25?e&vVu)~@y7Ue9X~E-;#}U=F5n&k9icNVa3i`m?Q_CQc*}}j(D54W2r-3e3NC2=0&FCH!a1+Q# zIOl`sa~*A)?$ifV`D8|aTO*48Y^B0kQ|3*^ra%PdO&rfI4ZcGt_OjRQXCxEHXdgdc?BA?LasQP*G)shsS~Nu03)3wH8-wWgexR~pyY1cqIj|e zDr@ctmfYQ1RHM)QLVN?5E`w((I6DGKh0&nqbv`zU@gQ}qf!XiKH-f-dY;FpbRs3D2 zoW)6Q33X$OmtGE2;!}&fZuDX8+A0R-9H38+N>MEm@s7$&G|&cTx=-Lrc4(#<$H*gF zCG;?=FG`~A4L`oZj_g%dag+@Kl%SqmP=v07 zfOYO4R2K6MJ_gxeL8VPp*e%V2RSB?dW=C;X-BN{M)76CB$Dl|0*&dm4P&t}%7R7S3 zP`8~UOH#@xSjwzU%4%E6HcQHuDt#J&>5ogF<(IJ!mO0H2BgRQMtHe%8!qvZd8WQ3Z zhdK-w#GyrxeRw&xE00bquTfJ(zs`dvNNckM7?v&yGr+@Wgu7Dop(XZ^iVTeQio*av zf(~~p;=JKc8op$^P5}QSRbZ7Yp_1ZP-S>VMp&zkpLgeQMsiaA3(s%ap{=LGBEy2Nq z;MDAe(xdt+oKGu`if8ix>~|s{PM@x*aV#|FiX;oX``eNojR3dK3q4E*LGfTI7P)^3 zB^O7_gqN@PdHs;juRTh=@}9vfDnW)wbc)6dj8@z1x7PJIboCg!<*MGN@sY_yY*A5N_Hh||blJVdl0lV^?vX;K}dZLL4i zOLbQJ)P6lvRwW3i7d8InZw%fQEK`6Leiy=$amCe!l6Y+B z_v2%yfyh*zi#s?t^9_-(AAyJEAygy3i>gr?{3_ATZ?L&P{uIc+$eP1Ji>kNrAe;f_ z8jnZ<0hT(C!y0RU@_Y5f9!IOSC&5+fiB^!=Xdr@pqvB!4D)>wGq@U(aq&Fy@VEI>e z=?6RV+sEMJiJ%b3A(0;nThbeuIHf&)*N@J10#>r}@NzSmj|j>ghp2c0?N5n8C2IFLDMLMGwyh>K7hPQtY}_1qK^mL zF~1mEKrWwsZg>Jbk+Czh>y+;iux<>In1lFHqpQEM1$-B3O#sznu&8u?RRWmd0QRcE z;S_f7N%Qs4&VqxF^TF}n6d6#Z2>=9>3<=vdyxuW44kV~eOmf5a*-Nhp0tos!Y}*A5 ze^NjZRhkT0Dq4P?2G5#&Grfh=CwnndaJ6rlh7KPM-G4$M-?R8tTZ>)}3or`1)2ASZ z2aMdpck;~Wht2FYn)}S}1#!1_xZmVrslSs6i*`<__(e`L$W5l*`AZy5yGwT4%b?PR zRA}`?UuKO+xY@OL#EtULcm-EBf?1M~OCa2&?|(?0Z!xaIy?j(HWz2K6=;U}w_%dJk z@(5<)`_<{o*od!UOC&3kHy&9vRnL3h#G2%b^2*tUG=w;y`CA;fmd9OcPqNw%Cx9huB6^09wi5UMx~ zx_N;Toq3{S(jmsv44HYL7&sI*SQ?S(6+~mP#>3xhtbH#!hD7N-G9srda2h{Va+{wt zu9qLSDBv9+T+NNPdcR{cN#ByZF8#&WhV16LZ@*69nIdc4yR7PP!@h_aXsOIsxjo&w z@G4TBoxZy>cPxX@g@Mn#{63%Y0n#hw^>2J=Ulk(sVahUh|D*hA_+#ELLMZd#HZ|I@ z5?r!1ik(uGX3N{u-MV)bL_xpsi~KDey-?{0=P$2Mb|V3D^<25aD+#ru`}DtGgn|`d zv3ttHAm#A`!Yy8I&Bw>+`jQ_Sye%v8D;lL{@@@x|aq*4D1@mfI$EP&To9V3g7(-s=WK;7@hI=Y?dYg%kGBT*`ls6So zBNDh?IjH`;+SD4ZMw~BF^E~$;?oS8TeXU|#Dn1{HRUSbEYHCY5mX~sF95L)meHO0t zQtS4s#v8KtY}XIcLnLos`klX&e>?C7aR~Bmyac_?p*$&xu~IJQn1&~k&1-0b$Ossh z(M2p+w)4apNg?e=N?aw`-twJW$U}Y7UHQ__GF%wB+zowP1DWWoU8Gy`9j4`W(O&_t4Awacx|kRJC@^L(__s zxR~TVQx@3c)zrlj#=}%RpW`6S@gL{IW!9iIZLexoqWXbPwX}=P#_zJ;#uwLL(J!p! z%(i1EVk}NxF^A3HN!M_^H)?L#@8X?l<^3`1J1FH@*wq;_#QMGxj`ziV59~JCP%l&| z3e4g7TVftw`tXnP8Lmy#A1m2ApD|OAwjU9bS$c{?a`B?eFlG?#N!`$gk#+!^n{!cm6y8 zcB4JWI>|wFe(9%iB(vzGdZiB`<~t` z`>L+2^pE(GJ1gh$gw^kN%)fO79?wc5<;(uYJSos`E>YE|YRQ!zF`8EtuOG@o-^5E| zq`lq?qpw)_*n<@>DoRuL;wyC|=c<)&DDrhW z{C4jJ|F{C8S1H>AOUZxW^uH)csFV|rp7`ymUV+%CwWQXyG}^zl%#&a4M19F`{qo@7 zm!jD(BJD!|1};`I!tLe`fz`(Rfe3b`r}*SXH|~qbJVbsD(sT0LJGXC-qb?RkGucPu zM+x)4N7|MI%V!` zFWMweMiO~JfI!CqhOTjpv1XfRjl7y zyjRk_^-|kn0E34Q-nG9sWBxozHf(=8>1^@0P#Iv|3O?^k`fH8LtxZ%1I1DK}EHod? z;71~WSI?0Xpp3$>-T`|t1aez_6iV{vU=HU1EZnuQ96_Bs?`3g(5?v1(o4OtS`1av&l9ccGb8~hsctCA(hF*K0U`dHx zSm{tm6b2M>!qnxyvH%Vu(`B_;EfxRq4kbzD$C(pzF%>PHFzFvy$b1|&YVn+msMkV# zOlr#_f1*gPqhnkEd22xDf=Fk#noJM&u8NH#aWPF}Af9&bvL^SMs9%|gCmpZIih{JA zCM=D8yjm8E(mxfhnUetBBMVR@(UBpi!NsNQdM)6^u;ZWLTF4MR#Al2>NcS=&&%!F zKRx}3$H8rN;Kc1@0C3%Wd{we44*(FFyS#|OS7Usb!ISy=l7~cv+lHfLg=oNr>)8u5fmOto<{^q$X1sl4jbaFn~`xsSXhUk6wZ)Fj?$o<6R4TdV1Mr5~A7 zX?HYVUB7>f52^H=Xl$>VH*WLZ69Md#H!h4ghUiWz1;oyA#Hd|5vRD;>kvs7;XNfk? zA?)2CKdse?4Z2af+N|Fdd|nS?FcqS4!&-*4_i1OCadl5^B2l(A^KF{?$$|y`e*0^S z3@K!zZ!u}K>|I<3_sNv1+#kEjikoQ2W)` z)~0Vklx)U&o3KULmfex4U)w_(KXhrP$NN`^+Y_w)^j^OdFmmf1>%xI0Z_nRZy({%( z6-(}i)jz*nTI3;~mb6jtXgTZJED`m`=^e!2;NW#cB<)X`3^rEssMw0vo)}B# z(B6zjr81$irtRF$n+)s_lS9Liay<-U@`L)WK^@k{(_g~*Q^7sSbvF$K*QY5}q76OH zG>8f}&f{E)KK1(?2%(pg6Ec5aU_pJ_Bt&cJZzbQT!P{dAnp;UWsl}K_rrq++Tgi?? z#X|LcMhX|(1vD7OxZq6#{K-WA0Fq>UFp!38Af#1p%^yy{b3>Ov96HSd-l~+|n!if$ zov({=v5Bvb&Mzb;T6*oQ)7qgYmV(r(DIo3ua=*Frc2?dHt5im#zu!7yJDZ7kNYo2w z5F8g{u&TzAGo~UG@S4aa%bT?sz&+aM;fMIm9E)UbErgdpop(HOoiZ@B6KB;omXHKZgp3Peb*@3;64Mqi=>!M5*p%-;=Uj+N;O?TPD)U=OD>$LA^a0~xAg`Oia z4!VXjmv-kG?>BB7Ol%=flNOrhkTqew7PH&;_M1~T4Pn^%xV?xD0k=&?Xgkk^W2M3^ z2Szor3>t{9NwngnL0=V}Gd1he@BNxlCiIqh0_#%?v$j3X3{-(K{P3o^)4P8sK3-#J zp7-WKC!V)9(Zq6r|8^L;VX)n+8XPzO{m$8pIMRjZA6sG52R)j@wdtWli@1+ioE|X+ zlL+u#Z!_RdBW~YTBfxAA4*E}@+@$$i%*n4#(%ODU!al~^x{F+eiDG}My6vv2n2YaF zl!L47n#Xv3Ho(Gx>CXyQEmyTa{u&M&epa|2xk~lHcK^8+;}?D@6(=>BD#pK)YV}Og zX#Xi9oD`3?Ydhs5HDCXk?!up_t}<=+VSpL0kZWMi)RiLKNVsv$UP>K%w$ljrXrNQ6 zl(xqVF@1H%VvB@&=Z_zcp?f(t{=SbuwQ(xHC&B8r;oIj`cE4 zYA@41wn~+>uL}gT>vAYR(}oV=;mvrRzqEPO?S?K{>!8GXdiAgn%3#FuPDmgqFB$f( zdFQ;Su|Ud_b97JkwhaeXv3pT8B>4zTwDQ`sKPzI_>phtS?xcS@pGQn7dNu*}xf)=~ zz!4ONO&vQir(3<-0OBqrERRjy8$Vkc;O(y!T>N!tNp~mWEI1&Q$+M!|0|&FhXq>MF zL4zwvk`{U#jt`68Es9J;78vRuTzmK>lBE(D0zn5V{mgk^vzhB#PK1VBMIbwoBnd;F z-?($GRB`==MTXqPpZeX-&TkTH%wHOu4zQi*yimkFs9?9B6FO>*GDr}dDU!L3S!xS@ z6EarFxR@#n^2dgoV%lwr=a^{V^GzFb-sA&`m9p0~r5 z#3Xr~G=-P#BfuxT#JiO*cQux5r?&sOGgI?4s^50p=a+_^2BR zno^0H-WgCHxO&q=UNBMcgsslc1VNTD5TT|iN1!4MHL%QmR2W|a7Kuv1_Z z@h3ug>~sqx;O2D&v(OkTm58GKNPiII=zEM4R*-@BsnMsSGa;fNNGcdRC;|Gzk1l?y z2xI^!Y85BgyxFIU`!uvPfC6dZkB@c$@6{6ARS`#n#P-qX9zcm>R640nQP_s>$P`}y zn@b8w7-c5bNHUUJ-gm{wyL46?X((REzI*}m7T?BGv zTkTI56DuY7ZMTteM4V_QBvqe_pM<#5flhgJY$Pz<#s;G`>hI|#Dhq)1<}yiCs9D>I zhpGtgyS^`jc_={eYq;%1ZjRl2(2Wot8ah1?grsQ3_7klJv69{yTq9VpotfUv&K&EY zv^Wh!nzzvSUT^>^?(e?6shk`Yj;66maH}Ptie7JidL(})WI{z+Zxn)_WLi*&PW$!d*0<4Be+*EY3(m1u1a9m==d^Q4gx7OeD3&V&x{xI=+Wpr z3MuKbB>V;_k`f4trDbt~F6VzL%P)9bz(9e}TK=LLoR>TAgeUPA+vInWxt>$RS=Eox zJ$QZ@cgN%IMTev-;eo{`m)TexGzV2pbPHKeGVW9>P?8q1ynLmMZoC`B^TI>$ZrzO^ zUobcU+PUAf^H}YRzaCsb7dmJaCgt2w@i4hT5Ois+B`M-%?f<5BqMIhEu09+;!S zId?c+sgE@I@1lwnry+n*5HE*3x;~R@bd!w&1b!|`f6pYMpC(z|0k>uX@_CSdV90)8 z`B9bPJ8*PgFu&Ju`wATB zwC5w>pYppWd^*7okbzS{0xgd!KzB=KRj(Y9xt=(pQRv4z3wIE0$sTo;5mMYf^Hrj3 zesp4$)fV0K=0o-3V3LExT(aho@@TXz=EnYGac}OW1>HsM6XAeC)oQBo9RWO! ztvc~$+vBFiFFKIA^(Y*XEF_vS;qu^}FKE`k0W$lp1dmu)2JrB5Jb z07mW?wQK~!NeIJ?6M4S{h+9Y-0Ad!XoKJtut%8XBMaqhg(|IBw7ZbNI~ft$ zpsDu`sj=O^cySx;R+sgO>^9k=4d$J_+HPT;MQ~5x&Ch}EETbKQU+8}yf7V;9HdW34 zH~*^BK(eLK197ZY;=gSP%?D~vit z2vlc?X11^&1Y)3w>HBEZ?Lt}}ngFyM`2a80LU1u3jZ1*IV8CAsXg~q_rYQJdT{m?> z|MrI-esU!-Gn+(h_w)SkSdQFL*RCTPFY)8{e)*R5ROoGsYYIatdIuWDbQ|Vskc9;s zbfzi;Zq^;@Jeexk^VD~9vCoaeLM6USbb+G=rCn9hSYVMTn+XBqX%gPg_&2_`eg#2H z7UMsuAc|4Y>GoF+<8Us#%tthi^zZdkJCk`e%+(_8h`P?dJH@guYQ~=*s^e^t{F4&e#D~;-@7bmC&frULwd>UYfU8sFseZcEP5lIF906>szgwZ*H71k&1INCMzm99)^`*gT`S zQ~*xQf32{k4T31~+4nsX6S5kRD5K-`Hnn-cNa+?~_W3WwJN{DV_88BwPc+U+xtJarhBc^8?FcQe#me!oD z!tWE{G-V{({SOW1foIwUHHy>@XbQTbPmylGH#OYofHr`D0@-BDfdW3UIjBe4VdxC7 zmu~iFevs%K>F*o<3}d#rYh);|kOlK7^;JXVZFi(ciVPnH|Gn8S-PJW>MU&kxT!=O> zN1ri64fLm3IN6?Dze|{-#Ud-iEI;kS8$U6t_DfCqpi{IJy(hw-2nYS~T;iAVN4rj~*aKfYo?q zz8W%D+w^MUlcXLOfoc=3nmKwW`pej}qa-^%_88Im(}Jf-R%hUZ@#b;+S0Q`$B7s$l zhl}^Tm2L!7`M;>Ykr%|X{+m{3M@u`g7%S+1HX9S$`tNHSZlmMmMwi=0Pt-zcXI-D@+G>!tgV)bALU^G_rrXyQq${G~$88_$nyq`u%-k`URt7yRzyd3Qwk zC?so8qaFt*Sj}yt9`!#J7@p+S% zwR!KF)!B1`TZgn+lPYu5&+n^aMyW!bZ9yoH&$hfTU64XB32ruG_Rr0!tG0aZJ(&6o z=<#%%u%N;Ugr}cdpN;afY0>5UGvwwmY;X6S-Unz_v}vX zow=~*aB8ox`cue*Lg$qm(;r^ajE-IGfFkSoE8Y17B~EP*m3)tSj{7NPZ`g?ab9ct& zmkpPH{p+xuY7UJT)dVFO%UN?tmgwgoa^&etBRPAU#^HK$FLSs)weBy+{P+_9?P1=e z+&HlM^X>8}vftHh|E+f-6vFLOg7C8> z`=^Hgd+i&_jD0mnKAxy=Rh+Q=jEfoq+h-epK_`9%Z6ONLS3laj8p|!Yw+ENp)*ZWk zm_GTtdPTwAom=aj_UwU5I0PoA_&2b9{mG2HnH(WI<==EuE@^T{7>x8IK}9S7W*G3^ zc3;+1P6tsgqSTYne3 z8gM6EX1hQ|x#ymTw9BNQp?N@0q4&qhs)3NShR8*U0Y`S-SwK}gI%S1+p)DH5C*wM` z>E1>Az}#F)+tBf0Fmh^}CYKv61GrDd6ul357iy^L^v`wrT7TP#pu&rvA|5hz^#_+S zd!_BWLdp`!ijQBU;%lqp_-KE=3+A|&N*9V5YaI((EBh}l&F$0gzGS`GMiFF;#+9p+ z@z3*O6+VRTetB1O>GSO2P1?cMa#OF-nxRnd#m^5)&aCWL9bRKpJr3b${!Wf&zcXk- z#dH;3+!xQWIcB?JzF8gRrlQ{neal)1EDk9wDpd}bQQtC54$xXRn28VS#0c5ujFp7U z_;?wMyWEtCjo3yli4zTzGL^0}+!tbnvd#pX$=q+AkC!RQ2{xCncuw0Mkh0Z`PP??N zQ&Hlw#OTNY+=C4=_+b$ojp9y! zo=4t^`ZP!Kh}vesvnaIO;rZgs<^}BuyO+aeMG79-hh^5@_oe%6leYs6aOGjgeXzDeEed&0^Mn5dEHN>GOBzSGtEsAG+O~`&_ z{py9&pNGothWhy=Ro_lHU zmTYX&Y-Y+_%xsay?IrUNMXamWyYNeeH-G+Gb@2aEch0x!o7D)}B7kI?cgI0!(KuBF zEb52;WQ}@sOS@N7nduWdR>sAM=gARN?P;pKE^kxyJV>hXo7Sy6ckx7ENRqFF>q#$i z+F9FVO1H4TNzn%`LR*0H@t=zf>Jbb58`LLxH!^rYq5f|WChuvtL*{zEl69WE{Gx3` zQi{UM@sClf-yb9Ea64RTb?Je0Z0TEi2vq{d6nKXl20|2Y_$ zy@=};t(EEPt?}9~Pt7mN2A^P`7xAr>tN*cMSfz{ z{NIvyA7;&R5(+Q9A&&p5x$xIa|5Pw3HJf6NA(`F-YIl&HXZqm!+#m1TEmUl1ZA$i- z>I28&M^Mj8FrD&GrTW#V~uO)ZiBVFuzWHVNuxJ;*vHZ&q_L zLWt+Qq>XIH%L!WFO;f+gbJWq>7sia@5+jWH26_xL8v+v8=!Q7XB?E2YX&X zUrm2i$dCvJUQ@WfD9I{mhbItO!6?ns;S3d>CC{UW2PU0fAlN3h|6X;UKzN{h+PQWF znq1#!-eLgJlke1l2HiI|`zAOj;NHbVm6GC@G`Z@KG$|xQ&-z}ltj`(mD|SzXh^9uT z4RuW=qVHsf=*ICd02_IOVqX2{ZO7KbvcKw0W0Zucg<`c*%wi;pdd6U6(x>zy7eDml zItPDE5N#}U#3z>KHrDhQu+QuD@-a9fpY0XDbi|k$OTLm|9{CC?YPRNR{P9}4RGx>) z9;us0au2BySj1_uyEf^DW;K;Ns^&OFCx=2oF_=XA)E;(QUY}}vyL!A3&AonUwsx=< zYoHS}_SrD+gRQIRqNXb;PWOojI*xaz$Ypt*#w(e8O6>eQ$x(QRWyGkq z(V(oWx{{nS+y6II=WSj&TXVCe0)s_?!#2)7jgj_qM_M>G$*e2nz`+O!rpVRjCF0o+ z{k#Xqc=bqhkR2QZQ6?h>;~k!tNc-!2zx3+$Lx&d?-u{>W%LIe{p1pYX(ErN&2$aJt zEmtup8XR+Cvi3hl7uI#`-U&J5JC!%iHV#@1ozt@;g#C_H^GK6v8G7=OLy*X~-`B3N z)(oQ`ck zbu9lsmn0UiV>{19Qj9`-ihse&<2~3Onv!CqJK~aKC-#I9&al=Um4NMncU6;vT^61+ z_lI$34(e2MoQcDh%PPl#(ZflY^(e{4q2-|4tOeJZr-AkLd7{bEc6Y_aJ}BnTb*O!v ze<5|QTji&3dv;}{x8fzIL6?=_yDb~u7?N>^Cc~J!Gy@{7Tikx-V#xhZh&hupw2sz} zXpS~Ff`{&^)5y)JI5{`;`N%@Y{*&z!&I5lnJlA-nuoY`({4`Rh4ttdUJYGq^73sSI zcCvB#YbO)0=|fvCVj&bb_M*>t^~;>&p|s7|srGGMcIyT+r#dzIFMhORajlg@l)8)t)g z+Gil_pG22l;^QyQ`UHWk6*I{N`1%Z)FtGfR`z3QLS$x71D~CURt=TDb5IZr>v|qVGbV6L+bLA@tcdggJX5b#QV%r?WOyj^ zI={DgXLV+}Q?t0XPo4aEb7S`4kjOnX=*H*s_s_92E<&x|S%Xm|+NJ9Z_^A`0Mt|bG zIlY3l}HA1D_<=b^`=@Blc^&=yaJJe6PkRZ*RQ!%UJmN;X9R{zO0{M)Q?#P>SdbpJxNimI+O15_7s1qdLc^rg4EF zHgvJ4M!&Lw=s}2P5?ysI!39s(EajqP06E;Cf-pk}szKZ*mYq<}heu1H;^3rKcpntm z7ag_vRCf)89i@LMNqV}LfI&eiMCMIX5K^Qm#FRcn>cmr<^esU)m^jP<2Y9OEhF|O$ zo*qSnBy4aZi7_rHzRF7sQjIMcRUmgRFO{B{BsIwB+Km(I5RaeVfE={$F zuSH9$_qxEri(>afO@W_Z6HrqY_ek+@lqL}c5WN891|T1xJ<=tchQ>x4)Ahk9KpD${ z9PR2u0RUE~jR}R6Tc$kg!!*QU!kVI zTPensk)+ySrHe3%%}DrQGx%f_gQ-aPPOyHep3>Zu8MO|_lFSR~165r`kGry;uvxfW z1E`4s+Im4!^=`ALtMvnT#hb7PYrdib8N(0S^rWDtbAx)zn$7a5^e{HiybqFJt`eyh zXGykv5raf#a8VjcGE%UQr*BIo0+cBv2>&L(oRt7YHGB?h(M5p9bYDl&qn~e@W)via zI$DKFaf4Bf>2OA)hnb{LZxkIY*#HA>#j`C%KrF=ixmVrbA~;uDf#2iwb~9oMuKALS zVW8UBrPljHJ@soB8QDC*H#9J0VV4gddG>qY=mWTwQ%cDze@CyzyI)HQFK2H%+?g}FEs;RsQ94hx_wY!t;(>#-oAleFc8ht_aKUfiiHDB z`56h&H431Khm@lLHk!3GRu=&NsdS1z9n{Czjt#vH8Ku8U8oWxkx5op?>cA<;XoP8W zG&HpdVn07ueM?(`S9?(5Vb8w<7KdOooVjV6$F!C+N|k<)y>ZQss*6O6JIL#`ZwWFAw=?gKM0utaKRa|UhfMb-LYP8 zEHi)`6C>_V-|zh4ICs6zlByGq2Sdf-7Rhv;KuGf(IPe>DJQYCV?KIaDj2i%f()_5f zcvX_nuRoLLE{|avIBlut4wsnua||%!-9H117O`~RX@^Haw3z5Phk2shdrGq63tsnk zdKGpP%MymJ*L0uw*|nj>nTt+sZLJ+`iS2X4)Kt4uuQ>O2oZ9K}2}4B}k@5<6KEwbe zF+{UCpwT3YnADgapu|ETQ83>xJ=NwsPF;mZ)^7*=wK*9K;i6jQ2pZiT(?2dz&E$T? zKR(U%5$S}aVV!lyJLu8J+pU(fIJq(s{uAsz5zcuXHT}8)czhg@EROIDJlBEEe0a-l zGEn8mM#-~Z@t?#Ij!PvKd*zj|nM>gh7@AKWTAnLUX+Os-TPM17^u^RYZ>b(QH9bRE zFd&qCB8Ukf@NbhFpYU4fDLfB4|Gp1|$ue_$BUEtQmDYYxU%>6AdY#y-*UFDXJdi^0 zR4VQ0UD&m{u;*VjJqj6k$OUO|y@&lw%g$||d!LUw$^P&h&b~hA-a}GB)=Jma;@ktB zx2zg|N!!=1b(MT$CE%Zagu0z+cl?xk`TUyyAgx2kzAu{J{mC*K=EMJXM$AEAQvF0- zddVvX+7<3JtAV*uZ@e6X+21_7UyyW_*$Jnc1ge@1JrB z%%c{0OaTg+dd~+sByJ(lXhcc`kE7Ancm=&bW2x0A?Jnn>D3+>=To zF(8U5NYd04iRA#o5-w5Ay2&6p5;#5{%-ZBT0yRl0^hvH>6NmPu^sJ@kuAQD~O7b>|3B2PxlV@ox0MG>FaWD z;6#pk@$E9Qtk*N$){}&M%L>=7M*2L?1vvVa+738=dzlb7O{|TRdBWmNmj}zaSkdpH zsZvzn*Og^UcxNbkO^5<$0>Cg}^lI7sd@I$h@FJ@5eoQ$HJ&|-bg^RdgdeOVpXw}Zh z^*Ayx{-Wu)Kug)h)_IvYx|5i(osO|bV9!0Cz!?lSK57kwe2{cn!hSJ^jqLJThrAa) zj6c_sfC__wQP#wAPDoe+Jdu5)FQAy%HpkS(5S2OJfO%ZNf^xv($$%y6?{o1Ia(Jlr zKOP8{gLeH}G>!^Dtl6P-z}^&msQ6|yH!0%|7Xl3v);nhoEHwJWK38MjE9K&I7dHxDL<9{jpt2a{EeN*OZPEgOGGrxZ=-{xc%&6*gI0cij`7WXcQZBNwv8sIS}4 zP++cO@BeB@{R~337~>%co=aa1?~RvAJ85anXsffd`{r;&Cts=1e>uM93$$^eRL|CS2>^m?MzK&@tHC#jUNvcy^G`Qu z_33%rze_5S6SxmxS9F;(#K7$bZHYrtI``MzxDT)5=HDLwVv`hVNqJCGz#}FXX<02w z3Yt8}(e@r`#FkBde-&0eAeLD@Xw>%h)gU;d#^jrnWcU@Hi0$h2fq2|R#H7oNmn?1L zfwb4~e%Q#)Xa2O!62n9rginCu=Ulu@;PG8jQoj!HA8Q&9h?QI3e+Si_#LllAw7%on z#pikw6qESHXzE#(pho!j^&jX#nz&v?N8jq)V1cajqoA+kU0(sSwF?vX0`??FZ)-;_ z$G-bqsH>Pg+9`MQlH}VO!OpqBgr>7o4R-Z*$!^vTb@xhNIzIU$+azt~-jT0hUI=54 zHhZ_Y@3c9t`#8LEktbR0`|R`ZXL=L;e3wk^ud^D&)vMir6#RLpv@$5r1j3G4 zD}Q-?3w?)Y`ZHO?R66HXd9@Gm!ssPmqS%7&M486rr+$;o!AI&_^X*Z4TboB3p$i=q zF-Jw|nH#Uxl%Jb!>Fzcuzpb}_V0Bz>6GI(-u^|j@y_#oHuBdqB-`}~`PiH%lGymy% zzKnl&1$qKKdHK74|InZ=s8pW<+=|5OVOGN&jMf;WEUULyI^#)I)?G+TsGUf$AJVQ7v%sfM(9l59#Qy`Wr{V~sol=)dN`E0J<&%`3Pa}!e+ z-2W4t%Z&xTkHXxv`znPg5c?hFTHweoZOz`_S~sqB`fRtTMMX8(bHUwW>aMPVy995E z>Mhqo16-$}tXH&_q8UC_=aVCR5q1##{DTA@g_Fv>xDqZyL+|Kz({ozGZw-kr|A5 z%PbatA_l1h{NbFZvN|eOX1nMgj{Ls&9#x zEPutdjxFP77~7zVltSRSj>SbU@cn~tIttK_FL-+8)*}6SPj$%%CZ1`6`6$c)?AoLe z`=cr}=CN*m7Nx&rm$Pqsd~oUIRIrfp5ZmIefQeYEq@UfYx?SkdlcLP=Mo-Q&kIh#r z(p27-k?(3**!IRV!JzwF^_*@=k>90-q?YX5pBCzhLv_tZR)^6qXpn4~XVQLZp4;*Zy|5>819@{fo+fB;Hdv z>HQvD5jtDXH(u&&cOL!g*zzvpmiiP_>G9%qCb@O~g%~#?)NXewUAKbB^WdviU4fFu zLCpngyLu&{`@Qjs{MR5_`NsOoE{=E4 zo4LWdWYk703l4z)%=aJ~1x8DEv02&hX48|HWE#3)Ry%wjIWAh|uoBD*J>|~E7{O5m zo&U*57=5t+2szx#!PJ;Jr^y*4bYId$_P3-`LwhVTit^6xVs*b!Sh;UD^h$Iaa6w5F z(GuyCJ{h}*8BPUN+TT{Q!$TitUiiDrAMk@nW-$SZl6296le2xyvZkbvShVnG z3AB)h7TuE`5S27ciY>4gP5P1-UM@$(B8?E|32cgh&0I{o*lVXVym+E=@B!RRLS)Ez z^9=@q;$aS#H!M&tz{6!HF>%TW5DOCE+c1@;6u#3d_%d8C{+>jEw3rADLV&vml6%`` zCTc~;xQnOKKd?G8!xVA3p|x|hN+K~F(mp<^AF-y&P5U{XH8^=={Yyv*A@qrA z3QHF7QK-{e(RZS4WZN@@P0^;X%-|Qx#n#7B+5@ocv3Cl?l(+c;)zFOn+# z70G@$?4BSn@e+o+(rhS^WuI@9oWFv)$yb=Aqu+j~O{4%lM(Nh}#tc3=Wksp<1``Pt z%NV~AzcU`rSyN~1R|FC{pn+g`lk1>*IXXme;jUj+9+mg|WXT$!&N#2Vh-Ox({niBZ zS)E#5=IetrrGK?r{FI_RFW%^Q;KW6&eGm~U#^ZOt*fZz)%qu-_!;b*+s>S+>x#}CA zymMujeNL}7r7a4|y*y+{Cxq$VKd+ldMn04e_f-wNFyJ+!_nsFJ;*Kfkaum~HWuG2? zTvfMIcWb(Ev{^=DLU^EcL(BO2?%3eu)15O~gZK#OE-{8RvRclA9J696I?<^ZSuTgw>QfG0LxEfp&Y%0_l~hwboN)R69;ZD# z!Ebq$t$|pXP$#%2k{q{f6`%Q$PrWMUETZXBN5;9KK{QuItF$UNH2V_vmq34XGIQSw ziy}^pjku2IBGD3S>u^jmZ4?$2twxPU2 zJhNEz4Z7H^5H>gb9v>H3)P7w5>fhop!M_T;_*FvLxFP_IynXKW+jV?&~R^~6K;3a$9L%6Al>A&7^WSNeD0ZCn>GVk|)QQv2mwp)s@SrU}N($ptRuEGiy? zP122jH4^a7qOjLVC3M^;Yx4Dbok2;`&AwUc=MUSj?~?};OqboizwBvmDO5>PvAv`d z;n_`}xz>3_KI1&jpgqLNwAc=*awlzOw&NQ#e4Y4s0{+_Y130n^Hk!YV`&6)9Q{U@q z_dV+|Q8X3$xWuhG=u*?Q3nwXOZ13I3{3j^Jj4gSnFN60NIV@2PtvP>!vL1b^r-+>B ziKf5sbx0v&R!|UCz9i{Cq$2}&;#wOtJOaJ>XNKrEI? zXtOt}^V9X{9u+vqTpY|!$8qdq*jq5xPZ`!RO7wj|fcKufR#`HciAZ}kAr zgjlbBh3qSjOrIQQ4GIuX4$cbx_wIVI&;(x87)zbphv?Ch)p#!u)pJhG&IwBPH9=G5 zU$u4_Ak~Dr!f0Xf&P0)bq?lkM533RsbU1AJ9s9{|BKr%e4hXpzdEsoxQ&Ct(lHICJ zn#*LoJW^t(^xX40ONfQ&3IL%>coPR!Jyk--5_6Y8>{zOus5GzcN_;ev)nd=Jc>mm% zE;!i0f(00tA!YrVWRK{b6$=zXvh}#EyHPIP)E-D&1f->Wjk6$d&>A*k z@^fdTN$!G?SBD?nz2hTaB2*xnf8nA-5guG~x~?@)pX zLL~T#p=eMi=;JC5J!RnMC?gc@pDUbIbRx^lzw?QF9mga@xm%Lc7!yq>RU8XMHr|@) z4aeZ4S^}t3b>Wex;wmnoInbB^P>nW~cAe7l$CY23yOSeYoRdVjnxuI5Tg70a(z@NV zC-of*M5YY$&O%2+SJ}GVw7Lro&(gX(XxfLewb=PJ-}JjDxMCC;DT2=L?+#D;E@Q%Na!*q43@^B;2qyQX z)ASDeJ6bp!{~(>F>4t_oan+cYC4AqVa)jWjEz^SP78!LDZv>v23_p@zxRyN9See~o zh}F%p{dxc6zhHwb>1Qc#IsCvP*DlP1jBLYzP}=km^@~DVhIA6+oVbH6%>(AP`96-! z&w4FM(x_AsamMz$2){pNoVh| zc*_91m4W#l^27qf?$h=8pXIwzBKP;n@B-3*&wb%-Q|yWgCP<*oKBaV_86$@2wqoXV zI~M@uhbGf?Ys=POp6ZNES*nZUCm!T6aocmXi?_`gMTLHH@wD$&k>4^`6dmW8UWsHd zY?xJJVL$Bp78+SJ?DnkcN{k{qu1*HcvUJR#FI z_{XV#ih0l*p!i;|kem#zADhq_Ok=Abiny9CV*`>{VWfw>FYj8-w(WkIEktJeBUF7* zSv0>rIJ#xkYk!h**|DvQ*{9o~r4U$_Dugu9hmJV>&E$(>mVi*N{V6#8XQ_efRwzpj z7bdWQxpFH&Y)o-?_Rev#38(60Wd`gCyWxc670AtA>QrKd42pnHWRd^!=OK2sFe!b@ zG@IALxiK{Jwt1@Z}T*2NWxusDan0bdFjqEY==n*!abo;wA2K6hEysUC-= zbN`Hz|AG2<|9d!WWvF4>rvW!{?hSXYp-AsD-FGqctYKg+lI0AT`H}c@=sgGbhYpws zWepVv6NX$tSm{d<%Pp|351xZ&AS5@^)q>}%w7V=Y!12tiq}KgD`6u98(dOBn9j;&= z-q)wux(vBbp50%TUfo(dUx__zHG@s2Dp_1vqb4c&gd;~x>VsV%?hNK63JVdybZXIC z8;SxbMm%`^f&?Qb9Or(If`23^_MyCr63Jx-XOMDY*2nB0c`jgIFCY>Un&n_a1psg$ zT?5bxUY~ct0ZC3UI!VkI+D~L*w8oX!0C9MIA^-qk$RzmVDx7ME_z` z;3o;|8NYzzUcrm>@8x;AmPBF>XP{U;b$uD-*&#BMQWwp|GlW`TR);j8wogA=wXOb* z%+DNSOxUnkuzwAbV53Y0JRdH@!fDb&k!7&G~)W2-(5 zoIM|7`Vj6@Fo=kJ?t4iBS+hVOOMXaxp+==$e4pShNY9%nPf(VY!gH^bBHWQoEk6y- z`|2I{l+!hwSQB_P6t9{aUYR70-BUF;P;z7d_>?2`Z5qWsL^?p3S_pkwk*a`Ss*ZpN zeV-_>dxSo`&zuK$%NPLN$8jP7e@M?Ak|?D8M3>6n3M*fkrH$A%qX)NJ!8vAc^ZAy01EIigF7g9N@N zL{ESJEbo>FR1mDzk0sZHyi_G|+fcc!4GZ5HfaQ6vORYDzC$Q`Tu2icwwU0Ds63xw% zudV@&51AxZmaDCk%ObMQi${Z!_T}m+q@@__l^+knflNysV1Ov>qktny+rixpj_Z>% zVQA4}^^+JXfA@ba4Qs^qN0$*GQ|*zsSbZo{RBAi+@Lkv)vNp4In}+kR#*@=8*U4wd z-&Df+HU}GX;}R-7B-(xm@m6SB+JNBWY>nTi?&3Dz3+N2|Ao*EJOcJJUqb^42x*&5M zKO*pu*3%M+@J^9I=GNlM@@wCeOLD_ZEyv6{?4L*X94QMhau5_oMwH&#(T5rEiP6w2 zUc4Oq$5vp)IajmE)VvNrqLQ-^&;pN8vW z9C1y1YP-5Ot_dVA2OzobxX6@JwZBdnn{`?a8j#&MEeitmWc8R3Cm8kFu(&o9k=~O(#+$4Lmer{vBfI)nu-6U$Pd(mZNm~-i-(NL15Q>)N;QfhulDSH|Tmw=6Tuh`+CWv38z?QNQ_~D(QasBCMs(wCsTQsOg}YB z>3OLyk!ttW&nW)8NyBAp$~NEOjmZj-oG7|M;}#Zju#&Edmv7WQMFq2;8r_Zzzb|oF z$j^*=Z$96Ah9IQCh_JB_R!4?6KTMU4Bs6M=!Ip_ zm6qvfkGRwEo3}BNN#M6^2^{uWS%uEOh%!u!3&|oyr%MOJH|HZ#Qv+pOcY7!vlP{hB zJZb&Lmm&)z`Rdc#4ZFNP&x+rcMz|Z)L6|f#Z@^JRM^=$*D83-QjvRP}&CA)>Q6$6C zURfX~UpD#3xxj+{iAT~K@{j*hO6HN>;9okTwtUJ-PWC345Y2K6MUwt}TNsR`s=Xs2 zFTaQnlsAL3C}L~#?g=XG)q+{%&^*sy|Lx~xx}f+?Ah&E&iU%pYF`n(b^5%04=ScPss6?`ma=(M~=4$1n-YO(vLk^J!3yn6xl}Q7)|1C6DmD#Cc~T_ zMckN2jc12+z}R*eo91ELcCN98r-QhC+qd7qxtZD7K7JDCiBouWdp3eT=BTT2J(O>6 zEQa8?KXIU@j-PbA7$@$%qsJefSuu_}xD(?2j(0=b@(0#FM1b6n^k)g)60s%u6Ms1` ztp0c^@in`xHJ*T#y!iTNB=YF{laHca$00|Osc>D?V`+QLpyy9vuo5LrnVWW4eTADM zo(m=h{1yQ1{2eTSgk!&e^1N^#A)$q$9t>nv8U;l~C3xQEG7}vK@C%Ba7T-_-3)~c? zwj~zmbt2#yc@Pg5L{N1@#lQt^fFV-2u--|i947Ab%tyuKa!iTz=_M90D!j#D3tnk& zgCfA`D55y;I7I!CHFtqOTB;6`v5;Sh`lO~B*|IYt?3FXrJIdOnM$ACRF=zgL9qn+8{uZS~S!HUtWmKv2Ti zON!wo{nG(xIF-3+HF!J5;5~HRDXMcbj=P`|YIQh&6Yh_dV?~evkdRQNl6f>M5RG%r z@?-}*lH|hfq~W*l;KBk^WndDB!|(@`iGc#LCY~lxJfQ&H&uWJ0ICjp7TwzQX+M0Z0 zawNnP;Ge`mEhbO_774V*1OLH~MwIx>{d1OZC1ux%S`hxwjA4 zE+g@2X}W|^F>z#mE}NQ&YG?}FA>6KVD4a<`+|Bl2eY{FQ1g_eKV!!jkz|ct0h!dra z;1o!7)uRA;6=&7{A*)$KaQ->!yJW>pUv5)u$=y|58M8b83T{*-R76NRv7mj1VKfdP z&!BTbH8K$>`rKGAB@8%28$|D}r`bsZU&n)OP_Bikr#2#>hJiWrU-O?VG&>X z^e_(wSDm-9dK3Rs9C^aL>0cIs#O1WsNFoel0BX>Z!pgMIY+P~58^UQeAofy|%{;}J zNtD7sYshyQoAmSUik5_ijpxZIyQwr!M%v|`wv;}WO0sw^nW1ZvhvNF=7Cf!e#4Etm1EU)8rL=f#ep&`skX>?==wprQ*AwB=*hgV^ z-B;(}bAkp6Z%$G|<{{ro<;1M6L2_(d^j%viiAd@YE`rn1+L3#?r2s;6PX6S$Ti%=RnpUo%MCu2fbVSBCM*Z zC3kI9C#!}0-X5QDbFGaT`b{j*xoJ&g#?$$;*PlRO-0aZP9RG}~amyEzXC>=X*6$6E zr745)^3NcmB?7HtryPiDmdUP^y?6GiX8OS3Zil|zvKg7>hq2GKTP}x!_jF{a$ zvCuG-Sx!$J1?5@-ULG)$=F#YOU%35U(tjgKdyzq+=M*{mp3A$Zk_RBGq! zllz<}>N^*HjTishzpEd+%>U}o+`E52pM0bWEiGuJRmKtjafa+gef?4YAg$4p(cG$A zv)U%|>W`ZIAJSb7o)?v`B#mE@CrcMkxHD*guOfnCkPr)b3Uur`>7m!Ea7fTUuUZi* zxr;}340I08q%sETWbjl}iWC@3hvSrfnTS;2nXKW=_BPBKEujA>2q!p;w+)Lg7P%z0 z4G8FuulhTh%et1l0tyxXw=5(XoMR!&N*S9GWEWGft}e;_CyF7^u#te!wJD%zGo0hG z4aYMYN5>e)D>$@=#&Lk->TMB`K0W``09f`J0`Ku0<#_f#2^?2;^wcq5Ee3>kZMY6; zCtOE1T+^ON5prK31|?yx~Pwi&L9~&fXIptJfta5Hngtm6k=u>dzJ?vwJIkbEB|v za*OfUiSg%)2~CKd#e)INZVbNlP(6!J`pWmh{%lKW4nOw#f&Z0v()!f4 z&dflh-$La4qq9C#xT^A=q!c^9LBE{@3ZE(Y!&{d?Sqr0kS6F%$XVLw`P&3?9~#lj>W)n6_T9XTeI&#+;u^$hjB5-Jd0e%X@{oW# zYse!VW%r*aGxnmT2*XZ>Y>zu43QkssWlT<6^+5`F&|73N<{i6t>{#KxFqNGNL5L?r z*}*`gz(8lpK%bXr;6OAkAR6rdx_4g`xGQAFZ2UBgaLAKOuVhV9RH$S&D)&?NjKu$$ zoc{M%gVh12pQoqX1^E-+*IV6Nc`i0YEkRR>Wss%37HvEiE&VM%DPaRXh zjdH~8M)nyMI_OL~q!Q1ES}B$huC*LZ!3xi zz2^PT*jWFO>A-L*C#*DtRUPDgnGqX+yd{d6b9}^%p zH~m%k9Pqi`nM~{?z>i-ThD$ZHJyG$bM}o|hXvPXt+^g-rSn$wuhpC z7ZQBKmq1$>W<`?xq8L~az)q<~ECpS@Nr;^5hP23XyblS21xoDLUjrB6LWXovOdQc1ZKciy24@iPLc4CpM(1LeoxzBYyu%nEh`NDn)lSK zT}%Qj%d%>2p{Wc2W(H7`1Xv{FL1y1UyA%ls0nkQ~U^ilVc{=w{tU&?@LMUz$s3`qz zBp3y;I2EBiktNG#FmzTk|3_dZ$rWH}NeXZ)6%dk;D3btmuu^`B2aI+p%>(4JD7Of7 zR@|loOygV_hmpF}4dV^`vVD(qult&FAJQ!L*xOmt{AcI@_sxn7E9TJ%RN2mw5n~f>NX5ANcrIsi zg`{mH055a_pcKGgDXzdnK2ZlI2o7WhqMa-0oyq{P&nYX?(fI&faRwUR7LCLKrdpGw z)F(NU00fku9p!YKS8%IWU31O=iB zP_`GSI%K-!);jywB>p%*Qt^+}@UN4u(Gn*J{cJudk=CF6>DMHE4hbbG)L?j^BrE}E zJ^Nh%LwPK(sSpAL)DW0{)~mh#UDIU%Snh@)0giuwf~5B3n^toBB@omAnopCJ}vTBWshZ!AKf-V1bV;zi&`@aEUxI$&IZj1)-SM#l#&l2CS{j4)) zoFb4G+MXCXTCph!y(zDl2;>G-=#xQ3v}GXz3g~hqL`w&XGmsbXfRG!hGQS1DxHujP z5DF;kZpXz`^}YXb#VYKiYS`@}r+FchPF3@`^T0UF&fKs3=!W;Ep#*^_eckqEVSOdYiL8 z&$0VFY3#frtFQ#bPxpR0=Y#}kP~UuT`!-LlyW_k)`}9vOtn2ZP8`<4Bldu#!c_w3O zu^>Pv=y*8;e^Jlv9#cnYlkxSxR_BG^>3BT;Ph={_MiR&9kz01Hg_lSzJHNl3v+@W^ zwA>H%sHX8^iyZpA-4*UFi1#Bfa^lSEM{)eqhDveq$$F2{I%nT(&mNHeg)lETXI0+* zW%o&auc8v_evC1Py(f3-S!8K_Slt$sLS?fKA7wC=NpF8*Z7v|b>Oa}YFLX))~CXb$@%G91A0$n z13Z>f1$MzI9N0w(sZZe0go{_Y=}q$x$?}K!I2Z5cXor8jD_tMS&M&LBx?N|u^-Z%z zUfr&Yd?_A2h6D%LO^>|giCY#k7lGiv-AfRDzqt8PbnEklt+k4+jm52P(e1qp+e5(T zB>?RPN7K)SUk**qnSSdLL%Ll|$ak`srjn-bR^*677lzNEHAmisEet0HMcBpe>$k zep+rUNRp=CZ+ud1T08l)_1P))J?-y zb|*uHv|}MkRlVuFa#p%)DfvB7*p$u@`!D5vc^LCsx-vi9(bD3!?Xhesbr$)@f$#sv z0=z2Ky^*tD*IYhUNz4?~%TT>LQEO4!-NDy;ezN*{8gp>HRr#~THSqI*Q;Eg-j?ai2 z)Jxt=4_z=?N{-{$a}=P4ZnyK3CnKlWv9R*X3GD;N~o ztdyJMpFQ;J+#X&z0d{D$`WI*ff0@_Pe)6q-n+g?jMPi}e`L8-he830a79h-*;-%kS z4PNwnE1;qrr1_&_tgXb2Q?ETTvnPB|PC8JrCo1H$&EO8_;&z9TyC`AfhlTIT&UgN& z>Ia>jf7@#>S5)NcUw78{advTe_qAsagZ?25_~jjvzskl3N`X}E=O7;JAJU2aa9ht5 z!5L;4l;t>zPU!CnT}YA4SG^#>ResZ0Gz9-+IDLGmIw4EF$l1PfmnIx(d=aw#)Tsza)GND+UcYYi9?U&)IS^%?g9Y*;1N?lWKd9-oskC7d(#`fgA zC~|689$PnT?|@UnrsmnQSy3tXW}i~IJxh)F$iUW6xHwBFaV)82TVodvG|-bp?~gZ1 zEPU9_-fgQqQK@a_M(n*Yx|7ss=58XxZf>0U6G`#=V@}i6VjQ-bf_O@=M#@CRebzE^ ziw!OelFW0N39cftk=ee`FSTH-AWz|2=mIE$ zcq3IVwC|Dr6#Z+*-odZ!J=o3OT1A7dCxq0xwUi+vsl30YgBg!v;(g@@WhG)DZ^_3R zFj+}!W9w^A;tus}Ck#5cZ2#7N`65@-zP(*`=I`GLJ^Pf$d~Er}gx!5?nn)nf(e9NSv_)jHG z!&E%|n_^#i7cbFKqiKNI49J~k7{(fsc4IU-bIaN2zc$Y5;LZHRQfNQJb7_w zz^h&rYg%(LZr`DAnwE=M+u?SmMABwd$KB@zMh4l@8tb}fX2|z(@7qOv_tF)pJC8U? zFY=AJhtX{K%s~~>7&i|{iR(DC+S=hXo8d#t^j~2|R2{;4%7e0l3(|BI$CcVR)=JHmyZrbnGt7kLrOpb?Tf+e<4``U;!)<>$!WAUp9f#%sKGF^wf9 z>mu?bpjT|qacC9NUGlUFqj-g{1g{|L0-LMet1#`g^N0u)J+g!M&vt&nahFYWc-sh<68)qxOZ@FNW zVkoX>!L=MP$Pxne#CZE*^x1na33Jv-c1hd5@#@J>yB{iuJbU36`D#Ue4%FkfPnPb# z$R-V^gNFZl0x^bp;NDM&GAg#vFw2&uFRivZ>IV9JdZ0t|pQI4_<3WYAq9m6;Trua% zg-%Qs15N1%PtLUn%TrBK^qya9x|yF~#QoYqwlHK&Glb%8NYt9-#ULJM=oM*AzngnB zHD16I;~|=bb4zASl3N7+RC{~_GW)(UzA>Sl<6tmi+QqjbKy15@-$|W3^ODJjij%&Y zhAXB$`swp-LL>yu{cf#0$yXwP2%$xnY+GK=y{=#$!H2BiF#Ty&r&$>pd=+OUgBAa4 zz7oA-%A>4ytrO=k8;%!P*sH3k4`|(r=GbN5Tg(WLbrrqwjXa*1%$7;}W4MjAvk`|F z`Q(MEBC876Wm{I@X=`n`K%rRLUuqks%H_Suh2lhHEfJ1ZP1m>p{fr!m?pJMon0-1} z{8_lCEkt|kVQ#|2)BAthlsxupT3_mTIdvL<`U8`!MZDC$SM3OB4fdo;5LQ3wX7wzA zqVE5>ocv*PwrpI-f-Wz{_f#_U2M^0*mu(8dlpgnPuQatj;p^U zbQY>)YK;}{L}9v~$2|66mu>HeSUPwAF4*2rI%4$@VKtu4tJdmV^EJbLqP$tV;jU{| z%gMD<`d6bQsdGDfJ9gv7$G=<8UY)>pC>*g%;FH4_S6VU+Hdcj+4D~xygEtWmEKXzm zg+2P>21jvCWY$uOk!-PBS-k(!t5-ZOy_LZs>DG0<_g6kxq!L@+{Z<$If43NrEBrNf zO?UhGTQcW9og;3YAT{vj%mx0mn>YTufzBJ=F1P?}RusQgNH@#EMto3%$rr?2t37|X zU!d>t4h_@@oeZQ-e zauhVlB%@K&vL{4m(Lz!`O5e-4v2lJ@-#q$$pSn}snAIzY$(Pt{XOSRH{ODfJpQ)yk z_ygP_m^~;V^6?JWjwAs(q;i64L_GAly;iw1Z|7u&Jj6uYd&9~4wnZ7OpXoKAhcFyZ z3`K*(GQfdFd_qJaizPc0TP6zNml;Rk9v)VYIebKx;+=+m*QGr+prtJy$P?pDGo}%XYxR`nPuX~v z!BHv7yNkZ$&%L(8*+XNCp`7y_BkS7GK(+M|r*-fLDE0#&>;Wx9CrSv>mhqn$zf_yl z5o@Npo!3yZB1F}4yStppw5S ztNux%-PkM>n)(8a$r=mgQnL+fpDnDl4Re*rxZuiF>*?WIT6nxe6D(!b8spkMZ4wXV zdW(hbj&VK10R&wI$^Bi0+cZ$jSm7Cs;sXXCdyFeqhHIN%cqO`M@t5H7SiXLDasR21 z@|=44ch=Xgl3fzCc#i8}r!>n|5SyNRY+^BITVOVX{JGD`e*({fj|aH!Wn;&Tbt`i` z6Vk&XvrK-I*LrV3Z8bQ;^jxVvDq`$Aq>KYGE-2O=OaqWX0On&9I~%+lgoW0Mv9rL- zlX?LkX>3puRO1H}gaQm!@|fZPEO6jk8tXVehK{u3xaY$<2M0gxcT>8mNg2F_0coc(h~ zhD-vixpLtdqWh#%IBVcN6fyWiLzOVj@MxYlF`Up78&#f`(f z^J{382G~gzv5jR@y`UcNx%hEx@-tl7BpCpuL2OK1j9$nO;O3{50Cw0BC`h_?o%)Ze zbK_-y?XIf5uQrA2^cf@giZkb3SRVQcEEbYJ<{{Fj;>~XrJiP^HE`h5=kQ2G`m{KIT zh6D0y!wSb*iq(G9i6Cyt>@=$70p5o49E??Jd9hzwuk*di69WJM#u~F*Doz7SvDfqP zsLC|ch8P(j+yC%gE(8vPMKx1T;7&&L%KO)O`fbeA$Me;IOfh0r`~ap|Hy%7L+uep| z$^;BTJpePjMA=npwaD>?Y22kGB+!uUa>Knd1ZDc;vk;Zr{Cin?EwoSF|whgP@H&*`K}TBaMDcW9__JS_Qw$ zCIrli;rMwepNoU(q8OYvQ{LBx<==HK$0DXlSFQpQRCL_@-?u8X4$tTm_IXb={sYgj zFWh6s>@}Yaa-69OP*8kQ$f|Z=F6v-dD_+yqTjQk};0eafhjYEV#?2Y?JXtBvap*Rb zLb5GmLJcvK`Dtuov?sL)Fh5P}lx$|9b6cJltV4B%{DJKWR`_ugV?%*saG>uEAl0U# zZmAv&&jYnc8|}aZ0XnBObH^wk;8A^PXaziwJ)8ElR_tM`0Zvk&`QmgYpFcl?#AFqQp^lL5!U)P0rX3<#^W8igRXRD`GnM$}$Nx*LA)y2~~}D=6YcM zCbl$cF5%5Z^Qd5X_G}bz2^sm&jXPxLUT^Pzm02j~LII^f#1CotQ3xSse#kE`_eU7l zW8kpQNv|}}lA!iI_OgR`j|l>l$Bpy^pMt|*e~37Ji(pcH-Vfjt9f-hF$x;5Z=R8ZJ zwsHJ|9#LoU;NOiIw>P+}YLuCLhIe$)P3MLBuDixo4`15lG%H~$#ioTSrPSYhl!D>n z2aNnGyoZJ9F)KCIIBh@($Q3oj2Kx=utiIfBK zNhakyY8$?mK1hY%i+@OihQ+7Xh`I%vgl6ApyeZp5^xJs?50p9&?|K&>{2nJA-(now zdy+UOv`b7;IIEuz>OUZ$m2i;@)o&rP&w_+95drL#30qf3z#d~{W7u2XeUem=aKOSj z+~nkqh1c~83k`9g3VM*_CxdgrijUbojL$e|Uj}nJ*DL_99)wEU*gUghV{r}3*$!Cp0NgmhL^$ilo@KO=Eb#aKlF9y3Hx#jmigyJpDg295z=8*W1xb&W z{3poJMbUQc%*_{GepBeNF^CM{{Vk=Cn|bO%aU6Kck`Q>=n;*$f`NSz^B3A+yJyGJ` zp)pG_ScXV2kFC{}PyHgp4?edQ%n0vvsRZ=;1&x7T zNd}c~eZ3QMS)Br^;`~+{@=ZDvWWa#zkc1@oRzG45%`283-(S;udG)Qasyo=}nib!u z3jqgZUe_UKRCkTN&tEHBO)7o2VqI)UdUg@z9k$=fQGwf>4%vK{vpLti`C)W(VQaHu z2YBP>=4Y$_qv$T9nta$WfImw{Z{$cf8mSSA28JlDeRgN*8Hd{QI(XhVIs5P&ax-;DJiwvSmBmp2A_seO>~iryHB$d*T ztZl2lJDVBmZOJ}%IX5L$HB>AiVk{ffc74V%G`KlN%OYr3aYM^J!(hC3vt4NO$6`jX z0N8BwjqRn0S(NrGUQ4;=omZT$rGu;UBm<&S=%>tx5$VQ{N;|iB2?U0knj_cV!q&;e zh$hQ>AKoo~Z}n_6%YgLi{21_^Pg;gIIG*Kj-fxT9Ck&#n0^b!vEjG!&T8zvuua0>b zu0+(9d&&+*X-M^-iVylJt;KpNIjqDFg2VnQBW``$4(^=0l>uQJ> zFSI%Oqkd)V7KZKBl%C}A-xy|xVXWC<&S!)?CEIK8+*U5@VWsiDf8=;Wxmko z&p5O3MLpi_tN%g8ovOODa&GXM<~x0JNc>lL>gWGC){1kr9URu;LI#B!A{pCVsby~VgjAFIbrj=ju>91Lje?_d%zPum_mp^C?_ifcL&q?_C zJba@#Prxaj8>O3C+<*I4@i86UX)ICvduQbayhn)x%&*FGywcaWz7v#6?#iXEXm-6y zoa^_>?PUOg^1;(&MHAE+xdszKTN6!FVcQ>Pxa=1)yYhtn%NxuhR8J;kHF{hbCUgyR zdK(-`klWS^pk<8G-DsofENlRsG*rDm_uMo?j}H(#h^6UzJi)C8}oig$MN$ zm|7D;mYr1<-r6=@)sEi_y&S6iOQUJXxM%a!A@ib(ljtS5F~6yfy{V{Qll_?^e!En` zpc5r9bB)(RjtYpaUWw!-?k+vOM>=tcWi!)bA0EwddLN5r3#FrNCY(!7C6nQGnQ}&S zTOpszR~wHv&1A5Hfal~#iRD_?Oxl~tEXJzT&l>M)}|fRlIAv@?*Bx8Hct zp^N&rhrT&7AC)B-Ld;v0R5uGt)_kdaQZU{?m*eG7Z3^!ykR1PS28p6G zI=+6`7GAC0j(vKS|0Ua6J81uPVnalF?Y6OCk5a;1)Y(_txTmXms*R-f=K#B5u>q3R>bsN@d%K3r_kko zV_Bg|{V3`HS)(S^K!=2Uy28g+St-1Nu_`_~U0S$R0`cynZl{;AfrUGZTwdma)RN#t z#MK4CNB6;H@Vxi9G7>mtqPd{o*ZcQDgBO&L6fBow5baq@ zVJ6Xp?eDwLL8<=eZdrmkfr}o}@?V^lkX^vIz7cTdk!OHhiZ7i~3hO^Ho92AkirbhT zVwA!d;&m^eu|)Nh%pC3ug!$_D$kHEVJgFVK2CP2oacgu=C+{didOpgR`a|P z|0XKzYmaMdIs$VXH4Sb#1Do|hryCFI_EdeOO2*6dlq6Q}Gu-C(KeoPt1%<@vyD#w( zAE>NVJ@CDETO?BwCaz~EaU@k!%)UpF6BmF+iD(Ny#c~&L}^;{Wxb3{zlOyuOuj%G-CT3&I4prXeqTF`GufBJ;T!8TjC*l z&q;7~X2Gcm7v#SHD31;ulj< z&Z&|3n@x`l1Fv5AG(FMVLTpyP2lHVZ3^e|@G`+c#T2H_t{BU4YmS>(Mqoh6$Y3gvH z)hWq@Bh%uGy+RX@fPMM5Ph8~huWH9V$aRmTMeUr^xE;D5;eFaZnCkMNks$sW7eY=3 z{4Kcfm;2&^! z-JJn_aV&6X0i41i9ic&)V8F_G6uy804QB%Ab}(F0fTV&b{_xwB+UUzS-UOb&vi zvi(w;^1O1a6)C?Cr%}Dljf<@m&cT=4y%e=d3RlvLi%O6~vN=dFYs4|JmQH0wC(S$& zCRUq+z~M<|O1LT{On2v$gT%?7u}Lf6IdVT~R$`8EvPo2i_OdnrMu3h`j8uM{nwAB4 zmGYn=Cm|tim7b&vA*TXJWaH#xK1?ISCms5?6A;cumGozB8A)1JV(NWeW(cO%QN&GG zS=qKK&#lr=KFl!7ctKApB8TAgtHs?VC`922LYDTL0(oM|lZ4)VO-IMSs7iL*Y+!hE?MK4Lx(qoF z;GBI>6K;?_l0?oTWyW&(Ac3{y#9#(V3j-KYd3DXnEajLugHh7osx~v??YII#s+=4} zkqn0YX0UO&Q@7)gC)oI_wyx6YDhBC`1RUI_qc=qM%GZWxB>dg(K;YnDwn2(G=Hm%C zoW*`Y4QG*5y=yv-2w+_ZA#5~YAj znbf3AMkSlkTJy|_rWEhjadxUxJ6g976F-}SE4!` zvdZD{hfSb&$qq3LZ~)vKFj@ga7y!Hv1S5k%o&aa$Q)$vM@CltNIut-sp>`U)ojH*O*b$WW_7!=G6J-|b#Gm-QkT05GX( z;AGyX(|mB2#L$83;z$TjAatgW>vmRfb>c8Mc}K<~+rYxnvcJYR^}bN+tqcyCaE8s_ z9=Z%g@`Sm}WdGkV$SpzWLm`+W!=N+2_3bmnlM*UUR#HTVc8ks9Ma#D;PM`EQnJW$?pZh^6D4)ME=7xc58QW4c6?i7t{P z8U0>9ybwwwX|YY-O);l7^ofgnX)5Dc2IFCtE?##SOtrjG~y3TI?z^(l_e~I-D|9052TFZ#zQ~9wV-M$yq<{pSx1?y z6<;3jN5@;o#90gy>`A(o%!cH#2GckxI5+rn%{kdVGR7S2K(+a&8RwPDxtPOoPRR1d zb>45tdwC8?;W+TLlQ0Y?_IS^ z%_Guddu*ZsCXI!fAtDM_KRQPzh_y{6Rec<|NEb3k{lG8q+fFMbH%r{7Wc$gch}Pz2 zi`KXOh3x`gawcQTKe+xR<1NW0T^@h*`YE?A?A}(zVaepFSW)$-d~^#qJyb*+JmyIakQvbzT#dfx zT*B?+X*Tg_p-}T;1@1nVmuW=6ELVs0q+jCimnI4CdK%Q+nqb@$QBDpulX44liA8hx z0TAzGb4Rr{izr^)j2!j){(8=903^c>tJ$>{r4&t;xV1#I3LT$YE!10zYn7aF+mD!E zD)l}dC&xpUa2}Y~TAjk`36z7l5Xwx7Hs=*<&;6Py?Jw>FuThu(*{G8&Ye@0mwiZsd zuGvaPBINs8ej@{srB=Qz;E|-zzn?b@#y5#s27*qIX zLR!tJ*zZh|ga_MTEI+A;1mF5<6dlUW6c0xgJUX*rMsp^jL#gfCt zKbn*;IsBED-tB=)ifKz;A@ly|8s)rMzkOxY%`YW4M#idC-#2)tM#5iBU3wB3?^mmu zpHXcYL*!ubo?rI2SPp$ES?RlwjZ0pcXHt^HO0yLS zNrJ=iiHLv~m7B(#Gys%=1!$z691dM`fV~oOjZM1Q4uDDEC>$^q7R!x<0p`{dIV31L zo*nMgG8sbB_#jLpB}6CE6X0cYX$&I36PF~o2{z1dD>U$4O|kihf^`&mZzG3iKDaXd zzW5Jc4zIxMoS=QJzFbW~<``zAU57eIDwW{C`O%%>7{iJn1oZs!*uq*O?jz?SCBv1Q z4RV%P#790|j?Se@cv5_&yuxqa$-E$0;Q*Nm8|ChEO0R5>IS2x3XWB;}M^T2rIckSMis{i1D7ZFFj zxq}f8^28|Ld)I$_T>trHow&Zgz5(CZ6xrBP*x2j<3FV63b5FKyff3?Ub0dR+`YS^z zKqUz@c+(j2EFFmiwqlbchK$~)kp^r)7fZMvq@5-pz~=hIjQC^~sa1Y?4+V zgExH}ot_^}m$*v|fB7UJ|9WQbg%f=j|8jomr$4=^O3=pMMboU(3>^WW7bDChk=_NQ2_mBt_5*x4Oyytd0A_4nLDD>=s|7r;#_v4|5V#sMNJ!D~n{ubw4mx2gYR2WgNX zv`QXPAbCyhyaNL!5kK*jH!`1h2HCu+8QoZ3Rh2SuDSMQ-f?0pwnBORJ<80lG|D?~TngOm;k&116*)PbG= z%cdv4M%!!RDoVMWx$D_frD+pHdFA5B&>bebCoFI&PjkuG9e6<#H=QD@z4^|Z=1qAe zw(b7@_f4+1a3|s8?_cngjhw%UXYG|exL=#=NE{1kC5|9&^8REEpGb&aNFq4YJyLLg zA;R#=qEDxW8W08pD7y%d0m-)|*Lx7#kXs3=YER zZI17C=ist8rJ`T8{(bZuP!ssu?9XLC$}VB!&N}#=eeW;5@5qytqs*_-*VKMpJs#Q0 z|BSe)^EczUYBMlPyAS_St!$DoOEqZy4`>G z`9P>;`rUJ;7*fWsKujkE&gn1^9nQ3{oLog|{+Jyf;zNl&(wk8c^Ay6{{p zRVE0UCnrC6pC@+K^IfpQ+x+9fOYELH#gSs-sVl{vD{tk;O6SR5;q4Cy%Or!5 zx#zXFLu(&1Dg3fFH3u>|P02@$(^Q`;*#PiE>*1Ox?XSUmVND+mdH&aTWPx7EwIegfEHF+xas5&^TTF&dFVo>(?L4 zt~*@B9oMaj<=41tz@?C)P&}Be7;>21v(>ifJygFGzR{7GH1a5=>eesESMSUP_5v=@ z$xV+&r^(z}G*`a|$IvCzEW_+BBKl=^f`lm1)hP?JYmJt%pyYHevh?!gI!h!<88^~2 z4TltEj6d=;NQCGv<(Ap>BNUejbCcsn+}V+o3PBCf=hz>eXM(N;&EXHtl2=Ob_4g8`sVY?i;jK6hMX&Pry z-WDKod=>qq=7hjFMeY3Ift83_hW3-Xldg`^Pfta6_i{4+=3fuCnn9+RbVES`*Eqx> zv6#G;$O{VYO^m`pn#`s*pqEfU+@$|V5}=*WJ`-j_wQ>yFda4U zNqNNrOV%?Xg6{R59I}P=T@a#i5ANULA0BxMen&%R+o-)>bA-n+zW0dW%$194=KZ~f z+w{W50hMPnje{E9lFx=TCt?RVt>}*fa7tFCH6tBSUw@S9OdmWO)6IOhY(yOC^Uu)q z8@je&=vQAY{HpuMJtO2i_~XL6)|V3dX4|Bjb(8*QKB`Y&yVi1*v6|R;+45$X{g`$u z%{yuC_RehUJP{(zScnz$U@Vg7mwu_#?G_y@-qrt)u|iDN8v3Lbl?~%M?mPDU>%&;- zw$-w1kG5}>1;ag@#!&5=6n$CZ?WP}kQQl9!V_n0IN1p9?*s{2dG<$Z>zzDG-vDPknTGk^F#FO)TyE{rM|NV$%5cWU`Y&!JAet1!I4=zBo??2dK~GknlVju(TemZp=t}`;C_xu#=U%r<3+XxpM z4CvJ^)l9wWE-UeA)$AHE2g`pY2JRwb+NmzB7TY?#u)fJ!eZn6ff?Yo9r7(cO2GsQO z-dLpFnX~8j>BAuiec2d~P;MXmvigO#Gkgn`Z0l}0-r#S!K&JVCqri&T_PBY6+4BrL zkF(T@$oM1IuCObJRz$ju<&B5OV%-?qmbZEy)$K;U;QahFy{Das!URts_FgNv3R29Z z*;QZR_--M}f)P?3fXJ+r+{qj-E{J^m^p+gK_ltX_^2FmFFEq%F`>_a_CXwq2HEZ75|&hf|%#>w_+hR1i|9gLror z;^U^ga1u8jXA$8y2pX2k`oZ3LM;n`HUqiYRa5)f`=jBJJoRhS_!wsVk!N<9d(kX;JYa`)%Lyr#2OL#&hOa|>7$DDMYr0$OJz3SBP;gJn3rOoY|JHaK4u$)&S zjHQ)Axdk}8(7*8CsM~n;(xYo8@jWWapZYIs&4!l>MiCP*bo}W3K6uLG_liyu-zN)s zj#X@6j1*Rn%EW%$md;z?)rl|3t`Jyz%y7b5^L&tatR=@$+bDH6ejR0~tqJmg-TZ$k+0)&#qq$%j(?^2HSjPCtukFRM_+c z3H%$l*hDPq7OH)X&~b8mV1S@hrrc4zeMG|Leg|tAe8#leA&@h5@UIEd@Sp}>G(d1y z0K|1rCVcCi67Qvb8<{g}`iG(9uc^%_xoLi(Fgz4Z#c+uCfp}D}Qy11NTl)f`@>*U= z7kD5w!0ldb<}swTYY#egwKHDz8a-?B`wOk3#A4+e?j{?D=$w)zujp~N2mZOO7{fq~ zY>7W}q-kZuPLZcW_%@#pa9Kfp`^xFH(9iH&&tK~-UNAiD^wTc*t)WYXgVxEPHBXlf z<=ThK+Q?26koGM{kw2|h#wzm;eoJU?jl?YQe1$*!Eg62NH}9H&r`fHkOwXI;dGpr= zq4t7M_W}w96LXC!S&j#bnv>(w&pHZdi)OQ2KU}?RRz-fIwI*4I@y?LXnRNZc-^VyU zp>Kq<^uZGvT(H;mL_e)6$i8@Jd-JP)z}01>*L{Ttbtp~|B8n3PLU>Z|GCeP{n08O5SeC{9_46vtb>_}WHfPXGa@)w%y62rKi=;qi@u`;3au$ud| zThsuc-1!rAFiufx|>e)KZ zR8i+88a15SsktH*ggm5lJ6+TGQZN5_M|?MJNqIwdWSxh{s`(nc_RHFrcg-JtUe?;L z32APVSRfmYMql+bxvBa33yqT5K5^#K;TVnW9|OfNnJVo1 zY;8e25{ffD&BF;%+KmO3vb)$0vBuF5&pBfBE12jZC>4sL?`$95j-3b;EQso4RBJ%K zyKNkPOhE6~@N53O6#VDY^4y+V3U`(s+~s}hqFGl3RQk@hfHM~_9ctNF0MSi zClz^(?;h^E(Ka96q&lQU%u-h0zZa&{LN0>&xT>5!nwsR)Uoj0d&uh4nsl0+ey~4uF zVm|p(0k!5L>-HkO-$>-Wv$Ori1rd3mm~I%FomgeU{?1xanJw(pqZ|OKN`RvA$oJi9 z^>pY=w;B}>HYaJ0N`OYGM34alg~KDv0UcjT$3>>@`zfn(VEF^}?PS+dWQqJY2XxXR zI?GEee^IQo1nq4NMs{-kowLFL0{h0`888pv4mM#ny*1(GmE_Z6@?IfMTJY91|aq^wF^_vzamGiadY1J;U0r?D&3<_B!;Dz3UY}pVLG4tf8{L zpd5YSEKXuH7Z(?aBnAP+7$fQtI;sz#qYEu7coWPl&K#JF!|&A2nCFl?KR|U5c#-~C zAcuVQVu~is$PjOg*ahT$Jx}{Ce%qU%qajx9UMqng4H3)|CF$#p#pO&qby44- z8%Fcs`*9)pPAo>J)EeqOx8)<3WW*&mFO*4yn~D28AdhOkG)A5(Qe!4OM$9ghpTpR#)@~DjW&|;WC^3=yMb>zO>8-Pic>k52T zK?0QlfKa5I$W<&nb%nm^V$A@7gjn*ue)DQ8gzZjWUPA~3(WOSK0x`&%n2OJS9;ZXd zaGqGq^~=`KM;M;Z?iK((D`{pA2y40I$ai0&Jed$vgxvEba@pNa_L{Xx_4B8S)~Y>z zPWzlK-M5d&h$0ufW?=R9yq=A-)=U7hoBVsPd%F`Kv@aqGtl&oFK`VH}6*fN?>-y3N zZe)x--nH1tE?B&OSdpKy-IZ zu>>loP{|!IStqN&1x!dNG!Xb1 zfEeUz78dkJotMS|kn1AyChY$QLf%Uwg}cQl;(w;5&7J~sB>2#W2Vv!x*b2=GQ=euO z)L12Hxhnm8!eSs+LJkP!%0=NwP#$B$9QTSaPX%)l-+ni)@O#HocLc)OL`_xXnAFsP z)2lh7p@x8x^<_Z^*@}A^O1n(2h@l8VQt9JFluqV*zei`;1olj6juBjSR(0eg?JRq( zHM))_-sn@4>m2EDj{Hmr#wtK+`ZOqVbJmv;RQ5S+;Ta1t9&^$ME~bhJiCzSO^g$qa z_V*wlpdiS57DRsGaoAf$9d|tMTPoD#0klv47+)+r@|{|HEc8IA>(&k+S;JT3KYX)d5ac|F023_EgK>$RyW%xV)s&Q`AA1VKlw0FF&)n-Ybb4Al zS59j9FhHKOatG81ox8`35XeqLir@(?Re8HiL~ohi29WpERN@_cR%mL#g@|55 z1FE_}pnZze(8rB?CRhHuX*Rr6M{wf5-_hOT{P;7^x>{k8bvd(rR&p-K`L(YRUjUj& z10RL6)~ui)At&in2Z|;Y*lw>?I#Es-4>X*tedTM!^;g}x$$5sV)&RaBjW0g>9O-m4 z==Al$fs4#rMr7rKr^K1z9$g$>cIYdv0?3N+Aa=M4B!7)bx;h7%X)w%6F$EH1%ELT@ zh)c7$o~N;#u0O?%!7*or;DnO@&Ld0$p20OVKyJV2U#oZVC%>7Cp~>&J2Tj!;rLDN^ z*|=NBKty-({3#o$k#b+gdvu%7uTsi_bLw23q_{Ijr%a$`#4nw3&pXv3^Tr)|^zDoq zwl;jw108p@%6@Dp*}`Y7rYimhTk_ygA)xD#GCa3~vnHWx7PT2w4I`GZ2)}memTW=)cR-d`Cq4LYAK< z8cc5PK2GE#PISX+&d5;It%nq9#2Yk!R45JYEX?&7%HfmB0il>!5Ks5p_@0?!W3>D6Im!6@V;xhVzB9t_WY2RHJe>kppqfy}qB6`?FTKA*$AKKp4rC&=l z3zn5^Huau(yqU4!Cnu}V@pv)SN#Drt7xEj=>PY-rA@0(1!faz3i*dG@evkgjfkapB zIR8Dfl&>ff5I{1PBi9bQ`Pvo$&gL;703`>^V@od9y7MD6MY!_Ml@xe;=N6Arp;X1q ziuW$=-=-pU8WWpRGIX*Z?XB~EFej8qbjgGFM+b#aLBy#0{?eo->s)YjAiu#2soM#H8&CvDwei_q z2b0>y=QB66Key0++`PTFkIw}QjlOYQSB=!mN(p^;7h%*obtJ_PCbW3thCizZ_ZJzb zLLcoCioX_L)O6icGAQtwmd_?CG)Gg;A1zsm=I%q`>T z+u=gr#t#GP22oUGYS9a?5{SfB7?p!Kk&QAnc|Bu?;RL z<71;Qx5vggnV4v{nK83nJfLRPjH2!~*wih7OzWxI)TcA}nqS6ICP@vuEXz;WGxt^g?kG&mCzrjO5CgA-!^o?U4A} zDQnQROlhbml<@adKFf$VdGV_$BZb>YU>9FZbqv=WZ#pa3@(g;8+xNFbf{T&-3wprPO$;l)CT~FjV_&Wl*Gs^vMRPfH2#J@4Ye{Unf z1|~_NzMY(yJMYi_d++#HN1MjTR;TTa>-Kq(b~}N&D`y z{@rDX*zSZr`-QG$xBot?bcjZRzG&S2UiHsIt>@a`bEXmZl??<%@dWsTuknQ^X3E~E zwB7Y@zw<*B`n7i3rRulKu+Y*MLwk2r@t-@JtoXluQ3CRWtpr{dCDyh!aYvuEV@%vV z`)c<|>`HQ)VKEJcKVS6x z7-Z>|+nQR9mARQH6w2UZ1W#mm5@I<@@}l>Ce9}wc$C~Aw$n-W%7S+7`o{`D&Ig@@w z>aN7)wU2~sMf(b;=UIMdW4r0N8+zH7tqXKRzr25*?Qi=KJmNjCuoB&>ix$m&bd9i3qPQBQx9#e5l$DX-wkQe4R_(0SDW4p`F6RSL2vcTsk9bd~r z`^vk2uTEZ^P|?2k?J)ml;HTDZ;=rIId~@QKU4?Ya`HEvf%c|`ASfBlw7q>>m?~y_B z{Eysvjie_!E_$S2c-fjD%D-Rxi4p#$@dSb@ySZK*{qV&@;H|9-je47Q0+(lGx`ZxS_n&mx#nwXMT*rz%{!Z|MpaJL>gdRD zOf>_sV^hf(6+D%Y@;r_xpi99RW%#e4ZrI6WCx?!oz2+mJ{v7u+r_vIZ8hAW*u3whJ zemoiuKW+GyQ(We$!8b|5c7VAt*-|a`!})i&;YrR>yyu+@gUI08bJj_g6)dmQ=WNC) zjJ8K9Lecasm&?uil?i?gKkqAA7ww*(A(ytvTLsMEK9_gWLY9k&*Y31MxUV?9PLmHv zJFJkF(ZW_sqZviNj{BqtPkIK=qo}&|D;Awr4!1t-dAgXAOQj3)1z*#o8A!h;iRmgg zoiZ#G5bic-`9#EdN3|_{V`*8VhKa9aCUbK*Wy0MeUs*U$1Rp9J*U{sHzedr-0qc#s2_IyUi*A~`g>g`pWBayPHDJH;czC$Nu?@?){93Uu9llH z&a%Dwwx2`yzT9H0w$h~r8FeLXm#wE)y?;i0lSKcx<8M<+kAwv`4U;JXtYXEHzsI8v zdkYngWoH~n>6V|jkh*4oHQD>%5Ap$CwO(pC(ezlLm2)B>=UKIR5+vJAmcAQDeI%ve~&xBBJ+Pg#Y0-Ekm|?# zFWhU#uE2$Cs^#t@C?3R%N`UZdevGQc)NxA!y zGe6{ll!`6hcOrx)0Z~((B~)gviM)DdqV!IH@yiy;j(?8ZQd=R!P7U>b)@trrq? zJ4e7(_^Wac+ACx!$R%F@M&2gCo?#Z)B5_aCH*crTI(e8>OLCn%VEh{J3!i+n9;msC7 zmS57XPj2K^nntHU)*8x3Ej>IftwfIV%gHw?H+MvTu6$NpTS!mH$g|?lli?hIW@s|% zk*o}3;?X%!#>pDmS;~z!?$f(pY}N>ELUSfrieh~Hyi2@p?1b;58(;Mo#YXHegJ7e5 z6^@0*_p7Ah@8@TG{psMWbc5?y+Tkk+-pfe!8!6GB$O|G@O8IPkSzZX?=2cV8eEZa( zoHD7s0JHDE>@)cmq2kT+rg=>GwHe2PQ4--ak#p^B_zXvdMJHPY&P+f?bqgwD5)8=V zP9JWl?JJp*meBKCTd)z5Tg{gF^~od*`InYxVQQ zhHQMN<%jJQvt6c7n;BKua(#X0dc2)$t#>`ubYd!1j zz+kSX-i(QroZVZ$)hXezmG+?h^WJ(ou~*-fFaOsI#+|o@6lQyVQ!H=#x%E3+Dg&j_ zGIhxYXOKK@7eT*N`K$LV+8ngn%nte)zav7f=Y6_V{sqe!rPLq9YgWI?cfz2!$P?t( znQWPo+arH`E788tTevQ_$I`Arztqzw~|PZx@w=E|NhbO=BhvTyxOd-V~so^XKxHGn~a>S zijvk+54Zd#YrON}gjlcjIR{m&RQ|`;L4&1NAiS0i zfjxR^y`)ZS?>m8^>Ej>OA|R=yYM#}73Zu%CAtOp#D&^?1$~S7ZP7F-?HXcH3*^aR} zos!9+aHlu>>UYh?E3#;%5ZwJX>sGb8WGx9-*>}sm8z-d~xq`|*OBpp-Fw~^OtnW!Y z_=#HY)+?wVU0cz;Z&k0%Sm;JXf?)f1&&u-PQlfLtKq`Qv@+{BM^lDr`!hLf^4IQ%i zCVCm$lKDm@Ya#x^iF1)7=Ad|@JzJAfOke6kJ`dCG2u)K_9iIn&PmI}YyBd{oxQNyp zKo{fR&H0``?Pf9LnL*;0Yca@Sx7r_O!Qh~cke&^=anc3I&pzH4fzxGA z^9o=7!E$Ix6q3(YQ$HMjFuY|gQABxOwj$NUB5_=wV~Mn!1!6^!ru=dw0U7*-fmm)G zypJ3p4kfA1tf;LJL0mc{0g?@9hmJxlm@Tb?FJVDpmf##2(A^ES4lGzz5#$U7?ot7z zMKN`}G1?Ptga?#LIA{w}a&(k92%vi%*x+FlI>#iOD+|M!LVC=gGqpfg`TXC*R>auK zWEFRCpIq$p#^C_u#npg61GX(4BgZ{@<@+?MI!cJB-M85to%>u*kI-W+Qc6 `l^# z%%&b4p)Aabt8+c6uNyVFbJa-Nh}q{*I;UfbG@>5~dwOK8?1%&N>s+a{W!;fAg3!7b zyMP)bP-mcpMXY&EF)$p{l(Tep(%X}Rl{|`$#{$8NV7Lu9$QDe;tg?@IjYtCOp6K?s zL_oRZh|QK18%k=SSV(@aGWjm7gk5Kl5MdZZnw`*(48@qwQJY_+Qt)BhcKR*Dkxe1o zVo$hDmDVG_h;<>W%FE|ZUV#{Lu9}Pp4$AU-+B*)DLBl0EYh*xjdO5(tp$u>t0h$mnduE-XqU^W-jIl-SroI`KKG^!oSfN}k5DvF#lc1VM* zy7%$iS$H9XETE%tV^?ddgRCi=4Ki5X7jHzRg{h;BFU(-pZVu;AtfVj+{8d`vY^!&@ zVJ9>p_?v4Nwl@r|h2<9+ zxFX)UTWEWWW%l*dK#=n`{m?zTx}HC!(Lp6HN#)^);XIKb{iR#z5|-m=hDi>@%#*5_ z%0$pG>;(eEx7MJzWEF~s3@?JS0Iq{N4uHfRupw;(zz6W)n>$Atj>hUtUUyqX;j!5{ z?@%~SkZTKUMCSqOP%&db$O@Q7;x@n`5cu_)aKH=m&79w2c0pPIFC>GJCYU49q+ekR zL@}8KFqq?n7_e{dIKq{%cC{*Hf9JGX)v3w1=bi>BT;17mRuEJY(EWqmP#;s%Vm~w( z(9Lw=giZiGrWTI{IgkV#m?*YCXtba^KtD1A=o^#R_kqxkPPEZD_BMw7861S=XYt@6 zr}l7R7%;oAlTHB7XbDU7+WrnE8WwyHpLvHiF!mBT%Ksi_9R*GppqE%)MyFtoO}$rn z>JSds$-$>fN?pg9?eHp>V2%6nufc>=f>25WM;M(W+S8;219L8ChS-5m8|#>K3dv#E zF0F-Xy%QI|xC2SbYeQ1q4nq18O9ugbvE1xE43FoU)fnK3hvW(?5Rai&J;*@Nh>$oL z1k@c7@0?ZAFjHsB%ytO@B!M|EIb8D*El`}mf~4=ZL>fDUuqcH)uNb^29lZb!x@`N& z=%K;=8rdH#e`c#??q! z>j$#-&=NtF$NuG(IGu_De0(s~!vD&$v$mqk>2JagBiaa=27*J=JlyzObbdrE8-dUF zM9g+asD&=@iNLK0!u;D2v1tPC4-^tgW%5jr2mtuoPlt?r1-LIey0{!c@? z`E^8igH^CK#*=wkF@X9pC*+aJtxXDM@eFcS`4D_&A#L~;3#{l6TYY6+S79JFP(0c2 zjmZ>)=y|feS`pJX$Y;i0QQY}3p~!P94B2|bs`KWpO^Fa1sBuTlwR`utrt+UfX)6L! zi!k47hir7bIDBr~3KaDDi||4kVA45&LAlLtO9eG!pW}r;gMbBVB-fsrj%XlzN?YH| zW1zcMvn;XV#kmt3o|E!l*VdK}#$f}b8Ei{DcliNuLrdT1LD=$EMo{P1AOb8~og>V1 zZ%#%q8ZS)>O7TiQDc&h4YG>~yV3qqzZ8}69!0@YMw#3QN_i}+SjDr|PaJtJVj6KC$ z*;%3JOnJt@WK`Nr+C0cy61nsySV*9^HO}9LQz)JF;AYj@@zKIq+=<=_7c|d#;Q`llz|(W8Nzk5?X$a7BWj)a|o>xW)99SfrhsirAOY<*1-w^Su&bWNyv zV8JP#yN6h)Omu|Lh25qTFrjFQ7=}~sm)+@B8PKK^Y*CsA2nfRiOPYcLo^-LVpC}b2 z8W_aD^56H0Awgua;G~PU9$iTN1nc^eS?PH8X_pqWspvVsC&cxMgx?<(uAdkropYaI z#;}bnX>_m*gC`8{dkzIRDLXk!F`4f~BQRT@m^B{2C~0eL*o)jNv>+s+?z6SUvmkc8 z2}QRC?u+AOeT=e0ucw7fBf~7zsyV^~4B1)Qjt#cMqX>JbyelSMfn(#nzL|!#bTP1^PFC{;tOl z^BYR1m5nI0o#Si&Q7&XcPO^#x==G0`Xu|EU{O&6f*HH|9HQogg_@S5_M&Lu68U=GA z`vEZd2H`Hsk$nO9dn}fnKBizqvjIRI&tYr7yM_d2D3CuU`hmd;JG=`<7Y5_ERukH;ijDeSm`%$ff0GBw{6C=I$Eu>kz0wnCm9LMFoQQ`h-A z<6VHsn+DMxaz&v za_vU#z;7NI3~XERv(5(<3@{NxU(LV{#G%_yj^fNbxCBJ)JOz$leBMZh6_>nzRjT)N z2ti?XF^p_rCH0LwCVR#lvmG2U+F)7s0;;S(%ueszhxPCglh8(VEAcK=hDX^ zoS4{o5Fik4aasVwiw`hB!J*lpFgz1=5hBOr6h z)t8_jap~XiHjlIefGpVV%C02PW(%@*E*NrLxx6HHyk8vb(c)~#?ezF~{&M4khGUHo zT$4&Wh3s^jDL(mq#s1Xi7f+-DE{0llo)Gm=>yr!`a(}SC2)eaC(9&``NI|fC4;& zqT;a*qu!=aum5yjF2%O%z-Bsr{;SKk3^#{P`po^0qO%NZ@@?brHpbXS4Hz-X(IZ5< z5k`y@DQS@I4k=+lr=ujK8w3Rv2~p{k21Nx4K><-hRPe9v-TUo1o)6Eb=eUpi_+7v2 zJYBh8EHe*rO}!HlJ^%+35Tb6!u$RY-lw%gH6L$Ow>g5UF&6Bs)w85^glb)-_c>c;g z<{*f%k%xGmVn@sv#@z9NGlnnEB;7ckUY;-c{cXwpQ}Ng^8WS9G*gl}Xx&U+qeu0!(!)AKJS6wp z>0%j=sq(inZ(%sj>vvy48R4A1>Pc`u<0s7)weut{^{FQk0xTkGzALD(OTAin#w!aH0tS*oE!g1d@XWk6wdc0-P5c_uz#&6owu{E5| zCTX)4vfeCGB?&J+stob*T-5qOREM^>D?D-BOr{ zs7)kU8$eUURtA#j(@JSaA7sr>rL)FKU(a{b4l+rj_)ASZ+sCgy4%nDY_^od2m01~I zr0eo)p?B^*N7NQ6-{ec`baBs2h-ypDnv?C^%a@dziiDpV1id1q`P$UHGo=b_S-gc{@|8wuC`ItfPhr*OMZ)tlqst zPLpM4RdJ5^ePM*zA?tDMFRPQFtQF>`1Fph1jywh57uf95&WRKWH;d@pS*N?f@3pME zTjJ@$j`sCpwJVcxe46o&^tr#l5k+Fqb;NV(bY8+0J1D_~^n?^KhO8v>l22Yo+5+%K zO4VKegO*zrONMs3Des^y%~Tvrs>ayVvg)quzW2Rir|kx-V1-9b?IV?O-kobjO4AQs zy(k_~f=S|4GmQ#my$}|XkIbJ(40fFV{Bq?5snE^E{IlU?$VT~R>#SfpD3Ms^$iQ(6ed{Jq$# z54ly7gSCGr6KqcS`*YQk-sh)9?HPLNb;D<~qW68*i@zzuq!fItuZtS>ES&G1)`x~1 z9d>Uls}}ZbsTz?w-W|2Ir%|pPcG4)v-|q%2$)A0G0m@8{hG-O56vPD z(<YC$1&PeBj@zVyvmWr0HGZb#xed6wniC-+;plXA?u;lu|Bw7l z#n)9~D7u(rVa*HZjp7hh7#}<3p2o~&%VWk;Dy4~KL7U%jUox+-xLOs)g}+|&q5Oq~ zxrSSMChow12~=7UT|W?kV_3A>$%FB&lWK3s>lm|6E#5CR&*WvT7ok8fW2L;;Y7@T&x3pih9Uk zSLUlXPpofqe{AXAqhqKxXLuPAI%!JsAxbl)R zhC6X#ZN#>i*EA5Xz2aImm1I9dY#7PeOowzeXU0MYxaJWlvI^9#;KHqJ z>BbRS1S^mep9GwXNyYre!XVnZfY_LE$z^y-qDB%Rq#;F|YNPj8G#udiR+m5-E=Qp) z3kSL%d9#|8$+m|!m=3mLtSc9iL@KD*fKu{z-kC@d7L4NjObG5#WT3GYy+(5IK2NVZVaj6<%1pP! z)exBln2$o4gB4Su3&i1WAIl%`*FIqJ4oosjNa68dRw#lp^^?APdspBg%dAuoNGuIl%-fk3npNEPnOelm#v$i(sv%th`EFC zLwf54)Sr783Bl*9GgelGs6@)uRm4rUu)iLF3JlBE9)Zfy97AXw-k=2Kpy5u9H$mPE z==3|_;xa85TCT>zZNpsCwg)xWVWJwUK9$aH#zj``EwT<{3K(M>1H@};gQZb7=gIlv zehdQDboQ&oU$tQ3633`rTEKt8a4@;p76M@Dum=DDy|e;g03#$Uf9X3@U!k$4wwMzx zK%`h(8)m>A&mwellmalx1Kf=Uo(=$<%$WtqAUa!Ty5eeWm=M3106_X_sZ%q*CyTZb znI>>lXmN?@65FMRv4~s?Sn;+tOxq{};YFqa58CVeWD+pHzVg$3#~v|;h~ZAVP9+v@ z@B52)cnY{y{T-lH)dm}5;FW~{LpIQLF96WWi*B&=scZpzDVRN7XT`TT01FeqQ)-)a zYWh5CxXNnudTXAB3q0DYwHs7@@1~>YD)tZoHi#jg%G|ihVx#Clbv`s#AKa`J)V%qV zfi%w)6pOH?CP1w9Bl{jd#o-3Vh%rFmMu(?4RcAs#{33(=ia->52q!cs<_$AsGno}p zkt$EB;;QyC=IM4#I4q(oMHkcSz)-+D#UeT|buBO!1{)A!=%~Ip}XtU~X zH4x(i7~bqOFSGHYl0~U(KW&&ef)Zw=?E!XW$ZaX5GDWpsx1A{!SL(pDuf%RaxV6DH zEqt$jKJn*ayQ>3zBdXf3YXr7VgizL-fD#*J@9m4!Eu z2p%^U(4{gJ0s|f!+Dz3veYNU+by+M_iXY51sQy`B+xx!uXHU(iANcPTa2~LhL3w|f zGqYGQ_wlgwW+8qIAe!8|{>m}@+-|->t^JSiMSs!^x}o!9fJIf?f7FkH5*tkJxc{Ku zKqr)ktyUYh`5t48UTFY>M3JyWIs20nbTqq^Y}3%D?D+M-;B{S!wybHFrWCt)DG$pc zbaJ(*hZq`6hWWJc_$=SL9=tYQ#Wq?eKMJk^o<{-`jKHyIe-5AGmwGUMJPf7{(7g*T z-Vdb}pvFgD7lTouz1KB2-51_6n^Pm^dkb2Xdr{z7ea(A zLnApfRj8mC8(di4Q97SyUfgOO@R%sU@6c&e>zQHYmc0mamZfF?7zX5Znspa_wwyfq zcb>)x2%zT#J`B4~hvg`C)`wkk3qbsra#;ui2ZYuEWEgO+6u3AU;A8+eUg`-ZgHBxm zg=psK+la+%gISB2vXhpzlbJ$Su*P##TNAK#r}kiM4#wkPz%p^HHerxY7oPFl;)Fp{ zLf{KMtzPx=1JB8-;&px4W#?oBcGF!5Q}P7P*u($;vw%n|P;fQjYAcWv4ech6{v|K{ zJLS_nwxh}KT03DcM1bj<3!mh{aD-LeG>g`u6o~a7w>PWMM=DfSKU;pR<@6ict z+UZ^siY@lR!T7}qh#Q6oaWnIFoFMg)HZ09@tHG8HCGhRvKi8MX8;2 zAzf~c@|gfGkAaWbLG#743B_mw7#h6FJQ`Q5M|}cD7JC`M#Mmed+_{LJP*5fMb9Xp1 z&?{C}thmwuCJ3Z09yEZVU?t#C`pACIVhV9j^} zhg8A#0uax?Ao86Nj5iVga%5-1-h#tt)@#-lBPAHaZ8SrvmtRAes2CDSWjkN7W*6O{ ziQTNJlV!Lu(INd%{qC$1=m>G^m$ku&RH5q+&Ej{4RF-O6@lQh-T0e#*E72x;4eIps zv9o}Mcaz=G5;~0C*Bk-P*cD$Q?*3-_)ywuHK>o*`85>WPZ1|APqBQcY_3uZ5P~FVm zcUiBG+3(Uu%jBjbN&H074*qr^a(?RXO$uDeZY~ z#{a~z6dVR~Po~FObXQk=JfHb!Ko-!?6}D%s??>n{m3h8h+%+kY^_s1>U;1>dTs7(& z@7uZ0bBR=4_(%KYy@mV_ZCz;a^1fBGMT1#`!^*)*RgXEsGHva7htmL&mRZzkHPqh2sglo|;XAJa#ylHUU+|I99@IKG%G@>6*B6wO#z@-X2IX09Etfzzp%$u~f*ue@J*=(ACIp^A-d zd&bOg`8tz@r)K!hIa95;gRX?!@#mGiuy0TLx4M55+&Cn5Ds)&L zBkey@=)>_lz`|UYy{kULWgpva;TR`%C#xQhRAlC2y%uHTaQSv-x5LYKp-^Or>qlGH zB=%c&?pcaI?K}%^JYD5+=-3@`x)~?(6dH7IcI@3O74IiV=N!o;Z6lbW<(rNA#BMxo z2uek#R|mgJ{*!sjCGLHEnp%iof02GTuJXCy%ZBtHvQU%eYaOyAp+Z#p?(P|J7qTVxI&F;bdH8H#^`v7 z9RKs;zK_F$U%b+;^*?xpC?v$LPvgCR7`G-r%O<{nz_+{pZI55a1w456voC;M<7DXI z;k1%^fa9hG-Mi+OvEwbu#>A;bk7dLiK_&hMMM|6zggjz$Z z)KXJL4w{hoQ!)=rvWqYGWDSe8Rf0jA9JKNFnQx8gUy%??+k~jsj?sUkR^4n3kmB46 z9$!}@-X#23Vqt9<%kgJx$bRp7*py}}?DHs_FDui!G9tA_DxE)j0%?JT_l=gkK;kVQ zMIZYFow4s$+?FbkOTY4Ra<5%X>-IUPqGsaHS=jbn{_{r4{fNpi8k|0#sL#_ArmP-x z=8qeOj^Vm+Jo1z(vDHCRK7{H1t|o&8!Gny!h-3`e`c6zj2wte(ya=LnKYDFj@aXpeTB-wYpS;SiC;dBWoV6^aqNXcCVDP2Zl6gr3j&J^G>g#vTUjT z+=Bxw9Gnk6NFU;2ItpO1D~x6t`J6q5e^%G_Kz;Kd7(I0w&hL-UB-cA)KitNeI* ze=1ZzkU>EQFg#pN3xC`~$3XTKJr8xucuT?|n@W<9hT1gj;z`uIGK&V@+fwFMm{7v( zB2C~-0M263Q`}F;z>4(~!Q9yE+H4+7C|xioRn>&mUx-;Mui+AQLa*`zHhD*LKlU^x zRmjaspKE6Tx-U<$7Bvmaa#YEUZ#%IRbL`c{uAP`J{I|ieudg8GY*kLe?y3x9GO=Uh zi;Ctj#`eRO;EnYlx08Pu;GY}JwNRGx8X`^HzaW)YjzPBn}AGTx3lnW!` zgo|y!B;vp2_>Ianxb8J|3q^?&D1gpxfJQWw$avcoj2E*TEC$FVvHx*}URh;dbG(rEX5W#DhS_MFg}({E@Fw9%-9hEtS>uyH_tgkF9{4Xy_qV8;tYJ55iaK+R8r}7&CdAJ|w=3CvskipOJ-HET@ z-&Ypl&oZuXXq&<*oJ5V>*`r-q-H@ewS5cU_zBS`(%8fcp-|nTL!Q3%z;2wIK2Tj_KPJ=mF z+6??m^lyx-%|;V{{{3hRE}bBo#H`<06OO{Kk{C@l<2U}h3PunHXcEP4Z!zLyM#NlB zbh-Ms`8J}JZhzciBV9+$i=Q>Tk$JOliW8CE`cj_ea`v)vROE?l-#DT831L4CQc1kH zt36CP6;P$Ol|()<@Vc5ErFA~|a%)8F1xR~m_E)b!#nnS^INUabx_l*ki_6!UX6-p1 zu-fG^8dHEPEi)pq%qD8?+^_ih`$P0vo%pq3?pq6yvLQ=N&pAR6)Iwi})!(mp{GZP^ zDgS-OQ~px$lxZ%f5{=)0VwnX^srUU(`~7n~nsm1ZJ|ZAX>J{Gzq@(%AwR=P{A!-WJ zWt%R-Fv1wErUrImewNkz!3t{Wh-Jsqt8hAzHzeC80o^@3SAnH{5m^#Dx)_FS68}K< zQElb4-HQqH7pB9V@%NjqMW|n=k?WPbVqr!0_lG&|u?G;er4*R1_W*ffAfQ0z=`4{| zisay5ivb5W?fGdPMW^8^^1xD^wnsAxiy96w7hn#^H3rs8^g`97;N^nDYtamcBe^9Z8TU@=4%gXY(S~q*K9oSRvZ2@vyV)TVC@Wb9$u?V7RNUMLpa}1^8eB z5sS%>gJZRw+KWG4nKK!-DA!+z7-m~)Moiq3{ZsK0&QP0iC6BCGnvWPg7`9ov^h{lE z$ao|qNp!0q{k)n5gC-rt;hQujk~QMf644~L36N3`odbhGwKh>|fDLw0UIwr5u)Zu` z&#r(phF=crc&N#hlFuzT5bub|HXSo07$=m}*=K)|8E>dIq zcqlI^T&hMZxrtPG|5kZXbh2S?c0p`*)vYO@L8-{YbBuDekRv@F7K;E2ZPtkqfyMoX zrBv%i3%xXT=mj9A9U58`8b(Ck@sS~gXe$%SjWnhvtlt{mKTmZTQgsIzG!bg=lj%G* zC-2mcD>{gR`;^XACdZna!mwgi7~{coeyFVRllly#gJ{~$q{W%hlq3yIj^uANov$-x zVKLR(L*R{}#ke*>crPvO7}k79XR~!}9XT6!4BJG``kZI%9H!y-z?u7z@DFF#TF7bJ;LQ8NHJ(-z zZ$4g%9DdcKVv=7ii9ufl5{;{$i$*%CUGx2 z5s^$^!U#zR01$-FQl`}O3I-^IuoEJv@H;dC5|j)GG6%`f(u)urCxAZ#%>}8)TLOdz z?>pabM1ou+n);e^AyPIYX@Cg$YXB7!IKKLbAl(K+{96&aV$sNgI{!85P}6lodv=%mffE#?W)G6dL)1%OIP99X2sBqgGy zo1i2ub}%L;4LXmbiAi@hJai{yfti(o~zmT+wE6Z zwxM<`;1{UQPeWD?k9Z5tF_?6*Ji`i~4oKuR+bCz-TS2zb3AO>%$@~P{xb`=lz@YOmJ4~7{Mh0|vp32j09X*dE9g8&e>m-z{-j8uiIS5YD%Rd@nOCGTz@ z(s}r<*s`ba9>aL$AfV$6a(JfOJaun>`X-|doyEKE9%~Td84^v1o&XLL$-K=k`ol6M z+65>j={MfV{5L`KLyBJ6RW&~*Ete05n;;1^J3f2vi0_|qe7^4Zf{&E8??^3=J@-md zty^t-_of*@*b9Z%GqP}eO*8&6)0_hbK^I%9!H9~2v;<~$ImnxS-Qmj(nzZcMGeqHe zf@GLYtPEifnPNlT@L_Gnr7OcJDUgzUf}1M!*)u;7f$=7Rv{4M68hD><{#0Dv-?ZoLRFB20qfLo@l$U5(K$ z?LlC`0FLH(?GFn933mY=1-_9Hk{P!-Pm}{NAMD@^3ABG<^((b{yP!2M%f@TKt{*OyK}bl) za)=B-f00lS`PK~$umISPsM28!0T!W>5E+m!q0RJI%s9pzhAyS=7!bOg^oBN5YAi0~AkA=L6g_}lSLSxi0g&1p;=1YA!VwPH81 z2#6&Uz&*5mkx7tiBw&=av7tEz?Sf(_>~>jmO(Y}jOhE{O)`3fv45`0&FP?1a?qyF^ zDLp^0sKO*^{eUrcSojBdRY9$R1jI#xuGKrvu{c_Wry0#~;eo?@5-{pfSHJ<)cdukq zw%@}?Id~tKI?NqsyrF;B6q-~8d53-u0EBu8;3_Yrq?&f^x!J9^e@kK6D6dEn7%L9xJ0eI|c6=v_f>%*iyj$X5dRF{IZyYMoKYBxGD zv&c#PEy5sTNp_kxFU~iazS+Bl%TdD;fw$NLO1*upktvaXrvJ>m=*@i6Mf{hW_rOR% z>FZ#TzSPFN02qSc7y~IkQp)>kS92o>miN)$Of3;s=a-aaQsVcT6Cq=r;>p;>MwCRXZ=7iY=l`PLsT0q z?Se&eG38N%Hy@)FShKS>WS8;?Y2LsymHm?|Yu~xyr{WMm)+~@KxGMxrsHKI+5`aMJ zIp4CCC_{3BWX3~5vhC!9o0<XRxMcv`|lK*a%G<_@m?q6t0 zMh(_>l!=eaKwKPRmG{{4V=kJ+m`ZJbUunJQU}mWeVT!CZ#YaB6@`z3-_X=CcCT*E| zynEw+o_3P0%*dsl6s|O-w*^I?2F??>H;uagyU+Dscd@`@gWDes|9iC7s!@mPzWIkQ z^FO(_sRqrE@y|wyG0Dc}563(57gqDYeMsk@qkvDKd=0t|JbZux{;op2^O9k8)xtny3=PPda6f_(rZ@(QQ0J1)N{8-=mB3P5g{c+`E&wUzK?9F!9UF#KXD7 zuP^^4&i^rKPF<_|?Fa>h*!=2d2b(KD@Qn8U zn=U;BM8_)SdyF1RhI7gV?j3I_`8*sqvk=azO`Y;>*pJO_uHcKO|na z6+0ElAN-KCOaHusk}M!)wYY7}%4&$Uj$eN22CKX;289=ECZb)JaKQNjAKk|-66oS`VRbI(HNeoVB zh<|h4|y&+q-|w@3IFc*)7g27cmh#rIj&oWenU z)E$B5dDZSv0YlBP3N`@|Ct+6t9o`E}3uMu*BZ9@b$QfYB8G%KCH5owy&nJ_IR3SUW zJn>Sj0FZ$tk^{>G+bjam+;v7KLu8<#Sr}M7ng}_1b&H73qxgm9LO7EVK!45)h!fE* zQ&qGSKi=s^qq^AAD|&B!D;y*s$~Vr(nS+j@M-CHgMXM5iUo>6YW29{k{SdRnRGT5u zhpEd^`;Dp3GnQz<#5vVrWGz+X#hY>=2AF2ePDKV0ubqL>%SZhMi0EMLu$q*>hU&DN z+>-@K;)T$a>!^nc?jvcc);nYJ8@{dpPSh@b^}1+*hy@M>)}WbjXK;_a1J={?HiZVC zMKIBn_PFT@mC82X4}z`j?S@pP{+8}Px$8p@*A-q8eQ|N{Ws*@ze?oUUUInspei*O9&Q%-JHEZg6x#HXgUCFD8t1`B{sEpaiJoBfJK)MJ1tdnIY%wPZI5K}96)FkNY3$0 zWPmgNui`3AtnLI1^CVR1o!|ZEO7DZ7{Z(2En`TsoV^$ZtLXURxzzP9j@1s|V^wK@U zNjz5zjJ*z3vq#d@E0>7Fp<%>m{1ShpnW!E)aBWij=9BPg7HhD+P>yd_9*g;z)paHr z5d;>h_A-L#m1E710v;g~EJS~tC3!+`ObHDxz59d!GyI^g9(&?~*S~#SFwtsMd)6_i z$MH7crRJ~o_=lQo=yYZh-Nrrq)!+N~wH98EQ?LbT+n@}=!?PdnwY;}};lI4JVY>IF zV!ZXI*uZp4D~l$Y#G_Z!q?reSkaL9gxgK62#B|V5Psw=Y_731yrlW3ZtRH2*6S$T_aGg zGebiasZom=6}Rlnw857P%IXXNAY244+dqO-hsc=hkS<>g{j|C;QfpmXfM9l8h&MX_ zr3&{qJ}O<#1QO09bUQ7VhIcx^iI#R)!}s#%P+e3d%V3A8jL?q>uxIK*60g))TE^{E z*=!)n3OopHF&^>}PMb7!e`FjYtJ6X`jM1K!`Nm%MN|>r%z^BmLB(rJ~bY@Ewq`0pp zL8YdlZ~kCjxsg&X)7OG&ddM^bgBBuA$FG!k^5~QikIP=0PVhdA?TpwN;EI&RF>$n9 z8_zLD22Ws_`7zAB=^#k$L><_IgAan0%;Up)YDRG6n=U_Q;C*E)GcHcO=jrj+8e)Zn zQJf)TC%jJh&?MQ_AvLT_K{ZE|yV4bjhu~hf>k=m=xF#PsW`_VRC9J4cn?sc4EGy@G zsrKIAc_m8YstPN`taDt86-=Hp+iFU(CB4Hh*!-6D#5T`39D{A1EV2ihwFi`* zS(3pp&_g1R$wanmvK(}X(I}=`C4SNHksL>p_YX-=+VKhv1QwZh!enyqJl9s6z9DB* z=aP`*yX5t#CkHJ0C<*R7;pWjN?{BBc$^l>U8*aLOIfHA=zFuAbG|e_2aXFMzw4+~J zH`WPX62Nw42O@2VL(jXK>m^A#YZcwjo zvi$JKRxMDQs2F*$e!;rO`^)Zu!`fnxH+K>=HvNG%wfwQN*gde_@_?9$}$-vr{fSc*Th^U+slut^T!>H(rOvN*V2jXUeo zDh5iGH0uI3iTz1k@?&A8JF*{k8`02M^J+o-AB68JwHVz2DXf#)6}Rf0vUwqQ_i@{< zq?i!B*m3f0qSU7fq!csGyj~kq)2FhXK_3wI%^0onNtJDPEb{fFs>?#sH&$?0G+uOB zwG!Cw$fb)g8pz^dExS<`L&K?ydA}yuuEb;RuvqZ)?EN3sCas9yqDL`feE&%l4-FM$ z)Q8;=1sRqdVtSVzxTIIyyjBq1AC>22-r6=f!eJ3QIMWeikIL6hmQyg{-%EkUYYakl zWS+g9O;mbP#~6gLCQuj`U)*~9)*KHEjNo}!*`fbnc%9r=(#tYt-o^9#V1%X}+zIL( zH!c(T7Ar6O`Ybga^Ujy-$W^oQONpsDi7fFsRoPo$0w|#M6QPURMZI4se@Ulvnaeav zl!5Z=mXbHe6@qz3JjrxR3gDU8;Bp1Y650tK7Qdk2d^&m#LG|3kWH!Ab086>{QO(ylx zTi!ubzy0#&xdDH<%`GBtGk=N_h@pBDVq7wGk2AC9`k`bxZy^>LaMQ3d2c6k;FX0ns zU&_}wfUV}X(U>V^XLjr~6#u@9P?n#3<4^%~E-d21WZONLUq4Rc6~}ZvuEeRmq~X3kJuP|ADJGC)^UL*10{>}vFu%`sO~{9Y z9>tdzg_iDl5|c?MHF&nCp&Z4q6GCVoRo4Hk&MxeTkZH~XECWUq}xBo3A6pwzEnVK(b&Ymi|Eol{k4>S8hZs27we#I)YxSV>miEv z^u$@G=F*+#AcvkUo~tWFb9(-5{S@adY&NF|ruwY;k`@cG0#7igMmY$f-386wpafk} zo10%RFNwv`&K6^-`)mHEWD#YWSZgmGus8UkFyD2!7gvO;sd`BY;&Fb-~Lat(jT!0=mKeXWr&>GDuoq7v|IqBc3tq?V( zCfBf0!S#C|tL#0t^w%YYM=-Ai^{C8T1kw6LZ*dok@HJxH!RoW>S6(xNf2#g!XNNm2 z6fYU2Rpiv$V`S*4Kl9XCJK50Q!7{73n9P>ymErjeM8MzP?p%MDOxYzU$a!8{u6Zw> zuI55Urw?6bNRrh?{Hqxer23%O9Q6F$MYVxI{#`}qbXBG#%;AjMr@=2<&dDa1-@9N#Cd!Cxl@@rgdkNP8nzI&XZd z3x(MicIzr;ZqV0>MZlNV?$7An=t#cmRYsNv<{-C*=)PA0xa%6CI6BeS5rf*kvVJ); z%^`6H?iDnh#QmT8RM6k5Er)?*pR2i5$TS)Zgmfax%E0gd{+eQW(n z5BKd~=sYlGTSE;R>MUNYi*1kGHO6CG-*nJ!V~H$KQtOWE!7di=>z~VdV42bK=A=Le z+0d-iFd$|4(EPym$+j9wcSZOTa&z?Hf=vX2P~f`9_j*`H_!6fuaHA9?+;|YPR?&LK z@1O+w)5f2)qc1Au`AAySQt%Urf!8K))UUNLTa$0LvvQobxf8_M;$1P_g@3(^w4pR_ zH#Pi&Bhz()_$9qlISV{Nk6UZ63=;TnrB?naLk|qlW}9~A2G~8>j^s|F#9(gow8;SZ~1IlXSzMbuOqK zJhmvGy(rm+>!8I#owVU4O_8|fRS7)}={7v++E_DcqoFBNQx~sJJ1CicpR%5RKk;9E z(%VXZG;2W3MnI*7xs``5pTKOz;& zG7L+m>kLuH43`btXFa~03Mj16JZ$}3j(p`KGPubu9WGV-FkJvyzWl4~p;cuhKUXouW4Zg6}6spk`Y_|@#v zBhmZ&X1+%p5J4d}XG9s1SX1Id1j-|OInm84{iQubh4zL!xm>_7_vZEiY zbLAqp;1_p=aS!po-nF;6$1Y(t6Dl(oK}d588x+IvWYd+%!i6XxGMS=Ey zhr_1YRr-I8vjhQn_9wsaNm#2?wFunDcX&FtYaJ_tuYc2`@-~}#c-j{HzoL!({)(WWeQ$^pm|P1yo(b*uDLn`hEOf4 zpN!qYh%C(>ck%TEze1l|>5v;7CTxzJZHLnA%J(~l1~&Q(;X4DaE7XrMWeuiyGA1o| z*5yGQP}P>-sHtyC?_tG-`>gp_hBnF z;(Gx?&>x+r5Fp3;Mv78NJ%dz(#pTE_FHctS-Sz?-Jn~zvymup-0qOv8y9K&TjNViVE1eYZZ5esxj{-A(eEqj0@dO9kOUsjo@{u(Y1}M# z6r%YA4;H-$6>S?+5U}iE1;#cq)xBt|NN>;HZow-Vdq_ z{;odPuI!^I+9_`v9c6?6aUif@9)g4G*i*|*{%^3B73%pdgOk;`OtXmxPwus^{7(O>Y<|L z^(C`kOMk|?oalczjk|lwwYtJMSVBF+uS5@r?}Xjstz+#DrB@7Hva)!ls3i8oLB7k% zAPRqD?rZv(l3Lc&->p-Z!S9nXE1sOKt)4P0n{9hGr*@&QJ33R{LT=&aP8jmNEW}a% z?J|koGDBsw5$noCm2#e!Qf;pJ;)gSXz|D)iU;7P@)gDIJc7M0^=}8SyNsB&Lj*e21 zY!9+Lw_LdSe;yxnD3c@d$U=ws$YXt9To=E(^`T+u21Lc&;irW$jPi3`j*~2y8~?z+ zBZ!eAD}o^EMo9Pr+y1K!AWx4NH#aB8)3Of%Fwc1O4Aa59Zwj?86pLhXQnhuLX@VkE zOL9l$kA>k9WC7xo9oKNif3`FpyAG>c$_iCQb*^46UaIlE0(_^=7^nKyq2HO}LuHbuZt?6?{6uO?Ub> z76hA@s>7c$BuzMXKULc%dj}5Qk(lBW#;PR`+zre1Y;Egqw0)X*&|rvZV?|BN*JBsa`kJw-mefY`F0 z7D|M)zIOY5qQ9|zj&n{Qo}1_vsc8U_aYfOdc-`D=NHDV86YvzasxC9406tEOByuUG? z$;Xi$2xGNSCavX*O12uOHOBFmfZlrit8mk7up+TIsl69}b}HYf_g|*OtzCTe@C)Lr z-OHv|Y?Tg^;4ej`6^_DjsDT7H!+WW+7v}linzubY$DC^?rPtHK_T58NIP6{Kohv~j zs>BT_heFK!W^eRqHtYbHWFllWT+OH#!{UL<;ZePHcFy8ym?va(S0R?w%eY9={;A$u zX75-3uc=1FviX=*sE52TImI888yLsNa45zh{VW@lc_6)Hp*f07mPrUxfOX?|3L1Bs z-m1bbpx`^Fw@U=+(qvPiTPQZJLVu^;h?md#a4p$+2=2H-KgqknM$cjjl4GXy=PwUP zBxP>13hJXQh~ln$>ir@jd>4x5)(?QiAD_wPnBTee7tgKaopPjCWRwRo zcK4V&7<+*Ut+}DBiDnZQGF+DNcviELLjPy|*)y;7dT3r!7Ei`hsrs+Ke?JX$PNzNh z-=UUb5&r9*aU6Vsd)4k$C`o*!2@|9EYi~Kxql$AJy!C}Gm?%iw{ErRM58=Cw7=k(< zFf`5y_ak(ZSN*Tm;Sel7HJ>UM%_Z(BVN$Kc(g5Wa^aE)G}V>S`;qT4X?>*c){v5guQ+`S8o z(mjTEpvtG!J9i@h4Care+12Pwt>pZKnU$wx(LuY=ytm2S+>cV{(zX%s*O9|Z`j`X0nuO*TOVud{$8vqlru(zVCDM{4kR~TbguG_`9N+lDEtN%{**Ql$E1mbX z1KUPKBV%a)Y3e)qG^D}=Fq60*C-0oYHuTgmw%Q(rzaASw5TNDyBjs_;DtPN#>8I8% zhW)r|3pr+K62VQ=r|Z6DZ(GHC+j>-v-;QiL*rHVVY0e9tqy83mF01E%A^W^m*&Xkz zebSH5jz+#Girhi}oSxCQWLEStmFBp%MfwpgGc+zt90ZoX}$z-@WG6ME41${gKUuGV0d5h$Y$Wn$*?Xwl8e3z&Fd&&A%C3&gL! z>`8%(y{_WhO80vR?^Gh99pg>v{r;r>aGc)R7I=geb-xuaKBm~qZr%iJ!1MObJJayOOQqjc``$L(O3U0its26RZh555jl|H?Iwb{s$?00i9Y@k5t;L|s{m;RB_w`mv=Y`++|&<}(_ z7e+tQSM^Y(n8?qu`K701x(p-(F8{!C(Kj-8Wqru3uw(Q0kchPWz64B<;0!&ECem{q zCE&MD^UCUl)58@>tly_U@_#nay85Y;6w_AC_cfi;B+)p<7G!pKn1-!>27E1}8`E99jwD4AHU^QwRupL|~ z>K-Dhv^!$pvK|@*;Uv8=@fbU}{Df6s)B*ogqPpFWI;5_hgiK6^WbY<3_oskw3S@eL z!cCR^?FUH53SWFTL9Ao~WEG`I4x*)Yiagl&?dNn~%a#+9Z7x0c%Y?cx1~}Zi26WcZ zHV^OQ|8eX-jxiMV*1QbUG|%I`oM901V%y)I$5-r{SA9r{4&q`Tc&828Mv~nt!HeGz zZ;0u&%*H0s=h6HhNO-OI!xOoWvJm;Xe2x)H3CJ0 zG=$iK(K=9_`C^=%QIHDzp{o#n#d3(B;Yv^v$WF&vG5?fHve_eFLr zi~kqm1|9jn*ic=Smt3Zh7JR;2VGC={S8qKww5eTfWuJfv!SM91$y}r;mWj6)kXq2Dgf9SQ zP0`#A%>o;!<7W13Kd1ALR@Hx9Z+mk@*Z0N@6ows=(r|$i6)}oBaB+emOu`chSNOsi z-f)LM9O4m|_{1q*afP!11vh}kw)G9M*~nv1@7fs)Ik=*zP6?BMMHh|X-0`l?%7&Z0 zV7SWp!e1$7lUeNe&Uqfh=k=}UnZZ+`8d3%z^F+);xB)pV)pVyn9qNBkm-^JH4s}nQ zfDP&hdQ!FsA24+@sfm0f6rMCYDHS(_L`yaQx4WLL5NTG7&0G{J&-hl)WpXyr-TK~l zo^QU#r3a}q$t#S|2oc}J8@50JA*$YZ$3GtOk?+S1vcU1bQ)RvLO<}V3L~`pS=!{kr z1-@RDa~fawgccs2|{@V)m9>$=sVz6dW#|inB51S3sq^L~xAr1I z{DhWhkCjZ1qY6+5FYoks3Ae7`2BGK}Wf^ksG_w z8^2HZaPJz&ksLR|7|RhPUZGQ3LLl4#2HYU9N`>!*M;R3jz9y;T%B<~r@pdYT{9y;SjobDS!4C)Z0`HTSuEE4)W z(kPD-DF>?~k@6gdfe7S*hUlbLLMlg&BT%pes}z4i6!ZWWkdQ)5;R4=)LV7?DBySjM zqCO~OPn-@KU_c(A4hG7S`EDTsi2srRlM*lUQZKh>DVfsOj6y8l20daX2cSxxro;^x z-~C1WBJ0001>pdT~H8r3gk^nLRAP@S1G&xgc1i%0QAO!BA5e%R-L!cjw0WvS3 zG$g<-_fk5ilREXLFOd?aE(R&EWoh7rvy31G6q7M2L>?LeH-n%qDS-gap$GEdEoGv# z`auw&^PISVLPS9V>cK4PfdI-9GW3KJLQ{Vp0N@q+fdKw01)iZZh9Ln8GzIb?2?XFB zUV%2_vLDXm765<}?jZr*MgUyxBm#`#KqoV%ptCbBvN}^#MOSoEuJb6*A+bmZIzp%s zV89E;^FEc+0B!*$Dv}7~GA6zu1%?48zM)tMAOgg)Gj0Jo&B7Q2AV~9IIBhZ$2w;Ct zo?@}SK_+`3C*Lv$+@K%sfdCkyWg_Dh7yu?hKumG~Cf+dt%;YwP!9-;OMOoBN?-WnL z!$pzOFB;J$hQS-UvN2;q6qGXoc(fnN;2wT7Gl#SvXro4uR5Ux|6`B+Zp0r2t{$qkq^I`b4)b5&P0BOm|( EJJG_;$^ZZW delta 580724 zcmV(&K;ggefhO~UCJaYOMmRZPt_8pY=l`(`_yP(1{`>y_{Qv*|lMMqG6377m04x9i z006E9zykmX{s8@#ZI%Z)lWzmV0{)Aa0pbT0lN+rTlV1eV1OBkalQaeV1peN9_meaR z`vv|$45AQ|Gzaht|k7+0}mIHag#I&_z3<3*1Tp2 zN1>B63OxbqlX42VBmP59@{yaoQq@jgvGDqyheu*$qz&{*f{{lUiqT zBUzFclQa(V4gL^zm*@tO`3-nji_McX5AqKF(2xfx$OX_asDe6;0+Tcl@elq4n5wZb z8?!P?qR5$THj^|F@DTo>yv%EwRcj2U@CwbVySbAz67LcI^$DPf)2?|a$P#Nkg}ajS zlQa|V68@K*tT=D#(b=|;`iZvL^9{k1G!*R<{))zxD7KIs+$ea;qY1FsEuK}AG!^U= z{sP&r|LmU?Yqh<2wv?Q)I@_OQlQb6V75?fbU-Bq_@+yzJC>q+n3al=A2X>P*7wZ-N z?t!|dE)MXOs{8>T0%{ul&rk-hlQbAGf4}+uFZcja@18({0}U2Lcu-+Nh6^1wg!oWm zM2ZtFR>XKwBSU{3J$?ikQshXIB~6}0nNsD-koD@xdj}0#Mw%1t)$0`Lj))8yNc;pE z6lew!S6l!9P{6@b6FQD;ITBUs)EF^Vj5t$k-^FJ^yb_Ta7TUKmUW2$n*vRb6e}P=g zK4bWf$k%6e?cRO%jbJ^uL7>QM`&IA5VP^9X6tq_0E3bc(cojA9+}t3UYF^%$xt?6F zY#A2*_UrN7jP!of`*;rEm73$f7yqBe+Rx$I1w&T%tLuB8?DQ}d3pqNj#2SmbAh-ampzXLbEWCqec<)P|=MRN8~YtMt|}6h%j4 zk_-U`B8wE#L_t)fvdR#kI5A5tJ>+s24J`ugFNuUZOgJ|j5)iG#hD+$1Z>Fp6y6hUd z{t!djzQhweJ@e#qPe1zv^iM$pt%)T=4@ERlMHlVJC75Iy^rD+`v4f|dX23Ml8GxE0 zgCq9fqmoLPK&pW=#z1uke^sZ7s;U)MsNsk+IN@X*jn=etHX^VyNI3Z#?8`NW;)5bP z29JF&u+Z3RtU`zQ^U6Jepsj46HQjU0JH|STl+a-*o3Ej5lc*5a^Pu2@vLJ352tWG{ z5~K`y=Z$tC*0?J}ly|)4*C8?!k_QTI6CyLdI_MC6X1C|Y zL}TdMd-Cwgl+T)7e^YLU-m7P0vmQ=ayPYfUHgLcP7rb!74>x=uMi*ziamP<`G^X8L zqP%j;FULG{mweg5ClWLTJ<~Bl6}5CGRG^hsNX}q=j3Z(gVT4%6U>%awnR!Wbm)^Mz z=h-Zy+}Fwb`U`VnRnxt4;*noo^Z06esrNEmI{B4)E+n@ef4C<^K6&rI2S54vCck|9 z_2lUt=jrQ-<-wHYHloaJ19Q@bAYz|9>riCq4HsCsY5t@;}S{D>}t;S+(g4`HT^=22ze>3G_-M@^&D32*((lp^#|au|4h(C_XsTj{~^_ zy$R}$LtO$$=A-hyDglTktxnE)t2MKRDX(n&Vo?3vH17_=PpP;5ptMvjDd zB1%LJia@&P60^1ok!xmio83Gp z;&cj5p6=AAJq4f6kCIpu!dGK)OYWiU*FAWL6)k3pHYH zW0ZIjiH4yr0t!!*(lH)o5cs|36-Squ15FUPNx8U!C}&=DB_8QG{=Uo!2xSc8r3K?7 z!g}}$E}47YOU}V72E7AU1=Z^u_h-4jK(=xX`(kCis+?lON3$(C}bh#$FqW$Q&#mqI#2>UV<=F(JH z2)Z^2NubbYLp$28y2QAs1#o}`JYWJBf7rm5Girhr>^P*N)*CUnlp&)^20u`>s+F(- zR<)|teSObeskGpGHU>SjG7GJjJ6`zO%0kOp&Ol;xRPP!wS^i*NXsrjnA73wLNiMh~ zW?mUah@F$Y`naaO8M+tr7}PKr;~0OcrQG&zydSgNLVG!@9JQ?6Sun25n)h($f0P|~ zS|{HR%3VAe20K%^n~_;SZ>8^q);vNY2n>N-&W?Z%v!?Y_?m}}dEt%_h#q*jdyF_>{ zgrIQ?DkZ29%`MR|hz8N^Vics7{%#XJI$rXcx6qOHnH|}CIb6-PPD!ng2;bb{HF||Y zC(}!I!rWi~3cAd<#&xcBz3T}sf7sW*K2(G8coSa~5_A(L0~r!|;nJNz3>(&}JHqEH zTT?E6U+Hj(iz850#d4jDd^vXo0k$r24j1}jJ(vDm8ww2rG_Mg$nOIW}8U_l2 z{e(l>%iSyAF7ay;&dol(9fn#2d`TJQt-CYV!<}d8-Zy$zyoaZeauUxOf0rB9=XST| z+Y1L@!&wGo6DL(dm8qP+X-KU)BhOcIrCfc07iIZ9eCF&+Pm2#s=ueAC(q79_S5$EnX#4FqS_*OyvqEg5e^kL03?I95!58!gyu%|b*t;N6(pAibSasHzJB72$%g601ow1{`syr4tbvNkFG-y6u7*k1EBfQ$Pre zz)-6rf6EmvsF}(kr(DTAeUc64OFInN!#?E2Ui8JcgTY_~HW{>|cMyU?>bn~RJRRIY z5Ac90_(58!e+B$8kT%pJ*?np9DIwnD>I98j5|i;4k*Y2 zE1N^%s6NcmGHZ$`G@6OBAgs}_MJ`C26#f(rb!3}NP(3d2LdqEkf}9P5oI^aT2mjy> zL3oW{>?_~lf|8j-w*ZUFv8O+DgIKg2Y9SwZd!H5Je-0ub5QHo&nK=uPytPT}xt|Mx zAJUR%$gZV%E=n-K13X0$si;tV!0j>xB^jDcvoxAi#Z~MQJ6b~#Z79NZES^a)ij)kjSW{$gmX4vBV=`G)n~|#yWxqOG!p$Og0N+HXiVR6X?MoSgxk}8rHm|@%m$0ONQ^AYi$nSv-l^iC5QV@n< zn|G)S5h1QV1k3li4`cWSW8f9cX&3&|G|jzG$vHHL(u4+M*p1K}%Z-pl&1sP6Iw9Ib zh}#U!28qoUiN&aFmkv20)_h5r{Io>F#2b-`e;-M-VURxO5)0^jPUnO!?6l78+{EkD z&ZImq39KXAM9p(B&0!c$(Hsq*b4hiAP1$4{vXsyHq|Yom%ljm%w4@_t;E4?IJGUf* zz+=Wr-~qZkfgA9_tdh3-@wa)@9bBQ1@!&(}aL_wEL2vB52I`Gru*I_om&~aT@~a0z ze;hXy3^Cv0NW9S<6qUsgv!?g4AkgGZgv1!!tWg*x&KRvvV{uU&wb2_zO%?sJ%<0h- zMV4Jt&QA-A<|L6si%k)!3kZq+Qto8V?`%@-O)CY@g%VFXI?);ZvSayCGBwg8 z-O)5v(>1Le{B+Yj($6{a1rZPd0R6kse;LpMHPEZlDo0?1+`*s9^d3P)&fdh$LLF2@ zC5U%0hFdU&SIA5;B~;&J(@M3}OT|PX*PI;*v}y)lxN8xqH)8ZJaoj zqdB$H8^lu`EKn4nOB+yv1f>;6xB(Pk1b7HkMO{(WB-BZ5&0posiAdDn1lCh!f7WJo z)=N#&P?gqcrPi|f(r3liY}HmcQq^u<6jptsNqJSbT(-D80TiHB8{h#Ycs59?0vmu= zLh#jXrPq43*LzjdO|8~_{^i$dWkq}i*nr(uZzb4ZdkM*jtXvpZauvK;MOPG9S00c9 z4SQD};DLBGOo7$djpf*m9k6Kaf7g%|SyjANk2Tqo9m|4MS(N}+Hxk!{y+KKs1SPmI zB{)}fJpo%)0*jqkF|gR5y;wHDj*mmxqBYv1ePoUXFve_1RSQJo#vfbI81=^pr0XC>Nsdd}8g_yIB+vTb(uA$1Pi+WdgV$+P$^h%f(!fQ(C*_ z+`4_*%mv-hy&u0NUH*>%+!_*Gt@VRth2C0*u~Am@Em=&gs=?V^{M*&RGr>=oN}-CnZ=+8YQ4 z@MYfuHela1U-VVrP*vXpc3_WnU%!Rl2$Ej_^#gCH-^@`BT=-hqf5l$L{olt0S{v8} z0)F5VM&YPU;00!3@ND1|hT(dR;J%&U{Hb7Nv;%A~VK%D=YY3HDJ^6;x6XmEFN3{RfY`yVl9>jc{qne z8e6fYRsS7gwY33He}IQG_ToA&<2$zFJjUZa*5f|r<3IM}KnCPN7UV)E(xe=5$8qbynwgX6JWy=Xi$a zd6wsTE@MvqX6Ag(5}%>xezxa-_UC{G=z$h!ftF^KZLn58<1qd^Y`zIb-i0hM(20iF zakgl3J^@B3e`tc{=#BR1jt1$E7U_^C>5+b9Cf4Va4$o0F>5_KomWJt6v!wriSXLmg=ac>Zy)o98&45 z)}gZjWvj00ulDM&25YdkX@X5?R$l0st%vBnWOz^qYh$nlPhbQ>AOtqZg1g4+yw+tvGz{&}zm&nE5C-e!1kYjsfV)n;wgwg+{P2Xz?jcX$WWmOKO`f97ngX6)z|D-Z!? z=eF+a#_sIa?(OF8?)L8Bw(M>_>sB7z!hL8`hKbCU?DeMX_Ga(*cJKI>=j4*^`W7n> zf$#j5@BP;Arw(sb#q8x)Sm=fA?>6uQNALt!@C9e^26ylWhw$ZwZu@3zuI`Ws$M6i- z@D1ni4&UtmPSx>df8|_&UeTuHbExh9=I<3(@fK(C7gurmrttb^X&1L~7{~D&*JKcX z(*Rd))h2G*_V6Jm@*+3#BS-QiSN?L(mhl;nZZ28!D3|gnr}75paX0nxD%bKY=khN1 z@-GK-2{%qBx9{o>^D{^DG_Ue2$ImQR^EZd{IG6J|r}O6ie;G2DZZog*J?Ha2XKpsX zPdE4TK_~P=H}pdv@-fHrJil;6cl1X`bU;5#L5K88xAaTL^h`JHJ74t0)^kk<^-!mB zNjFPM7xhy|^;B1NE?4wU_w+_z^;nnn2Pbu5H1%1>^<3BWUDs|-cXgHibY2(sVL$F# z55`+3_GDM~e`RNOc`#08{&js0_GYK{YS(OIhrxk(6lk#aZujxc@hi`y}b3pirm-vbQ?tv$GCx-Ic*7)?^_>SlJe~C;PEC`?5#-vp;*S?{lf2dc!>j znXr1cf2aGp2Y8_0d9Od;D{}h2_xryG{J{n@Ae z+PD4N$Nk*b{oUvN-uM0A2mas}{^2M7;y3=|e@Fi0SN`Q^{^o~$yr_$SlK$zZ{_3~> z>&O1=*Z%G2{_gkw?+5?z7yt1m|MEBg@`rvWb^i5d|MqwP_aA=MUwub`2~|>j_{abJ z*Z=+J|Ni&?{|5+Y0tXT-Xz(DygbEijZ0PVI#E23nQmkmv;IoVxGiK`O@gvBPB1e)e zLuv9P%9JWsvTW({CCr#IXTIFH@gmNgI(PEy>GNk#e?o^6Eo$^A(xgh4GHvSgDb%F* z>eZ7c?;X~xTHl@P{_6EBm$4TIBY&@5Q>LO4E^PQP;>3yeQ-Nvu^GBHSE~3XVb22`!?>}x_9&L?fW>hAb|xMcp!oaD!3qn4LbNBgpr9EVIoz`z*B4N;@sJ zW_I^$sjm!HEw{Ov@=8{&QuSH4<(hjgy6LLBF1w5dn1AiI?aJHex8Wu`?!5WxyDz`} z`unf5*8+?u@l~oG=d^FNYE4}o?pq1*C zC_Xc7a(~ob#XL3ES!=yD*Sb2bGSpIcJ#p1tT|GA1X{)_9+xG2z@j2f_-L}I~e`7Y$ z_sUE+-+lZ2H{eGJ%yZmx2i~vURugUX;f*`~IOI8h&GR>Miyb+;cHccTvLIvrIq0E_ z-tXjaS8n*|=W333=k+X{I_$B_K0C3cdv*Hdw0}#xI^IRQt~>C-3qQQO*uovR<-|kF z`_R6>uKqmq(MvzQa!wPDJN3jeAFk)wd;dN7;S2sN<=;Sqy7;w54Nc{&>zzLQ@yj2) z^`@IIN+$Eqnu#dvZ_fSi^e4ap5-@P};|lW5m%r_NLl8M&00bkDgGQW#ImX~Z4v3J4 zdw)+-76ppQ>enJ4bFZF+)f@KD8m^tpb^Sx#0*nVj|WEJ8#o$< zw(3C?9{LbsBy1n&^!B_JGO>wHY*}}nmO%VzCkQjV0131IrX@J3^kxbEA|ja zFknC#heF08j)XB&@co86$A|-*hM}{;G06Z1%F4mXbK1li-<+32P#vru!-K{9lzAT!s-#jl)7R5 zVbG93N+pm;mb%m*&-kSbBj5&qhIFJQ9jLSl3P*%8w5wkIYMmU~6^Qn7IVfd-PM6|O ze+)IH41mJ(pz79^`s1uL>>@1vkqNiXP*{1*CKzErO$cTprf6N*1Nq7Ye}7byhGop_ zEBML)4Y*XUNp0%0w2H}Eg0-`r{p^3ln#bmxwTn=pgbY1u3`(F6(0Xyv zP<%3liW!n%DOlSf=&;HuduguZf7vdr8cBiEio zkq0z}D&_)**@wV&^M6o2EyPC?CClfH7Mu@i=T)4l6UPoHlsDAG( zGui5KlN+?JE~|3RnuMhc)xU|&b*LeD-Vi5S%gb(ZBui`T6lPbu-jgmin{(^dX+JWO zJ_dNV3rz2XgW6fC7I(QLE^&pPJI9?lIJJ$!XSVXYD7)@A!x8OTVH4ZPL@5Km|C((r z2m7AUerqO%;X@;E8ssD&dBV4yaY@H`SPz%;#OL*nZ#=^omm%i`6MsYf8Q+L!&_7%8 z=46=xL`F)P8Bozb9|gqirgH-p=t5YPI)zCrcHGrsYTe>~)qj$Fidb?ePRIT|x+6q-*~h{Vn~+}n$GZ?B!q zRX+1j4j%WRqFyLNt$%xR_+J+Ng@;vZwx7!eSpu+9nvqL31Tus<;b zgRn#bi`c$0ubc^ zQpF!ps9ppJ!YxpOAbFjzl^W@ZAE!aX4E~itBk&i6y_5_EK{8z*%gNrd0h1`)f*^FD z*BR3uwwK?T85*fBsE+=3`17BIbz|G8HQ86MCvV6`Nm0YZ!u9v~BXk^?@VWI<38 ztkwRd6rlNoO@D!)y;T#w)m-~YR-p-;TfHAqpdnwm(iZ9*8#Pu+r2y7_;jal`t_0u| zriyUgb=*7f zmmopGIZT)&u3a_}m>|KxGF6#>DN-2-5+yC%k$GYS34g*8Jz~8{;3rIFhORgjVjbyRB10`?)48TAL5CI{jBtpU%OghU;8cb0BlnvRZiviKqWp(C4W}d%2o2qS9ay}Xr()HrC7>JSn5k! zo~81TI(n)G^OL88k->@ceJ|}cW zr*uvybylZ!UgvKbCv%#~a^@pSZl~}#Cw7h}d6uVno+o;yXKZRGc;-lVPRo0?r`m|8 zde*0X-Y0(M=XtWHe6mPhseJ51BB*Jd zv}q!kLumf!L*ZDF7OJ5hYH%JYl3EamCdNCYz?BA~8bxYU0Ffz(Q&W5cq#EQfUFoT5 zR|;UNX0T`{)MgskgLWuGCv*ayqJPRhY(Z){H4uz_+ zDpAxyYp$t$s6%TC0sgK+Mz7{VF6_f&_$m&_>f^*HqFyVuUaO)ms&b^}qf)9;7y-CG zMGlPXQk21!X5_bqK%@D?IY8>yjcTK62C;tW7SyV1@@jU}tFfkvuHt~c5`TrXwx*a8 z#jCbvBKU)}Qma4kD^dI_oXQ8S0_?Ir2D0LSvPSE#R;!?n6`^J;$cF5pZY#GUhqpp% z2uP|>a6t;BfXW_)xTb7VK*N=Cs!*WomA0!t;OxsD#)Fost#-n)Dg&%;Y;!rtYc)Z#!f5Q=Gn)pt>}(!g|cniZpPcn?0-_=6E`W-Gx*cO z#B4o%W2r$Z41A-eh5*XytQYETHYP@Z{wof&YEh^IzCNu#*aNQis;~}4v*v;_Xf9D; zgR?>}@uKR#uBkmhuJcxJ^j=EiBCkIfFTe^xYpO%j;y@ZW@A6u1u_iCHBCS0rgR_3G zQ>a7r<^u5+#XiulvwvzX=N1L^I_v%xh5za=_a?77)T**>uTc!J{wBrz)-S4#EytEE zkdm$jZ?Jy;n(paBM(VyS7wE1_Raf5T?J7b8%APPCCdF`tK;I5U%o+ov@~$9UX_f-- zyf%hCm@f{9@8BXVn_V?{x2^I)#AXiDlz@G<}pVto-V8~r?LCi=HePfG0Udm+Cv+|X28}l8pCF*?t>B= zbHmD};D+;Sj&nE5Cc`#P#nAu`$TL09vpv@{KHsxG=YKOlW9%#g>pll`KM%A(7c@a1 zv_b20CphaC)N{e+0zyZ0LQ6D7Pqal>G)7;vMkBN(UovB4a=23JyIyG!h{7-oVcre{ zcI5$1h5&cvK{nPyOCy(+hJX+-L4PFYLK`2on#+tWa$AtzyHj;wse!?84eZRVOP} zWB%`#MzZh%GEzM9o7$=4LT)+3fwQK8Yf@_f8?jO2HBbwM@E$=VcLE+eD;(@K9AvdW z2)466aysX0Vhi#+uV%;Q(MESRK_3D7IXMM|-qlfOJx9awuP^KS6;i%kY){ZZ_4hCY7tqeknU;nUBn6>&QZ&KswcSo+#R<7_Kv%;RY zQGbB3z}hORCiCK!H|K`6dY3l7YA?QOvGXQ1V1q693b^=ULxCqRzNP{94z;ZM@ofHP zw${Wm4RpANe>jMTxQLH9hzkMJCT}NrxDXt5Av3 zkN3Eb|2U8b`HbhbZX*V7BZY70t&|x96n{X0uWhnY7{iqkK~ZQImL4~|8ijCQx!+E= zV*IOpGsRH1>Q;|8)F!oUcm<8Cb?(>H!I?t3FeA#e4V>fHvbM!p(cQ$e;YGvizxsys1OAvv#S7BX-Fr!hgq4JkvKl zkK6jK6GpCY9=LWg3%fz<`2&baGVsQmBJEKbz{@O##f4)<9LIV4PGAwYut3&H|c#gyQK0vgJ!+G*DzvoB4^nWeKV!iyjM*@GjH zB*{+x)a@v_lB5uO_Ea9Rx9mRb$ah!J%4%f-i1AP?%OwLqB7A?yOwQRw{PLLRaVg6k+^EN2w^+dsX4ne zgs6kMua4T2P?fPgY?STBopl#;EX+68+C=u8?(4Lc@6*jRjtF*I`EF^`r%|U?z4~q@ zQP9ec4g2nEJ+4}D=hnTOcW>Xnfd>~poOp4k#*7Jnmm)-nZ+}sZG=F~a+^ak}nl}&P zFuL=o#+tiv7$kdn^C3hzY}DLTaOB1dDl-W+PUZfNPX7T6FqeJOAnAxi za>2%)j2vUoA^vdiFTe>MgKxN{;8?1{j5--fCx?z|%E5_>;s~Mqq_RhuL{21Wz?>49 zaKe3h`l*hHWPc1$B!MDxsz%HTx)G@y0klWR3PFqrq@|R!(8(vEpz(nivKmFn}?4mHo1acoZ)JkL@x^n5n7daDg(;#L3OLIkR zlmXHlWw4=>GQ7%Es1r8*Axy2#LdsK_J5kbWF-P|_h=0y4K@C;ZQQvY6wr^w$tG3&s zY}M6QVU1PRS!vzrz2KIU4mw?{{OB7fvYW2E>7E*=uV_KMJ?ZTC>8Y+7r|CM?M_3$m!h>N&K*& za}C<|DSvgu<%mf*@Z;7Yg6WD0GY)qm_|A-)T384nw!rMietRrPVsw$($-{~r!Vun* zQI@i+uVx#oth3T06*VEuX*0Eq>Dpo~A=s2OFc7&k)6bA}RtpZy_6bBSHr4ry2AqYa znXbY(%ULbK<_wxN$4D~_(=$6e(@%SJ_Qhkij(;xbGnv_L+ikZ@jqRM-R=p?XyYbFj z@4fk^>Q>-#^$x0G{SgG(k6_T%AHt#fhPfOZlINCT{wS~f73N^L<(6}}Ywne3on*41 zoif5<4399B=a?`uo4?@eq;(mlzL|Fs=pZ9R&(+IRginc^h{ zB7bP(qu6ywUZ3n)Y6s+(`f1~b1d6`PbSEm8V-}Rwy>!(c8b)e*3v(18E}4RyOYnbMzu*j1T;X}pIYiS8LRCK zX@%Jc*&1UGY)BA(B{ZRy;)bfZsY)KUv430oU>HLg&X9&v`P**-$GO4jiW1GqK|L-4 z4dw{0JJ5(qu2P^wr_|$dAE`&;1oxX87RMgE8~zCLN|%O!6|ZC@q13(d14i)$FLveX zqCmb#5{1doL)rQW>&*B&5ycT?NJ8CX$k!twNd%0MsbaY{_D7^(DpIVI2)S^OhJP}t zgLk9rN%RDXFozIGNrJQ%(MWQ>8g3F+>#Nc`T!y8Xoscc;@`Y;nazPAs2~8A4o5t*N znEz=FfF7$uB5;YzIsoZEba7e&>+q?c*>8gWs7%`S(WkcHESWS&Wm|^n!AU9vfuywN zHBFPkRMDn|tm5Q1!5L0*Mrn$@34dG)h>%WorZWeESe&k)Q653O!a75Q0TACAf)td2 z3rYluvYr8zTtSXHk9(mMrDY*7HZK>sNJRcna3H=rIz*62X=od@*b^S*38Gsd6UgwD z28&b_4i_bf7CFfz(=pGBAK@1nLu!%XO%x<;+`^bT(W1lHbWElD*7ler1b>_cg)cts zX-Pz*(IdcNm&t3%P+RI$k|3xPRxIB*v05BYw(LqU!{%Dpq{_A|^-~0Vi!6^IzqJ_+ zmbLVUIrKtHB#q`UVufY|X{o?jmQpm8;maxc8mYpBh9`jf>R}P9%@uOfg;{m%V<8*a z<0KS^LtIvN#KX8AUN!{7P=Akkhy{|(QbM2xy~r!VBb;&;M;YW33eP4fu#dRzwLog4 zP}YT+qs+%lu3d@uXgIyxf<(B$eW@}2S8|e>$`*G*fm&T~%a`r>Lr6AR3?+r)82oe( zNY!OYWV~w=O-h!$R@rL%M2T3wF!C;JQU<%arWjvFvn^@+UtQg@D1S;lB^S>u4SlO- zOd15Dt!;@xU!Cd7o#B-RF(}kdaDrF~lTxvC>*hhrn_&%am~X4y%@aABojl}|Shv8A zdAMLiht$IjmJRJk&cPfalr{v&x=0zC<5|!_X+c7{4D3+EliN10Ovw}o`F3y_-u)z* zuI3>Aa&#wK)JchB2UlPuy^!IV!HoKr!r_ zX&XlwdHT8nFZ8RCnf?eyGqX>xUUdXx00*Hjx)VV9-^G!hY^5ii=}T|)vKi~m zh)^T16DiA+2gDYjv@Yo$O`T%ESeomVc(N9a0@vbf$?$gA67bYKg&h zqB-qWY%m(<9bELc2(y^YAXkui7r+Ap7Z%x@m* zbu;_tL0{f+ixXHW(Dqb%pig-WXY`?W{p(?`VYItX?V7j!p&2M(M8_FKgtivh4lO^! z;}y|)uYW7wZA}9~vBm9|NhO-;rorIO9+@aS#cy5DoDV5itgMQ0ktr5gjorgpiZup%Exy z^`t@`D4`LI5I7?K@ew`o6F)Hu2Q3jv@d^`RjVc8Ti;gbvqzd(;)y!~QM2;ZL#0vQ$ z4@E8z2rk~Na24MK7ey}49P8S)Xs(FY&0AtiDmr_LM`@**)ZBQ0LvAdr2Pv{7`!6E@4<$`Uc8#p0X*Waw=UqVoO1 z@+--*EQ!M^w=yl&axK|%5WCVW;eQgV#uE1CGA`}%F3oFYK8fN+u`T`bF99b2Lq} zG*R<3RWmhNb2VMFHDU8LWivKub2e?WHgWSdbu%}4b2ojnH79d3g_Bk?(|_!Wb2yRn zCoQr!ne#WDvpJ#jIi)i?sdGB5vpTW!IfFAfxf4o`lkC2;JH=BXJ97oh^E}ZrJ=JqP z*|R;}^Zq^IGd|^WKIyYQ?ejkIGe7lnKlyV$w{tuJ6zm!@JOi{qLs2ieiuU?*K^e3` z9rQsVG(shGLi6)K4fH~DgMUD+PD3%YLw~S5DKtbybVNzCL{0QW)zd;fbVY5*C%w@{ zSu{pj5=2q7Ms4&)aWqFuR7GX9M=ydysSZeebV$jLMs@T^ku*t_bV<*XM~PHMg*4)# z^hv1{;*69@u{2AybW0c1NvqUDrxf49^h?Q5HCwfH zTe-Dcz4cqcHC)AYT(h-uC`WTNhn`-+6huJ~Ml4vzDGvle6ih)~(RFgnm0b08U-`9P z{qrn817YF*?-kt;Wb|8wO;AzUMDtU zK{jMXHeyS1WKH&DQ8r~&c4b+%WnK1VwRK`EmYy#5T{9M(ao({Ne$Lw7aMXs=a*Ky*oWCrpfsq~;T4AA#XbTW=Jyut_Z}?KAKu{>OhMbeVHm8|7~X+Q!54j@@_a4Cdg0H3 z71;IE*MB$Iw{+e2BJx0VHJAY|pjj>=|2CKbh=5q?K?8W#AId;t0ia+L zdo+O~IQE1WPZ_v|sd9pQqk?t!f(yb3URZ-O;UaeTg(X16-r)jxmkTs_g@<8?MOOn( z&qrxkhY>S}jRS&hSc|ckDS4PTe7Jjo_!cyn3x6hu5}23)B;bJk!4x#u2yVd`j@b14 zca1d|24dKSIhYcR;T51jgSU=IulS2U?}oXPiw#-q!gx2v_#?d6A1Fa|Xa39~mRNKr z*Mu2hko^G~);K*bS#)z?IzoAJ3!)K3*9fSzkQW&|y?Fm#`70HoqZ7>F}L zbbs&ZA}9e0`ss~D86@DC0aQ7ZVc?)b*@OKT3ZS5fpOlqpc|&7aK!>-Qr+I8=moK3i zN^sdBB3TrkI1h%wjdOXEucR!#;Tc3%0_J9oxz`^;K!bxI6YMvdLsErCk`c4nA+ebg z|5++$xt}d{o72vAakqU>IEY^u1_Z&B-G4xeH+Y3F*_}TkibFU8xIh%D*oyOcn>>%8 zF*l&ajii}!pheo*Y&kc+xga9>lr=eoxxj8hSE5z;8!kDSK>`|*V2J)Bz@!yZno~O3 z5IH$fdZ^9RpldfulJ^!A`h5%HEXp9J8DPXRxT?`26SP{ZMFB0qf{9bWaphqc1b?9i zHdq4gnMH*fsr3}7cjKrJ*{#_cag|z>5W1?LB6)jvHa-G8L^rH=mzgcujW0Q@FXA4; z;v?kYhy`0l*Ltowtf-L_t|42pRvI_18m($0`N7&JHq2G$>*VT~=?9|D;HQ1>E=*|6t% zp&g>NQ5Ur@f}I&40!+CeYC4`B8@Bh^adrEFZ@ag{n0EfMJa)*t2+<+w;(Pjgmc`InOLXqx2J_%$5LFMSKPa295Z3uDrCIsru@le zsK#;Q8*afhGT{}{+9A%t+JB7U%l)BXp~0N1LLQ)D&7T3ymz=>#(#l^?vWF8TM+l-0 zgyHl?8E)wx!4M>J;g;%P3*~$_u>89pyU@dl&Iy%)$p*J@{s05{#=F??BdQCy6kRtC zUAGfG(>duI>){>N`B|;)9XeVns@y5h7eXLCk&Y2>&WzA6qI53F(|UO|Ro>=hc}HBS90Qe8^RcgTc_P{J$7^oH7_!rHqXH?+O7xd2>1qrd&$ZzmG} zw-K&eDxRScZo$pJBg!eV6NP_EAQY+Bl8YzItrISVCtku(%njfw-hu!G6+wdDd+k3C z4y0V|uAzQ20oXOvTU@AT&THanjp5wd3+&+rr7oB>)!M^^n;uR=?%t2z^A+8g> z3)59<3-$w!O5(>n#DolTVt~;OuW-(Q$ zY+ZIs2y(%<@CD)Qj3uP3>uV?ypCJ#(9V*U2a-|&MCo&RehmI&}3$`E)ZptARiY0ze z?QLQiTU{U!Zq}{-j39mF;oj?U9W5r`AuvGfW1t4SVTBgHq!wS^sSCFx|ML5tdW~VV zslp9VU+*O{5;aO7xPE^pSRJ=?C-!ZhdltXaVd_H`p7dwm^c~{s*%}KpKL&hZg;Z(= zL*H7?%qDzC_|Yd4jUlOW!GO>j;!V9aKvGCvcy35@jU>W4Efs-a2ab z>HYgRQ2nV#`YE4E64b6uUk{GI4(6r{TO!^70*63>N0h1SG*EwFE`REFB+R8w*^z|G z{8_w+F{8$f96Nga2r{I|kt9o+Jc%-;%9Sizx_o)jm@$&Z%J{oiFQ?9(JnP-_S7zo& znJ|kQJ&H7GOrav}IV5OLWW#|%qFPjz@Q6L939(t+x23DUTv@Ghh1xOELw{#+RK19G zpjM@H>)O4GH?M!*m-leWllLz0xpUvXK@*i}uj0jw8#{jNFVlz^lxwJahb>{Edt|9v z9h-2=KD7hQU2F>nvg*~WTf2T;lIX}`W$OLCjXO8){(5DG(PRocxNwlNn?A)cH)Lml zoic`n+BOYFgaf@U$I<*^8o8e}l2+(1uE0bX!EzstKD~eX^(yBwbj)Gbd*keZDRi$7;X1dj#m1?fZ=3Xd%L>qh_&bgsH zz5Ui^o*}*Ikv&3Cv{73-*s~)>aCDX9MLQB%?zWxgAIq%##;;okE z%8_{krKZt!T1je=jW{01tVN5iWh7c0LCPa|up*bHvEq)~lBQ3}CndS+uIo`IqI6L=18sbfvfxe(bIj((7bSf$-yEvDsrE+$4XWY*hfiQK z=P`E*Zh>>sO0!lmMhZ6!^*0ZT7%_=U+t>ayM)n{$P$E|CYi30%q6t?*(uK^ib@=)N zWVrdF%+{aPDO%Rmc85AMujtx*ci&0sykCFMJu4*ibu4n6nvE4*RL8P@KMuJ`SzGjU z)C^CJXXNTlPWPEdaP7E7jmwn_HUWZ%_O)S_>~^%++K8R$W}2*d?G)zCX}Y%WuAa`m z@I1H>>J)CK5T&0Ew>f4e;^LvJ^A0`aoL6r7ub3Ab{cD`RJ*z;2UNm<1qOnD-jw64s z-F8~onsZ$DFJ}+`dbscHY5ev#rZY4(2u^PuAl~b~Bf-vdt-GHA)x|kuxIu5#^VMM9 z;RdUDWP1lh3I#2KJkJg6D(y4V7OXP-IQUf3x+>M>R6>m%wbAp6p)t0JQx%=at>4E zVHkR=q%$S?j$!1%6z9-nFLSgfXssJyBCPf|H&I7Oeo~ zD_i@D*v7y$v9_zB)aq)}yZ$nEvPU^=Q7l_b%u;rk`z? zQnjXSEt3jLvpl-?YN3CHEJSCETjKoHO}jlQYjX?Sj(8R?ln{h*kBi(!^fn`}$U`Ft ztCx6+;v>aHu5q0^jHQA^6vllnNY2rS957(K41hrd8!HNEILx*>46b`I^4rrcccSrq z?|X-P7x9vJzx##3eElKb2z+BP0UmHi^vhrJMj#LU?8rMVKwf{UaikFm-ft4#p}E@D zslE1fZ+%hn;XQ`9!??w-U4l^I1V>=LC>HQyT5MpBpg6?}enm$_v0s~XgbVwP01R{9*>UxFb`x za+WDEk-(|-rr z>QyJU&`LQpN6r!CF>Cr3ueb%HEduF_oCBz@Fh-^|QV)M<@Ol{Z?8vZ>YT#gN#MTKT zZzJG5UO~KKyfE-G6vnH8V`oIaZpcVIp0Vs-7cA8rmUU@kjnY>O6yENRE3K14ZH+v5 z(HOZ!2>43`MhE-;8EGDG1l%C*dX##@ zHoq;h?G$U^%>bq`7vM`{gR2`OJ;#VCz!4w z&pD?o&NuRQ1IE4WI^FqEW_*iCciSRYAFmN^VGMtvk2eL{b@Ve7{6te=yu4_o@OZfZ zcu}O_(k0!un4d_S9#8!uRTnnUF@kYQSmYL$CbC5Yuj7%B{_(dM1|=XIcGI(K?W3l9 zlWne)x!X_o{f1Vb6fPTcqO*5QL^-Z*oU}JM2Zb2-qcUbOyw5 z@?(D%{ED5X;LT@$()tE@jK5)N2^hE@(8q`y#_M&A$f-ltA1@IwVtJ9TJH6}OaLjw( zSG;Et0+>An2!I3idqn|p6&3-L)^}T%5q)O_M(1MCU}?#x5nBdPqYlz=d?fGS2cf+q+I^9_d=cGU!Cp_Xx8W`%@z6UQbG(4c58rh@85awX`2GsqEP zcoF}Gf&LdEg%MDFGv;!XlVUE0V+OZOvNth6c!akH8%DT0ez=F%l!PxaV{TvyMU{UC z7wCo-@q$t3Vqn&H@rQrr7ZZ(O1k4wH{!oK^rx69_5uIldXy^~aKwcLB1E&ZBlQ&*@ za1mtye;M%%Nx%iOIEx)OW@pD=Fdzu}_j3IASpLU$fe2PZhrbdiRF7?E`0#&uDLb*DF8PyiC#7h{cp ziTlL`wI?omh>TxFjD}T=_c&R~*b<=lf?lU+d4L$BVvs_W3H=0#uULMI_70c`1(`4m z19tv)E=F>kIBwwxj@ZW$)3e08{p&h~klCJJQ`kF2h+69#*csaxQ~@J zliGBQG08~&$P%E~cY7yzafpqkSA~NoZvG&JglC1{M|J$Bg4K5sKc|r<$dMXhg$y8i z{s4AbD3vW}d8xM%-Zx+yvQ}!SaV^<@Fo}~;m5+RwmPw|QD*=jG_!Hh(mYT+GKLKVn zIF%PMeOm{1*9d}Rriv_y5nzA$iDsA)-vE0OKxEt4d>lcN2IhDS0F*JIhyxa2KDcR& zXO=IyXY|;XuqG3uNt&f;nx~1Hsi~T)$(pU{ny(3)u_>FgNt?B4o41LZxv87G$(z0Y z>6^a^oWUua!%3XHNqon73(3iAukZ?oC6auilG^*p&QDf9qOSU3ZlcOpvt+N z&B=)@7hZUO6BNw;%{1`9%Iu4{FE& zP{5q7n2BU&a5>pJ5NM0~N3C+1>{tWPR z1v8)pdY~k#pu-TNaVn>C3Y-&4r+12{d8(&-%BOwmr+>Z`LVsLDyGC%P9~HjWq31LIYt{_qN5%4NX8 zfsFPI(+7JIAP9fZ^$uGGdoXYddg*+jc&3~-4Bw`H9ucnl6#`r!3dd%JjOh{giK3Jc zkApW|p*Ca}A+P+kU}HxxuG*#vDyzRLumcO5cB&f(i?9i+unU_TGQqG93;wYYE3p$x zu@!5v7kjY|i?JKau^sEN9}BV}E3zX?vL$P>CyTNQ%OQW26eU*%PmZb{7-bAwMH26H z4$mNd??6u^g^e;HZe1u7KKL5SCRD!mT{vqDn@TQ}uuoX#l9j|EDJ!*8OSM&NwIOS; zSIe*(tF>M0wO{)eTnn~iOSWZewr7jBX*;qjD@iT;3@;m+O2V{nTV?AdT3n^Jd8@a3 zYqD93wqk$Fw}BhDfGfC#Yq*DtxQUClY@0hS%d~NuBy;Pub_-cHrMQ=ixq0iiXG^%5 z>$xf0xt}Y#qf5G_Yq*SyE{+SckV_$wTepvOx0Y+VvrD@kySZi?y0!kRyAhkayX(8Z z3%tS0u&C>{aLc+S;<`@@O&cb>%^SS8TeiLHyug1ez0+&G*NeS}OT31my2p#WP5Zh~ z%e>i3zMc!cWLv%ETe|71zU}M2?|ZV_TV-$ixZc~k;H$i`3%~mtxaT{z?90E33%~(P zzy)l;xgo#ZYrhkcyp$WY2Mobp>%U+-z!AK+6>Py7tijHk!1TMi3v3?^47-)P!6kgM z6a0U*7;M7PYYZ#w!Y|ys9c;WHj2|K_zIY46Ic%{g?6oSa!&>XZKP<#UT(&Y?zchRw zHoUwzOvFumusqzgLF~jRE5%c6#aFzx=hCEq$tR9j)$eo32uDMu0aX&WMyEi%-QU|%?!HNjJCIM4pb|R3C0DV`^@G{ z&m649smsmY{LQQp&Q6=ZlU&d5i_TQ6&Sv{<8o*Cd8)OWY{u|#01M*zZ0WHxG+#zkt zy;mj*lu(rWyc+dD38LW7cH1Enea?SVG|w1&(ip3%2(8czz|gqi&=5`1GabPd-MClw z3X~wU9IYC<{0boryCY50ju#97jeB`x4_g_joF!v*)LtVkG


    D!A`VHW7`Pw z^t--246q%*qHSewYYaIJqe_2;+I!Jcu2-gq-P%HJ+`cQ@Bum>0d#1NN*%wd-I$Xi4 z7tdb{l|$ycMc?&p-}jB*`K{mk&ENg)-~SEZ0iNG(;06M2 z;0KQ28ejkhFrfG?-|`KkGC{X48Pub^-n47W?_d}r`lfF>v)}OA3%h^a2z##H-GwHr zs&>s^oNc%j?2Y05wbKa$Ax^q4e&O%i-t+t3!(e>zJ>LipLZr=~y2p`>g5@+bFK$ z6~^M7edfAh<1r3llj?uFbB^WZo8vl8nj=c0e-7w@F6e_!=!I_RhmPopuIP)-=#4(; zOzY^8F6ol~3NVlWl!@p8OXcG%(iWcQqie}J%U!Q<+8A*TuRx&mB(Wo&8);5nS$(qK z76QB+WJ%BnN-7B9#Q?M32n#L1ah@A^-nG+d=esNHpMJex4c33pP?~-|>CX=B(Jt-N zPVJ9Q3Du76&~E9NZs?l6>EVmkG>+_-Ysue$>bw{d&!BDJu+6doZYIj!(a-gfK&;qYzA8X@VG(j;;tJTHVg&IOt%qD25JnlJIjAj zR}#P6rf>@heXxJ;Uer-n!%dwE${L#5A%)=>5BfZrf^)DK%(50 z>pwcE2ny)`nw*zpT$I4^f-dxsB&+@|kPNkF=;GVG5Yk?q3V< z4B!S*i|;Ni5B4q(VgKfWyYDxe8-nnj-1Z6$zj6$a1J{2%49?1FmW}as&hWUQnB@fx zl^TNb;GR8334f7v#f}?F%FuCt0SUSGZEvl{U}-RZ>{B1ZjO#1Ot@10+@-grEpAY&i zKl6#s1qqM=5{{+|M-p9g@BW0(G;ln?^1Z~8q@^vanDuD|+} zUPXvt$5b^}dgTlbzL4*v- z2pnc`;XsHHB~GMR(c(pn88vR?*wN!hkRe5my!e04q{(wB&wcv_&E+$g#$?JIR@0`< zTRC;^`=;;P(kXwzcPTfgq ztkeh$MCF8$KyA;Pac9be`OGCWmi{Wwois_(?_a=y1rH`%c%whWi4`wq+}QDB$dM&a zrd)s7GGv(?hn2aUxXdt)nGIj$k`SSr(Gm;75Ga8`9(ke+LLvEv3Me3SR?g24-ohTU>=Q1`j(Dc)@ae=ve}N@)*<5C#2U z5P{4e4tzEIU44UGOohG9K*BkJ1UU$X_9%Y|0ilO}n{PJ;8+7nN2qUbBuXprfsil`< zn#-8DI`r^E5JMDkL=sCh@kA6;RFOocP<$$?92#jP2dki;9uqGXs%pi(QY6rgJ4 zmJpzlzzs?S=^?7mhIyqbsJQ;PF$t-D9HuQRMcmMsy6(DAFA60Tb4)VJjIc8_(^P+R zO*Yx=Eu&jFL$i!qLNi88;Y?!ywFg&g?X}outL?Vj%mnGpj1&~en1lEf=$jM_5>FmR z4Qd1;X!^@Up5H5~#dMH7d_P+5|G?AR|5vlz~Zak)pl|xS z^HeS(Vh%V?MK$zTWRq1^FbXS7>B4`zps8z14p+4HT5Pk`c3W<1$|MsPnY<}TA#c(E zDnV+>(T$r#f$D-=o`ecqn_kSJrWYC5(I&HeG(}yv{tc1ZF1^evrDm5^cwvUeZ1Z7= zBbL}QL@$cdpL^=jcw>(KJFe#*f9SMR!aPlL)ImRS15`E!XiK@!hAXQ0B1M1YDZ?Nb zXr=U^I&WrRSBUa32)5r&-kDfZb++hKn+OjmxWm0lt6Q!@22#DcUbp3X^TpOEfFfn(MnFp=muY2Sy10~GaYw|YVIMqiG&VX9Q;3od zFu#FTWgu92xMm;%{x@y`Fd_koji@_fYd(E``s=r-S(PrtP+GY=_y2!?00vNi10-Mp z4R}BVB2X0VA>fP3u%QDLWIq0`!$HOetUF`q|IM zRJ5a|%rtAdlANY?FFd{7ObuhZp9WQ^%PbU8mncow(#%-Q+muA;*t{SZ1Zz{&KnP$n z&oa;otLA^iNLQJg)pxveIt95RuTVNgtBI$5e( zy{~lsVd!A!S@CArmZaryhZkeoc%nA7I9=^9F@k?okU%2>x8f>LlL?`r0*|)Q9A0iM zQZt{W5h9gZaBUc11c|^e4^L|<*TP9o>>)X8?BptQ+vkSs0C^x>tnQ9+_70X!r)>Rk z>w+}=WzhamFCsQCh0jZ0G^bh3Yi4ts*=q_@+*e2Wo%4R{9FxXrAd^?C3n#fyQX|Z{ zNC|%^LrF2mfMG<~Es6e|nJIkJG5-+Elb)f6EzMyN+jPVv-l>T%g5r+|XvL%A;)`KC z<7UbjnouMiQftQJBNdj&)sY!E(e^&-wy@;Qqw4>$JbY7mi6 z{zOqdcTd|cPV}Cm5RB%&~kLQz5Rb}aEDvm<0f~x&3$fkgZmY)C^zOP@k)2Y zTSu?V-~>c~-T%zdyA3C#X!Qsx5g3L`XhiUFlen`O#&*L=cWH>n)U?SZ^QQCi z4w1E%LGv6nS8rXW7`Lpb%si@%D|I}XyAAU0Xu%KdcTxK*@naA{ymIPNqC_6fituMl-3UzSIl`eq<$`y6#99a9)&33N zu!tu8te)BK1Zse=| zrb5%#^v{0NzvJ8(H+{RF@^GO9@BVz-&pyhO{smslz#4kvxca2bV(#^jV+O za}D)CAB>X@lv<;W*rKjcqY}&-9gCiKf|`kNr-aBrXEQy8XsbneK=!*q(*gb_^cz0s z>p>sQkXnf@T*V7hKTPb!)G$L}jF@SH22qfl zP3**X7>25fDvJO$j`#)%$N@pH!(}N0Wy?c@(0~PGz!`c48Tm$V1jn`h@(Z;i3`_Jy zbW9daJVsq?M|XTjc#MBXc^orUoJUV|KnA$Ic|0Dd(ZzL4M}QoRU>r!9;)ruFhDOMP zOn3%%jI?LSgglTWK|Hbk^9z<(n{`yF8lxer$-@h9M2?`1uhJgDILClINioStl(0vZ zY)O}V$yKaJm{bY8aK+cKM}9O0Q&dTiL`k322!aesG?}J%_=bOBs5>s;0w`EYrfkZj z%!3{IhMtrNXZ*Wpn9_>-R z+)*D5QjveC(eVUQ^b}G#L{cSP()Kh`CVf(hAky1(Qjeh0{j^dj%~CB5FD%_sCY4g# z?9w7i(I6F5FFjK(;fv=zg*K#BU2xRQ#P$rJ7pU>y;Bx-)4t48ity9f1XMj8 zR6<=uIW1H+&Z zRD$$WPc_R_B~?~!)g2pESM^a-9Y|D#RTo{=Sk2W>#Z_Ik%2^G@TD4PN?NwqO)L|{w zfc#Zp4Awe5R%CtFGj&#I?L}oZLuO@DX`NPWUD9mbRviAU)-t@-H0@SzE!R%9Rdbb4 zaD62|aYa*dO;>mw(RYnkt6Nw1Yu9K+S9;yoNS#-Hy*7JQJii)Pmo4K1A_g_6fK9xO zm+%h<8GqTC=-7ypiF-8$nGIN)y;+^zS)T1#pOw;_{aK+MTB0plnW|Z%bvDVqC?Y7{`5F z$(>xv%~ZIpT+Gc}Q;l4}yMWluHPo!VlQso zEIz|9UY-B@hFo$DL12v4z~V1%V-D_OH;&^Qf@8ZKfhh#6!Z zx<*f#m?Fl;-GhNX6=hI<tU93X-`Se-^t!({$sYGxg4 z#^!Q1;B(GpSw<5))?;0kldx@P)uA#yZj*O#4GmD|%E*<2NCAI_n0?MiY_5lOyM=Xj zV|z{=d|qgFCg6G~%4cXLsjP?2KxW-}=5w&2&{Ji&3)6RoxZsL_d}fnfgMWd6mW+B( zHVqKy)JbV8ZFU2p9%`b_13_pC+PDl-Mk1BlY1R2? zkRIkt{#BhO=8nNzndWIN#%YPEYO&5=&aDSxY=l?n=*Tz+ZxjV+ z=<2`pFs2Ua8~Q({5I{;OKz~Z$qLrfrKJHrnsy-j5PLo}OX^L?O8%pe%MkA~~YZfN! zh%sx)=Hl{whp;%}%6Nt$q2kJ*(g$2jiit_M<)oy+!N*M05Sw#C`*ZQI6ey6$fB?rkRtso-Xe;Vy2(IBtJP z?($Y&&iw|K>x@^Rgu||54jb)oE^R+PZ3Xm))b@wfZg9f3jP{ATgvgn52rh&m0=J$g z8(xfWNQ?h~<}peLD1UJ4#DE43ACPCb{uz@-Zo?Mvj?9ltffavlgqH(0Y~DX$LyUS* z&C^ilhtBRufoaDUpM)@i{?1|7t#I?XvJ6iM4o3_RXN(Zn8WJ~+6JHD!zwj21@%;w# zCHL?0eTE_HjC05X&aUgxLhxLQ@HBpKI2Y;5;BkbwfK)z+9De{A_#pzu;2JLQhu_c> z4WMft(~}E$@i15Pf6hb2_6KUaaE!KdH=y#}3v?J~bg+hO=e}w?ujw&A;U*V9JlB&w z_lG|Jb7>xQ+cR`hS9Jak_3q_dW2kgAaf9-0VmE*D1xIazQ0-oq@YbeqgfN1duu1RW z5GlMSe}D#@+kfeK;^h**07@{1SN<3lOy0Hgxuea%Ks}Ci$QX19sPUjki*B8Zb_(IqAHTMUAR&sx6AE;4`L5KB+mkbRT55=H&n0ELGQ0T-EpDRKP zt=<#G(C|!H3@iHfqUZOozHT()coZJ>m-h#b5{$$fcPd*;JHi;807Hh&)7jX3Kia~_t2j_Xn77s_@*!jgNzgXy}Q65#XXQBNh%CE zvVY{qmM&kyj45*_&6+m;Z{p0Ub0^Q9K7RrYDs(8(qC1Tl!zt*XppQ!#h;n-HRZ5W& zpgn}q5v#ut2N?v(r17Ilj~~xT=s1?6S)*>>!i_6;F5S9z@8XS#^yaWK_2j(^EO;>C zck=3$8Fq88T`pfAAIy{29~1^K6sB!z>VKfd%9s6Rp8S)s&8N4dGN3x~scL_2m@0gB zFv!DUkUMe_CD9+W2$orX%%OJU@S36;x>Si$>uty=88UGA(O4rR8X%ixJNBXSlHIGW z6fJ%{`SRw^qfZ}~bWPK!TPtM!@HK;`{$zu#eLF(#-MxRiBwTjIsaK$Z2O^lDf`8?) zH&b7L6&B%O@+7uc{!(>4)tqFLd4^ei{%poTXO?+r--6R!SlU$Yu^8JyDM<8`DHB1H zL~{`&Wm|YHX-8aJGd*SEMBS;>++!M)1f6sWeZ$<1YY_+@O5d5ZL5!ritZha`gbVTdDs7TT7H##APmW5&lRqe-Q}fCTciB+oE{^zfJj-=GCxO9Ddq zqfDV85(5L={I{s>@r>6SVs;$BR=R>gixSh4{!W*x=Zei<^pMVaUFF}SXhA5wK^}8r! zjy_tcq?}O(sij)h>yozqJgn_;4CDb#t^V8-*hlhYC`wseP4rD#1IikcjwpRZYj|vd zJmfnM1*;HrIDRDOviH4NC4b7$&fBxkKLh<%!!1S3Da4#kT=B&jzdRAgUwLd*%S?_O ztGhvCowe4{4n5Jm`s!Qgp*3~=(>D=hqL9Fmk=J3Qnj!4=!daHRDYs(YZ8v->h45Mg z&*&FjOOA0HD#b)&WVKBqA9PyHIX_iKOG(RoF(nr(RL^y0UBrOn4}Wz&rL*9ryS3@3 zqYiZ4+rs{Lbl$8FDN^4Z0j`jaOFl$6RfZG1Ey-7>9=!0w=e7FRU=J#GzhwXG)>3Gz zEnM#fmk46q3V(QM^D-6vy6dgecTnkO{vF(k6EmQuLbrVN`%Q#P&~k<-S$t61+!-F# zLYB`4C;Ux61qu^kUw_W1q!J_%FM$eNAc9u+y>7v7BH?ot&S+;oZlOTqQt-t zf-rM9U?NJG!Gwe*2X?P@55vzvUN{l2bi@s&ctsS;yNTKU4w3^=+(Hl{ z`AQ!ClET3t+~O5m437cVm;)~0PK0{gBOi~ly-XP`hace~1AO=pBMOm-M#M@IzrsW( zLQ!#joFpZMw>o3E;ZOKFA*%}IhCjZ4B|Rrmi@G%jCKQ4X z{D?y#+7gLAhN5J-Xht^*rH&dkt6HVoK|cbGT(Og#$P4Q^MA3+;$wZVnVPOv%(X{jY z@|NS&o_}a=TG5R`Bd^bRYpI}tlx^OOC+|RtG5GqDZxGh7GI_^djj__f8e=ypX{=(m zYT3)0POU+)D@-b)*UGBRuZ|@rW=m_@^sMe2rpUuEG)dOihV>o8$b%_XN)yf61T8se zgl*H~264*uAR+iBUGs9bFiLhPf&HpJo9kTXs(;7G# ztWF-!phO-ru~*oVm9d$~10|k8Du2a=ohPaXed}xA`zE)zc6moVZfZlJ1h#X*o#A&2 zY+$0|R|%zFIL0y#UV?RUVsoX~C)}MekG<=_dYl6q!+3?SifrT}zhW5BI7fa*SmGXw zm@f3xpHbKamIpd_xch)nZ%c3%!s=J37*mo-Us!ZR!3=SDGl9 z#`2~w?debNvD2U)^_4}K;~s`kmOWnAl6<2u*Uy0xx*ZCPIL zN!P&kHL;7GpkW(3*_A|glwr+mWkWmKcX_t7tF1t4Gg{cMwl=rBEgEclJKR|VcYj*9 zjc0O;JKgGT3A)$q?rOC=l;&P@yx(naYuEeU`tDM`QKapC13ciT7C6B{*l%vmJIn_! zIKx4`@Pa@|MHA z+2?urIcXmAo0rz+IN!Oy3#4;u{!pCfLN{m7hkkRESKR1IZ*|d^&T^#w=ep@mpSqS) zPW7wz1nRh!I@Y_MI<9*i<5nLE*!fuYu%lgdV^2HV*-hBD!#(bDpF7>_Zuh(6J@1ps ziZXxr1A9L7q8~l!OKGppCKZm^f6*1LSiIJ;sQS6 zBwAtyDkA$;;w2j2CUW8@f?_C2ARvE=Vkw$nCa#|+nxY+&;wrl0E5f4w`L$v!(xL*Q zqWY=gE9PP?@?tM?;x77PFm~T9rXMhxqA?0%GAbh-BI7bTqw^6X`ZZ%HPUACL<26oU zHDcp7{$4b`;Vg2a?rCE-fPJYwKG%A+!_dn;ysQc5Q+jyL52o^qZ39XLQ-T!n&3fNq$?_9_+e!6 z-NHGHAEOuv7f@hFX5>ktWcQh5N`B%-g5OI1US(*2dxc+GpE-BcLSgvhY9?U<=4J{bVs3w5QQ{M0no3TQ zVPOj3I|u@6H6Jv%K@JcB6numBi9#L-f?;@OZWd!_>LxFOW(@Ks_-#=Lkmg;M!5O0F z@fm}!kY@QYjt1an^63ahH0SjR2x-29C=5|=a%W;j;BtyWb#~_qD&#wSgI^6pdaCDo zrl&DL!#6}H^1&okk*0rmV&75n=vSzv_f^JlofHEkK_mQ=AT)wN_2W_^Ad3ngk;3Rn_$cswgPL}qj`HZ57U`Vw zo|y(8n$qbhx?FI6!;{udO+15=&Zc|{srFTALx5jS%EMAl3}Y%PnewOd6^db;!!VeF zDYRDa{fX}NDL{YfUZq}YVf`L7+`ZIOuTxv&zFaxGJnZYXYw0S7}g9n8I02X`z2&-=RiF_&qAc$U~#b!@ce& zW=iKld>D3QlJ3=Lz5dgI?&v?Y4<+#4Fc{8b$Y!|$m$-`KFk!^NGM_07OG;^_jPBkm zZ0u|lXLHypUg-~biorA(+ z1jFv0!`^>J#I|X3yy);*Y!JyQ&FQUaN8{`3_Xh0t1fnju01hAGU6sToL?(T7r1VFDTe6Ppm>sJ^7rV5y? zx+Z^2d8lfpRFYAG;lz*QBJ8KS>C0|~DY!uy{*+P~P;dCEZf;iYS6o2v{%%SEEpQR9 z@nS^sF0lPH??F6o^jhgVP_Ol3ulBZ2_kM4W7Vzp8*c0A@>*7QiKqjHe?k&M>*ZSqx z-lXluFzycE0fX=E(Z}EH=q}|zb2`-h>K=c6(Cr!l;#16Q?wQ8!-IN3*r93!P=^`H% zg#fG0k6{#See7Px7E-D0P!qU zClQbCK^*ZxB=OUD4inGP6G!pVva$ZH;|XI-qA7Acl!OY$F8R$a`PJ@0IO^@g*i%Q8g|i%)cNa^n;$DZA|{D`+ZL~WG8g=X^3?+u)N+uPUnGA^vL(+jCGV>{kFE0QND%m*LqP#FOYs@cWM296 zH{4YAZfp>=*YDx!$uKBFAD@nZvhIaQ$p(o(`!jT`3N$CPuYT&({w)9IFEZmMobGQp zPv`;LbMN7^J`-|`_OnC-bYx^PSQNDH9rTMNG);@N{f+ZqNQi_S20A}-__2R;JJ&GS z!n4?7GCd<|?;&)IR~w_4 zgHN-t_yO)?ys%K`9-|huJRg7cziRRP)%6$WUg$PQ;u?dE?8l8NF~#PdK`=m8^Il*d zU+46z{(9`;j`dsT9yC1ceyFuuySB@MH2MxU<;wMy)@f$vulLOKccr!gEo= zu24ccpFPR#1o{pvpS1Wm2BWzihxB<&`5ac)?^Z14P_>8g&U{LvU7x`%KvyOy7lG}8c6E^E2 z-z{(fbyqj5ywr*ppmr-?ZxHQ^d$;<@cs!5yJXS?%k9w+xqyOc%SbJcB}LEJx`$vM}MPVq8qi1FErt%Zs#wQZLDiDuZO#+0HQIF`!ksPD5pCEU~@b(d|!VBtP?!5e9*IM}cg zzx?#}(NsrBxys9bGY!)?c^hOVo9%7LGPsKu>0(q~?(S`*ys6mjX5Ug3FW>2sY;;2I zV|+eXMBygpbMbAEKIq@l=Xd^#7Wd}czW5Ek8>~~| zhmD20fpSNC_W`a1l>CWe)R=Oqmoi0|UjLQ?ckXUG#oR)F$Rcla*1A@UHTTBnGk|tf zt#~5{HaQ0t5^OFy6^C_zemJ1{V~DIM-_oqL-d-?YD`xkIv!Gj4GHhdVd;G;I52@-V<^0{x5DOZksPL!Z$HW$;) z@ia%HZ+K~n3VRwg+RdbccCM+EK^cS4j160j^PRDOXGgoLJojw3v2!@j8T+RD=kZ~} zWh#F*x@Pg>(B*ww{@!&ubJp9rclZ7se0cHW=eni|TedvfwsqzX-rM8v;KOfbzfB&0 ze*OFFCiedyzyJjt&@c03f<}})hI!|*1{-{5m^?%osU`!7`bL5r1ffp9GTch-Gz|pd zZzceB@j=Q-m#97JhiFPGF>Xv&rDgu)FVBG z9EOrmMIBXCQT{B|6xCE!U6s|GI5o)AR%M-4)Etr+8s ziz=AojtzsDE|4|ePUMkIJ{jedIlEZpmRJ5!41bz@5BQIyu!i{Z~XDdC7;|N z#wowNCdD;B(sH^nU-|RUMF+d{(M^|U^VA0{Jto$hcK!6&Wk(hE*(X;W_y1~tch~k! z9q%3Z;Pc&Gc)@c&{xIT=`u%j}i+>*ahnbI_dX|vCelX`f(jI5-tN$MSy1WmcYV6Jb z%lq+LR-gR#-Di9H_lrND{=L$VUv&26-=F_W@$Y|j^{e0go~J(m9&mpKL}1GVNIwD= zPk{@R;P)apL6bG`d=C^}1ufW3!QgSwgB8P|`84>!|Ai2RDfFBQ131F#l@NF!WZ?`a zCqo+k>dS>MJjxDv*h3%w5Qsq(;t+{gL?a#%iAhxA5}DXUCq5C1QIz5osh8{40U3Xu z5shh7V-|CWyfwZNj&YRZ9O+m`JKhnGdDP<``PfH4{t=LY6yy}!c)X+J?U033q`{1_ zNJl;rl980;BvY42N?sC^nbf2UEqS$0ZZeXe1m!46SxQr$QirHiB2XyPvAGtYcN{T7i02 zw!Zb6X-$k=m+Dry-W9JNqicU&`KrmiCUmQI?dxC(3#h*q7O{#vEMXR_H+f`&RFn`! z0~?!I%fhv?m(?r@1)I;#w$T(4NR0-bF~HAm7XG!V&1`B}+sxArYPEt{khnerA}*M= zwYk-;VqUvj-%_@>GfiAVLhy}niB=GPNeB#Z`&;McHnz`|Zd*&LH!goTrMTciu5t&` z-06NdvC#$ZcuATz@$%&xA7VgulT?NvycfO^NdbJ}+Y|}V_r1x@-FeB|-+=PhzX1*q zfc-(>0;@L!??{qp+tlEO7&5p5mhedpJYfs-wz^l#gK-b6m0`3OzabWJi1mxR3SSt- z+B|WJSxn&bUYEVbaLa#fl?l&V^I!d10rxi!GLOnV_4ke zBH0+nIvxycNP*(6T->Jk@gx z9_PUg!4~$g^B@RH++rkim2`k8trhlSTGOcEw23`kYr;U`0usnI(OR8?Qy-=tAwWwb zAw6e?cv~;6hQWWgbHav! zF!C&A0YL-l#!#@w`GR#l=eizLVL%G%QFgxI8|k)@`JK)Fm!_xt+-?s0x$A+tlaIXG z@Gkn%lkS(L>*eikZ~I=RUiG{C_MG3KM9B^F3QBC-&mF6|Q*Ms)?gE|SJZFH+6@0%$kv1sWi6HtYTK3UO%peexZ&G`?n^yVFm^X*T=QWu{=(r?{BF_5x%7KEed=#d`@Ogx_Oi#W_o=_xb3P*v z{6@?<^3d{`7i{tp3D?he-gAG@pW;Hdc$;XC>-&Ez1~{!O%@F z0PQqH!XIEGEbOedP=m(&!Tzs@s1rx&!ermDej{zzVT<|ZsG>`L`3-b=HlJZV0MB(L5V*2#L{|rzAoDLckq6z&0 z8cu&A1Vq6YD8Vfv!0HM^0wsVFjNuhhO(7UC`iuZs9FXjYjLR+n*pf>D6@m+pMHHGZ zB={`@_k!ax?iGUVB;26bUjA^xMvwt=zz7=w3HM?Nq0k8xQ3|Qh3bF7hwh#~tD+a;r zKgz%cm(4J6PzQIg^D@u=c(CFQ!vSXyHllwJ15L2#+`{|bu=2Pp3BIi!Hi7{zurOpV z1=j8jolh^$g41-5%=F?Os)8?MugdnK`9|#y;f@zQt{#YS5(AM4O_32(QO=%m_4vXS z^#T@Ukrr(+8V9Qq{ox+U&K%DX9m~$mFcHh(4@5fA6M?Y)Ht+xBG5;QVrcSF9-ZW^OBJ3KmzspA_JF^1DkR8Hj)9u z(ZAkJAN7JCGw>hzA|U=7ks$lxASECnL6Wb|X&B0&?rx$U%7FZ`M-vBw6AMNZeXt&N z?jAc0^ia?*GEx=&;TaS{0?KV5*R6lR6e0;E!!phyCv~MELe2ou4bTLEDk)(C>= ztsMD6J&aEolM)Yi&*QibCdcdDW>5)}G6Tc1DfQwh-AwqVvNFEm9IWy!`${Hhl2;Ta zVVVvn+fiJ4(kDN$2Sd>@N6|0dz8!YR^n)tC|8+Oi`96TIH7GmG&uS1mLBVKW(^Gd~kVLenLZ!vn(t9>wMUCn3`wh0wK%GBlBL)qc|wKZD-<;W<-7&<3L`m2USc5JXb5Hosv2 zQ3Ez}Ee|_VHH*`^NX|0fb1#3mlO#ie!@?6a#*;pc^QUH#LHg1_`|=##VL9IqSL6Xb zDbHi3lO9D;|48xTbQ3@I;v7V)DF-qM_QC*-z!Zo@SVSRMBrz~v0RYXD7mZIsNHh~n zG!sy#EnO1|GJ#V3!3cH}KG{+}YcoLWGw-sKM+uKZ@hu`gR4+nwL`i>CMo~0KziKcG z^gydLK^L?z%8KbS`5PP1RKI^ny*@?o9`EtfurUs}w;M^eO`* ztiB-vs*FqJ&r2f{C@FswA1!oIG1N@e(jVs34NebF86r0ILfkI0Fb;%_-u7Aj1N!+bps(4@yL`ul~t@Hl~ND%N-tGc znv*ZaY*PX6GorIGtJ5>7GfXA3Fo>>MPc=4M%_%H#FSv{#_~L&zb(JTXatqP$)NW5M zxy%wh?l*&g-MCBwk_^zGfcKo@SQ)U@K+sxS%RbxnPzw!SqoptCb?xpokDwjXJR zVWTcpZL9Vc@Y{?JHFA^d+7m=9k#dLc`D)W>@fB_nYhr&5S0ka8?EGR*EwFRDDhAsR zGV0cD(JvEm2^s+IGP3nDM4=I&6jNu{A5gXj8lfO%rgWq2DIo7U z^F}?60)kU+hZfi`io+P70fPghN^(Q}a>E#$?t`^XfBVb8CfGPmcz=KD985t$q-07S zBtlF<{jzj+H#9VE_-fohy&NnAjCXJ~r+*{Mh&z9lg*$h}lDLHt!z&U6N@RFSs^m&2 zVKeeJP#`09(vgeJk)VzTavQ9InK)vXn2bMm{n9iK%79TG#f`b344y+;IN6i)3ZTldi_tNYb+V2_S(P(vl?BYjK)Ih9xs~-;mS>s5V0n?1s+Mgz ziFcWoJ?fQH`Im>8x`_EUaT$w?%9o89ZkL&vBPy5~8JVB?mz|lKpK6*JRGG8+np@eL zyLq6ZS(mvPoJ0AW%h{jAd6UtZomYpP+qr+H-dU54>YU*jChOUrTS}e_SeWzKpCS35 zHA|i8d7lGXfAQI%$7!IqIiVXmqa50E3mWC58KN5+p(}cu7MiFo8l&5dqdPj9{xrI$ zJ{qJAjigI@nM9hWP8y}HjHO$;m{dBaUK*yWPoir&mTwx-C>p2HnWlR>mt-2Jej0zM zU0J7#TAv3=sh66mo7${t=F2Z+uE()8m{A7uIHMr>)NjG8n5$OulJg-``WMn8n6Rfum_v43)`>{8?h5x zu@{@M8{4rT8?qx?vL~CeE8DU!8?%2iTeCNtvpd_fKO3||TeL@;v`gExPaCyUTeVl4 zwOiY@UmLb#TefGLwrktAZyUFBTeo+cw|m>Se;c@iTeydtxQpAkj~ls@TmHG1o4K3Y zxt|-lqg%SCo4Tvpy006%vs=5jo4dQ)yT2Q}!&|(^o4m{0yw4lG(_6jQo4tSA+r8f# zzT;cI=bOIk+rIA`zw=wa_nW`_+rR%Czyn;s2b{nQ+`tbU!4q7;7o5Qx+`%6l!XsS5 zC!E47+`=y$!!umNH=M&e+`~T{#6w)fN1Vh<+{8~D#Zz3xSDeLL+{IrU#$#N@XPm}s z+{SMl$8%iAcbvz2+{b?$$b)}e$cLQBi`>YM9LbYh$(NkTo7~Bt9Ll3y%BP<K7=3 z9Luv@%eS1%yWGpa9L&R9%*UL}%iPS*9L>{Q&DWgG+uY6H9M0oh&gcG|&giZ{~XW*UC;-e&5LtWHIozzR+)K4ANQ(e_poz+|2)n6UfV_nu~oz`pJ)^8oxb6wYWo!5Ka z*MA+@gI(B%o!E=r*pD6AlU>=Do!Oh+*`FQSqg~pko!YD2+OHkkvt8S_o!h(J+rJ&$ z!(H6Ro!raa+|M1|(_Mev*PY$l-QC|E-s4@~=bhf`-QMpV-}7DH_nqJS-QWKm-~(RZ z2cF;y-rx@&;S*lr7oOo8-r*k};v-(-C!XRf-r_GF<1=34H=g5nJKp0z9^^w_9_Mpj=Xajxd){5=e;(+AUg(FO=!@Ry zj~?lhUg?*f>6_l^pC0O?Uh1cw>Z{)BuO92OUhB7>>$~3TzaH$vUhK!7?91Nl&mQg5 z9=8Jm03rDV0LTFU04x9i006E9zykmX{s4^#mu{8^IFoM!!vg+`mjU7j6_XpS=mY++ z#*+rF`~?1;hf$LTuloi5Kn$V~lLoN*2L8g;4B?rR2C?}E{^>$tdO6LL2D11F{+id! z<~FF^$Y%Rn23tFh>X~Xj*|wq z^bG!xA-RqyS(7$-lQ@$Gx8xk@54~BL2>F*ki5bROoZ9~Rno+r#%84<|8J*&j2DtJL z{)AenfoG?`nUG$`r;L*Zx$zJFnS$`KEZc@JyRt2-b1;(zy6_PG%Y>0zvrw14Nf^5y z%e?-R2D|S({!Tc;QFq5mh{Eej$UOJLF>G{(T$U&t0q}cruW-a4OR|dR1|XXQJiNk* ztFZo%1R!e!LF*600Jf3rlQFy{0sfO$yqyyM23$}Ed0-0P><{N)3WA{0ERzPkbq(zg z?|{;7jmwC9a*6Geh`nJo{$Sd`eR4nz$AA6}+GUUl@eB>1UDRCt)Q(*buaE~sZPY4v z3*f!dqTtjfjnc85zm0v{{*xZQN&)_pX})(R{_w(cYsDch<{`V|E$8MBZ4CaxjL+`6 z!jdcDZcqKzz&HW^lkmV5 zlLo;se@pYb>(l@tiormF1q+Jum(ZUvgAE57QMl0HtB4gNa&g#D!Z&{~G7c=0aM(wK zDegrKLNWq_3j<9i6htrr$Al?K7R(fJV@Ll|WM{^T@KOpz-BN`9m%LYX+U5JQCo zDhLWHq4|&{f(DxrEQ9o#Dts+?ANL!q%cu~@;K+6TME3eBy_P6%J0s7^>zm``mvBDERFf5W1VdzGl5 zx+B_AYlUh&v7)jFikXsQ9A;*m20XZs|gCxA1|q`C~|LW z^=YeMqoZxCtQp8WB84~2&a%A~0(+Mli~w!R|4_Np-gTDdj|A%&`^6nugTroAxTFM|Eq z)~mkS_@kuyouF#_A4uo6gC>w@k1BJcOGN(S1wDu5Kzqy^(RRAhKIP1=A}I)8Xb#t# z#t85;6&wl%BUqOS9?xkfL}3b5$e^5whdew}4||L@!=M?#hBw4ve-3rHLmu|fhd%^j z5QR9z9|mYR=`m0a$iuW1LTxCl+E_%+F{hwxZXg!)$2an@MK_@3S$S|+5)9ZTzpSW6 z%mE-!5(2>pOz$*q*diEDHo`T^gl*q4VcrI!f!!qpd%r2xiWKt4gm{TE3Yl9_&XET< zG_nlmo60xlLbZ57QG?eKG1%g8og)wen&^&_1naq5ea6Us(H?Gj59|fsL zTlhj4`b<7VL}^M@x>A<5)TJ+l=?#rIpaLmvJWN!f6Nhq-cl5N6v}7idav%z0N^MAP zV~fhlST>n>fAM@hr$87vE7TQouigBY27Q+-rHa0jT`~tu0 zIBY{v8zt}(!c(7GBwj(O5&y6`{!j7&FD&=0h`3~<9D@484fV6mLpw@6IBqd&#KcK= zbXgHzDpn}aXlx#65|#_}t(+A(D@HfD)@7=3jstn?f6RnRRH|?jCEkCfL#*r+>XY+@_L}&73z+J{J_ZnBD4z0~Ul?XL$?X#~(5UAX*Egxf z2Cq2#Y^X&__&+}yBxcVrMtaq|-h?o>yTY_C^|E{1grIl5_YG-&^}Aow=~)os z6H@{gf7rkWMsR|Q2%jS&@jOdgP)_AKm3w?+7@5ce6i8)SC@zKzl#nes8$#ZOEfs(X zB$+ymTaXZxfwe5FYDMG$js90Oq73upDq>H|o8Fm*HXm+8kcB+A!kWgjG9iK>{E1t* zDx|^}&hUouI^lQbk(b`ho<%h2TAk>aIzQWhZRriaZ&^8fKh>{_Ky{{U$a} z<&HdzF$_^08E;eZ6goFqELA1ioE4?c+~}wrH328abBMaa z^~blv?U0}xc8|X`o}gD$+cP(cHYuBpf0vOXuKQ63d#uZ|WpHE28u68$xVB2Rz%H>O^5aUa|4XIEje%LCvT!iU;#hkM-R9z<&=f9=6H z-8KrlJ)3EfWNXlxQvdR)eS_iWgdCHWPkWmqdUZz|M%o~ zs}Rr61=H36{7nX5b+Su5YC9htfAm#P&T;qsBSAmCP)b;uuAUIOb*bE>ji;HX3;i>h zXov=mk8_&8KGxM4=bB*$_ChNJ{RUD0$-Rzt{`Y@oYG1ZyYmd2`wj?mX14O_CRKNvf zzy@@{2Yf&R6r#Fwuxv|^ygQ-2`>Jw-f;`9siPAV9ske-PAAu`_X(2H&e;YK#)1Z87 zuY}ROfxtK}BcL28gGP9TZ>X)7V?LGhiS-)_D~f<5fPx}4!Xqp~F5tb+y9GCxFM{y^-Rvf^8MM8;%P#${wi2b4g%dyfXge>Nq;yQYz_>8m(` z2!VLw!R8o-&$AVWsgu=Fx(`Y`DTJ$TjEHYAg+q(J`V&WkBgH+0u?V6KQ41462r@xP z3fjoC6aLaPL0}9{VYeFD#uj9z*jTcyv$8Hkv}_zSuw#Bj#cNuhOl5Y%|!CF=G%&#^DYa0mq0O#no%XLd?0nkiHdk zL_A`EgG@1uszimTCkc25FPTW|NV9r;Ka22`R)fkC%*ms4DExCtxll@rSe;gK#op+F zA|!(D@{&^P5gbv=wL}rFlt-+|pE@cC4RFb{TuZk6#k|x@f2s&Z^k6mtb2c4n#=#`a z!ZggoEXHR9A_|;FZL>yhqcY>Uywj+kSI|h7Oo(&Pl&??@wz3S)J2x|;GBcwDeH4U_ zB0c^<#JRBA!aVB6m7$I4Lda`k#i6(bQ@~A^tcz#RNI~F8g(x9ZVz;oEOe}=URdQYG*beYe?L>JiUf^^!c3tQAdN8ys^p#AGbQSbD}KW=z$zbHs!gG|&D;bcCJ#f3qD}TaHrP%M?}7y6DRsXuH77 zA;gr?8Ku!0Z9v6bF!zu~o4UZq)Vs*UG8hmFa107fk%0E}hlkp=LQ~0BR8ZBsIo6~E}p*{l{H!A(;*Pz8$1^(lz|u{^dBrXj5# z74e{pf8s*bt3)&_2(e*P>HG}iI>UrWN`v#1nRqDk!M$37$OjSB-zwDcTaC}bn=fUZ zx74lw$o8IzRnljn!-#-T{M=8{S;Rz~u$~(~I;F%3Wl)EJ9#lz zsHat3s9PZjDA1mB$SxtJM?@T<8qol@fl%m~o4`qodI}g%c~{KKjGs!RG!h_otq6DZ ze;tF00E`<3#=y@#%@<-F5}J62n;bs)D-`XCm!xu-RyM2bOc$wKT*vMiivaN`~gMsK+omzDn z;mOSG!Z_^F0J^=~yUkl{mE3F9*1voYVhq>J)!fZJz;W%Na=p7AZ6YAON&YS{gQK&mM3OaaFH(ts>>UP5u#A8zA1>emH7VWG zP2GlAU5-%MqfnHFI9XY}GKyk})@a(#K#~o4mF!Z~gz&6%Io=PY4vS5KGSI^-`(M}< z1pz)Ep(~*WiC*&M-a_fJRAJzIe^ANs)n4;83-PN9*9`&Ky#?8oGk=J@N4kYX*($3A z%2ZWEpB%-lRj2kvUDah>U=2ct&`r>&z3~-=JYWnmql(W-l2@p)zT^HB34O?^&|i{Z z4vWnJMk1A{3$NK6hz}ElT+>d|uwpD;;kfuW1fJY9_P@#<#%|@@H-_UGfAw4*0$mF{ zS0BwlApJPJK#1!7h?wCrB&pWZ!q*v2*utY#wplpYkUy)qhc3+^*A+bNkz`smxbFpx zcgqM1+T+^j zc)Y8W)y?=fka5{Zamg*FNTcx6>L_QkfTUE;h*wnOc|JBa225`LjnO#P=Y4L*Io_c< zrbf}dMnnG7#`wBy=EYR4V0$Ix+$Cs6dj|e}p5qf`*`Pwa&$03As^phVf}s|P#l;xQbcn_v!=4C6X`o=gM{{nd<{%ctNq zL<6cluL5b8Feaehi*^QF>v#u*9tcqzYH9sfQhvT0en(cl+Gqx<((~b=(3nA{j(P|@ zhsXhwyDW+R)N_N|e{eoGycikCN}sww=XCxEBS2B1pzEnsmy)LE!0tbLzFarv=fg&9 zXY}VB0%*q^UFD$)Q)mt8y@yz~zd<~TGKh|MC>F7D7$k;D{tYlVwNb-_wW7eUx3 zQLr})p5ih&CMbO5=N-j!nBtieigSoO)T+*r28|}k2qIXre|Mm$-;nHq%SFp3K1%3; zp9ld#0F&B+p!;!02erI(yfM^D0_Sx>Ymq+~5FmqcAgGMWFC!Jrm~LY-hVO2rz8UWf zoy*`zd7&K0B(m6?(WD0%=QV*HcutirwMv*(#~qG&a!4^iUSv|<9?LnzLM(Z zJ=zE;{`PMee{j~%uq5k~lJ$e)1fPnNCJC_$l76HhDu{s>|g z>>AH6!iG^CO6(oyaRgj!#*W5oGgovqJ7@qBN_Yj?GXC;CS-)?%g-oD?On8N7ctaJ1 z22-GfD8B`tCO#bTt>!@8J)x58iSl7k>*~-nC~t%he*}$sc$F+S1rpv47=dz1FoiL% z3*#*pi&%0bS8^Z`1r)D{j=5bGQpdIm^F|N_IsYhK*>jai^hLiJH$Ui^vmU zzxHq!cWbwd79aO>NB4BMc4a35%yss6k7H=BHXZ&I^3c6wb;tL7-^({nVZamVR(jTkdyennR7Vrh=PV&s9wZ) zhZv;d6@urt*lT@1`hQyvn|B(qD}y5==B0NCxRCm-=X#r8dBb-3u=iG&Hv*Zz_pV3# zf3*J~8E=U0Vi0%D>p|i5xu<)&2TPj?^SamjEH`(WKk~iT`@2_r>#$fY3?0DV`@SFi z#b^9^{`!0t`^c|RvNwAoZ~V)z`MxCy32=MvwweAK6XbdO;AZhGy2hBAmje^9CY-=}$ty@1XK{^B?Oyj1-gYW?Lm%-5&) z*bnHlKmO^r_IG%OVYp3N&#!Ngs9P8YXsB}P7k^}TbL_tba3BBlSO2sF50diBHimrW zmw#n+{`S~Wd!POF*Z=+J|Ni&?{|5+v0tXT-Xz(DygbEijZ0PVI#E23nQj~b%e?kfv z8gp>W(W6I@AVfwIF=Fzhkt8ZtvTW({CCr#IXVR=`^Cr%mNl^0a>GLPhpg~EF4EeD~ z#~d1Aq_|k|Db%P^r&6tI^(xk^TDNlT{_6EB*sx;9k}YfYEZVec*RpL}^Qg7)yn6TY?fX(F;J|``5+!2ve`C_6O)t1@?D#R{$dV^hu59@-=FFNmbMEZfBF3dj z>53$I*DvbSs#ohA?E1CP!iSM|RP6LK?%cX}^X~2YH}K%XhZ8TZ5U%LCAWM%t&H6d? z=%`^+uWnLykfVt;E;jD{JNWS8$CEE_{yh5P$mMb*ZMUcN`10q|v~J%ue|GJ)x!+Fj z|382M3OFEv1sZtZTI<0u+H&vlL?46^&UYV$faQnSeoXa;Acq}#_#ucPiZ~*PX(`BH zU8cb&A&V`hRw0Z(T}T^8wlP*Bj{Z67xFe4}`uL-7Cn}d>b1S-dB$7GBxFkw5Zl~dm zKuS3!l~r1KC6-nS*8iUfyYC!yjxpcF`w+bH zej`=A^E#C8y;1Q?Z@l_aJ23vi1sgn>wGL*xEyFK%Yu~pUVs%a#3|M?I7yT~u4oO_x zQ_wpSTs$(#C0i^?e?hOLKynBuE2J{ZE>8sW$Sp5K^Tjq(gtNsuC&V)YJ|8?Z(M21b zRlpa<3MEFS0cZ&18!Nk59tHvp zH*n#B(yN_6?aSjf0&|I3km>>qyh>=O)9GR=0#ZoAc6<(}`4LM;9 zRmegY3IvA?kfA?lXfYcy#D_o>A_Iu%4wBk&A@d_!bIG?zbyF}CS7BnTkXm_bZ2GAI!W{7DKrcE#ilZ#hcXe ztO5- z42S?S1)@xgG1DK;gl0jeIn0J!Gn)R;W;MH+O<`8Eljb}pIz9Bs!-*1$q%0*#q;@8W zO$B|ED+n#~ClEE(5`zfATvoNnNTxuKLx1gmtPdeJfn!3Y?dkB&MRAsbFeql@D4h1!yb?K!f_i%QbQ>)t5YKE>U zv8y}rY9Uk72$f%)N?`q&+k@axun93Ne?lnRxq^^ZxR@o|EJ17B(;|1d#6_(?DC^ne zBA2<9buK|j%i8U7x4Q`%)`Es~tpLK-ce0&rJc(D^Q(Xn1vMVG;{n=Z89t5}tq3k^i z0u5nK4!`~f+3&^f>DcK;cVzh`2!JJ5-~j)Zznvwpe+`mg2K3j!4Q8-+Gpykae{aRL z<-Ov0V?*1i(NijX#VSI!s>H7WmA(eCuR&f>!^7e*g)$x@jbSK5=((ZCWiW<7Zk)s% zqtG8hmH?4+_{}`@*bP4p@{D<_V+#%6sm!mG`U=K)?CUPV_UIx2)(zGg=@X z7W41OOc*nJvBXgUXb=zrO9q^@&H)({Bl^?yj<>xL%t{7C z&>#tnB}YeI@`9Y)VygbHf65Qiat6q9l@OFUt`lN&i>;g>Gp~8fO`day6TRsEgLuTZ zHF2I)d@4Ve@W_EU@4|+BAoU>MHVslSi)I|_}9u`ptblG9b>p@G}SEIYeQKl}$4s2$*vw16+xa`BHuH3DIT*hFdvbCR3RC2l&Wg(}U#~ z{^?%;B2xhZApZ?u|ACMJF5ox$-vb_#_*I|<`U?4#pQD_g5!jwo_|OI9K_09b1^&)o zi`8F1Da2L{QXaq`1C)VQkqi{HAj!a>!g1RQE)ornj197te+=&6#q{8UXfrwG2%B-Vhwqdbp6{TiVY;@niyIl9*!a@s)-)%A(-@GdUa8;p_x?- z102fFzjYkP#Tkw> zV+|f-DORI3PRS{vB9f?LQK%P*x#CqogD6k}C1iqIf3<}?+yXgL!Yfc?L8Rk4wj)%) zBRR?=L(n5Tx}!tjqdhJJKkDN(2BbjZNH%Jti*Tc1c;lH3Bttf&LsmvX8sw;)pA|l& zMP4LEdPPJ^WP~83JXIPvW~4}tBuPHSMslRBP^2H;R!O#`OTHvRoFq!hhes+TNWvse z<|Izaf8|Qh#9`i~XO^aE<_>6vW_@I)O>X9Cz9wuQO=_y9OpK;$#wKp&Cbi5aZNh|YHl=O` zr*Q5HZ}z53{H9$Fr*bZ56oMCuj8~W$XHr7uN*Wt;UMF@^k#mZd!oZYt`XqG%WnHpm ze|DCqd9q1yey2(xCwZQyd%h<&Zs(UlCn%{WRkG$;)iD1qW9 zShgpKt|*J{#)cYahqkDU&M0oYXm7?Se~sp-j#`F|(k71XD3AuJTlA=E{-}^1DUy-} zk%p#`CaIG?sa7oMWi~04R;iUjg_Jrbm0l^Ac46~_Do!%**=4hTyWraS2pbjdb7OJ5hDxxN;qAn_XGf0pLHPGP_PE5S}?z-r*Y7OcVw=E0U@2$U^iqW&%2daT-l zS$($c+wQH=$}MTuE#MNR*SZAX7B0^E?QI6G;=<(MwuIq6uF4`Va4s(8ie%%m1mtF} z$x3c=R<7qh5{JLK5gp8?kKKq61481uJ zu@N6J5+|_|FEJB0u@gTre-zii2yAZ+Fa!_#FcfF87H=^Zcd-`>u@qA=LrgIS^zav- zF&d|_8m}=Mb8#4BKou{<6}K@R*RdTp@f*jn4c~2mmgoTCaP=~+S~_j+PQ(^a@f|0! zA}{hH&+ir}#392mBTq6VC$b|$^7dA;CT}t)ck&TjazaEh7JD)&f0yzQgED;HFd$#8 z4ogKRAhHnPfGp24E!VOw-!d-evM%p3FZZ%9|1vNKvoH^{EDOQ@CvR^jAjB(6ax4?G zGe0vlN3%3fvoX(aG9Sb;`*1T)GdFj$H-9rYhch%+GcqePEQ>Qbr?WZ>GdW|xHNU4S zuX5G4a#S2a86Uwqf9JD4?=wH+z!6OG5g>#;j{rXRvq2xUI{z~U1oS}+bU`CDL`Sql zAGAUvBc!ahE-3ch;3C$+O$i^Hbcz6xYB;uQW^Rv-XYv2B9<@w=_-H z^h?KdO4qbb?=(-}v_Z(UO7pZ(@3T+CGarZaYlRX?=dd7af9B*qMLlfqHV-vbSG6q* z!S?=@)k(EgclA1BH3n$)FlMq>m$g}cGgymt#&Pvow>2@R{&gxJwbF>RQZF^wnsi0T z!}hv0U$Zn=tIa#^bzcwmF#okd=rslqwqh?fEgN=VCpKe8_AEQLd&aX}>nvVV1Y%1z zXRq`20yaTxe|Be&wk&`4VGA~Cr#4la_G7EIVzYK-%QcD8H5OeqE6-F8Xh6uq0Pl2Q zbp~%l+yi3M01gnhaThmoAGdNRH*+tyb2m41Keu#8H+4_9byqibU$=H=H+OIMaeubx z2>14ScXyXJd7rm=r#E`9w|X--coT$pV*q>4cYD*fHhtGOe&4r#=XZO*_dvw=eg`;z z3pjxfICT3ry2^HMCv|Px_B_8643sRz{dQFVI1OAlhW=-hz~~o$dpLsogN3KKi@!LG z$GD8oIESxzi_Y-$U_$fXYcZ? zN@#>i1i_rAER&b)o!8!onXCaAz@KY*q^G*7Z}_9Hx~unitG~Le&w7l}mz1D9%*LS_we?8cTz1WXE*_S=ntGn5!z1pw-J=?dv+iN}A zzrEbgJ>A#6-QPXi$35QfJ>U0z+5h}<(r&LO0TI}2mnnI%k1W|1zOfJTctZVuvP=2Y z`#t4XzU5oK-e11vZ+_orKIeZv=!ZV%d%ozGzUjmLq*uCb(k{d&K@RM8#E)zaFnAJB z{I9D7#T&a35dF?N{?kXjH50JYCuXDK_DMJ#uLBDhJf*B2;_%6 z_wPUdcfI!aKS0C~IFMjLg9i~NRJf2~!-5zv$h+r{qQ86cAUf2z{*hxxj~_vX6giTl zK!_46TEvKH6*`n?QKLtZCRK`m^TLG`Frem; znxjXLAVG*^C1QkYS0hQlh7~)OY+18s(WX_Kgh&IoFu?Y3+n`6-9#DgX4dRvvSQv9t zgf)_4F5D!0=O$LXn6YB7k0D2vJQ*_M9)lN$Aab>8kEuDJ!q8yAz*3bNMXpf2nssZ{ zFIjtZ-8$yR*aBDAJ~%Uf$zF!%6|#YOV2=$mEg;K5-cw=Laxb?YsaQ)KR( zdO_&d5q1ZkJR(ig=h3HEzn*=2_wScBeHwLY)vH>!LSDO{fBt{}V*h$8F1G^nD>1_` z%V9vb6bviC2J%Yc!Nw@8umQ?2%rHaCFjGu3&Z_z>w9!giYP*nsR_Fi&6)W%FGNY!RU^GOxSlyf5@;mcFcJ^Ad@&p(BdFSO6> z!%C~j{!3KRMUBvZps)q&N8YKrn(7#TI-v*~tGNaeo1MJjmd38FuA`Gq0wMeA6|(jR8z7L^@s!VUKa@x`6a8}h0CmHBDz$*qpz@H!Z8y6wk14?XF# zhZWs)%ui3fYspoIyCPq>4y2;9wMN}`f&BJI>f3pLHyzK3fe&8z;fV*TVi*Xu%FwJd z?%4U~B|LD!P0{+a(o6kPFhK)n?%rmb7lVPTQ!`VQRia%r+QcA?AXq{bLXLp8`7zhP_zLAfu>0cjPQ$K7JOpbxb#S68@BtWj_ zkBd}|?DnAy)%;IxbaUh-StB`2x<-fo@WmYU7>G_F@^YIrrD}BexKpljHmOV;D>b=F zJQlJOxywZ&K)D503I~L*bmRzk37uXBlbFSXr3Kr_%FpeQH7BecBqeFgOEyq<(zIrO zaDZ4uZElmB-3-bR>!ZAkEwPE^>`xe>C^92}CN%AYN~c_wmrs3VFJD1f7iIQEQccP+ zV@!Lb=PQZ*30*K-u=nmXjpAIyYPL>0LL zB+1Tg#mZzvC3@1;P_mRFMQN0X8Ov3Fb`o5(sh=uWV^N2)W=uLwX>07o&fB53=K|GS47lfqU2mAH6|r->Kdq86{#<6<}gpXqUQkfqaSVPY<&7th=7n- zM|CMmThmjJ&StA2Z4FB8%2cn$6qyKJB^GiAJJ*o3qaV$yQW;aleGJpPy^E0fGz&4LXi?*$W*H`!<-hM71HOZ z_?ejT^{k9@3EKJ6n7%b8C7H&|s8;?VqDQm^qOB*zf+*3}T-8uiMOe@Q7GfBjWP5UN-QtLbomidVdv86()e zj6HIK^(){e^AyEU?(u&6s$WyS@Vl;|VI|?KUKTTXyhn|1T&-JV))*MO!SU}KF)UtU zUAcl-a4|!z%U7y)md}0mGw?qCOO?k zGn88~k}p*PZB;>sib%1HM0oQ7O zDBH+^sCBdh;SFsg%iJ{q+(mV04;}b=hbh7Wu^l(0bALDXY!O5~CFh{HDl0 zd~s9^K)Z(ejxh$>216Sk{FU|V1hO61iI5&!-2PBTkqKK9U3Q@0HbMBe0my1pt2(D8 zugR;A+UsAZB-rc4g1X%u{s_imHp_J0f4Yb*_-C(OR;#BCyPqPwl4*{(N6bFI;tE0ESW zk+;x+Ez-u<@bR*Cb|-Hl*r}SQNWr|eUa>&&iekJXs-$FpPP#18kl%#Ykgtg^n508> zAUhWm?zq-qI)dGs+{8TJbL*u!aM#2>CxS)2O{_o6UROjNmYfY)8EzA)hu_sT@V)FU zpZZB2pZnQWeopMc^8OWc{w9)NLHcX5505&Z^`q`@U|WNv^l$RMj{YJKsGJqIIsgf5Gl@X(DGxn*scVV!0qNDLf9fI>P|6K z(4V9z2`pvrK&9_6MAK--pXLIfI!$N>D!2}BeG-qA7H>&w#p$gOH`s79( zIzi|nNeMYhun_+73m38ZHYwm%N&3F(;1V%|F6Rv)Q4UwYmA22jyzl#DZv~+35d-1q zy3Bol9#Io#ClJOkchq4Fv8&eh;S{5dZU)h93b7(MkE0S1k-BUIz|a#DX$7WH5t$C` zp3nbIiOaxihH5Cg{zw|TkQ&`)1fp&=2x|b@(9I&yjdZ{n&2b-CF8!2Y8M{!=SnmV* zu^;_0B|5kLP$kPbwRfN#zh%ob3 zW4p+)nk-M}6sdl?On<2G;P%foiV&3|kg%?B=^RIa&Wklb@`Y;ZkDTqx)FGutQXBiv zuV$*&awU}hE-n(+uoZJBnjQ%NZ;u^YvDO4@*O)2dR0-#DZv~E^*1RDqQ$p%a&BW|~ zsK}^tqG}HmF-a6pi4<4raG2==!Hg$;GIu7imyWFqzfl&oErT?$ZQ?K194G0#a_Cfp zt=e)XSpzOF=PCE7lcw_Et`O*8k~L&f{~WUe6e;57Fcf=^m;|sf0k9n>b2Tb6ZH^1s zRPTkZ5CSjnIZmzW*kSR)X9S*Fa zc(Mj=4>kdFYs%BhWKS{Y49@^)7-vx_Z7rhWks_EYG+ATU2y~GQbd%V!4L8W5LiBQE z5G9N<(v856^qL8UmZ_o4%@SR5-B_vyT(mz|AVU49c7!fRJ=7oIM?UG3 zB7DK=8cG_2F&62tsyGspgiiH;;1Z|)NJ=BBK8>)3itPHHPUtoh>fTWc-}67~K)%vEqL%Eeely#peYcqOTF_w z;r^?kt_vxv3G{}KTM}#X?17_B)Fv@?NOcDfpDUEG;6W#}H5ydMT6GJ5U-kZwR3^3N z!`=%|gR(mh)ZlO;a8%>Vd|_4f=(#GQ4zew)UX}Wobp-${aoQ%;Lg`p@=Mrm81E~*i zjFq`?Zaq76HKG+L>(Ek7X+;aHp`NSOV3g)`6{b4r+`M7u^iq{dji}&~BA}sIh00C; zEywVT>n>+mo3DSI6%?_5pbip8RjajCnU0jUB~~vhEjyE#OftK^tS_0g)`q2FA5`el zCfm?cZV+(X-01bN%`K;oE~(ESK(Qq$Hc$Cx{B9G$!=|D%F~bP(^ML8#*p0Z^9dVA<53qljzkwv#^zH3H0E9k8Y?71Df_}+p$Oo zGbQXHSAGiMDwdU;E*ISPZLt9Z6=?K6k)`Iae-u+oJ@j#KRVQ)*4_U)~bOIalmXlCu z1G&_i2zEC15Lna!18azG5cF)trfeM%)^v0h>qqs5)LtXc3&|_SkVOX^7jh-eZkLqu zx~rwM_HT}$yHJgPUnjOe*LIikbzy0h4Y}6y=w@@zRCm5CZN~Ft-}Jx2aq4y`ZWu6i z?^JDNwq~I>dL!m$2l6*{mb7{{d-v{Ve-=N8R#GW-Bb63YbpYpZqUyNLCWO`gFKa?r z-w0-xbwpjaR^iKhZvtwwkZfC#Q(sQac(q(zk7H+puwZ9@cX_W4^mo}P&oA@RMiYl_ zQ{rzQ3@~@X3y+sIpEYKC)m&Bda9NDoJTJQj=J>oJO@Y-Pyn%yvw3KE4-S>}cpuAs0%ar4BGT*ld5!Sb zmY2$Wq>xEjk^gqFj^Gv__bbVjlZlBZo!BG$N1R8QoU0jt%D0jzsf4o;vWi!uE||rr z4wqYhBV%t9j1Bsr;X{m@r;M8jjU75v)HpiXI8q7fjm0;P1z9y(mUG{^Q?Y3WXJ{k+$O69=pfBxBdC|k2NVl;*} zm$67r%x;>SN7Al#*ku6|8m?>vuFHfe!oxqnrArMauF@wU1I1le`*hVYsW0 zH>YXP(kmx3!Us0TA~&^mC>ZDFVQBYCs|&w889lJag3b44l=?Ezaw)UmYs z{wNIUazQrNnU{U_k^s!*Rx_S!jjQbqb5+xCwkF=%MnaL7g@a?MSu8~LkF<>p=QNpr zGoKfsUHi3^xN(l8 za}AL&JCKkMOk9)+RU?#_`xT!KB{6fjh0VAz0J)juk8YYxeWd=lJAkCVd6%J&ZTne) z8PG2`%(&IN29o-(J9{aYTSdasRw<%?tuwE}X#gHHBw4;^JDgQT2fhJi|Lczz0*Ohl*n>oN2>xhu6DV18cuS z6iPq2q$*syw^avnyaNc@!P`54wPr)P`Nq*J$L}(0r}gr9QkO$J;ix;x!Ng8~89Z4R z95Y>1lkD!x(nh#e&HMvw$>0T$)Q{t=nD6Atqcbc!!nN84Z$VVJK zqD&ptSL4)}dE%;iSmhen|1(y>7KPXvB06b4HFVcuJTLz!gH|2URV~n?U3bc}Otl=4 zXnm3T7S8`j+dC|G(@&K`E-T&fL-nv&K%JooYc^l~-1Cvz!tTrUecx|?BFwFK%*ou$ z1s=`&PR$F?Xxp5)R)R?wUZfnp;UWIvC7wwHDdN))lKB^fhs7B&UJ@s?;g=aRCtjp7 zNF7XG9mMS~86Fz9g;V2iytz+mFFp=a!W%BJf2Mr|{&4jPePPqjapWUD=p8<9gudv3 zo~DG}0Yz%(TV4`#{^46 z?(EfG;>Q}|Gsx@lZCjZBkbSquv=sDGFDf`GtKtzy774=f6yD|k;HO_@A~VQ^t1r(lx8L&Pj}p5o$IonJ`bk zH_e(pjXJgJ)vQ~)ehoXe?Af$y+kSoFLJIyEymR=@(W6I@AVi82F@ij~ktEEUJAV#6 zy7cMPt5=&_O>Xbyz1Nxx^rX0%nEK zRvjLR(MK;~fuw*+T6JSdG!lfLU|sf<*OUxe1eZVR1VSNLEV{T4j8QH*C54^6RL4Lr zapVt_iwzXx2%WeX*_}FIgQi9m=^0i2Abk4MNex0K=8ut{WjT^BLo%hIL>*$8=3$zS z`O=qv1BFQ{Pmfyql!JwJ8kw1S`8k4Ei1lQcX^4S(5r-vpV8Nj$*^>)Y50Yx?ZOSgo zY_rZj3vIO0)|T6Dz5N#4aO|09UANwT3vP7cjZ1F1=6a`Iy6Ucb9(zQ%7hHVv)px;t zfu&075rJS4NN4(X5@P~9?Mm4zYLaMIb(HnsezaD^;vVchoDKW*VHcmu3|;qVTK z!{aG%hw5Qqyc|d$2+~G^AuQtfKxo7yjtPlMY@!ok!o(+XZc!{bAqriH#Vl%Zi?e|t zw)_V?^4M^Ue9NI30ms8Mn#YHV^9|$b<+!RzagL#jA|3BIli7huj(F^22<_-c$>pMi z_9!HQtTsO`DsqvG)S?%C2gVtHijjxN>sa)j*6WPdE%5s*sdn9lo2|yZFa+lP>q%V`VNlkVVh|`N>^<0Te3+j<-$4n+F zm+228I&+$%Yb7${OgBCdltbiaKa)An zi)JsQ8trIb_6bjqZWNjWEvZRQT9AUuFqe~5XhIv>(1${FozMHBL_bMA476^iJngAZ ze+tx~3U#PNEviwEiqxckDs`zt^&C^53e~7eb*fYis#B|q)vRiDt6c5sRIlpQu!?o8 zO#LTH&x%%As`P&cZK+Gg2~&p3l!pMVw50ui>}X4i+SG>jw5n}w zSj`&Q*vi&^lMQEbEPGkZ{!+6F&p=EHi5pyDR85U-qnIMxan>0QlIPI_(B)H@Q?Nez}`y$r%Hn{t0=xqmk-M}8UrH>8oA}3qfD7L9x%^SyeAZ@J4~4)d7HeC9N-xy^5m^PKDa=5;uSy1BjboeO>FL@&D0kM8iE z|D4?KI0qJx4)v%@ed<)N`p}g=ZlJT=>Rj)-*Kv+@j*H#P9`|p^&+gKZuWQ-&Q8|HX z@b$QV%YE)tuK^Z(;q!Vd987M<0^I2i_`plu?s%W}-cQx{zYBixjBotc37_}FM>Fw` zue{|84tc|mee)VFJDJfA`h==oU2JdrJDPru)T@5=tZ%*RUl04(%YOE>uf6SWkNe!~ zUiJQ`*SlHn(QMT3e)z;MzVVNbe4lIHE5JX0ee$0V{pd@7`qbZj?(}$h3BCEpIr?;!zU}fjz5Cw}|M<&){`9ZE{qK+e{Of=J{O`a2;V%!qXBxoQ ze*b5H2Z(?PsDKN2dI5-id{=-BD1j45ffZ$b&uTgFgs_K`4YnD1xaGgG7jg zNvMQN$b?KNfJRskO(=y^NQG5sg;(f=57-e$h=pC~giGj$7lW2*Th>3YfhMP!TW{8HLC_HMo9&D&$ZpewJh>EGGimS+qt%y>d z2#T>7E~3~jq6A|imCaa`MtOcn*&a1za?BW& zTgjDO>6Ifml~fseR{1Vi>4Hr8m1(J#Ysr=+IFBAyX&)z+J4ul-*_L;IiI;h)my8sa zb9r`Tsg$LNmU~H6w;!nZ}lx_NJAfiJGaYnuIBuqj_ninU(ywnzKopwHcMI>6-FZmp#duw*KjxzX_Zj ziJQ6kU$A+W!HJy7shkdfIh@3aTgLgA%PF1HNuBH1oX+`H&?%VJ$(`NloxpgV*r`z3 zxtZT-p67|4oEVn&}Cj@hP7j_m<$)U&Y9tjJcah>6qI&pZ^J<0cu_AnV-Al zp0X*R396tA3Qz=EppZ3mjq(LgALrSDYYNSVsq)7T6 znt4<7kpfRjqc|$1Q);8t@}$<%VUV<>Ny?>N>ZM-_reP|kV@jq%x}^U0)}&X;p;XGI zZ3?1R`lRy_Wc{;1rDbZTWm=Fk+M905r+u2BIvS(Hqo;mKsD-MafEuWMGpNe)4UOuk zj|!=gDyfr7sg-J}mx`&Gs;QgGsh#Snm!Eh888slcF2Opk>8h^MS+3^lE$F(g z@hY#i*{<%2jNKZTum}qs z0{gHPYq3umu@dVX6nn89>#;SNu^NkP2WgidYqBT1k$)j8vK))DE$gxjsj?c&vM)=s zH4Be1E3q?cvpdVP-iWgZtFt{Tv_lJ?*cz8+C$i|Lu-_`7MC-Iq+lxO7ut5v8Rcp0l zD7E%VwO7lvUAu8v>#ketwPQ=RWM{O*Sbj-6o=a<*;99nC3%5Q7w&p6facj4Cn^tqX zt#ymHeShn>4SKe=m9}czo@^VNP5ZZrtGEHBx4+7_iwn7t+d_?-sE;eTmy5Z&L%D%! zxtZ&^pX)Z88>5{Ix}|HnEsCX-hPEe2xJpa1ym`8@E4yzax+6-uvx~dA{+k-Lo1wR> zyT1#(DHXW)Ik>HBpoV*#!K=K>`(vp)jDpL$#(!H$zRSGTYr4Gqpww%<+nc%ATcFy@ zz2OVF-kYD`E57IJx8%#7=8L}V8@K8!p6u(s^Ec9Wn8$Cb- zXvdS!BOhv-0wJKlA)o;bECCUqzY*NM!#ifhdyN2$!T!r>AqSIog}_g`zzpob5G=tZ zJb%6v45P-#rSQ=k9DI;LkX`X{Z%zuOduqZrOt$>{E$+g@Jxl~Wz#AKknfVCA{IjGo zET>l*r8vyQSF6L`;=@sF96$^M13bjpNCdsXqel!xPP)Xfp$v7Pc>b5?cr{Q50FoN# zU<|L232p4gZw$wAEXQ+9$8~JScZ|n*tbfOQ%*TD~$A1jSfh@>_Ovr_7$cK!`iLA(r z%*c&w$FE=v=g`C~@xv*UbLV4eihm4bjs4cUxc<#z4*&jnNsc(Hniy`0UXi4bmYk z(j!gM_Dl(xU=02254@~`%`?zfOpBS!!^Z5kvN3Bla0Eht13S&rJ?+y!4b(v`)I&|w zMQzkajRQh(1b=@A85eO591Yb`E!9&^)qz~nSB=$Kt<_r{(rqlsy3EolILri{jJ&~C zCfc^492-X<7)OBAZw=RRE!T7H(?>89M^F?NUDbQd*M05R7|qpzE!cxi*d(pbDGk>D ztgWu;(l9{Qw8$GhOw(%p8uu_0H9*&yt=XHM*>E=khsS@$e=XXhP1>c+$c2sCsjb?p zea@Ni3jG|`8amc6EsBx;J<|(&%&}1qG@;X+?c2W%+&(P?G?8@$VcN%y+{vxnb*#>f z;M~s*-O(-G(@ovgZQa+6-Px_(+s)nG?cLuE-R#`Pu&vlgo12aO*tK27x6QO``Wk@| z-0>~nnmvEqHNg_gjo4&*;B~FuuuU zz21Klt{Pz?=%Fs^qfY9jKIo2x=&7#it3JojE$Ocg>#;8Dv)<_8ZRt#m=>x>(n-1BX zPUG){0j4hO!%pnPPUx(T?8#o}u1@RC?(EMF?T-%Kw~oWPE5`4xHC@?(5d$<3 z@B=UK1yAq?Z~pKJkMIkx@D0!K5AX015AhRU@aCTG7mx9zz3%YN@g48+j{e-#e#6#I zJlOu4ZochkeYoDv8lX<`FE8;i5A!oG^ELnS7N7Asuk%yA@m0R_qK)MtKksQ?-VJ|# z-kSa#L7>2nopvbC?{RL!s}UwKK=oB`^;eJeS+Dh5&-GpJ^Ynj$hSQ-uCp~VQ`{Xvl5hLBf5^^#<+v}^l<(XkZ^9%msGDE+cb@>DKfMn%`UFt=g#-Ag zADanq_ar>{vv2*^kNsI+`@GNn-LJ>GZ{^;<(QL2JLx1T-Z@~^u{LIn$$KPi3qEGs! z|MYFk0MnmS*&hQdumbzm0VA*iSq~5^1jZm((BMIY2^B76$PfbtdH4KHq?pO#MT{9W zZsgd}<42GoMUEs{(&R~$DOIjy+0x}pm>3&1qEypnOq@A&?&NvVrp=iaDVGat0UUpY zdf`F}7-4ga&C#PrkRU|V5;4-&t&t>g<<6yB*X~`s{(1H8Z|ANNJEfqRxQ-`Z z-u!v=>D8}i-@d3K~!j3`L~+<1R=LRVdTd1~_~ zlkj~vTnWlpw_uPcNjHy&Aw~($GzHaoV~+ki_W0vK36*aw`Vv)Z(MMBOIps(v9WbvE z2{Rytu{0<(iI_{Ii@^|ZzTm(J7Nam$BQ)e}RUEN46TAXljHK^C}jTF9sZ_vZUE>DnCSABKX zTW?)tLrF%|KF(IQ{dWCY4vgeX=RSrjGNsP-dTr_6bCww*85Aeg6CR z|9=3k&u&9`NxM8J2m~Uq4EX|=5pwdl%CW0+38Y{GN%xU<5utQA*<9l`0>Q{l?nNHl zAWJZp9nCQ=4-|jQ0~an3t_Xatgqhe}2rf87$DJ-?uH#`3efUElW(9UFDjDrEgF7T9 z(Js2Pi-0QSGU3fdLvabhfjku#0Yv~VKeG^LmUlg)sfv1SK^pgD<-N6FjWy%j+O8fF zzyM-{BDq*!A}FE`{JBk9=+n(Z95}8;RC-yEf_yLT8qG$vO5GDn3v6;?{z_h8kg^QsHqi4ow zgfg0uYF2-{8u(bWMi{v` z1d!0D11yX{kXY!1FR;K)0YZdH1azbZE+`0anUWHMK-UUy$Wb?V6eBhHR}4`oxX#fs zbKY4OB{GOYgt2s_Ekzf0vOf=|~4Q zr;Zd=5P?J3yh19&MhG<~ss4NuB9sO(u$F_t!0AiS=V*fkn#(#6ua0h#nVf~CRVYFWzR2V7tE6lvzU`z*)e_hGQ9xi zg_wVe;-+2$mz;UZ2nNE;^a!w_ZcgZ%;S}dv(5NDF-pWPkET2Cinm%7p1P=YH1O5B~ zKYQ+R1sLcA+9sloI{Zc;$4VAa_TdZVwhtQ^kfT5L0l(X#kC788U`jve)leqkT|}tZ z0>>2rD9}j|3>4SizGT2yBGr^M)Bpo4DqeqgaUm064dFl;=Uv^iPfxY2zhwI2Lp)pAqtdS85+agk4*agzm18_kBBi4UbFn;xW zZ+CSWgzq8~#3wmyc%|8*nMn7ze9G&0`I?4X9+$SorEtv|md-0@dCM1T=OX6$Eqtaq zqx>wFWCr>`PJ=-Y5<_UYFld@wR1nTcdmjch`n0As5o;&iy{gqJw&J6$rfth3V*PpcVHSJvD0M|U52!4Xr^+C2zq8R}a( zg)tz)c_>4({u!`BPvJNRP%y$!r*mw_C3~cEHB%AARJtj4E*2=A1=fvLGg^I$(BJc`iz#|4X1yP*N_=d;1!{nmV;P~D` z2JX!be*v+l(%W&t`#9H~D}x{;T<%QRyS^K}Y2z^?Ko@Nzy}QCO_wqNDb2$i%KnY|W zn8UNk(zBY&K>q*?N!foL%vvI*xd8k?G%6wt28fv_x+Y0;i)f0K)M|@uQY}b(I;bNj zHNq7(;{F<}@ETc*pR(x$b|SSJC>v!!pFk)ZR?8pzF+%#;pSRN=x$D8Y#d|1daQv0)wO{lQ)i=Jvv!37?24Zd&PXw zq+1M^FtZ5RgC&2Tkfa{yMAR!k0E3(e@TtHvxQl26U30t~a#87fD8e$+5%Y@MqJd7j6 z3_Ho6v@sAOL{L1uXxmAd?5~ShGEKa@I0P4wGdHRf#qa1UHq5@1+P|(OMsPW@R4hIc ztCu`1L;ipA!%l3RJOoBOEQ>U8$%h%7nA|@}l!<)dFln2^nmj?lT*kPx#t(YHZd^?M z#%#>3_(qcnM`a>M%ETECbeR*xi<~LCa|AlM;FP8zfX0vudlbQaG(pKA5e#XI7KAjX z^NI^t&2h4tfYcftyb2za!neC0|LLPvQ%JIND1v_?sDPpf_|XA}1gJo0uKjVRt=mYB zq(X|a!jJsKxDo{XY8WTnDRH((}+yMxR?A-HzYWbbEOh9FaZNV{)$POluH;&0>bi+^i~p3C&Wt%(3Ctx_uGPNpG% z5Ks&*GQrR&v;!#~Pm`Ov&gYi2jAk6K~=&i+{lh}qzFg>O0Coos5a7RAs8?@4}v%e zAc7ZNQSg+G4qc^`3`#eIN%m|qQQc3!tzu=v*s8O`B| zkY#F3Oi2OP5?DwRkPv~>8O#iToH~C@BLE3dr;S8AwdtQd^#>rF$c{DCIv9cCLJE6m zRJR#a=!{O~s6v(~Rgh=|!y`9yaRXu%mlyliQH>7BD^T=I)%D~`R&`Gyl~Gu|#Qbzt zTIJ7tF~)xJHk*ByoNZN|EizbT0K`efg}F*rRaRxKJ`7!{>r>i`D9U8aLur3SRd7wJ zZ4Ckl8l55IL}e9HGch@EIkPtW&~~xKspK3ok=eXiTX{ojfSV;j*<}rdUUB@K&qEJ~TBS--fIS>hu%~=5v0|?lAd<)P$5PamvGn&&h zqF6gU5RBc(>aqtN03`OIi2Hw;NJ5F+d?}~q*wyd%M1RN6m zoVh(!{|f?bLpk$3UvoV^7V}^Cos;X$-t84!ktE!B{RoUH(#4JbVHu{g#+6LSb<%$2 z53~Ru$fyj0jSE5R8P9DC@Ckw(kW9BY3$vg>CZ5-RTHR;(%3dJHk61Fk-PXpLTR{G0l^kD=IAEX! zmzdn!XANYXG)#Z9^yHpwHoC0J8Er!b@YZ=dOC%?N`= zWRzgmknqXD**CI8L5yH$y#O&Y-Zf?6hV`K-l1l%u4dSUttWpjloap^ZSFyagqA=Uq;M{q zC0qeDKD)A+u0etZ;6*!S7$kSH2P=+i=xyVCCTNb(#r~y=qnWWt7&+ zybKpU1{b7l7e9ozjt-nWM9Td`O75^Wfy0P&b=ua8fSgQMGFdXZ)G)#QobcW3lGHT> z`aFKIwv&He)yIZtR0>txVI+h{Z#t2r7)%Of;+>YZ^bX(E(t^|#2OagC>_A_}dj9 zaRFuMP)f5P2r1y+N-S|;eQ{C3J3R()?Q!F~8V?srI`I@=akY{p7w>@-rYay8@^JCB z(k_3!GEnjyP?wa>ZI#~b1rFs&vQ(2(u@$NIkU=;B@An}ulUv*Y*bpTh21OMg(SEzpq9}2Y@fp}huT$uG%?{!~KXX%{Yz^-1$ znc0Ow9BU)DJ4|*{qz)MmC1W3#mux+RwuqwsTo@lGs$bRTFA)U09hWW?x4`|JWIwNE zSGT6tuy=V$;Le-TDf8vN&@{J>Z5Njdh~Gqa7h>l%5;8ud&GYmHFzBndyW}y8!sUO9 zNLP2kTgu@aw+&2Rj^mV@bW5*zi*FB1*H=wP?i#Tq02)d5Mkh+6)W(#_wNW z`Bv|Q<{}Dp2!vJV^%2Mg| z3Duugde^%k)m|zbW7WbB_y(-o^I9_@g8>k;A%(`ZExP~(OM1Y6q^5^M1RGq}xB8!d zqWFuSecB(7j2Fj^-}H`$T$1noSuuIprSFtSb)heQRcEMN5MJdqep%P2KuD-ouY*94 ze(1;bfvN%K&wiqhjz*Z1shUQk8Y>D~d>rcj?k9hrn5yr;nDI}XyK-Ocjo^*z{_xj` z?zf=6Y5%t}f1iLo*w+bwXw--j!Gf884kApba3RBn3n>X|BuEe?hZZklOek?<#f%vt z=E>8jp+txy4>rm}(Id-?1q%v1=x-*?nl}D#;>@XYC(oWfe*z6EbSTlHMvo#*s&pyS zrav!SNC9JNj;c9&^av7!2v;ITzJfK9WGvaTX3wHct9C8hwr(;JWQ@XfwqOW<4Kf+!LJ{&N zrIlBfhT%yfa@nPqUxFECh$LQD)rs5X6{DJKuJt0D;nf&tjW-Gw1Yz<`MW1~@64|Gp ze_CdwfLIb*sG-3%A{%XoT1n+gj6zDNmNJQ1sil`8L@(BKzQ^$7$NEv(G{s z?NXe2s$Gg{o|^57rfU0^s^9j-{_1;n#@2m|eTpVt-6H%Qw+XiiuEXrX;NG}*nhR@vKHk@VuErZz_U=sjf*i8Q z`Q8iV$S1SJZ@??F+_KBj4Qyt?*B;EU&2BAxGh7ezY=pQ(xLUEE81u=o(d~Nd56Vk3 z-E@>CJ010}yk+>Z)mLMk^;1!mNU*h`Vym;*aLEPZ*gN;^@XtY4#pA`YW_+~Vw!;&ptee`j%}a&cDnar4ABxa_mjZYkks(i~ReyDxrr;|?FzgT!nfR{5S7 zn4Y}y%QN5n=@^7|dT`Na#iqsmx~`63u#Es znURf(T%>Q-$Ve+ut!ifNqfa3 zTj@&H2xK?MdL@=3Ig(dYGMBn^*(DE{NlkVKF~S@sF{hG8JAyKT#Y+J*o0)e(R32kySq4RqT208)Go17z^5}jx!&WTQnR){#oImtQT z>Cul~N}l7RCrL;CwNE+Su$0DYN8NIOC5)TeSuqzEi2Rn>HV(m16QkX{tz<=!y@h9Xp^V)YzDNAQm5 z<>VGRU8^E_s?@i_HLh}RDW63zi#WQG|KAawyHy=wDcTx04rk~ z>)6LaHnNhPEM+Sz**ct~raZwZTRZF7&*rA7dmSxlnbX%g0yeeg5$s^=q|z@kCb7PM zz(NfyYunr6Hn+L00Tz6b*?I_3C+9E*D?w}A<0AJWuP{bOOY7X{GUc>iQ7v^{#M<=` z)_sRfS4-b<{*Ld0H@xBrc;8Xe;6gXP@|`bz>uX=YNmne@-EUp4 zi(S`h)s)G(M?K`>-UA~z!3th5gB$FB;0HrE!V*TXJb-)O3u8FL8s0F6zsI#Q70!V3 z#xkBUjcaV<8{;^~I<7E$JM7~h13Ab-&TxK9Y-F{Z z7{vfCtbmZ7iuN?CTkYys!`g(Zo;9r_t(9Bj zI@h}1HLrW^>t6#q*uoz6unE(D#R~qDnQV?VvzzVgXG2>|)oG?M4I?vS6Fb}5-Zr

    -md!6u-pFFq??{j72 zz44mgJm>TN_?n-H@}eJowk0&G<0C)$&GvoLVK4mWkDmBtS3dQtZ~Z|tKk(3h-#+D~&+_YoKm6kV za`xfQ{qqmr`^FzX``hpS`j$WP=%2sXYajLRzd!%_-~UtN?|k|L;L@od?(ts%8sGu` z!jb<0UjQ=T%?+TyAz%bb-~`H%*2Npw{a(E_;0Bi515O{w$YT*`2VHIBC7iwD;a^V<~ArE>X7@}dYiD4PCVH-~1 z8Kz+z(w?!Y-0eZ28{%Oex*r_MVISVw8tP#n3gY$gp&#~tAs`N7A}XTp6`~==AtEkf zBuZlAHKHS;AtWBfH)!G}a$+ZX;wOS)D2n1Jl42>E;whqHDyrftvSKT`;w!>p{w&Jk zEYe~v+TtzZVlL|9F7je8`rW7k<2G_*H+thYf@3&}<2aJzH7cVsF=GZkqa#M+B)a1}rd=hjqZPIzJlf+u z4xK#CBNNsmKKkQ7_TDmLVDmZSK7L_80%SreB)jROLE<1nE@VVXWUMu$L!ux=PGm-E zq^4D*MRFiUZe&P`q?~o6M>Zfxj$}%zq?eVXNd_Q)O0Hx~%B0K*B+_Z0OVXnX+8qJT zWKa5}khLUE&L2$vWKkNWUInF4o*z*jWm7uknAIc=HlqzH5l>&K+y+W^fAUO!TI2 z#vO1DXL7z~aT+JsA*XUeXK6BLbJkyN-rxFvMQ3(uXH85ebu!&^ZfAJ@Wp{d~&Vi?R zq92JKkFuX&Ls^pIXskaSa0ubqPJj0U;$20UqmwKsi3zF5 zm<9)4=|r5$X>j0@nQkPSrfC?e;4?{ofCL!8kP>O3CI@HT>2HXZO(1HbDyp7ZXl$xu zpRQ(n&_@Co00C%fjut8b4CJ>aVJ*M6znD@>&W=KmyDvtww+Zgg^?cU^DH1DpMfo zu--?=$-dYGqz^o>yFZtGo_M!QRHaDrC6!X$UZ@rY3*}+Ztk9|&IZYAzOz?Z6I2?0PQj`fljXt|Zp(T*j@Y zj_i%@Zo3BWLVRxQ4lnV4Qm*`2F0tjn!_uwtsxDTtZtpg)LOieQMsFk*Z&juM#OAK{ zChmJS#rJ|Q@P6<3E@Jr#&9!T$z9X(Yk_O27~I!2do86#Q=l8wv(1 zunzO^4*PI$rY{}SFkCvITE4BP&R42!b1QF+xbh6|00scSK68 z@k0D_CI|vg`*9`3QfNju_6r?g6!8~56CMXY#K|aVZlrBR7KAa&kpU@eW5o17C#uWlk@*p@vmt1Dkp?wAMrs1^l76o5%BP5BSaW4w1OZqXrqK|Y%|IZQEN!{a5uy^ zhcrTeJivLDH0xw87llBO2w?dS`b;AGw zXZJQ=v&@Py856f*w;^KtAq+t5Vl#GO{;lp~_k>GJ@p=DBETQ)QmH|vN$V`hdXzOxP zi!n_X_=B_dMP#=?D+EEq_97c{BOo{r8^J+;RCq-nM0dk>CP;P;tn)!MLhL_NcdduwuI|A6H7oDICzJTaETW-eb@K&5@z+{ zS`Os*#Od#8M6`jEhK>Jk3K&6_lQ0730T&>0MT7bO_CQE1G6}P>jbDJUxB(*1wnNN+ zv~2_VX>0g}Y;Xi{!4>av18*{NX97Nd^g-ZuInVdUxU^;iw@17*M+-|H;P`~k_z1uG zP0Kk%5J3`jH53>@8N@|WZ+dGl#Qq|yGe!V;n1_UBGXNWRHw;L%8?d?)uQ>-dkQ7IN zr#E_&J9!z}cN{7do3&OEtf5SED84s zSNIfn^hK+4H~0HFgLw}uglvyQU1xhHgmG9aglyLaK?nS|EBr=#H&uV~uvGXeKl%uR z_GJtB%DQ_?1Ukq+_{jh7hbR8TWtXzcD@4qP_GJ6^IseKh%lxnWwn7XwxFbYB(|RF$ zy}h$?#E;>xgCWLe{7U03QlzneaKC)5mv%+e^$BNnN=PxZYqo7)NHE(qAk+K8E4+^* z@=`N`9sbt9yA;V-g?)BbWpJld0?+Fv2tw|!%O*D%L3E^NRw zG?&4@J-gj2u|`jD3OM-bw>7_iv1UgwAbT}JfW1QywCQ6#;nTE&OSmAb^|0XcHhZnP z>oB{wdvUKu&@;dDk3a7I2SIs{u=saB`J+4b14O1wlq4|FKuKUig#wK*U_fMHL`sbs z2_(qi0z!p(970Ses7w!k4Gu{%a3c_kLnskBK8*MhW=wxGY1XuP6K77HJ9+l>`4ebR zp+kulHF^|jQl(3oHg)DIM-cWwj%26Eu_+t#evv17r)=$b<7)0m52QcB^F3*$zA5ey=1X@q6Lk|hrg zguy^U#f6k|I6QedVoH;YGquc^sR!hRs!wmW?J$TyF0xA>+#NyC#S|J@7teIlcyY(F z7f-IN`!-3BsArqD{`fh9Xp03qFIEkM_{$_J2NnzwzI8{7rK7LTM4P4hvdWuLd1_5W)y0R7(N^2!LTiycX;s!5j)ZEHS6% zV{AN(&|An2hdOJ?qmM|6z=$&Ppolw!D8WvtMIDvYQvT_>z(Nd7m21NeKkO>RrW}1KO9sk_X}RYbMaiM_=&UhL2A~{G$Oxd^ zl~?x|Ylwk=Bw8Bo%}kOsQKCkOqa*~-@}TUbi-`D5kXd2!Qe1^Pjy?V>1PfF6_^nnGe3h$MnNnEC&*TjEB|LS1m2bTtU3_jw26Bjq%+T6wF`ktPVFOneoOMPr;lEE&ZJhi=}eTe70D)zyo4GNGHn}O?$dP5cK+E%Wfu$bMS79`5l^n8q)hi( zsn#gb(lzI$MQ=TU9RK`9g~7sq3-nK`a)bd3h%zUl;gN|@T*!kPprDzy zi7sKuVn7ev;07X;Bm!Ir&gvKd12^OW3jE?4yoBbRl7J_6aH1Vt`25BZx@$YL@02)MhS_$>pf+e5@m*Lo_1+F061kWz2*dWO&idArq%L z)#*-o`o=kaMRCrI=1}oiAp-Q}j|*~DHdD6Em8H)jf^b6_J{VHT2%-_e`A8muGp{BJ zf~$Ih>Z~Xs2(xN+tY1Rt=E^#M)v7Y)tZ3yNSJRrDwlYagTEi+Zf6;V_=Bd*#Cv>LeRXXAe~A`z+Tp?m(}cM7qeE*eipQ$P2XWjTUyhe7XGzN zNo-IV>sZ$|j(-G@oMe@HGNxiqbE0MI*>u}msCw47!4>Xs)t6f19v8WP$u;b17kif1 zKDT2zFhC!NI@#L}QYpk0S7x=_-4OxTyWthD4V7D7^PU&IY-DaTTMJ$Hl4_a^AgXo0 zLcrXPS2#NQCei+9a?r@HI+}#rY z7|1~u@=+43AfdJx$#ls<{~SwS!`Rq>J(eU;mnLt6p%c!>pGw%lgcLr0|+|-R$BR+S$5|346udZt;?3>)vp8y4LB| zw?Lr1?@_xO-~ktSec(N5dAFF45qR&tv6~cs`@7!{NA$ocUU7>jrQkjOICzUu#g~OA z8=IVNI8r38?}(Rt&=*(v%2}S1jBmWbvBflW8D8O}yaMHa9;7c)vd$ z@?FMuinSeo@4!cM>xB3EvZI3c#>^QU(8*4Li~_QG}Oa=-h3-Uol~#eaVL-yi?^hhzHHKRJ5O zcH!VJ{%mjl=&$$m4*?TU0T*y$_^7U21C@;iZB7THPzt9I2TKqLR|Qq7slh(L_KpwNxkV87K2C2{t z&kzmI&kC=QRANh27!3YC013ZP3B_;*JunUPP!IPo^45?IMg9f*;;swlFbV6B31yHB zp^y(3kr5kl=y;H|s_zd?#oJhH1!=DW2(b`(HNueKq&9aSPd65 zgx=ne7ctQl4Y3JB4it|O8?#XxPwN(&5ibG}$ym`B=MWgHae#(#8`Du8*KwG-(HrZ3 z0tJ&y9H)^J&9T1Fksb3afV!DP<9($1*A`8WA z9)FP<5%Cl8ks&iuBR5h&9P%OC!u2GQAStpQEz%k>k|X|8QYBZiDm?Nd*P=osFw7>B z94pcjPf`(CQYUwkCzZk_UlJ|oEd`^0u@ys9+k2! z^HML<@h$%`WDKCv=CT}-@-Fw%Fb{Jb`_dc#FEB^4CeczD@e(m7lQJ1mF`H3;7h6p* zZ89<|(=$Ia6fg4>L$Wck@-geOGeJ`|R}&3Ia}r~c*ftX~xso+&(>8C=HQ7)oW79Fu z5;bY_HiJ_*6EHWgutKu0G>h^yweldZaX6n7I-jrpI8AUjdDAp~lQ{|V37>&GyVEsOsAAg%T(a1H0%Dd)%3zbEMx(6 zbTHF01)dZD2*6D56i-VLDt-QcP%5VqE~ZNr^~|W+^i8jGN52#XLO@IbKr-{xQ6F{4 z&=ls>G|W;U1cXFO+tf|rlrsrKQN{E}AyrgIb;~4m;{cV#U_b*TAX7DUx-_*=UGz}5 zGLYtUQArh7bG5`wRo*BSx^f@{*i=&!AObER1XN{My_8nL^j3A%S)Ubu8JB7LRO|=+ zbS+Tzxq{VICE!@yLQ5qe2bA?7*$+yg)m+cjxOkP=e6_K1;K8;uN#jBc)< zU-MO`)V0;Hm8f!HP!m7{hO#Z7nWg>iC-y=U7^Wc+f;`h1Ys>FOaY)+ z8%nvT~0 zrm5amWom*Zz5BcX)~FIQ0j49mjVY^?9?`Yoj-ss&jIx*Ls(CS=CZ|%Qtbk z7n-IQe~c<}N#=UZ7k>8^eOs@1*Ed!2=Y8W>f75n;@hCID_kQtLe4F!k_t${+wSWJI zG#dx9lGl9)*j@|xfG0Te>H+yo0fRGGgEyFiJJ^Fi7=%NAScFHIgfp1<&Y>QrB7r-1 z7fGgp?d0o5G46hmVSwROoGs^`{7H z&kJ94e4F=wi|_t;?cM={zu_MIp^tN_kH5hbUZMFicTcIcPv7yy8fTBFDVpLiE1B1D z@tBgWPLPe^k9n$(jlmSY;VIIXaX9y5^x|`)i3{a;IVCxEE18uk?jAUqkNa4b{g{?# z*_LY=mv32@bD5WS*_V46n15NAdAS}gIg^(Hlt0FQdp+jC+V_s)7!$oPPE)g$t6Amh zp^W8Wn728YyP2E6*_*=|ocmahU*U;W>yRt2T5(Z*KL)a1@0nP!0$s3E@z$E}nd0OD zgX`fQ_}QQP8KD1JpaYtq2il+u8lewbp%a>+7uul{x*jlC9t=4dr&XO385ftCV`7VU ziHZV$Ef53#DVKclnWOrC-{m3tFNnT3UM$k^d1@WHP!258Nnl z0B^4V!|wn~ny9Vq9!|O*l3J;knyH)Gsh=9Eqgtw`nyRbXswMiEF)p2#aoZ+QWJrdT zjTeu6I{t!MsEZn{sqLuu*{a*xt=}51({-u@26?ctm`kW z2Rqf)+7#p(u@hUd7n`Z+8l5ZJrZ4&(g_u+>jm2V*tOI+f37fMS&9Li19za{PLz}cm z+q6p?wNG2MQ=7F{+qGL8wqIMeVcW5h*{+%K3g1?lnJn%$Pvo$-vxB?D3|k(G+qjQ^ z8@ZEP{<)W%xtrU$pBuWPTe@r8CvUr}uTYd%CBBfX@PymDw~V-@TfE1cyvy6Xqno<5 z8nPpCov%=%OQp}a+qa)gVZYnHg)F?!Tfg_4zx&&`(;F$0*;LwjRp3s(HEjmzdzJ5- z!3m7G-$5QAT*4!q!YACqD;&cwT*EVeoWnQV!#fe$QW{b7%J!)F5TAQ z+!VCAt`VJ2Uz564JaQ-v(ovm%*ab_~*_;x+{>NZ&)wY3y(QWv2aKSmI_Mn6JxfF(-O=6N>mA?kUElMa z-_L#8fkFTTzyPW}+qfFVtuM(}h5bBE*k7F1z8&Jh9fRYY;w#?bFCOE68{te$p(g%} zfugj8;(fUH}LHBYN#c6`-NyfiLfGf$vHF?*AU` zO&;od;&3h?0$$ZZU_J!4{zeD4R2Dwe?akrqyX-T+jeNfF?Ouii!5_r^;{5?nYs1h-8S-~wQOfaU`)wr-yOjWsXq zL$1^{pZPcc>~TNdU!iTFLG&-a`I7n`=AHpWf#Qw9kua$4O`$zB;2S9Z8BoVI$^iN) zUiWumy%2-(;kELKjlf5x(dA9&n_vG2hx4KT76=6(>YW=%u%N$#_XY}Lz(CZ%ZyFdf z;49#$G{@nCD?f1`!$=GnGRkrGhYH$|PvYl)QTfbL!m5v!}lc z7g7jK0i!6497vPq2ob>mfu{r#Xvh(ys@1DlvufSSwX4^!UUNL93briP5M#}%UCS11 zkGF7@A~gzQXbPY{la8z(fBw0$<;$2eYu?PcbL4ySYR()zvk}7_??}g7*l?g}hNg5H zi@>nVxn~y`KGXS{$hUK^C7jclhoOews*~5e$rz|zx^i>C?E&LL1gKFZz-Y_8yZ5Ua z2t@eaRV;xJ)+46zyJRL1}NZwfBu0cT5`#;<_3ZV zvF6Y>Au!~|LGs9R5G4#r@C-D$H3V8c{=5SP6q#&v(lGvfcu-3W+`=Gf%i*+`b3;KV zS9?7Kk%I;jU}t~?=&dK@dQJs7WK}&faO9HQ;rO0W@m)aRlvGw}<&{`wspXbjcIoAp z`59=+H)NJ+=9y@ke_8$<15l*tCYcU?!y!u0)MKVR_2d?TMiDa9gG%4X=#QMaX#mYL z#w4l{7Yya`=A&oE*j!LG8WkOmSfwC=Q&2?^0$EIk>J?fBgqlKHqNeJVlXAf~lzd>u zD(kGY)@tjmxaO+su9S%h>91sp!cYpoKJ!eOaT+5512EWme`Y;FFeJ)5!$>4i1W<^A z=Qqk)>#am(01K(4Gg2y)jY?sv6$~0AfSpvJs;cUB8ieZ91@h*r6{||QDio}|1}yNv z1Q%@Z!3ZZTu&?4iS}g+_J}N8&Ilw6`x4EUT(V6E&+nPVnB=IduuYB{$$lr$Bu$$y6 zM&r4b-q>DPe@wmpJC(jnnisE-7Z6Y?&OisjZ&Lo=C8fehC$03-OgHWH(@HbEa+=0S zl*ywPF_47C4Sn;>vc#t8u_)hI=#MBQ_2iUSZu#Yye`l_FVN&01<~uCr0F9f_M8tp~ zT$BBcp2nbAI!YV1zLPwxF+j=c3}_&zPVc-ksOM(pjo9AkCZ(=gAqf5y;f%Ls-gZke zKmg*+=j(V5MFYI~_S|>x{rBL9PhjW3fAh}73^$$(jV;Ol%u-2Qk_j zi=x!yfB*mb{u@*I$RxbO5U+2>`&(O*haFGtta@ERi;|Xv76LpFf=kN52P&X|4th`l z2MD1EN4S6mi0_0bOrZ)_$if!tgnZ?@N1qtrhSAASB3y7*oX&y1Mwm$yw4vcUZo!ZU z*o{o*s0aUen4u$*=@zt!z%T|#K%4Atc;@0;e^D-_K<+H%I-U9y2D{=eBMr$Z^%`TS zRKURxx^aYXJU|IuNXI(b@s4=R zf2oabeEwq`ML0)2`tp~+45l!L`4}-l8%v>j+y98XDJJF%A=$bE<<7^RJ3Eu!(~u$wj5FqaGA>w?vj@# zB&I+IO3;EDw2sD1r$U+OJK8PFTPeZGe>sp@&C&Uz9(jmj5xFp&i7GN=-^u}s-eFOB zlmViGz{x^addzharkz|=Wd_?dxSK&xXG^*c_42tMekz~>DhL7*hFX=l5UnUiJE&5Z z%G9Ph6=?}&DNB8K5NN!DZlW+oPDqMO5SOvAp5axlc9TNLdJf9_B@ znW+9$73$K#!1SGONfiu~H$7JZEj;5n(y1)S)Agv}gB9q35r)dxqPpj(bPs;Wo$iYGqzOCDT)f8e+PPVRF4 zn`>YOyZFU0j&U#@JmDJK_{J+;Z(!P+!1oqTxAPShPKT5v5|f9x9{g{L8#`d;LN=)~ zjlw0)Me+p;hGTEpFuCtyu z&FM~ix=MWhv#3X{-U_=ci^co0kH^U2-P-g!q!KKuBF&vizZuQ}Uh$Kk`{`c$`q#j= zjF>h=>_Zv5*vLM%vXjm1Wjnjs(0;bGqfPB;LnF|B88mKR8Q*zAf3G|AI+D9W{!Hsy zd+^pM&h?ykec)ib``z%4H(-Wsta{t~-uTY9zW2@Ve*63101r5}`=aea+fq}5^J`hu zte#kxo6YC0w7M~kJ?FMt-Wbog#(_=mfP4JoAP>37|2=TN6nxc%%|XvxUFNNvIiA0L zxVhadaTTvS)4blef6jN#a~XGhdZW5p?rUQf>^D04|i z9M?9dIFxeU^R9dS>zDmG(Z^2qvg6z6dO><#A0DB3Iz0#uPxFYY&K6tO{L%!jc#FZ# z_rCj`sbV*~;0F(MwC6?bIjTCs6CHJ}gj+~G3M}30e)V)`e?9M8zxdxb&;I$&BQ!Lo zAiU^D4|2o%1o4yCk@6UzaE6n|^{EGo-LZ~$C*xf6p3lARcW-6Tlm7R>-?!;|g8JC2 zoKpsH<;x*Iz1UNp=Cqfy)-BHa>RbQ%lM|CM5XJrNd%yeO|GxOcPyX?nzx?Pwzxvb9 ze)X@!)Otg;e{Bhk^2slKw?=}xkOup9=VN8|mREap#&y_NfChMgwgMW%U<$*aC=U35 z5Ey|HIDr&cffjgy7?^>^uz33)po0uJhJrYVgy?`}s8*hZhA@+as zgPz!nzW9rM;fEXX3N=xT#(0d#n2gG}jLg`K&iIVb$c!a(NQu~rsYZW}$b@m19@}V( zTTzJx0b zja&hamxqa&XO0GWke!H*z4Iar*^mzTkPtZvNgkFm&vvqm2I1(04LkOIhV z2sx4@sfFmsUA>c%D!Gy@*^(~#k}w&QGC7kp`I2~2krug#uZWEq*@DftN^|FtS^<)m ze>i}+SCUAXl)QJ6+trj#`IJx@l~OsCR9Tf)d6iO0k&;n-a#)M%F?B-O6-23v1c`@A zd6sBdefO}IY}uA>`Ic}QmvT9mbXk{nd6#Ty8C)rga>r<48IZUbj%CSMi zmr}@rW<~&1_m^2QmR6XF1c;cHd6}45e}T}hY|n)$hn)!xtz@D z7`^G6F6Wrg$yWrh0D(D>{gMK<2>=ApoZk7J;F)dm$A~?eKp7@!cyZM~wc@?hdjVTZU*3q8W;hwk{XO7h`^og9Msh3{69DFr$>C{dvANokd4i3v(H+F73vTB0V3nG)Kc|LK2$BVWrC0T&PgDG)CW zREY>WV9j-$$$6qY+M`ODqR^S1e7!CQrRE5v zzPX}wcYIKhJ4oskr%@G5dU-}krDR&BooJUYsGO&#VX3AM=cm)tnNxV6e@%*c7l52A z$f%s!sh;PkLdmCVwsMntBw>no*;$`a>Zz>Ss$KUobg`4$HlbBfq5G8roI){7a;kRN zT$zfTisY)oI;`=AP397-7-^ogih_@6WPX|efhr`tN|sMb007Vk#G0+z+H0=>tJgTI z6Z(%vhO1A(m`gIPlW7zJe>x%C+OFWtX;aRmWrLbmIUt_ zumTHXL=dC&ijm8D72+yfFo0<0x<>sPlDDT-16#2c8(qTys@{mNJXJvnu&<(mu7#?1 z^VxJ4d$K6|RMj!B7l~KON~;UYQ-VWgKr^j!3R!ib0T+-5DciF?e=AUBfT%LsvLg7h z{mHT1hG+_>DolE*gIO0w!?Qm-wNz_If*`aYSg-X5ozjy=g#`mnVE`a2FHP&OnMS8D zK(ST3wru-7!$1TQ03<`JlUlovz8Qsu<*}4%s+Q`sPb(Ax3KT}*2CooSZF{(g+cf9! z3XK2-aT}u{Ft@H4e;wtSu+sCeV^l$q7Bpr4%VbVkU{EU*I(q?eTe_wj0jQh0s=KTfNqMy|3gs)%f74$Vud9*OIx$&mINP~i|GnEL&3q3s{t?^!!isMG+e_r ze8V`L!#cdfe>~j7KK#Q#9K=FA#6(=g{ziPnNSwqwF`yu}iLx)J;o6#Tu*lK?1zsAIgtGQ7bYY!|Lrw>`<4ZwNtx z6Rm|~Ybcz;Hut%pOUEw!!f|}ah@8lZyvU5)$d3HTe~=u>l03bJ0NRVjY23YQjJj`p$SzFB$eOInYEQP5 z17I|^eSE?x+@$hZH1}J`oxIG<+|17W%+MUo(mc)7jLDxILQ&kx+Pux&+|Azn&EOo) z;w;W)f4sc}Fv}Jk0d9Q7h+N27yZ)1vJE3_TW{4)t!|Y~%{Go!`D#)D7g>1;x9MA$i z&;(u327S=ge9dw+&J5kq4*k#&9nlg!(W_j>v3$;HJih9j#EbmTx_pi03Br5&lWLR$ z^_)+BT+GHyaU^TMfcwud9n&&B(==VvHht4Lf1T4hz0*A1(?0#vKpoUVJ=8=!)C#=- z2%XeQz0^$I)K2}>P#x7$J=IiA&~XdPPJzx*kpOX<$uRxVAPuf*`V{sIG$u{fJmtsv z9H-P;zeq*LFJ07fJ=b(y*LHo^c%9dJz1Kf|)K&f0fF0O^J=la@*oJ-Bh^@(_OB-38 zf6)fe#v07XF%8$T3ePwq(trLbs`QN3X??$NXL5CVt{5p5iLL;w;|cE{@l`4d4^(GbmxOfAmV8 zuz1Ii8MBPj;3U>x9`xM)Dy?p!P2sT$;z*w4O1|Vw-sDc+!3K~3Fd70N9nuKykw_afK2BoI{Zm9R6>Z-o#tlsLb{_3zE>#{!Uv|j7Be(SfM0)Bn~jNa?M{_DUV z?7}|m#9r*iu0=X40f?5Nj~$~haM=@zw}$o$zzUmtA{sD1x?C$>V@E-5-KJWBi?|F{uj&9Bc%GE(}<{o(+@vASf(wCzwiv-@DBg*5FhapKk*b_@fLsa7@zSPzwsR3@fr{4_8#&gKk_7B@+N=s zeBSEao&q|n??pQ1drN}zf1ED^zqz#LQwX2)9>4QE-}65I^FSZ;LO=9G&+&Df?ns~X zO270>-}FxZ^iUu5Qa|-nU-edx^xdw6IdB&duVxFYyPiGb$|JopZS`<`JBJ;AD{w*ANry{`lMg_rhodVpZcX=_#eEFTW=jM zs`x;{_@heFJsuS?FLReI=d4fNs{i}IAN;~U{KQ}U#((_CpZv=I`@K)OyRYL{@%l}X zp$rhCxj#mbS~x35fBUib{CU^>;2-|tKLD{nPQU)-U;gHQ{^+0n>c9T%-~R6Z{_r3F z@<0FdU;p-h|M;K(`oI7D-~ayq{{WFg;6Q=}4IV_8P~k#`4IMs&7*XOxiWLp^z?e~E zk02dAegye3M~Vv(3=lwx(trdFFhahB8B^v=njr`Qt&#tzN~NRqIx+UA=w<8&>RCvSrPNJ-U%7$Q&UqFi4q_!G#bx z*52KDG6decFi`I08(1*Sv>VSJMx0pjV#bXfKZYDx@?^@DEnmjWu&~C#A7N;aa5C=7 zx}5{tG+-KNj*}8pzkfEp*>GmswQb+Vom=;A-o1VQ2L4+%XV@M`zKx4ALd)Vn;R0?R z6K;a&)$b117+m{y?%lnA2OnPic=EfkXS5#jv&rPkf2IeS0;OH}A0^DMpFijMwB`N( z2QWYZ2PCjS0}qUfzoYaEsjbkGo9n&w?z@1&AbN7iLJeP1uzy1jKLjyE5l1AkL=#U$ zF+~+uWU)mT{$GSKMj23a;70ilb|%!?sHAQ9EfbQOgW1hvraqj#4}Gl z_vEurKmPjgMqfd?kIV1o}vIAMhsX1HO8joTJc0wPc|K9uz9sn~}%=D1^z zKL$Bukw+%EWRvaHfXHqijevj(>r?h*nP;ZCW}9!uIcJ@B=DE1GT22X7my(1J&Yq7( zI%%bsW`DYAr=NzJ(QQW~AkGO1FhJv|x8}NQufGO6Y_Z4AFJhEbwN5?)&?dWWx8H_4 zZn@{CyJHw|g0In+>*l*}zyAh2aKQ&(tKa5Y#+z`(7iYY2#~+7$?u;X+ymHGg$2@b* z3vSf#%|8b{bkRpAz4V>niokT$S7*I-*I%Do1An4nr@eODZ^u3NhhtW)2Nf!)Kmp;0 zC%*o8DMp>#xT?d+oR9zI*S#2OoLhKhRot^Up{BU87_B zAbj`Vhd+M#=cm7Z`|rm;|M)8?KYjoI2VkLCl{M4kKz|2BU;-7mKn6C@fe*Z&3IaGm z3V&A6LQ6tcUka*-vLJVa%LmJl5hBw4v z4t2Og9`?|OKLlbBg*Zea7SV`DL}C(^xI`v4(TPvIq3?o6JPE4MidV!UOa|D$p-fSD zAAn*Q#W+SXmeGu7L}MD&xJEXLQ3b{0Vt*FpI7jp~DuaA^U`AS~)(wDykW-yD`$wb!Dn8&XDk?1ZOzKInE1ZY47I>~zC)1U{1n%X`|6n75Pp$~;A<5beF*~Nzn=Zv^PIVeh{+Q0x zr$0rQBdep*ofg%oN6lnUfx1+t&WoGPAu3X(I#sH+QK?PEYF5$38UnC$s5h-@Sj9Tl zi4ISzXGJTzShY{6f|aaqg=<`Yc-6Gl)vg;_65EOsRk`-nuX3g9T?ISX>VF6|J$(IZ zVih~pz!KK6k3G%oY}(JoR@SmeZ7gIpyIGyw5|Z-cwsvp)5+#rwDiL=0Lv$?kExT8(tX-*uLUq000CKgf>Qa!s(4Ido_GlR#AsR ztj!@22mpWtAUK>F@GlJb>t6uNcf=)5Vge*g;g9zBzcUoD0Zcs22!G7DjXl2bK-Ifp zB44zoABJuZMGOEPzcU0Wo?#CJAOH;nc*IRcB9Lk9Q5&~7h9bVnj-{#PANyFy87^{~ zncA{P5|^?bmLZefm8Uis6@mxcs=r_AL zG%yTua9<+inTPq6^naEcjLI4s!P7oOFP&9PU?XsvhDM0AU1+#y9DDxS#6*bnqh;u6 zAot+Ypf)wEVSr#%FF4OZNVTauo$4J|T80Lmw6JIBY8nE2!J1A2s{3r}Qb+OGHe_}Z zFsRQsuUUslUjBFYQ`pdCCqO)OOYZ{W-+p?~qy1BjVTYqmG+BlB%mW_RCV&72S z;O^m|3q5dTW^_Y|z9Fk)=okMs+$T0n0eu?|0O^`Jx)OJ|osD1s|H4>?Fle})btvG8 zL)pe5ST>4F&Hx4{LE%YIIRH=|!yfc_;#s3P#v3jGfMa{a8Zg2I{2OE<2z}xL;O?m#C}o$LgMTNPhYA-v(-HjnqYJ)=v~PHo zlK^u6&fMizmw=MhZt_0IaN}PO`IoOuV7=>p_sA#y*KHqS`hsTX6kam&oo{30KUn$D zcm9(F00WHM-Qo-P@7bRZeF4}y?h^-aj_+#(tRvs|*@wRRe<)@4Z<+td7XY^7zW)0- z=&LjBGJin*(>$m%HQocjN5enl(?7@?KKN@et4q9NGeGb2{-M)Ly$_@#&q@!`V1N>o z04f?HR+~Ei@`;h)IRA3K6}&o0OS|ifuD?qG1A9RLU@*O7Fq`8*9WsLJ!n&?HFUTvt zBrw0Mqd|$|x!+^5`&z=J3j&EVfEJsiWpj%&K(ld!tDn}(w< zBUri?gn=cTIWrUhG^{`FqO*x(fD~A|&`ScNTSKdB!5nD5B=|1#qcRPkE>_b!257?M zgTmYs2)|>y9V)wqy8waEJFSa043q#&v_Z;)K~Ma|Dl9xKbi&8eA?};QNNl*7BRmO^ zG=IH|LqN%MhPG_tZOkdw67Hmz#yVC0W7{AP(vJ( zxI|n&BUnO_>qGz2MB~G{;EN$33&R@1KYyH%A$HU>m9sCBgEu;hfEYr^8B)klo46FX zuN9oYC3`{}ld~jPK`*m0=Yzr~lSdfDu@`Kz>}$UnvN=A~A$DXt-djl-)W;l9HoIs-_KM6i#<#g$yVVuVO7lt@dQ%JZ8@hJTzP zhg`BvoZZ6MG~t= zAriF}>pY*VA(gALB`YzcoFP(t!o;L60k|)7yGE&QFvV0h#k4dU`oBpOuqU)RfRrIpY_gD~p)SM4+U&E-q_|UiK>#yD(8NrM(@f5E zNgX1dX;cXlya2jLf%YWJ_e7!Lf|TS?%iK!AIejL zKrbAkzud_dvpIb%>vnafWSGcof^I-?W0SA&72Tg}e8}h^dgf{CGJ`HubAV8(&M8!&MzvjF* zB}KxlvrrJVx=F+#rX)xr4Sz@|6GIm@zE!Ky;?u|(IiT+jB@(;S4)K2@DV(+{9YiTaE$A6hgSwIQZN)Ea_J98x(4 z%^|&W%ml?!(aTO51JDapyPsS?1H{rs4Kc10w!QRJ02{!D>&*KiReu8nFgo+i^;18M zL@))rfcHy6Ak9%v!_<3g(fAuNqNGq9{=G)GOHNXR&8Vz1{ySEn^sxgqKuEnI+uR|W zV@~}1vS6{*&Ra5IRZVat)*7gLYuJQ)F$6o+aizH>Ji722S>$U~iZwp2qcGkRNsj%x8~sLueL+<0RVX}G zT-_lm#UcLdNMx0?!V@zxlUeA4RvRKy9WpmoJ6d2gOuX!~gn!*J>rC01Em;7RQu9PJ zp%@80b&GnH*AkQ%cs1Lz4VtkXTmE~kS470u_B=~|mD_?@Jq8I_R+$fiB}n{y)I=rP z9z8Xe^`S-W*_BI998yqAy|LqySdcBer?WWTi-2^^*K~cfk1beh{jw8%+Z52R9K^Q} zsJ|I=u+IHl(SKyw80t}@YdIP0s&#|4TV+jy-BqQXHqBMs0sYx;#Ube|+7uJ8WxdP` zlU&N()6d0O8-m$s)zC$A-InNq*wt3ttypgzuo{xq(tX?04c*brRFUf1{PjeUsqt&DMb;0+a03^CG9y{4x_};Z+;WxeDKTG2YHK-}RIL^ko3`W#2iK*Z0j+ zJ*D3`E|X{kSp5xx&g=;O&C({5NoWgBx>Po!17WEfIuqkP{WCiIy2Gv2AqWOi^Lt1d zGCB)hVh7|dCS5jBz0yJMUlE=)tb%9x4Ism5On#Ir8 zG_jK8!6}wv9{S^Dd%<1)>&_Zt$4Yy_S0zr$n?58RNX5Nn#(d=`jzW!8HmAhp1*Obh zY_^+Z)&rE}6dPqFOx+yvK#qCau*K6eme)0gkUPCsxRv8}Rt?*d#N$4!PdgMa1Fj!sRRE^c%VmNLLJu!JX595O_aMmZ3Ym!nSb2 zB9%Djvts{&@s=t;970A2j4li`#%Ci@w-Dn=!-)OYij(gaO5LW`U+ZH4S#V~jZO}4 zxDjV)-$bKxX`HT;HohP3Aol-k!{1mL)cdg0?dUt z4v&CGb3n^1!ovk%aYWhbOF6ZZX;(hh1RuPTTbom3m6VRmCC)pjQ#={!0Rz7}=KMS= zBe2y5WbF0xpc^_)gMl5SWPjM4ZVBT@w!6%oZDk<3W2ke9uXcRq6AY%F>)GWpO!dCgAaYoCjI-S(H)tPZP> zSE2d-wKSvbYxGXF$Q>N;r&r<>*Je#J!xW!Xx%%)1x1`uJK#l(+Iz!P06LdDJ`qdeU zl$Xo2$}EcQhdRpW9xX6s?iC^FCyG?Qsg53quO!1 zx2kBDquD>>IaOe~==$6@DYLv^)yfIrFQOg@fe1)|1n$J*2mhlA76#~k1_%MuZ=%=_ zAn7WJ+7FuBcUpPrN2az;59b7(o^?Z0PVI z{=|q9CsM3v@gl~I8aHz6=h&wwupU!{xWGVQK>`URXh`u=EZn$q=hCff_b%SNdMz>_aN%o?1Oy~# zw23n-;>3yveQ-Nvu>?1gv+lI z)NVAaASdPAx_9&L?fWkIi zotc#u0THxCBac1$_~T|581$6}1Q>vycZb2aB$G`#`Q&dh0y*VzFlb<5TJ@DiC6`@# z`DI2kc_kA8TNbD!lxeEDCYx;{B_)_~nwCR+^9`h>oO$ZG=Z86dB^Y5Iq39-|g&KP3 zi*WL}=v{QmmzItSF*+%wm9oa2e^){#0Mej{dip7-q54LIqL!K(mZNs&r~#6vy80@t zDFq5sO9V*j(Fa>qAS$oD`u_SWRVwHjE3sfXdZCdB6}v36%}&(nNjV7g7f3l+Mg*|g zYP&7BI@$Jtv*DsdX9SS4DlWR|s>{)A42)#iuHDKzFTHIFyA`zS>YI^ge|BcLFTeqd zTWh`>-HT?u2`juXgVqi!FvJ--3h8|jTYNE}A`+lMNwz8Q0k#Z_JTl3^9eh*47^}Px z3>VlLLB%V}JoAj@x?7UNmVJ;z$vyl0Gh#ic;PGuV8|_dIR~D4Oxko$wv~}C1+)>V! zS$#FuS!=yD*Ij%4HP~T`e?2zYWt)9A+G(r3Hrs8x{Wjcjs|_yHPm^2FXt&nAH{Wv( z#PPU9%RM;Zg&Tf2;)(t%zBuEJJ5IO?eb*fy-V*37Ip&$eWiw27|=~ z`r#E!tvSXiNN}M6O|w2b?IlU=)zE^YzB})|`~Exd!JA$|1uCpwe>=u7%tjE?$0Pqd z^g(D0c=Xj+qf1jG!02BcKG4yeEdGVoN&D+vA?C_yd}Y=RcNAO?XH0_Lbib{YI& zgrwyc5R$NjCd^L`f0j2837F}GE{x7>Hub_8(y)f7Ngsv^Xu};MXCRm3ArONoM9y5` zLK5)a5R-V4X(_S(iB5dtMmBiA7DBO#BGHr;v#3QYK5!re+#(n?LMAGPv5aPPVGe{e z5CdEQiD!Hwpx!9QIhrs8Msy<`fmoy?<*|=`lwbFnK3Ug0g`cV_zv#smjcW(jcT<<^K{0%URMA zYOQ=^Ef-k9T=KG)7`x>weK|f6QcIY|Jm$JunSfv-vwB!;CN!hTtTdv90eF-q=EjK4 zZF2KeY8=cpf4lj#zRXIT<~(Pd8aa?QqLX%e3}-vzDbM|=b7S+A+$ry=&wYL<1Z@wEc`8_W)dP>wO{>ToDLc)&tat9qV>}Bh zTtP(ykI7W6wNocR?RwG?b=5K{iYs6ZMXIfoRj;#yL1?mSSW^*Hr@9I(WD#{RsxG#6 zMOtS`e=9qqUFxe-k^QWjER$KyS`4p6CGD3`8ZgU%wzX|S5o&3dg2awCw*Gl0z*y_r z-9pK8MgFtx>@I6h;Bx7Z0@E#Tle;9}7M8e}%Yph7>s%j^6Jp3!E_PYO-09A4q$%a@ zhc3FW>t?sS4B9SuHK$JLrgua6wC+~SE8p`Hf404>O95JnY2OIR623Q?FMtt*-u*sz zc?n1=f!V{^WqMotIQdJeD}#VmNJ;9Twg8sZ94{FCYi;8TOofr%x=C4mD5bK955%t za<=}9m{tbmH}m-^alZ4+)R=%Q1NtL_nc1HEtZ1eDxzHXJ2p^~y=+1yfZEpHNd)Yjln`WFD&Q73xyAEu0|fj@{~kA1f70#l zfYqu@3nw4Eejud_az}rwE0s#P!fPxDL2JAay~<~_N2PmJJf800|L zKau*go2nBc44o)gr@GE{Vgw^te|;xh@48RMZt)(B-RvxvUe-y@^R$!QCq0;Y*ppy{ zAcULXXcr3H!44E6O#LN;FMQthUiO_7p4?pD$=J!=gJ&B)>TUn}Tem)StzW(ImM1sn z3%>Whx1A@LPrOlzu5@+P$^au?eUjHmAgA*L>RZ2nvU3jr6Qv#Zy0=xne+ixxBVb$i z0I;S`_^xb<@1*zwSo}|V5N_i?KM4|`w$%e;d;n-5;B()J^p#zGoyf%ayiWoQ{0Sirn!K@!|4~~G7=iEQ z9(VKv5j>#y+1gIFopsQs2deq53sp5*VTSfAt{s>6#kqo(5bX z4(1-uSRn+e-5nC)2;M|(x<(|eRLH@oyVh%)N z?g`&f*k1%tT?kBH^%+3{R-YG2z)|$T1r*66${MyLVH--~eLz?4g}@Yc+#*gN0qB|* zc4E?GBDZP41x#Nme}(`S=AW{a$t6Bu9VP(?vPv&1n+Gyu?l~g}Mx)AU;u0R8C;DF+ zI-;^UVTN$D2J!9@+872Y%E$Uql3c!6_)&JF^vM~S*^uQ-VBl-cOFlqoXYN6$H z1}H*bQ25pYT!4L~0752YLK|M4<)x zWCAFKyqRPYEFr~Jq94ke07M&8)}Hu9+cid;PtMy;1c4)Vqxf+iC9b4MLSw*zAW$ZO z?@`=u2=1Gyo1uO?f_9bN&gd)D;yz$^S`b1h{i|?6YPxu~E4u$VUBTp3HGNNDZ zjbdIJg)8br5=LQO{#*5N+$El4!EqoPlB8W$CHrkAe_Sr*3!WxhK;>{YUvbu8$O#_@ z#w2eBpd(!a>RckBrxu$K}NuRfBq$Xt|^oCcRkeW@cCYcmguO4TrOfLbpodg ze_~~?GFwaD;k-(skUw_w;BnY zE~tJE>S4ZKJCVsl^68XRNPX6aVVcAWy#=DqTli_Hg~9-BCLXB@#l4Pb?lmb=WU9V# z9H#;W$1R|9il?wbr>quZPq?M}G2aBne;`=ip2QB}kuEI4)@l+8ovu=x?SgQo9W*-e30iYndN{E#R z(i-I_p{hh7E=0TnYZ>}Pqt;u6_Qbq0tY$vyP-LT(E(O3|90#Umr~V&GVkv6Ee*j`W zTMAHRq#i{!ZcIHA9+v7|XGp*T_MFtYPTeZ!!kXp8njVtg>fS!7lm2SQLaDgWW*n%k_GLYH+r_2s!&06HmcB{@>q+n7co?4PR+1b!WNPHlRbNDPq zG_B!It>02xQ$=M@EFrxD1&7+4f03l-*oqq-a;+4?roPGBqUyx!5P-E2t2}0&^7;f3 zv})iU#k4J3!9m^gk{k?#CzJpVF*GDutvDZ7%Lr#^j0iL=Y$&&~fZhs9Xpu zrQsH@QVJl+*|2YNUqc15#W44uudl<@CK3 z5MxVAhA+RuE(u$x0rX@B*P_P>A;n$gz@Fv)#jfvAT;)f;v0S$CT3*0aN@wvgn{Qs_ zZe}0~lP}4cZ`>9!R;Hy_f@HwD@5$bq$~mX}4J-#Qhz{w+8eiq^J=+65>H`vx1}vZ~M_oAb;sZuNv}vKoWuu4=MJ4)$1hk?Q z3ZXl8hBv;I$pT?{W};i*S68Z}$fY8~PA~U8oBfGg+CCr)nQb`Q8a}6hKF1?@W}^V` zV*!q@-U{)q`fBqU`h~06b^oGClMoA_XnaQbLxgWU5METI7Nn zW*Re4%mr;Q&)D(aNik!_8td&BnqIQH{vXTAv>1-+^!c%c0y0rJ?8UvN)*2sBn(PXP zj2`NqA0}%wKWDQQubGePS_qUmG;Y8Mac7)K8wu5@uo(HeP>X+UVWfi3aH-~~V*J0D}N&E-w&-7_1X zCF=xEmR%p)sE`;miE7{d(V#@%F-gMVJmzH(v?vU-Z1?J(CGT+!F81&FArYW91QMYP zFJiJ1MiPo=fAPxOIx64fCZMi@?F3R^_dREa&SX}C_R5uKR1b7wvo$(N>zSTKW!k5p z`m+LU$DZPKX0*~^oOE7X=1|B~5B>ycP{{8>FNI89Vqz->5tKKjCI#W;Ko5LrQ+PLc zUt-<`!FRuRP}ui(!*|p@uQ4`tRv3YNpZ5&EMCCD>f3wCGnMNjjQl>@nvi_KBU0e4@ zxDNBYbZT&SQ|NY2$X^}8Y+_OcjsFs#csTIIv01gD0fwLMp-O}NYh7jPq@qg zt#(tmc5o02L5GGjg;VyN^SyV%LNrw*H4d(Kco0F6Z?tZB;ct^)N&<_EFBY`oa$GY{ z7fbi|e?S*s>3CgC6lO`sk1qwnikm_DpLU)&vpU5Uk14brh!Lo0?ICDm3!8*1q1*g`2+yS0RI3i00000t_8pY00{m70sWV4mIpbLZv#G;xsw5f0{)Aa0pbT0 zmsOMj=mY++#+N{q0b3A>Klaq?@4o;C%+nE{Cb zgqO;h0d5HX0MC5oOr&|2K$-#f3I35Sa*dzY)UlU9n*sI;{@UtqqX9jlUpJ?G+sU1qmq4Ha@elrb5bJRFu&vz+vea3#HkUx50q_w1yNzpFhj(dd z&^v#C=b*%wK%xQf5&lSQhp`)j-$;eSp?GS^FzuH>qXF&_{%}yR(l3joUCXr|y;LZQ zSeHMf0qzt28-tuE-8gucpflOaT(2xm4Mpadf;T1pQ)V9#DaO~iel)7uBn6aj0kM>cVs=s18Xk+E^O84xx+yMBYZ`d zOTllYhvjW4If%@9?@A|$n~?9odbj%4Ysr_g=%ktzc4mCg7~NZ(5i4h2k2L6dPy~J} z*z(zPf0~<#7ype=o^{^!6rZj)K77HCb+P*uc)p-TBG!{nPu{XUK|vr9eiq~*QM~)E zsLob{?Vh;=g6uOQ-ZN;r^8OmmAPNz>5Td>!D(|2S8yc{{q&Q3tq5cF4u%6FML&!GF z>S@oe|Kd|Gi1)PPPN5J*b6@$!N zjIR`n%t=BB3xY;Afrf!^w$KjpZBh2K9$)I2p0%7m^=Af?MYP3uZ-%peAJF^FLg>Wm;& zpDm~)25&t|LdX_UY&FZ`%S|4|CKTw?G|kPB!;#cY=tFFaBJojU#wFsMs9d!-D!v|+ zuCy`=I|tQ9_VVw~m>zvMVTBiFxM7DMf7S^}B$cG9z(Athsw)zzv~mwEvsA)^EWRX} zlWB1Jc2kF%4Pv-4Pm@!jHxXti=7LaNH6d#SDkGst&;EoEn$qav6ygYPq=739k?Y4t>e;d|% zf|ZC8Q?#b5=dyVkGwGzs<9W?r55k!XuodOkpc*?oHi)7NQnU=PGZ}YVMHRoLkeUt zwv)(Yn6~S^n=UBy=DYCSug5-nf9P6EFzzZ0E6*S6&Mux-Re?eMf!%dyg zAZ@Eqvldtzra91W`Qzd6W_S?A%xWP!To!Pw#>0`E2wk-E5Edgfk}b9ii%aww4~xfz zZ}<>6%5tC+=Of0q5sZg11Y;cKI7d3xks;oD3i!VB4k3_lW9K`c$JA$*ka?gA?`umf zbfPE_Hl%k2d>Ibu*DwCne=J8Ntd9mWVMc>&ZBUIV-Y1Rrjv`7g56$u6UyittW4(_;DHmg50RYEMd%a)#T+^WX90>N5}}YAH{C|2;_)9Sp}8%E zq$WBCF{-3?=QO4Qf5)j!HK$V_V$pk?V;G{qsxg|G9dN$m8P70=R;7A9s49f1&@jd~ zjByUC0tc;XWvf}iSr4_MwJH{!V^j6|{?xhR)v3>*iZLkS*S5A5tzflRL)|)zVXW1! zODts~!8owW`FJBBsJe=~-W*tyo#7__bM$GYAyxwA2K(#H)-u4e^b4ZJuqm_`Lr0WrG_mNBw?Q!Fi!Ch5@YM+IHB(9|rM3Zp#!s z62})J5P?ZeItG-cl(@y^0dn8s2>wQ-M<)v5Hm9V&M`O1ueCKe-dQO7DwCw1&n~lPL3?uu`2en zYfdj_e+=gy$JxhI#&e$ayk|c5+0TCl^hK+D_{|0!#1wL?s z7u?(EE_PBVvhHL{B0UsIM!X3L?N-w}%=gClf4#HKggkZI;2#Hh$VEPKl9ya`2ro5J z7S3)f`QaT-vN1X2kR^Jzz~-&ac+RPfZ6PDj+3g2)*f=Wn zPzM_%PF$Pc{Ng&-IJSpi4W}F3>t6?Z*u_3}TqE5oN=G%*>j87erH9#Nc{U2FPVYx! ze?9B8zCkd$PIkWc-S2+~eBkG3cBP|*bN$9(2B zzj@Agz9@uOxS>&P2W(vaUOe@%23fp3cUMdP?zP5+={-Mt+SlIpxBoiIDYAPT^WOKp z2mbGQ&u~C0L-E4zy*yHarFzr*%XN4Cf4*(t33w!8F{#FJpu)bi*u! z!#9+}IHbclw8J{Y!#mW&JmkYYfAqsX1jIiS#6V=f6;wcm>jW5dzjFu%MgV~mV5ycu zLp78>dTS{{$cRDIL_*xePUOT-^u$mE#ZeT+QY6JwG{saz#Z^4QE=0gD1SE%Ry8MF& zbub27@B~I61VUhgEbzr&1jb-wgJEO?TTsPgWW{7e#${B-W@N@^bjE0ef5vH)#%iR- z{?k1f)W&V(#%}aRShNyJkOzDC#&R@A8H9&gREKqB$97ytdr${?PzQ2+hj%!~d&I|l z)W?0~$A0w3fBpo>fE37qB*=m^$b&@4gjC3dWXOhe$cKc;h?K~Qq{xc2$cx0tjMT`D zw8@%Gk8EVfLu0yl z{K%gK%Agd=p(M(pG|Hny%A{1vrDV#cbjqiM%BYk|rwmC1%mtUff4$_(zVyq#L`t>< z%)ung!ZggoM9jog%*ABP#&pcbtV_X!%*mw8%CyYO#LUdp%+2J?&g@Ld^vuu{&Cw*y z(lpJ}M9u!xRL#T$&DC_x*M!a3l+D?s&Dyk0&1}uv)Xm-G&EE9Q-vrL!6i#!@&EYi8 z<3!HnRLdyA*uM!2HnaQwb2{J(Hzy$)O?3?_)i#(3aMaFsvy!LHPRzR(j-;VC1uhkbbNnI<-@1e-P0T4bpn_(L43iKLyl471Ti`)Iv4XLq*g?Rn$dg)JApGM}^c#l~g4C zhONv~J$<-()YMJoQb+%#H&sB_-RkHQTdA z+q6~NwPoA3b=$Xv+qjk6xux5>wcESJ+q~7=z2%o4j{_Zlh1|%M+{vZf%C+3f#oWx* z+|A|O&h^~S1>MjU-O(l8{?aww(?#9XRo&HP-PU#8*M;5KmEGB;-P*O?+r{17)!p6Y z-QM-x-v!>_72e?`-r_ah<3--&Ro>-g-sW}Q=Y`(rmEP&4-s-j9>&4#e)!yyp-tP6@ z?*-rR72ok4CExNj-}6P^^i|*WW#9I7-}i;z_?6%JrQiCs-;TV?T8K&VHw&5Gb;T+cC9p>R4_Te7};vg2{AtvG?HsT{j;v`n$C1&C# zcK+fghTmS&xa|q>8z6GRxhEX2nQcl=bcICY> z<;(pBO6UOw(B)m`Wf%|yJoV&Y7G~WQ1YYLlGQiqlFlGjbfLedv*BSxC zk|t>`Q0br+>cXXjht7j@PG@JXX^*~Corc_^zUW4%EVQBoC~)YL9_p;t>b!-4hn9eP z_~)cvXQn=AvSwU<-sU_|YNwEgMri15ri87Q>$!$onXZ55RgP(~9%!?!GN*3bdeDGv z)&Q9Ho_EM)ZGLIGR_w(#TQ^wd9LRxXPJ*xw=(YZ6W8UeG@@vMmX$D9Ea~?7NtJY>H zVC>R1ZLWQX3y5Z@-e}8qTDxXu*Dh<2zU+2x+|<@)33$;wjqG&3=hFu6;Eq~TNM|2? zYhI3Mqdk8IV;JROsEVh6ZJI9G%@$GW9x-` z*mF?rWd>@oRS@kq1=DKUcd%rzF$UqrZ#oWwWhMfq*l%M_fqN*~a}b2U*5)L@Yk!Dt z*{*KB&S?jim~)_n-qz+2_+^-thnk@05O9OvUfO?gUS>CdS=MgtBY6ck$b%Hm12?$u zj)DdiS8+iQ@Y}kDB-rK%=mH-~lgXGq6A@}F}Msg>H{)Xe;Wost! zT}JOq%>$ASf%{(Yv8L^&hVZ9&hcfW#YSQwO&Vd$3+FL+u!p_rrSZ`w{@}@v&to9zX zo@svuXNp%?^LhqzJfd)27HBUgY3Y`jcW7y2#^@wBbS6e<-DV1wUS>GIt#?QPpJsqY zcm}=h=q#UY%Fc zqyL_1d+BOzzVls<>)R3qZT99vCw3$TabABW?WVACASZ7;Qtwe`fIa7PI)89XZ*Xfr z67z<3M@JH97<6c#?4vz)UOw_z&h6cPbz>Isj*|87k@k`n1n9-{nH@ zsBC8TV)u6cL}g)dQJfY-)J;H5=J*>ZvXe3-{F_%<*#-M z+$QvD-)0za14?LwGU#+mU+QH(r6F_rr-%9>8-`uibR-FQV21o)lKL`HFpLryacwNo`4=1tE zX8;WV>*>~JxLy#W}1(_apS3N_So`0j$vgc5o}{24T1nTf`ZCXDdy zqadbIr&4|OFa%SE5^phe{@PF!Q!W$19&N-i>QPZ3mmW=VuOM2TZYZ+V>9ru&qgEr< z!y0sG!yr?WynRSEXr73ajz)lX^D*Sek|$HHZ22esVx@BTgf`0_WqVQdVd?xRK`(oU$B;yDNfMcrAp z8F-sR^&K@#Md0ZKgLlwGa8zP{6e+0F1>RxQ2x=fbGa+jl_6J==(lsU9KmiWu7)62w z1;uy(Z$Oa01EL5~azWka&^P`d>h#}yJ^J_~kUEE5Q78tC+177_SzAk3=r$jh>ga^ zC!-aOCZm*UR^*pXJUN7GLNIdre=WG-iaRd3<(hl0b$_i|5H!qbIMA*MQF0NW1a8#o zof~#oP_p?_6fXm*Et=5&aWJ*&=SH473avs2J5=wC11A(j4i#H`vBkyNSuK+kakyYZ z5^j7h$t9b7GRi5dys}9-l9q=XG0Qx&RKXSmu$L0WYcH$&V$`aa5MKnae<=h>?C?q- z2bGkT3lr3=Qdt_!rGdCf%rr$VFWnHN*^0+9*kOx3HrZvH?Ov-?>x4ByI1gRwiA{Ie zvvT}OZ4tK&KzFoE$u7H*#0>B#tkWgl?N8up=gM)k6{Sn1l?3Ho;n|g2emUluYreTf z*eZThl??%$$=lS<=JQ2Be>WubMSLrI6x%KV9KoC%&0agqPp^a3_>=L_4qhGwK$pZg->>0nDT8G3CvGRunyR>_Edq z-VoH$9iF)FgG*er_|`Mvtv^aDwCD2zD8K;{uz&`v36`X{8`0^{e_v5ET}Qshz26O_ zUN-xe{sdOIVlhQq6FCP^Vi&kmF)SfS1K+cz7L*`B;~6e|Aq-D?Q|6MUiZd#5#!kqarzM-G`^w5Z$N{F8fjN{ zCQ=WLxr!*$N{d7Kf5Sh+LC_yY*q}!whCMb0MJ-@Vp%C+^$360~k0(3S|J+c9K@zf% zg>=J{lvt2A>i+3n5rM+t?$y1DNN^xY3`x~BsLNgQvX{P;&-olUk*-lme>|H+UQzVNvy4;{icjKBeObF#1LMDY=V4MwmNPhsxpTk9OnGUIkG2$uDMAc9s zb5soI&XKo!{<5F`{3k#I+LALC6C(#fUGx|bJBax2l@XkzCMQWug@iLAX}S(URJ6jZ z=|~|w?1(Lye=@j&45Op9X$(4#VoiwTjX)qgO8O)+Q3e#Mp4ZEWMs>mk0hLLW0{tma zgDTXaHl+Se!NbT13vx-jT<{@O#AAV;8PR9X?nc?u9|M~DIGtz{A_=t15xqH;krD(N zSS1ZaX379zy3(Noi6)ik`5pbSkCqvMD^(Us)V=bxf3JT1p^})=)QSMIi32ezmBMUfi)YfNuoW6) zj4VM+e`^=EHdG;G-3Sqzk_W{m%Pv+W6;sOKNb5?4XXgt+W#3^=S2@J47+I=R{^r~L z^0&YK{sq@oDaw(rvQn~B$$==KagJ}i0+sD;t~0Sq$>$=LJPFp~D@3akSw1A82|@4{ zMM2H~fi@%1;HgwDK!jNJV7h3ONPx*Hif_Dwf58N9tcMwpgjdk`9K)DwQ;=z2$Vw%q z|2;C2ldR+(8FxC!v=&2X^cj2p$A}h1IWL~b*WQ532vD2OeCIFr!&6sj(_}LvtHB|-zc$1i)0v%fv=bFX_P z591p>Imed+zx&}6zxc-YWqBKU{N*#h`Of!Y@^y0a=TpD>*1!JBqTem-bHDrE|30|1 z4_ok)zx?Ju|9qmK1j(bn{qBE1{BKfTB1}L2_P;;=^AD1GoC6y2ga-TR0iT!7#sM?| z{+9yB0X=`A6FwmnMxhi=Ar)4k6<#4g$-_Aq!z-AA7k(iahM^daAsLpT8J-~;rlA_H zAse=#8@?eN#^D&gf-#(fJlxk7cA*^ZAs_M~9iBq~Vxb_yAUyS% zG)^NmR--juBQ|EEHf|#~cH=XG;x`1NDdb{CnSv!^3WJSfF|K0@)`Kwmn=;CSJkBFM z)}uc?-XlKdqdx8 zQlLC+qQA+*PeR^3+~hah;vE*mJqRU8`U6B-C01UgR%RtvZlzatC0Kr?ScWB8j^$O> zLr*ScQLg0zN}@4vWbWN$Fv?XubfHp~{v}?TWnS*3UiKwlf@MR#0wl73C1ECDFhav! z;$BligHDE=7k(sU-a}+greszoWnQLbW+rBCre<~~XMUzPCv!HZ zb3P|@MyGU+W@7TDbsk`UZmK2c6(?~X&zCwaan ze8y*alBRZ|r+s>#dgdN`5(IbFLr4Cne+DRk4yb?@D1jcRfhH(|E~tVwD1#noe%_~q zzF&mi-hK`QcN(OIZYYQHV}(vAi2jz}hsxfCTBUb(D2k>ihmL4}h_)#Au_)}7XojjN zjn?Qt!l;Yp==0$y>dEMdawL%6q>u(Fkq)Vm7AcY*sgfoslP;-~HYt+wsE$sl@ky!a z`KWJtqn2(dmrkRVQYo0`-j|XdmYPDBo++Ag>6nHoo5J3jZl`y)XO^ZZoz|%}zG<80 zDe2*+5BzLkvR*gn<5DJxsDT>o*8Ovij*|AZrMJlm@jfE0UyKvf7@bdZC<} zDVG8lRpbFY+Ube;N3d2Wv92rU8EZpO>j@PE6f`Rggut^VtE*;)F(hkg>}$1R$~mM! zvxY$ZVd$e8tZ(|8!r~;uF08{gEKcTFP$a=8LhQp{ti@*Rf-Q+CV64Y#th=(S$eLQn zmX*LZ+dI{NYY5y&Pe3cneTKjGE5Bk&84PR)^czK#=`4m~exv|2{4CIx9Zz-a&YEJN zWmC~=qPmVO(>h(rYQ#GTYz}yb%jPR)#B2r4tfkz7%_b|#HiXV5ZGP|>PcBIrByA}k zEdz{W(LQa{zU`6WDKrfJ!z&C!9Xf*M4n}h_CX7KrT5%*cNTimf|~nj2nDn zoe*s(Vo3zl;km-VFf1AoC>ZN5iS5oJ?&>Z9>(-c>ukwzcH`C2J;RZsrawgfwsQ z_^$E>%c3T0qC{=7{!VR132X}JCPi?;`8KP6%$9Hn1OafWM+TR0qe7~;!f*i3V*8e& zg2b<{CPXHTV$I!e{|Q;={?@W2v58Mpxn z$12P)1O?Ya=5oO;+=3entZd{!vZe+T!xurALb75jIyEbT8N&k)>ZVc6X{pauPcqF^+}#!g2b7{f3SfkNCu#XhqC-mb7F zu_u#kCu>q0H^lSe&lIPz8|bFM_5fTEu0LpN2n-Grmxjwi2lqxRK|JrX-Hh-4Sr#)e z69=xcHEY{l%KrN9Y<{pD7>Yt8JWk+$ej+2&?jz5x+BO=;G{O)E%IvmnD1Wjy)7h80 zte0@FDNn8mi;mQ`^2_3c*E)o@ni=ulEKxx5^$tY6ita^Zf*p%62Lm%PZ%eX{-liNg z4I79NcqBCYhcs(*Br^c@Ap08M8~nQjqK2enWS^-~%M7x14J zF|trkGZz7cQ`FcNjc7tJ@M z^Ftsn{XscsrtF%6UL>A}k_b#x*_^Stp%THsqU;}ktXaMXUwkS}-luUI} z`^Qpib5k?Gu?2O0xWN`hMG6eIW>cg_bG2t1aA$iJ3X?VXB1}oAwM?@$(Ydm}!ictp zBQDRYthR_NuMSQNbUfEokNEK)<7yy#?jW1OW)F5=zyOU1#WOUv;5;^e?L@X!6SfJ3 z*!~Q+U~BbfUw06F=`ROCBiw>xkaKB=_gR}X;jT4Ww{>mbvpkUS)N*iA2rRcwg!$$x z*NSvscW`}%MhlI z0(WhVf9tnSIDm&PQIC;hf5ZF+HKjH8f(x~SN3{f)oR15&bziuY4^W1O_@l8lPMfxB z5AHg*G{U@dqWG&G$1;Z}D-n!zMzFC4pS4Jb05WB72nh6iMDW0WZZ40@@RKV!kPr1a zxCnwTIZ%@Yk^@C%H+Xbwk(G3Go&$B1Pr0Kv&zIUUJ>Bx1oU%=~&6ZQ}PKUsTQ)>u> zlNVdCY^3pyPlU>yYiws zk^F757A$aoYtX(t%*TAt-nz`!iO+V?nrZXT5<1LBG|+Oa%m=;9H+sYu{n?DE%lhse z%Y(`ifj`9a2|G;oYI|4J1Ia5ZIrei`X{*zBgelOu8`Sp+LpzQZ{A{Ag z!{@Vo;UB*3t4!@*#w|1gCG@`U|2`!&!i$^BqW(62=fXV0HOCyt0UbZMlHE5@a~nl)9+reS}_mOYzxZQHkT zzkHpWw@TC$<358XocM6##gQLZo}Bq|=gpx%mmWR&?%vn2XV<=+dw1_Ht$!E)+h6c! z=h3HEzn*=2_wV7ymp?zec>DM9=hwfV|Nhzf{dptLy!i+$(7*!`RPR3k8EnwO2O*5G zzXc_PO+dO5%uvG(6QqBz!Vf_VQN$5Re9OZUHM@@QN|f*tkK3BL5dMa z!c;Wn#UFu0P{$pKEYip$k=*VTQ{Evh$;Iw@$COtd^D)RPv2<_9C%Np>%P+y~%NSF> zaWc%n?)k)L|Gx2Qeo-xm<(wKi|1}&7(Lk&$7(M1() zl+i~WjTF*JB~5h9(fZ6((@i<;G^3?%dB@YV-nk{BxrQ-i7-Lv%)zw#FjaAlJX|2`P zTW?Kun4w1P)z@Evy>U!7gB5F7jO6}_^A=p0ZPwXmp?%fVXXGi?+H0}R7QcER`bOKW z)HJ0YWW!xG+H-%=O_$kJ$*lIw4#Tjqh@y8)w{GGi10k7Qh%Q633aCs*0-1E;t4_)-pNiW^>(@B@79=$Vf z-SyXD&vt)vd1$ZQ_S8V%# z`0KIHUiLIU;q92@8AFb z0T@654v>HaG~fXdm_P+CkbwB^ z;20Q3I0Ar<0FdJy@t8+F?vamu^y41^8Aw5wc+~j*^t66r?)X zWfFk7k(Id8V;${iN?Fd5mbJ9yEpeGkUG9>Xz4YZTff-C;4wIP0tfV^RLJ}#hsg=#7 z)hp|0OleM&n$@)CHL;mZZElmB-Sp-+!D-5pn97;ulod4T7*2Jrlb!8!=R4sUPkGLh zp7pe697&>oCN`RrpXdZYGgJ{!U^GLP02SyZvEd6}%+jE$AgKN|^`M7^HWC|BxPTt+ znNdS30~iYBBQ`n;3}x_MSlkAtA<0A9dWIIiIZam8aZ>RQxWV#5Fckb)z9K}P|Mw55CWX<7M* z*0CaXsV~?>9-r`2g3@)YKqcr^D=XBk3ep!YaDf^0hz%D6fV89yL>>cCTGG0JuyPcw zY6Eb8+QQD3wzakGZBa^+RK?U)!{7n~fD472<`I@}9RLUZJ$QUe8F-1xQ6BC zG^uiQ>vD@K-RV+wvwrlbFTA=?lM1pK+a17T&7s}puHlaAeXCrbJAgRgF%CIJ?jW&| zf{P+F0e%H69SwjDgz6Wg*i`O$`RHEbK32PbJiQ|@bV^VTn0K<~jW8ioOIi!$Q5mo; zEf(&WVGp;~93A#>06>ge6|b1ZEq3uRNwOr~8eD0+WMA z^feDW?rPc`gGjPYpOECsewnIYenFV|2#nE|)&+T7HEXdU;#|bB z6QvCVNM9|-S1TjdrJePQfgNmN51ZJ3lOzd5W?Tjw<5#fX#Ns9n1<%J zH6snk`XX44>Roaiw>^OF#uwUD9>BB-2@DUn`i`|G?V8PDhSkP#)v`4QHhc|#M~epl z;~EbD#fzPCm9L!TDL*zdW2)>3WU_i4==KjwQFOKSro z?*vgW1yxX+CU3@ClE$PUN_58@`LQjsa zZtLi*-T+PEx@-Wj?cQ9^kI+of=;)#%Dx$#87y1g8=qLiB&kO4atP~IW;7IuZKmfqc zkMzI`!!G;ih!Uj03)L_T#Sj3rqkPM%X-(WznCb5qwp$DX39sg|{ z>4*WOKmzC}2gXqf%JCfGXdAf^5BUxs=}*xR(GlAamewyE0pR#T001*V4kN%3%|RXo zKo2q@5JP~F2GEZRaIUE8zOL@w7LEZs?-&;;6ys>_Qp_BGEYRW#Z`BCP;;_KvfPp3B z=p{+3@Mf_meex%PQj%)X1#wXZ{s{)Byo?DQj@z_q0kg5-a4)T&!tKq4K<*3kbGN!1nA3rs^s4$QME&1s*aDVIT$c@&cSM z5Mco0*6{*=yip&sP2aHLE)6lP%utR1lFB{7cL5FzVG z02(1O`7I$0!2X;sA-fWe9^rWkpRm zQ^5XQOW*FYG!e=C#_{;paU(_ZK>Nr+<476$iX!_j4GAkD0f5oU&@t!lL;2_~w~&tb z&lp92PX{kCHm{4l=5ye1(-U_S6i@EKNDIV#VdIXo)ktup4sY+06DN1{MthVxku*t_ z6c(+Mx3JUMh^vm8p*D4@Dwp!el57n&OFW^@%dk|*%yY>6=o1O4tG?=xmK;cM#8DNS0@W|hKApZb>K^qi6MH2@4^iT;4 zQ0qug^C(X@bRIjiLs95sHP!Ca?Ca`)K}OvU zPW6aP38^>D0VhxK<2G*9Kx!0+G)IXu0NCnCm$X=o^;nCoNn`3sb1q8L=~j1dDr?k# zDJ>D(h>`bD^*kHYJqd{!S#+)J^FGf~Rt=CEk1^qn@lGpLQQ-&@6;Y1-6pko?4g-+y zLRGL3^griOG|`a|IW&&u6_x~`UK4a*^^}g*@KXI%L(SnKA#)o~RgN$;APx2;S5z2V z6uRgv&~7tU#TAceb&zaz?&^Rh|8!S>i_-!Liv(|yv`Q9OOKcR8b!KU{W@k%T`H5LK z&sk#$jyCq8LUvsKgE#@KNc;oHEC`1?rzn?)~rZP5w<)|CR?$zDi>yL_H#iu zbma+W%}HlxkZ13RBpnWNskIsP=-r?SJxOV4F&4cPmrZjjcKHq(pR1^fEcdb&ZMU|+ zy7o`Q@l)@J67KR}(=npzXmA04DNoh*j_@{lmsdk6bsg>Yj_}bl8Fp{I@1y#vcoE4p zT^A&gOWvZE{&8m%X&K2@kF&HAk2&d%1Z6gUc@4JamjqiAMaDi@P*X+O?%shFjcj0K;woY9?ie3GG?Oj##3i}ok z1u`A~%0t&yc;_+>sh4@@Qic09d(#m7_RxFv_G|qO0NOE*@DWiH$$Xhs?e2?89js%a zmPWM|k!&?s1Ar&tC|5VP-YkydQjv;-RcZtHi@`XIcWHo~spfJKN_i}r_%4Bi_PI6~ zW34rhce(X&ncu@N*0*KHY47J?k5dgY2 z6A&~Z$^NkH_K-vkNqS{ij__8J@raTOR*~~4A%lQ3KNC~G_iN2z4FaQ%>%N+SkmKxV9T#NnSdgMv#9GVZEI_~(?>DQsIXMu21Zhpf7Fmp?d77#D zlgPM@OhFJDp%DZDxL{zLw|Sq?#h=#9r}*qj*C3tKx%A>oozP^nvc?~2@=%%2Y zpMZ{oHk0r;POVF)SX! zx)WaVtC86iCzqLjm0k+(4gwzYs8mR{Hx zmy56&ce)sVETPL)!YHhjb*;3nTUbR=yIGB{!8^Rg8X zuk$(u_&QpUa;KP3utQRgDEqCZOrVV`X@!oe=+oYEijPI<8PRQz22S*(&{`F{&f;jQ z_B+7?oRBDCwJ-8Q39@d%sxv1_{OHxS<}d89L4`wqApWv2#6_Ixuwg{&h-|kpZZ{NB zF$ z>y70IR_FVz!tCDsJK|1EwOVVsr zeI7t?USV{8=L7!djsEDxBk1vE=!u>+kpAhR{yLO?UYNe=DI@B!KI>;=>gA>CtzI&; zKJ3NbGq^rpy#DL+V(is^?H8l$;pOaq(f%&je(vcWFx>uK;6CoFqVD;=@6+P$za{VW zek%OF@C_d<0AE`KfAFK?@E!m0k0SB2W$_u`At3+qF@Gl{A6qED@);uYK|l0mV)L)1 z^F3c7L_hUaKP5=tT1@}+FMjo9fA&A3^^4{8Vc#HVfA@L+B5Xfca6k76qW6h^fA@bM zSb|^pkH7etf9;pwSCU`(n}7O4|M`0*`h}nRwg2+1e^jtP_qRX%55NBVJ>~mhU;NR( z@5#SY%>VS$|NZBF{W+!mJs!Lp z2r;6>gQiSDyNEHP#*G|1di)4~GGxYyBukn+i87_il`LDjd+asn4JUsNJ-YPi)T>uTZoTUA=G-@C{|-L9`0?bWV=pgsd-wI5(Yt>SKfe6= zbkEbzsh&Oe`~3U+{|{h)fGXK1AWZq`$6SF5F34bm4rWK-gD4${Aa4?0h+&2rZU|X~ z97054g{yr?Vu>c6h~jnar8rTDB4);7j55whV~u33XyZaIzF1g}KK=+~kQaKGEvcKOtpYOcvL~|s;H*E3TvzvzYKHCGS5tN%{JeRbIv;N z%yZ8^3;fD3&wN8}(MBJQbka&M&2-aFKMi%%Qcq2F)mC4P{+HG20bGA4%5#n}DgRvW zE5@ADtmv*!3k}Ms&rbWtL1U{}J;v0&uQBz!4m_{*sLOlt#$P;)W9L+Ki!soAb58Wq zPfvaI)?bf(_S$cMeWk{@drt7;zdGqFLvxRQ`s%OGe*5mf4}bjf&rd)0CL`L|@}{gV z{X72u55NElaDW6XpaFjmh(H2L?;E=_Mfk=C!AwEJ6t95ZITDD$3~F$L9PFS6KM4N9 z5Q=bwBrIU`zVQmi)T1KLxS;>!0mB%|aE3Ijp$%_{!yM`mhyOdrvwj!BATnxFEqvh) zkBG!1DshQSY@!pN2*oH$af&wdpAQ+cyVboz9`&flJ-!IWFp7V1jASgM8PABuG@h}G z@_0w-_9w(R3JOsd0#@!*(lB84bYw4t>JdGJnY{GM&FiNxk zHm05YENF`SSv7zH2BCi;s6ih}B8E;)p$|ESAaW1^1~7jh0yz-GDE$GAJmevHNs;JY z9GaAR2x6oVZD~a>n$eAR^rK0kgdhar%?SyVo;h_$BQy%sphiFx1Q~`#%ODk#>IJ1W z(H|`t@(u|wKm+)6sZ|T|)1V5Ks8q?pMv2A}r9zFO4536&FOUPT1~r)eFos6mkkp;d z1+ETZDouZk$^fK7^#Tk?K~=AMRkH#Wt!f1-TayxmueL<2Pa~^Bpb=9T{ybwK^~gj_ zB>+UXGJvi~VeCK@xza|Ev>?VPt!Yb3Q<{h(1dXVxIB}v`&MM>`7eMP@0ozi@YE-h6 zwX8-lJC(#n0IY$LYt|f_5L5hil|=>WEd&zVxXypmA*&66b*;WEHAuQ% zp*mj*go_60qJsUw#Rm(-v!=K%42W>u6dzXvgL5D-F%@di_~BjsyQ^I(u?Bvz=uCszzID)(CJp1V^n0U;m70RtI&{1u3;wFHmVBL(t4&Tb?d?5YhYrp+b?3hzT=m^<&d&}3?8zb*2_+6hh%7t4DgA!2JamjyOXY`FC zJ<0@eL$HT&i^AuX#cpo8-%THH0Bbb|aHTg4L}8580+poxJ7+-VjRAWGw6^2T))Cng z`fY{0{MKA>AZdY`15|ce>`?e^whg;wzKY$|K&|{_GP?D#Pj?~vUU3c@?a1&ZIuA)m z0@85;4e`SLsR&uP1~i&(D!YG~sC@spK;}$xvL|o;karOffZ1nr0&#x~kbjf0fBqm? zL_iRHH(8%2KFfDj&G%8zpl4_24|NxFn&o{1adLFEbalph8bx9R(E~^Ie?a9<{*YB% zFc7cc25!)R3L$1oLv$xreeA{s%%xdkHxRbAE=K2p0#SId_JdqCb$Wl+9w(QB1z}$b zR$VmMR&dv73Fi`dU<6+XhF~an1#xM07jOencSeAK{v%fqm&aM@)o~0EgO2xsewJ@` z=MR~%TfRgIP{0LTXn~&t4L^8pK{ybwH-G{Gd#weBvS)(9Mi3__0p4_Zur_B{I1t|; zRU8))JE$~0h!FBH40(SraY6NQB3M;MmO`j#5F+?Hb%%v!B>{R^Zvs($M3`^K*ASCt zT~E~r*F^+*00qEU6_;jY9`y~$n2b{=0T-lwD+Y&+=yUi)hsSsh%D9Zbbq*)DQ6uGf zfjB;*NQ$QRal)ly*mw@KsCET$4!1CIkH`QLKwm~ShK6$ zrdj;x54NU{2O)o4^!IQHC=s5RG@v*TZj^BKPSh^*9ulR#+$b4V{LR2Kk09Nq`!qk93%qGAR&^uz*^{ z1wfaRm}8YcmX*)IcOo}cj+9mG29I*5R!eCR)3#!C7?po+88uXifW>ECZnlbm*_8`f zk74N#C)WrZD2FomRX;_RIw+b2Az~hele7LMUUn%Imu72uAPSjaoX05&#o0!Db&-Cl zU4W^O6Udy7G@QkGoW_})nJ`VzFqDbO0EyU`oRg9uW|o5K4_*}kjbNR~nVjoFZx56ftQ6qQ`Qe^~s7=jul3jL*) zxY?fyfoN=Z5YeMx{}uugNE2QdhG7Vt{$Pv{3YMz`n$O9Q1iG5~xot+4YgLJ&{zF=_ z)|}y~InyVg<|$mbL!;;Eg5^e^E81~i=WpUg1de~CWiHB?;@6~&^`8Y%4-06dRf(D` zIuHtJbW8bzXP8(Q5Ng}Tmr^yNADEl|0#ODYmRcM-h2QX-H4%SW=#=L5S_gqw%_*Bd zm!giSqOuj8R+@l!T2*;k59qmpScjvYGo?Wapyp|0ReExHprZvLSeZ$wzolSisHFJ` zs5gJw5K@UWr0EaAH*N(X4=dMqaL#kDEs@e!r zKy`pyR&4dgeG%Y+!w^%Js-M0$feV;_OWJ>_u=EOCzy)O*bQOwx5>XE|7XfatNSWXU zz&3nb8JG+ZOrci>w;&3I^jS3znTKC@D9~xT`3Ty zDq(Qkg>Vb8aNNjt!;lH(8gtCbk{Xo+T<~Aeihvx)vXtPmFuNJt2m_RWD5{5v+q!=_ z{R&k78n6QkRKw>Fs%HeEFbq7|Zoi{e5s(KAONgE(3IeH`2g?8)S*s+k(F#jO zrmX}4sz)QL{@|vR`JDrSS7Z36?`3KeX%JUST;^7#82O|Fr-cN!WrGHHwK@~LcwI?2 zcgA`UwsvlXCbKO$nMa1RDGIr!hHrmct5xo1w4M{Ul$nSvcy713kd!%G2K%%kcXsIc znbX;EYm0_w>NIaVZ{7)1LH^*BnnN~R!HG4-$YfOE~Lx_OAZ)LVM-___>`1X=25jM=tEW4oC+DWX7N zDj^RoG&D2u!1V_WxblrnC0@PC5DP~TrQ^-4JhnG3`BbJ+d1Db!4<3!3{+Yl z{5+KL494)n2jMNNa}yg(E)jox7D6l)t2Msat4}b@#J^$0;zGn`@x)0%to5h8O^n5| zF~!yr#bdF>UqQlIEXMw#@x|2A#bQy$T@l7(%*LIO#>sNVVe!UWvBqs|$B;3{#}db2 zk;hq4$9D|Ki_yovvd79nyLR!%fvm_2b;x5DzjQIkSCPnzOv!l>$-IA3$X;>D#dyV) z?8%JGy#FZ3jZ7Dl{K=;b7n>|AncNksJQb#l%CU^br2L7GJQs-a4o`X)IYvRU?8|uJ z%CN%9T(LXfpv#D?Ds%eF%lsC>{3^q|6`gdP&ym?fezEJk30a!h5kol(HDAVhR%G&ih;ylL8I#{E5}e7fz+m zccD8K0f_vp&_^-;K_JUD@4%Udp+C3KLIn*M=n`ZIz0ezd73K2{7VR|kfI|0~7zcek z_ToDntxT-r_A<~`r&UEk_` z-|U^=?!Dje-QV*4DDmAU0!|$S4i^SaI0%1U8w#!_3?3#94ki$891;!|6y7%$?iv_= zB^pj89Nr`z{u>~k6(VjoBn}!Tt|TZPBq|OhEFK#!E)_8THZmR=G@c_iP9r!zBRW1B zJRTH2jy6EfO8;FaLXILvej-TT8A`qrOfEJ~?%hQ0CQ!~HRK6ir{uo$Z6I#wST&{oL z{+Z(C5)$SQGUka<<}Y#PR+HujF69H><_-SlW3uLdG3P5$=TdX$W0B_qvgiNN=l=2M zUyc}o&NPJH6^70qh&~^RE+34}=4Kw}g5&5|G3oJ9>D(RZ51#4Cap~l@>3`GdR1xav zG3w({>f&+gUXkjg_33`I>O}GC;1Pf8)ZOYEUhBg#>uq)Gdz0%x(d*Um>(UYI(J}0z zzU#z%>}0bK@Bk0{5D~@R||HTKXB?O+ejj_=eQ&+q;jUj*G^Lhyh7ZvOBA z|L?J+@R05|{q7I#Knt|c4g#_63?c9gfiCxm@M5y@Al-@!-yk2~@n1sn5RvgIaq{dg z@tS_kKFZ+@%5z@_DT;XWzWN6zw`y7_Gj<( z#_{%S|MYs3^X;GtgFppTFbEVc5L>TwZ9f+G;6tU=LTC>rgulbaaQJb5C5mtO3C-$X z0{M(D`Q;|~c~SX|?-1Ia`Al>3_RtJ^zxQFV3C&>f82|ZT5wfS{N{)YzC9VE%TCd;v ztKT28Us|-U^R~|)xbHmiJN6Ob`?|0Cb$>VOz7O7@3URRadmjgWzx%L(E`{AfPa6Ec zME&Pr{k%^f(|=mkU;9|%{iX%}(N7%XANkvF{6irMeh39U%n#XUrHwh!G_oe3pa@6iO2{ZsaHm%EfaN-*F*X z<{n3l_s&5x_ROI}XzxyPbcoXCO`JJ(F7ya;=g*B5FJ=TcFO9{Vxv{2Es8N3k z#VFP4RjmG0vUK^9Daxl=p;9IKv!{`VV%4sNx;3d+nPAgOCCh&n<;bD_b~n213NtRv zxq1cb44V@#7n4{8F<>kTXb#2^zD*p)n1P$YnKf_bjC0>XY@5PRAzeC-Dr^VgX)Fx2 zGGok}Av2IH@&e-&O?TE~snp@KZ}Q}JCVuej+}X8@FMNF&=kc|?eg6(VTyN)Ep z>f7i)wOX6+HU@H2%rg0YN^UZVjA_q44mMOtToC@qiwcul*LVsek&qe2M@)1a^Y@F`599e(c@v3(q{**APBaI@`J&uIr zaV;g6e5T12pXBh$qN=nh${DvLuFLbl%ur0A$Q%sK2xn@pjN;hDalfqSobpU5X_Sh^ zy8@jsKA;3CqYO6$ZD>O^BbAgh72#<{G!sh;LzQM$^G8L92>nS;%@k$SQ9m8!=$&tt zOsYB^C(VD#On*#mbWBKXq7^GvU3CdoC)Y|()TAumv>D2H{*)0(cif3fg)ix))4qp?U|V zM_lGFF?6?cZfsN4 zhUC2|FzmK!Cyq4xT2qx=pRr?3lW5^=5n)Tp;-*q-QR zog#<&OUjFS+G@@f*UE4(6W^Kuen5f0A;ukYoKKtTnawh!Dbd=p2yQ*-mJoQStr3OH zG=+CTP=bOn2{4q03*JK*8_UPD*Pfw$>=BVfhD$_$5urDMn!ft}>lL~s1>+e=0_IyF zgl)Fyoriu|A8E^@bCf4~8k%Q7Lx2MeU`wthEue@(1H6ksS#}p2X#DP62gySOsy7VU zmBDxyC<}b*H#`c7qJ0d2K|P+wfDig(ea0ig*$zUdMvRR%WkJdLEMvV~t4k{#B4B!GZV89q6j1Y)5 zq}~nFcM#thFL)<$!w9`2#2aF8d-ie(0FQ`7t{kv@30xrX$^t_fHgF*}eBv2V0*&1< zzz8QKn?I^}MayJSitVe=Im&1m9?p?!OEKbq5tFFLiQ!En{)A)z4qrqk_ZWdOW87jF zzsLYQJ#vE?nWXVv@r+#gqlpZ3gSX(5f>#vJGC8;%M2dJvI~p)nKiOsVqL|2=$r6jS zgeKA~xkIMeWNLkrAMYHa!4Wbb2q#%yIAc?#SMUx`$WoR+>cPHn3Qd#qBv_>`HJT}Z zgl`H{*i@$u0xE@UQzGw-XOS`yPYH~tob9`fayGI><$yDso|_L#LiUYwWK^STy9cyD z$q<1`M54xv4JCMo!QBPnh45^jV-9kTZt{)`2Z4`6kCZ>lq@W%U#Sl(`H$gN((s&b` z8%y1Ikb{OZp+2qCJB;d0oDl7z8r^7rJv#c0k1iyn5dLXnI7LAx<9$__BW#{OwrIR+ z&Q2o)jc05|YP=xSQl0*As#oVJ)p}r*s^{RSN0+FRw?ZnY;hZ2_&6zi$Qj)G?ZJ!{( z3e{&g(J~TnAwBzM(Y)@JuU7Rd0!#N)lF}7I-Wk?edFO_AFxGgEjcH>@rOFY1CX}YM z3FjE^f?2&%wXazHD^9vP!mqZL0b-3OYk}*J$i}v^IN9pZV8GZAq<|&7gF++DTASVF zKoE?Kz}b9s3nIbtCOq|r3@`9SNMg^j>)nz)6BdU!3}O)Qg|B?;%PCI*^dQY$hJ~hQ3g(wk%KH?@1W^Lz#f%i)2h2*gj5GoYDgT?=z zQhov^uqkew;|T3o#j9-bayL@p3u_p|c7#TUd;2cNa6u3~u#5)I@!aZvN!Lfg6U100 zl&qKr0mr;c!HzlNU)sJXyw$z#H@FZ2NJc;dj+rtEtE@>Z15nFb=CFrDB}$2e_=zJv zFz|{@K_ur{o*DsiDHkZe83WG^ZO-XC&@5tThzxj%obY-Ry=X=Y(953937*qiY0?@B z&Lmy)sNHO8=t@^k<~8bn7;)Oo2t0$r7FP8BqPaZlJsL)SVkR?V%zWxeL!;NE#tpF3 zoND<+fYtqb^iNbhJ0E%HLUfA2c)nXJ%aq}F>YWHr!D?q?%$vRYmeoGgfsJMaT;KyA zI5s>L?WoER-})xfl&a)I6e`<`LM9+?Ry zm4qjGtYr-BkLG9qLlVV|k7+Z(u^E}h$s-)C$O~jBoa7rU-7#1b^V^PNP2u}xcur?x zCNsr+=DFm^$3+hEt7qNhHzAS)X7Wneo_k8Ws}P-7$(Y-OfD)9LOux8knT2mC-3k#B zM8_lbgV21QRwsFX)=R!jX`>veE53)({@LjCdo!Kt5@}9>(J12}5zg!<5&P3EV+7%t z+A>7tw^m4w_0Nlr)(?Wa;*~70O1~NHrCj?r$o@hIjj{6k-Ox9Fhdh`=GStv}$TL7) zqtK5W`OMF_2Pyvl0MpCnE1&d1-f`~{HH74m*ZzGC|>{03~vc1hf$E3IL$dQ%p5Tb70+ zw}BeIkqf=l$q6DcvUhlgdgvn;^M`aYo|&30#-N14!x*RY5j&H<3plTP*f|btB%f23 zbI=+I@RkIBq!S5Xo5|yd_G=3Kll}+&y9NI9zY2jm%kaPnga-R-8U_@;tb0J_La;^v z4vm66crgb3lOuT$1qnNevl^#`a65>&!TYN{?c+YJFoy3t2=aor4*WnObe-%8KEL}o zz*|DbkU$Ewz?guL8HB=>^TNkNJ&l+O^%h1C8n!U!z!Px$53Sqnnc~}WMf;A(GmU7bxR4gbU`K<^MokANS$$%(j zlL!<hy zMuvd0$RGlj62*nMg*@m1k5oO{*b!D7$!pAi6=NVqW5kd%lt?wIxPh_AbcD5xbhM3R zqJoN#gA5df45-D6h_OKuS|Lf3G#&{VmX>Ruq(d}x+%TPl294Azj#QtPJUyW)tzCo& zr94TN8yPb3kxW>~FoQ^*LmH02Nvd=em2FQW_`pJ7tK7G8Owd_Z> z$@7EZXEO+eA3b=``L+0=}G`s$Bl?ig%C~W7|abSOw_~>8M4RhWR_l;P2KX&tO}^CJc!(kr&nlBysQ)VgiJpy zK(@rdn>o)BYDzljEu@!{>h=)&fKah4*k#l3ebb_qHrQlkwmT(wLhoy%Q6GZoxm1GVgMo# zg!oa<{3`>G#0;jI$MplqV~Iw}>{7wl2QUp&F%{F|1B?)JW$vai8|^e0^*8ds>XMJ%z->`13}FLK@fyINDKJ1i~u|`V);LqlNK+VC(>AqEMs?ItE7UyT(O}F|WJ;nw z_0jjF1Sm)>`52Z81RgGEgoQ%Mf@BCAr9D3+lO}!AD5cLcY*j&hRYR43)oghOTD>D% zokW}XQ&%NaL;f{}QB9RCElTKQ&m;NO$OzV9UD5Kq3|EEJ0PH+#<)m8WQ*AZVWCgG~ z70E3s)`LJ-+E7;`B}=)pj2A^#EIK1|aLz zI)zUZo4x)mvaRBzKK8ISn%xkREn8<5&a2JZy?dI|IfuRNOsDljsNKH8Q`-`4TT3KY zWBt^H5JbZrkJ+n1)NGPr*gi__yu2k4&AnF}EVIYu$|-?{P)yvWSlxe}(0wIE%OHY< zc*mAK(PKS`+I@+CFWKC=)Cs>O)W99w<~s@DRlbe2ExTh5*;UfASupbZhB6Eq$-pJt zU>k*ex8F)Oo3J$V%NN$n$K3Qyh+9nRWDGoi@eu{?A;{xLE-*jxVks`r zUX~jEQPCw;86n}5sNWay-xmtt6kOqeoGcF>rktp-x0sHroZy?N+3Q3Ub|XKVn-WPo zJGJUz^IMSlz2SK~K!jC}v!aA|IEM}vP$8ZtB8K2~!QG93&j_$!3=Y>o*l_Q?sz_Vfg)z=`Tt7_YtE&3BYE?tqWvx_`XSQWQz~#Ah zQxcAUycJVt%fj3?!5t&19&yICwI$|e4lR^Oq&br@u##wdo{W|YF5bz3-;n@nr6-f# z6K~GwdOod#V62!z$v>&-?Yikoqvc`4qGCIVeQe(MENGNQiG=P=L11Wyjwy&9rly`O zoQ^4-9;TlDX};BodVGa<3@7;*P^Qcm6P;bm22qG^h@Z0SYfn=ox(JcxQU1)4?Mipq2>t6&Z&IC=h-ptV}{Wpy_w%u&$hNkyPC|}j_%tg<($y&gFtR= z4(>oKDCKBtQlVP1=D>yhWKgzk$-c9AQor8*Zu$&wn`}wETk81qa4exNqu(MZiB2yBlg?Ro3K@f#C zO)#pB^u~|{qZ0- za0OB$8af9AuZJ=kt>nNU*dlJE+H!K!a0W8*bGC2BSnyar2#+ojH{T#Q*CZY%5IWy) zlxU{NFmlvBP}?$e;6d~c-v~sio-}$>!(6?~`16-S@E@hx9FU{Qtn@kWvN{I}P>&2z zSBOAr1aKOoK-UdES9LdF^#hlGF(QdqO{a|rk6RS-jiJXZWi7@%lNeOhP z9gex#ng`i>=NpNDcbgAJa1Hv2AbE~B_@?i8f&X|kwuzqaGKrA-7kc{h_WH#s`ZzK7 z;y|PvNPD$kdl)eLd4KzV%n)*L3H4TqVYFX+yl;ErkcS>$3V=@;{hjeIVEU?dg4g0^NOrjvC>g6b+t8 zIW_*90R9U_edCAz+3g)iTk1ECl4Y%W2Dc1c@O_WZd&r^Yx5sv^iO|e zUVr9-ebc83TcYpk!#``ylY;*%Ia07s7WXvDIf(8#FOsH@nLxu04L3`H_p|ODz zFJjE7aU;i$9zTMA3~BIT#Elh2qD-lBCCio~NlMgs(k0EBHvVtoYzWh&#hgBWcHAkm zC(xn{XBthabScxOPE{g(Dz#(1bLZ;uoT_yz*Mn8BX6-6=tkSPohmuXJ7N^;;Yu_?# zixzInsC4h*&8zn)(Y;Ua-Q#H2FJZrc1@|qinC@Y>iyyy#Z9Eq;S;-j(Q*MYkGw05p zJ8u50HMHo`YDt?;y%n|U)=pWw?h4!U?AW$%kFAZ{rtaOoUH1MhTx9U!#z7}OF17e$ z=E|Q#M;zU!^Xb+NR=++yyLRZ@yNCR~S3LOg!pWO2h(5jV_3UlCfA2cJ{Dt)2-mkA8 za(?ao_kZ?(AAp$&I3Rz?83+}EsTG)@gK0JB-GUE37a{(IdRcfOUKz5-;Zz=K7-CW& zo;RV0zma%iZYHV-AB!z&_9AUE!dT;4DYi$Wji%wa<5NBQ*c6b~4LPEYK_dC%kq;J` zq>)bE)+ChFE$O3_KwUYdmRT}brIv+p`6X~(ikVk`n8TfUQ<`PkxaN#(s^%t~AjvtW zM|DO?reb*7c_E(%_9@z*fnGEyp)nq6C`yVRsvo0-J=$nOkxHtbo|Ue4X^WKJsVSzI za%!fhpDs5lsZ50`X{wy9I@zhOV$>>JvL&Tcx88g6-M8O=10J~G zgA=|pV)7uKxZ;a5-niqBLms*0lT%)~<(Ffgx#pX5-nr+WXI>HLqmy2`>4=Mdy6UU5 z{@%Llf2^ney6m&lUi<5@+n&4byYn77?!5yayyL$MzxmoadY`^~%mzWnRQKfnFn*MGnM)#txI z_5D*1ef}FD0SnkT02*+12uvUYbGN_-LNI*Ye~Tamh4;V;I&gyQ6CegV=)vV(@PnQU zp$JQOITD&Mh2_(r3R?&|6uOXfFf1MiV`#%0N{@v&G~En$n8F_V@Pj}MVh4vv#L5|w zcQrgB6N8vUC+e_?QhcHm4M@c*67Y&xRGb!Tx5O=iQG{Kz;TXeczcQMUeP~Rh`u^59 zf5j-0?~81lV*=gyLpaj0g?7y2_V&of?D>(1fjpfY|7b}04bp>&JS6lM$w=in@{!1k zBpLAtzCyO}8)eXd2B7c`9Eu_b7tp{hSl5UoG~#`fjNtQbA&5K(LKL55oG}`ahdfZ? z8IXI08*~Y|cf9f(nB%1{9ruhfl+tqNe-Onh1(8g~xe}JLq+B%fAPQkJ?i-o7Wg~_` zIc;*&n~ZyhDVq6;Sw@bW=0qnKZHdcb4iApej3ql0SIb-Ga+keKXE2Ak&&M6}nUq^* zGX?}&VF8wn2-zT

    H(gCY*0>tv`jQwmXuPRA_YF6c6mlOO>O?8= zPg=?orIB;0OTQ6Ol8Up5tMnWudzS}P7VjMzU;tS)kWv#;f~;f}1?zIafHIUW6Gr%Bf!>!a)Ga8f2FJA+zMC731Y37Gc4rtzyPyWE)#-fKnPf%h%*`ce(5(uSv7p*Zaz2 zx`M6lFAc0#32e75YU z95g}@{QaO?BLK^+o-VwHe;r+7Q!wMxp_qZVdqxEMnu8*bH*sZ9D-2Sg);u)Mu_Aul zFw}}*$w_&~j57sUVen(bi9)R*a4T6Mj5s128OaRD@Z#PP0=4E~tVss5n90llFF!8K zWF0e;XNhL>9=XgWX56165NHNSz{E@LvXjXQ{uPl#qJXgzHe`MjY!Ob{$JDbjXwsUa>dDrc!cy~H3UX>$&V;hFo2(KGB>3Uph z8UEXlxRWgxKKTv>;~C8nNamoqaqxv68i<-ypLa z5Jz!~WBjao4nc*bjAxiLprpb^cAMoa^yiXy(G`C;5gh$Ef2IC^@*GXibfnG8@oYm{ z9yix$&R1RJjU%q(5&k&Jdrk7mkzK7RUwO;F40M^(9C3@zgT-YlZ^do>;$8lvG5VU! zVLaCqkK3@T-(i)*hkNN#ce>-gvDZu({_xg)MI)eK?qVk{?_JmU)e$}RldIkA(3V`< z^PN++tJUR_e|tOPW=__cC)?3=H~r!?zju}T9dUpstKc0+cw7NE-&NmPs z`NR&ssUG4?7_p@l19%p?0hf!x*VDDz{T?b%Zkq85k2T)#=-$UPQ1ona6DAmY?l$=%>yEg==k0~;#N8zRmT zf?>FQ8sz992g<`YDB+iRnisa-UPa+v)!ISrRS*=Q98MvH$wPf*g5umCAl_jbI!+m? zAtX9Z!7ZRXTp#f@Ujlk!;?SWT0wMOv!ymezf7}(J{^Ypg;s~N(E#b3u;S(aF;VB{v zHsT{Pz#K{|~Rp3bB7TxI{q8WqbiCOpHn|GO` zev|&z3Szqa+N<{#K)z<~6+3V$$8%$&nXpk{HcbF@O=gLnF*% zQZb-A9wd3;+bh)9fh~?Hgq(RLWE~n96og~1`Qw9C7$x9Z2t=2!F~BXnf@&?}<1pMw zz9WH!mo?$qqb=m<(Ij8lWKKDmP!i>3f3YNVW&WKTPGkre6i@!wPgdnnN+Le8S3^>c zF-#bHxq-!v71&iHD^8LI8Q(Teq9nRwQKH#l8AA%>z+W08m-*FLm7Hz)C13*LIsD~c z7TGN_V^A(mczIb}E>2<s<To@VT&=3cF49U>ZkmBC`Zr8&kSdAX=3)k> zdHtqbmXB~I=3)k#V>XVbB`2vdf2ZO&XG;FD=P?pnb%rB$O__Gu9CuF6bZJ1CEu(7g zWO+&sG^kePMWag5Wpq|1ZZcqoTHcPiScYLJ1AONW{f;2u+67c1;wWb^j;B1pByP=Q zjk4EFQbJ9JBa0rS06~L;^;K5o13g)JY1M~<>@Gn-kp|}Fd1J?x|*6oCR8m>k;Y=|5vratYD6HZT2<-fcp8a) zQfW!REY(*JJe&eeBXBCte{>OBcr79R%qd)(VQ1qQUt8`~VQylq_9=fdPL<85Mmd>} zodc>$7l>k}ym4VFO=qMs>0K@;v09!4oP(z!SX!Z}@e!M}E)KPN6RWb5t3q0^+Gm1E zBDV_YSiP!%;wp5asDj$+b){?LkgMeI>YxFokG%sW<0y<%<;%I9i4qMMX79~JI4kUpfm}9bjtFR)@zY1zFcB{FL)4Lkg zUeRi}uAa$8rpR(C{t4)>zQeudYuFiMzb+2Q1}r2CDlEfxREhT8GFmIe zYJkOJtj0XGv;>0iYw;y9MNwW(lL;Mwxk zn~Im5Ht1vtkkUnxv=nIT|!r6_R0P9AsT9r~jnwJ}F zt>iGA+}5A$hV1J-!#R|ekKMv8xRv0V0($u_@Cxtj_LcPh>El9!>q2VcC@&>Iu9Sr; zd72mVF68rOe^&%Z?>SKJH~#o7S;>=I-#do=Uleg;-HyX0j z>@6Cf`?+HX@S@6U7RxHm*sk8<=$Rqb(**kz$}SGR!Ifh@@LeA0?Yi(yPN$yr-WOsY zz7d-ZFOIZUa6O5zJ?YR17a0Y+l?4ZJ1~<+Icd*y0f9os4sLN7LJ(ci7@e%__;#@$eYt@Wkrik`Azes^<_(a2Nk@%VIDHGcmq7u`X_C6zl4B zk?`#*4m4bG7XMNlBjXnf@G<5h`mPp^;adaarVdxwT29tXT4wjO-&PVaGQuz>Z>TYt zf)%>$e~HnhYFz;6HZT74iEy%VVW#OWCGalpy7KOlRqvA3@7CY>*5WD8F9Qe=XZ4$c z!WB?n;Tnq=G!)q)>Zy-8F#pZw0;6c<1f)RqV0R+(i=mV8F|(BJqq<%51Itz_b*iR0 z@MwiIBTgAG`=W>iB!SiICoP&u>KWZRnRUJce;rOvc)=b}+8g5ZpuzZ*~| zA_gQSG#XA>&_w-dD$Oe61ThOUPQ>DIO@S~LO7SoaX*xcaNh*$uA(%}cu?UtW3(CV$ ze}CwCaV!r1RRj9|(o-ub=&CRsrq5-}SsHRZw$viqQ@|U!&~Rvj9>dZtJ!H83QHY*m1!X7s{K9!Br0$ zlx03cHx?^TL-D;OZkyHfIZ$^~U-#{Op}OgrP@kxmdD?S&nn_YM!!37na~QsXe{>Xk z87B&;Er|jb{^ltf=P`Vf)k$lVTO}HWS`K_Ox8gW=6pwCuF0>T$cMq?Xf$w#FhpYv2 zXle^MN9I9A1bCYfI5beW{Z@A~N{$ukkSLJUEX_96wT~r<0+Ko6MGtRdr?tGH*`q0T zAch}AK*I;#xP|pJkNfzKBj7^ze>daAI9bhj>ppI z4?9_nmvt#R&XP00i~}W@A2uN7_+CfXH}p6}0J-AWbgr_Kn-_VUD-M!p(*+n<8#fM_ zH#sea>7zD|i)q?sQDl}&xChrFgS{?>2euIBnT%W9m(%o+&n}J^nU0Hff9t)uk#jd$ zi?E&hIR2g|j-EqjpHmK?6X>A3bRIHZAYLEr>Dk{A>l3P$qf_e7Ihis?tzeHirBkJq zuXcZ$7kkyV39Tnlc6d%f#3%LkdExg#>G!nflzvCJf7haoO;Ah>1uZoK7Z@sB5p|Ha zB%hxPgcL}^>&Le z(zE^U{dWJQn={(HT0Q;ZWP+rpyS!1I8_-@R48O6ghP|HN9oc91LAe#$ z_cPoRJ!6OYsjoddf62XS(|upr{o?l*-~SRsSfFb8*ojskuihjr7JiMP*L2OOK5u2y z<2^dTzKO~H1NtM|t0)VhWM3tL{POe?PA9?_-(Dqr$+LXAbJ*`+DJ3X(0vjpTw|h&5 zE!Ow^uR{*Lv3r9$HJPpz_2;|&MiI=HW*=fb!1JZ_&sMuXf4sep6%@C)mQoJGwKwDb zr2JJ6^pqI``w!ve1B5(zf*3@ACXir?cLIkoaQF}+11{$-FmNMKAVmgcFbWL9Kw(FL z#*ATX$Uv0Hjwc(bBxq3MMT`U^Mog(vUrke_@_muU=(xSmxG2nh-TW%~;WD&yOKz)+~$B%|jR^lP+yKw5rH&Xa^QN z*fEb^oG<@k3~cb^O^iQL;!G@(amux3|qc}r9_UTZuf6_gTVUE{1Co9t}4(1+qqUYB$4j^BPW2tHh8Xd}qE-ooAJ70QTO&KQS! z7^c69V7P_7ks6doI{6A@;J^eKSnxrN-jT{6{5<>+#1KV%P9#Ajs7VlyU_;2ScQn{Y zC>LSuf9a%}9^wL_6Cv`4#~ywBvB%8h(TIT@8W{$fyq145P`i%#twbU?kE=B$ec-JT9N=Q=V_W zIfpSpUNO^@tui!k#>kq3Y0Kfd)Y73mlg#cse-_zHh^ImiHB-^@`pgs~A^y`0^qY4a zEzz&?$~4o;SK#dFy4D0GucC)q8j)9{gcJz3jfAsqMM`nRZblYuywO3DoK*=>J3o`h z*=hyakTXhp3bu@4huTx5lftai&ha`TH%K>B6=kZAEG%gcQCB*CCm3;?~=apvm=uTa5Y!nv4a?fdL7= zfwwPraXKeSF&*-lRW@<@wM~3Qu590m`o%R_)t0R{;enxz^x$HHwpJd7BXGDQh$W`j zBB3zGX=8^v{y1cnNPd&#pL3(CWsz{Zf2GD{%}rQro1y->EG}S3f-?TB7?7{my6wK3 zD4Iv2go`I%#`!bC02TC{!3XseQAIm4hVLhVeEdft8S^r)lc3P-D|tk5Xc&>6X(XjA zU7K_`je_dvofJwwXLANR@96W;k)xQ+&3478>Ye7Cymfa<(b#e`FBJe-e|NLa%g9HBAZ^McdRe3 z_1BvO-KhN{V6G#QS>pmYCsG;S2>7ekNdRUl0pI}F7qx<&%WFL&O0?27Euoa5HnYOW z0TW0R8JSCNP1y|7lBOW&jSzJve>BFE+6RS_X^>hh6p0JpB##CDvgafg!wh>k*pnNs zq+&zs8Ng~{FuY(-TF}D@1}_9dr7*C8w-ZSZaq^B3uuq99fx(5$BEP4s5NH^oAq?Fl zvbe~scS13sLlEf0g5WTS4@A$hj3^Wes>p&pVImBi_(rHnu_IJei50Opf1EAuqm9hsgFN;_Nr7Mx zQtC3x@g!2bSt3t-@he{`PpP3iKn|GxFqK1I0l}!rC;|yo5{#hnq#T&#BN8}==(h9` z9ZoP;#oVAnj3AF@773cGe@oMv{-QjSbp(J6XvyxrmNMTh{)w2zyvV4g;z9<52_v>k z-3yJ0NuiLAkPPVHX1aOD=&325v5|@QoI?;w?z4umxyAV;h%oh)(2p3oVL%7UO-gy` zo6oe9G>fvCohgrc-g)L`@+is=2}wp0;2%y}w=;YC)1ZN>NF%W+e?iY=ad8d_f)Zmo zQ=`xZpBN?SH!mZ%RR)SAe_<+AoSHP%?XOgo;)$~svQMmzM5~%(Uq^0gucW<5rb4MH zS^p+Sp(u%`NAzhDeI-;So}`;>ZKsCNYE;Z^=UY)#Q8GK?oo;${rV|or&=x|I@yWxS zH$#_7VA@x;j#Z3pe+^z8=d;rf8PuXPvn!1}s#l&CRsJ_w39K>x@)5#f#IV(2$UmvfRk{AYf(~PdFbqT`N+8R01IU;t0RiPy zBXPAO5-SK`M{eg9XD$hu&73%)oMdSb?i%&{#DE}RS#ATwO}~C~=yN2RE;K>!J6@5i zUC~m<8K$(Q)10d{L!-?~G%o|nd}f%bnayl`Go5)Rf9IZnHhmW>jh@fSO)L-NHzZXF zpG`e7{+#(Qixx=h-izXgyiU@$*mQQ&OkK$O%?;1^hJ2a*C>T0uF%BuUpLJTXRF`bk zU~ctiS?wnk-wV&QmJtJtaO+&(jK-wyGpKxxl24f_*q9}@iwkT&+7$-V2T`6yq)lyg z{u9ijf8I8|4+>UqPdit64fDh)e$fdnElI=v=vqo>l7oWhszV?agoxQVF_DwJWE1nT zvA>wDjOT#7k0a%aekRg%+Q=&3pm{{F3a`;E<=VlcBSv&?Bdji1X6H~{O(LMpEWzu- zciOp)asV#6%459|VcN}Yf*;boa#>H6u~YaofAyL?lVpKwC`4|;b+&)BvlUI>ue``~ z=qxfKVkcVBnGW2Zw5}#Ns{8G@%)5$A9ed z(mE2qMZ{t>NedWjVzG5cgpx{P)9E4NO2)t4teF?j;+MtK$*;__9=biT&K(u69Sg)f zfA^=d%=V6D)?Qg@u_uc7YNOOwgiQsG{)iEvHzqYgNA~BXy^U;NZc4ctC%f;+yoU1S zWdVO#uWb(7j6YxQCqJ_`dhEZ64*kDKA0pI8cyF=qwX>zmOJwhy4vzp`>R0d%B2qv& z3eV{NBI_W6X&&$)whsB^0RbH%=78i^f1Y9K93li>fi~EJ5_T>KD(&Dh0s*-|sw6=D zD8*uoVd(}?t|%evmWC=)@GsWH-w;Z=k|qy6&(NSl2Q`KTJ1Dx|NhUZDB0ew#y=eY) z0xB|L>OSycbfi*tkSMH4Gp0~6Gy;&a$AOgaA(+r16e2EMsV^ex93)J$8o>&Sf8=$% zFp-{+izw?seh?ypU{*$O3A?Z&zW(qWaL_|E4kF%5AWUJj@~I5%i~xls`|5Dmrcj#( zNp-GCh`^5$9Y~KLktp`#XD;Fq(a^x zGDJZVm#hDN#+7h!2`4drDzT?9e=!&}afvu_C_HglUa&rFO(;T9$QVf=RIzE^5C-Vx z9BffTR7VDHp#vG<0={sYdT@sl5A~qJ7sn77QAZf%iWu3^7*zs;M6oB7(MFhY`uvXg zoG%(HkQ%ST8tH~&q-z!z;~T><3*)g=c5zwgQAMsP9XZ1d;H>~FvR7zEe-MB$0`i3+ zrj83Siem116KwL#$Gi^rk6| zfflJIDvc5;MdTYuLMSbAf0RC{D7=y_l+m1`=;5?cOpIPCH&hXX8!gD?xT01xvS5;M6L6O=G|6lQhjr zWA5_WrZOoj^E9baC0J7+pz;|Gf+c+fG+)D9JaPE&#tm}7H-8g2e{-M=;Hor>vw(sV zIbi?`MLr;TFDZv0TdSY61p`vpv(3^rZ1_UXSS_GeF-n%=+^j z12k{u(?KItLOn!5e<##JFLWU>R6{rP`Z&}>FX}?$lSe8vP`>dYIxjA1=^a0mMO#!b zT~s_t6h>#XIAxSZZMA}Rk=!4 zR&!NTEp<|De^plnlvjThRD)GmIn-AP^;lgBSBsTdTa{RA^;w(MS*6uWla*R)G+JfV zT0?1Buhm=8b6c-8T)(w2$CX?y(p=BA0Mivt*|l!ERbAs1ThG;9<(1;<)n1beU-Q)f zv-MT?RYc&GUk7$f>D6BewmSh9Vedv^7j`=xHbWy-e?%3AVk_2SFBW4nR%17oV>{Mk zKNe&|R%Az(Wd2LmWKR}lQ&weHmStPkWnUI%V^(HomS$_#W^Wc}b5>_}mS=m`XMYxG zgH~um)=!7lXpa_YlU8Y$mT8;TX`dEqqgHCCmTIfkYOfY+vsP<4)=!7jYrht3!&Yp^ zmTb$`e{9bdZPQk5iF9q-)@|Rmyo>>E=az2k)^6{%CGl2o_m*$_mL*BTZv$6w2iI-^ zmv9gFa19r6>sE0W*Kr>gawAuAM^tVn*K)tMaW7YMH&<^nmvcWCbk){#Lzi?*7i>q@ zbW>LYD_3>B7Ij+}c4Jp|-S%~7*Kuptc5@eWe{olLdv|hq*LQ>Wc3D?=Ll<~;_jrp} zd6$=Tlb8N^>DGClS9%$jc&FET`8Im5S9{Mkd$$*FyBBuB*L%m8eA8BZ%lCAv7k$^4 zd)1eH-#2~XSANCzeCO9}>(_Mg*M9exef5`rWzK#7SAbV{fD0Ia4H$vtGl8l1di{5S ze;*im0~mt;R)Qy3ejV6?g%^V}*mpOWgB93%J@|u1SaU`Ae@&Q#Wfz50SanyJh4Z(9 zUpPKpSaNAthHn^qV_1f57>7TXhkN*Pe;9~=SBHlv32okKv0YHMx=97m_8Jk~1)r=XjKXl~Eaw1!0vl(33ein9d;&*qFTRQel9Em>&$7T6vCNp^lAN zm1p^Jav7KcS(tG-54;(b4_QQgxsiD}oNt-Dh*_Iqd5)7g=9U?hBiWS2IGQV#n`QZ$ z-4~v>8JIGmoPR``2gZQ8IB*r3f4uTwjS06Nc1{EI8Gad06x^0%+yHGw0Sd~1q8WIK z{xkO(+>bYYl?_c86o17d%7kX;fQ_2tP2sXdE}(6daJQ|ZnS!(!&zNE zn!LW60YV_GA7`w|Yp?Bssv%;ngQ=)}U!zQo`N)L@QhaQCycZ{JsT>!wZ~!Kzt18dqkJwz74{}m&Lky zgbf$lrvp20U);zye;i2O!P9=^#Bp4Ji@S4u`)+}Ii{FTUKf(y+R?W>;xWE1v%{jPn z@k@;Bfi`;ln<4;_wL851p_KAq=cfBdBAUpX8IKG7&_$sUPB8@_9JFZ;n$O|OP@0LL zyB?Ux(0!zJ6p%+~qR0Iqz46)~ss_m+r_{weS4NqM)|!>Fe*&0ZU5sOWJH-1*{G3Ms zUC3RWNATPq^!&k3J=gzy!gyrR869#Eoy--T&>zCk>%q|>UDA1k*(?3hfrQx&ESP{@ z*teV5e~U+kL4bN4NLKySTOCNAi!(|ba=;uV?%^HgecpSdb%KINScF)4cHT5Ofn@Dl-j3_uN6a3o16Vfo zo=4=Kyr7bz>!IqsR_dXl?>Sd-;XMVG9^O89?=U{KpV~*>A*Ulg z<|bsdfBga8;Tzsz4zw#G2!P3jr2OfnoZY#=HopRLdjI#g8qqaC{-7~`xQqYzDp1hKpS-I<^IBOtEjhMnC% z_?J9LfM3j)!s&-x_a8^X+{{Pzf9@|IATBUSe^Jk$L4QFQMEK^95(5UA4*C=H(3HVp z4>K}wBe7!{jjuY2yr}UZM~UwwluU7Nq)1~A%b*kpqou@;8AG~!Cy=1TgbV#OdN@qt zPzFUuK6}VTB1{G{5i*ca&|pdiOXFyLj{J-OIPHU%F^To_qG}-Cv2{j6HihGVx-sjt!3-T>kj**^b|ml)P}Oqo4>G zBV=;3QR3Lpc@Dyq@C=H^5YPOn)hL0nSDq0lxz2bYtZ9uAQX7O8LBhGC7eLE3SkH~} zhe2WwFMj-?28C*Jb8C*wIiye`yxM zIT8_9+E*#$V5Lbey`s=78VsqIF;!kUWm-L0nFDAneM$aQM+wz*)r|%XSkp*q&3RB< zULHlHKklWO-bs3P)aN}<&1KL>L8+x*p&lv3kdsgzB&DHFov0R^LOJRWq)~!)5}0OU zIg*wUaOvfdVdfwznP#GC6JAdRe<_un29av&o--+Fl%^CJCy^-+>GY>Tfl4&!ZyJ!w zn4)hvM3q-CFaSeGr0Hjtr2=8f8hGYu_^7V z{*-$vo!KtQ>r~$Q1MI28W_qtd$0qcwRVqP|q_xhn_0mJ|iS?n0Y2~}?e}M*lqf#TT z8QQRsAcrjS$P`N?NQ)PUa!{M}-ZB+q?Y-Ag%%Mg8eBpc#X?W1d8Jt7$@M!q{aJBGn_%xRK{^k3i?% zrTe-f(ayS7FLx9zOZ8+ziCy)s+L2n6F2z726KN`uYZ4e1Fhzm;f5ZJdPwnXt+mQ!V zen;n{WxPc7+^&_Su}ef+Q19E$0PYCGv2)x_W?;rqq zK!(5fV~8;*!MKB{>OkMq-a*E95R5S37F$`CKn^q(4P*ieJ%EBBZ1aleA*LtCd))&K z5<#idjydw{-|q1Bpa2HXD}MwO+(3?pE(9J#ffegv19vjR#%XGT6*R~N-;%-banLnk zyVHb{L_!9b5QQqVjrE9NlNfccWU^^u4Ob>DsTit+CrsfATSvz_+VOG+*$&fmV!-Sj zC~hU03m}(6L#9E_A9^f_&rakIoNxge?`X|K!g4wZnBpDKz|To=5`Ud{ykb5Pd5RZr zfl0-w>>0NZND_we3WrH;M3gwkIYbl~P8Q~qlU)9qe+1EpTQDgTuZWEvEg8x2oKci6 z5`-y48G>QF15XbTge-9q6-Po+k`RfULSlIiJx0ZP5BkP4m4lXlafW>A%8KxAaf@iJ zvXyv>KqJoajS3azX@3!LLnh>rm%%lXDc9UhSPtg~NFIeH!x)CDG;oVoFvOgsG)6kL zsStghV?Mc&r8m*V$kU1D9p^aBv*J0=dS>LFILqfgZRtr+4zNp-gXA#Wb3KEMMvC*h_aV4JAd4G^^++r-rVWdPZs;eY& zWGp&8$hHI)6|nXzQw#Z5gH)3?wFE`2W=Y%rP+8YU)o9g~Ko#n|;Do5R3}spe339*OG8-l*>4Os4aSR!G zU`Fb(D1To=@l%( zU*wp|L#GmS%d-AJmO{S@O5GGQa-kGD|8`o2P;~8ZkwNrvsXd4m{ z(3lx?rmbECoquAHobDk93(%*fxK~aeh;5PbH#We0`#FGdkAAW*=?X;^}9Ay4j+} z;AXN}RcdR~+SoqnLGHmc3A|dX@UE#rtcThn&j#Sx_Orw%ZdbN4fCv)V&KcjUsUE{+ zi+>*1Es!tE#)Uiu<9QVDL7tHbM&gMh)}xdMIf5ryfe#u_(eA9-=xC9sMadlr@-x@u zY@9!YsM7Wap2(}Qc|CQW#F@Dt6Hz2CIe!DK&JalZ*{NfEU^nb*Y zQt9t?`qZ!9_7>hXy}FP3?~9~D+SPuN-mQ7gA5TONNx|L3JA6Mfq~>(dpwCre1b%J~ z-))g1Q2~7;A$<(cc}C`Zu7-V-qJE3^da-AIxzcleM1Hb266hx>q_=*2VSot8X&NDa zF5!EuLu&u>5J3|>SJ8h5ffI8QI)6-4e_xS#2H}CPG9Y>8Aak*T{`XG+czHlbN3UZb zc~BPjKu^zSe_bbB%mH@(qOgPxQx8t4ccpeoL}-L67bTRy1&V_un8twFgMznrdpYqG zITl+*c5VV;dGIA7V@QU<0v2(lZ`LAya%gcmrfNE zC-H@z;V5dMfN;l$T}XFCb$8e24}(~U8Q}(vIDw!?fF2Y@9ifQMCQbrDRc`o(InhsR zkryJ8gF(0#u;vfvKm-}V1vU|GKUiq00%yasR`B-_A#iBnmu?{V8uVfeIAdudArI++ zU{wf)b)|>mw}=ZNjkwf!3c(hMK8bT zBFC7F5@C+afsT1G{)@G!ZglZAP8E%g(sV3Fayt=xQ#2Cy@CwR-E3t@=;CNKl$Bnw! zVdn^D#8))y$cgP(jQ$ai8DVMk7-JNP7YvCN`bZ_xGmWEo4*ih_hJVpb$TojzL6G}G zh)EG4(Ue*r_KhSdF=3U4?;w-m^NK=gBt2DOp^*fl;4K`t5P|qg19)k?fr?mSlp}(a zS!ic8B{$ayhVQsI$mfuIm}~Z!aifMd4X=8CjB8~n84OtmTJ12f! z=V-{pG*4MT-~urs0e@euH-b|_muGVlOG!nQxR*s!2DbHwl~^fo=@n5In2b1XU8xbE zcw0X>5`&0xL;eXX2$u<=u_tvUZtHS7g3uwifG1@z3=Ax6@17B%riEmkId-8>+8JUt9AEnue#E6l=7Jp5w>6)-9FdBiI)XAB) zSvRlZIJ=o>*2zxT8GI5!nj^w~a%P#u=_uo23hXot=|Ks%&}3e5b6yc?;{t2yxh2w> zgm+n;WOto0k)5|mj(+)_j}xA{$(yQapiq&WUh$u$q8IH+jZp(%k=B>v!G8JH6+Z$! zPXSf_8J&Z7fPcm?pENNHG)SD#6PhyG7l6hyCDCxs314{$L|vknW(gI<$2D0v8#zD; z#&8aP^%VY~IAP!eOt0{yuW$<;k{HQ3plOH~YO$GPQH=`Wq);lQw?L%=lytrFcE*#I zX(^YbL8L~Cq&FsdfGCbm^9~*)2n_|Luh1bcDldsxrGLapio@t++Ek=RN?3_mk+oB& zcM7F?`j~L)oOon*KKBmwAd0|eTTP0mQ97ln2vzdnG)a^ZtuWGBj8LZyPI<_jS29c}VWSM3ut<;){*ZK}HkOG$H zBuP}KU&VM8jK!=juXsR1=t&G{AB+IYyLaH>2 zc_RT0&h;Eck(jMEr#CtCFlln@eWiH3m+exR9U zb?3C05w%;nYHJCYTSXHgHZWU9VM6%P1ynSh_x7k7!rY;7lwkBGKl zd4H(clXn{kKa>!+j!HbZ^)pZ5e4l7NOWRtDH*Q*s7+jkd73Pf0#+`Zu1qQ;bG}}{n z{vrW=hGLZatFx++Omm3c`LHE<6Gu0D=ku+VnLF-btf$i%coId!!;J<}yYM5nQsR8v zGkd*@IhVB^p}R#KHFuut62wcQ#;dJAYJa+En~{3qg0r`+C#hS>n>&AFea5O=wkx)d zb2ivmcO9{%rv|o3w78s$Y@fTltJYP~Taji7z2J(vdqKS;VZFoHxqUV~i(9Nsrn@jA zwoLM^BAY#D^czfxvLc}>5R1Hh_OcpL6TLEj=3BEEd>5}n36#)-^>7-gp&3LKL4Rst ztXJWf0%5``Q3h|@go4Y#9t^DLg@?N`B|owvv%wK>MlS|KK2=J!^s7xdh9^q4!~Wvc zJ;#}JQgUK1*0uRJFlx#XW>~`<0hwvC8&R}$MplOc^SN_-5HuAe>d9t!>z7h|WK^7l zl!#@X$Hhl+vg=~Sf>)1`XAs028-HP(o@>UcM$ASDd>5zllv1KJc#>zQ;}aP?gZh&l zUhywaxf_-HzAP2dtV`33${AzLWKru;2ubU}<1X-@=c2On5m(pzjBZ;XGEaaL7BMOFS z$9M+Q)a1lg>P4c^N|n4#NP27yqDMKP(_NFq8Z=C=5J|gnOjTO8fpfMmjY%@Cw%0d^ z^Yjks;VYecN!0xRg@3 zjMUXl{SA6FC-D?ZW-V}qX4FUx%*GrOkc3NM)^rG_BSCz5KjT40U<{i6nx|iu2h)UX zkR{2urWb>G$>_S8nH?Klw?B|%4Bv>+@B2~Epf01MBS8QS^59X@ksbp`+qR9{@_QZi zkV;^C75e5y!w}r_5q~kW!P)146UpX@YlhiEyV;$MI-`4G!Y11BOxhJ|+M3$19Ox33 zE!}vU6!DY^AVmqvfm*tqV6}bQBLdz_gE7R-5K^)c;^f$AgQSsLj^*wqfk>9c= zFd1dx89~Vb&H$G^;edXp61jdw-s)etGIV^h2C(6I59e{1b*U zo;S_=F!Vh_&2l!~@nwAT>#b@?Hqq;;c-`=v>E%wVyk>pMp8nwZX_nmi?A(!#)6O(E z5^0dmMcK|Y+b&&2Vm%BYG!pYcP_5@Rum z2=os40XIz_7;2%6MIU%LiCif{1W}U-QV$?wzkiG{Poq~uv08F*h{f?3{4=sS!LMQ} z{;~97me!%I7X#9Bq=CDCHNTSbS*p#7Hy<^z?eb&|Cz1YNGaVrhbzTPaqGkD-`K;sj zzzF!Iy7#+IE>bW0o_C0YpBIIH7l&W);}Pb|?iHkebg$9~DC1+*UirF8nMSn5-ZBhm zPJf8x^$v9o1E{a}W=Sicuk>Eg`q*vm&tJ8#uNSdjx)4tL%YXa1KX;Tr_vG~qjGqLy z!TA?4{1-5)xKAFV8mYD4nPJ&M=->VZU$cPr9l%irJ?O6*!38c;hX5f;0|SB#LfrC~ zFrmPL1`#Us*GNLc9L69viqJ3uM2;O58h^9M@g1~h=M<8B=T9EAdj8(MYYC0uD~|U_ zK1=CQ=T4pp-(6rpvmVKBJ8eG0*;DD!q)MIg99i=IWYeb!&mk?U^BYo=Rh4d)>eXx1 zuKrMp-3hiVTAm(MV;%!ECE2F-VFRmh+LJMMTC+TxCz(RL0UTlQ?ZT*nfPOmec*(>!78 zbRD}lZQ{kN2Df@Rb3(AJIS%&-z2oSPSD`Ieio9nK1{y3UG(vC~@jN-qpGTjg%+H|4 z-M`lhE&Y5LAqk6uF&}*o=kRH3_kT~J^7K2ei~!#Q@IVC@WUxWt>~k*2oN_Nx; zwHu!r@x2#;l<}S+#VT@1C!Zv)NhqhJk}oN%#4^k5IMlMs2ff_uN|#dDQGbHG9DHY@ z9>HX@P2%=p|JM-jI!ae(>@Xs#iq@;&|7`Q+XB_+(m2rdDIG*Uq$ zoo!M|FD=eeOg9BhQ%*k>%u`S^eN@s?M@2PNIZsuU)K*`ukX2Y`4O3QHZ}m!4Ttl@r zPhEEfcGh2mg=kn}Q8o5hPJfeC7E@-MZL3#2pM|zsU!}FyDQdIrb6alf^!D30!{wG- zJ+T!xU1!l%mm_oA_3~YKv6OdSD(h_)U;aPYwYT3k^+lFne>VkqU>gfI*h7SYU3kHU z6Nb1_f#H2P;tMObc-4$w#aP&h!R&Zrk*yBynk}bFULG{%{S+~bI(5qJ@m1qmZx;nPe(m<)mLY|b=O~qJ$Bh= zr@eODZ^u1%-FN5Rc941pK6v4WFa7>^;*Uo@dF7W+-gxGphdz4gnRkAA>#xT?d)KSi zzI*Ri=l*-#Nf*C-^Up^={oKP>zkS@g-E*~4$+7Lyx|d**t;SwF?dZB zUk;r(MJm28iGNib9u%=?L@jQShh6ld4#5~k*^M!KQY2#%jPy$_lzu&+k$>`ZW^pPE(q?{%^R!m;g;DHAkAqhpO|LBwVOPSr^WXzH_4uO`S(+7t&@9 zwR9yt=}t|z(w6cRpD_LB>t>qIMt(yS{sd`I*MEsRW8~DWIj!eJi`vYiMsuN=fQIT0 z+Dw8n@`1I~ULJ}mzIPm|I}$zOj|w7+U2kFZ+tUqW zB7Y5$z)p<|qvW2>voQ)S39p#h+}8HFFp4hfR9o89)^5A2W32`d%evRXD7L7ZZ9{h1 zy4&V>MSz<+H~z%P2ytCej9wwT9xxyRc|F}Nj$4or0GO@Q)oei`VBOCaB)g~k z?&@Y++t@`ob>SWF>dM=Y^PVof1z9ia+JCE&_nI!g1tD(h>Ih z7D+jfC}tB(kOiOa9QiFs9#FSjHSU*|O?}u~^!%_OYxpg^a#@I#CeVv3~@S zK#Ws2=kB&M1SmG_eoNP96aAS1Gh!X14dQ4>%Y)4RH8Y?M9911fAkz#mQ<@6`12!|j z&C)&Hr9FLSMa%v}&5thisZVF;I1if6v!gYh^&C4%hrrQ+Hu0z(?FK_5x*)CYHKnKg zYEq*Zn(rtzj3~^y+5We<1-SqbMt_aiF?agYXf7SIW39SevyRZTtuvf;{pdZLF4+YE zcIo1qpMns3b?-RzlN0T3KX03L;&yiF(hCC{Gr-G{?l+(X4Fx2%mZ+)rHmgeqZDY6D zbgKSzsL2@S+$kA*S{C1&BY@>8a>oeF19|Ye-0mzQIg`~p5*K*+juk_i%zqIuXP!YV zXsjBx*3LETv|VaapQG*_nGl4d5B|Es66B}f__Mu%e(TjSy1=EgGo+i&fS*1?v#mDz zp}PL{a1%uS=JGg)woUzSv-2|T7PoXpa9%o{$2*!yjQPxK&dZzUT;W$Gx`W#pblCM= z@XkfL(pkRsr$@bXWR}1`hkpSS3CwrVCIWEQXD;@~lbwV-Hs|CCVv4H*B@^#BN9t1_ zJ7Xx=z!c~DRsTFxn73UXG~adK>Hc?l7+&4?Cj9Gse`mxey&y|BDCc|c__%{w@|3r{ zJglDiT{FJ(GXWpyODB2;kRBRxSM)RToeAJOUG@aA_eh|z{`G%@JAV-Xd-3pYd)%{I z9*VuXJj!crSKI#WYu`ARlOHs?3S&5odwT{0oU(UFKx3G)Zvcr+E1mxOIer2OsB*3b zoU(KH{+(9CI!3$&^Xo$cV1c~Tt9mkTo(~&`A z072AAz;;VH(`msTyni^5kf@bYogK76OLIUUgF)7T9}%QM9(0`_>?{qu9SBsw0CJvl z01D)jhyJoQ#VRp*n6T1uGVEeIs%xDz1TA|SFoFO>b2B@V0*Xo#H8xBghKqnCdcSLG zJqF-{Ya_mBszcKuG{|Z?@e;x>9Ir(zC@(ZPULr#mLpM|_9e+s#Lov*&N;I7_lm|CF zJk@!%OLQGk{5{k8hB%x=*deq*RK+{wB|9U%Pb7#+;zK|5FF-uR?Mp@UA_(#`9SNRWuJ9D;`MXow4lzdSfI-6|2Odj~t?u{?mLIZG~v zqcODGu@8J5FN^?U>c}#@J(Kh=lq{YI2(;O{fHOnCiGOngpbUZ=G6tUfhS>@xP&0-G zGlF-(2nEB(jf4@8jK-k&upT=dQ!p(5vPnwVF(`P*c4A3*129U6!xQN&GP#9UmpZ%8qO`?qI1nbTgbRwEGK7L8G{~d`s$jg{+}`Ld(Javo=W#HRwd0 z5*ba%yts^rN6!RJV9d_aB(u|u%|DAr-*ipbL4Qrv)JC+c&4F|sLeouldk4qjO?eoG zTj)XIOeo@%M^kLhR#Z-b+d1a~3ex%!3D`i&lfe4YfIfrE`m(#?xdk`!P5`w2#qiv; z@hqP5TuswKPvBHf4{cA{JP+GsIr&7L-FylAl&JjNPg&%%?qbfSOU|H_#|dQ*d(=*X z1b?+}qRzZh(bHkk&wEEq?7i=FF>C8D9Gg*ev(D4OPKjJ1?h%AIyZ~X8haw$Dn54U| z)GgHEN)l7dzVt9G{UB&~NgfNUvV^XQb2RnxMIfm>Nz@1`y-FZCwffux)!m9#G$kfz=5QNfW zfU+#j)k(QXjgd*+QY0DGg1fa)?T&By%u-b_>H-q0;vP3`2-K2Q1{l>O{hg#tGxdtEqawedp!q2qp%;D)LOMRe^pHX+9z|Jx^2BX zA*5E*vDUmo*v=XVh3zbIwWn`o4{-%3i&dS7jZbvFJ#|GrpcvI#XoN<{gku!Qcr`zL zMIC;{S9$Zt(t(DFAg%DC1SrUXB!3WuO%vEnOT$-aJ-BRAc`yb_Xaq_iIcP+?hAo|k zZPjd@*o|#ZQf*g_<<_btQuu5gmE1In^D;>+*^^D#)`3^HYS|tw9sZb|Ez;^O-mKX> zGcAoXohr+#pT$p*1zW_TxO+v?X?0hqWw_|nImRN`(*f9}Wyty45zg{hd4FIn{=3wA z3$`7})E(i}(rK`Q_#ido0%PF7MSU-3r6*%pRN_;!9za1;(5&4$Hw`ca4zw}GGKNIi-hZz$D z-;KGl6GYH8U2CLN|7ry7J%7YNG&N`_DIM7-M!SI6HHJTfMQ$>MSHLaY)m_Dkfb9L< zf)FM<^)EM|gidW$((+vI72VRwFKs#p;qtOuxCO_8T~o*__I3VW_zmCxr36*&U1-?e zg*6@gr3Bx#vu4F#{WZz{O|S?6;Bye(Z(v>UWx7Lw0?@sMeETVm9DhVYn^5uL9*H6| z@#53_d|Y9z!0-rxk#$$$)gmZ&B++AH$mi~wR6T~V$jV%3>qNyJ)nj7J2fV<&#n9B#9GD~Pj8 z!f$=oZuMgD%quSLtbc8U;mVAW5-Vc^lSDw~ze+w`8R@|u_CYvq(V6ArIjq$pHV7)F zol%a$Db10+h zWQUWeU)G;9Cd51zBF~(N3vk~3wZF%8S@X>U^rZy#HQFHKW`A#{U=Fn4aHM7tW`GaE zEk>NUZe`|aRJZSAk%FpKFVo=>6+sQc|_&AiFmHljjLN5@R~I zQ`6ykTIh-;zI86B9Sz@B+uQz&J$^H_zD+@DgdOfkKQ;q9#X31Vh1S?fIf|BO(pnK& zn^FR;G>`5qPJcvcXtGm`7FiDtS%W>DBe@0GD`|o_Y0XM$fu?6^bR9vwB)oSeO{%~cTG^DVzmE&8!J&Y`!w$4y! zaM@?3Spy`{Q}tJ@4(svyAFCEtejehjMqHov>T?3?t$!73*CFeDO4dg+tSAn!y}U}u zlIy(AD-}dRY6CQ==CLv;g8^eF8EhzfJG*Sr%xt)pu+FYV2GGaaE;Q0!DmG0WNo;MNersUVq%~CW6y}S!@PXW4^q#Q!{*IG?XM0yL$&tB((509Vqyqqs6lFf~-KSwKRk* z#TqGkeFj(dF^dhtn7m!1YwvULZ}1H8T^nQmWBxSkj@ua_G(2=hx7Kaqv+vWvZ%R#R zKAw?;?5t@Lg+L6}?#|uX>N*>?5LgXB_=aB zJM7*&4EA%u0nhDFPN=P5KiwcVQQb}eEc2+Eh_&uk?ilE~?$~)&WC)eO!)0Z?PGd6p za(_0ipDibdE;sb7o;Gk^0mU)fX!LpMkB9yG~deO(w*^xRaPXSnm>hAcfl zLbbK6(>14XY%%T?^79To?c`m~!qpR(t4RLi@|kP&zBct_mUL6p^!2m!%Et6IKXYQ| zbUF8J4FGi=u_{t`*Rw@+MHW6*&r{X4W`FE^olXxf^G4Ft3H9cVCe4aBK<9C2{-YZT zM=OIN_2VV-{x1=qI1;D#+avKEf97`2gLjuAY2pG{g)HG#a~>s#3qW{;Uo1isV@ln! z0(Tv;O&xtruY_0l(;_sRL~X1CSvJpjLTq^M9$t zbt$zIL})_gIlfQY=F@?u5!2C2hr9Ita^>2vuWs(!=-YEFuLp|wv+IHfb8C0>RR1rL zFJ7Uu`@IHhE)7VYvU)Uoj)xJ&hjd>@ z{MdDtgOQOn8_g~Na&_^B4*FKIe6Fh}{y&8xc4|i2;{W#ZTK7ac*I)5FG`WB;(meSgA+_nJB7YgUnduXwMtK}xf*g4AokWTjL8lI{I&8Es z&#hOlGA%4~%i_eB8roVNiB;y`fssE>Cg{c^jIIL>GeE4EamL3o34c5MW|^tdl6hi} ztteY5#Iyt9Mr4`0B=3fq2L`6>5N}Dn3;FszXk1?LlB`D`PQ19W>dhWI4}SgAxmnJX zKQ9dJkT(?R-qo)p=$d5CrTzqG+n)Ii<3cFs=;nMT)5uqf-7*^kjlA;CdlEbs-EpE3 zq#Q+kT?8Ir6js)rJb$E_c3o9s}kPgdTe$4m90B?`ag@ZS~1? zpGW%nmcw3x7-pkPD5}U3Z!VUEoI?;Y=i5jN2~yH;=Y3@tl~r1KC6;$N$dm*`JxP;b znkkhMh9!|{q*rH2rCFCB`ST{6amooNk~=YwLnFgLBjZC;I)C&`1SHW%i7AB^`buKZ zMET-+K-slZCM?kh&7g)Jn&_fl)i;|2@@3^*rkT?CX-G}tsVAQ>g87cB@_h5nIlKw- zilwF`w&iz01y>t1t-Ai@sGXQ#(S9X?E6+|bwHV%6@^W==x{Q|DT$s?}+toV;QAEjD zYBC^iNqjm4QKX}O3Sv?~(lnX11m%S5TYg$hkwEXptL1HtwOe7oAJ4n7!S?=(tFNE- zE7ZyZ1-u-=B>{S8NzgQQ<9cZ}OxwfH5d=+>7u>?qIe&Xfy3o-q@o2Cp@1&5Tab&)n z@497-T#$wEj)ZWC^hTxfzex8yV%IR2EzisYvvhL~Iq%Gp!YwWIv&0xd>=4ll8Lieg z&^&|GXA%?Cb_6y4&qQ9+K4A^g)&!a2Q4ii`+}p+xV0ZMZ>8S2uJL}v|5P#Dp7dL--I3<(_JyY_=1-c6qOMj9@U!t&?&a_LNDA9$ZO=bem$k$#4+PxZN(1RzO}5grqK*mD?7Gy+d~daX=anIZ!s^`M0`!hd;M zMA;bm$F@SYP&o#5BW8HFkb#N`pl{6FQYNBJwoUD%HN8c%&K61}1rmk>;UXbRH&OE|s&xXHto%!2F<0qz5*TrWB^{(G6YP7)qHyFbKQk8TMF0 z9+4*TdHX?Vm^2h4t}F$wl+mm6Jby?K2txHw1Pe4jjd60*DC7J6h?RwBa<@GRoHHP+FJCn$~PaY6$mtke*K*0_~ zIoJM_TtOfE%jZqDB$XY=Wl!c4%~FZ8TRqZe-zr*%ly-4 zv|T3+mMchU>svRHOkqOyt3G)lTAMdD-U?+QP~|Fq^XoX#NUw#@I3sL)`iv4<1f+Cr zB}t`t-KCbar=E(;E&BVXcHS;`CIad+P&7*fQ=}2b5Ji(#LeU&xn4&T9XL}5y0a2JD zz543#t4PCufsVMuYm$~iY=1;i37Fy>#z;aX_hV7rV3d3pK1YTQwLm0sK7N`6@e_nZlLm$aLYS;+y_~K@#?0?{MU%=k z&H!Hc*TL5G89QCcU0Rp{md13`?#XFrcv@&MU3GJF=rU8w!yIPyrxmm0Ti%7T2$^`&D=9B!*cOp^{;j$Q z-bwGK(w8o{t46}YTG6U4y}pI9^)2b4OBzWKOx&J^0dFAY;D6rvj*XL{{0%8XGU`#T zBqZN#WoTge9Y?1`(sMvz+to=WYT747qY{*FnEfS!tyo8~f*+u@F~l}>Puo8tSR4cG z98)7o1O}QZxEGlu=g2!za!@yQ3kF3mQC!+z((ssGhLJD%t(dh8FT}q@my$HjsI&u? z$bY0EFj_ps%YPfms2J7qAHn?D*v|Um`b2s}iOD;j#=S3izq{g3RF!$g@p@cs?P9pabX z_@o}mj;0wTTT5gyVgL;uN7rK;8uY;MmPcx;(?*Ekp>s0(l4q zCA3`-0Ds#ty@=~^L0?HgJ?)MR8N=UA+50g7B{*Kebd-2!Bn!bXy&I%M}J9C60t8!b=-ciPGs^ zxiBDTEmfBBUia zz<(pk1U7y|5O~YR@uEX`V@Ea8ifG~mnASP=n;gCcL?Tgxz~lE(20MC$A`a5jR3pPx zNj^#+DEdS{p2a@~rMy zgE|T$Ql3!06y#oLO-tk*7o?z1A!0k01bgRr7snxV-Cb{@{Mg~r*VSkGgt&z{>Djk(hmbbOh2a73AsTI z#HW1DXAYDBb6qEXa?ZxlCw&U!=YM1Z4Dcs?c4vM<)8_CXdy@Cj;WbqD3PA2nto=Pt|@7@X@f?oJEf^iG(rxT-316j zb>^2FSel#u>6XGNpeiMu4yvKzm!KZ1q7LbzHfqros{W($V4~uuoK9+_UMi;6g{5XH zxXE9cYJaMzo+_apX{DyBs=jKZZYr!I>8#c&t=j6V$||m^sjlv- zm-=X{`l_&EYOm(1s}8H77Avv_>ai~CnkuWaLMu`RtFubMRawT3IYGHbeq>beqZv$pG!mMgqgsJzx{z2?`s&a1x4=zqOBYQOI5guW}l z3hBQNESU=Ix)!W6<*UIqEQBiTp+c;~vMIt&?1xIM#qJ=+g6hV44#Q?F$Wm;>c5KK# zYRQTz$)4;prL4QMYLv39$^tCRn(55O?5f&qlh*z$&N^ttqN~nYNvp1m&<-up7Ol}9 zEz%~f(k?C2Hm%b>Eq~NTt<+8})mE+5UM<#Ut=4WW*LJPfel6IBt=Nt&*_N%@o-Nv@ zt=g_F+qSLSzAfA~ZP3Oo-PWz$-Ywqdt={e}-}bHF{w?4JuHX(X;TEpp9xmc0ZL7*D z<2J72J}%@&uH;TG~>3^1PSeh>CrmpI)E?9B} z>%K1R#xCi)uI$!s?b5F8jxO%zF7NiP@BS|EN-5|DFYz|+?iMfdCNJt9uktpp^I|UZ zKCkpnuj589^<)4a8*=O-^7EcU{uyY125`YWkgzG_zy)Z)FmP@oKmjE@Fd$2D z^bSKK;}Gb*bxE9XHr zvlA@KgDi_PExUm&pC>NsGB3|bJ=_8{1M}Dkvwt8E^PDI%GB=?)i@JJcFav#a5Shv$eEcJQX zo<}}IoXFVawbMDkfZJt)oZP4`Q5as=G@Uf!^L()}__YKK@K?8!R&zC9cs2L2(^;c+ zSx0GO-%eUvbvv=O6uLE>+=5_#@m$ySU4JJYU$fI*$I0Bi1VNMCX2;1p5CKuFQw_(- zR=b@kDE9R+c1Dmj1I#vL-!?g~CuOsfWp5&8zlmnQ#9&icXX~9^$H{27Q**xwT$hJJ z8+K~H30z!`f4n+QpH9KKM1bjnKV>K3=H|a*T>_S8iP=X+kFporlD1W2( zdHSK{HpOpuE>YWYoFEiMJKd`*#EYIM%TfimUkZkoTJmc!4i;f0Is%2RNL_ zxJMXxoH)2PFL;A%`0YUWh=V1Chkv-0PPa>3z&2LIhHs^Z!%2yQ28ny5iNi^i0=Y)p zxQj;=mP5Hmymxl{LyO~hjDyCE&xwyK_)G;kPds>eNVuFFxrHmaM=<$bOthSE_?-fp zlv_EK12>?@`I~G83|#mALF6<Sg`{b-H~V}uceu6Mw49R@q8dw(RidufDwpxFAG;QDlrIjsAGtP^^z-+GWe`zaIq?KFFv z3;V!(gs=1av2UZYH=)4;HM9%)-+?>3QxtgHPPT75p?kZzlR3Jx)3={H)4WL_iNZZp zJGHkHyt8{hy7tDu{)zNNK)yTio9uhPM>v_+JHXqO!1KDovy;Kk7=J$}ypS_{!zcT) zze&VDd$hkv#lOiGHHM%c1jp;V#}^36kGsf!ytqg6H`IGIL&M)a_3gC0yu0>2yb9Y> zi+Gc}=E4ci&pKtN-NI}4UTiuHe|qXh??J&<&Bw_-2&SHcG~r*nYIB9&ze%dgDP1nU z;lmck!Of7j);uX^lLCoR{0oS;69mvUX|{+q}?g$X7-*TdmAE?q*ybALynpoF&YWQ48HE^_&(^e9*cViiJN>MyC+qdglwEIE>5*R3uu@kLhHE$l(NmK7z!$5;FC0f&{W42^P8gvR(>O!hkv1ie*obxKot$*Hz_0-hVU`<4i#_rDf;Op+}cKefn>l z=bk-#M}6TpW6z5B^bY>*`0YKhYv*oV;kP6}FWmAdNcP1DncPGkn5SlhJZYyAdgp?^m0GQ8WXanA{mmBN|c^42*>_*+zTfdDH|rpFrQQ?N(QE^vbYNE zyD&QbXij3zjfCo!&O8vPsTa>olRkp-f6}8)iOpO3lW1mac)pg&AHPl&yt@WZ? zyMNrQFv8%)30ULu3b$a-a`Q)7OAl5E*@P^TKscccs`k~K02_{h96TBpr%4S0S6qLz zh3(waKvfr79#2g*-jwHswq98imULf_PVyI~`5I({uja1le>el{dNT4FP8cCoMt7l%Ipg0XOnwmzCxw3n*95O-leEiJE zB7iVVvffz^vIw}GDRi1poZR(;4WB%5tY5#iSjdvg4>An`KkA(jGhS@-z95|XD1R&m ziv4uH3Z0yD7$Ae&AKiiCo%Z6}tlesl7H3Q$;eq1(`pX@kJaXuuf=pMzBxd`fAo5K+ z?MYo;D8J@TqBXA07OKY}uqky9gQ&_m{#zH$YG#?FC}|{_?fIqtNeY1G%3{VAwy0@J~+| z6j;avSi|QS5Nx=j)vD^|5eGiyB7Jk80AE0$zYzNZEe(RucpltN(L}fqD~9bMnSeqM zpdg3^sSs*CTgmLcRw^^HTPIkXuA6|zLAI1zt@ zP^6s{FS5amX_175OiUIPV#SRBav-jnmO)l%Mi=%3dLP^#LEdP?MfOW$EBnbU+JZUx zt&)|jTpNofrx!oE&q}vroI-4g7&uPOdj5bVPzYzi#c%-{?+{RLR#FvHyaO5#O3Y&Z zf}ovO3@Ib&)4(m52|WhD?HRW)ND_aB@rsGv5hav3$2oGu7idb8nwKGvAR2LtOJ(8} z27;vomiY`CeK36t2!S9>k-iWN;~mGjh#=0nn6(VEn8zFpf*f>CSgyq^UiwBs>k|@F zL!$+?n8CjqLnpV!wg8rCTr*7jaxfwBjBLUlcf-@35I>xV#nPmYx{#Lad z0c9=d$(qlC!XLTfGGoPBi)4SxWw4YOl}iAE7a9G*S)>$9ukkxrcGcJ zdji;c2zEM!?c;0rnAo$(#H^w;q*KKN*_&WiYn8W>b8IK)Opsz+5bl%rmlKf!F^LxROe>fWVGLHUMkau6)wA|$Tc>Qr)!aQ;9B4T)cZ ziQJ7hrsIh)@gkLK%cg&jY+qQuCVhg6tC6PM!^l8S12M)Rf&jc6is;UX$NA54>`NTz zFxYee7K{;=8iaA)HZffGa%2isDQqze1`9!Rxrl=xIroJDVa6>R{frU|!qQG{UdWr{ z91$evSRp%>E>4VkJ`7!iV@u71>G&n#_0-iP%6rXtyVPADK3ad#v>WV@h0Iq-(XG{Q zy$BTYU^)hf4<|oSRbjVFnC8?YJDUnNUhq500PE!&&$tUDmyK!-PgOiaI`yh$EnJ*z zGQ81|_Fs+|9a_&08cFhTeZb~xiUsA^v{VVPpJD6_3;Uu^ErT1HbgTYmKU>-E9Bhyo z0Oya9x;xfBBu;-$D{XEMu(sGn_@SXQ?vRwbD(AkKN7SwDc2n`)#jfPMk^K$9&_*E! zjmf^aQ9J$>Vkp+X$x~l*of9C@yw)#>udeF^i@Z;wr09E3;Q} zeASDdn3H%iDyv~24vo?>IqCW5&$a11SZEc%u*L`UWGrtl2Z$P8+-_ABcpdtO~SDuRpjr>Ds zoQWi<5*kGLH3m;YX?l}4$!q_{MvccpO!u3G$bWda5q%WmzfU*z=Wbseu2aOP+n`VS zsPBhjE!Fa+0Hf~!aRN|yjYclu70y8vngm=h#QcBO?IzMMexMCk)Gy>z%m`*`AQt2F z1_J)dM*bEedzNnC-eUh|qyKj6|2FWzn$OilBKr)m`W(&%cToEr&?6vFAtaCjEs!QK za91=?+d9zw*e?X#P0+w&u>edVR8ZwksEKOj)h}@<;_04H3^kj^o_NC=P!k7Lj8lKx!2ajXi!sdVFGyILH_Q=BP5! zdn_+FoFhlZFb#u`;Ld>vXo3rLL)vl%0N2G;)X#uAYq6Y!9A{DJAR`3w#TX99tq9^9 zCPsG{FTfbDuukg(l#Up_NBo|L4amyc-&m>L7btuUsp=)S3kJ>6tCc#l5;)}uN zF1RkkzO*qcUCDv~&OVYr6t=}3Q=)$rVa*jW4i=~JRmu`2&eFv=Wp8dtAP+7hfkG4n zQW6so0YQ>QlI?Z2NiT0sF!W0S1yUeQF*2Y5FICauU??uiuAe5yE$J-42<#buP!};o zGCiazfzk7H12Z9`48{u?!;L-kay2>=Fvg27Yl8k6Zwm!q(doo;8*xJX>_dN5PAlQC zO)vyuN^ape%3v5&B9~yyB7ZVpn6t}v=^|iYF=yysoO3VxjVV8ngY?oCJxf*8EjS@g z`24{)PX!3C54IG7IE@oImD6ZyLOOl&{-ASrf`xjj^E!}(3vNL+^a#}ot2;@uG^Mg9 zzRowy2odGc_qwn*nFBa+OE`bK#XUy~K9>_evr|3glXdcwCiZjn`tv^xNkB1(K#iyK zq!PvKLl96%mAeUQMrs%|V;Dk$MoeKxUxE^DVa|5*NUNi++6FS* zN-gctALoXgz=ylbxt33H7PS0c@8^8;vMP%JUMPQQG!g(v`$rt zL;XP>5EVCQg20fpM!SE{CQ1<87SJFx(^aVy^*{n7tke2R(pZnxS9*YNaN-RsPf_m( zTE5gkyQJly@Gl?&JQFlGRV6_Wi!H4ZP~6~G*OOd*k~V2#EsnKW5wIO`>s^smA)d9I z977(aZ#$kcKppN{Kky{9bs%vCBIH5#{vPmVz}5B!;wfiiRu6xYLXZPpmsMR8F&bUR zUFkJhKXX0V=v|!^THiuni^(7QwOUmUZUR;=5VYR}Ryf`v`UE157FO79Pj^BM1|@b> zC>AxV=N>#~a`JIkiB?hw4jM9n3p&J3ob*r;)oI^BQA5>C9~Cx|RyMG7>$D9jAkQDJ z&TB8Cu0)O}{&IiSCeboR;_1+?pu)D)D1mIvC^5NkA%II~9>q_m26?phVhfe!V3sJ# zKma|W@G`+jA8zkn&m&?_SugS?_Le=JG#GV5aOtpcNp)Sy%QdS;A(m0~6u08W>o~;r zZJ#5^bnoNn6PBQ$N^Y-cBUV_FB>rayf-r9nDlm{(U#EXf>yuXB;sm+xCqj+Z^e9nS zwmts~Y`3X@er7i=!Zr=aDLN(UJc#?Ex8|;bc2{EwVG?;IKz4meG(78ev5j3XjuJa| zSslZ7GlvVxKvjT_`-pd71w%fL!+Dn%C+b&uTWgYfhd)v@Or>|}I)wprRcB$hD!dna z!#7?rc5;7o_hN4g|9V1w1%`dy_XP)+D&+Tg^H!Cva<2~H&hWu7CSYwDq0&po84=q=Rd02ZN zG{69?SQ0}~ZUsSZV$$+u^Gg1xXV}!+h6KQl1I~Z2_%E6UF&C**q%4vAZ7L6x<{%O8 zf}&9UVL)s%?v##_ip6^m7intGMgwl_q62x}n6fr?!#Xur&~8EU_;R5obFapLPM0tZ z#7FqWH{oo&#+Ye(rzZr%6+7j8j)P+a!y0cG_4LFcR3rv5V;}}3XBn4vhL*6-mo(pk z;*5U}hSm1?+O{WjhZzm%gOH4rlSzEiqd0=OB#2X&bq^}W5E`JtS7yw!I?bqDv6Wr! zl&hk37lN7@sgHMe-C`NmFa8GPNT(+IPM3k;k7Q(6C3d5#tKn<0NIoPB4O%d01B84wMqGJ5VU*g1q7>6q=A zn6tx}>zS&&&n#R~FzAt=6(X7gEE}sg5D8kF6(U~}ViMy=bzHiXZ5k%kj#FMmPdUX; z5IT8j4@l^^=zs*x6nS5A8bh-7LCb%#L0hr z`9}5xqtL8+W3U>o;&zno=@pD=K=`Rod*(46m#LZQsR20-4@0Hi;Uz%Bg_r3Zd?0GHX+`Pczc$0P`n%(;)MI4!l_Y51eTKE8ANI_UJWX&3L1Z)8&_sy zyY1t;S6REOyTQZMCN65V-AcSak*U9%3#p~QWA&@8%e`y6QR90^(tEw@yHb7oSMr;j z0fxB!yQ~9zz}LvL4m`nYY`PU9OR-{6mEZtB?nadsZ=7bWsk3~Ld^Ht+$j610vpRILCa4#UfKw) zTnC_Y7nEdYD{K)k?E_+Q)qzDq&ii94%y?-^M|QYGAZ<~u4+PL5BM0QTSAk+eehDZT zI#R#Kbq3JB_t=dLlVeR&|+Tb zu`3N}SJ{Jn>C>Qq!e3RK;qW^!gByWf$6Z}V176YuC#Rq%94G{(Xr8?$1ci9jhbFDW z+JR!Pe0?W&ZYOz3H@ck~H+0x&u>a!N2LiZK(=u2+NoZ_7MA`E8Z(1Rm0BVnP5V(xJ|EVq)d7bXR&Y1a6^1 zj3N{Eh6v2NZ{&g1xxknuU{P)o8cYW)?+8Ee#v_g%D3V@sW@mpaD5-}-B$@yu>yu~c z`2`gJ;zFRI=YO8%%SZ8S(rW?T=d~tGasW~X+i%|f`3!;{*DScso+r~jH~7Qt$o@be z#CGJKFGfP|{lQh(-XOPkKSG2E$|e);9yb)fR*|01KgM%p#AJ1|=omh>uZB5mVz1tX zK#)@Ym)ms3pJRVHW-&i-6DMfIb?$xkQKAee7D2ndY!H0vX|Gax|3|2P(pe*4{!Ag{ zf%wZHeoo&>Qs0yr=5S@fdbtyfN_tYLV`&cID81v z&A@Y9AVv_hFr%Od5QBVE2rY+2N(4nY%tJ8bNQK5kKD=N`VG0Hk&JC1T5n`Ey1SvVZ zm64^ffsG&|4wfXL-m)PbTj z{y+=?w4{IAhj0}(k^nJBSAkzGYJIwQ;5%r~Yzid&&YwJJ_58hi*BF|^R|@YDewO&} z-**=nP_9Qt%oF3tQ8!YToI0##q-k4w`*=#_O8n zcfXHcv2?BDou+3Unp4@GN0ocM(HC2D`7KD@di<3}o_lB2`Oi@SZ?MI5fhC?BU0?jZ>F^~g;IUV&73_?;?XQEU+0~Vo^Rp*nV4n;tqqEy|<;h>Hp zWNB8Ib_!~!+Zz)(Iv+#1`g={IwBA-m|#wfSCa?60Ze6q~7CMuPXG=_!`|^(Q7J=bkplPjquMM z@j4!P+LTXwdF5#?uKCiOcdqv5MrW=#SEiqidg`jL&U)*vzYcrsvd>O??Y7^Jd+xgL z&U^2^{|aWj!`|iIFKkeHaBTxPI-;aO(`tQ$w|Nj3EzyJzxfCMa{0S}141S)WW z3^X8D8VJD%N^pYpdmsfbh`|hMaDy4NAO}AP!Vrp(gB~2A2~UW^6#B1(Dr}((_s7B) zB5-~(tf38Wh{GHzFortpAp(DS=))ii(S<-9q7jcsLi!aB5t|6b{ylMtG>oDZ zuZTq*PBDvGR3aC@2*&(vF^ptH;uy<_#vqd}OG z+@l{i*vCH#l7fL8Bp=;)MK~IAk&LuqAse~CM?!Lv1C%5sFUdbk{$hV}lXXNSCsU}& z8Y=RWq%5TaJ!wh^hBB3uTqP?R>B?6cGM2IwAT3om%3A6&mA8ClFL$ZOUjkE(!WzAyb68Or|qoiOOdJaGBBsBQ>ju#cN{IirNHbH#gYKZHhCF-)vC1C6F`VwKXEuN3SxlmmqzJvaG{GH5^pP>eo3WtiZ5qZc1C7#j0SGWPC{di&Q49j4gnD@5u)s#+DR zbtI`tX9`1978RsiRi;-@n$neK^`$XYEB|WR(*Wjlr*XC4J6^HX02Vb5N9`X;kvhO( zxWN?qdxlpV@eKgZ^`mtqsz>>n*s$tvtRdB(G0qCs{K<5!c>t^cH~Utb5^%9Rohn{G zdR4tXwy*hv22g*aI@J0-b*W9oU!R77+uY*wwzpq#PoO42ond5zIsqf;2*n`jLgk=!`|eYgbJ< z1iktjum1!A-U8Z}e|gw$clVcxql5sF_%%vH`}YkP7$txH0+hgCQH+uU&+6~NdGYKp z@B$GD;Pb*p(eVA=Azud2*RqIg?}Fi5KM)^f#Q57U{|by^0lHX!{pGFz18kH5-|rO| zeuxBW%fAFKxWW4S23}$ez;ATM#_>IhApD2px42QrMtL%c@n_=qemKM|zTb@X2V?z; zxPLYNul|3cOcyuuxWho+AC<#GWcwM!C`OQQ{wVyA2;c8dM|nm6_-qtErW9Hv59)jfm@!EgX{`oV7tyOJ5ljzAMQL!I|pY#wn z822m=$$@rft!xH_fZ2xPU$(bhD3QLJ0TGC`Lvr<2u#_7t=&oO_4}xo5U)g@+UiU+0 z{q9>oMBeuscK?2z-vRo2f5tutvg`M4W=kNlnr6TSm>q%4ewf;Tx;CSU5O4wB$l&*b zLY{wdP1yY<+{cm4Zl1Lp=T6?Q-sYZ+{qh}e8qfFNz$W0p^;ht2190T$X821WE^)h3 z+}ihkd7*Hf12glS4*q?|oV)$%2;8*b3zscWwM%ULnLOhN zKKw%x|Bg%ug4wAC;NmeVk8ij{+VXhykD1$Nz6#`VoG)Sk^o3`~G)_27d58H+&utZ(Qw0-KUP9`Gp2tZC!dC2VsBP*SteGuk0Wo^}+5} zx&8Kzd|J1B!qa1ei}+TXm||ymKxCMQ zP~<`9&>0XIhFupaDHmq)&~T5ZdsGKNwP=0zV`N4Ffxaewyx1At#%m)e4+R!Y9v1;_ zv=kB7fx*aJyNG4bC=UQP7AAkkWVYA-irs~cSjCDAP!iPwt|@r(NdaAhZq1_y9)XIE}fc>1G^%t&r5#cQ}2K;s6ESS1qe zRzFK20o}Ea_oHMScOwH?KN$&!9q2{h=!!rn59?Sz?YLsMSabhVkNAIck2?rJ71@u5 z@{b=TkR4e+1-WMjd5jK`2|Ff@bwrKU7>&*-Ky3GiBk2&b$aobrWj*E)zo=jq@r4iB zgcA8*6|s!wbctMHlLADDZD>DV8H$CtKZKZAqXZ2b_Y$pVdX2(hd6J07Wf6HGQop8Q zKd5c=^Nkq>miWVp5nz8&5OD!1{?~K~Np5smK@qTU1$F_D#uO>$29Op6UStf4`3(g& zQ8Zx;nqma+&=-?}d4Q;v4sij3_KkR@W%pwWW5;0lLkaHX5Kz#SiiekMSU;5DlzSp| z3~&o;)^+~#h-qmQrD;D)!BOvVKIF%v=MJnSGf5W zC{}^dDUxy-7Kmv}VHg%}(06VTeD-6W^LRoYbdmhFlU;WszxjK;W(+CNBR^?B#ufo> zA&=9fpFHxD=inn8k&pTVaCNm5hVl*~p>y^#47UK5-x#6tplsnqj$+r4JNcL%8e=Ax zY5;_osLw0R=_je$apy@_7yu>QE@U zXiA!s-KJm{nkNt$qx~qKHM(dmx^g)xq&y0v1o)qB5u|?|IHU!7pdI=@Xj+{9xRg+8 zgu}>H8M>1i*>s&jUIoT^8!Dm>7mpBOWf%E__Q^kU$F2)jTY%Javtd7bgESw<+0 z<&pTKtQn`U7KnnuI<47y6x$j=0CyDK_F$Omp;CYHPanyr_+t=Vw-gX)asc^ZkXmG& zfu5bwr!m{3IhmmOW0y@>fvNV5Pj(B9&!hwp zZcTNOU01dD;}gi*vRw7D!)1FZdp|3?XkSZ{3;?rsC9^(7vqEySY@4%%y0iGxvmZ7S zBbtAd_`_>a_l=4Er?g^ps6W|Xiz%o8Hw^C}3Pj6V8LPFnxtoct(xQ)=bt%VmsV0U*^LLnA# zoLhc3DzDMzY>twM7t63XK`4r}dxdBW-b!R6dsOdhi?hc7Jy3_H;9d;|vKn9tX^5pA z_zIn21h-%c$hMb&`3=yJkhN87`1=j|ySE^#ZBbQKQ7fA4S$w;&S!YYL0D!~fe7;a{&c~cnj2qc^g-D0OzPO#6o5D zYTT8r7h1*8+psx%BO$PYqdPzsX}Yyqw~U5A%2voHL2<`dmNe^^GF!NC$g~BOJJ zggIwXXa1}uinK7gwd@#_Jh_wZR}>PEoe&Y4vmAtF<;ih4v=YD(p}ET`*2^5h$iTd$ z!VJ6iQ_SBOw;7wskG#wu*31l0cF(M`O9_aiT&=wZUA0DJsvN$D%(Uit6mcfbVCX1B zhRgB%i$$c%tqic;{LK%c&boh_m6rsccyR$9OvfCj%S>Ctd2qv&ki*Y-5-t7GVcf&w z*NY|n#)_n1*;u;XM{aIpo?`(G9FcVWsJG2_pQd&r)Z9ON8CQ|AtwRkcSap#{t*S#= zdQ$DEIe}3iD{qDjSX?c)Mwc_8k?)X>0knS3=uG(T|bOJ znucVq4bS5ko?KZ!!;OEsU8{c%>K579bR;DT9%yCmjo^QVpRu{6Y^bRMwB|^DKT2M!qLo=Y{$W)4=v}_zM|+b$-N{i^m)F`w-Ub%h>(er>&)YrE zucm9VEnDD`gpwGg`K-&fp@g<>KM84N{i{Tx02UJa3`c)9;bXBBoSo?UlkAn~{>oCz z-1=h@0IS;}H505p-bEzr4x#E5i0uUa>Kqa4->vJl-i8HU+ZIXdNXYBqi?JI?nqSPF zzL!76zUSzE&9^v^^xfh{5Lvv2=>FsD`qKlDmZ#dD>V0?ZC;p2C4(lS0V&%^6=dR$V ztM0oFmIHqe?+z#LJtm*3XY2(Hqu=1Iy!pJ!uHlN-i*-h?lb(5a!SW_*aNf@FWtHM} zyWIW$@8>3A^+{=Y$BWXL2TC*;(BM!wu4vu(VRo+cc4g;`em~JI^hBRam&FALJ7y9t z^7VZIUElS<25`n`<#YSG43OuE-r`>W_29+2w)}sah1-T-+xB{HjA|~1R{oFmWATmD z{IwU!>O}PtYG1$qGk03Q;v~`;f6wQ{hWE=V>0Fh&^xbf351VE1jlRA2>0W$*zvuKQ z&~keD#7&xlFVOiDhWp&&DDOz9^bNS@&mxt~Y3k}?n5~?BD7{{K$G+)Ponq99`$j?M z$FIy>)%$cFf*!qc)(R1@9PL}*=ESOD{^FgTVij6RNrYpOEmWJ%7M2N zjNNl$c9(7nub0=|HOIFFvtysm{`h~w_-HGSqW%nnuBg1EY7Zv<_#l!`zyD+dtfGqm z%Z#tWGSpD70}FEQDd8TA%pmDTY^k<_8c`^ZTV8o3lh|lF>FEs{${_F6m6EM|pl!D9Es&NC}EpoVxPJ-wwjjx&);fi2(^( zd}Ww~BrxTccWiQxQ3>1f(;_jYdgV2kRP!)2Gr2@l9yJAIb45wxEMra@72=b(MPbrW z&l*|!6wryj5fmjt4JGPMhj4%VrXq!)IOne=O@dOYxSk=b{#sS5YPBF!(^^$BCu`zK zxkl+cbVGF0RadZX1(6^qLC&NpyYYIj&BUU@Up5q%vAAzuSK)hk_5T4cTmI zKIofsE)?VykS~2DR6IG_^2w}h{%gH~VKx}yY{BK~su6YO8RQm?L6+uS!{qebZ@zhF zXs40=7-WiHL7Av+dF%Q}lr~-WZ3Cn9?Ahq@bSXVoCHpTuf%@)N9-~EE%kRV9kq#?L zU}6}giJ&1htXyN`m}P%1Z-p@2%D*nxwg`X*o4?Lkwf-9-yyIqaZkW=(8zy9H4T^8V zLk(PB0|_VV@CXub>l|pFc~+FolQmLun6eBODA%%z(`|WxS9l`BpVA#m4K#?VD~ML@ z)~1N)CUJBIO2>6T*KdN|_BH*=I(F!yN8B&oeQ!Q^MTb|2V|jm|+3AAop`G@zmGI6n za$`XMGIFqvDI_YYBVDP0MmuW#Paeygi1T*yjub#@fN>VNB#1jX(;OFlMjaL8LET$)E(z z0Yo7ZF=2-y*_eOEWf1+TMP=n;*?Mr8H>^#^CAa?i#=E$%xOjDi5|jW_&{oqd=zZj5 zvvcDY5d}feJ*_}!gx(t8akX4!tv&~ejrgpz7Y6XKBXWcWvY_Y_DFP-j5wMm)KGGw+ zB=TLM^B~+z<-ABqg*Y9G&4cm=q5tG7iAWTjV9M~Pv&nzsQ9YmuG~T7kdflo`L;2A3 zWC@W9!K9biQ;RX2B0fQ#j3~wo-$atBl~L-80j4w^K|pChgG7^4)EwV$%9j#VN`;Ps zl*%hD63cUNfeVMiiv%`!FPs=;m&K#sBL9;WMCM^4rl_YmVGv71GS4N76yWdGf#f;Cpy*1P7M+zmo%CF=Q~;xzBUqNm>mITrXb}#A4yPJPMpY( z7=clu5#BSYo8*z0U=rk&MS}<#%{WGylremK zR3jVDAylH~p@slcO>9bnO8L|aDg`;qOQ7MDkc5Bdq(;P1MHsqD27J|;3gPKlK|+&? zIB{gtTAVwxIj$`}uZu77s@~)i#>fl{e^z8bBQOZ5vCsvZG)0R?OL!^<@RW{3Sx;fX zibu}rF_@24CMs_MNFxbSHIL#XK}KoW0``!u4IPkO)A|^e{^diZTO1GL6p{~p>!*ZS zEqQ+=P)VXVuXMwj>}Jn$uD?iBxTWGt{z&C9EV=gSiZJohVjsm)yPBv?`aFn0xDsBL zj8~Q!q$IZ#a@)Q&MtUJr zB$tyiX{mDT+6^@}USc zLM+^i%4T)-5?K@4&<)wulH$?`Q)qt$*usVevH!RzT+bAhsD?%+D}-!iLxb70P1K`B zjo-hNwG6&vv{3T5$x0qBoM`@wpcz1fVf5@$!kea8%qamdZPq0=Ugf@<1{j;LNZ`IQ?e zI;8#vxWKP>^PEE};W#c5!%_asO3=0?;!@(###l;aK5AksK~+LFENZ zk!oh8HRf7bdeh~RNtV|dD`Vbrl@!wR7ZM!j&^Y+ekv=}7AB*U%ggC6so=GD>UFr<4 zI3=v^ad~uH6jK9P$wo~eke`1;a3@?hZB##nUMVpx=pHbj|B;_}VeQ?=`C(v+ZFkWG1zz#O-qc zXw(RdJjY3EChy~r>EEQipF84HVp$~bkTv^?vDRLlbdC%LB?4ci(&v9*4Y=_KB?nC7 z`=@R)J?q~j?`r*xIfFv}US~5l`1_BL_=ppl78;q2{KE})GYI-ixdI^ePIt%6~!kb7cGSR=bPz@C9ktSq7$hkY(+P~wuyR>PG z!?}(0$RZG=DA(A*iTE5k*|!a|r=ZwDB-Dy7bdB^VD|j(MRJsi*ybTtNLA)!B7Ob0M z^O73OALOyMzsNyWvlcnL!4>0{9!v>S^0b>U0*q)wf`G)7kVJpXT0uILiY+`H%<+mf z^sh{#3qa8NuY?3j0fm>3Kzzz&|`xD%XI%fa#EDNI!)LfmiU85ut?a z3xb;4lyk^IE;ug<7^@>OC1XfGzltu1palENouoiUu2>GXLl7{Nr95cFARwf*!=hz0 zpsVOcw}LWWtc!nNR6eh$GA>D~Oc+03oQOt{y7$_~qmii|qQ~8E3Fes)e%!^B@I_#J zhaX9*H{*(PSTz~D$CStc5d%mG+_kh&NUezya6}nlEXaR^h=4ps^C3m^n7~9M2!7(R zI{QOYgu?$wDp8=v>|!E*TqP`|$+s$qSHeiC0FB$ACQ5(6M~l!$2H8l%Fv(;yh&VB_ z`anv7TEH@33I5XBpme&4sH`BF^b40f4w%eJjmn^$Jd&I22(T1Go+JvNbPZF2hGBUM zLC`coFiN6`GYebFty4&Zya_1CuqJYhv>d{gSU?z<$+qJaBV>t~EJ(s6i?Y127xatA z3QKv|Nlkw{s{dHaw#16J1dO;$i6CIeL3F|i!YOWX$}(KX3vipMEQnh`&45fyl&Hr| zBnrxWG01|6yc`siNv3x)Mc+I{G3m_EC7R(oXJ# zGV~-2Xh{Atso2l^gd6`PNp-PLu-H!e+>ijBy$3qa-ozsQ91Qu)Pd*fg_WVtB0U^Oi zPz&AA?lgu5^^5nk3<@O-Br}Ez(WrIe4xN^Q4amhRAW)X2+>lMPx;{> z9sPgM!psmEol#S)QSiLc`ShR?J<$sj&|rB`q6pGz>M0#PP$8|-egm5Qe1`AyPbyV1 zCvBWBbr4$8#gkhBjmhdDH^>1vjZ-q=rw|Z@XSjvKxh_04vDS0Rm8NUhd(ZG;?{z6;n)A$0?QlUIOMS9%TDP8wK(Jy;Ji zScF~J54u){eb^3CScn}iiM3LDb=Zl`SdA@MjNRC`saTH<*&h{IlGPB9E!m7cS(M!n zmF3WjU0IloS$64InVs31z1b(ES)6~}*p}VdckNl9-7lS$)Rzrfqa|6Oh1sM%T60}m z{-*s_r+wOIjoPE7TG=aFsmLMO(HVSF%mn zvu#^}ja#`5*SD=(NxfT{)!Uw0TfF^SW93_h6}_FyTl zVl2*L3cg}3?&9c6VlS3pE)HXVGA?5?7U3~IVK^|m724XxuWJQMILSAG?-rz=lWEPHOGNxonzGO`9U`x*A zEly-k{$xo0_GC~VfwTWP-RodfeX-pVG!X)fP&EoV~Bp_D;|bM z$b+4!hkt6r!?XmEp#K2v!x1wqJzg2_eY1vHX2 z=}h~V{wHE-MZ?9V7HUzhhgS${LAV86lxnKB>XcsTmd@#YMvbF?KIyGyX_tm+nFh3) zHd9i?=^EPUp1#hX4(g#sje59+tgh(mbZdZlBDwD8r*7)8&I7}PKCCurt*)U=IO|;o z>yj30)KF=%_G;8%Ynry{wN`AM=4n!mYoUH?)2QpaK2yC`YEkHGQl$jTK8?bz>Z_h= z!v?g(mTYQ1jj(=y1(arNq&|m#d56-5>Rt|M83yhe$^(za;d;nE4Zv$sZbDHI;g4_w z4i*I{C-@mjdfS2#?nU-4javwCgQjrQa2*)X?HbbW(}2}R*eU?O2?3{}*a+{%E{*I~3AcbV~0^uZHmhpKytjK{LJZ%4i z0bbN9)UXNf<|>qU1yTic8X6Ugi1U9*$-0<8C-R5mP9+-pD3Hj$!~Vb!m_Qg%bbtQ9 zbWO)=JC6W7H$EB)^-azx6XE_kTI|UXM*j4|dHUcG*t%KSB0)Uv^F}jd}nLLN|?WCv}~+b`rvN)OeI> z+V*~bSD@0M^n68$`X&wIfPwi|zS{6iaXZy|(r}(2`1*z5de9Je)3Cm}pKX6w`;?z|Zg+~7_lL6&{FUB+ zd4P#@g~0h5%KL)n4}7904ZeDkMp;XP=^p1oTraiKth62Cn>$Iv0P zi4-|%^r%pvv5q4BVSJYGWXgyqU*=%oMr6v6FCE@{xiE;#i6o)f6zT9;y@`Stq{!*9 zrB0j@f9f=5?Bvj(6Km@9=Tu=mjzzPceD|#6SFmQ!qCG2h;kye+%lhLPAt5Dy1`IMu zJj%e7!eMs*_B>ez@GG-^0S9I)0dZ2Z#_X1n*p>kdr5?F3h+tAAl+2ssdiM9sE*FW9 z8AKL2nnCG>rz6+Xz(7OU$&Ee6o?Y8*5)l1HPU0wPd%IPbZle;-FQ3#f&Zco19vq!QZM$n})0~ri7q8CBR*nn>JhC=>3TT z-ATly$J=q4CFh<(&Gl7XbRrG*S7Bg1_t%5ho#&W*{ZV!gc;b;47IuGWS0HK%6N`aGEpXO{RM@YFCl8zp#EesA_E~zcxeg@40Fq3 zbAiRvgK8r7szN;F@rto|`qOBO#PVvFuUswL?6c3f73-a}PK2gHsb)8fuErv(U%AdA zsF1graf=o-JUQw{PW6lzfxJ7}l`NGDdqm)5{l1%TkP!s6WMIaBq>zKgc3mjZE1A6_ zo5C+)tZ_`%rPqVWOqNKpT?ZpvnMgP;I?-e@FN9-ZlQJ;!c#Y{59nU`7cv{DUK|9%? zMo)xvLK9caWXDx<2(dqtLJPH$QY%#Q#wgbcvdE5Q{B_6v@2IRX%l{4d@sWQ>`3?`mM05^%6tu5P@GHaLgCBl>yH+k}>;Mt7Gg;QbmD z0Yn)~KJQpobt2#C#0Erd?h(SY}Crm1{Is5-iv z45GHD6KSP?34(Fj5x^3-x8WfMeg2RkVrXch9-_;HFf$GL{1HG2Ew6+-bW0DvLBIqa zP%R~tnQ!pZyC$(sd{T1Q*d#(HXz`>(xe3oi>fx5kJ?tKC85vmY2)5U?tZnR5$nXN` zye8>nG96n@lCd)5aL)OOz5J zWgYJ*l05G5Nq!97wQ$$ELHeVR9}pce`dl{0h+NCC4y@P^M|Kg!R3ns{n9h_>$}!RWrdq1htcC%3_(ojeG!;`M zs;I^|S!;Ask6YYgcuaLFjv~;Ab9}>F@?u~F+>iLC7)DGq zaEn(Ye+01%XwxMb%h$|!_8d{AgkLu_K8g57PVYEJqs&;@%Vzd3n+;cIKYP`80JXGP z$qfUbFw&Xr1vHJobE($)L4mh zCIXxiiEagJGgsQlyAgvH&r%a$t7uNP9~>wre`9Nx1tU;!<^9imEmy~hjJGHgep@_w zRJrxGHsNDwpL&5F3 zCe}K_&4yJ`yi6i*hd7L0bWQ}s?tK!8f!SgD{z@dphpCwi-wVk9z*xq1p>g2W6Q6bp zfBDRXsiuP&0Aa1tuE-57rG7BH-yJleqjI^$TmYxZZSqL- z71+TR)-F?dG+{+y*dQmiw(xleq7`kYmr7)K8nDbe{jn^~ysjtTU{Ve&)8C|Y*CrNw zkP-g)K-c|FTqt3+tNk-nh&_g4X=0Lqe@m65Ul%Wu3Nf}Y10!luyE0G^URGmdNIx9q zX;}jPky(z+K?wWC2z(8~Q|0UI*+4rm0D~CwCS|>aXxlyr3KY3{ABw*#0F4_Ra(cr&#E;xa%ssN>Y_;`DCzVe{;$s zhO(6dd6xEUa>%U7%3(&9-VFl>TJ^|MNMOz!1d)`%D0zlSavtGnlHB8I%=kq%ZlbBp zx8n*esEND`Ra*;PN+_QS#Btmy-({u397If)X8w$tH#z3tkk!E}rJ`@}Jedv@C?jj> z&?;)&E*uX!Xg)r*kc${CO8>Ide}||QVaXa!wB-9lwhoxC(|kBB_Bos~fT+#>G63#s^X)68kitJUq{f2fy-j9b{@ zOh2s2J374gJ>VweByHY6%1PRUSOV14;z`>7VQ|%k+K5N#xfCzCu?O&tJeF>8SUf1wNv@<2@Z+<)ky0&QU1pjj3E zoJd6B+8yG-A^>|_o$8e)V1dKg2vXda~41?tJuUVNhtHsmQbB;!pX6RKHG zb{`e-S51-zPSTxDMr8#aoKpPc%^BrT4qj1i6;djtk_gdX03%f{1j=PXQ9J_@Enp;y zonEk4>anESf2mwy99S(bV#0YPYk(!#h-Dg*rB9{{TDIj_;vsqjpA3!+?tt9ghq~Ij{fv!octy>YK8m=$W?!Q=5St!cLD}Unw#~V#B(^MZf*zMWrCxO%zPys$@zvLG|MgENEr-6LZB3= zsHGa_f7lFJ1{1*m5uTC!QKpA2=v3aPD=w&Xnxy3M4c%}kNQR_GWN4gh=#_P-b$n=u zj%bONR1>ABxs}L?qMJ%l1`(b^uyDaGFp1MCqd!ClyRg}c;AlJY(qJlHLbcp#1c5O? z)-ZU-Fu+~L4JrP}@Z~1XXp%zKlA0)s&L|j0e<`1`m|eVRYhtOECRAlUj@)pFMxCj` z=%{HvYR{-Co8E$(QU;uAp+w9n>6k@(Jjh}>iXdo`16gRPo=Fu6LI!1CmpamWo|tUF zrwMIHsv-t921*W4f-&TuE1d>}UL81QMp(VVu)ad2-U3Fn7#!Ot zf3hZ2c>axm37vT^o+|>;t>)^k24v6;B;CD3xgLffyjHX}iwRoULIx&&(p}btYs7tP zeEH|!+{U@4E3vLCfr=|gnx3Lc#yix*j;dXiX^*fH>$HYe%g95ya>!pOsHsxTCt3(- z)v1ZhC&J49Dsf81$36rOuH}`1tWR$#)c%wo@~m#ggn?_MA#;mO6C0>M1C-yV8U!Y zWFHkCs){Al@dc=p{+7jZ48{^3j|i=G5bcH>EfMBt)m919E(FTXk4`}COo@(ZfBr04 znh{n;MT^y}pdBa9ddAoS#;BET&z^0&gg~gYQir_*4HiX2QS9Y9h6zGLCU8Ne$cu0) zUAdMkxti--?d!0vtCDaolVIcUrQvDVAAkT3j#SY%P(oo`CzPdQBDqC{!dvu7+Ev_c z;|b5Eq5VHGt z{+)0og@^*5nJES-1|IN9xNz|_@X&TF1k)&#!e-M5>i)e07nH%ssGbF9Y_0w66OyXz zG5`pB$33jZ2)4&BtuTI(2jc#Pgch){An>vgE}Y#k+vc#|fp3@U1O)@J5EF6j@vuZ7 zG5*%#Xrpco{4Nt4KOB>wfAAG|tQM2sL3HtI^b!w=1{9P5pvgueVs0Xn%;iu5C7^9R zG)kTXiS>ok^uh(Fp@t<(hV^1^Gp=tVKQas(TJb;_&qPPTz)Yabgu(e?0OoFD>@Ixq z3WVW^xUn*NxH3yxtHl(X+!3$NX{{of8Lu&*%#td;l?|9Y{0`9m~os=!TW^Y9w?)KI_?=N(tq z2*adr{G}qt@TJHGA8w&!LSX)h zN(@_%h^OwH+Gd;veRK`hqOY-mYMa5GZRQ|05mKL^h%4YI=Y37%5p-R+(J9z zjL1lY`K|kn}g3G)%^(MC3#)_p?h6FiOWXLGSQQ z+w`rCY))esL-TYz;htxfh+YWwFhxXB%Q3zY?nyt~Q?FoDf79YrGeqTODpubbAvP)S z3AAr4B|j53fk@v41FLr#VZI>q>*Y#gV_>c6MU}_HLu+OdLk2

      UzVzi$av;(yNrfczzZ`$T|+V}J=w|+L%eju2Gzp)QI*n>UD5J09j zESMZ7sC_A+9`+G_xiWrIwJ8377|3oK*jgOeNgk+!R#+PLpblENg{K8*_*ob$ExEC>L4Sgcn2EIL7yGafBDfhONQ4f;Af#9yiCAfi_#2J*8;`gfk@y>v zxEqyN8<$v%(C8Prs2RIx9HN*Uhj=i-NN&Tp8^yRA$A}xrxEsre8_g&i&lrvBs208{1mzDRIk{FnOnf^DJ09KfXS(*hSm1jbg-qD!2@tB6uHqn9tm8l$e35j`0jC-ky zoau(1sf?g0hoT8@qzR4CusDi=Ito!aNNE?vse8wPnwX-R+rgT(;hKU0n<6ut34xg_ zc#po9k4>kPu*j6g=alOx4~_6WhVc^fND%Un2~cngsVEnJ?~tB&_8Dxc7;ed!@o0bT zi5l1`o0eG++=+(XNq*qTCBI3D!MS7LVV!;vpxQYdwTX5aKLc}8h|%lVn&DVXG0faa-= z5sIP$Av#5W7i0b)HxWq65eVf5(Pl=suxLB>6CCQH*}-=T*9dMPPwB%}15poJS`a%{ z5mo9BC3>P|L@;M+5Mo+=qJt1+sxxD{UMhMpELt5d>KZVr7cx4a1W}-hc%V%vFFop; zKN^}r+JQqFjYTRD-(U)Y06?TD55v$103-@+l%xuOC#jiQoRkn4@9+x71$NMdQO0FI zuW+FU20*KtTZMHFrjQ95!4sG&2=eJ+a%N!2_CGu&2q95RlZvZFkrb0k37L>h32_g` zKsEqm3oI_r*`rFr?i=T)>RjSO1m(j8jeuLqbPklDKwU8mWl%m1G&>2! zL>sF!!@yjF1xA_RL|o87)K#BHBtNY5bv9dn1`hi@2-O4SlLS2^p9tkWP|#B%v1Vfz z1VjL4k43R0O9}U&vSTN$F+olgd$Q}AHi{XF{rQ0ZSsL`Zoi!?sH~Nq{%7#1YmxoG@ z3(AZQN{0`ciA8!3rqE1{@Cpe91;YRh+w?v60|TmN5Y83^b7Tsl5Lw3HQ3YhQjT>@* zwuDBn{s0Zk6h-f_O-V4glUtv~1)oW9KH`N&td>eo@l(L+oLfq0p*pIRAh{Yv3Akni zK>(~g6a)rj5Qv)tCn~uXx>FcM31z^zJGOeS& zWBzR=KhFSHh2{@}aJ4E+Xq!_85PM2ym3zK(r$Lja^q00LxVEAJzXu_x!bqr2wwsXH zn|Ld~db_Ap*r?Dbz}f_{9G6gy09{-V6$O>K4Y$1bHa0Mj0!a|O2cb(zP*^YpRv0lz zf{dQB1i^Sr1Rl#h#*|BaB*+DSaSsLcQq0Pw95iPh)yL6l$pi5UR=agang>{vJFz50 zqq;sEw?MKXK}A`FN?rO5 znUDlTu)+heML?wz5tpVDI?zHyz+vPD5zW!geqGYm<88w+ygG22hTvx7uC+z;ZbOmYl4+P7j?22R6ebS!wB&|g5_lzv^`{m z$yhY1yVL`cYy_`h3epN(ttMjQqd;76wf9h0Pc%(51r511O^U@_4(_~oT znh}|k`7~~StH#dk*`m?mpYh?F5#paQ;+s+8nQ`Lp*PQY_46Ao>lEcBmwNod1aOcqK zt*#OZR&mfOJ4>Npv%~7Ho-E;Y3*Ie78Q~R=LvdW8JDejPk+bWiZVWkt5#-xAx&G?9 zLvcg_4YPU^ysi-f!+fuh=OOj%t^PjL4Hdd$oVy->6P7CNPmxU+G5+Ocv_C&FI?R_o zIc235o2k?u#>R76?M+nxW~N!Reae>6iiP?^kkr zUZ?eN!pO=)L{L)e;|3n@@uhU60e2NZUJu3~2p|9P0VML(MW;)eI}+?0Vp^xUgDj?y z@*bam2d0OnV=6ky6>%yXt~Q^$J(faUKWr0xi%Rho+cQeOFOeY(zG0{?A+tl_Ob!mq9NMym*M%l z;q|N9#>m>d>Dprn+ZQR@&Pdx&IdNaq2$2bvz8D4?>16LWs_b%O9_0Uy=lW+6KQF2(K9mpBW6V84jNrZ$BA- zaSx$1FCuYT7GhfaB|-UmVfpJj_M`ak+{XFT?D?gU{h6`-n9=>2@%@+){s7@m;6Q=} z4IV_8P@zG2ri|5F7*XOxiWMzh#F$azMvfglegqj(Y3$WG zm^C{NyYNh3Px`Ib4WokHO)V_*C zUd`+HBiF!^KZYGP*0M*lO*elA9a^+dxF|PyYP*_SBg)ZC(2J>e;b#-;O))xC@WF@dU)`yYsk9FT3^JgD^ti?7_wt z3opdb!d&9YZa?n&v+lnRWfQP}DE11>?!fHI%WlE!{u;FIK?!H1Q8fFo!7)c2al{Y6 zmf8bxIuZ3FaYYlqL(w`Ftz*$T7p;SFIvH!EvP#3?!!k=O@8eLvAK42Mx*?|v(8$89 zOLDp;r)zS$C#Q=Nx+$%svrfmJ>+nndX8LYS=*ndBNCKD2O--rTYZJPEH=&EOxjF5G zG*Y(k^zuyW!fejZ<^W|=(CpHLZcXSCZEn%#8fETLNmpg{DN4!I^Fuzl)08<*NA(m< zLd83jyadhbiak`9Q?*rNkA>;gKV!`fQ|4-2?$+i8{B&0XdqprZH-kk_EcF~J6j^b{ z4N2KenZ0yY|8inaTIQ&K)itqe8)Nd<7P)26TmA>ZCAVLHJ2KZ+bfcCP9k zdOxe@Yn^*O&ckNa4Uyf}$YBEo6i#S?4SJX^GEhrfwl+m(#cU}?XZo|Ggr*XN+9?tA^wG~rs)!05E^tk6vsOjOJW-e;wre3*k=1gU7 zSmlZ(ye4@v2_h4JdD!yo89@a32BC8VF+%>7iY%|{n>-q!*)7b0)0|+mh4lP1&_$P< z?&hb>mEpXV+`C@R{oel6z#SdD_MqOOBnL`%PwSprG6~)v-}9*7Amw{7UZQ$lsax`v zSH878uQ$>A*z`IlHqcxTZrLkc0C6R`)P2uT42;?MUiUYDMx8HJ=~LfM@;~SJqnK70E4SAdc8hv@k4G(e;W4HkdKT+H*UNMz=Kw~M!c*pR9(7$i! z&>!+(p)s;zj52V6ehhpE47sLSu1cJV-;-HbyCGM}}?mM-Jha z!@PWBh6Fi(VN3F75d-Q5fk0DWrm8po8V8O}d*taJ-Vg^q3{p*l%Yof;WOp3eg)k)} zbe}x#F$_Ueu?&KEh6yR*1}?ZE4|(X{LF!?QPI7?@jo?ZiC=tAGGy;?d8D0-y0ENgw zBNLRM$yG>M%3DA~5KHujG4gOrpLkLbH*7=>c6m5|gm6+65c$UMXmX1@R51_35Qs57 znF*R0^LJc$=0V=k2>RKt5uOyq?tE88H;CdM4atKs#=^~OuCklN+#NLD$%QgdqLB&d zqiz1koS+S7Xo@K$1cfI-?on_#syhzroaQ=6a?pHP+t=1C$rDUcq8`sELl2p2YEoAK%U`-{u~&Arl1r>5`sVpUk0HJp)^P?cj*Qhasdi7#9a;u zT86G1QxrSxK^|Tz3Mlv#3O#5LG}K9wfmHZZPRN@{ogT`lAso%^_TA+JiDY zB&LRSLn97~kc7@=q04C~a2{%yh>~ui1YB7np=QBQg^x`fRZb+8Gs)ppa-<}gk+)FL$O8;ukb_2qOFM2L zf?1R4N)`se2!#2t!aCF=o9{0E-n517#O&Z7cp;D?yRk9m$3grVI#1uqG zK@u!_j_%%KEa2tBGj5RsDC}`~Qig$s1@ej{z~HV>fPz-noP;)?70&rhLMVq(25CYe zv7%_VQ|f?x7H>CE9sb7mo7!OCrz@ag4NVP#!=h%b90->!(^5Y-}+#kcsL6P(7685E)$%1fl zL8v%o_gUG5&3-nQjX{Jj;}OKy?SVo4F$@ng?-|d4#y!r#W`(>NzxQ~5MPreA6?St0 z8n2-CE1s@q0M{G-A@@dVSb|IhaK8K%a7}_#AYG3JhghIF=AQqn??dCSK&t{XBzyqo4!9-?5h zD3VCS;mx!Wg{>?uQ>yxZEqi^uE|rHFvpH1w_BV?>)oL7HI`!OX*B>!O^13U1Lu1@f z6hwc9F^n+O^Dd5uhK2ZopK%Xpgb}y44?sFIVllbfA?l+!$rV4t3&qMj-P zF>?dP$%G`x{sHJBf~?XmXyB>1DkWoxDJZywFaxQAXfSWfC#HLOy_Ozb=85jw3w&3mN}wAT|h{2XZaN zW55I?z!-bHmx(-xfw&vgRok1CGL*jl;7G0E`X+j4e9s4Fl8-1k?=$%&kO}q($tlMs&oE$f2q5qBqK) zFER#cn=qDtc!h>^NF=I7V~|J)YofmRhNlxEhiu54C?bi3!WU|a6_Us>k~$|k2Q`C; z`D-CA>PU&yNQX49k+h+2kPn1(AeMH1R z?2Z1*vyIFP$dnL;$LgjdYl&qHMRamYf7?q~M5>+mAH594SsIB*{7Y5bBfmh$(NM?U zU`N<+N4uy8daz7-Fg_C^%hL%&j~hg(DZ~s?5PsY(f7~&EoXeVcyHJe96|$w@fz8Pa zCrW^SP1xi`o-mx+{GB0+iPx-6V7in1!%8r*L!sfyXxU7W>rCK4%h^!N*6@^X~)lV0lF&Nz_8SRbAdrQ-#HPqD6DFqE4-3%XX4ImW_AyrU+ zBK1!rosIo$js9$n|7_9#wLB=rJSnBpHcg9t(2_W{58+&v{AAGL+)8siATKRbFog{< zRShy#4KvjZC*4prjY~Fl(?zWc3dxX1^@R<&$~w);J~>W=;nP9&)7Ajg(-73tAXM8h z)EY(94pme}Mb)FYksV=GIz1OVEm1swRjo@cn@k1OO;wFf9gR;NjZmG9QJoDN{%s8$ zP1Rzp2|3jgR}~mo{ZUyRHCoLaTkTa`Jq=wI4PF%uUxf`|g$-d<4PrIca8*@h?HFdw zQfIw5XoVhW%~on14QmYzYz+-*QT5tG z1=|l2Tfep3m<8O-5nL4G+O^eF!w8#q5Z%!w-O^1W%iJA!=oGk3*u_nM%f`*e$NkmF zCDqAAxyrrV-2GcQq*UVU+`|3bj3b0BAl~9N-s3%9LP!R?QCwE5t>A@D+1*y!4OYD! zR=(9;s^C6|G#2@Ii_C?d%?&lel}y9UxVCYd^i|*WU0)^;%hhe(@U&3q^-t-oSL)4M z+kHUXWmNLxG`si)Q)oMXbsEONj1-m-1XH2}1Lh;}Js|O|IP#@W^F{tVJ>0_!#^4M- z0rzEIBynBmgx$5Y--pHDd$rzg#onB0BKcJa5ez=H@FJpw#8+z1=x~FglL&bL$@aK~ zGDrd_NCLxRV2WT`!*JTrfLhCtTCKPq_H|ZF7A^=*Y6MdVM)#rLA{LAz28|?^j3us$CVt{4j^Yj; zlqx2~D<0G=9@;H`UfC|5*e^zw0H%aVmd59cxtS<1QD}rlc!m1@5kFlPgLbf^hRNsWK~n z12>tf!a{3oA7-z62u6NehzZPtj`oL-){BtdjFDDEbCam}N1`gCbHmmw2vea)WF; zq0k#W;?84&n7~8(r4&>$uqwYKxIg`gCL+j#YtFhn(BU%JC3eaLBs%9?poF}-CX~9V zVC;c^Okl;nLnUaS1Q>8`O29)pYbjBfZ>jUJS9qdma{g}vvoxN90kTqVpok^{2gMYS zDnSr4eiJA`5QU?AhI+{GS3)bltHti3goRLwK0EIQ=G|AVR9d0nceLQvPU+X4;@IXD z+Rj$n1`XT>4c*=h-i8$4=E1x`t{^aq&~3keyu#r z5Yt8lgKy6+&#uDl;eur6`1fGPqC# ze=8FPbph9mqttH^^CAn+k32sU?#{~<7V!?!Ez zz?cgjm})mDAnwrabC=j8C~!ej9wCQ+v$IfQHaSmp@iQ!-V7e@S25`f0r3MdE=mA04 z9Z3^iuBNGQUo3AHh={zhyeI5+|NUSjs-jgCq!Sg(yEN!}ZZd zb%j#(bsYHt+Ynj&p7V z=mrWrk2^V*V}B?=FRXZh8}AlJ2)g@*&yu3v_IzQp7x5y-j2bs`?0D{$v5q1~ zk}QeQ*sF6USF&sgaoDYQE@#s8M~=;j6DW4_?CEpo2`0E|5-s}h+pCa9molw4Y*;;} zQi27f29(B+MZtCWt`UqNf?kPw z_3ajvB(E~{AR_AO!_7e&Jv;X*NFI58VQ|UB+SoZaMuv$>GVmF+GBDhje_{@GnBzvs zMKxj#B?zU6&@xfF)rbg@XYca5Y4jOJaAW>N+>G_iW$d#gP{s;M(#T&urt~04RE_-g zOdc?lAxeTla3zQl{~a`p5h*x`ktjjr;07ff-Xc#)6IC=9iT-?~REZ>_)K!Xc!Gv3j zF5Q&VPc`-w)KD>!g%pZ9f0pGGR6eRz6^uf9wH1p-W);?0N`7_LFljl7Raw zgD6ryq)-q+aJgC@L^LALIeC=XBSMiTnujsBU?!86IVj4-ZFxXpf6P0XB;n{e!*CIV z5lQ&Rmmoa^;l`O<5-}^K2{qc~rv8ZjvL>sy3L=j&Ni<|k8D-!H=UDEA7oek2Q9^_u zTdipX6hwA~YrpwJn-tMB{W}wp0~3X$f59{{nJ`UI8p^OqRSFF8yITn&NOxU9Lx~`iKr`OOqL?B`Cg;3^ z(krh>O5i)Dq~y;tq9}FFT{5MbvM7GN^3FTw`0VpLAU>1H#f{t&mM8`JGoC2w{roeC z#&DboG+RX(=9w2$^42X}`SZ>zl>pEG09C@EJ$ma z3}a%=CZ=q&$TArtvq9r2WsKhS{@mSK$IuMkqda*vB2zt215IoujcBgc3Ok!}S;ny}->IWPem>2EU8F#eVsloIk?tYnVGpa`*P=%k{oDdT!x*3-V&V`5 zrt5RcyU6H1coEZy@FA%?ojfLyg%z@}g)Uqn8&*gNe`}NwA-1cbLvn{hgz#>M2l<^3 zVJ4O!ehV!`lo$~QMhPxVrv6LS@`xZN!-Z2d;v7qa6BfBJxsJS|FjXmZY$un*t%26JTgAw`0FkV4QkT`EHCtIZ` zHF>3b3}b6ixksB|`3y9gM2o>f22^7nLBi*A-zUKY5I_wJGACE^Qq4y zh7+A2xn?eJo^^ z;fq|1D6T!N;Y~kiSLNZAp?X~=Uz7M(f8_;Md4v66VU>4S#j>`wy;KJ_Vk_I(HWsep zlq)&udPfZ=v9l-s{p?6XOT^I{ue8QHt@2W9yw$q4xz5dqJ)$e!=|=ahmA&m{OPE>Z zZPs|c6)JFXsMQY=cX-Af9&(MR+~G0zx$b@MMx0tz-0pC@#_R6zzT4IC`c$}zf3ylE z&kMZt4sX4{YcG5gtl)#l_pbECp?!ykU*PE%u>39Ic>N0CWXdy$1O}dgdxzi!li0lt z=InzptJ`7nc8;P@=T)=&!3-O(yucH1?>^kS5bJKlCH`@1Pwd?kYj?%wZ1FE&Y)~01 zkH*8Zaqn=fJ00te$G7t_kh6SNeyFRt zq6axLt!a$-*}Q;mvY_8dXhs@3nuvy^5haN{N3&Y7knXdjjYsL*S^7%!e^4P4a-D@q za9YVB{j~AW+&eYjj?J`lGwl?O!d17r*=2q8?Z#POBMTJCp`!;ITA&l1u%Xv5&MT9H z9cqP+`kAG6=&7ff>S{83!SDD+kqq|kcxOd8=pqYW`udQ0GYs0alQx8@-BD{(4BOhK zQMWOa zO*%%GRUX-skdGiGrElb+5#L5gBcOnFjcENOgkiZsU{eN|*MydYf0@`hAVhMGydw`u z@Rq^+tvZ0KSmFamyTO8v0&aVI$qf(Ea3Rjnb5H#1E48>pF%Dm=V(E-6m>waGAah1wZIrtBa8g%U_y#XEeA^K_t}oKj-^e~2M##c;8ZiUzp%UY(POC?2Q1r$7Ga2aj=}miHd%>0pfrWeh7ja zLc})+LJq({e-h9NG<4lq&`cTVN4K1Yf51#0l))`j&JyVdro`PHCV`otkukVI>U~50 zBb0$4ltHUJk(T6PwuqfdoI@rw#$+@?K}4b(+=BGv%*sK8#=V{OxB(P25fP9f9VX&r zJO~k#fpQp2Wt;{R2?`~E$S_!99&8Dy6iyUL(Siiue=xwJ5rB$JJVP>y-SS05@sZB) zeOb?G*3U5v6A~Rn7+sVh-SjyeP$}L?F&dVgDKo$M>@vZMG9!-L0&+{ zGZf`bcI4UAB%Y8=HA2KShE6u-**0=lHzv$Ce>x$x)I(`*9KHk-+nw2b= z)%iy~P=bQg$Eqw~QD%bMaO6L-q8&m(1!a>#hyoEfBDT=tq2xgjyh2(?h8sYKVdTL% z1Yd!Op`_fS0Dea%Sj&DSM-QOSMVRD2M#r{%LtX$Sb|~Pxm8M`AWYA!y5mZKS!b!9Q zf2D8=C2Jzfazw;D3c_hdg*;5=&g5fce#1F91s4>jKfFQ?7$j1}qT`?k`uL{U;)aCOT9$q5ef32l?^`&6_rBS&fQo*Bp73Sm! z0t{?X3sTD~7=d?UWv}d8Mixm6-l6V=f1E??;b#`kL;568f*lda3Kx)}e1L{%jEZcc zVr`yeblAq%#0pTV-B`$jdY)oza2(8(K@X7JD?kAZu*pT#oIyrP4s21}8379N(5-N$ zJa{I7+(TAI=FrF?msG|p^uTE9Nn?c4dNxTTtPyH-PXDgDRIoO9WS%Hze--(~qc-W29>i}L0gO$l?j=yF{$nIMqBC+SL#PC@ zKtWjOh0>HMDLQB#vMN(nP9_xPK)k6{tgDbJ~&TnebX@MqC2T#Oyw@D1Bub`^n| zmVt&9f{vAfJ{NXb3~7>(Y28s6M1;fAl~mI7Idg z%9lRFtt{Zjl%H+}DMrL%|7(&5MYDS#wO@#LTfLR`?P*%}SZjnMD zuHopRbV2vKGC*{>8Si_lIEkCd<^|jkJQ}CE95JK(dSdy=k4I< z>FB4D&}%r_E4bmSXzeRtfA#Be{VQPwtYHnTb9te9LcuFQLv#G3Zl(Yjxq(PVCPdJN zK^ns|OrUP6Mw$Aic>HYng=*Y^hFA>tvqf$-*c zEZ{95t8Q+{#~H|d{7DeBXn|uaokOaFjjJpmG*^-Xg{u)WpA)H;_>wkJ}^|>u+ z!7X~pEo#xNYT0dbG3c)bL1@sdMh>a0v`S7sZf($s8%|4|G_0@gTRs{K;y$E5sLD#B zVHI6u8x>@=XbBO}e~Bf6oPj9t=u(TUwk${9DvPd4o!rBa&h8%?fn$0G5u{$H5N-}- z$fJzqO496NEbITgLQ#eWsMO)zNvp$V!mS`DPnv>ilwxCmoT(_vcMOQ)h)=SV!QO^l zvRts(c5lLb?-ACigxx9g<>}xxVShd$+a{j8Mw$B#82l!efBY^N{km4tu7+k@BAxIC z^_U0AtPJJU10VCTL8KfSTH-TMLb)zYBjiEX_HiGlmHss9;>$I{Ykpxd(#dU%T$q?b zdz9bq!busBA|dMH*(kCfOQR`33ek+k8iqv5T~e})VN`(4Bol_V=7F1jjVW~9c8YTF z8AAr8)h1hFf7w)qDEl$kxIrMt#K@2#f^;o3R3b35s^toTDagb(WX>m+9we6mwS*=m zpy8@cpeY10uX-q)g6({XE$xJF=!~zB)PpCm0X)MquZ{6tW>sF27yA}h8t2#=t5+Md z*Biqz1bxjnR1I9*11ce|su*M_iGo8vG=M=MyqO4$e@IW8JoH1~0vmn9Hy~`ty<7UF z&;5MMJ4i_fR?QD`G)iE!L$}RHhY4;-PyF%BNu&NtO&`SDn1;)-w8-pINgu@gb@bN| z3s1ko%<+YGriJ>LpGk8xh;Z~PmxlTzkmDfkkhbtaAavDiCxGxp`mF>O8;lnBjyh8e zI}3?Cf6#+kul4z^ZThzD8L!zsPgy?~*gqdwK>t`932;%G$?C=;VB-n&K|~}5b}Z6L zS>%jidtzDO?qatoIXlETgq>l_-c;bO02A$(VntXF%vj$i7l)Y_bC%kA7CN6!pq|d4 zhEAbU%%M_DqB6{)I+s&SG8nxD-*$*@iw(S1e~KvVc88SqR3Hv;mq%nvh5HnDCNLQ% z6*u8bg*?218*s86XGLfK%V&>mXz$r*dzWzjM_(BSXn@Uio7FXl-L;72HINC1K|7n1 zfOHIc#dY&bcBk`pS6X-H*LUyNX_HQBlg?^WOlw2TYeP(IC(LZSw}KDPeAi2Tt4>)% zf6Q5X?|xfYe}~S0hfaV)%z!gYfiuj38%%;PIEm9vgUicrRlf%aDgoksHjBdv=n0+LAL&f0HxJlN(Hw56qMg%#Qy{k7v1@%gdHyNtaX1 zmk&&sWA~UJU707$nI}w|56qhXOPl}8{+shloX&CrqCI%bwGB zpO0gu4@{sN%%J~Eq4Ue3^Gl-F%c3{Bs*?p{`GTvzI;^`oWr2>)0n~ldT7Fa5f2R9s zr~k{R4@{`@%c$2&sn^S?%S)=Sy0UY&IyAepKf5}3!=$5Fum8)X^Gl{X_^ub*pGM(5 zYgk+t8+jj_dGA=2U)hx}ySh(FonOhFL(H|;OSYrybF1~e_!9d(@wtg%f8D?zrV}Bze~Wg%fJ`B$4kY*J4wP@ z`mHBnu8-NnS7F4f&cxSC#aGO@KmJ|0SJ}D8S-O8b&O7?F4_UQWQpxA%$@f{x6Y9#N z_P5tqJ`ZSbzwcnrZD6w7VCKBk8%fA3$;cbb!oN$ybNSG-cG3Iy(LY(zf8SWlXWz`f zSLl4&zEh` zV~IR?0wKin47IgshxedufByA#zM+18)P?@kt$pbSKTDjxkEFiO|2=o({(cfd3-E;O z(Ee}5{kychyu3WSTs*sEJiGpEyz2u$iM)g5s_$B4lMtfP%+-)g>&R=Ie@lc?;9m*x z>xl8o{pyQv>)RR|3|>w=KRu(bJp(8}_!BsgU_pZi5hhf)kRiicf5YB2lsJ)MMT`Ev zi8^4voeAr(ljY=n5bt7ZkGt ziXpfZo)b0lZQp|5e_o08J6K@7VZ)>n?)%WK;*W8Vax6U60JLQzTn@=QGU4E5)Q@1F5}Cko1~#DnFD${|P|NqKpQP_jKS>(=F6 z!3uuf)hy}hI@vb9E-~)hx_9&L-Ch*1^zk$eBR;Q_@p_$*f8Ecd9DgQe&Ge1K{C(b7 zlF_>2$JWVcjT#OKZEWu@W6ZONwB~L1$qgpjox{s5$Wm)@Ew^2SOpA^iUoN3qzA2EY0MoIj-<4 zDY`sas*cZ+vitL({_g0q3$KLqs!ULh%3IW^!zgrAqV^grQG>>f9H5{Cv0l+yS6KqhGUk>>8hPbLgHWQx)w9%1*54G3aoAxlWs>J$Y85r8 z9%yV8e}s}^1xccjgA>lFR(AO+?-qwesm-uyGchGFfih&bHg2aqIaG!2^i;O6gs#ZX zq8kbnQ$oKx6#k(^k#4Bbs2NJ{XR7_76zhcG+ZrKFy*{YZunPi}zENf5tp_8B=)sMj zpvj|$AdCnCTWtrD=M^Y&Ai@YEL@|YnAR2i`e_tXpS=b+WG6^Du90*Sm3ah@6du_MP z60x3Jl7PaLdcF}P2qM}Rv@EeMsR214J=#zU>5WjJc-4w0`=^^$Y zf66ch8qqHYyg0$v#vO((-)W@owu^spm@**8wic!DZ#n22_s-!4x~cC!uOLDaWS0Xj zeaI9^=o|j-deDdtH6nR!!=69(7CR&4!F3G@8rcZ4ClngwXe$(m(kjI?LxrbfD4d!L z3j-+}3Pfu->>3WihQqPB&}{J;6>bJ6e~4~zAqN_y1iwTvjPcR#Yr!f*V1{v^O#BU8 z#H*n|DhC5Myy6?716(~Qu{uiBu2{^nT@xd+MJ~XL3q%-3<$7>M7Zt=9Tu4E`jB$=U z1VIqS(BEi`0RZ-lqD@;P`C_zQjtG-MGs(5mGN-_8qi>3A)mq}jU|CKPf3ER(uX}` zJmYEzIiU+xxWiIGY=eJ zL%FX;Yy?Ve`4oWpEW!>^&tAnre>fq%uG~Br=p+)`+ic7E#btP{AF904)O}RF^|PW7zzk0yZ)Op z9N2H1y=ke+usl;pwl?|;SB>jPCer{gkD{n6Ltu+S+2WA4E41yRa7#md7Sx5m#bI!5 zm@XiSM4uK-$qtjte;@z6v)<8S!JXQZOcUH_FL6U|Kv4=wrFGp0R<>1 z>@@?%=t#<{VN@K>($l7jX<%ASsZE9O_oG6f37CK5K9SBd^BsMGtZcm zbB|k0aBoHsnP$B?R3tC&6-*Y$_@10|p=kXj-$G!P*8FuG}U5bv12jN_#Wr83i8=(wr1L}*?Sm;Do zD-U8ENr^%vf0X38wX~95QauLb4-PiCzMOWje<^{XcQoi5hATPtP_U5! z4c~v#_tRXf(BAG4;42h(ztzl8HtQ4NVN>{aLX+VXSGf^5uLt!=+3|P&msqD%&BTV% zPE14cD>K&>PQPe_9IH4`mbZ^t1#cQzz-7q4vvjdoP6Yk_|0ql4TU-ng+5OR zLyx#dZ-+|HYfkSvo`DMr?I|AUZv!uI~>dn+Gik)0XdWjHo^)DI;VG1 ze@SnQs&gjCDLT+Cf+`5OKrCLbv1HAtChH7)!00l<;kXV8e94!9DyX>Nd_brJBW3=Q z;1flIHcIA!dgpUwtg-$^k)q=D>Zo&|B2v=Mq>|v&+R!RItzS@La3IkV-{r72BJCi~ zx!eGPl8lg=O&R%XFB;*2NYQlQ%{g%Be+E$~2fbzo^CT`%-~`n12X!m+glqE*N{4_e zs>Un{3vLO8=*@8G38&*6yhSGN!Bh~&6kH^jWatqk5GE|7Osacj&GKjdK&f0$q~nSc$FLCkoI+QOy?$>s>RhTzD?;K-&4w}u}D zB_m9sCu$0NJTMPT(=<7fB5dU}O;ZpAYb>6jG+DC|x={XTl7ux)GZQq;IG74Im53=e zhbD(XsLt*x0&{ByGi(YIYYr1X$iW(*(;CQOA$H4$;IYi)@yzIvpzJX-fAP_w^pS{s z#WrteZpvU7^o73;tiQ@Y6&s?O`fEM;D{;oc7~DWT4-5(TjCpTVci<3D;ktS z=}n&^G-@U^YAW;{FLY}%)N3}BYC6(r&$n-fP1Wu4N-gf9g zx28#v=1GwzN^R&m|1CQMZaaObJ3aI}jVnAMYCOv{PzANkk`zjpe{{C^35VdcXykO@ z=#&WU)bsFEq4ZQs`SjrY6uAI(P&u_zx290z^ia7hQEN+9^r=z%?NQ05O1I`pqb5tY zW=prGOQYsXJ@r;`6+b|gQAJhqNY%?sbxqxIPNx)WSQTqr6=`4b6f;Ak6)lrL;YL1m^lC@}-m1>%mYM!-dqLo_ZbzX_W z9@GIF@HJoc6<@b?-pZ9~+_Y#GRcKIE2vv15{v*{0CsnvAHC8WmGcz^eHkDo}wqmgY z8+;*SHFje&HW$FrU&%CJg=SzqrC=}fU@>{U-n@^G-6F}VsCb8sg@;nm0*3AFo9MF zg*LW`)?kgcXpk0Zl-5(27HMD>X=K(@XtrwI_HFHnXSFPAm2+#$v1_&LYfm<8g=TDv zrffImY>Osse~V^qH>GXiHgVzhYD<=GA=GXS6K^3;Z#fiYsZ?b(lw}95VVTuwxfE)h zP-+!-bV+w07B*Ge@P^jsEgJ$F_?w=+ezbbWVm5Iqm@$CZUFH_=a(K5^$mCG#G$4MSwkpfGfp-Jq3Y1MS&}Yfm0Yn zWn_s3Vucr_g>CbDoiu#$10M+C23r1Li@CUqyZ8nOfgFSsY}*WoH${jgg@`xBh&Kg^ zB}Ivy_#k-Y8(LH{J_V=>r|ragiChkx_sR zilv9kxPz1SgRwM(ad(8Nw}i!0E900%#&qjkCD0t>M}VXdGwEsapwAk@l-rE^NYMzO z0+4}5kOh;Fy*GMC7LnsahU(&%fjJ#p0Fr4Lc6}IA%otP9*iqCNQ`i_&+}Kgz_-#=F zf0GR&#op#KSmi276mkmN`W1VNr5 zkhUeGp&>wi<{oBap3Wgg5<@^Dg>&X8e?ho~$3Vvz+-Lp7StSI7KxpNqK}yz4wWc#z zA#lTe2*jf+B<9p5OTr2@KE{1!ZmFZ zvKD26sRUt_VS}mSFc09S4IwBMe-orP-GJka1f(>ZmUm}fw)$gx_Lwmxo*iYLKZC5x z`mFWYI>B2~lDSfrc~P3VQl7a|qIprKxnj+UeGWU2B1>45WEk?G3<8{Z21IoHW0Mri zoS2Q0`li1s3y(ykrZDchULgQ;6Dfk=><%aAXwQJ4Vh)F`n&!q04!bQ*f7_dV?6O2h zz{4;wbV{-|`_D{MPXkN}^WYXhx(td_uGGmGaEAzz%z$Kh1yMt&_$A5Oi{sh`skrYZ z0H}C^N`3Tme5@;{jQ*gzbgL(jqq(|U-o86z!J2h>d93B4P|Et3(Oaz5b%Vurt^c;I zmG!L!T9e&3vNhF{v)9KJe{G}#va!er=2VoDxZt(pP&c{Hrap$hexmTW1Sy!T!k5oJ zC)^;ysC`^gM%aoAbSj(W*nwChOLR=^LL-ANGC)?@s#s?vIA@X4POV-c36h{K@<)ua zXp9$)OIi}qO06~IN|w&@dz2)rWXWKBX(WINabgp_l0xk;T9j_Ve-i0ws$%9^D51|| z9VigIi8`X4|Ky!FMZ6b(7{2O6G* zUeWqeC=XH~1?=p$Wf>DQUN{mzHP}nPj!czeiLi&vtx8@nW}9zP{kD;VUnn6!I<7Uo zOXdrH)5yvGA07;}WUgOGgzj-{=3YAwpy2JXMd+`b5Fx0il3>rg%QSQf=_!KS`NZ49 zd+IB=>d)iqfA2Y;z20ouyl>Mzh}V5-+ufPp{fNn)ZOz_oj{+gPitTes?Q6{+Nl7eb zJHZtl@AbY^_#WZ&Xe8L`<7$G5=zmBU%Ye=0p;1be={`h`6z6j0+cN z{+?yz*$4_HfAviD5LC}!qodH4a(VU22$yH~1Q}^m@LfzO37?JpS`Z=^i(tzZwC4+( zJ+o}v{{! zFi-@M2{h+GW08kMnDS0N-F-6*7d>DF);kr6LPQY8tn{KWln?;}gf{V%(@s78q?AxZ z8UBToQWD0L2Q;E=SC~~W5@bpcFyzpXS?4?>qA`N>7@J*asdZeL2HoPBnaA-JrkcMI zf3}*NfGNgUWOkx3S!Hp`IU8qb@`;;hqyg$%YI_pO+G}kh%35u=F)G_`!}JCzYrzdy zX{LGIQidCOa6yD1@*Lz`BgSmyhIB7NvttfH1kr;`<#ADlSD7TioP9B- zJYyXflswZ27kRir1QafT^~|SvltIK;_4t|us8@=L;7z*f#FI}@0A*xRK_t19AWi<| zaISRM!JEXCFL99q6b6JGH7td3>Te^-3eKqf0og# znV&psCMdB#17_&Zm?>)XVU8w!n52|4t=FYOL#?SXZWvLoARFDosSyQoTCb;2xIv;Q z^9qv3!}Y)IOw_P$JJu@dEqQJB_&ERlZ!*e|-)mpw`<^J&$`mOeRoZ%|#}DG0yi(;t}!Qy;di2 z?!K9nr>tMk2qK6@bLU}^*`^RROc}#IR7OKq8)1Z}=MH0bNCnlyBw4V>4mJ_~Qc<2) z%2eys^7wzjY(8Eyr|*x=C%g}T{P3f|204ea-W4BFg71Kea! zY~#tS@FtW}3+2#6*C{AC=TB_u6|Kqu1sIf}DXYU>5)vec`+?=mifNgh6iKv9%|3+K@q*C^45fCiKjx6tEPjFAa}3iK7aoXt62 zq0oUYZe^x(14D^oe~gF*RFj99CMK&H%4=dWo13IbG$mQiO`da;=_Ds?OxaFPzO$qZ zlqb*RfkNe+<7set2Q<0^jU>uLTMW{cLBDd3p)#nbN1aVQh+0&6@cuMY?_lEfl)BUq zBF3YY1gR%QT1k>(@*SX5XJ_QOQcSjVlk3D}OgA}GOvclue{`*DUAZbruWk~ok`!xK zMe!)Mp4F6Tt>juU+15(Fb*!U=XHD%2*~nrNuY=SpCiyx@zvdONrc5a&3oFUPRx+`Y zTx=!dTFJRecD1baA7ue)*-By-keaP5XU)pWN`iKfqDARTLupz`qE?csZLM*So3z*V zF}9MNZ69k3f7{!VbFiV@Eg*Y4NYV~6xPu(-Ad8FK@QOE6%5@-f2kBe~LU*~+6>M%L zY27|z7m(ToBzFPnT|k0&y!`ENYt(@abqILC1I7k6(xNhI{!#v(=~du*&D&nYQrDcL zY3c#z+sFF$@xFb`Zy)#T-w=yfF!ngGFHVf&6#F6&qr13p-R=ZNrVC#=#6 zn-aqg)UX3N>_87Y5X2)cvXSLt4+1M$zz1%zg5TTRzL;{x2dr^mB@AUi#@B&7R-lg+ z2xJ8cS%E}8vYF3(m-5nh!Pb;=D5*?9D|Z&hpVe_FyL`YeA27@Z9PD;En4l2PL8Vcx1QuP0sTxsKo5G;gD!NO5B=3dW3w>wZ1i-q+{Y~| z5YGhEGXX6fKuiZv)10ognLW*DQ14aLBPF#dP3_)C59-eTl6L7yw_8?sHkPKCvbD20 zjcfnCxQ{U=;G(5%W0cZaw~%IOvDr##b|zb?LS%j zkJbb4IBN^ezJnhy;rm;7rf@NdNibsNjPL|K<}z>k)_ahuzHq)ryKlwzo3WE3WxzcS zfAsKzJbfZ3Aj$D(a!BEW5G{yI7OVmDE8iH^4y^gaZ|>V;+jr-u{Wo!c9(2J-oT6bT z$k5GK^Z*{6eMx7O``(8FCO{o551)F(cZ~Jy=Df6A{ui#-zYF%V>jfIGpo*r%C@oX) z5Ku#%DgnG-j{wQxg%M%LokI_8cW1iueYx&-W%{zVfj%?lgz2}Z? zw(0@X2qHA1p~dM=Y@m@UvdBIYIoSSNR5T3nf94(HDJY;iWDH}H7{Q_RJ^p|{e?H*@ ziujH({#WX*ZJl>OpXWcJw?Ct|Kc$C1r-whOcYS7&edbp}74j80#eE`CRd4h+BjI=) zu`l=1A-5oZ(BTmEKz^=Q4^Oi;Ve$=vzy;RB7XCP;egr{+JVk;8aSwvzMzQfUlyVMj zLMc^c7#3(r6etqCv4evF1-~?be_H1jyazS*$3FN+HTuUV{1ef2Uu8_keN{fe!Hvl;8$sP%XEy8=v9^z+(z}1Pz%Ghh<1j?d?<&xe@76k;)t)% z8}Jf`c|eH`f*l0mC7l91lyDC9a0`?$h;m2?77;p`kO})D2)E!7WuS_cBN~^Oi$tM_ zx9A4c;RY}e1l2M_q7aA6c!6L6g-j!b=|ld7P-BI1f`$G!5&%eV0!TmvxIYGnKM06F z3dlYT$UY9ZhF=kZY$OQve*#MnV-c^w1wlY7Mj#XS(1Ok~6szf#s$rAj;e-QFe5A;$mACd%$ zR|Z!C1*T#O9T|$%_!ZZfG}&lA+sHKC=qBH&h2hABUgm{W2Zp>QhCN1x8`gJcNPWyU zcJU~W+V>7LVg#9B3= zfW;g+P#}4PGW6$Sx^`K-7FNDyNQ-eZ{a2RAah7?OmY%nkf7jKPUiOwh7MJ*ih99Pe zcIg(`my5oVEd-$-jc}DGLJ-&D28Y6aFkmnu5g;Kc6RMdNf^$4QAvhDn6^&p_Cg>c{ zlO;JJn3T{Q8et(p0E1w05B^~|{;m) zk3H~=Bbl2^3M$rP9t-m?3_%dDfGs}t9$rzC8j++y;F_Y^FI*5X@dryTArG1;2p6GD zx*{Mw@Pb-75?pCCUO7HsYW_50Y9?fQd}f$@r1mU6MVFa=mk*XxG zq96)|0e+o27rt4Q&&d;vC3Vl%z#VSI`s;nLRH}E2=8qtAmA+1A0f2~wwtwV#YWumQH$gN%It^Nb9>?5w} zL$2v#uI7WT=A*823ZS9$jL*PE){zMvv{-ilCzwd1q3{@I2(F`8zxO4(^2g98H&dOy#y9{@!O z6rm6y5)-Cb7P(oY)zPRA0V|V339fU3f8Vea5&?;-DzEE1L(QQ)6bgP08ZR&ql{zA? z*_RT`A+IlK8EG?;^BR)=c&R75x!2OUEL)SHYaYJ(5cUBcnLr8CiL=M5x(E^~n@I_9 z^O->)w@B8dA4Q)2vSprqg`VcKv}XCc@EN@LGrZH%**@@J@gEz@kBpGDF6bD#(#K^Tk~J}2B&DI8^<>|iZyKID)LAuZCb{K_#L%QMVAH9S5yOf@-7H9HJ7Jq$HJ z{L2WNf+8`Tg3t(Qb0W7;e@{ALPukN%Y;h_;4b-NY6hmDWJruOEchg7s4zI9JM*TT{ z@zhTZMNf@`N_|jh(+C@jotVMYZ~;XssMTqP(RroO=(WO`<@wmgb10oYDor&k4K*$eH89OIGR-tJ4cPzJm!cpXeo=Y?Uo~d3|wv-DiEBX@5Q2 zcX8Gt#?|L--5MU(2>#yV z6W{(!Gv7&5-$`@dMw8!0v)>mE>+gM&;2J=;NB_G`!~p*yj%Z=MY}$?KtQSSm=jN>Uxo628QaXj_Pj? zTX8;ek#231&R(BxyqBKRnSPF&F65m~+YU3@IEy1{xkIcGxp9i_E~{x0zV zNAY9N_i<77gmU&SllG9Z_Oh`}-b48s2 z4iNqX4kTF6;6a256)sfx%oH?+5hYHfSn=SyTf^>I}+>$@45Br6i5sNxZic<+D6)Y)+IXsDI2Wf%0OWESiLpOQ;O(`NC!|>eZ{~ zZW+VcHL9^!=e&+JIqa5CvT6O%W7FaU3KTacDLf|%qFTHYzrEs`*Du3*!-kRkTlio` zw1yQ6{3c55;>8Z{LRO4XBW1%OM`GrTcV)}Y{^Y%C)c2eqL6Z$qGNLo$XhD5VLTi+o zVt+kKf`<8{O=)RKd3lUH4NY-s)#7-y%1*8p?BnKS%_fdck*!;}a_QQwPIfQz?pOm4 z4(}Cld9RJ3qlYS)Jyy#b-@DXIc>eu^p9ey_CF<5Ug5DX%KWLu$Mm~hT>5s4ecc3u_ zK!MJA5FumAAfl0H&Z*~?TOO|uKx}=OTMHFL}ss|cq6b#Fl|K7=j2q?CC<{7kbgzzAFeDv>|ck*xwnpa*K${z_$ zu~8r#s;nfe>%$tghu(mn-ORLwZ5titc4NF5~^!XKz}Xz zF42<4vyV}W&V#fg^(N)WJxeW$FH?%@;}kvpHY@`QQr);ECfZ2lV%1F6BWOc6NQJ_U zVNBtoCf)=w^`LVa=^+Iojyfj~+s?^^3t3Af@f~CGIO?8n8bKC{JRWQ${s&;oIQ5_k z-5AvfE+{$Yo?*&xHm6|($pa$Vrhnz45dgn2D%LWLp(KxCDog=}TaC(OCnd`;1))?` zbxaTV1cj z9*j^I3K;U3vLIu)$N>t}N((!O*bo-#qD+k7S2t)nHs&5tjL2bXK{6(B;~s=kGGU{L zsHyI^H#6v#9$-kq+aQcEr5;mqfMHe+kqr`SvbAhY9%#6L!H6<2bp9g=QJ!%Fh8*(9 z?QX7IM9QFZC<&r(bvqO!hkqQnh!qCkdL`D4AfSLKa>dIKyNq*&&0XCf(5}+LJn~3* zn5%=YO_UfTO8M!5TIPNrn7^MH(wYg~4$-7}{yt>E9(`Kq?<-v&>F<~3-s$xX`%Iop zat|`>AY`IKFwjZtdf+vLTnh@Oc!xZE@(LmZD+ye9-9ez?h8|F&9)I=thTLWXgEBm$ z9`D!*YoZqfE^y@=V$%&Jm}HD|xFH8cL&z|CKm8$18F`1W^DeY*2WR*jm-A z9&q6qokJUx^Ca(4b{1Plc6jv*E46K zO?>zwXqbXj4kAJiUVkBvnFQgCqR8DVZo!2B%8Pa1_)SH&_dV~eFMQk^AEU~r9Yb+v zed_z(&(;S&?#b_a^y8lWo`%2t{pB;KF{6AM5rjO5!is$vh7k~GkSC_jRYk#85KhLc zJoF$FjFHH$T6+#+~l2@cwt6$IiOE`rP{iXJx-xicbD6uG&W8~X;43Vrhy zxr0z6wAL#a5P>F8fM;EjP_;~;PkVJ*Un0M^$fq?jQIEX-8LuQ6KfZ+0lHbGR(=^FF zPCm_(Oao;o^M5kRJP?pDE_6dCppb;2+(K|)8^}A#kb@E#L=?Y*LIn@quXq;+-GaP^1194~ESn$eV1S7ankAFfOl|$|ZA%ww85HJXWR?Faq zSMbK8fC~}{y}iEs=jdr8cE1OM8ENkaVhVs z2@&hMM1SKeH}vqVKG~)*ZrYby7$FJ6=o}a1icY&eX1bjHt@{QNPT?ZxuTK%|dVR%( zt+*kv{&>X!87t5p5OGMr6GZF=`3-3Jq)LKl-Yfho6^bHqD=sB%(oEYY)JD>|KwIt8 zTKhigg)+8Gn{Co+YqZK zniIc7FankZ;;R%S!Kej+Moh~v3<+ge5P&W48UAnQ24ko)2)B5%gRMHM0zn}MZlHt= zfq#mT2nKP>O*RIieK|uZPZT*K2G3~2lD`w{DaAPZXeL9NzZWy@wdtebCLa>Y8xJkV zI`)cwsO-%i9MC4i@Em8k>qVRiq6fw7*exh=H!8Z&aR|8@?v4_-6kZwz5l%uLu67{- ziOp-wPUubRC68+-@s0UYo2E%Mwt@N5zIrAdGPw|F1cnpLNH*s! zpa86k<#5pi9_Se}F>NCdld7gT>w}?DkGA2ij%Nka?Z!r`(nq zzSFEdy5gR-m}$9A+OCZ@jE76O4rr!Py1yHqI?jSz*epnpxU zOnA!;r4fa0Oq3fB6RZ*DLNq0E7&2qb%MBSzI;G`ti##*}8uv)M)}hXuM$lv!srH9h zZ{@Ah1ojz%nDxzxpX*3ILIrkPk!4JO(fYktTOZ-Wk7V!!CiDbcAcy4xDc?()=Np>n zW3gUCsb9;V>(iO+(>}i#jek`^ma0J#j**pF`4WaGG&C%grNawR7!@?U6;T+g!Wfp{ z+KJO>Ln!!|vACKE;sPU3HnJE(rYS{KLUcJIgt`1t#GYZqn8_LDE5@OzLZHb+oteI&sXn0j zM3)IgQB;e1Fca~Ej(gaMbd1D_SjTiMjCXv;?&-y}Ai#of$G}Pp4&t&5Qwu{(8bnMQ zXw(^LbQx-tLXo<}OMi+n)!IZFqs30j2zC@Uk@&E2yhx1HlYcatfHazcjG2O5nS-=7 zYXll>>=|v08Eym`Z|oUxTp4l9NSU0;;@HTc>Byn^NS6u8PZ>!{EXkfZNu5bams!c4 zX-S=V$xn$%nrup^EDW0j8vdLF8l7Ajp5zptgsnN=`Y<)m%=sWK76} z%+{pM$@I;bsY_0|OH0vA>Elc)^h{m*%S;kYmL$!gGtEyzP3BBb^%RJGSWL+TNXon@ z+$=5KL?qtSOX^%1;B@{O;rtZhj2Yu}8RX0q4Y63jLG|&6EsXnGOAv z4xN+_4N)D<$PqQj5;ejTow*bx!WG@l7Uh%|<&+q;lo|b$8s(H5g_IoKQ7Q$)9u>(S zO~m;`QGd{s$|9A}Bh8c~&6FjblqThrC(V>7Wt1tcQa8n|EA7cF704|W(k_+DFU`3L zy+-fsIq)DfE0(M19IQ#mYJL$T|(aA?=fOq=*RB(=fG^F}0L3g_JYR zlr*)JHBFQ@O;l3;O`1i;%SO$~M_t57y%Tc41%Ft5LW!u{ICbQ`idB;vCd( zEL7V%)KVSRaXi)ROjW8})o5&0{P2V!fQu}^g;~whTD{as$<#^NR7UAkOZn7E2~|KD zRbn02Pcc>qMb_I4Qb@g1_Hfn{H~}V*)>oufm9*8M#8oTQRbErq@x(co1XdgqR&u=x zB!6MJJG=!Est5rw9Is#&hP228$rpr}NX{@abWw z!C04RP)m8x{^Ic1c%4>%tygP>6q98Xlz#;jm4y_RWfYgy6PS%zSc*NUf((4(S8g!^ zH$dByu)HMTig213-hd&6$U!&AJc!|fV7iDWQ4FL_6s27mrkxb04GpLT*@d80s!d4p zSV(qNNnQm|U;Wq1Fie3JTa!qhEP1Md@ISWA$AmC5zO%;wX)-v24l|<;c}O0Tpnp5H z{oJHT9)$p_a-5OTQ6clYheF$xgs|I(3Bsb4jn&=2&C}V=O`g#`hg73jh4@=Q3EWQ^ zTu3S0&Ny7DRV|YJO?=Hpef_z})zHaJN`R$EhQ-{HXpHEQmLNE$XSf&DxxXMamPVN0 z`HiTDdH#jTBag=L4|ORG7Q6*@QGXY-X+HoamdmIIV`zlvX%(w_-M*s)UObviP?uC{ zzaN1J1Qwu;XgXragZJ`T7?Pd4F+YSbg@(aj)xlub4eX!g%s&!6zb&+>z&uB9oZpHl&ejYtnHJnWt6T>lz*?ildz51 z*hzwf(KO-W4e6O6?&?_!a)VW|BLC^JX;CP+Wg|hXBWU|0CSj-)&{lSY6@BW4pnrz7yoZlbm3ALxV5{BS4`|ZYX(> z7Lfj_5O84$XkfRo@uH3KrhkihmTx#LJr+bkFp*KAG(&n%jzm`m)mV_^*!f7}kX_>8 z%wCsyVnB&vJ(=P}sbWC6V*JSBfps{bB8}PcuuKR7(lLdYDyNI+4^hY^FvGJ8v6U!m zr4$h4+mS9i3lYybBg@kxY<3<#a*9`ov2LOj3^K18x<70tATwj5fq#LRPrEol#@`;0 zWE+B)x=EdJDq*ZDvTv>jgwY^A(pwvmm&W;7<2~tD; z3(BPw$~S=9JXr#;fq%KOa3X>?lD&zwh{=Av$({`qP>Lv&izFXdCn1A&q4qfCLBR~p=i*G)AL(8*o zJeCcIo2CPT-GZ=eJ;Cfu(d<6qY{)2ZS_5YP6lVS0UX<+K&=hkgMQs69ZR}(1)+UN; z>1YK0B^HsPSbhg7D(Qm&xf}YIhHEByc)>~L0u@i8Tj1i%k{2_=2!`=6nEu!aFoIz> zs1mhpgnx&+y08*5ELyJ>0wTc@8_N4B%7dJnTDI|KqrD~M#o1sehiJh1bXdoTj)|9F z$^?=KG9oA<{ARkC;;gXi?~20ZjJ5OY9PoA>f_IHtB|h-b#zZ>DLTUF@Jm*k7_d-7J z^PqJbH<($}xua`2E8$21M7M6PVF^~mKcoVo6MuwY0ZH+PS@8>c8yTu&GRy7zJ3&KN zont{81Xf#UsOA?r(Qrx~X@U0+;b=j4xKFz_t$BBYuIpC58x}!`x^Z%7P;cMz4cJRR zD5$Q8*A0LclDrcU&1W?K^jK z8-E>lP%L+I-W5a3pr_S34}2r_2_q?>J) zItl{8`Nfs*fy&yAu*oKf0T;Y3x#mfjXR8ZM3!aE`VF6_zY{aJ6g1ZpHD3%*V1gECfoQ$nYCL+c zB<zjqmJuO z^w_?dRW@D>v)*aclqt!rN&53f(kJDyj(M(84mV7HvUGWP{<*)LUzr~kntvPmU!}*X z|GkABccdoeZ@-+(mUWS@#S@_&ezJZpXPj4?wJX^eU6-SbV6M1FJQR_81^WRr!xGt5vG zwsc{SJ!!ZlP#t>N6Nq7QIU<=inRup6DXM8xi*4?gAe=JIIOj|@+8L9MdGgt(pNCoa z=Sf`(dJ>q0inJ7>A!)a0L9aYlD4UDExhRZ_*10I2i2m;BsHdNT3V&0eh!VQ!p@ojQ zXrq=w>glA2R*LAQh-QlDriFGIs;;{tI%=V%B6_Nzsgi1@tgSlwYN4?fI%}b|7J93o zx$;`AwQ%|>D6oYJ`zNu#8rv+giz+K9vw}J+D71n~`zN*6!W%D++5Wk$pxyfUt+wH! zJ1(N-{(0`7>HfLypMUN8`L4VRGYs86_t+D$#1q@2PL`_b^eMi5>RT_r1N$p!Sv&SA zu%86`X|SIN_o;BhFT+fiJ=pkCv&}cxoQra#X6)(4e0J>Ts>enuD8PQ2+$YL?s@x~b ze7YR7)KhO&F*a9Yo%Pih^IY`3S1uZKpF@l5wV(b+m&}%o`HirIG=^* zd3c_Q+qt-%jnDbGkG$c~C`C+TyVJ}2sMs@}Ts$NTg8 z=CN}-`>?gE`hPL*p63a=oxRWbJDkD8IeeSNBOiX`%G+u0pMBe@dHK>So%^5G&xt*p z+PBGlo8GGlzWDo7P5zwb`w4w>Qs4jBhdB40ZFiJXo2BIRth+c0e`nGk|0?)0|G^1> zeG(v?2$(?wjt+r@Qy{DuC@Tl53xb;B%c3a$Xu%i83x9)c(jcEW*d`Co&`k8O1`m7a z!yZ<}OKJKS372KUW}&cPDzp^~IVBj9!7z#plcAbum?s;m35QiQ6E3oFgCT%1jA8sD zA%fVWA&N{q3p~=ml9;k3MyrXqdZHBhcaL{CX<<}6lNIfR#WQJRC(2+a0KrasQ;k7NquDF<1| zM8a~BjTE8?KUYL^9T8oYY#SyIrb%~kvXjVbL=r^d4`T>I5bAM}Z`xwYFrgAostgk= zq3KFP!cvipbR>=-`Mz5QESGIu<{RC^{zUn-Fn^fBG@c-Wkcnd^lae8VhA3VEMW;N| zOVMo8G`&SNkwHg@P+Tf>bPO$>L!AcEmPvG? zb9;t_xKIWt^uUOZ4l*sExF4_o_ zh6EbPlu#aw@r_25fv^7g$TujFS^h3GVt=P)fI$w*(1>%KhOdG!$vZ^h2F!Bdt2D{y zmiDQpes1ZXwGF61QJ_^?QqNBCgOmK)L_amzPfZdelMB^)tqA2H37O474vcvVC~)B% z@(=_Jf?x!==70#_4MKG%l9GB{7Y0E!tzvsXgjakc4?V~OWIBK@X&RpPSION47l^ZhNF#4f8gu+XSww zz=BI{=mulAy!YkhKj574#od+(1c8OEWXXGKgnV4%1LaUN<Bzzxp`3V!qOjGg`_p??RakcWhGAXri4fD4xW$UQP)1TK8^9BwG`GemJ_ zV<0&Nu~l(QGR&h5lc~cz`tY4YoXy`Bw3gK~aoWP_yRibVtZzFj)Y2N`m(?Q^K`4WH z8$kpeyS2wTsH;CHkpw|(1k9&Z$v2=-UXiL}KEvkacs;}%VpwB31;S$<#C{`Mxx2I^v zF*v7AgUClk!QB=ebn>K>*b0uiuc=IXLr=kIQH&wpNT!ajK^Uq>x@N%&wJVYPiu_t?HPo29o)y^@td(7><^=;fAdF_*D zJ9OJ#D4i4i??tYt{`&2cu#{`9+`ka&{JXz1QUS^`W=QK zq6~J@FFEr`czSc*9R%^`016UuGhcIHgj*aF!TSBiEf5SDC{)(5k)2Oy9F1_Cmv|hF zfSdtd(+N1>143Z$b<^+BUGQDc@QGFNdE4=UTk>tv@-g4SkbebW<$>je(p=S5fK`@8 zd=iC_g&T0xYz)I*QAQ>hLnBm{edR$8lA0i7LT+@{XXOD$-NP-oL1m~$Yf*+W?S&h- zm1(KeUvZ#J6kv=T;Fcs{j4YrPHXsB-APPtzS5aV_)ZLlf9hu}^nCxAc^xc;HT?jgg zLr971i4rNTlz;wtMVMTy6qJ~PDEOft3I<3(1CgA=O1Vcpe8V6f#2@;hv36DOUL_-YXFjy8th~Ci=pxNmS?eUZCO_iY)&~{kEBoM+Y!eT5E!X)5= zC{Ca$o=F*!$r*-88hXhZdPy7h$Q#08-DJXN9TsTWqJK;nVKS=LVC6>@y2uswNEW)t z7EYtFGzM^RV>bdv9-T;6*^+FXm@3Xsx7E+Loldwp&bSq$Crws7GM-J`U_2(%4{nGw zt_U^qh&8SVHtr(``OX-si7ti-FA5PlB9}S}4?CugJHBK1+`~jtBsi)>MOLJT=;Mj- zBaZmviGKk8Bu5Hl_7G$)T44NO;Gt;@B#DHiO^)RJ zlq8uLq?aIMK`CU*F=Y2RB=JDxOfqFYl49lHB8Q=(-{79m^rUh9WS9iymJHonk9YIWQg2kjO1j9?4(*=<;rEHmw#+!k96fDeWeie0_Jz5 zrH8Dgi?pSOyd`1ACEm^DmeeJW++{cAWoB(UXH(3KGDzoiQl~Q1jO;C2Zg$*m4p46b&To=fvYaGwrDXQBW{~-7&#W2ryM7GCMO#%XUsL{dnzbT z>;v9VjK$PvDQZl9-q3!oPfKFueWK=grhi;{p5sa`B!-IROGZ>oF6fD(MQ38jXP!uC zV#sK&Xn)Gxf8Nstnxk=@Cy%5jkE|z+v?q!JX-%l8ey}KsxTt=>D3Qvj?`f!Ta;V*T zD2{@tQHm&Ek|B{9Oj_fFo^k|Iy=zo^V z>HbM{DSmvZhlDA9jH#WLsTejXjtpmv6sL;}sE!tBFdnGiCFqXz0iecb7WWX*+ zr|M;uTIQ8DT9%gTuh!Y4mJix((0`-?Ql);7t^#MI#>lL$NUg3%J&?n+Qfsx6!#7xK zwrcCPa%;C*t31HNUw~?ii0X=vDzKvHslG?5V#un#N2{*us}gFgHmj!YBCdwWJ9t7C z@N2*N>%RhQzzXcZ5-h;7fhW{xSd6QSlPi>W90o2fR+Kynn8UI1c_A zfNI-~t_VF4!tRl5$(roR#?lQC!ZNzW!mdcelE}kC?0ZV=c~ESASnPRVtj%g{IMORK z+3P{#tA~iJ7aHx+B5l$t?b3Q-3#_cT9x7}msv9oq%+6`ej>pa7N6wDN&Sq`Tk_g9Q zo5ud`;=-=lE-uufE~4_PxcTbjc2Z$c!Z0*~8w_Dh{ePh#xB)11b<}?V z@0NkCmSpGXI!fPi$L+KavjXklqNLf9$m_=L_Ll7IzOCc(DCEkf z{)Jci!a!`*!|BDJevvLn+->v{YYiD|>0+-n)$WXnr}dI&{}L@BydC!t;5mu;2@Ig@V#S(kV125JKSimDsvG4{h1^y4f#6p+j@CN!6{%qw)GIWk*GINK|^kOvUM-1Zm zm+|oi>j(a=Y-3AXZ4NGW$HMk+SnFk-2lOZZy`YUp8o|Mv;BKwC zgGz@tXXCk*;|J;%{)_33LFtxp=|Tx9kD+$_Y4DSkd-#{*0S7RDfBZB(>;Q32;5~u_ zhb0LTB#FFw4IMfZl`zjkiWMzh#F$az#YPe_b__DH9+!mB9-bqT@L9!oMpQPG2PFuR zTM0$gLm3j}%7y~v6}(q3VnR0~O9Cx8P~%XM8=XFdn$cLoAkdH!6qJz2yLuptR?OqE zYeG{LL6&`Zw&5^;v1~(DRGX1$RJwJex`o2fVwr?SxO%lniRjq36wf6|!bP6Ib4U*+ zT!?Y5UCNa$U&fqS^JdPSJ%9cN9a{A0hR~qB)9X+5YMAd#ldk721_=@~Y~Qv`a7te%3VLvOy!+Oudp z`5KxkpAntViF5G`(P^Z#fAi%>BNRW zvaun)h>$6Nyb<*(h{uVt3gWAWh7x2W7h&9NJpG_+j=Z6W@X;aeM2gFyFn=U#t+uEW zv&^M*q|!3HSagWVAfgohimDqMDlnl&>fv!KiiT3GL?2UJQ7$&yL^M%F7iF|jM;}Gh zG}IbejkS8@aqZ5}?5W3>3~0kGx82AfLyX`~{p|#QF_L37Id`4{slkg(ac;R-Zv+p{ zp8E4Jqg)#rN+ARv%JQX$1mps{R~(`>yXokJbR+J%EKR_Kh9Psnix~88N2-$G$x8*_ z0@0X~hK*~r$75lx4_Q4m|Bta~UEsxo{yxCvnT`U}}2?DUgIJuVQG2fMrS&9P+wioqX~VLBFn~iU#LGb!&}w2;Bxj48rbz zhO7+1luDX{0|Rq&EJdEr+%U_#jqtpNK|op~P}9an4p|HEphk|FrN=-$vXL zgybzxE843QRsIC=3Q9N*T5?K4CWaxWpVh;CP!JyRm={0}cF=<#1R>S7MyalSO;e!3 z!q{#T22!=KH);zG+fG0nxFLrbP~ld8-zXwJz2WL_hT#?ZO!h?wyr?R5ZpK<8+>i8HhzK$>Rna;xaEH(2jh3T1EJikSKY7E|I2d z2$S@exBqdHK_dC&%{ci;iYP9TT=<3{9W*g2MpBL%V&f(0NJ?7P(w4WxB}GP9kFI&` zQkf#nJB0qALbJ87g=vE!aM>GDYF;-o^A-VwwYa&&`9_@cRcK%0|&VrUB zpcNewluRI$3kvm}ck1UwLA*ntq~y^hxnK%{G>-@;yoWsS=!R5Q1Vndw7Devy4s>dT z5(X__D@$ri?ij2*dSu8wM1fFuie)I91S*Ea(l}XV%ciyzqjR_*h(HQ|^d7e;k%2@p z&U-YJie-3)Dkn~Lk$aqD z7@`2JF)X@JmA>N{&oG96Xf4Y@%yQO2&@jd~jL{g^_A|D#y(m@dAzRp*M6;x5U2xWXkb zPG&od6EXL>!-(!?d20;cj@7>R#czH?Q`Qpx5^SPThFUYgR%0T66Rwru0bNgU0~*5Z zRX8l(Q{yRAa~R_lhZZfChYQ~3mf6vdhIFJQJ!wi;+R~SP^qDP%ssDy%&2DxxoDZB} z6wuWMO0ctW9B~5_FajQeZgiiUD_s}Idds#ZEqZGmUPWVn+Sk7ZcCdv#Y+@JN*kRSQ zYSIiEPUQmB&o)&uNnL6i@IVQxLQWNIux&!T`q`m3<-SBR!BqpH* zwJG5Q^`^Fe5|qI2e2anKAMZCCfR@*Um)zthM|sLszH*jZZQ=fHH)kFWaZ*L1A)cW4 z+D^~{jH5u~8oxKlKW+myP^;xfM|#qgzI3KH{p2raw`dqLbBIggQVdB(&Xsxbdh`6$ zKL`8ILC%Dgklg8KM|;}UzIL{^J%CULW@x5Pb5i+#;T@dPGdbjNDxSB%@AVFQ;5Ytp zCYa0YZby9L6~B1KH{S8}fP1GsGk2Q%K^=OANj=oT2CXO9?|=Wg;Kly&Az1zc^N&Y; z>Q%pb*0-MAk&k(1C=YMT*8}g>yVS&gV{sIGp6AvU{pk69gJ7KA^~Xnk@|C}Q=7$jW zQg=9i#Qoq6+h2qsxX5|mKi~V{gB|2H*oE<%KYsF;-~8uCzh#~;_vk;{4%iSsMHpfY z-;mn(_U^v$`;CkEr{Dko2fzRnK*obThA==qJHP})zy(ym1Ee}*QikelKm?SB#W0ng zo*%{wO8xHMf5^Obi_u4#7C6GNTkF`w8Top#7oq~OytB(^u$gC z#ZMH)P$b1sG{sVcywl(sG=sS-tiT7{Lr%%TUC06jvqgI|#9TbYQe%Wu^hH$s#b5-+ zVHCzjXZ`!E*=(MgV~mm@B%%MP2Mb zo}(*5;0kF}M`~opb#%vegvWQ3$9Sa2d9=rR#K(Ko$9(j{=cB?`{xl}5yF4O<2X!z8 zTkr%%AOu2SgDh~!hlI$8WP^!h16$a~i{wX))X0qF$c^;Kjs(e%6v>bz$&oaF$rmg@ zKUB$;WXYCvNq{63NstG7c*&ZyNk4>#f>ei{%`9N)XweX&hGS1}C&;m8k@9fV5RnP@x&<1tT2ZhiGl~6xS&r<3iZ$s710rYCD9T!(GzV@b;F!YLD3d<(HDi$7?sf(~kX~9*)1c8E z<>+WKGjn( zwb2_j(h3ArJoVEq zR6DIePc@BGMb%VQ)m3HHR&~`^h1FP<)mf$0TD8?%#noKZ)m`P)UiH;q1=e5{)?p>q zVl~!dMb>0h)@5a9)@F6qXNA^imDXvc)@rrZYsJ=VH4RHe5=f0yn7|ZIrBrPt*K#%2 zb4Ay5Ro8W8*LHQ+cZJt@mDhQt*Ltl)^EMKq7>MHElS2P*n>sb zgjLvuW!Q#w*oTFa?|LI7McI^9*_CD4mUY>eh1r;u*_ox;nzh-ICD@z|!BPF!7W2cL zm&1+&HYFw7vNhYYMccGh+qGrewsqU0^+Or^*E1C)m3-T~#oN5q+r8zN9*+YTe-&7~ zl-s$bQu*W5$)()NwcN{XPL+aOZ?&>I^xV${-Ov?X<|~N)&4pYg8{O1Z-PL8?);+kZ zFx}Xd-Pxtx+O^%=#ogT1-QDHg-u2zz1>WEl-r*(Q;x*pmMc(99-sNT9=5^lZh2H3u z-sz>@>b2hM#op}I-tFby?)Bd9BL&~^72okC-|{uz^F`nERp0ex-}ZIi_l4j1mEZZL z-}<%Rm*2GlMSl`D;S)yT6jtFCX5kig;TMMC7?$A~rr{d4;Ty)`9M<6-=HVXp;U5O# zAQs{wCgLJC;v+`lBv#@jX5uDx{^BQw;wYBlDW>8ow&E+s;w;wUE#~4b_Tn!F<1iNE zF(%_OHsdo!<1|*|HD=>BcH=jO<2aV%Ii}+}w&Od-<2XFl<2~l%KKA232IN2%M&v|RfGIfqD&Xo`M@>V>>p zkmy@5iHCotX>%5YlE&$r*6EYJh2I?pon`8b{2ratPdF6ju!gLmLvE;#C8=v}PVYMoA+Mkec#hJm4G-b@hbvX%gl9$?}s~_#Rlp!$X!av zY6(bb+%*OXknF5B1;Wm0{=;tGcL40pW&kdD?9_(l$Tn@9PJ+XZ-5k(ry}n&zXzi>X zY(y?>*e>3BDDBp6Y}FQSWoGT$CTY~J-DfE4wYJ@7cx#{zfjoGH<1z+OCD(Uf@*d{M76jNzt<*xT&vJv`K59Xz-8?Ak2xw{Aa>=5FN$>zsxHZ_R`dzi)pCm+@VWY`Hew;tp`xeTAI900zft3E(B! zHSnB{0RA>zQvm85(EjZKSKf2L?vhS{Acxcv&uI^z@hi9G8lPPxhjH0Wf}G}pAunkg zH{CDSX)?d@5LapxXY%B=?3^ZVNY#LxW^OFE^I6vN*%bw#w(`0C2Js$gs)p~8&T)U! zg$AIO@7k5^oNk2b?WfUE9rog0*BAzbzUXiG-Hmw#(3-aiY08FN zF5l{8-}Q-CK6CbH_vIWSGFE#Z5Z%sNGx*S>m`hg8N7>^`p)qTgzs_twhS`Lj>^+6U#ek6pXR z={o<{wkPRkR}IxCX@&2aUoUCJH(t;uY2MG>dXW3Y4*9O}{gNK>t^t30oz4SqeSXPi zeg&`c>DTJf>1^cw{GX2i#g^{IddPkDh`0Uxzx^|}UFBy0 z?FI;c0tXI!cYy)Igl-5zi;yrO7lIN8=6IO!)x?Y%H)b?w5n+mZ97htY`0?b)Mj7jc zI4R-W#1toCh@8j;CC)963nv=O>GNmK2+$^a1o|^bO^Jd&hSZqH=t(FUhkcCjjU<$h z8tRek{_6EBm$4TIB7e8*(vKL&2B|u9qo}IJBo?;#l4H?_5n$4d{IKcSn<#_iY2-!& z&cPC&p-dw2so|+PlL%^*hYM(zS0(lgT82_1lRtBG&Lb0L9-;?*@^;-Lr=X01FDE6K z7$pohu62J5F7e&1VcNCd;>xk%N2!0ulP_=nJo@zN*RyZ${(n9E`10qsUH(vXSn3H) zan#QM^FWlGlnF<*Sp=Y5mL-JXShoO2*+91p=h!BZDb&GbtxF+Suj*} zi$?V;(L29Av=fRSF2>V~Ldxh;g#-o4Km>Sd)Ch1Bfytmh zs9D74M4~EE3PD|YJa9zs%n~%sR#+xVO57}1ZP4y zNGj1i?>PG`w9!7>sz8B)3YLR{al{gj)K*(0LPYqJ6;s|$`q7FTacU8x10kExi4!$u zkyRV*m{7fzf{ReRz`8{3r$WwY-Tp>p6l>OVA5oGk!woz9FvJl{JTb*u?n;)bDy;;r zn-e9x=zp6hwW|?TP$9(MbzG>tG8a&kvFulVJ`~YN?=tYKL?s(FEXh^*c&tv4wh6O8 zs*1O%&)4m18c9pyTr^WcUii>fBh5K7SW_QJbj4kJ{WaKOi#;~kTV6bts)J_qN=k%+ z%c@4uu*oh-Hv7cuTWivbBuDcmbhM{XrJKY8sJi5#E_6q;`QOhiO-~ z59Q$1#|S;9(URf0Wz5`~H!A)a<+Rlf^yEr&-c?H$bu3W81nK?I@gqTV($ge;V<$?N zzkj|y`|Z2`KK${QceYsc6ZDOXr*g#hM@CS;b08c=$zgG5AnP;?~WQ^67IpN2R z+{>oOtaihXaDi*YyeB^Msn317uaI3CA2WX?Vla4GgpUK6CqgI*7It2fpr7Q(lk^zP zW)gv*;#`m_FGx;wp3^JuXyET`S<81iR3ZDzUsYns&y}*Yr7nG`!~97S9qNyxC8e1} zXqpfrhQ*RNtP3`m`MCi3g#M=)iD*z@zy&vmLLLacD4j6a(T(aWc3oj0z0@f}jM9Iy zq=HgcPM{!0n3A=uW<4uf;Um+L)bSydES^I{Xij`-b(F>g>PL?BjJ@);uYUdOSVsdC zf@x%N4;_%vPv|9OO2{=OdGF9Qte{bg2IWRqzTVlwk&j2Y9IxB{Ufk8S}{0V9pM z87c5_4{TuR4tc>VG*!KqiG`%tMd*J3tmR2ly3&@8DQia}A)HWB&3^8RXjkQ}OWAfL{A)2$SjI9JZ{&6f zk*kUyqOsyCPavKCe62`@JP{o8nZmm6GaEISU=Dr=xN|;OS0_x=k63^9pdS$zK{A;F z(OtUQ*1k5jspKn;$iot~lAks_SUb%>c(<%h!Mgb}}zzsEyR4ygW^Gp#bij*$D)pxXw1lGp_NClSfrS zQ1!};P+rAMgx^0L%aVVfJrQzSgbT)oMk99YBclYqSIUAHmE!!Gu*lbz}gmEwHm z9G3ZBoFYDn{y+}6PzEj#!RVQ$IdaF6NrWWf72kNrIY_C;x`uz*B21GA%bCOwqX$70 z&p5|7hVktxz5_|`DtLfnjBv^pH51T)#y4(}c;k61(M_I-=biNoL4b3_{;;zrHDC7G z)4ujvRuw$)R$kYC<$I&bQ}TR=ZZla+-lKyj?{{uI8wUz*0}1LaHBghwLw4AIHnP*t z4!XSG7@h-WtXF?!Z~O6+zx=4h{Vj*S##NP*B#;(2;^FxcAQ69Gm|p8$!U`Bk6=Uf@2Vp8%BscWH$3F@XP+ z1+NtpNB)GGQNRF9jG#6#SkjS2i49iC$=u<2ApZ@73FdzUnT^l@G93j@77I?0?-14k z{+d9nn@x<&k56UG{+w-(=U$EKL5H*Ys)DWm#X(C7d-$p=@!4=`lnOrU>sLAUshZ z6egk~F5-Xs++a>zfKpL{#P!^&NZMC?ApB(nG?ane%?#i8R&KrFP8r`<)B`1C4^D(Y zA^L;=nMhacU8(7aPlSLVR7#?pg!BcNo(;q^+}D2*pz(0tLqS6w?hX?g)-DpnD?lHn z)FS?(D4_ts8Y504HCAKT-GjG50V*;8B$mM|SmJ+qK!Yf_ff2v}4CFu(FclNt1$N|t z5##_2^Z*y+fhm;HD{z5157N%hyCSsSpA_pax<01zre{Cv5B5Z*W;DB%rCvg_1 zaULgfCZ}>PCv!HZb3P|@MyGU6CvytH79@f)@TPWdC)soYb$%y!hNpOrCwZ3VcM3r# zaHo2%r^LX4dA=un#;1JF=X=5dd)_C0){1x1Cx7;*fBxrure}T*D1mB;G5{!oCa8ig zsB#KHb`~gve?};MBm#q0sD)lAeI|m0ZYYPc#};5Hi2jDCh)U-cbf}4*=y(L@h^{D$ zwkQq=0gA?`jNXNd)~JnM=#1v5j=IH-_Nb5k=Z*%cke0=d7O9cG=a43;l9I%cHmQ?R z=aNRLln%s`R;iUP=agotl3pp7cByb`DVPSSmyT(be}<`<&M29t>5-l(o1UnezNwG4 zshn;ooYrZL&MBT2sGaWVisq@G;wPU5Dv16mp{l2#9%_Xas-p5HqBd%RE-IwrrlU@( ze@3dM(k7*5>U>@*r@|(J?!Xq*K_aBV4rr>H@&z_X#6F}dmc2p}NPqwUfUM3ct=6io z-YTx0YSA5wNO2 zB!Y%s85bxkxt6QBo-4YhtGcc$ySA&X3ILiZsxs_AxYpi(3g|@aXSQy_@sex`vv`0I=AgNKc)!@exc#;nZFEY1FEyl$kvGAF^J6}SE?`Ss@xRO?1y1Id;w zUDyL0aH&9 zwrxgZ0vTa$ILn0t94jcg- zWUTxk?QouMSUfKC9zh*IFI&*A?E+JO)`3^3!`&ih+HyqbR%jhm?tvm_y>o%|hKQIKFs{~B1 ze|hY~+p_A&mTz2G@N@oBtN#Kp>L`Q!CZ`66h4CKe76`3en1kYKuQDhu^S&XT&~?uY$5_5Fh7*nk^gzY!a_&UDyNEHs=s$gy0?l za2Dqe>xDg(>Jfl2Nu+^rez8V~?f)h)e|yAg1g|k0x3L?~ssNPg4$m&*eyeb9@$Qar za@Ii?H!m39(>{nXac+V9{_k9%F+pT-m*i{}lf>SR@fOJO82@ou>_Z*%uzWVMM&z$? zUZ?Lat#9`3BQL0cp09C|u|FJW6WhZR53XH|@^Z#-M&xfN$nLcgCnd{8AEPo@f7CDa zCNcP~FZiy<8yBiFj&|FbP9pcax#Vj ztUwqrakg)l(uF$Y0^~X;z2*Wof1q<8JM%0L@LudPI%mX0^K(HD=l&8iGKaKCkF>%n za|K^=d8q6+zmLRTtV_T2#cHwv6ZAyvZ8W!qI@|F9ZwV*cb65!Q2WQ0Hesm?Hv_8jY zMsI{3BPT!$gg`s(*=|86Vs@dz*R+dA(Q`|nh5e|2$Uu{jUt z7OeGIuXS3J{-<2z!csq{;Zia-i|iy11S<^v_Akg z0x$DGH0?_JgV>_-UQfh69JbO*1S@N+?2 zL-a%#M!&63`-5uFc4E6Wa}z|$I_*9f46B12bF^+F zHb=}iZ5y@Je)kK-C)tt(P&;Q+`-4CW!RN;BI19M{Dnn>*Y;ij1e}d1oc?*Oo4<{)v5oujK;W}I>o9whglR{)lux;?qBcih!&^tV zK)`qN^7tSFa5}t~d9h zQ#k%87iYMdGLk3puWR&&(scb2xP%)oDxWxb(nT&*I&}JYMi?+r4}>{XGs`yfby~VV zNP18sxN=gqwV!n^fBK}OIt81%0-G&zlCc5TxIavEq675~B=tl9YYu$YFyw(6H*5}2 z!YkYYC8Vyce=-0)-hwFP0RiM{1iS(z4E(GbfqShN5&WtVWWp^Vd=KcV!80(hcKpXv zLdef*4rF|L<-i4eR$oP(tTMbQI6TDP>OS7WD}0-;=KRhZtgWU%BOJUaxGb*BM|{>=xkeBz3G4W|%kEPb=QH22hX?y`+WUtie`l$_y17SoojY;56X#On@t}+T zIA3$UCZosNPd7*G=f0b7HQT$?lP}dL=NIen4WIqnvq#T1=P-{1kbAoBUj2*OySXc; zx5hn1w@@aJg*tFO-`0AA%ept+UBW}n}v~UJ*E|~MS z3&hBN^JFirI^S*5=0cj=XA*C*L3`_dvUpbjYpG3yJmdhw3PAZ(6Rn=VKPdhC3xxX9 zY6QSc3h*ic5M2;CeXWK7K=|_~2tYxD2N4o5e`NRbFLQ9O5nBubSm4RWODv1Lu0H*wb7fddCNi~{#<;fbS8u3U+R9FeGP1`QT ze>*ghvQArZ20~d-Up3xtB3#*33}Z5x}2812!!8E1babpj)A74jZvcD0DNh82tSzW%@LV1a_` zvrYnJ4Zfk^(95&_{?ku390+7;w+IEoW+?qEY)`|X$jeL_AvQ$tH;fXKj-h?ju}8%f zQxvN>{k-DIw1&{qFFq6f>8>ZHu0n1s2rp|cE`jWskfF$C6cRv&vh%65wYH+Le@5_f zI!LB>zRB_%XwG5iodl{3Q_Nw!3`wAK3@E6FhPq@329kofN1!(kS`#CUa9T*9cg*zX z7Lp=>Mj(R1)YBh68Tu0dN-iq{8Z8CgQ_Cz< zfrJu=G>CKxj>9VKQM1T!tTrzmhe9sCEq*IeFA#C)&mU}@BJis%8lnydWjn0bUg6N1 zu_1d>Tv0`J4*RtsVOtCLM5!ngvW|(Iz1LyQ_CX819q9e(t8V3jw%6?Re@nI@W__{; zU3J+#_TiIJPFbK3S>B;n%$U`1z5@Z9dB3f?Qqo|5n#=h!pN*3)zC;d-SVRen)p^L~ z$h~<7ZWGS$jYbxr&rBN!^9ATH$e$5cK3p8@lNNEHPjk4&Dq z^>BZTNZ^9eN6*Pp$2Dgtg@T})F+X+B zHw7@|%vni#^&RAuXEjYx{mG=DAV@VFA?mFM0Q*u48Xkbl1D#H;UB4OD8uCv`;N(-VOAHxfooV!fsxvj(5DlLaH127LZl-w@rh8>{?FS&qn@{^#~fQpkc4d3 z#41J+X)dm-bf7M27wF% z01@z1JtAO`3&`n903bl0+=LL`)q?@LVw689aS-4U>J=3Eq)mPqOoI%wCXDlkF3&klp#B2CqAqV0D}djchywSbHE{Wl zfJ=#}IUeaqC7I(tMH%1^#iNcyz=5SLb?Hl6nh1?%3}FWiO;=PDFM@t6p(FIE9iN6$ zdnv;xMa_l8h7!UobVnZzDKSg3;MEW4{h+Dzf zU7F@LuYrbWHxrr{6?E2%dL}|$Y?>br6rmJ$6*MZ##ae^7gD+krhfqtyVbcJI-RLo> z4Dn+c(3Kw%QpBlv;pnpV_>;JWY_W^{8y17+QIiR2HS<~^{D4EaLA6Ac1KG%G4MG4x zu~MITe~2FJt~sZgL~nHliGZ9~X%UAYL{G>w#yS3wODZ~@b0PD@l>j2OTw2w0QSU%* zn~pio=+=q42#LU^tob|G8brA7G)Qsv`3~j*;8g_Cr$645-u1Q@0Kn6psMK3u>e}}p z_%%RziK|NZp_t0ds^p3~{TP39DB(ldp*=V+)8)EV__`VBLJCfPoR5Atx)X6}H%3fOVmZ z87LQZ0PM6RBr#VWJirLY#l$xy@|97>SM;P6Gbj#GS!?#5edwreC%vd|x~yRNScWcb ze>K|CiZs~udFG2jiBMFQVj=asSj#-AAcCR(v!NX%A{W@;;YA{|12Z#ZN*lf?djMl2 z`psB?BmQZ|j1;#K8B$*6v7};7i`8TOh=RgVt!qj0I#6x$ARz$m_#$@^)VW%~BuPnr z4U&U;vicyuq=_(VZObk}BNLB$1|=kre-m`p(zl+1Mkx;>-0YHM+59XBHJ_bofw-Y2 z(xfM<4dQEknwNC~5d{Ef4PRIf_aN~+H}LRu&TVQyzwCzhLCEP34e&bL6(KNz4UA~> z^tfAw^e{yn9U}lG{K-n0a${XIq>5q;#1{3I2lMjq^Y}zIktNHbv24RPN|c4#e>wDB zBuul=5d6n`1hSArelMk^0ptS5fuiLS{uy#5#>o@K#=$ilHI18x!!b8HaQb{6IgjW! z_NcO0Rqhxuleuq1HR{#vvJb0Yy{H}sjegW21oiCoi$$tB*Nuu}#5^kzGb4_eA)Ry_ z^pMQxDf&d>_SLTOgt5KahYZ_7e`svjqimU_n^G(tWVtx-L4WhCX&#=4PE4Nil@D0p zDjC&j;U$o!)+G4w{Y!r&o8RlEZLBGg0vK!)n(NMpO{{%hfsEu;+5>>J#jEbzb_6E` z=u)e^2~K8zd)f*4{d%j7ka#*b_n)jiLZl4?IhoJ&uGfh4{e+Nl13)7Ff9UNp1)>q) zwZ8k`F9^J!AMNq}8+`$%G=e&j3;uIZhvhak%|aoZa)tNZ!rwy6S9>Zsl5AftEb$ER z!n)%&DsKTt#E;;>HID4)ESsxG zZh#y}^Fq+^E+cNtp|c(@;wUO!G^#st0Yo6Lk|IOzHsjP(<+^Z*e^5$q``*iN3?L+! zEeDc-l*Ht=k}Z0q>$SiQ{B-{BaZ=?L48{9yg4W0l0G4gNG_m+v?bYy2_>eDBjKC(m ztspQFAvURSw2u`fkrHw7AjZx24h0o~=S>(O36fxSfDzX?5e91Ub5xQ2!o>Z63ghZ8 z59OtfHmijAP7hxOf1)IVB9!FrY>MIvFvKp7=`QRm^lr%xW61>PL%gHrjIJ}L<_K8K zSFEMvs%ii`svq-E@%GOK!9(NdNH`u&u4qUtoC+M{@fs;28`be5*ij%U@-n~>J$kUu zz-)<#5HQB>Yn13CBTb3U!)6SPDcFMwO-qADk|oQGDZVgTf7t9Ix8j75=sYl~qUi9V zB8&eV%N+Gk(kg84YAEskgs|r>;z!V;!S0A_bciSp&tw!%|1v2|Wa1*^#JLvn)%58b zAmW@{F>yj5Qv8A2*elplG54fPCQJd^LZD4pNdkxC!H3 zDI}QB6vL9df7t0JX7MaLu^=|dPCjY-NWw95(ILZUhs`1iZP8%UEI8%@(C?-7YV{3+jDqx5rqyq7Fe3G0rc25K;{1;vnKS7QTe+C$D>6V9j-SbOzFPUI{0Qy{2{JRQ}f1VUoqAhNmw ze;XuEGe8Rtj3UrRvNee%{yJf;76ZWm<2h0ze=f8IF{0B}@*gp5D7GN}*n*=ZMmoZE zJWkE_7UfJJB1TV--N;lKt0V=$iJf4gM>i1wTI-mGQQ3qq0fg&G!?Yj>AY#2#{U{;_ zD%O@TwjgM2_^xzeKeJ)G6l9~+7gHA9e_rk04x*S|FZYHNW^8HIG1C z5mY=?3Z}@mY+XvGvphs%Zc8<;EoqkVRh00#@$=T{d+yf0PC) z+Llf06G67tYw$I)w!_4v#yiB-YwnS6V@_5>!#w#4J-uV{(*FdJjXfgCBpao-Ij&!$z zUgASvG)7-PZ(##dAXoxmt4~S^e-MwEPL*_&>;e}NOn>p){+=&?lo z=t9tD4%I~+OiT!NYQnZLUiT1OI&WEuBMq>jj%Sk`dCD`W@iuYfjm_$Sx{#VhRXjI1 zh%8XUI)R!6gG9gMn&mZ{`EHwAu7-}pYi2|?t|D^7Q!7k#JP0o|{21?~;wmnhA-uU} z!uewW%Pp3;BAeJge`&R(HAM0z?~K!h@>rTYRk{;i8l{axsER~v`WZOPVQb^>B>sa# zjm$wAa?B{^2FQZC&}cX2e$Fi9F=b9IyB_vrUngM6Z7&625|f~m5#pEXwQyu@}2{o)@geTC1fwOmtcqg8DapE{uUgjm$zE$%9{* z%qUCnsIdWcomHrXI;ksipQk3LACjjF?J|P8vmMJUV0)zNMPxqrVTg+Il%eK;`nA`G zsDv6bRGYPZf7|G2T1I@+9A-MYt^2yMd+vk^8^tqtpSx_PNFA!BOnhVN14oYF<%9kR1bLk!Ae^q-EKkk33%q!6+B6Iluydlth$8*BU zCn}#`bB8T`(>cA{)h6D20oDB5Ho=nZD_r{^_AU>TAN&v2_coPzV+y>9v0AxxVYmW9Y#??48`} z$-WS!{_N2{?bUwm)xPZA{_WvD?yX$xf9byNOIq&n-XGY0@AOiJ?{m7@Cm>0 zx61AjKk?Dn@Vy@J9sltmKk}iT@hQLZE&uXwQ1LZ?^FsvlwO;Z;KlDX^^v}ohP5<;! zfABef^;thOQorX(KlWvR_DTQsZU6Rhzvo+j_jf<{|DE=MKlp{8?R~%ajsN(kSDg2i zpYxGl)rbH2p+EZPz4@uX`mLX&mp}Uv|N1*!`n~`A!5@9NfByW*zx=sM`_W(Q&40?n zzy00+{Yhf|<$wO^A3W0k{)fK)dmR4xzyG~o{{h0Em&Y{-9)CM}tcW9|$dM#Vnmmaz zrOK5oTe^G+Gp5X$CqLS}i8H6pn*e(H{0TIu(4j<&8a;|MsnVrPn>u|8HLBF9RI38r zi8ZU%tz5f${R%d$*s)}H(mabct=hG0+ny~8H?CZaP%nfCA;7BMy?p!n{R=p-;H7g5 z8$OIUvEs#y8-JH|3pujn$&}qXzKmIC(-gz%5&V{CFjWG#pp{`Nz>TO|rt(M-Y9!OG zTa*+R!ZDBa^*vN40H3d476w5yL*2I zEt5|uvR^fVAv8s!SI4;|%|VGhB_l$avv!TtzH_1EZ-1Fb?^I?24oF~u1|Epuft4-D zV1o`$mSBVu4#boo2|Ys{08l^!O$t*9P?~sCg&>e1OF?muK_>b{fH4LQli^PUoa2u% zL5YIUEl8DdkbC2$SkQ_`@o12bLA~NnJqh@PBtte{@Qp#Qf?r*ogz3LrETu&?t~7J`tJ2hcpgD z)deXEc~t%n1OX~gXg#?o&^zx8q$5xQG>Irs!ywdYQI76O5P+i!1gSrgiu!1&LFG}Y za6SF`%{S0oDp0Muiedn+{+#2PK&JR)p+NFLdVh|rLVa4PugUgw>aPT?8dS8YO4{q5 z!TK{SQ+R$0Zn)x(OKw7So{Mg}l9kJDS4-U@P^LaLGEf9P5eZ;a0z?^VREqk;s8N&} zBOC+p`ty!(r=A7?tO9umKnn3~+b;k!)=0nw8f$E?zydw9sT=2m)3k5aG zQVYla^E*Bdly6e)ehqfmVvjwR5bBPhEdP&e!OWK^9};5j}h z{=K9L1PV>zLI{0ADHu$!f&jQ+QB3HMilxwH-}oS)bQikbDGLB`%h^wgWuneeOK1oa zP7ZVkfaXE0CpU{B09FVT7yj#VP=Dkc?0Og@AbP8PT-I zsZ4c9C#VV4sH&wCO$8rQ99EEf7~`ltfgykha4DKHF-3@aCpdXHGJkm$t6M$sBcgcr zj8g_{MGQzn5@2c+7z9gD5+!L7bLd3i?e7`4@YO4f(mcO~GAMct6k;_vDDVC`^sm=L zDNzVuw?$qIhb^?81nde_HxA@EjNPkGa*zZfFw!TD^@pH@f?0tqGMQDat!;0sOgKn& zx4dl$9B@0frZ7Mdlz%G7VLx}do~L6M>21^vZ4M-(*|OS z#B~nKO~Ud|w)`kwzaQ~?SI86Wu!qdfM*bC>0SOR6J=20 zs&~!iPF+kMd1$x0-B5-+l!>$4PRf*rL0UFLl)x#n(UO0X02p9!{>PZ-TV!Rj}RE#}FdZYXk0(wu<;aeEI0-$L=Yv5}hX zn9F?TG=HzT%@d*&D3}bkmjbs2iPSIidK!l~E}1_f?gmP5p_u-N-?GUJ(xnZBQrU$e z3scuGB0;z($Rvu=3<^u@dH3W<`MSSv9815U8?c-`)#il4vz#{4GYtg_5omzJD0i--N;yPwT$Xw16MqN}J(W>XE6m3FD*_Y?8W|Gzic(KJAagZDgLzQUh5Z6-i=$9hm_*( zX)9&pezMgFsnxDPjw_Vh?|wO{O@eZyKNJBJpDKm#zW2Wm{_sCqDgCTb8A1SqB&hlA zKz}#qOv`WqA~1_(1Xz3A5a^oRZ}kFP0F;Sx4-HVj8@{q4O>%YwXBzJIZh0|56qEum zP+U*8di{b=xgvlHBwMQm6wOl)We@@Lq&bh5fC`ugZlHn0!%ZFtg8uM!oYrfq)@uzI zZzPBx!SfA4Z~^59KCCw@~% zg;i*UR|pqN!E~TvdvGLf2$3viHd#sdcm@GcFoX~*(mehVltc<4R&hoU#@AQAh7k3j zVL=fAnMM$ZL{SDIRz#792eEHTCKMBdL)(N$mzE=ur8*6mhHaP-G}VBG(ucMNYkv#z zaz%lKo5+cs=!u_L5lxW?0#qr{HD$vR1Og-vphgg{&{0O9V@c8o z4%8FHa&g+5Uv!A!hgY19x{!&V*uZH5Td7vplFZxh>!U=g-?+JT;K-q zc85dZUP+KIL68L1)fDTLS{Efjp*I-mN09zvS3$u3kVYW^M37n%SQQtUk)qLHpml;C zDU$j~k|k-9Cs}PGsgf(nk}c_yFA0-Ffs!*xlQn6R&J>e5sgpa&lRep!H-8C~K`E3& z`87Uilt+n_NvV{yMU+kHlurqj05U7crJ=$(C(7mT3u>aVeK^>6UeAmv`xtbE%hm$(K!umwySEfhih&NtlIcnEEJ~ ziK&>285M`=n2!mW`^cD+Nq?D@c@>d~nVG4XGiRBd>6wYSnV~6~qlsdlX_}|$mZYhg ztI3+!gqp7jn^)PIvq_t^DL#<)9l1FS)wm7bpn+wu8m}M- z2q_ft(Tq9p8@?fXM4=w+Aqv%5oW~iG>9HQ|aT~b-JhzaWxoH|Vrhgju(To5P8?#Xx zNzotwAr$Z-AK6tD^hq1`iJr|7g;rsk{|TT0DlXcz5P$d-NHT~{kvF7f5c1%3J&^*W z=sW_cYq>ZrNKt-!qHJ(jj1OuQ8Hy^(#v(L=dgJ(rViF`Wa*Z)mkpyO;{xBuu$t5DH zDQY$lUGfts8i#rzpnpB;qd%G;1d1@aLZsjD47_<1IKo*?VQEO=UB^NY5Oa6GvMor{ zNOlu2v^OSEXDrAfD^Hp#0r^8<=B1j#DgNWBEBjLqM0yTxU@aGFbO_@szt|HjMWw*z z6Z2;&dtzsL>MXMIrUPN6WC^5&YN&^b7(zOzRiUIvY7|Xs6o0w*cF2|~#lmL-&^!$V z1(Yy@wF&EICE0s@0b2M`of)QgmbjppV+7m{TQ4upSD@PPo7Zg>KGJmx& ziuW*ZRw-L!JzpnhUF9@GL99%3tT$ycQKK^4hp5pit<&l*N`Y%d@mYu?6bTRnTtJbX z#aZdc1@s62fPX^-Spun`v~xaThz7uP4;nnB_hn=TL%x@D$is+;LL>*+t_Fs$$ki!` z7)S(HG1%iemV+Ld)PkMmsbgrb&13!|%_DbiH53y26BKK(7K5-=L9HF@u^-D3LTY@X zRvanHA~s5n#iKiAAVBZ1gY$AnHS%9mM2}kMf(gTM^nc|&l<*EVL}rk(B_lnzI$%2I!$vo;AcOa690g3x4YRx?7zf+>&# z%rbIlXbLKnN!IgI2L%+T2LLdTA}l0y1ji`vRDTc)I}|VFxHXl%>q->nYel{kQJWjJ zL9tT`#ZZ)|I;QIrMTILzWm7nXTw>L~oMZuP0zI6jw|8 zZ3Zwq-;g-wwM#t(6lv!a{seG}9@VfV%!rJFJJ9eP1i&okkN^S@1n~u56scgf<~L-F zr#v*rbZo~_Qg^`UH)tHcL{VYkbtzYjw0}f_VZXXN50(@THV`Y^i}~fKUh~A6tjVk? zq>X@n0td=4U@!ktL8Wx9pOP>Qcn|l`V|yk?%ky7M!3EpGN4Q4Fj~6gamN-9{1Gp0; zhon+3Jj(;YN4^|~z>LTLHWZyZ6v&2mp9-~W7ZiFHtZ+vVWJeTnhRv*sXFR&e-+v6w znJKbxySb)JG&fR2aeEN;lOo>`6q1`1ZoFq6O2~6)!XvwZ0?`BN+-k4HB(EgTl|s)k z+t2R%&jvk#Hw?A&0{*A^)={sMpuPretL4nkK%?oQV-{`Ag31@+?9m@xnB(k>!$Yh96asP9UStT?!m`rtwu9T$2nqG$_IKVL3OLg9B0Gl5O1syM#1&gntcq3lA89JcKJI z@`N-f0Y4}R4c=}50z*o|<364TLnvEJxPZ6o-b>EpB^kwdWYe3rh1AI0Tv+7^(Oy9E za^h6uWfusDt);^|gy6yjJA<+!w$0@YaCzICfC!$@UYM=7Bpj{}5` zV?re7NQ_|ej(d2?M}g?e&g@kPlXBdV)(Kiffm)3t7y{4(-YXpyKHRuP5dt9D7vFxw zss4}-iJHxh?&)55vB~c3ew6AC?-Nm^^G@&eZkJ3|2OEF>@CmQ*2=DF<@9;3W@DVTZ z6HoCKZ}At8@fol28_)3_@9`fm@eTp%rhHoZmYMpy&-+=C z+CGs4kkZ&xQFLA91{o(BcZaHgo%@T~`_B*k9&y&$1Jw-%1wfSmJ-|3Q5F@Ws{ig*5 zP+-{E`oVg_)&Sf5lo|c&&;Fsf`VHF?dQ@X;=SS#4FXtcu#oI`o>R9}&Jq832!vG8f z(8xdE!GrPy5xj8EUmjBk4=xYXKsE+STh z{+TS^RLfQ%00G8ynyTnfXPJQ^3}ZCO(O-^?0-hbzP}A~d%$YTB<}5SpXV9TVk0xE( z^l8+oRj+2<+VyMLRL7Pz8POk%wQVyx#_0IqH&JEs9DG;iLXKMpp8lE)!5H+dxF z{%i1{s1(Q%BHrtdQVJN8{TVBq_GEH@DKt8U+}Ztm`0?dyj$PmWef;_L@8{p&|9=1j z6cC}={Gq5M#{|i3q}}53?LfF5%ZROa5&#Sue{La&IsgcGr=yH0IS3j7_VQ>T13s*9 zpd6AQLIAVSVz9y6=CkpCMjUg5Z$KV<^zla^gA{T|B8xO~wE_=Z?;t1^bZa2qHvT&3 zuYpo%)ReMD9pjX9PC89G@=iSS)N@Zh z`}FfqK#fZBqzG|Klg8c-GNL3H8X1Nygt+tzO@9O;>mdZwvPeUJN@rAQK?@PBb5v4G z-7!#9Q&n|UR$F!TRaku*w4^cn(v6@fqhttDoFuK3QcDlYBvUdSq7tw}Sz=G42-`FD zS!knmDOPH$wf0(Uv(MBbzFF6nrpWCW}I_Is=#>xNC88P&Ful9MBxNr(~K_l zl~Z^HA|NDx+{nl}5=p*SHvwEs;!KlrAa3J+jcw90-tGw z5=uk?`XEZ^oj8aP1Tm~ea}_(Nkq~kaJ6!d0t49#&y0p%pOtOnG>Z9%D7=REE+V^UI zz6s)j!~KEcY|=~T>}}LjSABKXTQ{x1os@)ur=l4T2Fv)@%LSM4Bye+L z+H~ZTkI8j^=9_o^dFZ3Z33iFoDQj(c@WF;+mNit4SlbeJICO!E{Q2vl6{uc;A2s+3~RH`wQ zt7K&>mk5&-3x2w`3;Fc==3d zM)PJb0nkVvn9IQoB)8<9&eW90z}{f5is~LaSlWSNoeRPVIqMXK!FGs zs7Xel5W$iNAP8JYf-$=W1t>5uBN){vM@z!ejo!13s|f)BxNwz){8A$4-~!h!@Hkd~ z)%2!QF@_Pi#sHoAqaIv<0vDX>R3iu`pi`wPYK{r1B$PxD^8Tm?419x-zvRH5N|jo7 zh*qWXDfK|QQ<6qPAk|2X0BJy$zpiq?BS=I%l?U$OEizvf31X)9H-9{%rip0}lX_I<0=h*AZ8#!8exDN&~v5HN%a{%+LUqtgV1G_ttYwu6i-6{1)yojYL}~~ zKVAWY5Ijz`A{tx>V1OQfNcNZY$ioG=P?zOYm?ikgw3GR8wn4HW<)U^#VKwGn9cxUjOd<<;*g~3|W@IOODKSB6{$e3U5P1>=3XPC7V5~FT zdICVIDg_lwpj;11Fy=UrNi@7hBJ6%;gaMRzg&?ZAoQ$ME6t{pv@wf@?iF`ZU70Cf) z^eygwms(n<&M`qk;8Fnm61U+~e;-cn@eX-FgCtF6P!hm;CMf56yi7Z|qYZSD2sC`y z0MNiqr1mh#2;vrvXoM(;tZbOayeAzML}3XX?Ti>96UN{l@OZ~7rPf2}=UYh%0DI&- zxTV}q5*&FbOdFFxwjTEo`rn3ObcrC{Em)7AF_JFufm3(99wESSpJAb)e^P`1aCrwt zFuju%9XqQ>6NFz0;OszEm+WRoJ8jl361M|gA`oxwnf^+i9$N8L=8I?iOL7oXja)!H z6vH|CnEU6S_vD)=d23F}YB@!pn`IHj_s5^k6Y_K(CsISK=fRFov0qBHP41ir=Vh(;O+?Fx|moi0k(4U7gaD!@d zKlnok1|mRzQmNN#q1cm%d-w)p5QRKAlg#Iv_9F;4poD5u z37u<%Q%V6yX)%^Lo;e!{ix8KLs6b;FsvOiV0~s2BxVP}3ghr5nBwWHK97BR&LJ(O% zQX#m5U_FgEyjB~$e-!MA4fF>>>j9`429lUTD)TWn%m_F%!AO}yosfsaA^=V&b8Z3FC5p2`5f%v84xwBCNfCpNjlaQ*x8VKvi1CvQO;xGW> zvcsvt8)5T@MVT}vfwT&8j509ASbPXxB#2a0!!;xgcl)a9e{z=N5Cq5w#KR*FLWDaW zlnBJ&!!862yu*`Xl!#+wh#XKxk}Hd5bTYaA(1~+^CjtOLop3tEdqj4es+R~IV<-a? zO9B*YG#|_ZD7ch*Fb*P+ugX#!#*u(oEGiePi-iyX^;tLoI2`jcF~3QRJgXG&@C^n* zwG;>e-%tKDkSdCK96>NK$EXm7%mars48utXlZ1hX93Ek$lhVKl$m+en*dK&Y zEoPL%apZ|QRF4$633dDvnHsVeE3CYUg)qx>6ac%-31hIR zmyDCrfQ%p@O0xI{3n?s)%1NSJ$}EdU{)9L=e;O777%M#iOu_7&!jy=^Ov=Q}iAF<5 zCgC(9Q>`(nO43ZHn8*W>BaV3yfXN|@AP_lu@UK=xpN2E09uT}&s57amC5A|>ddP%R z0>Mn+e+W-`pRD^sM7)V_*cSodE1v32-viEy@J;FLOPC~$xf7dus70~)3-5sj#z0JG z+(~HkF<3krpfQCrnZ-WIPNm{b;Q7w*)Xecz%()YVS1<+r)X(H-v>vDn*5ks`98hzb zi4@=hH&8#5T8R)K0tzCC2U0+jC@>cxHJ3<$e-_~zlSqIEA_74(i3x(BlMsL&*q{Z~ zhzr`#3_Zn+z|bVnQ0p9((x8$A;Qku?NjCsU0EDZEb65!5+f1HhOr|7=2oNU4%o7^5 z(GS5<9gVymCAq)Bz??YIkYNi3x&W?;Oak4~Xo^uU{UFnzgwPV7AFDI-B8he>Go5V9 zf5G#Yj0+dPa+t~a6ERhp@j1JsF$OgK1~nzh<-^hJlaa*&fE<8JE1yPu{RdG~El~hW#R7|CcbGQXlC_qGARaaV6S2Z10 zja6B-q*tv~vzb*~%~eRURbCYsUHw&He}yBy>{VhV6#)oVWKGsC8p2|2);f^@WsO#8 zRiavSR%?}$;_(riFraNEpl<%1R&ZsFR~Xi7Em!zpO@Csns?bp$eP%9Sg&{mBbbhe{n$?t!#PdYidEQ)HCc^KSv2~?miWmZ!Pw`4S?7^i=Al`Yy;&|| zS(R|vmqpo__1T#P+L{$woGn@^(%F;X*`6&~rY#_wHCm{JBBa#_rCr)}ZQ6E?TCF9c zsjU(sdD_+i+g>8tuqE5AJzF2@e_D`TTebxww0+y7ZCkmWTP%WGyN#khtlGQXTfXhv zSYbWA^;^LmT*BoOHwfJR!cAPoU0lX(++1pgH2?w#m|O?|f;DIc$IV>L-Q3d{1PB-a z&<$PDg@8feT+=;W)U^s#_*~I#-O*Kr)NNJFh+W!!l{IKx4w!;X*xgN-e*z9@-O{aH zP@*K^E#5%6-OzmkY|sg8cmmN~1LMV{VXTRRpk9BlUhJib>eb%s<=*U_iR}g7?iJtf z^^EZ~-|}6a^HtyU?WFZ}-}Zfp@I_zlH6-L600@BHo><+`4PMlM22U--TbRnDsD}ZL zUIdcS_m$rTo|XnKq6ZG3e+XWQ`DI`4y6zvhnc<3BW9{>X1w&3V{Giq&SCRkW2(dA$<2a_{pRnUPmg76_-Zb{(HwNT67UVuIB-UkNp?G1?g@Dm;n%fe@ zoN#2NkYuH>;UPAX&Cud4E@MrGVovVlCH`bku3}LJWj)SgKHg(ImX%aS2IO9@Wmul&od9N9j^$w9^y{#uRbBhKPcHfK{VWpP$#O(y1HUgmceW}R^7d4}hD zmgk(H=X|#3ea7eM;pcz$=bO;yfJSC!Ht2Xp=zCV^erD)`e^zFJmI#6#Xfr+|)_nq_ za9;e)iHB2;Tab~LBRt}O8YIKApoka(5RZlU$x{`wo^aGu1-7iz;hh?WAlu<*$j@6K zG8;b2Zyu16Sm&T7XLMfY?ImYYK5C+7XDyy+i3Vtjj_5O%>M(|AhmPv2erl@r<%;I& zKelSH?rNu=f9k9z>#+W6v4(1`F6*r(qh<)*7A6X82wlwOi+GYWQSi~$qG_jTggO&| z>C-?nExG=MxPU)20H7fcOlzyx1PV%ihq*Jj@%aW8{j~AW2(kQWpbqDDHVLFI>dsc` zb7tx*2JNO+YqL)4tsd*DR&BQS>Oem0*miB%9^~5Yf8*Q6<=ocg-L~aHLS7C)ir*ao z=(X$J@aFY`s1dzJoisSp^Ads>vtN0bMiYP#;2P=n2}*7c0)PP}6$yBf0EF8)%+~C5 z%?P39?9mqObY^d(?%vNPZ7ODM*tTujzHhY7@7dOG`{r-l#_#GQ(4exYz#A{v9Y5`%7V;(r@)W-7mO zD_`0&0j9+MR7x|6{`HvTMLE7?C|8jR9xp0j4SPx6bu6GNs&*_GgcMyaV9od=2 zaKK0bD4+<|8U`p(6EidTg9ng>$6{qybY>rNh@beVfBKOR_mU_1jmPS$=lHB&f9tK+ z`m4A4u21c;Px6p&=mcNz27hoce|JdsFusm?33+v#Q1fuqbDAzyf4DBWnpjwWnEvc) zPJ}R%ag>K|_$=(9X{1kj|5$n_9_M9`e5aRuiZ6R=zjn;ec5E;A&A0f@_xR5@`OwGu zv*-HK2m3$%ZO{$wqZsbu)<*NRf8kkz3=#*rc~5hizx!I(VR%a`j-va1M>+J7h-He1 ztjgA#|2RNre5F_KWOsaNkM=1xZD@yjiKqIoH~o_r`_uRO>_`3XC;RX>`|a2M(dYj1 z2mkb6>p^;J(2eV%m}}6z19fP5glIJNGCvS!b=x;j+`l`3@aInu01W~Kf4)lqz?8pX z00?|b2%v{NfyNY;IjC`C!-wDe!2n@XY zC(oWfe*z6EbSTlHMvo#*s&pySgD9T@Kp}N1RjN>-R?Vsvr9hNgzfR@q&nwukV#$`x zdNyiWwE()l{kazCT)A<@fBLMO_io(1aOv^|{MYVazhnFM)SEak;lqL*H^#fT@MOq_ zCG%9A8FACjo=|(j9^5)N@e^0M|J$uo$+e)pC z7Jhub^sU;zUmvdi{fGObpxI6W>O^2p2I7R^O$yq?U`-Bk1mQ*!YDA&_Q;c24*?<^U zhS`RcZ3g0pmUXCMfgFa|o{B57Xw)F8sREQLs)Zp5Ypw~D1OVe{B#L$bL?Frqw;3Z6 zO$YU`BTw}tS&%=Ce*gf3MiSlPPm#HCaTNho?sVlx{=D>51h*sy02lt;q7YP24Cl`` zYVy{ioO9Ayr=54=xgl2Y$;Xy{YXJyoe}ev*U!fwNi0FYRj_6^cB&O)-qKYQS=%bQW zO6h`@P6}zGCU%PHrkXa$X{MfD3hJq%mP+b_s^%H1tWA+Ne*gs7_=HWZ5UA6MjjiQ$ zk1^0ZtU$-sbI>r9Ec>NK-<(tJtlxqg{;s&=g4?IL=lXdc zq3RmSE`NizOP_^b|S?Ywy4M){F1G6Xxr0!3ZB*VZs70>~O;u zPE2pW1`q5nf4LiToYRb@-O7`$F#6N$ul6)`5(6u<+_C`8E>+J33A9`@0YP+}v(7v7 z-1BQce*+$P57IJK4-`QJo_G&DzTLRv zk3;@;(34YM8(ft`bvfp1ZO(b;nr{v|=Au_lI_0KMj(X6l1OClAx~`uKyScLWNxOWu zTYgyQeC>|=RlaAg`{%-kPWHmRw$#zxrKD2X!3W1qeD+jIZ(?%(GQ zKJZVCfA4wm;u~Lj`lqj-di$%t-+KJ7&mVjJv)^BP{=))fZF^EDOf8tRF(LxsvphnHBi9KeR3nD@>ic*{+ z6{~2)D`GK=I%~!nfB=CnLI4PBXhsGl@x+@@&qhnsA{+kO=*Bm~F^+PaBON`M#!loU z7bmMDAN%OXKLRq4f*j-^?NPm)G|@(9BqSpn>BvVyGLn*<+;WHr0#&2NG;oZ>u>C3!N+PSkLm z>Rcx~+v(16jA{fQkjC~?Bp$c6nLv8rW zo}4qCHZdX+<%YM0Vl<-~-6%&lcF3O4^OQAVVMSm5NVkrnG^HwCDNC7h&0JMao*Idp z10g6u3R>`@E!`mEr#*b zji%LpfyLio^;cN_9oBzb-Ktd=>)6IZHmi@7Y-GL4SjuAdcu(!;PeSUNkJ&1z92r?@ zPP!s_7=v(m0DuJA#F6OyHKX%Mf7-ha^&fcE)?K%??QDydTiUj*Y-|f##k7XC;hv3a zW-BgngR8a-9XGmblP+|p+nA?5^e1|2&8|f2{=8ku3AQl_!A-;)TZZ=bXi6&!(tNvH z-KKYG>jg`D)yv%E3YWSi(r=`Yd*A;$*SOZLE`j$uU;`JpzmIvUHun}3f4xzGWIq9I z3|S;zf(XYZ@-PKM&fC!VvUk2_5%E???3UlU7sT{6aaEeT+yJ}Szc9w{a}jJ~1s@p4 z3H}s+d7NV(8|JSceXy@obDE$)4+1s0$2$b_j9bv;9o`~NMG#VDV+_L>Q{Krd9wIG* z0OuUHxW#jD!dFv>f^Z&wf2WE^jAHw~7r1RM@tfsa=Mdj{#Q=UWpJm+V9;XV>{1P<4 z2u(0QBihgkOZ0W0`ck2&;|{9vE>5tan*M`sWH(s?fG9Am5ijAU5o!W}05Y)?i9E$1 z1i3(%P%;vWz$7M62||!Ib@&ANv3pMM-^e`60C*@ag2p%oo9 zXO9iqXp^>b869LttC1))`ujz(bXo7xDV%s4?cb!4rar%0e$u$_(o zs)Lev+q#{&ZZV7T6KpyU+s*;U^T737aA6mF$H!K9vKhW?hc}zqi>7vdCH`!SM;qf) z`)gZ=0))~`fhbcbe+>{g0-J|4;u-buw6$PzZb4KlOdZxqmkID?lx)c#@*slCVG@!Y zRC%@O&Nn;v?eBvF+~{HH_tO8JYo-4C2kHT$v49~yagA4-;x*Rsje*?tg@3)_VSo78 zBYthCmQkiasa1uAvI&rxoYc)?xj@V<9^xn@4?MXrN}A=He=5NiHs29QJ#vr&y%HUq zQm?PT7ti#l`=Aheu#eOKP#j(0l0nSb=gN4T{y zzj^C-9{Zmce|_k+KYH$$9_*+Wzv;{WsJ~`6D6JpL>y?~o-Ot`hEx(O<{jrbEPi$KW3FhBuHoXHO=o&q}H5b~BELSPaS zA`xERe-wV-6;d4*lAabe;RYgN20kJbPNEc6;>P{eqXosbbz5nGTkgflYT;WO(jObz zpBv@`-^HN~_Fr^714^jb{so{SA663}!j~W#p)K}WAqHX=G9vgnq6jjfFgoESCSoLZ z;4p&VGAd&;>Pu_|RZuj;r1_F++yOS7Vw_CJe|X5lYV{lpW+VQU8SNEE?MVbhSOiAU z;y3ByuNfjPBH=so;w}2(BHH7gDIqaJqA^OMK2oATS|UJFAuyD z^sI(3wPBplhJ2JFILhD~{zh=PRyo2}nJfnn4&6F-Q#(rDJno`AqGT^N;x8&9KGvg4 zf3D;`&KOJrBTEuw)^JqXX`6)k1ZYvguWXQxFpfa@h?ocokr;_aB*{?X1e2hIK|qO= z@L!dHnm=rsZ6KaVM*b5?Ql3e=V<38^*R zreek;Sf=DHh9%{VC1rxCt2O6 z`wg61cfl*&nvy2ue(5CqKV4jjReB4{1Zklx6Wl#1z? zl4+S1MaL<{K!H&hjgdbj!j=jFmnOn?lF~m>nwjEhp6Y3o>ZEye0-PS{e-^+&A|&W3 zMbw@mYN9G?kFFkhD8m+P>6UIm8b|`2UJpHm>7r`trgEx>on~mLLmEV?oF=G){gaM% zYO1R0s&dr+r-lX`jOwIrXQ|?dGqLKe;%ct$lXRlTpGNAWQfjH55iRLzu^Q{Kf)jv> zMlL{Vq^1FamZ~JwP%xn?f3jNZwPNcnRZnb&1~!PQv({+!kdi>yYPOo|xuPp5Q7Ndx z>aW7Vjs8mBY*V_@YrWblBdu#_)IqGm0i}xT1|90X3hclVY#X6zd2~Xva_OM5#*zvI zrWS0(O6$$ zz}^H!l@tWn)yd*)&g$%(xGHF1L%X)DzmAeDh15+9DUr(6&MNKFGA()Z>}S{m&?4!( zCg?d${!dKRgn$l!i{4a>)>YGjZPX<3cGQM#Qg3@$ukz)lWHu%OV=n@S?*oJH_@*!WhGzU$F#KY${A%z8Z!mFAuyJNE z2V=~d<*Uz1f32Vrsu^W2M?e@)_5_7J4i7@XFnFT@2xbAZ$N^Jd1Mjf*e&sDfa1cL& z1g|d$cQE=EF%es^2Pbj)2B-NpalV!CXQ;!zg6rjWLeH9vk)a0MaRie&8I$OAx@32n6)@fFDsHJl9OqB+=4TZOT zVjDumxb0uv^_+YR9lN26-sK?lk{Sa%g{kcXY{=e0=!h#DGkYZSA0qQY3$ikE?=wfR z6UQV?wq#jCF(hmAH)}LEZ?rda^hZOdhmkXRe^jodYHSMU0`c0!7lxrwjA0pu8_KDi z3)V9Lz#N#QL=M79Jvik*&VlAbijE|VzR#QM3N7l+%+ zf5e|F+ntiIqTWe_RCWXa^f7o8#65h2QRgw5po9u%tD}4; zrE{wXWiKXG7kG3#G!S1mc5gTNHuwjJwpfq$ghx1Ohc{ZMwukX5dQ`E(KC9X)e^MxO zt3SB+Pr$b-1Grv)V}0x1eFL|CqXYwlz>L%Q1^hNnG)I50mO%@+c@X$+R(Ft7_mJ<} zf;0Gqd-g?NGKFWigCjX=KRI`gw9ksJ#G_&ocA8UlB3 z-ynW#3ycfEU%mrU-$W3|glqXje{&Rbk3YA8138fw@^t@juNirTGx>u@ag;mxlykM9 zD|(aDrHWd4dPqXE;;$rlsg<+0KX76@d!m>hgeXoKY|uA~ySRCI+Wtfi<#981PCy9& zG|Ma6G@j=YO)*Iwq2%h>`}WvOk{0jdqi&g z%#nIwzcNbD%s=-;cmRMuf1`vTuseFV`vUiHyfd@OJ9DnvJHM}dqPIM-_xrxP{I1`7 zT9a_Lp~uR)E413>3U5RM4FL1bm1?9QLG$%N6eUUo$&h?RkvRUfQYwj313;=@i7sNez2~>Rzf1V$H=0O0?0NjGq-(5~(kheU{6-+J;JYxm$PCTU=E~em*l-b zx$o2ZV%EFd*FEu%e?HwGzwukR=r6y_&ppwNv}km=NxSU4`g~EzjnKpX=WdMQAP#u` z&En|3ii|D*ng3MfE#K_26PZ9OV<$U9609 zkaX%DA#6(jAHV@bshKr^fB-}Y zfLNoM22DH>#S~Rsk;N8Wd=bVN9h^utW#DMYAqpkZ@Hu~!aZbh{g&dN|B8@x}$t0DO z@j=wEfy1~h(x_v{$}pP9o+Y*1lFKf={1VJC#Z*$m`kbsI$}Q@U?81RGv}nvY<(!kw zI_wY;(dPa(L`{+?b-WZv=2|5c)>vhomDXBqP0vs`QR_rPDW|khL-wweB+FZk zJr>zym0gzEm~vH(R~*wwu1YN3oJiMZwcVE6ZoU0>&SxRKv7u>y6WLMHLhl2Y-FDr5 z7v6XST(3rQk1&@_Q?tcX-hTc47vO*eK8)UR8G1-fi6k+#M|BTHsKXCK9I?cKExs7z zj5Sua;4@XFk}^}-^T!^33CeFj{!H7L<(6H38D=@J9GNj}0DXwUIw&(#5>9Y zF3R!Bn{d4`x12A|{W8n*z(W7K@6iXp8z-_Elcd+kBvH73<7*|EqPpu+3c4e%QQzIL z(tY3j_rFXpJb2KH4}5sdg+Kmq<`Z{*ap)VD{&DIfw|;WoX(|)iUd@Gl-NYoZqq;s* z!bZOVYzI&f00hw5V0!oihzP$eKti3m|b!i1O*At*eE3Jt=-g1C?%FdT>s zmr}zE{)Pp9dmM?ew)KpApqEcq?4eglM}6cLARON)#}LwSj&>X&9`C5f z67q46emo%{|JXzwN}`B}$pyK5RR^6_L|+DpgGbia5>-zAQ)*WMq3hckb+DhE_bQR74q_zzHA{dhbhb#67!hG zj3F|Am#NGdGV__vv>{pAIL0)B@+DF`jg1Zl5uJt1Pz@W1ex4wcxk2C{_jrduo^gwF z`2OP^^(e?Y3d9)0Forw_B9C?YBcH|?$T@~F3}_tW8wGs^Oxh=uPhwM6q8#Nb z3z*7QQZ$OOoTx^p$k8f#k(OY*r8KQc%{W?rvzjKYBT7%I(mb+sr7rCwOkXO~K+<%k zHZ3GhZ>rN=@^q&@?Ilou3Q?A*ZJSW4lzc7~JAX98BZ*`QHat=>g5+R{TmXtF+NS}6 z7y|$V_=Z1*5!OZ|ASeRJgh1YbRzZXyC=qyu8-&t>d-O92LWzJVT++~>Jk+S9l4wVN zEhBfo_aQ@p#7;(gGtn(lD7Vs zrY$XMld0O&vbLG7Z7po0DcjfvRwYQS+0CRR5erGe9~3E;K&r+8nrN+VR~-sh0T2R$ zlvS)`MT!6f+E%ONfvph$iU1f8l(n*dM4j@0h5?p{TUZVDqmLylR$A%G#+vfIsATU& z;|t5kN>+j+wQX%_TiG*`RKNW#sc272;ME%VwFsWAf@3S-05h1s5ccmi9V}sN>J~9J z>!#}zrVecap(9hE3GD^|1cPAJtk0-NSDzc!v6dAmf{-U&0bmMvqIHS&s7DWffILj$Y;;Hyi2NWJV&#ah1j zw(qM|t?!ERo7Vo$=&W506kCHi*Sm%@vX7nFVly1#4o~)GA1?8RQ+(nx{l( zen9#PJ>N#JGSP=_@Pj*laN!n@IL0sD*@{zr>Q>MAXRogHs&jqoJN5ZMI)0dF<%p@5 zvJ8FxJb-G$1m!0#@6Es6=yVH&+&8Xq-ER(K2800cfxp1HSyJz@I|}Hetna@Q?_Q+u zyYUzeI@6Q>bf_o1%?ii-oHeiO%zvKqpzk`^zaHAroBo=Ix6i|WcsQX7P3BO3_6LXu zAmW*j*u>gid2~A1bK&&m&|k3BVOrQj{f}O2Tpm&M}N!M zXSMcW4FJnOBI;O|`uN3Pey?Yp^yo)@=|>xS_n*G~Vt&5r5Yq`p>PQm5{n^3>QrzQq zB)Ni*Ao2hJB!HiP`~j=L&E^Df=J-zCGR+^L!O^}d)JDzuo{v$c@9`Lq`}(Z|1MUNh zuDv8r@x1Kd!teaZF9ppn{Zz08S#bT-FaE-A2HCH`;BTnrk498ukq!p`_-tkd0;)Pf zs#b!kI>M?1a3CmQ0_A}Ke{TWtPUeQs0sTSIU;rq-%NWpqD#7ua0IDu z^287gL$K+*&+<_41y>Lb-%t)?&;{ub25V3TZ;%G{&;}1o{#5>t{tknimJ0ST1c>~B zoXm;PbdMk;uqYJp?htSa_pavduB+}U-T)v0x+@D|!tlTlQPQvsH?aCd@#w~|6xFcu z+Hmu5&=2>2@D=+o!CY|`VX+ovalvRY7j1DDtLga;F{w6;O|ZouWN$M%qLNtRk^t=> z^6JqDLKFxfu0-Jiq6@4H;GG77#~$q+v`f_RDXg;53b$<_py2|9!pCBw8v)=MK`|6n z@%ly)70(bJ$?z0Sap|1y>Dc1suw zQ6uiG?TE-FjtJXc;vS3v8j|lYj6oD$L7?2xiRv*Giq!`&v>g@6jb4ZysM#A5X9qf0Ea3QYeEGCH!LZfCwHFG9j}E+R-aB5O^+gc`(y> zG!uESr*b^=axzmiH*+*QlQWy=G@mCmqh~dLr>8X^=lNu>X+W;G=A&ndkr)Z$Z9bw1 zN1`MGuriwnG&^TFKgTpn6E%@jHIqIYV>C)5INQ@b-xEH&!#mB+dm=(Oh;dkkt0~h*Xs|6A;}bxC z15`i<6gB44FwSmTsN_O=#4iN`Wm2X;2vkBRltL@iCd%$Wy+=WV#Z~w(AtEM3M1({v zltfF^L{HQoFtjl2ubTv=O;R*`P*g@|ltwL7MG0dNAHqd}2pDVBM}HJZEAvJNBS%-| zJ`t%yvn5EElu4Vk&NO8QAp}jHCNTMbWk{VAOS4oHT_AcQzFXeVG;j%65R=^+% zX?enC(f~3f1ALwq5%~iunQe&D!}o|H1P-ukYvXIXFKksZa>J4=&vq@{GA`-%E%kD9 zH8*qBwrxT8ZACY3Nq264HSYQLb|>=GZ;_)AgM(O(m6Bk>Dc?>!tZRvkAro+nG8=a` z9@hgW7i`H=c)QPXWwIwfmu*LPc}utcc~AFw>y~qs_aCcwZLK#4pN}b1gLVHFZ4gOn z5-}#uDV_eQo!%*)=Bb|8YJCHWas3IO^l6|9O56$}p@zXDSwa+lV$7bj%K&}XamUsS z(Ncg(ktK;2fyMTCJCA`yk1f$QbfdR(O;>t7_im^6f-P9T!0=87<4|FhPqD?%hHwB1 z7bT8x0FqD&%Yew3%iF*a63;5FzRSAaDz4_LuI@@8cr2`bEXaloD2!|vkc|0I;+_V=7^Lxq`PhgH_$lG?C5{0jht8m>c?cLyQt6t(6u<(`v zS~dha9t|3Dd0L4PdLN^?qJum3!>a9*&+cri;jXJ!`l}&ujiXSq$-4MX+V28S@a#At z&Oy6>%*w5QH?HaWj|JJZ3mT}^60a9pp`E$4VVj|=IJTj=wxv05Au6#2gPZ=g5T$J~ zKzYPEBKK5ovej51d=LH-Va}^D8~Dc2tI7)b=4qEvA_af~3f6bD-&(GB8nwThiNQPj zn7N_5pd%gMky$!p);ro(*A?jBVLurpvGL*w2mV=CC!i*VlAPDe( zrEs~8+qt8V#T=0`aZF{!!ny3fcsNI{e>3hUY z+`LQturW|viF8eTTREPCPe|B=69R=*ShAO}37_!58ITIg`lS8ArnQg@tGgMu>J?0X zLCKZ8ttVVHD*Plz%T=7qzkJKX+{?v0%fWoi&wSU6s>ONx z#VI5?loB^FM0{cXf_&u;upE&ndc3)RJhP!26N6$Cy^AHT8!7hK$qxg{517h-TEj7X z!|(dTUAx0^JH*u+&9NNQE1ks4ytcr9aLs+9d*g#L)b7sCvubY#la9Q1qj4ImajdRU z9Q|P%;mI3)IUF&~q|Ff>=S?N*A+(DE8xK7-5}km%+rk^Y%2krQX*<$o`?Wp%*(v?D zhkDxmI^jJ1V^1U0DP+`Xgeixp#t8x=Qx3d1GJgvLBt=r#yMx$`+1R6e(d*iO%D*hr zt6jwFz1g9C(y2Y)rM=&83Jg_OC%QL0Ekrr$gL|CPB8`x0snRO765_dX-AVi16&=|b zJ&-S+kU6}({xu!NJ-*91{nA0+%thYh@jdebK1NpuJl%9WX%$Z1=42BBF;#Xq7}GH$ zGcqT0;&Xaqhqh*m_GX1XXNi7)XLYt`k^X0W)@7LL=zUis{>5cYiR~|7~ zhe|KRA}G=yb~A364Pb$@=flKcmB;CWcI|3SsVzKwTGW-%*sUt#1-}PS~NK3ybtlC8%1ndg}L?>oMVW0PVA4O%KCukq> z*-qen-~RZIe?m0`RT(1}rqoE&WmAw}`lla7ciShF)^DFbH}9MJyWjia)B2OP&CT^l z;pFzeAN|wcE5n~BltE7%6SPksW_#72{_DRlH@artU*tG+Fp<>${~sXymu;2@B^%V3 zOOhmIBNbvxlx9nsG>PCy>SR*Vz><=3rISdk9e6C7+ckM@c@pe%6kMBXyAbeCaB9(3nUaGk<+X z&>G=EFyKP|g;(ODpoU73ply<>rm2aZ>1LX#F4^j!tHv4YoU_(h>z%jend_dr_Sx&7 zzXnR`tECPbY@x><`ew32k2LJ?!w@@M ztV9z>R53&sKa{aU8#mN3Lmw{`vO*&#R5C&*AC$5|D;Lxb#b?`Gw#I3jy*9^fv;DTmal<_~$aT|wx5#!@Run&CC3}^wHPcrrTtzJvw8F2c^v7 zKIbP%d>H=s1Vua|ZO}dM2=vS?=lJtZJq5iZ&p`GIWXv$e&_9qo?f&z}{z4~^a|~k` z&^X994)~0SlmP$)XrMq|kt9=W?}AV%o#|vpFxJ72gHog221Q6U5`S9lYqtxV?y8r) z>&Y#9*mI%XVE95A>Wzjow4vW{ctafuj)yz+q2Yk|Lm~2xh(k0Y@Q`>!3?e2+L)i-B zyfVHVH6uJkdk8kdGn|6t003p+LXt!w2mok+Aeke;H~cXS2sR=CN)kXO1o93z3L*qb zioi2Gm!uxt<9|t5Qhx+Q@en-rfFSjNqXnJF$fRhngdVJ&39&XwtyPk1m-JdBM@Xd< zs&IPNyIBoInL|?c(3C+$Wf4`GL{=VAiNs^#5@CrvS;mr<%(MPwEpZ7wUFMRP)bnL8 z8+jKfY6X0N!OCYaH2@A+WJCiXk{6joMgW9>AZ=758>0j|`G3KYOH7kr1k}XHCbQ=a^k z=GCry-RfLK3KYL&%SJbX(SI&ikxd*8rk}gTKx&Fj2!G}@Np8R&9GM1G7~C{TjhL(% z@9_<&a)BVXNYX1H@(uzd367x^tZYS+R!ojnt!D*mZ8e!&vhFsYx3z7U*eW)@8aJ?T z#cN&v%G~8T*RRhVWpklRUFk-5hO>1GPO<=dK!m^Olg(tS4mO%eqZVrs#|}V!1o@&; zMQc-m;B=b=NZ@}!zE}ot0$`o*6$oW@+6ZX)7aExZ03jbz4;WlvNs`U3f;aNpeKz=? zy~XW=I~ZZ%PB?_Y_9uUZt$aFFV|ch_e#DvGjOJR?`qp%Qvz_bQ$~o^^*S)^;D*-F&s_gkuGqR6N z#Agqjyg;Qabtz?O84D#r#2eR{U+ z!R)1_&3xvNrcIEmO2B{q_+qxd9r8yj(%V0V24>NyT801>3e&v6A0JM0dcPXqt~NE! zZLV{Bqt)lS1wgE2?P0xsxah<-y3&)*^n^3~;0|?q)Ef@bx7OS#u^|eiC~o7y#ODrJ zq{x33!P7-YQ`;YuSh+l|+Grt{oZBaN@4D4(({$Zee{cl#LSh=ub1)U=SEu>TPaSWd z`_}LhM!ba;zvXwsy3GA1IKU?_`N}JN;FmA>(-%Jeb*NXJ>Jqhj;Fkg;tz*k)H9<*^ z!E?Ter0*a^9*D}FG;*^S8RbdVz#jt)fCPV(GfC-A2=4`OV61j}^z0tI@)b$qad5(eCFG`ARAMP;znl#)(fSV2Oj|BvQEn zFt#9p%q&4f;o2k(@&ekFIR5AeADNAh1Y=~2z|>pbdi^NlxKpLw|SVSf|}QPoQHic=zTE=e&07t8h8}yr#L<1 ze$nw6Lc?7NVKnCz5%*vW&@gU0VGN@13Irq({=*;&qF@RncY_<~fzl^}R(Nw+*mEHm zW+NDDCCGv&D26HMf-7i-EeL~WsD^)OD1&S0hHZF$R9F-_=shf9A21<4X3-zu;t)bu zE^AQ_7vOsgP>6b2xe&h=pBvZ(FEu(WiwSNP^Z!esMK};%9!Fh>6NogXQOm zVTgufNQytSh(uuz-2)Wg^AZ%2AL8Q=jp7f?bY7E#5S;=ZpOO~!00lun1i*h7jKSCg z`iF{wXDrZ?jL5i*h!QQZqKwT5jmsj9tm2Ht5{=9_joDa@)F><6NGsoHE6`$$GiDOj z0zNg;br@0;_CSl`;UfqUB;p}WpjteP{4VjO|Vvj(via&orQm(iygAr&M=?{@39vuQAAp%XFB9SCnk|ueQC}|fI z=@S-d9VV5LmLd?j7$5RGzkIs$CFIi zlur4SPxF*%of*_VF#m+w@UJ&~3$(KuKs5ZM8kin*AK*_cMMm&hiR zhRKeyxRQ=}nV6ZGn%Nu?_Y+&U1(E(z78@y;n^~Hsd77vx8J+19TW1}C0Tx*SnW;IO zv{{?B`4+3$6I(|i=@@^LMtPgS8Jxm7oYuk}>vxB*C>S{*5UOYrhe4ds8J*HOofnaH zJA<6yr62O)53)&}-ua#2Ii0`h6Rl_;lcJq5Vx8f+p6uD4vpJqUaTPBC6GXX{TKSXy zup7Pc8^KX_?)jep8lX`LpFGi$=h+D&1)E1nlbN!M0{Wm38livoNT54GQa-bmBDEKB zfRqe+G`A?BAR3|~%5KfMl_Rwo7m5|e=?|(XB;fIpAR;0pGNLqEqc%EQmRX?FF(Xvu z3!Xs`7{_`*D39Smk2gAH0*NgCNvbRed5{X}qznnA4k@J&Nu~SvkXLG@{fMPos-^$P zrC-{Nnwg_Jp^AUWMn#D!5VdHNoI)-RdZgL4q|G9y&jO|aX{Q5ekb2srd-|k(8l`_a zrGZ+dgPNs<+NFmYrpq#><|rL!iXHn=e?#MkZ+?6hn=XZp9rhT6_=DbV~>BTqK*245Fvzyh8WP`TH2y1v^-8MGYO3QGt+$`cPu-vo|avG7 zvx(=d)pxU^__MMaw0S$U2V1l{+p|JjtAQJ^a>%e72_Y=;irW$&80SnC0Zm3aWYknf z)+Tq1<|N$&RNzEMJ0aRs>M)A6~c>%X`3$t;nx~Z$Lb!)S*d#ZP9xBg1I zK5KuwgNwI+tG6$fncJmxi7OK(g&WT{04r4y823`sBvUk{WGJUo0KilJ;8XSmMnhFp z5Oh>YwNy#sR8XaN3{g}OP!Dcrx`=_gizm0~%ew39y8NcQL<_ikE4cGZxbnNZe4DV| zs<8aKzy4cGnwgtCQJ%tUjwUf^6H#c8B>;a3Mt4AVSpZOsK*d>bBxOq$TXaWSriEH9 z*AR`6B<)hZ=8G5T+j#72!ZLfp%~!LsTf4E_uebZcI7`3yOT+QYyEmM_{R_aEDZo3? zihee{G=abpp}_vr^nechybvT`k_&gBdt^VxU;hPQW7H4<=2U7s!X(_bIVXuF+xLd#Q+qd_d!#1qOIPAtc{J(yCiaord-h-k%SVfXR#1Ub{w)bO7ys|{r zZI=6Olg4BXAprnz0rl`lsTE&ZoETsHd?{SX_iD*MN5(U3zi+I+2)n~`499Kk$)QZj zYb?r~%tCSL!zm%3;bU=oY&qpsvD<%^XwF;6P+ZG|oXZZZQKQXMs-Az>y3u zT+DM`oXnuQ%wY`1Sg6UfE5p;=!ZB>kW=zd4jLm3_Tmk%}QH00ps0>$8yvIus$;)w~ zRk{9vZI>nf$R-B>kv4bXhJ^A{ZV7R2RbphvOc#^H#dx&< zpnwZ8#nA8#nGjh_+|T~Z%>R6am>jbN9dy#X%?a($X*|(!{K;}G(GlIrOMTR(+|;)U zmliz|7|o-4=ns+rcHjZE;v#>wJw^~|7s39pcJRu}Z+CmRoO=Rccjr`rQH9SAkxdzh z({i!Xlo-$e-P5b zx}4II{x(`J-Fu%Kd~uz{bR8FV9jbcWx<9?wV{Fh3ZPk47*%r}+HW!1&WzVQecGm7(64*jMorsD-QG`K+pX>1vMt}QP2aIy-|x-CnAx0; z5)|ZmAL{52d*Q2EBs70SXoU7W5=zK~;GG)c{kA$C;m<74=>5KhjoTN#-nKo|texS3 z9pCvK;x?F>WIC>q0^s7K4qw-Y4e^JTIu(LQh=*v1iP+$5JK=e4-tDX7VqD>U4dU}H z;{Ny@=Tbxx>vuIG8KE1Oy34ja5xu90C; zivs|U2SKD8H|E=O=GU00cj~2)4(WNi=Z0$Oh>Gcos_B!?>6PB+nGWikF6y05>R|Hc zQI5E-iIpf}Dd~Uuq8<{XBB_gvUhB4g>ve&&BGCyfah0hj;NU!y9qOUDUhKwx>{X%b zA;Igbo|kp#9mB4a$Ug1VUhPMr>>yDRc&LurL6Q3DmHlb$;y&)=p0tO1$JQ~Kjf(#5 z<=*b@{_YdO?j9kZp?R3xX~*!M@A|&)4#CPJaTWKGjx~Ql80yLI27mAe@9!c})#(UD zvxx8zAMx+5@FBtQH)fpjF7X<_@iv4;0XG9aIh$qLHR2 zAM-LF$-6BQ?J4-z((OklBV$)ugKq3EyadD{v#}E6>FZ<80{L#<+#$Wx&Z!4uq^cw;2 z=6Vv?J0ECeivBjK8Qs>Txq6q#ZNZ=b!qD74!MVeIUQl?FvG6aA^{_0e!Ri`%e zN6_lls#vvh1?x2`SgvBpmOc3OY}d47*HV8K`$+Cwx^?G@&4?GH-iv%K`mG3y zB^r*17^32bj2${&2w9=zgp{Eg%-c5eQ_h_|e+CWFWlhi~;lN3>GMCFZG6$&2sLJ)G z&`kvakU&JY?jGlA{{|jh_;BLF)xMQ#7Wvh-%b6Q=p1gT<<<6@wk84*uUCWrW9_)Xs zd9V2J-phaAjDEcNzvSDq59~fZ{K4|;(=Tj4KmNn|`}0o>yy6BVu)wB9s|}lzP^&|d zPI3v!wI?=uLM8|ty5}8%p1H-hcj{5-9eEHEQ6OW6F$P40#9F}*C4%>B$v6U{Z%{Ey8y-Ha2?00CT#&LQ{Y(?3bd3{pJV3{1&$?u{=VM;3Pva(tK?v%4mJ@dq~v1h4`7Fs*8 z-PYP_AG_AuY`L9Q+-|}BHnMQh9ar33--VYmlUPbn5<%?^Nx}XK2jHM14rE%RQJ?^r z6o3#C+BDNm2PQ}W7)NC|Vjh1*ji6x$ehQVMQ3-g(ltvo4C0>z7Cb{JBkp0!yUe8LI z<(0{58RnP6YHQgl*Ht&&$#C9T=gN5g+2_mJW%t~ii9Xuso{?T!>7bc@+UcRC_KxZ+ z%Pkt~qfO?z>x<@9rZ#mXNhFPvLK^!8nWpGy3YmZqnA2zKp%h_EhgyFa06{=h6#!Zx zezcx?dI&1kXFN?ophzdy)BpesXi=hbJRLX!7`?_kbIpI;RpnxVJr>H(Jr7;j(uqCY zSb}WU&RL^0+F?)6YS*E*J@?zM_C5IZc*lMIc-?>BJo#F|{*7$28+=I; zoM_`{6AJ|vAY_5+Mks$_ya{rHL{EE!7>4c%Y6Kxl@A<}YE)u$hpjRl$n}Hk%2&zWq z_vgQVr8;-n=mMBJ&;d|p0z4oANoPQs8H9Dr0$T5k_q*L4?{&bNpzyk+!7x!Ug2mfl z21OXc^Z3t%xnf>zoJY2Nbz*uQv0hFT;;6)_4*-=qpTaPuKY@QZsto>sSjIGUHyc)J zVU2)B5}|P^g3%9s0;xwC2-1TSR?&)A><`Ljh86@iP=NzvAQ!)wz%Xt_Sr2?f1wD8{ zHFEGwZfp}A=OjmI(NSA#BpwDyxFsz1@kjH@Mf1vr{x6ZZP$tydNDMDD!$&2Adjj#^ z5Hp1k9|F-+w1R&m5w~~4AxaDY@0-XylE_HN`O%c8++P{35;HHVk}X{nqbuREMKjWJ zl{G`-T|TJC4%(4izBCseEttnPjbVN#Mq*`yH(!hjI8)14O(Vn%Kw&S!tt)1JAK<#Wc^O1GdcpVEP4 zEdzSXfxhxPxzr^acUjCmCbOZ-eCQmRS;vXq5tv!KXwvX_PmVq%nw^-YB&vB$WvFcc zY~zUA1^~BiI?`YXc?A(8Nl9{EGKc8&N8L6hfc_jMyO4p%&-Zmq;)eX&h@$ICkeRNb}Ir6 z{1M~=KM6sr8a!hf$0xq*wJ&LNY+BI5S9Cje@9OfqP$DbT$iyr%l8-!0CJWQaYXzZz z3oK=~*0{zYk;HFI)1bQgqZ3qc4QwzXn?rv(WJlpkh$vbNfCz}<0-3s$0epiH!iDc0 z6`P-lFg53#0^qXHc=1mN5G#v3L%TrXf{d-aXhyf!sg9QFd~rPK9!t8?^%ZhzliXw| z^OC@1MKD?wtkzMpmCCjaHK|iQv_?1n7|U8dNrRy050l`8CdjolM($w@X!O)6#t?tS zD=wE2XxzdSuQ+V0mEB-U-}ut^b#}CWOzEwrannyOb(BAi>Qf)M+X-H^w)s2YaZh>M z>{@nuUtJrMq=bbpQIH*6b8Cv+B+Ry|#|2Q#0DkwI-vkIkx&_YYW>5RsIxe{B67Em@ zYI@1o4!5Vv4eD=;I^z2M@5H%%Zg78B{AmN{Etc(3uX|w`8)K8izGfokh^PW50~o}u z>H&oy?8fH2K{Lp8u3k^}{3izSxhH`h^oR|e=mRtQ(SQCOr6Zl`Om)-Ggp?m4muKKl~p7N-NJnP>s`?#mx^|ZG;YAAA-ucgme)Oe3ed>Q#Un9KtjPJ2*SARf=UM?qo_~*@X>zCjB=SP3~)xUl} zV&5|%FkJ;G{F-@!4y=%^zwf|& zafv_*LVB6QKJ>#s1jK(p#FaYq43n@X2Z1nwpg%!m#71<)M}$PB@IrPY8$3+JF1*A@ z)Wl8X#7^YIbpsT#={uJYK^YVYPejF3RK-;cLr~PakQhZ0+(SKVzz2-L39LX?)Wu!o z#a{eBSL{7lgs>U(hmZcUKk=Kl=JUm6WX5K6#=8SX&pZsZz4s2Bhc05?!Hmt3@kt14+zxP#*{v1GVv z!#HxwxV2Qvwlp{Xjq^5+vpA14HMvwZx)ir&qDKM@DV2X<2`sZUseA~kTqLha0fxDN z!Sc!)1Iw#2%dj-K$Gj@EytYnD%eQ1p%ydhNv&*`K%Z}^Jyxh!Lsd^pRj%xsgm%;e0^oXgNm z&e4oZ=DdGP=d>DLJWbAkkO~6{4AY2xDa?e(kPYb&AW}>q8WB-B&lgcl7BLYNiIExM zp@g`RVeqwzC?7?WEZnp*;Os2n?9a#q(6J;=B_mKyL(a=wPUR#`1&vMy1x@MnOb2bw z(xk`hd1fOv`sqq+cpcm`1r1)}QF{p8I^ z!?b@qeJX_GQ%LJm9y3yib5aCd&?%iz37t*|70oC;R7Y)4L|xP>l}AGCjCp|`Y%nR5 z+6a|mDG&>aMwpvNI;T$38@?fkzoDEm5FEn66vIIr#UThZ9R|nwE{UL_@(K#=xl?BQ zQ{U86X#N{g$Q;We9kL}=(gGz^>TFVGMb>{pZB|BoQYnR&dbCu|h^F78O8v_SMZz9l zCDlk;n4E(c^6?b&$t3eCBKC0~_<^j6&{Y6PfHmXQUQI9l1kz!3OnnVbgBw)id{$>w zR)8H?CJk6=HPlFj)UBb%YxSs*fF77Avu%Y&l+gwsz?%iJR?OvaKUVm-l^)`n|)jBZD0((U<&473yxwZo?ztsVguIP zJtkH@o=oXo;_$6rL7rYF2HyzQ;44;SDPCkMW>|V0Nv8mb3iF57e2v$fScNzlWGiOmVs>O>Cgy)*R_0`8<}BvLNIsB|=rTi!N?iNfP~x@vWC~ytwvDKVTNnmW zh)-I^V_Y_3T{c@kR$yLc;6u(}VMgXej%RpYW_hk>dTwTXwqVpda1YYYwVo3&h! zHw!6Y!+SZa6q0<~H-GClfFtJ_v*q2)VFVS~?>p$Fn3uXz2a`}a2vBH+SV@?ZIhwmUo9lrej_BU<$<_Pm)-%eP9O{}Z>YF_3 zoJ{JSTYqePqmF8%o@%A8YSFVt3w4TlF}Y|`$H9}zfb7zq2E(A9J^r(!X}&qK<2zermkFYoXrjvqs3QMn^1_=6u0hNf1#%(2JZKoJ)Y1-}F2JYY%ZW{b;rs#j|1U%Z|R_^6yZY)gh&j>*g z+~Ve@?&`Mg{R82raETa%=Ii$E?*{LI%Wg>?##03C@J8?SR&OgC?`W>ZktpuGIY?zEuA32$%2)uN4MY z3Mjn9_@;mG4d?Io$jLI+trXr)xa7>)0bm z4Nr4_y*K;(FbTyxJHm`Iye{{c2bThy7I`8y5 zpL5IW^XG2Dknq04uI746=-7k;E^zf%cXcDs=0pDftYUw3g=cZ44%@y%OU`^zuJh%kR%i13ut@%)ewIZyL!&lE`r_H+^V6bKoi z5gWk~l87k);8A|>_igv~Tn_kj9{4}*^^NxRhbQ-YhWMR7_@3uyp?Bt9-(m@);NphMF;vlxuo2mqet zu_ubM2h_<2R z=o?if2v!{&S3aDT^@qiwRa-Owge{J8sjXRfaTDt)7 z)&;+<@HtyUVGj7yf5TrP*X#fBZqkt zD)#IExJ1;z(l_|<;>VLOZ+^C6w0hGY{>xr1;==yy*S|+W|NZ^>@g0BjZw#BTY?Z|q z*?_YJcv*o09++8z2rjr;gA6{{S%eTy_*sP%UKm=27;ZRPha8?aB8erEmy1TqjK^DW zB~^6N04Wv3lS|f3SJQPk4n*Tm07xfLb4FR06ayg`dE^B`J(QG^COY{flu=6Q+g<79 zXJvf#*_UN~R~qJJU|@gprB{DtKJp)$A%gf}X==8~=4o!m31@0@)=B4TcIJs^YkKy{ z=WBij3aD&xDF~o}Q!2VBqX;RJ;))u*FwztjO)(M>#T_SP2n4-CU62Ct7~PFU`IFpC z0z6aDJMTmU1po$|( zFwz>D5|jr364bLJsMHbMQ;$F9XlioMK;;HOPXX}CLp>S+FUA>byfJ}Viiy|9bcNiP zvBMJPv9P=Tl`MaM%d)$!x^uoPbIfRY&vVVVq8ciUxE*IVG)9Ge@?&X$m z4)fuc)3D<0!{D+7NL<1RoH4m`AsajE_3hqkO~%fFeFde}?>@ z`|LMGD=w00S7c-r9oa~yJyMd9+!h@(c^)o~hm&^Lni$C#oB=41R`s}m^BADYRjQHz zLC}9BEMv*HJqm`0ezavR?}$EHs^^cl)MXHfSi~bjl8VMGCNeE~Ol2mMl9Mc#{}M?| zW?oB{*3=D1#-k366tEM!LrEzWqKZ)t!dCTwLJ)|c&ULP{2NJv{Jmc9Y;L)?5@v$d9 z@0rhf{$-#1B#b}l+0TG#XQ2KB#g0J_s!xA~E>xijJ?KLT8qt7G)SngI=SB0GQG9OH zo*n%sL$mWyfdVw7BL&_`3tCc!rc|WzeCb!RIi75y=$j*X!WavJ4Vw<2UoibCP=h*~ zkTw*eD|P5mjY?F#5VfTUO{zqpN>Qm=l&ToDsz$NOQLTEEs~}yfQ<-W~uab1BMU8)| zS;KnPsiu{zC>1JPEh1C#WK&)U+!_a_u!+5HVha1(+5pNK*TE9Du!cP>ViT*_#mbU5 z>WGvi)WOAZ0=BZ1g+MqNtJ%$RwzHo7ENDY3T8wBUr@rA0HnvGXfed1lmW`1EM=RUe z(zdp?y)ABYD<#y92WuSu0f{{Z(u{vK009EVg#ZxN5H{U1x4F)JE_9{__P9{ztYh(qjQ zPG(pV!8NalQ>@|@v$)0IC2_q<>kk;axW+cVF^+S5*tv#>xAx$%j)N@ZArraC*8{R| zP>fv}8@b6&elnD!Y$7DGI^SRG{{xg5+teXG}I+NGl&>#12<>Zhg}sjXHjs9%lNSks!+vSw?o z$I9u4_!m6(0%c|qLfMfpde?8hbybmlRb^+D*;{pXSE2n?TjyHU(k_2%wOyTUSW~;( zsNQx|jopx5pH>o|M2Tq=QpT+Xn`6cZ0(M|PF>%Y5m+b*2Aa988U)mc!xa{S>?=4LA zP1Bi|gk~iTzD$KT6XDO)T*4iGaEL#=UG!GSf_obhbVFp_G2$E9fR~4hi?-io>0>TU zzNLWsd*3M!gyywyMdBiC`anAj`=s~wQ$qz!~cu|4f z3*1_4HTDey06+kO7{*JW0>`9xR^=!MxXZ!rbzi<*-(z1{%?mE^g|EHgLTCHi-A?qQ zgO2W^vpebVt~0JH9mCeeo$h#7kbB_m8Mm0#I|fFOcgSNPj2C~1F@|9b#S26pc#Hhj z&oPW)=)@r3*!&uo1Qmtd0xEmsJ7>=xc7e+rkFDS3>w_8leA{v0kW+iysmXBNx1IO7 z-#zeqAN=17zxc#IKGIPy5JHcKfn1}=*G9tSgK+^WqDamN)7A)}Kt%vb>X9HqV8Lg& zAv--#UgiEFXu*FLN{=Qn?9n@wzgX4NfA;*gf7xq~f3feM^)<9Z z0h*59q0R!TP6NIV+$EpGt>k20ZG9?g&+(}`58}-7)L$Sg9p)sR6GMvpiuPz zfC&x*)!j`)yh8{9Kn^s55kLhDL7@!pp#A|OR%7$oB%o@0xF^c5}_h7Vh}>& z5K3bH5&D1L4^mOmH7|u%90y79kGe4X)hk(cmwB9qaL6+9BctKB5vf;W92_Gd3b7`rr~;;xrE5 zC1Q~u3R-S>;)@6#!JrT+f*?(F-r67xP&7%XpbCFfEJagDj4mbwG;kjS5WyV^z%cA$ zV)dfq3}X%!V7yVC;y&7BO&a1c24FG{ z{-l3FQX@eEeL<-ZCv-qa*qxLQW(@+NDDxWl`$oQ7YwL`ek1- zCF+q|ZcyO>R7yo$$^c|aD0bvae4s#lN=Sd2%1D}{@mM7(rb_6cVhkXX zm0HdmTbia@=A3@U{9g#h{q1gu=ZbdDxz@*~UP;C2FIX}aENx*lup9C-qrYyN+> zCTyaoY|>?J;^tktXI_pAZ_>_qG{e@U8ySJkRE8YHoK5gBhryI%RvwRX#*GRtkKJ@; zLZ}}UxJo^s&Bb8nV0|a-g=dAPW?Qc2PL^jPu4ioO=7+*(Z5riml4yIH=zF3lvvAtd z`9^T$&P06EMq=ea{0{IK=fMnUavpz$@-UC{K#xPjB2=i*e@f_tYG*KVXOem+l2WMb z1>`V=sEUp#UzTWKd2SJP-*5dNe#GJkaI6?!F2G2V##5b%&c7y;q8mUk@DeVCyhN32;YG{A{T&gFC zN~)Ji>6TJym||(Bf@zmd>6fl(q%q%9a@<3B+>AB^8mZB3w9y+;#5b6ND2T$=A!>G7 z=#w@oc;f1XLTZ#!s;64&rgG_04(nfb>ZN`vrW$LoChNN3qZHx>jKm0jD%~ix+mxJA zDzVZkxe~4Zp|0NQqrPRhs^)(_j;DF1Cx`YVuqrFN5^G?(tCcP*vhLG3>HAY_mb^vq@~UQS7vRo5E^a#&Vmk zRis8d7K8EBr70axX`sN?n8BKw#bVpW&RWYREXKy{!oF<7((J?9Y{Y-!{_Mo+Y{l~I z#m=nEA{9PbWNv6g(*agq@l{{_)z%c-$v!RAMr}4S8Dwojp6MB8Nv+myE!SRLZ{`My zwU(Ztgqn7(*`6)hy4ly_M#lLAmK_&zDOcLgE#1~_l(nrF`JLVFE#LNSk&&rx{FsLM zE#Vff;jS3q>V{(-F5`bTuHz<{;0hRIP3z-UuH{}Xc-iZ1tS#ntuIGO4Z9N!yBrfQd zuIZkxW)UrJh$`x~uIs+;V5u%{EH3QUuI=7#EKQom%?s}KuJ8UX0CDc-(kJj1ukjvl z#^9}Z5Zc>nncN~T^hU4rj;`KDN}7#rj5zJ|ZZG$C?^Td)Zrp#t)&9O#)_Sk`o-g_~ zgy2eoc^KLNByFNGEwQC9{nju2YHn><+O*rPq5I+B<+UBU6rpmk(2*|766Nt1=s8Yn{Wx=>lYU*7<1|$3#%Uwvdy5S@P37vCeVM=HIV+){Eh(R!7!8oA5vZ$ z+eQ=T-?(O|6l-!FM`{*x@h9VP73*;*lX4#a>K?c26UwCWUgS*$9w-i8;T@h&P_j=n z9xCE8=25ccL7wCdgywPHL44jYY-U5i3L6WAC>&@dTQVHau^eZzqi*udQL(zFav(D+ zA;+sKzw3WCYic)hb2iWG&7mdueuaIGgc&PD^bNrDmE#+BU{&U6NNyi?cpoYPMfi>1 z7(&JQ4TLI`pnmvv}I^D5G*ITl6V=F*biQAak@J zdoxIj^D5)xIeW!#VqpI|2M0c8DFP@xjbIt>lL>#`Y1O6vU{E;3N`U4{B+o+E#wF_@ zG-I+vM|3Cm>P2hxQS{Ti*zBQCH{7WVIn3)C?;d7Gy!NK7k=bb zhF}=_!x&0O8J58rrXdUY0~>yaKfGZa&Y^-L#7ZbML3jsb_B2@hv_C>LxDIt-7d62> zHC2Bfc2j$_HcRzlBX(3f_EaY}W0Nx?ji%^sBO`ZXEE9|XjADUWWm3FTDw1GKUWY3p z4=hrhN(uxmK4(I}8v$&c8qYIddxc*MV;l=Lx$bt%Su+-Av{EN^I7>EUH#Tt}_hTb> zWG6RsgQ~Ila92R3L`Y>Qg5zh8^*A2qITC-UIx0m|{y2s8HH5@Gawf2(Sl4z}+_sXc zHFqdH%#;>cKj$a3o6xl;1Lsm zF-!YZhglRKe{JHvx}xi`GR$GXEWyo7ErZX_}hqbbxK1e+Fwn+`Ob z<|zjaM4g@>43f~E!u4|gsTzMmMBQXZ1l;~3z>kH&>-oVG_0119lPCPDn|Q>}`ov58 z&;Plq19~)Jr+^PQsfyBzFGQ;T`&Y2)r!R>p+=9%{Jk8fRsbBlnXS>d$I?o%u*cUz7 zd-&L!ec6LDc8>9F0P#8x#1LDnlwhm2vJ$sqy;unRk`jD-XE?U+yitD-{kZr1l|ww? z3wqE4z2H~2mJ_dUbg?o#u;^Ft4wL@qU-0Rp zaOtDI>9;=XzrO0HJ`1mYRbi*1n>1jJtU>T^PicOl60pj8e(;0-0q1b>)BfzsJ`5xO z@+&_LGyn5DKMq6x^h0lkYGWB2N5PzxR7B(hYuk}lsJ)M zMT-|PX4JTmV@Ho4L537Ll4MDfCsC$UxsoMCWs)Q*YdMoR=k*TW57^oK_sef+UUA%`TA#T|`&F~}ru zT#`v7jf9d&DTSo+M=N>6GDj_KXMNHNW9@kFFLgfGB_RN;?33MV?} z7GuJCMU;PY5_;wqXx3a)%>E_^RkF}Dk90CgL|0^VOhu!FbV^C9r1VNlv&3{uO}pgu zOHacb)KNl31+`35&wLV9HCIja&{i7_lqgmH192g2xM3xo4N6W3Y-I>n_8_20 z5CB5hs#ggFdM5z@Oi`BEh&6=P+;M4KndKl|o;F;UXI{Bxa~(z~=Yw`OsON%yCMf8E zhNgef=y#8vH|cwq-nahgf1e&W>Vv0Vxaxmg`ug1&jh z6jQ!Ah;C7ak?0+H{E=rEJnwnOpm%Z)Id^%C8OCmiK1N<-0ABIOm|Je)({|r)iT?Hb z@{YOib6XyK{c7KTIenTh*BoNf??*j<*7tY4b*fkAzy1O6e^v`10S9Qn0%Gle3QT_> z0~g4(214+G5{w}J*d}Pf3S#hlIcY|FD)ta;vBl}ej08j?q0f24(a9gibWPs}}M29OX!}i=zLlwzTAVB~C6~D18 z72U9d_L-l}1n0gm>MedU8>9NpNWXu~b?$*F z8V?u9`mu3?bSxwu56QamY|>|#ke!~u&)XaFTD$SqV90x1GO5CUmo5>i9}EuPGK znM{F-8sU&(u;_c9Q02NU}AV3u2J0g%G32ZVGkDPzc7jc$G zFt(43%1kCR{?Q0cH9FFdo@=Ba0Xax)Qj?H>5V+Mht5C_sL4FGF*+e@WD zo{@qSe4~l~6v#qxsL&K8fgliNAwT!Yy@4$B77lS+$VLW(wgMmry(E7r00e+1flwj@ zY<((uTK@XCmnycVFuiFsA3M#&29BvgZR%2yidm&zma~@itWP@&+Rx%Iw4@!a=?=@1 zs9J=owL8c?$SR6FgcO$i+#X*kK`enZ0;3wmt4P%ffVvLEt)7JI#x4M?-^z_4E!*QVlyUr{xi-qPxCz5Hvh0N;zi_&RXD z6s)fX?+ZuFYF8zzooZDL;szH<6t29rVHxR>%e5<9x5wNG z`xR{6XH=36{5NoxfQkx=iX+2~rRE-~2?}Ut?$pZ6EtR=<8n|$8bEKJ?nVM}gEeCF# zX}KyhHTTX@R?7W&-~Si?^PK15a1Jkc!F4#-Vf?P&=lgvOykZG~lsd4?x*NAo3ZCy- zo!~8e@gn8?i)77PmwxqPn$>ct&r+9@UH`6NT6D*JZga7-+=$@;9k?H-NS6vY^Mtsm!?bsP|4{=*z;^*l?)Us0IMYnWWchpi zm;2s*z4;?#(X{?_y5{>^hR-ti*(v1e@os|RZU)b>B_hwIAwBgc6mf;ORIS3;dthHr z(zG@pjMuG{Y>Y!ZT;rMeuq|5Q^wKNDxbItVmblZPx8I@ov;~h;yP%@WlW&)Z3BBJ% zhGU@8YC~Vn8a{gOUcPg0{lG={KIT!cS#XNvz!XX6(&ypg7q4%mtc%>x>7JZ=@#Yr7 zo16QVRA!OXEqUz>pA^&Qpem|!rJT6MrT7sv=lndLm?^=!b!LBb?BO2MO%uZ;pBka&*JyVUT0KAj%p`DJ5;|=N|L?|i5k^*^KE}HLtGP@Q)sPH6SUw42-Hgo)J&0@Gc(I?d4^fsF=#vr^`Yc_8{jU?HTGjLZV>tjo0F zv785d_3w|i6$UmnS+R!lMBjqa?-AaEJ$g29QyZa#-Z zxq=G0I8HwO<&?saN9FoM&H#V$-ai`szunqzR{Gt>^8MZQ{CBO=e(z9fQHJF6bQd>8 zJ5Vg!y;)6@GBHIg5QF8}v*0<3{X7HWfH?86WdbFzrDI6_a^x?up*4HcHKoCy&wbYi zl^|!8z7s8Y3pPQj%3w`Du#ROqx1Z!^2p%J0_BqCE&O*bCT6Po+2Tz#hcSZ@^q9vV{ z;0ebH`2FC&pnz_wv~iplj8w)9Lu|#^F!95f3v5DyahS97m^fvj`+h=ql&vcKtg2sF zImQX9mz{@B8zK#e#^qQSNtxOgqUUCXPq7_SB^}d@JEqflOrK56uv2ImCA?}W{2)_! zXIS_bO60FH_1L*iQRV1ApJNDO5ChyYn#V;gQEG;bZChMiMM9eQwCS+^pBOZ%jBVLh zLi?qp-nQfwSEPUEQIushZ<|=o12w$A^u?FbS5#D<8|u`z>zFWIRzPf5Rb+$wWy4gI zYaZ}vyp)ae$0feRjf13}4Wv{3pt_hjKgAb|L92#^W*2OtkPx;ekm!PeMVH7N1)Jm5TR zQRvVw&9r{!-qDVq;pd+7KE(saXIP@1+A~Xezy;WE7>jg$y*f%P54Z?(mb9S;hCF76 zM?wDN10-|}tqw+gZhsmbIheJpaZ6S5@Be&>nisbS3*=yBTMxlc*lD)2^1w^53gDBx z>Lr*NJ8Me_lKNM(nwke(hDnJXmiq22Oje3txkpIqqm@d|2?#pLs7m{qxn^XIvhoa< zVYPy+jr6wbG`An;3EAw$NJ{^OudI@dUx#_zE3ji6i#+d@F4_s zDl8ena=E?8ju>WQ`w&Q%0$x& zm4JMb2i$^P0j}m<-GaFsJ+y7X6j7{e!r9T|)T%v>ML^~Fn*?{4R}~I0q^#cb^*1sXuZNkj|;jB zv@X3oTeydGYs&*}!(ISa@?LGjbof}(_lfJN56#>?-&GRhe z%v22uaXw>y^5eck6n!6(2mA>;>SJaFx;p|0M8h;6)`8x+Qt?<6P8`jxx|H4*7vVZ1c(*JGoWVTk6glHu(k{ zljyD$=?33YSe}?nLv53$a%F1>+L}SJQRV3pjV>NW2a0$0lZCDD{n^9Qsu-DoT^94k z_PjPm97@gkYhQL2L!4};UwJjmmMQ1m(Bz5m>bTWTjEoWZ*EhCvE+}VU4U$Baq@;Kx z2<;8!N?GwtltdJeD)EaZ2=kc?X_}ZHxNmgd@i6@7`q!5D<5yV`fA-eDb>F}G&u!M> z;Xg1Xe2nkuM^ARVdrlJAP8qLBV$cYK*ZErSun!Z8qYMZinYY%(ObjfBxN5BIEeN=| zumOw&I(DY6VC%!uYtIy?VN#pQIi*;#)~;*L5J+xqdKfh1K^+#`Mp|BsGQ39^34;!S?| zUj<$3xa1vv>MFu%l!hdREr2kCQC;s;>Ri>Mb*C~HZLC{_d`z9=rR(fn(@7c^-Sgb* zE_&XN(~tlS)biAOJ~!FC=b7^2d1H97P{qkVgUTKlwxB9RnI*_k@>xu*}6*(4JvW)gg-f)9s9tnuH4WQnEhmicY`FQ3P~w(Lzg!)K}1SFq0`^|j=8I9pY^H>RVP$==lzrNA*3i5^V8HwhB zgv${;hr*O3T<&m+r`mUPYdu+`Za7u_v0*{yF&{g8DEAtj~WaMp}hCGFm%h-AD6!a4zw^Q_8f zHQ1=ty|yq_V%hulYdgCB%YtTLPoZbgPC*yVld3zf zZ61E)8-H~y>Rt6qHllyL{|}^cYR)hDvXn}R**h5FuXe9v@GCLM`SK7`zvKwpkut$f zobLOEp_t&22gf@{ayP%(Wo7BG?!8&RtQs9|@;Zw8J`XDN<)q9d>uOv6;-iIhE3OkZPlHDv?|+&& z!+zTKW$tK=C+Wte(YLmBb)!#w;%+$1pSF8DHv05>WE?K#1#7fn)KF8i>4p=Vj(wXz z?$fZQPAg}t^pNR3bVdC&H))-Vy|!a7^E&0-kKa;pj2wM+pY3J@Q+l*JcdYgi=_d91 zg?E?6>c(C_i@WLNu5!FhOso#^lli>*h0FaCUA9_I1ysqcz@lS(!jCe3w`}0sdIk1esi9*)=Je*DsESJ{1 zW@7jGLta;4g?iI9vyjgp??Zzs_Zk-<{SzaP$U)WG>;Q0zd_W#o0$7_GT=%-^rgzAf zFW;bVo{mqh_~w0?TqK9Q9c}UsX1)3Hb(0>_IIrguKK^CuS64`LqMoX&c-Ktma9 zXMCdtC#PXXp>6!lzI099@q^&d4(T&~Np_R7g59BA>dk&>A(LN_ze1U$GyYk5lXJ30 zVZ9fc{o`Yb=1;|k^}C-5pxzyyT+rDtUelHGb{`GZ}3X7OdZVqV`oZ5;vikRhZ@ohUb^&>r=8ZjrW|NEL| z@pfK!#DaQD81r)R&V5+q5=sBh^INRq-A6`|D;HYA?-myCJ&TWAbJzbH`>y!so9@Vs zpq9VseX*m53z1uK`Vq5&|4h_r-rl~`60y)C_Iuzzop{)(fB(;T_wAprTY^`8zWx0M zqwYW0xaFVxKR7XdDw4YQ=G)szcNsi6dfbp!lvFG4q%KPBU!2IVri*rK z6^w1gMCb_S{AUwKP78Om9-C_w6V-+9wqnuLHu0L%M?UdioVcqcY11YX*e2_yi;evc zCyubRWNO>wN7@v0HRb32(}^{@a_DvywRTm6<|$&kx?8(Oj=Fl_zdBJRwq3inU8kn) zbVs}1T)X5*`x!_FzOhvw-C>~CftKnpwCN!AGl*_SII)8*u*10gsH~LQa<;a^bcFb; zz2p3DhxtLX8Kl!vs&k32)9OekiZ)x@blL`Xesb-!%jvwR+hkwfdAXyr|5c~Mkxr~R zqq5uSgeFgY>2y}>axH)7LhN#PBiCAYdBk>6`Uy@sUDs>7uBIe@d)IYyj@)sp%lpVC zb~N~)yZzOSawWS1Y`UpI=!QVI?vPlc#X=*tc8VfbR zNAcmKdZ^|7y|w*)9sT_y{R4CTgS-9jAp`xoQQ}*WdWZgxwd$<59lA`ZSU2B+@z{Zh zoPkf}1D|ULzH|&sjtqRQIq!f0p`SxP5(j2t)kn*9eeggHd?Cn$F5(0Z!VWG)sDIBn zI=VQ%EeK~X2u*`6@0zWo==y9y&~j+aE$ zr6>)fIHF!U$+0_E`79}fP3IDYG9HGrfiv=nz!TI^)-b3~E+Uc7wUV;{Db&TwakS1& zsmu%6N{g36QUTVP;(|36*4tYrbD~P=J(kDLpA%!#-3+H=X~0ty;KABmq?Qmn767CH zsTe3o2>=8D0;B1RSQFcTNBb4h+Ns*lAEG zfCll+b2bL1Q3Y=&$A!WInYtIA#(k6r?ThJ^E#XwEpWtvKy61!5*mE4!{v4b3L0 z13eT(VTEEGSkV{=wg-qzTTj8GsF#hSuj#>{%oy_@@%E%g zQcC#&OF%CZ8Ye9ppGiAP8TVl!X)k@jlE_$5Ci*oBke)$P*@w#ZaA14D%?SW16GBj; zL$=^tM{y+y0AY37%}bq4J+0C2lEidb!XgX^5v>vtiQ zFiT%B6-SF!@r}`?KrER^y<_8P>_FS~QQdm@i3UL63h+`~q7;&rEin=i87HO$1<}|E z_>mqZHX!2;+IXT{8A^`>QI4WhX!5qIG&m)#DiIDI;^I;Qa4}2KxKU>eJu41Kc>ZN> ze^ey_00!L2PqOK?1UsX)l1sUqu!_I7Pb$Sc%JGO$@kGX$&^N)I|ci_8UI{rvwEj z*tA9^`h@bC0B9=ADELqkBs3n5ivbLRXYM7Y9%FZfN1Ha>L+098;=lmLCwM5EW<6l! zVeDljKtwrVZFH)7jym=F!5uO!T9-!es(yX^$%O=8Fw}`A5e8%2n~wOQn3zE0oM1h= zwij==uGkkAU#oPa zf{;3t%ZY}}c#3qQ$p^(ihWHYuK~P^8YDp}_7(k3-7k)~escf97H9Vn+=R97gj;&x< zqyb&0qmjtbC&4|4RNFAbeSTeGXh$b&YhL|Q1v~r6;tOXy9t66-h@lrB`*tP3 z#WnAK`~^DDM!BddR#+?yB<{#&u6dPEi>jk#NMyflM@<^OQvYS+fv*v2|_VPoSzow&ul=}Fu8zF_=G=52^?PJHg7IA(b$ zc60w;3z8ZL-2xIlIg2@ti>-Tfr zM++!Z0@G|`LmzE%(|zM|d`m>2iZr^DHH?f0h6g`b?^xjpVIqG~wQo0b@X5vslq_l9 zg`khVAcKEI#rhSWyoqz#Tzu;1pYcHS+tDr@i+TKgLkJD{m0u$Ww=FsDCuZkQb=xk~ z$?8knmh1j;qGda_-@ot{eN0+I&Ifq3Bq7DK6D=BW*EO4r(@?Vu&@}Kr>9bR4>OIpw zHzlTV`lxFBeE=Il?Rt4jK6}&rMFF0X8IbWVGMz=p(4NH8ISe}r3xsv0~+a1EM zA9{;=l;@vabYP)d{%@F3w<1$7PlJr_r;{_S-bBWuaq|iIOZRzg*l`#0mjcc2pS2io zau43P+6UIt3Ywy19~B>)D$V6Bak?Kbh@}A~Y0yhSF@r~+|J@9$K1O!JK0SE}_{3#r2NX+D2wVYIdt|lxb!loMNh04 zlHJ4@u34X$JmltD0y*a$^RE8^mi58t*z}E52^U zKr3k?l{?ggX?sg%%mG)_Iqqn*5?iyynBKK9JEeT&^f&AffSLeMWX9@M2t1nL!_?m! zKM(w%4tK2wNHW|sX)|Od7zl7iR!oq!Y!x#Au1cYl1b`-z4Op3zrK^7wQor~S%5R$*+()G&;Ri?@90%ZkSdeIn1iaaUNp)A7 zQes~zaU5ZDMT-<HhmgXcZ)<93v{lHQJk2mC@KpIcn9&NwMVZtgFViMg;X?ksHOEX}|-E8;XI#uVu8;VR|_Ru}X8u;RGVYnaE4T z07<-O@xZuf4i{=pCYddK*V33Jc}FTqPOhr{H0Iee&Y;FXqRVrmvA^oOI@wrESDNP` zYRi=SZ0Ma>-7wvs1^OZz^K@Ln6($!}<=I&P6qiK< z-|fl7Qu~!PA4H$xAVOFu3kYEgf2(f)$QpDpJEJf{6iH+8(L$?pT}-L_Zzi*TD>KED zbya$#3nNr|uRMf$^eVht`>5)GkQ~t|@XhQ|-TzuXSX4y_AV5&DzG>L*JDD_ypy)|B z80FwZ!)bT)Uh-$vJJTR2W4#+7f@hn!Gl+_RWhZo$cl1?TP$~m}*ko8dM!hj%;6d7x zhKYbwWRyDib@V9w14`==0VEE`!lSd*MOIJx{@zJKb@JX-52uA+paT|utG|J)yjV=_ znM^u!dU-SQ*4wwruUC1ghrd0o&Pv1uN*alf)}G_RTyq&HoYkAmf}DhxylRU%x2VS> zx0)JtcLsF+Vd?G;pSwSn=X##1$IS7ak+N8vv!p9ee(wlyT)5Z%fwGO+Ij^^M+ciZu zT6gEXBr1a>aQzqh8{AZY>*E7z$_$U| zN}C-I$iPqxVA`Wy?8muJK+5b0rBa_Zwl)8X^-yzb=+?2_PQA=*k7Ln>eb5tP^0_mX1}7 z&fJ}5y~qs7iI?b`roYgBnRTZICJ4bYJp192dla`i{G+a{)Q!kDNRniDCz~XOm?R89 zrA+#gjv}KI9yCfT22}N3+5RB!xhAcY?a%#V1dJQXc&Hn0b(GX>d5d}7O@fWtf6eOE z-LWDM8M^C$1$4*lN7Z|3eTj35om<&e_zKW~Tq}FlaaigZ=0n%)$APd9uZn**diLH< zd>{FFPE;GyB@&heGwJt}?Q4`V;Hw^rQ>iUwP0Q$8{~Y+8uqrx)q4Qc-q3C+1Sy&WF zAT`7wh*VoHqbFx#{h2SaI0t?EKRSejJLd)ZA~3>I{KsCcio#zXIhA@%Ns z;kt;!{e#SDw#0c8~`fLjJl!ngEt>j_p(Ay;kxy^4|gt_zrGsSUJ#L$s3Mgs zJM|pOF@gN_bz9)cSbZzk^vtC`|9*#|P?F?i$w)Mz<{n$lqVI!~UzhKyJs-I_rCKs{ zp&0YvInfd(V#a`)PnR_==>0D~j64xN$2W|iDmHS-Dq|MEls+o?re`TcbUax?=;z>1sltA+vZ+a{UcY90m<6$*5O zJdjBRXrHbB@L>s9VTr)^c8GpPt&H!<5~gI%W=jB%qRG%!Hc87hm zF?cD`z=<=@zLZjt4O1g#!q8W!ZryM8>4i#HRxW-#{q`Q~>E6Y`A4wm#Aj0lQwx==Y zCpnsDpLjcE$j>awIIF(jj{S0nqznhef zt7~|mg245s%Ua}LeEa>FjccO}#G-_FZT`t-(s zuK-6hW=7L3bt}=j$dvW8-lC*6?)qZvpjEpTvq9>zJi(f-{u;qiv9}Sr|yoTQCWON=}sgeEq6v-J8nNnf$K!tKnnQMsSPH)XpIE za$r-~=5OIgYBGzZVpUsoecf5UXZxM1%|Ftpx^vDSLaeCAmNxN4t&$t4#J^IQUrnt@ zgHwNovzvD-^z~M(pZytq)x7tVdbsL?o$68D6cq_HI`$~k6|RsE(QZ4%`jOpou%NI1=i@Wh z-m8|w4XXY@Hp-D)8T03qh_>@7|BC*r<02Qe2m;7x05=vWPy!Ugf@DfSide8l378ke zR|F8BxLcH?tUN=Xo9 zsD)dHv@&qjwyNQh>||Fhe+hN7QhoPt=N-gu-jq}-d!T2_r}gwfPh*VS+>{cU zN4|GfkDPmY&J(8{>e8wBP`X^Xe7}< zs4N)M_Lx)eX;)RYCWM#Y5TEN9HRK7znnhfY+?Nm~&DBtnBuN9WcWs374Q|x8IyQn) z`>A1W(H4$nx(p-^9$f`enBWSsmAx?+6;c6k{1YQbkp*JFD0%YH zUmM456hN)p2|@@mAn0KitV{+%!-+xu#2^#=)j#b@a7KuS!L?8lhgU2RKm(LBVsc?r z!BE-TAUwRW{UjC#*oC8rfH77ImvqLw-9dSj|AJjjJhq5<4xO3Yj7xb-@iR%x-UayY zj;h82q?by=LQR6m484f!BP*@?AO@hSDw#O9UV$Ef&3&H0KD&{g`q`nnolWg1QfNu%z2DpdJy?Kqrs=xRF=RMhAIQ^HBqUIWc)opXK0aPu_)d@bKV<^Yyn)X{XegSy|{7%>D=7)po8H$r;e!Ov!j*32|f z&gXEq8;ln=#Y|DBQ!f0>%0^neV2_4iXfQ2F`$+>3#v-OSDa#-Rx)1I869dU_(`!6` z!x#7zZUOOypwX3C8W*{}P6X|Pt0tp?DX9=qE00|-|L1hbgy|vze3XGolHvp+3}uj% z3j0*6IGR3V)UjXg-u9h)+l6wWf@a9E4)UZX5u>9{HIQjEa2S=mI0rDKBspcop|C_3 zHx~CbhieZ_IMi&qo_KV5(BhOUOI=t{fY*SdlhBIIIkgFDH*PVx4V6uVAT*p2m*{K>3T5=2HGzu<#0(q#NA>THcZ22 z#HpDk%yUfu<~w@FTDjE;6#OkHUFw(*jbIv`1GKh`Cn-H^$HI5cq#WJ%9FIK7=8lVx z(Zy36IJkpc7I~MmWjeulgcI;2HnSr(Vsc}-<5G6HfJf_STvbkXhJuc>7rY42eVa|% z;>lHE0A=40k^n+@RVz=VLlE5|tCblFjW??yqVIq-yT<6RV!duQ`SS3_3{W` z;1_{wLK%=cc7wTE;-MDIo&k}D5I96D?P37|THUJn_`tcEpa??_-F7x#;OAVzV{*J_ z4sd3>T0LdIP*%UFfq_(^&;yiI+J za$OAs_-6k;oCLxsS#ZZ35*Z4*=Y%qDN@mO(n{It`fU2qCthng_P6G}IGs!F&l|Y{_ zr{T&O1C%jMDQ*O%_hYUqMK|UE50=IN69>1r4xG6P$GG}L)TnrZwqngd&O#qMJu4;s zvsiT(r5Xik)f_4Pg1i1zD!#(@M43%8Fcb#1`C(ViB^c2z-%^uMPUkA2cRh? z8R$vjt1IpT8x6&L<$xVsY!OP&3{D)o4F-gY04hH`+x_tz%?ZTefFnO}9b34=J zbDwstO~kKXp>9hH7I*&Hup}Xp^Wg@ufQ1jEXup#wIV^%v{<_XJ)B!Yr*aX$x_^9pR zD^|m4R^s7l165PP%P?pF*TGi~0UDgq$_{{ZWxUJouqK>VJx>$#>~dH831`R85nsQs zXwyEZ(P`hIFkfMddQaMLdREcTgbZS-hCE0D`#81z+SQwc8?%JV7Yw?;eETdE=Nx~& z;w~!1&CJ(TPl|98*++-AhI1UHnV(QI26&1v6e>AfvdD|XRU5vjhLM;M9t)6Hysn$l zMVcoiB18s`PpyTq)5Qa0Uv?=#_UIrQbfP$9+%V(MQN9=WxUw`Y6N)Bw zQ}qccQKw-daWb(`Akd;M>Q)?<=u4$)B3O7zQLv27TcPDfA2@1=_l`QL-`_xMi%@PX z9~8Tlg3LraW&u93i7yxi^T`r`_yvd)l_WL-6;<^vMfCb&VxTq(;ZAASq!^lRZNbr+ zFLr9BR4)?%<&#&LUPo{?4(llL8LZx&C*)7e)rs6PB75zH^AsWDewtkI9 z|58U-{CRb~t^ws>U+OH`|J|Z~`q!M+v8r{odAy7l)?~9&6A8p?-_ZVQTxI3^mZPm= z#pZG4D&PTs<|{I#`bK!Ly7%~J_ps#pqGVE{7T+-@o|w=&`Eoy)NXv=+Lmf z+lc{e^nhxDNVNG{y=&FULW^vc`NuT(pJ=X+zimD>G`PQzmIdC2lHftA#P8+O4ZygF zhhZZz5<0P3hbL_}*~kh*Z%&Qq9ju;2zcah0gGix2J$x(~R?(<_wi_{OubSC1oyL%A zO#;0CbRBWU*K#6hQk#-qPaXb9?kAb=pT1P^j_ab8UPMMcVeEyZj)mO2I@EdIDF|Y0 z|DI60@3)*1b8C3%I}Y`Xvnn|e4t$O891g0vSDkX?;LKk&S%icHCkGVxQ0HLkn-YeQ z-LtQXA;kXO{T2 zn^S^2$<$Jw-@c}sk6D`(yw2;qe5i?z{B|nmOG{+XG0$(EndeKzT?p?+zaE~dqp6#+ z=6v$)#+4#Q;^9JtNiEdZ@|MwZ2d*|Z6Fp?2aFBz(M#I>|rZgSZ<>QApFE?|!-X8BX z{cI%qD#ZL#Sc&H3G@R8ustV-TD%d)8cBSwGuR4lkGkiCpO_ee9@=79SSEoUmrJiqx z6bGAb=<(fE8}?D zkJPjNKm5@BT%O4L$e5q~`7i8JdwtKAiYJO#4ox*B|IVTqQM~J!C*S5&^@fB&Uf38d zb(AOms}JvHRI8Uwjl}Wmkg7r5%A;Cc}>_2>XFFa^<{DtRu+SjWulf1&;?Fvv+xUTIi4nMMM5mPI4>X_-y z^RYVh?f0)Ai<|j#wyf@e=*dtoDNWO`9+4ZpuI8<5LS7bnGVrtKTiE4lwweQ?!@Kjz zlT08K*GrkZ<6Qh(rfz&tuAPelsYPm7rt!TOmNC_7?Go|px9mO^wl>^-JPKjO6qpDl zx8-68Q7TlKe6J5ZvZX0jy>eAGA9_!|=(p;VkEa0H%+mrb`<1U&KRcx|Y?@u^98Ez9 zOVX8>i^yevIEr( zOz{L=$ErskTc=QL9G0DX%n^R03)jXYf3J^!qXN0@r=!>vC#G*FTE(XdM5~X@#EYlZ zV{ceEQ6Jn3SH8{lIYnD$LyTQum6=L5Io+MFp<{M&{?0`)uZqj3O~u(wVtdD*UarR} zyIb3OX%7U`oeXA66BYkd3wYDI7a!(*&R>4yd;^k{p&9Nmm6ga8Sb5U0@!1p6lKBTD zDXdJs(Ky%>oBXO%UB>2_9OM$^!Sd5D8&Mgy!W+~kegC_gErf8{csoc@HfJ@dH^z-2 z@u=ns=x;q7rAfgO?hdP}3F>5h-b3O$8?uH0umGs`dc zU`d8i{qbUdfApL>N^?t5CS`~vFu}rMCRdk>ZhlJBAhiwfAcB^EZ?=f9?2NxyWm9q= zq8U)NK+Rv`=8id(5D(xO0{h0szmpBrmS$c61{Qt2tYcQjJ}-drJ&dI~!_E$3Afs>4 zP9nyYrl8rTc=6i-Ct`l7`70gU$cWe5ouTqXVTj_azBm%Z)T=xfOvtE=dDv^1QXa%z zG@9L7ZPtJ)1Brdb7Cg*V07%r-RfdRj<&?0gFA-38E#eddjW33p zv-HG2_k`fVWQ!wnw=vw20f5D(sR$LRCEY*SrDC-Qzo+c&oMak-#2e2T@YVz3u*@7d zIYAEXa$WA;f>hxNt4xvGm(E`YEhWB>khzyr(C-S(uF3&;$b|QtyFzV_F7hlYd~swB zHC;$KLkYz|Q}IFiZn?M|6RevpU}-dL{e*Ho#b}AhTNDQoG`S1WuF14(+PL#9a&Pr2 zP{h8KJNJ2<`kFJU(7yAhx=`%?h0Ee!cwH+MAUud=qiClT51}oAydW+C)$$zwdnaYd z#s(Y)B~fB?40y%70ad6-+tjjgHurUn`+48X*~&*hMVk0~PDUk@GBB=GXqXY41;GL& z_P>ilkpLj?G!KP{2OvVbl@JU7U~c*t6o3!pbV=bv;sGcnx_-*cByt~&hZrBN`e2hT zYiR=ad_~X_({xVVJWh5b05MeP0O~jkVx=-fvNI-MQ4I`_oQ@AcV(Y83n?!@0J>q(R z6B1Y7dfeUEo9AE{423fQZ?YYTkr>qxKZLS7FS>1}l?gAcF z_IFTE_E(U6_}^f?;ZeHYeo?oGP7O*(hK`j5lqw~D51@nq7PG{Y&_!>s?O0lIOh$CL z)N~DM`e-bh0dt)zK@2Du>Z$o2t;S#>y?g*QVW2BiBnIGlJ_=hRPCIvlW(LV912$)Z zkT3!W`Wp)Lc18d(WRQbaH?`?f1{Zbpk_<#k5ivId#W>w<7l%jdr#LwS%cn(@ou3OQ1RYeTwDwR z4nYb84D|C~z1##PT8(G}4Y`-zSR|uczA)r$u=w4Rq}YKFvuXsInN8=Imq! zsu$;Rz%n%ZH`38mn?q4Z#*eyHJ#inH*cq8~t$JG9Y00-YL@TfcB8FCXLd=V|FZHw` z7tV*a-Qvx`zKydgGGO<9xGdV9FPM7;bS}G~47D^XXLhv^W|AMhoY|Yv+47T{*YPzr zwH{w=xWxse6#<6!o7#ZByFVu*oH@wF}L zc_1cN&xTOLL$XDvsh=nR;#G#f9_{e!%0ZO@YTG3N)T8-?yWmMe36_UW0B!eA_+z^q z2anGV9!CJ6PEP$8uhc1@k1R7EH6DHFCg|#eD9DVmDHLkbO^cP^t|{9#-GYeBZmXDs zR~0nqR02|-17~A!Yh+F+%jp|P(*J3nmJIBX(_<7d>ZjVdf$0T-gc8(HCAPW$_%>|~ z;nZM-2Mmfs29tQhv3xn{zf+ut5TJKjDVzq=$KWL^30Ob_;O`UxpoM_);u{&}2PL$F zKWjnF2p7USH#M=0xPi+Y?d$>$Z zq*eN!)B~apKvvBJ^fm*IRMxcJ5DJY6`EwqGVSq!^v8GV~03M9MYe@kB1`O)v0w(}! zihlY0RecrbsS;tXY5s#5As`-<2I7F0!K#tMl8k9Zr^V@v0|O^a0$$`4fcJ-z>@GK8 zWa#+*(D83QSom}!1lX*Il|-Y~y%mJn@Ze}XZ-LT>X7M%zEl^DvqBXp!i`sJY(<1(m zOs8B~$4es?7%nzou&Bi0{RHE1TDQd*_bN;&gG8YU^#u&rnn+v^lva67qR zCn(nc*aWX2|l__vti-wP@Egq{g>G6 z3HJ39E6%FUJKl9ve~ry(wI{D|6$z3j;M2`BV&P7c#%=2Cv$9DQV#Jw${(v5&tqDN4 zS;iT+m9U2l9P$qe{wKob>thR2&Vrx5fGXO0Cfi>x?f>djsuqDPo0 z;SFFDafW)2C=Q>S5&N8D{{99=5o^Q_`XQX&;J^HPN<1$*waj})EOu5XIA#^WvUzJd z!yj!R`!vCFz}irlUmSNGGX)>C^~O#k1}rwj=;9Z6W-ZURJ~n&eAi5`@7S|v{LJtmN zY}1A!f-paAtiGa+U)Mens$O;w{WZJpJB8b`fI?X%heXv&|AEf&WlE=pG9n2dig zNq;T)jO_+!Z}i#1hlTN~3@r{qGSj9FlmD;J(8 ze!18H3$k*OIV?q~(RdNlAOs_dnatb2a_ndwad!_ls0@U>z%~90mj`LU^58tjZfP_{T)?Yo(}>mja8$;y2i_?TFd!#!`=$ z2@b1rGB2-PCV_-i`m|OJ0y|XyZSV*!8Np0G>2T;bb0A0!zWR-{n-{l;D}FX{Wbhc#n>^zup!t z9DNEjm~`HhFg7%N9D=`BUoz$dAo2VJWtClunY94Y+95N?h=NT38ZU;5?aIQx&%*!2 z?vUAl9DWAFyWDlNsHNg(*jT%_ZAKvPs3tW$x_s!k$p%YAtt4WY{RB*ca@pI>KoHw* zaZG3kBM<)pJrB}KSwOsAfs_^j7#MB&o91{93%S{nUcHZN+QuiMV;&Pk-uP+2t$km8 z11z7}0CLRwK|%ytW0-4m-`pC>C4Ae(J!LMrCqgcT z(ZwaTi%TwZrzmNJ2)T5lq{$_BQ<5Z&h^{VCDy^U2U$DnxkMlU^^ZC4ApXbY7CD!h_ zqg^1Tk9M>0m|hL5^q5~@iTWoH>~77wHZcu>I?;Z9zZwH{GT@Xj*+1}j`=H^@b~(U} zds>9GE46V;f^4IG>&0{))*qzsYhHOfzvnlNc2XQRIwXF-WPH#bc)I$yKo``;U)xrd zHfoUkF7wdnOQLE166`5m9E^-y8N*^YN$)-Dj~&XGZ`s{3`b6i;IpL>&M2}?z9fB-$ zbsKXS?TqeGLEuLz?3CllC*@+w?c8>WHmt#z{KQg5ind5PV@_%GZUMK_f9Q(ksX83< za@dJ8BevNmomm;QG`GIs)h(A{*Z<1ZBn@;;CNJiuyR%$V#7c7L8WK>UG_HbIyng5c zlx=-EgPhiwa3K+hnN<}j-&CI^L9k=FgG4Y6B$P7c6u5&~zta9&p`!8v>mPx&6;|jJ zqgPA<8r3fx+3oHUTn#>ch20+ZAudv1Xm&uH-FE5N7iA5OJ~`)I{m&ZGgq!Jnl3 zF@A+6y1gEcrT=KdN?~1;+xiAXI&^d^UcIK_Y~je-n4Apv}F09GiL zKi81^Eo}fDrMkPZ-T#Azh@)`y`=AR+2Q_y%0S?iJKJpBcL#Qf9so2<21Rpm2d^j*V zt=a5-S#gb`A08!Au{8BrA{P5yYJnOH8Yq`2s*fESh3GFqw$tKq1D8l;n z5_|Va_y4}nM&Z7iG<&~yImhrkQ$m@iQ%>v14K}Qrd|W1G6Qi7;DxOs(DFyd>cl0W! zh+pb`Ds(O;#;MPl6=j9gIqii$_Qz$$)^ccK*9=1<+7#Nlu6*)-SmKn;AMEs#?=H z{E1qRxFn$MMt1&4KaBZJcMB${K^`quMzKk}XxscJ}*x#YCN}<%-vyp~5q#++SUWu+Wb@9+#Pf?8$x`XunOdRg{X;6M%gRxz z+yBj^f4JH6(dTpSy+yj>G+)L+bvtz!# z_D_ydg%A5=KYg}$78IiAIpZ{YW{?Y){kN(+^;P<(?1wAriNIb{@)tXD!o$;VpC_!i zJn{NHdx21;g42i--?8X)LsQGyAKvAC*}HDX zo2e@}AK722!fH5Opr0_XvElabw|?%9H%%#;myPGSQZvl-{}cSOa6q3OKPV{N8_;91 zSK|KdZ#lOQ*i08~=f1;6u?2hQmEr{i&XbMr^fX}p?E@xM?1IJRx?%HVh_lnWuwi9S z23psrBi-;uU$&A3J#iauRMn?PnJ~1>FsdFb*7JQB9B-gy9MojA$OJ<13Ia#q+x-;XuJjdgyH60apXBhfCNPp(*HnKVw{JNf3} z`A4m407Sr_ew_95>m4*$xcrgOjosaGs^vAtuxh<-=@jdI^a}{{)`fv0(*nhU-r(60 zoeHa=o@!wPiFYjazwCXRzCB~GCBhGnS+8zHp|NYjzMroJq1H3}NpZ`z)w!Om>qQ4% zwXHAuy=zBpY$(e)-u)}e8g?HvROLlZVB_mI zPu~6TDE$o`%b>Gd-+C`C%%ue&LJGQdVe=Pv89%_NIx>#by+4>L>z^K&rV!O1cv0m7 z{8+kL4qPo+t9(l;_Dn^;+J%*>fMGWt7dyAdVfAvsg(os-gs47p-XV`H;Sii}tCA61 z;Ak)qTzKd|k&q%+m4QAH@4Ic2<|`B5GZjfi`4@409qM$|OULhT<>3Cvdq~Qa77e^K zuc4v%=6wql$gTh?7o@i(tbv)6yQ-tyPmi7!cZExX~>P6Eg zxGPMK8oWN@%z9$rJ7d8q_Q*4I__8Hq?1wnzdgaB(fiFi^jE$5_nJ`Mk?GNQyQ4c?j zW|C#QjHe?z5^Qv$ABQ{GDU&9z6iCVW)-Cv{3@R*)?CJW#x*YQfBuf_;4iYzBoz9vt z=OHK}a)d1{gMOVQ%}{yRwNuIuY`gUO9d-`Lmjh{rec4Ivwyt+|(y7 zg5NY8^X{*!dNw8na}rFSeg4D2dv4?s-Kyl+PnTI_fqz2Ieu-Z;=5v*_ zK&ravwZ94nsA1Oqp!F@nBZUdIfi8`HO-Y}`;G9-U56 z986AIU#^fmzm>g&p?B}H(3`hlhxMqh(=M8ytQ7w{4sir%4EKggJJEP9>cT;$Pg#*+ z8~1vghU(GRk9K{U;lWX7W3qD-Vf;is+2q1kSwHHqx$$c8Zt&Nv3qiSv&}D0d%rEj| zgg!#l+Rjm^nsgB-WwzsToh1_P+&jCc@GqjH}b_h-Xr+>hJsXCZC~Pmslp zzh#V_`;Y!UucklcC)`~Rr$H*J-Z2J^Y0Ts_tgxy*NjH!PRqrW)KWWgA%Nek6_%OGR zwK}V=IBxG4e5|SxSLa|8m|1kCval(v?!Zw|x8lK#DlN^A_I3)-b7PA@sIcS4^xrSS zYxek0D(NJT8eV0ZJrf5-q_}%Lhn!Y8b1TPDS(DD@rwQN6zx5@CFLI98S*7=qSjr$< zO-pr0$M@wuXiHU7F2(PcuP?lvLef9AdcYvqra-yc+Tm!7SP)K#+1nz!rPOW4QdU;}hDm)hQW*opb_ z_2^K)sJ2wIpmVz{>L#_X9>}D}(sEg5oA<3_B*mWuvVGH8Zf3g4HdzBgoh)=U^So_% zK|JMI%?IT;oo+&U)}=d9_wOtmoDGf1(j2ceH6y$yA4NVjFR41Hnct}G*8$4i{uK9Q z`)d>ZpBNtyi|0ZOnwjDarJKJeF(LZA6LgufE7iPHSYDfoYt_qV(PwVy`b<|CTF;Tx zG`ippX2(?&@S_J+-{WE{BA^cvbaWr{JD!DfvDER^4l02CSe~XQz(6 zGnP>%kbA1-y1UR#T@sUW2K&Kcz{((?OXqr@VqZU5tP}Bpkd-hTY_LN+us;~r(-THc zdYt;sUt6eyR|=N`(PhbZo~M>|*V(rylV(YJ3O}{A`T$RffE}~dWL`F<=jPz7OB&XG z?{1T`AA(0!nQS-z;c(yz@6+Z#=0r!-rC~`kn=dqw`S}h{8UT zm_gd3;A(ECK|-E$!2QQJmcPrGI*{N6dBPW5ew^rLkW@x*U2#0{J#T4M*m!m>HxB77 z7`V7?ThsnXhCb7sZ9TiOM`bxzsi=ztNN(wOiseW9`Kbta&-Fh^O;Mfc!CFo6Fl#|1 z``mUfM(O9zydmV%LW)K^gnlvCJZC)=N%VqTw(C-DE&6e0=7TXp@0Mn3aJ1sH0Y|#n z?Q?c{<`o)3iPQR_9iVK*89;9F>%$;86HNAd^l_(__Yh$7V{WSUeA@dnf7z(R*Ws)> z%6NfdBzf2}7_6{~TzEr|*Gk7ud6@pbZ7;ENd=A5O-`=zbA#lf_Jx7V)O&)Q6HvtQ= z27nq)j~|AZpblU`ur{tmxhe>z0Sxu=EJ3VQi8L)P21R`g&C$yb-Kp*nps>JcC_ml> z!vH~-yR7{489jDf42ePJ3g!Kpy}1TP5Qq}n`8+s+9`qB1B&sW1XF&D-Dapancl79ANJfFC8cf252rD;td)geK{152vFQ%Qmf*AR5?{ZTzpY-dK_MI*}I7c zkpob20S>u8CFGFNQGCg4EXZ@xgi1TVpbb~Q3{>(Z1w~j84*(o-pp{Srlz@m|jGhjK zu4qfb0C5y2hyxPB^P^$d6G;_2OYkW;G*4K87ae~%a0ku;^CiETU{!hH1WteLg1(6|zo&e=-a27xC@(Uu%`0SBkSgx9J6bWIZ|&w zD10hLa$E+j;t5MA5z#%1WSy0gA;T#^_Bby4g?;wRu@WX~T<1RmoCYXu38i6%B?G&av{KKe3V|1~vgeu{a=Wey)J#vfmy+ zaX^~@gaQ?UIAGO76P2|?3YGm3QIA;%1GPWJQK$qc5ddRF@k9{; zH3F3n&=b6JtdTKb4hTxfh`@v|mSmuqHlwN0?ePi~@FItNe2FU-(PB=UKUi-MR{^q| z>rjOZ@O?rNR~>GS)H$3X=h9*d2%)c@lEE!loV)Mv@{a6mfH;K##&Myu%a@`>F2~?6 zQwWjTT-ba(+8l>saRH@_{fIm;jGrk*H@tY;{v3-d*Aj}TY1Z`11F{34o6WhENT?Nn zav(Ua$i+nEfk!VU{sbWVKR6)A^VCgq1!qCs0fjQZVq{7S+Af2?FQhIZ3sA&hbP+@l zFIkC|)Da96SbE8Uxk3&A%D+Ik^n;{{2*fPmynwI0`cu5@Dci>!O?J3Op@YN&*Fy=e z$_}eUJW6KQvTWA-B5HWKjB2ra!Zq=>_y;ZJ5R%$jhAJrlNaP4t+o`$I#6AaJ|E#Lk z%@KCx$gC1omF8vGFEC3Y=7-5}Q+srxJZcGBR)hr}FiJ7jGy}V+hFu&He;VNmAe8R|)8PxY(bk9p}V&Q#!k~tQn`zuAHI*3OD z(RXBqB0jS>0i?i(ZCCE@!-A{{4ZhFI1XNgqv2}O9W19k03J@aEJf1q1o$zYZ_4YlJ z5B8&ha+f4O|E#xOXb8c^B&-Gj> zaW68^tm#`z;k78={Tx-oVQ<1o;$^ze%~su^R{hVdyX#wPV<9)ecMLv*41MqDpU^SA zdPgryzC7<1c@3e>J+sSKC4~lADY(2)i|UDWqJt0iZ``e9lr`$OPaZp=c0v|ee~+-H z^j{a6Fn%cNj-1k)mV@DUKghQpIfsmMwWEZKJJGOB-B4RH}v2<{qqC5 zSlfj^A;&IXw!V6QVNT?Z*;$fdy#k2xju$?*(MEEQNzri+OU^p^M#itc{X$FLWJ@3& zwjaMH;UH5RcCM|t?f#VmmnZ{fu#$(xE|Sk}vuq9T+9*CM7UZ`%o_OSR^-=O~?{%8b zwMXrZmy~*Z*$y91zTkeQr|WW+M2#ZE=E z099Y#zCshj6J;xL#tpw86$OeH1wBerd{Wr*C`$~TeeTJJU*3dMMg3Xmva6j`-u{L_ zBrVel0TOrMMJB9CtWqCKlK?rq$|e`sX-q<^!cjmcuu}?yEqosc2%V#&mWLj`qd8TE ziQ`t0e--5yc$TSIj|g8dzaHJabi7;iVK50<4^A@g`x8N&k#f`VyY7bUUcHa&!asyl zx|d5##`g09 zpkOXgLxowstU=|Mn+1EXwD0Ryr|bK5TOg#ixX@9zC)FnhRhZqjAaRZH!KDQMr2vcL zL-(OG_u%=^6(J$n6Uf`eQZ6BK1P!FIs%VgC9)b$e4nWaWGcFuG4f+!g+M@=j<54-k z5Ftch9SQDV=#p}gEZv!?bJGNb{&RtfEe^oa4bY4%*mhM^MxkB)Swdr+#MP?m2Sw0# zgaiA2N}#xrBASv=Zi-)+sD+LwbXA~@$2!P-fr&6iRk@0URq-9lw3Pp>k`ph@RM8nUWcIemwTUw-?Qx|RJK_jsu0}9 zP$$~}$I3aX;T6Y=AE>tl)=+c)0jbRc4n(Kkc|UKalfpe0rA_;6Ygb(LFwe&y(j3wT<-LS7Bz*+uDmpt0^T zkC-&2hgw2{)`_psP#SnH0Ul1*PL~78@r@YD6<>rQP>yC47KHel+G7XZ7bMdE-EYj@Y>*oN)lXq=`;U+Z-pqUN z-2cA#@B73vj-Uqz@w5ex5}<~TBzg8Pr|v6`dH&qcTA<~oV(+vdouc7&vB*Aa!TDc` z_60Rh-eHGQsMZS^FC{O^w-_}s7rimUOCRVmGY4fFIV|eDSRN#b;||sUiT?dUjYZdn zik4BJU7fYNe)6=ORbH*OFT!wVEo)@MAC=~s*jjd z;w*yVGQDbprI6r5D|E?(fP!glxJTQ0>~R3LW0MN<;{VE4XeEG*@kuZ&zzPs;i3Xxp z5d~9~)t4cgL_qvz!KNx#yE4iGLpNpEz}3^^^CZ%V!qT7J~GP zhyS_jMN2?Pe1|X%EyonrkGWk3>P~CtuFJ-}P)?mzNOk%%M*sI$5`qAb3McvNo;>Vb zB;LBXorfDpbwVX#_INcTAdcp0#Gt%^?36f}zs333oj3{|#W6MlZj`4LizsvvcF0EU~i75#pG4f z6OUMp^pH1`UwzxU3c_70=DUS`1rhJRExviRa+J#y5`Gi%^@Mj2boV(w;MP}}zZxQ}1rke|lPN)4r`lh5XZU z-hT9crAN%vV7rWSYIBCza;WTF*Es;u-1c7ZeWg6_8z`$IQ;XCk`$nw@xhG5~<%7JN zLQJxlx+_C91>1(SMu&|W#*jshGQkRH-_vWo{S720hab_f_UP2i)E61YlN;K^UCHQJ zdsX^cAQ);eNbAnFWIX&dU|T6Da&>e}55TgRtoN*}MAzlDYhMG9!FwS7`M2#{paD^L z#GWw0Fc-h|XW2Uwa4sf{%>&X@q6g-#S;kH^Ad0pPdody$hd?KKEMEo4}GY2-%WhqZPBs2q3%xeGFwL9#m&_I&OK!qL2>U(HQ(ZHpmt?{JUb8(S#%Sn z`FOEzI?lkJDF#l|i<=!$O5u4X;rFbS#r;124CAtnOHN0}E;F+%t>yblLcRP<=dhn1{rMKsAV%qTC z9wB-a(e43*Y3aK2m|GL=npTJF$h_nc`v9keHZDOU0Fl7O3iAlu7zNez+JhtFskh+5 z?KeO!NKIjfKWVUbWC5B^?1E!9gi&b~>hv&xDFZ5gbFz0gV^P(ilz(}Jarr^$0yo3C zN=-)iX1&hwaoWfY>-h4nR8MxQRLJTPt8|{l^L}+{DlHjTsd;6z;!w`1svzZ^{E5Eo z?K7+Siki?leH@LMT?EW2#tA(z;%nz->1D0LWIzUOVhSjvHClgM{iL?AbvoBfI;rCF35n|W5F3v5j*bT z{8+j_A3T^;WM*h46O*YXRGO!Nm>o}F%77}&vx)FoQgNU)5iEj4qolB-PI@+AgguC? zK?8}TjACspE0e5SR86VJFloV6y$j4+_)G59oe$?7j|}Q2_yxrD!Jm53@c35i-6C0H zBj%l652T8LBqUi z-~;CV%+3+oX^-thTa|QfsuLey?OQGL4sGYZF?p-{V@Oj36T+KvY61cgImTbpf2zw&oH=IkfY@-9 z31`?Y1UO~4ap|Tf8$e2rx-k2G2pfp*`s}7~1angAILR}p3mas@XjKazrEtXpV4<(c3 z)4GBb_h2a~vkbsk=3!iO(jJGt;Q~3qrfPx(d7EX+?a)dOq~3dfCj$EHmEcF5qQ)%k znKAuc|Cack+ZK8}Ecx2A%BA6ji!Jg`oMZ(O<%n1f|Fx{pr1#4I;`{CV*9&hZE$Z)2 z@C%-UvI%N$6Yezy29qn&&V9YV6jNxoPOF%HHGA_w=IHsERZ$l@qX}Dy$uaz*wL4++n_nFYFD>qRBA@ox z$Y1!}uM-+So|jQnuAfL$4?cYO<)dX;-OPU-Z~XoVKQm)5VfwB|eIM_QhpA&NgDP5P zl2Os|FfB8tY#32Ei3n#CMaqbx4MedHB4U^*-a|y2fym32+nxtbmz}G^14ylfT2V~B4&awewAD|j7+@}Tg(cAXkjQudXznu6k{jKUO$S-Ns4LG z&cv}>4-4(id%EXOnk57D^j2Qf&(n942zEc#tT$TW7yKOHhC0=qP1+tgkP<$*A9yo`MLUu%*=74q>5^chiQh+GjS{XO(pX2PBWU zUF3epF&(O=)%sI{5qQM``sHJGg~nz2Z+fOtVm7bKw)PQ+O2myF9j+$bS#3)#{R~=0 z4YW2}=sg^0@v=|TFutf%9LnoZ#DHr~I^IljtYtgal{wyOaIEhbymb;Jxt$hx(sq*u zd440smt$O^Cyv1l*^nLX-Yp5W2h09a^9G4|4d}EIY@S?2gmJs>(PxH)lVNTX_YR=5 zhvpt!Ey{Pm4X&$Sl;xNR#<@Z!T*rR4(?Hp9{{|pw2CC`NvU!APL#13rLjPMB8ux0x zf*yJ4a^S^wN7v3T!@-#%pcHi0v?@0g3qIg9iNPz+VvuR_p{dr!unaNT;ZPW zea>beXRNrFyYLb{>CiI!(8sbvpBfH*?l|;i_|V6VG|$+FcWT*Uq&_3~z|0r?Y1QHN z6wHK^v!^#fYY=54l^eaLDEQ1g?4&dk==^tB{ab7@wjVOEKJXHb-%xUhwjb5w3Z3wx z!Xl}{=c({)s>n5}Xd_juniovV)1&n|r%0X-7d_yFm(HhKJ5C{2BP##8wSugaWL|*YxYamd#Rxqx3C#y z7klt~Z&?i~XsmRKZ>{*QknuRL0(Vh=sK=>rM)?~|8k+dL$K10yJ}tSz)BJI+(z5dR zPaxx$9-~TW*T7RAE*`8#c4+$jQzciL3B9efWPTCbk^EeaB<7oUm_6c^{njhzt5@z{ zFM8g~sQ^CjG0u8=e*Vz`X;j=~q>V)kaIW5AO~DPl8nY3P-e0cqlJ)wi z77^L2ERzzNf^x=yVH-P$pl@5YZ~HahhxEobT3@+(STgH8EZZ9N=&vtV+^p%c(JpAc4@LaV z?SqpeC66X#vQNcddgaLKuXFyLGUfOp_kEB|G*<|X0sGmfiHYwN;rh?(`@gsHpQHB* ziH}TMZo+m3VVaLxZyqyOi}vtqqM>A_CCt;^!!(EWd89u%?TR`W*W)q_g+haGlv zTfJl);+&y7J|CV!`6%}V4@ruA+qbOvSElTxW|=@pygyuGyLd?Y;5$$eI}wTUWWr3^ zK(S_rL=5gObjU>>*>N|BIrXZdlh5)u8f`7RzDDbQG+S?wY zOsy(swFVJb>M}~Bfy%RiD&GQk2?DVa$5aJ@OT$on9&2Jb&?OJ7B*E`vWN)9DALjeu zRkEl(-KULn0#~)KW)>M!>|UHl2u%b?7`)%tnx?o#WLWX!4TDTgbo|Wdpc8<&BY`!e zJb^BmJHEgPuMSLB$daC&kyUx{NJ;#}D%AQ=u#JDPZB($`g<$)fV2AQx$Fsq^+ji~_ zEJzX;%x@oS#RNF3i0ep%^mXOxch?2s53X(92zo5;xP09^0k(WT$VW>#7Q@nOdo6eP zqU47fA3;dr+>z0G_S^hS`or*$*8T&BW#@(q99rUq>sGHO;vV$zbxmT)ydA+);6+4F z5u*jiX%e9^QB1{VkRp$J>KoCdmf~ug{h!t%Uhs~S68bD}GBuSIA`kn0PN_GYeX2dk zv)z^q9A_C^_B{?ca@@r~>OCuJITrOH<8jp)m29L&XGRq6T`ve zpS=iUe0AV*l_#6nIs596$x$ zeJjJPInh`X!FU|emvfTN|MVSeBMcv9$AuYL

      u4$ks;gx#i2D zxe=^Yy9!70(1q<616PZt*YguGb^o5PgeM-$rQi7~I-e!uo4y$R~g{_vm1 z^~9Jf-iDQGU|JwBWduK7PspFj9)W`r>fjh7`3}(ys8KyXKs?(>N6pSd{pJamJYaM^ z8r^o{J@q3QllV5B$_yDUsi7g`eJ@tFk8D40x)>K?=%?{I= z@NJX+EPwny+C!!S6EG0f%zO4dO6K#!RKBdpl{?LGS92BeCI1PKIU@{^a!bvxJYmu9 zA7ix=hOgOiSjje*eYTtR;6S9}(VvTTwY*yfpLd4gp6zkID(fW@SQn&X(+1`7X~l}O zb717^7J_V=p7;;N+MWKJk$Xo$f+;EaZ&{9>+$Ox*sW1<}{SWtZBriVo$=Q75)<~Q1 z)NdFTFZr)a16;a8hezN3QD<3eh7X<8RT95x@7rV6+d5@b7R|0qW}$aydEeYjQCu0A z|2FM$CLoTz*ZD}NVR^i<1de#74}Yc=x9_oZaoG8^11~Z(Mm#uBKbq70E}!ZUkU&%)J^_ zMv*R@ZtBkPG}%)DmCpvaOS!gm`=k#$!SO*RI_ye_bcSV^0%!|%95i9;9H?IxX$Xkmiq3`@a{pml=c|q#$n5-7bN|UX!7_8eZYu%IcsnaXZ~GFu z%>J@VU}3)f{4%3;e_xLr!07*ekeGu8*uSJhxVtE6ZB)Ww(Z#HM%Rc(-wLdj6>d$gi z_Fp)$@Ikr1K+DE~P%yq=N-A}lyt>8Kd=}RvV7JM`8W+S())m0wI;U~tmA1KR2XCFm zyK$@@pSQ5Xj~%+=(R6I&)*0PbxBb1#rQOpOUbP+{t2|gQ`;`iay?UBG^wPKe;n|h> zk@~pZ?>aC1`%p}7C{wXIL(g#NI(Kkz?~_+NZiKK`v_->`k;1v+x_qo@$RJHbJekCj zPDvxrsbzNUq2m+bKDaSrDW?*i^j(-T(Jp2#H6eXPEq6cc-Avf(&u>rlAw>plYr>i- zPXdA2t}qQ_Yb{3@=T~8XUa!z1)Vdu{KSVB16p0P%7f1V~Z~AT1L!K(%b%HKRrG-;{ zRm_vz$}OQCZfVf?mAwuy)2ij4oQX=ZDDZvs#5f#u{c)Ey*Q=phb z{_$M|hb_nsiT-XQk73AzV{;9qpu+AdeowiJ`XUH*5_Em<>7f*XA1~byMa<`#liq?E zwyH^mU~)EwRD&15iGJm!TMi%e*%EHHT7tImZVLc9=Zv)4Dul!k=bp3V65(ot7MQNo zS)T`0NU*n-e0Qp$)&NBWUDROx!n{3pClO2MZ9H^w-oMvo!ki*|Pmem*QFxLn=Wx#5 zYS`n%gqy;^i_(mDEJeX>pFXHdL)%j2)!TlNkE?G#@=16xYSQ8E+VuG6{MDw;KkILr zp8T7yhjNSI+tFQ+C0!?THP-1-3tqYrN-kt!9Ijbe5<8^djZwFUCva$2sC6RiRr_n` z3p`4sptfTdTKw(%Tr8lcgGR_ITn?t(5y28q;4xSx6xvace3le*Sp49LNI*l;bg`B)J~kEM2TbGQ#J=L+^q{HoaUQK5s)B=g0U1CtlF`x znvQs)*v9v-s8!1{?!`&gqN!!o~C{w~CZ7L)~f5%nPEfY+&zV_3uyGON}A9 zRPcBWtz`gVA3Yo|mBNvl5m-WqR``v#JW6@(sZzgtqg9gxE9-!7bi1YW0&6R+YeDQ# zmm~M|5<^xaG&RgU?z0Lqie+q&*N*jDAw7w2u?tKxUZGV3R}>k(pj?DY-i9r8NhYSk zAAnUw4x}5L%^pd?mi20?tfe2|64H#yRK-t(bt}#jF7BIAl{{BVk^8Sw>K!m+t8_`k z0Lm20Zc8;RZ;}2*C*8=V38DFyzAG-*CDc4n~;6qC# zDsP&DZidMx0~-S=hH3d3o@GH862Y7DA6iYOQYt0Ym+nICM6nz~)P%>vj;ZemzwnOA z!peX*t?c{^F-Hmo+H4*!z{bS-(mUG z=XcRA9ePnj2RD^{GItWgID5+q@l>7uJmbH?arSE;56Q<{Hj9km{@bi`J2ZfD)Z1O` zTC4-?-uJTp*h>xE@U4e;*T}J6gob0v^v_;%U0HUkl@;AIiUeo1pusXIvvv-=0UcK| z19MOb{nK}$&DJ|vI^N&WeTP*^{FZzzlpY-Byl0T9 z*N$X~=}Cwl_+edp{$k|GVV+>ZcXC5QY7CSO5rB)yP0*J^yfUD1;-B$xS z0^Q9Lj0t-NcIL${=vjL9CWd82-+J-*+;5)^jGAcY&Mi517|th8Zv37G3GA{dgG!5$ zDeRomsnDY!IoO-BTfsQ<9VRfHhMl($U=kg`05p&Bhsf;T0VVxGG?0#Td%plqx>P~S zJr&02ztFEm%~bzU6=q%+14CBVnGYSQ2^Bl5nXS4jFsfGu&ZVSfpNap$l+Nx`cK z7qBYfHf#?5iAe)pLEPg6opoM|5E4Lr??- zv(FBV57N;^EGxW=2FI$iQy8^lUKnh?##r;oU^uVB2*WjBu(99sB>zjsJ_Kx1)4lb^ z;hlj5S%pgM*ENv?gSkXVwfdpFylW5$Qgw%(nh8n!ah+3A&|6+^=259z^b)%r;s?K< zS{?%e7uhu-Vki#_snH7N+sbA_?T$GqyXNT=wJHvrrD~&m9{nf)lQvfeT^&NHBh+7@k9KkP|%d%}5{VQ(cb@%#C#h(kD{HJUD z$Ki{pvLl&>XumQ_FWcUBaptd=ka1g=?|6!330SU;X%o_GK4Y!YjHWISFVg!d!wL2Q`h4xZ{=pO}e5abrM6$jF z#O3e^}a zFF$3Pr;0#ZOVQFCtGPEosdz3?(7{qH6E=okfD(;99*nstAbj8z1w zyA?Ehh29@dJA9a>>HRX<<2S@)pSwDZYI@}$97EYBp{6+z=s|?ssmt`Z``$c6#pLgT zrJ)zq>*9W?N&T%?83+Ayp#9Pk3s25dE^Hj7gDolHnm`kw;@mVT83&6*0A*F}U&hzEbP?D8<75+Em(AgdXn-EW< zcRMjjgy>R?kBy7kk{x!@YoE58|FOOH6Yl?op7GIkp)JOGOndr=#wFgQ6k&aMV}A6r z@At)KV~Qw`i6(iyye-F0HA~@DK8eJroPO( zth-4pw6TMLh->{$WBpg34O}FJF+j^j7e2v&Fd(c{i=hUUGY&hw0ZdFJUzntf_d6w* zk3_cF;7|-8jFE3H3xp{fTLTO_E10r%{E!DCF=FZPwPgG4<-Ej{SgDUlhxs_2@STn( z>(t=_GT!U>Qog@TobAVA9ZjD~AqSyQzdSpYt=dN!JdX~F-b?JiX6qysp-?#as@hA} zPRP~!bzW+{CilaI*R{OA;_PN1=o!u8h-v*IF^)uCzEt?BW8sx1i|Y3DP@kQTdfD|# z$4|8f@fDQ?#fKtla%2q37}~)%JS-v9v^r0xS3+eK9vr*N^Un4Rm_U=OU|$>VQdW!O z=|r0?RlK}~E2(ywpxnOY_~UE1-NyS`pHKX|>9=G*?^gJfr*^AmU;TdLa33N?me6&_ z=LE=`a`Qq&t3-||v|Gf9{`6kvXGp?R5if{OKSf3Agl9iRvIExgkcmH?`fq2?5&X3N ztaU?9eF03@Zo-io``KM+AQ9H!V(>ZF;vy*qsDhmc*_^N(v@>;w4lyEQ_pjL)4B zUt9^6q8*X5pT5qnN51>r_@0r`E3Q+eSBKk_#dZxv^;k^39br2(ebF1+cXRB&+ll+W&0Kxf;I%WB#m2O)&iXRk3N2@OA|z1H#sfE-aZV}AVulHr&Npr~kL zKLEnDi{xe+1}iOQ!;TzbScWm68yP$9OMZU|4twN7J7`f!g`+`~5AL_fb8dVr1%w5g z6u%utGG{l4$Y3qGupAf=0*|zPjRWf%Zn2ua z>zYBaS|`@E_N*HuHSC)r{EwpZ@Q3<;;P~ej#~lubv(BB7v-dtaduA3Y$&4g>S8kYD zol()yktCxeN|EfHQHYYx2x-`((tUsT`wQ;#xX0b&^LW4Buh;X*MU7Hd)|9^;C8yTF z5=k)CENYOcX;??vsk&?5rgPQqxV%wcdUd&NiFY;1!XW)zZ($ z>GvBr-;|QKATLq*m$;V)QV79W8b~qFVfV$cz6zuid#eo%h4W`xqeW4`M!KcWnIla8 zy?)4rRPN()oFzr7U%#ccU3K>ViXH!X53g12H63Mpd&l;Ej9tx+-Ln|`mL0NvSB%5Z zj>AK3-9psyu^7jpMl6NfE}Mp~N^wN&It#|yzbJlDH57F@$D^1gj;nFsDIi3!IyeJ< zXvPN#SM{q{_o&Ne9cw}S!%*RucBAJvB9pd{UajCKSX_Q8=v<$xfC2z*QU3lrUT)xv z13HkL<@E#P<~hRcuiGZkedHP2x{EraGsR~4J;RA_OJIWzUbkn>7xN7U%;AR?yrjv( z;oPHWPx@Yt2n25K#gEpS#9!Mze?P8yY;-jtRot0&?4J3Ns-l+W%|PQ=+km2l&NPh( z6+u*II3zv-u^%BAA1S^cDH9*1v>&ClAN2{CqcymbPK4Y?P@VTH$=91AlC%=fPzcV) zW9Zw-vwbESsl9~hwC3fg{HM^%g(>IHgzmgID(1UT;GHo<{w|>~Zy_|C6 zGZ@6KZck@HiKDQbYtS7E2yRx^n2w7P>3rl%vuGC(IBVWY27_We4jCT2s8ofWK3?nW z@g-1fg=o;C)lGR4IKb?*pi`Hhe$j+ zhum?mI}joAl=_-}u&?u{mZSMab1m zD`djK1Y+}Q#5v$qQ*>+P%4pEeVA<{*aQju=@fMF%OEP!GXOG}ztNa*S&L%CJ8(14t zJ{u|)2_d$p7(!!R{vM7({@L?AtICe;^G_SQ+d5v8RwVv4PK0GSo7UT{T^p|(9t7Lit_`O8u^aH8t7qF##u+J}BKm*!^}D?LNo%W3f6yx`sfzog=UB>Y(=i zl~-OI|N37;$I8ys-M{JT=k<0|zoVi)d({c;9Eyvy`|NO-Id2?s&4e=L3 zhv?j?w?+-YBTwv_TUE@M)=hEc<8^w_vC+{{`<#TB(uK%$J!?U_xHd5 zAW&5*gi9D%Ir2#_4I!!)*{yVb^eJ3wE>dTf-o-1QO>}$W8wD49tNU3(U0siByRyJ;nVg6kCGO4&YIxWFXs!e2)Vr`HGe?<;1sTs#9 zGmEk)^3Bm$7Iiui`(0EuS#s}Tk`GS!q<33-xnJb(edc*r3?d1OF%eyCH1kl8i7cDD}I?i;7 z#?UQ!|K@}vC*AKS1z}uj?W8#g6G}3CrUd@4H+&h?!ImVYLC>WE@g&YHD+`^Hz-Kou zmM9-pdsMi5>tdVe%AMG`l9kdcEYVdmL+WLKIAx$5l@_pf=GJ*mE`yHX;s@?FXCB;c z6T2|bZ!mv0E9xWH{C@;$IVWZf|6M|{%WA5nehxH>X4#1;T+6N*=_)SpC{5IFyHQ5< zD}MCtn0Qtb?C4?nP5v*LcS00J{}lAg-Mh0jsPaZ)dsye8#Fl+}^3ry{(TB@BZ^_4Q z&U*P4Tz#cbQ8yyCH~m|>8!{y#k|=_%&A+=p{dNB1URt$akFCp~4IcJveRIBCw5m%*~H_WxjKi&4d|C0&Duk4Rm~LOcBzhXK}2(f$y!K-S)?3$mF{ zu(7ZAStXUm2|vW$f5ojFg{*y#Kl6#~B;fG0;lJ5(q!^(PqPLZY#o1Djk=JpLNLT9o^I?^}{Cv?frq#Q0j!hpwjTnH&(_nfjDrO>a}kpnBR3Y*&0Em-D_KWzLc z9^Mo4Kx0>*GcailqetqJa(m!##3BDt`hqqIm0RsD)s>b+u*16PSjwOaGkhJ`S0Bn< zeuQe2i!9YQ>J+(@9T}%F$}h5G=_AC~(b;MtIL<`pucq74c?hI%ZP2`Hz_&oB!RG`{ zg?esn?TBmlN%%Vx+Uep2xD(Gs7g<$X>g}$AyS%^>9g#&f#fn9DU2?EO8As$mM4oZ6BBu9#BXDv5=oy;bF;X7di#M^8HbXT>AdF4YGI;X) zrlI_<&6EM(u)>X-sho0z*IjVe4YPM>H$D#KNgnl8s>T5rhCzY2gN`|5Ztv%w{L=*w z|4AM)C;))=Kn!%Ik5W53Npft-(KEDUO#vx68?2I)Bq4l`{6C z>ajw@$^;_j1gh~$Pg(dTl1H6^{9;4<%iXl#`tw*I3u7h2(!F5U>7#Plf-jMhyR4a+ zfY~KkEAl|2?(e_WN@<`9Ewr1(!Jn(CF6SDRatCJIeFqL=KQ@`;J7v zIUuka0MGHIypn52KLLOlo(P1hED1$87XtPCB+|{@37iJEpD-f@@Hj7k!xGW~AIi2B z#vE7%k+9xnRAPiFB=8G2sCD+H@%u~a$$Ne5m7vlZoA0%NCu@tSGWiI>B$SwXyE-)D z8Wx4)v$dffb)_UhV)Qgjn-k&M?8%>Y=d$85qEu8JSxdVABvBn;W|(rVCCh-0%Cg$k zeHc_Lb~#tT#>`K%1F&nSyii|_>2UD%SHT4=F_&4g|Cn^_C)I#sLhZxenKEaL2=cqw+ z)Jd%D(pFPRqTdm_F({{>+;I|&*nuE|L_9ZSh^DaCc45bQ*~_vaA1i+kLbwCu;Hn({ z&@z7AbbrHik5UjB-^73n35Pucia49Gv=zqTY&f#JXXWmT(4G21=ZW}v6+lNSi@t@%zg)_5u{J` z@`muO?|;{*#|8X1xO*@+WS@^v({kqhL;5kUK+Pb|{a{o|mcbs_c&IY?fp$2VQTAS7 zaf>*ns?E3Ai>UEtW8rGB5sp~b_K}=!1G(zaflk}`_z-?NA=Q!aY zI9c``iG(U8c5tMp$QZqoIVvD)?ka1&CF9aA>jc19`kHd~QF6BL{MNlfX-KW@p3E=OO1rB9RDu*eI7g*u`-8;VaQC;-{n#;-y5>+Lrd($ ziUf4aOTYj?ngaJO;|!@&>wM_Eh6nFNR3k6iAJD|Z%XSVy{I}VAAHjCn8tzjlaMzpuIR3(2w6!DA)Z(*CTaYD-cY=+iWXQ-D*z7VvT2R?U#12pHVM%^|$zyznXoy%rK4QZB6@hB}BqdNf% z`u`=xU^q1FPmF7l*SA$P@q*g(&0DU`!8<~?lwbwFgfS%S4PFZwipT{=ogOD3A_w8c z+Q!0}!vx>GBpfWqK_kLZOpE%$r4-_%lER%4r4 zfgtlktSjWVxp5ErsJg(-Pzsl#W%wn-EAY%P44z+40AH7eLUC=d?)`e z5(oV=h{>3jgyO#cF>9izzz=;_wcA&MzrSSQVZmCUI5}{{>W}~lVg^hdzwl@a2V0_g ztC30I(s`l0qOIq=84M6`$4-ORe){Xv8R&gCUmV!X2N@n*UB4j72B@-JnXCHt5TEPM z40XYSV_TiaPv(|ioU?*RGa;c2O8HN)D#iPOHE=`?7L_NUi3caRgFT^)Vp@ZGcs3rHkC-eRwOK5`_; z()?0YE<98C?bs&PjN)JPUGkwI!|=vOi?ZDVv!9u+8g27U-EPzZSl+Pt;g8PiMM8>< zj*5CSRB73#Ukei69dJM4#p1V~#y6FVcMDrh&m6&@`kaPUg~~5in3dDj3UbMhbkD62 ze3M@7N_tcO87^aQjiRS%gbd%LIK7!1{AEU)tA>w+ zV8XtOf22DY@%W{$*H&@9O>+4Vr}uivmuNHWb~ukMoe$Mr7S$}f6E@_QQwEehsbNqZ zZYtihuo?AnkC6vE%vxckUC5~F?rs6*xoF*wnjPft^TIi0*moXij5|8U1Cn%M`-UPi zIymW$TKvqqkWHxFa!!SxN4zB_+8SdA?V>&%v>zKD8J=Qz7q%6=P#W<~I$K7`h^roc zPV;z#%+Oa3P)>LQO-k#I*wsvY8^7`kqy~y0-cCD8Q{pt-Q1-@8k%{{Xo!Pg+iuk}i zin++Pl`iFLN{pxi+XZmORnKh=kC>+D=2dGrOu4d6tl=b8zH-fv4brcoVob5?MHAPq z33a34xaYr2>2J+@F%|!Mk^4vdg+?(BIeMS4PdM*7u5vruM&Yz%?qic3QE>jFnd6TD4LwqB;dK~Z)mJNdDZw* zlVDmV-{qd)^XK{bTQcR?97-`HL8Z$fT=gTKkn`xJa&2N>y=a`|M`xyI zUQkNj8N-r~GkNEp%eR|P^xo%Rwo(~P5i|{ZcsK*kWZ2#x;t30c8=S-Jo$FNI!6R=~ z;?I|7^eDU}yYOt5t7T{1e}9Ave*QYs4BJeRcTQqxZiWsgsUQEC>zPz72&-%<>6$aV z6`FL%OrZ{QgJ`x%Y*@$$lk?Y3hrb_zCw}!e<9d3@Ri@(B_`fyPj5P$7BBeaoAszUY?cJE_{35c2y49NhuLD7YrRm80&_T?lAa8{f7?ja z?sU52>E=y~rl2n2>4O%jtEeT|RmQxqKH-_)b%1B%*(W~{A|rRmD4$r?EN(@=>SMAF zE*5hjx14vteChl=aqDQYMaw#^B+r(Y;5UTTz7Zxg(X7bd2O9hxHFYcO!-WT7R=b{+Vgn-~8sh=)8=n|*ZWcPIK) zh-v2~T{zW>h!>r@^Y+v1+blZc5+Vv%&l9qU7f&HO8mn3wGplJ6nXO}o*{=BlR z!Hv5!w?EG$np+Z0Bh0=DHQV-VFDunIoy>b}mbEzZ=ATv=X*L{~9r~xAWq!oI7jyWM zzqdeLVq;Q4R7|YHJ%?yJl{p#7Efxx&nkSo0!7`^x#x>qYVb~p$h}BRopPvofM)&r_ zb9yvJmElN}Z&malr731?jh(j}n1$Gm$}?Yp;Z3j((f(FtD^G|MJA-R^v6)!k9%!_rStcEnQNndwp(Z3e2gaKmk4}f zY3fvd`Lgh3CSz*!$ayIQP>uMdX$rhk0x`C675vmuNC+x}YqJr33{(9(Y4m;FgoG@BltN&0>N>>m}!5ZVJ`4v?}_UQwvi zF?99bkL05kp!dVK?6&4q_Drq>-&Nai#O7>fXagJ;Wm~(EyuWF`GD#5qr%v0dlzYoa zjz$4o+i!83v9py#XVreCuAZ~T9Obge!?=#KS*KUvXbJeS`){y;P*a0GApN#4Yp`fj z#B72j6R7IEwVy+Jt?8^gHPpF$yU?K)Ltfivo5x;=z5LekU|cV8r7BC({(0yO)^spc zXXts|rx!M5*UuMq9lPHYkv14z$QeHOh5`|<`#^xp1t;GtjW)NfU+fD#d!5R`dfD)G z$X<_^J^q(k9zFg#F(lvNnJGu_;cICi4KAWr;)-<09{?Ju0&Q|HeW1%q!C^iY>f{^J zsZh)be&@l+}rs@bsu4`gibP+y>rFkO*O_tzrq(Un+NrrbG9f z-ms>l*VDO@yweMw#$TP0X9lfDm<|~ABCD9RFa6iAa+Eal<9yarwecyK4XJeF3f=L zk_1g<_%BC0fy*mM0|r1qgNr~vmurQ1LkEdut;F9!v3zmaZOQ7>@AoW85uLV0KA=Eu zlNT-4D#H!xwecxES`KGW&l&AP7a6I-hUnxXXubuuTsEPsjIJ2VD@P8r2vInl=v%$c{R_6e#2d2Cc`>VbTS=7DNzAul0N`=53*xrE}NpxB!1Zl-WS-f z^b8x8?B)etRxug;m?(CbRaXktz`+@tUqd!JE}pz@&25!Q+vqYe?&_&UdHfKpvC6b4 z2^@_H(WpyAh_gE-?HkI3{$ZV$yB)zKQiy#JFvmJ3b3{RK8q7|o>ZI5eLtSFbO4muT%xpyd{s zTGRk`Ccz&Kg6m$EmG4+vh*tW<cbj zch6|@pP@3|?fBMFVhYLuLqlcQbW4kEX4*KE3*7AQK4Q3e)%m!eFtJQJcnQy`RZ&JE z%yX!y5;(_Jsp7BTVxo^nvyUfQV|_L{A1EIY48AisON(gJSL{#!QOK_v*WYKt@dWGO zLpB$ek`Y5Qm>ii771AOD{Z@us`ShF$S$W$5yHlGJN9#(Xm4-E_3;i9C)M2n}D;3V$wB#!<{k+9?D%6ToP9g zf=yAXtn^-)Urk!LqfnTEOTxi(?rbt#6FKGB9mFELuM*yZFp;FO!J+;hhU#nbsFoPM z;lm>hzD@M1Uqi9mu&nmw9Q_WSBh>5w{oO~-xz)!X+D%rU+O76{TkT%9c2$`!SV8BK zw(hf=n!igy`RGVS4pN57xqB~$#SXJ7oQt3`F%c#!ln$*^VjhfEu%$zO@${cRPO8wR zC*cNbyR99W!VD&qo?9CrZ$CpGJhj(QY~0~5b$g~i>iIK9(+~e=&hOh37DdgJYeGDR z^!OOpO+304{I&&@vfjGidjTv)XDZ&il**`k5j$uQh48AdSbaXaY8oMRfBO5QMe$Of z)R>3%pVv<9znXsgI8Jo_Dz|zkVs1B8zLlA_>KNbZ&HNSjq2-Jd6oNfJNfrvpX#e=< zW4z4c+$UaTrd&G#VK2kyr77D5VCSDJ_16+J`jb|2IgAcEPaoTraYf5;Y=w=vIzV+?R`LgU*%Q=!k=qZB9)i?)v47q6W_?3 ztj}wz4LbJy)TcjF%_B{Z!>^>^6Bb^AKVBmiZ1MWa+&IBFH*~e|O3_Px(_>)-NM69l z!Dtc{(KRAe@y+O4*oDS-bPW#V7Rx#4oSs~qGo|5Wt}9N^we#_9M+}BqkR~`y;YsyL zY1@Chp8WUk7l5z1ozroHOyg{Y1;OI3upR|g+(|(+{!n@_`1;rIfvAU2JhyggQHEwa z`|;z8ohO1vPhI@H&hZYWS3*Ns*j)W9%sEQYlXzp7zU3z9kmlHaGY5P@66fbTK@2>=eqhK_cogH zSyQ#7TKNmiIo3yVd#jA#8J8DJ)Pk$sJZ}_6;T`JJP{_D^+W?!TEByRKLS89_hxY@I{BPM*sR6QggnNlElc;gD+zYk<~_A(J4ciD|hz3j$e+fLSAJhcWyt9GIC zGm@=*p)xi!vq)aN57J_T2iA+Lnu0jALouFL_+r5)=~u+}ka&Klr!p7AniLH)Xr`u# z1)P_CK@gM(+TIh*qLG4j^v+i1LwG-m#F>wGl`o70Je5fwYtJ}sg9_VF;%D-$ z0~zl+Vcx~i?gHp81C&qQi%_G$fo@lm%Lm9sY2cD8>O&BBR|GA9YKilVjMkL1 zSdEx~SY3FZb#W7kQ@^6y?4nMZ2uw9?(ez;ysn#e<^rA#B_p+zdFMkvB>qi@V6Imx0(_Ewgv>3|D@##THL|uq01D z`47q!NLasv7^6U8bl}k@H-EkHr%u`^09c@)>OR1e2#{Sc zazz>tDibXsLHFr_<@K46r;4lvJ{p^)IR-)cc#72DxExhnYI^fWJ{OW@Dk!wcOFJ+r zcqy=_Uvj+!pjlnw+TbZufYS!tZ_S_D!13+Y6MG{NB=Y-69~pRgy~mUcUlB^thbuV1 zA3CCUJwsC@uEVm=R%omp`YN+XuQu&NIf#r6AbRAQd z>{%{s-Z{dg4azD4L}N%;_z0b<5)DLw18#Dz>6rSk{69T0kq=K8fJuY2YDx z*G=Q@%5V{_uHOWj_O6TaXy2(Rv#@Pjk1b9ylT3YT-JrNAx1Dai-Fu__@Hne*IR?DFV4h|Zq~aso%; zjBAx~NX*gkhq-j<$7oHU|Dlr|I+xBlV21{Fxs4;Fs|ix>NkyK6)~8NMpYcamxgGC< zK|c502{;w4_#T7zFiGBYO706|V|5)L;-c30dKoOxBM^SlS6Mtiv%NvFnR-w z&mNdufb$HMr%laY%~eO5Me&hEHuRw&o&W-Ozc2Tl1KXVGyfbj%g&QB9@&rarA3ev< zhD~dkVjr(mpXe)p=NiIdgG(||5SOZzts*4zs{2-r!={s0Sr(h0SwCHC3ZlGMEM zKPZe;ym7@GQ`@QN`pJ3}^+G1~sJ_UHFP>*3RwA-XG>$-w0%f@U>}V@==rM&xGt*$j z{uIYoID32Tf79^aA5=`-9!czqolCo`#EKH^jVCD7d#DB`9#1;$dpdG$O>k^OH%<;c zv=UP_mGZzyc=MFjJ{iYdLgy+U!|?W;ZLVqAk8XL{(()+i>=pai8Ucao%EwNX&7_V8 z^J}fb1JSn&o+AZb;zVo?TK*1D6*d~vd1~U1)r#wxL6Aa;^3!c0IA>?y*k)UU~ zsK%f} zif5nDTlHl+v4)^nXLTL&f>^A52$Icp=DwjN@Gf?3CNy>^T7)Rnbg53$3)QEj(2Enm zYAQ%?b;`MWz8k72>vZamdp(g@#BwIxH^70i6?DKjIEK)+9s&9)2sZM62oY5h0N(I3 zE1!sh7UATpNPck7UwSU7q>B!Og(wTKQD~?)7S6rM%}P8@&DB9p8d@Bn3~GMGufzpG zAa%bPnB|&v5NAVA8sXJ-Qh&xEtel&5;$J;Jphf#bhdu=Ii8R{e8_N{pHHckWUR@5p z$XqLvG%vpj3%{eYxUdcG>w<=Mr@OVw`3-OYS*Sn<%ip&6S5m!R%LgObh2u75N|e>e z=F$Q8Es04GWZea2fdd7)#?}TPksI6`?N+1ACZz+RQFhl9hk8;^Yy83Y7z7*nA*fga zo2$QFNog4xM48fAkcVbmohJts68g>grK^;kQr#Q-I0wG(@n(t(?=#YHy69iqkIC6_ zh)2hz@BSG0z&&VW4|OUTJnC|JM<0BUEFVyUL9V~@c;SFrIL&8;12}LX zqv+y51@>Q)FOYdPJy8~1_A;?)7`n-;zysvA!@#Hs=Z47AW{+nQ%Af9l6Th`#H4Jyn zlp$SxcmCcd{lQn(OBt&hvOqi%jqH+V?#08V(<*^H8lbT{~$O{)k`j!yZxusM+~A@$V^?Ms|J&iEPTC zN&Ple_|QvuLBTw@EO-immWdlzmJz8(cP?i{4i7w&()2Q5pWMySc`wlG4}&3KK9Nc1 zxHBILBfwHRLZS{qPh0Meh%3cgjvw_kOANHXt&GD;J~fQu18^$G%SF$Dc|;;WV$N?1 zPu}lRM~2w&?_U9Zp#$gR$%+HSYREhX5(VZ5U_joN5lIjp;P*m-6zY5A3{=?UbFkJI zDSR+uj=Az9d-kf7!ABj%4#JV)u56cCoXDHJJErELv#`kIh4YfiOM=?hp~cAA+6nYb z*A*6WfhR)`x#p z5)ixU$QKdIYungf5?8M45rjT+D}pa*D>R2M0(2t*y-KnBgNY;ABCk!oT_9_BQ+ z5H=HwHn{(b7ti!24iTkyfmNF7q^x7CumSSuMW|bJh4M|4tJEnC>`}(dq+8I`^0_3h zKFkZ?afY8Fxvxr-)Qrn{a8ge_<$3b==F;!4+P=5WeQ#rZZ^!=VH2KjM21#$U)qNN} zVzqXi^RkC3`q&;%+~)f-mO}bly^tLVFN(AgS{liJE70Mu+w+^y(GD z*v(pE^W5kNZHyt*HO8hD&DpfQy#5#yR{*sD-jG#~TKOXY~45K*snli+P@njctec7~g2^}81 zdu%Uufc$x5ly}xAK(T3X+oEh!T#C4$YW=0mQLZVNGM?Q2wg?Ox3vDT^90?8RTH^ux zJ&&QuOK$|h0*mr$sXuW+(H+O{;bOH>K^yIl&Le4(igA%PBMYhgf%vq4vB;A&PKJuQ26Si^jc{GQ3BIL4 z=tY~zZ2eg3PNFrpij}ynIF=e;(Im##3_r_U$%)j^-Cl96JeKGhb$^2O?RfQxy3z~X zU&TdSdR<$e{h^O@d5OF&wY=^A^VIW;pUG`HLmX-H;m5z23Zg3$D)z&J9 zOX&iSmp0YRxCFVY<)9a$yNVFvwrN~gteo+^FYHm-zuC{Xj+Q)6xdbHh+>3wZ{Z(T>_80cm2%Uu;(7tG7His-Tq*5yEda(qlEDyS{eji z_c-_RtHVWut^3hQJX4JtABv{m2pNwgdeOu+df~z8pWo}7Cb7OrOgB@?^NkW!;bN+a ztlp|6`W#mq>}i{(-{TcxMh%VOsdkMRqFRO3GZcJ0Nh~$}q!L`&>^=GT^>q|L4 z)mhY6TfPiivm@OETR*#T<5zNARY5O?FU(pVZUobuwYhiF2O~UL`u<|Jow9wsgVKgx zr5Pqk-yS|}OZxt#&*ak27c1oCKLJO^$HjUi2QOc^YY}PA{H)+_yR|E2qQ*sz!lW;d zEM2zu_F;!?MsBXp_)iY)!g`@_GM|Q(z-&B;xtd4)XK<~CgJu$JQ+FCUGij3~L$oZz zA$BPCa7%qNfg59~BDTFe4UD|2t>wtIZF?Ez{r0ZQHi_C2HzyCvs>yX;bf?i(&}0OG ze*~l+#``;q)O)-ZFI}M*9^w}!fw5%H(%`W3U-Ss`EolTB`E`^f@0_r3c!Hm*yoJBm zWOMKH01#}0ZgV}gUoa7umF4yu!7mZt=keMcBCptG1O>6gOkqET{@nL{!>E@&VkWB# z{)3d_Gb38q70A1h8!Iljj)@SM0S@ZgHj*CK3!^7e5j0OHIea;2mk&fTR+PMxSnkQ-o`>s(YYDr==*doch>=pT5{k!YWx`Rx1Pv)O8TavV5 zSQtrJ4@8D3ad9aH7mwsa-HNKPl}sOo;{u234l|vrPAt?UUeHvxz)aVcEqbDZgRI6e z_+}Cunl)};YN@GQH*DmnAmoqkb6)9`GmG^rVogVMpts`Z&!11$g{Re= z?iiH6?<@^h&m{S@j^{<~Oge;=X&=Y*pj^hhY`JG9V9jIzoGL*MRentIA`wh*B`%_cTBWy#=f$q1Z z$n-~@P?6*I2-D$N8O|s441PPsdp;-{mpLUY=7G_f)rH(wA9}*#np!@uo};1`5EDj8 zbP6=nPrS9NTW+God$0P1fSpaa+`CWSe^^r=+V<+8YR~y+40#rX|AR$PU_>TxTbi$& zjEkTN-C_RW@xWEYZn$#Dwp0gxm9#qj%^;{>@Iepz%06Zw{Eg|SPdBE!&kbA**0j0K zfU9&O=Cd--hF*7w@V&Vkd@HBK_2xz+&iW`v6g`a?AT2V}Z*8KKkNUzbq?&CQZh&OGsqDDEMiBWu3tL_cBv)W;JSOHtKoj-@|(n3naFBge2^ z2NEH#{7QWa4!X&07qeypx@7|w(n3`;iCfMb+%jTEUy$mKIh4##);HJ;;H)ckUY~c2 z|M>nMM{Jd)WQf;<3dRBJVm!eHIIObbu}{)tYXglhoci!_>V-*oP5d_Zo&K&5Z^#^R z^=ZbTej=ZyJ05eUCfr>PdOS03kZ8@Ty>m-eIo(H@)*nQ;cq^K;D3lw~B~5kbOS0=8 z4NmM+S<@H_Z>6c2+b^nnMP)b6-uDwJ>o)dLSWW4_@B3-lR;D>Kc${R~HySor-+DJC z2`K~1W##JSM5f>Y!+TS~rotaPtq#ARmbF=WFwL{rqsvS0r%{HTTY*&g^n!sR`QBI5T>oYW z(N8kL!~RPXRW1&MRefUPe&Xz}IR8Ltt`1r{e`fi1w^4v+=doKZP!J4*Jc9~;>;n@K zP^@_PRkT;1`~4dYCcauMu(fc6W{-K##h6CzCOmdbjN#-<;nH_zBgZR?I&e( zd$ys?)sB2wDVi2+qVbu&OU4FtVn(ar+K1zRD-OCaU>lRQKr-L-!^?XPZ{Rar(&%Bg z2Uvp$$=fhfuHKalzZK`l#3aha-xL2z(}QGQ*2@`grnlYZ1c=!e6H+@a`t$!9XJ1E4bztf2fR&|E{wUREG2O!ZZh9QAT)66WeVnAKU>S_HcN1ZnPuqXf#dtKp@{thY z9lrnU4dL2&&{B)FKFO^X4htYnMUdQ&Be<07xMwY{_Je{1^r?X1Ii<|E8`s4IOz$_t zTv!Y!RqM8MOo}EGT*(E3I8($4mg|LNAbuzXU6lbRK*-H7@Ihxb+e37~p$Dhtu9`z4 zUPuI|TG>#t;8a;u8l3(b-Vf#J2a)ybiK`&Y80AX~yka8*je?re?_IJ1LGbiw8=$?o z6W9LAzyV~9PD3AXfJvPfLsNU$M}0y{@(w2tA>}j*u-o8-c=CbNO~~2KZ^D~7Pk{8f zhPwBwwc+sBLHe&HRzYHSdQZc_GljR}ZGa!!87SwWZ1r>&JVK9vqF@66 z)n-bTPbIP0IxLk+wpuj>qX1P52M@`j6$JnZhR+yK2+o13)Q@gXLo*G@Kz_utR~e0& z!GbVygnsa}jY~7dEHyoPx*ta|xhyNrRh6l!eoX6fw~Z`3k(pB)z#WlYpJapmczLjf z*&a@MSV*DT4`01^+)hBi4y?@{JCiDu9iREw^+YSM$kT7{cq(?R-kGD`YFxAkCm63x zV8WJc3v1yV2fbNz3b6`K7`vZ5klNB{I6OwlLP0hBnv*rs6HisaQP6iN%0Odgk_`}V zp8M9Q3za$iZJQHA$x81Yo*kmT&2bnFZ7zh<9)GlvnPyp$DPWmq*#6~oL^F_d>21nE zHbACYsy0_uW#Q)UIF@we#GR(&itlGWYdIe}@?3w!06%ylpyz&QduPjdR>DYC)S&PQ zc*J)^2{<<#53VxE;yTQYu7M|xQLquO5bYYxKR_E=2n_qzj)u9|Q2Zrq_Yc*BQ50f7 z6d=_Zq21h?fr3FQLS2%KN`;f&C>$dLLQvq31^|F49{Lt`{Sbh3PhptaaHos&tJQTB zC3*~K01bs7xVqseRBeh@^k|X|D3OS0xb$vTT{SwZ8rondlP0utz+t15Ti`QJ$Bp2w zWJ~ZAN}d?;@XGZpn#4b7Up*8!z6%FfWqg3a)&0eznW)^dgcEf7yB`QLh|7jbdc`T@ z0}#|H2j)2vV*nQ$B-(BrW@%vE-S>7F$~9)SBf$}Uw=H~iB6%2ETgWNRctb^1u*Cv; z2i&R@cUbu_d<+o%EKLq(OVmmEn;Aib6wl4JpwF^A$EcT3urSL}Xaii8e$Oe*;tx>z z*_jdeR#dOp!{;pzIdehcCg~lk#^ree2V?O4$AT`m*_JYgk!KX~=3?b0oXK+l(X`>> z33b(Z#rgBnOe2MZtc>>#d?YP2L-Q<8Vf0rg!qRVNCTq}s2fIHX16)>x0*Q)MaLWY) z{El-khcRA{PI=qyIlWZOk~!qrOHr6EU_7W3d)fv%<8wHAj{?o~}(jPw125X=xIQ`)$ zG)$6>p>eyKOGMyx0Z;--W^BG?c*!K4ni2?ztDX(>Vy6D_8dG5;7YZp}<$ABrJ85kA z^)qvs<)DG;xJKy~^?g+-P&%9bn@mULixJ9QyyL9F=h^SBbgcd@dzFDptrTPvoHS*^ zfl7;sg@Dvgkr}DE+AyvvPOj-*O&>~@vPf+T$kYZT({9a!;Q$gcf;N^Etw0LuG;%s^ z$L5+~XfI87HN`4eFw`o|J2+i+)iNr$GBP+lFF19XM&?*W=r7tc$H|*)iFR_@|0uc( zx2E1U4&d9^V8DRUJy(=;2?*#$cO#9XBt#meWI=bQfRvPgfJi9n2oX_2N))6^R7y}p z-S6f77tT4?bDisczW3(~Jurzy<#zoMxNg&(Bt<&BEQL$Ci3!s{k-42ovIEysMIUl{(rIQRYY*7v2>@4~qtEd!6o zawIq64UmY<401D;sF?p8!QAZDUyg_RnEV(L|%fKGvyPBTg!!t`RK85)(8^W%>iZrful2 zTR=CON@tY_5FJ{d$4H zaIGi#qndo;fbia-a3TU;H5Yu?MNF=E z0_$mXPTLuFWd{S3u2eo`a5wr;$4G9a?%wT6W`+m;cRTKR*#ueG+&bCj@f#SVRhL0F zI3Zin(1gbWnUUp_7wE_mNKN280JFY)FEwIMoPGo4H*ml5(U6$FIs(jQ1d$JpxK+{+ zrwc-~L|$6f;gg{kLJh&zeA;&rM0z|_2e`73D!v`FlDlsqBT0QWpCVU2)S3OBk>h#q`|M`zs^n?TNPRyz; z->`XM@=b;=WZG$L4u9Sov;hxG3yMB(pt{5C4QRM*B(9IR%_x?s{Uq}^qsXl6YVr_t zm2s2~*j`=yNI-4MD~XR;l$5zw_3u;nzlGQT_IyPI+vsafIa!Ip7yX~tAOOKr-ApZK ziE+JoL#m$_p=TT%J!Jp*Bc(JGz7cY5to_>U$Gpp&sj=U@F{<4#mkg299vX@AAl**U zP$f2sMYogO>99Jpray;p|977Ney&dbxBVaO*MI*pBy)loIHY_IkOT2dybAt176e+F zR97E96tgQhoK6%_3Hi2jh#AQip7^|WWP%;dzi8Lt|D^-=AoLuzY4-ggI$m};@Wt0d zBSewj_ENzcOsLpY`jY<>>gK-0OoLVZl@_s!oO3424cG1_*m6B+aekd!K1vVitM$7T z_3)>U>&G7R{gkIn*P0i577)Hp(RsCveU$qz-j=?}@9PgH2D1G4dMfXkZUf{y;Qg(z z^)7oNW%n=un?JlfZmED~+kUK2o$1H(W$d1^Q-}9->#aYR_7-#jB1`WK|QZVaLZ`8;GQ|>(3z$GcigUp2&n)|T_OD|}< z%AnZ5mm2H8&xjYyTl@aIO1kV~F<}tNm+)RqjxTRoQLl_^w1XgSCL{B=wQ8gjfwam$K-AwMna~Ez{UBj$XKO~s$Zzz_j5CNzSLGr8#zeS zWk09WH4&bri>u`0*B&z4-e|i~*SH3*5xl(>IWBpA71~r+f(vGA>8#`W^4H~$>eeqd zeSI%&2v75N^XhvR(A~`^FZukp5%ZwRKKO7{t&D%wk2JSP1wWa{4IPBzrgq0%`>-ZU zQdwBeO8L4z#r9}LnTGwaPZSLWN_WIhOK5XK3X!Wf18$!LDPNH{q znWO!>;Ip{K%^HIQW5n@{snzWHJ}Y3w20Y`^K=jIB%Hb9JMg^Z&m>k7`x3 z2Z>gR3Q)9HNtwB*78O8O_AQ^ED6hv_W}t1?RG-xu!y!`rgB=fbH_m_;IrhxNIV^=- z4*F5qRN0@f&626~RO$vM5C}=%IS>@Vixw=&r*lEwuQ^ca!bW(3*5J&-;YpJG6?(JX z`H*rk%J@&EBg2kz3}n7D-A6QRN?}_0u~lx>7Yg)pL3^*lf~DwmfzfeHobb6 zV)>L1rQs3ehR--fDKoyQ3#CfeTpml8x5@_H<@q7>^4pH(ENH61Y@6TEzD%#xLopqO zVN{%bqrPa7=!7e1`b^x#Hynkt%6L{cA74zj9NX>oZrBmLpZ}&TDpIsWe99{%A^m(? z0g}?H*nIKd!|Qda^A)qUkr9tMd#`O2-$@>T<_CRs$D^|pI+;ATRoxu#=-F_6QhMAw zTB5vuNnkzyn&vjr4#DT^$nyftphhrr%fP++xTVwHN4xt@ZFGe=k<~X?;pE8gxpNUS zv=uDAA0ZP~BgxkH2J>|8Za^p@fADJp_Va9Wm!%%`p5 zqjBUCB9xO$^qa4FT0v0;JmE=t|JJUfN|vzvfkA2X=Q}pPo&?7RiS9Ui%NU$af2`}y z&<$qx(dizOSmj`Ho6DQzak48@8!x+|S4w_jaE~Y&(QgM57B8ME|B-%bl$>!`Ffy1; zVCB4-`}y0<=-cH-0!HD1{ns(I0D6TzF`5W{Ine`=wUrS{SNrBeoj&Ah6d!B)o%GS{ zcpU)|cu2$rn@L zz3VyP@w|>RHL@PYOgZw&A4$&gR&f(r^8_(x_TdG5+j$u7NNzEm)TgZdF&Z&a+Ef+e z!Iv^Y$7@sE1~7RRdBOPIn`@Ai(C2aTo*;l0-qLeG(!}PT+Kl0ht3m z1mbm`wX7Rb_aPZnRxTdQtPp=g zj(`Mw0U@_-L2)CO5IooIl_LCIUkl>aV!EK%2ND^AKDQiDw6S}cem}&7ujM^Vb`#kL zeQQCuuIITFYcaQs7vgu1XD-CEaxEiXxw3X#VE7r|EtN zVDF(>!L{<+8bzgTdK@E&@R9g>mbed3s%77#fdR8u)VS#D5QwhjvV2Z`|I~$3D@3Is z!WpMxn<5c}hi*{q88f6KK$mXRTnfOb-0+Yi3@b)3rs4*bZ`Y`m*6_va+18K!D?*^JBe2#HWYUb8s_Y|0Ok?AW&P3d2plqHBqzT;h8#OV^Npz=k;4)j1o)M14&#d4tH z#Omgp4s`?13ddt6%k=+mh5|K@p)!>K4E6z8i>$qyq_Vi7=~JI{1Qmqg<*1(Ucis_u z@vv7z1rZ26XWJVwJiN8k((?b4r-jKN7+;MZaV$Fp!5!P-iin5Uj-zjKxstCW4!Ynz zRBf5!w40rAcwb%v935vHK1a$Vq3P8f1>aEB9hMmJ;!m%!oz&zmSFscE3|hhiP2PHO zr!(C|N-yw>aN`Nyi(dy&8emNMwzc`sXBWtC%gTON)x5%LGnBEKy{8|diP(5PC;5%& zLSt3Z(j@Q{bl{VYlKVY5Pg3(>~OyvkYlY$PG zHV<5PGd-IUZl{U&yj7*_HY9!}aJmx~wkTVNv2xajYkh9(X=pDKx`H}#zJ4Cbr!4!G_=D9X)>%H`vkN7veAy*0(KXfD48V_u{QPSpGaqqx4i1ABd5FSQv zSfe~ZH~~Cu`}q1In@a*!aZYE%eV4^&cd;0iR{3poT-vVFUtn&;5gfP9C9?dpWqGYO zIY9EX2dVcCwcY>yI-HxAN+sHteDhw?dYG&QJzc}IznXtfm~AS*a|@~OiO%1gvS3MA z3MSKY39mfhjLW1~qMAZ`(qFdR>gsA)X>xDIN*awwf8LU@nU~fB_ea6-%qEmQtGC!n z9FJ=2a!`v2mV8BmfpN{+i5_pA!a59(VoMh!< z&oLq{oaqtp>RW6XuJ^{&X~o1N$om6`2_20C{9+(9L6&)GRJv_`h9gVE2XLh4yj4+d zeZp(tKPvTHJ3U{nl&H#ER16Za&|)o~ulxAX=@XyG&b=OK!b@p^3pY~!r8a-t1#2A2 z_E;qdI^#qQUO93HMfRL)NpWok2`@e6?JOurPjB-ANH4-Cad+r_CrMf3;4xO489wk0 zVg7ISLh=Sck>79LG6o# zBxL={^@-1*_!QfO^J+^DAlIUkrP6>KY*6Qow&tlJzbE-Yrmm*JtA9(fHXHlb#UF9j zH`@eF+r$i#f*?|%x=hKGtM80#;d)H4IGfHR>mV}w!I8Xi0=GUy0i&pRH38fQ;?HYb z7^^j-YV&jw0cvf*9eglB!h4DKYr#Eo4@vx^@BP@yWImo9GXKfGgeS)&ay55O38ST0 z&u&IOon;JUdpLOz!xwiPvZv2SCmI)=9#oo`K!rHtpguUT901zfVY2}Qcq*}<1G}C| zfDO{sMJ+us67TCHj{kJM7>_-U18tmfdOxi67OaVSNz`E-8bsuLhK;2vVsVI5y7z_& z?4HMBz2vJtRO1gth5~@|j@SIbn7a4Q(W($|HT8Bm@!ij#=M}YaUo%c2nTh z!GUeUWS@}%NPAElOGSk?NQnZ@CtvM+Y%|hfI&UW(nn15KAP2?qT*qPhkgggKB^?~* zl&=0<9aX|81qCx%P&BY)BpeSW#&-g-2BI`R0Pm?blT@gp>uW1x2zxE*YBGqf7z6}_ zLSy)W@$EoTCq0xGxYGo%4+qDH(VY%T!l|s3FoXf=ssZVRA(h9EDs7M;7C~ZGPjn$F zx~inQKAVyQ+x+JnA@Cx_WQbqwy&`)<;*?02Q98iKNilW6{#*lSw^e~Z&H z6wev&4HxC}XAz`YOnlOO#1DE{%I7%_wc3#{NLC&*L%oWg7(1e7JMBd&+T zb>!F0?${jsGy!B7Jv!zzhrpay75)s?@`RXbzmogAYh6aCPlq!%e_oEpLtx1iE5kq=T+nQnFbozL=m&SkMZ>|wTcnlXhFWL*qA5*;Hj79g3+ zPi;DX4|i^o#Q+A+LYTnA<#dmE{#@jhqdqsLE|0a9o4z`4pm_l4x9+2W%LYgOXhSvP z%+n@XV&aJ2$=wq1fEIn)>%l*Ra{ej}42nel?nd*r_&bUT4=?9AM_?f73jBlCYo)ib zK(37v+t~9lT2;yfSmf_ySy`%C$>?;GOMBoVt9zl1%y;cKZw`Z#;>hjxhr;affV6(9 z*z`TJGa#C~)VF4jaxz~yIn>=ELkJ_Fk;D<$E|R~$L8RCT3u7uWeR5a4^s1>*yl*(51lP58fY z`S&^CG%q(-qwIdn=)Q34F`(>f3<7p0mrQ&2LBm@8Lc@Kz0DZ{*6}=|9yOWR4TrXrj zCVCXfV_g%!x24c>awTf5WgL8C5E^UUlkKLR#D=;QKO8@%N#vCeSzUXylo05Q zh6CE(7aP!Z%ei*mj?3vzmgQf%_ycyslz2}i_+tFpYKhdw2g{6Y53k(FZ7QGJjhJ0GE4rU%VQ%bay(a z_2CR6QXshZSH_4HpI-)iY@IDpf;fydfgndR005)AQGV?2yv!z;SG|1kmm=0eq1Fdyv0`A}G&Utuz^7M&%S=uh4Vt5brVm{x&^v z^}m@$EgEx%{EcF_fPM=uYn>`jLsn%9IvGpYK{1Li79SOOFBw`Q@K4q2*;TL#1AItJ zQ(Ie`iAYf}aDtJ{Y#r-7d+k_90`t--^81s%|3JsK^KNXlwdsBS==Z3|+=F&i@~7yn zLK9V~8d%0?kjD^5QY3vhk^7`G1*uXA*G1TPcaC&x@C*h&D6DuclI)n`WniFa`X*k* zsBF>nx}^encsxJ-x7{+C{ipO>;mtU`{M$O1v3e5@=qkStg;bRu8Um*#PnpG$CHMXY zzZb*v^-%NmYW=__`5Y?`(@o|;KLo8|79Z=q6|T(PHzBCL(^GLJ`l@6(Emd`z3A!QI zXoB}iar&c(e zE}|LmK044NY6AS6EWBXM7<5dK&hUBb_+d(rt2!^9P2_xr)nLZzZ!e5?W@6HTg1Dj# zz(|2r-F-tX_CG%>`dq#`O1&%myj92`+S{Tdvc^&~T-cz^z3U#$l)u|knLRzEZ*w63 zng6{HQHb|TZPI5$%$M3zJ++_uVWFKhk1E`n?v)MtvidrUR@B3NG~+m&NqFe^YuU|7 zeMA6spw-57AX5M)?P3j|Ni0#>=oRt)OPvMMcb-NMtFQ8#N62iuJ3ijlJS;J9IrQVh zTO?~}0Z?f0Q6}WPkmN>Jy`}3^Ee2WBS7u%UPP3~ z34q=r89BG>k&fsgS1ME|KW)%|vx=jmmu|5WRWGqFcMMb%v4?k0ZQJgUGo2-*fK@t#*BUMsyGNAeEiq=ON$e2y;{%6TZ ztI1i`2P7{>ILPNpmpZv3?fi7*lPKT#?w`p#aEH|K!W&m7#r(FrheWA-PzOCd^v= zr3}^2UjoTa4-Aj}-C+D$9RqYx^jwG;x3SyThD2gVNWthV#3EWR zpYUZ?CnrhNj1gr=jBAN-6hV8K%;P3U2H?%cG)?C+D(8?qJ(pR5Uf_A%3!Zf^uj*at z1*gIKimL2%!PUMX#t!4gp%Klc1v+{kBuGpF$Fw3yfjX~%7O<47$yA$9d`Qkeio>NM zfVfNLHHhSx;nMrQHQ*iyMkg5u#ZD#%8sI=TV;Q)1kBOPknW+n2s$syl^;<$ndsWdZ zCKB{jzGR0GFNv}BvpNiDn@Oe0T-r)y! zlkA@VT*$10!r;Q}(~%vbx*jZRLSKeo93_fbP!q)i?=8%aYt`RZPQRfhi*~t@(LJ&t zF2=H6NFYYPs!+&bY($P-IFf@@bDS{I$jRsuV|n_8!Vu;(?DU0+~HNzrs^3_ z6KpHl_v<%bD}mMxLg$pyIX>1s+5q2a4L%}}&-1O@?$Fmhppb$O@!rosKK#YckA5`R zY(ZXykEy7`#B2YGYOVwgysac7h8C~Y&Eb-hPKYqkG7?K3_0@D0BD~RDcsH-7pAp($ z1GjOqUm5{1C9Z@uWtD&PQu*`oeZ}o3kA!zC2y{&Tg!GGpnH>0r@ZS43&+}a^D$R|r zgHO6dd9Ie-618F9;5b?Hu--GW*e_o)0#c=&B(8I&*`iie}*BAu- zUPf=q-v2$eeRu!ve#{s1`)AXE{VW@?Wz#rr_Btoo3`Wl;rT$@PbfZ(RUk?Z zBoYKdO)_h&@_%wuHIZhZU(cLbjr@h+dz^lbbM7{+aQQTHY5&+*b2yUhKcpD z{q>)8(FpjD4-M`AHCSiv`;;iFtv{QA@)y5pZl}Rl*IE{I%2z&2seb|0xDnN_M&lJqg-_9<4sJe?!jrcC&cR(-Qt~_MUz@IM4 zabS-fz!y&rby85H4;ZrL7*kdNL~Z#nBRZU`{*T-4&;SCDZ#&PeL!MxH-tb`F$ada9 zc8D33*J%Z;QTZ!Oo-a_InSwu+`ZlAG)jp2my!2ghyx;y>FzW0Z&pU(uC9;bK9U^n| ztxT|xx=erAfp8RV#-vJ4h|=9^*}sdA;n3o`yuq+6k6yyZ?B&>(1#K1*TOq|rBqRjM z*nwm@Lb5BMxI<7h{tlGz6um_y#J`{A%^=r7+yLt~EeIrVV^f%Y2yykHfWxTb3~GPm z5c|(f1WHDWJqj+&mUB$E`5FA!c}l$p12zc|MQPDlnTka}xDd+XZb9I%=n!x56bT@0 zPhnQGn+M1HgEa=Z5X7$nPuw{|Wi$+O*)6D?>A6Cht6{5e0Gyd_2_xy-3C>4q3$z zpy~8gM|qmP=>Dff4#n_1Ws-;7)ta0|LWGXgYDlG+TeX1UcY#eeNzcbsYiS&%d8j@h zuOk5Yc;#}tqWY6i_1?$o&wr@*D{8!&R?jj*Kgy}ghHUY8)*!quVT=M;CBZxGis2oi zpRH|=LuP02M9Gv8m483XirHU_$6e9xU{9y2V%88=mGXAq|Le*Tlvw9#A;S|ZwWN== z- za<)RDh7Sv!P*m1U_u1*B2_+$$LL8!S%N!@^jxEox{ZQ)!nI;_Yk1JU*4w;$4%um8p zek*kmolQ5>`=L9EnXT+s)TKCAvaHq3Q=uT~H-Uc$++jN*!QWmfX8^4B^Q7@VuKgOc zHip?~b1Ui>=r`zY%}x3M@E^>;2ehiYbCo28>ni?!f4#5i8233by6(B7)7DU0oN z6mmTZ7A=Ija{d2Y{RZ`wqDw+ZZbzI7~|Oi&_sx!)xI zF!y5hxJ!3_;v9U7e`!<$YaWz7q!T$Au;2iAh01m5prB>fwsanw>VvKEtpY6OL&=qd zqPaPtuQnVOaHQ6@&!vFS^Ouwp+!LpMbZsk&>UeTnRjO;CvarvkUSnr|uY z*B;Mz1%-Yev9Z>5NFwy|+U^0FDj%aNR&`zS%#@{~U#p0*77cI|^_u{R0sy2bw6Z0H zx+aH)!l-U61ViYGFcvT*gJd&uHv*HGjX(sY6~RSP*iAb6N;06*7Ua*1Nyh^yR4y4L zXJVDOD;^ZJ5+`(1)=CSJ{Kbb484F$Ekt^olBW>`k3xZdKSO;0E2cVS_;3(3x`^})d zpf0O;d2M$5r^f?C0UhV%?jiVY^!~cOOxjNADGtqbWHn)1MY9tN?@CR>#BxgoC+Wel z)V|4Y6r(t3&)Q5g8l*!8hA=RNAlaPt#r+sj;;Up#k9Je|9V8~s6a#7=xI3~@KyL~- zBHf8WNA^;ft#EhHD|d5|rAuFh7Cog1JyT#3hY3$IF@^45vt6}}zF>F6(Wt?#eFI6Y zk7*)FAqrmwdAQJ*;tPJ$X}IPYa#~nShkfw*tw=q6JXFk)3m^wImVsKV+RH#Lm?5*|BY8ntWVt}S#VG! z*q$f~fs%ml@+GDClVq_9@u22PYF^w1^w1nZi%L%Qn&X>n9)dk7Dp#220<||6C!Mv(E34^U9lTbgX`Jqo0Qieyyy)XI6_KIRg5aa_!AMetSWXy8R*Pgd`HdgD z(t)*?3IGBMwdi(VD-A`~KGp`z46yJK2{wO$0*;pxk^(O77DKtV(X6Trv=Zh;aA`n) z8Rv#KYkK6^dqoAW0_oieR|ke4$w87||5Gihk>?1GYIE?@J)ZxhAt1rlcQR6mpSN&; zW_h;W8mLh*PA{o197R6UD}5xlCTmnz_~Igvr7~ZE+>cNJev>K9_(d6K_*q&{e0eTI z>XZ_&R5VsHUXVo!83*JbmA`~3zd+Hz9LOgx$;Es~Q2Hu2n9vn?nsXlycz2$or2-n> zJf!OswfH9FBF=TesJXv@^6xTM5pVbGJ&VCxuYzYmPT%qT?%K%9mi0ZoZlE@04A zzi|gdg`1@b@J>)1=9+eR)hbbEyw?JZZZse2x7ko0xfvvbeLgMtdE{#&dPQjEk1~@k z!Yb|M6O78pLyN+-m(?jRBsP@kGi6^?IdSy~CH)Z@Hee_^HaT%*aH)No;8o)~^-S4# z5lWHH=t7l-KYD&oqJ={MdWn;wY^r8eWV(o=KI;H4D0j9eWd82IDW^Y3 zh<%!8#e>csc%^Cj+~j7BXx9&C8dr-X^P|wk$L?x3pK_yjR}Z`XD)x1IKhZMPiJ4X9 z2zc#`x?R>DRi@9GDHI(Vm?JgD(&S*8S&6zWGsZi*+3;wgtQ{`73Ah;jJ#_NbE0(jr z8zBHo3Sjb3_O5Yc>vBvxaqX{*bo(0-EzObx{~V9p*N03b}IOL$PyF*EK5kK1QKqzCr1E%r) zU&QC?WF^_m&LM{wsk(otrA{lTIxtSYz1 zYT@=#MHS4#r5`ap+pRf~@bMGv0~o`9}pK_lvzdHR)$=yTfIpu$=@IcVZ+SWVf& zgLVb=qWV;U`Qi(4G2$vX2zcF1t1=;I_T~dbF4fL%?~a-4&<1KI1hI*icoybPa{q|5GiI$L#DzHbpzQUJ^V$%qY8%(dXekOS&^PZ+NBFwdoc!rKYt2e*T^n3 z-TWtFAO7hpUg?X->I&vs#PZ|OM~B^ie@Kt=%p-LgU!Jx6ccz}tn79INd#R;~`y-vt z!Uq(%Oc&mKG*I}G7^d@T0pfb?!!`KDmjgpBQ_&af)QnJz7tpS&=*WKtECNRW&~V$G z$kDz(iE>o~2{6ISF;MyHp%gAj$FPW2_WU?r^{Bs4h1!uE)V~sqFKi?r5$p6chVlKq zr?-`Ee(Fn6tRFAeh(0@J+;@ZrUJczvU3y%8Iedw$f^#F$gQ!KZbaQ;3%IyaC;QGp6 z6(;j=zJ1xXfUI}v)>}Qw4DIP%W|daw;%#w$o%s=#p-{D+@qf&!K?9*Bqw#l(*;D&M zB_^3-g-*{|04tQ!ix}=r!vZP!Xd3L)CEM-74?PWr!THs|eFgq6u**NSpJj@T3O;x2 zS_mCZhrMlymLe+2)KLVlW-e~F2_3;xFaJ3x7sONZoRa>wvh$`)MrpkDIei+DQf+E` zAuG1^Mqb(UvkU*tj76q&_h>Ar!|1;){aU`hg&x%%p5MN2yw7(y8UtZ67^1LVf$}NI zVQsFT2i)kjL*W4iR=2$j!psnn@sL-NLat(2^kS{^Q&B{TmLd1_;PSldUfks#L(=uZ zKprhGPxQLeBfhE?|5XT$KPOBR9r-CtSAo$hd(uB4Ti)Vyb5dPy@yIh@Xl_&Q7MsJH zS!pl(olUP{u$xDz>(lv)dm)K7=1`xd`H>ut5b~U~#XH|wb+w?6Ibj)L5;ZO)nW3WM zv@5&FU?>s4zTip;dM{eb%1W0|!zyu>fyDToeNd~0lf@e$1B^9ItrxNUUKci3B(wu8 zKrHMppYK}UHuRa#xvFWAE0IT+OaH~X_-NEWq&xA1mFtymzE;!N=+AM<1}iY6(PIocDRn!zb|3A2l!K{TfT3(`$?R z*>9`%^758IV}gA0X}Ck-WC7_LqaEx{;b#0v0e>rck#FA=z2L%gYCT^b(FVg#(%-@H zHHlo0-YAT&55LPQHHsgV`)`q>fj{`LR$LJHkKgSlDpl6;XoG;5!W=%VnVC!R56|8{ zfXWkZT1b7#mNz{1QZB(-qOa$Ns&b^Evzea_^!1q#^ujZdPwAY^*3iVBjww1L zU$j5LXQ7=}mZ_C%7iTTev0c%_-S&!)#6F`LVKFH^f0bc71~!J zH@kt+w#1i32&($bK~pBEfa%qU&4S@dfkCmVkgci1#dYI)h4~8Ml_+*;F(Q+@Az1u(CUl5Ng*?Ncgtrw|J+;w_Cs?&i`$`^`*t)pojKz%I zF2KWu$ioL68*&zUj4Af=^*~-7ZRw|mmphJb00P^kA5|S`y^SnU+00?|l|SOg?@Vlw z9C{?4OjPiE5!GI0^>nBsMk-#gI_yy zwc*mPeJ42OyhDGW{AM!5XdMhb_t|$0DSH^iWU$!Hhv^wpfIy2RXHyCulel`9jg^na z3AetqZOq>C>$piY*`yR*tY81$Z_AF@w_|f+ZvD5_+5KK zE8>6|b>JIW2jSTTT^bv|&>MwFj=%y5_IQOfEStgk2D7A~n~wK<{pPfiA?X`CqvpD> z^a-ye`3sRn@0`zQ4tVnLI?Gr4w^4JQtxsAU?ov1xqdxS-K4HA>A1D~`F_TJ%-87UzkA&;|7!`nvrOP6QYzTlYc=Z^q!bqe9OvIX)fnr$ zE$5^;0#?Eq&di=_)1A%=zvOx0AEYQGFD$zG*{T#nZXEA|9qP$h$blnze72Mx^^OH* z{wVtzv#qhLrbci*=oVjGfOI$|>X|9%{|X%7tn!B_x`PW}kNa2|rgya5W}QK~Y#68r zwMc~!GqK=a8N>_dgF+a4ID8HKW~X;yne|uC+q-*WC>>@fYf)$|kMk{>NYn=}kM!gA zItCx&$o2K@t4vR^U)|AsT;CJBTSCwF(J6)D!tP8vSWi_eZ>Z(VCe}-3xUDeR=U>dT z&dQ=ekS=JH<6ENy?=p>pFZ96a z?m?>j>N@|Wqm6t238PN^RvzZb3)jQX$;?NU3Ul3e85>B<2BDvS-3?&3D=(6n^p3Tt zCSIPdmhtyskF;>_!U#jrB=)My_(XUan*4!NI__1_S$)jCX%VMTXIfh>-FFrC{9Fc| zcoM83PQBqSD>T-=N+e!OW&&==%_q{A4{x((e^eR4GDCMEguBGzHY47^Vkpw(HKo&A zg_6Yg2tfBM2vt{Ut6jiw-2=w^8m=}%jJr9K7_uGb*N-PDqAd+W1j$n^pm>*Ec*wnN(!)>9}@3F zQ`LVZdg6WPYjnb)B9&{(tDU^S%^0~*R?lCkk6!j>{V^|-v=y+CrH48V_Uyz5$?TEU zNTs6$>Mz#8Y{p2WtQSecA{$h#5;{>sIzcjh1+qMXr=IKT0?ewZ?ruCNRk5GjX$n(r z1G6@CwI-VkBKsfc0+8|KkWBu%q}>R)iLNy9_i5|m>`#BelXc^g!H_FIeJ2R8M?n(2 zQ~)yQ?$<+BSP<){U;p>>h~3$MudDQD@}ZL!IhRzVWI}-5@H@k8IgIleI8_vbcJi4x z<_5*K;76+2sz6AOxbvjdN)Lf+P#i3tXJwu6TP5;hq`LbF{3n!MTg!QyO60A~mO-^4tE-FEuDQspOwH$v6MP2L7ELnN3F{BH1Qa*nSYtvVYpA^{kzUi6B+j{rjs0Ls%y#-9C5ZT`GvI@MdD8H<P2s|&x9_eYzeoTyja&Z(eV1><9TnZHgB=VQl(ytsr0s`ukyJb;N?vT^OA>kbmU1gy`oPXGY$;1hJs7XX&4hw+0M&x)d% zia4?X95^ry4!jJaTet{y^a8UxHi^5z%ECC%z8rFsJp55WId}_ElpXx8xSq62uY&}9 zH3DF-RY?q%Ed7eZ<}gOF%ZJM+4l<#H*RLG{<)8M-zn-T9rX+e+)k~CWs(=l;c$>RE z`eK=Data0g!J2C!;ddD6CRNMW;VlchY+wQt$|7SK4xHa*qpvK-ZnEiERK*CCpP^d4 zzpw>V)|p5G%!y=`*FLZbTs`P(aXtVL(_b$z>Ev4aXcdH-z-Q}GbtE4@- zN`aknnnFLe}nM(o^o#?q3xdGV> zgGeGkpB~`lNJUka<^Lhck!4GrkRF}AWG|-QVLBhe*u&kQJ`saPq!_eXYgc$1(0DXr zKdnAJ6lvzy;1+pRgNepeBX9gUg8lqTe>4EedrxQdrFrp_A`JjxpX8&R_Y&>)nE?sx z*{Er9$@O)v7`F>}xkpaBY;TfF1PJHSBoTPap-MCIQJ9=Ot-M26FZ)1#cDmYl#8xzWqcrU3S>=$dM z{QDsNbN}^GwSBL2j-lN8)vEOIk*cc_yAN~1RUUj3*~i7JWiv+i2zF=F@#QcoEfRR! zMwj*8?(}Iu+nXvNBb>nj0Vi;l@(k0@GiyQ)R;SZq3X>Aw5gi3^qfhKp#G{LJ9H63l z9*UJz<{d*lhG{Egr>Qp{An7Z=-d;g5ttq-@LvjlG@Z`yCgOpjTbTloJd zy6#6&mAxzD8cC=W8HKJHk`xyw39_S$aigKgVoBuc^*aJ+-h zX1Vn^73rXH2?@aN;1-Y+*_85)f=OCoN=TQ`?5WwM0C4n|Z~k+FM=fw8x4M4I$gF2^ z6=@&uRJ+Yic2oG}OSq@;u7 zG?}#_m7V4~HU7GI_@YGDT(*m_Fm?`}aaJxL0^UJvr?V+AAnbkKJbA`c*wZ{Q#ia{U~26UStD)DR4^498i zAKhmuE2};yu5@WZbI#qe6*B)r;*siNvq!f*IuK#AQ4RNUGjE#Pc-Ubjjw|n-!?a9H#w!euP|*lmjX7w1aME-2hmEtn%|U{7WcsLufGmJ z{MQ)id4TYgaLqdK6npB@$%naSn$7s5^u!MbB}oTEaCua z{!rtP1QVG9N&eQCyU#VXj4lXB$6r}+l(4pMs0lNzsHclB3n6VWoO|tpAytFqY zB4&Ta#um{|d{OSeG4z9U>VpY1*j5Jd^?tWX#n=vJ5WUL$5Stm@iWnJSUqnfl z2Y>31+EOuZh`aJnxBT#D@-FW&o}0GxAwEFj2!-hMhRD4zQ04nWVinX8UmzUu3Fu%x zV5F<-ixZDSc}?m2eIvSmoMwYN@0P1ex)l z2D3hQ?tQ4(B@(`6MMM~$imRH(HNuD6_O`L7L{`)#x77Z9#*4)Q;raoEhv$cxcp4}7 z@eFjikdTOwvn?k@Y@T}n$L<=IO^z&_O=nwL!kPum>+|dqp0oU?MwhJe)j!y4ihNqO zDbT;YX}vnMVn+hN4i?Lc+2)!JBd^8ZSiHQ}81C@m#FFSypShcB4X%LkujZa7C(1^bntPq*4oo?5$D?h2 z+d}o+mJr70$ON`MS`@l6T#^O)r<7eu9DBO+29Yz2f_>x)44%ucl@zwTe(vqb+=T)q zwdioC7=Mp%lb=1O+oLRV&twB#yb!k7S2qpaVl z(btFpMd8{LBon#YM|z=)EAJlF+kN^g5X6X0rgY@f7BzfNv(jh!d{B%gI!qz(3uT1> z(;5ph;WgObHh~-LpXG93W^_D&J2?lRxt&|uRpmmJ!PbXT83AVEp+ERUF>viwAc*~vuHkzY}*%C|a&BShIa}rv1;^p?J z@J;K+yD|h_YlENJ&%&t(18BeG$~w6UDs!xIyXU%W5`0vk8$-k4v7fjhPa;IH&pm-gT}C-EF&H)sL}Xb)N#n8q zTw3A};j1Hy8KO5MR8ObeY~sljeua)H=>3tlh>+zga`6ST#RLe&1a)xhdM;boGL}eQ zYV3d4UNpI!S`k7`Cm)4qJ<#+$@iQcfjVmwRlckkqIJf}f8c+m3TKhg|@q8=kp|8YV zA$^1q@)UY$Gw?Pkz9c1r|#jDM!Sj5GC-b1t!Ev9)0aE6En(6Z#~_VFXcj z?gkepaFc-H8>sCP6&&*xZe2aY8&!+(V_772{W$y52SQ1@4ofkNs;!}B_a|g(4V|L{ zQ?8)=-0|jkE;{q|-%Hj=d-$jUfnoMskBCl`Ws;J+&8?f&Tm(lfP3dq#UiJWw6Sz-E z_27(s{BDp>jk!!}9{fom>7|!ttiSSiO+#6s@_@_w3C78|q$|@M$ZRYhGHSccFcBoW z!I?Jf=@PFm4>|Gj){<=$z5nv7OGzWsqt;Qmpyqr&`Oy$j1oym!L%dUgj?81z{dnu7 zAIiE){yZnYB_TIt@w$1Gvc+^~KH@Yr&-1v{Vz5lo$q;UvW+iy-|d-nK!5`ch|{(fA}H5zh)_6R*v`Cr(j`kJ1Tn5|))jbnG~-$HQZ=35+B zGLzjBwUElu*QuobyEPYQ;uex4{ud=CQgb>?@1Ub7rwPy;mxzI$uTSzyv1fNr$-e*J zOQP_FHNLz5R!Y_0E|4}gV4ati#4cK2C+zd{PcQmAhA-X>B;7gmV6wY3b;<<E zrTp;94+UA?OYXdx&B&|4?dp56*Su^T&eOFqzzerx*^ls<|G}9;YE}e()*dea_58vx z{ibD({-c*vQu$Tcdd_}mJ3BnVwM#}mILRi)7x~1s`@}DZcJ`rH^Svwb-iJL}MI{t7 zZQmyWdJ>4JZ8yBXFZSnlz!`FvMbmddt>gYrf62dtdA5k2(d^Z)72=|BF`Z(Gv&q+&=sLXy;)RpqmGV zW=s~%@!`kx`B>h78>}oQyXogYf)(au;K5N=Amg~%??i0rzu+{qIa=nmERXR?`%G>k z6IA;>w2*Bs&5VHqi(=UvoFzaUvlxglxn3hoLUwd4*UGbZyR}P9IKMI$u8M1i=WpL2 zo+WvF`BV}1`q`bEH?AChN$RI+c*QIcC-`n`{9&J{hns?fM(mE0IwAAC8 z`8|_f+4qV9?#uTTTOa|6ErQIO!CQxeiY0i)m zU8N`q?ePG6IRIF5EHc1{tT|meBCU|3L897!hFYHjrO;_W(+u*R3+Oypts2k^44z~y zN~FL6hq5etNtCA&sL>n!WYn4+lKTb*{-cD*(H*J~YEOd(%k6wwM{l^MtaCmLUD#Xlj7dNTYlX{uf$lpz$ zk-@)zqK}tXps=Oz`##g#6v}|b-9H;o=goKIev|Rvrnn2b^Byyw2Hvm7^<*~c+-wbf z;0ZZ#H+a-v{7K!(o+*yxFYt6Tq{$zx8y{-FYb;R~HPpd~7~kY@b0&x#8*hxv`U6eg z7{%#x0FQQ@cH|5HvMLikku1Pr#n3F$F%$}01eTZ{f^W0Z6CeSbSvv1J01vh(HCO+! zcp{-X+**luGBo=lCGvWyLQM&Er<8PvK#v2R32rIB-iM>@YUivd=%7rq2sYVjJ{mb5 zo~Tqvd}xgmt<-ywH_Yq*h!IHU#)jf)u0|rx_d>DF3c{;Y%vi626H(C)ooSFm3}=S&}0LH(_l0i&X88>B5b{lgQ5XE z%zQ}1DR9J_VsElb#A)I&&2|5z-!Q?buFW7i0ZJqk)9XCoT-p#3waE@MI8NgP^!6zk z^czKwDae#Sb3q`vFwGbV9N0pyTfdJ?ZC7 zb{8!f?|61Jo5M@Mq7dF8JJw`~J%+H!CW|0&6M(&!@*f!l^agN}pG%TYg7oT%5CGns{Bsn* z9$`holG$XaP`w1WIT>Ppwp5FB?8GEA{IL2eK<>$Jg9FG@pnE>%A0YOrKkuKskxb?W z03^SfBxGB2VzG+mfH&Igx>z#Q8xXp0&Mm|50JlmUuCBzXXFK=hARp&0_Kzj@1xIs4 zqF*I|&p5zNV0tNWvY_HIb=?SEJk-(w%QhgO618BMOBA&v(p!`8q-G7alkSgSF!QuR z9*Wi=J_*>~o3#9$oKz7hDm+K+Z|8nh{%#YB@xqXzdQ?@n+)wi=eis-ZJ+$35Ha5_80^1>k4JAnLoA@XO2E*at$7+aNFEy zp+q%#K)W^e|QP9fU*=D7_pdyW`2*+J9UcW|!0yhc^Ed zZRes(MQvd_Q)W` zULeXGcrKQbyb279ZZzwqx!~X~F{2+VxoBAU>Fy+SY@T%l{2Z5{bqKyPDDnJlPQ%vo z=_s+6+K-VW8xU>-iogBfl;e0_(s8haewFjoLrxhy6XZ3IRL-Qp<$39)HMF$Ofo}`LHsj_-!*0QCXQ0%E# z+el(}>7wnpD?K(}Nlcsi6`AW{-YB!7Vo9b(NRXQB-7IfDD?0Jktso+yy8Iz{O0%Zm zesH%-shomX5D1^|78%FBM})8{?}~tFkC6C}8ctsxb%cN|mR=SXc)S^Z@#dSwr{JpAVAR9=x=s;s9;?zdSleeO zCq$Y_aqD`}?$RND1cV2~ZK>Vb-ow(> zBO%ze@s4iOcwb@tIgd;HtM!v{UE`h`l+%_JQpuXvZ?|L9n7WqZMH+o!`t4<<>x{V5 z*9Du&bwgLXu9kr^HLOIqbmJ)XDIv1O2+i2lJUS&;(q-|L>~Oiwno zvaLTYeMxGL8iH@J@gol}z)~NKZ``}!mhR%rG4;`V<>`e>9T#n4nruxyUf=v$sVHG# z)J;mZp>DUj{@5rJA0}T|uRhf|=f(|`nwi=B_gEZ#EL~D>?^%rDB{Z{~pF}kZdCjbq zH!r5>N*;YZD7?7P@2h)BmP23c{O+tK)LmjOqj-)JtikB^gI>SMO!fBG$$cb55G3Y2 z`T$~!+RZ zxPkGKHDxKuD(SmYGKqtG#QFV{d8OyR42k$ZcjDU#{L^$IsCxCa>ej@f<+Th-u3)*Vor6Wji4TIww`oapFR1o zZ*0_o@_Mt@DD$cjbi$>#rM1`nNN(MFOvTorT;^rv)g`_iB5gx6?o0NCxhz1@4^=+-(CQ&4u%hialS& z5#*1jo2)><@YRpWfMPQ6V`?YZBMlBG?|``^rU?_NGWUj@=A<&*+ z&cl+GM+vfvsy9C}ojxR)*|8(P{aKD86D#p6Z{ygWx z;Y+5t1t$EGhbjL~aDkIY@eVw$+F;lE!@-b`LdOz57LeU!=BE6*UO5(u(;EN)VuP(Q zCAG4|ZG+BG#H~_E)J+hI-T163z;$d={+7)UN&>kOR&bw{%Ge!d_fs3 zUP0;&T?O)xQ+~{;j&;nrN5{1}^2z-<{g8Z8?rD$Mynh0gU0mpR|Hq~aWyIeQVeuFJ z22(w3GdSp4!CQZL$Dw?3(sf_!7c_$+tm_rEd4pp=dpa?6rC&`?26OcYuwm%aev90@ z{s+UrRCD2!{>LiUv`2(eX^``*SsFbuf(H%J9q7)r$K~d50~S zlxR4ul~mgjvZNN}Qu^N^!}uY0EvBND@9nV&jAp(?L+LJVeJe=^ef`e}e?NRaPg3~e zOCue!V}?54=YKf3H5&I<4-@mq_24`4&YOw{WwUo89-K+DjC{g}ua);DOCRBL!H0|!}i{)mNuDy1&al0jk+#a_Z=*k5@sex3EWu`)cffs{BGqLXE6`TGp!^opj^7l-AAeH zkpcs71vtNKmPg2xTwWH80+JqA&E`akjJ{`kSf6vy|L{&B(>urMA0h*_mH5eB1L zSCx7`2+APvr{MfZYf%B()TfQ{Zi#b^3u}pr%`2xPc#*|6#T z&d#7UD2y8TjHoSN-Mq+O~ie-l`Z-+=U_$}YNpmUk^*j9GsfqY+Ix5>~n=AKgM zrwfhW!WRk~XQKDMHO`XO?|k_7qQ$q7NuIrPMcRGQrxsjk_z5d4Yo~t>okdN3i#2wYkEAU|-&1(XjWQX^58+;% z>wI_jkQ&(u6|;tAyki~i&A$8ZKQNXI68B~qlHdK)AW?W|tNo?K+~hC#TDhhOiJ{BF z?c?JM?9EP4?ZvqCeLY3(4Mg`Z21Y8f#SfJoWFkUmG1%0gRQ}FkyM3Y`4AY{1cLg)@ z<<@Fvam-c%AsMPYjY~>znLoLzWyZz-INhU#ASqmc5v#OB#d6I}aA-NOL-7mR0YO&y#nR#^ zlwTYkN+^Wv-19(zMMHKz8HLc>q(e3&ur<>18X{e`RDS6pR~Pfe?milk-(6h0TWfQ* z-=)J*zL9xB@#W`OkAe`(pyZ`z%TNQpKGI-nOM!!=&u8eagpHdd)l4!3n;eSkL{gaq zxEcVAS)7Z>e^~VJTIBO^Ss%X{@hkY#7A#w9-cIO-dOT%(xxT^-Eq*%A+6+Q)^Et*W#;95{Vu^`X~3J*wg=R=j=x!o(jlq#^vt8w9b0 zsxzt5_|E)86;WEnSqT-NE2U1^ZL!b?_Uj3(@BPvK zAl+e9k9DG@Vp&fnO;!KBYDm@?D{hhD9D?ca5qFHQEmv$r`GdR z%!r4X9Qc7-MR5{a5qut_;hu~q#yo4vCDI}V&{B?^$KDRAs1FcanH;sZ-6&|!&6yvV zx0_LAtVV9B(ZQJhrVw}&0kQB!99W->k4+(eJskekm_j#|x8Awu<9hPqP=r|nmxz&{ zb}0wSw}yObS8aWb)Eizw-r=jZ^1F5Qo)e!EU&P^V@}{I|?V`l4!^dhneKve652VU^ z#wWS8R#yAu&$C7aN4&YLvDhxvr!GcE8kf|g-PExZcHrjDqN;J(hb%;>j9KjYeAfxP zVTOwJVunKQZxT-7!d4D{BXcp=aK8W zU}Y~1_z~Ug^oqgxeqTQQ2Dwr=`MAP@9G*`FvZHbG@xbFc2{Z`82?RpDAtZ+P7vb85 zqP=v01ByhS%8uaFF1QH*wpf9`wUG_`kF#bdfM^N;*C+$Hf}HFS5fRgC|MN51+LR9a zb*E3OLiPBkS*~9oy`2Wf88Q$6r*5nXtL7L&RG<$T?z(21NUa83BD2otQQ@sHZl2#s6;^SMd6tjhMRdUqyg?$ zi8~^efqy?elI(n{ZWQm4_V;t=>0eLmKwu#N$fp6)!D?ay1dPOXJ;H!`;J9fYG6pO> z$>IUk9(GR;o2KN>fqwTwtzq8biu-7R@{^1kfNdV0#1@#+YM_wGGbL5~D{naqIJ!u>UtL7cS1bJQ4s@K8UQF!wJwqdW7p2H@$B`u<4+y*m zCcN69+N);J`pl_T$d{B)JWIokO@)TpA~_-qT~9JlmJIX+%)B&(+TRDr;y_sP@sz9bRpEnzr$<$; z&IyJ12uIBcQ+-eYC1bJxL;+y)B?Js9F_vYhG5~FIIEadvRnu@=H<|@zq_d*{FeDAq zV#0YZ9%zp9Z#4z2y2+5G5D}g(BJH}S04`DTvu0Bbi%}<$WBwK3swYsPCnA@P=Ub#4 zHuvhwI*@=Eb644TrFb4-)qDc-BS_7(Y4xH-(0%DMP9?20Z&-8ObR2mnW!fy8@~#=84J9M{ zVgcEBB0gCo@%Z|d33MMIXHGXZmtGzDT#PcYq?gQ}VNDo8arE>wqtl)6sDi?B9G*C5 zVA#oa!;809IO$F-8Z~dV_-J~h40Dgb1(g!G!){t7l6!Snz10L>UWy-^Z`4aZAmso{ z*w8+`uFCA@lv40x>7K@BKQ?Z{Z~Y;e@Woi-&1Rl>GK}39 z0@^@8*L&PrN77{nPZNsdfkqOKy!QHik>V81grRfv0&p-T866V5)&4eC7TL>UF^UyPcCBd zJEh{nKhoTPOeDLq8U=gbOr2MYCu_iF^d}#mh2lu^>9a-M(}^cAu$GeE9p666BaJ3J zyw(&1U1>z%-zRmjq5Y>t$V*G)#DfxJq%<CSTaSVvoTWq($aB(9nRl@0FfmlZcQnhW0wEXz4Wp<NNd&(Aab4HsPz^4BJOzQu7~LS+;Y{F+`Aki-6Wui* zGvhPYRBql%*U5OVDvKX{!xpv#yh-{e2c|=Hef_7EE<xnI&C=sZaIk~VKFyXz;_mRQIbvSgXEgpWNn z4n`yrcqb(m{&Z>)$yW1EXw6DiO%#k5u5Dffc7zJ+`0mwC(*Q^oNsyicJQuBSK(m~7 zBg4(=HvhXxxjQTF)Ts?^U+>R-;-23q->c>HKDmO)0)u32@B?Rv?iB&!gt7atm$j0UB|$T|jI zRpiO(F@kof{QBV>L@MlWdr`do zpTKL1x|>K)TvM`w0Fin#_Z+DjG~9OiGeuOn@td1QjEz=*0@q@IReHW6u6^Ry+RY{W ze9GvdwA|cZ@ayDhHdh5?K4|lms#OdatWAyD_2oLsUm8&$RUv}Is-9HzOSH{lFeR=K zTJIV5D~qF7rg=Kogz}48pKHhh8Wert2TyOt0jQ$un_+0A=TLa6DIYB>C7Z%RU~jhK z#_X#s3yiipn!9v^@O=BY--C%~ECY^Tl{EfATgW*j^Xw-F8pnmtx;TGHy?n80LJs1l zmjmI&)E3D%NT~JltJ1MR1zoeRif)e=Us&v`BZ7d}JuxuDK{x&&w)4|x;TO3oIg@xL zO>@y;?d7(qLNOXy9s#%~QdMvl49o^<%i{QM3Fau3Kyd9wa3H+Sl(*&S&3@^vU_7_f z=7U}n*w?}bV?joqj7Vtdas%yJ={2}m?2UUM1JZBRlk()Ga)78918ZNW-Qh9wDn}}w zVc@u9sjnhrQ1X7MUA4CiZ7zv8PsF1Di1$q7c3elz2~5iV^-h zCLw3HsYNHFyW#$TPULt49(C1Q{L;tVlo+(5jQyuOP{QlxS=q;D_?vHq*mYr<@YuJ| z5R(Jv-lV?Mrp$&0C6i8V{n2qW_g(BWqdBI~tHxSv%GI_`p*HH-Iu9WNzfFvZP#Xcg zkW4FYxK`|R18OM>e!>3Alyv7kuO#@9A#+e&5c2%0m9^b|LhpOsX^@<_sxYcOzJQHO zW5PiDTjsl4rhKEc=tNp{q}?K_CEXWZJE^Y?LLG*)2=>&RzXyUj5w#D>ti{*if?}x; zc|OMY*EaX7q5j0~4?HTKD_(fG^@ff8Z5G@?@bn)9!qrNryH9D{g!H&MFkg#?aQhoe zj;u)NGO+G7QOqt$Fg+D{$rR_G_%5<)HY++j^K(Su)IdK^-XZaGr$276E z9J%$4?)X_=PsVdEEYUOMap)VxU|_Yv;b4cUzn(+gZ2g1uKLzx(tl8X0*Mio|=LI^zj_6>8_4bB=!>SLiPgEXaDMsFYP-4Ef?6L%#91N2|*qn{rDZ zayOe6Gxk1Ix;;#bfy`3 z-a49!4Sa57dMd}1>sx}NdFH@i(}%DBh79>%PW6p*L&G9EX2Exv+$^TR-8nJVoD6GD zth1eDZqFpln~?4<3bYAFP!@&TzNg>uRO-4mmSbX*t0XQUjBAI4D#S-LeKCNX*fm;H zF#S^3mZj{nwq-=D%w`V$v%@(Q%>LWlVn1Zn{DI50Z-u6~z)3q||Lw?-c9s9z)iU0z z1~0sY+G+o{?_|8o7mIX$x106`fwQGPyF2APeqhS@xlC%u;)Z^&t3*Bcw|+=luVWn; zLImVanfikif1S~8B^0NBVeOIrJ3OA+bm>#ToI=_Wa34Y(w}{TIgYCISpNfDxbrA?Z zvlKD?u>Avszj;RIahZH1|0T;DWw(Y~fA}w3KhRPw`T$uHw@KHh2gh_jeqvi&kj__!j--2{71F(y}p` zbMc?}U)@L}ufEi~5{J&O{63a`WJ$!FC3+q^krt+sz2T7`hF{z6{cgOn_x*DtNs4s$ z-{GI%zyB+K7pNJ$AVmBn(4BI{OQ{nDEB@tV~U;CGf1ueOg7b^?+yA$CaX*-m3B_#l6z< zrlwaPOPa`^3@ORYas0YeG$H-UV!PIm;2i^v5t5mOz^j@%@j_(9nH_DOlPNQOG8_&q zQS9uelx}m1Vjlij702%$$d8+;e5r>yobBEeQM4@-{OsxShV9;S!cwnd)~%erMnm{V z#VU_}^AZlhG>P2C&?vajmur61Z1d}{|MgUQJg4>WICTlcL3L*$bLF9p8$4snrquW+ zfomjbOjf}fh4SULn^uL(oqrO)B=Ej>F{F;$e<$sbifcX=dJf5}6mAE)dxWl+ zR(HPN{o0)R1&5LmhC!#v2`+mtQl+s#fSJ zHBVrnR~88A%1(NHVu53kqHC3piyp33J!>P0uGhS7i^{)v;ndlpdAj+a*$RzSmBBOA zbFh&4Sx4Y}?(AIYOtP;^nbG+>LJ(qpE_qX*4}77vp>1A{@}OCUy7a@e&~7dUT`*NaBoLU@dTH^L21RuqUf2Rx92M1lFOCn z(_5q$kDW-@;(x!sk)&7$OU1>kzZcO19T1 zE?tBA$z`&S02^wmAtQU0y8@-!JSkV57euEf&2-xksW-*VxR^UBdT$(Xmv9Rt-J=YU zAC$m%eXKOfdcPennJEX2?l$!R42GFHj|o-F*+ z=^iHG#CMa_GW++Z*NCMms+tmkBnN>>Nj@E3k}UO8p( zfDMWBdw3@D?D_N_VHY!7D=g}w&V8iST}e)^`LV>SkdC-}Pj~g%vr8?<=8{xkz*0oc zs;QE@!+kGgnaJbSzuI3$qzmY!Ke7}BCcRT4+W1AWO)JJ1BU69XAT!lU=Si6N-;)H= zV+-}izuOvEK#|xC)7fK|m|mL~`s|ftAyvuh+=5+2#Thp-$Y*6p2?csHgWrC2`Ro`z zQJ|j8<0H84+UI`0u1N41>*~36_o*)v^_oC7W25iV;SFO?u8p8z7%q$5xL1b28|D4^?W^vmc^j63IsVDVt1GS(Exb3?Ciw$@ z%tNdr-ut+Hb)T3oWz!)(RJrh1=;Mr#j80^UeM{F9;gieZuJY{Hsuknfp;HCl=2L+QBdlG0bBOEC5a(81PMh>)j9Z{BlOtEH=pJ8GMq=K z3hm|hc`j>SoJZsho5j8>Jt6wFsV`W5Gc{V}gSorwhshWv8%DiHnu?-~Rqo#Lv}bXo zpp*1EIzGpz;jLjp!iOn=9+}JSd{LgFpB_x?SLlDcYDW?sE7`0k8CM}L8)VAI9~oJs z1X>ea*0TpsdtH)wbwvDl!f9MhB;Fd|p)Ah%cf4Kleu)#9$8r(CU__h2Z}1eE|4tQ` z6l$Pe*-5-vJEh!1X5p<|jcx{pYfdQMj=lHr&Qy6p{E!l3&*0Umcc-5HPFJ zo6HbgPodP9Nxm?rig^W(^%}{r2?#aYqa^hcxMLT33L{9}ssUO#M8kET-+Gvmk=Umf z&r4E+Vu>D^RjxnHDzWAaj`+6{mgZytb^9#8HXzm!fddas4le#xLt_pW#4VT$u5Bei zFl0{T-`z7TXw6#Mmp>+Oi){`$0vzOkS^$B1JB=Db?;~jdE>^@DK1VuGH}MSe$>+CX z{d5YPH0`_B@cA%IJ{aUN?Tc#HWq0=0E+~sWgZ81SEH0mGtn+gqhtVV>*2i!Ti zfN-&71pG_c%I6t$z2sQ`_vmeE^wUIJ;8&yF)Cm#;j1{(;l4CNs6H7Vx>823h*Ai+( zoUbfE6oOXnrKv0exwOgrw*mlrbOe-sW{kuOx+o1K+JNq2fI(?0ZPFl71)wPeaE%5~ zoD)$45K!JI0uG>9!jX9F^U)EaKmeWc+ZM-0rE~PKb7i7Alj)UBXg-LvKuy;{Icw0c zNLnD7(J*0VCE0sK4>TwrqE!wZx^o|%U+}jtJ-oFhCR#Fs{VwLk$2=<0qmjB$qS9Gy z+gY+#pRB4tw_$T3J<#Iqn=O!-^8`r(x|e{Np_#%745`b5QWIrqL8&Th?R!ThlBRQ# z@jw6yx7{EkK&4Bjrf`SgRLUV-$G5+U!yG9-sQoP=HO8zg;Fm0bD{N3|7Y$LI1I6RN zveNjEq>;1jfJDFvnk?xh*;3}%c~iu1G?Aep#}}vVkxYB}YuynO%M!`x;ME}AD&COF zS}wkIE)DYy;<88s_DMQqRxJsnVX>maC(tOv=ZtV1qyPu!veqa7mXP|B)d8_nOX<^m zL3E8r^)&8Z_Hsl&&Ic8IjSGDLC_tgKG)JajLzyJZOy!`GBniWTCPy}@OvzLp%dUCx zH6RxR2Oy_%L;gQz1e*FF<4lp(!LyF^$}Wt6ELs47mQ_dd$pTj5DM&n@?A3G?cC|+K zj)Thyl~D|6KMtP1;9EhE)Rg8>1d2RcQh3NO^@1QOm>^pHviAAQr_*S@wK??QCB3#x zs3>rdL|e4D_GC7jlgpH{GVOf>GGtW=GivCA?yGHNGHnCektfe8bQ9DtxXiK4;7kAy z8&F{H#Em92rzu{B(_49CP>pR%3|^-p;;reeE-mK6QXIixeY<&bpD*K*-j7M|>;g^* zu83$eB};dUC%}s5xvcy76|~P(31|l-w@dNY!j8wjKIG#Ha{u$e$m@=v8T{VS960jDsx1NC z7J|~0=7UucMd$WX)x)%(7@<^Mb{T)1wN6&--mDru5zeKY5ok1vmmCEnFV5+GBAK1r zWtdt+G*TGQL^KG+0K4BbD4JV=;J|}(piBVQ+6xWb+5s^`E!*>Zupz-(K=Gn-@s5R87u5RHdq4N)w2= z{z+<4jsHDH0EGkblYdzUqhJj9dtVeknoUuf4Xy!1p&`UF*ik!Al@{cfEhI3-l1pR+ zUvcms2MC!$$H0O%)9DX}1A_yp0YqhSs%Kw%AJwylwVG9~kc{qV2ta!1yExB+0C~?! z7@W&5z8z}mWRuik{Fxo6tF+gq(Y`&e^(md7KO~DC8P7F_GL=R}qdB7J95^JGBhHkH zWfxA@R5L+Z<3TD&ljFltpleq%~ z%_qO!VU+b8@uKvl6&a>)mj^seVyhjsiMJv>pOB(su3e<4E&8Cix^~F~%Y#=p{y;_7 zo=Fmiul)|09MzH!nHOQv0z()XP2JBo3BJ#-`Ygxi1%+v+{=AWSGRq6Uf8u;wfHY82 zhok6cW*eS$Y#;!FoU$r&6u0M%k^t;YS&Dg`1Zamgp%hl&qw*08%xpj$mwtFV!b#rr z+2!&5^VJ<*pH*bhrk;6*p2Gb9K6?8wkmkX*7sFQ++i&~uD_}15veX*!@|^0dk0wM~x|xhz0s@-Y zP~5B>746^G|6W1WzJ?iwf6p~Jb#L29{z>nbG%Oj1rjS|CY6zOcJWWNjgkurE(h>1GvkZaF00`#DJ)=eT&BI1OfL$gC@&Zxeatey( zz~lfm@hT`;j)Pt^?3M~Hi>K=|roh*-U|p~vo$}Pe9grv2u^2p~i`&q^$6+gbRmJd`ckyP{6`-TH)lL5SGD#vv0raBaZX2ZR< z^C1J0(R>DW=Qx?55HuU_5Nbl>oDU9Qod)n`PCgfK;3OLY$oVqfsA=@oAs=>@0Gu5L z;AsRWI+dP(ne*$O8Qjzd)ux%lF`&f0n#2YKI0D%HEB8WhB+&Be&p|w(V&Q@M_%4Rd zail4L0X&TRj7(_S$;K_=v#~7~n(s@3CuuP_A$V^G2+kd%2Q+bC$}n>b-eXK>+fh*5wzv0p7z0Pv zSR0fkGj7fD=c>9Oc3@1A%k_wS8$jkSUoCv>Q9HD2Tg2L3QX9>JYM%#_^5?xkH&{fD z(0JLrgwsy*7VLR4MF(lx<%rEWJZ3;D0KZK-f!HCYV7fpbZiQAtdf}Wh2ecQGiVI>v zfll^fqNm+#9Od}RZ)=j(4_f4$_i)?GN__rrm$&CjW9=oi8qL9LS za?y6$8JleVkdBkw6i#KFQy=r+Bz~BhXSq+-!0Qk+B zAll18!d=KaKL*4>b;EwFA_2n{pq;JDZr{ZB&Ov;=GFOV*yP54?66MR$1*gwNZ_6&q zY^Jf(G93{6-wz!wAudvPGp4F=3HfO?u_M=lOJ800aI`Hq6&0BTOwMkJIqy`9XR91T?#z%owVMzdGJ z5t+=v+wUmj_;!Gv&&}_U2hiKLq|U;Bl()T+)WVK@-`)OjnS)A|DhX!(Ga(v&XS_Ib z>X^EvD595%io200G;#-@bO82PNb(B>LL|!Pv!oestn)bcTujs*&qTpQWG-&nB5C7N znX!fZk#$xs;^RG@Q>70%!GO@I$)|KvblQ^vN-S)cNl~0#%6AYPkgx^>Ft8h|g$vP` zP(Jdk(7(CBa9l5^3!PXm6JlTBofb=?jnR^Z=wGk8*qN{{XFOV>)=O9OmZ!_qE<(_H zeIbu+AkGJqdAH!*7YFGbAL~t?i(|atmz~bqj7G*PiFA*%tdu8eO3!{$z`5XJvm>2zyR~{j@wsUCD^Mtf!Q{!7JGzKv&C65(~yq* zAW2?L(t2%$8xJe?4PQUWeDrDeS6?atW*wr1PR9M~TMf{;wyNZlykYrFL=Q~-r@pe8 zq@_&H|LU7|OD03OvfevyS0{ejf@VMW1g|*nJ~)_wlpr25t{t)n9kII|aXq9R@vR*R z;g8-^A;HwQQJ&h~iKm3a3&E1?vHg5c$_MlUha4v+@Fr`=((V_Uv!;u9zjc#7DSVnx z7`t`|g}s@%_(dR4v5(5zxAN6h)PGC+9QnE}{dV+6V~)O5(66B3%EBA}rGu)48pX_m zgc-k%{B9BY^F^|`T`2xt@A$(?d~!N0_D94UEcXmzV(#j{S+{>5*8Z&t(LTD-HvVHv z?AykI)O7Guzp_+*V{ZNy1fm*#@fo#Jv811DtD2*& zx<=5=xw&))#yrAnuv6-3B50Yfkg3%f^wV6_`Oem)lG?lVG!;}bdNWmwR_ME#hOVjG zOqZ!BLI_K?mQU$-C2=iiKHVaxC~&f=yF~mp`Gwvh>s@%iph<$UxmbSh#ckPs^Ah=x zXSV^F7B3L1+s58o`DrWYy`Z~Y$LzGAaZzHT@T#A5b_c^6qnsL5#T305?}m;Ba}4&6xOaJ(+Ob9_RK+dCRR(#({rf zZTyrLRXDfS5B+bxhqIc_uzp3Ok9`=P5uJ^<1AUebL7jM`aO^E3^P4i;xeh+Zhoj%k zCKfb}dw5TmmVzd3yR5?IDPT3Ys224qmrRpOEKgY;>0^Oz9+=~)5lwYA82#_pZdJhT z@+{2tCHY`W!oSmZUW`6ZtSXLC<8P~Xt`yKdJiA8mP=m`Td)it^7tF1YhXg&p~4o8{hh?yU6g2X@V3|s_3i1r)|)jjjgtzD)U!|N&NCyq7l zq2)Kc6%7!bc?Eno^^jSi49efh38sfLZjg^UG`7d1hWE4+U=&{X3oms_;Q5I79~WW< zx9pJRgYx{FgXBR!Zf_V!eCFMVJqeuA%um)jy_MtMi^EwEq2l4FR88KrTYml@1*@%D zKD?>9{DFO1EfxpXvH!yFo$4u5d;OVDq?ef1(WotT11>>b2d^V))kln{!b^UX?wU$! zIFTXFWLCTTT9dNYhgn6=>pa%TS&X33UL%?8%}tUAv2LoH`rl(Ngvz>0q=uG7Z2^+Z z%U4TZT3Kqjr=}ZxH)IACmqk_eMi?M+ZMJgY0xP5nJ0*|_+xT2#v1K}UweK#C;-UfP z(!;Tf@9(3^exzu*T4lR$3A0oxCo5bQPuDsQ=a!l=k+{ItE1;mq5sYKG;&xfMMt~^5%WzKMA%*KQTpUD7pj^2O7v!P! zuAZur*lLQH+>GxJy_#E89sf71i9_9>L|%qV*XF@&f_Pa1V=vWb?shod}Xhb-c304dPq9-oTF61?k5P$!LlaVZt%9fb8sccc2J z92*LY7Mv3ilaw@{L*M_CF01thMP3VjfwKsmnH<%VpcG<)-yfuLck%trNug10Xu)}b z5XvnrxW0M2N01@}SV6qSrLWJ*)u@anXcsfJJu^64`je7eb~m}UdL_TkqHI6+!oQ-; z#FIGOYoTGE0LK*8&jx|PgyO?3d5*Y61GHg-_0UeuOA1^Yt|EF%!AKbdiMh?`tbmw# zdl_^l)YWIX%SD|3!Amn}71eHayhg!k!&MQRoWaLphm|UU`9hkM3T?P7aF;Tp4VP1Q zCW?u8bL|jhD=LYN~dahfWIye&fi~fUJDv4UWZybe>C~SvPhG=cLSGmgCSK)>{v8eEH<^yE@4C(@v#_Pbn^TW1W8^S0nm5HF1_0RH^OtJ%62?6|kK=1# zz$R#h2l)Uac94ZP0BGY4?JItwkjGgJ=1kRj<49SsB&F*qm|bXTp$Is?zkibX?*si5jqR2Fer5`xy2fQl$ZB5vo3a$WT?0a zuqgyVXCeyfJybMnrb>89!mCLHPup2kv3!|uujd+unacO9Li1T=94jjl2*(9gJ%85p z;n}0-Pb^}(5J19-X47eH3+E||?J20)}x!Z-;| z0V$X~GIOo3)65+KCj%G?U$k*_Rx5R~9x-8&;QkEW8ca1a{#-%8e8g~O>j1T43C1xn zx49s(J!c*|+SFKMdahAvju{+yLruKQ4`Dm{c;T#%XCs5g_f3Ra4EK_c{#>!^N|e>i6H1^_;MQBmWN{XPpq?E4QS zk=Mkt__`-F5aIrr5UzlzA%`Z5<`rUt^GjQcXHqOF4YmP~Wr!CAueaG(c%@mpXU>6K z6izrkh5`AVr#vK{FC>o9X6T%N07fogOPB>zfij~9M*(R-QNvvN@U$0}M+ohU z!k-x-dentxud#NX^B4In!VUgreOP^N+?`37(lc$a0!pVD=H1@xl4H-JmBA$a4FU7?@{sGX(eWn5(GZTt zi}bN$3GXX?D$g#Lz11BQ1wMFdzUQ=u@;2kphNHUR zGFc#LSqhV3@4?}dE9J{`kr$c5?X`iS6rd4Y0uAOkM>b#5IwI^wTjg3b z3#U{;YslL_Z4C!y@^L5HG>W+qoDWz^@i&5_5WcJ#@%B;6Fz3_9&XhtUxFFvZcD{!S z;r;e=f*kK{f47Ukbklzquu2wR{H0BqHah?K=^lAY{X%TRBn}F2xBI_PDR5)B2L5n6 zkJSxuWVWhFqCuEm=Fxw#m%uucxHocBa_gLCrd<9OT75LQC!ieKd6~##^U1VCB`E%wVpDq+0R6iH=Cwr|g}l9iTq?HRv#v2KyOP9f*I@39hSl?x z*hw2=wO4}*V$3;MV{lGa=E)-m_#2d&2iol05-DJAXEZi0CFhIl%e((*5Pt3^1-WUk zkZTMxF9&)!Q$dDK+~^*^cCOe)uEt|l5p7&AiY;fZ)n|q(e0p||krlgR`6^k!=``@( zwK!T;*&najN}Id?K?VOt*%>(A`tPpKe*dH8opI%GCMPn~WMuMXK>m)sAYN41ntM!i z|B>J4?Md20L;T-Ts@ClMkEBA$l)4t4ORjfnXY%Vz^LedGb~!z##y(#;Q5*v%$F6#| zV(G0OPu9}={P*!y4m04?D>W7CoT?}?(U&fgOCHCi-#zsm5IGk z`!N#iYN_ivjB32B+Xe12ig-%{(QLi;o(OAhdE@(23N@@~aQ)8Uy{`jouaBwq2N$Hv zz)BNhqJw6puhz#l6OGlq9-FB?>fV5UoG6vb77I>0>Wn~3eSF=!I#!vZ_**_$Di8@EH#idK~1e=wRs?AbV*ljyTgV&6;qwkuBC6>rX}LvK9ASJq4I4V+>$Ko zKj7Zb8v2DTaU>>}sO?S%H#|hu(LF`{i6bt=x33WCXSc`s_Cn^Q2DiJWWQu$I@%)T? zm;By*2KC(M>>wyB3g;%FHZ?BMk7rzuTGRhlw$pz1M8GO{)KlL&izu8nzYbAfFB#=H zlkk0q*#IYT-RyEt!3!m;d!$)W9X&EGB^SYwj8H!Xxtnpd^p{S0l2YTk?w%B6yDLs| zTYV>NKK`~$^}?8G3~Ts2dVu9wjp(vGdJ~Ze)8Qz~Hn<^5%8NHHyWQ00%~fAp^US}o zW%zPHlilH9MqMXVGq53uXjtZ5O#39Dmb-i5Nt6zC@lp)r2rrrFz6pKIs&**%W;iO7 zR_-+-s5}Zx7c%1+ZksdR;tLwl2-YJx=Hx~h`DqlU8$Jc9L+@9CxGRo(YC7IK&wbWrW@YCS#|F(OjCatl+-1|FnS1OomQQ`R0uu%IHZTCXEeKfG5c@wT(6gQp9fs3YAWD;4gvEf9srS1+1 zVMoDbt61=+lyS`W=wuNGI(_b~Uf4FHtwd8)#-RH*VxiwQN>ueEF@t)rk!*H2hWlj@)u6;lqmJ7Xrt79+cN2E@0 zXYu$M$?2qtb(Oo2Hb_#(#|&uSEuNgM%A6^MfzovJwI#Ym`QSF8|5``VF!ymR=m=9a zyqJwq^!s&eN51v63dxB(%U+^uCe^k(pw`Bp#cqg~J?CDhL?c2+`#bf*<5)0nu)L1L ze;}roUI{He9$O1ZaNP^DDdmtb1f4$iTm*0tuDy$e$x(?M)J7_iA=kKyVe(1;=L=bV z7d@u1*W`AkaoA{mq@vUN(85mK7A6SqZ}~W05&s4LF9ABLMy1$1%s*v_yZAPeyJ{W8b!g5^qE6M&QbdNiv;798 zUFYWfxW(D5B;;mNH(u^vm2Y*6+5XHr-!HBArfyL5__-VHbn(@HkWd0d3kSlx>AZqU zgpxR&aSov$12vw*xN_IWx7zh&N_De7-Kfr~u|JcaAz**DdLd&*D^1pUx!ki$z-X~u zr7~eq24k!x)-sQ+*s=zFUb=LWe_XubT^f;eu<@(GHi%s{$N5nnKXnD+`XSvbE+g>m z@BY`E*fw+z_HE+eIEx!o%6JRaVGyjZmh`sAYeE+3Jq z!Gp`G*yMv5DWzZ08mRAh{ipkMq-Sh~aVCf7(bUh^20Wa>=!QRvc#R!LhHwAY9)?Q!9)M}FH(4$d9|FOy>? zykF*@&F+(S<|l%K5Zis&2*3e?xU?HWFYl3(2V%OLn)bQm~A(>uG0*$LlDgh!2NuBHG$ z$Y|6zJpG+@$kph6u6S7dr8}EPQ^_HzJ=o7u1L`rZk$esO_N#S+;*r`BQ&I7YkGm38 zwQ7sL#NNI{o9R`Oc*y3f_dwL6dp{w%20b2?bN+6gskKiAg(NsFXUMHZN*4G8)7TI9zg(d)2cGpYu0G9&?oF#^sKLTW z7Y4`veM`ar-M)(dy|_mEIr85>sbt2HS3XY84KJh~_c-)vP5$Vi)5B~alV{ik4hWzx zbxPD-p!lDR|30QhMJL&(_r5o(r84%O5jB??J1s=}*gigGmqFrJ=wyOEK5^L929&Ek z=2MeQ&nK3kmoeB=`HoQ52J!tDdX+W20Sz@YhL=Wh9mdzVyeb}yjz37AOKr!| z@uBFIwgzi^li0h`;W;Tn#9F2W-4KwjHS2)6mp)iem$k7DN##>p5O}ef@zA~Hp075= zgWMlJVDlH%#W?ticd+-O9+wyFG?W#M0)4Tr)fxp~#>dg)Q6%?KkZMX*kgHQ65L+bs`A9(OtfK742lT%yac69L)q=X9B5jZ#Mj&Wc?{M&k3+aC3Yq zyTr&G0Gu0CWoa114+;U7YdK5e1EnJlI3i6))3=bf1BD{TiFz^BO*3dULZ#zJSPT+; zZl)I%Cld1#M+;0521G`=^AjlmFtNjkX6`&1Vw&wEPYU6FddJK7a;}kixl!F&vU9Em zZhW}PQKX(s?^$P@891VJSpu}#eF88ZB-CafirO5Gy~GJZevP%>fwNt%NoX1~?-?7B zPJk+cd|9uN8OF_c#%sF>{Lv6MrQ}PB_$2fgn=(LaUvS> zbRzwrRsV@&W==PoW>fh`K)&Mc_)}-&%+Z3!9mdIbvSU-L9FW3OiPx^@kSmK{*O2K! zWcr%j*QKdu?@iNRDkp;vQxZqhs*Y1^>L=eXPsWa>EiX^{%)mE}>9?2RpG?#7OFd?b z$CG7*ah8iuh_>2)d7h4XF}D7tr9LHrzG@+2Mu9eG`U|C(e9ug0WCl7+v#loR^c1v@ zWn|xgMN9q6CJyPbU4TX0`HcerG3Kw%V-{qBwv;7MJ*A2O1)*mQ4pYR?mLPcuy*R;X z6!=}nQbaz9=`NHGZTT$65)UvUnxZYwfj29Wbim!Tc>n+kv)o)xi!dU9Ch71-0N=C8 zlT8v>p1|QaqxjAkDGum|5ldk}H3AW22CS|#zp#|Y!EI>A_w+{(TZw<|NvP`5VRBR# zV5DkVUOWH9nSm#%dv5`*%*3qn2Kh5n2pd!T7HJcaeofcU%mzOLz^{XVoDimHdOF7$ z9tgndkY4HqL6LSyzh;t-8nwX^NKvT-KV^bX7i0jz=?MU2-~chHnQ`wPt}PHuaRi9$ ztP>7!&Rl{=L!l_!X#y0`bskWdDO)EoV(i+D05{kZIiexJR0~>dP!n-ZqGdkPPlh z#TEd1@C0COmHFUuMTR3Mwu?_hQ%)nPdK3=NB7tu2!{%b30m#KL@x^eJ#R&by$WExq zF7Vc^#VGB?;rlj^tu0h>=|w!rAH-~#uCw}AbymrfFfI(gqtgQvY?#ArejJuDKAwvi zD!yEu_jYhLWETjh*7K)EX}Rd?eMsYcz^J1Ps<@w5S$wdMS?w32?cg5f{xU4 z;w5N(rFR18S^<{YCY4g{Kj_=x;qnk(YZnQg8~9P6ni0c_lPUZx8C*1ct|kVHj>5s$ z`S2(}?0ilfdJHU|oETlqNKI@`vOhoQc(f1vUDtXRn(!K&Ib=Du7?*7G9u|#DVDAFz zAlHEs>mb#2I)inv{W|34I(Q1i(M9~b?Ht0pyvKAN+X3X7o)alK&dnsRtB_zid4NtE zA0GX%$_NM~INM6w(xnod$SJUQGEm>dkS>DIi8bQ3eHl50Bh925h1Ddr5POC+eEH8#Y7#f#M z?;8RCK;S0pb2qZ8vslx*xN=P0WPrOr+NLN6} z?k3QK=M;@r%8EAM6LL|Z8B@GB{q{HIa~$!h03Exvm1SGJI(8H62{?Q2US~C$zu9jgp9_zScSy2-?q9 zOXWe&(stt#i66f-F$#b4;d*swg2bEa0xtjnjJKemAb(P_ani0k z+NI$%#T`a87zLx%=lJc?wV+F;bq2uEw5W4ZCSWIr$5ulLaGsXF!|GUe=JQ_tQeJOi zr(AUcG%qnYox=K5z@8yj`T5&-!=a6Bqi@KuzKM*{pK1OynAnV~z?{p@ld%<+zU-n{yW`BeEWTFz!-Zv0#Luhf7W2|&=h&tv!8Ufl%TcuwCZ z0e~Y3PSMFl6PM-c=;>O&mM^SJ}8d&qT4FIu!+)|G-n&gcn3=}_l0wY1v1$rMGfS4+BZJLruW z0BCs=>>dU}67ulSqwCq>R`P&zf-NwOaQ)J%D%VYI9;hhtpg;ZiGCuU}M2|w?v4K#S zUxg62Z)XvA?c}Z6p}-dXhev^nWW$H2CMA*h!^wA)@Bd_^Fs3mLKK}Aoui( z8jgm=Kkv8*DhouHt^X`N{JFZ^R1FInRIlfZNs@AjQf{gw@nd1UCQHr*QL zKM|JR(!7JONH=5)S%*;Q;yQx=c*!@`?BvaKK{%hbzWpHIofaLp?`DAiQ@V=9PNoR6EzS#9tNbII;K1hIPpUSFgoKCz_xy4W+V8*X4k6Ic~fh`nOHGr1AeTt&(q|kOLG}_M3l(AQd{)FTFrFvBF%`kfS=PsJ59}RxWMKk zF)Mc%{vBs2;_!8;|5ihUmRkPRcinw5GuKXliU3lb)}U-MzK5v(Ln7j47Fh^h_V?%ASs?%&O*L1hd-nbllWhC;z`gI{6Ge(Xb4`9fB&JF!&IL+)KO|?U zu<0=+dkF}a&A`jDfM|2uY$vm%i&_pANDjFe-Xua;#gJikjdDKEdn0mNtg~_t!)KNf zs=@T%wBMgX-nF*2h@Hhzj4dT%Xs+;z<6uVy?wx-8gG#Yo`w{oB@?R0UUhg#V-%)tY zYZ?b)9DO}jG})lv%KHlq4s=O{Nt4=7+kNJngZ`*~U4CZeWi_(2uHoywysYVi7;mh9 zMNta*`d1?|wO#jKX7CHDT0nnv*tf6$G|vt@Frm6*TRfoPDQ%5Es>K z#Utmvg&G+!0Hy2EY0^RWq#ll!+POW?^ zCx43NTbb|(@Trn3`3kT5cW)PVm10suCh$BX`l6>RpdtBjvu=s|LC^f=B1K+|m8h+D zL`XiM#q4)WKD_(;mn4-4y_hbzxMiHlF$H_(`><;Am#?K6lddt3Mrhb;&2P5enS7QU zMmQU(-J^oQa}65?pc`I@9L84=WZjZl1g&yIYh`}iPVoA3yqO`w&$@o#-yq>FB|X; zW4EX=z@2lm>(~DCOOavrBbRz#W3d6)B*-24$QiA9BW8KncR!sOWj1i%gf`tY6C(M< zoVfy)+iytf;k9R*jNgI+hPTEkcoYbU!d`cV0Wgn;lD#J&0B{`EiMEAF^G4nEOr%L= z0YW1|qHzmiJuuCG$_ua<-BCEYBUI(>^?(;Di!O2hRNlE~FsU-2-_3RSf7{9dE1SxG zxV;eeT3ZcEo4fl{e+OorTrz;jR9hE&qlVz(LP;K279*{W!np(fvX0;d4QE(6vXpZJ z9StxW7TOxHd<F*RIMkQR`1>EEL!?1YaV|JD>LMj`Kk-V>qlwX)1-HM#;QhrKg zyxNn$f6xzQzm9&T^J~As%2!UP8-VyuC62RFty!ZDd1QPMfIxFWIE=&x zo>5xYsE@z$x7^|YoGr}OmTI6S!j@r_iIgXtv$71>D$zrDDeJcMLreN4%X3*nR*RBl zkq~kKZJ^$rblDTJtO@Re#Bgl!8yHRTd^Y45i)3K%-D9zt+DKN^@+8zrQJwa4(z+3q z^cud5h+k(2Pi{}S8W@RHz5v{Z49>}P-Xc>yz$VoS_?V*)(!66vz8E?N_yJ5j762*O zGqRVsgNL$S{&b;csIBmcmYL0WT{tvUm_C%faX6pj4o8p1hPjhtpt zM#ta{_$Z|Gq&kkAU-eYVFR(RnE`}r#&3U(8q{5aDvTyz2In09WpX0G{)goD1nna3@ePY^w!+}yF*;T~EY}v{ zIOSXPP>cE!Os==BweSs>)1^ORR{6r^YXk$f!iwv)Z$`L_k!~zvp_f7mp@I&KGJ$RT zX^-wbdGb4#Voz<;l%t-cK*H&`_7#zM-`!g36Yb{viW$zkr5_c0yY-o&9nJ)mZJvSA zL{>3!0$}q<@(a>0aNxe2Dz%sQdZ}FO!^nGafG8=8Uwrae*hDSvD3m2N(PzGKExy3Y z{Ev6AM3+21^v;6&BnI#Nj;_K*Tf2*#T zo7c1@uGBqd9wE=#h{w?eAL0;^3AM7qaOk7oMSSgGsHNXjG1=nq>&{q!y z(#AAgXX*AXBt8y_U(k4P)QETDK5y`irt|%NzmO44$E91h^p5S33ZMLq*^JxAi!VVn znl?8?zeOkWLfk)vY~3ooCty&ksPJk4Avs!bMMZZx)BjrDn{4mHy@uKa4y(}eiEr}B zBjaSSNIucw`);sa`$xC|vf|zF71j2=qSKfzs673#8NDjU{WKRjaj7s^|*dYwh1X|J-Ys(?iEm)PL-^p|^a)ex1_KIjYH_FWQBr zAvkc$Izsh=b}+Y&}f!)r)-tNg=( zp~!6y_ctk82mv&6KDFA>rVEG-T3=p&XjtyP@j|z#8@Ta)^Zs>_oigLmvIHq4uuP$H znTe50PhVDNuojb`u$#)ejbS(@WV5mhvaSf zVRa401ywDTA228AZ&?)gTZxfF^%IwRdySYaBa@o5i;7Vzi=i@R@}k3KNXl+-mT?lQ$)`->s{$XoEy z#yMwaMvV9gk>BM_;=jqTY?WAp8^YjMaCof?#RzXV3D>*Ju8jv-%|Tu+O%>-)xb{%s z$k#+QWpNcuB~D^VU>KMI%D7SfrS@ z?V}BiE8z7nX2|?L{|m61qnzMziLUiccS{o_m}tm3EUh&VZju;Kp?~X%u@_##^3s#p z=Hp)4XvHnwA{P_Ek>f(FGbZq5tv=!!TPF97Ts^vYPrjNmsAD=xEX8xxDb&c|r(a2p zR&L#f&w8eaa*qwOvMC{vZS&YY)C)4bGWn&u?=_R|wbha(COTFf-`JtrweAX5uQknr z+CUn8_ARjhEs4DoK(3Y#aS6YiR>FDSm-c?Y_`7^}o}8-&n#{gEL({6p#b^34SX^!% zf}C=O^u>K)~_Tn&IztEfo^F2lTL$Y1u4k z7p}v?4Kys2xJi<_KT+@S64bzLv&7S9z-)ciU8~PcZy(Q*X<%#v#tU+A6H#6!h>XnEwWjTkr# z!?G=uNpV!v-JoCg9{+OJTQt>TZ!g8(s~^K4ZA%lY-KKw~U}trZWmxVwH&(i*ZHlKg z1l`(SEGzY8(s%Xq$7f!sY2+6SFc>L~Pq?*bB*7t#9%e%zu>>!j5^@Eqw-;iW*%JQw zEQnSUzR4TUO%!G&8U_vBR~66mzlh28)XfdJ(^QT|9?4j_Vr12wi&YY6toV0p!GXx8B#xO+sgbVt3F_TDEzfjV+-(Z+_zv{eYE&TCvx+-{Mu)^pcbYI`AA%u>*`^m_lxptVnMGsZ6%XMfWOO17hPL5p{tjzbc^Zaj7R8BD95n%90Sr~v9<^Qm6h*vb3bM!+=yYT*QL`>cseUpe@LFh ziy(Vuf4a_t##!Ws@ffz?gT%$RfVzV$e6M_Lh`ucv+ZP$9)Vlg2RBNuBSX32{v%xTx|xJZ-OyebVEDkv|4EiK z_3h7FIzCxzgb&G+dtPwwB5PpnCu9S@@umNxzC1-bPr0rEBs;FPO z*H0-S!Y1~y+NHSPLn?!+oegUnP77|QThJ)wvq>pA`)CZ z9rG_6c8B!=(lS;>9W4Ka%CL`qsrC!L_3eIdnP2^Eb5^yd{m_>e9&D++I@gqs@$rer zbK6b|de#~p?h*IHQ1w;c51rY!)rv^G?^z$+WN>~WEcDsJck`CCb%Zl5AiZCP&2 z9S^qu8R@Xoq0pwRRrbv>@5Gd%a#2L0e5t^N6{!DPHIz?A|NH3(tAFs?sp;aW#(QN) z>Br_#w=Dkau;jaC?c?j`6XDGlT7zFEHA7M@I8`~XUhsHvuq~))GfL-mwB(t3>M8WZ zQ0&zHlPx*yE&bD{nwv{jV6M37k-nrbT>Q95asLval$NyS>=S_vR8l+FW2ki{Tvboh zJ+pcBh?HkNVf91E^~V3%c`#C{2m)o@AD>(KVa!jrNq8kJ&t~bTSxK7(2%mObt;ppo zk|F8unHj_|q9Bg!yXtV!XRq-rIUvgkT$1E_n?8xHpCIuIF34Jl1j!MAF9QQHk3N!5od@jhQM z37*wpZ<)&>#HNcLtVC2Qz9zqgr=Z2RXhjmMrQ66Edm5ZWH&9OW*H8;$rKP(2|Wd#JeqFM+r2LtoV&VYn`grZZ9p5KwVbaz ztDS#F1Gsxye9hOKvc&J6ycy^}4r|x>Qg)SI?h0MC7l-SkKliVS{5$;<@}r_X{@=@Q zi=Mo?2cI|%HvC##??yB=E>SiQkJKN$O(JIagf8n$ed@i-#JhkG5grK zdaq0R%PrlqKhA4_V(f&zNxieyz}jnph|yx z|BK>HkttrA-}{E5?>^s5_oy+gtKA_=MgG}jydvea5^POkMaORZi~aUr)r}_%XzS{- z?D*I9)&6m(Jxe)6yqx6nzdg5nB5R#eOZ}!>%kIs$1NtI0OOTSi_`45l_P^g0&Gys( zFjTK%^OyH+?65`ge6MRA*J9H^FNcbmV-&3UUA>!L=+QmE(NPhG1gS)!RMdhPW z-_4)xVGE6Tu=t8(o&ki}W7BDJL7{N2h#s?CFmJKRi)Si6Yj^eI8|!1=H5vp{!SKF$ zvl<_A!heG5xu29ghq5Ea%7yi(O^3H$WCVv*oj8!FRcEmHp2(lyHby8{g(_B_MW4(q zzIs?SJqiCOOv;Tqlb-3NyA<2>z^70|B<+Ja-zUB!S_XbvDQ2t9n5>m))DJ_Cqv@U+ zD8_gfO`sAG1930AlXA7XP$H>f)Gk>TESh!uf+v+olS6~|GWI${9!Qzz8>mJmFv{hM z_Gf9KySGaZ2S1vQj!a3-<(u-34P+@6B1B+v-$%u&V(*BtT=Ts5cMc{w&CsHQ+H0Dh z(){%}8!z|w@F4H1%{XqdykD_+ep;8+JJ8Gu{ELl2L?&*VPlG8lt=A`*H7S>=Plhg? zISXY&HVuwxeh~LnA#V~RO zS@l*xvTA0g%XQ#Tz|%BlmPz!c)~PX@+s5pO4_@GVlb~ne=4PLlKCofj<$ciNN|eIf zkLI5BM^P;iRPwQ~?|9eHDjrzVpk>dOD4e3GNFyd&qY@g&p1CJ$D6&{&&7`IB$CBDV zM1*pNbZxqVl~Op)W26j`#EQB11a>w>ea`Z z@x)<$?gWB+_w}%Oh0cdgbZQlhS$UwP2ALXC|F_v33-5_m;%j@Xz2Z_UYKjt6weJeAWS66B+SCDuo*LTqO ziLvX?jIxPqj6~4Xy9qhiD?_<2%AV`b^zlehWmumgb^V^p>HBTA`>so27IlKat8Oav z?;=>PtS_2x_J5iy9Fp0;k}`bAH>PHn%lMjBpjrf*d|EA*x7%%W84RZ}d)SvzUR3+M zp}w^Kck(`D?XEDSqW+mrlPI&t54V`1d|5v1ce{u+FQxG|a4#PAXJW_2$^zByO^Wvd7e2BlJ-QYxuDk&k@Ob{0DjzjXz<%Hr$rbC_ zjF9PYCl21~`Z(ij95bpLyr3_v*FE9zDt%e@i5~nv<7vl{8@aW)81U;^=pC+J{&mz4 z#7le6fMjVeiY|X4f6prJgQzF>pZ^6E3G4O;#((~K(TiR3VWZser4YerNp*x(jf}-4 z4>9RBPf}KnOoSq7r3ik!y3)&1M5ODyLKH_zQix#w#iSZxk8mf_;J2#s8l!FMj#?Ci zq!xBVQI1l0DkP=s&S5nIL1Y+QK;0=}X(u6JN)(xx*rQAarD0sMh74mLA%#Of`{BkGGZ;&1q(v_Jkel6nV;7T|##)uJk;QyP&wK{7 z$2d!UMENBmKO>KN)Z-jS8T=5#Dn5a%W;eRts z=fEmb_jzolO0225r07hiV9tuIqfF9rkb@qG&s;RhVNNmla=L_x4pGfTz%`VXK-{P_&Q5)k!z*L z$Wuep!zzfBl?k3<=1Of*14Um! z=ZUNA61AQ0xspA(BnVGgt_R67#jT)|T>b$Mcs{@-u%A*{hE+y{UxHZhfgud73*`+- znDrF-P%|k)hE%XfeIGkL%syF03s{`YrX6u)3S(?I!Yy{OVhsFZI-SD^qJQ8pG?25x zw|K)F>hST8fgEHZ51GhCHu8~?oMa{c*c&o-Oi-+I3r127moPA$nK+g!4m%lMy$TPt zjw9L>6bnMq?b+5A$YnLJxhgScGk>6wgL+h?5jlXNTY^CdJ@1*%efIO80Uc;T51P=0 zHng4@VT~rg87{PClYV7|Ie(rQM#b!8biJxO3}H>rZ?1y^l)MtyOOKk=Msahg;d2fz zZTTFpNJ2-~;tfx<-~=eJ^{sK8YhC{Dn%BMd^{;^)Y+>K}1WX9VqEY?ib-qJ~N;2&Q zaCR~rbqjqzyK}N<>oA$VB4lFbPUF5~)RtnK+~rQ8p2lYG_&@>HQ-8Y!7bpP}wWtFZ zCm6ycJdtmG@0;KK_V>R59&mvVoZto5HycoJ185*y-D`;!nyA(79uV+BJ;f0q*V-%) zZ@J;qA>ySx2-QbQvY8hgl~)ES#E!3=q}ragX}r@gz`Tsr)8MKJlM~ z3*@rXtaP=v{N-!kOAj7rkyjZ0p%D)*iZ^V50zwd-?|t|C-vRHB8)SjuZkJ1SyBk4Q zfr)}6Ba3ac_(M^2Yf>o(v49WxGI_2lQ%F*h5o$&jte(s2Kz~hV)V6MT)w8_qt8dFW zL;(ekPsd!>7K*GBZ$_oB%ijA>I@ljl)z3tQ8RjRt(n z`4EICq2l3hpZg4HefQB)40x_EU%F2{x1c)@B38nD0)EsDx#jiL=bH5_B5ITf zz1T|>%zp_1Pfs%tNwyX-7G)6+W$_~T<+w`181f(l@<84&q7V@=7==+0YY~M|&>$!< z8I^GvlW`%^t{9;)8vZGQ7Nt=lULjD1LL}S(2Hb$HDgqdVF&uZ07^|@y&G8)FjvCRC zFrc7Nh=SQxhYQpSbyDvjz|lTxVh4XkPsC&Q9)H9Apg~W_aUBV=APthy)bSt>0vduK zB#J^jOktUtX2o_3A)GF|vp$z;1CGNpKHl*ji!NGoxD}2uw zU_c;)Pa$ctCT&u{67nWBV;G1)9ylmVGKHZ!#5ByLN8sr(L_rU50SEQO6fWQ$^rQy_ zA%F0OAup^$Pr#(-paBNt;pbprDtqr1B0whTPbbatEYb3Ya#AN%%?WE+z+f ziuowP4H)19PC))6>mm~X005xCA2i@ECF3tM!9LC*0RVsiszm^bMgWY#E<}L<@WN{R z;T8k{KeAE~5}>yJG6&>=G6_IW@*p!cz<(c-;4c9n5By;=7ZYLxzyJUs1n!{`41h92 zz#oj^FE0QvB!Dc_GB|~EIB91sb#kNpMcE)DW~!y8j35Q_axe8n9va~^gJ3KvfdI~- z2lAjR?PARQK@fiPnYchtL_q@T0VnE#0H%^GzJwAWQyu`|75sq!)G7s@!9Io|0e|Q- z1@a&X1mGQBVKl$eADHA80DuziApxpJ03;*=jKMwylcRpqJ_eFFF*HLp)Lo48CeMMc z4sU4cp%Gxuzzd|)Jaf|kZh6 zNiq`%08Bh0&b|R8d*CFyG6&qiAAjzF0OUazC!-8rfdS|u1db#J=z<#)U`Z%r7%J2* zE_6f9^i0vzKsr<>qv9=xVi>$(DfzN4L?Jg5U`73*4DP{25mQF}Au=-LMkCWcUV%rU zz(-lMOD+Hk2ml%;lSy>O7;+#ZnlZ;sUmmE>eIq`4Pwsb`&_% OG*wk~Rox>X002AC0QV;V From 7e03a36f072475b29526234b634001e47352ac3a Mon Sep 17 00:00:00 2001 From: Guoqiang Ding Date: Tue, 6 Nov 2018 03:21:20 +0800 Subject: [PATCH 180/808] [AIRFLOW-3193] Pin docker requirement version (#4130) Since the change of AIRFLOW-3203, the dockerpy client has upgraded actually. Therefore, the requirements config should upgrade also to avoid potentially misleading. --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 0abe23c6823fe..c187b9b7ff51a 100644 --- a/setup.py +++ b/setup.py @@ -157,7 +157,7 @@ def write_version(filename=os.path.join(*['airflow', 'sphinx-rtd-theme>=0.1.6', 'Sphinx-PyPI-upload>=0.2.1' ] -docker = ['docker>=2.0.0'] +docker = ['docker>=3.0.0'] druid = ['pydruid>=0.4.1'] elasticsearch = [ 'elasticsearch>=5.0.0,<6.0.0', From 412985dc6f7f5af4440dc91dbef1770603a0071a Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Thu, 8 Nov 2018 14:34:41 +0000 Subject: [PATCH 181/808] [AIRFLOW-2799] Fix filtering UI objects by datetime (#4061) Our conversion to Timezone-aware date times in 1.10 missed this case - any filter on a date time would blow up with `naive datetime is disallowed` - this makes our datetime filters timezone aware now (they respect the default_timezone, and they accept timezones in input even though the UI won't let you craft those.) I manually tested this by changing query parameters - a value of `2018-10-30+01%3A05%3A00%2B01:00` matched against an execution date of 00:05:00+00:00 --- airflow/utils/timezone.py | 4 +- airflow/www/utils.py | 42 +++++++++++++++++++-- airflow/www_rbac/utils.py | 71 +++++++++++++++++++++++++++++++++++- airflow/www_rbac/views.py | 24 +----------- tests/www/test_views.py | 18 +++++++++ tests/www_rbac/test_views.py | 13 +++++++ 6 files changed, 143 insertions(+), 29 deletions(-) diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py index 6d49fbcbb3e85..5adaa2f5c4a5a 100644 --- a/airflow/utils/timezone.py +++ b/airflow/utils/timezone.py @@ -164,9 +164,9 @@ def datetime(*args, **kwargs): return dt.datetime(*args, **kwargs) -def parse(string): +def parse(string, timezone=None): """ Parse a time string and return an aware datetime :param string: time string """ - return pendulum.parse(string, tz=TIMEZONE) + return pendulum.parse(string, tz=timezone or TIMEZONE) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 1566faa91ac1e..3f869e625180f 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -33,8 +33,8 @@ import zipfile from flask import after_this_request, request, Response -from flask_admin.contrib.sqla.filters import FilterConverter from flask_admin.model import filters +import flask_admin.contrib.sqla.filters as sqlafilters from flask_login import current_user import wtforms from wtforms.compat import text_type @@ -419,7 +419,43 @@ def __call__(self, field, **kwargs): return wtforms.widgets.core.HTMLString(html) -class UtcFilterConverter(FilterConverter): +class UtcDateTimeFilterMixin(object): + def clean(self, value): + dt = super(UtcDateTimeFilterMixin, self).clean(value) + return timezone.make_aware(dt, timezone=timezone.utc) + + +class UtcDateTimeEqualFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeEqualFilter): + pass + + +class UtcDateTimeNotEqualFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeNotEqualFilter): + pass + + +class UtcDateTimeGreaterFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeGreaterFilter): + pass + + +class UtcDateTimeSmallerFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeSmallerFilter): + pass + + +class UtcDateTimeBetweenFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeBetweenFilter): + pass + + +class UtcDateTimeNotBetweenFilter(UtcDateTimeFilterMixin, sqlafilters.DateTimeNotBetweenFilter): + pass + + +class UtcFilterConverter(sqlafilters.FilterConverter): + + utcdatetime_filters = (UtcDateTimeEqualFilter, UtcDateTimeNotEqualFilter, + UtcDateTimeGreaterFilter, UtcDateTimeSmallerFilter, + UtcDateTimeBetweenFilter, UtcDateTimeNotBetweenFilter, + sqlafilters.FilterEmpty) + @filters.convert('utcdatetime') def conv_utcdatetime(self, column, name, **kwargs): - return self.conv_datetime(column, name, **kwargs) + return [f(column, name, **kwargs) for f in self.utcdatetime_filters] diff --git a/airflow/www_rbac/utils.py b/airflow/www_rbac/utils.py index 0176a5312c373..b25e1541abd85 100644 --- a/airflow/www_rbac/utils.py +++ b/airflow/www_rbac/utils.py @@ -37,7 +37,10 @@ from pygments import highlight, lexers from pygments.formatters import HtmlFormatter from flask import request, Response, Markup, url_for -from airflow import configuration +from flask_appbuilder.models.sqla.interface import SQLAInterface +import flask_appbuilder.models.sqla.filters as fab_sqlafilters +import sqlalchemy as sqla +from airflow import configuration, settings from airflow.models import BaseOperator from airflow.operators.subdag_operator import SubDagOperator from airflow.utils import timezone @@ -378,3 +381,69 @@ def get_chart_height(dag): charts, that is charts that take up space based on the size of the components within. """ return 600 + len(dag.tasks) * 10 + + +class UtcAwareFilterMixin(object): + def apply(self, query, value): + value = timezone.parse(value, timezone=timezone.utc) + + return super(UtcAwareFilterMixin, self).apply(query, value) + + +class UtcAwareFilterEqual(UtcAwareFilterMixin, fab_sqlafilters.FilterEqual): + pass + + +class UtcAwareFilterGreater(UtcAwareFilterMixin, fab_sqlafilters.FilterGreater): + pass + + +class UtcAwareFilterSmaller(UtcAwareFilterMixin, fab_sqlafilters.FilterSmaller): + pass + + +class UtcAwareFilterNotEqual(UtcAwareFilterMixin, fab_sqlafilters.FilterNotEqual): + pass + + +class UtcAwareFilterConverter(fab_sqlafilters.SQLAFilterConverter): + + conversion_table = ( + (('is_utcdatetime', [UtcAwareFilterEqual, + UtcAwareFilterGreater, + UtcAwareFilterSmaller, + UtcAwareFilterNotEqual]),) + + fab_sqlafilters.SQLAFilterConverter.conversion_table + ) + + +class CustomSQLAInterface(SQLAInterface): + """ + FAB does not know how to handle columns with leading underscores because + they are not supported by WTForm. This hack will remove the leading + '_' from the key to lookup the column names. + + """ + def __init__(self, obj): + super(CustomSQLAInterface, self).__init__(obj) + + self.session = settings.Session() + + def clean_column_names(): + if self.list_properties: + self.list_properties = dict( + (k.lstrip('_'), v) for k, v in self.list_properties.items()) + if self.list_columns: + self.list_columns = dict( + (k.lstrip('_'), v) for k, v in self.list_columns.items()) + + clean_column_names() + + def is_utcdatetime(self, col_name): + from airflow.utils.sqlalchemy import UtcDateTime + obj = self.list_columns[col_name].type + return isinstance(obj, UtcDateTime) or \ + isinstance(obj, sqla.types.TypeDecorator) and \ + isinstance(obj.impl, UtcDateTime) + + filter_converter_class = UtcAwareFilterConverter diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 50f30649039bf..51af6c0e2a415 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -42,9 +42,7 @@ from flask._compat import PY2 from flask_appbuilder import BaseView, ModelView, expose, has_access -from flask_appbuilder.models.sqla.interface import SQLAInterface from flask_appbuilder.actions import action - from flask_babel import lazy_gettext from wtforms import Form, SelectField, validators @@ -1645,27 +1643,7 @@ class AirflowModelView(ModelView): list_widget = AirflowModelListWidget page_size = PAGE_SIZE - class CustomSQLAInterface(SQLAInterface): - """ - FAB does not know how to handle columns with leading underscores because - they are not supported by WTForm. This hack will remove the leading - '_' from the key to lookup the column names. - - """ - def __init__(self, obj): - super(AirflowModelView.CustomSQLAInterface, self).__init__(obj) - - self.session = settings.Session() - - def clean_column_names(): - if self.list_properties: - self.list_properties = dict( - (k.lstrip('_'), v) for k, v in self.list_properties.items()) - if self.list_columns: - self.list_columns = dict( - (k.lstrip('_'), v) for k, v in self.list_columns.items()) - - clean_column_names() + CustomSQLAInterface = wwwutils.CustomSQLAInterface class SlaMissModelView(AirflowModelView): diff --git a/tests/www/test_views.py b/tests/www/test_views.py index bb183d4a649b7..4075f91d537a6 100644 --- a/tests/www/test_views.py +++ b/tests/www/test_views.py @@ -772,5 +772,23 @@ def test_dt_nr_dr_form_with_base_date_and_num_runs_and_execution_date_within(sel self.tester.test_with_base_date_and_num_runs_and_execution_date_within() +class TestTaskInstanceView(unittest.TestCase): + TI_ENDPOINT = '/admin/taskinstance/?flt2_execution_date_greater_than={}' + + def setUp(self): + super(TestTaskInstanceView, self).setUp() + configuration.load_test_config() + app = application.create_app(testing=True) + app.config['WTF_CSRF_METHODS'] = [] + self.app = app.test_client() + + def test_start_date_filter(self): + resp = self.app.get(self.TI_ENDPOINT.format('2018-10-09+22:44:31')) + # We aren't checking the logic of the date filter itself (that is built + # in to flask-admin) but simply that our UTC conversion was run - i.e. it + # doesn't blow up! + self.assertEqual(resp.status_code, 200) + + if __name__ == '__main__': unittest.main() diff --git a/tests/www_rbac/test_views.py b/tests/www_rbac/test_views.py index 6f37f44e54614..d7f099d6be23b 100644 --- a/tests/www_rbac/test_views.py +++ b/tests/www_rbac/test_views.py @@ -773,5 +773,18 @@ def test_dt_nr_dr_form_with_base_date_and_num_runs_and_execution_date_within(sel self.tester.test_with_base_date_and_num_runs_and_execution_date_within() +class TestTaskInstanceView(TestBase): + TI_ENDPOINT = '/taskinstance/list/?_flt_0_execution_date={}' + + def test_start_date_filter(self): + resp = self.client.get(self.TI_ENDPOINT.format( + self.percent_encode('2018-10-09 22:44:31'))) + # We aren't checking the logic of the date filter itself (that is built + # in to FAB) but simply that our UTC conversion was run - i.e. it + # doesn't blow up! + self.check_content_in_response('List Task Instance', resp) + pass + + if __name__ == '__main__': unittest.main() From 7e1cd4212c91bd3cc82dc7b7620e7ffc6eaad197 Mon Sep 17 00:00:00 2001 From: yangaws <31293788+yangaws@users.noreply.github.com> Date: Thu, 8 Nov 2018 13:59:38 -0800 Subject: [PATCH 182/808] [AIRFLOW-2524] More AWS SageMaker operators, sensors for model, endpoint-config and endpoint (#4126) --- airflow/contrib/hooks/aws_hook.py | 6 +- .../operators/sagemaker_base_operator.py | 2 +- .../sagemaker_endpoint_config_operator.py | 67 ++++++++ .../operators/sagemaker_endpoint_operator.py | 151 ++++++++++++++++++ .../operators/sagemaker_model_operator.py | 68 ++++++++ .../operators/sagemaker_training_operator.py | 20 +-- .../operators/sagemaker_transform_operator.py | 41 +++-- .../operators/sagemaker_tuning_operator.py | 24 +-- .../sensors/sagemaker_endpoint_sensor.py | 61 +++++++ airflow/hooks/S3_hook.py | 18 +-- docs/code.rst | 6 +- ...test_sagemaker_endpoint_config_operator.py | 92 +++++++++++ .../test_sagemaker_endpoint_operator.py | 126 +++++++++++++++ .../test_sagemaker_model_operator.py | 83 ++++++++++ .../sensors/test_sagemaker_endpoint_sensor.py | 110 +++++++++++++ .../sensors/test_sagemaker_training_sensor.py | 2 +- .../test_sagemaker_transform_sensor.py | 2 +- .../sensors/test_sagemaker_tuning_sensor.py | 2 +- 18 files changed, 830 insertions(+), 51 deletions(-) create mode 100644 airflow/contrib/operators/sagemaker_endpoint_config_operator.py create mode 100644 airflow/contrib/operators/sagemaker_endpoint_operator.py create mode 100644 airflow/contrib/operators/sagemaker_model_operator.py create mode 100644 airflow/contrib/sensors/sagemaker_endpoint_sensor.py create mode 100644 tests/contrib/operators/test_sagemaker_endpoint_config_operator.py create mode 100644 tests/contrib/operators/test_sagemaker_endpoint_operator.py create mode 100644 tests/contrib/operators/test_sagemaker_model_operator.py create mode 100644 tests/contrib/sensors/test_sagemaker_endpoint_sensor.py diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py index 44f33d49ab9e4..5716eb3c0305e 100644 --- a/airflow/contrib/hooks/aws_hook.py +++ b/airflow/contrib/hooks/aws_hook.py @@ -182,7 +182,7 @@ def get_session(self, region_name=None): def get_credentials(self, region_name=None): """Get the underlying `botocore.Credentials` object. - This contains the attributes: access_key, secret_key and token. + This contains the following authentication attributes: access_key, secret_key and token. """ session, _ = self._get_credentials(region_name) # Credentials are refreshable, so accessing your access key and @@ -192,8 +192,8 @@ def get_credentials(self, region_name=None): def expand_role(self, role): """ - Expand an IAM role name to an IAM role ARN. If role is already an IAM ARN, - no change is made. + If the IAM role is a role name, get the Amazon Resource Name (ARN) for the role. + If IAM role is already an IAM role ARN, no change is made. :param role: IAM role name or ARN :return: IAM role ARN diff --git a/airflow/contrib/operators/sagemaker_base_operator.py b/airflow/contrib/operators/sagemaker_base_operator.py index cf1e59387a784..08d6d0eb6a83c 100644 --- a/airflow/contrib/operators/sagemaker_base_operator.py +++ b/airflow/contrib/operators/sagemaker_base_operator.py @@ -79,7 +79,7 @@ def parse_config_integers(self): self.parse_integer(self.config, field) def expand_role(self): - raise NotImplementedError('Please implement expand_role() in sub class!') + pass def preprocess_config(self): self.log.info( diff --git a/airflow/contrib/operators/sagemaker_endpoint_config_operator.py b/airflow/contrib/operators/sagemaker_endpoint_config_operator.py new file mode 100644 index 0000000000000..a94cf3022904b --- /dev/null +++ b/airflow/contrib/operators/sagemaker_endpoint_config_operator.py @@ -0,0 +1,67 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerEndpointConfigOperator(SageMakerBaseOperator): + + """ + Create a SageMaker endpoint config. + + This operator returns The ARN of the endpoint config created in Amazon SageMaker + + :param config: The configuration necessary to create an endpoint config. + + For details of the configuration parameter, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_endpoint_config + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + """ # noqa: E501 + + integer_fields = [ + ['ProductionVariants', 'InitialInstanceCount'] + ] + + @apply_defaults + def __init__(self, + config, + *args, **kwargs): + super(SageMakerEndpointConfigOperator, self).__init__(config=config, + *args, **kwargs) + + self.config = config + + def execute(self, context): + self.preprocess_config() + + self.log.info('Creating SageMaker Endpoint Config %s.', self.config['EndpointConfigName']) + response = self.hook.create_endpoint_config(self.config) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException( + 'Sagemaker endpoint config creation failed: %s' % response) + else: + return { + 'EndpointConfig': self.hook.describe_endpoint_config( + self.config['EndpointConfigName'] + ) + } diff --git a/airflow/contrib/operators/sagemaker_endpoint_operator.py b/airflow/contrib/operators/sagemaker_endpoint_operator.py new file mode 100644 index 0000000000000..4094fbe59e2b8 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_endpoint_operator.py @@ -0,0 +1,151 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerEndpointOperator(SageMakerBaseOperator): + + """ + Create a SageMaker endpoint. + + This operator returns The ARN of the endpoint created in Amazon SageMaker + + :param config: + The configuration necessary to create an endpoint. + + If you need to create a SageMaker endpoint based on an existed SageMaker model and an existed SageMaker + endpoint config, + + config = endpoint_configuration; + + If you need to create all of SageMaker model, SageMaker endpoint-config and SageMaker endpoint, + + config = { + 'Model': model_configuration, + + 'EndpointConfig': endpoint_config_configuration, + + 'Endpoint': endpoint_configuration + } + + For details of the configuration parameter of model_configuration, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_model + + For details of the configuration parameter of endpoint_config_configuration, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_endpoint_config + + For details of the configuration parameter of endpoint_configuration, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_endpoint + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + :param wait_for_completion: Whether the operator should wait until the endpoint creation finishes. + :type wait_for_completion: bool + :param check_interval: If wait is set to True, this is the time interval, in seconds, that this operation waits + before polling the status of the endpoint creation. + :type check_interval: int + :param max_ingestion_time: If wait is set to True, this operation fails if the endpoint creation doesn't finish + within max_ingestion_time seconds. If you set this parameter to None it never times out. + :type max_ingestion_time: int + :param operation: Whether to create an endpoint or update an endpoint. Must be either 'create or 'update'. + :type operation: str + """ # noqa: E501 + + @apply_defaults + def __init__(self, + config, + wait_for_completion=True, + check_interval=30, + max_ingestion_time=None, + operation='create', + *args, **kwargs): + super(SageMakerEndpointOperator, self).__init__(config=config, + *args, **kwargs) + + self.config = config + self.wait_for_completion = wait_for_completion + self.check_interval = check_interval + self.max_ingestion_time = max_ingestion_time + self.operation = operation.lower() + if self.operation not in ['create', 'update']: + raise ValueError('Invalid value! Argument operation has to be one of "create" and "update"') + self.create_integer_fields() + + def create_integer_fields(self): + if 'EndpointConfig' in self.config: + self.integer_fields = [ + ['EndpointConfig', 'ProductionVariants', 'InitialInstanceCount'] + ] + + def expand_role(self): + if 'Model' not in self.config: + return + hook = AwsHook(self.aws_conn_id) + config = self.config['Model'] + if 'ExecutionRoleArn' in config: + config['ExecutionRoleArn'] = hook.expand_role(config['ExecutionRoleArn']) + + def execute(self, context): + self.preprocess_config() + + model_info = self.config.get('Model') + endpoint_config_info = self.config.get('EndpointConfig') + endpoint_info = self.config.get('Endpoint', self.config) + + if model_info: + self.log.info('Creating SageMaker model %s.', model_info['ModelName']) + self.hook.create_model(model_info) + + if endpoint_config_info: + self.log.info('Creating endpoint config %s.', endpoint_config_info['EndpointConfigName']) + self.hook.create_endpoint_config(endpoint_config_info) + + if self.operation == 'create': + sagemaker_operation = self.hook.create_endpoint + log_str = 'Creating' + elif self.operation == 'update': + sagemaker_operation = self.hook.update_endpoint + log_str = 'Updating' + else: + raise ValueError('Invalid value! Argument operation has to be one of "create" and "update"') + + self.log.info('{} SageMaker endpoint {}.'.format(log_str, endpoint_info['EndpointName'])) + + response = sagemaker_operation( + endpoint_info, + wait_for_completion=self.wait_for_completion, + check_interval=self.check_interval, + max_ingestion_time=self.max_ingestion_time + ) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException( + 'Sagemaker endpoint creation failed: %s' % response) + else: + return { + 'EndpointConfig': self.hook.describe_endpoint_config( + endpoint_info['EndpointConfigName'] + ), + 'Endpoint': self.hook.describe_endpoint( + endpoint_info['EndpointName'] + ) + } diff --git a/airflow/contrib/operators/sagemaker_model_operator.py b/airflow/contrib/operators/sagemaker_model_operator.py new file mode 100644 index 0000000000000..4332daa9db341 --- /dev/null +++ b/airflow/contrib/operators/sagemaker_model_operator.py @@ -0,0 +1,68 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.aws_hook import AwsHook +from airflow.contrib.operators.sagemaker_base_operator import SageMakerBaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.exceptions import AirflowException + + +class SageMakerModelOperator(SageMakerBaseOperator): + + """ + Create a SageMaker model. + + This operator returns The ARN of the model created in Amazon SageMaker + + :param config: The configuration necessary to create a model. + + For details of the configuration parameter, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_model + :type config: dict + :param aws_conn_id: The AWS connection ID to use. + :type aws_conn_id: str + """ # noqa: E501 + + @apply_defaults + def __init__(self, + config, + *args, **kwargs): + super(SageMakerModelOperator, self).__init__(config=config, + *args, **kwargs) + + self.config = config + + def expand_role(self): + if 'ExecutionRoleArn' in self.config: + hook = AwsHook(self.aws_conn_id) + self.config['ExecutionRoleArn'] = hook.expand_role(self.config['ExecutionRoleArn']) + + def execute(self, context): + self.preprocess_config() + + self.log.info('Creating SageMaker Model %s.', self.config['ModelName']) + response = self.hook.create_model(self.config) + if response['ResponseMetadata']['HTTPStatusCode'] != 200: + raise AirflowException('Sagemaker model creation failed: %s' % response) + else: + return { + 'Model': self.hook.describe_model( + self.config['ModelName'] + ) + } diff --git a/airflow/contrib/operators/sagemaker_training_operator.py b/airflow/contrib/operators/sagemaker_training_operator.py index 69036925f34f7..d90f7e6555697 100644 --- a/airflow/contrib/operators/sagemaker_training_operator.py +++ b/airflow/contrib/operators/sagemaker_training_operator.py @@ -29,23 +29,26 @@ class SageMakerTrainingOperator(SageMakerBaseOperator): This operator returns The ARN of the training job created in Amazon SageMaker. - :param config: The configuration necessary to start a training job (templated) + :param config: The configuration necessary to start a training job (templated). + + For details of the configuration parameter, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_training_job :type config: dict :param aws_conn_id: The AWS connection ID to use. :type aws_conn_id: str - :param wait_for_completion: if the operator should block until training job finishes + :param wait_for_completion: If wait is set to True, the time interval, in seconds, + that the operation waits to check the status of the training job. :type wait_for_completion: bool :param print_log: if the operator should print the cloudwatch log during training :type print_log: bool :param check_interval: if wait is set to be true, this is the time interval in seconds which the operator will check the status of the training job :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the training job hasn't finish within the max_ingestion_time in seconds - (Caution: be careful to set this parameters because training can take very long) - Setting it to None implies no timeout. + :param max_ingestion_time: If wait is set to True, the operation fails if the training job + doesn't finish within max_ingestion_time seconds. If you set this parameter to None, + the operation does not timeout. :type max_ingestion_time: int - """ + """ # noqa: E501 integer_fields = [ ['ResourceConfig', 'InstanceCount'], @@ -87,8 +90,7 @@ def execute(self, context): max_ingestion_time=self.max_ingestion_time ) if response['ResponseMetadata']['HTTPStatusCode'] != 200: - raise AirflowException( - 'Sagemaker Training Job creation failed: %s' % response) + raise AirflowException('Sagemaker Training Job creation failed: %s' % response) else: return { 'Training': self.hook.describe_training_job( diff --git a/airflow/contrib/operators/sagemaker_transform_operator.py b/airflow/contrib/operators/sagemaker_transform_operator.py index 7be570cdacd6f..9d1c665f9eea5 100644 --- a/airflow/contrib/operators/sagemaker_transform_operator.py +++ b/airflow/contrib/operators/sagemaker_transform_operator.py @@ -29,26 +29,39 @@ class SageMakerTransformOperator(SageMakerBaseOperator): This operator returns The ARN of the model created in Amazon SageMaker. - :param config: The configuration necessary to start a transform job (templated) + :param config: The configuration necessary to start a transform job (templated). + + If you need to create a SageMaker transform job based on an existed SageMaker model, + + config = transform_config; + + If you need to create both SageMaker model and SageMaker Transform job, + + config = { + 'Model': model_config, + + 'Transform': transform_config + } + + For details of the configuration parameter of transform_config, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_transform_job + + For details of the configuration parameter of model_config, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_model + :type config: dict - :param model_config: - The configuration necessary to create a SageMaker model, the default is none - which means the SageMaker model used for the SageMaker transform job already exists. - If given, it will be used to create a SageMaker model before creating - the SageMaker transform job - :type model_config: dict :param aws_conn_id: The AWS connection ID to use. :type aws_conn_id: string - :param wait_for_completion: if the program should keep running until job finishes + :param wait_for_completion: Set to True to wait until the transform job finishes. :type wait_for_completion: bool - :param check_interval: if wait is set to be true, this is the time interval - in seconds which the operator will check the status of the transform job + :param check_interval: If wait is set to True, the time interval, in seconds, + that this operation waits to check the status of the transform job. :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the transform job hasn't finish within the max_ingestion_time in seconds - (Caution: be careful to set this parameters because transform can take very long) + :param max_ingestion_time: If wait is set to True, the operation fails + if the transform job doesn't finish within max_ingestion_time seconds. If you + set this parameter to None, the operation does not timeout. :type max_ingestion_time: int - """ + """ # noqa: E501 @apply_defaults def __init__(self, diff --git a/airflow/contrib/operators/sagemaker_tuning_operator.py b/airflow/contrib/operators/sagemaker_tuning_operator.py index 94c995072a8da..dc1282b5b8273 100644 --- a/airflow/contrib/operators/sagemaker_tuning_operator.py +++ b/airflow/contrib/operators/sagemaker_tuning_operator.py @@ -25,24 +25,27 @@ class SageMakerTuningOperator(SageMakerBaseOperator): """ - Initiate a SageMaker hyper-parameter tuning job. + Initiate a SageMaker hyperparameter tuning job. This operator returns The ARN of the tuning job created in Amazon SageMaker. - :param config: The configuration necessary to start a tuning job (templated) + :param config: The configuration necessary to start a tuning job (templated). + + For details of the configuration parameter, See: + https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/sagemaker.html#SageMaker.Client.create_hyper_parameter_tuning_job :type config: dict :param aws_conn_id: The AWS connection ID to use. :type aws_conn_id: str - :param wait_for_completion: if the operator should block until tuning job finishes + :param wait_for_completion: Set to True to wait until the tuning job finishes. :type wait_for_completion: bool - :param check_interval: if wait is set to be true, this is the time interval - in seconds which the operator will check the status of the tuning job + :param check_interval: If wait is set to True, the time interval, in seconds, + that this operation waits to check the status of the tuning job. :type check_interval: int - :param max_ingestion_time: if wait is set to be true, the operator will fail - if the tuning job hasn't finish within the max_ingestion_time in seconds - (Caution: be careful to set this parameters because tuning can take very long) + :param max_ingestion_time: If wait is set to True, the operation fails + if the tuning job doesn't finish within max_ingestion_time seconds. If you + set this parameter to None, the operation does not timeout. :type max_ingestion_time: int - """ + """ # noqa: E501 integer_fields = [ ['HyperParameterTuningJobConfig', 'ResourceLimits', 'MaxNumberOfTrainingJobs'], @@ -87,8 +90,7 @@ def execute(self, context): max_ingestion_time=self.max_ingestion_time ) if response['ResponseMetadata']['HTTPStatusCode'] != 200: - raise AirflowException( - 'Sagemaker Tuning Job creation failed: %s' % response) + raise AirflowException('Sagemaker Tuning Job creation failed: %s' % response) else: return { 'Tuning': self.hook.describe_tuning_job( diff --git a/airflow/contrib/sensors/sagemaker_endpoint_sensor.py b/airflow/contrib/sensors/sagemaker_endpoint_sensor.py new file mode 100644 index 0000000000000..ceed9c10097d9 --- /dev/null +++ b/airflow/contrib/sensors/sagemaker_endpoint_sensor.py @@ -0,0 +1,61 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.sensors.sagemaker_base_sensor import SageMakerBaseSensor +from airflow.utils.decorators import apply_defaults + + +class SageMakerEndpointSensor(SageMakerBaseSensor): + """ + Asks for the state of the endpoint state until it reaches a terminal state. + If it fails the sensor errors, the task fails. + + :param job_name: job_name of the endpoint instance to check the state of + :type job_name: str + """ + + template_fields = ['endpoint_name'] + template_ext = () + + @apply_defaults + def __init__(self, + endpoint_name, + *args, + **kwargs): + super(SageMakerEndpointSensor, self).__init__(*args, **kwargs) + self.endpoint_name = endpoint_name + + def non_terminal_states(self): + return SageMakerHook.endpoint_non_terminal_states + + def failed_states(self): + return SageMakerHook.failed_states + + def get_sagemaker_response(self): + sagemaker = SageMakerHook(aws_conn_id=self.aws_conn_id) + + self.log.info('Poking Sagemaker Endpoint %s', self.endpoint_name) + return sagemaker.describe_endpoint(self.endpoint_name) + + def get_failed_reason_from_response(self, response): + return response['FailureReason'] + + def state_from_response(self, response): + return response['EndpointStatus'] diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py index a64c331cdd09e..ff3b98b6a91da 100644 --- a/airflow/hooks/S3_hook.py +++ b/airflow/hooks/S3_hook.py @@ -69,11 +69,11 @@ def get_bucket(self, bucket_name): def create_bucket(self, bucket_name, region_name=None): """ - Creates a boto3.S3.Bucket object + Creates an Amazon S3 bucket. - :param bucket_name: the name of the bucket + :param bucket_name: The name of the bucket :type bucket_name: str - :param region__name: the name of the aws region + :param region_name: The name of the aws region in which to create the bucket. :type region_name: str """ s3_conn = self.get_conn() @@ -405,19 +405,19 @@ def load_file_obj(self, replace=False, encrypt=False): """ - Loads file object to S3 + Loads a file object to S3 - :param file_obj: file-like object to set as content for the key. + :param file_obj: The file-like object to set as the content for the S3 key. :type file_obj: file-like object :param key: S3 key that will point to the file :type key: str :param bucket_name: Name of the bucket in which to store the file :type bucket_name: str - :param replace: A flag to decide whether or not to overwrite the key - if it already exists + :param replace: A flag that indicates whether to overwrite the key + if it already exists. :type replace: bool - :param encrypt: If True, the file will be encrypted on the server-side - by S3 and will be stored in an encrypted form while at rest in S3. + :param encrypt: If True, S3 encrypts the file on the server, + and the file is stored in encrypted form at rest in S3. :type encrypt: bool """ if not bucket_name: diff --git a/docs/code.rst b/docs/code.rst index d577b768eb79c..ca6ca593ca345 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -183,9 +183,12 @@ Operators .. autoclass:: airflow.contrib.operators.s3_list_operator.S3ListOperator .. autoclass:: airflow.contrib.operators.s3_to_gcs_operator.S3ToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.sagemaker_base_operator.SageMakerBaseOperator +.. autoclass:: airflow.contrib.operators.sagemaker_endpoint_operator.SageMakerEndpointOperator +.. autoclass:: airflow.contrib.operators.sagemaker_endpoint_config_operator.SageMakerEndpointConfigOperator +.. autoclass:: airflow.contrib.operators.sagemaker_model_operator.SageMakerModelOperator .. autoclass:: airflow.contrib.operators.sagemaker_training_operator.SageMakerTrainingOperator .. autoclass:: airflow.contrib.operators.sagemaker_transform_operator.SageMakerTransformOperator -.. autoclass:: airflow.contrib.operators.sagemaker_tuning_operator.SagemakerTuningOperator +.. autoclass:: airflow.contrib.operators.sagemaker_tuning_operator.SageMakerTuningOperator .. autoclass:: airflow.contrib.operators.segment_track_event_operator.SegmentTrackEventOperator .. autoclass:: airflow.contrib.operators.sftp_operator.SFTPOperator .. autoclass:: airflow.contrib.operators.slack_webhook_operator.SlackWebhookOperator @@ -223,6 +226,7 @@ Sensors .. autoclass:: airflow.contrib.sensors.qubole_sensor.QuboleSensor .. autoclass:: airflow.contrib.sensors.redis_key_sensor.RedisKeySensor .. autoclass:: airflow.contrib.sensors.sagemaker_base_sensor.SageMakerBaseSensor +.. autoclass:: airflow.contrib.sensors.sagemaker_endpoint_sensor.SageMakerEndpointSensor .. autoclass:: airflow.contrib.sensors.sagemaker_training_sensor.SageMakerTrainingSensor .. autoclass:: airflow.contrib.sensors.sagemaker_transform_sensor.SageMakerTransformSensor .. autoclass:: airflow.contrib.sensors.sagemaker_tuning_sensor.SageMakerTuningSensor diff --git a/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py b/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py new file mode 100644 index 0000000000000..658fd813df73e --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_endpoint_config_operator.py @@ -0,0 +1,92 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_endpoint_config_operator \ + import SageMakerEndpointConfigOperator +from airflow.exceptions import AirflowException + +model_name = 'test-model-name' +config_name = 'test-config-name' + +create_endpoint_config_params = { + 'EndpointConfigName': config_name, + 'ProductionVariants': [ + { + 'VariantName': 'AllTraffic', + 'ModelName': model_name, + 'InitialInstanceCount': '1', + 'InstanceType': 'ml.c4.xlarge' + } + ] +} + + +class TestSageMakerEndpointConfigOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerEndpointConfigOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=create_endpoint_config_params + ) + + def test_parse_config_integers(self): + self.sagemaker.parse_config_integers() + for variant in self.sagemaker.config['ProductionVariants']: + self.assertEqual(variant['InitialInstanceCount'], + int(variant['InitialInstanceCount'])) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_endpoint_config') + def test_execute(self, mock_model, mock_client): + mock_model.return_value = { + 'EndpointConfigArn': 'testarn', + 'ResponseMetadata': { + 'HTTPStatusCode': 200 + } + } + self.sagemaker.execute(None) + mock_model.assert_called_once_with(create_endpoint_config_params) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + def test_execute_with_failure(self, mock_model, mock_client): + mock_model.return_value = { + 'EndpointConfigArn': 'testarn', + 'ResponseMetadata': { + 'HTTPStatusCode': 200 + } + } + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_endpoint_operator.py b/tests/contrib/operators/test_sagemaker_endpoint_operator.py new file mode 100644 index 0000000000000..710daceff5b57 --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_endpoint_operator.py @@ -0,0 +1,126 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_endpoint_operator \ + import SageMakerEndpointOperator +from airflow.exceptions import AirflowException + +role = 'arn:aws:iam:role/test-role' +bucket = 'test-bucket' +image = 'test-image' +output_url = 's3://{}/test/output'.format(bucket) +model_name = 'test-model-name' +config_name = 'test-endpoint-config-name' +endpoint_name = 'test-endpoint-name' + +create_model_params = { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role +} + +create_endpoint_config_params = { + 'EndpointConfigName': config_name, + 'ProductionVariants': [ + { + 'VariantName': 'AllTraffic', + 'ModelName': model_name, + 'InitialInstanceCount': '1', + 'InstanceType': 'ml.c4.xlarge' + } + ] +} + +create_endpoint_params = { + 'EndpointName': endpoint_name, + 'EndpointConfigName': config_name +} + +config = { + 'Model': create_model_params, + 'EndpointConfig': create_endpoint_config_params, + 'Endpoint': create_endpoint_params +} + + +class TestSageMakerEndpointOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerEndpointOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=config, + wait_for_completion=False, + check_interval=5, + operation='create' + ) + + def test_parse_config_integers(self): + self.sagemaker.parse_config_integers() + for variant in self.sagemaker.config['EndpointConfig']['ProductionVariants']: + self.assertEqual(variant['InitialInstanceCount'], + int(variant['InitialInstanceCount'])) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_endpoint_config') + @mock.patch.object(SageMakerHook, 'create_endpoint') + def test_execute(self, mock_endpoint, mock_endpoint_config, + mock_model, mock_client): + mock_endpoint.return_value = {'EndpointArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + self.sagemaker.execute(None) + mock_model.assert_called_once_with(create_model_params) + mock_endpoint_config.assert_called_once_with(create_endpoint_config_params) + mock_endpoint.assert_called_once_with(create_endpoint_params, + wait_for_completion=False, + check_interval=5, + max_ingestion_time=None + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + @mock.patch.object(SageMakerHook, 'create_endpoint_config') + @mock.patch.object(SageMakerHook, 'create_endpoint') + def test_execute_with_failure(self, mock_endpoint, mock_endpoint_config, + mock_model, mock_client): + mock_endpoint.return_value = {'EndpointArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/operators/test_sagemaker_model_operator.py b/tests/contrib/operators/test_sagemaker_model_operator.py new file mode 100644 index 0000000000000..6b6d12de10264 --- /dev/null +++ b/tests/contrib/operators/test_sagemaker_model_operator.py @@ -0,0 +1,83 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.contrib.operators.sagemaker_model_operator \ + import SageMakerModelOperator +from airflow.exceptions import AirflowException + +role = 'arn:aws:iam:role/test-role' + +bucket = 'test-bucket' + +model_name = 'test-model-name' + +image = 'test-image' + +output_url = 's3://{}/test/output'.format(bucket) +create_model_params = { + 'ModelName': model_name, + 'PrimaryContainer': { + 'Image': image, + 'ModelDataUrl': output_url, + }, + 'ExecutionRoleArn': role +} + + +class TestSageMakerModelOperator(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.sagemaker = SageMakerModelOperator( + task_id='test_sagemaker_operator', + aws_conn_id='sagemaker_test_id', + config=create_model_params + ) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + def test_execute(self, mock_model, mock_client): + mock_model.return_value = {'ModelArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 200}} + self.sagemaker.execute(None) + mock_model.assert_called_once_with(create_model_params) + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'create_model') + def test_execute_with_failure(self, mock_model, mock_client): + mock_model.return_value = {'ModelArn': 'testarn', + 'ResponseMetadata': + {'HTTPStatusCode': 404}} + self.assertRaises(AirflowException, self.sagemaker.execute, None) + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py b/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py new file mode 100644 index 0000000000000..2e438e4e193ae --- /dev/null +++ b/tests/contrib/sensors/test_sagemaker_endpoint_sensor.py @@ -0,0 +1,110 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +from airflow import configuration +from airflow.contrib.sensors.sagemaker_endpoint_sensor \ + import SageMakerEndpointSensor +from airflow.contrib.hooks.sagemaker_hook import SageMakerHook +from airflow.exceptions import AirflowException + +DESCRIBE_ENDPOINT_CREATING_RESPONSE = { + 'EndpointStatus': 'Creating', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} +DESCRIBE_ENDPOINT_INSERVICE_RESPONSE = { + 'EndpointStatus': 'InService', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + +DESCRIBE_ENDPOINT_FAILED_RESPONSE = { + 'EndpointStatus': 'Failed', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + }, + 'FailureReason': 'Unknown' +} + +DESCRIBE_ENDPOINT_UPDATING_RESPONSE = { + 'EndpointStatus': 'Updating', + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + } +} + + +class TestSageMakerEndpointSensor(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, 'describe_endpoint') + def test_sensor_with_failure(self, mock_describe, mock_client): + mock_describe.side_effect = [DESCRIBE_ENDPOINT_FAILED_RESPONSE] + sensor = SageMakerEndpointSensor( + task_id='test_task', + poke_interval=1, + aws_conn_id='aws_test', + endpoint_name='test_job_name' + ) + self.assertRaises(AirflowException, sensor.execute, None) + mock_describe.assert_called_once_with('test_job_name') + + @mock.patch.object(SageMakerHook, 'get_conn') + @mock.patch.object(SageMakerHook, '__init__') + @mock.patch.object(SageMakerHook, 'describe_endpoint') + def test_sensor(self, mock_describe, hook_init, mock_client): + hook_init.return_value = None + + mock_describe.side_effect = [ + DESCRIBE_ENDPOINT_CREATING_RESPONSE, + DESCRIBE_ENDPOINT_UPDATING_RESPONSE, + DESCRIBE_ENDPOINT_INSERVICE_RESPONSE + ] + sensor = SageMakerEndpointSensor( + task_id='test_task', + poke_interval=1, + aws_conn_id='aws_test', + endpoint_name='test_job_name' + ) + + sensor.execute(None) + + # make sure we called 3 times(terminated when its completed) + self.assertEqual(mock_describe.call_count, 3) + + # make sure the hook was initialized with the specific params + hook_init.assert_called_with(aws_conn_id='aws_test') + + +if __name__ == '__main__': + unittest.main() diff --git a/tests/contrib/sensors/test_sagemaker_training_sensor.py b/tests/contrib/sensors/test_sagemaker_training_sensor.py index 5861d7a6fdb15..6642449c6a795 100644 --- a/tests/contrib/sensors/test_sagemaker_training_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_training_sensor.py @@ -101,7 +101,7 @@ def test_sensor(self, mock_describe_job, hook_init, mock_client): sensor.execute(None) - # make sure we called 4 times(terminated when its compeleted) + # make sure we called 3 times(terminated when its compeleted) self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params diff --git a/tests/contrib/sensors/test_sagemaker_transform_sensor.py b/tests/contrib/sensors/test_sagemaker_transform_sensor.py index 1394920d5dc3d..810680683cde3 100644 --- a/tests/contrib/sensors/test_sagemaker_transform_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_transform_sensor.py @@ -97,7 +97,7 @@ def test_sensor(self, mock_describe_job, hook_init, mock_client): sensor.execute(None) - # make sure we called 4 times(terminated when its compeleted) + # make sure we called 3 times(terminated when its compeleted) self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params diff --git a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py index 8c0ba11380c1a..beb62dc47b1d1 100644 --- a/tests/contrib/sensors/test_sagemaker_tuning_sensor.py +++ b/tests/contrib/sensors/test_sagemaker_tuning_sensor.py @@ -100,7 +100,7 @@ def test_sensor(self, mock_describe_job, hook_init, mock_client): sensor.execute(None) - # make sure we called 4 times(terminated when its compeleted) + # make sure we called 3 times(terminated when its compeleted) self.assertEqual(mock_describe_job.call_count, 3) # make sure the hook was initialized with the specific params From c39c961d8d0e9443eac0d3e65854c3e3bda2e750 Mon Sep 17 00:00:00 2001 From: Felix Date: Mon, 5 Nov 2018 19:41:14 +0100 Subject: [PATCH 183/808] [AIRFLOW-2780] Add IMAP Hook to retrieve email attachments (#4119) [AIRFLOW-2780] Add IMAP Hook to retrieve email attachments - Add has_mail_attachments to check if there are mail attachments in the given mailbox with the given attachment name - Add retrieve_mail_attachments to download the attachments to a local directory - Add some test cases but more are coming - Add license header - Change retrieve_mail_attachments to download_mail_attachments - Add retrieve_mail_attachments that return a list of tuple containing the attachments found - Change IMAP4_SSL close() method to be called after retrieving the attachments and not before logging out - Change test_connect to not check for close method because no mail folder will be opened when only connecting - Add some test cases that are still in WIP - Fixes a bug causing multiple attachments in a single mail not being correctly added to the all mails attachments - Fixes a bug where MailPart is_attachment always returns None - Add logging when an attachment has been found that matches the name - Add more test cases with sample mail --- airflow/contrib/hooks/imap_hook.py | 280 ++++++++++++++++++++++++++ docs/code.rst | 1 + tests/contrib/hooks/test_imap_hook.py | 266 ++++++++++++++++++++++++ 3 files changed, 547 insertions(+) create mode 100644 airflow/contrib/hooks/imap_hook.py create mode 100644 tests/contrib/hooks/test_imap_hook.py diff --git a/airflow/contrib/hooks/imap_hook.py b/airflow/contrib/hooks/imap_hook.py new file mode 100644 index 0000000000000..c0b3126a8f382 --- /dev/null +++ b/airflow/contrib/hooks/imap_hook.py @@ -0,0 +1,280 @@ +# -*- coding: utf-8 -*- +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import email +import imaplib +import os +import re + +from airflow import LoggingMixin +from airflow.hooks.base_hook import BaseHook + + +class ImapHook(BaseHook): + """ + This hook connects to a mail server by using the imap protocol. + + :param imap_conn_id: The connection id that contains the information + used to authenticate the client. + The default value is 'imap_default'. + :type imap_conn_id: str + """ + + def __init__(self, imap_conn_id='imap_default'): + super(ImapHook, self).__init__(imap_conn_id) + self.conn = self.get_connection(imap_conn_id) + self.mail_client = imaplib.IMAP4_SSL(self.conn.host) + + def __enter__(self): + self.mail_client.login(self.conn.login, self.conn.password) + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + self.mail_client.logout() + + def has_mail_attachment(self, name, mail_folder='INBOX', check_regex=False): + """ + Checks the mail folder for mails containing attachments with the given name. + + :param name: The name of the attachment that will be searched for. + :type name: str + :param mail_folder: The mail folder where to look at. + The default value is 'INBOX'. + :type mail_folder: str + :param check_regex: Checks the name for a regular expression. + The default value is False. + :type check_regex: bool + :returns: True if there is an attachment with the given name and False if not. + :rtype: bool + """ + mail_attachments = self._retrieve_mails_attachments_by_name(name, mail_folder, + check_regex, + latest_only=True) + return len(mail_attachments) > 0 + + def retrieve_mail_attachments(self, name, mail_folder='INBOX', check_regex=False, + latest_only=False): + """ + Retrieves mail's attachments in the mail folder by its name. + + :param name: The name of the attachment that will be downloaded. + :type name: str + :param mail_folder: The mail folder where to look at. + The default value is 'INBOX'. + :type mail_folder: str + :param check_regex: Checks the name for a regular expression. + The default value is False. + :type check_regex: bool + :param latest_only: If set to True it will only retrieve + the first matched attachment. + The default value is False. + :type latest_only: bool + :returns: a list of tuple each containing the attachment filename and its payload. + :rtype: a list of tuple + """ + mail_attachments = self._retrieve_mails_attachments_by_name(name, mail_folder, + check_regex, + latest_only) + return mail_attachments + + def download_mail_attachments(self, name, local_output_directory, mail_folder='INBOX', + check_regex=False, latest_only=False): + """ + Downloads mail's attachments in the mail folder by its name + to the local directory. + + :param name: The name of the attachment that will be downloaded. + :type name: str + :param local_output_directory: The output directory on the local machine + where the files will be downloaded to. + :type local_output_directory: str + :param mail_folder: The mail folder where to look at. + The default value is 'INBOX'. + :type mail_folder: str + :param check_regex: Checks the name for a regular expression. + The default value is False. + :type check_regex: bool + :param latest_only: If set to True it will only download + the first matched attachment. + The default value is False. + :type latest_only: bool + """ + mail_attachments = self._retrieve_mails_attachments_by_name(name, mail_folder, + check_regex, latest_only) + self._create_files(mail_attachments, local_output_directory) + + def _retrieve_mails_attachments_by_name(self, name, mail_folder, check_regex, + latest_only): + all_matching_attachments = [] + + self.mail_client.select(mail_folder) + + for mail_id in self._list_mail_ids_desc(): + response_mail_body = self._fetch_mail_body(mail_id) + matching_attachments = self._check_mail_body(response_mail_body, name, check_regex, latest_only) + + if matching_attachments: + all_matching_attachments.extend(matching_attachments) + if latest_only: + break + + self.mail_client.close() + + return all_matching_attachments + + def _list_mail_ids_desc(self): + result, data = self.mail_client.search(None, 'All') + mail_ids = data[0].split() + return reversed(mail_ids) + + def _fetch_mail_body(self, mail_id): + result, data = self.mail_client.fetch(mail_id, '(RFC822)') + mail_body = data[0][1] # The mail body is always in this specific location + mail_body_str = mail_body.decode('utf-8') + return mail_body_str + + def _check_mail_body(self, response_mail_body, name, check_regex, latest_only): + mail = Mail(response_mail_body) + if mail.has_attachments(): + return mail.get_attachments_by_name(name, check_regex, find_first=latest_only) + + def _create_files(self, mail_attachments, local_output_directory): + for name, payload in mail_attachments: + if self._is_symlink(name): + self.log.error('Can not create file because it is a symlink!') + elif self._is_escaping_current_directory(name): + self.log.error('Can not create file because it is escaping the current directory!') + else: + self._create_file(name, payload, local_output_directory) + + def _is_symlink(self, name): + return os.path.islink(name) + + def _is_escaping_current_directory(self, name): + return '../' in name + + def _correct_path(self, name, local_output_directory): + return local_output_directory + name if local_output_directory.endswith('/') \ + else local_output_directory + '/' + name + + def _create_file(self, name, payload, local_output_directory): + file_path = self._correct_path(name, local_output_directory) + + with open(file_path, 'wb') as file: + file.write(payload) + + +class Mail(LoggingMixin): + """ + This class simplifies working with mails returned by the imaplib client. + + :param mail_body: The mail body of a mail received from imaplib client. + :type mail_body: str + """ + + def __init__(self, mail_body): + super(Mail, self).__init__() + self.mail = email.message_from_string(mail_body) + + def has_attachments(self): + """ + Checks the mail for a attachments. + + :returns: True if it has attachments and False if not. + :rtype: bool + """ + return self.mail.get_content_maintype() == 'multipart' + + def get_attachments_by_name(self, name, check_regex, find_first=False): + """ + Gets all attachments by name for the mail. + + :param name: The name of the attachment to look for. + :type name: str + :param check_regex: Checks the name for a regular expression. + :type check_regex: bool + :param find_first: If set to True it will only find the first match and then quit. + The default value is False. + :type find_first: bool + :returns: a list of tuples each containing name and payload + where the attachments name matches the given name. + :rtype: list of tuple + """ + attachments = [] + + for part in self.mail.walk(): + mail_part = MailPart(part) + if mail_part.is_attachment(): + found_attachment = mail_part.has_matching_name(name) if check_regex \ + else mail_part.has_equal_name(name) + if found_attachment: + file_name, file_payload = mail_part.get_file() + self.log.info('Found attachment: {}'.format(file_name)) + attachments.append((file_name, file_payload)) + if find_first: + break + + return attachments + + +class MailPart: + """ + This class is a wrapper for a Mail object's part and gives it more features. + + :param part: The mail part in a Mail object. + :type part: any + """ + + def __init__(self, part): + self.part = part + + def is_attachment(self): + """ + Checks if the part is a valid mail attachment. + + :returns: True if it is an attachment and False if not. + :rtype: bool + """ + return self.part.get_content_maintype() != 'multipart' and self.part.get('Content-Disposition') + + def has_matching_name(self, name): + """ + Checks if the given name matches the part's name. + + :param name: The name to look for. + :type name: str + :returns: True if it matches the name (including regular expression). + :rtype: tuple + """ + return re.match(name, self.part.get_filename()) + + def has_equal_name(self, name): + """ + Checks if the given name is equal to the part's name. + + :param name: The name to look for. + :type name: str + :returns: True if it is equal to the given name. + :rtype: bool + """ + return self.part.get_filename() == name + + def get_file(self): + """ + Gets the file including name and payload. + + :returns: the part's name and payload. + :rtype: tuple + """ + return self.part.get_filename(), self.part.get_payload(decode=True) diff --git a/docs/code.rst b/docs/code.rst index ca6ca593ca345..12db9b71b136d 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -401,6 +401,7 @@ Community contributed hooks .. autoclass:: airflow.contrib.hooks.gcp_mlengine_hook.MLEngineHook .. autoclass:: airflow.contrib.hooks.gcp_pubsub_hook.PubSubHook .. autoclass:: airflow.contrib.hooks.gcs_hook.GoogleCloudStorageHook +.. autoclass:: airflow.contrib.hooks.imap_hook.ImapHook .. autoclass:: airflow.contrib.hooks.jenkins_hook.JenkinsHook .. autoclass:: airflow.contrib.hooks.jira_hook.JiraHook .. autoclass:: airflow.contrib.hooks.mongo_hook.MongoHook diff --git a/tests/contrib/hooks/test_imap_hook.py b/tests/contrib/hooks/test_imap_hook.py new file mode 100644 index 0000000000000..63dc13a874e63 --- /dev/null +++ b/tests/contrib/hooks/test_imap_hook.py @@ -0,0 +1,266 @@ +# -*- coding: utf-8 -*- +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import imaplib +import unittest + +from mock import Mock, patch, mock_open + +from airflow import configuration, models +from airflow.contrib.hooks.imap_hook import ImapHook +from airflow.utils import db + +imaplib_string = 'airflow.contrib.hooks.imap_hook.imaplib' +open_string = 'airflow.contrib.hooks.imap_hook.open' + + +def _create_fake_imap(mock_imaplib, with_mail=False, attachment_name='test1.csv'): + mock_conn = Mock(spec=imaplib.IMAP4_SSL) + mock_imaplib.IMAP4_SSL.return_value = mock_conn + + mock_conn.login.return_value = ('OK', []) + + if with_mail: + mock_conn.select.return_value = ('OK', []) + mock_conn.search.return_value = ('OK', [b'1']) + mail_string = \ + 'Content-Type: multipart/mixed; boundary=123\r\n--123\r\n' \ + 'Content-Disposition: attachment; filename="{}";' \ + 'Content-Transfer-Encoding: base64\r\nSWQsTmFtZQoxLEZlbGl4\r\n--123--'.format(attachment_name) + mock_conn.fetch.return_value = ('OK', [(b'', mail_string.encode('utf-8'))]) + mock_conn.close.return_value = ('OK', []) + + mock_conn.logout.return_value = ('OK', []) + + return mock_conn + + +class TestImapHook(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + db.merge_conn( + models.Connection( + conn_id='imap_default', + host='imap_server_address', + login='imap_user', + password='imap_password' + ) + ) + + @patch(imaplib_string) + def test_connect_and_disconnect(self, mock_imaplib): + mock_conn = _create_fake_imap(mock_imaplib) + + with ImapHook(): + pass + + mock_imaplib.IMAP4_SSL.assert_called_once_with('imap_server_address') + mock_conn.login.assert_called_once_with('imap_user', 'imap_password') + mock_conn.logout.assert_called_once() + + @patch(imaplib_string) + def test_has_mail_attachments_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + has_attachment_in_inbox = imap_hook.has_mail_attachment('test1.csv') + + self.assertTrue(has_attachment_in_inbox) + + @patch(imaplib_string) + def test_has_mail_attachments_not_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + has_attachment_in_inbox = imap_hook.has_mail_attachment('test1.txt') + + self.assertFalse(has_attachment_in_inbox) + + @patch(imaplib_string) + def test_has_mail_attachments_with_regex_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + has_attachment_in_inbox = imap_hook.has_mail_attachment( + name='test(\d+).csv', + check_regex=True + ) + + self.assertTrue(has_attachment_in_inbox) + + @patch(imaplib_string) + def test_has_mail_attachments_with_regex_not_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + has_attachment_in_inbox = imap_hook.has_mail_attachment( + name='test_(\d+).csv', + check_regex=True + ) + + self.assertFalse(has_attachment_in_inbox) + + @patch(imaplib_string) + def test_retrieve_mail_attachments_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + attachments_in_inbox = imap_hook.retrieve_mail_attachments('test1.csv') + + self.assertEquals(attachments_in_inbox, [('test1.csv', b'SWQsTmFtZQoxLEZlbGl4')]) + + @patch(imaplib_string) + def test_retrieve_mail_attachments_not_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + attachments_in_inbox = imap_hook.retrieve_mail_attachments('test1.txt') + + self.assertEquals(attachments_in_inbox, []) + + @patch(imaplib_string) + def test_retrieve_mail_attachments_with_regex_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + attachments_in_inbox = imap_hook.retrieve_mail_attachments( + name='test(\d+).csv', + check_regex=True + ) + + self.assertEquals(attachments_in_inbox, [('test1.csv', b'SWQsTmFtZQoxLEZlbGl4')]) + + @patch(imaplib_string) + def test_retrieve_mail_attachments_with_regex_not_found(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + attachments_in_inbox = imap_hook.retrieve_mail_attachments( + name='test_(\d+).csv', + check_regex=True + ) + + self.assertEquals(attachments_in_inbox, []) + + @patch(imaplib_string) + def test_retrieve_mail_attachments_latest_only(self, mock_imaplib): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + attachments_in_inbox = imap_hook.retrieve_mail_attachments( + name='test1.csv', + latest_only=True + ) + + self.assertEquals(attachments_in_inbox, [('test1.csv', b'SWQsTmFtZQoxLEZlbGl4')]) + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_found(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments('test1.csv', 'test_directory') + + mock_open_method.assert_called_once_with('test_directory/test1.csv', 'wb') + mock_open_method().write.assert_called_once_with(b'SWQsTmFtZQoxLEZlbGl4') + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_not_found(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments('test1.txt', 'test_directory') + + mock_open_method.assert_not_called() + mock_open_method().write.assert_not_called() + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_with_regex_found(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments( + name='test(\d+).csv', + local_output_directory='test_directory', + check_regex=True + ) + + mock_open_method.assert_called_once_with('test_directory/test1.csv', 'wb') + mock_open_method().write.assert_called_once_with(b'SWQsTmFtZQoxLEZlbGl4') + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_with_regex_not_found(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments( + name='test_(\d+).csv', + local_output_directory='test_directory', + check_regex=True + ) + + mock_open_method.assert_not_called() + mock_open_method().write.assert_not_called() + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_with_latest_only(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True) + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments( + name='test1.csv', + local_output_directory='test_directory', + latest_only=True + ) + + mock_open_method.assert_called_once_with('test_directory/test1.csv', 'wb') + mock_open_method().write.assert_called_once_with(b'SWQsTmFtZQoxLEZlbGl4') + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_with_escaping_chars(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True, attachment_name='../test1.csv') + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments( + name='test1.csv', + local_output_directory='test_directory' + ) + + mock_open_method.assert_not_called() + mock_open_method().write.assert_not_called() + + @patch(open_string, new_callable=mock_open) + @patch(imaplib_string) + def test_download_mail_attachments_with_symlink(self, mock_imaplib, mock_open_method): + _create_fake_imap(mock_imaplib, with_mail=True, attachment_name='initrd.img') + + with ImapHook() as imap_hook: + imap_hook.download_mail_attachments( + name='test1.csv', + local_output_directory='test_directory' + ) + + mock_open_method.assert_not_called() + mock_open_method().write.assert_not_called() + + +if __name__ == '__main__': + unittest.main() From 66d0d05ea0802aec407e0ef5435a962080db0926 Mon Sep 17 00:00:00 2001 From: Bolke de Bruin Date: Fri, 5 Oct 2018 20:03:22 +0200 Subject: [PATCH 184/808] [AIRFLOW-3164] Verify server certificate when connecting to LDAP (#4006) Misconfiguration and improper checking of exceptions disabled server certificate checking. We now only support TLS connections and do not support insecure connections anymore. --- UPDATING.md | 11 +++++++++++ airflow/contrib/auth/backends/ldap_auth.py | 14 ++++++++------ docs/security.rst | 3 +-- setup.py | 2 +- 4 files changed, 21 insertions(+), 9 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 6fde229a2038d..0f108ffadc1ba 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -83,6 +83,17 @@ configuration, so creating EMR clusters might fail until your connection is upda Ec2SubnetId, TerminationProtection and KeepJobFlowAliveWhenNoSteps were all top-level keys when they should be inside the "Instances" dict) +### LDAP Auth Backend now requires TLS + +Connecting to an LDAP serever over plain text is not supported anymore. The +certificate presented by the LDAP server must be signed by a trusted +certificiate, or you must provide the `cacert` option under `[ldap]` in the +config file. + +If you want to use LDAP auth backend without TLS then you will habe to create a +custom-auth backend based on +https://github.com/apache/incubator-airflow/blob/1.10.0/airflow/contrib/auth/backends/ldap_auth.py + ## Airflow 1.10 Installation and upgrading requires setting `SLUGIFY_USES_TEXT_UNIDECODE=yes` in your environment or diff --git a/airflow/contrib/auth/backends/ldap_auth.py b/airflow/contrib/auth/backends/ldap_auth.py index cb4675757b75a..dc17dab5124cd 100644 --- a/airflow/contrib/auth/backends/ldap_auth.py +++ b/airflow/contrib/auth/backends/ldap_auth.py @@ -56,16 +56,18 @@ class LdapException(Exception): def get_ldap_connection(dn=None, password=None): - tls_configuration = None - use_ssl = False try: cacert = configuration.conf.get("ldap", "cacert") - tls_configuration = Tls(validate=ssl.CERT_REQUIRED, ca_certs_file=cacert) - use_ssl = True - except Exception: + except AirflowConfigException: pass - server = Server(configuration.conf.get("ldap", "uri"), use_ssl, tls_configuration) + tls_configuration = Tls(validate=ssl.CERT_REQUIRED, + ca_certs_file=cacert) + + server = Server(configuration.conf.get("ldap", "uri"), + use_ssl=True, + tls=tls_configuration) + conn = Connection(server, native(dn), native(password)) if not conn.bind(): diff --git a/docs/security.rst b/docs/security.rst index b47a58f7418f0..1ecc958d52ab4 100644 --- a/docs/security.rst +++ b/docs/security.rst @@ -61,8 +61,7 @@ LDAP '''' To turn on LDAP authentication configure your ``airflow.cfg`` as follows. Please note that the example uses -an encrypted connection to the ldap server as you probably do not want passwords be readable on the network level. -It is however possible to configure without encryption if you really want to. +an encrypted connection to the ldap server as we do not want passwords be readable on the network level. Additionally, if you are using Active Directory, and are not explicitly specifying an OU that your users are in, you will need to change ``search_scope`` to "SUBTREE". diff --git a/setup.py b/setup.py index c187b9b7ff51a..49de5e6205423 100644 --- a/setup.py +++ b/setup.py @@ -188,7 +188,7 @@ def write_version(filename=os.path.join(*['airflow', 'snakebite[kerberos]>=2.7.8'] kubernetes = ['kubernetes>=3.0.0', 'cryptography>=2.0.0'] -ldap = ['ldap3>=0.9.9.1'] +ldap = ['ldap3>=2.5.1'] mssql = ['pymssql>=2.1.1'] mysql = ['mysqlclient>=1.3.6'] oracle = ['cx_Oracle>=5.1.2'] From c75b6eb9178ae54054789d448677147a5a1c422c Mon Sep 17 00:00:00 2001 From: Dana Assa Date: Wed, 7 Nov 2018 17:08:42 +0200 Subject: [PATCH 185/808] [AIRFLOW-3297] added more failure states to the EMR step sensor (#4152) Previously, the states 'PENDING_CANCELLED' and 'INTERRUPTED' resulted in the Airflow task being marked as successful. This PR fixed this by mentioning those states as NON_TERMINAL_STATES and as FAILED_STATE, accordingly. --- airflow/contrib/sensors/emr_step_sensor.py | 4 +- tests/contrib/sensors/test_emr_step_sensor.py | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/sensors/emr_step_sensor.py b/airflow/contrib/sensors/emr_step_sensor.py index 3dddf01bd83a5..44aaea410dd1e 100644 --- a/airflow/contrib/sensors/emr_step_sensor.py +++ b/airflow/contrib/sensors/emr_step_sensor.py @@ -32,8 +32,8 @@ class EmrStepSensor(EmrBaseSensor): :type step_id: string """ - NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE'] - FAILED_STATE = ['CANCELLED', 'FAILED'] + NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE', 'CANCEL_PENDING'] + FAILED_STATE = ['CANCELLED', 'FAILED', 'INTERRUPTED'] template_fields = ['job_flow_id', 'step_id'] template_ext = () diff --git a/tests/contrib/sensors/test_emr_step_sensor.py b/tests/contrib/sensors/test_emr_step_sensor.py index 7f72c73befd03..ad0b2469e861b 100644 --- a/tests/contrib/sensors/test_emr_step_sensor.py +++ b/tests/contrib/sensors/test_emr_step_sensor.py @@ -83,6 +83,35 @@ } } +DESCRIBE_JOB_STEP_INTERRUPTED_RETURN = { + 'ResponseMetadata': { + 'HTTPStatusCode': 200, + 'RequestId': '8dee8db2-3719-11e6-9e20-35b2f861a2a6' + }, + 'Step': { + 'ActionOnFailure': 'CONTINUE', + 'Config': { + 'Args': [ + '/usr/lib/spark/bin/run-example', + 'SparkPi', + '10' + ], + 'Jar': 'command-runner.jar', + 'Properties': {} + }, + 'Id': 's-VK57YR1Z9Z5N', + 'Name': 'calculate_pi', + 'Status': { + 'State': 'INTERRUPTED', + 'StateChangeReason': {}, + 'Timeline': { + 'CreationDateTime': datetime(2016, 6, 20, 19, 0, 18, tzinfo=tzlocal()), + 'StartDateTime': datetime(2016, 6, 20, 19, 2, 34, tzinfo=tzlocal()) + } + } + } +} + DESCRIBE_JOB_STEP_COMPLETED_RETURN = { 'ResponseMetadata': { 'HTTPStatusCode': 200, @@ -158,6 +187,17 @@ def test_step_cancelled(self): with patch('boto3.session.Session', self.boto3_session_mock): self.assertRaises(AirflowException, self.sensor.execute, None) + def test_step_interrupted(self): + self.emr_client_mock.describe_step.side_effect = [ + DESCRIBE_JOB_STEP_RUNNING_RETURN, + DESCRIBE_JOB_STEP_INTERRUPTED_RETURN + ] + + self.boto3_client_mock = MagicMock(return_value=self.emr_client_mock) + + with patch('boto3.session.Session', self.boto3_session_mock): + self.assertRaises(AirflowException, self.sensor.execute, None) + if __name__ == '__main__': unittest.main() From fa9088699696d89a7d00f3cdadc9b1360978a86c Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Thu, 8 Nov 2018 23:07:03 +0000 Subject: [PATCH 186/808] Version 1.10.1b1 --- CHANGELOG.txt | 129 +++++++++++++++++++++++++++++++++++++++++++++ airflow/version.py | 2 +- 2 files changed, 130 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index b4ee1755b467a..f10de13cf35d3 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -1,3 +1,132 @@ +AIRFLOW 1.10.1b1, 2018-11-09 + +New features: + +[AIRFLOW-2524] Airflow integration with AWS Sagemaker +[AIRFLOW-2657] Add ability to delete DAG from web ui +[AIRFLOW-2780] Adds IMAP Hook to interact with a mail server +[AIRFLOW-2794] Add delete support for Azure blob +[AIRFLOW-2912] Add operators for Google Cloud Functions +[AIRFLOW-2974] Add Start/Restart/Terminate methods Databricks Hook +[AIRFLOW-2989] No Parameter to change bootDiskType for DataprocClusterCreateOperator +[AIRFLOW-3078] Basic operators for Google Compute Engine +[AIRFLOW-3147] Update Flask-AppBuilder version +[AIRFLOW-3231] Basic operators for Google Cloud SQL (deploy / patch / delete) +[AIRFLOW-3276] Google Cloud SQL database create / patch / delete operators + +Improvements: + +[AIRFLOW-393] Add progress callbacks for FTP downloads +[AIRFLOW-520] Show Airflow version on web page +[AIRFLOW-843] Excpetions now available in context durint on_failure_callback +[AIRFLOW-2476] Update tabulate dependency to v0.8.2 +[AIRFLOW-2592] Bump Bleach dependency +[AIRFLOW-2622] Add "confirm=False" option to SFTPOperator +[AIRFLOW-2662] support affinity & nodeSelector policies for kubernetes executor/operator +[AIRFLOW-2709] Improve error handling in Databricks hook +[AIRFLOW-2763] No precheck mechanism in place during worker initialisation for the connection to metadata database +[AIRFLOW-2789] Add ability to create single node cluster to DataprocClusterCreateOperator +[AIRFLOW-2797] Add ability to create Google Dataproc cluster with custom image +[AIRFLOW-2854] kubernetes_pod_operator add more configuration items +[AIRFLOW-2855] Need to Check Validity of Cron Expression When Process DAG File/Zip File +[AIRFLOW-2904] Clean an unnecessary line in airflow/executors/celery_executor.py +[AIRFLOW-2921] A trivial incorrectness in CeleryExecutor() +[AIRFLOW-2922] Potential deal-lock bug in CeleryExecutor() +[AIRFLOW-2932] GoogleCloudStorageHook - allow compression of file +[AIRFLOW-2949] Syntax Highlight for Single Quote +[AIRFLOW-2951] dag_run end_date Null after a dag is finished +[AIRFLOW-2956] Kubernetes tolerations for pod operator +[AIRFLOW-2997] Support for clustered tables in Bigquery hooks/operators +[AIRFLOW-3006] Fix rrror when schedule_interval="None" +[AIRFLOW-3008] Move Kubernetes related example DAGs to contribe/example_dags +[AIRFLOW-3025] Allow to specify dns and dns-search parameters for DockerOperator +[AIRFLOW-3067] (www_rbac) Flask flash messages are not displayed properly (no background color) +[AIRFLOW-3069] Decode output of S3 file transform operator +[AIRFLOW-3090] INFO logs are too verbose +[AIRFLOW-3103] Update Flask-Login +[AIRFLOW-3112] Align SFTP hook with SSH hook +[AIRFLOW-3119] Enable loglevel on celery worker and inherit from airflow.cfg +[AIRFLOW-3137] Make ProxyFix middleware optional +[AIRFLOW-3173] Add _cmd options for more password config options +[AIRFLOW-3177] Change scheduler_heartbeat metric from gauge to counter +[AIRFLOW-3195] Druid Hook: Log ingestion spec and task id +[AIRFLOW-3197] EMR Hook is missing some parameters to valid on the AWS API +[AIRFLOW-3232] Make documentation for GCF Functions operator more readable +[AIRFLOW-3262] Can't get log containing Response when using SimpleHttpOperator +[AIRFLOW-3265] Add support for "unix_socket" in connection extra for Mysql Hook + +Doc-only changes: + +[AIRFLOW-1441] Tutorial Inconsistencies Between Example Pipeline Definition and Recap +[AIRFLOW-2682] Add how-to guide(s) for how to use basic operators like BashOperator and PythonOperator +[AIRFLOW-3104] .airflowignore feature is not mentioned at all in documentation +[AIRFLOW-3237] Refactor example DAGs +[AIRFLOW-3187] Update airflow.gif file with a slower version +[AIRFLOW-3159] Update Airflow documentation on GCP Logging +[AIRFLOW-3030] Command Line docs incorrect subdir +[AIRFLOW-2990] Docstrings for Hooks/Operators are in incorrect format +[AIRFLOW-3127] Celery SSL Documentation is out-dated + +Bug fixes: + +[AIRFLOW-839] docker_operator.py attempts to log status key without first checking existence +[AIRFLOW-1104] Concurrency check in scheduler should count queued tasks as well as running +[AIRFLOW-1163] Add support for x-forwarded-* headers to support access behind AWS ELB +[AIRFLOW-1195] Cleared tasks in SubDagOperator do not trigger Parent dag_runs +[AIRFLOW-1508] Skipped state not part of State.task_states +[AIRFLOW-1762] Use key_file in SSHHook.create_tunnel() +[AIRFLOW-1837] Differing start_dates on tasks not respected by scheduler. +[AIRFLOW-1874] Support standard SQL in Check, ValueCheck and IntervalCheck BigQuery operators +[AIRFLOW-1917] print() from python operators end up with extra new line +[AIRFLOW-1970] Database cannot be initialized if an invalid fernet key is provided +[AIRFLOW-2145] Deadlock after clearing a running task +[AIRFLOW-2216] Cannot specify a profile for AWS Hook to load with s3 config file +[AIRFLOW-2574] initdb fails when mysql password contains percent sign +[AIRFLOW-2707] Error accessing log files from web UI +[AIRFLOW-2716] Replace new Python 3.7 keywords +[AIRFLOW-2744] RBAC app doesn't integrate plugins (blueprints etc) +[AIRFLOW-2772] BigQuery hook does not allow specifying both the partition field name and table name at the same time +[AIRFLOW-2778] Bad Import in collect_dag in DagBag +[AIRFLOW-2786] Variables view fails to render if a variable has an empty key +[AIRFLOW-2799] Filtering UI objects by datetime is broken +[AIRFLOW-2800] Remove airflow/ low-hanging linting errors +[AIRFLOW-2825] S3ToHiveTransfer operator may not may able to handle GZIP file with uppercase ext in S3 +[AIRFLOW-2848] dag_id is missing in metadata table "job" for LocalTaskJob +[AIRFLOW-2860] DruidHook: time variable is not updated correctly when checking for timeout +[AIRFLOW-2865] Race condition between on_success_callback and LocalTaskJob's cleanup +[AIRFLOW-2893] Stuck dataflow job due to jobName mismatch. +[AIRFLOW-2895] Prevent scheduler from spamming heartbeats/logs +[AIRFLOW-2900] Code not visible for Packaged DAGs +[AIRFLOW-2905] Switch to regional dataflow job service. +[AIRFLOW-2907] Sendgrid - Attachments - ERROR - Object of type 'bytes' is not JSON serializable +[AIRFLOW-2938] Invalid 'extra' field in connection can raise an AttributeError when attempting to edit +[AIRFLOW-2979] Deprecated Celery Option not in Options list +[AIRFLOW-2981] TypeError in dataflow operators when using GCS jar or py_file +[AIRFLOW-2984] Cannot convert naive_datetime when task has a naive start_date/end_date +[AIRFLOW-2994] flatten_results in BigQueryOperator/BigQueryHook should default to None +[AIRFLOW-3002] ValueError in dataflow operators when using GCS jar or py_file +[AIRFLOW-3012] Email on sla miss is send only to first address on the list +[AIRFLOW-3046] ECS Operator mistakenly reports success when task is killed due to EC2 host termination +[AIRFLOW-3064] No output from `airflow test` due to default logging config +[AIRFLOW-3072] Only admin can view logs in RBAC UI +[AIRFLOW-3079] Improve initdb to support MSSQL Server +[AIRFLOW-3089] Google auth doesn't work under http +[AIRFLOW-3099] Errors raised when some blocs are missing in airflow.cfg +[AIRFLOW-3109] Default user permission should contain 'can_clear' +[AIRFLOW-3111] Confusing comments and instructions for log templates in UPDATING.md and default_airflow.cfg +[AIRFLOW-3124] Broken webserver debug mode (RBAC) +[AIRFLOW-3136] Scheduler Failing the Task retries run while processing Executor Events +[AIRFLOW-3138] Migration cc1e65623dc7 creates issues with postgres +[AIRFLOW-3161] Log Url link does not link to task instance logs in RBAC UI +[AIRFLOW-3162] HttpHook fails to parse URL when port is specified +[AIRFLOW-3183] Potential Bug in utils/dag_processing/DagFileProcessorManager.max_runs_reached() +[AIRFLOW-3203] Bugs in DockerOperator & Some operator test scripts were named incorrectly +[AIRFLOW-3238] Dags, removed from the filesystem, are not deactivated on initdb +[AIRFLOW-3268] Cannot pass SSL dictionary to mysql connection via URL +[AIRFLOW-3277] Invalid timezone transition handling for cron schedules +[AIRFLOW-3295] Require encryption in DaskExecutor when certificates are configured. +[AIRFLOW-3297] EmrStepSensor marks cancelled step as successful + AIRFLOW 1.10.0, 2018-08-03 -------------------------- [AIRFLOW-2870] Use abstract TaskInstance for migration diff --git a/airflow/version.py b/airflow/version.py index be4038b4af24d..1080bf44f6953 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.0' +version = '1.10.1b1' From 0586c2e83aa6d7f41e0e0650eaf5017fbccf097b Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Fri, 9 Nov 2018 22:36:51 +0800 Subject: [PATCH 187/808] [AIRFLOW-XXX] Fix a few typos in CHANGELOG (#4169) For 843 and 3006, the original JIRA subjects are actually ok. For 3008, understand you copied the original JIRA subject. But may be good to fix it here. --- CHANGELOG.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index f10de13cf35d3..db77cc2903110 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -18,7 +18,7 @@ Improvements: [AIRFLOW-393] Add progress callbacks for FTP downloads [AIRFLOW-520] Show Airflow version on web page -[AIRFLOW-843] Excpetions now available in context durint on_failure_callback +[AIRFLOW-843] Exceptions now available in context during on_failure_callback [AIRFLOW-2476] Update tabulate dependency to v0.8.2 [AIRFLOW-2592] Bump Bleach dependency [AIRFLOW-2622] Add "confirm=False" option to SFTPOperator @@ -37,8 +37,8 @@ Improvements: [AIRFLOW-2951] dag_run end_date Null after a dag is finished [AIRFLOW-2956] Kubernetes tolerations for pod operator [AIRFLOW-2997] Support for clustered tables in Bigquery hooks/operators -[AIRFLOW-3006] Fix rrror when schedule_interval="None" -[AIRFLOW-3008] Move Kubernetes related example DAGs to contribe/example_dags +[AIRFLOW-3006] Fix error when schedule_interval="None" +[AIRFLOW-3008] Move Kubernetes related example DAGs to contrib/example_dags [AIRFLOW-3025] Allow to specify dns and dns-search parameters for DockerOperator [AIRFLOW-3067] (www_rbac) Flask flash messages are not displayed properly (no background color) [AIRFLOW-3069] Decode output of S3 file transform operator From 0e1a18352761a5a558cc21a96fcdf64021a1ab25 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Fri, 9 Nov 2018 14:42:43 +0000 Subject: [PATCH 188/808] [AIRFLOW-XXX] Update Updating instructions for changes in 1.10.1 A cherry-pick went awry causing some thing to be duplicated --- UPDATING.md | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 0f108ffadc1ba..ace0bdae60f9b 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -3,15 +3,7 @@ This file documents any backwards-incompatible changes in Airflow and assists users migrating to a new version. -## Airflow 1.10 - -Installation and upgrading requires setting `SLUGIFY_USES_TEXT_UNIDECODE=yes` in your environment or -`AIRFLOW_GPL_UNIDECODE=yes`. In case of the latter a GPL runtime dependency will be installed due to a -dependency (python-nvd3 -> python-slugify -> unidecode). - -### Replace DataProcHook.await calls to DataProcHook.wait - -The method name was changed to be compatible with the Python 3.7 async/await keywords +## Airflow Master ### DAG level Access Control for new RBAC UI @@ -53,6 +45,12 @@ To delete a user: airflow users --delete --username jondoe ``` +## Airflow 1.10.1 + +### Replace DataProcHook.await calls to DataProcHook.wait + +The method name was changed to be compatible with the Python 3.7 async/await keywords + ### StatsD Metrics The `scheduler_heartbeat` metric has been changed from a gauge to a counter. Each loop of the scheduler will increment the counter by 1. This provides a higher degree of visibility and allows for better integration with Prometheus using the [StatsD Exporter](https://github.com/prometheus/statsd_exporter). Scheduler upness can be determined by graphing and alerting using a rate. If the scheduler goes down, the rate will drop to 0. From c71e078e03be0618630d7e7d87ae875a043a13a3 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 13 Nov 2018 00:56:09 +0800 Subject: [PATCH 189/808] [AIRFLOW-3325] Fix UI Page DAGs-column 'Recent Tasks' display issue (#4173) A new task state "Skipped" was added while UI was not updated accordingly. This makes the last circle in column "Recent Tasks" not displayed completely. Fixed both www/ and www_raac/ --- airflow/www/templates/airflow/dags.html | 2 +- airflow/www_rbac/templates/airflow/dags.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/www/templates/airflow/dags.html b/airflow/www/templates/airflow/dags.html index 0a7a6ec2a4791..5473189d439dd 100644 --- a/airflow/www/templates/airflow/dags.html +++ b/airflow/www/templates/airflow/dags.html @@ -402,7 +402,7 @@

      DAGs

      states = json[dag_id]; g = d3.select('svg#task-run-' + dag_id) .attr('height', diameter + (stroke_width_hover * 2)) - .attr('width', '240px') + .attr('width', '270px') .selectAll("g") .data(states) .enter() diff --git a/airflow/www_rbac/templates/airflow/dags.html b/airflow/www_rbac/templates/airflow/dags.html index c7aa33801488f..bba0d191dde35 100644 --- a/airflow/www_rbac/templates/airflow/dags.html +++ b/airflow/www_rbac/templates/airflow/dags.html @@ -401,7 +401,7 @@

      DAGs

      states = json[dag_id]; g = d3.select('svg#task-run-' + dag_id) .attr('height', diameter + (stroke_width_hover * 2)) - .attr('width', '240px') + .attr('width', '270px') .selectAll("g") .data(states) .enter() From ab5f5438d7787ec0f6b839ed528317276e2dac50 Mon Sep 17 00:00:00 2001 From: Neil Patel Date: Fri, 13 Jul 2018 12:12:39 +0200 Subject: [PATCH 190/808] [AIRFLOW-2723] Update lxml dependancy to >= 4.0.0 Closes #3583 from neil90/master --- setup.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index 49de5e6205423..fca659689958c 100644 --- a/setup.py +++ b/setup.py @@ -224,7 +224,7 @@ def write_version(filename=os.path.join(*['airflow', 'click==6.7', 'freezegun', 'jira', - 'lxml>=3.3.4', + 'lxml>=4.0.0', 'mock', 'mongomock', 'moto==1.1.19', @@ -287,7 +287,7 @@ def do_setup(): 'gunicorn>=19.4.0, <20.0', 'iso8601>=0.1.12', 'jinja2>=2.7.3, <2.9.0', - 'lxml>=3.6.0, <4.0', + 'lxml>=4.0.0', 'markdown>=2.5.2, <3.0', 'pandas>=0.17.1, <1.0.0', 'pendulum==1.4.4', From b5b9287a75596a617557798f1286cf7b89c55350 Mon Sep 17 00:00:00 2001 From: Stefan Seelmann Date: Mon, 12 Nov 2018 18:29:53 +0100 Subject: [PATCH 191/808] [AIRFLOW-3072] Assign permission get_logs_with_metadata to viewer role (#4175) --- airflow/www_rbac/security.py | 1 + tests/www_rbac/test_views.py | 107 +++++++++++++++++++++++++++++++++++ 2 files changed, 108 insertions(+) diff --git a/airflow/www_rbac/security.py b/airflow/www_rbac/security.py index 2405aa8f803de..72691565e64be 100644 --- a/airflow/www_rbac/security.py +++ b/airflow/www_rbac/security.py @@ -75,6 +75,7 @@ 'can_task_stats', 'can_code', 'can_log', + 'can_get_logs_with_metadata', 'can_tries', 'can_graph', 'can_tree', diff --git a/tests/www_rbac/test_views.py b/tests/www_rbac/test_views.py index d7f099d6be23b..f378783e90a42 100644 --- a/tests/www_rbac/test_views.py +++ b/tests/www_rbac/test_views.py @@ -786,5 +786,112 @@ def test_start_date_filter(self): pass +class TestLogViewPermission(TestBase): + """ + Test Airflow DAG acl + """ + default_date = timezone.datetime(2018, 6, 1) + run_id = "test_{}".format(models.DagRun.id_for_date(default_date)) + + @classmethod + def setUpClass(cls): + super(TestLogViewPermission, cls).setUpClass() + + def cleanup_dagruns(self): + DR = models.DagRun + dag_ids = ['example_bash_operator', + 'example_subdag_operator'] + (self.session + .query(DR) + .filter(DR.dag_id.in_(dag_ids)) + .filter(DR.run_id == self.run_id) + .delete(synchronize_session='fetch')) + self.session.commit() + + def prepare_dagruns(self): + dagbag = models.DagBag(include_examples=True) + self.bash_dag = dagbag.dags['example_bash_operator'] + self.sub_dag = dagbag.dags['example_subdag_operator'] + + self.bash_dagrun = self.bash_dag.create_dagrun( + run_id=self.run_id, + execution_date=self.default_date, + start_date=timezone.utcnow(), + state=State.RUNNING) + + self.sub_dagrun = self.sub_dag.create_dagrun( + run_id=self.run_id, + execution_date=self.default_date, + start_date=timezone.utcnow(), + state=State.RUNNING) + + def setUp(self): + super(TestLogViewPermission, self).setUp() + self.cleanup_dagruns() + self.prepare_dagruns() + self.logout() + + def login(self, username=None, password=None): + role_admin = self.appbuilder.sm.find_role('Admin') + tester = self.appbuilder.sm.find_user(username='test_admin') + if not tester: + self.appbuilder.sm.add_user( + username='test_admin', + first_name='test_admin', + last_name='test_admin', + email='test_admin@fab.org', + role=role_admin, + password='test_admin') + + role_user = self.appbuilder.sm.find_role('User') + test_user = self.appbuilder.sm.find_user(username='test_user') + if not test_user: + self.appbuilder.sm.add_user( + username='test_user', + first_name='test_user', + last_name='test_user', + email='test_user@fab.org', + role=role_user, + password='test_user') + + return self.client.post('/login/', data=dict( + username=username, + password=password + )) + + def logout(self): + return self.client.get('/logout/') + + def test_log_success_for_admin(self): + self.logout() + self.login(username='test_admin', + password='test_admin') + url = ('log?task_id=runme_0&dag_id=example_bash_operator&execution_date={}' + .format(self.percent_encode(self.default_date))) + resp = self.client.get(url, follow_redirects=True) + self.check_content_in_response('Log by attempts', resp) + url = ('get_logs_with_metadata?task_id=runme_0&dag_id=example_bash_operator&' + 'execution_date={}&try_number=1&metadata=null' + .format(self.percent_encode(self.default_date))) + resp = self.client.get(url, follow_redirects=True) + self.check_content_in_response('"message":', resp) + self.check_content_in_response('"metadata":', resp) + + def test_log_success_for_user(self): + self.logout() + self.login(username='test_user', + password='test_user') + url = ('log?task_id=runme_0&dag_id=example_bash_operator&execution_date={}' + .format(self.percent_encode(self.default_date))) + resp = self.client.get(url, follow_redirects=True) + self.check_content_in_response('Log by attempts', resp) + url = ('get_logs_with_metadata?task_id=runme_0&dag_id=example_bash_operator&' + 'execution_date={}&try_number=1&metadata=null' + .format(self.percent_encode(self.default_date))) + resp = self.client.get(url, follow_redirects=True) + self.check_content_in_response('"message":', resp) + self.check_content_in_response('"metadata":', resp) + + if __name__ == '__main__': unittest.main() From a01be426721bbebf0ed477f512e719569b69d43b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Szymon=20Bili=C5=84ski?= Date: Sun, 11 Nov 2018 23:26:29 +0100 Subject: [PATCH 192/808] [AIRFLOW-XXX] Add missing docs for SNS classes (#4155) --- docs/code.rst | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/code.rst b/docs/code.rst index 12db9b71b136d..0bf04cfb8f956 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -193,6 +193,7 @@ Operators .. autoclass:: airflow.contrib.operators.sftp_operator.SFTPOperator .. autoclass:: airflow.contrib.operators.slack_webhook_operator.SlackWebhookOperator .. autoclass:: airflow.contrib.operators.snowflake_operator.SnowflakeOperator +.. autoclass:: airflow.contrib.operators.sns_publish_operator.SnsPublishOperator .. autoclass:: airflow.contrib.operators.spark_jdbc_operator.SparkJDBCOperator .. autoclass:: airflow.contrib.operators.spark_sql_operator.SparkSqlOperator .. autoclass:: airflow.contrib.operators.spark_submit_operator.SparkSubmitOperator @@ -379,8 +380,10 @@ Community contributed hooks ''''''''''''''''''''''''''' .. Alphabetize this list .. autoclass:: airflow.contrib.hooks.aws_dynamodb_hook.AwsDynamoDBHook +.. autoclass:: airflow.contrib.hooks.aws_firehose_hook.AwsFirehoseHook .. autoclass:: airflow.contrib.hooks.aws_hook.AwsHook .. autoclass:: airflow.contrib.hooks.aws_lambda_hook.AwsLambdaHook +.. autoclass:: airflow.contrib.hooks.aws_sns_hook.AwsSnsHook .. autoclass:: airflow.contrib.hooks.azure_data_lake_hook.AzureDataLakeHook .. autoclass:: airflow.contrib.hooks.azure_fileshare_hook.AzureFileShareHook .. autoclass:: airflow.contrib.hooks.bigquery_hook.BigQueryHook From 24dfbd22c08aa5ba90e0c95dace68a1a91008338 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 12 Nov 2018 18:23:49 +0000 Subject: [PATCH 193/808] [AIRFLOW-XXX] Sync changelog between release and master branch --- UPDATING.md | 130 ++++++++++++++++++++++------------------------------ 1 file changed, 56 insertions(+), 74 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index ace0bdae60f9b..200266bf33d1a 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -3,54 +3,8 @@ This file documents any backwards-incompatible changes in Airflow and assists users migrating to a new version. -## Airflow Master - -### DAG level Access Control for new RBAC UI - -Extend and enhance new Airflow RBAC UI to support DAG level ACL. Each dag now has two permissions(one for write, one for read) associated('can_dag_edit', 'can_dag_read'). -The admin will create new role, associate the dag permission with the target dag and assign that role to users. That user can only access / view the certain dags on the UI -that he has permissions on. If a new role wants to access all the dags, the admin could associate dag permissions on an artificial view(``all_dags``) with that role. - -We also provide a new cli command(``sync_perm``) to allow admin to auto sync permissions. - - -### min_file_parsing_loop_time config option temporarily disabled - -The scheduler.min_file_parsing_loop_time config option has been temporarily removed due to -some bugs. - -### new `sync_parallelism` config option in celery section - -The new `sync_parallelism` config option will control how many processes CeleryExecutor will use to -fetch celery task state in parallel. Default value is max(1, number of cores - 1) - -### CLI Changes - -The ability to manipulate users from the command line has been changed. 'airflow create_user' and 'airflow delete_user' and 'airflow list_users' has been grouped to a single command `airflow users` with optional flags `--create`, `--list` and `--delete`. - -Example Usage: - -To create a new user: -```bash -airflow users --create --username jondoe --lastname doe --firstname jon --email jdoe@apache.org --role Viewer --password test -``` - -To list users: -```bash -airflow users --list -``` - -To delete a user: -```bash -airflow users --delete --username jondoe -``` - ## Airflow 1.10.1 -### Replace DataProcHook.await calls to DataProcHook.wait - -The method name was changed to be compatible with the Python 3.7 async/await keywords - ### StatsD Metrics The `scheduler_heartbeat` metric has been changed from a gauge to a counter. Each loop of the scheduler will increment the counter by 1. This provides a higher degree of visibility and allows for better integration with Prometheus using the [StatsD Exporter](https://github.com/prometheus/statsd_exporter). Scheduler upness can be determined by graphing and alerting using a rate. If the scheduler goes down, the rate will drop to 0. @@ -105,6 +59,7 @@ The method name was changed to be compatible with the Python 3.7 async/await key ### Setting UTF-8 as default mime_charset in email utils ### Add a configuration variable(default_dag_run_display_number) to control numbers of dag run for display + Add a configuration variable(default_dag_run_display_number) under webserver section to control num of dag run to show in UI. ### Default executor for SubDagOperator is changed to SequentialExecutor @@ -134,11 +89,13 @@ Run `airflow webserver` to start the new UI. This will bring up a log in page, e There are five roles created for Airflow by default: Admin, User, Op, Viewer, and Public. To configure roles/permissions, go to the `Security` tab and click `List Roles` in the new UI. #### Breaking changes + - AWS Batch Operator renamed property queue to job_queue to prevent conflict with the internal queue from CeleryExecutor - AIRFLOW-2542 - Users created and stored in the old users table will not be migrated automatically. FAB's built-in authentication support must be reconfigured. - Airflow dag home page is now `/home` (instead of `/admin`). - All ModelViews in Flask-AppBuilder follow a different pattern from Flask-Admin. The `/admin` part of the url path will no longer exist. For example: `/admin/connection` becomes `/connection/list`, `/admin/connection/new` becomes `/connection/add`, `/admin/connection/edit` becomes `/connection/edit`, etc. - Due to security concerns, the new webserver will no longer support the features in the `Data Profiling` menu of old UI, including `Ad Hoc Query`, `Charts`, and `Known Events`. +- HiveServer2Hook.get_results() always returns a list of tuples, even when a single column is queried, as per Python API 2. ### airflow.contrib.sensors.hdfs_sensors renamed to airflow.contrib.sensors.hdfs_sensor @@ -152,6 +109,7 @@ to have specified `explicit_defaults_for_timestamp=1` in your my.cnf under `[mys ### Celery config To make the config of Airflow compatible with Celery, some properties have been renamed: + ``` celeryd_concurrency -> worker_concurrency celery_result_backend -> result_backend @@ -159,73 +117,76 @@ celery_ssl_active -> ssl_active celery_ssl_cert -> ssl_cert celery_ssl_key -> ssl_key ``` + Resulting in the same config parameters as Celery 4, with more transparency. ### GCP Dataflow Operators + Dataflow job labeling is now supported in Dataflow{Java,Python}Operator with a default "airflow-version" label, please upgrade your google-cloud-dataflow or apache-beam version to 2.2.0 or greater. ### BigQuery Hooks and Operator + The `bql` parameter passed to `BigQueryOperator` and `BigQueryBaseCursor.run_query` has been deprecated and renamed to `sql` for consistency purposes. Using `bql` will still work (and raise a `DeprecationWarning`), but is no longer supported and will be removed entirely in Airflow 2.0 ### Redshift to S3 Operator + With Airflow 1.9 or lower, Unload operation always included header row. In order to include header row, we need to turn off parallel unload. It is preferred to perform unload operation using all nodes so that it is faster for larger tables. So, parameter called `include_header` is added and default is set to False. -Header row will be added only if this parameter is set True and also in that case parallel will be automatically turned off (`PARALLEL OFF`) +Header row will be added only if this parameter is set True and also in that case parallel will be automatically turned off (`PARALLEL OFF`) ### Google cloud connection string With Airflow 1.9 or lower, there were two connection strings for the Google Cloud operators, both `google_cloud_storage_default` and `google_cloud_default`. This can be confusing and therefore the `google_cloud_storage_default` connection id has been replaced with `google_cloud_default` to make the connection id consistent across Airflow. ### Logging Configuration + With Airflow 1.9 or lower, `FILENAME_TEMPLATE`, `PROCESSOR_FILENAME_TEMPLATE`, `LOG_ID_TEMPLATE`, `END_OF_LOG_MARK` were configured in `airflow_local_settings.py`. These have been moved into the configuration file, and hence if you were using a custom configuration file the following defaults need to be added. + ``` [core] fab_logging_level = WARN -log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log -log_processor_filename_template = {{ filename }}.log +log_filename_template = {{{{ ti.dag_id }}}}/{{{{ ti.task_id }}}}/{{{{ ts }}}}/{{{{ try_number }}}}.log +log_processor_filename_template = {{{{ filename }}}}.log [elasticsearch] -elasticsearch_log_id_template = {dag_id}-{task_id}-{execution_date}-{try_number} +elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_number}} elasticsearch_end_of_log_mark = end_of_log ``` -### Custom auth backends interface change +The previous setting of `log_task_reader` is not needed in many cases now when using the default logging config with remote storages. (Previously it needed to be set to `s3.task` or similar. This is not needed with the default config anymore) -We have updated the version of flask-login we depend upon, and as a result any -custom auth backends might need a small change: `is_active`, -`is_authenticated`, and `is_anonymous` should now be properties. What this means is if -previously you had this in your user class +#### Change of per-task log path - def is_active(self): - return self.active - -then you need to change it like this +With the change to Airflow core to be timezone aware the default log path for task instances will now include timezone information. This will by default mean all previous task logs won't be found. You can get the old behaviour back by setting the following config options: - @property - def is_active(self): - return self.active +``` +[core] +log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ execution_date.strftime("%%Y-%%m-%%dT%%H:%%M:%%S") }}/{{ try_number }}.log +``` ## Airflow 1.9 ### SSH Hook updates, along with new SSH Operator & SFTP Operator SSH Hook now uses the Paramiko library to create an ssh client connection, instead of the sub-process based ssh command execution previously (<1.9.0), so this is backward incompatible. - - update SSHHook constructor - - use SSHOperator class in place of SSHExecuteOperator which is removed now. Refer to test_ssh_operator.py for usage info. - - SFTPOperator is added to perform secure file transfer from serverA to serverB. Refer to test_sftp_operator.py.py for usage info. - - No updates are required if you are using ftpHook, it will continue to work as is. + +- update SSHHook constructor +- use SSHOperator class in place of SSHExecuteOperator which is removed now. Refer to test_ssh_operator.py for usage info. +- SFTPOperator is added to perform secure file transfer from serverA to serverB. Refer to test_sftp_operator.py.py for usage info. +- No updates are required if you are using ftpHook, it will continue to work as is. ### S3Hook switched to use Boto3 The airflow.hooks.S3_hook.S3Hook has been switched to use boto3 instead of the older boto (a.k.a. boto2). This results in a few backwards incompatible changes to the following classes: S3Hook: - - the constructors no longer accepts `s3_conn_id`. It is now called `aws_conn_id`. - - the default connection is now "aws_default" instead of "s3_default" - - the return type of objects returned by `get_bucket` is now boto3.s3.Bucket - - the return type of `get_key`, and `get_wildcard_key` is now an boto3.S3.Object. + +- the constructors no longer accepts `s3_conn_id`. It is now called `aws_conn_id`. +- the default connection is now "aws_default" instead of "s3_default" +- the return type of objects returned by `get_bucket` is now boto3.s3.Bucket +- the return type of `get_key`, and `get_wildcard_key` is now an boto3.S3.Object. If you are using any of these in your DAGs and specify a connection ID you will need to update the parameter name for the connection to "aws_conn_id": S3ToHiveTransfer, S3PrefixSensor, S3KeySensor, RedshiftToS3Transfer. @@ -401,10 +362,11 @@ The `file_task_handler` logger has been made more flexible. The default format c If you are logging to Google cloud storage, please see the [Google cloud platform documentation](https://airflow.incubator.apache.org/integration.html#gcp-google-cloud-platform) for logging instructions. If you are using S3, the instructions should be largely the same as the Google cloud platform instructions above. You will need a custom logging config. The `REMOTE_BASE_LOG_FOLDER` configuration key in your airflow config has been removed, therefore you will need to take the following steps: - - Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/incubator-airflow/blob/master/airflow/config_templates/airflow_local_settings.py). - - Place it in a directory inside the Python import path `PYTHONPATH`. If you are using Python 2.7, ensuring that any `__init__.py` files exist so that it is importable. - - Update the config by setting the path of `REMOTE_BASE_LOG_FOLDER` explicitly in the config. The `REMOTE_BASE_LOG_FOLDER` key is not used anymore. - - Set the `logging_config_class` to the filename and dict. For example, if you place `custom_logging_config.py` on the base of your pythonpath, you will need to set `logging_config_class = custom_logging_config.LOGGING_CONFIG` in your config as Airflow 1.8. + +- Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/incubator-airflow/blob/master/airflow/config_templates/airflow_local_settings.py). +- Place it in a directory inside the Python import path `PYTHONPATH`. If you are using Python 2.7, ensuring that any `__init__.py` files exist so that it is importable. +- Update the config by setting the path of `REMOTE_BASE_LOG_FOLDER` explicitly in the config. The `REMOTE_BASE_LOG_FOLDER` key is not used anymore. +- Set the `logging_config_class` to the filename and dict. For example, if you place `custom_logging_config.py` on the base of your pythonpath, you will need to set `logging_config_class = custom_logging_config.LOGGING_CONFIG` in your config as Airflow 1.8. ### New Features @@ -413,8 +375,10 @@ If you are using S3, the instructions should be largely the same as the Google c A new DaskExecutor allows Airflow tasks to be run in Dask Distributed clusters. ### Deprecated Features + These features are marked for deprecation. They may still work (and raise a `DeprecationWarning`), but are no longer supported and will be removed entirely in Airflow 2.0 + - If you're using the `google_cloud_conn_id` or `dataproc_cluster` argument names explicitly in `contrib.operators.Dataproc{*}Operator`(s), be sure to rename them to `gcp_conn_id` or `cluster_name`, respectively. We've renamed these arguments for consistency. (AIRFLOW-1323) - `post_execute()` hooks now take two arguments, `context` and `result` @@ -438,30 +402,36 @@ a previously installed version of Airflow before installing 1.8.1. ## Airflow 1.8 ### Database + The database schema needs to be upgraded. Make sure to shutdown Airflow and make a backup of your database. To upgrade the schema issue `airflow upgradedb`. ### Upgrade systemd unit files + Systemd unit files have been updated. If you use systemd please make sure to update these. > Please note that the webserver does not detach properly, this will be fixed in a future version. ### Tasks not starting although dependencies are met due to stricter pool checking + Airflow 1.7.1 has issues with being able to over subscribe to a pool, ie. more slots could be used than were available. This is fixed in Airflow 1.8.0, but due to past issue jobs may fail to start although their dependencies are met after an upgrade. To workaround either temporarily increase the amount of slots above the amount of queued tasks or use a new pool. ### Less forgiving scheduler on dynamic start_date + Using a dynamic start_date (e.g. `start_date = datetime.now()`) is not considered a best practice. The 1.8.0 scheduler is less forgiving in this area. If you encounter DAGs not being scheduled you can try using a fixed start_date and renaming your DAG. The last step is required to make sure you start with a clean slate, otherwise the old schedule can interfere. ### New and updated scheduler options + Please read through the new scheduler options, defaults have changed since 1.7.1. #### child_process_log_directory + In order to increase the robustness of the scheduler, DAGS are now processed in their own process. Therefore each DAG has its own log file for the scheduler. These log files are placed in `child_process_log_directory` which defaults to `/scheduler/latest`. You will need to make sure these log files are removed. @@ -469,15 +439,18 @@ DAG has its own log file for the scheduler. These log files are placed in `child > DAG logs or processor logs ignore and command line settings for log file locations. #### run_duration + Previously the command line option `num_runs` was used to let the scheduler terminate after a certain amount of loops. This is now time bound and defaults to `-1`, which means run continuously. See also num_runs. #### num_runs + Previously `num_runs` was used to let the scheduler terminate after a certain amount of loops. Now num_runs specifies the number of times to try to schedule each DAG file within `run_duration` time. Defaults to `-1`, which means try indefinitely. This is only available on the command line. #### min_file_process_interval + After how much time should an updated DAG be picked up from the filesystem. #### min_file_parsing_loop_time @@ -485,9 +458,11 @@ CURRENTLY DISABLED DUE TO A BUG How many seconds to wait between file-parsing loops to prevent the logs from being spammed. #### dag_dir_list_interval + The frequency with which the scheduler should relist the contents of the DAG directory. If while developing +dags, they are not being picked up, have a look at this number and decrease it when necessary. #### catchup_by_default + By default the scheduler will fill any missing interval DAG Runs between the last execution date and the current date. This setting changes that behavior to only execute the latest interval. This can also be specified per DAG as `catchup = False / True`. Command line backfills will still work. @@ -518,6 +493,7 @@ required to whitelist these variables by adding the following to your configurat airflow\.ctx\..* ``` + ### Google Cloud Operator and Hook alignment All Google Cloud Operators and Hooks are aligned and use the same client library. Now you have a single connection @@ -529,6 +505,7 @@ Also the old P12 key file type is not supported anymore and only the new JSON ke account. ### Deprecated Features + These features are marked for deprecation. They may still work (and raise a `DeprecationWarning`), but are no longer supported and will be removed entirely in Airflow 2.0 @@ -545,6 +522,7 @@ supported and will be removed entirely in Airflow 2.0 - The config value secure_mode will default to True which will disable some insecure endpoints/features ### Known Issues + There is a report that the default of "-1" for num_runs creates an issue where errors are reported while parsing tasks. It was not confirmed, but a workaround was found by changing the default back to `None`. @@ -571,7 +549,9 @@ To continue using the default smtp email backend, change the email_backend line [email] email_backend = airflow.utils.send_email_smtp ``` + to: + ``` [email] email_backend = airflow.utils.email.send_email_smtp @@ -584,7 +564,9 @@ To continue using S3 logging, update your config file so: ``` s3_log_folder = s3://my-airflow-log-bucket/logs ``` + becomes: + ``` remote_base_log_folder = s3://my-airflow-log-bucket/logs remote_log_conn_id = From 6c9683cd4a24a5ec57ec3eaffdfa2a962346a764 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 12 Nov 2018 20:49:55 +0000 Subject: [PATCH 194/808] [AIRFLOW-2779] Add project version to license (#4177) The Incubator PMC requested that we add version information to the license declarations to cope with the possibility of projects changing licenses on us in the future. Where possible I have worked out the version we are actually using, but where that hasn't been possible and the project is still on the same license (they all were) I have taken the latest version as of right now [ci-skip] --- LICENSE | 37 +++++++++++++++++++------------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/LICENSE b/LICENSE index 2288b870b4239..7f2532df77c07 100644 --- a/LICENSE +++ b/LICENSE @@ -217,9 +217,10 @@ The following components are provided under the Apache 2.0 License. See project link for details. The text of each license is also included at licenses/LICENSE-[project].txt. - (ALv2 License) hue (https://github.com/cloudera/hue/) - (ALv2 License) jqclock (https://github.com/JohnRDOrazio/jQuery-Clock-Plugin) - (ALv2 License) bootstrap3-typeahead (https://github.com/bassjobsen/Bootstrap-3-Typeahead) + (ALv2 License) hue v4.3.0 (https://github.com/cloudera/hue/) + (ALv2 License) jqclock v2.3.0 (https://github.com/JohnRDOrazio/jQuery-Clock-Plugin) + (ALv2 License) bootstrap3-typeahead v4.0.2 (https://github.com/bassjobsen/Bootstrap-3-Typeahead) + (ALv2 License) airflow.contrib.auth.backends.github_enterprise_auth ======================================================================== MIT licenses @@ -228,16 +229,16 @@ MIT licenses The following components are provided under the MIT License. See project link for details. The text of each license is also included at licenses/LICENSE-[project].txt. - (MIT License) jquery (https://jquery.org/license/) - (MIT License) dagre-d3 (https://github.com/cpettitt/dagre-d3) - (MIT License) bootstrap (https://github.com/twbs/bootstrap/) - (MIT License) d3-tip (https://github.com/Caged/d3-tip) - (MIT License) dataTables (https://datatables.net) - (MIT License) WebGL-2D (https://github.com/gameclosure/webgl-2d) - (MIT License) Underscorejs (http://underscorejs.org) - (MIT License) Bootstrap Toggle (http://www.bootstraptoggle.com) - (MIT License) normalize.css (http://necolas.github.io/normalize.css/) - (MIT License) ElasticMock (https://github.com/vrcmarcos/elasticmock) + (MIT License) jquery v2.1.4 (https://jquery.org/license/) + (MIT License) dagre-d3 v0.6.1 (https://github.com/cpettitt/dagre-d3) + (MIT License) bootstrap v3.2 (https://github.com/twbs/bootstrap/) + (MIT License) d3-tip v0.6.3 (https://github.com/Caged/d3-tip) + (MIT License) dataTables v1.10.10 (https://datatables.net) + (MIT License) WebGL-2D (git-commit 9a7ec26) (https://github.com/gameclosure/webgl-2d) + (MIT License) Underscorejs v1.5.0 (http://underscorejs.org) + (MIT License) Bootstrap Toggle v2.2.0 (http://www.bootstraptoggle.com) + (MIT License) normalize.css v3.0.2 (http://necolas.github.io/normalize.css/) + (MIT License) ElasticMock v1.3.2 (https://github.com/vrcmarcos/elasticmock) ======================================================================== BSD 2-Clause licenses @@ -246,7 +247,7 @@ The following components are provided under the BSD 2-Clause license. See file headers and project links for details. The text of each license is also included at licenses/LICENSE-[project].txt. - (BSD 2 License) flask-kerberos (https://github.com/mkomitee/flask-kerberos) + (BSD 2 License) flask-kerberos v1.0.4 (https://github.com/mkomitee/flask-kerberos) ======================================================================== BSD 3-Clause licenses @@ -254,8 +255,8 @@ BSD 3-Clause licenses The following components are provided under the BSD 3-Clause license. See project links for details. The text of each license is also included at licenses/LICENSE-[project].txt. - (BSD 3 License) Ace (https://github.com/ajaxorg/ace) - (BSD 3 License) d3js (https://d3js.org) - (BSD 3 License) parallel-coordinates (http://syntagmatic.github.com/parallel-coordinates/) - (BSD 3 License) scikit-learn (https://github.com/scikit-learn/scikit-learn) + (BSD 3 License) Ace v1.1.8 (https://github.com/ajaxorg/ace) + (BSD 3 License) d3js v3.5.17 (https://d3js.org) + (BSD 3 License) parallel-coordinates v0.7.0 (http://syntagmatic.github.com/parallel-coordinates/) + (BSD 3 License) scikit-learn v0.19.1 (https://github.com/scikit-learn/scikit-learn) From 8b4a3038043bd61d4fd22eaecea2ec155d8ba20e Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 12 Nov 2018 21:18:32 +0000 Subject: [PATCH 195/808] [AIRFLOW-2779] Add license headers to doc files (#4178) This adds ASF license headers to all the .rst and .md files with the exception of the Pull Request template (as that is included verbatim when opening a Pull Request on Github which would be messy) --- CONTRIBUTING.md | 19 +++++++++++++++ README.md | 21 ++++++++++++++++- TODO.md | 19 +++++++++++++++ UPDATING.md | 19 +++++++++++++++ .../example_dags/example_twitter_README.md | 19 +++++++++++++++ .../www/templates/airflow/variables/README.md | 23 +++++++++++++++++-- dev/README.md | 19 +++++++++++++++ docs/api.rst | 17 ++++++++++++++ docs/cli.rst | 17 ++++++++++++++ docs/code.rst | 17 ++++++++++++++ docs/concepts.rst | 17 ++++++++++++++ docs/faq.rst | 17 ++++++++++++++ docs/howto/executor/use-celery.rst | 17 ++++++++++++++ docs/howto/executor/use-dask.rst | 17 ++++++++++++++ docs/howto/executor/use-mesos.rst | 17 ++++++++++++++ docs/howto/index.rst | 17 ++++++++++++++ docs/howto/initialize-database.rst | 17 ++++++++++++++ docs/howto/manage-connections.rst | 17 ++++++++++++++ docs/howto/operator.rst | 17 ++++++++++++++ docs/howto/run-with-systemd.rst | 17 ++++++++++++++ docs/howto/run-with-upstart.rst | 17 ++++++++++++++ docs/howto/secure-connections.rst | 17 ++++++++++++++ docs/howto/set-config.rst | 17 ++++++++++++++ docs/howto/use-test-config.rst | 17 ++++++++++++++ docs/howto/write-logs.rst | 17 ++++++++++++++ docs/index.rst | 17 ++++++++++++++ docs/installation.rst | 17 ++++++++++++++ docs/integration.rst | 17 ++++++++++++++ docs/kubernetes.rst | 17 ++++++++++++++ docs/license.rst | 17 ++++++++++++++ docs/lineage.rst | 17 ++++++++++++++ docs/plugins.rst | 17 ++++++++++++++ docs/profiling.rst | 19 +++++++++++++++ docs/project.rst | 17 ++++++++++++++ docs/scheduler.rst | 17 ++++++++++++++ docs/security.rst | 17 ++++++++++++++ docs/start.rst | 17 ++++++++++++++ docs/timezone.rst | 17 ++++++++++++++ docs/tutorial.rst | 17 ++++++++++++++ docs/ui.rst | 17 ++++++++++++++ scripts/ci/kubernetes/README.md | 19 +++++++++++++++ tests/dags/README.md | 19 +++++++++++++++ tests/test_utils/README.md | 21 ++++++++++++++++- 43 files changed, 757 insertions(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 21cbd5010c4ee..194aaa7b277ca 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,3 +1,22 @@ + + # Contributing Contributions are welcome and are greatly appreciated! Every diff --git a/README.md b/README.md index f5fe2e1bdcd4c..c1893cfba8013 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,23 @@ -# Airflow + + +# Apache Airflow (Incubating) [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) [![Build Status](https://travis-ci.org/apache/incubator-airflow.svg?branch=master)](https://travis-ci.org/apache/incubator-airflow) diff --git a/TODO.md b/TODO.md index cf19035e1b995..1e4e6ed249e1a 100644 --- a/TODO.md +++ b/TODO.md @@ -1,3 +1,22 @@ + + #### Roadmap items * UI page answering "Why isn't this task instance running?" * Attempt removing DagBag caching for the web server diff --git a/UPDATING.md b/UPDATING.md index 200266bf33d1a..8336d73cc9337 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -1,3 +1,22 @@ + + # Updating Airflow This file documents any backwards-incompatible changes in Airflow and diff --git a/airflow/contrib/example_dags/example_twitter_README.md b/airflow/contrib/example_dags/example_twitter_README.md index 319eac39f6953..28ae9826f1f06 100644 --- a/airflow/contrib/example_dags/example_twitter_README.md +++ b/airflow/contrib/example_dags/example_twitter_README.md @@ -1,3 +1,22 @@ + + # Example Twitter DAG ***Introduction:*** This example dag depicts a typical ETL process and is a perfect use case automation scenario for Airflow. Please note that the main scripts associated with the tasks are returning None. The purpose of this DAG is to demonstrate how to write a functional DAG within Airflow. diff --git a/airflow/www/templates/airflow/variables/README.md b/airflow/www/templates/airflow/variables/README.md index 3fd539f8b54fb..e3b30a1a64b93 100644 --- a/airflow/www/templates/airflow/variables/README.md +++ b/airflow/www/templates/airflow/variables/README.md @@ -1,5 +1,24 @@ -## Variable Editor ----- + + +# Variable Editor + This folder contains forms used to edit values in the "Variable" key-value store. This data can be edited under the "Admin" admin tab, but sometimes it is preferable to use a form that can perform checking and provide a nicer diff --git a/dev/README.md b/dev/README.md index 084020cfabb66..cd4a932adb468 100755 --- a/dev/README.md +++ b/dev/README.md @@ -1,3 +1,22 @@ + + # Development Tools ## Airflow Pull Request Tool diff --git a/docs/api.rst b/docs/api.rst index 4ea19c8969f92..194809abc6330 100644 --- a/docs/api.rst +++ b/docs/api.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Experimental Rest API ===================== diff --git a/docs/cli.rst b/docs/cli.rst index f05cbfbe27ed2..4d68d0eef3b8d 100644 --- a/docs/cli.rst +++ b/docs/cli.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Command Line Interface ====================== diff --git a/docs/code.rst b/docs/code.rst index 0bf04cfb8f956..afa0dc4c112bc 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + API Reference ============= diff --git a/docs/concepts.rst b/docs/concepts.rst index f1fd14f5b187b..95f6e6a7a60ae 100644 --- a/docs/concepts.rst +++ b/docs/concepts.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Concepts ######## diff --git a/docs/faq.rst b/docs/faq.rst index 46212084c58fd..42dfb278f832a 100644 --- a/docs/faq.rst +++ b/docs/faq.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + FAQ ======== diff --git a/docs/howto/executor/use-celery.rst b/docs/howto/executor/use-celery.rst index 0e1bab060b0b2..71cae66a6f5b9 100644 --- a/docs/howto/executor/use-celery.rst +++ b/docs/howto/executor/use-celery.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Scaling Out with Celery ======================= diff --git a/docs/howto/executor/use-dask.rst b/docs/howto/executor/use-dask.rst index 769ce17e01a02..6d3efcb349a76 100644 --- a/docs/howto/executor/use-dask.rst +++ b/docs/howto/executor/use-dask.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Scaling Out with Dask ===================== diff --git a/docs/howto/executor/use-mesos.rst b/docs/howto/executor/use-mesos.rst index c3bf95a63aa36..d67579ddcb96b 100644 --- a/docs/howto/executor/use-mesos.rst +++ b/docs/howto/executor/use-mesos.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Scaling Out with Mesos (community contributed) ============================================== diff --git a/docs/howto/index.rst b/docs/howto/index.rst index f9f160e6bdbea..8a19a9ecb40a2 100644 --- a/docs/howto/index.rst +++ b/docs/howto/index.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + How-to Guides ============= diff --git a/docs/howto/initialize-database.rst b/docs/howto/initialize-database.rst index c85142f35b1c2..fad75be46d25a 100644 --- a/docs/howto/initialize-database.rst +++ b/docs/howto/initialize-database.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Initializing a Database Backend =============================== diff --git a/docs/howto/manage-connections.rst b/docs/howto/manage-connections.rst index 4d0adfb1009dc..9c324d259d776 100644 --- a/docs/howto/manage-connections.rst +++ b/docs/howto/manage-connections.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Managing Connections ===================== diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index 6333e32dd7d67..2c84a7f88e888 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Using Operators =============== diff --git a/docs/howto/run-with-systemd.rst b/docs/howto/run-with-systemd.rst index 1dc138347c37a..131fc3ddc0612 100644 --- a/docs/howto/run-with-systemd.rst +++ b/docs/howto/run-with-systemd.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Running Airflow with systemd ============================ diff --git a/docs/howto/run-with-upstart.rst b/docs/howto/run-with-upstart.rst index c18a203f17b31..adfb85a31effc 100644 --- a/docs/howto/run-with-upstart.rst +++ b/docs/howto/run-with-upstart.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Running Airflow with upstart ============================ diff --git a/docs/howto/secure-connections.rst b/docs/howto/secure-connections.rst index b3b9ba193d295..719db2f3f9655 100644 --- a/docs/howto/secure-connections.rst +++ b/docs/howto/secure-connections.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Securing Connections ==================== diff --git a/docs/howto/set-config.rst b/docs/howto/set-config.rst index fc8d59da189ee..cedce11324124 100644 --- a/docs/howto/set-config.rst +++ b/docs/howto/set-config.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Setting Configuration Options ============================= diff --git a/docs/howto/use-test-config.rst b/docs/howto/use-test-config.rst index 5cb4790287992..4f272afe96f35 100644 --- a/docs/howto/use-test-config.rst +++ b/docs/howto/use-test-config.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Using the Test Mode Configuration ================================= diff --git a/docs/howto/write-logs.rst b/docs/howto/write-logs.rst index 44fefc64809f3..935e79717c971 100644 --- a/docs/howto/write-logs.rst +++ b/docs/howto/write-logs.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Writing Logs ============ diff --git a/docs/index.rst b/docs/index.rst index 4cd96eaa74e43..4c18ce5ce6df9 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + .. image:: img/pin_large.png :width: 100 diff --git a/docs/installation.rst b/docs/installation.rst index 9e9ab3edc289d..5faca5efe4d20 100644 --- a/docs/installation.rst +++ b/docs/installation.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Installation ------------ diff --git a/docs/integration.rst b/docs/integration.rst index 2d9bc84c35928..1ab8e609d0dc2 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Integration =========== diff --git a/docs/kubernetes.rst b/docs/kubernetes.rst index 0c8b956ce2317..372b27c009acc 100644 --- a/docs/kubernetes.rst +++ b/docs/kubernetes.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Kubernetes Executor ^^^^^^^^^^^^^^^^^^^ diff --git a/docs/license.rst b/docs/license.rst index 3c53035713267..bcb2b76bf3de6 100644 --- a/docs/license.rst +++ b/docs/license.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + License ======= diff --git a/docs/lineage.rst b/docs/lineage.rst index 719ef0115eb0c..c94fe70e045c9 100644 --- a/docs/lineage.rst +++ b/docs/lineage.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Lineage ======= diff --git a/docs/plugins.rst b/docs/plugins.rst index 8173e632e5f3f..61b49577c7eed 100644 --- a/docs/plugins.rst +++ b/docs/plugins.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Plugins ======= diff --git a/docs/profiling.rst b/docs/profiling.rst index 09102334b1c29..c4f1c0b4869af 100644 --- a/docs/profiling.rst +++ b/docs/profiling.rst @@ -1,3 +1,22 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + +.. TODO: This section would be removed after we migrate to www_rbac completely. + Data Profiling ============== diff --git a/docs/project.rst b/docs/project.rst index d1f2cc010c939..6e4074f0918ba 100644 --- a/docs/project.rst +++ b/docs/project.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Project ======= diff --git a/docs/scheduler.rst b/docs/scheduler.rst index 84dcc4a7f9db4..72a3d8f7a5cb2 100644 --- a/docs/scheduler.rst +++ b/docs/scheduler.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Scheduling & Triggers ===================== diff --git a/docs/security.rst b/docs/security.rst index 1ecc958d52ab4..0be3609073495 100644 --- a/docs/security.rst +++ b/docs/security.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Security ======== diff --git a/docs/start.rst b/docs/start.rst index a3e21f93f68de..e3b16b28f7fab 100644 --- a/docs/start.rst +++ b/docs/start.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Quick Start ----------- diff --git a/docs/timezone.rst b/docs/timezone.rst index fe44ecfbb9f62..078f948c4f903 100644 --- a/docs/timezone.rst +++ b/docs/timezone.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Time zones ========== diff --git a/docs/tutorial.rst b/docs/tutorial.rst index 570cd75c809d9..69670d7b025cf 100644 --- a/docs/tutorial.rst +++ b/docs/tutorial.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + Tutorial ================ diff --git a/docs/ui.rst b/docs/ui.rst index 5a09ce5aa0a92..a2f6b8514c89a 100644 --- a/docs/ui.rst +++ b/docs/ui.rst @@ -1,3 +1,20 @@ +.. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + +.. http://www.apache.org/licenses/LICENSE-2.0 + +.. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + UI / Screenshots ================= The Airflow UI makes it easy to monitor and troubleshoot your data pipelines. diff --git a/scripts/ci/kubernetes/README.md b/scripts/ci/kubernetes/README.md index 5d1f9c195c0be..675be9fc4a042 100644 --- a/scripts/ci/kubernetes/README.md +++ b/scripts/ci/kubernetes/README.md @@ -1,3 +1,22 @@ + + # Airflow on Kubernetes If you don't have minikube installed, please run `./minikube/start_minikube.sh` to start a k8s-instance on your local machine. Make sure that your `kubectl` is pointing to the local k8s instance. diff --git a/tests/dags/README.md b/tests/dags/README.md index c5d6acb0f4c2b..615e8ab7f6968 100644 --- a/tests/dags/README.md +++ b/tests/dags/README.md @@ -1,3 +1,22 @@ + + # Unit Tests DAGs Folder This folder contains DAGs for Airflow unit testing. diff --git a/tests/test_utils/README.md b/tests/test_utils/README.md index 8a5c90dfd25f4..a93a0aa34c13c 100644 --- a/tests/test_utils/README.md +++ b/tests/test_utils/README.md @@ -1 +1,20 @@ -Utilities for use in tests. + + +# Utilities for use in tests. From cc12b35b7797a21706b42dd68d19f53017c1bbe1 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 12 Nov 2018 22:11:33 +0000 Subject: [PATCH 196/808] [AIRFLOW-XXX] Changelog and version for 1.10.1 --- CHANGELOG.txt | 4 ++++ airflow/version.py | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index db77cc2903110..92e62698132b1 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -24,6 +24,7 @@ Improvements: [AIRFLOW-2622] Add "confirm=False" option to SFTPOperator [AIRFLOW-2662] support affinity & nodeSelector policies for kubernetes executor/operator [AIRFLOW-2709] Improve error handling in Databricks hook +[AIRFLOW-2723] Update lxml dependancy to >= 4.0. [AIRFLOW-2763] No precheck mechanism in place during worker initialisation for the connection to metadata database [AIRFLOW-2789] Add ability to create single node cluster to DataprocClusterCreateOperator [AIRFLOW-2797] Add ability to create Google Dataproc cluster with custom image @@ -42,6 +43,7 @@ Improvements: [AIRFLOW-3025] Allow to specify dns and dns-search parameters for DockerOperator [AIRFLOW-3067] (www_rbac) Flask flash messages are not displayed properly (no background color) [AIRFLOW-3069] Decode output of S3 file transform operator +[AIRFLOW-3072] Assign permission get_logs_with_metadata to viewer role [AIRFLOW-3090] INFO logs are too verbose [AIRFLOW-3103] Update Flask-Login [AIRFLOW-3112] Align SFTP hook with SSH hook @@ -66,6 +68,8 @@ Doc-only changes: [AIRFLOW-3030] Command Line docs incorrect subdir [AIRFLOW-2990] Docstrings for Hooks/Operators are in incorrect format [AIRFLOW-3127] Celery SSL Documentation is out-dated +[AIRFLOW-2779] Add license headers to doc files +[AIRFLOW-2779] Add project version to license Bug fixes: diff --git a/airflow/version.py b/airflow/version.py index 1080bf44f6953..5521427e4c327 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.1b1' +version = '1.10.1' From 7b9911eb9a60ab5423b9f651654d2e066874b439 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 13 Nov 2018 14:01:44 +0000 Subject: [PATCH 197/808] [AIRFLOW-2779] Add license headers to doc files (#4178) This adds ASF license headers to all the .rst and .md files with the exception of the Pull Request template (as that is included verbatim when opening a Pull Request on Github which would be messy) --- .rat-excludes | 30 ++++++++++++++++++++++++------ docs/conf.py | 20 ++++++++++++++++++++ scripts/ci/check-license.sh | 24 ++++++------------------ 3 files changed, 50 insertions(+), 24 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index c9487dd8c49c1..05f7b98000f89 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -1,16 +1,21 @@ +# Note: these patterns are applied to single files or directories, not full paths +# coverage/* will ignore any coverage dir, but airflow/www/static/coverage/* will match nothing + .gitignore .gitattributes +.airflowignore .coverage .coveragerc .codecov.yml +.eslintrc +.eslintignore +.flake8 .rat-excludes requirements.txt .*log .travis.yml .*pyc .*lock -docs -.*md dist build airflow.egg-info @@ -18,16 +23,23 @@ apache_airflow.egg-info .idea metastore_db .*sql +.*svg .*csv CHANGELOG.txt .*zip .*lock +# Generated doc files +.*html +_build/* +_static/* +.buildinfo +searchindex.js + # Apache Rat does not detect BSD-2 clause properly # it is compatible according to http://www.apache.org/legal/resolved.html#category-a kerberos_auth.py airflow_api_auth_backend_kerberos_auth_py.html licenses/* -airflow/www/static/docs parallel.js underscore.js jquery.dataTables.min.js @@ -37,6 +49,12 @@ bootstrap-toggle.min.js bootstrap-toggle.min.css d3.v3.min.js ace.js -airflow/www/static/coverage/* -airflow/git_version -scripts/ci/flake8_diff.sh +node_modules/* +.*json +coverage/* +git_version +flake8_diff.sh + +rat-results.txt +apache-airflow-.*\+incubating-source.tar.gz.* +apache-airflow-.*\+incubating-bin.tar.gz.* diff --git a/docs/conf.py b/docs/conf.py index 8b5d8305bbb15..fa2a3d2274acb 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -1,5 +1,25 @@ # -*- coding: utf-8 -*- # +# flake8: noqa +# Disable Flake8 because of all the sphinx imports +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + # Airflow documentation build configuration file, created by # sphinx-quickstart on Thu Oct 9 20:50:01 2014. # diff --git a/scripts/ci/check-license.sh b/scripts/ci/check-license.sh index 83c942a0a4f12..035283f9061b5 100755 --- a/scripts/ci/check-license.sh +++ b/scripts/ci/check-license.sh @@ -73,6 +73,9 @@ mkdir -p ${TRAVIS_CACHE}/lib exit 1 } +# This is the target of a symlink in airflow/www/static/docs - and rat exclude doesn't cope with the symlink target doesn't exist +mkdir -p docs/_build/html/ + echo "Running license checks. This can take a while." $java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt @@ -84,24 +87,9 @@ fi ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then - echo "Could not find Apache license headers in the following files:" - echo "$ERRORS" - COUNT=`echo "${ERRORS}" | wc -l` - # due to old builds can be removed later - rm -rf ${TRAVIS_CACHE}/rat-error-count - if [ ! -f ${TRAVIS_CACHE}/rat-error-count-builds ]; then - [ "${TRAVIS_PULL_REQUEST}" = "false" ] && echo ${COUNT} > ${TRAVIS_CACHE}/rat-error-count-builds - OLD_COUNT=${COUNT} - else - typeset -i OLD_COUNT=$(cat ${TRAVIS_CACHE}/rat-error-count-builds) - fi - if [ ${COUNT} -gt ${OLD_COUNT} ]; then - echo "New missing licenses (${COUNT} vs ${OLD_COUNT}) detected. Please correct them by adding them to to header of your files" - exit 1 - else - [ "${TRAVIS_PULL_REQUEST}" = "false" ] && echo ${COUNT} > ${TRAVIS_CACHE}/rat-error-count-builds - fi - exit 0 + echo >&2 "Could not find Apache license headers in the following files:" + echo >&2 "$ERRORS" + exit 1 else echo -e "RAT checks passed." fi From 456584f7ad64c8d40d18aff0932fe92d84a0b2f9 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Tue, 13 Nov 2018 15:17:56 +0000 Subject: [PATCH 198/808] [AIRFLOW-XXX] Correct date and version in Changelog --- CHANGELOG.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index 92e62698132b1..f7b419177bbd2 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -1,4 +1,4 @@ -AIRFLOW 1.10.1b1, 2018-11-09 +AIRFLOW 1.10.1, 2018-11-13 New features: From 17034eed465c8b4ab1ddb25acfba15f8a94f6b75 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 14 Nov 2018 21:35:03 +0000 Subject: [PATCH 199/808] [AIRFLOW-3339] Correctly get DAG timezone when start_date in default_args (#4186) --- airflow/models.py | 3 ++- tests/models.py | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/airflow/models.py b/airflow/models.py index 1536b37df906f..81d179ba246a3 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -3249,7 +3249,8 @@ def __init__( timezone.parse(self.default_args['start_date']) ) self.timezone = self.default_args['start_date'].tzinfo - else: + + if not hasattr(self, 'timezone') or not self.timezone: self.timezone = settings.TIMEZONE self.start_date = timezone.convert_to_utc(start_date) diff --git a/tests/models.py b/tests/models.py index 0c34878ee5696..46b273ddd4665 100644 --- a/tests/models.py +++ b/tests/models.py @@ -214,6 +214,12 @@ def test_dag_topological_sort(self): self.assertEquals(tuple(), dag.topological_sort()) + def test_dag_naive_default_args_start_date(self): + dag = DAG('DAG', default_args={'start_date': datetime.datetime(2018, 1, 1)}) + self.assertEqual(dag.timezone, settings.TIMEZONE) + dag = DAG('DAG', start_date=datetime.datetime(2018, 1, 1)) + self.assertEqual(dag.timezone, settings.TIMEZONE) + def test_dag_none_default_args_start_date(self): """ Tests if a start_date of None in default_args From 31f6775007d5d025dba74753a58209488de58511 Mon Sep 17 00:00:00 2001 From: Bumsoo Kim Date: Wed, 14 Nov 2018 01:22:52 +0900 Subject: [PATCH 200/808] [AIRFLOW-XXX] Remove duplicated line in Changelog (#4181) --- CHANGELOG.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index f7b419177bbd2..030a8194cb18a 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -144,7 +144,6 @@ AIRFLOW 1.10.0, 2018-08-03 [AIRFLOW-2710] Clarify fernet key value in documentation [AIRFLOW-2606] Fix DB schema and SQLAlchemy model [AIRFLOW-2646] Fix setup.py not to install snakebite on Python3 -[AIRFLOW-2512][AIRFLOW-2522] Use google-auth instead of oauth2client [AIRFLOW-2604] Add index to task_fail [AIRFLOW-2650] Mark SchedulerJob as succeed when hitting Ctrl-c [AIRFLOW-2678] Fix db schema unit test to remove checking fab models From 52e00329aaf80fa8248f56405a2bf6abfa00da53 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 14 Nov 2018 22:03:45 +0000 Subject: [PATCH 201/808] [AIRFLOW-XXX] Include 3193 in the changelog --- CHANGELOG.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.txt b/CHANGELOG.txt index 030a8194cb18a..abb0563d71771 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -51,6 +51,7 @@ Improvements: [AIRFLOW-3137] Make ProxyFix middleware optional [AIRFLOW-3173] Add _cmd options for more password config options [AIRFLOW-3177] Change scheduler_heartbeat metric from gauge to counter +[AIRFLOW-3193] Pin docker requirement version to v3 [AIRFLOW-3195] Druid Hook: Log ingestion spec and task id [AIRFLOW-3197] EMR Hook is missing some parameters to valid on the AWS API [AIRFLOW-3232] Make documentation for GCF Functions operator more readable From 76a5fc4d2eb3c214ca25406f03b4a0c5d7250f71 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 14 Nov 2018 22:06:03 +0000 Subject: [PATCH 202/808] [AIRFLOW-3343] Update DockerOperator for Docker-py 3.0.0 API changes (#4187) The API of `wait()` changed to return a dict, not just a number so this Operator wasn't actually working, but the tests were passing because the return was mocked in-correctly. I also removed `shm_size` from kwargs passed to BaseOperator to avoid the deprecation warning about unknown args. --- airflow/operators/docker_operator.py | 14 +++++++++----- setup.py | 2 +- tests/operators/test_docker_operator.py | 12 ++++++------ 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py index 727507b270c42..04dbcab5691cf 100644 --- a/airflow/operators/docker_operator.py +++ b/airflow/operators/docker_operator.py @@ -104,6 +104,9 @@ class DockerOperator(BaseOperator): :type xcom_all: bool :param docker_conn_id: ID of the Airflow connection to use :type docker_conn_id: str + :param shm_size: Size of ``/dev/shm`` in bytes. The size must be + greater than 0. If omitted uses system default. + :type shm_size: int """ template_fields = ('command', 'environment',) template_ext = ('.sh', '.bash',) @@ -134,6 +137,7 @@ def __init__( docker_conn_id=None, dns=None, dns_search=None, + shm_size=None, *args, **kwargs): @@ -161,7 +165,7 @@ def __init__( self.xcom_push_flag = xcom_push self.xcom_all = xcom_all self.docker_conn_id = docker_conn_id - self.shm_size = kwargs.get('shm_size') + self.shm_size = shm_size self.cli = None self.container = None @@ -191,7 +195,7 @@ def execute(self, context): if self.force_pull or len(self.cli.images(name=self.image)) == 0: self.log.info('Pulling docker image %s', self.image) for l in self.cli.pull(self.image, stream=True): - output = json.loads(l.decode('utf-8')) + output = json.loads(l.decode('utf-8').strip()) if 'status' in output: self.log.info("%s", output['status']) @@ -223,9 +227,9 @@ def execute(self, context): line = line.decode('utf-8') self.log.info(line) - exit_code = self.cli.wait(self.container['Id']) - if exit_code != 0: - raise AirflowException('docker container failed') + result = self.cli.wait(self.container['Id']) + if result['StatusCode'] != 0: + raise AirflowException('docker container failed: ' + repr(result)) if self.xcom_push_flag: return self.cli.logs(container=self.container['Id']) \ diff --git a/setup.py b/setup.py index fca659689958c..ee3935b76c271 100644 --- a/setup.py +++ b/setup.py @@ -157,7 +157,7 @@ def write_version(filename=os.path.join(*['airflow', 'sphinx-rtd-theme>=0.1.6', 'Sphinx-PyPI-upload>=0.2.1' ] -docker = ['docker>=3.0.0'] +docker = ['docker~=3.0'] druid = ['pydruid>=0.4.1'] elasticsearch = [ 'elasticsearch>=5.0.0,<6.0.0', diff --git a/tests/operators/test_docker_operator.py b/tests/operators/test_docker_operator.py index d57a7cd132a16..b9154456b9b31 100644 --- a/tests/operators/test_docker_operator.py +++ b/tests/operators/test_docker_operator.py @@ -51,7 +51,7 @@ def test_execute(self, client_class_mock, mkdtemp_mock): client_mock.images.return_value = [] client_mock.logs.return_value = ['container log'] client_mock.pull.return_value = [b'{"status":"pull log"}'] - client_mock.wait.return_value = 0 + client_mock.wait.return_value = {"StatusCode": 0} client_class_mock.return_value = client_mock @@ -96,7 +96,7 @@ def test_execute_tls(self, client_class_mock, tls_class_mock): client_mock.images.return_value = [] client_mock.logs.return_value = [] client_mock.pull.return_value = [] - client_mock.wait.return_value = 0 + client_mock.wait.return_value = {"StatusCode": 0} client_class_mock.return_value = client_mock tls_mock = mock.Mock() @@ -122,7 +122,7 @@ def test_execute_unicode_logs(self, client_class_mock): client_mock.images.return_value = [] client_mock.logs.return_value = ['unicode container log 😁'] client_mock.pull.return_value = [] - client_mock.wait.return_value = 0 + client_mock.wait.return_value = {"StatusCode": 0} client_class_mock.return_value = client_mock @@ -144,7 +144,7 @@ def test_execute_container_fails(self, client_class_mock): client_mock.images.return_value = [] client_mock.logs.return_value = [] client_mock.pull.return_value = [] - client_mock.wait.return_value = 1 + client_mock.wait.return_value = {"StatusCode": 1} client_class_mock.return_value = client_mock @@ -172,7 +172,7 @@ def test_execute_no_docker_conn_id_no_hook(self, operator_client_mock): client_mock.create_container.return_value = {'Id': 'some_id'} client_mock.logs.return_value = [] client_mock.pull.return_value = [] - client_mock.wait.return_value = 0 + client_mock.wait.return_value = {"StatusCode": 0} operator_client_mock.return_value = client_mock # Create the DockerOperator @@ -207,7 +207,7 @@ def test_execute_with_docker_conn_id_use_hook(self, operator_client_mock, client_mock.create_container.return_value = {'Id': 'some_id'} client_mock.logs.return_value = [] client_mock.pull.return_value = [] - client_mock.wait.return_value = 0 + client_mock.wait.return_value = {"StatusCode": 0} operator_client_mock.return_value = client_mock # Create the DockerOperator From 2ff994fd9efca5243933559a0bedac784121c1b5 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 9 Oct 2018 23:01:58 +0800 Subject: [PATCH 203/808] [AIRFLOW-3175] Fix docstring format in airflow/jobs.py (#4025) These docstrings could not parsed properly in Sphinx syntax --- airflow/jobs.py | 35 +++++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index 03610bd983607..1faf9e2d034b7 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -349,10 +349,10 @@ def _launch_process(result_queue, :param file_path: the file to process :type file_path: unicode :param pickle_dags: whether to pickle the DAGs found in the file and - save them to the DB + save them to the DB :type pickle_dags: bool :param dag_id_white_list: if specified, only examine DAG ID's that are - in this list + in this list :type dag_id_white_list: list[unicode] :param thread_name: the name to use for the process that is launched :type thread_name: unicode @@ -424,6 +424,7 @@ def start(self): def terminate(self, sigkill=False): """ Terminate (and then kill) the process launched to process the file. + :param sigkill: whether to issue a SIGKILL if SIGTERM doesn't work. :type sigkill: bool """ @@ -452,6 +453,7 @@ def pid(self): def exit_code(self): """ After the process is finished, this can be called to get the return code + :return: the exit code of the process :rtype: int """ @@ -463,6 +465,7 @@ def exit_code(self): def done(self): """ Check if the process launched to process this file is done. + :return: whether the process is finished running :rtype: bool """ @@ -544,16 +547,18 @@ def __init__( :param dag_ids: if specified, only schedule tasks with these DAG IDs :type dag_ids: list[unicode] :param subdir: directory containing Python files with Airflow DAG - definitions, or a specific path to a file + definitions, or a specific path to a file :type subdir: unicode :param num_runs: The number of times to try to schedule each DAG file. - -1 for unlimited within the run_duration. + -1 for unlimited within the run_duration. + :type num_runs: int :param processor_poll_interval: The number of seconds to wait between - polls of running processors + polls of running processors + :type processor_poll_interval: int :param run_duration: how long to run (in seconds) before exiting :type run_duration: int :param do_pickle: once a DAG object is obtained by executing the Python - file, whether to serialize the DAG object to the DB + file, whether to serialize the DAG object to the DB :type do_pickle: bool """ # for BaseJob compatibility @@ -783,7 +788,7 @@ def update_import_errors(session, dagbag): def create_dag_run(self, dag, session=None): """ This method checks whether a new DagRun needs to be created - for a DAG based on scheduling interval + for a DAG based on scheduling interval. Returns DagRun if one is scheduled. Otherwise returns None. """ if dag.schedule_interval: @@ -991,7 +996,7 @@ def _change_state_for_tis_without_dagrun(self, :param new_state: set TaskInstances to this state :type new_state: State :param simple_dag_bag: TaskInstances associated with DAGs in the - simple_dag_bag and with states in the old_state will be examined + simple_dag_bag and with states in the old_state will be examined :type simple_dag_bag: SimpleDagBag """ tis_changed = 0 @@ -1062,7 +1067,7 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): dag concurrency, executor state, and priority. :param simple_dag_bag: TaskInstances associated with DAGs in the - simple_dag_bag will be fetched from the DB and executed + simple_dag_bag will be fetched from the DB and executed :type simple_dag_bag: SimpleDagBag :param executor: the executor that runs task instances :type executor: BaseExecutor @@ -1373,7 +1378,7 @@ def _execute_task_instances(self, 3. Enqueue the TIs in the executor. :param simple_dag_bag: TaskInstances associated with DAGs in the - simple_dag_bag will be fetched from the DB and executed + simple_dag_bag will be fetched from the DB and executed :type simple_dag_bag: SimpleDagBag :param states: Execute TaskInstances in these states :type states: Tuple[State] @@ -1482,7 +1487,7 @@ def _log_file_processing_stats(self, Print out stats about how files are getting processed. :param known_file_paths: a list of file paths that may contain Airflow - DAG definitions + DAG definitions :type known_file_paths: list[unicode] :param processor_manager: manager for the file processors :type stats: DagFileProcessorManager @@ -1788,7 +1793,7 @@ def process_file(self, file_path, pickle_dags=False, session=None): :param file_path: the path to the Python file that should be executed :type file_path: unicode :param pickle_dags: whether serialize the DAGs found in the file and - save them to the db + save them to the db :type pickle_dags: bool :return: a list of SimpleDags made from the Dags found in the file :rtype: list[SimpleDag] @@ -2027,6 +2032,7 @@ def _update_counters(self, ti_status): """ Updates the counters per state of the tasks that were running. Can re-add to tasks to run in case required. + :param ti_status: the internal status of the backfill job tasks :type ti_status: BackfillJob._DagRunTaskStatus """ @@ -2071,6 +2077,7 @@ def _manage_executor_state(self, running): """ Checks if the executor agrees with the state of task instances that are running + :param running: dict of key, task to verify """ executor = self.executor @@ -2102,6 +2109,7 @@ def _get_dag_run(self, run_date, session=None): Returns a dag run for the given run date, which will be matched to an existing dag run if available or create a new dag run otherwise. If the max_active_runs limit is reached, this function will return None. + :param run_date: the execution date for the dag run :type run_date: datetime :param session: the database session object @@ -2161,6 +2169,7 @@ def _task_instances_for_dag_run(self, dag_run, session=None): """ Returns a map of task instance key to task instance object for the tasks to run in the given dag run. + :param dag_run: the dag run to get the tasks from :type dag_run: models.DagRun :param session: the database session object @@ -2226,6 +2235,7 @@ def _process_backfill_task_instances(self, Process a set of task instances from a set of dag runs. Special handling is done to account for different task instance states that could be present when running them in a backfill process. + :param ti_status: the internal status of the job :type ti_status: BackfillJob._DagRunTaskStatus :param executor: the executor to run the task instances @@ -2463,6 +2473,7 @@ def _execute_for_run_dates(self, run_dates, ti_status, executor, pickle_id, Computes the dag runs and their respective task instances for the given run dates and executes the task instances. Returns a list of execution dates of the dag runs that were executed. + :param run_dates: Execution dates for dag runs :type run_dates: list :param ti_status: internal BackfillJob status structure to tis track progress From 0c28b60dfba53eccf69fc99cab25c57d6aded7ad Mon Sep 17 00:00:00 2001 From: Jacob Greenfield Date: Wed, 3 Oct 2018 18:16:20 -0400 Subject: [PATCH 204/808] [AIRFLOW-3004] Add config disabling scheduler cron (#3899) --- airflow/config_templates/default_airflow.cfg | 4 ++++ airflow/jobs.py | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 07a9458073dae..ae09f26adec38 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -480,6 +480,10 @@ max_threads = 2 authenticate = False +# Turn off scheduler use of cron intervals by setting this to False. +# DAGs submitted manually in the web UI or with trigger_dag will still run. +use_job_schedule = True + [ldap] # set this to ldaps://: uri = diff --git a/airflow/jobs.py b/airflow/jobs.py index 1faf9e2d034b7..e94eedef8d686 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -791,7 +791,7 @@ def create_dag_run(self, dag, session=None): for a DAG based on scheduling interval. Returns DagRun if one is scheduled. Otherwise returns None. """ - if dag.schedule_interval: + if dag.schedule_interval and conf.getboolean('scheduler', 'USE_JOB_SCHEDULE'): active_runs = DagRun.find( dag_id=dag.dag_id, state=State.RUNNING, From 5cb7fa599f81d1545c98821bfa861b447495ccbb Mon Sep 17 00:00:00 2001 From: Chengzhi Zhao Date: Thu, 9 Aug 2018 21:24:48 -0400 Subject: [PATCH 205/808] [AIRFLOW-2882] Add import and export for pool cli using JSON --- airflow/bin/cli.py | 56 +++++++++++++++++++++++++++++++++++++++++++++- tests/core.py | 36 +++++++++++++++++++++++++++++ 2 files changed, 91 insertions(+), 1 deletion(-) diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py index 2940d2e1232d1..1b0bf936279f0 100644 --- a/airflow/bin/cli.py +++ b/airflow/bin/cli.py @@ -271,6 +271,7 @@ def _tabulate(pools): tablefmt="fancy_grid") try: + imp = getattr(args, 'import') if args.get is not None: pools = [api_client.get_pool(name=args.get)] elif args.set: @@ -279,6 +280,14 @@ def _tabulate(pools): description=args.set[2])] elif args.delete: pools = [api_client.delete_pool(name=args.delete)] + elif imp: + if os.path.exists(imp): + pools = pool_import_helper(imp) + else: + print("Missing pools file.") + pools = api_client.get_pools() + elif args.export: + pools = pool_export_helper(args.export) else: pools = api_client.get_pools() except (AirflowException, IOError) as err: @@ -287,6 +296,43 @@ def _tabulate(pools): log.info(_tabulate(pools=pools)) +def pool_import_helper(filepath): + with open(filepath, 'r') as poolfile: + pl = poolfile.read() + try: + d = json.loads(pl) + except Exception as e: + print("Please check the validity of the json file: " + str(e)) + else: + try: + pools = [] + n = 0 + for k, v in d.items(): + if isinstance(v, dict) and len(v) == 2: + pools.append(api_client.create_pool(name=k, + slots=v["slots"], + description=v["description"])) + n += 1 + else: + pass + except Exception: + pass + finally: + print("{} of {} pool(s) successfully updated.".format(n, len(d))) + return pools + + +def pool_export_helper(filepath): + pool_dict = {} + pools = api_client.get_pools() + for pool in pools: + pool_dict[pool[0]] = {"slots": pool[1], "description": pool[2]} + with open(filepath, 'w') as poolfile: + poolfile.write(json.dumps(pool_dict, sort_keys=True, indent=4)) + print("{} pools successfully exported to {}".format(len(pool_dict), filepath)) + return pools + + @cli_utils.action_logging def variables(args): if args.get: @@ -1448,6 +1494,14 @@ class CLIFactory(object): ("-x", "--delete"), metavar="NAME", help="Delete a pool"), + 'pool_import': Arg( + ("-i", "--import"), + metavar="FILEPATH", + help="Import pool from JSON file"), + 'pool_export': Arg( + ("-e", "--export"), + metavar="FILEPATH", + help="Export pool to JSON file"), # variables 'set': Arg( ("-s", "--set"), @@ -1762,7 +1816,7 @@ class CLIFactory(object): }, { 'func': pool, 'help': "CRUD operations on pools", - "args": ('pool_set', 'pool_get', 'pool_delete'), + "args": ('pool_set', 'pool_get', 'pool_delete', 'pool_import', 'pool_export'), }, { 'func': variables, 'help': "CRUD operations on variables", diff --git a/tests/core.py b/tests/core.py index f466c4c4f94db..60bfee69df7dd 100644 --- a/tests/core.py +++ b/tests/core.py @@ -1430,6 +1430,42 @@ def test_pool_no_args(self): except Exception as e: self.fail("The 'pool' command raised unexpectedly: %s" % e) + def test_pool_import_export(self): + # Create two pools first + pool_config_input = { + "foo": { + "description": "foo_test", + "slots": 1 + }, + "baz": { + "description": "baz_test", + "slots": 2 + } + } + with open('pools_import.json', mode='w') as f: + json.dump(pool_config_input, f) + + # Import json + try: + cli.pool(self.parser.parse_args(['pool', '-i', 'pools_import.json'])) + except Exception as e: + self.fail("The 'pool -i pools_import.json' failed: %s" % e) + + # Export json + try: + cli.pool(self.parser.parse_args(['pool', '-e', 'pools_export.json'])) + except Exception as e: + self.fail("The 'pool -e pools_export.json' failed: %s" % e) + + with open('pools_export.json', mode='r') as f: + pool_config_output = json.load(f) + self.assertEqual( + pool_config_input, + pool_config_output, + "Input and output pool files are not same") + os.remove('pools_import.json') + os.remove('pools_export.json') + def test_variables(self): # Checks if all subcommands are properly received cli.variables(self.parser.parse_args([ From a521266851d450bbea97d7e9d0d22247573d676d Mon Sep 17 00:00:00 2001 From: XD-DENG Date: Sat, 28 Jul 2018 20:48:17 +0100 Subject: [PATCH 206/808] [AIRFLOW-2814] Fix inconsistent default config value of min_file_process_interval in config template is 0 However it's supposed to be 180 according to airflow/jobs.py line 592 Closes #3659 from XD-DENG/patch-3 --- airflow/config_templates/default_airflow.cfg | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index ae09f26adec38..150ac5673aba5 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -432,8 +432,8 @@ scheduler_heartbeat_sec = 5 # -1 indicates to run continuously (see also num_runs) run_duration = -1 -# after how much time a new DAGs should be picked up from the filesystem -min_file_process_interval = 0 +# after how much time (seconds) a new DAGs should be picked up from the filesystem +min_file_process_interval = 180 # How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes. dag_dir_list_interval = 300 From 2d386cd4aef0cb795fc374b7cf31c5a898cec96d Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 3 Aug 2018 09:27:06 +0100 Subject: [PATCH 207/808] Revert [AIRFLOW-2814] - Change `min_file_process_interval` to 0 (#3669) - Change the time (in seconds) after which a new DAG should be picked up from the filesystem --- airflow/config_templates/default_airflow.cfg | 2 +- airflow/jobs.py | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 150ac5673aba5..d8a05e5c8cd84 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -433,7 +433,7 @@ scheduler_heartbeat_sec = 5 run_duration = -1 # after how much time (seconds) a new DAGs should be picked up from the filesystem -min_file_process_interval = 180 +min_file_process_interval = 0 # How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes. dag_dir_list_interval = 300 diff --git a/airflow/jobs.py b/airflow/jobs.py index e94eedef8d686..4d74648bbab55 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -596,8 +596,7 @@ def __init__( # 30 seconds. self.print_stats_interval = conf.getint('scheduler', 'print_stats_interval') - # Parse and schedule each file no faster than this interval. Default - # to 3 minutes. + self.file_process_interval = file_process_interval self.max_tis_per_query = conf.getint('scheduler', 'max_tis_per_query') From e43c14a0a2987f8616bec86cd533c24c6c34aaff Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Mon, 1 Oct 2018 06:15:20 -0400 Subject: [PATCH 208/808] [AIRFLOW-3129] Improve test coverage of airflow.models. (#3982) --- airflow/models.py | 25 ------- tests/models.py | 181 +++++++++++++++++++++++++++++++++++++++++++++- 2 files changed, 180 insertions(+), 26 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index 81d179ba246a3..0a03ea979223a 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -594,21 +594,6 @@ def dagbag_report(self): table=pprinttable(stats), ) - @provide_session - def deactivate_inactive_dags(self, session=None): - active_dag_ids = [dag.dag_id for dag in list(self.dags.values())] - for dag in session.query( - DagModel).filter(~DagModel.dag_id.in_(active_dag_ids)).all(): - dag.is_active = False - session.merge(dag) - session.commit() - - @provide_session - def paused_dags(self, session=None): - dag_ids = [dp.dag_id for dp in session.query(DagModel).filter( - DagModel.is_paused.__eq__(True))] - return dag_ids - class User(Base): __tablename__ = "users" @@ -4135,16 +4120,6 @@ def add_tasks(self, tasks): for task in tasks: self.add_task(task) - @provide_session - def db_merge(self, session=None): - BO = BaseOperator - tasks = session.query(BO).filter(BO.dag_id == self.dag_id).all() - for t in tasks: - session.delete(t) - session.commit() - session.merge(self) - session.commit() - def run( self, start_date=None, diff --git a/tests/models.py b/tests/models.py index 46b273ddd4665..8375a95a30760 100644 --- a/tests/models.py +++ b/tests/models.py @@ -44,17 +44,20 @@ from airflow.models import clear_task_instances from airflow.models import XCom from airflow.models import Connection +from airflow.models import SkipMixin +from airflow.models import KubeResourceVersion, KubeWorkerIdentifier from airflow.jobs import LocalTaskJob from airflow.operators.dummy_operator import DummyOperator from airflow.operators.bash_operator import BashOperator from airflow.operators.python_operator import PythonOperator from airflow.operators.python_operator import ShortCircuitOperator +from airflow.operators.subdag_operator import SubDagOperator from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep from airflow.utils import timezone from airflow.utils.weight_rule import WeightRule from airflow.utils.state import State from airflow.utils.trigger_rule import TriggerRule -from mock import patch, ANY +from mock import patch, Mock, ANY from parameterized import parameterized from tempfile import mkdtemp, NamedTemporaryFile @@ -640,6 +643,38 @@ def test_following_previous_schedule_daily_dag_CET_to_CEST(self): self.assertEqual(prev_local.isoformat(), "2018-03-24T03:00:00+01:00") self.assertEqual(prev.isoformat(), "2018-03-24T02:00:00+00:00") + @patch('airflow.models.timezone.utcnow') + def test_sync_to_db(self, mock_now): + dag = DAG( + 'dag', + start_date=DEFAULT_DATE, + ) + with dag: + DummyOperator(task_id='task', owner='owner1') + SubDagOperator( + task_id='subtask', + owner='owner2', + subdag=DAG( + 'dag.subtask', + start_date=DEFAULT_DATE, + ) + ) + now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) + mock_now.return_value = now + session = settings.Session() + dag.sync_to_db(session=session) + + orm_dag = session.query(DagModel).filter(DagModel.dag_id == 'dag').one() + self.assertEqual(set(orm_dag.owners.split(', ')), {'owner1', 'owner2'}) + self.assertEqual(orm_dag.last_scheduler_run, now) + self.assertTrue(orm_dag.is_active) + + orm_subdag = session.query(DagModel).filter( + DagModel.dag_id == 'dag.subtask').one() + self.assertEqual(set(orm_subdag.owners.split(', ')), {'owner1', 'owner2'}) + self.assertEqual(orm_subdag.last_scheduler_run, now) + self.assertTrue(orm_subdag.is_active) + class DagStatTest(unittest.TestCase): def test_dagstats_crud(self): @@ -690,6 +725,25 @@ def test_dagstats_crud(self): for stat in res: self.assertFalse(stat.dirty) + def test_update_exception(self): + session = Mock() + (session.query.return_value + .filter.return_value + .with_for_update.return_value + .all.side_effect) = RuntimeError('it broke') + DagStat.update(session=session) + session.rollback.assert_called() + + def test_set_dirty_exception(self): + session = Mock() + session.query.return_value.filter.return_value.all.return_value = [] + (session.query.return_value + .filter.return_value + .with_for_update.return_value + .all.side_effect) = RuntimeError('it broke') + DagStat.set_dirty('dag', session) + session.rollback.assert_called() + class DagRunTest(unittest.TestCase): @@ -2465,6 +2519,35 @@ def success_handler(self, context): ti.refresh_from_db() self.assertEqual(ti.state, State.SUCCESS) + @patch('airflow.models.send_email') + def test_email_alert(self, mock_send_email): + task = DummyOperator(task_id='op', email='test@test.test') + ti = TI(task=task, execution_date=datetime.datetime.now()) + ti.email_alert(RuntimeError('it broke')) + + self.assertTrue(mock_send_email.called) + (email, title, body), _ = mock_send_email.call_args + self.assertEqual(email, 'test@test.test') + self.assertIn(repr(ti), title) + self.assertIn('it broke', body) + + def test_set_duration(self): + task = DummyOperator(task_id='op', email='test@test.test') + ti = TI( + task=task, + execution_date=datetime.datetime.now(), + ) + ti.start_date = datetime.datetime(2018, 10, 1, 1) + ti.end_date = datetime.datetime(2018, 10, 1, 2) + ti.set_duration() + self.assertEqual(ti.duration, 3600) + + def test_set_duration_empty_dates(self): + task = DummyOperator(task_id='op', email='test@test.test') + ti = TI(task=task, execution_date=datetime.datetime.now()) + ti.set_duration() + self.assertIsNone(ti.duration) + class ClearTasksTest(unittest.TestCase): @@ -2819,3 +2902,99 @@ def test_connection_from_uri_with_extras(self): self.assertEqual(connection.port, 1234) self.assertDictEqual(connection.extra_dejson, {'extra1': 'a value', 'extra2': '/path/'}) + + +class TestSkipMixin(unittest.TestCase): + + @patch('airflow.models.timezone.utcnow') + def test_skip(self, mock_now): + session = settings.Session() + now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) + mock_now.return_value = now + dag = DAG( + 'dag', + start_date=DEFAULT_DATE, + ) + with dag: + tasks = [DummyOperator(task_id='task')] + dag_run = dag.create_dagrun( + run_id='manual__' + now.isoformat(), + state=State.FAILED, + ) + SkipMixin().skip( + dag_run=dag_run, + execution_date=now, + tasks=tasks, + session=session) + + session.query(TI).filter( + TI.dag_id == 'dag', + TI.task_id == 'task', + TI.state == State.SKIPPED, + TI.start_date == now, + TI.end_date == now, + ).one() + + @patch('airflow.models.timezone.utcnow') + def test_skip_none_dagrun(self, mock_now): + session = settings.Session() + now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) + mock_now.return_value = now + dag = DAG( + 'dag', + start_date=DEFAULT_DATE, + ) + with dag: + tasks = [DummyOperator(task_id='task')] + SkipMixin().skip( + dag_run=None, + execution_date=now, + tasks=tasks, + session=session) + + session.query(TI).filter( + TI.dag_id == 'dag', + TI.task_id == 'task', + TI.state == State.SKIPPED, + TI.start_date == now, + TI.end_date == now, + ).one() + + def test_skip_none_tasks(self): + session = Mock() + SkipMixin().skip(dag_run=None, execution_date=None, tasks=[], session=session) + self.assertFalse(session.query.called) + self.assertFalse(session.commit.called) + + +class TestKubeResourceVersion(unittest.TestCase): + + def test_checkpoint_resource_version(self): + session = settings.Session() + KubeResourceVersion.checkpoint_resource_version('7', session) + self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '7') + + def test_reset_resource_version(self): + session = settings.Session() + version = KubeResourceVersion.reset_resource_version(session) + self.assertEqual(version, '0') + self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '0') + + +class TestKubeWorkerIdentifier(unittest.TestCase): + + @patch('airflow.models.uuid.uuid4') + def test_get_or_create_not_exist(self, mock_uuid): + session = settings.Session() + session.query(KubeWorkerIdentifier).update({ + KubeWorkerIdentifier.worker_uuid: '' + }) + mock_uuid.return_value = 'abcde' + worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) + self.assertEqual(worker_uuid, 'abcde') + + def test_get_or_create_exist(self): + session = settings.Session() + KubeWorkerIdentifier.checkpoint_kube_worker_uuid('fghij', session) + worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) + self.assertEqual(worker_uuid, 'fghij') From 2a37694121b4646e055ded5a7d56fccea232fbe5 Mon Sep 17 00:00:00 2001 From: Kevin Yang Date: Fri, 26 Oct 2018 01:37:10 -0700 Subject: [PATCH 209/808] [Airflow-2760] Decouple DAG parsing loop from scheduler loop (#3873) --- UPDATING.md | 10 + .../airflow_local_settings.py | 43 +- airflow/config_templates/default_airflow.cfg | 1 + airflow/config_templates/default_test.cfg | 1 + airflow/executors/base_executor.py | 2 +- airflow/jobs.py | 378 ++++------ airflow/models.py | 53 +- airflow/utils/dag_processing.py | 676 +++++++++++++++++- docs/img/scheduler_loop.jpg | Bin 0 -> 46864 bytes docs/scheduler.rst | 5 +- tests/jobs.py | 148 ++-- tests/models.py | 78 +- tests/utils/test_dag_processing.py | 137 +++- 13 files changed, 1108 insertions(+), 424 deletions(-) create mode 100644 docs/img/scheduler_loop.jpg diff --git a/UPDATING.md b/UPDATING.md index 8336d73cc9337..618888ea95a29 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -24,6 +24,16 @@ assists users migrating to a new version. ## Airflow 1.10.1 +### New `dag_processor_manager_log_location` config option + +The DAG parsing manager log now by default will be log into a file, where its location is +controlled by the new `dag_processor_manager_log_location` config option in core section. + +### new `sync_parallelism` config option in celery section + +The new `sync_parallelism` config option will control how many processes CeleryExecutor will use to +fetch celery task state in parallel. Default value is max(1, number of cores - 1) + ### StatsD Metrics The `scheduler_heartbeat` metric has been changed from a gauge to a counter. Each loop of the scheduler will increment the counter by 1. This provides a higher degree of visibility and allows for better integration with Prometheus using the [StatsD Exporter](https://github.com/prometheus/statsd_exporter). Scheduler upness can be determined by graphing and alerting using a rate. If the scheduler goes down, the rate will drop to 0. diff --git a/airflow/config_templates/airflow_local_settings.py b/airflow/config_templates/airflow_local_settings.py index 95150ab3bbbef..45a2f2923c058 100644 --- a/airflow/config_templates/airflow_local_settings.py +++ b/airflow/config_templates/airflow_local_settings.py @@ -20,6 +20,7 @@ import os from airflow import configuration as conf +from airflow.utils.file import mkdirs # TODO: Logging format and level should be configured # in this file instead of from airflow.cfg. Currently @@ -38,7 +39,11 @@ PROCESSOR_LOG_FOLDER = conf.get('scheduler', 'CHILD_PROCESS_LOG_DIRECTORY') +DAG_PROCESSOR_MANAGER_LOG_LOCATION = \ + conf.get('core', 'DAG_PROCESSOR_MANAGER_LOG_LOCATION') + FILENAME_TEMPLATE = conf.get('core', 'LOG_FILENAME_TEMPLATE') + PROCESSOR_FILENAME_TEMPLATE = conf.get('core', 'LOG_PROCESSOR_FILENAME_TEMPLATE') # Storage bucket url for remote logging @@ -79,7 +84,7 @@ 'formatter': 'airflow', 'base_log_folder': os.path.expanduser(PROCESSOR_LOG_FOLDER), 'filename_template': PROCESSOR_FILENAME_TEMPLATE, - }, + } }, 'loggers': { 'airflow.processor': { @@ -104,6 +109,26 @@ } } +DEFAULT_DAG_PARSING_LOGGING_CONFIG = { + 'handlers': { + 'processor_manager': { + 'class': 'logging.handlers.RotatingFileHandler', + 'formatter': 'airflow', + 'filename': DAG_PROCESSOR_MANAGER_LOG_LOCATION, + 'mode': 'a', + 'maxBytes': 104857600, # 100MB + 'backupCount': 5 + } + }, + 'loggers': { + 'airflow.processor_manager': { + 'handlers': ['processor_manager'], + 'level': LOG_LEVEL, + 'propagate': False, + } + } +} + REMOTE_HANDLERS = { 's3': { 'task': { @@ -172,6 +197,22 @@ REMOTE_LOGGING = conf.get('core', 'remote_logging') +# Only update the handlers and loggers when CONFIG_PROCESSOR_MANAGER_LOGGER is set. +# This is to avoid exceptions when initializing RotatingFileHandler multiple times +# in multiple processes. +if os.environ.get('CONFIG_PROCESSOR_MANAGER_LOGGER') == 'True': + DEFAULT_LOGGING_CONFIG['handlers'] \ + .update(DEFAULT_DAG_PARSING_LOGGING_CONFIG['handlers']) + DEFAULT_LOGGING_CONFIG['loggers'] \ + .update(DEFAULT_DAG_PARSING_LOGGING_CONFIG['loggers']) + + # Manually create log directory for processor_manager handler as RotatingFileHandler + # will only create file but not the directory. + processor_manager_handler_config = DEFAULT_DAG_PARSING_LOGGING_CONFIG['handlers'][ + 'processor_manager'] + directory = os.path.dirname(processor_manager_handler_config['filename']) + mkdirs(directory, 0o755) + if REMOTE_LOGGING and REMOTE_BASE_LOG_FOLDER.startswith('s3://'): DEFAULT_LOGGING_CONFIG['handlers'].update(REMOTE_HANDLERS['s3']) elif REMOTE_LOGGING and REMOTE_BASE_LOG_FOLDER.startswith('gs://'): diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index d8a05e5c8cd84..01d975d5ab6cb 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -68,6 +68,7 @@ simple_log_format = %%(asctime)s %%(levelname)s - %%(message)s # Log filename format log_filename_template = {{{{ ti.dag_id }}}}/{{{{ ti.task_id }}}}/{{{{ ts }}}}/{{{{ try_number }}}}.log log_processor_filename_template = {{{{ filename }}}}.log +dag_processor_manager_log_location = {AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log # Hostname by providing a path to a callable, which will resolve the hostname hostname_callable = socket:getfqdn diff --git a/airflow/config_templates/default_test.cfg b/airflow/config_templates/default_test.cfg index 9dc1ef94ec044..7d8347efa2341 100644 --- a/airflow/config_templates/default_test.cfg +++ b/airflow/config_templates/default_test.cfg @@ -39,6 +39,7 @@ logging_level = INFO fab_logging_level = WARN log_filename_template = {{{{ ti.dag_id }}}}/{{{{ ti.task_id }}}}/{{{{ ts }}}}/{{{{ try_number }}}}.log log_processor_filename_template = {{{{ filename }}}}.log +dag_processor_manager_log_location = {AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log executor = SequentialExecutor sql_alchemy_conn = sqlite:///{AIRFLOW_HOME}/unittests.db load_examples = True diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 3522f7c449790..25bfe3cd91064 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -141,7 +141,7 @@ def heartbeat(self): queue=queue, executor_config=ti.executor_config) else: - self.logger.info( + self.log.info( 'Task is already running, not sending to ' 'executor: {}'.format(key)) diff --git a/airflow/jobs.py b/airflow/jobs.py index 4d74648bbab55..48726733982ff 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -26,22 +26,18 @@ import logging import multiprocessing import os -import psutil import signal -import six import sys import threading import time -import datetime - from collections import defaultdict +from time import sleep + +import six from past.builtins import basestring -from sqlalchemy import ( - Column, Integer, String, func, Index, or_, and_, not_) +from sqlalchemy import (Column, Index, Integer, String, and_, func, not_, or_) from sqlalchemy.exc import OperationalError from sqlalchemy.orm.session import make_transient -from tabulate import tabulate -from time import sleep from airflow import configuration as conf from airflow import executors, models, settings @@ -53,16 +49,16 @@ from airflow.utils import asciiart, helpers, timezone from airflow.utils.configuration import tmp_configuration_copy from airflow.utils.dag_processing import (AbstractDagFileProcessor, - DagFileProcessorManager, + DagFileProcessorAgent, SimpleDag, SimpleDagBag, list_py_file_paths) from airflow.utils.db import create_session, provide_session -from airflow.utils.email import send_email, get_email_address_list -from airflow.utils.log.logging_mixin import LoggingMixin, set_context, StreamLogWriter +from airflow.utils.email import get_email_address_list, send_email +from airflow.utils.log.logging_mixin import LoggingMixin, StreamLogWriter, set_context from airflow.utils.net import get_hostname -from airflow.utils.state import State from airflow.utils.sqlalchemy import UtcDateTime +from airflow.utils.state import State Base = models.Base ID_LEN = models.ID_LEN @@ -304,7 +300,7 @@ class DagFileProcessor(AbstractDagFileProcessor, LoggingMixin): # Counter that increments everytime an instance of this class is created class_creation_counter = 0 - def __init__(self, file_path, pickle_dags, dag_id_white_list): + def __init__(self, file_path, pickle_dags, dag_id_white_list, zombies): """ :param file_path: a Python file containing Airflow DAG definitions :type file_path: unicode @@ -312,6 +308,8 @@ def __init__(self, file_path, pickle_dags, dag_id_white_list): :type pickle_dags: bool :param dag_id_whitelist: If specified, only look at these DAG ID's :type dag_id_whitelist: list[unicode] + :param zombies: zombie task instances to kill + :type zombies: list[SimpleTaskInstance] """ self._file_path = file_path # Queue that's used to pass results from the child process. @@ -320,6 +318,7 @@ def __init__(self, file_path, pickle_dags, dag_id_white_list): self._process = None self._dag_id_white_list = dag_id_white_list self._pickle_dags = pickle_dags + self._zombies = zombies # The result of Scheduler.process_file(file_path). self._result = None # Whether the process is done running. @@ -340,7 +339,8 @@ def _launch_process(result_queue, file_path, pickle_dags, dag_id_white_list, - thread_name): + thread_name, + zombies): """ Launch a process to process the given file. @@ -358,6 +358,8 @@ def _launch_process(result_queue, :type thread_name: unicode :return: the process that was launched :rtype: multiprocessing.Process + :param zombies: zombie task instances to kill + :type zombies: list[SimpleTaskInstance] """ def helper(): # This helper runs in the newly created process @@ -386,6 +388,7 @@ def helper(): os.getpid(), file_path) scheduler_job = SchedulerJob(dag_ids=dag_id_white_list, log=log) result = scheduler_job.process_file(file_path, + zombies, pickle_dags) result_queue.put(result) end_time = time.time() @@ -418,7 +421,8 @@ def start(self): self.file_path, self._pickle_dags, self._dag_id_white_list, - "DagFileProcessor{}".format(self._instance_id)) + "DagFileProcessor{}".format(self._instance_id), + self._zombies) self._start_time = timezone.utcnow() def terminate(self, sigkill=False): @@ -475,7 +479,8 @@ def done(self): if self._done: return True - if not self._result_queue.empty(): + # In case result queue is corrupted. + if self._result_queue and not self._result_queue.empty(): self._result = self._result_queue.get_nowait() self._done = True self.log.debug("Waiting for %s", self._process) @@ -483,7 +488,7 @@ def done(self): return True # Potential error case when process dies - if not self._process.is_alive(): + if self._result_queue and not self._process.is_alive(): self._done = True # Get the object from the queue or else join() can hang. if not self._result_queue.empty(): @@ -534,8 +539,6 @@ def __init__( dag_ids=None, subdir=settings.DAGS_FOLDER, num_runs=-1, - file_process_interval=conf.getint('scheduler', - 'min_file_process_interval'), processor_poll_interval=1.0, run_duration=None, do_pickle=False, @@ -584,26 +587,27 @@ def __init__( self.using_sqlite = False if 'sqlite' in conf.get('core', 'sql_alchemy_conn'): - if self.max_threads > 1: - self.log.error("Cannot use more than 1 thread when using sqlite. Setting max_threads to 1") - self.max_threads = 1 self.using_sqlite = True - # How often to scan the DAGs directory for new files. Default to 5 minutes. - self.dag_dir_list_interval = conf.getint('scheduler', - 'dag_dir_list_interval') - # How often to print out DAG file processing stats to the log. Default to - # 30 seconds. - self.print_stats_interval = conf.getint('scheduler', - 'print_stats_interval') - - self.file_process_interval = file_process_interval - self.max_tis_per_query = conf.getint('scheduler', 'max_tis_per_query') if run_duration is None: self.run_duration = conf.getint('scheduler', 'run_duration') + self.processor_agent = None + + signal.signal(signal.SIGINT, self._exit_gracefully) + signal.signal(signal.SIGTERM, self._exit_gracefully) + + def _exit_gracefully(self, signum, frame): + """ + Helper method to clean up processor_agent to avoid leaving orphan processes. + """ + self.log.info("Exiting gracefully upon receiving signal {}".format(signum)) + if self.processor_agent: + self.processor_agent.end() + sys.exit(os.EX_OK) + @provide_session def manage_slas(self, dag, session=None): """ @@ -739,25 +743,6 @@ def manage_slas(self, dag, session=None): session.merge(sla) session.commit() - @staticmethod - @provide_session - def clear_nonexistent_import_errors(session, known_file_paths): - """ - Clears import errors for files that no longer exist. - - :param session: session for ORM operations - :type session: sqlalchemy.orm.session.Session - :param known_file_paths: The list of existing files that are parsed for DAGs - :type known_file_paths: list[unicode] - """ - query = session.query(models.ImportError) - if known_file_paths: - query = query.filter( - ~models.ImportError.filename.in_(known_file_paths) - ) - query.delete(synchronize_session='fetch') - session.commit() - @staticmethod def update_import_errors(session, dagbag): """ @@ -1110,7 +1095,9 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): # Put one task instance on each line task_instance_str = "\n\t".join( ["{}".format(x) for x in task_instances_to_examine]) - self.log.info("Tasks up for execution:\n\t%s", task_instance_str) + self.log.info("{} tasks up for execution:\n\t{}" + .format(len(task_instances_to_examine), + task_instance_str)) # Get the pool settings pools = {p.pool: p for p in session.query(models.Pool).all()} @@ -1301,8 +1288,9 @@ def query(result, items): .all()) task_instance_str = "\n\t".join( ["{}".format(x) for x in tis_to_be_queued]) - self.log.info("Setting the follow tasks to queued state:\n\t%s", - task_instance_str) + self.log.info("Setting the following {} tasks to queued state:\n\t{}" + .format(len(tis_to_be_queued), + task_instance_str)) return result + tis_to_be_queued tis_to_be_queued = helpers.reduce_in_chunks(query, @@ -1479,72 +1467,6 @@ def _process_executor_events(self, simple_dag_bag, session=None): session.merge(ti) session.commit() - def _log_file_processing_stats(self, - known_file_paths, - processor_manager): - """ - Print out stats about how files are getting processed. - - :param known_file_paths: a list of file paths that may contain Airflow - DAG definitions - :type known_file_paths: list[unicode] - :param processor_manager: manager for the file processors - :type stats: DagFileProcessorManager - :return: None - """ - - # File Path: Path to the file containing the DAG definition - # PID: PID associated with the process that's processing the file. May - # be empty. - # Runtime: If the process is currently running, how long it's been - # running for in seconds. - # Last Runtime: If the process ran before, how long did it take to - # finish in seconds - # Last Run: When the file finished processing in the previous run. - headers = ["File Path", - "PID", - "Runtime", - "Last Runtime", - "Last Run"] - - rows = [] - for file_path in known_file_paths: - last_runtime = processor_manager.get_last_runtime(file_path) - processor_pid = processor_manager.get_pid(file_path) - processor_start_time = processor_manager.get_start_time(file_path) - runtime = ((timezone.utcnow() - processor_start_time).total_seconds() - if processor_start_time else None) - last_run = processor_manager.get_last_finish_time(file_path) - - rows.append((file_path, - processor_pid, - runtime, - last_runtime, - last_run)) - - # Sort by longest last runtime. (Can't sort None values in python3) - rows = sorted(rows, key=lambda x: x[3] or 0.0) - - formatted_rows = [] - for file_path, pid, runtime, last_runtime, last_run in rows: - formatted_rows.append((file_path, - pid, - "{:.2f}s".format(runtime) - if runtime else None, - "{:.2f}s".format(last_runtime) - if last_runtime else None, - last_run.strftime("%Y-%m-%dT%H:%M:%S") - if last_run else None)) - log_str = ("\n" + - "=" * 80 + - "\n" + - "DAG File Processing Stats\n\n" + - tabulate(formatted_rows, headers=headers) + - "\n" + - "=" * 80) - - self.log.info(log_str) - def _execute(self): self.log.info("Starting the scheduler") @@ -1554,84 +1476,51 @@ def _execute(self): (executors.LocalExecutor, executors.SequentialExecutor): pickle_dags = True - # Use multiple processes to parse and generate tasks for the - # DAGs in parallel. By processing them in separate processes, - # we can get parallelism and isolation from potentially harmful - # user code. - self.log.info( - "Processing files using up to %s processes at a time", - self.max_threads) self.log.info("Running execute loop for %s seconds", self.run_duration) self.log.info("Processing each file at most %s times", self.num_runs) - self.log.info( - "Process each file at most once every %s seconds", - self.file_process_interval) - self.log.info( - "Checking for new files in %s every %s seconds", - self.subdir, - self.dag_dir_list_interval) # Build up a list of Python files that could contain DAGs self.log.info("Searching for files in %s", self.subdir) known_file_paths = list_py_file_paths(self.subdir) self.log.info("There are %s files in %s", len(known_file_paths), self.subdir) - def processor_factory(file_path): + def processor_factory(file_path, zombies): return DagFileProcessor(file_path, pickle_dags, - self.dag_ids) + self.dag_ids, + zombies) + + # When using sqlite, we do not use async_mode + # so the scheduler job and DAG parser don't access the DB at the same time. + async_mode = not self.using_sqlite - processor_manager = DagFileProcessorManager(self.subdir, - known_file_paths, - self.max_threads, - self.file_process_interval, - self.num_runs, - processor_factory) + self.processor_agent = DagFileProcessorAgent(self.subdir, + known_file_paths, + self.num_runs, + processor_factory, + async_mode) try: - self._execute_helper(processor_manager) + self._execute_helper() finally: + self.processor_agent.end() self.log.info("Exited execute loop") - # Kill all child processes on exit since we don't want to leave - # them as orphaned. - pids_to_kill = processor_manager.get_all_pids() - if len(pids_to_kill) > 0: - # First try SIGTERM - this_process = psutil.Process(os.getpid()) - # Only check child processes to ensure that we don't have a case - # where we kill the wrong process because a child process died - # but the PID got reused. - child_processes = [x for x in this_process.children(recursive=True) - if x.is_running() and x.pid in pids_to_kill] - for child in child_processes: - self.log.info("Terminating child PID: %s", child.pid) - child.terminate() - # TODO: Remove magic number - timeout = 5 - self.log.info( - "Waiting up to %s seconds for processes to exit...", timeout) - try: - psutil.wait_procs( - child_processes, timeout=timeout, - callback=lambda x: self.log.info('Terminated PID %s', x.pid)) - except psutil.TimeoutExpired: - self.log.debug("Ran out of time while waiting for processes to exit") - - # Then SIGKILL - child_processes = [x for x in this_process.children(recursive=True) - if x.is_running() and x.pid in pids_to_kill] - if len(child_processes) > 0: - self.log.info("SIGKILL processes that did not terminate gracefully") - for child in child_processes: - self.log.info("Killing child PID: %s", child.pid) - child.kill() - child.wait() - - def _execute_helper(self, processor_manager): - """ - :param processor_manager: manager to use - :type processor_manager: DagFileProcessorManager + def _execute_helper(self): + """ + The actual scheduler loop. The main steps in the loop are: + #. Harvest DAG parsing results through DagFileProcessorAgent + #. Find and queue executable tasks + #. Change task instance state in DB + #. Queue tasks in executor + #. Heartbeat executor + #. Execute queued tasks in executor asynchronously + #. Sync on the states of running tasks + + Following is a graphic representation of these steps. + + .. image:: ../docs/img/scheduler_loop.jpg + :return: None """ self.executor.start() @@ -1639,17 +1528,13 @@ def _execute_helper(self, processor_manager): self.log.info("Resetting orphaned tasks for active dag runs") self.reset_state_for_orphaned_tasks() + # Start after resetting orphaned tasks to avoid stressing out DB. + self.processor_agent.start() + execute_start_time = timezone.utcnow() - # Last time stats were printed - last_stat_print_time = datetime.datetime(2000, 1, 1, tzinfo=timezone.utc) # Last time that self.heartbeat() was called. last_self_heartbeat_time = timezone.utcnow() - # Last time that the DAG dir was traversed to look for files - last_dag_dir_refresh_time = timezone.utcnow() - - # Use this value initially - known_file_paths = processor_manager.file_paths # For the execute duration, parse and schedule DAGs while (timezone.utcnow() - execute_start_time).total_seconds() < \ @@ -1657,60 +1542,47 @@ def _execute_helper(self, processor_manager): self.log.debug("Starting Loop...") loop_start_time = time.time() - # Traverse the DAG directory for Python files containing DAGs - # periodically - elapsed_time_since_refresh = (timezone.utcnow() - - last_dag_dir_refresh_time).total_seconds() - - if elapsed_time_since_refresh > self.dag_dir_list_interval: - # Build up a list of Python files that could contain DAGs - self.log.info("Searching for files in %s", self.subdir) - known_file_paths = list_py_file_paths(self.subdir) - last_dag_dir_refresh_time = timezone.utcnow() - self.log.info( - "There are %s files in %s", len(known_file_paths), self.subdir) - - processor_manager.set_file_paths(known_file_paths) - - self.log.debug("Removing old import errors") - self.clear_nonexistent_import_errors(known_file_paths=known_file_paths) - - # Kick of new processes and collect results from finished ones - self.log.debug("Heartbeating the process manager") - simple_dags = processor_manager.heartbeat() - if self.using_sqlite: + self.processor_agent.heartbeat() # For the sqlite case w/ 1 thread, wait until the processor # is finished to avoid concurrent access to the DB. self.log.debug( "Waiting for processors to finish since we're using sqlite") + self.processor_agent.wait_until_finished() - processor_manager.wait_until_finished() + self.log.info("Harvesting DAG parsing results") + simple_dags = self.processor_agent.harvest_simple_dags() # Send tasks for execution if available simple_dag_bag = SimpleDagBag(simple_dags) if len(simple_dags) > 0: - - # Handle cases where a DAG run state is set (perhaps manually) to - # a non-running state. Handle task instances that belong to - # DAG runs in those states - - # If a task instance is up for retry but the corresponding DAG run - # isn't running, mark the task instance as FAILED so we don't try - # to re-run it. - self._change_state_for_tis_without_dagrun(simple_dag_bag, - [State.UP_FOR_RETRY], - State.FAILED) - # If a task instance is scheduled or queued, but the corresponding - # DAG run isn't running, set the state to NONE so we don't try to - # re-run it. - self._change_state_for_tis_without_dagrun(simple_dag_bag, - [State.QUEUED, - State.SCHEDULED], - State.NONE) - - self._execute_task_instances(simple_dag_bag, - (State.SCHEDULED,)) + try: + simple_dag_bag = SimpleDagBag(simple_dags) + + # Handle cases where a DAG run state is set (perhaps manually) to + # a non-running state. Handle task instances that belong to + # DAG runs in those states + + # If a task instance is up for retry but the corresponding DAG run + # isn't running, mark the task instance as FAILED so we don't try + # to re-run it. + self._change_state_for_tis_without_dagrun(simple_dag_bag, + [State.UP_FOR_RETRY], + State.FAILED) + # If a task instance is scheduled or queued, but the corresponding + # DAG run isn't running, set the state to NONE so we don't try to + # re-run it. + self._change_state_for_tis_without_dagrun(simple_dag_bag, + [State.QUEUED, + State.SCHEDULED], + State.NONE) + + self._execute_task_instances(simple_dag_bag, + (State.SCHEDULED,)) + except Exception as e: + self.log.error("Error queuing tasks") + self.log.exception(e) + continue # Call heartbeats self.log.debug("Heartbeating the executor") @@ -1727,40 +1599,34 @@ def _execute_helper(self, processor_manager): self.heartbeat() last_self_heartbeat_time = timezone.utcnow() - # Occasionally print out stats about how fast the files are getting processed - if ((timezone.utcnow() - last_stat_print_time).total_seconds() > - self.print_stats_interval): - if len(known_file_paths) > 0: - self._log_file_processing_stats(known_file_paths, - processor_manager) - last_stat_print_time = timezone.utcnow() - loop_end_time = time.time() + loop_duration = loop_end_time - loop_start_time self.log.debug( "Ran scheduling loop in %.2f seconds", - loop_end_time - loop_start_time) + loop_duration) self.log.debug("Sleeping for %.2f seconds", self._processor_poll_interval) time.sleep(self._processor_poll_interval) # Exit early for a test mode - if processor_manager.max_runs_reached(): - self.log.info( - "Exiting loop as all files have been processed %s times", - self.num_runs) + if self.processor_agent.done: + self.log.info("Exiting scheduler loop as all files" + " have been processed {} times".format(self.num_runs)) break + if loop_duration < 1: + sleep_length = 1 - loop_duration + self.log.debug( + "Sleeping for {0:.2f} seconds to prevent excessive logging" + .format(sleep_length)) + sleep(sleep_length) + # Stop any processors - processor_manager.terminate() + self.processor_agent.terminate() # Verify that all files were processed, and if so, deactivate DAGs that # haven't been touched by the scheduler as they likely have been # deleted. - all_files_processed = True - for file_path in known_file_paths: - if processor_manager.get_last_finish_time(file_path) is None: - all_files_processed = False - break - if all_files_processed: + if self.processor_agent.all_files_processed: self.log.info( "Deactivating DAGs that haven't been touched since %s", execute_start_time.isoformat() @@ -1772,7 +1638,7 @@ def _execute_helper(self, processor_manager): settings.Session.remove() @provide_session - def process_file(self, file_path, pickle_dags=False, session=None): + def process_file(self, file_path, zombies, pickle_dags=False, session=None): """ Process a Python file containing Airflow DAGs. @@ -1791,6 +1657,8 @@ def process_file(self, file_path, pickle_dags=False, session=None): :param file_path: the path to the Python file that should be executed :type file_path: unicode + :param zombies: zombie task instances to kill. + :type zombies: list[SimpleTaskInstance] :param pickle_dags: whether serialize the DAGs found in the file and save them to the db :type pickle_dags: bool @@ -1885,7 +1753,7 @@ def process_file(self, file_path, pickle_dags=False, session=None): except Exception: self.log.exception("Error logging import errors!") try: - dagbag.kill_zombies() + dagbag.kill_zombies(zombies) except Exception: self.log.exception("Error killing zombies!") diff --git a/airflow/models.py b/airflow/models.py index 0a03ea979223a..5248c76e07ea1 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -448,39 +448,28 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): return found_dags @provide_session - def kill_zombies(self, session=None): - """ - Fails tasks that haven't had a heartbeat in too long - """ - from airflow.jobs import LocalTaskJob as LJ - self.log.info("Finding 'running' jobs without a recent heartbeat") - TI = TaskInstance - secs = configuration.conf.getint('scheduler', 'scheduler_zombie_task_threshold') - limit_dttm = timezone.utcnow() - timedelta(seconds=secs) - self.log.info("Failing jobs without heartbeat after %s", limit_dttm) - - tis = ( - session.query(TI) - .join(LJ, TI.job_id == LJ.id) - .filter(TI.state == State.RUNNING) - .filter( - or_( - LJ.state != State.RUNNING, - LJ.latest_heartbeat < limit_dttm, - )) - .all() - ) - - for ti in tis: - if ti and ti.dag_id in self.dags: - dag = self.dags[ti.dag_id] - if ti.task_id in dag.task_ids: - task = dag.get_task(ti.task_id) - - # now set non db backed vars on ti - ti.task = task + def kill_zombies(self, zombies, session=None): + """ + Fail given zombie tasks, which are tasks that haven't + had a heartbeat for too long, in the current DagBag. + + :param zombies: zombie task instances to kill. + :type zombies: SimpleTaskInstance + :param session: DB session. + :type Session. + """ + for zombie in zombies: + if zombie.dag_id in self.dags: + dag = self.dags[zombie.dag_id] + if zombie.task_id in dag.task_ids: + task = dag.get_task(zombie.task_id) + ti = TaskInstance(task, zombie.execution_date) + # Get properties needed for failure handling from SimpleTaskInstance. + ti.start_date = zombie.start_date + ti.end_date = zombie.end_date + ti.try_number = zombie.try_number + ti.state = zombie.state ti.test_mode = configuration.getboolean('core', 'unit_test_mode') - ti.handle_failure("{} detected as zombie".format(ti), ti.test_mode, ti.get_template_context()) self.log.info( diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index 8b77c796d2030..47f473e9aa3d3 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -22,17 +22,33 @@ from __future__ import print_function from __future__ import unicode_literals +import logging +import multiprocessing import os import re +import signal +import sys import time import zipfile from abc import ABCMeta, abstractmethod from collections import defaultdict +from collections import namedtuple +from datetime import timedelta +import psutil +from six.moves import range, reload_module +from sqlalchemy import or_ +from tabulate import tabulate + +# To avoid circular imports +import airflow.models +from airflow import configuration as conf from airflow.dag.base_dag import BaseDag, BaseDagBag from airflow.exceptions import AirflowException from airflow.utils import timezone +from airflow.utils.db import provide_session from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.state import State class SimpleDag(BaseDag): @@ -121,6 +137,45 @@ def get_task_special_arg(self, task_id, special_arg_name): return None +class SimpleTaskInstance(object): + def __init__(self, ti): + self._dag_id = ti.dag_id + self._task_id = ti.task_id + self._execution_date = ti.execution_date + self._start_date = ti.start_date + self._end_date = ti.end_date + self._try_number = ti.try_number + self._state = ti.state + + @property + def dag_id(self): + return self._dag_id + + @property + def task_id(self): + return self._task_id + + @property + def execution_date(self): + return self._execution_date + + @property + def start_date(self): + return self._start_date + + @property + def end_date(self): + return self._end_date + + @property + def try_number(self): + return self._try_number + + @property + def state(self): + return self._state + + class SimpleDagBag(BaseDagBag): """ A collection of SimpleDag objects with some convenience methods. @@ -308,10 +363,243 @@ def file_path(self): raise NotImplementedError() +DagParsingStat = namedtuple('DagParsingStat', + ['file_paths', 'all_pids', 'done', + 'all_files_processed', 'result_count']) + + +DagParsingSignal = namedtuple( + 'DagParsingSignal', + ['AGENT_HEARTBEAT', 'MANAGER_DONE', 'TERMINATE_MANAGER', 'END_MANAGER'])( + 'agent_heartbeat', 'manager_done', 'terminate_manager', 'end_manager') + + +class DagFileProcessorAgent(LoggingMixin): + """ + Agent for DAG file processing. It is responsible for all DAG parsing + related jobs in scheduler process. Mainly it can spin up DagFileProcessorManager + in a subprocess, collect DAG parsing results from it and communicate + signal/DAG parsing stat with it. + """ + + def __init__(self, + dag_directory, + file_paths, + max_runs, + processor_factory, + async_mode): + """ + :param dag_directory: Directory where DAG definitions are kept. All + files in file_paths should be under this directory + :type dag_directory: unicode + :param file_paths: list of file paths that contain DAG definitions + :type file_paths: list[unicode] + :param max_runs: The number of times to parse and schedule each file. -1 + for unlimited. + :type max_runs: int + :param processor_factory: function that creates processors for DAG + definition files. Arguments are (dag_definition_path, log_file_path) + :type processor_factory: (unicode, unicode, list) -> (AbstractDagFileProcessor) + :param async_mode: Whether to start agent in async mode + :type async_mode: bool + """ + self._file_paths = file_paths + self._file_path_queue = [] + self._dag_directory = dag_directory + self._max_runs = max_runs + self._processor_factory = processor_factory + self._async_mode = async_mode + # Map from file path to the processor + self._processors = {} + # Map from file path to the last runtime + self._last_runtime = {} + # Map from file path to the last finish time + self._last_finish_time = {} + # Map from file path to the number of runs + self._run_count = defaultdict(int) + # Pids of DAG parse + self._all_pids = [] + # Pipe for communicating signals + self._parent_signal_conn, self._child_signal_conn = multiprocessing.Pipe() + # Pipe for communicating DagParsingStat + self._stat_queue = multiprocessing.Queue() + self._result_queue = multiprocessing.Queue() + self._process = None + self._done = False + # Initialized as true so we do not deactivate w/o any actual DAG parsing. + self._all_files_processed = True + self._result_count = 0 + + def start(self): + """ + Launch DagFileProcessorManager processor and start DAG parsing loop in manager. + """ + self._process = self._launch_process(self._dag_directory, + self._file_paths, + self._max_runs, + self._processor_factory, + self._child_signal_conn, + self._stat_queue, + self._result_queue, + self._async_mode) + self.log.info("Launched DagFileProcessorManager with pid: {}" + .format(self._process.pid)) + + def heartbeat(self): + """ + Should only be used when launched DAG file processor manager in sync mode. + Send agent heartbeat signal to the manager. + """ + self._parent_signal_conn.send(DagParsingSignal.AGENT_HEARTBEAT) + + def wait_until_finished(self): + """ + Should only be used when launched DAG file processor manager in sync mode. + Wait for done signal from the manager. + """ + while True: + if self._parent_signal_conn.recv() == DagParsingSignal.MANAGER_DONE: + break + + @staticmethod + def _launch_process(dag_directory, + file_paths, + max_runs, + processor_factory, + signal_conn, + _stat_queue, + result_queue, + async_mode): + def helper(): + # Reload configurations and settings to avoid collision with parent process. + # Because this process may need custom configurations that cannot be shared, + # e.g. RotatingFileHandler. And it can cause connection corruption if we + # do not recreate the SQLA connection pool. + os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER'] = 'True' + reload_module(airflow.config_templates.airflow_local_settings) + reload_module(airflow.settings) + del os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER'] + processor_manager = DagFileProcessorManager(dag_directory, + file_paths, + max_runs, + processor_factory, + signal_conn, + _stat_queue, + result_queue, + async_mode) + + processor_manager.start() + + p = multiprocessing.Process(target=helper, + args=(), + name="DagFileProcessorManager") + p.start() + return p + + def harvest_simple_dags(self): + """ + Harvest DAG parsing results from result queue and sync metadata from stat queue. + :return: List of parsing result in SimpleDag format. + """ + # Metadata and results to be harvested can be inconsistent, + # but it should not be a big problem. + self._sync_metadata() + # Heartbeating after syncing metadata so we do not restart manager + # if it processed all files for max_run times and exit normally. + self._heartbeat_manager() + simple_dags = [] + # multiprocessing.Queue().qsize will not work on MacOS. + if sys.platform == "darwin": + qsize = self._result_count + else: + qsize = self._result_queue.qsize() + for _ in range(qsize): + simple_dags.append(self._result_queue.get()) + + self._result_count = 0 + + return simple_dags + + def _heartbeat_manager(self): + """ + Heartbeat DAG file processor and start it if it is not alive. + :return: + """ + if self._process and not self._process.is_alive() and not self.done: + self.start() + + def _sync_metadata(self): + """ + Sync metadata from stat queue and only keep the latest stat. + :return: + """ + while not self._stat_queue.empty(): + stat = self._stat_queue.get() + self._file_paths = stat.file_paths + self._all_pids = stat.all_pids + self._done = stat.done + self._all_files_processed = stat.all_files_processed + self._result_count += stat.result_count + + @property + def file_paths(self): + return self._file_paths + + @property + def done(self): + return self._done + + @property + def all_files_processed(self): + return self._all_files_processed + + def terminate(self): + """ + Send termination signal to DAG parsing processor manager + and expect it to terminate all DAG file processors. + """ + self.log.info("Sending termination message to manager.") + self._child_signal_conn.send(DagParsingSignal.TERMINATE_MANAGER) + + def end(self): + """ + Terminate (and then kill) the manager process launched. + :return: + """ + if not self._process or not self._process.is_alive(): + self.log.warn('Ending without manager process.') + return + this_process = psutil.Process(os.getpid()) + manager_process = psutil.Process(self._process.pid) + # First try SIGTERM + if manager_process.is_running() \ + and manager_process.pid in [x.pid for x in this_process.children()]: + self.log.info( + "Terminating manager process: {}".format(manager_process.pid)) + manager_process.terminate() + # TODO: Remove magic number + timeout = 5 + self.log.info("Waiting up to {}s for manager process to exit..." + .format(timeout)) + try: + psutil.wait_procs({manager_process}, timeout) + except psutil.TimeoutExpired: + self.log.debug("Ran out of time while waiting for " + "processes to exit") + + # Then SIGKILL + if manager_process.is_running() \ + and manager_process.pid in [x.pid for x in this_process.children()]: + self.log.info("Killing manager process: {}".format(manager_process.pid)) + manager_process.kill() + manager_process.wait() + + class DagFileProcessorManager(LoggingMixin): """ Given a list of DAG definition files, this kicks off several processors - in parallel to process them. The parallelism is limited and as the + in parallel to process them and put the results to a multiprocessing.Queue + for DagFileProcessorAgent to harvest. The parallelism is limited and as the processors finish, more are launched. The files are processed over and over again, but no more often than the specified interval. @@ -324,48 +612,320 @@ class DagFileProcessorManager(LoggingMixin): def __init__(self, dag_directory, file_paths, - parallelism, - process_file_interval, max_runs, - processor_factory): + processor_factory, + signal_conn, + stat_queue, + result_queue, + async_mode=True): """ :param dag_directory: Directory where DAG definitions are kept. All files in file_paths should be under this directory :type dag_directory: unicode :param file_paths: list of file paths that contain DAG definitions :type file_paths: list[unicode] - :param parallelism: maximum number of simultaneous process to run at once - :type parallelism: int - :param process_file_interval: process a file at most once every this - many seconds - :type process_file_interval: float :param max_runs: The number of times to parse and schedule each file. -1 for unlimited. :type max_runs: int - :type process_file_interval: float :param processor_factory: function that creates processors for DAG definition files. Arguments are (dag_definition_path) - :type processor_factory: (unicode, unicode) -> (AbstractDagFileProcessor) - + :type processor_factory: (unicode, unicode, list) -> (AbstractDagFileProcessor) + :param signal_conn: connection to communicate signal with processor agent. + :type signal_conn: Connection + :param stat_queue: the queue to use for passing back parsing stat to agent. + :type stat_queue: multiprocessing.Queue + :param result_queue: the queue to use for passing back the result to agent. + :type result_queue: multiprocessing.Queue + :param async_mode: whether to start the manager in async mode + :type async_mode: bool """ self._file_paths = file_paths self._file_path_queue = [] - self._parallelism = parallelism self._dag_directory = dag_directory self._max_runs = max_runs - self._process_file_interval = process_file_interval self._processor_factory = processor_factory + self._signal_conn = signal_conn + self._stat_queue = stat_queue + self._result_queue = result_queue + self._async_mode = async_mode + + self._parallelism = conf.getint('scheduler', 'max_threads') + if 'sqlite' in conf.get('core', 'sql_alchemy_conn') and self._parallelism > 1: + self.log.error("Cannot use more than 1 thread when using sqlite. " + "Setting parallelism to 1") + self._parallelism = 1 + + # Parse and schedule each file no faster than this interval. + self._file_process_interval = conf.getint('scheduler', + 'min_file_process_interval') + # How often to print out DAG file processing stats to the log. Default to + # 30 seconds. + self.print_stats_interval = conf.getint('scheduler', + 'print_stats_interval') + # How many seconds do we wait for tasks to heartbeat before mark them as zombies. + self._zombie_threshold_secs = ( + conf.getint('scheduler', 'scheduler_zombie_task_threshold')) # Map from file path to the processor self._processors = {} # Map from file path to the last runtime self._last_runtime = {} # Map from file path to the last finish time self._last_finish_time = {} + self._last_zombie_query_time = timezone.utcnow() + # Last time that the DAG dir was traversed to look for files + self.last_dag_dir_refresh_time = timezone.utcnow() + # Last time stats were printed + self.last_stat_print_time = timezone.datetime(2000, 1, 1) + # TODO: Remove magic number + self._zombie_query_interval = 10 # Map from file path to the number of runs self._run_count = defaultdict(int) - # Scheduler heartbeat key. + # Manager heartbeat key. self._heart_beat_key = 'heart-beat' + # How often to scan the DAGs directory for new files. Default to 5 minutes. + self.dag_dir_list_interval = conf.getint('scheduler', + 'dag_dir_list_interval') + + self._log = logging.getLogger('airflow.processor_manager') + + signal.signal(signal.SIGINT, self._exit_gracefully) + signal.signal(signal.SIGTERM, self._exit_gracefully) + + def _exit_gracefully(self, signum, frame): + """ + Helper method to clean up DAG file processors to avoid leaving orphan processes. + """ + self.log.info("Exiting gracefully upon receiving signal {}".format(signum)) + self.terminate() + self.end() + self.log.debug("Finished terminating DAG processors.") + sys.exit(os.EX_OK) + + def start(self): + """ + Use multiple processes to parse and generate tasks for the + DAGs in parallel. By processing them in separate processes, + we can get parallelism and isolation from potentially harmful + user code. + :return: + """ + + self.log.info("Processing files using up to {} processes at a time " + .format(self._parallelism)) + self.log.info("Process each file at most once every {} seconds" + .format(self._file_process_interval)) + self.log.info("Checking for new files in {} every {} seconds" + .format(self._dag_directory, self.dag_dir_list_interval)) + + if self._async_mode: + self.log.debug("Starting DagFileProcessorManager in async mode") + self.start_in_async() + else: + self.log.debug("Starting DagFileProcessorManager in sync mode") + self.start_in_sync() + + def start_in_async(self): + """ + Parse DAG files repeatedly in a standalone loop. + """ + while True: + loop_start_time = time.time() + + if self._signal_conn.poll(): + agent_signal = self._signal_conn.recv() + if agent_signal == DagParsingSignal.TERMINATE_MANAGER: + self.terminate() + break + elif agent_signal == DagParsingSignal.END_MANAGER: + self.end() + sys.exit(os.EX_OK) + + self._refresh_dag_dir() + + simple_dags = self.heartbeat() + for simple_dag in simple_dags: + self._result_queue.put(simple_dag) + + self._print_stat() + + all_files_processed = all(self.get_last_finish_time(x) is not None + for x in self.file_paths) + max_runs_reached = self.max_runs_reached() + + dag_parsing_stat = DagParsingStat(self._file_paths, + self.get_all_pids(), + max_runs_reached, + all_files_processed, + len(simple_dags)) + self._stat_queue.put(dag_parsing_stat) + + if max_runs_reached: + self.log.info("Exiting dag parsing loop as all files " + "have been processed %s times", self._max_runs) + break + + loop_duration = time.time() - loop_start_time + if loop_duration < 1: + sleep_length = 1 - loop_duration + self.log.debug("Sleeping for {0:.2f} seconds " + "to prevent excessive logging".format(sleep_length)) + time.sleep(sleep_length) + + def start_in_sync(self): + """ + Parse DAG files in a loop controlled by DagParsingSignal. + Actual DAG parsing loop will run once upon receiving one + agent heartbeat message and will report done when finished the loop. + """ + while True: + agent_signal = self._signal_conn.recv() + if agent_signal == DagParsingSignal.TERMINATE_MANAGER: + self.terminate() + break + elif agent_signal == DagParsingSignal.END_MANAGER: + self.end() + sys.exit(os.EX_OK) + elif agent_signal == DagParsingSignal.AGENT_HEARTBEAT: + + self._refresh_dag_dir() + + simple_dags = self.heartbeat() + for simple_dag in simple_dags: + self._result_queue.put(simple_dag) + + self._print_stat() + + all_files_processed = all(self.get_last_finish_time(x) is not None + for x in self.file_paths) + max_runs_reached = self.max_runs_reached() + + dag_parsing_stat = DagParsingStat(self._file_paths, + self.get_all_pids(), + self.max_runs_reached(), + all_files_processed, + len(simple_dags)) + self._stat_queue.put(dag_parsing_stat) + + self.wait_until_finished() + self._signal_conn.send(DagParsingSignal.MANAGER_DONE) + + if max_runs_reached: + self.log.info("Exiting dag parsing loop as all files " + "have been processed %s times", self._max_runs) + self._signal_conn.send(DagParsingSignal.MANAGER_DONE) + break + + def _refresh_dag_dir(self): + """ + Refresh file paths from dag dir if we haven't done it for too long. + """ + elapsed_time_since_refresh = (timezone.utcnow() - + self.last_dag_dir_refresh_time).total_seconds() + if elapsed_time_since_refresh > self.dag_dir_list_interval: + # Build up a list of Python files that could contain DAGs + self.log.info( + "Searching for files in {}".format(self._dag_directory)) + self._file_paths = list_py_file_paths(self._dag_directory) + self.last_dag_dir_refresh_time = timezone.utcnow() + self.log.info("There are {} files in {}" + .format(len(self._file_paths), + self._dag_directory)) + self.set_file_paths(self._file_paths) + + try: + self.log.debug("Removing old import errors") + self.clear_nonexistent_import_errors() + except Exception: + self.log.exception("Error removing old import errors") + + def _print_stat(self): + """ + Occasionally print out stats about how fast the files are getting processed + :return: + """ + if ((timezone.utcnow() - self.last_stat_print_time).total_seconds() > + self.print_stats_interval): + if len(self._file_paths) > 0: + self._log_file_processing_stats(self._file_paths) + self.last_stat_print_time = timezone.utcnow() + + @provide_session + def clear_nonexistent_import_errors(self, session): + """ + Clears import errors for files that no longer exist. + :param session: session for ORM operations + :type session: sqlalchemy.orm.session.Session + """ + query = session.query(airflow.models.ImportError) + if self._file_paths: + query = query.filter( + ~airflow.models.ImportError.filename.in_(self._file_paths) + ) + query.delete(synchronize_session='fetch') + session.commit() + + def _log_file_processing_stats(self, known_file_paths): + """ + Print out stats about how files are getting processed. + :param known_file_paths: a list of file paths that may contain Airflow + DAG definitions + :type known_file_paths: list[unicode] + :return: None + """ + + # File Path: Path to the file containing the DAG definition + # PID: PID associated with the process that's processing the file. May + # be empty. + # Runtime: If the process is currently running, how long it's been + # running for in seconds. + # Last Runtime: If the process ran before, how long did it take to + # finish in seconds + # Last Run: When the file finished processing in the previous run. + headers = ["File Path", + "PID", + "Runtime", + "Last Runtime", + "Last Run"] + + rows = [] + for file_path in known_file_paths: + last_runtime = self.get_last_runtime(file_path) + processor_pid = self.get_pid(file_path) + processor_start_time = self.get_start_time(file_path) + runtime = ((timezone.utcnow() - processor_start_time).total_seconds() + if processor_start_time else None) + last_run = self.get_last_finish_time(file_path) + + rows.append((file_path, + processor_pid, + runtime, + last_runtime, + last_run)) + + # Sort by longest last runtime. (Can't sort None values in python3) + rows = sorted(rows, key=lambda x: x[3] or 0.0) + + formatted_rows = [] + for file_path, pid, runtime, last_runtime, last_run in rows: + formatted_rows.append((file_path, + pid, + "{:.2f}s".format(runtime) + if runtime else None, + "{:.2f}s".format(last_runtime) + if last_runtime else None, + last_run.strftime("%Y-%m-%dT%H:%M:%S") + if last_run else None)) + log_str = ("\n" + + "=" * 80 + + "\n" + + "DAG File Processing Stats\n\n" + + tabulate(formatted_rows, headers=headers) + + "\n" + + "=" * 80) + + self.log.info(log_str) + @property def file_paths(self): return self._file_paths @@ -472,7 +1032,7 @@ def wait_until_finished(self): def heartbeat(self): """ - This should be periodically called by the scheduler. This method will + This should be periodically called by the manager loop. This method will kick off new processes to process DAG definition files and read the results from the finished processors. @@ -498,7 +1058,7 @@ def heartbeat(self): running_processors[file_path] = processor self._processors = running_processors - self.log.debug("%s/%s scheduler processes running", + self.log.debug("%s/%s DAG parsing processes running", len(self._processors), self._parallelism) self.log.debug("%s file paths queued for processing", @@ -528,7 +1088,7 @@ def heartbeat(self): last_finish_time = self.get_last_finish_time(file_path) if (last_finish_time is not None and (now - last_finish_time).total_seconds() < - self._process_file_interval): + self._file_process_interval): file_paths_recently_processed.append(file_path) files_paths_at_run_limit = [file_path @@ -553,11 +1113,13 @@ def heartbeat(self): self._file_path_queue.extend(files_paths_to_queue) + zombies = self._find_zombies() + # Start more processors if we have enough slots and files to process while (self._parallelism - len(self._processors) > 0 and len(self._file_path_queue) > 0): file_path = self._file_path_queue.pop(0) - processor = self._processor_factory(file_path) + processor = self._processor_factory(file_path, zombies) processor.start() self.log.debug( @@ -566,11 +1128,47 @@ def heartbeat(self): ) self._processors[file_path] = processor - # Update scheduler heartbeat count. + # Update heartbeat count. self._run_count[self._heart_beat_key] += 1 return simple_dags + @provide_session + def _find_zombies(self, session): + """ + Find zombie task instances, which are tasks haven't heartbeated for too long. + :return: Zombie task instances in SimpleTaskInstance format. + """ + now = timezone.utcnow() + zombies = [] + if (now - self._last_zombie_query_time).total_seconds() \ + > self._zombie_query_interval: + # to avoid circular imports + from airflow.jobs import LocalTaskJob as LJ + self.log.info("Finding 'running' jobs without a recent heartbeat") + TI = airflow.models.TaskInstance + limit_dttm = timezone.utcnow() - timedelta( + seconds=self._zombie_threshold_secs) + self.log.info( + "Failing jobs without heartbeat after {}".format(limit_dttm)) + + tis = ( + session.query(TI) + .join(LJ, TI.job_id == LJ.id) + .filter(TI.state == State.RUNNING) + .filter( + or_( + LJ.state != State.RUNNING, + LJ.latest_heartbeat < limit_dttm, + ) + ).all() + ) + self._last_zombie_query_time = timezone.utcnow() + for ti in tis: + zombies.append(SimpleTaskInstance(ti)) + + return zombies + def max_runs_reached(self): """ :return: whether all file paths have been processed max_runs times @@ -591,3 +1189,41 @@ def terminate(self): """ for processor in self._processors.values(): processor.terminate() + + def end(self): + """ + Kill all child processes on exit since we don't want to leave + them as orphaned. + """ + pids_to_kill = self.get_all_pids() + if len(pids_to_kill) > 0: + # First try SIGTERM + this_process = psutil.Process(os.getpid()) + # Only check child processes to ensure that we don't have a case + # where we kill the wrong process because a child process died + # but the PID got reused. + child_processes = [x for x in this_process.children(recursive=True) + if x.is_running() and x.pid in pids_to_kill] + for child in child_processes: + self.log.info("Terminating child PID: {}".format(child.pid)) + child.terminate() + # TODO: Remove magic number + timeout = 5 + self.log.info( + "Waiting up to %s seconds for processes to exit...", timeout) + try: + psutil.wait_procs( + child_processes, timeout=timeout, + callback=lambda x: self.log.info('Terminated PID %s', x.pid)) + except psutil.TimeoutExpired: + self.log.debug("Ran out of time while waiting for processes to exit") + + # Then SIGKILL + child_processes = [x for x in this_process.children(recursive=True) + if x.is_running() and x.pid in pids_to_kill] + if len(child_processes) > 0: + self.log.info("SIGKILL processes that did not terminate gracefully") + for child in child_processes: + self.log.info("Killing child PID: {}".format(child.pid)) + child.kill() + child.wait() diff --git a/docs/img/scheduler_loop.jpg b/docs/img/scheduler_loop.jpg new file mode 100644 index 0000000000000000000000000000000000000000..a4a5e285e54403afdbb6eebc4d0328b8ab677775 GIT binary patch literal 46864 zcmeFa2UJwcwl&-eNDxGF1|?@CBdG-?3rNl&l0iUnrWFJf34#I&0+KUL&OymJ=bUrS zp=rL2C+WGK%e~)s-y84!|G}|)47>Ny9Tx-r`gL?7Y#dBnGGYpHGGbEFThy#{w|b8&G|(A^c{;}B%!_ocq?GNZUE~^$f<7{(4KBxvkT?`7(cG_iirrh(oKI}MP|@%S2#JVkXzAz~7`eE4 zc=_(~-+v$`E+Hu;t)#4?s-~`?X<%q%Y+`C=ZfEb{=;Z9;>iyEk*YDM9|Il~u!@?sz ze2k1wNK8sjNli=7%P%M_DlRE4tEsK4Z)j|4Zt3am>mL{#8XlRRnVp+oSX^3O+1}aR z+dnuwIzGAZ>k0_r7q@`dUmW|BUs!-&R}m2r5RotZx^mSCI1sQ9k#4hJ!xm9M*0;f- zYiYf<+z^yWxGyNDC^=XCT7!`ErpQw-Ov29?EljcJpqHCp z&p~_KOOPXWzjKg(0vx9r`5e?W72}oQ(W(s>V%3UzOA=%N=Eu!5nEA5fMl#9a@Hcr2zqtyrTcr zK9f|IVX2f~7}4!!MYLE^Ovy{v@t8^Aj;IMY5AnNT>9i@vRZxp$&oFwYC&Xjpq=%C(BZ`S8!}dp z_!`_|7<*P6t&I-qOYU2ror63-G-{tJm*mCyC(+KfWmgPNE-z>#a%&2S4_KI#Tv${a zio)oDBIy5QB|KEClgk$WWJ&DB!VURl+g>Ng67QZwjRKw(By zhyk;cTj{|L5S|Phi>CukZ{v9k#h*Btbo(W-s^thi#5LesNh+*lJ0UiXG>`V#i@>z8 zwjpk`v--lP6m~1XODIsK*TJN!B8GuV`JqeroP+a@8r?QUmW=mc6F%#^XKF{;5v^_q zgYrtMkrsEYplntqGUC4UkL`#;AF%59X@kT;%WAiGU*2E8#z}XV-3xhDfqrNp%t7Ha zZM0ezO!39zc6u>!baY0U>>ag7s5*(CNKY`$)vAjmsXnxRm^<|mv7EfeNNO}xovBc+ zy(+t}>H046{v+=FoFW--j$_t)YU`mk3Op&MR1&*Vn4u-&)GzvGi5e^AnagpvaSbX4 zJqE^G#Uss0-i*3EPB!D&m;)t-3O!@Uo755`wH0i^mQyI_C4#F6=B+6~rFcGtN39gl z8lfktUq$(fT$9k=}egRkxrwuNc(Z zJ*If5Ac=D8rU-HzZsA(&Mkq)AWx9F&u_ z34hH0l8MQk75lj2jb zuTJ4k{va4TrTr^X$^NPiYI0-*~ggwxYasVu+6Z7W_7Om>HU;i_QAcr1(*2C;oVyRp&kSJ_o!mC0Ut|7D$_7 zyOzBvTKlVZykZ7tVb<3hscl8ykYvQ^-dm4J7*kPV&mXila&wE2q6(mrxXwpyBKTrc zOFT3oN^QY%-=rBO&EiJUj>V3N@U||O)1l1uM($BMq(_s&MbC%vyuY$H@^ z#V8A7R`kQoh>y2Bm*uonl#-|wTDs~sAkLqlEjBTH3q=?_N&SutM%@dxRqJ4(UI?94 z4RL5s&8%uwaW4xlcl+oo14KjgeYzp^cWW5p7X89Ej-1@aoP(x05_k?ip1`-yLF@${ zBm)85Y-S3quUZ8g&t#Ap*!(rw)CT8*6?>l=^z4-j3FABY4Gky@NHNvlwwsyb||%l#9B??>EJcs!&}}{G!tr1 zeZtrk!Ofr%qjC;HivV8YB3sWvNV4ak#-($Rk5oEPJ#`wLgS?W@L9_cSfYJsVKu@(u z&Ou1zTOGZ+Bs-Tc@pLse59VRp& zaBU9csX_JPs$dix_aNmQbYCb1juv$e3hsf_ME9M8mP`tM{jU9V1~+;~%=G{#*qKQv zq~=p!^^)mtAA+Z4SM>I{G$B)I)kkNpZ8b*8GnT>0k`e5^&!) zdN4fRa}WhoCq5!KWe8kka}L_wj}ZgRG*AG{j34~f%w_Pj%nBH|H&Bd!yZJt@8=8mT zSz+)uFDZ@`M}p5V@T-rQfM{KKsrKl%%PoQMuMRFHtuX)r%&Y~BX#|XsHmBI*(t=E7 z0hUs1C2?b(5L;GnsREI&^zs*m${gQ6wZ3z4@9oVaMJJ$Rz%}~Y$%#-PB4$NEM6`hD z9}qKmXh!S=|8{!qmMF{%Jk3~4J}YAl7t+m2Tn;~B4KYg|VYsRFfJ%FxZ^J`M@ZE~} zI$k0}4Fy_m^XtYZ?5YR5wnc+Lk0*2fKiBQ~Pj;dHzSrOF@;3(l+c8kPow%!cL{P_z zztfDqZ5F{s8j`Cjj>V3M)pTn&ax5Ft2-%SrlsN}I&VZib_Fb79t~dvM?kGJ6dB1_e z?}}T4mpHQ_Q!}gQpdh96486LtOjjGV-5C52JlDoQzgQiNJQDM-Yc*@#G+E-P(Ko2% zJ8Q~~>O4VTjXmNk17zyG@h;q(e-VCzwfbmYEeOuwaCZ$)uickBj5q&O;M0p{SbV7%vKq^ zqHcYBNKuN>zIb-cp%X@`1KDoj{Nd7Q$+lFV57Wwqu_6+K?BLxUJvYoW{p!Ur&pF!f z9aH$h;33+e1(Zdf+ImV~aWWJk3@O=}Z6*mtJvSupnnDLTur0|Cr+@DszrUV#9i+3_ zV@){87a)I|)3;+kn4iEKyGJw#-m(EY2aOE5h3{fW^g{#5#YYUXQ(SmP(U&v1yYbJh6E1xPSQUCYZA(9xwYM5W>t8e!+!dE2>Y>-s~aU z4$JqUx}LgH_Hb*3Co(3>h9=WXwK!HGMmZXJLNt5MwiGn$?ftO0)OUrA2Nm&kN%r*0 z4a^lrO`C=ped@7^=uzApK8>$QNKoaF93pNG`gs;Ensn?kb|`#C zw<8$qo{21^or7vPO@JC{WD$&9wqvl?hf2P`SuC@vMWqGQ zQ(t0_{ixjtC;xPc1dNIYvU>((;m$6Qg#+!l5Obi|+5-wbI=Fr_c?0i{Cv=~KP}*Z* z=OByNBSD~fXAASiC`LYP9-o0>sOO*$wi3C4e?Fz)Pi8*tmN-ZjfJ5ZZL02IQr2jR3 z*l{>_@P@CR3W)b}3s^{)E|wNP7$WlAdelMGBuMrd&Yi{VXrKxUGf6p&J@yVD5Z%1Y zQHX3`7Iz>IhG>mGcjj^xF9H$8{jP(0&D3zC;5Rm5X35n+JlgKw9%=xFYmGbMkit$F zxh`aT1UyZY4>ZF11?fb@!W@fC)8KuLkj`bGT$G2^h6)8--u6+{Z*VQCa+L3Byk0{B zuM7STgS%8iK>7Mlzpr4Ob*ehwcVj>$e8Lc0f^yG@>Xv_`-^bbB{H%Yy)nsQ_^xeVJ ztLqxmoGgjr_X#bNa!0unQ*6CgIF)zNzlz9@%5}=W-Ybrcydmkx4pgx}dL8^{d}l>6 zf(zLvcTXTkIOm`{KzD=WN5q@{$*DDfYgU1{@d&J1n$36Q)`x63c8JB549yj98 z`kh#P&qOgjeSp`ivR4J?aX-=NE&^`!m@Z#RpiGXmlwslV;y%V>0u~I=2xq?Jz2p0 zaQp6P2|@szt1^saG|ZzRnBqg~E`NUUQ7qKv2e|x0GpBqDXuvt>k2L0p_`z}T_f4+6 z?qpS*W^APJY{^jlGz{@7goDvsC=h5xTavciRoC8cD)leHgy1^?`KSJui;>qv>_AJt z%g}#jqRNP`MY)^HXI~8!>XIiR3@dY2B`KcM1{+)io%^G~gg-Ex04Sp&#XrS=!Z-L| z|2L5KYrB3Yr5|N}z~T>I$hZy+FzB^ZO?^jIEqecgNyz`XxcNI&_o#5Q@SlTJV`jSR zCQYIgOFq(x9L_RsD2IM4vNV+y!^z*N&go#6_omjr;K+b zzfuLwl!Y_$7Zpo+w2VHrt)SjhaN~jPJ5727IgU!|9f?kr6{0&}=GQ*ULSs+IsBBEy$Oj>IifiXoT}_6EN9YMG%-0UJ|u+)r?gsdx23U! z6@EGxqseTWy3@HS{8)foJ|}&|sN#-JYP+Uv84il@9dG+2W!}7Y*P8K8>B-FdT&Z2_tZK{;5-5YLBYNA%kdUZp z0XI{3-X{l3{zg8;qSbXnM4goip)HM%wpG(Gj>m~dYielUrh*|X&)r-P+U->j90y+1 z7gjv-v+SJztkc*xlPc_EVUw<-GhUC;A{UZg&n)7FJKiZ=o+e&)@Gg3!da{gB-Dz#P zrg_wJl=EuGG+o!;8-Wam%5Vermt$=Tf&~V?2Hvp|R@X2jND3QYG8=I65=YjN-{nwo zpSeadWDk~Jqsz%JdretV=%wEKUV{pqU^Q?#}c*e|e># zedcHNSazhQzTy+}${6jvbbVV)sA`Ald}q*Ie*ODn$w>#0mPVYQ zmLdz=ymnd~ggD_(#nj#LkbtN(3qKJ;I$+!tOmv{bbW7rZNXBgN}D(b zWo1?G+l7~w`%gaDR+POXh0zv=8zZ+gXoX~27YCDhoXJ5V{Y+$Ym&T$wd8l1Lo+Ydz zv~119^nE@nyO&Kb-ksu?(s{2Euo<7Kp>Ys&QOF^a_jA-WjIcPsLeH)T;XCy&rz}ja z*hcbvw3;BO3PQ#1n31COS?EBU5eHGo-2oH#Zq0?zjWYYZ=ZV1IsIvVsnqXUd`^6zb z+(JQ6)RT4r5&>JFC#wDM{$cEBIiXlg+cI z{OTsMH3QvRC+m~27N?cqEN6F=;dxorfrn|QuNTD=V-IYbgcH?@)MCscg*%bKuW02b zR)}LFi;hXN$|h}_$r30*8bsV z4^>gVgxw+~edfXBe`Mogp+7}$$-3Y_G5gR=ze0MdAHXRUYUx8|G?9 zkGZQ*Qd?X3!*)V!gR|{j&k8jaw1WP-iWzP^(E}%X$wIl)y(i1rqkg zTk7OUS{F)TnVh!rgy^yD-WC~YL{2b`aCPUeug~q7?#xukZR@~JqKN}8T14mf9-I# z;mG$Fj@#*$NZAi@YmuoRtEs8kQXgz3Fo-FmNrX2a@ytfy%syY>cS?-JBxwObZU85aX;?G=T8-? zqY3h1>gxhd*a8N0cw@i-%H4R6P&fHCH_Uu(@JBANiEJqN@KpoqVG1VZ(pOrM#G#il z-2N)3{QBLVDD!;XAu{KbwRz*A)xMDPDglPs45D}d1DK`~Dy&r_kDjv0X|wz6-k0ob zGEAxB-dH(5nY^F|`Rb=R^nvh<0E&j>D7oCyXp@+h35^h9?oi3Fj|wG5h(UfOn@UO) za*y8Q>SiCV7Pw7{4-BqnNBX1Bz^;>O3TFv>+vuA{Mh+(~zv{nWv6BlOrWEdzQS5P7+s0xpO>Q`hS%~)K@1o4 zvWNq9+h+G=NLJ;soh`HXT{e{`_)w|YKP%1if+_u*w1gzo`U_pcWl##Q%Ss+Q z5fHHEeZ|fTLyNP?jo1K_A;XkRW}HkvFy&Lr@6uY){T@n%OSzcX^_FrDMeGB z|5=_4T0Ul-e}l61;7RUMcUzxgg@g2<6ok`_iAi7>o`N$g6J-&+UgXt*SKb>ZJNQlq z*WR`~qUF&Id8*h5p}ScEr}W&hM9Vqj8=>QSY1GdBO&Y@X)46?ipWM)vO?vECqo9TI z`W7q=H8B=7nWK_liKzJMrLM)AJbiLUQv}N&%*YY)T!DX`-1|jiA=q%#a-Z&06f1@B zE7we9lG^GVH*^C93>7$t3wQ=eP*Y z$AV=tjq2C^2~^}$IGVb=C1r(MjtSgR{L-nV_SATyU+yB>-YH!yG#3s@ZbqtL(>L$d zy7~A%b_f<3Pk_ylN*=;h6hv|qp;dX~b(*o+aCW<%uRP;s;Mq+hXsKDNn^5;YCfCMp`=dbnLi9J$x=j>)#XLk$6>Hs*{FbV=+Ct ze^_ftQ|$8xt!2kYulo|?`S6_5R&F3ujbv7aWxlyuS-_lg$Mn89N#(Pelo8rn54n$uCO6ZE1Qp&mu%2(f)miK1SMe}LU_-Ode zi18mv^%l|wLNi&c0wB*iv8yJN#c0H!)th6b#tPxhSuDxhM~+UTGFmdUu+5HIH;a^` z*g8v_XK!nm*ENOKzdp)C57w>quPMsOuj~;-IqXVIovT}?yZ_UL3S&|5Kpr!0&IdwKw@SUR6NnDb zCC#=Q`hh+hw~YcDQl`djYdXLjE35J)zPMx87EOofo2Lvz28!rEW8wj$oo@AddyK$f z)s!qSVo44>0NBKXbY(z*(Mz6Km}_Oa|4wyoPG#(b8^!2{s?`lawLhH^d?Zl;%lav;oTg!8Sv`lzB=J4?8G%lE^ADV@OPe}j&z=#gYtj@6#uo< zq~PYQp<@O$xz+sKg_^{KTwwg&bQQcuX8}Hh19OQ&>EYc2kR`bR$mSXFl#yD^NX+FN zw34ked^UgT^2z!jC?CM4Bk%%Qt3IX-UMDkwzg#zgpM!=WJ7SK?3h?y#@z@FS2@{gD zgiK&MQZ(=~dCCTtOgNR$0A|bx058=Cdaw|kFkDD_sS^OX;*XabLGjIm>GcJh>Oc7e zE^vrF!6}SAJqH0ZpPKQ86(mkFA{-&kZhxEmcXaXiCznXd2TP9{qRIxjTy4 z(Ss%iZNlv{Y$v#d@-P%2?`HS@;i4?hO7&epw6)9gPdo3N^1iBGQtStBQFa0o?79!E zu^?SZ>5wTgUP6N+8!|l!2!KZ@_S{#Bc>L<4!aj zc+&m?9|FvP6ZRLbR0scU@Y3HA{trW)#moZ@~c5VmSrrQO)AR!cqvXj$b%#h}X!-6!sOqvwpdA5LSf@hx&4B zc00zLo|(&njPvz}AN^XW30nfJL!1!=EWl*DFp0vDQ{ zWJNb5K;<1C=hK(@tKU+Xf9Lx&(Oi6NV-?0N+Sn+3r?i*FO0S+kf8K}saZynT;&VB$ zgG6CqwqjFm%;2zw?%-tH9u3}<{5DBlK26Z4vbC-Wv4e;h!{X-!%^%$3MFLVAjtQP~ z7GGlP1%Mfp800(6_BHiPtdDwS2!c>zf2$^94tK^sTIV^8v;u{et$RMH# z<~?hFA1FB*Jr40rRXhZ3a4@oH_Yvg(1|2zs_NNo3*W;3$cg!0l0A%Yl5*>sWB(gWilaC3{ZBWz$z0ts zkUom_DSo{g%>0Sdh)R)X(MfMQVMIa{U-pgtqT-17;jxul7T)f7aFDycU9#p_wLfcZ zE1Kk2OUq}dpiiDFDUeqaG#{Dm=Ld~_ci0Llc4;b*k{{O2W1}01^EOpni7s%HoTv?# zdE@VGrYYB|dlGXh=B(lu3=>OPDRLqXsub$?YKvcCc>F>lT#hI&B#p$b9Jsh^|ZM#MNF}{him>I1)(abU_^> zxtD^9%T01gv^m%6(~6cP@EnV!(;>dfEqA*!fYrWLD`H!0$VFd@kR>bo$}ODWapld+ zP!$Xy+qT)p=8+|n)Ud^*(xfRAEob{2^s3-YwPRLcFr_R<H5_jsj;i zLL23D98Ik$Ph~new7zH>wh1fTYEU+CfAfWCRF(NMPz9Tjp=j{1^hV2=kxb5)ZWO3> zItt(2_UZU&VGE}($7E)pLMkl$LVeZdG(Gw*n^q+U zZ{x?N509Di?;WaM=XjHv$LS{c4?MWZm?pZ0srHTzakSQidS&jPa4Y!I^j%BvaLXX zzm}QqqQ1IF2S64Xz*p^d| zp3tEWOZLBbbkWF3f7QrI>joBKXFdT3RV^(CRcDxS^w29{ESM}`ey?~rv)bFJ3-aj8zz#YFfbBzj1 zqBSIZvAkPPXy`B4B;8<}qOSLi5SK+>y$fpT1!eQflIyd7%Vp_mdZN+x3(o$A?03>V zd76hpC;nS+z^}-udL0A=vhM4~Y|2aZ>4>pZprJpwV9*@K8yD@I`B&{+nQ3WTo$~`_ z)6IZir{KGq4jTFI;MiW!msO--&65k+unO~h#915}(>a6uYHwVwWA)KsN_lUO1HhNo zSG}<=fR%~n4baE04P7FgR1mniGj{0uPlW2BC)H7;jY)E#qdN39FPmVq^|j@MHd@LH ziC9*fM-XsH`3>9HW}2!px{K8cx>zRL2z9PjDJBl77QN%k%lrVgi(iPmy5LY%B$4sK zBlEDoV$29Yrqm98;A}*g#5UJVr!4VWuCA!n1%qMqN_=?uz?(=Rum{?5&2d+0_)=sE zCEDLM94batE()oHYsGAexgY)&*3yFOAUJZ&d)?WscyqM#7b-Q*D`>qTXmq?;|q4p*3JnBljagJt{HHVQFZ-71CX6dZ7iw&>Y$E~NQO>NlxW z{435>ybvmQ&G)?hJod5-Gk3cbS`r08FPWq8rx}0M=B0n4&Fn|PGiLQMk=(;iDqfCV zC>2R^@SP9ebRtRbJ^K1WY(e1KUBi-+RH-C04HmbUGcjke*g8zC<+AALfjFqVQ%d3r zP#;VnU8B4ZXLUSeKrPc%|B5wr0j<&-`hmA`F3m9cOQB_Y5YpxI?#UR!swUJbr<@aarN^> z5hndtyeR`{71QbU@2UIhr5TpCb~&_ut-@ZW|EkbV|4gA9FC`Jr9^RaJp!STT4NEdC z+;H2+o5sJCOVj?U4okZbEQr2=5Hv5Z5?XzKNt)@4{AD@DEp#cieyzlwh5f40d;dhE zKU_*8^1s!d)I9~6;gy}9@fDPpa%n%)UzK8P79`rxHD;5-168%c!zB+?F=y0EQ!M3D zZ2eY?VgIVq6Mw7p4>kY)cU77`zCiGE{J*#V3&Ok(>qFCU1bB1;@H9m1K|rHkkRJ0Z zcqg-`$&;@MG!Re=WJ+bt#vQx+w$JTg?LkD-IA!bNsT;_!+>Cebx4Y(hGoKDB^grPV z6gb|l5SZt9KVHoq(ir~y@eO3xS#_6Pp5X(f8A0CsB$NhCZV&v{qRKN&;!?*@PGWJR zQ27ulGM%XB)vs?UYigHtGhc(uPk_%5)r!pYQ0*<5)C>9`1%+D0g>R)pw7CSX-*~wm zD36S2*N3*~&axFO-B&@AsN0-n!CrA-BdN&aZ65l0%9zq-pEja@vDIS50-mq0OE<#T z8a^5aX0rfJr|Bn$k=;DhiC$Adgn zMJ8WAef_TJ+_b5wAlDbN)AJ75z6vv%SV?N*?mbo`OfGtQ)%WZ6O4;MpF;5(Wfn|bt z*qmgZcir=-P6cz1qu*Q$2!kOo0Ar47x%FEz4}5Wbh3vGk;(ZjVCZF?B>@;M{Zu#Ah zkju^2kx+Rx!nL{Bu(kQVMadD?AVIEq$~biAh`GVR~fVjVf$6EvS6 zk!&*~3YQXo5a8fwbrnwiLcz?|;6V<>-Z}-`i)oY#`m&5CEK}1QJffL3I?_ZVnN3XQ z<0tCakmBuicLhn|%~benbOvc8&@=FSAX`Rf_+a1QnQ={pe1S~_OIBXx zCRC?L(1s|enzTv->nuhc)=}>*E)Bo(9vEv+?{6&#`Ow439P0j>aAj=_2_|*5GTizl zKWUjmVB`varg(P>M3~plJy|iTc!7g&rwjecmrc2zD(s~Z)m$^+xv-gev9-mmFv=2B zM|?8WQR0{zD2u7M6Xa@|OGe~0al|SfgYC5I@x@{Lr+b3(GVh3Ko}YtKfK@(wSPE3^ zgR>E>l;HeUijPA%rpi5$>qiJL&OyxPbCW3cckNe6 zxceN`0~6%n?$$SsA13iyw;9^ZeUdHS(H6IP){sx9SXVMkgby# z&=B&`@ZFqbqnhy)+l(Y0ei)Cv0H%dK#nBS<>1EE~bvWxylfO%KHg1^Sk2dBpyp?Eq zt^X?(0z1y=z%Q^i17IC<0qcnHQKkuaxqBzOxB;}0E&Kt*Bpl3{`?x!_`neH!i?Psl zo!Xcypp`n7Qk=h>_X-629@Hpm`i38Cw1IJ1PzF|k*<@$n^d!UklZdqM%4J)R6I=2VOOf$G;7|&^9&QGXRhJ(WAO3_kiW{3Z4O8CJ59lBJB^&&2)l#7{jzQ;=5TQr;S}8#??A?G9wBRHnVvu_U!s zN({22yF{cerq@7IPI3B+ZKL|aRJCsWj{=Mm)ViORTTFIsXGEA$u_qd9P#OKJ|p{sQSOfSjQG3$Yew^U|cglgPdrITeV@7+0#?f+mAxq z7Vd7)E-Gp{tg3Vz+O9L^huIz*N*uTt>}Y~)=cBaMJ>6{ioxG)a~h>nmhAR#e7zBcG0njd+!|n(oE%0k1MLke?o0=vuj)EzmTu-$=%zF zaQ+px=6^6+AV6Vy)AJtM!T2RSsJ-YbjAh1KhNvix=o8$81mniBt}j6yGuqn_GcL^|brU91R+8K0GGgWlTdVTV1vAV`Gm zUU%!&>UriJHLpK_}N%fTrJ~Q*s zbuX$)j$P%!o4Gq?)N1h*>b5#)&@k(0jg{%eaZ^i*;^WHidT+iX?&bvOVTE_AWaIWX zL}NBohU>i~CRrX(eSHCGwP@I$b1@)@B~=DgFNiO+XL&@gMwZ}V|JpuI+67P3f21sN zb|j#%|3eTLlYe23X!+lnW4GS^dp$(7YMim&NTvfD@X(T{qg)Wnw5^pZF;|-u4Sl>uGSYOz96xD{DWVrXG0Sj@}8>3NOxL=^I zWT(}3qOsar7c)XQ&o?gD?0jCIXRe9=DigJYYig*&lJWm(Wa-B>745%(s+dL@^O z6CGu@8L6ih%K}YHIcfKC4ss&09RZm7WPfPqXgsRU;#R;*{2qw~*WX1*^Jnc2HM$H$ zjuBh$#Co~igdVJ1h;|FDWHvdOPviJ^6tbM{YXcM@DN4T=AQjs`RD2>N`OblcSZ}>X zBw?MZzjGSY@H@c|6iuD9{(^u2SMM+NylHBkqM=E9Q55Mq?RL*xsK*ST9*EzlI$=ou zV^NK^gztgGt zO9|3@QG!Sa1e97@hHZMdt~rnW6h>J;Yxu#u)-CV*9{erc+<^X=?^eeI<6lTUL7U+- zs$ZyD=13}jIkGY)-hoAReccF$S`G6?&FsdzB5pxatjDm@$b28pX&7dqX&MhA!@>xC#j`(6~AiPk5$0$O%3S*R!)zLm0IV&fky zsm!>oBzKC+jyI}iRv~|LCA4T=5K|SDwNu! z*tv(3sY$RP07CMu(t}wH8(qSV(Md4sZu@rKP6hsN9hldlOn!d)HiuGw6xOa5ai1gq@0YZFu&h=#X{LcyH}l5_uDz`-WaWpqIf4+)+x?S=FP&(T`wCvsEQ}}`hy~H}88>0yne0)H(`~Q2R zo!ffRU;OlafAMENe*C_;Ht&IBo0b_vO;V#}RLb2n9yH_C*5;I)CjQn#Px_FKV+GG%gXL6f+1amky^|exKa8(mLMI ze|maI+oR{V-Yl`XCHoNg<9cdo+$oQrh85*jx9Z8!2?g3>g-B>8~$hv$V`D_OUAa| z#;6E!iOED`|D_A8{nP5>(SR_)(w4TffwcdrVh~SRIp#AcK@^n~{*D$yaU$9^+-G3o z!OK8kjmg5@)j}#G&b2bZ7AZR_W|{flq>Kjt6BvZ|=fzdV&&nb0wLI9Zu1UGP0@;hP zor=xHg!+qaa6QlsM)`FhJXF>ASQ|rTS^TzZUYx z_?}Jb#UKdiG0-tDW~;%#*jr_Ka4`mw(E)cA)B?u$#`M@jz65?70dqqI2AgF5CXfBo zj#lyf2)VAkQ=&!Ac1ufA4r$k>me2Sw$#$K1|25$||2O-6GR1E3!d{i#T4SC?Sh|K} zkJoub0e=uk1}V47IJP;XNBHSrZ#Lwsw(}Ee_#$F+XHto>61Ph&>NRr|qRR*I{S_G_!*?H3ml9kU6FVZwI%wSW~PzI4D` z`lY}6>e_Qq*#J!5ddM7=G5A1tg5vCh5McKZe6=yMO$Tv+=UuQ}nZ0$l+U{SN0GM^9 zb4Ad3Rg@|%zkT7Q|L2v8-~NB4a}~n&8pZ)H=4?Fv zvItDK&-N-k2i=5~4f^QyDnIk|?U{lIs)_t`8)njT&}&awiCQH{r;svac3_w87uQ*i znfIEj7h^Jj`Om=*f%(DtgFnB)&z2$o%kSC2QO-djoU&vC94RY=-WM@P1>O*Ea#CW9 zO$~YR`NnsviGEvE^!X~_9V)EoBoi{$Z&j^v9kSNn1+-vr_-Y-nHIkhQWPlG?t!w;O z7oTAsdeUwS>P?n;S$9kT)z63naJl}D^R9OA=>d;KQDmM$)wb=EP- zpSbj-P#kWk0~={1$N~+$3NG8$^G!Ji#e`~W0xK&XeFPr_g6we>#6Ukg>`tKMoLk#M z=uR?6>-WzvfIFvy&zQ!0&9YpZMDvVyx>km^2m|b;93mUo*_Q9qZo&9@yZ{f~ z<_86pWA||a{klegTzq8lIU6E=?5;ASydOCs-+@PQ%JNs&rfk4(1M6pG_Yz@_(vvV- z;9ED&K~CGq(pEjo5&oVTG^G@A=AfmY@f1_+w*1EE17I#+3~?sK@l7u9sp5sz7CT3p zF2H_rBNlGZ(|rypU=>dnvWOV&qbol-kil6tGqLFJx;pvc!uKhbE^ODCt&1U`i>vqu zp~#np#Qk-P5!u86eTT8Wy#pB1*xfVEiwxnOttrMlYySKWe*6)0?=P<96qG_9sbI+% zofWox-yEEdrL>H5mLZl?(&0YpJPB60mf1=eN$|RAHBISgfXyqB;|QmAW72(j;IuNT zRU2OV8Dr#U=>LCtZxKd(uZtRShnr*{CN?&l8S4+MWCVLJ^z8|xWffX7pQWSTe#%Kt zQE`6-ZEk9xk;&%>-ys%T*Ibx-z-m5nVnV~9;7y_hSGLi|ehH%1{XJWON5~~Vr}Gpn zion8sYG7q5&E3OYHSp2A#R?C*xvN8a!2C%e2)f)JV>qVy8 z4OB`?yoeM94g^Wwm|id2w93pR-51;>>OcZ{2=p2L{HrqHoAq$gh|`4yiVk4k;8VFW zwBy-Xj7fh%teJ9)b~>dJ9iHPDH`CEY{N4BLO*psKn{2<_;27t_y|Q=5=!s|!job~) z`!ed}Xx+(O?qtd8Is%n5^$XhNMGI!1Hf*IRNo;cmH_qOVNV+EnyJrZRLCQsU_+jj-9UG6@Cpg+3S|C8Si00b+DZ%aG} z*e<$bIPn}*CE8eEZRVMZztd_ z`Z*|O{W?_mmU2fryCXR8P1Q&taW zSQW8rvzHaRXndSdp_&i5M;PEJBx`d*@P-@n>XD}fqMr=UosscY>-kYh9}|?F~3|h3BB|V(vQUk~%?_RMbaRPi`wajQ3I05}hFEBDs6G0`s5^ zW6K?qB9yC4x*xS}?Q?8C9jr9obn31SS&MNB057q{0jqA~t>ID$9ov3qNPA!yt}L)3 zkUK%lUH+sw64E&d3kT1+cs7s)Yzj0`>=ASU`+zC%0;?Rm7oBV2UMCcAG+JPpWMp@I z@cuhMuw$!_nITO=0Rn`BWx)YYndd7AG=;XFy98R|M=@J725G1S)`IFxh>$<2+Q)DW z0;8KiUgB>`>F^M|(kd8Gl-^78C^4_(lJ zjzZ#d_Ei!w941V$TVBYI#XKUA7gTnM^OO7!GofxYJ3W(ZA9pKcUuAfGb$co>AJ(&Z zJhoCjO}Xs{0|q@IM@&%{i#7peebYbztO2_SLAEtqGSw8?C7g2q7=rKV0namz-90=9 z(F7mkKx)KhF^_?n2!QKq7p^M+H4Z@l7%1oN1H0^LPr^bTbxZUuLD*($&kWnai<@v6 ziXFO$M%PSo+-IXsyN+HVETZWw_@%x)S8KofLyyhKx_8R*T_)Y3 z9C@(3&m``d_JRx9E)#cvt%2onS^Uw-9t#-a+hP{h;K~N@p1x_EMWf9gLm>rwZbnta8 z(eWDUuyXkBa{;fZEe^uHtkz-6soawAwbUZX;Utz`Q_{t)Y@Ay53VzCZYQ?%`V6@uy zQyCWL9Q%fZ>v0F=`yBfO{u}M(PD`g_XE@JumD*EOW4P^XGg68UdSxt6V!RDz*R3qs z9{EDny3^}zjEiFmR7 zOL?K&yP%w?5|CcnyH@ZPj&8RePLzrB%ANRS`G}7KONIG``x3oRw%(^bp+i}#Zu)a8 z{{t?7b*32bcR}4&Zy;ti;t3a64Ja}LA`DFh0F{{^B+Oc=1j@+rY5JMTjn}VeanZ1yY`eW zd*_%sQIS(u|0tSg7)>o1);@Y=n?b_(gh^UM&6o zGm3ou2UqOD5!)tge~>M}zkgyEWou8W)j;bc5#z|G{NC47_mLter0@)V;t(EVGoZE7 z+k`q(O=PNFPbbuyx#?^G32%^`e+Rz(mP>+S6UneMS5JCp*I%J_nnpD|5TYZWHU%5S z3vQeBtrvD~ZD!Kynd}d&Mmu`*Xe;3D$orlM$i;a8(jpIYKJS5c|uM@B5-z&d-pP=!GDX_jHiBaz0;j>e58(rQ#I?nKq( zMS^I73b*0FNMzLyK==oRa;8|N1sV$s*#Uk4C}i=J(2=v^2iQtP&tjfj2MC}3g77WG z{lVX*^sR$C|EIjS4y&^19)&lGf=GyjG%6sS(gIsVN~F8H1q7sHBT7mL2nZ;kbT>!{ z3J3zy-6h?zVFR1(H(MXyM}6Mwd*5@;^*jHZe}Mbuo|!eP*UXwV>rT#ydHqCr;g<0e z{&D4tj}rY7$7I#fh&2fz4lsWy*d{`_rT7m+k&mZdkd7#;nG%>t)QeKQ3Z;{wAsvt0 z!Zd5!H{UNvJawjDXqmB|V`-g=h6>KH%~L*U73O8MO0Qu&di!5!64s&bsziJC8{`*d zGW`cv;!cSU^9J+LZYZvgzOo*^Ldw%*M=xU;$`51z7*!TaPDZy`RYB5@yvvnxeyVw= zE=HJ^K?yk?E606|smVA((O!8uI&(rjVKUAU-&pmE0*QGi>H1*2;oZuKc&(v|4cDdn z$;f3V+04m{dCsRPocI`Lp$#ja1>ws23L*2*ONcmAycfsvui}QH_{KWT>8Bg4Eu}7o z#M{h~7~w0>7C4b)Y>zZfG4IHR$lPLUO8(N+Nlx3bisx&tE7&4|r(vU$p1)px7`-u? zQn~11PZKWKm00+TUiGz!bj~ z@Mo)j@5a`H&2f1jMVGQsh00R;e^?sJKW&AD_kCLn`SWy???(pN7w!bn=$WW#SbT$6 zI!ke{YW5WluS~FOFAAR1y;KAaZW&BILKS**l2~ZvpL3VM6H^;JP~5x!UucA5#hf0L z;inY#UuOAFA4HY5#rJw&GG(WN9g7biy|OF&dU{A7&qF=#grJV+&8Dfm?f%EQ?ra{MK%d#v_~&Y8_ZoUG38z88i%EVrBLe6AW#o4cQ5!<`gflCEq9=+Sl%iJWXQg z>`c`ti<)3_DrL3b^PAi~-^HCgSdxyI*ND7Z+uv>-WkXsz^2tJFv{~$aKRxD4mIy1}Xh;XOz=Pi~fwq*Z-`Yv!$U!lN=8<1~1RaqusYiP*1bCB6wjUBe0#IC@|fp?!0g{Onyv)V9X`6O#QLKBJsq^;h1$*{McJB|16j)Lgcu7OU|EI|eH(lZ zBsMVHWN?sAs_HSPeZhQ$Jij&>n`+(dxGOjpuHEc$$rJm{T;vx)q&*0prae=Bc-ZFQ zS%ojm>%Xesy4U$=a0h|9`o-ELw?9CR;YM84sK8gJmKRj2R{zmgiT_)V{2M-NPHClB z@3S7_pB}9o3)W2^z#Qtkmq*WzO-U@1&dT<&MRq3AbB(nVu2C5?*R|$8?PTpFDdL9= zzv`}Q53M7NVeF0XtB(vQ3o{U)Dn`V2lGZZVdNQ19?Uq%`;XTgLc~&0Xe+(4PpQQXZ zPAu$)6Z_@=+=-RSd_4DFyG{~6zEJc@s|Te|oR_X!&|zL%_!F6<_OJfjhm>Qrp*yFJ z7KyC4Z0W)kY(ED)gE<+EKS?=IJI{{#O!waFoX5)=$ zoV8a~9=2Q3EPv-ui1#?MBUk>VU)w_O!n!79sEBqEV@QM;wBzB8kZD)8o^f5|(e z^sum*dA@VgkAX$^Iw|1PDk-n5i5AfHc(fSPK91{X#FZe$O(rVX>Bg(+N`NPQ&LuFC z{e)^Hk$6Yxz-5?cPY{Z@;|)A^3DS6UGzUeyn}58hZD*}r<7;>cv~Kb4*3oVR0qRo9 zB@o$u%0MpzD1wSZf4(T(4HYZXJ$iCNzyabWrrq~)jh>q+Z)|G3dl`XO6xi6LTXZd*KqOj4SzCkv5gCbU?FT2r4m z0&=y-ZYTP@4~lUy=|>hP;1sj6n2K3oz!pTC~RfPE->p6Na_#fFuG7)or}6-^W;8&TsgR$e<0&i3Mme5)>g5Kc)Nz!QA}@ z*_BtV*=E>$W0@_wSz>?ux?|i!^MXkSF<}!+`z0xJ`b~48TP;q|8rG_}RZZsa?(->H zOC^TYM)5CQA-75R21{qBCm;E z1IroD{nUzK`O|kf#=4T@^hlk6?5rKVzg&a%OYBBdM&GzW6`QwOi z=Us0;Hh2un8ncOWu=Y8lcor^2eR9duU)Wy=E5rC`EGy8x=H_aR;L5|i`o$4gB^FOM zYMV0X;Y5YA$u+8b73Q6*p6=Mvobtw37*1dYQhf1ENze2dl|m|l=?Cl!+0EsON(iZJ zG4oim;ukNA8;#xynmQ=(p;jd*kW(VPU7xwoP7!s$(<=qqQ zvL32*-t?>Wqc?rdJynatlJj*0$4NDuK9MMgabPCW5B{>yeyrcHScb7Sc|k>=dordo zHl0OFA$6vj{KE>M7q((BY}up_y|~7RE+OV2>BUN@&dN+9e?JLB#ojqR$+JQA35Y}( zo1y#M)*z&b`D+d9w->B+Z&T1b0sTG&>7>=<%RB0E@y;%mdDijqgi`wKE%DdMxx0FN zdp!0J_t%sqw@+Psu*7To_(s&yZ+0+zsgzQz;QJUpdJuBO`aBn;?Zbx+gCxC{I9RcW zqT^cuQsKapji+Ct;>+R z7=U%|a~IaVb*4g`1Xt1wsJS2`<-h)?M8wzkRS_(ofT&?SHGoy!u4#kW@<9nSP71cH=U~ z1lfnPjAzeQF#aiK`U@oMeR;ylN>56qlY%?AUM8H}8yLu#eDx23l>RqOVcG@AmLDu9 zjr9*k(6D$mb5ca8z2-&{mxh^ab|q0B1nE31=f60~SEN6P7KfNUB!CJ~Plxac^df~| zJ~*52BCcOm+-jd@vqe-ZBL-@2yiDW|GR;i0C+(q^?AE;M)H?Ahr6p5+RaQq)LWf)f z=jq@*wZ7=phlbnvpe!8u1L!tTrpPEA3I{(IcD$(a*Y=&{1$6#jC#^dEC&b(mEl=aDJKgJf6%^ zzLM({v!E}EuX;iNYIM@}Hf$?$q8I;v7eL#*x9@%!!m!i)lUG^4IZSE4HyzfL{9D1! zFK#E7<%re^*H;*eCCAX0>5f|$5y_Wt;%H|+yz&q;%<25CLAn`H3)ZB{^5ON%7dCVi$Tk)0-9D@tN^6A$uP5N z;mR{R_Q_A}^v8M;Ym4o>v1}Z~R9EN(saXvFu)6vrt*SRZs3pBxxiM8V5TGI__a7;* zct8Wz1I3j7$tC$m^)!?DI;NwxT+$v7s-QN>bD!He%wEsC`Mi}Je?kT^n=x`N+>Cn3 z5@&MeakQe%YNOG^5h3T^T$Yq$W9y|iu=I>CpU4zB1vt6wzREWmrl|)G%zIsVs1n>ML6|CDwo^tz)O|TR)+Ob$v^jqjGtriQCYH zRyB(#$K@EOHE{B5dOz#a+_)8koR9sO9mQRt+ViNoG0u_&V+e{+dlbO?G z=^QJf1}-)AJQ?z%`gtjR7cI_f!2}VZY`Ki&26NUW9oiE!nNd*am=(Cd7Z?`fcM;Fz zqQ|(wU5-N@@2bQ$9@j$=mp5pqcrp5vmB#eP4aZttEFDQc)jeTmQy|y!BK(+>{d1aU z&+LgO_pO4Asy&8+fgtapzOx5jF{mC|$L_=76s3_*is|nDcnThDS%GrYDkPJ9FA!5$ zhc6h(~GMPqk^uYg5G>i#lvAu&^P!7Y0AI0 zbAf2Yl5lk=lSKMdK)I3jp!KuY`ieeFX$_VXZaxpT5T??Z-d`%)Yeors`njBC>C+=3i)2D!-#$m+`P0hyn|nxPor$1FFXNu((&D*zCr8^!~yn6`u<3tAxX%$bv>2|We<1sak@7) z3bgG_{CC}w)Faziv$u&>Ju{Lc&4%4d*TXL5LfE=m-{wYWM;Zz7fg6`6*UCO+uN5CD zrD0X{U%)LP>|JQZV|*y2z7Z{;c5ENQ{a#;JJBq`F3p+$rO#N+g9+~K=-tJTrj=&#T zPe1Ij+_Ri{f9#f}Xic>47Qq22IZwgGGg?R9LdJ`CV2gC#6oO*e4Zem=HYr<5SmGkv zV;}Y=1Vo~_)Rk9dSAesKq}NPrXq&AiD8CC^yyo_DBd7*O+^~I#_#4Ds?YX-K zd6wLJr&W1_4g9Y}*;?1da@c=L-B;tdRQaJLAnhDula{ zK#!m1E?fOlwvP1%s7qEO_4=WmdF^EbmlO6<-hSA0`MHu!DVCyfq-6GvX+~4lT~Eav zvBTqFQxOzL))+{2`&6@|5##Xiptx&zIP~Gr5R+w;;)%Eu(P}e+XAp|eyzZ#t{k=Wa zZY_Oewpg-`5!m4Lt{SQ=uw&1+>znGEVs$49h@i*L+XqPvpH8GhDmW2`Bodh;S$Zlf-N~Fi5aEe zARSYQ>G!bmJ}4+6F^@=q3SHl|&3Nl~6L{F&+MyEm-BjDJ%f|25G6cM*R-_x*7Zs;V z?9G3zd%JS*M(Botocuk=-7CdLGEYvjM0k$#&Om8wVhbBwGAE>s=d|XkVn^t&?3n55 z#q>C%3c3%q_K*yBZ#9N@kgn-b8IZ0ZeKwPW%7s5j^l|*P4g-ffON=)YYXE$CapUK_x@F}9^jSrRs9er->sELkpv`r zbhkL*r24%&$*XB;9jI=})yc<@vOL?!b3J|g*&2JAE_e&1%@425e~As6G!!2!X?3FS z-Dlb#WvETIs2kXX5KYN&eZi7c?!Dxy3>Ol*;ln6h-fMTCKX>7ERVQyjy=+QtEe$X` za9oVghQDq#w zNeGX)wp%o?&@jf=@YA%Yxv*Qx_YoQ@)%%abuuBDTmwGkxXpOZS#%p3h!l_u$t!pdJ z*)g`X1I*_o5UC*ltVhE>V-Al|#wFqyMV96G!jF2l;iRNcy8&Vobbz0gO@Mf{7Q;hR zIcAfjg$nq{L;LZASZr42@|gYeuo|Bi{3l1Ql1HzO%Un!1Wm9pxH^Lp1uUdOj>Ndj` zmegQOrh_JW)yMqppc+4iL$IB#PHheK#)fRglU<3rNDC*?ywtaTEF;#H!uk!P`zM0z z#$B!wWa23Xw+3^6<%0i-zga_fVi)Sf=z2Da;5qy6`Q16bY*F`C;cK>!UG(Q3nDHlg zu;-64)@#7$GMAU+LwnzPtI%YiAhqh9b|L2!t=Rc@MwD%5FQ+g&Eff<|E%6P_c@@~U zT;VL#8sO@rbLq+bPy(7&BGo^=Idcg?QC#__{6cKq0Rl#+l z>|Hr{TPb+uc9xdDz0j9ik>zYQx0C6TCGbg{ZuMZLs*!5O9f!XVh>Ol2+GBG6Y7j=? zxwi(pEaRdNy=Qk%10)wdO(Z!;3uzGMlY2FugOS&O*7VcS1k+&({nVd7tyh?QrL$xEN%@ z;4T-Sq@JMmIF5|o^wrGHCx|CieB6$~M~mIu35CKPDbHd(d%Bf(zAmDUs2jzozMY03 z_VB^Z8cN;14Lj!W5#Ocorr0Gae*$XT(CQuUj@0FTIhcrN#{1W=n>yJ~?v^pX2)cfi zTcAC>uJVjO9>j~CBR(k8{!iq8fpDG)$t60%n`DBf3%!YKvq zO}b`EQ*SAnh!VV8EMY7jK#CHiWUX;`Sfq1~x`fCo$zrHmo~%XRrmXtiX6CqyqRMk#+xZYHGd*&B$MB#aHs>xDcI6oPVQ+1=E5lr?m-Sb+sH8Sf zR{ApqiwDb5<_|igWMTXelB{pf4RQ6RQtx;>9IN5se zR`T$@Qab;lMWhg-68hARPKfcNRVE9~I=+Zz#LX8Cp6S7?An7c>64$4zFG00^FT+5Hn5S4YSky*`>1@2M zyk;Z9j?a9M*X3l0W4w^P^_1Jh*M)Y%G(~E;qB_&2Qn)jEazu})8Q<&zCvgjTo2OEvKjGEn62$OmDhGt2{A^sq>1&^)jdv7L7)H}w1D5I zE{U6@{y?pBW&R=9`46OSb*}nh8#D&k?BMdkbB!AU;}cT7pX*b@d|fO;Ax+zz@aD=G z>zb!YaCSaq$OJsr@g>-ycw9(uG9F|zQRvWod-*t^D8z)e=CN;g_G6?*^AgBlFSOf( z)tJExpI^Ft^()1{P8_f%_rd0j=Q=G_L|PvZDfk^6Nga{-Xg8)DFX$Y@26oAammb*>3cE3t8fIBYkQTHhmVK#{;uOW08vI`7>es&Yc{Ks}LU94JCuA zv6V_y1A}fV(w@Q)&dnaka8)wr z^QH`WhRmlIv@3{jnboXooE`Ns+d@L8sHwvP3KLB6VqaW1SI$~NC+>%Jv^AA=SF2Z< zm?irIhWB`)2l$6p4l@!Ovyt+q;Ij4A#D85YWLt2Y{&E8WzF zSXUhl7wc|3D4A^a$&u&C%f8mGr#}-Md3rOdycMf0eZL*`NQdS?jyuhEtmk7`){orRJnoaLutYQF983+6G#})9}$gx%7g^{J; zBB(2tpMsEnkfa%@lBip~5GWN#w)oC1g3(B=^o<}E=HulHEv!1B5Y{vEjT%sEuTBo_ z@@K=hl65bWn28mUpQBxs^1&`RW;**CR4q=&wMq0`<)1HDOw4kpDZl{!>h!tD;sS7J zw;vviJLWT?iJfJ;4&|YhGRXA_K3_OwkT$y^p!!O$<3cE0>hk8~7geVHsdw~VRFTTJ zDPCwoBMDciMcz9NxZX0VIlCVLHfO=6H$_#J`!o50O}u<(UV&Vm12-m<1nI(-k94PR zqL!`BbY*3XtdYuFX6vSJb7R@MXi$^OENHpXVQP@q42rOvnL15V?`=iC&)s}iVR&ql zmOu>ik(!;L(Wwb#Izy>j>$Rp;Sil(J<)Oyw)ci3vD4VW8UJpL>*wNVKM%rwfY2~}9 zXWix;zMW29(OAb-&}rrFo%4B~3h&QnvBgDyVo1)w{xYOGBOH+wUzU}|A>Ffcn*yDA z&o9$wmCQ_NAfS>D)-XiH3t>$>^(lPafrZqG@*=D{wXYKFbT_FES^a}fdlaV%L#w|* zh;`-9=&l_omrYp*yiOC3;j1<6jHGUZ0LS0@$`lA~Sy**nI@n#Ssf;rBH`Atz^5ltn zEt*%aXGzm z_0AsHG6fUd5H1lV#BvoOaJtQ9O@UcrfM=+kz{wY=^5z9C%b=oKfTH+n>!Ll<2!hS_ z2<$o>u>p#L%@jBK{M0t8QiwZ#l#rJGoB+_%XxN8(G zrLI=4wC0r{Iv6lD$MkF^z@Oe`f3l|WppMi0+?e518%I`l7ORW91X=g#P4R$8)ENx; zjmqV-ms=YT9_nYD?w8qkhB{SgKIIPk2I=R8u)(%35#;by$;c?lUd3p;QT%ns^<^Kg zu*2&-J+K(6sK|SExzCXzae79Tm353d%%X`R`!tmxPf2og|X~(Kadr<*%VyeiGR_k`F`OZon5P z2Nl)D=Fb=Q-lp0*FsM$)T@em$s4_fq9+-$b+sgeyTOQM{%jrKp0V-MJ4&y*EpvQ;X-ZtjSl@6{5Um4|ovX z7ar8q$IzXVJbWBE0bQqkelXgWiXRa=?-Ch&EcQH6ue4fb(_yH6#o`=&xrNl#nq=u~ z9xG3xqQE^|XJ7V&^uWg(@MWWHW{Y_HJUz@CJT85MICe>&aHLf{f`9Oq-O&hnBmTgF zlcrm<0#>o|<)H87f;lFyuSt#`Pu!iGEi*oVlP=cT2}C?_c`v2DJ1;L8JqAS4yL^nG z-i>He9^w{DGWbQqHAY`y_8JfXDbW|WF-JIYb((XU%|aqVbK}!^tHHbA=+yhVj5NJE z>xRpzRA$0y_;p0^R04(VfVyIY{g_uQtDC3J!3<6AZuA0d=9Yh?IZ3saFYyj;GE=YV zU|@oyu`QIp<6+a6C}_Q!Pmh;#x53LV^-RMXb)2(Fr#5TnZ)rUXV5FRWTQt{9(Bj9p zc2RxU;!2iGuW`hdItzvT*CUNkPmy%rU@?mWa^q(i?#bUEEuNymJ=(@Zx-ExeQF(azE1c%_;@Pu{Tc653BC; z6?a_ZVS0_rLlW|34o9#7(_15+tyow)P7RxhkV%lXPkp1e5PtEUosj`<^gHF`51uJD z%@mbY1;`ry!W6CdRSBuW0UZ}`-nGoq zK-9?Rze3DtD&Axx(xm%2#5zP*+dRhlChzyBoLKl(&4(#rx{6&XsjQVdX^mGIh{>3o z#cxB+`H?~q*fd84HA~R)I@z3^tQA7*V%j73L0aee0SR)j*z{O9#YPcvHx98mTp>59 zDrz^1OoC;-csT)`X1`6!DWab?Hc^qTN#NsIN2!b5*>RyYM_^`}(;%9Nwxq^Xms1 zHN03TaFOR2W9zB&9YV$WgJX@!*Rh!?4=RT=ipOuKxH6eyr!8ZVh(es38c8+j#VVX# z^6Q-|;U9y;XrNCvm`r&L3j?ce6QU1q$xc`&i@MvEqO9nZX zRv8JAx3N@jlL;4vm*ZVgu@gI3g;iWbbG79$5o zfsZ3mI3Q0AF-W^z>V5MYL=;YoBnCTtnu@9H}YVhVO!Y^h*Mq&arrk$RW9s0>#0Q^*tQ&=>KaCoN(aK0D#2c` z|FZ@ZpEQy!5)amGCkOT$ufUKt?`m{lZOqT6CjZH2mEHe0hrP9 zgeL{)SLbKHi@wlp~$C^=Hd<4lVvPZYmzx~9+eVe4h~3mQ4uF(8 zsywTz1l7?ON7|U4pSFW28=#RKNNAvc9(tVv7Lc z!^6m4HI_!%F&>W(M?o2`FJ`3=UVVeSfQ=KO)SP_J_U{v5=F%c$y*1r63*PlP&{YC= z(pMR|r~M5=g48c`Pb65U#alVf*U|N}FHBKE}zMc%I0S#1c zj!Z)hrtGFkpoliG`Za-wvM&Jpw=;5}@h&47Q!TY&baxk`LuPfZhZ8=INmXYO@GPlrE4c zbxC$;Q3vo&%a@QDDG0?!(9Kn#R2aUCQ^R($6{z(A+RCEa=Dtf+J>CcX-h;J+Q&~^< zBv6YbpdUHVk2I1ok@k=;5_uUdc~4bBVfyjL;4Ev$kt}Rhq!(J951Wbp3F}nIX(w0# z!i)#7re$QoM_p+To;JLIAsIaxk$XeXAH$(R-MR}joo6fR0w)YH$RC$7%rF8oj5nU% zer=Vq*A0O9$N-ck8e0WdP#$Zv2>SPEJ>VagGPE#KR3ujnYjgpJbGt+Wd&kX=?TG_(>id;FSLaJ=#>zx z2cdfv>}^;=UmNRx3-qempL1$_KnuVHn)^aaB>2>&2>JJ?pxi}7#b4iQ1h7N%#hQ-$ z*HvO#J?WLpqown&GqHUcW0-;9r>C$~?oZz9*X*~0Q(kyF6pu|GdK-I!pyVs9>E}YZ zPYJbUWxZ<8UF2e`Vrci8&+MYCGGH@z3f*-n>?`uE{OtUMr^kxSWu2&{`y^ezK`Q+# zvQ(3#92AOfl?m4{y{+K?w{=;5E;RnH{$BF-csMk5W>_DeI{#~k%(~tKtUa6MyjMHO z8x9cLm*m`IA|tfx!1A52GaGa@cOeVMlN~A8tg@frq(Vf`2}j8oVGChBx3=9g-Q{k4 zd^>}<{Z6QkxSW~_u}U9~j#WB4Q+fu8Sn+l5vZa*j8HxL@fR~lk(eC*P_Rs1(m z$_{qU=|0KKh5^=?nNEpbPrPO-#^i9jarCU&mCvuj`w(m>zC>o`X+BR+-Jtl ze)X2^IAcWLL(;{YME-P)bmPfyi{GX+jRiGL6BMT9Bq?$*#a(>&DoGQQiF13uEHt)3YYH9Pek}{Ub<25 zymHHle8&4}`-{QV8xFykLt5|yXAWrVEQSsh--@?@1(R+g~WdHk|SEWY0L>xmkx@wuCi>UfExt;Z8s4;7A+ z!(kvzzRHz!m=f8sQuoreP-@*W<=&edM3X|_7Gnk1)THc#H=FVq)lqf>yBDMSBOHgR zhPm_IBNd116law)Z%yd7ddulOy)OC*pO`4@m6KzCXu4g&qXr^(|7pG93TkQxd!0F~ z@F?SVAwl;`sY>_hpaGXs^Q)^Wr@68M!{deBDmM12zIvW(4Suxd{;0ZcD4JbCEp&8V zFOZGr!af-ZZ_THCiiOJOC#XS3qAxEE<89z=NZWv+wEpr)Tz9F(mO1NvFFXZC+l2G? zTKB>`{RA=Rvof@UbSj3$m{x9%gwPDKS*pEbQa2YOA5WH{zS$4GcQ3SJXbc4@ULDq~ zYzS&$fInp19kTxhU1adFLiV6E9wsRUbtN1&ns*6!O3!4~G! zf_sYDX32Dr<(Kj^Sy^?uZ&TM$v$OCIf`73`~GUw+%k zG}Uqme#zLRf{)*B%r}UQZ|r+$2sLp%uM0@OaCr@mIsK79^?&fJtT*23X3JOJQNtXS z<^osYl9ZE#xEs8ts;ulZe(3CMN~a?iHTZmE66};fxDJdpS!;*9F--H;Zmx#ptnw4- zv0R87XeyOb_G=b-;2efeAW0&JA_;Fe#jROdumz@xbs_JodWMEd=3r7kY*&nvs7ZZE zM>H zMZ15~HYFB#7G!DQ3Hq<6tkWkUD67*gWY8~d;)p8-fU{o?5<2aicY%eQ2ZvyOeFDdr zN}!#zz@9$`S2^b!ejn9o?iFh$0Lhl%6nE^$jF@4wI-kHs3xD+yC;+w-3L9~cT#TGY zNy75nixwk+&HcP6d8Wnzq0tNL=`QruXh51o{`!z|Ktp#FKaT<{_?p~?-$#t<>>HK_ z^>wD8$K>$t_Wo~>Fc8r-Rz;!$eVliJ+cr-OlIXaphkvwk{Hs-$%CID8HTQQb5PRXw zLYFOvp{$B1ZpD);gp;!>18rvXl>gg?nP4lN9O;Qf{eAjlq7YNEjr2N1@ zBXI?Yf^jxr|LRy8$qJDECQT2N^09*=K0((#PtnsseE{(lukRsBids-(7|Hq(pjN6H zWSxz{c0vsinom%H;m)X|pEy`MG>kkVIKd+NJ|8mw*SgT{BCJz9;ic0(Yrt=4{sAna zs_?T5*eJ)9QRRa8&Nx4aBqgi_uI}xy6z%LEt zz}XB8fQ`JL5J1;+CjmAjVJ36X1VAx!JN*pT@T^L~tZWG6JfwgVahnnqSnv}Outty> z%LnN32~Cfz(|Fw)B_Ll{GVSpy8Hd^Qm5#nZG>>@P& z*uFuIXi^Sb4E~yt*)V!W%(1v!9|(t|6i94Qm$YHawt!aPDs*XZDAD@wr>xW0nl-3; z2~_)cl51p;nEv*Eur%|O#^IQf4t$~luT!)u@{AOMec_woz0{vdG#NV5k>^Ib>d z2pr+ugf80wL`sYw#q2xz7wT9Wtb)eIRpPiyf8@qLi zcf53ZT>70Zn$J;rV#_tmHj&8dZVlhZTkZG@R)xbKV`#$w4ven>2)=9)rId)p=kLp1 zEM^pY{#m4oafDP;tWDEfOl*4YwW%sq(}SF>J6@!Q$tuv9`+4xNK7Dr datetime.datetime.utcnow()) - dag_id = 'test_start_date_scheduling' - dag = self.dagbag.get_dag(dag_id) - dag.clear() - self.assertTrue(dag.start_date > datetime.datetime.utcnow()) - - scheduler = SchedulerJob(dag_id, - num_runs=2) - scheduler.run() + scheduler = SchedulerJob(dag_id, + num_runs=2) + scheduler.run() - # zero tasks ran - session = settings.Session() - self.assertEqual( - len(session.query(TI).filter(TI.dag_id == dag_id).all()), 0) + # zero tasks ran + self.assertEqual( + len(session.query(TI).filter(TI.dag_id == dag_id).all()), 0) + session.commit() - # previously, running this backfill would kick off the Scheduler - # because it would take the most recent run and start from there - # That behavior still exists, but now it will only do so if after the - # start date - backfill = BackfillJob( - dag=dag, - start_date=DEFAULT_DATE, - end_date=DEFAULT_DATE) - backfill.run() + # previously, running this backfill would kick off the Scheduler + # because it would take the most recent run and start from there + # That behavior still exists, but now it will only do so if after the + # start date + backfill = BackfillJob( + dag=dag, + start_date=DEFAULT_DATE, + end_date=DEFAULT_DATE) + backfill.run() - # one task ran - session = settings.Session() - self.assertEqual( - len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) + # one task ran + self.assertEqual( + len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) + session.commit() - scheduler = SchedulerJob(dag_id, - num_runs=2) - scheduler.run() + scheduler = SchedulerJob(dag_id, + num_runs=2) + scheduler.run() - # still one task - session = settings.Session() - self.assertEqual( - len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) + # still one task + self.assertEqual( + len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) + session.commit() def test_scheduler_task_start_date(self): """ @@ -3006,7 +3020,9 @@ def test_scheduler_run_duration(self): logging.info("Test ran in %.2fs, expected %.2fs", run_duration, expected_run_duration) - self.assertLess(run_duration - expected_run_duration, 5.0) + # 5s to wait for child process to exit and 1s dummy sleep + # in scheduler loop to prevent excessive logs. + self.assertLess(run_duration - expected_run_duration, 6.0) def test_dag_with_system_exit(self): """ @@ -3030,9 +3046,9 @@ def test_dag_with_system_exit(self): subdir=dag_directory, num_runs=1) scheduler.run() - session = settings.Session() - self.assertEqual( - len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) + with create_session() as session: + self.assertEqual( + len(session.query(TI).filter(TI.dag_id == dag_id).all()), 1) def test_dag_get_active_runs(self): """ @@ -3168,8 +3184,8 @@ def setup_dag(dag_id, schedule_interval, start_date, catchup): def test_add_unparseable_file_before_sched_start_creates_import_error(self): + dags_folder = mkdtemp() try: - dags_folder = mkdtemp() unparseable_filename = os.path.join(dags_folder, TEMP_DAG_FILENAME) with open(unparseable_filename, 'w') as unparseable_file: unparseable_file.writelines(UNPARSEABLE_DAG_FILE_CONTENTS) @@ -3177,8 +3193,8 @@ def test_add_unparseable_file_before_sched_start_creates_import_error(self): finally: shutil.rmtree(dags_folder) - session = settings.Session() - import_errors = session.query(models.ImportError).all() + with create_session() as session: + import_errors = session.query(models.ImportError).all() self.assertEqual(len(import_errors), 1) import_error = import_errors[0] @@ -3188,8 +3204,8 @@ def test_add_unparseable_file_before_sched_start_creates_import_error(self): "invalid syntax ({}, line 1)".format(TEMP_DAG_FILENAME)) def test_add_unparseable_file_after_sched_start_creates_import_error(self): + dags_folder = mkdtemp() try: - dags_folder = mkdtemp() unparseable_filename = os.path.join(dags_folder, TEMP_DAG_FILENAME) self.run_single_scheduler_loop_with_no_dags(dags_folder) @@ -3199,8 +3215,8 @@ def test_add_unparseable_file_after_sched_start_creates_import_error(self): finally: shutil.rmtree(dags_folder) - session = settings.Session() - import_errors = session.query(models.ImportError).all() + with create_session() as session: + import_errors = session.query(models.ImportError).all() self.assertEqual(len(import_errors), 1) import_error = import_errors[0] @@ -3220,8 +3236,8 @@ def test_no_import_errors_with_parseable_dag(self): finally: shutil.rmtree(dags_folder) - session = settings.Session() - import_errors = session.query(models.ImportError).all() + with create_session() as session: + import_errors = session.query(models.ImportError).all() self.assertEqual(len(import_errors), 0) @@ -3293,8 +3309,8 @@ def test_remove_file_clears_import_error(self): # Rerun the scheduler once the dag file has been removed self.run_single_scheduler_loop_with_no_dags(dags_folder) - session = settings.Session() - import_errors = session.query(models.ImportError).all() + with create_session() as session: + import_errors = session.query(models.ImportError).all() self.assertEqual(len(import_errors), 0) diff --git a/tests/models.py b/tests/models.py index 8375a95a30760..9e09f76a43900 100644 --- a/tests/models.py +++ b/tests/models.py @@ -23,43 +23,44 @@ from __future__ import unicode_literals import datetime +import inspect import logging import os -import pendulum -import unittest -import time -import six import re -import urllib import textwrap -import inspect +import time +import unittest +import urllib +from tempfile import NamedTemporaryFile, mkdtemp -from airflow import configuration, models, settings, AirflowException +import pendulum +import six +from mock import ANY, Mock, patch +from parameterized import parameterized + +from airflow import AirflowException, configuration, models, settings from airflow.exceptions import AirflowDagCycleException, AirflowSkipException from airflow.jobs import BackfillJob +from airflow.models import Connection from airflow.models import DAG, TaskInstance as TI -from airflow.models import DagRun -from airflow.models import State as ST from airflow.models import DagModel, DagRun, DagStat -from airflow.models import clear_task_instances -from airflow.models import XCom -from airflow.models import Connection -from airflow.models import SkipMixin from airflow.models import KubeResourceVersion, KubeWorkerIdentifier -from airflow.jobs import LocalTaskJob -from airflow.operators.dummy_operator import DummyOperator +from airflow.models import SkipMixin +from airflow.models import State as ST +from airflow.models import XCom +from airflow.models import clear_task_instances from airflow.operators.bash_operator import BashOperator +from airflow.operators.dummy_operator import DummyOperator from airflow.operators.python_operator import PythonOperator from airflow.operators.python_operator import ShortCircuitOperator from airflow.operators.subdag_operator import SubDagOperator from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep from airflow.utils import timezone -from airflow.utils.weight_rule import WeightRule +from airflow.utils.dag_processing import SimpleTaskInstance +from airflow.utils.db import create_session from airflow.utils.state import State from airflow.utils.trigger_rule import TriggerRule -from mock import patch, Mock, ANY -from parameterized import parameterized -from tempfile import mkdtemp, NamedTemporaryFile +from airflow.utils.weight_rule import WeightRule DEFAULT_DATE = timezone.datetime(2016, 1, 1) TEST_DAGS_FOLDER = os.path.join( @@ -1710,31 +1711,28 @@ def test_process_file_with_none(self): self.assertEqual([], dagbag.process_file(None)) @patch.object(TI, 'handle_failure') - def test_kill_zombies(self, mock_ti): + def test_kill_zombies(self, mock_ti_handle_failure): """ Test that kill zombies call TIs failure handler with proper context """ dagbag = models.DagBag() - session = settings.Session - dag = dagbag.get_dag('example_branch_operator') - task = dag.get_task(task_id='run_this_first') - - ti = TI(task, datetime.datetime.now() - datetime.timedelta(1), 'running') - lj = LocalTaskJob(ti) - lj.state = State.SHUTDOWN - - session.add(lj) - session.commit() - - ti.job_id = lj.id - - session.add(ti) - session.commit() - - dagbag.kill_zombies() - mock_ti.assert_called_with(ANY, - configuration.getboolean('core', 'unit_test_mode'), - ANY) + with create_session() as session: + session.query(TI).delete() + dag = dagbag.get_dag('example_branch_operator') + task = dag.get_task(task_id='run_this_first') + + ti = TI(task, DEFAULT_DATE, State.RUNNING) + + session.add(ti) + session.commit() + + zombies = [SimpleTaskInstance(ti)] + dagbag.kill_zombies(zombies) + mock_ti_handle_failure \ + .assert_called_with(ANY, + configuration.getboolean('core', + 'unit_test_mode'), + ANY) def test_deactivate_unknown_dags(self): """ diff --git a/tests/utils/test_dag_processing.py b/tests/utils/test_dag_processing.py index f29e384b8c657..94c39feb4eede 100644 --- a/tests/utils/test_dag_processing.py +++ b/tests/utils/test_dag_processing.py @@ -17,11 +17,26 @@ # specific language governing permissions and limitations # under the License. +import os import unittest +from datetime import timedelta from mock import MagicMock -from airflow.utils.dag_processing import DagFileProcessorManager +from airflow import configuration as conf +from airflow.jobs import DagFileProcessor +from airflow.jobs import LocalTaskJob as LJ +from airflow.models import DagBag, TaskInstance as TI +from airflow.utils import timezone +from airflow.utils.dag_processing import (DagFileProcessorAgent, DagFileProcessorManager, + SimpleTaskInstance) +from airflow.utils.db import create_session +from airflow.utils.state import State + +TEST_DAG_FOLDER = os.path.join( + os.path.dirname(os.path.realpath(__file__)), os.pardir, 'dags') + +DEFAULT_DATE = timezone.datetime(2016, 1, 1) class TestDagFileProcessorManager(unittest.TestCase): @@ -29,10 +44,12 @@ def test_set_file_paths_when_processor_file_path_not_in_new_file_paths(self): manager = DagFileProcessorManager( dag_directory='directory', file_paths=['abc.txt'], - parallelism=1, - process_file_interval=1, max_runs=1, - processor_factory=MagicMock().return_value) + processor_factory=MagicMock().return_value, + signal_conn=MagicMock(), + stat_queue=MagicMock(), + result_queue=MagicMock, + async_mode=True) mock_processor = MagicMock() mock_processor.stop.side_effect = AttributeError( @@ -48,10 +65,12 @@ def test_set_file_paths_when_processor_file_path_is_in_new_file_paths(self): manager = DagFileProcessorManager( dag_directory='directory', file_paths=['abc.txt'], - parallelism=1, - process_file_interval=1, max_runs=1, - processor_factory=MagicMock().return_value) + processor_factory=MagicMock().return_value, + signal_conn=MagicMock(), + stat_queue=MagicMock(), + result_queue=MagicMock, + async_mode=True) mock_processor = MagicMock() mock_processor.stop.side_effect = AttributeError( @@ -62,3 +81,107 @@ def test_set_file_paths_when_processor_file_path_is_in_new_file_paths(self): manager.set_file_paths(['abc.txt']) self.assertDictEqual(manager._processors, {'abc.txt': mock_processor}) + + def test_find_zombies(self): + manager = DagFileProcessorManager( + dag_directory='directory', + file_paths=['abc.txt'], + max_runs=1, + processor_factory=MagicMock().return_value, + signal_conn=MagicMock(), + stat_queue=MagicMock(), + result_queue=MagicMock, + async_mode=True) + + dagbag = DagBag(TEST_DAG_FOLDER) + with create_session() as session: + session.query(LJ).delete() + dag = dagbag.get_dag('example_branch_operator') + task = dag.get_task(task_id='run_this_first') + + ti = TI(task, DEFAULT_DATE, State.RUNNING) + lj = LJ(ti) + lj.state = State.SHUTDOWN + lj.id = 1 + ti.job_id = lj.id + + session.add(lj) + session.add(ti) + session.commit() + + manager._last_zombie_query_time = timezone.utcnow() - timedelta( + seconds=manager._zombie_threshold_secs + 1) + zombies = manager._find_zombies() + self.assertEquals(1, len(zombies)) + self.assertIsInstance(zombies[0], SimpleTaskInstance) + self.assertEquals(ti.dag_id, zombies[0].dag_id) + self.assertEquals(ti.task_id, zombies[0].task_id) + self.assertEquals(ti.execution_date, zombies[0].execution_date) + + session.query(TI).delete() + session.query(LJ).delete() + + +class TestDagFileProcessorAgent(unittest.TestCase): + def test_parse_once(self): + def processor_factory(file_path, zombies): + return DagFileProcessor(file_path, + False, + [], + zombies) + + test_dag_path = os.path.join(TEST_DAG_FOLDER, 'test_scheduler_dags.py') + async_mode = 'sqlite' not in conf.get('core', 'sql_alchemy_conn') + processor_agent = DagFileProcessorAgent(test_dag_path, + [test_dag_path], + 1, + processor_factory, + async_mode) + processor_agent.start() + parsing_result = [] + while not processor_agent.done: + if not async_mode: + processor_agent.heartbeat() + processor_agent.wait_until_finished() + parsing_result.extend(processor_agent.harvest_simple_dags()) + + dag_ids = [result.dag_id for result in parsing_result] + self.assertEqual(dag_ids.count('test_start_date_scheduling'), 1) + + def test_launch_process(self): + def processor_factory(file_path, zombies): + return DagFileProcessor(file_path, + False, + [], + zombies) + + test_dag_path = os.path.join(TEST_DAG_FOLDER, 'test_scheduler_dags.py') + async_mode = 'sqlite' not in conf.get('core', 'sql_alchemy_conn') + + log_file_loc = conf.get('core', 'DAG_PROCESSOR_MANAGER_LOG_LOCATION') + try: + os.remove(log_file_loc) + except OSError: + pass + + # Starting dag processing with 0 max_runs to avoid redundent operations. + processor_agent = DagFileProcessorAgent(test_dag_path, + [], + 0, + processor_factory, + async_mode) + manager_process = \ + processor_agent._launch_process(processor_agent._dag_directory, + processor_agent._file_paths, + processor_agent._max_runs, + processor_agent._processor_factory, + processor_agent._child_signal_conn, + processor_agent._stat_queue, + processor_agent._result_queue, + processor_agent._async_mode) + if not async_mode: + processor_agent.heartbeat() + + manager_process.join() + + self.assertTrue(os.path.isfile(log_file_loc)) From fa8ffdde364c6ad4a47b71d5d0d46f50d85d664f Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 7 Nov 2018 11:07:15 +0000 Subject: [PATCH 210/808] [AIRFLOW-XXX] Pin Flask App Builder to Fix CI (#4150) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index ee3935b76c271..3af146d5a301d 100644 --- a/setup.py +++ b/setup.py @@ -275,7 +275,7 @@ def do_setup(): 'croniter>=0.3.17, <0.4', 'dill>=0.2.2, <0.3', 'flask>=0.12.4, <0.13', - 'flask-appbuilder>=1.12, <2.0.0', + 'flask-appbuilder==1.12.1', 'flask-admin==1.4.1', 'flask-caching>=1.3.3, <1.4.0', 'flask-login>=0.3, <0.5', From c4a7159ac126d66c4a8e06bec5b13c7b1683a3b1 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 13 Nov 2018 00:32:19 +0100 Subject: [PATCH 211/808] [AIRFLOW-3220] Add Instance Group Manager Operators for GCE (#4167) --- .gitignore | 6 + .../example_dags/example_gcp_compute.py | 36 +- .../example_dags/example_gcp_compute_igm.py | 143 ++++ .../example_gcp_function_delete.py | 6 +- .../example_gcp_function_deploy_delete.py | 23 +- airflow/contrib/hooks/gcp_compute_hook.py | 198 ++++- airflow/contrib/hooks/gcp_function_hook.py | 4 +- .../contrib/operators/gcp_compute_operator.py | 276 ++++++- .../operators/gcp_function_operator.py | 8 +- airflow/contrib/utils/gcp_field_sanitizer.py | 162 ++++ docs/howto/operator.rst | 298 +++++++- docs/integration.rst | 46 +- setup.py | 1 + .../operators/test_gcp_compute_operator.py | 718 +++++++++++++++++- 14 files changed, 1790 insertions(+), 135 deletions(-) create mode 100644 airflow/contrib/example_dags/example_gcp_compute_igm.py create mode 100644 airflow/contrib/utils/gcp_field_sanitizer.py diff --git a/.gitignore b/.gitignore index 0e53aaded07b6..5712aee6c549a 100644 --- a/.gitignore +++ b/.gitignore @@ -138,3 +138,9 @@ rat-results.txt *.generated *.tar.gz scripts/ci/kubernetes/kube/.generated/airflow.yaml + +# Airflow log files when airflow is run locally +airflow-*.err +airflow-*.out +airflow-*.log +airflow-*.pid diff --git a/airflow/contrib/example_dags/example_gcp_compute.py b/airflow/contrib/example_dags/example_gcp_compute.py index e4abe2e152a00..51a55b6a99053 100644 --- a/airflow/contrib/example_dags/example_gcp_compute.py +++ b/airflow/contrib/example_dags/example_gcp_compute.py @@ -24,7 +24,7 @@ This DAG relies on the following Airflow variables https://airflow.apache.org/concepts.html#variables * PROJECT_ID - Google Cloud Platform project where the Compute Engine instance exists. -* LOCATION - Google Cloud Platform zone where the instance exists. +* ZONE - Google Cloud Platform zone where the instance exists. * INSTANCE - Name of the Compute Engine instance. * SHORT_MACHINE_TYPE_NAME - Machine type resource name to set, e.g. 'n1-standard-1'. See https://cloud.google.com/compute/docs/machine-types @@ -37,19 +37,23 @@ from airflow.contrib.operators.gcp_compute_operator import GceInstanceStartOperator, \ GceInstanceStopOperator, GceSetMachineTypeOperator -# [START howto_operator_gce_args] -PROJECT_ID = models.Variable.get('PROJECT_ID', '') -LOCATION = models.Variable.get('LOCATION', '') -INSTANCE = models.Variable.get('INSTANCE', '') -SHORT_MACHINE_TYPE_NAME = models.Variable.get('SHORT_MACHINE_TYPE_NAME', '') -SET_MACHINE_TYPE_BODY = { - 'machineType': 'zones/{}/machineTypes/{}'.format(LOCATION, SHORT_MACHINE_TYPE_NAME) -} +# [START howto_operator_gce_args_common] +PROJECT_ID = models.Variable.get('PROJECT_ID', 'example-airflow') +ZONE = models.Variable.get('ZONE', 'europe-west1-b') +INSTANCE = models.Variable.get('INSTANCE', 'test-instance') default_args = { 'start_date': airflow.utils.dates.days_ago(1) } -# [END howto_operator_gce_args] +# [END howto_operator_gce_args_common] + +# [START howto_operator_gce_args_set_machine_type] +SHORT_MACHINE_TYPE_NAME = models.Variable.get('SHORT_MACHINE_TYPE_NAME', 'n1-standard-1') +SET_MACHINE_TYPE_BODY = { + 'machineType': 'zones/{}/machineTypes/{}'.format(ZONE, SHORT_MACHINE_TYPE_NAME) +} +# [END howto_operator_gce_args_set_machine_type] + with models.DAG( 'example_gcp_compute', @@ -59,7 +63,7 @@ # [START howto_operator_gce_start] gce_instance_start = GceInstanceStartOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, task_id='gcp_compute_start_task' ) @@ -67,14 +71,14 @@ # Duplicate start for idempotence testing gce_instance_start2 = GceInstanceStartOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, task_id='gcp_compute_start_task2' ) # [START howto_operator_gce_stop] gce_instance_stop = GceInstanceStopOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, task_id='gcp_compute_stop_task' ) @@ -82,14 +86,14 @@ # Duplicate stop for idempotence testing gce_instance_stop2 = GceInstanceStopOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, task_id='gcp_compute_stop_task2' ) # [START howto_operator_gce_set_machine_type] gce_set_machine_type = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, body=SET_MACHINE_TYPE_BODY, task_id='gcp_compute_set_machine_type' @@ -98,7 +102,7 @@ # Duplicate set machine type for idempotence testing gce_set_machine_type2 = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=INSTANCE, body=SET_MACHINE_TYPE_BODY, task_id='gcp_compute_set_machine_type2' diff --git a/airflow/contrib/example_dags/example_gcp_compute_igm.py b/airflow/contrib/example_dags/example_gcp_compute_igm.py new file mode 100644 index 0000000000000..dc24259f9f119 --- /dev/null +++ b/airflow/contrib/example_dags/example_gcp_compute_igm.py @@ -0,0 +1,143 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Example Airflow DAG that uses IGM-type compute operations: +* copy of Instance Template +* update template in Instance Group Manager + +This DAG relies on the following OS environment variables + +* PROJECT_ID - the Google Cloud Platform project where the Compute Engine instance exists +* ZONE - the zone where the Compute Engine instance exists + +Variables for copy template operator: +* TEMPLATE_NAME - name of the template to copy +* NEW_TEMPLATE_NAME - name of the new template +* NEW_DESCRIPTION - description added to the template + +Variables for update template in Group Manager: + +* INSTANCE_GROUP_MANAGER_NAME - name of the Instance Group Manager +* SOURCE_TEMPLATE_URL - url of the template to replace in the Instance Group Manager +* DESTINATION_TEMPLATE_URL - url of the new template to set in the Instance Group Manager +""" + +import os +import datetime + +import airflow +from airflow import models +from airflow.contrib.operators.gcp_compute_operator import \ + GceInstanceTemplateCopyOperator, GceInstanceGroupManagerUpdateTemplateOperator + +# [START howto_operator_compute_igm_common_args] +PROJECT_ID = os.environ.get('PROJECT_ID', 'example-project') +ZONE = os.environ.get('ZONE', 'europe-west1-b') + +default_args = { + 'start_date': airflow.utils.dates.days_ago(1) +} +# [END howto_operator_compute_igm_common_args] + +# [START howto_operator_compute_template_copy_args] +TEMPLATE_NAME = os.environ.get('TEMPLATE_NAME', 'instance-template-test') +NEW_TEMPLATE_NAME = os.environ.get('NEW_TEMPLATE_NAME', + 'instance-template-test-new') +NEW_DESCRIPTION = os.environ.get('NEW_DESCRIPTION', 'Test new description') +GCE_INSTANCE_TEMPLATE_BODY_UPDATE = { + "name": NEW_TEMPLATE_NAME, + "description": NEW_DESCRIPTION, + "properties": { + "machineType": "n1-standard-2" + } +} +# [END howto_operator_compute_template_copy_args] + +# [START howto_operator_compute_igm_update_template_args] +INSTANCE_GROUP_MANAGER_NAME = os.environ.get('INSTANCE_GROUP_MANAGER_NAME', + 'instance-group-test') + +SOURCE_TEMPLATE_URL = os.environ.get( + 'SOURCE_TEMPLATE_URL', + "https://www.googleapis.com/compute/beta/projects/" + "example-project/global/instanceTemplates/instance-template-test") + +DESTINATION_TEMPLATE_URL = os.environ.get( + 'DESTINATION_TEMPLATE_URL', + "https://www.googleapis.com/compute/beta/projects/" + "example-airflow/global/instanceTemplates/" + NEW_TEMPLATE_NAME) + +UPDATE_POLICY = { + "type": "OPPORTUNISTIC", + "minimalAction": "RESTART", + "maxSurge": { + "fixed": 1 + }, + "minReadySec": 1800 +} + +# [END howto_operator_compute_igm_update_template_args] + + +with models.DAG( + 'example_gcp_compute_igm', + default_args=default_args, + schedule_interval=datetime.timedelta(days=1) +) as dag: + # [START howto_operator_gce_igm_copy_template] + gce_instance_template_copy = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=TEMPLATE_NAME, + body_patch=GCE_INSTANCE_TEMPLATE_BODY_UPDATE, + task_id='gcp_compute_igm_copy_template_task' + ) + # [END howto_operator_gce_igm_copy_template] + # Added to check for idempotence + gce_instance_template_copy2 = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=TEMPLATE_NAME, + body_patch=GCE_INSTANCE_TEMPLATE_BODY_UPDATE, + task_id='gcp_compute_igm_copy_template_task_2' + ) + # [START howto_operator_gce_igm_update_template] + gce_instance_group_manager_update_template = \ + GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + resource_id=INSTANCE_GROUP_MANAGER_NAME, + zone=ZONE, + source_template=SOURCE_TEMPLATE_URL, + destination_template=DESTINATION_TEMPLATE_URL, + update_policy=UPDATE_POLICY, + task_id='gcp_compute_igm_group_manager_update_template' + ) + # [END howto_operator_gce_igm_update_template] + # Added to check for idempotence (and without UPDATE_POLICY) + gce_instance_group_manager_update_template2 = \ + GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + resource_id=INSTANCE_GROUP_MANAGER_NAME, + zone=ZONE, + source_template=SOURCE_TEMPLATE_URL, + destination_template=DESTINATION_TEMPLATE_URL, + task_id='gcp_compute_igm_group_manager_update_template_2' + ) + gce_instance_template_copy >> gce_instance_template_copy2 >> \ + gce_instance_group_manager_update_template >> \ + gce_instance_group_manager_update_template2 diff --git a/airflow/contrib/example_dags/example_gcp_function_delete.py b/airflow/contrib/example_dags/example_gcp_function_delete.py index 30f5369af60ef..d87eed39c530f 100644 --- a/airflow/contrib/example_dags/example_gcp_function_delete.py +++ b/airflow/contrib/example_dags/example_gcp_function_delete.py @@ -33,9 +33,9 @@ from airflow.contrib.operators.gcp_function_operator import GcfFunctionDeleteOperator # [START howto_operator_gcf_delete_args] -PROJECT_ID = models.Variable.get('PROJECT_ID', '') -LOCATION = models.Variable.get('LOCATION', '') -ENTRYPOINT = models.Variable.get('ENTRYPOINT', '') +PROJECT_ID = models.Variable.get('PROJECT_ID', 'example-airflow') +LOCATION = models.Variable.get('LOCATION', 'europe-west1') +ENTRYPOINT = models.Variable.get('ENTRYPOINT', 'helloWorld') # A fully-qualified name of the function to delete FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(PROJECT_ID, LOCATION, diff --git a/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py b/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py index a0e44957b9204..606cc181b0ca4 100644 --- a/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py +++ b/airflow/contrib/example_dags/example_gcp_function_deploy_delete.py @@ -45,11 +45,14 @@ from airflow.utils import dates # [START howto_operator_gcf_deploy_variables] -PROJECT_ID = models.Variable.get('PROJECT_ID', '') -LOCATION = models.Variable.get('LOCATION', '') +PROJECT_ID = models.Variable.get('PROJECT_ID', 'example-airflow') +LOCATION = models.Variable.get('LOCATION', 'europe-west1') SOURCE_ARCHIVE_URL = models.Variable.get('SOURCE_ARCHIVE_URL', '') SOURCE_UPLOAD_URL = models.Variable.get('SOURCE_UPLOAD_URL', '') -SOURCE_REPOSITORY = models.Variable.get('SOURCE_REPOSITORY', '') +SOURCE_REPOSITORY = models.Variable.get('SOURCE_REPOSITORY', + 'https://source.developers.google.com/' + 'projects/example-airflow/' + 'repos/hello-world/moveable-aliases/master') ZIP_PATH = models.Variable.get('ZIP_PATH', '') ENTRYPOINT = models.Variable.get('ENTRYPOINT', '') FUNCTION_NAME = 'projects/{}/locations/{}/functions/{}'.format(PROJECT_ID, LOCATION, @@ -70,11 +73,7 @@ # [START howto_operator_gcf_deploy_args] default_args = { - 'start_date': dates.days_ago(1), - 'project_id': PROJECT_ID, - 'location': LOCATION, - 'body': body, - 'validate_body': VALIDATE_BODY + 'start_date': dates.days_ago(1) } # [END howto_operator_gcf_deploy_args] @@ -103,11 +102,15 @@ # [START howto_operator_gcf_deploy] deploy_task = GcfFunctionDeployOperator( task_id="gcf_deploy_task", - name=FUNCTION_NAME + name=FUNCTION_NAME, + project_id=PROJECT_ID, + location=LOCATION, + body=body, + validate_body=VALIDATE_BODY ) # [END howto_operator_gcf_deploy] delete_task = GcfFunctionDeleteOperator( task_id="gcf_delete_task", - name=FUNCTION_NAME + name=FUNCTION_NAME, ) deploy_task >> delete_task diff --git a/airflow/contrib/hooks/gcp_compute_hook.py b/airflow/contrib/hooks/gcp_compute_hook.py index 5fa088942b706..617e39cb4006e 100644 --- a/airflow/contrib/hooks/gcp_compute_hook.py +++ b/airflow/contrib/hooks/gcp_compute_hook.py @@ -68,14 +68,14 @@ def start_instance(self, project_id, zone, resource_id): """ Starts an existing instance defined by project_id, zone and resource_id. - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance exists :type project_id: str - :param zone: Google Cloud Platform zone where the instance exists. + :param zone: Google Cloud Platform zone where the instance exists :type zone: str - :param resource_id: Name of the Compute Engine instance resource. + :param resource_id: Name of the Compute Engine instance resource :type resource_id: str - :return: True if the operation succeeded, raises an error otherwise + :return: True if the operation succeeded, raises an error otherwise. :rtype: bool """ response = self.get_conn().instances().start( @@ -83,21 +83,26 @@ def start_instance(self, project_id, zone, resource_id): zone=zone, instance=resource_id ).execute(num_retries=NUM_RETRIES) - operation_name = response["name"] - return self._wait_for_operation_to_complete(project_id, zone, operation_name) + try: + operation_name = response["name"] + except KeyError: + raise AirflowException( + "Wrong response '{}' returned - it should contain " + "'name' field".format(response)) + return self._wait_for_operation_to_complete(project_id, operation_name, zone) def stop_instance(self, project_id, zone, resource_id): """ - Stops an instance defined by project_id, zone and resource_id. + Stops an instance defined by project_id, zone and resource_id - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance exists :type project_id: str - :param zone: Google Cloud Platform zone where the instance exists. + :param zone: Google Cloud Platform zone where the instance exists :type zone: str - :param resource_id: Name of the Compute Engine instance resource. + :param resource_id: Name of the Compute Engine instance resource :type resource_id: str - :return: True if the operation succeeded, raises an error otherwise + :return: True if the operation succeeded, raises an error otherwise. :rtype: bool """ response = self.get_conn().instances().stop( @@ -105,50 +110,178 @@ def stop_instance(self, project_id, zone, resource_id): zone=zone, instance=resource_id ).execute(num_retries=NUM_RETRIES) - operation_name = response["name"] - return self._wait_for_operation_to_complete(project_id, zone, operation_name) + try: + operation_name = response["name"] + except KeyError: + raise AirflowException( + "Wrong response '{}' returned - it should contain " + "'name' field".format(response)) + return self._wait_for_operation_to_complete(project_id, operation_name, zone) def set_machine_type(self, project_id, zone, resource_id, body): """ Sets machine type of an instance defined by project_id, zone and resource_id. - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance exists :type project_id: str :param zone: Google Cloud Platform zone where the instance exists. :type zone: str - :param resource_id: Name of the Compute Engine instance resource. + :param resource_id: Name of the Compute Engine instance resource :type resource_id: str :param body: Body required by the Compute Engine setMachineType API, - as described in - https://cloud.google.com/compute/docs/reference/rest/v1/instances/setMachineType + as described in + https://cloud.google.com/compute/docs/reference/rest/v1/instances/setMachineType :type body: dict - :return: True if the operation succeeded, raises an error otherwise + :return: True if the operation succeeded, raises an error otherwise. :rtype: bool """ response = self._execute_set_machine_type(project_id, zone, resource_id, body) - operation_name = response["name"] - return self._wait_for_operation_to_complete(project_id, zone, operation_name) + try: + operation_name = response["name"] + except KeyError: + raise AirflowException( + "Wrong response '{}' returned - it should contain " + "'name' field".format(response)) + return self._wait_for_operation_to_complete(project_id, operation_name, zone) def _execute_set_machine_type(self, project_id, zone, resource_id, body): return self.get_conn().instances().setMachineType( project=project_id, zone=zone, instance=resource_id, body=body)\ .execute(num_retries=NUM_RETRIES) - def _wait_for_operation_to_complete(self, project_id, zone, operation_name): + def get_instance_template(self, project_id, resource_id): """ - Waits for the named operation to complete - checks status of the - asynchronous call. + Retrieves instance template by project_id and resource_id. + + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance template exists + :type project_id: str + :param resource_id: Name of the instance template + :type resource_id: str + :return: Instance template representation as object according to + https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates + :rtype: dict + """ + response = self.get_conn().instanceTemplates().get( + project=project_id, + instanceTemplate=resource_id + ).execute(num_retries=NUM_RETRIES) + return response + + def insert_instance_template(self, project_id, body, request_id=None): + """ + Inserts instance template using body specified + + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance exists + :type project_id: str + :param body: Instance template representation as object according to + https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates + :type body: dict + :param request_id: Optional, unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again) + It should be in UUID format as defined in RFC 4122 + :type request_id: str + :return: True if the operation succeeded + :rtype: bool + """ + response = self.get_conn().instanceTemplates().insert( + project=project_id, + body=body, + requestId=request_id + ).execute(num_retries=NUM_RETRIES) + try: + operation_name = response["name"] + except KeyError: + raise AirflowException( + "Wrong response '{}' returned - it should contain " + "'name' field".format(response)) + return self._wait_for_operation_to_complete(project_id, operation_name) + + def get_instance_group_manager(self, project_id, zone, resource_id): + """ + Retrieves Instance Group Manager by project_id, zone and resource_id. + + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance Group Manager exists + :type project_id: str + :param zone: Google Cloud Platform zone where the Instance Group Manager exists + :type zone: str + :param resource_id: Name of the Instance Group Manager + :type resource_id: str + :return: Instance group manager representation as object according to + https://cloud.google.com/compute/docs/reference/rest/beta/instanceGroupManagers + :rtype: dict + """ + response = self.get_conn().instanceGroupManagers().get( + project=project_id, + zone=zone, + instanceGroupManager=resource_id + ).execute(num_retries=NUM_RETRIES) + return response + + def patch_instance_group_manager(self, project_id, zone, resource_id, + body, request_id=None): + """ + Patches Instance Group Manager with the specified body. + + :param project_id: Google Cloud Platform project ID where the Compute Engine + Instance Group Manager exists + :type project_id: str + :param zone: Google Cloud Platform zone where the Instance Group Manager exists + :type zone: str + :param resource_id: Name of the Instance Group Manager + :type resource_id: str + :param body: Instance Group Manager representation as json-merge-patch object + according to + https://cloud.google.com/compute/docs/reference/rest/beta/instanceTemplates/patch + :type body: dict + :param request_id: Optional, unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again). + It should be in UUID format as defined in RFC 4122 + :type request_id: str + :return: True if the operation succeeded + :rtype: bool + """ + response = self.get_conn().instanceGroupManagers().patch( + project=project_id, + zone=zone, + instanceGroupManager=resource_id, + body=body, + requestId=request_id + ).execute(num_retries=NUM_RETRIES) + try: + operation_name = response["name"] + except KeyError: + raise AirflowException( + "Wrong response '{}' returned - it should contain " + "'name' field".format(response)) + return self._wait_for_operation_to_complete(project_id, operation_name, zone) + + def _wait_for_operation_to_complete(self, project_id, operation_name, zone=None): + """ + Waits for the named operation to complete - checks status of the async call. :param operation_name: name of the operation :type operation_name: str + :param zone: optional region of the request (might be None for global operations) + :type zone: str :return: True if the operation succeeded, raises an error otherwise :rtype: bool """ service = self.get_conn() while True: - operation_response = self._check_operation_status( - service, operation_name, project_id, zone) + if zone is None: + # noinspection PyTypeChecker + operation_response = self._check_global_operation_status( + service, operation_name, project_id) + else: + # noinspection PyTypeChecker + operation_response = self._check_zone_operation_status( + service, operation_name, project_id, zone) if operation_response.get("status") == GceOperationStatus.DONE: error = operation_response.get("error") if error: @@ -161,7 +294,14 @@ def _wait_for_operation_to_complete(self, project_id, zone, operation_name): return True time.sleep(TIME_TO_SLEEP_IN_SECONDS) - def _check_operation_status(self, service, operation_name, project_id, zone): + @staticmethod + def _check_zone_operation_status(service, operation_name, project_id, zone): return service.zoneOperations().get( project=project_id, zone=zone, operation=operation_name).execute( num_retries=NUM_RETRIES) + + @staticmethod + def _check_global_operation_status(service, operation_name, project_id): + return service.globalOperations().get( + project=project_id, operation=operation_name).execute( + num_retries=NUM_RETRIES) diff --git a/airflow/contrib/hooks/gcp_function_hook.py b/airflow/contrib/hooks/gcp_function_hook.py index d89b5b0ec80d2..29cef1716c327 100644 --- a/airflow/contrib/hooks/gcp_function_hook.py +++ b/airflow/contrib/hooks/gcp_function_hook.py @@ -65,7 +65,7 @@ def get_function(self, name): :param name: name of the function :type name: str - :return: a CloudFunction object representing the function + :return: a Cloud Functions object representing the function :rtype: dict """ return self.get_conn().projects().locations().functions().get( @@ -78,7 +78,7 @@ def list_functions(self, full_location): :param full_location: full location including the project in the form of of /projects//location/ :type full_location: str - :return: array of CloudFunction objects - representing functions in the location + :return: array of Cloud Functions objects - representing functions in the location :rtype: [dict] """ list_response = self.get_conn().projects().locations().functions().list( diff --git a/airflow/contrib/operators/gcp_compute_operator.py b/airflow/contrib/operators/gcp_compute_operator.py index a2fd54529429c..a872c17227ddf 100644 --- a/airflow/contrib/operators/gcp_compute_operator.py +++ b/airflow/contrib/operators/gcp_compute_operator.py @@ -16,18 +16,24 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from copy import deepcopy + +from googleapiclient.errors import HttpError from airflow import AirflowException from airflow.contrib.hooks.gcp_compute_hook import GceHook +from airflow.contrib.utils.gcp_field_sanitizer import GcpBodyFieldSanitizer from airflow.contrib.utils.gcp_field_validator import GcpBodyFieldValidator from airflow.models import BaseOperator from airflow.utils.decorators import apply_defaults +from json_merge_patch import merge class GceBaseOperator(BaseOperator): """ Abstract base operator for Google Compute Engine operators to inherit from. """ + @apply_defaults def __init__(self, project_id, @@ -61,10 +67,10 @@ def execute(self, context): class GceInstanceStartOperator(GceBaseOperator): """ - Start an instance in Google Compute Engine. + Starts an instance in Google Compute Engine. - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform Project ID where the Compute Engine + Instance exists. :type project_id: str :param zone: Google Cloud Platform zone where the instance exists. :type zone: str @@ -72,10 +78,12 @@ class GceInstanceStartOperator(GceBaseOperator): :type resource_id: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (for example v1 or beta). :type api_version: str """ + # [START gce_instance_start_template_fields] template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version') + # [END gce_instance_start_template_fields] @apply_defaults def __init__(self, @@ -95,10 +103,10 @@ def execute(self, context): class GceInstanceStopOperator(GceBaseOperator): """ - Stop an instance in Google Compute Engine. + Stops an instance in Google Compute Engine. - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform Project ID where the Compute Engine + Instance exists. :type project_id: str :param zone: Google Cloud Platform zone where the instance exists. :type zone: str @@ -106,10 +114,12 @@ class GceInstanceStopOperator(GceBaseOperator): :type resource_id: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (for example v1 or beta). :type api_version: str """ + # [START gce_instance_stop_template_fields] template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version') + # [END gce_instance_stop_template_fields] @apply_defaults def __init__(self, @@ -135,10 +145,10 @@ def execute(self, context): class GceSetMachineTypeOperator(GceBaseOperator): """ Changes the machine type for a stopped instance to the machine type specified in - the request. + the request. - :param project_id: Google Cloud Platform project where the Compute Engine - instance exists. + :param project_id: Google Cloud Platform Project ID where the Compute Engine + Instance exists. :type project_id: str :param zone: Google Cloud Platform zone where the instance exists. :type zone: str @@ -149,10 +159,14 @@ class GceSetMachineTypeOperator(GceBaseOperator): :type body: dict :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str - :param api_version: API version used (e.g. v1). + :param api_version: API version used (for example v1 or beta). :type api_version: str + :param validate_body: If set to False, body validation is not performed. + :type validate_body: bool """ + # [START gce_instance_set_machine_type_template_fields] template_fields = ('project_id', 'zone', 'resource_id', 'gcp_conn_id', 'api_version') + # [END gce_instance_set_machine_type_template_fields] @apply_defaults def __init__(self, @@ -181,3 +195,241 @@ def execute(self, context): self._validate_all_body_fields() return self._hook.set_machine_type(self.project_id, self.zone, self.resource_id, self.body) + + +GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION = [ + dict(name="name", regexp="^.+$"), + dict(name="description", optional=True), + dict(name="properties", type='dict', optional=True, fields=[ + dict(name="description", optional=True), + dict(name="tags", optional=True, fields=[ + dict(name="items", optional=True) + ]), + dict(name="machineType", optional=True), + dict(name="canIpForward", optional=True), + dict(name="networkInterfaces", optional=True), # not validating deeper + dict(name="disks", optional=True), # not validating the array deeper + dict(name="metadata", optional=True, fields=[ + dict(name="fingerprint", optional=True), + dict(name="items", optional=True), + dict(name="kind", optional=True), + ]), + dict(name="serviceAccounts", optional=True), # not validating deeper + dict(name="scheduling", optional=True, fields=[ + dict(name="onHostMaintenance", optional=True), + dict(name="automaticRestart", optional=True), + dict(name="preemptible", optional=True), + dict(name="nodeAffinitites", optional=True), # not validating deeper + ]), + dict(name="labels", optional=True), + dict(name="guestAccelerators", optional=True), # not validating deeper + dict(name="minCpuPlatform", optional=True), + ]), +] + +GCE_INSTANCE_TEMPLATE_FIELDS_TO_SANITIZE = [ + "kind", + "id", + "name", + "creationTimestamp", + "properties.disks.sha256", + "properties.disks.kind", + "properties.disks.sourceImageEncryptionKey.sha256", + "properties.disks.index", + "properties.disks.licenses", + "properties.networkInterfaces.kind", + "properties.networkInterfaces.accessConfigs.kind", + "properties.networkInterfaces.name", + "properties.metadata.kind", + "selfLink" +] + + +class GceInstanceTemplateCopyOperator(GceBaseOperator): + """ + Copies the instance template, applying specified changes. + + :param project_id: Google Cloud Platform Project ID where the Compute Engine + instance exists. + :type project_id: str + :param resource_id: Name of the Instance Template + :type resource_id: str + :param body_patch: Patch to the body of instanceTemplates object following rfc7386 + PATCH semantics. The body_patch content follows + https://cloud.google.com/compute/docs/reference/rest/v1/instanceTemplates + Name field is required as we need to rename the template, + all the other fields are optional. It is important to follow PATCH semantics + - arrays are replaced fully, so if you need to update an array you should + provide the whole target array as patch element. + :type body_patch: dict + :param request_id: Optional, unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again). + It should be in UUID format as defined in RFC 4122. + :type request_id: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (for example v1 or beta). + :type api_version: str + :param validate_body: If set to False, body validation is not performed. + :type validate_body: bool + """ + # [START gce_instance_template_copy_operator_template_fields] + template_fields = ('project_id', 'resource_id', 'request_id', + 'gcp_conn_id', 'api_version') + # [END gce_instance_template_copy_operator_template_fields] + + @apply_defaults + def __init__(self, + project_id, + resource_id, + body_patch, + request_id=None, + gcp_conn_id='google_cloud_default', + api_version='v1', + validate_body=True, + *args, **kwargs): + self.body_patch = body_patch + self.request_id = request_id + self._field_validator = None + if 'name' not in self.body_patch: + raise AirflowException("The body '{}' should contain at least " + "name for the new operator in the 'name' field". + format(body_patch)) + if validate_body: + self._field_validator = GcpBodyFieldValidator( + GCE_INSTANCE_TEMPLATE_VALIDATION_PATCH_SPECIFICATION, api_version=api_version) + self._field_sanitizer = GcpBodyFieldSanitizer( + GCE_INSTANCE_TEMPLATE_FIELDS_TO_SANITIZE) + super(GceInstanceTemplateCopyOperator, self).__init__( + project_id=project_id, zone='global', resource_id=resource_id, + gcp_conn_id=gcp_conn_id, api_version=api_version, *args, **kwargs) + + def _validate_all_body_fields(self): + if self._field_validator: + self._field_validator.validate(self.body_patch) + + def execute(self, context): + self._validate_all_body_fields() + try: + # Idempotence check (sort of) - we want to check if the new template + # is already created and if is, then we assume it was created by previous run + # of CopyTemplate operator - we do not check if content of the template + # is as expected. Templates are immutable so we cannot update it anyway + # and deleting/recreating is not worth the hassle especially + # that we cannot delete template if it is already used in some Instance + # Group Manager. We assume success if the template is simply present + existing_template = self._hook.get_instance_template( + project_id=self.project_id, resource_id=self.body_patch['name']) + self.log.info("The {} template already existed. It was likely " + "created by previous run of the operator. Assuming success.") + return existing_template + except HttpError as e: + # We actually expect to get 404 / Not Found here as the template should + # not yet exist + if not e.resp.status == 404: + raise e + old_body = self._hook.get_instance_template(project_id=self.project_id, + resource_id=self.resource_id) + new_body = deepcopy(old_body) + self._field_sanitizer.sanitize(new_body) + new_body = merge(new_body, self.body_patch) + self.log.info("Calling insert instance template with updated body: {}". + format(new_body)) + self._hook.insert_instance_template(project_id=self.project_id, + body=new_body, + request_id=self.request_id) + return self._hook.get_instance_template(project_id=self.project_id, + resource_id=self.body_patch['name']) + + +class GceInstanceGroupManagerUpdateTemplateOperator(GceBaseOperator): + """ + Patches the Instance Group Manager, replacing source template URL with the + destination one. API V1 does not have update/patch operations for Instance + Group Manager, so you must use beta or newer API version. Beta is the default. + + :param project_id: Google Cloud Platform Project ID where the Compute Engine + Instance exists. + :type project_id: str + :param resource_id: Name of the Instance Group Manager + :type resource_id: str + :param zone: Google Cloud Platform zone where the Instance Group Manager exists. + :type zone: str + :param request_id: Optional, unique request_id that you might add to achieve + full idempotence (for example when client call times out repeating the request + with the same request id will not create a new instance template again). + It should be in UUID format as defined in RFC 4122 + :type request_id: str + :param update_policy: The update policy for this managed instance group. See + https://cloud.google.com/compute/docs/reference/rest/beta/instanceGroupManagers/patch + for details of the updatePolicy fields. It's an optional field. + :type dict + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + :param api_version: API version used (for example beta). + :type api_version: str + """ + # [START gce_igm_update_template_operator_template_fields] + template_fields = ('project_id', 'resource_id', 'zone', 'request_id', + 'source_template', 'destination_template', + 'gcp_conn_id', 'api_version') + # [END gce_igm_update_template_operator_template_fields] + + @apply_defaults + def __init__(self, + project_id, + resource_id, + zone, + source_template, + destination_template, + update_policy=None, + request_id=None, + gcp_conn_id='google_cloud_default', + api_version='beta', + *args, **kwargs): + self.zone = zone + self.source_template = source_template + self.destination_template = destination_template + self.request_id = request_id + self.update_policy = update_policy + self._change_performed = False + if api_version == 'v1': + raise AirflowException("Api version v1 does not have update/patch " + "operations for Instance Group Managers. Use beta" + " api version or above") + super(GceInstanceGroupManagerUpdateTemplateOperator, self).__init__( + project_id=project_id, zone=self.zone, resource_id=resource_id, + gcp_conn_id=gcp_conn_id, api_version=api_version, *args, **kwargs) + + def _possibly_replace_template(self, dictionary): + # type: (dict) -> None + if dictionary.get('instanceTemplate') == self.source_template: + dictionary['instanceTemplate'] = self.destination_template + self._change_performed = True + + def execute(self, context): + old_instance_group_manager = self._hook.get_instance_group_manager( + project_id=self.project_id, + zone=self.zone, + resource_id=self.resource_id) + patch_body = {} + if 'versions' in old_instance_group_manager: + patch_body['versions'] = old_instance_group_manager['versions'] + if 'instanceTemplate' in old_instance_group_manager: + patch_body['instanceTemplate'] = old_instance_group_manager['instanceTemplate'] + if self.update_policy: + patch_body['updatePolicy'] = self.update_policy + self._possibly_replace_template(patch_body) + if 'versions' in patch_body: + for version in patch_body['versions']: + self._possibly_replace_template(version) + if self._change_performed or self.update_policy: + self.log.info("Calling patch instance template with updated body: {}". + format(patch_body)) + return self._hook.patch_instance_group_manager( + project_id=self.project_id, zone=self.zone, resource_id=self.resource_id, + body=patch_body, request_id=self.request_id) + else: + # Idempotence achieved + return True diff --git a/airflow/contrib/operators/gcp_function_operator.py b/airflow/contrib/operators/gcp_function_operator.py index c0013aaea9bf6..7f7da1d3ec5eb 100644 --- a/airflow/contrib/operators/gcp_function_operator.py +++ b/airflow/contrib/operators/gcp_function_operator.py @@ -93,7 +93,7 @@ class GcfFunctionDeployOperator(BaseOperator): . Different API versions require different variants of the Cloud Functions dictionary. :type body: dict or google.cloud.functions.v1.CloudFunction - :param gcp_conn_id: The connection ID to use to connect to Google Cloud Platform. + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str :param api_version: API version used (for example v1 or v1beta1). :type api_version: str @@ -105,6 +105,9 @@ class GcfFunctionDeployOperator(BaseOperator): :param validate_body: If set to False, body validation is not performed. :type validate_body: bool """ + # [START gce_function_deploy_template_operator_template_fields] + template_fields = ('project_id', 'location', 'gcp_conn_id', 'api_version') + # [END gce_function_deploy_template_operator_template_fields] @apply_defaults def __init__(self, @@ -276,6 +279,9 @@ class GcfFunctionDeleteOperator(BaseOperator): :param api_version: API version used (for example v1 or v1beta1). :type api_version: str """ + # [START gce_function_delete_template_operator_template_fields] + template_fields = ('name', 'gcp_conn_id', 'api_version') + # [END gce_function_delete_template_operator_template_fields] @apply_defaults def __init__(self, diff --git a/airflow/contrib/utils/gcp_field_sanitizer.py b/airflow/contrib/utils/gcp_field_sanitizer.py new file mode 100644 index 0000000000000..c0a8985281e1a --- /dev/null +++ b/airflow/contrib/utils/gcp_field_sanitizer.py @@ -0,0 +1,162 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +"""Sanitizer for body fields sent via GCP API. + +The sanitizer removes fields specified from the body. + +Context +------- +In some cases where GCP operation requires modification of existing resources (such +as instances or instance templates) we need to sanitize body of the resources returned +via GCP APIs. This is in the case when we retrieve information from GCP first, +modify the body and either update the existing resource or create a new one with the +modified body. Usually when you retrieve resource from GCP you get some extra fields which +are Output-only, and we need to delete those fields if we want to use +the body as input for subsequent create/insert type operation. + + +Field specification +------------------- + +Specification of fields is an array of strings which denote names of fields to be removed. +The field can be either direct field name to remove from the body or the full +specification of the path you should delete - separated with '.' + + +>>> FIELDS_TO_SANITIZE = [ +>>> "kind", +>>> "properties.disks.kind", +>>> "properties.metadata.kind", +>>>] +>>> body = { +>>> "kind": "compute#instanceTemplate", +>>> "name": "instance", +>>> "properties": { +>>> "disks": [ +>>> { +>>> "name": "a", +>>> "kind": "compute#attachedDisk", +>>> "type": "PERSISTENT", +>>> "mode": "READ_WRITE", +>>> }, +>>> { +>>> "name": "b", +>>> "kind": "compute#attachedDisk", +>>> "type": "PERSISTENT", +>>> "mode": "READ_WRITE", +>>> } +>>> ], +>>> "metadata": { +>>> "kind": "compute#metadata", +>>> "fingerprint": "GDPUYxlwHe4=" +>>> }, +>>> } +>>> } +>>> sanitizer=GcpBodyFieldSanitizer(FIELDS_TO_SANITIZE) +>>> SANITIZED_BODY = sanitizer.sanitize(body) +>>> json.dumps(SANITIZED_BODY, indent=2) +{ + "name": "instance", + "properties": { + "disks": [ + { + "name": "a", + "type": "PERSISTENT", + "mode": "READ_WRITE", + }, + { + "name": "b", + "type": "PERSISTENT", + "mode": "READ_WRITE", + } + ], + "metadata": { + "fingerprint": "GDPUYxlwHe4=" + }, + } +} + +Note that the components of the path can be either dictionaries or arrays of dictionaries. +In case they are dictionaries, subsequent component names key of the field, in case of +arrays - the sanitizer iterates through all dictionaries in the array and searches +components in all elements of the array. +""" + +from airflow import LoggingMixin, AirflowException + + +class GcpFieldSanitizerException(AirflowException): + """Thrown when sanitizer finds unexpected field type in the path + (other than dict or array). + """ + + def __init__(self, message): + super(GcpFieldSanitizerException, self).__init__(message) + + +class GcpBodyFieldSanitizer(LoggingMixin): + """Sanitizes the body according to specification. + + :param sanitize_specs: array of strings that specifies which fields to remove + :type sanitize_specs: [string] + + """ + def __init__(self, sanitize_specs): + # type: ([str]) -> None + super(GcpBodyFieldSanitizer, self).__init__() + self._sanitize_specs = sanitize_specs + + def _sanitize(self, dictionary, remaining_field_spec, current_path): + field_split = remaining_field_spec.split(".", 1) + if len(field_split) == 1: + field_name = field_split[0] + if field_name in dictionary: + self.log.info("Deleted {} [{}]".format(field_name, current_path)) + del dictionary[field_name] + else: + self.log.debug("The field {} is missing in {} at the path {}.". + format(field_name, dictionary, current_path)) + else: + field_name = field_split[0] + remaining_path = field_split[1] + child = dictionary.get(field_name) + if child is None: + self.log.debug("The field {} is missing in {} at the path {}. ". + format(field_name, dictionary, current_path)) + elif isinstance(child, dict): + self._sanitize(child, remaining_path, "{}.{}".format( + current_path, field_name)) + elif isinstance(child, list): + for index, elem in enumerate(child): + if not isinstance(elem, dict): + self.log.warn( + "The field {} element at index {} is of wrong type. " + "It should be dict and is {}. Skipping it.". + format(current_path, index, elem)) + self._sanitize(elem, remaining_path, "{}.{}[{}]".format( + current_path, field_name, index)) + else: + self.log.warn( + "The field {} is of wrong type. " + "It should be dict or list and it is {}. Skipping it.". + format(current_path, child)) + + def sanitize(self, body): + for elem in self._sanitize_specs: + self._sanitize(body, elem, "") diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index 2c84a7f88e888..41fc6fd1e5904 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -103,8 +103,8 @@ template variables ` and a ``templates_dict`` argument. The ``templates_dict`` argument is templated, so each value in the dictionary is evaluated as a :ref:`Jinja template `. -Google Cloud Platform Operators -------------------------------- +Google Cloud Storage Operators +------------------------------ GoogleCloudStorageToBigQueryOperator ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -119,22 +119,31 @@ to execute a BigQuery load job. :start-after: [START howto_operator_gcs_to_bq] :end-before: [END howto_operator_gcs_to_bq] + +Google Compute Engine Operators +------------------------------- + GceInstanceStartOperator ^^^^^^^^^^^^^^^^^^^^^^^^ -Allows to start an existing Google Compute Engine instance. +Use the +:class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceStartOperator` +to start an existing Google Compute Engine instance. -In this example parameter values are extracted from Airflow variables. -Moreover, the ``default_args`` dict is used to pass common arguments to all operators in a single DAG. + +Arguments +""""""""" + +The following examples of OS environment variables show how you can build function name +to use in the operator and build default args to pass them to multiple tasks: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py :language: python - :start-after: [START howto_operator_gce_args] - :end-before: [END howto_operator_gce_args] + :start-after: [START howto_operator_gce_args_common] + :end-before: [END howto_operator_gce_args_common] - -Define the :class:`~airflow.contrib.operators.gcp_compute_operator -.GceInstanceStartOperator` by passing the required arguments to the constructor. +Using the operator +"""""""""""""""""" .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py :language: python @@ -142,15 +151,42 @@ Define the :class:`~airflow.contrib.operators.gcp_compute_operator :start-after: [START howto_operator_gce_start] :end-before: [END howto_operator_gce_start] +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_compute_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_start_template_fields] + :end-before: [END gce_instance_start_template_fields] + +More information +"""""""""""""""" + +See `Google Compute Engine API documentation `_ + + GceInstanceStopOperator ^^^^^^^^^^^^^^^^^^^^^^^ -Allows to stop an existing Google Compute Engine instance. +Use the operator to stop Google Compute Engine instance. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceStopOperator` -For parameter definition take a look at :class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceStartOperator` above. +Arguments +""""""""" -Define the :class:`~airflow.contrib.operators.gcp_compute_operator -.GceInstanceStopOperator` by passing the required arguments to the constructor. +The following examples of OS environment variables show how you can build function name +to use in the operator and build default args to pass them to multiple tasks: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py + :language: python + :start-after: [START howto_operator_gce_args_common] + :end-before: [END howto_operator_gce_args_common] + +Using the operator +"""""""""""""""""" .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py :language: python @@ -158,15 +194,48 @@ Define the :class:`~airflow.contrib.operators.gcp_compute_operator :start-after: [START howto_operator_gce_stop] :end-before: [END howto_operator_gce_stop] +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_compute_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_stop_template_fields] + :end-before: [END gce_instance_stop_template_fields] + +More information +"""""""""""""""" + +See `Google Compute Engine API documentation `_ + + GceSetMachineTypeOperator ^^^^^^^^^^^^^^^^^^^^^^^^^ -Allows to change the machine type for a stopped instance to the specified machine type. +Use the operator to change machine type of a Google Compute Engine instance. -For parameter definition take a look at :class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceStartOperator` above. +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_compute_operator.GceSetMachineTypeOperator` -Define the :class:`~airflow.contrib.operators.gcp_compute_operator -.GceSetMachineTypeOperator` by passing the required arguments to the constructor. +Arguments +""""""""" + +The following examples of OS environment variables show how you can build function name +to use in the operator and build default args to pass them to multiple tasks: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py + :language: python + :start-after: [START howto_operator_gce_args_common] + :end-before: [END howto_operator_gce_args_common] + + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py + :language: python + :start-after: [START howto_operator_gce_args_set_machine_type] + :end-before: [END howto_operator_gce_args_set_machine_type] + +Using the operator +"""""""""""""""""" .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute.py :language: python @@ -174,26 +243,163 @@ Define the :class:`~airflow.contrib.operators.gcp_compute_operator :start-after: [START howto_operator_gce_set_machine_type] :end-before: [END howto_operator_gce_set_machine_type] +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_compute_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_set_machine_type_template_fields] + :end-before: [END gce_instance_set_machine_type_template_fields] + +More information +"""""""""""""""" + +See `Google Compute Engine API documentation `_ + + +GceInstanceTemplateCopyOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the operator to copy an existing Google Compute Engine instance template +applying a patch to it. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceTemplateCopyOperator`. + +Arguments +""""""""" + +The following examples of OS environment variables show how you can build parameters +passed to the operator and build default args to pass them to multiple tasks: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :start-after: [START howto_operator_compute_igm_common_args] + :end-before: [END howto_operator_compute_igm_common_args] + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :start-after: [START howto_operator_compute_template_copy_args] + :end-before: [END howto_operator_compute_template_copy_args] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_igm_copy_template] + :end-before: [END howto_operator_gce_igm_copy_template] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_compute_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_instance_template_copy_operator_template_fields] + :end-before: [END gce_instance_template_copy_operator_template_fields] + +More information +"""""""""""""""" + +See `Google Compute Engine API documentation `_ + +GceInstanceGroupManagerUpdateTemplateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the operator to update template in Google Compute Engine Instance Group Manager. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_compute_operator.GceInstanceGroupManagerUpdateTemplateOperator`. + +Arguments +""""""""" + +The following examples of OS environment variables show how you can build parameters +passed to the operator and build default args to pass them to multiple tasks: + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :start-after: [START howto_operator_compute_igm_common_args] + :end-before: [END howto_operator_compute_igm_common_args] + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :start-after: [START howto_operator_compute_igm_update_template_args] + :end-before: [END howto_operator_compute_igm_update_template_args] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_compute_igm.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gce_igm_update_template] + :end-before: [END howto_operator_gce_igm_update_template] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_compute_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_igm_update_template_operator_template_fields] + :end-before: [END gce_igm_update_template_operator_template_fields] + +Troubleshooting +""""""""""""""" + +You might find that your GceInstanceGroupManagerUpdateTemplateOperator fails with +missing permissions. The service account has to have Service Account User role assigned +via IAM permissions in order to execute the operation. + +More information +"""""""""""""""" + +See `Google Compute Engine API documentation `_ + +Google Cloud Functions Operators +-------------------------------- GcfFunctionDeleteOperator ^^^^^^^^^^^^^^^^^^^^^^^^^ -Use the ``default_args`` dict to pass arguments to the operator. +Use the operator to delete a function from Google Cloud Functions. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_function_operator.GcfFunctionDeleteOperator`. + +Arguments +""""""""" + +The following examples of OS environment variables show how you can build function name +to use in the operator and build default args to pass them to multiple tasks: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_delete.py :language: python :start-after: [START howto_operator_gcf_delete_args] :end-before: [END howto_operator_gcf_delete_args] - -Use the :class:`~airflow.contrib.operators.gcp_function_operator.GcfFunctionDeleteOperator` -to delete a function from Google Cloud Functions. +Using the operator +"""""""""""""""""" .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_delete.py :language: python + :dedent: 4 :start-after: [START howto_operator_gcf_delete] :end-before: [END howto_operator_gcf_delete] +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_function_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_function_delete_template_operator_template_fields] + :end-before: [END gce_function_delete_template_operator_template_fields] + Troubleshooting """"""""""""""" If you want to run or deploy an operator using a service account and get “forbidden 403” @@ -208,7 +414,6 @@ The typical way of assigning Cloud IAM permissions with `gcloud` is shown below. Just replace PROJECT_ID with ID of your Google Cloud Platform project and SERVICE_ACCOUNT_EMAIL with the email ID of your service account. - .. code-block:: bash gcloud iam service-accounts add-iam-policy-binding \ @@ -219,13 +424,24 @@ and SERVICE_ACCOUNT_EMAIL with the email ID of your service account. See `Adding the IAM service agent user role to the runtime service `_ for details +More information +"""""""""""""""" + +See `Google Cloud Functions API documentation `_ + GcfFunctionDeployOperator ^^^^^^^^^^^^^^^^^^^^^^^^^ -Use the :class:`~airflow.contrib.operators.gcp_function_operator.GcfFunctionDeployOperator` -to deploy a function from Google Cloud Functions. +Use the operator to deploy a function to Google Cloud Functions. + +For parameter definition take a look at +:class:`~airflow.contrib.operators.gcp_function_operator.GcfFunctionDeployOperator`. + -The following examples of Airflow variables show various variants and combinations +Arguments +""""""""" + +The following examples of OS environment variables show various variants and combinations of default_args that you can use. The variables are defined as follows: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py @@ -240,11 +456,12 @@ With those variables you can define the body of the request: :start-after: [START howto_operator_gcf_deploy_body] :end-before: [END howto_operator_gcf_deploy_body] -When you create a DAG, the default_args dictionary can be used to pass the body and -other arguments: +When you create a DAG, the default_args dictionary can be used to pass +arguments common with other tasks: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py :language: python + :dedent: 4 :start-after: [START howto_operator_gcf_deploy_args] :end-before: [END howto_operator_gcf_deploy_args] @@ -252,11 +469,14 @@ Note that the neither the body nor the default args are complete in the above ex Depending on the set variables, there might be different variants on how to pass source code related fields. Currently, you can pass either sourceArchiveUrl, sourceRepository or sourceUploadUrl as described in the -`CloudFunction API specification `_. +`Cloud Functions API specification `_. Additionally, default_args might contain zip_path parameter to run the extra step of uploading the source code before deploying it. In the last case, you also need to provide an empty `sourceUploadUrl` parameter in the body. +Using the operator +"""""""""""""""""" + Based on the variables defined above, example logic of setting the source code related fields is shown here: @@ -269,9 +489,20 @@ The code to create the operator: .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_function_deploy_delete.py :language: python + :dedent: 4 :start-after: [START howto_operator_gcf_deploy] :end-before: [END howto_operator_gcf_deploy] +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_function_operator.py + :language: python + :dedent: 4 + :start-after: [START gce_function_deploy_template_operator_template_fields] + :end-before: [END gce_function_deploy_template_operator_template_fields] + + Troubleshooting """"""""""""""" @@ -294,13 +525,20 @@ and SERVICE_ACCOUNT_EMAIL with the email ID of your service account. --member="serviceAccount:[SERVICE_ACCOUNT_EMAIL]" \ --role="roles/iam.serviceAccountUser" - See `Adding the IAM service agent user role to the runtime service `_ for details If the source code for your function is in Google Source Repository, make sure that your service account has the Source Repository Viewer role so that the source code can be downloaded if necessary. +More information +"""""""""""""""" + +See `Google Cloud Functions API documentation `_ + +Google Cloud Sql Operators +-------------------------- + CloudSqlInstanceDatabaseCreateOperator ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/integration.rst b/docs/integration.rst index 1ab8e609d0dc2..9d3e6c9828f59 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -116,28 +116,28 @@ field (see connection `wasb_default` for an example). .. _WasbBlobSensor: WasbBlobSensor -""""""""""""""" +"""""""""""""" .. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbBlobSensor .. _WasbPrefixSensor: WasbPrefixSensor -""""""""""""""""" +"""""""""""""""" .. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbPrefixSensor .. _FileToWasbOperator: FileToWasbOperator -""""""""""""""""""" +"""""""""""""""""" .. autoclass:: airflow.contrib.operators.file_to_wasb.FileToWasbOperator .. _WasbHook: WasbHook -""""""""" +"""""""" .. autoclass:: airflow.contrib.hooks.wasb_hook.WasbHook @@ -538,6 +538,16 @@ Compute Engine Operators - :ref:`GceInstanceStartOperator` : start an existing Google Compute Engine instance. - :ref:`GceInstanceStopOperator` : stop an existing Google Compute Engine instance. - :ref:`GceSetMachineTypeOperator` : change the machine type for a stopped instance. +- :ref:`GceInstanceTemplateCopyOperator` : copy the Instance Template, applying + specified changes. +- :ref:`GceInstanceGroupManagerUpdateTemplateOperator` : patch the Instance Group Manager, + replacing source Instance Template URL with the destination one. + +The operators have common base operator: + +.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceBaseOperator + +They also use :ref:`GceHook` hook to communicate with Google Cloud Platform. .. _GceInstanceStartOperator: @@ -560,6 +570,28 @@ GceSetMachineTypeOperator .. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceSetMachineTypeOperator +.. _GceInstanceTemplateCopyOperator: + +GceInstanceTemplateCopyOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceTemplateCopyOperator + +.. _GceInstanceGroupManagerUpdateTemplateOperator: + +GceInstanceGroupManagerUpdateTemplateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_compute_operator.GceInstanceGroupManagerUpdateTemplateOperator + +.. _GceHook: + +Compute Engine Hook +""""""""""""""""""" + +.. autoclass:: airflow.contrib.hooks.gcp_compute_hook.GceHook +:members: + Cloud Functions ''''''''''''''' @@ -572,6 +604,8 @@ Cloud Functions Operators .. autoclass:: airflow.contrib.operators.gcp_operator.GCP +They also use :ref:`GcfHook` hook to communicate with Google Cloud Platform. + .. _GcfFunctionDeployOperator: GcfFunctionDeployOperator @@ -588,6 +622,8 @@ GcfFunctionDeleteOperator .. autoclass:: airflow.contrib.operators.gcp_function_operator.GcfFunctionDeleteOperator +.. _GcfHook: + Cloud Functions Hook """""""""""""""""""" @@ -697,7 +733,7 @@ DataprocClusterCreateOperator .. _DataprocClusterScaleOperator: DataprocClusterScaleOperator -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +^^^^^^^^^^^^^^^^^^^^^^^^^^^^ .. autoclass:: airflow.contrib.operators.dataproc_operator.DataprocClusterScaleOperator diff --git a/setup.py b/setup.py index 3af146d5a301d..b519e77a3610b 100644 --- a/setup.py +++ b/setup.py @@ -286,6 +286,7 @@ def do_setup(): 'gitpython>=2.0.2', 'gunicorn>=19.4.0, <20.0', 'iso8601>=0.1.12', + 'json-merge-patch==0.2', 'jinja2>=2.7.3, <2.9.0', 'lxml>=4.0.0', 'markdown>=2.5.2, <3.0', diff --git a/tests/contrib/operators/test_gcp_compute_operator.py b/tests/contrib/operators/test_gcp_compute_operator.py index 449c4e015fdda..e8f9bf01656b8 100644 --- a/tests/contrib/operators/test_gcp_compute_operator.py +++ b/tests/contrib/operators/test_gcp_compute_operator.py @@ -18,10 +18,15 @@ # under the License. import ast import unittest +from copy import deepcopy + +import httplib2 +from googleapiclient.errors import HttpError from airflow import AirflowException, configuration from airflow.contrib.operators.gcp_compute_operator import GceInstanceStartOperator, \ - GceInstanceStopOperator, GceSetMachineTypeOperator + GceInstanceStopOperator, GceSetMachineTypeOperator, GceInstanceTemplateCopyOperator, \ + GceInstanceGroupManagerUpdateTemplateOperator from airflow.models import TaskInstance, DAG from airflow.utils import timezone @@ -34,12 +39,14 @@ except ImportError: mock = None +EMPTY_CONTENT = ''.encode('utf8') + PROJECT_ID = 'project-id' -LOCATION = 'zone' +ZONE = 'zone' RESOURCE_ID = 'resource-id' SHORT_MACHINE_TYPE_NAME = 'n1-machine-type' SET_MACHINE_TYPE_BODY = { - 'machineType': 'zones/{}/machineTypes/{}'.format(LOCATION, SHORT_MACHINE_TYPE_NAME) + 'machineType': 'zones/{}/machineTypes/{}'.format(ZONE, SHORT_MACHINE_TYPE_NAME) } DEFAULT_DATE = timezone.datetime(2017, 1, 1) @@ -51,7 +58,7 @@ def test_instance_start(self, mock_hook): mock_hook.return_value.start_instance.return_value = True op = GceInstanceStartOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, task_id='id' ) @@ -59,11 +66,11 @@ def test_instance_start(self, mock_hook): mock_hook.assert_called_once_with(api_version='v1', gcp_conn_id='google_cloud_default') mock_hook.return_value.start_instance.assert_called_once_with( - PROJECT_ID, LOCATION, RESOURCE_ID + PROJECT_ID, ZONE, RESOURCE_ID ) self.assertTrue(result) - # Setting all of the operator's input parameters as templated dag_ids + # Setting all of the operator's input parameters as template dag_ids # (could be anything else) just to test if the templating works for all fields @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') def test_instance_start_with_templates(self, mock_hook): @@ -95,7 +102,7 @@ def test_start_should_throw_ex_when_missing_project_id(self, mock_hook): with self.assertRaises(AirflowException) as cm: op = GceInstanceStartOperator( project_id="", - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, task_id='id' ) @@ -123,7 +130,7 @@ def test_start_should_throw_ex_when_missing_resource_id(self, mock_hook): with self.assertRaises(AirflowException) as cm: op = GceInstanceStartOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id="", task_id='id' ) @@ -132,12 +139,14 @@ def test_start_should_throw_ex_when_missing_resource_id(self, mock_hook): self.assertIn("The required parameter 'resource_id' is missing", str(err)) mock_hook.assert_not_called() + +class GceInstanceStopTest(unittest.TestCase): @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') def test_instance_stop(self, mock_hook): mock_hook.return_value.stop_instance.return_value = True op = GceInstanceStopOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, task_id='id' ) @@ -145,7 +154,7 @@ def test_instance_stop(self, mock_hook): mock_hook.assert_called_once_with(api_version='v1', gcp_conn_id='google_cloud_default') mock_hook.return_value.stop_instance.assert_called_once_with( - PROJECT_ID, LOCATION, RESOURCE_ID + PROJECT_ID, ZONE, RESOURCE_ID ) self.assertTrue(result) @@ -181,7 +190,7 @@ def test_stop_should_throw_ex_when_missing_project_id(self, mock_hook): with self.assertRaises(AirflowException) as cm: op = GceInstanceStopOperator( project_id="", - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, task_id='id' ) @@ -209,7 +218,7 @@ def test_stop_should_throw_ex_when_missing_resource_id(self, mock_hook): with self.assertRaises(AirflowException) as cm: op = GceInstanceStopOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id="", task_id='id' ) @@ -218,12 +227,14 @@ def test_stop_should_throw_ex_when_missing_resource_id(self, mock_hook): self.assertIn("The required parameter 'resource_id' is missing", str(err)) mock_hook.assert_not_called() + +class GceInstanceSetMachineTypeTest(unittest.TestCase): @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') def test_set_machine_type(self, mock_hook): mock_hook.return_value.set_machine_type.return_value = True op = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, task_id='id' @@ -232,7 +243,7 @@ def test_set_machine_type(self, mock_hook): mock_hook.assert_called_once_with(api_version='v1', gcp_conn_id='google_cloud_default') mock_hook.return_value.set_machine_type.assert_called_once_with( - PROJECT_ID, LOCATION, RESOURCE_ID, SET_MACHINE_TYPE_BODY + PROJECT_ID, ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY ) self.assertTrue(result) @@ -269,7 +280,7 @@ def test_set_machine_type_should_throw_ex_when_missing_project_id(self, mock_hoo with self.assertRaises(AirflowException) as cm: op = GceSetMachineTypeOperator( project_id="", - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, task_id='id' @@ -299,7 +310,7 @@ def test_set_machine_type_should_throw_ex_when_missing_resource_id(self, mock_ho with self.assertRaises(AirflowException) as cm: op = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id="", body=SET_MACHINE_TYPE_BODY, task_id='id' @@ -314,7 +325,7 @@ def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_h with self.assertRaises(AirflowException) as cm: op = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, body={}, task_id='id' @@ -332,10 +343,10 @@ def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_h "'zone': 'https://www.googleapis.com/compute/v1/projects/polidea" \ "-airflow/zones/europe-west3-b', 'operationType': " \ "'setMachineType', 'targetLink': " \ - "'https://www.googleapis.com/compute/v1/projects/polidea-airflow" \ + "'https://www.googleapis.com/compute/v1/projects/example-airflow" \ "/zones/europe-west3-b/instances/pa-1', 'targetId': " \ "'2480086944131075860', 'status': 'DONE', 'user': " \ - "'uberdarek@polidea-airflow.iam.gserviceaccount.com', " \ + "'uberdarek@example-airflow.iam.gserviceaccount.com', " \ "'progress': 100, 'insertTime': '2018-10-03T07:50:07.951-07:00', "\ "'startTime': '2018-10-03T07:50:08.324-07:00', 'endTime': " \ "'2018-10-03T07:50:08.484-07:00', 'error': {'errors': [{'code': " \ @@ -343,35 +354,688 @@ def test_set_machine_type_should_throw_ex_when_missing_machine_type(self, mock_h "'machine-type-1' does not exist in zone 'europe-west3-b'.\"}]}, "\ "'httpErrorStatusCode': 400, 'httpErrorMessage': 'BAD REQUEST', " \ "'selfLink': " \ - "'https://www.googleapis.com/compute/v1/projects/polidea-airflow" \ + "'https://www.googleapis.com/compute/v1/projects/example-airflow" \ "/zones/europe-west3-b/operations/operation-1538578207537" \ "-577542784f769-7999ab71-94f9ec1d'} " @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook' - '._check_operation_status') + '._check_zone_operation_status') @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook' '._execute_set_machine_type') @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook.get_conn') def test_set_machine_type_should_handle_and_trim_gce_error( - self, get_conn, _execute_set_machine_type, _check_operation_status): + self, get_conn, _execute_set_machine_type, _check_zone_operation_status): get_conn.return_value = {} _execute_set_machine_type.return_value = {"name": "test-operation"} - _check_operation_status.return_value = ast.literal_eval(self.MOCK_OP_RESPONSE) + _check_zone_operation_status.return_value = ast.literal_eval(self.MOCK_OP_RESPONSE) with self.assertRaises(AirflowException) as cm: op = GceSetMachineTypeOperator( project_id=PROJECT_ID, - zone=LOCATION, + zone=ZONE, resource_id=RESOURCE_ID, body=SET_MACHINE_TYPE_BODY, task_id='id' ) op.execute(None) err = cm.exception - _check_operation_status.assert_called_once_with( - {}, "test-operation", PROJECT_ID, LOCATION) + _check_zone_operation_status.assert_called_once_with( + {}, "test-operation", PROJECT_ID, ZONE) _execute_set_machine_type.assert_called_once_with( - PROJECT_ID, LOCATION, RESOURCE_ID, SET_MACHINE_TYPE_BODY) + PROJECT_ID, ZONE, RESOURCE_ID, SET_MACHINE_TYPE_BODY) # Checking the full message was sometimes failing due to different order # of keys in the serialized JSON self.assertIn("400 BAD REQUEST: {", str(err)) # checking the square bracket trim self.assertIn("UNSUPPORTED_OPERATION", str(err)) + + +GCE_INSTANCE_TEMPLATE_NAME = "instance-template-test" +GCE_INSTANCE_TEMPLATE_NEW_NAME = "instance-template-test-new" +GCE_INSTANCE_TEMPLATE_REQUEST_ID = "e12d5b48-4826-4ba9-ada6-0cff1e0b36a6" + +GCE_INSTANCE_TEMPLATE_BODY_GET = { + "kind": "compute#instanceTemplate", + "id": "6950321349997439715", + "creationTimestamp": "2018-10-15T06:20:12.777-07:00", + "name": GCE_INSTANCE_TEMPLATE_NAME, + "description": "", + "properties": { + "machineType": "n1-standard-1", + "networkInterfaces": [ + { + "kind": "compute#networkInterface", + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "kind": "compute#accessConfig", + "type": "ONE_TO_ONE_NAT", + } + ] + }, + { + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "kind": "compute#accessConfig", + "networkTier": "PREMIUM" + } + ] + } + ], + "disks": [ + { + "kind": "compute#attachedDisk", + "type": "PERSISTENT", + "licenses": [ + "A String", + ] + } + ], + "metadata": { + "kind": "compute#metadata", + "fingerprint": "GDPUYxlwHe4=" + }, + }, + "selfLink": "https://www.googleapis.com/compute/v1/projects/project" + "/global/instanceTemplates/instance-template-test" +} + +GCE_INSTANCE_TEMPLATE_BODY_INSERT = { + "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "description": "", + "properties": { + "machineType": "n1-standard-1", + "networkInterfaces": [ + { + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "type": "ONE_TO_ONE_NAT", + } + ] + }, + { + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "networkTier": "PREMIUM" + } + ] + } + ], + "disks": [ + { + "type": "PERSISTENT", + } + ], + "metadata": { + "fingerprint": "GDPUYxlwHe4=" + }, + }, +} + +GCE_INSTANCE_TEMPLATE_BODY_GET_NEW = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_GET) +GCE_INSTANCE_TEMPLATE_BODY_GET_NEW['name'] = GCE_INSTANCE_TEMPLATE_NEW_NAME + + +class GceInstanceTemplateCopyTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME} + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=GCE_INSTANCE_TEMPLATE_BODY_INSERT, + request_id=None + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_idempotent_copy_template_when_already_copied(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME} + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.insert_instance_template.assert_not_called() + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template_with_request_id(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, + task_id='id', + body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME} + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=GCE_INSTANCE_TEMPLATE_BODY_INSERT, + request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template_with_description_fields(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, + task_id='id', + body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "description": "New description"} + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + + body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["description"] = "New description" + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=body_insert, + request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_copy_with_some_validation_warnings(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={"name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "some_wrong_field": "test", + "properties": { + "some_other_wrong_field": "test" + }} + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["some_wrong_field"] = "test" + body_insert["properties"]["some_other_wrong_field"] = "test" + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=body_insert, + request_id=None, + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template_with_updated_nested_fields(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={ + "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "properties": { + "machineType": "n1-standard-2", + } + } + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["properties"]["machineType"] = "n1-standard-2" + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=body_insert, + request_id=None + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template_with_smaller_array_fields(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={ + "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "properties": { + "machineType": "n1-standard-1", + "networkInterfaces": [ + { + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "type": "ONE_TO_ONE_NAT", + "natIP": "8.8.8.8" + } + ] + } + ] + } + } + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["properties"]["networkInterfaces"] = [ + { + "network": "https://www.googleapis.com/compute/v1/" + "projects/project/global/networks/default", + "accessConfigs": [ + { + "type": "ONE_TO_ONE_NAT", + "natIP": "8.8.8.8" + } + ] + } + ] + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=body_insert, + request_id=None + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_copy_template_with_bigger_array_fields(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + task_id='id', + body_patch={ + "name": GCE_INSTANCE_TEMPLATE_NEW_NAME, + "properties": { + "disks": [ + { + "kind": "compute#attachedDisk", + "type": "SCRATCH", + "licenses": [ + "Updated String", + ] + }, + { + "kind": "compute#attachedDisk", + "type": "PERSISTENT", + "licenses": [ + "Another String", + ] + } + ], + } + } + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='v1', + gcp_conn_id='google_cloud_default') + + body_insert = deepcopy(GCE_INSTANCE_TEMPLATE_BODY_INSERT) + body_insert["properties"]["disks"] = [ + { + "kind": "compute#attachedDisk", + "type": "SCRATCH", + "licenses": [ + "Updated String", + ] + }, + { + "kind": "compute#attachedDisk", + "type": "PERSISTENT", + "licenses": [ + "Another String", + ] + } + ] + mock_hook.return_value.insert_instance_template.assert_called_once_with( + project_id=PROJECT_ID, + body=body_insert, + request_id=None, + ) + self.assertEqual(GCE_INSTANCE_TEMPLATE_BODY_GET_NEW, result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_missing_name(self, mock_hook): + mock_hook.return_value.get_instance_template.side_effect = [ + HttpError(resp=httplib2.Response({'status': 404}), content=EMPTY_CONTENT), + GCE_INSTANCE_TEMPLATE_BODY_GET, + GCE_INSTANCE_TEMPLATE_BODY_GET_NEW + ] + with self.assertRaises(AirflowException) as cm: + op = GceInstanceTemplateCopyOperator( + project_id=PROJECT_ID, + resource_id=GCE_INSTANCE_TEMPLATE_NAME, + request_id=GCE_INSTANCE_TEMPLATE_REQUEST_ID, + task_id='id', + body_patch={"description": "New description"} + ) + op.execute(None) + err = cm.exception + self.assertIn("should contain at least name for the new operator " + "in the 'name' field", str(err)) + mock_hook.assert_not_called() + + +GCE_INSTANCE_GROUP_MANAGER_NAME = "instance-group-test" +GCE_INSTANCE_TEMPLATE_SOURCE_URL = \ + "https://www.googleapis.com/compute/beta/projects/project" \ + "/global/instanceTemplates/instance-template-test" + +GCE_INSTANCE_TEMPLATE_OTHER_URL = \ + "https://www.googleapis.com/compute/beta/projects/project" \ + "/global/instanceTemplates/instance-template-other" + +GCE_INSTANCE_TEMPLATE_NON_EXISTING_URL = \ + "https://www.googleapis.com/compute/beta/projects/project" \ + "/global/instanceTemplates/instance-template-non-existing" + +GCE_INSTANCE_TEMPLATE_DESTINATION_URL = \ + "https://www.googleapis.com/compute/beta/projects/project" \ + "/global/instanceTemplates/instance-template-new" + +GCE_INSTANCE_GROUP_MANAGER_GET = { + "kind": "compute#instanceGroupManager", + "id": "2822359583810032488", + "creationTimestamp": "2018-10-17T05:39:35.793-07:00", + "name": GCE_INSTANCE_GROUP_MANAGER_NAME, + "zone": "https://www.googleapis.com/compute/beta/projects/project/zones/zone", + "instanceTemplate": GCE_INSTANCE_TEMPLATE_SOURCE_URL, + "versions": [ + { + "name": "v1", + "instanceTemplate": GCE_INSTANCE_TEMPLATE_SOURCE_URL, + "targetSize": { + "calculated": 1 + } + }, + { + "name": "v2", + "instanceTemplate": GCE_INSTANCE_TEMPLATE_OTHER_URL, + } + ], + "instanceGroup": GCE_INSTANCE_TEMPLATE_SOURCE_URL, + "baseInstanceName": GCE_INSTANCE_GROUP_MANAGER_NAME, + "fingerprint": "BKWB_igCNbQ=", + "currentActions": { + "none": 1, + "creating": 0, + "creatingWithoutRetries": 0, + "verifying": 0, + "recreating": 0, + "deleting": 0, + "abandoning": 0, + "restarting": 0, + "refreshing": 0 + }, + "pendingActions": { + "creating": 0, + "deleting": 0, + "recreating": 0, + "restarting": 0 + }, + "targetSize": 1, + "selfLink": "https://www.googleapis.com/compute/beta/projects/project/zones/" + "zone/instanceGroupManagers/" + GCE_INSTANCE_GROUP_MANAGER_NAME, + "autoHealingPolicies": [ + { + "initialDelaySec": 300 + } + ], + "serviceAccount": "198907790164@cloudservices.gserviceaccount.com" +} + +GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH = { + "instanceTemplate": GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + "versions": [ + { + "name": "v1", + "instanceTemplate": GCE_INSTANCE_TEMPLATE_DESTINATION_URL, + "targetSize": { + "calculated": 1 + } + }, + { + "name": "v2", + "instanceTemplate": GCE_INSTANCE_TEMPLATE_OTHER_URL, + } + ], +} + +GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID = "e12d5b48-4826-4ba9-ada6-0cff1e0b36a6" + +GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY = { + "type": "OPPORTUNISTIC", + "minimalAction": "RESTART", + "maxSurge": { + "fixed": 1 + }, + "maxUnavailable": { + "percent": 10 + }, + "minReadySec": 1800 +} + + +class GceInstanceGroupManagerUpdateTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_instance_group_update(self, mock_hook): + mock_hook.return_value.get_instance_group_manager.return_value = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + body=GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH, + request_id=None + ) + self.assertTrue(result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_instance_group_update_no_instance_template_field(self, mock_hook): + instance_group_manager_no_template = deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + del instance_group_manager_no_template['instanceTemplate'] + mock_hook.return_value.get_instance_group_manager.return_value = \ + instance_group_manager_no_template + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + expected_patch_no_instance_template = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) + del expected_patch_no_instance_template['instanceTemplate'] + mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + body=expected_patch_no_instance_template, + request_id=None + ) + self.assertTrue(result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_instance_group_update_no_versions_field(self, mock_hook): + instance_group_manager_no_versions = deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + del instance_group_manager_no_versions['versions'] + mock_hook.return_value.get_instance_group_manager.return_value = \ + instance_group_manager_no_versions + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + expected_patch_no_versions = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) + del expected_patch_no_versions['versions'] + mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + body=expected_patch_no_versions, + request_id=None + ) + self.assertTrue(result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_instance_group_update_with_update_policy(self, mock_hook): + mock_hook.return_value.get_instance_group_manager.return_value = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + update_policy=GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY, + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + expected_patch_with_update_policy = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH) + expected_patch_with_update_policy['updatePolicy'] = \ + GCE_INSTANCE_GROUP_MANAGER_UPDATE_POLICY + mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + body=expected_patch_with_update_policy, + request_id=None + ) + self.assertTrue(result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_successful_instance_group_update_with_request_id(self, mock_hook): + mock_hook.return_value.get_instance_group_manager.return_value = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + request_id=GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.patch_instance_group_manager.assert_called_once_with( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + body=GCE_INSTANCE_GROUP_MANAGER_EXPECTED_PATCH, + request_id=GCE_INSTANCE_GROUP_MANAGER_REQUEST_ID + ) + self.assertTrue(result) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_try_to_use_api_v1(self, mock_hook): + with self.assertRaises(AirflowException) as cm: + GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + api_version='v1', + source_template=GCE_INSTANCE_TEMPLATE_SOURCE_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + err = cm.exception + self.assertIn("Use beta api version or above", str(err)) + + @mock.patch('airflow.contrib.operators.gcp_compute_operator.GceHook') + def test_try_to_use_non_existing_template(self, mock_hook): + mock_hook.return_value.get_instance_group_manager.return_value = \ + deepcopy(GCE_INSTANCE_GROUP_MANAGER_GET) + op = GceInstanceGroupManagerUpdateTemplateOperator( + project_id=PROJECT_ID, + zone=ZONE, + resource_id=GCE_INSTANCE_GROUP_MANAGER_NAME, + task_id='id', + source_template=GCE_INSTANCE_TEMPLATE_NON_EXISTING_URL, + destination_template=GCE_INSTANCE_TEMPLATE_DESTINATION_URL + ) + result = op.execute(None) + mock_hook.assert_called_once_with(api_version='beta', + gcp_conn_id='google_cloud_default') + mock_hook.return_value.patch_instance_group_manager.assert_not_called() + self.assertTrue(result) From 3dd1d5b55bfdfbfa67557fa9f86799d7187389d2 Mon Sep 17 00:00:00 2001 From: Jose Luis Ricon Date: Wed, 28 Nov 2018 09:57:23 +0000 Subject: [PATCH 212/808] [AIRFLOW-3384] Allow higher versions of Sqlalchemy and Jinja2 (#4227) * [AIRFLOW-3384] Allow higher versions of SQLAlchemy and Jinja2 --- setup.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/setup.py b/setup.py index b519e77a3610b..d6324cf5d9449 100644 --- a/setup.py +++ b/setup.py @@ -276,7 +276,7 @@ def do_setup(): 'dill>=0.2.2, <0.3', 'flask>=0.12.4, <0.13', 'flask-appbuilder==1.12.1', - 'flask-admin==1.4.1', + 'flask-admin==1.5.2', 'flask-caching>=1.3.3, <1.4.0', 'flask-login>=0.3, <0.5', 'flask-swagger==0.2.13', @@ -287,7 +287,7 @@ def do_setup(): 'gunicorn>=19.4.0, <20.0', 'iso8601>=0.1.12', 'json-merge-patch==0.2', - 'jinja2>=2.7.3, <2.9.0', + 'jinja2>=2.7.3, <=2.10.0', 'lxml>=4.0.0', 'markdown>=2.5.2, <3.0', 'pandas>=0.17.1, <1.0.0', @@ -299,7 +299,7 @@ def do_setup(): 'python-nvd3==0.15.0', 'requests>=2.5.1, <3', 'setproctitle>=1.1.8, <2', - 'sqlalchemy>=1.1.15, <1.2.0', + 'sqlalchemy>=1.1.15, <1.3.0', 'tabulate>=0.7.5, <=0.8.2', 'tenacity==4.8.0', 'thrift>=0.9.2', From b887603bc57250074a83aaa231c64e2995efb222 Mon Sep 17 00:00:00 2001 From: phanindhra Date: Sat, 17 Nov 2018 05:23:09 +0530 Subject: [PATCH 213/808] [AIRFLOW-3352] Fix expose_config not honoured on RBAC UI (#4194) --- airflow/www_rbac/views.py | 17 ++++++++++++----- tests/www_rbac/test_views.py | 15 ++++++++++++++- 2 files changed, 26 insertions(+), 6 deletions(-) diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 51af6c0e2a415..e2f439d20e037 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -1611,11 +1611,18 @@ def conf(self): raw = request.args.get('raw') == "true" title = "Airflow Configuration" subtitle = conf.AIRFLOW_CONFIG - with open(conf.AIRFLOW_CONFIG, 'r') as f: - config = f.read() - table = [(section, key, value, source) - for section, parameters in conf.as_dict(True, True).items() - for key, (value, source) in parameters.items()] + # Don't show config when expose_config variable is False in airflow config + if conf.getboolean("webserver", "expose_config"): + with open(conf.AIRFLOW_CONFIG, 'r') as f: + config = f.read() + table = [(section, key, value, source) + for section, parameters in conf.as_dict(True, True).items() + for key, (value, source) in parameters.items()] + else: + config = ( + "# Your Airflow administrator chose not to expose the " + "configuration, most likely for security reasons.") + table = None if raw: return Response( diff --git a/tests/www_rbac/test_views.py b/tests/www_rbac/test_views.py index f378783e90a42..5e9e51c8160a9 100644 --- a/tests/www_rbac/test_views.py +++ b/tests/www_rbac/test_views.py @@ -428,7 +428,20 @@ def test_refresh(self): class TestConfigurationView(TestBase): - def test_configuration(self): + + def test_configuration_do_not_expose_config(self): + self.logout() + self.login() + conf.set("webserver", "expose_config", "False") + resp = self.client.get('configuration', follow_redirects=True) + self.check_content_in_response( + ['Airflow Configuration', '# Your Airflow administrator chose not to expose the configuration, ' + 'most likely for security reasons.'], resp) + + def test_configuration_expose_config(self): + self.logout() + self.login() + conf.set("webserver", "expose_config", "True") resp = self.client.get('configuration', follow_redirects=True) self.check_content_in_response( ['Airflow Configuration', 'Running Configuration'], resp) From 5d50e9b56ecf761e77f880108a14850cbc8fe39d Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Sat, 29 Sep 2018 15:14:18 +0100 Subject: [PATCH 214/808] [AIRFLOW-XXX] Don't spam test logs with "bad cron expression" messages (#3973) We needed these test dags to check the behaviour of invalid cron expressions, but by default we were loading them every time we create a DagBag (which many, many tests to). Instead we ignore these known-bad dags by default, and the test checking those (tests/models.py:DagBagTest.test_process_file_cron_validity_check) is already explicitly processing those DAGs directly, so it remains tested. --- tests/dags/.airflowignore | 1 + tests/jobs.py | 18 ++++++++++++------ 2 files changed, 13 insertions(+), 6 deletions(-) create mode 100644 tests/dags/.airflowignore diff --git a/tests/dags/.airflowignore b/tests/dags/.airflowignore new file mode 100644 index 0000000000000..93361c17e24d8 --- /dev/null +++ b/tests/dags/.airflowignore @@ -0,0 +1 @@ +.*_invalid.* diff --git a/tests/jobs.py b/tests/jobs.py index f106bafaabcd3..062a69661f787 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -3319,16 +3319,22 @@ def test_list_py_file_paths(self): [JIRA-1357] Test the 'list_py_file_paths' function used by the scheduler to list and load DAGs. """ - detected_files = [] - expected_files = [] + detected_files = set() + expected_files = set() + # No_dags is empty, _invalid_ is ignored by .airflowignore + ignored_files = [ + 'no_dags.py', + 'test_invalid_cron.py', + 'test_zip_invalid_cron.zip', + ] for file_name in os.listdir(TEST_DAGS_FOLDER): if file_name.endswith('.py') or file_name.endswith('.zip'): - if file_name not in ['no_dags.py']: - expected_files.append( + if file_name not in ignored_files: + expected_files.add( '{}/{}'.format(TEST_DAGS_FOLDER, file_name)) for file_path in list_py_file_paths(TEST_DAGS_FOLDER): - detected_files.append(file_path) - self.assertEqual(sorted(detected_files), sorted(expected_files)) + detected_files.add(file_path) + self.assertEqual(detected_files, expected_files) def test_reset_orphaned_tasks_nothing(self): """Try with nothing. """ From 5600c2bf43ac3ebefb2d48aaefc265f16faff2c1 Mon Sep 17 00:00:00 2001 From: Shintaro Murakami Date: Mon, 26 Nov 2018 06:48:11 +0900 Subject: [PATCH 215/808] [AIRFLOW-1561] Fix scheduler to pick up example DAGs without other DAGs (#2635) --- airflow/jobs.py | 2 +- airflow/models.py | 12 ++++-------- airflow/utils/dag_processing.py | 7 ++++++- tests/jobs.py | 14 +++++++++++++- 4 files changed, 24 insertions(+), 11 deletions(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index 48726733982ff..bf4d12c50db01 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1670,7 +1670,7 @@ def process_file(self, file_path, zombies, pickle_dags=False, session=None): simple_dags = [] try: - dagbag = models.DagBag(file_path) + dagbag = models.DagBag(file_path, include_examples=False) except Exception: self.log.exception("Failed at reloading the DAG file %s", file_path) Stats.incr('dag_file_refresh_error', 1, 1) diff --git a/airflow/models.py b/airflow/models.py index 5248c76e07ea1..4745c771c8d28 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -277,12 +277,7 @@ def __init__( self.import_errors = {} self.has_logged = False - if include_examples: - example_dag_folder = os.path.join( - os.path.dirname(__file__), - 'example_dags') - self.collect_dags(example_dag_folder) - self.collect_dags(dag_folder) + self.collect_dags(dag_folder, include_examples) def size(self): """ @@ -516,7 +511,8 @@ def bag_dag(self, dag, parent_dag, root_dag): def collect_dags( self, dag_folder=None, - only_if_updated=True): + only_if_updated=True, + include_examples=configuration.conf.getboolean('core', 'LOAD_EXAMPLES')): """ Given a file path or a folder, this method looks for python modules, imports them and adds them to the dagbag collection. @@ -536,7 +532,7 @@ def collect_dags( stats = [] FileLoadStat = namedtuple( 'FileLoadStat', "file duration dag_num task_num dags") - for filepath in list_py_file_paths(dag_folder): + for filepath in list_py_file_paths(dag_folder, include_examples): try: ts = timezone.utcnow() found_dags = self.process_file( diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py index 47f473e9aa3d3..1b2eeeff2aa79 100644 --- a/airflow/utils/dag_processing.py +++ b/airflow/utils/dag_processing.py @@ -215,7 +215,8 @@ def get_dag(self, dag_id): return self.dag_id_to_simple_dag[dag_id] -def list_py_file_paths(directory, safe_mode=True): +def list_py_file_paths(directory, safe_mode=True, + include_examples=conf.getboolean('core', 'LOAD_EXAMPLES')): """ Traverse a directory and look for Python files. @@ -284,6 +285,10 @@ def list_py_file_paths(directory, safe_mode=True): except Exception: log = LoggingMixin().log log.exception("Error while examining %s", f) + if include_examples: + import airflow.example_dags + example_dag_folder = airflow.example_dags.__path__[0] + file_paths.extend(list_py_file_paths(example_dag_folder, safe_mode, False)) return file_paths diff --git a/tests/jobs.py b/tests/jobs.py index 062a69661f787..231944de3e8f5 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -42,6 +42,7 @@ from airflow import AirflowException, settings, models from airflow import configuration from airflow.bin import cli +import airflow.example_dags from airflow.executors import BaseExecutor, SequentialExecutor from airflow.jobs import BaseJob, BackfillJob, SchedulerJob, LocalTaskJob from airflow.models import DAG, DagModel, DagBag, DagRun, Pool, TaskInstance as TI @@ -3332,7 +3333,18 @@ def test_list_py_file_paths(self): if file_name not in ignored_files: expected_files.add( '{}/{}'.format(TEST_DAGS_FOLDER, file_name)) - for file_path in list_py_file_paths(TEST_DAGS_FOLDER): + for file_path in list_py_file_paths(TEST_DAGS_FOLDER, include_examples=False): + detected_files.add(file_path) + self.assertEqual(detected_files, expected_files) + + example_dag_folder = airflow.example_dags.__path__[0] + for root, dirs, files in os.walk(example_dag_folder): + for file_name in files: + if file_name.endswith('.py') or file_name.endswith('.zip'): + if file_name not in ['__init__.py']: + expected_files.add(os.path.join(root, file_name)) + detected_files.clear() + for file_path in list_py_file_paths(TEST_DAGS_FOLDER, include_examples=True): detected_files.add(file_path) self.assertEqual(detected_files, expected_files) From dccb8f8bb83c27d16676be7b259980bd29246534 Mon Sep 17 00:00:00 2001 From: George Leslie-Waksman Date: Sat, 21 Jul 2018 13:42:29 +0100 Subject: [PATCH 216/808] [AIRFLOW-2731] Raise psutil restriction to <6.0.0 Closes #3585 from gwax/upgrade_psutil --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index d6324cf5d9449..a76d768c2284d 100644 --- a/setup.py +++ b/setup.py @@ -292,7 +292,7 @@ def do_setup(): 'markdown>=2.5.2, <3.0', 'pandas>=0.17.1, <1.0.0', 'pendulum==1.4.4', - 'psutil>=4.2.0, <5.0.0', + 'psutil>=4.2.0, <6.0.0', 'pygments>=2.0.1, <3.0', 'python-daemon>=2.1.1, <2.2', 'python-dateutil>=2.3, <3', From 4e2f80975bd2e3c3232b261d71a0541fbeff2fcc Mon Sep 17 00:00:00 2001 From: Jon Davies Date: Mon, 20 Aug 2018 23:41:11 +0100 Subject: [PATCH 217/808] [AIRFLOW-2908] Allow retries with KubernetesExecutor. (#3758) --- airflow/www/views.py | 23 ++++++++++++++++------- airflow/www_rbac/views.py | 23 ++++++++++++++++------- 2 files changed, 32 insertions(+), 14 deletions(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index 40bb8e577e88e..483e12f66f284 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -993,16 +993,25 @@ def run(self): ignore_task_deps = request.args.get('ignore_task_deps') == "true" ignore_ti_state = request.args.get('ignore_ti_state') == "true" + from airflow.executors import GetDefaultExecutor + executor = GetDefaultExecutor() + valid_celery_config = False + valid_kubernetes_config = False + try: - from airflow.executors import GetDefaultExecutor from airflow.executors.celery_executor import CeleryExecutor - executor = GetDefaultExecutor() - if not isinstance(executor, CeleryExecutor): - flash("Only works with the CeleryExecutor, sorry", "error") - return redirect(origin) + valid_celery_config = isinstance(executor, CeleryExecutor) except ImportError: - # in case CeleryExecutor cannot be imported it is not active either - flash("Only works with the CeleryExecutor, sorry", "error") + pass + + try: + from airflow.contrib.executors.kubernetes_executor import KubernetesExecutor + valid_kubernetes_config = isinstance(executor, KubernetesExecutor) + except ImportError: + pass + + if not valid_celery_config and not valid_kubernetes_config: + flash("Only works with the Celery or Kubernetes executors, sorry", "error") return redirect(origin) ti = models.TaskInstance(task=task, execution_date=execution_date) diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index e2f439d20e037..2bfca2770621a 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -675,16 +675,25 @@ def run(self): ignore_task_deps = request.args.get('ignore_task_deps') == "true" ignore_ti_state = request.args.get('ignore_ti_state') == "true" + from airflow.executors import GetDefaultExecutor + executor = GetDefaultExecutor() + valid_celery_config = False + valid_kubernetes_config = False + try: - from airflow.executors import GetDefaultExecutor from airflow.executors.celery_executor import CeleryExecutor - executor = GetDefaultExecutor() - if not isinstance(executor, CeleryExecutor): - flash("Only works with the CeleryExecutor, sorry", "error") - return redirect(origin) + valid_celery_config = isinstance(executor, CeleryExecutor) except ImportError: - # in case CeleryExecutor cannot be imported it is not active either - flash("Only works with the CeleryExecutor, sorry", "error") + pass + + try: + from airflow.contrib.executors.kubernetes_executor import KubernetesExecutor + valid_kubernetes_config = isinstance(executor, KubernetesExecutor) + except ImportError: + pass + + if not valid_celery_config and not valid_kubernetes_config: + flash("Only works with the Celery or Kubernetes executors, sorry", "error") return redirect(origin) ti = models.TaskInstance(task=task, execution_date=execution_date) From 52601433e57d758c6b9d4934fe93ad186e4e3f66 Mon Sep 17 00:00:00 2001 From: Guoqiang Ding Date: Tue, 6 Nov 2018 03:25:03 +0800 Subject: [PATCH 218/808] [AIRFLOW-3132] Enable specifying auto_remove option for DockerOperator (#3977) --- airflow/operators/docker_operator.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py index 04dbcab5691cf..cf9838593fe18 100644 --- a/airflow/operators/docker_operator.py +++ b/airflow/operators/docker_operator.py @@ -48,6 +48,10 @@ class DockerOperator(BaseOperator): :param api_version: Remote API version. Set to ``auto`` to automatically detect the server's version. :type api_version: str + :param auto_remove: Auto-removal of the container on daemon side when the + container's process exits. + The default is False. + :type auto_remove: bool :param command: Command to be run in the container. (templated) :type command: str or list :param cpus: Number of CPUs to assign to the container. @@ -137,12 +141,14 @@ def __init__( docker_conn_id=None, dns=None, dns_search=None, + auto_remove=False, shm_size=None, *args, **kwargs): super(DockerOperator, self).__init__(*args, **kwargs) self.api_version = api_version + self.auto_remove = auto_remove self.command = command self.cpus = cpus self.dns = dns @@ -207,6 +213,7 @@ def execute(self, context): command=self.get_command(), environment=self.environment, host_config=self.cli.create_host_config( + auto_remove=self.auto_remove, binds=self.volumes, network_mode=self.network_mode, shm_size=self.shm_size, From 775a466429b238a335df6f24a87f4cfcd87b5794 Mon Sep 17 00:00:00 2001 From: aoen Date: Sat, 17 Nov 2018 06:35:37 -0500 Subject: [PATCH 219/808] [AIRFLOW-3233] Fix deletion of DAGs in the UI (#4069) --- airflow/www/templates/airflow/dags.html | 6 +++--- airflow/www_rbac/templates/airflow/dags.html | 7 ++++--- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/airflow/www/templates/airflow/dags.html b/airflow/www/templates/airflow/dags.html index 5473189d439dd..a34c6720f6520 100644 --- a/airflow/www/templates/airflow/dags.html +++ b/airflow/www/templates/airflow/dags.html @@ -191,11 +191,11 @@

      DAGs

      - + + - {% endfor %} diff --git a/airflow/www_rbac/templates/airflow/dags.html b/airflow/www_rbac/templates/airflow/dags.html index bba0d191dde35..cf23e3fe75195 100644 --- a/airflow/www_rbac/templates/airflow/dags.html +++ b/airflow/www_rbac/templates/airflow/dags.html @@ -192,9 +192,10 @@

      DAGs

      - - + + + From a5aeb1c4d12ce6ec297bc590695f644240646b4d Mon Sep 17 00:00:00 2001 From: Marcus Rehm Date: Fri, 20 Jul 2018 22:46:59 +0200 Subject: [PATCH 220/808] [AIRFLOW-2596] Add Oracle to Azure Datalake Transfer Operator Closes #3613 from marcusrehm/oracle_to_azure_datalake_transfer --- .../oracle_to_azure_data_lake_transfer.py | 113 +++++++++++++++ docs/code.rst | 1 + ...test_oracle_to_azure_data_lake_transfer.py | 135 ++++++++++++++++++ 3 files changed, 249 insertions(+) create mode 100644 airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py create mode 100644 tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py diff --git a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py new file mode 100644 index 0000000000000..06a3998defa49 --- /dev/null +++ b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py @@ -0,0 +1,113 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.hooks.oracle_hook import OracleHook +from airflow.contrib.hooks.azure_data_lake_hook import AzureDataLakeHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.file import TemporaryDirectory + +import unicodecsv as csv +import os + + +class OracleToAzureDataLakeTransfer(BaseOperator): + """ + Moves data from Oracle to Azure Data Lake. The operator runs the query against + Oracle and stores the file locally before loading it into Azure Data Lake. + + + :param filename: file name to be used by the csv file. + :type filename: str + :param azure_data_lake_conn_id: destination azure data lake connection. + :type azure_data_lake_conn_id: str + :param azure_data_lake_path: destination path in azure data lake to put the file. + :type azure_data_lake_path: str + :param oracle_conn_id: source Oracle connection. + :type oracle_conn_id: str + :param sql: SQL query to execute against the Oracle database. (templated) + :type sql: str + :param sql_params: Parameters to use in sql query. (templated) + :type sql_params: str + :param delimiter: field delimiter in the file. + :type delimiter: str + :param encoding: enconding type for the file. + :type encoding: str + :param quotechar: Character to use in quoting. + :type quotechar: str + :param quoting: Quoting strategy. See unicodecsv quoting for more information. + :type quoting: str + """ + + template_fields = ('filename', 'sql', 'sql_params') + ui_color = '#e08c8c' + + @apply_defaults + def __init__( + self, + filename, + azure_data_lake_conn_id, + azure_data_lake_path, + oracle_conn_id, + sql, + sql_params={}, + delimiter=",", + encoding="utf-8", + quotechar='"', + quoting=csv.QUOTE_MINIMAL, + *args, **kwargs): + super(OracleToAzureDataLakeTransfer, self).__init__(*args, **kwargs) + self.filename = filename + self.oracle_conn_id = oracle_conn_id + self.sql = sql + self.sql_params = sql_params + self.azure_data_lake_conn_id = azure_data_lake_conn_id + self.azure_data_lake_path = azure_data_lake_path + self.delimiter = delimiter + self.encoding = encoding + self.quotechar = quotechar + self.quoting = quoting + + def _write_temp_file(self, cursor, path_to_save): + with open(path_to_save, 'wb') as csvfile: + csv_writer = csv.writer(csvfile, delimiter=self.delimiter, + encoding=self.encoding, quotechar=self.quotechar, + quoting=self.quoting) + csv_writer.writerow(map(lambda field: field[0], cursor.description)) + csv_writer.writerows(cursor) + csvfile.flush() + + def execute(self, context): + oracle_hook = OracleHook(oracle_conn_id=self.oracle_conn_id) + azure_data_lake_hook = AzureDataLakeHook( + azure_data_lake_conn_id=self.azure_data_lake_conn_id) + + self.log.info("Dumping Oracle query results to local file") + conn = oracle_hook.get_conn() + cursor = conn.cursor() + cursor.execute(self.sql, self.sql_params) + + with TemporaryDirectory(prefix='airflow_oracle_to_azure_op_') as temp: + self._write_temp_file(cursor, os.path.join(temp, self.filename)) + self.log.info("Uploading local file to Azure Data Lake") + azure_data_lake_hook.upload_file(os.path.join(temp, self.filename), + os.path.join(self.azure_data_lake_path, + self.filename)) + cursor.close() + conn.close() diff --git a/docs/code.rst b/docs/code.rst index afa0dc4c112bc..b9b4923e2c4c0 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -188,6 +188,7 @@ Operators .. autoclass:: airflow.contrib.operators.mlengine_operator.MLEngineTrainingOperator .. autoclass:: airflow.contrib.operators.mongo_to_s3.MongoToS3Operator .. autoclass:: airflow.contrib.operators.mysql_to_gcs.MySqlToGoogleCloudStorageOperator +.. autoclass:: airflow.contrib.operators.oracle_to_azure_data_lake_transfer.OracleToAzureDataLakeTransfer .. autoclass:: airflow.contrib.operators.postgres_to_gcs_operator.PostgresToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicCreateOperator .. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicDeleteOperator diff --git a/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py b/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py new file mode 100644 index 0000000000000..e02e6312fb740 --- /dev/null +++ b/tests/contrib/operators/test_oracle_to_azure_data_lake_transfer.py @@ -0,0 +1,135 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +from airflow.contrib.operators.oracle_to_azure_data_lake_transfer \ + import OracleToAzureDataLakeTransfer +from airflow.utils.file import TemporaryDirectory +import unicodecsv as csv +import os + +try: + from unittest import mock + from unittest.mock import MagicMock +except ImportError: + try: + import mock + from mock import MagicMock + except ImportError: + mock = None + + +class OracleToAzureDataLakeTransferTest(unittest.TestCase): + + mock_module_path = 'airflow.contrib.operators.oracle_to_azure_data_lake_transfer' + + def test_write_temp_file(self): + task_id = "some_test_id" + sql = "some_sql" + sql_params = {':p_data': "2018-01-01"} + oracle_conn_id = "oracle_conn_id" + filename = "some_filename" + azure_data_lake_conn_id = 'azure_data_lake_conn_id' + azure_data_lake_path = 'azure_data_lake_path' + delimiter = '|' + encoding = 'utf-8' + cursor_description = [ + ('id', "", 39, None, 38, 0, 0), + ('description', "", 60, 240, None, None, 1) + ] + cursor_rows = [[1, 'description 1'], [2, 'description 2']] + mock_cursor = MagicMock() + mock_cursor.description = cursor_description + mock_cursor.__iter__.return_value = cursor_rows + + op = OracleToAzureDataLakeTransfer( + task_id=task_id, + filename=filename, + oracle_conn_id=oracle_conn_id, + sql=sql, + sql_params=sql_params, + azure_data_lake_conn_id=azure_data_lake_conn_id, + azure_data_lake_path=azure_data_lake_path, + delimiter=delimiter, + encoding=encoding) + + with TemporaryDirectory(prefix='airflow_oracle_to_azure_op_') as temp: + op._write_temp_file(mock_cursor, os.path.join(temp, filename)) + + assert os.path.exists(os.path.join(temp, filename)) == 1 + + with open(os.path.join(temp, filename), 'rb') as csvfile: + temp_file = csv.reader(csvfile, delimiter=delimiter, encoding=encoding) + + rownum = 0 + for row in temp_file: + if rownum == 0: + self.assertEqual(row[0], 'id') + self.assertEqual(row[1], 'description') + else: + self.assertEqual(row[0], str(cursor_rows[rownum - 1][0])) + self.assertEqual(row[1], cursor_rows[rownum - 1][1]) + rownum = rownum + 1 + + @mock.patch(mock_module_path + '.OracleHook', + autospec=True) + @mock.patch(mock_module_path + '.AzureDataLakeHook', + autospec=True) + def test_execute(self, mock_data_lake_hook, mock_oracle_hook): + task_id = "some_test_id" + sql = "some_sql" + sql_params = {':p_data': "2018-01-01"} + oracle_conn_id = "oracle_conn_id" + filename = "some_filename" + azure_data_lake_conn_id = 'azure_data_lake_conn_id' + azure_data_lake_path = 'azure_data_lake_path' + delimiter = '|' + encoding = 'latin-1' + cursor_description = [ + ('id', "", 39, None, 38, 0, 0), + ('description', "", 60, 240, None, None, 1) + ] + cursor_rows = [[1, 'description 1'], [2, 'description 2']] + cursor_mock = MagicMock() + cursor_mock.description.return_value = cursor_description + cursor_mock.__iter__.return_value = cursor_rows + mock_oracle_conn = MagicMock() + mock_oracle_conn.cursor().return_value = cursor_mock + mock_oracle_hook.get_conn().return_value = mock_oracle_conn + + op = OracleToAzureDataLakeTransfer( + task_id=task_id, + filename=filename, + oracle_conn_id=oracle_conn_id, + sql=sql, + sql_params=sql_params, + azure_data_lake_conn_id=azure_data_lake_conn_id, + azure_data_lake_path=azure_data_lake_path, + delimiter=delimiter, + encoding=encoding) + + op.execute(None) + + mock_oracle_hook.assert_called_once_with(oracle_conn_id=oracle_conn_id) + mock_data_lake_hook.assert_called_once_with( + azure_data_lake_conn_id=azure_data_lake_conn_id) + + +if __name__ == '__main__': + unittest.main() From 7f1095f23477e13c76b7e74aa760faa16e7c8d78 Mon Sep 17 00:00:00 2001 From: Marcus Rehm Date: Tue, 31 Jul 2018 16:22:40 -0300 Subject: [PATCH 221/808] [AIRFLOW-2795] Oracle to Oracle Transfer Operator (#3639) --- .../operators/oracle_to_oracle_transfer.py | 90 +++++++++++++++++++ docs/code.rst | 1 + .../test_oracle_to_oracle_transfer.py | 80 +++++++++++++++++ 3 files changed, 171 insertions(+) create mode 100644 airflow/contrib/operators/oracle_to_oracle_transfer.py create mode 100644 tests/contrib/operators/test_oracle_to_oracle_transfer.py diff --git a/airflow/contrib/operators/oracle_to_oracle_transfer.py b/airflow/contrib/operators/oracle_to_oracle_transfer.py new file mode 100644 index 0000000000000..31eb89b7dded2 --- /dev/null +++ b/airflow/contrib/operators/oracle_to_oracle_transfer.py @@ -0,0 +1,90 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.hooks.oracle_hook import OracleHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + + +class OracleToOracleTransfer(BaseOperator): + """ + Moves data from Oracle to Oracle. + + + :param oracle_destination_conn_id: destination Oracle connection. + :type oracle_destination_conn_id: str + :param destination_table: destination table to insert rows. + :type destination_table: str + :param oracle_source_conn_id: source Oracle connection. + :type oracle_source_conn_id: str + :param source_sql: SQL query to execute against the source Oracle + database. (templated) + :type source_sql: str + :param source_sql_params: Parameters to use in sql query. (templated) + :type source_sql_params: dict + :param rows_chunk: number of rows per chunk to commit. + :type rows_chunk: int + """ + + template_fields = ('source_sql', 'source_sql_params') + ui_color = '#e08c8c' + + @apply_defaults + def __init__( + self, + oracle_destination_conn_id, + destination_table, + oracle_source_conn_id, + source_sql, + source_sql_params={}, + rows_chunk=5000, + *args, **kwargs): + super(OracleToOracleTransfer, self).__init__(*args, **kwargs) + self.oracle_destination_conn_id = oracle_destination_conn_id + self.destination_table = destination_table + self.oracle_source_conn_id = oracle_source_conn_id + self.source_sql = source_sql + self.source_sql_params = source_sql_params + self.rows_chunk = rows_chunk + + def _execute(self, src_hook, dest_hook, context): + with src_hook.get_conn() as src_conn: + cursor = src_conn.cursor() + self.log.info("Querying data from source: {0}".format( + self.oracle_source_conn_id)) + cursor.execute(self.source_sql, self.source_sql_params) + target_fields = list(map(lambda field: field[0], cursor.description)) + + rows_total = 0 + rows = cursor.fetchmany(self.rows_chunk) + while len(rows) > 0: + rows_total = rows_total + len(rows) + dest_hook.bulk_insert_rows(self.destination_table, rows, + target_fields=target_fields, + commit_every=self.rows_chunk) + rows = cursor.fetchmany(self.rows_chunk) + self.log.info("Total inserted: {0} rows".format(rows_total)) + + self.log.info("Finished data transfer.") + cursor.close() + + def execute(self, context): + src_hook = OracleHook(oracle_conn_id=self.oracle_source_conn_id) + dest_hook = OracleHook(oracle_conn_id=self.oracle_destination_conn_id) + self._execute(src_hook, dest_hook, context) diff --git a/docs/code.rst b/docs/code.rst index b9b4923e2c4c0..30cef507e1e85 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -189,6 +189,7 @@ Operators .. autoclass:: airflow.contrib.operators.mongo_to_s3.MongoToS3Operator .. autoclass:: airflow.contrib.operators.mysql_to_gcs.MySqlToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.oracle_to_azure_data_lake_transfer.OracleToAzureDataLakeTransfer +.. autoclass:: airflow.contrib.operators.oracle_to_oracle_transfer.OracleToOracleTransfer .. autoclass:: airflow.contrib.operators.postgres_to_gcs_operator.PostgresToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicCreateOperator .. autoclass:: airflow.contrib.operators.pubsub_operator.PubSubTopicDeleteOperator diff --git a/tests/contrib/operators/test_oracle_to_oracle_transfer.py b/tests/contrib/operators/test_oracle_to_oracle_transfer.py new file mode 100644 index 0000000000000..83d25e05a30f0 --- /dev/null +++ b/tests/contrib/operators/test_oracle_to_oracle_transfer.py @@ -0,0 +1,80 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +from airflow.contrib.operators.oracle_to_oracle_transfer \ + import OracleToOracleTransfer + +try: + from unittest import mock + from unittest.mock import MagicMock +except ImportError: + try: + import mock + from mock import MagicMock + except ImportError: + mock = None + + +class OracleToOracleTransferTest(unittest.TestCase): + + def test_execute(self): + oracle_destination_conn_id = 'oracle_destination_conn_id' + destination_table = 'destination_table' + oracle_source_conn_id = 'oracle_source_conn_id' + source_sql = "select sysdate from dual where trunc(sysdate) = :p_data" + source_sql_params = {':p_data': "2018-01-01"} + rows_chunk = 5000 + cursor_description = [ + ('id', "", 39, None, 38, 0, 0), + ('description', "", 60, 240, None, None, 1) + ] + cursor_rows = [[1, 'description 1'], [2, 'description 2']] + + mock_dest_hook = MagicMock() + mock_src_hook = MagicMock() + mock_src_conn = mock_src_hook.get_conn.return_value.__enter__.return_value + mock_cursor = mock_src_conn.cursor.return_value + mock_cursor.description.__iter__.return_value = cursor_description + mock_cursor.fetchmany.side_effect = [cursor_rows, []] + + op = OracleToOracleTransfer( + task_id='copy_data', + oracle_destination_conn_id=oracle_destination_conn_id, + destination_table=destination_table, + oracle_source_conn_id=oracle_source_conn_id, + source_sql=source_sql, + source_sql_params=source_sql_params, + rows_chunk=rows_chunk) + + op._execute(mock_src_hook, mock_dest_hook, None) + + assert mock_src_hook.get_conn.called + assert mock_src_conn.cursor.called + mock_cursor.execute.assert_called_with(source_sql, source_sql_params) + mock_cursor.fetchmany.assert_called_with(rows_chunk) + mock_dest_hook.bulk_insert_rows.assert_called_once_with( + destination_table, + cursor_rows, + commit_every=rows_chunk, + target_fields=['id', 'description']) + + +if __name__ == '__main__': + unittest.main() From 488717b90a1046f47c2b0f0801c7b524410177e2 Mon Sep 17 00:00:00 2001 From: Jeffrey Scott Keone Payne Date: Sun, 22 Jul 2018 21:54:21 +0100 Subject: [PATCH 222/808] [AIRFLOW-2753] Add dataproc_job_id instance var holding actual DP jobId Closes #3622 from jeffkpayne/master --- .../contrib/operators/dataproc_operator.py | 61 ++++++++++++-- .../operators/test_dataproc_operator.py | 79 ++++++++++++++++--- 2 files changed, 125 insertions(+), 15 deletions(-) diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index cc7ff21b0df2a..82d0654c64ff2 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -716,6 +716,11 @@ class DataProcPigOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] template_ext = ('.pg', '.pig',) @@ -763,7 +768,10 @@ def execute(self, context): job.add_jar_file_uris(self.dataproc_jars) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataProcHiveOperator(BaseOperator): @@ -796,6 +804,11 @@ class DataProcHiveOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] template_ext = ('.q',) @@ -844,7 +857,10 @@ def execute(self, context): job.add_jar_file_uris(self.dataproc_jars) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataProcSparkSqlOperator(BaseOperator): @@ -878,6 +894,11 @@ class DataProcSparkSqlOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] template_ext = ('.q',) @@ -926,7 +947,10 @@ def execute(self, context): job.add_jar_file_uris(self.dataproc_jars) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataProcSparkOperator(BaseOperator): @@ -967,6 +991,11 @@ class DataProcSparkOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] @@ -1017,7 +1046,10 @@ def execute(self, context): job.add_file_uris(self.files) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataProcHadoopOperator(BaseOperator): @@ -1058,6 +1090,11 @@ class DataProcHadoopOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] @@ -1108,10 +1145,14 @@ def execute(self, context): job.add_file_uris(self.files) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataProcPySparkOperator(BaseOperator): + # TODO Add docs around dataproc_job_id. """ Start a PySpark Job on a Cloud DataProc cluster. @@ -1149,6 +1190,11 @@ class DataProcPySparkOperator(BaseOperator): :type delegate_to: string :param region: The specified region where the dataproc cluster is created. :type region: string + :var dataproc_job_id: The actual "jobId" as submitted to the Dataproc API. + This is useful for identifying or linking to the job in the Google Cloud Console + Dataproc UI, as the actual "jobId" submitted to the Dataproc API is appended with + an 8 character random string. + :vartype dataproc_job_id: string """ template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] @@ -1239,7 +1285,10 @@ def execute(self, context): job.add_python_file_uris(self.pyfiles) job.set_job_name(self.job_name) - hook.submit(hook.project_id, job.build(), self.region) + job_to_submit = job.build() + self.dataproc_job_id = job_to_submit["job"]["reference"]["jobId"] + + hook.submit(hook.project_id, job_to_submit, self.region) class DataprocWorkflowTemplateBaseOperator(BaseOperator): diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py index 4d8540eb86202..aea74a7abfc71 100644 --- a/tests/contrib/operators/test_dataproc_operator.py +++ b/tests/contrib/operators/test_dataproc_operator.py @@ -45,7 +45,7 @@ except ImportError: mock = None -from mock import Mock +from mock import MagicMock, Mock from mock import patch TASK_ID = 'test-dataproc-operator' @@ -83,6 +83,27 @@ TEMPLATE_ID = 'template-id' HOOK = 'airflow.contrib.operators.dataproc_operator.DataProcHook' +DATAPROC_JOB_ID = 'dataproc_job_id' +DATAPROC_JOB_TO_SUBMIT = { + 'job': { + 'reference': { + 'projectId': PROJECT_ID, + 'jobId': DATAPROC_JOB_ID, + }, + 'placement': { + 'clusterName': CLUSTER_NAME + } + } +} + + +def _assert_dataproc_job_id(mock_hook, dataproc_task): + hook = mock_hook.return_value + job = MagicMock() + job.build.return_value = DATAPROC_JOB_TO_SUBMIT + hook.create_job_template.return_value = job + dataproc_task.execute(None) + assert dataproc_task.dataproc_job_id == DATAPROC_JOB_ID class DataprocClusterCreateOperatorTest(unittest.TestCase): @@ -502,31 +523,51 @@ def test_cluster_name_log_sub(self): class DataProcHadoopOperatorTest(unittest.TestCase): # Unit test for the DataProcHadoopOperator def test_hook_correct_region(self): - with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook: + with patch(HOOK) as mock_hook: dataproc_task = DataProcHadoopOperator( task_id=TASK_ID, region=REGION ) dataproc_task.execute(None) - mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) + mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, + REGION) + + def test_dataproc_job_id_is_set(self): + with patch(HOOK) as mock_hook: + dataproc_task = DataProcHadoopOperator( + task_id=TASK_ID + ) + + _assert_dataproc_job_id(mock_hook, dataproc_task) + class DataProcHiveOperatorTest(unittest.TestCase): # Unit test for the DataProcHiveOperator def test_hook_correct_region(self): - with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook: + with patch(HOOK) as mock_hook: dataproc_task = DataProcHiveOperator( task_id=TASK_ID, region=REGION ) dataproc_task.execute(None) - mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) + mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, + REGION) + + def test_dataproc_job_id_is_set(self): + with patch(HOOK) as mock_hook: + dataproc_task = DataProcHiveOperator( + task_id=TASK_ID + ) + + _assert_dataproc_job_id(mock_hook, dataproc_task) + class DataProcPySparkOperatorTest(unittest.TestCase): # Unit test for the DataProcPySparkOperator def test_hook_correct_region(self): - with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook: + with patch(HOOK) as mock_hook: dataproc_task = DataProcPySparkOperator( task_id=TASK_ID, main=MAIN_URI, @@ -534,19 +575,39 @@ def test_hook_correct_region(self): ) dataproc_task.execute(None) - mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) + mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, + REGION) + + def test_dataproc_job_id_is_set(self): + with patch(HOOK) as mock_hook: + dataproc_task = DataProcPySparkOperator( + task_id=TASK_ID, + main=MAIN_URI + ) + + _assert_dataproc_job_id(mock_hook, dataproc_task) + class DataProcSparkOperatorTest(unittest.TestCase): # Unit test for the DataProcSparkOperator def test_hook_correct_region(self): - with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook: + with patch(HOOK) as mock_hook: dataproc_task = DataProcSparkOperator( task_id=TASK_ID, region=REGION ) dataproc_task.execute(None) - mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) + mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, + REGION) + + def test_dataproc_job_id_is_set(self): + with patch(HOOK) as mock_hook: + dataproc_task = DataProcSparkOperator( + task_id=TASK_ID + ) + + _assert_dataproc_job_id(mock_hook, dataproc_task) class DataprocWorkflowTemplateInstantiateOperatorTest(unittest.TestCase): From 2e9615949c001829ae36c0875e7d5db6d4fd4882 Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Wed, 18 Jul 2018 08:54:19 -0700 Subject: [PATCH 223/808] [AIRFLOW-2705] Move class-level moto decorator to method-level Moto decorators at class-level make other tests that sends HTTP request fail. This PR moves them to method-level so as to fix this problem. Closes #3565 from sekikn/AIRFLOW-2705 --- tests/contrib/hooks/test_redshift_hook.py | 15 ++++++++++++++- .../sensors/test_aws_redshift_cluster_sensor.py | 9 ++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/tests/contrib/hooks/test_redshift_hook.py b/tests/contrib/hooks/test_redshift_hook.py index d5e791b4b7c80..c69ed8a9dca2a 100644 --- a/tests/contrib/hooks/test_redshift_hook.py +++ b/tests/contrib/hooks/test_redshift_hook.py @@ -31,10 +31,11 @@ mock_redshift = None -@mock_redshift class TestRedshiftHook(unittest.TestCase): def setUp(self): configuration.load_test_config() + + def _create_clusters(self): client = boto3.client('redshift', region_name='us-east-1') client.create_cluster( ClusterIdentifier='test_cluster', @@ -52,7 +53,9 @@ def setUp(self): raise ValueError('AWS not properly mocked') @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_get_client_type_returns_a_boto3_client_of_the_requested_type(self): + self._create_clusters() hook = AwsHook(aws_conn_id='aws_default') client_from_hook = hook.get_client_type('redshift') @@ -60,7 +63,9 @@ def test_get_client_type_returns_a_boto3_client_of_the_requested_type(self): self.assertEqual(len(clusters), 2) @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_restore_from_cluster_snapshot_returns_dict_with_cluster_data(self): + self._create_clusters() hook = RedshiftHook(aws_conn_id='aws_default') hook.create_cluster_snapshot('test_snapshot', 'test_cluster') self.assertEqual( @@ -70,27 +75,35 @@ def test_restore_from_cluster_snapshot_returns_dict_with_cluster_data(self): 'test_cluster_3') @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_delete_cluster_returns_a_dict_with_cluster_data(self): + self._create_clusters() hook = RedshiftHook(aws_conn_id='aws_default') cluster = hook.delete_cluster('test_cluster_2') self.assertNotEqual(cluster, None) @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_create_cluster_snapshot_returns_snapshot_data(self): + self._create_clusters() hook = RedshiftHook(aws_conn_id='aws_default') snapshot = hook.create_cluster_snapshot('test_snapshot_2', 'test_cluster') self.assertNotEqual(snapshot, None) @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_cluster_status_returns_cluster_not_found(self): + self._create_clusters() hook = RedshiftHook(aws_conn_id='aws_default') status = hook.cluster_status('test_cluster_not_here') self.assertEqual(status, 'cluster_not_found') @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_cluster_status_returns_available_cluster(self): + self._create_clusters() hook = RedshiftHook(aws_conn_id='aws_default') status = hook.cluster_status('test_cluster') self.assertEqual(status, 'available') diff --git a/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py b/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py index 5bee24aeda548..921b2dac1fae9 100644 --- a/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py +++ b/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py @@ -31,10 +31,11 @@ mock_redshift = None -@mock_redshift class TestAwsRedshiftClusterSensor(unittest.TestCase): def setUp(self): configuration.load_test_config() + + def _create_cluster(self): client = boto3.client('redshift', region_name='us-east-1') client.create_cluster( ClusterIdentifier='test_cluster', @@ -46,7 +47,9 @@ def setUp(self): raise ValueError('AWS not properly mocked') @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_poke(self): + self._create_cluster() op = AwsRedshiftClusterSensor(task_id='test_cluster_sensor', poke_interval=1, timeout=5, @@ -56,7 +59,9 @@ def test_poke(self): self.assertTrue(op.poke(None)) @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_poke_false(self): + self._create_cluster() op = AwsRedshiftClusterSensor(task_id='test_cluster_sensor', poke_interval=1, timeout=5, @@ -67,7 +72,9 @@ def test_poke_false(self): self.assertFalse(op.poke(None)) @unittest.skipIf(mock_redshift is None, 'mock_redshift package not present') + @mock_redshift def test_poke_cluster_not_found(self): + self._create_cluster() op = AwsRedshiftClusterSensor(task_id='test_cluster_sensor', poke_interval=1, timeout=5, From e0f7cb3bc06bcb1dac2f560ca84a9d66bffe1849 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 8 Aug 2018 00:18:42 +0100 Subject: [PATCH 224/808] [AIRFLOW-2867] Refactor Code to conform standards (#3714) --- airflow/contrib/hooks/bigquery_hook.py | 32 ++- airflow/contrib/hooks/databricks_hook.py | 3 +- airflow/contrib/hooks/datastore_hook.py | 4 +- airflow/contrib/hooks/gcp_container_hook.py | 6 +- airflow/contrib/hooks/gcs_hook.py | 2 +- airflow/contrib/hooks/salesforce_hook.py | 3 +- airflow/contrib/hooks/vertica_hook.py | 2 +- airflow/contrib/kubernetes/pod_launcher.py | 2 +- .../contrib/operators/dataflow_operator.py | 2 +- .../contrib/operators/dataproc_operator.py | 6 +- .../operators/gcp_container_operator.py | 4 +- airflow/contrib/operators/gcs_to_bq.py | 8 +- .../operators/mlengine_prediction_summary.py | 6 +- airflow/contrib/operators/mongo_to_s3.py | 6 +- airflow/contrib/operators/mysql_to_gcs.py | 3 +- .../oracle_to_azure_data_lake_transfer.py | 228 +++++++++--------- .../operators/oracle_to_oracle_transfer.py | 4 +- .../contrib/operators/s3_to_gcs_operator.py | 3 +- .../contrib/sensors/emr_job_flow_sensor.py | 3 +- airflow/contrib/sensors/emr_step_sensor.py | 3 +- .../contrib/task_runner/cgroup_task_runner.py | 3 +- airflow/executors/dask_executor.py | 2 +- airflow/hooks/S3_hook.py | 10 +- airflow/hooks/druid_hook.py | 2 +- airflow/hooks/hive_hooks.py | 3 +- airflow/hooks/presto_hook.py | 3 +- airflow/jobs.py | 2 +- airflow/operators/check_operator.py | 2 +- airflow/operators/hive_stats_operator.py | 3 +- airflow/operators/python_operator.py | 3 +- airflow/operators/s3_to_hive_operator.py | 4 +- airflow/sensors/hdfs_sensor.py | 4 +- airflow/utils/cli.py | 6 +- airflow/utils/helpers.py | 2 +- airflow/utils/log/gcs_task_handler.py | 3 +- airflow/www/views.py | 2 +- airflow/www_rbac/forms.py | 2 +- airflow/www_rbac/views.py | 2 +- dev/airflow-pr | 2 +- scripts/perf/scheduler_ops_metrics.py | 2 +- tests/cli/test_cli.py | 4 +- .../executors/test_kubernetes_executor.py | 6 +- tests/contrib/hooks/test_aws_lambda_hook.py | 3 +- tests/contrib/hooks/test_gcp_mlengine_hook.py | 3 +- tests/contrib/hooks/test_mongo_hook.py | 4 +- tests/contrib/hooks/test_redshift_hook.py | 3 +- .../minikube/test_kubernetes_executor.py | 3 +- .../minikube/test_kubernetes_pod_operator.py | 15 +- .../operators/test_dataproc_operator.py | 24 +- .../test_hive_to_dynamodb_operator.py | 3 +- .../operators/test_mysql_to_gcs_operator.py | 11 +- .../test_oracle_to_oracle_transfer.py | 3 +- .../test_aws_redshift_cluster_sensor.py | 3 +- tests/contrib/sensors/test_emr_base_sensor.py | 24 +- tests/core.py | 2 +- tests/models.py | 6 +- tests/operators/test_docker_operator.py | 3 +- tests/operators/test_s3_to_hive_operator.py | 8 +- tests/operators/test_virtualenv_operator.py | 3 +- .../experimental/test_kerberos_endpoints.py | 2 +- .../experimental/test_kerberos_endpoints.py | 2 +- 61 files changed, 302 insertions(+), 225 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index f9abdbaaacdab..b255bb1686b7f 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -206,7 +206,7 @@ def create_empty_table(self, dataset_id, table_id, schema_fields=None, - time_partitioning={}, + time_partitioning=None, labels=None ): """ @@ -238,6 +238,8 @@ def create_empty_table(self, :return: """ + if time_partitioning is None: + time_partitioning = dict() project_id = project_id if project_id is not None else self.project_id table_resource = { @@ -286,7 +288,7 @@ def create_external_table(self, quote_character=None, allow_quoted_newlines=False, allow_jagged_rows=False, - src_fmt_configs={}, + src_fmt_configs=None, labels=None ): """ @@ -352,6 +354,8 @@ def create_external_table(self, :type labels: dict """ + if src_fmt_configs is None: + src_fmt_configs = {} project_id, dataset_id, external_table_id = \ _split_tablename(table_input=external_project_dataset_table, default_project_id=self.project_id, @@ -482,7 +486,7 @@ def run_query(self, labels=None, schema_update_options=(), priority='INTERACTIVE', - time_partitioning={}): + time_partitioning=None): """ Executes a BigQuery SQL query. Optionally persists results in a BigQuery table. See here: @@ -546,6 +550,8 @@ def run_query(self, """ # TODO remove `bql` in Airflow 2.0 - Jira: [AIRFLOW-2513] + if time_partitioning is None: + time_partitioning = {} sql = bql if sql is None else sql if bql: @@ -806,8 +812,8 @@ def run_load(self, allow_quoted_newlines=False, allow_jagged_rows=False, schema_update_options=(), - src_fmt_configs={}, - time_partitioning={}): + src_fmt_configs=None, + time_partitioning=None): """ Executes a BigQuery load command to load data from Google Cloud Storage to BigQuery. See here: @@ -876,6 +882,10 @@ def run_load(self, # if it's not, we raise a ValueError # Refer to this link for more details: # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.query.tableDefinitions.(key).sourceFormat + if src_fmt_configs is None: + src_fmt_configs = {} + if time_partitioning is None: + time_partitioning = {} source_format = source_format.upper() allowed_formats = [ "CSV", "NEWLINE_DELIMITED_JSON", "AVRO", "GOOGLE_SHEETS", @@ -1007,12 +1017,12 @@ def run_with_configuration(self, configuration): # Wait for query to finish. keep_polling_job = True - while (keep_polling_job): + while keep_polling_job: try: job = jobs.get( projectId=self.project_id, jobId=self.running_job_id).execute() - if (job['status']['state'] == 'DONE'): + if job['status']['state'] == 'DONE': keep_polling_job = False # Check if job had errors. if 'errorResult' in job['status']: @@ -1041,7 +1051,7 @@ def poll_job_complete(self, job_id): jobs = self.service.jobs() try: job = jobs.get(projectId=self.project_id, jobId=job_id).execute() - if (job['status']['state'] == 'DONE'): + if job['status']['state'] == 'DONE': return True except HttpError as err: if err.resp.status in [500, 503]: @@ -1075,13 +1085,13 @@ def cancel_query(self): polling_attempts = 0 job_complete = False - while (polling_attempts < max_polling_attempts and not job_complete): + while polling_attempts < max_polling_attempts and not job_complete: polling_attempts = polling_attempts + 1 job_complete = self.poll_job_complete(self.running_job_id) - if (job_complete): + if job_complete: self.log.info('Job successfully canceled: %s, %s', self.project_id, self.running_job_id) - elif (polling_attempts == max_polling_attempts): + elif polling_attempts == max_polling_attempts: self.log.info( "Stopping polling due to timeout. Job with id %s " "has not completed cancel and may or may not finish.", diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py index 4e05ddb0b1af4..cb2ba9bd00fd7 100644 --- a/airflow/contrib/hooks/databricks_hook.py +++ b/airflow/contrib/hooks/databricks_hook.py @@ -74,7 +74,8 @@ def __init__( self.retry_limit = retry_limit self.retry_delay = retry_delay - def _parse_host(self, host): + @staticmethod + def _parse_host(host): """ The purpose of this function is to be robust to improper connections settings provided by users, specifically in the host field. diff --git a/airflow/contrib/hooks/datastore_hook.py b/airflow/contrib/hooks/datastore_hook.py index 5e54cf2a65384..b8c3ca00a0f46 100644 --- a/airflow/contrib/hooks/datastore_hook.py +++ b/airflow/contrib/hooks/datastore_hook.py @@ -172,7 +172,7 @@ def export_to_storage_bucket(self, bucket, namespace=None, """ Export entities from Cloud Datastore to Cloud Storage for backup """ - output_uri_prefix = 'gs://' + ('/').join(filter(None, [bucket, namespace])) + output_uri_prefix = 'gs://' + '/'.join(filter(None, [bucket, namespace])) if not entity_filter: entity_filter = {} if not labels: @@ -191,7 +191,7 @@ def import_from_storage_bucket(self, bucket, file, """ Import a backup from Cloud Storage to Cloud Datastore """ - input_url = 'gs://' + ('/').join(filter(None, [bucket, namespace, file])) + input_url = 'gs://' + '/'.join(filter(None, [bucket, namespace, file])) if not entity_filter: entity_filter = {} if not labels: diff --git a/airflow/contrib/hooks/gcp_container_hook.py b/airflow/contrib/hooks/gcp_container_hook.py index 3cc3744c03686..970d4ec8992c0 100644 --- a/airflow/contrib/hooks/gcp_container_hook.py +++ b/airflow/contrib/hooks/gcp_container_hook.py @@ -44,7 +44,8 @@ def __init__(self, project_id, location): client_info = ClientInfo(client_library_version='airflow_v' + version.version) self.client = container_v1.ClusterManagerClient(client_info=client_info) - def _dict_to_proto(self, py_dict, proto): + @staticmethod + def _dict_to_proto(py_dict, proto): """ Converts a python dictionary to the proto supplied @@ -93,7 +94,8 @@ def get_operation(self, operation_name): zone=self.location, operation_id=operation_name) - def _append_label(self, cluster_proto, key, val): + @staticmethod + def _append_label(cluster_proto, key, val): """ Append labels to provided Cluster Protobuf diff --git a/airflow/contrib/hooks/gcs_hook.py b/airflow/contrib/hooks/gcs_hook.py index b70825eb61f39..7b5b7194c004d 100644 --- a/airflow/contrib/hooks/gcs_hook.py +++ b/airflow/contrib/hooks/gcs_hook.py @@ -326,7 +326,7 @@ def list(self, bucket, versions=None, maxResults=None, prefix=None, delimiter=No ids = list() pageToken = None - while(True): + while True: response = service.objects().list( bucket=bucket, versions=versions, diff --git a/airflow/contrib/hooks/salesforce_hook.py b/airflow/contrib/hooks/salesforce_hook.py index ee18b353d2e82..24b67f49fc402 100644 --- a/airflow/contrib/hooks/salesforce_hook.py +++ b/airflow/contrib/hooks/salesforce_hook.py @@ -135,7 +135,8 @@ def get_available_fields(self, obj): return [f['name'] for f in desc['fields']] - def _build_field_list(self, fields): + @staticmethod + def _build_field_list(fields): # join all of the fields in a comma separated list return ",".join(fields) diff --git a/airflow/contrib/hooks/vertica_hook.py b/airflow/contrib/hooks/vertica_hook.py index f3411de994d7f..e6b36b51d5937 100644 --- a/airflow/contrib/hooks/vertica_hook.py +++ b/airflow/contrib/hooks/vertica_hook.py @@ -41,9 +41,9 @@ def get_conn(self): "user": conn.login, "password": conn.password or '', "database": conn.schema, + "host": conn.host or 'localhost' } - conn_config["host"] = conn.host or 'localhost' if not conn.port: conn_config["port"] = 5433 else: diff --git a/airflow/contrib/kubernetes/pod_launcher.py b/airflow/contrib/kubernetes/pod_launcher.py index f34de419ae02f..8c8d949107494 100644 --- a/airflow/contrib/kubernetes/pod_launcher.py +++ b/airflow/contrib/kubernetes/pod_launcher.py @@ -113,7 +113,7 @@ def _monitor_pod(self, pod, get_logs): while self.pod_is_running(pod): self.log.info('Pod %s has state %s', pod.name, State.RUNNING) time.sleep(2) - return (self._task_status(self.read_pod(pod)), result) + return self._task_status(self.read_pod(pod)), result def _task_status(self, event): self.log.info( diff --git a/airflow/contrib/operators/dataflow_operator.py b/airflow/contrib/operators/dataflow_operator.py index 792488156dcf3..4a1be8c918027 100644 --- a/airflow/contrib/operators/dataflow_operator.py +++ b/airflow/contrib/operators/dataflow_operator.py @@ -332,7 +332,7 @@ def execute(self, context): self.py_file, self.py_options) -class GoogleCloudBucketHelper(): +class GoogleCloudBucketHelper(object): """GoogleCloudStorageHook helper class to download GCS object.""" GCS_PREFIX_LENGTH = 5 diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index 82d0654c64ff2..f8b515961115c 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -538,7 +538,8 @@ def _build_scale_cluster_data(self): } return scale_data - def _get_graceful_decommission_timeout(self, timeout): + @staticmethod + def _get_graceful_decommission_timeout(timeout): match = re.match(r"^(\d+)(s|m|h|d)$", timeout) if match: if match.group(2) == "s": @@ -622,7 +623,8 @@ def __init__(self, self.project_id = project_id self.region = region - def _wait_for_done(self, service, operation_name): + @staticmethod + def _wait_for_done(service, operation_name): time.sleep(15) while True: response = service.projects().regions().operations().get( diff --git a/airflow/contrib/operators/gcp_container_operator.py b/airflow/contrib/operators/gcp_container_operator.py index 5648b4d8a01b3..9c1eb12d7275c 100644 --- a/airflow/contrib/operators/gcp_container_operator.py +++ b/airflow/contrib/operators/gcp_container_operator.py @@ -94,7 +94,7 @@ class GKEClusterCreateOperator(BaseOperator): def __init__(self, project_id, location, - body={}, + body=None, gcp_conn_id='google_cloud_default', api_version='v2', *args, @@ -143,6 +143,8 @@ def __init__(self, """ super(GKEClusterCreateOperator, self).__init__(*args, **kwargs) + if body is None: + body = {} self.project_id = project_id self.gcp_conn_id = gcp_conn_id self.location = location diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py index 3a7798030cf49..533cf01de5666 100644 --- a/airflow/contrib/operators/gcs_to_bq.py +++ b/airflow/contrib/operators/gcs_to_bq.py @@ -143,14 +143,18 @@ def __init__(self, google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, schema_update_options=(), - src_fmt_configs={}, + src_fmt_configs=None, external_table=False, - time_partitioning={}, + time_partitioning=None, *args, **kwargs): super(GoogleCloudStorageToBigQueryOperator, self).__init__(*args, **kwargs) # GCS config + if src_fmt_configs is None: + src_fmt_configs = {} + if time_partitioning is None: + time_partitioning = {} self.bucket = bucket self.source_objects = source_objects self.schema_object = schema_object diff --git a/airflow/contrib/operators/mlengine_prediction_summary.py b/airflow/contrib/operators/mlengine_prediction_summary.py index 4efe81e64151f..5dac0a44a9dcb 100644 --- a/airflow/contrib/operators/mlengine_prediction_summary.py +++ b/airflow/contrib/operators/mlengine_prediction_summary.py @@ -102,10 +102,12 @@ def metric_fn(inst): class JsonCoder(object): - def encode(self, x): + @staticmethod + def encode(x): return json.dumps(x) - def decode(self, x): + @staticmethod + def decode(x): return json.loads(x) diff --git a/airflow/contrib/operators/mongo_to_s3.py b/airflow/contrib/operators/mongo_to_s3.py index 43b5d8b6c357a..8bfa7a52f80bb 100644 --- a/airflow/contrib/operators/mongo_to_s3.py +++ b/airflow/contrib/operators/mongo_to_s3.py @@ -96,7 +96,8 @@ def execute(self, context): return True - def _stringify(self, iterable, joinable='\n'): + @staticmethod + def _stringify(iterable, joinable='\n'): """ Takes an iterable (pymongo Cursor or Array) containing dictionaries and returns a stringified version using python join @@ -105,7 +106,8 @@ def _stringify(self, iterable, joinable='\n'): [json.dumps(doc, default=json_util.default) for doc in iterable] ) - def transform(self, docs): + @staticmethod + def transform(docs): """ Processes pyMongo cursor and returns an iterable with each element being a JSON serializable dictionary diff --git a/airflow/contrib/operators/mysql_to_gcs.py b/airflow/contrib/operators/mysql_to_gcs.py index 4d1bb7b329e6c..eb47465e8e8ec 100644 --- a/airflow/contrib/operators/mysql_to_gcs.py +++ b/airflow/contrib/operators/mysql_to_gcs.py @@ -218,7 +218,8 @@ def _upload_to_gcs(self, files_to_upload): for object, tmp_file_handle in files_to_upload.items(): hook.upload(self.bucket, object, tmp_file_handle.name, 'application/json') - def _convert_types(self, schema, col_type_dict, row): + @staticmethod + def _convert_types(schema, col_type_dict, row): """ Takes a value from MySQLdb, and converts it to a value that's safe for JSON/Google cloud storage/BigQuery. Dates are converted to UTC seconds. diff --git a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py index 06a3998defa49..80cec8f462d9a 100644 --- a/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py +++ b/airflow/contrib/operators/oracle_to_azure_data_lake_transfer.py @@ -1,113 +1,115 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -from airflow.hooks.oracle_hook import OracleHook -from airflow.contrib.hooks.azure_data_lake_hook import AzureDataLakeHook -from airflow.models import BaseOperator -from airflow.utils.decorators import apply_defaults -from airflow.utils.file import TemporaryDirectory - -import unicodecsv as csv -import os - - -class OracleToAzureDataLakeTransfer(BaseOperator): - """ - Moves data from Oracle to Azure Data Lake. The operator runs the query against - Oracle and stores the file locally before loading it into Azure Data Lake. - - - :param filename: file name to be used by the csv file. - :type filename: str - :param azure_data_lake_conn_id: destination azure data lake connection. - :type azure_data_lake_conn_id: str - :param azure_data_lake_path: destination path in azure data lake to put the file. - :type azure_data_lake_path: str - :param oracle_conn_id: source Oracle connection. - :type oracle_conn_id: str - :param sql: SQL query to execute against the Oracle database. (templated) - :type sql: str - :param sql_params: Parameters to use in sql query. (templated) - :type sql_params: str - :param delimiter: field delimiter in the file. - :type delimiter: str - :param encoding: enconding type for the file. - :type encoding: str - :param quotechar: Character to use in quoting. - :type quotechar: str - :param quoting: Quoting strategy. See unicodecsv quoting for more information. - :type quoting: str - """ - - template_fields = ('filename', 'sql', 'sql_params') - ui_color = '#e08c8c' - - @apply_defaults - def __init__( - self, - filename, - azure_data_lake_conn_id, - azure_data_lake_path, - oracle_conn_id, - sql, - sql_params={}, - delimiter=",", - encoding="utf-8", - quotechar='"', - quoting=csv.QUOTE_MINIMAL, - *args, **kwargs): - super(OracleToAzureDataLakeTransfer, self).__init__(*args, **kwargs) - self.filename = filename - self.oracle_conn_id = oracle_conn_id - self.sql = sql - self.sql_params = sql_params - self.azure_data_lake_conn_id = azure_data_lake_conn_id - self.azure_data_lake_path = azure_data_lake_path - self.delimiter = delimiter - self.encoding = encoding - self.quotechar = quotechar - self.quoting = quoting - - def _write_temp_file(self, cursor, path_to_save): - with open(path_to_save, 'wb') as csvfile: - csv_writer = csv.writer(csvfile, delimiter=self.delimiter, - encoding=self.encoding, quotechar=self.quotechar, - quoting=self.quoting) - csv_writer.writerow(map(lambda field: field[0], cursor.description)) - csv_writer.writerows(cursor) - csvfile.flush() - - def execute(self, context): - oracle_hook = OracleHook(oracle_conn_id=self.oracle_conn_id) - azure_data_lake_hook = AzureDataLakeHook( - azure_data_lake_conn_id=self.azure_data_lake_conn_id) - - self.log.info("Dumping Oracle query results to local file") - conn = oracle_hook.get_conn() - cursor = conn.cursor() - cursor.execute(self.sql, self.sql_params) - - with TemporaryDirectory(prefix='airflow_oracle_to_azure_op_') as temp: - self._write_temp_file(cursor, os.path.join(temp, self.filename)) - self.log.info("Uploading local file to Azure Data Lake") - azure_data_lake_hook.upload_file(os.path.join(temp, self.filename), - os.path.join(self.azure_data_lake_path, - self.filename)) - cursor.close() - conn.close() +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.hooks.oracle_hook import OracleHook +from airflow.contrib.hooks.azure_data_lake_hook import AzureDataLakeHook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults +from airflow.utils.file import TemporaryDirectory + +import unicodecsv as csv +import os + + +class OracleToAzureDataLakeTransfer(BaseOperator): + """ + Moves data from Oracle to Azure Data Lake. The operator runs the query against + Oracle and stores the file locally before loading it into Azure Data Lake. + + + :param filename: file name to be used by the csv file. + :type filename: str + :param azure_data_lake_conn_id: destination azure data lake connection. + :type azure_data_lake_conn_id: str + :param azure_data_lake_path: destination path in azure data lake to put the file. + :type azure_data_lake_path: str + :param oracle_conn_id: source Oracle connection. + :type oracle_conn_id: str + :param sql: SQL query to execute against the Oracle database. (templated) + :type sql: str + :param sql_params: Parameters to use in sql query. (templated) + :type sql_params: str + :param delimiter: field delimiter in the file. + :type delimiter: str + :param encoding: enconding type for the file. + :type encoding: str + :param quotechar: Character to use in quoting. + :type quotechar: str + :param quoting: Quoting strategy. See unicodecsv quoting for more information. + :type quoting: str + """ + + template_fields = ('filename', 'sql', 'sql_params') + ui_color = '#e08c8c' + + @apply_defaults + def __init__( + self, + filename, + azure_data_lake_conn_id, + azure_data_lake_path, + oracle_conn_id, + sql, + sql_params=None, + delimiter=",", + encoding="utf-8", + quotechar='"', + quoting=csv.QUOTE_MINIMAL, + *args, **kwargs): + super(OracleToAzureDataLakeTransfer, self).__init__(*args, **kwargs) + if sql_params is None: + sql_params = {} + self.filename = filename + self.oracle_conn_id = oracle_conn_id + self.sql = sql + self.sql_params = sql_params + self.azure_data_lake_conn_id = azure_data_lake_conn_id + self.azure_data_lake_path = azure_data_lake_path + self.delimiter = delimiter + self.encoding = encoding + self.quotechar = quotechar + self.quoting = quoting + + def _write_temp_file(self, cursor, path_to_save): + with open(path_to_save, 'wb') as csvfile: + csv_writer = csv.writer(csvfile, delimiter=self.delimiter, + encoding=self.encoding, quotechar=self.quotechar, + quoting=self.quoting) + csv_writer.writerow(map(lambda field: field[0], cursor.description)) + csv_writer.writerows(cursor) + csvfile.flush() + + def execute(self, context): + oracle_hook = OracleHook(oracle_conn_id=self.oracle_conn_id) + azure_data_lake_hook = AzureDataLakeHook( + azure_data_lake_conn_id=self.azure_data_lake_conn_id) + + self.log.info("Dumping Oracle query results to local file") + conn = oracle_hook.get_conn() + cursor = conn.cursor() + cursor.execute(self.sql, self.sql_params) + + with TemporaryDirectory(prefix='airflow_oracle_to_azure_op_') as temp: + self._write_temp_file(cursor, os.path.join(temp, self.filename)) + self.log.info("Uploading local file to Azure Data Lake") + azure_data_lake_hook.upload_file(os.path.join(temp, self.filename), + os.path.join(self.azure_data_lake_path, + self.filename)) + cursor.close() + conn.close() diff --git a/airflow/contrib/operators/oracle_to_oracle_transfer.py b/airflow/contrib/operators/oracle_to_oracle_transfer.py index 31eb89b7dded2..1db95f7520bb1 100644 --- a/airflow/contrib/operators/oracle_to_oracle_transfer.py +++ b/airflow/contrib/operators/oracle_to_oracle_transfer.py @@ -52,10 +52,12 @@ def __init__( destination_table, oracle_source_conn_id, source_sql, - source_sql_params={}, + source_sql_params=None, rows_chunk=5000, *args, **kwargs): super(OracleToOracleTransfer, self).__init__(*args, **kwargs) + if source_sql_params is None: + source_sql_params = {} self.oracle_destination_conn_id = oracle_destination_conn_id self.destination_table = destination_table self.oracle_source_conn_id = oracle_source_conn_id diff --git a/airflow/contrib/operators/s3_to_gcs_operator.py b/airflow/contrib/operators/s3_to_gcs_operator.py index 2898af1071773..64d7dc7cab976 100644 --- a/airflow/contrib/operators/s3_to_gcs_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_operator.py @@ -184,7 +184,8 @@ def execute(self, context): # Following functionality may be better suited in # airflow/contrib/hooks/gcs_hook.py - def _gcs_object_is_directory(self, object): + @staticmethod + def _gcs_object_is_directory(object): bucket, blob = _parse_gcs_url(object) return len(blob) == 0 or blob.endswith('/') diff --git a/airflow/contrib/sensors/emr_job_flow_sensor.py b/airflow/contrib/sensors/emr_job_flow_sensor.py index 31d16a066b241..4193c2d53ed4e 100644 --- a/airflow/contrib/sensors/emr_job_flow_sensor.py +++ b/airflow/contrib/sensors/emr_job_flow_sensor.py @@ -50,5 +50,6 @@ def get_emr_response(self): self.log.info('Poking cluster %s', self.job_flow_id) return emr.describe_cluster(ClusterId=self.job_flow_id) - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['Cluster']['Status']['State'] diff --git a/airflow/contrib/sensors/emr_step_sensor.py b/airflow/contrib/sensors/emr_step_sensor.py index 44aaea410dd1e..fc37a5ddb6ea2 100644 --- a/airflow/contrib/sensors/emr_step_sensor.py +++ b/airflow/contrib/sensors/emr_step_sensor.py @@ -53,5 +53,6 @@ def get_emr_response(self): self.log.info('Poking step %s on cluster %s', self.step_id, self.job_flow_id) return emr.describe_step(ClusterId=self.job_flow_id, StepId=self.step_id) - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['Step']['Status']['State'] diff --git a/airflow/contrib/task_runner/cgroup_task_runner.py b/airflow/contrib/task_runner/cgroup_task_runner.py index a97eac2af88ef..faa2407f09a97 100644 --- a/airflow/contrib/task_runner/cgroup_task_runner.py +++ b/airflow/contrib/task_runner/cgroup_task_runner.py @@ -193,7 +193,8 @@ def on_finish(self): if self._created_cpu_cgroup: self._delete_cgroup(self.cpu_cgroup_name) - def _get_cgroup_names(self): + @staticmethod + def _get_cgroup_names(): """ :return: a mapping between the subsystem name to the cgroup name :rtype: dict[str, str] diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py index afdf3ad2d198f..80527a2512545 100644 --- a/airflow/executors/dask_executor.py +++ b/airflow/executors/dask_executor.py @@ -43,7 +43,7 @@ def __init__(self, cluster_address=None): super(DaskExecutor, self).__init__(parallelism=0) def start(self): - if (self.tls_ca) or (self.tls_key) or (self.tls_cert): + if self.tls_ca or self.tls_key or self.tls_cert: from distributed.security import Security security = Security( tls_client_key=self.tls_key, diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py index ff3b98b6a91da..76188f109c244 100644 --- a/airflow/hooks/S3_hook.py +++ b/airflow/hooks/S3_hook.py @@ -42,7 +42,7 @@ def parse_s3_url(s3url): else: bucket_name = parsed_url.netloc key = parsed_url.path.strip('/') - return (bucket_name, key) + return bucket_name, key def check_for_bucket(self, bucket_name): """ @@ -223,8 +223,8 @@ def read_key(self, key, bucket_name=None): def select_key(self, key, bucket_name=None, expression='SELECT * FROM S3Object', expression_type='SQL', - input_serialization={'CSV': {}}, - output_serialization={'CSV': {}}): + input_serialization=None, + output_serialization=None): """ Reads a key with S3 Select. @@ -247,6 +247,10 @@ def select_key(self, key, bucket_name=None, For more details about S3 Select parameters: http://boto3.readthedocs.io/en/latest/reference/services/s3.html#S3.Client.select_object_content """ + if input_serialization is None: + input_serialization = {'CSV': {}} + if output_serialization is None: + output_serialization = {'CSV': {}} if not bucket_name: (bucket_name, key) = self.parse_s3_url(key) diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py index 48a09e67b20df..817295e89609d 100644 --- a/airflow/hooks/druid_hook.py +++ b/airflow/hooks/druid_hook.py @@ -70,7 +70,7 @@ def submit_indexing_job(self, json_index_spec): self.log.info("Druid ingestion spec: {}".format(json_index_spec)) req_index = requests.post(url, json=json_index_spec, headers=self.header) - if (req_index.status_code != 200): + if req_index.status_code != 200: raise AirflowException('Did not get 200 when ' 'submitting the Druid job to {}'.format(url)) diff --git a/airflow/hooks/hive_hooks.py b/airflow/hooks/hive_hooks.py index 93e1f45fe6402..db6df226d06b2 100644 --- a/airflow/hooks/hive_hooks.py +++ b/airflow/hooks/hive_hooks.py @@ -137,7 +137,8 @@ def _prepare_cli_cmd(self): return [hive_bin] + cmd_extra + hive_params_list - def _prepare_hiveconf(self, d): + @staticmethod + def _prepare_hiveconf(d): """ This function prepares a list of hiveconf params from a dictionary of key value pairs. diff --git a/airflow/hooks/presto_hook.py b/airflow/hooks/presto_hook.py index d6b5293fc0f97..c7ebd8f5dcb70 100644 --- a/airflow/hooks/presto_hook.py +++ b/airflow/hooks/presto_hook.py @@ -56,7 +56,8 @@ def get_conn(self): def _strip_sql(sql): return sql.strip().rstrip(';') - def _get_pretty_exception_message(self, e): + @staticmethod + def _get_pretty_exception_message(e): """ Parses some DatabaseError to provide a better error message """ diff --git a/airflow/jobs.py b/airflow/jobs.py index bf4d12c50db01..7a45056905523 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1183,7 +1183,7 @@ def _find_executable_task_instances(self, simple_dag_bag, states, session=None): 'task_concurrency') if task_concurrency is not None: num_running = task_concurrency_map[ - ((task_instance.dag_id, task_instance.task_id)) + (task_instance.dag_id, task_instance.task_id) ] if num_running >= task_concurrency: diff --git a/airflow/operators/check_operator.py b/airflow/operators/check_operator.py index 5a31737fd5a6c..a0d213cf6619c 100644 --- a/airflow/operators/check_operator.py +++ b/airflow/operators/check_operator.py @@ -147,7 +147,7 @@ def execute(self, context=None): is_numeric_value_check = isinstance(pass_value_conv, float) tolerance_pct_str = None - if (self.tol is not None): + if self.tol is not None: tolerance_pct_str = str(self.tol * 100) + '%' except_temp = ("Test failed.\nPass value:{pass_value_conv}\n" diff --git a/airflow/operators/hive_stats_operator.py b/airflow/operators/hive_stats_operator.py index fe83284193757..b0bb874956a92 100644 --- a/airflow/operators/hive_stats_operator.py +++ b/airflow/operators/hive_stats_operator.py @@ -91,8 +91,7 @@ def __init__( def get_default_exprs(self, col, col_type): if col in self.col_blacklist: return {} - d = {} - d[(col, 'non_null')] = "COUNT({col})" + d = {(col, 'non_null'): "COUNT({col})"} if col_type in ['double', 'int', 'bigint', 'float', 'double']: d[(col, 'sum')] = 'SUM({col})' d[(col, 'min')] = 'MIN({col})' diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py index 88f3b1a8a6347..a75f3a0207e41 100644 --- a/airflow/operators/python_operator.py +++ b/airflow/operators/python_operator.py @@ -351,7 +351,8 @@ def _generate_pip_install_cmd(self, tmp_dir): cmd = ['{}/bin/pip'.format(tmp_dir), 'install'] return cmd + self.requirements - def _generate_python_cmd(self, tmp_dir, script_filename, + @staticmethod + def _generate_python_cmd(tmp_dir, script_filename, input_filename, output_filename, string_args_filename): # direct path alleviates need to activate return ['{}/bin/python'.format(tmp_dir), script_filename, diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py index 5faaf916b7417..b82ebce6fa295 100644 --- a/airflow/operators/s3_to_hive_operator.py +++ b/airflow/operators/s3_to_hive_operator.py @@ -261,8 +261,8 @@ def _match_headers(self, header_list): else: return True + @staticmethod def _delete_top_row_and_compress( - self, input_file_name, output_file_ext, dest_dir): @@ -275,7 +275,7 @@ def _delete_top_row_and_compress( os_fh_output, fn_output = \ tempfile.mkstemp(suffix=output_file_ext, dir=dest_dir) - with open(input_file_name, 'rb') as f_in,\ + with open(input_file_name, 'rb') as f_in, \ open_fn(fn_output, 'wb') as f_out: f_in.seek(0) next(f_in) diff --git a/airflow/sensors/hdfs_sensor.py b/airflow/sensors/hdfs_sensor.py index c9bac08ecbfe7..d05adef71c276 100644 --- a/airflow/sensors/hdfs_sensor.py +++ b/airflow/sensors/hdfs_sensor.py @@ -39,13 +39,15 @@ class HdfsSensor(BaseSensorOperator): def __init__(self, filepath, hdfs_conn_id='hdfs_default', - ignored_ext=['_COPYING_'], + ignored_ext=None, ignore_copying=True, file_size=None, hook=HDFSHook, *args, **kwargs): super(HdfsSensor, self).__init__(*args, **kwargs) + if ignored_ext is None: + ignored_ext = ['_COPYING_'] self.filepath = filepath self.hdfs_conn_id = hdfs_conn_id self.file_size = file_size diff --git a/airflow/utils/cli.py b/airflow/utils/cli.py index 4a1e57a062b4b..32303cd90bd8d 100644 --- a/airflow/utils/cli.py +++ b/airflow/utils/cli.py @@ -94,10 +94,8 @@ def _build_metrics(func_name, namespace): :return: dict with metrics """ - metrics = {'sub_command': func_name} - metrics['start_datetime'] = datetime.utcnow() - metrics['full_command'] = '{}'.format(list(sys.argv)) - metrics['user'] = getpass.getuser() + metrics = {'sub_command': func_name, 'start_datetime': datetime.utcnow(), + 'full_command': '{}'.format(list(sys.argv)), 'user': getpass.getuser()} assert isinstance(namespace, Namespace) tmp_dic = vars(namespace) diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index db58e650d8ce9..45d0217e230ae 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -127,7 +127,7 @@ def chunks(items, chunk_size): """ Yield successive chunks of a given size from a list of items """ - if (chunk_size <= 0): + if chunk_size <= 0: raise ValueError('Chunk size must be a positive integer') for i in range(0, len(items), chunk_size): yield items[i:i + chunk_size] diff --git a/airflow/utils/log/gcs_task_handler.py b/airflow/utils/log/gcs_task_handler.py index 8c34792bb2138..e768882ac5df3 100644 --- a/airflow/utils/log/gcs_task_handler.py +++ b/airflow/utils/log/gcs_task_handler.py @@ -164,7 +164,8 @@ def gcs_write(self, log, remote_log_location, append=True): except Exception as e: self.log.error('Could not write logs to %s: %s', remote_log_location, e) - def parse_gcs_url(self, gsurl): + @staticmethod + def parse_gcs_url(gsurl): """ Given a Google Cloud Storage URL (gs:///), returns a tuple containing the corresponding bucket and blob. diff --git a/airflow/www/views.py b/airflow/www/views.py index 483e12f66f284..6bf60d63878fc 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1898,7 +1898,7 @@ def task_instances(self, session=None): if dttm: dttm = pendulum.parse(dttm) else: - return ("Error: Invalid execution_date") + return "Error: Invalid execution_date" task_instances = { ti.task_id: alchemy_to_dict(ti) diff --git a/airflow/www_rbac/forms.py b/airflow/www_rbac/forms.py index da9d12c7adfd9..61c34888e3568 100644 --- a/airflow/www_rbac/forms.py +++ b/airflow/www_rbac/forms.py @@ -93,7 +93,7 @@ class ConnectionForm(DynamicForm): widget=BS3TextFieldWidget()) conn_type = SelectField( lazy_gettext('Conn Type'), - choices=(models.Connection._types), + choices=models.Connection._types, widget=Select2Widget()) host = StringField( lazy_gettext('Host'), diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 2bfca2770621a..39d9fb58a3633 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -1577,7 +1577,7 @@ def task_instances(self, session=None): if dttm: dttm = pendulum.parse(dttm) else: - return ("Error: Invalid execution_date") + return "Error: Invalid execution_date" task_instances = { ti.task_id: alchemy_to_dict(ti) diff --git a/dev/airflow-pr b/dev/airflow-pr index 65243ddf876ae..c7cea6446b61d 100755 --- a/dev/airflow-pr +++ b/dev/airflow-pr @@ -757,7 +757,7 @@ def standardize_jira_ref(text, only_jira=False): # Cleanup any remaining symbols: pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) - if (pattern.search(text) is not None): + if pattern.search(text) is not None: text = pattern.search(text).groups()[0] def unique(seq): diff --git a/scripts/perf/scheduler_ops_metrics.py b/scripts/perf/scheduler_ops_metrics.py index d4e472d34feca..def829e7cdc3f 100644 --- a/scripts/perf/scheduler_ops_metrics.py +++ b/scripts/perf/scheduler_ops_metrics.py @@ -123,7 +123,7 @@ def heartbeat(self): if (len(successful_tis) == num_task_instances or (datetime.now()-self.start_date).total_seconds() > MAX_RUNTIME_SECS): - if (len(successful_tis) == num_task_instances): + if len(successful_tis) == num_task_instances: self.log.info("All tasks processed! Printing stats.") else: self.log.info("Test timeout reached. " diff --git a/tests/cli/test_cli.py b/tests/cli/test_cli.py index 944ee16fad167..81a0a16bda2e9 100644 --- a/tests/cli/test_cli.py +++ b/tests/cli/test_cli.py @@ -70,12 +70,14 @@ def create_mock_args( ignore_dependencies=False, force=False, run_as_user=None, - executor_config={}, + executor_config=None, cfg_path=None, pickle=None, raw=None, interactive=None, ): + if executor_config is None: + executor_config = {} args = MagicMock(spec=Namespace) args.task_id = task_id args.dag_id = dag_id diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index 0a389204b4d6c..a42a87dd17996 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -26,7 +26,8 @@ class TestAirflowKubernetesScheduler(unittest.TestCase): - def _gen_random_string(self, str_len): + @staticmethod + def _gen_random_string(str_len): return ''.join([random.choice(string.printable) for _ in range(str_len)]) def _cases(self): @@ -45,7 +46,8 @@ def _cases(self): return cases - def _is_valid_name(self, name): + @staticmethod + def _is_valid_name(name): regex = "^[a-z0-9]([-a-z0-9]*[a-z0-9])?(\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*$" return ( len(name) <= 253 and diff --git a/tests/contrib/hooks/test_aws_lambda_hook.py b/tests/contrib/hooks/test_aws_lambda_hook.py index a35a8fcd1e269..0b9744cd22285 100644 --- a/tests/contrib/hooks/test_aws_lambda_hook.py +++ b/tests/contrib/hooks/test_aws_lambda_hook.py @@ -42,7 +42,8 @@ def test_get_conn_returns_a_boto3_connection(self): function_name="test_function", region_name="us-east-1") self.assertIsNotNone(hook.get_conn()) - def lambda_function(self): + @staticmethod + def lambda_function(): code = textwrap.dedent(""" def lambda_handler(event, context): return event diff --git a/tests/contrib/hooks/test_gcp_mlengine_hook.py b/tests/contrib/hooks/test_gcp_mlengine_hook.py index bb3c5b62586db..c3bc7a9c0da0a 100644 --- a/tests/contrib/hooks/test_gcp_mlengine_hook.py +++ b/tests/contrib/hooks/test_gcp_mlengine_hook.py @@ -61,7 +61,8 @@ def __init__(self, test_cls, responses, expected_requests): for x in expected_requests] self._actual_requests = [] - def _normalize_requests_for_comparison(self, uri, http_method, body): + @staticmethod + def _normalize_requests_for_comparison(uri, http_method, body): parts = urlparse(uri) return ( parts._replace(query=set(parse_qsl(parts.query))), diff --git a/tests/contrib/hooks/test_mongo_hook.py b/tests/contrib/hooks/test_mongo_hook.py index 7696edea05a30..2d4c28f013426 100644 --- a/tests/contrib/hooks/test_mongo_hook.py +++ b/tests/contrib/hooks/test_mongo_hook.py @@ -23,10 +23,10 @@ class MongoHookTest(MongoHook): - ''' + """ Extending hook so that a mockmongo collection object can be passed in to get_collection() - ''' + """ def __init__(self, conn_id='mongo_default', *args, **kwargs): super(MongoHookTest, self).__init__(conn_id=conn_id, *args, **kwargs) diff --git a/tests/contrib/hooks/test_redshift_hook.py b/tests/contrib/hooks/test_redshift_hook.py index c69ed8a9dca2a..029dfd38016c5 100644 --- a/tests/contrib/hooks/test_redshift_hook.py +++ b/tests/contrib/hooks/test_redshift_hook.py @@ -35,7 +35,8 @@ class TestRedshiftHook(unittest.TestCase): def setUp(self): configuration.load_test_config() - def _create_clusters(self): + @staticmethod + def _create_clusters(): client = boto3.client('redshift', region_name='us-east-1') client.create_cluster( ClusterIdentifier='test_cluster', diff --git a/tests/contrib/minikube/test_kubernetes_executor.py b/tests/contrib/minikube/test_kubernetes_executor.py index fe38efb81bec6..2d1105111b5b2 100644 --- a/tests/contrib/minikube/test_kubernetes_executor.py +++ b/tests/contrib/minikube/test_kubernetes_executor.py @@ -42,7 +42,8 @@ def get_minikube_host(): class KubernetesExecutorTest(unittest.TestCase): - def _delete_airflow_pod(self): + @staticmethod + def _delete_airflow_pod(): air_pod = check_output(['kubectl', 'get', 'pods']).decode() air_pod = air_pod.split('\n') names = [re.compile('\s+').split(x)[0] for x in air_pod if 'airflow' in x] diff --git a/tests/contrib/minikube/test_kubernetes_pod_operator.py b/tests/contrib/minikube/test_kubernetes_pod_operator.py index 2cf615f83cf33..595d7aa8b43e5 100644 --- a/tests/contrib/minikube/test_kubernetes_pod_operator.py +++ b/tests/contrib/minikube/test_kubernetes_pod_operator.py @@ -39,7 +39,8 @@ class KubernetesPodOperatorTest(unittest.TestCase): - def test_config_path_move(self): + @staticmethod + def test_config_path_move(): new_config_path = '/tmp/kube_config' old_config_path = os.path.expanduser('~/.kube/config') shutil.copy(old_config_path, new_config_path) @@ -80,7 +81,8 @@ def test_config_path(self, client_mock, launcher_mock): cluster_context='default', config_file=file_path) - def test_working_pod(self): + @staticmethod + def test_working_pod(): k = KubernetesPodOperator( namespace='default', image="ubuntu:16.04", @@ -138,7 +140,8 @@ def test_pod_node_selectors(): ) k.execute(None) - def test_pod_affinity(self): + @staticmethod + def test_pod_affinity(): affinity = { 'nodeAffinity': { 'requiredDuringSchedulingIgnoredDuringExecution': { @@ -169,7 +172,8 @@ def test_pod_affinity(self): ) k.execute(None) - def test_logging(self): + @staticmethod + def test_logging(): with mock.patch.object(PodLauncher, 'log') as mock_logger: k = KubernetesPodOperator( namespace='default', @@ -184,7 +188,8 @@ def test_logging(self): k.execute(None) mock_logger.info.assert_any_call(b"+ echo 10\n") - def test_volume_mount(self): + @staticmethod + def test_volume_mount(): with mock.patch.object(PodLauncher, 'log') as mock_logger: volume_mount = VolumeMount('test-volume', mount_path='/root/mount_file', diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py index aea74a7abfc71..173752d722ecb 100644 --- a/tests/contrib/operators/test_dataproc_operator.py +++ b/tests/contrib/operators/test_dataproc_operator.py @@ -522,7 +522,8 @@ def test_cluster_name_log_sub(self): class DataProcHadoopOperatorTest(unittest.TestCase): # Unit test for the DataProcHadoopOperator - def test_hook_correct_region(self): + @staticmethod + def test_hook_correct_region(): with patch(HOOK) as mock_hook: dataproc_task = DataProcHadoopOperator( task_id=TASK_ID, @@ -533,7 +534,8 @@ def test_hook_correct_region(self): mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) - def test_dataproc_job_id_is_set(self): + @staticmethod + def test_dataproc_job_id_is_set(): with patch(HOOK) as mock_hook: dataproc_task = DataProcHadoopOperator( task_id=TASK_ID @@ -544,7 +546,8 @@ def test_dataproc_job_id_is_set(self): class DataProcHiveOperatorTest(unittest.TestCase): # Unit test for the DataProcHiveOperator - def test_hook_correct_region(self): + @staticmethod + def test_hook_correct_region(): with patch(HOOK) as mock_hook: dataproc_task = DataProcHiveOperator( task_id=TASK_ID, @@ -555,7 +558,8 @@ def test_hook_correct_region(self): mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) - def test_dataproc_job_id_is_set(self): + @staticmethod + def test_dataproc_job_id_is_set(): with patch(HOOK) as mock_hook: dataproc_task = DataProcHiveOperator( task_id=TASK_ID @@ -566,7 +570,8 @@ def test_dataproc_job_id_is_set(self): class DataProcPySparkOperatorTest(unittest.TestCase): # Unit test for the DataProcPySparkOperator - def test_hook_correct_region(self): + @staticmethod + def test_hook_correct_region(): with patch(HOOK) as mock_hook: dataproc_task = DataProcPySparkOperator( task_id=TASK_ID, @@ -578,7 +583,8 @@ def test_hook_correct_region(self): mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) - def test_dataproc_job_id_is_set(self): + @staticmethod + def test_dataproc_job_id_is_set(): with patch(HOOK) as mock_hook: dataproc_task = DataProcPySparkOperator( task_id=TASK_ID, @@ -590,7 +596,8 @@ def test_dataproc_job_id_is_set(self): class DataProcSparkOperatorTest(unittest.TestCase): # Unit test for the DataProcSparkOperator - def test_hook_correct_region(self): + @staticmethod + def test_hook_correct_region(): with patch(HOOK) as mock_hook: dataproc_task = DataProcSparkOperator( task_id=TASK_ID, @@ -601,7 +608,8 @@ def test_hook_correct_region(self): mock_hook.return_value.submit.assert_called_once_with(mock.ANY, mock.ANY, REGION) - def test_dataproc_job_id_is_set(self): + @staticmethod + def test_dataproc_job_id_is_set(): with patch(HOOK) as mock_hook: dataproc_task = DataProcSparkOperator( task_id=TASK_ID diff --git a/tests/contrib/operators/test_hive_to_dynamodb_operator.py b/tests/contrib/operators/test_hive_to_dynamodb_operator.py index d1f75b50c40eb..e5b2c3e65a257 100644 --- a/tests/contrib/operators/test_hive_to_dynamodb_operator.py +++ b/tests/contrib/operators/test_hive_to_dynamodb_operator.py @@ -52,7 +52,8 @@ def setUp(self): self.hook = AwsDynamoDBHook( aws_conn_id='aws_default', region_name='us-east-1') - def process_data(self, data, *args, **kwargs): + @staticmethod + def process_data(data, *args, **kwargs): return json.loads(data.to_json(orient='records')) @unittest.skipIf(mock_dynamodb2 is None, 'mock_dynamodb2 package not present') diff --git a/tests/contrib/operators/test_mysql_to_gcs_operator.py b/tests/contrib/operators/test_mysql_to_gcs_operator.py index 6e2e3f90dfb95..f0eb709500ca0 100644 --- a/tests/contrib/operators/test_mysql_to_gcs_operator.py +++ b/tests/contrib/operators/test_mysql_to_gcs_operator.py @@ -27,7 +27,8 @@ class MySqlToGoogleCloudStorageOperatorTest(unittest.TestCase): - def test_write_local_data_files(self): + @staticmethod + def test_write_local_data_files(): # Configure task_id = "some_test_id" @@ -35,17 +36,15 @@ def test_write_local_data_files(self): bucket = "some_bucket" filename = "some_filename" row_iter = [[1, b'byte_str_1'], [2, b'byte_str_2']] - schema = [] - schema.append({ + schema = [{ 'name': 'location', 'type': 'STRING', 'mode': 'nullable', - }) - schema.append({ + }, { 'name': 'uuid', 'type': 'BYTES', 'mode': 'nullable', - }) + }] schema_str = json.dumps(schema) op = MySqlToGoogleCloudStorageOperator( diff --git a/tests/contrib/operators/test_oracle_to_oracle_transfer.py b/tests/contrib/operators/test_oracle_to_oracle_transfer.py index 83d25e05a30f0..9c738dc8f76da 100644 --- a/tests/contrib/operators/test_oracle_to_oracle_transfer.py +++ b/tests/contrib/operators/test_oracle_to_oracle_transfer.py @@ -34,7 +34,8 @@ class OracleToOracleTransferTest(unittest.TestCase): - def test_execute(self): + @staticmethod + def test_execute(): oracle_destination_conn_id = 'oracle_destination_conn_id' destination_table = 'destination_table' oracle_source_conn_id = 'oracle_source_conn_id' diff --git a/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py b/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py index 921b2dac1fae9..95064c2655ac0 100644 --- a/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py +++ b/tests/contrib/sensors/test_aws_redshift_cluster_sensor.py @@ -35,7 +35,8 @@ class TestAwsRedshiftClusterSensor(unittest.TestCase): def setUp(self): configuration.load_test_config() - def _create_cluster(self): + @staticmethod + def _create_cluster(): client = boto3.client('redshift', region_name='us-east-1') client.create_cluster( ClusterIdentifier='test_cluster', diff --git a/tests/contrib/sensors/test_emr_base_sensor.py b/tests/contrib/sensors/test_emr_base_sensor.py index 8d00db713fb2d..2215edd09e0f6 100644 --- a/tests/contrib/sensors/test_emr_base_sensor.py +++ b/tests/contrib/sensors/test_emr_base_sensor.py @@ -33,13 +33,15 @@ class EmrBaseSensorSubclass(EmrBaseSensor): NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE'] FAILED_STATE = ['FAILED'] - def get_emr_response(self): + @staticmethod + def get_emr_response(): return { 'SomeKey': {'State': 'COMPLETED'}, 'ResponseMetadata': {'HTTPStatusCode': 200} } - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['SomeKey']['State'] operator = EmrBaseSensorSubclass( @@ -56,13 +58,15 @@ class EmrBaseSensorSubclass(EmrBaseSensor): NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE'] FAILED_STATE = ['FAILED'] - def get_emr_response(self): + @staticmethod + def get_emr_response(): return { 'SomeKey': {'State': 'PENDING'}, 'ResponseMetadata': {'HTTPStatusCode': 200} } - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['SomeKey']['State'] operator = EmrBaseSensorSubclass( @@ -79,13 +83,15 @@ class EmrBaseSensorSubclass(EmrBaseSensor): NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE'] FAILED_STATE = ['FAILED'] - def get_emr_response(self): + @staticmethod + def get_emr_response(): return { 'SomeKey': {'State': 'COMPLETED'}, 'ResponseMetadata': {'HTTPStatusCode': 400} } - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['SomeKey']['State'] operator = EmrBaseSensorSubclass( @@ -102,13 +108,15 @@ class EmrBaseSensorSubclass(EmrBaseSensor): NON_TERMINAL_STATES = ['PENDING', 'RUNNING', 'CONTINUE'] FAILED_STATE = ['FAILED'] - def get_emr_response(self): + @staticmethod + def get_emr_response(): return { 'SomeKey': {'State': 'FAILED'}, 'ResponseMetadata': {'HTTPStatusCode': 200} } - def state_from_response(self, response): + @staticmethod + def state_from_response(response): return response['SomeKey']['State'] operator = EmrBaseSensorSubclass( diff --git a/tests/core.py b/tests/core.py index 60bfee69df7dd..156b5a9ec6244 100644 --- a/tests/core.py +++ b/tests/core.py @@ -2467,7 +2467,7 @@ def test_init_proxy_user(self): class HDFSHookTest(unittest.TestCase): def setUp(self): configuration.load_test_config() - os.environ['AIRFLOW_CONN_HDFS_DEFAULT'] = ('hdfs://localhost:8020') + os.environ['AIRFLOW_CONN_HDFS_DEFAULT'] = 'hdfs://localhost:8020' def test_get_client(self): client = HDFSHook(proxy_user='foo').get_conn() diff --git a/tests/models.py b/tests/models.py index 9e09f76a43900..c6b6b5e075f2a 100644 --- a/tests/models.py +++ b/tests/models.py @@ -284,7 +284,7 @@ def test_dag_task_priority_weight_total(self): match = pattern.match(task.task_id) task_depth = int(match.group(1)) # the sum of each stages after this task + itself - correct_weight = ((task_depth) * width + 1) * weight + correct_weight = (task_depth * width + 1) * weight calculated_weight = task.priority_weight_total self.assertEquals(calculated_weight, correct_weight) @@ -1398,7 +1398,7 @@ def process_dag(self, create_dag): dagbag = models.DagBag(include_examples=False) found_dags = dagbag.process_file(f.name) - return (dagbag, found_dags, f.name) + return dagbag, found_dags, f.name def validate_dags(self, expected_parent_dag, actual_found_dags, actual_dagbag, should_be_found=True): @@ -2814,7 +2814,7 @@ def test_xcom_enable_pickle_type(self): def test_xcom_disable_pickle_type_fail_on_non_json(self): class PickleRce(object): def __reduce__(self): - return (os.system, ("ls -alt",)) + return os.system, ("ls -alt",) configuration.set("core", "xcom_enable_pickling", "False") diff --git a/tests/operators/test_docker_operator.py b/tests/operators/test_docker_operator.py index b9154456b9b31..ae076eb2e3a53 100644 --- a/tests/operators/test_docker_operator.py +++ b/tests/operators/test_docker_operator.py @@ -153,7 +153,8 @@ def test_execute_container_fails(self, client_class_mock): with self.assertRaises(AirflowException): operator.execute(None) - def test_on_kill(self): + @staticmethod + def test_on_kill(): client_mock = mock.Mock(spec=APIClient) operator = DockerOperator(image='ubuntu', owner='unittest', task_id='unittest') diff --git a/tests/operators/test_s3_to_hive_operator.py b/tests/operators/test_s3_to_hive_operator.py index 2f8628c621abe..426566c40c7e5 100644 --- a/tests/operators/test_s3_to_hive_operator.py +++ b/tests/operators/test_s3_to_hive_operator.py @@ -145,15 +145,17 @@ def _get_fn(self, ext, header): key = self._get_key(ext, header) return self.fn[key] - def _get_key(self, ext, header): + @staticmethod + def _get_key(ext, header): key = ext + "_" + ('h' if header else 'nh') return key - def _check_file_equality(self, fn_1, fn_2, ext): + @staticmethod + def _check_file_equality(fn_1, fn_2, ext): # gz files contain mtime and filename in the header that # causes filecmp to return False even if contents are identical # Hence decompress to test for equality - if(ext.lower() == '.gz'): + if ext.lower() == '.gz': with gzip.GzipFile(fn_1, 'rb') as f_1,\ NamedTemporaryFile(mode='wb') as f_txt_1,\ gzip.GzipFile(fn_2, 'rb') as f_2,\ diff --git a/tests/operators/test_virtualenv_operator.py b/tests/operators/test_virtualenv_operator.py index 82abe9c8098de..8196d636f4c1f 100644 --- a/tests/operators/test_virtualenv_operator.py +++ b/tests/operators/test_virtualenv_operator.py @@ -140,7 +140,8 @@ def f(): raise Exception self._run_as_operator(f, python_version=3, use_dill=False, requirements=['dill']) - def _invert_python_major_version(self): + @staticmethod + def _invert_python_major_version(): if sys.version_info[0] == 2: return 3 else: diff --git a/tests/www/api/experimental/test_kerberos_endpoints.py b/tests/www/api/experimental/test_kerberos_endpoints.py index 9179cdecc027a..1cf30635fa407 100644 --- a/tests/www/api/experimental/test_kerberos_endpoints.py +++ b/tests/www/api/experimental/test_kerberos_endpoints.py @@ -65,7 +65,7 @@ def test_trigger_dag(self): response.url = 'http://{}'.format(get_hostname()) - class Request(): + class Request: headers = {} response.request = Request() diff --git a/tests/www_rbac/api/experimental/test_kerberos_endpoints.py b/tests/www_rbac/api/experimental/test_kerberos_endpoints.py index 69a8117868f3e..54bbd865b3724 100644 --- a/tests/www_rbac/api/experimental/test_kerberos_endpoints.py +++ b/tests/www_rbac/api/experimental/test_kerberos_endpoints.py @@ -64,7 +64,7 @@ def test_trigger_dag(self): response.url = 'http://{}'.format(socket.getfqdn()) - class Request(): + class Request(object): headers = {} response.request = Request() From 5f0b400f17a04b58784946986a899ee4ec307d62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Victor=20No=C3=ABl?= Date: Fri, 16 Nov 2018 14:20:42 +0100 Subject: [PATCH 225/808] [AIRFLOW-3251] KubernetesPodOperator now uses 'image_pull_secrets' argument when creating Pods (#4188) --- airflow/contrib/kubernetes/pod.py | 4 ++++ .../operators/kubernetes_pod_operator.py | 7 ++++++ .../minikube/test_kubernetes_pod_operator.py | 23 +++++++++++++++++++ 3 files changed, 34 insertions(+) diff --git a/airflow/contrib/kubernetes/pod.py b/airflow/contrib/kubernetes/pod.py index 5de23ff5bdf81..bad5caa738e1b 100644 --- a/airflow/contrib/kubernetes/pod.py +++ b/airflow/contrib/kubernetes/pod.py @@ -54,6 +54,10 @@ class Pod: :type result: any :param image_pull_policy: Specify a policy to cache or always pull an image :type image_pull_policy: str + :param image_pull_secrets: Any image pull secrets to be given to the pod. + If more than one secret is required, provide a + comma separated list: secret_a,secret_b + :type image_pull_secrets: str :param affinity: A dict containing a group of affinity scheduling rules :type affinity: dict """ diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py index 91e73e0c53df3..a29b61998d181 100644 --- a/airflow/contrib/operators/kubernetes_pod_operator.py +++ b/airflow/contrib/operators/kubernetes_pod_operator.py @@ -45,6 +45,12 @@ class KubernetesPodOperator(BaseOperator): :param arguments: arguments of to the entrypoint. (templated) The docker image's CMD is used if this is not provided. :type arguments: list of str + :param image_pull_policy: Specify a policy to cache or always pull an image + :type image_pull_policy: str + :param image_pull_secrets: Any image pull secrets to be given to the pod. + If more than one secret is required, provide a + comma separated list: secret_a,secret_b + :type image_pull_secrets: str :param volume_mounts: volumeMounts for launched pod :type volume_mounts: list of VolumeMount :param volumes: volumes for launched pod. Includes ConfigMaps and PersistentVolumes @@ -108,6 +114,7 @@ def execute(self, context): pod.secrets = self.secrets pod.envs = self.env_vars pod.image_pull_policy = self.image_pull_policy + pod.image_pull_secrets = self.image_pull_secrets pod.annotations = self.annotations pod.resources = self.resources pod.affinity = self.affinity diff --git a/tests/contrib/minikube/test_kubernetes_pod_operator.py b/tests/contrib/minikube/test_kubernetes_pod_operator.py index 595d7aa8b43e5..6ab69f31c6927 100644 --- a/tests/contrib/minikube/test_kubernetes_pod_operator.py +++ b/tests/contrib/minikube/test_kubernetes_pod_operator.py @@ -81,6 +81,29 @@ def test_config_path(self, client_mock, launcher_mock): cluster_context='default', config_file=file_path) + @mock.patch("airflow.contrib.kubernetes.pod_launcher.PodLauncher.run_pod") + @mock.patch("airflow.contrib.kubernetes.kube_client.get_kube_client") + def test_image_pull_secrets_correctly_set(self, client_mock, launcher_mock): + from airflow.utils.state import State + + fake_pull_secrets = "fakeSecret" + k = KubernetesPodOperator( + namespace='default', + image="ubuntu:16.04", + cmds=["bash", "-cx"], + arguments=["echo 10"], + labels={"foo": "bar"}, + name="test", + task_id="task", + image_pull_secrets=fake_pull_secrets, + in_cluster=False, + cluster_context='default' + ) + launcher_mock.return_value = (State.SUCCESS, None) + k.execute(None) + self.assertEqual(launcher_mock.call_args[0][0].image_pull_secrets, + fake_pull_secrets) + @staticmethod def test_working_pod(): k = KubernetesPodOperator( From 0cc7178d3a1e7d47bdfe922b2cf0be6b8f415678 Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Mon, 12 Nov 2018 12:02:39 -0500 Subject: [PATCH 226/808] [AIRFLOW-3306] Disable flask-sqlalchemy modification tracking. (#4146) By default, flask-sqlalchemy tracks model changes for its event system, which adds some overhead. Since I don't think we're using the flask-sqlalchemy event system, we should be able to turn off modification tracking and improve performance. --- airflow/www_rbac/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index 82f1b37c7c423..55ce6856e656a 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -50,6 +50,7 @@ def create_app(config=None, testing=False, app_name="Airflow"): airflow_home_path = conf.get('core', 'AIRFLOW_HOME') webserver_config_path = airflow_home_path + '/webserver_config.py' app.config.from_pyfile(webserver_config_path, silent=True) + app.config['SQLALCHEMY_TRACK_MODIFICATIONS'] = False app.config['APP_NAME'] = app_name app.config['TESTING'] = testing From 8ddaf0874c3ef5abf7f5c8f610fdd633876865fd Mon Sep 17 00:00:00 2001 From: Iuliia Volkova Date: Thu, 15 Nov 2018 20:36:11 +0300 Subject: [PATCH 227/808] [AIRFLOW-3353] Pin redis verison (#4195) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a76d768c2284d..05c85c4f674fd 100644 --- a/setup.py +++ b/setup.py @@ -200,7 +200,7 @@ def write_version(filename=os.path.join(*['airflow', postgres = ['psycopg2-binary>=2.7.4'] qds = ['qds-sdk>=1.9.6'] rabbitmq = ['librabbitmq>=1.6.1'] -redis = ['redis>=2.10.5'] +redis = ['redis>=2.10.5,<3.0.0'] s3 = ['boto3>=1.7.0, <1.8.0'] salesforce = ['simple-salesforce>=0.72'] samba = ['pysmbclient>=0.1.3'] From ed4fa75014005bf05b987f97567c332a0ecaa2c0 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 6 Nov 2018 18:54:18 +0800 Subject: [PATCH 228/808] [AIRFLOW-3301] Update DockerOperator CI test for PR #3977 (#4138) --- tests/operators/test_docker_operator.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/operators/test_docker_operator.py b/tests/operators/test_docker_operator.py index ae076eb2e3a53..23577d7a2830f 100644 --- a/tests/operators/test_docker_operator.py +++ b/tests/operators/test_docker_operator.py @@ -80,6 +80,7 @@ def test_execute(self, client_class_mock, mkdtemp_mock): shm_size=1000, cpu_shares=1024, mem_limit=None, + auto_remove=False, dns=None, dns_search=None) client_mock.images.assert_called_with(name='ubuntu:latest') From d584ede109688c45a9e6f48f09b8e6e946966d20 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 30 Nov 2018 22:52:29 +0000 Subject: [PATCH 229/808] Fix flake8 issues --- airflow/jobs.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/airflow/jobs.py b/airflow/jobs.py index 7a45056905523..4daafc22ada18 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -428,7 +428,7 @@ def start(self): def terminate(self, sigkill=False): """ Terminate (and then kill) the process launched to process the file. - + :param sigkill: whether to issue a SIGKILL if SIGTERM doesn't work. :type sigkill: bool """ @@ -457,7 +457,7 @@ def pid(self): def exit_code(self): """ After the process is finished, this can be called to get the return code - + :return: the exit code of the process :rtype: int """ @@ -469,7 +469,7 @@ def exit_code(self): def done(self): """ Check if the process launched to process this file is done. - + :return: whether the process is finished running :rtype: bool """ @@ -1899,7 +1899,7 @@ def _update_counters(self, ti_status): """ Updates the counters per state of the tasks that were running. Can re-add to tasks to run in case required. - + :param ti_status: the internal status of the backfill job tasks :type ti_status: BackfillJob._DagRunTaskStatus """ @@ -1944,7 +1944,7 @@ def _manage_executor_state(self, running): """ Checks if the executor agrees with the state of task instances that are running - + :param running: dict of key, task to verify """ executor = self.executor @@ -1976,7 +1976,7 @@ def _get_dag_run(self, run_date, session=None): Returns a dag run for the given run date, which will be matched to an existing dag run if available or create a new dag run otherwise. If the max_active_runs limit is reached, this function will return None. - + :param run_date: the execution date for the dag run :type run_date: datetime :param session: the database session object @@ -2036,7 +2036,7 @@ def _task_instances_for_dag_run(self, dag_run, session=None): """ Returns a map of task instance key to task instance object for the tasks to run in the given dag run. - + :param dag_run: the dag run to get the tasks from :type dag_run: models.DagRun :param session: the database session object @@ -2102,7 +2102,7 @@ def _process_backfill_task_instances(self, Process a set of task instances from a set of dag runs. Special handling is done to account for different task instance states that could be present when running them in a backfill process. - + :param ti_status: the internal status of the job :type ti_status: BackfillJob._DagRunTaskStatus :param executor: the executor to run the task instances @@ -2340,7 +2340,7 @@ def _execute_for_run_dates(self, run_dates, ti_status, executor, pickle_id, Computes the dag runs and their respective task instances for the given run dates and executes the task instances. Returns a list of execution dates of the dag runs that were executed. - + :param run_dates: Execution dates for dag runs :type run_dates: list :param ti_status: internal BackfillJob status structure to tis track progress From 8e26b75e1660d7d75b1c9279c6433cfa6e4b536a Mon Sep 17 00:00:00 2001 From: Verdan Mahmood Date: Sun, 22 Jul 2018 09:23:57 +0200 Subject: [PATCH 230/808] [AIRFLOW-2691] Manage JS dependencies via npm --- .gitignore | 8 + CONTRIBUTING.md | 57 + MANIFEST.in | 6 +- airflow/www_rbac/app.py | 3 +- airflow/www_rbac/compile_assets.sh | 23 + airflow/www_rbac/package-lock.json | 12408 ++++++++++++++ airflow/www_rbac/package.json | 55 + .../www_rbac/static/bootstrap-toggle.min.css | 28 - .../www_rbac/static/bootstrap-toggle.min.js | 9 - .../static/bootstrap3-typeahead.min.js | 26 - airflow/www_rbac/static/connection_form.js | 78 - .../static/{ => css}/bootstrap-theme.css | 0 airflow/www_rbac/static/{ => css}/gantt.css | 0 airflow/www_rbac/static/{ => css}/graph.css | 20 + airflow/www_rbac/static/{ => css}/main.css | 6 +- airflow/www_rbac/static/{ => css}/tree.css | 0 airflow/www_rbac/static/d3.tip.v0.6.3.js | 302 - airflow/www_rbac/static/d3.v3.min.js | 5 - airflow/www_rbac/static/dagre-d3.min.js | 2 - airflow/www_rbac/static/dagre.css | 38 - .../www_rbac/static/dataTables.bootstrap.css | 333 - airflow/www_rbac/static/jquery.dataTables.css | 495 - .../www_rbac/static/jquery.dataTables.min.js | 189 - airflow/www_rbac/static/js/clock.js | 41 + airflow/www_rbac/static/js/connection_form.js | 80 + airflow/www_rbac/static/{ => js}/dagre-d3.js | 0 .../static/{ => js}/gantt-chart-d3v2.js | 0 .../www_rbac/static/{ => js}/jqClock.min.js | 0 airflow/www_rbac/static/nv.d3.css | 788 - airflow/www_rbac/static/nv.d3.js | 14260 ---------------- airflow/www_rbac/static_config.py | 63 + airflow/www_rbac/templates/airflow/chart.html | 11 +- .../www_rbac/templates/airflow/circles.html | 8 +- .../templates/airflow/conn_create.html | 8 +- .../www_rbac/templates/airflow/conn_edit.html | 8 +- airflow/www_rbac/templates/airflow/dag.html | 23 +- airflow/www_rbac/templates/airflow/dags.html | 17 +- .../templates/airflow/duration_chart.html | 11 +- airflow/www_rbac/templates/airflow/gantt.html | 10 +- airflow/www_rbac/templates/airflow/graph.html | 10 +- airflow/www_rbac/templates/airflow/tree.html | 7 +- .../templates/appbuilder/baselayout.html | 35 +- airflow/www_rbac/webpack.config.js | 128 + dev/README.md | 15 +- setup.py | 19 + 45 files changed, 12990 insertions(+), 16643 deletions(-) create mode 100755 airflow/www_rbac/compile_assets.sh create mode 100644 airflow/www_rbac/package-lock.json create mode 100644 airflow/www_rbac/package.json delete mode 100644 airflow/www_rbac/static/bootstrap-toggle.min.css delete mode 100644 airflow/www_rbac/static/bootstrap-toggle.min.js delete mode 100644 airflow/www_rbac/static/bootstrap3-typeahead.min.js delete mode 100644 airflow/www_rbac/static/connection_form.js rename airflow/www_rbac/static/{ => css}/bootstrap-theme.css (100%) rename airflow/www_rbac/static/{ => css}/gantt.css (100%) rename airflow/www_rbac/static/{ => css}/graph.css (87%) rename airflow/www_rbac/static/{ => css}/main.css (97%) rename airflow/www_rbac/static/{ => css}/tree.css (100%) delete mode 100644 airflow/www_rbac/static/d3.tip.v0.6.3.js delete mode 100644 airflow/www_rbac/static/d3.v3.min.js delete mode 100644 airflow/www_rbac/static/dagre-d3.min.js delete mode 100644 airflow/www_rbac/static/dagre.css delete mode 100644 airflow/www_rbac/static/dataTables.bootstrap.css delete mode 100644 airflow/www_rbac/static/jquery.dataTables.css delete mode 100644 airflow/www_rbac/static/jquery.dataTables.min.js create mode 100644 airflow/www_rbac/static/js/clock.js create mode 100644 airflow/www_rbac/static/js/connection_form.js rename airflow/www_rbac/static/{ => js}/dagre-d3.js (100%) rename airflow/www_rbac/static/{ => js}/gantt-chart-d3v2.js (100%) rename airflow/www_rbac/static/{ => js}/jqClock.min.js (100%) delete mode 100644 airflow/www_rbac/static/nv.d3.css delete mode 100644 airflow/www_rbac/static/nv.d3.js create mode 100644 airflow/www_rbac/static_config.py create mode 100644 airflow/www_rbac/webpack.config.js diff --git a/.gitignore b/.gitignore index 5712aee6c549a..114236d467885 100644 --- a/.gitignore +++ b/.gitignore @@ -139,6 +139,14 @@ rat-results.txt *.tar.gz scripts/ci/kubernetes/kube/.generated/airflow.yaml +# Node & Webpack Stuff +*.entry.js +node_modules +npm-debug.log* +static/dist +derby.log +metastore_db + # Airflow log files when airflow is run locally airflow-*.err airflow-*.out diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 194aaa7b277ca..481ded4e4ee58 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -258,3 +258,60 @@ $ alembic revision -m "add new field to db" Generating ~/airflow/airflow/migrations/versions/12341123_add_new_field_to_db.py ``` + +## Setting up the node / npm javascript environment (ONLY FOR www_rbac) + +`airflow/www_rbac/` contains all npm-managed, front end assets. +Flask-Appbuilder itself comes bundled with jQuery and bootstrap. +While these may be phased out over time, these packages are currently not +managed with npm. + +### Node/npm versions +Make sure you are using recent versions of node and npm. No problems have been found with node>=8.11.3 and npm>=6.1.3 + +### Using npm to generate bundled files + +#### npm +First, npm must be available in your environment. If it is not you can run the following commands +(taken from [this source](https://gist.github.com/DanHerbert/9520689)) +``` +brew install node --without-npm +echo prefix=~/.npm-packages >> ~/.npmrc +curl -L https://www.npmjs.com/install.sh | sh +``` + +The final step is to add `~/.npm-packages/bin` to your `PATH` so commands you install globally are usable. +Add something like this to your `.bashrc` file, then `source ~/.bashrc` to reflect the change. +``` +export PATH="$HOME/.npm-packages/bin:$PATH" +``` + +#### npm packages +To install third party libraries defined in `package.json`, run the +following within the `airflow/www_rbac/` directory which will install them in a +new `node_modules/` folder within `www_rbac/`. + +```bash +# from the root of the repository, move to where our JS package.json lives +cd airflow/www_rbac/ +# run npm install to fetch all the dependencies +npm install +``` + +To parse and generate bundled files for airflow, run either of the +following commands. The `dev` flag will keep the npm script running and +re-run it upon any changes within the assets directory. + +``` +# Compiles the production / optimized js & css +npm run prod + +# Start a web server that manages and updates your assets as you modify them +npm run dev +``` + +#### Upgrading npm packages + +Should you add or upgrade a npm package, which involves changing `package.json`, you'll need to re-run `npm install` +and push the newly generated `package-lock.json` file so we get the reproducible build. + diff --git a/MANIFEST.in b/MANIFEST.in index b069194984511..ec99c1f6b2b1e 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -24,6 +24,7 @@ include README.md graft licenses/ graft airflow/www/templates graft airflow/www/static +graft airflow/www_rbac graft airflow/www_rbac/static graft airflow/www_rbac/templates graft airflow/www_rbac/translations @@ -31,3 +32,4 @@ include airflow/alembic.ini graft scripts/systemd graft scripts/upstart graft airflow/config_templates +recursive-exclude airflow/www_rbac/node_modules * diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index 55ce6856e656a..7a2b18418cd1c 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -32,7 +32,7 @@ from airflow import settings from airflow import configuration as conf from airflow.logging_config import configure_logging - +from airflow.www_rbac.static_config import configure_manifest_files app = None appbuilder = None @@ -68,6 +68,7 @@ def create_app(config=None, testing=False, app_name="Airflow"): app.register_blueprint(routes) configure_logging() + configure_manifest_files(app) with app.app_context(): appbuilder = AppBuilder( diff --git a/airflow/www_rbac/compile_assets.sh b/airflow/www_rbac/compile_assets.sh new file mode 100755 index 0000000000000..2c115a44a528f --- /dev/null +++ b/airflow/www_rbac/compile_assets.sh @@ -0,0 +1,23 @@ +#!/bin/sh +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# first bump up package.json manually, commit and tag +rm airflow/www_rbac/static/dist/* +cd airflow/www_rbac/ +npm install +npm run build +cd ../.. diff --git a/airflow/www_rbac/package-lock.json b/airflow/www_rbac/package-lock.json new file mode 100644 index 0000000000000..90d94cd0054f6 --- /dev/null +++ b/airflow/www_rbac/package-lock.json @@ -0,0 +1,12408 @@ +{ + "requires": true, + "lockfileVersion": 1, + "dependencies": { + "@webassemblyjs/ast": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.5.13.tgz", + "integrity": "sha512-49nwvW/Hx9i+OYHg+mRhKZfAlqThr11Dqz8TsrvqGKMhdI2ijy3KBJOun2Z4770TPjrIJhR6KxChQIDaz8clDA==", + "dev": true, + "requires": { + "@webassemblyjs/helper-module-context": "1.5.13", + "@webassemblyjs/helper-wasm-bytecode": "1.5.13", + "@webassemblyjs/wast-parser": "1.5.13", + "debug": "^3.1.0", + "mamacro": "^0.0.3" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/floating-point-hex-parser": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.5.13.tgz", + "integrity": "sha512-vrvvB18Kh4uyghSKb0NTv+2WZx871WL2NzwMj61jcq2bXkyhRC+8Q0oD7JGVf0+5i/fKQYQSBCNMMsDMRVAMqA==", + "dev": true + }, + "@webassemblyjs/helper-api-error": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-api-error/-/helper-api-error-1.5.13.tgz", + "integrity": "sha512-dBh2CWYqjaDlvMmRP/kudxpdh30uXjIbpkLj9HQe+qtYlwvYjPRjdQXrq1cTAAOUSMTtzqbXIxEdEZmyKfcwsg==", + "dev": true + }, + "@webassemblyjs/helper-buffer": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.5.13.tgz", + "integrity": "sha512-v7igWf1mHcpJNbn4m7e77XOAWXCDT76Xe7Is1VQFXc4K5jRcFrl9D0NrqM4XifQ0bXiuTSkTKMYqDxu5MhNljA==", + "dev": true, + "requires": { + "debug": "^3.1.0" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/helper-code-frame": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-code-frame/-/helper-code-frame-1.5.13.tgz", + "integrity": "sha512-yN6ScQQDFCiAXnVctdVO/J5NQRbwyTbQzsGzEgXsAnrxhjp0xihh+nNHQTMrq5UhOqTb5LykpJAvEv9AT0jnAQ==", + "dev": true, + "requires": { + "@webassemblyjs/wast-printer": "1.5.13" + } + }, + "@webassemblyjs/helper-fsm": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-fsm/-/helper-fsm-1.5.13.tgz", + "integrity": "sha512-hSIKzbXjVMRvy3Jzhgu+vDd/aswJ+UMEnLRCkZDdknZO3Z9e6rp1DAs0tdLItjCFqkz9+0BeOPK/mk3eYvVzZg==", + "dev": true + }, + "@webassemblyjs/helper-module-context": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-module-context/-/helper-module-context-1.5.13.tgz", + "integrity": "sha512-zxJXULGPLB7r+k+wIlvGlXpT4CYppRz8fLUM/xobGHc9Z3T6qlmJD9ySJ2jknuktuuiR9AjnNpKYDECyaiX+QQ==", + "dev": true, + "requires": { + "debug": "^3.1.0", + "mamacro": "^0.0.3" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/helper-wasm-bytecode": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.5.13.tgz", + "integrity": "sha512-0n3SoNGLvbJIZPhtMFq0XmmnA/YmQBXaZKQZcW8maGKwLpVcgjNrxpFZHEOLKjXJYVN5Il8vSfG7nRX50Zn+aw==", + "dev": true + }, + "@webassemblyjs/helper-wasm-section": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.5.13.tgz", + "integrity": "sha512-IJ/goicOZ5TT1axZFSnlAtz4m8KEjYr12BNOANAwGFPKXM4byEDaMNXYowHMG0yKV9a397eU/NlibFaLwr1fbw==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-buffer": "1.5.13", + "@webassemblyjs/helper-wasm-bytecode": "1.5.13", + "@webassemblyjs/wasm-gen": "1.5.13", + "debug": "^3.1.0" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/ieee754": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ieee754/-/ieee754-1.5.13.tgz", + "integrity": "sha512-TseswvXEPpG5TCBKoLx9tT7+/GMACjC1ruo09j46ULRZWYm8XHpDWaosOjTnI7kr4SRJFzA6MWoUkAB+YCGKKg==", + "dev": true, + "requires": { + "ieee754": "^1.1.11" + } + }, + "@webassemblyjs/leb128": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/leb128/-/leb128-1.5.13.tgz", + "integrity": "sha512-0NRMxrL+GG3eISGZBmLBLAVjphbN8Si15s7jzThaw1UE9e5BY1oH49/+MA1xBzxpf1OW5sf9OrPDOclk9wj2yg==", + "dev": true, + "requires": { + "long": "4.0.0" + }, + "dependencies": { + "long": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/long/-/long-4.0.0.tgz", + "integrity": "sha512-XsP+KhQif4bjX1kbuSiySJFNAehNxgLb6hPRGJ9QsUr8ajHkuXGdrHmFUTUUXhDwVX2R5bY4JNZEwbUiMhV+MA==", + "dev": true + } + } + }, + "@webassemblyjs/utf8": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/utf8/-/utf8-1.5.13.tgz", + "integrity": "sha512-Ve1ilU2N48Ew0lVGB8FqY7V7hXjaC4+PeZM+vDYxEd+R2iQ0q+Wb3Rw8v0Ri0+rxhoz6gVGsnQNb4FjRiEH/Ng==", + "dev": true + }, + "@webassemblyjs/wasm-edit": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.5.13.tgz", + "integrity": "sha512-X7ZNW4+Hga4f2NmqENnHke2V/mGYK/xnybJSIXImt1ulxbCOEs/A+ZK/Km2jgihjyVxp/0z0hwIcxC6PrkWtgw==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-buffer": "1.5.13", + "@webassemblyjs/helper-wasm-bytecode": "1.5.13", + "@webassemblyjs/helper-wasm-section": "1.5.13", + "@webassemblyjs/wasm-gen": "1.5.13", + "@webassemblyjs/wasm-opt": "1.5.13", + "@webassemblyjs/wasm-parser": "1.5.13", + "@webassemblyjs/wast-printer": "1.5.13", + "debug": "^3.1.0" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/wasm-gen": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.5.13.tgz", + "integrity": "sha512-yfv94Se8R73zmr8GAYzezFHc3lDwE/lBXQddSiIZEKZFuqy7yWtm3KMwA1uGbv5G1WphimJxboXHR80IgX1hQA==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-wasm-bytecode": "1.5.13", + "@webassemblyjs/ieee754": "1.5.13", + "@webassemblyjs/leb128": "1.5.13", + "@webassemblyjs/utf8": "1.5.13" + } + }, + "@webassemblyjs/wasm-opt": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.5.13.tgz", + "integrity": "sha512-IkXSkgzVhQ0QYAdIayuCWMmXSYx0dHGU8Ah/AxJf1gBvstMWVnzJnBwLsXLyD87VSBIcsqkmZ28dVb0mOC3oBg==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-buffer": "1.5.13", + "@webassemblyjs/wasm-gen": "1.5.13", + "@webassemblyjs/wasm-parser": "1.5.13", + "debug": "^3.1.0" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + } + } + }, + "@webassemblyjs/wasm-parser": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.5.13.tgz", + "integrity": "sha512-XnYoIcu2iqq8/LrtmdnN3T+bRjqYFjRHqWbqK3osD/0r/Fcv4d9ecRzjVtC29ENEuNTK4mQ9yyxCBCbK8S/cpg==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-api-error": "1.5.13", + "@webassemblyjs/helper-wasm-bytecode": "1.5.13", + "@webassemblyjs/ieee754": "1.5.13", + "@webassemblyjs/leb128": "1.5.13", + "@webassemblyjs/utf8": "1.5.13" + } + }, + "@webassemblyjs/wast-parser": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-parser/-/wast-parser-1.5.13.tgz", + "integrity": "sha512-Lbz65T0LQ1LgzKiUytl34CwuhMNhaCLgrh0JW4rJBN6INnBB8NMwUfQM+FxTnLY9qJ+lHJL/gCM5xYhB9oWi4A==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/floating-point-hex-parser": "1.5.13", + "@webassemblyjs/helper-api-error": "1.5.13", + "@webassemblyjs/helper-code-frame": "1.5.13", + "@webassemblyjs/helper-fsm": "1.5.13", + "long": "^3.2.0", + "mamacro": "^0.0.3" + } + }, + "@webassemblyjs/wast-printer": { + "version": "1.5.13", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.5.13.tgz", + "integrity": "sha512-QcwogrdqcBh8Z+eUF8SG+ag5iwQSXxQJELBEHmLkk790wgQgnIMmntT2sMAMw53GiFNckArf5X0bsCA44j3lWQ==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/wast-parser": "1.5.13", + "long": "^3.2.0" + } + }, + "@webpack-contrib/schema-utils": { + "version": "1.0.0-beta.0", + "resolved": "https://registry.npmjs.org/@webpack-contrib/schema-utils/-/schema-utils-1.0.0-beta.0.tgz", + "integrity": "sha512-LonryJP+FxQQHsjGBi6W786TQB1Oym+agTpY0c+Kj8alnIw+DLUJb6SI8Y1GHGhLCH1yPRrucjObUmxNICQ1pg==", + "dev": true, + "requires": { + "ajv": "^6.1.0", + "ajv-keywords": "^3.1.0", + "chalk": "^2.3.2", + "strip-ansi": "^4.0.0", + "text-table": "^0.2.0", + "webpack-log": "^1.1.2" + }, + "dependencies": { + "ansi-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-3.0.0.tgz", + "integrity": "sha1-7QMXwyIGT3lGbAKWa922Bas32Zg=", + "dev": true + }, + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "strip-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-4.0.0.tgz", + "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=", + "dev": true, + "requires": { + "ansi-regex": "^3.0.0" + } + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "abbrev": { + "version": "1.0.9", + "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.0.9.tgz", + "integrity": "sha1-kbR5JYinc4wl813W9jdSovh3YTU=", + "dev": true + }, + "acorn": { + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-5.7.1.tgz", + "integrity": "sha512-d+nbxBUGKg7Arpsvbnlq61mc12ek3EY8EQldM3GPAhWJ1UVxC6TDGbIvUMNU6obBX3i1+ptCIzV4vq0gFPEGVQ==", + "dev": true + }, + "acorn-dynamic-import": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/acorn-dynamic-import/-/acorn-dynamic-import-3.0.0.tgz", + "integrity": "sha512-zVWV8Z8lislJoOKKqdNMOB+s6+XV5WERty8MnKBeFgwA+19XJjJHs2RP5dzM57FftIs+jQnRToLiWazKr6sSWg==", + "dev": true, + "requires": { + "acorn": "^5.0.0" + } + }, + "ajv": { + "version": "6.5.2", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.5.2.tgz", + "integrity": "sha512-hOs7GfvI6tUI1LfZddH82ky6mOMyTuY0mk7kE2pWpmhhUSkumzaTO5vbVwij39MdwPQWCV4Zv57Eo06NtL/GVA==", + "dev": true, + "requires": { + "fast-deep-equal": "^2.0.1", + "fast-json-stable-stringify": "^2.0.0", + "json-schema-traverse": "^0.4.1", + "uri-js": "^4.2.1" + } + }, + "ajv-keywords": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/ajv-keywords/-/ajv-keywords-3.2.0.tgz", + "integrity": "sha1-6GuBnGAs+IIa1jdBNpjx3sAhhHo=", + "dev": true + }, + "align-text": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/align-text/-/align-text-0.1.4.tgz", + "integrity": "sha1-DNkKVhCT810KmSVsIrcGlDP60Rc=", + "dev": true, + "requires": { + "kind-of": "^3.0.2", + "longest": "^1.0.1", + "repeat-string": "^1.5.2" + } + }, + "alphanum-sort": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/alphanum-sort/-/alphanum-sort-1.0.2.tgz", + "integrity": "sha1-l6ERlkmyEa0zaR2fn0hqjsn74KM=", + "dev": true + }, + "amdefine": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/amdefine/-/amdefine-1.0.1.tgz", + "integrity": "sha1-SlKCrBZHKek2Gbz9OtFR+BfOkfU=", + "dev": true + }, + "ansi-escapes": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-3.1.0.tgz", + "integrity": "sha512-UgAb8H9D41AQnu/PbWlCofQVcnV4Gs2bBJi9eZPxfU/hgglFh3SMDMENRIqdr7H6XFnXdoknctFByVsCOotTVw==", + "dev": true + }, + "ansi-regex": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", + "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=", + "dev": true + }, + "ansi-styles": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-2.2.1.tgz", + "integrity": "sha1-tDLdM1i2NM914eRmQ2gkBTPB3b4=", + "dev": true + }, + "anymatch": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-2.0.0.tgz", + "integrity": "sha512-5teOsQWABXHHBFP9y3skS5P3d/WfWXpv3FUpy+LorMrNYaT9pI4oLMQX7jzQ2KklNpGpWHzdCXTDT2Y3XGlZBw==", + "dev": true, + "requires": { + "micromatch": "^3.1.4", + "normalize-path": "^2.1.1" + } + }, + "aproba": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/aproba/-/aproba-1.2.0.tgz", + "integrity": "sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw==", + "dev": true + }, + "argparse": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", + "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", + "dev": true, + "requires": { + "sprintf-js": "~1.0.2" + } + }, + "arr-diff": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/arr-diff/-/arr-diff-4.0.0.tgz", + "integrity": "sha1-1kYQdP6/7HHn4VI1dhoyml3HxSA=", + "dev": true + }, + "arr-flatten": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/arr-flatten/-/arr-flatten-1.1.0.tgz", + "integrity": "sha512-L3hKV5R/p5o81R7O02IGnwpDmkp6E982XhtbuwSe3O4qOtMMMtodicASA1Cny2U+aCXcNpml+m4dPsvsJ3jatg==", + "dev": true + }, + "arr-union": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/arr-union/-/arr-union-3.1.0.tgz", + "integrity": "sha1-45sJrqne+Gao8gbiiK9jkZuuOcQ=", + "dev": true + }, + "array-union": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/array-union/-/array-union-1.0.2.tgz", + "integrity": "sha1-mjRBDk9OPaI96jdb5b5w8kd47Dk=", + "dev": true, + "requires": { + "array-uniq": "^1.0.1" + } + }, + "array-uniq": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/array-uniq/-/array-uniq-1.0.3.tgz", + "integrity": "sha1-r2rId6Jcx/dOBYiUdThY39sk/bY=", + "dev": true + }, + "array-unique": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/array-unique/-/array-unique-0.3.2.tgz", + "integrity": "sha1-qJS3XUvE9s1nnvMkSp/Y9Gri1Cg=", + "dev": true + }, + "arrify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/arrify/-/arrify-1.0.1.tgz", + "integrity": "sha1-iYUI2iIm84DfkEcoRWhJwVAaSw0=", + "dev": true + }, + "asn1.js": { + "version": "4.10.1", + "resolved": "https://registry.npmjs.org/asn1.js/-/asn1.js-4.10.1.tgz", + "integrity": "sha512-p32cOF5q0Zqs9uBiONKYLm6BClCoBCM5O9JfeUSlnQLBTxYdTK+pW+nXflm8UkKd2UYlEbYz5qEi0JuZR9ckSw==", + "dev": true, + "requires": { + "bn.js": "^4.0.0", + "inherits": "^2.0.1", + "minimalistic-assert": "^1.0.0" + } + }, + "assert": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/assert/-/assert-1.4.1.tgz", + "integrity": "sha1-mZEtWRg2tab1s0XA8H7vwI/GXZE=", + "dev": true, + "requires": { + "util": "0.10.3" + }, + "dependencies": { + "inherits": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.1.tgz", + "integrity": "sha1-sX0I0ya0Qj5Wjv9xn5GwscvfafE=", + "dev": true + }, + "util": { + "version": "0.10.3", + "resolved": "https://registry.npmjs.org/util/-/util-0.10.3.tgz", + "integrity": "sha1-evsa/lCAUkZInj23/g7TeTNqwPk=", + "dev": true, + "requires": { + "inherits": "2.0.1" + } + } + } + }, + "assign-symbols": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/assign-symbols/-/assign-symbols-1.0.0.tgz", + "integrity": "sha1-WWZ/QfrdTyDMvCu5a41Pf3jsA2c=", + "dev": true + }, + "async": { + "version": "1.5.2", + "resolved": "https://registry.npmjs.org/async/-/async-1.5.2.tgz", + "integrity": "sha1-7GphrlZIDAw8skHJVhjiCJL5Zyo=", + "dev": true + }, + "async-each": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/async-each/-/async-each-1.0.1.tgz", + "integrity": "sha1-GdOGodntxufByF04iu28xW0zYC0=", + "dev": true + }, + "atob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/atob/-/atob-2.1.1.tgz", + "integrity": "sha1-ri1acpR38onWDdf5amMUoi3Wwio=", + "dev": true + }, + "autoprefixer": { + "version": "6.7.7", + "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-6.7.7.tgz", + "integrity": "sha1-Hb0cg1ZY41zj+ZhAmdsAWFx4IBQ=", + "dev": true, + "requires": { + "browserslist": "^1.7.6", + "caniuse-db": "^1.0.30000634", + "normalize-range": "^0.1.2", + "num2fraction": "^1.2.2", + "postcss": "^5.2.16", + "postcss-value-parser": "^3.2.3" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "babel": { + "version": "6.23.0", + "resolved": "https://registry.npmjs.org/babel/-/babel-6.23.0.tgz", + "integrity": "sha1-0NHn2APpdHZb7qMjLU4VPA77kPQ=", + "dev": true + }, + "babel-code-frame": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-code-frame/-/babel-code-frame-6.26.0.tgz", + "integrity": "sha1-Y/1D99weO7fONZR9uP42mj9Yx0s=", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "esutils": "^2.0.2", + "js-tokens": "^3.0.2" + } + }, + "babel-core": { + "version": "6.26.3", + "resolved": "https://registry.npmjs.org/babel-core/-/babel-core-6.26.3.tgz", + "integrity": "sha512-6jyFLuDmeidKmUEb3NM+/yawG0M2bDZ9Z1qbZP59cyHLz8kYGKYwpJP0UwUKKUiTRNvxfLesJnTedqczP7cTDA==", + "dev": true, + "requires": { + "babel-code-frame": "^6.26.0", + "babel-generator": "^6.26.0", + "babel-helpers": "^6.24.1", + "babel-messages": "^6.23.0", + "babel-register": "^6.26.0", + "babel-runtime": "^6.26.0", + "babel-template": "^6.26.0", + "babel-traverse": "^6.26.0", + "babel-types": "^6.26.0", + "babylon": "^6.18.0", + "convert-source-map": "^1.5.1", + "debug": "^2.6.9", + "json5": "^0.5.1", + "lodash": "^4.17.4", + "minimatch": "^3.0.4", + "path-is-absolute": "^1.0.1", + "private": "^0.1.8", + "slash": "^1.0.0", + "source-map": "^0.5.7" + } + }, + "babel-generator": { + "version": "6.26.1", + "resolved": "https://registry.npmjs.org/babel-generator/-/babel-generator-6.26.1.tgz", + "integrity": "sha512-HyfwY6ApZj7BYTcJURpM5tznulaBvyio7/0d4zFOeMPUmfxkCjHocCuoLa2SAGzBI8AREcH3eP3758F672DppA==", + "dev": true, + "requires": { + "babel-messages": "^6.23.0", + "babel-runtime": "^6.26.0", + "babel-types": "^6.26.0", + "detect-indent": "^4.0.0", + "jsesc": "^1.3.0", + "lodash": "^4.17.4", + "source-map": "^0.5.7", + "trim-right": "^1.0.1" + } + }, + "babel-helpers": { + "version": "6.24.1", + "resolved": "https://registry.npmjs.org/babel-helpers/-/babel-helpers-6.24.1.tgz", + "integrity": "sha1-NHHenK7DiOXIUOWX5Yom3fN2ArI=", + "dev": true, + "requires": { + "babel-runtime": "^6.22.0", + "babel-template": "^6.24.1" + } + }, + "babel-istanbul": { + "version": "0.12.2", + "resolved": "https://registry.npmjs.org/babel-istanbul/-/babel-istanbul-0.12.2.tgz", + "integrity": "sha1-5yPwfJokMtiAVVILwi519cI5Fhw=", + "dev": true, + "requires": { + "abbrev": "1.0.x", + "async": "1.x", + "escodegen": "1.8.x", + "esprima": "2.7.x", + "handlebars": "^4.0.1", + "js-yaml": "3.x", + "mkdirp": "0.5.x", + "multi-glob": "^1.0.1", + "nopt": "3.x", + "object-assign": "^4.0.1", + "once": "1.x", + "resolve": "^1.1.0", + "source-map": "0.4.x", + "supports-color": "3.1.x", + "which": "1.2.x", + "wordwrap": "1.0.x" + }, + "dependencies": { + "source-map": { + "version": "0.4.4", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.4.4.tgz", + "integrity": "sha1-66T12pwNyZneaAMti092FzZSA2s=", + "dev": true, + "requires": { + "amdefine": ">=0.0.4" + } + }, + "supports-color": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.1.2.tgz", + "integrity": "sha1-cqJiiU2dQIuVbKBf83su2KbiotU=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "babel-loader": { + "version": "7.1.5", + "resolved": "https://registry.npmjs.org/babel-loader/-/babel-loader-7.1.5.tgz", + "integrity": "sha512-iCHfbieL5d1LfOQeeVJEUyD9rTwBcP/fcEbRCfempxTDuqrKpu0AZjLAQHEQa3Yqyj9ORKe2iHfoj4rHLf7xpw==", + "dev": true, + "requires": { + "find-cache-dir": "^1.0.0", + "loader-utils": "^1.0.2", + "mkdirp": "^0.5.1" + } + }, + "babel-messages": { + "version": "6.23.0", + "resolved": "https://registry.npmjs.org/babel-messages/-/babel-messages-6.23.0.tgz", + "integrity": "sha1-8830cDhYA1sqKVHG7F7fbGLyYw4=", + "dev": true, + "requires": { + "babel-runtime": "^6.22.0" + } + }, + "babel-plugin-css-modules-transform": { + "version": "1.6.1", + "resolved": "https://registry.npmjs.org/babel-plugin-css-modules-transform/-/babel-plugin-css-modules-transform-1.6.1.tgz", + "integrity": "sha512-Iv98dKRgQnhuHFcn2heHH1OpHo9LGyoKjlsAkj6/Q3wkwpVyHrNfVua/WHnrwe2f7EHy1KANnOSg+q4AJ6ZzaQ==", + "dev": true, + "requires": { + "css-modules-require-hook": "^4.0.6", + "mkdirp": "^0.5.1" + } + }, + "babel-polyfill": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-polyfill/-/babel-polyfill-6.26.0.tgz", + "integrity": "sha1-N5k3q8Z9eJWXCtxiHyhM2WbPIVM=", + "dev": true, + "requires": { + "babel-runtime": "^6.26.0", + "core-js": "^2.5.0", + "regenerator-runtime": "^0.10.5" + }, + "dependencies": { + "regenerator-runtime": { + "version": "0.10.5", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.10.5.tgz", + "integrity": "sha1-M2w+/BIgrc7dosn6tntaeVWjNlg=", + "dev": true + } + } + }, + "babel-register": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-register/-/babel-register-6.26.0.tgz", + "integrity": "sha1-btAhFz4vy0htestFxgCahW9kcHE=", + "dev": true, + "requires": { + "babel-core": "^6.26.0", + "babel-runtime": "^6.26.0", + "core-js": "^2.5.0", + "home-or-tmp": "^2.0.0", + "lodash": "^4.17.4", + "mkdirp": "^0.5.1", + "source-map-support": "^0.4.15" + } + }, + "babel-runtime": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-runtime/-/babel-runtime-6.26.0.tgz", + "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=", + "dev": true, + "requires": { + "core-js": "^2.4.0", + "regenerator-runtime": "^0.11.0" + } + }, + "babel-template": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-template/-/babel-template-6.26.0.tgz", + "integrity": "sha1-3gPi0WOWsGn0bdn/+FIfsaDjXgI=", + "dev": true, + "requires": { + "babel-runtime": "^6.26.0", + "babel-traverse": "^6.26.0", + "babel-types": "^6.26.0", + "babylon": "^6.18.0", + "lodash": "^4.17.4" + } + }, + "babel-traverse": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-traverse/-/babel-traverse-6.26.0.tgz", + "integrity": "sha1-RqnL1+3MYsjlwGTi0tjQ9ANXZu4=", + "dev": true, + "requires": { + "babel-code-frame": "^6.26.0", + "babel-messages": "^6.23.0", + "babel-runtime": "^6.26.0", + "babel-types": "^6.26.0", + "babylon": "^6.18.0", + "debug": "^2.6.8", + "globals": "^9.18.0", + "invariant": "^2.2.2", + "lodash": "^4.17.4" + } + }, + "babel-types": { + "version": "6.26.0", + "resolved": "https://registry.npmjs.org/babel-types/-/babel-types-6.26.0.tgz", + "integrity": "sha1-o7Bz+Uq0nrb6Vc1lInozQ4BjJJc=", + "dev": true, + "requires": { + "babel-runtime": "^6.26.0", + "esutils": "^2.0.2", + "lodash": "^4.17.4", + "to-fast-properties": "^1.0.3" + } + }, + "babylon": { + "version": "6.18.0", + "resolved": "https://registry.npmjs.org/babylon/-/babylon-6.18.0.tgz", + "integrity": "sha512-q/UEjfGJ2Cm3oKV71DJz9d25TPnq5rhBVL2Q4fA5wcC3jcrdn7+SssEybFIxwAvvP+YCsCYNKughoF33GxgycQ==", + "dev": true + }, + "balanced-match": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.0.tgz", + "integrity": "sha1-ibTRmasr7kneFk6gK4nORi1xt2c=", + "dev": true + }, + "base": { + "version": "0.11.2", + "resolved": "https://registry.npmjs.org/base/-/base-0.11.2.tgz", + "integrity": "sha512-5T6P4xPgpp0YDFvSWwEZ4NoE3aM4QBQXDzmVbraCkFj8zHM+mba8SyqB5DbZWyR7mYHo6Y7BdQo3MoA4m0TeQg==", + "dev": true, + "requires": { + "cache-base": "^1.0.1", + "class-utils": "^0.3.5", + "component-emitter": "^1.2.1", + "define-property": "^1.0.0", + "isobject": "^3.0.1", + "mixin-deep": "^1.2.0", + "pascalcase": "^0.1.1" + }, + "dependencies": { + "define-property": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", + "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", + "dev": true, + "requires": { + "is-descriptor": "^1.0.0" + } + }, + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "^1.0.0", + "is-data-descriptor": "^1.0.0", + "kind-of": "^6.0.2" + } + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "base64-js": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.3.0.tgz", + "integrity": "sha512-ccav/yGvoa80BQDljCxsmmQ3Xvx60/UpBIij5QN21W3wBi/hhIC9OoO+KLpu9IJTS9j4DRVJ3aDDF9cMSoa2lw==", + "dev": true + }, + "big.js": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/big.js/-/big.js-3.2.0.tgz", + "integrity": "sha512-+hN/Zh2D08Mx65pZ/4g5bsmNiZUuChDiQfTUQ7qJr4/kuopCr88xZsAXv6mBoZEsUI4OuGHlX59qE94K2mMW8Q==", + "dev": true + }, + "binary-extensions": { + "version": "1.11.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-1.11.0.tgz", + "integrity": "sha1-RqoXUftqL5PuXmibsQh9SxTGwgU=", + "dev": true + }, + "bluebird": { + "version": "3.5.1", + "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.5.1.tgz", + "integrity": "sha512-MKiLiV+I1AA596t9w1sQJ8jkiSr5+ZKi0WKrYGUn6d1Fx+Ij4tIj+m2WMQSGczs5jZVxV339chE8iwk6F64wjA==", + "dev": true + }, + "bn.js": { + "version": "4.11.8", + "resolved": "https://registry.npmjs.org/bn.js/-/bn.js-4.11.8.tgz", + "integrity": "sha512-ItfYfPLkWHUjckQCk8xC+LwxgK8NYcXywGigJgSwOP8Y2iyWT4f2vsZnoOXTTbo+o5yXmIUJ4gn5538SO5S3gA==", + "dev": true + }, + "bootstrap-3-typeahead": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/bootstrap-3-typeahead/-/bootstrap-3-typeahead-4.0.2.tgz", + "integrity": "sha1-yxyWkESFaGIJb8jHHMIbOsu1BBI=" + }, + "bootstrap-toggle": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/bootstrap-toggle/-/bootstrap-toggle-2.2.2.tgz", + "integrity": "sha1-K4hTT8G5mGdPh3+Yug2LW3Q+lv4=" + }, + "brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "braces": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", + "dev": true, + "requires": { + "arr-flatten": "^1.1.0", + "array-unique": "^0.3.2", + "extend-shallow": "^2.0.1", + "fill-range": "^4.0.0", + "isobject": "^3.0.1", + "repeat-element": "^1.1.2", + "snapdragon": "^0.8.1", + "snapdragon-node": "^2.0.1", + "split-string": "^3.0.2", + "to-regex": "^3.0.1" + }, + "dependencies": { + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + } + } + }, + "brorand": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/brorand/-/brorand-1.1.0.tgz", + "integrity": "sha1-EsJe/kCkXjwyPrhnWgoM5XsiNx8=", + "dev": true + }, + "browserify-aes": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/browserify-aes/-/browserify-aes-1.2.0.tgz", + "integrity": "sha512-+7CHXqGuspUn/Sl5aO7Ea0xWGAtETPXNSAjHo48JfLdPWcMng33Xe4znFvQweqc/uzk5zSOI3H52CYnjCfb5hA==", + "dev": true, + "requires": { + "buffer-xor": "^1.0.3", + "cipher-base": "^1.0.0", + "create-hash": "^1.1.0", + "evp_bytestokey": "^1.0.3", + "inherits": "^2.0.1", + "safe-buffer": "^5.0.1" + } + }, + "browserify-cipher": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/browserify-cipher/-/browserify-cipher-1.0.1.tgz", + "integrity": "sha512-sPhkz0ARKbf4rRQt2hTpAHqn47X3llLkUGn+xEJzLjwY8LRs2p0v7ljvI5EyoRO/mexrNunNECisZs+gw2zz1w==", + "dev": true, + "requires": { + "browserify-aes": "^1.0.4", + "browserify-des": "^1.0.0", + "evp_bytestokey": "^1.0.0" + } + }, + "browserify-des": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/browserify-des/-/browserify-des-1.0.1.tgz", + "integrity": "sha512-zy0Cobe3hhgpiOM32Tj7KQ3Vl91m0njwsjzZQK1L+JDf11dzP9qIvjreVinsvXrgfjhStXwUWAEpB9D7Gwmayw==", + "dev": true, + "requires": { + "cipher-base": "^1.0.1", + "des.js": "^1.0.0", + "inherits": "^2.0.1" + } + }, + "browserify-rsa": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/browserify-rsa/-/browserify-rsa-4.0.1.tgz", + "integrity": "sha1-IeCr+vbyApzy+vsTNWenAdQTVSQ=", + "dev": true, + "requires": { + "bn.js": "^4.1.0", + "randombytes": "^2.0.1" + } + }, + "browserify-sign": { + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/browserify-sign/-/browserify-sign-4.0.4.tgz", + "integrity": "sha1-qk62jl17ZYuqa/alfmMMvXqT0pg=", + "dev": true, + "requires": { + "bn.js": "^4.1.1", + "browserify-rsa": "^4.0.0", + "create-hash": "^1.1.0", + "create-hmac": "^1.1.2", + "elliptic": "^6.0.0", + "inherits": "^2.0.1", + "parse-asn1": "^5.0.0" + } + }, + "browserify-zlib": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/browserify-zlib/-/browserify-zlib-0.2.0.tgz", + "integrity": "sha512-Z942RysHXmJrhqk88FmKBVq/v5tqmSkDz7p54G/MGyjMnCFFnC79XWNbg+Vta8W6Wb2qtSZTSxIGkJrRpCFEiA==", + "dev": true, + "requires": { + "pako": "~1.0.5" + } + }, + "browserslist": { + "version": "1.7.7", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-1.7.7.tgz", + "integrity": "sha1-C9dnBCWL6CmyOYu1Dkti0aFmsLk=", + "dev": true, + "requires": { + "caniuse-db": "^1.0.30000639", + "electron-to-chromium": "^1.2.7" + } + }, + "buffer": { + "version": "4.9.1", + "resolved": "https://registry.npmjs.org/buffer/-/buffer-4.9.1.tgz", + "integrity": "sha1-bRu2AbB6TvztlwlBMgkwJ8lbwpg=", + "dev": true, + "requires": { + "base64-js": "^1.0.2", + "ieee754": "^1.1.4", + "isarray": "^1.0.0" + } + }, + "buffer-from": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/buffer-from/-/buffer-from-1.1.0.tgz", + "integrity": "sha512-c5mRlguI/Pe2dSZmpER62rSCu0ryKmWddzRYsuXc50U2/g8jMOulc31VZMa4mYx31U5xsmSOpDCgH88Vl9cDGQ==", + "dev": true + }, + "buffer-xor": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/buffer-xor/-/buffer-xor-1.0.3.tgz", + "integrity": "sha1-JuYe0UIvtw3ULm42cp7VHYVf6Nk=", + "dev": true + }, + "builtin-status-codes": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/builtin-status-codes/-/builtin-status-codes-3.0.0.tgz", + "integrity": "sha1-hZgoeOIbmOHGZCXgPQF0eI9Wnug=", + "dev": true + }, + "cacache": { + "version": "10.0.4", + "resolved": "https://registry.npmjs.org/cacache/-/cacache-10.0.4.tgz", + "integrity": "sha512-Dph0MzuH+rTQzGPNT9fAnrPmMmjKfST6trxJeK7NQuHRaVw24VzPRWTmg9MpcwOVQZO0E1FBICUlFeNaKPIfHA==", + "dev": true, + "requires": { + "bluebird": "^3.5.1", + "chownr": "^1.0.1", + "glob": "^7.1.2", + "graceful-fs": "^4.1.11", + "lru-cache": "^4.1.1", + "mississippi": "^2.0.0", + "mkdirp": "^0.5.1", + "move-concurrently": "^1.0.1", + "promise-inflight": "^1.0.1", + "rimraf": "^2.6.2", + "ssri": "^5.2.4", + "unique-filename": "^1.1.0", + "y18n": "^4.0.0" + }, + "dependencies": { + "glob": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.2.tgz", + "integrity": "sha512-MJTUg1kjuLeQCJ+ccE4Vpa6kKVXkPYJ2mOCQyUuKLcLQsdrMCpBPUi8qVE6+YuaJkozeA9NusTAw3hLr8Xe5EQ==", + "dev": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + } + } + }, + "cache-base": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/cache-base/-/cache-base-1.0.1.tgz", + "integrity": "sha512-AKcdTnFSWATd5/GCPRxr2ChwIJ85CeyrEyjRHlKxQ56d4XJMGym0uAiKn0xbLOGOl3+yRpOTi484dVCEc5AUzQ==", + "dev": true, + "requires": { + "collection-visit": "^1.0.0", + "component-emitter": "^1.2.1", + "get-value": "^2.0.6", + "has-value": "^1.0.0", + "isobject": "^3.0.1", + "set-value": "^2.0.0", + "to-object-path": "^0.3.0", + "union-value": "^1.0.0", + "unset-value": "^1.0.0" + } + }, + "camelcase": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-1.2.1.tgz", + "integrity": "sha1-m7UwTS4LVmmLLHWLCKPqqdqlijk=", + "dev": true, + "optional": true + }, + "caniuse-api": { + "version": "1.6.1", + "resolved": "https://registry.npmjs.org/caniuse-api/-/caniuse-api-1.6.1.tgz", + "integrity": "sha1-tTTnxzTE+B7F++isoq0kNUuWLGw=", + "dev": true, + "requires": { + "browserslist": "^1.3.6", + "caniuse-db": "^1.0.30000529", + "lodash.memoize": "^4.1.2", + "lodash.uniq": "^4.5.0" + } + }, + "caniuse-db": { + "version": "1.0.30000865", + "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000865.tgz", + "integrity": "sha1-gv+2TUD3VnYgqsAtOmMgeWiavGs=", + "dev": true + }, + "center-align": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/center-align/-/center-align-0.1.3.tgz", + "integrity": "sha1-qg0yYptu6XIgBBHL1EYckHvCt60=", + "dev": true, + "optional": true, + "requires": { + "align-text": "^0.1.3", + "lazy-cache": "^1.0.3" + } + }, + "chalk": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-1.1.3.tgz", + "integrity": "sha1-qBFcVeSnAv5NFQq9OHKCKn4J/Jg=", + "dev": true, + "requires": { + "ansi-styles": "^2.2.1", + "escape-string-regexp": "^1.0.2", + "has-ansi": "^2.0.0", + "strip-ansi": "^3.0.0", + "supports-color": "^2.0.0" + } + }, + "chardet": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/chardet/-/chardet-0.5.0.tgz", + "integrity": "sha512-9ZTaoBaePSCFvNlNGrsyI8ZVACP2svUtq0DkM7t4K2ClAa96sqOIRjAzDTc8zXzFt1cZR46rRzLTiHFSJ+Qw0g==", + "dev": true + }, + "chokidar": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-2.0.4.tgz", + "integrity": "sha512-z9n7yt9rOvIJrMhvDtDictKrkFHeihkNl6uWMmZlmL6tJtX9Cs+87oK+teBx+JIgzvbX3yZHT3eF8vpbDxHJXQ==", + "dev": true, + "requires": { + "anymatch": "^2.0.0", + "async-each": "^1.0.0", + "braces": "^2.3.0", + "fsevents": "^1.2.2", + "glob-parent": "^3.1.0", + "inherits": "^2.0.1", + "is-binary-path": "^1.0.0", + "is-glob": "^4.0.0", + "lodash.debounce": "^4.0.8", + "normalize-path": "^2.1.1", + "path-is-absolute": "^1.0.0", + "readdirp": "^2.0.0", + "upath": "^1.0.5" + } + }, + "chownr": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/chownr/-/chownr-1.0.1.tgz", + "integrity": "sha1-4qdQQqlVGQi+vSW4Uj1fl2nXkYE=", + "dev": true + }, + "chrome-trace-event": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-1.0.0.tgz", + "integrity": "sha512-xDbVgyfDTT2piup/h8dK/y4QZfJRSa73bw1WZ8b4XM1o7fsFubUVGYcE+1ANtOzJJELGpYoG2961z0Z6OAld9A==", + "dev": true, + "requires": { + "tslib": "^1.9.0" + } + }, + "cipher-base": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/cipher-base/-/cipher-base-1.0.4.tgz", + "integrity": "sha512-Kkht5ye6ZGmwv40uUDZztayT2ThLQGfnj/T71N/XzeZeo3nf8foyW7zGTsPYkEya3m5f3cAypH+qe7YOrM1U2Q==", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "safe-buffer": "^5.0.1" + } + }, + "clap": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/clap/-/clap-1.2.3.tgz", + "integrity": "sha512-4CoL/A3hf90V3VIEjeuhSvlGFEHKzOz+Wfc2IVZc+FaUgU0ZQafJTP49fvnULipOPcAfqhyI2duwQyns6xqjYA==", + "dev": true, + "requires": { + "chalk": "^1.1.3" + } + }, + "class-utils": { + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/class-utils/-/class-utils-0.3.6.tgz", + "integrity": "sha512-qOhPa/Fj7s6TY8H8esGu5QNpMMQxz79h+urzrNYN6mn+9BnxlDGf5QZ+XeCDsxSjPqsSR56XOZOJmpeurnLMeg==", + "dev": true, + "requires": { + "arr-union": "^3.1.0", + "define-property": "^0.2.5", + "isobject": "^3.0.0", + "static-extend": "^0.1.1" + }, + "dependencies": { + "define-property": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-0.2.5.tgz", + "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=", + "dev": true, + "requires": { + "is-descriptor": "^0.1.0" + } + } + } + }, + "clean-webpack-plugin": { + "version": "0.1.19", + "resolved": "https://registry.npmjs.org/clean-webpack-plugin/-/clean-webpack-plugin-0.1.19.tgz", + "integrity": "sha512-M1Li5yLHECcN2MahoreuODul5LkjohJGFxLPTjl3j1ttKrF5rgjZET1SJduuqxLAuT1gAPOdkhg03qcaaU1KeA==", + "dev": true, + "requires": { + "rimraf": "^2.6.1" + } + }, + "cli-cursor": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/cli-cursor/-/cli-cursor-2.1.0.tgz", + "integrity": "sha1-s12sN2R5+sw+lHR9QdDQ9SOP/LU=", + "dev": true, + "requires": { + "restore-cursor": "^2.0.0" + } + }, + "cli-width": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/cli-width/-/cli-width-2.2.0.tgz", + "integrity": "sha1-/xnt6Kml5XkyQUewwR8PvLq+1jk=", + "dev": true + }, + "cliui": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-2.1.0.tgz", + "integrity": "sha1-S0dXYP+AJkx2LDoXGQMukcf+oNE=", + "dev": true, + "optional": true, + "requires": { + "center-align": "^0.1.1", + "right-align": "^0.1.1", + "wordwrap": "0.0.2" + }, + "dependencies": { + "wordwrap": { + "version": "0.0.2", + "resolved": "https://registry.npmjs.org/wordwrap/-/wordwrap-0.0.2.tgz", + "integrity": "sha1-t5Zpu0LstAn4PVg8rVLKF+qhZD8=", + "dev": true, + "optional": true + } + } + }, + "clone": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/clone/-/clone-1.0.4.tgz", + "integrity": "sha1-2jCcwmPfFZlMaIypAheco8fNfH4=", + "dev": true + }, + "coa": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/coa/-/coa-1.0.4.tgz", + "integrity": "sha1-qe8VNmDWqGqL3sAomlxoTSF0Mv0=", + "dev": true, + "requires": { + "q": "^1.1.2" + } + }, + "code-point-at": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/code-point-at/-/code-point-at-1.1.0.tgz", + "integrity": "sha1-DQcLTQQ6W+ozovGkDi7bPZpMz3c=", + "dev": true + }, + "collection-visit": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/collection-visit/-/collection-visit-1.0.0.tgz", + "integrity": "sha1-S8A3PBZLwykbTTaMgpzxqApZ3KA=", + "dev": true, + "requires": { + "map-visit": "^1.0.0", + "object-visit": "^1.0.0" + } + }, + "color": { + "version": "0.11.4", + "resolved": "https://registry.npmjs.org/color/-/color-0.11.4.tgz", + "integrity": "sha1-bXtcdPtl6EHNSHkq0e1eB7kE12Q=", + "dev": true, + "requires": { + "clone": "^1.0.2", + "color-convert": "^1.3.0", + "color-string": "^0.3.0" + } + }, + "color-convert": { + "version": "1.9.2", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.2.tgz", + "integrity": "sha512-3NUJZdhMhcdPn8vJ9v2UQJoH0qqoGUkYTgFEPZaPjEtwmmKUfNV46zZmgB2M5M4DCEQHMaCfWHCxiBflLm04Tg==", + "dev": true, + "requires": { + "color-name": "1.1.1" + } + }, + "color-name": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.1.tgz", + "integrity": "sha1-SxQVMEz1ACjqgWQ2Q72C6gWANok=", + "dev": true + }, + "color-string": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/color-string/-/color-string-0.3.0.tgz", + "integrity": "sha1-J9RvtnAlxcL6JZk7+/V55HhBuZE=", + "dev": true, + "requires": { + "color-name": "^1.0.0" + } + }, + "colormin": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/colormin/-/colormin-1.1.2.tgz", + "integrity": "sha1-6i90IKcrlogaOKrlnsEkpvcpgTM=", + "dev": true, + "requires": { + "color": "^0.11.0", + "css-color-names": "0.0.4", + "has": "^1.0.1" + } + }, + "colors": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/colors/-/colors-1.1.2.tgz", + "integrity": "sha1-FopHAXVran9RoSzgyXv6KMCE7WM=", + "dev": true + }, + "commander": { + "version": "2.13.0", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.13.0.tgz", + "integrity": "sha512-MVuS359B+YzaWqjCL/c+22gfryv+mCBPHAv3zyVI2GN8EY6IRP8VwtasXn8jyyhvvq84R4ImN1OKRtcbIasjYA==", + "dev": true + }, + "commondir": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/commondir/-/commondir-1.0.1.tgz", + "integrity": "sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs=", + "dev": true + }, + "component-emitter": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.2.1.tgz", + "integrity": "sha1-E3kY1teCg/ffemt8WmPhQOaUJeY=", + "dev": true + }, + "concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s=", + "dev": true + }, + "concat-stream": { + "version": "1.6.2", + "resolved": "https://registry.npmjs.org/concat-stream/-/concat-stream-1.6.2.tgz", + "integrity": "sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw==", + "dev": true, + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + } + }, + "console-browserify": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/console-browserify/-/console-browserify-1.1.0.tgz", + "integrity": "sha1-8CQcRXMKn8YyOyBtvzjtx0HQuxA=", + "dev": true, + "requires": { + "date-now": "^0.1.4" + } + }, + "constants-browserify": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/constants-browserify/-/constants-browserify-1.0.0.tgz", + "integrity": "sha1-wguW2MYXdIqvHBYCF2DNJ/y4y3U=", + "dev": true + }, + "convert-source-map": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-1.5.1.tgz", + "integrity": "sha1-uCeAl7m8IpNl3lxiz1/K7YtVmeU=", + "dev": true + }, + "copy-concurrently": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/copy-concurrently/-/copy-concurrently-1.0.5.tgz", + "integrity": "sha512-f2domd9fsVDFtaFcbaRZuYXwtdmnzqbADSwhSWYxYB/Q8zsdUUFMXVRwXGDMWmbEzAn1kdRrtI1T/KTFOL4X2A==", + "dev": true, + "requires": { + "aproba": "^1.1.1", + "fs-write-stream-atomic": "^1.0.8", + "iferr": "^0.1.5", + "mkdirp": "^0.5.1", + "rimraf": "^2.5.4", + "run-queue": "^1.0.0" + } + }, + "copy-descriptor": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/copy-descriptor/-/copy-descriptor-0.1.1.tgz", + "integrity": "sha1-Z29us8OZl8LuGsOpJP1hJHSPV40=", + "dev": true + }, + "copy-webpack-plugin": { + "version": "4.5.2", + "resolved": "https://registry.npmjs.org/copy-webpack-plugin/-/copy-webpack-plugin-4.5.2.tgz", + "integrity": "sha512-zmC33E8FFSq3AbflTvqvPvBo621H36Afsxlui91d+QyZxPIuXghfnTsa1CuqiAaCPgJoSUWfTFbKJnadZpKEbQ==", + "dev": true, + "requires": { + "cacache": "^10.0.4", + "find-cache-dir": "^1.0.0", + "globby": "^7.1.1", + "is-glob": "^4.0.0", + "loader-utils": "^1.1.0", + "minimatch": "^3.0.4", + "p-limit": "^1.0.0", + "serialize-javascript": "^1.4.0" + } + }, + "core-js": { + "version": "2.5.7", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.5.7.tgz", + "integrity": "sha512-RszJCAxg/PP6uzXVXL6BsxSXx/B05oJAQ2vkJRjyjrEcNVycaqOmNb5OTxZPE3xa5gwZduqza6L9JOCenh/Ecw==", + "dev": true + }, + "core-util-is": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.2.tgz", + "integrity": "sha1-tf1UIgqivFq1eqtxQMlAdUUDwac=", + "dev": true + }, + "create-ecdh": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/create-ecdh/-/create-ecdh-4.0.3.tgz", + "integrity": "sha512-GbEHQPMOswGpKXM9kCWVrremUcBmjteUaQ01T9rkKCPDXfUHX0IoP9LpHYo2NPFampa4e+/pFDc3jQdxrxQLaw==", + "dev": true, + "requires": { + "bn.js": "^4.1.0", + "elliptic": "^6.0.0" + } + }, + "create-hash": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/create-hash/-/create-hash-1.2.0.tgz", + "integrity": "sha512-z00bCGNHDG8mHAkP7CtT1qVu+bFQUPjYq/4Iv3C3kWjTFV10zIjfSoeqXo9Asws8gwSHDGj/hl2u4OGIjapeCg==", + "dev": true, + "requires": { + "cipher-base": "^1.0.1", + "inherits": "^2.0.1", + "md5.js": "^1.3.4", + "ripemd160": "^2.0.1", + "sha.js": "^2.4.0" + } + }, + "create-hmac": { + "version": "1.1.7", + "resolved": "https://registry.npmjs.org/create-hmac/-/create-hmac-1.1.7.tgz", + "integrity": "sha512-MJG9liiZ+ogc4TzUwuvbER1JRdgvUFSB5+VR/g5h82fGaIRWMWddtKBHi7/sVhfjQZ6SehlyhvQYrcYkaUIpLg==", + "dev": true, + "requires": { + "cipher-base": "^1.0.3", + "create-hash": "^1.1.0", + "inherits": "^2.0.1", + "ripemd160": "^2.0.0", + "safe-buffer": "^5.0.1", + "sha.js": "^2.4.8" + } + }, + "cross-spawn": { + "version": "6.0.5", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-6.0.5.tgz", + "integrity": "sha512-eTVLrBSt7fjbDygz805pMnstIs2VTBNkRm0qxZd+M7A5XDdxVRWO5MxGBXZhjY4cqLYLdtrGqRf8mBPmzwSpWQ==", + "dev": true, + "requires": { + "nice-try": "^1.0.4", + "path-key": "^2.0.1", + "semver": "^5.5.0", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + } + }, + "crypto-browserify": { + "version": "3.12.0", + "resolved": "https://registry.npmjs.org/crypto-browserify/-/crypto-browserify-3.12.0.tgz", + "integrity": "sha512-fz4spIh+znjO2VjL+IdhEpRJ3YN6sMzITSBijk6FK2UvTqruSQW+/cCZTSNsMiZNvUeq0CqurF+dAbyiGOY6Wg==", + "dev": true, + "requires": { + "browserify-cipher": "^1.0.0", + "browserify-sign": "^4.0.0", + "create-ecdh": "^4.0.0", + "create-hash": "^1.1.0", + "create-hmac": "^1.1.0", + "diffie-hellman": "^5.0.0", + "inherits": "^2.0.1", + "pbkdf2": "^3.0.3", + "public-encrypt": "^4.0.0", + "randombytes": "^2.0.0", + "randomfill": "^1.0.3" + } + }, + "css-color-names": { + "version": "0.0.4", + "resolved": "https://registry.npmjs.org/css-color-names/-/css-color-names-0.0.4.tgz", + "integrity": "sha1-gIrcLnnPhHOAabZGyyDsJ762KeA=", + "dev": true + }, + "css-loader": { + "version": "0.28.11", + "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-0.28.11.tgz", + "integrity": "sha512-wovHgjAx8ZIMGSL8pTys7edA1ClmzxHeY6n/d97gg5odgsxEgKjULPR0viqyC+FWMCL9sfqoC/QCUBo62tLvPg==", + "dev": true, + "requires": { + "babel-code-frame": "^6.26.0", + "css-selector-tokenizer": "^0.7.0", + "cssnano": "^3.10.0", + "icss-utils": "^2.1.0", + "loader-utils": "^1.0.2", + "lodash.camelcase": "^4.3.0", + "object-assign": "^4.1.1", + "postcss": "^5.0.6", + "postcss-modules-extract-imports": "^1.2.0", + "postcss-modules-local-by-default": "^1.2.0", + "postcss-modules-scope": "^1.1.0", + "postcss-modules-values": "^1.3.0", + "postcss-value-parser": "^3.3.0", + "source-list-map": "^2.0.0" + }, + "dependencies": { + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "icss-utils": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/icss-utils/-/icss-utils-2.1.0.tgz", + "integrity": "sha1-g/Cg7DeL8yRheLbCrZE28TWxyWI=", + "dev": true, + "requires": { + "postcss": "^6.0.1" + }, + "dependencies": { + "postcss": { + "version": "6.0.23", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.23.tgz", + "integrity": "sha512-soOk1h6J3VMTZtVeVpv15/Hpdl2cBLX3CAw4TAbkpTJiNPk9YP/zWcD1ND+xEtvyuuvKzbxliTOIyvkSeSJ6ag==", + "dev": true, + "requires": { + "chalk": "^2.4.1", + "source-map": "^0.6.1", + "supports-color": "^5.4.0" + } + } + } + }, + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + }, + "dependencies": { + "ansi-styles": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-2.2.1.tgz", + "integrity": "sha1-tDLdM1i2NM914eRmQ2gkBTPB3b4=", + "dev": true + }, + "chalk": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-1.1.3.tgz", + "integrity": "sha1-qBFcVeSnAv5NFQq9OHKCKn4J/Jg=", + "dev": true, + "requires": { + "ansi-styles": "^2.2.1", + "escape-string-regexp": "^1.0.2", + "has-ansi": "^2.0.0", + "strip-ansi": "^3.0.0", + "supports-color": "^2.0.0" + }, + "dependencies": { + "supports-color": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-2.0.0.tgz", + "integrity": "sha1-U10EXOa2Nj+kARcIRimZXp3zJMc=", + "dev": true + } + } + }, + "has-flag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-1.0.0.tgz", + "integrity": "sha1-nZ55MWXOAXoA8AQYxD+UKnsdEfo=", + "dev": true + }, + "source-map": { + "version": "0.5.7", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz", + "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=", + "dev": true + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-modules-extract-imports": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/postcss-modules-extract-imports/-/postcss-modules-extract-imports-1.2.0.tgz", + "integrity": "sha1-ZhQOzs447wa/DT41XWm/WdFB6oU=", + "dev": true, + "requires": { + "postcss": "^6.0.1" + }, + "dependencies": { + "postcss": { + "version": "6.0.23", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.23.tgz", + "integrity": "sha512-soOk1h6J3VMTZtVeVpv15/Hpdl2cBLX3CAw4TAbkpTJiNPk9YP/zWcD1ND+xEtvyuuvKzbxliTOIyvkSeSJ6ag==", + "dev": true, + "requires": { + "chalk": "^2.4.1", + "source-map": "^0.6.1", + "supports-color": "^5.4.0" + } + } + } + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "css-modules-require-hook": { + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/css-modules-require-hook/-/css-modules-require-hook-4.2.3.tgz", + "integrity": "sha1-Z5LKQSsV4j5vm+agfc739Xf/kE0=", + "dev": true, + "requires": { + "debug": "^2.2.0", + "generic-names": "^1.0.1", + "glob-to-regexp": "^0.3.0", + "icss-replace-symbols": "^1.0.2", + "lodash": "^4.3.0", + "postcss": "^6.0.1", + "postcss-modules-extract-imports": "^1.0.0", + "postcss-modules-local-by-default": "^1.0.1", + "postcss-modules-resolve-imports": "^1.3.0", + "postcss-modules-scope": "^1.0.0", + "postcss-modules-values": "^1.1.1", + "seekout": "^1.0.1" + } + }, + "css-selector-tokenizer": { + "version": "0.7.0", + "resolved": "https://registry.npmjs.org/css-selector-tokenizer/-/css-selector-tokenizer-0.7.0.tgz", + "integrity": "sha1-5piEdK6MlTR3v15+/s/OzNnPTIY=", + "dev": true, + "requires": { + "cssesc": "^0.1.0", + "fastparse": "^1.1.1", + "regexpu-core": "^1.0.0" + } + }, + "cssesc": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-0.1.0.tgz", + "integrity": "sha1-yBSQPkViM3GgR3tAEJqq++6t27Q=", + "dev": true + }, + "cssnano": { + "version": "3.10.0", + "resolved": "https://registry.npmjs.org/cssnano/-/cssnano-3.10.0.tgz", + "integrity": "sha1-Tzj2zqK5sX+gFJDyPx3GjqZcHDg=", + "dev": true, + "requires": { + "autoprefixer": "^6.3.1", + "decamelize": "^1.1.2", + "defined": "^1.0.0", + "has": "^1.0.1", + "object-assign": "^4.0.1", + "postcss": "^5.0.14", + "postcss-calc": "^5.2.0", + "postcss-colormin": "^2.1.8", + "postcss-convert-values": "^2.3.4", + "postcss-discard-comments": "^2.0.4", + "postcss-discard-duplicates": "^2.0.1", + "postcss-discard-empty": "^2.0.1", + "postcss-discard-overridden": "^0.1.1", + "postcss-discard-unused": "^2.2.1", + "postcss-filter-plugins": "^2.0.0", + "postcss-merge-idents": "^2.1.5", + "postcss-merge-longhand": "^2.0.1", + "postcss-merge-rules": "^2.0.3", + "postcss-minify-font-values": "^1.0.2", + "postcss-minify-gradients": "^1.0.1", + "postcss-minify-params": "^1.0.4", + "postcss-minify-selectors": "^2.0.4", + "postcss-normalize-charset": "^1.1.0", + "postcss-normalize-url": "^3.0.7", + "postcss-ordered-values": "^2.1.0", + "postcss-reduce-idents": "^2.2.2", + "postcss-reduce-initial": "^1.0.0", + "postcss-reduce-transforms": "^1.0.3", + "postcss-svgo": "^2.1.1", + "postcss-unique-selectors": "^2.0.2", + "postcss-value-parser": "^3.2.3", + "postcss-zindex": "^2.0.1" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "csso": { + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/csso/-/csso-2.3.2.tgz", + "integrity": "sha1-3dUsWHAz9J6Utx/FVWnyUuj/X4U=", + "dev": true, + "requires": { + "clap": "^1.0.9", + "source-map": "^0.5.3" + } + }, + "cyclist": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/cyclist/-/cyclist-0.2.2.tgz", + "integrity": "sha1-GzN5LhHpFKL9bW7WRHRkRE5fpkA=", + "dev": true + }, + "d": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/d/-/d-1.0.0.tgz", + "integrity": "sha1-dUu1v+VUUdpppYuU1F9MWwRi1Y8=", + "dev": true, + "requires": { + "es5-ext": "^0.10.9" + } + }, + "d3": { + "version": "3.5.17", + "resolved": "https://registry.npmjs.org/d3/-/d3-3.5.17.tgz", + "integrity": "sha1-vEZ0gAQ3iyGjYMn8fPUjF5B2L7g=" + }, + "d3-collection": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/d3-collection/-/d3-collection-1.0.4.tgz", + "integrity": "sha1-NC39EoN8kJdPM/HMCnha6lcNzcI=" + }, + "d3-selection": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/d3-selection/-/d3-selection-1.3.0.tgz", + "integrity": "sha512-qgpUOg9tl5CirdqESUAu0t9MU/t3O9klYfGfyKsXEmhyxyzLpzpeh08gaxBUTQw1uXIOkr/30Ut2YRjSSxlmHA==" + }, + "d3-tip": { + "version": "0.9.1", + "resolved": "https://registry.npmjs.org/d3-tip/-/d3-tip-0.9.1.tgz", + "integrity": "sha512-EVBfG9d+HnjIoyVXfhpytWxlF59JaobwizqMX9EBXtsFmJytjwHeYiUs74ldHQjE7S9vzfKTx2LCtvUrIbuFYg==", + "requires": { + "d3-collection": "^1.0.4", + "d3-selection": "^1.3.0" + } + }, + "datatables.net": { + "version": "1.10.19", + "resolved": "https://registry.npmjs.org/datatables.net/-/datatables.net-1.10.19.tgz", + "integrity": "sha512-+ljXcI6Pj3PTGy5pesp3E5Dr3x3AV45EZe0o1r0gKENN2gafBKXodVnk2ypKwl2tTmivjxbkiqoWnipTefyBTA==", + "requires": { + "jquery": ">=1.7" + } + }, + "datatables.net-bs": { + "version": "1.10.19", + "resolved": "https://registry.npmjs.org/datatables.net-bs/-/datatables.net-bs-1.10.19.tgz", + "integrity": "sha512-5gxoI2n+duZP06+4xVC2TtH6zcY369/TRKTZ1DdSgDcDUl4OYQsrXCuaLJmbVzna/5Y5lrMmK7CxgvYgIynICA==", + "requires": { + "datatables.net": "1.10.19", + "jquery": ">=1.7" + } + }, + "date-now": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/date-now/-/date-now-0.1.4.tgz", + "integrity": "sha1-6vQ5/U1ISK105cx9vvIAZyueNFs=", + "dev": true + }, + "debug": { + "version": "2.6.9", + "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", + "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", + "dev": true, + "requires": { + "ms": "2.0.0" + } + }, + "decamelize": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-1.2.0.tgz", + "integrity": "sha1-9lNNFRSCabIDUue+4m9QH5oZEpA=", + "dev": true + }, + "decode-uri-component": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/decode-uri-component/-/decode-uri-component-0.2.0.tgz", + "integrity": "sha1-6zkTMzRYd1y4TNGh+uBiEGu4dUU=", + "dev": true + }, + "deep-is": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.3.tgz", + "integrity": "sha1-s2nW+128E+7PUk+RsHD+7cNXzzQ=", + "dev": true + }, + "define-properties": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.1.2.tgz", + "integrity": "sha1-g6c/L+pWmJj7c3GTyPhzyvbUXJQ=", + "dev": true, + "requires": { + "foreach": "^2.0.5", + "object-keys": "^1.0.8" + } + }, + "define-property": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-2.0.2.tgz", + "integrity": "sha512-jwK2UV4cnPpbcG7+VRARKTZPUWowwXA8bzH5NP6ud0oeAxyYPuGZUAC7hMugpCdz4BeSZl2Dl9k66CHJ/46ZYQ==", + "dev": true, + "requires": { + "is-descriptor": "^1.0.2", + "isobject": "^3.0.1" + }, + "dependencies": { + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "^1.0.0", + "is-data-descriptor": "^1.0.0", + "kind-of": "^6.0.2" + } + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "defined": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/defined/-/defined-1.0.0.tgz", + "integrity": "sha1-yY2bzvdWdBiOEQlpFRGZ45sfppM=", + "dev": true + }, + "des.js": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/des.js/-/des.js-1.0.0.tgz", + "integrity": "sha1-wHTS4qpqipoH29YfmhXCzYPsjsw=", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "minimalistic-assert": "^1.0.0" + } + }, + "detect-indent": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/detect-indent/-/detect-indent-4.0.0.tgz", + "integrity": "sha1-920GQ1LN9Docts5hnE7jqUdd4gg=", + "dev": true, + "requires": { + "repeating": "^2.0.0" + } + }, + "diffie-hellman": { + "version": "5.0.3", + "resolved": "https://registry.npmjs.org/diffie-hellman/-/diffie-hellman-5.0.3.tgz", + "integrity": "sha512-kqag/Nl+f3GwyK25fhUMYj81BUOrZ9IuJsjIcDE5icNM9FJHAVm3VcUDxdLPoQtTuUylWm6ZIknYJwwaPxsUzg==", + "dev": true, + "requires": { + "bn.js": "^4.1.0", + "miller-rabin": "^4.0.0", + "randombytes": "^2.0.0" + } + }, + "dir-glob": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-2.0.0.tgz", + "integrity": "sha512-37qirFDz8cA5fimp9feo43fSuRo2gHwaIn6dXL8Ber1dGwUosDrGZeCCXq57WnIqE4aQ+u3eQZzsk1yOzhdwag==", + "dev": true, + "requires": { + "arrify": "^1.0.1", + "path-type": "^3.0.0" + } + }, + "domain-browser": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/domain-browser/-/domain-browser-1.2.0.tgz", + "integrity": "sha512-jnjyiM6eRyZl2H+W8Q/zLMA481hzi0eszAaBUzIVnmYVDBbnLxVNnfu1HgEBvCbL+71FrxMl3E6lpKH7Ge3OXA==", + "dev": true + }, + "duplexify": { + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/duplexify/-/duplexify-3.6.0.tgz", + "integrity": "sha512-fO3Di4tBKJpYTFHAxTU00BcfWMY9w24r/x21a6rZRbsD/ToUgGxsMbiGRmB7uVAXeGKXD9MwiLZa5E97EVgIRQ==", + "dev": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + } + }, + "electron-to-chromium": { + "version": "1.3.51", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.3.51.tgz", + "integrity": "sha1-akK0nar38ipbN7mR2vlJ8029ubU=", + "dev": true + }, + "elliptic": { + "version": "6.4.0", + "resolved": "https://registry.npmjs.org/elliptic/-/elliptic-6.4.0.tgz", + "integrity": "sha1-ysmvh2LIWDYYcAPI3+GT5eLq5d8=", + "dev": true, + "requires": { + "bn.js": "^4.4.0", + "brorand": "^1.0.1", + "hash.js": "^1.0.0", + "hmac-drbg": "^1.0.0", + "inherits": "^2.0.1", + "minimalistic-assert": "^1.0.0", + "minimalistic-crypto-utils": "^1.0.0" + } + }, + "emojis-list": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/emojis-list/-/emojis-list-2.1.0.tgz", + "integrity": "sha1-TapNnbAPmBmIDHn6RXrlsJof04k=", + "dev": true + }, + "end-of-stream": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.1.tgz", + "integrity": "sha512-1MkrZNvWTKCaigbn+W15elq2BB/L22nqrSY5DKlo3X6+vclJm8Bb5djXJBmEX6fS3+zCh/F4VBK5Z2KxJt4s2Q==", + "dev": true, + "requires": { + "once": "^1.4.0" + } + }, + "enhanced-resolve": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-4.1.0.tgz", + "integrity": "sha512-F/7vkyTtyc/llOIn8oWclcB25KdRaiPBpZYDgJHgh/UHtpgT2p2eldQgtQnLtUvfMKPKxbRaQM/hHkvLHt1Vng==", + "dev": true, + "requires": { + "graceful-fs": "^4.1.2", + "memory-fs": "^0.4.0", + "tapable": "^1.0.0" + } + }, + "errno": { + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/errno/-/errno-0.1.7.tgz", + "integrity": "sha512-MfrRBDWzIWifgq6tJj60gkAwtLNb6sQPlcFrSOflcP1aFmmruKQ2wRnze/8V6kgyz7H3FF8Npzv78mZ7XLLflg==", + "dev": true, + "requires": { + "prr": "~1.0.1" + } + }, + "es5-ext": { + "version": "0.10.45", + "resolved": "https://registry.npmjs.org/es5-ext/-/es5-ext-0.10.45.tgz", + "integrity": "sha512-FkfM6Vxxfmztilbxxz5UKSD4ICMf5tSpRFtDNtkAhOxZ0EKtX6qwmXNyH/sFyIbX2P/nU5AMiA9jilWsUGJzCQ==", + "dev": true, + "requires": { + "es6-iterator": "~2.0.3", + "es6-symbol": "~3.1.1", + "next-tick": "1" + } + }, + "es6-iterator": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/es6-iterator/-/es6-iterator-2.0.3.tgz", + "integrity": "sha1-p96IkUGgWpSwhUQDstCg+/qY87c=", + "dev": true, + "requires": { + "d": "1", + "es5-ext": "^0.10.35", + "es6-symbol": "^3.1.1" + } + }, + "es6-symbol": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/es6-symbol/-/es6-symbol-3.1.1.tgz", + "integrity": "sha1-vwDvT9q2uhtG7Le2KbTH7VcVzHc=", + "dev": true, + "requires": { + "d": "1", + "es5-ext": "~0.10.14" + } + }, + "escape-string-regexp": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz", + "integrity": "sha1-G2HAViGQqN/2rjuyzwIAyhMLhtQ=", + "dev": true + }, + "escodegen": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/escodegen/-/escodegen-1.8.1.tgz", + "integrity": "sha1-WltTr0aTEQvrsIZ6o0MN07cKEBg=", + "dev": true, + "requires": { + "esprima": "^2.7.1", + "estraverse": "^1.9.1", + "esutils": "^2.0.2", + "optionator": "^0.8.1", + "source-map": "~0.2.0" + }, + "dependencies": { + "source-map": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.2.0.tgz", + "integrity": "sha1-2rc/vPwrqBm03gO9b26qSBZLP50=", + "dev": true, + "optional": true, + "requires": { + "amdefine": ">=0.0.4" + } + } + } + }, + "eslint-scope": { + "version": "3.7.1", + "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-3.7.1.tgz", + "integrity": "sha1-PWPD7f2gLgbgGkUq2IyqzHzctug=", + "dev": true, + "requires": { + "esrecurse": "^4.1.0", + "estraverse": "^4.1.1" + }, + "dependencies": { + "estraverse": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-4.2.0.tgz", + "integrity": "sha1-De4/7TH81GlhjOc0IJn8GvoL2xM=", + "dev": true + } + } + }, + "esprima": { + "version": "2.7.3", + "resolved": "https://registry.npmjs.org/esprima/-/esprima-2.7.3.tgz", + "integrity": "sha1-luO3DVd59q1JzQMmc9HDEnZ7pYE=", + "dev": true + }, + "esrecurse": { + "version": "4.2.1", + "resolved": "https://registry.npmjs.org/esrecurse/-/esrecurse-4.2.1.tgz", + "integrity": "sha512-64RBB++fIOAXPw3P9cy89qfMlvZEXZkqqJkjqqXIvzP5ezRZjW+lPWjw35UX/3EhUPFYbg5ER4JYgDw4007/DQ==", + "dev": true, + "requires": { + "estraverse": "^4.1.0" + }, + "dependencies": { + "estraverse": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-4.2.0.tgz", + "integrity": "sha1-De4/7TH81GlhjOc0IJn8GvoL2xM=", + "dev": true + } + } + }, + "estraverse": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-1.9.3.tgz", + "integrity": "sha1-r2fy3JIlgkFZUJJgkaQAXSnJu0Q=", + "dev": true + }, + "esutils": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/esutils/-/esutils-2.0.2.tgz", + "integrity": "sha1-Cr9PHKpbyx96nYrMbepPqqBLrJs=", + "dev": true + }, + "events": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/events/-/events-1.1.1.tgz", + "integrity": "sha1-nr23Y1rQmccNzEwqH1AEKI6L2SQ=", + "dev": true + }, + "evp_bytestokey": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/evp_bytestokey/-/evp_bytestokey-1.0.3.tgz", + "integrity": "sha512-/f2Go4TognH/KvCISP7OUsHn85hT9nUkxxA9BEWxFn+Oj9o8ZNLm/40hdlgSLyuOimsrTKLUMEorQexp/aPQeA==", + "dev": true, + "requires": { + "md5.js": "^1.3.4", + "safe-buffer": "^5.1.1" + } + }, + "execa": { + "version": "0.7.0", + "resolved": "https://registry.npmjs.org/execa/-/execa-0.7.0.tgz", + "integrity": "sha1-lEvs00zEHuMqY6n68nrVpl/Fl3c=", + "dev": true, + "requires": { + "cross-spawn": "^5.0.1", + "get-stream": "^3.0.0", + "is-stream": "^1.1.0", + "npm-run-path": "^2.0.0", + "p-finally": "^1.0.0", + "signal-exit": "^3.0.0", + "strip-eof": "^1.0.0" + }, + "dependencies": { + "cross-spawn": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-5.1.0.tgz", + "integrity": "sha1-6L0O/uWPz/b4+UUQoKVUu/ojVEk=", + "dev": true, + "requires": { + "lru-cache": "^4.0.1", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + } + } + } + }, + "expand-brackets": { + "version": "2.1.4", + "resolved": "https://registry.npmjs.org/expand-brackets/-/expand-brackets-2.1.4.tgz", + "integrity": "sha1-t3c14xXOMPa27/D4OwQVGiJEliI=", + "dev": true, + "requires": { + "debug": "^2.3.3", + "define-property": "^0.2.5", + "extend-shallow": "^2.0.1", + "posix-character-classes": "^0.1.0", + "regex-not": "^1.0.0", + "snapdragon": "^0.8.1", + "to-regex": "^3.0.1" + }, + "dependencies": { + "define-property": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-0.2.5.tgz", + "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=", + "dev": true, + "requires": { + "is-descriptor": "^0.1.0" + } + }, + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + } + } + }, + "extend-shallow": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-3.0.2.tgz", + "integrity": "sha1-Jqcarwc7OfshJxcnRhMcJwQCjbg=", + "dev": true, + "requires": { + "assign-symbols": "^1.0.0", + "is-extendable": "^1.0.1" + }, + "dependencies": { + "is-extendable": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-extendable/-/is-extendable-1.0.1.tgz", + "integrity": "sha512-arnXMxT1hhoKo9k1LZdmlNyJdDDfy2v0fXjFlmok4+i8ul/6WlbVge9bhM74OpNPQPMGUToDtz+KXa1PneJxOA==", + "dev": true, + "requires": { + "is-plain-object": "^2.0.4" + } + } + } + }, + "external-editor": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/external-editor/-/external-editor-3.0.0.tgz", + "integrity": "sha512-mpkfj0FEdxrIhOC04zk85X7StNtr0yXnG7zCb+8ikO8OJi2jsHh5YGoknNTyXgsbHOf1WOOcVU3kPFWT2WgCkQ==", + "dev": true, + "requires": { + "chardet": "^0.5.0", + "iconv-lite": "^0.4.22", + "tmp": "^0.0.33" + } + }, + "extglob": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/extglob/-/extglob-2.0.4.tgz", + "integrity": "sha512-Nmb6QXkELsuBr24CJSkilo6UHHgbekK5UiZgfE6UHD3Eb27YC6oD+bhcT+tJ6cl8dmsgdQxnWlcry8ksBIBLpw==", + "dev": true, + "requires": { + "array-unique": "^0.3.2", + "define-property": "^1.0.0", + "expand-brackets": "^2.1.4", + "extend-shallow": "^2.0.1", + "fragment-cache": "^0.2.1", + "regex-not": "^1.0.0", + "snapdragon": "^0.8.1", + "to-regex": "^3.0.1" + }, + "dependencies": { + "define-property": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", + "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", + "dev": true, + "requires": { + "is-descriptor": "^1.0.0" + } + }, + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + }, + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "^1.0.0", + "is-data-descriptor": "^1.0.0", + "kind-of": "^6.0.2" + } + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "fast-deep-equal": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-2.0.1.tgz", + "integrity": "sha1-ewUhjd+WZ79/Nwv3/bLLFf3Qqkk=", + "dev": true + }, + "fast-json-stable-stringify": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.0.0.tgz", + "integrity": "sha1-1RQsDK7msRifh9OnYREGT4bIu/I=", + "dev": true + }, + "fast-levenshtein": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz", + "integrity": "sha1-PYpcZog6FqMMqGQ+hR8Zuqd5eRc=", + "dev": true + }, + "fastparse": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/fastparse/-/fastparse-1.1.1.tgz", + "integrity": "sha1-0eJkOzipTXWDtHkGDmxK/8lAcfg=", + "dev": true + }, + "figures": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/figures/-/figures-2.0.0.tgz", + "integrity": "sha1-OrGi0qYsi/tDGgyUy3l6L84nyWI=", + "dev": true, + "requires": { + "escape-string-regexp": "^1.0.5" + } + }, + "file-loader": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/file-loader/-/file-loader-1.1.11.tgz", + "integrity": "sha512-TGR4HU7HUsGg6GCOPJnFk06RhWgEWFLAGWiT6rcD+GRC2keU3s9RGJ+b3Z6/U73jwwNb2gKLJ7YCrp+jvU4ALg==", + "dev": true, + "requires": { + "loader-utils": "^1.0.2", + "schema-utils": "^0.4.5" + } + }, + "fill-range": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-4.0.0.tgz", + "integrity": "sha1-1USBHUKPmOsGpj3EAtJAPDKMOPc=", + "dev": true, + "requires": { + "extend-shallow": "^2.0.1", + "is-number": "^3.0.0", + "repeat-string": "^1.6.1", + "to-regex-range": "^2.1.0" + }, + "dependencies": { + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + } + } + }, + "find-cache-dir": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/find-cache-dir/-/find-cache-dir-1.0.0.tgz", + "integrity": "sha1-kojj6ePMN0hxfTnq3hfPcfww7m8=", + "dev": true, + "requires": { + "commondir": "^1.0.1", + "make-dir": "^1.0.0", + "pkg-dir": "^2.0.0" + } + }, + "find-up": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-2.1.0.tgz", + "integrity": "sha1-RdG35QbHF93UgndaK3eSCjwMV6c=", + "dev": true, + "requires": { + "locate-path": "^2.0.0" + } + }, + "flatten": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/flatten/-/flatten-1.0.2.tgz", + "integrity": "sha1-2uRqnXj74lKSJYzB54CkHZXAN4I=", + "dev": true + }, + "flush-write-stream": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/flush-write-stream/-/flush-write-stream-1.0.3.tgz", + "integrity": "sha512-calZMC10u0FMUqoiunI2AiGIIUtUIvifNwkHhNupZH4cbNnW1Itkoh/Nf5HFYmDrwWPjrUxpkZT0KhuCq0jmGw==", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "for-in": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/for-in/-/for-in-1.0.2.tgz", + "integrity": "sha1-gQaNKVqBQuwKxybG4iAMMPttXoA=", + "dev": true + }, + "foreach": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/foreach/-/foreach-2.0.5.tgz", + "integrity": "sha1-C+4AUBiusmDQo6865ljdATbsG5k=", + "dev": true + }, + "fragment-cache": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/fragment-cache/-/fragment-cache-0.2.1.tgz", + "integrity": "sha1-QpD60n8T6Jvn8zeZxrxaCr//DRk=", + "dev": true, + "requires": { + "map-cache": "^0.2.2" + } + }, + "from2": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/from2/-/from2-2.3.0.tgz", + "integrity": "sha1-i/tVAr3kpNNs/e6gB/zKIdfjgq8=", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "fs-extra": { + "version": "0.30.0", + "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-0.30.0.tgz", + "integrity": "sha1-8jP/zAjU2n1DLapEl3aYnbHfk/A=", + "dev": true, + "requires": { + "graceful-fs": "^4.1.2", + "jsonfile": "^2.1.0", + "klaw": "^1.0.0", + "path-is-absolute": "^1.0.0", + "rimraf": "^2.2.8" + } + }, + "fs-write-stream-atomic": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/fs-write-stream-atomic/-/fs-write-stream-atomic-1.0.10.tgz", + "integrity": "sha1-tH31NJPvkR33VzHnCp3tAYnbQMk=", + "dev": true, + "requires": { + "graceful-fs": "^4.1.2", + "iferr": "^0.1.5", + "imurmurhash": "^0.1.4", + "readable-stream": "1 || 2" + } + }, + "fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha1-FQStJSMVjKpA20onh8sBQRmU6k8=", + "dev": true + }, + "fsevents": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-1.2.4.tgz", + "integrity": "sha512-z8H8/diyk76B7q5wg+Ud0+CqzcAF3mBBI/bA5ne5zrRUUIvNkJY//D3BqyH571KuAC4Nr7Rw7CjWX4r0y9DvNg==", + "dev": true, + "optional": true, + "requires": { + "nan": "^2.9.2", + "node-pre-gyp": "^0.10.0" + }, + "dependencies": { + "abbrev": { + "version": "1.1.1", + "bundled": true, + "dev": true, + "optional": true + }, + "ansi-regex": { + "version": "2.1.1", + "bundled": true, + "dev": true + }, + "aproba": { + "version": "1.2.0", + "bundled": true, + "dev": true, + "optional": true + }, + "are-we-there-yet": { + "version": "1.1.4", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "delegates": "^1.0.0", + "readable-stream": "^2.0.6" + } + }, + "balanced-match": { + "version": "1.0.0", + "bundled": true, + "dev": true + }, + "brace-expansion": { + "version": "1.1.11", + "bundled": true, + "dev": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "chownr": { + "version": "1.0.1", + "bundled": true, + "dev": true, + "optional": true + }, + "code-point-at": { + "version": "1.1.0", + "bundled": true, + "dev": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true, + "dev": true + }, + "console-control-strings": { + "version": "1.1.0", + "bundled": true, + "dev": true + }, + "core-util-is": { + "version": "1.0.2", + "bundled": true, + "dev": true, + "optional": true + }, + "debug": { + "version": "2.6.9", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "ms": "2.0.0" + } + }, + "deep-extend": { + "version": "0.5.1", + "bundled": true, + "dev": true, + "optional": true + }, + "delegates": { + "version": "1.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "detect-libc": { + "version": "1.0.3", + "bundled": true, + "dev": true, + "optional": true + }, + "fs-minipass": { + "version": "1.2.5", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "minipass": "^2.2.1" + } + }, + "fs.realpath": { + "version": "1.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "gauge": { + "version": "2.7.4", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "aproba": "^1.0.3", + "console-control-strings": "^1.0.0", + "has-unicode": "^2.0.0", + "object-assign": "^4.1.0", + "signal-exit": "^3.0.0", + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1", + "wide-align": "^1.1.0" + } + }, + "glob": { + "version": "7.1.2", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + }, + "has-unicode": { + "version": "2.0.1", + "bundled": true, + "dev": true, + "optional": true + }, + "iconv-lite": { + "version": "0.4.21", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "safer-buffer": "^2.1.0" + } + }, + "ignore-walk": { + "version": "3.0.1", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "minimatch": "^3.0.4" + } + }, + "inflight": { + "version": "1.0.6", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "inherits": { + "version": "2.0.3", + "bundled": true, + "dev": true + }, + "ini": { + "version": "1.3.5", + "bundled": true, + "dev": true, + "optional": true + }, + "is-fullwidth-code-point": { + "version": "1.0.0", + "bundled": true, + "dev": true, + "requires": { + "number-is-nan": "^1.0.0" + } + }, + "isarray": { + "version": "1.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "minimatch": { + "version": "3.0.4", + "bundled": true, + "dev": true, + "requires": { + "brace-expansion": "^1.1.7" + } + }, + "minimist": { + "version": "0.0.8", + "bundled": true, + "dev": true + }, + "minipass": { + "version": "2.2.4", + "bundled": true, + "dev": true, + "requires": { + "safe-buffer": "^5.1.1", + "yallist": "^3.0.0" + } + }, + "minizlib": { + "version": "1.1.0", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "minipass": "^2.2.1" + } + }, + "mkdirp": { + "version": "0.5.1", + "bundled": true, + "dev": true, + "requires": { + "minimist": "0.0.8" + } + }, + "ms": { + "version": "2.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "needle": { + "version": "2.2.0", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "debug": "^2.1.2", + "iconv-lite": "^0.4.4", + "sax": "^1.2.4" + } + }, + "node-pre-gyp": { + "version": "0.10.0", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "detect-libc": "^1.0.2", + "mkdirp": "^0.5.1", + "needle": "^2.2.0", + "nopt": "^4.0.1", + "npm-packlist": "^1.1.6", + "npmlog": "^4.0.2", + "rc": "^1.1.7", + "rimraf": "^2.6.1", + "semver": "^5.3.0", + "tar": "^4" + } + }, + "nopt": { + "version": "4.0.1", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "abbrev": "1", + "osenv": "^0.1.4" + } + }, + "npm-bundled": { + "version": "1.0.3", + "bundled": true, + "dev": true, + "optional": true + }, + "npm-packlist": { + "version": "1.1.10", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "ignore-walk": "^3.0.1", + "npm-bundled": "^1.0.1" + } + }, + "npmlog": { + "version": "4.1.2", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "are-we-there-yet": "~1.1.2", + "console-control-strings": "~1.1.0", + "gauge": "~2.7.3", + "set-blocking": "~2.0.0" + } + }, + "number-is-nan": { + "version": "1.0.1", + "bundled": true, + "dev": true + }, + "object-assign": { + "version": "4.1.1", + "bundled": true, + "dev": true, + "optional": true + }, + "once": { + "version": "1.4.0", + "bundled": true, + "dev": true, + "requires": { + "wrappy": "1" + } + }, + "os-homedir": { + "version": "1.0.2", + "bundled": true, + "dev": true, + "optional": true + }, + "os-tmpdir": { + "version": "1.0.2", + "bundled": true, + "dev": true, + "optional": true + }, + "osenv": { + "version": "0.1.5", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "os-homedir": "^1.0.0", + "os-tmpdir": "^1.0.0" + } + }, + "path-is-absolute": { + "version": "1.0.1", + "bundled": true, + "dev": true, + "optional": true + }, + "process-nextick-args": { + "version": "2.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "rc": { + "version": "1.2.7", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "deep-extend": "^0.5.1", + "ini": "~1.3.0", + "minimist": "^1.2.0", + "strip-json-comments": "~2.0.1" + }, + "dependencies": { + "minimist": { + "version": "1.2.0", + "bundled": true, + "dev": true, + "optional": true + } + } + }, + "readable-stream": { + "version": "2.3.6", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "rimraf": { + "version": "2.6.2", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "glob": "^7.0.5" + } + }, + "safe-buffer": { + "version": "5.1.1", + "bundled": true, + "dev": true + }, + "safer-buffer": { + "version": "2.1.2", + "bundled": true, + "dev": true, + "optional": true + }, + "sax": { + "version": "1.2.4", + "bundled": true, + "dev": true, + "optional": true + }, + "semver": { + "version": "5.5.0", + "bundled": true, + "dev": true, + "optional": true + }, + "set-blocking": { + "version": "2.0.0", + "bundled": true, + "dev": true, + "optional": true + }, + "signal-exit": { + "version": "3.0.2", + "bundled": true, + "dev": true, + "optional": true + }, + "string-width": { + "version": "1.0.2", + "bundled": true, + "dev": true, + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + } + }, + "string_decoder": { + "version": "1.1.1", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "safe-buffer": "~5.1.0" + } + }, + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "dev": true, + "requires": { + "ansi-regex": "^2.0.0" + } + }, + "strip-json-comments": { + "version": "2.0.1", + "bundled": true, + "dev": true, + "optional": true + }, + "tar": { + "version": "4.4.1", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "chownr": "^1.0.1", + "fs-minipass": "^1.2.5", + "minipass": "^2.2.4", + "minizlib": "^1.1.0", + "mkdirp": "^0.5.0", + "safe-buffer": "^5.1.1", + "yallist": "^3.0.2" + } + }, + "util-deprecate": { + "version": "1.0.2", + "bundled": true, + "dev": true, + "optional": true + }, + "wide-align": { + "version": "1.1.2", + "bundled": true, + "dev": true, + "optional": true, + "requires": { + "string-width": "^1.0.2" + } + }, + "wrappy": { + "version": "1.0.2", + "bundled": true, + "dev": true + }, + "yallist": { + "version": "3.0.2", + "bundled": true, + "dev": true + } + } + }, + "function-bind": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz", + "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==", + "dev": true + }, + "generic-names": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/generic-names/-/generic-names-1.0.3.tgz", + "integrity": "sha1-LXhqEhruUIh2eWk56OO/+DbCCRc=", + "dev": true, + "requires": { + "loader-utils": "^0.2.16" + }, + "dependencies": { + "loader-utils": { + "version": "0.2.17", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-0.2.17.tgz", + "integrity": "sha1-+G5jdNQyBabmxg6RlvF8Apm/s0g=", + "dev": true, + "requires": { + "big.js": "^3.1.3", + "emojis-list": "^2.0.0", + "json5": "^0.5.0", + "object-assign": "^4.0.1" + } + } + } + }, + "get-caller-file": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-1.0.2.tgz", + "integrity": "sha1-9wLmMSfn4jHBYKgMFVSstw1QR+U=", + "dev": true + }, + "get-stream": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-3.0.0.tgz", + "integrity": "sha1-jpQ9E1jcN1VQVOy+LtsFqhdO3hQ=", + "dev": true + }, + "get-value": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/get-value/-/get-value-2.0.6.tgz", + "integrity": "sha1-3BXKHGcjh8p2vTesCjlbogQqLCg=", + "dev": true + }, + "glob": { + "version": "5.0.15", + "resolved": "https://registry.npmjs.org/glob/-/glob-5.0.15.tgz", + "integrity": "sha1-G8k2ueAvSmA/zCIuz3Yz0wuLk7E=", + "dev": true, + "requires": { + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "2 || 3", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + }, + "glob-parent": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-3.1.0.tgz", + "integrity": "sha1-nmr2KZ2NO9K9QEMIMr0RPfkGxa4=", + "dev": true, + "requires": { + "is-glob": "^3.1.0", + "path-dirname": "^1.0.0" + }, + "dependencies": { + "is-glob": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-3.1.0.tgz", + "integrity": "sha1-e6WuJCF4BKxwcHuWkiVnSGzD6Eo=", + "dev": true, + "requires": { + "is-extglob": "^2.1.0" + } + } + } + }, + "glob-to-regexp": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/glob-to-regexp/-/glob-to-regexp-0.3.0.tgz", + "integrity": "sha1-jFoUlNIGbFcMw7/kSWF1rMTVAqs=", + "dev": true + }, + "global-modules-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/global-modules-path/-/global-modules-path-2.1.0.tgz", + "integrity": "sha512-3DrmGj2TP+96cABk9TfMp6f3knH/Y46dqvWznTU3Tf6/bDGLDAn15tFluQ7BcloykOcdY16U0WGq0BQblYOxJQ==", + "dev": true + }, + "globals": { + "version": "9.18.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-9.18.0.tgz", + "integrity": "sha512-S0nG3CLEQiY/ILxqtztTWH/3iRRdyBLw6KMDxnKMchrtbj2OFmehVh0WUCfW3DUrIgx/qFrJPICrq4Z4sTR9UQ==", + "dev": true + }, + "globby": { + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/globby/-/globby-7.1.1.tgz", + "integrity": "sha1-+yzP+UAfhgCUXfral0QMypcrhoA=", + "dev": true, + "requires": { + "array-union": "^1.0.1", + "dir-glob": "^2.0.0", + "glob": "^7.1.2", + "ignore": "^3.3.5", + "pify": "^3.0.0", + "slash": "^1.0.0" + }, + "dependencies": { + "glob": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.2.tgz", + "integrity": "sha512-MJTUg1kjuLeQCJ+ccE4Vpa6kKVXkPYJ2mOCQyUuKLcLQsdrMCpBPUi8qVE6+YuaJkozeA9NusTAw3hLr8Xe5EQ==", + "dev": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + } + } + }, + "graceful-fs": { + "version": "4.1.11", + "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.1.11.tgz", + "integrity": "sha1-Dovf5NHduIVNZOBOp8AOKgJuVlg=", + "dev": true + }, + "handlebars": { + "version": "4.0.11", + "resolved": "https://registry.npmjs.org/handlebars/-/handlebars-4.0.11.tgz", + "integrity": "sha1-Ywo13+ApS8KB7a5v/F0yn8eYLcw=", + "dev": true, + "requires": { + "async": "^1.4.0", + "optimist": "^0.6.1", + "source-map": "^0.4.4", + "uglify-js": "^2.6" + }, + "dependencies": { + "source-map": { + "version": "0.4.4", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.4.4.tgz", + "integrity": "sha1-66T12pwNyZneaAMti092FzZSA2s=", + "dev": true, + "requires": { + "amdefine": ">=0.0.4" + } + } + } + }, + "has": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz", + "integrity": "sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==", + "dev": true, + "requires": { + "function-bind": "^1.1.1" + } + }, + "has-ansi": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/has-ansi/-/has-ansi-2.0.0.tgz", + "integrity": "sha1-NPUEnOHs3ysGSa8+8k5F7TVBbZE=", + "dev": true, + "requires": { + "ansi-regex": "^2.0.0" + } + }, + "has-flag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-1.0.0.tgz", + "integrity": "sha1-nZ55MWXOAXoA8AQYxD+UKnsdEfo=", + "dev": true + }, + "has-symbols": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.0.tgz", + "integrity": "sha1-uhqPGvKg/DllD1yFA2dwQSIGO0Q=", + "dev": true + }, + "has-value": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-value/-/has-value-1.0.0.tgz", + "integrity": "sha1-GLKB2lhbHFxR3vJMkw7SmgvmsXc=", + "dev": true, + "requires": { + "get-value": "^2.0.6", + "has-values": "^1.0.0", + "isobject": "^3.0.0" + } + }, + "has-values": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/has-values/-/has-values-1.0.0.tgz", + "integrity": "sha1-lbC2P+whRmGab+V/51Yo1aOe/k8=", + "dev": true, + "requires": { + "is-number": "^3.0.0", + "kind-of": "^4.0.0" + }, + "dependencies": { + "kind-of": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-4.0.0.tgz", + "integrity": "sha1-IIE989cSkosgc3hpGkUGb65y3Vc=", + "dev": true, + "requires": { + "is-buffer": "^1.1.5" + } + } + } + }, + "hash-base": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-3.0.4.tgz", + "integrity": "sha1-X8hoaEfs1zSZQDMZprCj8/auSRg=", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "safe-buffer": "^5.0.1" + } + }, + "hash.js": { + "version": "1.1.5", + "resolved": "https://registry.npmjs.org/hash.js/-/hash.js-1.1.5.tgz", + "integrity": "sha512-eWI5HG9Np+eHV1KQhisXWwM+4EPPYe5dFX1UZZH7k/E3JzDEazVH+VGlZi6R94ZqImq+A3D1mCEtrFIfg/E7sA==", + "dev": true, + "requires": { + "inherits": "^2.0.3", + "minimalistic-assert": "^1.0.1" + } + }, + "hmac-drbg": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/hmac-drbg/-/hmac-drbg-1.0.1.tgz", + "integrity": "sha1-0nRXAQJabHdabFRXk+1QL8DGSaE=", + "dev": true, + "requires": { + "hash.js": "^1.0.3", + "minimalistic-assert": "^1.0.0", + "minimalistic-crypto-utils": "^1.0.1" + } + }, + "home-or-tmp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/home-or-tmp/-/home-or-tmp-2.0.0.tgz", + "integrity": "sha1-42w/LSyufXRqhX440Y1fMqeILbg=", + "dev": true, + "requires": { + "os-homedir": "^1.0.0", + "os-tmpdir": "^1.0.1" + } + }, + "html-comment-regex": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/html-comment-regex/-/html-comment-regex-1.1.1.tgz", + "integrity": "sha1-ZouTd26q5V696POtRkswekljYl4=", + "dev": true + }, + "https-browserify": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/https-browserify/-/https-browserify-1.0.0.tgz", + "integrity": "sha1-7AbBDgo0wPL68Zn3/X/Hj//QPHM=", + "dev": true + }, + "iconv-lite": { + "version": "0.4.23", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.23.tgz", + "integrity": "sha512-neyTUVFtahjf0mB3dZT77u+8O0QB89jFdnBkd5P1JgYPbPaia3gXXOVL2fq8VyU2gMMD7SaN7QukTB/pmXYvDA==", + "dev": true, + "requires": { + "safer-buffer": ">= 2.1.2 < 3" + } + }, + "icss-replace-symbols": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/icss-replace-symbols/-/icss-replace-symbols-1.1.0.tgz", + "integrity": "sha1-Bupvg2ead0njhs/h/oEq5dsiPe0=", + "dev": true + }, + "icss-utils": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/icss-utils/-/icss-utils-3.0.1.tgz", + "integrity": "sha1-7nDTroysOMa+XtkehRsn7tNDrQ8=", + "dev": true, + "requires": { + "postcss": "^6.0.2" + } + }, + "ieee754": { + "version": "1.1.12", + "resolved": "https://registry.npmjs.org/ieee754/-/ieee754-1.1.12.tgz", + "integrity": "sha512-GguP+DRY+pJ3soyIiGPTvdiVXjZ+DbXOxGpXn3eMvNW4x4irjqXm4wHKscC+TfxSJ0yw/S1F24tqdMNsMZTiLA==", + "dev": true + }, + "iferr": { + "version": "0.1.5", + "resolved": "https://registry.npmjs.org/iferr/-/iferr-0.1.5.tgz", + "integrity": "sha1-xg7taebY/bazEEofy8ocGS3FtQE=", + "dev": true + }, + "ignore": { + "version": "3.3.10", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-3.3.10.tgz", + "integrity": "sha512-Pgs951kaMm5GXP7MOvxERINe3gsaVjUWFm+UZPSq9xYriQAksyhg0csnS0KXSNRD5NmNdapXEpjxG49+AKh/ug==", + "dev": true + }, + "import-local": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/import-local/-/import-local-1.0.0.tgz", + "integrity": "sha512-vAaZHieK9qjGo58agRBg+bhHX3hoTZU/Oa3GESWLz7t1U62fk63aHuDJJEteXoDeTCcPmUT+z38gkHPZkkmpmQ==", + "dev": true, + "requires": { + "pkg-dir": "^2.0.0", + "resolve-cwd": "^2.0.0" + } + }, + "imports-loader": { + "version": "0.8.0", + "resolved": "https://registry.npmjs.org/imports-loader/-/imports-loader-0.8.0.tgz", + "integrity": "sha512-kXWL7Scp8KQ4552ZcdVTeaQCZSLW+e6nJfp3cwUMB673T7Hr98Xjx5JK+ql7ADlJUvj1JS5O01RLbKoutN5QDQ==", + "dev": true, + "requires": { + "loader-utils": "^1.0.2", + "source-map": "^0.6.1" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true + } + } + }, + "imurmurhash": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", + "integrity": "sha1-khi5srkoojixPcT7a21XbyMUU+o=", + "dev": true + }, + "indexes-of": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/indexes-of/-/indexes-of-1.0.1.tgz", + "integrity": "sha1-8w9xbI4r00bHtn0985FVZqfAVgc=", + "dev": true + }, + "indexof": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/indexof/-/indexof-0.0.1.tgz", + "integrity": "sha1-gtwzbSMrkGIXnQWrMpOmYFn9Q10=", + "dev": true + }, + "inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha1-Sb1jMdfQLQwJvJEKEHW6gWW1bfk=", + "dev": true, + "requires": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "inherits": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.3.tgz", + "integrity": "sha1-Yzwsg+PaQqUC9SRmAiSA9CCCYd4=", + "dev": true + }, + "inquirer": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/inquirer/-/inquirer-6.0.0.tgz", + "integrity": "sha512-tISQWRwtcAgrz+SHPhTH7d3e73k31gsOy6i1csonLc0u1dVK/wYvuOnFeiWqC5OXFIYbmrIFInef31wbT8MEJg==", + "dev": true, + "requires": { + "ansi-escapes": "^3.0.0", + "chalk": "^2.0.0", + "cli-cursor": "^2.1.0", + "cli-width": "^2.0.0", + "external-editor": "^3.0.0", + "figures": "^2.0.0", + "lodash": "^4.3.0", + "mute-stream": "0.0.7", + "run-async": "^2.2.0", + "rxjs": "^6.1.0", + "string-width": "^2.1.0", + "strip-ansi": "^4.0.0", + "through": "^2.3.6" + }, + "dependencies": { + "ansi-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-3.0.0.tgz", + "integrity": "sha1-7QMXwyIGT3lGbAKWa922Bas32Zg=", + "dev": true + }, + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "strip-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-4.0.0.tgz", + "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=", + "dev": true, + "requires": { + "ansi-regex": "^3.0.0" + } + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "interpret": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/interpret/-/interpret-1.1.0.tgz", + "integrity": "sha1-ftGxQQxqDg94z5XTuEQMY/eLhhQ=", + "dev": true + }, + "invariant": { + "version": "2.2.4", + "resolved": "https://registry.npmjs.org/invariant/-/invariant-2.2.4.tgz", + "integrity": "sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA==", + "dev": true, + "requires": { + "loose-envify": "^1.0.0" + } + }, + "invert-kv": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/invert-kv/-/invert-kv-1.0.0.tgz", + "integrity": "sha1-EEqOSqym09jNFXqO+L+rLXo//bY=", + "dev": true + }, + "is-absolute-url": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-absolute-url/-/is-absolute-url-2.1.0.tgz", + "integrity": "sha1-UFMN+4T8yap9vnhS6Do3uTufKqY=", + "dev": true + }, + "is-accessor-descriptor": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", + "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "dev": true, + "requires": { + "kind-of": "^3.0.2" + } + }, + "is-binary-path": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-1.0.1.tgz", + "integrity": "sha1-dfFmQrSA8YenEcgUFh/TpKdlWJg=", + "dev": true, + "requires": { + "binary-extensions": "^1.0.0" + } + }, + "is-buffer": { + "version": "1.1.6", + "resolved": "https://registry.npmjs.org/is-buffer/-/is-buffer-1.1.6.tgz", + "integrity": "sha512-NcdALwpXkTm5Zvvbk7owOUSvVvBKDgKP5/ewfXEznmQFfs4ZRmanOeKBTjRVjka3QFoN6XJ+9F3USqfHqTaU5w==", + "dev": true + }, + "is-data-descriptor": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", + "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "dev": true, + "requires": { + "kind-of": "^3.0.2" + } + }, + "is-descriptor": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", + "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "^0.1.6", + "is-data-descriptor": "^0.1.4", + "kind-of": "^5.0.0" + }, + "dependencies": { + "kind-of": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", + "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==", + "dev": true + } + } + }, + "is-extendable": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/is-extendable/-/is-extendable-0.1.1.tgz", + "integrity": "sha1-YrEQ4omkcUGOPsNqYX1HLjAd/Ik=", + "dev": true + }, + "is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha1-qIwCU1eR8C7TfHahueqXc8gz+MI=", + "dev": true + }, + "is-finite": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-finite/-/is-finite-1.0.2.tgz", + "integrity": "sha1-zGZ3aVYCvlUO8R6LSqYwU0K20Ko=", + "dev": true, + "requires": { + "number-is-nan": "^1.0.0" + } + }, + "is-fullwidth-code-point": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-2.0.0.tgz", + "integrity": "sha1-o7MKXE8ZkYMWeqq5O+764937ZU8=", + "dev": true + }, + "is-glob": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.0.tgz", + "integrity": "sha1-lSHHaEXMJhCoUgPd8ICpWML/q8A=", + "dev": true, + "requires": { + "is-extglob": "^2.1.1" + } + }, + "is-number": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-3.0.0.tgz", + "integrity": "sha1-JP1iAaR4LPUFYcgQJ2r8fRLXEZU=", + "dev": true, + "requires": { + "kind-of": "^3.0.2" + } + }, + "is-plain-obj": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-1.1.0.tgz", + "integrity": "sha1-caUMhCnfync8kqOQpKA7OfzVHT4=", + "dev": true + }, + "is-plain-object": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-2.0.4.tgz", + "integrity": "sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og==", + "dev": true, + "requires": { + "isobject": "^3.0.1" + } + }, + "is-promise": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-promise/-/is-promise-2.1.0.tgz", + "integrity": "sha1-eaKp7OfwlugPNtKy87wWwf9L8/o=", + "dev": true + }, + "is-stream": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-1.1.0.tgz", + "integrity": "sha1-EtSj3U5o4Lec6428hBc66A2RykQ=", + "dev": true + }, + "is-svg": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-svg/-/is-svg-2.1.0.tgz", + "integrity": "sha1-z2EJDaDZ77yrhyLeum8DIgjbsOk=", + "dev": true, + "requires": { + "html-comment-regex": "^1.1.0" + } + }, + "is-windows": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-windows/-/is-windows-1.0.2.tgz", + "integrity": "sha512-eXK1UInq2bPmjyX6e3VHIzMLobc4J94i4AWn+Hpq3OU5KkrRC96OAcR3PRJ/pGu6m8TRnBHP9dkXQVsT/COVIA==", + "dev": true + }, + "isarray": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-1.0.0.tgz", + "integrity": "sha1-u5NdSFgsuhaMBoNJV6VKPgcSTxE=", + "dev": true + }, + "isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha1-6PvzdNxVb/iUehDcsFctYz8s+hA=", + "dev": true + }, + "isobject": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", + "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=", + "dev": true + }, + "jquery": { + "version": "3.3.1", + "resolved": "https://registry.npmjs.org/jquery/-/jquery-3.3.1.tgz", + "integrity": "sha512-Ubldcmxp5np52/ENotGxlLe6aGMvmF4R8S6tZjsP6Knsaxd/xp3Zrh50cG93lR6nPXyUFwzN3ZSOQI0wRJNdGg==" + }, + "js-base64": { + "version": "2.4.5", + "resolved": "https://registry.npmjs.org/js-base64/-/js-base64-2.4.5.tgz", + "integrity": "sha512-aUnNwqMOXw3yvErjMPSQu6qIIzUmT1e5KcU1OZxRDU1g/am6mzBvcrmLAYwzmB59BHPrh5/tKaiF4OPhqRWESQ==", + "dev": true + }, + "js-tokens": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/js-tokens/-/js-tokens-3.0.2.tgz", + "integrity": "sha1-mGbfOVECEw449/mWvOtlRDIJwls=", + "dev": true + }, + "js-yaml": { + "version": "3.12.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.12.0.tgz", + "integrity": "sha512-PIt2cnwmPfL4hKNwqeiuz4bKfnzHTBv6HyVgjahA6mPLwPDzjDWrplJBMjHUFxku/N3FlmrbyPclad+I+4mJ3A==", + "dev": true, + "requires": { + "argparse": "^1.0.7", + "esprima": "^4.0.0" + }, + "dependencies": { + "esprima": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/esprima/-/esprima-4.0.0.tgz", + "integrity": "sha512-oftTcaMu/EGrEIu904mWteKIv8vMuOgGYo7EhVJJN00R/EED9DCua/xxHRdYnKtcECzVg7xOWhflvJMnqcFZjw==", + "dev": true + } + } + }, + "jsesc": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-1.3.0.tgz", + "integrity": "sha1-RsP+yMGJKxKwgz25vHYiF226s0s=", + "dev": true + }, + "json-parse-better-errors": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/json-parse-better-errors/-/json-parse-better-errors-1.0.2.tgz", + "integrity": "sha512-mrqyZKfX5EhL7hvqcV6WG1yYjnjeuYDzDhhcAAUrq8Po85NBQBJP+ZDUT75qZQ98IkUoBqdkExkukOU7Ts2wrw==", + "dev": true + }, + "json-schema-traverse": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz", + "integrity": "sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg==", + "dev": true + }, + "json5": { + "version": "0.5.1", + "resolved": "https://registry.npmjs.org/json5/-/json5-0.5.1.tgz", + "integrity": "sha1-Hq3nrMASA0rYTiOWdn6tn6VJWCE=", + "dev": true + }, + "jsonfile": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-2.4.0.tgz", + "integrity": "sha1-NzaitCi4e72gzIO1P6PWM6NcKug=", + "dev": true, + "requires": { + "graceful-fs": "^4.1.6" + } + }, + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "dev": true, + "requires": { + "is-buffer": "^1.1.5" + } + }, + "klaw": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/klaw/-/klaw-1.3.1.tgz", + "integrity": "sha1-QIhDO0azsbolnXh4XY6W9zugJDk=", + "dev": true, + "requires": { + "graceful-fs": "^4.1.9" + } + }, + "lazy-cache": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/lazy-cache/-/lazy-cache-1.0.4.tgz", + "integrity": "sha1-odePw6UEdMuAhF07O24dpJpEbo4=", + "dev": true, + "optional": true + }, + "lcid": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/lcid/-/lcid-1.0.0.tgz", + "integrity": "sha1-MIrMr6C8SDo4Z7S28rlQYlHRuDU=", + "dev": true, + "requires": { + "invert-kv": "^1.0.0" + } + }, + "levn": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/levn/-/levn-0.3.0.tgz", + "integrity": "sha1-OwmSTt+fCDwEkP3UwLxEIeBHZO4=", + "dev": true, + "requires": { + "prelude-ls": "~1.1.2", + "type-check": "~0.3.2" + } + }, + "loader-runner": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/loader-runner/-/loader-runner-2.3.0.tgz", + "integrity": "sha1-9IKuqC1UPgeSFwDVpG7yb9rGuKI=", + "dev": true + }, + "loader-utils": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-1.1.0.tgz", + "integrity": "sha1-yYrvSIvM7aL/teLeZG1qdUQp9c0=", + "dev": true, + "requires": { + "big.js": "^3.1.3", + "emojis-list": "^2.0.0", + "json5": "^0.5.0" + } + }, + "locate-path": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-2.0.0.tgz", + "integrity": "sha1-K1aLJl7slExtnA3pw9u7ygNUzY4=", + "dev": true, + "requires": { + "p-locate": "^2.0.0", + "path-exists": "^3.0.0" + } + }, + "lodash": { + "version": "4.17.10", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.10.tgz", + "integrity": "sha512-UejweD1pDoXu+AD825lWwp4ZGtSwgnpZxb3JDViD7StjQz+Nb/6l093lx4OQ0foGWNRoc19mWy7BzL+UAK2iVg==", + "dev": true + }, + "lodash.camelcase": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/lodash.camelcase/-/lodash.camelcase-4.3.0.tgz", + "integrity": "sha1-soqmKIorn8ZRA1x3EfZathkDMaY=", + "dev": true + }, + "lodash.debounce": { + "version": "4.0.8", + "resolved": "https://registry.npmjs.org/lodash.debounce/-/lodash.debounce-4.0.8.tgz", + "integrity": "sha1-gteb/zCmfEAF/9XiUVMArZyk168=", + "dev": true + }, + "lodash.memoize": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/lodash.memoize/-/lodash.memoize-4.1.2.tgz", + "integrity": "sha1-vMbEmkKihA7Zl/Mj6tpezRguC/4=", + "dev": true + }, + "lodash.uniq": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/lodash.uniq/-/lodash.uniq-4.5.0.tgz", + "integrity": "sha1-0CJTc662Uq3BvILklFM5qEJ1R3M=", + "dev": true + }, + "log-symbols": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-2.2.0.tgz", + "integrity": "sha512-VeIAFslyIerEJLXHziedo2basKbMKtTw3vfn5IzG0XTjhAVEJyNHnL2p7vc+wBDSdQuUpNw3M2u6xb9QsAY5Eg==", + "dev": true, + "requires": { + "chalk": "^2.0.1" + }, + "dependencies": { + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "loglevelnext": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/loglevelnext/-/loglevelnext-1.0.5.tgz", + "integrity": "sha512-V/73qkPuJmx4BcBF19xPBr+0ZRVBhc4POxvZTZdMeXpJ4NItXSJ/MSwuFT0kQJlCbXvdlZoQQ/418bS1y9Jh6A==", + "dev": true, + "requires": { + "es6-symbol": "^3.1.1", + "object.assign": "^4.1.0" + } + }, + "long": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/long/-/long-3.2.0.tgz", + "integrity": "sha1-2CG3E4yhy1gcFymQ7xTbIAtcR0s=", + "dev": true + }, + "longest": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/longest/-/longest-1.0.1.tgz", + "integrity": "sha1-MKCy2jj3N3DoKUoNIuZiXtd9AJc=", + "dev": true + }, + "loose-envify": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/loose-envify/-/loose-envify-1.3.1.tgz", + "integrity": "sha1-0aitM/qc4OcT1l/dCsi3SNR4yEg=", + "dev": true, + "requires": { + "js-tokens": "^3.0.0" + } + }, + "lru-cache": { + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-4.1.3.tgz", + "integrity": "sha512-fFEhvcgzuIoJVUF8fYr5KR0YqxD238zgObTps31YdADwPPAp82a4M8TrckkWyx7ekNlf9aBcVn81cFwwXngrJA==", + "dev": true, + "requires": { + "pseudomap": "^1.0.2", + "yallist": "^2.1.2" + } + }, + "make-dir": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-1.3.0.tgz", + "integrity": "sha512-2w31R7SJtieJJnQtGc7RVL2StM2vGYVfqUOvUDxH6bC6aJTxPxTF0GnIgCyu7tjockiUWAYQRbxa7vKn34s5sQ==", + "dev": true, + "requires": { + "pify": "^3.0.0" + } + }, + "mamacro": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/mamacro/-/mamacro-0.0.3.tgz", + "integrity": "sha512-qMEwh+UujcQ+kbz3T6V+wAmO2U8veoq2w+3wY8MquqwVA3jChfwY+Tk52GZKDfACEPjuZ7r2oJLejwpt8jtwTA==", + "dev": true + }, + "map-cache": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/map-cache/-/map-cache-0.2.2.tgz", + "integrity": "sha1-wyq9C9ZSXZsFFkW7TyasXcmKDb8=", + "dev": true + }, + "map-visit": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/map-visit/-/map-visit-1.0.0.tgz", + "integrity": "sha1-7Nyo8TFE5mDxtb1B8S80edmN+48=", + "dev": true, + "requires": { + "object-visit": "^1.0.0" + } + }, + "math-expression-evaluator": { + "version": "1.2.17", + "resolved": "https://registry.npmjs.org/math-expression-evaluator/-/math-expression-evaluator-1.2.17.tgz", + "integrity": "sha1-3oGf282E3M2PrlnGrreWFbnSZqw=", + "dev": true + }, + "md5.js": { + "version": "1.3.4", + "resolved": "https://registry.npmjs.org/md5.js/-/md5.js-1.3.4.tgz", + "integrity": "sha1-6b296UogpawYsENA/Fdk1bCdkB0=", + "dev": true, + "requires": { + "hash-base": "^3.0.0", + "inherits": "^2.0.1" + } + }, + "mem": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/mem/-/mem-1.1.0.tgz", + "integrity": "sha1-Xt1StIXKHZAP5kiVUFOZoN+kX3Y=", + "dev": true, + "requires": { + "mimic-fn": "^1.0.0" + } + }, + "memory-fs": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/memory-fs/-/memory-fs-0.4.1.tgz", + "integrity": "sha1-OpoguEYlI+RHz7x+i7gO1me/xVI=", + "dev": true, + "requires": { + "errno": "^0.1.3", + "readable-stream": "^2.0.1" + } + }, + "micromatch": { + "version": "3.1.10", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-3.1.10.tgz", + "integrity": "sha512-MWikgl9n9M3w+bpsY3He8L+w9eF9338xRl8IAO5viDizwSzziFEyUzo2xrrloB64ADbTf8uA8vRqqttDTOmccg==", + "dev": true, + "requires": { + "arr-diff": "^4.0.0", + "array-unique": "^0.3.2", + "braces": "^2.3.1", + "define-property": "^2.0.2", + "extend-shallow": "^3.0.2", + "extglob": "^2.0.4", + "fragment-cache": "^0.2.1", + "kind-of": "^6.0.2", + "nanomatch": "^1.2.9", + "object.pick": "^1.3.0", + "regex-not": "^1.0.0", + "snapdragon": "^0.8.1", + "to-regex": "^3.0.2" + }, + "dependencies": { + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "miller-rabin": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/miller-rabin/-/miller-rabin-4.0.1.tgz", + "integrity": "sha512-115fLhvZVqWwHPbClyntxEVfVDfl9DLLTuJvq3g2O/Oxi8AiNouAHvDSzHS0viUJc+V5vm3eq91Xwqn9dp4jRA==", + "dev": true, + "requires": { + "bn.js": "^4.0.0", + "brorand": "^1.0.1" + } + }, + "mime": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/mime/-/mime-2.3.1.tgz", + "integrity": "sha512-OEUllcVoydBHGN1z84yfQDimn58pZNNNXgZlHXSboxMlFvgI6MXSWpWKpFRra7H1HxpVhHTkrghfRW49k6yjeg==", + "dev": true + }, + "mimic-fn": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-1.2.0.tgz", + "integrity": "sha512-jf84uxzwiuiIVKiOLpfYk7N46TSy8ubTonmneY9vrpHNAnp0QBt2BxWV9dO3/j+BoVAb+a5G6YDPW3M5HOdMWQ==", + "dev": true + }, + "mini-css-extract-plugin": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/mini-css-extract-plugin/-/mini-css-extract-plugin-0.4.1.tgz", + "integrity": "sha512-XWuB3G61Rtasq/gLe7cp5cuozehE6hN+E4sxCamRR/WDiHTg+f7ZIAS024r8UJQffY+e2gGELXQZgQoFDfNDCg==", + "dev": true, + "requires": { + "@webpack-contrib/schema-utils": "^1.0.0-beta.0", + "loader-utils": "^1.1.0", + "webpack-sources": "^1.1.0" + } + }, + "minimalistic-assert": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/minimalistic-assert/-/minimalistic-assert-1.0.1.tgz", + "integrity": "sha512-UtJcAD4yEaGtjPezWuO9wC4nwUnVH/8/Im3yEHQP4b67cXlD/Qr9hdITCU1xDbSEXg2XKNaP8jsReV7vQd00/A==", + "dev": true + }, + "minimalistic-crypto-utils": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/minimalistic-crypto-utils/-/minimalistic-crypto-utils-1.0.1.tgz", + "integrity": "sha1-9sAMHAsIIkblxNmd+4x8CDsrWCo=", + "dev": true + }, + "minimatch": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.4.tgz", + "integrity": "sha512-yJHVQEhyqPLUTgt9B83PXu6W3rx4MvvHvSUvToogpwoGDOUQ+yDrR0HRot+yOCdCO7u4hX3pWft6kWBBcqh0UA==", + "dev": true, + "requires": { + "brace-expansion": "^1.1.7" + } + }, + "minimist": { + "version": "0.0.8", + "resolved": "https://registry.npmjs.org/minimist/-/minimist-0.0.8.tgz", + "integrity": "sha1-hX/Kv8M5fSYluCKCYuhqp6ARsF0=", + "dev": true + }, + "mississippi": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/mississippi/-/mississippi-2.0.0.tgz", + "integrity": "sha512-zHo8v+otD1J10j/tC+VNoGK9keCuByhKovAvdn74dmxJl9+mWHnx6EMsDN4lgRoMI/eYo2nchAxniIbUPb5onw==", + "dev": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^2.0.1", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + } + }, + "mixin-deep": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/mixin-deep/-/mixin-deep-1.3.1.tgz", + "integrity": "sha512-8ZItLHeEgaqEvd5lYBXfm4EZSFCX29Jb9K+lAHhDKzReKBQKj3R+7NOF6tjqYi9t4oI8VUfaWITJQm86wnXGNQ==", + "dev": true, + "requires": { + "for-in": "^1.0.2", + "is-extendable": "^1.0.1" + }, + "dependencies": { + "is-extendable": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-extendable/-/is-extendable-1.0.1.tgz", + "integrity": "sha512-arnXMxT1hhoKo9k1LZdmlNyJdDDfy2v0fXjFlmok4+i8ul/6WlbVge9bhM74OpNPQPMGUToDtz+KXa1PneJxOA==", + "dev": true, + "requires": { + "is-plain-object": "^2.0.4" + } + } + } + }, + "mkdirp": { + "version": "0.5.1", + "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-0.5.1.tgz", + "integrity": "sha1-MAV0OOrGz3+MR2fzhkjWaX11yQM=", + "dev": true, + "requires": { + "minimist": "0.0.8" + } + }, + "move-concurrently": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/move-concurrently/-/move-concurrently-1.0.1.tgz", + "integrity": "sha1-viwAX9oy4LKa8fBdfEszIUxwH5I=", + "dev": true, + "requires": { + "aproba": "^1.1.1", + "copy-concurrently": "^1.0.0", + "fs-write-stream-atomic": "^1.0.8", + "mkdirp": "^0.5.1", + "rimraf": "^2.5.4", + "run-queue": "^1.0.3" + } + }, + "ms": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", + "integrity": "sha1-VgiurfwAvmwpAd9fmGF4jeDVl8g=", + "dev": true + }, + "multi-glob": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/multi-glob/-/multi-glob-1.0.1.tgz", + "integrity": "sha1-5n0qtEKdJ2BubrTbNQlK/JF4h1A=", + "dev": true, + "requires": { + "async": "1.x", + "glob": "5.x", + "lodash": "3.x" + }, + "dependencies": { + "lodash": { + "version": "3.10.1", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-3.10.1.tgz", + "integrity": "sha1-W/Rejkm6QYnhfUgnid/RW9FAt7Y=", + "dev": true + } + } + }, + "mute-stream": { + "version": "0.0.7", + "resolved": "https://registry.npmjs.org/mute-stream/-/mute-stream-0.0.7.tgz", + "integrity": "sha1-MHXOk7whuPq0PhvE2n6BFe0ee6s=", + "dev": true + }, + "nan": { + "version": "2.10.0", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.10.0.tgz", + "integrity": "sha512-bAdJv7fBLhWC+/Bls0Oza+mvTaNQtP+1RyhhhvD95pgUJz6XM5IzgmxOkItJ9tkoCiplvAnXI1tNmmUD/eScyA==", + "dev": true, + "optional": true + }, + "nanomatch": { + "version": "1.2.13", + "resolved": "https://registry.npmjs.org/nanomatch/-/nanomatch-1.2.13.tgz", + "integrity": "sha512-fpoe2T0RbHwBTBUOftAfBPaDEi06ufaUai0mE6Yn1kacc3SnTErfb/h+X94VXzI64rKFHYImXSvdwGGCmwOqCA==", + "dev": true, + "requires": { + "arr-diff": "^4.0.0", + "array-unique": "^0.3.2", + "define-property": "^2.0.2", + "extend-shallow": "^3.0.2", + "fragment-cache": "^0.2.1", + "is-windows": "^1.0.2", + "kind-of": "^6.0.2", + "object.pick": "^1.3.0", + "regex-not": "^1.0.0", + "snapdragon": "^0.8.1", + "to-regex": "^3.0.1" + }, + "dependencies": { + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "neo-async": { + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/neo-async/-/neo-async-2.5.1.tgz", + "integrity": "sha512-3KL3fvuRkZ7s4IFOMfztb7zJp3QaVWnBeGoJlgB38XnCRPj/0tLzzLG5IB8NYOHbJ8g8UGrgZv44GLDk6CxTxA==", + "dev": true + }, + "next-tick": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/next-tick/-/next-tick-1.0.0.tgz", + "integrity": "sha1-yobR/ogoFpsBICCOPchCS524NCw=", + "dev": true + }, + "nice-try": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/nice-try/-/nice-try-1.0.4.tgz", + "integrity": "sha512-2NpiFHqC87y/zFke0fC0spBXL3bBsoh/p5H1EFhshxjCR5+0g2d6BiXbUFz9v1sAcxsk2htp2eQnNIci2dIYcA==", + "dev": true + }, + "node-libs-browser": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/node-libs-browser/-/node-libs-browser-2.1.0.tgz", + "integrity": "sha512-5AzFzdoIMb89hBGMZglEegffzgRg+ZFoUmisQ8HI4j1KDdpx13J0taNp2y9xPbur6W61gepGDDotGBVQ7mfUCg==", + "dev": true, + "requires": { + "assert": "^1.1.1", + "browserify-zlib": "^0.2.0", + "buffer": "^4.3.0", + "console-browserify": "^1.1.0", + "constants-browserify": "^1.0.0", + "crypto-browserify": "^3.11.0", + "domain-browser": "^1.1.1", + "events": "^1.0.0", + "https-browserify": "^1.0.0", + "os-browserify": "^0.3.0", + "path-browserify": "0.0.0", + "process": "^0.11.10", + "punycode": "^1.2.4", + "querystring-es3": "^0.2.0", + "readable-stream": "^2.3.3", + "stream-browserify": "^2.0.1", + "stream-http": "^2.7.2", + "string_decoder": "^1.0.0", + "timers-browserify": "^2.0.4", + "tty-browserify": "0.0.0", + "url": "^0.11.0", + "util": "^0.10.3", + "vm-browserify": "0.0.4" + }, + "dependencies": { + "punycode": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-1.4.1.tgz", + "integrity": "sha1-wNWmOycYgArY4esPpSachN1BhF4=", + "dev": true + } + } + }, + "nopt": { + "version": "3.0.6", + "resolved": "https://registry.npmjs.org/nopt/-/nopt-3.0.6.tgz", + "integrity": "sha1-xkZdvwirzU2zWTF/eaxopkayj/k=", + "dev": true, + "requires": { + "abbrev": "1" + } + }, + "normalize-path": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-2.1.1.tgz", + "integrity": "sha1-GrKLVW4Zg2Oowab35vogE3/mrtk=", + "dev": true, + "requires": { + "remove-trailing-separator": "^1.0.1" + } + }, + "normalize-range": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/normalize-range/-/normalize-range-0.1.2.tgz", + "integrity": "sha1-LRDAa9/TEuqXd2laTShDlFa3WUI=", + "dev": true + }, + "normalize-url": { + "version": "1.9.1", + "resolved": "https://registry.npmjs.org/normalize-url/-/normalize-url-1.9.1.tgz", + "integrity": "sha1-LMDWazHqIwNkWENuNiDYWVTGbDw=", + "dev": true, + "requires": { + "object-assign": "^4.0.1", + "prepend-http": "^1.0.0", + "query-string": "^4.1.0", + "sort-keys": "^1.0.0" + } + }, + "npm": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/npm/-/npm-6.1.0.tgz", + "integrity": "sha512-e38cCtJ0lEjLXXpc4twEfj8Xw5hDLolc2Py87ueWnUhJfZ8GA/5RVIeD+XbSr1+aVRGsRsdtLdzUNO63PvQJ1w==", + "requires": { + "JSONStream": "^1.3.2", + "abbrev": "~1.1.1", + "ansi-regex": "~3.0.0", + "ansicolors": "~0.3.2", + "ansistyles": "~0.1.3", + "aproba": "~1.2.0", + "archy": "~1.0.0", + "bin-links": "^1.1.2", + "bluebird": "~3.5.1", + "byte-size": "^4.0.3", + "cacache": "^11.0.2", + "call-limit": "~1.1.0", + "chownr": "~1.0.1", + "cli-columns": "^3.1.2", + "cli-table2": "~0.2.0", + "cmd-shim": "~2.0.2", + "columnify": "~1.5.4", + "config-chain": "~1.1.11", + "debuglog": "*", + "detect-indent": "~5.0.0", + "detect-newline": "^2.1.0", + "dezalgo": "~1.0.3", + "editor": "~1.0.0", + "figgy-pudding": "^3.1.0", + "find-npm-prefix": "^1.0.2", + "fs-vacuum": "~1.2.10", + "fs-write-stream-atomic": "~1.0.10", + "gentle-fs": "^2.0.1", + "glob": "~7.1.2", + "graceful-fs": "~4.1.11", + "has-unicode": "~2.0.1", + "hosted-git-info": "^2.6.0", + "iferr": "^1.0.0", + "imurmurhash": "*", + "inflight": "~1.0.6", + "inherits": "~2.0.3", + "ini": "^1.3.5", + "init-package-json": "^1.10.3", + "is-cidr": "^2.0.5", + "json-parse-better-errors": "^1.0.2", + "lazy-property": "~1.0.0", + "libcipm": "^1.6.2", + "libnpmhook": "^4.0.1", + "libnpx": "^10.2.0", + "lock-verify": "^2.0.2", + "lockfile": "^1.0.4", + "lodash._baseindexof": "*", + "lodash._baseuniq": "~4.6.0", + "lodash._bindcallback": "*", + "lodash._cacheindexof": "*", + "lodash._createcache": "*", + "lodash._getnative": "*", + "lodash.clonedeep": "~4.5.0", + "lodash.restparam": "*", + "lodash.union": "~4.6.0", + "lodash.uniq": "~4.5.0", + "lodash.without": "~4.4.0", + "lru-cache": "^4.1.3", + "meant": "~1.0.1", + "mississippi": "^3.0.0", + "mkdirp": "~0.5.1", + "move-concurrently": "^1.0.1", + "node-gyp": "^3.6.2", + "nopt": "~4.0.1", + "normalize-package-data": "~2.4.0", + "npm-audit-report": "^1.2.1", + "npm-cache-filename": "~1.0.2", + "npm-install-checks": "~3.0.0", + "npm-lifecycle": "^2.0.3", + "npm-package-arg": "^6.1.0", + "npm-packlist": "~1.1.10", + "npm-pick-manifest": "^2.1.0", + "npm-profile": "^3.0.1", + "npm-registry-client": "^8.5.1", + "npm-registry-fetch": "^1.1.0", + "npm-user-validate": "~1.0.0", + "npmlog": "~4.1.2", + "once": "~1.4.0", + "opener": "~1.4.3", + "osenv": "^0.1.5", + "pacote": "^8.1.5", + "path-is-inside": "~1.0.2", + "promise-inflight": "~1.0.1", + "qrcode-terminal": "^0.12.0", + "query-string": "^6.1.0", + "qw": "~1.0.1", + "read": "~1.0.7", + "read-cmd-shim": "~1.0.1", + "read-installed": "~4.0.3", + "read-package-json": "^2.0.13", + "read-package-tree": "^5.2.1", + "readable-stream": "^2.3.6", + "readdir-scoped-modules": "*", + "request": "^2.86.0", + "retry": "^0.12.0", + "rimraf": "~2.6.2", + "safe-buffer": "^5.1.2", + "semver": "^5.5.0", + "sha": "~2.0.1", + "slide": "~1.1.6", + "sorted-object": "~2.0.1", + "sorted-union-stream": "~2.1.3", + "ssri": "^6.0.0", + "strip-ansi": "~4.0.0", + "tar": "^4.4.1", + "text-table": "~0.2.0", + "tiny-relative-date": "^1.3.0", + "uid-number": "0.0.6", + "umask": "~1.1.0", + "unique-filename": "~1.1.0", + "unpipe": "~1.0.0", + "update-notifier": "^2.5.0", + "uuid": "^3.2.1", + "validate-npm-package-license": "^3.0.3", + "validate-npm-package-name": "~3.0.0", + "which": "~1.3.0", + "worker-farm": "^1.6.0", + "wrappy": "~1.0.2", + "write-file-atomic": "^2.3.0" + }, + "dependencies": { + "JSONStream": { + "version": "1.3.2", + "bundled": true, + "requires": { + "jsonparse": "^1.2.0", + "through": ">=2.2.7 <3" + }, + "dependencies": { + "jsonparse": { + "version": "1.3.1", + "bundled": true + }, + "through": { + "version": "2.3.8", + "bundled": true + } + } + }, + "abbrev": { + "version": "1.1.1", + "bundled": true + }, + "ansi-regex": { + "version": "3.0.0", + "bundled": true + }, + "ansicolors": { + "version": "0.3.2", + "bundled": true + }, + "ansistyles": { + "version": "0.1.3", + "bundled": true + }, + "aproba": { + "version": "1.2.0", + "bundled": true + }, + "archy": { + "version": "1.0.0", + "bundled": true + }, + "bin-links": { + "version": "1.1.2", + "bundled": true, + "requires": { + "bluebird": "^3.5.0", + "cmd-shim": "^2.0.2", + "gentle-fs": "^2.0.0", + "graceful-fs": "^4.1.11", + "write-file-atomic": "^2.3.0" + } + }, + "bluebird": { + "version": "3.5.1", + "bundled": true + }, + "byte-size": { + "version": "4.0.3", + "bundled": true + }, + "cacache": { + "version": "11.0.2", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "chownr": "^1.0.1", + "figgy-pudding": "^3.1.0", + "glob": "^7.1.2", + "graceful-fs": "^4.1.11", + "lru-cache": "^4.1.2", + "mississippi": "^3.0.0", + "mkdirp": "^0.5.1", + "move-concurrently": "^1.0.1", + "promise-inflight": "^1.0.1", + "rimraf": "^2.6.2", + "ssri": "^6.0.0", + "unique-filename": "^1.1.0", + "y18n": "^4.0.0" + }, + "dependencies": { + "y18n": { + "version": "4.0.0", + "bundled": true + } + } + }, + "call-limit": { + "version": "1.1.0", + "bundled": true + }, + "chownr": { + "version": "1.0.1", + "bundled": true + }, + "cli-columns": { + "version": "3.1.2", + "bundled": true, + "requires": { + "string-width": "^2.0.0", + "strip-ansi": "^3.0.1" + }, + "dependencies": { + "string-width": { + "version": "2.1.1", + "bundled": true, + "requires": { + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^4.0.0" + }, + "dependencies": { + "is-fullwidth-code-point": { + "version": "2.0.0", + "bundled": true + }, + "strip-ansi": { + "version": "4.0.0", + "bundled": true, + "requires": { + "ansi-regex": "^3.0.0" + } + } + } + }, + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "cli-table2": { + "version": "0.2.0", + "bundled": true, + "requires": { + "colors": "^1.1.2", + "lodash": "^3.10.1", + "string-width": "^1.0.1" + }, + "dependencies": { + "colors": { + "version": "1.1.2", + "bundled": true, + "optional": true + }, + "lodash": { + "version": "3.10.1", + "bundled": true + }, + "string-width": { + "version": "1.0.2", + "bundled": true, + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + }, + "dependencies": { + "code-point-at": { + "version": "1.1.0", + "bundled": true + }, + "is-fullwidth-code-point": { + "version": "1.0.0", + "bundled": true, + "requires": { + "number-is-nan": "^1.0.0" + }, + "dependencies": { + "number-is-nan": { + "version": "1.0.1", + "bundled": true + } + } + }, + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "bundled": true + } + } + } + } + } + } + }, + "cmd-shim": { + "version": "2.0.2", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2", + "mkdirp": "~0.5.0" + } + }, + "columnify": { + "version": "1.5.4", + "bundled": true, + "requires": { + "strip-ansi": "^3.0.0", + "wcwidth": "^1.0.0" + }, + "dependencies": { + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "bundled": true + } + } + }, + "wcwidth": { + "version": "1.0.1", + "bundled": true, + "requires": { + "defaults": "^1.0.3" + }, + "dependencies": { + "defaults": { + "version": "1.0.3", + "bundled": true, + "requires": { + "clone": "^1.0.2" + }, + "dependencies": { + "clone": { + "version": "1.0.2", + "bundled": true + } + } + } + } + } + } + }, + "config-chain": { + "version": "1.1.11", + "bundled": true, + "requires": { + "ini": "^1.3.4", + "proto-list": "~1.2.1" + }, + "dependencies": { + "proto-list": { + "version": "1.2.4", + "bundled": true + } + } + }, + "debuglog": { + "version": "1.0.1", + "bundled": true + }, + "detect-indent": { + "version": "5.0.0", + "bundled": true + }, + "detect-newline": { + "version": "2.1.0", + "bundled": true + }, + "dezalgo": { + "version": "1.0.3", + "bundled": true, + "requires": { + "asap": "^2.0.0", + "wrappy": "1" + }, + "dependencies": { + "asap": { + "version": "2.0.5", + "bundled": true + } + } + }, + "editor": { + "version": "1.0.0", + "bundled": true + }, + "figgy-pudding": { + "version": "3.1.0", + "bundled": true + }, + "find-npm-prefix": { + "version": "1.0.2", + "bundled": true + }, + "fs-vacuum": { + "version": "1.2.10", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2", + "path-is-inside": "^1.0.1", + "rimraf": "^2.5.2" + } + }, + "fs-write-stream-atomic": { + "version": "1.0.10", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2", + "iferr": "^0.1.5", + "imurmurhash": "^0.1.4", + "readable-stream": "1 || 2" + }, + "dependencies": { + "iferr": { + "version": "0.1.5", + "bundled": true + } + } + }, + "gentle-fs": { + "version": "2.0.1", + "bundled": true, + "requires": { + "aproba": "^1.1.2", + "fs-vacuum": "^1.2.10", + "graceful-fs": "^4.1.11", + "iferr": "^0.1.5", + "mkdirp": "^0.5.1", + "path-is-inside": "^1.0.2", + "read-cmd-shim": "^1.0.1", + "slide": "^1.1.6" + }, + "dependencies": { + "iferr": { + "version": "0.1.5", + "bundled": true + } + } + }, + "glob": { + "version": "7.1.2", + "bundled": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + }, + "dependencies": { + "fs.realpath": { + "version": "1.0.0", + "bundled": true + }, + "minimatch": { + "version": "3.0.4", + "bundled": true, + "requires": { + "brace-expansion": "^1.1.7" + }, + "dependencies": { + "brace-expansion": { + "version": "1.1.8", + "bundled": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "1.0.0", + "bundled": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true + } + } + } + } + }, + "path-is-absolute": { + "version": "1.0.1", + "bundled": true + } + } + }, + "graceful-fs": { + "version": "4.1.11", + "bundled": true + }, + "has-unicode": { + "version": "2.0.1", + "bundled": true + }, + "hosted-git-info": { + "version": "2.6.0", + "bundled": true + }, + "iferr": { + "version": "1.0.0", + "bundled": true + }, + "imurmurhash": { + "version": "0.1.4", + "bundled": true + }, + "inflight": { + "version": "1.0.6", + "bundled": true, + "requires": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "inherits": { + "version": "2.0.3", + "bundled": true + }, + "ini": { + "version": "1.3.5", + "bundled": true + }, + "init-package-json": { + "version": "1.10.3", + "bundled": true, + "requires": { + "glob": "^7.1.1", + "npm-package-arg": "^4.0.0 || ^5.0.0 || ^6.0.0", + "promzard": "^0.3.0", + "read": "~1.0.1", + "read-package-json": "1 || 2", + "semver": "2.x || 3.x || 4 || 5", + "validate-npm-package-license": "^3.0.1", + "validate-npm-package-name": "^3.0.0" + }, + "dependencies": { + "promzard": { + "version": "0.3.0", + "bundled": true, + "requires": { + "read": "1" + } + } + } + }, + "is-cidr": { + "version": "2.0.5", + "bundled": true, + "requires": { + "cidr-regex": "^2.0.8" + }, + "dependencies": { + "cidr-regex": { + "version": "2.0.8", + "bundled": true, + "requires": { + "ip-regex": "^2.1.0" + }, + "dependencies": { + "ip-regex": { + "version": "2.1.0", + "bundled": true + } + } + } + } + }, + "json-parse-better-errors": { + "version": "1.0.2", + "bundled": true + }, + "lazy-property": { + "version": "1.0.0", + "bundled": true + }, + "libcipm": { + "version": "1.6.2", + "bundled": true, + "requires": { + "bin-links": "^1.1.0", + "bluebird": "^3.5.1", + "find-npm-prefix": "^1.0.2", + "graceful-fs": "^4.1.11", + "lock-verify": "^2.0.0", + "npm-lifecycle": "^2.0.0", + "npm-logical-tree": "^1.2.1", + "npm-package-arg": "^6.0.0", + "pacote": "^7.5.1", + "protoduck": "^5.0.0", + "read-package-json": "^2.0.12", + "rimraf": "^2.6.2", + "worker-farm": "^1.5.4" + }, + "dependencies": { + "npm-logical-tree": { + "version": "1.2.1", + "bundled": true + }, + "pacote": { + "version": "7.6.1", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "cacache": "^10.0.4", + "get-stream": "^3.0.0", + "glob": "^7.1.2", + "lru-cache": "^4.1.1", + "make-fetch-happen": "^2.6.0", + "minimatch": "^3.0.4", + "mississippi": "^3.0.0", + "mkdirp": "^0.5.1", + "normalize-package-data": "^2.4.0", + "npm-package-arg": "^6.0.0", + "npm-packlist": "^1.1.10", + "npm-pick-manifest": "^2.1.0", + "osenv": "^0.1.5", + "promise-inflight": "^1.0.1", + "promise-retry": "^1.1.1", + "protoduck": "^5.0.0", + "rimraf": "^2.6.2", + "safe-buffer": "^5.1.1", + "semver": "^5.5.0", + "ssri": "^5.2.4", + "tar": "^4.4.0", + "unique-filename": "^1.1.0", + "which": "^1.3.0" + }, + "dependencies": { + "cacache": { + "version": "10.0.4", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "chownr": "^1.0.1", + "glob": "^7.1.2", + "graceful-fs": "^4.1.11", + "lru-cache": "^4.1.1", + "mississippi": "^2.0.0", + "mkdirp": "^0.5.1", + "move-concurrently": "^1.0.1", + "promise-inflight": "^1.0.1", + "rimraf": "^2.6.2", + "ssri": "^5.2.4", + "unique-filename": "^1.1.0", + "y18n": "^4.0.0" + }, + "dependencies": { + "mississippi": { + "version": "2.0.0", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^2.0.1", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.2", + "bundled": true, + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "buffer-from": { + "version": "1.0.0", + "bundled": true + }, + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.4", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.3", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "y18n": { + "version": "4.0.0", + "bundled": true + } + } + }, + "get-stream": { + "version": "3.0.0", + "bundled": true + }, + "make-fetch-happen": { + "version": "2.6.0", + "bundled": true, + "requires": { + "agentkeepalive": "^3.3.0", + "cacache": "^10.0.0", + "http-cache-semantics": "^3.8.0", + "http-proxy-agent": "^2.0.0", + "https-proxy-agent": "^2.1.0", + "lru-cache": "^4.1.1", + "mississippi": "^1.2.0", + "node-fetch-npm": "^2.0.2", + "promise-retry": "^1.1.1", + "socks-proxy-agent": "^3.0.1", + "ssri": "^5.0.0" + }, + "dependencies": { + "agentkeepalive": { + "version": "3.4.1", + "bundled": true, + "requires": { + "humanize-ms": "^1.2.1" + }, + "dependencies": { + "humanize-ms": { + "version": "1.2.1", + "bundled": true, + "requires": { + "ms": "^2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "http-cache-semantics": { + "version": "3.8.1", + "bundled": true + }, + "http-proxy-agent": { + "version": "2.1.0", + "bundled": true, + "requires": { + "agent-base": "4", + "debug": "3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "https-proxy-agent": { + "version": "2.2.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "debug": "^3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "mississippi": { + "version": "1.3.1", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^1.0.0", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.2", + "bundled": true, + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "buffer-from": { + "version": "1.0.0", + "bundled": true + }, + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.4", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.3", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "1.0.3", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + }, + "dependencies": { + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + } + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "node-fetch-npm": { + "version": "2.0.2", + "bundled": true, + "requires": { + "encoding": "^0.1.11", + "json-parse-better-errors": "^1.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "encoding": { + "version": "0.1.12", + "bundled": true, + "requires": { + "iconv-lite": "~0.4.13" + }, + "dependencies": { + "iconv-lite": { + "version": "0.4.21", + "bundled": true, + "requires": { + "safer-buffer": "^2.1.0" + }, + "dependencies": { + "safer-buffer": { + "version": "2.1.2", + "bundled": true + } + } + } + } + } + } + }, + "socks-proxy-agent": { + "version": "3.0.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "socks": "^1.1.10" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "socks": { + "version": "1.1.10", + "bundled": true, + "requires": { + "ip": "^1.1.4", + "smart-buffer": "^1.0.13" + }, + "dependencies": { + "ip": { + "version": "1.1.5", + "bundled": true + }, + "smart-buffer": { + "version": "1.1.15", + "bundled": true + } + } + } + } + } + } + }, + "minimatch": { + "version": "3.0.4", + "bundled": true, + "requires": { + "brace-expansion": "^1.1.7" + }, + "dependencies": { + "brace-expansion": { + "version": "1.1.11", + "bundled": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "1.0.0", + "bundled": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true + } + } + } + } + }, + "promise-retry": { + "version": "1.1.1", + "bundled": true, + "requires": { + "err-code": "^1.0.0", + "retry": "^0.10.0" + }, + "dependencies": { + "err-code": { + "version": "1.1.2", + "bundled": true + }, + "retry": { + "version": "0.10.1", + "bundled": true + } + } + }, + "ssri": { + "version": "5.3.0", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.1" + } + } + } + }, + "protoduck": { + "version": "5.0.0", + "bundled": true, + "requires": { + "genfun": "^4.0.1" + }, + "dependencies": { + "genfun": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "libnpmhook": { + "version": "4.0.1", + "bundled": true, + "requires": { + "figgy-pudding": "^3.1.0", + "npm-registry-fetch": "^3.0.0" + }, + "dependencies": { + "npm-registry-fetch": { + "version": "3.1.1", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "figgy-pudding": "^3.1.0", + "lru-cache": "^4.1.2", + "make-fetch-happen": "^4.0.0", + "npm-package-arg": "^6.0.0" + }, + "dependencies": { + "make-fetch-happen": { + "version": "4.0.1", + "bundled": true, + "requires": { + "agentkeepalive": "^3.4.1", + "cacache": "^11.0.1", + "http-cache-semantics": "^3.8.1", + "http-proxy-agent": "^2.1.0", + "https-proxy-agent": "^2.2.1", + "lru-cache": "^4.1.2", + "mississippi": "^3.0.0", + "node-fetch-npm": "^2.0.2", + "promise-retry": "^1.1.1", + "socks-proxy-agent": "^4.0.0", + "ssri": "^6.0.0" + }, + "dependencies": { + "agentkeepalive": { + "version": "3.4.1", + "bundled": true, + "requires": { + "humanize-ms": "^1.2.1" + }, + "dependencies": { + "humanize-ms": { + "version": "1.2.1", + "bundled": true, + "requires": { + "ms": "^2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "http-cache-semantics": { + "version": "3.8.1", + "bundled": true + }, + "http-proxy-agent": { + "version": "2.1.0", + "bundled": true, + "requires": { + "agent-base": "4", + "debug": "3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "https-proxy-agent": { + "version": "2.2.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "debug": "^3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "node-fetch-npm": { + "version": "2.0.2", + "bundled": true, + "requires": { + "encoding": "^0.1.11", + "json-parse-better-errors": "^1.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "encoding": { + "version": "0.1.12", + "bundled": true, + "requires": { + "iconv-lite": "~0.4.13" + }, + "dependencies": { + "iconv-lite": { + "version": "0.4.21", + "bundled": true, + "requires": { + "safer-buffer": "^2.1.0" + }, + "dependencies": { + "safer-buffer": { + "version": "2.1.2", + "bundled": true + } + } + } + } + } + } + }, + "promise-retry": { + "version": "1.1.1", + "bundled": true, + "requires": { + "err-code": "^1.0.0", + "retry": "^0.10.0" + }, + "dependencies": { + "err-code": { + "version": "1.1.2", + "bundled": true + }, + "retry": { + "version": "0.10.1", + "bundled": true + } + } + }, + "socks-proxy-agent": { + "version": "4.0.0", + "bundled": true, + "requires": { + "agent-base": "~4.1.0", + "socks": "~2.1.6" + }, + "dependencies": { + "agent-base": { + "version": "4.1.2", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "socks": { + "version": "2.1.6", + "bundled": true, + "requires": { + "ip": "^1.1.5", + "smart-buffer": "^4.0.1" + }, + "dependencies": { + "ip": { + "version": "1.1.5", + "bundled": true + }, + "smart-buffer": { + "version": "4.0.1", + "bundled": true + } + } + } + } + } + } + } + } + } + } + }, + "libnpx": { + "version": "10.2.0", + "bundled": true, + "requires": { + "dotenv": "^5.0.1", + "npm-package-arg": "^6.0.0", + "rimraf": "^2.6.2", + "safe-buffer": "^5.1.0", + "update-notifier": "^2.3.0", + "which": "^1.3.0", + "y18n": "^4.0.0", + "yargs": "^11.0.0" + }, + "dependencies": { + "dotenv": { + "version": "5.0.1", + "bundled": true + }, + "y18n": { + "version": "4.0.0", + "bundled": true + }, + "yargs": { + "version": "11.0.0", + "bundled": true, + "requires": { + "cliui": "^4.0.0", + "decamelize": "^1.1.1", + "find-up": "^2.1.0", + "get-caller-file": "^1.0.1", + "os-locale": "^2.0.0", + "require-directory": "^2.1.1", + "require-main-filename": "^1.0.1", + "set-blocking": "^2.0.0", + "string-width": "^2.0.0", + "which-module": "^2.0.0", + "y18n": "^3.2.1", + "yargs-parser": "^9.0.2" + }, + "dependencies": { + "cliui": { + "version": "4.0.0", + "bundled": true, + "requires": { + "string-width": "^2.1.1", + "strip-ansi": "^4.0.0", + "wrap-ansi": "^2.0.0" + }, + "dependencies": { + "wrap-ansi": { + "version": "2.1.0", + "bundled": true, + "requires": { + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1" + }, + "dependencies": { + "string-width": { + "version": "1.0.2", + "bundled": true, + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + }, + "dependencies": { + "code-point-at": { + "version": "1.1.0", + "bundled": true + }, + "is-fullwidth-code-point": { + "version": "1.0.0", + "bundled": true, + "requires": { + "number-is-nan": "^1.0.0" + }, + "dependencies": { + "number-is-nan": { + "version": "1.0.1", + "bundled": true + } + } + } + } + }, + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "bundled": true + } + } + } + } + } + } + }, + "decamelize": { + "version": "1.2.0", + "bundled": true + }, + "find-up": { + "version": "2.1.0", + "bundled": true, + "requires": { + "locate-path": "^2.0.0" + }, + "dependencies": { + "locate-path": { + "version": "2.0.0", + "bundled": true, + "requires": { + "p-locate": "^2.0.0", + "path-exists": "^3.0.0" + }, + "dependencies": { + "p-locate": { + "version": "2.0.0", + "bundled": true, + "requires": { + "p-limit": "^1.1.0" + }, + "dependencies": { + "p-limit": { + "version": "1.2.0", + "bundled": true, + "requires": { + "p-try": "^1.0.0" + }, + "dependencies": { + "p-try": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "path-exists": { + "version": "3.0.0", + "bundled": true + } + } + } + } + }, + "get-caller-file": { + "version": "1.0.2", + "bundled": true + }, + "os-locale": { + "version": "2.1.0", + "bundled": true, + "requires": { + "execa": "^0.7.0", + "lcid": "^1.0.0", + "mem": "^1.1.0" + }, + "dependencies": { + "execa": { + "version": "0.7.0", + "bundled": true, + "requires": { + "cross-spawn": "^5.0.1", + "get-stream": "^3.0.0", + "is-stream": "^1.1.0", + "npm-run-path": "^2.0.0", + "p-finally": "^1.0.0", + "signal-exit": "^3.0.0", + "strip-eof": "^1.0.0" + }, + "dependencies": { + "cross-spawn": { + "version": "5.1.0", + "bundled": true, + "requires": { + "lru-cache": "^4.0.1", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + }, + "dependencies": { + "shebang-command": { + "version": "1.2.0", + "bundled": true, + "requires": { + "shebang-regex": "^1.0.0" + }, + "dependencies": { + "shebang-regex": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "get-stream": { + "version": "3.0.0", + "bundled": true + }, + "is-stream": { + "version": "1.1.0", + "bundled": true + }, + "npm-run-path": { + "version": "2.0.2", + "bundled": true, + "requires": { + "path-key": "^2.0.0" + }, + "dependencies": { + "path-key": { + "version": "2.0.1", + "bundled": true + } + } + }, + "p-finally": { + "version": "1.0.0", + "bundled": true + }, + "signal-exit": { + "version": "3.0.2", + "bundled": true + }, + "strip-eof": { + "version": "1.0.0", + "bundled": true + } + } + }, + "lcid": { + "version": "1.0.0", + "bundled": true, + "requires": { + "invert-kv": "^1.0.0" + }, + "dependencies": { + "invert-kv": { + "version": "1.0.0", + "bundled": true + } + } + }, + "mem": { + "version": "1.1.0", + "bundled": true, + "requires": { + "mimic-fn": "^1.0.0" + }, + "dependencies": { + "mimic-fn": { + "version": "1.2.0", + "bundled": true + } + } + } + } + }, + "require-directory": { + "version": "2.1.1", + "bundled": true + }, + "require-main-filename": { + "version": "1.0.1", + "bundled": true + }, + "set-blocking": { + "version": "2.0.0", + "bundled": true + }, + "string-width": { + "version": "2.1.1", + "bundled": true, + "requires": { + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^4.0.0" + }, + "dependencies": { + "is-fullwidth-code-point": { + "version": "2.0.0", + "bundled": true + } + } + }, + "which-module": { + "version": "2.0.0", + "bundled": true + }, + "y18n": { + "version": "3.2.1", + "bundled": true + }, + "yargs-parser": { + "version": "9.0.2", + "bundled": true, + "requires": { + "camelcase": "^4.1.0" + }, + "dependencies": { + "camelcase": { + "version": "4.1.0", + "bundled": true + } + } + } + } + } + } + }, + "lock-verify": { + "version": "2.0.2", + "bundled": true, + "requires": { + "npm-package-arg": "^5.1.2 || 6", + "semver": "^5.4.1" + } + }, + "lockfile": { + "version": "1.0.4", + "bundled": true, + "requires": { + "signal-exit": "^3.0.2" + }, + "dependencies": { + "signal-exit": { + "version": "3.0.2", + "bundled": true + } + } + }, + "lodash._baseindexof": { + "version": "3.1.0", + "bundled": true + }, + "lodash._baseuniq": { + "version": "4.6.0", + "bundled": true, + "requires": { + "lodash._createset": "~4.0.0", + "lodash._root": "~3.0.0" + }, + "dependencies": { + "lodash._createset": { + "version": "4.0.3", + "bundled": true + }, + "lodash._root": { + "version": "3.0.1", + "bundled": true + } + } + }, + "lodash._bindcallback": { + "version": "3.0.1", + "bundled": true + }, + "lodash._cacheindexof": { + "version": "3.0.2", + "bundled": true + }, + "lodash._createcache": { + "version": "3.1.2", + "bundled": true, + "requires": { + "lodash._getnative": "^3.0.0" + } + }, + "lodash._getnative": { + "version": "3.9.1", + "bundled": true + }, + "lodash.clonedeep": { + "version": "4.5.0", + "bundled": true + }, + "lodash.restparam": { + "version": "3.6.1", + "bundled": true + }, + "lodash.union": { + "version": "4.6.0", + "bundled": true + }, + "lodash.uniq": { + "version": "4.5.0", + "bundled": true + }, + "lodash.without": { + "version": "4.4.0", + "bundled": true + }, + "lru-cache": { + "version": "4.1.3", + "bundled": true, + "requires": { + "pseudomap": "^1.0.2", + "yallist": "^2.1.2" + }, + "dependencies": { + "pseudomap": { + "version": "1.0.2", + "bundled": true + }, + "yallist": { + "version": "2.1.2", + "bundled": true + } + } + }, + "meant": { + "version": "1.0.1", + "bundled": true + }, + "mississippi": { + "version": "3.0.0", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^3.0.0", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.1", + "bundled": true, + "requires": { + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.4", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.2", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "3.0.0", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + }, + "dependencies": { + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + } + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "mkdirp": { + "version": "0.5.1", + "bundled": true, + "requires": { + "minimist": "0.0.8" + }, + "dependencies": { + "minimist": { + "version": "0.0.8", + "bundled": true + } + } + }, + "move-concurrently": { + "version": "1.0.1", + "bundled": true, + "requires": { + "aproba": "^1.1.1", + "copy-concurrently": "^1.0.0", + "fs-write-stream-atomic": "^1.0.8", + "mkdirp": "^0.5.1", + "rimraf": "^2.5.4", + "run-queue": "^1.0.3" + }, + "dependencies": { + "copy-concurrently": { + "version": "1.0.5", + "bundled": true, + "requires": { + "aproba": "^1.1.1", + "fs-write-stream-atomic": "^1.0.8", + "iferr": "^0.1.5", + "mkdirp": "^0.5.1", + "rimraf": "^2.5.4", + "run-queue": "^1.0.0" + }, + "dependencies": { + "iferr": { + "version": "0.1.5", + "bundled": true + } + } + }, + "run-queue": { + "version": "1.0.3", + "bundled": true, + "requires": { + "aproba": "^1.1.1" + } + } + } + }, + "node-gyp": { + "version": "3.6.2", + "bundled": true, + "requires": { + "fstream": "^1.0.0", + "glob": "^7.0.3", + "graceful-fs": "^4.1.2", + "minimatch": "^3.0.2", + "mkdirp": "^0.5.0", + "nopt": "2 || 3", + "npmlog": "0 || 1 || 2 || 3 || 4", + "osenv": "0", + "request": "2", + "rimraf": "2", + "semver": "~5.3.0", + "tar": "^2.0.0", + "which": "1" + }, + "dependencies": { + "fstream": { + "version": "1.0.11", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2", + "inherits": "~2.0.0", + "mkdirp": ">=0.5 0", + "rimraf": "2" + } + }, + "minimatch": { + "version": "3.0.4", + "bundled": true, + "requires": { + "brace-expansion": "^1.1.7" + }, + "dependencies": { + "brace-expansion": { + "version": "1.1.11", + "bundled": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "1.0.0", + "bundled": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true + } + } + } + } + }, + "nopt": { + "version": "3.0.6", + "bundled": true, + "requires": { + "abbrev": "1" + } + }, + "semver": { + "version": "5.3.0", + "bundled": true + }, + "tar": { + "version": "2.2.1", + "bundled": true, + "requires": { + "block-stream": "*", + "fstream": "^1.0.2", + "inherits": "2" + }, + "dependencies": { + "block-stream": { + "version": "0.0.9", + "bundled": true, + "requires": { + "inherits": "~2.0.0" + } + } + } + } + } + }, + "nopt": { + "version": "4.0.1", + "bundled": true, + "requires": { + "abbrev": "1", + "osenv": "^0.1.4" + } + }, + "normalize-package-data": { + "version": "2.4.0", + "bundled": true, + "requires": { + "hosted-git-info": "^2.1.4", + "is-builtin-module": "^1.0.0", + "semver": "2 || 3 || 4 || 5", + "validate-npm-package-license": "^3.0.1" + }, + "dependencies": { + "is-builtin-module": { + "version": "1.0.0", + "bundled": true, + "requires": { + "builtin-modules": "^1.0.0" + }, + "dependencies": { + "builtin-modules": { + "version": "1.1.1", + "bundled": true + } + } + } + } + }, + "npm-audit-report": { + "version": "1.2.1", + "bundled": true, + "requires": { + "cli-table2": "^0.2.0", + "console-control-strings": "^1.1.0" + }, + "dependencies": { + "console-control-strings": { + "version": "1.1.0", + "bundled": true + } + } + }, + "npm-cache-filename": { + "version": "1.0.2", + "bundled": true + }, + "npm-install-checks": { + "version": "3.0.0", + "bundled": true, + "requires": { + "semver": "^2.3.0 || 3.x || 4 || 5" + } + }, + "npm-lifecycle": { + "version": "2.0.3", + "bundled": true, + "requires": { + "byline": "^5.0.0", + "graceful-fs": "^4.1.11", + "node-gyp": "^3.6.2", + "resolve-from": "^4.0.0", + "slide": "^1.1.6", + "uid-number": "0.0.6", + "umask": "^1.1.0", + "which": "^1.3.0" + }, + "dependencies": { + "byline": { + "version": "5.0.0", + "bundled": true + }, + "resolve-from": { + "version": "4.0.0", + "bundled": true + } + } + }, + "npm-package-arg": { + "version": "6.1.0", + "bundled": true, + "requires": { + "hosted-git-info": "^2.6.0", + "osenv": "^0.1.5", + "semver": "^5.5.0", + "validate-npm-package-name": "^3.0.0" + } + }, + "npm-packlist": { + "version": "1.1.10", + "bundled": true, + "requires": { + "ignore-walk": "^3.0.1", + "npm-bundled": "^1.0.1" + }, + "dependencies": { + "ignore-walk": { + "version": "3.0.1", + "bundled": true, + "requires": { + "minimatch": "^3.0.4" + }, + "dependencies": { + "minimatch": { + "version": "3.0.4", + "bundled": true, + "requires": { + "brace-expansion": "^1.1.7" + }, + "dependencies": { + "brace-expansion": { + "version": "1.1.8", + "bundled": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "1.0.0", + "bundled": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true + } + } + } + } + } + } + }, + "npm-bundled": { + "version": "1.0.3", + "bundled": true + } + } + }, + "npm-pick-manifest": { + "version": "2.1.0", + "bundled": true, + "requires": { + "npm-package-arg": "^6.0.0", + "semver": "^5.4.1" + } + }, + "npm-profile": { + "version": "3.0.1", + "bundled": true, + "requires": { + "aproba": "^1.1.2", + "make-fetch-happen": "^2.5.0" + }, + "dependencies": { + "make-fetch-happen": { + "version": "2.6.0", + "bundled": true, + "requires": { + "agentkeepalive": "^3.3.0", + "cacache": "^10.0.0", + "http-cache-semantics": "^3.8.0", + "http-proxy-agent": "^2.0.0", + "https-proxy-agent": "^2.1.0", + "lru-cache": "^4.1.1", + "mississippi": "^1.2.0", + "node-fetch-npm": "^2.0.2", + "promise-retry": "^1.1.1", + "socks-proxy-agent": "^3.0.1", + "ssri": "^5.0.0" + }, + "dependencies": { + "agentkeepalive": { + "version": "3.3.0", + "bundled": true, + "requires": { + "humanize-ms": "^1.2.1" + }, + "dependencies": { + "humanize-ms": { + "version": "1.2.1", + "bundled": true, + "requires": { + "ms": "^2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "cacache": { + "version": "10.0.4", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "chownr": "^1.0.1", + "glob": "^7.1.2", + "graceful-fs": "^4.1.11", + "lru-cache": "^4.1.1", + "mississippi": "^2.0.0", + "mkdirp": "^0.5.1", + "move-concurrently": "^1.0.1", + "promise-inflight": "^1.0.1", + "rimraf": "^2.6.2", + "ssri": "^5.2.4", + "unique-filename": "^1.1.0", + "y18n": "^4.0.0" + }, + "dependencies": { + "mississippi": { + "version": "2.0.0", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^2.0.1", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.2", + "bundled": true, + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "buffer-from": { + "version": "1.0.0", + "bundled": true + }, + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.4", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.3", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "y18n": { + "version": "4.0.0", + "bundled": true + } + } + }, + "http-cache-semantics": { + "version": "3.8.1", + "bundled": true + }, + "http-proxy-agent": { + "version": "2.1.0", + "bundled": true, + "requires": { + "agent-base": "4", + "debug": "3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "https-proxy-agent": { + "version": "2.2.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "debug": "^3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "mississippi": { + "version": "1.3.1", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^1.0.0", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.0", + "bundled": true, + "requires": { + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.3", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.2", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "1.0.3", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + }, + "dependencies": { + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + } + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "node-fetch-npm": { + "version": "2.0.2", + "bundled": true, + "requires": { + "encoding": "^0.1.11", + "json-parse-better-errors": "^1.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "encoding": { + "version": "0.1.12", + "bundled": true, + "requires": { + "iconv-lite": "~0.4.13" + }, + "dependencies": { + "iconv-lite": { + "version": "0.4.19", + "bundled": true + } + } + }, + "json-parse-better-errors": { + "version": "1.0.1", + "bundled": true + } + } + }, + "promise-retry": { + "version": "1.1.1", + "bundled": true, + "requires": { + "err-code": "^1.0.0", + "retry": "^0.10.0" + }, + "dependencies": { + "err-code": { + "version": "1.1.2", + "bundled": true + }, + "retry": { + "version": "0.10.1", + "bundled": true + } + } + }, + "socks-proxy-agent": { + "version": "3.0.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "socks": "^1.1.10" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "socks": { + "version": "1.1.10", + "bundled": true, + "requires": { + "ip": "^1.1.4", + "smart-buffer": "^1.0.13" + }, + "dependencies": { + "ip": { + "version": "1.1.5", + "bundled": true + }, + "smart-buffer": { + "version": "1.1.15", + "bundled": true + } + } + } + } + }, + "ssri": { + "version": "5.3.0", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.1" + } + } + } + } + } + }, + "npm-registry-client": { + "version": "8.5.1", + "bundled": true, + "requires": { + "concat-stream": "^1.5.2", + "graceful-fs": "^4.1.6", + "normalize-package-data": "~1.0.1 || ^2.0.0", + "npm-package-arg": "^3.0.0 || ^4.0.0 || ^5.0.0 || ^6.0.0", + "npmlog": "2 || ^3.1.0 || ^4.0.0", + "once": "^1.3.3", + "request": "^2.74.0", + "retry": "^0.10.0", + "safe-buffer": "^5.1.1", + "semver": "2 >=2.2.1 || 3.x || 4 || 5", + "slide": "^1.1.3", + "ssri": "^5.2.4" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.1", + "bundled": true, + "requires": { + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "retry": { + "version": "0.10.1", + "bundled": true + }, + "ssri": { + "version": "5.3.0", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.1" + } + } + } + }, + "npm-registry-fetch": { + "version": "1.1.0", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "figgy-pudding": "^2.0.1", + "lru-cache": "^4.1.2", + "make-fetch-happen": "^3.0.0", + "npm-package-arg": "^6.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "figgy-pudding": { + "version": "2.0.1", + "bundled": true + }, + "make-fetch-happen": { + "version": "3.0.0", + "bundled": true, + "requires": { + "agentkeepalive": "^3.4.1", + "cacache": "^10.0.4", + "http-cache-semantics": "^3.8.1", + "http-proxy-agent": "^2.1.0", + "https-proxy-agent": "^2.2.0", + "lru-cache": "^4.1.2", + "mississippi": "^3.0.0", + "node-fetch-npm": "^2.0.2", + "promise-retry": "^1.1.1", + "socks-proxy-agent": "^3.0.1", + "ssri": "^5.2.4" + }, + "dependencies": { + "agentkeepalive": { + "version": "3.4.1", + "bundled": true, + "requires": { + "humanize-ms": "^1.2.1" + }, + "dependencies": { + "humanize-ms": { + "version": "1.2.1", + "bundled": true, + "requires": { + "ms": "^2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "cacache": { + "version": "10.0.4", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "chownr": "^1.0.1", + "glob": "^7.1.2", + "graceful-fs": "^4.1.11", + "lru-cache": "^4.1.1", + "mississippi": "^2.0.0", + "mkdirp": "^0.5.1", + "move-concurrently": "^1.0.1", + "promise-inflight": "^1.0.1", + "rimraf": "^2.6.2", + "ssri": "^5.2.4", + "unique-filename": "^1.1.0", + "y18n": "^4.0.0" + }, + "dependencies": { + "mississippi": { + "version": "2.0.0", + "bundled": true, + "requires": { + "concat-stream": "^1.5.0", + "duplexify": "^3.4.2", + "end-of-stream": "^1.1.0", + "flush-write-stream": "^1.0.0", + "from2": "^2.1.0", + "parallel-transform": "^1.1.0", + "pump": "^2.0.1", + "pumpify": "^1.3.3", + "stream-each": "^1.1.0", + "through2": "^2.0.0" + }, + "dependencies": { + "concat-stream": { + "version": "1.6.2", + "bundled": true, + "requires": { + "buffer-from": "^1.0.0", + "inherits": "^2.0.3", + "readable-stream": "^2.2.2", + "typedarray": "^0.0.6" + }, + "dependencies": { + "buffer-from": { + "version": "1.0.0", + "bundled": true + }, + "typedarray": { + "version": "0.0.6", + "bundled": true + } + } + }, + "duplexify": { + "version": "3.5.4", + "bundled": true, + "requires": { + "end-of-stream": "^1.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.0.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "end-of-stream": { + "version": "1.4.1", + "bundled": true, + "requires": { + "once": "^1.4.0" + } + }, + "flush-write-stream": { + "version": "1.0.3", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.4" + } + }, + "from2": { + "version": "2.3.0", + "bundled": true, + "requires": { + "inherits": "^2.0.1", + "readable-stream": "^2.0.0" + } + }, + "parallel-transform": { + "version": "1.1.0", + "bundled": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + }, + "dependencies": { + "cyclist": { + "version": "0.2.2", + "bundled": true + } + } + }, + "pump": { + "version": "2.0.1", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.4.0", + "bundled": true, + "requires": { + "duplexify": "^3.5.3", + "inherits": "^2.0.3", + "pump": "^2.0.0" + } + }, + "stream-each": { + "version": "1.2.2", + "bundled": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + }, + "through2": { + "version": "2.0.3", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + }, + "dependencies": { + "xtend": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "y18n": { + "version": "4.0.0", + "bundled": true + } + } + }, + "http-cache-semantics": { + "version": "3.8.1", + "bundled": true + }, + "http-proxy-agent": { + "version": "2.1.0", + "bundled": true, + "requires": { + "agent-base": "4", + "debug": "3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "https-proxy-agent": { + "version": "2.2.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "debug": "^3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "node-fetch-npm": { + "version": "2.0.2", + "bundled": true, + "requires": { + "encoding": "^0.1.11", + "json-parse-better-errors": "^1.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "encoding": { + "version": "0.1.12", + "bundled": true, + "requires": { + "iconv-lite": "~0.4.13" + }, + "dependencies": { + "iconv-lite": { + "version": "0.4.21", + "bundled": true, + "requires": { + "safer-buffer": "^2.1.0" + }, + "dependencies": { + "safer-buffer": { + "version": "2.1.2", + "bundled": true + } + } + } + } + } + } + }, + "promise-retry": { + "version": "1.1.1", + "bundled": true, + "requires": { + "err-code": "^1.0.0", + "retry": "^0.10.0" + }, + "dependencies": { + "err-code": { + "version": "1.1.2", + "bundled": true + }, + "retry": { + "version": "0.10.1", + "bundled": true + } + } + }, + "socks-proxy-agent": { + "version": "3.0.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "socks": "^1.1.10" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "socks": { + "version": "1.1.10", + "bundled": true, + "requires": { + "ip": "^1.1.4", + "smart-buffer": "^1.0.13" + }, + "dependencies": { + "ip": { + "version": "1.1.5", + "bundled": true + }, + "smart-buffer": { + "version": "1.1.15", + "bundled": true + } + } + } + } + }, + "ssri": { + "version": "5.3.0", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.1" + } + } + } + } + } + }, + "npm-user-validate": { + "version": "1.0.0", + "bundled": true + }, + "npmlog": { + "version": "4.1.2", + "bundled": true, + "requires": { + "are-we-there-yet": "~1.1.2", + "console-control-strings": "~1.1.0", + "gauge": "~2.7.3", + "set-blocking": "~2.0.0" + }, + "dependencies": { + "are-we-there-yet": { + "version": "1.1.4", + "bundled": true, + "requires": { + "delegates": "^1.0.0", + "readable-stream": "^2.0.6" + }, + "dependencies": { + "delegates": { + "version": "1.0.0", + "bundled": true + } + } + }, + "console-control-strings": { + "version": "1.1.0", + "bundled": true + }, + "gauge": { + "version": "2.7.4", + "bundled": true, + "requires": { + "aproba": "^1.0.3", + "console-control-strings": "^1.0.0", + "has-unicode": "^2.0.0", + "object-assign": "^4.1.0", + "signal-exit": "^3.0.0", + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1", + "wide-align": "^1.1.0" + }, + "dependencies": { + "object-assign": { + "version": "4.1.1", + "bundled": true + }, + "signal-exit": { + "version": "3.0.2", + "bundled": true + }, + "string-width": { + "version": "1.0.2", + "bundled": true, + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + }, + "dependencies": { + "code-point-at": { + "version": "1.1.0", + "bundled": true + }, + "is-fullwidth-code-point": { + "version": "1.0.0", + "bundled": true, + "requires": { + "number-is-nan": "^1.0.0" + }, + "dependencies": { + "number-is-nan": { + "version": "1.0.1", + "bundled": true + } + } + } + } + }, + "strip-ansi": { + "version": "3.0.1", + "bundled": true, + "requires": { + "ansi-regex": "^2.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "2.1.1", + "bundled": true + } + } + }, + "wide-align": { + "version": "1.1.2", + "bundled": true, + "requires": { + "string-width": "^1.0.2" + } + } + } + }, + "set-blocking": { + "version": "2.0.0", + "bundled": true + } + } + }, + "once": { + "version": "1.4.0", + "bundled": true, + "requires": { + "wrappy": "1" + } + }, + "opener": { + "version": "1.4.3", + "bundled": true + }, + "osenv": { + "version": "0.1.5", + "bundled": true, + "requires": { + "os-homedir": "^1.0.0", + "os-tmpdir": "^1.0.0" + }, + "dependencies": { + "os-homedir": { + "version": "1.0.2", + "bundled": true + }, + "os-tmpdir": { + "version": "1.0.2", + "bundled": true + } + } + }, + "pacote": { + "version": "8.1.5", + "bundled": true, + "requires": { + "bluebird": "^3.5.1", + "cacache": "^11.0.2", + "get-stream": "^3.0.0", + "glob": "^7.1.2", + "lru-cache": "^4.1.3", + "make-fetch-happen": "^4.0.1", + "minimatch": "^3.0.4", + "minipass": "^2.3.3", + "mississippi": "^3.0.0", + "mkdirp": "^0.5.1", + "normalize-package-data": "^2.4.0", + "npm-package-arg": "^6.1.0", + "npm-packlist": "^1.1.10", + "npm-pick-manifest": "^2.1.0", + "osenv": "^0.1.5", + "promise-inflight": "^1.0.1", + "promise-retry": "^1.1.1", + "protoduck": "^5.0.0", + "rimraf": "^2.6.2", + "safe-buffer": "^5.1.2", + "semver": "^5.5.0", + "ssri": "^6.0.0", + "tar": "4.4.1", + "unique-filename": "^1.1.0", + "which": "^1.3.0" + }, + "dependencies": { + "get-stream": { + "version": "3.0.0", + "bundled": true + }, + "make-fetch-happen": { + "version": "4.0.1", + "bundled": true, + "requires": { + "agentkeepalive": "^3.4.1", + "cacache": "^11.0.1", + "http-cache-semantics": "^3.8.1", + "http-proxy-agent": "^2.1.0", + "https-proxy-agent": "^2.2.1", + "lru-cache": "^4.1.2", + "mississippi": "^3.0.0", + "node-fetch-npm": "^2.0.2", + "promise-retry": "^1.1.1", + "socks-proxy-agent": "^4.0.0", + "ssri": "^6.0.0" + }, + "dependencies": { + "agentkeepalive": { + "version": "3.4.1", + "bundled": true, + "requires": { + "humanize-ms": "^1.2.1" + }, + "dependencies": { + "humanize-ms": { + "version": "1.2.1", + "bundled": true, + "requires": { + "ms": "^2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.1.1", + "bundled": true + } + } + } + } + }, + "http-cache-semantics": { + "version": "3.8.1", + "bundled": true + }, + "http-proxy-agent": { + "version": "2.1.0", + "bundled": true, + "requires": { + "agent-base": "4", + "debug": "3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "https-proxy-agent": { + "version": "2.2.1", + "bundled": true, + "requires": { + "agent-base": "^4.1.0", + "debug": "^3.1.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "debug": { + "version": "3.1.0", + "bundled": true, + "requires": { + "ms": "2.0.0" + }, + "dependencies": { + "ms": { + "version": "2.0.0", + "bundled": true + } + } + } + } + }, + "node-fetch-npm": { + "version": "2.0.2", + "bundled": true, + "requires": { + "encoding": "^0.1.11", + "json-parse-better-errors": "^1.0.0", + "safe-buffer": "^5.1.1" + }, + "dependencies": { + "encoding": { + "version": "0.1.12", + "bundled": true, + "requires": { + "iconv-lite": "~0.4.13" + }, + "dependencies": { + "iconv-lite": { + "version": "0.4.23", + "bundled": true, + "requires": { + "safer-buffer": ">= 2.1.2 < 3" + }, + "dependencies": { + "safer-buffer": { + "version": "2.1.2", + "bundled": true + } + } + } + } + } + } + }, + "socks-proxy-agent": { + "version": "4.0.1", + "bundled": true, + "requires": { + "agent-base": "~4.2.0", + "socks": "~2.2.0" + }, + "dependencies": { + "agent-base": { + "version": "4.2.0", + "bundled": true, + "requires": { + "es6-promisify": "^5.0.0" + }, + "dependencies": { + "es6-promisify": { + "version": "5.0.0", + "bundled": true, + "requires": { + "es6-promise": "^4.0.3" + }, + "dependencies": { + "es6-promise": { + "version": "4.2.4", + "bundled": true + } + } + } + } + }, + "socks": { + "version": "2.2.0", + "bundled": true, + "requires": { + "ip": "^1.1.5", + "smart-buffer": "^4.0.1" + }, + "dependencies": { + "ip": { + "version": "1.1.5", + "bundled": true + }, + "smart-buffer": { + "version": "4.0.1", + "bundled": true + } + } + } + } + } + } + }, + "minimatch": { + "version": "3.0.4", + "bundled": true, + "requires": { + "brace-expansion": "^1.1.7" + }, + "dependencies": { + "brace-expansion": { + "version": "1.1.11", + "bundled": true, + "requires": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "1.0.0", + "bundled": true + }, + "concat-map": { + "version": "0.0.1", + "bundled": true + } + } + } + } + }, + "minipass": { + "version": "2.3.3", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.2", + "yallist": "^3.0.0" + }, + "dependencies": { + "yallist": { + "version": "3.0.2", + "bundled": true + } + } + }, + "promise-retry": { + "version": "1.1.1", + "bundled": true, + "requires": { + "err-code": "^1.0.0", + "retry": "^0.10.0" + }, + "dependencies": { + "err-code": { + "version": "1.1.2", + "bundled": true + }, + "retry": { + "version": "0.10.1", + "bundled": true + } + } + }, + "protoduck": { + "version": "5.0.0", + "bundled": true, + "requires": { + "genfun": "^4.0.1" + }, + "dependencies": { + "genfun": { + "version": "4.0.1", + "bundled": true + } + } + } + } + }, + "path-is-inside": { + "version": "1.0.2", + "bundled": true + }, + "promise-inflight": { + "version": "1.0.1", + "bundled": true + }, + "qrcode-terminal": { + "version": "0.12.0", + "bundled": true + }, + "query-string": { + "version": "6.1.0", + "bundled": true, + "requires": { + "decode-uri-component": "^0.2.0", + "strict-uri-encode": "^2.0.0" + }, + "dependencies": { + "decode-uri-component": { + "version": "0.2.0", + "bundled": true + }, + "strict-uri-encode": { + "version": "2.0.0", + "bundled": true + } + } + }, + "qw": { + "version": "1.0.1", + "bundled": true + }, + "read": { + "version": "1.0.7", + "bundled": true, + "requires": { + "mute-stream": "~0.0.4" + }, + "dependencies": { + "mute-stream": { + "version": "0.0.7", + "bundled": true + } + } + }, + "read-cmd-shim": { + "version": "1.0.1", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2" + } + }, + "read-installed": { + "version": "4.0.3", + "bundled": true, + "requires": { + "debuglog": "^1.0.1", + "graceful-fs": "^4.1.2", + "read-package-json": "^2.0.0", + "readdir-scoped-modules": "^1.0.0", + "semver": "2 || 3 || 4 || 5", + "slide": "~1.1.3", + "util-extend": "^1.0.1" + }, + "dependencies": { + "util-extend": { + "version": "1.0.3", + "bundled": true + } + } + }, + "read-package-json": { + "version": "2.0.13", + "bundled": true, + "requires": { + "glob": "^7.1.1", + "graceful-fs": "^4.1.2", + "json-parse-better-errors": "^1.0.1", + "normalize-package-data": "^2.0.0", + "slash": "^1.0.0" + }, + "dependencies": { + "json-parse-better-errors": { + "version": "1.0.1", + "bundled": true + }, + "slash": { + "version": "1.0.0", + "bundled": true + } + } + }, + "read-package-tree": { + "version": "5.2.1", + "bundled": true, + "requires": { + "debuglog": "^1.0.1", + "dezalgo": "^1.0.0", + "once": "^1.3.0", + "read-package-json": "^2.0.0", + "readdir-scoped-modules": "^1.0.0" + } + }, + "readable-stream": { + "version": "2.3.6", + "bundled": true, + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + }, + "dependencies": { + "core-util-is": { + "version": "1.0.2", + "bundled": true + }, + "isarray": { + "version": "1.0.0", + "bundled": true + }, + "process-nextick-args": { + "version": "2.0.0", + "bundled": true + }, + "string_decoder": { + "version": "1.1.1", + "bundled": true, + "requires": { + "safe-buffer": "~5.1.0" + } + }, + "util-deprecate": { + "version": "1.0.2", + "bundled": true + } + } + }, + "readdir-scoped-modules": { + "version": "1.0.2", + "bundled": true, + "requires": { + "debuglog": "^1.0.1", + "dezalgo": "^1.0.0", + "graceful-fs": "^4.1.2", + "once": "^1.3.0" + } + }, + "request": { + "version": "2.86.0", + "bundled": true, + "requires": { + "aws-sign2": "~0.7.0", + "aws4": "^1.6.0", + "caseless": "~0.12.0", + "combined-stream": "~1.0.5", + "extend": "~3.0.1", + "forever-agent": "~0.6.1", + "form-data": "~2.3.1", + "har-validator": "~5.0.3", + "hawk": "~6.0.2", + "http-signature": "~1.2.0", + "is-typedarray": "~1.0.0", + "isstream": "~0.1.2", + "json-stringify-safe": "~5.0.1", + "mime-types": "~2.1.17", + "oauth-sign": "~0.8.2", + "performance-now": "^2.1.0", + "qs": "~6.5.1", + "safe-buffer": "^5.1.1", + "tough-cookie": "~2.3.3", + "tunnel-agent": "^0.6.0", + "uuid": "^3.1.0" + }, + "dependencies": { + "aws-sign2": { + "version": "0.7.0", + "bundled": true + }, + "aws4": { + "version": "1.7.0", + "bundled": true + }, + "caseless": { + "version": "0.12.0", + "bundled": true + }, + "combined-stream": { + "version": "1.0.6", + "bundled": true, + "requires": { + "delayed-stream": "~1.0.0" + }, + "dependencies": { + "delayed-stream": { + "version": "1.0.0", + "bundled": true + } + } + }, + "extend": { + "version": "3.0.1", + "bundled": true + }, + "forever-agent": { + "version": "0.6.1", + "bundled": true + }, + "form-data": { + "version": "2.3.2", + "bundled": true, + "requires": { + "asynckit": "^0.4.0", + "combined-stream": "1.0.6", + "mime-types": "^2.1.12" + }, + "dependencies": { + "asynckit": { + "version": "0.4.0", + "bundled": true + } + } + }, + "har-validator": { + "version": "5.0.3", + "bundled": true, + "requires": { + "ajv": "^5.1.0", + "har-schema": "^2.0.0" + }, + "dependencies": { + "ajv": { + "version": "5.5.2", + "bundled": true, + "requires": { + "co": "^4.6.0", + "fast-deep-equal": "^1.0.0", + "fast-json-stable-stringify": "^2.0.0", + "json-schema-traverse": "^0.3.0" + }, + "dependencies": { + "co": { + "version": "4.6.0", + "bundled": true + }, + "fast-deep-equal": { + "version": "1.1.0", + "bundled": true + }, + "fast-json-stable-stringify": { + "version": "2.0.0", + "bundled": true + }, + "json-schema-traverse": { + "version": "0.3.1", + "bundled": true + } + } + }, + "har-schema": { + "version": "2.0.0", + "bundled": true + } + } + }, + "hawk": { + "version": "6.0.2", + "bundled": true, + "requires": { + "boom": "4.x.x", + "cryptiles": "3.x.x", + "hoek": "4.x.x", + "sntp": "2.x.x" + }, + "dependencies": { + "boom": { + "version": "4.3.1", + "bundled": true, + "requires": { + "hoek": "4.x.x" + } + }, + "cryptiles": { + "version": "3.1.2", + "bundled": true, + "requires": { + "boom": "5.x.x" + }, + "dependencies": { + "boom": { + "version": "5.2.0", + "bundled": true, + "requires": { + "hoek": "4.x.x" + } + } + } + }, + "hoek": { + "version": "4.2.1", + "bundled": true + }, + "sntp": { + "version": "2.1.0", + "bundled": true, + "requires": { + "hoek": "4.x.x" + } + } + } + }, + "http-signature": { + "version": "1.2.0", + "bundled": true, + "requires": { + "assert-plus": "^1.0.0", + "jsprim": "^1.2.2", + "sshpk": "^1.7.0" + }, + "dependencies": { + "assert-plus": { + "version": "1.0.0", + "bundled": true + }, + "jsprim": { + "version": "1.4.1", + "bundled": true, + "requires": { + "assert-plus": "1.0.0", + "extsprintf": "1.3.0", + "json-schema": "0.2.3", + "verror": "1.10.0" + }, + "dependencies": { + "extsprintf": { + "version": "1.3.0", + "bundled": true + }, + "json-schema": { + "version": "0.2.3", + "bundled": true + }, + "verror": { + "version": "1.10.0", + "bundled": true, + "requires": { + "assert-plus": "^1.0.0", + "core-util-is": "1.0.2", + "extsprintf": "^1.2.0" + }, + "dependencies": { + "core-util-is": { + "version": "1.0.2", + "bundled": true + } + } + } + } + }, + "sshpk": { + "version": "1.14.1", + "bundled": true, + "requires": { + "asn1": "~0.2.3", + "assert-plus": "^1.0.0", + "bcrypt-pbkdf": "^1.0.0", + "dashdash": "^1.12.0", + "ecc-jsbn": "~0.1.1", + "getpass": "^0.1.1", + "jsbn": "~0.1.0", + "tweetnacl": "~0.14.0" + }, + "dependencies": { + "asn1": { + "version": "0.2.3", + "bundled": true + }, + "bcrypt-pbkdf": { + "version": "1.0.1", + "bundled": true, + "optional": true, + "requires": { + "tweetnacl": "^0.14.3" + } + }, + "dashdash": { + "version": "1.14.1", + "bundled": true, + "requires": { + "assert-plus": "^1.0.0" + } + }, + "ecc-jsbn": { + "version": "0.1.1", + "bundled": true, + "optional": true, + "requires": { + "jsbn": "~0.1.0" + } + }, + "getpass": { + "version": "0.1.7", + "bundled": true, + "requires": { + "assert-plus": "^1.0.0" + } + }, + "jsbn": { + "version": "0.1.1", + "bundled": true, + "optional": true + }, + "tweetnacl": { + "version": "0.14.5", + "bundled": true, + "optional": true + } + } + } + } + }, + "is-typedarray": { + "version": "1.0.0", + "bundled": true + }, + "isstream": { + "version": "0.1.2", + "bundled": true + }, + "json-stringify-safe": { + "version": "5.0.1", + "bundled": true + }, + "mime-types": { + "version": "2.1.18", + "bundled": true, + "requires": { + "mime-db": "~1.33.0" + }, + "dependencies": { + "mime-db": { + "version": "1.33.0", + "bundled": true + } + } + }, + "oauth-sign": { + "version": "0.8.2", + "bundled": true + }, + "performance-now": { + "version": "2.1.0", + "bundled": true + }, + "qs": { + "version": "6.5.2", + "bundled": true + }, + "tough-cookie": { + "version": "2.3.4", + "bundled": true, + "requires": { + "punycode": "^1.4.1" + }, + "dependencies": { + "punycode": { + "version": "1.4.1", + "bundled": true + } + } + }, + "tunnel-agent": { + "version": "0.6.0", + "bundled": true, + "requires": { + "safe-buffer": "^5.0.1" + } + } + } + }, + "retry": { + "version": "0.12.0", + "bundled": true + }, + "rimraf": { + "version": "2.6.2", + "bundled": true, + "requires": { + "glob": "^7.0.5" + } + }, + "safe-buffer": { + "version": "5.1.2", + "bundled": true + }, + "semver": { + "version": "5.5.0", + "bundled": true + }, + "sha": { + "version": "2.0.1", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.2", + "readable-stream": "^2.0.2" + } + }, + "slide": { + "version": "1.1.6", + "bundled": true + }, + "sorted-object": { + "version": "2.0.1", + "bundled": true + }, + "sorted-union-stream": { + "version": "2.1.3", + "bundled": true, + "requires": { + "from2": "^1.3.0", + "stream-iterate": "^1.1.0" + }, + "dependencies": { + "from2": { + "version": "1.3.0", + "bundled": true, + "requires": { + "inherits": "~2.0.1", + "readable-stream": "~1.1.10" + }, + "dependencies": { + "readable-stream": { + "version": "1.1.14", + "bundled": true, + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.1", + "isarray": "0.0.1", + "string_decoder": "~0.10.x" + }, + "dependencies": { + "core-util-is": { + "version": "1.0.2", + "bundled": true + }, + "isarray": { + "version": "0.0.1", + "bundled": true + }, + "string_decoder": { + "version": "0.10.31", + "bundled": true + } + } + } + } + }, + "stream-iterate": { + "version": "1.2.0", + "bundled": true, + "requires": { + "readable-stream": "^2.1.5", + "stream-shift": "^1.0.0" + }, + "dependencies": { + "stream-shift": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "ssri": { + "version": "6.0.0", + "bundled": true + }, + "strip-ansi": { + "version": "4.0.0", + "bundled": true, + "requires": { + "ansi-regex": "^3.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "3.0.0", + "bundled": true + } + } + }, + "tar": { + "version": "4.4.1", + "bundled": true, + "requires": { + "chownr": "^1.0.1", + "fs-minipass": "^1.2.5", + "minipass": "^2.2.4", + "minizlib": "^1.1.0", + "mkdirp": "^0.5.0", + "safe-buffer": "^5.1.1", + "yallist": "^3.0.2" + }, + "dependencies": { + "fs-minipass": { + "version": "1.2.5", + "bundled": true, + "requires": { + "minipass": "^2.2.1" + } + }, + "minipass": { + "version": "2.3.1", + "bundled": true, + "requires": { + "safe-buffer": "^5.1.1", + "yallist": "^3.0.0" + } + }, + "minizlib": { + "version": "1.1.0", + "bundled": true, + "requires": { + "minipass": "^2.2.1" + } + }, + "yallist": { + "version": "3.0.2", + "bundled": true + } + } + }, + "text-table": { + "version": "0.2.0", + "bundled": true + }, + "tiny-relative-date": { + "version": "1.3.0", + "bundled": true + }, + "uid-number": { + "version": "0.0.6", + "bundled": true + }, + "umask": { + "version": "1.1.0", + "bundled": true + }, + "unique-filename": { + "version": "1.1.0", + "bundled": true, + "requires": { + "unique-slug": "^2.0.0" + }, + "dependencies": { + "unique-slug": { + "version": "2.0.0", + "bundled": true, + "requires": { + "imurmurhash": "^0.1.4" + } + } + } + }, + "unpipe": { + "version": "1.0.0", + "bundled": true + }, + "update-notifier": { + "version": "2.5.0", + "bundled": true, + "requires": { + "boxen": "^1.2.1", + "chalk": "^2.0.1", + "configstore": "^3.0.0", + "import-lazy": "^2.1.0", + "is-ci": "^1.0.10", + "is-installed-globally": "^0.1.0", + "is-npm": "^1.0.0", + "latest-version": "^3.0.0", + "semver-diff": "^2.0.0", + "xdg-basedir": "^3.0.0" + }, + "dependencies": { + "boxen": { + "version": "1.3.0", + "bundled": true, + "requires": { + "ansi-align": "^2.0.0", + "camelcase": "^4.0.0", + "chalk": "^2.0.1", + "cli-boxes": "^1.0.0", + "string-width": "^2.0.0", + "term-size": "^1.2.0", + "widest-line": "^2.0.0" + }, + "dependencies": { + "ansi-align": { + "version": "2.0.0", + "bundled": true, + "requires": { + "string-width": "^2.0.0" + } + }, + "camelcase": { + "version": "4.1.0", + "bundled": true + }, + "cli-boxes": { + "version": "1.0.0", + "bundled": true + }, + "string-width": { + "version": "2.1.1", + "bundled": true, + "requires": { + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^4.0.0" + }, + "dependencies": { + "is-fullwidth-code-point": { + "version": "2.0.0", + "bundled": true + } + } + }, + "term-size": { + "version": "1.2.0", + "bundled": true, + "requires": { + "execa": "^0.7.0" + }, + "dependencies": { + "execa": { + "version": "0.7.0", + "bundled": true, + "requires": { + "cross-spawn": "^5.0.1", + "get-stream": "^3.0.0", + "is-stream": "^1.1.0", + "npm-run-path": "^2.0.0", + "p-finally": "^1.0.0", + "signal-exit": "^3.0.0", + "strip-eof": "^1.0.0" + }, + "dependencies": { + "cross-spawn": { + "version": "5.1.0", + "bundled": true, + "requires": { + "lru-cache": "^4.0.1", + "shebang-command": "^1.2.0", + "which": "^1.2.9" + }, + "dependencies": { + "shebang-command": { + "version": "1.2.0", + "bundled": true, + "requires": { + "shebang-regex": "^1.0.0" + }, + "dependencies": { + "shebang-regex": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "get-stream": { + "version": "3.0.0", + "bundled": true + }, + "is-stream": { + "version": "1.1.0", + "bundled": true + }, + "npm-run-path": { + "version": "2.0.2", + "bundled": true, + "requires": { + "path-key": "^2.0.0" + }, + "dependencies": { + "path-key": { + "version": "2.0.1", + "bundled": true + } + } + }, + "p-finally": { + "version": "1.0.0", + "bundled": true + }, + "signal-exit": { + "version": "3.0.2", + "bundled": true + }, + "strip-eof": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "widest-line": { + "version": "2.0.0", + "bundled": true, + "requires": { + "string-width": "^2.1.1" + } + } + } + }, + "chalk": { + "version": "2.4.1", + "bundled": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + }, + "dependencies": { + "ansi-styles": { + "version": "3.2.1", + "bundled": true, + "requires": { + "color-convert": "^1.9.0" + }, + "dependencies": { + "color-convert": { + "version": "1.9.1", + "bundled": true, + "requires": { + "color-name": "^1.1.1" + }, + "dependencies": { + "color-name": { + "version": "1.1.3", + "bundled": true + } + } + } + } + }, + "escape-string-regexp": { + "version": "1.0.5", + "bundled": true + }, + "supports-color": { + "version": "5.4.0", + "bundled": true, + "requires": { + "has-flag": "^3.0.0" + }, + "dependencies": { + "has-flag": { + "version": "3.0.0", + "bundled": true + } + } + } + } + }, + "configstore": { + "version": "3.1.2", + "bundled": true, + "requires": { + "dot-prop": "^4.1.0", + "graceful-fs": "^4.1.2", + "make-dir": "^1.0.0", + "unique-string": "^1.0.0", + "write-file-atomic": "^2.0.0", + "xdg-basedir": "^3.0.0" + }, + "dependencies": { + "dot-prop": { + "version": "4.2.0", + "bundled": true, + "requires": { + "is-obj": "^1.0.0" + }, + "dependencies": { + "is-obj": { + "version": "1.0.1", + "bundled": true + } + } + }, + "make-dir": { + "version": "1.2.0", + "bundled": true, + "requires": { + "pify": "^3.0.0" + }, + "dependencies": { + "pify": { + "version": "3.0.0", + "bundled": true + } + } + }, + "unique-string": { + "version": "1.0.0", + "bundled": true, + "requires": { + "crypto-random-string": "^1.0.0" + }, + "dependencies": { + "crypto-random-string": { + "version": "1.0.0", + "bundled": true + } + } + } + } + }, + "import-lazy": { + "version": "2.1.0", + "bundled": true + }, + "is-ci": { + "version": "1.1.0", + "bundled": true, + "requires": { + "ci-info": "^1.0.0" + }, + "dependencies": { + "ci-info": { + "version": "1.1.3", + "bundled": true + } + } + }, + "is-installed-globally": { + "version": "0.1.0", + "bundled": true, + "requires": { + "global-dirs": "^0.1.0", + "is-path-inside": "^1.0.0" + }, + "dependencies": { + "global-dirs": { + "version": "0.1.1", + "bundled": true, + "requires": { + "ini": "^1.3.4" + } + }, + "is-path-inside": { + "version": "1.0.1", + "bundled": true, + "requires": { + "path-is-inside": "^1.0.1" + } + } + } + }, + "is-npm": { + "version": "1.0.0", + "bundled": true + }, + "latest-version": { + "version": "3.1.0", + "bundled": true, + "requires": { + "package-json": "^4.0.0" + }, + "dependencies": { + "package-json": { + "version": "4.0.1", + "bundled": true, + "requires": { + "got": "^6.7.1", + "registry-auth-token": "^3.0.1", + "registry-url": "^3.0.3", + "semver": "^5.1.0" + }, + "dependencies": { + "got": { + "version": "6.7.1", + "bundled": true, + "requires": { + "create-error-class": "^3.0.0", + "duplexer3": "^0.1.4", + "get-stream": "^3.0.0", + "is-redirect": "^1.0.0", + "is-retry-allowed": "^1.0.0", + "is-stream": "^1.0.0", + "lowercase-keys": "^1.0.0", + "safe-buffer": "^5.0.1", + "timed-out": "^4.0.0", + "unzip-response": "^2.0.1", + "url-parse-lax": "^1.0.0" + }, + "dependencies": { + "create-error-class": { + "version": "3.0.2", + "bundled": true, + "requires": { + "capture-stack-trace": "^1.0.0" + }, + "dependencies": { + "capture-stack-trace": { + "version": "1.0.0", + "bundled": true + } + } + }, + "duplexer3": { + "version": "0.1.4", + "bundled": true + }, + "get-stream": { + "version": "3.0.0", + "bundled": true + }, + "is-redirect": { + "version": "1.0.0", + "bundled": true + }, + "is-retry-allowed": { + "version": "1.1.0", + "bundled": true + }, + "is-stream": { + "version": "1.1.0", + "bundled": true + }, + "lowercase-keys": { + "version": "1.0.1", + "bundled": true + }, + "timed-out": { + "version": "4.0.1", + "bundled": true + }, + "unzip-response": { + "version": "2.0.1", + "bundled": true + }, + "url-parse-lax": { + "version": "1.0.0", + "bundled": true, + "requires": { + "prepend-http": "^1.0.1" + }, + "dependencies": { + "prepend-http": { + "version": "1.0.4", + "bundled": true + } + } + } + } + }, + "registry-auth-token": { + "version": "3.3.2", + "bundled": true, + "requires": { + "rc": "^1.1.6", + "safe-buffer": "^5.0.1" + }, + "dependencies": { + "rc": { + "version": "1.2.7", + "bundled": true, + "requires": { + "deep-extend": "^0.5.1", + "ini": "~1.3.0", + "minimist": "^1.2.0", + "strip-json-comments": "~2.0.1" + }, + "dependencies": { + "deep-extend": { + "version": "0.5.1", + "bundled": true + }, + "minimist": { + "version": "1.2.0", + "bundled": true + }, + "strip-json-comments": { + "version": "2.0.1", + "bundled": true + } + } + } + } + }, + "registry-url": { + "version": "3.1.0", + "bundled": true, + "requires": { + "rc": "^1.0.1" + }, + "dependencies": { + "rc": { + "version": "1.2.7", + "bundled": true, + "requires": { + "deep-extend": "^0.5.1", + "ini": "~1.3.0", + "minimist": "^1.2.0", + "strip-json-comments": "~2.0.1" + }, + "dependencies": { + "deep-extend": { + "version": "0.5.1", + "bundled": true + }, + "minimist": { + "version": "1.2.0", + "bundled": true + }, + "strip-json-comments": { + "version": "2.0.1", + "bundled": true + } + } + } + } + } + } + } + } + }, + "semver-diff": { + "version": "2.1.0", + "bundled": true, + "requires": { + "semver": "^5.0.3" + } + }, + "xdg-basedir": { + "version": "3.0.0", + "bundled": true + } + } + }, + "uuid": { + "version": "3.2.1", + "bundled": true + }, + "validate-npm-package-license": { + "version": "3.0.3", + "bundled": true, + "requires": { + "spdx-correct": "^3.0.0", + "spdx-expression-parse": "^3.0.0" + }, + "dependencies": { + "spdx-correct": { + "version": "3.0.0", + "bundled": true, + "requires": { + "spdx-expression-parse": "^3.0.0", + "spdx-license-ids": "^3.0.0" + }, + "dependencies": { + "spdx-license-ids": { + "version": "3.0.0", + "bundled": true + } + } + }, + "spdx-expression-parse": { + "version": "3.0.0", + "bundled": true, + "requires": { + "spdx-exceptions": "^2.1.0", + "spdx-license-ids": "^3.0.0" + }, + "dependencies": { + "spdx-exceptions": { + "version": "2.1.0", + "bundled": true + }, + "spdx-license-ids": { + "version": "3.0.0", + "bundled": true + } + } + } + } + }, + "validate-npm-package-name": { + "version": "3.0.0", + "bundled": true, + "requires": { + "builtins": "^1.0.3" + }, + "dependencies": { + "builtins": { + "version": "1.0.3", + "bundled": true + } + } + }, + "which": { + "version": "1.3.0", + "bundled": true, + "requires": { + "isexe": "^2.0.0" + }, + "dependencies": { + "isexe": { + "version": "2.0.0", + "bundled": true + } + } + }, + "worker-farm": { + "version": "1.6.0", + "bundled": true, + "requires": { + "errno": "~0.1.7" + }, + "dependencies": { + "errno": { + "version": "0.1.7", + "bundled": true, + "requires": { + "prr": "~1.0.1" + }, + "dependencies": { + "prr": { + "version": "1.0.1", + "bundled": true + } + } + } + } + }, + "wrappy": { + "version": "1.0.2", + "bundled": true + }, + "write-file-atomic": { + "version": "2.3.0", + "bundled": true, + "requires": { + "graceful-fs": "^4.1.11", + "imurmurhash": "^0.1.4", + "signal-exit": "^3.0.2" + }, + "dependencies": { + "signal-exit": { + "version": "3.0.2", + "bundled": true + } + } + } + } + }, + "npm-run-path": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-2.0.2.tgz", + "integrity": "sha1-NakjLfo11wZ7TLLd8jV7GHFTbF8=", + "dev": true, + "requires": { + "path-key": "^2.0.0" + } + }, + "num2fraction": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/num2fraction/-/num2fraction-1.2.2.tgz", + "integrity": "sha1-b2gragJ6Tp3fpFZM0lidHU5mnt4=", + "dev": true + }, + "number-is-nan": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/number-is-nan/-/number-is-nan-1.0.1.tgz", + "integrity": "sha1-CXtgK1NCKlIsGvuHkDGDNpQaAR0=", + "dev": true + }, + "nvd3": { + "version": "1.8.6", + "resolved": "https://registry.npmjs.org/nvd3/-/nvd3-1.8.6.tgz", + "integrity": "sha1-LT66dL8zNjtRAevx0JPFmlOuc8Q=" + }, + "object-assign": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", + "integrity": "sha1-IQmtx5ZYh8/AXLvUQsrIv7s2CGM=", + "dev": true + }, + "object-copy": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/object-copy/-/object-copy-0.1.0.tgz", + "integrity": "sha1-fn2Fi3gb18mRpBupde04EnVOmYw=", + "dev": true, + "requires": { + "copy-descriptor": "^0.1.0", + "define-property": "^0.2.5", + "kind-of": "^3.0.3" + }, + "dependencies": { + "define-property": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-0.2.5.tgz", + "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=", + "dev": true, + "requires": { + "is-descriptor": "^0.1.0" + } + } + } + }, + "object-keys": { + "version": "1.0.12", + "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.0.12.tgz", + "integrity": "sha512-FTMyFUm2wBcGHnH2eXmz7tC6IwlqQZ6mVZ+6dm6vZ4IQIHjs6FdNsQBuKGPuUUUY6NfJw2PshC08Tn6LzLDOag==", + "dev": true + }, + "object-visit": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/object-visit/-/object-visit-1.0.1.tgz", + "integrity": "sha1-95xEk68MU3e1n+OdOV5BBC3QRbs=", + "dev": true, + "requires": { + "isobject": "^3.0.0" + } + }, + "object.assign": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/object.assign/-/object.assign-4.1.0.tgz", + "integrity": "sha512-exHJeq6kBKj58mqGyTQ9DFvrZC/eR6OwxzoM9YRoGBqrXYonaFyGiFMuc9VZrXf7DarreEwMpurG3dd+CNyW5w==", + "dev": true, + "requires": { + "define-properties": "^1.1.2", + "function-bind": "^1.1.1", + "has-symbols": "^1.0.0", + "object-keys": "^1.0.11" + } + }, + "object.pick": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/object.pick/-/object.pick-1.3.0.tgz", + "integrity": "sha1-h6EKxMFpS9Lhy/U1kaZhQftd10c=", + "dev": true, + "requires": { + "isobject": "^3.0.1" + } + }, + "once": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", + "integrity": "sha1-WDsap3WWHUsROsF9nFC6753Xa9E=", + "dev": true, + "requires": { + "wrappy": "1" + } + }, + "onetime": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/onetime/-/onetime-2.0.1.tgz", + "integrity": "sha1-BnQoIw/WdEOyeUsiu6UotoZ5YtQ=", + "dev": true, + "requires": { + "mimic-fn": "^1.0.0" + } + }, + "optimist": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/optimist/-/optimist-0.6.1.tgz", + "integrity": "sha1-2j6nRob6IaGaERwybpDrFaAZZoY=", + "dev": true, + "requires": { + "minimist": "~0.0.1", + "wordwrap": "~0.0.2" + }, + "dependencies": { + "wordwrap": { + "version": "0.0.3", + "resolved": "https://registry.npmjs.org/wordwrap/-/wordwrap-0.0.3.tgz", + "integrity": "sha1-o9XabNXAvAAI03I0u68b7WMFkQc=", + "dev": true + } + } + }, + "optionator": { + "version": "0.8.2", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.8.2.tgz", + "integrity": "sha1-NkxeQJ0/TWMB1sC0wFu6UBgK62Q=", + "dev": true, + "requires": { + "deep-is": "~0.1.3", + "fast-levenshtein": "~2.0.4", + "levn": "~0.3.0", + "prelude-ls": "~1.1.2", + "type-check": "~0.3.2", + "wordwrap": "~1.0.0" + } + }, + "os-browserify": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/os-browserify/-/os-browserify-0.3.0.tgz", + "integrity": "sha1-hUNzx/XCMVkU/Jv8a9gjj92h7Cc=", + "dev": true + }, + "os-homedir": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/os-homedir/-/os-homedir-1.0.2.tgz", + "integrity": "sha1-/7xJiDNuDoM94MFox+8VISGqf7M=", + "dev": true + }, + "os-locale": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/os-locale/-/os-locale-2.1.0.tgz", + "integrity": "sha512-3sslG3zJbEYcaC4YVAvDorjGxc7tv6KVATnLPZONiljsUncvihe9BQoVCEs0RZ1kmf4Hk9OBqlZfJZWI4GanKA==", + "dev": true, + "requires": { + "execa": "^0.7.0", + "lcid": "^1.0.0", + "mem": "^1.1.0" + } + }, + "os-tmpdir": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/os-tmpdir/-/os-tmpdir-1.0.2.tgz", + "integrity": "sha1-u+Z0BseaqFxc/sdm/lc0VV36EnQ=", + "dev": true + }, + "p-finally": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-finally/-/p-finally-1.0.0.tgz", + "integrity": "sha1-P7z7FbiZpEEjs0ttzBi3JDNqLK4=", + "dev": true + }, + "p-limit": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-1.3.0.tgz", + "integrity": "sha512-vvcXsLAJ9Dr5rQOPk7toZQZJApBl2K4J6dANSsEuh6QI41JYcsS/qhTGa9ErIUUgK3WNQoJYvylxvjqmiqEA9Q==", + "dev": true, + "requires": { + "p-try": "^1.0.0" + } + }, + "p-locate": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-2.0.0.tgz", + "integrity": "sha1-IKAQOyIqcMj9OcwuWAaA893l7EM=", + "dev": true, + "requires": { + "p-limit": "^1.1.0" + } + }, + "p-try": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-1.0.0.tgz", + "integrity": "sha1-y8ec26+P1CKOE/Yh8rGiN8GyB7M=", + "dev": true + }, + "pako": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/pako/-/pako-1.0.6.tgz", + "integrity": "sha512-lQe48YPsMJAig+yngZ87Lus+NF+3mtu7DVOBu6b/gHO1YpKwIj5AWjZ/TOS7i46HD/UixzWb1zeWDZfGZ3iYcg==", + "dev": true + }, + "parallel-transform": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/parallel-transform/-/parallel-transform-1.1.0.tgz", + "integrity": "sha1-1BDwZbBdojCB/NEPKIVMKb2jOwY=", + "dev": true, + "requires": { + "cyclist": "~0.2.2", + "inherits": "^2.0.3", + "readable-stream": "^2.1.5" + } + }, + "parse-asn1": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/parse-asn1/-/parse-asn1-5.1.1.tgz", + "integrity": "sha512-KPx7flKXg775zZpnp9SxJlz00gTd4BmJ2yJufSc44gMCRrRQ7NSzAcSJQfifuOLgW6bEi+ftrALtsgALeB2Adw==", + "dev": true, + "requires": { + "asn1.js": "^4.0.0", + "browserify-aes": "^1.0.0", + "create-hash": "^1.1.0", + "evp_bytestokey": "^1.0.0", + "pbkdf2": "^3.0.3" + } + }, + "pascalcase": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/pascalcase/-/pascalcase-0.1.1.tgz", + "integrity": "sha1-s2PlXoAGym/iF4TS2yK9FdeRfxQ=", + "dev": true + }, + "path-browserify": { + "version": "0.0.0", + "resolved": "https://registry.npmjs.org/path-browserify/-/path-browserify-0.0.0.tgz", + "integrity": "sha1-oLhwcpquIUAFt9UDLsLLuw+0RRo=", + "dev": true + }, + "path-dirname": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/path-dirname/-/path-dirname-1.0.2.tgz", + "integrity": "sha1-zDPSTVJeCZpTiMAzbG4yuRYGCeA=", + "dev": true + }, + "path-exists": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz", + "integrity": "sha1-zg6+ql94yxiSXqfYENe1mwEP1RU=", + "dev": true + }, + "path-is-absolute": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", + "integrity": "sha1-F0uSaHNVNP+8es5r9TpanhtcX18=", + "dev": true + }, + "path-key": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-2.0.1.tgz", + "integrity": "sha1-QRyttXTFoUDTpLGRDUDYDMn0C0A=", + "dev": true + }, + "path-parse": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.5.tgz", + "integrity": "sha1-PBrfhx6pzWyUMbbqK9dKD/BVxME=", + "dev": true + }, + "path-type": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/path-type/-/path-type-3.0.0.tgz", + "integrity": "sha512-T2ZUsdZFHgA3u4e5PfPbjd7HDDpxPnQb5jN0SrDsjNSuVXHJqtwTnWqG0B1jZrgmJ/7lj1EmVIByWt1gxGkWvg==", + "dev": true, + "requires": { + "pify": "^3.0.0" + } + }, + "pbkdf2": { + "version": "3.0.16", + "resolved": "https://registry.npmjs.org/pbkdf2/-/pbkdf2-3.0.16.tgz", + "integrity": "sha512-y4CXP3thSxqf7c0qmOF+9UeOTrifiVTIM+u7NWlq+PRsHbr7r7dpCmvzrZxa96JJUNi0Y5w9VqG5ZNeCVMoDcA==", + "dev": true, + "requires": { + "create-hash": "^1.1.2", + "create-hmac": "^1.1.4", + "ripemd160": "^2.0.1", + "safe-buffer": "^5.0.1", + "sha.js": "^2.4.8" + } + }, + "pify": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/pify/-/pify-3.0.0.tgz", + "integrity": "sha1-5aSs0sEB/fPZpNB/DbxNtJ3SgXY=", + "dev": true + }, + "pkg-dir": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/pkg-dir/-/pkg-dir-2.0.0.tgz", + "integrity": "sha1-9tXREJ4Z1j7fQo4L1X4Sd3YVM0s=", + "dev": true, + "requires": { + "find-up": "^2.1.0" + } + }, + "posix-character-classes": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/posix-character-classes/-/posix-character-classes-0.1.1.tgz", + "integrity": "sha1-AerA/jta9xoqbAL+q7jB/vfgDqs=", + "dev": true + }, + "postcss": { + "version": "6.0.23", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.23.tgz", + "integrity": "sha512-soOk1h6J3VMTZtVeVpv15/Hpdl2cBLX3CAw4TAbkpTJiNPk9YP/zWcD1ND+xEtvyuuvKzbxliTOIyvkSeSJ6ag==", + "dev": true, + "requires": { + "chalk": "^2.4.1", + "source-map": "^0.6.1", + "supports-color": "^5.4.0" + }, + "dependencies": { + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "postcss-calc": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/postcss-calc/-/postcss-calc-5.3.1.tgz", + "integrity": "sha1-d7rnypKK2FcW4v2kLyYb98HWW14=", + "dev": true, + "requires": { + "postcss": "^5.0.2", + "postcss-message-helpers": "^2.0.0", + "reduce-css-calc": "^1.2.6" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-colormin": { + "version": "2.2.2", + "resolved": "https://registry.npmjs.org/postcss-colormin/-/postcss-colormin-2.2.2.tgz", + "integrity": "sha1-ZjFBfV8OkJo9fsJrJMio0eT5bks=", + "dev": true, + "requires": { + "colormin": "^1.0.5", + "postcss": "^5.0.13", + "postcss-value-parser": "^3.2.3" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-convert-values": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/postcss-convert-values/-/postcss-convert-values-2.6.1.tgz", + "integrity": "sha1-u9hZPFwf0uPRwyK7kl3K6Nrk1i0=", + "dev": true, + "requires": { + "postcss": "^5.0.11", + "postcss-value-parser": "^3.1.2" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-discard-comments": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/postcss-discard-comments/-/postcss-discard-comments-2.0.4.tgz", + "integrity": "sha1-vv6J+v1bPazlzM5Rt2uBUUvgDj0=", + "dev": true, + "requires": { + "postcss": "^5.0.14" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-discard-duplicates": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-duplicates/-/postcss-discard-duplicates-2.1.0.tgz", + "integrity": "sha1-uavye4isGIFYpesSq8riAmO5GTI=", + "dev": true, + "requires": { + "postcss": "^5.0.4" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-discard-empty": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/postcss-discard-empty/-/postcss-discard-empty-2.1.0.tgz", + "integrity": "sha1-0rS9nVztXr2Nyt52QMfXzX9PkrU=", + "dev": true, + "requires": { + "postcss": "^5.0.14" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-discard-overridden": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/postcss-discard-overridden/-/postcss-discard-overridden-0.1.1.tgz", + "integrity": "sha1-ix6vVU9ob7KIzYdMVWZ7CqNmjVg=", + "dev": true, + "requires": { + "postcss": "^5.0.16" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-discard-unused": { + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/postcss-discard-unused/-/postcss-discard-unused-2.2.3.tgz", + "integrity": "sha1-vOMLLMWR/8Y0Mitfs0ZLbZNPRDM=", + "dev": true, + "requires": { + "postcss": "^5.0.14", + "uniqs": "^2.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-filter-plugins": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/postcss-filter-plugins/-/postcss-filter-plugins-2.0.3.tgz", + "integrity": "sha512-T53GVFsdinJhgwm7rg1BzbeBRomOg9y5MBVhGcsV0CxurUdVj1UlPdKtn7aqYA/c/QVkzKMjq2bSV5dKG5+AwQ==", + "dev": true, + "requires": { + "postcss": "^5.0.4" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-merge-idents": { + "version": "2.1.7", + "resolved": "https://registry.npmjs.org/postcss-merge-idents/-/postcss-merge-idents-2.1.7.tgz", + "integrity": "sha1-TFUwMTwI4dWzu/PSu8dH4njuonA=", + "dev": true, + "requires": { + "has": "^1.0.1", + "postcss": "^5.0.10", + "postcss-value-parser": "^3.1.1" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-merge-longhand": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/postcss-merge-longhand/-/postcss-merge-longhand-2.0.2.tgz", + "integrity": "sha1-I9kM0Sewp3mUkVMyc5A0oaTz1lg=", + "dev": true, + "requires": { + "postcss": "^5.0.4" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-merge-rules": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/postcss-merge-rules/-/postcss-merge-rules-2.1.2.tgz", + "integrity": "sha1-0d9d+qexrMO+VT8OnhDofGG19yE=", + "dev": true, + "requires": { + "browserslist": "^1.5.2", + "caniuse-api": "^1.5.2", + "postcss": "^5.0.4", + "postcss-selector-parser": "^2.2.2", + "vendors": "^1.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-message-helpers": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/postcss-message-helpers/-/postcss-message-helpers-2.0.0.tgz", + "integrity": "sha1-pPL0+rbk/gAvCu0ABHjN9S+bpg4=", + "dev": true + }, + "postcss-minify-font-values": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/postcss-minify-font-values/-/postcss-minify-font-values-1.0.5.tgz", + "integrity": "sha1-S1jttWZB66fIR0qzUmyv17vey2k=", + "dev": true, + "requires": { + "object-assign": "^4.0.1", + "postcss": "^5.0.4", + "postcss-value-parser": "^3.0.2" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-minify-gradients": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/postcss-minify-gradients/-/postcss-minify-gradients-1.0.5.tgz", + "integrity": "sha1-Xb2hE3NwP4PPtKPqOIHY11/15uE=", + "dev": true, + "requires": { + "postcss": "^5.0.12", + "postcss-value-parser": "^3.3.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-minify-params": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/postcss-minify-params/-/postcss-minify-params-1.2.2.tgz", + "integrity": "sha1-rSzgcTc7lDs9kwo/pZo1jCjW8fM=", + "dev": true, + "requires": { + "alphanum-sort": "^1.0.1", + "postcss": "^5.0.2", + "postcss-value-parser": "^3.0.2", + "uniqs": "^2.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-minify-selectors": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/postcss-minify-selectors/-/postcss-minify-selectors-2.1.1.tgz", + "integrity": "sha1-ssapjAByz5G5MtGkllCBFDEXNb8=", + "dev": true, + "requires": { + "alphanum-sort": "^1.0.2", + "has": "^1.0.1", + "postcss": "^5.0.14", + "postcss-selector-parser": "^2.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-modules-extract-imports": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/postcss-modules-extract-imports/-/postcss-modules-extract-imports-1.1.0.tgz", + "integrity": "sha1-thTJcgvmgW6u41+zpfqh26agXds=", + "dev": true, + "requires": { + "postcss": "^6.0.1" + } + }, + "postcss-modules-local-by-default": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/postcss-modules-local-by-default/-/postcss-modules-local-by-default-1.2.0.tgz", + "integrity": "sha1-99gMOYxaOT+nlkRmvRlQCn1hwGk=", + "dev": true, + "requires": { + "css-selector-tokenizer": "^0.7.0", + "postcss": "^6.0.1" + } + }, + "postcss-modules-resolve-imports": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/postcss-modules-resolve-imports/-/postcss-modules-resolve-imports-1.3.0.tgz", + "integrity": "sha1-OY0wALla6WlCDN9M2D+oBn8cXq4=", + "dev": true, + "requires": { + "css-selector-tokenizer": "^0.7.0", + "icss-utils": "^3.0.1", + "minimist": "^1.2.0" + }, + "dependencies": { + "minimist": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.0.tgz", + "integrity": "sha1-o1AIsg9BOD7sH7kU9M1d95omQoQ=", + "dev": true + } + } + }, + "postcss-modules-scope": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-1.1.0.tgz", + "integrity": "sha1-1upkmUx5+XtipytCb75gVqGUu5A=", + "dev": true, + "requires": { + "css-selector-tokenizer": "^0.7.0", + "postcss": "^6.0.1" + } + }, + "postcss-modules-values": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/postcss-modules-values/-/postcss-modules-values-1.3.0.tgz", + "integrity": "sha1-7P+p1+GSUYOJ9CrQ6D9yrsRW6iA=", + "dev": true, + "requires": { + "icss-replace-symbols": "^1.1.0", + "postcss": "^6.0.1" + } + }, + "postcss-normalize-charset": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/postcss-normalize-charset/-/postcss-normalize-charset-1.1.1.tgz", + "integrity": "sha1-757nEhLX/nWceO0WL2HtYrXLk/E=", + "dev": true, + "requires": { + "postcss": "^5.0.5" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-normalize-url": { + "version": "3.0.8", + "resolved": "https://registry.npmjs.org/postcss-normalize-url/-/postcss-normalize-url-3.0.8.tgz", + "integrity": "sha1-EI90s/L82viRov+j6kWSJ5/HgiI=", + "dev": true, + "requires": { + "is-absolute-url": "^2.0.0", + "normalize-url": "^1.4.0", + "postcss": "^5.0.14", + "postcss-value-parser": "^3.2.3" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-ordered-values": { + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/postcss-ordered-values/-/postcss-ordered-values-2.2.3.tgz", + "integrity": "sha1-7sbCpntsQSqNsgQud/6NpD+VwR0=", + "dev": true, + "requires": { + "postcss": "^5.0.4", + "postcss-value-parser": "^3.0.1" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-reduce-idents": { + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/postcss-reduce-idents/-/postcss-reduce-idents-2.4.0.tgz", + "integrity": "sha1-wsbSDMlYKE9qv75j92Cb9AkFmtM=", + "dev": true, + "requires": { + "postcss": "^5.0.4", + "postcss-value-parser": "^3.0.2" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-reduce-initial": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/postcss-reduce-initial/-/postcss-reduce-initial-1.0.1.tgz", + "integrity": "sha1-aPgGlfBF0IJjqHmtJA343WT2ROo=", + "dev": true, + "requires": { + "postcss": "^5.0.4" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-reduce-transforms": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/postcss-reduce-transforms/-/postcss-reduce-transforms-1.0.4.tgz", + "integrity": "sha1-/3b02CEkN7McKYpC0uFEQCV3GuE=", + "dev": true, + "requires": { + "has": "^1.0.1", + "postcss": "^5.0.8", + "postcss-value-parser": "^3.0.1" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-selector-parser": { + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-2.2.3.tgz", + "integrity": "sha1-+UN3iGBsPJrO4W/+jYsWKX8nu5A=", + "dev": true, + "requires": { + "flatten": "^1.0.2", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + }, + "postcss-svgo": { + "version": "2.1.6", + "resolved": "https://registry.npmjs.org/postcss-svgo/-/postcss-svgo-2.1.6.tgz", + "integrity": "sha1-tt8YqmE7Zm4TPwittSGcJoSsEI0=", + "dev": true, + "requires": { + "is-svg": "^2.0.0", + "postcss": "^5.0.14", + "postcss-value-parser": "^3.2.3", + "svgo": "^0.7.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-unique-selectors": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/postcss-unique-selectors/-/postcss-unique-selectors-2.0.2.tgz", + "integrity": "sha1-mB1X0p3csz57Hf4f1DuGSfkzyh0=", + "dev": true, + "requires": { + "alphanum-sort": "^1.0.1", + "postcss": "^5.0.4", + "uniqs": "^2.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "postcss-value-parser": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/postcss-value-parser/-/postcss-value-parser-3.3.0.tgz", + "integrity": "sha1-h/OPnxj3dKSrTIojL1xc6IcqnRU=", + "dev": true + }, + "postcss-zindex": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/postcss-zindex/-/postcss-zindex-2.2.0.tgz", + "integrity": "sha1-0hCd3AVbka9n/EyzsCWUZjnSryI=", + "dev": true, + "requires": { + "has": "^1.0.1", + "postcss": "^5.0.4", + "uniqs": "^2.0.0" + }, + "dependencies": { + "postcss": { + "version": "5.2.18", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-5.2.18.tgz", + "integrity": "sha512-zrUjRRe1bpXKsX1qAJNJjqZViErVuyEkMTRrwu4ud4sbTtIBRmtaYDrHmcGgmrbsW3MHfmtIf+vJumgQn+PrXg==", + "dev": true, + "requires": { + "chalk": "^1.1.3", + "js-base64": "^2.1.9", + "source-map": "^0.5.6", + "supports-color": "^3.2.3" + } + }, + "supports-color": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz", + "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=", + "dev": true, + "requires": { + "has-flag": "^1.0.0" + } + } + } + }, + "prelude-ls": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.1.2.tgz", + "integrity": "sha1-IZMqVJ9eUv/ZqCf1cOBL5iqX2lQ=", + "dev": true + }, + "prepend-http": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/prepend-http/-/prepend-http-1.0.4.tgz", + "integrity": "sha1-1PRWKwzjaW5BrFLQ4ALlemNdxtw=", + "dev": true + }, + "private": { + "version": "0.1.8", + "resolved": "https://registry.npmjs.org/private/-/private-0.1.8.tgz", + "integrity": "sha512-VvivMrbvd2nKkiG38qjULzlc+4Vx4wm/whI9pQD35YrARNnhxeiRktSOhSukRLFNlzg6Br/cJPet5J/u19r/mg==", + "dev": true + }, + "process": { + "version": "0.11.10", + "resolved": "https://registry.npmjs.org/process/-/process-0.11.10.tgz", + "integrity": "sha1-czIwDoQBYb2j5podHZGn1LwW8YI=", + "dev": true + }, + "process-nextick-args": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/process-nextick-args/-/process-nextick-args-2.0.0.tgz", + "integrity": "sha512-MtEC1TqN0EU5nephaJ4rAtThHtC86dNN9qCuEhtshvpVBkAW5ZO7BASN9REnF9eoXGcRub+pFuKEpOHE+HbEMw==", + "dev": true + }, + "promise-inflight": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/promise-inflight/-/promise-inflight-1.0.1.tgz", + "integrity": "sha1-mEcocL8igTL8vdhoEputEsPAKeM=", + "dev": true + }, + "prr": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/prr/-/prr-1.0.1.tgz", + "integrity": "sha1-0/wRS6BplaRexok/SEzrHXj19HY=", + "dev": true + }, + "pseudomap": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/pseudomap/-/pseudomap-1.0.2.tgz", + "integrity": "sha1-8FKijacOYYkX7wqKw0wa5aaChrM=", + "dev": true + }, + "public-encrypt": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/public-encrypt/-/public-encrypt-4.0.2.tgz", + "integrity": "sha512-4kJ5Esocg8X3h8YgJsKAuoesBgB7mqH3eowiDzMUPKiRDDE7E/BqqZD1hnTByIaAFiwAw246YEltSq7tdrOH0Q==", + "dev": true, + "requires": { + "bn.js": "^4.1.0", + "browserify-rsa": "^4.0.0", + "create-hash": "^1.1.0", + "parse-asn1": "^5.0.0", + "randombytes": "^2.0.1" + } + }, + "pump": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/pump/-/pump-2.0.1.tgz", + "integrity": "sha512-ruPMNRkN3MHP1cWJc9OWr+T/xDP0jhXYCLfJcBuX54hhfIBnaQmAUMfDcG4DM5UMWByBbJY69QSphm3jtDKIkA==", + "dev": true, + "requires": { + "end-of-stream": "^1.1.0", + "once": "^1.3.1" + } + }, + "pumpify": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/pumpify/-/pumpify-1.5.1.tgz", + "integrity": "sha512-oClZI37HvuUJJxSKKrC17bZ9Cu0ZYhEAGPsPUy9KlMUmv9dKX2o77RUmq7f3XjIxbwyGwYzbzQ1L2Ks8sIradQ==", + "dev": true, + "requires": { + "duplexify": "^3.6.0", + "inherits": "^2.0.3", + "pump": "^2.0.0" + } + }, + "punycode": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.1.1.tgz", + "integrity": "sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A==", + "dev": true + }, + "q": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/q/-/q-1.5.1.tgz", + "integrity": "sha1-fjL3W0E4EpHQRhHxvxQQmsAGUdc=", + "dev": true + }, + "query-string": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/query-string/-/query-string-4.3.4.tgz", + "integrity": "sha1-u7aTucqRXCMlFbIosaArYJBD2+s=", + "dev": true, + "requires": { + "object-assign": "^4.1.0", + "strict-uri-encode": "^1.0.0" + } + }, + "querystring": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/querystring/-/querystring-0.2.0.tgz", + "integrity": "sha1-sgmEkgO7Jd+CDadW50cAWHhSFiA=", + "dev": true + }, + "querystring-es3": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/querystring-es3/-/querystring-es3-0.2.1.tgz", + "integrity": "sha1-nsYfeQSYdXB9aUFFlv2Qek1xHnM=", + "dev": true + }, + "randombytes": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.0.6.tgz", + "integrity": "sha512-CIQ5OFxf4Jou6uOKe9t1AOgqpeU5fd70A8NPdHSGeYXqXsPe6peOwI0cUl88RWZ6sP1vPMV3avd/R6cZ5/sP1A==", + "dev": true, + "requires": { + "safe-buffer": "^5.1.0" + } + }, + "randomfill": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/randomfill/-/randomfill-1.0.4.tgz", + "integrity": "sha512-87lcbR8+MhcWcUiQ+9e+Rwx8MyR2P7qnt15ynUlbm3TU/fjbgz4GsvfSUDTemtCCtVCqb4ZcEFlyPNTh9bBTLw==", + "dev": true, + "requires": { + "randombytes": "^2.0.5", + "safe-buffer": "^5.1.0" + } + }, + "readable-stream": { + "version": "2.3.6", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-2.3.6.tgz", + "integrity": "sha512-tQtKA9WIAhBF3+VLAseyMqZeBjW0AHJoxOtYqSUZNJxauErmLbVm2FW1y+J/YA9dUrAC39ITejlZWhVIwawkKw==", + "dev": true, + "requires": { + "core-util-is": "~1.0.0", + "inherits": "~2.0.3", + "isarray": "~1.0.0", + "process-nextick-args": "~2.0.0", + "safe-buffer": "~5.1.1", + "string_decoder": "~1.1.1", + "util-deprecate": "~1.0.1" + } + }, + "readdirp": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/readdirp/-/readdirp-2.1.0.tgz", + "integrity": "sha1-TtCtBg3zBzMAxIRANz9y0cxkLXg=", + "dev": true, + "requires": { + "graceful-fs": "^4.1.2", + "minimatch": "^3.0.2", + "readable-stream": "^2.0.2", + "set-immediate-shim": "^1.0.1" + } + }, + "reduce-css-calc": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/reduce-css-calc/-/reduce-css-calc-1.3.0.tgz", + "integrity": "sha1-dHyRTgSWFKTJz7umKYca0dKSdxY=", + "dev": true, + "requires": { + "balanced-match": "^0.4.2", + "math-expression-evaluator": "^1.2.14", + "reduce-function-call": "^1.0.1" + }, + "dependencies": { + "balanced-match": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-0.4.2.tgz", + "integrity": "sha1-yz8+PHMtwPAe5wtAPzAuYddwmDg=", + "dev": true + } + } + }, + "reduce-function-call": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/reduce-function-call/-/reduce-function-call-1.0.2.tgz", + "integrity": "sha1-WiAL+S4ON3UXUv5FsKszD9S2vpk=", + "dev": true, + "requires": { + "balanced-match": "^0.4.2" + }, + "dependencies": { + "balanced-match": { + "version": "0.4.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-0.4.2.tgz", + "integrity": "sha1-yz8+PHMtwPAe5wtAPzAuYddwmDg=", + "dev": true + } + } + }, + "regenerate": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/regenerate/-/regenerate-1.4.0.tgz", + "integrity": "sha512-1G6jJVDWrt0rK99kBjvEtziZNCICAuvIPkSiUFIQxVP06RCVpq3dmDo2oi6ABpYaDYaTRr67BEhL8r1wgEZZKg==", + "dev": true + }, + "regenerator-runtime": { + "version": "0.11.1", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.11.1.tgz", + "integrity": "sha512-MguG95oij0fC3QV3URf4V2SDYGJhJnJGqvIIgdECeODCT98wSWDAJ94SSuVpYQUoTcGUIL6L4yNB7j1DFFHSBg==", + "dev": true + }, + "regex-not": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/regex-not/-/regex-not-1.0.2.tgz", + "integrity": "sha512-J6SDjUgDxQj5NusnOtdFxDwN/+HWykR8GELwctJ7mdqhcyy1xEc4SRFHUXvxTp661YaVKAjfRLZ9cCqS6tn32A==", + "dev": true, + "requires": { + "extend-shallow": "^3.0.2", + "safe-regex": "^1.1.0" + } + }, + "regexpu-core": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/regexpu-core/-/regexpu-core-1.0.0.tgz", + "integrity": "sha1-hqdj9Y7k18L2sQLkdkBQ3n7ZDGs=", + "dev": true, + "requires": { + "regenerate": "^1.2.1", + "regjsgen": "^0.2.0", + "regjsparser": "^0.1.4" + } + }, + "regjsgen": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/regjsgen/-/regjsgen-0.2.0.tgz", + "integrity": "sha1-bAFq3qxVT3WCP+N6wFuS1aTtsfc=", + "dev": true + }, + "regjsparser": { + "version": "0.1.5", + "resolved": "https://registry.npmjs.org/regjsparser/-/regjsparser-0.1.5.tgz", + "integrity": "sha1-fuj4Tcb6eS0/0K4ijSS9lJ6tIFw=", + "dev": true, + "requires": { + "jsesc": "~0.5.0" + }, + "dependencies": { + "jsesc": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-0.5.0.tgz", + "integrity": "sha1-597mbjXW/Bb3EP6R1c9p9w8IkR0=", + "dev": true + } + } + }, + "remove-trailing-separator": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/remove-trailing-separator/-/remove-trailing-separator-1.1.0.tgz", + "integrity": "sha1-wkvOKig62tW8P1jg1IJJuSN52O8=", + "dev": true + }, + "repeat-element": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/repeat-element/-/repeat-element-1.1.2.tgz", + "integrity": "sha1-7wiaF40Ug7quTZPrmLT55OEdmQo=", + "dev": true + }, + "repeat-string": { + "version": "1.6.1", + "resolved": "https://registry.npmjs.org/repeat-string/-/repeat-string-1.6.1.tgz", + "integrity": "sha1-jcrkcOHIirwtYA//Sndihtp15jc=", + "dev": true + }, + "repeating": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/repeating/-/repeating-2.0.1.tgz", + "integrity": "sha1-UhTFOpJtNVJwdSf7q0FdvAjQbdo=", + "dev": true, + "requires": { + "is-finite": "^1.0.0" + } + }, + "require-directory": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/require-directory/-/require-directory-2.1.1.tgz", + "integrity": "sha1-jGStX9MNqxyXbiNE/+f3kqam30I=", + "dev": true + }, + "require-main-filename": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-1.0.1.tgz", + "integrity": "sha1-l/cXtp1IeE9fUmpsWqj/3aBVpNE=", + "dev": true + }, + "resolve": { + "version": "1.8.1", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.8.1.tgz", + "integrity": "sha512-AicPrAC7Qu1JxPCZ9ZgCZlY35QgFnNqc+0LtbRNxnVw4TXvjQ72wnuL9JQcEBgXkI9JM8MsT9kaQoHcpCRJOYA==", + "dev": true, + "requires": { + "path-parse": "^1.0.5" + } + }, + "resolve-cwd": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/resolve-cwd/-/resolve-cwd-2.0.0.tgz", + "integrity": "sha1-AKn3OHVW4nA46uIyyqNypqWbZlo=", + "dev": true, + "requires": { + "resolve-from": "^3.0.0" + } + }, + "resolve-from": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-3.0.0.tgz", + "integrity": "sha1-six699nWiBvItuZTM17rywoYh0g=", + "dev": true + }, + "resolve-url": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/resolve-url/-/resolve-url-0.2.1.tgz", + "integrity": "sha1-LGN/53yJOv0qZj/iGqkIAGjiBSo=", + "dev": true + }, + "restore-cursor": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/restore-cursor/-/restore-cursor-2.0.0.tgz", + "integrity": "sha1-n37ih/gv0ybU/RYpI9YhKe7g368=", + "dev": true, + "requires": { + "onetime": "^2.0.0", + "signal-exit": "^3.0.2" + } + }, + "ret": { + "version": "0.1.15", + "resolved": "https://registry.npmjs.org/ret/-/ret-0.1.15.tgz", + "integrity": "sha512-TTlYpa+OL+vMMNG24xSlQGEJ3B/RzEfUlLct7b5G/ytav+wPrplCpVMFuwzXbkecJrb6IYo1iFb0S9v37754mg==", + "dev": true + }, + "right-align": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/right-align/-/right-align-0.1.3.tgz", + "integrity": "sha1-YTObci/mo1FWiSENJOFMlhSGE+8=", + "dev": true, + "optional": true, + "requires": { + "align-text": "^0.1.1" + } + }, + "rimraf": { + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-2.6.2.tgz", + "integrity": "sha512-lreewLK/BlghmxtfH36YYVg1i8IAce4TI7oao75I1g245+6BctqTVQiBP3YUJ9C6DQOXJmkYR9X9fCLtCOJc5w==", + "dev": true, + "requires": { + "glob": "^7.0.5" + }, + "dependencies": { + "glob": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.2.tgz", + "integrity": "sha512-MJTUg1kjuLeQCJ+ccE4Vpa6kKVXkPYJ2mOCQyUuKLcLQsdrMCpBPUi8qVE6+YuaJkozeA9NusTAw3hLr8Xe5EQ==", + "dev": true, + "requires": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.0.4", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + } + } + } + }, + "ripemd160": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/ripemd160/-/ripemd160-2.0.2.tgz", + "integrity": "sha512-ii4iagi25WusVoiC4B4lq7pbXfAp3D9v5CwfkY33vffw2+pkDjY1D8GaN7spsxvCSx8dkPqOZCEZyfxcmJG2IA==", + "dev": true, + "requires": { + "hash-base": "^3.0.0", + "inherits": "^2.0.1" + } + }, + "run-async": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/run-async/-/run-async-2.3.0.tgz", + "integrity": "sha1-A3GrSuC91yDUFm19/aZP96RFpsA=", + "dev": true, + "requires": { + "is-promise": "^2.1.0" + } + }, + "run-queue": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/run-queue/-/run-queue-1.0.3.tgz", + "integrity": "sha1-6Eg5bwV9Ij8kOGkkYY4laUFh7Ec=", + "dev": true, + "requires": { + "aproba": "^1.1.1" + } + }, + "rxjs": { + "version": "6.2.1", + "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-6.2.1.tgz", + "integrity": "sha512-OwMxHxmnmHTUpgO+V7dZChf3Tixf4ih95cmXjzzadULziVl/FKhHScGLj4goEw9weePVOH2Q0+GcCBUhKCZc/g==", + "dev": true, + "requires": { + "tslib": "^1.9.0" + } + }, + "safe-buffer": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.1.2.tgz", + "integrity": "sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g==", + "dev": true + }, + "safe-regex": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/safe-regex/-/safe-regex-1.1.0.tgz", + "integrity": "sha1-QKNmnzsHfR6UPURinhV91IAjvy4=", + "dev": true, + "requires": { + "ret": "~0.1.10" + } + }, + "safer-buffer": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", + "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==", + "dev": true + }, + "sax": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/sax/-/sax-1.2.4.tgz", + "integrity": "sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw==", + "dev": true + }, + "schema-utils": { + "version": "0.4.5", + "resolved": "https://registry.npmjs.org/schema-utils/-/schema-utils-0.4.5.tgz", + "integrity": "sha512-yYrjb9TX2k/J1Y5UNy3KYdZq10xhYcF8nMpAW6o3hy6Q8WSIEf9lJHG/ePnOBfziPM3fvQwfOwa13U/Fh8qTfA==", + "dev": true, + "requires": { + "ajv": "^6.1.0", + "ajv-keywords": "^3.1.0" + } + }, + "seekout": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/seekout/-/seekout-1.0.2.tgz", + "integrity": "sha1-CbqfG9W0b7sTRxjrGaaDgsuxuck=", + "dev": true + }, + "semver": { + "version": "5.5.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.5.0.tgz", + "integrity": "sha512-4SJ3dm0WAwWy/NVeioZh5AntkdJoWKxHxcmyP622fOkgHa4z3R0TdBJICINyaSDE6uNwVc8gZr+ZinwZAH4xIA==", + "dev": true + }, + "serialize-javascript": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-1.5.0.tgz", + "integrity": "sha512-Ga8c8NjAAp46Br4+0oZ2WxJCwIzwP60Gq1YPgU+39PiTVxyed/iKE/zyZI6+UlVYH5Q4PaQdHhcegIFPZTUfoQ==", + "dev": true + }, + "set-blocking": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/set-blocking/-/set-blocking-2.0.0.tgz", + "integrity": "sha1-BF+XgtARrppoA93TgrJDkrPYkPc=", + "dev": true + }, + "set-immediate-shim": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/set-immediate-shim/-/set-immediate-shim-1.0.1.tgz", + "integrity": "sha1-SysbJ+uAip+NzEgaWOXlb1mfP2E=", + "dev": true + }, + "set-value": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/set-value/-/set-value-2.0.0.tgz", + "integrity": "sha512-hw0yxk9GT/Hr5yJEYnHNKYXkIA8mVJgd9ditYZCe16ZczcaELYYcfvaXesNACk2O8O0nTiPQcQhGUQj8JLzeeg==", + "dev": true, + "requires": { + "extend-shallow": "^2.0.1", + "is-extendable": "^0.1.1", + "is-plain-object": "^2.0.3", + "split-string": "^3.0.1" + }, + "dependencies": { + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + } + } + }, + "setimmediate": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/setimmediate/-/setimmediate-1.0.5.tgz", + "integrity": "sha1-KQy7Iy4waULX1+qbg3Mqt4VvgoU=", + "dev": true + }, + "sha.js": { + "version": "2.4.11", + "resolved": "https://registry.npmjs.org/sha.js/-/sha.js-2.4.11.tgz", + "integrity": "sha512-QMEp5B7cftE7APOjk5Y6xgrbWu+WkLVQwk8JNjZ8nKRciZaByEW6MubieAiToS7+dwvrjGhH8jRXz3MVd0AYqQ==", + "dev": true, + "requires": { + "inherits": "^2.0.1", + "safe-buffer": "^5.0.1" + } + }, + "shebang-command": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-1.2.0.tgz", + "integrity": "sha1-RKrGW2lbAzmJaMOfNj/uXer98eo=", + "dev": true, + "requires": { + "shebang-regex": "^1.0.0" + } + }, + "shebang-regex": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-1.0.0.tgz", + "integrity": "sha1-2kL0l0DAtC2yypcoVxyxkMmO/qM=", + "dev": true + }, + "signal-exit": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.2.tgz", + "integrity": "sha1-tf3AjxKH6hF4Yo5BXiUTK3NkbG0=", + "dev": true + }, + "slash": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/slash/-/slash-1.0.0.tgz", + "integrity": "sha1-xB8vbDn8FtHNF61LXYlhFK5HDVU=", + "dev": true + }, + "snapdragon": { + "version": "0.8.2", + "resolved": "https://registry.npmjs.org/snapdragon/-/snapdragon-0.8.2.tgz", + "integrity": "sha512-FtyOnWN/wCHTVXOMwvSv26d+ko5vWlIDD6zoUJ7LW8vh+ZBC8QdljveRP+crNrtBwioEUWy/4dMtbBjA4ioNlg==", + "dev": true, + "requires": { + "base": "^0.11.1", + "debug": "^2.2.0", + "define-property": "^0.2.5", + "extend-shallow": "^2.0.1", + "map-cache": "^0.2.2", + "source-map": "^0.5.6", + "source-map-resolve": "^0.5.0", + "use": "^3.1.0" + }, + "dependencies": { + "define-property": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-0.2.5.tgz", + "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=", + "dev": true, + "requires": { + "is-descriptor": "^0.1.0" + } + }, + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + } + } + }, + "snapdragon-node": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/snapdragon-node/-/snapdragon-node-2.1.1.tgz", + "integrity": "sha512-O27l4xaMYt/RSQ5TR3vpWCAB5Kb/czIcqUFOM/C4fYcLnbZUc1PkjTAMjof2pBWaSTwOUd6qUHcFGVGj7aIwnw==", + "dev": true, + "requires": { + "define-property": "^1.0.0", + "isobject": "^3.0.0", + "snapdragon-util": "^3.0.1" + }, + "dependencies": { + "define-property": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", + "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", + "dev": true, + "requires": { + "is-descriptor": "^1.0.0" + } + }, + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "dev": true, + "requires": { + "kind-of": "^6.0.0" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "^1.0.0", + "is-data-descriptor": "^1.0.0", + "kind-of": "^6.0.2" + } + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "snapdragon-util": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/snapdragon-util/-/snapdragon-util-3.0.1.tgz", + "integrity": "sha512-mbKkMdQKsjX4BAL4bRYTj21edOf8cN7XHdYUJEe+Zn99hVEYcMvKPct1IqNe7+AZPirn8BCDOQBHQZknqmKlZQ==", + "dev": true, + "requires": { + "kind-of": "^3.2.0" + } + }, + "sort-keys": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/sort-keys/-/sort-keys-1.1.2.tgz", + "integrity": "sha1-RBttTTRnmPG05J6JIK37oOVD+a0=", + "dev": true, + "requires": { + "is-plain-obj": "^1.0.0" + } + }, + "source-list-map": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/source-list-map/-/source-list-map-2.0.0.tgz", + "integrity": "sha512-I2UmuJSRr/T8jisiROLU3A3ltr+swpniSmNPI4Ml3ZCX6tVnDsuZzK7F2hl5jTqbZBWCEKlj5HRQiPExXLgE8A==", + "dev": true + }, + "source-map": { + "version": "0.5.7", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz", + "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=", + "dev": true + }, + "source-map-resolve": { + "version": "0.5.2", + "resolved": "https://registry.npmjs.org/source-map-resolve/-/source-map-resolve-0.5.2.tgz", + "integrity": "sha512-MjqsvNwyz1s0k81Goz/9vRBe9SZdB09Bdw+/zYyO+3CuPk6fouTaxscHkgtE8jKvf01kVfl8riHzERQ/kefaSA==", + "dev": true, + "requires": { + "atob": "^2.1.1", + "decode-uri-component": "^0.2.0", + "resolve-url": "^0.2.1", + "source-map-url": "^0.4.0", + "urix": "^0.1.0" + } + }, + "source-map-support": { + "version": "0.4.18", + "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.4.18.tgz", + "integrity": "sha512-try0/JqxPLF9nOjvSta7tVondkP5dwgyLDjVoyMDlmjugT2lRZ1OfsrYTkCd2hkDnJTKRbO/Rl3orm8vlsUzbA==", + "dev": true, + "requires": { + "source-map": "^0.5.6" + } + }, + "source-map-url": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/source-map-url/-/source-map-url-0.4.0.tgz", + "integrity": "sha1-PpNdfd1zYxuXZZlW1VEo6HtQhKM=", + "dev": true + }, + "split-string": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/split-string/-/split-string-3.1.0.tgz", + "integrity": "sha512-NzNVhJDYpwceVVii8/Hu6DKfD2G+NrQHlS/V/qgv763EYudVwEcMQNxd2lh+0VrUByXN/oJkl5grOhYWvQUYiw==", + "dev": true, + "requires": { + "extend-shallow": "^3.0.0" + } + }, + "sprintf-js": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", + "integrity": "sha1-BOaSb2YolTVPPdAVIDYzuFcpfiw=", + "dev": true + }, + "ssri": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/ssri/-/ssri-5.3.0.tgz", + "integrity": "sha512-XRSIPqLij52MtgoQavH/x/dU1qVKtWUAAZeOHsR9c2Ddi4XerFy3mc1alf+dLJKl9EUIm/Ht+EowFkTUOA6GAQ==", + "dev": true, + "requires": { + "safe-buffer": "^5.1.1" + } + }, + "static-extend": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/static-extend/-/static-extend-0.1.2.tgz", + "integrity": "sha1-YICcOcv/VTNyJv1eC1IPNB8ftcY=", + "dev": true, + "requires": { + "define-property": "^0.2.5", + "object-copy": "^0.1.0" + }, + "dependencies": { + "define-property": { + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/define-property/-/define-property-0.2.5.tgz", + "integrity": "sha1-w1se+RjsPJkPmlvFe+BKrOxcgRY=", + "dev": true, + "requires": { + "is-descriptor": "^0.1.0" + } + } + } + }, + "stream-browserify": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/stream-browserify/-/stream-browserify-2.0.1.tgz", + "integrity": "sha1-ZiZu5fm9uZQKTkUUyvtDu3Hlyds=", + "dev": true, + "requires": { + "inherits": "~2.0.1", + "readable-stream": "^2.0.2" + } + }, + "stream-each": { + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/stream-each/-/stream-each-1.2.2.tgz", + "integrity": "sha512-mc1dbFhGBxvTM3bIWmAAINbqiuAk9TATcfIQC8P+/+HJefgaiTlMn2dHvkX8qlI12KeYKSQ1Ua9RrIqrn1VPoA==", + "dev": true, + "requires": { + "end-of-stream": "^1.1.0", + "stream-shift": "^1.0.0" + } + }, + "stream-http": { + "version": "2.8.3", + "resolved": "https://registry.npmjs.org/stream-http/-/stream-http-2.8.3.tgz", + "integrity": "sha512-+TSkfINHDo4J+ZobQLWiMouQYB+UVYFttRA94FpEzzJ7ZdqcL4uUUQ7WkdkI4DSozGmgBUE/a47L+38PenXhUw==", + "dev": true, + "requires": { + "builtin-status-codes": "^3.0.0", + "inherits": "^2.0.1", + "readable-stream": "^2.3.6", + "to-arraybuffer": "^1.0.0", + "xtend": "^4.0.0" + } + }, + "stream-shift": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/stream-shift/-/stream-shift-1.0.0.tgz", + "integrity": "sha1-1cdSgl5TZ+eG944Y5EXqIjoVWVI=", + "dev": true + }, + "strict-uri-encode": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/strict-uri-encode/-/strict-uri-encode-1.1.0.tgz", + "integrity": "sha1-J5siXfHVgrH1TmWt3UNS4Y+qBxM=", + "dev": true + }, + "string-width": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-2.1.1.tgz", + "integrity": "sha512-nOqH59deCq9SRHlxq1Aw85Jnt4w6KvLKqWVik6oA9ZklXLNIOlqg4F2yrT1MVaTjAqvVwdfeZ7w7aCvJD7ugkw==", + "dev": true, + "requires": { + "is-fullwidth-code-point": "^2.0.0", + "strip-ansi": "^4.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-3.0.0.tgz", + "integrity": "sha1-7QMXwyIGT3lGbAKWa922Bas32Zg=", + "dev": true + }, + "strip-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-4.0.0.tgz", + "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=", + "dev": true, + "requires": { + "ansi-regex": "^3.0.0" + } + } + } + }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "dev": true, + "requires": { + "safe-buffer": "~5.1.0" + } + }, + "strip-ansi": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", + "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", + "dev": true, + "requires": { + "ansi-regex": "^2.0.0" + } + }, + "strip-eof": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/strip-eof/-/strip-eof-1.0.0.tgz", + "integrity": "sha1-u0P/VZim6wXYm1n80SnJgzE2Br8=", + "dev": true + }, + "style-loader": { + "version": "0.21.0", + "resolved": "https://registry.npmjs.org/style-loader/-/style-loader-0.21.0.tgz", + "integrity": "sha512-T+UNsAcl3Yg+BsPKs1vd22Fr8sVT+CJMtzqc6LEw9bbJZb43lm9GoeIfUcDEefBSWC0BhYbcdupV1GtI4DGzxg==", + "dev": true, + "requires": { + "loader-utils": "^1.1.0", + "schema-utils": "^0.4.5" + } + }, + "supports-color": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-2.0.0.tgz", + "integrity": "sha1-U10EXOa2Nj+kARcIRimZXp3zJMc=", + "dev": true + }, + "svgo": { + "version": "0.7.2", + "resolved": "https://registry.npmjs.org/svgo/-/svgo-0.7.2.tgz", + "integrity": "sha1-n1dyQTlSE1xv779Ar+ak+qiLS7U=", + "dev": true, + "requires": { + "coa": "~1.0.1", + "colors": "~1.1.2", + "csso": "~2.3.1", + "js-yaml": "~3.7.0", + "mkdirp": "~0.5.1", + "sax": "~1.2.1", + "whet.extend": "~0.9.9" + }, + "dependencies": { + "js-yaml": { + "version": "3.7.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.7.0.tgz", + "integrity": "sha1-XJZ93YN6m/3KXy3oQlOr6KHAO4A=", + "dev": true, + "requires": { + "argparse": "^1.0.7", + "esprima": "^2.6.0" + } + } + } + }, + "tapable": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-1.0.0.tgz", + "integrity": "sha512-dQRhbNQkRnaqauC7WqSJ21EEksgT0fYZX2lqXzGkpo8JNig9zGZTYoMGvyI2nWmXlE2VSVXVDu7wLVGu/mQEsg==", + "dev": true + }, + "text-table": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/text-table/-/text-table-0.2.0.tgz", + "integrity": "sha1-f17oI66AUgfACvLfSoTsP8+lcLQ=", + "dev": true + }, + "through": { + "version": "2.3.8", + "resolved": "https://registry.npmjs.org/through/-/through-2.3.8.tgz", + "integrity": "sha1-DdTJ/6q8NXlgsbckEV1+Doai4fU=", + "dev": true + }, + "through2": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/through2/-/through2-2.0.3.tgz", + "integrity": "sha1-AARWmzfHx0ujnEPzzteNGtlBQL4=", + "dev": true, + "requires": { + "readable-stream": "^2.1.5", + "xtend": "~4.0.1" + } + }, + "timers-browserify": { + "version": "2.0.10", + "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.10.tgz", + "integrity": "sha512-YvC1SV1XdOUaL6gx5CoGroT3Gu49pK9+TZ38ErPldOWW4j49GI1HKs9DV+KGq/w6y+LZ72W1c8cKz2vzY+qpzg==", + "dev": true, + "requires": { + "setimmediate": "^1.0.4" + } + }, + "tmp": { + "version": "0.0.33", + "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz", + "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==", + "dev": true, + "requires": { + "os-tmpdir": "~1.0.2" + } + }, + "to-arraybuffer": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/to-arraybuffer/-/to-arraybuffer-1.0.1.tgz", + "integrity": "sha1-fSKbH8xjfkZsoIEYCDanqr/4P0M=", + "dev": true + }, + "to-fast-properties": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/to-fast-properties/-/to-fast-properties-1.0.3.tgz", + "integrity": "sha1-uDVx+k2MJbguIxsG46MFXeTKGkc=", + "dev": true + }, + "to-object-path": { + "version": "0.3.0", + "resolved": "https://registry.npmjs.org/to-object-path/-/to-object-path-0.3.0.tgz", + "integrity": "sha1-KXWIt7Dn4KwI4E5nL4XB9JmeF68=", + "dev": true, + "requires": { + "kind-of": "^3.0.2" + } + }, + "to-regex": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/to-regex/-/to-regex-3.0.2.tgz", + "integrity": "sha512-FWtleNAtZ/Ki2qtqej2CXTOayOH9bHDQF+Q48VpWyDXjbYxA4Yz8iDB31zXOBUlOHHKidDbqGVrTUvQMPmBGBw==", + "dev": true, + "requires": { + "define-property": "^2.0.2", + "extend-shallow": "^3.0.2", + "regex-not": "^1.0.2", + "safe-regex": "^1.1.0" + } + }, + "to-regex-range": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-2.1.1.tgz", + "integrity": "sha1-fIDBe53+vlmeJzZ+DU3VWQFB2zg=", + "dev": true, + "requires": { + "is-number": "^3.0.0", + "repeat-string": "^1.6.1" + } + }, + "trim-right": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/trim-right/-/trim-right-1.0.1.tgz", + "integrity": "sha1-yy4SAwZ+DI3h9hQJS5/kVwTqYAM=", + "dev": true + }, + "tslib": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-1.9.3.tgz", + "integrity": "sha512-4krF8scpejhaOgqzBEcGM7yDIEfi0/8+8zDRZhNZZ2kjmHJ4hv3zCbQWxoJGz1iw5U0Jl0nma13xzHXcncMavQ==", + "dev": true + }, + "tty-browserify": { + "version": "0.0.0", + "resolved": "https://registry.npmjs.org/tty-browserify/-/tty-browserify-0.0.0.tgz", + "integrity": "sha1-oVe6QC2iTpv5V/mqadUk7tQpAaY=", + "dev": true + }, + "type-check": { + "version": "0.3.2", + "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.3.2.tgz", + "integrity": "sha1-WITKtRLPHTVeP7eE8wgEsrUg23I=", + "dev": true, + "requires": { + "prelude-ls": "~1.1.2" + } + }, + "typedarray": { + "version": "0.0.6", + "resolved": "https://registry.npmjs.org/typedarray/-/typedarray-0.0.6.tgz", + "integrity": "sha1-hnrHTjhkGHsdPUfZlqeOxciDB3c=", + "dev": true + }, + "uglify-js": { + "version": "2.8.29", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-2.8.29.tgz", + "integrity": "sha1-KcVzMUgFe7Th913zW3qcty5qWd0=", + "dev": true, + "optional": true, + "requires": { + "source-map": "~0.5.1", + "uglify-to-browserify": "~1.0.0", + "yargs": "~3.10.0" + } + }, + "uglify-to-browserify": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/uglify-to-browserify/-/uglify-to-browserify-1.0.2.tgz", + "integrity": "sha1-bgkk1r2mta/jSeOabWMoUKD4grc=", + "dev": true, + "optional": true + }, + "uglifyjs-webpack-plugin": { + "version": "1.2.7", + "resolved": "https://registry.npmjs.org/uglifyjs-webpack-plugin/-/uglifyjs-webpack-plugin-1.2.7.tgz", + "integrity": "sha512-1VicfKhCYHLS8m1DCApqBhoulnASsEoJ/BvpUpP4zoNAPpKzdH+ghk0olGJMmwX2/jprK2j3hAHdUbczBSy2FA==", + "dev": true, + "requires": { + "cacache": "^10.0.4", + "find-cache-dir": "^1.0.0", + "schema-utils": "^0.4.5", + "serialize-javascript": "^1.4.0", + "source-map": "^0.6.1", + "uglify-es": "^3.3.4", + "webpack-sources": "^1.1.0", + "worker-farm": "^1.5.2" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true + }, + "uglify-es": { + "version": "3.3.9", + "resolved": "https://registry.npmjs.org/uglify-es/-/uglify-es-3.3.9.tgz", + "integrity": "sha512-r+MU0rfv4L/0eeW3xZrd16t4NZfK8Ld4SWVglYBb7ez5uXFWHuVRs6xCTrf1yirs9a4j4Y27nn7SRfO6v67XsQ==", + "dev": true, + "requires": { + "commander": "~2.13.0", + "source-map": "~0.6.1" + } + } + } + }, + "union-value": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/union-value/-/union-value-1.0.0.tgz", + "integrity": "sha1-XHHDTLW61dzr4+oM0IIHulqhrqQ=", + "dev": true, + "requires": { + "arr-union": "^3.1.0", + "get-value": "^2.0.6", + "is-extendable": "^0.1.1", + "set-value": "^0.4.3" + }, + "dependencies": { + "extend-shallow": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", + "integrity": "sha1-Ua99YUrZqfYQ6huvu5idaxxWiQ8=", + "dev": true, + "requires": { + "is-extendable": "^0.1.0" + } + }, + "set-value": { + "version": "0.4.3", + "resolved": "https://registry.npmjs.org/set-value/-/set-value-0.4.3.tgz", + "integrity": "sha1-fbCPnT0i3H945Trzw79GZuzfzPE=", + "dev": true, + "requires": { + "extend-shallow": "^2.0.1", + "is-extendable": "^0.1.1", + "is-plain-object": "^2.0.1", + "to-object-path": "^0.3.0" + } + } + } + }, + "uniq": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/uniq/-/uniq-1.0.1.tgz", + "integrity": "sha1-sxxa6CVIRKOoKBVBzisEuGWnNP8=", + "dev": true + }, + "uniqs": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/uniqs/-/uniqs-2.0.0.tgz", + "integrity": "sha1-/+3ks2slKQaW5uFl1KWe25mOawI=", + "dev": true + }, + "unique-filename": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/unique-filename/-/unique-filename-1.1.0.tgz", + "integrity": "sha1-0F8v5AMlYIcfMOk8vnNe6iAVFPM=", + "dev": true, + "requires": { + "unique-slug": "^2.0.0" + } + }, + "unique-slug": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/unique-slug/-/unique-slug-2.0.0.tgz", + "integrity": "sha1-22Z258fMBimHj/GWCXx4hVrp9Ks=", + "dev": true, + "requires": { + "imurmurhash": "^0.1.4" + } + }, + "unset-value": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/unset-value/-/unset-value-1.0.0.tgz", + "integrity": "sha1-g3aHP30jNRef+x5vw6jtDfyKtVk=", + "dev": true, + "requires": { + "has-value": "^0.3.1", + "isobject": "^3.0.0" + }, + "dependencies": { + "has-value": { + "version": "0.3.1", + "resolved": "https://registry.npmjs.org/has-value/-/has-value-0.3.1.tgz", + "integrity": "sha1-ex9YutpiyoJ+wKIHgCVlSEWZXh8=", + "dev": true, + "requires": { + "get-value": "^2.0.3", + "has-values": "^0.1.4", + "isobject": "^2.0.0" + }, + "dependencies": { + "isobject": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/isobject/-/isobject-2.1.0.tgz", + "integrity": "sha1-8GVWEJaj8dou9GJy+BXIQNh+DIk=", + "dev": true, + "requires": { + "isarray": "1.0.0" + } + } + } + }, + "has-values": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/has-values/-/has-values-0.1.4.tgz", + "integrity": "sha1-bWHeldkd/Km5oCCJrThL/49it3E=", + "dev": true + } + } + }, + "upath": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/upath/-/upath-1.1.0.tgz", + "integrity": "sha512-bzpH/oBhoS/QI/YtbkqCg6VEiPYjSZtrHQM6/QnJS6OL9pKUFLqb3aFh4Scvwm45+7iAgiMkLhSbaZxUqmrprw==", + "dev": true + }, + "uri-js": { + "version": "4.2.2", + "resolved": "https://registry.npmjs.org/uri-js/-/uri-js-4.2.2.tgz", + "integrity": "sha512-KY9Frmirql91X2Qgjry0Wd4Y+YTdrdZheS8TFwvkbLWf/G5KNJDCh6pKL5OZctEW4+0Baa5idK2ZQuELRwPznQ==", + "dev": true, + "requires": { + "punycode": "^2.1.0" + } + }, + "urix": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/urix/-/urix-0.1.0.tgz", + "integrity": "sha1-2pN/emLiH+wf0Y1Js1wpNQZ6bHI=", + "dev": true + }, + "url": { + "version": "0.11.0", + "resolved": "https://registry.npmjs.org/url/-/url-0.11.0.tgz", + "integrity": "sha1-ODjpfPxgUh63PFJajlW/3Z4uKPE=", + "dev": true, + "requires": { + "punycode": "1.3.2", + "querystring": "0.2.0" + }, + "dependencies": { + "punycode": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-1.3.2.tgz", + "integrity": "sha1-llOgNvt8HuQjQvIyXM7v6jkmxI0=", + "dev": true + } + } + }, + "url-loader": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/url-loader/-/url-loader-1.0.1.tgz", + "integrity": "sha512-rAonpHy7231fmweBKUFe0bYnlGDty77E+fm53NZdij7j/YOpyGzc7ttqG1nAXl3aRs0k41o0PC3TvGXQiw2Zvw==", + "dev": true, + "requires": { + "loader-utils": "^1.1.0", + "mime": "^2.0.3", + "schema-utils": "^0.4.3" + } + }, + "use": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/use/-/use-3.1.0.tgz", + "integrity": "sha512-6UJEQM/L+mzC3ZJNM56Q4DFGLX/evKGRg15UJHGB9X5j5Z3AFbgZvjUh2yq/UJUY4U5dh7Fal++XbNg1uzpRAw==", + "dev": true, + "requires": { + "kind-of": "^6.0.2" + }, + "dependencies": { + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==", + "dev": true + } + } + }, + "util": { + "version": "0.10.4", + "resolved": "https://registry.npmjs.org/util/-/util-0.10.4.tgz", + "integrity": "sha512-0Pm9hTQ3se5ll1XihRic3FDIku70C+iHUdT/W926rSgHV5QgXsYbKZN8MSC3tJtSkhuROzvsQjAaFENRXr+19A==", + "dev": true, + "requires": { + "inherits": "2.0.3" + } + }, + "util-deprecate": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", + "integrity": "sha1-RQ1Nyfpw3nMnYvvS1KKJgUGaDM8=", + "dev": true + }, + "uuid": { + "version": "3.3.2", + "resolved": "https://registry.npmjs.org/uuid/-/uuid-3.3.2.tgz", + "integrity": "sha512-yXJmeNaw3DnnKAOKJE51sL/ZaYfWJRl1pK9dr19YFCu0ObS231AB1/LbqTKRAQ5kw8A90rA6fr4riOUpTZvQZA==", + "dev": true + }, + "v8-compile-cache": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/v8-compile-cache/-/v8-compile-cache-2.0.0.tgz", + "integrity": "sha512-qNdTUMaCjPs4eEnM3W9H94R3sU70YCuT+/ST7nUf+id1bVOrdjrpUaeZLqPBPRph3hsgn4a4BvwpxhHZx+oSDg==", + "dev": true + }, + "vendors": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/vendors/-/vendors-1.0.2.tgz", + "integrity": "sha512-w/hry/368nO21AN9QljsaIhb9ZiZtZARoVH5f3CsFbawdLdayCgKRPup7CggujvySMxx0I91NOyxdVENohprLQ==", + "dev": true + }, + "vm-browserify": { + "version": "0.0.4", + "resolved": "https://registry.npmjs.org/vm-browserify/-/vm-browserify-0.0.4.tgz", + "integrity": "sha1-XX6kW7755Kb/ZflUOOCofDV9WnM=", + "dev": true, + "requires": { + "indexof": "0.0.1" + } + }, + "watchpack": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-1.6.0.tgz", + "integrity": "sha512-i6dHe3EyLjMmDlU1/bGQpEw25XSjkJULPuAVKCbNRefQVq48yXKUpwg538F7AZTf9kyr57zj++pQFltUa5H7yA==", + "dev": true, + "requires": { + "chokidar": "^2.0.2", + "graceful-fs": "^4.1.2", + "neo-async": "^2.5.0" + } + }, + "webpack": { + "version": "4.15.1", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-4.15.1.tgz", + "integrity": "sha512-UwfFQ2plA5EMhhzwi/hl5xpLk7mNK7p0853Ml04z1Bqw553pY+oS8Xke3funcVy7eG/yMpZPvnlFTUyGKyKoyw==", + "dev": true, + "requires": { + "@webassemblyjs/ast": "1.5.13", + "@webassemblyjs/helper-module-context": "1.5.13", + "@webassemblyjs/wasm-edit": "1.5.13", + "@webassemblyjs/wasm-opt": "1.5.13", + "@webassemblyjs/wasm-parser": "1.5.13", + "acorn": "^5.6.2", + "acorn-dynamic-import": "^3.0.0", + "ajv": "^6.1.0", + "ajv-keywords": "^3.1.0", + "chrome-trace-event": "^1.0.0", + "enhanced-resolve": "^4.1.0", + "eslint-scope": "^3.7.1", + "json-parse-better-errors": "^1.0.2", + "loader-runner": "^2.3.0", + "loader-utils": "^1.1.0", + "memory-fs": "~0.4.1", + "micromatch": "^3.1.8", + "mkdirp": "~0.5.0", + "neo-async": "^2.5.0", + "node-libs-browser": "^2.0.0", + "schema-utils": "^0.4.4", + "tapable": "^1.0.0", + "uglifyjs-webpack-plugin": "^1.2.4", + "watchpack": "^1.5.0", + "webpack-sources": "^1.0.1" + } + }, + "webpack-cli": { + "version": "3.0.8", + "resolved": "https://registry.npmjs.org/webpack-cli/-/webpack-cli-3.0.8.tgz", + "integrity": "sha512-KnRLJ0BUaYRqrhAMb9dv3gzdmhmgIMKo0FmdsnmfqbPGtLnnZ6tORZAvmmKfr+A0VgiVpqC60Gv7Ofg0R2CHtQ==", + "dev": true, + "requires": { + "chalk": "^2.4.1", + "cross-spawn": "^6.0.5", + "enhanced-resolve": "^4.0.0", + "global-modules-path": "^2.1.0", + "import-local": "^1.0.0", + "inquirer": "^6.0.0", + "interpret": "^1.1.0", + "loader-utils": "^1.1.0", + "supports-color": "^5.4.0", + "v8-compile-cache": "^2.0.0", + "yargs": "^11.1.0" + }, + "dependencies": { + "ansi-regex": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-3.0.0.tgz", + "integrity": "sha1-7QMXwyIGT3lGbAKWa922Bas32Zg=", + "dev": true + }, + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "cliui": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-4.1.0.tgz", + "integrity": "sha512-4FG+RSG9DL7uEwRUZXZn3SS34DiDPfzP0VOiEwtUWlE+AR2EIg+hSyvrIgUUfhdgR/UkAeW2QHgeP+hWrXs7jQ==", + "dev": true, + "requires": { + "string-width": "^2.1.1", + "strip-ansi": "^4.0.0", + "wrap-ansi": "^2.0.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "strip-ansi": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-4.0.0.tgz", + "integrity": "sha1-qEeQIusaw2iocTibY1JixQXuNo8=", + "dev": true, + "requires": { + "ansi-regex": "^3.0.0" + } + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + }, + "y18n": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-3.2.1.tgz", + "integrity": "sha1-bRX7qITAhnnA136I53WegR4H+kE=", + "dev": true + }, + "yargs": { + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-11.1.0.tgz", + "integrity": "sha512-NwW69J42EsCSanF8kyn5upxvjp5ds+t3+udGBeTbFnERA+lF541DDpMawzo4z6W/QrzNM18D+BPMiOBibnFV5A==", + "dev": true, + "requires": { + "cliui": "^4.0.0", + "decamelize": "^1.1.1", + "find-up": "^2.1.0", + "get-caller-file": "^1.0.1", + "os-locale": "^2.0.0", + "require-directory": "^2.1.1", + "require-main-filename": "^1.0.1", + "set-blocking": "^2.0.0", + "string-width": "^2.0.0", + "which-module": "^2.0.0", + "y18n": "^3.2.1", + "yargs-parser": "^9.0.2" + } + } + } + }, + "webpack-log": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/webpack-log/-/webpack-log-1.2.0.tgz", + "integrity": "sha512-U9AnICnu50HXtiqiDxuli5gLB5PGBo7VvcHx36jRZHwK4vzOYLbImqT4lwWwoMHdQWwEKw736fCHEekokTEKHA==", + "dev": true, + "requires": { + "chalk": "^2.1.0", + "log-symbols": "^2.1.0", + "loglevelnext": "^1.0.1", + "uuid": "^3.1.0" + }, + "dependencies": { + "ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, + "requires": { + "color-convert": "^1.9.0" + } + }, + "chalk": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.1.tgz", + "integrity": "sha512-ObN6h1v2fTJSmUXoS3nMQ92LbDK9be4TV+6G+omQlGJFdcUX5heKi1LZ1YnRMIgwTLEj3E24bT6tYni50rlCfQ==", + "dev": true, + "requires": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" + } + }, + "has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=", + "dev": true + }, + "supports-color": { + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", + "dev": true, + "requires": { + "has-flag": "^3.0.0" + } + } + } + }, + "webpack-manifest-plugin": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/webpack-manifest-plugin/-/webpack-manifest-plugin-2.0.3.tgz", + "integrity": "sha512-FZcnB3MMQ0CT0aU1+LItwywXWAixLTGUEAtN0fw15dScf2LudQwheLPUCj+QMhDlwZT+9ysfKqUFTcfUGc8bXg==", + "dev": true, + "requires": { + "fs-extra": "^0.30.0", + "lodash": ">=3.5 <5", + "tapable": "^1.0.0" + } + }, + "webpack-sources": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/webpack-sources/-/webpack-sources-1.1.0.tgz", + "integrity": "sha512-aqYp18kPphgoO5c/+NaUvEeACtZjMESmDChuD3NBciVpah3XpMEU9VAAtIaB1BsfJWWTSdv8Vv1m3T0aRk2dUw==", + "dev": true, + "requires": { + "source-list-map": "^2.0.0", + "source-map": "~0.6.1" + }, + "dependencies": { + "source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true + } + } + }, + "whet.extend": { + "version": "0.9.9", + "resolved": "https://registry.npmjs.org/whet.extend/-/whet.extend-0.9.9.tgz", + "integrity": "sha1-+HfVv2SMl+WqVC+twW1qJZucEaE=", + "dev": true + }, + "which": { + "version": "1.2.14", + "resolved": "https://registry.npmjs.org/which/-/which-1.2.14.tgz", + "integrity": "sha1-mofEN48D6CfOyvGs31bHNsAcFOU=", + "dev": true, + "requires": { + "isexe": "^2.0.0" + } + }, + "which-module": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/which-module/-/which-module-2.0.0.tgz", + "integrity": "sha1-2e8H3Od7mQK4o6j6SzHD4/fm6Ho=", + "dev": true + }, + "window-size": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/window-size/-/window-size-0.1.0.tgz", + "integrity": "sha1-VDjNLqk7IC76Ohn+iIeu58lPnJ0=", + "dev": true, + "optional": true + }, + "wordwrap": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/wordwrap/-/wordwrap-1.0.0.tgz", + "integrity": "sha1-J1hIEIkUVqQXHI0CJkQa3pDLyus=", + "dev": true + }, + "worker-farm": { + "version": "1.6.0", + "resolved": "https://registry.npmjs.org/worker-farm/-/worker-farm-1.6.0.tgz", + "integrity": "sha512-6w+3tHbM87WnSWnENBUvA2pxJPLhQUg5LKwUQHq3r+XPhIM+Gh2R5ycbwPCyuGbNg+lPgdcnQUhuC02kJCvffQ==", + "dev": true, + "requires": { + "errno": "~0.1.7" + } + }, + "wrap-ansi": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-2.1.0.tgz", + "integrity": "sha1-2Pw9KE3QV5T+hJc8rs3Rz4JP3YU=", + "dev": true, + "requires": { + "string-width": "^1.0.1", + "strip-ansi": "^3.0.1" + }, + "dependencies": { + "is-fullwidth-code-point": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-1.0.0.tgz", + "integrity": "sha1-754xOG8DGn8NZDr4L95QxFfvAMs=", + "dev": true, + "requires": { + "number-is-nan": "^1.0.0" + } + }, + "string-width": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-1.0.2.tgz", + "integrity": "sha1-EYvfW4zcUaKn5w0hHgfisLmxB9M=", + "dev": true, + "requires": { + "code-point-at": "^1.0.0", + "is-fullwidth-code-point": "^1.0.0", + "strip-ansi": "^3.0.0" + } + } + } + }, + "wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=", + "dev": true + }, + "xtend": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/xtend/-/xtend-4.0.1.tgz", + "integrity": "sha1-pcbVMr5lbiPbgg77lDofBJmNY68=", + "dev": true + }, + "y18n": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/y18n/-/y18n-4.0.0.tgz", + "integrity": "sha512-r9S/ZyXu/Xu9q1tYlpsLIsa3EeLXXk0VwlxqTcFRfg9EhMW+17kbt9G0NrgCmhGb5vT2hyhJZLfDGx+7+5Uj/w==", + "dev": true + }, + "yallist": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-2.1.2.tgz", + "integrity": "sha1-HBH5IY8HYImkfdUS+TxmmaaoHVI=", + "dev": true + }, + "yargs": { + "version": "3.10.0", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-3.10.0.tgz", + "integrity": "sha1-9+572FfdfB0tOMDnTvvWgdFDH9E=", + "dev": true, + "optional": true, + "requires": { + "camelcase": "^1.0.2", + "cliui": "^2.1.0", + "decamelize": "^1.0.0", + "window-size": "0.1.0" + } + }, + "yargs-parser": { + "version": "9.0.2", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-9.0.2.tgz", + "integrity": "sha1-nM9qQ0YP5O1Aqbto9I1DuKaMwHc=", + "dev": true, + "requires": { + "camelcase": "^4.1.0" + }, + "dependencies": { + "camelcase": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-4.1.0.tgz", + "integrity": "sha1-1UVjW+HjPFQmScaRc+Xeas+uNN0=", + "dev": true + } + } + } + } +} diff --git a/airflow/www_rbac/package.json b/airflow/www_rbac/package.json new file mode 100644 index 0000000000000..5f0c4bc592d49 --- /dev/null +++ b/airflow/www_rbac/package.json @@ -0,0 +1,55 @@ +{ + "description": "Apache Airflow is a platform to programmatically author, schedule and monitor workflows.", + "scripts": { + "test": "echo \"Error: no test specified\" && exit 1", + "dev": "NODE_ENV=dev webpack --watch --colors --progress --debug --output-pathinfo --devtool eval-cheap-source-map --mode development", + "prod": "NODE_ENV=production node --max_old_space_size=4096 ./node_modules/webpack/bin/webpack.js -p --colors --progress", + "build": "NODE_ENV=production webpack --colors --progress" + }, + "author": "Apache", + "license": "Apache-2.0", + "repository": { + "type": "git", + "url": "git+https://github.com/apache/incubator-airflow.git" + }, + "homepage": "http://airflow.apache.org/", + "keywords": [ + "big", + "data", + "workflow", + "airflow", + "d3", + "nerds", + "database", + "flask" + ], + "devDependencies": { + "babel": "^6.23.0", + "babel-core": "^6.26.3", + "babel-istanbul": "^0.12.2", + "babel-loader": "^7.1.4", + "babel-plugin-css-modules-transform": "^1.6.1", + "babel-polyfill": "^6.26.0", + "clean-webpack-plugin": "^0.1.19", + "copy-webpack-plugin": "^4.5.2", + "css-loader": "^0.28.11", + "file-loader": "^1.1.11", + "imports-loader": "^0.8.0", + "mini-css-extract-plugin": "^0.4.1", + "style-loader": "^0.21.0", + "url-loader": "^1.0.1", + "webpack": "^4.12.2", + "webpack-cli": "^3.0.8", + "webpack-manifest-plugin": "^2.0.3" + }, + "dependencies": { + "bootstrap-3-typeahead": "^4.0.2", + "bootstrap-toggle": "^2.2.2", + "d3": "^3.4.4", + "d3-tip": "^0.9.1", + "datatables.net": "^1.10.19", + "datatables.net-bs": "^1.10.19", + "npm": "^6.1.0", + "nvd3": "^1.8.6" + } +} diff --git a/airflow/www_rbac/static/bootstrap-toggle.min.css b/airflow/www_rbac/static/bootstrap-toggle.min.css deleted file mode 100644 index 0d42ed09cdaa3..0000000000000 --- a/airflow/www_rbac/static/bootstrap-toggle.min.css +++ /dev/null @@ -1,28 +0,0 @@ -/*! ======================================================================== - * Bootstrap Toggle: bootstrap-toggle.css v2.2.0 - * http://www.bootstraptoggle.com - * ======================================================================== - * Copyright 2014 Min Hur, The New York Times Company - * Licensed under MIT - * ======================================================================== */ -.checkbox label .toggle,.checkbox-inline .toggle{margin-left:-20px;margin-right:5px} -.toggle{position:relative;overflow:hidden} -.toggle input[type=checkbox]{display:none} -.toggle-group{position:absolute;width:200%;top:0;bottom:0;left:0;transition:left .35s;-webkit-transition:left .35s;-moz-user-select:none;-webkit-user-select:none} -.toggle.off .toggle-group{left:-100%} -.toggle-on{position:absolute;top:0;bottom:0;left:0;right:50%;margin:0;border:0;border-radius:0} -.toggle-off{position:absolute;top:0;bottom:0;left:50%;right:0;margin:0;border:0;border-radius:0} -.toggle-handle{position:relative;margin:0 auto;padding-top:0;padding-bottom:0;height:100%;width:0;border-width:0 1px} -.toggle.btn{min-width:59px;min-height:34px} -.toggle-on.btn{padding-right:24px} -.toggle-off.btn{padding-left:24px} -.toggle.btn-lg{min-width:79px;min-height:45px} -.toggle-on.btn-lg{padding-right:31px} -.toggle-off.btn-lg{padding-left:31px} -.toggle-handle.btn-lg{width:40px} -.toggle.btn-sm{min-width:50px;min-height:30px} -.toggle-on.btn-sm{padding-right:20px} -.toggle-off.btn-sm{padding-left:20px} -.toggle.btn-xs{min-width:35px;min-height:22px} -.toggle-on.btn-xs{padding-right:12px} -.toggle-off.btn-xs{padding-left:12px} \ No newline at end of file diff --git a/airflow/www_rbac/static/bootstrap-toggle.min.js b/airflow/www_rbac/static/bootstrap-toggle.min.js deleted file mode 100644 index 37113200bff5c..0000000000000 --- a/airflow/www_rbac/static/bootstrap-toggle.min.js +++ /dev/null @@ -1,9 +0,0 @@ -/*! ======================================================================== - * Bootstrap Toggle: bootstrap-toggle.js v2.2.0 - * http://www.bootstraptoggle.com - * ======================================================================== - * Copyright 2014 Min Hur, The New York Times Company - * Licensed under MIT - * ======================================================================== */ -+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.toggle"),f="object"==typeof b&&b;e||d.data("bs.toggle",e=new c(this,f)),"string"==typeof b&&e[b]&&e[b]()})}var c=function(b,c){this.$element=a(b),this.options=a.extend({},this.defaults(),c),this.render()};c.VERSION="2.2.0",c.DEFAULTS={on:"On",off:"Off",onstyle:"primary",offstyle:"default",size:"normal",style:"",width:null,height:null},c.prototype.defaults=function(){return{on:this.$element.attr("data-on")||c.DEFAULTS.on,off:this.$element.attr("data-off")||c.DEFAULTS.off,onstyle:this.$element.attr("data-onstyle")||c.DEFAULTS.onstyle,offstyle:this.$element.attr("data-offstyle")||c.DEFAULTS.offstyle,size:this.$element.attr("data-size")||c.DEFAULTS.size,style:this.$element.attr("data-style")||c.DEFAULTS.style,width:this.$element.attr("data-width")||c.DEFAULTS.width,height:this.$element.attr("data-height")||c.DEFAULTS.height}},c.prototype.render=function(){this._onstyle="btn-"+this.options.onstyle,this._offstyle="btn-"+this.options.offstyle;var b="large"===this.options.size?"btn-lg":"small"===this.options.size?"btn-sm":"mini"===this.options.size?"btn-xs":"",c=a('

    NAzq6e_ z$7Q|4AZmmGf8*3Sq*f0rBnC>=*eTa=Z*Nchwq^^r+`g>#NK-;6_i#1$I4f;G&yEC4 z(>Iig%v2U{Nw?c{T&d*1Y}d0vqsB}{*S#GQ{{oB-2P1(Th7lM;w4$!Py=jwR4{lhzht^iFOhY1-^VOT}GH- z5!QXD28bg}lW@3)pAl_6gI6Y^ir1rwE7glXGtb(1k>@zC?D&Dtv_Vs?{$nC4Sm*_j zNBEH^e>rxn`9A&Hlk-ZH3p6mFQHy^$1Elz~bop#3&Yw^0YG&$~p_CF?yWEI(+E%R9b`Ym%J7C1%*?fDPWHlp}P-~pb zj~&vITPT6Ft^djuV*FBqiyRr_w2+%t37Of%1UrNz&`VoPb@U4|+5WS{>j!`0OqgB@ zf5uxzz<*1o{<~6mJQk(-O|N-Dx4FEUP-^s-fMWZV?Mrri1;t;T#j`tN0JO#vOah6p zVe)Lq<8uNHypI?BS;TFFbJ)sbbX?A78=Gx)>@&^xXhQ9Lb!7ZOtwhx$M$aSq;sQMj z!FyenP_tUPBBRN>lL%-PP6=&K%>RXof4IGLo%=zfJp(u=Gyr4>+(M(=Mkcrq5qvrP z$iusHfl>3>6pFAY9o6g~o_oSqC4Bx{OK=}G0I#Nh+SAYseH<^A5b z`WKC0V*<#o>wV@JOAfHBHS3D%m!Vq>w!Cr+-ftD@+lZZ9Bsl)R1ERQ4>+i^9e=KE8 zoc>l?*hJ)krVxPxWrFN;$Y4i4U`+m*P|&LEEd%^Cj)-+`5{Q%nhIy5apUmYqCuGr| z#j*?@op61WJ+KuszE1QQU<82!LBsW5+w3U`{`&;3mjQYJL<$BH&iOMaj|&5Z3d`Ix z`0$m6g$zR6dU)}nF=G!KP4vhBe}+SY5hqr}cu}C6lNjG!VDM^X%!esu`f~$PWKDlb zK2B6=@La-$5d`VPcr)b5R|1FT$tg-;$)vn|?(L9G_eYp}2WQl<(cIDH6}e}UAK&?Ml+ zICf&$uLK`rUZ^?d*|Zqf8tfSO?K`1vEnYNMb>Rh5hiTWgeH*v!(i|gLEh8^uh>2Wo z4gB^vuEM#n zRO>`Fc?6Qd`_h#288-3M^Pvt`A`lunjVe>7vosjfA0reRCd)i>cofn|b0`D8H!Z#N zFsdY#6oxqa69tA%Bkkc%Oj#rpQ%6OGv@{T0}l zdJUG?Q#~!V#bF;Re?vJ@pc&>Y26E7HwYFfW6xnQbdZxF)EPId=ZyAVyPi<>bmRfP& zGWRfa*?kw@65WlL-g;}B*WP^fg>&D2b;8$QTaVT0oy8`SAjw(ZsSLe<9ey}VfFX`m z;)yK|w&II5#x>)OJ*IfzkKgS$UwkR)A+8Gq@&0SpJd6lgf8>}qCRyecX|5UPWMR&k zXH#$P8R*=84%*0};We7*q?PUu>7@tV+3Bcxg<9&WO|%;8sk6?tX>+^Y8f>D!27BzV zc`n;*nbA&Lckn3 z*YU?kUR-kAeserXY6slUi%BVKf8E+ zzwdtb@%s*+{G-uNUt9CpzxsXB)~{Ei`tAO|AOHOImvGhrJ_COyLYFaG0~CKb)Zq?! z*h3%wFn}hK#}J8FL?a#%iAhxA5}DXUCq5C1QIz5osaQoTUJ;8~)FKr%vPCa`5sYCR zVi(0&Ml+rfjcGh18P(WEH@*>$YGh*^=~zcQ-jRuO#N!_Mm_$AH(TYL@IU2)~nbc$&C%H*aF0qrD6yzsKSxQqLGL)x8WGYqJ zN>j4Z{*|%JBrIiFOGwhvmXE~cBS)D_U;Yx9NZe&G-8f8Q9+Qs7MCLL%iAiQYvyaMz zW-^_5%3e+ro7ps^G_{#UZgvx#Qv7B($qCA8mJ^*@9A`SexK2P`lbwI@lxHK)InOY< z)1KtSXFj#LPkv(4pZ|nnK<@}ofgY5f1Wjl?5!z6NHZ+(Ih3GFOI?*Cl6pjY9=tjGV z(Rg~4qp<{ONKZLZl9tk>4rS;kHF{E)5;LXxT~C(xx}1WKRFdQ!%zwr$KF{ zPqRr>p)S&>M=fMhl{$Y&rhas$n*=I4zflGaXaEZDn5QUmZ~+b6;*5<*LL*f9RAwsj z7XE_BgCInq)gv0C5qZc1C7!XwSGb|BOMHh~&oRZj<~51WC<9odcn(phRS<+dVp`=Y zif=rTv3U?hUyt~WOvtqn!+4@)F{{}lzQYuVeT7?@2wKsS_M(4rohxAdxKhTl6}3c! zt6b@7SG<;%uYSF)5(TT+CmI&9z9r&2UUAwe_BM%}%|m9N2n|YLcBQ6W>u96+S3jyY z6VOm1S`ph;VXC&aO0=$Cb1TL126nJg1nydeJ4E6d7mCP5qH>w*MCU?xzRz8*XQTKH zH;~nc!w~R*1^j>5x4O2wu`S{|mYc-&CKjS*&Eiz?n1`_f@*T<1fP%>xu7!x=j8*>m z%@;F~1WLtG!pPL4F=yF%usN!8xBH9FG84 ziQj;MN3LST(@Z1>lvU!Bqa$T`31>LH5Xyi};ExD?|ePZQ`939OsF^K)Xxa%0!G{WiKC$%SNp8 z5YG{80_H_aGQ53x8A0QL~|T#`tqGsRb=@EuKE$Kpt<5rUZhF*vKz z7(uuijAt3K7jK+`ZJROLFeP%gB9Mb_y@ZI&+Vx^EeZ(G*d5OdLuoUk-?jxFF*XJ%G zWGDg~<4}6jFi6g^-D_lNCFd1HBzOo6ek~>fG7oOtsEKRr41HnYk_&hmJ>CFVx7dJ^@GV}8#(i8;5p6EUu3peubSMo&A^ za}9rYg)lvdPLGaT+pZ8iH!;r29(Y^ueo?Ns+v`SrhD!uLc;)!qz&(9-{w1evM{xsE z-w^6j?ek!J z+-E`%OCNpi835pKz{U+jjB^eQ?BTT^5i)-vNc6oePJ93_{ya&v{t&SbZ+XZ&_3}6e z5u(y`g@k|n<(G%+VGroYZ@q}9S3T>s%Kjn7-oU)~z4jOGME!r}_kwRB3Qw=%!uXI+ z&6q+EC=K2XZ{ZX3q~OpQ2ZKD z`-JfBP-_Cmr2=mR13~buIIzn;P|YMF0$%I_&8~@zVJLvD$5P-N&JYc)Ot#KR{$h~s zUZD}7Ul$PWS0!9-Dspg|E+Q6to_)X)NnRM8kd zPVo-0i9nBtN|6?cND^a>0SJ(YWYHBt&K7eK6;m#-G(r=LNEUaI7mHENdXayM9&r>s ziH)L=0DVpHJd6^GXw;C6&CE~tmMG{@kchtF&wg?A%8m<)=o|iA!s6oWoT}pQj7Sb6 z;GE8)=;T2R8K51BsO%bXjLxr!0&)v|jI^+k!*cKM6jC0z(Hp_=z;e+gvhmj*QX#sL zh`v!C%25Z8$mDvki98Yr&oO@;fsyP8@*Lsu9m>x0hD#!3Z@x|v3SHtKhsY8cKqEy> zA*K)~XO1BgvLa`UByUd}S&JhLk0RBvh_q3=AaWzYuqee*BFQlv-H|4ph#lRLC0&vZ z4{jzsf&ioNCgmX~chVObz$eXe=E_d%f~^EgYcU#ch|Z!c&1oquvhjZ~a1H@dC1;N% z^>C$RaT%ki8Ch`_5px)CL>YlYH#$*KCgK8Gj)?N$-L8WES@KZL?u{|>z^(Lc%j8fl zFewTT#u+Ix0>nx((Q-4b!ZXJx0)8&eF5unp(7oKCHiMv=j3G9^AEbL8cQp6R{;ljZ)8#HQ3aM^Ljfhz;^dfn& zHw2O&tBFG2QXX!MNi+0G$H+us14=_w6h_pEIMhQ+lG0R^DVxYejR;0tvMQ^zO>@yq z$I>N~5FCq7`D}mFO=Z%(T;ey9w1}2eyN)!`#PmeTR33Z{>Ku>MCe$UU)JnArLxD|0 zwRB9obX>kvB=cxXQ8Z51Q6XMb=gMx*c2oj8RRZUmWz5SX=>prP1?g1@x$D&N82xT4DfgCI(?FaDQET|YwW$h9Y+030I#Fb3iy+VLC= zW++-N+3FQI#l<7M@LL%}v)a`o6ie3@LNh~hAv9ny-LW<=l3^D@2`N-N$uo&S&s%#! z^Xf7%7Lb20Yi?qFtgkF~D9)@c+AB)!u`P0JWEr4hUE*TVq+Az5UZ1G!KEmQoLjv3? z&PZjcs_7iSw1{p(@Xmq&bMGY=3|1@4B^cBtNOT9AHe=6}i9i$BkdWy-j82WAGH!G! zZ?7N8>QPyiAlwuP%eIK*kRU|WKgUJ0C{*y=HU)p-RvwTRwUkyrhc;7($Y`|-Z@+SB zhl^^t5-sjRYCo%Rk0@}_^b&_lAcQm{#qR!1k*IO^^l?Lw0SGJ3Myw-llrr#*Ziy&d zI#zB!?k;Xm4vADAa&A$DmS~CSZ|~M>4{~rZS0E`AYWvoSQg?|im*%9^y&{)Qb&_&< zw`zZr)+K~j-Z&TSJ{Pq@_cx%S5h%gRw&KN3H&xF=4Cyj-<%$T1fD0n@8e`XQn=~TH z6Kk)ocJ1i}d*a1_x5pYUVewQR>K7J^^dlU0h{TKmdI|iDff4~jBYAeRjG^cz`F@gByW|Rk9x=0~%w-Bft%;_ZETBhhx}yX%`Q-<220e7H)waO~DkjYl$cLH3)xl zi@SJZ$3=*tfeGP{bQyyZuo%#^%8CWSk3UuXHjRs|mxRB;fhV_#hz|+`_!f)|z#_99 z%g$8+NsnS{<2DRl!4`P!aB+u7Q9|GyqA+odh(e*%V_8)Dj0iIRk#89w2;zzcDbt8H zmUW}>H+t-n%Qnvbk(V{`Q6a+$&$WLCqfqdOIlyqNlVj_X$Ecn(xi>huDm=M;QyGay zxs;<;im1=6((I7=40GcN;R?QsxiRWoTE8_wjD}<2_c+0kk^WcanA&L2M zA$S^z7X?St6ONtO4r$t^d#smel(c*o3Hh0b6m)ufBT;`(T< zp~0$2i$b#+STirt$Xcs0gRFmYR_iQT%7g;Rd{-WvnsD*0G$~VSwMyQQ6EZwgAoTPd zobri8V|ZnCh&;^mO0%8M8U6_I8h1SwSON7;*A58@+bJE}t@avn!I_PsTF`{z7Ld^U z7F!`2yT#g>t*;uC8QP(5mirVUQA?8&m_lVGiOcGktaO*6fl-%_sC0h|mYWMpp;s9} zk0>WQ_FN5_0c3lz+BCQY+7}D7F=A)-Od;74wR_joGoD_P05#lX&~Jz50`b z`)xZGp^e*6)fBdyXt_13AN7KmxFI=?ZzypO2A8Q{OK8;RnqXP0>*&gsC3h;6k> zt)r5*^ML$ld)Rin#d`iVSWlBJ?N(WhnF!VvENGVi%l5Aw57UmE=OQDsLD4D8F34TH zmyCfD1WnI=qaSbY+J3{sgv&~w!7VR*+;FjpqVRSz`y9Hwy1{>(?#{LGAfs19_gc#C z`N|GC!yELd+{$Mz3Ll$xuO%oKEUrYMAL&%Q4;u*k5xUi^DiFq2p(xL*yoj#cW6#z? z6HW965@uC=vX+xjxe)sMM*V zN));!2vkoXo_K$~7kA_8?Y-@gN1y@OO;OB$w{{T(*^`|XIg?Ji?<2gL`zmA3g8M97 zd&ezTT4HOM(f82)p-ES3@X=oP z;&`3I!dvE&5TRyzTu$A+kX@^C`qu(|xCUM7-D}Vbozu~A>7A)m&V`}iiVH&aH?A$) zVb#zs0PTO#{=>?ynCG3$nJ5YkeaFu}?Ke#n%8oT-TpoD&09G^XoR{^;u79GM98$iAfm#U`bQ@EalQGs9zVRBbaIq8mT$KZB%~67fO1nj}Ao zP+#yz|4!4>@h7?|HN5q|eU(<~8*1Di-3p%<8fAbnPi=pdf76hY+CQS0znK7+n-a7NAdJCaz#zywc?c6Kl$R+2!3eh!3Stle z+QNTf1Vt^}DDT__24y%R%tMd?42v9%8RPhkNJ%aS<$e1Ya$yjYF%xNr+LxH=WS_DilS~X+nRMEL{?WDRN>(kylG$j>E1Q}V~Vx9{J*5g!iW)wVp03^Xci zDsN(Oq@s{3XG$~SO~aITf?ddkA~fgEGfKbc9U&!xofY;N`))knj0#aJypGTq@#lXb zna^kcP6JQK;k|SjeaB_B5o$Z_l^KEdg;$el@s-ycdk7&HAQuTPSKWSOEhJol$%Td7 zWD)*Wl%Yfc!bjOnFxbUWY-?>MY>hY-4h6pu6P#(9u@=7L!-C_}V zB~_*2e^2>1p;u1I<3>p^q$kfa4t0MRWMLfXmr!dV$)^xWj~ytXQQoOlmy2 zi4d1Op-G7;AcOo$k7OPC(oCBjEufS|dk?=-Y)2j<1Na(dZQ5^KeWcsWbH!;r*(|lQy>NL0?N#*kWSGE9&tbXS-V3`n5LS!O4%_0;CoNX_zi<<%^a1Oaz{> zke@iIJKoBR>rj8VFB?ry0t{40cUHMJ1-TJLDN>m8K!?A|jLIbFtB*s{t?v z7XwPTzmK$tBRFE_P;g=lhxm>zH%o~%7lKWT%u-9pv84wQ^2%5?PV0Ofw_|zh42+2<^+NEV25$Iyrh0O!fav{GAkSPA>n1_Yny#8EDKkxEsg%to zE^iUZqVbB70sY8J0xeuHv(P%ZnmP7sjg-+I?0r9S0vN)5^}7#PpKGad*=NH zd-s2IsIcI7zIz>>i()lP>!vTD{%LMl3Y-w>BDgII&J2TJ>)?WHG{QNVFdy-ht^|H$ zi!#};GHcqH3nME{_6_JlVk6&D4HhB4Z0K@RJm7hm3X^O}u9mkPmJdyjfzKewUM8a2 zEveEXkW~m7<&kE|GBmY$bIvWkycMrPOo)F;VeK=X#TXn#pb^FpMOSKq${gq!m8*HH zQ`VyaQJ6xzfBUoB48;Jk7W&Zrof%*WD*;oS!x%|`=&I64{;I?lGSBDO^GhHpNeaP; z0fKOwbM%QL3Ct84hC(iAIV@?Hl_Vmc-3S{|t?HF(RBo~Rvnec6>s#lV)yZ;fJZ*n8 zYfnQ1mCy8ZBfjD5|B|@4m=<1TS_6{k37Q~axoMM^r%Nt(IXy5U!Z4inw9tu}+rRYZ z_}bkr6Yp}6@Pss*4s{E9cLp~h@boi&+2oBNhlHQ5;Ba?UkmwnTmPc6}s)Atghjhj{ z8o0$fUV)X#&{2_j9(l>_trvVlbfbTGW`Mlu9T#%D!R01>@xT2ok{Rn-eFQf+LQrW0 zV~iYNJntol^b#0Tpmikul}825MQ@aVT;wFb@X3P-ss7f$(rHa|QOswqLzdqh8u=bE zcKf~HwI91ONG-V2Izx1$zXItyUW$d>(DbLvBjXgz2-#p}5lpuH8|P*KUS)rBW&myL zml{wwD)l`$I>~(AWmnU<;x2S|d!Cga!Xya;GmNQEDF@F$JXRj_>R5kc)_-ZetRecB zW^?=5qNfDub6{uSOw2z4>L4odBQ4;2MZT~*S#CT!J_n7-_@E)Brq^_iEEXpMV$~Sz zOr0O-7bgc@!{4*KQo`YTCFy_rF-%OoLgZkJrTZxg%1;OeHs}mJJR0eGtEa6cpFHQK z>xUHWcQxPOeGgG(D;6HB;eUl?MtJvml<^_*Ln;0oRvgw9emkcb7|0PZq&F$@8st_S z%(M{!2vPY}P0wLV&fyd&SRA%wFvYP{t&t`$=oL$)OcwZVxD*~vf-8UL#}aN=D(JTm z8@Lf2C=$z~fM(PcNZ~te(ScLp6(JB4D_BexgksMjgR&J&8)zZ_7fceEB0Y$IKnQTi zGK6j8c1Os8AGn0i;e;M2C{c(MQy4ou5rWU591}QLGst|K#uNrf5jz!z0dhKjXDUhf zh7h=jfny^jNDwKoDmQ=ig$OZzNKrzTSQ6z&hVt-;y(ckR@(k8T5FzjimeL`VV0=N4 zbtG~Q0mudIB>`sDF1BP0*7s!pMhW9bLVWZ7iYOsQOe7~|wm*5`LTiy8z?eX?$UtP4 zH*vKwqsRcIc#8fPUb}LGnE-vIxDbs1FvS&&e56jPNQcw06Agd0B6(@?@2vlp?k?7ogFVm;sgqHkM_X5NAm-oEU#V&_|X=l|{HHjCpeomPiB$ zIG2+WgJD@&o*|ePIhsP{FEJSbhQk-o#|36lX4!a@3h|nF$&L4z1K`*kr}>G#VJqM8 z490*nzPT!?b`X7&7Oe9QzG)1;#Y#J&5y)u_n;B5axtyBuoXVM8z4@C75uBR=4MN5d z-l?5|6P|w!sV~_HoY%Q8YC`*285L@w zT!x_#s-Xt@q4sH@CiI*Dq=O_HqyDKv-uVp77oLChX-4uHpW+FgJ9-z;X%V^^9O4ry z3~(&C@sj)E2075APWq%dPzEt3qf|O0Gzq0p3In&9FPQ)XSPG>%TBVi|qVC3|U+Sbk zDW;{Vrfk}#Y)YnX8mDrqWpFyDc6z6cSfqHmr8MYn<1{`dah&YtLR#+>Kdod+OF#w zukxy&@H(&fVy^a@ulSm;@w%`4imv|puU1;GtqQ0CE3gO)t_Pd2u*I+r3!@Jkv4=sh z5(}^tTd@Nxs_%-i9Lui@o3J0-vAi0xBCDz-Te7BlvLdUp?zXWg`?3mqu_ep0F#CV0 zG+VQddb2pYr#dUKJsVUmyR$<3t3PY8L#wYcd$hS)v`X8nO-rvDYp_ZCv{W0TOiQj; zTeZPDwOVVlT-&vsnzac2E4C&Sv|xL-oBFjCOSWhmu4>!1cgnVK`+0JUv{5^xbvw6u zTX|@kw|kqcZ~M1x>bHXXQ-s^EflGgyW?Q(7>v@MetdHBcAey+6YqgYHxfU9^=c>7z z@vh19xu6@mqC2{zTe_xux~QAFs=KTfNqMz1W+*+Pl5n+r5{HWC09+ zru(k0n!f70zU z{D~@&!2S%}zz!V13;e(mEWr^x!2n#r7M#HvyulpY!GYSp9vs5zi@_pX!X{k6Bz(du zyu$UH!Yus4FucAl9K$s1dmvoH={v)JIK0C=+{5{+!#ev9LR=z$Ax^zcb3PB+{iY6?8uP3 z$dNqBL`unDEXIj!$(lUHjJ(MT+{vEY$C(_;O+3n^oW!Pl%9R|(sjSMbT*9ra$g%v& zJv{!)v|Pise9MFk%DRlVyo}1f+{?m@#ld{bxjf8KT+GM}#mBtNu&m6^Ou(!R&Cy)V z6Wq*ujLp{E!P>mdPVCLo49=f_jLFn&#;&LV8o&kj5Xh#G0~eqH!;rs?00p~&!`*Dh zE<6m4kOxdt4-f+ll#mB`kO|NLMCek`UL^>$;;IJ?M9|;{@jRsRU7On zEeN-;&b_D4{QPJT!$krOXrD;Z7M;iS@CqA!3!k{sEgdlcEzkt*Rr$Ps&;9(*0X@(N z9Wf1^&!g^c|g?>qtNoe&}|(tMSa#oiqstK&=9SgMGe)yA=Q%O z)@l8@ZjIMi?NwgQ$j=ae(1IWeT+P1_Q`czy8-4xQKXeXxP|uKj&(~be0h|Z){J`~K zd>Y`}Kt$V2m@D7y6}k;E z!7bds4XEi2*5e%kw_VlQ&B((Y+Y#g3#~0nbQ4ixSYM(ew#O)i4+6dLnsJ-#r{^7xU z(%c)h4I1&?8@c^M?fus4J){%P8x77U3OwHw9^gQP-_Y$F{2iB;4ICq099t9|91cX& z4Flic8}FbP0tnfE?rqz?0pdtO;6HQ|5%A)h?b&P$d{? z4;FF(4_OgFbRR%rd{}MVA@Gnez~;Qc=YH5cvyYX0YThUe1yYrQdv$LHgAM*fee-Wyw) z1EYA@zai*Ng4SF493K9`pZ*Y(zFC%T5}5uQXddaDKBURs>Y&c*UN!2B4(q*f>VWPW zs;=j)zT6LgLG4k^?G)kT(vIj6qw7gA*`v7X1d-%q%^Txh>AaEcC86KIadF;$$W{Kp z`TmJe?!bQJI){7{M&Q5w9=}qq#!kM;7>sD^To0Cl>UtFc5v}HNu2p$pd}IC_qrL2i z9ncuB@uCo|C6NNxF5y~iM-()vwj~sen!L|F*fh+SfSLv zq|W{z54Zv55cBivEc8R-?hYaGyg~6LzMD_X@c!WNm$J?gKk@$U4;KFa@e#A}1YzcO zw)G{E@lsFqYwlG>AM``d8}$(K4@vS~74wfx^BxUY_wWvQe-NLDD{#*n5(xv3Jv~A2 zL&O1p-@Ne+&kgqX;5Ll^Haw2&_uCs~|L|77^jP0pydn0Uj?u;k_8oumWB(hIU-f9e z7)QVMQ}6a8U-JGS_cKrWa_${>pZ9V7Lw29{de0kakMxH^_~h*mi2wD04vB&P5KkZZ zyfOM`FZi{8TvHxq-5$yN-oP%LSHF4uym1eI-YM83zWtL89PjX+z)|pk`keQ^-`@X2 zd4=4uQR=>-{R|33>;D@X7CpTI|J)z++mHRcVg30J`~YF>**Aaw1q~YfCh*-sh7Aqo zVBp4K!e{3k9)xD>-o=AKA~KYh5t>Dg8K2c-D2PFdjSVR-EUEEeNMq1A`ZH$iqDhi} z69x9%DYIZbktlC+eD|!T(W6bBK80#9r@?m@RxaiOsFH!Hg2TL?{dlnq z+E-4^qD@Ox0$i3(jd?9|uqp!>_wF)C!Y~8_2|y1*VGzNlL9I5QdF68OZ(fIw8AO(f z*Izvi3^Xv^Tel%+&!DZ=?fNyWSF#R&rxpy2RV!DoU=50G{T8i6wQSwOjq5LV$HW9V zlKxEf^u~8{(Te7XSTXVG{u-5zY7kn4v(Oy+Out&>OM<9;ozgrxQmc8Z6)TM>#n8f5?idG#tOPFIjAZe&maVUl~d3) zugoJay{sTyjRF58b8o<_3iKxl!eA=MMerC(;jFE)!!bu4chvDUhR%8B85nb<$Uv6_ zGLoT}E&@`>9DOY48yC^K;FgrQ^D2T&+z2iYv?2fzC&HdV?5q%;Q4uW(l60u52q-Dj zs|)Hg$V(f;+^(?e7`On-0Ecpa!%qf+K(nAc&)HK0vm}6q3$&8Brx8EjJV;8os(g=u zTg1w;HG^Im%tlHW%EYR(pcwL|IK5i4piCHKAcPa^ss3@8Tjsj)D3A2Blu9clEbkRq zLA291S55T_)uC#=HL+)`+Vo3L8G@2RK~orpR9I!zkXc-fD>kt(pMu7JBOBe=sGh+h z$d)m$PUJD(hJ4&>TF#@2rOUsxPasb#X0S|g5u~#~Smm_}xhA24UdSIA4=x+L} zU3dHXYg&KAN;uQL4qDH|dqqBIu)4(JZO?>F_Df)f*1B=8R{t6pW`gqFm%D+bO0iys zuDcOujC3w&;eZ|f)L?{v?fUm<0ewCQXg1Gt*ye{L?hq-fJ~k+1{~U$lCX^e32#V`O zJ7+F?C8tzV;@YiC0tyNF(^?3KF3#B%=bx9NuM_NU+o4kr01qX*Y(@1NyU z*yV$)eg|TJC3djQtuyz!Ah5-j+@H_$4!t1Ct!h>7O6vwT2ZlO-r?zLuqm~Z_z8K=N z)q}j$!0@T%1wMG;=S|T*ST7Le;>>R$%s^1T9F^cuvy9%LQqfZOw&QVx@*8BoOmS7X zc&n#jxWX0_I9A`;>&xF8S06nM&Bq?U2;ziQvv+C)E`ndpy5Ct86+Z&{$9w9FfKJ+{ zK8x)sKL+6va5RvAsx}EqKKucWi1revT-{_>QTm>Z++!~FrN?$~NuEC<2$u<}h=b1} zpv6$c5b^cLAQ*&;2Cc%uZJLCDfl5 zlCZ8MP(?S|BSZSAlPG3xih1i<6Co&{LLTIhRlK6+%A-X!I>n1mQ)EG&_`dY#p4ebppld^*@lRJrg#T5;=~*N5b7PTFkGEwOk7>` zuYrMKaCc{LdvTY+r9hFE(o)>rT?S@wcXxL$4#i4wX@OFTyL++1aJk7pxpH!n_2DEt zIXU~o$$r-QJtX>wtt^Amo6s@;5$@XLP}3ep`FX*GoM~5F&CZH4IT*XY6(y&v;1(%l zVwiLGMhm0xhC)|vPe+~}q&q2yI^?4~IxFxFp~X@PW+w7qW}SDJOyy5!pAi|1X3oo$ z(-aa?8sc?N@mtv`Zm6Z0^SH$$JYH|E5@n%TYxbn&f_f?#RxBb zafm?=jNWUP$mnl&`7rM-wQSj|5Pdn{hlPdcTr?stT6AfU7IDiJrMPHDGM}7h zibSj=9;=n-rsn%F5lg%mx)G+2hiee*{w7qi_&T@Gn^mg5Hz|qz=`92KyF5_mW3NI#K}WWxrx; zk1)R8n1}vnR2s;yuCXLIg``O}{FD3NtnUp8^I6ATkyEa1W-?ZZfxS(3QoaY%AMbU_ z1!kBHc9yT*&f0IuT{zn2dZ4a(s36*A1(~mKV_+Vsd)$QG9c(~09Vhj7>&$Z2kJJv5 z&#hY_oj!O|daVX;ppNcTmJ4+}qcu4wpXeU}lt!+G+57`C6ystX3p&YFD{K~0tVx^e zsQjT7QKz2rVZK*SBBpY2%-y}rrV9&~c24gS=#!Xs5X~!?1u3XA&XVSQ9N!zG$m@54 z)^>ui_3wu(j$QJ|P`B{HF^7np#!LFC$2My3WhML85`2X4pfq^|W1PhdHk8_exhNe4 zM?bGZw#c}{%>1$(CzEpascj|C9s+hKE(^XT>hF@%N*4 zPjQ_llbk=Kz^7KL|6m=ow>*E|;2*4I;MhhI`XcZZM-&xk<9tQ9(+|DHgm?soB-2F} z>7?3?ncTKBJD9f;#MHEZrJOiJhot4c(h-D8;zwFAKy&{7k@;zkv;s~lNsO=e4=ZO( zXd#!Ggky#ZfOPu_nP6=b5b>iUv7ou4Vbv;U-T`NFj6#ktuTot?$pK=ANHdDB?47|M zMA5ZP@csExJMTMGu0R3uomCG5OgduSYHU7cAn@l-v+AVYWwFKPMjVGImM2KQJ2}B* zDv_%!lqht7MS!0@6vswXmbewf9NWoM(@j%bdgK>#=^sIK)%zur;NKhYH#kNUgvBYF zSW8C&SlQI%4#JTvX1{@S0R}jC;tA&E_z(dDw?tjosPda83Oro$o$`aMTo|+#;!!s8 z)xkqC2MSB~3IdYiQp82$yd-y&jcn8kE`-F;@r#H}dFhFk{!g)FegkQe`mz4JT%@I6 z�o$iO@_|uuX?$s%5J_uP81Kfz&o&L~dE#+ftgH!FIJHi2Im-(T&#b(Y=-ZQI9~o zRwWDHu&-c7@`IN znAno&jG|aPHc?p7gpJ;% zDlQ7+Z?FdiJjS*SzR_u`j@pd1UV|#3#Qc$osy2rztsAPDl98#ORne=)^gAQUP*sg% zbDNeF{yYA*pxPCfopTIo7bH}L`&6FVMYdgGHc`b&s+afXj!#sMxA?2BNr37cqOwlZ z1O`-DDAdy()N>@?yiuPB-)3oZ;5Sky+3^G)gCTY6zv0yycNs<;)(!^$cWYvnZv2!dkw3b_QNK& z^4j`lg`zhrBP8zziab(N!G$pv2sMwBp&|2uH$B7N0m3%H)3MFUXbd>(1{{C+G_SYg z4qiErhXkG))=ezk8vgX-MD`sjG@k#U=`S(!RJZ!oy?oj+-;n}Vte{Ng!=t4Jqskl&)7jTYi$u z6U2PYh;grFPA`Pv6Xrk=Luo!5Q_XZP3v1Y_-&%#jGtf{b@2O-ByIWK-&@TKylsZ2 z)$|ND<)ZTY=Gh7zoF^ua(oo9_Qw2(4#XjqkS&Byf z{g^+eFIe;&n5*wfNuCa+;lO;vRmN7k)9z!EoV5;xF{#Dy++_#ui7Nl)5SSon&#AFPR@KQ!e2z6Bs4YI2s+V2FC(_F$9vflbjq_~E zyd7)Rn4pFxWP*RkVJ+|S=yoq^g3#I#8&^`T7zrz)RUWawJk%o}V9Gs{R?5DtTMx(Zo_Fk$2c82aZx(5!{a zd|wWVy<^RY?SG4&2sBS>Gtg~`PHQ{bptq%tI=sm`?D|H7LyI~85mYy8+o0dW!iU=u z|MlCkUF>3fNl=N_R4UbPO_jRQI=unbHKH{YMkJUWB!8bc>09>$$Oh+&vzToYPh{=$ z?mZQW>B!#Tt;qc8n?_hT$zbff50MRiSkMiWI(X=$Ia}h4h5!d2t-vn&wd2!UCC;P7 zOpnTODsmUMn8Qe|=Ap&XXTUP%277YR!D`xcm95shUp{8TI8~0g0e|acAz?tR&ZUQY zllI+dFXAK>^Xx2)TKW$&OvP!bi)(jbFMxY?Ye3%UF+Adt{6LsXbAUU-w~{*Z^xY{` zN!N94jsNpZEv`DzN%j!lHvym@jjqQ8pxG=9dvl))o8ME)ARZqJkK{tmw~|RT%8GsJ zW-d9^j+~x9gFYDOyKy2%fTe$Ofrob~sFT{qpLX!E^)o>X@h^5M78gK4&qN>=806yT zm~qQ~;l7a|Dr|B;aGZMUNKJNOcBTR)H@-Y^bh;XKv01wCFV8P>qrW3}MVC=;g0xy^ zjtASj!bUr;5*JEQu&YhqIrUvq)BU2jy9BVhsZw55YhHut#wPS!DH`OyjV1qrT`iz- zw;`&p1dC&3-q!6du6F9VNxry2C1Oy%pYvyA<*O}DkK@y2xzpgCuCqjgIF6u$Zz>UI z&Pb)So#UK1Ybn&^?EGH|w$c~2Zyj^(J+-#Yc5$62e28>j>?HKc&#*h<{pdH5cserU zgL|gAiWKo%*p@)I|20J|3nxCy78-XB2cMjJ>w9gChB)#p-dEunD`V9nzK=x}5I9&J zwRjo5G)Td7qtzIkWw4sb*F&J}##+fMT*wmCa#(J3W%e`4{nDUW6x>N0a?* zzMHHhoE=&pPb|ADR4#(OGQLc;*I4ShqQ_?#*=>MQ^oz|qKq(*~ZoRf$QUuoFCm)3Q z@86-$QK{`kA|D1s@s`g5er&7E?87yp?$86Q(?C^4t2y@T_9}lT?!Q5`jtJ8{;Rl=Q z`RW{wC-=5Krh)g5BW5Vo4}i{xq0+-=ExI`46T%~v=X+luxlD>OR`Q+IHpFiQRqJlk zDm{ZAGdA}v`rm;OpAD4|Q?;D?u{0Oin8%BMEeE&vwmA9LNc*8cKkN{LMxwdV&_f0o zc>}F)zc){!jZjJv2o41zUTpz?2oI2>#zk;LQOdftzRQsk%ZkN&wY#A|eF@JfOz zbn(iLmz)hk>>x=LAP9;w<7#S>D2pnKLdI{w#x^Jml~~@Hu39HYnkGlL8B!DTO>NgtYfrl|w=5kY<&$o@{3T+2 zgd~&l*c6(N15@g9Ymn)OVf+(lTPj427WcJ6Xnfe{h9)oh&h}|6gsL(ae?BE!N z6O3P+(cIH#7;uLDBYJ=B>Ts<{C9LDmi+(xz8%HTKD386>y5st53Qk?V%y|!hUJxkz zZC2_*>Zh^ioHp>i)I(B+E^LNtsSrY96j@|>zSF)QK9^^{8Hy8V&fr9i=}{cSp|oon zgL1iF43*(3S21_Kd>?A|PFSsi0seCUgcGl*_D%PvLt7o zU9d+e?Ki88^wo#SwA({+!?4-;L)Nl1xrz!IC|FK@FPc3Ya-yIju`p!MA5qSFJV~rl^VKpaqf*Bho_R!w)P(?pwCr4u&M~pqi z^dm;ym)~~Xj%`bB|F}Cge;m|x>}LD(n2i*7^OWs0=Tvm*q?u}giXj5(HP}0|6>d3y zp^rTMpQHrV&Ct74qFg~ylrCKxxBd$@*fo?5!z2a)+~3ITd-!^&{hwUF)5lDH9s#Ft ztx9k_Gs&hd_m!j%yhaqbXP^C`uAdS)=5n2u_!sLmBZC#^ju?{%P~S|d{F7J3+c%_F zF^aV$B(-+U^-X)TgXLU0sX0>D6NS?M+F9qE}Y&YvXR%|oZo~L!c z;F!AWKrjXIaJW})Bz{cNAbodS*Amxu;`H69d!+rpdWrU)ZG``ME8tw>0-MNKvK#sx zjMsWGvCZvsJ^jYH=_*-S=6QC({H5;pPfcL&oml#6;Qg)uZqV^g&Ux+QvB7fRKjZIE z_Q#eP=u7tn+(-sNeY_4`xkrcm!qD=2(fx^xPTq$aUmuLX6wC+EW%hv75?_ZhR<^y0 z2XNdP;j~ZmAbGYyg5<_f7D)z7U1;SXNpoWuPXGh9IolA$kH&D}9tK?Z$|0Jo#t7*r z27EZ%Fau#zq>>~fVM^sNGk+31-j0#DgzX!CG>bz zql)~^N%0mIjCAZ{syfZd39+Tj0##!gZv63?CroVeha+#J-4d+1m^pN-#^1j_FLhw% zy1O6O|IwUYlv~E_UN!NC(mbQ@sSNCjF=;xeo!J*H$;_8hHEG%WBXiQBOrU^$%2LTa zd-29nsG(}=la+f;>j?{Y67+D&HMu3%5w${$HEP-ufmIUzd&1|3Zt8h$g~a}t zy6aX;0pjzn)XU-bV9eG+Z>CCFqEG699IZvc3YGF07;`#T0>z}gm0T3j+VYwnB^alb z$}bo{5^vp0nd(YZ#VGKR7#>iON0DBTxR742dbEB35(t6xj6tjqJf{}4nITLes1?U5 z?tjh950tpqH0sUJ-b5GAP$L6)pdey0`pE!Y$nhEeM|q7t#|B7k zO*3)iGyn7xNGg)}>`0C9=KNip(N76@1R{D^CGs#b`l$d9=mSN(7!7W~Lebm@$KdcYCg zJ|4`)j}e&sn-ziQl$e0w>o)}iXkB2_VWdd12Ou)W7yvVDhj{0W*>R28pJxH8iRfAV z*{LSjB4k-8zd;I!L{tR|PYaA}-&tQ+Mn5Cq1M+?bIT&Duxr_Hc8x^LN^st{13I_1t zBI@?>mLD99?F=nG;`wIug8}r&h>T4zK=!qOnE>F|qRa%a!$V}tnxM8{)Q__#X(7RL zGy0hT1<1!4Cd>dWd_)ZK_fL4SonK~6y2tFYvNHOa0p|n=qxZnijV+SVA)4fV6*9XSic28qhl7*X*TjhPKO+aJ}u>!bX7r$f$^bjG~ ze}OR2c&)H|HI~WsTS}r6=Q8@)04Kxb_4TuH0(haz zrfHL_NU~8N0$Ie_0(_(lJ;NRNEOey3m-Y37YI5Y2NUd_BsmQCetZl^^1%7f!CQ zJD!%cvnrvFXC;-zk2lieo5C`kD`Y=CY^z)T)j$bENqlskn-e+1PjIL`yV2|X8OFqC zHK`IWxlWxw0T_zolNBpe7ksEz{{V%`7j=#eBr_A6OL$^F`?gza6+ygjpRtx+y{Q7| zcMCBhmFq*{WUN*v`n;P1soa00az)%NhT|TL5Q`@t1!o2m|DFl;iZ$)cREfO}&->+h zINnP43n<=wy}y*}UPDWhdE>a%6Iih3NOf&LkpXST#m((-^F1&0l{)+VR~hP_eD-+^ zBKPET#iT$TQUX80BKBfq0b?UQ#r!&^HtRGiZ~4{uo0<*OSgTP7|eHId+RQs2*qp|f2&AulFg!gmn(za6kLk3$BI+2qe%)>QCV4PBf~`K)GjrkH>jaa{$66 z3vG{SFzJE!>u6P?iMaGVnE~;q^ zzbm8eAaB)OZ00=snQRnNj4l!$0?anpU~H`8`r0(MXqKYFmqLS{R#i>S#$LswmR&r` z7%Rw#qMaEDGh@n0g~BTT@WKm+!Sj<-INSMZjN_Fs4g(HGb9xlAsy6jQQ+;h<^5wL9 zq1Zl+n%jZtZ>xf|2py2E)tGkBDuU@(TjBRG_g3Ms5y$S*br1FJ-BfV)!B%q0qfJ$Y zd-mvjt1#tZ)yL~)yQVyWIET3o#HW+#{(8zI;RWew2k=}uwC=cdZcYEJ=kT_w>v*}C z-DLoswdX?OTvZ|TLM5}v^*arnJhs?i&@+^r|!to?pG#3*}@qU^f8JVB$B>!>Ik!GH~z<| zM-Udwp+_o`xkJ@+WrBp3B&S>YhR0M*2aKdFS8l?=XwYSX^)0h^wP^-)B)<-ezrqdc z*rbM)fQVxuTvBt#Z{aL4Z9gGkH8oW427vmzcBU?bDtZ?45?Qu zTvj8MaS^?b8ch-|RxBsqByHHC^4J>A86XF%j%>U^EpZ#NNYcqW4n?WEQVvzxiDy{| zNtdK|BQ@xZA!LEcjt5!fcGF*LBs{%C5fMsthZKc`AY=SNYQr8T@(9wDy1w$7R$s0B>4&+F)DR<< zaX^&UV58V$iT>xp1=**vp|DF=Mv{jTsM3Wcznr*3OeD7MIB>kGU(ib^VYND(@gqJN z4gixEe^?;%eke!M9GrEq!k~$mW4~AtmbHPb97eMTP?4{w zdsM-P21dDqB477(oI?n5^HM%lWVYT!{FgZmAn_<~4Z7*(bDJ26Ndx+TO<7v+ZTjD~{;&@42lr={9GCYc1kO#&WwXfS`Eyqe;8K$W}Jcm3;Zc=D%Vw!`Rppl%eaKeK7;mO(jIyvrg>|J-$;zvD9=iKc-f&OgLLHE*O zPY0wCNv399VLe}0KwxrY@Lv;5vgm_8I?^Y96QMCNO6`)GXcVAQ=Ir@aCC{h#cY!(| z4m0E)u|+;Fco$cGY{L%PM=O^=*LpF0AB`FJ!GRf%S^%UVTsL4%4h7>`Ms}%&GbFQw zOUO2e)(o)s;(t;NB*$_~E1&vAi2OX~?JU*}zwsQQR#*9i#3V`80Qbv3_-|kZ;b{{f$A0x!jM)t-q1~Z*YgGg$tBqCuf$6Xe1nk(M|888XxS~$o zxEo+N8ULuS^GRLd=0v7 zUw=N2fIeaEUwm2vP*QnWk=H}QM{Pe3vmJ;gOt2XG&=2|IcWi*F;8GmmR&NgeKUu7+ z?Cnesd_!WOpyI@4t-?%R5WfKkrJA^01*Ha5mGV%)PY%+Bx^#xf6Ux=3e^4o=97rF1 zB$W;U3twf;n+n=hwL7aM*14p5VW2d1gk^?=+dp)|z!Kd{rD6DJLYoD*RbQN4l9_A} zsOCNq)`g*zrl79tI!r$_p^)(XH~oXQF_}?fCMB&jW2j~|bnFaOep)>?;L>8hL2s2r zq6hM_;OPIL+G6FR-539W9B)u`wNjk%(KxOdBI#k0{e$ioH7{dKqRH|n6y36^X;>|M z9B`PyAIu`S9NK`?3_ofAX0MF^_iG;5CTEnxzlN#iFM_G9yAzUTLj{x=C={2hY=xoL z3^7F2@nZIILb~-4p?Z?QdtcETTKXXcp@rh*l8{F|}MLL9*qbkxS!A5ny$ZWcw1QsAnvf4CG;xX3G*| zy(@zIN8a+GMT%n-k1N{NsLH-J$Wdn=beBL$4yA05!FGVsY05rw4Dxx^aT6GZk07jk z2gXES7HSK^^o^!gK@n-3xQ)t)7n7LqP{k_lZS;UQ1%&y?t2qiz{ZyDj%)gsju$c1O z82NY=0m~FeLUZXjY;qz=4To}IisQ5xyM7sqi5(LWZRTnpJ_XxB_ z?FRWpqwu)jwy*(yjb-SpOi0RoTmM(0VWt>Pk&aORI_#m2y^tcQg=kI|MHf5Pe*XM6 zakl+w3-=vv<>~> zh^lrGji#2TMS`lfl$<~gRKtfZmewp7E9y~lcNAeRYAH&q`JlZ?%`(j`HAH(aI~uB* z8num8FX0-E&jCr{4Kud<;C(c!7q~#2v>1HN_3FAyfaG!D=+z>s7ktoCjcQXnIJ(?b z7MR(iOcTF^lMf-oGL3fn!xl*HTfW-kESU8E`XWgk`m7foR$}?QV6F&_3I6!&L*RRF zjh`uszf<4;PUHS5RLAU_*nwTCAw~n|sQjs>1tYo$rczNxwg3v}mA*z*{}q%sFCFD{ zK<%d__T>e>Ybrvz=K5QubxMHBgVfSYgeC$hxM^D6;O9@&q9ZPg+ASK@r4dC-?Rs~W zhmy;*!O)%pktZ3fZrAt&4Xybzpxx5F%7xbYu3hnuFLTG&=e(;S{lNu;E~%;kPr=qO zaA)41d)?2aOnMPokdBo8!RNe{kAHd`R|Y+02L68Tiu*j|X@HpfOH*k1d;b0Eg5~Ps zdwDE*Lq0!rF8q+JG+mj(pW?eAM6(}h4?eoMq4I%%k*&CfF@|Ce8BqCop=>rIzkxp) zAz9mX5zhK+SMFa7c+qrCGBkS!c4%_V9LQK`bf-^gNm6728Nt^Y#)m?>H)&-LgMVDb zb^m%A>cfnC%GL`i*PnOS5r5ZT97UJvm_3(J{-o*h_ZzNCArtGHtPAn+L6C(Tfj+?z zXJUqD4|zS`qJEryzJ>N~w55s9Gp|phZ4e4?5-DvG>ur))ZIXIzJ`inuY)1FIfCJi0 z#a00;lOKUf$h2B0_8feCjUf!vFpQv0u+kQj-WIdf7K;~Ri&fJsql}rdZ$`~%XlZEzv_wAXf z8SSplZK!u6dRyF{r`5if*S>f7zRzC1ZTR*lX%h#;;@thdXM84VA+8V$E9{Fq;@%jnoE=J1(Vv)HXw)3ewDQ+4WLAL(>wwcpe=+QP*`3LaE_Uv$5v09+m#u5~&Y4?1bU#yy=0I-OfQSwx%;<)1>AcpSzMc8x*yi=j+aYZgM&M%0JU2)%DQjlF@d~l`6a3NWD z!E<)SB6B2PccE^#iScr!{O^i7;euucaYQ?Ffmv}$*Ko=7;ff92jVj}cBjSo`-<4I_ z?PAdlLF>lD>P9DW#XIB5@}DdJ`4t!R+*N?gP2|Omu)vL@-BpOrO`HdqnAAeH4Ou$f1oRbpYSQ5J zPw(_StCyL`?UIt$8Lh+i^!c5%bCuHVL%Qd$daorbuO=@q8)es{K4;UH+jX8>dut~f zXRocjV}}I$`&+LXpttGUTbCJUTZHo6A3g7Ot2xsl!9D*3eEdOJa(cO&sWhfY4<`+W3fd|DR0VeNO(|J{el94CA~_iMk8 zdh4B(aqr{oo6PDQvhQGTeV(v>pHbkIJL3f}xEnR{a=Y_MeCwN0;gV^5^p5Ni_wD_s zX*AE#yY=u0??&KT6Sze0Pol;UL%baZC*(Ct_@qTA^{V4&NplR5z0@9(uni8uTEdAy;Cy+rdb~&UILQvp1*x{p4|6#cpLEj zyU*;t@8V2A*?PcKgnw^A0JPKk$t2=#W9A`9=6Olkcey@br#&E+^}+S4|J-*w)p$gd zvwyYMUrm?CqY8xItfhS;+FKJ;IeJuK6F3*|%e4&jY6j)X8UQOt=s9~1Y5I6)yxe_o z=&rbX{PlvrZrk$xyz6#Pw8p_~kfpe3es zJ)FxGEaHh%oo8F9QP&NcL>#``o^QlW!u&${-hQr5vRnGOdht}h<^9k6xNK6#kKG?H z!*}1L+v;WdeR<5I@`3JcBnFuy9IG37|MZ7{W0cOFcbgqCZ;DQ1%W&BmPyH`pn(L0; zBI#;P_TS!L&%xIpy9Wk&rR3fs@Ypzfe}=BdeSK7_#p+4)YG*;_n)g2GT4wu}XSJ@DPfOW!MX^Y_j*GG8d7#{}WGu@8IH6ta z6?uM-?3G2*9J85pZXF+WTUCoSVU<@mxW!af`sL9r>5*2@*8I-RRL8udW8aRWY|a#} zt4EfmdbujDMmFI;)%7c~e;DHCwz3IYD#okd)!_iUj~o9~-#fPD`(p8YIf$=uN*h$; zJZrgt<UAyO)b>lUOuo<9<(cH^YNeFq9R%EA z@u^)M{o(z#^eN`N^O2IhSsk%hdO1b_@8KImCVS!>UFBIi>t(KSsGs2O=5!tXH7MyG z@gV)=1V{J(+){Ut7H!eMP?a(Px$VgLAS?C5>|f!OZWlUxL%dEpbh&!WpMu1_Lcju9Rrp3Pv1F-kbvcA$dJ+Ai|)FLETAQQ1q3;qAexE?0{mv?sfT7a}BkiFF z3RDgt@$*8d{y#RVKZslX|Dw^S4f!_j{|Ai@@Zp-lpyCD48>@dx@wxD;Mu+e(*Y&vs-4lK?0(KuQ2k zDx&IHC&WlrO~@WUm}4$Vr}Y(5vMC90j61tzXet31$Xm;WxvP}itAFNQwN|t`RLawR z`cg#r1(6Z11Q5dyNl=6xzKCR z$W{h`38eX=<$E?rS~|Y*1o6dAq#fJ8>W3LptuCLlFm+Zo2DU>a>e zzD<2+Bfi*D@5wJCkMDagzBI(+B`YXLJaW+}Ny%_4iCscVBaBc2cO=5FW_!=bW+8Yh ziBaATz2`wM{;3*CC>Z#C7vVCJt3pU{rkzbHDq1BFE?o$JuKCc6%g_oqF?h;7{~9~7 zAI)zT)SuFKQw~LMW)n0Co^I*;n_~?@Mtb zl4DUY1b4{dsRUza(h)<3P;U$I(v8sgAt+jd0_1Q4N_m1c6y{V}{K_@-LE2M|C4o{1 zF6QP1lmJ3I3J9Z+zaLmfOC8|0DY94q2!iBr12\Jz|{sLTxUS&**8$eDMLkTKWM zB!@yRqhFs9s+M5-)}ToJi6 zxgfbLi5i49G?)%pfJ|-{8ihhhx)mA-z*c}76H6C!_L)s9n$auiGlw&R1|dk&nQTjp z0GoaZNC-DeKdw@y^teJQ4`TV|w3&vZ%s zZQfL-g-)7?1;ENG{iY%W6+a*762g8#FG>kuMjb#SUVEbkQIAH^5Py}ugiv9ZQK4=L zq{`B%^L-%C2K9-5w4+)T1Y>_`8s4N@OQPGN-d(~C_Q!PsQ%lQUFXV;SWNJN2(fL`_89q zLl>$FbmR)v(&-__2uzQ3`i_mTD?9iCfcf3#a~5kP&Cx}fZFtlmvtEJtt*w$;xr}IO za2K-qEz*B$HtuSA6>0@QNC-Uyne3vFFBIuU6~Yn820`{9UR-@IyNZbO{M0TS(TP!(Q!#2*#FeK*G^3%8q z;8P7$#$xc>?QVJak{G@$uaX0N6}~`7r|4GQZ-j)3=@>xh@GZUWw^3%JTqfV}QJC;x z6Cxb8NVgP?*jbEbB#^X~5=8lOl+V&7O9(;U0M@_WMnBk4%8|AhH@uaWI1BAi5*HSkq=e05_?-m*FVekdbm72l1+g<=2J;}B9%DQb^n#Cv*7Ii8CA z+6z^OkIU;@V)hqN8^3Ss&*k)&rI(iDtr73Z$WBc0s4J+{c`%pjt+R>?_JTXo9WeXX z%#F^?aUHCuwuL+ELqD3`u-HjKkXJ zDeFZLuFr41U(?~lR&TJ-cF9~TIDus@E*B%0I>4pm=f7PDd^S96uD=$~BnU7eY^;%^ zgz&St&_x|eoKn{?1n~7Ov*0vK;scFz3o9`vN!22LG<*5(B@a=CE@2f zto5OBTsi_gxFsPHXFI=5vuo5sO@SQe`7R%}dM5kqWmtJXt!~YY_!4P}Er_0nkIWr` z>0&Z1T^0WNviZhGM*3@JdoQc6=K%t_R<54CgF8l*zB+Qu(0o230coaWOh ziRepX8RMxG4H{JfC(R&u9=t?!DtE1@Y&T(it!xPX$Uwj;AsrM+uT|$OyYL&*Q5}1b zzV#sG4Te3B&tfZnc)x(e(j=ofSGyp$xqkiPy85*MN#EYvD?T!O$7pX)aR5b<$&3hceGJbBq6 z{;1zAY~g>yeD|QrYt5m(FrNK25T#>+oh!3)Em1|AopK)gy=S7M#?WqDl z*y(Gr$-f~9>hqf@Sl@C*W_kbTiN7lZ#Zw4iyQVQ7^(y+DCmxPe+kz{@q+$5AGx}@Aqa8b7Y=$UG+Boe zyqLak`!D+zX}-=2`zws zHHNpb95NoBv8}Qb1?Y1Bqjzi2ysCW5VR2!^Hi?m$50h{jAbI_d+suCjs zeBVi$VF;>5`ZdVnm)equ$=|p(Ybd?#6+y-+}LE!HCmR|Sp5NCu{jx`JDWVO1!esAp! zmUzoQL#F9cVW}bB9u#KCK)e}C4*YK4@gmgq_Cn0d`jU>)4njNW;<+%yTkCh@E5vF5 zZFm#3)@<$Z*RAfwivN;XmgnjOowWEky7^2AX)eCwV$RPx5bi85RxiPZ^oJqAnq087 zwM#fN>*}D%?fzoQQ2^OoEWxkEJ78Ur5gRpRG)9tO2ZCO26ztb#F(5w0;vK5-_Z#5! zYS|d26avPE^Qolp^$aFPuyk*ew3XYvCN8IS#S{Yt!I_)I+avVnema1I=jG zjW8jlZi75gzo^X|c~ZSIm6{8ElBWFttWvOR2nR%ln~^=5n8|AID{767vg+1Ek{T$Djc3tRfjmN@vn===%t zE0*xjlIV(Zr|kqhk#hH3q4K>p3+bV5EB2@upOei3G9vuFE6JXvv3oA_bFPfRs1&r< zhFNIs=Fo(HFdZvsSFKEuY3sNuwnvTwKBZUx}lt9Z1Ldby<04p-4#SI$)Gox>0#8OKM zF31szTWpFk1Pz+>j?ECD7;&OupzyE<#V)FUM-UHT@XW$P5lvLlMHy|>(MKVTw9$L& zxyRB=G0n82OnDTEQcX3I=NVB1al?%~1j)phPA9swqD(DK>78d@$)u4+C>iFQ@I0yq znp!ELM3h_3VHKlFUH<*ZQd)zh#1v!9tZ7zLn$7hcUR#BgJ#8iJ*4uBv4OiT8$t~A^ z+;dZ^M_qNX6%A99lEo?Bp58^*-h1)QSKod4?bqLb0dA1ebqOxmV1nBXSmA{kZrI_6 zA&yw$i4}wfhBYzHSmTTtkdESyK@M5ukx4GuKnQq$Yr=d>m8)&Gm+Ul#ZuG-kBx$fHQufYym z?6Ju%+w8N^PFwA@*>2nIx8aUk?z!o%+wQyZ&Rg%j`R?2AzX1G0$A{%{lMf^UpyKUG&jOFWvOhQBPg<)md-d_19s4k6rfJ zX|LV(+i}lb_uYB#-S^*t4_^4;i7(#x#@&X`|Y{! z-eG%kNo0%h$uHmh^U+UV{q@;z-~IRDk6-@z>93#sIyy1O`~CUvA3|l|AcP&zfPe)w z-~kbsKm{(4fem!v10fhe2~LoIf)%u20dXLNE#$&~9rWM_QIm@_;D81=FyRSNm_ik< zkcBOD;R|6HLmAGHhBdU|4RM%40v;h3`vBn&ff&S^U;_?0G~yAFm_#Kmk%>s8!4a_G zLm^I)id7u`NhgF*!X|E!i(T~M7q>`*MA*ZMWi;a%DN+Y8u91yxbR!#oX|M(Tp^=Vt z6yg@N_(nbMk&k_JAq}u$M?ntKe|YrcArYC#H3E{5jdW!845>&-PLh&0WaJ|;naSrx zvXY(j%Ht#sw?PN_;+&Jv8QeB~{18M|20l9#aUie|Ur{La=dGuCcU+a4-iM?V*e##4v)tyy^=Xh)4`hlBOe>YaWH* zi_GQqg^55}Cw6cI(cTqOxnN-&b5YcasAH?OwXJOx8zG@_Vb+E2hefyGvPB$aKC9Xw(fO}Bj7MC=PrJ-FFf>Rg{ zH4*J~?|a*8UKJjpz08f_4%iFdIwaMN1Uw;r#Yf4`isZ8=!om&if*U2-%bt*crWFY`I$_;9BtoV4!lDSq7!nYR7`rJMFNRrblDWEYs@zqG zGFiblRNP zv5uE2*)3>VQs%hyj$zH?A*;9(IL$Hvg#d@0s${rPM2I~KD%{sdm4G(Z2#9fNz%3g) zBxbIIY-5ZP@YVW{RIZ3*VNJpe?%1MTCgBz;l8YUPAk=Wkz^WHzj>-C0uO%ciMnt&7 zG{7N$4O52gOgbTgBd{9YQ#L6@urXyQra=T~PEDI*_+}2X8-;SVhdLmR(1%AH8+F+5 z46~}|BFcUsgPF~^n3oYtwMZ$yM~l5 zHV~f+I1CVqGU>zq(?s|5) z99yFd_0nM1%~xX_-LIhPfbs55BZ^3W?bgmD5^Tz3D2_Jb_GkbOuFv)!OCi*OqZZ5_ z%%K7=BCX<&$bQY=B8(y~Z~|ci*ivf025$>U?7JvpAO4T3CgQl(gYWY8jrt@6T*ghH*zMzJD)B$3}B5Xfc>-B!aKyefm3?IHy0iAJ!%fb7>U&>R@i z6-nv<#i|6sF9AI;nZ8UTTrC2tYPTG$x&mppkPCl=Eg{sw7eZ{PTrk`w3*0PkvY1K= zw-FnMs_r0*t9;GlxT?2)O$uYH7>S zOV`F{C$(?Uh%2$WY8(@PLD)J$d;o0mOh^dQpbpfppf2r%*z2tTi!6yUhyL&*Ca>Xk zvB(G!`mm89w~?!&@Eb`?8P4vnwh|m~3dMYg9vLbgrH~=tQO~3hxwHznI>7{!G81R8 zA^oAC>JGZrQ5k3}8(Sk=EVdSEF)z}=NF(4ZXcx6lrz}E$Aopz^x6&>R4JGxm z6Ry!44U4zDafF7yJQIPM9xvDYD?EjuIU6w%#BRELfq*XR=G4v?_zOiZ3LDz<>!eBqSkwNp zOcObxC0*#vR8jvfZYGDXAAc>(2FpJwV#a<_s7lDO5JLZdbjT=G=)n9zd^m`-aG>w} z0jKmQxaRW=C!!N@010c2gL)Cvd}~3$)VJvBmw+@R)IrQr2oCtt5a&|zkV-?(Q`q*? z$4=DRv`#}s;L`Zd2uR=LF#$Cq!LIPN zlr$r_v#MMzgT55g3QFKeEcFhoz$}W|XyDmqi>@+%4zUQ4ni^FC(J+hXORWqxkhBjV zQK~4>OC2T)4yI{kPU;gth4OT)(TjquU=342v+OHkb}xluXgIn?8-=Uii7N7vaV6j5(|IO)+8Zo$dqA@5O2TO zbh26z*tUwAR&Li05+l_i(U7lBKjKb-4uxRL9_8}+GIFnUt*Zto4ag|j0Fc^>Fu_PD z*Lb!)J?;KB3eNI%-FAxDEEoR1bYp+5Q8^WV4NXXY@brmD402QU|6UXJa3Bq~GO0`p zHFx&PifuH^!OUXqcVn%`PM1j+waTW72;h@XxECU@`#Op zXipXgB-Zx$j@gu9W)Z3v$!uUXkn(b?WjkRX#B5Rb!R1VdW+Q8|y7x0GHiiWBBIY!= z_Td7tAsy0T{1|HE_>YW?^c2%7gH-N>`ij^L4o!nDw=Aw?Eh0W8Zh-0*X9LOQX0217 za)sq@xlE`P2iR6q!f08DNT*9|S&wCZMXw+8?tT&C$P`Qa_9}CWP^cW&2S@cvxz=w_ z7^P;-iCMF3ap0tgO$MdvE6G;9b}FeCWDTegm0&uj(t5&f{&;z=XXw7PZi^? zPs8m?Y4K+}D%tqeAFNbjArOfJj)XG)wX+=!&m;s$g+{z?A`x5YGx8T9$?rRhsj8&=~6XR?SFnm+yj3&(L%S zpl>I@hoxu$s_fCo;^?Ff^nlV7_Qtp}%`8SW5KRH{WUDYC{MD`=?_aBwgx2h{epnfb z4YGK+jUCIU=qlx;^@L%sNzckt{$EoilsL6`%!HWu#+DX}nKgf0Yun&|?#SXw%qA-h zfD5NykIY(jki*z(gSm_&>5R>`QwIpPt}gyk>y7W1j7Er$1*?lu^Ms5TJ;^wyOl#8E z5^yAJAb6k6R zd`uhIw2x?)S3d1?b^|FI>+hK#+9&2Jc<~UF8&=M6d9!0$A?DOJ)2^ykPL4EejCzU& z3(oJ{(g%>P3s3EjHDy2jfwWb zeGT@gO0b*^zO=7C@yM|Pj=n;ww_NU+llb^#DAhJf5}2!~FZ@W6I?AF>8xvxLc5Sy5 zHU6{{A=Z+^uvCYCc1p~`wyVkZtHX5RG-$+p^Uu!OAJSUT1dC?TkUfJkkZ!7hXY4EI z+K-J0kTIg-v}oR%wk)?V1N}PgEHl8IY>|=j*AN>U;*Yx^kp>+br@A;HaLTcyN~ai6 z&(rj~H9O=;`O*3+@3{WFN=MuIgc9|#2&&+0ae>*jN4C^|Jl)gRm4h>g$W}PV*3E+* zGr%Ugm=}7XcsS>@PdSY{HN+3`CU2Sv*_sEamVK&3p$(odnP?|Mk+BHC-fK0P+kO8^ z#tEzH=+`szSD_wFfk4fK69~R%8IysnJwL2>Ev)^E!cW$o6(Tyx4W;z%##s6ytW=Kdfq;LFhv9dD z*z`*a3cmoTt|l1-Q9?)~uoGPl$CXycgE>jN{p?0ar)06%iyU=dE6E909>MmMwaBZ} zxH1`#g!Htb3=NL{(Yh$-_VtLcDTmvl>@fnzI=aY^D%5=V z*YPNSE76-G4sfpaI)Qc|*_9pKr&;c?{UV;-JM(w5rwo6#lEhMqx74UN-D~zubG9h& zvxK0k?0c_B?Dl8BquMy7#vZXv$ed{cIAz>vpZl7#t9PFW-)#{M)r-D}3r+o97aJ-t z{p<%>jl(Lj$+NnOH3I|y2&aU_>Nb6YIpps_`nzkP|0bn>Tc3ypz0 zPTD(y1Dii=FhACj=+BxiQhLV*}H<`M!YCqs){WOD4ZPhFjrID{n1nTV=? z$E{p1%2Wt8tk|&$vB6mx6024rA{QQ^VHIK6pE!<~Bim^Q+D^H!3EU!4C7iEnn<|5= z5$sP9C)H&IjICo*u#(MnI?^)iVH|DA=By*4Y1mu??{*S`Bk|0$lM`-MXDx*_jU88s=Q<(+4on2Sn@M24SOf2niY4t3OK8*RA# z2VaB|PDo*e7G8*9h8k|jVTT^JXAd@=V3SKlG$f)-F3QN&o;j=u=VD!8Jx4=-F7Vic z<31spBUN2E{x}mhERN+14c5JN6Ih_ZVNWg?iS#3m*sy2McyI`GRVQ^Zw8e)Snj>Fw z110I1GBB|cU56@(iRnOp4JpQuY6)$T5m;(=bs|Ei5ws~v_l=oQ8au#IlBQcE zri>5{sYioQGXdpMQ5*59X`H(Xp;1Z%?I6=jG0jv{t_kJ!QD$@nHPlC;9yQr!1f_A8 znT<^~kX34Bnkfx_T9}})KA+*HMjWuEeC1=7x3t zS*`?b0gqxZ0d|zMIOcSJ@;*$Tjdt2w|B2DY?m| z`cS6~hdOZd$_aZaL5X%Ef{|>Q4XQ}bg%48fN8EbWpwHHHQK;s;Wh6$4u!r4AKpQ!k zvN9o86^-mKeK~K>YN7D}D1rosw`+u!BKHG7K}+VrDh4ny@Tq7V{HnaA2B^DWqeqK?%tG zL6OwV2qryQ&cyO!oy<`zN>r@kkcLAe>=i8$no)+DSA0{6&veJ6As_Fo!S3!x*{HC}?+4SXIu& zNXPU98%g{PNL0j=Kho94%s|SLerdr5xU>j8tyr*E4k?=TCEUQHb}EjsMpQ>Mll_>gk|P8T zBqI`$4jU;Vit2T*e9aBTxL6lfnnOgcT8VXZW)*bCYIbDoQt-}*v}^v3$aq%as3XO4 zltRLPr%;}36di?RjBMgX_H4zP9uZd1f!l7%It*vcui`(2X4U%+| z(A{7psCca;W_ppUYmBsmn88q}Ir|}b>b4|_?58fi3mReIah6nlkZNpXEW5!VEPN?2D#a#K{^9F8)6vx6bkj=UB09A+e@ z5OokGglbu%hqRYdRMoS3Qp2aG^tnA#h3R)FgAJ$dH?lz@q(=;W8X@E7i-#zRa6qx{ zNOr>_|8;V12*Htuw}=B_A>>g;dCL(}oDQrXp$;!FN7J z1Drrny{kIO=?sjTSTfQqEG@L54~=M;jZ%%vi5Y#jpc0XA5ia?q3LKO{G;(ryaa7#_ z4xI$31LC&JWHrvvR-)eOo|0KB@f>}BgHyduSDJw8I16Jzf@L3(@!XS<*bszJn<7(QA2!DMESA#8pve^`Zn5EM znLGz?Eh_D?W|b1VROtwndPsnF&^n~tn^@SWN+_S*UWKAo{s^HRnCIenUM`A%>ypfN z>1CE(I&dTLbGkC<=O)Y{2q&3(5gVmSMWu064G4)FwiTnKgrejrNJzBR$>OfM;MSIQ<^bsRYZomS zt^%-0c!{jfV8-n>Z%Q4IqL|l@93_D#xJIPH(j6dvk^b{T6NM~;8H=cYRyfp#!|FXZ zdwZ`Q-vDj+M($pCM)`XB)UUqv*LE-wgl6|pMyK>2;{CCxBSpDT5yH_~eieZTofZv? z_gOUl^m|b($DIE|xPx%qUiwYBFfVaJRDOKn3qqn3B0+5@BVg#{3o0TcB+`F%!f)g^ zfHB4$0#<=;gn#;hfJMW99p!g28mJ7@F@7KTVDhwa&x2YOrV!=BH?ol&FtY_4#u8fq zQ*~f`o@N^cL~jjpgW)n?!bev`eGYX2ZOD1Y(PbD zHV9Bw_=5xiXkOPVV0R(vF)RNpNvw?2NekXGnsrOBCs2CSGf-%;I_t$wm>kS){xB;aU~7nxLl5;<*SGm#lWBAyo^K+=&T zNs=XLk~vdF)OeCBSzC)oWFX0sW3i19MrG<)lK@te2dR@g$&)?llRw#CF?lwnG%2-l zDJUqEaAQSFX+=rtlurqjQ7M&ENtM3llEea&P&t!8$(3E{m0t;#Jc&V7i6QQ!7t8Sx z&PSGO$(C(@>6UK^mvJeV9>Pn>*OprumU*d{d&!qSIhJ$j9wJf)X#i;(1%`l$n2D*F zi^-Uc>6ngmV{eIX_=R)DTn!(8^L~>xtX5{nxQG0qe+^jnUs@>nW?Fot0|VA zX_~JIo3SaIvq_t^X`32?nyjgtyUClR{^FXq37o-yDV)PeoW*IJF`1jashrEHntyqm z&k3E;DV@_voz=-S$;q7AshxW1oYv``-wB@KDW2mAo7lOX=ZT&^*`4Icp6%(L?+Ksr z>5%4`p7m*;Iklehsh|7FpZ)2d|7j@&d7lGHpb?gz0E(aqs-O$XpyWxP1qz`BdY}zT zp%rR>p%;pwq4}T@%Axf+p&1IIAu6IHN}?vYp&g2%+WDa+%AzgmqAv=giFBeVN~68G zqA`l2IjW;O%A>_GqcsYot9he6N~A?yYw;G0lxQ3Sy4gnSj3alE5tIg`Hm*{u{Vgde_N_hiy z3AJCVwD<51=RlXB7X}oU-FX8!3tP7dQ4iy#$#a$L|_C&Ajm`z1h=rphfK5YkOwIc0veD4ZV(pG&;ubr$(5V}7r+H& zEEf0B2pSLqFaXLhkOHrO$fXRk?~n<3&;Q-3ON7+qTm_OoCkf(A>TmDqM*?m&6i(%14w_<3VqDUOcvh&1tH)D!u-sQ z5CTxp2%_N38?w2W3)W#R)?-c9Wo_1Hjn-+c)@#kyZSB@?4cBok*K?h@7a|W&Jsg4n z1354Z!?DXL0N6%A37SmIg{`u8-OyxV3`7tDjUW%iaSL2f23*hsDWCz4ZP=N8vWN}U zV&MiMKm>o!@E)R&1StRmhOF7CEwY?V7N!6LAt2H4kqNg<399~W+aJr?V=>8G5XSIf z3`Wq{kA2(2ow2zc7MUQ*jX>B6ArE9L++raLliUavP2ANTv11DjpX~;l``rFu1S!DR zVo?vKzy(}T+X?Xv=THw`%*vx|-R}*t@6ZTIU<7|T0NzOe1(~1_lAHob(A!`k-+)~P zn(GEePzGf{1ecr#Rt?_`uCVXm28TJPfy7)nXwJjR4rmTn}6@;f$ODgYDoY zuCVnW4=JwV@^BAek>b=17Wa_iEuIkXFbt+33d6A6CeGs}dk-|;<3TRuLr&yHZsbRf zWF)Deto0;t+oR?_ojFFs?S(FbGrt@l>z|n?PFy-|GmE?+foC(9j4Ol;H6n|M8-r z3TrS0C2#Un5C?jY2_Vn%EthLFbu}9&Bj0rVNmiB z5Ajog^1guf2u=5OZeSS1>fM+>rMHTuMQ6n_iq38YoH4E zQ2Cz^`k^oSqfh#!Z~CW?`l+w_tIzta@A|I~`?3C)Lx2M-fBIb?;rQ+k&|nI$&U51)0zO z?hpU*U+eP^5dH)X^f!!12tn@%E@ary;X+_*rcj|+vEr04@E&gD*wN!hkRe5mBw5nr zNt7v7u4LKLf8|S{+yF)vjgh6`HYITvE6(W(?ZEy8hq34TBN_#;uz0 z22rKDMpawHi4`wq+}QDB$dM&ard-)FV?jZJq|g8(e~6K#{_2$>Ql?Ch&VtU_Y&NyY z64*}Or&U*7zA^y$^FXYcr)v0<-_>Fj33M-lw_@2jEJ-v5690~ByT0t+!EX@<=3;RB}lre+05YA>;mFa!M+zwDL+Uv;3{Y zC-D<$LMu9=QbajZLVSh)6sG6?9NS z3pJFy7rSHdjcdY~Vv0C4I&@M>E4B1eOu4#enL^(1&AVrlY`~+s7OjA{LRaRSd zbt8GYS!Nko-?;CMcH($c%2#{!^;cknB{P>uij71PG`h0{m`A5jMN(m-m3CTctHlr- ze@Xn%c3VZvu+NHi26=RiIIT7JTy)b_*Ee&r-S%BS%+Rh7U@CfcU3~M^cVB+P$^~9{ z?@+_RWyFQln|>2kcwvScR>_=50ESoIA_`H(UT7WGcw>(KJI+@bMI?Ur4*aZGM%O9U zz{X>iTXy;7R+SNXWEko7)7gw+)_G^1e|s)d=9#rsq>%fHD0tiveO7vDrkmaon~;N+ z?XvWVcHp$s(uDka7>jLd@S7b$U@rRWAd??t@A;EV0ZMfqWFPEq#jwHPL?z5s9 ziKf`3T)6`md~m`Qqg&)o26D0pO9o3B(842^d~(XOQYK=6B?-@;Zq~Gd7KyS3f4p+i zOE>*=mi93x7uGuwS{cRjBgOGGAV>Xn+;i96A#5<0WT1OI=X;7LY}@^K6Qw01t=OG`w!3_1YUIHz6K@3u=e}0ez zssv&Q7?FZ}1ioTb+o|*|g(>#*NE% zI3!{bjkqEeZmWmE)6VDO@PsK`<%mT*` z4x$CyXt+vT=2Dj?!Q9;lf08I4s9_UG+hsA0dCY|X?GDOFqb0WUp!y90kI1BEHBpH< zLK1|M+j#@+hG4!+PkaI+%4qKfQNf!8o5{mD~O9Ce~Of7M?I>%UMf@` z0mRQA`gXsMrc|Z6A1LBCb>LK}LlqpzBuS9Y z>CIqNxF=Da`qZTzG@&kSkQ&zDr$Gc2s$2f$D$k5s(VaR|J7#DWEw0E=0u4%JLbRj(hQFeQ*Vt)C4L|+cV_o}N;?!dpc`!ltO4&S8qX2!s+j zP>OIHLfjD01scy_uz|s{z}GQ^q&}ejP4((~VGJ+k9)jR6N6wK1gbKl#2!axgpme1zeQ8W*I@6R0MAQ<2X-ZVm3Kb4RASf|uO{ZGb zt7dhpUHxiU$6A+;{Rbs~{&af|(e zdl=iML~n`UUCrj<7C)B9fOFu6f!tyk@3;rp{ZiSxGtUnWB*-X>(D$BmUiiYVZf`?i z+!Vl|5&qcl1&r%${wm}hg3y2laO)3w$OSr_$w6?@f?|Y!P9Cv7u)I~|JACy2T^_H< ztRUR)-O3Kb{qvvyEv)k%K{Q$*_n-t9pkM?p5Cr>6iv@C@-D?P7sxT_Zr1WdRwyTGC zh(HN^2hX~P37kN(gFp%#h<8{pyAZLo*dRr42!{H-L~=kBOhMm(q%3NPZ6d#DQb8G< zLCZL(Y?3g4?fCvZwh2BO>_Hz?3l==8!}k0KC~iD1Ab ztU@cyidM>^$pfDzsxUaYLNE-&qG&=Fv=8!QEip_(HS7s;>ZgVHk^NwsrdmTftizUA zAwQ6-h3XD3%se~%LqHS>dx#p%DZ>z{2(t=AMO?&xkkCEe8;Ft0K{q)iMyy0jWQYwa zM21MJD69ySxC4{F(%*D#%xtgLzZcM?; z(UwksXhr7!LmTa=ut7kWZcIlRe79?K#aY@#itsCSoX7M7KTX_3Q%spTq(^@IJ3Pe2 z)QUxE>_>tGJXUlFS<((ET)={C$iJf-_W?-gu{S?)F^9ZJtCOmB6o@s_LHqGTj10;7 zVy|DU4~678kxa=98>Ef&p>o8k^HWKgbgXZG17yTXvBasb`X7e? zCLyt(v0TfiG6#0th!U(yzhg_f#HpK;gqwW9v|NK(tV_S_D7-tshfvCUqmjQn%#JdD zho~7r{;7;hDgYD2j7*5a1_Nv_htNxks5;5qOnf>cKj2D%fWs)%NzN=yae^d5yvbLT z#K+u6(|pZb@=RseM(DAXD5Q-@flb{!CdKrhn{>%eRLk8Q&Rk-jZDA&$bfz|GP2y}$ zRk8dia2M~4_tebi79J)x-FhUh z2oosH(%oUnjvy%$v`8%t(=my_@~BAj4-r zREBT^zYv9CP&yc>j=9;Pj<|=8q>Vwq&`WLA34x1Ykc&h02WaRmsPKn#;DQi1aBXc!y!A3u>)aYrR%%%~ow0-BxbxR=fW2 zg~e0`_|Vp8z=TN!lV_M#Z>|61MF_+*iFb%teN_f{$^~cG)@LAwRIQD;Wm~+>TfN;| zzU^DT{ae5dT)`b&!Yy3GJzW08OZ@%=mBixF@E8}%IsXyC1;%f1wV zg84Xq4-v$M@Z6#mdy^H-Uktkj6a|t$h+dX~3sHcA6ySoz>($ZdEn|3xJb+wE__6aq zP2cngavaT1gkae6To>~GTy+V757}H`R}`z?>qR4WVVv532T6ek8omY}UWPvyV>M6%*!Uqu=@RbX zj@Vd&G9F`C=ms#p1{(gJhB1ZIOFcmtF96O1F8Je2*k1XKPLGIB9i%BOPAiRAPHWJt$$v1lCp=t+O zh7mQ+!PqF{M_z_@{?G^=W)traWqEjHQlMosZr-%P!EdE*|I2O0KUfgXr|eVE`FS%!rW=z#Xzc930i1{G@Njt^O9SfJtG z>?Bt@Q-nlO&2`;+7N>St21s54YNq5Ue1k$LW?F#WZT4I$7Km%;WppkA2muKho?$r_ z=|AX{N#19E?qu72P)GVzEi7av?dXptDB0~4P|*)I2I^U%;~_@oYZw?!{sW@DD81B1jE*wvHopFvOyR{s!IAjFseuEWGoG&*H+=m zd+)N2Z!+44kUniQ3}&FlU7Oekd#G@GFo)Ht8W&V4c?K1WAwt4WAiBFqUH} za!KffP7rcsi1H5yaT*7IiQArC)K2jwD1@LMr--?OsqzSU_Rh_%aX(TAG0t*DS?>B= z33~wY){%r_QG~j=13Ndspplq6$Mfc)^I|D-C|8CLuX2vChtjs}9Cr|6=J7eAD7?(; zRGn@!AES`|1Nz1&VwUiic=I6dgh)7YB+v8i74O>I zK4l0>lEg2ZbSsKoF_vk)5o@w`2_FY?Pu~_k7n&sh^CExpP5^ayk%S-@aw2DPIX4zM zXLdjSa3A+k1W8~WY#4NVAaRDE1*Xnr@91H50CyBJJNA}T_kLzvFQVcuY}I}POMnGJ z?}!V>2G%k1V(;^RV>$I)==KUw?h-N_XA1VTR z*JfnCpt20@(Dc(Li0pSqq1ess-G23JK=+Nv_gpY?ZQ*k|-}Y1&Z}5|tPEdJ&@Np0C z^iOyAB;R&#ABjp&ZHy{r7SDCvGfNeQk}@B8lD`H^XAs_hUWSQ(2nt^v%mJ7r2kol& zhnjzvM|TNx2=Y(Qb7!aWk${J&CwKlJVk+|NR|1JK_Y%f(`V4}H;x_Lz=IsRUh^u#7 z{)sVl{{9HJ?D?(7d6Cd}Ij8dt4|pmkacS1=98?2swj%J0r;q4{R8<+fzaTXK0|vzp zde`)nP#t}LkAz}BmVl255BGSnR|bX$825>UC&z|w2MJb>=yGod6FwqW^4>zuTuSVG z{m}?mQ1i)W^p(JRt?z`0?|QLdd6(DvZ4VFcj$KRNd#v90{F&d7(8U?ye&R13T2KQJ z@s9ow|ABz}cWJWQuM7%@2W(jO#wYZ40HMRK;3FD;-gSQy{E0WKqej+KeXU}1~KUQYKTtcdy(XL*gDnr~c%$*CVjo6jL2c5;MgF;xY*;NFjrN z4q|v?LFQo7Nm)Y*wNOh|!8D&0q^M)0l`)dllOo_bB#$L5x_G6SW0F~>nP=YkS71>V z;UiZ$<-$`_l2BKqnL6dt31+Wx)?`BJTmu-JR=i;vnuikST2GPq*-$ry%-7#oNlo2;_S)(F~F!f+F2Ekt4Y@uvQLMW6;+wC&`_`rAvpp^cuS7~!_!M>bw#JxdoWZ+)&oY+m zc&i%c@>?-S*hHs@G7vH(uW0p#Yq7{9lU%aN`ve-;uxYKp}wqGwbFqrFtg@p7hPtK z9h?w6%N#f8&sj??w6CW&%@ERmA2Z#y-FM@icX}yXZABr=cpXuaruy+%;Ogq6r&F@# zjJDNFEQhY%{K|QUdlAh&m>QcLKDp&#> z5Hh?aa4v;1oFNSl^%)(1dB}nogM*Zz!n4NDX#CN&}87L8;N zA4+0LWV=;UC+UIKlzte+c1!6a_^yu?7zEKa1?i z`4kuv!hoTXnH(i4OUV(O(BdUnfgUN22omuDg~+{9hJEsX--F&Uz$l1%Hhp( zqBETX2~;HqQy40LiKLn!jN54oy7MlTK5!qadZLyhF|&P4@N`;{WlP@z%kI!qA+y1M$`WWn)S^N)sujCN5-X^c zZc21}72Tpeoe~tYQFWU%p_o)rb5urA1Zj~vVOiVi)|2r~i57$fR-@T2nDcq-U; z?9>j&4QgV>d~03q_?o;X2H#b>u88}3m0>=D~}W*s1?Z5{?x%*H@xD#PL8@eAMX-S zv;P8^YfZAC_Ox)67hM=OGzwS`3AP|-BgG#E_|frZ#D47pf1*U{K?**a2e|-VFoX3n zphIHC2Wq(8%5dc}lqBg|vn6Hq$^t9@`IWyBbEmmHo1>}`2v_uLup$S%;e#xr5NygX zi)(CSyto9y5&l$6-q2r}q9kUpimpPzrZ`s_Y-7^p*|2a0$6>9^T%W!1Kuq+jig3!j zCfp&FyXW6PFf;+OL4c}Y zIs-b;3$cWb#r!ifn?xz5iey>UC^KMP<91?>aPIyrX#RuP`KuL)v6inoEqnZx6r`Xl zcrR`0Q{O}#gr0P55M7eQU2DpvBvt?FgphsEf@iFje~B7Ojp~HhSrYM8$X~3c2J`&2 z5N}w5707&`Uqf5a?Py+icMZKck0dDw><*FN+8Uc>2E@0P&#rwS=V(L3$`rjw>56Me zC`&kX=$?0tTSC=JT3f=t9wRUfrdt6+($)r9Mxi78hHm)h9`#r_!xz4ghBI7m(!7H$ zFOp{^e-<1j!KF9H!`sDcv^$5^F2avYk`!yQ@x@XVqE9l@MuO0$+*8`OaOCj{QQ$o1 zqR50X5&{jK=lmNImy;cQijj6m+S-3&rx-gdpu|55Wb>Add_+LIZ0Le}M)>F*)C9>9iJcnDeZ?ai;@bs-HQI zmSIvD(@bR<_@0T?UyWTxviOl)s}4|Q{GlbNVe#at$<=vt4leW{^rDA>2Hfy=gWRJL zA%FoFTF(V2^!)BV6zD~+k@4EtB=EX#X*(RsBgoa&A61DM=1WeJqr$Drv7J*?u0bF( ze|YN-eXlk@X+E4=;3B=j7pG4VH>*P8^U26%HbT+AsUK;FC;?#)ZrP5f5IP> z-^$d6kLkfJyuuzD;vphpA}Zn{GGZe-;v+(0Bue5WQeq|6;VZlXAr_(~a$+ZX;wOS) zC>mlW{(HTC8Pihq`))ge?aVI1T;`WCRpP&Vq-RH<2G_*H+thYf@3&}<2aI| zHc~<2Cld;_-+Y^Z^=#-Aq7EBUt0ktz$mw<393ZKlu79qgg_EV<3-%VJ0N9JD&$ir2tL*_u)p4o#pfE?j{%H=JGXWc_rUS(Ul zI>ay#K?rzX@_e8jyvBDr1fVQKeFh&( z(5J0P-5dBAW-;g8q(#Q{65#&fTGE{&DKN)&9^XXW!E|6#)2W+Feg|s(6>9}11>S`* z2xT6egG8t&d!AnMe~{5nV2eng%)>YoAQDP{Vd#JA3p}V2tDRfNQK6p2)ODW3)_C1$ zHJxE~r@<5n%Ah7q7D@K~F? zK_P_71S+VDT8U1n!?ayPnu1(?iW-0Y!J7i6n1P5Tys2d&f9Nu(X*)E>m_k-NzygK2 z)5OS|Tu7c7Vd?IfgDz^x_Rt@A0>(b&4z}sVI|PAwqNhK|1101@4&;H8WCR z1rWhAxDV^Z%5EYAl3)Y=k2XYp9><8?%8H6wMBRyt4(Em}gQ!*39~44EssVrUXNqxY z1j;Fa2rGXcf191c0-m-5pB5F$JZjJUjAzY7^@xqZ3>(Mw+mI=Pb}ScAH0nS|>%0_! zxo`!JdWD8+02kOGLCAw3q+J;ZggkIT3T$c$gn$v;!jrlWtmV(R9!Z=;Lcw-|ss_)r zQI>Ib>4!|1Wzj;5{)bZJNiG~}(C9>~GF^TQCoS-5e{nV>v5FPO!CP47Tgd_f-9T-=lxtp!O}^SeB0!q)^znqBJBk<2!b1kLWmB;Jq$x0X5UOT|tfWeWatDE3#6C0wWd&=o<}K#gDV|QC zIwY=Ykde3igp!fWm~=3A!{N0fo0j}1nP$}l#%|S)yO=a32Ik=fLfaFDxE4Dm!fQK ze;{i^)~Tp9EA8qjQY^%ZK+mzgsqq$pB_sr>4ezc}Rw??{0DIT+_7|SQ-#KC6S^7=- z#s%0g#)mPC`%XmHH7IqcPH(Zv->7S#q3VyC5jL2^6jN~(TX7Xj@fA-oQX~a7OdCTs zo8WqH7#GH|(Sn+$sTXVU8ndxfxbYjqe{mek@f_1}9ZSW@{jO8IiyiZEAKxx|@Ra`f zuX?;Ne+33iA#aM=aLZgnH?#v{Ag)Eg!>^iwQ<%YoNvu6|(eTQ!_onPSIqN^A3=c;R zU98zp2%G-nT436?PLZTwkN9xl(Ha#0gd*7TE#q=7=W--SLMQn0F9UNh`?4f#f5IS` z!7(FqGAlDOgTWwdLL&6CFiZ0^`?AGa^EG30Hf!@Xb8|OatSh?Yba?YPle6Wv>N%rx zI;-XA%5Xg;~f5R4ON~}t&TJD1mg-+a6tjH%3TLd`wL{JpTb0$~? z;%p=+DlNT#K~gLAQZsc^JM~jT zbyQ3K^;A!Fci40|OF~v_^;UCrS9|qWZ#6lK^)-KMS(|lP1H&;l!Ea(7fAhMv2b%#c zD1$q=!#}S|756h7<5HHGpvH_5*k&*yD+hu;auYiceg5LDN*NI;Qd_nXM=WVi}q-fc4?dTX`^;(tM+Qwv{B2ncVuFNfi=?X;`#Guty#<3jTTvqgD18g;pvX#O7v2 zEbpE@Kp}TD2yTNncz-iBlXrOob1tYtF)#Cav-f%%^{e2rY|Hn2(|3K__k928ee3sr z{?~U*v-W=jcz_G|fQz;+dk0K+r%%Q4avzbxR$&D-n|UCJ7P%Qyf3;Uv5RLxXGLsng zbT5Sc(H=nzAqB~iq&gk)2{K=_gU2=`v#}b=!g!2tFk83vv9;-GFx)-pxQ;UfcIC0x zsd9v{YCL;%cQeSgjCGQ4Ggvb@R~Kv|Bmzzo_>`OA+g67g|MMxO(#Uos6Lt9K`~?p( zUJeJ@NQX1RdbmgQf7=5Wgg3+}X~05f$8-*^MmsbE*urVBb}z9C_saaO-^%#d{wA=( zX?+@cp0)!p{P_NH;K-mHcMLLJ46h-7Z&)I0fB7rr&NDWIHC*oRILDE=?! z!bg*EPJpfVMhWE5)tBSV4OwVl26++tx0V=Anx6!StEzDbe+G!Wtf{3@m2WFvATEL= zh&q%*nl>vev?*nQEU?nJLKU-PXu#iGAPFQ2-s`Kf-Ekk4Fr`><|Rb@_dE1#jzLed#(! z_nWZ6Sd8wOfBKzV%V9QpWI3`E#FGlIo}w`t8@ijOaEr5brRS~Qa=TOX9SYMz|MAP# z{zS3AS~I+9L<9OM9vfXvAWlTGSjaoPyYZthjuhWdo>a)9+wz2B1WvR=H|YKdoH;Bl zv`;gbBJ^AsgJXokbB72vJk0P^gJhHQL`o~EpBry+f35QdvL}ezpQ)a@ebYNK+88p) zyJMmC5%ifsoUT%{d2oM8{j%5}8=-yhP*vs)uHRxFvY`;He?^9;cu2$X{lITu)5&Kc zNLWF=2{Hc2xI`SWQj)$!;8!`!&1hM)1n~_+D>4l(SyHGXsb$?d!o&F#YE+iyaw)QN z$B&;oigKC5Y0sWHbINjcij);?Rz;CIDf227B)oRZNEKQ_aG@GYv_8W1`xkIv!BMY~ ze^N74pbsl*NL_O}7_hyvXVYR$d)d!h$eTHL_WT)iXwjpeQdML*QZAD4uKf{+@C~WH zq-oc-eH(Xf-Me{*Y*#3zLYbDXEi(?wBB?(RjT$xhJh9uydvksnUAUEHe)oy*`KM5Jp~9gXfBj=|*)_-N;E^vc+8%7m>ZFrgu(3%hu)-2h z5~%f-JB zZM+f39Ca)yzy{TlWJ2Faxof6Bg4_|wB$Zr}$sCutOFDs;xTYkz^wZC{au%7jqcR!4!n%qq@^7k+qrN|g#JiLoon1wQ;qA&q|YFQmPk{n{Mt#e z$>_C(wt7xiUgb0mboyve?L6EmDpm9J=R8e&Uo_4&!GE1K<=?bmBZ0My&6 ztF9C4thH7M-mST{`XGdix=x}$Dl$&LcGS^XNr5MWkX8O=W_>D8QWj2ZCJh@qnQh$m zDJ(v`i+BU$4(%a}MYOgmfAGMQ!V+ulz9pZW*u!Dvs#1`yvBcuRWuqMQ&_(a?8szZH zn6`buf~V+?T<@kErRhq}Zi;NDsE~7|Zf;Dleg7SJt*xG}If!n*$k2}X8$C&Q0xava zU0KaMDtPF!OU%V8LY;Zg_E~1(q43g+qYkN3b=O?nioE;w-Cr`-f3zg&btInOAZSyF z-`}7A{%>aON_wK5wQN^Aq}7n*5Qsq(qRrY7 zKqDSePj6D<5}C+Ee^R_5YfEs`M)cDRz76qH40<6}Sk<_{*aLhc5}aOO(T%lX(Ij=V z#MHv(upjp3d9eCX4AEFeI|fD>1$!Auu2Be!`JfP;y5k@TS;)}s!!_s;j?fH=ERjV* zQdoHr^`g=ayB$VwM>)|o*f+xp{UlRGuma(h!9CrCgk*-BT62 zxx~cgjIY$?E_s=xn0X0bMNoq&dzq-PDDxsimP;w!>Zsan->+Jh&Ku_}{{syIJTv{8b&f7C~%d7o0M1E4Ci5-gSzO0fma zowz}zu3SXNt=^Swu|!KQW>rh^;T5of73^UB(z5i~Bdsin7R78DoW=ZsBg^Ay5wD5O z>8K&6I~~n2CHhq_5*D-olT2DVF}ET4ApW$eRqbl=YC{+Ob2Ij!MKq-|C^*5xDF+2i zigY`ef5+gq9m@fxpoHb2eQYcjnCT}aS=-zujn-v|g{m5yTV3m3x0tTo6>K--jmi?m z2ZK_|H6#le+#0bzp2@A1D8@!bj&egb3Xra3vRRm6gdFC2LS%gw)bu+B#e~De(dXN%w(nbbe)l&&v;ty>s5|)vb ziCNrY7lTf9P2E)v)my~&ev505B?UgE@}wxKr?)F<2Q8M9QmR?FG350}kZY4Fe)_>c z34ZaEH)GDM@RiD0*7BBZ6=Qz#Dm4D^kOw>#I^6Woi^nUu29&t9g!NRC$h3_T2^n*! ze|s$1M%d`Atfb3jKZnL1bj?a>|6FK8AG$ZlJ@991+zqxGd4#p4SCZ4FHs`rSk;6&R zK7iYiD|JIT%#mQEMfupsGWj-R=`0SzifC6$IJxI!?yG5CYg<3%m~vDESVbe3A|fX~ z?hTW48Zsy~l#R`Ewn9ZH0-xNe!DmzZe_EK9pq;ZRPe)|w*d0EGuv>o{Du>f!MZsNe zbD!HEsO%Uxr7RWJ0dRKE&QzehHl|TiEl`6ptD$h)ZrVsDnabpLloN~YatEkjiXgYa z8Q$y zR{r$&V2UU^i)<-d_a=6Avc;W#p{*Qo)zR}L%1lm;w zE1kAZ;Vsn1(^j;C<48d0y8{;uG;60DT)Ssi``SSZb(a-r<^Uo^rtxKoxDQ?1TkL0) z+3s@J_8stnw`xRV#p@f*DboXaf5RJ%W8^7c`HQBiV2e9?hphL}ZvpvO zafF-t?rwQQPu%pWSG_I?uS}zxJvU*r4yRDPqX4B+Mr^U`>UrOL-~T>zt)HVgzTRuu zDfw0#%X_R+!)V|?ANtXk{ywpCB~eP1o_}sHJeH^5eeZuC{9reWH?NO`$~%NsbN+0ymHYJ23omg`Wnl0*fzVj_Yv{P`M%m^*oRTThIkp&vLwJ zE2gK<*v16IuCpeJ^j?q$e|yje1&`?J>zgQr1Z(Ad0EX$sN;>9+UAC?Xe6a68?rE z4@QzC-QXEO5+(j^Z9#<34LK4T7jhdLG9?4>9va~SG#~_We=-C#Kon-uCu=Pigs?Jb zQVt8ZVg;cR0&7wO1fd?Rk}FS-5EZT$yE6K)G7mzI zGwJ~?(^4(hk}ccPE#DF@<5Di?k}m7gF7FaA^HMMOk}vzxFaHuS15+>ulQ0X@Fb@+k z6H_r4lQA2UEHLk(9`dNaSPe2A(=sm;Gc!{&HPOqEkD!lRLZ9JHHb=!&5xRQ#?hp5kde4dLRk%z!)F{ z8Ym$X;+Lq31SEe{K^K%k8`MD`6hb3ZLMN0$E7U?ER1`8aLobv=JJdrz6huQ*L`RfF zOSD8qGZSDS2VfutQUC^CK`di5%Z`giWz+@F!3}aC1bU!7ZPZ796i5q65Og#Xg49Tl z6iH8L7>FPQ8bKbo$sNYQ9g>tvtJF$);}wFSN#B7q+<|`?dI3zqbQ;3)O3Tzt&D1RY zK@ead1bT8b#-SH!oH8`rAx`IXPMLuo@>EgX z4l~kvX9HIeKdx0BDRb`B! z5ty!B;~SOp`%ZBSs$H;T)cnHsI7&>9iNdK}r$C zPKQ-mYmFazby<4>AEwog;8aep^-!5%8m7Tn$$=i|VJo#2T^)`bexX}aRTrWmU9pHz zz13IiwOzxM8Nd}>r-2)|VIK4$9^xTh2bRm?fgOLsAsmw7QK#WkmsL&~HebUvT%FZf z-N9nJkzhe(7&O6Kd%+Os)LujOUVq^i{+IzA9yJ=0K^qX39pZr>G8Sg7D;~mu8ni(h z9+hNCwqth|U-Q*x_qAf9HD+~$7(Vu1ft3qbKnj-DIwc_#LZKGsRAlWnXGs=PalvL+ z7GZzGfgM^lVEN%?hgNJwDqs&b9BdXBa&~I#m0s7@ZGW|ABQ|K$bZl$o8qh!!;5Jls z!4Ozr1O9dcSU?73zzobl|8~u3pEeXgc5O-a6CU>ya6xjjwq~h;X15k$v%zb#wQf6i zkm4a5Xtrjx0dm>(SK(F@Qa5!Aby+pR5MF;5b`3#wSC?_$Hg3grZatSt`alW{L1aA^ z3A&&QkiY~0_iqQc3BVu>mNyLIVE*iIF%c4XapUxDa~5(ZVH>h{8%BY9x%U{T;TXP` z8Z4Jpbz?Bc!hs= ze`P=l40vg!zzy60c2&1_L-t!IR!W67eIer-_O}dX*AS$D1<*juh(Umf*LVY029lQw zs9*|G_=Krod7n3W6?bph_JMuXaVKFCSYZ@cAr*Ew7I-*(#dmyJc5^wmf{Qp}*7q1v zffXjfbQ!l7T-Obx00~&YZ%u#&g13JU&_EAZ*HCpq6VN~!&Vd+U0Tu$d3jiSs(%1y7 zn2P`QiuqRx%AkQcmU^f5P;vg(9gfS0d!reqm<75(imA8{xQ7^qcZ!P_a7}=4!GMw1 zU=2<w3f@3<`XoI}eg>;V?gV1K)ym`_5C zLzo2+`EN}C3D5uy%wQ1yc@Tf%z>y(&k|lWxs2~mo;dysWai6w}L$;GS8B87b6F}Lb zdAO8KnUqI4l~q}Y*&!eP@*y7ZVWh!!H1GkJPZ^cN;gxGSW^Z|#V_}JZA!>j1mnESN z%s>W=_lo727)rt!&_H$#b%Q^JpbxrvuJQDLb;n;)L( z8paq6WPpmhIkxv%{tt#>9Ky9>mzkU=HjgRTi;Yp8c zC+51YA9;mSSiMzvg{fc-24NHQFsyAgu&;G~f8i4jTZd11vHO49zZJWO!#A?I)?h_C zU@iNyQGy*}VV6NwheLa*_905eRgX#A-1s3LdK(o|VG>TaTO}b5Rv5PP83|y4wow8e zhyk7l7Y&@@DdOQ67W))X;TYIKB%H?^XrZ`|d#XE_7|3fMn)|sQI84PgkLkAZ_JND{ z7FV~dko|a#dAffKV8I!VbtLuy8{|5m!@#cF8J+c>tnfEdDf$?QSMHJNQG z+ypDH(64n={S1Wn7rEVd(}|(VPaGto1sKdbpat5?(_6hKc?#Abp~<|=r7;)K{AV3D zhQl0GHbIu=I3zO}(}C!3X9xun6C-Crrlm%LDKovILP)~(9b zf3}>nJN17WJ;rC8!v(utDV@DXT%Y>@*li;mGFqu?93^DS)OS5_(SXY3!7Rag$ZJ=u zja+r~J-_~FRCN?YLH40sN1h%eJW4wjyEop?oFUh}o5Xwl9O~hq)WI8?0T_mzu8m#2 zQ5e~i{h%G0um5=yHo?rH(OD&SzM*@~!BnCNyJ~+a+JIf#cC=92&NqtIA82{?Hvznjf7NKA~HEHO=o}4G#K*F+GvH zz{SfW!(|+_RiYi7;hjfZ%KJba&NUZm?8*&Z1`ER&;qn~?mxO+v}`mZ1RvA?6YpZmKX&%GY}!M}Xbos~Od zA8MVx$6?Wj==}W`?Ul+NKtE37{{Pq^v3-9#6{`MWH=(_!;I;7?AdnFAN3fv5g9sBU zT)2>3EL0Ft9mB`4AUt)n(6HFRv7-qVXa*wOM{?gdX1#Rj>cz69OJ?KtRh&t)rp=o; zbL!m5)1W(dk;e1!i0&AV3G83Ca-@U z&XOeCHB+T(P@FYusK}i=Wv*SjQ}X6r6KBw-O~8P)8w?MgyL3RO2`#$RsLIBWBTJr4 z{<(5vm6$VY#(Y%Qug{=Eiyr-}+xBwn-fa)9@2FUV z3um~GSyQLLhbv#soZ+~})0#`4PQ8D+^{ecTp*n>s)}*7;n#HL**UUzaEc%?gZc}^2 zRP1WXY)4B5i}PU9pEJ9V-bt6owX{-d?&QYafe2PLn?LsWvky><3CLMWl5LimWlT*7 zAWPE}nBa#XCRmR=#PoqgS73F-LKl4yBh60x6eeRk*fc|nDw_=A)?0Mxwhp=b#4dY3QMd?&(e_ZbC!oqmV{=U3S}@ z5(_IEPJ_)RRMa)(AZUEWkwAZv8Wc_}+*umVP4M`0O-A>nxFRghRMgvIC|yWlQ!tGK zRjI&YMbb~N4TX|~k97uSOQAIDY_pO?D{Uf*RBNpwcf?`G9krxsR49o=>ugFFa@K2d z!KSM&LG_&DOo=C^h#nGR5hIUOB*|D zsq@Y|Kj)^eJO?eb&^Q-;v#~WNt#r~dH{BA+iusD}x>Q%~XLb*9$4Y5v_Tnoradb-X zAkB0k>IP%2o82kg#X^5gP8VGR3mP)WibNN%v`SoWiRG&6OQ}g6;%rk}J<@|gsrF`M z%l1Oll;TX2OConvo^8VohoN~GJ?2&L6LrCbc^r~>+qEK{K0$@%aopjDB#i#e5=?~~ zZu^JwoFfgq@_uxmShZdiqrU14Mq`sU;;18B1cQF?j|oQ~{lb4?C}KLAs4uKNT$^|A z{rBL9FTVJ3jBoz==y%_`_onLs^N!v{6C1Ce9sBu3tFMtx+( z0$;d`J@{xa#Y^3xiNy&tTp>!IP1m`Tt zIrD+R41q9&A#CA^QJ8WRtVpFQRXK@Peu6ZgF(6V}M#~{Wrk1$O{-rM4M8h;z2!LKf zielW+#$poPe}FB6alnQ&@WTRo8IB(#so0uL3MhlO1biA`eYj-enc zStC6X8fFf3BM8nq>$tGk0Fx;3V0tn-*4rHVYH;S5;xp$qW>i&t96A{X6*J$O_{ z@w|aNKK79<5zEoTM%OUyQ6Ua!+)OCKM$qjU^{9VHO{!9t%G9Ph^{G&0YV_`a&J86J zil57BRtLH{FnHArX3&Em1hI&+rUHU&i-=mg6N|RGwXLa$tNvW&n#xn!l@+bb>o)ef z4P1QUub_DoIu<9(!W!0bOQYGG7CR?#I2Ik~n1(NWVF*g-;j&W5tY$aM+0J_QvY^$2 zCaQl5AWdAh3{ohAC^eyp{AuPTK_N*B0Yd8Z9NT^$WfTy={4l5{(u@F9^_LGuSS10Wx?Mk9EoqG)HPQ@~o#%%et z)X)iNdfLZJ5u%nQy2Xh;0v$oa&=-R?1EGKFD2z_pF^KAgq87bq)*aTciSHT&9%@<3 zAjqh?Fx=scG@RWy;z`>MTSa%oFeyrV_d6ER3Q{sE-eCS{+N2tV(WbN@V;s(i;ASZz zk#SH(V0O&o9{c#mKn}8yhfL%m8~Mmc7P2a4EM8q?YhqB&X^}AJM6FtRE}HY|C((bP zhdclR5Y(W?17mTFGN1LW5Yc9(*v4iyuQeiXjq_XOTIOSfIULF0200G%lPC4eXn=-I zWCBg-Lc@1xK}!^(c*7gCI7k`3&_f?iO0WA^+R|P*n~G$>X(XUw(xC1GEJh6rVmRZ} z?=r;oQWA<5!9vm~Z6Ojet6D=mw7q|d;jo8yeYFQxiWRTyT1@=Wj6r0OZ15lke!5^s z1%@LP+B`-)rlO8_(amlwdZA9p`I@5K6Q4wRuhi!LI}>-FiI^}d37*ocIKr+sn7EEw zP`XiEA}6mV6_p@*<(Zgpl5MLTE$kAC534Fy}B9nOEKD+2NQ z;!+tYF-pW063bfDFS_0EHle!JX{&r1Eq1ZRfRQ9W?kIK6b6#HTvIhP_Ocl+5Mm+Mtj;dQd99YkKn73Z%W5o3#X>POF z$NuIy*O?mOV8^tD6U3yD7vp~kc@-ME5bI0JOIDpm1F1{B@l&7U?=qF?&t)VE`Z*>P zdVohg)B)mA_u)LhyPFgO5!5jS?xS$ed{soK{>(|}Ru>&2$TOE7Tz|j69>&8f0FWlQ!r!Fq6p&=2Mfm&7eOQbawE#qFnA$I1l1RQ0d?5m zUnKrW8467a`Hg>esZhf*gZ zNAl7YEWl}K@M)t)4A*cC*gzHPpa-#|8UCh)dj)Y0A=Yu5h6ZM!2(aT?k702H*o9L; zDQT4opU^d4(;)ERTY3=(z#tQkhaO$ff5pUl5it>0VIy?ogLi*J4C-JN^F&YPriHvz zK)AFN;TC+|WpCUB4`SpN0=0B<(HH-ggwzoZmBLnQ1|K?s4FCgxKcxtM(-a=%g`$`c z^6)P8XMaC7a)wtWBx-nrqsSca01OZ}1)D@LYRG|9Q5Y^p3pWxlO=NvZ z7dj5(U(LXMJTZTDS66~!R%dP%5i6KhY?TTwh;~)Cb~o`H{lgOb(@Y*Fd+#zgbk~D- zw*i($FGDD5e0L2EC>+k92cl#Mnh^d9qNE2VH0sPR*-7+D+yVP?h;~@=7aO1 ziu(tLk#dWh08o4ZM!uMo*FlwGBRXJ2i&9ZM5omM+13F5#U;g%Q@i-Oy0D@OXf+kpI z(^!pd(^h}kh<5R?T*Sc*xF8zazzwu@4NN(NmFAS_!BXwmcSx9(!{H33Knax42#^U2 z*I*N+2n!?D5dgV<+meO4GE7?jnUMToTa^fGT>}&G1PsPTFbwuW@?eo4#2+E0krI&) z!C?=x7hTgeY5Pc$=M@{PQ8NFdB_*>OH-jf7Q5%1Elsr`#7yMQi{0b%X%cLQn@>aZ2mBm-d3DeI?E#&IEmBa`)+d05ks+b}_)D0*R& z7je*Y{_vXT5t|-1p>uel{#Kz5%`k-0g?}w1lI+1Jy73^yQYhFU84)}I0OJY{-N+?StHVW1>k~_ zh=L2+keRxZ*lB*jq7L5RT%-D=EF^zx`AQ2Esd@JR3n{e$NFb2T2|Df?3ddqDzJjIz z%RuaqO>Hv@zwnTuM-c6x3dbgirU!X_0Cx_-X4DX&!m$q-fsQItFVZlQelmQs#0)O0 zF6vMtH!@)abEgFbb&#qYlIm+>r&inn3ZOs-WD5!n%BBy?9hr)*pPCS=`CETaz+kzc zr#JhuUbvK}cpvxol=)g}&cLt!svOX;afDl_c6+GxHGu@%i29Yaj=Hr{@vY!0mzCP7 zZMsd7LXM_5cS!#GgR=H%8+Q%bCW_wRkFIKkR;zvzl@8`)E*-+BhZ`cfdRyEp^xvUHokw8MoSeE_W<<`91*G_=xThxQ>WE#(Z5%N&5h8JHoC>(X%i$%t1=rg3m2 z%lIMuV4Ys;W^Cnx4?zb}kZC3m1#@7w#{icNTd6A8Hg;BZZTnzL22H@wx+;OYu`7zX zGlf0q6?f-5QQ9J>1`X#RyggA5&cGEP@eygWzss?deP_6a1B}$SxCei;t&r%b>JYWU zQJs@IjhEY?M2jF6duk=Zp@ylBzcYlya1I?@p9@S)uu#7f2^7czkq8XIk+P5^)E(N8 zKNOT4*sz-ZL!t^C_Peh7v++l~-0_pkX$JNIl1M-+6nq{3@e;6W9aAe|V@e);p|xAQ zc`=Jta2CG?aSU^?w;O*D1#|$#1<}1ROATiQw!2CVYL^f;$F5Yc3D~ehTPUwQ%)ju` zgL=!x^P;bRD-GVTAF_!PA)AiIHON;1N7U2@A$LrgSe0S0xCQGYYv30fgbjqDP|cC6 zbj+r)AhxUwcBg>H={g*EqnXstM++%xsE>Cxec*o$HVj;6>N64&uO3Xv zA;NlU77Nj^CDdvVMGVFhRS?oZTpZB_UFgKIK(W+83sT&nDiTio`pE2&7{gX7!vld; zX_MO;{>r82C;X5PG7D#9CZ`9XZ0Sg{;=~Xn=&)gj3fh^M>S}CaBY^GXTIsCJqu7ro zQkVd|&mUKW``UlV{VEUgu*1$^m|uYey~)rxq0g=w7X1uTOiF!#^vMSEQ`x5^%NStY zHyqbV2Y|q78^CFRKnH0i5wJYJQ!zX;kul%pTZa2F>cMRvfqpIV05z zan1vv{RIro+Z_B5vy})DWEKwi5F@u?8}iTv`+Si%F`8b7wrRD#@i4z<$j6(|4pzMq z8~&En2)UFRN-v1Ko{>j)lU9U(`*#<84tL3pnn-PTBxkioD`U;OAeGeAN&;GC} zWCWwg<`3m0A!?M&tj!#2lPalz3lp?P1aXVqJq3fnv;J@nW|PgaO@Je$*7#f<_FyAY z?4kAI4f3$pvn3-ok|UgR*8tkm@?9PGP!AA}(Gq{2I}VgwtgM~caD;@r;e?yrlI$Y6 zgV8%|5C`gO0qr*I5TQjZPy%QV&r2a+UET?qJLqY;s*QiZ=5q)qkD;^|$tT!&c zCA%f#)PY=TMhyTGIrD*v6!EAql4PFpjwL}S_` z*}IHW&Txhu;S&BXrWk5>bB*jmo)L!k?E^G9X?b^QW+@9^B_U+&v?&3b~`|y8k zDcH^O5O|EILPJP5MNRDj{1GyM2DH%N&2A9&aOod(FYZkjYtRf)VGqVkE|4Db!7-uD z9KT3}3$6;4&J(ac z;SzjO_S!+#zV%$c^-=o58C?-+4Yq%sP7vaOFoV!4Ef0Eq4)Y<^Dy^cxOqun%-t~5W z_jsT8dcXI4-}e%&dp@T1Evg{wf!b4_06$ zjGC_zs?7p?cT!&l2t3$>$!quP=xH7=q&4UE3qhnE(FNLeLGMik@ysATlahZB+4ih| zDr$BN?xYLxiY{lEL7RXsuplbcq?JfoLZx0E&EVDOXA|WOaUb8_@s8E>_2iU%W`^NRsrK{ImX5+Tr#hX_z!g$9}{XKsbOWRzyy{1(y z*lrWrDO6M$?23{_*u9l4vx{}=FEz^$_rbH*j20R*GHA9$(xne9V)rce@hTH+-geD^ zZ5k9O&RO1!pG;XZXcI7Ohzp)O_c0_%lSrIDhiMe02^JG}|DR0!l2p)LYf-p)cvm)lg4Q7AyXPi3KoJjsb@-x!f^sAb2)u47jxlvT(XS>QOV$gD}&KKSA4qM;&jj zp~Z|QOq=PY)cPRi9Ef0}={k9?@}?P8aEoINQG1)BR8!*^4jX5#aYnq7+a4Tw zC+b*8%_NeL{E&Y`XuN4B8^8WSC6;hBny{XGmSxtpgy6a7q?%ah!yk;~k>{KZTkXS2 zm2Sn-Ab#Ru^A!Dx<;)*!HW`o#FtePdutVpix86YO>EovUYO$gaTYvurIADK~Be-CL z4@Nj)g%?)X8;2i;IAVz>cKDnsH{fw5usY z!OEMyIY+4rqz}3$TWE;sHsk49$`B!Z<|a0kaQsoM{!1zbX-6Og<7U2j=N;-3W8F9C zqUz_bzw{=`vTC+_@4vS;TZS2y1{(9vM?Zb_*N1<`LlNJHKmPZnx!5Kd;8)U{bJ#g1 zKU9hl9`*zXA!$f!1BfGtV%8wF0?jT{+;g?nCyD@uV- zAn~9^DqN_aV}!#Vs)$;t)FKcFuF#KvoKpS1)&2!%01}YVn@JboX2=Ty(I#kc2)X=X zu3{adZj8hvMWlhL1MW*8>HtRCO7$gd6y$#%5F}HXw3bN$e$9o&7{xEf)(d9110fbgOT?g9V@pa-T^S%93PZpen*tr(pKepARu7tdM^| zDeL|7dCoAPO1e-+gF5dThEoX=3d#^kn#-R2XlO(PqEWT>jdNFM#_nQb>=QvKj29u*K`KUkCp^62LQlAE~$2i0$L0a0Suu@`& z9K{*S4)(&92q8-@4-!2S==8D&e!>wOn8w^)$Vko&L)Hir6IWjfM-eiRPbQ&}at(bW!H{>-ocEa}H~B3Cj474JaE zo7Q3+Ll2ETo`PjOV;a|(Uhud^d9G1ci%4>?b+obbWNSnl*1Qolz3_uyMO@)Gs9$VTMvw+Sk7Z zcCaNP+&ASz*lHRwkDO+Z&^mU1x(9K`WD|7dWBUV+gH6RYIy(-27R1x0q8dVQx@<$I z78MRQXgkobYRSleZ3^xwR#)3kZ_N2C>LkdN=a>pJP@&rY26({ztnV_TTi^`hWVk}O z2^eppzzCTynJUZXKlijytoTJOrVZeOe0$5!=99F>2<&nN#^C;JhpLf(h=zCH7kH^S zIV)#PBgEo+AeU~b9uB^9p7*@4A;vT{zUXtyB;2&0WGuLW(d~gmR@!o*wxQ8V$y9h^ zmWH9YKm>dih~{Y-{~ezd?iZFlZ&w+dfxTE zcSeS@%tPndO#dEbuIDO$aP}DP}_xlNuP^ z={_2N+Hi>TO@L47Cad!m`V=G_ay&)gWj}k`D{YvnGkfi=(ghw75+Uu_jnq^D@`KnT zULDj0rK9}vPt<}iFlC56fML8rD5QSm4t6}+^p|<-UQnf{<7o?jxcb;RBkb1hpZC?j ze)d~NxD9SUMb2TRg(5^93^&y#umNeS9iOuQWjgMfI>_qWE56EWj2KG@gz_kW zD?Jy2q4<-M{4%{`ur64jIgh&jHRhtg4+OywWQfdilb;g7g^&mOQz3)choqw^U1A7S z@-I4Zyj19-$RiAYl!`tDygY^YIuqyufg8J15V{lW3=1q26Qr&o^A@n+JR^j{DD=L* zAQn_mLMbE&Z*ZF@^t7|k6aFKJCpkXJQ$Fk3B5_KEPk27V2*3cdJa@@DScnZI=)h`& zwJXFztQfn^P_TN-7$8isJQT!1BsvQu77QfBf$%GgNvy1OT??7hI7Hg>vD#7D!fyK#aOJgm&-jtltal-w1ij; zFXXa^h_+cG8ye)RGYkw?{&*?qa|n3glEx?y6ugj;ggC?fSViuF2Tno> zGOEUJ1jjyqGdK7<#c|6;gmA&Mc({fzDY<~Un1ZX!Dn?B!40vn^Wqhy4Scp;U0yh#k z&qKX%JcvCslqhq+uUL?Cc!p;X$cA)C<{3w7jKW*YDTUaFW(b1?T$_lvnlj0n(PN}z zC~4d&rvun{1SSF?pcK4XBKF$b%e!fuBqPH~5By z5Cs^>fulsqTbRkEWJ;|#HDZxTK^(PTvPj@tKz9K{gorGO%RvOpB<&-~UnrAcjELev zi)LI1Q9Q?nX zL4*d8Yea)kMi?V0&u|M#^T@jJN&(`BaL@*SPoT#qv&V-BO%mFZeTsw2iVxV`S2)B5;?Bq6}y37Q5$BO7eJzB?v&_Z0KJ-1uUa7RsWIi=G&wbMJr z(>&GFJ>}Cr_0vBE)Ib%~K_yf>H5x8|vO;`NheBo4Ms?IbRWFnPi+XrddBCOU!iGDY zDtSg^hf#G0Q*}58_N0b)SX5$V)@F6qG&LH`5GQK*hGl)$Yc13Uc?KvR2!SLR z2Ec5D3kU%iAc8WWg!8kCZ^#5uVApna*LRmnjsq!we_e$9h=pT#g@u*aiKW;x(1Uw5g@hA?ie=iSP1sDZLGyCjsioSgmDg8*Jyci(tL0jz zWe{jA;DQjy0a2)`gBXT9$OA4&0sgV2g#E;dbGU_JklVSX+q$*eyT#kQ)!V)0mmZG; z9e;&f!!5Ztc?E2U+{?w>%vD@t7zP7Ef>+>Nz2#gjqk=HlgkcC>!sT3Jn1)$(GQu!f zO_A`shewbA!g%*mb7JbcWQ zP|%JOO{~-kCZj|rqb!vb)lsFmeUQ90oD+*wRi33RxcG%h7zJuLQI1T><01Y&g+Q>f ztvxb(lPe`eei#~tQ76T}QgOJukbl*h z_1|e**;Qp#cM!dC@W?fU3$3vS(Q^k?^;DHbS;D~3NnnLkNCg<4h%NC#ci}OHfQ6J$ zB=%#>6+J=alt76DL1=&i4Il!gl!#$q-$p>!7KUSil;04B!f~5V>BOT_`A}fU${=0U zPu1Za&Ow!(EFylX)+J;)*N2r(@t?DU3k68N^|LfqxJv17m=O zg?t9MxrbYDgFHCQjdI&}a03`HOj)L9IW9#r6T~m$;wL)iimN`<@GZ+r*1!a9;;b;K^tTd{UHv1)f$2qU{p>3uwJyh9Nd!PCP4WsHdjZiY-}#0e>N|h=?BS`#?_C zQ4rMX_DsZgFz3rY=Ydej%od1epoB&c1WNczxF&5_lxsq?V~HrwE&)+AaR?s*TIXOvzsBPmu7x=^q0C~gega}c*nW&$>IjtqEVQHPiSN)hO^&c&%a281Jw#|fPbznYtr-uv(t>i2v?k+XY)3nu_1@bK@`yiR|pX2jZJV6b~u+Rj^a%fpHD^D0Krjzq2z7|&29+*o z-*ZwYLVL&ZP(cfzwg`$u0=LGbZ7B0HuZDGJC72`@fG2T$5P6aJhjkDJbVvDZpoa@; z_x67GQ-7R?lSlcDZ@+k-_jwnFYS4#Dk4unOzs3gcdrUbP8!77E3O$$hDm?ayxWC5{ zK5St1f>7{aig{N^_ec+>cmNwZ)`xlU`mbLHZ4mqZu^0PoF#EGddyLQcwP*XbcYCWx z`E+;twy%b?H+!vYDj#P*9O}gd3+!TeQ^m)(*>F4|P+lJGR`_%Wom?wYW7Y2t{ZbL)`n3Kvmr1OSo2lx67%b(^R zBkWR?cGYBmhX4qBzy__EVld36X8sm3Z0PVI{=<5GOl&aG)BHn(4ek{5-mDZmaA5%madxR^eNP+ zQm0a_O0^``tXgGR?dr9p(V;<^#J%$hgb?Ce=s$)_cSc`j}GH0so0^LD*l6jsUr-Fg>)p&oG-sUPnbws&9Y5Qr|ooL zPChAE%~(K%6_#!$c|}!vO+}?ugMVW!xFLca>SS6qibY78hdOP@;8-FmHlb8lg%y;7 ztdxfeE84`Q4?X-vhL};+%%dZaK?1i8SFW&eq$?-!CL~}ix#SXTsg=0Ri3?_UWrGhI z1=f*lh&Ym#VTw5>nPrZNC7NkASS6bZPKlsiN$NI>Flf1^kCQI>#U!77`hR&BJI0xJ z410eXdT2)8bRnHZ@;r839Pk;mhIii3C0;6UofnHz>801okAm6L%q9hCx5jscIfI0B zEWiR;o=-lBnVnIpHl>1~MTnGx5AOcDYk(56DB7GauGteSOP$gRD`u5e>t?dbS{OdM z>bB%Y^hgVnOf+?7qE9Ivm48)NC3T8QE3Rl*&4+xYl@g*15lL>H@!D&sHbZr}uezZI z#t%LL<3mrZExEQ1!U=o(CQm1#875c5N<1-CBe6k<7a3a4ykb%^aK@gsG^KMFUOXd!gcetXhneN zHRaw6w;8UQiHR98i(nebFG&>sn~c<%$GTY}U!83>Fu{VNLe0}N6B#U4!t4p$WSGr# z(@uc*N)$=wwnF$+jDN$}lUWJN^fEbO=NQ~NjPL8EEW+G8H1MDc)8&dtb-4|`2IJR@KR)^8m%sS*9-Az(i$*yP`-jc*h47tX z2X7ZXJU*qaHlN?$kj}Wqv;RFrAKOS0R9rzI0@8&Ttl9tzh<_0ocmQJ)k$S=tn6?RE zV1pdVn+H2EWgf0kEfsvJhDc_iwN`k|LJHK2X`aU}T=>F*Q^4R9z+i|^SSfIV(~=V? zvH>G}30TxS;#XX>yL}l>{!3V@(wm;;vbe6B`-}xg_^pg(vYVX;Z+{28{Bb2TC;1~I8N!ZC2{0bf z^ankv(M)Gn!;06;2t4kSpZz$bVB~lWD-i}mTo~dGRKOt>DAkEwAVwIhNJV)_=(S2( zOBkOp1VBH*F*iz2SpKdc#3$X;V~ zCyY5R?|(J25*Riub0~I{Cl-aWSN;;Sm}-QPfz(h|a^|7V1pP!@Uy>MDocKOscF2q$ zji^XOD!&-w@^23t(>Eit4)Ez_TJJaD z13+^kje!v4Ltmn=MV#Bpy1L5BDaOrq{5fD%Yk&GpmhQ}d_tVC%plPo_;*pAuJ(xD! zKv7yYELRzl(=VOk3QgQ$q4Ck-BxE!-bgd#ZVETmGv@?Y;e4>cEQWqf%5eC(AClG|d zXfVqsj0c*45&Zt#D9S*&8bhiCACt(!T}{FUT&T4pZduAPtcf#n^i8k?42QB@>Je_4 zk$)$b-4Z6-7%{AfhId;r60yje+z0`aB+VsEFkG7zFNqJhKq15=@~huTcmfyx{jV*8 zA>aWU!x`5&FoF}TU}rG6!3`b;gd;5B2~)Vj7QQfsDQt{eFk-zVnS>~ek>L?z_!-bR zF^W^HVrYEh9WH(`jAJZg7|Wx^HokF<>wj^N9rL)yLc#|M=cL%X5KLg_93{9+m!+a$ zRag@$N`qo|mb$^lu~vTBAnBnGYi*?(gZwg>!z|`8e|aC_n9(r?M&>oMxy@qZGKS_s zuq)HK&SW(Hq)uJ=EXsQKyG-_?6Qr{NC#dgNnoTGe7K*_qfI$z7SVSsPv5Jm?<$n`5 zqynZfok2}-k}Q+>1ZQ!$oiN0q4!BU0aP78^F5qVad2pAV9}7-+(7M*PzV)r;Q4bv7 zc-Op^$2-&^>>1Bk9>bV~hq+=4Z!p^%!ftl6)q!hT|F1a5p@z)fYcF=B=cR!mWWspb zQ$L%JuA79Ae4Bjk>y1U*g2~rw`+vAQLeQJu8oM{X^R4fF^Sj^v{x`q_E^vUSyCiHF zIKnmNiG?$~;SPT|#25bGiBr7d6<3WcGp_MOx>Z1=*?27RJUZqjIeG=fYOsJBRF1OA zLm_%C{w`xK^O>LVT*sg}&U3C4>#^%^KA)>%d_}#g20_qYQ1{}V5?d^Q0e{ms#9$cA zV2C|1f@7$oQyAEm=}fZ?qHvt#HgKVdPL%5D@d-m6H1UbONQvfxXf+f(q4SkRyzX|l zyEbrf_lWy8*|Q*oCmJDmTiAjUhCh5JUSW#Gi{ci>5b-d)0S#y~TiDXJN9Oo6O}NBD zi|$_OTb2?I9oYjP);K*kl7I2vzU{Z(uCI5Ox`((38A=dUMj!&&VN4i#mD^UQ~%_~zdq|kQ@!wBwEN{+#izV4{`3xi{KUt; zy8c-qL=zAK{SneyEK&goGh}*gT93mW>afE}5JD4L-#XU?f?@~*LVp=TAlyNsF;sj| z0T^fk$C%$NOcbM7)DgMg;+)U)5ycX{-o6n*4?sZ~a6uU~f*>@)E8M~@Odhlen?2aW zJv0YPnMW*)-tYBEJj4PCd0ubOnmRaJv!NIY-U1~EULKTz8@K@%aDf*vLdH~`>jeez z2nF$E6n$Ncy=6=ugnt-|wU|8613!GBz$jA<_6cHHpc$Sa8m6Hdt|1$?p&Py-9LAv> z&LJIM75JgY>eQhg?jafy!uk~)_Gyi$(8wor*4C*OA_5=)j@r`D*4BlAA+%qzS)l$a z2+;<#A0K978y=4KIDr?a04QpJ2K2xO9$_Ph!YdfVH+;i6oPPrgwqWM$Lt)$$*QA1U zjUkZ4Ln@HaG1x`sJPkbTnv1n!D?-C9h(aSAArS-t5rE?Mm=cm`;<+817jS_fm;yAo zSUuDlF8;|n03w(GSM`k}IhJDvmLoc*qdI~kL9L@ZzT@+qBRoA~?zZB&}gy zJ*dMwc*8fC7)zQ$CJ2HXltB`J;s-z@eN~0~X-O{`T7P+YViua%H^{>ab|aIJ1Dv_Y zkadKcl_i;p99pKOTBc=N?Mj`jC0xd(@+1RY)}>w2CE?|zUhXAd>ZK?2CE4;THF4R!EAZ6Mq<{3P3>;h@f>gLME8PFif5ccIDIL zq(cOX4C>&a*uy(GLxN)7G93d6r9!qX1U$T84_c@x2%a7&WggrcRMJ=W>F1n;6j9_= zSG9|Mvg1hc6fX%#Qe6xq*g_`Uf;x1iSL(>ToR@V4DbD1EZj7ao28QJ|sgw2~YaDW2jfRq2KoT!5ec=>QO#T@r>5`sC1irLK?G`2jkooQGf2Uq=D@P* zK|SB88GBETU|gfK_hyjKfr@6ASQXS!7EUk zVBAAJj4a8PtjV4%%BHN!o~%8rtjn70uf=SQ1>2Ka-cXuCczXV4BS?V?a4c4^fd)i? za0)8X0;u*ekPVEhYP@1|0oxY^fCu+c- z!T=EztfGcPF)T&68j308fD@?5!S%p9+>1BdLO75RD0C_)dI2;FF4Bb*S%2UJ!cdNL z%tOJ^i8qnuJRrt4Z4$;9EX4{3U~Q8tU`y@V%Quo&@BGg08p<$a=6rF1r39~z^qBJA z!#VJ5x+$g<1lu`4n01x`5xi*#7%tHcDxgXL43t3%o&%8nY90xl!qeW%%MnY%cB3=Q z?W@p5J#3z`=%xmY>IJ}Q(tnBqG@Qdba3x1H!vZei6aa&v$b;2l?e5a&Xc!iMP=@FL z4ZwnzNnF+8V9Y0iVjFCz8|Z=H-GVc$pgPzC-1f#T9A81i{0zHJdXCVJ0jk z6c9lUqyQ*>L4{nEeijA%paq`rgWdv(0+U3&5Sg|BjZDQZm|%?lN`D*j7REjx10X!w zA^^fOjPMit$?N_QyG(2r{|P+Yf)SXK208&0Eb15ki5`gJVQQck7_KQeM2%S*mq|?JhC2S_FNB{xM*xdYl3%c&!AaK;kySFnohh zj&Dc6gDNznDyXoI$bW-H_5nr&?-mn9V=(ZayyxiHsNk&VBr?vq&;j9IunGhL6hMKR zq5*b_LXR?7D9*9DM_HSS;gBmPN zt2O~C{0VeW)PM0J?-mI2+}|SAg^WZ&qQayZMIT+^CuYn^=D_tXD-5)0 zm~y8m5Q83I;(2+&cSduVuBi-=br1vrDdrpZERpqLb%i8QCGAD;VOfVV&^^DwKx@Gp zkODAD#*raLdWpp(yx7z0Sdzy*jZ9~WxUs)HtXB#vBB^l$>6L{#lUwfu#1hzgjziHHIYI-XLb?1 z!Fc^bi{HgNY}n4eX`qhkaGEp`G=D-i@&_$)&G{NiK$~`H7s80wt|%ur zIlaRz=s^#tKy*({ER2FR7y}?fh9!Ks4}ADOoB4|Ki37FDa?|+lNp%@*L$Z*NM}En!U{ak}ku8r+5(*!i)n4Oz;Td@CqX|!ZR#2XLkdP zgMYY+!?|#f0*LFg4}i9(2ZyGgIf%bOahJMaI5?Vv_>>cdN1K8mltB;ZF#`x^2$Z!R zm_jKp$C6*py}Y=GpE;^0HuAm$a!W@SfHbx`K^uUAro=@lAOkEc1GZ!M=JkR&M&tv;F=$gvj!+)ul ze4`um%K9&>*F~9^ybokN@^19YPCeD5tUqYF(MNbNR=vvtX-7Z9Ei?ibX;uiub^my%B8+3K3dQ;I< zdkg7^o09p|1-=o>m4%EV61{rR7ji@Zp~5i?He_(UJly@y7r*g;JLIQ&w;Mn68^1GH z0}@w^y6_hz5ckN|GpJ|1_Wo}__jiBEPd@mEe`#Aj`Io=>pTFX3Kjjw!`hUB;`@cW@ zvwP0ZKmCJy^WQ)IZ@alWePZW7{?|W1kP*>*zhw!-z0P)Va)0`e>!YXnTJz@J zhJHns3Ki00uz2DW`_Lgt@87?p<%rlLTJYyD7QzE)5${>Edb=XzsY}Gx_wncVM~ZxZ z|A+|-P{08REYQFM5&W#b1Q|5&zXu`gNW2LttkA*>C;Sh?4L6JsLXH{?QN#oP8<9jS zHef*pmRyQ}feU0{VSkD7BrI=3@1g*uI#f!*XcZ*fd?L_?%`7&M=7QBr+;?ZajrQ`n*%E>R^S8f zH$awTvPmZsB1sD!k(vWd@%jUi%2`7!k=6u@&;k%v+7gbHWs5cu|Em+K9rfIZ8ps5;Yhy)g~oz%I*zvKl0%XQg)(v zAXCZGXd<(=Y>Oi`*+ft+IyF*=$%1n>dv;q^WE z2;o_7KjG=D+(L%VTW_z5?%Qv$JPy26i3u-!tf>97BI;C~br$W%A&>m*$SLQr@r)MF zT=R#J1xfMEK`#jjDPXuDf(YD|@ys+!oEL0Fe(<+ITV<~jU<}iXI7orQ>H~=t+dVk7 zQ$SI8%YQDesg%+_iCu;~A4t)!9_3}$shef`(2|8dnzm^niYUC6!Ne+jY{-I$3_L6= zaL%iczdIIO{vr!!9HV@5tPT_~?Rs8BszG%%_>4jevg^;hc7zBmln-~`0o~IK7r~bi zq;ntH*#5^Pn88yqBqEkTh9`vKinq;;gf=6O6@PYCLKUtMKXq$gM+)&M7Ip=GHMF78 z=9fc{S%Q54vVsl;cS9jsk9jj3Rz2*77)uz@ACG#JG4v3G5VYha8kh-oy0JsTJqZxQ z3)|&J_PEJe4vcSr;O-0pmY9i5^WfM7vew#Pzv6ar@&>jn#nfdoAE zfqyGMiJw58CbckbO)#^|nP1E$k_vi`L*}YhoN_j<5Y2&Auw#=YB^17n;9(TJa)m2c zv4{d{%{*n)h^g?PBh?&(X%)Mk-~PzN{M@4u{aKAZ2F5SrCA;>`dQJ!@{Wgrw*pUVo_h%zI>lbW?0m1dKq5VZ6gniP}*r^6`|t z>1~K$nWTg`cEfoBMV#Kag&sn{t_!HZ35r5WEK(5;e!$|6DS`!|O4yBMdgKi$RAC?o zX(*o=GAuy4h_W(-vNhp~YqbMqm_&)YQmEz>3RIRW&FC}`b}*Hy#Da%a4D&;XR+X!N`lempkjt)) z)ihV_;bA0X2^m%siln0=yI|l3Q9z@Jtzids4mwmO@`pUmBr8|sVGK%0K?4Fk7+k>q z7K>xBhAH*vLqk~ri@MJ3s;?4>C4a5~NI`C?59iROJnj+N>>LD0Ap@Uzr1hc7QO-N4 zBAk+zL5yRtl$O`9UOhsaTiw>9HAyVYT`a0Xb^sMALj7y8cqqP(AeE|Ux*FPLYQZX+uQbQ)B5f zQay{t)1U;6#NB`gcMF9Nye1`{a`R^U-hg7c-4!WGS&D!2!x8=$bBO(W44LJ!cUi3fFNd1IcT_bUA@ zp*v@Zk3y5?mdnWVq%oaoO>g>{^;ibSI34O^-a-mfWYLSHa0_+twtqDIphlR*Ea@56 z=-tq`N3Cg{VZgUn|)bwL`UwhVX!^M)fkdn$L`(J^Gr zGacJ{ymL!bjj>H_agUqaOoc}~nvw2yOuO9ecK4>CIAE5n>jFe*#4}9eYW`RsLmy1^ zv`&zH+cl+YjwvM8CR%uqN zkRRbN$}BIQa+R+fKl{MOAmXqFna4pDQ)T(hao$~Zkh&!-dUL8x6OK93Ip>y9haO7T z>s~v74zRWkJF$l&0AMThuG^f!EUGRSS<2e4ITpowNPoD}?7z8$+KP-EkeH!C% zNA0t3D0{eG7t8O(IDSuFbu^;A2fqLe1VLZX-~-I7IX%zDKMnn(zq<7>h7=SJCgKd% z7LFP~t~4mF>MV{}OfD8!;Q?bI7RIbth;7+I59jv5^?zP36heXbdLjLA&c0M577kE9 z$glj$kN1p^XTD6~hEL_@PWe*M`ND67>|ysj&;zHT2J-_RRACALZwi3$?^J;Y^JDy! zF9&JR2=T)mWRUq@Zw1M#9^Szeq#y-OXWl}f3`~KYFeCWDt{6eu8NU;Xhp%q=x z710eC(rp%L5g6h?2!SvR;vfvF4+!Hx7G;qZg@3UbfMMNUu@&Lr4394ZpYZj}ZwX70 zG~9t1Qm_nr!S{UZ9EPD0a)4b9E(Cf&4=Ri+_<`&S@DKeY`I68XXHOZo$r#E21};wm zUZf4^pbhWq7{Z|h@dFyVz)iO2>O_MbSm76R!4M@u5<)>2{`fEs^W(AHMB?CX8bTov zEq_uG&oAy;@StAsKB&>v)Lax?8< zA+>=YOG6&IK=mZjy7nP8RdY32GXv4?HDNP0WfL~Vq5dK-(jp~+8D?`gd9yct^M5yi zGdP74IFkXWMvxkY(>IgPHg8iAYC$)X^EcCh8^Fvu$e}f{lQm@w&~~r<5RxQk5Sp-a z6~#?3b)gm{K@sj?5udQ`-19x*Gd|^WKIc>a=#D<|6F%>g?sB0MI-x%$Arb^M67FCS z3>4;;QY)R3`WSTjaPdJQG(sVi3V*_2DsR&c!azbbbVG%(K^as*WiBb>;6Mjq6I9_u zfgu-C6dTN;9n^u#V3f<~VMbkJMro8OVl+o}bVu7VKj>i^QZN)0VMsf${A}`>b~H(q zbV<96NuBgbp>#+6VH)Ey2YoL}u{2AyR7sga?RNgo@~#X^f`Agfp-Z`p@_%%6AHa|! z?_nRBbW36MDR8h#Uk@h5^BfvMiVSX`?(x3D5FHEU9kRzwkf09rfgKPvQQ09JsG(6E z)lo?=QX`cWG*c?6(hlrE5+*@XIh9i%^-~))91t}dO4BX+fSq(JRqugGi;p+YE;jA6 z`}B_)Vl`IhlumIqS9Nt)d4KN^Z_^?>kXMCuSc$b*jrCah5ZFfHSgZ6PL-JSsKoXc$ zS3d!-kb)Ja)mZaWKKCy=WAjzLH8zzm6n63NDs&-r6I@+YIA@hUi`Cu6Vfhqc7rRng z2Qyn`a4q%KC2OG;dchJoFd-9iNQd+e4Ax-&G4&|9GATVY2st!DpMM}?C3a#bb_hLm zVKLT1e-%S16k`{*VW)s%MfM3E_CYf87q;OOJ^>dR_Y=0k6C`(XC3hPp;c_weay6kUH`Wd`A#*_& zbSZaqD|Z_f;cyM1W>t4}Rrgkz)g_x%{#W~Tc4?P(W%hP)H-C3^_jaMxA{TLYfj4-C zcX)}ncyG6Mk+*hx!4n{_ApI3r_mw7l!B=~CA*1(t^|coG@NV%iU$OUl!FOuKcUTo6 zLnl;d?{$3R5`1YAUVGsVkdiYK7AX~heeIX`09IfLHY5+WDigLSxe{bGbYuybVkdNA zF*SiPRT1}>V1EVnfggBxCBY7QHxU(5cp>Re;Ifr z6B1yV?|!3J5uj2Rd2tHDU})9%SnW55b$Ew)7<<_7eQS)Bz{q7rdBG zr&trDKub&j3N(QsS=S(=cXi8ef<mrSE!mGNIFmJblR3GQJ-LD}Ig~|tehzx}kSBmVx)7DVm{Kx1mXQ zp$%c9Ioc2cp$w;8Gw6WDk2MK~WZt6+S_xb65r) zS$`pW+K78$50Ze4>5)(S^uACb71sFmuK8(ux)%%?kC4C+khgdxL1R6bGpVwHU3rqF zK=ouG5d!wFHF%Ua_>&cTu^GFu9s98%JF*|!5Jmwh7R3|%V4EeoIW1X0306QSITAWy z6z(<_Y@rhb)<69-7g)RgwO#wQVVku9cYhalfp7`8wk6lL372vw;Q@X7x3?jlb;3^pu2oqtslN)jwx4Nynx`X?-wR^jNI~Bb9yS>}puz|0k)ebgc z7J(tXPgE5)fkcb)DIFH2$*iIg~eflpB17ad)E~+7Lp4qCMfECASegyb%Ne5ClOG^dJxNzzonJ5X_*y zF}6V=Hexk&`c#@IS-NE%ID%z1KAYP@F*&oTKoR^lXTv$1b5>_(bw88VJ&SzF1-Hoq zSGR4OxOIqr;C9KE96mSjSw%8f`*vrO92}zImSe96cn#tD zRN)pb6sAlGsyp|~$s(>1-iJ>j~m8y6f}wNV?0_xE7IKn4I34g5EOy?@dxJ9vRP z78fa$f?xfFKiLde9goZa4mNwUMOmHIdDw}a*hl$gC;WjEA(of@*(KT3RU6vVxyOV3 zlLKKC)Or*GA+xVNvMHFdx7`pjyPeIwe-AX)z4Ac$x!v9U-TV2U<-Nb@y}$8$#PL1f z`Mcivz25`D5dJ>g5O^`3!G8eYKm6bk-op=i;Te7r9RA@UKH?=l;w9JJnn7csHRcwS znMrnpsn26QHrCI*sHGAPOdh}WTfrARzYPJ^D><`gJ?3S8=4rm>6L^ztKIe75gHzb0 z>0QA!VRBKEFk}4p#i0cadK68J|B`o^dgRNp||2GJn=d)Ij-qoss5%CTxoAD{X@-3h739uOYp6|aq?~h>-+CT-Sh+SSp zCfpzsf}#HI>wZz|?|&%X5K-eH_GN$eWuG1N0UvOm9&n!=aDh2*b7t9LMsXh>;=cIb zUQrcQ_3i#sBXtxO!3&@u=``T5nhx<7j`P9WyPY56KRn+_+@F;mzd;(^>sw*tmn&CY zu<<$8bKb{0RGz7z%N6?$%%Iq001FgxvqwJH6?p&e01g16j(@;`feIQrXsD6Gg$g}+ z8U-<;#EB78#fp_GqsEOKGsYT~$d8{pa7vcEBhpaEl`J(!{W~SF;J{A`8$OIUG2$tI8#{gsIWm~ZlL0qo z{Fk!k&73=X{;U}<-ja$&mi31^wd&QZTf2S@JGSiEu7TMdYv=FMuX^>0#w%8l@ZrRV z+wF}!Io;o)Z`&;nJ-VVu)T>*M(`}A&ptBjP=Dp*>G~lqDSdiFfz&5@4`;Z^ zp(lS!U8*#)-@$|z$5E8tdx|6?Nhl$Ou||RlF38{-aSQ@sgb|uSONAEFqDO`dZMYC1 zi+l)Th#}HJM=B(qc%q3bm3UE$E`kDMj4j1TV~sZ6h$D@9+%;Zvg5(HfkV4jY)muiQ z{(_{DN`EfNWRpe~gGwxPw4s6q3|z^;1t|p4MjI9tQw&D2AW7yaQ67V4nrg1eCYqjz zgi?57Y0_q$X}TB-n0oH{q?sgADS-qUsL%!-V;-4`h>HNi=puqVDo7p~W*CSWKLLf8 zPd2H3Dk0{$EtU&;ufr{cYh~+7kR@%bw_%8pGTk_`B8x{WXI-*@8XU# zRqw)PfL8CM&t8ss=FIlNdFPpbj(H!{GTKME4T~k3=%cUBdh68A0ekGS&rUn-r3@(~ zp1L1;x+SK=UuLB}+ZKQ`MS0+rGk$-xU=kSgJ$M3g*(eW)cISwtPc z69^V|WQ7(f5|TbLibag?CYnHqA%8jn#JJI|I>uR_$OJe*z8s5Kk6WBy?iZ>!_@!}8 z+)Y!?a=P^?q9jA>q8GmyG-U**h(^5C7ClyyIwT@uiJ+JpdoscdQ9=)*Fa)24^nkNu(bVF}P47F@3B|&fjKL#aPO+STc)Kao}XQqoA)mUWuIKehEx8bZ(f$ z9AO?zh)h9n$QBQIi5HTfgAiN*coMkM1-$nmiPR8dNJ<4vzL^)1+|mzrP=qGFISLLn z!iR>4*g)=Z1tAE*1uh_-3V)o?hT~!EBO4>9D93U)=@m4TU|dNPpZght$e;wzRFSZEJgo-0F6>yzQ-TeXCm8xsG&_a@Ppm=uR|p zG(N)dbOpu`zwFb*~HEGGceT-0kiyhoT5Wn9vZ40Iw;{DA2Md4X@-82^@3~ zO$|`nPJ0zklhZ3JABtg#dpd`HGEe(58ybHwD_>tYvNK+KMHc<4eOCd;|c@zh?R`7usmtO#T-H4!Yv=@d^I zhGhK`l%4fgRA1QkVHhTWfkEk#mUie6kd~JGN=ge#mmmle)X<$0LrF_2AR*md(jYBJ zi=e{r@;vML2j273S!bQS&N^%Fb@o};ecji``>qv^1!)3q;mAvC(C-L&zectzHEcS@ z@`Wx*A>T}4*{ief-Fk_C}RkYJVO30 zt-irDkH;SQR4grrIyJVe!H)fhTUIJrfwrlw2QL^g@XEc$ueY!h)a^ggD0V4Tn#tl& ze%bHc#69+jdOKLH<&l}!bv@C`%~HSfr|4}<`jJnLJUbmszqM>%@a~46rBqnB*}6)j z2R^K33|~ty(eyB#xqiufkZ3RSh~u^fKKcrz3818V0a@n{Rgm5Czviq~$6>$7KrmZh zR6|)24n6rrik&hjLNXR@_MlaS{?9v=t~=SL6+-yPdIAmaIQcz3;47K~A}crDr@hZH zgM9uKuepG988fdK-$rxqoR--qh`bkq3rA@Uy<`t;7P&7o_AFOwFI(Kfx{Hmnd93k4 z4Gu9Rms^tVKG&)l|HmpS_`*8Fp|->0BmN~B+{vk}G8Yj37+7%5MXT~QqWHDw9wD5j zQ&W=I24q@?7cGguuHOkG9z|@My2gH?R6v5*0o~^H=Y7TNQcT(KoL8kz*rQt6h+#E%JKjo z`9V(7&D%?$WKL-Cyz=cgB;x@3h8FVE1_@APa#Fb?v?5tJn2Nqz;))j^Uip= zOQu&P|AAkINMb`ACWF7InNva_9yPBNc|yCBLR9B9ukZUzbNY0P>M-DlEKcOsr*&dc zya8xuSswHgCz@A{Vg9wmmfx(eRKQq}{4%FaE)Hx`LwwdiOfqK@^NE;VJk(n~!n@Ru zcZWauKDBis*f`znOL17Y2t6{~&!7FjhzKlJ3Bu4^hj8D+uH6%sV-t4&rS=|-zUL2l zi-RPaz}fa9;pbQrg^)t^s8O~M@_L#=*02YPFG9`{={)Agv?1ecP*=7v`aumTDHe+ZZ}P;oeUCjfp?#wl*O+RqwHc!Wa#>W|%jOT#}ErcM_lb`U0Sm#5WV@p0Sp{$mQSwJv|D%HHjjQ@T~{A znSTL6hZ)?Z~JNy9_{&pSof;HT3FY$r&o5C7lZ{eiKGH;I) z=RiOI_*qa8eY7_mQrIOB>clt3O>`ELQ3=g{<_O7%G><-c-wDmOwt*BPxOqdfTokkK zH9;PfBB5HesVib6hLPDz^fZz1oLVfzFEiscuUI1-fH)y5`wrrrAZo;=4X`FQQRl** z=3TtXyNt~HQ!N6Gv>8cl##-RE{GZ!oe10PMhP&`SS0V?CZYRlUT#c6#aBS}+6+#D>~t_ga+DYFYvS;(kvIX8LgA=9$z5~iDBKB{ z;{ii~$WWo2rXVB$lC)F!TO(KDoZP}A_*EoBfnIPX+~glYu|7#Lo*<7tAciHX*tERZ ze5lxRzu1c8!+#9XHkuzCUVm_m`d}JRNSpP+b^n7qNlA-up{Hhv@9UBmG9`ZHC4oaF zNL!f?K_sQ<#ShMWrD3m2t+GlY%1fjF<|_=9nwS+mIVV?40K7${m3oq^!2y^Pa%33H z;9R0P0m@C1+YT=;i7GEEFRvIXui7uKCaI|0C$(iLDu7osMpZPISG1H@6j;YBIhD8a zRd#Du_AZteD3tfpRrU>4-sLEM`CIW1=~OYUSvC6_U**18ahqvA9ZaPFCgQ1W83{@wR;N``Sr3_^ap5)ScW$F7AnOT7Bzt8vANnH(#ed*DJ zOA_Gd8fZlg>2M9~poW69b_hY%AOo(3*V0DU(pS_n9@Ko-tmy8jW#g~o(5izV|B-?z^hoS}G^Iy3XD&73^(ewf>wlscviQFgYJGVT&2Xv!PJRqtfPX1I_)@*p zRGJ9O4TMCank-0Lnk!no`CIG)AwH;<7XH@mrO&qS^X>n^{wcJ&@_(v3MW+52&2VfD zyjd)>G$jE>Lk1>VmxkM3{HkB_hb%6&ZECgGp0+jlLMqMM4~E-IJ=+@S(fdp7=Syu1 z3gz}%U;nhUEg+iDD!$&D7bUf%oDy^pyR^-degRc>eEQwKyZ4pUqJ7-Ctn3t~qR{r* zlp<+3`HHlY`TF@4t}}zD9m%5IrA*W{nPraWu;}8~W;~wgcxXm)*3l*Uv-S8OX+W!6 z`X|GgUa@2hB%-2QafIRYq^lnbp%Cbi*6um|6I`6oqw%gwQv0*WbMQs%^Eq?FC8(;C)Wn-7mC#Sr_}B2_=RoalErF7Xz&^~+&cwj{vQ(IN%m7Pkk7w(bxa$Gb1qlNI(F88{=Z{>~#r^>W z4b+Kxip3!OdT^kFX8=xdsW-&+Zm3jts7kx*ia;ZuX}E5=vmqN+DlilqGu%w35i~Pg zbUjSWMA0HJ6yHN0fwUO;%lLVKY$!nmKQgNQwSjCnJ_Rc={B2LGl-gXUgq%s4Zb zS+&hwdXQ$aB1sAoa#yrw-E?xFgVx+2md90`lS;x^2N)Wg+iSH9~L^S2bS@Q;UmTd_lEPAt#o9xcKSK zLM==zOJ}KJa-oq~tfp$|^PB0m8_}1r<@)@^Zf3?RSLAYa=hC2{SpDd7rOncK6+^S& zN>%Ohl+NO88$(ysO1b{>N-VetcL z!yMArL2IAG$NuH4&bF=+@`er z*9#BwJ`d79)I_B|#W4Ylikb1udP_}is}Die%}QaA!*Fo|lgE>3;O|bhoJTk1<;jj? z7|zIGMl4%TLbml*XD?1(HzJ(t*eHz1#2i?@ApS|ZJ8$E*6gkHKTb-8G z-8IHD)Vv`bKOuiPwiBE?ZI0ix8*2{uxEqzr5cqNXxMTYT%l57ZX_(ND^2r^ywk@6D zy^n;j*JI=Ex!b8h6a@sa#(}v79Sm!%B=%n>m|+Fx?!;=^d96 z!p89jYf(_BDoV1M?n(nS3g&g@;~$vj=u^`HF%QpqK+JI$TV1w*-&W}gfI&n zAvo{kCKq7-ZjO}Vup<|2=SD_$eBcHb>$dKEi&{8F9jkN@TcE0W6`%$;F%0Vm2*i|v zH^m1y0D_#$^dOYe`$fY9n2uw(K(wih6U)KRM?Ow&KmBce2-+g!fDnDg zlc|#sZSVT3uwH%qe#}cioZ&Vjp9k=A|23z3)Q2m32Kyte2Lc>3+(Z7fx1>;K0IE`? zA#j&x9lx2Moi?13+}Ti-n32aOg$|g7`gjSN&4aJp?MS8SBIfrvAdmAdjI|1Ea)7l3b zF}{gs=c1ES>2+9^LY9|mPiQh6Xl<=@m=}mef1%@4p?A7()j2>ob^2c~?cL3nE^jyLrR`+IlPv!xuXh$E&w&NHe4gnlO9^_;;-1M-u8$ z0Y2m&`yRbML;hABdh1VLg`W!L7WJ)+%bo(-D+H|^zUs*lD$A2nTAU7%d8njG^K1A0 zg!bzmJ_>J6_v%<2M3D{ii$6*X&u$RK+t7|a&BTnAozg%o9;8%Y!c67Bmitp43C`ml@|i2X=}-e@#o`jBmw^ zX+7fpy)3FzK?oL8YSXTvLM)4vk5UQ)825RB#JsQJ>P}s-=W>c(>+JD1;81{qZ*z-- z!c!=QJCcUiR=Cs#S0foAixKU^QZ}RVOF2o%njYIFT?sEyDM|e(ElPVm^q}TvSZD0I znI>}ld!E;XzuggPyFjhRV4#p2u^=Nx5m;MU`w&aqoY8=DT0P3y_A>cl#(kjeBK#v* zCX2K>OOPY~UGxy!tK!IEl@OM?7kiBCLbxJ!bscA5zDHz-afiqW!$hojHx0S4zQaGL zx?S)PW5It`;^C0gZ!`mRbXd4&2UEr@qd2lPKvf#c^gE@5^6N) z#F8`+Qc4(^@a~K}pKYVQ^G2mwontBR8SY&&Pxl6V8J?U(3L2@1$a`^m?@iy4DS7xJ z?boT@$Q?^rAjc4ovLr{HM1VR+3=f~r4l~DWVe7G{SH|{+@q95(gdZGh{N~I8rMMP}?y~wRS%fMIf2NHE9$T%j(gJ<-a#8AcWS;@2na__TRc!jVF zZ(V~Ygpf+3FpM9m0KJ(59BSyU|Jc8E_Q~4uT0q8rF0+!*DzQ;iV6Ztf zM6p$~&q7jo2IWkicC$F4Ao|^G7RYve&{wKqY1_zLp}?v4jvI zA)pTctp>qP^uvK{ug!R)%Kk=hDZ7=hm}{YZX$l0F6^qPU&Zh-&-;1`bkT>~r@m?S)h#3ug9QXDESU;gmESdj@;}CH>Dh1>hDJTPlSq z9hZ(xjC=MbjcWe%TcsV4UG}DQVhXpaZdWfFF`ATbx-P-Ep%D9SlY0FfxarMYA=NcS zwh9}7a>+E&n=BgL=Mf1)I`NF$59d|s_UBSI#FWKt^`>Gq@QNufaN=J@!@j_aSdt4vP8c zMVH`vndbg|>9RSI|6%$oJ^Nz#n|;*3lM%%R1&0_B87w2L;eZbc$Zia;de=dz%m|Z( zfMgKj-km}t%IR%)(H`gbek&D{`_{R4}^+TBAktzX-C?d zTzO80teH&;6zZH5TdC-ZSXXr$aXnDuUom3CDi$S_QG!jVb|#U`LJLO0ENZ5(hH-?3 zR&OIwHY4Hv$v-Y>B<)j!Rwyu24Z-6^>qQ!WqtAM5O@+tP%v*%7pEt{SOsL6*o1qcq z;NL!O30RSb7&4H^uqTBcG@?t0l2hQVQ#4tA$9qTCp0;hv0lk>V@TAw)p;D$pK@_6m z;VzL%!@_{btt&IUgo+F;(fXtNJj)U9YE6Oppr!h#49R;ODHUuzBIH97$EpX>7tEh&)cL9*0~qx?R1ur{7vczbmA)X!_vNz0?HI zc$jSP0xOHGBnev_?qV@hBXf2B zx9NH(mHC2G!20$P67_E=S$@bk@+8dk;jxD=z;#UQKc%|X6>p!7~c8zo1yC+NQ+yP z5JBZfEV{hnu59u*R<=@qi^uFFKD%q;bqVr)FZ4%&a4xeq_FZirmQ?zzkD2wfbZ45-^QB{ z-vw$k&mW01e(Lv z&JkWAfJyH*E27j};d~nf3JXuzXRnG4FKT20IAtB6?;rkdp)t5cj#;bGW&nK~S>%lX zMFAAfqOlrEtT1(eJ|cV)C5Ge?hzybc5dtx6qC4QnIw~;HzXdz_gGS7`kO>*&JOf#u zuF_5z@2A*3nTk@e6_&kujs1s5fTrqm&HXv=9Gp}cReukpWwcpEEwHYI$pI*khZok( zpkrMcSKzN|Oph(qdR#68F7U_$wZa`k!?)1kX@n1By>S6j5iCsvhC>bH+Gq+DwcN$* ztxPQWFd^f0xMKntR5>3RaFRi?Ny`2%xiwnRxdIHZEezwK{!autrN^Wg2#S52(}N)1 zhR35bs+k^l*=Te@_$z7cAx`UZl7V2m4vI&7d`KSZokf8ULtTO@lDx^x0-hD6i;{@0 z+`r{L<^oJlY{<VbN`=WdAKG7qk`KJ|n^!0mGQ)<-s zqwD4$QhU+_Z3yX?Qsd44Fza`;KA)&7=KJi8AU;hLLTmy~Nqen0Qa1*n#Z1sIXNj!1 z&_cfG7!xKLJN9T)eZ_T_to@@^>lD-FdMie*eMncaiWb_RI0p_f%wTpwgMvFN>e)Uf z4@hkK*W!3Oh>*bk_>H13&=GYJlB-hix^iWpNTzd4!TJU^@9JT_s9Hn)XRKJ%=XboQ z?2pf1r}byE1HWS){Ns94DdSOBE|B=tB){J(nolbrtllWz9a|t^D*BT*A-q;5Vx7^D z^$W%zsh+Qp-9nH=QOVgETG)XtUQSUbJcnDdxQM#Np|F5=I7g#H}}zDQyBY5#4@Qu;72OE!VsXn4(Mpd3o=m*#MpFsCq|r zwmnQbgMRd1VdcE*_wIU?32cd7UhTmmQQOi()e8Lk5bDr9ITKGJA%8aZ4jmFxzUF&E zD^qooO-PiM5;g&?zd<-GJP8x(7}Qsx6Z4wX6OKiD@=QUf>3F1mH=#&b4Z>@3ua!9d z{fVTCh6Rn(%ldOVAJEZ8Pc<7-xOfpMLJFhPNQ7S>JifQC=4LFnJ(uK=xqo4i)mQ6K($&EO3`78bDp$NqgG9iM-f4` z5=HRI6LM<60`Q8jS|Iu86x}Y>#i?vJ?BI>1uZFjS9aTjrU5FN;N_y5(U3VkXZ0giPSo7X=HlH}uqF8fYrx$uvk_ zEn^Fl2N54bIOl{6I~J!$nXKLP0g8ksCF#XcC5R)+AJd^QYV$1YHE|(&P@?Yu4r@U? zi{S_>0+Yu47+4fjnqwv(l@-T^c?wHfFa7u_F+v-@$l??#tH6{M)h2?74JH8yS?fre z$g?{wu*^5JhrW*xp*lW5`|x6=BH0ZC!l*ULBypi%TB;&uIYUmENqc5FkqAsPa|_G* zVxS1XMWegNdxrk*wbub#&{$a$t&(*lw(yh0Z`1a&S=yK1F{}g{3)XC@XDMW%0B$zwd@lSVV*mYf z2*BSrt~&BtV^V~ywdt2IjNG5_-3tbxnojVR*~{kG(ZA66A=6L@M#7Rc?GJS&Jw$OJ z*StsY2X=A5)pTiIAc^6gkVnc?@7N-YJjVc%_1<*K$uNpC_i_t~0VGEFhXANuA&%GI zJ(7}~fwpBXWu%FfHUrS24Ye=5fYsVVfL(^lD2larfG6M<2% zxNpt?sxFNvJd5I8jrM~)9PHk3x3*IXjk(8%6->(x79nk$qT_i}Jm^uZTxQFgxskG6 zEzxH?HEc@@eTZa{25Tb5g7xF&?doZNuV16p_?DxfedlcGI+Nn($l0WPPRO$ zN)k$_@iK&e1}h;{#7EQHR|gl8=hDXk4e6Ll1ky&-H+a||oY!;>UxE0=V)*_`6inN0 zWUysV%jS5KPD+?@7bYru`rz+=lBRBn7$+I;8$0=0TcmTv#!C=gO*u&?Q?i{9`CkO_ z78D@D9V1E;Df)ZoM}edKV|(^hQpG@Gy*KrIX-syk`Sa5vUogAMiMwL55#9=y-W*jzD(r*}nfl{z^3_oG$w}#WdfmzMnr7nj^BbFy{i#@@eAm;Gzy<-TfmE7)iDz z!JF5OsMm$$ugle7z_|Ubq5bcBFGyxAs%gmgL|*SP9n9>}eb`ZKcoS{TL&TwAY54kp z$uo)KgL9*leXA~vG_kKu9_EM+PsTfE>OF|&{Mn~hj7VJ77=em&Vy70mbIk2<)$QoW?HKQNLhgRb;(jLNelG9+OV|D4h5K(e_sd}S zt2p;Rxp?=#AKm}8yI+sF->kahkKAwZ?gSJbcTPaSBM+h{9>8ZF#8w_4cMmYq0}}56 z&GR6s_8|T0K{oCITk{}4_Mo`+fKzxUapD>qhhnt- z!dhWJ?j3js+-?c;&22>}|z2z}%LK&4b zh>8#-E*v~pCnTCc8Q&h{PyA}U;gPf^Aj+7I7ziOn5U@}9mSfNmbyPFpNJb20rxUA& zKmjLE_G_0f^G>kZ1fZIcP5gi1=>*V|zVXjs7rpSi3W7Hd>hkbHw}d{bC|aB{48n>c z?LZ27-M{>0k!b$nty^OJ?;@<9f0&woSf^Wz*ON=1ba{tYp^Tk=`Rjl;?i`-?LjBDZ zNF~(?2vLNQI6oo+v&>R3B>yFoCLw?oh4Jz_JbJMcq(zyR6e zMS>p+5H9(YhcKWR3L#+o10)eU; zCqQAszv;pXKovrK`WpqHGNkCqKLz^YaC~~F0+505NBXh?kd+9JlS#i-0A2&}S|Tsg zZxw;DgvIHxNKd7i(UVbuZdh5xF?o7Yc0UlH0dm>@#gg(YGIz)=p zX{MrK>9@MTqkH)78>${q(ig(tQwX_xk2ZLP({G;v5AWka`KGe@og>)e?H9P+`)}K9 zebaCCfIq45L(%=8+8D0(=Hr|hj|Qp#@GZKh-|7PosPR;1vGYb^EtBPrxx@R8E{IEq z74`I61K>Ul{?_xeNgw&SRBG_*^F)b78J6r2+MM)TLm&d_=>3N#l7KC=k1`N{XVOt7 zdRh_!V`gYbS7#$Try z|D${P+LCG{Zp39Mgtgo6Ig^e?vAkUz7B^Z@b}?7sHVCh}&%~xQZNGw5hGbg|rMSY( zcJ1_#4s1mSU^_f*sCq)p6`m{oIU#psbun-rt#{O2mI$MQLC2$s^yYR8patj>@)M- zr*BSPYiXWfaed!Xu2j+yLUZ}FAd9K)A zCK@40@nmE)TUU*<_KxnQa@vtyqj(6Zs~ZaY!tAuzl{@_rQyYgPGgM>t#ohnO8Yq+h z9VrS_juJ|omB(!IRR$B)mSn~i)M@fp_nbYFB4IFS68NfZ_qp-sw3deO!RviiUt1#) z^)8;pdj-k+>Ud6Cp8J`#_Zu7drgcNS7EZMk%Zl>9^1OZQUciBooyoKKqDk>k_Ejm7 z=2#rA^P}M3A)kDX$8xrYXr8XBk{|>>RSEI||GklH85wF@L+wvg=Ys5ZErWTwn5YJQ zD;*v(X2fj|<=Cixhq@JL;u9OSo*oX2$2DOGv^4}m4 zVh>A;0-OdSZCBWfBMU^Gjk;S$#cDqB48@Du#bd8n1FA+E9Hi(z(JjY(Z9A+|VF@vy zkGP;NogN{R6tt`r=F@wfU^~4gN%&h-s@LeN&95 zO*fFwjL&lJQA2<86mIYwef5O1d1SM{`@B6;V@vA)ITlUZu-a@y_Zbb&S zIT5Sua1IZ3b;9GMil;Dp(>_g8q^{3g8JUC%2RkI!-WIDgYof`_M3wp07E1)w)WeW^ z$94h4^MNuh0rF_gl97^{SgIOsnf*1RzUj`rria#9kgp!7y}tj1LAyA*_$=(^+MnB7x=Mn*LvbcsN+T12REIq6;N+%G z0^R%w4RLR^@_RGe+0vs2jI8Nrs}E^EVpUJPETn{Hi($Xt59VoEwh_Ua=rLMuIq%cXmxR4WoaYhUjkn`x%uu&rP zm>;(0kw}{i+wuY5P@?UTWx3OJ>xXs`f`wMVz5>M_g6v?Oh+CK6?5#^|trzRIP=&~@d zzL1&Bm);2b0Qc0@N~Myf{JzbuNE5rr)JC zkU!Vq{SUBZ;4(w(;H|{V6lyk=BhOjdDMtRj!K~I|0{X^Uwanc6-d$7f>oNzf@++JP zKCwVs*jUp>6Mm7F{$9xF5wFe<@N6x0?G#mIU~@T>=j-Vh;3v#ILw@fRqkB1I(7H(R zNQ>igJ#5cjd7yq0@`*0SpyZd>BmVpYr5uj`(v#k(`!6sXZDoDa=-ZZPCJ^$tpIoLp zW?bK2ocpTwufhF-UwQJyQP1)@3(}^bN)tEgdXC-P=layn$S%^HeIwSSAAO(9V<0ca z!dJqbzSZ~(9@^B)7%s5T)Y)xURh#(e^XCgl3S;sOjm;81%OD+U^L+V@Hoj!yXnqSa zm{llREPc4zQKj#_nU()@@K|$!$d&Qql;wZZ#@?Kd)MnY@L;l+&t>03*r^B8C-$&Tu zlL<^TB27mW7E=Tq2FBj)a zM_WIEGbpy-Z$bu{4Dfa3+UJgv&SJvG+hkUB%v?&xrxO&XBF`pV>@f=^A1O2@$g?1o z+Wen-=(xp77)RePqNZl6Ss)sZos%jXq$kiI^1SCaY_MD8m6oV#Z1v!#!l{UF&hrgO z^Jlo$2c}~6VYMF*gCu@}#4qN%xMw6Uvan*I;>AuYzD>PyM3@a$_i!Ce zc}r#|we@{QrW$0O^7$rouXXJ;eC2!+SFgQ7^YrbIal6;JYtM*4gfy2Hvv=0Ffe>BbloU8%!Z_|UiK1oi>C=G85CNYLk}E%eT9?zs^1kKISN6p()0-teNl3g{ zIk2LAFqtg>rY_wQrv(mNoE3+(Lf{GzfW@YG`^$`BN{%KuDE(;Nc(SH{Og~>!ke=^aL}IWlYpso{OYfg z{_8dAuK{!a7SBr~f`0xF-Rs-x??&379rLe&tKLl~P2k{@uqW>}XxFa#cXRdwzrNdf z&-7f(vY|{_R@HT@K#Sy&BhUjw5bQ zN8^M4lD@b(?MLD-z9R9LYX|4mUkNx{{+)jXh;J78j+u!9atq){ieeJ71iXB&U#{S-wv%}&~ePR5>2 zrWwWP=uY}yiZOrNn5nwhIJ?-zy2^fZJ}^?UC+*_!>3Vnv_Y&yhwCLiq$J6olbP3FK z3BFV0Yv|&uR1~7>7Ut|0p?dP@T8WEHS=7E;!lzrZQcg@;`GJM9R73aUo^IKnk1eyh z<&jh=GBe!@RCjnU6`W!C_tS0_DiXGYPf9*LPeXguU7BtTdNiq`;*xr_XL@wDdah-d zwX{_-6dH6ndkw^T4QJv}fZj(yG6Kck=XZ3k&|U_!UNM2ByyRYsnO;k#p6}(Vh*s_w zRDCv_eUjQm?W~+8hJ6n9eJ_3b97Fq@Qjlu39*JrKea=07E&^TRHkFa*|3mlkpz8PJ z?DrDu_bzIbF_m()?|NcSDP%V?z6ex%Q5q(5|IAZ28*2stvuIpo(dGBPtVx-~NPYh)ZaIzcr$$vHYD zHae{`ik~qWowXmG^BMgXIy#>+x==K_*f6@(GrBx8y0SI8`fGFzIJQnT_MLNV;||`X zGPY$jw(TQQzhUg4XYA+9*x}aL(XX*%;P?sE_$lZ3nb`Qb3Ud6H z(fEb^_-~)_%h2(wl<_}B<9{2*|MiSt&y3&Ew(!5kZ-Em8%;Vmy69DlEBGn0?@dUBM z1ju&+95w+-oq!fkkbIgT?VTW-oq%mmkY7wt5Z{GYOj2@9-V>j^uR3`rc~Uz}()do& zhE39?PSO`oGJKk39Mv11nPlFcM6z5=J|LcArJiErnqn89;!vI9G@jyenBw-GdKfmv zlZv0>EuP~0G{xULB``ZBxIHCwG4+Ue8bLiR%rz|{{{IEWoJ5jOM2qt=<{!*%_Vf8QqJSXT-C5caSHpSp)G|L)BR$<5^>e+5dcJpNGww zq|TZa&zgOjHSe9Zn4PuUo_%pKYehU~eaG(NnzI$3vs0b3H=c6{n|7oF*`ekf<-jkC z|FcGeoO=JWy9#gm7w!@^=a>*N9~gdDj)5T&&tVWkp^44_e~|m^-0L7MA$9cIoNsUh z#vJun5=9t**FnPv(AwBG*#s1j)dbTaf5))3j^T zBId;cxy4i+v-wR-ox=j$Ka5#6q6}sRsxk8%4u#{)Qez1Wugq$Q&FYz#8b!^D+6ao} zb8FN?|7jJWKLT$yGH4?9;>R7szg8?_;Z>W*Pg{rGt#xHhMHFjN1 z_ikiAI$4myU`#?bZQ21B5UhlXKYOkgC-}&e{c+c_){}RB{AW zh7}P4<9cH`L;XVY9&sGCH6iypz-^W2qva=Wt7Wb>n+&TsUn2INwRJ8&eV*@RFBV`s z*54n4?fi+O+QN{IrC?-WgmcdP^QJJroGd?_O%yW-x)bS67tTO)hhqz8;>Odx^$#~M z{r(`>bVYBCVhrY_Zfoct#=g@?pSB?s4$6kuT#d$hd(1u8+kEhGU0HF1;b?;oloqh} zpZISZyM#@O#0U{7YdWW;l~Ox)7CYuU7iFK_AlnNmiEyMuAJi$sL?0f;_`<^at)1dd zEY;2nwN>a_5BsQ~Fy%QI%^REZu#NK7KzvhmjZ7d?wA}ATI=q-7J8{^vh3{yq~mck6Czr>h~$5{mW_uKvtu2hP0u|8 zL6Q?FuaDby@>r1<>p3Wp;s81*jR1zgXq!0XHidCI?YQAza(n1w{Xl8p;wB?2`4MJP@6rdlT&Ec>JRh1y@)H2BI~!H zb`&e!kKED-TN4v*UNP4aVofA+^0`yYI5ylhpYAiDJzx`J%AR)+}7dasAH&pwL zMXChEQkSqaZ5y89mlphIqk@u z_b<0GZ^J&(tTcacki7a4k`SSWMS1C>6J{ettPf1a(1W@M_>y)MP54i<-v>!Mo2GFD zijZ$=>o2EXqD+KB_|m|NPCI0AD6ILN<|18p++gh}suTAJ)UH=J8jBB~dGitk2kL}_cpy$2 z!AGPXAz1l{nzZfYATT<}CGFy9N)GD>!JN^ohwlI;B(M{QJD-DyFKDchf7=O2AHJ$j zGf$NVkCA(v7dy~N9YqWPiRR6+a!d`{cdeu&SWfVVCLbM;OplyIF`gk#PuP3ca+K_9(K+EB9Cy{%QK9Rh>)bt{T>WuMq@ zWpGbQJ;=IoK^t4=Bt(?ZT&Tw%QaK&x5I8HudqPo|z=Y)%9TeQ(Hv|DvU;DLHe9YA0 z6(JSDsbIlA?;GJC&MLKN<+v~(j1o1XeS%L_+NL2u&S*1?Hf3JT^pd&zvS?Sj4lxG2?`&C)jMi71E|%39DmCCPw@0usy7K;oaS)-ILyXT;hjL4PK=6R+!0c6moZX;Ia)=R$c zz5Logo>U{5*Fq>1ug1Bjsd!1ZTIfSq`810UwnaC0vJw_UtCX$t-oPQ0N;>I93a{SG zd{}{uL94?2k!30(LqfJOy&&9=f^jx=@zN1KsLCjjq%Mpp9W98KA^&pvQV03T;1MLL z@#JpvDokdg;UOH_TmIsA{(s!Bi2IM+uBFblhW>~9buH}^ikRd^uo9}pGqI83#Nq5d z%0iz=gU#o8_afTS`dG>ixXK&4)#sF47p+_nqX*jxp>Ll44NcaCZlho5ye*?s@)e<{ zoL8R6kV+u5kR5=FNndYmy>t9St4N`e4-E-y5)?~9>NWLEAO(5#94$%Ys9xm-zHO!c zroeDRYYSJqdS$Hv+A3xxqsOQ8WI|xvsZZ~P;|f3q1}Q{hv2tp9_T-tVg$m0-i^|0X zAGg9^e(?p-IE|!A*!a>~m#hRnu|%*dy~?szp#LQED>hHu@*+9VYY8wsqb4k{}2nc0tHcrglz z0DJ|Ry54N?jUu%YeJmZ(sG!@#k8egdxq;CgCfu3=O2;aD>4dxq&L4#fwkDO){avUN zr3Z7I1;#2dlGL;*WOihI7JFK7+R66hr=7>K`~!mHvdECyk)xa$4rRrGB0{RS8D6!6 zDnAAEC^Ff7NE3izBdQ$TvZC5wNiqJafw=Xe?PqaeBivmP^5o-jrxp#y!0qBJ6 zOi<8BRJee9aWxbvcwLDO64Q0V?J}y^)i_^(iSxU%LamYc{~{T@=QE+6`0uLE#Q?tL zjl4F^XmkN(7THHg|ZfK3OEUNOE&_0A$0z;vH5tK#7t_ww}!a$GBzD#hI+a z3Pq_5h;pkXSkpKpu}}g}d3dPlCJfjb=ebl3YBa*KH~I4h327hyuJs+8maT9$7N=I~ zO>Na`pUB+SsR0CH1fl%UT6o$7kRbsbo4@Q!Rg+OcA zrEW7(ED0av^gDNz!znRQTi@T~{Ua!;<_99^h?40C(aK|;VR@kw_yIlJjiXGe>^K>r6ip@Bv2(Pc&qkcJabGqZaM+h{8N*2J-oo83~M#(I3c7k^>|dE zo=o~RmQzELefcw6lrpJxx?K#A=0hEn66o|q>3Jv?%(Frj%cHO~7Jwvx#gD0CP316C zQ1U4b#rYW4N4V9CI_mHyz$+ogw9u`y!Kw!~+lU#G&Ug7-6?pPQ9=&@VVH*X|+$jgF z5xO^mb;kp#sA0CUB5N`{r>~s#(>eIN4n8Sjcy#>-OcFk1|6e8QBrAX z=}=lwT0luj1*Acc1_8yrzFhC?bAA7V=MT^8T<1K_{oHSQToI^ECB}LxGNjncqe?;K z%r~gkDE$DkwXX3}HXZt_El#;1ncy-pJ!)OIU)L6+@rP%?X!N}ElLAgSJ?>Y7S~Z7e znQ(#@`dhl}th=~Df(H4*dpow|T_zh1;so8XZ#lU&Je!8iL89oT9-6Rt)e)q!uSFKK z2DB4jH~EWQ=%Q?CB_*luq>(PDcP2i}*nV!j)7#i=YXxcFm3%`!plktEQ$>07mEDzIZ)*~N%VHs`qPH?m! zwgyR(dq#{6DaA*#ugJ`LD{p>b4<_kaJ${0bylz*miJ(Uu{U+6TeMxtM67_rgw89gk zjMqNu8E{rH5d$?Tc<)x1LV)XekWX<(YjFLEFXnc;pq7TP7>@11JJQMH52n}vbxlR8WUlIv-6Jm5io>5rTh-r>&OE^Z7$vBky^ZWCc zALVvMKME+sCYW{A97%m5)50`?i4_Pj59fVn|w7~Vre!b9N|eE9m{aB^Tp zG=w3g_I;?2a&gB#Bgl#cls{G%t8K)e;F}FJHe9(j)FGeS@LL&_>;b`R!A;9+j~D>| z`6N1^ztavBeW!c@2(|Q;8g}L-X%$fUZz#*RFz#!;B_~%r{sCkIUme45_|kVJt{c6O zH9rVvGwsG|EHsn|9pUIX;+j62pPrMF=(vX!H2U4D}i> z$-8we2vkv>iapyov$kz^9SwJ0;ZTNvkRBz*W-3q<+A&OnCHN1U+5L#jm-8RMNlk^r zB9`-avRbib!o}H$Y%+-6pd8#pwY}Im-WvOYacC+LW&J7Y3HJ}ZBdrXe5#06 zs%Mu>s6>C@EEjh1UvY|DvH28|b}0=O6H~fqt+b}%I6neka1!tQv$-Zpyciwwut7l` zIJkYPxG`5hq}{jOn5)p6s+7m9Mt3TLzegm+Vg)Q+#WRchpV`Jn4B7s%M}OJoG+T;) zY-_UPj4mAGepnIrGubPZll32~k96#HVF{y?11%awYRC~N@gJnpU!RapIvmAF(s;8d-Ydj>+?c~tBOL8-CUSm*VI zUFCcpm60mvKUV%{BS;I~!~u7|1AE=%H-;OO0jx}1lRDs|ktlMNy8t8o{Mn#`eJR5? z$`u$kl}0xIA*z<65u^&U1KW^L9@ruDC${*_|J<({|B5$RwVCdtIoOImUIlt_g>BWL z{PayX-7n_0q2=LbHfr(g@!QGied|iiXrop)O`_dr!;bl8;JAn8bRvCv2WljaFBc-7 zp>0+R$F&q|$p%&Xl42L#jtyDe0YDA{jE$bH#-G8O$B98*`(*>Wu5)-9xA-(FJo3DT z;|5;}%kZ_;aFUisz9EPWPDS@8H;mwiF@QfQHc&5Y3EhvR_z=5srJ%L zh3ZCQs4ZSe!I5gIHt#uK*UQ83GLMkHr0zz(_w98+Xy1LJv^gT4%QulqkvKY91yd4} zwl!b#k`uq!x7_wyDF*j5@OUI9cKCUM&1p?}S!4Wrwfgr|C@-=7^2C@Ssn zt$RG0I$_9a^jy0;ndJrOjXRj{2;)tlOU_$PT&u)U*S|#ePH#HsoPSXyMzn0Rl`AR7 z$mcBP<$j2cK-mOtMR^5V(e*)KWyHs<4Cg@+oWHszu(s>-rVE(OsLFGu)#N!JWR^7A ziG_>B@_6e<#20;hxc!n7)u%A@gzaZBowrF^dH-o;RpAzO;!g*KdN2(52J=(k=Vk*F zsj@0D;a(+Spu(#UDqKqIV=d|A^v+gXJp&UOYP94-V`W?_p;b!lXywLFd=8BCm#FIP zI{SE3;F-DFr2uzJmAZr8PqU`g{HjOUv{jy{)^nzMTlFJi5a)WMaA*8xNVy-*^DBsV zj?NA6t-^N{kAtN?Q=&v)29rx?*@Syio&ABTO|Zr{-9oa;lN!gVG^(ugmRoaN08g?R z?=LKW!6{Hms44oBYfR1aoqj$PUL#Jmp(s*i`$K*))h|lip4~A)fh$->;#UWcXWGb% zGI0lmzzqdGdNJ67CXdOI|H>_`j;yB8R)y~wf;9htlRe3PQHEeb>SfOitERRIczeOl7Y?G^GJR}^yRS&!D`z((~9`-dX;`HNM z%@cBO%**C}*Laq{co^ny1Xh*hrY*=jNDjq`T6n*pOHan0BK04`p{v#Bth*x`@t%J7 z`c_4H)HbTY-%-um>(S({)MF8x-;Ima&BXHg_j#2VCY5Sf`N?c!JOhQ>-^LbBvmtwQ z-CUdb!a`hY0%Vz7oyDHJT>z;Bptq4$Nq(5%?F^cFe1w=1#mFzd>i~#OwKhJwckEW; z9+`9S24)e&SM%tP%r7y8R-O$2y}4eTa$VQh8O2k65mu@=(yU?Rtvd@nPg%lwn zohXXc;If<(|B++!b8lW{fAfb=OVofT@_jFpA99Gz1?xT?&6W;a5ttplc2)zuP`&7&7rc&g;b>|A@i7)FFjfxjfE=~0hyX`q1`y|#S>c_3zCP+;4K>pnZPFRTc^biZq* z7X*7JXc+v(+i49}suwILI)Ee|$aey1wnKfLCP^Q>(%c`Ct%kKVlxijT8|EslO|D8h z2v#uS;g=I@uHGN-n8MNmv_%>?iqDiS-l|1SXzY!Ye`C>46i+&S25!FEc3`|mzLc`x zS#{VPKE(VMqg3ShZVu^*?P36|Z-`^AJ1|1jpLLg*F~i+deQH2xqjOH^hi424Z)FP} z2-htTHpK_<!G?$GIgIaA<*^B&3v;Vp% zm?)sjHrtcW0<54+m0N!_ETS0SMg&nr5)kpf-50??}!m4eAtSn zW9dvCmbkifucgJ126h`%Om$;o0g9@zzOVadFhVlrl97JNpbC}}cZ=WotILuV4X6?m z%xj{dh@b3dy4`cDdz^9p%8!0a2A(JW2j0jTKH{14%X^y>T-TWxTz-&h{NCfy+kA^v zsW?A-0)&}QR2q)b%UA8NR@ujblIiWf+8+KIN*-nLBdUD0+L!WGzK@VTfXtl1erQlPSFbcL;O{tik}5<8u^AiFpfcV1yTO|MQA?~X#$hE(t5UtHfNlZS$ZJJVnMU-8jV+>=qR1J?|fvPPeV%0I~1hf z;mmz!w~8B_`X-gwkXVINvnc4q%IFDo{1W>8Z6Cd3zI z6Z07CxMNz?!Ov+xXqz?CBa#orsLSoT;pFJyQTYQ!W6CM@)Q+6zTJG}KrHq9AA!R*M z%4ZjR+oA5kX zsRooyu&h~U@m*7a9O}QRcrLs`+ienUhNTthUnK|H!{JzN#H!LGrbkPrm>)5>gF%D7 z^}E=&4B*Sf!HSom$`ve zEbYWmB50nG3X*Ph8k#646M=i#J2gJ4GA0J&G}KsB<5lEoym7wr9! z%pZ3=QfAjjGt$JMk70_KY~we1f*K*=Uvu?}1z7B$_}Pv|4HBysm7|re+J&AQBRR6K zd~M%{o$b#5&;7yx3F+i={n3L-l-w`>Hhfubkmiw>4I{VWVsl`3U%Am4_ypz+4EzF= z7*caWFpG}f9V|<_k1$qh{4^bko6%Hd;jpt!eL&CCp+C@3v1g;V@vSffvjW zYR3W!m+MD z5P~}2-=Mx9POC4}2BaYnrYcGpl7wt1%jjmLF-Xu; z6U43~eNyO#DX}^L8B0;A$h}q*sY;qVhkt&ccuKuOM`njIlG=CGwIp@# zI?o>}Vc}7`+D^}P1b{H5xI@}ys_lWH-|#e(Sxw#s%-pQD@Px@%UFn3+xipVbQFDf> zK4So$==+L=!}Bb7Lq>_Bh?JlNBblg?cFuHZqC@yj$%@|QM8|(@seu)nt+-Tx^iJ+q zQ*j$=vG(#~qnk!GY2g9!MFZZHOJ+&euM(qKzeRN?yxT&|=D_1pNo-EA5IS~PGHvoe z8c#nR#@79l=q8WTBqv1^-zV)wQFH0oSNcRMj47Ex56)>3&VD=Q14cpBkhzA69e1td zT9c;^1#c_do%e=%kL?~k)}loS4v62d6Hyf%PI@=wcun1jRYMV?A-cESDciM4$-9l8 zyS`TD`F{KPru4bhK~2rBdv4lJ!YF|4KxMyJ4s-flM*5BKhz2NyLPT|;K15zj%)y7s ztl`I(<6O4X7fkQVTrZrgy?9u!a;++P$Cd3rih2>x68_ht7kfOuT1`rXW4sSV;(ZhM z@o^JEj+hmgRcYvaejK>?dQ6`ftJcGlQgU7irqdlZF9Iz#H2yP8y=)%~rt%8i`g2p7 z-hh#@EXkx4u`5}O;JSmCAe7$e2&S*e0}W|97&hg8e!HzGwmdEf##`}Ue)ylOcK=@O zvfqz?*BdYFakoTFTy`JHkf7^0-x~80yNyh6vwL4Kq2lWwk+I?h@B{NSBD1V|rH=I_ zAIFj9`L{f83lCIPXp+9>0JqzV`YkJ24VEz_JycNax#58#x3@L1NWi0ir|Qy3hhkl} z_%0CJ5KOtI@JXNu+jPU!gA?dFnWo7bl8K)@;Wg@9JZ{p0IsGoxpKO|H5%6urgEL~YSSnek77UHpc{;$rZk_f=I(l0|d18?P1+r<@=L><<9mLNH zm)>6?Fmeqw#5eMDIot8r&WjsEpK7-bewO#`r#4`O<7Y3Sao=dFUwg5s_&iTRvA>0+ z>5tb*AI}xJeYvT~syKs#DvT!7sMNPvIG(57sIj}=>VQo6(o^KVvKG-WVetIFLOMo^ zi^xCqcEJMqk}EV=>3Ltor(Wa{-0#x&WI)sF=fZZf=Qc=1D=}} zqRA8)?9e*gU047*qz(v42UpANpnV=6>b00t8b_PfI}6kS*{0+6ZC4vJP@1LGsei5+23H9L{=I%4d!vWqzeF;fyDy)Mg zDhz??v>jS-!UzjNC4~-sHmgw-qR+?oKSq#bu49tQJ~`=#7d>VPi~yB$3MeOCer3zQ zZUh^kXTSlA%~U8n_lr+Tk^Xi|i8*rF`xCO!`{s!_()zSYTAR-BMaYn31c(d3sk^JD zxWZl#;CUE8RAMI~v%|tW>TMwu^EIB7or2fkG?oDsuZCK1r4J9VAY-QI!AW)UJt<~m zD8X(Ud`kQ+fT@cWL2N1cJ_jOm56xN%T?G8{KHDama-H`juvVb(=sQsmBUHQyWRvzQ zryu(|^PU>1ncxXB0lrP@ruggRU7kN4O&|^m5={aSD6T89ifp_Kf`Ev5J7Wlx((!l- zUpBLjUkFsJ%*&}02<}1gi+6OS$nwczn*)@o9H6ln5qtQAuRA#Del8xqVygaUx+?u`twIrQPiH@ju-rdX7N3#* zRR~wr(>V^jO4g|=sYY=C9(!l^#s@4+Ywo}W)AFL|my4wA{3QCBwGkV2S$Kl*%jdlA zD@rtZCGp=kd}-ZtV~@&U&?#B*gY`Qj)~S;BtDMR51yR8 zczt(49WCs@%K~0pFbb}%_y)z?)?6PGy}*^ z0hGwQNg=jbAz(Qdh2AK@f={rz#+Nz$4wHTY+AI5yCXTAwyG#rTR`myKHiPqAf%nT0 zWBOwp*{01Y{+7-bQWJVpK{$It%joeMH4W}~h!-(kWyK|Yj38nz|MTP&l7S@3@h3VY z&4VGc#vOiuy@_l5i8XtU3KW6ki?F;3fQt$hhH_(`Pr_w6A4K>+xGj5Nm#40ol=zH-Luf~ygLEkSl&L+X-cbk1aqcjV`fDP>9TLd>@nmmh z>YslGH-$G0+z-LEdJ&^%b=NTl5X-B*--0uD5)>fpy0l zEJK0(ahZI1Loo6e0#tM7$8400pQIJ6-PNkyd#T+orZXI@({{=7aEium zgYJIlu-vM+=u+=YO#doa|F%{C-=#iK+yFnsfT+!Y?8<;r+>kcJ zkfF_x<;oBtZp0a4#M@>haAhQn6E_wQF_ykxqb_I|pPbND;0bBAndn}b7>b*khL~Em znc7~NqQ%WzLd-nc%zUoQ{!h|sGk<(#jup4~|BzOOxMfa=WnP#jEI-h0MG+-5k$W~|L-;>u=5+;%?1HiGsXkYRg260q9|vHRI(cXVZU zCT@QfVt?Ca|L@A)Ek_&>>Oj=)K=#{#QUXmIie_j>v;0PPT+yx3I`XzV3jB5ymT(de zb&_tsCsWW#8&0aB&YJDcy1$(bC0tBHU2f-L+nFwC30Id;SC4jApWm(lztOf5ZV~Nn zkAJ&iCERfdq3$U!F(zT{?imsuIiVhT?H)zHJqEAbOG7gJsj$Nmg6<{+j~aBXFk+tx!uQU-g{NTcPG^MXS?qao%_ab->Xo++jhTyzx{|L z{D5Kihd?<}5co|(D_T*=nbHcSr13l}9e#Uw%^Pg+`*8mVWm$(@n=|=nDW`m^} zjinFgpT}YmuxLwjmZfz9-x7RCKjV4_8oz~A?k0{*%EESw*+x7{3FaZY6t^R3hxb$P z@H*kYt0&rR5dm1}(t2V*xZ^b~grFqpbxDjL?SmK$nk=RyIjxgnO_AkUcna15dJ1%Z zx){D=%90Df*INfK^)araZC|~L@De{tL_8Y~hZUotDIpwLDCmlj>Awx8V}3f(`&cS% z!P1rZ$jw#eR{Yo%D~{|T(*_-JtZe}#k*LeWCD&x5pE_|pmEjg_(8qX}lk^yt?1eLB zHNkA)M@nTyS^10T2^LR;0W!__a=D}?$$@GVTkA95sA*d3N6QMqW>^F0-R~)T-h^p>eA2&c zDm9d9G7h?b7~+=;!AhW5I0dEz&ezG|JTw1wh&O+(jny+_o;)fd&_VNy={tNSyzvrJ zceVrrd|Oo!qNSg-sgXr5WKtT;GY<2}x^IeFIyxQ~bQI7>oy~T{eQ%G7GMkcw>PvMQ zheQu&Mw{Ps?cA$lkpR1x@X8yOvL&XaZ%IEGg^FamE167-tj#`0;kqzW)`G?nVZ%{^ zU#+|P|MvD5JXm0SeOubIW!<}I{p7a5F8gXWcYr)H$5lk7eqA z*AbZ$M#X#!#{G$8DKy0|c#U%ycxKo}$-^zGv(0K# z*tQzQzg*Dc!~7{o4*C!CcO+_Ml6k<8uHcjYRd|wI^9eLQa;s9QL^RZ?iash~@o$&u z)zI)cgZB<7di+80hDh@uP0XLsl)~|+(Vsps4^OQ0y2MJi-nm>Y4TVI18IEoz>1ZFr zGJgPe=&uuSZ7|#y1T|)OTtK383}qKNLJ}W5iscE@_#tY^<)ErArv_C5dmZT z_wr1}^g$;>3=rn7NNWOw<@Y!CmNLL$Oa?4X55lq9wk&c*bnZY{Gw_Ki3sX#K_iGf^ z^l|wPkf}Qgpmb}Z&oZap%fMm5Z_C1z`B+fDmmyugHh=|Yf~7NH5$XlT(!%E4Z0j^l z=j(dEHU0Z;yNY3Quw8i1@?;+I-N|;Yp?8@UHZO@-!ot7%v3&pjZ}q}zzI1gZp!b`{ zzrOz-&uJDdyUkCX$Za$LYww52W5BNhtBj2QW`>P6xCa@Lt1}mWSXiIX!~S8Ctf&85 z(GkW?1J9W6u9^Ns*7#V?650&r)jUV~@c-LDk(Ll9_7`AqyiQc}+0RNN`FWjSk!vt3 zCqdI_#e?#_#a#ZMUXsuH2PZ%rT)(yV@dW(PD7`E3g+!Hye zi#Ym>JHR}8am?<=e_?mnQheNCder;m*ID5C@Hw51BmdCs108Hme0lP2F)G$hx2xF1 zqjit7+=RD$-}=Y#Sm0j5+F9uz?GGDt$p)a^xPw)&(AUP}ln*EKcGnY^4DrRke=-NX z_{03x0g_vSzq>q;=K!g|IBuY+n*SqM>Z5~Ti>)ywxT$xGj1cS}hFb!;4cm>b*d#c~ z_VR#yX;ag^I*$Ho*v7B`$a-{(FVX)UxVb7DQY+OX6`nrG=zR{BiQXgA7|<<{BJGp{ zK-#fhRF2F5;0B2U#Siqm@H3*1nG`L<*i%ZS!nBi9-H3i*^22EUA3j=A)o-Sg)=YYD zq%iA5$Pn@!235$fF?INaR_$Or0dKsxX@k~~boKftdfS3w>DmyeF4i9XTov}o`m-XB zkniwFtH(QD0{eK?s0>f#7Zg?(ulho0z_cM{9`BQEj2nW9=(!WmFnRX zYDu~NsOvN$XZt zh9$JTlNzae=T);E7a4&=ayR#BUI{BHRc?pfh{aS;nM1bZ6kN5~jly14qi>id<1Y~m zYIrg&N}tWG5n86v7Xx1EWv8Iw<(Prc-wCG=B9&G%!5AWyz69aD=HTaW#3k|z2C0(! zDh4RMG7A;ULzMzknHbUINuc?!cI48eqX0bQK17)!d+uUdx?}s)4n!5qlg*?^9z+tc ztBP_AHd8aFpr?WX@WtZB*d1Q@Mzc>#`&Iyk`BYy1R3k_BoKMPUgU_5G1RT%-0fTHe z0R7acF$E{`E*u-d!ItO9@B=e<@eId=*oG5pUqLoq8BXGtUW2D;h=z-#Nq_57ApJ8^ zvYC_=A0(#@C6^JTE#0KcMJ6Jv;Pnsl2=h+lz9v)BuXwEC+mPZNO1e}`D-+9$_+y{@ zcAxDjPeY7&QjrE_UYaAFyrYs48vbke5q*&^UAx8ikk&!tXW#e7A21|-_g|^)*ZX_z zZ=-Nr<=-^2VgW*bHvx++sGN83?~{+?Md|o1r|Tl|0{~IQJeh?hx@LFc*L{iNZ;B3e z%`|l9zc?Itiv${xw#4h|uq&6O${he+ifGhk2UDp%yyDOo4e>CgePo1f9ns<+#?CPQ zefiXM0uLOr1=q6|o5g(S>lDX{5Eh(nr%PHm{hOIM(Kr$Vdtq%aXaa?oNGF6dwxJ4r zo$}}KyV1CbY2Ft@S6v54t^hX{jJGfocPsSN{c{l#?6Ls=&^%NT!Nw#vz^TU^Ou*iQ z>?6(E(i{SE>q!Tj>6x#FEGxEfrl>e}d;cP;fRE>c7?`5K82L?2qHxeM{rg|M6@MlL zAR_50M^~hHgNgXW+jhsFwH&lG&$kWTUvzs>xM_|s`eT_s2^FANPi~YofA>?nm&?W0 zKZpiL+-yxtr(M$D{c3yXtODlErUczxX@Gn!7<=EgRtg>+GWqUj5L|LnxL@`G5c{_O z0{vg>6jDTiF!A=co7p(R8+?7xMO-2iDHO|Wp&!c=AN#1JKLH0tSO<8n{{n!?SOvVI z-n~n2&2IYu0V^6+X()mM0SV+H+V(geyv}ucpkXvmth_AVd zjj&m9V9SvxYh@tU;pvM!Z5FnmZ<7Q{5*2c27b)cevlxVEcbbkj`Ix2FGuNEgz4Z)t z;bXi}Any+~I;WRwN51o{2MV4k*C-)QK2+~j{48ZDH1939R7GyS&39Rq6iP_8 z4m=POSH*wZ{{m!? z&#R@;x>0j@lKva!yUO;Zf(G|#rcG`SgDW|DUN~LK%sEqZ0t#KP?#VruW1YSyh_nU|jhk`R*%d2jWq_02`V*X$t3qz$aZDzE&c^KN#r(p zF%rDs4<;z`9*rIHM^LD-J|~Me$mRBiBfpmLIm7b?xJ(qE+Im(X4odK$#mIyg-bRf} zC?;?W5+9BNxQZ}=lTDc?c;$dYw!AL3g16A<`VB8p0&4?_4LbO#3abicmr={EnL6kU zH^^gbIXDH5qj2`!abpG$Jjzd`{AjYx&Wx&DY|nGdO(yezhAUabD870z?MwzIU;;bK zZRsKt9>lGm(e%;LdrtZ|^E}`AdQ(c?D^CXNTT)5BecAYNnT_Pa*5=cQrQUNQxBf&~ zX2g2SFkTTo;({Gg5401XkKA261 zVaR^nLcYh@avDDQag#`CKDU(5g*Lxcd?53U_AwTu5{6nVg z8#OPrAUw6gIOd9v+)SCuj7k=9mkX!y`#y+nYGgn49;TUqok~IjUlhTB>ol;Tn+#|k z)PytiF2a)j{w9TEo)PXf@7EsR~r40W9pNha2#DIlabF&Nm4 z3x7Cm?R3m9k;sZq4CQC;?-cd^b0k@S3X@WkW+52`{O`vY$asUGj6{$Q5*k14!4gFN z-av00OM-VsUhC>nr%vAR39@}kUJ*u4Mu)Dz5GHo2vHZ(mBk?CC+W+B&xh>hPy<>JP~F5r=M?X}ql@+hvf zD?Zvyia-mBbB70uq3YLF92t&&Ve0+?JV{kod^Qf|_;F&w0) z#FSt>Ff{ne;SqTy@Mz0-UG&lUy017Vq;ekeQ7NR>l+_oY_H~Ar3V=@*Kz?BczxGdE zparhDgsKIr{Ad|0)k>vkK3|3Ecl~32eZ$k^u7%xhF;( zPm&`LyqWBpnVcS(hamf`3i4B^Ot~<)3h0?16q!0mum_+JmX75A$ynS^aJrtg;NY-N zL%Qw2)aXi5HbQWKBpMTr-eVIjI0ZAqbA;fr)?aBzH{Bj8J6gm!^N+&0HZVG*MvkF| zV#h<^a52p)NC z+(!`o zlqGXeZ_WHQxI42$^eCYx<`-2KFcB#&1(mI4?#GpoKa$d*aP`O~kVfriC0+_C-vctc znVtSw{L&6QD2PO?ekVGsB$Tl`wn&KwIYpV}YQdtNNoY53LnI%S{Kt-4)pId^_hp10ZO5f z$<*pT9YIQzD6qvQ#oj6LFJ9uMcNUMAQs-Rc`^tGKge^XT@KkaTcCD#CG|#l1NIy9} z<8KYTDu4DR95@LfE2CA_l}ziHwzvl>mX_GRL&#QP!737@2m@o1%rb&?nf-m%N2z7; zv>;385YoO73Je0m1i~ZRuq9e2sDhT_%1+t9`PRHBMaCLfFof*+DB+7J-WMr?&?k%no(I|7#u+w1`pL~oLPyTt&5Kc`tGEhU)sfEiU_RGk>j0ta6#2N1w z^xa~}?o{H3BI17lJh$PAJf}Fl69fB=V8&A@sbZH0X;eZM7ekFz0UlbqI6%sDNf0z_ zBu+>+bxNG7D$DOiES(NQlmJEVFG!V?7{f?DmjT_WOnx+HoJ^-wvn798czptgTxWnf zN|Hs*OJ;N7z6a0|B%HE@^(sA$YTZz*4g*x61y(hqxK)%^O{l&G@QY}_lRYS7`^H|& zLzCT%;6({uIS?bKS@#ElW#rB-kwu`mM!vFJ*0rru0o4F}n843f zx4bmhr^Mur?V^Xqzi7o?g11)okC>rDdA4O;}N>{)Hk=g&P8hduk7)w`gY z63H6*7W=n*h3|zugvM${5N`zlwW`v!?+t3Te=_J=3Wi9CH+_>NN7S`HggQk~dO66* zW9zy-UzUtf^{I%dU1$&HWA>7oFi&f8;{eaElgK8zg>DS!XS=m+XG9leBo4jk*T$od ztUltE5;tl!?0@S!9RFyI)e4QszlePG&nlKTi2k39CZMk1K&AO(N?genBav4;H%j_F zZ(@=%gJd`;tDoknSb}#+%e*vrnU99AYfu37_Lt*ZT1J}@2NhBSf*TDYAjTO0h5{|# zmxomckhTVU!hkq>D2Sm=Op!?-K&jRL@U=amub{ZEXo!))cV9h5Ae|9~k;q6-yBGca z>D)k`Jiu>kq3IebFz4;G_5F>w! zK#v6f%P#=EFAyfh6pG0eN_y+#osntlAeH^&vjY+qkVC3+9`9e(Qj`w;_&#*#_%^Zj zwZ<&>s}K_J19lfUBn@t=asTj{3u2ItXHm#DRu$kb*9xQoNt7A=Xm5RH;Tu#R;84aN z7HxS#gr9fcoRPLXCOc|7PDM$hj3qQ|oi|2UajV8!w;fn)oDRXUwt+7ZcoX{oXmX;I`&%w2EgVDXFW%1V=**W}qzzLg1;uRjA$ z_MwT)>?1aE>ma<+wB=Uf*R*&=G@2Ncd}2&XJ$BUsB61u zag=EkOGt={UJi%o<_vINYF`-duMocMAjCR;Sel5mPLG1Ag_^0EHmc=6r6EO|g)#pH z@{EPmpMpNNRGNFEDr#et_llzFYg`{bQEXBYIavpa$OQ$SFZ+hb0VsX8REb&wA=Jd% z9!X*beLh81ufu(<<|Rp?7%oQ=Xfq``vfeb#~)c9C6J}*{_!N1Ab^* z(YFBEj)vX<%044U>ot{yr<`@1A#JD(+6Ob|WQr)oy!7vXTb{*6jelqu@1GbS@>yx= zsc<>3Bu)RJcKM=Rxr-DEHcT>pm-Wz${>!2+hKMiZ3(Xa2o8Dmz@q2=F@IM(|V{;t* z1uM@H3*X7=@<$Sdv}xUByUCI7qualKFf^SpO`RQ3t(%FgU)fMCMy(Uw!?gh(dv5dhMq9N&>_wnp7wk65=E4dCxgP+hzJ;TQhmO^jzhX1RF}+jO-5 znSc}L_ZW7U708YFdC#V$n;Itk+dUuqx^Y0%_qlqbW5w=K!u3&4+W5!b@u%O%F~ekC z`+^gq&#nQ zP;CuFRvE!GjAnOw<>*QMYr@_l;=r*V^27*+`x2}L>7Q&lqCoT*K{OJ%zPtVY1?l)N z8msvO(|XG`-<@yshvv=b;pdzf>xEyItse7@F5g2-Ti&2IA{YZVhtf20r?mr4Uk7|& zIz-ymemA8bPC*o}WfTO1hS3t+^}@GaK8s|Zyw`NkpAKZLH{6)?!VLZ-$VK19DDue1 z401g==s}mKr3L@wo^btYmz);zQ!$`U;;(!R@yEI~sn$efwS+{-Nwa_uh76&1sXHt! zmKAm!dZid=^~x@;p<;g%nom?DqR=7d@T1^jkh}PnD`aB(fw}2j!b8994DOG*#kpl1 zeyP^VQa?wcN*SJ*{PhKxmVMUvxWHOJQGo5q@FV+G9YmI!e45hKjo*-|o+BP}#?=N{ z98IT-?vwe%7#Qfh*+RVpx$h`p#ySrQ)5IFSX=EKc1HZc;izgM*h2vDP>C8mZUxc~V z3_oezcXREsSSLn*c^6a`mjwN&=fK7={rj9_f-B(58FTZ2@CXD?3|7SzanSWC7fPiO z^Yr)}SxcHpaGpd(MwQ-|RsoZFQd7N{PFhQf6$z(u;xjPp~{}rRGMaf&cc%dQTg+`HWDcw1JGsIu4NVEKmtVWmOiP_cMF71|w;IzEQdy|lV2(95^`u>G%Z&&5KN z--o}AQO0j~m8hB|=r{Rv`-m5*iuuW9e-Ws4GE>usaMk^K`$gjL*Sp61Cl62uiM-Rc zC6m0(;D;ky4#W@j9#nK}#(quhTzLLI^#Oc@X+3yhVoP0Zcfmq=2uQF2y*`5j)M!@; zTm0eFs@r{-p3ED=gd1*xJSgP&;A>HwxSDyw1MhEMWOumJ1wgDg1ZtA5xRYBDOJ^GR zph*eL9bJ0Gdm6QOeF=FHzJpy$ApqRV2MG`KVSOYE$Yfn_r6>l_$Wm({xiwTN&V6$R zy40+wmaeR$DwAo;_O60j?5C#TrXk7F^ee%(7)s1RHM1-8D|~%ttOieKf6HWcl{jxB zM^H;Ck!0>Onbn`%c}O+tBm^;Aj0z|Mng_oh(9)2yTkVD8x4}-UHC(iz9Wf&{B^v}} z&b2Nl`*Uv=q-u?3>M7wFsY;WGWHnU)v97h2M(pqBLFt~>u_a#@#YOj9uh@@V^+n9jujEtf3@^NH+NShaj zRgF4Ddz)}y0>1qpwgJ9p!lYVYgexh0pyjR~zpyXH!61S`kMVbNvTnU2{pY0-ncpoW z@!rmyvX#N;mZn-xrnQvbSttgYbMi2?K7jm^ zvF2u#;KxdnNFTs*(qi;a6)R>vxiDGgxo4Yv^n5TfeA zaX_Zrjv`GqG@kN44lGx#oyoX_YtsG@DyYsYl{D+p#R8d2xzs1gMxcQja&wUaEC@-F?v=~Yr zL~3j!2T1v)m6B=j(+7WzhtvAz1UL^e)z-loY@y0r3J+hnfxkEc6yB9x05q!$mr8_p zg5Svh^Ds!Vn)B%ww(yL6Rn^VWfgyTky566R_(fMO&aqML3iTV=GOZsntvAp3g|j5l=-CE3A?^U79#SZUv4<}j`_k8J3(OSy8qo+0{x1@D zF=)u;-rFQwjaVu#p~fe4n6Ip4W^W8EscS;yTBTk9Z@*cNJkh5WKTnmJ>`$KKlIyI) z_w>@O=2&X<|9=35KzhGSe^E;9P0^hNil3H;P8uR`)x;*O*yG@{J#BI!OG{%I3@3sD zvIK%~3zDGB4gNKN0S+R86N8Z;GiQ0t1;(-V;^e)GB9#G5)wEYA4QABUK*;zgfe1kT z!PLC~xlD3kPd(V?0z340e^TL>2tw*wcT~hWEar{+_rVMhFGKP`f99#Hvk|Zy2#npO zm2Vt!W8egfj_Jtqs9yjEA8p#5{%+}WoP#724r{VB4}!6GXpCc3&gPZ3)y*yQ{T*1& ziB-;#ua(<;TS3U>M4(RQt*i1|UrPbaeQq%g5dIx~HUd`s9(tQgfNNbJh@FBEfvi8c z2Y|cBEpp&`fpOs(e`6a)$u>P|=tp1r)2Dv*=~n8mz@w*+NsJKBbc4s_+Sl)ZM=%;U zCKNPDx#}Rt`EwnNSU2e%i~#=)a4~)r0i)TwE30Y7H|)={HjIe3;<1;Km<|*{4y;%a zeM2i$fhx0t0Ln56141D3xhn2?3i$Yr7w|T`sachR~G5)JH2rYT=hAAklC-6XdXa@Uf4ts%*^}(w501Jcjg!F3x zD2zfW6sm8q5-2p19YFy$_y$~vHNugt!+R)j3@}G1HK5*06YtU3qTl#Foh_>IYYdFGFZNn zxCN;5sDtV+IYWtI=r2RWBC#6ohu)-%K$_{Ner2Q+HMb_oKjLy1eNF0BxR5ctGw`iKZ<1WZGf zJ4%7Xf0Gz+Di$vIMBpPN;n4Hdr&=DRyo#qRi~LARRcMBj8$s?s%B;l64v|V# z00ylri>IQWtXzwz#7YmeFt5M{U_eV3vC5mQf6KZ=H=-2En;Zx*Jb}-!23+ujpTtRn z3x=M=1tS5=y%a6}y9CR&_=a0}1z%#xg?xs{G=*V6sj?U-Q@Dj8<0E~lhi8z?TD%Hq zcm+{-g<+7ru{Z}~Fa_1r%(6I#TQG&x{LHbyMixuRXsjF8OwCwKsj~4D<$FqFcm>Rq ze@%iUnB(-NtT=~ZfX!Pl2Fm=3Z-`FStWL2wP20Q$?UYII3{UYKPx537nk-MTFiP@V z%#>2kaU-??Kr5oein~M$_e=}+w9ovEslGHh^d!$(`J%b|Pq*-gN&g~<>t(odVRs$)hZEs0mqm!@b@ z^pv>^h#01zgzw~v<FDp@Oz#)R#Jk4I%_M zl+?Ttgfegg7sJ#Gtf2j#qstAej>pomH|Z zB~!vx0{sS3qLj4IRbCxdVlCEAe{)r1ZBhZrnq=jh`fOISSX5|*K56xeYMoYW%~ov{ z3S`|@Z~pyOa1B>+9anNKS0wFLFhvb~z=vx!S5Sioc!h^sbys?=S9`rzd^J{drBA!$ z&}IOJrDs99Xll5Oo1>;hSc?OC7wS)gr6 zi5-l008CF{gh@!+HoegA2tuB932ab>L5PF!pjz`7w(!W>@Myst#fD9w+Ckvjg40^9 zrG=**3+=N3-~C(#J$kP z{n)cuAgqD_$aS}@ngY-LP|*;P&ix7k44&%68_=~2(UnQQxPcpRAzLF0e}FNTTVD;p zQmd#3gAkN~xr$-H6M>;Q>pENE-CzFg-=HO2-0H#|8k1Vv(B;+N7tAVYlV07)-W$E% z!0KMCz=pkAw!P6le>V(Yn2Zj?I4b<3CD5w~;M-7F*s`ohGa;FER8-6?h^64#(cSz%p8VI+7?~JG*x`3e^AN+OoKFs_#!U2*gBN} zpu=H2&SO1pSi+500ltg8h}Xf0S9yh3j3qzFqF8va*fNn3TDrAGB*wLM_)QE_Y zMGdQ1S59761MK9ss)y*HkEThD1g2h2wlD<54GGSO2*Fu?&_GYd3Qm?*b*%@2BZ=5h z2Y!$-c2zERe^rrPj*uuW4a&8!U5<`mo-t)>WmJjfl7$hTRb)bjS3Im{j4k9y{ss+d z){BZg2Wy^Ky(lF`#z{h^p>L+-#evrjjtoeylQD(@PohmWmVn^Ys1u92FS=(%5j$5B ziebnE&D&#wE@*?+R>FNDHUK~H^&o|&nYmztEXV>Xe`!PVy#X~m6}^}QCb;Mu$bvQ0 z5=zYig(hDnumxBnUpBCY54H-gT^|UD>6iwB%vy+a1A-^W>6``v=co!F+-b5T4Vj*T zs~rTK9-S%BX(E*l>}_Swk!c7Zos3w6Cjf#LVGvc==}qVgN5$zJgn%g^!kFeO3u}fa zfNBnKe;yRfi0l~aH7Mo*IA&rgE2w?}*bt6pzKS4y0jP$67l?zU7OiiX1pbm%Xo@6gO$i4v^AOvv6EA8v(i)I4FP#i)aX(qsh)$(Y- zc!wO4=*~zX#(rpc))#p`2}>L#Osl91fC94_e-}t<2ti4Kv=NF!K4{}UZsg|Gq5Xyo zo-On%VB8u*9wJ|jiwufQhB4GQ4;o5qFy4$y!!&UWDyihijRU{}Yv$DwA)P9s77haB z0BF-Tb=bePg4_da>X4mp$MWk@KI;P-gzoLXf8z?W`i-%!j^qJwR2eqMYM}9Hw>5}B zf3uQnxZ;5Q?!dUn>#DGg21g+BMmG@};K%R;w*BlUyb|yRDioLmaGw4I^c#Zk7H^iu zt#eptSxY}3;pTJ5LLP$g(~>_cK||eoj2@cpeaT08_=aXmfdcy)gVBIY5QT$Cf@5i6 zOgV}QmUaxPEQ9F0cM0bOtSR_#Emgkk9@pSO;G(frrLWI4Q{s8X^aef7>m` zS?R>yaal_(AlJC#nzbOlmw3)cK%#_g2?4-a6gW)*XlOnHtCXw$2?0@%(k|b2ZtwOS z9oloq0w%};8+dMUA9o*uA?IcSCh&`)TD2Q8jKgRFHZdv`8225@ZW~B7uBh_{ngZcz zHXums>v@3*VO*$nEDq?{(r7_-f1m<`Cmz2xcuR-cK|c==Y#_Cot7@ZMd5Dl#euvxO zfPo`$NCyHDd4VL1Yi!^LS|A>SZ(w#M!sOAnWA;JS$!W7fHn^(d?tO27r|RW6^{^N% z%w__4*Wg!Ap@}9E8_22CrghZP!p=C1EMOCLzy;mXsm6&RjuUAfzmkv6e?mgo40eyT zpt=EeANMYfnq;>LZ9#~8u7@(%mK-3OrC@-D>StiVKK=D}{=g4>!5_7uB@A5H1wz1# zyC8&Q@W~-kccz~NT(AXC(4cRyh7P^~Lf8dd7@-qjbE8TOPau*Zm<-3^_pR`TP2gz= zi4b{^tOtn*b%*Y}&X@je=1VQeJ%elO7C+=SPcmjY;oxp<2-x&C2 zfOsGfEOlu9n%0T}Hw~!jDzNInchIZ<00zC{0AR45^Z=a*ACF^RjSSg@ksSo-afwks zz~Z1@`taH(@E;~D*vaQLrSB=b7y^K>#pncz6Kt#vIdNhLE_jm&f1-#;DA*($1W(*x z1kV~9hz4=lB6Q;-lW*$YsXNCcV!<11JlVp24Uwss@pLTluhD9mEbmPHh7)DaThf^%MV#+waVLxswk7XzMi zP=zljK79OuJRk4`8(N{*AY_Lyh#$Xt{Ny#mM%B7E4$mR1f8kes`cQ1{pFiP8^n3Of z*vMmp-Fv{ObDw{kfcG3W&1CmPGu_R_&mai`Az}V@s_@ZFC+D2>4m`Nv zb^;Ud*m4nPe_0g7kV6omkVR*qDTD$c{@9XGL_?5s**RHUGz4)etyGYh-<)PmGRAdg zra@_TwwF;zDFxv$5k&P0Q1^(Ef&n3zQjb$nrSJ?>{2F}VFV>$O%|%C zqmo*xsi&fvs;aB9+N!Is!Wt`3mH7paCz{n7Nk}0+s_^Ttf6b@}1a+$1r=5k_0RvRR*;VMx z8YKcTf5j6|Oi?iewA`3uyghyk#1m_5;*GeL*^=avO+FbV6Ym5=n<71RPQ zK*6(%X1+by8w!)0V~>-;DRjdm$f`u!4Gjf_XV>cJsT;O@{h8EKLfs-kpqrfG>{009R0&BfWW5?;9Be}^NUxZ;a5-niqByGpBNw^n`iTf8dd z=GZ~OL-l4@2j zf5-kU1gfMAukbzF~|P!k}(~0E<{OP!?L< z0tPV9Km-gRIO;(zh(a795sPTVBO)=0e@ax9SLYMS_7{A_; z@?4c1iaZd+$9mDCMA9MN03iPG0)%qFUmO4iL>EXD>tRoO{emPmV28UVp{J6agn<4s zxgvh7uS94ZDp6PJo#OTJEi&3fx6=5cVd^E95s9D#b@oaE#`1u(^i?nxe~6HaI1m?d zQqnFdhY??1@C3nxs<4Vlnq&4Qncew@Aef0wa6%;kfCC&<7!ZM1$ip9$2!R1K(1?2k zMIQCYL^zen&c{MFvXY%FWh-mh#ke!7X0@6;r>cXcz}N zM!4|9rrf48eBUFhh${iM&1bi^J|I4RPcwoFAUdXP}OL5oeO0=mP5 zsUrb7(|;{UMK91DJ{bA_oP2e10HCzq^)|UvV8~}a){)Op=$Ie5$Ox!|KxppvVjLsY z41y@ZQ!51sO9N_&Xar#dN^I2{F_9@rufY=`3UXGg#YwHgjEix?e`XER8plsZF&kRs z@r)QUAW-!%LR2JhffJ)xZ3$=u+>8+f14O_G>q;yo8f($Z;yA}T-Z77R?BiHvHZSnR zNEC-M&(H2^f*7>MNyJJU-8f6NU=--J?y|7&y`XD8V&i>JQoH?zFOD|`if=Re4)Jlo zM)xJNk5nNKW29!9%D0Rvay+wJam1Q{TgA8ddMCEmF*JSMiWRKgI2m$!hRiUp~n zp&Wn^J{rBoi2magjcWi#r?rJgQQJiw-@t_@y53-xyrk=%xQSH<$m*;Hm`f;N0xk@g zCC;2p? zz$@x;k4&ThDj|4=P#R(5cY{9kq8~l!ORu$Qyrr&L)`tay82Q&(#(0bpgf? zhylN(e|^LT6~g~hl!VulK!DJUn6DtJG`fA8VC!6L!lu~cpEit8wb@&()mP^qZLon! zz(ceI;M@LTOWsKs{c#S}k(7uSj$)y4YJvU7v|lKJ_bI;5W`3E#P_91NBj8DRrL)d*Ecocwv3cgc;L;X#E2s3IpCfE#?` z4Z>pnEXv|6(jt%LU|krP2VjH}I*9>!02ACB2c3gijfO#V6-wNO2mInc8HDP{2`|cp zH*7-uy#Vw<2ZpFa957)?p#?TT8VpiYe^SLoGaz3UE)dDZ&I_o*!7L2E02QJA3v*G& z^+AWsNgSYAR~n*4Gr*%74FcAw!x}<|H3m;8F_rpl!ds9d^QBTbev5K@aT}1B3t+s9<8X3C+=IW}gW$%Q)5!er7#83Ds%ssh%z z%O=FZ8i=Ms5ry<&ro%NRft2QEA|_N}!)2<1jWDKqT;?XIra=zo8swZhw2NShX3ljF zX0{h-il%Nt7pEyza6V@E00X7v<{%s$46@~2j%9UP=U9?MTc%}NfW%*(egPT@1@*FkpdoMu;vDX|V-t zY|e+0mXm;}h_2{fyqP!1e<-%Cm$<|idEjV{lE;hz#d?_;QPik(o{x^&(VMA*jC#YH zy+x2x1&#vIjq<2atQT6~C{UOOZLteJ9O;Zc={v~iJG>c_a+Ex5X{X5pKHQj(R;iRu zN12`jnik1v$f=y3#AzT+i=Kv&h-aFp1z2R7k);G%7zteD>0PL3f1kjp<;BgSQjem_ zh2^yiQ=Di$2ZM|8rebJ^ib|J}sHSSFo-Ru*tyyJyDyX7rs;cU$vg-chaB8c< zYN&{6(O6=v+Ul+1YOd-kWmM{}`s!d9$v}jO0{v>S8tbtltFgXnvNCI37zvbQLA2sz zvr=odTI;pW;<92Zf40_VFa$#~Y-_lR>$s9Dx@l{$%7uy zz1r)&4o1A`si4N=o^j6A~rD2UwbT*ydu zz@RtOY}P_X&N^5Y>;lgsj*5P&U&uov&`lYfTV5zulnw@@+Ql;k0RyA}Hzn=f@@?M| zk;xi~0of$6e+CiNLXy?$#UNPh3#gM*zDmx<Tdd~@A`%c?^4d&i5AeTdunapeT*Q};=5KeXseF(JgxqM&+T5BB1&)FUmQu%# z4zG@O&5b(oQ2cNZLvgzZ2@oqWUmWU)+VBn2X^G+vANabUs494i(g@9_zPawv-|!NTQC^hyh4LH>I(!YfiqT%PASv>hAB zG8=$vovH&a5W+08fyXA`FcN}$PJ%3ZY%H?@BhU$quFpvULGJYMxg?#%E-nxZ!hb9f ze>o0<7ec@jT(fTG*}80kEaxmbsFke_oc)~9eLNWwHE<^MT zlS44i@_9Omk=cSPF9|WrGPuTFTpHWjf5D}B*0P__l@S<05m-I zcmxXEz+YQp=QwQ(41wnsoO3!3g1DTsX%E$QX^bEd6&kIdl$z3NLsH2x1wYV2 zYzx&3%-_u#2QEbIV9&+}c~yhie*k9Z24qVMlY9dcTy~daL9YN5*ZdC~SVS=n!9;A1 zVLxqRM@g-HgR^Y6M_|NYqrfWXa8bA}SrEYAgDFh7-_&>x|^hV2z_8-`#^y1%t-qv zFLlHp6iUaH0iVa!_t%7ue?>CE^g0PgT*Q?xc5T1|IkbjFj;%?cL@QyrgRuc%E;*Cu z207f?A(vDaf?HID8-zftSJMT= zdI)F$6iCi3|7OcAPxov*+2;OvM-T+N$KJY6d7;ON)&O=R z1`A9s%Me(Lv>XiP&_>Mzwv$WbV9&e05B|0>=Q$rMWdcM15!`|?xB=aax_^&F59EO< zyiguLaAcZ*i&G6LYSGMKQ$4P*gavO!{lg`RXtdG7lv1Dw^o+6@Fl&qHH!F5D&%fU)0 zDDVa$Pi$HwJYj>}JB{!tCPabaMixes*bHGq?wlt^f1a{DL@~ru!AZf+G1PbyXU=!> zj2STyA;-IV^85*sz`y_(X!72j8%C*~zkBtf8cUJjf*^kyXT@18$nL=1!&#`LB<7zt8{@|Q_Nf)MBA-HMPv z(6^!&e-dbDa^=gJ$2NwTHazLLhMYM50aLnkoE$3(9D>VVc5ApARu`DXn)GBXKAu2tjfwRv zhW3>lKgz;`(KP(%))+sQzAJXG*Td;%PR166>T5y*E` zNpKK;E*wx573JXvf&`z?a5i)k#Zb|_u zr4W45cvdM6g2QmYoc;?2` z>KUeo!VcSYu^xtrlrfE7xy&?qdPpFFeqaK#;$+;Yu57u|H# zU6dp+C4@_j)UlEgMO)za9d*6|Qy?3Hfloebk|7fXLf&Dd zJA7Z`XOk!eoB}ayG#+5Vj#YVr$OJJ2=EfyKriDbDyqUt}HB{MzW|arP1}AJve|{lY zoxIr*b$8QC?rT~shhqr zh7vc*D8mgmFqJ^7oK$5MHF=(ae<9WjDwn*BJc3|AvctLRb=IE$BG%X&6-yw8TkyI{ z*Jm+1=W}XP$V2kc!AgJ#dEq@C`Q(*fp84jTe;)eirFYl(+UT{H?0oaaPA7N>spvY3 zQ?jUqWXOrnJ!^J>*zQRx-kUXyKi=dfe0F9~o@SUf*<=nF#7Si@U|CMEf3ZLQ@egM= z(bAf=xB+!)iB+ty zlDSl^DgmuR%`#;#b`IMxiSScZ|^VrnNCpX3f&pIXcpy%?4h+`>}R z)8iib*hfG95s-lt+*2Da$Y(zEp=W&wY2W)`BfkWx zA~E;F2R2}q!T$a4CIK8^&J1YEo8@e04ZO+E7#P8!siJ5KvY>zlf8mW*YH*cS`=BF5 z*fkTD=^QB}8=KlQiEs3c5HUQ*K?Kr59tHzUn!KC78cD?M1W|9dF=7IcxGPF9QB$7d zf&pR`fe^s{00udbfLu5 z1{!m`Ko0!b&lC{he^`Xn=teo(QICEUq#+e)att|}>m^c6+5=w};Pyy1AViYqpk%@p zc1iWI?;R=ZlQ&c2$)Y6OF>G+6?n8cWvDl1H|j zATBGIG+stksvuJZ2V>*GED@7yA1UDplUb2vy3Lwail%;WfAI|(R?~*?NJcj6w#~e; zl&2d3=Mcq7IC3WOoK3t(BQ6txG8p3=k*$Xj)?&r8+%q-$i~wXK+gZ;hk)W+XW9$;z zMux(1{&x{IKw~`R+0Tv?wy~A%Y-w9t+uk-@C6&!dRj5+=wDg;b%AUdIx+qIFER%iJ zsl{Bwk3oEFe+@TL$Hx}rT$2Hz8UIoehES=IsXg@_xm1uUno(Z!n)gxpv+8QF8Z@l} ziL1Kw>Hr2I52iujyywLRgluD0`Ji=#Yel9q)x=DJFad2IoD)i@c>*@1CvJ-BD-Zt) z+aHhXf;G;ET%0a9 zu>Ly1htv`T0gd<+WJDcA%wD7prZE*KCjwcBX<~9&?cS$AJf5YRiT03BpqAC2O@Q9 z@J{=~2tG05PZbU`Z-@B8N5RG)l#o;iUI87&WC|Mf_y&oyl1H9gMaA}Pu}+`?g;=>k zb-QvgC2DNO)r=OQiP^DWdhF0Rz9R=};lhv;f3swBpBvrjR`*dNuIC1*$>9 z2%+O5BeLNMVImL&RNW&J7XY@f%U}QrKmk=mNy254ovOWKMks^U$T=vXmJ7UMC|udf zFa(;IycnR3f+sXlj1h#@8e=GSo}FpCfBlY!^2H7FY>YoPB^6VMs@68)f(al-n_`XL!Y&zd=pp_OCZ0*{+(0en zqE0d)1Y)I$l*sV%;2Y{;Fpi>B>M2$7;4(z#7KY&#AnPW2r!(^48Jc1k@_?cmA&)e| z7(yTc+Ab4}VHk!1^=i-1ME~o!RQ7Ro$1zGV_0&4og^R! zazF^xA^|j|9&Q21)*=Ub0CWt1TKWzrjA5RF44_hg3xMS&OhE{PES@NeGa?`lesCw8 z?I+$T1#+MfpyL^c0H9zXe+7=ho%$&VzM%@a?Ft{NPUJz?2mmX}5e7Je04RYC7m^_x z(jgyG_1v&E=;a1Hzyof85oj-)JOIIX3Lb)i5ENw<`|V%g!y>oL5B@#C!Ey=(JYc70 zW92@Cy8g#Pj*kQGYZ2vP5P;zLAm9;qVq_W-WauX+ykQVtz$P;>f2;(A6MrPqWNyAn z@f|jSCryb1Ho^XI!b&^=6}vJIJ>WaoKqXJ0gyzNr%mhnZl3xtWJq}_OaiL#;D2KYT zZs3v^t%D<{tb0JxEx(cxG;lUvA>0gr017h%NQD42r74`y00^@%Wu=SIk=!&w5Q1QF z3R3`5z<79N?;Nu*e+i)NL?Kw@VHnCl2tShn@&G!t@iPgaF+Gz2E?_BAfB@QvSO`D^ z94Y|_ATtIp6D+eeT@x9H0V0djIFA!KlT)K4k~P#J7#M*N2x0yvXb%^nQzC3AHQ?bJ zCV~yJAPd-l5a7}#&H*|l;v1}C5~7nlY{5G9(luxS2wva?e}Di8fMF1x;0b`Bm$<@2 z;&VQMAZpftKG%S%x(^!$VG8gwKM7Riut7lOQ$Fc)3UJBA@&`c`^ap-OYq7HWbvNRtbUfC!8L z3Nqmq+<*&2p~INM6vA{&p}-BG;Y{7&7F_2Z!mtduKu>%A6DI^C51@bu-joaS;1*7- zPoW`l^pp$e6eqqR4-^$r6}1d{QxLeI5rPdG8evd1f8rVPv``Vp82&U+$#hUZRXI-; zRZ~?}SCw3tlQps^F(Ni%)sbUAmSZ&{VsBz({zaDb5|&~?g)hj0Ra@3&UlwL# z);Y5BTV5z_1hzONk2Q3*Hzw_Pd^TO`5U!rde@bwcZyW_)c=lQCEI5>wU7S{CqgHCC zmTDo=Rjc;MgqDQ>6KlKHYrht3!xmMqR%~g@-ty3z%2sXHmTlYCZI6s>-xj3otp(U) z60lQl^Hy*7mT&tudE(Y@H7Xc@4i~(}Xag5<6IXE;mvLbOa2vOe{-tjqmvSrDaxa%} ze;*ffH4geb<+Lf7cLw+ZTT0SAOT$$ljNJ?-zgbSAPM?e)rdZ z{}+G*SY7&8fD71w4;X=egMbs5fg9L?A2=JQ7lJ34f-87}7ubR`Sc5lMd@-1VKNy5V zIC(u-giF|jPndQ|7=>4ugJz!&r>RcwE4k zjL#U2(>OTHSdH7*jo;XDg&2iu)Ll2bqwESdc|3 zFhE0N3z?A{*>ey1qx|6;OhFLxe?Sm=&5<)%lQ$QVJE|TsfeQ|@RK{(SOWBkcx05*v z8d9J!BR~VZ@g7cDmSEOfHOo5?-P4XxW#4Ic#m2km?~1m}pi=PmY0EnU^^^ zg?W(fL2(Sg4U(CetJ#_z(wPOR9=Kp}rumw`8Jx$Kn$O}>D)wSoBeKpSe^dC{~o>e;w|@6ksf-BR~QMIt1k98BZsQWLj3haW={Vs-xO3ZsVlA znWe89tJ`gKaiuW0KoA6Bco-lBl2Bxexl~AFF1X+-k|5ahNiK+>5ukvbVnvJ);~cnv zTG~z!8len;EC`$$HtOLOGCBzg+prHi35Y-taKj&?|5VEnu4`Ebs210W>=0HlX1ZOhLmpoWnP~mtVtCLm&h= zg8{g}RP`IhQ+$v{*B^#~3#c&#Vmt*R1G){|0E|F2o`IENe?Sys`6h-TjO67U;u3$2DRY*1UR1?(K1ZCqMM1i7so44KjHtylWh}+u-p5RG}nkiwP@_;hzBB&vK%+;OUH3GYn z-~?lvvI<(rPvb_2`7-|QJvK;_bS6jSM}BfX+cv8D$_rlQS3Wf~7grPt)8)EU&OP1N zJvCAre=8LJQv^YZ*gU>Zufurf@AyTQ2;-Te9!vpKf`HYBK@>=3rps_$?pfSZ{^{f1+ecS%=!l@R-oPPV zH3WN(z$h%jUMOCn@9Aix+n$>Z;M(P0@fRO8e@6El@?f5*=mPQ}69!t$4LsOg%orS7 zv~NM~G?OcyC^EjtGs+uX5?U`(pig;qMYBmQKisg z{X14>B~?oM6*9s8_;2By+l4fRaG>Md4f5ReqrcX77oK^;U=!o7d6p>D zN~=8LrJwv~y_4&KuUkct8{sIji1(GpzbPRPC?^wGzx?lir91hiHQ<$l01CzsAQB`9 z0_~l_g9sBUT*$DY!-o*%z550Y+Bb-kA*?GGPnyOg{(R@pO#}i45->23G|8}`D1}0k zkpmWg%Aio?-O9toNRJ#jlH@^awyxc~c=PJr%eSxJzkmY^9=w$Lhy0z=quX~(R6d(Q6XQi4|r-n+r_-F1$) z&c40-_weI;Wd7G1ru*J56T1)V`+ZCWp+pQUusDO#IncoJA13`BB+Y$zM0lS)IGv*) zCd7m%--aA^=;4RL(btzfuuNbgF_zhbhXW3jprK8~nCL)*CfW1D1PlJ(!g6ql`biNRvIzlwcwrN`ffmlvGw}C1E1=72}C1;t^z&GdTk!jXY|BBOWyp z%t?Q>&}JC3xEmo4H! z%%lb$grGF%Kq}`=bWUJrpP+^+>ZqiDhV z1Do2^>8GUzE9|hu=6R}KsXnxiJ?GeC&$0+T3vENv?zj)M%c|4tKGaV8Y@-TYOX0Wg zked*zt~OMUKkVLvl9(ovd9JwQPOFf#^n!aYw9=MqFTS?oajm!u>1t%J3z6%8tiAr~ zyD!7<{Ca1x6jyBV#mXIm>Yl;}=u5HyW&;YJyPybG&B7UF@k$jvHgp%xKFsD-=;Ws3DV zE#r(we(GRivKCHG@Wan0qZ#ght_UZzPs=6Xj0w+POX~1JY{LSs#25!I_~3-wrZHX| zLygIzEdE#m9?m$6iHSe#ZIG9@$}@SZj5`ixLg%23W8-QId~T7q;^8FeKS*;f=IJU_ zk1>MmZl-KL+0Z#ouHAWy4c=IM@v$3ZFs~DTtGGJpC1|1H zjV0X8&^g>=@^{+67HY_k?;VE{VWQJHanuKoHZmhG4CA!d|Ve*S3;hl5lBD4gfqP8 zLnXZlNNQBj$FLZ`wi#`9Mo>xwa(5DFoNZQJE7Hjd@s_yE{-rKmLPfhc zYi}e>Wc(m@6+h zj2_7|rNMdifV3ISz}8!cpro<-b?f&rXk`9J4l80%=%F zQ{e#{kmNY2ewM7HZZfDm-2@VO7^4g+fB_7WP?K++V~&%5l7{nk8m)R(5=UBq2Dlt% zqbIqMA=8Un%?$Vhhg!;cQB`U;_u&k*bybLUvYbZO%GS27m7{-w-TqYB*i|A?&LE2t zUjJ^jkKR%4gjqdLi702XoYl{h3_Ms8O@bg}I5t2uY2?pvm^P-m$*JEXpIY6t)T@ee zhv70HD=oEuxgSI>p4q(LIlSS*#k%#iu#N34;o29Hs%@%p+nM}YwOV`*t$zgc>#YDg z%6C}kdC2WmGflhMkI3V>FzO%ho@d!1VYY>>f}}&LdOyz&RI62Wz%aWuSkuiz5vAsIJ0)ZQVh#8dLsGZEE&ht}M%P%pf7~ zu-#4Ab?@0K*HWvq$|~-;@(Q~0IwPynSP_t*F}WN;HohoMv5NltG~d01(7uUns6FcI zDKEVc55*O(2?w0x!A3Qp2o>m%VChQ$n(&`imGGG=DPxFmx4LrEaI`o~UdlzzjBzy6 z3rp62-xYiL%U}jki}hkxwY`qLEA`}!ex^3jX@OfhCN5}?+p6|Hq?tK9Nn;)RjG5T> z2OCcEy(SF05@Rhr4TFzzc6L@H?RaaG%ud(cCS*)rd|S=}3CibX)ZUPH?BUi+F|dvoobdu>v7RQO&Z6hf3y-C^&&#Wka3M3IoaGOn3yB8r5v8n>U{ zf1iIp=R7{2a~|is-_KY39dVEKFOR+hMZ?}*I=ovx-YeMcH^Y#kvWwh%^eO!TnsvO4 z7A#@Vb`v)Ghe8ewd3|S4(Cr1L(MOeZ{muq|CR`KDmKxW(Ks#m_c50M{yl=x^EY#IhGHAp#v~1QF;Hm4?)m#}8yC4igC-EP8J>6~4bH z;}U6Kc>Kw(;_rlNZTS66c}l?JPM-%Gd`_v4;C{Hq#ozz_0Ycc`#%SbrsyQmF{BpZ+ zQX<3^c%e}^_)m#wQZE``iuMr^f|erLx5Ll}Om*j48WkR(&fZHQsImz&{U!;KbI1&n zWK_B-W1t&zfY8%pBz}r)*n9ytXvC04PB=m{y5Yu-7=_S*q5iDE+(`45NC8D6EyR;$ z8O>zlDDN07Hz6!F7tk~b<*36Bn#R;Yxg z+=SM_tBoxQ9j6JMs6>W|gl@;g{*c5{)5O7+#F4o~!kedwW2mG_jU);zX*winE;p&( zBWXUCuP_%8iba$iGL@#%C9TFU_&`3_$^gTBUwP zr4yohl674Tne8HkCloks@8>WR>NCPDbY;<8TNM>iIMMo?Ge>fT`3f+x(2V zAB(vCCfnzur!s41zg2eKYPRYc%NAju-s6k|CPgol3J_XN5$4I|Hb~b=(-2ux)rw#@ zJXD;oV4K8(iKd9XFxP#OU0v6{CcLT{D|X-4`+J#2inVgWP8)0h%WHu`qkJYWDm$cP7tRkM`*?NFX7UZ3SDvibXu*Nv%Z` zsIJ-K@Bm&c;#w#H*^h-tVM=kTrTRvYN+Pn+7KTCE>2H?$0dlwe8TkSDi0a$fsLBW20RZgIiI-^6`btLM)|dAmcveFtnmuzgWw)SSa8w zGZB$^i(RTuMh+o*+|1f;tFJ#O5^7J$dy{v*m*E4}pfe0C>kb*!<%4U|hgAm%sO zig0yK?&xKWff6S5@Yb9P+saD}6)tqNd@#WcifA*E-(3}JAT_4+k zyjWdN!|*b{MdCtP_J7qgr$Nr+96k0k2=oVxp{aCr`%D{J+Rx@ntKU>xA?=*-j4*A+ z_95svn)O^dHAdPc+%Cibp25$DSj(>em7wAQqwzBpL$sZ#7P}J?B!z|8g_f$pFh*B(~N}BBi&q$_b`BCAcaTJA8 z@6~NszV@4bT57qw8KQI*#+?e_2DEyZA^MSGhx?34^(r2#uuNhJogyRU5<(Xl=!9?0 z#M3L8W_z5ytu8>Wq#`JUg;ozrSzJ?O?}Dp#mK!wMc!{H#@%0lGiLM%^#Ccfja`v@?-m_vmaY z1uz*SDy)Cezmft5U3*3EBjym!evLuR>r*(BgS>}Ws1G?LIGCVf^XV})#@jeZFS%Fi zEKNT*C)Cs~Px-aX_ZZF41(Oe89lb4awbuc1hx!neOE#av_2?|t=aKr)fkMPgQEQe` zJcMR~13=I~C(50GXF^8a5Z+XPX(9(39`f)PGebU@8`*2#1<*wFvZ3KLNtOi|lnoCs zJKYd5p^ z6KEmKrSa4-S)k1#_=`5{-G|z*h5$(C30*vWh)#uY`*Pcsq-O3T1J;b>+L#GpAd zu~k3;kZ0l*gb(r@)AW>m#qY|vTtLBQf&4nlR;X~xd=MFMjo~-*IB`hpeX8yy$ayeV z%vUz99ot5y_ZY>Q){*xei*avfyS*-klPh{zPu;tvCa81H$w);nT zo)wH=p#P598(yVG3?9II9hMZZKz4#PhcNe0;h9u;=LoaxwAkMUug9x z+y2F{_Rfml`KNS;9Y0EV_mPj?<8mi+$@{ncQ0LKE!7)LP{lviAT#K!g2d>J`RTLm*FVz_QQ-2F5o8`Q!OCM+;s=SCM@f z0}EfK=C>UDwS}YAe(>AR6++rO>4?bd%Z-;Rbp4rekNQ2Z{eGP#w;nS)0j4k*MtZ&u z{zt%)5abckbWgJ#Wqm~-3guQ@JJ&780d_UHbn3h_3=OM!7Mb9-_~nICpTd%t7HjGO zYwbP}dG%`ov^7GTynqmm9m1_U&?u=qt*Eyk@P4!Td6}-QnEJRmTebRJe_zk zot;^rwf$sl|553N0>jPIKq2!xrr-JB(b%b(LSGJa$82}o?nOB9bpP4i}@>7)M^)2F-R~ zztMb+U~Bm>v$FX-df|cIL%yzSb-q9U($`si8T06et-=)6ZKbHb-+nQ>OTjNNoPPBE zE*<|};Nwghe_zBjoOt1Mr+pgs9R{zmj_zg-Nsm^vl09I^q< z=!BZSNgeV4m&@|$7&UiFimG(+pDUTcRsjBIgvT?*JULrvj>CQWYFRtMU)l8vv&uq0 z)l3oQ2g_zFe>(g&BS?NBO=@es+SUXeuNvLAf8iZxGYFWNF|yh#td)u`G&4JFJnvfgRvwaQ}c3ryua10F1WGmut+*qUy#7{ zNKNO1B$N^5Az2iJ)zHl!S4mQqrWsLe-4{1l$~(sJ{*@6TvPbwyQZTUzXd?xdV+HJ z#qqy@q&S%^YVoBu=D!9$25v=!PU*|{FBBn|n~?X9dvFMy-_qs`mus4ImrEqtqQ3!B zR1&r@k7kt$)-`pCiw)@h@ml(ga+jEzqTxTZy*$_q2b+$ZQpga6^3DqxT`~zr78R!0 zh&IB*Ller0N~b~>L;+b{8Y4WJER(3RPA-eTNbg;4VWDp(mhD8YrXV>iFpW`guJWC)2#KrWmCppRrrRJ2S~xTk=`@dF=_W zY@NAGU9Zw5x#o?RmlWPXlr0qh>IaS-ju8T2>~dB={6IRwI&!MDPjwETR6UG~MWt4~K&W45AsD26 z{kUJAdG+>}3OF0l4TXQVLxA&K7SdX z`kWRv4A?OZ7@;}!5$I_xvj{A}& z%?oUgB*qYSzvFVLc+CO9oRg1JK*C4aLt6%K(_=HM6(V~r#3B()pS`h5`q;>Z4;}Ex zk;zJKjj!pE|L<;V!75!v-PQd1d#?p+?0z0fJw(U2`c& z)f|`~Hr>y6w^-0Hd#ymp?N~pxDCxDqZD*y;dH%#V65rbpK?Y@lvKzw}%mWDkhHr#T zS8n4n{%)Npq8ZEqP@m{Onp{yB21P;T;C!nd>1AC2p;pJz@~_+FY8R8x4= z`secH_f3%xow;`N1os($yPdI_1oZxcuRQ%w6reb6P=rp4)i8ngkTG?4z*h)v)v+fO z&rxs40e(BQocv1qwd`~KQQpF64V4!x?Ke@1ah-Bs4>AZd5o8##OXljJjQLOWz5ZNR z!XZ*iGSeGnXx8=Op_`VF$G*f5^Ye7&nbZs7=l#*4&byC6Op=L`P)RKdLYJ3B34oc3 zdZld4#+FGpDKMVsX?T-&ro~tdG#=u+!J$&vz;d`-rW_-Dt&pFR|J>TpgYxE@!7hsS za%Vy|oHWfe=q}ErH94+uh@WEloU3vd83-EwnBL+Oa2~ZEUxEG_VlqAJg zZw*!{#n2*Q+|lXS80H3Y?Fi1b@*{gmCmKN;DKSzyDB{RW6ZQmBjVE9xw4&qbr+U!a zR#C&*>Yy1dy-u)fRavr09HaV-H*)|Um#FPqp+@2PXniw6cLg0NdmNirS!r%RomAm9 zXU(km)xD#Ua64XQdCpeiqXX-gxU6Trlp@{o66gmn66!`qbbe~`~* zE`?KYPqkn(}KXvgikzoHdnBN=^@w~-Z zo#!yXc9G{tuE%4<%Op?PLbWyHI=folv|t~prvh{~_MZ+|@^e&P@#gjG!ShDo+OD#& z&w`;?7I1AYBx5zu{<8|ex1=*l(6AE#3BVJA5BIkqI_IuYPJA? zqV?Y^}Ox@hTF56pHF-YdQd8 zN{pk)(l;26eX;Zee0K7tzsuYMr&I5oI?sm*o&nQVkc_{lGUQ2*(s%gDMBKk#K{5j< zip@;|J6TtBf8S`WKw47ZBT%p?K0G4kOp#QSaU)8UNgmExjSFerifN1$l;pKWR3 z_*+jq4NpmA%(WTck(|LY8ZyexycqJVmqRj?ngUr>&2=DF{4@w1ZW7YKN}}&8n7?%U zQnGqC0luz_+izE5cetRWzj!tt^Y~}Q_x_FFnWzx>z2k#brf|%N^&R>jPAhAS`!PK1 zA55OzA5^&I^>AGH7)u}=|NEMAZ#M$9S*$2KtHK-qQ17?fvW#L`(Mf`y;1v1k*LT%t z?|6E^_VkR>S}IH;jL|KUrKTYB!AL;M;rxd|;%N^lU<2vB`6_r`V;>rdQ;U7&~;r126Nol1a*wm>CKRiax81zV-k zG2}b%fx`7KhNTer8^WR^V6RWz@{z9{sLa2~DyA|Ku4wL16ij`;TYdvntIdF01<9if z{6Deu{)*TPrzS8)SMC3+LmjR%!s zEdb`i4vXnt!sU(LE9H!FKcq@O(y5e4Pr_U4=40&a`c4*Nl7*rRO3LgA*&W>d^c4gxGRO*d+QTxiT98Y}4EUSOIV!avgQ~b36GCGf3KO#n+?C*r?VXd+C`MQOD7HpG2AP_j1Dn+4OVj?qlOb0 zjo%XaJ9`7qgnmnQ<+;Cq9rT{o?9-4Rd1;XHvbgb5LJ!!b;8JY(o0u+fC$@G`)~L&O zlt(2gFNxWA;)vrP-}eWN3`ANir@bFEOneoX{GJe$+*Up^I?X7Z4tPCvY4HEvED=rw z8gj62q!SrJU}Hg}A0kKRA`^!)N8g>IbUH^Dr$?6_zMTI#+M_`?@%2(a{EAyZ?$TlZ zirv_z``Bjr=;!i;7inW(%CCHh9NXy~+npZ!wsEB-Y-|rQzRxj!AUb}iVi|ij_QP)c z$el2L95j9sIsP+p{H%QZSJU|K-toN5@q)m?KR?I+L8t&uDsU$Eue|<52x&jT=`q0-Jkig8 z9?o4Gafv8uI>g;K!9O!0uxZW7H}NlzP-(0TDA=A5R-F_vniS2lpch~uY|DX7L1I~x zQWcZZuO^31Cqf@iO14kRA5AJir!W<HEo(TZRV^&P?F;j zIaS{5#@3ciUpbn#gwCur*d0?Kh=lWuasUcGWA8C@HFyToFddG9*=nZJhtJ&Tn{k?% z>8rPsK5S%w8(Py$+nmK_-Bf39zJeRKNBws*)%R@-pV6#W)~t7h9o9DzNfhcYnf2YA z^}9T+_Goh8$83Puhd_H<|3_JbW#tdS9v|-8+XO|~JjnVGTH&B3XXm0g6*lwX(dGx> zs7+`Ko%;3#jc*^ws&f&}^8hC`=sbmU zUgg)v64m)Kqw5J$^W`4%mBI7!*XEyQ%~w~ys$reYx}YoWDop>6ZS+nI%qqlMtjg?F5bT~B5^ z#TI*v7M)ZVdp#EW|4#P>FAfs2oD8EDhh8nde=;@Pw>Y}#gqc|!gDz1YeW>MJnp9m< z{<}D3zcd4#nD$uu5VZsgUYe^|n(rI`_-bi!#`(B+Y3XQb1vcA2a%z8lKWpXJtN)6^R({W{{9Q`_v$^sQx;nhS0uWyXeZCGhUWMHM@7K|2 zf!HqifJl#r1*O;Yog!>&k4I={M(T)0QU}mjw1f`y4%w0{_pO3i4(%v-b-a~^jEFp zG63qBGLLnTCT(QA4S?7#F}*52nofk9l1r~wwoIh}yQ#<>7JNKFlCoBi+YBuIB!@tQ zw@Es;7~E*TUU6)H%02s}O3s)2q}1)0|BClkt$$__Pf9}z9hgLtEZVreoM6s1Xz>^( zpIcup(gcT-jFFp!2c{&33c2gQ;#?&?9nt|P?76O8;!2TdE$~PPf$1a?Z_K8%3j-Q! zwp`b{e3o2nx<%9uVUD@davHa~90e~)d)<1RxfXJZ^&CD%>EG%1dT+J%;r-`N6B{Sn zBny89wuLGV?3OzxTGM4D^G|eeGC=+urEZC`Ad>Zeb4w7uU65Q(2!HLNXl;mF#WVw2 zRHI23JIXvf8Gud3`xX0iex5VIF6d4R9_Gla`h$>(*KZG7&#bFG*G)I z;nlg-NN}q595r4EALuu$dMj1k89Q+8epS}>t0ZPda`Nr!N+at+Bc~ep4Ow4nyL`_@ zX7`7A4=6^HS9ouV=5K2B6+7NO_>H(}7xS?;nO}t2=-2LKvX`+q!}L_vtLd*Bw1f*s zbT?PY)*>Kv6b!fRMalJ(N58U>0b6^OPzL=ZsRa6yVF>qF;w$DI--R?9f@G1~&tJ78 zO%|H=sntl{@r;#<#_s+5=u>ZD?VL`RFRnVP3=`h=jz5id@CSF}U3}eM;5BaXIMKm9ix6n3+U?J0MSJYB;2_PzRpv!Qtlsdb{3UiBNj*6< zVT1PeGLI*A51xB}@vCFYtq;bipd9V5iofHk#r-g35NYK0 zX3;t;(=>X&@#*gxaI1TK6&-GyL_%tNO2>c1XzMU(t6d=@CEw5PCZuf)5-L6XTr3(W zUH>I)=8~fw5TqM}ahGb_*x9&o?m`3*DmC)s?)`j`XX*uG`hCN1(* zI-o%o>O`q&B7>`@eP5kj$?VD}tvqFi)Wj@ZB?b z@^fBCxYpe&b~9mR{6Ed}RPN7SNrkpv34A=Kwq8$UlsxG)7hJY)Y_%lFv=WZXVmsG6 zaADh$7xv-=8CcG0WCj8`1g(SbSG-JotM``r<2a^}s>(M+>T@Ihkk8W}X+*YMXwRtl z`7oNHEHc96G}5!q0ezQi{oE3cCj3NJedkm4q_n8ZM59VoxCk(2Ts*1 z3-XA^RK2J2;aZVMVbed#Q-ogCc4a`v-t`D&R^X(Ei`xGAA+OSdilqEtxvQt8dLf$M z^<1M5+PBH+gQWMNo;~f2Q1e`}(~^JaV7A-El`Lmnnst+iRZr-8dc>CFF87u2JZs?y;;q zan^ILy_y(3tsMKX5-v>--0z5r9Ek|Lsa!Mb|s(TT*M`F@SsHS`?@3_9d^Q_DNF{P&;sEGsZdDcExPYYMMb6BlK zxWvPGi{<#RI{!Bx#~z<1bw5u97<+-8KeMyxyFhp3_kG4*k;|?$eJ_wiCKZ%`5+^}i zKSrF63D&Sd6oY@pQUw*|^qNvts`p(g84xND?|J_aA1l;)aJZZCVpf14JzFD~G{kV* z0tAMIE-(FfP;23c@M9ZRKKlAZk8t$$EIeDdrt04Juei8WK))Q}@GH+0@+$a4(UFB1 zY3)?eeL!khbXn>A(r3PZ%ZabcIv8{e-ngSjDR1|jk(|4ELwc4&M4er*zJ z$Ywe?HGi{RM@iL+ZFd|+NUUd#7SoZCu74LE^fxcQ6WxmmJxn~pAW4HRp(nhg%d4$OxQT@J$@6@%D(}^~`dlsjyfaJK1u%Y3?yE{ihf?%aW2%|0V3bKnd|}AyI-4} z1)VY#r7E(ryNn1TH?BwL-71vYu;d98zdt79dVYqFjoAlH}jcbr#FLduwrHj&amfPH9o5cX}Gi^=%BUbFLjiNlgJ^mAfB&3KZY# zD37WxJX9Xj+-XxD*U+BNc>Wn@u;5{ZqHr0mFdD@1WgTO-H5V$-zT)vkH_gkMq_~#C zoWq^Dp6fUhvOnIE{4G)l#3X)={?CK)(J|D!C+rIKW$;lRFqE$o%Y(#$dnV#qGx@w{WYWNU3Bl3d->i!yvg%rNbloN>Z0CDF|D*yWO{N2Uh-`@Va_~)SPpRy&6fvlL-k zj{BcSL|ngVzOyUVC#A*W@w+pUtTTE0K*l_wkmZeYVDuHrA*U#s28_Pp&l9 zJAxNEyG6BMHwdl#NY&kxnqZ?y;_q-2Tki?cGmRLgMgEaAHISZ6OK&O{m&*{kM>aL; zh)z4CL|O#BF?2_OZs+RYK}#HOGV=+&c~(S{XoR%g?IMXxb(}{%TY9%(YbC4ZLuu%d zzWPo%TARM)24-hE2O`=fCMSDA%q@iHKdG?`X7J1Ky`0p$P7K;-Yp%%0gO|%5Hr`O; zkkDjZAG!C-^CDJ28XxYTaNirka_soj7Y%wTZ_~^7xL|J2_zNR}JP`sCyNM&@NGNiN z{S?)5E7M=BvI=JdH|**R8C?_44AVH$da5hsb5myF=jVSahgRw|)4hWb@I5J9a8Hxw#@5k3Jc55+W zsP;cn?OP~vboGa^+l)ELUPZ!>t1G6X(8GO0uT0~>yRQmO;*g}#U|Y#xUAMsgGbyjU zJv|SipF6$bT(rhjiM>+cn9gRUV9{RuKZ{%sy0E&D{D8)>i1YnSL8=g^Mx#=9?;T$o z6Hay`LDeLz)7|Hv1b(y(T%;+cW+*x5K;-C_S_hO5HeUFOW09^R7GeZMRLx&e3Zo)~ z8W_56A-{o!yU>(!BT^%D7%%%2ntMEHS{qgxQIsz-D34Y|q&GlM6358|3=_lph!U>mFa;q=MD zDgv4QcP0FcG0}K0<>Ft|av2&bnql;a8GH2h3!vG|)IE|eI1_?vDcdp$bnscI*)9re zFo|bczZU+8&=TVI^VO{a2QtSaLt$*7*>tE%X?h`;-FbJy?ATTGp_HLvi$NIC| zVxjtMR&AZV?zB(21dRrDwF28@N^|Re&#fp4a z-P+D9+4*sSBdKld{&MFai=CloVTAEMoO~Vw?v)t#7hr}p_?hbL@k9XkH@|lw~`81j(U5Lv`i%f)FAQ5{3M8GnyE$`#Baph{Q5WuKmlTzDP7##HNcb z4gH&oX&j^~mzzI|Bnr;PP#14pS=6UD6c)Q?LOX|x7!8HF2dU~7PL38C-~Ogd>KZ}! zOzC0x22HN#uCcDcxrx}sbJ(Ka%8)+ZkUPs68|TO)khy1-sr#n%w)0$aXGQ~o2O-xn zCaW|kE!+#j z8%zF7{Pb^d*}|IpJLgm>f)Jzq>?YFc8q?4xj>=CHnO)N=3WX2RGpE-n~|8@P#)Jm2F!)UcA10FV>JC)@Q{sP$PDOelw9V zN)O{m-{h$Fxx!^n?7!p9{GS_7WA%ANNvv2LkB$jXOV`@BKA9FE$NzmkoM7il`NmBL zcx8D@mkh>=#iBgtVFdnaJf9Wlo+K4xLEn0mEagw)Q|W$+Ub(3+&e1|5Ef(kn6*skU z*A*IwoU49@l>|OU198#H%0F~s{#|$Vf!whPczS*Hza}E(mjrddNYy+m9fPSFhKiQY z*EdObxhh{6KD}<%mw+el(#k2jA*=65h78%1tg0T(CW_I+C(6CdVK002L-0}c=Zm)) z!^K`yL!^xwi}>7@`6{V@vDcN;E1b^zAn4cQBU~gq;eDEbjO&i&|2>wB zjjoDKmUPw>w)Emwd1&z1g095O7OH<$X!@ ze5i^JI5DhBA5IVUCyBg{E-s940!SHZc*l1`uk}I=_p6eQJBe>@(H?oXgkDNXmdbu? zUQCGzBA80qV`Eh`42Iw?zNzrY!t51Y1oWPrcAx&`Q6RzWWslz>V$sXb8~z%G3#xwOp4@)fMx6HNF$|yjxzjM@`l@>A9^6 zq4?kKruXFcnH3JW>JFl%zX{eI$}9X(yZaw~-4ElspH~!)y19N{RXDy?ckHL| z+ezW%vBGI|-6^fq?l8ITtVrQkWnFGa-LKbmzdIEEw3?muEBu|U`@8&ocfF3br|@r5 z;rFoufUX`;y!Vf}9(Vx*GVlXRV8H721n|~3kX}8+5(9nb198B>JnLcMU!iyF=?ED3 ze>dr3Fo>Lb#GM6raXozm7DxuZj@-1BzKy4FW3xuNnfrds*f8Ca8oMB!)(mF~3x4dzKSo zP{90B8B;*JTnHBCRuVLMDY}v^`1|ryDo=i=ewetVVeixZ!Q>m+h`C2fkJlqrAmj5_ zUP{*mn%ciC_%p1bdSKUfNz%m7iZEP}FnG{w=n-LM)(#g-Y^ zuFySYoq@)YkRPCLnX;GqG~O!e7dFC<13p0G$TU;VA!QtQ(5T}#&4&>kJOpny)d63Q zFWHh!ikLgaoh$E1%6wam-vB-AvCJdvh&WzbrWl^rbM2j2E{TC`$T&cRqXp* ztmhBAtyeC#(B5ToQN`sK{y>wN4QCXVrKPLH5Wv_HUz)_prwe#p5N>S{cjLdAi;DPN({URGsPliy39Cmg1FwB#`Uowv;GD7Epxa25 zMKa&Uoc3ZL)QbGs*#*8x)`^CSVXOUDU&D04k7|mpsFdHs<+@wJye>t|vw=k}n0iY@ zs?ey{g$5XmO&jf+)H6_z7H&|q@zDnE2jj7KWK?j=xa$g z0ik##M%DFq2f{Z_{@YbA+*~jXl)1|maz%1fX1XP5B_yeUteD$m-NLSAVAyW@MCG(P zVgaJ)B$@tRLfof0l+co)_lC68D8Myyc~1SYx;2xuLjqr_I3d6$)cUpCNK|ghS#E^d z4mQNsaGFGh3*49k_gixXyu9;AIrGDtC*ONpZZl^sK;l20V1i7te>4M!3a*eeEi<9C zyyI<^c;WnaVD6y`Sz)iVGL%!lhu!CS`-!tBceS3rud_mR5#KOXAD>MCwGt#wC=N;b zIuW|RdzHO>MF_pEHREA5??6*(MrA24rJ^e$0DWm0O7#k%5_iI$a=(3&a_67V*o)#2 z*0VR&?;^6xeodoOy&v1K$irAA-9-Jgo1bFgjwg0iV6~G*lWhUn+^15@8J4#~q8mq0 z4RAr!+ctyH*#DNt<-+t+e-%F+t$y1A^umfbG9~99U6Cai-?TarBmcHjX)W)6sw?)% zPUWo4QYSn`=C^k=x_4)NBfK$3>z#o%q1a$j(VKVddEy1mL$OM}8lCseI^6C^`*K<- z7{&EoLrEaiz+S)2j(>A!U!(6|n~uS*Vi%VE^9}JtXyen<4!W>}i1>dYt~4wCYbP6Z zmB}AYy+)b|*WnkdA6R;Ym}t0vd1$xa_QN!y7@$Ob`slm~Psd%q(izLM*ez!TVzuKP zkonta65c8#Kq#NVBrXh%TbyIo$02wVRGc0OZHyV5zh(rzWx0#*HmKI!yrm7qU{7HyLw%0Xi?Gq)Z2%jTe}p(k0>D6#7etWJsOfIS0NOdb_ zY5Z3>um#X>(|;t|?BdZ~64?hW7aR>0A0rAV4+rDEB=e^|u*ahzV~@Xi{zK73f@W{& zw_XW~c^Gk8XP-42ecH(WDdb&%#Lv^R6p?W7pR)El8iDhV(XlE)SLclXfM1T^dG{|| z5sS)@J4wdlBIsC!b(;5`^kZ0<8>q6~)~{t~eW~T);(hXr{ZHWk^_8+T{qE z!FUK?IZ&Bi)@3GX;`XhX-nLIION_I$vToZycWB@e$j`dtQ1@R~G^cj9&$UL+iAwvo z*`KezzPr|+rJdvF{Pw}Y=b5+7&u(-OfOK3sx&E#_F{~1TyAA<22U3JgvUTzTJ>CcK z{5RW{ch7UY7#G8JS>uP-)KlxHSKIT0?|fjVT_U{8*5#{iBQ*tV&9)aj=$>mQ(OI5- z``8%WohISX;dJo*4I56d?!2G&-p&>zVC`@u?7{byuBXAwCyN;KQc7s#v3X|0mptCS z6os#R_^;IVqTjHjf9hYd?#7`>$VG1qGC4yVK`)3WGsp%GI;qV z3=wh}*x-slunnrR=gYn_y*AF#JuUqP{4bs+&^^65&(AjP#d z3M2E2^erCw?nq@4zeU?KLGv)vE+!hG$R&;lM1|PdOc?R_HVG@g>6RzKhb0dM;MGl& z`i!dO_bTuHVTZ!N!@El2p-ORR5NiJ6Jqy;p$hGXSXTK^Fbwo_pK|<`N-Kgu&4<%TT z8K)CTX3&P|36DJAW2|X7M=!u!uCHlWM)`W3ZsFb=F*=(PRfN64HKXCnLVN_CTe-&i z7_2F(rC_+&7@LTO-lT-W=z%=QUc0KunMwGQ4W3&L2>PI_Yf(H`<((oo4hD)zUk_qt zS|_ViYL0QC-c@YKEq8pU`^2II+W1Ys>zs1fdrJq>S zz-~cqKUR+C2epFZrUWwWt&PRgjikZ|cW88a> z6X`^;nj5n6uJ{{pelmKgEwmGy6tp3t$FBX)Ll}}77y(0_pXnN-=>~>K5EX2kh<*(Fu*EbQNcjfX~>b%I!GM5u-yJ+kB26#_q)b;)MgZkWo@%XcB{A zCsUCUVNjhGizHjzw#j9~vzu*8sKf>cL7dAzM}DnJ)v-G)U?+2biO#yDxM_p6K8&5J zyf(4>@vDx#lD=!hQ+DzRsLmB_`>l((l?tnrIx3no}APO#w zSmvmN{W)U3Iry`n&SH)Az}sA-Aw&Oo#E|9TS9vA~E*p9VE23mZT)nEy4y!g{YVX%J zP0p(EFiI3PGe%_GUIjr`-C5>?flM}*pd>Fo6~8{B1tmRQDE}&RI-n9$eb-hKL#R>V zp`l5FC^fpbGWiMvhQjQ|TrsS1QjAUcX7YLRENM#@xMQiafk~nT$DdG)a12`5U#Nb@ z;wo;Bq*<*`I^QjIHAM%qm?PKfUDri#x>^Y_@E9Qd273b13Kn%*){1$mCC1y1hAABp zaN@nKr9;dMT>B^fNl{i-3tnLY6GW;3bT%2QkAxtjB%#LmI>kSV(}*mgz(WjZ^|i7B z6ABG!)|k+g&j+Y)?|VFUwq@?`C4LLq=8{6Og2Ota_yO{uEZxCJVpe7#Ckh?ZgUII< zoB%;$L7J3r5=&VTk#WLs&F!vQmm$XkOTs+dCF(0$xag%azs!R8ugGsV!CPnZQII$I0v+9^MdHYUsMb zqte#*LKq7PW{QaB?X@dPM8|u!b;H@vSyG_0BnP`UjE|RhmCrlhMYnl)cjDd5%nNuz z{e|FugOHBIGz397QVJ4RL&q_9Db5osGNohTUgkY_Equbt8FOf=YcX+Abu@qWcI(<# z{zIn*BQJfhxGCE_Pui|c6Pi1V$oB?~W>Rr8*MCM;_G6fS5%mT1Ezcj~^MFM@u{r}r z>aRs%<6GZb1IO`OMs~?*Hq9L;H894!3k*4v@Qb?5 zF&w-Qdwp|YZ2|wiyzw>}OJn;G*Zqm`-CeG0%Yn|{>9;TjEEb%#dGnI4RUC|aPmXeV z#B3^i2u-AeF~;qmBaz#yk50z#L2XDH%-bS9-z65`G%3XJL@2fT*g8Zb+?#F+@fsxV z;RK&yh-zu>z99&vg`u2IZ7!t&Q)N|10zw2K4_aggcQ6mS%DEDc<)C5YIO*{c;JUbL z@`^11hycDC!KBJX6gIH)0tXuArIvzVAV{x5A<6W5U`Oc43n#Obl;g{;K6Fu=0jpF|H%LvA&0r@ZJzzmfHkO(UA9mMWX zW-lXu6p&gP@MScGXL=z2D)8$tknR#qP$;48(gOi6BGPEa7&r(Di~tItAh1@DXSVH? zxS$bCVHnVk5ZbB`q$HpSdq+To zOIW%l1$g9KBmiqf1`T1LstCft1dPDGP`Jc@#uRQV2X4V1{uT-Sn6Vip%+FH5zYGG{ z41mZUf=03<36g*rukHE{f)O1NA?kr%8nG4MAwU=)_d3MKF2D_LN)sm~{y?!NZSp2( zLKLfzX0(jUP!RxCv0YfPL(YXjL}C^h5Mpdmtm+{bw{90NuuDz{#)K&Zh`$y~0vvGNGWl)34?PVzRL$&Q1^Q^FHy#Bjj=zh@eWQ>%NW0`F#2BmIlNh1rO(ftNk+fNz^(K|nXCjaR z^l6lkQ_Q54D2LTKuhb2oQ%iA?ORe)#EbvQ7Qu69Wb4W`(iL`$BuO(zDfc`_F0LxXgx7nAEGBCfC77|Mx#_Yr*vD15h5gig_*vURKt}z$5kH6 zwHF`aWYyJVzTtvY&sC{smm=pCgd|R>HdP=KC0njH*A)tgfFtepZaI<&QuZQrZ7#zk z1uBvZ{l}LE6k!Y0FA!MHh6`X z$&eOwb`*zfLjh~)X_qqtr8Xo^XIfq^JEJpeyA)z9Q2C_tN;+g9sKj%}6(jn_a>Ldj z&@B&urGgk}Jj#oRLBf{3j&{dWqj_T5d@@g z9qe!&m0=S!QW=+06Vo7(D5mlRqw2Q|%5wEGa95(T>LkE}`bHqR-~)Fy0+0}i&fykR zbVX5jR?lH)?*&+5%TR3eQn%9pRBTqEK_H>v96r!$$`ChymLe!&Ci8cAkvNH&rFat} ziPE(=%D_;6mNz+@SE!yBB+!J0bN(O`Ou-ahfk(E7DW~&#>CWP4Xg1r8&dh9s<>ZUC z20(!-CNtt5@=5}TAQLE|5xC{Fz?;P4wirMx_i9iDKEJmI|HOC}~y{f^ejT%h> zJtW9gh~RospgS?@Ka6alhQW|r1%VURKo>SJ71x1(85Ck8_#h0)U24gTK1LKAj9MGv z8%zO{TKTa+t%2yHGsh&9TIS0V%LpzG-ww%|`mKc>0vcQ;2d-oYU8RO=ES6!Q5niE- zWl7t*h5?!okim@sW{-ELwqT4c7L~Z1&3TEZ79wuW04UEPzN9=gKm+{Dx0p9-n=_ps zL|=q|NtZ6*0yH2KlqQT>PH6tGmz(?fP7Otw4q9D}VN8@Mo|kDvx(%C#;ah6yp!e#3 z_A9XnE%RnIfyg(5|~2+Ml~7ObQoF(h+eR zf|LjIjT$#%e~@S9x0aYyi$a82*-A(ZU`~F2IG4S~pCfup{=*ph6GkEWO?st&?u%t- zmAl+vOhj6qMFg4)LPB7)pEbZBK{`Z48paafwS>5L?}v2r;E2yTul3qHp6pLDqzs7h zW;MyJxBv0gtniBm|8TRJ;)+ z`EG9E!`tr_8sXAv^TSpA#&7(?X-sP}AqUaNxX|OPL!#qs9K;FY6^iS31c5$(p5eo* z${0j^xj38>oFj#C{#=iPqQtLU6kP3J&?FD0+!&gi$_qloUm=GCjT-$gKDfh}8lmU< zMigFzAf6$Q(XLyHZm1#V984j}-C)V9YI{UM6r|kJatLBH@_LL*5H77qtc1ysH$ zZu&=JTy^+J<}G>sEkS2{W$+>HLD_v+ZOR6=yfJi&{WfqN-GQZJIOZKfZzUst+^eR&K5}Mo zHPd7Eqt+jS-~MTx1e@5}A7tDEo>rz$XQG|u9J7${o!i6rAq)mc$=TH{{^Df=*59cl z45=d@^5a3Cwq(^Q?j+<*o*6fUQH(|&V0i&=uzF|EBD~!vK#(QGXge@I=XKsSH2x+& zgaCj(=!Jgh<2vGt;uZ{lVCa=z=o6sg6XeUMhs>P*A_TfIW=6<&KI^r9GkV@8NLzs1 zC+u@41Ym%d7DVjPUVn}{LEuoxs;LaHUgnMAg6Mne?f&j3yz6b^7DSO&zVGY( zD2%D^0srv*9L`-^P{5kL7)2=h17;& z4~ozx9K^ z_>n*PmEYBmfBBvN`Jo^4As;dfQT1Pf)CON8s-HfI0Ty5(M%Ka^V1XFC!6VMW8Jr>e zSHc^-zx~I5^>4y|8@ituq#^q4|Nhyc`HzBWZsHqW>;eGd*1LfO3mQC#&|tk`NhIVj zmJs2y92Yb|jEIq-Jz^y~Vmg;Gq{xvZL)zQ7&mP7f9#xXGhpFR_k~9-yn$R()7H2MN z`ZS2LCq{d)e&EC;^rzCLOq)7=3N@)pyhAjB}VA>}=Tk$}N4rW;E*nD8+zRG36Mpan3#&06ZzhX6D1j%b;?=BMyM7Hjw(QxoYumn!J2&mKx|^PB#6W@@kY2y_ zV8DQGRl6{M2x!ky-DD*ZgGY;B1qI7{pLsRkcQc-Uh+k?7?#N@0KK=+~kU|bgWK;?E z{wR+>{?tQ>gL&Meq>=|EsboF%R2dPK^3(&RKYS51VR8yZxzv?P-ecvL2SI72lF-EE zp>-f4CgMRsy>pU1dZwfiN(Y^kQiONrxu~&(hx(}zd6)#XXLx%;s$ZUX0{YZB zF?M!;qd|b``RR!%fqG{_hsN}sH;YP|qkcuc3Tv#g&Pr>ow%&T=j`9pcNhXv)kw64F z^y+IS8xnMmF`2kgMza|XrshEMKtl<$&AQQuF(QUpp*-KbLP@wvonuNSqMVzFDc>X{ z&ntOcOUW}qOjn&!*U=f#dYHgMREz!+Gmxf#>`7w~EbIY1kT>kX@(wI5YJtZ`uKq4a z411Uu<$z-rV?wZajymISi=7VS%*G2Rq>{sUK-?lG&eZ9XrZR?iPQ@WdOpKm(+8i<_ z2)}cVzL<2Eal{t?To6$|oTlTgPCpHG)KX7Pb=6bls$e~CRCEDya%lke1tEx1Pr$E# z7!iU21guSf1U)qJO|-Xgp@G_K8*l*^M{+AIgD{BAg;LP;pg{;Bq!2<67ac?i3=G_@ z0T(bt5a*UwUXJ2NYh;jutaE&7NNVuQ3*fbVVGv8VM1C> z=eXAqXTJBYdZV0aK{V;k02NbCG9hGt4>T8hXFJo!E+sRkswvD;^oKbH^U(g#+de_* z0ms<8?>Lo!Q`1SeYIXbWzYl->^3T5?)(6&;#|8RJfLsJ&i%qtHe8UX{NPq+mmx2o< z0B?;jh%s_t00SN_1PasuCHw)V>#_# ziTRWlJdbf^Gvy1Pi1tA{oOlN(Fsl(~teCNp=`M&Pvj-Bn7{wAG1zA0`Oz%#^FQ9?0 zi|5OU`tDeciZB!LwEltC237)Bma z5CH?^;1*>AT9X)%H&f6s4=~7qC^jcR4Zebdf~Z>rk^qfT#w-(pphOv9h(d!vBL#uc zfE$d_h!A+qL-%k43{n7tXi9UMFbF{=JVgwas19ZXc_L0`H?$IHA!l!Y#FrK)ps0Zm zb4keXQxV;Zwq$M8fRoM^#e=;O>i@nkda;RGSlU?OT@VKCR#hbh&A&p}Yv z5iRU1H0VpCOLlk0eG-GBMN~=9=II^OYz%4lah+q}smPS7bfqk9sY`1U!u!=jg-)`Z zl?rl>TQqP<@<`G!LXd!e3kU*(Kx;&>%CM!L`XgD-C`gzDl0O45{s0Q!(B)6sWDFvl z5IO{@haePqfkt@AAI4B7z24yuXHs*m&%EZJMB+`P>}jK>E2rmBgiaN$^F7em6VW{L zlRdd7QKQnC@QyO7C$7#@C)reblID!1$+58ekOnchNt1@E(HjPTBh*PyB}bQd!=0E= z2dTV4jNFOmv-D9Fs7&(G*vfXcw5_dejl@VE{pP5c9Z1>EagJ{Qw;solKpu#?+X%Q} zT^M<4{sO~=xIJYdJ=m(6>Jb+MM3*n;phRFiRF6h*0Se?TuX)LvgbLNMgiU)NeV%|*#>Z!=yuKp^TjgLpq;SFz`0~(T@ z<2#_yPuz*5Mhm^Ihd&JB5R3S>F;!44mQ8*}jG3Gj$L`s# zOPsG>&&Xwe({vQfL^Bb9u|sCkx);sNbLa0Qxs%*74@{wyO=&^kN<3_q z4=x^mSvT7!VZL>F1_JS2pD`ku!h;Zf$KP=cJD$q^9;Y<^uHcW;ab*=fBYh=KpFXpE z@r-Z0<5BK%0>M0! zQ5jPw9A-LDaSxf01BL?wjZlE==Y&rPg;59~?I$3=;UGPbCOrivWncgjzy(B=BwRpl z`4=!aw>!00XyxLG|zr zz;agaU<|jQh>OUGi}(sLND>+t5zughouM;e6d4~#d?E;DgY^T@P*_QGf^e6D8nGuq z1S&n@4KKqdc!DS}*c8%$cd3Xc<Zb5dZ^(*^V+)FRA8x z7y&wG6qJ$SY6Ia|o0yR@xHQ`nl>0>x>d;Sz*L?X^nk?Zvt%*B37)U=rJg#{fB=r;L zaA2pw68n`hVWPIfMnm02kpo<1Q0TXF>)HQ7D&cd z6EHD4oADR}fto(Fe5j)t*?}nj+M_*qU^+$dJ>bJVnc_O%Gg%i#JKED3bOw|&%0&Y9 zClnPuBUdJ6fE+=U2~(+^P3oji`gq*AmBx?FbV|V+ciWjqA#L<$iDXPzT4$U((qspmK z!5F292mGa*qPi1}>Z+lEs-;CUM}(=!nS(#!Ek+aEFY5KBd^)@mmDCm*Gf5&M8z#2^hR zaVS8jgN$M-@=6$sg08<96e1ET0Rt%ZYOg(!C#(38zPK3w$~qfilD#-7@G2fDA+V`s zSfcW;WpWpP-xgM`SF9PUu^TI5#|k9>MiN@M6yH!u;bO8Ud$O4jtQaA(w-K>Q!C59F zvnE4jtMOVjXfQK7vpH!mFKZGb2o=%^Lky4u9BZ^ki?p-lu`G293v`}O3$;oPy`4bNZyTL2G!%MuWi@U|^G--mo%gemYTOh~FA1w*t8#w91>z}x2=`0D6>P!y8!PoNF9Z;OO);1Z5S|8* z1m=1*11g_+_;mhM1gP;6h3&#WAt1UJEWVVs**@DL@HEe132l7-?F=SB%9pe8ahs!#nI0Y^iC_AjMV@CBp!J zK}-x}(_#e9A$3wH3doTIF`UJ7OvlB$#k0W|Jyj1noB$Jy5cZc_Plj|CAc#q|ZFwxc zpBBhKW*28{KYDx)Q1Fiz^ke`mvebLXU6pycRZ5qf#)#a*>@ri*vWzW@$lFHBshr78 zmdKy>3}z(|bu7!XjJ$Uo8{aStreF$xuK*EZ*vAL~4Y~XZreJea1P#Ujx?Q6I#sa$4 zic`mY3pOztkD}?ySDrq(C}3e6m`Dr$H+1a!18SiWCXVmA$|b`DUbs^W{+Um zN(BrwnE(X}RG`8_2^o!qP~6hrnFDir5ZjE#+6o1&6atGD99;?nNs!SbK`lv;jNcXk zMj#4N{SLPfO&a0`ZApI-l4x>&u|Ofv2yT!AL=eRVk!U)-K#=K;O7RSW&;v>LZCzM% z2yqWb{qgf6Q_`lCbnCWgQP;^o#o7HsImj`>17QqIO*yF@EKpz&ie>nJ)b?{~ zOpV%$N>c|=3=wN+WeydWk3m{AP7L^ zX#S-3!}xaFz#%NUeHYx!NR8lPWdKxg?PAz6zXNfM732mX{+{C~rqc2XzRdt3PzGrI zL1OHl>IMV8@}T2LhwUr~Ns6YT&Q2IeA;s6?pftw3Hg zhei$s_W2G$pyVRnHC|I8kt|gb&=w+o97a%j2>}gpu3iHU=z-3$1)dhv6)ZUb5))=t zzZ`(ZUv@9D=}Hz?J^X z$1*^=M@_CY5D-2Lv2Nc6)IbH{56=};d7x+CU-hB(| z5)pdI0I@9S_m1xb4&b`c?c^2$x9}zCCKrAjUO-%Wc4(G=fH;=%^q?1D3Q~-|Pj-wj z1OpL5>M1@Qs9w}9e$?C6>aU)U<04#87GCYBr=1t%$WaD66yCrQ9E&4>21JK`k==g| zG+<(m7O3-S9m~PR%6j|`4Psp3`drqiCT`{rfcxBg7%(o+?$4lADPRivhj9~<0D+zF zS+DihN$6#N;fHdO16K77%Jj<|RC%Z*3Y0`iBAzxA9`fhV@DI-}{_ri~VlHm}K=J#T zbfxZFr=IFA&MqHMwqtC7_aN?ezbq!t=0Lt^5bzSxr{j2)J{Q8r0;Y{^$FU86JLd|~xNe2UZkK(4D@qgd(2qE}I zJZ~o*0vcfbp>*<$|HO_R`KfOZL?H9X)d;^DZk6^5&Z@2c)^BKN3Q1`G%nuL}7)Z$T z&Y(d)e|at#NOI7jL3ue6ND$=DAv72a7`$=_#Q=i{Az*M3bnBkHhY!E?{$P-x(Ycf{ zWzM8o)8YXKgA292T23iVzYjX^9Plm|sZ0zvQOeRK1X!3DT+;YtW0 z=-$I~8s3_!7t>gZvj!&4Z3)Anpa==_d^=dL5y%`|9oD;*!vKt}GruJ+G|j&gfFcJC5)ceVQTjTkMK{t^bImmKh!2_&sDq)5MmR)K0yiGw zE`l{)pS!%JN5KaP^IEA z)Z|c%015~rO02>YZR5?k=VlwR#D|QDGlD#zX-O()N zNh}3Q-eK;ZMqJf@Q=xHXiYNhGA%xS=$%K5RkuoHr?j@Ow5$J+pNVWK4j5F4FV~#ua z_+yZJ8kN+q5?~+)mGnYOt#jIn00w8X?RM6f4H`xu4Ze{qp~PY|XvSb65`^H$Qj*6Q z_EH!|S+$_yECfnw=1DYZdyq#m2}{Gawb#m^ayCH>czLpaVR{W1H|6{>hKR&awh)FK z;+A{v9N;$PokoB?hbwuY0S?c9FNlCQnZj#{i$fzo7;S&bd%> z)6af~G4&5h=q!}qImeVd1krzg{`Y@|_{jA|9`_K%ivYJBg$xct+bP46QXq=x)#Dss zS=>1o<0pCWj7B{I%LYUDK@f&egd-$j2~C)}(SfRe9>xfQ*n$wE$0^QrDCEg=F|(grJ786ai6m2vY8hV}9deFaqOJ@L zG+7-ja!Zsaa^1(3n($sa2v8s!WP*N-&Cm zv~OG>m?2U6Sja|JvXiB3WwpXIP_4y(l`)N}Al#q<2{ht6M>7IvIs47godX6Epd!DUL=#@kB?-oW!dt9okZ;J6sHRN;wJ>XwdI$oxr)|V5G@yaBIAp6KfbRl-VNS(5bfp;(-$--U4O_|JD_)cUU|PypP-v2s3*8byBp4@4HpX&)A^mDt$6D62 z2HB%&J(P;3NY}gOwE?b+l%qjT0vG53XiM`zKLe;9rpSl{G>`+pv?K)oyLGj#eQj)K zn^d;eHgg3rPjH7@+}KP{rh8?Yq!J(pce~WGLqLEri75d@_;0uEeQ$i{Ti>FB^1f^0 zMSur9;AAz*CL7@fEyzFm`XI$eO*EqL1o-OHxT;$Sw zO1qw63}ZZbQ%bfX$}iXNkH=i*GpG53Jzn$uggmn($9d0v{&S$4^yWb)deM#kesrWK z{U}3MdQ*Yibe?1w>QkqB)fLTjtDoG*R>ui6P_AmOhh6MrCn?r{YjO^>hYIZkpH12? zwRX18A{K&%;T<8JB$bNXlC%%+spNSdsSMN@pQxHo# zyD0KtaiL(ZDp1m&DPRuX+9aS1fJcz#XM#~d4E-665JZ8aVvIbnDHEh$is@$pr4$eV z$%^NF?|tt%#>?b?8+jlA19aLcc_51ceKp#A2ylT$*jFd4cJ)*I;So?ve!Gu66@<4xk3_yfue7@i=w~0R*$PoGeUjO^&f2D$7{-xSLC{6(s z==nGHfSP||E@u0d-;s)YV61O3KNA@mT+zD&NiP`Ct=pS_r3*L?oyarDkcwpkEwuu* z_@fHJTdgu^z`_}ZBcql6i$NKjK?M1~*$65A+H48zU%SL9gHfkl+Fd zbff8zfQ!o%`&$y%(1c{7%oUa8azWZOhc@& zLA=1hPXQEvf=EJW@If=2t4t6=B7BMmJi;V&2=s#u3p|h~G=L~HnknotBNf= zo-WM1I|Rc#ObLG3vlCQ9Ow2@0WC|Nxi*G0+H^dE;_{4J%vQfk^wf?ZZp12ygQ$_F~ z#SD{(I5Zl+i;$Ht00dmc3=)Mg;(~8D30F)p`}xFwrvs1oNjpc`Ln%oWo+FFjAQ&^N z3Dc7x#=wzaR7H|Q2@66optu@;a3TXJgX^)q+H07?IPlZ7jt{l!->XM5jZ$ zYOD!rtVT!iMqVVuUu;I)V4)wW!%ZAWf-K0I2tI${70$|mbMhBL$pkJ~uVI6MaGDIi z;Vcb*D1&Dx11Ja@p7;h)h)D8U3~L0g@sfZb7=|Di2p*J?lZ1j>XrWx(iM4?OF3it?IMd= z(tu2GuG6{&9H9)3%%*I@G~74`S%R(*(567NiFi~NXrKfmn2VUo0kNb+QBV|&G=ix* zh?=~~oZLx;^GTUttjdu~gIrD4R71x5MwLM~xY&(@ctF7sD#S{Gta6+laH*k6jV|kd z!t_cF>QJhINP$~$mcDU=))5Z1nneyvg62%CvzW=8I0wqn2<_ZXJmH&Yks;%h0OSM@ zF1XG17>=xT%HC+8RiQ3>VSp5fJHd-7H-Lhm(lsM^hH^3fLJ45E_=o~&7%7Lg%3~BqpRqDxqh)fJ( zg93%)f=nO_N_xH|aDzNx8*d7qdYB_I$;dotn2)$P^;EJ6p^jX^3kn>brZNS^Au8J_ zOKE|R{UjLIix47Vq<6TP)?omO3zIGB2~lv<^1REQ{wz%|OTN@dk#7(V^Gr&VACFO z(}jyu7>yQOtV$c*R&I?!#>S15@Z%V^A-G$rpq;q8s`mXcz{D*-ax{#&0+x4|=BnEJOnX&m;g59?Ku( zc}5AS)OsKi0VPgBX|-2KCs*wW9cr~BIv$Pvn*QW-SE}?nH#n#!Oxj5T2}Nz18k$$X zGnYJQgzwvlU%d!HfPzMU$XLF+1^c{=QaTx>v08I(>1DAm}9M7$n&d zAbB{VS=>^S!Pf{mhh9p6iOmUp?8SamMa^}NcgO>RaRVyJjh5hl4FS?HXvm@m+>LX9 zCI-j@vSN!Xk;d$NAsq9EbNF4Bnh2i-T06R#-Hnh85`;0rj7RiFa1=-NAxCiS0b_`s zdoYHKhyYP2F69t~*T`BSsi6>fh3FB+-yQynu|0@$I5}8pD_qP3o9 z#`WL|y9MrAk_u@VvFQP4F%|=|9x@}C}ZE-2r|2}ZRD;$Ce*_7D`+q+m#n zWSO(z$RP`8C19+;b3SzjUk>6P6(h+9w`)D;;o>14TyR5^MAh_u5 z*~y^Zv9Mme=oLtc8Hq{Z ze0_z^RVs-_H4eKFHyEuQq-6;7&!(ZnpZI4ZnktvS&X*->zmYBFKZb#f@DKXa~kc&pT+m&TEcKXpgj?P_*h# zy8sw4f_dTsiFo5szKlLgB2yk6oimzeAcEo(Y{H%=$XOW+u?;SW892n@q$U)V2Ew_n zk|n-?6sYVnxK#2%2(?Xt0#z#Vl`o%;01cpjRY3zlu?e*-ehnj_=FhfNFt%Rb;bzP5 z=AQUQC=_RN?2W5_Ucqs-DfY2nsZ$#D)P=pdITy<%0-6`qPtyy?{Jv4ka;v z09I)WLBSDO^lS*MR3YKp9~P(ceh9*yh|$}XBn~oTpoE6O$(9(s+Wb;{d4+27MVV*> zZ<5a_sp4$;Z6FP9ff()_DQ>7vZh<+(05|97)@qrsh+Alch!mm$7K#<+LX`-cC(Q)L zU5=;u;S3rE5Lc8}+HAQ7==w|86v#?{_x^G)pS6TuLB4s`4IbSkDG0uShAVdNXXq8r z=nde)+*odeAdt#^{Syl6fmG>S%$8w0#^@V1X=t1bUcu#+zHoZ6 zk9!Rt_1sb!V#>xiwG006a@=Sqf%)xv(2}6FCa1>n2u5S&4lU;Paif9m+fxpI_6w@M z_K8wfa!WQ9(IPPlPrwXo98*XqqssD@MlXAnrZ7)-WoIRL$)ozSCc}ma>stT#y)BPI92$MKmPNMmmdvGGwt#Ia2p2o%S@QPP2b;Aw~$Sm}*sHWHZTjz+aWUsuWI8b9@9Iph0FW4>JO z9>;2if_1zwns$xZpV%_^)K&mn9wd7Wo4vx5ee zSq%W&q64Z2042o!MKq#fa;$GW9r)5lN|4Y|fd+6F_cogM!jKKsm7(~5A|Q7Ur*D{! zSvCmwooFig3yHON*Yz|5C?q9wJ3*o!!A$v&;FGUG<^$HGr-x60=R>UsD)PkVV z7W{5u@s|l7dMb4rCoEKd7K_b+iov^$XZ4PE?srP$*Qg)2hXxmO`)`1#TJH&ycL~No z#%Cn_cQE^UM~*XjF{d4WdH7X(wwHXPe8ETFdEC$avWwtCC1jLK0x;#4Sx*s+g@(ZzZTHu8r?2$PqoqGMrE*NI+`$b)BiEf@W+qhSez;R{tzzS9sj9f(VI#frdM|cD?_e=6is6cTQA)1O^G*KzpZ9p}bcG zFML}lk)gu~WzJ2UD2f3`BDah|k-$-c8-EIii42iYVZB>948SlZFC)#G4O1jYkU~%< zN|{!^lLv!=ghnP6LMxLXLMDs~pP3RRRVjlDQblDlcJAIQ7!i_WefEk7M+Ty5qOExj zNh-9ws!L61hmpbg_9A%O%gfhR)4={JukpT=nV zbF(;y)3ldwGzJ%!?!E*Mf9sZs`dbK&5stSd>PG(J;Je5T+GXmro+zBo3)*F%LHI&w zc~Za#e{rI`Z%_ZzToMqF$3(-x8LI(qJa_?+3Y`O=t{0R60|6n3QjcoqWYPnpp-Lc#f$w~yS)-0N zMu~zb$|Fb!j_T?bg=tD^5uM-oXh06PC@YaDcJ&YuP4bwMe^aY94f7zT^x;U~QNt*h zLX5WIwFyRT)=Rwj)q$6H}yPY}J!_^Y zj~Qdc#s>U#o(6ycn+IPnH#TlY%sR_(vK?NLz8yc3w<-V zC}e|9Q|5IWqqbHVJrl%Tx1jaS=Ab>33F4FA4y@}Wf1We=>#Jd|x!NSQ-Z*4xTYk6k z&-Z@L-LGUqwkXQO2Da;#LuI{Ih+(f#>C`t;KHz&xRJ`$f_nU`*Rl^^@{PWXazy0^) zpFjOqlQz>iXb0sT^=Oww{C5tF7-)bKsfR!OQ4e{@Ls^30NEs~W9ta+dJO@M|LkPID z1I{FZe;G93IYRhAhO8!n15{u}b2&akk6BrM?wV_3ieX7C-RU_jB(V#Bn|1Z^b@ z$`5ha!V?BiH7Z;od62k34|Z^cQe>hG1#-Y7#zu-jBVhnV*g`Hs@H0#N;XCfphzk%P z2u9N%8{6o{H^MQFa-1U_3&lS>)q@gwc!fL~e|V2fSSbM(OB$D;5S~LKGLbOo!7!wD z4!>o>f_9Pzh!knb@T?1I@)!mbz;nq&LV;aO+vL`6k;6KoGL@=aB`aI$%2(R)l)%Z1 zs(QeMG6dmU3CIEZ1SKOhGAx+FGynn2GPMWF(l?vb00D;SOadez6G)>DGo3k10?dp~ ze@eq3EW7E=Z-O(N;vA376*33%oF_f&NrXm{Q#6$1LOuIQ z&mPduQ19Ua4E*U&D7*rS`CKSN8|u)9LNubPG$(MVQwAdB00!X60T+g$P(uap79t%f zNlO|yiK=CACS9pWB`GM|!HuOO9R}Q7e~QwZ;xwl^-6>Cd`WlJ?=N|R2-77?43S$WM zX=9usQ$LE+r8ZTH@Qi9xlj+l|Vl}H;-6~fnYE|GsaD`v(Dp|{F*0Z8Dt!iDX{D8{V zx5729a-Az(>uOh|x%IAk-78=F>es&lcCC30EMW_4*ux?=v5KwVU>EDy$3ix;f0CUn zWlPi8%3?ONn%yjCJ6qQN%X&7nq8%-1OKaLUg7&nkT`g-{>)O}$Cbh7gEp2OS+uOQU zwz%CbZ+q+8-~Ls%z#T4ei)-BD;#9cET`qH*>)hwIQMu5aE_JJG-Rts6y4c+=cf0G| z?~X>h;2ke{%WGcdiub(gT`zmvf2-Q`x;MV^oiBY63*Y+UH^2JbFIw-*-~R$QzyfZl ze+O*f10y)Wa4fKb8|>f*LwIQnjxdEQY~c&@X2KZWFo!$rVbN&#!y+Cri7g!B5~Dc9 zD*o?@SM1^!!`QqnjxmjEY~$z7_{KWkF^_SZ;~oP!$U+|NkB4mJBO}?de??9*lbh`1 zyDs_3Ql2uE$r|M<{$n}IT9)XQx9sIFgBg}x4l|j{Z06{W^~`EsGn-pE<~GAQ&T_JI!W8_ok1Rp4T(ne-MI9l13Cc2!SAMnl!PPBG$6@+Nf5M{{2qJ5Kt<1_-@f6X3vlb@{{x)6om z^8!hnrVScxbN(9|0^d_+B+xN+HnUMMFh##%vropf+cRCllm-^_C@2{GG&1r^2wa_; zS6rZ-9wj)M8cjSjAD2ML z(E^PS-Ngug%fVtKe^3UeBJCcg_|^q_nH#nkK;boC`31)=efTh!Nae_K~50Dc7;(e}J6a6y5hD)Q=y)@jG|U;c$mI z`)p@rkF2|M_Q)!GlTq0jQFqQd^Mhp5kyT_AMMn0{u7s@2WL6T=*Y_W|AMf7xdcR(e zCr;mk+W=)Ix{i67p7tgdFhJdRAyKws4+ahh0bnp@pBe4vWf*87+12T58n z3G6`Bmx&)E>uNM#V8LP@_T)00Z&@});G^q^E>4d(d;*hIhDBoDc=k$(LEXLQGOFl( z*G#UT3M~ToeaNd+p$TFE9)d35W?OFawKC(kK=o`2t?%6Q=V2++HNzkeswf_DMqMz< z^E6hUk7|U`b(qk`llZg@V&-{L6?ln;!W<)N}Ez}q3_V*Id%D^(>OKKCr-g(p9 zZmC=^tc*_UAk-93gg{C2P!h<(&@5zGKOh=Cn}V$w0o@yNJa?UFj>dKpEthHZ)&R@M z=s*Gz2;^9|fEz8TjQrQ@88#OO{vL}G!n$vHCinta&2Z+_P`#fRvv1U&PTac>Z}HkJ zf4`AHyAh3cNeOIrIW@yM!*$@DeebY94;urp3zru&Iz3|JJsc|cWzM=(1P>XqXsnQ4 z2UxV2May9rt33aW_ExO9nGsk0nd8Jp39(vHx2QZ=BJgmOtmp(DqBkpmU|{Ta5WNUx zn+d&zr#kNI55OB+k?%A198jl&DvH|9%}0GY|J|F=3s}a>QD-!Z(2ottebK2bGvGjO z;gVB*3d}y*HA*Y1(kN$Gc}Uo+pdFPi}gC z%L4Q4wWcrA3|W>8eo`9W-0ev(%l+Y=6O$0Ju<*{ZC(OhUUm!;$=t0e97&0v7xtq zyBmzv<%@A(Zw%b5e*onYR=flGY#Jy8U_Te&%@)e!j%}NXk%a^Algp+q`wz0qi#HuL znp56Q&_^azdQua!w zdNr(QMki|jyE9k6v2|G5CodzhJGTc-M(*LRM!;V@K<=jBk?@3Eo#wsM%z3q?!lUPg zl)HjIeS^fO(7R*@BZK~xAyRxfA`o#l`B_9}LV-l`#^9poZIuk6`VFR<<@n{5CgVCV zrLd-(;J^9a-{6jgwZ*sZSG@Zc2QDD~8VcQx;Ap^7+yz8GklUyMGAOzszKkyFDa9m! zM+5Eg2jEsKx>}c4yrX-*`ZTTBF^i?KBq>_8BO>rFM|k{y^?P)B&*-ny!~FGTuu=>X ztK(nl-+nvzam#Ga!D>`-_;J#K*B$6|%aOzqVX9-UVciovoxd${5+nP)Brh~|qh9z6 zZ>Fi?uIek51(}+G!3S!X+MC@BKPP6ggf72vflYK1 z98%N>a<|eFjycvZ4j^}{3I$;gq6C1oh)UFbNRo^Aen8ur@C6z35sdxx_##@RvQK#p&=R50a+ql*A~SFG1Px`F zj3p`z73nb642|}#a;Jgv(S{{G&z*SCSPc)lWkKaps-e+V3DrNI$hbR;2l9Sqxt=2Z}Z zKZz_KKy!%ic$N%<=8gYy@g`@xa??Bt#zJ0YiAKGN$%Thw!{XjvNm^F}`3OPZ+72XC z=o?~|Wr`$suD$A=&GQ*3axYvvFu?G?zW^A~{3mNysIBi?j(_?qNHo{(42}>|lryMr zYj48FNQefb5z7-_Wfh#JX(8+8{Z?=}gI;-kXR0ehPr;!+FuX|f@5r9tk@G9wWC7L_ z#CBc+DHb68PTs3|QwD6|NwB0>kmNO*iLv$%WuI^KqQ{GPRH2Ds#FMx>L-|N>x|YZ5 z8>3(t*Finj`Jnf;&zp&uwN-O5Bi<|>FxS!t3v`$k=(>F{^B--Gi%oU4S6(6CrFley zf2fdD!%P;m<@0GR7<-f83X`3R3&Tyb9Y!~xIHy6E7@^{@u-#AT1ebn_!&7Wbn}Z_< z+1V^gTTuP=C!fY29EW+%&L+z~7mUP8tMprW2mW--CRcIVb%{8pfL?4(Z-X$Fw~$rX zC*FCYkrejYl7>G}j>t#X%d!#cZQX3D(ETru)KS%otB*5X1IEoV`UGLg^oJVwdwj*z zgDA(OtMJ;~9VsD#-N_)xV(Bi9GCVCTm&PmQT^NH@4*GBBe#GCCwsb?&SPVxNmJTyr zCLon*uZ)`~%j#birn{yBu?nGYBFk<&JptDT0PBtncorsUli(%)1P({>j$`h%<1VM z@$D_q|4F_gfW0f=(apgZSSAnH(dULcT8gc1iCk+~{W3Xls{yZr5u;=g0yZ^LtJi$? z^y$os5%U`xiP)yB=%D@6_m_#N(hQ+U$UBxt&RZ5)6&YB<^I1&`x?FBqS%B~zC4HOu zMpgX^8#RKXhl9K;ovflIJhxXKzSf3gL1d7WW-)N*=9kUM{ZEt%-2CEf49ewarT1fG8QK%u3JXk=j|2=2N}1tFG$ zo~)TOMJd1nq7@?2Q{d6ul^8)x&+{hEg`=cj1A*@&;R@mOFnvk=Y%XDpId3PCdlMjR z4`49UjuxlFL!;3PZI6C4@^?or4R^v-;JUo-w1|&Svjp{?-e*-ER@)Tm>w#;e-!p3FvdehkD zUq@XnSq_^>xu=<+>}T1Pjo4gA=GHZMMK66(Og-3Z@haLBxv4rAc>7br?a=s}O!5QP z24J*3O;!{9J2;XCP;uOu zg954Q%tJztxs7yyPYn2hkg~!|LvcZPS-)_Tv6vq1n{(!iQJFsON27A@)mcoc^3V4R zb&tYX##Gi>Jt`Fves~e7wKip(z{qVuYv*__hySwdItgT(&KmO$O z;O~`LmpC4!caDh~Y@aT>!>D;~`SmqkN3BURN0@`%*O8%}3q+k{U`P0Lc@QhXec zD~sW8t~D}9mi(dDBLg>W1|NS0a?xjvPw_z0-npOd#aexcbc1RKH^6d~v&+*2b&idsnZQdKm}DLFbM{9^M;|=<|F{ zG3Ww-I(e&$?Ojlqm}^gPwS8GH-6E-Ixb?3Ycpr#${g@(DT`LJDcY)cwmW%hfjSa6l z()&17`tuoIdjiOgA3>q0lr1`Dv%s*~;BaHXCk`?8naJk4u}p(w5E;1<`VS_Ho^y$);m@Rt%?EV1;bS5NcDGp0lXb^0@02qPAW zqVn8w{qNfV@eJ#SG)TuVGH+@D=mPHc%Alw!RRkivyMQdvl54CwtUi>(Xtt>FspG)* zv3nD~xhO{{*UfI0u?YzYy5j>!OL%!YwF%8GGO0A*3)vh7vILZl$c8g( zBCXxMOc7-WGf92>EibKCQ6SD+tqcDjxK*jMZ_};yNsgm-NSD=KV-$+gIuctf8FRsjwHNKcG=I( zkteUfykR=#;MSS8IdVE#`kNoai!Dv|VKXm%E8*lIU4cW2SyDxsi+|fnDQ1r4zT(fP zu+}PEJAZwG=_A3m-mexL)YBYAhYm)2C!DDufs5P!(*Q>=XMvKCXT9VEOi9Vo;!%Iu#iRb8y}3Vian%| zZLGf5?*?az6k2uoA9G0Km4eZCV*AY#ILBiUPfN5+^c5n!ty240I5odi-FO~>kt&QS zm?Ja_DJLLJr;D@EU+L=pRNuZ*=bi2v02I{iq|+w5fOjC#^y4;k3d5*qF&z|51qvt8qoQm39Gsy=L>&wL z=?tFL4iP>9PB6PMse9Cupd3E5jTji=Hs@r45kDtfc>D8Zeyercq z0P-O3ng&}g$xu)?Mk~_QVDG2+wTl-r=IT)<#*%#;|1_4P3T`={r`5K12@TWa<+Cwg zy8&lTx1e%CB^g~hPyO#Lp(HhaOV%($1g18KuQ$$~Tj#Zbne@%#m3VqQnusRo*Bau1 zAj!+w6$y~3?gyks!n;Ob&7Bh9hOa`SaZSi*scWe&FdLg#EUzlII>r%suBNX{qm1SZ|jiwVXO`>i)_A?JBS1bpg*n$|W(i>umkR}%wiXU^KCy7e$_P=lKQJ%0K+(A` z{Ya7?jZRiTB^h#o3}2dD5c$ypOzvf>*MY{z)DT~IL?PFgZQVD9E(W?4o!zg%*&fOo zyMD2=F`Z3#sI2B+AGU<%C0V~YBxvvR^P$#d7L|jT%W?Ny6Os)=IFGLQkn3DS&aH%A ztrtWy9}?Y9yEy-a7?vQ*pqllZ8d}8rm>I*Yv=x3eLW5t~Ntw#Yy zv=GaRBrUNFswFp7s{Ru4(*&t`?yVYLt;Ci)fqRg{EZ0vG14q=}9&p1G^-YV_w?54h zex9N&Fx&f(_|QAYguC{pUjm(#8Lp;khvuA2Z$6M&q)(yA-fKS?e{>OFh&`T7@%KmmdXlnDTplfDb^2;*7R(xAd%*I1}G zky()xdfN}!iB$R|F7(rmscqzL(#%yuRSK9r6V`vfvEwbCV0Lko;~&dSCdIoMKV4LF zT2?r{ldxfQ_%k|oAN1{v=Un91mcTP=d%L)t^FENsnj?vB-rCoFZU1+?TO-7I+S_h+ zn`=Hf;q0MJOX}%+5RC(KE)aco*Z;tz^L< zuQ1a+NxwTA1TlW55J{jk>R9qyV)&h(oGW4uvC#ZAwk(Ebo9oHVwYu@{0}2$_5i!uz z46Xzgzcl#Z7Ys!tq6LMhl2xntR^ocMG5r2=>oCfxl+p0k&PK)BQCl@ld>8I$EtcZ4 z1MY(A8Ssil&kbYXZt*JNqpYNu*7{MveHYJKNg$z_EZPAiB0B*TA~U74HU5t1oDej3 z9RBD(Xn_Ag*}$HdTXd+G@9Yjy<$&kAe)l;OS!HYg!1NzfI@))O`38(-tWKP9zYCXq zXw`klD05_Y=pf>cWrRE)Lnm3#J|6zYEf-Di#Q%boniG8aQVMW<=c!5Q+WTQzPN!(* zQv%K0d6&?M3^l9lZ;S0gu^01rENCK;p4V_9g+ArHi@^oMJ&MEFLS~2m?wxW zGk`A6s+|gfEHYj9UXL&0_RhwxpNk+keZ^S_n5|Nhznxc=)!h_*M*ginKI`XB%o{wc zL1zoq{Csr$-?vWPhB;;>VtaJ})-X0s!b@-b*6N;jFEj4^Wg?Q*vuCOb%=n#KXmWp} zujiqQy)^t0YhRCMvKR%+gGQ_yiV;%)*E&|Nd^byy9A>2u+4*!PrUXaH2nG&}Xu3H^ zQNretP}{>bHWDC-MA=yE8-byyIaewuD{Rf34DnQ)ZQmIau;#rYhC<71i(Ldd1^}eH zhhgFwjRh@a#+bE3hKjPx6Azq3VnmGjH7Eeiv0X{|m7xp+}!eF8mIra0^M#@ud>8ChXR^J|ZxY#eJS96CI|^!e5iFW{?3 zg$b34Wr@q4;n-ba<0OH-A*55u)m;nDhE-mtK~DLvVlWgm1t`D9zdpFbdqkmGgJD38 z1NGfJdPqU^1%|w13&bY$d^*F3^S*^i1gwfO4i!hw@5Z7V&c0m|~AE=&_yydh6QY ztDQmLunHeQZ_-R!i9Kq6qoK?gss(^}IUXIpic%VpM|Yuk1qU9|SFV*HY9$|b%q&b8 z@)Kp_L&s1Pg4YS0E8Kf4qA(O~ElUyIDsR>S)rH(A1)?rxB{_7};B>;sBSt#fUE%Er zpc%z4DNo&2#@o>at;(Wj262+Fiwpna>OI)C5x{8Q7^C=$mx}7=KyLh&&6Ug~@z!*Z zyPN?`)8tnLvyfD!RGouQUjG~P#taRwle3)?@|5o0J9(@WIW#qDB?BuC09Szm+TlwM4cuDjlz0VH{wEB-_52DRoH$;yULXdelSU5UXB3 zIesonKxKdl)%Os^Yb4Q5S!Ak;lx%~KT$K{YbSNK(3f*Gnse(6z_u0|lL+~Gl*6nzCTR29e9L61)wvjdyi_r3oxQOb^C#8=@}}1OQB+cR~f*T z3VfeLf<)Pql%EAr$J)p$mnDsb6ZNQGt6cAXr7QQCiEH!MSPC=u*W#1`oqVZ#gjWKg zXu;ck)2RCFwyjKsZ{g}S-C-R4yd>)f1uEyY+kYA9BdFzA1bx;h*7zHx`Bb5g3*5K{i|7B|s zvH?|D$v-@DN0dGZP~unw?tW5=r+F*_8R69HNxR^PiL-m3!Qks{7(zyqUZQkPZBqI% z_1L8rt0yW+zil5>EWl6Em9HaCWGI?%KHu6>e+|>G$#(fy6_eaXm z7H*nn*w7p2(g~pBq+39haFr0Qy4sE}ESGq-cmxNKT91j^sG6wH=rthGDpxQ`6_=(; z3&Z7=d&gYujH%An)raQmom?#j(qZHDkE~!IT~KFh8jb!Q8d@A}B8*82Rg>4W54}F&vBWyrAa4!#H&=Ps#qfSLkBrFxH@;Bt3rN~Vz!*n z?`_|2M0Q1A}jJwtmOpus8*t_Szy!e095 zd1bZV48i+97Evbj__Kd}be0=q(?|)g>>giXgWqS6IG;5Xqsr`5Xp_>wuwO#&Z!N9q zcA|uJL7cou;c!x)@T1xn0(a48-WNGk+ypi{6!mP@alhWRfs)VxhSq^EsD|hTK2G4) z%iib=s-dg)Lyx2jL4yO!Y)GvuvqJ6MPyS07620Ozn*Wf{iC$G5b+->>NEgVr({5K6 z&kbHdTxDUlgFkP3ukCc>{Igb}(snk9zf6NE<-Nem41(n*H+K)8Z}a$%^p>qDB=IVB zzLf;5U4R7hRwfB|NC>a-vn&9%A31G5>VP@;luJD57h&T!QLOkEUO`h=AZ$kXVjG~i#WxjPGX4l5;lxc zKlAoRSnq6OCB4{HmxcXklTSSte5LJ%+oX8mN4-~cHlCc{YSE~@QrgF2qwvN=5DI$C z>SpjET==8h^`m93tr=2aRGuKQzTi*&`rm;ouNFa<3p~b8dF$Wn9P<>(oXIM1St?$k zOZ#f{A$wWMfBO;N_Kj1t_vosM!ZVpaRt?umP>Vvs4~5aMI|lz-b@uVWjf6%u2^Pl! zU_^imW^G#QhW?^f>C2!fF#BT-8sjr==%Y^u=Um1mh-F`~J0i$UKGa%!S~fp5ng>CoXc%2hJ`(Cyx>z4UH+_`K0C%dB{UcFtEY>}?Kt3~_tg&?$*kT+=(>-2 z{w#C&%T@IRy)7oM#noF;Z~m(WHBOn`500m0BHVmdZwUKfm@6aOWmt~2OtIJm(}Wi~ zD(oyv*VJq_s`te*TYjL`nwir+wK19rwrV|UZJJTJB%!46ftkINSJ9fMS58%tu|}2WELG?R)wn-G1FkoNy76N$<%T zCj|F{6_doSMv%Z7#SF381Fo^Lzj8B4K!Y|xrZ7ZSps9u#J^(9nk(EJ&q@GxEfu!1h zj1u4dnbr?uUm#7%8(ipjVyGNu!KtwbkC$^A4laXn!DyadfbbG*MqisYTnKKgK+Q27 z!4eUZb)!BUl&R3W9G$2PGWlgoYjCN{=R88}#u(7v%nL(i<~B7=KVs*(nJ@2)P|cGc z_n>{4O)#9=`R6O79O9imAT_zZ?3lKHGELFm&9jIf z#t*VB6RCNTSQ5RG7%-OE!n=eE`L?mdf8yjv(tQ!jpk%S(&v~m4C8Y`T3hc#W&CXEC zb=lE-p{HbTN79#jhXeVx)Y2Uefn0(8=|dI5Ia=Wcr=BN}jX&87-DlNoTD`?HW)1e$ zUPYqEfbEODw%>ba`<*mb&kuUp>nbbyuiDMei7v7Og?}SYj8ka7Y&wJA>>fRP}L?%_%_9Oj-IPA(y}Qs3&W( z8bw8Su4@X8%`JopD4&3Dc!u-+%^l$0gva{5f65Q@Lui&i7!bK2Ag}-TQ@J2As|%0) z$;95=-;LTDeZVNETMNchO!D0BDtVZJ7uiZ^Z*1W>-p>1z1j!P6@JKP-^_82JOuago zKu`fis1&PagrLJkwEW&x;2jIJGtZWQcfoICd27C z{<8D^;>v(^mJsdL99!-$gYHPXN9Cx@04^zu+oG>~X#brjB!a0X96&H@;EsZ8!YdRI^eaTP7R)m}}$$&6P@(XesN;T^tUw zTh|$qJ+^;_Fk#nfgqrc?ZtNqo{nw5g(iN=?1(Pd-qrSeJc{Zq8Jo(aW9V(btG^k2( zO&MFH;yisN`AMwQE{|EaDK~Z3Q~4UqoV?mdAvo@qV!AnJTyXXgz#-q*5xdomZSJWP zWI)^q{wrI2OPi;ZLKauUvaizHIMSYjP!MCsIUC!Ucdh@sWV<9JOm1I4=XLkR{N-vT z=`p*@PO&6Cv-|8#*7XmP3fI(9#;2^`mN)H692sSN_El<(BK;>O$Dd@a9+Oz;Sud6r zOj!PaJP{}<-;b0YaRl#g`oEFdC#coTZ#vjCqBN!A9o58ARAS!he6IB#FIik?WN#f0 zln%QMqo>+2XdV6XNGk;}fXd2!sc-r+GiP*7M8fx_X?9hLv)cMF6~`d$t-zxY<1&RJ zL1`DpSGiv;7Q|T~btJl@z=6?QGPm=FZxN(l<&7sNaKDRCa&LXVY{%`jcl_sV*vH|= z4R;XS|9U4{&wfOK9*C`!a*UO&bx|AncSm|$ z;Zdh)Xzj?OaN3yMk}dbUHLYL1z0Q8SKv4hvcK+|)r?hK}zl3Hw&i?K{$Z6GpTqm4# zb3HsL)NEU&_gDAtMem8ww5|Q^_&scYag;kAwfLa+&sgNeN0onVTYA?|rwT59{x_xe z&3gRwW9!AQ`v2O$dokS`o4z+$bwnZWe!WWd>Q%m0)uj{=@#x#Miuti{q(|3Dm@ zgf{jpU0buZ)QAUK(Jl!510ac1DnxLV<`KK^FBdo@orZv;En3wyTh>l^sd=#*%;$TB zK#G4rrB7ST3{JPcJTE5+sF3Qt9_EQKnN6Sm_xp(p=RFz8Q^W@zsfjg`M0l;v^)N~a- zl~|{fI2UAmLFHY&T!6%&D7IcHFMya3#PlW|Wa%Awbii%$M>M+B=@joQc33PEJO${O(lYlz}N?5h*8z+7<{5$ruxqP;e9UWWW0q}~m8=t-x+)~E5OgLVfa`a7u|h*p;n?T$Wp+ftLUDb38q z*>f54W?ABb;6nGeIZeh4QX%79w+s<$NmG^K3saF_$k9v%fR=Nq3=R-i4;l6HvX(Mv ze>XpTl}XD(o8{M;755QJlfgH4^4NtV+b!?@LbHe6h5pSI-3(4*>^jIqRo+i{5>&UW zJ>JMX+QWSNl@dY)J`Mn}!+}sypbXJx>m-L%Z}*EJBhHQiYg#kGXgy^Da?~H^@E^q1 z1TtkU=MYu>L>^H!;}CTy;FTMYDG@(d0Dgu>)#q6nvx_2gIHLocssQa*oC$uXkc7jr zC}C;Q6X=al;40bpBAtel0+juey_$`32xj!eC=TUO-QFfJn=^eq$%)S6QaPlf!QSyy zC1fxDDO%%DrMp8MH?3T>%0AwftMI#){q?g=6%}^#$>G_LDsG_H?M5UP-Ou|5ETvr=?OhDXc0l4 zp86`#FPQ4Y14doD?JcxfJ>^SAMoahOC(I=Qv*|3Zu;f!Z?m$V({VG-^(N$Nl84CL5 z05(ydAyQhr-oqo>&0BMyKCAPB+V24N+XbeJ;YLqD$BC6XEH|~*;8+x0tuM@?rCQ@F z1PuU9U3)P@2bz4FnFR+E9F~=9uPLYs0wvIh>)+J>MbI}VWh`Neq8?FYVeDGn1e;0X zKEgl)7rKdq=Xu18>=wRZtfv2I9e}Ie4OyAe1n+kf$F1W78cFiga1QE3S#u-E1ZXh0 z{1=KTGo1dyjEQ?kY}18q3B^*QQdmRaS-3`so-T(QsV}tFa8{Ci53| z5-%un@csjW0Z3dXXIbt49zsc`H*D^?cDr`v| z2wxGeCvr8D2ogkatBrx@%?%(!mO%AiS2+`#d*j3#XcXP#2?M8#935Tq*kN-&fI%&j zr#kwjctI|5>Sd`|iLz|Mz1*Z}Wud~`u%~VSKS~IW3wY|C{EW7yCxpJhqbBV>JOaSb zSrxh9Nta_m7%XV3h6ojS-iwfV?PvD-yL=1C2UXxkfCnld-gFus+Pz`D$D9z}@G+U? zK>^^0Z1Yw@$G6s~xN4hUodS5X_Kaub;9M`}O9nX|tV?;W5kOJkO5F@-VNY*~yG3`{ z{O)-^NY(x21vCF!M!UDtJI}6Q6K=RCJS2%X_EMh}ygNm=VvVXfCmhC;xu`1?`-jaL z5l>U$1o4+E63ux9ajfI%{6)esITP~8s-+hxzb)oA4 zaL1Z=B(!vxaraqpNA4zHZou(J+G-l=zeB$=ZfjKH#N@K`cY+;UNN}pg7UNs@Ugn&3 z7TkYZ^fr(+D(-;=pC1JXcFAwc0$_I1p;QQhmTWgn#TXB0QV&MdVVGA!f&V>!*dZ4| z6h(Y(&b#)56eC&7Oi%qfhe%dTZ<2e{;%37(i@=&e^6E=8t4gm^Govm`QcX6`KJ-!Xy3Yo&Jj3iG-?1 zmf$;G4}vt`)?{&?ziLcInb>)g?j&ot41#><%5G_~h3kQI?EJ&-J)wSLHZs_Lr+c!s z=a&65<*p&70?WVO%C~aI9Paap8;MG+)YTPE(6vvPu#c35^vMjS9M$)|;nscw?)kkO z3`}VWo6Vug0KJiUc^_m2Q-?FE7fBN4bNhb2WD9&>qh=ZT6|H&zlRu>UF5mhPrh8j< zjL~+m|CE_V0BUwQo$i+BzVgCe?Zu_9O54y*lffn55I4%-E^woL+IH-33h!6Y#P5R! z?+amIWJoKUE6nF@1?<5`@1l=$8xv1|cF#hz?44{y$*W8jKS1Op)Pnd_T~M|KGc$8d zVj~u5-R2V#?TCQEd~XR_GYFO5M@mg%*CD^C#e`d&2D<*CZ~J`PmwD6vkEKG2Uoz&O z+_#CG)4jyKs~o%|Oc0R_pp!6yrf_Q7mz7y91xle_7e8t=AIeZYB6^{O!V--$(5=GC zRv8ai?7ihQ``{ZjS1hK{Z?)khIJhHf=}+}@>4wI2eZ3yyhu3C zy0`|3$+pK5uNFFZR+u$cDBy*X8MD7(gJR9~_A{-XF3I~Ybb!Cy@1+PgeTHw6AMj1T z(9^)-9R=wa4*Oq28Q#&E=llnY5@Pmyws`Of1j~Z=aJ_G z&yNR)m|G5Fw+nt5$T7-(I8k3#c49KjLQPuoDUc`XPGL4%uNa&V??wPlIxBY;H~gH` z*?0}z=8cvBA(28*r9ei)crw3?OI=+)P0Ejz{2Qww^L2WPcJFO9rvJj!sKq~GW8_Q` zuOxxkbjj{-WvU@uzkKN`Py)e6mW;Caf?xO#?q49xYf-J6mie@K0TXJ$;twJ&-VI;- zw!|5OU!AG^gXob-Rg0MeJ0aFn_%&QQUH9q~t~@>@3E(e-t4CGWqRYH96>9UbW0d3vQEb?2|CqV<@(Bz|qpDS$bOcazwDxVX!- zzoYDjI#1X?atgRch>F=0D=LiqM$MufUF{7+yzXnS77jM?MN9`kMpa(NEyTj|KI^0 z*fl)(uI*=Vzz{bc%)%VdaCv}adKDhzA5q>cgn`%dj)7t{=#ZJYax*m&x{ z?Ptfk4jQCi=LipdnN`(@Uu%C_XVXhC=_nmGAzO3ir|Q{KjvXF8^@8|B-lga)QRdB| z81a2!?(A*=%o4doD-I(aY$zNLhGV-~8 zQHlW~O@6?cZAmRmIPw_kpR(vLw0)(oY@kkn@Z^)ldM7v&uURv5veH7x>aA^gwd(Gb z{v-vqSnMTv^>a$v^0NH=xi6JZuh`*F(8l{Btpr}j(~xhuQq}{E8erhwmv(|^`lie- zH=1$*4RbGR0%LV6Ke12?QnM?Qj=+~IdEGBMc)xhj$Y2OS8llT%&*d-!OU+)kcMZa) zAeqQ2QdWT-CMHRy8r_cowt zg4Yfy_QpUtef|X2^}NI46pG=!`M=n9P?0bXJ)puZdW7z?r zSH|x{2@3N4X<}}#Oq5%=N1p`8%L=m?8moGh0Mt(7P1T$Yq)f0h!i{FyOgaja+Y?re zlPxx-WECL)-C=X1t9=TVCd%sydb(=VidN=&!cA6|*L6OOT(|UWx?$}cr+CxWtNh|u z(T&@Eiq?){>rK|q1joIbE(K}{%ves@K$%#0WyFt#QFQp`POd5jW6vV4f&}UzL&ptCj6sTSzfE@+u_Cv zO<>wUiI@BvmN$qlUr2x$&8+c@iq^Du5B+pS4wY?G?B(J@1YST#4Zns3HDh9QF^slZ z1bXSgl{$a0Y6gAuWnjze3E?<-IpT-PWGZN92>ULfD3d0i8`^o`*7kf9o9@rvN|OW@ zILWOImP4y1R=;{A$|9ZO-HS&4jBKfWX;l09s<@@gvJ_>%U(LTA#5PwOOuoqO^aky0 zV|{XUNEI}o(hI3aa9Du#RBFpa-A+2;iLBf08R*Y}RC?sZAE5~S4$9rP-%8ec;D9rc z|G>z>n1yz)%J$HC$6YnYPU@@a0e_EFqRM*N-<&8L&XI7v{d+5Xv zGoAax-7?a$gnXC*PP*sj7($eGU`nfzf$=1pbXkQ^FA$}J5l}q4=Rx@ATKt0&mi~vj z?w+W)wnHdM3&`g2(6?pyE;JL~n7gJdd{_w7yvdwp$mpmH@Gh5IM~k3_`;q zbVX{b`qkS6LkqzV;-teo@5L+Y82K_`Ycyzcw32PDKM$+}3|^d6Pvp>Vf+bHbuKGI))I(*KjD zsz_o!&K!_p+ot%}99`Lm$MLpcXw%lpm9pSEOakeb(A`x2KBptqWzE|gic5(g#+OO9 z3>tgjSYL7vsrjM%<(i`l)O{R>?hESN|2SRzyN2*W}(}VdzTU9vJ81Rq|Iv93@|VW0Se5HWC^MpMgLd%T(HNK)hBN5bN<+nv|Od z=~z|IfBn5i&vu@gk`AWL9RyHXtF$?RF(PrD24>5P%AEWr`$8_2?h@p;hHTkFtJANZ zP~Y`NJROfgy5Bv`GXqT@?qk3m_?C(}|HlttOIsD$uCSpMGKy2;UO$#t#Pb{5To^QL3 zpw#AiuQah3`cjESWN#F`M3w4y1zH|h$0)enmI*i1T{%{H5Jyo|m5CQE2@vpVk)b%m z%>D^|B+zvK6M%D~_VOOBy#3NiID1SBGXvZ25IUrRJG(=d{mbZ2<;2L;iCya2nfy;e zlgP!#ozgn3?!3Qe2SWK@V4G!3XaiQ*&VNeZ2HHXfg8rX0i$`s3Xjp&jVFf$gzH4SD z$B_P-H@xV&2xWmBWqb3oO~S#>7#S^x6ui*nF-1i<<;b|3;d)!6T^uP5gHKfl^k>(ZQ~D+6=j$MFd$7W$t99Yr)%DMB9iS#OgE zVLrd8*$%k+>#MQcHZT1W=6C%Q&QAMH5`8Kz-<}`pjkbzCD%95z7!T8p!b!B!-cd2KOiwocE0O-x~vltcBvMYvGhPo;olKoO8g}HqY z!DC2~Qa-O`@}L_y#I0VCs2qLCEa%}R`pLt;J# zXiU=9z_@T-^4;G+t@RQySIctcPRc8nHTSE#?R)ziVjoKw6W3O83y)}h#5wL}GVKKM zo^olIrsun!&?){bxc3UpVFYxWPn}Qd`5yp_Ky<%3lmIAj58qk98GOD6l0ZPLK*^B@ z&l$$In#P0T8ri9bM$oi>?}`A~Yq{oe!**H0noA+3xkRLs!PwixSF=Iys-8~_#r@+6 zA1p=gA(>O(?kdqppF9(G|hZ6g^5Xggix$d2?#OyNaeq(oqhD(Xstb78_~ zaGfF)8KEe{GxRQJBq28Gr2b6EFOTv$XkfxoGyrRqLpqF&)x$i0Z)`|#1ig?6L<>BK za$^7k=mAmCNu8vGmwTggsIxNgH81=g*$E+S5{}o2fJR6=gE^7wvqW}5MBnpA8sx-) z#6f~QK7%|7gapEc6hb1LzOZ9Ovuubt@g||r$}&0$rew5_G|a<9%t8stZMnU@lLzG! zi=4PdRjfY)NP(+=V?R^SzmT%F@hO8$ctkf~I^Rf(tYI;qECblegV=xu1;j(5WQtpe zt3Py`q?8;=V6h&MrJV4ub?i%M=oUe!DWQ6o6Ep&H(;Dy5F%qi2x^X&`xWu%?zS!GC zgh&F`625_SAyAAUrIkYgk~!z(&2+{}g)1Q^JH1eH#vAVR1z2!gZ5bC^SlxfpqPg@fwG z=vzL2XoR+)&Dl5}$Wcmz5QWx&lJSB*s+0++h(o%2%Ytw@*?Wj#*s`W04t&v%T$=p$=d8+QjC#x4r$_Y$t)bs9TV9pZ+`X zQ>H!@4osJqyak^E34fP3y#-YY{=j`}BRTqT2<(@ay#@6P{%fWTT6I*S_Fos5m%asH z8R!qXcxGW1ZO1`*3%7zJM~BZSjkcDF(&(2EzXf{?{%OGGUvLIvx7T3gmzTc<;~oCM ziRGu6NcokJ8H?BHm*@W1U=Mb1ciEd!34}tIm%s&h0qK_z!3B8^{w7FqKlz|fiVaYR zn8lZu!3FUT{&#kYqbwGtzd>143376mm%;_`5dIoiHwj@cX?8Z5V-lEE;g^@g1@965 zVTHM8mYWP~51u+_EqjGNmzTr^?h^iV7N>F}C4nL24` z6aFa6BGr{=hPy*HwPv||W^BEqmzTx`>=gd+pseleh@_f?BYxAVomN#?P#Kq(#|7&Z z{sUBgnpUp5BcG|!Y|n&97=V2dpqH1(1?mC*mx0Lz6_=OE1u%ad$~FI(03l=xHh=z< z!8y_iTN)t>u|3SyDVMTb*z94;mFS3&MEeSU1Q}AKKYN{UoJ5&YrOG3Ta=nBZ6J{bI zD^VuASf?gRj_T|sN@TH9nj<)K9jWsW&dD8#)G6E2)E?B6b<{1inRO+NMCL|*1sn2b z%`~~f7MyydDZzh_C50S`cw~#&g60zKfOD`X+M`E&<=V2vAw-4t&K^dbSn*=UjU7LR z99i;Y%9Sl&#++I6X3m{I!v3R*b8#mjc|Y3S(aEqsUt1FTvD{3u=Eofs`rkzNBIc@w@vJ>STxGu%RgpisaIbJvelmP^^spD=5a3cnmWU_ds&1 zHg(oiXGjQ((`id(CRHlwCfI`Y5))-18vXZny^`w8diek0ZR+(eY!jw$Yi@DrLGSeI? zcMvE+z>>1B!>0@-5vm=6yy`w#Q&ov8uP(!?HpDXBv9RxQNdqs8h{X}GhNwJo#k=4- zh`6cXkhV*8*JZa|ci)9KUU}!G7qZZ>Q;f736}$^y6I<$}lkJjHh9SW^(MU9r?5s*R z;QW7zuBbW!W^_w<_F<0(0qybS2pq_i_q|S(VQ)u|;qpbIXBRH{(|{9+iaGMkTkm5t zVHU3rifRHFW^EV#RMem+zhXGOl>8mxb#ZiH+$464!so;%1PZ?!W>ifgJ3M5GjY zlH=_TzV9_3vpizUcx`i(!>h6Jh$IP(jJ$5oK^0=+P8gi4V*&ZPQzD)fNMODJ>4PG#1n(tx7Adh4raNF{%c z2+9+vJ-q|UB9c4t#V{&^3YKf9jCBe;jo^?E#pt){3bVfOMa-9M4@WpEoRH&RR5k`A zvhgfJ^buXQ{}`AR7#GYMi{2KNe(OOp_hM8400e84McNciH)*0!koInMQk*iO19o&ooX3w zBWfv)Sr!DB{2+ubBZ-j*3xc#3CTfVYL0#8~#}N#&%W4+_PS)ZgtPf$SEdmqIMu;LK z6orsi=xZCU!WB0b9qU+cnW5W4D3{Rv4Q{&{q@j936}*&dKL+BJpKkTIk7$31YVl&v zs#t|TmU+yNeoP1mZRMKK35qW%vPcP+xJ1>wZd+DDo#d1uN`bs-QDk(K6_EqHIr*@K zZF^75{Qe{uM8$ANaC;2&=9iM=c*mK~bf)F1qcdnu#4g0t+*1q_5p0O5Yg94KR02~z zp)E5d&brJ`1hX>4z{5pQj1_-2y9m!cLUM_$dLanE7@-RG)1UtYXg~!zP=7`ABL`9N3Axt9bV4zq?RuMG4#A^zQNWcoymXr-@WC(>>e(_DPCE*>QbOC{=@*o=P^qBfP+8FP~% zjjiz{9vTTo`01#W1*{?wq3QU%>J^RwwT0aD5>6%Ji;423rx-yJTvu}uocdyX5t>TC z$|wF*w21YhN8RdK8A`98z#$R7_{-?(fYP(7)vMTmXiq6}QKW_~7rwYh*l;=#n_{GV zPj!xGZDWqfN(3+xr)BJe%*9{?Vcr|;QE?jH+6IgwMLkbM z1J+i8w%5Jyg>QW2>%iFDE^R{vi+7y25AQ7OJ3h(ie<^pC{?dQQHqI-N4h=kD%`x^} z7y0N#94wYLnM1*IWpINj=biV24o9$Qi-h>=VEu~49tak&e^)c&2|p(`(yeY$?VC3K zW%$Cr6GQ2NP2At_Y`DidW-*#Fj238?@tHp?aY0zT;S1}xz(fXdh=ttZE=@%l^6`#v zNL*we^Vlcw&@q3JB_v>H64@hRCbB{5kOtMZEknl+uuHnfv^g0a+llOUP4H#5)och{`+~_ z?}m50v5UfU*W2Fr#&^E;z3LWc)$f(TQiwe)$W#e!WF*oY75-q4~KZfB|h+5{FWloFUv|SYK5~+m+~g-mdCFD3a&UJX$K6IiN-RMV0deW6nSY-p<=|C6y(xpCis#o3WSI2tEeg1T=Cp_v| z2YcAXK6bK~-Ru+Gde_xH?XRER?Qe&B+~q!Zv!{Q3?RTFyJL$f6zW3ele+PWV?T&ZC zQ=8s_M||QHzj($s9=?MweB^D?cE?xV@|VYa<~5&~_ekFJ=A``QML&Adm;T=LWrzId zJseh@eEN%EK9{O3o1`qjUF_P5{sfZRU#y=G{6Hyz`a{CE@Qw3bifCMzzCGU2+XJiw7?6* zzzo#D4dlQM^uP}U!4MR|5hTG9G{F-@!4!W~!4+h|7IeWEguxh;!5O5%8nnS1#K9cY z!5!qm9`wN<1i~N`!XYHWA~eDyM8YIg!X;$FCUn9lgu*D4!YQP}DzyH>E5yPq)WR*~ z!Y=f}F9gFd6vHtj!!k6(GepBQRKqo7!!~roH-y7Dl*2is!#cFXJH*30)WbdG!#;oX z!#@PXKorD5B*a2A#6v{HL{!8@WW+{v#7Bh0NR-4$q{LN|hfBo7Ow`0pcB*juR#ZyGZR8+-PWW`o=#aD#ISd_(Cq{Uja#aqP1T-3!~30Jcnj<#%F}aXq11(X{5$#w8m@1#%$EaZREyo^u})l$8Z$KaU{ob zG{gvf}L$cd!LinPd!#KlU~$c^O4j`YZn1j&#T$&n<U1k11# z%dsTOvNX%HM9Z{P%e7?7wsgz4gv+>;%ekb>y0pu?Y{7dthi5Q`Verep1kAt`%)ung z!ZggoM9jog%*ABP#&pcbgv`h^hG+PH26+ezdH4ptjLgj}OvZ=E6xP@VO259(( zA=#ZXbkFy6&vSqV--ONJv`^$a2fzFV^u);i^iTBshQ54;%=(CW=u2q$%mq9hE(Qc>vFIs0ReRhj-}9`YZ_IgoX=c(IpMh;k<=!;L#|B z{yiTZ1|2;>104n<^#|ho1`GAlF9p*u71J>#(=r9qZ zSbwO8VbEMICsZr=f>U=tRJcR*kYW?=fo z;0T`J3YOu2kwD=TP~jfPgdlwe5f0xRpkP5DT+H}cUgbt;a03m9fDj=50x00!2)KYU zP}p-U-s8pMdQ09$2#H(B0S#ybd7y+cz+lpC1mQi0MtI#kfC3|c0zr5NOf?2{^SY-}Qfjq!bs^x-PU}YF^f_}B!XI##2yy7gDXK>r%kjR7-{s;lfS`phJjl+ho*+#VK4%K#$(I)<|@v{XMh3*$bl+NPao}p2d=_XaE`wJ9;QkBfYkzo!CpLzLzGKDs z1~-6Y0)2-7ZUiNU0XK-zdJu#Z=-+p^1xjcHHyGJ?{`8V$Zm1ZTtpP zNC62Dg#C2JcM#{wrBDFwUEjR`Q2vG_PE*Gwac|232GD@^#per%fD~9>rp9c4f6#1z z7HA28f(8|Z_I+R?eTG4GXpq3%;*Q;ari2iNX2$i0>n8qd3pLyvXyk$b>JEm19w1~S z4~bWRf&Tso*^cd3*k#haZTcSS+b#ngEo~ye+IzS}ffj0!(CjuphuaneQy_vK_+W;0 z0|t-aAb{_8P;w)^1tTDWOlX9EQvL{25MQ_UZG;|%9)R=trG#4`0s)uNJP3h2s9U?f zU9*PpOepAAnAh}lMvv~-Hx}Ut$mrZ1@e*hCYM2i zS0>PNpaiVWgVc@l5GeDH80jE*1(2ZVHV=tLZ*xEXFJA<<^N)Ck+g4^#Uh1EwUmZ0D zC=ggO2lP7E>D|7C)fM0626F^|g#o_vd{5>~uZ#z$PWfyE34nntmCsCo+qiWDvAyRh zeofugfC~@=^R!N0z1dcOcllC-Y(oA^%7%jE+~->_2!E#WB)9;W_H`i+_K_I&@f3vs z?g2``(Pc++XD90nX6YyYX)efv3C&y>fPqYq%}nraO2`CbFoi5H={P^~*v*3uE`x4< z2mIc8FBb%K2Y7*R_mB97@xJqTcl3zQ?>vy;doKuy$8?6ZOn>Kp0v$c&ATVs)Hd49n zfp*v6AZ>Kz#`pAX1VQF|hAs5J?~J>yPK?)h{!5k3G6(^GJ^B821JQnkde!R)i0*S( z`Bi`U*GIMFUETuC1JMTAo5%GD*7XaBZ=?qL%S`RXklYSV0#SfcriNx4&F`jH-KW3Q z?|gn#&I8@8bMYmAf<|z5Zg1J|6@(y7WR%VUH#qIGE(2M2bS@BuwoiAYhI@CX`>&Aq zh`)S$k9bSR{Jt;r!DsG)NBobt1z86OQ~m}H9M+?Uz=7xTq-gXiNDfg1-w8_L<{ZI{ z8aHz6IPl)HVc!UXy!GrG%9JWsLZc)h#jS7Wo>6k+rc4qo2{~?+xd6r3Iho(wNcb6a zCDN47UO~I(@hQ}(Qm0a_YV|7CtXj8n?f&ZZE0?hs2N!=OnBzhtjhRxwxOI+;1VQx% z?hTO%T9ON*2KyV9!>~VjGM#3WXKa|fs`Ys0$l>M|<8NhP6G9NiiIh^u_xFkj5mBhQ)Hju*$155c(}Q{bh?W8+!x)i6 zCIl@MLLT*%1(HZ4J!4W*8Ga**26#bpluXbxazPHm^kfuD&=B>LhDj~eR9!L3I3tZU z+IS<5{yFNnBac1$_#==(3K`X1d1VA#3Zk?ZL3e-9+yX@eh9RaHV~#;)OdfeaqmgBp zRb>npDHxGSZ0zL$Lm9EX!%!Jp4%FIfv0<6VaO}C~1~l@Vv*(`g7z2?glqm>tm4`Ii`VNlxQG95M1=2 z0saG0Cbn2Xk2UBoOr}M48K_d#BM1x(zH@)^#rNEkYY?Ky^Uf_qG*lm)unlBP5~O`o z56B`nA|XNFgjt57-{jiEWkkT>#xpEq?9naDy<88NDWr*Mff}`!!mRh4;~EUsmXX>p zPG{2DE6Ck43`WC<3|~FZOx@-!J-{$@(QTHb=s9DvO-aREyFqn5Pv=0W!(=gQ;VpmV znwwH|7hJ%_E6+f4p&JC3NJ$79ZRkqa3E8YVm;3R4J;Z-7Z~M94!U@?eC2ZL4okxIhTFAc(mEOeEh>ArOB}Xbh&L zYjj3DA`+9R#3eGZiB5cCR1O!hn3ZrKmbpdWW&@4E%w$nN%#NQZMUXB{5h}+E0bf#} zM!sZ19`X&P?Clt!xg<4kCT!5U#;GLFdp;}&81 zUJ7!cC5?Oo3Oz6#B?Xa#J*Iyk38olKayAJ`D14HFIp~-_>M@K*B2toDQY9-xAj#Dn z=pv!gBqzDx9eY^V|Le5X5~;goj_LwI=*#58|5MTx9^M|<}y zUfwpcAwgt`R0FvMJmL9Aj)X0S_Y_5$#wd!zJmnm(ASeutVVZN8VtB#Zoj;7BJU4-a z5?MlDvuNT`h_c5zw*b{Zi3d_%Ui6o z6Pvg}5J1^xPf_U2BBp<{s#d)!R!iv(>LJ##ij_+jD`Q5edXJC2BcL6* z2gV#mRQ}b9bwW8M$xfpR#A|`8 zQ9&^wWgT@{$x4-oUHvR*Lo3?RlD4#>{iLtqZ;iOx$kMjB#s#8ngR9)-GPk+TeQs%i3*F5P*O?s1bl-R*L>yWagS zc;6-6@XD*Yt;H^SqxuZ(lDEC?eJ_0DE8nV&cfLb1?=$kN-~IBpzyALHFMtEQU-sI! zzy>}rf)gCw-YI`LNca^ngd;5B{}Q;t7QQfsGpymx)px@*dhmopEMfpx*uy41F^W^H z;<$8p#c=`gh+`b%61%v@Hoh^AbNnzb>G;Ku{fLu>A>$#Bn8rOmGLn<5_)jy49|JHLPPT>siyf*0uhIoRG_ioK0^{)xQ2U zu!Ak^VH1D5*o8hvV@<8>Wiz|kQ$h7*iY@JFQ@h&MzBabAt?g}dyW8IObqUeRG8WSL z7Oyxry3?)hb+fzO?p`+*R}ENa)4SgGuD2G6tnYpEyWjr)H^BXELS%QO9(p>QrxX6> zlbk~y$-XzlBQEiYi`w8AyEw)(uJMg?yyF;mIK>o0F7lC+yv$*pE6P)@@|Cl^<%}+u zD@g+;mni22G6DXVTIU5Ff1d37um|E3ulU6mzVVKK{AcZf4PQ{c@|Lf>I_v@Y&2zr< z;BY49pLD=;OiAi3=UxD&|n4D z%LQg&4$8$2hRz52;46&ay#yitiGmY0pb>`73MSzKz918dP6USG=0KqhQehRs#T80l z4;s-AijEgzAsC_{2!c)-ijEQzVH$?c8ctvhs@@I8;atq2e+SlK^=%;#=93WaArX$D zA0iV(7r28p@#P zxFI6K;V802DRxd4x{*B41L)jg=;WdFQDQAxVk~0f{w!*uAaY{8cw#Ps;x39}FNVc0 z2H@6^10k>he>mbo6vB%!f(|lzpfZBaD;nT1VB@^dqB;&D=ul(m=pyJ$p*v=yS8U_v zhLg4M^WJlJEN_GxQ2B1q4AS}itM1GD$dY?^p4o=pdHQpmnLf(DA$5uE} zn{-7{X3kLtq)DP==QJhfJY_#dCFeNg5w@cl4*sPpW+mrrE-6|CFkfQhXE$ht==VrJ)Pf970rrsjBN8~Wu=@?y|^10|FJ7q~%| z1PDk7!cf`eFa%E{4At&@Lnf$a8Dv6s*pV9ul^{S448?~r2*P39!hdqXBz43?l!0`t zXV%=qFyw)LQo=|yQiA5eYB&-XG{S#=hIIaRgmc;pbSh;^rs6SLCpu#1y=dnp+GQr< ze`R)pCj{bRJ(6c=n&;2G!=V^~dUi%$4A5(E0h+8vCYVhRB*E7>N+TG7k0wFK5KR*F z02h=&qM(d$I7O4<(`ZyF&3FWF!a$Oi2N$#>B^+iM*N|*M=r96q45<$w`4_1t5 zy_Be9Vjqf{qlzvhi`t8eHY1Eaql~sFe~n^}c={fWa*jq8U8!7><+wo*Gy*jR43%*~ z$H>FgB>Cq0P~#Kg*g=_xsfBeiM* zm5YjuYDhsr5SRi=48|>_j2qCIl8O=jRoE%L;Ax1;3w36WWTGWyYEGb1<)F@Ff1v{E zq25cPW)5gFs%Qe;1yX5jG(r-*48;(EO)Li`Xlf9Y<)476DU^qm5-Ajf%bb#GWVl9~ zh(ZwH3F;^kzk;c$t_F`v&|2w6BLFL~6bHdR$CJ#8fs}!()M-d!;npB!beZ(vex+X^o1dLE|fhouXCA`pJJOjIK zh}j(GP&vw({zq(n+(3J|O zYEOB%D!|Ul#kl4@C~nXWe=V3y>TsUJ7D3Jru_|^P=W=+;NQBD|Q4FXuD3wl=GBMDA z%<3~pYU}z5N>EIil&yipu4O_6?Z(Sdl4Qr;?#jkX$UbM>w&>h$PTgi^%1-0}!%NHF z%gZM3-|pz>27!|vL|#AxtN!W`LA{jqGA_Hirp+3`Jq_q9uqS+;e-9{)#_b%ZO~7ZX zo}Yh+t#l9p%?OKUxWR%D4wf`kG_@}Np6&iVMgPLf|DHeqKPK+3r0zl`@8WF&Yt909 zX9Hv91HVfI+sg!Jj`9v2dfG%ISgatNj396=6cA25q%b8og^D!7NT4UIaxHwA2-oV+ zbI5}x;YmTb0UCE|e_fe~dO|@GltI=Q${hz)a}0wYb8Vu8)IDK@q406qW-J{J139ok z7BE2;h-2HT;@z(80R!q3XHFG!j__J5@lK=ha^)d^qm zkqqP*LunEd3jwpToP+)wWYenb8ZjG4mbs>HQdyaV4KaH!f3#8Asz^qCaPK%~#1X$s z7Lo%vTLUqEa*Bp>?T&KqmNMp?^5&#+US_eoY;nDGG3I=6&}nK@NkHn&c8 zkW}`&T(-Pqwq}dPU=K-Qe-3Ed%T+h)O+&Og-wQi?bx#j3YXgaEy9;c`i)_y}Sk$(p zonl)be|BVwwgHp2yO=g`qjn1pw<;I+j~sWqBsaV)H*-e?=#?INPu~4$tY>FVZr96h zi?VgM3wGN}a6>kCzjG5BB1tRvS(mqYPu}GXcrL6r9kw^-yf?kX_c_luyVQ5R+;?i{ zH(2lYC%*Du{>CzZOGW6VH|cSD>KwpRbPeQUS9sCMt_^}K*LyNI`V z{r84%g>whVZId&fnz%Qk_*|=agtK_PxH#UzILo$jUt)N9KJ|_BFNgypbQ`vJ`v`;k zFOV~~kkgBVZ?}(MHxPz|?gmXEO(~FTqbeKc1 zf0BPD@_Mn2qj@E(d5E*Q+nQjOm-d#=OP9Aeg-f=DS0j_>sFMRaN8~t=?6?3I`Xt7= zy3Dzv*ZDg(`f5M=jYhh-YB;3_dIt`=Lbo}Oy!nrF`jB|KyexW=i~5VZ`i_jbx|F$B z{duYrW&V_Nqlnv!f`cq|+qA3;39ZMAf34d&t{=Is>jJbNVN;ewXaLILtLP}LQLouxBEy;e|y3Z zD6z?(Jj$N{!Yh2mH{znMb;-?kmJN`Cbjh47k<>%GQU4B+%KIGR%=3l<(e?H@rzN+&e zz`s4*Grim=yworJTX#L)Up}5y$wciA(d5@Fh=O0d!}_P9Qc1rk zP~Unj1^br^aZs8@L{AwAD|E;|Kp2zf&)+#?2SwegN01@Idd5s;5@qb&vtiH}T71|3 zkmI>qqC^eb`0gM4cq7)XIFCoX*W4BD%N=iYsM^^Ksxh^C%pL}~MuF`fjW z4Liq?X;XW?u-R)mm1d_cOTtiNv5IdHv*sxiwX;r(HZBt{fmbR5Umu}Tz zw>H+*yLaw6Q5g9K7Hsv~tFD6)Cw4e&*h0mR1K$z}neyImqT*K8f6VroDZraSYt1?A zUTD)+<(V?(_;l;U@pxBW;Xu#tlJ|f(x@4o!}X#@)4dMKkS00qRb zz(x+t0}6%KdMYZ{xMYYcufQrxOcc&at4qG#%IwV6?#gRTf7kr_>`l`OJ8aI=>S2tv zJ3*7oO+G=pOie&5>r75UD;sS=L@7&cHAW%jsKFeNd}X<30L&qekAecwh&)7b1HX5m zfg%MX1S!f5NhzfASEd2PlHNr4f+a=s^&9c7=fmPz4Es3tNfc!p?@=gMkZ5o-w8j7~C-B z#9r$$g$Q>WA&seg<8uQfyjJwqm^^ZrZK`KvrPWnYn%pQ>b?X6ESYnMuw!%E>ISz&- zpp}-|>Y%`J+x>#bD{kY1$-y2S`08o5Of^Ui_B zIZAkXX5KuE8I+@8o@?Y8>2A%nAYM(W*gTQmX6~Ctf^ef_D1xASMN0m^d1lG3-Z|$R zW30^sf86v0!EG4j6l{c2y$O0-5S~OKqU3t0)ASrujOg_puzM0j*uUi&NWA45KeQp4 ziGX*YdLZUP;w03)hmynJF`c<{7563b7!Yy9ako9fEbY+l^ ze<+Rx4O7sFDnQBsC743J9EdB5TUiwimtuDv1&G!=4$P!jL@9bt zJB@II8 zR5FEut8B;vT}coL#*&4S$s#JhM$5pAe{CRgT;Vq|Q6X==17baJ1GEH@2~v%RW>O4h z4}91@?FgbrP!L6)aHXpp_OKBI$wcJj;k*(+ZX>_j_|)w3;>q~#^Ulu0sevVWe8Whh@t%2=L~mI&0P zKUzskSjKXew5(t(bNN%RImmOdTcJEEVmfTS!aA7A98cudB9o9rs?FrcJ>+qYVU!^U z-3i<`0M@Ht`Oup_e1|wWR;qFme-WKXG{%xBp~S}3Foutil3Dcfs5@`CfsVi|B%Q*d%r?K>@F8+hM(;DissIfe%e=C!!RJtI~ zgr*RL5mNBM*-fN%C3BB4Z($LIWR+%&RgE6u*%WAGVi*T2t3e=_iC~J$SCr_1MfEtF zJ33CT<|NOPE;qX3-C|WA>?=cBh)`@*rLYAP1Vnn|o7gy(DzJfOzix30%^G1_Bg#bk zfZ1LuPN=F%$}UCmGunief3|VggF+cP`qB3A2O<^{g(=eGhVb#qPM>vF46));;T8o< zyR~0$I|-Mc2G^Ct)n##Ad0bj1*OfzcWpiEm+*L+bx^vl%33*diXo!?~%3+^BUcny$ zohNSGp_wt_+sELxW*tM>(DWijrHS6(=RV^v7 zPPzxDEh-8lP+}vT_MCfkEX5){>A&*o0fCHQV~?_$AeI(H2CpLICKH(;o&d`Uu$^sf zYde3wRkALbyh9+CZ6#SN*u-*lz>{t6?6`Q7X&0c zAtv3Xc}!6(G2ta|h>d!1i>)@?v-dC&URbF9?|>(Kf0L>f{;rVB zofsEj|9!tdJOQ5pDB4!+8T*X7|? znfOvJzLbqG<>N&edC4P+LqK7Jq&>K1e|z!(fA}+J_ftV0r1cMEl8+-O z#QpSdh=Qj6GNJ#fLhb~Q?sN(JN=f@fiTl*e`+7>1kZUrCYq@C2{6q=;K1u!74^RxE zy$&J;8SqWc!6x$J9xNgSQ}9pJ!XRYuAb@NuZcqh@rUgB)rIv5V6e^Y^P?RX}lPpm2 zFwnR(e{jk;@X3Dg$~;44e{Gk>N z@fJ}j7cFQPA*dHM=odvP7(Zzkx3L&M=@=_186jvHoiQENkuB5#8{F|7;n5v~aN35j ze-j7srf}}3w9$gN@sqr=e+0vmkcAt5p%A2JuVg!$?b z7w-|c@-eq`s{1Yy7~v4{Cn1O@704$c2q-m3 zC@qL6v(hLn=p-lTAQfmOpYkkOQW`^%8gnuu0dFI7DhEf=4HYpY!*3+33@j%|EFp+2 z4ah9fGBDLqEqReGMTsgK$SMs8D;r2FEr=^8$SWIDFCmC88^|vQ2rvb6GiQ-7f3wjr zKM64vNHGV9F%`%$CkQejNHR~8G8@P;6$mo{NHaOJHZ9RJdlED&Ni+?JGy%vo4G1+M zNHrUXHG|VN6$myBNH+V2Hf{4cu@E<}Qa5$cEwc|Ug{vbk5GVc7E>YT!c*{&Qr*H!so;SztnV(>L8-{k81K{FloGm_ zay=1L7TdEl-*bYz!5iS=ryevye+nKTv_ie%Kb7i1H}pa~uq4BBKpn3@x9mU@^hEn` zL4T9l;DH>HVftV+MrCwHX|zU{+Dri)e>6#zbV-@CNuw_r=m9GNbR0$W{74ku9tA}s zCFpQMGl;J$Fc0WZZ*7rHXMQ~42DWWhX|+~u^;U5;S9Nt)d9_!4^;dy4 zScP?1iM3db^;nTLS(SBJnYCG+^;w}6TA3hHuVP9GvPwhnN+HigHT6!AC$Nl(JM}_d z1ZQDb1RCCfn)GbFL;(y-vpPhouk%HfMEqXL+_~efDR8Hfa8Zc4&#Ve`pc*1SnQg zKeaNk6&bZP{kYX*O>a^5ME3wB=H6;5?qMOoWwOHOt2D>@z=tG~W3n{IO)LjxRn{M3 z)=uu>Y_FmoN<(v4V{3u#8wLejz9CgPLT;rfpL$|%4GM27&ub{);Hb=Ik$5?_j5rvV3D?BtHN53l2yy`X-6<>rB*TIVd~=5 zh6JI8l!y`E!XZXw*}d5h6RJnuaO;Z*H~5|X71P!4&~16jO^Bc7rD5p0(We`ezwh?hIyj+nrs z43sHP1mWkPVO!22abeF5V4z9p#xLqO^lmFE>5(DqHFO!cfgSjPAvl61c!G&`bd~m} zGFE>!wre`Jb+h6gYNHJDDRHu6!rsPN+@K!op<9BWK!V^}T&JOAPNZ-}Nakuq(j_?D zC7+1pJ+4LO(sp{+e|AMUMxR)uLZE;>rUHm81PTIcHtNAzq{KcNK_2V|SIPiysNxnz zq=o%vonEYt(9HelLql}v*=)!*(Dh~8#e^E6C7ifO6y|{8CxNRCXDK+3^>~l@xR3q# zkAc>LtMy|0Cxhc>gSDoEJs2ytgOQE{LbC3{Qw&J;qMgxZo9l z!~UqlnA*!)1cx!6fkKENUW1r=(I|u$0+q+AdZ*%fG(}tJ1+5|r3Ow#@rpk3hfhPDi zZPddJhJmpNBqZ|SzJ5hm!pl_>jALxb2+Ap46?am&AnTyvZJdNAB9|jvHGW_fj{&)y z&H0?sIi1xRe}VKa*MI~8xk*fw_W<@v>hC^V+qk7;Ykc2`7j&LNe9X_ftYF`lyjQsg-)E z1-6|_myq?xki!O%uSSvWxhmd)sj9;b=EF*UtqgjgjhI3n%D@QNBYJktALPNH&mppw zBRQ6+L{-)N#1Okuf>3&vW{<3fytSvP5ZP_JGE6if}MKQpgMk}x@)MK zYOMOIsp3}Lj|*^YZ4CRn#99D(K!(3k%KEHN1dY~Ou-e)@W1%DkaVKRt|^B^yLnG#GvxesH# z$f~tIOQ0@lK6HAs=Sr4{*M#O-Ey6i$#@TXL`@j)A!4-VL8Md{j!gNCvbq!K=t+1YL zn=jhs(nRV(RtS53n?Zt`ZHHT}4Z4Mv!|TF|R4PP5_IEjE2eg2sdQVn}_u4j6!(EI4 zNPK8!{eQu`7XrL>OuX>}CoH98G$(Gjd6?NdFGRsyUg387SBm!b%Vy(wbWC1^WxsiP zh4^Z3R|E!ZOTZ_06Dya&&HT*KJk8bIERuGR-#IzsdBOwSK!f_i5#yi)s~+?S2HJbZ z!n(pj9Ic7_Z5Hs$$X68FCVFzbfmurlUnzKAa$Ye6Vjr zygw$HiX}hXrV*r@M0kA3WvN@<1<_5Rh1|gNoLFV3YFx-F1#UyLOL|;};Tb3))|DiZ zzL_BgylV=4&5=FXm3`UI*} zS%0lV#8+%cRG_J~Vl7%09XZ?pe4^!A$Rk7SM#toVSmG7VzNM{4YF`%JxE}}h-B5pR~dbXZEpCNAAtLTZ|vy}dD!(OfPZ{yl4 zFRq6%j6$u+e#~$p?Hj@%Btv8?0+roFta{z+SqSLa7VM#lKHF>98zSYe=GbGt@qZov z@gcv#Xa1>Ao>FbTFL9m{b$;h7gVd1Y^H~H-)Z!j)Y5o?)U7KQ2L=N;nzp{;Qq+*vT zNS}q$jP>1Dnr6Qta=#0$;#GDZ2oZR+3BnpCq4n*Pm4Eq}zxkd2`Jq4hrGNUV zzxu8J`msOzwSW7$zx%!a`@ui_#eaYP{K>!k&A~vhJ9_*G z(%`vU#t7b%M=zzyl`LDjd)L&HYuG)z zeEW8kXUbSTzk~}LK8(2SxuA@{os*Z?ug%7eA5Ofv*5XvkO(Q49sL``%wj)c9J`K{@ zFm3wQPy%Zh4!HyMLg!{$aOs`-Z0)o^RanhB*^H9CzBx$z8{t+gvtYz0q3( z7F<2M_U+uedmjjT+k!@Fz@|m z*K!2%_1uAcO}CeVZ(TQ-gAz_iVTF8!XCX-CWf+n%>2df`djo=Kk$-&jkyz1v`Jq@5 zf5Sw`B1L(P7-K{R);LjuI8MaijuAoVV~|1)DH?`E`m^DY19_<4kRmEMkcm*L2&F$R z!bs$dRo-}{j!O1;B#>2(NoJV^8A)W4N-_!MlTb>jB$ZHJ38j`(ss*K&M}kSDm}dS7 zsE}w5spgSvD(R+@aDN^tr~Z<3Drsk&cq*wUk$ev6XP}yH`ky`OdR z=#hyYxhRp19{K2zkuo}ImvU8ksgRj+%4@IN*@KNQ!VXKUu(^cB>Zw(xO600(&Nn4~ zRmPg7fLqo|WTivq8sx6NehY4H>R>}|x#p5fs-nt5$*hy^Re#y)kgpO+?U1v&itUiv z3dyaH-i8Zsz@zO$aKQ!pGpLY-64|bh&Jqf3oYKxY%!_qeduNkE?i*ykJN_$h$RdxH zaF7cR*>I2#D;zPS6FXWlmiA&Ssm4Ka?4!pxf;@81KKoU&k0%S6vX3j9+_I}L!y5C& zGz!UOwzop5bAOFI|BU{1)*%JGP zT;u4q(H|oE?we#oZT67$3K`(jK}N0b)L-V+V4s1>TzBM>pP6p5D-sFZEwLR58RTekQ=tH8e`s&_<-g=Ow_ok#e#>MUA-(6$gJAMO=e~cDi zr93!M4Gr5q-a|a(K|=G5^c^p+C^UxqOFfkLP~W)k;?E6yj2pr;Pc@eml>#dsfedV* z10M*%2ug5*6cph2F6f~0vFLo&xP%8k2>zE{l>!_Etf38Wh?jnq0u~ZP8)DEw_ozqs z=%zswaqtq4sF!pN1|okO-w4Mz%5jc#tfL+8h{rtYagTiLqaXjs#w8x;9Ip_?E54zi zCx)aRj8tS@qDZ1C0& znJ|VfvqKgB%RIvkT)3P^hQS3Qa3Kjf;KDF~Ryk9d1|A(`raRvW&v?pnp7gAzJ@1Ln zKss|h^$i76iP4pTr2A)*k48(_eNMi|4Q z{z%I$L?H;Z$m14*5CkQ1=?_>oqN%MM2sAR$R8y>lKcW~0B^sg8{5+$Y_nB%^t#S(} zsqyr>{@Ac(cxBAi?Zsw^kMOAh&xvh=O5eea9k{OWhVShTEf zzL5tQP~skx$UzFyo7U?7lnay?YD8|)gL2mO6cYkp(nk*%s9eL7>DtM9~A>-s2oO z4l#ch9+HP1h+;3#C_@sA@eViSz|sB~MuxR2YT4F95JYIiKsy@gxx53g)jbhNs%)Vv ze|guu?zOLf4Q%~}IXA{g0u)RE1y3_LvoCVfZFpa_eaSw2+ydX=>gAvgD3@T4tXch0c+0Tylw5xsXX#DtW-a(0K6JZ#l zsCf`vEe~g}{O((Y^or#nkF!)1FI*6?pebVc(>LEvO@#GPqiV}ZPtxlFWveY?{ld^sGw6DGGZ|`Nd%LbbMJxDGu$S6$CjD1WABxTmTKPKu$@pQO#8n0H#VS=w0WK2cI^2)AVTimjg|R zUgv;HBp7LMb!!HQHwfr}S*U-7TgZi7cu5fG8uh?YaPI1sQT3UX*y_wWpTa}a+Rc2RN*e3(}Q$W`<44#t2n6O{?3@C@n3 zi2eZ$PFGaX_X?T#3PbljI%P=ba7`3}YZY=n^RNvSgo>%CimS+qt>}M>uLz5=D2ua5 zi?wKrw}^|b2!^r|J~ESTuMlqw;Rbood5#o#XE6}Jm2?Ji4@$9&$+%Dk@mmNLRhFTQ z4bqEB;Y#<=iw(s{3YCmXL4cxYg~78f15p;~cohCX4_v{H>IjcTQB*?lj`3)ZJHd|c zpgrh#kNp@E>?juX=#PH`*%DY66oWyK3CR-fsE`ebj?j=20@;xF$o`It@emWKkO&tP zdx4P&Srzmkkq)_w+Hns)RR$-ClBDJ{qevmbQzO8!8c@;J3DEUBk3zSRxwZMbfE|?EIA=AStEb3)Fyz)Dt~i0fg>a? z0%tfOBbvLJLFN8vKIdpCd!s6l2{Dq{ zm6Z9KmN_$+i6e-!na0^Mxal}-$(z+#AEY@*z{xPe*(}7#GSJzY$mt`n=_9fUHMH3_ zwz)0Ri7C2io$r|(@L&(~NuTxUDZaTu+KDpU=`PF}Jgk4&oUS>Z$|)q~c{u2~BZ3(t zgh?Wx37-#o7{F326Y2}cQYs}So<4$_ETWqK37!F(p*TXGJ7S=DiJ&;5pe(|mAmX49 z>Y{z&Dd!@i!SZ?-iYol+Fy4tYA6g@NS(z4-HvSnS&RLs0I+*OKEAIKCOX?Z+>7?_C zpHifsM1y~$Dx;%gW27wNqsa-ReJP|`dZIOwqA0SW9MYoxOp2z4ft>_mnk2HJDAJ+k z1E4)BpgS_6I8vgS379A{rX*6P8FHp+DySC{rI9qHUsI(wIv2A!Yoo1!`3gEoB(*^ux##i9)QqM|u z`9Tir0GDEU13-T`tFa>qul2z}m5@YJOSM&NwO5O^S*x{M%e7tWwO1ch(0h%N@PNAk4rp=)((xQC0l ziEDdwQxC5|1W9mP(V85MV+=7DmJ_j5&1D4L_6lDrxP^b(OIWD5qf5G_Yr4EfH}X(m zf&jSYkyAIdBzYhRmn#vE#szL52p;}rPP{U>>ruF;3%tQAyu&L~0P}7f=oou3X~Z*Q zIA&Py&=ffaSKnX^#z2KgQoYy{RnOpJ?;#dcF|m(95H^8O;0tO$M_jjna^G+ZQ5RK} zXbj9TX-a?b3=^Aff7m?6pjN&$5Y#&i&k&YySiM7MC5|x=lLm?!3BA?37TOlS1ObIh zF|k%PvHnpH4y?TqT))&~Jml+o7mBN~_qN1K!X<3NC(KL$Q@N2>YIHRh85MIu091L9 zt_FcrwNwTUkq~*C2iQ@$dD~rs@eL0}T#X=DFn52FxLaO%CkTI-yG8660ai{;oKgHM zf_ZyeQ-*?#&0silT68#Y{_62E~b_Q z&~Sebf*@|*g<)lIUN%Qwj&xo2ZTtpxU zoaS74g_1--1fpPOO=w&kMO;deN)N|xTp$q7pjN`AR@h|_#$W^*2FKQRU6EQ4PZtCy z*;@THUYQ5Ef>B`BR$-|8T0wAkaW#Izl>~nuD9vld$q#jsMBrIPHGUl?Z*H&_;FfP` zYY>n;9+G^?2aV7PtO= zl}ab~4K?R&eP#qYSbXWly*V~xf&fj%mL5x%Y>Rl!Z>3{Ba9q%gX3+-)NkC`)J7a$) z=npX%(gI}+9VQpmQ{k0(?oR#$>hf^b-P-iBc*P+e(Nb@wp9#t>v|wr^;5 zNTOf_`BrD?cfIcly7DEu3r*K`ZP$1G2mmu{=yeO5`%ifW1w)nt1Sb(#-EtuZR8tkt zT7}g~AXJ`51nE~x)I~i>FnlmrTl0TWTmH5bM-^nf3~t>t5bJevqJU11tRsZa@U6oMgrP z-VhPrgHd`tT)qD1TU4UpQGq=S_}pLv{BDrjt*?+_V!VDk&bo~dbPg`saTekJB;zc; z($E0SCx;{(p3O$BWLKqUIyezxn1WA;;&CQUDt>S+9^Bn_+{88GHLicpb2kh}#=5CA z-9DZloK;RA<++r@wpZx3-`(V?uIj6vfB}O_FpvVct1?OE)MGrRN4qk>P*t+WmBcO>GtL zTRqWKb+rp>@^D^dX5N1~sAE9o=qzqwkq+aOUR0Le+{WN{6ARMR{o~&S1L9W0A93W~ zk>spy@CT3n@W2)@WA^W3%w)^=5Vk%MN7s67TzKoP#!g4a*HuoO>}TT@Z+!L*M1D|i z2JvmIR_&#KL0}5S3z1K%A5pYUm~ z_G_PC0<%p$99OyfS%nO7*CYzemwJ`kUbh>+l(6@NCQ)MEUS$AMM#yIcp<57@hdCb0 zg1}9VHs$MGy_SC)_kaHikO2)bhf`ApYB^3^K#Xo8xc4+{J=?VRZ2S(4Px!bybiil| zqwNL`{t;(+Fvb$ya_2BrbAkJLz!ZMp_}uhXf}m&w0qS!vcBO8Pr_T1v&-~3#Ujp-P z)ng@;_8ydmdVkoy+3$1~DH4oW!IM_M6+FI9X}#QdkQaYNJjR2;lve)wXbdb*(=7)_2-u-O3;4* z1sSHH{R*7znMMNqkCKlJ^z55P^1w~9*no2ILdR~42_6hPBnlpHhEc~u5{L4}nGs7= zQ73<(I0r6aR$a4NrZ<+9((isvcr9n7r8|D@|>!j@* zQBKp7wr^gU@Xyrh8Hh|mQG3OpL}&X(lstbM?d%y-{FL;wcWxQ*QqTG`22oB!6?Ifn zXJfO}$KFI0GGpdk^|3oUdKD=?EuFQfKnJ~*sCtGO=AvDT(p1)9hk7*9Vs)a_)?|0e zbkk;clE)NdfQ|NAY_rAiRBioHwOfZ;MHbsw!Hua_bN$IRU738fw%cLZJvQBC*?oUj z&1|QwmR)}P^*1MOwfz=canmg~U3AS|H{D);O*h_j=QX$9bni7+-*^Ajc;jpS1I~8f zZVNW|U~>}&cVTlI&Ny3$6^4}FO6}DYUu-ev_-33h-NzoDd-j=Ub$qh0TadLC8C;UV zHMv`q!BzQTMk9WC;+QL*`C@8U&Ut@puA#IC8(+c}d+e}tkxl8P_&vJYq>1h`VnNw` z+T4~;o*G<=&9xX@o4fY=?;h<@2OGi*H=J;xOHP~JwY6rhC%1KWm)2Hx{%vW8H+s9L zZ?^huudg|K?(4T7H1DGY|61{{O`iDWF|Bz{dtU9>cRQHDE`C?z8~yJ0KnOY}f6dxo zwB+QzWc}}Z0X*LTXIH>(Ey{m&vCEvc;D;^wUCVb7WMK;db~>UhE?v&JMT!V^KUjd#yyTyrxZqO zg?416_V5@iRpBBFH=t#gjxYlme@tX1Q<=+TW;31nOlU?^n$x6aHLZC~Y-SUfY_Nu; zQrWCkwo;s-d*!ZF<%uDj00rx0XFJ{bPI$&sp7W$9+y3)VO#oW7Z?aE?=6w=e|`^-~Wtg^^bwf2|Ejxfr8z z-s+s`RA)q8`cjz2RHiefX-#c8PzD#%0L7nlp%HcfrcRR z;6$N(BM*YO31pxlL!GR~4NLyr77PU@Lv*&Zr(-2+SDB;4XCAZ18uQaY=Z ziq)-~z58A8hL@+N zB@}9JrP|esN-58{$T#Bk4qurf2|38aFw8hzW0*p}uXx2RcBn;BG$IrGl7~VlMF~+n zgFu0zL?$u;1>ibX6b;_~S4dt#2}&fLmtKm@Y~+ee~27Fp%HDY2aPG*SfX6^RhboUk&S$0Bqv$POWrJb|76}-p*Ovk zy2mi|xQ|>ujp|epz!9cs>=;!C<7OWV1%c7 zs}XO0a}Kz06X?F#2yPgLy7mm=!VXrc5aL23#$}8$zyJnNe-K2V7JH9gxk1fZmWYer zK?zFl@D*i%LQMp*MJ~V~2T7o+G2Bp-Mk{eALS7Y-nS5(p=UUgh=5?>z>0~`Y`BhS$ zvZRbb1SlBe8*WHJ{trYU4@%^q5tJZ84i>xAOf@1xNoY2nzTD_C^gs|Gg@UTfkO!n5 z*Az(ID;NCCe?`_THr#?ZN0(cJ;nOJk!fL(+ZJU|{af{*|nIJ+QwCILOPn)kThJt>R zU<6Ts)U|GC1Q&vk38}&W&!#|7jxqcSHx7l?sIs-MO@4Ber(ESLw<)ku3B}$?vDm0; zEfGv{3obPK)R1lCQAJ^tTR@>T?=S`wUJH#(7=a5fe{pvnUg8EK2s{rkXvAxRp4%?> zAQR7^L=U#yWg=jlK)-df&4*$PrFSIhf_TR(g5YeNJQmLu7ezxvK?E*P3$Jk}^MfqwIczsjv! z2!b3SK&KLg9crp_(>joHH{q)^<3m1&Si5t`1ZyLf=To)+^Apxcf~lG+Hp{*Qe27u& z0T?I)s1p7;@zWpjQ$HX4K_Co5A(SchD-&T`6=I9OKN&1AD}!4Q1XUw~Lfb#k3yNoe zf4&Iw2V;0RfzuLo(XyLxJ=lZ3Ot^^d!#XH%w$96v+9JBCa|4#Zj69exdJ~G{_IET-RsxK2aYGY`{r-K15c!h{)9Rv)+1(XLtNC7Cw1W<#; ziO7Q#$h&9Ilyi87Ob`?(ghNHU1yPtUO6a1Z3xYnmhqbG{6exp5>5~>qf}0D(dUH2I z8^!_tF@_*W0*w0xQ=o)aX~ahiiby1t%sK&=UDO34mngFV7k_?4flaiMPArvA1VuQ> z1QFwmVVJMtJBL54LY;uL6bQ2dc%KjjNi{q09q)8pChjW0w>$`6}jLtWN8^PVBr+MsUvU%uet8PVfv*@tlP298dE+PxMSr^;}Q(Y)|)m z&-3ig@`O+Nq|f=RPyEbJ{oGIflukzQ1bP4}BXmC%lDWPN6-Tc5`OOCfJ`%016YV@Q$xxZF77h*3 zm{|VNhp^DW)4{g@IIE_;weTg@fQ#(yl zFU`$5y;DBT(%zzoCmqvE(aTZc%L5G+uOtn^0=_m;EJf8TS)-EVR4=UR77!5;9zqdz zX^|xX7bW>29TCuE`H>yj)EeR;8i^y?QY8aD)VX>F{)$!p0>3u7Dp}>PFx;#Q@>*MDdMS8Neimho3F0$A!1Sk56>j51h+4Ouu* z*hXR4U2)iAu~!O`*k+;FYq3~tAy*QL)LTKMErVONpeSrsAKL^0W0LD^tY*(+k% zWO3POf!S-3*?*U&*+B~GRipV;wt-l;$-$%5*NWxWi+|10 zVv!iU;n>~r*r5VhtBu?J_!_dQTVF7nq2W=r@mj|L+t-1cz)D)PjaiJHSdD!j{+D#UvjTWZOSf_oJ(+!r?m6X+OUFC(1%FUz8 z4WG;%AGC#~-3=C`ZI<6{mf$tq;a!#D-Ie2Ql;mAr@|B?Gg{9}^pXl|U%>|&&wcgJa z+#oWb>D?8z)ttm#AUTRTX)Ry>HI?%*6p0xm0sR6Fn0`j*KCEe~tl?k4e3MP~b&i-H-b_ky( z6ri0Ifmq>E5r_lEtOUN5>-F6R*4_tB+XxPp7Jii%{*xG<;S8?fKe^#m`OG$=mns(4 z5eDJ~7UKC`pB{dd`!%8beG;Pmojhc9Vl>v#yrtckxCe50m+KJ+9wp>Eo(%IL))Eeu z6E2n%hF~2gl_Xx3CDxNBPUG^0Vm*l^HedpmE)oYHe-`B|zy(u+j6?p`EjHmHM&Tm1 z-Xq?XNT#{PO|8bYOlhfSD~V<}iARk5XMhf9 zfgb32e?$RmuH{tGW-!iWMc!pb?q&P^<(ehubRFjND&}GgMmA!Q7L{Nwl}QdojLzu% zFa^@<=%l(Y{b(^GV2Um?zykBHp}4y;(1}m`e?6f%2VYqcer}UVlU6eHyY5hjIFm90mcxwDE&7HoF{yhNxBdpvYiO{RMSKQ0 z(!xRe2(fgd!(_4CtWk@|Ykl^XXDCWi>o9ffvC$B?c5X&DIEclrvH`0t$#(2fbAyut ze~Dt;>l)1xG^q&0uI$J7>M{vyv0gl~PUtM&X0*PQo!*t6_T`@rm7pe-p@x%6Ze24Y zO>GkdjPn*z$OAXX1M>L>UVH`X7KD9iyY7a*Iz*!GeudYN2bOjNH!wiPhQpa?gj;AV z(7bNxs)q@?i5}ZAkSMO}cC%#j7IieUe6%e=7#p>vt9nygNwGk(A#WQhQLGe;i@w8@f!Ug9PcWdH> z>!_9MVWVqfehFx31PY|LqJubNf5?L%V7rop#aCFj+-k8eM+k&tyDn38)?hnP$SeOc zBPkaJJL708#L7}1I#e{f8!np;$MY(88o*FF~mCdoYC_X!WQ$!wgJV{b;Ee_g|S%k0W`-oECt{_wNTY2j9D;%4h@=5+7%bkqfPBouXH zhBR1YiceFuns~V33UVkgP0SdEyW;{s0eZHJw%jYV{(|FBN-zQ#2!cG|GQv**MJI|| z_`^4Y0k%u_E^qnO>w%_^c{`u>n&*t0Pl7VI2xu^ZEvtu~uXD@Le|91;d7_8FJEP6# z%K;4Jf~Vey5Oczs=qvN3`th@R#KU@(+jpbr_m=(lR|)u88Ths?cu6_-WvAqFX6&9 zzln-zcroE;rqnWy-1l8zf|4)0m1oKrU(B9AgAOh8ouC|0Ov!V2Zx|6Gq6VKEypqe{ zcU-s_TiGMiJ7Z;F2zs|N2&LK@zd?HkEl&R`O+?Lwlu}@kSfYt1qL`wJD`q7YS!|&NnTInv6xUod z&VQKKUO3Vi*kC-)h?rtPisqP%M21G$FqR?tnP!|d8Ksn=8AC(}LAa4e9)cKiP98lN zv85ZYtpr7FP(U%vau&M5QV1}S)&mi42$F|ST7M%q%H&`f`VoPUJc1BGi9FM7Nd!gy7z)Ff z+qITu65YL1Ql=@@V@eMxlrf}+IO#OyWgw;`;t9(#+pM$CLL05L(^6Zlwbx>st+v~8 z+pV|XK8yZ>i)F19qp_ITcrIo-W~lCBKDsNJknu8RB)yC&$>hC$J=yNQ{{k$NZ+{rw zhUhRG34-QrycwfqZCV<$TA~=a!3A#!b3w3tuarSBmm9eu->PM}@exh$WRfwWWh5cr zEi5})u_^U@lQ2eQkgS$(uld8VKqk{POvnU%Lnk!#Wa7pv^}JFr7lI^637RmimoY|y zSa*&fZX9V0OAWcTVX*<*h3ru`fPb5|+H14jw%c#Rz3sR#rbTvN>Dql4yLsCMFW+^| z`}bS+3jS8Ug}2o&;)^4BPHwkA9++a^3}cEZw?K1GZ_f~P&oj>qR?jo9n4+dJ-*k4) zFrr-kp1JC;h=RF8y&e6KF@X(ZjPARCLwW1QY*x>6ro8eDG^f{iOYEN;Q-9m(1ocMz z?$aBiUiA(=_e^TX$E3O`v1^ok>8W{Nx?-X?FVjU5hD|KvW|{3zGK8>Ezy0^)pTGY5 z^WVS!{{t|90vsR#3uwRtA~1pbBZM{FVy^cI1#b`x%HAdj6u(*UCxOdgPYB1so*eFj zJ5ihnOK8HGFb!uaBpJ^lqkkDUtPp1x6U{jq(F5Ui(HqVnR~WtF4MP60ZhvZ&U>GYHMlf!1 zj94t=7SD*qG_Em|g45hPI@!r{c+oKR@FXciiJAFujxK)OA|S&!j8_N?i^I6ZkJNa@ zNNRD7j@%+7u_(zMQWA@n+#x14Y0P74a+O$Y;p*~H>&l37Gpc>sMzy@lw5cUr=pZjq$63~D?xS=169RfI|9;8H)>R1Q8>s%l-UD%F9Fx5729 zZ*|`vi=s)a%JQdMrRGV8dcv=YFsvUOYX-~8!Lw#Ct!!N^V}D`~Z1^IwlAWw%IaF7_ zhV+>u)$26Z*niM@s#J}F>VPd+;0zvEgb7CHf?GOYqHZ^v_l;zJ zC%NA>UU+dC-Y|_h+TjU)*n+QQXLWh2;P{%j!6JMx2P1632}7{L?~5_wW?bVZ|MSKW z%rOMv;eU)_sAG*+&axWep^jS|GLXaYMke8*m(+yO%!6DbJiKuWbs%zwZYJ{@)iI1U zr}-9mhBBYkD&_aZFU}Ta5R3`^PE$Jg9cV5zf2A45dz?dGmpsR!)!51YcRISyny%@e z8D}Q}=$)(5hlHLYuH>z9u*0Tn3hV0Sdus z5@lK}lNv%H2~cQZTZ|zJA*=D?g{$_pS#I>BBR%O#Upmv9?)0brLp|zJ|961t(*rk9 zLKKzm&^JKme-LO00wFvxMs@_zh+%BKC(n@)CCZQqM6Xfkf+PCW>u&eE<2~tjFr+TXt2*?K#Th;B1qFNz?*boNZo&KQ|!L?hazyJ)K z-cX^;t+WOdG*9{UKoCH|EAYfh@CMg!&n6j+BS7=fG^ zfq9rh5Tt+=3c@YqL2CrT4W;0hSV*ZzO&{LSGb9WpKw2XZ0hr`K67WVnPy!f|2OBQU zPK3l4mWrkjK^|y?p`?jF>|u8Fz@C%=O1y&|#vZCfBK{FH0yNmr%$)}mWJ1%hVJPyL ze`DF!GT|0HSX&YjV=)@zF(P9!D&sPC8(U>UcYMPwBtZ($VM%boFt7(MYEMn{UmoZM zEj(;yZ9h8R$jC<=-m|19Jq2Zv=rLpdd6PiuSlc56I&%WX%CS1~hz+&xvEi za7P~M#pyIc9BR)(6v!)dg#A%MD&E4&e^ud_xB+m;-%N-?5OBd}G!Oq>-6-bVM~FfW zzyRk3iV;-AH!z1NWMmn@4w}5eE2N@z+yW7BfebEU?(kqu_+Z`uAu|GHPzvQx5@k^u zWyg7+5nSUXE=ueKK_0}QQu@_B93*bw#YV7%H=_O@_HaiMq(x_p(l;1GY2*RqeqS{JOgt`N6HY#faHMIRVLINQ!LrL_2D6|V-D#k)kf=cE9C3Hq<#NH(~0zt?mV%Ef( zgQTIhu?9a|{`7&^*$pa~cD%x=~MX*7qUAx0TYgdpr*pCrtDo=zZ^K_ir3SvDt1 zLZ=Z}hj8S!cY?=vhUe+|B}3HAd8X$}&`Ksa#Cu-kOh5xAPzOqu z-bboL5=5qcqGye`!JJqMe=Yq+j2>JAsss^)$qlImi?W1Ovc+kVO=@Cjo!aT0;%T1h zsktqdDO^erd`fZ3N=z^$oYKTSJY*?8#Kd(H*v)4z%0q0BAv;0=5oE$QJfs^4Pnbe! zK}2V4v|?+Rf++^3LNv$ma6zUtXGuba{^4y37mQ~f5~drZ;9ydMf2Bt04NYlEc#<)o zhj(mBUoL5vj-;xj#I3RkC{c(Ta0U#*26|WsgML!~foGgLC_zDJ(e3HBa%;DG>$ei+ zV$nwm7y&E^LLZ`K&)6hp4a1?ZlOZ5f3yYAd)R zZPF_3(lRaIEmp>IO|Psg%_52ep8m{qfZ=WAM#q>0E>_$_e`G=u+(sy>$3iHjNyJAR zRjNxa2ZHn_Mof)HT*MhRP0VFX5Uk>QbOf4+qDrRXEH;A0{0Aj)>5y7X<)w^$T!eY- z#B`uZbbN&0a_QQlM4-Iggp>-Qyqq{@Ou&K!6`tXGh^%uO`F_zDvf1Mb`4w^OGEkxQm^iJ=7LrloSFgR@cox?epf{fChqWNy`I*!8%0>ptt zCI|v0+`_$fUJMdomsAde2(KUrf+9hKDAX=xuwUt90wpAf44Pd;!ivMXA@CRl`I?86Mucy~q{Kb=e;oVH4mNtPB88(zU=IL4uhp7` zoCeINtphJy!Ul722Yc`bgK!9o@CcJ|37hZ({UZ!@g3uF9_#TQ^Kl>h@gD2AagQ* z!xZygGCOlGE*Ww-Gcxa_U*v_4*@YMX3zb38jv>@mepHDe)rtMpnXM5!f-_RtTsOU# zrB#L+Iax8wGm+fG*Jg7mSJf%2QHXueh!K=HixoMW5Rqk&kxdYiy^oTy(L0AxJkRq( zLy$J_3pc+HKXZ^jhjSBula@d`Sc=tCI*Zjh=TkeI5ksR<@IZ7*XOcwU3q@10X89CI zM;Aa37Do@6M?=(#`I1Oe6iHW-Nna95tMpJ~5lhpHOD~Q^XAnl~6ish*C1(&pTmFzi zLl8nUjzTXElNF9b6LnX26h23nK8KM~>lilwvqnppO@EM7PY_jqzmHWf4pt+MRtpYS zd-Yx82yDsKUPIQ<0wYv|7))oCS!c9osWk_&H3Ydee<79Z@Z1~@Fc;0-t-Gq3+jChIrcwd~jv7|WP zc=Kizw~JT!TVHr;FLw!{^F2McQbqSSO?Nm|_m5+FUjH_M1@{iM87!}nAn9@m*%_UM z5igs0nhA4%nmGe8Z;_n=vwkJ{l2v&>U3r${xn*y;hv_pt`1zj$dY}vXpc8tb8~ULm zdZPDEqBDA$i3_Q&hvcF`~1%Xeb5X2&=Y;pb49-IyH8|0z-;@zbpCtFOPryY z#vVypJ!E8Yv_eu9;&a%N6)2&J*$hax&Z~tcen%ExZ`}=EB@j$e&aj-<3oPr(>v0SJb%r- zQ`NoQ6UOD8Mdi&-u24wqh0-SX0M=^`G(5vw;svEC$1^BKZ^lRM_=A8{4_JBaPGz6t6MnnBvjMx(*6cWO3;3Cw2xs&N^&P)FG zbAR`H|M!D``1d^J!#&Ao_2r{g=4-xRFk)UV$DoK2CD@4(kgIfL8tMKQM=NTlbnwL! z%V@>`gqvaVLLze&>C@p!bPFKXN~gYf8pZBSf)Ib+&pab&0k27BA*Fz2(g~O zccRL8gckBuM<{uQsf-B)ikoNt7z%oME>p{#rq200M`>eAE}4pcYxt1TyOC3=jQk8=WZDjnT!dF6DZiWb^8`>T)A`U*0p;VZ(hB7`S$hu7jR&~g9#Tl ze|#8mV#SLYH+K9OvfLAFl6`$<>ZMrCn>lyh`tH`Sd!9*`HceHYDPt*5w|4y+c5K$@^x~O_?50%!(rZl7k4&9C^N3hmpk1 zD`ff6}|^yGB7~x)GI z9DI7`95+&c;uQzK94n9Q_>}4!0s+mFi*JU>&XjN5^eqV^{$Ba!oRli$;u!>UpafMf z%Yf#a54k9d3nGXzRHt*0I_0Be~bYR zq)IbMsh}Lnum}c0&RLI(8T-s3BJu?JtdZUFctxsVjL3lti|A{0L@pY6e`d7WUj==auY?sWk|N>dE6ic`^y22RDxMELd~3G?qqWt6iGurk#Eo>Zqljnz3t@ z%+_cvyUiNQaJl{(?5x)_qLCi*c9Ox&G%Kaw7>k{jp9EsT}O|o=Tg; zFb9y!C}Sx^p(vv$qud~3e@2VBD(Pb)qN#c&LE)NG+Z@80RyF1o_dpUD zr(tSM!mNAv?NE@&!DJCDVZ(+IX@6==mg?Sp{~h??g&&@HzO3$Q$=jveTKN~fZXWvR zn}o&<7@!E^l{|dHVsf4D9Q z`1wZm>~jloiKS@LgH7&sS3DQK5QZ_7;S6cGnB)FE4r?le4fAx^n&en7gI4M}ocCD# zdf9ZJrZ|uw*0nI;+#)9D&Gck*UmMVyyl*A`SX~eqp#1wxxB@ZvThen106eyUlaI|yLO23jjYp{U{ zffejv30qjh9u~2QRqSFHn+3)`7P66*>|_J$Sjt`&vzdjgWi{Jb&tjIdpB3$BNn2Xe zPB#9ur&aB0S=(B{62cm~R~!3{}0d}~!WCGH-@ZPjvrrIK%Wi&1bqZvAcs#^#RX9^WvAG47~NXtWAB zy@it;ZCYKM7WZZ%>FsYBrKW|PmuQXLZB=sT4{{tB!3kFIf*IUk2R|6X5ti_TDO_QH z3tt$+8P@QIIox3ne;C9e7V(HlTw)WS7{w`8@rol%4}ZYpYP%dTw?b3Z8T%upH`X!N zF&0i0%RcPm9G@tae;RnlH#hP$s&0#jH4V9IeCN1O;(39PFH1kc*7h1 zxxFzwo`K~mTiF;;UTKzbDB~U1cE&k>KJ%Qnbc;sd6VHt>VY%M%3Owf-H?vi;j+xwK z(*3r{!??wemn;f5-*|*Gs&YN*C*~W6hS56i4x)EF(kjDwz;dSZsHb#~dYl^7^|0}_ zOr7de>vqsLF7%Fv@d{4sSkal5wWH4*={Zw6&{f9toHu=DPe&QlyA?I5nceJv0mpjA zv)1vU&z$QySM$*`_Vt;8ZRr^gyVs()^^K3+-+*o^-{d%!wi$Ho)9v`AvSL_jXU;G{Lf%io&_B!@|ve4P=H-31O zBcAk$FFn{zpLw{)J!87h*zSDA``$UZWJ?C+@GJm3=5Ic`1Qd*71==(8oeZB;*D);BuawSRouBmefv_gn6r#{T`(bH3xv z{~r6j4?bUqKilH(_V|QaO~9EdJ(?0e!CiRO+~BExZykB*qId@yKN2KEQY1%`B<%tqtq~FaLroEfQ7jOo2jWSn zt|U+L;2}eQVw=7~53NiP+#o%+!W6(`2S>s=ybXxz!4P3%BOR$DOOh#@(kY)3DqZL# zb!Z@ksUT0#81zIm2r(>Pttz~7BwkGvP~=y@Qoq3RCc8o&f+{5lq7h!M)k4A)azfX@ zvL4(82A#+({Q=ds!Y(b%)apgmsA4Zg0xX{)4?ZG)EDIBbOpnk|Pj{3iDkqaNE7LMB zGf1XVg{tzGu5#}RC~=;l8`UX+GKQ(*gusk}8&^|-LLws?;Z&^b7Vt%wh0=BVP>d;dLTqR7^tpn${0kU{t@UWHR#eP$|5BYg%WI& zxcnr44?<wjbKGk z10jPzMP15gV6<+S3c5I@UqHqUKnGIZ3OI5FI|c`6g8q~zv`0muV2O6-A^N2UTuNPk ziYRDcMt!=gsb8sRyM zWM&u)& zp>8;cIVr)O6zD-PqK+=+6<*SREV=|_Vl%)B4^oc>qcqlW9~W{XcUm~Mc|10VKo;xH zAy-6UPH{pIGNA_!=2M!Q&?K=f|RMs6cUBDkt1zTtY1GkOU}K>Rj@_;P%GgI7c1HwMUe6_hI&G>9Bj zasya^2bh3wgmMKkC6#R@Su$Q;5=9y!ca6X-Mt5mZBSR#|zbXYq@Cj9GHX}!;bhsdK zYL_8kpnLEPH$X#s3Uw!cg7--=u6VN(YL|0)MWT6kBO^6;W%7)ks<$q&m%rqrZ9*bL zyj5%|VKhv{dk)fkp#eO0f~?GfeS;u@66rng304di3ev-8&gd3ia%SYAA#IdZB$n|g zRx%4%jn|ls+c+-{_;xH8h%guHN~Z^y{$rm&qzpQSU@JH*oA;l8L?UC#;Nrrom<|PY zVa9w?*kZb)P>tao`in24>qQkQHHu?blhoUq)_ackRu*VkSf$`b!=% zrcA3MV_a&wh$s(=B24@VZYd#tu1M9Ui6)(xbffsK+P78|MN}BY)LP;da7>Bqmvs3O zj`Ac2bc&G__c0@XHI3aEo3mM)2N;gSMvi-kj;{_HmdRAUArG8%d50lo{TM|P>3m?u z7#`+&7(_K!<5oY|LWVg!$iry|#%vPk72-r+6zxG2!aClvQXs|BhPHD=r6ZCViT?Q& zhyqJXVt|grM3NvtYb{X1R7xu^QT})msGvc1$1L|)+@)?+%XmudOdX0@_Q(DZu9 zM!8JnVD5&m5AurOC@0RUBczxMwi+>kCdd>BYH@5V-6?tDS`co*OK>7+(y=Q17l;7( zsec=|gZn9?`fH|Ig{s=;*mM5no?%vz;~UTif+AUex}hs0{!9=k;Uz$$oD$O*vMWH< zq&qZjB`Tsm>8CA+VY}awfhdBzL*g0Cn!HueX*O3iGu~l0$w}OxK@|4v zx!T+6py$l1Ue0y2&h7~K|_B{icv@~IKh z{Bymr9^`?4g_0=Ud@30Jb0k(G&F?&)dXmn6CpkO`Q_jOu#nJrE>2|vQ+%OFtM%uj3 zljDT;94vUdhjK?AtRWdH-O?`|(=%PuH=WZv-P1oE)I(j=N1fD5-PBJV)l*&7SDn>c z-PKo;rkekPxjV#D0AxqMK z=h(%Xop%hKo{(*XRok;~5Rd@zi=pm;=e5i=Hc(=b6)CqE99eR1V6mqvzeHp6!jM>Y?ZAe<|x%YU>rQ>-~>^25-y; z(QoWMu$%I2O0gP#9VKHtJ#;$Oa^Y;W#D z(EXks2&4Y+M_+li4jWD%^;19Ws5z;>e%fr!co z4Mm^yhd*ojPX6$YfACnJ=UqR4-e7;;DBl8yex+=G#`a$9WnY*$U;3nfr3Sxw3ZMAH zpEULX9vI*J)xq%<4)P7n`5g}Wxt{w^iuwo4`ZJ&A{{Ex_zx#n77=>T_{~sXy2^>hU zpuvL(6DkyF&lfg(3nNOLNU`F=bGM9H+{m$`M`N$fc^pZy7~2p1J*ULrGi%;|j27bNl59V-ge$rv-O`|8qDFdi^+>>8U2h~TyCdS-8yVx? z2wC!OiIrpj9!|Wt=FY|yhEBeaG;@TfpCiO8{UG1!2Vr9`NP9cM+ujB04ll4b`QYT! zt6wh>dHCh&orjN3o;vyK>9v#ZeqU+&@#L8o9(wE*XyAJ8c?TbVdBcR19)0hrc3*4x z^(5SO{DlWzcmRecpmziwDB_6ECAi&!-ob;CI_Wtx42$Wh166wXk*8mH8G5&&cO827 z;dUXCDCCf}nTQ>V-g)BqlZ%bIMq>E@f0v#k|nx*5!c*j4G>l^7DG-j3UKiQSjgg-Pb5l-gycbVypmMhiD| z>glJThAQf)q?T&xsi>x^>Z+`^>gubo#wzQlY>-1Eg*IB4UY3P!$>EIYK`Nc3l@>dx zJm(aK)uqoNNe&w)M3YNGGlI(tkvYtnRJtZigs# zyjI6(%k_GA?|}Kz?6r_KNW%!0XC;`Prlz|A1c!x2#5QIE5q7l&8A`vp-k9TCE2mO1Z z8w5d#s14+dOn-D@4)XAgdl)ejx8Ppy-rLG|8xItiU=)o(VL4SlE(DEGmYF!VU`4lL|#E7E! zMjmpI37#0C48x$J18=cQ4{qd;!%3tg*U8Ry8V{0rljL(K`5a3!2b0gyBy%`v98ZGs zp9FbD195VT9th$bg2(|8+>nPS3ZfoLEJN?MNJwK4AqVO!14fMj4L9V#hr`I(EpGXW zSG4aN-+$03NLl9(1V+&x&uHQ`lK{Rmy5W>T0HYvaK!hSfAqVF>C)(<>wt}?Hok&fp zQkMo#D(OvXEld_HB=V1J5)gzYEL$lpxKC`uxjf_t~nCJfB# zy)wKanhHVZZ`8Te&U*H<6l-dso{AlMGRL0D!6$R{nH+u=$DdsV!^b_>8^6AP>-x2TC;@*>_MP2X2VT7E=IXm0)CGxqazD z{=&vx#eTQJ{-}p3asUHv<_v*QFvUCG@qY}!8FHxaDJnsZ8s8!t8L9JqUVSGSRjR#E zM^%m9RcV-2u5DGoSoVh~ewu?GTvUj}oF-b!KnaNcwyhNQ$18q{RxZ$woM<(|EJcD) z7z9BS0@)&3n<+%?imkhg6^JfjKuAHCwG2))#Rn-!py6w@Qew&{`PWZy>)?BcLJ-s~qbOt@ms;<-f0#{3qiZ8Hw@Lcj;I12`@BV3N z=G@mf`7O?Vjnm)a{C7A39yQ{^m`%$+!XOivtB1h80nOaKBR2B*;5|myILy7nlu{hX zUY$vhccAnf@~8(iL_zV2i(+*VA4tFjQjd&t{NDvZ7U|yMn~H2(Hb|-mE+)Z~3$z`7 z`pa>#Xs624+?cMUrvLUIPrnS*krK7M|9KBe$d3z2z-aG`vgRa2`ylVwGWW&~g+|~` zEGKsjlKf4Nw!8i9aF4s(=T7&!+x_lw7ail?ArHL!{qKMeJm57vu)Yia@Q6?R-9

    ").appendTo(this));n.nTBody=j[0];j=q.children("tfoot");if(0===j.length&&0").appendTo(this); -0===j.length||0===j.children().length?q.addClass(i.sNoFooter):0a?new t(b[a],this[a]):null},filter:function(a){var b=[];if(x.filter)b=x.filter.call(this,a,this);else for(var c=0,d=this.length;c").addClass(b),h("td",c).addClass(b).html(a)[0].colSpan=ca(d),e.push(c[0]))};f(a,b);c._details&&c._details.remove();c._details=h(e);c._detailsShow&&c._details.insertAfter(c.nTr)}return this});p(["row().child.show()","row().child().show()"],function(){Vb(this, -!0);return this});p(["row().child.hide()","row().child().hide()"],function(){Vb(this,!1);return this});p(["row().child.remove()","row().child().remove()"],function(){cb(this);return this});p("row().child.isShown()",function(){var a=this.context;return a.length&&this.length?a[0].aoData[this[0]]._detailsShow||!1:!1});var dc=/^(.+):(name|visIdx|visible)$/,Wb=function(a,b,c,d,e){for(var c=[],d=0,f=e.length;d=0?b:g.length+b];if(typeof a==="function"){var e=Da(c,f);return h.map(g,function(b,f){return a(f,Wb(c,f,0,0,e),i[f])?f:null})}var k=typeof a==="string"?a.match(dc):"";if(k)switch(k[2]){case "visIdx":case "visible":b=parseInt(k[1],10);if(b<0){var m=h.map(g,function(a,b){return a.bVisible?b:null}); -return[m[m.length+b]]}return[$(c,b)];case "name":return h.map(j,function(a,b){return a===k[1]?b:null})}else return h(i).filter(a).map(function(){return h.inArray(this,i)}).toArray()},c,f)},1);c.selector.cols=a;c.selector.opts=b;return c});r("columns().header()","column().header()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTh},1)});r("columns().footer()","column().footer()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTf},1)});r("columns().data()", -"column().data()",function(){return this.iterator("column-rows",Wb,1)});r("columns().dataSrc()","column().dataSrc()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].mData},1)});r("columns().cache()","column().cache()",function(a){return this.iterator("column-rows",function(b,c,d,e,f){return ja(b.aoData,f,"search"===a?"_aFilterData":"_aSortData",c)},1)});r("columns().nodes()","column().nodes()",function(){return this.iterator("column-rows",function(a,b,c,d,e){return ja(a.aoData, -e,"anCells",b)},1)});r("columns().visible()","column().visible()",function(a,b){return this.iterator("column",function(c,d){if(a===k)return c.aoColumns[d].bVisible;var e=c.aoColumns,f=e[d],g=c.aoData,j,i,m;if(a!==k&&f.bVisible!==a){if(a){var l=h.inArray(!0,D(e,"bVisible"),d+1);j=0;for(i=g.length;jd;return!0};m.isDataTable=m.fnIsDataTable=function(a){var b=h(a).get(0),c=!1;h.each(m.settings,function(a,e){var f=e.nScrollHead?h("table",e.nScrollHead)[0]:null,g=e.nScrollFoot?h("table",e.nScrollFoot)[0]: -null;if(e.nTable===b||f===b||g===b)c=!0});return c};m.tables=m.fnTables=function(a){var b=!1;h.isPlainObject(a)&&(b=a.api,a=a.visible);var c=h.map(m.settings,function(b){if(!a||a&&h(b.nTable).is(":visible"))return b.nTable});return b?new t(c):c};m.util={throttle:ua,escapeRegex:va};m.camelToHungarian=J;p("$()",function(a,b){var c=this.rows(b).nodes(),c=h(c);return h([].concat(c.filter(a).toArray(),c.find(a).toArray()))});h.each(["on","one","off"],function(a,b){p(b+"()",function(){var a=Array.prototype.slice.call(arguments); -a[0].match(/\.dt\b/)||(a[0]+=".dt");var d=h(this.tables().nodes());d[b].apply(d,a);return this})});p("clear()",function(){return this.iterator("table",function(a){na(a)})});p("settings()",function(){return new t(this.context,this.context)});p("init()",function(){var a=this.context;return a.length?a[0].oInit:null});p("data()",function(){return this.iterator("table",function(a){return D(a.aoData,"_aData")}).flatten()});p("destroy()",function(a){a=a||!1;return this.iterator("table",function(b){var c= -b.nTableWrapper.parentNode,d=b.oClasses,e=b.nTable,f=b.nTBody,g=b.nTHead,j=b.nTFoot,i=h(e),f=h(f),k=h(b.nTableWrapper),l=h.map(b.aoData,function(a){return a.nTr}),p;b.bDestroying=!0;v(b,"aoDestroyCallback","destroy",[b]);a||(new t(b)).columns().visible(!0);k.unbind(".DT").find(":not(tbody *)").unbind(".DT");h(E).unbind(".DT-"+b.sInstance);e!=g.parentNode&&(i.children("thead").detach(),i.append(g));j&&e!=j.parentNode&&(i.children("tfoot").detach(),i.append(j));b.aaSorting=[];b.aaSortingFixed=[];xa(b); -h(l).removeClass(b.asStripeClasses.join(" "));h("th, td",g).removeClass(d.sSortable+" "+d.sSortableAsc+" "+d.sSortableDesc+" "+d.sSortableNone);b.bJUI&&(h("th span."+d.sSortIcon+", td span."+d.sSortIcon,g).detach(),h("th, td",g).each(function(){var a=h("div."+d.sSortJUIWrapper,this);h(this).append(a.contents());a.detach()}));f.children().detach();f.append(l);g=a?"remove":"detach";i[g]();k[g]();!a&&c&&(c.insertBefore(e,b.nTableReinsertBefore),i.css("width",b.sDestroyWidth).removeClass(d.sTable),(p= -b.asDestroyStripes.length)&&f.children().each(function(a){h(this).addClass(b.asDestroyStripes[a%p])}));c=h.inArray(b,m.settings);-1!==c&&m.settings.splice(c,1)})});h.each(["column","row","cell"],function(a,b){p(b+"s().every()",function(a){var d=this.selector.opts,e=this;return this.iterator(b,function(f,g,h,i,m){a.call(e[b](g,"cell"===b?h:d,"cell"===b?d:k),g,h,i,m)})})});p("i18n()",function(a,b,c){var d=this.context[0],a=Q(a)(d.oLanguage);a===k&&(a=b);c!==k&&h.isPlainObject(a)&&(a=a[c]!==k?a[c]:a._); -return a.replace("%d",c)});m.version="1.10.10";m.settings=[];m.models={};m.models.oSearch={bCaseInsensitive:!0,sSearch:"",bRegex:!1,bSmart:!0};m.models.oRow={nTr:null,anCells:null,_aData:[],_aSortData:null,_aFilterData:null,_sFilterRow:null,_sRowStripe:"",src:null,idx:-1};m.models.oColumn={idx:null,aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bVisible:null,_sManualType:null,_bAttrSrc:!1,fnCreatedCell:null,fnGetData:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null, -sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};m.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:[],ajax:null,aLengthMenu:[10,25,50,100],aoColumns:null,aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:!0,bDeferRender:!1,bDestroy:!1,bFilter:!0,bInfo:!0,bJQueryUI:!1,bLengthChange:!0,bPaginate:!0,bProcessing:!1,bRetrieve:!1,bScrollCollapse:!1,bServerSide:!1, -bSort:!0,bSortMulti:!0,bSortCellsTop:!1,bSortClasses:!0,bStateSave:!1,fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(a){return a.toString().replace(/\B(?=(\d{3})+(?!\d))/g,this.oLanguage.sThousands)},fnHeaderCallback:null,fnInfoCallback:null,fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:null,fnServerParams:null,fnStateLoadCallback:function(a){try{return JSON.parse((-1===a.iStateDuration?sessionStorage:localStorage).getItem("DataTables_"+ -a.sInstance+"_"+location.pathname))}catch(b){}},fnStateLoadParams:null,fnStateLoaded:null,fnStateSaveCallback:function(a,b){try{(-1===a.iStateDuration?sessionStorage:localStorage).setItem("DataTables_"+a.sInstance+"_"+location.pathname,JSON.stringify(b))}catch(c){}},fnStateSaveParams:null,iStateDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iTabIndex:0,oClasses:{},oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"}, -oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",sInfoPostFix:"",sDecimal:"",sThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sSearchPlaceholder:"",sUrl:"",sZeroRecords:"No matching records found"},oSearch:h.extend({}, -m.models.oSearch),sAjaxDataProp:"data",sAjaxSource:null,sDom:"lfrtip",searchDelay:null,sPaginationType:"simple_numbers",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET",renderer:null,rowId:"DT_RowId"};Y(m.defaults);m.defaults.column={aDataSort:null,iDataSort:-1,asSorting:["asc","desc"],bSearchable:!0,bSortable:!0,bVisible:!0,fnCreatedCell:null,mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null}; -Y(m.defaults.column);m.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortMulti:null,bSortClasses:null,bStateSave:null},oScroll:{bCollapse:null,iBarWidth:0,sX:null,sXInner:null,sY:null},oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:!1,bScrollbarLeft:!1,bBounding:!1,barWidth:0},ajax:null,aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aIds:{},aoColumns:[],aoHeader:[], -aoFooter:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:[],asStripeClasses:null,asDestroyStripes:[],sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],aoStateLoaded:[],sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,nTableWrapper:null,bDeferLoading:!1,bInitialised:!1,aoOpenRows:[],sDom:null,searchDelay:null,sPaginationType:"two_button", -iStateDuration:0,aoStateSave:[],aoStateLoad:[],oSavedState:null,oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,bAjaxDataGet:!0,jqXHR:null,json:k,oAjaxData:k,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:!1,iDrawError:-1,_iDisplayLength:10,_iDisplayStart:0,_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:!1,bSorted:!1,bSortCellsTop:null,oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return"ssp"==y(this)? -1*this._iRecordsTotal:this.aiDisplayMaster.length},fnRecordsDisplay:function(){return"ssp"==y(this)?1*this._iRecordsDisplay:this.aiDisplay.length},fnDisplayEnd:function(){var a=this._iDisplayLength,b=this._iDisplayStart,c=b+a,d=this.aiDisplay.length,e=this.oFeatures,f=e.bPaginate;return e.bServerSide?!1===f||-1===a?b+d:Math.min(b+a,this._iRecordsDisplay):!f||c>d||-1===a?d:c},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,nScrollFoot:null,aLastSort:[],oPlugins:{},rowIdFn:null,rowId:null}; -m.ext=s={buttons:{},classes:{},build:"bs/dt-1.10.10",errMode:"alert",feature:[],search:[],selector:{cell:[],column:[],row:[]},internal:{},legacy:{ajax:null},pager:{},renderer:{pageButton:{},header:{}},order:{},type:{detect:[],search:{},order:{}},_unique:0,fnVersionCheck:m.fnVersionCheck,iApiIndex:0,oJUIClasses:{},sVersion:m.version};h.extend(s,{afnFiltering:s.search,aTypes:s.type.detect,ofnSearch:s.type.search,oSort:s.type.order,afnSortData:s.order,aoFeatures:s.feature,oApi:s.internal,oStdClasses:s.classes, -oPagination:s.pager});h.extend(m.ext.classes,{sTable:"dataTable",sNoFooter:"no-footer",sPageButton:"paginate_button",sPageButtonActive:"current",sPageButtonDisabled:"disabled",sStripeOdd:"odd",sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing",sSortAsc:"sorting_asc",sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled", -sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sFilterInput:"",sLengthSelect:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",sHeaderTH:"",sFooterTH:"",sSortJUIAsc:"",sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"", -sJUIHeader:"",sJUIFooter:""});var Ea="",Ea="",G=Ea+"ui-state-default",ka=Ea+"css_right ui-icon ui-icon-",Xb=Ea+"fg-toolbar ui-toolbar ui-widget-header ui-helper-clearfix";h.extend(m.ext.oJUIClasses,m.ext.classes,{sPageButton:"fg-button ui-button "+G,sPageButtonActive:"ui-state-disabled",sPageButtonDisabled:"ui-state-disabled",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_",sSortAsc:G+" sorting_asc",sSortDesc:G+" sorting_desc",sSortable:G+" sorting", -sSortableAsc:G+" sorting_asc_disabled",sSortableDesc:G+" sorting_desc_disabled",sSortableNone:G+" sorting_disabled",sSortJUIAsc:ka+"triangle-1-n",sSortJUIDesc:ka+"triangle-1-s",sSortJUI:ka+"carat-2-n-s",sSortJUIAscAllowed:ka+"carat-1-n",sSortJUIDescAllowed:ka+"carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead "+G,sScrollFoot:"dataTables_scrollFoot "+G,sHeaderTH:G,sFooterTH:G,sJUIHeader:Xb+" ui-corner-tl ui-corner-tr",sJUIFooter:Xb+ -" ui-corner-bl ui-corner-br"});var Mb=m.ext.pager;h.extend(Mb,{simple:function(){return["previous","next"]},full:function(){return["first","previous","next","last"]},numbers:function(a,b){return[Aa(a,b)]},simple_numbers:function(a,b){return["previous",Aa(a,b),"next"]},full_numbers:function(a,b){return["first","previous",Aa(a,b),"next","last"]},_numbers:Aa,numbers_length:7});h.extend(!0,m.ext.renderer,{pageButton:{_:function(a,b,c,d,e,f){var g=a.oClasses,j=a.oLanguage.oPaginate,i=a.oLanguage.oAria.paginate|| -{},k,l,m=0,p=function(b,d){var n,r,t,s,v=function(b){Ta(a,b.data.action,true)};n=0;for(r=d.length;n").appendTo(b);p(t,s)}else{k=null;l="";switch(s){case "ellipsis":b.append('');break;case "first":k=j.sFirst;l=s+(e>0?"":" "+g.sPageButtonDisabled);break;case "previous":k=j.sPrevious;l=s+(e>0?"":" "+g.sPageButtonDisabled);break;case "next":k=j.sNext;l=s+(e",{"class":g.sPageButton+" "+l,"aria-controls":a.sTableId,"aria-label":i[s],"data-dt-idx":m,tabindex:a.iTabIndex,id:c===0&&typeof s==="string"?a.sTableId+"_"+s:null}).html(k).appendTo(b);Wa(t,{action:s},v);m++}}}},r;try{r=h(b).find(H.activeElement).data("dt-idx")}catch(n){}p(h(b).empty(),d);r&&h(b).find("[data-dt-idx="+r+"]").focus()}}});h.extend(m.ext.type.detect,[function(a,b){var c= -b.oLanguage.sDecimal;return Za(a,c)?"num"+c:null},function(a){if(a&&!(a instanceof Date)&&(!ac.test(a)||!bc.test(a)))return null;var b=Date.parse(a);return null!==b&&!isNaN(b)||M(a)?"date":null},function(a,b){var c=b.oLanguage.sDecimal;return Za(a,c,!0)?"num-fmt"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Rb(a,c)?"html-num"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Rb(a,c,!0)?"html-num-fmt"+c:null},function(a){return M(a)||"string"===typeof a&&-1!==a.indexOf("<")?"html": -null}]);h.extend(m.ext.type.search,{html:function(a){return M(a)?a:"string"===typeof a?a.replace(Ob," ").replace(Ca,""):""},string:function(a){return M(a)?a:"string"===typeof a?a.replace(Ob," "):a}});var Ba=function(a,b,c,d){if(0!==a&&(!a||"-"===a))return-Infinity;b&&(a=Qb(a,b));a.replace&&(c&&(a=a.replace(c,"")),d&&(a=a.replace(d,"")));return 1*a};h.extend(s.type.order,{"date-pre":function(a){return Date.parse(a)||0},"html-pre":function(a){return M(a)?"":a.replace?a.replace(/<.*?>/g,"").toLowerCase(): -a+""},"string-pre":function(a){return M(a)?"":"string"===typeof a?a.toLowerCase():!a.toString?"":a.toString()},"string-asc":function(a,b){return ab?1:0},"string-desc":function(a,b){return ab?-1:0}});db("");h.extend(!0,m.ext.renderer,{header:{_:function(a,b,c,d){h(a.nTable).on("order.dt.DT",function(e,f,g,h){if(a===f){e=c.idx;b.removeClass(c.sSortingClass+" "+d.sSortAsc+" "+d.sSortDesc).addClass(h[e]=="asc"?d.sSortAsc:h[e]=="desc"?d.sSortDesc:c.sSortingClass)}})},jqueryui:function(a, -b,c,d){h("
    ").addClass(d.sSortJUIWrapper).append(b.contents()).append(h("").addClass(d.sSortIcon+" "+c.sSortingClassJUI)).appendTo(b);h(a.nTable).on("order.dt.DT",function(e,f,g,h){if(a===f){e=c.idx;b.removeClass(d.sSortAsc+" "+d.sSortDesc).addClass(h[e]=="asc"?d.sSortAsc:h[e]=="desc"?d.sSortDesc:c.sSortingClass);b.find("span."+d.sSortIcon).removeClass(d.sSortJUIAsc+" "+d.sSortJUIDesc+" "+d.sSortJUI+" "+d.sSortJUIAscAllowed+" "+d.sSortJUIDescAllowed).addClass(h[e]=="asc"?d.sSortJUIAsc: -h[e]=="desc"?d.sSortJUIDesc:c.sSortingClassJUI)}})}}});m.render={number:function(a,b,c,d,e){return{display:function(f){if("number"!==typeof f&&"string"!==typeof f)return f;var g=0>f?"-":"",h=parseFloat(f);if(isNaN(h))return f;f=Math.abs(h);h=parseInt(f,10);f=c?b+(f-h).toFixed(c).substring(2):"";return g+(d||"")+h.toString().replace(/\B(?=(\d{3})+(?!\d))/g,a)+f+(e||"")}}},text:function(){return{display:function(a){return"string"===typeof a?a.replace(//g,">").replace(/"/g,"""): -a}}}};h.extend(m.ext.internal,{_fnExternApiFunc:Nb,_fnBuildAjax:ra,_fnAjaxUpdate:lb,_fnAjaxParameters:ub,_fnAjaxUpdateDraw:vb,_fnAjaxDataSrc:sa,_fnAddColumn:Ga,_fnColumnOptions:la,_fnAdjustColumnSizing:U,_fnVisibleToColumnIndex:$,_fnColumnIndexToVisible:ba,_fnVisbleColumns:ca,_fnGetColumns:aa,_fnColumnTypes:Ia,_fnApplyColumnDefs:ib,_fnHungarianMap:Y,_fnCamelToHungarian:J,_fnLanguageCompat:Fa,_fnBrowserDetect:gb,_fnAddData:N,_fnAddTr:ma,_fnNodeToDataIndex:function(a,b){return b._DT_RowIndex!==k?b._DT_RowIndex: -null},_fnNodeToColumnIndex:function(a,b,c){return h.inArray(c,a.aoData[b].anCells)},_fnGetCellData:B,_fnSetCellData:jb,_fnSplitObjNotation:La,_fnGetObjectDataFn:Q,_fnSetObjectDataFn:R,_fnGetDataMaster:Ma,_fnClearTable:na,_fnDeleteIndex:oa,_fnInvalidate:ea,_fnGetRowElements:Ka,_fnCreateTr:Ja,_fnBuildHead:kb,_fnDrawHead:ga,_fnDraw:O,_fnReDraw:T,_fnAddOptionsHtml:nb,_fnDetectHeader:fa,_fnGetUniqueThs:qa,_fnFeatureHtmlFilter:pb,_fnFilterComplete:ha,_fnFilterCustom:yb,_fnFilterColumn:xb,_fnFilter:wb,_fnFilterCreateSearch:Qa, -_fnEscapeRegex:va,_fnFilterData:zb,_fnFeatureHtmlInfo:sb,_fnUpdateInfo:Cb,_fnInfoMacros:Db,_fnInitialise:ia,_fnInitComplete:ta,_fnLengthChange:Ra,_fnFeatureHtmlLength:ob,_fnFeatureHtmlPaginate:tb,_fnPageChange:Ta,_fnFeatureHtmlProcessing:qb,_fnProcessingDisplay:C,_fnFeatureHtmlTable:rb,_fnScrollDraw:Z,_fnApplyToChildren:I,_fnCalculateColumnWidths:Ha,_fnThrottle:ua,_fnConvertToWidth:Fb,_fnGetWidestNode:Gb,_fnGetMaxLenString:Hb,_fnStringToCss:w,_fnSortFlatten:W,_fnSort:mb,_fnSortAria:Jb,_fnSortListener:Va, -_fnSortAttachListener:Oa,_fnSortingClasses:xa,_fnSortData:Ib,_fnSaveState:ya,_fnLoadState:Kb,_fnSettingsFromNode:za,_fnLog:K,_fnMap:F,_fnBindAction:Wa,_fnCallbackReg:z,_fnCallbackFire:v,_fnLengthOverflow:Sa,_fnRenderer:Pa,_fnDataSource:y,_fnRowAttributes:Na,_fnCalculateEnd:function(){}});h.fn.dataTable=m;m.$=h;h.fn.dataTableSettings=m.settings;h.fn.dataTableExt=m.ext;h.fn.DataTable=function(a){return h(this).dataTable(a).api()};h.each(m,function(a,b){h.fn.DataTable[a]=b});return h.fn.dataTable}); - - -/*! - DataTables Bootstrap 3 integration - ©2011-2015 SpryMedia Ltd - datatables.net/license -*/ -(function(b){"function"===typeof define&&define.amd?define(["jquery","datatables.net"],function(a){return b(a,window,document)}):"object"===typeof exports?module.exports=function(a,e){a||(a=window);if(!e||!e.fn.dataTable)e=require("datatables.net")(a,e).$;return b(e,a,a.document)}:b(jQuery,window,document)})(function(b,a,e){var d=b.fn.dataTable;b.extend(!0,d.defaults,{dom:"<'row'<'col-sm-6'l><'col-sm-6'f>><'row'<'col-sm-12'tr>><'row'<'col-sm-5'i><'col-sm-7'p>>",renderer:"bootstrap"});b.extend(d.ext.classes, -{sWrapper:"dataTables_wrapper form-inline dt-bootstrap",sFilterInput:"form-control input-sm",sLengthSelect:"form-control input-sm",sProcessing:"dataTables_processing panel panel-default"});d.ext.renderer.pageButton.bootstrap=function(a,h,r,m,j,n){var o=new d.Api(a),s=a.oClasses,k=a.oLanguage.oPaginate,t=a.oLanguage.oAria.paginate||{},f,g,p=0,q=function(d,e){var l,h,i,c,m=function(a){a.preventDefault();!b(a.currentTarget).hasClass("disabled")&&o.page()!=a.data.action&&o.page(a.data.action).draw("page")}; -l=0;for(h=e.length;l",{"class":s.sPageButton+" "+g,id:0===r&&"string"===typeof c?a.sTableId+"_"+c:null}).append(b("",{href:"#", -"aria-controls":a.sTableId,"aria-label":t[c],"data-dt-idx":p,tabindex:a.iTabIndex}).html(f)).appendTo(d),a.oApi._fnBindAction(i,{action:c},m),p++)}},i;try{i=b(h).find(e.activeElement).data("dt-idx")}catch(u){}q(b(h).empty().html('
    ",{valign:"top",colSpan:ca(a), -"class":a.oClasses.sRowEmpty}).html(c))[0];v(a,"aoHeaderCallback","header",[h(a.nTHead).children("tr")[0],Ma(a),g,o,i]);v(a,"aoFooterCallback","footer",[h(a.nTFoot).children("tr")[0],Ma(a),g,o,i]);d=h(a.nTBody);d.children().detach();d.append(h(b));v(a,"aoDrawCallback","draw",[a]);a.bSorted=!1;a.bFiltered=!1;a.bDrawing=!1}}function T(a,b){var c=a.oFeatures,d=c.bFilter;c.bSort&&mb(a);d?ha(a,a.oPreviousSearch):a.aiDisplay=a.aiDisplayMaster.slice();!0!==b&&(a._iDisplayStart=0);a._drawHold=b;O(a);a._drawHold= -!1}function nb(a){var b=a.oClasses,c=h(a.nTable),c=h("
    ").insertBefore(c),d=a.oFeatures,e=h("
    ",{id:a.sTableId+"_wrapper","class":b.sWrapper+(a.nTFoot?"":" "+b.sNoFooter)});a.nHolding=c[0];a.nTableWrapper=e[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var f=a.sDom.split(""),g,j,i,o,l,q,u=0;u")[0];o=f[u+1];if("'"==o||'"'==o){l="";for(q=2;f[u+q]!=o;)l+=f[u+q],q++;"H"==l?l=b.sJUIHeader:"F"==l&&(l=b.sJUIFooter);-1!=l.indexOf(".")?(o=l.split("."), -i.id=o[0].substr(1,o[0].length-1),i.className=o[1]):"#"==l.charAt(0)?i.id=l.substr(1,l.length-1):i.className=l;u+=q}e.append(i);e=h(i)}else if(">"==j)e=e.parent();else if("l"==j&&d.bPaginate&&d.bLengthChange)g=ob(a);else if("f"==j&&d.bFilter)g=pb(a);else if("r"==j&&d.bProcessing)g=qb(a);else if("t"==j)g=rb(a);else if("i"==j&&d.bInfo)g=sb(a);else if("p"==j&&d.bPaginate)g=tb(a);else if(0!==m.ext.feature.length){i=m.ext.feature;q=0;for(o=i.length;q',j=d.sSearch,j=j.match(/_INPUT_/)?j.replace("_INPUT_",g):j+g,b=h("
    ",{id:!f.f?c+"_filter":null,"class":b.sFilter}).append(h("
    ").addClass(b.sLength); -a.aanFeatures.l||(i[0].id=c+"_length");i.children().append(a.oLanguage.sLengthMenu.replace("_MENU_",e[0].outerHTML));h("select",i).val(a._iDisplayLength).bind("change.DT",function(){Ra(a,h(this).val());O(a)});h(a.nTable).bind("length.dt.DT",function(b,c,d){a===c&&h("select",i).val(d)});return i[0]}function tb(a){var b=a.sPaginationType,c=m.ext.pager[b],d="function"===typeof c,e=function(a){O(a)},b=h("
    ").addClass(a.oClasses.sPaging+b)[0],f=a.aanFeatures;d||c.fnInit(a,b,e);f.p||(b.id=a.sTableId+ -"_paginate",a.aoDrawCallback.push({fn:function(a){if(d){var b=a._iDisplayStart,i=a._iDisplayLength,h=a.fnRecordsDisplay(),l=-1===i,b=l?0:Math.ceil(b/i),i=l?1:Math.ceil(h/i),h=c(b,i),k,l=0;for(k=f.p.length;lf&&(d=0)):"first"==b?d=0:"previous"==b?(d=0<=e?d-e:0,0>d&&(d=0)):"next"== -b?d+e",{id:!a.aanFeatures.r?a.sTableId+"_processing":null,"class":a.oClasses.sProcessing}).html(a.oLanguage.sProcessing).insertBefore(a.nTable)[0]}function C(a,b){a.oFeatures.bProcessing&&h(a.aanFeatures.r).css("display",b?"block":"none");v(a,null,"processing",[a,b])}function rb(a){var b=h(a.nTable);b.attr("role", -"grid");var c=a.oScroll;if(""===c.sX&&""===c.sY)return a.nTable;var d=c.sX,e=c.sY,f=a.oClasses,g=b.children("caption"),j=g.length?g[0]._captionSide:null,i=h(b[0].cloneNode(!1)),o=h(b[0].cloneNode(!1)),l=b.children("tfoot");l.length||(l=null);i=h("
    ",{"class":f.sScrollWrapper}).append(h("
    ",{"class":f.sScrollHead}).css({overflow:"hidden",position:"relative",border:0,width:d?!d?null:w(d):"100%"}).append(h("
    ",{"class":f.sScrollHeadInner}).css({"box-sizing":"content-box",width:c.sXInner|| -"100%"}).append(i.removeAttr("id").css("margin-left",0).append("top"===j?g:null).append(b.children("thead"))))).append(h("
    ",{"class":f.sScrollBody}).css({position:"relative",overflow:"auto",width:!d?null:w(d)}).append(b));l&&i.append(h("
    ",{"class":f.sScrollFoot}).css({overflow:"hidden",border:0,width:d?!d?null:w(d):"100%"}).append(h("
    ",{"class":f.sScrollFootInner}).append(o.removeAttr("id").css("margin-left",0).append("bottom"===j?g:null).append(b.children("tfoot")))));var b=i.children(), -k=b[0],f=b[1],u=l?b[2]:null;if(d)h(f).on("scroll.DT",function(){var a=this.scrollLeft;k.scrollLeft=a;l&&(u.scrollLeft=a)});h(f).css(e&&c.bCollapse?"max-height":"height",e);a.nScrollHead=k;a.nScrollBody=f;a.nScrollFoot=u;a.aoDrawCallback.push({fn:Z,sName:"scrolling"});return i[0]}function Z(a){var b=a.oScroll,c=b.sX,d=b.sXInner,e=b.sY,b=b.iBarWidth,f=h(a.nScrollHead),g=f[0].style,j=f.children("div"),i=j[0].style,o=j.children("table"),j=a.nScrollBody,l=h(j),q=j.style,u=h(a.nScrollFoot).children("div"), -m=u.children("table"),n=h(a.nTHead),p=h(a.nTable),t=p[0],v=t.style,r=a.nTFoot?h(a.nTFoot):null,Eb=a.oBrowser,Ua=Eb.bScrollOversize,s,L,P,x,y=[],z=[],A=[],B,C=function(a){a=a.style;a.paddingTop="0";a.paddingBottom="0";a.borderTopWidth="0";a.borderBottomWidth="0";a.height=0};L=j.scrollHeight>j.clientHeight;if(a.scrollBarVis!==L&&a.scrollBarVis!==k)a.scrollBarVis=L,U(a);else{a.scrollBarVis=L;p.children("thead, tfoot").remove();x=n.clone().prependTo(p);n=n.find("tr");L=x.find("tr");x.find("th, td").removeAttr("tabindex"); -r&&(P=r.clone().prependTo(p),s=r.find("tr"),P=P.find("tr"));c||(q.width="100%",f[0].style.width="100%");h.each(qa(a,x),function(b,c){B=$(a,b);c.style.width=a.aoColumns[B].sWidth});r&&I(function(a){a.style.width=""},P);f=p.outerWidth();if(""===c){v.width="100%";if(Ua&&(p.find("tbody").height()>j.offsetHeight||"scroll"==l.css("overflow-y")))v.width=w(p.outerWidth()-b);f=p.outerWidth()}else""!==d&&(v.width=w(d),f=p.outerWidth());I(C,L);I(function(a){A.push(a.innerHTML);y.push(w(h(a).css("width")))}, -L);I(function(a,b){a.style.width=y[b]},n);h(L).height(0);r&&(I(C,P),I(function(a){z.push(w(h(a).css("width")))},P),I(function(a,b){a.style.width=z[b]},s),h(P).height(0));I(function(a,b){a.innerHTML='
    '+A[b]+"
    ";a.style.width=y[b]},L);r&&I(function(a,b){a.innerHTML="";a.style.width=z[b]},P);if(p.outerWidth()j.offsetHeight||"scroll"==l.css("overflow-y")?f+b:f;if(Ua&&(j.scrollHeight>j.offsetHeight||"scroll"==l.css("overflow-y")))v.width= -w(s-b);(""===c||""!==d)&&K(a,1,"Possible column misalignment",6)}else s="100%";q.width=w(s);g.width=w(s);r&&(a.nScrollFoot.style.width=w(s));!e&&Ua&&(q.height=w(t.offsetHeight+b));c=p.outerWidth();o[0].style.width=w(c);i.width=w(c);d=p.height()>j.clientHeight||"scroll"==l.css("overflow-y");e="padding"+(Eb.bScrollbarLeft?"Left":"Right");i[e]=d?b+"px":"0px";r&&(m[0].style.width=w(c),u[0].style.width=w(c),u[0].style[e]=d?b+"px":"0px");l.scroll();if((a.bSorted||a.bFiltered)&&!a._drawHold)j.scrollTop= -0}}function I(a,b,c){for(var d=0,e=0,f=b.length,g,j;e").appendTo(j.find("tbody"));j.find("thead, tfoot").remove();j.append(h(a.nTHead).clone()).append(h(a.nTFoot).clone());j.find("tfoot th, tfoot td").css("width","");o=qa(a,j.find("thead")[0]);for(m=0;m").css({width:n.sWidthOrig,margin:0,padding:0,border:0,height:1}));if(a.aoData.length)for(m=0;m").css(f||e?{position:"absolute",top:0,left:0,height:1,right:0,overflow:"hidden"}:{}).append(j).appendTo(k);f&&g?j.width(g):f?(j.css("width","auto"),j.removeAttr("width"),j.width()").css("width",w(a)).appendTo(b||H.body),d=c[0].offsetWidth;c.remove();return d}function Gb(a,b){var c=Hb(a,b);if(0>c)return null;var d=a.aoData[c];return!d.nTr?h("
    ").html(B(a,c,b,"display"))[0]:d.anCells[b]}function Hb(a,b){for(var c,d=-1,e=-1,f=0,g=a.aoData.length;fd&&(d=c.length,e=f);return e}function w(a){return null=== -a?"0px":"number"==typeof a?0>a?"0px":a+"px":a.match(/\d$/)?a+"px":a}function W(a){var b,c,d=[],e=a.aoColumns,f,g,j,i;b=a.aaSortingFixed;c=h.isPlainObject(b);var o=[];f=function(a){a.length&&!h.isArray(a[0])?o.push(a):h.merge(o,a)};h.isArray(b)&&f(b);c&&b.pre&&f(b.pre);f(a.aaSorting);c&&b.post&&f(b.post);for(a=0;ae?1:0,0!==c)return"asc"===j.dir?c:-c;c=d[a];e=d[b]; -return ce?1:0}):i.sort(function(a,b){var c,g,j,i,k=h.length,m=f[a]._aSortData,p=f[b]._aSortData;for(j=0;jg?1:0})}a.bSorted=!0}function Jb(a){for(var b,c,d=a.aoColumns,e=W(a),a=a.oLanguage.oAria,f=0,g=d.length;f/g,"");var i=c.nTh;i.removeAttribute("aria-sort");c.bSortable&&(0e?e+1:3));e=0;for(f=d.length;ee?e+1:3))}a.aLastSort=d}function Ib(a,b){var c=a.aoColumns[b],d=m.ext.order[c.sSortDataType],e;d&&(e=d.call(a.oInstance,a,b,ba(a,b)));for(var f,g=m.ext.type.order[c.sType+"-pre"],j=0,i=a.aoData.length;j=d.length?[0,c[1]]:c)}));e.search!==k&&h.extend(a.oPreviousSearch,Bb(e.search));b=0;for(c=e.columns.length;b=c&&(b=c-d);b-=b%d;if(-1===d||0>b)b=0;a._iDisplayStart=b}function Pa(a,b){var c= -a.renderer,d=m.ext.renderer[b];return h.isPlainObject(c)&&c[b]?d[c[b]]||d._:"string"===typeof c?d[c]||d._:d._}function y(a){return a.oFeatures.bServerSide?"ssp":a.ajax||a.sAjaxSource?"ajax":"dom"}function Aa(a,b){var c=[],c=Mb.numbers_length,d=Math.floor(c/2);b<=c?c=X(0,b):a<=d?(c=X(0,c-2),c.push("ellipsis"),c.push(b-1)):(a>=b-1-d?c=X(b-(c-2),b):(c=X(a-d+2,a+d-1),c.push("ellipsis"),c.push(b-1)),c.splice(0,0,"ellipsis"),c.splice(0,0,0));c.DT_el="span";return c}function db(a){h.each({num:function(b){return Ba(b, -a)},"num-fmt":function(b){return Ba(b,a,Xa)},"html-num":function(b){return Ba(b,a,Ca)},"html-num-fmt":function(b){return Ba(b,a,Ca,Xa)}},function(b,c){s.type.order[b+a+"-pre"]=c;b.match(/^html\-/)&&(s.type.search[b+a]=s.type.search.html)})}function Nb(a){return function(){var b=[za(this[m.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return m.ext.internal[a].apply(this,b)}}var m,s,t,p,r,Ya={},Ob=/[\r\n]/g,Ca=/<.*?>/g,ac=/^[\w\+\-]/,bc=/[\w\+\-]$/,Yb=RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)", -"g"),Xa=/[',$£€¥%\u2009\u202F\u20BD\u20a9\u20BArfk]/gi,M=function(a){return!a||!0===a||"-"===a?!0:!1},Pb=function(a){var b=parseInt(a,10);return!isNaN(b)&&isFinite(a)?b:null},Qb=function(a,b){Ya[b]||(Ya[b]=RegExp(va(b),"g"));return"string"===typeof a&&"."!==b?a.replace(/\./g,"").replace(Ya[b],"."):a},Za=function(a,b,c){var d="string"===typeof a;if(M(a))return!0;b&&d&&(a=Qb(a,b));c&&d&&(a=a.replace(Xa,""));return!isNaN(parseFloat(a))&&isFinite(a)},Rb=function(a,b,c){return M(a)?!0:!(M(a)||"string"=== -typeof a)?null:Za(a.replace(Ca,""),b,c)?!0:null},D=function(a,b,c){var d=[],e=0,f=a.length;if(c!==k)for(;e")[0],Zb=wa.textContent!==k,$b=/<.*?>/g;m=function(a){this.$=function(a,b){return this.api(!0).$(a,b)};this._=function(a,b){return this.api(!0).rows(a,b).data()};this.api=function(a){return a?new t(za(this[s.iApiIndex])):new t(this)};this.fnAddData=function(a,b){var c=this.api(!0), -d=h.isArray(a)&&(h.isArray(a[0])||h.isPlainObject(a[0]))?c.rows.add(a):c.row.add(a);(b===k||b)&&c.draw();return d.flatten().toArray()};this.fnAdjustColumnSizing=function(a){var b=this.api(!0).columns.adjust(),c=b.settings()[0],d=c.oScroll;a===k||a?b.draw(!1):(""!==d.sX||""!==d.sY)&&Z(c)};this.fnClearTable=function(a){var b=this.api(!0).clear();(a===k||a)&&b.draw()};this.fnClose=function(a){this.api(!0).row(a).child.hide()};this.fnDeleteRow=function(a,b,c){var d=this.api(!0),a=d.rows(a),e=a.settings()[0], -h=e.aoData[a[0][0]];a.remove();b&&b.call(this,e,h);(c===k||c)&&d.draw();return h};this.fnDestroy=function(a){this.api(!0).destroy(a)};this.fnDraw=function(a){this.api(!0).draw(a)};this.fnFilter=function(a,b,c,d,e,h){e=this.api(!0);null===b||b===k?e.search(a,c,d,h):e.column(b).search(a,c,d,h);e.draw()};this.fnGetData=function(a,b){var c=this.api(!0);if(a!==k){var d=a.nodeName?a.nodeName.toLowerCase():"";return b!==k||"td"==d||"th"==d?c.cell(a,b).data():c.row(a).data()||null}return c.data().toArray()}; -this.fnGetNodes=function(a){var b=this.api(!0);return a!==k?b.row(a).node():b.rows().nodes().flatten().toArray()};this.fnGetPosition=function(a){var b=this.api(!0),c=a.nodeName.toUpperCase();return"TR"==c?b.row(a).index():"TD"==c||"TH"==c?(a=b.cell(a).index(),[a.row,a.columnVisible,a.column]):null};this.fnIsOpen=function(a){return this.api(!0).row(a).child.isShown()};this.fnOpen=function(a,b,c){return this.api(!0).row(a).child(b,c).show().child()[0]};this.fnPageChange=function(a,b){var c=this.api(!0).page(a); -(b===k||b)&&c.draw(!1)};this.fnSetColumnVis=function(a,b,c){a=this.api(!0).column(a).visible(b);(c===k||c)&&a.columns.adjust().draw()};this.fnSettings=function(){return za(this[s.iApiIndex])};this.fnSort=function(a){this.api(!0).order(a).draw()};this.fnSortListener=function(a,b,c){this.api(!0).order.listener(a,b,c)};this.fnUpdate=function(a,b,c,d,e){var h=this.api(!0);c===k||null===c?h.row(b).data(a):h.cell(b,c).data(a);(e===k||e)&&h.columns.adjust();(d===k||d)&&h.draw();return 0};this.fnVersionCheck= -s.fnVersionCheck;var b=this,c=a===k,d=this.length;c&&(a={});this.oApi=this.internal=s.internal;for(var e in m.ext.internal)e&&(this[e]=Nb(e));this.each(function(){var e={},e=1t<"F"ip>'),n.renderer)?h.isPlainObject(n.renderer)&&!n.renderer.header&&(n.renderer.header="jqueryui"):n.renderer="jqueryui":h.extend(i,m.ext.classes,e.oClasses);q.addClass(i.sTable);n.iInitDisplayStart===k&&(n.iInitDisplayStart=e.iDisplayStart,n._iDisplayStart=e.iDisplayStart);null!==e.iDeferLoading&&(n.bDeferLoading=!0,g=h.isArray(e.iDeferLoading), -n._iRecordsDisplay=g?e.iDeferLoading[0]:e.iDeferLoading,n._iRecordsTotal=g?e.iDeferLoading[1]:e.iDeferLoading);var t=n.oLanguage;h.extend(!0,t,e.oLanguage);""!==t.sUrl&&(h.ajax({dataType:"json",url:t.sUrl,success:function(a){Fa(a);J(l.oLanguage,a);h.extend(true,t,a);ia(n)},error:function(){ia(n)}}),o=!0);null===e.asStripeClasses&&(n.asStripeClasses=[i.sStripeOdd,i.sStripeEven]);var g=n.asStripeClasses,r=q.children("tbody").find("tr").eq(0);-1!==h.inArray(!0,h.map(g,function(a){return r.hasClass(a)}))&& -(h("tbody tr",this).removeClass(g.join(" ")),n.asDestroyStripes=g.slice());u=[];g=this.getElementsByTagName("thead");0!==g.length&&(fa(n.aoHeader,g[0]),u=qa(n));if(null===e.aoColumns){p=[];g=0;for(j=u.length;g").appendTo(this));n.nTHead=j[0];j=q.children("tbody");0===j.length&&(j=h("
    BigQuery table to load data into. If is not included, project will be the project defined in the connection json. (templated) From ca2c6b7389779f238b47e056e5a1fecd09f039ad Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Sat, 8 Dec 2018 09:14:43 +0800 Subject: [PATCH 267/808] [AIRFLOW-3479] Keeps records in Log Table when DAG is deleted (#4287) Users will use either API or web UI to delete DAG (after DAG file is removed): - Using API: provide one boolean parameter to let users decide if they want to keep records in Log table when they delete a DAG. Default value it True (to keep records in Log table). - From UI: will keep records in the Log table when delete records for a specific DAG ID (pop-up message is updated accordingly). --- airflow/api/common/experimental/delete_dag.py | 12 +- airflow/www/templates/airflow/dag.html | 2 +- .../common/experimental/test_delete_dag.py | 141 ++++++++++++++++++ 3 files changed, 153 insertions(+), 2 deletions(-) create mode 100644 tests/api/common/experimental/test_delete_dag.py diff --git a/airflow/api/common/experimental/delete_dag.py b/airflow/api/common/experimental/delete_dag.py index b9ce736b48dfc..e7d15772c961f 100644 --- a/airflow/api/common/experimental/delete_dag.py +++ b/airflow/api/common/experimental/delete_dag.py @@ -23,7 +23,15 @@ from airflow.exceptions import DagNotFound, DagFileExists -def delete_dag(dag_id): +def delete_dag(dag_id, keep_records_in_log=True): + """ + :param dag_id: the dag_id of the DAG to delete + :type dag_id: str + :param keep_records_in_log: whether keep records of the given dag_id + in the Log table in the backend database (for reasons like auditing). + The default value is True. + :type keep_records_in_log: bool + """ session = settings.Session() DM = models.DagModel @@ -41,6 +49,8 @@ def delete_dag(dag_id): # noinspection PyUnresolvedReferences,PyProtectedMember for m in models.Base._decl_class_registry.values(): if hasattr(m, "dag_id"): + if keep_records_in_log and m.__name__ == 'Log': + continue cond = or_(m.dag_id == dag_id, m.dag_id.like(dag_id + ".%")) count += session.query(m).filter(cond).delete(synchronize_session='fetch') diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index 9ef4cce5c1c46..d202a977ee56f 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -311,7 +311,7 @@
    . Got: {}'.format( - destination_dataset_table)) destination_project, destination_dataset, destination_table = \ _split_tablename(table_input=destination_dataset_table, default_project_id=self.project_id) - configuration['query'].update({ - 'allowLargeResults': allow_large_results, - 'flattenResults': flatten_results, - 'writeDisposition': write_disposition, - 'createDisposition': create_disposition, - 'destinationTable': { - 'projectId': destination_project, - 'datasetId': destination_dataset, - 'tableId': destination_table, - } - }) - if udf_config: - if not isinstance(udf_config, list): - raise TypeError("udf_config argument must have a type 'list'" - " not {}".format(type(udf_config))) - configuration['query'].update({ - 'userDefinedFunctionResources': udf_config - }) - if query_params: - if self.use_legacy_sql: - raise ValueError("Query parameters are not allowed when using " - "legacy SQL") - else: - configuration['query']['queryParameters'] = query_params + destination_dataset_table = { + 'projectId': destination_project, + 'datasetId': destination_dataset, + 'tableId': destination_table, + } - if labels: - configuration['labels'] = labels + query_param_list = [ + (sql, 'query', None, str), + (priority, 'priority', 'INTERACTIVE', str), + (use_legacy_sql, 'useLegacySql', self.use_legacy_sql, bool), + (query_params, 'queryParameters', None, dict), + (udf_config, 'userDefinedFunctionResources', None, list), + (maximum_billing_tier, 'maximumBillingTier', None, int), + (maximum_bytes_billed, 'maximumBytesBilled', None, float), + (time_partitioning, 'timePartitioning', {}, dict), + (schema_update_options, 'schemaUpdateOptions', None, tuple), + (destination_dataset_table, 'destinationTable', None, dict) + ] - time_partitioning = _cleanse_time_partitioning( - destination_dataset_table, - time_partitioning - ) - if time_partitioning: - configuration['query'].update({ - 'timePartitioning': time_partitioning - }) + for param_tuple in query_param_list: - if schema_update_options: - if write_disposition not in ["WRITE_APPEND", "WRITE_TRUNCATE"]: - raise ValueError("schema_update_options is only " - "allowed if write_disposition is " - "'WRITE_APPEND' or 'WRITE_TRUNCATE'.") - else: - self.log.info( - "Adding experimental " - "'schemaUpdateOptions': {0}".format(schema_update_options)) - configuration['query'][ - 'schemaUpdateOptions'] = schema_update_options + param, param_name, param_default, param_type = param_tuple + + if param_name not in configuration['query'] and param in [None, {}, ()]: + if param_name == 'timePartitioning': + param_default = _cleanse_time_partitioning( + destination_dataset_table, time_partitioning) + param = param_default + + if param not in [None, {}, ()]: + _api_resource_configs_duplication_check( + param_name, param, configuration['query']) + + configuration['query'][param_name] = param + + # check valid type of provided param, + # it last step because we can get param from 2 sources, + # and first of all need to find it + + _validate_value(param_name, configuration['query'][param_name], + param_type) + + if param_name == 'schemaUpdateOptions' and param: + self.log.info("Adding experimental 'schemaUpdateOptions': " + "{0}".format(schema_update_options)) + + if param_name == 'destinationTable': + for key in ['projectId', 'datasetId', 'tableId']: + if key not in configuration['query']['destinationTable']: + raise ValueError( + "Not correct 'destinationTable' in " + "api_resource_configs. 'destinationTable' " + "must be a dict with {'projectId':'', " + "'datasetId':'', 'tableId':''}") + + configuration['query'].update({ + 'allowLargeResults': allow_large_results, + 'flattenResults': flatten_results, + 'writeDisposition': write_disposition, + 'createDisposition': create_disposition, + }) + + if 'useLegacySql' in configuration['query'] and \ + 'queryParameters' in configuration['query']: + raise ValueError("Query parameters are not allowed " + "when using legacy SQL") + + if labels: + _api_resource_configs_duplication_check( + 'labels', labels, configuration) + configuration['labels'] = labels return self.run_with_configuration(configuration) @@ -884,8 +925,7 @@ def run_load(self, # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.query.tableDefinitions.(key).sourceFormat if src_fmt_configs is None: src_fmt_configs = {} - if time_partitioning is None: - time_partitioning = {} + source_format = source_format.upper() allowed_formats = [ "CSV", "NEWLINE_DELIMITED_JSON", "AVRO", "GOOGLE_SHEETS", @@ -1163,10 +1203,6 @@ def run_table_delete(self, deletion_dataset_table, :type ignore_if_missing: boolean :return: """ - if '.' not in deletion_dataset_table: - raise ValueError( - 'Expected deletion_dataset_table name in the format of ' - '.
    . Got: {}'.format(deletion_dataset_table)) deletion_project, deletion_dataset, deletion_table = \ _split_tablename(table_input=deletion_dataset_table, default_project_id=self.project_id) @@ -1532,6 +1568,12 @@ def _bq_cast(string_field, bq_type): def _split_tablename(table_input, default_project_id, var_name=None): + + if '.' not in table_input: + raise ValueError( + 'Expected deletion_dataset_table name in the format of ' + '.
    . Got: {}'.format(table_input)) + if not default_project_id: raise ValueError("INTERNAL: No default project is specified") @@ -1593,8 +1635,29 @@ def var_print(var_name): def _cleanse_time_partitioning(destination_dataset_table, time_partitioning_in): # if it is a partitioned table ($ is in the table name) add partition load option + + if time_partitioning_in is None: + time_partitioning_in = {} + time_partitioning_out = {} if destination_dataset_table and '$' in destination_dataset_table: time_partitioning_out['type'] = 'DAY' time_partitioning_out.update(time_partitioning_in) return time_partitioning_out + + +def _validate_value(key, value, expected_type): + """ function to check expected type and raise + error if type is not correct """ + if not isinstance(value, expected_type): + raise TypeError("{} argument must have a type {} not {}".format( + key, expected_type, type(value))) + + +def _api_resource_configs_duplication_check(key, value, config_dict): + if key in config_dict and value != config_dict[key]: + raise ValueError("Values of {param_name} param are duplicated. " + "`api_resource_configs` contained {param_name} param " + "in `query` config and {param_name} was also provided " + "with arg to run_query() method. Please remove duplicates." + .format(param_name=key)) diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 037b9fc43e631..c773ebb2e6f21 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -75,6 +75,13 @@ class BigQueryOperator(BaseOperator): (without incurring a charge). If unspecified, this will be set to your project default. :type maximum_bytes_billed: float + :param api_resource_configs: a dictionary that contain params + 'configuration' applied for Google BigQuery Jobs API: + https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs + for example, {'query': {'useQueryCache': False}}. You could use it + if you need to provide some params that are not supported by BigQueryOperator + like args. + :type api_resource_configs: dict :param schema_update_options: Allows the schema of the destination table to be updated as a side effect of the load job. :type schema_update_options: tuple @@ -116,7 +123,8 @@ def __init__(self, query_params=None, labels=None, priority='INTERACTIVE', - time_partitioning={}, + time_partitioning=None, + api_resource_configs=None, *args, **kwargs): super(BigQueryOperator, self).__init__(*args, **kwargs) @@ -138,7 +146,10 @@ def __init__(self, self.labels = labels self.bq_cursor = None self.priority = priority - self.time_partitioning = time_partitioning + if time_partitioning is None: + self.time_partitioning = {} + if api_resource_configs is None: + self.api_resource_configs = {} # TODO remove `bql` in Airflow 2.0 if self.bql: @@ -177,7 +188,8 @@ def execute(self, context): labels=self.labels, schema_update_options=self.schema_update_options, priority=self.priority, - time_partitioning=self.time_partitioning + time_partitioning=self.time_partitioning, + api_resource_configs=self.api_resource_configs, ) def on_kill(self): diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 40d806677c74e..4a97ea7aa4d3c 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -25,7 +25,8 @@ import mock from airflow.contrib.hooks import bigquery_hook as hook -from airflow.contrib.hooks.bigquery_hook import _cleanse_time_partitioning +from airflow.contrib.hooks.bigquery_hook import _cleanse_time_partitioning, \ + _validate_value, _api_resource_configs_duplication_check bq_available = True @@ -206,6 +207,16 @@ def mock_job_cancel(projectId, jobId): class TestBigQueryBaseCursor(unittest.TestCase): + def test_bql_deprecation_warning(self): + with warnings.catch_warnings(record=True) as w: + hook.BigQueryBaseCursor("test", "test").run_query( + bql='select * from test_table' + ) + yield + self.assertIn( + 'Deprecated parameter `bql`', + w[0].message.args[0]) + def test_invalid_schema_update_options(self): with self.assertRaises(Exception) as context: hook.BigQueryBaseCursor("test", "test").run_load( @@ -216,16 +227,6 @@ def test_invalid_schema_update_options(self): ) self.assertIn("THIS IS NOT VALID", str(context.exception)) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_bql_deprecation_warning(self, mock_rwc): - with warnings.catch_warnings(record=True) as w: - hook.BigQueryBaseCursor("test", "test").run_query( - bql='select * from test_table' - ) - self.assertIn( - 'Deprecated parameter `bql`', - w[0].message.args[0]) - def test_nobql_nosql_param_error(self): with self.assertRaises(TypeError) as context: hook.BigQueryBaseCursor("test", "test").run_query( @@ -281,6 +282,39 @@ def test_run_query_sql_dialect_override(self, run_with_config): args, kwargs = run_with_config.call_args self.assertIs(args[0]['query']['useLegacySql'], bool_val) + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_api_resource_configs(self, run_with_config): + for bool_val in [True, False]: + cursor = hook.BigQueryBaseCursor(mock.Mock(), "project_id") + cursor.run_query('query', + api_resource_configs={ + 'query': {'useQueryCache': bool_val}}) + args, kwargs = run_with_config.call_args + self.assertIs(args[0]['query']['useQueryCache'], bool_val) + self.assertIs(args[0]['query']['useLegacySql'], True) + + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_api_resource_configs_duplication_warning(self, run_with_config): + with self.assertRaises(ValueError): + cursor = hook.BigQueryBaseCursor(mock.Mock(), "project_id") + cursor.run_query('query', + use_legacy_sql=True, + api_resource_configs={ + 'query': {'useLegacySql': False}}) + + def test_validate_value(self): + with self.assertRaises(TypeError): + _validate_value("case_1", "a", dict) + self.assertIsNone(_validate_value("case_2", 0, int)) + + def test_duplication_check(self): + with self.assertRaises(ValueError): + key_one = True + _api_resource_configs_duplication_check( + "key_one", key_one, {"key_one": False}) + self.assertIsNone(_api_resource_configs_duplication_check( + "key_one", key_one, {"key_one": True})) + class TestLabelsInRunJob(unittest.TestCase): @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') From 1b552cddcb6b22594f7df4aace3ca383d3a5b5da Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Wed, 20 Jun 2018 20:36:32 +0200 Subject: [PATCH 292/808] [AIRFLOW-2640] Add Cassandra table sensor Just like a partition sensor for Hive, this PR adds a sensor that waits for a table to be created in Cassandra cluster. Closes #3518 from sekikn/AIRFLOW-2640 --- airflow/contrib/hooks/cassandra_hook.py | 19 ++++- ...a_sensor.py => cassandra_record_sensor.py} | 7 +- .../contrib/sensors/cassandra_table_sensor.py | 56 +++++++++++++ docs/code.rst | 3 +- tests/contrib/hooks/test_cassandra_hook.py | 81 +++++++++++++++++-- .../contrib/sensors/test_cassandra_sensor.py | 25 +++++- 6 files changed, 176 insertions(+), 15 deletions(-) rename airflow/contrib/sensors/{cassandra_sensor.py => cassandra_record_sensor.py} (88%) create mode 100644 airflow/contrib/sensors/cassandra_table_sensor.py diff --git a/airflow/contrib/hooks/cassandra_hook.py b/airflow/contrib/hooks/cassandra_hook.py index 704ba0d8d03e2..0e0b47708d5a5 100644 --- a/airflow/contrib/hooks/cassandra_hook.py +++ b/airflow/contrib/hooks/cassandra_hook.py @@ -158,6 +158,21 @@ def get_lb_policy(policy_name, policy_args): child_policy_args) return TokenAwarePolicy(child_policy) + def table_exists(self, table): + """ + Checks if a table exists in Cassandra + + :param table: Target Cassandra table. + Use dot notation to target a specific keyspace. + :type table: string + """ + keyspace = self.keyspace + if '.' in table: + keyspace, table = table.split('.', 1) + cluster_metadata = self.get_conn().cluster.metadata + return (keyspace in cluster_metadata.keyspaces and + table in cluster_metadata.keyspaces[keyspace].tables) + def record_exists(self, table, keys): """ Checks if a record exists in Cassandra @@ -168,12 +183,12 @@ def record_exists(self, table, keys): :param keys: The keys and their values to check the existence. :type keys: dict """ - keyspace = None + keyspace = self.keyspace if '.' in table: keyspace, table = table.split('.', 1) ks = " AND ".join("{}=%({})s".format(key, key) for key in keys.keys()) cql = "SELECT * FROM {keyspace}.{table} WHERE {keys}".format( - keyspace=(keyspace or self.keyspace), table=table, keys=ks) + keyspace=keyspace, table=table, keys=ks) try: rs = self.get_conn().execute(cql, keys) diff --git a/airflow/contrib/sensors/cassandra_sensor.py b/airflow/contrib/sensors/cassandra_record_sensor.py similarity index 88% rename from airflow/contrib/sensors/cassandra_sensor.py rename to airflow/contrib/sensors/cassandra_record_sensor.py index aef66122e90c2..493a6ba6b1e08 100644 --- a/airflow/contrib/sensors/cassandra_sensor.py +++ b/airflow/contrib/sensors/cassandra_record_sensor.py @@ -29,9 +29,10 @@ class CassandraRecordSensor(BaseSensorOperator): primary keys 'p1' and 'p2' to be populated in keyspace 'k' and table 't', instantiate it as follows: - >>> CassandraRecordSensor(table="k.t", keys={"p1": "v1", "p2": "v2"}, - ... cassandra_conn_id="cassandra_default", task_id="cassandra_sensor") - + >>> cassandra_sensor = CassandraRecordSensor(table="k.t", + ... keys={"p1": "v1", "p2": "v2"}, + ... cassandra_conn_id="cassandra_default", + ... task_id="cassandra_sensor") """ template_fields = ('table', 'keys') diff --git a/airflow/contrib/sensors/cassandra_table_sensor.py b/airflow/contrib/sensors/cassandra_table_sensor.py new file mode 100644 index 0000000000000..5a85995acaaec --- /dev/null +++ b/airflow/contrib/sensors/cassandra_table_sensor.py @@ -0,0 +1,56 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from airflow.contrib.hooks.cassandra_hook import CassandraHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults + + +class CassandraTableSensor(BaseSensorOperator): + """ + Checks for the existence of a table in a Cassandra cluster. + + For example, if you want to wait for a table called 't' to be created + in a keyspace 'k', instantiate it as follows: + + >>> cassandra_sensor = CassandraTableSensor(table="k.t", + ... cassandra_conn_id="cassandra_default", + ... task_id="cassandra_sensor") + """ + template_fields = ('table',) + + @apply_defaults + def __init__(self, table, cassandra_conn_id, *args, **kwargs): + """ + Create a new CassandraTableSensor + + :param table: Target Cassandra table. + Use dot notation to target a specific keyspace. + :type table: string + :param cassandra_conn_id: The connection ID to use + when connecting to Cassandra cluster + :type cassandra_conn_id: string + """ + super(CassandraTableSensor, self).__init__(*args, **kwargs) + self.cassandra_conn_id = cassandra_conn_id + self.table = table + + def poke(self, context): + self.log.info('Sensor check existence of table: %s', self.table) + hook = CassandraHook(self.cassandra_conn_id) + return hook.table_exists(self.table) diff --git a/docs/code.rst b/docs/code.rst index f9b5c23eba539..744c9b6cf60ce 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -233,7 +233,8 @@ Sensors .. autoclass:: airflow.contrib.sensors.aws_redshift_cluster_sensor.AwsRedshiftClusterSensor .. autoclass:: airflow.contrib.sensors.bash_sensor.BashSensor .. autoclass:: airflow.contrib.sensors.bigquery_sensor.BigQueryTableSensor -.. autoclass:: airflow.contrib.sensors.cassandra_sensor.CassandraRecordSensor +.. autoclass:: airflow.contrib.sensors.cassandra_record_sensor.CassandraRecordSensor +.. autoclass:: airflow.contrib.sensors.cassandra_table_sensor.CassandraTableSensor .. autoclass:: airflow.contrib.sensors.datadog_sensor.DatadogSensor .. autoclass:: airflow.contrib.sensors.emr_base_sensor.EmrBaseSensor .. autoclass:: airflow.contrib.sensors.emr_job_flow_sensor.EmrJobFlowSensor diff --git a/tests/contrib/hooks/test_cassandra_hook.py b/tests/contrib/hooks/test_cassandra_hook.py index e420ec009583a..9cb0739993173 100644 --- a/tests/contrib/hooks/test_cassandra_hook.py +++ b/tests/contrib/hooks/test_cassandra_hook.py @@ -39,6 +39,25 @@ def setUp(self): conn_id='cassandra_test', conn_type='cassandra', host='host-1,host-2', port='9042', schema='test_keyspace', extra='{"load_balancing_policy":"TokenAwarePolicy"}')) + db.merge_conn( + models.Connection( + conn_id='cassandra_default_with_schema', conn_type='cassandra', + host='localhost', port='9042', schema='s')) + + hook = CassandraHook("cassandra_default") + session = hook.get_conn() + cqls = [ + "DROP SCHEMA IF EXISTS s", + """ + CREATE SCHEMA s WITH REPLICATION = + { 'class' : 'SimpleStrategy', 'replication_factor' : 1 } + """, + ] + for cql in cqls: + session.execute(cql) + + session.shutdown() + hook.shutdown_cluster() def test_get_conn(self): with mock.patch.object(Cluster, "connect") as mock_connect, \ @@ -117,16 +136,10 @@ def _assert_get_lb_policy(self, policy_name, policy_args, expected_policy_type, thrown = True self.assertEqual(should_throw, thrown) - def test_record_exists(self): - hook = CassandraHook() + def test_record_exists_with_keyspace_from_cql(self): + hook = CassandraHook("cassandra_default") session = hook.get_conn() - cqls = [ - "DROP SCHEMA IF EXISTS s", - """ - CREATE SCHEMA s WITH REPLICATION = - { 'class' : 'SimpleStrategy', 'replication_factor' : 1 } - """, "DROP TABLE IF EXISTS s.t", "CREATE TABLE s.t (pk1 text, pk2 text, c text, PRIMARY KEY (pk1, pk2))", "INSERT INTO s.t (pk1, pk2, c) VALUES ('foo', 'bar', 'baz')", @@ -137,6 +150,58 @@ def test_record_exists(self): self.assertTrue(hook.record_exists("s.t", {"pk1": "foo", "pk2": "bar"})) self.assertFalse(hook.record_exists("s.t", {"pk1": "foo", "pk2": "baz"})) + session.shutdown() + hook.shutdown_cluster() + + def test_record_exists_with_keyspace_from_session(self): + hook = CassandraHook("cassandra_default_with_schema") + session = hook.get_conn() + cqls = [ + "DROP TABLE IF EXISTS t", + "CREATE TABLE t (pk1 text, pk2 text, c text, PRIMARY KEY (pk1, pk2))", + "INSERT INTO t (pk1, pk2, c) VALUES ('foo', 'bar', 'baz')", + ] + for cql in cqls: + session.execute(cql) + + self.assertTrue(hook.record_exists("t", {"pk1": "foo", "pk2": "bar"})) + self.assertFalse(hook.record_exists("t", {"pk1": "foo", "pk2": "baz"})) + + session.shutdown() + hook.shutdown_cluster() + + def test_table_exists_with_keyspace_from_cql(self): + hook = CassandraHook("cassandra_default") + session = hook.get_conn() + cqls = [ + "DROP TABLE IF EXISTS s.t", + "CREATE TABLE s.t (pk1 text PRIMARY KEY)", + ] + for cql in cqls: + session.execute(cql) + + self.assertTrue(hook.table_exists("s.t")) + self.assertFalse(hook.table_exists("s.u")) + + session.shutdown() + hook.shutdown_cluster() + + def test_table_exists_with_keyspace_from_session(self): + hook = CassandraHook("cassandra_default_with_schema") + session = hook.get_conn() + cqls = [ + "DROP TABLE IF EXISTS t", + "CREATE TABLE t (pk1 text PRIMARY KEY)", + ] + for cql in cqls: + session.execute(cql) + + self.assertTrue(hook.table_exists("t")) + self.assertFalse(hook.table_exists("u")) + + session.shutdown() + hook.shutdown_cluster() + if __name__ == '__main__': unittest.main() diff --git a/tests/contrib/sensors/test_cassandra_sensor.py b/tests/contrib/sensors/test_cassandra_sensor.py index 0f0e7f5eb3b56..c07bc0be2a44d 100644 --- a/tests/contrib/sensors/test_cassandra_sensor.py +++ b/tests/contrib/sensors/test_cassandra_sensor.py @@ -24,7 +24,8 @@ from airflow import DAG from airflow import configuration -from airflow.contrib.sensors.cassandra_sensor import CassandraRecordSensor +from airflow.contrib.sensors.cassandra_record_sensor import CassandraRecordSensor +from airflow.contrib.sensors.cassandra_table_sensor import CassandraTableSensor from airflow.utils import timezone @@ -54,5 +55,27 @@ def test_poke(self, mock_record_exists): mock_record_exists.assert_called_once_with('t', {'foo': 'bar'}) +class TestCassandraTableSensor(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + args = { + 'owner': 'airflow', + 'start_date': DEFAULT_DATE + } + self.dag = DAG('test_dag_id', default_args=args) + self.sensor = CassandraTableSensor( + task_id='test_task', + cassandra_conn_id='cassandra_default', + dag=self.dag, + table='t', + ) + + @patch("airflow.contrib.hooks.cassandra_hook.CassandraHook.table_exists") + def test_poke(self, mock_table_exists): + self.sensor.poke(None) + mock_table_exists.assert_called_once_with('t') + + if __name__ == '__main__': unittest.main() From e9207f2f577567177887525db62c234d3ee2ebaa Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Wed, 18 Jul 2018 09:04:25 -0700 Subject: [PATCH 293/808] [AIRFLOW-2758] Add a sensor for MongoDB This PR adds a sensor for MongoDB, which waits for some document that matches the given query to be inserted to the specified collection. Closes #3611 from sekikn/AIRFLOW-2758 --- .travis.yml | 1 + airflow/contrib/sensors/mongo_sensor.py | 58 +++++++++++++++++++ tests/contrib/sensors/test_mongo_sensor.py | 65 ++++++++++++++++++++++ 3 files changed, 124 insertions(+) create mode 100644 airflow/contrib/sensors/mongo_sensor.py create mode 100644 tests/contrib/sensors/test_mongo_sensor.py diff --git a/.travis.yml b/.travis.yml index defad4f8ebd9c..fae1ecc5f04f3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -23,6 +23,7 @@ jdk: - openjdk8 services: - cassandra + - mongodb - mysql - postgresql - rabbitmq diff --git a/airflow/contrib/sensors/mongo_sensor.py b/airflow/contrib/sensors/mongo_sensor.py new file mode 100644 index 0000000000000..3ed49a6f9d64f --- /dev/null +++ b/airflow/contrib/sensors/mongo_sensor.py @@ -0,0 +1,58 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from airflow.contrib.hooks.mongo_hook import MongoHook +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils.decorators import apply_defaults + + +class MongoSensor(BaseSensorOperator): + """ + Checks for the existence of a document which + matches the given query in MongoDB. Example: + + >>> mongo_sensor = MongoSensor(collection="coll", + ... query={"key": "value"}, + ... mongo_conn_id="mongo_default", + ... task_id="mongo_sensor") + """ + template_fields = ('collection', 'query') + + @apply_defaults + def __init__(self, collection, query, mongo_conn_id="mongo_default", *args, **kwargs): + """ + Create a new MongoSensor + + :param collection: Target MongoDB collection. + :type collection: string + :param query: The query to find the target document. + :type query: dict + :param mongo_conn_id: The connection ID to use + when connecting to MongoDB. + :type mongo_conn_id: string + """ + super(MongoSensor, self).__init__(*args, **kwargs) + self.mongo_conn_id = mongo_conn_id + self.collection = collection + self.query = query + + def poke(self, context): + self.log.info("Sensor check existence of the document " + "that matches the following query: %s", self.query) + hook = MongoHook(self.mongo_conn_id) + return hook.find(self.collection, self.query, find_one=True) is not None diff --git a/tests/contrib/sensors/test_mongo_sensor.py b/tests/contrib/sensors/test_mongo_sensor.py new file mode 100644 index 0000000000000..876cb99ba4251 --- /dev/null +++ b/tests/contrib/sensors/test_mongo_sensor.py @@ -0,0 +1,65 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + + +import unittest + +from airflow import DAG +from airflow import configuration +from airflow.contrib.hooks.mongo_hook import MongoHook +from airflow.contrib.sensors.mongo_sensor import MongoSensor +from airflow.models import Connection +from airflow.utils import db, timezone + + +DEFAULT_DATE = timezone.datetime(2017, 1, 1) + + +class TestMongoSensor(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + db.merge_conn( + Connection( + conn_id='mongo_test', conn_type='mongo', + host='localhost', port='27017', schema='test')) + + args = { + 'owner': 'airflow', + 'start_date': DEFAULT_DATE + } + self.dag = DAG('test_dag_id', default_args=args) + + hook = MongoHook('mongo_test') + hook.insert_one('foo', {'bar': 'baz'}) + + self.sensor = MongoSensor( + task_id='test_task', + mongo_conn_id='mongo_test', + dag=self.dag, + collection='foo', + query={'bar': 'baz'} + ) + + def test_poke(self): + self.assertTrue(self.sensor.poke(None)) + + +if __name__ == '__main__': + unittest.main() From b1f69889ea5178e05d13d9b461d850d07cffef5f Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 28 Aug 2018 20:36:29 +0800 Subject: [PATCH 294/808] [AIRFLOW-2916] Arg `verify` for AwsHook() & S3 sensors/operators (#3764) --- airflow/contrib/hooks/aws_hook.py | 7 ++++--- airflow/contrib/operators/gcs_to_s3.py | 14 ++++++++++++- airflow/contrib/operators/s3_list_operator.py | 14 ++++++++++++- .../contrib/operators/s3_to_gcs_operator.py | 14 ++++++++++++- airflow/operators/redshift_to_s3_operator.py | 14 ++++++++++++- .../operators/s3_file_transform_operator.py | 21 +++++++++++++++++-- airflow/operators/s3_to_hive_operator.py | 14 ++++++++++++- airflow/operators/s3_to_redshift_operator.py | 14 ++++++++++++- airflow/sensors/s3_key_sensor.py | 14 ++++++++++++- airflow/sensors/s3_prefix_sensor.py | 14 ++++++++++++- .../operators/test_s3_to_gcs_operator.py | 4 ++-- 11 files changed, 129 insertions(+), 15 deletions(-) diff --git a/airflow/contrib/hooks/aws_hook.py b/airflow/contrib/hooks/aws_hook.py index 5716eb3c0305e..9d4a73e1c0c6b 100644 --- a/airflow/contrib/hooks/aws_hook.py +++ b/airflow/contrib/hooks/aws_hook.py @@ -83,8 +83,9 @@ class AwsHook(BaseHook): This class is a thin wrapper around the boto3 python library. """ - def __init__(self, aws_conn_id='aws_default'): + def __init__(self, aws_conn_id='aws_default', verify=None): self.aws_conn_id = aws_conn_id + self.verify = verify def _get_credentials(self, region_name): aws_access_key_id = None @@ -166,13 +167,13 @@ def get_client_type(self, client_type, region_name=None, config=None): session, endpoint_url = self._get_credentials(region_name) return session.client(client_type, endpoint_url=endpoint_url, - config=config) + config=config, verify=self.verify) def get_resource_type(self, resource_type, region_name=None, config=None): session, endpoint_url = self._get_credentials(region_name) return session.resource(resource_type, endpoint_url=endpoint_url, - config=config) + config=config, verify=self.verify) def get_session(self, region_name=None): """Get the underlying boto3.session.""" diff --git a/airflow/contrib/operators/gcs_to_s3.py b/airflow/contrib/operators/gcs_to_s3.py index a87aa3af5c531..0df6170eab377 100644 --- a/airflow/contrib/operators/gcs_to_s3.py +++ b/airflow/contrib/operators/gcs_to_s3.py @@ -47,6 +47,16 @@ class GoogleCloudStorageToS3Operator(GoogleCloudStorageListOperator): :type dest_aws_conn_id: str :param dest_s3_key: The base S3 key to be used to store the files. (templated) :type dest_s3_key: str + :parame dest_verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type dest_verify: bool or str """ template_fields = ('bucket', 'prefix', 'delimiter', 'dest_s3_key') ui_color = '#f0eee4' @@ -60,6 +70,7 @@ def __init__(self, delegate_to=None, dest_aws_conn_id=None, dest_s3_key=None, + dest_verify=None, replace=False, *args, **kwargs): @@ -75,12 +86,13 @@ def __init__(self, ) self.dest_aws_conn_id = dest_aws_conn_id self.dest_s3_key = dest_s3_key + self.dest_verify = dest_verify self.replace = replace def execute(self, context): # use the super to list all files in an Google Cloud Storage bucket files = super(GoogleCloudStorageToS3Operator, self).execute(context) - s3_hook = S3Hook(aws_conn_id=self.dest_aws_conn_id) + s3_hook = S3Hook(aws_conn_id=self.dest_aws_conn_id, verify=self.dest_verify) if not self.replace: # if we are not replacing -> list all files in the S3 bucket diff --git a/airflow/contrib/operators/s3_list_operator.py b/airflow/contrib/operators/s3_list_operator.py index b85691b005fb9..a9e005eed3f65 100644 --- a/airflow/contrib/operators/s3_list_operator.py +++ b/airflow/contrib/operators/s3_list_operator.py @@ -38,6 +38,16 @@ class S3ListOperator(BaseOperator): :type delimiter: string :param aws_conn_id: The connection ID to use when connecting to S3 storage. :type aws_conn_id: string + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str **Example**: The following operator would list all the files @@ -61,6 +71,7 @@ def __init__(self, prefix='', delimiter='', aws_conn_id='aws_default', + verify=None, *args, **kwargs): super(S3ListOperator, self).__init__(*args, **kwargs) @@ -68,9 +79,10 @@ def __init__(self, self.prefix = prefix self.delimiter = delimiter self.aws_conn_id = aws_conn_id + self.verify = verify def execute(self, context): - hook = S3Hook(aws_conn_id=self.aws_conn_id) + hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) self.log.info( 'Getting the list of files from bucket: {0} in prefix: {1} (Delimiter {2})'. diff --git a/airflow/contrib/operators/s3_to_gcs_operator.py b/airflow/contrib/operators/s3_to_gcs_operator.py index 64d7dc7cab976..81c48a9e157bc 100644 --- a/airflow/contrib/operators/s3_to_gcs_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_operator.py @@ -41,6 +41,16 @@ class S3ToGoogleCloudStorageOperator(S3ListOperator): :type delimiter: string :param aws_conn_id: The source S3 connection :type aws_conn_id: string + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str :param dest_gcs_conn_id: The destination connection ID to use when connecting to Google Cloud Storage. :type dest_gcs_conn_id: string @@ -80,6 +90,7 @@ def __init__(self, prefix='', delimiter='', aws_conn_id='aws_default', + verify=None, dest_gcs_conn_id=None, dest_gcs=None, delegate_to=None, @@ -98,6 +109,7 @@ def __init__(self, self.dest_gcs = dest_gcs self.delegate_to = delegate_to self.replace = replace + self.verify = verify if dest_gcs and not self._gcs_object_is_directory(self.dest_gcs): self.log.info( @@ -146,7 +158,7 @@ def execute(self, context): 'There are no new files to sync. Have a nice day!') if files: - hook = S3Hook(aws_conn_id=self.aws_conn_id) + hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) for file in files: # GCS hook builds its own in-memory file so we have to create diff --git a/airflow/operators/redshift_to_s3_operator.py b/airflow/operators/redshift_to_s3_operator.py index 9c1b621dae965..e6682c78df3f7 100644 --- a/airflow/operators/redshift_to_s3_operator.py +++ b/airflow/operators/redshift_to_s3_operator.py @@ -39,6 +39,16 @@ class RedshiftToS3Transfer(BaseOperator): :type redshift_conn_id: string :param aws_conn_id: reference to a specific S3 connection :type aws_conn_id: string + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str :param unload_options: reference to a list of UNLOAD options :type unload_options: list """ @@ -56,6 +66,7 @@ def __init__( s3_key, redshift_conn_id='redshift_default', aws_conn_id='aws_default', + verify=None, unload_options=tuple(), autocommit=False, parameters=None, @@ -68,6 +79,7 @@ def __init__( self.s3_key = s3_key self.redshift_conn_id = redshift_conn_id self.aws_conn_id = aws_conn_id + self.verify = verify self.unload_options = unload_options self.autocommit = autocommit self.parameters = parameters @@ -79,7 +91,7 @@ def __init__( def execute(self, context): self.hook = PostgresHook(postgres_conn_id=self.redshift_conn_id) - self.s3 = S3Hook(aws_conn_id=self.aws_conn_id) + self.s3 = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) credentials = self.s3.get_credentials() unload_options = '\n\t\t\t'.join(self.unload_options) diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py index 5111d455a832c..d1d8cfa58eec8 100644 --- a/airflow/operators/s3_file_transform_operator.py +++ b/airflow/operators/s3_file_transform_operator.py @@ -48,6 +48,17 @@ class S3FileTransformOperator(BaseOperator): :type source_s3_key: str :param source_aws_conn_id: source s3 connection :type source_aws_conn_id: str + :parame source_verify: Whether or not to verify SSL certificates for S3 connetion. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + This is also applicable to ``dest_verify``. + :type source_verify: bool or str :param dest_s3_key: The key to be written from S3. (templated) :type dest_s3_key: str :param dest_aws_conn_id: destination s3 connection @@ -72,14 +83,18 @@ def __init__( transform_script=None, select_expression=None, source_aws_conn_id='aws_default', + source_verify=None, dest_aws_conn_id='aws_default', + dest_verify=None, replace=False, *args, **kwargs): super(S3FileTransformOperator, self).__init__(*args, **kwargs) self.source_s3_key = source_s3_key self.source_aws_conn_id = source_aws_conn_id + self.source_verify = source_verify self.dest_s3_key = dest_s3_key self.dest_aws_conn_id = dest_aws_conn_id + self.dest_verify = dest_verify self.replace = replace self.transform_script = transform_script self.select_expression = select_expression @@ -90,8 +105,10 @@ def execute(self, context): raise AirflowException( "Either transform_script or select_expression must be specified") - source_s3 = S3Hook(aws_conn_id=self.source_aws_conn_id) - dest_s3 = S3Hook(aws_conn_id=self.dest_aws_conn_id) + source_s3 = S3Hook(aws_conn_id=self.source_aws_conn_id, + verify=self.source_verify) + dest_s3 = S3Hook(aws_conn_id=self.dest_aws_conn_id, + verify=self.dest_verify) self.log.info("Downloading source S3 file %s", self.source_s3_key) if not source_s3.check_for_key(self.source_s3_key): diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py index b82ebce6fa295..85f05325f65be 100644 --- a/airflow/operators/s3_to_hive_operator.py +++ b/airflow/operators/s3_to_hive_operator.py @@ -78,6 +78,16 @@ class S3ToHiveTransfer(BaseOperator): :type delimiter: str :param aws_conn_id: source s3 connection :type aws_conn_id: str + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str :param hive_cli_conn_id: destination hive connection :type hive_cli_conn_id: str :param input_compressed: Boolean to determine if file decompression is @@ -107,6 +117,7 @@ def __init__( check_headers=False, wildcard_match=False, aws_conn_id='aws_default', + verify=None, hive_cli_conn_id='hive_cli_default', input_compressed=False, tblproperties=None, @@ -125,6 +136,7 @@ def __init__( self.wildcard_match = wildcard_match self.hive_cli_conn_id = hive_cli_conn_id self.aws_conn_id = aws_conn_id + self.verify = verify self.input_compressed = input_compressed self.tblproperties = tblproperties self.select_expression = select_expression @@ -136,7 +148,7 @@ def __init__( def execute(self, context): # Downloading file from S3 - self.s3 = S3Hook(aws_conn_id=self.aws_conn_id) + self.s3 = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) self.hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id) self.log.info("Downloading S3 file") diff --git a/airflow/operators/s3_to_redshift_operator.py b/airflow/operators/s3_to_redshift_operator.py index 0d7921e9ed0f1..8c83f4437267f 100644 --- a/airflow/operators/s3_to_redshift_operator.py +++ b/airflow/operators/s3_to_redshift_operator.py @@ -39,6 +39,16 @@ class S3ToRedshiftTransfer(BaseOperator): :type redshift_conn_id: string :param aws_conn_id: reference to a specific S3 connection :type aws_conn_id: string + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str :param copy_options: reference to a list of COPY options :type copy_options: list """ @@ -56,6 +66,7 @@ def __init__( s3_key, redshift_conn_id='redshift_default', aws_conn_id='aws_default', + verify=None, copy_options=tuple(), autocommit=False, parameters=None, @@ -67,13 +78,14 @@ def __init__( self.s3_key = s3_key self.redshift_conn_id = redshift_conn_id self.aws_conn_id = aws_conn_id + self.verify = verify self.copy_options = copy_options self.autocommit = autocommit self.parameters = parameters def execute(self, context): self.hook = PostgresHook(postgres_conn_id=self.redshift_conn_id) - self.s3 = S3Hook(aws_conn_id=self.aws_conn_id) + self.s3 = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) credentials = self.s3.get_credentials() copy_options = '\n\t\t\t'.join(self.copy_options) diff --git a/airflow/sensors/s3_key_sensor.py b/airflow/sensors/s3_key_sensor.py index fa2eb786ffa03..9eb7d2fb6553b 100644 --- a/airflow/sensors/s3_key_sensor.py +++ b/airflow/sensors/s3_key_sensor.py @@ -41,6 +41,16 @@ class S3KeySensor(BaseSensorOperator): :type wildcard_match: bool :param aws_conn_id: a reference to the s3 connection :type aws_conn_id: str + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str """ template_fields = ('bucket_key', 'bucket_name') @@ -50,6 +60,7 @@ def __init__(self, bucket_name=None, wildcard_match=False, aws_conn_id='aws_default', + verify=None, *args, **kwargs): super(S3KeySensor, self).__init__(*args, **kwargs) @@ -68,10 +79,11 @@ def __init__(self, self.bucket_key = bucket_key self.wildcard_match = wildcard_match self.aws_conn_id = aws_conn_id + self.verify = verify def poke(self, context): from airflow.hooks.S3_hook import S3Hook - hook = S3Hook(aws_conn_id=self.aws_conn_id) + hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) full_url = "s3://" + self.bucket_name + "/" + self.bucket_key self.log.info('Poking for key : {full_url}'.format(**locals())) if self.wildcard_match: diff --git a/airflow/sensors/s3_prefix_sensor.py b/airflow/sensors/s3_prefix_sensor.py index 42e574c5ee2b8..559e5a1348344 100644 --- a/airflow/sensors/s3_prefix_sensor.py +++ b/airflow/sensors/s3_prefix_sensor.py @@ -40,6 +40,16 @@ class S3PrefixSensor(BaseSensorOperator): :type delimiter: str :param aws_conn_id: a reference to the s3 connection :type aws_conn_id: str + :parame verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + - False: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :type verify: bool or str """ template_fields = ('prefix', 'bucket_name') @@ -49,6 +59,7 @@ def __init__(self, prefix, delimiter='/', aws_conn_id='aws_default', + verify=None, *args, **kwargs): super(S3PrefixSensor, self).__init__(*args, **kwargs) @@ -58,12 +69,13 @@ def __init__(self, self.delimiter = delimiter self.full_url = "s3://" + bucket_name + '/' + prefix self.aws_conn_id = aws_conn_id + self.verify = verify def poke(self, context): self.log.info('Poking for prefix : {self.prefix}\n' 'in bucket s3://{self.bucket_name}'.format(**locals())) from airflow.hooks.S3_hook import S3Hook - hook = S3Hook(aws_conn_id=self.aws_conn_id) + hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify) return hook.check_for_prefix( prefix=self.prefix, delimiter=self.delimiter, diff --git a/tests/contrib/operators/test_s3_to_gcs_operator.py b/tests/contrib/operators/test_s3_to_gcs_operator.py index 807882c324936..97d6eae916d28 100644 --- a/tests/contrib/operators/test_s3_to_gcs_operator.py +++ b/tests/contrib/operators/test_s3_to_gcs_operator.py @@ -88,8 +88,8 @@ def _assert_upload(bucket, object, tmp_filename): uploaded_files = operator.execute(None) - s3_one_mock_hook.assert_called_once_with(aws_conn_id=AWS_CONN_ID) - s3_two_mock_hook.assert_called_once_with(aws_conn_id=AWS_CONN_ID) + s3_one_mock_hook.assert_called_once_with(aws_conn_id=AWS_CONN_ID, verify=None) + s3_two_mock_hook.assert_called_once_with(aws_conn_id=AWS_CONN_ID, verify=None) gcs_mock_hook.assert_called_once_with( google_cloud_storage_conn_id=GCS_CONN_ID, delegate_to=None) From 0a634aafb0da1d7feae440b467c76334af2799f8 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 31 Aug 2018 09:36:24 +0100 Subject: [PATCH 295/808] [AIRFLOW-XXX] Fix Docstrings for Operators (#3820) --- airflow/operators/s3_file_transform_operator.py | 15 ++++++++------- airflow/sensors/s3_key_sensor.py | 2 +- airflow/sensors/s3_prefix_sensor.py | 2 +- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py index d1d8cfa58eec8..4a0b3ad2ee32c 100644 --- a/airflow/operators/s3_file_transform_operator.py +++ b/airflow/operators/s3_file_transform_operator.py @@ -48,15 +48,16 @@ class S3FileTransformOperator(BaseOperator): :type source_s3_key: str :param source_aws_conn_id: source s3 connection :type source_aws_conn_id: str - :parame source_verify: Whether or not to verify SSL certificates for S3 connetion. + :param source_verify: Whether or not to verify SSL certificates for S3 connetion. By default SSL certificates are verified. You can provide the following values: - - False: do not validate SSL certificates. SSL will still be used - (unless use_ssl is False), but SSL certificates will not be - verified. - - path/to/cert/bundle.pem: A filename of the CA cert bundle to uses. - You can specify this argument if you want to use a different - CA cert bundle than the one used by botocore. + + - ``False``: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - ``path/to/cert/bundle.pem``: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. This is also applicable to ``dest_verify``. :type source_verify: bool or str :param dest_s3_key: The key to be written from S3. (templated) diff --git a/airflow/sensors/s3_key_sensor.py b/airflow/sensors/s3_key_sensor.py index 9eb7d2fb6553b..4935561d057f1 100644 --- a/airflow/sensors/s3_key_sensor.py +++ b/airflow/sensors/s3_key_sensor.py @@ -41,7 +41,7 @@ class S3KeySensor(BaseSensorOperator): :type wildcard_match: bool :param aws_conn_id: a reference to the s3 connection :type aws_conn_id: str - :parame verify: Whether or not to verify SSL certificates for S3 connection. + :param verify: Whether or not to verify SSL certificates for S3 connection. By default SSL certificates are verified. You can provide the following values: - False: do not validate SSL certificates. SSL will still be used diff --git a/airflow/sensors/s3_prefix_sensor.py b/airflow/sensors/s3_prefix_sensor.py index 559e5a1348344..4617c97cf360e 100644 --- a/airflow/sensors/s3_prefix_sensor.py +++ b/airflow/sensors/s3_prefix_sensor.py @@ -40,7 +40,7 @@ class S3PrefixSensor(BaseSensorOperator): :type delimiter: str :param aws_conn_id: a reference to the s3 connection :type aws_conn_id: str - :parame verify: Whether or not to verify SSL certificates for S3 connection. + :param verify: Whether or not to verify SSL certificates for S3 connection. By default SSL certificates are verified. You can provide the following values: - False: do not validate SSL certificates. SSL will still be used From ebec5e211b8654e52263e77ad923ab48413bf5df Mon Sep 17 00:00:00 2001 From: Gordon Ball Date: Fri, 7 Sep 2018 18:41:03 +0200 Subject: [PATCH 296/808] [AIRFLOW-2997] Support cluster fields in bigquery (#3838) This adds a cluster_fields argument to the bigquery hook, GCS to bigquery operator and bigquery query operators. This field requests that bigquery store the result of the query/load operation sorted according to the specified fields (the order of fields given is significant). --- airflow/contrib/hooks/bigquery_hook.py | 25 +++++- .../contrib/operators/bigquery_operator.py | 7 ++ airflow/contrib/operators/gcs_to_bq.py | 10 ++- tests/contrib/hooks/test_bigquery_hook.py | 88 +++++++++++++++++++ 4 files changed, 125 insertions(+), 5 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 75c051e9a575a..bd337d53ae5bd 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -496,7 +496,8 @@ def run_query(self, schema_update_options=(), priority='INTERACTIVE', time_partitioning=None, - api_resource_configs=None): + api_resource_configs=None, + cluster_fields=None): """ Executes a BigQuery SQL query. Optionally persists results in a BigQuery table. See here: @@ -563,8 +564,12 @@ def run_query(self, :param time_partitioning: configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications. :type time_partitioning: dict - + :param cluster_fields: Request that the result of this query be stored sorted + by one or more columns. This is only available in combination with + time_partitioning. The order of columns given determines the sort order. + :type cluster_fields: list of str """ + if not api_resource_configs: api_resource_configs = self.api_resource_configs else: @@ -629,6 +634,9 @@ def run_query(self, 'tableId': destination_table, } + if cluster_fields: + cluster_fields = {'fields': cluster_fields} + query_param_list = [ (sql, 'query', None, str), (priority, 'priority', 'INTERACTIVE', str), @@ -639,7 +647,8 @@ def run_query(self, (maximum_bytes_billed, 'maximumBytesBilled', None, float), (time_partitioning, 'timePartitioning', {}, dict), (schema_update_options, 'schemaUpdateOptions', None, tuple), - (destination_dataset_table, 'destinationTable', None, dict) + (destination_dataset_table, 'destinationTable', None, dict), + (cluster_fields, 'clustering', None, dict), ] for param_tuple in query_param_list: @@ -854,7 +863,8 @@ def run_load(self, allow_jagged_rows=False, schema_update_options=(), src_fmt_configs=None, - time_partitioning=None): + time_partitioning=None, + cluster_fields=None): """ Executes a BigQuery load command to load data from Google Cloud Storage to BigQuery. See here: @@ -916,6 +926,10 @@ def run_load(self, :param time_partitioning: configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications. :type time_partitioning: dict + :param cluster_fields: Request that the result of this load be stored sorted + by one or more columns. This is only available in combination with + time_partitioning. The order of columns given determines the sort order. + :type cluster_fields: list of str """ # bigquery only allows certain source formats @@ -979,6 +993,9 @@ def run_load(self, 'timePartitioning': time_partitioning }) + if cluster_fields: + configuration['load'].update({'clustering': {'fields': cluster_fields}}) + if schema_fields: configuration['load']['schema'] = {'fields': schema_fields} diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index c773ebb2e6f21..98e5cfb6beb36 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -98,6 +98,10 @@ class BigQueryOperator(BaseOperator): :param time_partitioning: configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications. :type time_partitioning: dict + :param cluster_fields: Request that the result of this query be stored sorted + by one or more columns. This is only available in conjunction with + time_partitioning. The order of columns given determines the sort order. + :type cluster_fields: list of str """ template_fields = ('bql', 'sql', 'destination_dataset_table', 'labels') @@ -125,6 +129,7 @@ def __init__(self, priority='INTERACTIVE', time_partitioning=None, api_resource_configs=None, + cluster_fields=None, *args, **kwargs): super(BigQueryOperator, self).__init__(*args, **kwargs) @@ -150,6 +155,7 @@ def __init__(self, self.time_partitioning = {} if api_resource_configs is None: self.api_resource_configs = {} + self.cluster_fields = cluster_fields # TODO remove `bql` in Airflow 2.0 if self.bql: @@ -190,6 +196,7 @@ def execute(self, context): priority=self.priority, time_partitioning=self.time_partitioning, api_resource_configs=self.api_resource_configs, + cluster_fields=self.cluster_fields, ) def on_kill(self): diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py index 69acb616594d9..003e828a8aa9d 100644 --- a/airflow/contrib/operators/gcs_to_bq.py +++ b/airflow/contrib/operators/gcs_to_bq.py @@ -114,6 +114,11 @@ class GoogleCloudStorageToBigQueryOperator(BaseOperator): Note that 'field' is not available in concurrency with dataset.table$partition. :type time_partitioning: dict + :param cluster_fields: Request that the result of this load be stored sorted + by one or more columns. This is only available in conjunction with + time_partitioning. The order of columns given determines the sort order. + Not applicable for external tables. + :type cluster_fields: list of str """ template_fields = ('bucket', 'source_objects', 'schema_object', 'destination_project_dataset_table') @@ -146,6 +151,7 @@ def __init__(self, src_fmt_configs=None, external_table=False, time_partitioning=None, + cluster_fields=None, *args, **kwargs): super(GoogleCloudStorageToBigQueryOperator, self).__init__(*args, **kwargs) @@ -183,6 +189,7 @@ def __init__(self, self.schema_update_options = schema_update_options self.src_fmt_configs = src_fmt_configs self.time_partitioning = time_partitioning + self.cluster_fields = cluster_fields def execute(self, context): bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, @@ -238,7 +245,8 @@ def execute(self, context): allow_jagged_rows=self.allow_jagged_rows, schema_update_options=self.schema_update_options, src_fmt_configs=self.src_fmt_configs, - time_partitioning=self.time_partitioning) + time_partitioning=self.time_partitioning, + cluster_fields=self.cluster_fields) if self.max_id_key: cursor.execute('SELECT MAX({}) FROM {}'.format( diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 4a97ea7aa4d3c..0006b0c616b87 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -448,6 +448,94 @@ def test_extra_time_partitioning_options(self): self.assertEqual(tp_out, expect) +class TestClusteringInRunJob(unittest.TestCase): + + @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") + @mock.patch("airflow.contrib.hooks.bigquery_hook.time") + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_load_default(self, mocked_rwc, mocked_time, mocked_logging): + project_id = 12345 + + def run_with_config(config): + self.assertIsNone(config['load'].get('clustering')) + mocked_rwc.side_effect = run_with_config + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + bq_hook.run_load( + destination_project_dataset_table='my_dataset.my_table', + schema_fields=[], + source_uris=[], + ) + + mocked_rwc.assert_called_once() + + @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") + @mock.patch("airflow.contrib.hooks.bigquery_hook.time") + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_load_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + project_id = 12345 + + def run_with_config(config): + self.assertEqual( + config['load']['clustering'], + { + 'fields': ['field1', 'field2'] + } + ) + mocked_rwc.side_effect = run_with_config + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + bq_hook.run_load( + destination_project_dataset_table='my_dataset.my_table', + schema_fields=[], + source_uris=[], + cluster_fields=['field1', 'field2'], + time_partitioning={'type': 'DAY'} + ) + + mocked_rwc.assert_called_once() + + @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") + @mock.patch("airflow.contrib.hooks.bigquery_hook.time") + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_query_default(self, mocked_rwc, mocked_time, mocked_logging): + project_id = 12345 + + def run_with_config(config): + self.assertIsNone(config['query'].get('clustering')) + mocked_rwc.side_effect = run_with_config + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + bq_hook.run_query(sql='select 1') + + mocked_rwc.assert_called_once() + + @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") + @mock.patch("airflow.contrib.hooks.bigquery_hook.time") + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_query_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + project_id = 12345 + + def run_with_config(config): + self.assertEqual( + config['query']['clustering'], + { + 'fields': ['field1', 'field2'] + } + ) + mocked_rwc.side_effect = run_with_config + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + bq_hook.run_query( + sql='select 1', + destination_dataset_table='my_dataset.my_table', + cluster_fields=['field1', 'field2'], + time_partitioning={'type': 'DAY'} + ) + + mocked_rwc.assert_called_once() + + class TestBigQueryHookLegacySql(unittest.TestCase): """Ensure `use_legacy_sql` param in `BigQueryHook` propagates properly.""" From bfd8454004def2c12b1bccfa327e8cffb14dd435 Mon Sep 17 00:00:00 2001 From: Iuliia Volkova Date: Fri, 21 Sep 2018 17:46:59 +0300 Subject: [PATCH 297/808] [AIRFLOW-2887] Added BigQueryCreateEmptyDatasetOperator and create_emty_dataset to bigquery_hook (#3876) --- airflow/contrib/hooks/bigquery_hook.py | 75 ++++++++++++++++++- .../contrib/operators/bigquery_operator.py | 74 ++++++++++++++++-- docs/code.rst | 1 + docs/integration.rst | 8 ++ tests/contrib/hooks/test_bigquery_hook.py | 25 ++++++- .../operators/test_bigquery_operator.py | 25 ++++++- 6 files changed, 195 insertions(+), 13 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index bd337d53ae5bd..b93f5d3b3e679 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -960,7 +960,7 @@ def run_load(self, if not set(allowed_schema_update_options).issuperset( set(schema_update_options)): raise ValueError( - "{0} contains invalid schema update options. " + "{0} contains invalid schema update options." "Please only use one or more of the following options: {1}" .format(schema_update_options, allowed_schema_update_options)) @@ -1350,6 +1350,72 @@ def run_grant_dataset_view_access(self, view_project, view_dataset, view_table, source_project, source_dataset) return source_dataset_resource + def create_empty_dataset(self, dataset_id="", project_id="", + dataset_reference=None): + """ + Create a new empty dataset: + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/insert + + :param project_id: The name of the project where we want to create + an empty a dataset. Don't need to provide, if projectId in dataset_reference. + :type project_id: str + :param dataset_id: The id of dataset. Don't need to provide, + if datasetId in dataset_reference. + :type dataset_id: str + :param dataset_reference: Dataset reference that could be provided + with request body. More info: + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource + :type dataset_reference: dict + """ + + if dataset_reference: + _validate_value('dataset_reference', dataset_reference, dict) + else: + dataset_reference = {} + + if "datasetReference" not in dataset_reference: + dataset_reference["datasetReference"] = {} + + if not dataset_reference["datasetReference"].get("datasetId") and not dataset_id: + raise ValueError( + "{} not provided datasetId. Impossible to create dataset") + + dataset_required_params = [(dataset_id, "datasetId", ""), + (project_id, "projectId", self.project_id)] + for param_tuple in dataset_required_params: + param, param_name, param_default = param_tuple + if param_name not in dataset_reference['datasetReference']: + if param_default and not param: + self.log.info("{} was not specified. Will be used default " + "value {}.".format(param_name, + param_default)) + param = param_default + dataset_reference['datasetReference'].update( + {param_name: param}) + elif param: + _api_resource_configs_duplication_check( + param_name, param, + dataset_reference['datasetReference'], 'dataset_reference') + + dataset_id = dataset_reference.get("datasetReference").get("datasetId") + dataset_project_id = dataset_reference.get("datasetReference").get( + "projectId") + + self.log.info('Creating Dataset: %s in project: %s ', dataset_id, + dataset_project_id) + + try: + self.service.datasets().insert( + projectId=dataset_project_id, + body=dataset_reference).execute() + self.log.info('Dataset created successfully: In project %s ' + 'Dataset %s', dataset_project_id, dataset_id) + + except HttpError as err: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(err.content) + ) + def delete_dataset(self, project_id, dataset_id): """ Delete a dataset of Big query in your project. @@ -1671,10 +1737,11 @@ def _validate_value(key, value, expected_type): key, expected_type, type(value))) -def _api_resource_configs_duplication_check(key, value, config_dict): +def _api_resource_configs_duplication_check(key, value, config_dict, + config_dict_name='api_resource_configs'): if key in config_dict and value != config_dict[key]: raise ValueError("Values of {param_name} param are duplicated. " - "`api_resource_configs` contained {param_name} param " + "{dict_name} contained {param_name} param " "in `query` config and {param_name} was also provided " "with arg to run_query() method. Please remove duplicates." - .format(param_name=key)) + .format(param_name=key, dict_name=config_dict_name)) diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 98e5cfb6beb36..422e58660b523 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -528,12 +528,11 @@ class BigQueryDeleteDatasetOperator(BaseOperator): **Example**: :: - delete_temp_data = BigQueryDeleteDatasetOperator( - dataset_id = 'temp-dataset', - project_id = 'temp-project', - bigquery_conn_id='_my_gcp_conn_', - task_id='Deletetemp', - dag=dag) + delete_temp_data = BigQueryDeleteDatasetOperator(dataset_id = 'temp-dataset', + project_id = 'temp-project', + bigquery_conn_id='_my_gcp_conn_', + task_id='Deletetemp', + dag=dag) """ template_fields = ('dataset_id', 'project_id') @@ -567,3 +566,66 @@ def execute(self, context): project_id=self.project_id, dataset_id=self.dataset_id ) + + +class BigQueryCreateEmptyDatasetOperator(BaseOperator): + """" + This operator is used to create new dataset for your Project in Big query. + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource + + :param project_id: The name of the project where we want to create the dataset. + Don't need to provide, if projectId in dataset_reference. + :type project_id: str + :param dataset_id: The id of dataset. Don't need to provide, + if datasetId in dataset_reference. + :type dataset_id: str + :param dataset_reference: Dataset reference that could be provided with request body. + More info: + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource + :type dataset_reference: dict + + **Example**: :: + + create_new_dataset = BigQueryCreateEmptyDatasetOperator( + dataset_id = 'new-dataset', + project_id = 'my-project', + dataset_reference = {"friendlyName": "New Dataset"} + bigquery_conn_id='_my_gcp_conn_', + task_id='newDatasetCreator', + dag=dag) + + """ + + template_fields = ('dataset_id', 'project_id') + ui_color = '#f0eee4' + + @apply_defaults + def __init__(self, + dataset_id, + project_id=None, + dataset_reference=None, + bigquery_conn_id='bigquery_default', + delegate_to=None, + *args, **kwargs): + self.dataset_id = dataset_id + self.project_id = project_id + self.bigquery_conn_id = bigquery_conn_id + self.dataset_reference = dataset_reference if dataset_reference else {} + self.delegate_to = delegate_to + + self.log.info('Dataset id: %s', self.dataset_id) + self.log.info('Project id: %s', self.project_id) + + super(BigQueryCreateEmptyDatasetOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, + delegate_to=self.delegate_to) + + conn = bq_hook.get_conn() + cursor = conn.cursor() + + cursor.create_empty_dataset( + project_id=self.project_id, + dataset_id=self.dataset_id, + dataset_reference=self.dataset_reference) diff --git a/docs/code.rst b/docs/code.rst index 744c9b6cf60ce..aaa4eec5f7213 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -140,6 +140,7 @@ Operators .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyTableOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyDatasetOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryOperator .. autoclass:: airflow.contrib.operators.bigquery_table_delete_operator.BigQueryTableDeleteOperator .. autoclass:: airflow.contrib.operators.bigquery_to_bigquery.BigQueryToBigQueryOperator diff --git a/docs/integration.rst b/docs/integration.rst index f0d2cecd2556a..c463f89a4a023 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -559,6 +559,7 @@ BigQuery Operators - :ref:`BigQueryCreateEmptyTableOperator` : Creates a new, empty table in the specified BigQuery dataset optionally with schema. - :ref:`BigQueryCreateExternalTableOperator` : Creates a new, external table in the dataset with the data in Google Cloud Storage. - :ref:`BigQueryDeleteDatasetOperator` : Deletes an existing BigQuery dataset. +- :ref:`BigQueryCreateEmptyDatasetOperator` : Creates an empty BigQuery dataset. - :ref:`BigQueryOperator` : Executes BigQuery SQL queries in a specific BigQuery database. - :ref:`BigQueryToBigQueryOperator` : Copy a BigQuery table to another BigQuery table. - :ref:`BigQueryToCloudStorageOperator` : Transfers a BigQuery table to a Google Cloud Storage bucket @@ -613,6 +614,13 @@ BigQueryDeleteDatasetOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator +.. _BigQueryCreateEmptyDatasetOperator: + +BigQueryCreateEmptyDatasetOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyDatasetOperator + .. _BigQueryOperator: BigQueryOperator diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 0006b0c616b87..84fe84043e582 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -337,8 +337,31 @@ def run_with_config(config): mocked_rwc.assert_called_once() -class TestTimePartitioningInRunJob(unittest.TestCase): +class TestDatasetsOperations(unittest.TestCase): + + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_create_empty_dataset_no_dataset_id_err(self, + run_with_configuration): + + with self.assertRaises(ValueError): + hook.BigQueryBaseCursor( + mock.Mock(), "test_create_empty_dataset").create_empty_dataset( + dataset_id="", project_id="") + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_create_empty_dataset_duplicates_call_err(self, + run_with_configuration): + with self.assertRaises(ValueError): + hook.BigQueryBaseCursor( + mock.Mock(), "test_create_empty_dataset").create_empty_dataset( + dataset_id="", project_id="project_test", + dataset_reference={ + "datasetReference": + {"datasetId": "test_dataset", + "projectId": "project_test2"}}) + + +class TestTimePartitioningInRunJob(unittest.TestCase): @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 7c76ab73fec81..9ce3b478d3338 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -22,8 +22,8 @@ from airflow.contrib.operators.bigquery_operator import \ BigQueryCreateExternalTableOperator, \ - BigQueryOperator, \ - BigQueryCreateEmptyTableOperator, BigQueryDeleteDatasetOperator + BigQueryOperator, BigQueryCreateEmptyTableOperator, \ + BigQueryDeleteDatasetOperator, BigQueryCreateEmptyDatasetOperator try: from unittest import mock @@ -136,3 +136,24 @@ def test_execute(self, mock_hook): dataset_id=TEST_DATASET, project_id=TEST_PROJECT_ID ) + + +class BigQueryCreateEmptyDatasetOperatorTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') + def test_execute(self, mock_hook): + operator = BigQueryCreateEmptyDatasetOperator( + task_id=TASK_ID, + dataset_id=TEST_DATASET, + project_id=TEST_PROJECT_ID + ) + + operator.execute(None) + mock_hook.return_value \ + .get_conn() \ + .cursor() \ + .create_empty_dataset \ + .assert_called_once_with( + dataset_id=TEST_DATASET, + project_id=TEST_PROJECT_ID, + dataset_reference={} + ) From 6f0dda0a399e0f05f35f7cb5c30f02052d84edce Mon Sep 17 00:00:00 2001 From: Iuliia Volkova Date: Sun, 7 Oct 2018 21:49:50 +0300 Subject: [PATCH 298/808] [AIRFLOW-3055] add get_dataset and get_datasets_list to bigquery_hook (#3894) * [AIRFLOW-3055] add get_dataset and get_datasets_list to bigquery_hook --- airflow/contrib/hooks/bigquery_hook.py | 80 +++++++++++++++++++++++ tests/contrib/hooks/test_bigquery_hook.py | 62 ++++++++++++++++++ 2 files changed, 142 insertions(+) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index b93f5d3b3e679..6d2239f4e22f6 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -1441,6 +1441,86 @@ def delete_dataset(self, project_id, dataset_id): 'BigQuery job failed. Error was: {}'.format(err.content) ) + def get_dataset(self, dataset_id, project_id=None): + """ + Method returns dataset_resource if dataset exist + and raised 404 error if dataset does not exist + + :param dataset_id: The BigQuery Dataset ID + :type dataset_id: str + :param project_id: The GCP Project ID + :type project_id: str + :return: dataset_resource + + .. seealso:: + For more information, see Dataset Resource content: + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets#resource + """ + + if not dataset_id or not isinstance(dataset_id, str): + raise ValueError("dataset_id argument must be provided and has " + "a type 'str'. You provided: {}".format(dataset_id)) + + dataset_project_id = project_id if project_id else self.project_id + + try: + dataset_resource = self.service.datasets().get( + datasetId=dataset_id, projectId=dataset_project_id).execute() + self.log.info("Dataset Resource: {}".format(dataset_resource)) + except HttpError as err: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(err.content)) + + return dataset_resource + + def get_datasets_list(self, project_id=None): + """ + Method returns full list of BigQuery datasets in the current project + + .. seealso:: + For more information, see: + https://cloud.google.com/bigquery/docs/reference/rest/v2/datasets/list + + :param project_id: Google Cloud Project for which you + try to get all datasets + :type project_id: str + :return: datasets_list + + Example of returned datasets_list: :: + + { + "kind":"bigquery#dataset", + "location":"US", + "id":"your-project:dataset_2_test", + "datasetReference":{ + "projectId":"your-project", + "datasetId":"dataset_2_test" + } + }, + { + "kind":"bigquery#dataset", + "location":"US", + "id":"your-project:dataset_1_test", + "datasetReference":{ + "projectId":"your-project", + "datasetId":"dataset_1_test" + } + } + ] + """ + dataset_project_id = project_id if project_id else self.project_id + + try: + datasets_list = self.service.datasets().list( + projectId=dataset_project_id).execute()['datasets'] + self.log.info("Datasets List: {}".format(datasets_list)) + + except HttpError as err: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(err.content)) + + return datasets_list + class BigQueryCursor(BigQueryBaseCursor): """ diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 84fe84043e582..77a31f032081b 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -360,6 +360,68 @@ def test_create_empty_dataset_duplicates_call_err(self, {"datasetId": "test_dataset", "projectId": "project_test2"}}) + def test_get_dataset_without_dataset_id(self): + with mock.patch.object(hook.BigQueryHook, 'get_service'): + with self.assertRaises(ValueError): + hook.BigQueryBaseCursor( + mock.Mock(), "test_create_empty_dataset").get_dataset( + dataset_id="", project_id="project_test") + + def test_get_dataset(self): + expected_result = { + "kind": "bigquery#dataset", + "location": "US", + "id": "your-project:dataset_2_test", + "datasetReference": { + "projectId": "your-project", + "datasetId": "dataset_2_test" + } + } + dataset_id = "test_dataset" + project_id = "project_test" + + bq_hook = hook.BigQueryBaseCursor(mock.Mock(), project_id) + with mock.patch.object(bq_hook.service, 'datasets') as MockService: + MockService.return_value.get(datasetId=dataset_id, + projectId=project_id).execute.\ + return_value = expected_result + result = bq_hook.get_dataset(dataset_id=dataset_id, + project_id=project_id) + self.assertEqual(result, expected_result) + + def test_get_datasets_list(self): + expected_result = {'datasets': [ + { + "kind": "bigquery#dataset", + "location": "US", + "id": "your-project:dataset_2_test", + "datasetReference": { + "projectId": "your-project", + "datasetId": "dataset_2_test" + } + }, + { + "kind": "bigquery#dataset", + "location": "US", + "id": "your-project:dataset_1_test", + "datasetReference": { + "projectId": "your-project", + "datasetId": "dataset_1_test" + } + } + ]} + project_id = "project_test"'' + + mocked = mock.Mock() + with mock.patch.object(hook.BigQueryBaseCursor(mocked, project_id).service, + 'datasets') as MockService: + MockService.return_value.list( + projectId=project_id).execute.return_value = expected_result + result = hook.BigQueryBaseCursor( + mocked, "test_create_empty_dataset").get_datasets_list( + project_id=project_id) + self.assertEqual(result, expected_result['datasets']) + class TestTimePartitioningInRunJob(unittest.TestCase): @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") From 06949ef33bb0009ece1e375f3fdd68698c232607 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 20 Oct 2018 13:05:17 +0100 Subject: [PATCH 299/808] [AIRFLOW-XXX] BigQuery Hook - Minor Refactoring (#4066) --- airflow/contrib/hooks/bigquery_hook.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 6d2239f4e22f6..17337444e2f98 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -229,7 +229,8 @@ def create_empty_table(self, :param table_id: The Name of the table to be created. :type table_id: str :param schema_fields: If set, the schema field list as defined here: - https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema + https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load.schema + :type schema_fields: list :param labels: a dictionary containing labels for the table, passed to BigQuery :type labels: dict @@ -238,7 +239,6 @@ def create_empty_table(self, schema_fields=[{"name": "emp_name", "type": "STRING", "mode": "REQUIRED"}, {"name": "salary", "type": "INTEGER", "mode": "NULLABLE"}] - :type schema_fields: list :param time_partitioning: configure optional time partitioning fields i.e. partition by field, type and expiration as per API specifications. @@ -526,6 +526,7 @@ def run_query(self, :type flatten_results: boolean :param udf_config: The User Defined Function configuration for the query. See https://cloud.google.com/bigquery/user-defined-functions for details. + :type udf_config: list :param use_legacy_sql: Whether to use legacy SQL (true) or standard SQL (false). If `None`, defaults to `self.use_legacy_sql`. :type use_legacy_sql: boolean @@ -536,7 +537,6 @@ def run_query(self, if you need to provide some params that are not supported by the BigQueryHook like args. :type api_resource_configs: dict - :type udf_config: list :param maximum_billing_tier: Positive integer that serves as a multiplier of the basic price. :type maximum_billing_tier: integer From 53e34797abbb5ad5d6892353a9dd0e41b69c6f4e Mon Sep 17 00:00:00 2001 From: Iuliia Volkova Date: Mon, 22 Oct 2018 12:03:22 +0300 Subject: [PATCH 300/808] [AIRFLOW-461] Support autodetected schemas in BigQuery run_load (#3880) --- airflow/contrib/hooks/bigquery_hook.py | 17 ++++++++++--- .../contrib/operators/bigquery_operator.py | 4 ++-- airflow/contrib/operators/gcs_to_bq.py | 24 ++++++++++++------- tests/contrib/hooks/test_bigquery_hook.py | 8 +++++++ 4 files changed, 39 insertions(+), 14 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 17337444e2f98..248fb986a8168 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -849,8 +849,8 @@ def run_copy(self, def run_load(self, destination_project_dataset_table, - schema_fields, source_uris, + schema_fields=None, source_format='CSV', create_disposition='CREATE_IF_NEEDED', skip_leading_rows=0, @@ -864,7 +864,8 @@ def run_load(self, schema_update_options=(), src_fmt_configs=None, time_partitioning=None, - cluster_fields=None): + cluster_fields=None, + autodetect=False): """ Executes a BigQuery load command to load data from Google Cloud Storage to BigQuery. See here: @@ -882,7 +883,11 @@ def run_load(self, :type destination_project_dataset_table: string :param schema_fields: The schema field list as defined here: https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.load + Required if autodetect=False; optional if autodetect=True. :type schema_fields: list + :param autodetect: Attempt to autodetect the schema for CSV and JSON + source files. + :type autodetect: bool :param source_uris: The source Google Cloud Storage URI (e.g. gs://some-bucket/some-file.txt). A single wild per-object name can be used. @@ -937,6 +942,11 @@ def run_load(self, # if it's not, we raise a ValueError # Refer to this link for more details: # https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.query.tableDefinitions.(key).sourceFormat + + if schema_fields is None and not autodetect: + raise ValueError( + 'You must either pass a schema or autodetect=True.') + if src_fmt_configs is None: src_fmt_configs = {} @@ -971,6 +981,7 @@ def run_load(self, configuration = { 'load': { + 'autodetect': autodetect, 'createDisposition': create_disposition, 'destinationTable': { 'projectId': destination_project, @@ -1734,7 +1745,7 @@ def _split_tablename(table_input, default_project_id, var_name=None): if '.' not in table_input: raise ValueError( - 'Expected deletion_dataset_table name in the format of ' + 'Expected target table name in the format of ' '.
    . Got: {}'.format(table_input)) if not default_project_id: diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 422e58660b523..f861f064c7218 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -306,7 +306,7 @@ def __init__(self, project_id=None, schema_fields=None, gcs_schema_object=None, - time_partitioning={}, + time_partitioning=None, bigquery_conn_id='bigquery_default', google_cloud_storage_conn_id='google_cloud_default', delegate_to=None, @@ -323,7 +323,7 @@ def __init__(self, self.bigquery_conn_id = bigquery_conn_id self.google_cloud_storage_conn_id = google_cloud_storage_conn_id self.delegate_to = delegate_to - self.time_partitioning = time_partitioning + self.time_partitioning = {} if time_partitioning is None else time_partitioning self.labels = labels def execute(self, context): diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py index 003e828a8aa9d..11c2c83bc174b 100644 --- a/airflow/contrib/operators/gcs_to_bq.py +++ b/airflow/contrib/operators/gcs_to_bq.py @@ -152,6 +152,7 @@ def __init__(self, external_table=False, time_partitioning=None, cluster_fields=None, + autodetect=False, *args, **kwargs): super(GoogleCloudStorageToBigQueryOperator, self).__init__(*args, **kwargs) @@ -190,20 +191,24 @@ def __init__(self, self.src_fmt_configs = src_fmt_configs self.time_partitioning = time_partitioning self.cluster_fields = cluster_fields + self.autodetect = autodetect def execute(self, context): bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id, delegate_to=self.delegate_to) - if not self.schema_fields and \ - self.schema_object and \ - self.source_format != 'DATASTORE_BACKUP': - gcs_hook = GoogleCloudStorageHook( - google_cloud_storage_conn_id=self.google_cloud_storage_conn_id, - delegate_to=self.delegate_to) - schema_fields = json.loads(gcs_hook.download( - self.bucket, - self.schema_object).decode("utf-8")) + if not self.schema_fields: + if self.schema_object and self.source_format != 'DATASTORE_BACKUP': + gcs_hook = GoogleCloudStorageHook( + google_cloud_storage_conn_id=self.google_cloud_storage_conn_id, + delegate_to=self.delegate_to) + schema_fields = json.loads(gcs_hook.download( + self.bucket, + self.schema_object).decode("utf-8")) + elif self.schema_object is None and self.autodetect is False: + raise ValueError('At least one of `schema_fields`, `schema_object`, ' + 'or `autodetect` must be passed.') + else: schema_fields = self.schema_fields @@ -234,6 +239,7 @@ def execute(self, context): schema_fields=schema_fields, source_uris=source_uris, source_format=self.source_format, + autodetect=self.autodetect, create_disposition=self.create_disposition, skip_leading_rows=self.skip_leading_rows, write_disposition=self.write_disposition, diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 77a31f032081b..aff9b4cdf7a4d 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -443,6 +443,14 @@ def run_with_config(config): mocked_rwc.assert_called_once() + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_run_with_auto_detect(self, run_with_config): + destination_project_dataset_table = "autodetect.table" + cursor = hook.BigQueryBaseCursor(mock.Mock(), "project_id") + cursor.run_load(destination_project_dataset_table, [], [], autodetect=True) + args, kwargs = run_with_config.call_args + self.assertIs(args[0]['load']['autodetect'], True) + @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') From 7367529353188824e44a4ef0e34e408d0e052a73 Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Fri, 16 Nov 2018 14:31:24 -0800 Subject: [PATCH 301/808] [AIRFLOW-3355] Fix BigQueryCursor.execute to work with Python3 (#4198) BigQueryCursor.execute uses dict.iteritems internally, so it fails with Python3 if binding parameters are provided. This PR fixes this problem. --- airflow/contrib/hooks/bigquery_hook.py | 3 ++- tests/contrib/hooks/test_bigquery_hook.py | 8 ++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 248fb986a8168..5f3899fc2e846 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -25,6 +25,7 @@ import time from builtins import range from copy import deepcopy +from six import iteritems from past.builtins import basestring @@ -1700,7 +1701,7 @@ def _bind_parameters(operation, parameters): """ Helper method that binds parameters to a SQL query. """ # inspired by MySQL Python Connector (conversion.py) string_parameters = {} - for (name, value) in parameters.iteritems(): + for (name, value) in iteritems(parameters): if value is None: string_parameters[name] = 'NULL' elif isinstance(value, basestring): diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index aff9b4cdf7a4d..f47ce00c2f9ae 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -316,6 +316,14 @@ def test_duplication_check(self): "key_one", key_one, {"key_one": True})) +class TestBigQueryCursor(unittest.TestCase): + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_execute_with_parameters(self, mocked_rwc): + hook.BigQueryCursor("test", "test").execute( + "SELECT %(foo)s", {"foo": "bar"}) + mocked_rwc.assert_called_once() + + class TestLabelsInRunJob(unittest.TestCase): @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') def test_run_query_with_arg(self, mocked_rwc): From ad78c6124259d36b962e1abedf344bf8761f0c92 Mon Sep 17 00:00:00 2001 From: Ryan Yuan Date: Sat, 17 Nov 2018 22:52:03 +1100 Subject: [PATCH 302/808] [AIRFLOW-3332] Add method to allow inserting rows into BQ table (#4179) --- airflow/contrib/hooks/bigquery_hook.py | 73 +++++++++++++++++++++++ tests/contrib/hooks/test_bigquery_hook.py | 50 ++++++++++++++++ 2 files changed, 123 insertions(+) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 5f3899fc2e846..75d5fd406a144 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -1533,6 +1533,79 @@ def get_datasets_list(self, project_id=None): return datasets_list + def insert_all(self, project_id, dataset_id, table_id, + rows, ignore_unknown_values=False, + skip_invalid_rows=False, fail_on_error=False): + """ + Method to stream data into BigQuery one record at a time without needing + to run a load job + + .. seealso:: + For more information, see: + https://cloud.google.com/bigquery/docs/reference/rest/v2/tabledata/insertAll + + :param project_id: The name of the project where we have the table + :type project_id: str + :param dataset_id: The name of the dataset where we have the table + :type dataset_id: str + :param table_id: The name of the table + :type table_id: str + :param rows: the rows to insert + :type rows: list + + **Example or rows**: + rows=[{"json": {"a_key": "a_value_0"}}, {"json": {"a_key": "a_value_1"}}] + + :param ignore_unknown_values: [Optional] Accept rows that contain values + that do not match the schema. The unknown values are ignored. + The default value is false, which treats unknown values as errors. + :type ignore_unknown_values: bool + :param skip_invalid_rows: [Optional] Insert all valid rows of a request, + even if invalid rows exist. The default value is false, which causes + the entire request to fail if any invalid rows exist. + :type skip_invalid_rows: bool + :param fail_on_error: [Optional] Force the task to fail if any errors occur. + The default value is false, which indicates the task should not fail + even if any insertion errors occur. + :type fail_on_error: bool + """ + + dataset_project_id = project_id if project_id else self.project_id + + body = { + "rows": rows, + "ignoreUnknownValues": ignore_unknown_values, + "kind": "bigquery#tableDataInsertAllRequest", + "skipInvalidRows": skip_invalid_rows, + } + + try: + self.log.info('Inserting {} row(s) into Table {}:{}.{}'.format( + len(rows), dataset_project_id, + dataset_id, table_id)) + + resp = self.service.tabledata().insertAll( + projectId=dataset_project_id, datasetId=dataset_id, + tableId=table_id, body=body + ).execute() + + if 'insertErrors' not in resp: + self.log.info('All row(s) inserted successfully: {}:{}.{}'.format( + dataset_project_id, dataset_id, table_id)) + else: + error_msg = '{} insert error(s) occured: {}:{}.{}. Details: {}'.format( + len(resp['insertErrors']), + dataset_project_id, dataset_id, table_id, resp['insertErrors']) + if fail_on_error: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(error_msg) + ) + self.log.info(error_msg) + except HttpError as err: + raise AirflowException( + 'BigQuery job failed. Error was: {}'.format(err.content) + ) + class BigQueryCursor(BigQueryBaseCursor): """ diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index f47ce00c2f9ae..24864b202fdf9 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -315,6 +315,56 @@ def test_duplication_check(self): self.assertIsNone(_api_resource_configs_duplication_check( "key_one", key_one, {"key_one": True})) + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_insert_all_succeed(self, run_with_config): + project_id = 'bq-project' + dataset_id = 'bq_dataset' + table_id = 'bq_table' + rows = [ + {"json": {"a_key": "a_value_0"}} + ] + body = { + "rows": rows, + "ignoreUnknownValues": False, + "kind": "bigquery#tableDataInsertAllRequest", + "skipInvalidRows": False, + } + + mock_service = mock.Mock() + method = (mock_service.tabledata.return_value.insertAll) + method.return_value.execute.return_value = { + "kind": "bigquery#tableDataInsertAllResponse" + } + cursor = hook.BigQueryBaseCursor(mock_service, 'project_id') + cursor.insert_all(project_id, dataset_id, table_id, rows) + method.assert_called_with(projectId=project_id, datasetId=dataset_id, + tableId=table_id, body=body) + + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_insert_all_fail(self, run_with_config): + project_id = 'bq-project' + dataset_id = 'bq_dataset' + table_id = 'bq_table' + rows = [ + {"json": {"a_key": "a_value_0"}} + ] + + mock_service = mock.Mock() + method = (mock_service.tabledata.return_value.insertAll) + method.return_value.execute.return_value = { + "kind": "bigquery#tableDataInsertAllResponse", + "insertErrors": [ + { + "index": 1, + "errors": [] + } + ] + } + cursor = hook.BigQueryBaseCursor(mock_service, 'project_id') + with self.assertRaises(Exception): + cursor.insert_all(project_id, dataset_id, table_id, + rows, fail_on_error=True) + class TestBigQueryCursor(unittest.TestCase): @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') From 5055341ac3876a6e8ab09767a19cf57c9c06b4a2 Mon Sep 17 00:00:00 2001 From: Ryan Yuan Date: Thu, 22 Nov 2018 10:16:18 +1100 Subject: [PATCH 303/808] [AIRFLOW-3371] BigQueryHook's Ability to Create View (#4213) --- airflow/contrib/hooks/bigquery_hook.py | 19 ++++++++-- tests/contrib/hooks/test_bigquery_hook.py | 43 +++++++++++++++++++++++ 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 75d5fd406a144..39645ebee4640 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -218,10 +218,11 @@ def create_empty_table(self, table_id, schema_fields=None, time_partitioning=None, - labels=None - ): + labels=None, + view=None): """ Creates a new, empty table in the dataset. + To create a view, which is defined by a SQL query, parse a dictionary to 'view' kwarg :param project_id: The project to create the table into. :type project_id: str @@ -246,6 +247,17 @@ def create_empty_table(self, .. seealso:: https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#timePartitioning :type time_partitioning: dict + :param view: [Optional] A dictionary containing definition for the view. + If set, it will create a view instead of a table: + https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#view + :type view: dict + + **Example**: :: + + view = { + "query": "SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*` LIMIT 1000", + "useLegacySql": False + } :return: """ @@ -267,6 +279,9 @@ def create_empty_table(self, if labels: table_resource['labels'] = labels + if view: + table_resource['view'] = view + self.log.info('Creating Table %s:%s.%s', project_id, dataset_id, table_id) diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 24864b202fdf9..9334fd6dccdfc 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -23,6 +23,7 @@ from google.auth.exceptions import GoogleAuthError import mock +from apiclient.errors import HttpError from airflow.contrib.hooks import bigquery_hook as hook from airflow.contrib.hooks.bigquery_hook import _cleanse_time_partitioning, \ @@ -365,6 +366,48 @@ def test_insert_all_fail(self, run_with_config): cursor.insert_all(project_id, dataset_id, table_id, rows, fail_on_error=True) + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_create_view_fails_on_exception(self, run_with_config): + project_id = 'bq-project' + dataset_id = 'bq_dataset' + table_id = 'bq_table_view' + view = { + 'incorrect_key': 'SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*`', + "useLegacySql": False + } + + mock_service = mock.Mock() + method = (mock_service.tables.return_value.insert) + method.return_value.execute.side_effect = HttpError( + resp={'status': '400'}, content=b'Query is required for views') + cursor = hook.BigQueryBaseCursor(mock_service, project_id) + with self.assertRaises(Exception): + cursor.create_empty_table(project_id, dataset_id, table_id, + view=view) + + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_create_view(self, run_with_config): + project_id = 'bq-project' + dataset_id = 'bq_dataset' + table_id = 'bq_table_view' + view = { + 'query': 'SELECT * FROM `test-project-id.test_dataset_id.test_table_prefix*`', + "useLegacySql": False + } + + mock_service = mock.Mock() + method = (mock_service.tables.return_value.insert) + cursor = hook.BigQueryBaseCursor(mock_service, project_id) + cursor.create_empty_table(project_id, dataset_id, table_id, + view=view) + body = { + 'tableReference': { + 'tableId': table_id + }, + 'view': view + } + method.assert_called_once_with(projectId=project_id, datasetId=dataset_id, body=body) + class TestBigQueryCursor(unittest.TestCase): @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') From 1a92832547213e4f6ee2aad18efcfd0ff704aac4 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sun, 9 Dec 2018 22:29:11 +0000 Subject: [PATCH 304/808] =?UTF-8?q?[AIRFLOW-3438]=20Fix=20default=20values?= =?UTF-8?q?=20in=20BigQuery=20Hook=20&=20BigQueryOperator=20(=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …#4274) --- airflow/contrib/hooks/bigquery_hook.py | 5 +- .../contrib/operators/bigquery_operator.py | 12 ++- .../operators/test_bigquery_operator.py | 87 ++++++++++++++++++- 3 files changed, 93 insertions(+), 11 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 39645ebee4640..9752847dd08b7 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -570,7 +570,7 @@ def run_query(self, :param labels a dictionary containing labels for the job/query, passed to BigQuery :type labels: dict - :param schema_update_options: Allows the schema of the desitination + :param schema_update_options: Allows the schema of the destination table to be updated as a side effect of the query job. :type schema_update_options: tuple :param priority: Specifies a priority for the query. @@ -586,6 +586,9 @@ def run_query(self, :type cluster_fields: list of str """ + if time_partitioning is None: + time_partitioning = {} + if not api_resource_configs: api_resource_configs = self.api_resource_configs else: diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index f861f064c7218..481568d9291e3 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -112,13 +112,13 @@ class BigQueryOperator(BaseOperator): def __init__(self, bql=None, sql=None, - destination_dataset_table=False, + destination_dataset_table=None, write_disposition='WRITE_EMPTY', allow_large_results=False, flatten_results=None, bigquery_conn_id='bigquery_default', delegate_to=None, - udf_config=False, + udf_config=None, use_legacy_sql=True, maximum_billing_tier=None, maximum_bytes_billed=None, @@ -151,10 +151,8 @@ def __init__(self, self.labels = labels self.bq_cursor = None self.priority = priority - if time_partitioning is None: - self.time_partitioning = {} - if api_resource_configs is None: - self.api_resource_configs = {} + self.time_partitioning = time_partitioning + self.api_resource_configs = api_resource_configs self.cluster_fields = cluster_fields # TODO remove `bql` in Airflow 2.0 @@ -181,7 +179,7 @@ def execute(self, context): conn = hook.get_conn() self.bq_cursor = conn.cursor() self.bq_cursor.run_query( - self.sql, + sql=self.sql, destination_dataset_table=self.destination_dataset_table, write_disposition=self.write_disposition, allow_large_results=self.allow_large_results, diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 9ce3b478d3338..0e52beea02657 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -21,9 +21,9 @@ import warnings from airflow.contrib.operators.bigquery_operator import \ - BigQueryCreateExternalTableOperator, \ - BigQueryOperator, BigQueryCreateEmptyTableOperator, \ - BigQueryDeleteDatasetOperator, BigQueryCreateEmptyDatasetOperator + BigQueryCreateExternalTableOperator, BigQueryCreateEmptyTableOperator, \ + BigQueryDeleteDatasetOperator, BigQueryCreateEmptyDatasetOperator, \ + BigQueryOperator try: from unittest import mock @@ -157,3 +157,84 @@ def test_execute(self, mock_hook): project_id=TEST_PROJECT_ID, dataset_reference={} ) + + +class BigQueryOperatorTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') + def test_execute(self, mock_hook): + operator = BigQueryOperator( + task_id=TASK_ID, + sql='Select * from test_table', + destination_dataset_table=None, + write_disposition='WRITE_EMPTY', + allow_large_results=False, + flatten_results=None, + bigquery_conn_id='bigquery_default', + udf_config=None, + use_legacy_sql=True, + maximum_billing_tier=None, + maximum_bytes_billed=None, + create_disposition='CREATE_IF_NEEDED', + schema_update_options=(), + query_params=None, + labels=None, + priority='INTERACTIVE', + time_partitioning=None, + api_resource_configs=None, + cluster_fields=None, + ) + + operator.execute(None) + mock_hook.return_value \ + .get_conn() \ + .cursor() \ + .run_query \ + .assert_called_once_with( + sql='Select * from test_table', + destination_dataset_table=None, + write_disposition='WRITE_EMPTY', + allow_large_results=False, + flatten_results=None, + udf_config=None, + maximum_billing_tier=None, + maximum_bytes_billed=None, + create_disposition='CREATE_IF_NEEDED', + schema_update_options=(), + query_params=None, + labels=None, + priority='INTERACTIVE', + time_partitioning=None, + api_resource_configs=None, + cluster_fields=None, + ) + + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') + def test_bigquery_operator_defaults(self, mock_hook): + operator = BigQueryOperator( + task_id=TASK_ID, + sql='Select * from test_table', + ) + + operator.execute(None) + mock_hook.return_value \ + .get_conn() \ + .cursor() \ + .run_query \ + .assert_called_once_with( + sql='Select * from test_table', + destination_dataset_table=None, + write_disposition='WRITE_EMPTY', + allow_large_results=False, + flatten_results=None, + udf_config=None, + maximum_billing_tier=None, + maximum_bytes_billed=None, + create_disposition='CREATE_IF_NEEDED', + schema_update_options=(), + query_params=None, + labels=None, + priority='INTERACTIVE', + time_partitioning=None, + api_resource_configs=None, + cluster_fields=None, + ) From 21e75dea2997e51aa5d80dce5aa2a2da416961c7 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 29 Dec 2018 11:00:20 +0000 Subject: [PATCH 305/808] [AIRFLOW-3327] Add support for location in BigQueryHook (#4324) --- airflow/contrib/hooks/bigquery_hook.py | 60 +++++++++++--- .../contrib/operators/bigquery_operator.py | 10 ++- tests/contrib/hooks/test_bigquery_hook.py | 83 ++++++++----------- 3 files changed, 92 insertions(+), 61 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 9752847dd08b7..05db8f4e987f3 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -53,10 +53,12 @@ class BigQueryHook(GoogleCloudBaseHook, DbApiHook, LoggingMixin): def __init__(self, bigquery_conn_id='bigquery_default', delegate_to=None, - use_legacy_sql=True): + use_legacy_sql=True, + location=None): super(BigQueryHook, self).__init__( gcp_conn_id=bigquery_conn_id, delegate_to=delegate_to) self.use_legacy_sql = use_legacy_sql + self.location = location def get_conn(self): """ @@ -67,7 +69,9 @@ def get_conn(self): return BigQueryConnection( service=service, project_id=project, - use_legacy_sql=self.use_legacy_sql) + use_legacy_sql=self.use_legacy_sql, + location=self.location, + ) def get_service(self): """ @@ -201,7 +205,8 @@ def __init__(self, service, project_id, use_legacy_sql=True, - api_resource_configs=None): + api_resource_configs=None, + location=None): self.service = service self.project_id = project_id @@ -211,6 +216,7 @@ def __init__(self, self.api_resource_configs = api_resource_configs \ if api_resource_configs else {} self.running_job_id = None + self.location = location def create_empty_table(self, project_id, @@ -513,7 +519,8 @@ def run_query(self, priority='INTERACTIVE', time_partitioning=None, api_resource_configs=None, - cluster_fields=None): + cluster_fields=None, + location=None): """ Executes a BigQuery SQL query. Optionally persists results in a BigQuery table. See here: @@ -584,11 +591,18 @@ def run_query(self, by one or more columns. This is only available in combination with time_partitioning. The order of columns given determines the sort order. :type cluster_fields: list of str + :param location: The geographic location of the job. Required except for + US and EU. See details at + https://cloud.google.com/bigquery/docs/locations#specifying_your_location + :type location: str """ if time_partitioning is None: time_partitioning = {} + if location: + self.location = location + if not api_resource_configs: api_resource_configs = self.api_resource_configs else: @@ -1106,9 +1120,15 @@ def run_with_configuration(self, configuration): keep_polling_job = True while keep_polling_job: try: - job = jobs.get( - projectId=self.project_id, - jobId=self.running_job_id).execute() + if self.location: + job = jobs.get( + projectId=self.project_id, + jobId=self.running_job_id, + location=self.location).execute() + else: + job = jobs.get( + projectId=self.project_id, + jobId=self.running_job_id).execute() if job['status']['state'] == 'DONE': keep_polling_job = False # Check if job had errors. @@ -1137,7 +1157,13 @@ def run_with_configuration(self, configuration): def poll_job_complete(self, job_id): jobs = self.service.jobs() try: - job = jobs.get(projectId=self.project_id, jobId=job_id).execute() + if self.location: + job = jobs.get(projectId=self.project_id, + jobId=job_id, + location=self.location).execute() + else: + job = jobs.get(projectId=self.project_id, + jobId=job_id).execute() if job['status']['state'] == 'DONE': return True except HttpError as err: @@ -1160,9 +1186,15 @@ def cancel_query(self): not self.poll_job_complete(self.running_job_id)): self.log.info('Attempting to cancel job : %s, %s', self.project_id, self.running_job_id) - jobs.cancel( - projectId=self.project_id, - jobId=self.running_job_id).execute() + if self.location: + jobs.cancel( + projectId=self.project_id, + jobId=self.running_job_id, + location=self.location).execute() + else: + jobs.cancel( + projectId=self.project_id, + jobId=self.running_job_id).execute() else: self.log.info('No running BigQuery jobs to cancel.') return @@ -1634,11 +1666,13 @@ class BigQueryCursor(BigQueryBaseCursor): https://github.com/dropbox/PyHive/blob/master/pyhive/common.py """ - def __init__(self, service, project_id, use_legacy_sql=True): + def __init__(self, service, project_id, use_legacy_sql=True, location=None): super(BigQueryCursor, self).__init__( service=service, project_id=project_id, - use_legacy_sql=use_legacy_sql) + use_legacy_sql=use_legacy_sql, + location=location, + ) self.buffersize = None self.page_token = None self.job_id = None diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py index 481568d9291e3..853e80894c10d 100644 --- a/airflow/contrib/operators/bigquery_operator.py +++ b/airflow/contrib/operators/bigquery_operator.py @@ -102,6 +102,10 @@ class BigQueryOperator(BaseOperator): by one or more columns. This is only available in conjunction with time_partitioning. The order of columns given determines the sort order. :type cluster_fields: list of str + :param location: The geographic location of the job. Required except for + US and EU. See details at + https://cloud.google.com/bigquery/docs/locations#specifying_your_location + :type location: str """ template_fields = ('bql', 'sql', 'destination_dataset_table', 'labels') @@ -130,6 +134,7 @@ def __init__(self, time_partitioning=None, api_resource_configs=None, cluster_fields=None, + location=None, *args, **kwargs): super(BigQueryOperator, self).__init__(*args, **kwargs) @@ -154,6 +159,7 @@ def __init__(self, self.time_partitioning = time_partitioning self.api_resource_configs = api_resource_configs self.cluster_fields = cluster_fields + self.location = location # TODO remove `bql` in Airflow 2.0 if self.bql: @@ -175,7 +181,9 @@ def execute(self, context): hook = BigQueryHook( bigquery_conn_id=self.bigquery_conn_id, use_legacy_sql=self.use_legacy_sql, - delegate_to=self.delegate_to) + delegate_to=self.delegate_to, + location=self.location, + ) conn = hook.get_conn() self.bq_cursor = conn.cursor() self.bq_cursor.run_query( diff --git a/tests/contrib/hooks/test_bigquery_hook.py b/tests/contrib/hooks/test_bigquery_hook.py index 9334fd6dccdfc..a7da79039f43c 100644 --- a/tests/contrib/hooks/test_bigquery_hook.py +++ b/tests/contrib/hooks/test_bigquery_hook.py @@ -249,9 +249,7 @@ def test_invalid_schema_update_and_write_disposition(self): ) self.assertIn("schema_update_options is only", str(context.exception)) - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") - def test_cancel_queries(self, mocked_time, mocked_logging): + def test_cancel_queries(self): project_id = 12345 running_job_id = 3 @@ -294,8 +292,7 @@ def test_api_resource_configs(self, run_with_config): self.assertIs(args[0]['query']['useQueryCache'], bool_val) self.assertIs(args[0]['query']['useLegacySql'], True) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_api_resource_configs_duplication_warning(self, run_with_config): + def test_api_resource_configs_duplication_warning(self): with self.assertRaises(ValueError): cursor = hook.BigQueryBaseCursor(mock.Mock(), "project_id") cursor.run_query('query', @@ -316,8 +313,7 @@ def test_duplication_check(self): self.assertIsNone(_api_resource_configs_duplication_check( "key_one", key_one, {"key_one": True})) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_insert_all_succeed(self, run_with_config): + def test_insert_all_succeed(self): project_id = 'bq-project' dataset_id = 'bq_dataset' table_id = 'bq_table' @@ -332,7 +328,7 @@ def test_insert_all_succeed(self, run_with_config): } mock_service = mock.Mock() - method = (mock_service.tabledata.return_value.insertAll) + method = mock_service.tabledata.return_value.insertAll method.return_value.execute.return_value = { "kind": "bigquery#tableDataInsertAllResponse" } @@ -341,8 +337,7 @@ def test_insert_all_succeed(self, run_with_config): method.assert_called_with(projectId=project_id, datasetId=dataset_id, tableId=table_id, body=body) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_insert_all_fail(self, run_with_config): + def test_insert_all_fail(self): project_id = 'bq-project' dataset_id = 'bq_dataset' table_id = 'bq_table' @@ -351,7 +346,7 @@ def test_insert_all_fail(self, run_with_config): ] mock_service = mock.Mock() - method = (mock_service.tabledata.return_value.insertAll) + method = mock_service.tabledata.return_value.insertAll method.return_value.execute.return_value = { "kind": "bigquery#tableDataInsertAllResponse", "insertErrors": [ @@ -366,8 +361,7 @@ def test_insert_all_fail(self, run_with_config): cursor.insert_all(project_id, dataset_id, table_id, rows, fail_on_error=True) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_create_view_fails_on_exception(self, run_with_config): + def test_create_view_fails_on_exception(self): project_id = 'bq-project' dataset_id = 'bq_dataset' table_id = 'bq_table_view' @@ -377,7 +371,7 @@ def test_create_view_fails_on_exception(self, run_with_config): } mock_service = mock.Mock() - method = (mock_service.tables.return_value.insert) + method = mock_service.tables.return_value.insert method.return_value.execute.side_effect = HttpError( resp={'status': '400'}, content=b'Query is required for views') cursor = hook.BigQueryBaseCursor(mock_service, project_id) @@ -385,8 +379,7 @@ def test_create_view_fails_on_exception(self, run_with_config): cursor.create_empty_table(project_id, dataset_id, table_id, view=view) - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_create_view(self, run_with_config): + def test_create_view(self): project_id = 'bq-project' dataset_id = 'bq_dataset' table_id = 'bq_table_view' @@ -396,7 +389,7 @@ def test_create_view(self, run_with_config): } mock_service = mock.Mock() - method = (mock_service.tables.return_value.insert) + method = mock_service.tables.return_value.insert cursor = hook.BigQueryBaseCursor(mock_service, project_id) cursor.create_empty_table(project_id, dataset_id, table_id, view=view) @@ -440,18 +433,14 @@ def run_with_config(config): class TestDatasetsOperations(unittest.TestCase): - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_create_empty_dataset_no_dataset_id_err(self, - run_with_configuration): + def test_create_empty_dataset_no_dataset_id_err(self): with self.assertRaises(ValueError): hook.BigQueryBaseCursor( mock.Mock(), "test_create_empty_dataset").create_empty_dataset( dataset_id="", project_id="") - @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_create_empty_dataset_duplicates_call_err(self, - run_with_configuration): + def test_create_empty_dataset_duplicates_call_err(self): with self.assertRaises(ValueError): hook.BigQueryBaseCursor( mock.Mock(), "test_create_empty_dataset").create_empty_dataset( @@ -525,10 +514,8 @@ def test_get_datasets_list(self): class TestTimePartitioningInRunJob(unittest.TestCase): - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_load_default(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_load_default(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -552,10 +539,8 @@ def test_run_with_auto_detect(self, run_with_config): args, kwargs = run_with_config.call_args self.assertIs(args[0]['load']['autodetect'], True) - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_load_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_load_with_arg(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -579,10 +564,8 @@ def run_with_config(config): mocked_rwc.assert_called_once() - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_query_default(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_query_default(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -594,10 +577,8 @@ def run_with_config(config): mocked_rwc.assert_called_once() - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_query_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_query_with_arg(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -644,10 +625,8 @@ def test_extra_time_partitioning_options(self): class TestClusteringInRunJob(unittest.TestCase): - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_load_default(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_load_default(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -663,10 +642,8 @@ def run_with_config(config): mocked_rwc.assert_called_once() - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_load_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_load_with_arg(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -689,10 +666,8 @@ def run_with_config(config): mocked_rwc.assert_called_once() - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_query_default(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_query_default(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -704,10 +679,8 @@ def run_with_config(config): mocked_rwc.assert_called_once() - @mock.patch("airflow.contrib.hooks.bigquery_hook.LoggingMixin") - @mock.patch("airflow.contrib.hooks.bigquery_hook.time") @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') - def test_run_query_with_arg(self, mocked_rwc, mocked_time, mocked_logging): + def test_run_query_with_arg(self, mocked_rwc): project_id = 12345 def run_with_config(config): @@ -750,5 +723,21 @@ def test_legacy_sql_override_propagates_properly(self, run_with_config): self.assertIs(args[0]['query']['useLegacySql'], False) +class TestBigQueryHookLocation(unittest.TestCase): + @mock.patch.object(hook.BigQueryBaseCursor, 'run_with_configuration') + def test_location_propagates_properly(self, run_with_config): + with mock.patch.object(hook.BigQueryHook, 'get_service'): + bq_hook = hook.BigQueryHook(location=None) + self.assertIsNone(bq_hook.location) + + bq_cursor = hook.BigQueryBaseCursor(mock.Mock(), + 'test-project', + location=None) + self.assertIsNone(bq_cursor.location) + bq_cursor.run_query(sql='select 1', location='US') + run_with_config.assert_called_once() + self.assertEquals(bq_cursor.location, 'US') + + if __name__ == '__main__': unittest.main() From 3355fb73cb28db5abba783064f65be703396621e Mon Sep 17 00:00:00 2001 From: Yohei Onishi Date: Sun, 30 Dec 2018 08:09:00 +0800 Subject: [PATCH 306/808] [AIRFLOW-2939][AIRFLOW-3568] Fix TypeError in GCSToS3Op & S3ToGCSOp (#4371) Fix TypeError on GoogleCloudStorageToS3Operator & S3ToGoogleCloudStorageOperator --- airflow/contrib/operators/gcs_to_s3.py | 2 +- airflow/contrib/operators/s3_to_gcs_operator.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/operators/gcs_to_s3.py b/airflow/contrib/operators/gcs_to_s3.py index 0df6170eab377..d0a7947f9f54e 100644 --- a/airflow/contrib/operators/gcs_to_s3.py +++ b/airflow/contrib/operators/gcs_to_s3.py @@ -100,7 +100,7 @@ def execute(self, context): # Google Cloud Storage and not in S3 bucket_name, _ = S3Hook.parse_s3_url(self.dest_s3_key) existing_files = s3_hook.list_keys(bucket_name) - files = set(files) - set(existing_files) + files = list(set(files) - set(existing_files)) if files: hook = GoogleCloudStorageHook( diff --git a/airflow/contrib/operators/s3_to_gcs_operator.py b/airflow/contrib/operators/s3_to_gcs_operator.py index 81c48a9e157bc..b831adba19c72 100644 --- a/airflow/contrib/operators/s3_to_gcs_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_operator.py @@ -149,7 +149,7 @@ def execute(self, context): else: existing_files.append(f) - files = set(files) - set(existing_files) + files = list(set(files) - set(existing_files)) if len(files) > 0: self.log.info('{0} files are going to be synced: {1}.'.format( len(files), files)) From 75a831f459e7f1a16429f30e3bdfedc01a9e5cb8 Mon Sep 17 00:00:00 2001 From: Cameron Moberg Date: Tue, 7 Aug 2018 09:57:41 -0700 Subject: [PATCH 307/808] [AIRFLOW-2863] Fix GKEClusterHook catching wrong exception (#3711) --- airflow/contrib/hooks/gcp_container_hook.py | 4 +-- .../contrib/hooks/test_gcp_container_hook.py | 34 ++++++++++++++++++- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/airflow/contrib/hooks/gcp_container_hook.py b/airflow/contrib/hooks/gcp_container_hook.py index 970d4ec8992c0..0047b8dbebf2f 100644 --- a/airflow/contrib/hooks/gcp_container_hook.py +++ b/airflow/contrib/hooks/gcp_container_hook.py @@ -23,7 +23,7 @@ from airflow import AirflowException, version from airflow.hooks.base_hook import BaseHook -from google.api_core.exceptions import AlreadyExists +from google.api_core.exceptions import AlreadyExists, NotFound from google.api_core.gapic_v1.method import DEFAULT from google.cloud import container_v1, exceptions from google.cloud.container_v1.gapic.enums import Operation @@ -146,7 +146,7 @@ def delete_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): op = self.wait_for_operation(op) # Returns server-defined url for the resource return op.self_link - except exceptions.NotFound as error: + except NotFound as error: self.log.info('Assuming Success: ' + error.message) def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): diff --git a/tests/contrib/hooks/test_gcp_container_hook.py b/tests/contrib/hooks/test_gcp_container_hook.py index f3705ea4ce7d4..6e13461395bc3 100644 --- a/tests/contrib/hooks/test_gcp_container_hook.py +++ b/tests/contrib/hooks/test_gcp_container_hook.py @@ -61,6 +61,22 @@ def test_delete_cluster(self, wait_mock, convert_mock): wait_mock.assert_called_with(client_delete.return_value) convert_mock.assert_not_called() + @mock.patch( + "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.log") + @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") + @mock.patch( + "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.wait_for_operation") + def test_delete_cluster_not_found(self, wait_mock, convert_mock, log_mock): + from google.api_core.exceptions import NotFound + # To force an error + message = 'Not Found' + self.gke_hook.client.delete_cluster.side_effect = NotFound(message=message) + + self.gke_hook.delete_cluster(None) + wait_mock.assert_not_called() + convert_mock.assert_not_called() + log_mock.info.assert_any_call("Assuming Success: " + message) + @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") @mock.patch( "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.wait_for_operation") @@ -107,7 +123,7 @@ def test_create_cluster_proto(self, wait_mock, convert_mock): @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") @mock.patch( "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.wait_for_operation") - def test_delete_cluster_dict(self, wait_mock, convert_mock): + def test_create_cluster_dict(self, wait_mock, convert_mock): mock_cluster_dict = {'name': CLUSTER_NAME} retry_mock, timeout_mock = mock.Mock(), mock.Mock() @@ -135,6 +151,22 @@ def test_create_cluster_error(self, wait_mock, convert_mock): wait_mock.assert_not_called() convert_mock.assert_not_called() + @mock.patch( + "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.log") + @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") + @mock.patch( + "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.wait_for_operation") + def test_create_cluster_already_exists(self, wait_mock, convert_mock, log_mock): + from google.api_core.exceptions import AlreadyExists + # To force an error + message = 'Already Exists' + self.gke_hook.client.create_cluster.side_effect = AlreadyExists(message=message) + + self.gke_hook.create_cluster({}) + wait_mock.assert_not_called() + self.assertEquals(convert_mock.call_count, 1) + log_mock.info.assert_any_call("Assuming Success: " + message) + class GKEClusterHookGetTest(unittest.TestCase): def setUp(self): From f31a32af6e4d267f8123d0fd8e76a7a2a546eb20 Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Sat, 29 Dec 2018 19:12:17 -0500 Subject: [PATCH 308/808] [AIRFLOW-3550] Standardize GKE hook (#4364) --- airflow/contrib/hooks/gcp_container_hook.py | 83 +++++++++++------- .../contrib/hooks/test_gcp_container_hook.py | 85 +++++++++---------- 2 files changed, 90 insertions(+), 78 deletions(-) diff --git a/airflow/contrib/hooks/gcp_container_hook.py b/airflow/contrib/hooks/gcp_container_hook.py index 0047b8dbebf2f..84173f249847f 100644 --- a/airflow/contrib/hooks/gcp_container_hook.py +++ b/airflow/contrib/hooks/gcp_container_hook.py @@ -21,7 +21,7 @@ import time from airflow import AirflowException, version -from airflow.hooks.base_hook import BaseHook +from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook from google.api_core.exceptions import AlreadyExists, NotFound from google.api_core.gapic_v1.method import DEFAULT @@ -34,15 +34,24 @@ OPERATIONAL_POLL_INTERVAL = 15 -class GKEClusterHook(BaseHook): +class GKEClusterHook(GoogleCloudBaseHook): - def __init__(self, project_id, location): - self.project_id = project_id + def __init__(self, + gcp_conn_id='google_cloud_default', + delegate_to=None, + location=None): + super(GKEClusterHook, self).__init__( + gcp_conn_id=gcp_conn_id, delegate_to=delegate_to) + self._client = None self.location = location - # Add client library info for better error tracking - client_info = ClientInfo(client_library_version='airflow_v' + version.version) - self.client = container_v1.ClusterManagerClient(client_info=client_info) + def get_client(self): + if self._client is None: + credentials = self._get_credentials() + # Add client library info for better error tracking + client_info = ClientInfo(client_library_version='airflow_v' + version.version) + self._client = container_v1.ClusterManagerClient(credentials=credentials, client_info=client_info) + return self._client @staticmethod def _dict_to_proto(py_dict, proto): @@ -60,13 +69,15 @@ def _dict_to_proto(py_dict, proto): dict_json_str = json.dumps(py_dict) return json_format.Parse(dict_json_str, proto) - def wait_for_operation(self, operation): + def wait_for_operation(self, operation, project_id=None): """ Given an operation, continuously fetches the status from Google Cloud until either completion or an error occurring :param operation: The Operation to wait for :type operation: A google.cloud.container_V1.gapic.enums.Operator + :param project_id: Google Cloud Platform project ID + :type project_id: str :return: A new, updated operation fetched from Google Cloud """ self.log.info("Waiting for OPERATION_NAME %s" % operation.name) @@ -79,20 +90,22 @@ def wait_for_operation(self, operation): raise exceptions.GoogleCloudError( "Operation has failed with status: %s" % operation.status) # To update status of operation - operation = self.get_operation(operation.name) + operation = self.get_operation(operation.name, project_id=project_id or self.project_id) return operation - def get_operation(self, operation_name): + def get_operation(self, operation_name, project_id=None): """ Fetches the operation from Google Cloud :param operation_name: Name of operation to fetch :type operation_name: str + :param project_id: Google Cloud Platform project ID + :type project_id: str :return: The new, updated operation from Google Cloud """ - return self.client.get_operation(project_id=self.project_id, - zone=self.location, - operation_id=operation_name) + return self.get_client().get_operation(project_id=project_id or self.project_id, + zone=self.location, + operation_id=operation_name) @staticmethod def _append_label(cluster_proto, key, val): @@ -113,7 +126,7 @@ def _append_label(cluster_proto, key, val): cluster_proto.resource_labels.update({key: val}) return cluster_proto - def delete_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): + def delete_cluster(self, name, project_id=None, retry=DEFAULT, timeout=DEFAULT): """ Deletes the cluster, including the Kubernetes endpoint and all worker nodes. Firewalls and routes that were configured during @@ -124,6 +137,8 @@ def delete_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): :param name: The name of the cluster to delete :type name: str + :param project_id: Google Cloud Platform project ID + :type project_id: str :param retry: Retry object used to determine when/if to retry requests. If None is specified, requests will not be retried. :type retry: google.api_core.retry.Retry @@ -138,18 +153,18 @@ def delete_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): self.project_id, self.location, name)) try: - op = self.client.delete_cluster(project_id=self.project_id, - zone=self.location, - cluster_id=name, - retry=retry, - timeout=timeout) + op = self.get_client().delete_cluster(project_id=project_id or self.project_id, + zone=self.location, + cluster_id=name, + retry=retry, + timeout=timeout) op = self.wait_for_operation(op) # Returns server-defined url for the resource return op.self_link except NotFound as error: self.log.info('Assuming Success: ' + error.message) - def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): + def create_cluster(self, cluster, project_id=None, retry=DEFAULT, timeout=DEFAULT): """ Creates a cluster, consisting of the specified number and type of Google Compute Engine instances. @@ -157,6 +172,8 @@ def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): :param cluster: A Cluster protobuf or dict. If dict is provided, it must be of the same form as the protobuf message google.cloud.container_v1.types.Cluster :type cluster: dict or google.cloud.container_v1.types.Cluster + :param project_id: Google Cloud Platform project ID + :type project_id: str :param retry: A retry object (google.api_core.retry.Retry) used to retry requests. If None is specified, requests will not be retried. :type retry: google.api_core.retry.Retry @@ -184,11 +201,11 @@ def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): self.location, cluster.name)) try: - op = self.client.create_cluster(project_id=self.project_id, - zone=self.location, - cluster=cluster, - retry=retry, - timeout=timeout) + op = self.get_client().create_cluster(project_id=project_id or self.project_id, + zone=self.location, + cluster=cluster, + retry=retry, + timeout=timeout) op = self.wait_for_operation(op) return op.target_link @@ -196,12 +213,14 @@ def create_cluster(self, cluster, retry=DEFAULT, timeout=DEFAULT): self.log.info('Assuming Success: ' + error.message) return self.get_cluster(name=cluster.name).self_link - def get_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): + def get_cluster(self, name, project_id=None, retry=DEFAULT, timeout=DEFAULT): """ Gets details of specified cluster :param name: The name of the cluster to retrieve :type name: str + :param project_id: Google Cloud Platform project ID + :type project_id: str :param retry: A retry object used to retry requests. If None is specified, requests will not be retried. :type retry: google.api_core.retry.Retry @@ -212,12 +231,12 @@ def get_cluster(self, name, retry=DEFAULT, timeout=DEFAULT): :return: A google.cloud.container_v1.types.Cluster instance """ self.log.info("Fetching cluster (project_id={}, zone={}, cluster_name={})".format( - self.project_id, + project_id or self.project_id, self.location, name)) - return self.client.get_cluster(project_id=self.project_id, - zone=self.location, - cluster_id=name, - retry=retry, - timeout=timeout).self_link + return self.get_client().get_cluster(project_id=project_id or self.project_id, + zone=self.location, + cluster_id=name, + retry=retry, + timeout=timeout).self_link diff --git a/tests/contrib/hooks/test_gcp_container_hook.py b/tests/contrib/hooks/test_gcp_container_hook.py index 6e13461395bc3..f4a21da7fc311 100644 --- a/tests/contrib/hooks/test_gcp_container_hook.py +++ b/tests/contrib/hooks/test_gcp_container_hook.py @@ -17,18 +17,12 @@ # specific language governing permissions and limitations # under the License. # +import mock import unittest from airflow import AirflowException from airflow.contrib.hooks.gcp_container_hook import GKEClusterHook -try: - from unittest import mock -except ImportError: - try: - import mock - except ImportError: - mock = None TASK_ID = 'test-gke-cluster-operator' CLUSTER_NAME = 'test-cluster' @@ -38,11 +32,8 @@ class GKEClusterHookDeleteTest(unittest.TestCase): def setUp(self): - with mock.patch.object(GKEClusterHook, "__init__", return_value=None): - self.gke_hook = GKEClusterHook(None, None, None) - self.gke_hook.project_id = TEST_PROJECT_ID - self.gke_hook.location = ZONE - self.gke_hook.client = mock.Mock() + self.gke_hook = GKEClusterHook(location=ZONE) + self.gke_hook._client = mock.Mock() @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") @mock.patch( @@ -50,9 +41,9 @@ def setUp(self): def test_delete_cluster(self, wait_mock, convert_mock): retry_mock, timeout_mock = mock.Mock(), mock.Mock() - client_delete = self.gke_hook.client.delete_cluster = mock.Mock() + client_delete = self.gke_hook._client.delete_cluster = mock.Mock() - self.gke_hook.delete_cluster(name=CLUSTER_NAME, retry=retry_mock, + self.gke_hook.delete_cluster(name=CLUSTER_NAME, project_id=TEST_PROJECT_ID, retry=retry_mock, timeout=timeout_mock) client_delete.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE, @@ -70,7 +61,7 @@ def test_delete_cluster_not_found(self, wait_mock, convert_mock, log_mock): from google.api_core.exceptions import NotFound # To force an error message = 'Not Found' - self.gke_hook.client.delete_cluster.side_effect = NotFound(message=message) + self.gke_hook._client.delete_cluster.side_effect = NotFound(message=message) self.gke_hook.delete_cluster(None) wait_mock.assert_not_called() @@ -82,7 +73,7 @@ def test_delete_cluster_not_found(self, wait_mock, convert_mock, log_mock): "airflow.contrib.hooks.gcp_container_hook.GKEClusterHook.wait_for_operation") def test_delete_cluster_error(self, wait_mock, convert_mock): # To force an error - self.gke_hook.client.delete_cluster.side_effect = AirflowException('400') + self.gke_hook._client.delete_cluster.side_effect = AirflowException('400') with self.assertRaises(AirflowException): self.gke_hook.delete_cluster(None) @@ -92,11 +83,8 @@ def test_delete_cluster_error(self, wait_mock, convert_mock): class GKEClusterHookCreateTest(unittest.TestCase): def setUp(self): - with mock.patch.object(GKEClusterHook, "__init__", return_value=None): - self.gke_hook = GKEClusterHook(None, None, None) - self.gke_hook.project_id = TEST_PROJECT_ID - self.gke_hook.location = ZONE - self.gke_hook.client = mock.Mock() + self.gke_hook = GKEClusterHook(location=ZONE) + self.gke_hook._client = mock.Mock() @mock.patch("airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._dict_to_proto") @mock.patch( @@ -109,9 +97,9 @@ def test_create_cluster_proto(self, wait_mock, convert_mock): retry_mock, timeout_mock = mock.Mock(), mock.Mock() - client_create = self.gke_hook.client.create_cluster = mock.Mock() + client_create = self.gke_hook._client.create_cluster = mock.Mock() - self.gke_hook.create_cluster(mock_cluster_proto, retry=retry_mock, + self.gke_hook.create_cluster(mock_cluster_proto, project_id=TEST_PROJECT_ID, retry=retry_mock, timeout=timeout_mock) client_create.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE, @@ -127,10 +115,10 @@ def test_create_cluster_dict(self, wait_mock, convert_mock): mock_cluster_dict = {'name': CLUSTER_NAME} retry_mock, timeout_mock = mock.Mock(), mock.Mock() - client_create = self.gke_hook.client.create_cluster = mock.Mock() + client_create = self.gke_hook._client.create_cluster = mock.Mock() proto_mock = convert_mock.return_value = mock.Mock() - self.gke_hook.create_cluster(mock_cluster_dict, retry=retry_mock, + self.gke_hook.create_cluster(mock_cluster_dict, project_id=TEST_PROJECT_ID, retry=retry_mock, timeout=timeout_mock) client_create.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE, @@ -160,7 +148,7 @@ def test_create_cluster_already_exists(self, wait_mock, convert_mock, log_mock): from google.api_core.exceptions import AlreadyExists # To force an error message = 'Already Exists' - self.gke_hook.client.create_cluster.side_effect = AlreadyExists(message=message) + self.gke_hook._client.create_cluster.side_effect = AlreadyExists(message=message) self.gke_hook.create_cluster({}) wait_mock.assert_not_called() @@ -170,18 +158,15 @@ def test_create_cluster_already_exists(self, wait_mock, convert_mock, log_mock): class GKEClusterHookGetTest(unittest.TestCase): def setUp(self): - with mock.patch.object(GKEClusterHook, "__init__", return_value=None): - self.gke_hook = GKEClusterHook(None, None, None) - self.gke_hook.project_id = TEST_PROJECT_ID - self.gke_hook.location = ZONE - self.gke_hook.client = mock.Mock() + self.gke_hook = GKEClusterHook(location=ZONE) + self.gke_hook._client = mock.Mock() def test_get_cluster(self): retry_mock, timeout_mock = mock.Mock(), mock.Mock() - client_get = self.gke_hook.client.get_cluster = mock.Mock() + client_get = self.gke_hook._client.get_cluster = mock.Mock() - self.gke_hook.get_cluster(name=CLUSTER_NAME, retry=retry_mock, + self.gke_hook.get_cluster(name=CLUSTER_NAME, project_id=TEST_PROJECT_ID, retry=retry_mock, timeout=timeout_mock) client_get.assert_called_with(project_id=TEST_PROJECT_ID, zone=ZONE, @@ -192,18 +177,26 @@ def test_get_cluster(self): class GKEClusterHookTest(unittest.TestCase): def setUp(self): - with mock.patch.object(GKEClusterHook, "__init__", return_value=None): - self.gke_hook = GKEClusterHook(None, None, None) - self.gke_hook.project_id = TEST_PROJECT_ID - self.gke_hook.location = ZONE - self.gke_hook.client = mock.Mock() + self.gke_hook = GKEClusterHook(location=ZONE) + self.gke_hook._client = mock.Mock() + + @mock.patch('airflow.contrib.hooks.gcp_container_hook.container_v1.ClusterManagerClient') + @mock.patch('airflow.contrib.hooks.gcp_container_hook.ClientInfo') + @mock.patch('airflow.contrib.hooks.gcp_container_hook.GKEClusterHook._get_credentials') + def test_get_client(self, mock_get_credentials, mock_client_info, mock_client): + self.gke_hook._client = None + self.gke_hook.get_client() + mock_get_credentials.assert_called() + mock_client.assert_called_with( + credentials=mock_get_credentials.return_value, + client_info=mock_client_info.return_value) def test_get_operation(self): - self.gke_hook.client.get_operation = mock.Mock() - self.gke_hook.get_operation('TEST_OP') - self.gke_hook.client.get_operation.assert_called_with(project_id=TEST_PROJECT_ID, - zone=ZONE, - operation_id='TEST_OP') + self.gke_hook._client.get_operation = mock.Mock() + self.gke_hook.get_operation('TEST_OP', project_id=TEST_PROJECT_ID) + self.gke_hook._client.get_operation.assert_called_with(project_id=TEST_PROJECT_ID, + zone=ZONE, + operation_id='TEST_OP') def test_append_label(self): key = 'test-key' @@ -251,11 +244,11 @@ def test_wait_for_response_running(self, time_mock, operation_mock): # Status goes from Running -> Pending -> Done operation_mock.side_effect = [pending_op, done_op] - self.gke_hook.wait_for_operation(running_op) + self.gke_hook.wait_for_operation(running_op, project_id=TEST_PROJECT_ID) self.assertEqual(time_mock.call_count, 3) - operation_mock.assert_any_call(running_op.name) - operation_mock.assert_any_call(pending_op.name) + operation_mock.assert_any_call(running_op.name, project_id=TEST_PROJECT_ID) + operation_mock.assert_any_call(pending_op.name, project_id=TEST_PROJECT_ID) self.assertEqual(operation_mock.call_count, 2) @mock.patch("google.protobuf.json_format.Parse") From f6a07a8c2226ecb3cfdf127da5b7339118138412 Mon Sep 17 00:00:00 2001 From: Tom Kunc Date: Tue, 24 Jul 2018 01:12:09 +0100 Subject: [PATCH 309/808] [AIRFLOW-2712] Pass annotations to KubernetesExecutorConfig --- airflow/contrib/example_dags/__init__.py | 0 .../example_kubernetes_annotation.py | 47 +++++++++++++++++++ .../contrib/executors/kubernetes_executor.py | 14 ++++-- .../kubernetes/worker_configuration.py | 6 +-- .../minikube/test_kubernetes_executor.py | 18 +++---- 5 files changed, 69 insertions(+), 16 deletions(-) create mode 100644 airflow/contrib/example_dags/__init__.py create mode 100644 airflow/contrib/example_dags/example_kubernetes_annotation.py diff --git a/airflow/contrib/example_dags/__init__.py b/airflow/contrib/example_dags/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/airflow/contrib/example_dags/example_kubernetes_annotation.py b/airflow/contrib/example_dags/example_kubernetes_annotation.py new file mode 100644 index 0000000000000..058baf69901f7 --- /dev/null +++ b/airflow/contrib/example_dags/example_kubernetes_annotation.py @@ -0,0 +1,47 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import print_function +import airflow +from airflow.operators.python_operator import PythonOperator +from airflow.models import DAG + +args = { + 'owner': 'airflow', + 'start_date': airflow.utils.dates.days_ago(2) +} + +dag = DAG( + dag_id='example_kubernetes_annotation', default_args=args, + schedule_interval=None +) + + +def print_stuff(): + print("annotated!") + + +# You can use annotations on your kubernetes pods! +start_task = PythonOperator( + task_id="start_task", python_callable=print_stuff, dag=dag, + executor_config={ + "KubernetesExecutor": { + "annotations": {"test": "annotation"} + } + } +) diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 4021bce356a40..5104d150f0065 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -39,7 +39,8 @@ class KubernetesExecutorConfig: def __init__(self, image=None, image_pull_policy=None, request_memory=None, request_cpu=None, limit_memory=None, limit_cpu=None, - gcp_service_account_key=None, node_selectors=None, affinity=None): + gcp_service_account_key=None, node_selectors=None, affinity=None, + annotations=None): self.image = image self.image_pull_policy = image_pull_policy self.request_memory = request_memory @@ -49,15 +50,16 @@ def __init__(self, image=None, image_pull_policy=None, request_memory=None, self.gcp_service_account_key = gcp_service_account_key self.node_selectors = node_selectors self.affinity = affinity + self.annotations = annotations def __repr__(self): return "{}(image={}, image_pull_policy={}, request_memory={}, request_cpu={}, " \ "limit_memory={}, limit_cpu={}, gcp_service_account_key={}, " \ - "node_selectors={}, affinity={})" \ + "node_selectors={}, affinity={}, annotations={})" \ .format(KubernetesExecutorConfig.__name__, self.image, self.image_pull_policy, self.request_memory, self.request_cpu, self.limit_memory, self.limit_cpu, self.gcp_service_account_key, self.node_selectors, - self.affinity) + self.affinity, self.annotations) @staticmethod def from_dict(obj): @@ -79,7 +81,8 @@ def from_dict(obj): limit_cpu=namespaced.get('limit_cpu', None), gcp_service_account_key=namespaced.get('gcp_service_account_key', None), node_selectors=namespaced.get('node_selectors', None), - affinity=namespaced.get('affinity', None) + affinity=namespaced.get('affinity', None), + annotations=namespaced.get('annotations', {}), ) def as_dict(self): @@ -92,7 +95,8 @@ def as_dict(self): 'limit_cpu': self.limit_cpu, 'gcp_service_account_key': self.gcp_service_account_key, 'node_selectors': self.node_selectors, - 'affinity': self.affinity + 'affinity': self.affinity, + 'annotations': self.annotations, } diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index b21b6db9871b9..ad19426f0f565 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -185,9 +185,9 @@ def make_pod(self, namespace, worker_uuid, pod_id, dag_id, task_id, execution_da limit_cpu=kube_executor_config.limit_cpu ) gcp_sa_key = kube_executor_config.gcp_service_account_key - annotations = { - 'iam.cloud.google.com/service-account': gcp_sa_key - } if gcp_sa_key else {} + annotations = kube_executor_config.annotations.copy() + if gcp_sa_key: + annotations['iam.cloud.google.com/service-account'] = gcp_sa_key return Pod( namespace=namespace, diff --git a/tests/contrib/minikube/test_kubernetes_executor.py b/tests/contrib/minikube/test_kubernetes_executor.py index 2d1105111b5b2..45d4124d07973 100644 --- a/tests/contrib/minikube/test_kubernetes_executor.py +++ b/tests/contrib/minikube/test_kubernetes_executor.py @@ -150,8 +150,9 @@ def start_dag(self, dag_id, host): def test_integration_run_dag(self): host = get_minikube_host() + dag_id = 'example_kubernetes_annotation' - result_json = self.start_dag(dag_id='example_python_operator', host=host) + result_json = self.start_dag(dag_id=dag_id, host=host) self.assertGreater(len(result_json['items']), 0) @@ -161,19 +162,20 @@ def test_integration_run_dag(self): # Wait 100 seconds for the operator to complete self.monitor_task(host=host, execution_date=execution_date, - dag_id='example_python_operator', - task_id='print_the_context', + dag_id=dag_id, + task_id='start_task', expected_final_state='success', timeout=100) self.ensure_dag_expected_state(host=host, execution_date=execution_date, - dag_id='example_python_operator', + dag_id=dag_id, expected_final_state='success', timeout=100) def test_integration_run_dag_with_scheduler_failure(self): host = get_minikube_host() + dag_id = 'example_kubernetes_annotation' - result_json = self.start_dag(dag_id='example_python_operator', host=host) + result_json = self.start_dag(dag_id=dag_id, host=host) self.assertGreater(len(result_json['items']), 0) @@ -187,13 +189,13 @@ def test_integration_run_dag_with_scheduler_failure(self): # Wait 100 seconds for the operator to complete self.monitor_task(host=host, execution_date=execution_date, - dag_id='example_python_operator', - task_id='print_the_context', + dag_id=dag_id, + task_id='start_task', expected_final_state='success', timeout=120) self.ensure_dag_expected_state(host=host, execution_date=execution_date, - dag_id='example_python_operator', + dag_id=dag_id, expected_final_state='success', timeout=100) From 53a8440bf11191b1b7780dc08ff874ca723445f2 Mon Sep 17 00:00:00 2001 From: Ravi Kotecha Date: Fri, 22 Jun 2018 16:37:46 +0200 Subject: [PATCH 310/808] [AIRFLOW-2661] fix config dags_volume_subpath and logs_volume_subpath Make sure you have checked _all_ steps below. ### JIRA - [x] My PR addresses the following [Airflow JIRA] (https://issues.apache.org/jira/browse/AIRFLOW/) issues and references them in the PR title. For example, "\[AIRFLOW-XXX\] My Airflow PR" - https://issues.apache.org/jira/browse/AIRFLOW-2661 - In case you are fixing a typo in the documentation you can prepend your commit with \[AIRFLOW-XXX\], code changes always need a JIRA issue. ### Description - [x] Here are some details about my PR, including screenshots of any UI changes: Changes the use of `log_volume_subpath` and `dags_volume_subpath` which are now passed into the construction of the worker pod's volumeMounts instead of the volume section (where subPath is not valid). ### Tests - [x] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason: Unit tests have been added but I'm not sure how to add integration tests for this without breaking the other minikube tests ### Commits - [x] My commits all reference JIRA issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git- commit/)": 1. Subject is separated from body by a blank line 2. Subject is limited to 50 characters 3. Subject does not end with a period 4. Subject uses the imperative mood ("add", not "adding") 5. Body wraps at 72 characters 6. Body explains "what" and "why", not "how" ### Documentation - [x] In case of new functionality, my PR adds documentation that describes how to use it. No new functionality added ### Code Quality - [x] Passes `git diff upstream/master -u -- "*.py" | flake8 --diff` Closes #3537 from r4vi/AIRFLOW-2661 --- .../kubernetes/worker_configuration.py | 41 +++++++----- .../executors/test_kubernetes_executor.py | 62 +++++++++++++++++++ 2 files changed, 86 insertions(+), 17 deletions(-) diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index ad19426f0f565..86ee270abc1b2 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -109,34 +109,31 @@ def init_volumes_and_mounts(self): dags_volume_name = 'airflow-dags' logs_volume_name = 'airflow-logs' - def _construct_volume(name, claim, subpath=None): - vo = { + def _construct_volume(name, claim): + volume = { 'name': name } if claim: - vo['persistentVolumeClaim'] = { + volume['persistentVolumeClaim'] = { 'claimName': claim } - if subpath: - vo['subPath'] = subpath else: - vo['emptyDir'] = {} - return vo + volume['emptyDir'] = {} + return volume volumes = [ _construct_volume( dags_volume_name, - self.kube_config.dags_volume_claim, - self.kube_config.dags_volume_subpath + self.kube_config.dags_volume_claim ), _construct_volume( logs_volume_name, - self.kube_config.logs_volume_claim, - self.kube_config.logs_volume_subpath + self.kube_config.logs_volume_claim ) ] dag_volume_mount_path = "" + if self.kube_config.dags_volume_claim: dag_volume_mount_path = self.worker_airflow_dags else: @@ -144,15 +141,25 @@ def _construct_volume(name, claim, subpath=None): self.worker_airflow_dags, self.kube_config.git_subpath ) - - volume_mounts = [{ + dags_volume_mount = { 'name': dags_volume_name, 'mountPath': dag_volume_mount_path, - 'readOnly': True - }, { + 'readOnly': True, + } + if self.kube_config.dags_volume_subpath: + dags_volume_mount['subPath'] = self.kube_config.dags_volume_subpath + + logs_volume_mount = { 'name': logs_volume_name, - 'mountPath': self.worker_airflow_logs - }] + 'mountPath': self.worker_airflow_logs, + } + if self.kube_config.dags_volume_subpath: + logs_volume_mount['subPath'] = self.kube_config.logs_volume_subpath + + volume_mounts = [ + dags_volume_mount, + logs_volume_mount + ] # Mount the airflow.cfg file via a configmap the user has specified if self.kube_config.airflow_configmap: diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index a42a87dd17996..796a8e9ad1c1e 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -14,6 +14,7 @@ # import unittest +import mock import re import string import random @@ -21,6 +22,7 @@ try: from airflow.contrib.executors.kubernetes_executor import AirflowKubernetesScheduler + from airflow.contrib.kubernetes.worker_configuration import WorkerConfiguration except ImportError: AirflowKubernetesScheduler = None @@ -74,5 +76,65 @@ def test_execution_date_serialize_deserialize(self): self.assertEquals(datetime_obj, new_datetime_obj) +class TestKubernetesWorkerConfiguration(unittest.TestCase): + """ + Tests that if dags_volume_subpath/logs_volume_subpath configuration + options are passed to worker pod config + """ + def setUp(self): + if AirflowKubernetesScheduler is None: + self.skipTest("kubernetes python package is not installed") + + self.pod = mock.patch( + 'airflow.contrib.kubernetes.worker_configuration.Pod' + ) + self.resources = mock.patch( + 'airflow.contrib.kubernetes.worker_configuration.Resources' + ) + self.secret = mock.patch( + 'airflow.contrib.kubernetes.worker_configuration.Secret' + ) + + for patcher in [self.pod, self.resources, self.secret]: + self.mock_foo = patcher.start() + self.addCleanup(patcher.stop) + + self.kube_config = mock.MagicMock() + self.kube_config.airflow_home = '/' + self.kube_config.airflow_dags = 'dags' + self.kube_config.airflow_dags = 'logs' + self.kube_config.dags_volume_subpath = None + self.kube_config.logs_volume_subpath = None + + def test_worker_configuration_no_subpaths(self): + worker_config = WorkerConfiguration(self.kube_config) + volumes, volume_mounts = worker_config.init_volumes_and_mounts() + for volume_or_mount in volumes + volume_mounts: + if volume_or_mount['name'] != 'airflow-config': + self.assertNotIn( + 'subPath', volume_or_mount, + "subPath shouldn't be defined" + ) + + def test_worker_with_subpaths(self): + self.kube_config.dags_volume_subpath = 'dags' + self.kube_config.logs_volume_subpath = 'logs' + worker_config = WorkerConfiguration(self.kube_config) + volumes, volume_mounts = worker_config.init_volumes_and_mounts() + + for volume in volumes: + self.assertNotIn( + 'subPath', volume, + "subPath isn't valid configuration for a volume" + ) + + for volume_mount in volume_mounts: + if volume_mount['name'] != 'airflow-config': + self.assertIn( + 'subPath', volume_mount, + "subPath should've been passed to volumeMount configuration" + ) + + if __name__ == '__main__': unittest.main() From 08c1713c289bc15a272ebbaba15d4575baa35f40 Mon Sep 17 00:00:00 2001 From: roc Date: Wed, 20 Jun 2018 20:37:39 +0200 Subject: [PATCH 311/808] [AIRFLOW-2645][AIRFLOW-2617] Add worker_container_image_pull_policy Set worker_container_image_pull_policy in default_airflow.cfg As AIRFLOW-2617 added worker_container_image_pull_policy config to the section of kubernetes, but the airflow_default.cfg was not updated, this PR add worker_container_image_pull_policy to default_airflow.cfg. Closes #3521 from imroc/AIRFLOW-2645 --- airflow/config_templates/default_airflow.cfg | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 7159c137f4c25..b5182434ccff8 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -568,9 +568,10 @@ elasticsearch_log_id_template = {{dag_id}}-{{task_id}}-{{execution_date}}-{{try_ elasticsearch_end_of_log_mark = end_of_log [kubernetes] -# The repository and tag of the Kubernetes Image for the Worker to Run +# The repository, tag and imagePullPolicy of the Kubernetes Image for the Worker to Run worker_container_repository = worker_container_tag = +worker_container_image_pull_policy = IfNotPresent # If True (default), worker pods will be deleted upon termination delete_worker_pods = True From b446c56b2f251134703ac19721ef8225ef17e23b Mon Sep 17 00:00:00 2001 From: Shintaro Murakami Date: Wed, 4 Jul 2018 17:48:51 +0100 Subject: [PATCH 312/808] [AIRFLOW-2655] Fix inconsistency of default config of kubernetes worker Closes #3529 from mrkm4ntr/airflow-2655 --- airflow/contrib/kubernetes/worker_configuration.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index 86ee270abc1b2..50f7374e78c22 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -19,6 +19,7 @@ import os import six +from airflow.configuration import conf from airflow.contrib.kubernetes.pod import Pod, Resources from airflow.contrib.kubernetes.secret import Secret from airflow.utils.log.logging_mixin import LoggingMixin @@ -84,7 +85,8 @@ def _get_environment(self): """Defines any necessary environment variables for the pod executor""" env = { 'AIRFLOW__CORE__DAGS_FOLDER': '/tmp/dags', - 'AIRFLOW__CORE__EXECUTOR': 'LocalExecutor' + 'AIRFLOW__CORE__EXECUTOR': 'LocalExecutor', + 'AIRFLOW__CORE__SQL_ALCHEMY_CONN': conf.get('core', 'SQL_ALCHEMY_CONN') } if self.kube_config.airflow_configmap: env['AIRFLOW__CORE__AIRFLOW_HOME'] = self.worker_airflow_home From f2d77789aa898366eec54bb18b616dd78f5c04b7 Mon Sep 17 00:00:00 2001 From: Aldo Giambelluca Date: Mon, 6 Aug 2018 21:44:48 +0100 Subject: [PATCH 313/808] [AIRFLOW-2755] Added `kubernetes.worker_dags_folder` configuration (#3612) It was previously hardcoded to `/tmp/dags`. This causes problems with python import of modules in the DAGs folder. --- airflow/config_templates/default_airflow.cfg | 1 + airflow/contrib/executors/kubernetes_executor.py | 2 ++ .../contrib/kubernetes/worker_configuration.py | 8 +++++--- scripts/ci/kubernetes/kube/configmaps.yaml | 1 + .../executors/test_kubernetes_executor.py | 16 ++++++++++++++++ 5 files changed, 25 insertions(+), 3 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index b5182434ccff8..dd43b980aaa6c 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -572,6 +572,7 @@ elasticsearch_end_of_log_mark = end_of_log worker_container_repository = worker_container_tag = worker_container_image_pull_policy = IfNotPresent +worker_dags_folder = # If True (default), worker pods will be deleted upon termination delete_worker_pods = True diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 5104d150f0065..9769dc3c90332 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -115,6 +115,8 @@ def __init__(self): self.kubernetes_section, 'worker_container_repository') self.worker_container_tag = configuration.get( self.kubernetes_section, 'worker_container_tag') + self.worker_dags_folder = configuration.get( + self.kubernetes_section, 'worker_dags_folder') self.kube_image = '{}:{}'.format( self.worker_container_repository, self.worker_container_tag) self.kube_image_pull_policy = configuration.get( diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index 50f7374e78c22..18239c5cd2a24 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -84,12 +84,14 @@ def _get_init_containers(self, volume_mounts): def _get_environment(self): """Defines any necessary environment variables for the pod executor""" env = { - 'AIRFLOW__CORE__DAGS_FOLDER': '/tmp/dags', - 'AIRFLOW__CORE__EXECUTOR': 'LocalExecutor', - 'AIRFLOW__CORE__SQL_ALCHEMY_CONN': conf.get('core', 'SQL_ALCHEMY_CONN') + "AIRFLOW__CORE__EXECUTOR": "LocalExecutor", + "AIRFLOW__CORE__SQL_ALCHEMY_CONN": conf.get("core", "SQL_ALCHEMY_CONN"), } + if self.kube_config.airflow_configmap: env['AIRFLOW__CORE__AIRFLOW_HOME'] = self.worker_airflow_home + if self.kube_config.worker_dags_folder: + env['AIRFLOW__CORE__DAGS_FOLDER'] = self.kube_config.worker_dags_folder return env def _get_secrets(self): diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index 5796599683eb5..4c86bcaa6915f 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -176,6 +176,7 @@ data: worker_container_repository = airflow worker_container_tag = latest worker_container_image_pull_policy = IfNotPresent + worker_dags_folder = /tmp/dags delete_worker_pods = True git_repo = https://github.com/apache/incubator-airflow.git git_branch = master diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index 796a8e9ad1c1e..c203e18d5cf8e 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -135,6 +135,22 @@ def test_worker_with_subpaths(self): "subPath should've been passed to volumeMount configuration" ) + def test_worker_environment_no_dags_folder(self): + self.kube_config.worker_dags_folder = '' + worker_config = WorkerConfiguration(self.kube_config) + env = worker_config._get_environment() + + self.assertNotIn('AIRFLOW__CORE__DAGS_FOLDER', env) + + def test_worker_environment_when_dags_folder_specified(self): + dags_folder = '/workers/path/to/dags' + self.kube_config.worker_dags_folder = dags_folder + + worker_config = WorkerConfiguration(self.kube_config) + env = worker_config._get_environment() + + self.assertEqual(dags_folder, env['AIRFLOW__CORE__DAGS_FOLDER']) + if __name__ == '__main__': unittest.main() From b15fecf1e18c9ad413fb16b4dbbde58bf4301769 Mon Sep 17 00:00:00 2001 From: John Cheng Date: Sun, 19 Aug 2018 22:07:53 +0800 Subject: [PATCH 314/808] [AIRFLOW-2917] Set AIRFLOW__CORE__SQL_ALCHEMY_CONN only when needed (#3766) Only when `airflow_configmap` is not provided and `AIRFLOW__CORE__SQL_ALCHEMY_CONN` not in secrets, it is set as an env var. --- airflow/contrib/kubernetes/worker_configuration.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index 18239c5cd2a24..dd620ba66ecc4 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -85,13 +85,15 @@ def _get_environment(self): """Defines any necessary environment variables for the pod executor""" env = { "AIRFLOW__CORE__EXECUTOR": "LocalExecutor", - "AIRFLOW__CORE__SQL_ALCHEMY_CONN": conf.get("core", "SQL_ALCHEMY_CONN"), } if self.kube_config.airflow_configmap: env['AIRFLOW__CORE__AIRFLOW_HOME'] = self.worker_airflow_home if self.kube_config.worker_dags_folder: env['AIRFLOW__CORE__DAGS_FOLDER'] = self.kube_config.worker_dags_folder + if (not self.kube_config.airflow_configmap and + 'AIRFLOW__CORE__SQL_ALCHEMY_CONN' not in self.kube_config.kube_secrets): + env['AIRFLOW__CORE__SQL_ALCHEMY_CONN'] = conf.get("core", "SQL_ALCHEMY_CONN") return env def _get_secrets(self): From 27f48702ecb224b35c4c591a1e331b7899c3ecf2 Mon Sep 17 00:00:00 2001 From: John Cheng Date: Tue, 6 Nov 2018 00:28:01 +0800 Subject: [PATCH 315/808] [AIRFLOW-3022] Add volume mount to KubernetesExecutorConfig (#3855) Added volumes and volume_mounts to the KubernetesExecutorConfig so `volumes` or `secrets` can be mount to worker pod. --- ... => example_kubernetes_executor_config.py} | 34 ++++++++++++++++++- .../contrib/executors/kubernetes_executor.py | 13 +++++-- .../kubernetes/worker_configuration.py | 2 ++ .../minikube/test_kubernetes_executor.py | 4 +-- 4 files changed, 47 insertions(+), 6 deletions(-) rename airflow/contrib/example_dags/{example_kubernetes_annotation.py => example_kubernetes_executor_config.py} (61%) diff --git a/airflow/contrib/example_dags/example_kubernetes_annotation.py b/airflow/contrib/example_dags/example_kubernetes_executor_config.py similarity index 61% rename from airflow/contrib/example_dags/example_kubernetes_annotation.py rename to airflow/contrib/example_dags/example_kubernetes_executor_config.py index 058baf69901f7..3aa70f862b4d4 100644 --- a/airflow/contrib/example_dags/example_kubernetes_annotation.py +++ b/airflow/contrib/example_dags/example_kubernetes_executor_config.py @@ -20,6 +20,7 @@ import airflow from airflow.operators.python_operator import PythonOperator from airflow.models import DAG +import os args = { 'owner': 'airflow', @@ -27,7 +28,7 @@ } dag = DAG( - dag_id='example_kubernetes_annotation', default_args=args, + dag_id='example_kubernetes_executor_config', default_args=args, schedule_interval=None ) @@ -36,6 +37,14 @@ def print_stuff(): print("annotated!") +def test_volume_mount(): + with open('/foo/volume_mount_test.txt', 'w') as foo: + foo.write('Hello') + + rc = os.system("cat /foo/volume_mount_test.txt") + assert rc == 0 + + # You can use annotations on your kubernetes pods! start_task = PythonOperator( task_id="start_task", python_callable=print_stuff, dag=dag, @@ -45,3 +54,26 @@ def print_stuff(): } } ) + +# You can mount volume or secret to the worker pod +second_task = PythonOperator( + task_id="four_task", python_callable=test_volume_mount, dag=dag, + executor_config={ + "KubernetesExecutor": { + "volumes": [ + { + "name": "test-volume", + "hostPath": {"path": "/tmp/"}, + }, + ], + "volume_mounts": [ + { + "mountPath": "/foo/", + "name": "test-volume", + }, + ] + } + } +) + +start_task.set_downstream(second_task) diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 9769dc3c90332..0798a7b967d22 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -40,7 +40,7 @@ class KubernetesExecutorConfig: def __init__(self, image=None, image_pull_policy=None, request_memory=None, request_cpu=None, limit_memory=None, limit_cpu=None, gcp_service_account_key=None, node_selectors=None, affinity=None, - annotations=None): + annotations=None, volumes=None, volume_mounts=None): self.image = image self.image_pull_policy = image_pull_policy self.request_memory = request_memory @@ -51,15 +51,18 @@ def __init__(self, image=None, image_pull_policy=None, request_memory=None, self.node_selectors = node_selectors self.affinity = affinity self.annotations = annotations + self.volumes = volumes + self.volume_mounts = volume_mounts def __repr__(self): return "{}(image={}, image_pull_policy={}, request_memory={}, request_cpu={}, " \ "limit_memory={}, limit_cpu={}, gcp_service_account_key={}, " \ - "node_selectors={}, affinity={}, annotations={})" \ + "node_selectors={}, affinity={}, annotations={}, volumes={}, " \ + "volume_mounts={})" \ .format(KubernetesExecutorConfig.__name__, self.image, self.image_pull_policy, self.request_memory, self.request_cpu, self.limit_memory, self.limit_cpu, self.gcp_service_account_key, self.node_selectors, - self.affinity, self.annotations) + self.affinity, self.annotations, self.volumes, self.volume_mounts) @staticmethod def from_dict(obj): @@ -83,6 +86,8 @@ def from_dict(obj): node_selectors=namespaced.get('node_selectors', None), affinity=namespaced.get('affinity', None), annotations=namespaced.get('annotations', {}), + volumes=namespaced.get('volumes', []), + volume_mounts=namespaced.get('volume_mounts', []), ) def as_dict(self): @@ -97,6 +102,8 @@ def as_dict(self): 'node_selectors': self.node_selectors, 'affinity': self.affinity, 'annotations': self.annotations, + 'volumes': self.volumes, + 'volume_mounts': self.volume_mounts, } diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index dd620ba66ecc4..9d206e62cfe9a 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -189,6 +189,8 @@ def _construct_volume(name, claim): def make_pod(self, namespace, worker_uuid, pod_id, dag_id, task_id, execution_date, airflow_command, kube_executor_config): volumes, volume_mounts = self.init_volumes_and_mounts() + volumes += kube_executor_config.volumes + volume_mounts += kube_executor_config.volume_mounts worker_init_container_spec = self._get_init_containers( copy.deepcopy(volume_mounts)) resources = Resources( diff --git a/tests/contrib/minikube/test_kubernetes_executor.py b/tests/contrib/minikube/test_kubernetes_executor.py index 45d4124d07973..a8db7e12b7554 100644 --- a/tests/contrib/minikube/test_kubernetes_executor.py +++ b/tests/contrib/minikube/test_kubernetes_executor.py @@ -150,7 +150,7 @@ def start_dag(self, dag_id, host): def test_integration_run_dag(self): host = get_minikube_host() - dag_id = 'example_kubernetes_annotation' + dag_id = 'example_kubernetes_executor_config' result_json = self.start_dag(dag_id=dag_id, host=host) @@ -173,7 +173,7 @@ def test_integration_run_dag(self): def test_integration_run_dag_with_scheduler_failure(self): host = get_minikube_host() - dag_id = 'example_kubernetes_annotation' + dag_id = 'example_kubernetes_executor_config' result_json = self.start_dag(dag_id=dag_id, host=host) From 63bf106e18be127f447ad9a35efa0dbc8756a042 Mon Sep 17 00:00:00 2001 From: Kevin Pullin Date: Sun, 16 Dec 2018 23:05:26 -0800 Subject: [PATCH 316/808] [AIRFLOW-2770] Read `dags_in_image` config value as a boolean (#4319) * Read `dags_in_image` config value as a boolean This PR is a minor fix for #3683 The dags_in_image config value is read as a string. However, the existing code expects this to be a boolean. For example, in worker_configuration.py there is the statement: if not self.kube_config.dags_in_image: Since the value is a non-empty string ('False') and not a boolean, this evaluates to true (since non-empty strings are truthy) and skips the logic to add the dags_volume_claim volume mount. This results in the CI tests failing because the dag volume is missing in the k8s pod definition. This PR reads the dags_in_image using the conf.getboolean to fix this error. Rebased on 457ad83e4eb02b7348e5ce00292ca9bd27032651, before the previous dags_in_image commit was reverted. * Revert "Revert [AIRFLOW-2770] [AIRFLOW-3505] (#4318)" This reverts commit 77c368fd228fe5edfdb3304ed4cb000a50667010. --- airflow/config_templates/default_airflow.cfg | 4 ++ .../contrib/executors/kubernetes_executor.py | 10 +++- .../kubernetes/worker_configuration.py | 47 +++++++++------- scripts/ci/kubernetes/kube/configmaps.yaml | 1 + .../executors/test_kubernetes_executor.py | 56 +++++++++++++++++++ 5 files changed, 95 insertions(+), 23 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index dd43b980aaa6c..384ff9df31f43 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -583,6 +583,10 @@ namespace = default # The name of the Kubernetes ConfigMap Containing the Airflow Configuration (this file) airflow_configmap = +# For docker image already contains DAGs, this is set to `True`, and the worker will search for dags in dags_folder, +# otherwise use git sync or dags volume claim to mount DAGs +dags_in_image = False + # For either git sync or volume mounted DAGs, the worker will look in this subpath for DAGs dags_volume_subpath = diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 0798a7b967d22..dd9cd3ec53b15 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -137,6 +137,10 @@ def __init__(self): self.kubernetes_section, 'worker_service_account_name') self.image_pull_secrets = conf.get(self.kubernetes_section, 'image_pull_secrets') + # NOTE: user can build the dags into the docker image directly, + # this will set to True if so + self.dags_in_image = conf.getboolean(self.kubernetes_section, 'dags_in_image') + # NOTE: `git_repo` and `git_branch` must be specified together as a pair # The http URL of the git repository to clone from self.git_repo = conf.get(self.kubernetes_section, 'git_repo') @@ -204,10 +208,12 @@ def __init__(self): self._validate() def _validate(self): - if not self.dags_volume_claim and (not self.git_repo or not self.git_branch): + if not self.dags_volume_claim and not self.dags_in_image \ + and (not self.git_repo or not self.git_branch): raise AirflowConfigException( 'In kubernetes mode the following must be set in the `kubernetes` ' - 'config section: `dags_volume_claim` or `git_repo and git_branch`') + 'config section: `dags_volume_claim` or `git_repo and git_branch` ' + 'or `dags_in_image`') class KubernetesJobWatcher(multiprocessing.Process, LoggingMixin, object): diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index 9d206e62cfe9a..d83be81c1f704 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -38,7 +38,7 @@ def __init__(self, kube_config): def _get_init_containers(self, volume_mounts): """When using git to retrieve the DAGs, use the GitSync Init Container""" # If we're using volume claims to mount the dags, no init container is needed - if self.kube_config.dags_volume_claim: + if self.kube_config.dags_volume_claim or self.kube_config.dags_in_image: return [] # Otherwise, define a git-sync init container @@ -128,32 +128,19 @@ def _construct_volume(name, claim): return volume volumes = [ - _construct_volume( - dags_volume_name, - self.kube_config.dags_volume_claim - ), _construct_volume( logs_volume_name, self.kube_config.logs_volume_claim ) ] - dag_volume_mount_path = "" - - if self.kube_config.dags_volume_claim: - dag_volume_mount_path = self.worker_airflow_dags - else: - dag_volume_mount_path = os.path.join( - self.worker_airflow_dags, - self.kube_config.git_subpath + if not self.kube_config.dags_in_image: + volumes.append( + _construct_volume( + dags_volume_name, + self.kube_config.dags_volume_claim + ) ) - dags_volume_mount = { - 'name': dags_volume_name, - 'mountPath': dag_volume_mount_path, - 'readOnly': True, - } - if self.kube_config.dags_volume_subpath: - dags_volume_mount['subPath'] = self.kube_config.dags_volume_subpath logs_volume_mount = { 'name': logs_volume_name, @@ -163,10 +150,28 @@ def _construct_volume(name, claim): logs_volume_mount['subPath'] = self.kube_config.logs_volume_subpath volume_mounts = [ - dags_volume_mount, logs_volume_mount ] + if not self.kube_config.dags_in_image: + dag_volume_mount_path = "" + + if self.kube_config.dags_volume_claim: + dag_volume_mount_path = self.worker_airflow_dags + else: + dag_volume_mount_path = os.path.join( + self.worker_airflow_dags, + self.kube_config.git_subpath + ) + dags_volume_mount = { + 'name': dags_volume_name, + 'mountPath': dag_volume_mount_path, + 'readOnly': True, + } + if self.kube_config.dags_volume_subpath: + dags_volume_mount['subPath'] = self.kube_config.dags_volume_subpath + volume_mounts.append(dags_volume_mount) + # Mount the airflow.cfg file via a configmap the user has specified if self.kube_config.airflow_configmap: config_volume_name = 'airflow-config' diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index 4c86bcaa6915f..d10cbdc728a9f 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -178,6 +178,7 @@ data: worker_container_image_pull_policy = IfNotPresent worker_dags_folder = /tmp/dags delete_worker_pods = True + dags_in_image = False git_repo = https://github.com/apache/incubator-airflow.git git_branch = master git_subpath = airflow/example_dags/ diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index c203e18d5cf8e..f93a9d81e142d 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -105,6 +105,7 @@ def setUp(self): self.kube_config.airflow_dags = 'logs' self.kube_config.dags_volume_subpath = None self.kube_config.logs_volume_subpath = None + self.kube_config.dags_in_image = False def test_worker_configuration_no_subpaths(self): worker_config = WorkerConfiguration(self.kube_config) @@ -151,6 +152,61 @@ def test_worker_environment_when_dags_folder_specified(self): self.assertEqual(dags_folder, env['AIRFLOW__CORE__DAGS_FOLDER']) + def test_worker_pvc_dags(self): + # Tests persistence volume config created when `dags_volume_claim` is set + self.kube_config.dags_volume_claim = 'airflow-dags' + + worker_config = WorkerConfiguration(self.kube_config) + volumes, volume_mounts = worker_config.init_volumes_and_mounts() + + dag_volume = [volume for volume in volumes if volume['name'] == 'airflow-dags'] + dag_volume_mount = [mount for mount in volume_mounts if mount['name'] == 'airflow-dags'] + + self.assertEqual('airflow-dags', dag_volume[0]['persistentVolumeClaim']['claimName']) + self.assertEqual(1, len(dag_volume_mount)) + + def test_worker_git_dags(self): + # Tests persistence volume config created when `git_repo` is set + self.kube_config.dags_volume_claim = None + self.kube_config.dags_folder = '/usr/local/airflow/dags' + self.kube_config.worker_dags_folder = '/usr/local/airflow/dags' + + self.kube_config.git_sync_container_repository = 'gcr.io/google-containers/git-sync-amd64' + self.kube_config.git_sync_container_tag = 'v2.0.5' + self.kube_config.git_sync_container = 'gcr.io/google-containers/git-sync-amd64:v2.0.5' + self.kube_config.git_sync_init_container_name = 'git-sync-clone' + self.kube_config.git_subpath = '' + + worker_config = WorkerConfiguration(self.kube_config) + volumes, volume_mounts = worker_config.init_volumes_and_mounts() + + init_container = worker_config._get_init_containers(volume_mounts)[0] + + dag_volume = [volume for volume in volumes if volume['name'] == 'airflow-dags'] + dag_volume_mount = [mount for mount in volume_mounts if mount['name'] == 'airflow-dags'] + + self.assertTrue('emptyDir' in dag_volume[0]) + self.assertEqual('/usr/local/airflow/dags/', dag_volume_mount[0]['mountPath']) + + self.assertEqual('git-sync-clone', init_container['name']) + self.assertEqual('gcr.io/google-containers/git-sync-amd64:v2.0.5', init_container['image']) + + def test_worker_container_dags(self): + # Tests that the 'airflow-dags' persistence volume is NOT created when `dags_in_image` is set + self.kube_config.dags_in_image = True + + worker_config = WorkerConfiguration(self.kube_config) + volumes, volume_mounts = worker_config.init_volumes_and_mounts() + + dag_volume = [volume for volume in volumes if volume['name'] == 'airflow-dags'] + dag_volume_mount = [mount for mount in volume_mounts if mount['name'] == 'airflow-dags'] + + init_containers = worker_config._get_init_containers(volume_mounts) + + self.assertEqual(0, len(dag_volume)) + self.assertEqual(0, len(dag_volume_mount)) + self.assertEqual(0, len(init_containers)) + if __name__ == '__main__': unittest.main() From 6c5066493515bd777b5ad8ffbc81c9bd44ef9a0e Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 31 Dec 2018 12:50:01 +0000 Subject: [PATCH 317/808] Fix Flake8 issues --- .../operators/test_bigquery_operator.py | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 0e52beea02657..9e4d315945ea2 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -42,18 +42,6 @@ TEST_SOURCE_FORMAT = 'CSV' -class BigQueryOperatorTest(unittest.TestCase): - def test_bql_deprecation_warning(self): - with warnings.catch_warnings(record=True) as w: - BigQueryOperator( - task_id='test_deprecation_warning_for_bql', - bql='select * from test_table' - ) - self.assertIn( - 'Deprecated parameter `bql`', - w[0].message.args[0]) - - class BigQueryCreateEmptyTableOperatorTest(unittest.TestCase): @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') @@ -160,6 +148,16 @@ def test_execute(self, mock_hook): class BigQueryOperatorTest(unittest.TestCase): + def test_bql_deprecation_warning(self): + with warnings.catch_warnings(record=True) as w: + BigQueryOperator( + task_id='test_deprecation_warning_for_bql', + bql='select * from test_table' + ) + self.assertIn( + 'Deprecated parameter `bql`', + w[0].message.args[0]) + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') def test_execute(self, mock_hook): operator = BigQueryOperator( From f64a4c8e68099c3f844848c227c0e91b1d283816 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Thu, 3 Jan 2019 09:35:35 +0000 Subject: [PATCH 318/808] [AIRFLOW-3560] Add DayOfWeek Sensor (#4363) * [AIRFLOW-3560] Add WeekEnd & DayOfWeek Sensors * Change to using Enum * Fix Docstring * Refactor into a Single Sensor --- airflow/contrib/sensors/weekday_sensor.py | 103 ++++++++++++ airflow/contrib/utils/weekday.py | 51 ++++++ docs/code.rst | 1 + tests/contrib/sensors/test_weekday_sensor.py | 163 +++++++++++++++++++ tests/contrib/utils/test_weekday.py | 42 +++++ 5 files changed, 360 insertions(+) create mode 100644 airflow/contrib/sensors/weekday_sensor.py create mode 100644 airflow/contrib/utils/weekday.py create mode 100644 tests/contrib/sensors/test_weekday_sensor.py create mode 100644 tests/contrib/utils/test_weekday.py diff --git a/airflow/contrib/sensors/weekday_sensor.py b/airflow/contrib/sensors/weekday_sensor.py new file mode 100644 index 0000000000000..4cff263f38325 --- /dev/null +++ b/airflow/contrib/sensors/weekday_sensor.py @@ -0,0 +1,103 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import six +from airflow.contrib.utils.weekday import WeekDay +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.utils import timezone +from airflow.utils.decorators import apply_defaults + + +class DayOfWeekSensor(BaseSensorOperator): + """ + Waits until the first specified day of the week. For example, if the execution + day of the task is '2018-12-22' (Saturday) and you pass 'FRIDAY', the task will wait + until next Friday. + + **Example** (with single day): :: + + weekend_check = DayOfWeekSensor( + task_id='weekend_check', + week_day='Saturday', + use_task_execution_day=True, + dag=dag) + + **Example** (with multiple day using set): :: + + weekend_check = DayOfWeekSensor( + task_id='weekend_check', + week_day={'Saturday', 'Sunday'}, + use_task_execution_day=True, + dag=dag) + + **Example** (with :class:`~airflow.contrib.utils.weekday.WeekDay` enum): :: + + # import WeekDay Enum + from airflow.contrib.utils.weekday import WeekDay + + weekend_check = DayOfWeekSensor( + task_id='weekend_check', + week_day={WeekDay.Saturday, WeekDay.Sunday}, + use_task_execution_day=True, + dag=dag) + + :param week_day: Day of the week to check (full name). Optionally, a set + of days can also be provided using a set. + Example values: + * ``"MONDAY"``, + * ``{"Saturday", "Sunday"}`` + * ``{WeekDay.TUESDAY}`` + * ``{WeekDay.Saturday, WeekDay.Sunday}`` + :type week_day: set or str or WeekDay + :param use_task_execution_day: If ``True``, uses task's execution day to compare + with week_day. Execution Date is Useful for backfilling. + If ``False``, uses system's day of the week. Useful when you + don't want to run anything on weekdays on the system. + :type use_task_execution_day: bool + """ + + @apply_defaults + def __init__(self, week_day, + use_task_execution_day=False, + *args, **kwargs): + super(DayOfWeekSensor, self).__init__(*args, **kwargs) + self.week_day = week_day + self.use_task_execution_day = use_task_execution_day + if isinstance(self.week_day, six.string_types): + self._week_day_num = {WeekDay.get_weekday_number(week_day_str=self.week_day)} + elif isinstance(self.week_day, WeekDay): + self._week_day_num = {self.week_day} + elif isinstance(self.week_day, set): + if all(isinstance(day, six.string_types) for day in self.week_day): + self._week_day_num = {WeekDay.get_weekday_number(day) for day in week_day} + elif all(isinstance(day, WeekDay) for day in self.week_day): + self._week_day_num = self.week_day + else: + raise TypeError( + 'Unsupported Type for week_day parameter: {}. It should be one of str' + ', set or Weekday enum type'.format(type(week_day))) + + def poke(self, context): + self.log.info('Poking until weekday is in %s, Today is %s', + self.week_day, + WeekDay(timezone.utcnow().isoweekday()).name) + if self.use_task_execution_day: + return context['execution_date'].isoweekday() in self._week_day_num + else: + return timezone.utcnow().isoweekday() in self._week_day_num diff --git a/airflow/contrib/utils/weekday.py b/airflow/contrib/utils/weekday.py new file mode 100644 index 0000000000000..1fe4ad3a59011 --- /dev/null +++ b/airflow/contrib/utils/weekday.py @@ -0,0 +1,51 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +import enum + + +@enum.unique +class WeekDay(enum.IntEnum): + """ + Python Enum containing Days of the Week + """ + MONDAY = 1 + TUESDAY = 2 + WEDNESDAY = 3 + THURSDAY = 4 + FRIDAY = 5 + SATURDAY = 6 + SUNDAY = 7 + + @classmethod + def get_weekday_number(cls, week_day_str): + """ + Return the ISO Week Day Number for a Week Day + + :param week_day_str: Full Name of the Week Day. Example: "Sunday" + :type week_day_str: str + :return: ISO Week Day Number corresponding to the provided Weedkay + """ + sanitized_week_day_str = week_day_str.upper() + + if sanitized_week_day_str not in cls.__members__: + raise AttributeError( + 'Invalid Week Day passed: "{}"'.format(week_day_str) + ) + + return cls[sanitized_week_day_str] diff --git a/docs/code.rst b/docs/code.rst index aaa4eec5f7213..6f82166b10ff9 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -260,6 +260,7 @@ Sensors .. autoclass:: airflow.contrib.sensors.sagemaker_tuning_sensor.SageMakerTuningSensor .. autoclass:: airflow.contrib.sensors.sftp_sensor.SFTPSensor .. autoclass:: airflow.contrib.sensors.wasb_sensor.WasbBlobSensor +.. autoclass:: airflow.contrib.sensors.weekday_sensor.DayOfWeekSensor .. _macros: diff --git a/tests/contrib/sensors/test_weekday_sensor.py b/tests/contrib/sensors/test_weekday_sensor.py new file mode 100644 index 0000000000000..55a4c4da4bf7d --- /dev/null +++ b/tests/contrib/sensors/test_weekday_sensor.py @@ -0,0 +1,163 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +import unittest +from airflow import DAG, configuration, models +from airflow.contrib.sensors.weekday_sensor import DayOfWeekSensor +from airflow.contrib.utils.weekday import WeekDay +from airflow.exceptions import AirflowSensorTimeout +from airflow.models import DagBag +from airflow.settings import Session +from airflow.utils.timezone import datetime + +DEFAULT_DATE = datetime(2018, 12, 10) +WEEKDAY_DATE = datetime(2018, 12, 20) +WEEKEND_DATE = datetime(2018, 12, 22) +TEST_DAG_ID = 'weekday_sensor_dag' +DEV_NULL = '/dev/null' + + +class DayOfWeekSensorTests(unittest.TestCase): + + def setUp(self): + configuration.load_test_config() + self.dagbag = DagBag( + dag_folder=DEV_NULL, + include_examples=True + ) + self.args = { + 'owner': 'airflow', + 'start_date': DEFAULT_DATE + } + dag = DAG(TEST_DAG_ID, default_args=self.args) + self.dag = dag + + def tearDown(self): + session = Session() + session.query(models.TaskInstance).filter_by( + dag_id=TEST_DAG_ID).delete() + session.query(models.TaskFail).filter_by( + dag_id=TEST_DAG_ID).delete() + session.commit() + session.close() + + def test_weekday_sensor_true(self): + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day='Thursday', + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + + def test_weekday_sensor_false(self): + t = DayOfWeekSensor( + task_id='weekday_sensor_check_false', + poke_interval=1, + timeout=2, + week_day='Tuesday', + use_task_execution_day=True, + dag=self.dag) + with self.assertRaises(AirflowSensorTimeout): + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + + def test_invalid_weekday_number(self): + invalid_week_day = 'Thsday' + with self.assertRaisesRegexp(AttributeError, + 'Invalid Week Day passed: "{}"'.format( + invalid_week_day)): + DayOfWeekSensor( + task_id='weekday_sensor_invalid_weekday_num', + week_day=invalid_week_day, + use_task_execution_day=True, + dag=self.dag) + + def test_weekday_sensor_with_enum(self): + week_day = WeekDay.THURSDAY + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=WeekDay.THURSDAY, + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + self.assertEqual(t.week_day, week_day) + + def test_weekday_sensor_with_enum_set(self): + week_day = {WeekDay.THURSDAY} + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=week_day, + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + self.assertEqual(t.week_day, week_day) + + def test_weekday_sensor_with_enum_set_2_items(self): + week_day = {WeekDay.THURSDAY, WeekDay.FRIDAY} + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=week_day, + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + self.assertEqual(t.week_day, week_day) + + def test_weekday_sensor_with_string_set(self): + week_day = {'Thursday'} + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=week_day, + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + self.assertEqual(t.week_day, week_day) + + def test_weekday_sensor_with_string_set_2_items(self): + week_day = {'Thursday', 'Friday'} + t = DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=week_day, + use_task_execution_day=True, + dag=self.dag) + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) + self.assertEqual(t.week_day, week_day) + + def test_weekday_sensor_with_invalid_type(self): + invalid_week_day = ['Thsday'] + with self.assertRaisesRegexp(TypeError, + 'Unsupported Type for week_day parameter:' + ' {}. It should be one of str, set or ' + 'Weekday enum type'.format(type(invalid_week_day)) + ): + DayOfWeekSensor( + task_id='weekday_sensor_check_true', + week_day=invalid_week_day, + use_task_execution_day=True, + dag=self.dag) + + def test_weekday_sensor_timeout_with_set(self): + t = DayOfWeekSensor( + task_id='weekday_sensor_check_false', + poke_interval=1, + timeout=2, + week_day={WeekDay.MONDAY, WeekDay.TUESDAY}, + use_task_execution_day=True, + dag=self.dag) + with self.assertRaises(AirflowSensorTimeout): + t.run(start_date=WEEKDAY_DATE, end_date=WEEKDAY_DATE, ignore_ti_state=True) diff --git a/tests/contrib/utils/test_weekday.py b/tests/contrib/utils/test_weekday.py new file mode 100644 index 0000000000000..961652a4ea8e6 --- /dev/null +++ b/tests/contrib/utils/test_weekday.py @@ -0,0 +1,42 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +import unittest + +from enum import Enum + +from airflow.contrib.utils.weekday import WeekDay + + +class WeekDayTest(unittest.TestCase): + def test_weekday_enum_length(self): + self.assertEqual(len(WeekDay), 7) + + def test_weekday_name_value(self): + weekdays = "MONDAY TUESDAY WEDNESDAY THURSDAY FRIDAY SATURDAY SUNDAY" + weekdays = weekdays.split() + for i, weekday in enumerate(weekdays, start=1): + e = WeekDay(i) + self.assertEqual(e, i) + self.assertEqual(int(e), i) + self.assertEqual(e.name, weekday) + self.assertTrue(e in WeekDay) + self.assertTrue(0 < e < 8) + self.assertTrue(type(e) is WeekDay) + self.assertTrue(isinstance(e, int)) + self.assertTrue(isinstance(e, Enum)) From 0eaed4bd0f796cb082c620ec42be931b8419d20d Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Thu, 3 Jan 2019 10:51:46 +0000 Subject: [PATCH 319/808] [AIRFLOW-XXX] Fix WeekDay Sensor Example (#4431) --- airflow/contrib/sensors/weekday_sensor.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/sensors/weekday_sensor.py b/airflow/contrib/sensors/weekday_sensor.py index 4cff263f38325..1a7a702aef97c 100644 --- a/airflow/contrib/sensors/weekday_sensor.py +++ b/airflow/contrib/sensors/weekday_sensor.py @@ -53,7 +53,7 @@ class DayOfWeekSensor(BaseSensorOperator): weekend_check = DayOfWeekSensor( task_id='weekend_check', - week_day={WeekDay.Saturday, WeekDay.Sunday}, + week_day={WeekDay.SATURDAY, WeekDay.SUNDAY}, use_task_execution_day=True, dag=dag) @@ -63,7 +63,7 @@ class DayOfWeekSensor(BaseSensorOperator): * ``"MONDAY"``, * ``{"Saturday", "Sunday"}`` * ``{WeekDay.TUESDAY}`` - * ``{WeekDay.Saturday, WeekDay.Sunday}`` + * ``{WeekDay.SATURDAY, WeekDay.SUNDAY}`` :type week_day: set or str or WeekDay :param use_task_execution_day: If ``True``, uses task's execution day to compare with week_day. Execution Date is Useful for backfilling. From 15d9f2e2d358a46de6451f5c4afb734a3f9aeceb Mon Sep 17 00:00:00 2001 From: Steve Jacobs Date: Wed, 2 Jan 2019 00:57:08 -0700 Subject: [PATCH 320/808] [AIRFLOW-1921] Add support for https and user auth (#2879) --- airflow/hooks/presto_hook.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/airflow/hooks/presto_hook.py b/airflow/hooks/presto_hook.py index c7ebd8f5dcb70..20c918c1a4f49 100644 --- a/airflow/hooks/presto_hook.py +++ b/airflow/hooks/presto_hook.py @@ -21,6 +21,7 @@ from pyhive import presto from pyhive.exc import DatabaseError +from requests.auth import HTTPBasicAuth from airflow.hooks.dbapi_hook import DbApiHook @@ -45,11 +46,17 @@ class PrestoHook(DbApiHook): def get_conn(self): """Returns a connection object""" db = self.get_connection(self.presto_conn_id) + reqkwargs = None + if db.password is not None: + reqkwargs = {'auth': HTTPBasicAuth(db.login, db.password)} return presto.connect( host=db.host, port=db.port, username=db.login, + source=db.extra_dejson.get('source', 'airflow'), + protocol=db.extra_dejson.get('protocol', 'http'), catalog=db.extra_dejson.get('catalog', 'hive'), + requests_kwargs=reqkwargs, schema=db.schema) @staticmethod From 9d0c9c385e0a1a1f66fa1e0481048bd590e91b8e Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Thu, 3 Jan 2019 20:50:09 +0000 Subject: [PATCH 321/808] [AIRFLOW-XXX] Add license to Contrib Example DAG Init file --- airflow/contrib/example_dags/__init__.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/airflow/contrib/example_dags/__init__.py b/airflow/contrib/example_dags/__init__.py index e69de29bb2d1d..114d189da14ab 100644 --- a/airflow/contrib/example_dags/__init__.py +++ b/airflow/contrib/example_dags/__init__.py @@ -0,0 +1,18 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. From 36a62da96950569fdca7e3c2f5006eb318ad14e2 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Fri, 28 Sep 2018 11:51:04 +0100 Subject: [PATCH 322/808] [AIRFLOW-XXX] Fix Kubernetes CI (#3957) --- scripts/ci/kubernetes/docker/airflow-test-env-init.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/scripts/ci/kubernetes/docker/airflow-test-env-init.sh b/scripts/ci/kubernetes/docker/airflow-test-env-init.sh index aa86da72dfa70..59d830af63c59 100755 --- a/scripts/ci/kubernetes/docker/airflow-test-env-init.sh +++ b/scripts/ci/kubernetes/docker/airflow-test-env-init.sh @@ -19,6 +19,7 @@ cd /usr/local/lib/python2.7/dist-packages/airflow && \ cp -R example_dags/* /root/airflow/dags/ && \ +cp -R contrib/example_dags/example_kubernetes_*.py /root/airflow/dags/ && \ airflow initdb && \ alembic upgrade heads && \ (airflow create_user -u airflow -l airflow -f jon -e airflow@apache.org -r Admin -p airflow || true) && \ From def7a9fc136b9f567e3c8ca436daab6e6430cb6d Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 2 Jan 2019 12:36:23 +0000 Subject: [PATCH 323/808] [AIRFLOW-3578] Fix Type Error for BigQueryOperator (#4384) --- airflow/contrib/hooks/bigquery_hook.py | 5 +-- .../operators/test_bigquery_operator.py | 32 +++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py index 05db8f4e987f3..9eb940c905fdc 100644 --- a/airflow/contrib/hooks/bigquery_hook.py +++ b/airflow/contrib/hooks/bigquery_hook.py @@ -23,6 +23,7 @@ """ import time +import six from builtins import range from copy import deepcopy from six import iteritems @@ -671,8 +672,8 @@ def run_query(self, cluster_fields = {'fields': cluster_fields} query_param_list = [ - (sql, 'query', None, str), - (priority, 'priority', 'INTERACTIVE', str), + (sql, 'query', None, six.string_types), + (priority, 'priority', 'INTERACTIVE', six.string_types), (use_legacy_sql, 'useLegacySql', self.use_legacy_sql, bool), (query_params, 'queryParameters', None, dict), (udf_config, 'userDefinedFunctionResources', None, list), diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 9e4d315945ea2..44ed2e3bcf7e0 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -19,11 +19,18 @@ import unittest import warnings +from datetime import datetime + +import six + +from airflow import configuration, models +from airflow.models import TaskInstance, DAG from airflow.contrib.operators.bigquery_operator import \ BigQueryCreateExternalTableOperator, BigQueryCreateEmptyTableOperator, \ BigQueryDeleteDatasetOperator, BigQueryCreateEmptyDatasetOperator, \ BigQueryOperator +from airflow.settings import Session try: from unittest import mock @@ -40,6 +47,8 @@ TEST_GCS_BUCKET = 'test-bucket' TEST_GCS_DATA = ['dir1/*.csv'] TEST_SOURCE_FORMAT = 'CSV' +DEFAULT_DATE = datetime(2015, 1, 1) +TEST_DAG_ID = 'test-bigquery-operators' class BigQueryCreateEmptyTableOperatorTest(unittest.TestCase): @@ -158,6 +167,22 @@ def test_bql_deprecation_warning(self): 'Deprecated parameter `bql`', w[0].message.args[0]) + def setUp(self): + configuration.conf.load_test_config() + self.dagbag = models.DagBag( + dag_folder='/dev/null', include_examples=True) + self.args = {'owner': 'airflow', 'start_date': DEFAULT_DATE} + self.dag = DAG(TEST_DAG_ID, default_args=self.args) + + def tearDown(self): + session = Session() + session.query(models.TaskInstance).filter_by( + dag_id=TEST_DAG_ID).delete() + session.query(models.TaskFail).filter_by( + dag_id=TEST_DAG_ID).delete() + session.commit() + session.close() + @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') def test_execute(self, mock_hook): operator = BigQueryOperator( @@ -208,9 +233,11 @@ def test_execute(self, mock_hook): @mock.patch('airflow.contrib.operators.bigquery_operator.BigQueryHook') def test_bigquery_operator_defaults(self, mock_hook): + operator = BigQueryOperator( task_id=TASK_ID, sql='Select * from test_table', + dag=self.dag, default_args=self.args ) operator.execute(None) @@ -236,3 +263,8 @@ def test_bigquery_operator_defaults(self, mock_hook): api_resource_configs=None, cluster_fields=None, ) + + self.assertTrue(isinstance(operator.sql, six.string_types)) + ti = TaskInstance(task=operator, execution_date=DEFAULT_DATE) + ti.render_templates() + self.assertTrue(isinstance(ti.task.sql, six.string_types)) From f47d41d720ac43ca37bc6bfd57057b9fd120e465 Mon Sep 17 00:00:00 2001 From: Chinh Nguyen Date: Thu, 3 Jan 2019 14:39:24 -0800 Subject: [PATCH 324/808] [AIRFLOW-3583] Fix AirflowException import (#4389) Looks like the class path changed and broke wasb_hook --- airflow/contrib/hooks/wasb_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/contrib/hooks/wasb_hook.py b/airflow/contrib/hooks/wasb_hook.py index 4d4d123b40af6..19ee442f91b1d 100644 --- a/airflow/contrib/hooks/wasb_hook.py +++ b/airflow/contrib/hooks/wasb_hook.py @@ -18,7 +18,7 @@ # under the License. # -from airflow import AirflowException +from airflow.exceptions import AirflowException from airflow.hooks.base_hook import BaseHook from azure.storage.blob import BlockBlobService From 625c7212d124330dcbfe29ad996fa1fda3bd8231 Mon Sep 17 00:00:00 2001 From: Conrad Lee Date: Fri, 4 Jan 2019 00:18:05 +0100 Subject: [PATCH 325/808] [AIRFLOW-3316] For gcs_to_bq: add missing init of schema_fields var (#4430) --- airflow/contrib/operators/gcs_to_bq.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py index 11c2c83bc174b..ee762d6527eac 100644 --- a/airflow/contrib/operators/gcs_to_bq.py +++ b/airflow/contrib/operators/gcs_to_bq.py @@ -208,6 +208,8 @@ def execute(self, context): elif self.schema_object is None and self.autodetect is False: raise ValueError('At least one of `schema_fields`, `schema_object`, ' 'or `autodetect` must be passed.') + else: + schema_fields = None else: schema_fields = self.schema_fields From ba9b1c2dcb77edc5022eb7b5bb560555aba76e0d Mon Sep 17 00:00:00 2001 From: Dariusz Aniszewski Date: Fri, 4 Jan 2019 14:50:15 +0100 Subject: [PATCH 326/808] [AIRFLOW-3446] Add Google Cloud BigTable operators (#4354) --- .../example_gcp_bigtable_operators.py | 149 +++++ airflow/contrib/hooks/gcp_bigtable_hook.py | 231 ++++++++ .../operators/gcp_bigtable_operator.py | 424 ++++++++++++++ docs/howto/operator.rst | 130 +++++ docs/integration.rst | 64 +++ setup.py | 1 + .../operators/test_gcp_bigtable_operator.py | 540 ++++++++++++++++++ 7 files changed, 1539 insertions(+) create mode 100644 airflow/contrib/example_dags/example_gcp_bigtable_operators.py create mode 100644 airflow/contrib/hooks/gcp_bigtable_hook.py create mode 100644 airflow/contrib/operators/gcp_bigtable_operator.py create mode 100644 tests/contrib/operators/test_gcp_bigtable_operator.py diff --git a/airflow/contrib/example_dags/example_gcp_bigtable_operators.py b/airflow/contrib/example_dags/example_gcp_bigtable_operators.py new file mode 100644 index 0000000000000..48c4245cbafbb --- /dev/null +++ b/airflow/contrib/example_dags/example_gcp_bigtable_operators.py @@ -0,0 +1,149 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# 'License'); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# 'AS IS' BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Example Airflow DAG that creates and performs following operations on Cloud Bigtable: +- creates an Instance +- creates a Table +- updates Cluster +- waits for Table replication completeness +- deletes the Table +- deletes the Instance + +This DAG relies on the following environment variables +* GCP_PROJECT_ID - Google Cloud Platform project +* CBT_INSTANCE_ID - desired ID of a Cloud Bigtable instance +* CBT_INSTANCE_DISPLAY_NAME - desired human-readable display name of the Instance +* CBT_INSTANCE_TYPE - type of the Instance, e.g. 1 for DEVELOPMENT + See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance +* CBT_INSTANCE_LABELS - labels to add for the Instance +* CBT_CLUSTER_ID - desired ID of the main Cluster created for the Instance +* CBT_CLUSTER_ZONE - zone in which main Cluster will be created. e.g. europe-west1-b + See available zones: https://cloud.google.com/bigtable/docs/locations +* CBT_CLUSTER_NODES - initial amount of nodes of the Cluster +* CBT_CLUSTER_NODES_UPDATED - amount of nodes for BigtableClusterUpdateOperator +* CBT_CLUSTER_STORAGE_TYPE - storage for the Cluster, e.g. 1 for SSD + See https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.cluster # noqa: E501 +* CBT_TABLE_ID - desired ID of the Table +* CBT_POKE_INTERVAL - number of seconds between every attempt of Sensor check + +""" + +import datetime +import json + +from os import getenv + +import airflow +from airflow import models +from airflow.contrib.operators.gcp_bigtable_operator import BigtableInstanceCreateOperator, \ + BigtableInstanceDeleteOperator, BigtableClusterUpdateOperator, BigtableTableCreateOperator, \ + BigtableTableWaitForReplicationSensor, BigtableTableDeleteOperator + +# [START howto_operator_gcp_bigtable_args] +GCP_PROJECT_ID = getenv('GCP_PROJECT_ID', 'example-project') +CBT_INSTANCE_ID = getenv('CBT_INSTANCE_ID', 'some-instance-id') +CBT_INSTANCE_DISPLAY_NAME = getenv('CBT_INSTANCE_DISPLAY_NAME', 'Human-readable name') +CBT_INSTANCE_TYPE = getenv('CBT_INSTANCE_TYPE', '2') +CBT_INSTANCE_LABELS = getenv('CBT_INSTANCE_LABELS', '{}') +CBT_CLUSTER_ID = getenv('CBT_CLUSTER_ID', 'some-cluster-id') +CBT_CLUSTER_ZONE = getenv('CBT_CLUSTER_ZONE', 'europe-west1-b') +CBT_CLUSTER_NODES = getenv('CBT_CLUSTER_NODES', '3') +CBT_CLUSTER_NODES_UPDATED = getenv('CBT_CLUSTER_NODES_UPDATED', '5') +CBT_CLUSTER_STORAGE_TYPE = getenv('CBT_CLUSTER_STORAGE_TYPE', '2') +CBT_TABLE_ID = getenv('CBT_TABLE_ID', 'some-table-id') +CBT_POKE_INTERVAL = getenv('CBT_POKE_INTERVAL', '60') +# [END howto_operator_gcp_bigtable_args] + +default_args = { + 'start_date': airflow.utils.dates.days_ago(1) +} + +with models.DAG( + 'example_gcp_bigtable_operators', + default_args=default_args, + schedule_interval=datetime.timedelta(days=1) +) as dag: + # [START howto_operator_gcp_bigtable_instance_create] + create_instance_task = BigtableInstanceCreateOperator( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + main_cluster_id=CBT_CLUSTER_ID, + main_cluster_zone=CBT_CLUSTER_ZONE, + instance_display_name=CBT_INSTANCE_DISPLAY_NAME, + instance_type=int(CBT_INSTANCE_TYPE), + instance_labels=json.loads(CBT_INSTANCE_LABELS), + cluster_nodes=int(CBT_CLUSTER_NODES), + cluster_storage_type=int(CBT_CLUSTER_STORAGE_TYPE), + task_id='create_instance', + ) + # [END howto_operator_gcp_bigtable_instance_create] + + # [START howto_operator_gcp_bigtable_cluster_update] + cluster_update_task = BigtableClusterUpdateOperator( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + cluster_id=CBT_CLUSTER_ID, + nodes=int(CBT_CLUSTER_NODES_UPDATED), + task_id='update_cluster', + ) + # [END howto_operator_gcp_bigtable_cluster_update] + + # [START howto_operator_gcp_bigtable_instance_delete] + delete_instance_task = BigtableInstanceDeleteOperator( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + task_id='delete_instance', + ) + # [END howto_operator_gcp_bigtable_instance_delete] + + # [START howto_operator_gcp_bigtable_table_create] + create_table_task = BigtableTableCreateOperator( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + table_id=CBT_TABLE_ID, + task_id='create_table', + ) + # [END howto_operator_gcp_bigtable_table_create] + + # [START howto_operator_gcp_bigtable_table_wait_for_replication] + wait_for_table_replication_task = BigtableTableWaitForReplicationSensor( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + table_id=CBT_TABLE_ID, + poke_interval=int(CBT_POKE_INTERVAL), + task_id='wait_for_table_replication', + ) + # [END howto_operator_gcp_bigtable_table_wait_for_replication] + + # [START howto_operator_gcp_bigtable_table_delete] + delete_table_task = BigtableTableDeleteOperator( + project_id=GCP_PROJECT_ID, + instance_id=CBT_INSTANCE_ID, + table_id=CBT_TABLE_ID, + task_id='delete_table', + ) + # [END howto_operator_gcp_bigtable_table_delete] + + wait_for_table_replication_task >> delete_table_task + create_instance_task \ + >> create_table_task \ + >> cluster_update_task \ + >> delete_table_task \ + >> delete_instance_task diff --git a/airflow/contrib/hooks/gcp_bigtable_hook.py b/airflow/contrib/hooks/gcp_bigtable_hook.py new file mode 100644 index 0000000000000..5d1b6f01c9aa1 --- /dev/null +++ b/airflow/contrib/hooks/gcp_bigtable_hook.py @@ -0,0 +1,231 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from google.cloud.bigtable import Client +from google.cloud.bigtable.cluster import Cluster +from google.cloud.bigtable.instance import Instance +from google.cloud.bigtable.table import Table +from google.cloud.bigtable_admin_v2 import enums +from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook + + +class BigtableHook(GoogleCloudBaseHook): + """ + Hook for Google Cloud Bigtable APIs. + """ + + _client = None + + def __init__(self, + gcp_conn_id='google_cloud_default', + delegate_to=None): + super(BigtableHook, self).__init__(gcp_conn_id, delegate_to) + + def get_client(self, project_id): + if not self._client: + self._client = Client(project=project_id, credentials=self._get_credentials(), admin=True) + return self._client + + def get_instance(self, project_id, instance_id): + """ + Retrieves and returns the specified Cloud Bigtable instance if it exists. + Otherwise, returns None. + + :param project_id: The ID of the GCP project. + :type project_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type instance_id: str + """ + + client = self.get_client(project_id) + + instance = Instance(instance_id, client) + if not instance.exists(): + return None + return instance + + def delete_instance(self, project_id, instance_id): + """ + Deletes the specified Cloud Bigtable instance. + Raises google.api_core.exceptions.NotFound if the Cloud Bigtable instance does not exist. + + :param project_id: The ID of the GCP project. + :type project_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type instance_id: str + """ + instance = Instance(instance_id, self.get_client(project_id)) + instance.delete() + + def create_instance(self, + project_id, + instance_id, + main_cluster_id, + main_cluster_zone, + replica_cluster_id=None, + replica_cluster_zone=None, + instance_display_name=None, + instance_type=enums.Instance.Type.TYPE_UNSPECIFIED, + instance_labels=None, + cluster_nodes=None, + cluster_storage_type=enums.StorageType.STORAGE_TYPE_UNSPECIFIED, + timeout=None): + """ + Creates new instance. + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID for the new instance. + :type main_cluster_id: str + :param main_cluster_id: The ID for main cluster for the new instance. + :type main_cluster_zone: str + :param main_cluster_zone: The zone for main cluster. + See https://cloud.google.com/bigtable/docs/locations for more details. + :type replica_cluster_id: str + :param replica_cluster_id: (optional) The ID for replica cluster for the new instance. + :type replica_cluster_zone: str + :param replica_cluster_zone: (optional) The zone for replica cluster. + :type instance_type: enums.Instance.Type + :param instance_type: (optional) The type of the instance. + :type instance_display_name: str + :param instance_display_name: (optional) Human-readable name of the instance. + Defaults to ``instance_id``. + :type instance_labels: dict + :param instance_labels: (optional) Dictionary of labels to associate with the instance. + :type cluster_nodes: int + :param cluster_nodes: (optional) Number of nodes for cluster. + :type cluster_storage_type: enums.StorageType + :param cluster_storage_type: (optional) The type of storage. + :type timeout: int + :param timeout: (optional) timeout (in seconds) for instance creation. + If None is not specified, Operator will wait indefinitely. + """ + cluster_storage_type = enums.StorageType(cluster_storage_type) + instance_type = enums.Instance.Type(instance_type) + + instance = Instance( + instance_id, + self.get_client(project_id), + instance_display_name, + instance_type, + instance_labels, + ) + + clusters = [ + instance.cluster( + main_cluster_id, + main_cluster_zone, + cluster_nodes, + cluster_storage_type + ) + ] + if replica_cluster_id and replica_cluster_zone: + clusters.append(instance.cluster( + replica_cluster_id, + replica_cluster_zone, + cluster_nodes, + cluster_storage_type + )) + operation = instance.create( + clusters=clusters + ) + operation.result(timeout) + return instance + + # noinspection PyMethodMayBeStatic + def create_table(self, instance, table_id, initial_split_keys, column_families): + """ + Creates the specified Cloud Bigtable table. + Raises google.api_core.exceptions.AlreadyExists if the table exists. + + :type instance: Instance + :param instance: The Cloud Bigtable instance that owns the table. + :type table_id: str + :param table_id: The ID of the table to create in Cloud Bigtable. + :type initial_split_keys: list + :param initial_split_keys: (Optional) A list of row keys in bytes to use to initially split the table. + :type column_families: dict + :param column_families: (Optional) A map of columns to create. The key is the column_id str, and the + value is a GarbageCollectionRule. + """ + table = Table(table_id, instance) + table.create(initial_split_keys, column_families) + + def delete_table(self, project_id, instance_id, table_id): + """ + Deletes the specified table in Cloud Bigtable. + Raises google.api_core.exceptions.NotFound if the table does not exist. + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type table_id: str + :param table_id: The ID of the table in Cloud Bigtable. + """ + instance = Instance(instance_id, self.get_client(project_id)) + table = Table(table_id, instance) + table.delete() + + @staticmethod + def update_cluster(instance, cluster_id, nodes): + """ + Updates number of nodes in the specified Cloud Bigtable cluster. + Raises google.api_core.exceptions.NotFound if the cluster does not exist. + + :type instance: Instance + :param instance: The Cloud Bigtable instance that owns the cluster. + :type cluster_id: str + :param cluster_id: The ID of the cluster. + :type nodes: int + :param nodes: The desired number of nodes. + """ + cluster = Cluster(cluster_id, instance) + cluster.serve_nodes = nodes + cluster.update() + + @staticmethod + def get_column_families_for_table(instance, table_id): + """ + Fetches Column Families for the specified table in Cloud Bigtable. + + :type instance: Instance + :param instance: The Cloud Bigtable instance that owns the table. + :type table_id: str + :param table_id: The ID of the table in Cloud Bigtable to fetch Column Families from. + """ + + table = Table(table_id, instance) + return table.list_column_families() + + @staticmethod + def get_cluster_states_for_table(instance, table_id): + """ + Fetches Cluster States for the specified table in Cloud Bigtable. + Raises google.api_core.exceptions.NotFound if the table does not exist. + + :type instance: Instance + :param instance: The Cloud Bigtable instance that owns the table. + :type table_id: str + :param table_id: The ID of the table in Cloud Bigtable to fetch Cluster States from. + """ + + table = Table(table_id, instance) + return table.get_cluster_states() diff --git a/airflow/contrib/operators/gcp_bigtable_operator.py b/airflow/contrib/operators/gcp_bigtable_operator.py new file mode 100644 index 0000000000000..640851e76e490 --- /dev/null +++ b/airflow/contrib/operators/gcp_bigtable_operator.py @@ -0,0 +1,424 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import google.api_core.exceptions + +from airflow import AirflowException +from airflow.models import BaseOperator +from airflow.sensors.base_sensor_operator import BaseSensorOperator +from airflow.contrib.hooks.gcp_bigtable_hook import BigtableHook +from airflow.utils.decorators import apply_defaults +from google.cloud.bigtable_admin_v2 import enums +from google.cloud.bigtable.table import ClusterState + + +class BigtableValidationMixin(object): + """ + Common class for Cloud Bigtable operators for validating required fields. + """ + + REQUIRED_ATTRIBUTES = [] + + def _validate_inputs(self): + for attr_name in self.REQUIRED_ATTRIBUTES: + if not getattr(self, attr_name): + raise AirflowException('Empty parameter: {}'.format(attr_name)) + + +class BigtableInstanceCreateOperator(BaseOperator, BigtableValidationMixin): + """ + Creates a new Cloud Bigtable instance. + If the Cloud Bigtable instance with the given ID exists, the operator does not compare its configuration + and immediately succeeds. No changes are made to the existing instance. + + For more details about instance creation have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.create + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance to create. + :type main_cluster_id: str + :param main_cluster_id: The ID for main cluster for the new instance. + :type main_cluster_zone: str + :param main_cluster_zone: The zone for main cluster + See https://cloud.google.com/bigtable/docs/locations for more details. + :type replica_cluster_id: str + :param replica_cluster_id: (optional) The ID for replica cluster for the new instance. + :type replica_cluster_zone: str + :param replica_cluster_zone: (optional) The zone for replica cluster. + :type instance_type: IntEnum + :param instance_type: (optional) The type of the instance. + :type instance_display_name: str + :param instance_display_name: (optional) Human-readable name of the instance. Defaults to ``instance_id``. + :type instance_labels: dict + :param instance_labels: (optional) Dictionary of labels to associate with the instance. + :type cluster_nodes: int + :param cluster_nodes: (optional) Number of nodes for cluster. + :type cluster_storage_type: IntEnum + :param cluster_storage_type: (optional) The type of storage. + :type timeout: int + :param timeout: (optional) timeout (in seconds) for instance creation. + If None is not specified, Operator will wait indefinitely. + """ + + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'main_cluster_id', 'main_cluster_zone') + template_fields = ['project_id', 'instance_id', 'main_cluster_id', 'main_cluster_zone'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + main_cluster_id, + main_cluster_zone, + replica_cluster_id=None, + replica_cluster_zone=None, + instance_display_name=None, + instance_type=None, + instance_labels=None, + cluster_nodes=None, + cluster_storage_type=None, + timeout=None, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self.main_cluster_id = main_cluster_id + self.main_cluster_zone = main_cluster_zone + self.replica_cluster_id = replica_cluster_id + self.replica_cluster_zone = replica_cluster_zone + self.instance_display_name = instance_display_name + self.instance_type = instance_type + self.instance_labels = instance_labels + self.cluster_nodes = cluster_nodes + self.cluster_storage_type = cluster_storage_type + self.timeout = timeout + self._validate_inputs() + self.hook = BigtableHook() + super(BigtableInstanceCreateOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + instance = self.hook.get_instance(self.project_id, self.instance_id) + if instance: + # Based on Instance.__eq__ instance with the same ID and client is considered as equal. + self.log.info( + "The instance '%s' already exists in this project. Consider it as created", + self.instance_id + ) + return + try: + self.hook.create_instance( + self.project_id, + self.instance_id, + self.main_cluster_id, + self.main_cluster_zone, + self.replica_cluster_id, + self.replica_cluster_zone, + self.instance_display_name, + self.instance_type, + self.instance_labels, + self.cluster_nodes, + self.cluster_storage_type, + self.timeout, + ) + except google.api_core.exceptions.GoogleAPICallError as e: + self.log.error('An error occurred. Exiting.') + raise e + + +class BigtableInstanceDeleteOperator(BaseOperator, BigtableValidationMixin): + """ + Deletes the Cloud Bigtable instance, including its clusters and all related tables. + + For more details about deleting instance have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/instance.html#google.cloud.bigtable.instance.Instance.delete + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance to delete. + """ + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id') + template_fields = ['project_id', 'instance_id'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self._validate_inputs() + self.hook = BigtableHook() + super(BigtableInstanceDeleteOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + try: + self.hook.delete_instance(self.project_id, self.instance_id) + except google.api_core.exceptions.NotFound: + self.log.info( + "The instance '%s' does not exist in project '%s'. Consider it as deleted", + self.instance_id, self.project_id + ) + except google.api_core.exceptions.GoogleAPICallError as e: + self.log.error('An error occurred. Exiting.') + raise e + + +class BigtableTableCreateOperator(BaseOperator, BigtableValidationMixin): + """ + Creates the table in the Cloud Bigtable instance. + + For more details about creating table have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.create + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance that will hold the new table. + :type table_id: str + :param table_id: The ID of the table to be created. + :type initial_split_keys: list + :param initial_split_keys: (Optional) list of row keys in bytes that will be used to initially split + the table into several tablets. + :type column_families: dict + :param column_families: (Optional) A map columns to create. + The key is the column_id str and the value is a GarbageCollectionRule + """ + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id') + template_fields = ['project_id', 'instance_id', 'table_id'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + table_id, + initial_split_keys=None, + column_families=None, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self.table_id = table_id + self.initial_split_keys = initial_split_keys or list() + self.column_families = column_families or dict() + self._validate_inputs() + self.hook = BigtableHook() + self.instance = None + super(BigtableTableCreateOperator, self).__init__(*args, **kwargs) + + def _compare_column_families(self): + table_column_families = self.hook.get_column_families_for_table(self.instance, self.table_id) + if set(table_column_families.keys()) != set(self.column_families.keys()): + self.log.error("Table '%s' has different set of Column Families", self.table_id) + self.log.error("Expected: %s", self.column_families.keys()) + self.log.error("Actual: %s", table_column_families.keys()) + return False + + for key in table_column_families.keys(): + # There is difference in structure between local Column Families and remote ones + # Local `self.column_families` is dict with column_id as key and GarbageCollectionRule as value. + # Remote `table_column_families` is list of ColumnFamily objects. + # For more information about ColumnFamily please refer to the documentation: + # https://googleapis.github.io/google-cloud-python/latest/bigtable/column-family.html#google.cloud.bigtable.column_family.ColumnFamily + if table_column_families[key].gc_rule != self.column_families[key]: + self.log.error("Column Family '%s' differs for table '%s'.", key, self.table_id) + return False + return True + + def execute(self, context): + self.instance = self.hook.get_instance(self.project_id, self.instance_id) + if not self.instance: + raise AirflowException("Dependency: instance '{}' does not exist in project '{}'.".format( + self.instance_id, self.project_id)) + try: + self.hook.create_table( + self.instance, + self.table_id, + self.initial_split_keys, + self.column_families + ) + except google.api_core.exceptions.AlreadyExists: + if not self._compare_column_families(): + raise AirflowException( + "Table '{}' already exists with different Column Families.".format(self.table_id)) + self.log.info("The table '%s' already exists. Consider it as created", self.table_id) + + +class BigtableTableDeleteOperator(BaseOperator, BigtableValidationMixin): + """ + Deletes the Cloud Bigtable table. + + For more details about deleting table have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.delete + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type table_id: str + :param table_id: The ID of the table to be deleted. + """ + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id') + template_fields = ['project_id', 'instance_id', 'table_id'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + table_id, + app_profile_id=None, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self.table_id = table_id + self.app_profile_id = app_profile_id + self._validate_inputs() + self.hook = BigtableHook() + super(BigtableTableDeleteOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + instance = self.hook.get_instance(self.project_id, self.instance_id) + if not instance: + raise AirflowException("Dependency: instance '{}' does not exist.".format(self.instance_id)) + + try: + self.hook.delete_table( + self.project_id, + self.instance_id, + self.table_id, + ) + except google.api_core.exceptions.NotFound: + # It's OK if table doesn't exists. + self.log.info("The table '%s' no longer exists. Consider it as deleted", self.table_id) + except google.api_core.exceptions.GoogleAPICallError as e: + self.log.error('An error occurred. Exiting.') + raise e + + +class BigtableClusterUpdateOperator(BaseOperator, BigtableValidationMixin): + """ + Updates a Cloud Bigtable cluster. + + For more details about updating a Cloud Bigtable cluster, have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/cluster.html#google.cloud.bigtable.cluster.Cluster.update + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type cluster_id: str + :param cluster_id: The ID of the Cloud Bigtable cluster to update. + :type nodes: int + :param nodes: The desired number of nodes for the Cloud Bigtable cluster. + """ + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'cluster_id', 'nodes') + template_fields = ['project_id', 'instance_id', 'cluster_id', 'nodes'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + cluster_id, + nodes, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self.cluster_id = cluster_id + self.nodes = nodes + self._validate_inputs() + self.hook = BigtableHook() + super(BigtableClusterUpdateOperator, self).__init__(*args, **kwargs) + + def execute(self, context): + instance = self.hook.get_instance(self.project_id, self.instance_id) + if not instance: + raise AirflowException("Dependency: instance '{}' does not exist.".format(self.instance_id)) + + try: + self.hook.update_cluster( + instance, + self.cluster_id, + self.nodes + ) + except google.api_core.exceptions.NotFound: + raise AirflowException("Dependency: cluster '{}' does not exist for instance '{}'.".format( + self.cluster_id, + self.instance_id + )) + except google.api_core.exceptions.GoogleAPICallError as e: + self.log.error('An error occurred. Exiting.') + raise e + + +class BigtableTableWaitForReplicationSensor(BaseSensorOperator, BigtableValidationMixin): + """ + Sensor that waits for Cloud Bigtable table to be fully replicated to its clusters. + No exception will be raised if the instance or the table does not exist. + + For more details about cluster states for a table, have a look at the reference: + https://googleapis.github.io/google-cloud-python/latest/bigtable/table.html#google.cloud.bigtable.table.Table.get_cluster_states + + :type project_id: str + :param project_id: The ID of the GCP project. + :type instance_id: str + :param instance_id: The ID of the Cloud Bigtable instance. + :type table_id: str + :param table_id: The ID of the table to check replication status. + """ + REQUIRED_ATTRIBUTES = ('project_id', 'instance_id', 'table_id') + template_fields = ['project_id', 'instance_id', 'table_id'] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + table_id, + *args, **kwargs): + self.project_id = project_id + self.instance_id = instance_id + self.table_id = table_id + self._validate_inputs() + self.hook = BigtableHook() + super(BigtableTableWaitForReplicationSensor, self).__init__(*args, **kwargs) + + def poke(self, context): + instance = self.hook.get_instance(self.project_id, self.instance_id) + if not instance: + self.log.info("Dependency: instance '%s' does not exist.", self.instance_id) + return False + + try: + cluster_states = self.hook.get_cluster_states_for_table(instance, self.table_id) + except google.api_core.exceptions.NotFound: + self.log.info( + "Dependency: table '%s' does not exist in instance '%s'.", self.table_id, self.instance_id) + return False + + ready_state = ClusterState(enums.Table.ClusterState.ReplicationState.READY) + + is_table_replicated = True + for cluster_id in cluster_states.keys(): + if cluster_states[cluster_id] != ready_state: + self.log.info("Table '%s' is not yet replicated on cluster '%s'.", self.table_id, cluster_id) + is_table_replicated = False + + if not is_table_replicated: + return False + + self.log.info("Table '%s' is replicated.", self.table_id) + return True diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index ced4d80345fc9..e9aade62957cf 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -355,6 +355,135 @@ More information See `Google Compute Engine API documentation `_ +Google Cloud Bigtable Operators +------------------------------- + +Arguments +""""""""" + +All examples below rely on the following variables, which can be passed via environment variables. + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :start-after: [START howto_operator_gcp_bigtable_args] + :end-before: [END howto_operator_gcp_bigtable_args] + + +BigtableInstanceCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceCreateOperator` +to create a Google Cloud Bigtable instance. + +If the Cloud Bigtable instance with the given ID exists, the operator does not compare its configuration +and immediately succeeds. No changes are made to the existing instance. + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_instance_create] + :end-before: [END howto_operator_gcp_bigtable_instance_create] + + +BigtableInstanceDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceDeleteOperator` +to delete a Google Cloud Bigtable instance. + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_instance_delete] + :end-before: [END howto_operator_gcp_bigtable_instance_delete] + +BigtableClusterUpdateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableClusterUpdateOperator` +to modify number of nodes in a Cloud Bigtable cluster. + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_cluster_update] + :end-before: [END howto_operator_gcp_bigtable_cluster_update] + + +BigtableTableCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Creates a table in a Cloud Bigtable instance. + +If the table with given ID exists in the Cloud Bigtable instance, the operator compares the Column Families. +If the Column Families are identical operator succeeds. Otherwise, the operator fails with the appropriate +error message. + + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_table_create] + :end-before: [END howto_operator_gcp_bigtable_table_create] + +Advanced +"""""""" + +When creating a table, you can specify the optional ``initial_split_keys`` and ``column_familes``. +Please refer to the Python Client for Google Cloud Bigtable documentation +`for Table `_ and `for Column +Families `_. + + +BigtableTableDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableTableDeleteOperator` +to delete a table in Google Cloud Bigtable. + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_table_delete] + :end-before: [END howto_operator_gcp_bigtable_table_delete] + +BigtableTableWaitForReplicationSensor +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Use the :class:`~airflow.contrib.operators.gcp_bigtable_operator.BigtableTableWaitForReplicationSensor` +to wait for the table to replicate fully. + +The same arguments apply to this sensor as the BigtableTableCreateOperator_. + +**Note:** If the table or the Cloud Bigtable instance does not exist, this sensor waits for the table until +timeout hits and does not raise any exception. + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_bigtable_operators.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcp_bigtable_table_wait_for_replication] + :end-before: [END howto_operator_gcp_bigtable_table_wait_for_replication] + + + Google Cloud Functions Operators -------------------------------- @@ -1140,3 +1269,4 @@ See `Google Cloud Storage ObjectAccessControls insert documentation `_ for details. + diff --git a/docs/integration.rst b/docs/integration.rst index c463f89a4a023..6a051134d891d 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -1,3 +1,4 @@ + .. Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with this work for additional information @@ -773,6 +774,69 @@ Cloud SQL Hooks :members: +Cloud Bigtable +'''''''''''''' + +Cloud Bigtable Operators +"""""""""""""""""""""""" + +- :ref:`BigtableInstanceCreateOperator` : creates a Cloud Bigtable instance. +- :ref:`BigtableInstanceDeleteOperator` : deletes a Google Cloud Bigtable instance. +- :ref:`BigtableClusterUpdateOperator` : updates the number of nodes in a Google Cloud Bigtable cluster. +- :ref:`BigtableTableCreateOperator` : creates a table in a Google Cloud Bigtable instance. +- :ref:`BigtableTableDeleteOperator` : deletes a table in a Google Cloud Bigtable instance. +- :ref:`BigtableTableWaitForReplicationSensor` : (sensor) waits for a table to be fully replicated. + +.. _BigtableInstanceCreateOperator: + +BigtableInstanceCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceCreateOperator + +.. _BigtableInstanceDeleteOperator: + +BigtableInstanceDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableInstanceDeleteOperator + +.. _BigtableClusterUpdateOperator: + +BigtableClusterUpdateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableClusterUpdateOperator + +.. _BigtableTableCreateOperator: + +BigtableTableCreateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableCreateOperator + +.. _BigtableTableDeleteOperator: + +BigtableTableDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableDeleteOperator + +.. _BigtableTableWaitForReplicationSensor: + +BigtableTableWaitForReplicationSensor +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_bigtable_operator.BigtableTableWaitForReplicationSensor + +.. _BigtableHook: + +Cloud Bigtable Hook +"""""""""""""""""""" + +.. autoclass:: airflow.contrib.hooks.gcp_bigtable_hook.BigtableHook + :members: + Compute Engine '''''''''''''' diff --git a/setup.py b/setup.py index fb136ff69497f..c650567742d01 100644 --- a/setup.py +++ b/setup.py @@ -190,6 +190,7 @@ def write_version(filename=os.path.join(*['airflow', 'google-auth>=1.0.0, <2.0.0dev', 'google-auth-httplib2>=0.0.1', 'google-cloud-container>=0.1.1', + 'google-cloud-bigtable==0.31.0', 'google-cloud-spanner>=1.6.0', 'grpcio-gcp>=0.2.2', 'PyOpenSSL', diff --git a/tests/contrib/operators/test_gcp_bigtable_operator.py b/tests/contrib/operators/test_gcp_bigtable_operator.py new file mode 100644 index 0000000000000..8417efc9977dc --- /dev/null +++ b/tests/contrib/operators/test_gcp_bigtable_operator.py @@ -0,0 +1,540 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +import google.api_core.exceptions +from google.cloud.bigtable.column_family import MaxVersionsGCRule +from google.cloud.bigtable.instance import Instance +from google.cloud.bigtable.table import ClusterState +from parameterized import parameterized + +from airflow import AirflowException +from airflow.contrib.operators.gcp_bigtable_operator import BigtableInstanceDeleteOperator, \ + BigtableTableDeleteOperator, BigtableTableCreateOperator, BigtableTableWaitForReplicationSensor, \ + BigtableClusterUpdateOperator, BigtableInstanceCreateOperator + +try: + # noinspection PyProtectedMember + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + +PROJECT_ID = 'test_project_id' +INSTANCE_ID = 'test-instance-id' +CLUSTER_ID = 'test-cluster-id' +CLUSTER_ZONE = 'us-central1-f' +NODES = 5 +TABLE_ID = 'test-table-id' +INITIAL_SPLIT_KEYS = [] +EMPTY_COLUMN_FAMILIES = {} + + +class BigtableInstanceCreateTest(unittest.TestCase): + @parameterized.expand([ + ('project_id', '', INSTANCE_ID, CLUSTER_ID, CLUSTER_ZONE), + ('instance_id', PROJECT_ID, '', CLUSTER_ID, CLUSTER_ZONE), + ('main_cluster_id', PROJECT_ID, INSTANCE_ID, '', CLUSTER_ZONE), + ('main_cluster_zone', PROJECT_ID, INSTANCE_ID, CLUSTER_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, main_cluster_id, + main_cluster_zone, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableInstanceCreateOperator( + project_id=project_id, + instance_id=instance_id, + main_cluster_id=main_cluster_id, + main_cluster_zone=main_cluster_zone, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_create_instance_that_exists(self, mock_hook): + mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + + op = BigtableInstanceCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + main_cluster_id=CLUSTER_ID, + main_cluster_zone=CLUSTER_ZONE, + task_id="id" + ) + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.create_instance.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_different_error_reraised(self, mock_hook): + mock_hook.return_value.get_instance.return_value = None + op = BigtableInstanceCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + main_cluster_id=CLUSTER_ID, + main_cluster_zone=CLUSTER_ZONE, + task_id="id" + ) + + mock_hook.return_value.create_instance.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.GoogleAPICallError('error')) + + with self.assertRaises(google.api_core.exceptions.GoogleAPICallError): + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.create_instance.assert_called_once_with( + PROJECT_ID, INSTANCE_ID, CLUSTER_ID, CLUSTER_ZONE, None, None, None, None, None, None, None, None + ) + + +class BigtableClusterUpdateTest(unittest.TestCase): + @parameterized.expand([ + ('project_id', '', INSTANCE_ID, CLUSTER_ID, NODES), + ('instance_id', PROJECT_ID, '', CLUSTER_ID, NODES), + ('cluster_id', PROJECT_ID, INSTANCE_ID, '', NODES), + ('nodes', PROJECT_ID, INSTANCE_ID, CLUSTER_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, cluster_id, nodes, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableClusterUpdateOperator( + project_id=project_id, + instance_id=instance_id, + cluster_id=cluster_id, + nodes=nodes, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_updating_cluster_but_instance_does_not_exists(self, mock_hook): + mock_hook.return_value.get_instance.return_value = None + + with self.assertRaises(AirflowException) as e: + op = BigtableClusterUpdateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + cluster_id=CLUSTER_ID, + nodes=NODES, + task_id="id" + ) + op.execute(None) + + err = e.exception + self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(INSTANCE_ID)) + mock_hook.assert_called_once_with() + mock_hook.return_value.update_cluster.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_updating_cluster_that_does_not_exists(self, mock_hook): + instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.update_cluster.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.NotFound("Cluster not found.")) + + with self.assertRaises(AirflowException) as e: + op = BigtableClusterUpdateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + cluster_id=CLUSTER_ID, + nodes=NODES, + task_id="id" + ) + op.execute(None) + + err = e.exception + self.assertEqual( + str(err), + "Dependency: cluster '{}' does not exist for instance '{}'.".format(CLUSTER_ID, INSTANCE_ID) + ) + mock_hook.assert_called_once_with() + mock_hook.return_value.update_cluster.assert_called_once_with(instance, CLUSTER_ID, NODES) + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_different_error_reraised(self, mock_hook): + op = BigtableClusterUpdateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + cluster_id=CLUSTER_ID, + nodes=NODES, + task_id="id" + ) + instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.update_cluster.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.GoogleAPICallError('error')) + + with self.assertRaises(google.api_core.exceptions.GoogleAPICallError): + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.update_cluster.assert_called_once_with(instance, CLUSTER_ID, NODES) + + +class BigtableInstanceDeleteTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_delete_execute(self, mock_hook): + op = BigtableInstanceDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + task_id="id" + ) + op.execute(None) + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID) + + @parameterized.expand([ + ('project_id', '', INSTANCE_ID), + ('instance_id', PROJECT_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableInstanceDeleteOperator( + project_id=project_id, + instance_id=instance_id, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_deleting_instance_that_doesnt_exists(self, mock_hook): + op = BigtableInstanceDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + task_id="id" + ) + mock_hook.return_value.delete_instance.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.NotFound("Instance not found.")) + op.execute(None) + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID) + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_different_error_reraised(self, mock_hook): + op = BigtableInstanceDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + task_id="id" + ) + mock_hook.return_value.delete_instance.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.GoogleAPICallError('error')) + + with self.assertRaises(google.api_core.exceptions.GoogleAPICallError): + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_instance.assert_called_once_with(PROJECT_ID, INSTANCE_ID) + + +class BigtableTableDeleteTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_delete_execute(self, mock_hook): + op = BigtableTableDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + op.execute(None) + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID) + + @parameterized.expand([ + ('project_id', '', INSTANCE_ID, TABLE_ID), + ('instance_id', PROJECT_ID, '', TABLE_ID), + ('table_id', PROJECT_ID, INSTANCE_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableTableDeleteOperator( + project_id=project_id, + instance_id=instance_id, + table_id=table_id, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_deleting_table_that_doesnt_exists(self, mock_hook): + op = BigtableTableDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + + mock_hook.return_value.delete_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.NotFound("Table not found.")) + op.execute(None) + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID) + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_deleting_table_when_instance_doesnt_exists(self, mock_hook): + op = BigtableTableDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + + mock_hook.return_value.get_instance.return_value = None + with self.assertRaises(AirflowException) as e: + op.execute(None) + err = e.exception + self.assertEqual(str(err), "Dependency: instance '{}' does not exist.".format(INSTANCE_ID)) + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_table.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_different_error_reraised(self, mock_hook): + op = BigtableTableDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + mock_hook.return_value.delete_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.GoogleAPICallError('error')) + + with self.assertRaises(google.api_core.exceptions.GoogleAPICallError): + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.delete_table.assert_called_once_with(PROJECT_ID, INSTANCE_ID, TABLE_ID) + + +class BigtableTableCreateTest(unittest.TestCase): + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_create_execute(self, mock_hook): + op = BigtableTableCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + initial_split_keys=INITIAL_SPLIT_KEYS, + column_families=EMPTY_COLUMN_FAMILIES, + task_id="id" + ) + instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + op.execute(None) + mock_hook.assert_called_once_with() + mock_hook.return_value.create_table.assert_called_once_with( + instance, TABLE_ID, INITIAL_SPLIT_KEYS, EMPTY_COLUMN_FAMILIES) + + @parameterized.expand([ + ('project_id', '', INSTANCE_ID, TABLE_ID), + ('instance_id', PROJECT_ID, '', TABLE_ID), + ('table_id', PROJECT_ID, INSTANCE_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableTableCreateOperator( + project_id=project_id, + instance_id=instance_id, + table_id=table_id, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_instance_not_exists(self, mock_hook): + op = BigtableTableCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + initial_split_keys=INITIAL_SPLIT_KEYS, + column_families=EMPTY_COLUMN_FAMILIES, + task_id="id" + ) + mock_hook.return_value.get_instance.return_value = None + with self.assertRaises(AirflowException) as e: + op.execute(None) + err = e.exception + self.assertEqual( + str(err), + "Dependency: instance '{}' does not exist in project '{}'.".format(INSTANCE_ID, PROJECT_ID) + ) + mock_hook.assert_called_once_with() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_creating_table_that_exists(self, mock_hook): + op = BigtableTableCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + initial_split_keys=INITIAL_SPLIT_KEYS, + column_families=EMPTY_COLUMN_FAMILIES, + task_id="id" + ) + + mock_hook.return_value.get_column_families_for_table.return_value = EMPTY_COLUMN_FAMILIES + instance = mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.create_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.AlreadyExists("Table already exists.")) + op.execute(None) + + mock_hook.assert_called_once_with() + mock_hook.return_value.create_table.assert_called_once_with( + instance, TABLE_ID, INITIAL_SPLIT_KEYS, EMPTY_COLUMN_FAMILIES) + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_creating_table_that_exists_with_different_column_families_ids_in_the_table(self, mock_hook): + op = BigtableTableCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + initial_split_keys=INITIAL_SPLIT_KEYS, + column_families=EMPTY_COLUMN_FAMILIES, + task_id="id" + ) + + mock_hook.return_value.get_column_families_for_table.return_value = {"existing_family": None} + mock_hook.return_value.create_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.AlreadyExists("Table already exists.")) + + with self.assertRaises(AirflowException) as e: + op.execute(None) + err = e.exception + self.assertEqual( + str(err), + "Table '{}' already exists with different Column Families.".format(TABLE_ID) + ) + mock_hook.assert_called_once_with() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_creating_table_that_exists_with_different_column_families_gc_rule_in_the_table(self, mock_hook): + op = BigtableTableCreateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + initial_split_keys=INITIAL_SPLIT_KEYS, + column_families={"cf-id": MaxVersionsGCRule(1)}, + task_id="id" + ) + + cf_mock = mock.Mock() + cf_mock.gc_rule = mock.Mock(return_value=MaxVersionsGCRule(2)) + + mock_hook.return_value.get_column_families_for_table.return_value = { + "cf-id": cf_mock + } + mock_hook.return_value.create_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.AlreadyExists("Table already exists.")) + + with self.assertRaises(AirflowException) as e: + op.execute(None) + err = e.exception + self.assertEqual( + str(err), + "Table '{}' already exists with different Column Families.".format(TABLE_ID) + ) + mock_hook.assert_called_once_with() + + +class BigtableWaitForTableReplicationTest(unittest.TestCase): + @parameterized.expand([ + ('project_id', '', INSTANCE_ID, TABLE_ID), + ('instance_id', PROJECT_ID, '', TABLE_ID), + ('table_id', PROJECT_ID, INSTANCE_ID, ''), + ], testcase_func_name=lambda f, n, p: 'test_empty_attribute.empty_' + p.args[0]) + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_empty_attribute(self, missing_attribute, project_id, instance_id, table_id, mock_hook): + with self.assertRaises(AirflowException) as e: + BigtableTableWaitForReplicationSensor( + project_id=project_id, + instance_id=instance_id, + table_id=table_id, + task_id="id" + ) + err = e.exception + self.assertEqual(str(err), 'Empty parameter: {}'.format(missing_attribute)) + mock_hook.assert_not_called() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_wait_no_instance(self, mock_hook): + mock_hook.return_value.get_instance.return_value = None + + op = BigtableTableWaitForReplicationSensor( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + self.assertFalse(op.poke(None)) + mock_hook.assert_called_once_with() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_wait_no_table(self, mock_hook): + mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.get_cluster_states_for_table.side_effect = mock.Mock( + side_effect=google.api_core.exceptions.NotFound("Table not found.")) + + op = BigtableTableWaitForReplicationSensor( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + self.assertFalse(op.poke(None)) + mock_hook.assert_called_once_with() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_wait_not_ready(self, mock_hook): + mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.get_cluster_states_for_table.return_value = { + "cl-id": ClusterState(0) + } + op = BigtableTableWaitForReplicationSensor( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + self.assertFalse(op.poke(None)) + mock_hook.assert_called_once_with() + + @mock.patch('airflow.contrib.operators.gcp_bigtable_operator.BigtableHook') + def test_wait_ready(self, mock_hook): + mock_hook.return_value.get_instance.return_value = mock.Mock(Instance) + mock_hook.return_value.get_cluster_states_for_table.return_value = { + "cl-id": ClusterState(4) + } + op = BigtableTableWaitForReplicationSensor( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + table_id=TABLE_ID, + task_id="id" + ) + self.assertTrue(op.poke(None)) + mock_hook.assert_called_once_with() From 0d55d2f37deb621ac7a97607b6c3cab4a1a53e0b Mon Sep 17 00:00:00 2001 From: Sumit Maheshwari Date: Fri, 4 Jan 2019 19:25:56 +0530 Subject: [PATCH 327/808] [AIRFLOW-3340] Placeholder support in connections form (#4185) --- airflow/www/static/connection_form.js | 8 ++++++++ airflow/www_rbac/static/js/connection_form.js | 8 ++++++++ 2 files changed, 16 insertions(+) diff --git a/airflow/www/static/connection_form.js b/airflow/www/static/connection_form.js index 8ed42c2dbed2e..8517ad2582bf4 100644 --- a/airflow/www/static/connection_form.js +++ b/airflow/www/static/connection_form.js @@ -59,6 +59,9 @@ relabeling: { 'host': 'API Endpoint', 'password': 'Auth Token', + }, + placeholders: { + 'host': 'https://.qubole.com/api' } }, ssh: { @@ -81,6 +84,8 @@ $("label[orig_text]").each(function(){ $(this).text($(this).attr("orig_text")); }); + $(".form-control").each(function(){$(this).attr('placeholder', '')}); + if (config[connectionType] != undefined){ $.each(config[connectionType].hidden_fields, function(i, field){ $("#" + field).parent().parent().addClass('hide') @@ -90,6 +95,9 @@ lbl.attr("orig_text", lbl.text()); $("label[for='" + k + "']").text(v); }); + $.each(config[connectionType].placeholders, function(k, v){ + $("#" + k).attr('placeholder', v); + }); } } var connectionType=$("#conn_type").val(); diff --git a/airflow/www_rbac/static/js/connection_form.js b/airflow/www_rbac/static/js/connection_form.js index 09034f8785203..17ca29066a21b 100644 --- a/airflow/www_rbac/static/js/connection_form.js +++ b/airflow/www_rbac/static/js/connection_form.js @@ -51,6 +51,9 @@ $(document).ready(function () { relabeling: { 'host': 'API Endpoint', 'password': 'Auth Token', + }, + placeholders: { + 'host': 'https://.qubole.com/api' } }, ssh: { @@ -72,6 +75,8 @@ $(document).ready(function () { $("label[orig_text]").each(function () { $(this).text($(this).attr("orig_text")); }); + $(".form-control").each(function(){$(this).attr('placeholder', '')}); + if (config[connectionType] != undefined) { $.each(config[connectionType].hidden_fields, function (i, field) { $("#" + field).parent().parent().addClass('hide') @@ -81,6 +86,9 @@ $(document).ready(function () { lbl.attr("orig_text", lbl.text()); $("label[for='" + k + "']").text(v); }); + $.each(config[connectionType].placeholders, function(k, v){ + $("#" + k).attr('placeholder', v); + }); } } From 08bd37b803f9d77cc65be88885a7aaeebd7d8fc8 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 4 Jan 2019 14:54:33 +0100 Subject: [PATCH 328/808] [AIRFLOW-3527] Update Cloud SQL Proxy to have shorter path for UNIX socket (#4350) --- airflow/contrib/hooks/gcp_sql_hook.py | 89 ++++++++++++------- .../operators/test_gcp_sql_operator.py | 45 +++++++++- 2 files changed, 100 insertions(+), 34 deletions(-) diff --git a/airflow/contrib/hooks/gcp_sql_hook.py b/airflow/contrib/hooks/gcp_sql_hook.py index 43e664d15c857..7dabf74789f97 100644 --- a/airflow/contrib/hooks/gcp_sql_hook.py +++ b/airflow/contrib/hooks/gcp_sql_hook.py @@ -19,8 +19,11 @@ import errno import json import os +import random import re import shutil +import string + import socket import platform import subprocess @@ -44,6 +47,8 @@ from airflow.models import Connection from airflow.utils.db import provide_session +UNIX_PATH_MAX = 108 + NUM_RETRIES = 5 # Time to sleep between active checks of the operation results @@ -378,8 +383,8 @@ def _download_sql_proxy_if_needed(self): download_url = CLOUD_SQL_PROXY_VERSION_DOWNLOAD_URL.format( self.sql_proxy_version, system, processor) proxy_path_tmp = self.sql_proxy_path + ".tmp" - self.log.info("Downloading cloud_sql_proxy from {} to {}". - format(download_url, proxy_path_tmp)) + self.log.info("Downloading cloud_sql_proxy from %s to %s", + download_url, proxy_path_tmp) r = requests.get(download_url, allow_redirects=True) # Downloading to .tmp file first to avoid case where partially downloaded # binary is used by parallel operator which uses the same fixed binary path @@ -389,9 +394,8 @@ def _download_sql_proxy_if_needed(self): raise AirflowException( "The cloud-sql-proxy could not be downloaded. Status code = {}. " "Reason = {}".format(r.status_code, r.reason)) - self.log.info("Moving sql_proxy binary from {} to {}".format( - proxy_path_tmp, self.sql_proxy_path - )) + self.log.info("Moving sql_proxy binary from %s to %s", + proxy_path_tmp, self.sql_proxy_path) shutil.move(proxy_path_tmp, self.sql_proxy_path) os.chmod(self.sql_proxy_path, 0o744) # Set executable bit self.sql_proxy_was_downloaded = True @@ -409,7 +413,7 @@ def _get_credential_parameters(self, session): elif GCP_CREDENTIALS_KEYFILE_DICT in connection.extra_dejson: credential_file_content = json.loads( connection.extra_dejson[GCP_CREDENTIALS_KEYFILE_DICT]) - self.log.info("Saving credentials to {}".format(self.credentials_path)) + self.log.info("Saving credentials to %s", self.credentials_path) with open(self.credentials_path, "w") as f: json.dump(credential_file_content, f) credential_params = [ @@ -419,8 +423,8 @@ def _get_credential_parameters(self, session): else: self.log.info( "The credentials are not supplied by neither key_path nor " - "keyfile_dict of the gcp connection {}. Falling back to " - "default activated account".format(self.gcp_conn_id)) + "keyfile_dict of the gcp connection %s. Falling back to " + "default activated account", self.gcp_conn_id) credential_params = [] if not self.instance_specification: @@ -450,18 +454,17 @@ def start_proxy(self): command_to_run = [self.sql_proxy_path] command_to_run.extend(self.command_line_parameters) try: - self.log.info("Creating directory {}".format( - self.cloud_sql_proxy_socket_directory)) + self.log.info("Creating directory %s", + self.cloud_sql_proxy_socket_directory) os.makedirs(self.cloud_sql_proxy_socket_directory) except OSError: # Needed for python 2 compatibility (exists_ok missing) pass command_to_run.extend(self._get_credential_parameters()) - self.log.info("Running the command: `{}`".format(" ".join(command_to_run))) + self.log.info("Running the command: `%s`", " ".join(command_to_run)) self.sql_proxy_process = Popen(command_to_run, stdin=PIPE, stdout=PIPE, stderr=PIPE) - self.log.info("The pid of cloud_sql_proxy: {}".format( - self.sql_proxy_process.pid)) + self.log.info("The pid of cloud_sql_proxy: %s", self.sql_proxy_process.pid) while True: line = self.sql_proxy_process.stderr.readline().decode('utf-8') return_code = self.sql_proxy_process.poll() @@ -489,16 +492,16 @@ def stop_proxy(self): if not self.sql_proxy_process: raise AirflowException("The sql proxy is not started yet") else: - self.log.info("Stopping the cloud_sql_proxy pid: {}".format( - self.sql_proxy_process.pid)) + self.log.info("Stopping the cloud_sql_proxy pid: %s", + self.sql_proxy_process.pid) self.sql_proxy_process.kill() self.sql_proxy_process = None # Cleanup! - self.log.info("Removing the socket directory: {}". - format(self.cloud_sql_proxy_socket_directory)) + self.log.info("Removing the socket directory: %s", + self.cloud_sql_proxy_socket_directory) shutil.rmtree(self.cloud_sql_proxy_socket_directory, ignore_errors=True) if self.sql_proxy_was_downloaded: - self.log.info("Removing downloaded proxy: {}".format(self.sql_proxy_path)) + self.log.info("Removing downloaded proxy: %s", self.sql_proxy_path) # Silently ignore if the file has already been removed (concurrency) try: os.remove(self.sql_proxy_path) @@ -506,11 +509,11 @@ def stop_proxy(self): if not e.errno == errno.ENOENT: raise else: - self.log.info("Skipped removing proxy - it was not downloaded: {}". - format(self.sql_proxy_path)) + self.log.info("Skipped removing proxy - it was not downloaded: %s", + self.sql_proxy_path) if isfile(self.credentials_path): - self.log.info("Removing generated credentials file {}". - format(self.credentials_path)) + self.log.info("Removing generated credentials file %s", + self.credentials_path) # Here file cannot be delete by concurrent task (each task has its own copy) os.remove(self.credentials_path) @@ -690,18 +693,38 @@ def _validate_inputs(self): self._check_ssl_file(self.sslcert, "sslcert") self._check_ssl_file(self.sslkey, "sslkey") self._check_ssl_file(self.sslrootcert, "sslrootcert") + if self.use_proxy and not self.sql_proxy_use_tcp: + if self.database_type == 'postgres': + suffix = "/.s.PGSQL.5432" + else: + suffix = "" + expected_path = "{}/{}:{}:{}{}".format( + self._generate_unique_path(), + self.project_id, self.instance, + self.database, suffix) + if len(expected_path) > UNIX_PATH_MAX: + self.log.info("Too long (%s) path: %s", len(expected_path), expected_path) + raise AirflowException( + "The UNIX socket path length cannot exceed {} characters " + "on Linux system. Either use shorter instance/database " + "name or switch to TCP connection. " + "The socket path for Cloud SQL proxy is now:" + "{}".format( + UNIX_PATH_MAX, expected_path)) - def _generate_unique_path(self): + @staticmethod + def _generate_unique_path(): # We are not using mkdtemp here as the path generated with mkdtemp # can be close to 60 characters and there is a limitation in # length of socket path to around 100 characters in total. # We append project/location/instance to it later and postgres - # appends its own prefix, so we chose a shorter "/tmp/{uuid1}" - based - # on host name and clock + clock sequence. This should be fairly - # sufficient for our needs and should even work if the time is set back. - # We are using db_conn_id generated with uuid1 so that connection - # id matches the folder - for easier debugging. - return "/tmp/" + self.db_conn_id + # appends its own prefix, so we chose a shorter "/tmp/[8 random characters]" - + random.seed() + while True: + candidate = "/tmp/" + ''.join( + random.choice(string.ascii_lowercase + string.digits) for _ in range(8)) + if not os.path.exists(candidate): + return candidate @staticmethod def _quote(value): @@ -754,8 +777,8 @@ def _generate_connection_uri(self): client_key_file=self._quote(self.sslkey), server_ca_file=self._quote(self.sslrootcert) ) - self.log.info("DB connection URI {}".format(connection_uri.replace( - quote_plus(self.password), 'XXXXXXXXXXXX'))) + self.log.info("DB connection URI %s", connection_uri.replace( + quote_plus(self.password), 'XXXXXXXXXXXX')) return connection_uri def _get_instance_socket_name(self): @@ -778,7 +801,7 @@ def create_connection(self, session=None): """ connection = Connection(conn_id=self.db_conn_id) uri = self._generate_connection_uri() - self.log.info("Creating connection {}".format(self.db_conn_id)) + self.log.info("Creating connection %s", self.db_conn_id) connection.parse_from_uri(uri) session.add(connection) session.commit() @@ -791,7 +814,7 @@ def delete_connection(self, session=None): :param session: Session of the SQL Alchemy ORM (automatically generated with decorator). """ - self.log.info("Deleting connection {}".format(self.db_conn_id)) + self.log.info("Deleting connection %s", self.db_conn_id) connection = session.query(models.Connection).filter( models.Connection.conn_id == self.db_conn_id)[0] session.delete(connection) diff --git a/tests/contrib/operators/test_gcp_sql_operator.py b/tests/contrib/operators/test_gcp_sql_operator.py index 516fcef4aa85b..a1d5ef632a8a7 100644 --- a/tests/contrib/operators/test_gcp_sql_operator.py +++ b/tests/contrib/operators/test_gcp_sql_operator.py @@ -511,7 +511,8 @@ def test_create_operator_with_wrong_parameters(self, get_connections): connection = Connection() connection.parse_from_uri( - "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type={database_type}&" + "gcpcloudsql://user:password@8.8.8.8:3200/testdb?" + "database_type={database_type}&" "project_id={project_id}&location={location}&instance={instance_name}&" "use_proxy={use_proxy}&use_ssl={use_ssl}". format(database_type=database_type, @@ -728,6 +729,48 @@ def test_create_operator_with_correct_parameters_mysql_tcp(self, get_connections self.assertNotEqual(3200, conn.port) self.assertEqual('testdb', conn.schema) + @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections") + def test_create_operator_with_too_long_unix_socket_path(self, get_connections): + connection = Connection() + connection.parse_from_uri( + "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" + "project_id=example-project&location=europe-west1&" + "instance=" + "test_db_with_long_name_a_bit_above_the_limit_of_UNIX_socket&" + "use_proxy=True&sql_proxy_use_tcp=False") + get_connections.return_value = [connection] + with self.assertRaises(AirflowException) as cm: + operator = CloudSqlQueryOperator( + sql=['SELECT * FROM TABLE'], + task_id='task_id' + ) + operator.cloudsql_db_hook.create_connection() + err = cm.exception + self.assertIn("The UNIX socket path length cannot exceed", str(err)) + + @mock.patch("airflow.hooks.base_hook.BaseHook.get_connections") + def test_create_operator_with_not_too_long_unix_socket_path(self, get_connections): + connection = Connection() + connection.parse_from_uri( + "gcpcloudsql://user:password@8.8.8.8:3200/testdb?database_type=postgres&" + "project_id=example-project&location=europe-west1&" + "instance=" + "test_db_with_longname_but_with_limit_of_UNIX_socket_aaaa&" + "use_proxy=True&sql_proxy_use_tcp=False") + get_connections.return_value = [connection] + operator = CloudSqlQueryOperator( + sql=['SELECT * FROM TABLE'], + task_id='task_id' + ) + operator.cloudsql_db_hook.create_connection() + try: + db_hook = operator.cloudsql_db_hook.get_database_hook() + conn = db_hook._get_connections_from_db(db_hook.postgres_conn_id)[0] + finally: + operator.cloudsql_db_hook.delete_connection() + self.assertEqual('postgres', conn.conn_type) + self.assertEqual('testdb', conn.schema) + @unittest.skipIf( BaseGcpIntegrationTestCase.skip_check(GCP_CLOUDSQL_KEY), SKIP_TEST_WARNING) From c18b1b5f9fa63df12efa54d0a5c078dcee54d1eb Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 5 Jan 2019 13:06:56 +0100 Subject: [PATCH 329/808] [AIRFLOW-3480] Add GCP Spanner Database Operators (#4353) --- .../example_dags/example_gcp_spanner.py | 140 ++++++++--- airflow/contrib/hooks/gcp_spanner_hook.py | 236 ++++++++++++++---- .../contrib/operators/gcp_spanner_operator.py | 229 +++++++++++++++-- docs/howto/operator.rst | 162 +++++++++--- docs/integration.rst | 41 ++- setup.py | 2 +- .../operators/test_gcp_spanner_operator.py | 170 ++++++++++++- 7 files changed, 839 insertions(+), 141 deletions(-) diff --git a/airflow/contrib/example_dags/example_gcp_spanner.py b/airflow/contrib/example_dags/example_gcp_spanner.py index cec3dcb855bc6..0aeb1e63a0bd8 100644 --- a/airflow/contrib/example_dags/example_gcp_spanner.py +++ b/airflow/contrib/example_dags/example_gcp_spanner.py @@ -21,15 +21,16 @@ Example Airflow DAG that creates, updates, queries and deletes a Cloud Spanner instance. This DAG relies on the following environment variables -* SPANNER_PROJECT_ID - Google Cloud Platform project for the Cloud Spanner instance. -* SPANNER_INSTANCE_ID - Cloud Spanner instance ID. -* SPANNER_CONFIG_NAME - The name of the instance's configuration. Values are of the form - projects//instanceConfigs/. +* GCP_PROJECT_ID - Google Cloud Platform project for the Cloud Spanner instance. +* GCP_SPANNER_INSTANCE_ID - Cloud Spanner instance ID. +* GCP_SPANNER_DATABASE_ID - Cloud Spanner database ID. +* GCP_SPANNER_CONFIG_NAME - The name of the instance's configuration. Values are of the + form projects//instanceConfigs/. See also: https://cloud.google.com/spanner/docs/reference/rest/v1/projects.instanceConfigs#InstanceConfig https://cloud.google.com/spanner/docs/reference/rest/v1/projects.instanceConfigs/list#google.spanner.admin.instance.v1.InstanceAdmin.ListInstanceConfigs -* SPANNER_NODE_COUNT - Number of nodes allocated to the instance. -* SPANNER_DISPLAY_NAME - The descriptive name for this instance as it appears in UIs. +* GCP_SPANNER_NODE_COUNT - Number of nodes allocated to the instance. +* GCP_SPANNER_DISPLAY_NAME - The descriptive name for this instance as it appears in UIs. Must be unique per project and between 4 and 30 characters in length. """ @@ -39,16 +40,21 @@ from airflow import models from airflow.contrib.operators.gcp_spanner_operator import \ CloudSpannerInstanceDeployOperator, CloudSpannerInstanceDatabaseQueryOperator, \ - CloudSpannerInstanceDeleteOperator + CloudSpannerInstanceDeleteOperator, \ + CloudSpannerInstanceDatabaseDeployOperator, \ + CloudSpannerInstanceDatabaseUpdateOperator, \ + CloudSpannerInstanceDatabaseDeleteOperator # [START howto_operator_spanner_arguments] -PROJECT_ID = os.environ.get('SPANNER_PROJECT_ID', 'example-project') -INSTANCE_ID = os.environ.get('SPANNER_INSTANCE_ID', 'testinstance') -DB_ID = os.environ.get('SPANNER_DB_ID', 'db1') -CONFIG_NAME = os.environ.get('SPANNER_CONFIG_NAME', - 'projects/example-project/instanceConfigs/eur3') -NODE_COUNT = os.environ.get('SPANNER_NODE_COUNT', '1') -DISPLAY_NAME = os.environ.get('SPANNER_DISPLAY_NAME', 'Test Instance') +GCP_PROJECT_ID = os.environ.get('GCP_PROJECT_ID', 'example-project') +GCP_SPANNER_INSTANCE_ID = os.environ.get('GCP_SPANNER_INSTANCE_ID', 'testinstance') +GCP_SPANNER_DATABASE_ID = os.environ.get('GCP_SPANNER_DATABASE_ID', 'testdatabase') +GCP_SPANNER_CONFIG_NAME = os.environ.get('GCP_SPANNER_CONFIG_NAME', + 'projects/example-project/instanceConfigs/eur3') +GCP_SPANNER_NODE_COUNT = os.environ.get('GCP_SPANNER_NODE_COUNT', '1') +GCP_SPANNER_DISPLAY_NAME = os.environ.get('GCP_SPANNER_DISPLAY_NAME', 'Test Instance') +# OPERATION_ID should be unique per operation +OPERATION_ID = 'unique_operation_id' # [END howto_operator_spanner_arguments] default_args = { @@ -63,51 +69,115 @@ # Create # [START howto_operator_spanner_deploy] spanner_instance_create_task = CloudSpannerInstanceDeployOperator( - project_id=PROJECT_ID, - instance_id=INSTANCE_ID, - configuration_name=CONFIG_NAME, - node_count=int(NODE_COUNT), - display_name=DISPLAY_NAME, + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + configuration_name=GCP_SPANNER_CONFIG_NAME, + node_count=int(GCP_SPANNER_NODE_COUNT), + display_name=GCP_SPANNER_DISPLAY_NAME, task_id='spanner_instance_create_task' ) # [END howto_operator_spanner_deploy] # Update spanner_instance_update_task = CloudSpannerInstanceDeployOperator( - project_id=PROJECT_ID, - instance_id=INSTANCE_ID, - configuration_name=CONFIG_NAME, - node_count=int(NODE_COUNT) + 1, - display_name=DISPLAY_NAME + '_updated', + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + configuration_name=GCP_SPANNER_CONFIG_NAME, + node_count=int(GCP_SPANNER_NODE_COUNT) + 1, + display_name=GCP_SPANNER_DISPLAY_NAME + '_updated', task_id='spanner_instance_update_task' ) + # [START howto_operator_spanner_database_deploy] + spanner_database_deploy_task = CloudSpannerInstanceDatabaseDeployOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + database_id=GCP_SPANNER_DATABASE_ID, + ddl_statements=[ + "CREATE TABLE my_table1 (id INT64, name STRING(MAX)) PRIMARY KEY (id)", + "CREATE TABLE my_table2 (id INT64, name STRING(MAX)) PRIMARY KEY (id)", + ], + task_id='spanner_database_deploy_task' + ) + # [END howto_operator_spanner_database_deploy] + + # [START howto_operator_spanner_database_update] + spanner_database_update_task = CloudSpannerInstanceDatabaseUpdateOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + database_id=GCP_SPANNER_DATABASE_ID, + ddl_statements=[ + "CREATE TABLE my_table3 (id INT64, name STRING(MAX)) PRIMARY KEY (id)", + ], + task_id='spanner_database_update_task' + ) + # [END howto_operator_spanner_database_update] + + # [START howto_operator_spanner_database_update_idempotent] + spanner_database_update_idempotent1_task = CloudSpannerInstanceDatabaseUpdateOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + database_id=GCP_SPANNER_DATABASE_ID, + operation_id=OPERATION_ID, + ddl_statements=[ + "CREATE TABLE my_table_unique (id INT64, name STRING(MAX)) PRIMARY KEY (id)", + ], + task_id='spanner_database_update_idempotent1_task' + ) + spanner_database_update_idempotent2_task = CloudSpannerInstanceDatabaseUpdateOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + database_id=GCP_SPANNER_DATABASE_ID, + operation_id=OPERATION_ID, + ddl_statements=[ + "CREATE TABLE my_table_unique (id INT64, name STRING(MAX)) PRIMARY KEY (id)", + ], + task_id='spanner_database_update_idempotent2_task' + ) + # [END howto_operator_spanner_database_update_idempotent] + # [START howto_operator_spanner_query] - spanner_instance_query = CloudSpannerInstanceDatabaseQueryOperator( - project_id=PROJECT_ID, - instance_id=INSTANCE_ID, + spanner_instance_query_task = CloudSpannerInstanceDatabaseQueryOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, database_id='db1', - query="DELETE FROM my_table2 WHERE true", + query=["DELETE FROM my_table2 WHERE true"], task_id='spanner_instance_query' ) # [END howto_operator_spanner_query] - spanner_instance_query2 = CloudSpannerInstanceDatabaseQueryOperator( - project_id=PROJECT_ID, - instance_id=INSTANCE_ID, + spanner_instance_query2_task = CloudSpannerInstanceDatabaseQueryOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, database_id='db1', query="example_gcp_spanner.sql", task_id='spanner_instance_query2' ) + # [START howto_operator_spanner_database_delete] + spanner_database_delete_task = CloudSpannerInstanceDatabaseDeleteOperator( + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, + database_id=GCP_SPANNER_DATABASE_ID, + task_id='spanner_database_delete_task' + ) + # [END howto_operator_spanner_database_delete] + # [START howto_operator_spanner_delete] spanner_instance_delete_task = CloudSpannerInstanceDeleteOperator( - project_id=PROJECT_ID, - instance_id=INSTANCE_ID, + project_id=GCP_PROJECT_ID, + instance_id=GCP_SPANNER_INSTANCE_ID, task_id='spanner_instance_delete_task' ) # [END howto_operator_spanner_delete] - spanner_instance_create_task >> spanner_instance_update_task \ - >> spanner_instance_query >> spanner_instance_query2 \ + spanner_instance_create_task \ + >> spanner_instance_update_task \ + >> spanner_database_deploy_task \ + >> spanner_database_update_task \ + >> spanner_database_update_idempotent1_task \ + >> spanner_database_update_idempotent2_task \ + >> spanner_instance_query_task \ + >> spanner_instance_query2_task \ + >> spanner_database_delete_task \ >> spanner_instance_delete_task diff --git a/airflow/contrib/hooks/gcp_spanner_hook.py b/airflow/contrib/hooks/gcp_spanner_hook.py index 96e8bcb71c784..23f3a3c86d400 100644 --- a/airflow/contrib/hooks/gcp_spanner_hook.py +++ b/airflow/contrib/hooks/gcp_spanner_hook.py @@ -16,13 +16,14 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from google.api_core.exceptions import GoogleAPICallError +from google.api_core.exceptions import GoogleAPICallError, AlreadyExists from google.cloud.spanner_v1.client import Client from google.cloud.spanner_v1.database import Database from google.cloud.spanner_v1.instance import Instance # noqa: F401 from google.longrunning.operations_grpc_pb2 import Operation # noqa: F401 from typing import Optional, Callable # noqa: F401 +from airflow import AirflowException from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook @@ -41,11 +42,12 @@ def __init__(self, def get_client(self, project_id): # type: (str) -> Client """ - Provides a client for interacting with Cloud Spanner API. + Provides a client for interacting with the Cloud Spanner API. - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the GCP project that owns the Cloud Spanner + database. :type project_id: str - :return: Client for interacting with Cloud Spanner API. See: + :return: Client for interacting with the Cloud Spanner API. See: https://googleapis.github.io/google-cloud-python/latest/spanner/client-api.html#google.cloud.spanner_v1.client.Client :rtype: object """ @@ -58,16 +60,15 @@ def get_instance(self, project_id, instance_id): """ Gets information about a particular instance. - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the project which owns the Cloud Spanner Database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The ID of the Cloud Spanner instance. :type instance_id: str :return: Representation of a Cloud Spanner Instance. See: https://googleapis.github.io/google-cloud-python/latest/spanner/instance-api.html#google.cloud.spanner_v1.instance.Instance :rtype: object """ - client = self.get_client(project_id) - instance = client.instance(instance_id) + instance = self.get_client(project_id).instance(instance_id) if not instance.exists(): return None return instance @@ -78,21 +79,22 @@ def create_instance(self, project_id, instance_id, configuration_name, node_coun """ Creates a new Cloud Spanner instance. - :param project_id: The ID of the project which owns the instances, tables and - data. + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The ID of the Cloud Spanner instance. :type instance_id: str - :param configuration_name: Name of the instance configuration defining how the - instance will be created. Required for instances which do not yet exist. + :param configuration_name: The name of the instance configuration defining how the + instance will be created. Possible configuration values can be retrieved via + https://cloud.google.com/spanner/docs/reference/rest/v1/projects.instanceConfigs/list :type configuration_name: str - :param node_count: (Optional) Number of nodes allocated to the instance. + :param node_count: (Optional) The number of nodes allocated to the Cloud Spanner + instance. :type node_count: int - :param display_name: (Optional) The display name for the instance in the Cloud - Console UI. (Must be between 4 and 30 characters.) If this value is not set - in the constructor, will fall back to the instance ID. + :param display_name: (Optional) The display name for the instance in the GCP + Console. Must be between 4 and 30 characters. If this value is not set in + the constructor, the name falls back to the instance ID. :type display_name: str - :return: True if the operation succeeded, raises an exception otherwise. + :return: True if the operation succeeds. Otherwise,raises an exception. :rtype: bool """ return self._apply_to_instance(project_id, instance_id, configuration_name, @@ -104,21 +106,22 @@ def update_instance(self, project_id, instance_id, configuration_name, node_coun """ Updates an existing Cloud Spanner instance. - :param project_id: The ID of the project which owns the instances, tables and - data. + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The ID of the Cloud Spanner instance. :type instance_id: str - :param configuration_name: Name of the instance configuration defining how the - instance will be created. Required for instances which do not yet exist. + :param configuration_name: The name of the instance configuration defining how the + instance will be created. Possible configuration values can be retrieved via + https://cloud.google.com/spanner/docs/reference/rest/v1/projects.instanceConfigs/list :type configuration_name: str - :param node_count: (Optional) Number of nodes allocated to the instance. + :param node_count: (Optional) The number of nodes allocated to the Cloud Spanner + instance. :type node_count: int - :param display_name: (Optional) The display name for the instance in the Cloud - Console UI. (Must be between 4 and 30 characters.) If this value is not set - in the constructor, will fall back to the instance ID. + :param display_name: (Optional) The display name for the instance in the GCP + Console. Must be between 4 and 30 characters. If this value is not set in + the constructor, the name falls back to the instance ID. :type display_name: str - :return: True if the operation succeeded, raises an exception otherwise. + :return: True if the operation succeeded. Otherwise, raises an exception. :rtype: bool """ return self._apply_to_instance(project_id, instance_id, configuration_name, @@ -130,8 +133,7 @@ def _apply_to_instance(self, project_id, instance_id, configuration_name, node_c """ Invokes a method on a given instance by applying a specified Callable. - :param project_id: The ID of the project which owns the instances, tables and - data. + :param project_id: The ID of the project which owns the Cloud Spanner Database. :type project_id: str :param instance_id: The ID of the instance. :type instance_id: str @@ -147,15 +149,13 @@ def _apply_to_instance(self, project_id, instance_id, configuration_name, node_c :param func: Method of the instance to be called. :type func: Callable """ - client = self.get_client(project_id) - instance = client.instance(instance_id, - configuration_name=configuration_name, - node_count=node_count, - display_name=display_name) + instance = self.get_client(project_id).instance( + instance_id, configuration_name=configuration_name, + node_count=node_count, display_name=display_name) try: operation = func(instance) # type: Operation except GoogleAPICallError as e: - self.log.error('An error occurred: %s. Aborting.', e.message) + self.log.error('An error occurred: %s. Exiting.', e.message) raise e if operation: @@ -168,39 +168,175 @@ def delete_instance(self, project_id, instance_id): """ Deletes an existing Cloud Spanner instance. - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The ID of the Cloud Spanner instance. :type instance_id: str """ - client = self.get_client(project_id) - instance = client.instance(instance_id) + instance = self.get_client(project_id).instance(instance_id) try: instance.delete() return True except GoogleAPICallError as e: - self.log.error('An error occurred: %s. Aborting.', e.message) + self.log.error('An error occurred: %s. Exiting.', e.message) + raise e + + def get_database(self, project_id, instance_id, database_id): + # type: (str, str, str) -> Optional[Database] + """ + Retrieves a database in Cloud Spanner. If the database does not exist + in the specified instance, it returns None. + + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. + :type project_id: str + :param instance_id: The ID of the Cloud Spanner instance. + :type instance_id: str + :param database_id: The ID of the database in Cloud Spanner. + :type database_id: str + :return: Database object or None if database does not exist + :rtype: Union[Database, None] + """ + + instance = self.get_client(project_id=project_id).instance( + instance_id=instance_id) + if not instance.exists(): + raise AirflowException("The instance {} does not exist in project {} !". + format(instance_id, project_id)) + database = instance.database(database_id=database_id) + if not database.exists(): + return None + else: + return database + + def create_database(self, project_id, instance_id, database_id, ddl_statements): + # type: (str, str, str, [str]) -> bool + """ + Creates a new database in Cloud Spanner. + + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. + :type project_id: str + :param instance_id: The ID of the Cloud Spanner instance. + :type instance_id: str + :param database_id: The ID of the database to create in Cloud Spanner. + :type database_id: str + :param ddl_statements: The string list containing DDL for the new database. + :type ddl_statements: list[str] + :return: True if everything succeeded + :rtype: bool + """ + + instance = self.get_client(project_id=project_id).instance( + instance_id=instance_id) + if not instance.exists(): + raise AirflowException("The instance {} does not exist in project {} !". + format(instance_id, project_id)) + database = instance.database(database_id=database_id, + ddl_statements=ddl_statements) + try: + operation = database.create() # type: Operation + except GoogleAPICallError as e: + self.log.error('An error occurred: %s. Exiting.', e.message) raise e + if operation: + result = operation.result() + self.log.info(result) + return True + + def update_database(self, project_id, instance_id, database_id, ddl_statements, + operation_id=None): + # type: (str, str, str, [str], str) -> bool + """ + Updates DDL of a database in Cloud Spanner. + + :param project_id: The ID of the GCP project that owns the Cloud Spanner database. + :type project_id: str + :param instance_id: The ID of the Cloud Spanner instance. + :type instance_id: str + :param database_id: The ID of the database in Cloud Spanner. + :type database_id: str + :param ddl_statements: The string list containing DDL for the new database. + :type ddl_statements: list[str] + :param operation_id: (Optional) The unique per database operation ID that can be + specified to implement idempotency check. + :type operation_id: str + :return: True if everything succeeded + :rtype: bool + """ + + instance = self.get_client(project_id=project_id).instance( + instance_id=instance_id) + if not instance.exists(): + raise AirflowException("The instance {} does not exist in project {} !". + format(instance_id, project_id)) + database = instance.database(database_id=database_id) + try: + operation = database.update_ddl( + ddl_statements, operation_id=operation_id) + if operation: + result = operation.result() + self.log.info(result) + return True + except AlreadyExists as e: + if e.code == 409 and operation_id in e.message: + self.log.info("Replayed update_ddl message - the operation id %s " + "was already done before.", operation_id) + return True + except GoogleAPICallError as e: + self.log.error('An error occurred: %s. Exiting.', e.message) + raise e + + def delete_database(self, project_id, instance_id, database_id): + # type: (str, str, str) -> bool + """ + Drops a database in Cloud Spanner. + + :param project_id: The ID of the GCP project that owns the Cloud Spanner + database. + :type project_id: str + :param instance_id: The ID of the Cloud Spanner instance. + :type instance_id: str + :param database_id: The ID of the database in Cloud Spanner. + :type database_id: str + :return: True if everything succeeded + :rtype: bool + """ + + instance = self.get_client(project_id=project_id).\ + instance(instance_id=instance_id) + if not instance.exists(): + raise AirflowException("The instance {} does not exist in project {} !". + format(instance_id, project_id)) + database = instance.database(database_id=database_id) + try: + operation = database.drop() # type: Operation + except GoogleAPICallError as e: + self.log.error('An error occurred: %s. Exiting.', e.message) + raise e + + if operation: + result = operation.result() + self.log.info(result) + return True + def execute_dml(self, project_id, instance_id, database_id, queries): # type: (str, str, str, str) -> None """ Executes an arbitrary DML query (INSERT, UPDATE, DELETE). - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the GCP project that owns the Cloud Spanner + database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The ID of the Cloud Spanner instance. :type instance_id: str - :param database_id: The ID of the database. + :param database_id: The ID of the database in Cloud Spanner. :type database_id: str - :param queries: The queries to be executed. + :param queries: The queries to execute. :type queries: str """ - client = self.get_client(project_id) - instance = client.instance(instance_id) - database = Database(database_id, instance) - database.run_in_transaction(lambda transaction: - self._execute_sql_in_transaction(transaction, queries)) + instance = self.get_client(project_id).instance(instance_id) + Database(database_id, instance).run_in_transaction( + lambda transaction: self._execute_sql_in_transaction(transaction, queries)) @staticmethod def _execute_sql_in_transaction(transaction, queries): diff --git a/airflow/contrib/operators/gcp_spanner_operator.py b/airflow/contrib/operators/gcp_spanner_operator.py index b803fcc30ac15..8ea08ea0cf46c 100644 --- a/airflow/contrib/operators/gcp_spanner_operator.py +++ b/airflow/contrib/operators/gcp_spanner_operator.py @@ -26,21 +26,23 @@ class CloudSpannerInstanceDeployOperator(BaseOperator): """ - Creates a new Cloud Spanner instance or, if an instance with the same instance_id - exists in the specified project, updates it. + Creates a new Cloud Spanner instance, or if an instance with the same instance_id + exists in the specified project, updates the Cloud Spanner instance. - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the project which owns the Cloud Spanner Database. :type project_id: str :param instance_id: Cloud Spanner instance ID. :type instance_id: str - :param configuration_name: Name of the instance configuration defining - how the instance will be created. Required for instances which do not yet exist. + :param configuration_name: The name of the Cloud Spanner instance configuration + defining how the instance will be created. Required for + instances that do not yet exist. :type configuration_name: str - :param node_count: (Optional) Number of nodes allocated to the instance. + :param node_count: (Optional) The number of nodes allocated to the Cloud Spanner + instance. :type node_count: int - :param display_name: (Optional) The display name for the instance in the - Cloud Console UI. (Must be between 4 and 30 characters.) If this value is not - set in the constructor, will fall back to the instance ID. + :param display_name: (Optional) The display name for the Cloud Spanner instance in + the GCP Console. (Must be between 4 and 30 characters.) If this value is not set + in the constructor, the name is the same as the instance ID. :type display_name: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str @@ -91,12 +93,12 @@ def execute(self, context): class CloudSpannerInstanceDeleteOperator(BaseOperator): """ - Deletes a Cloud Spanner instance. - If an instance does not exist, no action will be taken and the operator will succeed. + Deletes a Cloud Spanner instance. If an instance does not exist, + no action is taken and the operator succeeds. - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the project that owns the Cloud Spanner Database. :type project_id: str - :param instance_id: Cloud Spanner instance ID. + :param instance_id: The Cloud Spanner instance ID. :type instance_id: str :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str @@ -138,13 +140,14 @@ class CloudSpannerInstanceDatabaseQueryOperator(BaseOperator): """ Executes an arbitrary DML query (INSERT, UPDATE, DELETE). - :param project_id: The ID of the project which owns the instances, tables and data. + :param project_id: The ID of the project that owns the Cloud Spanner Database. :type project_id: str - :param instance_id: The ID of the instance. + :param instance_id: The Cloud Spanner instance ID. :type instance_id: str - :param database_id: The ID of the database. + :param database_id: The Cloud Spanner database ID. :type database_id: str - :param query: The query or list of queries to be executed. Can be a path to a SQL file. + :param query: The query or list of queries to be executed. Can be a path to a SQL + file. :type query: str or list :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. :type gcp_conn_id: str @@ -197,3 +200,195 @@ def execute(self, context): def sanitize_queries(queries): if len(queries) and queries[-1] == '': del queries[-1] + + +class CloudSpannerInstanceDatabaseDeployOperator(BaseOperator): + """ + Creates a new Cloud Spanner database, or if database exists, + the operator does nothing. + + :param project_id: The ID of the project that owns the Cloud Spanner Database. + :type project_id: str + :param instance_id: The Cloud Spanner instance ID. + :type instance_id: str + :param database_id: The Cloud Spanner database ID. + :type database_id: str + :param ddl_statements: The string list containing DDL for the new database. + :type ddl_statements: list of str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + # [START gcp_spanner_database_deploy_template_fields] + template_fields = ('project_id', 'instance_id', 'database_id', 'ddl_statements', + 'gcp_conn_id') + template_ext = ('.sql', ) + # [END gcp_spanner_database_deploy_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + database_id, + ddl_statements, + gcp_conn_id='google_cloud_default', + *args, **kwargs): + self.instance_id = instance_id + self.project_id = project_id + self.database_id = database_id + self.ddl_statements = ddl_statements + self.gcp_conn_id = gcp_conn_id + self._validate_inputs() + self._hook = CloudSpannerHook(gcp_conn_id=gcp_conn_id) + super(CloudSpannerInstanceDatabaseDeployOperator, self).__init__(*args, **kwargs) + + def _validate_inputs(self): + if not self.project_id: + raise AirflowException("The required parameter 'project_id' is empty") + if not self.instance_id: + raise AirflowException("The required parameter 'instance_id' is empty") + if not self.database_id: + raise AirflowException("The required parameter 'database_id' is empty") + if not self.ddl_statements: + raise AirflowException("The required parameter 'ddl_statements' is empty") + + def execute(self, context): + if not self._hook.get_database(self.project_id, + self.instance_id, + self.database_id): + self.log.info("Creating Cloud Spanner database " + "'%s' in project '%s' and instance '%s'", + self.database_id, self.project_id, self.instance_id) + return self._hook.create_database(project_id=self.project_id, + instance_id=self.instance_id, + database_id=self.database_id, + ddl_statements=self.ddl_statements) + else: + self.log.info("The database '%s' in project '%s' and instance '%s'" + " already exists. Nothing to do. Exiting.", + self.database_id, self.project_id, self.instance_id) + return True + + +class CloudSpannerInstanceDatabaseUpdateOperator(BaseOperator): + """ + Updates a Cloud Spanner database with the specified DDL statement. + + :param project_id: The ID of the project that owns the the Cloud Spanner Database. + :type project_id: str + :param instance_id: The Cloud Spanner instance ID. + :type instance_id: str + :param database_id: The Cloud Spanner database ID. + :type database_id: str + :param ddl_statements: The string list containing DDL to apply to the database. + :type ddl_statements: list[str] + :param operation_id: (Optional) Unique per database operation id that can + be specified to implement idempotency check. + :type operation_id: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + # [START gcp_spanner_database_update_template_fields] + template_fields = ('project_id', 'instance_id', 'database_id', 'ddl_statements', + 'gcp_conn_id') + template_ext = ('.sql', ) + # [END gcp_spanner_database_update_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + database_id, + ddl_statements, + operation_id=None, + gcp_conn_id='google_cloud_default', + *args, **kwargs): + self.instance_id = instance_id + self.project_id = project_id + self.database_id = database_id + self.ddl_statements = ddl_statements + self.operation_id = operation_id + self.gcp_conn_id = gcp_conn_id + self._validate_inputs() + self._hook = CloudSpannerHook(gcp_conn_id=gcp_conn_id) + super(CloudSpannerInstanceDatabaseUpdateOperator, self).__init__(*args, **kwargs) + + def _validate_inputs(self): + if not self.project_id: + raise AirflowException("The required parameter 'project_id' is empty") + if not self.instance_id: + raise AirflowException("The required parameter 'instance_id' is empty") + if not self.database_id: + raise AirflowException("The required parameter 'database_id' is empty") + if not self.ddl_statements: + raise AirflowException("The required parameter 'ddl_statements' is empty") + + def execute(self, context): + if not self._hook.get_database(self.project_id, + self.instance_id, + self.database_id): + raise AirflowException("The Cloud Spanner database " + "'%s' in project '%s' and instance '%s' is missing." + " Create the database first before you can update it.", + self.database_id, self.project_id, self.instance_id) + else: + return self._hook.update_database(project_id=self.project_id, + instance_id=self.instance_id, + database_id=self.database_id, + ddl_statements=self.ddl_statements, + operation_id=self.operation_id) + + +class CloudSpannerInstanceDatabaseDeleteOperator(BaseOperator): + """ + Deletes a Cloud Spanner database. + + :param project_id: The ID of the project that owns the Cloud Spanner Database. + :type project_id: str + :param instance_id: Cloud Spanner instance ID. + :type instance_id: str + :param database_id: Cloud Spanner database ID. + :type database_id: str + :param gcp_conn_id: The connection ID used to connect to Google Cloud Platform. + :type gcp_conn_id: str + """ + # [START gcp_spanner_database_delete_template_fields] + template_fields = ('project_id', 'instance_id', 'database_id', + 'gcp_conn_id') + # [END gcp_spanner_database_delete_template_fields] + + @apply_defaults + def __init__(self, + project_id, + instance_id, + database_id, + gcp_conn_id='google_cloud_default', + *args, **kwargs): + self.instance_id = instance_id + self.project_id = project_id + self.database_id = database_id + self.gcp_conn_id = gcp_conn_id + self._validate_inputs() + self._hook = CloudSpannerHook(gcp_conn_id=gcp_conn_id) + super(CloudSpannerInstanceDatabaseDeleteOperator, self).__init__(*args, **kwargs) + + def _validate_inputs(self): + if not self.project_id: + raise AirflowException("The required parameter 'project_id' is empty") + if not self.instance_id: + raise AirflowException("The required parameter 'instance_id' is empty") + if not self.database_id: + raise AirflowException("The required parameter 'database_id' is empty") + + def execute(self, context): + db = self._hook.get_database(self.project_id, + self.instance_id, + self.database_id) + if not db: + self.log.info("The Cloud Spanner database was missing: " + "'%s' in project '%s' and instance '%s'. Assuming success.", + self.database_id, self.project_id, self.instance_id) + return True + else: + return self._hook.delete_database(project_id=self.project_id, + instance_id=self.instance_id, + database_id=self.database_id) diff --git a/docs/howto/operator.rst b/docs/howto/operator.rst index e9aade62957cf..aaa27e6a1484a 100644 --- a/docs/howto/operator.rst +++ b/docs/howto/operator.rst @@ -661,21 +661,117 @@ More information See `Google Cloud Functions API documentation `_ -Google Cloud Sql Operators --------------------------- +Google Cloud Spanner Operators +------------------------------ -CloudSpannerInstanceDatabaseQueryOperator -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +CloudSpannerInstanceDatabaseDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Executes an arbitrary DML query (INSERT, UPDATE, DELETE). +Deletes a database from the specified Cloud Spanner instance. If the database does not +exist, no action is taken, and the operator succeeds. + +For parameter definition, take a look at +:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeleteOperator`. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from environment variables. + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py + :language: python + :start-after: [START howto_operator_spanner_arguments] + :end-before: [END howto_operator_spanner_arguments] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_spanner_database_delete] + :end-before: [END howto_operator_spanner_database_delete] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_spanner_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_spanner_delete_template_fields] + :end-before: [END gcp_spanner_delete_template_fields] + +More information +"""""""""""""""" + +See `Google Cloud Spanner API documentation for database drop call +`_. + + +CloudSpannerInstanceDatabaseDeployOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Creates a new Cloud Spanner database in the specified instance, or if the +desired database exists, assumes success with no changes applied to database +configuration. No structure of the database is verified - it's enough if the database exists +with the same name. + +For parameter definition, take a look at +:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeployOperator`. + +Arguments +""""""""" + +Some arguments in the example DAG are taken from environment variables. + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py + :language: python + :start-after: [START howto_operator_spanner_arguments] + :end-before: [END howto_operator_spanner_arguments] + +Using the operator +"""""""""""""""""" + +.. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_spanner_database_deploy] + :end-before: [END howto_operator_spanner_database_deploy] + +Templating +"""""""""" + +.. literalinclude:: ../../airflow/contrib/operators/gcp_spanner_operator.py + :language: python + :dedent: 4 + :start-after: [START gcp_spanner_database_deploy_template_fields] + :end-before: [END gcp_spanner_database_deploy_template_fields] + +More information +"""""""""""""""" + +See Google Cloud Spanner API documentation for `database create +`_ + +CloudSpannerInstanceDatabaseUpdateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Runs a DDL query in a Cloud Spanner database and allows you to modify the structure of an +existing database. + +You can optionally specify an operation_id parameter which simplifies determining whether +the statements were executed in case the update_database call is replayed +(idempotency check). The operation_id should be unique within the database, and must be +a valid identifier: `[a-z][a-z0-9_]*`. More information can be found in +`the documentation of updateDdl API `_ For parameter definition take a look at -:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseQueryOperator`. +:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseUpdateOperator`. Arguments """"""""" -Some arguments in the example DAG are taken from environment variables: +Some arguments in the example DAG are taken from environment variables. .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py :language: python @@ -688,37 +784,36 @@ Using the operator .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py :language: python :dedent: 4 - :start-after: [START howto_operator_spanner_query] - :end-before: [END howto_operator_spanner_query] + :start-after: [START howto_operator_spanner_database_update] + :end-before: [END howto_operator_spanner_database_update] Templating """""""""" .. literalinclude:: ../../airflow/contrib/operators/gcp_spanner_operator.py - :language: python - :dedent: 4 - :start-after: [START gcp_spanner_query_template_fields] - :end-before: [END gcp_spanner_query_template_fields] + :language: python + :dedent: 4 + :start-after: [START gcp_spanner_database_update_template_fields] + :end-before: [END gcp_spanner_database_update_template_fields] More information """""""""""""""" -See Google Cloud Spanner API documentation for `the DML syntax -`_. +See Google Cloud Spanner API documentation for `database update_ddl +`_. -CloudSpannerInstanceDeployOperator -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +CloudSpannerInstanceDatabaseQueryOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Creates a new Cloud Spanner instance or, if an instance with the same name exists, -updates it. +Executes an arbitrary DML query (INSERT, UPDATE, DELETE). For parameter definition take a look at -:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDeployOperator`. +:class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseQueryOperator`. Arguments """"""""" -Some arguments in the example DAG are taken from environment variables: +Some arguments in the example DAG are taken from environment variables. .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py :language: python @@ -731,31 +826,29 @@ Using the operator .. literalinclude:: ../../airflow/contrib/example_dags/example_gcp_spanner.py :language: python :dedent: 4 - :start-after: [START howto_operator_spanner_deploy] - :end-before: [END howto_operator_spanner_deploy] + :start-after: [START howto_operator_spanner_query] + :end-before: [END howto_operator_spanner_query] Templating """""""""" .. literalinclude:: ../../airflow/contrib/operators/gcp_spanner_operator.py - :language: python - :dedent: 4 - :start-after: [START gcp_spanner_deploy_template_fields] - :end-before: [END gcp_spanner_deploy_template_fields] + :language: python + :dedent: 4 + :start-after: [START gcp_spanner_query_template_fields] + :end-before: [END gcp_spanner_query_template_fields] More information """""""""""""""" -See Google Cloud Spanner API documentation for instance `create -`_ -and `update -`_. +See Google Cloud Spanner API documentation for `the DML syntax +`_. CloudSpannerInstanceDeleteOperator ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Deletes a Cloud Spanner instance. -If an instance does not exist, no action will be taken and the operator will succeed. +Deletes a Cloud Spanner instance. If an instance does not exist, no action is taken, +and the operator succeeds. For parameter definition take a look at :class:`~airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDeleteOperator`. @@ -794,6 +887,9 @@ More information See `Google Cloud Spanner API documentation for instance delete `_. +Google Cloud Sql Operators +-------------------------- + CloudSqlInstanceDatabaseCreateOperator ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/integration.rst b/docs/integration.rst index 6a051134d891d..284c372d47454 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -663,11 +663,38 @@ Cloud Spanner Cloud Spanner Operators """"""""""""""""""""""" +- :ref:`CloudSpannerInstanceDatabaseDeleteOperator` : deletes an existing database from + a Google Cloud Spanner instance or returns success if the database is missing. +- :ref:`CloudSpannerInstanceDatabaseDeployOperator` : creates a new database in a Google + Cloud instance or returns success if the database already exists. +- :ref:`CloudSpannerInstanceDatabaseUpdateOperator` : updates the structure of a + Google Cloud Spanner database. - :ref:`CloudSpannerInstanceDatabaseQueryOperator` : executes an arbitrary DML query (INSERT, UPDATE, DELETE). -- :ref:`CloudSpannerInstanceDeployOperator` : creates a new Cloud Spanner instance or, - if an instance with the same name exists, updates it. -- :ref:`CloudSpannerInstanceDeleteOperator` : deletes a Cloud Spanner instance. +- :ref:`CloudSpannerInstanceDeployOperator` : creates a new Google Cloud Spanner instance, + or if an instance with the same name exists, updates the instance. +- :ref:`CloudSpannerInstanceDeleteOperator` : deletes a Google Cloud Spanner instance. + +.. _CloudSpannerInstanceDatabaseDeleteOperator: + +CloudSpannerInstanceDatabaseDeleteOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeleteOperator + +.. _CloudSpannerInstanceDatabaseDeployOperator: + +CloudSpannerInstanceDatabaseDeployOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseDeployOperator + +.. _CloudSpannerInstanceDatabaseUpdateOperator: + +CloudSpannerInstanceDatabaseUpdateOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDatabaseUpdateOperator .. _CloudSpannerInstanceDatabaseQueryOperator: @@ -690,6 +717,14 @@ CloudSpannerInstanceDeleteOperator .. autoclass:: airflow.contrib.operators.gcp_spanner_operator.CloudSpannerInstanceDeleteOperator + +CloudSpannerHook +"""""""""""""""" + +.. autoclass:: airflow.contrib.hooks.gcp_spanner_hook.CloudSpannerHook + :members: + + Cloud SQL ''''''''' diff --git a/setup.py b/setup.py index c650567742d01..f01a360dce59e 100644 --- a/setup.py +++ b/setup.py @@ -191,7 +191,7 @@ def write_version(filename=os.path.join(*['airflow', 'google-auth-httplib2>=0.0.1', 'google-cloud-container>=0.1.1', 'google-cloud-bigtable==0.31.0', - 'google-cloud-spanner>=1.6.0', + 'google-cloud-spanner>=1.7.1', 'grpcio-gcp>=0.2.2', 'PyOpenSSL', 'pandas-gbq' diff --git a/tests/contrib/operators/test_gcp_spanner_operator.py b/tests/contrib/operators/test_gcp_spanner_operator.py index 38ae985f26947..525f79647126f 100644 --- a/tests/contrib/operators/test_gcp_spanner_operator.py +++ b/tests/contrib/operators/test_gcp_spanner_operator.py @@ -23,7 +23,8 @@ from airflow import AirflowException from airflow.contrib.operators.gcp_spanner_operator import \ CloudSpannerInstanceDeployOperator, CloudSpannerInstanceDeleteOperator, \ - CloudSpannerInstanceDatabaseQueryOperator + CloudSpannerInstanceDatabaseQueryOperator, CloudSpannerInstanceDatabaseDeployOperator, \ + CloudSpannerInstanceDatabaseDeleteOperator, CloudSpannerInstanceDatabaseUpdateOperator from tests.contrib.operators.test_gcp_base import BaseGcpIntegrationTestCase, \ SKIP_TEST_WARNING, GCP_SPANNER_KEY @@ -46,7 +47,7 @@ INSERT_QUERY_2 = "INSERT my_table2 (id, name) VALUES (1, 'One')" CREATE_QUERY = "CREATE TABLE my_table1 (id INT64, name STRING(MAX)) PRIMARY KEY (id)" CREATE_QUERY_2 = "CREATE TABLE my_table2 (id INT64, name STRING(MAX)) PRIMARY KEY (id)" -QUERY_TYPE = "DML" +DDL_STATEMENTS = [CREATE_QUERY, CREATE_QUERY_2] class CloudSpannerTest(unittest.TestCase): @@ -240,6 +241,171 @@ def test_instance_query_dml_list(self, mock_hook): PROJECT_ID, INSTANCE_ID, DB_ID, [INSERT_QUERY, INSERT_QUERY_2] ) + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_create(self, mock_hook): + mock_hook.return_value.get_database.return_value = None + op = CloudSpannerInstanceDatabaseDeployOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + ddl_statements=DDL_STATEMENTS, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_database.assert_called_once_with( + project_id=PROJECT_ID, instance_id=INSTANCE_ID, database_id=DB_ID, + ddl_statements=DDL_STATEMENTS + ) + mock_hook.return_value.update_database.assert_not_called() + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_create_with_pre_existing_db(self, mock_hook): + mock_hook.return_value.get_database.return_value = {"name": DB_ID} + op = CloudSpannerInstanceDatabaseDeployOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + ddl_statements=DDL_STATEMENTS, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.create_database.assert_not_called() + mock_hook.return_value.update_database.assert_not_called() + self.assertTrue(result) + + @parameterized.expand([ + ("", INSTANCE_ID, DB_ID, DDL_STATEMENTS, 'project_id'), + (PROJECT_ID, "", DB_ID, DDL_STATEMENTS, 'instance_id'), + (PROJECT_ID, INSTANCE_ID, "", DDL_STATEMENTS, 'database_id'), + ]) + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_create_ex_if_param_missing(self, + project_id, instance_id, + database_id, ddl_statements, + exp_msg, mock_hook): + with self.assertRaises(AirflowException) as cm: + CloudSpannerInstanceDatabaseDeployOperator( + project_id=project_id, + instance_id=instance_id, + database_id=database_id, + ddl_statements=ddl_statements, + task_id="id" + ) + err = cm.exception + self.assertIn("The required parameter '{}' is empty".format(exp_msg), str(err)) + mock_hook.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_update(self, mock_hook): + mock_hook.return_value.get_database.return_value = {"name": DB_ID} + op = CloudSpannerInstanceDatabaseUpdateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + ddl_statements=DDL_STATEMENTS, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.update_database.assert_called_once_with( + project_id=PROJECT_ID, instance_id=INSTANCE_ID, database_id=DB_ID, + ddl_statements=DDL_STATEMENTS, operation_id=None + ) + self.assertTrue(result) + + @parameterized.expand([ + ("", INSTANCE_ID, DB_ID, DDL_STATEMENTS, 'project_id'), + (PROJECT_ID, "", DB_ID, DDL_STATEMENTS, 'instance_id'), + (PROJECT_ID, INSTANCE_ID, "", DDL_STATEMENTS, 'database_id'), + ]) + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_update_ex_if_param_missing(self, project_id, instance_id, + database_id, ddl_statements, + exp_msg, mock_hook): + with self.assertRaises(AirflowException) as cm: + CloudSpannerInstanceDatabaseUpdateOperator( + project_id=project_id, + instance_id=instance_id, + database_id=database_id, + ddl_statements=ddl_statements, + task_id="id" + ) + err = cm.exception + self.assertIn("The required parameter '{}' is empty".format(exp_msg), str(err)) + mock_hook.assert_not_called() + + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_update_ex_if_database_not_exist(self, mock_hook): + mock_hook.return_value.get_database.return_value = None + with self.assertRaises(AirflowException) as cm: + op = CloudSpannerInstanceDatabaseUpdateOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + ddl_statements=DDL_STATEMENTS, + task_id="id" + ) + op.execute(None) + err = cm.exception + self.assertIn("The Cloud Spanner database 'db1' in project 'project-id' and " + "instance 'instance-id' is missing", str(err)) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_delete(self, mock_hook): + mock_hook.return_value.get_database.return_value = {"name": DB_ID} + op = CloudSpannerInstanceDatabaseDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_database.assert_called_once_with( + project_id=PROJECT_ID, instance_id=INSTANCE_ID, database_id=DB_ID + ) + self.assertTrue(result) + + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_delete_exits_and_succeeds_if_database_does_not_exist(self, + mock_hook): + mock_hook.return_value.get_database.return_value = None + op = CloudSpannerInstanceDatabaseDeleteOperator( + project_id=PROJECT_ID, + instance_id=INSTANCE_ID, + database_id=DB_ID, + task_id="id" + ) + result = op.execute(None) + mock_hook.assert_called_once_with(gcp_conn_id="google_cloud_default") + mock_hook.return_value.delete_database.assert_not_called() + self.assertTrue(result) + + @parameterized.expand([ + ("", INSTANCE_ID, DB_ID, DDL_STATEMENTS, 'project_id'), + (PROJECT_ID, "", DB_ID, DDL_STATEMENTS, 'instance_id'), + (PROJECT_ID, INSTANCE_ID, "", DDL_STATEMENTS, 'database_id'), + ]) + @mock.patch("airflow.contrib.operators.gcp_spanner_operator.CloudSpannerHook") + def test_database_delete_ex_if_param_missing(self, project_id, instance_id, + database_id, ddl_statements, + exp_msg, mock_hook): + with self.assertRaises(AirflowException) as cm: + CloudSpannerInstanceDatabaseDeleteOperator( + project_id=project_id, + instance_id=instance_id, + database_id=database_id, + ddl_statements=ddl_statements, + task_id="id" + ) + err = cm.exception + self.assertIn("The required parameter '{}' is empty".format(exp_msg), str(err)) + mock_hook.assert_not_called() + @unittest.skipIf( BaseGcpIntegrationTestCase.skip_check(GCP_SPANNER_KEY), SKIP_TEST_WARNING) From c184da85d4cfd4fe2e95c42518c6700a7cdb4108 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sun, 25 Nov 2018 21:44:07 +0000 Subject: [PATCH 330/808] [AIRFLOW-1196][AIRFLOW-2399] Add templated field in TriggerDagRunOperator (#4228) * [AIRFLOW-1196][AIRFLOW-2399] Make trigger_dag_id a templated field for TriggerDagRunOperator * Update dagrun_operator.py --- airflow/operators/dagrun_operator.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/operators/dagrun_operator.py b/airflow/operators/dagrun_operator.py index 53814af41024d..8d339894f444e 100644 --- a/airflow/operators/dagrun_operator.py +++ b/airflow/operators/dagrun_operator.py @@ -35,7 +35,7 @@ class TriggerDagRunOperator(BaseOperator): """ Triggers a DAG run for a specified ``dag_id`` - :param trigger_dag_id: the dag_id to trigger + :param trigger_dag_id: the dag_id to trigger (templated) :type trigger_dag_id: str :param python_callable: a reference to a python function that will be called while passing it the ``context`` object and a placeholder @@ -50,7 +50,7 @@ class TriggerDagRunOperator(BaseOperator): :param execution_date: Execution date for the dag :type execution_date: datetime.datetime """ - template_fields = tuple() + template_fields = ('trigger_dag_id',) template_ext = tuple() ui_color = '#ffefeb' From 9af4770c6a4822ff08626d651bccc037aee8c543 Mon Sep 17 00:00:00 2001 From: aoen Date: Mon, 31 Dec 2018 08:31:11 +0200 Subject: [PATCH 331/808] [AIRFLOW-3581] Fix next_ds/prev_ds semantics for manual runs (#4385) --- UPDATING.md | 4 ++++ airflow/models.py | 46 +++++++++++++++++++++++++++------------------- tests/core.py | 32 ++++++++++++++++++++++++++++++++ 3 files changed, 63 insertions(+), 19 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index de68e065aad69..45ab7dc96942c 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -31,6 +31,10 @@ Examples: * `ts_nodash`: `20150101T000000` * `ts_nodash_with_tz`: `20150101T000000+0000` +### Semantics of next_ds/prev_ds changed for manually triggered runs + +next_ds/prev_ds now map to execution_date instead of the next/previous schedule-aligned execution date for DAGs triggered in the UI. + ### User model changes This patch changes the `User.superuser` field from a hardcoded boolean to a `Boolean()` database column. `User.superuser` will default to `False`, which means that this privilege will have to be granted manually to any users that may require it. diff --git a/airflow/models.py b/airflow/models.py index e7dd02a78eb17..484dd821c4fbb 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1849,13 +1849,38 @@ def get_template_context(self, session=None): if 'tables' in task.params: tables = task.params['tables'] + params = {} + run_id = '' + dag_run = None + if hasattr(task, 'dag'): + if task.dag.params: + params.update(task.dag.params) + dag_run = ( + session.query(DagRun) + .filter_by( + dag_id=task.dag.dag_id, + execution_date=self.execution_date) + .first() + ) + run_id = dag_run.run_id if dag_run else None + session.expunge_all() + session.commit() + ds = self.execution_date.strftime('%Y-%m-%d') ts = self.execution_date.isoformat() yesterday_ds = (self.execution_date - timedelta(1)).strftime('%Y-%m-%d') tomorrow_ds = (self.execution_date + timedelta(1)).strftime('%Y-%m-%d') - prev_execution_date = task.dag.previous_schedule(self.execution_date) - next_execution_date = task.dag.following_schedule(self.execution_date) + # For manually triggered dagruns that aren't run on a schedule, next/previous + # schedule dates don't make sense, and should be set to execution date for + # consistency with how execution_date is set for manually triggered tasks, i.e. + # triggered_date == execution_date. + if dag_run and dag_run.external_trigger: + prev_execution_date = self.execution_date + next_execution_date = self.execution_date + else: + prev_execution_date = task.dag.previous_schedule(self.execution_date) + next_execution_date = task.dag.following_schedule(self.execution_date) next_ds = None next_ds_nodash = None @@ -1878,23 +1903,6 @@ def get_template_context(self, session=None): ti_key_str = "{task.dag_id}__{task.task_id}__{ds_nodash}" ti_key_str = ti_key_str.format(**locals()) - params = {} - run_id = '' - dag_run = None - if hasattr(task, 'dag'): - if task.dag.params: - params.update(task.dag.params) - dag_run = ( - session.query(DagRun) - .filter_by( - dag_id=task.dag.dag_id, - execution_date=self.execution_date) - .first() - ) - run_id = dag_run.run_id if dag_run else None - session.expunge_all() - session.commit() - if task.params: params.update(task.params) diff --git a/tests/core.py b/tests/core.py index ef8c9497e7ea0..2df021412d4d3 100644 --- a/tests/core.py +++ b/tests/core.py @@ -1023,6 +1023,38 @@ def payload_generator(context, object): dag_run = dag_runs[0] self.assertEquals(dag_run.execution_date, utc_now) + def test_externally_triggered_dagrun(self): + TI = models.TaskInstance + + # Create the dagrun between two "scheduled" execution dates of the DAG + EXECUTION_DATE = DEFAULT_DATE + timedelta(days=2) + EXECUTION_DS = EXECUTION_DATE.strftime('%Y-%m-%d') + EXECUTION_DS_NODASH = EXECUTION_DS.replace('-', '') + + dag = DAG( + TEST_DAG_ID, + default_args=self.args, + schedule_interval=timedelta(weeks=1), + start_date=DEFAULT_DATE) + task = DummyOperator(task_id='test_externally_triggered_dag_context', + dag=dag) + dag.create_dagrun(run_id=models.DagRun.id_for_date(EXECUTION_DATE), + execution_date=EXECUTION_DATE, + state=State.RUNNING, + external_trigger=True) + task.run( + start_date=EXECUTION_DATE, end_date=EXECUTION_DATE) + + ti = TI(task=task, execution_date=EXECUTION_DATE) + context = ti.get_template_context() + + # next_ds/prev_ds should be the execution date for manually triggered runs + self.assertEquals(context['next_ds'], EXECUTION_DS) + self.assertEquals(context['next_ds_nodash'], EXECUTION_DS_NODASH) + + self.assertEquals(context['prev_ds'], EXECUTION_DS) + self.assertEquals(context['prev_ds_nodash'], EXECUTION_DS_NODASH) + class CliTests(unittest.TestCase): From a22462625324707ed754a16fd3a0f2979668bca5 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 5 Jan 2019 02:17:45 +0000 Subject: [PATCH 332/808] [AIRFLOW-3150] Make execution_date templated in TriggerDagRunOperator (#4359) --- airflow/operators/dagrun_operator.py | 28 ++++++++++++++++---- tests/core.py | 38 +++++++++++++++++++++++++++- 2 files changed, 60 insertions(+), 6 deletions(-) diff --git a/airflow/operators/dagrun_operator.py b/airflow/operators/dagrun_operator.py index 8d339894f444e..7000c6c90bc18 100644 --- a/airflow/operators/dagrun_operator.py +++ b/airflow/operators/dagrun_operator.py @@ -17,6 +17,8 @@ # specific language governing permissions and limitations # under the License. +import datetime +import six from airflow.models import BaseOperator from airflow.utils import timezone from airflow.utils.decorators import apply_defaults @@ -47,10 +49,10 @@ class TriggerDagRunOperator(BaseOperator): to your tasks while executing that DAG run. Your function header should look like ``def foo(context, dag_run_obj):`` :type python_callable: python callable - :param execution_date: Execution date for the dag - :type execution_date: datetime.datetime + :param execution_date: Execution date for the dag (templated) + :type execution_date: str or datetime.datetime """ - template_fields = ('trigger_dag_id',) + template_fields = ('trigger_dag_id', 'execution_date') template_ext = tuple() ui_color = '#ffefeb' @@ -64,10 +66,26 @@ def __init__( super(TriggerDagRunOperator, self).__init__(*args, **kwargs) self.python_callable = python_callable self.trigger_dag_id = trigger_dag_id - self.execution_date = execution_date + + if isinstance(execution_date, datetime.datetime): + self.execution_date = execution_date.isoformat() + elif isinstance(execution_date, six.string_types): + self.execution_date = execution_date + elif execution_date is None: + self.execution_date = execution_date + else: + raise TypeError( + 'Expected str or datetime.datetime type ' + 'for execution_date. Got {}'.format( + type(execution_date))) def execute(self, context): - dro = DagRunOrder(run_id='trig__' + timezone.utcnow().isoformat()) + if self.execution_date is not None: + run_id = 'trig__{}'.format(self.execution_date) + self.execution_date = timezone.parse(self.execution_date) + else: + run_id = 'trig__' + timezone.utcnow().isoformat() + dro = DagRunOrder(run_id=run_id) if self.python_callable is not None: dro = self.python_callable(context, dro) if dro: diff --git a/tests/core.py b/tests/core.py index 2df021412d4d3..fd0d803eee074 100644 --- a/tests/core.py +++ b/tests/core.py @@ -44,7 +44,7 @@ from airflow import configuration from airflow.executors import SequentialExecutor -from airflow.models import Variable +from airflow.models import Variable, TaskInstance configuration.conf.load_test_config() # NOQA: E402 from airflow import jobs, models, DAG, utils, macros, settings, exceptions @@ -1023,6 +1023,42 @@ def payload_generator(context, object): dag_run = dag_runs[0] self.assertEquals(dag_run.execution_date, utc_now) + def test_trigger_dagrun_with_str_execution_date(self): + utc_now_str = timezone.utcnow().isoformat() + self.assertIsInstance(utc_now_str, six.string_types) + run_id = 'trig__' + utc_now_str + + def payload_generator(context, object): + object.run_id = run_id + return object + + task = TriggerDagRunOperator( + task_id='test_trigger_dagrun_with_str_execution_date', + trigger_dag_id='example_bash_operator', + python_callable=payload_generator, + execution_date=utc_now_str, + dag=self.dag) + task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) + dag_runs = models.DagRun.find(dag_id='example_bash_operator', + run_id=run_id) + self.assertEquals(len(dag_runs), 1) + dag_run = dag_runs[0] + self.assertEquals(dag_run.execution_date.isoformat(), utc_now_str) + + def test_trigger_dagrun_with_templated_execution_date(self): + task = TriggerDagRunOperator( + task_id='test_trigger_dagrun_with_str_execution_date', + trigger_dag_id='example_bash_operator', + execution_date='{{ execution_date }}', + dag=self.dag) + + self.assertTrue(isinstance(task.execution_date, six.string_types)) + self.assertEqual(task.execution_date, '{{ execution_date }}') + + ti = TaskInstance(task=task, execution_date=DEFAULT_DATE) + ti.render_templates() + self.assertEqual(timezone.parse(task.execution_date), DEFAULT_DATE) + def test_externally_triggered_dagrun(self): TI = models.TaskInstance From 2420dbc7aa11e7536b340d98c5ffd87dabee906b Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 5 Jan 2019 13:15:56 +0000 Subject: [PATCH 333/808] [AIRFLOW-3634] Fix GCP Spanner Test (#4440) --- airflow/contrib/operators/gcp_spanner_operator.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/airflow/contrib/operators/gcp_spanner_operator.py b/airflow/contrib/operators/gcp_spanner_operator.py index 8ea08ea0cf46c..1cb4c7d47f069 100644 --- a/airflow/contrib/operators/gcp_spanner_operator.py +++ b/airflow/contrib/operators/gcp_spanner_operator.py @@ -326,10 +326,11 @@ def execute(self, context): if not self._hook.get_database(self.project_id, self.instance_id, self.database_id): - raise AirflowException("The Cloud Spanner database " - "'%s' in project '%s' and instance '%s' is missing." - " Create the database first before you can update it.", - self.database_id, self.project_id, self.instance_id) + raise AirflowException("The Cloud Spanner database '{}' in project '{}' and " + "instance '{}' is missing. Create the database first " + "before you can update it.".format(self.database_id, + self.project_id, + self.instance_id)) else: return self._hook.update_database(project_id=self.project_id, instance_id=self.instance_id, From 6c4e9407f28978ba646bcdd21f080d6af22a4007 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 5 Sep 2018 01:04:42 +0100 Subject: [PATCH 334/808] [AIRFLOW-3005] Replace 'Airbnb Airflow' with 'Apache Airflow' (#3845) --- airflow/contrib/example_dags/example_twitter_README.md | 2 +- airflow/models.py | 3 +-- airflow/operators/slack_operator.py | 4 ++-- docs/scheduler.rst | 2 +- tests/jobs.py | 2 -- tests/sensors/test_http_sensor.py | 6 +++--- 6 files changed, 8 insertions(+), 11 deletions(-) diff --git a/airflow/contrib/example_dags/example_twitter_README.md b/airflow/contrib/example_dags/example_twitter_README.md index 28ae9826f1f06..0876bdb4d161a 100644 --- a/airflow/contrib/example_dags/example_twitter_README.md +++ b/airflow/contrib/example_dags/example_twitter_README.md @@ -50,6 +50,6 @@ CREATE TABLE toTwitter_A(id BIGINT, id_str STRING ``` When you review the code for the DAG, you will notice that these tasks are generated using for loop. These two for loops could be combined into one loop. However, in most cases, you will be running different analysis on your incoming incoming and outgoing tweets, and hence they are kept separated in this example. Final step is a running the broker script, brokerapi.py, which will run queries in Hive and store the summarized data to MySQL in our case. To connect to Hive, pyhs2 library is extremely useful and easy to use. To insert data into MySQL from Python, sqlalchemy is also a good one to use. -I hope you find this tutorial useful. If you have question feel free to ask me on [Twitter](https://twitter.com/EkhtiarSyed) or via the live Airflow chatroom room in [Gitter](https://gitter.im/airbnb/airflow).

    +I hope you find this tutorial useful. If you have question feel free to ask me on [Twitter](https://twitter.com/EkhtiarSyed) or via the live Airflow chatroom room in [Gitter](https://gitter.im/apache/incubator-airflow).

    -Ekhtiar Syed Last Update: 8-April-2016 diff --git a/airflow/models.py b/airflow/models.py index 484dd821c4fbb..8e54c9b2a1a9f 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -3799,8 +3799,7 @@ def subdags(self): """ Returns a list of the subdag objects associated to this DAG """ - # Check SubDag for class but don't check class directly, see - # https://github.com/airbnb/airflow/issues/1168 + # Check SubDag for class but don't check class directly from airflow.operators.subdag_operator import SubDagOperator subdag_lst = [] for task in self.tasks: diff --git a/airflow/operators/slack_operator.py b/airflow/operators/slack_operator.py index c5a69456fbe86..3382bc2788dd4 100644 --- a/airflow/operators/slack_operator.py +++ b/airflow/operators/slack_operator.py @@ -115,8 +115,8 @@ def __init__(self, text='No message has been set.\n' 'Here is a cat video instead\n' 'https://www.youtube.com/watch?v=J---aiyznGQ', - icon_url='https://raw.githubusercontent.com' - '/airbnb/airflow/master/airflow/www/static/pin_100.png', + icon_url='https://raw.githubusercontent.com/apache/' + 'incubator-airflow/master/airflow/www/static/pin_100.jpg', attachments=None, *args, **kwargs): self.method = 'chat.postMessage' diff --git a/docs/scheduler.rst b/docs/scheduler.rst index 4f539be2dd6f3..377fdffac2837 100644 --- a/docs/scheduler.rst +++ b/docs/scheduler.rst @@ -114,7 +114,7 @@ interval series. """ Code that goes along with the Airflow tutorial located at: - https://github.com/airbnb/airflow/blob/master/airflow/example_dags/tutorial.py + https://github.com/apache/incubator-airflow/blob/master/airflow/example_dags/tutorial.py """ from airflow import DAG from airflow.operators.bash_operator import BashOperator diff --git a/tests/jobs.py b/tests/jobs.py index 231944de3e8f5..bb729424eca38 100644 --- a/tests/jobs.py +++ b/tests/jobs.py @@ -421,8 +421,6 @@ def test_backfill_ordered_concurrent_execute(self): def test_backfill_pooled_tasks(self): """ Test that queued tasks are executed by BackfillJob - - Test for https://github.com/airbnb/airflow/pull/1225 """ session = settings.Session() pool = Pool(pool='test_backfill_pooled_task_pool', slots=1) diff --git a/tests/sensors/test_http_sensor.py b/tests/sensors/test_http_sensor.py index de9513724415a..5e55aa56e9c49 100644 --- a/tests/sensors/test_http_sensor.py +++ b/tests/sensors/test_http_sensor.py @@ -140,7 +140,7 @@ class FakeSession(object): def __init__(self): self.response = requests.Response() self.response.status_code = 200 - self.response._content = 'airbnb/airflow'.encode('ascii', 'ignore') + self.response._content = 'apache/incubator-airflow'.encode('ascii', 'ignore') def send(self, request, **kwargs): return self.response @@ -178,7 +178,7 @@ def test_get_response_check(self): method='GET', endpoint='/search', data={"client": "ubuntu", "q": "airflow"}, - response_check=lambda response: ("airbnb/airflow" in response.text), + response_check=lambda response: ("apache/incubator-airflow" in response.text), headers={}, dag=self.dag) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) @@ -192,7 +192,7 @@ def test_sensor(self): request_params={"client": "ubuntu", "q": "airflow", 'date': '{{ds}}'}, headers={}, response_check=lambda response: ( - "airbnb/airflow/" + DEFAULT_DATE.strftime('%Y-%m-%d') + "apache/incubator-airflow/" + DEFAULT_DATE.strftime('%Y-%m-%d') in response.text), poke_interval=5, timeout=15, From 3c8f391d3f690b766bb8f4a55c84923bc8ab7a85 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 29 Sep 2018 11:35:26 +0100 Subject: [PATCH 335/808] [AIRFLOW-2238] Update PR tool to remove outdated info (#3978) --- dev/README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dev/README.md b/dev/README.md index 9f16b9f8fc032..fe3900e37795d 100755 --- a/dev/README.md +++ b/dev/README.md @@ -23,7 +23,7 @@ under the License. The `airflow-pr` tool interactively guides committers through the process of merging GitHub PRs into Airflow and closing associated JIRA issues. -It is very important that PRs reference a JIRA issue. The preferred way to do that is for the PR title to begin with [AIRFLOW-XX]. However, the PR tool can recognize and parse many other JIRA issue formats in the title and will offer to correct them if possible. +It is very important that PRs reference a JIRA issue. The preferred way to do that is for the PR title to begin with [AIRFLOW-XXX]. However, the PR tool can recognize and parse many other JIRA issue formats in the title and will offer to correct them if possible. __Please note:__ this tool will restore your current branch when it finishes, but you will lose any uncommitted changes. Make sure you commit any changes you wish to keep before proceeding. @@ -76,14 +76,12 @@ pip install click jira #### git Remotes tl;dr run `airflow-pr setup_git_remotes` before using the tool for the first time. -Before using the merge tool, users need to make sure their git remotes are configured. By default, the tool assumes a setup like the one below, where the github repo remote is named `github` and the Apache repo remote is named `apache`. If users have other remote names, they can be supplied by setting environment variables `GITHUB_REMOTE_NAME` and `APACHE_REMOTE_NAME`, respectively. +Before using the merge tool, users need to make sure their git remotes are configured. By default, the tool assumes a setup like the one below, where the github repo remote is named `github`. If users have other remote names, they can be supplied by setting environment variables `GITHUB_REMOTE_NAME`. Users can configure this automatically by running `airflow-pr setup_git_remotes`. ```bash $ git remote -v -apache https://git-wip-us.apache.org/repos/asf/incubator-airflow.git (fetch) -apache https://git-wip-us.apache.org/repos/asf/incubator-airflow.git (push) github https://github.com/apache/incubator-airflow.git (fetch) github https://github.com/apache/incubator-airflow.git (push) origin https://github.com//airflow (fetch) From efaf90ed81562efd2adb61b81d5a61f588494a01 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Thu, 24 May 2018 16:05:37 +0100 Subject: [PATCH 336/808] [AIRFLOW-2238] Flake8 fixes on dev/airflow-pr --- dev/airflow-pr | 75 +++++++++++++++++--------------------------------- 1 file changed, 26 insertions(+), 49 deletions(-) diff --git a/dev/airflow-pr b/dev/airflow-pr index 867d3298498bf..c374956c1054c 100755 --- a/dev/airflow-pr +++ b/dev/airflow-pr @@ -54,7 +54,8 @@ except ImportError: try: import keyring except ImportError: - print("Could not find the keyring library. Run 'sudo pip install keyring' to install.") + print("Could not find the keyring library. " + "Run 'sudo pip install keyring' to install.") sys.exit(-1) # Location of your Airflow git development area @@ -66,10 +67,12 @@ AIRFLOW_GIT_LOCATION = os.environ.get( GITHUB_REMOTE_NAME = os.environ.get("GITHUB_REMOTE_NAME", "github") # Remote name which points to Apache git APACHE_REMOTE_NAME = os.environ.get("APACHE_REMOTE_NAME", "apache") -# OAuth key used for issuing requests against the GitHub API. If this is not defined, then requests -# will be unauthenticated. You should only need to configure this if you find yourself regularly -# exceeding your IP's unauthenticated request rate limit. You can create an OAuth key at -# https://github.com/settings/tokens. This tool only requires the "public_repo" scope. +# OAuth key used for issuing requests against the GitHub API. If this is not +# defined, then requests will be unauthenticated. You should only need to +# configure this if you find yourself regularly exceeding your IP's +# unauthenticated request rate limit. You can create an OAuth key at +# https://github.com/settings/tokens. This tool only requires the "public_repo" +# scope. GITHUB_OAUTH_KEY = os.environ.get("GITHUB_OAUTH_KEY") GITHUB_BASE = "https://github.com/apache/incubator-airflow/pull" @@ -205,7 +208,8 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): except Exception as e: msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e continue_maybe(msg) - msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + msg = ("Okay, please fix any conflicts and 'git add' conflicting files... " + + "Finished?") continue_maybe(msg) had_conflicts = True @@ -216,7 +220,6 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): if pr_commits: all_text += ' '.join(c['commit']['message'] for c in pr_commits) all_jira_refs = standardize_jira_ref(all_text, only_jira=True) - all_jira_issues = re.findall("AIRFLOW-[0-9]{1,6}", all_jira_refs) merge_message_flags = [] @@ -315,7 +318,6 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): if primary_author == "": primary_author = distinct_authors[0] - authors = "\n".join(["Author: %s" % a for a in distinct_authors]) merge_message_flags.append(u'--author="{}"'.format(primary_author)) else: @@ -327,7 +329,7 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): # reflow commit message seen_first_line = False for i in range(1, len(merge_message_flags)): - if merge_message_flags[i-1] == '-m': + if merge_message_flags[i - 1] == '-m': # let the first line be as long as the user wants if not seen_first_line: if '\n\n' in merge_message_flags[i]: @@ -376,7 +378,7 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): run_cmd(['git', 'commit'] + commit_flags, echo_cmd=False) if local: - msg ='\n' + reflow(""" + msg = '\n' + reflow(""" The PR has been merged locally in branch {}. You may leave this program running while you work on it. When you are finished, press any key to delete the PR branch and @@ -390,8 +392,8 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): else: continue_maybe( '\n\nThe local merge is complete ({}).\n'.format( - target_branch_name) - + click.style( + target_branch_name) + + click.style( 'Push to Apache ({})?'.format(APACHE_REMOTE_NAME), 'red')) try: @@ -459,7 +461,7 @@ def fix_version_from_branch(branch, versions): if branch == "master": return versions[0] else: - #TODO adopt a release scheme with branches. Spark uses branch-XX. + # TODO adopt a release scheme with branches. Spark uses branch-XX. branch_ver = branch.replace("branch-", "") versions = list(filter( lambda x: x.name.startswith(branch_ver), versions)) @@ -692,18 +694,19 @@ def resolve_jira_issue(comment=None, jira_id=None, merge_branches=None): action["id"], fixVersions=jira_fix_versions, comment=comment or None, - resolution = {'id': resolution.raw['id']}) + resolution={'id': resolution.raw['id']}) click.echo("Successfully resolved {id}{fv}!".format( id=jira_id, fv=' with fix versions={}'.format(fix_versions) if fix_versions else '' - )) + )) def standardize_jira_ref(text, only_jira=False): """ Standardize the [AIRFLOW-XXXXX] [MODULE] prefix - Converts "[AIRFLOW-XXX][mllib] Issue", "[MLLib] AIRFLOW-XXX. Issue" or "AIRFLOW XXX [MLLIB]: Issue" to "[AIRFLOW-XXX][MLLIB] Issue" + Converts "[AIRFLOW-XXX][mllib] Issue", "[MLLib] AIRFLOW-XXX. Issue" or + "AIRFLOW XXX [MLLIB]: Issue" to "[AIRFLOW-XXX][MLLIB] Issue" >>> standardize_jira_ref("[AIRFLOW-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful") '[AIRFLOW-5821][SQL] ParquetRelation2 CTAS should check if delete is successful' @@ -725,7 +728,7 @@ def standardize_jira_ref(text, only_jira=False): 'Additional information for users building from source code' >>> standardize_jira_ref('AIRFLOW 35 AIRFLOW--36 AIRFLOW 37 test', only_jira=True) '[AIRFLOW-35][AIRFLOW-36][AIRFLOW-37]' - """ + """ # noqa jira_refs = [] components = [] @@ -770,8 +773,8 @@ def standardize_jira_ref(text, only_jira=False): # Assemble full text (JIRA ref(s), module(s), remaining text) clean_text = ''.join(unique(jira_refs)).strip() if not only_jira: - clean_text += ( - ''.join(unique(components)).strip() + " " + text.strip()) + clean_text += ( + ''.join(unique(components)).strip() + " " + text.strip()) # Replace multiple spaces with a single space, e.g. if no jira refs # and/or components were included @@ -847,7 +850,6 @@ def main(pr_num, local=False): click.echo('Working with pull request {}'.format(pr_num)) pr = get_json("{}/pulls/{}".format(GITHUB_API_BASE, pr_num)) - pr_events = get_json("{}/issues/{}/events".format(GITHUB_API_BASE, pr_num)) url = pr["url"] @@ -875,33 +877,6 @@ def main(pr_num, local=False): base_ref = pr["head"]["ref"] pr_repo_desc = "%s/%s" % (user_login, base_ref) - # Merged pull requests are either closed or merged by asfgit - merge_commits = [ - e for e in pr_events - if e["actor"]["login"] == GITHUB_USER and - (e["event"] == "closed" or e["event"] == "merged")] - - if merge_commits and False: - merge_hash = merge_commits[0]["commit_id"] - message = get_json( - "%s/commits/%s" % (GITHUB_API_BASE, merge_hash) - )["commit"]["message"] - - continue_maybe( - "Pull request %s has already been merged. " - "Do you want to backport?" % pr_num) - commit_is_downloaded = run_cmd( - ['git', 'rev-parse', '--quiet', '--verify', - "%s^{commit}" % merge_hash]) != "" - if not commit_is_downloaded: - fail( - "Couldn't find any merge commit for #%s, " - "you may need to update HEAD." % pr_num) - - click.echo("Found commit %s:\n%s" % (merge_hash, message)) - cherry_pick(pr_num, merge_hash, latest_branch) - sys.exit(0) - if not bool(pr["mergeable"]): msg = ('Pull request {} is not mergeable in its current form.\n' 'Continue anyway? (experts only!)'.format(pr_num)) @@ -1059,7 +1034,8 @@ def setup_git_remotes(): error = False try: - run_cmd('git remote add apache https://git-wip-us.apache.org/repos/asf/incubator-airflow.git') + run_cmd('git remote add apache ' + 'https://git-wip-us.apache.org/repos/asf/incubator-airflow.git') except: click.echo(click.style(reflow( '>>ERROR: Could not create apache remote. If it already exists, ' @@ -1072,7 +1048,8 @@ def setup_git_remotes(): '>>ERROR: Could not create github remote. If it already exists, ' 'run `git remote remove github` to delete it.', fg='red'))) error = True - click.echo('Done setting up git remotes. Run git remote -v to see them.') + if not error: + click.echo('Done setting up git remotes. Run git remote -v to see them.') if __name__ == "__main__": From 7ecbdcb64d0cec678cff66b3aca0d587f337ca51 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Thu, 24 May 2018 16:20:16 +0100 Subject: [PATCH 337/808] [AIRFLOW-2238] Update PR tool to push directly to Github --- dev/airflow-pr | 46 +++++++++++++--------------------------------- 1 file changed, 13 insertions(+), 33 deletions(-) diff --git a/dev/airflow-pr b/dev/airflow-pr index c374956c1054c..80dc22ef38504 100755 --- a/dev/airflow-pr +++ b/dev/airflow-pr @@ -65,8 +65,6 @@ AIRFLOW_GIT_LOCATION = os.environ.get( # Remote name which points to the Gihub site GITHUB_REMOTE_NAME = os.environ.get("GITHUB_REMOTE_NAME", "github") -# Remote name which points to Apache git -APACHE_REMOTE_NAME = os.environ.get("APACHE_REMOTE_NAME", "apache") # OAuth key used for issuing requests against the GitHub API. If this is not # defined, then requests will be unauthenticated. You should only need to # configure this if you find yourself regularly exceeding your IP's @@ -175,7 +173,7 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num) target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper()) run_cmd("git fetch %s pull/%s/head:%s" % (GITHUB_REMOTE_NAME, pr_num, pr_branch_name)) - run_cmd("git fetch %s %s:%s" % (APACHE_REMOTE_NAME, target_ref, target_branch_name)) + run_cmd("git fetch %s %s:%s" % (GITHUB_REMOTE_NAME, target_ref, target_branch_name)) run_cmd("git checkout %s" % target_branch_name) had_conflicts = False @@ -394,11 +392,11 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc, local): '\n\nThe local merge is complete ({}).\n'.format( target_branch_name) + click.style( - 'Push to Apache ({})?'.format(APACHE_REMOTE_NAME), 'red')) + 'Push to Gitbox ({})?'.format(GITHUB_REMOTE_NAME), 'red')) try: run_cmd('git push %s %s:%s' % ( - APACHE_REMOTE_NAME, target_branch_name, target_ref)) + GITHUB_REMOTE_NAME, target_branch_name, target_ref)) except Exception as e: clean_up() fail("Exception while pushing: %s" % e) @@ -421,7 +419,7 @@ def cherry_pick(pr_num, merge_hash, default_branch): BRANCH_PREFIX, pr_num, pick_ref.upper()) run_cmd("git fetch %s %s:%s" % ( - APACHE_REMOTE_NAME, pick_ref, pick_branch_name)) + GITHUB_REMOTE_NAME, pick_ref, pick_branch_name)) run_cmd("git checkout %s" % pick_branch_name) try: @@ -437,12 +435,12 @@ def cherry_pick(pr_num, merge_hash, default_branch): continue_maybe(msg) continue_maybe("Pick complete (local ref %s). Push to %s?" % ( - pick_branch_name, APACHE_REMOTE_NAME)) + pick_branch_name, GITHUB_REMOTE_NAME)) try: run_cmd( 'git push %s %s:%s' % ( - APACHE_REMOTE_NAME, pick_branch_name, pick_ref)) + GITHUB_REMOTE_NAME, pick_branch_name, pick_ref)) except Exception as e: clean_up() fail("Exception while pushing: %s" % e) @@ -809,9 +807,6 @@ def main(pr_num, local=False): - GITHUB_REMOTE_NAME GitHub remote name (defaults to "github") - - APACHE_REMOTE_NAME - Apache git remote name (defaults to "apache") - - JIRA_USERNAME ASF JIRA username for automatically closing JIRA issues. Users will be prompted if it is not set. @@ -936,11 +931,10 @@ def cli(): This tool should be used by Airflow committers to test PRs, merge them into the master branch, and close related JIRA issues. - Before you begin, make sure you have created the 'apache' and 'github' git - remotes. You can use the "setup_git_remotes" command to do this - automatically. If you do not want to use these remote names, you can tell - the PR tool by setting the appropriate environment variables. For more - information, run: + Before you begin, make sure you have created the 'github' git remote. You + can use the "setup_git_remotes" command to do this automatically. If you do + not want to use these remote names, you can tell the PR tool by setting the + appropriate environment variable. For more information, run: airflow-pr merge --help """ @@ -961,11 +955,10 @@ def cli(): def merge(pr_num): """ Utility for creating well-formed pull request merges and pushing them - to Apache, as well as closing JIRA issues. + to Gitbox (a.k.a. GitHub), as well as closing JIRA issues. This tool assumes you already have a local Airflow git folder and that you - have added remotes corresponding to both (i) the github apache Airflow - mirror and (ii) the apache git repo. + have added remote corresponding to the github apache Airflow repo. To configure the tool, set the following env vars: - AIRFLOW_GIT @@ -975,9 +968,6 @@ def merge(pr_num): - GITHUB_REMOTE_NAME GitHub remote name (defaults to "github") - - APACHE_REMOTE_NAME - Apache git remote name (defaults to "apache") - - JIRA_USERNAME ASF JIRA username for automatically closing JIRA issues. Users will be prompted if it is not set. @@ -1020,11 +1010,9 @@ def setup_git_remotes(): click.echo(reflow(""" This command will create git remotes to mirror the following structure. If you do not want to use these names, you must set the - GITHUB_REMOTE_NAME and APACHE_REMOTE_NAME environment variables: + GITHUB_REMOTE_NAME environment variable: git remote -v - apache https://git-wip-us.apache.org/repos/asf/incubator-airflow.git (fetch) - apache https://git-wip-us.apache.org/repos/asf/incubator-airflow.git (push) github https://github.com/apache/incubator-airflow.git (fetch) github https://github.com/apache/incubator-airflow.git (push) @@ -1033,14 +1021,6 @@ def setup_git_remotes(): continue_maybe('Do you want to continue?') error = False - try: - run_cmd('git remote add apache ' - 'https://git-wip-us.apache.org/repos/asf/incubator-airflow.git') - except: - click.echo(click.style(reflow( - '>>ERROR: Could not create apache remote. If it already exists, ' - 'run `git remote remove apache` to delete it.', fg='red'))) - error = True try: run_cmd('git remote add github https://github.com/apache/incubator-airflow.git') except: From 03746b9a0880f13b00beda4a92cb48b781fd390b Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Tue, 4 Sep 2018 08:34:20 +0100 Subject: [PATCH 338/808] [AIRFLOW-208] Add badge to show supported Python versions (#3839) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c1893cfba8013..036e695a64fc5 100644 --- a/README.md +++ b/README.md @@ -24,6 +24,7 @@ under the License. [![Coverage Status](https://img.shields.io/codecov/c/github/apache/incubator-airflow/master.svg)](https://codecov.io/github/apache/incubator-airflow?branch=master) [![Documentation Status](https://readthedocs.org/projects/airflow/badge/?version=latest)](https://airflow.readthedocs.io/en/latest/?badge=latest) [![License](http://img.shields.io/:license-Apache%202-blue.svg)](http://www.apache.org/licenses/LICENSE-2.0.txt) +[![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) [![Join the chat at https://gitter.im/apache/incubator-airflow](https://badges.gitter.im/apache/incubator-airflow.svg)](https://gitter.im/apache/incubator-airflow?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) _NOTE: The transition from 1.8.0 (or before) to 1.8.1 (or after) requires uninstalling Airflow before installing the new version. The package name was changed from `airflow` to `apache-airflow` as of version 1.8.1._ From 183191a28c1c1211eaab6832724b6a7f2f1066ba Mon Sep 17 00:00:00 2001 From: r39132 Date: Thu, 6 Sep 2018 11:57:15 -0700 Subject: [PATCH 339/808] [AIRFLOW-3028] Update Text & Images in Readme.md --- README.md | 26 ++++++++++++++------------ docs/img/code.png | Bin 218215 -> 285906 bytes docs/img/context.png | Bin 238360 -> 149588 bytes docs/img/dags.png | Bin 115068 -> 217069 bytes docs/img/duration.png | Bin 254497 -> 250846 bytes docs/img/gantt.png | Bin 213845 -> 121795 bytes docs/img/graph.png | Bin 333294 -> 152205 bytes docs/img/tree.png | Bin 163147 -> 128024 bytes 8 files changed, 14 insertions(+), 12 deletions(-) diff --git a/README.md b/README.md index 036e695a64fc5..4c22be6d8cf05 100644 --- a/README.md +++ b/README.md @@ -27,20 +27,14 @@ under the License. [![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) [![Join the chat at https://gitter.im/apache/incubator-airflow](https://badges.gitter.im/apache/incubator-airflow.svg)](https://gitter.im/apache/incubator-airflow?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) -_NOTE: The transition from 1.8.0 (or before) to 1.8.1 (or after) requires uninstalling Airflow before installing the new version. The package name was changed from `airflow` to `apache-airflow` as of version 1.8.1._ +_NOTE: The transition from 1.8.0 (or before) to 1.8.1 (or after) requires uninstalling Apache Airflow before installing the new version. The package name was changed from `airflow` to `apache-airflow` as of version 1.8.1._ -Airflow is a platform to programmatically author, schedule, and monitor -workflows. +Apache Airflow (or simply Airflow) is a platform to programmatically author, schedule, and monitor workflows. When workflows are defined as code, they become more maintainable, versionable, testable, and collaborative. -Use Airflow to author workflows as directed acyclic graphs (DAGs) of tasks. -The Airflow scheduler executes your tasks on an array of workers while -following the specified dependencies. Rich command line utilities make -performing complex surgeries on DAGs a snap. The rich user interface -makes it easy to visualize pipelines running in production, -monitor progress, and troubleshoot issues when needed. +Use Airflow to author workflows as directed acyclic graphs (DAGs) of tasks. The Airflow scheduler executes your tasks on an array of workers while following the specified dependencies. Rich command line utilities make performing complex surgeries on DAGs a snap. The rich user interface makes it easy to visualize pipelines running in production, monitor progress, and troubleshoot issues when needed. ## Getting started Please visit the Airflow Platform documentation (latest **stable** release) for help with [installing Airflow](https://airflow.incubator.apache.org/installation.html), getting a [quick start](https://airflow.incubator.apache.org/start.html), or a more complete [tutorial](https://airflow.incubator.apache.org/tutorial.html). @@ -69,7 +63,7 @@ unit of work and continuity. - **Dynamic**: Airflow pipelines are configuration as code (Python), allowing for dynamic pipeline generation. This allows for writing code that instantiates pipelines dynamically. - **Extensible**: Easily define your own operators, executors and extend the library so that it fits the level of abstraction that suits your environment. - **Elegant**: Airflow pipelines are lean and explicit. Parameterizing your scripts is built into the core of Airflow using the powerful **Jinja** templating engine. -- **Scalable**: Airflow has a modular architecture and uses a message queue to orchestrate an arbitrary number of workers. Airflow is ready to scale to infinity. +- **Scalable**: Airflow has a modular architecture and uses a message queue to orchestrate an arbitrary number of workers. ## User Interface @@ -91,9 +85,9 @@ unit of work and continuity. - **Code View**: Quick way to view source code of a DAG. ![](/docs/img/code.png) -## Who uses Airflow? +## Who uses Apache Airflow? -As the Airflow community grows, we'd like to keep track of who is using +As the Apache Airflow community grows, we'd like to keep track of who is using the platform. Please send a PR with your company name and @githubhandle if you may. @@ -277,6 +271,14 @@ Currently **officially** using Airflow: 1. [Zymergen](https://www.zymergen.com/) 1. [99](https://99taxis.com) [[@fbenevides](https://github.com/fbenevides), [@gustavoamigo](https://github.com/gustavoamigo) & [@mmmaia](https://github.com/mmmaia)] +## Who Maintains Apache Airflow? + +Airflow is the work of the [community](https://github.com/apache/incubator-airflow/graphs/contributors), +but the [core committers/maintainers](https://people.apache.org/committers-by-project.html#airflow) +are responsible for reviewing and merging PRs as well as steering conversation around new feature requests. +If you would like to become a maintainer, please review the Apache Airflow +[committer requirements](https://cwiki.apache.org/confluence/display/AIRFLOW/Committers). + ## Links diff --git a/docs/img/code.png b/docs/img/code.png index ac49291f1c3332bc62d192b18b70777c8f523d07..60e8f58a3382148b66a587d3685945a0a4b393ea 100644 GIT binary patch literal 285906 zcma&M1yq|smoSXGmf~8Vr4)AyPzn?(K#LW3DK5d?DQ+!pg%(=eEx5ZB2_eA?p-6%U zNPw^J?tbs?fByY;-<r?5Qc??_mYU< z??}M>gg6!!*%Jq4<#(@?l{wydy4g86+hSp<$9>5n)Xf;93xf;2Qdj>N{_b1U8r~nh zfF#|Y5lN|d4_=cID-9*{KWn4YRjK2w+=ui7HJwRT0@gb9L*=^r40`oB2)RO zUbu!IPHZ-|rz$G5739g0H&oEtRGuR-Hg^N|H=nd5<$zJN?W3x^1{~Bob@q2fJ6G%J zd0ec80}myinX|C*Oc?679y(Bqmc3Bmdz+FRUWI*K5ysb=bwblE4)7Sg!&Q5L=S0AO zRjey0CympfU_)c6>GeBxO-h++Jc7(Aav*HJ~;NFL%icDoh>g{;Xk z-P{5O;om^6q7kM`h{>pyy8d9C&qJd0l1G)#yZOeFQy3cj%|eHx+s(BD!OV7c`U zW6=IoQP_2~TmLhSoJ~sev1h+c0L|!8dc@Uceekt=J@4sI_)|qmC2HoHoFO*)%1_v> z*l8R@c(_ke`!gxZ!udHqB3%xndmhSv4I>}qpg9g(O||n=Y3NiVDmp2q!r;dda&v9t z>~sO%VO^NNY>F>u9Zf~ijjGv1@}vTl@J7RXzMsDDPlKfQTK}NNh|3WU=i=s7A$eGJ zybaXI9ud$C#C=t9GND8bb)BeRoySfQqk83Iw{PsU{9f#F1Gr7Nyr^9@XdlLfe#$2< zc}|}w;`J1Cj%*wdDE{yPK!pmw`|6e-et<0p!=HW_hVE-O{d$Qu_|C-`TDg-d)PT$Q z*eT<7OkeS3u~q4o?3SwjPo&e=Pkh1OjFhf(dSx-tIw=l?s74#)r!|Uj( zsD%jSPH5+Xi7)<=yfY3MSqZ|<}lMR`Se+~o<@(qheM|Cq#-Sr5qX z9n)a#A-n&)H6O6;!3pDHW%=3m{Of=%B~CGx-qrU#9OX7&!@k7)Fy>=|I}W@=ToDfJ zZ(+Q3(?M;TLl&~xW5yQvtAf6yOLpCV0N zzWo+PQ3G}dP{TL;uS|k?&7zUL%*Pp5v>5EQ*wDwbL^pWG5%l4z+0RG0-|~E>dPW2# ziX|%FXr0&aQ{#TXsPd*n4s)sf$#Bqc@cSV7VAr6&70(HEeUS?1pLFL}3f~wW9U>a=2KXmwc6+Yv=rnANVp574a8Sj~O`qI!4q5{$J?`(MP-|hMRRAHxa zobt1xw!{y~1&)30iq9R19R(||yHtafOXv7M-ZkkODY!6JC;LSB6kRGs*I3o|Yj0Le z>ipDqc-LNas{N(Pv4*KwR<-$E^NWkH{l|-D5)HZ0MyX83E;UBYE0)d7j<@?>-f9;N z7bX|q_vIF6ek#s$I2e44`sgkZFVW(dKkKJ5vR8J_AsgGBYW7A^6}iOiXyn)+-Y7mH z?(5Xzbm`#mvHjy?hpE|$CE1e2pz6Y* zMw3`}6piWr=@9!WHx^fZi8U*jE5O3aA^oGaO#}FMA@9(_fcSt{vqPRkQBIoSXNYrS zKqGY521p6)zBW1&+eHfS3#1Fs8HyVL8W-wq4C#!PjMth=46hA$3>OVdo1Yj38rU1V z)gkJXTufZ9Bo`&KoZlNu8HXPcK&R)T<_b%rOPyY>*Oji|FVikBLgnl1m(^D2AyO+n zEY^}wyeG#+?1zKA-MvRWet5TS|JanCt{<(3oHS=7(Tx0D`BNb_C9~jP2+TO+qc^56 zoOqd)SK|eK3RbOgsWFGaRtH@JVF9oy*gOmjT0fpW|8!xn@3EAz!MGQcinw9sV z8yr&G({-ZDd1BxqjdmW#Ao7gV%XY%xz-R}R$)t+|)H zFUMZOHpB+um=Vb0#p6=qcT=8{tN)S(xRZ||30?G4e zmzXq(Qi#kMLn3W^x9au{g12EKQ;fc%5lqzV?`cnje?8G=fv`lsOiD1K^_E%z7%{9e z;941oR9Yd1jD~lI?nDZu6?}p&u6R6>JsP*C2j{oThmnb)Syb6|fKAO3y%O=hv?_Uaw}ZD2mu@<;JaMnDBgg+o1sQJWU*f zm+fhc<#^gu?CV}G;5tM#{cb97D1wL_GWmqc@&yM6u=9Ls%sh)Ym@%C?+Ppp<_tz2& z5*-&Sw7s#Z89&@apYUCEMQ|v6+0fJXTPmwUK|~4YVRHYo$9dz8nsWYuc*dsUg>l{0oa&-HhLABrttJw;D_Lqu)M=fIOD!p zl$hXjQ%<|h?tK4j|GM2~qjckn9rKKtFuE^4OvZSZ6F3iVnG!4b0H_TNS;nxEJ|%6S z#~A*8r)zfZV|m)YYZbY5J(LncjQ-xO@?FKm(>e`RqLFYi6ft!4CO@mn__X=6aX|Ov z*Q%1kGwtbLXFalaM%M=$4T-i#6RVkSCQ2rcOgw_WuaC3t?C(VHr0AoXA0G0G=wxwJFmuj7gb*Afh%w;Q8i%m9-CVc$P$IA);( zDbvcs&vSDnhW>hI{a&a3OIx$WLB`6r6|o+FjDD+zn=zzC2pSCq`3xRO8^X;#wCaSy zPXn6FjMf{Se$^ze_JQ&)myaL~S@o^X?8`i>C=9A;y%w3WyzbMx-CG#BO9*B!VSncG zoXJk|T;h}&l&hU9OPL}ETRL0~+M`4(wzM*|>ax$tHejCad_0-!vyL9QF;=ba8WBrT zbj(}vfe%K}e9%f(goHfXT7b-dE!WB?mIfYH%|J4NW^jrkl=Qk^-_eT!Rd7P=o!9Nz zJCJeUz+==K4c0=|sRC=c6thcE5a#vqm-!EdAB0R2x|jUrPj04|GhDq^f3F4k@WGcV zq4=$Yh{>KeZN>pwM`lC8w`hs`hCA?@U~3H*WdyrP5X1fPG}m+~eahn$MPGWwukt_r z9M78&IrEK~g|mgpaGE0D9;z;eJ%Q^tx3w2S)n;rM=shwanC7O#JkFfF+wA5gYU;98 z&<%QHjYjrN1WO%vpGP8MS=r?k3a>w7{`~1ZpZPZP#k{IVJT&Plz*pGU{oNp4X_EkwgGwJ5Q#s)JkmiiP{5H%$Qg);*SMw>LU3;(%O;g;r8 z8VIXbV^^YVhj|41&U)T656zOf$=Saf2V!@1Lm(Cw-Q#~e*srv?kFl_@iyU+fybLtn z%38a*2wK{>S=kEuxw!v@#=-*l$^PxS*m_xV__;W{0%iT=IsXkp_HX|m!$O=K{|4ga zB+qG}@s2~;&C`}cLQq^#m{Wm-gM$O$X=5j=t@83e(Eq;4bAI&la+ehn^7Zu<^c53y z^RyQdk&%%R5*8H_6&3gkAprDu^|JI6a0PPxJCXlON5vLs?djm|<>2PZ@ejI|R&L&2 z@|>LiVD!)5zsqUs=kTALT!H_g)?WpM{xKsYA}B2MPuhP`0sk14edpk3>ujLn;9~0v z{L4c@R9Hk5@NWqJzp4LZ`Cm{C{}WY2SVsK6qW+hu|3C!@{X>QSqSC+H>)%HI8ccx% zAoNe;E0CyWUC;dW2aAJ>mhRsc=O51b+i?B;d;0I~-#*UA?&kKWCsH1+G zt`Y`4^u8fD=D~l!f&GF~J2LEi?alLeFKT>zKF=-4^SMU0%xwgJ=uC~T6+PvXFJf%q;6H3L=g39~ z{;%TvJAg1YK0=n_Uy%Oa!08_q_ug{Oo>TpAy8d78IKHg7|EhTssQy}q`eVSSe^J-x z;h^OI%jbVX=~X3q>fS>nF0b-Gy68X29fL?=k*VS0cpqn8{m*XwCk~13h7>srS5!2W zy4M}R#fr^{nJ}xkh2DX(A3tSKt^F7=p9glgon8^N69`EB-z5HzklIMc7Zh~YYQ-wB z40ty?F=hpap0IG+vVAn6FJ(96YJpVMq{~Sxwis1<)FqcKkk(7;<+Pefzo{$#|8J7?E z4Ja+K5K#!asK)I;;rq1$9DYrlUspg*+$-g;o6Bq6d_GGKn~)by|2pe9p0XLD>|qMzceW_L@`8NwFGCsFu>U-6(5 zu>^GwV(PX850{AE+~jut}W zwBFW{J&wSk9=7Ou$H51K5GzRVVyVX4Zp1D;pZ-%A&WQ;QdX1I)nL_2-%Hb0+Z#+G- zS!~kIejS?vZupl9uIhVllhSBpm8-o45}@@l6~|W$%=u4m{ExwlFX;X!9MpwJy6R%7 z8i-mvJ1!?D&3}#J&~Gl!;*Sn1TV2)(A;8h(w$yOvc)>7Se)@=fK=d6ifQpBqf}M{g zO3LYcFu$;sv)J5x_t06TXuP=Kz-!mB6;5OscvB5ZyV)r(giq8eLJ(_)4MuBlFwKGg z|0vNvQ){m|en>&D{MjPDu=ETOkqVmpjmJTS4AB4uXbPh?n&%oJJN6ffLtxIGn6+?gZ5V15sKC;vMWP6nlQ zH?@_jvG%|xjc&JJBl(lzBwRw<>ynY3U@O`)MOLi-1YApuQ0-$0{7-UGP-si<&qm%W zXQWndnsT9cra%#x%RBF1OL-=V&yASePo>>2`GHnnr*ZP7sW(DNU@_z4KH zSr`J4L;R+B?(*s&RD}of7KpnvnLPMU9FN>JmI7Xc7}E*V31> zper7HhC9fXgbE&I-(FwR=%p}|Gw7mc?zF&V^fdao#c~);YDj#*d z>}>%uam%OS{(@es*TJEI0=z*2i1tzckh_7k#_8TB%o>Mch{8xb(iWY6!s=x|QrJ}< zd3M5r7|~r3|L?!~iga~Oi$1q&bfA|ZAuH+dV3~ zy$Pz#GJIcYOWg9iNz>T8qdEg>-C8u04wXGZb%PHIE<7U7_SKK{<%*-v+mW z{pDd4c{eL4qtaDWmMhA%2<-;t(L_AwSsyyt2x$%80?izDBbu#1c8EaG*-Cp?OSK}X z@bFe8NPZ0rJL>pjxU1UYI`QI#GBBiTt@5fm)b734pNx~XMwd4WN&G3Ima2{NT&1=c z+dk33{1#aKMwYpvJ6tY9taUp+{=yx|*Aip@C4eB5eiD~E5iQi<_oyn=k%%Hl0Xw(? zuY?iqqtyZ=fKD3kg#e_99C)-n?qh-V^%_!m+@~k1{ua(_q>1KH#cizWm6NP4ZX`vx zKTXshIin|5pPGI9#0A>v)%t-RMi)x?lwFXMA6Q_DE9>D`9=f3?itUJScealp_8YO+ zw0z@X^5i2K87G6VXg^S>7I!(O9v3yr`IX^)x$Mk#4v&T!@83P~;YgzVA;r5A6-gbC zzrmvVf}rUGnpR|AM@D%g==3*J5I2Q&hwqEqc-quqslz@R9uZ`$OZLsSW${yE)~i3F zRr-1hyLng6Wvaqjv$2{_EDld4)cN0!)9Bv+YJrZW)w1WC^)cEUzd@neSAChM?LvB8 zF(*84<=js~`d*cvv@Ru48*#~a+fV)@H0?e9h&fLO8~{5&Uy5XiAwm&k z$@ieE3TWiv6)n1F(1BN&1c}i-@bScc<8>{%XQY5LGvushl^cA3xfR@<j3=Yo2jDBA;?T2uDI0 zb}i463jw}?yJrMId9Hokfp3v&^+I!%*%F(0CH||;bGB)k1eaqs8tVLpvZ&d$CEq8E zQzV4Nrn(aLN9N9J@uq!~+f@cLqM+>&chjV2KlD2uQt;&WtBKS<|0(JZC-JXp^idan zMtFzmKsdyVPb^!a?QzNY`R(X{d>XIm1R?|!hb$3&T$4?&WuZ>kUWH|afd%M!Gc8?P z-T0?*McJepuuk%)1M!Z&8v}8ECLg-AmyR-8oz*J3);bAr-H-Y@L!=u*G^(sJTeuO+ z?eGcAPsuYxh*$oVwaI00ravgK`byE*$MS>cWgX-+zhP23lU**Zkaoh;d9eLpY)#86 zCTFHmL=Mn8!aiW_l2@X-IR3QXSzO^n+-?LD<{^io-}8n~Oy>{7TTPLNZ;&piyX!Uo zHzI+k?jm;5HQ({_qR@-p)fP}teFMX}f9EAWQsbcP#_EQ=5c9*?Fgg0FMq$-g z#Prqv_Pal@NK&R^e zqM!=QC?Fj=^fy2JML}r>=?n47^ir_DNAYYV6p)dKWx4C)D$Jmd?6a?C+`xX%c&^Ko z>fLAZ!gv^&hp5dsaX|Mc44mXkRsr?ifiLg^~9~8NQkaw_SIF*<1@t6zxowk7Pb*YeC!aDIs^$o53f6 z!BhD4tCx@NU(QTV>-03A%3+*nW{&*%Q^YG{<(L+S)RT4d+R-Bb#PwO;%T~uk=swt! z4ZR~6B6Gzm^cN^)GwERWie$-Mc;Tp#j-$SgAS%rKA^bF7ukB%I+&Bu8`ggZ= zFgt#+_sehpCY;CA=adNC_WW}=DU6QJBzJcy4L~R}0Btv>+&*mXu=|u+J(n$`*iUHK zoa_^&2Pybo3^Fd;V_RrW2vq@Z7ER}h`3bSF_%AzAf(FSlphokvBtgcQ)*ElNAKv&= zCYNmNs5Lqgf7sh8q2>Leu*-oapy{*BckqMBn!$vena`!xuHeB{r=bjn zW5o{V0&pqzL)lpC9QD#7M!|LvBitb7)+r3PGx|a3$#@*AYEoY7egqTH&jYx{*ht5Nsp7ndPl6au1jPrn#Djk{ z?*d9CyANvA4wGNm@#9rG3iMG6)5)^Sefe}X|H|*U(oJ}0*r|~a0k|^X|3w!GZL-P3 zlmMtArJ0`MA25cQ-1qq%!iytQu7Ym&7X~}J6iX)Bc&E#nY7E#i2+bpB`}?lIB|CmT zIpr6(=0;L|)-2-g>Gb=c&?|@Vrs`Vr3cRvY8xTqbC&trj)VAtaaO#p;YUyXVK3UNb zHzd03B=%0b!mf$RkC-D`qI!kGXHjJtkgO{WV7Nh7fDvkT}1k5J~D=ep^K zMkY3U2K|W5-8WTqp+GTIBe|sXSU9mHRbO(R@U)agBz0A=Dz*SkfR_p>q7tqjoN^Jn zeZ5H=cA2HuJLN5?#sGzfcKXN{UWZhcPsO;%rM!oGFIyy}Q%@5bFE%}#>^ZZ0Yw`U4 zw-k$s#TNv?tN^?p80yrBY`k(+eo!d})#Z-`F`0$C->fRwKgv9W!LPA)u&7xq7XzgU zQeDFH8sAPTME4=tT= z@4V*bKdrZJOH1Q0*#`!(k=g(Vi0d|)^+b0f)`J~lrC0C6zf2erOxN@(^G#Q}1UYv%@#zEwdh@PF5tC}+}dF&cUuohu+aawHT? zDir?{e7WT?E=HSJ031P{k}cGs+6tf)$Bor9ZDWPolPgak8qY4;H2fe==aT1HAG;a#Q-OBsN#-Iq%)w z*btwH?T8tFS9gleH5xHoZQ#2{bT`-C4n&voo8)$*u*?o-amdHNyi_%l^TKfibvv>q zwgYDONMO*;M^?!!1NYUC!O2p9tNrFLpIy65;@?e8&zQqKf;%{Jt_QQftjj^B&D%75 zRx#sRJ%fR(a>bGZC-^}Ka1S)`JwM0(yPr6c?6Qg7<{=sL&raSPIO~38G17fHQL-w| zlm2u_-Q6OR@0f1h!p zkXr0wS*6ZoqD*Gb1q=_*Hjz*=+Z_6cKxIJ+@|He2kCAIO^2 ztE*&p<}ydB`0r}XJ_TOzM0CEPcQt67mT$}CMeU?h*^JM}ICibV{wZ?-AJLwx-;K*2 zhYa0}%gnS8e25ub9^mr;vI(M{x~E+wNt+~4H0xoUnXTc-d+?v*MaD#Xjwt%p)ht_F zgQ_h6_XYp>m#0x&Mcqji?5nZQzDK4Fr*Zc2GCI?f0j|$`L>hOf@c35}6C=sY4mn$F zQ%{JAFb+aEuLrr3T)6W~2l;H6bmd64-8!a;#{xnD#L9V+8_cy7n!+rV_u~eKS~k7C zw(o9N7(#CsmxpN7rar9oF?WGB*a#4730*~<`bRJux4jS((}^F?D^w%lxmMXn11Da zOe=9L61ii)HY*y7+%l864wXLI?qxv|sRWJO^L!&+Y>Wvrk z=NSyvr){iHJaC~tj9>Cw-r^y0P(hv0*}$me_mDqCZ~u%?(iaBFYUD-_KT^6fMty|a zxN=Qvitw}axN=F>t4=hs8aD*{SW*nCs{+)XU*sqVfzgU?xRgPeu$Ij(K4Y%jzVAt% z`{R@~y8vRrlCxqv@_Md2yu!SmPtXO%tS{$LnZ$Y<+S_|bZbyZdC# z4p*2{rftKNK(kp!behF#JDDR2xj7KD3Kko@gb&HNBZW6f5qvS3Ep=0qTg-v zB*46L?emcry4*x3u@X*(@bplT7XoP1LV3PSZDM{kp6)u;r@(tlS=#0!;OWqw?AXWo z9_VMe7ppUde7%~_S2bt?Qkw0^z{gjz)gW1ZM)iAM6{iV&u9L>3J$R`T@5yhKi zbw$?8cNDc<5YZU?Z;_1Ric@grA*YRU|EX&Evj@h99gwZ?u6Ow}tKrekC!JHiV@?HH zf@aYtlYFqp*)m({%xJEBFc>5gId4GS6I;CwGTNB>?l?y^$+|!o$S6Kz z1wH9}TW#Ql2&>FL7!FprUl|J(+`dq*#A%Iy5%CGKD&PegOwMP}o;kG(eC=dAg1Po7 z)`Dx9U|Fi^k2I6rM`#y4NNbf-5+LK#BO=q6QH{Aliv?1T1pkmd6P+yVv{h^A4OSl# zWtcyE&9fe7^<=wiCZE>kZqtl!n$O+s@8<{i0c_HZ?<}g@oPX44QG-7*I0l?)|#%Y&>-HbMgmT=phS@|Zypk-ZQb}R>xcS~*uIUhHBbA2`NbQ=3<3v8j`=UI_0 zFY0YUoQ@=G6K(kJ+35j30(Oj@6jjaVQ*rOkl9Lu2ZLF5FP=44%c{>d(|VFj|^ z){JaDKn*Q2>ph3??L!UY(yWY<&248PqXN3S3sTs4V zMXAdH>KKP3f9;m>^FL&J2O>n>1w6Np0oQT~ditk$h0Vfv&xYd<7XeC!k2{ly8pcsZ zWVBksKbmPelPY*WJ#MXow%DE&n@xYJcg0{`$8(iei3Z50^pSF$g`*%^AzUq?s(p}nDHo~E;^vXe*NNAyJJ%?>Mb+_g`Jj@K3PG3r;&cx zprep>n11bS%>UeI0xe^S@Wzo9&)}>% zoJDf(?uo$VwR2o-FNfIGz(0Y4ej#TO?xAg#0x(F4$lCUg3Da|ooSLbije&ym<}*0y z7`njd(aou&be|B;`Vgz?{OR}W_V>%zfI1=i)?a0h<=$qajVN}1Oj_3h)c!ykQjpbc z^JC(CQ*~mrtO^Bj(B&`Ng=H57O1#a7q$E95D^}(m6os$1Hoi7YKhK7BAItb&W+#pIQsovjV37gB6kJ*j9GI0X$P({s*cTl z_wrA&FzK%5KWcZZfwFfYlzkpVa)0>YzSzuE3qcq%c#v)(yiyS3zn=5Cs&{pTD<@^@ z#4HvG{S`Fhw!x9?WET=C<@a`XSIXv2yK3u(v@u|9I#zNhra;R18OZ@r`Y1vmD~n8x zx?7F%FzIo zmx&thkcJaNK(k|Hqa$g4d4uqv6u3R%%K@b*8H&O!w$A;Ok?zamphCIt7TDzN%xhN!N@etF=GB z4rsDhw$F^p^OPXBhY`o50{e1Dht(zb5Iuf^3&TU%RmB`r7c2dX`l;q=l06Wl3)MG$ zRF?_HR%^TnFe_VUlaA7orgbEV^gi%%0;47x&JLH_q{*JrhZ`G9jnMM9aPe!XQ4g@L9*y1N&n#@XK? zN0}pSw1}Z;Zp({fX8!^?>I_Kz1vQ1;@tXabkW{VHkZW|HW;h60e}a<|?&LpptFfyr zbiL_4$vQwFBP3VeE5A$ULpAdictkmA854Xd6Y^}ASLMP7T^)L-hvS=9)TP?q@x%Dr zBDn<(m#gO+=4YU_Ait^EW2ZSYd-}jeu-!oNm z8gIU}5W!=Sk0R4oI!)Yk7-E@p?^2@0i_TZM@pVeGlot=$brqu5CW793rieA{(CS|( z2wwwvQrFMHgzu0I@yk9>DVhxoT^|LzhJ)}iD0kQBP%)0i$N7(D5V3cYGO&b=LPp>* za`B*%&51T#0}=y$NwYdvCi8&Nq$%it%h~SxcTi9;n!GJtFNbHjqxG&5X$dKEQ?eyc z$S}B-|3v#Uki9d*t%?YapfmHinHiZ@FJkfkRa>({ve#eRZ zh^P%F9T<=@xR$H2=fMnO)5x#+QN}ql-WFkZy)WQluM9zr^aiu=Ddma4I*@s^ehJ}V zmKG1j>m5MI)niN1a<(564&;a5>~MMYMN=7PC`Zb#l`M{=6%dRC2e+&5)ezmSl&qwT zT6J41FFzR;n`ef~{!_T_!D9aSt6Xb+d!!#6q-pr(b$E;3L4E3k#$y5?|&;quKQ-&ju=$cSYN}5BMn9hMSxMH=JUQY#A^Tb{3eWt-rU@AWXVRRbj6V zM^C7YCl+|kCL!f+%lXnQsn%SK!{mnpmxp@pLWCZZo4QynSCzHRzSl7$ukqQ$j@IuV z^7yTpD4Yj8+>Et~B^IL^P74Lr9{g!~6~9aWfK*JG!MWzQ2h%Xxj}XB=GPspPyDm3U ztr6-GHsVqKs<8m<23%OMrS0rO!FpHOjAPBUU3&aemI71EwdM4GW*?Cj_1x@SJl;p# zNXSGqi4+=jFon9zFNhqD^*y@WnItQ?)CRyQY>NSzaGJcdAqk4|YDVNnJD8wg#|P6j zCUhDS9OSVbquWDfEv+#rp3D@ycktcslMXd(v4L1&wGKa}b}gH>4f=H6>>M)GCvw@{ zI`cFAyglV!?Bw)|i{}ON=?>U0184@>c!m<2VnfP5cQ8R+ejE;eDIk5ihz3rWr|{XA z?^FKwb=&)gbz3@`);Bl_JAaR}h50Dksx=?hdP@$%3IaXo&xnnyzIV|2uv+c3`kv&O zerPJ;MjlH>saiN886r44r+2y%3vVTc??=Pd%XXXoIIe#0xAp7!VDAH~j#7jC3i&0e zwc{U5SM+wf*=>T>=q%TdmTphu+7*|rP?9th35_&;#ViNzD`+1Ddsy`1*UXcb@--e@ zz?g3yh3hc-?q2vH4_pyX?9h4hc58!9(WQYtwjLCumR+PTfFjK%Wxf~VnqQ}bC;;yC zq)RU2hQ|C8paM%ox(+{?54{<@h9*14LyFSINBbeN@#dcW5N9?KeC;GmV={hlNoc@- ziL&z+X>(B4H1A|P%fVl6b|+n8Z8O&G zpJn+9tDqP`!dS~Xe!HNr8GFX88OqyWO zSggN$!aFd7iWrgdYJ)e|4Itnf0)_bCmRu(&x_x z=1Cl3YqO=MeF;_HyI&QPM;f#Dx5QTwJVh`ZRIqcO>CRB7ef32U4vJD*D$qE_6iUIa zBvAfB{$R7MK;USdaW;V~Kzj~TO(~LF`qA~HEDrh9u?t8RNUFyDX0x4|WFVx0B_ z27Z$-(}z5oWimEsHv|(9J)HfUNEM2WR59jRmzvS&c{02Yu!W*c{~z=EFj%E0l@f7G z%=+KM7ASp{1+AZ3`HSYrsq4b@m;e@AiDb*hscqLcKWF=w?XU%ut?X-vTITA8Dt>=J ztR#%=)=kw;jU3L;^|tFMU^bKMPVa<0i+lv38M-!Ekrd8iY~z&0)Gm3YGREN_HiF=) zj8-z1d;`Z52H^)WkZD3Zt-Z8YaqVl0t-bPb-3~dfE##6CeYY6l)!}0B>U7#V|Hz~% z`h+(rPk6Z}9=k63b&J7qOKxB8)V$@|+M~H>JVdwU{OcG!HG`>ki*+S3`wJ9bl~-u+ z0Od4L-xmM$1DT=k$?eYVqavLv-Be4EOBHh))FM@iu?k8gY3%5Jmo7+VTV3JJjoWUN ze>M~W)w+JXZMT&psWBVLZW37@8ci?P z>RmSTbmnaSK}$X4bzY#cxTh?w?uJFoh_Qd!kGkXeX?mLFQ%uGx;R$a~n*n{K3Pp!H zT*M~!g|wWK0X%W$F!CDOjL?BIbkH}sIZ^CiWpB2*)HX-8iaj?C2^WTEW!3iOS`#yj z)L{aXm(TBngL<5dW45wv@eu+aH$t%8XY|aGcWLJQkOu_`DEjz(98}dGVZU|swe5U& zo3!~=8-?%5$hKIm(0idK*C=dnr_DtPW_R|KDO$_}U(!m*>dvcVi>_!cLvwf&0ZP0E zC1wo=ehIW;fGdt4T}|X}7w}!Eu?Ldt1o(LEYAKHP;l}!hkii4n1O8~&{dsi=aM4-t zD?*tR^_<=M&VF=bmdj7;{}Xar|`RTxrB z#-mFz5NytX^b>ZLt%d{Je60zm9_mjUtC{n8KAUn{M4U3pJ^x^xwM9oeVGGI(kb+CJ z?qAQeIFC5e{jDCqkOSQvbQvf}rQ-)_oiV|pYvzed+Z(c=l1hp`yO?a~4O#5~ycv&O zF^1BR!Eruw1=!Q%Lv!b%x7YI9a=$$iTqL=}Uu#MzcHa?WYAC=Z%$29vB2fNK!{&n} z_mke5t~FRi&RItnDM!~*)Sl?$W6gC}gGMA)%-ihvipQ@l)(WJoIcskf$G~S#Nn;Tf zo2fd^t$z3j%dbe>{1E@;D_#SybZ8%rCGzosk;tkk7wR&Ewh5s?ys2Qoatlbu$C6peugb?fN(f4k}?91r2jZj>8NuyS)xbnv@wK^rc(^k#T z0^6JZR_-$iKYo}0LQTCy^0@ys@S(=jUyKhQos;!XJXC%$U};>}HKhc!(t zW}nQpZD5Zl>5qf`$3BhPAUnyTv4`07)AMANJ|`qs!IyxyCyZ@qSjr|Q`TYR3UP~f6 zmF2#$-9zj~_KYpjPeZ6HpjdH2(19tSdsc*YRGK=mG zTJXGgLc4C(=n-hyY^;4ov`|tsd}$Eq11Qy4K|`1~FP#Vwu3-{cr49U_LNXV@NY~XP zfW2PJulxNQ1x0jfD|6RCc?3nIOW@yyj@;YEb?7OP^2H|E)?tD&3nBgTr`}ll>Oox} z3HP2lz_`zR28!(*5EiOwLl_l97F+$RLxRuD+_T#vDimP;}p!%f9#6qhHj7L|E7r`SkwB2{O>hVl#sd34h zqzVX*FY0!W;$hAp7&+~PxQ*s4?_CAarh$;K3tM^K&||3{lOC0N+BvD96uso9aUU2N z)Qt>w)f>4vYcqWS!mKCAKv^dPl zLXFusqA=L7X#bnHiu*FEnH%|(-Swjb>x;28V?D{s^QHfu-<>Ml1Ee0F&9Vbz|^^XmQSW(QEW9kAC}#7W18w%I|T zYXE{~Ug!@#?376Z+ZS1kjKv?G-f+V?goX}QDwl`5JjR5@t?3;PTCM@0)0GPZsS!_Q zPqk<0N$=Xz0USvrjI;L^x`yCzoj0Ktb{$!tiq3yH0I^p!wR8$V#(Yga(Hw;Ht;b(iN z7SMOn9dNtoM6dihYw+9~(u7Hd{q~7DTDorL$V{Mw`M)!GX(lbgx|q_UGM3)#zd^71 z?km~D%7=+1v+R*k{ox~0 z63rS)rmms35lfTSlJzCl2ei7|NVWxA+;%&iOt~ZBtI0a+cD1dTy1(V-n=o#aJ>`y^ zrE}{{$<3#cDVKo-S1!!(xQE7kS(6g4h#jb!u04S{sc+<8O!q5cZKKVj`&bsOj&}39 zPp~OG3EhG%+H`I^Gd8u!*^ZH-n5%5s*mdEyBzq#qv#eN;rnjKh5leivj)x0d1PEiu zlTPHf95!Eqb=-=VTkUsUZG1THM~VIn^A*%^d!g7sF=zq^iY6XdlO$tW-!j=4Y3Hi8 z`t^(q)GfspcNntKRJ5^IoCT>VF&_xf1f9IkAJ|fkIdc;-PC~UJckU<#4O#-b)0Ohv zE`Fi`*C(yaK9sxSSd#0tKr&CN(S^JkZ_RIf(jUfR+6Jl)YhqChOBmqPONys5k$>__ zZ#~(Zs-?|ec@tRbZ{DpVt&h)7k;g>vq0xi8IEBd9Fqcu@hhF4C%7q^CC3Bgjzt)Tz zSngM-4*V!Az@O-E%!V?NBOc_0B5g|TdWE3Ao$IF0D2i`gP~Ol!l1)O$i;$zWTZX~y zfr?j1d%dGzR%3f7UZiW|t;=Rc0Rw48QYWgV&0PFN)diX`BcJjW+^4&!=x+fU7lPQK zX_-=-2FCf47J?w6iU;1%Ug~5?Qn*OnDZ)6dA85!keG!z4WOp()O;j?a>M16(CcO$_ z`3#g$8-_H*3uwl;uhHd^6hBckskKV*J~o++Ft<|J8g1PTNQCT9kD^@(C4 ztJ9oUq9~kYiawf4`xP2^LZ7S9QO=0Dw{9=F2>re_8)a*QvLMI{5G;t`;2!4=SH$|= zmOXSkB#vohCb8UQp(0rGR2m)V#n)ozxJ9;hJ>U5_d!d|E^Z1P?)R%|j%SbBYH0Tdg z`7h|p(7P{iYxpIlqO#*xs;GR8Vc-1IK_g7<4m}E%+`~41U3h7T8G|FI4y=Viqi=ur zmhp6U$S3}`$f0J5s`H!)f%OY5PmA*eQTp6$=nN8`GBmblEGIvUKjW33L9kGsuw>?~ zyZf}&xzawCx%z&AE_#$^{whHKX*3v(@{(;{ZeNhO^Fz3utf_DloXTO(M0O`AwqLvE z-3k1+H~!xp<7|@Ud^ioLXjTr5!oDJ-TecB+^!3MxlcA(J+RTgwVFyn47x3mDKvMJn zK0S)r|C?EB##SU69&`>b?jC8mFVt9`tZXbe)w~#(;4ILr67xk&1jwQYQLOHRI1M!N@yvT>&dj{eJd^Y1{Q9nI|JYaeuG&@0YOPhP?rjiYGmG6M zU01Y+B9GS-qhJx?5-$Ri>n6@ER|!-#FhO>zTKqm_)G~_1nQT9Wc|=Q*!ty=yT~3XZ zriL41WE^BcYHt~q3GSWmi1tf~>K0p1?~22mjPi#vh>sg2kIUll<8}I1x-gCO?K*5G zv&Dzl&8f=R^t#=l>0RI$4_z6m0*Sr;0?&L&*D2F!jQbU=8FnbAa4b*T9+{|+2vGx# zAjT*U)s#re@4BN~LY9O)sgjD0#xo|E_vh^J%wtG?A=+ewMALgc7L13}5##moeqj3w z-iSSMUkS-bVbChs5Zu6H_;rEFaK@Vn-+fEpW=aI&q-Sfs)M)>j8Zvf%ReRltxWX>b zK+wSxYzh5WxNe`fk^*yBI+?(IuO|u<#F!J7G`!Za!o`@~-a9ykb z`{9puGIS7Ap0|)`&;8twZ*YcAU(%mQA09Z zKSp0k<|`A#y}ypoQazNw;Gw59OIlZ%`G}mxpNBXzS%P3d-DiH2Y5U?_wNRS&*rw~} zR+xR;*Epj`i_GF_Pu?FqrFq>_dt6mh33fe7No~95fCC2#^oz#wL1P%?OHz6yWT+>Y zwLEN&f9j*+)*)vHgUp>%TU{Ety!sGeMJg4eGNa{O5Tn^QYq$r$wzy9GTMA-k z&;9hn>mCm^Xpc{ePTZdB-NswG09`Rf%+imCk_gz@wTM*{W6@b=Q}Qtk^z)sx{*rwL zK{7D{?)^-+j@)nUkRS3jDuoS(qV3J=5ipBU+sm$S^E z>5>2(=#!2*nVVq!^Y@)z9$x~STKbF3IH6>Q-2(C=VPGGP^Fa6n!qp}hZEvs10z=LI zJ@kQ%{u+_=A=AT^(9L=*1mVT|;W^&*cgyyC(UwPz8^e+GU}iC2beWL?K^Zn1r={c! zwqHShmGb_B#82uSsGodWUNApAk6U3x#o>1FJlam^eTOYKyro~o7{Pg#4?Hz};-V@%%>W^c=0ew7)nMK!8(bio<{d*tZ0q3Sb zD>XKCCVhw<{v`3F(sc|fs@PT6r)hfUAF1ADR-->M()P-nZt!x}n;gU;HkFXSU`cO* z=`H(p?<~af8LpOcLKj=A)zXT5*zYeqFIVnv7{07kM;ht!P?bfm)`;^2W_0ajx*0A~ z@%X3pwEuc&Y2fjNvr6&GwpqF#OQNFWCTt+o_FjNtkueRVqdsQFAA_tI!#?GOvDsF3 z^N{aF!gR%b94twpe8A$u_%C<+Ypnussny_iYA?Y79+jD=fTjnd+d`|+P^&+ew)eiz z=i_5TUW;n2>!8`ZalKO%fAR|*s<-JLN$anU>uiAtXP@KlAz$4OSiV(MmVkjk*R<`~ zJMu_^djd5V^*Jk&dJLv{Qk^t^23HN>LaG{?vLAIXT2U`sQ@z{YaGkm3)eTU{5`Umq>Bm$I`pSnK`XXcl7tiucbvXOMojTygHqVa8cJv{tT&^zq>pWM%s zA67FjiqMA=Fj=e7k+hNkGEy2NL(!nR%hK`RR2W0|I+{(hhfv?EE&*|S8@hOS*p1I3 zr&;%kJUl}8*2PxLr`eEVEI&e?`gE(_&}L~C?zRl3((PU zv@d*Iw3l4$8PB@H$-IGBg5_#GlUc92Z9~&)=SRwkq-kfSJ%aSl4Q^|Y>1gQv`;DJ8 z&y&(ojV+ddP}O5P!qqDgzuO-jZktrT)GKLIhP)%<-7#oyYxkTB?=jntl7JptoRx4^ zqa>5s8Cc%4IrC=Ew?4zvYFfx_tBKBVOgRmU6}OwlxL_Ft&S~3DLNP_1m<;RW0|4;p zT^*C$H~1f&)I}DC{X$WG!bP3?=E|Trmd1?v=9N$2$3X)6ys0n`R|3;R zLke21rY+PA`qci|?JByhWG~lYCbVnAM+oJ~Kw`?alK zHswpFV43acNHM(PbjAk+B}1!{cROX51BDQiWh@uAL9%0s3OB~rXmNANE5vHO(5 zI4$lDMbQzJZZPKYeKf6sIjNZiR4x{@t#LMwQ9spPuN7JW2{h)lH=AsL5SaY9lIzh@ zbGP5n+G`Gc*0h(xQ$K9?=pHKYNlbuI%}S>EFD9FDBB>z(_G2VdD@J-6oIg`(9vPlt zep6+7-$<8wRXW=E=9;@@^T~VM5Bs!><}-|npulczr{Ek-Yv`u5{=lKE8LMH0_`>u^ z`F$A^L)7Jv6LeBspx$_Hesyx;e%)lzmzpx(SGD%^3yGpOHh;$gFg~y~UKm7%@29-> zQ*cq@^K?k7Sz8=NaF_W~2Mvoi1@O5^e;upi*>-%14w~%4e$a#oVvlo`R8_mJX-!?iZcTHta2 zfJ(^B>1>z>Uw^4Qymr3B<|o^%>R6L>#=y2K#0?PiR_FOq!|Gj;nyl)^1B6-0K~q^g zqy>2+S>`%-ZHwAp2l}?#9INvg!>>ndW#BH?6HP&e1yv$QElfBBoUi1bF^sQKGBSqE zNMSmCZ_y?%`OAD8e`@WkZ&{$wjne7LZ}y+9B&SXGzFMe3_l%}*{P$L}43nA+8%uhc z?Cp|H7RLk&y3TG@c4_p5gd?5yj~Ov8^=FS3uzA2LnwHwhrq``I1=n}R5p*F6Q3b8K zg7&Y5u&HL6uPvF2PVQU(93JRThjD)d&P72ieS+$7h~X}8n{N?!CzqQmN-TI;`581+ z9b~^muGj=4_ZVy$Gs@t8%ucCz zL63&uL__cx!zz-iGks;k%~n5QfB2=d-p$xI%FLERVeQWWH(9URs0m;|y?g@G_u@Q- zpD&(!SPY4}|J2sj%4L){#NHKfmi~kTt?QJCdGn#01ZJ3?ee+Ew_lZ&)cn6`&gP*nN z%}D0H@*$Nr8G22<=)^XsU6OunvSu9-Z+k5jZ<-33kuX<+ZB-{BOT9sLZS7V3wvgsv z%}T${F0~bH&62hBs6Evz=noo64IQPxn&JqxngGU@&t1~5izg36wAAL@pgZTExbo(W z;2{gQX0*1FyBLmyaP$j4fX#8xiSgc7GsH%TyPbLQc%Iq4qPE-pZKl*;#Pg*0wT*UF z&S6qRHMM6VIAS!bW+`J=2W7Mx^IH!pk|y_>BV7f{--l#*$DP|R=G{Ozaf)U(yOV4S zR9?XJ-%#o21NU7&;EyfZz-qWXE*L6*?WJCEIo5w1? zZp~H0+{Tc_qL5nsKRvm>gyc5CN~{IfcZM|ML-KvTwt1*clK}O#t+1!HQo&J4tId&y zpz8MO`BKQm!s0c^Xvby$Er}mwy3kL|Z6%@f;#Lc4k1E3+N}f1`eE#OSH)>w z5${S+l8=daz7jpA1KLFd{i_$J{G96mls7Q;w z%b;Ixn{eM*bfGFrTCE7?hMAs^R3LUm7H5TRd#W45w9kVwhehV04YUHS$DQ4d9VH); z^^rH+0W!t@1+GAug7quC+9K&bs;Ma=fZzkcBGwCH)XHi5pL<^a^y>{57C(z4(p~e1 zBL{0yoovPTClqEp?nNT%rY{U1->MPe$<23%i0K{KS-6}N(c0dRGD`^fC}z>MglOgsufU386oIMY$0usOL}hh{nz)u81}GoVz;rQ)sMcEdcS;`?k2#hME@zMqLh~GdtyLrJBGZqW zYlNL;g&a~fWYxSxUMj0IbV44d!+8(u2X`_rf-hD6RNnvfblM-lG6!k+SNH*5)QPbl z14hPwsuB1$SPRDN>V$R344QHx3_l_8eqqemN|!_3zK7UjG#g-)&oElK>wjsN^|?Po z+fbm3JDj!Wa3A&=ocS&kfwEx*EI$uLOhp=fLpgX_Sg-#xJN`40O3d9LFM? zmuJ4eyuHbBg*Kg!F3=ohMpCyA+*ogfB@eVn-S!f|zT{GWqzkJC$HuwsrgEiP4aGdf z-bp!>esT2$>2y;S_WeMnpvn6YI`KOUG6w}u6B-A2#youa_dzp`{svHSf9wk=g@2ls zqp-%}jLE-K9!gqg^=J$uH9_Ablg}1pOy*FYo_5c&+n~-*G4bSzO~j|D?%(dD~L>2VcJMtw%mTlYOo5G<~7{?10u@ zz&&&YUZb*tl4V1eXbtA~Wlj z#}xyFwF`I1_j>v7&;MzWxc?%qf%6hVv`m5icXhj0=?>}WBD|y^;kRZTAQR({j+7Z{ zp5XaCq>=z{iq}h-u``~J{+F2i=?l+S?+}S@S9evK$bPTE{|^(umCf3Dd~|qm^6u|8 z;YthLbh-6s@Z!fky|O=HMarY>TTsipX8@!Bv1UskbhIkV!9L{|FAgDu`RT+Lq}w8#&^}<6D71;^72*~AuK2`kR&cHF5BqnrY@^m@i$*y zfNsCZ6qlZUSI4UkN)Y+YW#&q~6&<9HK9mV4YK7gN#nvkS-tu&1a0kPcwg12_^G~GX zPey!R;ExWvofnwk?)giXl*f&30nkSvRd_cu)N z)dE&C3L*I_L*f*9^SJt&nnlp0_1=>YV{5ETTOQ%)$jJP0jNd>Y3y=wHcGNJ4F8bYKNpH`?D8X4(BkKQ9;s5M0wBBa7j;qT=S65fxwrQ89Fi=on zLT{-dcTj*j;sCYZP_T1(+$bi%CGnxe2`j&v$v~!q8PrrXo)-yxa`$vGbeG9b4*Ten_LS-hMo z*Er1uPyfrs$d5J};YL#M^I~QhI&zV@AQsB=I>uz%WmTN=J9w!ihDkF10eotw#Bz+z)WKFUB zX?Pf|;DP0wb(FpbLiFryRa>lyNpARV%Ua8ddH%9TmgM5V>6Dz*8@4_cu}^+DVW#^j zxrYJM68f+!+)D+t0+N96CHBq&Yz)bkf#y;zQFDTQ3cp7*>*o`Phq5$gLYEg#GEky@ z<2!RcH8T7gy!6N9mtDFmptS^G=a?o{;u2#ekgrtuj$#&{Jj$}4v=XFKOj5M|_?U=! zAu{Ci#?phjWrGk8f@TM5F}_E1|2-2U#k zl&p&##V=PblYO+%*%BKoO>i0S7*0t}dpsR;tE zJk96QX^c%2)7rNx5{qpQ7wXU$wU3N-LU*`7xudI-<$P~dsim9F74)fni4;Sl&P>6Q zLux;PEOzGNv!MN`_|j$@+enSQB8BtuO(F$Dct-4WH=nb1aPMXtIM2pVAss@#LOilw z<1}lmGQ}n(b4OmZQJLQmU^Kk+vQ_~sgiokUz!^bJl|b=sW@jIwCLZiD0sCR~s|uMw z7WN_6ST5{!ext|Yf2?834JcF zvi0LSsLj5R9l&N+m{8n{5W#8Snjk`a(Y?r^2%{{1P3$m%pS z3w=G)&bQMKu9omSF-7HeDi2dTpP@Hb^$;^1QfxnG=)9_F+njCYol>WZ#501SSC%|J zX2zcfz8T&BQ&-^MO=)_V>_Pyl{%o_hFfoxmePwM4=d7gX>P6(pC(7Ns4aqUJ#>)Yn zf~Q_5OW%Gl8igyu3F4rHgT;cwwh*RYQ@d;>Pa#;tmEH3L)$ipi1z@08b~}3J3BU7_SBk$Oz%q-@gK-g zf~n8>!83%)t5*aC7h33Wx`z=XBiWIL@l3ztKN@_$BLJkNnldzYJECvBJ;QO%=WB4r zeNy@GRvWxy~m#E2=@Qt=3E&**l%rO360r| zaBlk<{^f?QdChy3E%Z!5v7iWrwku^orEwOU=sq-iBc;Vk29kM7rkR>!MZ5;Yo$pbX zn+=dpckOb>2!+d(&vvJoZo>9Rn9}`|`V&Us5=E9cD^u$2D&$kFmBTQa*=Bsd)t?l@ z`19gnDC@M2Ce1Sw$sISXa=-G}z5NXFx%v`%q$@F$IgSSmBgCYFFg;K2N*r zD>iy#4-N@yoSSofyg-3jv8=W;gKqZ$M` z_3RazfOgh&S7EcB?@Cm8{w0M#%{}J4}z6EkHmn&UIp=;z?y~m5GgywK7eQyj8 z8(J9_H1%lP{hvRqY_8m?0-r0mXF}1hFBo3M-PPDF$p$W~JL*4X(}?_3g1?xYao@OW zMtosQiUe?lcQVjeRbaLVgtNP0xm)fs3}alc4SN&M>kHa0GEs;1zSJb8p@|b9xNRsE z&@<4WUss{%`L@a>bX8L4bftj4aqHufIox8t^~zStxj6u0rD>}9HP>eRWqm0URX<(< zt+Q^nraakOc!Kno@}S-r(j{IDoBfP=BvYrefO>I1>4X2mo{jtMOPSkHbSpP?`GPh9 z3u$?PcV}ypRx=i&0^BlQGW3uYde4tw9;6Oo4l*uxQj-N>BlaITf_Q!RWQIDA#2QcM zol?s%WPJCib5H*=zV_w%W4(*|nv1h~RvO*25lCR%*9X z1<;nhVKc@+o}9Q)Y>DCqth2=@$Hxl@0J{(LM=PEgCi3b&-nnpN7LWK^w-E65EtlX) zU9SY3Lw{*#YTuQ9TFzRJZcIlaGt-++Nz8GnH=bOIp-Mn=?&R(yx%fm1SC837`k)L% zrpFDACKxUWM4yXX7|vAj5g+ z%f2{pW5L5W&+4&RlSDn-bSEOIcc5;9^fh^%OSt?_ujf}~<5UnnYOJNB%~pu5f>+d) zLO!V5XwFnTv}j%4KM>*7q}ajf&-kELhb~Vd_uO4 zTG=9>xyXg(>NAcbL|aA`kmQTr7{g7L+|oDC=c5*%6c%}m`c;h^Zc)Ju`$~Bpt&1-1 zp6K~E-&XG;rKX{HqII8-I{lYUJ~=F{ziG@^nctAoF0JaF3!Zg&V!4PbY(=X1G4Bl) zQYD4|+eTAR0Cw~voWStQt>LKdlDdU$jYZcN$fYZ;NT=)5X|G*j(HTr+w&&)TdCJVM zV2mvr(WOn=$Xz30I*bH~)Go1G%e=DojogK+iGZ)9PRbFnE=)G16il#S_3^5~*qp&> z#pyMLGV#UXUESCk9FBLv6^W1(uV#!dU&KeMZ$GkxhvOL#i?^VC4m065udoS^%a8jp*+9LR-UWS3OP25v-G!FLp_&2a}VaNN}cn=#pWgTQmll>#|3@HENri5OXyC8 z@~QJ*F@!Cs{kTw0lfeBfGfwP#xi0#e&y~!wxHgMHa!IjKNSf43Iy0ij%6mi0k)XRt zh!JCzHEQqdEIRXuHBJJs_9K`n$KhO%Tp!3!+0GB+~bGJ z`yy(iGmpSMGS`pa$c-JGKSAD*P*oMM)kl|b+LUw|4bL12<*Q}{0M3>lz{8di1wEz% zB-hO&_j!FRmg{FHWu z>*k<22#t(~v=qPmqnJ<3e5-}?3*KO)(GN~SB{b)@b?tdG zeVtfC*`L$*6v}kjm90~hCj6!J#I=@%-@`&0Ygl>Jzr#m#ojM|Uz!+9%#gzL;!fKta z`XD7UV2ZcV`M8L8!!;P_r_h5I-qMJt!nd$UtcMH*h1E^r#XCLqMN(BW!RYE@9FCcZ@@E1+FHE*u*5aL_MBm0MS-V>z` zh^J00`J6lNF90J@HPJR0lx50Jiw#FvHbL|=j%jnj(MNzHcdgD-t8CKcZ|?BUC|``@ zH!Ic7#et9}lVSCJ)P!fg}ULt4A|?Te8>!_VviJn`Z)9l%J7n3 z@e=Bw-^KfkKr=b~TZH{rjr6l2q0+$;Ghg<W(c-vM5W& z(Qi(CODPzs5DuIldmHgP@IYHgX?PfFrtiQpMUOEd%O{#g#;d6oFnUT=U?WOYtlqUS z9!kRRyK9K!l2U%4tJCBnJTS0>CORK@1A{j3xADQf1~;Mc6W;=C{P3n;a)IF{M$R&g z>l0UZ&Fc@t=0L4-aW>6Zct)DBmE0v_`-;JjVzjLIt|H#GZR_c-8unLIw>uFwniGu8Ik zgs3C3|84w3qFHGykdj2gXQsMp4yM(7f@kl-F__ge2=2?Jep`y(pt_AsTVmq0y5Y^7 z@8D9ClcV@vw$RCw`~$u99`CFaQ(u{kupI%mYHeIBbIdY#2B}e8P_=kf`NRsVJssy7 zb9p4B0wfc?{A*By4~|PU_|^2&>!p=dRY&d2V~jkR7-f{D_r`W?ImGz+_V=N#7T^AO zakGihVY7*E(Vw{4NGTDrb7^NgZ7>0|Di&9ablE)JSSv`a3{pc9J|kzVo7<9mA0i+dWR#F znH+i*6+2Fp{Q&rMp)$(O3El%vMW3)=pl{+}?MUAGmCfS+=CEo&UUQxBu1 zUHGaGWFJbC8QG8?X0S|7c1!KfRx=i?-8-gOeSPs3 z7c=bCysX&$L8~au{zMt`0DcgyIqQXx@EopfcT7XaXjggN7mIVab%#$OB zhU039Tx*_Mg58~DsNioM)w8(FcJ)KQ1RNErQI>tI`b9Fl8NWhn#~MX#9;n-wnjIXO zwp6OHk&@EL;t0c^%qSDzOA`t$da78q z9$x1wlTz0_7-^WXhqz`6P3bbp0lIIY7|mYCvtm=`vwT9iJH^^($J+gRw`9*thI*3R zDAY;owqTD-8XJ;k^#UJ%+F3SAkLfXXjLlWwP1L)2X_v1~*-#IAA96OM-Kq?u-E9ic zK4@I9<1_L6oXxvl8FpFpzq$^J*c}g+o(-gl?rVGOwVXw&y?7De{DXJzO>>}^sg?Kb zLWec2H`9)YZ7bB6Nq*BoVp-tskfP~B_Z{zI4B_ zL%$})e9(5yb7yVJ_F2HZ_q`krVlNzFVy`I?Uf3@FQ)9^n9#2|HSjWgTWgB3W6Rd(b zf1e<^@NNx2gZUue%CGZd%vnfpzc64o=yg*=fr5jK;gN_snb6dT^(x2O1#fH|+-=w; zEvz<;AOHB1nQ z7@0a+#o=AglxZ~ip8C+7J>^PDh2JAjC8q7Qyb}m7*}mPZF}NcuFRY`kc9AoO9{XOK z5fE%;Qs|CI(`~Zz9zcg-j=k}~hu|R=P<<@k2__$J^R@jp0${z|xKwAd$_hXQaNuLv z^^B}1F2%w}{@h?F!Gkr4clPH(v4jd~sSm{$Iga4{(#M}w9YNGv>)=;HxHJ|-B38_4 zaS<2V1fupxbMUv6gn$=gT=`o8M?_h3lI{HdDcOs|d8$cn^nl$Hle*ez>HHIA68qD( zkHtAzJG}#~@#(-h04W1mfVE1`n&jhy!|y$ZeOiE%;5$hHgXqJmrU&B<4m`hnnW~lR zbU%C}0%QVS=?90v1-eH*qfMFv`7|pNy%sYMe1ztX#5-`L_S!^hxsVbcUd^R25 zgb^QJ%mN&MG-l>(4>pKM?5O4M)Gaa zf$zaRBE&+_U1-c84#vUBSl*3ZO*bnx>^&hlJno{&PNU5?TJNldr`F=P^0$o{?`8+AQLOB#63CaK5$9;{ zW4gpdo;?DmX`mIuXM@eGnDt{HdIE0!Puk=yncb-qlTsj)MaKK6hiOYj3BJTPw(Oxp zjsP)p3un#9qLlkI zZ7>N0u{W>Lp{2gvwZqR*x5ay&Svb)1bDOd}_W$PT{>=}jc$RH|g$xmB4rLKtH|afS zSu(DTpf}%xejeHDBi=lCNmTm^tlGI5sZ-}c?@ZriCi{+%*K_X$!S$S5!p%1_EM#Lx zTDfzWiWn$j)ry_>N>5wIU7+M-hQb4Nuk+`-J$ut_4M{o-PuFrrKh7O4G5!cgX7)FJ zCa&p{<&==h_lEj`uc_K#Y`ZljmczuQ>BHU90!E0l5rie}1zDzVW3UluJx>Jbe&rfb z;pTL;RlmbNU*Bao%_*0n<_N+cu#fJQ5mXO06gKeTO{8{;B5w1E!bNm}A6XF!@Xr$P z+0_D7gtX>MC$}eU0Vx0AI=|kn75m_g;a5GW&tj>IqP3VV#J5tP-Z=d_v@dD116V#u zR?Z-;oPBiG^V0oReCXijQhc2itltDN=OAlmpu}qfR@5k}aG6lgmrQ>EK=oUuZ(gx- z3_N^BY~!&i3B}wsdYf!*bT-xS&K>E&hST?b%70(TXyOUm_gY$OC&&2_0 zo14N{%CJxNSXxuAJMFT*H_VQs2gT=`(`MyUN0%l%M4Oka>UgWu=%^Z|b3zxE@|i6z z%b_LtBMZw{QHKCLu3Q3YXcdu}3jj6+ulo)WH1ewwNA_&?B#d&hqMMRx;Qo4{ z*rHLkIlcDQar!t?(y8Ioh?cssYZT~klwaiX%tejT!A)4b)~^k=y}V#EPr)7;24OUP z8XSyiGsqt|f~d%Vf{(sQ3gm}VLTk{a0rPyOC*PcQqiE}hokJV7YdRMIbi|z1&lKL# zT|y<#Lxk0#FwDO{#q{N|>^*dJepb|`Q##1L_)`oujfkPGa6MoQpM_jFPg1wF(Xn&h1!KTpDIly6I*;f*mUyuRl zM^Vk`?4&gA|4Kl@NJdrL09Hs(TXEKviZx2ym>04+9998rK+y#T8GMz&hudSucTtl< zOcc&I`8-so60d7Wn*XS84<}KZ2uXJ06%$4pRG{tSyMea*Yr<0}>doAP#;>sQjlI|djg@~H1lGK}=c&jT=({r}sQo%)Av`fXOg z4;S_b4`Ehsopu7V*rggic5y6CDLF029`>SqIkx%nAfAC!S&+mFvdif_U^hlZz;vT93P-L!K+ zR*62Kk&RqYy9zu5~yl#E-Mypb# zzHGS3_Ak?pMcHk0CwmW=;b}=0duCse=8MWsf)tXS*bgjIr-c54J2V*O-pB5^?FoFf zHkt+9CVdIBn~cl&KQilm8x%-&ws}G4^!>H{wM!cVsa{qJ_>gS9MoXZZ^6 zvW`f}d4-$5g&R@LP3Y}iAu zFe2Gu0kc-6cb_zGG{2wlxmyC*8b;_&s?yD+*N(7S8BBcUxDtN|R_M;{tT|XTg8x{- z{~c5C0{6}o5Y5NzFER!sw_(jx`if`jA2RB>F@;m(%tyRFB+Yti%FsRaj%iQ(S4UH- z1^P!NHr=CQ9zzsFvp;UxMIZEO(J0b+_|)i_bN=(h3m&fhO|9Zr!zW@TdV}<@tMu+2 z4QF*+c&CloGwc+OA9iRrEZHM|Lg{{e8U*xJ+x6CYS*sL-2xvsHo6On2pCaD9^6etS zmiEP;8r+Wl=sfh@Uf%TM0IM#24rBRy&&PK;r!EHO2P>=Y8k=@Zh%x_ENLDx7ud&*6 zj8vH&$~wXI!_=d0^Gyco5JEPH>DPGq`4;OfPN|oSU6k{bv3FtnfzJLzvBz~L-PIRr zZEyC{DlIZNZBa+OZlcn!mydd)Ri3?4Msq7@{-iA}oagnxcSUaCWsFu=vh5+s{OLH3 zGA?R?%vTE7FZC+TAzEZag_YoyVs*L|p@)EWL_q<85>Vrc`1@cFk3HVps**Y>(cLoc zg^jK1cIucKEXgeVn0C31h+(#pPIv=y_63__!2`lS-d$GBgm_~xl)SYmI`v*ByY$0+ zVh_8N?_KrgVEb4}=^?y-v%^-?>C%uk>U;E7=r5|q!;T)7JRkwSb0^gAh(~ReKo(p^ zj!^Ujt2_y$cmZFcqXVNpyij@&kLgTQ+*7QbJlAcab7lR?r&x65Yvj|Y@I!9a?zDo? zuEl7aRWp(lDH1z&#n$AX&cwc!rX-U0-mc9%_K*&E*<v}x&9(M9Sih9Z%x)4wFnsfrWIF|fmLv@;6;9LkW%?`U0L4IuN{}WOd6Dsj!5XZ z9vmzM4%8eCEZZkl7iFba#z8F1nnECtdZgB4|wMhZ+Kd8G=q$H5Qg#{jf; z4kNMB#~qx-u%x}~cUk&DB;&%g6)V2G{9v5$rG=Ji$ax`fq;uIos=-K!}DuDw%P#UPE;)L*`SxKGC^N(5+7r3m6CUo29a$ zLQiLohq*jsiq`rkrHhx8?J%9~ZspVqooQbn~8pXaQ6`Kx@65N-+b!( z3!R3Zw|S9_f~0SI*yR-khEopaVA%D+$1|f2CTcI!OE$gnkCiP}IepQjA~Z0xU^ApjwyhBrr!WQ8yo7M)VS^}wzhSS~h9kJV$=x4X3o>!zzV zl4>c4?XeJbaC4QrI^d=u`y5m8*^p^fNPW1gk>%0jK!lL=1s_PnQ}eKc%j}Bn68u`f z7`;+d{pN}wa#T^^Q?j6D4#%cq()MlCX{6wu_@|hq^<5+NmT)I|GQMuYynT?>A*)5f z(ASs`Dkah@mkWZ2tXUGS1S{z*H+x2hOy<)MT|nC|QC~OA9HuC#WEeh&Dl89HjHhSr z;tFGdgc9PtKJ2>Jr|{O@l8b-$jYfBoA+e+CzM#O5E1}q=){a3N z`o^Y&eIT&|pju3?L*=`o(1UW^;6z2r)3G;7mYmKJ;p}Xrw_Ymm8F&x;dy_c{P6CVDshGy5X_jKJFGw+9EuwrmxLvaqj@ZEOf z_4>${7lx#7Qic@`&GQ%PCJRFz@bVfM7S^t*qzP-LP1dMnr9q>c$Ufd$rC-N3~%x)zQ(2xyb^oofi5les!v$* zE?E&L{h1Lrw%k^Pek6km%L7M+@`7Ck}|d)%f@`&ip*fh-c57 zwkdvAyE#PB3S5jw0?(hiL>;*^%kWXMbYY-{<%+!Fj8-!EDQj@;+Y;Iks`}Rn=Kl!{ zA9A7Rk


    (ggB|ieQ%V6lk6EJjX;LnQsJ6_S|{oiwaZwnzErg2C2gn3v<$SYCupMr?v6?!}OuS-gZ(`r6adP=Eh>%u3G3W%05bh|?M8 zHHN^h2W(=TXid<>N6#edm+}DmRmJJbvzoQucqaHtAVm<_G=Pj7%|247x`QK7n zbc*@1FQsPkzfbH!vb)_z&7Yz(RR2w8{xP8do6x&$W}Ez?nb-f)=5FJtR{}`Ya@*p! zvcnv^G*J(1DUB59nw%;!q(7`vSS=yF4G9&U;8w(w`cF*C{(d*@lth|Xu7AAlZ#O?2 z(F^>L08K$hi|_fd~@h=YJcz#9f!~mDKLq6X>G<>yZBbB!%ML&@_U@mE%E+w7s+FSLK{j)UxzxUY>|DYIa=k>o$C`s*)*~Y)UYcHStcYFG8&nW((t`sbY zf0rkhyDs}<((l^SR$~4uCH_9$hY@!~&cSt(`PT`_=8@lZ>9%&)ekhLP&kK5FHU1D5 z#cPm4JdCd*ZGbWOBsxZENu-I*-CNQzZw{}C&pR5*GO{LL>fJIaH~tK;qnCLHKo8L_ zU0QycRv~n}m^xBH=sUfgTsW;<8;+^Gq^+LKiiK>E7Q9C;trNj>yPU`)=Ln)OP?)QD z^*u`>`nq5^$7%1YwQ(6glPy&U-89ZhhrALg@8aR$BItHGyli3p`h*;Ps(K~3T zveL2M$ID=AhE&`0C-8_apPu#j?LPH89Di}5-%OOEuqKoeroRx?0*V!tIdY)ni1T3~j`gmnuA|Oraf`BN!MVj<3B|t!m^rlp)0t!g) zz4wkthmg=a2`vN&Np8+N@64Pz<9lcBhx;YJJUcu4S$nPjUc0Sg&T(V>N};X$xVFQ{ zFP7NlF(Be(E+lmA9MeZy;tSs-)wmX43;!U*%QVr-*%+?*YbgC{0cg8>3d~?*Nyy=II+HFUg z@EH$yU$cqllAr$O>#GD7a3U{^JAryCfl>XEW=Em~y#4!JRX_?|EEw`zM86UwC{10$ zPKNB)uZPTs6=+}Y`t>B+<0o>Sns@o#bA&|P`4Hiy^Ts_BwJp8wL|S&#lBb?O_f7qe z0oLTNN%+~*ax5oqos>(WIXo=ZvK4lx3ra;7JQ=4dLe+DhSCLxl^{M#|$^E#7eQ{iTyB_`&W!(OKVzv&awAK}!2C^`9 zKD=m`Y$fp5fTM5d7lfYj=Sq&gUfn0!$x)~QmbX2+x6kKV44O1KXN0t*RQ0evs5U4c z3t3}YjKwq@9gW%5QRf)+k66>|&upHzdf?0Xn&m!vasWf?Ywb`4QKI=vhTOaE8^` zXD-%@$0`Ea8(qg1?Mw2!b_*fqAFWvoyCc^rXMkbxXb-ls=OpqoC7R8(MjT2lEoj!v z3)iNqnm(o5-O(b25;8uDRQ%=%a2dBiAz%5mJfbTs{tZ z>-Ky6y)Sk(EX?1h(Lp>_zr8EgN>5Fscq5y8Q7RXWW`~{ozG(`!%FVOZGd)fK3p^Y@ zo}D^+U7{VlBwcB!JM3p6#_YE#)r?x-Fsm!VWL!Tqi`A2dv5o~r(mD$r(?K&uXmPwA z#F;m1+uR;y+WAlbTF(XdH1x;JRhzy`5~4&snc7pzTAznqIx@~R(tAI4{Gg8Payjy4 z#wQ%$Ho-b((F~~6ATdzmW+M27-r(mlOSo!~qUaD^(m-py>a<@mESYdJ`k7f1dnT@j zpQ;cHW2<|&-C9J`lq$sRh54CX56_MLVCeYVROMFb$zC{LG$Y5-5Wr>8DvRyyvg}vI zJ4OaoipO54g3|PjPlM?slBEHR|C|Z_v4;E_K~ZlTh~|?xU7k~eUGyCOVyauQIV3o0 zu`oRh1UwqfW*JwW0)S7%QbZjM7w;RvOd zUUqyYw~AxSI;+M-BnYgOP;wMP+H(wUiDLpk9@Qb@KU#Zsdgit)lC;Kp?2ixKp@xDZ z*Bqd^rB%ju69(0t^~|R}<27|rcA=tQVS`4)z6CM{g)ThiVtTy?HGWP` zh`q4Zl^$kJMRneX4Lpr}_#_YCW$kGZsNgV0hvoI-(~`Ib;TeOd1@HP3r`Mk_yB?Io zu92x^!>f=nPq0TB?lBZ($C+6GyG4l7{)vyOXY#|2v!zUXbr`+l4$_e_pdaI1r9UXp zVfh>G=^?i=Xn<&x%ZnQ+wc<#EI6cQ0CSh^=>jzi9aiWyW zEavVQ2u%aJsJLVgmwys~Ne5)Y=yA$Qk@#~Qr#A~h5LF3YzxbnD(A10|pByLIDn{9Z zMz1TtYI)4zbRg?8PQHmoJy&a=j=l=B)=)V(M2zX;cx3geZiU{dQKqapVRPOhi$wKE z(j!r`vF3;df^dZ07{Z>33DD5wbeQ{h&NVliiP&FOjji^0+-bSz!yE~vi_A7`-sA1f zyUF5nHPW$+%kr{y((=oz$~GSxEsW+o+n(*N?fyg>_ zi^pU&3jhoDsRXRd)p{;{EX`yd#O^#CZEX&+du8SRgyw|dLx-S3%YEtS9z=0>c_+C= zoq{A*0F*;Ps3^}^VC z?;~sWr-BsDxE1b)g88r?J?N40ar^q;SnWeb zi%|Z-HYL)s=LdstX2o}&S8L4_fXI!}s9HdW@zqQ034JsBN)zQ`d=fsffJ=q7HtI&S z115E9QBv5_zW{v6ysejQbNLd9Q z-+$dx(i7CzRu~V`aTED zDCoNHLNRQ+ytRcrcwp(Bux=$6BFT|DB%JdslVYBIH9spVtK-O+|9d~e*6)@;zA+5v zd&6f9wtr{$`A9ThO!~9urW9=JoHuKQ>~B?Wm^}&ZZB`tqkZh=t&fm!DjoFuQ-pmGe z7u87PBNttW6Qed0VY}61A2~tHX#QF{iJ>>nh{?yd?6@R@4A`aps`;5vZ5JrB>MP2Z z`(DMn9OJsT)R$ZQM|-+}_u`HjR{eXU+TGf0*cW0=`#C05*0-U4-N$F8R`o=14J}Xy z#_vVY*I(?u(`8M*s47?muh5*B;I~$b7x9nOTgf2axlL>phiD~sc}to1T!lr5$fJxa z=Q5^;TnmCnfpkrSRn5xg1`e0+A`fKfJQcjZct%WBK}m%cbXf>v8Qj9_x=N>?2;L>8 z)bJqMjFjjZmV9_S;7xwc!2nkPqmz7lGpL@DDVmR%*r5LW!}f%xnA-yZ8Dv6B&-&xo zyZ!?8tZkvmW{SCD_6Xgpu(o~OK8@duNM~&#t;VHNlgZoXm-qm&a+QyI_fCnU$v^J^SI9BbcKGiDFR`}BZi^l;uw;*3*?_hToD zb1BJ3PVBSl)o+?~$ssfv<)vQAoKz7q}JVxSt%JpQyhXNt}lVyK*P*ryQqo}7uVO>n$ z8)H=p^@X{{2>rtc!)nk*Duf>{NN1UY(%`bhu?|!MY1Z9%uaRU;?#;Ev3<*nqf(=d< z*pijQ#l70jZS`ic1c!S5B3&j9l|Ds>hL~@(ya=PAn8JKF2|2-Kd91HIS$-lgK00F9 z;(gunUJF-<0BvCTvF8i1qXFLSv4fMNGNn3_Q?EBh{N!7Gikj1{TJk#{^{=<_My^zD32Y|;YS-x`}6UZzJ)GdH!`Y? zBp1Z*A?zh1&VRl2KaS8{qOKlKUSigfq5~+;ZjU>pu98tcIL|{REy36)zo_OwW^7vF zcB__v$$PVPHGYHo1Lr$-Caw2-wX*Dcg7+lNM@ub-=iH;j=Q!&(nV@0BHcBq{AOFH< z%5-qWA`9Bc4W%-vjVWr}_)bN4gOyej4ySEk$B|j5;=9GNe1PwW9s&t zQuLti`{akkK!M|p3Sc$*YsPRzY^?*pt=OxI+T3>&t{?h?QFuM`?yG>QLz~@3F9+QR zH|zp^zCzN9PFlnGruZ9=AjG8p!*TjW|vjtnRX35p-LX++?Q#$iz zqxf{v+l9_DZkakL?F&SiD-fs9Ihbsr{}48HL%9HiX7Bw((jz7$s?6RCLC^>)<~-O- z2z$=xR{m8(=td-MBVse?D8}gH9b`^if6 zC0AVZqk3XL`m^`7#FyAyIrj4Q6W=<~@?X-(^Ai%^!#6aU)(b`?JTUO~j@eX;$1{M> zWD$tKFOCt470>f6&drA+}u|gq}60!PY8tt>W?S?pvq$1=O`H@$`I=c`lIP~h+m|$m* zXm6)Yt{`h+=Bd;ZL~{J>XOL8*NK*$-y^PpOtQUGTje_AJd-YQs+QIr_rr}gY_3THp zB!mO&M0{=Cb58KZ^A%1CEkVO|^UTC5HS?KjY|uLD6{|1UN|!=7FNVu+cpM zm?z6EPWwxDpl`q~`m`r;R?jHu(QyD3*NVzasR1`=Z!z*KDU-A1(leurnYO(W-cXP{ zhNfzS@Mvz=DZAf(Snq&|X2_{5gD6hADBw%iiSjFx*jN|QU*)11S-;z*6jBzv?=TKH z+v|53X)<27L)8^D!v}{j@Paf+(xC;8>~PVwK~;l8AyEbU*xbn*$35Y#RT<6QIKwQ` z*{nuFjs95dml5`#vN53jI5`u@*79{zuks?{Olrhpu{nYQR6XX2(;>sMjm!(D2EFfS})wOEB(r1vAEqZd$ZezGq4Gnb=* zd3FFw=iDChD=8;}`!K4&=0CdMCT5{edVt3)&n7aleM|yLZ7ol7e9pKB3EnE5V zHK#eRc`1#H7^au$8_0-(vu1ou*KM&=-k;pQZ8LVt+>T-2*3HURylzsE4qEqo5Al^zRTqwq!Kw|IEOEmww};I zc*fkFL?xo>&AJ6O=a0S-jUS#Vlp05m?T8&T1z3Ak2OmeQr}BDFCFA`CJa%6*bV!Yf z1Kz#pKLGd+4@p(Xv8O_K6v-x1m7$xqLYvH=#dRwfIrqBpzQ?v4mF?kowo<%};Kv4H ztl5~vijbKeuerTK*g_RMvp(O9x#D$tGE}XzqV{1r@WQMu>r^3`Jtt`9N%Dju^(kr@ z@Pq@lV!R;x+Q&CFJ#6c#;}5Ba?iWpElcpExAbmy{hBJMr5;Mo{wF8adkMMS6PTHJP zK}wQz22Ou2d&*~+qq&t+2-jpOkb+gWZgHUU=vDMKAR1{??IA65UO!&Z|LT`y^m=tW zuXdG_$KhR7+RuY#!Q_4`9P~VMj9y$JD(CbmVl|=tJ%4BPd@-!gsz&@JSkakCIQgwED4xSG{n$vJR1juWiy%#0Qq%Q2OgqZnN;8vTSUkT?W4i|kuovZ*kutnjq0hmwS~ekW#N=xjc6 z_OF1l94-b85}mN0G7UU0NH06J+A|8w1AyUk)n<-7WF44;IXCOWF^$8wRouhLuy@~q zz9UsKX3)+Ovucas)rQu#Dl%!D)Pt~qBB7VQLR#!T4uvDPQ?R%_d#4P@r53rQsNnRic;Z*2JeB$7|?MBbH zvR|6?m0J++rQO*h7g{|wiW@X#p7R584H^BUlJ+M-?gkf+_y_`&=vN6%4RXD`9yc(S z_n@5eyKiVCP;v20y1!Am?KcDLbev&zM$KL^+sygiM9Z>!dY>YGaNTkaI%jNo2W}Ru zzn)~1y%0dS==0WDb->Nj55P>p#@`xA1lfPjcL5&aG(rkW8P)LAYsAN$tVYG1evTbi5!b+@Tbu{4?fF zg=Zd4+LpdL;wrd$!9^!+PX$Hm238ES1%rBjO`77Jyd|tf)hVU%v)lCD$6{V{nvy)s zWB$TYnbC3aJL_(fb$mTTqYRDHTglgSztldV+f*0!N;QlB(l&ed!&r=Mgl=LSgPGv_ zepsy4ea}5V&Ii5kBH}FaSo@M8dugXaFA3Y?_?ra&|nNZ$+4a_0C*H^go>Ruw&VM}A{)N9T@ z?8;Fm^t|}haf~>#qtw2Y-*&_fW9#RNwHC17f+?Xs@vf~pHhy>jdQD$RIU+^_?H=DS zka*cWf4>ctD5x}6#ARBFz(o7aw`voa_!z)zZf1noLbK8a`Y+z{cuVY={FIQ^>{X$4 z2+RUG*#60 zPZ8RNeASsEtr$WbGI$TFtIC+`W3w~Elv>I-0|MufIflr*aP$JYWmf6W97RB82Av0J z24sVg#AcTszb;E3u%EvTmyGkWN%LT7l3eB=3-!xeDrJLdVX&#jXM$xL&N4vKPCX4V#Y)W=FH)h)yAzp5CBXn;c7k(5pS0A*-|4QN6I{Q zYP>DEko_2mNO%`aylKQe_H+}K725gTpU>lVa_Ca)9qMN|~p6&6_YUiaSc&8s3LVEYdXQHOFbI3!YieZ>A^2A^iok`@OYa zf{5S-ZYl>^Hg8wxrFxLXNS*YgK~8M+Q$pX2$|{g^`w*p_koI&yGj z(EB%<=d0u&TYeK0wsj7TT;mDe3iQc7_EyU!QWl$AUZHresLj*BmOKFa_JBX`a}Ds=ZyPAr`F`+;Wr`qh zme)xU6|3IPJCfU&xc2zHM)X?;)%6mM&yOau zRCYoitbE5f#*XS8bqPu$*FCJR$Eb3Tzs9eBUyrhM#GG{L`L8_ISzog;Z)$5SyAW~r z#uasigoY)X?b1OQ^X2)M`OQ?CCrpv{3f8bxNoDxWFE z0&(YR6xGpSMVq;v@*AjTZU(_jgS={;q&PpB-=7ak1(tj=ZI%(`l>HF#WSKcbLHO;! zDNj9zL9h;T#e|qpxb$!k0^fmEJge z8i%ImvJVA|Ds_Mxvjg0j+r6c_>rhtSo{wi+bt1iTY(2A%Cu@sp9%2sZu$MKre!~63 z@fVVOh)>R}tu))oO7d4mUn^>deE7_@O~{O6H-sC5h=9BaE@lkcVJ^XA=Tqbm2iW@i zQ{go%#F09H^H#(ycl-UiyIlvJy25|B6O{dc#ZwD4z z>&G%q-;*)BC9l5>U!!{1d)qdh772dMGL*ELcsJhDSEd#&6@AdP9TYpYLd>J+>r`};PYmLZVntMUv|+@v(X3e5Ri zM0+&>JmCIz^SdiZQVIx6?&hJ9BM%ICe9H*tUO1FqED0U@yxLMp=!tB$IfC|a-;LcJ z4!tQ}rVSLtXAqi`uF_;`&|N2KcuClBG@Qacc6*U~=yAE`_{U~{tC`%ql+_tG_>*W? zRcVxWlBFN*qfLf7lETT@b*9(QRf*EZRJdcK zu8S9p)#WwTQT&ZSS+#IM2&Nm!Pe%3?0%{D_cZ7LuI?Si%@g*}oug}t z!Q&vjv)siB^dit0RbN*(<6Cj%usfAXz*M3R8wd!!tl{fvcumI1B*LS^DMISfD^KG3lty=8Nsv84N{_~70#tL*Znqxi8$R5N!yL8Gmnl`8 z4-1eVVFTcWa#M{58rRLaeVvKiRdl`#_}I!>`scB_Qn zZ2HX|s+2$%FA}l---+M~!$b2@e!c){*f~cMlN+6Q$o2b;eV8zrq6z<%r&-t%ov2D+Inc3)vkHVx`J7TnllLaJcPHV0-@5cg?n z@eus(&y(U>FJy`eK*m_?aDkDbS<0vZ8hHBXBe+d=RET6g%G}QS(ALCs=syAM3sYpT z_it6Bn$+++)_9yz(*c?c%FXP{33kq3%H#|54OWQrKOX~_zZ4dS@zevm2mG_1XYxJ= zf0y)sF+KKn`V5CGSkQk#!MDy?@9()&eF}ya3vj=B?dBdXhhaz!k8GeXip}U=9GX$t z3GO|6mC1S9BQi!KIX_XUTHD4oc9owYg)Y~2e8AV=s?70=mErUxJ_y$F!d(bB8kZ!! z2_>*k;oCnjgP4zsBq{1*3+ciML>zC2RgCsznBAqO32FcFMzcV6P?hfd1L)Y9^C^Qa zB_+9-4dt~w4W;GbFZGv`Z%%8ICM2)DNVBAZfUz_Y&VI0|%*4DU__0@4$|%&gODq2<-ObJ$$4R6gmI#CH*wx-a|7&_$Ns@p3Mrq*HJb)-SM z#xcW1`jT*97#AfaK(*TzbZ#DTi)8KN`cjhniJ69~K!-vo1;6PUEw_deg{Kh!qE<13 zT=_Levl&4ICZVxobIZDm*qlk&tn!*@@-gMi(BsP9K0a7?Qiu{QRGfY?e75rNHS-n^ z)H{FuV~+}m;@hq1<9kma@7CJm`NOB3cXr}rKw zhl}Q@;EZ~$=%ge@)*S&l+zGfP=YZ_i;sSnGAa)Q9*Zthk>}{pwP}5A{xk&y%FG~N0 zinNb*HHDC#q;@ijs9l1jO$ptKp>Uf0X%VLDcHBZ+&3$94x;&BB$eHq`=JStcIKxMEA{hTSc&*jeHp(>q07NXqpZ%RNOmnV@5^88z9d~3UvRY$JjN3&w%mFo$ zU9%g71H+!;)@WJm#^f;ihQ9cCMJ?|l0MnbbP~}}5{s_#$iJk!Ulm9}*3!I0F?XD#A z9RuyVhZ@;k=W_t#Y)skCdo{a{8?vLmXjY6|)(-9USsALTWXkXQ^1~vfFUVV}kdn*Z z7S~wDLR;CVt$%p=e!Ef;)=qeo@ysyFPu0bT^vuDY5N?FGN;9l-bJjRfv-Xf9(((?C zH1pbWBBsP@RNDp2KYv$nLwr7^$)18)i|xqx5X%{Qe?A7AagFgZO6pwtNB+vA4YlJY z%W_NT<0!V*_tm=IZa81yWTVCq`uulL7X;^gwvas`j8w7opChuVJv zWm0myL&Db8_{|#b%RSrhJ$ED}=D>Acy;P7~E^1o$2BPO-0K(SIfRbQHjY* z*yZfpv$E=F1J3718Cblrk~3mN%|6j_@KO~1!Z=hlxlGynt8^H#i2ra(=y{5jWu3G} zQNK3E#dY9y8*DBER#$Cpkz`eKl$q3%?x`Y;Qt77l-wug7s!=KZ<$o2?7TZq2@j<7= z*(I_2ncb3&7xTfzF45&Gi_3R+JEQqV>VkKe?1y&%`JnrvBe!{glTO)<7xz|f^OWIY zP^g7%-et?0-`p9$vc0CO=In3VJ&qh$Dlw2?_yjGeo?2m)SrnMmykp4oLBk_ z&1b>B5nnfGFrMd-mY46SzMjI(?rDJQ2=ffnjSHr!ZLJptUY?O`X4bh8?$~SX6Xw+< zO70rpK|CKCx=iTjFpwU$WGBvdcM78E@36BYixh3;i!}`iBOg&nnCVK4v<773o~jUO zQeRGOiyV_W@f2@6wjxt>_ZV{dwDf%^C@Nn2?lO$^L_Py>2@AopGibBBj zg>X`2XrJKWeY;wjy7V->rn#1hs?g?l;E9as%7`I(dIRqu4k%)xY(2ET{b zvBmI%F{N~wcelX1@aOiof?g~mdpM+tK%WiX;ws*p#mjxHCv8t2`hgUOVD! z*fOM7drJObkisAg4Q(axWZ9o_4KMrEJ6om3{z$BrdR}zUCH!>M$1tnRz{p=^NK_hF zW7QvI4LE3PdECRCK!MtX}Up7gM&pXyMJ17TAOW ziC!DC$4t;IIZ6jb?^C7_)jBW8|Jvc-)L+ult z;b+9XQ$FrkjepwCWc!Wm=((%~(>(75&ULIjFJ@OGcwr6NG#fS^y8m=6zoe^XQrxQh z*h{Z=gNh$z$j+=Cg1;5WJt{(9NYnG)5ZsaYgl&%K#y$?erbQR5K{cPN@RszQ9ZYH+ zG+RI~-K@I~OybrsleeaYFq-QT#x5V-g_tu;_62--b9i{_>==cB3Zgm_RxbRr`7EeO_${7PdQ%l^y_i- z^Y*s3E_Z+5p#~XLx8K8g5GB85x`*Aj4NEBtd{CcF&dX-sq09CAG~foMPT?LiWj-WG z=IQ0KKU;2tmMUJ=)ReX{n?J4YzD`+^ogKC5VD|E;M?;_T#X9-DBSXg|x@qYLH%u}k zV?jEk!-=pLNYTux*5ewn!dML;Z0?wJ5IaQ~Jsi3O_q;@B_(b33!rvf_*%e;$&yX=i z>`GGxefpEB=*l_s!FqDNW6v#aO=c0>rC+Hc^}oi4oVHz4m0We}qp~nKnqPz{)1zi| zwfp^*o+}4ygY141+R{4H%D>y{vO7-^cvelRq}y<_70c}T3CBB2=>P zLez8Ysbm!!)iFO~y!>ta`9I-#(@S0+fiUD3L@|j^- zB?sC~_tcyysgYp_{ACrLjaN9`HJ|D?_CLGc;s*MkBd~=F57%vRE!@jw+Uk8- zJwdJ)`%I7U$2gRa2fUu=n|z@EboH_3eoAlxEy7|`Bx^BNS4O!nOFm2_S=>Q_{d*BM z2MoJdx31HrHJQYEuq~fCzoN`BewqMEJub6Z`ns7oDP=XO?t{rcJXf-F)~Hpr*IGf7 zV4gBMP-(RGP%g9fB?6>H!3}g#%Jn+L`k7|I4?T6f6Lol1oIMq9h&9ntVR=B`Ms`U` zKhD^hv{ShPMz!v(xgRMvWEq>wTygs&iZ$NK9d@#hCm3Ce~a+X-TC zC|fUj(@hz^7M>)YtjoZXjy=3E5e|OE_*qu&C;Lo*(P6rO-H<}TJ$UK|D{pSvacq9- z3-wb%!^7aHHjn80(F$2_4e3FlqAQ1qO8m!(VT>*CZezif46S)kPkm#=-}7AhIp#^_onrCaSI z(E`=Ic>W^V;O)D94Mk;J@Iv(iwpqO0s&4p8_a_ZBJN4_YJpCOT$Hp{5W;!p@fsbB9 z-f|h|JVX8v&{U{3YdfarM$(UIT6xE;E<9pGlA8!cUDM*lUt;=I&<{2MGpAvLk@H9C z<1rP|xpr{ZAwq<>WBQC`l#>70dn8luO#0?r>)^tXE-@F5mY5ItYaNg=ztSdAdnmd8 zi`e&Q;DO{@kfJTDXeEu!bNA{q0+bS?4G;f!2X*RP3^aaYu{Er2;w`b~jIc z!L>I#MiHM!`R7$|apaP7->E3m8k>zV(Nb$TYaf#m$4`-_cEavK+B_9)-@rDX_;{YK zG0&gA5;|3#zlp#94i>_obxr=J^ti~Bd!=0VKZ!X!7^T%?e&CS+m03I|CygbSBQ;Dd zikz);yKc}@Ju9~9QH+6gY+X`>)k9dCCkKa=AK1i(S55Q@JdYO$Z%^-myf(b z^+xk$-t?Wx$cz28aa+fK_|5$D%m3_qN#GHqm0dD6AS`4?RaO9DPjj>emP5vQe@xj4%+Kj)rYvkx81I^Q?BMA0nDU;yiJ~_>Oly+=`aZ{_^CKT#=beB?EUDyzDh{`h%aS>ZLiQWG`zhrJw>ZqtS znvB+Er_mxy0*ym-3L^%cKewHK=dCUaQ$?dQgHrqUx=I|!JBNxM9r>Sob75JT+eD)HL5jdkK8=IBG*?nyt%}z|pJL77S0OaY0-}iiz25-%` z5ng$^r|G)#qN7kZqF7bifX@O>_un6}n3?p-Il{$R>u<>J4x;#-RP{G?oh)?%9J)AD zC;r!3AQ925=Dcu|&snq65pAe73om2YikuZ%Ojd}JGQ7T~@AcyueD3WzDF^I(4MJ52 zMzbm5Oc?#gyuOLSnnvMfC@nTJD~#P=NLM3#C7#X{alDhKEw?#YNIK<*{E8`!mk=q$6kB0 z-cT8E-PS7~Me|yiBYbC_w&Q5A!m_NV%CmjEF zr_A&Z=n+O&!oSn}&sM*RT)hS_A+BKgGY#{`AMo|#shrzDa;1Om559ZV51zHRh_Ib< z{OPx6O(eIy9Kr=m@CmBX9tn)6Ek#YE7(Z<#(7nv-e{TG5Y&#pcMed9n@VIkF?w`ag zjZ%AgtiHo+ABcJiw5v&gyhK`~CShYtMj6f1RCV zKIjzA${)0t(vE{xg~pU#woUP6FGOebkN=rJfA4|l!)NW7kO8r)S9b>O%>Nz>;7{$& z%Psk9n~V>o;UZ%$`}8rn5l2TQUtkk0P1v#LD4Q-f;mNYT#S*5`e<9|7aS~w2if^8V z)s^^9E&eZl{@MM%eG8D#z;i!iW>Zq|&xrYryz%{L=O@Qwf`q(*;J>Jw!G$Mcbk{AD z?%(C{CqVxX8A~)ofi@Kv!cp^5D;u=wZ!TxU^ok{RAgI$vs2>8v9f0xGp zO2~N=kLb4Ji*zo8pH;+fDENa&#{j$%F#OQZ`F|kc&wYeT;)(Dy$vW=-9eV## zR`(2#ApU1-lz%Dz-J@sbvzJ#6M~g#!2G{pVRadvE2qc}$KO%=mog@KyqdhZt<3xz4f9tRdjZRyhx`k4e=C}w&+!ArGcIOL@V_8pUKcM7 zw6~k+f1%_^+3`#ZdVfFmzfc1|Tk+CZm`Q75`-^_~&EtD=JkyxIae!|B7et&=fIHGL6rU~*Lee*%D$ETFh z#=ZorX{nB7E*KF}@@zm;M z4-vV)Qf7mLZa3B}U(y(88x~kG`@Hb5YN=f>OZCHLFu9iTOBC*1|5&*DxgctQUZuH8 z+<;DOPxk-7*WZ~EV2u|=?vDZ;AE56`r;LQxz=a%*JQxeIKuMlEIbB70kRbb1+zrQyFpf>kKBp*;*t_7%nXyF0c1*3eo6CO? z*wa)z#0#Y6)vu!NlGOWSIS3&*3wro`MP6&U6m<+W^+`#*V)PVsOqNlOkNkPNY$8D%@>&sUV@ZaN8jZm@7f!c9cQ&Pbl?M0maPmuKP^Oa zmcy52D&(-T4dy8Esde8|`aykF;LI-7oA+nOo=CXT&3#`xjX|OoY_FN+H_$#0q{&2r zG?~wGjhi!Ra~Biqis}1qJ3mooXP}rWv(Vh|!KWmKc{Uq~@c$M+dZ##gb+5Sr0MQda zdKH4UyYO;!0v}<~Pgf*E0SaFxGN&Ua*voVcbqe-0q&4BoX4JZ3edh^JeRCF3Hg{O) z24Bngh#96y4@{58_jGKgiaC^KW{N@h3|W$vu_uM@jfey9mc&)0@nBd!@?wSd@ulK! zAsW3OE8$w=kU0?c(kFzLi;v%{#C0ER)k3-;Mkj9jt(5sP*AKxW%=6=m_XUzR2)t(0 z0jhaY`%MTZ5^4wE<$WiAsNEe;pM5&6pji{1T9fn5S`*ORReWlP^-l}7W47z&8ay)#^J}nSZU5=Y z@$g%TSDajxBN=rt=XTNR#oDLBvD?OgRtBUOaDx{EnzJTpR3fPdPC;k-cKP#}ed1@t z9Ol|yJCyunzFjZcp#D}a344h&6Q3Zk9oen;p!?Qy7DnxlEfRWIZftG219TmnRPu>7 z%x)_ke&wszE;VoZB>uS}_9A?~xf(r{>+NzW31UO%>H2mJPHT9)+G_`+ml`eZKR+@a z9xnzIql)Lvi819`b^aTM`U1fIg`QZre{=q>Dg=8?Kdyrq}p%mk{2#HcHsrQlai-kym(m@Ib0&Efy-GtJf~wMQfD2IJY%qWE(K& zvcv4hvViSPYBPw_>oC7=hu^&3(ep9yRDNM)m-prmPPnkozH0RV!Rq1w zL92_?R{ZxbM^oC|I1Fu#;6pU-2d7Ec zqJEp78G<*F`rWR+XVf?Ao+QtN>4kjxqE7N%H=fxL`Dg}kTbn`P!qfjWHm(Gaa+AMl zmrAgKPozrui2MTbLV1o`j>dsaUv_>tT@GC9y&gSW*~9;6K}=XuvfR+nq##Ir`&@vg zV3-b0re!h};JKmy#hGOB32{C8;n%hqp{5pQ@r!feRMaP?O`kcT$gD#vnq)gnH`j?$ zq-cI~twNSY8-Pwe8DX(W-lvQ5m<0nU0 zyt#R;H$Z9xcZU$dh&I+fjieDnX$Z|a?KaI&Gyhsf6isrvc$I7U?@v{E`(L~ntgy0! zNS`ZOZ&x-joI9Zwj*X}0DqZcAn<;NTP_(Fz)w|#-l6~|O)?ekl_~4aTg&P;0R15!) z3se1Fj^Z+yCwjgKGS#`yfcu45BrBUE2-!w`3|wth^vkYTTlA?s-Knt3aARC}%wYJ% zJ^O>t;LyCedHe2|0oaop*rWY=E_nUwjQ zr;_`uJ?T1&PK7`Cv|Z}@7^TSz*77k!D!{8ew;l-d+jYfLlMj2)|fbcSj?K4-SfbNg7#dUEZIROXR70M-JUgqqUZ+{@_zQHdTG6W8v_|e_A6w zE%;+wgQ#sQu<}Qg^zW;_AhwKPd`-(m=b(u~=au}de0IZqj!o?nNm+K%czuqWvHo() zTS{<@4+q~n{r=6?%Qj-r)wS96{~_kNh&&vednp5>hLJ)J+F-|P1WFJ9d9`h2eY zzOVawzu(t&-SgUXgjbwh-fwqLcr%^cvmzlpWH!8Z)547#r?o@tYL*0rk(2M>x zow4$UCt;OGSYo4ekZ|ocLf^t7e;~uH5(9FkY;|IYopIz!YV{IlPX{Sl2z}TXus)i- z$C#%As#nL#Ty2T|brq~M%lj*l#U|deW_xFTdV8liALX!t8MQj_70ZMstRH)s_>vDP zy1yln&p!zE80Fv7OLAZpE$KuCf#z={g#a+7Q*nnzJeyw6;L5udLOI zeS8Nm;~kTjUv2=^#8hRHyB|RV;Z~(_;K1JQIt5q<|6;q~nR>#mi9a1m3!*upNe)7 zhhq?482CivB8O|;Yy}u|&e<}Fc4_I_4N)J*F6S;+NTVlaIHtz)b@H6(Y@non#nM4f zqFw1h6-Ugj)VcZ@nAPbf{pvYg{I>^#N$h%Mh@SL8ppK$v)JO)>Lg=`1yZ@uDW!5|R zPOld6U^pD-{CK-EXOc{OCB!Ou6jYIu9=_opL(4|kJX7Rrn$OF8Um3XbDPbXYX z_`9d2S4gc)KiuzSpF;LZjE+4)C{g+A)*9iq{;I}46K3lbr!daY{Q>8%?-MlAi=+ZA za=!GRVggzQP@3`=_!sjXr>VM|r^+X!Im@1$J%_P$1Xrna zbZL3X!KM}D4oznXm?CWC7t2=%1jQdc`25=i-wjlIduP^#m0k&(v+6L-+m8wd_IlzMp7^2lbb?le{C1Z9hH0^go^qq;Z&LZ?{Z?;Ds4duh_grZ;?u5mb&+=@? zI`Q89Hq8ZHvD7QC62HzFxD}`@SPbkzT}8=ZC)`XjRGnBrmeq9t(UiAMKx3sTidO64yy4r81#T}j1G?_2F5cEAqf?3#n8!1n30S?bjdN1u72ufvZ8NcmgPBY(^= zDSQU4`oRtnevUw5E0*1CARIQY53tuTXTf(a{MC7sKk7XRiO4py-G|3>@}6T<&=H!q z82OYtB6Yu6giVBGv#RTV1(eCRjT>6$2s=g`c8UI`KFLMevdh8}pZyoR;$ z>(+?S?Hlb8n!)E>6Wf_m5PhmuS2FVB5Yb4ku?el&M=@bUh%GkITa}WfdtFwC1{Ub- zf&9JkAc(PJL+K3fmUt&S$oOu1gGFhaQQ(@;o*CF_64pia57B)eHo@zbwZc~x0D08I zAp-D0$q9(T%(Cq07BTH~xrmu__9Oh@HrizP-3M!er02}b5lgq`6?lK2{=o++7Rzc_o#@XXH zC-;MD`x~ZV-3=K71QaQPaR5I&Jzj+D=^my9bha<-E{Y-=F=GpIm}T#&EHxApJlbR6 zoC?838ghX4t7&>^RO{UYbK_n!BaEe5Td%U_b^sqnc3oL~b=uI#>u3zK5s!(%RtV)CD)x5AP(hLvZzhf+_1ekq8 z{Tu3U;Sq9Wq9Z6@sVjd6w+RUpp9Zo}jBBaz0YRK8_{GkW*x_y`+0Vsn=P+_`((zu=L4B)@M7Hx?AAL*16!Q7&sYLs%M`(uE7wS{0x^ z({G3NKKVLn?zbq)osq7BBhy`uft#(2MCIetd<~VTyRh6F~zTLE49`0cY-DC-qL9yXz0p(G@uQleD_{aU zh)y|M4@jFTmGgBYJ7IlzFx3`MyG+ewN!Y=k8=oO|-hr71Pq)bJ@8vYl(HL}uLk+g* z*>LpJkui#$^$S=wWwLwYTi&EKUnCaCxBaaq=aFFYW^6ru#-3=JeeA5r129%JC;oLXbzRN7vH!JCWb}k)O+swGjH9j=l7XIi#Mx zQy3;`*!cFx2UCYnSo%!TatcQ{gzy|4Xnc3z;Gzz64A!ZuaMRzl-LZ72y*ph(>1Hg& z-7QN~f#1O%ScS|X>G^|8Mnj8baI-Cf-z22`S|}j=Uftc(j`+4h+}QY%>F0oKhE>Mf zN+r4RTY-q(u%+dU@y%TaGv6Kohf(}C%ko&m1_DPN;T3l_nwR~{s-4n)_YR&x9$uX< z3JZM1va1Z6w!CJr2jp^v4Yl^ARF61@Th@E`Ri@5=!)V`a7`KANjLfs>4!3Oy)qNk) zbuurNiOWJrgie3Of4KMB6ypmgzG`)dtLL7VbpK@{nzAsRU$nsO+}nprAV_G$S#6gm6pW=UUQBAk4Q(t^|{88_?d$^VD74eBt%6 z5i9`0Rlp3>F;W)h&8T`6EUh55z|DnOBc*TED!YW$_NTBprTD-87Wk=Bc@V6tg#_*v zeUv@KGf7$ZDKR(Zp4@Xx5h=Iu;;gE1y}F3Y@uW}r%SNy_rfpQ~8DqafJMGon z-{aUO_|~Iep#vtsoPSq7d@CS{ip+z;tC9-?M?iL8>St0I%sT<%`l#+}&g=<;u7=L?>tx;M#6h z+VJ&zX)U1gkMYLvkWx-HWc!*Dsz*BTK zSnkBgixtA#?FUcRMebB6fk#)zm#1TgrNj%TIz|JaFTF)QEt+(hI&AG5JM+2sD9&f<6L`DEm*&jIjK1Lt;{d8xBH{hVfA6D&~8esQ${YI4aj?{z}>flt@Y32G?+9O`m!pIln9k6c>liO9IJ!m22_B){pDl(bumW{2|qC125M%~QtbEfm%RQlME~^!*{hsa_zW z8_IJXdgI%RjfZx#i;6JVi$~4+aw)aAOa=^GeK(@C&;qT3^c8umb(}Hc_fS!bsZ*Z) zTt8^uAHDnZdX=*4mMztoeaTV7BD}Fv2Dd{tBWfjO+SG=$F3vp^?GhT4JC^#$(lz)R zcrz1j=7N4dDUlK#Y&+Fz`b7&6UO3g@^B=94F@&7{?7Frlk;-Tb=5R|j}xtq7}JhGv=O0Fel+X1ATb zK2`079PfV_3(sC)qlKO;yB`B_yQ@r&1qY!z)J_I3xS(n9p_joPHJFH;W;cyLRHG*l zL`y^tf0*+d--D+2JDZh4E(@3>ZCAHUgK=Q<6eexS(@?cTau@+zlCK9V0@uqfnW%g_o&9{tfRoq z@}=qY5+d}_M7qV5P+FicH>9+7h%xvSAydySqw(b4Ta?@ep>*<>TQ!;?O6JoUw-8&)DjkId<^kl4Tm(`4UU2 zZLw{Dk*$n|(>EJzrKV99!{J`q*tZ7;l2Q$7+{)8AV(Z_+)_jjoy}J#}2naGLQfdw* z6G`F+)5R$Rmv>%_x*zq;9QP8ulEL^%_>%D)rS}VzCmfgq+F0Gbn zQQ-h3ZU1HF@yk66ybEPu>_;N{76C&J9d2^ssdmCGt6^S_zgDz9 z2EIR-lAZ=KD!FEG9FMbs7N@T9koY98TgU_a>*B^USdI0k$DhnJnu6#+Su>P!Qa>kO zqtn498)!1j)O056NwYD;mHJiW~&7CvB1UvmIw-YZ+P8JiQTbIk|2;F}<{rZB>*F0Z&7*#7cRkfJ|O%a5wVT;KV~%U`$fNX?WRsZVaX z;a32vk-jo1zwSm~rM7dbmx5X-#%<}gyRn=J*ENK|HnY-{~K&TD8ybRifX#0J62Fk-d!!M z--N!}I9nh3YLnx?^tgXWqk0t@BL?T8wfH&+911+jk(N9kFr;0G>>i|tCV!j;3(Q)0 zIo2&L{9LW_o%eDb@nh$=bND^sud=tu^Oq$>o7R?rCp0LB)&|5O54FtkLzZ3iT)iLk#R_cPZwM`*_$_PZt5ZXb1;oQ1R0d_FLmv*+dP{v*n+ z-anzB8~xpZ0a^nYI_3Nlp?0OCcf3;lOts8OTl&A+e^Y{%z0NzNA;sZz9Kr2`v|BNW z;==e4y*FmLng7qo)84sB;5NU#rNak5?;QusEo&ksY$42e^FOJa`1e%fY`z$c0-8_< z5ND3JbNkIfN>z%kD^aI0V%K^`u6+1F&Mq--XS`sXP&p$160NO!fdFgPSnc*(z_#Qb z`~}VYvEHTxEfHoqdk|GP-WUywAdt|TMok8dU4ji}Vs(|BR3i{X=~Hzu{y4Kg~G$gpR))%Pk&r)B_M#M-1G4AGTqSN9hR(x{IrNrj|C* z+6D>b-p8p}`GHL@(`?1|VC5}aK(W>F%&6}#nB*=tB*O;?F83Ydm-i`9Wr^f1{*ETv z6UQ3gR_J?eoyDQxoy^jMfj-a%odrHr1KH1OE7u-#tE<91v?}`??cVaa7z(%1Ui*W^ z^mt11d3eRluM|!gaoOt*>{+ndZ9_aScNsl=Gr)JdwdCD^RA z?(WxW>mg4s@bCHzLK)ZkTX3m#r*a8mX|xv5;9j?3kPs^9fC+wZ+67oYwAe-=uNeyV zrOQhXSCbPf$NBWgiOV!z0W13Qw}7|N6;(El~B2KS9tL>Z+Ww-Fluf1)v=MhV6=XJ?@19`q7e zoFaeXS(ujYgugp1eZ46Ek8T4}KqFl1=0Tnhk3mLWJ3>R^Xd{ou`8w$a>tci#YU6jL zE;P(MZ$Vh?IEsR=<{-p&2P4PilI5MhR8FyhX)MKt^XNUMCw9P#jk&$Vo}VA?u7CTU z)zzm857-W-Fp8QYRQ!BC8ApMwYu$-5$@NrW#e*Mj`vNw{^JCH{8qL7*%*Lebu1h4N z{7Nb30f*8=__I}+i-OHx?%kQ7rpG5pPYAi0J@C&OMi&AjDrZK(x_d}bRvFDb@Gv+U zVE+bpr%zMsTamEmCX(j`vSQ$_M8i4-a`4NxuNV3H9!%P+q0f3wjaqfRW7AU z%^VzOSp2C=&7jgKRO8o|+E_1r7G@n3Qt;4on%w8>rLzLdVj;0$vswbw;{o6A9**^f z&3u_XxxTwJ`*F3^J!Ge;$~AVy{cYPSu+Pbh!SN{W@Ne=)ey zK~Ip9ulJ4hHrBdQW?pr|EcNT3^S|jjZazK^;rdonLUapEveR??#Y%S7uNKEIlAWgW z$=T8t^mFrZasheb+&1SFY*Lkbg$G{+ZacYW+%S{&_)3=JjFwWqV~0|raX&v1uiWR_ z&E!0q{2DSGJ?-&o>m|CQpQrTK;lr8zZc3sl)RN&t|J5%>{4@5}F3ruJ8C`VRTN-8x z{(j}2?yD6q5gxPOmFcTfmb4SgdaEC)`Q%<$$K*l=j3=h&B*iO2FC-Y*$J;3AI`5u$ z@o1wK3ipqB$O%r19(x`~8R!DpG6BO^p|X@q5Uu%I#dP_{OcEn)Wy2V>5=P;s7J*uI8e);5Z7Zf!D2@BL6c#2!eXPLa-p8(ZJZvYPxw63 z903R1tt?L7zL@YwR+TAgSo7lc7i5f359K$Z$tpvcy2&aXofFm95o6*Ut3D0_Sf==h zkKl7?Nmr*V7~oYzix)|$0qYpi7s7eEC_M>?g`K;M1GP)D4mM307jt}8-g0=2xnbfc z(Ptw8&|9L)wA7q=IGxsBAHcXEHE^P&*laWZ_>Ou?gs zK_BFK6Ow?HTLt?!)lf`9{xP-RFDKYf;>_hGT;JPRs`KTDMlJV@mjSkmYa9vB#XK6e z6XTtFQT0Mq)3xzS&SPH9oQ$}1iO+~pIubV!R?C|YvRd=_VFSrM&aOh?%1Psta zhbNl@`+e|hB#q#XsH7|aWAqEL@3WK#G11kfG&mZK0l2^^~$G*C?gh zLVaK(OYYpoK*Mk1rac`3#RSG>$n3Y_SbzMW16bq&UusrdY$$`6p8z;No;}s8)4@GM zsWD*nt7z~Sn`1FIDsp7X$BR&>E%>%evr)f9>~3m*42teJ@f`Mhl8k5+Y8;IsmmnR0 zy{2E|y(?@hJudV$qurkcceTM%+|y@8bZc9-zX>kk8W?kO^-%@Y^@dj>NT7zeoB-<% zx+oH3dzyNa>l=9bfS8f4;4h`Aaql`}gR+9>i>hT4Jxm+W!qg=rUg=BLN#=awcr#rW zoj5nk3 zZ0o}Dia@ZbpnXI(0)}d3230RH zsIx#2AWf8b-&gkw!|P?oRO)zkibsEV8sEhB1=5LahsPafh^3-KSNP9zyrt%DJ3lX} zc<`X7Svco1clC%s*}&$meb8uPSd0bV$29BMJC)MBgR;lq$FqAzTl7SQLmt^!q%OW2 z7rb|7jn4sTS;f1yKx~K--jKQ9A201-(_|cwBgc&#mIIBs_RMC;|HzkM=o|%&dPaE* zo=}yP*;Wyrml~{Hajpw@EUgjUcKI*x@2vWtGPsU_vG=r;coQ@DvDX0QI6*C!c=qji zDq`y$9O4*f^H@iCr5F(*U_XzOYP@e-TYAXDYkwY_-Nh|d@5uQrita-ci^w#xRnrSS z)f8dd@Yo1~WcZO_vbl?fJ8OFi-@xp%-^n@ZsBYkL-pXq9DYQEDLwF7G$<}6u$J$Wp zOAVR_&GVaH@|95yXD$Rh;T8a@Rh3#RwU0m8kn)MdrW`$cqgQdn-M7ChYy>227&EDL zhrcl9MfEx#bf?}2RBcdG7+hjNZ^-9+#yugWbw+Vx7CaE{N+vlm z8@#!QDmpwwfzh<3quT?15(FH;_Km}obLBv!JJaGn?G7b=ro1fMIxT!faj5&L7!*4$ z9ut;9v0}aCU2xqkvXq-(;|M=Fu)}fHLRKSiFX26Syk-l^1-bwYobD4N zHO#M-8@z@~8{*)mv>pLLfxe8)qsH7h7VKv>%7V`0Iz`VQTn)iSW1aZzt#z_?bDRly z5srU(!!^JTk{lDO;WWnXS(m2JDR8D*Iz4y57(9AKgR?t{U?Y<;agdIEh_(byZv|Rx zlw$LaY&(FiJA41u3jhQ``bW(CBs<1&u;X9WXgh}2ox8|*&U2{9#p#UU^rmj}svWeh zcLqXG38Ao3WV*!EuL%H3%qo{S^vKDh+(Dyn00V{-Lk7oYjn4HhEGs z3eY2aJlM=~SOWK($};}-1Im2$;bT6X=n{RsZkEAPe=4sF=YKygHXfcSPIRr-U8x&_TL^kUjWqI` zB9G%cK(hg?YkSA+dwVm-?@Fl-G-joGg7m#@96j#HD*MlQT0!1_xEl7!anEDiJLHnLlDTsUfZyKQks3XSzO-VZU(3JdQ(5&N~g3C3N@mV-qRT;AsnezU9 zkse(7)NO_4;rIC*-*;a7UrJa{lGQ}pMwefoH5P7v`TMjzro^hpr?)avUB-%Vx~!=5 z_K|T<8L4}yhc1MvG7t1plZD~E)TJxl%0EmwsKq!yd8?*~Oh}pY@4NY!{VT>U4=?_3 z@pH)$rCzrbV{$LM#(g(|aO%yn$)j5EzW4FMyOqaNYA9l$NS}HBOltLY)BW40?JQ~J zx@qC^bC0z3MnBd|O_?70#brYq91~pKMLcq{Ml4>9*=fd&FX<&pM(bIhGb0nOj27Rm zCNAuyk&3wYHFvS(YfuM@d?tDt2eRHre-dsa3|vkKTt8T`Zlgo0mkjKrtnMtl?yES< zq0<4HN6kySz0Oi_%ic=2RZ=*KX+y4-(sz3GE8qCRq>1JEJuSPk@>2AES&B|bd75Zs zI44MSR>`_#+!<{*U=_z!$ve88N~gGLsQ9e5#f8abI*l8>eZga#P~j%k-T}0Sy4V5k z>`Wi9Q7gO7MwImW@8{jwKrFAl>hv=@bMu0O^FtQ7jaq!r?2oJ5DkSr%wH-zv{S3Nb zUaHTM&~&gd$Jz5J1I)RiBexa$mZQ9^Y_Lw+@@A8ns+H&t!w84T!H)f<#rVLSwt<}h zg`=@EI0{m(uUfIKFo^~0BQ%7rDs6oH<6-`IGw9WoNO22#*66Ri>Dfl}K(grGaWr6G zQ2c{=?Li$?={zu?^)O(6w=gO{AZX?!69K%#xQEbC#wyay>}+bB#vVYYPe2{Y^;PCM zLTKywaKDg3C4%0XXk`GoVePo-aG<5k2uZ@(9;j7k_g|#5uJhAs&pB7Qdb|f}oc4a~ zK3DX!eDi}s6Yj1=n+kwmWi{U^9@`8G9?M;;zJT%`i48t12J1IgwvCTQV^eoH5-0M4 ze@|w-VA&Nf6Z<|j)s&Pa^}ATdRBeHm>1AKtBZGyUdeb%?y+#iln}VQU`AUe5BhqwA zwtct^b=%blb{B~9@46tYyf#otd@r?S2Dgen{WKiOZiBSr%*X;2V>DX=%`$hbI{Zf& zqb5vfSvj!~9)M4IzhdT;cqVFNq+W0d$QiKFN^difbK2on8P~p*&Ag~qKw8OpV(KsP zQEg4dP{y6mZ;h?g_?PsMFp`YkxrdXe&i5~wi>ip_Fu=>Z%y2* zJ#lO?eNy#~s4HHn5ukGzscbbfmsoyUN=TbvJLU&fT4`b+?WK-u8F1VYmeb5v^Uvg{ z8#A%;DV09VSWlG@(EJ+bY;ypwU-mU^yfisbONlHERYczDRy~78ttFm0brNp+bQf#9 z4X=D=hXaYat})yhsF_=m%GX?=SmJ+@ckZ5@Yf)Q zH~=}g`%1~{&Yk|o#aBs(79_v?71q<3Q1GfuQLm*ER_=bu54epB#3gr~kPAHMJQUTR zSm50VF-C&4%edt%=pU~eCm57#T-V9{l3jl8l=i)8pI~i*&rbo!`E$npo4TKIzr`Z> zj(E-(u9Z`a7q6oNV`ldrPN-Ytz7v~{+^d$3L~#}KR*n~iPEc&0QmIldjUmA#5ZXY%;d7dI#ag|)?j+Z!LFbg;q_#vqq9txX{I*>YfWlI% z<`u+SUK7DMDUTUbzBaV2m_?kSH~n?XK%&~p^HW(u2I?#jN%fn$-SbX&FzKZ*9=iR> zBKf|`{M%Y;l?_jzL(K4PAKs3kedp^D0mdC{Q8@1Wdy3@Zgeu!=<(`KBQwnzyC8f8D zY;^!WktUhwps!H-?Y9N19%9G2_qtR4Cw~RGPe)0V0X?F;VJ*^udLsA4S*4pw{I37X zhlj4PNOa=JU#8=sFN=P!a$Mv|Srb!9%dE5O9lcJ;>_hL>^TX(EvVh#IQN`y~cN>)L z%@<43lzgkBg51;OEHc|QN7gE`!g+)8!;e7s@Y02niwU(Ho`GJv1xAmt0i_kVGq=EW-WNx57V>j9|%*sSfbO-AMz<@h6-0z&)5Z4S+~uQ zJjP>?Rb6Lh%IeO$9?C1OvXFhDQ%xxz>sESlnx?J;g%)Bq&&TT*;9d>c&qHI_gcU)R zP*iFa(B~ky7)7roC8<;$w{^{DiI)9m@8}*S9YDLY;2uKKrdd03<*ryJ7UNTM_Pi+c zioZfE;KCh;XCLEjzpr2dd&a?OPJ6P&jTM8)m%%kMpRCAL?$okCoIEDj?6Azn(GoO2mbU*Y z?j+_iVTiEi%yjz54p_JscK&c_;Oz5C*Lb{Zkr3M&HczU9`dK+eY4fo0HRiILLH@3Q7$bbQ_3dE{pCP?n)$bZ(nN#yWa7c=sT^~CE z^njK!xWi9Lh?E-EGpI0Y$#JY4$+lYNO_9VZ&i8`b`?YIJC`Kb`R*0RVag>4#zO)L} z79ECqFH|LZ-Qm)&?kA$?dR_H)lBQBmSl|suwYNWB1+hr|B)eKilMWK-X?3=#)Bl#7 zAglkynaRavOWOhKEt3eHR_tE344mvy!`d!3dX%g+?`+fkR6NQ}2fBF;gi!|=gqNvr z1CsJgNMw&!AZ`DX1+oxk(mh`by{WT~={pdLO0a~;3BvPpn*_xf@^0!ehvIsDp!APP z`CZtQ{H~6%ACq-FaQNBJoInHxDNc);9qH8jG zHFFCXp}1*oKdF6}#wPa&KfxmyH&&Ypg@!t`;k!?Nxatn`+3M~AxWZ)Rw^dxDwjR~G zFSsllM9fd7`_eKfQub#Nr+=U)3J~ayo@>l*0^=qmsMdVV;zqV8cZHWh!A!~Ad#5?w zWi=z2qP4m=8WZ5IhfE~9sXa{b*P(5&VQW9L881oUL2Ji|Fu})B1x*>i3}-yrJc#0C zl59lo@Z2Au9TZ|KZDsKt#WV=$P&#g?rW5Tjd_j0wxjp{(v&yZ;yCvjjvGd7J<#@0K zOp4<+#6l7Nb2S6SB`Mq!&*&x~($GpRpSgDY&>zVw|L2_jAoQPn_sH6;JT<^N68|X{wroxU?*g84 z5{AUanrZibin2Caq_JMc^zQvi0m|)fYZe*+yf!ZJPDDN|YS8r>iz_OwK%7-vn|cvl zNl9=HjM&pz;h|veB*3$@2_C2k1bBo9bn5MEP(xNDdl9kIlr|H5ojVeKMfH5Tfn`pS zrZ9AH*rQHX;630IwPU=h`K3ddx${GQRs8NsegiPA)(C)xl*alI!_o;rn>g;W;^KAW z*49K@8gOQeqt3W&t_Bf%dF^m8_eEW`yCWVByfz?>#VFNnyn4Pzj6vCUJU6$=)y{Gw zuaS1ccNFxUk*sGCeWhOE+z;D;#=qS?VkONQ&A2;2JNW?g?OY;x0_hYabE)tkCuhA- z9OC8PZ1=#cDC+EC)aSIBTjagnPHxyxy0kmm*%`P{x*zh7lK7+Ne+nV24ku4ztbL4i z4usD61hq$~R93&z2k~P9Kdt9$pE;7HduqITt8V0s=(U-E8Seb|zqTg`*tqvLwm6M{OtakEGDdg2q*r~E->jW7uwS5Pjm(&oQ1ic5OZmZ2BIj-1FVHJR_5+u7JLdZ9OGx5^;K|qABxtSO4B%3NVmcXp+_2yjnyx8F%U8vs7kHiQ z?Si_WIAfriIRe^apLr`VhfVP`+9M~Nb4tO(WRvb+>qXpX@Fe~8Pp-)3)mnxW4;66# zg3EAQEPEp%m0iX}fkWp4)4LD%yl#MF)5Qt~(H>VER@TFJKDcziy)WJY5J*tm_U7#< z|ND7j?duNK(1r_ysWPctR9o4`$kc(Dy`BPtsn`_)$}V5tfizn2W#$9Uy4x)9(W-Si zs#4uAv}S{&iNcVuHC7$ZbH5L{$lDr zG{o>k$y#3f;j?7U%B zGxeKFTNjf5MZX2%_X}yI9H5#d{n^u5cEnm$+N|8|A4^S*r7lO;&13=&n9p6@zVc7$ zOPSNOG98)hdz6j9p=1AkCf;35p5g~AyH1`Vp0p`>-&b7I^eU3nJg6dJyljuFe%gH# zi~UiLkTK*z7dbHA!>u1MNlE6-nj0N`My~f*HPiD?@m6bR0dCnW;?# zI|~})tH$tNgLEF~yu7DKJ>LqQZ(hTL)e9?!ljNPt1ynUwRA8=PMj~vIrDNBoo_PR& ze9t@_Rq}$9E9qM%E+W5X9lT7$FiD_AX9Hg29TBF|CX*2Y>Ti4)>1KtUhBe5d>7M#t zPWcq+G%s!Xlcbv|zjn~sJ}JK>)af<%K-*>WxHYXYf7XwyZ8Md`@kx-M5U9EGE68ik zq@oOh``)X3U3m>8WyqgFOuOl3E<|@Orce4$z7VNFDcSj z+U<5=Qftdw{_gGDY~$zqab)kJ(Pj#RX2$mWVSm&eRKWbW`POPwq$jZmwkNP6`3vs7 zSyvu+<3)OYcm~HxSLCvkze)93rl9~!71YCu9Yut%w5dXgee1nSl$eEAcjc+Q zR8JXzum#J(oM-i1vOGRO|GOZ}c&NAbKpV+xEqWC>BY12$x}`*{S1i33J{V`tI8mwX@gD3lWZ ztqc0mezI2sO946pq2X9yMk((58>58O@w)M{lg&5Gwn`vnu|E}$-{i^{w=%PNMMuPP zaY?%Lo=0ql0l1o-ynL8rdu3M9RW@<6m1+Mq({nlEUlTpKJL9G)zn z6hl#a<9tf<=?TN7X1_>9Rj4y-GXXI5i7%uVb?YhjeE@}hcECzOXMCnb%9P&B6Zsla zJK^lpA?v!QDXIh-8J{Tn!1Q3~_-cG_=`Yig!cqvne{NKdZCqXP=~(a$Vh~=$cS@GA z+ zIgBre>dT+-ZaXt&qlu zF?&42C4#f8p2&->vXOp3-6x6GEUN0IQl^*LL=#4m;Y&5Jk9hw^AF(okcP?zrEsc83 zJykc6{4`uo?5kXU3D-_4P^=bg;Is!va#5mYd$%O)pZBUlgKAGl3ZJ@f%-LS`b@MPVOs!qHu35F!JiH(DGX~*&Q|W@nsVs(eW8vlq zlQ8QfZ|A}|<;IL3Ub%D>;l#A&3~(zjAWu~@q0fJTEoohP#D7KU zwO(?=<%)Sk{U~>M4AT)4!OYX@_JucjnYbeMn49`rt8Z zrtn+NcUZs3-1B0+TP1a4@Y;t$)n{eVX+LTdSSlLs@Yxr(F^&uRC`j@Aj5j%&xOuZg z^pxedH8q?hojs$AWebUed+E;2!(_>(8?W&)GQ2rHb|){Z&`*%`>do0)ZcYSL3vqim z%WP=?l_Nd7a-ilu55+?y5^}xERl|6-bX!AI13LI~uj>d|-0L(CeA#(=$l70om?D;g zOH${0V|RDuqTBsnPjM!~nq?5{pjPc4->+T5v%5LO+>LBtQV(+CzWBlqUc;x+wYooX zIT#4QzmQ_P_i|6Zfm~w_RX7e*L-RM*uAaOQ@yu>3F}odsxO!1Nf@LwyIQPojtfs-b z-|p{)a7Rh)k6|W=lf|#wx8~D0Flh;e@p|P_hbgQVFa>uzm$QY%Ozgi%UA{F(HgbD; ze|Rd!;r)DlSp4MbWj9bKLc#d?_eIC|b+V?{gkV2KO&J-bZCIrrv%OWOfxdm<9ieKJY31jP{i_)oMb7*-d+Hx@ zO=2^9lOCtaxm`$LbCwq13Tk*}q(WYb(`_iP@77@A;i4n16VD`je{d|~ zmC$-7O~Fkl`C0qzS!8tRr{EXoY(q7_?U{yeb09x4FpbzWf7GSbAujkeyqrmHKD)XV zTX4zlbKaogrQRxvyowN~fTOA&?9=n7={c+kH=i2b`}Ad-{mPcCDQRXm|Cb!<{pcdl z)hU>8@#Zn0N$?opM7oICu=sq(HAOt8u4re1V#MgSy)P%(r`uTvzBn@!u%u$&+cI8tA4|od-mU@OR5Y-tW|ojd`?t|IpYPeSgd4w9>Hs9^4=HHG86TJ-qA_b za@|UrmaGbPn{MU z%q7$!*!Pq3O#vRZ7Y7zcW|wAnSLVx3b5`A()_C1mE4+80B#*!Cw z6o%Go%w@yRNb0Y`JAM}ItdT=! zdhHuMS4Dttv{A@Um0FFr)izhg&vb;lE@B+6L7mQ!T*hg=5~onYPiC#ZD75qYO@A6t z=#*sq;ugNI?7?2eULgVSkY4?=i{(%IvR`C)%J(N**ST_oO1enRH$F9&ILnlOBAr+F z1&s<4dN1-{E`2Bz!LRz`@KeOTMfmmpk750n7t;(r&FOLC&A9K9)<1}-ecO%j0bY%8 z*2kmw%$R-{Js1x9AKblVR9xM*uA2Y}5D1}gm%=@`Ljr|cfZ*=#?hv$acM0z9!QI{6 zg1bB1`o6XHI_uuM@5*lX*JqVf!_c`yY|~j$(rHH1OK|y- z;DD`-0NxMJZ#T%C)8-J#nK_`U#;|)vsSxIvv8=5ntn;bWgph^aM0Qn?@bdaT4T18P zB`Rzf9i>D@psWZ-z!LqsFGg49z+T@_e(P%({a9C@ajXZ&+g$|I3aJ(Tkf`ehX$M0l zxe*%P21sj^&74dQamv8Zw9OL0es*(D5L-DWI81~oR1zt`w?{z5>AtMMsZ{X#w4iwj z!=?W76ek%CC>@Y#$#wcz99tUs>Pt`N4F0Bs>$0X5_7ZCUV)%ZL_0{f2;qFJzC({3S zbI^bLPa@~r4=0`0lQ&ikSEmtYO4XR!Dpk{U&r@$sUU%PgdOea)zUhFyhx zIH_B5C^*3B>yDJ_&a&1Q<2xr`Lf!#&Yd>>i&(|@1IXI_I$q9DW3JSUvP788wp(>ym`cfN2TsvXNxWJ3Dgzj(B zD&iIkAm+X9AVvW+D@DMiyUuXv1yQL95sl4}+=rPB+M#FmW>}ae>&J{2JI0RjX}D7Yz@6t@>rhpbWllCVghyq75n=Mygf(YaIW8)Cu~CE zk98yKx^)Z4XSF3Ty3h9JP90YlQW0g>)UBrVW=NlqD_!!**^2Y8ZKB{-Lr^*nRpdF> zQ-jPENM6|o6ETbR%7oU6va0FB7n98@)*jlE0=R~v!+5d9uYS?fnkKcateQx>a}|ZC zPc&I#h7nc~NT-(dIZb+g1)=E*4Y#mgp^bIbI(-cUJ!!FtJaJo>R^e@jZx<}Jd<|1% z>)pxj^k8ZW3y&LnQ@0&ZEbvTXhx+`;r84J+QrfS8M=d_|xqe^5eLT+3YKZB4IWWTprup+#*W@&M1Z5h+9~p;M%ejd&@!j&DZ6Yg&e~o<)X4iEGfww)}6Nv z@ltkeHLU5Ku(v&xoy-F25f?YCS{f?kJRL!L{v1`LuIX&Av-qkP-2=L61y%gFo6p;f zcdAwPlqwXR$z;XgSfy9{In4o37VETJmq=r|*z;>-1sNwtqm5~N;KU7A68DNTzf9qy zB}dW#m*V6MO%fStTJtWKVtSr|2-wlk>vIa<@#!xME3seh`Pos8l@ZS~4MKL#;HcOK zAysjYYXR?C3}eF2Tz`!9lD?2w@53@{hx$6n^>g!~8n&iljsC0SgPNt0)^}?{V~gJ0 z8%Om6!oDShm~&w-cFQmkZ97{iT~xY$o7n(Ix6s2tg8ODo(78njD`U6lwHfGIO4Q!Y z(jz}b`S759tX>oRt<0h;pX~8-y$T>!SGJx3GmK(TmIQD#b1zm@;6Z1CT67(xxd$BQ z5+s@sdgK7S`{Wv=lw5!p{sEwm*$4}KWE^xb`Hs$Ep_`FY$8JxwYalg$ZJ}16xTFMQ zsYKeEqu}{7ca05(REh=Zni@wcq&U5EL8F8zOOWhBIjd?7Ipt%fO2`Z1jGRzkNl(42 z-!F1avB6zAvHOzh3xky2+_hLfO-#>>UsXhsLyi;@3|gQHr2}>ua2flG>(-HtJ6|2l z+R3UhS8>=7nf!a0DIu6-R8_yS&_>nKFRHFw35nd*{N3ydm}L01nr@3!v^(I6JzLMu z?B}-{)l#4t6tJSw?vfy@htpW6qzVF!b_^{#q4M!v*7wPq!-*sDu*XKAbH?-#&6Vh& z3)$v#DqOYAPT;$W5#)olF&gvc9f6vh@Ubpdut0h68tEk$8h6B;L|%x7=qVQs z5<%!mQu?*LMPozef zOcEhapVT%zhM{fJ(uJbgTID8t^-TTl&?pG%?{SL2v&7Mvf_O}tS6GoKZ1DClR@U|zCfNwbkf?W2=hqQc@j1jHYaVVuY4FIlR_3+by8vzWg?n=zd)AT#KN&V8_l$|aBqRA zC^_`ihB5qQ*%$QUgEnqz(Be759wySnv{4&Dp=lIQZV5T=<$a&z>>keIp_*A2{IhF2 zr6OX}EkL^xyeH+sOxpEq>w<+dAH~wNC<-speKa}U7uY7qWbdzUbJOiH_(ny$Bcw2` z3m&PeBppJZ7s{>)*2{ww{ei0^82OPAWC()FqKGJKvCFr5Bv@EgTVd9K-=iA6PM5jr zU&^L&MRGKi)Vc@cfzo^-?LN&=v|%_RGD{XU9FrABf*+tp;KzpF^gbusfuUyuxDZLp z+7EJI{J>{w4jWa|U7En_3ic1|r62+Hovo>6bK2;FwZ3NyGM_UQ3i3klhYhu9zb(_u8G4atIli+9CkaK1}R=z z8%&?=oOYSsw+5A53=f=O=4k8V63}E||MCh>$g^A*H=7;ipsuB;_^Y%#wE;ZFwODMG zzUW!)?33LLC6+X?8uMi&0qzg>EY#0Rg}HQvJI{7=i=Wmw8TO|P2!?bxpiPtv&y;4* zUj|b|Wn}s${gj6tud?$B;S1L{2)uVauijUB3B{)z}YpOmirrH=( zS!a}w<4fW>^D|=}#$!fxwMR9$r6!)_lp6yMD;3zNE@mb-~2v&lB z79O#za^1YBM0wi9?c5ySwv(5`L-wAVt1*yoM3c&ZK>#`7YA5vet=G*3&L7qZ&L}_E zPsBj?yf@i!C$|8)A8rp4D9bbb3#}|*TS>*9DxuNE!&SsYSIc$jk9AsJ>&=m*z`aVS z1n+_m%z^=Xd(Cv~m0gx75KFOV-!vT|(i z+}b%3i#SxBu~U%G^uHb?+80Wd=X#w+eP1QJ&AyJEaGX|3@0|nI;fN@MfZ^!d#H^~& z+{9ZdhBz|Yt?$k;z6a^~ZuD!{Tt9tZj2B%JdGP1l^gf@`4+`pYtK}p$*(gtYB^e}B z^S2$2^5O^I9fTWfG&APFRAzu(8O`;=;!3+@qi?`q zE3MVnq48Kz(DFfuuKKF#9rH#Jzg(3wGdpkY3nF@i5yj10Z4* zWC2#d@~wNETA!=Uj(I8EhN(Gmj>(+%*r*2DxaiTHAg$Et>ZI*jZi9 zB!5P)Ben)VJfv)3TZC%vsf);r-!$fD!cm;f6f@BiQ=WCZ~LI=mMiTp`h|_p5gRx zMc!QBq*Sb~y`T7r=-`E@uRP0qKQG;^0J~`?r#oYQ!6nV^aH11@eC$}o zi`5Cu2KWS7kpgvQ1pO(me!rRoDxZFr3GzzQhhwelw8f^eix*GkBvd9s;W*4&PC|Sp zEfMc;gkv|in@CAABusNHrObgW_R?CgmO+vC_mR^L^p~1ji;sFx^E;6ae89uy-6ogQ z85c)#^|Z9RJCw+#2uFskn)9P?gl1yEy65nF$;Qz${Mgzvb#fW~oDzjyykwnx*UO2o z`)WH6%S5+PUlb)n`}s)NTRtp7^ST+%K@WqI97bpVn}+0H{;gZxPe|Y(=my>Oz}ym< z8=#^P`6P(6yS~66eeBUTv!#7+emg-L)g%1tKH(Pa2J~+){P%vDznYK|$So#J-m>&R zkNAf(`mYaDjtyBFenq5wcT4oM68;bKtKHvw3`Y4b*Z4m|9_p>Ub(Qf6XIqsI?bk{A(4Y-!KG9Hcd|`3KL#%I z1(S9ARljkZF}I{fbJdcT^7mbk^0jFTTl3_9wy@i*(mMh){b4-i`5C< zcD8lfQTm<{wZo$(`z?2h;MvH@#|Fn67OoiEZ}A&4h-nwjyI^qH``@!px=xz#Y~+o8 zxu;_UeG{1=^<^dRh0&(`O9SLE%;dA)3-b~!Jl?eT@+a8Fd?ClP#O}Y$2;;G#seC6gu%T`53sn&krH$63&fXwx-O37zgYceZX!(oecNwr1{#`B2v|O3#e9 zQVt+89?4Yn!+Vem%Gk()@a*H2E-%>@B0&lM5Uib(0(!lmP-}-Y>ee##_=a8 zc{C%Q=pG|cQ&*9e;>6v@-@7aPLIsZ++mdlr$>(MG zqlv+C>T~B99n!q&l1~l`sjypfHep$s1I3w+lK}E4fL{J2oj+XWAqKL-o#V~!Q08`2 zWYH;vi*S4T5aDU=#Ji((U7EjvCe9@&OKj&A1$(lOyBJ8aNjlN^8Fl79T^eQTbX#E! z+qm`N*NGcp;;8Dw_D;8P6Nfy@Lbpl&@OI55Fp{kE+0u<4BR5y)24O5r>ogV1f|p^G zGveF?Y<*tI`$%w!`L^Fp>`xI+(+@JE(-fT~T0jkbM3=YwiPV)%Bpx;$9*A>AqglO3 zh&*oO6Dh=^!EanMsH!~SX2n1*3jys>YuOqjcO@OcggFbT1^3DcO zf@DIcH{<>V%PA2Y+ueydS!EWTbI&qyTB5;P;s2nE@9uW<)_!kykvpEA27lci4Lgb2 zx^X<>_jngC#T}I~?!#+Trl*`Z_7VJbAsk3M*;CQ3-uLY;8k1Fr_#q8qn8q}OH@&mFe_ztLw2oMXMOzhYQ!s5_5Skr^8ZViX@ z_d1gWu2P5jHKwepg43Epb4<|z7(FC6 zw-QI`5PJTFmp3gB?$XH_!4U{mQ3cf>ShSwCZmU1tIa}4&q8Uqv4x}~^*=>T^C}L|J zE3|zT`8p>;Q1cJ9Wh8&J)Z^a%e#Yp|**qhxsy*~Q=1}p1o(qWO{T3bxXD&4+{K9=g zAmU^u-LILYzt``i<Y%IDXiH2mg_BKS)h0HCmoO5Ez8H4Or1erVx-BPD&l-SJkc~RiV zMTAOg!)VyS@MWj^*O))j#U2L^hEzq8S_$7S-zmN zp`roN^{|%zSWneet@3Xn4=)=C`oysK3w=_a5(uRC{Lm>zUlXG3hV=d+W|~Fez|jB= z-_olB*u3DwOT{{}Q$A9jW_17fEmbhA0Br5kKCIfX*+s`Ya1V(1AW_*%y10lVv*G)= z?qB!Z1~ZgF_ybtgPtm6UFHkX;{W zy=~@OL{UEAE=#{2uF4ms(GzJ7g;`eXCwvElT6EW0WRf><9_{TPbbWDR4tL~3e`GI| zEl16q5S5BD3W{{hX~w8W_JvMis?@H+6-g=iVv0mv-2)~jV)uY@zCR;-a$s`@f8rMW zMtsA7Qd!)0gQXa!!83YHGnx7(_65qu!G1&ArG-OE^x^1Asz0~}d~AWGkw^xZOFld7 zp?!{phSq4oif5VnaQPRlW3#Ei!vEBhHI2qvHRrfpqW=XslgpjBwO`XGJw=LR=G0mU zDdN6TV<+YS@iRe0*TR0#$J(t`d>Zme+oN6h5ed)c;=X*KO zhD{GgtYI1G$eDruS~%Fcj5CZf;@oq7!nyVRjUT#l91a4HrO?34qiC&pR4s9*-OqcR zy-5w=y49FqY|Hi`Ds#kMo^8VP0k%rcTJENB#`FgKt<6rirW_$cBw+gX zAT;WkpZ6W=7E(7^E}TdYW8drQsTUu&b#@V620uW=+aKfpl~Op>ylfDQVoO)0Y)Gd= zD0=Nbr|6JJdEmZT&-u(Rw)S=L_pOht39m1FVc;ToVSk}zob-BqtW7+9d))KRsnl1t z(5kfuC&cR3K+NLU7O{k`x5Ftf=4OTXdUZ&a?>CA)1?jPESrA8xPz4oRF2dZTL|{lz z?F^|d6;vh)D^7TqzZ#kAVv6>`m*<4-JFY>&g2^=vR9QkIvtvhak-ZRcQ|5$xUxN6- z-Sz17xtz$8-!$M;g6PwVIB6@MoZCyw1HIIp5X}b}tr>p`e}DRho-!eiD1gQ%VVB;SQ(;!OjEjGc*-9}MJ{-K{q(%^SHzJ^`A8M>@qc(vzN)-gzF1*nKLGqgWS4Y88y4 zG!=|1Bg9qVjYbTkynuHowOS^iurCmzzq9(=4L zwT#oL1z&3QX$vr6qkNVW=2tIZ0z+;)vJ!#PT%4OZ)h<)?=2K}b!jG;KQ@6^BE*#_! zrFYGCl1|MN8D$AyorfC9)fJ^q)yhl8RQDYj zQj?9OM(@Iv>flYXqlSO1S6N%nwo@dFHeHNhOx%#`9HG|xa@fS zsJqCeSc0SBPD;z*EVr^=_G{y7FyBfwjg@^kDg1OudeFE;wz+@$tDR0lN-uS({_DQB1oI4IUPDrfyZ{{P9UGKr(9Hwwm6QS1UbG z4_C7GFwXPI8o$+pKxlFN?45lclGb@4%b@g$d#srQC>Ip)hfk5H_%>z-F@ zz?eJXx2(*W;dk=0(F4}yb^^QT7LL>tsZ2Q6M`*K`eKw*F_7lKt5w)I5q}s3AjLSmv>izEH(iQX$ck%V1!^s& zO3>YL>I|n8UMe_ysrVG#No~d=2dDKNR#NAqE&b%cUzqmz9p-h-OkpqL4{uX4fZm65 z+03#k^?22Ki|d%Q)L6%z?nMnLZ14%M-dF(} z_+2@FJbp1_rmwBvl6&P?)l3X!e`vBXGikk_dBRTM+5}Rj`Ux{ylS80HIHTTH&EE9W&lz^o= zS7Ga>k`|jA4n9J)A68`1D;$V#>8#|zphjw_M0(SGst`il ztZ?3J#mta<9f^zI9=v{*J>DjE%q0%5-{0L1MO31VRcPO%@Ejb)#U77+Hb<$P!SIMa zXUjlO713wiB%yrNTt^d0o8xWU+d{!5#-7@x3pB_<6x>#IhrStrnMo^ zZDHGav0&c0LI~;n#Vl6BVTd<@&-|YBu*dN*h+Lecoh#$U{RWbKt4bJ3Ywr5hS(cMo zpui&W9;jm0*CKw*Tzo>wR0KOJA=!xqh?vIKa|FbSJ~%7DdIkALQ)Lhrrjng}BuhKvEnIUeTh~Bb zE9nZfkU_@K``VK8IRgv3yFw#*VW~nsD~K*0FP0wjnv@BnnhgD}J?%YLN?r2Y$<%ree?_2J~!91f}E`2}Wn#vlNl;zvjCvm_DEUP0-%d?MPgCzOj`5+LXbV zB?th@HZ(;m=p03X$Z;Wp7+()lQ*$>pUl1Mt*MHE&A07mdRu+6~>a1@sqp2KPm1-qo)VOJh4=iK_d|h{405cC;LH z8nF0_bKSMLs=TSjPG7q=qAF=c_PJg)!6pHlucqdLUr{wsO4Yoy4n~bOwn9Nq^Y&lU zP*sxT%C!A;+crci)bZfioUbfJMiM5q8cy^-1NRTd)crM>aHcVnOdZ{7E}l$4h^MQ- zsBx=bz)a{Ra;k;$@0-4C0DgmTaxNi_F$(8MMU9bO|grDKpShjHvrNGG=u+d7qaGH|}(qJ9T7#8Wd zF$aYQ0_ne|No-u1(R{>zV8%-ug#LcZjF!JMgYDDkDnJeY;`EX6e!a!Pf z0_@UTC}CYLD$2;f^5jVG1*Ti5Fv#suk~=PA9hWxFoG(cytMhI1PbfUnn*tJwsiXKe z47(H%dDu7LLwchXUNKjb84d#a0HNz;!G#_sD9>pnvv0I=LLvw>9-kH#oG}PjF=Yqr zX=mP@HDTuPHCdQyI%Kms%VbYaw7yPHGePC11Xp-gQiZyfAW%RyDHWSkqY0d-f6)6j ziImY4E|9%i;VSo-of}q{G-|BXb(P#ci=ed`DRuh2&CKor1Fd}efQAOJ{#p;3HQN)c zcoXA{rc-oVR#&;TcuyUB%Rl8BWD8%38cp3MDYt!l;6X!(ut`@yIrvTf;Bm{)bo08H zxS@Ez6*HosHoSB+NS*H5HYOn+HYecP6uOk5%$4OB2qD~ca$*u?2+2|V(L&k}ot35c zKY}3MiVTp)A<#M>6y!yj1U)J?^2o8jg1lN&a0E!@@kDC1NcgoGmw#SsGUz8nx~VKn zAeWt=`T6jNTjDese^LvUDO*6MzbU>GU9vLFbFw&|>(7Y%`2>T382fACc&IQ7qauCN z{+s@3qO8lWHhkEcn}eI$!&Wlq%R}QFGA8>^#{^O1gWF%~zy-xQ(uB!*mp+%^K}>GT zRQ`}LvMGO8zy4JwPI$di%3VmA+G`H_7*XJ; zl1U_WV$(nRg#RlOFH1IPcAsPfZa7Nm)r1KoE_O6?GB+leZgVDp;5lB;Y^6dHs+N}F z$_9IRQ>rn=L=JLf4f2IlwuhQ5g!wzh;MXn{?A$AhlW^Yauew9!`hAVo2XY3A&T#4| zwsU6A=f}bzH$T zjeImD-QPgDA~G7o2{5N9FIXwAvCsvrF0FAPLZX0%myl$Qr2!#!tUXT0FucJhoT$2} zOVZ0;8rF+A`R*@z`9&9kP3o%t$51D-y_on51hSO< z?@x9955-mK&|EjDFSjy!a7nDNm%GCk%|$PH>nG?Tk79j9mg<#2OM~}vyZ&6|ADvjX zguQ6l40AR;*u?;89rephhEdHUOC=-op8D_l&$RbsF^WE zV(r6rX~P~^-RfNjAa*zA6=u9|d&-?yhH5M8lXfC>-hMpQ#-6^xY3;V>69SOl;%&x&+B;q2Dk1}cf2q_dIhglP+Ad>&dQ=p^zqwSyPY=fx*>h{9zuzS zTeC^&t!hpmGVJ-ge)vPG9Wr(f6j}QN2niEc!w%H6hMX@M__*I;lhXs_Dfd}4rmcpe zU0Nu{n+XovJA#KAC@j*4+Wsa}I~Ly_tk0ZTJ%h%&>xF341Da3Pgcoh{GNJbu?nNC^ zrluBPSvbn=Up@X(1}Hil#pahvFn|`s69oO{YT1q%y;fioPDTDBB|J-zgb*t}SN>Wp zkxMT`5_3nMcv?+w{)nrsflH2(m!&l+?6oAgli8vz4i+k?{cvM6f^w0Uc#nRU=PSqg zvcD3X_2S7iQI>b@NKUm~7|B5fJNh^)T>wc`l^Z(6rm#pjN?&c&~nprrgD7zZ_Mj4yNS1IZZrfD{y_7|fH74r zoe1&X4X2nh_hFw37pf{tf%6gYR6}ee_~C~VDO<$y*NvpZJ26dHX+Bh}Rekin2`;yN zeb#)j;^wMB-W|GV;r?-_V&>EL#Go5wO!^-~bo=W!NWEmnhp=lTKp6392Fl#?^c5uP zJ#e=9;}NS&Ax$Xk_X582jU=c5nN;Xa%NY5#7_o9a53T{O92AcN{iizJS3)aY*@Kh{ z2ggm=S{I(yMFl?)K-;`D?C$8mk8H; zJT~n$`lvXQ$0>VRY5WB5*^V-Oyh16kOw`KhfRxbNRY}Byhs%zpqZdqLJ2UG}@W}0b zeR)(+OLgNn~OLure`)aC60wNACfCOq;g->nsH z5l1CSRDG+H4LL~Lvk(@u;MUz0*<6hwu5G#eg_GujZf5_!=;I9%USTt0&i49GuDqAV zkM*-ovmko zne%I3XB*XirQICzV7_*pf%n=rbXxLGz<$DVp$JfqQcCuwl#}G{;Yu`+XY6=h{jyM_zu2K|ikEoN{rl7!$3r)5IK|vS)qz<&RC0p4O?|;?vowY^(ycP? z%pm5k?f0(vZ!F?B`QOv8C-28J|@skK5#F&E@R16@3i!I9G9ENVhk&fr>+ggd&$8y@dpBD0q zvr39t;seACY*b0`JAOk8k>69-(n?2hz05|O^)mG6%@+8TvSB@=|MOD4ok0W=1Uo%1 z+RhjfN}Mq;!Yg*Sk|}8}gqNcgoNr6-<%M@4)p#;sH9G%I9e9u1k0<0`hCpDRP_Jqp z`Z+K4<`D+KD2`;T3Zqz$!sd8X(42k{R<-EN=Nf)hT1upn41>QesNEc)1Cy*t2o0V1 zYKXZ2rK5?31~=r$>Dy%16^U+3POdcCM9#VU$0K+e(9hs^1oc@UzsK)G7 z^4(=3lLyE82p95t<9Zf7%U4h(9c`PuVq|##;&B&8FJiQ!tA2y(LgP5Tr9%OIX-d|j zlTu#am4jJ-ofV>)waO}lk7WdT*Ny{ubU~sm!KXiS$I?rK8ayo$PM!%oI9#%Dq+*DH zRaWgq^VUz!`R|-|4NME4gm^{^<5~Iv19!kBD7PhrO4(jxNS#^@k1FcHh)hnad-Y8J z`|lF(V@DD`uQS9zGKEI6Dul8aRridTfcK;?X^5_Y zonJ$)tK7adOwLn#Xj@ugo;#8tA}?U5f%M^mT7nqKkTlU;jk2!%%h(9HO@Xy>RL6$n zmHo9CjO||pujkQ>y>O!QZ-i%)>*B~~K9~^5Ve^y~Du~86I3TboiaQ8050zN)V~AdC z5hjueYH8Fxh0wA{R8KV-2K?{{HUqt~V{$%|L#I6WdWgrcJebj!!%iR-qF3?30$=<) zFgs8F$w!}M=*Xojc--LB7YwdZ|J|S1F+mnJf4zA|Y%(y#PE0cykiA*+*%akRpqm>b z5fUw-=4mgrn1{!B?8D+|VpHyg;*mRR-yXgg*lQ&~PKD+$%}gkL#_iqdjQ9bZjW~R` z*&8=P@mI{H3TeNWr#{THj%VfkGu{aW3IpIT07hI;#F_>;cR(!B0;Bo57L96@%fO^E zF|snHB2Zi9fjg1=_=|nhDcnot*Q}6ZMts?)@$09a~BJ`gK$@{<@u5B7Pyw&LpH2?m6(P@bB5xz9KTS zN`=*tmj=|Zv@bRj?*VZ=nFr?Y1`Sf?sW0hxJyBR|rHnsFp$BH0D@4|_rlZ7fRUuXD zfIr5yoFQ3qOm+WVd_ILP((`M3R>G?_ z)K00|{b)a+ieuVCap5&sv~4Dvhz5plxWV=^lS&C|?dADA_og~2;SZvbbtA}XMx)~? zpKyW*sv0Hjb}0v48ISPCtMT(IDGvVKVcS@T?Wd7v$PI2mE0U{ug<(e9x{^RvTR+5D zQHKS)mel&gseVTS{A9m=cEhAB`L!G?7HxdDtcxCy@VArC^h?MW)$c1m!sKA@3Gu^< zzF7tJ2fxuIsm&?|(m}Kj-IUwb^JJ#22oXnr@$JQjVnrSdn_4C{RKL)@Sz5s6O(Kb< zkaeSDoq9KU!W%YXs!Y2S0`2M?E(W$4i`mKHz>xkn1(4^~9;!|~4&Aw#tX9qpaa-|p z!HGni>&$}piT?VWV>S&##v*fubRk*!3iXfhoOXJzWXkIbgc_FqaHJ#JHbjZ|dHU;Y zRwM6w>6DKXvc+QfW$Lga4ri?lO{j0?yun zdJLys7Ss$%i)Pz|Pv_LgT|*6GbqclYBsz&+T!dO&pDbUQ0V`IkhW|003`-x3LGU49 zDFt05U>>RMQlWIo!m4Le^O~J6;x)wSc#?qbQ-A)QQ+nTQw80c^QzC7Yp%;C7%6b(UHai-sTWat< zHgK1ke$#CQUax)49i0M64}Z6G1(?LXZ03!1+9Ew`rSiZN%GJrZOg{R7t_tJ2=5EnB zV9R<#aiHpF(Yhftv==5tH18$|C81R#Rd6_^ZFtV)2zn5@`Lr!sOkjlaGxSon$sYHi-y^Q24Al`mwG-=AjKaFAM2PzklVtCZ#FETw+UtYiQMz`)2{}jM}pU< z)91uTc5LMowmu7E&F|LrD`kQTa~$LW=67GShO1FXCiHPTQ-l6N?i8i-J!>>T7gHy| zM$0(j_l?+vFK|)D7cD_e&$({AW-S<(XVG}6%ZWb-#P&6T6Y1j9N0V3p-y^{WisV>{ z-|x8&WBIa3i|mW3ioPd{1|OzHZ!Y5Tn!}BmqBY28py{;A`MJ=Ftc-qi#W6G~N2tUV z3ZhfYiV?z-@0JyWOOXp%>T)z$-yH{1n2luB-dk$yfmsn&fL9p7<4vlm?`}%08y2!y z@GUT9TMC!3bIWaHv%IHzz*#tJLXB-1GM0wi_Kc=~=^VvfJ+G0MKIHN>83w+eAV^&Jm|I;-nW)b@7d8Sb)1&;|0_D$itHA>c& zLpy;UHUMN}73-p3V%>9a?#b0yfd{4E-Z@~49NuqnT+JH{85e`hwT|WAH43$ehn>J` zB98AAM?beG3KF89#F)d$Th+48X}7_(@fRitL>dgl_dAu+cAXtxQkr5%8-cfcm~ulo z$ww)Q63dF-7EKCOR~0Sg6RTdn?^ShSdgU*X>c}DWswD+_ZidM-i;>A_Nq| zdOY*rBG1-380uerUMY8e25;F)c)6o&C{y>uhX1a5h9wq>KZ_CH&sL!Ibxm;XkgW-oQUqL23uZPp^LQ z55;$_gaL+sxY?>%xKH7swp;wE%*tbR9810rq#XE-qLe8b7|=>C2$O! z1b`UL{O&{$hoY!ocDnSkdjz?o-=RRseei{b%n&zlKa5Nl!T}L|Jd1N)MrGU!KI~Ki zwee-AvLw=60FL!)B9arqz-4jJ)0`}@-ExLxGpui32#asUkCmKJzWD8Nu=AXyM-WML zmirgX`>-9=WcQRq70Vw^4e><_3CDR{XTn}58LmaTWYkM)QYI@m*D2zp9?j(Txcl8R z{U<8Pcb1;@xC(t_&n+|$3)=UNk{l3@J}Dig#M=FdJth-6TGOM+VcFy9it?Bz-=ZwY ziB+mr9;667pZR8#K%`kAaH!Ee@VuQKb(shI_*7)m6kj9O?asShdxG{;I4u8qDAh#3 z-0wSKVm)4bl=KxxZ;x0P^1OqMW>(sYpcP0dKH}D#`8?K^D82WBguvWkCr} zxb#1FD)oJX#5?=$h4HP77?>Zk=1ja!u%oOvc!#}JgNZ72Wd=>_t+Td8ORB7a)DrOoic5y?-tYYsv^kjIMj(rFwhb%8~yYYm{m- zdnYl5R4={Q!1pt6k2R??fBE-CtRzH9c9?apCHG=Ms!x;NpK{QA;X0c_l+4p=Q2+@-P5_0KUmn_KHw4KR zc|me<3D_pHyk>DJwDI5j#PKiPy&N3{Zj7?C(+A&u`8gDCF79)g?p--!*pxbgwk?!ixqPVPNCSsDm+%{XmX*!v^gaVzsov<^6CO<_LpgFNRU<~FK*OUW{?E3f^tMF zVOkp~y%vj?u9h-EM}-J*9EUPt3HLlo3$;4Qbhv4gHJwJUL9Da+7o5aIPR1$}>E#dU z$oGGzOlooyq1hbnMvPz}{adf>r9>neun`iC-{7Rm#2~s}E}uAqk4-yTv!3j&h1-O& zCcf}SFVtg1Y5P7b`rtKIU+_+`inxJLuDB@^2^vgO!A12}2iTyb(Lj4$Rr~$M>WfoV zq!BjRhvh7*KH;3z8jV5n(Df9%g!2`5s;)>H))oVz%`78Mi%aeuSBwC1M%v-;A39b> z*S)YOhNaVmm6&%-?OCqnhdzum=ot?}X;YV&w&0k_0hu3$ zv+||EZ8ypyXy=cOgBupH?)$UT}O@GLF+`preEskX(q<` zRaz-$n#11%aq>OsnTD2>vQ3K?KI)PMFW1C}ZzUkAc)E5W-UP{Wz2)VUDeL3-r70b zg{Vx}?axbiDaaDu4#uvbU`&B*g{&MWY0`*rAk>Fy$nYCF!c$ay@0TsCq* zuKW>*aUYDsO0aoeI8%*(MLaow?jeO9&KH?=IVuPf+N=xk2Ax+rt#_*cFlS2oh~@NL zP{ss9jrIRY#~w6!1wbQ7srHyCxx((@(?JZwR9`)VG2l(PCz-e;ayAuoON7mscgL>+c#>|Jv*u{q2*GTB9@#jXBz~9Qp z5X7AL724Bvy&bbo*>1kq|5R$j!>B|?=f23*{FOTXZn$DvisLWZDC|qza#F9(CI=#% z*QCB)6GETvQC4yccyp*jAm33zIZs%5#;;L7G1pT)A(!aYrRx%* z{b*|U+~^QZCxRf!gd;?n%-p*AMx4Vn#0_^4AeY(Zw=5J|ppjuNe^g_NS%N zXp}Z(xy=;d1qB!B_06_4%-BqV*aIGtOixeCLK&RR6h}s6$@?WzvS0C?T8Bo`-YzRF;pccjv$Fy z(eK0;FSwf7RE7hGVS{Y=_L+rrtv_b#R3_Z zXLwHafa%ghalpkc6pnB^eI=F-TDK~DE?s74WCF?=LI|b|1=OO%*kCn2v+-%}P+@-a z65n-ep^-4zq^2->bxX@yqDIzPpWhPChGET2c8Opu*G{(YBp;LO2c;1}XTWU_b_XpR zb_>iiRberJS|HEU*km`uHK&BVthH#25lkC@1yilhC&bm!+#D!n*81R_8oE#rx-e90 z;i`Aim8CVdpJe^<3{9=NuGB$mX-13cs@xIb#& z3&J0ZYuy!Fv&1*!kLBHIJjz_npfelbE7Sbs8gYFwoM1b6PE#kPInFSb0Og(bFU8G& zKWHC@$ZbFUSSwewzI~wgHMQSsxkreK`;1V($VK+i9uBfOkFP=c-21kyn6{kbPd4B1 zPzfYr9kydB&BRbw7K4~)HEVlo0r<~M9bv1S5p;7R6`Trkrb`wDRlGf^-L(bu&btPD z0UnKjGyWaR`U}`UWUBx8lyH9;?j(KNC_6L`(ASKA1)F*JI-6&J%mtY0#60v_KiVUo zzErc_lUXF#_fovhhvusID_`zR?^&u{zDqs8>B`DMzw7%*qi)imh4OPF-0=I_iO{Y* zxacDoG|Ro;ud<({`Ao(ex?&6i70SsN^`A?i|9JFog+@9k@BEQL+&ZK8@eJdSlF$Eo zf)6IemyYM2Bp(QD9-sI71L*tLbk}5tAP`@yZ;)&8MGs=A{DZ*spHFZe`Rg0i{X*3M z`!-DKUkCfcV4W2r$l?upo4ZZ`!@@@VKYJ@8pSe(orIP4<>|ZD5IZOTz=(+lXHDvd> zL?->=Gk1xSC6M-7gVv4N_F%hhojJS3$-0ys?T}ef`-b6$;6b_XZ_6NqVb5y=Ecbhx ztwD=njVK=j_-fDDxxZyx9(Yle@QL_1-Py;Gtqky@Zme`2BlDJ2Q_e}ED?NKsrdZam zS77tt|1aL&GAOQSTiXqR1eeC$El7|A3mRO4CqQrs65L&aH}38h2u_g3-8DdPY20aO ztZ{B<@3X(N&#iM#zWe9?qo}UxT63*6dyesp_kF*C{&E{TtrXxtgWPo{qWO7Hk!&xL z9q{WBNzXI8<;qb>d#YoL&hZ@_?^-X!$}Hii@j~)M(+`?;fH(h16{B3=M2KQMH=&sS z>_B2Pa&GS^5$|wC;2XZ@RfNT!_mz2E>*$f(ixsZ54mLe@CHLN)0X`KRyUb7h#tfo| zkHAAF7=6KQ_T)M98xFn(fSu-!%TYg6ka)0;C(W?NE#B1KzadSt$7KT#?0-tp0ysJB zxj-P3iG4<)aAg^v$>E$`%$FI^RRPvnQ^Er4zKL1@CWzN?Vn*ryi)T-3eenEMbjOWR zGl@x~!#bM?NO)RVdVP8$OY16NhwRYCX~z}6inD6pz?F)#p>GvGj+gAWl-J6$#;Gft69JwI)Jn=ws8C=XJ zN0R)4By0bXw`#pARP%JTkGC~PGa$1t2XP8TiBdH1AJ7x52fq_IT=$NASSO3fBdnWiEKlx?~hjUEB91yPyu{DRzBz40MgQB{=0a5O&Y$O)r z@=cPs=YAIKtw#P2x@PEmWTwN5}l5{m(+P(wAJryq_ zMh#?$u2SZDcIsC+ny`_OI?4X>x_v{^2f+6@{jRIEB|qD~9KRNfh5aFQi2Hw4>&0FC zQ?2)d<<%_vg!Dp4;;RX+oV~9?tl~>3-JCz9H$sDfM4emovkF1Gl)~N9uK%D)l2wbG z&ri$=sF}vd{l7vcFMaCD0D@#8_SSX)T$eW6I*IWx72QYQLO)1L!v^^R_)_g%U{ckTn_Z(~5=QmM^Mq2ex zgaKtfyHc_p5VL@o_ZP|Rrxq(Fe&CeLk9Brg1Ln6^V1q3%>Cu5HC{rVo9ve`uZ(a0@ zm6NzO@>Zqx`h#+@{arQQI9I0Q+#tn~fD=W}`g$rEZs1P}DXlGD4PR?)!=+w{U>#&O zxpVu1rSH-oJF+sv=$c*+JcrdYp1j^hX9?#%4YAz&7Ynppsy=mJSEG*m{f4NAsn?$l z_EVAE_R12AFCNvf_{(!*PASlmHW_qb=5E}_5Ojn{rg+!6(p<~WgCBgUs;qixC|mEI z#h}b>!)*0QEr;*-x@56gi{qpVCb4ZrS3?;#4iTk)6IXSy7J zdhc{UnS8(5g#_cp8__Jo$hlzCOn{pijaHjiy>XF90B|34h>;JNFf8Su#?=wnl(y?u zCUhRlj!RG^STjgGZqOyN_La%+uam(4*jv1e9t^;K=vGhskMA~NS|a8Lm>jPgSydY= zu4_P;+ba@-wOX9k^&2g~V1DYa{d6YmTFG0YRZt59BFY$y_0erY=YF=XnhX37k%BlW zi2;p+Di}cKww1-0!YyZS>#(khvda=uwce7p(mkO|c3pEuPqH{2FRB($qVZ-%c=!d5 zOte971(2b!(W?NcitQlo>le!GotO~Z8^@a%R!x=CjGD9%!L{i5J|CiJT!peDU|tC) zk{PRSqMWC-Egb>vS6{nRW^;hyeRqn5H!i>YvUK03@#PS1q;L(IsJy~AO1wrT~@OyUJ`>G_3CloFc@ zXaGKfrRlyad#Z+kwa)J!*iw)BCmC5|)9G{{1Px2R54KPxx;@eOvcGP>Bi6AL3{I02 zIaD8V-DVT9NC*vODxfugVGSaK-UOQSv+1%sDo-SxmryC=6i0rP%*QsjtlHE_d;x}x z#zOP}U1e(1H7*`(AuT6sIk!9Oslf8RB(~@EHjW1TB8!PpO=z$NU+eM3;ElJaZP_o~ zE-6|cNhBVApm_^Jn4})(sXeV@jM^MYK1~P}WFpQ}cJgNMb9oykp%S&@7@&Z%lNmD) z-1xIdA}@bh4ryB}Mg%`NFp>5Ns3}t!-#iuWqwE$Fb~_Gx%K0pY zp#>;+r=Qw9?O~U0&5Sm;QFh|YPkq6#@zOUQ@rI)86=@RVuX!81W2$<&<8(_}OO`14 zj!C<>u`dF<`_U-{_DjvvC?zAE*MS3E)fNaf7zpIN9do<9=V?Y}ovMdL6BAUUyI1Yr zlqcwZLIx>nVL*HNni`;7uQ?p!aBh%V;}MyZp+K(E5$ByvWOp1H%WLu)=cc|XjDO*p z`i1fF&mu$Glw7EqMc7}M&!&mbN0ekSZKw+N8xm3^7C(jdveTC;KW)%+NxLw9^ed#ANzJ@7ZI9XfwL7*nTW>7; z1x?|s73S~YoK^MT7Y_e+WEK*HpHpRO9>~TYp%Z&CsbMo64^h|oF(Q-(_Hze&Rf3WA z4O*EC$w9gO)<)+kMGaRWm3gZ?bX8W0X^H z4u&jPh?J+7ptfDQ*vPghEvsW5`mptqAQMprn)Ik&TiQO-*#}sFyO@I23!73gFin7` z1EXi~J|pz>QTDb$a(^jz>+!V!@@TxuqyV(Jk$LqXxG7bHo1U_QmOnYOcl-1HneO?8 zYgqCv%=60bh*hyL@+@vd_l<7#&Hf#BDd-4AM7!nl@tr+?@A(zANC%Oa1+R+?9zfC?*w7hsBDPsIj>-b{#d3QZ86-Z49EuKZr`DbDgw zJ;^kQHy=}T$&5yIVx#m}rDb0I>Yb)c`GpSUpTLbdpDAs8#}ChvSB47B?4Gl`V65nG zXLG#LRQgRmAAyM1Cu)6e>kJNGv+bbVKvVC*dxBm zN6wQ54Oi!)sd8c)@^Xu!2_@8hiE8A5NbI{{FF)gxyD&F~1j-6syQuGvAhkIdn)>wO zBKm;2$Dv^qUJq)HjG)M>op>V*n&&=>`1l-9hcItiSve8N z&-u5RL*(7^sJjqC)_UC)0#!5BT4juv@~Q>lTeX)z<5TpB?s&RH%)FbD7|741Q57F| zrY@NY?fe)D>lagqSG+Wc9N$O|vg2@k?vrqPg3vV^+D-<&Etborv`)UYtiRovpRr$Z z^Xb%O%qxmusHX@9+>Xy@Lvsy3n%XO( zJ*~@s{-Bv)PcWemYC{Bzl6Q|vhG|!uMcXDDV0_-OMXHHGmA&`t1#J8C3-{t19j-`J0D^l2 zss1V_60x2iF!Bc3($~$T>Y7;@<749fiOYooX}|=lbz>zshloR!!BM!Ue=}N=kKY4R z=O=o-Gv08^Z8O}Y^|bq)a`#F{0jWFxz|O%8kwMDUJ1+g|1uiL}Y3wlb+iUTz!+;j! zZ`1?H^-HmU(%@8sV{Vnn>=(C7YE3fkEtnBPU>~KsZ{|$yQvd5nDcn-%lpJ24!@fP@ z9h6P156Gxq&w<$!sJGi=1_+j0Na3MhO(eziA0BjSak=tpF*QSItKV^9)zDg)dUy=o z+RmK5#U@y{4GguP^HUek&w;!Rj&TbI?dvPssrw}zzlahxL`twE>~wu}3t&?(Dp^grIXD>l@p4UE*H9CxX=3 zpD^|yU;{Wl9xuIGnb!YW1Uul`nsc+vcQmNC|Jrl6AT{{&+98l4u>^qumq!fgnxgdC zNk^4P4sh?TCddVLaW%7gfs;p|gWv>C=p`1zKSf+57b7Qg*tgO2z)E9HqmV?MFgAn~Vg^G4DgU7{w;E`2;AcHDZy zC~Q@AzQ)*wL=@yHi)f{rg#3jB%?f;O8^*hH}p4%}qBPc1*wil~A$n0DSrn zmv8ud5))$x*Khe>D`$tuVV}@_n8H4qo5r>XdAXi_)7N&;SQMfjgKv`Rk_x#T-Bjx? zTq+$o{krsx_*m8wIic1oB`Czej_k~OfD=jU6Z85eQ|Tv?SD3#CiSc-#K4A)LSe-Kk zbp64IDN5x1R7VoBDcejX zt4V@%$Xw$Quy5jH1Zu5ox)K|}~zUY!OhH3CEvk1(W*{)SI z?O*#?3i*}@(}*ufpi;c1>_(p0A0c)CXvL|wQv(e3OYI=tUudQ#ARtBrd0e8;#TRX593$ip=K?n62Q&{cl*#EN$a zmd$3InM~)S|im-DPfj|^}9#>I)4J}7n1#XCJr&RF7y zo1f~?YCt|jC2>B|VvN+%-F%I=dzk1}gt}ro1Oq{HvZVfG*fsAM(4Ug=16QN1m{Ssu zUxb5dMRqjQKR)kcE}+$jam@7+MBad)PM*u-Nl4T%uw$a;(GhD<3i?|p;BFL%Pl8uy z%&6Dn>2zlKpMaSQ!`?GfT&+0`FGtuDCYPqeST%Kw#Z!SZ`?J;)SU$pVp}ESb{LD^b81D*6b&ARzxR#ke=Y$RBseFjR zoAVWBm%7_(;x0}$yx#ddgJTq)P1w1YVNlsFGUHN(SacV?nTWEV+>6h37WV!rRyXKF z|GmKX*vH#uclzy*Ce)^_za?U(xNrmG$2>Y`(9-y_Qu+AM;mtiCF+Z-g7`KneF)29% zB5RWfaMQQHw48-F;Kx_RtqhKj>s3BRsW~;cVIllxp0=4U=^D$#b!C3Kw=>E=ZYIdu zdw7!8M?aJWc|B@%3xf`;nx%}HthYl?_m2@Y86Mlj^s3ymo#*7PoyfjN0KH^k+OBQC z90+t6uzC-jP`bSg31MXMY~>wCM;@nQ#iK*aYOAXVOEZD_X!Y05zs;5p ze8ceM8iuk}I@gp#sheyM$#Ihg?$!k1y3x!A&*KbrwKL$sEU|J=`@dPHI%txSjw*b> z>p>h-SkiaiG7Sb)#g1p)&M|wjaTgwmHP}V4w?{+lR zGa1u2^J#D(j0-F+n$7?qcGLwBM<$}?_l_*zi)CbZ<~^E)xvDO0l!|jHEv7-;Un*-y zyZTKc_&)k{UUCWjwY%DjsAU)9`Fg#| z>n`)LL&rx1$SLYW^$9%TDbe1hW5fqY$k#o%!MhTJeZiVJn}WHySy?lAMvWO#c@i8U z>_ogP?rjlL>z9U7`ju}AIAPDjm<}>{4X9q}-d7)<91_ySs!%OoB%&J+oL;XUZx(Xg ziB%mT8pZvjY#Z<{XUY6IP9b3lE(lDE3H(WDiiZ}F$zl^TGe2>!+h3@VcEa`k>NtAK zNA#;IlPas3-}0{m?e{&j=g`(8frE&t04;NSR@)L;AH|M7t}f`8*r zgnxZQ{)2`2YyKO5`u}*5Z7(mn9y*Ndk${<3*zWAThk3p?LP)U3S?PICaU;DN+ph0~eGXN~su|?02wRVSnK%QYV-v!U$y}G67!|vhbR)@*QtTv}t5Ilcw2SnDTd8^P zAAOSK_|SLHuar*YCb{tDf4x=yIT`S=s=(_&-pq#>#*3$wMub+|8}q_@_13!(-z)k6 zuct9f>vHt2Np@EI$J6(9T0d1XNoDJ%No`obfy%ZqYZ3W2sVYs#R4lt0Aft8#!$XOE zh+1ZPa=@EI!kB~FJ85LsBa!G6^TPEHkK(XnqRY`qU~Ay+8qgrPRF~Qc_LW*halGwz zRX;F^fKMe+Gici(#wPeQ-}hkzT-9!tr~#IgH^Yk0*RPz|w-Jdb;FC~|O$e8x)Z?37 zRQqfdA(ZW|FSJx|R>j)Gxa6amurx=sp7jIL%EE>ld$Bd_i$oY(bZo{=AN63 zmI6OgPE&4_9uSD^OOJ(Vf6$5z+ja`bh;46dAMNZHW@6VmH|Yuf=$*Bl4MWMoc(aSRx}#Oa-xAM zAR07}ma^DdSF)e|vrCg|P1sZo2nGZ`-%w}nRG4=ujnfK9$gpQ=J#7hHO-HZzRvp2p zUM6o0<)pn6bu!Ua)^0>d6Hi!v5MaYT@z=eaXNKRVa%b`hfU;jq0isuL>{_z-)8jYr zi7Jq2B2~<9ldD|V4ch>p(Bx5J2Y<=gnonf^gXDGtPqr%Fg@Ax^&_D%3@PK*K?7M39 z9j}q{x1Z7V`wP7wli!k%IgBq$#qmc` zZN*=1+!}j-v2P=GQevKzP8W=e&5V@e=<0}GW3I;k%g!Vu^2gzzHb=$41xxf%n3SYL z2TdyE=Eu$R@rQcs4y8Mac!P7KyX$~E>hTl`=_bTIQsPd!piW!_z%_$wTP6pd-N*)k zzoerWo`mEV4wim;s69t(1qy`L_<$19vmwy2vowC%@vgw04uk8tFTVDra zg}sM{c+p<(x-m{~r^xjt|X{nXyZ>~Zty*+BAs&uj^s`KeZKM`>wPkvPQ=dg1w z%?25*zbUkSy~Fwv!B{qyDFi7b)GHeW5`FCqwTBTdQ!s^NERV z-<1b{Qr@&B7d~b8+%1n|eyM%){Xl~MmoOQF<$@}v=d#6*S3LuK&e%JrD5L09-7st! zP7W^2mf68ZS>lI>Pbv28GHL3LL6U}g@{EpzuL#LE_giF~^oar+oy+mJ$4O*Ke->yg zVkUUh(!sz+Gju2yAimtX@03KpJuuiZGNLyyY;VwNpM{Nobk6F`cz0GDk|U(l7&s9rI?#>m}^$ssji^)=S$Np z#Kk+HnCjI_Hjq1;>_ghz#G2v}vp|$iz`6gd@O}=GPj%r)rpi3469h#mtmfB;&xa(e zXK`GXqQ8VLth}Yd#lF?=%@7!*^th4MbTp3(GZB`@x7kFumW~xqQ?oIBdFOjF!I%za!0f1(;T8Wz^is>}FH}Cki*<%Y-u08SRO*Qw0Q#rPQYu zSM(V{kNlK{?vghBZ~2hazt~C-aP&Pci>adhPT5vAth{PAxxW5hSeJSr7Ob~G*Av(F zmh2?Tq|%BVY1I$0qQ2%TZ2jsTzQ$WRS8e)A`0G7a;89_De3ygQ7PakuvTXAqMVR@W z8CJtcdS)R(e8<#DZPGm0o9y`3yY7QR_leV+dY5<6Pnam4qk36ufAcmF(uDuwyWC%e z)LxHd`AUvm`<=#AVXlXEtoEr~I~P`ZYX*@_Y(ihS^-b3UMC%NCRvRe4=v3_0FWA!< z6E<+9hYxC%DRX>uD$|QHcPJIc7qC+E-+7k^dKZ`7-ks!kqcKJU;13mQTOA|7aNSIP zptLquF0eLy8Eri>;5_HVWX*K7u6_pu9=*~6Q4_Y=U|x4nsMjd;5=g0ln?TPsaRQIa zrZ}AnF>~g^Rgx2fCDM$Usk=YV%4MJktY2GvV$44VuP0b#X=SyOycmijPPRX>qF=sS za!N7Qb7UijX{$^>#|IjU%1PFphbmNqkZB34#D^mRi4oNXWa&ZjfN>^%K^64}3dT{T zKF7(eam`ty`JcojB97)uO0pNv?%1Gi!zq0!;T_mgFZy~0wj`S{(nwSK~Q}Tqt>|!+@27D^~R44M1DAm-t`y4!aFqbWd~57#@DwIr|AsRSR>PYc<(+# zxvw}Ze>Emb%}bbK z#d@l!jM2W@fB-eQ(nCWM`ZDUqYEtQ^Q>fQ}pDx%e78-fuKyv=?49H2Rv6YVr_kp6K zac(YfYWmV5EGfgw;t1(m^Af3o+Gbu0jSstHtkKEURQC$@1=#kUDJ^q%`vz}*IM@Z3EuHMjn{A3RL#t*&$2nz z11ZHe7CAJNl4nvXZqB(y1=w>l4|b#?Bc4QY5)!F6bjut4UBc9w#a2|Q0wXM5o{_{K zGkadpZ+wBe)1pMyJiXMl8s~zA>Cbs9O3KZO&sWW^VS|uQKX>tJMXe$G?jmt6`yjiW zxRhGm+?G73V|Ya;^#DLBk8ZusVWg_=keBi0Ry7K8N!ymRhLqomQF(pe#%P1{5U8CD^@??5`vUVFI-z!b{Lkkxv%K zW^4c-TYhrdUsS!ju($7r96m|z-QL+E$PlG#l5JQy3Czx}62?r_Z$S4+L_0w1NM~@c zBaq_4aHzEq;)Og0H8be1p%kLouL%a?uV?rfWgPD_w;!Z1guQHGiO`k~j6Id+(;SR=39O5g9=$BZN`uV$OyAmIh= zBf@n}p+gz7wZm@R4^#l6bDSnkN8A*}SJde&*lP*AV_w2+W(d(XocW4ChlnF1WXk^B zLJLIP*6PRwkcdrtyA1)C=%=t;PG$gyybAA^RlKZ{MGz7AgHl(TOGH~qJ$ z4R}|1^Me@Z3|v=)?pctnvcFjMi)vdD2Cu!XsF!R>ckFo*fk(!CD#-m>*~Gc5{R%+3 z7z@!h_|_q?4zUsxAwH4fyF>2$;vS51 z_QeF!bTlTs@i)?UU;bx)t&wVNgX)le`PTWq%)L%I!Ru-1lxZGK|r zN;31_+9?mX!f<`j?j9RKP0SIH+i&wzVNmU0Cdg0`rA?B8*%G;9G4`= zHwP`6P6e;M$l#8ok}-nD0k)EHd6Mk1)|yi7TD8}_^7L+Gj`$VqxY(u~FI+xGK**#! z^o&74s$gcL-{vgYlM(gy7Z13McTT7>zIq(I9Qr~LPDv_rCKH-&3Xcee%S-)to9*6Y_L&}?f>dno5AV)4=w5NTL(4dzt_nHgJIt|j79>fxND%`{44;Cwu> znFw{<#t#Ix%}2a2_Ws#DaxIC$^|2t9xQdxbxslNHo-aJEcSY4lRs3OBuEfu0IbG_g zD(N>Cjee^$BHo^Og&U8_Fa#yL(AkQ8lYv+1d)O}pr-^Y! z@;>$o;`>DM_=pTakFCV2ofN{2ua}wt`?PCMA+csIRH&|_15X`Ymjr)n0s5Q6qfc4_ zE{n~e!*{IW`Ub#J{FQtz?DyMLn)WA_XkNqP!bKz&Va`AxM(kU*Fmrb+$RsH7A z3W}b%nriuLml%;I{WGV0uli8MjL54yY=#f*OMyDuv$pSxf}&=H4*GK6w~AGjWmh`Y zWu9@_s(#$eN4eTy7Y%tV_n49%O8v6g>b{*krp4ebQRCp2lz0a<25Dc@L(>(?N~Q!74V zP=)a~PQ9Q_iq+MAOT=`j+}twT=h{{Ac8QFXg4R~4UcZ2P3wPO$96frkh?Aj}C9SY~ zCmWQ6%=4IDm0lC7sRB{9$!QgLcsRv5*DbH*#Nju{>bL|Q@=mmHJ%PU z;e+SxwaPo*8~>(Ge<}lHJ;ZRH8)$6uJyD~@?YrxC>9LoFl-LqM9nc|^t5F+ksc>#<<>su+Szg9VmBlF1OIZ-O!%GVFvP9EAj z-tSE+tEe`scpqSH=>N>xxodELo2M-0mk{LGkw>xBb1uzs4!LCNa&U{Q!hIxN9BM_g zgk1ev7`tpU)c#)b1q)@!9?g~uR(#;Y;r&N`>zT8a4b*oFpt+hdflEoDuME~Vt2lte zxjuR3-2|ZGQshRe0&aKvi}1auT3R4Dx|hh~Mz)h8F^fM+n*`>})jWTIrg_qFl6~x9 z@YRP84*UD%V6$_=!AlHJS8?|Y8*+XcJo%}{gm*>qibzraM6?Rb+^?0!{2?ppuzHQx zxH%PL{iAtS^I8l|8|zp+t1~K5q^SGZa7L%*m}?NXeCc&e!yKn_i;6dpTD2N=(>fc} zt$yZXpb+ia=fx+Y6v1i-e99rYsd5QS9kUGbkWqzXZ597t?8xtfaK<~EWX>U zhID-tJ*U{y_+>;J%In+HCL40sJb4(KLP7ebj0ar`{YzJP-VnX!o~Ms~NdIQUXdkWz z4bn8bUCB#2=H|8w&vWyta* zGk#G2e1*b@w7fM6qhue=yD5;vHE>xb*pKhoO2Xr(5L+Y4_EPhVLDvv}LgZFXxv`KJgEqNP%nFb_B>$~7bRnC{G z2P(_=T-7Z!-tEfU#GuEW!usMZjmw`f&uqG4n$oBz{&r};frn>Ro>Sc~@!SvOJRxhl z8XcSc9=*}J%IH<4!~bLd`I%y9V<@WxtpxQo_JZn+=v5TqP{e&+HqnAE|A))wC`}2p zfATY_x85>}G~lSx9`Hs6<)mRY+H@ewjyqq?*oGrenVwE&Z71Tv9-+pI$9=oAa=~a+>55>-3A%e z9HNje|ImT*Lq)w-!WE0}DBYnLCw6n^HOj9eWcGGpJuC3(3)*q4L`tD(^-AD1;F{Ts zB3Vl^FTO`#GepsNEtisJ9VMZ4_Dw?XA^-H!p_W(QgCIAi0*Jr>ImoL zIB17^YfkxNaGNYif;dS+i)Y>|S_@HP)HsK0F*4+bc5r(O8JW-b7^8q7dkzhWM(q3p zG1T@j=iPQ*q>lyFQnWlrYM5^$1!?W&sYH3l$WH8kL->Rz@RhfoKpW}6qhUZm8eAua z0w@AmcpTb=`OD@J0~=#d@GFJ~&IAs=Nax;^O3~VN`Bbz)Q5HM>aW*^xMBd^)p(DXW&Op4sz0}<#ZHX4aTRX!_yiX^mn_RErJA7)iwchkPOvFv{c!E(|pKcAT zOiu9HcyQS_;JEkHQ8S119Cx~xSv)2n&#iAa)BY`W*#&&l9`oTn8xpkNpEtUHPYWuM z0a}aG(V;K`+CvGbm@*) zc^h}8+)Ck7k{7BR%wPzy+HRyW1b3b0`sQ6}eX*BB<0U%ZhsfSSSp!6vbB5RQVt{^x zyacyxn-4crz#}iooUOw&OnucH+$&~ML~P-m2*Ok_zWv2|YggCdao))3+Ydv8biA+N z5#|vuAxtaiXwl(aV^*&VEIxPW@j3I8xUvNw+6&nraqVpWNIv;b&iCJrH*`+G!?wI| zat%Jg*&+T!8*n4cJC}}dsc&gj)7JHFfg7I}JCE55F}c;~%$1^Smm+H3Z#HBE=GD77 zd%8zn_6z;&-V}0&9uT|Y?twDTV0X>9l$))#w!IIlJ=v!?igm_^(U{SJTJZmx?Y0{Kk-m&7)m2JAOy= zr4-VNaG#cS3Dv{(CZX`OHiK(v&b}O_G)P8C^J18rMxn9i-t>?7`qpryOp7075_1@= z$)jnvMQc(xyws1Otqj6ByX1NL zr^I-|h(yco2i;q{V667e#!tJyWu*vpjMp3EO*-je0)?LXy3+h)>resAd*G45vZd#j z*^VYskBv8drk1=3W$Z%oshMW1vOGHWcbi+VSYzg^*(#p(p$ij6<~(l|e*N6-Al79n zMCNv6MD0wamETqu9-?XOo$0#+%66}K7=18x`kxl)z4W*Ynm-f&K<<(RJnctvftS*^>64q~rc#I$9w+p!h0;WvhH8q`bHcQ~;_cx%%Zk zO0_++5c)vtos(0K_DLW+z)NA9(tDn8SE3Mhd;KisA!NwErJVNK7++;Vj{;5 zk6CZLBteNtmnDym?~an?1RbtuLOpE`F#7JlM8FaZw(5^lNBhQ!z4hu4pvJ|9|0Ke# z_0(j7n)d-DD*_QoOjSlBi?m-_*TLTkXb^)r=?l>he$2L1Q&g-Uw=u#fyP zrv;%^gevV-*a4@gMfTn!6l&CJ|~Bp*&C-sYNbqk5nAe&Gklz{2v%g z$8D3)ZiixH%$pU-J@@Ua0ek_spz-cJY)4MLfIUqL<}N-&fV(--S zxq}I-KgbfR;8dEbn={`tDI{2Sn~qMa{Vlc*QJ+y`J*icxwqY7CW*h8e)4%RX!7{H?K&NMyu44bN?9zSip6S#)uUS9vTj%@b(lY*i$bXMw3L*=kri<%)jv00 ze(&YzgX#*Sb>&_>1YJC|&$<=a2|8+8tY5c~L7{P;Yz8{tw(Xb`e(LOOfGpD5UJ9xW z;2z$oxLoTbUh)?$qkNa?rS>dAo-?&IE3O#uLN6AFTCAwgMV^G#p$ak>zYKpMNkC2$=EO;4jihm>4A{*S6c)}KnD-F) z3~PU9Fp}~8&I|G1={01%A}{Z+NF^4Eck$Is=bIog&6#(Y2q-vI(>tsy`i?;bdgjGj zg>KDO;2lv7`lM;BtWew>8Z?x7&BjM`K2t)-{W|oEZX)p+_v>MhGHMDgjU0aigVQJq z(sEDsG$o;Zdvx1u;^(fwzcmd0*vAi}T;ncRg{?h``T!qYSdtmN`jh;;yH;KWIMVrA zbTH=IV*jkrxLCAq#Nk3;B2tw4Qa%&P*X(6^!KW~Iq1_TBPBh`5&U|5ns{DuVuTCsR zmM_p$gBBgC;5s!Ml1>>9DJ^5*wi2>!y=!h+1y&A7sP|4m^T#Dds5H&j2H^0x#?ib^ z@JZ3o%&W3k%C7^v^>e>C*85{Jx}#dWwmfFRli^-I!py>YG0CY-^GV!fJrYVEK%@N^ zgDUeAl>OhWg!82)x;4|#9S&Qx$@iR_+XuEkXC($aw_@`xA%ZSZ22`Pe+)0St8_M+M zz@o|RFh+Tc3m(X?y&XQUrKL%R=g?51rPGF>l+|^Bely}7j9lhe7LGevPhG9~eJx64 zrhmWX`R%vXF5vLPn&Z*_A1KL$gl^+jKv#@biLeR~r_A?-7}tpNzu>c-L9usBHBNRf zDyd&CX%-pn$Y$7s5ad4dC+>rv_1xPiyH=91Vgy_C<0kBGZ1rR4Wwr(iwcF?n6<*=-~?-sa~5Sdr7$pD!=Rl>2P}!gZx3Aa%jzlG za6NU?l! z+gdn+qY8`0+~H@r@S3B|i}ro^Si};2xc?6VzhXU2Wka7f3U&NAA31JDVAtTJ0!*cv zOEaGuDI4z>CO`E)e}lOx?V>LM)uOG*0$hZX>@%+K^en^ngR|7fZ;?InB*+n?KOVS? zpHuY{AL<%Y=GT={f60_!+4=l6l$L+rGVCjZw0NxiD zmlM5TI!b#jzF-6>Vko}n8hTX~ zQdr>ieKyTJ8HI;y$HRtAn?`E|05^uNuL<{9Oc3Ms(wrS$mCg1l^)M`Ptx;0F1eY%I z`VX6i{_gj97v-ful>i8 z`1E=kt_T6&`%DFdkU-W%sym#$pH^ax@=;F7)beKLLY(3DcHz}z*Rd9Oq4L$RJTDRP zi3MB56aji(vLY{e2NvB~cp8^ zbjZNaCF8NwJ1-NyuY1=A+-M|LC60R?zIAiP-!tFy7OOJZJ9F>QE$xV-@AlEL%##XD z$ExP zv~+O~7-KNxpRGw7;HA%kMP8MpG6uM&wZamq_-RxRNh(Wn3{Pz&@~U~3M4PI3S~9F& z_*V3w%@gD1QkYgP>{;dBk`Lxag|~xu07!qb8c1tJPu~Q}#e^zpNPf+5&!w6Z{>AfN z1}`bk^90Q6CL@XsScf2jQrctq5_Lib`I3K?g$3iBEJj(jJ660yj)n*T!&M{(FRgoV z;)G9FG(Ewz1XZ@{^d>5~?&XIeXeo+u2uCj4*IKRKR3%A4)GO&>6+gmZ&2aftb(s_c zyiwx!eN?s!>Rmfm3*Fj(BBd=^U#Y0ibqbq8vN_H+6H+Pg%$*>F>UCcayzV1e`ITGb zNF`z&(FNez;gt09Ynut0av$=^4paBK${&traOw$bT`8{7&F9%#W8ocM)=2%#sL<#U z7v15HTH4U`q~9KYN1FE5H>s0YH$J(~v}1jF5Y!+goXmkNU9{Gxvlr@D#&V2EUTOm7M6Y$fZoq;^ij{{lTNjv5q(h_`@;dl+Wp#g zX(*doJAF<$SGX~VW{14TRd;Lm^CJD5HSWj5f~)0}?vxpdALNFlY$7{kT=2bfUA|brpfIyT;5Ba?i4?^^${75-@~k>&unwH z&+uJ>t2ttEu1wrn$=EI6xy5_GW?n<}p_CddZ_q*LF8*xf475Q^?(e>leL}vUk!-JN z`@%u78YPM`-bVt}cBM)Oy34~BUBPqXZ=vIU;I}9C>Lq@)_J_wdCyjvxrcA0_x$eQc z#_=NGjht$0MCruFo#3&}GQGR{fSZ)!9pf+%QM+3))%l`69ROhK9zG;rX)UEIZ1m$kP0g-mBZx|{K>U^oA2e5Q9w-Gk7L&twCOdZvBpgZohx@RkX#c2_@^jjoQ$bD zPOt_vC!dKyH9%Lg=^>?rs2p}ebHQes>CcLSyaU~bsJ>Bd=rP8l%*J{-hdPD2bskB~uTVWA%5;uR4Hk5}MC3*_RW z3h~b{Qqt)qlTo;ztnsx$^L!~*6=_4zK1*f|FHL#L2-5Kx@dv~mxkcpxVu|$cL?ql+}(mZ z1VV6kcXyZI?jGFT;dZ{Y_xe_yv-i$Ae@@lCb^p!o*;Ue zpW5i%*kFXt9Juk6Z?%ApjCv1as5qW-LQVb~4TIU?#)%;Gao!UlCl-+^Cc=9L! zxap(KlqtJmW%Nm`g# z4jor`Lp1=M7n5FLbeRPWZyF-*ECIflo-qj^Wgx=qe<}k>ehAdkX^4^2{jlA)yG#S2 zEX^F_Ka_Ysx%8eJ24i|l%TH1EYX87pn-|_#4cg}-?P%DlO8327MRC%zM~V7X@ZRM$ zVrx3A!6PJ~gx)#0Bq8+zht%|0+w73GrosJd{;Zh$!nyO1st<`zXaFI9xJ+!gR~Vfn z&Q^;^yUY;MThAh_=hE%Ok9F-59cFA!QYX)}BIw15pj&eVqn(~WKss6S+}IPE%y3a~}{p}ueoacHH`;I~4?gIXnfuL|!b33KtbuPI>!!6?v zyE1R@-*nGif2*59v&TtRu#T|AC>Nin@5m}WJ;W@iEy zc$@uPm4ysWsDxqX?JN8Xaf}NknAK9ZK5T=UmR4@0TKjR|tAU@wm?})u!}`C>;u*%1 z8|>46iGD_*KkC*nf*mWx@lm^az2)-M(nsx*;f|1Bq@9I2*O!W;p1HRz2}rMKCfeKz zzqY+V)S{#B*=xA*F6!$6DYU-s7`ize5tX1#&R#()^!Sk1N|xU{IFF0jqnX@>omljb z_0%;Qbl9J{F`-BYaRYK>-j3W@R)xfM{}eMik@{*lBqxl2%{^a)3+&Pl6pZu_SE z0zN!3FcyKcd9N1w#u!S3VJ=f96uQgb7f`Cy-4$(2k`;af8OGl9k{E#`Y&1ULlr*gy zWG+eoqWUfvO-I1ToxdnC0R38iB_<5MJGH&hDE5ORT2sK+(%rbrEy=T@PUnjRx9jB@J%s>&U4C}!8P~_98se!Vs zy9qo&I6y0$LEGeQptA=Aeovp zSo-9`_iL*OU+-Yg1>uKsab1U>p|7WWgfnu>q6F!~XPMVBg5>uH9zGdUHajiSVTaR) zdzm9pW?c5>arlBOHCt;zsIlNHQFs0yRf)EzsJBtjIaPL3Oji$oD5c+V0%fQ4zmc$p1vGV8$DMNe zDQ%+c2q1@Ot#M*Ec9iZ;K)_W@dT!~#d$1DCsxS$;+6c@u-HFu zn=koaNptT1fnJ<{gmz?|%bb~uqUHrpkAY1WTrCBvfyqk9RAPAGi8e!2xc73m6>rgu zxvYq_9NOv9;NR(-oyY#69Dur;I*Oo*;E&1im(3sm#T6DJ?tBVL11d&OJ;X<|YU0Py zE2Z)%l^@y0Z=IVhFB_Vs5^atJ7RV#a=4b*3%ReFxUTysxAY$N1x%Ow}mT!XJZ_mJ> z(BGbc|A(5b|1?AVKMeK}C){;WeiGSZ2N zquCg8jDRl$T?#!+X??FRr8;p-xtG}3j1u-o%eYKkBg!$$e?kW|hbQHB ze-teI^9QujKzK9yJ-yUh?8Eq=+VneRJ}MKFxj~=uXSF}#dOy!fKU?qEynFK5jP_|N ziP{|T(TQB|4R(##+lD*!=g@V=+(FE=(0dn+3?>CJ!N&l+3DR5ilm1wdcry8c{(tWK z|La*DORx?QhkzBH|K)4313R)Y`U;fIrnC(4>+7SSQNHkooX<1!hikC4^z)x5AS58I zlAUcU_|{@!dCuRm1g(t?hfa8;`&L8=fMwC=DHT@|{wIFmKZ^tiTD^dIzJ$0F0IXQF zLbw1nOGE+&jyn2rh^_83)-ZGPLH=~(tZr`I7T}NfAo6#42MB~(GO#nzKmu91=sjdBw>B(f*N^c3&j8fFo~1|f)lJ`^!#|!a=ymY& z#*Qaz&yS8v?|A_*3|-QzUC6zC&G2pvabS06+|FP=mwgtF26$<8wUtyM79^+G6+1xN zj7s(fjIn89g>@eGa`g6V#?EyVl4rS&!?xt_Cxvf9ASjT__N@p@W$!ht!y{69W<{CCLn&s) zr`sp(1!?63gWcsOq;q9vUC=9cozdG5u%bzIx7ibAl)4xj677;9Ea?6oHa4kaRqvAt^K(LiL47OAU0Q3wB?EE_7IWEI{Jh1}mZ zv;S9QzW<+nWc**gFfUF>7br{94k5Lvmc%zvBV^rCAU*KChY+4Ov`p`1piOJYDqOQ% z56FY0#70cbl>(o$Cni0~_db-b*ui;wYOy3lCo$~7u3hI`A*ewa?DwA7>0Jz?65~Np zwnKKu6F#ocW=O=3W}yr^+p={bMHel3C+eM+#~_B6NDgV&W^F^Ozo+AW1O0b=0t0zQ zzt~tId}(!q0+0SWzn4$6Gdj-`BkDak~b0j>xC^-{MmP6D9d#dY zK+ZY7k1Ct@23JstXT!3&|AO6T%%*yJe@T%&?jpYxkNapP>_?}?{$sdpxr(EjmSDQA zg>4}M)Z}YCa`O54TLmmJcY0fweJzY8%j?*2J?BSrti_1aP>hsWY25kpiC+q%T@!<= zN_WoWL>#)l;_cm;^bbVN6Z@>*Ok`Kef`aHOBH?iQG z^*>h!r5OS?phZh)UBUAC;;vQYBMRLt(s7SBz;=9GBd{10Th@~Eo*GG~_fRE}BJoOD z+d$j3*RRahG~NBIQ)ENGZMG3j!hTX}`0ZFe+*PG8Sf3A+V|*;E z^*xh|A>*2-hC113Z|MA3lQkGXK1p(g|J*kHr!Df(*33uHGoB}KqQ;%Ps}8)Zu4V>xmKaRE5~5h^flP`_!Qc)hGD7Pf~K1j zP@Ey?Hrsc-%W9k~(c8$<05m!-Z_3-fFw>j=z_>91lc(#dketI2kP9&_VAC?$jW5TH zfcG8@%#3*4OrD+E6N>tX?Ybvl|l5oDU zE>DML7Cj<6ufmA@LO$OF(JC(|PM_qx_ZePMu0A{-dr@3ot)OSn>0I-{BT1(nf-DMn zoks7y#lhMClO_-e8D~e|H<>jgj@e9q7Sbkm5APYD5dIM6!cZFm_LO3iqrGRH6d$m@ zXVTnLKut7VQc~4$(oW_b1d?I@#QqhcdS+|EI;ULM$z+A?9@xUfW<#2Ld1zQCB04}=(9}3@<4QB# zI2wI0e?w#AHm@kM8c#)!b|XgV_7p9SXbUy#WSr-@x7eDN2kq|2@*MtCSd0|?$#DeN za$N2Pc>`*@s zD{Gn8JZ(@NU=>qW733-DA}OvV)5cK2Pr4$@u%g{x2Hk4mUHK$1szTW8*h0@sZY~|z z*NOfm;Z7tkLfu0;>01ld@vA;1jr!eeGTSPgJ$_tXw}rQ$ciyT2W304081gUVlJ~Ru zKw|L8R~X)Ql|5{=YiyN#(%?3T9e>Eirv~$6roPA}e?%>8`yt%6M3iUC*y6W}FQ|ld z{OS9a+u5+@s*Ve0ilx_mb|vQ3C`d>Ovi4>@d$X;x8o%LfM(-)ip|HV;A8-o4r%Qjl zTp9sMP2#pW8$=U!{{)MH)@bK)zqdmz$j{f{#ll}BE>TtnUbJ*PL2L(*t;1hzhi*OVt_=&+)-P=^VlXBie0rDPjDLIiuuVWRn0W$ zEu_9Hrytnu9vm#N5;Yf!70)nZ~tIf!W!g?O@0blBM z`-1D2)yOuFelScgZ6c4a_!lB8%rUXUJ$XT&CcCne0QIXRNLbp}km0uNEH%r~KWHGC zwvn{GgB&9B21;6g1ozYJ>)VnQYX~65@Ey+`K$22>V=d4;e``aFe!KDtUf(-2wZbR; zh1169Z4(L*n3cz^j{V-7(*T}_hGA!1jb_T0U;N>H@q6FaR*)Im!S|DUMVJO!gB>gY zH6)GsKQyns8)_Rm#IVNJWFR~=A=%_ptNmSwYI(r9PC_-9%|KB2fGuxwetwY|^s%W( zXDgXMHXodB=r_-QvNGr9C1mWBFB<$q=xM2A#h^b+6alsfj_ zX$xKSp-W?A-F>6Gz75}R`z&m;Ggwd8$6ZCtv2F`ha)m}5X&zcQ0bBzKIpvakmky`R zDQps|0yJQo^^QzOZZ5pYt=a;Z*>__;+crKSo~|7U9~`)A5g2D~2K?2yBLI~DL&+X1 zyY=QQ-A4!5CGvax`JFAGk3?T}2@Jkrb{kcbe9jmRXnHG*xuRUI-d(pAUJ^2rVhumU zHKfejd-j*^8mq6E$QM(6P-B_8r<%?fP0=JGvo#yNE=!{DnNUfQWdBNyr-px6TEMap zd>+RVQ&z{nl=D==98)QG7<=hZLkO8T9GkJa(f3#-8R0&ft;lcu9ySE1J8h>PJ~i(c^#m%ng2WpzR}3-_DfuSLce_Fi z5Nm}Oy2KGlhRwrd&``e|SBE}|Sxq@7k%kshvQSaS2!nHes;tbWO}C#?Ssgfd-z&~e zb8ZK3S^KF9e$)6%`jK}^lWC&9-pt(3JK4`od)^PL^JiBcjOy!0q(ozNqK_^H(*s^8 z>KT_}q462r0R~pKB(d`q$vRx!lc?k5#4b2S-m3!gj*_(;3hzRjwxt{7#J*Gh|rF!qN>?cuiRoA6UX{ zKvo5)-bv7Y5x@3Ouh+3y>g@?BACvUG zUO^wV99T{_rR2vgvdbuQjKe-#if#&JAx4hB(CbUIKsH8*&NCxobu`Nam?1Fi$UPdf z4fX!qxo|Psgz{`WaqyMF{wP+Ml|K&JAXF#*`ps>xqKC}yu5X1O3`H9WFnUFu5<>(J zhYK(BO{So~A^Mn;q^|h~j=|HAqO#W_Iv!qxjMl7a&Om z@#hh+vx9Q-UZf?g@&t`|YIz{>$QU`5jRGMl8XuQWA(9Xv6P+SF4I`nQUY1MeBGG5v zS}-4;5i9=Otwoc9!tma{?3$@viMR#zTw5eGj`qA;CBH0wmrsoe?O3#W%hrG^$O!(j zLX&jM3nNS^UauEFB5O{{wL!gK#OZlr*HT-Yhi|>ah>0QZt8HygK_ok_0q*?!(ILU? z-qlob>FB|t*GO@@=LKrS508_pgcci7qem@E&sD%dSAkb5${mN^teulwJz6gOBU?t) z!HyL=g-gyabnJ~o$v-UNb!TU`uG<7?ar}+WCaS+suriYrYn8pxspF5FNa5WKD~heo ztp)6+GeU$STJw>KeAG=#-L2NeG-%mEwtH<$qf=)YIpZ3;DSa1?GF`cS!Rom}eeBS! zKp~4!qYfA4?D}tT)bT;hQk*~LGO=0d11Y3f3(TSh$zBC2gFO+lN_u#O2lPsIh;gM<$}K%g zjBR%V`y{%8&IqEVetJGQ-(r8+D&h3giM~_~m{={^w{8v*3@!aU_0)k)*nQ) zam`d4@y=vYxV`$m0_8z_j%% zHK%RvHf4r3j$NO!+2M19hsj?tKRzjH@>SFVDpJgcEe?A7V6w&gRo|{h4CyA%P#Cor z00(WN4fs_jr{5+Nh+`JCb$}06v|9^VvqbFf7SF3xd9Yzh{u#IHxsn72G1ADh z=qq!&o~)maZgRJbFL z4t05N-g@q7l9oH$QR}uQKoqhvm&zs5?}3-KctX4Y7u5gzD9>-im zYn9r74ejXqN7&RR8zJG$qgGt3hH#2@94z-CXp8S0^lGJQtt?o^)j@F==WhFc9?_4P z3&by)gky3Zy-$1J?|M1=W=*?I9X$UszWcjt^k2Ok`Kt=glCrKWSAMa=uJ7$IGRaDN zYNnSPW$kY3$b)e@IVn8C8L*(-Z@WA7KLBgSTHF9bQ;gFomJ-+W&OO9_PM^z)=Ld9YZlVJa<$Q zzClBINi7^#9z(GO`Lo8I-#KJLtCCMEIv@&7EFdkJ;mheBftC5v6Gdn@U%NJ{$dd|S zB1C8I@v~_d#r){xEBw0P=LaVMairs8xnCn$`G*$h6}qz${^Ktt9RbC`DB1+*!Te`4 zPef7Rl8?+v%UDrkfo0~HBmRHm>K5qXV0qKxoeJtBda;a2Xw zE%BeV>5|kFv!~O__Wjy}BVhZPmw0e8<_4mYzXKZwIPpG3VEj4@!DoGwl?7C9Y4vH} zFnU>W41a92XJsXAu2vjw0~=@-HEzd4+5FrXiYn;T$>rJP(=wcc@3U~aE!bzQ98Kjw zqKXmCDi&6;nmt{2m)HB1L*3K(r}AczHZfkC2LY?;yP^giM`W1k=JG2<^yNFZ%GJc> zL(lKA9~I%+1!)Bcz7l&NN}>mtA!KXf62RvDY5|&8w9?6MKvMY~dcXi7P+6D_P=*vzxJq2t&ObCPQab9_gL3oI zDZ?<6y^-f2mFN9no#)+Tw~)8%6gt!DT*7X(5+5HexpaP3FnjPBe?ml%luGNc%;xA@ zGH4T|Pq*ch6nm?MF+(5@@t_NLw{@!y7|f%&K@`fpb<%CG^!9&OIGLz1~hZ zc{8gI!;z6e=G67(Yfv=)`dygDf^|Qw{zP_}R2{*xu&~un^e$vQfCX1VGZ5;nFsAtq z~kQvo_)dHRVSiz43i4@Ka{_VBzNw_e}^+0EjP zThIF*-exV4{AH$U76rw&iWuzuS&Eo)T9-JubJkr+>g0W0Jj`%8Cib7~;FUC|XaKPR zh8b$eDPwaDb1H?)FGX>PgTni+gz8l2y(!56k@8{1`j@zNb#M~q9&c7=|J{AP z0dBL!1$2y{tS-Kl#>LQIrItR2QXUHPC~wK z%C9NfQ`c~@`gU=Y9YG?KNAhvFM}<{UUl~fZuV6(_U9y+S5Xs{$IEcr1xfQ2kP|$-Q znhJ1V*y+QzB?;n(pTwrV`*z{au)64JTJxcRoo?A4psmC!-(^_JeM5oJcyZ1-$&}A+#YU!=WMomPk!&lSeNTSlw5&~kaLkQQ zEN1Dh-ES?(IEX0ROG?P**{oKv;*8%$0kV_fQs1+?prnuu>m6)-o_IQOl^y?;mD}aQ z5k$cr>4e{enDAB$U>9zrkGbLD_5Yefn*QG{&HsMCvlM28pzCGNxhVQ2T$tA1Kp{#Ddq|h~M-IJv(>t+KJlgMF%jWYivZQT;wSwKtP zVf|)FI2us6NWbJ$qGoQ1FiQqq1zML{%~WBumGCqj(g*zgJ3rw7y6=xO!?;|@_z%=~ z)1)ugz6i3UcG*gYR`Y8D&6F>VN&Y5BjdTVnXB2zQg?O3szVRPVJZStcj|DIQT;fy1 z8Dm<+bxm>`+X6)yblFwOVHy})t!t|Md$nG6j<8ftK|g6rJkPXQFSQ@gzV;=ebn$*} zl(e$tjlA*S(37Ux%5)EqSUqb~ywiM73~0e1HBb#uxPg{W<;&L!Jlk}}<)071X}&Gu zxr^dJHwx8TQ=s=V4R(M>h$0aG{GL|36=7XPWn7j0!3f~5D^6ak6n_d1e6#DSWy+pGa z2N!0&JB~w6(yJybMf+M_fez9%a50aq4P~(+q_{kI5lax5HvWR33(wWnmV6(dt2&>3 zQ^sDBoBzn0FyA0K<5(pu>;gC6jn!!GG(nKEh9EGh`LIwgiu%Mj=YQ!iTjCeEZ9fh0jc*Y>P-n1U%nzYnUzVpSbNB@E29NR?4aPN4-!} zmPo_e6$B5BX`97X8$vhUvY=m=0AF3E-U3QV_V{wwHwgaJYkiNB$n5(-+&GREd57@~ zSZQmxeQGysqnhk3>jL(ki1waW3YUK{RA_8Ey_SBNo^qO}^mDV@y>=gQpLqeckv&S& zzL?zg2QeCb|9V8aSzc5`8hMiv+yCg#6RI2)XDVzP1)V`hBMZ%p1GbYM@Z!!X8U*!g zOmOq?1&h8Stek!wAGM2zJJHJTdrGQ{bibzU6?jMy1q#brE_){Nqbi z{I2elTY?mqO@-m9c_->i^0#)WPIf^dW6PBmx`U12paEo-`F2Y5#lUbYtzNUU?fi%& z`}OUH0n5vF_^{=MlRbU1UW7;L7}A#krNY-=Ekm8-2eMUZSh=HmpBQEBP;{QdUKB*E~^!t#%42>=_8d z-<}6SrwAd50K?U{s7yRuuiaX5^F9z?x;bW()#nDy1oYy=(T^?3ty_%;Sk-%U1}T|O2V9kG@4&Hj|IEz!2dlsr1LUdf!Vk*yY$2Gqhy5=AoWINLKlvx`U|RJ+ z)Y&ISkS6)%YrpFMymtTl>;@#r_IWiL6gzqf#nS$7Y)StC0n-1LO?!i6(=Pm(7ifRJ z<^KTCUAumVU?AK@MS_oBDQ%+7<6N!nwAW~B;-PeHI_^h()4d;UAEg$_L3 zH16=IMv&IHee0QS{O?=wUu|;!^{h7r#BSuq+N@T1J%P`Azt?xtuYMuD%9iSFY6Q6% zlUi@AqCJMl4W3wjq8@g8{7(!I!3iOW7HNy>g+! z>^1AmoqwAIaKDP2$nN))Q~nwLLq-2#qRz~GonB1%2A@}JbIQ@=<@U!{T3R>>QNq^O zg%!v0D7AIFd1V@vtyAoj7+V$s1XowUiq8Y+^lA^w~z_Rs$rPs8s^KgIOT z@cDF7#|@pKVCzEyID5n4m*p*kcHu6irM=H8KqJ-mAC!>o|JLVMA@q4;Fo+a7QnG)k z(Ka~r|AbHT3Nymrw?= z5Im*eoelA#32W;J9(Zf^)y?-$73n{R*&7NpUs~X&5<^aWRQo{Tq4_}4wKn&b+jA|* zzuPTgX9NFRyM=MA5BBwMO31%(f&R-kUH@@G#Zu;t4f>NyES90{7^ZJnZx?qJ7kV=u zMlFn~4rnV+n#uC>Yf;VZcCGb5e7MJETSFcQ?kz)Pw3LWagbGZVEBb%85`61C3+Mzr zt_r*bRaN=(`@MV9d~S(`kKM*#hH$Ia}%gPQRf6!Nd zmZU+n#T#vcJw06JMR2WYhRPsViiX;g-Kd#xoY^I7VjI!@cV{Vx34jB31)|C{;nmH* zqINX?#~Kjvlfw(^Gsl55s!wM}o-xr7E;N1gJ#nMUOltR2o5a1n4-aBSW5kaGND6U- z7iX8PXPH0m?EdkiIxv2Jp-&j`R`kZ}NB;5rf1SSn^Km*jh=cip2?45VpSG$0-D^De z`>VizRs8=8{9m)+uL6H(2#UUbdLQ|3*OyK>kab530k~w>D$*Cvz<+lF>o9Oq!1+K90DI$HDwve- z2Kno+G8H^oR=4jx9d5{ga|H%g@NaexGQqI~|1(4TUgf`j%5fbtdfwfAxQ;a@wl8ttOSGZVoZ((SjMa+joWs4y>^Y_fr{kqmwJfN* zLkb+(+q!$CB4ryEa;b#Dl4S4d^-rVbczNC&tkh@Km%^h(7Z463gsbM}*qg=<THQZt8^17`beHnf?pUgj#-o@>pVexnuysV(( zgF%_Z9fSM|Zz)J0m0s9;H)(dy{Y{sR4@2c(tAo@%MG#vH)i_>pf>y7YSQa~-)0_W6cRc2f z#I0&}*u0ZnJKOLPR5l|Q)W<68i@)EW>!gOHEXv8Or0x`CybXNdnw-N3sgr0AI49r# z`BLI~Wly~}L@E3^HJU=+11TzA+0cHK&a>GI{rfMN5qFb{s5)X$4?AUgT}U9-wBlRV zn6nzVVHMk{S*WAoeR1hWQt|$TC&e}wEA)CdpT#@8a@O^T^aE4}b&F;qVp@!S^P+J>|v!0*1 zBsA`8&9^g{RYZSS-zCcGtwEngH)}9rwqlnUx-00;`-#x`JvlP0j^2EE0H~$Jgu1eO z>7V+wKz~KslP5i>dlC~P-C}=dNAziYKhBfmohGSyrR!7ifoNj6<9tR;$SNhkkAz9P z&J=Tpd{_aaI5Z6PE=69=3uDMm>Rcv4CDO=;mJeZ|W>g{WaxB!Ir8hr|ZQfIxkHwjw zZWa`MdRFx~+qmzRUs~O-ox|6O7agh2iH}s4w$Hjt9#j~-A-xL%_GpwsOg3{f9)@EQ) zyQY#)v5$}D2(k_9LeVB%aUMQnqceNUUa35GSh~cRcJH?jnZtXwF3h~y3}li!T$Ty5 zzku_#I79IkGZHOiq=!e4tdqWjU;a^b-a6O1+}? zV)p*1%-z$utw;Zf+rT4!=(axDfrA+{$Vxw2!OljW)y={_%4q3)=gZcra}G}+Q$ZA4 zeet?ZI}3V+EP@^d7n+z;i>Z!Od*jmrRJvZ76ML%AW|}^wPo}XEH0(D&ppw!;Jt-0t z+n1tsFFU8Gw&0pgO%8c)JRzoeVlbdWD?0%--jNUdx)zPt4_<7T?awSfbpRyexT7zq zv4$p}%j>+>?^2Uup5W_c#7*KFNSVXzpL1W?--QP%2p^-Tj;&_uV3`+nFVCOqCmkyR zVk~ThZ!N|0VePQF-0DBnFK{flH`{8?9bFDWEOz{GDs8nETUJ?I)~w_8nGQos-1i9i zRl7QkIMxMWU?d2*|u(Kc{wWANbtib?6rVJHiRf*Ymz;eMRcAkU*4ls z%f}aSfJJ28N>jFDVsx=GLn`>Pt+X3=2`ko!Jokxx5F2V;4~cH+H6v8S1>EKv%5!Z#3>JY&~OENz3kFKuU~ z<6nm}yuIGgSxOG|Oh%U1{dlP#?Q){z`?(ZBc+_w|F-2||_jTByMJUc*T6X?>mS(~? zpH(VBFBZNbi7q1x;wUE`(J{@l%@fx5U%oMNqV7j*%}K)ZbXzxdIG`J-+|7m;;za`( z+1_Wx9yYf=a#;r)oP?f8I=ETX}=XOnUZwL@FEUo#;45vwPKRIc?t z#d|C?8zY4o%i@S>Mj9#ahjocaV4T^m$=gJAEt&Q8M3qbxD0qHT4HzY}t9sidJvOU1 zr~+2o#4nw0o!f1u1XC=K%i=PJu-l?iQce<2b3}}F59(~V%4wVPD=y{IizqekZL1YTH_eYz;$X>Mj3^oW66)= zG@awn9T1l_*goa!Lee4y~b9%7iMZV`W_9s9fQks#n;e3=* zD;YOksUrE^it1puT{=WU<`Zza#yL@U?h9L2)INkCGFoYH4~gOw;_yOK&87J197 zzFExo^gaV0Cak=mhi9n^F(dmQgdWs(U6pUwQGdRI!cH2#;hX<>X+R@pj?-t*x!Gme z;xjebYS$xMwrtzq$LqJ6wKgX30MFXqRzt=L{=vAk%g1O-fP{ZrbZkv8)0NP|j=ej? zM|mpwMcG+WO>qq$h$2P%n{p0f8=9f&*j|D*K1l=JutX@`s7<9kRJ0d z{hNPow*Nlpyb{xUg*i0@HnO{-ulSsr_t&w(SjNKPqk7i^!cK)RgF0*V0b z!zh9DNO@A%7BG(uA2;@c7IP=t>HS8GLv5Wz^W_}FpjDT%!YjJ&%d;T}_`{r_Q;30c zr_JTLqHMsey!zVo_}9U7A=lNS&~Ez!k|_0p^RDu{O7(I%2%++{z5%3dkquSFaf``j zC!&jc&k7WgV|1sY`Gn|@b-(Y6IG&=Pxj%8q2H6Q@-;`8=Vf1Bn(u`ndUwsBBh5sva zmxV=!57GV70MN3&S(B+_NBO;~*KG8@pLb}ELqE2#LpsRAG`{2G6xQiryu zrL3#m7+?gCG>o~1s7X$_+2G+S^^lG*7bVM68G4UF!@5|ZelCwOBSy02oFKe%Iz|Weuf_D-(~y zMvh*V&2f=FwHdXb?lJM%u3gbZn79b?Fn@{u5eq4o*hX4eKwX$pfc+rfg^5+c^rrCK zVSCLy1Mn)>e(p-xLh_}8eVN&?HUei0dfS#vp=0&^Uh-voJUls!&5BFZY{?k;+O03S zwuQzz%RrVfrAO~Uf33h+Pzg-Tol*KGL`hu0R5^+344c>T0M z??vl_@T!N`XcAT<_tK?7iu-!hk`n1Xa70GhrL`pWdZUs90b+>HOUurEGWs>8T8@R^ z{n5`W8)oCF!h;&j+;m|^53?Asr-cb!+i(+3AGVvgHPl*C5{yUoQ8bx$@y4UD-I*1~<D>PmGMHpT{J#gY{D6=lT^yi5AWl2BNu|j(O?H zI&z~{{Z$-XkUt(YYP0K#V1r5h7R9Mk(^Dw{RifCQIPFEvAss8zQi$R~4uy7mW(gnjo%Sm(zd zu-AT`!o`zLdjG54K&~#9^V8IEbfh%L7{v^7`D4u18AFJ5(0 zWj82Ab?u0(z)~E_bA+lT&0OyWVH^yqBrp>5+gb6@>NRC!>z(hKG z6^CSr_pdHzNy!$UYS({Hp{W`gT-u0$zBc+sB+4nm#EWgGh2d}meKp#QeAMJ!D|}4_ zR_PI!P0Z>Qm79M@^i18q_@87Bh3zJ5v&j?8CVJLsdffqVC6@LAzN8AQ$;atw=m zHdQF`+!|0}TE>Cz+ZXTIA;FOuk<-jP{e8FV1)|;-$Ifje z-eygX8Sy1AmHoLKOP3Hm4Z@<^Rc7!{9dF50&4Cs-Ln57Q!2U?bTaj37@04`xHNCxb zg)Xz$_@*pf`fxohf~FE^J5UfyC8mXTLb_5d+T>-U5rce`H^BK$EhsRYhrm3J|6440 zpQ8PzQ=B7mvAKceSgcQ+t&91bMiAU01~&&yuLH{((|A&?i};y8LPKCXE)g(Q(?K zNj8BSPh8iT#CdY^$ycyW@bAV%%eWGomb&;6}xLQR%0sL z)*X(?xwppU=1Ha;mc(0_t9GVbE~AC>c_5`rCWjGbk|K#xpPAA@a8^r$-hl^=)3&ip zDU&h2t!EnfA9|YS%P5Kbeor%U`-g`EGYip;SL5CRt+>JW#zbl#QKNle9^cMy-VVE^ zhE;)&#tIyw_#3hwNTNP4{f~Gy2F72H2uD?HRoCP9f3cT%2#+jALjg~{(WwlK%h4lG zmmaTmgY>G+w7J68LPYNxBF*u%WS*7H$>&2pQO1mkCObyQ%(w^GXDS|f<_NPuiF<9S z3V)mFQ>FRAwV9+w`Mw9~Nn_F$W1)>%1;UA&=7u%4*ppW$Iy~m__)fv@AZ$i{UP){D zOiIQH?lZYfTypzn_P}~CW6{k?=^I}g|EVgjU_vlUcQUf+|IX_u)hdMHaw3Ac=iygT zZuua=>LceDbE9$Rj%P(W+lInvQ&b@w3L)A-m7Nu9jc?eqPf#l%X)n>_`3l>vLuJ-m zGp{d5l}ryeAE)zXH66GuL{I<+s5+X!pd~}eOFzPAmD29j z%%|58+48s&E z68B>(!sz39a`F1In2Fib;FPesg~i0GR#`dgm1mk6sQmL?{Vw~X`_*J0g12*09cdDH z`B}(5gbPF8j?RARx>^S$39(@*X66GQZU*QMlRM-(_AE>EYfP`lIBhI6@cM21De3tB z1o}UyG*{0cDov!*Zz|2}5#j6W9XeoPl-(b>%^6iE+J}pEMFo(@E&(ju&YL?iE{*%5 znTL3H9QA%C{RyWXQ#nrM#FbV+_?L88Qkt=vm5+yit(fCp3bnIWtiWz=PR)Uyfe|u6o!nR_aoD6;nxA5jkrQ--*+ zE%>e9%1->9H+I_8^l0$mL8SJ&3buWFMS<|`bbU+E>mRkpXT@fVKZ3z3F_oC;4U7?Y43hA* z-v~J{zi&$d9Cnc3bKB6zlDe63~- zR4#JOwO4)7SQAu-XN7=qv6&{;Py65rq&pk_#!0#ZN0;8COQIRz4iUn{#kzyb-1eb` zZJAt_!tDT+=yETjsSJ2NnITlRag`TZA|6l`(-m z6d2juNC9KY^s6%qG`kTlr+k*ouWr~!y$6>Xc8!zmN676rGk0`PDUTuX3PiOvI=G*> zY>ngqI=a~}nps99>STFBQN?*eK^L=q9ifInbLR$bcjR``wnpdjOR@G;Zi=!Zb|1K^ z_cG_A9uZ!H{{Y~m6Z}T$G9a&tsu-VX-csV;9Qf@oq%L*GgLMI?=_$$6voOmHmWC+u znD}%tF%Ird=txpr_rCgswnXTGIFo_-^Zk~X3ViJhAHxL;;G)B>ba_ifpE&BjX$y;?F(H%0OUF($GZ{A9Z;w7Z;z`(M7eE`#5giv@Q!sqE&G3aF& zz!3TY*82$4^q;y(4nl0)euBowB$un6(t|EmOawrlQI!_~W39Mz%?``&4?HASi_;CG z&i4*{ILj|)^R5F`a*fd?7oDijH|8y?bQU9=q8ecr1La!>mhDIWPc_;W%sBTnO=HhH z7}w>}SnA!6`UC7}r1!&_77q|*5kYYVfPt88!~wuI-k)P0{D0F)`ZpU;R$dsKblMDZ zfi4v>&)ow1R$T`r=$?09f-AIn@O zDAbjwNMD|P5!}<9hia?jfLjD3Ow@y^d}G0X%44Af<*}SXN&kjppU`v^6t@XFxW4-F zpD0g{zQJZZ%kX90)Ib4Zsuq%}@T%}pPDjq98~;hT@pn0c=(t z@dII>SaqRE6mLt8j|N~s&1MPV;s)~k28Mg%1-bPc#&yPlX3d*_l~ggs@i#?GG}`|M zMa=(IVERI!@_&@y^W|&ekvYWhBf*5rHC&xBTMZT5p&al zZ6zJcVT(=0CfYgIf2zJyk9W0rVPl@C*Kk|pUMIFHW(*XC8-l2dabxRp+Z69BSw0Y! z+7K`LHDD(XnA)y)Uh!BOLvlRHpm%EQUKqM`z{S3YO@OMoXwO#sB+olV!=&*#N=6UJ zg><~pz)*+WQdGvyYSD8V=P*imf)xL=h9hu;Fwm2#`Au_?seZL1Zboi%ub;D^+;61M zF=nfcg=m`R{3VSBa}~k@O%2uGF`sV4R!wF~$>>26ef*_Vr`CV$P%Sj)f(2>$-#0uE z-5UNU##C&E=k7{WFmG3AczMdTgQn!1q2US?Tr!}}R1L^xCLGsXPnO5SG{gBR;Z8P#kAdFT!GIG=PUX2uM z(es~K0Q(13zTX&wt~_Ui`yWS4l(^BG^4njT)!5z1d-y14`&mbolk)A4@O)al`vB>H zi9QOZuS>3Hq_L-Ty(bFo?l*G#C$^{eOxX|T5J|isLDX2-_ZMB|I<{BORru+sd>Z;I zUzZP^h!LL}$yA1~OvHNP|4j0B&{N{I2I7Dy zxOC~2HkXnRJiYtk-ShAS+)2cNRV7X6YvrPT*QSt~A;t(ZdA`fC8?r-IM=Xg0Zk|yL z9g_El!h`4De}$~2WH6cEPLTUoD2ow9Kw0(rf`RBy+p5mApssK|sxnb;G7V}f<9G>E zl0HrILRvBQ&eartUG*X%aDOA8PJ3J*4KvJk$z|z?YLC+sVYbg#H*u|UiHXj$SGKHt zUkGoNy7voLMvAHdaw@U{nbC-1JiY=b&{6>u`%CSMLlB|mR-bndEz@dP^8Q6@*A?vb zLPzPr8Cb;47FNV)kZ)-ooHGi4zzQRmR4P#8`JK~EKZWI5u6@5FT zrN#_s#>laBk=H@))`*~GS??S@*o?bb759|j=+xN2-U$3VBqi6tKPzuKzFNLTyD4D` z+04ArGo!MiTWBMi%)Ojs#WO_`8F|SGcxOcaB^c~0>#-EjWOGsCKnKRjNID=lY1(v| z!~zI@jZ04Rj&B&{Vr;MIM9ID#%(LB)UTyvog$xvHu-ki4M5McS-iHfw7xvZHzNNs2WkNaGa&Q``5;JvvW7YUiA}jGzE#}W__EsBh%J6gRP%plEMbOnKk& zmuppyB*3ch_<6rqHb}x1c5LW*I zIuGNO86gKq6hAD7e^o)R-@JUVumyb$u&bi|A^4vpSbCzD+fXm;O7Sb#Yu*Z@DIQ5I zk?)VC%v(~o)|iGBx@&)ndPU+7l!?6~bzId0gFV2x5YTMXUOXauWm>Gu>zd+MUh9_P zSkKkj!oR^=F+^?N%+m7rhK&r2^fP#bVM{}qB}88aX$YkFdPh&qj2pE@c!1(R$DQQd za;rkRbs1b^z=^lSa>5Bq7b&Z^xbvEwAHg;X*4Jf`9e58Lo zOe7Xc>qgM9LpAVt>Jy{fp!YE zC5;fR!H@VlDtL#Ji&3N8-$hWkzUgzA804S9{AsakGIp*Mormh{xNE##B(Vn$GQp2y zi~ewqTjQSdG4Q6Ij;!pB2NZpFSuN+r?3Tpd!BvfWug>l0M)!DVNrpo|Zv@CcVhwm- zD^B7~d6A`hD$x>>Jg>vUwmrSJaIujL5X!9tOqh_RTA3xElF($VT)i}RzJ>mSi68sX zzWm|JwzLL-|7S&*>?R#1L~q&lV{u9(MfN_R<(NYAu$=^{5XBn9zO^rh zBxSb9VmLJ#28Rj#67C;}n$9GSAX>CmWg}qByB7QELV9%8^P`6pU|+x4Gb`!yC{0W~ z2vfsujEffo&dYZvXsR+08#q++NxFp-9~vs`^d{B_1%JG-rQj~c&-pll6JuRqql6R4 zxe%L}Tp_!j8l4=UOU}---6!yacI7Ljcn#h_sno4}mY-zaf1uPzk%L#vi37?}U600Cp<-{`mRRFu&lr9S$-dEs*x z?rt4QtDo%>wyex-EdTfVe(IKlgh6aI0~i| zO6WFCwM_-sM)Gd-e*oOl_{CgdTWp#*$nF2`ewHG}c5{#$DhkmcwztLA72oxK@-14; zUgl1t0cZSlT_z85WEJ*cz{VVHuD>&udGuzF6PKbb>u&ocvqbsXy$TU09{iybFO<)r zBdlo=_P$!**a`VbvfJllan%5gsiin&D-%<=?8tzHesrQPX zBG!6I4{lo8Lsv(%V?Zgnovc^)!3TP}5#{W1$(^KV-BCB&NTfmZru3OcN<}6`wmLiZ zeO&haILTpyct&sKfyoefF>l&&nNOe06!DD4r zJJPA~nEi`BcR>28zse<_qhfu71+!tU5)~nzuaT!WJQD=(`oCS=YhorGpR7Q9sc?<` z%2N-5?mhZM^_Zof{z*w=4zX?==(=Z6sWH8aY8o2CFMUxluXG*RL4Vw-M zkZRYk$;9RlGfpvMPq+|)L`BM_eO+W#{ z)>YC!Z>~Ik*ZBpgj(NLlfSt-18-w_&- zcO2G6>yEM$ZbAEI1?HiOf)W3{9blBm=ZWVE#USak(0OpWd&6Pac)L;FowYBcq$~At z3MrA@5F>babcGmM5UYsIW|dMz9Y? zg<{9UFYIi_Y7ruLc3E_#x_uTaPp(CdObk{euVRU;L%<2oVyUe&baS;y{*ST|Z&`bj1jU>MI>O0U$hSFm-Kh2Qvt_I>|C zXXn8WQLG~m(Vh6`oWBWnli@K9t=!9geqD_dcsrH39IN_xw4t$M__jxdzXDN|SMb=V z%<#oVh?Nx#qDr^!B0D7AoJ2{G*VswWudnlFq;GS}J$b9n`rDKO0$Cv^{+fV(J4UZU zkT!!pGv;2phq)_eMa+R9(opL=!YyhfB5~VkUB2*U4W^kYtca|v*w;bnRSk+7!+4hm zb6QVMD!OxtOY1{%L1N;hVbh2tJT`-hm_|InFZ(vR%efPWD!fmPOO*ZAhOf7L*r?+M^t8E)7b6Cb`u;eA_awj1 zfIf(EU+8M}IA%_8n+Nni>9e63<9prIxk|P2BKPMW7Zi}jtpu^S(nQ9ltN-ySP;e>d z0r@yJEM)gQk*NRfu8^KW9z*`q!T*reXVq7Ty-(J=7G{T6>ghjD*`H3at+~FqUxs;B zSzmt3ByVo~)ds*I-}tlzz+SmF?lo!RP+NMrBJ?AgnV%zOrHs_TEM&q8G$@tHvB|6F z9?B6A`s~0jO5TXxDru|iFFhjpDHZ{1`C^-MYfEhE`YTbs^~kG{nFDK=H3D$oW~Iem zw{c@3Mjik@DgPO?im|8cX>iII4lJ@T+ake2edn zPu<&xIA8WN*m;`Q++VDQG|j1Kd-&&1LFege8^DM^QMWo)2%c6>vW&6&3UKJz!g5`S zzM}5Y>3S(Yl$!*=&z3Fd_vzSUjOMQwFpmaA=C3>w?ndl@R*Mz*dfy>ew^aWa7j9$2 z3MmFYOgy?t7H*$658|;MDi~SF*(qZoRS&lIaR+~elgbewE{-+Kjx6e#lk}g$z!#?1 z_1P-+*MS(9>9L66&)R3-T`PmU{S~l}S3s(ZJh6)!{ib4u`1a9ZMlwDqc_xYRT*cp% z+U8SIR@PIfGTm%HSUWF99WumEX+9Dk?NEX1sIAi|&a&+ z3{L8)|7iD0Oq)jaz8;E|rq~U^l%`Zlw$g-qq%)axX@7n_UPY9{WQm{kGHX=D1__1K zlcyT~^a3tGom}p{o-s#Tf+g@HZo?h-GIKNV<2P1$nDlRXOy>T9I}OsaAPz-G6jbv< zUKom)nv)Z_xwK$IMKcna((o4s5|gg@PF=MF1(z#mD@iY`C_Ctni%0mogLP59F((X%Bs8 z7SpM3;M-yfByH{rhJY)JC(98_T7D!i=@;tn%g4s_ZyAc;*fDuRE2(z z$Llf94lLyK69Xn4BL|o-#*c@5iA9_UU*zc`QKW{!j!Vf{#J3)rqYMS!T6!nwLgZdneWkCN*8b2r4ybF0sF>mG;T^K>QJzcD2)6+OPtFY620yw>xy2jWmGX_OY#QR5=fQLPjp`ElG5 zS_pY4bg4~%37$?BN(+PZN4Sa<2{2!VYLl1dM9B8!C(bzumN~77by|4oB=}FY#7*6P zx&rM{Mz4u_q-$8SHsyAagd@=FcSNvTd*ZJcJ11j;52t-G8JV(^qd>IfCedP&TXs$si46}A36IXHqlo6dg*o^$kaTiM>+n2oMIC-Xv z3o7*26v^_B^ z%r0JSs=XG|PyxP3#;5I0{$+2B`^B%8Vl03kuSSE@?o@p0zcJ| z@Jfv{ljxSIeF}$vU8&{u!3`VT^Kk7MYP z@(She>@{kwkJVMUsP0rd1Cw8}xqvjG@T`>DJ%z(f9~uAvKuDhJtn4=&Ji5{oZ-?Ax z(9qJ|Airb?{C3ZTbh|@MueQCI%-+@1bvybnAc4;$(YFt%6DK}muacA;wmWTf^oX_f zXbXlijqaF;pjm;xDW?^x<;j}bh>mE}{7MrsAkBwP_7pIb>!Oz=Bq6W&C5%cIWTl4u zN>Jr}oxPE(m^ zLul0+k*i>Q7J#BOXCl5t2gj+YX?V6q3BX(>m7g^NqB}hf@(w7dSljcJRYU@D zn13$}tq0YS`o*r=iM}Qaq;hT`&&_^Su9rPKv(dOFxfL8PF+NKyyZha6lz?Ea@j?_& zzE2bjqa%MF3mXBPaESsgP!#H)s%6aOXWV*$MAFKUk)mXd)@|45#EvO60bkW`8R8hx z5WAK0)yYLWX>Qjudn+{|O9m4Ap_xC@n9mb31(Nm`TvA&Z7im+LjT6J)AZ#ptz8#6a` z?gI*(6NyFAY2BeNO(ltDbo0~-@p-j(qxhs?g_XulpZ*QlZiC4}af{Gx@)@yMOFN@QOtAHQDi=m<6|I!?Vj@#3bE)uxe`X{s$9r1hgX!F+~x(drIyvWM8@^tx86)Q4K^|Bll= z#EQEfnMkq{$quz!f2{DawEz{TU7r~IY~ZG}VWRyqC>=5Vnv8BY$(Y2qs2{@zP}3k_C*XeM;jH^E;4}qo zG#qI!H)-^A36)mCiiHfg!s3SIbN7iHTXo}xBv^}*+d*rutfts!Bhk+JK6ouOH>@ue z0d1g`YRlQ<%Kh>%bsX^s6hy@qj*xj0{77#uKAvOl#nt9+2VP0 z*{q(~2P~pMH1TNsu7;YtRQr5B)mPEBu}9eko= za{(}<>7fqdirl$jU>bJtJ0zgk>*S8hb9_}1{?!xP_!xCxy%WE)aFe9ZTdkTF*Lz)X ziK0B1;;r8B)S>86HydS@=G1TceWqS|0zFbIJ!qnPaY-U0=zUe)CDn z{L#Qxbg{hRv}v2qih?H%Phw;Bk4hBzKS1SzvJ0(EB?BCORD#nyJQ!InbGVb-X8MeE z-`xF$5XUXmZjtn}GDCOZ?N+L1XDiq7)`5zG5SrdexAJ4hUJV`X7}p~kHV=A?XEW#Q za&VjtQ?p%bli?t{1mRd99`Or^%g3R-coX;jxg#?iS0t9Og>>}2y+P8$%E32RrzIrf zi4o~utO-Q3ZTlF_N{|=5Ap63H#H(cA^RPOO3TMwhh_-Pay)U?jQfDm+kQw+Leh`J{ z)1_B5(B05^5XvqT+u@B{i#fBO?F6T>XgAlxG`ij>vB4j2K~-0!u0|PX9d=s!zP9*7 ztTYU*h%2e~CMW?5pt4 zVG_h00OfOHE|A8hZ@bKA(o2N^+IV3IjD%hrb}}Ndp7&HKV+-D9X50qyop8?RdmA-5 z8-_5hlc;Cs?!p~QWq_04GM1S0>KZyo(Cst;w&wv%jHG0C(eHaTjqUV!QEL3vO_1DZ zq@wV+!mD1Lu*cNi3J{}7?Xo7M6z3o5xk-|#KWTf;eOKL=-B9yk&f{%grlxLea4}(L z#lY@hy1$>KX#bEM{f3DjG2$eUhXph(xxx{y58b)W>-fESgfSb0ty(7n^U_MuXA`0O zCATTJ2yY>uJP$WAJk;%!!A?@8SC==lMf&V#Pcv;5Mj3p_vew6-qVoIk%{vH689GJK z$*Z}RHkJ&Y)7SLUBZiu{gtsR@@!?2FymsXL8r-#|JIv5ki8K`T_Yflk2Q_yMf`x;A zEwba+PuFVXzu$B`Fc1SXe{+b-m_Z9anhH`swW`=3b^kCb{#nIoh7eP$b*U=Gmf$K> zQz75Hrau4pZH-L~|MJHGpUH)^%l#%PGQ)}&+@*v92R*6cCmK&=U?@}n=&f8rya@_w zE#8zM5U3KohPE&v11x8^sOj6D-*jc7yLR0-%{!p4Ez`xsNefRrI+=O1Lly4iiFB{+ zZxMS|_(Fs3u==?ZVZDgy&5f-W|TvvxaGrQhhLs#P8-@z7PMon*VCY%I=sh`7@52V=-4 zV+K2~V(xuHMfVOs?Bg$*ilJKV=nM~RkB25>g1v1mb(-s`EhBa}P2el;0}nfLwngyv zO@usWXhmNNyY#4E>Df>~-$*aWY|Hg87DTBue@iNPEh)qO(Nk8#Uxa__%B01q=%@+% zAY1U_jZ~y?T3PpZC-OSPiEIw@C?g|fge5u*|rc zml`79Pl&KPU`DQ3Q#vd8L07RJq8YYRq@uGf8ZQ}_+&BznWTh=H8DNS~_cfH^`8YU( zsGx-ggAITrF2nGCySoJpMmTL63FvlYqq7fspuN3aKao83A62U

    WY#^$yZT zGNwKoT(^=Zm2=kO#I#Xgr{qn{a5R&4#in&JWb%B_p2MfZ5a2k@w7j()sVTK+cDD*b z2eq5j%k{W%aD?j<&g%0@tCl#~vvV(3lthh*#xR?}pJ&u1{%+2Ya^hL|5zjC;x7-L~ zntLh{dP=ZDe)~yFEhNUxq1gE3IN8Xvl zID7b-l?afUxF4Cf=Y;#w+o1^X(Ri)aHp`G{u>V*pn#|a()Zwb*@*aa-#%V%4tN_96P2ScjRfsXQ}c6udk<&Rrbtk1BOzachaRoePcEGsa^sZz*@86X@9cKhDO$bz`$|W z0mGEkMlEP~%Psgu;LF3N@-Yq#PB66Pj*z!TV?;&i9DSTA)fgsZX${9jpf}O+ywA>y zHLeUk6nQXEd{V-dp1@g8J1#=3dC)2r3B|fu9E`q4U|mt5D?_t-kn^X_p>mXPk6_nN zHP#FKA})~oassHz^4_X{%;ILRcv%=0G|V35s+ZT5(BMz^nJ*nD89)ra&h+l35OE%S zxqbFUyYc@~i!P|@z_~?ypol_3%RLvE7r;!)=WeENq-~bE^XmL>lgn##g$5(`Q?w~g3Zohjpu0_(1D+-Qp=h951D9wy} zZ=VE1xn>uQbJ1^o{{#utm1Lw}>dSD*gPzi^tOo-If*i>4Hs47_4fEEeE1sR^$Ds=; zYC4rt;P8RIU97DHjB>-Xvvc>FneiNx?>6D2Gk4BKt@UlN5T@-fy z;2M!j#kX5M%xCi#@wEJSvZ$Tr^cL?5tDvZGDWX{+`rRZbL+X)LrxLi+Hf^?Bz&6&9 zONtkU_&~IFw)w&}yaAt)y5|#f4j#D=PCJ0nx1)79H>37qpFAJ`A|;}TacuePJEqs` zm)JkNDHue}4wY^(r)P23U?a$h;pJw*Wq1cTh8(F|ovu>sHfjR6c8e40j|CC&?s(=~ zD$h(f>mHHIa@xFYy`w1R469FDkcTO)J3qdurAb}=JyQI{9;t$4i<($ovA5($*R4(( z;zote`N2|J>F1|&eHX-~f;L!!JOI`(rn3TO*I8u_YM+J9R4k9R8~Ti&P|Dqj_psfM z#EdeBPGPGcZ~Rf=VabV+sQb{~JI(RmT~%6sL$ke|0nU2v)LP2MEX81GsZQRM&B~}f zCFE1F$q12!>1~}c*{}~Df09gEsGreXIXCm^k+a4X_v&zMT;WN`WPsL#``4?%3e9AL>}uBQx3L=rPsJ6EDLplAmJl$`oa66}a$SMQAw<=c#<8RkiaQ04TCxI-Z=_)Wx z;^OZn7YnWT+ZZk>MaRC&|NDXQFGHr=Mr`vIdo_%%c`*im)o!Bc>V8P8{XrTw3+%GF zwbKj^o(8fy#QEiMfzN7h7ckgR|wTnR))%TGqPSDWNSc8t(FqX zLQ^E*j zE$qxO>tv^V;~Gx}z1%7KHrgYS95K77{j`~5pYK~;SzM~ga-D%ED9Kaw@5-9_ZJ6JX zzf^l6IrV;EaO13zAaxXF)tR4q)^CqL1W{L=GI-bZI4yuw=A zH{8NkxHsD6msIM2`?B8LKF%BJ%=}sWXK9|?(=}VhXf$&;P8&6M8}UY`3EM=%;@UaZ zDCj~}2|tE`n>^gD5-sxQJxplnJPtL3Rwozzt$%6pa3Cd#!TNY z!p%YNmFN7pVKa_Tz(> z)?CL=u}{*Q!7Bx%0Xq{5hq3Xv7pr8^*pG(UHdg7rhSHV+r59D{JAngzT$0(kUXD?j zVoV^yd8|R-)+Xr}omPPN5oVcpMSBL4zki2qQ)V)0}t!64Ri!skiG9DfT z;kvybwJa0ZI>l1PvmDrp;ZOhb_xKg&S#oYuE_&)=O8!LZ;uOZWrgqHV_KvC8Fz%x)IH4 zI;SnLTGy8Na98px5LTnm?S&b*uW^2AGO=~tj8nRR)2NZ%Eps--=53GlC(YQmZ8_3G z!X+1&pDXG+S36O5KW+`AN9 z!Py~ld|RmV+OK}Jw`gz?6*}9mg19)ifN&e*E1T9u_0}DgtNopHcPF2uE39tC#NG&V zeztbZ#1?vlRu`C|)Qb$(zV9xal-0?_|EN}RHOasAsXozC+M((U3Wk(`HU}<|016?V z_h$ptPinvU6vxVeqP@eI}9C zgh8=YcYfira1QMom|@`U)=L@PAd4ou_Dr(#K`RQC{;zkU0y{W9naksYBU3PWuxsDCtV~G<8k(_T7^P~mqkWyPUp_BZJ!&yHhr0b)FQkYDPfQvi2H{jV zQLu^|8=dn`8NGYc3yXd@W=z~NTKl60Y+cktjz0CyCxJ|QB3w2e=c*_lw0W7S2f;e* zS$qqfLj*Al{xoe}-!C*6*W&xTW9{fx65%dtgC!{vZeKjNod-K89uETEcqFZr(C^Ij z=#0BZ77I(_cG3^M0#2N=%q8l5Q=#;wfo*X`yEK2bTET#>VhCJRb|ka0L->s! zl{1iU1u<5?I6$P)2C);@lw9)uucpgj*KP_V|ABdh zwGa{iMYvaA0#}+*M-xWE1u~0c@n&We8I|2N(lC@Ar7sY&s}l~)^RAV&ALv-)dt)l-+8<}CiXk@$5ae#)j1Lq zm#coXUOJSiTl2K(j@Om>)vkL|ZyL{!S!BLL>hWx@48M~P6{$OjbBHn(9U#>E5Y>Hr zFB=`}C~QjL{+NAY;$KpCq3LF)1|n?M6@xOII?XCYOf|NLLeOFIP;8EwKUt$0oHUQQ zW{aNX+md=yU)3Dq#Kud`__Zb=(Zs3wA#Gyw$F=k&RM*pfrJKx)jI$0xvEseb8nxbo zHvUsG{~pR|>hP9x=VOa1ayX)~=r}XeM`;#@OUa2W1L9QOX)A$}1ughR@-_VSA8IHw ztsl!$UAu4#K?t@gdRVnHSU<}Md6XGqkt@rOKh;hZX8PoTcin5jgGQC_~Gs`7pw^DiT{ zH8FRblK~zaS4`Hz9P^y_6*r@UeBZMZGL0DiFJ45Jz(pBHvM18BGa&+^S0d79O?2{g zS;^UUgmHP?lz&g2Kp5ibC7r)`O>_IT3PgPLsZzit!xqTCffMOlZ2yUi3s(Vqn{{j^X7tfZ z&kIhu%%Ur1<*KCN2UmBS%5O7w8Op8edb55?#-ND1HQ=#2uW&;6T3G@c3pWK&<+_u1?_B5WhkYa`U-hV|jV9h_DrK&ah}Lwq72Ks%nInp028>VTtM zsSu-MlmR9)=&FDi<_6`6250GO;Flxto?CFdZj73xpwWijIOivyeui$*D!DUZxYiD^ zxHEU%fnsYnp_uoxoz`LXwqwkD5*IjL8DublzN|mHJ71l>rt!QR{?l>BRfn44i+;W^ zkCf$_-x|ls+5(S%Hi{UVDlVa-&PRBXr{vb#G28m7qL_OP3wivbwAoOV-@jyKkJV1( zytP0l7ZxVcm%05KI*GbM5$|JDu2>>=tSS(0fu0g z`~1s#_`lsx;y_nHK(qvdb>+)@5w95Un*`2se+Jm5Z$H*eaS${s5u*kZ7T$Q))63>3 zZUu|To6EF2^y!*w84#aRzcx=-bhaMM(SQ}Pds1;3IaPw0K} zZM=`?w8Nfv#%;^>B{5l-W(HQq514i;M{_niyrnpqnx|BEPwsf%?w~n@M~pjc;|z;0 zL&T4jfB_n{-7yr5KFoIomL_&C9JNbT5A`O(_F8P$ImLvI1e(~xBKtvuV6ZY4a%sdTYsj>^J1OJ>$iSo4ks4tZnKK`=< zdXyYx0?suukQ68xa>tEm8Ggr%w7zV*v>y2i=N(HS8@J93SzAXZ9j7|HyUss0EmWdSl zgzfn56{0Ql?Ox`zCq0P9X0D6_gF2fNj!kbG``#0X7Tj})TUy^)Als4v4tjzj01{L+rX`*YlDEQ`%< z3mR(;liF^kBlI}7I)zh5AN6A3S~Tgrd#TSArB*YdP04MnMtyl)ca~i4G~K@EG~fw9 zKG>e|vWtlno{z7k@GzgbOQbx(MsGX-ZGq_CM(Ca>oj8#zZbIl7+st^ccad3kdx5xKc-EZG8^o*ePdQF^y zjbY`jO|e1~4F#25eHBVlTE;XqgI1v7pkutPXqjh-wdHCJBa)yq13aTP(t4N4h?`>( z3tyWL?z~R9%)9_v!drY-l6*|X25`UtIE>?GQn{}2xk|cVJw#N=iueqd%W^w~+YvHH z9*(2CJxhH_NQlj7p%?iNLcls?4cEj?X@={<7{opGkHTv&Ux7N8PzenGW3lJ&WB6Gd z5hBSC34Hj9bgu59+(Lx_r4xL?*cMHrI;J|2zPj#O@ne;#IOM+_<^S~-_sQ58J-$dV zv_VfeIURXT(dMx53|M|Cw8#A4 zb}i%*8whA`Fw=nR|Z{a%7>#3v*DLG9%00P+%Bp^D;@8q7_{#+YHT$ zfa8k={JKsCocgY$L&Aw42`q6nH)jqF^Lrb0ECaQauo*SYI03hhC}{nZmEyor9$#T^ z`RkdX#H54O^VoxSgVUW;<;%pr*uZtM=ujDxb--bEPGG6ftXmb-MhLZWAJRs9@!h=K z`rcvRM43Bg9XaY4me0jiqR%$V@t?p(>ecf@^K9{&o6N4aEZ%u1^hE*jnooA`Enf0u zk|jp z$ivz_D@l*{lcINn$H9xy!qU?Hgs)- zs(M1Qdbh@$LQY?zhbu`nJa%Empn(!n_gNAe7lg^j>O18WhFk#84m2D85fJ;hp<3Clk8(onLeW9Dh z?@g2xay@$FcPOzI;x(?76@eNhl-8)n8~M`c0RXj;u+7;?pX2}KD|(4cdAWjceip;^ z?Ew_`+3E#;b098UJ0aB7VZse<1YE&0eVGCkJP|pRFmQdfW5koynHt9R_NY8*6OBHH zxypnqb9F@UzSuoK-oGA>uf;o_3#wxcBN@DVF%TLoHlN8n6yaxN^_@S~OfB36C9kj) z=I)%@$HF(VP|o66Sslj&uOfhMCH9XgPV3c>)rXA;$J*T<+y!G-Q5}+A%{Zf84+TevqY`_9+?Yc>#G$aH z65DW z=K}`=R9gG%l%5xq7lk;j?r*T-?h1R0OB;l-wcD9mhwGI*QWnik^=_ud@<x2JCN`!?Gm5I`KZ4qgqp5azyef#O~v_|_jgR&&wPXw+E?nuq3 z=#i4LUScWr7?<~xt6K-;R+HM*Fi&$80cx= z=mX@GTII)OIJP%JZIrZ&t?Leh6zeZ+dOJnddV6-Rfe zzn+eiT@L)P_nG1_-n!zwc@kt)@823iUGl6!B-CA(`jvjm7Y$1G@5Rr|CC+yDjA3UP zb#kLbLYB$e24TLu`m8&c9l{;y8dV0I88CG=8!xoLT<=WKg0hN0w;KOrzdk&and?D4#HdZ@U^duih1PpHW-f>_IuHjq9!{lRsQ z`fYR7>R?Se8BtW=tKXlv-rQbyb)q26l?p^-zd!lJzvV2|hK>OB2SKtbbyZ z-e-9voS^bz;cBLNZmp`af!NgGWXiL}e^sC1+59JbYL&HRP$qa|N5-{S_1Wkr_e;97 zs1^eL@e%=d`Pk5TG9rOp*+An52CRe0f$q8DV=qquI+C!amf)psy_cqnQ|)BKY{~Zt zsxQVxQ{Qo5#sfBbQ0_gqFvBDCqha*mN%0OM7kuoPj?`U2Nf-ox-sa9_^V%01oGa! zqa~G-6#J6Q2uZtNocYXFOtx&n0mq2vgI)6uUSm?s46rOC-87T_C&gqIF;Z?SW@CT* zy7!D(6?mWK&vcL~Hl;IT`E5f$`OP#B_oMKF2dH;sAHPR8@|biLW^7xseaEx1ZZ(#wXjg9ZBmwrvY&~`i zEwu{4Q|sgWd8x!1S5C=IYPE&}DaDXUP)XH2sz}@rNw1Z%13ZeNWw^_6&F>TJr{3#E z{>(?Xj77WakiYQ8W6!g_->!X6auqqzMV$(FO!R^4>6^cvjKFbq#W7S}<{}%25ADu? z;kaBBGel78Tm3?2_Y=DGFemnYv&x^Yu#8xo$bI`Bgib@^Lu*l>C$fPgYoVapjK}AnJUa)RyQVIdN5! zQA~nR<}6uK*&iJy7wIKsOIWUV13N-6<1z;L9>M~S*DiFOr|4omttjPLR(-;gLl>B0 z#{MZO)fY2|3io+cx^VVptbM*Gk!Gn_(IrHCS3L(dn-9dzQ4;!gha{&m4|~#DrYW1U z$K=tmysRuvnEO(py~CX{_t2ARxE?NLq%^H0lL2AfdN7bN2=mm8jzVd^nZ(TQy}`Bu zCQ$ZjTXJN(oO9)wE4AwUp>W*b-Nz+%t~`5i zLFB7u$4g#VB*582(n4wmHlq{o3p>Ks@K#fiTo0|ZglPxtG6F_Xh{Frrvb_r`@{!|o zsNz%eY9HjHYdftVsUSlPyT}ANWxmPaaDUcw&zHe}#&-~_Mm z`l+|3f15=M+lQL()1RW@`Iue=HP~Q)7hL)A!)n*Xc83ux`fJZ%&tXV3Yg&G%p`#5A z>9S**wuQV*tS$X+lZV|uCMJIP2>VwH&P{y9eINj?us&2H&s?C~AP`oiJ0 zppfdTD7MCV{=Gx`dwJTgTdT>sINEYN5d* zsnApv2iSD5dDQMJWH~sMq(Q z9kk&_d-Et(N0J)CYvIo?RxwB-{=g3q)|txgc`1PK?`@Y)5$%!JtPasJVJdp`ZQ-EM zk5dAi4A$^<`3k8dLRa|2&F)Yjhow~9tf+)}K7#MTra2GL>kb8(jqzL)wx??@_|xo9 zf%+~ujusc{_3NK!yc+n(;OCMUb%2kr=?$GSNlr7+B+qwt(0ZA@H?4XWE3s9&IW5$| zJ1TxqYwtMYeM}sbw;IG+)6FuP={&fZL!xTiAD~K!0-Krt06YUPyjGA_q6E>ADFV9t2PmA z&YW@=F&;SHP%?+K12%Clgkzu9haQi(P@Ctg7&%E@;CX*$#&J1JN>)Eud+`Q7lnX7q zb>VcntKVm(=!E|$Tg6Yr4Bt%WkkAup;lVk1K(C4ZwTrz*egboL{%<-ak~+)z$Buvxhz7nPZH!Oz&_n6^w)C zlaACWD?=BBj3x4!_lQY0gvRQ3p0v!qli{%32C18IA6Laur5t5N$%LC4nU-L(m}`FN zR=><1`RvWt&Y9yXx`XidzDN}91)Aek^c)wJ+ZZ)~Gj3Ko4}*e_d7ieOU1@d=G!KQ% z8W10UB7+-RXNoLIx-2_m8a6ftI3&NV&c-cZRnJ$wK@(JmuRfivg~Lf~T$hPNYaS+) zH@)6+IysVF5fCv)vXOLnll{T_Shawmo_ElpvI4P5vAUYhA@(cU!+=5n zPYt%-yDHG3(m@o}U2G>~Ncs^)%bNwpijv1(=T9fgzs!)zm-@YtDhIg`!P!06{eF?s zxW~97ngp(&I7O=!@Dj449WtpsMYuHZ;&sN5V=r;te3##V=@pb`Llxp`!O&R2eD`kr zB_8RGghfcXdTbj}zWj1nO|nR749YNc4-mU|y2o}5q8d?hbqgA?3s`id%h}gDI0#rX_4<} zb1ochuYr<*3ES@8MjM)R@YH<3i|76Lo=s(itS*-Vl1u(<6CQqjXl0Xbr7dy6^@EqK7V$INnE(%Q53l;)>a3_FF;8w zvwa|}gaT?!(~t6zP%_yM;ffS?^0vdi%M7A_4?J#KF+CpP0E<_%`qo`#X8W3{mn39& z7?x$tVVxtcILps>oo5vOenFPV;KM@3CdgaPt-5-=)E4Q}>!8=ezwAL-@vm$ZWW`7& zM_DKqK;T;O+7+`D%Gw_z@whNnT6(f_hJ^!V2DZ~O?lZ<3ibDMx`2Lur|)aq)R-*jggV{1S@1XQ0c<|TIy}CF$!0i%*8)hvy=`qZU$#T%R9}g=maV)<+vywquG0p0%Lv5j~(V! zNB{3Nj;tOn5yPSHE|5Gw(IyWeQ9^!*P6|SH!4Qh11;{dRm4@m z^(*{7;Xs=s-nSb-QKv#&apbwCLDc;aG3WRF^GvEds?6Yob$06-MVW*4K3r-*j8hlA zz0!rg4OKbnLy`EG{zr%hyOD+QmGW@kbS5= z5b7(*1KD|)0@gq85Zh44Ri$_$sN{e9(w5dOx2A{e7c(z5bhT2a%UeT~?2a2~Y6XGE z<*E5mk$&|fzr}HXYy=fSxuzP$YUIcqX{2n|2U%k6qC>CEAyN9eq*C9eljyiyKOt7w zhG0X|fIwTb^GQ;cj)x_(_m?3B!cp7oYIGKNBu9Fv@KAaQIZ?P4ms_YlL6v-&WQPU3 zDo0N~r$cPQXa$Oy{X)-IKKxRxvBUIv2^TFTuO;nXK23tQg>LPijJW*VAJ;IWgt+t{ zJLB$&%2^R@=p1^&dKpkJ0b8RhJZZHJoXUA9I|ex*S^;8GrU%SjKU&zONK{|a%!NZ! zy)W5vbFhL;qXtrsGAvJ3;l{KE!pdtwPLgxNNo!tVXeTVvx~>e_FGC}x@iS`OeaG6h zr>`=qer84tC<}t8HS(tMyD6je?8WiH4u{uTW0uDe+Cf}PbH!8+N+6u*stO@GO=AaB(OHvwH*Tjp-~*N-Wj>PmAtlnJ2}}R zq=@A}P~7b-2~{DRFE#5N#!se$lYUNthcT6zF8(_ji91lN_UYkjOO201t1h$f>&UKe ztFPZ3zL}l-uF~DDhAh56 zChE%~uRnYp&N)6aHacF?ekUZGGb*WNL}wEdZk(;X<(F1~GU&MJ33m*+sCsK1a^I`> zRzRi7dy9H6d*1Xej{Xciwz36(lqh zsY&6NA*w~7UUOZq#Vyp<3+nG__=@P^{Q*Jb1J_3D9SUlhXFrhS>7^6Ki4PzpoKM^W zs}J6YDV(zbet$tG2BjyZn#!&}Za`%~} z@7%J5S$Fn7xtAo~af;4{z#w1S;O}>A-JM81TI_DW!AZCpk0A+qsm*XVgBLlRt~sb5 z>Zzu0jiA1gH!tEMNxzC}$55i0j6K|AQeG?)P zv?iAoukN4STWVQYpMP>V zbb;&o4tOO9-Lv9=*{n;|M%8;wt)t$42hAFvyGgP}DRB0u9eYP&MI6_%8gFlM7o6tq z`CI~wk5w{^CG0e6oxBMnYA9sUuHU!tWr}~NJ}S5@aiPZ4xSEIs@eNg%P99(5xLJg* zUvP2&P#ZQ4J%u~2eswm?qK1d(sSB`AH(o?yk{WXKe?@AM^bm4hPW=Z_OXRpE@QzR> zaUYfKOw{1QQ01+5=Y^K&iC5K{Ql#6RNDAYx@k$ODt+y62ma$9jZYLbOd!SqI%NlI) zraQZM`b!mJrG7n}?#F7lAyVxXM_ORlh2v1fUK5#!UYwx4&V;uX1!a`Dc%<@?KL&{^ ztk5XvM~!KK`SVjzEYTK*?g_DNc}#-VXA-W0xPN5fcGa|dY9ZjYx=CX9%3%W`_? z*c)^&evD^bJ#pB;uPH8nFW-JhzvbIEs4l$IKa!$ocsM!>g|6fNH|2isGhVqXt&z3y z)3njME!!6pwnt3U^gFZ5%2rU$DT-YtB0@M8PokZ4hYyuSgw`E9X5_$tM|Q@dZBHiH@c3J3e}PF_ixg zS+svSs#h{ER9}FHPma$D93X%7O2r(S;;f<)OcMV<91>8z@piQlRf)A$WT4hyZ>sys zjojB0@b}Q@5FVtEN%*EVG8H(-aa7~)0!U6Ogm?w=*e4QTnBuM`TrQJNpEV~oi z;RfP~v~dU7U@QOopy7q`TX&Scg1yrpy0e<%U? z?`-Ef9?kEkB7AybM#U0!{x0?LuZ0Dbgnx0kx&Lo)xW8V?4+ZZWFAns2%Vc_LLefR{ z(dxK2+@;yaLXGzYUoPsju#vp9tjt((nomjzBC5zKyei()Cu0&kia>?e5262NI5n7h zePrb(Z~vk{|4MguU-(JJy^Lx5+Pb1y%ZYZu*JuDf6L6ea<6ModkJ^Ll;*Cye@mj8i zHF}vx>V#LL-m!m?ZFOvfGcW4PbC=+kslH~QLb1*HFVgtm5B;B~sL%cHY0HUzmm9(9 zSz#!DazI;k^c?N}+`dI2O%sN2S>I?FX68yg)|^sK2sbARtyM>8mQ&U(?Zf!LaCZ}1 zUZyrh!26WK(KU>}$UV0D)9aU8XJu<4QQj9EL!L92B0?pCF$su|i#MUKH;jKayqv}U zl{`3GB#@0%$cucfzi?~t8gI&fx$+m1Fk0b9%YD!u#Zljz#@yu83*<`ZxsC97Eq5ZL zp;{*&6Jt!q?$x3bs5;wk*!eA%QfiZ;iQDhuEal zzd7*6I&FvP#2cu`)e}+VGV}qGdg`vL#O@0p$$4_x0?TzGKh}8!J-T@ZrnpE~cM2lS zcsJg<-K43ntt=+Q{k=Ga1VjOCUOL9wIY{~6=y)*aq12~-YEdXO0&te#b55Myfro8g_#& z32&$Cy`6boK6Ia2c|-=u=X<4k+mkD~H_A0X2PQwUB>t1%LJSWDZH=E?R1Gs4GhGkW z^VwTThc7z5Ql~H6F#JK6iSbs=7*ww~j1ng%e`HKL4^EN%DL)xJ{Mxq{7;Q zj>U%5oNXG4A(i@7Q28tE9EAjwHcqd#OO!T3e6a=xB&r7}tP6s>X2eNv4Fy4=@|Ega z9yas>u55$ez6q(tvFUd|e9QU#Hkb~QIni>W%AHIsiFh_-x}><}+%BDPV&{1HKt7M> zteMPF+%n?NXv+W~x||(%tkxAjRu_Lfh+kYfD6c`_sLa0~vi>mX2F?qAICe&ibYAOQ zPb)wW*#c``JU#9L;%JJ$mh+@E8zw(U%MRz72BJUD+g9ClR)+vv$vdJmA zd+HhxYEMpHhDCODB{ozM&6$vQEIEG?D{dQ$g3E>@q4{M+} zl-tebBCK!udqWK=a-M=F+aaA2rLFh=xIFKLB_u667ni+Ucp^8`-F$BY?^a?RSz^$- z8Xw27@Z&>NTN{kX$Lc<1cRwi#?n;Mjl}nOFq!Pza5;9*=w3v>Y0$$a{XSUUnxD%Gb!H`O=PsqW>VLT);PWdcvI3 z+Mk!bNc7-k+jQY!j*PC=NniRD-w)!Jp+Db4 z(Z3(B=PBpZSW5N(d(ZjMg7jDQUE1&JJKVuOHIb!$eqwtHEl7>Q$! z8Z{p>{#kk#Z^vT-H16WXLUA>lxoqca4*-0gF^0#NIz%6q=59mSH zktQIYn7kRf#%fE=4m8`q^{2+EK2?y^qzEe?3DZAR^Rf1g{j1)ND*D`ZYz2kEw$7-5 zx~9_14&aSAx_Trzp_n>TD*_xkA;jIY#p76{+NX$-quWv0Jm0d?{}4p?tY_ z6MXo?b$Z+CozoD!dHN#$_#4sipk^|+JtiY1P4NmRF$bJTen`SdGvBu)tX$>BdsATn zLcTN4SqGy|ov1NBb0e3LB1THUwH?^xnros#8{#uNk2h@2itB$RKU>d^x;a}2d~oVXX9RhT`{C|*dMRUc5v&-b ze{+@`Y3u*wEG?(yFhbaY8kv=G#D`AbmNu{j!tYN9W>JP-3bZ`PKW>rQCfW&MEjv| zB%h-dy7Sb2qAFVcCd9s9tEtMz^#)!_7B!i7B}SO~ba;H(T0HcD1jeFYa%gVklN@Vi zeT{ozH>~Fn7Z9cXL4T^VYlR0T@y&ae7uCW@F4aS!!)@QCnfPF>DKk9rF~k1oGEY)& z%1Ld5FMN|s8ArJsd^y5vH&^)r{m%MUu(r!&yM*jX8V^fPK~I_wMklQtPH}H`4WgYz;2ehvV`COVlrQ^6bG1#znC%FyR!}y9Q)%i%uU2z^cC;|CX+9I6XVsai% zjQUp-vs%!vb$FFU@k3#Wq8!^+H&LOGoBS7B4UdU?ecdXUAN>*IX=u{0h90>BDPh1y z%c4sH931l{qp6MjU_rv+w{Nir#Zo0(P(D9*^u&}_`WntE3GI?c?EHP$9B9D!yCB7) z{hD~z2EyL-jHmIuHAWbj7FYiKB${T!_mtMQULkA}!JY2H>eN)j3%esNH&I*h3QJ^n)0TJw~StrEnkH1j2T+r`}1RV-AuRgY_Mua|06QfeIITm^ylP8v&)}mU zeIZV@wtf?*j+$adUiY_Bwr^|)n$-c5c196IgL^ZmX7zIqpjg?_IAT?NIW{p?#`xrR z0{&QQ+lu{nM0+&jzEvkCvH>gldg+wa zZiY)fzNoP8_9#NrRx>?}tdveAe<3y_c`vQLLX(kM8u!|C$Eb%|vT6=a8bIYexGY78@N~zIRhqYS)JQob8|8$u%UJ!*1*xi$tZn@R72MlRf|;l zMKpOw6MiXw0HX&5nR5P;!-U?z9w$XI)5bOQ24muR5gRqfR1yM@zmZ=>=n<}TbSE3v%q1!Yw7?&ZH8VLe;` zaZ6ypLLTDK6;#lKoN3#_-a#L4+<|-?({E z!?%OZY`sm}Ie_ONl1nVrhID!;t={L0))0Fej5q=~%LM9!C@5l6sG5mqU-nA~od@EE zfc>gh4@z6g@Z(_1!DYwiHHoHKkxyA9(fp52B7FuBBjO)&l3HjfjNC02v#ON!8>Q_O z#t5)#k%IVnU%RVSQpEV*I^m)o&F|M;&W*w?B6@_jqwVZjbL+VIWsw;fTOqUF} zNktYl!k`D(^Yh{zplO6+)_Cow_K^dB@EafA!?V)4yDPpzzq9&qG*t$63CRb11lkbe zy$Ry7(Zyi0A~Q^QP%89mMZxydnSLI-(AMIT1Ca{6g-?UEKkzT@~=` zyDkZ==NaLoSo?7M_gA%nl?lK@`lh)0+Awbv^N-2~yo$=t&@uIMr;P*?zbm~9`KwP% zYJ$9#RAJfb8#b}+4%YO)gb=M9H*sMHmVZEs;7X6*;l-KJxJO5=u7Gqz@t*cpy_w^Z zi?KD`smQ>Swj!u7ZQENH==)WIY5s+ox;jo`NM6a*{DLPZeZQYoxNefIE;=`wdi`Dd zrv@j*YT;FdYEN?b{9uuqwf#6nlGCVPS_z?B8M;>o+(v2KiBVcy{7rH#wfu1yQcJ_L zB1IN9%zM&q;>*Hf>v#M4b+f!IYi{E4L|fJf>?ckHS@mUFf*p$z6Y`!0G{G7K*%UL` zh7I0zT&1f6Z0fZMTZ^C0gkz1*j8?dF!?l`1Q^ZFEgmiHdDL9$$+p>i#%|gP9h}tg% z-^xKCH4sYX36h3$_dtw{uV;syCkJVO*DBj9Kcd=>Fji2&f?N@dud^8>UgoGpj)$<@ zRoA((E8DH0iSrw-xjK202g9O!AtLha*=yBk(0o=q)8%pqE=F&zD12>N!f%`_gY+U;8j{rGXQEyDGD(*FmIpdNh5 zSN=H&^$&3@nJwiJZVgb`FIn^W9F3ZRt1Y^{{nQ@0CUagQQP0P#YK$g>4dp@2=}q6o zNoF>JR*fb`ut>9Y(EZi>IiI&MVyN_i0n01omGen)OZm(9yIZw%4=NmNKU_juFobq} z+9IUA!dld`f#nS?B8oQC0F;>^jc9ov5&w0z;5WprpybNaiMorzi3aVKAWc2!x1(JB zh!oz0!35Ww45lJ0Xunxhq*u%$YpOwMca>)yYU%kE!rbFe{DaNaF)lqz6O=D9eQ8JN zRygPggPJnGzYAyNs`Gf5fASzaFLJ6H+bj*){*yR`GjJan{45Fu1@-(q1;;!1UsHpQ zR=hubvd9*>LyXiBgM~($c#lMiyA?{PkijQRCH-BuODw8(`mMdi%?wdnqciAbrB-^3Hrtl`B&+1TLsS|JsB1Qpu- za@z}9tZMZRCr?LuGyXy+5MN3Gs&ISyFy&P>u#BtU1L!M1UyHHf6F1@-p!ELT9}=?e=>$n-An| zO@~6OI@Pb%jx^)kpwAdGc~{L6_1EeQ#JVbE-z)ebVD-RGxNfMHu@%Ghzoue#!mY?z zjA^Ew0S23#>RYocim>)43~I}N6+*Nz>2dOc(6Ln7gDPWcu?U_hE2a&4^OVMnohNPI z-$2)U*PNdJEKS2p49ST@>Q6q2&0pvHBEpUzY#0x@5`$817Q{HwCypFpr=`yAL&=-O z_q9qs%xFJ+jX$Cnluomg+fYznjjr)^TVcwHg{7=_`{xIbs}KB|z#hWwJ$T>FQ;r%g*xkgON_*5B@xC;xRQ51;8s5@n`R*c1F zgw@eMY?CJ$=h$-7KjA%C`#{OVlbrU?L2GgW(8+BkDk{;s<{u7x-q(>Pnxq=_64LlN zkcK)u0eDk}YTN6(e;i7HHk4{} zKoZApe1JCWa1QeQYG=+zmdE}pRI=g>pQXr+j4>!c6xCD*qHN7 zc;mA>Jwg+C&?aXiRXJiva$B0g=e5on z7_8^S(9~GO!N;uNNQD#Xd!zh^{breykkb{cd}H|XF~{-GJ`#Fb z+gSpn*mHu>q^f5rf~Z9zL$`WYRAu~U`fBe9)SCe}hxrDSd)q>cb9iFyfPGHJmucmE z+GF@8?i*GQPX!vZb%|0@7sgKLS(!m<)jd06G4mPsc_EiIi7bPvlaxlYOO-5=b*81V zMGwz1MP&~@tPd_el9fUnLUprdf zOM5{r90ERo54(a$AC~6W4CvYf==B6@84P|1)ISy$a1G<{qi(eo<(m@6niLFwlVgAZcL3aWFzIw-j;xB;uO zjFtjDm!R7oDL}ioKNlTBj$0>6y+!R@>{XR2b;eRnKD#ohbfVOVfAP^HE{QN1|cT-RA7khW(SY|>iO$32H%5S*0S`2@WPl_iK% zccRN_AvlT1F9ghLWo`{uip7m~Q->Ltc;3jXRYEADA|lHpnn@)jOUY_bdq#rGQGYB!<$h)jd(6 zjteR753GB7ZK@QXAEFmxN1=`tn%IuXaV<-qW_cSQ?d)(>`3EEdV<~lXcAOoN1c&02 zG(Kw~sR%!ZOf@(^Fvs9~tl8e{p|8m;6;Lg5ZOlpBx9G|!Ijq>}D1q*}L|mFrSM|Jw z)?)lqDc6#j&-ETQa^}cMcF&E*I$`hV&f9o-@)QVRUPeGw`1|zuv!46LQ!(0Kd)AV5 zo-|xdMY!`f*69(0FYjl^L`F+z_NrG#4&6n@+pq8N{d(+ARqS^F>nY${w1&8hkN?&} z{nKuN&7cv$_RCqVcukPyX%tObybJ$EH1@R%o`-3&z`Ea}M@)poSlG>-&@+y;!U8?K$eCz1=%h3c zd9oQ7TJKc7d;cX5WcY%dZ#7`1ih=P39O@jY#b^tT0sW6_&s2Adn+*-^l{jn1ZjGCR zCzC53-a?}P!CwDf)XfAEha+GsWkofxmE?Gj?Qhop&r@|ggh5?i;V@kXBM<}rqfhv+ zjXUQ2^_;LbjxXM}^}pWzA6?1s4>XOz5jM3tNf0%)2KMXy&u-^mzHHgz_qv+$UjB$} z{n=i?Tz|=e@{2&^IuZOQHT>WH4w4Q93t{7}+n9t@X$A)5#Ti!R4aqJo-P1aDI+u_l zvZMD5&(nK#j0@ga9mhL=@|2{u-y9J1m}M(05e)5CVmCgB)(r7-(*iaA(Q&uDLM;M5 z-XGBLxgWrMT?(MuijMe9e6(&^)KC9x*L{T!l9uT94@xVP;uEQAZ%GC|-W40Ou<))2 zOLA2KKs=e~O7EnOR$isEGR#*I@pmX~(pU{;X;x(3z)@tLzw>ftL1@mb!df`2{rJ|E zSVY!)(QE|ctp8ILw1dwh$Erbm{<E4PCA`glvsB#+zjX zSQ7=4kLoax!OSa7yJSO8KT4HZ0%}h%^6QnDHZp?Kmp}pdmv!9$Y!4X@z8GRQDPu_F z!l{!;{tg_zJzF1~i2%M5s;mA5Y@2;AB1uiPoB;k&N-R9v-m%wZOD%`q<-5xp!L5m_ zOCO~_*vCWD?|t|c-|-#z=3{a{2;vfu^-!h)ABmZeha8)?sD>X>txglYJ&t=tlCrFN zEw?%i+f=FAoV^`H92*TCM$vKKLhtUH*L4Lr`pW>>P3DFU_m$|VP5K#j#@pH5K&Pz` zW=vjQGAEj3{MkIpLy1gsZjw^to(Zo!QaSgm?(I{|hw>JDw3@!}GBmTJ*_MLlG*9YP zC?`I8>?dd*?wXY7dS%kulgu*Yv^&cwC=APLk^Y??YAYf>^NE@n0*#f@lpN%6kO8Jtg&*(uHIsbIb86ZHJ3w|6Xv`cl}8F%ihi0dC#F#dlzX?tg74XFMjcmw{aK z9i{^*?)IMAdO8{mc^CHeIk$P+d+2p+9TR#%ixnrK6>r2>QH~sCh z1aCFWuDZeecr`o{&sd^;jPLA?p=ld-^>{i4?fVZ%T(q<9Q@TkFG@>nk5J>#ZA% zzp-z>qf|%G?_bZJpj|z*yC(6KE7Ee{B2CVEEy^4g-_i5C`E~qHS+@wR9eEMHXNBX3 z>sKbrKI^NoizXmtwSRUQ@+s&t1McWZgF3%@<`)U^@{+_^91v|4`qAPe(!$f_Sf$b}sI8c^|Q zil{HbMh2bKKrg%W3~D}})G`8@w=tJa8DnE|4M)Q46Dl8YBebT{UF%h|DOd@&N_Rgb zrX`*t`(3>s+vt zCHWw$Q-ISKb(?&OP97{ul=I%k4Suz#jUnrGiVXed4gr3?uVWg2U?zdbID{U7t&%Vh z;T2fGVFAv(%X;*u96|hN5Zz`dBUUViZKb`4iwf2ku)1(1P^)`elUCk7;le>8e#2?o;&s0D9zt0~{bqf0<@zlqQn7E~VAj&9 zgMVDoRjuAAf#@+UONt^4Z+qriFaaQ|_XX`ysU44cA7(*&+22Rt2*$T`;!n|!NlMAT zs%X6596ZbNXnt3raN@J>)5KO;&(T@^Y{zaXRXOUb7UbIQyc#5|l~H|V&B4=}-)Z4KKK0UL5@QN+{6`2;p z2Q9dzMWl8@%~3Oia8!GAOFujCI95Ko@;Z+;_ zxqUo(N{4GZAC?(z28K)*&fw>)wj&Ld2A@4eehL{pJ(vCB&7s0nXu+I36<}33ocff7 zpXRUHhBu_7L+hGys>gb6pWrz{>zKNQ_aipJkXI*2&y6?LKPaq^lKqP4itE@1WAJuz z+lRMv8oUshs)VpEyx<=)1YR@hxfA+yJ$dZ{pJCbrK+0%k#eP3j-%4J3D22T%CmjOF z6rgrpdFO9%{~_hm0_;h93ntCw@)A#D+Z$3}b&Q^gj618PT$*O=fjPltv!sC6M6`zH z56m(Ym1;|KDFo|)WP?(P9cGC~kdk8WrY4bE?mi*DJ+rDb?)XyCa~2yJ{hR4noICml zxj%47%V>dt1WiMZ`+bshsF6Iz+TZP*uWjf}({8yqP}3uN33VjBkHsJMEC{gp1n{Ro zt>{etwSld-n*5Q9nPL-t`0`#t*dhd_PDOnbwbbET8(4|)mvzw!t8>N~;X>3>Oa(a~ z2YU+MXe0fErQgcGt+wXNpx!vTINQhGX2QGB-nq=ilENL$h?r{SmEk)nij~&mPgt$& zH7(eD%sHrbbYW}*jdZ1?x5GcbxxA$#x_c1kH~5-)@`FI9fY0LZ zt>|oVw#DEX;JEm*4>hT3E|=Wa5$69V!o9@G;Rpk;fh{Q=snTZK+SV|?!9bEWEfPgQ zi|eHcKCC_w#gLIdjynCRib-VI^|=S+c;-byeqX4Sb{^5SBeHC}UI8vcFZRGo(!^CVWlxEO!aaZ9Iin2%CcVEE#pvX*|}kI%K7s=Ik7t#+gNubN_I zzc44?V&v(8x9roT<$I@3H%?Q4!WI^vhZRH>cYov{QDpR7{`vV^MEAa6C&P{ zjE~#CWc!$9Qr01vs6?IHuDNy|p6S7lN8~SY(&sr9{l1Hy0)dt@ICZns&7z@rvWtYq zY^trP5mQ4iIRZ@;ttg(WT0^21 z-wpxSdn>o5dyz$Z{^(*S$ zf@&|`x*9P`PH8dBM}r4VYM1$fQ;Gk*LmPyDhS4k2IQJsP&YO%t441}_34V%J!G)rub+00wYB+IYI@WkR!#7a$JZ zj={Nlzp>=hu6T{{swLaX1Lgrph-L4~D1QX!Vbu+R zaNrrdP!2sXRNMc9Q7_-Vz`Mv7csD|^fkVH^8BCO%Y>>C$@+$fRr(@*K3}R^DeZXVK zY4;zv5NPCi_7jhW*2oDC)IzE?RVZ@}!-kf>Swz=aXfZY{3{?Ekyf76KHAK);A zN6=rlj|ciHr2@$+BnB}OsdnNRB?gB~INt}|65fwrEV^_9eSv-k9vgpP2B6@NUNpun zGR*o&(D0o}#&uw4r|8#=jMm|+igJ_niu^P7Ka!(BW{%SrS6n~5Vc(r=N%KPz9qhMP zt`0N=7L1AZ2VUSu^UxXjq>d0a0W7xY|7c+dUx7fR<#M#53Z5S8=IBI&^bOJ1r15!A zgl~ng<4}2kD&82@c!%sqM=??*`E9VR-Z|DlU}lhmdl}kw0~bZ z;ERbR83eANy#>GfgCJLVT?FTMrWEb}LuPhd2YESzSlMkAwR4>#1P$V{btF2gCp50S zg;raDWAxFEP)6g)q}{}p_Yc%zyb%aQUM|OfJIxEEb~+i)6X&=zbSdA!X0cl(WI6Ok z33Z@#Y|I>xl-YkLPw0dhzAFhCf4LDs1M8e~)z{4)5bQP*0Y+<#8)|%evwmtpZW(Xq zBO!>1vx1}=3T?jA?lZr+gk&ObLm|D+^hY*GG!qE*qDy@GXPg}^W$)`7`lcwh{H)(k zD+{|fu3z|?@vY69F99hg`Ux=kHP^b%ARVa7OCt=cYEZRQ$S&q2Slc^Is}#G*M52x8nV^-7XhBop(zwl)x)XXY-MH$?V9(|Ug>@msVe}DE zBy$b=#*LDx3`dO)R+}YTlY0cm8|Ss-n8mu^y+4l!!XKQd7#1OApf@DLx8fTih)suc zF)idQI9aR%HC33Rr?ZVu^26F$PL6Lr_;Oy!+-UIC-k>$G9Qf!1KM8P1t{&^bJeUa> zKTXyYqLH_mUwHGKpz>Yz6^i^9pdcL|G4~b8>jyq)lKAlk&^ORh#Rn<1?aKF8`hU~JK*J%NX0ZAsihimP10?7 zyQS0opc1q`LZI}|G(3aUh z0R5m+%NvjZL;OBm4(2)7u{?Pw8h8UwR^--0l!vM+ z0c}EN6#ycyj%vAed!4udc1Otj1sgvZ6{GDYBk}=Nq&gnMR?#0F6^$UMr~({6 zKXQh&!>GChMN+#2;L0r1(;u-x<7$Rb2w-*6+wX(~6#&Ee?esKaI=tn%5@lU?Y{ECh zH03il2eKT-F+EPX!y)3LpVljea#+R0jI`qdy%i@LBknHY<=Y8^^Y^2wXuMfUUZ=6k zqSqpV4~;weHZ6h-7x@WPZ3+gNn+T1~3OM~e~8RSNX$ z1@&M6AF9zvob2~uifdAn@ln=k=PKJ$sCK5zWNw9uOGH~-6*BrWhrxGpw2&YiNA%RRr*(5_upQTk(Or%%8vjyJ6;g0gdneZ77`l<(cmaF)w z8nRKxI*7gd{%K@v*186T2fhgJOzX@|hV10ka#o|$EzH{w>m3Rx%gH5>Ur?#_zHRXv zi?D6gE#a)Xuz=upXWepL0CBNC#qI-_ooWA zs%EW^4NJR@sqgm0H=aX?l!U1zPwqdupCgAozAssq<}2u_iPMo7JlvaPUe4-v5)RRT zg-l^L<$zfo5qXFYs|>&8w08?D5`FcXhAwtApu6Z9!aSEtrrI8vuMvT%9O8GuHXZOx zht;oi1fbtt(G~mCI_5A3w!U$A82QwsQ}TfM=EFNXt!w|KXvR0ZUR}$G-?iIpLR+6a zA$g1U=uYMCKXHzJws-PPHWsj2$1U>`c1XnlZL?7~rC#>wU6VPc zW#RbNK@@YV_n~krgUet>qO61EUwQrR-}~#X$bKx1&RVRqLC&~~YcvY2m43ZHw-gg} z4=*p#N2ktl$b?}@wO7Q!j5UnCpA8qewJpXcHK580C?x!4p!6nu9v5{BYMXk;K{?hjW zo`scNM?~W9lYV)h77zD6jE848dYIg+IIv9J(GPS9RIBdVDn(msVA#I6ksoXqVuPbR zrX<2=wh`hL9XJL&kFZ878N*6{baXGcS7zP5bJfp4_n!$M477y21#kUjq-AfB$rwLt z@va}3>50OUVcug>pX6YPzuOBlRNX(Nv><4myonDhSu$vi#(cwGi?Ic4A)MnMUz@7nlAQWJK7GMga55N0QzIAx=|AB8%B%jeyqlIPx z&h${>5!x^P^c?lT^dkGha*=l}3rP$gnqp24+(7&Xt;dHm}bIZyFsEp zE;-Tv0jve-|Lgi5IE{)wo!KHE@;|v40_9Q@y`3g%3pDdb8&Wrs@}S|h{!YZF8A=Bc z{mY)Q)V}jp?mKi}Xt6fbP9@g6y$iR-7iV))fL*J$jNWe2es&Z;qLa-?hZ~4BxU_z z`NX0ONp@1{z8{3t?vP`TT-H+dgwcyP1e>&tjKZT2IDqH{%wlRr&Q`qqCZ?@a@Ptx= zsjj)yA_jpqowKwEWqbZx+|H;g)( z=Lf68k~U&IJ<5qzo?-a{yl0C_DuC7EXej>U-5wmwg?L)SVS+ct@_cb7O_mLO*c(L& z4_cR>rtcJ(9IgRu+#(Uul5VRm+$$j$BsTMq=VkSrB!1e2%mbGNxgVm@>+*vJVz@-p z){%M>u1}Om10O_Gtg+7DYHVBG#PBW@Ad{hI4&VEtrZD%5gYX?mD42bP%pc|A{O`SP z{6g+2FgAYkW^g-)F97Hy?zb%E(AjMd{U=f3@-(8WMu%VV+(YDPsy_V8lc#I#C>|$J zZT{R1YNv~~T122Af=rw4katx|&wlnAuOTu|nQbO#%Xt{xpn5We8>mlN;`_kLuv-&U z__un@TF*9P*{_(L@4YjAecLhp$wv{T+CT(#*vSmW>Leq<$b@ z?YmL9J0&YHrBKD=|7j+>dyQ@iKXF%LL& zuDE$Yq6!}ExDt(-J4kibdY@5?J;>@5`p_(xCj?;HHkq1r_7kIye1b(bo>M5bDS z*iZtc)oRQPY^BqBM&I$5oO&pi#q8rt0?Hx!&YCDUwP|A0F+yKt&wu3W52DuRqP6*` zAQoQ{6z}vkL-n=}b5T5gk|Xh)5e&?3cg)*W_Jw@)2Y;@{gpFoyXo=Jbf@PiM5D=k0 z8B!=>QV|qC=Af8}wubIeL~TM{9=YSN%AGIx;$)fQ*Oa6_f#sc|q{5t+AT4O@{+NY$ zlv#b-jmLd_t(rk!ICjeWXM;)NnzBv{L3_e@MUvfg?`TvZolUN7n$l3aCpVrk1*_lGR)X&K~3f{2QS172dGx0S4W9f#-PT&)V56fpE##_qw0HVFskv2c{k7mcwdfP8k0Au?XEj zLFIL8ULR&10J(Lb_Pqr6>r@#pe9A&}pdH_=p#mPz(6dmc69icEu}3s^b4SgDU2nIQ zhm+F}EGH~ZU#kc=`$5sl^aLzso659CD$x%owdzcpHVj_O1m;IHk3r0mPe_DrRmkz3 zfTUyhxTi;Ixb#bJ2@Fn&m*cwXy2aJm`=Sf^+t+Ft=(AtS9zc4Xi$qMQ+$E$@;P><> zfX4hY4OiX6QD1S&V``A&-DtvD6JZzJ;>bUNel@%u4I6sih6($0@IuZE%8uODMP2K- zvnThdm0=sZoEX&gb3Uq8isSy?m%856QQ7s$0nQ~Ib6&J)vM@?c&5Y@#mGl(YC5bn= z(b;cUQ}sp2w7?}&!zurwHgrmpi5gF_X=Cz%KfgozYDB5O8jtA%8>020M7iWk$%A!a zh;Dt&1^&s(*JAmv{*ok3Dkq3F^)cevkzvCH!@2&+2({p~pni_7!>J`>?oOgIqfuL}(S$a;0mjb2Eh;ie>~mC{9b{R+`tBbK zJA1f%-`>oZnyN(n7m;~lE5`plmi=#W(1{fxr8&Y6)O!^_Q|qtf$!YlF!_~+ANTHuxXy>&g zJL|z%*l*jiOCn3O2>M>*Kx${_F``EZn6wcTYdA!Z8gwwU_ey7OZflqpHiDS11(jA| z2U>4esxry3p|ZXNP!jjT*HZ3|&E_jwTahivV$OZlj*9OR*3QmtK=(9e2v@UgI1q^c zElpkew#{l%NC6hkM3Ss7(2545(#Yvsq{^?QT?k2dsBwg!roQ^DdbPl)dwO-}-y{{X z`^Jco2i{U++dKQYV@gaqC-gomyKjjc#Om!&zBQjlf{yhnwTTfwI;3C?;*y5=$V0I9 z49wtcyhsI+YL8FTuJ>227I({Kkq)T;^Xb=C=9f~j%GNNG_f79)M?Hy&?mN>T5#0T~cu1P6!5rXhLsuT;`so%qM++^KC6U zOcg<+dP};+LrU=)T?n9UdZ3DY@2s#5f(y#H12UMnZrMR_8+}Y$Z8s4J-cZw2RnXg^ z%nrRA2rzEE?F6n?yV}~H3{oW{SFER=bdX>kJOy6blf-UgSyzs7EMPLnt-o@ zf!Sy_E>7Tmc!iRSsx?wn=${Q zAhm*yj^i*GNB|D2%D(FEXlB1OtyX;a5V!J{a!w%KF?s*)jhRtV$C<^31)8-++gf*r z<5s&zcpE+F_&X#_F5-atKY;kxnmCAYZYc_ELwR$~AJYy`aEk#t@h&b}bGm+Je#4Il z68qRYOL;X?>5bWZzaM=TxWLUn%@7))xSrTEiV?TvjSf>$GI-uMC>p^v2j0WlU&{)P zCHPi%n8bg5kLIO*)Qw|P?M797M~({Ea(d-xO@A@NPtn#s8bY+ZrrlX$z5gM^E7SWS#1LO8^3B zg}aT=y(OcqbmaFj)J$c=)c_q7K)NQ{X2KGA^f*^eGy2Dt2eNd5(wDD#90^%i3GklY z`e}#-zbQfd1UP^#&@0*3=1p|W>gAKsh14dv`Kdk-l3JI3(#_fGv;*y8(rdk9(u7G0 zC&U_TNs>`b8_GaP!kG$-@8EAcv>U2AJJT2uoRB7YhNmlP5b7fBp`+VHkvvu?RomI1 zWj<8JzduZZ18M`h!^!Luf7C+3iOnKBvXSw%PrBX%d!sg029nmO>L-SJY+HHOZr(J$ zB+W(9N_MS1@;4^e|7Se3s*e3%kwv9)II!?eFPc6>7^pEpZn+H&^rgk=5E2|W__sH-&iYMsi0EV}VCK}bR= z#0c?ov$(Ak>Vjyz*CABh1^k8Wrs7pcf6lwE*iYrNOA~ob#6zF4BHVo=Gg8J89)a&7 zJXRKM0M4I3WA{4u^#BZ<8InxfRCq{jeXVlBZWgDocYZ^j_uTG#hEmUS99-6#Q4veX zMT8yOYfdmYpJ}4PNy|FH-qc#A46Vz;yL0h%A8kd`Sey1;flrPE?QO0}gg%5G4kgdu zW%ijwMCx-uhv)`WYUn5-Hu&%`Op5&NeZ5~Zg19@T%JQXEDxH=)TA6Ledt&)qj+>NY zwu{@A$H$rPy){M9{W9XX^x+Oe`a6+x5kAdy^>E3ig9CddpYJu@P?OUsWI3btd+pk0 zVeq3^SvL9vI8nuahiMBSED}9Tc!4hZk?E-X?9E8+1p$gtO@v4U*tN3vih$K3bCy9? zd?ajDi}o9jxCrQfA*KIc+>@({&;Noa3|B572GEMHD57$yw&9M6t^ZYJ*JYmwT~823 z89f1xmI9KSso0i>#x;85%7@E_^J>6DBPJ)g#tlQ^!>s6RfIkTSJ?!O+`1-tjO=DoI z>$`e&>KzI~v@VF3(Na5RnurhOeW8hoXb^Ps)WLcJUN-6o`eQ6G;>2nxcE8_=SPa+`2^%X=EO`P=ge#3{?0 z|3NOCagmQ2|CF0;SFzm1QMIo8&x*=P9}dFh(3kv|l#J>|;o*bK4>pzRjgkL?-8G~# z=L&x)t&own$Chvui=lR;+>XhO-~149)SQ()^pD35`uUen=j7x6uYEeZL3mIu)6(+O zFtbuJ3u!&U&A&`LZg6x?-rREOP<^%^3g}A@+qLL_!zKT9EvzbX^46jND$wyR@oKvX zWFs+peqRn=HZM0lRAIpIO4P0^XDjKsJf}Ur;KL6VlYnO(MWX~v$zpsKR-*V^c3uUs zro6h8<}QgQ{-USGYBb}K{%*yZ%eN?8XEz!#T;^!I7)NwV1E{)W?kk%C;s8-t`Tz}? z&Dl{(937wbB5XGm?@RLAOkPKb&Y?W{zFy&I4O99fnuEkr4yD}n8+0^-tuFG5X#T6rL=r5qD zZ2w4A94fx(m*Q!=p*!I9Ocgq8`7>8HjktGSRtNT51!C-wz%G-%YxnMvwQYi^^G!7`Z;bY2hsaL;V**LMWL9cMsfZW)(%pU7<##PFPak z?i;KJ1ev_74SBmP%uyd7 z$d1ALOx>=+(dn`0r&}p3*9TcVtG8>M_uB+d;ki{Su`3cDuJ@oh(vBArr)(8U7>>uT zyTu;Ne}_1`@&01qI@*R>qAY3KoCfTyhzH>bLl=$9+1*PBL4A*Pw6@u%W-$Z1^VOHc z_1j5}%7Ks%*d3f7ofhgyF$p93BNcyKAIz~NWCi_;jC#hhX)h%&{gS_HnNe06^%FsL z-(vrO16k<}iSX6w+}1hvnHgn_Y@c~+1KGzFuj`ZaXd<%thg%Z~`;(qDN$M9dr`^ju zFCei8VSFv997VT!zdC<)Druzu*8fJuzR?j$4XK;o6`O7rY$^mQLxTG=)Ja`W<-XF| zzO$f7qsgy&4a}2k^DN}tPvH`+JEMzk&@&KY>Aw7f0!2Vq)l{Bk!=tZwEc8<#^-3fA z0n4(h>Q@0s9?hu7*3FDB8)U_k&f41&JUd%D;|9|8Z1Ydy$;g*{pMF?PH>?!vVWS4k9y!<9PP7!y9C|GoZ#M20E6JfV!GQ2j{Jjya4bypS=g!sv_eEC{{NyxqUBM;Omt1WA z3tyUz%(`)VB2Q1(e9__`S}FsnMzPK*;169HcagZ)stb3NCG_gJyRZG1r51owIp6$0 zG*u$StI>v?xfkI`5|0UXh8t+${h45SVkbMxbbUhJayTCxy%jW>HG{%!@7MllD*5a` z*Xw*>*bk5daxirHyHKjb#*!V!defWEZeK|TlDM{u(J|HQ{_2!mr^u=xigB18yImbt z%4?;^efk!E&3m+H#2pYfo;r3?=L=KfOi4u6`mTUs*TQyWcJnGJ@RC$TzZ35gC*X?w z@eHy_Qe7|S#K2Bz=lnwd z6FWLps-q0YqxoIZs+LiTltd*9!CPkh_22DwyQ!y!gu0>amHW0Ni? z{>EzE$SgTy|JegOU?+l&tgWeugUK=$(}A4LqXXcL3vFBlkz^J68bnXgBD*x>z^Thl z=qq#v$9z~CsuF|r@Z5^<7~-z_3;u}Jotop4%c)Xwtu^H4lIfxom}`B*bnkSMuIHkh zGn~CN(@5#qj(lPB9mqnN`d(b41IxeEvxC@RL1UIxXtpACMBRC8o(=iI0(<)0`j5Qc zn-1pS*04517EpZ`C-Hsp^$4FsL;e&K9jZ*BZ-*`5f&?TQv8yj@bgCyJEQN9@f<2uo zS}n>}q6f)E*;Kxpwz#A5x9LaTJIxRKC7g}rbRqRE<|xog_O{J5-Q7&Zsi);5{-y-N z5CBE2ln}TpAJRkFR{x$H^_j{FrNY`b81L-sIyo1ejJNWMv(9mW?h$W3s5Clb;3##1 z@a!d%VosZN$&NbY2c%0yF{bXkK}uA9G0oP|D2>)Ev!mxGN2q3Q(IugbpnbZGASLfX z*k{`;Jq||&(v{N00d~x-KI!4S;SQ_b3;Oip_jRVHX97OzZ~|g?dK>i>oF1gQ6E2af zSoQkrlEUJTKvwek!&VESz&uQt9ogjkIh%21+-JAM$Gwl89a&El(RdoDem!HB{+myNgrJ`L+(vyhL2}ZZesZ#{SUY> zU@GB2j)1H4J<5n^p%aFua2`H%QHk5vDJXDi?8@`~Tp-y+rldYdw&R4Q5f{kDI{Avu_ z(P=;HP4Nlati{PYdC4!evY)af?=i_Ax0nlKk_X2u8Tciy(JKR0%dcw!zgo==J-4}K zo`(-LYd%3a@6SK@tXAtYpHpzS&CfQaIZ~YZ`*W1zJ_bqN>lYfE;e>2)Y`*4y=I)?; ztFc~05g6)FB=QvdVVdC%X{=Rer+N1%eLmpkwwagjTI3PDc5rPl>S}S8Am(~qf*P43wA_!l}$?y zorc__rSTT#=$vFdGX>Z6SGwJnrZlCn6;p9+x$M%A0#eaVlF&*v8<<2F`n*lD=bw%w zd0B>WA+|T>nbxTWibi~`{g{9B#u}A*fi6CxZaug}+NpUTBy$2xh~HQL{9#u|+&lK% zIo$IZl1q07g1X~MC#h)LyeW~RP)K*rsj)AyB@z^FwdLp5m{($=vfmJMs%+z9}SLMP}ukW zxJy%o5n%x%c`qW!z?Ve85iS4y-bqrk<59vP>*}^fpiu1OoJDJ4X^~xb#XyUT{;S|q ztEP;H5hv%``N;n9B>UqW52?9LaRK{7H-lB}nH56HNVhty?J;3*Zj8DlD1Jf@ZxV)T zY_qHh-UWp7V^4SKNB(s6DtAk-(J%8v8ct$@;@ZBlZ$X1Gm|GbzO9GW_NV?4CIQrQoeT=~+Q-srP>dEkM6nax*iQYn22*Xo5a zOUj~;CHOdDOR%doVXp6EAu)_=L97l)@?Zf+*7T1!H(m07;!`j{-AK4lF5;%Z%7`P`A67&pulm^AI3KF!1Q7i7U&7iKY7hYT z-(rL}oyeUZAIb}{NNp)S?eB4QCFcnz*P4~5h6un}6Y%9VM243@^pm;*e;iwTVz)_z zczS}c=_^54y(ix6C1b6@7i8y`!OC zN5goh3gKwWxZ^n=UVaVyuCvEVpD;(RSkxSl<`Q`6vsFYQm8}Ee$TW-98H!Kc|9nJk zK>%$Wb>eZtKIu8AH<{Jk(>eJjNUn0d?7v435?!ou^|yj`wCpj}7wj7`q?y@gIr1LP zD@Zn<%>}LNN@SGixgO+lFjXI%+}~2qb=^LMJhO4pjN}(|K27gl61WYX?9&rJZcV4Q zguq-=b@Dapb@LC5oKuL%QkeFR<|UOeqB2Ru` zXESDQ70mb?a_1N5imS`X6$T=ha!NZvF3}xUVU>yI;wB=#o61Ts{vT3ZU@J#JmJ4fZ zFhSuraTA3t~%O)PkQya9JgYQ&LyN@Jg%@RVMyQNlt#zL6Zj1dSdCPcOrMx_b;iHw22`p1Q|Gjbj#yn4$nK{o=Y!a24og{-}iC4I3khq-12$$QULPaR3IK@*%--wAxn1#rO6AJoxnLQs@?vz>c6c)(xTP7gaso%hF_!N_oY zRyyw@^)2l9DiLiuVzdE?Smm4bW@z{1(U-4N)ssG@e&S&QBT8u6sfk#HD))5a4LQGq zT?t-!j2_6I1CHNMnhSL6tc8Ex+sxB%IFLPph}uGRR$<}BRN+#4SoxxF=Giy%4TVtv zK!`IqW9sTK5rd02-5KmZ|MF)xM|32mPcKg;S^gA90YJZH<)>fb%bgrX@Cwt`jR*E! zqdFZzzkAoSI{ivY_+1ycKznt01Fj@IjK_(vk-pPftLF9%Z=(h=Akww(6xMcy-Qtt& zxudMV;UbE}{_;f+Nk@FoWr%I@zgO4-d*eW}BkKY7)y!_k0w`a`yq8uy2~wQ}tSR)L7m@ub|XO1=O7ceX5E-v8;x+ToA6_n}f) z1}X^+iVILWJC7zTFJr~VNG0TVT~-qe9pOMznX*-_=dWFY+n!NY=@0e^qS7$Dv07lU zL4sq?KTR>A>9Kimj3t~Q{G9xiH&?bc8J^ehW2a(qFc8`SFkZmG5YwpO}*5fLf7v$FFior}?Y z{zCtcnPw6iiz3nyc9Jh-5myJI)as9!_iAGu#$EHFUcOY3wr|_Zl&K!6ZA-J#&tnOXkts|MGXo><7L)4~jGbImhC+n7mKWXN^)5S)_#eD| zp1<>P+3c8QlEp0s3=9vWNhUH)@_c@-T*lrkPSZ>d60J58UTKZx&uxE3!IO?uWhGnU zujFInKi5i|^K-wrA~IlwJnY5k)S$p-f8gkAA1yiYZi3 zGm;}AE>R2TfB)kIHAU5dBv?kpg#v(UoQVyx_0bcq=KM%$~YRo zP^`#r^b!0B-ldkD9g_-uz>bQYb^m5t@r!Jjm(QGS)y4g(3HUtj1I|$Duf-L^AD#799!&u>AAJAQ5lt`%K* z;Bhdpl#k$c+KII4dqgpKd5AMJ)PE5o|M_S#F(>VC&HcW^{k|34G`hoJaSDr10-#f`H}0zO9r2}0)J zn>ckuMnhwmxt0swt+bY@e6E-kq?kSczVpvaIa2Ti3|1 zq4>X%iOjqIMVcZvO}QI;?bIw+(0@6deWOO}`xD$@RI1-h2HH_py`jxo`Ezj}&lNP$E{YA)>SZ~8VKt6mZVu9YK$t5_VemEdT>;9`$wCu(Kj$iN}$Ii zrJbSkM*?r(ZMy+b7j3l>9@V%% zAF$LJrP&|>XgK;dvKoA6ns0m|=criI#l%$C5Unv;-m(Nyp6~ok9MsP5A8VXh5>PFm z(XqM4-}EfEuIz2nmPOEW=T2|zv>oMvpJ;0)aU2Ir<0g{d$|VFXZn&{y8&!{;Wnn2d zhv~z#SM0WPAZ!OpIqv*%A*V(~I&}|=UKxMx@ zn$fKuInz0AVt+6BgMMn)bEN|4yL=`^`-!3N4|4w8ohBVLrYCtb;J86@LkxJghuz+0 zP{w%rVl-ZDuo1JU=w7a)*FW%^=~~Y0{3m?)CXk}{Q9I{~5LM><(^_igmuG3&S7&rZ zbXq9z9-E({YO#|(Z0bI2=9j@+!;Ui+(7AjkH*{c75!X*v()Bc?r`N@`7V3aXDr|c;ZHrEAyy3u^LLZp zV!~QAn#X=(;D|HcZi?`VTFw^uIuE#eIwTX^+tR=H?@93g+mx3&ke=DbA1ex zzYe=>?lM}CW!yg~a`ygE(I7L23dyx;mV~H>Sox1p|igfcerbxiCGX!C=+>7qD5HIU@?ZI3gwVreCQa6TY2_cl~K{H*qUfzC%! z&A@lnMuID{0GQ~$02s0*6RMWv!jC6kI_cQa$Gvrqarr(1Tek7+7@y&4EgjY@h;q=e zCF*9Tjry&y*UzpQ)y;KfV@I5zXDjJp^!Gd2EvT<)Rl3H#q&JV~mg3(@qru2bh-7p8 z*u5$SEZ|vs)-{KDgEPFW2@cSQ!gh#Cm)~L~*ZVHd6(@6ZMjagGW0twiZ5zP7adv%N z@B8)hZiBwH{&;pX_10?pUF*kd7s20VZHRqrkwB)VbcTMkr>T=*_2Kis)w-cG2%Y-@ zB!>7Sb*Jm{ylT=TFq+^!O=^dMctCnYSv=*6|PV3YC%T(02A2`k}pyI z`Pa3rbPKM#TsX%wZgYW*?-iH9&7k@Q31{fon8w2t_ zk^G(P_TPy7;jb171`0%Koxu(o_IG#9zrnne*{cir*YTI%?s6AK(A_D(fozD+XbdU0PESN%Ap< zk|-i?;%oxL*xpXgQ6sl)u4}2g;}M*CeFltdY99WiO(?iMN8}(}@#{+l>6dvZxt~OH z*-ul300M;~E-d`?O6tY9SJ=PYbTZsv-|ne#K${PNKU)cBBgH?Zwkb0r&FqVea?!au z%M`XexKPs~XgHJ!S2)x9Z@>P>s}~ZvM1W``53fLUV?4alw&gIUH6lv(mB@ck+ONH^ z{!%{4%pmw1HTt?!2c{^P7QGCWU+D8K0j%ouk7(&Glnmv$p}gBGk3ot7qSNqqN0&!v zusDr0tHTAcV+q9^xs7o%#Ztr)Js?&Z0e&zzbHlm{my9TX_{^?h4EQ7kt=YIi1(zQ0 z;znY(MHN*972&qt6wqwI<3f+0|?lo*op7_Dtv$sjHnZWBTX@UJ-L4z z>vUW59Ek*p`BEprzVSWOKNl1EyVQQSzIhND46inoejwHYZZ(dYh}Cim_%05>{uSKE z_m)eVy*!(EIJf^&dX4f`>am;1$<9y7ZkAl5t@DY*JH&UfbrYI+MN-SVPq-ZuEjK@G zC%4Si5_OzCx#Vbz#m<3BbU`X-;V_Q9TTISKYEqxV@Q|TsslI;P4lfp!P5U3w^ zSfa!7QQT=jc=!hLu~huUavQd5A?H-N7+vmYXO}6{@zCfKXKh?<-kAzLZGKkmPPqIR z`eM5JoxuHf^H(1UN1c&{p6ia*eLbozZnl_FOMf$$K0p-ZbN=EY)k0zP94m2_oU>lw z?vMu!vW-4AADj~Ao#KKkmw9OW(DTvSC_iu;ODZ#9f7rOxFs|8Z#z>v2$}pz8>8^Xp zfG!HJ0=2G%Qwn1)?p=jpol<$nQ4yTcisEEeor3r|q(QtaP>1Camj}&!JLhE~q6ZQ$ z!r2VVy=iYJ2ULsa+CCcw;%$@^u6br3DrqXli99@#v=RNw$mm;w`K0WYFO^Xocs`BW zLAD9B=X{E40U?hP3(icM!dEO4!DRID=ZLB=g2>ds_+JWphyOW9`@Vo*q66=29C~;k z@+ZunCZyKc11XFKSFmxdQC?j`H6Zd=42utPt(yJN=TmUfr6mdEz0Qv|QTFlA{<4fX zGhSa)Ke04eL>rT|j0>6-;@n-b^eo<3S5t75_41@h?*;?&w!JMsh&COjJW#>o7rVa0 z8{J=TchHxvIJ%2h?MZFzBMBuzYHj1i!s=98nJN=JlK5B*7l~K}M?MM@Y^CwQ)hBSJ zg;<%5U4^D2EpP6n=C^VdwWiZgS4JGU1Gzg|R@NM0H%758ioMrJPRYP4noq)?d(bPu zD@x52*P=ArbP4pN`Q&mKbIk7J?3`1bKx^j`y8S!*0X%r~c!%BxOX%42rn?zb z^`*6y^FA7Qbt0`&eq!&{eCO3CzoywJFI)10dvlQs^)qiyr401Sl+%|6nOxl!%W?`dL8}yCm<( zr!LxF9srisi!;q;Ad_TpoVD&in0w_48Hl> zme^H&4qho#IEU700E)g*EDqq|wsF=>Y6~#;8|sEGY1|Q)U}1G~ksR-td9d`<{Sbx* zO9GX%D4`UXv#!%ae?%I_&J2GBB!wN`{y_1Gt|0V=i;XJSw9~&?(A?Fk)8<#7;E>s^ zd!|b_if2h_2rxfQ(nsDwZ*oCO%khT{i-VE?!-*~h?$o+ z8tw_;b@}c%lpD-u=A!Uz--D;ewK!})d=Z`35HjY2CorKZp>Y)uP_8NCnp?;9EA@Aa z49carcq~4YMj)R7HKWifoT9|@#xpF;hSs-Q7n>Q}j zfmi~W;o$*qs8?L18+`ZBaaf!ToAQ81=;uJjFT-+(}UhrUBA`V_~ zc^sGi%9e7>&`4T1^ZcP1pL2z};N2?=zh-;5+IwV&^x^~f0rCS-O)`(xHo@SE>5qTA zR*1!u*Ih91&-&759+R5i-U|dhB96@D$B&eI8)}B=@Vm^mPrn~pIyPCxeI>CORIyy@ zDaKkOYc1fGes!F<3`@?MTV(bQN0Zt%qmy(N)&#gZ-hep z#e7~M{i$D3*zS{kf0=oB+~Vo8s@!|6JMj5pz_UMwKIA}mh}UsJPc1#|sz&ARIzrWZ z`t3}8U6J-KmDvg(dQ2x=!h91Y z3Vg0w%K}<4V{xp6n_Ak9*T(UgajYF>?ExcyuVbw0v-2YQ+V%V{?O$=5y7mpPNXCwk zyHYN_x03bQ(ghdMN(A5#zl=M%cWFOCbRINuEkEZLI#j0Jk&z3`j#fDj%FUNLt62^tzYm10COBs zNs_^<5n}IrIFwb#W~$pY*4cn$9AeiC+aIlHn0PP?WxO!}Tmrs$7iCP421)^B3`8 zCpYr_N5-FdRHA=Sp*u5i8RKX)Efmw_fU!& zxMtNHgN1lTdITGPSSO0crqs#MKSR1*T@KCqsTE@6#i`LvA2P>+mYxr!e{+HctDiu} zUb_@k2|sjqvg&e&p${2Z(5*!AR+V>|NOX;?|EwY}y%TfG*9T$xKz;F{TLD$hgI^`1 zO2%&dO*b%ICpd=J{9R$0iB?*I_X)U zLj~Lfj3ykjihahpVIJe#zs5P#xFg!uP-Oa5By&!xvD*1+P4VUtM|SJ+nc8Eg8U&p4 z$?W+LzfQ5x=WF*pF%_uSl^nH;)T{NyEm`e?KD9Z)6@)^dN?`jUa?oQcR^YdU}8n%Fszfv*ftAI~DFKFh< zf0rnkGX%cG9jjI7^<8BtQ`1;-lr7&QB<7E*>$n$J>o#b&_NOYCBekf~`E8bTv-o<} zBn9kkE>kL+;F1({-{A?rU^&Z)VKo~pphx!VdWTM&V^S>6$-AV(M1Wd} z`WK5l?Plzn^!r{7UvJAy0XBqr`b7mQPWISV)N2;Fn)iB|<<7^rx{ce_Oi_83-T1;s zNZLCuwnRvILUd{Ur#))apB!n6TW{L&=?b`sw|%^%Dm}xzQJhXRdvLn<1u@8xfb@`; zp6z4--AwE6)biWlztr+<;wFaK-mA4w+KGfkf{!p?IvaKP#@6-0Q-fviAA6H+Qn;iW zK*hXJKiC_)S3DT_Qk`pg;WT3<)v`8h2P@Uw@vAAG15b8jMi;4r#q90sjPL5>G0+XR zgf-qBV622PNPnoh%Iyqhu~yeob}lLMsC5bR$zF+MbvqW`6ll7{=? zNNrvOa+_3^>V)1yau zAJ0;{e?VClHwPxlS-lR?aZps*Z+_RiL{^`J6HzCIcFy8=+)9bu`N@xUIj+t-GbQ2E&FH^#NPB=-!G>)jHT!qmBE5 za$W@A^|8CT8ooMSitG)uWh~km?zea`&YoC)1#c+7`hzeZVD1rr(-=~$SARTjspWcq zKBUUXwdQ+F1}k4QC3w;et3tU*w^sM{I@~L^GVG?4^rDVX`U7<4%iN?{2rHzZK&;zY z+IcF}`*pl`SzQBhBd?J-=V#v(Y>anHelsEI_v|G$6acDth~>MLNXT~FwSwz4g}KoySaqRx`I2O!1AIdF#9>L4 z2E?jbQ4RYSc9UXF=P z=U9#UEyXL({M1|H4t6e@jWS))E^c7t(cv1lQHH)CJB^O18#CVB>TwgY;XZ%$>&q;A ziw#SzgKZVD(Q^G3lzHZK@U%U+O|!0(=dBLzf^PWTu0OHENGQ(8P1XGQcO6iLi3sYN z8&cZtN9>c1og&Q1^75sQi^-*p*|OI`CA~`|m7{*=@!{s-4XJ{?{)iyv0V?qz<20C2 z=uo=$2&dNxdqq4Otw9%&t;8W=FFNn3J>^pRG7Hvh>}BArI?)yNeUL^YNmGqPoyjy4 z)rf9LnJcDrlMJ$`Jnp>mRrSFJyAp%u5TSKB6ad0^u*Bq+Zd9j3ACs}>|DpT!L$CnM z^s!}Q4vM(DZT7>LA7@>5z5$z#u8u1yC|2TQZ~(b(#$Kzx*`bZy9Wp5Jg{h)~wr;u{ zd(IdfxZKRQ04O%X<#6U>HyrHj8b3pC`bsGc2Oil@2BR}vxQkQWzlg<5YxTt1t2He# z6C+z*8ecPg;Q@;GjO;BdcQ>h7eO&*A##u25X?G-p!MY#ZPgRE^)5Ai6UtDfto{z~y_XukrB#DHun{p#_}>Dix1t8WjOm z?WVxIpkScjT)6Asgbk&-{#7~1p{`1Eb6gnfH+;gEank+%jbO#eYqUg7E23FHWW^^} zya)+ia^^^aq1Mt0n?#P!048 zzthwSJx=XVe$@70Pz~DCKs*N^ZS=`a%f!aflL7hJDG>hWj5#rxIxo9k*=K-qFdrIBnj|Ucs z=`eaUz^Os-9+TL<&%@FkPYO-tBMWrZ$2nqtydzaq3pwfGns)BR~NC^-^ zlcsd(U8MI8p(7|rS9%Enq_>0~THplV_gm{*W38d@-ea6S_BlV`2gwNYdFIS~?s>JF zdtxW~JcO7ut3&!Z$Hh_}nqBu$QOs+gGS>4(zqPp`dKl2v9_iHcj2!7h0s$t=)7?I& ziNnET-IGO0V9Bn6y%3;Fp3s#3=ke1H9e_R;P58`4kjEX0v1Q@S=<*Pb?^@UQ*O z{Q*8h=FP`;<(`h%BIj%B<_9n+;Ibh<5;j_WHBr&HhWHxYG3VkyTMuSMe`lsDPgxs3 z%BC1z@%$(RkVifn?ta;;Zp%I&+F{y8SLybYN!Wf~>y1XCEm!wc>|XhqZyX|FS1k+( z@7ko?G-e@_;Z7@=uYf1+gK?o2uN^kJ819Q%-M3O7XH{9>>ubhB_K!}`iMXPnzZHu) z9Fp5f90zaV90xfSMpfl-ukAtb{U_Xh`Gdn|WO0$vd3T$nLz~SOeQP>eMF-#K!!d!U z`(shr2_Gt+o=PaePiF!`YQB1|u9PLGW`u9s=D4h)^bC(=Nrb6lLNOSYIX^yK^&c*Z zDc=KCo$S${IBZdsp07&-RnClXzh!;JSsaaR21|bm`E~_1E$^%nQ1j#lzxRzW!$i)E zA{M4ghbyfRPLjS^xYp7pA+JyQ+d(2EaIwYkDiKhz?g-Wsd3l@j2~K3!>+^;^&l7j5 z#M2emsNReH%o|x~b2%*O;$ zV}MM$?>ez_wi({UZ_I9Y+X0=oX1&e+;;qN@-)+rSDg^qHa7^}_tmcfFtfq_v`dEkS zda3jSd-sm}ePHaFr!A3t??7u4@S%I#Mns?LawZkVqEI8;($7j(JpimP&*j2{`i$%z z-OnD-kugXsFze(cHx3z|8k7DH;u^RX{Hp#xh-=#DV(BEjeN&7R`IR^#$kzDrD~^L{ zp4b9e-Jx=>eY!72QtCwuNioKxOHJgHTA)Qr&$+57#E7I-;3hlTDhZJ;%h;GWB>-AUy_!7 zz(Xbr=H#`sO`;)%g?A+TF^@x?Zc#@~p?H9YXoSGGprWzh{IQ0g^<20OLZ4QX5Do$C z6KwT9dF(pDXpLkt0cjfbX^e3~kp&whaNEc>S4hvR#a7Z~R4#(`w4p~!@JXg+?AOzb zyDHME`>+bxEtkz;ho=hbI3?VsQA&QjTi-p*uD+aC%=Bzx^~X*Y&qJg_4`7Z9)fQ#$ zq*3I6;2t^&5VLl{y3pL_PRKzhul#Cem8>x8hJTz~4XPrpheF2S*EmA8l zr&c{*yT}&dvtD{ROd0MP9&`CkaR+!#j|&wk&V@Bw0%ZKhaGfBxB=C|31DBUGnSEw% zJ!rMUhNJyX2Y%Y!spy8u%82KLI3s-h8urWo#cg|P|1h3zNpi>1GN--mYFMIcwH4sV!+C{Cd$ach z^M!DO?G*}&O@g{^!Mm9COGEF&9P5jOf(owK5e@1#(N+J;wwu!o$gvrMYV*BTYDM4h=+Aw zF5F}-9l0-SK6ZNeM$jDvr6q*g!@{HIX~}nBbE9UE&6Sx@detUrH?#D*eQMH2=oT~) zoIkuG&DViRckZgG@!qB0ERwzwQ+U(niLvT%wXYiqHlc%LoZ2r;dT=TTBP z1u5Cs+4w6-6CRyHr@~dFM@8C?_O@nIdB>0)-BIVe2Od##p>wVzzoSSyOu2q>*a!c{ zVRIpKq#D{ompw#?V~Rnxt}@0v&J*I4V{qM_^P@cPNjlXJMV%lc;of-6@SomZY=M(4 ztW9YxJOSrc8?Ya#VR04lbmgw0QE3iySu!4`;*rRMgT2puw&Sp(H!VDCj!9li){bhO zJ@syG=>e@g#;(-A8zDMMq!t}hE)#T35g(U8nXtc#eTJ-OYFrw4;h~(q2NXUM&Nj)X zAVe#@Hv~-iD_Vg?4QZOT1{wES5aboQE6vR1xTL^-@DeIY#%oZ*`w-Re{wI~si4Ie> zfHgxk#9PRFBd&GA1BVDrwfgWxr3> z?(jMzyVC!UaLoTl=yJD?TkxS5y0cg>qa1>nOUDVtoc4#woICfdB{ttO-k0Lu&hqo5 ziTyL3jxv1l;wHRIeQa{P3s*os;`Wo6EV5${EaaJt!OwmK^~?{(9yM9$!15n zQennd+H|j6jdxF>EHVT%5z5B*sJN#EYpHdJQWpz(7okF@LJypK&#$c97Y!)yOr~l2 zGTh5f)Mxr4H2guuYf%IOxK}cJ2l}Ma$N9F;5EZMvX<^vyXNha+vn#EB zbZ~=IJWA;>$0s{$n^jR}qy@Nd83S864_{LvX25>#WH|7s3T3SeGKojT)(k6)ZPHeY zoj(l93=I~UlM_+Gqy^INhy02P?wHHz``*njWAC)rh&ptltJr4scrj5-*n;?Kz0@I5 z6t>&e^$%LAL5lvaW2D|wcENj+aTxBuSwe_{;6&&kbqhNqxzP8U2u^~fep)9R=k^p{ z5{;+8yO;o0XP#kn_tY%^V69MIYZInUd3nbD$q)NtPc)wwaX^gkaXGF4Y+aK1JRLqd zj6^*tFuKn!laph-P)HE_9(g0aohE2pVa7!n6S-~{w`nZCDFb}noNf`3Zc&X5eWidf0+fVaD_%_Muqj2u zmt0KPp1*jeJ}$TnS5wd#*LpO|_StLf6_qE?94l{okO(oqv*C`A6>}eu>2u-BcDJ9@ zmZfL6_YOqpKt}kfn|7Lzy?fdX73jW_dD2lKCBHd8{iWr=cL=~Bf&g2;kvZgvy4H~_ zmDwiS#CMyCMP5nnQTqy};wpsP5AyH$Di7iX=Qc+FYk z{Nk+7>QxCp*L!{D_n>3Wrp}wi)M7b*?pH}Zt-qcnhN+pKCC*P)-=Df;c$BCkUy}Iv zfps!pbLs!YTeIci9*=-xs_uB&7u>;(b)3l@lPMm#2qweib=4r&{si*k?ciRkW4ay8 zUv4-J2#gpv<6erdqK1y*mq^~bQS@}af&TWaFwSFj+`qYISw>L;8Na-<{As;u)asl~w4(9?JkFRYWKHz~R-AEz0vytXjj}!QGVf zb;Gygw&Tr+=I2JszI_JBQC`D=H$UX!s(h^in>S&&vu(_VzD1hHl;K~#V}A*rsX8s- zXNndLyO`EDi38d7HY8L=Rdw!}5(wY1SOv^?CX#ncd$-Zn{W$cHi53fu7%_FOPqal= zr<3#JJa-H+y18R;;E+ICmqJsJ1A4)$5wx1xS`kpEj$^bvuHCFtDFK!RX(t{ZRYY^CWT zr0Kd!?pe?n4aq)~8HNV@qBLKhjpHiITU8Rf5ITKkASo2lXIz?7B0Am-$6Mwqmz(kf z5oHaBz11S;SsDEn?_B#mk73F(J>4May9Hc%Q8vXH`u)jR@LPsh;L`0kWp&Bt&Islu zdCtQlB9`bxO;kTP;9J~1?~QD$mh4Vb;DWCjrX@)#>srG{oBUy)B_d^OMUdO@RNU;# zDx3~!gTW&hjM-UZ$N?=^l&DuEfzUv3lon`azza^gktH<=JKMqDC+Tb!Qv%ZWa&pBb zk-2wVmn^e}ZY2L#Rx?GTwcx8LKrtqxl;-xm5lgR!A0P4Dc*m~+eit!k^Okl?nzb(K zexZaT@`dgP?Ct8ps)XLD+OO0QiRA$QWO8^YwxxMOGo}{XEDv-(qCq?2drLuw)Q$Q{ zgzSiw7<5Fq@Hq~*@59?^lG~d6lRaiKuO|H>J6gxz@gKz~{2~BBn8=@8_mjCsS(pfw zpQb=ri3<9`dTdBN2K6@W!0#GIAmts=AFk%TMuhq=t>5QOp7W_oE&P~8U4VLqDIi_1 zdoZ^i339Gjji9e*E9B-^lGupc+2W4BcGqyVZU|o z4^j_L82<-SOe$i66ZT~HTt@9}Ufr!c=7fhtNWsjoO$z;ab(PbfTyN_>*3dQH-yl3S z;kOoAb<7Bo^Rn0Jy!psm7jU)X5EPV~pa=$5>tbzTG~)^O17$*Ln*g;p1e;$s)~?=C zZ~YKWjEIt(`1tl@&z4mlEQ)oiRF^DPwIy~!_1x@MPBp`A58RB5{j{a0H}ng?KbPAf zntw4z?F+<(E)qnJPeqGZXtm(Qy*4+x0WX&n33M%5c8dSlx-^2;aJ6c59-PYG;Fhfk zx{8jx`=QUFD5U*+h|<2etKXgK3|XUZZ5x&XKl`4<)EY?x;*Srh3C9*i7@c_v?d05w zaN`UL@y_kp4*nZCBM z=bJLiZFuMmYNdh>>V32Pn|?kB+%IZ{os^%w9-h~k!gClFM!nu>kTAF`D>dT$)Vvz! zju6kx!$BV8tSrW4A{0ozsiXyDBGx{iz1Yb+xOb2w$E1)U_oJrOlzl#K%V@D26KTM# zTXBF5PpZ_P=`}ZRBttMg-~F0~Yp9=N@MUR4%TQnft?z%68-3?1b4ovW{9Nd^$X(fy zz68&a&dmL`86UOZX|JxELw#@lZ#2#DFzgI`Q@QN|Bj5?mV-sD`R7xt*a|0Z?(CF?j zdp|0n)`#hnd?-R_1}{f=GxlPpKRK2-;mw-DR~D_Uq20 z#S2E6*wBJpBU5`qK+yS)DEs1xG;WV0k>apBHiJqh;%U){h8TTChP*Lw>7l6~g@ty) zE^Y!#1j$bwnfh1A!WUf5^W@|~Rkj$4+%daSrg{mKWr%Bnj@C!R$R3pI~3Y{N(#T`ca0ts}_hI zUytb4Gk=;*)_$o{$I$}UO=)Met6_4sca6?x)8|p<8`>dSqfkPQ&ZGK$7Bs5iQRJM( z``~WU(5IgRQ%Y+?{X zXj7vdbRFO-Y`6q~&lzcp)4t&ur(4j_!Ob;okxSI9Qt6}%0b$xWy6s1cV9V%n>aEN@ z7^7VDKm;`7O=J+2f-2-z%0awh{j7)ONq9&tI3Akb(zPpFKXS2DHb%+;jgeo}v@_bj z*|gF`yx)Hzn!fW;CXf8b%d^W!KHC6LFgCnK;ubs;4>)|W$!~zNpf?)#4#0+gLwjcB zE{{gMWsHpwlul=2^{?JGQ%IW^UuK27(jFMmQ624s=^u2Jd|jIF@4`Ex7d0RcnvByu zDW?%U!TH(5Hvyd5Fweq6qMeVEWXQL)|Al!_TgYV7p@{z$e``qK{GOJyIS2GWOM;bv zycHU4RM&(iAXd@y*@^DfWfk0v2IgELS+G5#YUvyKYDounb|0E+aA~QfLyC-yy%yAp zd}SmZu^cMYT%w;Ejj#@M#-WqUXf9;`_~kUXEwI+2| z;unBVI`*!goohu#i)*E9M7z!Y>sAIHR3JCcC#5#hfU8vk8$Pl=R~VtNuDuM~dOSg* zMK>&I zkjjouprx1oF$4!+3ZiisUS}Ss5G~lR>2na$4>%Gp-B`}u2monxpM9RY^0j_4RAAoc zfHBTi&KbXz5|+nEbVJO&#GRfB2c2q~)zOOPUg{bK?yD?9s-GXI#M>Qya8s#3TKEJx zFrh>8ku}otIVx}cd#8Thv#Ni!7=ewHJUA0Ky>idC3?LEH8;xleAubdPDs}%*9y;6; z@HQpPAEtgeaIqtrEcUsz@s7glGlap3m6PYGb@+~I+;h|^8V{GW71UbJMEbJ6s zj4bb2miV7l0~P z6Uk4n#Bm5tQq#WQu88SwQIzB|kQ9nHmT$KwxxPTWzLljY)S5~t{q^=%B(V9??XvOV z{ns_kzI2EOOsLybl)=+z=duBm1Wv5}Qg7Gki(6`!)F~{u@+!VS%0J+{kBR4N7KXR4 zSBD)F@~5l|Y=BFZxO-j5oNOm4w!Oji?NRS@asqXrm26ST70e z6FPK)dx5h=OiOqCr=j^M*XQ0NOlL;9bH<@(FgJL=Hg5PElQz5#xOCF z|M{~2{jZN@?q7R6g8#Csoa*632lz=mRzV+9;w(1s*Lu z;>DTYoG{7yGhp~&JsJ~>(;49YsjZwwJ(=&-(}#f+GbYMHW%$NQ1&Xt!}Z+cV#;R84;uAKE2p0tUNm9MV%Y zOH!mL5qnSNVg%z3)A^lwOquQ2X3z+?1~&|o6}%jpaPx8K|Dt9APNKFit8qrlw;$9e zwTI%OrHN$fNj_oyIR*3^v0=s!aSi_5i0bkDwf_)y!+1jr`p`qBTk_t!7)rmMY1gFd zSV^5xqK~hXrJM;rukR6dKxgpuCZ$Au9)0qi6}oKB=xCMn;~nwEB$cP`Om~UBO-Ua4 zt@#mQE!q6rKjH_)TbD1r|3$$ZeaCV?kbQ?S2%UvGer{SnpF|wwSnW4W)vGdFDLos! zUgi8v zRt|zZ47VR=Zrh_|r(wl3q_;yHvm#v5q>sNWdstQgqFB}7PBIRV($EzpI@R? z4$*o%1HsQEAUem**!?^tePUl3_yjRrvOOFwi|~6H2^jq9NhX(@9>soyiFf1m0L7*r zX}DNk9&R%y+8z-uhJ_Z{K9SH&QyaC8B$*$kn=5AWsr+CPG)017L zyHlRB_G*}_#QI7qBMY^jmRwhNYq3x^Bi-CnH~!-O*vm^81Jm=v!x`U0YD@|;xs3C3c_t3MW-?* zxa{<;n^7Kt45{(7p62k;)eGG{+3=*IUCe1hi@DMD8s$(9uQlU-w5_p)kq39R4(-Oh zZXhkB8SM^9qAy@sV_eH8;9Aqy5us=fpRoJvvooDD)l_${r}z-8@Kq*yJAvnw+|#5n zv&i&+(A3-e!F9|a2|8vlprj0jhumDs6FrxK^=))~pQqzq#Eozd4Q&g&#DTLiIrEGH zN#Y|0e)%N=pK)jcO69woJZP>LgUyYwvk)x_J@O9Ht*DqE3;99h?RV{>Q1H(SP2H2n zbRvSJp*a^`U1Bgvtwt=bUobXe(@-z+>c$xbM8)5nMOK^$ohVL(q=JuFNk3V2=Q$t; zmIk3naToe)8)0c$(mW3YmKOsIac9XvC%h~1By=8C_8(&M z%^YHS8=A58XsWcC)TC^t9(ib7z+8+2noAamxF?0O^F662zz>3jq}c#6POfClfIhnASPsJ^tfQZK2Oe%L{D&4S35)=QJL8uD)=B*lo< zR7hW_550cmD?v8;O0|$J{z&@98yR21X#Cb7AQAtr*4V*R8@=ADjVCP+Kob9p;zHXK zOyKkt=Ct4V*w?9MPYTr}&Prv>%cj#=7v9m?iMI0&YMU6BNugdyP4%wUh4S`bglt^; zEO_bBOqbxMY+z@TLhPw0e#nA&)XG5T#QQauj`|zG=QX%Hz6&ycEKPwG9XIfy#(U!h zc|=V0FP9Nn-IlE;|KJODcsIyO~jCsp@;fq^3mxtK$ZcV=E(`h&N`v&|8 zxtYLa`CJ*x7EYOr@#F?~A|}f2{GdCAhlLz#i4BzW<9&Z&mHa9ToEV^V-O?n4FTSn& zyv?GcNBLYt^GKrmW(Om1^C@BNCPqEAcDs(@@!Tr z$M)FpV&#@=_)_79ZjEOTF7Dt#E`^!VP|`!LwVe@dX?deQRuz>~27t=x8J>W=6g@(P z&398yfZ=|aNY5sQ&n0_OZL~plLv8$UOyd5V0Zd!=ESRH;7xAi+vpL&&MFMKenkk^U zB{Yz9SpnzO*z)7;3cG7W(w5*kKttkEltVG_&5NrdE7kiW`yx+xjYqBsERu!+_3M*d z&m+QfXH3g>2%uX2M(n*;OCSw8Cpi+>?V+xzik%tn6SqtgG>8}_$*%5Fcx6W>Pd$to zXqlGBe&Xzkz1qLACQVQH>;h2Z#^GsNKZlP$6Wtxm-a1b_^Y8{}L-NZ--o0L}0?k8* z!$I|&{xu_&Q=|^ab_L##96jtH_m%3Ex5&jqK-Dc z&1GQs)~>V*d^1XjUUQY{4qxoaCGjDsMkX@qmE+3`?zsgl}Uuy+P{gV zArQQaQT(R5vd+#oR1Ahr8#;;;>~1xnd7w$nAkMYl@p(G6-zE@ul84knZ|Ez{Y9OMP z+V9VGihuNf12wGx2;bAo<(EEQy&JY-h~~sAv?83NN#!0|x-8&gx2N7pg0|`)l=%!h zU4xW0o0&}Sq)Ew^49vAkd)CoFuFh%FK_Zv}m9%6`YgC|0!hmLtgp=MGKRWdf+flBN zhK}PVa-|PgK(m&BmcEc9juQfs^=jFpHW06ETd@tIX*tvFrBgZ{^sOX)yGRhCwk^Un zQ?~}aql@Gyo*fXV=eo0kE$BEPJGJk#O9N}LY(GwAq@{c zx+qDmq7?AUrPwAJ*l^;8%(m`9cu;dUTlRFB(#;*8eAy#8bN6AUp|ngJ)FweR!q&q1 zP8J-1bUc%d4WLe`?M*E2!mLL~+1Peh$UKv?!Anmik$`jM6bZ1~Ac3L5GsPD&)IAX| ze8p?=j`B~6!ah`*7zls1#AIP=4YOS&F2UwK$?cxZJ*u}ZLm3g>*^987YCJ^yO5~<%MYmSbt`U9 zr2L;@u*U&T*tawnVE*sB6!y9zqY`%Jl5GI!4-7abY=61{H-5H*?!r{$JYU2CZCxxa zqa#r=$Xzw%NSo}UhVUeIfY;E8rP_-ndQtt^1g4OHpl^@khXm{$r|KS~#2~ZvDl4cx zmPhXa8xOv11r(y8J>5lk<=opUCA1dnoa|WeNtU|w4FV9|KqAXiySzBNJUc;k7a z_0Q)Wtsd4A4BfYSQ|gvZf?3K0Zsg{ngcRsDL+kpe_C)~fh&Id-X)|}082bZ+_9FMQ zBQ%hS3$7pg&!RKscovn|X?&YeG z1)f=mJv^EpnVt8nL}QLXR}BcxDlXJ70Nqmu7s_hFE#LWQgX+Z=1%%M0AxQ*3GRyn5 zH-SF7H>873I444!8vNk?xhd-%aWy>=~0Cku3k#fLAH!D9UNX$We8T zcSlp9^22MMJxkJZ%f3WHCgsal~OmF^|4=+7c`k2kB|OzdttsPEV_KJ zo-?h^^&7rE#asd?hFkCEy)BTTd^3Sa049KLpvL^k(02Ghzor?Cw3>^7- zD^Q@3YkjBrs2&AE+0mo^LQ#)fe8YKNM0Pz!N*lwf#_x8*3owOf=^5=ybknVumXO+f zwTprynXwb+qZ8lO_V|ariMb8*76y(U<~Bchvx9HqQIwk%oPSDL`LbBI3*D|*5xhHd zQYU=0$WQ2VtUDd*Bgrr4lw*`4bNSpy7<|`*$Jb;vROapLn#628K76QapDW` zk>qm)=7UX(6xue$)lj7F&=vrA(y*Y**~fZwF%@lKYe#E@*MjNFjL>}mV;f-^^ivJu z4-e)D@2^YPB-!~U=^iw=Pnjo~O9lXwG;(V?lq8WT%K@~~T?OP>)(P%pFU5mJCab$b zSm&GX9cS&kwf_xf>#fqf}xJTfIHy%qRM#tFt_YzxvjI@iHJBOsY6dgiri z^DFo%aRoGi4%(EvY1{{rJPG_0Ky|IHU~)!KhSg+as2Awq&4)5P8eqjo)HYrPfC{Kj z_e%aHNIjI4mhs?BQ%pPZwmqPk7q|8;^)$+q zCuzcAA{x*FX7!`kf!+B)87n>X?#zDuvPr1V`ptX_PH685d^yZF0NhORoo?7NS@p^N zV!fgBDB0rml_#T`rvpgGtjW9kBJ9Q%g#~Zs*1BfBHa=MNU+m6BZ+UnOIQ}MW?Js5f ztq<2y%kpC$H2lYB;vfs-ze`$C>`r)EgkW1eD&PXiB8%{q( z^&`>`$YL+X6(srYY4FOeDkuC1+xGnRm9TzGIq|9w{ej;KT?;DPj+>6StOe`|5Hr@cy zsJUFH$iF`B-=X=xvHt(GTi}cSC9Es|H({OV*t;tZ6n0bxPRyCA53+0(?D4r{4(Bb; zAD8*{zr@%4)z=lRJtT(AY?eC%>Xw{){g&hv+2m49Z6=&T?mboa#PHv43GIoRh(ZJpAUpKha={)W-nv#}u;vo7ow!Kp=YOi^w#GHx~*`1Y- z0-#zE8b4*)TY}~zvfi_@G)<1mU|<17u>JnoHFW%&i!nswb@N)MNQXxymBFBN!1)T^ z#oE;sR*2HVB@-4f`oZsdQvBOA`g8vh(WWN~@6Rhyp2jR%@Gb^q+c-IqV5AgC**)qE z^uJTDo!&STvAA?~H`-8ccc*vQMQMUoU`4|-h->8(DfcG^{IBe;jO)shn`Apr`!fu> zv3>uq(8yv~MNem;yw*t+i%z2zwR0Di(iN~a%-!! zJ@6Lp>fj1pd(``KD-t_k{ zOk4Z(Se^_meuEv9(FZN3sO(7kB0dz}sCP@l&y0@(=21frN9CRIYw>Gk<8;*s6oYJ@ zKB6Jfi7iZ{qYP}!LdDGo8mq;lpaPvR`?6u9tfLc$i?upfR&;5xlzst^c=~+6B4ggc zc)?hyR%$}+V|pp&MO-L&9}t`-e***i-s9i>YQjB1p9Xg_scAR+kVVy1PojHR?Y)th z9yRUi=T|00Te7idN)dSm7MMsgCWqgC)USW!qu!x_g$&RCiSnnAKvVN`J*}5-QD=pp z^Lt+T6_YPDA!_^>oDq(lq<>`Z*VgB~Er^t<`h zMO&(Zn*KxG7^xYy^k_jxpjkE;rqf9gK2 zO1&BtRa=)IKMm+;9SsEjepu*N{VVXUXCfOWM`j z=x8J1{(BRMVq&IBGtgFf0e16k!=Hy5eLYlPkU~-i>e%~@@E>P#!R!H@GxF8gZD{M` z9Na%l(QP{3FWBgJKGehn@_e@m{NuIZ^I@Rhc`LqS?v7yUZ)gA4l|51Q@0bh6V0K1g zMzZ|zT08C$Ul08;?*F?(N8Pd!E>WKtE1m{?`os_jaHebi{r49AY5>T%qX82crElug zRH`327KFPb*b-gU?6VhWB$125qN@AY94A>A?ceK|7Mb3x6PDHsrEWH`@$-6*c_tP= z_~&bH+l5Oecl)1KqL~4J5dIqaG=WQn0A5FF!6rFmn(aUDnkPsj34BagU zV#8Y%sVE(eZobzT{L$1h_dbK@k>6EYv<2s%C;b7o54tgAv5$F?QMbDG`f{kyr_b5N4XdU#opx8(9V_@cO}@>p4wZDFsSWlZKyB6kOuSQ5Hhw3 z%jW$1^~lIC`hQxO5BV{ri6BUtil%*VvBUQFrM=!%nirU-t;}{Ummf(l4;8FBy?VV% zb(yow_|E<8j)4G#-hxw^zS9ut4o8Vg<^WOIk^KZa_p6q*<-2*)ed2;@>{@sGppgD5 zDVHv%vJ8Ov9n=Tq8i52o8vtz?NeNf6=!1_!l-ds*2pb07;5Wr66Lhwv{Yr%aMg@g? z=H~lo(x4%=L+a;Nb7|K*+CNXxB-cVt!1*(sJ>SQX)NnlDM!U>KNmRP+#MDC11ELqB z?=*({R<9~wkqruw?9Q3Pu;KM*+OOdb6V>rHYWVSX;jZyjdHQKcgkC)7_X-}vk};|p zOz#CuM}FC}^;HTz;ZROy?x!Tve9AnvHmHfacm|RBWqsOlUJZ3jllPry`OgLL)V^i5 z%MqhybX_aIE${a~Ame|$KX+MWcGlj|KZ7F8$dZ?5w_oBB%&!209ZFy|+9mZ7`o2o_ zx0)Pi2d9Jn_yvdRU`nG4z4fkk_$YaeA!xTW*qi^0*)X@eiDW*Iz^C|_yGz7wUzIB? zbLMsUK^aeoxT)r2)em|PW75c)`vL{ZO*09 zT%prGH9VK};U(&1Ua&2RhbU769~ z_Nt=873Mw;`#0NWO|}jCLsbUL2XeoA&i?u~ht&H=quqEc4m=(~JF>A=@2ep8lRR4h zKkm`B{7>MvBAcui-7S}0ufT5kdB$vrOAb$Bh_ME#jaI7C=xeQQ%F8#ELKCCXIg+JU?-1USiwjLk>yXM;5uJCYGgje8qsT2~*>Ykj8yw~CM^R9BRf3TY> zbcCWC)Dn;-Z91qR`#gFyjiPMSJqJ6LX%2ikq#=P1b+mrLhR<<~;b<+wP7`X2vfH%9ipYyitMouR z4C26}qajuHcJ$Bu%w!uNg?&ZjuCKC}{a1A^WgekXXiW&nJl)~RAEqH7oB7wS#xWqc zUoa@R&lo62!YB54XNTH~Nn!nEziuR@uU4L)^hZvXRe}r>uKp=~Y6+8~kW~*gUk;@o z#GydSqclS3iqTssEw6I}`}f?1yo`Frc7hOeL(#cLc@X4z7r3<(Wfz~4m}gWI((HQX zmBxXGH}3FmLbdsz`A^%qGhbGjsm(hNvW#Aw)wZ6;p8)qirBnW~ z@Hh$i*LL}iqM7?U{041)&r+Pw+`v4{d$I8H@{+yIjXbO>j**1SaijX{=|oz`+=R2@ zh6=wWzei6$5p~&2S7=?gDtp#)iZ-lt(JbcUhaD*nC^Z*_?(ROvis*#0$mmFq^6S?D zEYKn0#ro$8D507t?`0=<=wYIA z|6DTpcVt}GDV>S`UoL|FY5n%!tDeah9_v4Y!>z8I)sjU&Q;j3HV0WPv#r1`%=D}a> zX2$ry^H9^~qoHGz^4$Flv6rLqL0%MF!LkR9IpRR+$D>vn8|pdLT3(LNk^l@LNslV_ec zvcjN?ZCQg?DVu(ig$mN%2GM8pE#GAPWK7rgM+BFq4k*BJgB2n)V>mJ<8@57cWk{i= zcYJn5vnB7pm`gN7`^gHx)LJ)OLFzN}X%gEr^RK6)e*d`b(7d-B+c8(X56-hAmr%-% zqn!v#JSe`bmEGwIe2_+!<5G>_(Xu#a3m!@@>g|#lRiMq#4xgq49OE25+yIF<%ar>K zoHdrC6VWo2!Jm3e=T>MPf%S+rAkX-QL0(L>hix?(x^lJ17!p3)`oT)mxPgb{R51Hd z^?L2W*$-j0&83+&;FW5@)k(xBnCU0222e7*a%%_9z4nKVeZh$cJ@Qr&FVj1YXu%M(N#hBpyYLR zM&-PHGR;(rQml*4QN<_K@r0q#f$bogmk2TNNsW20-zc}5=$=H%D~CvY@pAi{dLt)| zc+e`P`8`$R`9cktr>UABo5y2V(jleP@d%OBHg1xm_+B6X^DgZbKi`V-)z-E3C*RFI z*thRb%CbC!S0Fy9;ZzIM)#w|wIKBNTC&}tz0^pTNCxP>m!2LG$7cYlDiMTBu$rLVq zIC#`GD$KrpiY>J+Zfk5BD{XQd5;90I-jJ!Ne^PVMBr&}?aA|J;)Tmf|-)ypu!}U|8 z;BybD3V+vU#U{s1ZB3Rkr=D|UH82`TFM`i=&*4xwV9LYb>D9<`+1FsUL&ve9HZ?(+r4ReyU%es*nuw zX0Vf@=2omsG5wO->Dr~K2KM&&kN8h!vWM=(q<*_g_%~52$(RJ?yyJ_AjkdKnM}~6# zLnw}Clj^zbP}PddkL74(oW_RTIHr{DNej(BV2u6zvYSkBPBj*8!ZT+TmGU}@&y@e= z1*2`vxS3u)4`bE_=?roj*kd|X%PMAOoSI8wjO1kgw3d#Y(J(XLJUr-`B?il9R}t@W z1I1h^S>?~XO0xMGp?O!pzBo(Sq$tYT!v?NWLtW71B!@UCC_6*S}lmCgqP^5*PS)SiWS z>Zj?T@P`K!MzlZSEjabrZpFD<{%KR7HP=`iJ>y+@B>T$S!T9}zWA>H7%#UG)UxV%@ zbj%6B=U{@g;2ecQC(PZflpD~ihW9h~iA_(|q<-J94-|+5a;=5gms9BKrZLBy@-+b$U1B%eugcS( z?Ur|#wrh=X1M<}xikfN-g;<;kqfI}C?U{~jIsrH5LRVeg&nCTeg5j*-*!I%l7pOo! z(W9PR*l1fbCJ?pC1@=>#A3Z_NPK?h3TmpXEYA<($03Q56XZxgBL$0CDOeaHJl-IYbQTZQ%-2km@~s+3|I z!QM#ZmOWcBN}wqmN$^+$z8Wjkva_{)9En}?uPlHcZ&{qLe%vF34zKd)p%6_K=9j}F zz8_sr+Xd;p1J%>)Hh3;kTxu<6?}u8An6IdVDU9O_{iaogjzYXzIVz1eJxV&U4~{#e zcGFR)5{fpuUR~YUbjao9YHNEpYX9bQDxGc#EMNQmx%&E+q9OB_#Rr4q6gbImQ@I6p$#@^F%`;?7OujoqMV@;<~I`^q`Qyk?^>c7AJ$ zpLi(M-=~_f*mYVnav@L?zA2!Y@BV|mDu0t+jneUHe=E-PivaQ2xt ztp(}n(&o2KXd0UU#D-9&>) zz=|I*SIH1BjwxgN{v&n0oHhTKIW!Z{o328iYw`{TW9~o)%fP;t4R)&E;J$Y?W=p@V znFc^W)cf*jeysh98?3VRCC2?D5t z9?3%U)<~(k)^ov(b{<>n{m~gKOr+3hx6rajqla0*pMYs-|Mfa0d=n;a9g0=(sUTDeJ`E#$ccxb_s-Y|dp+A>O%o-UBIa2Ow|4zn{)H7`P8UlJRRe zZk~pCd^5M`fv)n^&}i>}!WRmz_;Cqk(wg@TZF`YU`rCbXA(P}9gIy1CT7ciYr4~}^ zU&mk&J_6L5leSbyru_W|1kt-jhp~U*x4lwqFX9GukI&}=rU~G(_)QVNFJ&I{U|iE4 zSO7X_+Nm(cKP);lBAd;9eQdi;3;_km6aNW)eEcE|_Z|eq_YoUz_VEYyZ*Rf%Q~lv9 zO!PbP+0aKq8$sDWJ?F7Ql7F3XmC^T%Lha8(r)Rkyxs?>+AwT)-#^2i7MinZia)tzS?P7R^3AeyHd$a!n(90!9Nz?<83fl zNELmmHad+IZpdeU9~lnGp|?e4YYVAGz+pIm?hSi@t5T@4y&#`=8yfxHnL9D!u^G=? zel|nYs-f(Xr^1hY4cvtdVBww9RVhq2*5}iGt`W!(1F~ z9U-A_;ku`ZCjSq6?-|r&+qI1bOl8L=}kI{fDj?{2%!iFhzO`O z>Cz!IsR;xKf`as36B2r;2|Yjv?CAYG?{~l7yKjAF&wMj`X74`?GhxVeo!5D-bFE_? z$2!OuUSv$ovc>nZiQI19`L}gTI2i#M`v~vaQh;e%KA9`1UgO)qo&k|)^Ny@ZcwzEQ zD+3vsQX9c{mgT+>T>QQu4HK!V!pWiQnzJSfvRR$hHntpDb=@mFd9{SX1aOT0j#ez_ z>otM{YaDeR)z=h@`#)S-RVCQ^`c+63Jb#eHCwf$yTL=nG(xn695wwDg+TD0{C+#$c(|aZGe7aA*7FobAnyy$l`UzRLR$G0UQ% z;yY`ifqM6FmNqXV|M9hLI>)-6)I@r|Pwm9XQeb0HTRPIat-fPA9!4doB-_eVLn2H+ z<&Z^Y^?zl0>o*4BiFX)MYF<5#@CY2me{Eocag8cJy5vs^g>D-zGg|^3&*7a9_NXg8 zrFKM^M=&ptK{|mAE>NY1huJFRay%9&I94*KoX5X>1VxX&<3D{FR-}zjk zpPAvCBcQ!(QclK*vn&rwX(gYRT3ZOQ1%y7mmbIkF0+3*a-GeBt`<=7ZBHTLQ zHAV^gWuvnBJTp1bvU`FU0pA8=xr1`+$o!pMwxJS~0rj|HowSvyS=q-`SP%YNbxieA zE_7${xV))LfBk(ahw$+3fbEsb)fQ}l80ld5!~N*2{W2)i5o7D^oteW5{7Iet5W^(B;#7VLQ>1wmN>P<=ZH) zWaPd{Vpnxr_>xjuA2eOe&~wwUZRhmCMsJphr|7Jf$Jon#WmbM97kcQq$y7i~u>Nu4KRY^eG{8YlOn$F@Ig(p*Y;J=n_pV`+<)RG8AGDRNgG|56HshhdKzJsxwB%>H8jgt<>wsF&}~}A3Pu%`OusTNk+ZR%3@Cq~ z41WHJ*kM(zr^hBO+;cOgft0!g`BbS5=$2j4v^H2yFf*{U0OajS-(S8MX^6~W*@r7K zo*hcG7xs53vDZ#2aC8%nlR$vF-4H?W+JL(~YV#LHZ_C?93@>P{oVIj!;evJ@3g^t{^tM*2qE}zJ_ zFdd{gxLZUYQBu*jbCu-Pq4J>Nuuc@yj(hvG8?gzeHFWazroY_IHghSart9s zYHcAGl3X{)Km6dRae~QG%%C5EGTd%Eo0WCEmY8m`UoBwgd=OfvG~zK;eaSA}H2)*& zbx#x8KOlhTpP}WZ_ZQX+n;K@_(;(W`dhngB9%*U7w=rFjcaa0@BXVnj`TCs@AdF)7xB~HB(UERSb5i zu_)gwa|*_JWC!6(w%PBfmGgfBDd-h5=R?+Ggl>uGk7{Ws;vxzqI&D^hY3h5Wdxmqy zF4tL`crC==lR_lst=sm(^TK6@H#D`p{NiQDC`V#)x}~T6k{aLnsbPF_w6MLFM#zZc zTbVwEt*c|1G_&39Bvq-652c0Pi6<8~ceg^y3hd`o>H-I~L)B`Q;%NG`OcaBNQAGPe z2QTc<6UEHHyk|e~P&5eQlM+nx%%x3J)`@n4NpD@i8>%MJ} zfz-%g%#lSJbtWZ`y{!@;NBNkfyeccW^tYo|pYXP*_69JcV#?VgT!OSfU*-%#W{@}vY) zXPVI8I%yaZ4r?*@2tA@8#Da~J!I1uCt^K!Y134Vz)>yIE?pkxtvyzF}0RUt-Y#_1v zdM9v75IXrapT^gKJ$_i4<$4?7!P`m=l>JCC!scRX*<@^Pu}Y*gHeaBxf9G7U+*VDP zZe8t<83oZxm0YWI}9lm#%@nSo{{snFt=3-W?K#QM@G#*nVu=PbHjkCgboli!z*?? z)iP-+piXWtQ)-gJWgm@_pW7XOF)2)SagCJFI4<>OlcFLVy!+aGDTVap^=g-5a^;Ki z=KN2Huw;o``xkmM3(#t5r}`(LUB&)w|0EypXKfH8q16Ebo4<8h1rF~U?Q=Ef+s@|) zSWqq!Uc|MMyMr7?S2j-OAk5biCrR?L^=?^)5@FU%ITjw(wZ;o%~w$H+-&m`s&o1e`|Co6Nr?Y(dG5 z#F*=ij(Ks%gIbRS@p5HrGRdTT-i3~jTMA{q6dj>kTs>rpKv8KrEpOm!Gud79eY!j%1RK@g|P}J z`MaxD9e0N&$fJeKlc{+jT3(Gl5AX0<==7pyXsXkQmIk$Ko!>201DNFyKQl*OCl9M& zM*z^;aRNA~KYD8cO_vTEWPrt?=8M*n=$uz zxrx{)YqYA&wycvJF2XKT8od1|%}wFa&F}j*>InbQvxqWvQClo%%nzv9HMS$ah^-Rg zF>gcA>m3~XYR5z`uBMREz=vC1vW@-2C+a|vI=?|&T&b!2gk15SFNXUO@tXbR{1z+DPaSf240|s9DYCRjUAABREpb3P@ zkU{7qe~}YudvppIF+!pFc_P`#$a}1}I3KOpwIY83YF&>1TnDf|0In!>7z0&?hO%$~ z7<>e>yK?1PO7&E1x7oVj?t8zE%WY;Oky~5C3GoniNy2{KqgEg=#fYdl+<@i z@W8x3R}k3xt50B%4>rlg@0e5#l`<4)GhAhrci5CUikPT#^uijFL8ir>Qi3d797p9n zsc4&*$;JG1T`gz$KFAnkwTXWDQC6b<-7Z{f%B9&0q=H;IvP`^*G)WfhMP}J{zo1;0 zKHp}qyr0Ccb?tnDprFuWG@*e}xRUZt*T+Y_Eo8GW(nojO^pVnBbEC#b8qzCCV&W`| z!dg^yq*|D6$&b1MWnyK*m2v|klb^puALo5s?qyLxt{(6HaKVIuK^prm(0*+A zN=;FnYtEf~lOt`$RX=M<2UsV}GIL)tui0Nckru>keB4q@SEdzu5D@GEt}LWbu3LqX z+hIW}S`U>AG0Ow#mfpKdkYk_}*u))AOX%AQSxk}57j|V7XTf3C$PPXXs5Wx*;~M9Z z&)V0(nh}$N%}14LI!T2E=Opn28>>=Ruk8arJNSqS)c6@d&zFo7!DQlu(8b*iOKa|4 z3v!W`*VO)YL4H<0 zF(P#z0Z$`B{ofKa=5y=|YTDdSkai|brxhj)VGeL1RNC}_=D7Em^3D#&RjX*!plvZ`3q32pnyV zJ($Ivm>8E)eo&yOi(zA^&*M#UsJf*$VRSn;WB$~^iutk3oM+l|3v2IMGc6e6VK@io zE+`bfMD)f^EKL$iPd1j7BGgAiuKSlxqpgaZ#f}hirES}R4ZF&A8-Av6!noddYou&R z0HML}Hw2%cOo1u3Ah>CE!#2K}_LG9q*Sq%fYYUYSC2up4QIeFT465Eec`)Vp# zgJR`rZG`2tcOM0+@6B7R{dchZ`f z4-#sNs6$p6uiRQjbTwGA9lr|(DdvYi5+!`!7m{KxU)mUgDdo4~ixB9NX&ikLBVVZr zdf8_au3JdYd^jS2B0FIg8)J)Sq~yDON5ZSQ17q!zI~<}o%T7>Rav60h%H7rth5N6q z#@k=ns$pTP;vRY0!%<-4vH2GDEEi8ZS#%-V(JB8J+lKC?zuaxx3STx%sPzMyJqjaR^Qp zVpChI_E~zq?8EmsA0m}E8oKi_$s1K>DsNz5RBsczY0%8UGCg}rj_ajsc!j3SoM3z0 zyBG#~X%})~bpbNoB_uv5(Vv=S%$8v_pIqmcOdKHXPA>S~c@ktA=wH175$V!DNN7oK zH2JhMJxIqK4-Kr&TMVX^)UVFX^GNe-qp}DBPc3Vi+4elaJ(_KkC$4!WjyO+3eC$QG zY8L3^EQ|}+GsL6^KQrdxN2>snk#Yl04%Venh`my5dMe{pL%ALf`CP3~dvA(CFLJO9 zS%HHmzy=YGJqkBIPNhD^9DX?4F+7;W2^38V*Gkt`e%=W5NtWr?i3i#V*rx0QTorzZ zcaw=o{-piuTd5E3E(onjNM3+(J-f6>T&_wb5;C;GSO*99^H(*T1FP_9~ryS@(QL6q@}*q;Tp zbwXMKN$?)&6bVopAnsb%(Dx^bgWi?EMJk-vfva&Ac)+u5l97zUFZH&yr(^1@MeMb< zAa<9RVb3t<7fiEBl_9_a?|Cj%xZ@s5+zT}qw>e;xw&x#qW$TfdOQ0guz#}7V+Apvr z!(^vjxN~TGfd;Gh;e)#5P!$7|eIv^Vb{B zWmZp-0;{G*+@&VChxbkc9>sg;J3N=yT?tc_ufA@Zg8j1iJdy!KCQEV5ff-yIOv}9x z`qx%Zu0;vguZ1HY+4=f81@V6arDY+@!c@UkLbj2~T_bmNazP=qR#tW@-b7=fFpBah zMGArR9g{V4fDAFKB&QBS%EpS<1oe7ax_ilc`Y8pWH}_Tk_JnXG2csJdlzzik%aJWI zjZ16Kxlr*2f(+dEa!e&#E%KetMS9ocK_ecfr8k=oL|TYf)<05pjs z0Ty45pPpY2t z{*2O(#}#zeD!E@|3H6?|H;CNl zKfZZI(bdg|{|zb45%m3#?a4%R-FVGp@6dd#R=0^(-~rfc$)R(&$lih^RZ|ApL-xsF z_j8GD?=E}xp{=1nC-@S)x}>{!ki*7W>8KPuUbMOfh#)3)Y#=8Uv9+WNQ#VGBlkt-< za}LnLbE?!M1#)--NImRz)QDh(ju6=3j6wExxcE5WJ#H03#1 zHw76*o~Ut7>F9cQDlL9K1okHBts-B*@L2|nOwzodr9I!`tnR2ME`bGbtL=u{7mr!# zVYL*g$;vc99ie|FEM+xj-?M!Q__7$B?<2iX>L<`Tsr*|6CXMMQ2ae2Jpt}W39xGk0 zR4WSG=^ts7Yx?r)s@7srRzu9}Prfn8`15oVr?b3n**=Zn+sL{kAz+HA)Gb+ijYW8P zHVgY|Qc5FUwd*xLoR-$eOfOTCHQ7%6aUP4hhYR}v!GllwH*0i)LWbwgRDh&esqOz^ z&A@+3m-~MrcW9tRIVT!gdZ+~iF-!4Ye|%(Rk@#oYB6($FUIEDrXJgB%fJ}zJBnhZa zw{<=;%LtXdS0&OT+*)eV-s$!lXLT@0|HqaWjO{1O6*x@&9bb@{E4)u;ouwBQ0I6R} zTNRLW<1_D*iFWHseTZ=trQ#B1% zKn$c^`O%Zu9)R`TTs2I8Q^OBRTrF8v5gsb?n-KOFD%qsSLk3a6ZS!(QX^mOlEBDi! zUyQsd&osNlI_xh3TA^7JDI6J-zXHQP_Ya7eQ4w%MI2`FO3jFb6`&66qDL??{S`Ek} zQ}_zY_MYGOM;TK`i-(g=6a)~?V#7^{nq>}lnlrymdSI#tmSg%bsQc>tW+!US_B1srTyGD+0x7Q$HwZE z71Mfulckf?Fb@C8i7;hKxAW{_;7+D#fHzGJ$!_*238eCxEiS13)%J$3IW+Zf1pof<`vK>8+bh0e7$~WG~tO+Cg2{F^(Pb|G_;n z|1HmQ_*b4)DupU(tAfrMc*%cbSCYPF?Dk}~PCAYd1lQt^looQHrJ7Hi6M9Wca4G#A z9QUuA#_^uSoIzRg4D?S37446fPSdip+64F#Hp%a^tPs+Jowe$^OW#QK)_pWCO}XUH zO=kFA@3TqT`;0nd(G*)QkDTap{qq5pv}?5gW1 z?av1Lt^Zdx*ni)^|3A$Q_#a`q9&6)Wwd_74>X_TiuQ6HR;->!Nrtf{x{NQLno%t6@Rz&!5>3K(_iQ(JI0uUWFDdIkIsA( z+D^OmZ29j>q-~3aNuSdc1Z{OR_dhqe{?&MI6e6q6;6vn24CdxAF3o7wm_t-_cs$`q z7&&Quo&p4=MEvniPhwcs$;K_!+PQ&cMon{tv*qB~5Hu_6#7;yy& z$3YQ)$!8Ql6TScQ|0zQO8vWs2Rny^gR1JVpv{8M^KsS(;s~*`be~e|~ag^lejIcaK z=0?M>%l-`=nmz0vi68&|z)zK;kvvbp%&Y$( zRPL5OGvB+x_4LJ4Yaot5vM-RPoS|8EME@4c`MydX)vd~ICnFE@ltZ-`lRMTybtPu) z8$}lymYf@II~?}in=1X&foAjLR}u$de2dXj)f{zW%k3NYVA*j7Pbl^sm9FI=&%=<~ z$*!_mkx-n8#<$5-81felBV7LM#p3atVRet8hEd}+i^hjYr&HAd1gq~flnlDE((XN)3aC0fB^5(!_@(fL z$%?~4&Oz}DvWg>v-8OjEc=BZ+;R9SG2)Ldh!V@XGw2p;TF~iMNG=d9V7*kbgg=f7BJC?0zY7jm!QPm3=&j4QEkB znw9ZMMg@FSLT=v`SX_jMoXAv-!a76qnpHznt&(iXghjI2&L!!ZLS&M)zN2?u1?`&A zOut$AMAlo?lV+9N_rLmU^`eOh^{A%?AIL_lH6?oOyVu9X?25$n{g%~uT_#GZJyLU5 zqnEci465wEUY=iF*3U(|Hc@UQy|qM`i-E+z_z^B-yW(3N;a{Z%ESoU6R`DO?QS}%9 zi5ak2Jg?Lj0IeX;cFa{B_m#`m)f@#5&#X&3o5z>V@cB>>e3@JEFOsw4e5S9I*vHT3 zICW0vrY{W(SJ=i`J?}p#VY^UzrNk`WE|}=btczthr)xb}nrJn|R0Jnj2F46+j!G}X z382Z5Eq{E|+6=(%GDOEm5l66RB!N@sM;BFOm8m{%cGV72GB37NGSk4O^fh=i)@bZDs6?5OA z%>tKC&>|3KEr@8w43nun1se6-T-of5Kj*GKep9EYwYn^G+fJmtZfZHdyLdjmkR$Jn zvhKwa^p^qV(oxnMCQe_U8r;vAD#*gG^5XyWl9O4K20u^u9&T|!&GG*#Qg)K{k6MWh zG$^`+wxX+FEty<=XOf*J6>|^UgR3<(fhkxcg9#=RxvvV!>!?cu!)mr z3@ssSSrt-@%8o)N*J*L#3th#?Zrc9a!CHR`3uG*S@9#(V*UF2_?eDE3h-V34HW+rV z#6Z2YG8k?iN{pk6rrqsd*^1}DzDp@#88?f=(GCF%nsQ*5v7hD5ByLw(Z^MGb{`q*{uHIB|tG%FoDGCqOR)fRT-x zt)nCcD-4l#fdo!;&)>qJ&IhN+xb*7IE*eC$;g|g9^#Hm3UmBt2hKKMZccoUv`NIqG zjqGK`6?aO;>rKn~;*CISM1fr6uv?ZO2=rQ8`GSD{xUPU`~2hF0Y?NsFBkCqc!z~+r$QkPv9~)yz%zH zSu=!t<19NaioaV9=P29n3-`-yh0ARIS-=d_{8=`Gra}KLf10!=AJtrI6yY{;xALe|jgxKfltU!ku!W-+X0GgQE6{unLzQy)8EU)+>@_%<~Q# zG+xj*&2&rgYwZHK>@;EZbjs3x!{(WiTNuMECph0_DLWK~685T1(v2bC;2t0CuoO@aTv9Qc1^5UP)U%7KlGM9mPo;c8CJuidWT z=4}{4n;MRE3zPZOgoUm*X;O(Lo3eQpYz8-DA1tYvRHrNCv~(L>U}aHQjI=%BfU zlV!tSki$>7yuVlTC^ue=n-roc>i*gVhiVW2A-0DSsiJ>BL;HvMcm2;`JD0Gww~P0BF0R zM9yTR$lK9FolLxbk4Zzt7FUTxRwESV!@l~B_SCTVntbmK%>PRq=Mvb2+A)K!HI!(F zJMcm#-I~P}H~TZT7tdO_;Oe9~(v&p2d)J0g4h|@b!Mp7K(#O=0V@rrVJ9vcYGecIp zqSr%YEsxx;eYQ%5PG1PPFr0$mwUo=5%a515klRnZrLlkeK8BLZ*PZsMHTV|&<0b&X}{LlINqifRE;@TO)3 zi#xJ01_y=~HXULNY>x2b=+R?E^4J{p$;|qX4awffvNhbP-`3DU$Y0P4@XN2+HF4Ki zkM1RWC?&B9Z(~p8?@YzDJ1Kb>XNT+lN|KX`DhzF1q~zQR4PlyEMU$DM|K`b-LX0C5 zO0Bc)+?djvh|e5LB`O}VQ8|5m``t<2Ro`u8)G=WFs{T##oCR`1Dj6gxbyHlWbL0k& zbI!CVKJ7!H3Ue#98oyIIt&Dn|%^O^jwhb62NjK4YTYVyAj$ot4EZtnnD=EihnF1bDbc zZ%t%C5%KU!WzzAjAmV^&of%8v^b@1y+uW1TDJsPpZX-tduwuPoM4)Bj)?3T`GlQfQ ztAXv;m(v?9Z^O)wyZnYd@H|#ynfP3}v>v<_WUx%*NPR(?4wAFs!2+|5y~#G$!cgOJ zuoq8nNhVB{G4(ndk5#%|0UV2;feN=pdR~)MU0Z1EvMz7^1PBD}^6WB3n;>hikw(iH zoL0pgRx}REx+Tsp9A%6b;~OsIi$z~tn!}@Y!SUJUm*KaxCe}3-ugsN6OCE9YHM-y) z94vplyuIzE4p6oyVyXHInm*yK^k%)U7O1u-OITAaORH=vT zaN^LWBTu+Z&Jk$C)Rr_F77aiPYTB)koC2CJKPPu#=H4DoYJB0xFQ95=&8v#Kju3Br zOtY#3y5+vt=7o`4#Urh6Whd&r8?k;%ru@||D;=VNcz?1xIw0Sr+d_5?c8!zhjKJ`NgB>JaigWGwt&Ba!Cz z#Ci_7jBQZvZzlRJ3XNeR6$GLgM@EqCzij|hxmI4 zte^S~K<{$@pP;#p>I9-4$r+fW#B~{xA$qCEKd_WtE(o4Vg>4C3ahdm z7Gi#*8Is{N0Xc|e9WsO)EV1=I2ef+KRA}ey26E0FTC_Kn)uOkjQA^tK8#EsdeV5QeRz3Mc7+%pm5Hp?a}>u z>q0>t+JIx73Wt{NHqgpquXTJN4zttnW6<}w(C$FUIXvAD&3Oa4{Wr}z+}9%iya{AB ziGxgKc_9m_KmPRj!O<*Jnr3^SEAvAg`iUZs%hIONb}LViZY0~0Uac{!zQug za86ndTb1c+4={(IOX1^p>(DQCxei@()egPG^SV=V0Qp6p(*+sn;+cB_9$(2K#!pQo z#YuuiDW~R{zFOiSXDPQg@FSv@QXSB?yee_u5pioZ6mLx|DvDVvlGa&KW;hXK0+eQCOHf zcYr1k{MW!YsNp+mftm}`EJ*c|cRB`It4{M}R#y>p;T(+9w~}&scPA1P#3glyte=k@ z_o7J;jJ(kMGl3>N1t+B*ytrzn>+$a5I}MKVC=K&VE_eCwAkU5dcsim2Z*1!FB4_!R zRoJczO4K%K)S|3hjr*#@s6cwoJkRKJx-w~!bDkRX(e}G$DH@+ryZ%9yo;rAowA#?C zbNg`pO++&oM8!yKtIQxx5c`zT#4UwZ$o0AEZ&^9lPs-JLmNi|K@-$Z2NQDGk=MQU- zlsx}V<^oUA(3_985152_dQzM~JWIOt(HvU3559c)#K0p|81RiSNj3(T2O&1Z2R5~@ z!gDCUPP8-Kib(Y3TTZ$59MXMHzx>{K-SZ0??{R~epqe9_>uRSns|DDXynN>p=2eeW zu4!3me?+_=Ah_=UhtKn4f)A3BZ- za-QzzsH#0tl3qM;%gW^{kN6<&OJ-(sna*FZ`27r6m?nj!kgE(^O8-l>j){^;j4kq9 zqj$tXit&aLHCy0?sLxM$tYSONGrW)+{@{K1S^*Kr05T|{?-w=Iw)2p!GSj0$lLyU4 z)Lw*{ii{6cO7cq3TI6KK^8Zww)}Tld3+JBV@Rd5(K9}M;t-Q)!g&MnvTwso13pHP`7s^diXM2lZ1~ZLOoz^lQ+azw=?-= zIjj}HlseZ-zLp4|t|x|gO1WBdR5H*tShmPlH(Ods>$3ahtfk~z-vvA+&4qo(%SS;o z9(vOJaAw)Mi#M_zhq-LPK2%-ed zVY`;)uwf8YVo%^}vAH}3>wOg;9<@=vDT!G}^}arvI_6CEbzaC_(0^ua7Cw{ zCi{a(oMWE?aP6x)LP7e0*{%2#658ko<}S8p@(CvH$Sw14W0?QvuTwbMu7ak?yRxqa z5x{TZ8T35QzzW-=X1d^I3H$vUEpiRoLQ`c~q?V%fUAk`*ZN@x5tUF}V5{S+Wgb#N= z(U%q*$AiYg;u5)IBtbcaEtkUY-&5shNo@_|ds!Id_xZ|uj<+9##wZJejS@;{&tQ^l z87HdZnU6hL^J>lW!_3u0ToJ=y7JpUTMEa@87P&4rXnRf16+iUEwc&bqI$+^S!%a;U zJ^Wsv=at}k#y~S7rBdP=9r}GAN=d?p>nL@T@&KJM{Ucub*^dpeo?(seb(9MdAfK}H zF!RmwdH$HsTG0W{z@4_F)EWL?G1f-KGVjEb9eBT33vG>w zagB;_atW!yA88Y7?W>2`lq1ey>BYo+>qj5g&$vKU6nQUMFk(5}e8A z=nz?w9Z_k;Te71C^!a;kv!eNeVN7{Xck$Ukg?JX+xc+o$53|VKvg|CYu(;lj-1B6Y zBvm042%qQ!ZxeQdp3V^PQw?*4<}Zp8^!Mk*;^D6Ixx#xBGb<86!k`5E;nvH?BS=UN ztNMG+Yg@I#k!G0zzR#u_DFTmhYV=%JIAFpoxUJOSc{;H`R?7#0Ps`{{CJYzYly^dI z_>sCRAJ69(f3W-LKkDsG7&M#aG~gvaaTCyDqAsyYYG_vu()IP|3}<{Mo-|xCp-Fve zgcgLo2-J|;i=SX!-_TUt6w{=D;p=qn74<0c@930e})7okjQl#}^8X zUZ$8cHS$az%|!WHlo-Oe;-**TuR06SK6vg7XnTJ-RKb!vIRjZ-;(kzQsj4W%yJ$>Z z?>Zs9+ZI90V_!L{d)9It@`D3g^SN^+6*MHXLZ?6QzP5;iL(M0PLgwnNA3IbMI|_fWgL>6QA<5&)1v}A_qP8b$9&rK@ zDsf@ct}3J9O$xNWoZE%kxQBCE=BjE3qLf~ci&wQHhJhakZJHdgyED|IaL*yp2Kbt@ zwN4=?v1SbGia-l_ROM&faj~PSIRIy_J-k@K;CZ{o9P5hVCC7_)c3icrV?lyOZ~JN7 z#Y)rDg{-^LLP>0mRQvIvW1rut;wmpSqWM;1*AXIeY<6It>;0uIJ=^+)SG${T(%{rq zVqhlBhmg}K2?t^@#Hsl4v~V`7-|Nnu{M9W-YxxZdrH3Rb&N%J`nHS4nxt=}Wn-&Q@ z6E6;G?6N(o>9WOHzNGxhBz&E}^jml#B2L4J233?Mj53+3qQo65s!b`??>QE~Tr*V| z8+=_^Zn`(|L)Sbk=D4NxG3vlEqAR@dbfq8Z%d;REzGdQGqDxAPWtdeN_^j}ul1f{w z&es|Kmg!?Y8rAa>pzCa(i->1AEtjbb?1Yq#<}W9OTNSx-)w|Q;Grlw_yL0%>^ZDIq z!CCK*-v_fCaZuqFKQK`>hWDD_@AhQ|e!I4H+4Jz~W!0HRbyP#K7~8w~yt7f(+=TlJ ziI{6*E%ZWLm7}vGb=Qk)8ZX_npAy9$Wv3!HJ-LH3qMB~Z%h?*_4g{v9G?DJ_#f@_7 z7V0N*pXdAt4vsh|MO|1rCV$-s#6H{bV+`aaCm~N~-SXor1uF{o%LZ1BfF}|xO;_vW z&4zrsn4fWaASH~&OS(NcBj4JBnclbz+3~i!Zbu_d@r87KNFOZeV|iUhxef zxACHPs|7_jg)1@%8^NMi@6fH=Q6n`^f$4;qG82{0TPZzWErP`$9u3LNFP4{Tj^}mv zbKdZPVXuu9ostD_+QiACs%b%VW`LExGRBtE4a)V;$S@37o@qYnaW1>M~5T z)1V*P(*iHgA3BqE@bMMT8!qT9>fvv_51Mi)?H^5{`}+7!!YkX^DT|5HjoG6Mt{`-9 z(AIev)Hah6H$8aO)ugohnZQT!Vr}>%OY!Xnjy^k+ECID(X-!th07yl`+nI(T#47l9 z8GX{MqBc!jTvOqRrSDqHyIAb2R?=#7(BI-1- zxclKY{$g_VIa!SsMFm_ws5KHgd`rwBGhZd_98~1|E9CVF&Tm5-MB=JC1djd%l{MV16D}Mkq~`Ma6T^ zb9j6ZZmB^giYPx=4@!Iaqfv$Ex^s{BU7?sG;&8|~UdbCn;_tfZ90cEr$YNTzXEGKt z+D;IDu6cA)N#1s{(S@$jd%?M@qDS7Ps1Bs$oBqj9*gm&TBh7;}Dv@&toEffNtbPR_ z;jQvk@}9$*E9S4a;njb*AiuaUaw=}q53N*C;cA>a5Ip4UqdqutX>&8g-}LMx+wk6k z{#pC_BMZfiw~ytc(?l{!>ujQx#8aT9)IE#5s{-QiB#)Ka%p~YIlZCk{KyEQ2O=zF{8TW#V%|6h0mWE$fR1e9E1JJamP%m{`J-a=F}t3@dGy*lp5hlu@n>oT@i>m=7!i+ z1y!`VzlOn`ZPVVZ{N5wxldg>#6^_~Ik$X5%+BIWtCu*X5AbL{Iu`mCbO=T&U5f!x! zeJt&ER>|#a9t~WVyk6PfSGSc^ad2v~JsXYX`Y^p5|D-+NsXVn6$;RB~xR(DoaF<`= z*`7)HcBb-z&g)GZeloy!PT@{ia0@q^v6fn-Z)dJz7KMs5BQ(Prgh zB~7(ac)!!FEI5YiW_$Y^q+iVIL4!V-NCgA5-)BBQdf2Joy&z4@WKS5*g6{gJ$y3)k z|0A0ln`An&&DPXuNms&&sPKOq)cj?D^A85QpX`4vUnY4C@62-l!yo;3ADp}-vKttq zivamY-^(+PljWdEagfQ4T1ceRfzo-x^jUrIWV z>D;*2nYmm~K4h-%^mXlvdtlY8wyCd4+-maK`II3a!)CK;BF%VB{00SgWJ4QjbnC2r zRmSQooppOkt?&1J(H{im2T(rjS`=6w5nrmXC+0t9kS;`H3{^eGlaulOx2X?Q*^G$s z(Qs~4Mc`M8mr-bi>Er;}fJ^AduqhYRAdOBSD`+vKypiU(L5`xxxGo4D>0m$j$Ca~A zZ=xr6aMxX0Ru=1)=;gVDrJe6C%8-XLj2&L-*%s)}v`yO4Ju`zqOq4etIOb8|5_($R zOc;b#v}T`43bv@)a#h%k<0E4QGuJ8n=;(>w*UAb_hII;WhkY{waWg@ycoc-rmfMlD zYvvAnZb|MwQaedCp>AYPJ!1 z>b8rQtkqkGt<(D_EDmoOAL4uzB$lwVv=iQ>6B(wA#}k90W9xqO3m9)keE+E%`5Qg0 z_YMgk?K7W+?)@DoL}ntONx0E$^v^E?_-I zvhuy9JiU(D7%j`%!rKwwR;`DoksHFmm5)G++Tig3&1uOc>h?k(&5U<1Pu|$@ut*#n zp;2bowfyR^-lB?wY>8mk!=rSjM=Bqz-y2A#LC!Gzam5JvKex;SOe&pzKYHX;G4km8 zw#P}^Exkh5QSRjjW+#*n&a@>HZ@S8)XlGS@K5>@AEElG9?bJSTX6n+=WUzM&(*-QY zE4^AnOYsViEhgr=OBbbF#ne1s+%j9f(nrVPJ)dw@YJdp=x&qO)kvHaw`glnH`t)N1 zT@U%U74EwH*pblhPlcCtM^l1YLo|NcXAuYc>ttDzK9~U?gz^gax1xqhmyI zj@m(Xdx&Z7G>WP89zD|+qj~oNyV(hN5^VTIc{VTDk}O8K?&=Uqd?q#JxN^fQNmiiE55KdC;DehE)3h%VecIz|QN71UQdNO7m-GV?2&#`+kNk_P?9#Le zq$4^MT=zmFvlr8N0gCN(*^%aLJ>b2^?i4R-T>--`Ib7n9yS^4*lje9p5u4tglccm}9nv)At>X8|G4sd4i#B3ahJwhTkG9lI%K$}a0BgtQ z?5i5phr-X_TUbyK+(lf=qGFmeztOT+8`~7I%!WU8U40&G`(ytm1xPdB5Q7&~`-WD@ zxksH1&82m8-@I`7ijvz$)fP~a9IrV3j+yVx-8#cQbKQ(9p}`DSbHAKtlH+mTo%?#p zJb0l?_i>lyB97zrhAZdh^0q+Xqt`CFNn(m;ob1K?wc`^h<*t2m2HvW#Gk+hO5RQ(M zVAmQ7%YZg?U3F8Gpxr9JI~v0*LVBY`uc~^;5a;sa!B(U5@}PrDNptk}sUNHa=g4?g z7TP|z2!naZsh|L!x3Lv{PuEgr-Mm;~fziv@j26K{9A`t1G%qP|4Pak4MUj5!2YO7LiF^7>=c8NBl|SG2d4ywdy76z;VF ze!abI~M!E#UCpI!*$*2bfEyUV{pg>J5B=QQ{M$jFL*r|)G)@c+91Pyk(cG>`NIx9 zlCyrb{|UH8G`H#K^_G5C&B3TQ2LBg#Zxs;dwrq_ixD!ZlhY;KeF2OB8aJS&@G_FB{ z6Ervhf@^@rH8{bY#@!lkq>;a~_C9{jTI-(w<=*=MZ*()iIcI$}MvWRZ<;U_Y?~%D__DMuLTG1H`~~);_>Iagfrrc=s)pt~9*txZL;=yxXgy{5qOmk$^T9 zr?KJ}IU1GM+joQ1_v~fx=I0yY5mi^Z&>!1Up@cr6txdxSrof~*pXaGqMfr{PRfkb% zs)eGHTKn!NMg@#d=oL?+qT#2{QVvsl(t(1GF1`&|sj)`t>xVl8<5xpd+D|Qtd`wjL ztvC{6f>@Lg%zUchdJHu;M|AG<)!We+21cKAul+;JD=}i>D{>ql2A#c; zsi)SxOR@Hc8*A71q913{3)8l>4|fs~my6O*Q%9IFi~x2J{{{pxXT6%A%crNY%T5Ym zc~1&zh|RT(nX{E4%7Yi4+%A|~_^y9W{ll`hG_l_$DFA5hdFG=ZjmW(jA=T1mjc!;H zMbrJ&GE(LNai^)unYQ4!q2EC#sB%_!OBN9*aL}w@E+Rk@I@W8|6SvfDhuZtnj`l=} zvZ1q091igIWdqWs4IkNjg{CuhnAAHq>-<-q!yB|+yf1JQqpb|05Ww}-6ij+?C)hHv z&aIsWJDxCGl2rbcWr#|*S;;##atw*OOy3WJO__S56u2R6N)o`bW#W_Y3P0FXos0Y3 zHzhk13qTHdfcff%n55+Jhl|-!2EW8nDt9rt)_ZPt zqqbJTT0n(^fS;ba&JOALxqT@FU{vl@`HIr zZQ5}3OwR@ul&khvAI?5orfb2K;xN@}T#|6zby_5Dgl^5|4hSH(Jm{ZyX?A6?n6&d$ zHMyrjx^RMg^l7nl%MO+0DRa!Rx9lsvRx2}(x1kyJx(@L{4nEos(-Hg{(%%5oYWEz= zmT)UqYo*6wl`V%Jf@c{FRC;bd2269GgzyMH>|rg!0a}hD40NNq6C6NM-Hje*c+Zk2 zQ(VIBDw{Tlt^MT0mw`PyHns!0L{&_Z7p@r22K5^ z`q6cNq%^>uqOWZtg@f*{_uODScr}y0F{RV+YmXnHu6C8tGyTeYj`wU8L63GK#NzCT(*10!vG3UG*M8Cp<5Gu|uUB5gpJ&apxF+^% zmv!k4ZzdRGnx!V2*ygIIUjy&;8FScBj!|XSPoEZM9sGaL5v&4=?SDV~a65W97XO9X5 z1N!Gxu%FKFKSYC43uR_~N)G$^i9`IfoZuDA_q^8fwc4YicHrmFM~E>X1Kw=*51#$D zOIuR0ob^7p_iCanK&RfEFM*xKgyxiwC{=c#cb5G z|C@}1A`vxwrm1DW=C1M|c$kLmS+uqrt^UG*!l1Ua3ZJ$^o(>gp=nctPZ)${Bzn;LvZ8wJet2-NB%Tvsu0tB0J9l1V7X&%{ zBw9H1pxpBD?AGfr+c{J_g96@*Y=;LQJvur0-Fw_ie@F|*{w8u&=DU7WM}!m5>qJV> zqsy@*W^)^G<4jS(@nY?;IP!@KYhd$>$rIMH#Y<4uUv6Xei!`F3mSk^F<7XDPjwnC! z^tPBp4N{-~AcC(YbM>vUn%FbhvG-X81hrIb;-iniLb74IE@Fdy+f)aB$<3_CnLEOe z(o+8eeSdoLPNCn4%571m9*?)&!cDQFeEw{EOp_nzWf^AiF;ikcR9D#{0cxE!yL>#+ zE{_ULtxGFpqo&cgaHv|@Gz3jNoZpnk!aG8)2CwD9QjDrQ^T(TwVkuJSKCTCsxI15- zlHZ$&Fwjt#H4*i$lOHj)Nd5)xp=YoB4Sf2O8)IUw?brNC<~*NHY@{`Au=wWM;#$9d z>9hDWnwtsOy1KZolJ4ZegezO>AP&e*{)^}KzVJ6=#ZA=c69~a0{Rln?em^F%!-?nm zHEw~A_k;2K=TV%wA<=MMK;gdE8%TfgqIJqa`OUIPbNv5;W&6F9FwXz=Qf}lCM)nSf zCEs;!RBatifeY}@jzT@}n%lUen$X+EsXFJDjy)<1{;R;`U%0_Pj>MFru?4Kn!9t%a zAk1W>_X}+l`_l3<>CLO<94uo9NJ_ibs#_cU+w*yCHSW_0gsim0`}g&bz)~V^=QfX~oH4Hb9a* z9+7ZRq@jkF4nHsmQ$HHHkWsu4L5Su{=}`R-tNAYwD`=mdT>|d$9e>%VgUDAn*#5V;IA{iaJrr=$f`XB_^ER_xMy93UXek<^<%_wEqF74#i~L$ z7~E;~^HVHNneHMEd2-l%t?2-UGw7n~@f+R5wX=ZrzD;l5qT%ZT=~yCqy~SaiWX+i0 zHzsyew%^+CJcBta9H+hr%{)d1hx4;*+QBtdNv@o5kVqjM@=L840Ua${;q+$s1j5aA zE}MNxn`!br1w*MF?XlkC9|D>nVW42kZY;U?sr5iaBtbyv4~wOm_W6en!_Ljw>4%{u zJbD3_*=A=}>Zg6L`>Yjy4`Ao-dVh{D_sNol?azIqmz+I`Yl%i7eoKT+f?cOHe@+}v zsT=pf#xiBBNma=85GyJh-8N4goUj@7^QEzKO;XkTEzF?&uC)yrF+Zr#^s=U}_MD_}wF4T`hdT zY3_OwQStl9=H;Av)MxG5g?1iOR-;$tC8q&9e5bbWApYEn>!f8g$FovA0YA<}JcT~5jO>vUGM7DBKb0@fd zXLv6IiLxR)mD&OayemF=5nVw+y?646HJse}Fch%<^_yv0iv{wd<-H4`{}oeRIz{)> zbw_qTZ$%KSg6`^r=TSf=Jy?@J*|(S2pmi+07{pF$tJLN*wzN0pg8Apq@)hioTEM0m zE<(gFY^@HT!$2_ZT@$<~w0_RneU|g*F^a3`bgSCSYtuFVw&lRgIw7|K_fNfGxP#J( zmN|Sk!|=S;SA{c*oKLm2do@hF@Pi3dklY_rA7aae_@#F&W0mAO*UpqAcPvaZ+_m>v zC2-D_D#brq5xAh0&S-ut9D3`Tffe?!Z?&8mH6mNFG4mRV{}1ZW+KAC2YEmT*bDW4| zoU*I>AABGk$7Ks?izsHHG=J7aF~p#1u4kSkA~M}?c1>%6j!Yxf_r+e7gqeI;+gNql zhl2FWp>-F78W%eQ_~>RUb#;+&4VyH29*x2QLF0f<~Q(@U*xHP5|vuubR{-RUTXG2 zT zV<X9EMD?kMFh_&Q{RW{psXZEfj$|# zk;+OKO0WgfI}A1im(FJw#SwWRvpU3<{)R9te|4KDDf6~Xx5<|tsXz9O zEMht0Rf!6H{1Nwmq9w;wAT(NOE${cpj(XQG-KIm$^Eo*PLqY+i8!}z9EeRzSqBr0#Ng+>s*%sm=V*g4?`Rk`u!Y zR@!t|X>XR>XfXS#+lohG-#IQ(73_Xi%fm2f*9mI?Hc&7qeJ)MZT(J8=SapKkY812k z_%J;%tPzky(oBLVND)-hLh*q6?fZ7tI}bV3V0V{w!yuH}@I{L>ffib|Dsk^FS|ad6 zu3sbIm|XH3^Qw^_PEUX%D&*jxutLoJ(cZwa0t3e=I-)I9_A{2Wqe8M>nRFz^U=F>+ zc@8t0jHUs*3zG|u(0o{39UN+4$Z?R5r9C6$6~NTd1oc};+qf!i`#Cu*PK79GOn~75 zQ6pI^%YrDI>q1R*-bw0j#9QhpucVN-%|rf0(2ek=7bfUdfb&j`v^{h>e{4F36E9Gs z7g`tAGNdP5QbYyy`x&0BWiy=s-fvkzSuK*(KbCb6u)1f)r&@Jy`QRh4umj`pPR@@$ zfk!18_bKHuv|d~e2&IWkLN5HcNuMr710uq=t5(X`?$Ogl1K`*QUawUcS*#b!S#O{h zk1dG`fv2+xqj|-{?uF%IqAF_Q#qxVZsH$9=*xB<`!Oh)#4-BOjq%<9^%P|KVQyh-_9&imAc6=yS9P6U3hZFgLPKeAk?#ZrH;Rzf#U00dRwOy;s>S^lM zvGs5(E)8vSwh}6>E3y}TwO=ghP!%bYPPO1KVtd_pD}(+FyI*~1d`oG)bQlL0ZU{8V zN|j3-i%$u>r7V~xmJ7!n_vM!E{^{i2_>f+$;F9GP&x`p1r%2){w>5fPGddG~*`~3q zqX47A-U+R?-j)F}ByH-2(OR)?8!hZfYM+PHh-(;o0F765XLEeZhq+y0F#3Z0)F5^n z+xEi7_WI*}9%%V^%FC0tHl9fh?O97NZDlQyv*DHMS6}VQB8+8zPGvw8Ruk>5sF0Qj4wT8YVLWP{}whD8fAGx}0i%|7= zGrr}*3B*0O`?JOn`Z}-iJq~44Yn^>kgMvY}%+fHHGfi1JUE}Vk8grwo8aP!oS#;;Z zo9l+eI*Yi^kV%vb?>qrcQ)20>~r1s&fo0}N|r_*V=PMVBU z<);@h$Rhse$XGm^QS||F-NJGC0#N?Ubh_uhZfyQnPto&jq9nV91HC!z3~Zj!7@*`- zv&_dgjUX}(*<=Epc&Y%Sjk*b3{e}yIzrYp$2$A4MM|i#msI>3RAiH=&ziB;&!NcOA zB|Eq13(?3&5_@1rd(>+7LAOMl)Ed;*d<9Un4WnR_*%$Dgm0 z3mc}3Xqc&q1=vbh9jt;=i3gBvo|{G^Y@kx52fGhg&B2-h`Z-W1W)kM>g|~P2K4@TC z7IrW5I{~u$m)u+0@|;DYMf!`gWDxV@=aSH_;mj1a0l77#D*RD}$~9K1@TX`=q3*J4 zhUMuuvU}yMhK3&=2@+cgMCc}KWTf2z{)6xQbk>||)jOklid`2CF{&}QtLpePS4I10 z%KR4jW$rV%l#!R=SUv+N&|1jX94Ov8QH-Rl&pxQ&wM(q@SxHh8n~8YRwz5gcc)yB7 z?s-v>$^yRitNv67|`F3^s1qX{bT^DBEmlSDZ3ulk%x2)9|=QozM`vq1N2m3b( zRRLVaD*VXjeoclh29v(B-MlYDS}+fdPzjrQY9-Ku)}8U|=R7mc2AlZ$ULxKfka2n3 z+fqG3Ozh*H7=4lYMX;m{*H?Q!(BpwAtPkhz{m50jE7xunUmT3@z)lQIP|))dj{t+xWq2Nken4F*oNjJCeSc}WWw$m z*ka$R$0Q!O9{P^aT@O2LN$d6A?~S{c=g!P->;=f=m`;m8UW~s@(tx|hFfYCW_u%rt z$Nb8NzXqjSd=2qak75U=n96uZeHuYNzUVFMiCJO{s8}z{AG&L9-8+@&kHWG^1VbNh zbDrSG$n41cRCO*o7ng0MYHL+m2(G@Q4(}l<&5VvBH>1ar1+}z(pptmJPHjX2OT*kN zd|f8=U+sgEAPdA8G^cSqD6)LQ0>j=n5voKY zH5qTr+F8|1UcJtnE>?6tO??{Z>zaK|k~7?43nQNmuFDe3KF^wD)N zT({AmHv^@O{#S7_!qtBiCnK?PcZ4-$Q{4zRpyr9WIzg62bIYFaHSM4{unDhJ+bsXU zs%{Msr^G z+;ww+ACx67lj<1#R{eN6bm)uOjayD9`Pck5UOxE#p=6tz5_C|MJ7=o5G(tf!vB>kN zdiL3meeOh)AcZh)W%(hox{;STgyrDpMpX(np)I)W2eeM9D&w&7cL=J0*V$Tl@$BIE zo;ia}y+;55Ieii7Oud`OQ(>Hh&7?pnERzk?f)_%DInQwMGkhV$;w2|-@5yp<+?OrP zJrR{iVJ}P%t(ZZ4N|-}!R6f+`h)s0BU5b8t>kpYN?Fmd@P4cTPA5Bh&WuS`rC(_j< zL4J!;SdtbK2%a`AwmVm0&Hu$C+0Zb;OvGri<(v0x<_8aZR@U9dr)u+W@*yM;xeqI^ zV!ouesaiTnu8g`7QBi4!<+Ew+es*Mu3nz^|s^zr9Ok%u!bg1L5r@ARYNF{)`6Z7+z zYPj(hBcSXl&|}lUggqbn@{vG8BAs_Y!lA7|`u+}MHOv^-w*XZPP_8bgtzie`c$#@DyU+D4ft#8gmeGkr} zw@OUan%*Y@iPAvoB~CU&18xCnA>C(F%bW$RgMmim+-p&9ISYK{3Bw-Gb8b>`;dd(i z&=y)QJvX23NmgzLw8ByXp2yj85f&@z2yC}dO9;MdUyS~xsx9iGVz@J}MjWPu1Yi%4ih0&jWs-YT)iI+80n6 zxQWk(0+*`RZ29!XpPOturg^q{LxZ?7$j^}MHaZ?W)>a4!lg*GV$|d>Jg$*zf8_*@RbD=l%Yf#`LnMMi4!M-We zUe>Gdxoy3d(_q;EM!j^$Y^T9jY2$Jqk!-z>5vc;b;>Ubrhgr)PXUp%y9?pduM3fm- zt6S;SA_geuK9f!>I{5oK+$*=9GQr!qpJEUAg1_pt7_)eWpLJlZY4wll#Q8o`J{pzo z-V`WGDQ6p}&!;$Y<+^+Re!d6;kBTkp$d#A*lbXW)pUOC4^{?+Cog*lq3V_XjAh%`M z;ByQ^biRvam+8S70R#uAU^GRgVTrm=hPjA?kuKe0NM7t{nBY7&PY<<+n81@l z-H14AxWZ89EdGWXex=|QKaxftG-K;j`BUyaHO^|Cxwz6`c=oWt zOt92qCODf%aKbqS!YEnDucaNw_fd-|U9H{*jH7qMgm?sZUfKUD`~mXaax45ynB_b@ z`r5u#or?z5YvRwp{yoia9V%#qh@i12Ixbz7>`x5e8W z4pM_pK?e7xxR2uSLuJulFus3$r&G~y@#%|9ub>uWS)osV!Q%g+XYJ%BFkS<9VOp$# zpA%#MrI6vba+UoT0%4<%lf-ZOzfa=NU%&rUy1wW7W6d963(q}!@fUgfKlHUz?*n10 z;7x-4viiGAd6}jY1Pvsb8_6*tbgeNPq#l8sTEMvY7qQ_gf6gfX1B<3f!*RWKqUdbF ztja(9evy?ChnLYnc}ddV+3D2^@fHlrqL!X znDj=SH%LY2gDmlY&i`PJruk-+(~jlqBCGdrVSrl|x$Pm2hgb~Z3EzE-WfD2whe z>jxbta(~yn$gTm;iUL+6utkZa2Oys19>l7TQ$LWn z{hZ+a55K*rymlwuxcnT{@)Ak(FNzKSJQ{g$T-;8`NCk@z?j;`B^NnPysI|0U3+|5D zi0r)3c}Tu&bh#f;m<9u6k9(i`w;t^X9<2BX?L;J8;qC@3$RpgaAl$L#KO9McTDM~` zodv618X(UwGFGIHeGJk%>|TEBY3JBc>+1A-Pweo|i}mN4bavB!>sY$*%lcuIH9Lj@TF!OC`->Fik8cekdh@%{c$LR^}cX4gy{}XjWtiol&fAIqN^Sv@o zNBzI78UFd_|0D4IubK?lf53`tT$Nu%d)RkA4t|kJJKnMI_pTs;@97LtxL-eXK<3}c zMgjfGVz0ZTVdi|*?oheuP~JQPcz(j0)h~g=;b|7Mi2=MD&1-ukwHR#H(+-;9I9%VJ z_e~YOTDzZkNq7lCVm3UD`}`C4md~ZhdHoe@7LbzNf(btsG_d-Y3ZLKa9#6`DUkXJ3 z)gP&-_u>xIR{x;{r^Dw0n8GVBh~I5F<=mf7&X=vJxgdweh0v+;+?!p-}x+cz_xw&8Sn`aa4od%aC* zzb0V@Y&tic_p`T6(=4sJwD_s+ZYce&5S97+TKYHN8eEReQN8<5wYWlYU&0E^`bJMz ztdxqSz*igHwMV zOaA#*mNgDT0Oy~~hwjXc|BHnFmzBt3M7AEb2{f*q4Sp1U2Nq}jo#5l79#Cv$q8Gr< zYYzLt>jTfX$gETa3(#Yw9Ufg#t~m1OS+L3WLMMtP+{$6?zet4c6UD{8SY^`}M&Wt% z$S_?@AS*`8x1fUqe^k(LM{B2FG#VfSoEFrgp7(Scnv+)riEcPNuoCj6r0^Ey9c-+q zS-RKqAu>MdG$?cGPs2L}dI^C4N*n%1{_+f_&ZKy}gd5dw4Whbus}J7q&kSX%HrRQm zg*N|m*T1?xUFgq@;rU9e9Y6Ew9phRXh4jx@miTnf)ZQPR%h5e&f*%%1qCvHG2#0~O zixPWrBk%N$c|Bx~#k5U`G|9BY-2+}}fZ6u*5_U{Gcje!UtxNyCXUfuP{SVKF z8|2xh&U&&Pwxz(m>g0yy<0o&J?SmQ04a}v*Or3|C)k<>Q4%V~K_N?g_a?s+ktUwYa z-Ze+@Dp@Q%2Ob|WdVxG88*;u7tL$-j2@l>R=rtT5_nJ-m(%Y!p@9bBHmBM)1 zh5o^WDlxK|UrTnrt&)ZZZfM&HZolO<`o$B5OeY-+}?V|2|jI zf_Kltks%T7wClk1vHc*f51v4nvIqUp*|zWnvKKp=Q=Q6Bm)f~8wv$KgeH=!`hV*`w()xtt%Lr; z7jnxHW)ECj3{|^fhqe2NR&=N2l;;!=$~jWBs>Z2_4s_403Pox7CyGQ1{;j0zRNL4B zHb|4e*mDYH)tm`u4ftZOBO3C9!1*GDm*?kk=g`8O@y)R_D3Djv4kN^W{n=D!o9Ryeg9Aufry?)idv zjWRNAK&$udZ|o$PDu@VXddyuX*9m48MSgzR9&@r+&ksUwR^I#t5c#QCWPs1A7e2t= zHbs1#ZyD4=Y73TBxg(z@vR6Ea*Ha6T&BAYFk{GH|Wdu#N!DOv`nSanT7z{bpXs}X$ z9T%#5+hR>umw=nwp2~vurv$`{hS`q$fO@Jm(MR(T?*rtrm9DU>6;ILKfLtJ?IbZM) zV%(c&c{=cFcU?FenVa3vHwSXmj7?f=KdEgqyLvn zX(T`#nVWdK7ZTXJa{cjSo(}E98Y|lGCZtfT=k528t5Oy`ivO_;{O@48@gn+vX<3bH zZtH2B^}~d(HUIpCyD z6nCL~|7?jZ?>Z~mofx0S>Sw*EVWlR%lN#-}$~l?qY%8lLS6mK~jYf5DqW^bEadgJZ zuds?CJo3lk@yz|jFoh7{RTN^FpJ2k=bQyWFsx!vtvhS=TS|s&ud5<4n-oMNi-%G{c z8Fn|9rmE@^DXIe{<-{M2Ii);R?b%Uv{9H}!F@&hX=OQaMf%WGxC%J& zv#~|@eSL&Hqx5=>g&q0X0L;A8xwwGc;CMcmK((_hj(fR_J#4u9vv5ju=I)Gnwj-IlO92^pmKy_dkIY_8_@x8}pksy0861LewZMHhIVcL%$b#oNDrQJG}VD3H95Em45|}umo3?g$z+EoHx-kVm&s%l`nviTN^wfZ;(HukitM z?|Ne<1XYb>!+X*cbySu1<``uuY6xU?2_$`|Ye@DH-AHO@hao0Sct6W~M0>2PAG4X% z%8v@J$%;AB?N&rvIrb0PU+RbNmkBu6%NzdvV@>~}{<8s5F01E67Qmlpi4ybP5gQ1R zc7R$xgv`zptkZOz-IZX)k`yr&%8t7rQGcc2K71wi0G6o6g~wCTLszS~@jrG+e&@C)32#@PLx1pdWY5?F4iqiqCe%MIFp5 z9%Bepy)TdZC8M*$<3$(Ncv^>NY28jPu(l08Sxr|zLEa& ze6W1A%NU6m+s34*SDg)S1`*e0PPUkg{eYb5fkmF_os~Z-wfpipKQ@`mjC4NMM2$D$ z_R2p2Ur^h*Z-Lez<-N2HLD;-;Ze=B6bA-P7Q2o`AuyPf)n7K-(up}pNj+&u}jP~tV z*WphQBKg}d(Kb%w0;NH%l$0i#HCjBx!=_2dN{gZxPnaTBHmW)5W=UH#qt;gEN)mWz z=UBz<6@pzgP_Ym}2cuNg=OZ{ZVbuG|-@# zsC?PVc4Z$~F$T@5{Jx3$5p7}MK|kaykoNlj5N6=FQcQ|Vwn=O)2P zQ~m76F?4ORu{&WW@Od&Aj$4~T)=v`Y{JPg#LNm^Dl*NEfg)#vBMQWU%b7Jvgy~tFm z8_8M~x|oY}@ZGf-hYG=e?i4@tM6z#*xjym41BYlGoD~I#1cg2|qVi4#yqR)+X5`b1 z5OC2dxAy}Y5N;62{QM!c1zr7j=(%j zmjg9)c5gQW;?Rsy;`WX(6!v!;{*}NN#PavD|k_BWw+D7 z(WIili^+A?#KH;LV+gc=Hi)aqFteMnS)I=ThDqd`t18;8wP@S5lANz*Ut3A;gJREqbi&RNmMa!&UWbj4jEuukHcIjO^) zF8w!Dz4%-%4RU4newz)m_@w*Qxvw?g>}XEM0iHH<-k=;6)*w5aotUfsguX1?S2C0f z4As{>JW!~ESizI*Rq$Slee#`|6mzYdrC!ibql_kK?S^Tug-%F_!tBl2pO?ng>x zR|iM{bjt$V6@~}UelTLyxHo9df%1ppz%kihNi0pv??q%{rkyw7630j>sSuZ{X?E6n zGtGYWD4pwXjG6|^D2S6O*<+39OQ_wJ{S~V2 zL@>|P2&jmYl-`s-iYGz1kFnKgdgHQ_SMi2`=+^z9A)mIwMsv8!a3%xH$kMRO>lz5Dyuke=6bc&MKdsm4N%hv@X-ct2Neh38Te+u*I-BY zGpN%13TGl_BSJmtQq5_@Ow7wPSHE;+6TR94P$bHB{>7v@bT&_+%D*x>OC`aW5c!lf zFfuHW1~KnKFz4U?9Q}@wti#kch{2M(B+#{Jx^^fjolqUZfW@XWSR9vvsAir*W zI_A^{sokFCGX@44-Sj164gwnCrQ0ynZInyD41Cj`W`taSE=@BnI1}7Pg8OFy-gl^; z?U_|Qs{9l{!zu=Sp(bohJVlg9g(D7vCLBAau9MSX0uGxxyc}>{_hlkqox;`adMfTQ z%dauAFSD0tg_AoE%9A?=Rh=m#UfNxqqYB7vrLPw3&X#_oj_7l&k8IG$jVvdFqL@^Of-22?D;fR7+o8cd!`$g^h=+h!v5Ik#2v>`T8{5jwF%I+k-p0}VfUhd zl1XYJSzP0V*xvhV2!2S+2)-BCT%`pb*3@Vpe|w$?A#p9QTAg#IDD5~lap4+0C}8Kt zuk0jV@JhS)$Cq8J1d%3cfT%jIa9l1vgDBNX?xnwO^A@W=U@kIsSni*nEFWMVQren^ z-jh(;4tJCwR;m^4bqr5v6}ChotG7%2kH9SuZZ5Mk5DiqZqpNvmx$aVf3F8X0uKj@- z-&{kUuL)1ZgbJncQ}hsF2o8^rZT<4%Br`9j+rS1Hy~0>G4$b9U*~{ckKXdi8Pv+Sk zl|ZqSc&>e1M#3>Ykp#wFwlzzNg2yeP7u*w{IK1EOKh{7NUDw*>;Q;yKQ%=02HBmvt zjh3eF$8Ti!n=+xVr&Idm6t-pS3(sdGynhxzhka0sta zqUl!&@!YacvhlpCn^w3E6K5V>3flJGs5JcHkk|erS;Jw29Vq8bw>>T+KbfGHW6{w* zb5`6BV=xI1d0x?dl#;8Rs$INI#?^TB+RA_^`rYyrKye`!()2!`ay_F+;ah(bhezep zmAA0`-a|RerWd!-(SRs4o)UPd0tYBe4$#vkWR8i6&}B)!NOdudY!6W0{6X=fO;zhe z_gO{v!%_d_`bO|l(7^BoVesegpZBg@X0^D2kJl#`?=^w5JN+}A56?%u6>P05S)hz> zU?9~H=CF!()cA4aVbK0yR^)N44V?%TPn_Jgvx7qwpZ#*A_Cp!|g8I}~|DU?M1XBY^3d8g8|=fbFIMLI~E zPH3DGfX3IK`nw@rj<19C)e_WMx?6pl!)@XnfjYCTGH(LfeEz72zUIn zkrvW$(RbA#;ea+Cwc<|6G^9EB>)B+7^Kg9z^|AS*zBR>R@7lg@19 z#!tjz&A#tr$9DQtkPO*Ov3J1&A{`LU-_qX z7X6m4KCOp*qW`u@Wa&8 zA3WTG>;I&<*wTW0I0VgJYzO$&l0XC65qS2st3R2+~<9;`ZKYneOd?LHXy58h66@2{CH~z}CPHPbmyWShxIJ4qnsQI?x*dsr1 zH%Pw(Abh`a$je6*#jK7+ej0U5kik>i678I6n7=U66bUdC{N#HZKi=O>El<*1Ja`07KK1lBGRda8Rk^%4RzMoOCGdjNEGjjgJ%)iuKUNF5tA{91o}7iOM$E71M?`gB6Aac4e# z*Al$Mo^nvbQuVz4(}4GyEkZ^gPjttu8cXp=N<+8BoQ?UW>q&s#>3cJ@d)J0_)cm+M z=5h_%*ks0j%iI}OJ(U+t&)lV9ClM)GbL+}R&m`(l5%cbh-oBO1Jaz^9(dZ4&@v+-t z$mYJAHA9Z~XO``^yZS{iH&tu+QZkKMZy(>cBa`sgu+SO~OnzteTK}4~5tz&@6$BU{;~w&g_FHqepmog9Q%iCywg+{VR=X2x@oOaHA-f#oyb&Ml_s5aRNWQjRoB~l4cEcMjZ7?iWO!#v8RvJ?umQRnje^MF>|l3u zrOS?Anv9^U>X!a&yPKnN2j5p6q{|&n>YLZvBby}QALwQ-LeL(Ofn!t9`QaEw8}p|< zQjiK^k;W+Sq)!mQTBFq{=vcbBMdh1>7Z>{m*ff9DDJ<8YGug#9Uyx;mc)R%G0i_0V>@yaqguKBZhx}c@MO|>xy8`oeByC=5)N>J2Yy=W zKRLX{et=a`;J1MRZI4S;fZ7j^4QR`y>AFl8`XSXO~L-DUL{NlWrK)MN@Ef#KBF<7wt*_&(3A?{0(->v&X&XN5bpj zaxa6oH{m(wpwt<59v3VKx~A5ANjQ% zPFZZWY5OgtVf{5GlP@l2PircZ!OJ*Gi#;>2^M*OAHpvU>fbV5sD31Omrs9&OyCB3! zFU#o7^kEJs(u+$GFsSbiYu0g^!BiT&NdkxKp*dUWR$)86GFFUctQ9U#ROKb5#|U;fdEq zJwSpEJO062p_>s^(3QkALF;wl?haZ3Q(`7ETMrL|l$J$2P%h(|(E6g?(*P$$65*qz zL#cpF4UPp%%wQO9W5#P(KgI&vTc=cd-QcUFw{L5KD5wzHMoy&&bsM(i(dRqG$@A$% zQI%O4=1nXSzZMvIg+9$Qb%ww!NAat48 zV4iffJ-QfXg6>11c>qenG9Up6&8{akZMPbqz6!~M4ug+aFd9njG-u4wr76DZ< zG)T3{C`1Z2C9G7>oEQ}fP$+fUu2-STeDmB_)pK>O;Fryez|FAlJOh#Y!Ab9XB67Io zlUk1{`8M@wN7<+^ww0t&lNW{0v@^I`PAXccJUtim)>brYFeuD3f9{=&gM{v169?42 zXi(#-^j&*AoMpv-gK!;0+g8n>o@B(tasZTopfVJPw!AyjWvds(fwC(Y~ z*u3V@{EjXc#W}d$inL$rYKjH0kA-;JffA5z@x;8sqAMl{^J zshQj{M<6x)kfH_2`IZ1ak#(*=s~Ns~LO_B|@zl{_d)~qoNg;0GOR&nt;Myq2xDrE) zgh>|CRO6cGXVY5t(AL`7)=mDc#cpOX+EOhk-oS^Ky8Baev$2jx8|)vehg)fy93O7b z=Q+9**WJZ0C^A*ZZ+Mx1povb9)ljIl&Um~$6X_)Unwz+92}oG3QKMIylgvIlF=>E{ zhdSU|;C(wuEuweGYF07uE{jvhZ%GW-7{7=u*z9PL=6&!;pTN!G#W7>v2sLVjJCFO< zUfiFofDD3`!sd6Ft^bF+_l#<)?bd&#i&6xoHvy4emEHuT_uiX`fb?EM5oszV^de1~ z2uM>(fIuihkg6cPgx*PLAwYnf_`Lhwd!O;&`|&y7&N*LTj5S7*weEGVd);%+-*wH2 z^#u3bT(DGx+b0{UC{Y!J*W1}=<02`UkD(RqtyEN_rT5!Z7}~kna5tn}KG%SN#ujW& zN-o^;$`xK&=d&3Mv;hLZr``>V26EPwi<)U4On$e1!2Syv)wkf`U9LH*gQr%Gl3)`? z_J&7%`?6P%$=F+^YP4?C%BAb})?<8G8YY`m#U>t|kBC*woO{1M^?fk){gF@pucyV`RE4^=s2@ui zbG6k5R@64ywP}(6T$s#k`3{5V63;ZEh(`0@!2?wWzr-ZnLJ4=i*CR(%(IFlQ!!DEDg#s&>iRljO*SmX zb2vXf0(?vstC?|CJ!a5R&8F#L#N|6XPl8q3EJ>%uNw*W3!wJkiN=^%~n3zrcwah8C z8chQK3zk-HGyQfsCf5NZFApC{W}gbJ^*xI~Du)!GuL;*bumRT;vD^) zT%nV0E8!w7i2CsBm&CR9#N?QVK}wtdftKKc)QR;0`QOyf^g2sAJ8#Y-l@Rc~Xya=P z_}6D5AW15wRkl_7u%-9iakxQ_#8|~Jc1SP@W6oQd$qL)O~9Sfpsz6UDx>z0 zh%@tQ>W~hH(1?)m;`Y_c_M@hR(~fKDql{1* zH;HkNDBDgil6@4-&sZ{Z=&n*znNhv%Rh)3r4Sce|y}FY2_AKi*f!#2Lu(=~9@{=Q3 zc}}#XAj3_1ImRfkXMdcoPSO3taJQ5sZiY29BV3{(+_P$G*BUa3z7EuM)rUxWl*}_e zE`y;>KEBWkcaX8gTp+TVVVvdd&P?KKg!fw>eZTLVF(MK_nb12qYkbq&B2)Kosmv@k zY$S`)jNEpgsiFa)noI=?zgzuE5us^BK}GgHX$MqRo_4HJVb*qKns~$OY`>E7R1b@L zsw`(}n3a}+j_0?o=qjp}T^wxraI7nIajYXw5B<$m-i9RJfLAf3QT3+WUiw4@mrMoo zRD8qi5Y779Wu1?-gkj#cf5*VXcq>G*E=rn%F_$mHpUuSuz8FluW5WXJbSDaDyK5Jk z$Z|2iVb2?{m%Oxvd+3+|YI!bcz^hm<5=mmyOorE_5Py4ZTdXL8cS zKRm!Qmza#WMJ!4o$M))*dKI+N!;v~!L7~LE#IUzTys+xHgS1>;vf{cUq*4m=UE z{k7^5Gpy-l8qdiM!3<=Fdwqnfxn)2RxRf3+tolQEGMYm((|&|QM|2D#8*DZ(`})fY zrQki31^?zfbUez&b#&=Pu-*g zp}0fK{}FNnq{N!=9}2{QKzN);ns1v04+kJgnA4?t?eg-@UW{lfatF z{_AS64(u#H{@Jxey8f1@%n>(+U(-pAqFJ~4Qa~g7k5)PG0+n7z{Ou$1?~5R@io~G> z1+KI+)s{>UCXfLY)_&c;+6#Q<=}$#gC|L_(en%%nVdF2Z>w}={_((WdAP_;uB?lS!jl=gz!nQ2r< zzlEhJ>7Q=~7EN^GjTj&FQFV7NBrn{hrimr; z(Z(wCxc@8U{U%e<{!)1JeIq&amY-1Zv%ckh9`imM0t846EB4_#OZZ23;0YD&ok&1) z!yCag>cc2ps6#9_?PfLSkAt7Y_z7gaCzc{i`Cur5IZix@#sze8Ka%}TPU&|jTN-vBv$>OGva?3AW$EcUqpK;{{LphCFeYG zWdhks#E%yiD42E%CD4g>$CaR8nn+%Rlai4MFQXqRLEhtd)tXu@oy3U-Rf*!i)rYmL z%=OLmob6i12_xZL^D=t_o&}bFo}^DA@Cogf7T=-*z)2DTYydI_?`NeatdN zo;2+E5nAzR%?B5iMBeORK5XZ9y-j{Fl_*%`V!*~6-`bFT?mWzLI3I^Cy9eNSqhzDM zacisu;b@E2+OHRsj41U&9?HEw|I~F=rY(SwhN!&(;FIf`HhB}~+J@5AzD-<}b^QL7 zEBKHD%!XhfhZM1z0WgX~v_$D0*+7if#gLiGmGpFUHE;p=Bujk3Hm_??Qm|yN4^X#S zG`AoR7O5R&>>Qkvf4UZ?o_#vpLcV;9vCWFc(@kkVc{}LrB|gkXb((uhOU_0?qUkDB z?y_BAbB>4!8S$;SvLxtenIv>{?p($YSbkkay%|T)={bE*2D$o)SB?y(GTQh*Ot=5L z5UGYL5{CxFQeE{8?y&jmI8Z4F3m8Zlgn88P_g<&b;6M~Q&*QzWhpwL6>MV6^W-!Vu z@OSy_+LA1%nu}A9bP#=wTGtb_iu?Ykd^MJ}a~p+uLj?PU ziw0$voy`pC&lcG__tn~PDH|R<{@gwueO&zl3@#^z;(oN>9bNUhcp@ve&xi|wmju-` z5j(}@)&;;HMUhBK@`CW*$7qJi>Fin}dY0=nx?$4N>pq-d0uj?k|vwd=bZCq57auzDLNfwx~ z;XVES&;cRtnC#vPS0x-)NH0OCL^|iWRT%d9BP#=0WYW=x;H~P4VY4 zCv45ZdmQAmS2`F*Ymohx%&DAp0~J5x=M^l?WYs4fL>ggxlEe8`3{ ze+TuL>IumT3kv@^0k*F0Ec9DVl(?F4ImUKBiV4IEyC`oL66b>WjAa>^C3O~Q07hU zD03%51J-TJKOo8h>{2?^TMJWbwZ?^ykRKXd+j^;yp?n>gf)mm=o#M2fE{~=Osf3j` z*Yn=aQ&hJXJ|)ua9^N13`pU&{tFz=bOCO38={?nbNdJHu(RKc+h}K2AJb(OcN|208 zQb2p&j^iG4-$yW&ETG6QD*M5xl>P-^yVA>pt3!G|7Sb3v7)+XZ>l70|vwAOZ6Ka3S zvnfiFh8Q$FMZ_O(hW-O(6pVmSZ|Pcck*~UId-vgTTs?c;)u@f4|E@y1iKtz)AuVDA zupNmeV63i%cJ-hSUeB;itbg?m8D%yi&wsb%#&X&>!|govFI z?Sek3pUN^gp^TTeA4;mCoYcMeACuql0$4{aPp#D^J!xrj?hjzTV)O`c_x6 z(nQX5cU8}A{y}kqZ7^dv(`_5qPF)(=cjrZLdvMcDjOL$X09yiV5;go&fb@>;h5aeH zgRsKu;eAImg@l6~T)2fGE}s$v2`|G7+&}SC`_%4??N*;c!Rea`d36^hlThvvcMwfCeLUb@+GL_Lu_VY3x7bS?^Y$$I= z_k|Mfj;;EWFe7q4a`y-d7j4$l9i(hT5VLOcj^o$`54&eUVt0)GO}_ylRF({I2&npD zNuM}0e^ESO^}H4z!oZ=|EKvz>T(WV&T>B248CBH)lhP~CB`qev4x#d$)Gk?z{+{&V zVx_lkYmjTIVSE#U%kOBSte5V8*0tXE%$M4#U-9YD98?z{}M@y(s2TUui2MLLUzw`@_&^T$n6-Pk){ z)jS(6PYOn$vcOy)Y3c&D9kTa}T%JG=dnEiW4f8Svj%y}m&ze8#WVtHcu2EG}jUr+) zJ|3t)5e+aeU40dOMa9CdBSn3=M=3?kl{-}va|3E713DE8aJXvxPOd)DI+sX(q0Ra8 zROi-g;scg!xbpHmM;RylLie$jTP6Nnb73U^OQo6jHVTwH*LYRJV_QpKF&fN5rBJ{L*BO_OzekeZor1_VBRMn{7RxjkZf9G%+T(6^Li31OLrZDwo~i|`VIL*Jaw?es{u%we zLMy_vut9)@ujs%*qwVZb(D`|mKJ^EqSEEm}&cmL=&uN=m<4!5hJ;L>8XUWN+O@$+I zkg(b?*`F3O2w`>qqbm!Bz((u#@C0a6JC_54fbksp@{2G-3tGU}`8z^bbJDD(K!oDF z9>>IjPi>cyx3`48ekA!v^|O#d>W%>QjN>bV^~+C2k`b_$0rNuX>yI<>rLqvYQBb&H zH!5#-6CnNiqYX4O`qiwI6*Zkg)BdtfBzgPvFiK%sEkC}u`TOQ@c&+_40Uh@yikMT6y8oaUwBFb8A7E%{Ou38IrQKA8kVGcRk&uSY# zn2OH=G0sr)hlY%_#~&Fs?<=&Vbe*^qI+(U{U8=Nlg?>Qno3wPTerFjZWj?w%sgYmm zvWO4yjWKAfHCo{{q#FVPo=P!+VviB+sZlVqZ)?Imr7S(0Y=7Le+}kSee06hPn!G(~ z;NmXNcV%%}$j@bvpikwhH2?`nFoH-E942viRq(It{?c4OkHq70ZCozoe7^*H*#~n| zO0lDl%YC4xWwbF5pHk}ftk?>_dZF4DI9jF7fjmABBB<%QEZT=;4In+Ledhb;I6iA- z)*XG0#-yYg7r-;$b_kjspiWsWKD@@o9UbdjJ(*9$?)!mLQ}Y6$8FQJ6 zaK5hs0F;Dv9IwDf0W5R-AsNj+4zJ`!+iec7M>RSloZ1@NYm=K$w|gfBNogo6F?`sz z=lfJgNfgfZXF~T%)x~lAw^YA5@{(yMq%`f!8r?|FL_$(Ck>Mla_qKAkT=#Zlwgm@d zwtF&i@vO>Oyb`(xACXVJHmM*-g3lmD^G|<5rxUdYNgn{3Ag{~UW0Ut(MSl5KE+%^$ z?NWZgKd|7q)_*R~XLtwnIN=8m40inM<>Is)=FjTRB@ukWr%Q`H<%Sm6w$Rv8i7hJf|93e3^Mdxy886kY9a>r^cj^pHtJIQ^h%eQCcdD6+Y zj^#eou$>K1*`Cq#CCNwu6V9Hb(_*+W%^cThFSb_^j3*d6pM+3(sCnDpvBP8i=i8}> zLLh9;O6d%BeV?kc8b<&}^nLU8{tQb`!|I%OgiBf&+}%-ArRuP7K<4Ne8N6oAlbye5 zXlk;xZtv#0-$R#O9bM)D4N}`U(s7mG%-H>|2AYXok?)9|nk5>vIv#r#7VQAPhNJTpV!OV8=(-Ry>fk5AEKB}p=B2W-z^tF!X zW(}XKr)&g*{S5w7Uc>*)eCml!tzB$`J0^dz>FVh)OPwFE;XK@oXVAJbfvM!^7Pktj_Mh3rg{!Axm;*7O|KUxx5Kzok`z^ zSkwhs#_1X)^>i%`IW>ZF`WB^GcQOn_>b_DR^ImEW+4Vu^E*6}Ki7jlIJ-IP5m(WEH zon14^Nwbm;v=*s>=Jl2By$J1hTa%|PGdV6au^cP(8B#_(CUabPY>vvlUQ}yIS@PZ1 zIanJjTm_UQxUTYDU&NUnDDFzSb8m7JhA^~!IpKC0@7UIU(55Z_0Z}t0KMMl3^7-z5 zQ~Rv_zJ%G-)g+mjt$WOxb8z?putGd-%l$Y`S%c%6vD}|= z$~&T!3iP@E&r#>^1-z4F9{qOJUDy^k&~vUe9o62XVk+iSbeG9#bVVk1DV8EDImmY4 zMn|`!^hjQ+Lz_;LcGZuLVx?`F_;DxcWpTHAxQQlc(5s|JPJkKh8@Q^@w*ic~M&*LZ zJ14deotEHBk0khW+BT2b(31Xyv_AQ62uh%~G5ZvTW#@_a1YmZ+O-25S^bS zWcgI)NBGq?RLm_JL@$oBbsPJtLxwbMPaSi#ZngdRsFys66d}4A_HbC!{PSkTiuiY7 zfh*iN5^a0i_u#e789KQ^Pxt+b<(=&wR&ua$DbxAutjZ8c4@Zwv7Ns1e`SDff#D-?r z_;$V2!uSy^4>htBh|SOxXW`Ulv`jih#ALmgiW?(d5!9Z&85ip?r@L8|fovzqVfnLL zRVU}~lc@{Cp3HX7u*A<`1r-_m)pD($8>}e%ORF+@+t#UYO0SJe6c?(AlUt7&J* zt!nGYn{-rO`^@aXe^3%&#^zw9OZ);?dDb;g)z($lwMpkKQ6?Wg^56UJz1*i?8R0(6@kX7$FSgPk8ugyWQlGKqM$93DUfX~_ zg8;)NrENocwW5Yc{y4L?nq@P+dZ4yf$mmqLSo+hx`w=;eo?kXs8^ui7Is8faE(F-d zj_TxGGHa#5E4#tco2D;Sn=(bdm9DWlsq$}-BmjhJ?lCnh{V7=VBE3lh>b!Ec<_KVs zE-z?h=q~XZ{grYw|891FLPGo*e_Z`U-yP@TU4?GrDa9PY^Q0Ry6k2Cm2BhqLLVs11v=SkG(j za6aY@$XH{Swly~mR?7&sAxnK>ef7hfG<`5tuZ>%W1sJ`QQqeiUc27+$yt0V?{Ji&* z%VSsec(a6z9L+l(g`VK%`Z>LN8NwOs`ZuICN{;#c(flelNqKi$9$Hitb`^`NmOkoy z*#77h^^eU9r{c;deIXncQYRTE|9|c$W zECy;T9DO!*mSww&d9OR|_C%>Y3mp%3)XrJ{v=rjpvYNxiA>Ls(A(L2|66b8{1iVm9~Xe`7SQeJ%q%bL^r#xly@=nfJDG8t4{0wSWkk48!L8re zGA7<>SlQ=a-{}yCn~|4wTDRzbm=+=We%xJ(+kq$A!rfo%3qr~T$a-a=88X-Jv1)h( z^rxO4Zk)-uRLI{RHn1*yf7|UdPvqQPf9b}2iTRIW{SIfk#?^8)dqY)<(`hk=iaJJ_|{pv4xUe1RNx zHfCHu>{UE82zO5Ek^Uq0?ww-z?8R9obyBFV})8T(~UlXG@-yW`i;u2qYY=(`!KzPY^sUV z(FL;ZRZUbF?QNGXmmJyoxN4lY!o11zCH!spKgOQ5KNu19@(e?p^9@u!eX(frG$!W$ z6Y|2uebgzor<&1%?N-aTV3Mw<<4;dgnw7A9MSEQgW)>zmjA1bb{2tvq#WJc$$h_AN zI}atA%d)Qu#9w@vjJMl?)>URO#&(>t3r09;?^`SZhHp&yskvf*uZ_{ehnAmeY^6hQ z_H-~fz3aQ=d;+-AV8}pQ$3ATB%Gt=%?B>L5{S<*_`D=elm%*(|>0}7oZTI+Ny;iGS zqD2Y0O+A?TAdk3W_FU<=1r`Z|)TH@~$7s=FzRmqS%k>NM`%}Q?olr-R$_My%h;dVO zn1CTgKPiz04dLHQX#Q1F{_$5(8P-7N*ao!MTM8NyB3K0@@#*EHB47P1$_^hi(xP>3 z=z*u#VMQi$s-tfdOTUMm2J9b!z?}=+(*dos^HKWj_<{Iv4FWLg$$tql!fXvfsz=9c5tmzTS;n67D+i4Ik_| zE`UB)_Flog_r4*x7l;+i^7y%Idc1q`IS(xmPbcbsom6>Z1FacdhWmkM6Ui+;gqPY< z!IN?yJ}6fCm*RziT2cSu{*G^3x;KF$oJx=7`uh5S9H?Z`&4=u}y?Pd)_R7<$Y&O*- zz1rf&{tv7$<}|L?ihKOtXCqHSp(9U+uvHbZsJfP?mB+`{RmT&##2T4ucD4J1AOXfA zaDCIMDOzUArXF#@ZTshq=h7_Ae*+I>b7>YvDqz^$5{;zsVqT ze@l|%AmimvF}b_HUk#XYP-yL$5T@33K-~jLpE_;(k_A?4MzYss=Hjr?e0ZK?AnZBT zM+c&B+%3tETRsA#d>`5erZkjK4Gy9-q|tsZPM%gu1F_(0&jri-Wd1r#lnLiMOuHzM zfR)Na!P2|k2HVk{lJg7pXYc=n2iSeMN%E$%9r}GQJ@crRmw(*>dm#qm^NHwo zGrFe{RgYo@5z!uhP_6zpIb`OyKhL-6b^Dob)5=>(*3-!*YB><{9a7a&^@*cjo4cmC2D#R!;LzjA|rXZJ)Sg8`*41Z0H4EtZB%*S0pn~eJOJU zUr{7XH)xe|4C`CgHN$!O*f`!yvb?V;>HYRq0gl@)Fpg#0<93o;=fe6bXs<1{{X3)K*-vEw!dk8jE zjYaU)006eJg;u0EmYT%R4BpijRo+|fC|fplFf(<`6`R=+4S!?W zo5)c1pUBYvG8ShqZvQ5BF2kyE16}yTD<_IgG68bwPokr-Z19LI`N@00)MV@JmfX3z zwYT`rUyE&hBy#Zgr|?++pHhzGoLCd?{|}&0 zHKnhH-*>k!d+qq4{Vr!_;yggX*)oV<(+aI;^QTmu~g& zr<&a-Pfz%)O!n;EduuMlq8BF60TuzqJtAVbUmu=Ccsh~o_oYOMMfVN&+-zQsxNF0+ zDj$AOO5*>xuWuoep<_kC*3DBcP zzSH^Wp&NK8 z9dPrif5Ys9RuU|LYQ)iyf#O+j_Z`Vr65hso75Mv(>Y;M!LCx^$C)$==sjJ=jgeNf_#x)O(dpWtnVh9B*#Z?4YFnoa89W{fWB|2DobW6Zqq3UOf~r zYMH&|KbNKLZ7k*$vU%ItszV=kN+KD{f9kMx@m!n?9737KoQO71I`fUws29Ihch@Rm zhPr!}-9M;gY)2&)xbk!_#*EsV9kyi$&t=@Q4QuNF1ILhQyhv;Bg&px@V_$VPgX6Ti z60sJH_|Ej{^1{sbGpOJ@VH9Vxbv>TEiC^=jTl=swkC5f}KH0KC^c|v(4 zeMg(K1+d;Wf42qdv6KcF9X>l5$$(tD|0RQo+|@%5S<5B9<^B6Ph2={L;HtYk_2}%( z&#*hpF@TRr_j+{ps=~;bl%gouGWNG*BFPz&y2P9xKci#^L@U64b+(teaU1xo>KW%o z^X5p-Sq$7sGl)$z&J|ks5p$D{iB-9+=%l@E4+s5l|71oV)eTDI(tf`Z)CP2dHVgDE z+v44Q@R>P6(qYd*-gMRBcy?RjP@N3Um9VR?9Dex21WbcTWt_6K{qD^KyI^Mw-^S|j zT`6L15~C1~xtnX#s6GsRWNmx=+(+w{&UUSu+An{-ZGH?chN0EaHAWvkHQg``#~M#= zS6Nh*J(Dz8dtbw&y@f%fR1-(!w%?&)i69O9ctTPgCQFT!3PcV}rL56G*Co{R4Cl4> z!%nxZFU8ewazB)gmg`w3126eSlY#5@-76Ek^$(=xnWG!`c$5MH1Rju^vgo@62R(Gj zyqLc^j}_k67-63S9t9H>0!pzpR#q?z@tbOb(Hb>q%2Ez317i{gZ`ubk%{eL^Q6qNO zUi!xayKl~VpPW;hjfp)Q=fHI1JXA{D}lE(JDFbZqVkvkJ>hH zUo3oyh~G!+K~5z@^;>Ki27a~Bo}WlX!^A9Lrqps5?m0doAEN!E*7xCQX+q11NYHlL zEienlGMMEI%5DBc4ns}$tiNRajauGkcXH0d<%nzQ%!pbe+X&ue?#uD8ABAm6X@dMUc^eZI?8qPKYpiu67Amz zi58FW!D2r>{yeDl`e%J>Q7i5FdCVz7@!QHozZG+J-YulP`%KK`o}1B*Ew$kv@4PoGE=TczW}!vOAN?_#C+Kc-rQD3Uihtv2YiRV zc>1kx(DP{vV}y{1I5n%syHH>?DOdk66k7|P6H>NA4ro1%b5*Tzi_ne%YcuqnB z%Zsu^{OD*NkhbPT6&*IadN&zZ--x;!9#yeRIWcmlk5m$e*e5C7)#sNL{@?{ncyfP*uv+3n%)TrGa=sKQ&%)|zAr`Znw**fJOjI%iE4H?7`ELBjvy=+IZmb)Ht{fV zeyUs|pQ)By{=9j_xXs$JD*3%-T(6{MeQ1&|*UCPn(PDFPS&R1Tj%TG#EGd&G7GAYA zMF?pv#o-BcduMFB3(*YfX!RwVdEJr+p)VW`HQBB?Aq2ht$%$* zUK~~rmR?*j5zkP1jmK9Ue1$_dC+V`eS}3eYCQ!G0VZm0)1{~-E@cL`BV`l`@pOYu8|9eVP|KS)Q5VWhMGGT32+~L{^BRh- z!B_W^pn{S6k&Id)UOWN=!3>h_BDn6H-j`_seoU8ma^F8?dU-$cVB}yS)asu0#wfX&QWo`OSytlt4$pxbC?eaks(C@7J!nNt4F8?7OyT@(anw1hQAb-OJ$(B zK74|+At$RVz*YTj#)~NcOc<9XAJ-+he&&(q{U~yXNf4(}1Q@Dk#r7}>zWGX)C3d_` zWr>V$kX!NBwAk9m=N|<4?iv$#`?PD`-gWwF-@=P?+^rZ%_VqM46f2?Q7XQ%jtUWqn zT|b9QBmPwclw$95@(s%l%K_~N*sn39e>XWC)uPqi&;Mf0t*V!iy*`j$8(2QsZq|CA zX``;Ue^9H_tf7bxf=%546g>j!i=S`fuH~{P8#YiDWrjR2bLwF%WOvBLYwYRWaCS4p zW-=ljKhHl-qqd5qa&4_^6uawhFawc%#x$ut!-2?P=l5iP&1+EK_l4egw8>nI-m=Q0jzesTf8$i8dR>V35VWW=kWVllj5Lboj8ifbnfiCL!*SVQLmEr=S#5#+u+k9B$K3XS7ijQn!K$0+uyUrZ>kc*%GD`4|5))`-Oao7WzPcqf0C&hSJfYfUrscw&c5#`)0)-U?5-dP ze3YsBk>@S>t<=8NyT_bo-Q!Jh_+5{^6Uuc%z)nWZr*7h~NB4b+vypaZQfS5%$_KD8 z&Jm3#2yVUIdIw};K*PTD8?x^MM2+G_qF0j5)Z2Zu1CF^#bu-f@Sq;W8CQM~RIwnn1 zGniut##D%-rYFA~`E%gJGSfiIIi#&LU+kLO386v=ZN+24SbPp;s zH!Bg5xV#SV+TD5x_I3KgPLhGN8o|L1z%tKr&6o+~qC&99znxS=^L&^hn9I7x->VMJ z-}(APDvLqkb2YZPulmszIU~2asz(@oK>9;-$yeX9 zsnjxWJW{VX0bcfMUsq8{O?rD1@xg_h*`Ly$hGn6{5S1nhG z1o+!hZn={A_(nEdB$?}1>xfd!e$U9PROQq2a)NY7z}D5 zSt|_;E^GO|=V6CtQKa zIsWn#>ymagGX8c;{JTWrI1O*j-_|k@jDM@F7F2ijn$Wa_1W8a5!#bn7!Y4hdoOgtz z?u~oqyd!p4h!V&dzZ&KV^=oBVKE5lAWLPv<1WAAi;gr9sQ$N(`P$#^3r(6dy>M@<{ zv?XoKu#Ckn$&wPTB*hzXTeqRfbyR3m`+m-M%Z9+#7v_*&RBIGTuyk`YCIn+ZO&RwyvFsCteEmeLsp98&jqYTxc@n2Tz0TJjhol9dp{ ziCVj?AaBusmi)^Zh~+i1(|Lkqy&#Kn;V6pJtJyl=oE~f3fj}othvZ|c2?P7fBqLhh zV?*7PCs=lJy$=`aU6AtX_!WNM%fSfZc~Mj7=1%$z%(>$bRV_i6qjwUIh2>X#XoZEB zjB;iFpb7u0dp??~NUrSHS7ncs%0dGfi=OW;id688X2kB#_B`N+`JQAVWNW;0=bx|@ zJ@m9tBf=6HJ|5c+_2$TZV$L}KDIQi!ht|2L4l6NkAwN%vD5Th^BJqDl*yH! z68_R$TFhWa#E1=O-#G8r$3=-iuUp>_*4;OTho`(a(XW4a7HF<#Zs-InL}S7G`j?Sy zyTgw?QocziS)PYo*wy#slHZfkviHKlkRpEc2Lt^3DS=0*w_@ zHhEWsWF>&;!#TvXFz-AQ_1{**Y-9c#w^@l$7I*v)ZgZ~fNx4xQL|_8W`gL;^aM$WC z%C)-Nx=Mr6{Z*;d!xSf46YWpom0=UK5q?{B{;JrQ<-!$o%v1aW@9@Ja1rQ@%= zM6_nV_Lh-?DL}wW1m4X+Er)%H$Lr?6!?y7GLQ_s2lde2rD~n$F>XDf0>hYwD?ZA~h ze7H&(rdO#fNbp*{IwWFVK{hlHf5*LS@Ze{fNd?#9>%)*3G*Tz>vWNz*;<~N7N7a8t zXX)BwQ}G__Pw`?TQaCXPw!2LQ|4AwG(3Q2s*V%oHx6yPE$yA-O@T|6BSjW>3AATA( zT}lt`Fvu^k&$01eT&2(IT#!v~s6{NR`+oN+-H?u8$)ipUIDR|kAveSA<(YDvmRdcW zQ*8pOifqHLcq_cse@qF|m1LtjnCbT;o_Ema&cL*ESpb4X9Wt+!2d*0qj4QkXqmKtR zN1!i_K+@qfR#mAO6vpw{xD62YTq`T{sNyv9NC)Z~QAq?>kj&5}Q1jKro1gMv^~Lt4B@GTJ=#b>s(lkX3^Q>gQ|AlOT zzo%hUfBV%b1GblNe)mI1DT39!jmpi-#S!C+>P#4S> z2~!ZmRCF^URZMIX{!j;Ur9Oh~$ZSgd)o9Ffg%Qp?LJ0et+-YRhU$8mzoBT0@Yk%zg zNx;flw~CMNRAMlvaJkPrQlkuK0K-oGd$JblRoNrLOkCqdy*b)0G1)0vT{ULl2LXmP z#MQK78977TWpoZNQmkUR8<`r=Vql*`oz8oKB(p{D2?r?n`>9Rv7=b2?hGdJ3>2UNh zKmwRYBq8_wKyKpVZPtLvs%LdNNVh%%ZYr(7Iu?LuIiAh{Z)58)(|oTG0H9^;s|{ zrWrIJi@2i2Kwt1!8%p=nDkXleSPI?Jvd{CucA-ZMs&is8 zVf(`FC<%JVl%okgw09YoTQ#3=*Q{Iy5KMOTctgV*17GHzV9r#vF2+agJ8ge9Jfq-) zu6?oQ83{v-Q-7}VE#qYkJ5WoYD|_l-yCN8PjdzWhj(B=j6Vb(glpRtMoGM~Z> zTpr*)k&4&{UmVv~8(#+ALfI7?(GUUAmeJprb2;wITgs5dzH`rh49C6RJXZ#Jd3E3y zF zjD2)Z3~jZ>YI2u{cZWilQ>7B7RCdu$Um@bzSiLiUYCXdS%ja4bLq76qHm0pQws7iG zQzg8;L)LM7mCZ5fEP#&dD{b5Jf|HeQM_NyprwT8{DjgbbK#hO9jy? zj&=&YZ1)hOudO6`e8%*DeHz}ZZcN9P;&e$}x(Ho}Yy>B+Ex)X+;Z!_%8IW>su$FqN zRF0^B5e_>i7h7F*?7jIN7FmJP!E2R1IB&x`h$Z-G>Zw^n6Z{$mxM@jQkQgb!y8FEI7wTt7Ld;Y`xxOd`v(X(E}FgmjlCY|~1;6zKyR zGQXAspKLyTA=FInx6@bIepECs+fNHO!{X;+7^vbVE`pL8h$6ik|Mg?ippf zfzOn|CXJtB-eiR2+L;SLXOfeXcGQT+n=Z)5-#@rH#PvmVX>wE{e0rp|?B(4tU)~0S zST)+=L?N|_wTJC5$Lf{s5WmC@F|6(@8I}Ic)@Dgr>T@XDtwNoiWS7EX&Nb8QMqcx3 zFVok{6a(Nj%AM~qtL%+zHau;HFsM>{yFq(+-!&@g?@z)!RfJ4k8fV`yi3tFH& z^N4gz03BLfVlnTpOLMTj!5C_OS#Wv?k=W3N#@zjsbq8Xo7>eetxD(MVY*Y2AxggmN zK#|A$eSa5uo=Bw)v5gzWXyVKA2vJl8f(muh20*8M!Un?y0JN^vo?SZc;d+u^c%sKhFEg z9Y)~q&3Fst(a9$|^tHI@c|M6+O2N?%FQssw$}a-Vm^PVR)}uYh{n3t(MYZ*Q>_?_U z#!BmpFskaZ&|;$@SKr*-HsoPRNyRNmx)!gzmC$W6OoG^`KbpeLjE}X=9o=6xZ4Oej zo@9f4bt}IW$l;|+T$73QTr@$%>@dQqu^vl?F@R4-{pFb>#+&AG71T2olBcWm4$ppA zaSor970aosEPk38To<{D(iCl2MZ9h-yTv>1i!?zKKBXUaCdu5{uC_on?5E>h1-I^? zByz4ib#is1mgNJN0Rtz*M^X`B%EJCvbaN`U3i{1f|^b49&Aub?SQKt>0ZUfd%~c6znLA)-dO+wcQV=j&ciEm;0vI zADOsE)3%>BaPmA9AoLeWd{PK8JPMmmf1h;g3M0eDPU`ITcxP&9v+aTD4}tMV&-+hE z1~v$Dy1Vo^4f7?2iEHz0C+LZhN>e2-l)zW-&tw0NQt*-`AjFc@%AoS9+e}9=4w6Xw z6SecH>*i^7L;DlmVqn#3r4;{hJHt8>{mu?5nl!opAluDS)J?WVza{x#PagtVfB~&c z{4Ls=(dcClNlg$bNyFWckg~~wbVLLckrE8$6`YfA8O=T%u@Wf49T|WKXE3;1e)9Oo zBHRK7HW7#g#uWyTh4r8~=JEPCDo(34InEXe%{?WZnDNgCsLiD=10Sk*RW*1^0*Oms zhVHpHl6k~iYF*!G)}4zX5y}tHC^L-%4q(wYQbq3dbo8g3eISzuO*>Q z-E>p%ihswx_|Ch9;=F^HM>YLP{&dENx?gy0x~VbFUI@ZaJJ;lxXHPnO$BX_+(Frf- z?CO%nQpbY_E7pF4yh$tb_W)@}#f0KM*k94yaSm~B539BGS>+%@>)G=)Icn@~+) zC|yr48f@HywrJWk<$6R`s}H|HlBG#%n_7gIu_{dk4B*(V3%=nP?{JSSG;!v6ZK5By zJugJ-q25i6f17?(8N}y5{SV{G$K=qGt;$=iGGWn+{ws8cKo`aXj{jF=Hh$QE z#y>8I|N6Vc0vJ0yHW-63!p`^nqvif@w<0cvNfd~MiOdefas>ao^lvN-g@mb##P0uQ zD5zJp*;n^Havpv}Z`ePsL)nBJyHPpb-m$Ca0HX@T;^_jA{Q0n6=OBsMShT|{v+EqM zr2iV-|Jk3>M2ZuM@u~Nq(PUk3?hYujKk`{p7PSu@4d)>M^X45W*dy;B_^m7Xm7m)^ z%*&3~_q-;0xLjsA=piGXq&7dQYQk;m@BGdg@(PO=Ec@C@5)>o5ICiq%WYGFR| zeDQh3m*A|5xetu|9r4nQW4$*@?!Aa(HEk!T&%Kmylt)Hj)Ir{Ppw|I1sKSkC&_z7S zD!vnlYZ5{?c>9A!SOIac;D7gB34>d~3G)UpIPZ~JTGHWM zhM1#AlHswX{Kfe3m2^N-dYrS8)6xF<#@1KC*xsXhhQsj$0&^nmnd^lDES0K3O_u$Jfq#nT+Tw&QP6S<-Ua zeosf3GAr>;GbyH$KBx|W5YkC?hARUZ%LPZ9s8?$RmnLn_EwKot;`Q`kdgA=|sI5l! zeAi)hn#pq1s6QRmA>m5PwR~#PY4D@2ImsK*#wTjmF3QrU;>R@x)&t%|*{z534Ub)4 zJwJ08Fu#sNT+(JL0-Jc&o=J(%EOSPjh#3UZdJKo^&wmiiupJhrAM-@=Ck6hI)k6Ca zCz1znyMAxQZ-;H*rFvKqlIjHn*Ee(&>G16~yfrEvL|srvDHeG2n9uqi2U&Xj)i<`) zW0@QHdn05)X>i~=pXl1|xZS;vnx6&CO7#{j+~)d?JkQ&9v>SAL#?yH`Rn7Iu__EmY z`CYmShLc!-24Ix}^VXh_$-dtlR^_U-xF%)w0ht_hDz;U&gq)8zjb7fBNg4fOm65n} z>(Amo$5XH@1CD{V1)G7om=p8Z_~egsY=0VD)91PiTA6O+RtH~d58l0j$G~^M*Od-UmxO$!X+twu6UX|-54yY8PtYyu8N_Hbz z&wN^Ak{$9WCS;Sm!e&ok0N}Rmxs-pB52C%f>^sxfKZ3&SdQmFRQGw8H; zTHm_H?E$~6>5xV`r1Y(^p`M1iH`d`WF(3_0+HiQ)BVIu-};!n`#W`eh;R>USDXUw32M^iLrRbVX@6qSuhv&wHCSA zg=vL5Q$$3?69IEbk%eNY@zPI_swYVgL{sS+xyfAEa#QkF?&_fXJ?+dH(>#+f3)hjS zl+Vt+Y@l}Vg*7Dy1AB6U$m(Rw$YGiI%S4p!J47J4i)>a&L%VM-`nsu3$g&diy5OAh zp?AnCKPum#IsEKTW)pp<1ESC?5L`wfu^h4X`rP>KV~$+ekNOk`P>E}KSJTB9)$^G- zWj^tEwcZ3yBie+Cf|x_FXR{KSOrjl>A-q9$NG`g7LNA8)b*dDlTRr|ED9^m_Bx&oB zggTxMmD+=*ryVE4DX^vB*_`wf-Q@IZ8o^72i?oi&BSm+Lm_ zh!n5DRys=8D#4-k{*;2)T*d^hXmT+_%MbM=_ENx0DtJAHGAh$m2!c@8ji2-X*_2up z)0IyE)|x#~xK5Xk4RMs=Kt8v%ym%Hh;fHk+Z{i5^Xg(SL9lBcO=JtvEbhfwADpd2^ zm}NVi!;W;c17g>9sw%JR*OFm%+xHo+hlbR38@?132CBe!$sM`J8`Fooi`RWGfGf2=bo5cG|~5FfnaY zWVw#HN80@WVARxcbJW-{h2RseugKZSql0bf^C5pr9Y|Y#g$jvbadwb-*?x=_C|jvH zd(gCz&Bk_3#Q%@mABEGr0O5 zJ0)civdy3fE~bm}4Ol)U(u84Bv?9jFJswLOJ@fw2#cT<%>qX77@ffg!0c@{LKaok-qL-cC z;zNo8Jjx+)hFKNAMH<)m=kK$rYQa^V`lSP!uu)LT7`xi*dX&$@o5OuU{41v&fLYq^ z+IC-Sk>I5goU+Fo4QkjttH?+1arZ=;aX;s3toZ z-amP0vkI!uhft zhPzAxgv;zd440vjm|8#gZEfMP*>*RR9t~} zD)0kWVij^i(mMc@_xVnB!^dOWqC}CS^OrbCQ4P(tW*ICw5I31ldT3%RfSeMZ*i*Xk z#Jbv|PrO)wT4(&aPHz}r9Diev&M_Qq`k6U()xd)p$_<10GAp*}rGk$2Uc=vQC;P9@ zym3dPna41FVoUFrOmfN}ob=Xl!_cSm=Of=kgM=+Xep=n8oxya0(cpw@t*aq_kS4+| zzsm#j;-tHOSaKqVZLDZg<3Qd1!4w`s+j;d#WOcpbot#m~d0V@L#jj5W-LiT(SH!|y zjWF=~pm%(Q-wU|+{1G`*Z^&|p)a@+3m{F1N4R@t*N2SiKb?KEBgAT7Q)$f7H!y>_Vmupd$BPAi@sAmK5dOKCF5tN745+4qGK=_umwY z90onR`WG121$8U!JMi|?mcpVdY)qvh0>j6sv@l#i*0yzqx3*}N>RJ5QDZEpxwVE3B zxs#3s8%Yt$I|-RE0}bVFQ*-1kHflu75&4OUkjtiy*n8icjdo=W&pbOiwr2AHbz^cD zyiCPF2gaEmXug?!9r>pxl}b<6@e4EyuC$cw1hCk`Ecvk8bcwtgb*pd7V&*Xr^y%`6 zqJWUjx&nlI3L&fpFG!Y~TsA2yR*T1S&}*EgkiE6eDE)eQ(jm64GZY*SJBxE$~h&Gf6WeXc&B=`upId zsL@4MoSSf&xtR=-cZJRKCinQ~d1PWo~jgV;V%*T+tavLuzQ*w(4;(J4&O#RHW;(ai$Y&E%?LOt)iC~QS7yRI5UqtWt#W1LL40C^I3cl*-iay7pP zqDA(TSBk-5r_oL7*jK(CHsG?Bs$)obJL9$8Lq16Kj?7Hi9`Kq?)#|$N&iu$IwX=)c z>}0vuAoDz~hxc7%o@N_zshnbO4^6v8^*FDp|1-PrTj?bp$8#9xh;sMKPTJd|_B%cNz zu6p16V4CBgM|$%?V`sv1wAAmj-XR93<{Oh|H~KA1B_Ni!$xKc^;Zh6jPk&@r49rki zELTmw)`u@Ya5P+AV@^*-`%=6`?X&OXaO_+IXI{w{#ILu|QUV7xtI+-FsC-pXI!2rz z_Vlkl>KVMak2mT3Omf4MvKxGT=31Ucscc=dI%fqhuVNQzbe?#TnBE7yv(Y{q{inp| z|5QEP1;V@Onk?5e?y&(ndlbsD0$un+gwKL2Q9mL6OgZK{U!_HeyA7|c%zxDR;`F|H zq_<7k*xoXkw@`NN#@uvGOrpD#Ia*#$xwE+$@hV=iB-pF{MNo-|Y+F&gxu%SInk1ft zBZC@eNC#uO02S*Se*mJtVNU8sZ=zbafsxXlX9v~vWq}O$ZTnM;oJ4u*b{`o%&sn~g z%_7Z!sOVqTr*lR6LFfyM?FqEbM5QUbd3(F9&;#LytDc+k=q98Xn~*a&MD-e9JgqgNSo?&(iaDNsN<8JqoBG#DGT@!Thb_u*(XEWG!*ZX*L5iW!SZ?V zOrkIMz7+u}UQ6UNA$PtWkjpV7Lb!Uj>k_g^HR%9Gz}5->FuPqzU5|*tt%o-^xB|xE zp05dFbH3jl92Z6d4o21Z%tRADyf|)i;>+Xe3^K*_o%y7ym-j)=#3prA*f>k|Q}XKw z-k6d5TS8a+aw-QnuBLT&g>9^4wQUcDy7loT!) zg8+*7d;0^Jy0x#KZO}+>hBvN3vZ?9Fr(}z;t83wmVAh20!?pQ^Kv%cLk5Rx@ZNCz{ z+Mp2r%K#xeX%`|9w!#7vNFOJ(85Yvvb-mcwQ8xGPj^xO=vN1WC7Bo7oOjvnR*vfZY z$ZB&X)YAP=-1qFec;=YYBH#PW*)1jX@gF|+wmB8dp-fhpqGBaEtcU^Pt(m-@oi={3a!i zeV41hL?tUgF~+^rEd12XMlWsC=AEKs`?mzsA7k&m`Y?CM-kgaWP zM72G(Gu^U#X<#j*b7|zZ@>3`7TbtCwRDdo-!d+OzUHPpZRCN@c5+UVU?rxAiX=xri zr&8Oef&-;7c)_PprBtO6Lc6u}Wt^OIrM=s0u>{_|o0o{^I$|6{DBi$gPrjXx^I&@} zQNVz^Xq4#f>->yS9zHKtgW_>ae@GtiSN(J*nhtp*cIMbGnN{=pK|Qq#Yt|N%fsK7U zHomXFXC<$ag$4Ghnv$6Ti%8*^4$oIhqfB03_2%bWBgAjr!o(jFwTdqGxWSAN>Kffo zb=i-7O0}p&;42Z$s(Eig?jye%3Y=ZYKlPg`BOfZ z$eGt-yTtiE_AFzXHpyaZ8kf6DyH<{SOXeP&!?4+5w=r~pG%9EmHq7NmpYZFiX4RZb zN&9k`G{PbtGdI7JJ}$CZzUCYOHnqJm4Rz<;5-j6 z_KkgoI=>FngA39jAzOhF(P}#KOv!4NpGW2f6Tfq4Y^tlgc}d+g^ln%DWYj`b|3tW0 z)%x!QQga-X7IY;h+AA^bSoD2-cl@G%#yZOykr-fM*@Nf9;@(w4hSQH#^GwO$*CtY9 zs>qAPmk0c6uN4$s;_CdhKy{7T?p*02Q+K@u?6+-g%{WIED*LLoixy35du*2-cNjc? z%_Wl=tv}7a#DNJ}34OnJPm$bFQrFo@YPbs*u(#eB5O?IWi~*aE4?%e_D3jb-16?8$ zUi(yg<$Kxs@i4AL)#I}z4Wn$`@VWPr&m4jTOcWuT1WADYih(@NrO|k6Q!P8DgCSEDV zAF%RLK;F7-6l)$cPrQ(A@Z@T6RZnTg@v*X7kypVu5TIpxvZGg;ph3<8GD05;^nQmjNhqy_xkGhrXLWD&R6FCJ@Z z8e5TZBzUio9GK8;#1Y;hn9()+77vnUzd3@Wx67$hR?#8{FPO-Bo`O zU172aC3SV$n9{4L5uZ8bNv@u6J?%z9YPp~-f<^LLa)#o0 z8$0*+#QB`jUj=cS?UjUEVa>*!==Szg_0htwk$Kwl3imA^zNn&T^V-rmj>1Xzrz()mj1hW=`!2M(v43CB1_*6*)(aiJB)|=y>Km z_V@DToHEMSvviOcuXUFMq!(W4$G*y-QQ`or5-xIJ=l*4uft zLxPWjRl29%>ix8i#l$m3X{8;WloL#-uC%3hSeUIb6#VFG$Ya4Wi~ z*B`y_CQeyas$)0}X!u-gsuN>C^(bEu+*CGdLwseaQS*K-Uqs#?FCKr1C(>@-oVTk~ z?u0pEbgyvvyt5U1Z?hGLn(l%92cN6cLtEq9VZoXEklFW=6e*s?%j-Nu;2a#IV*Dv~ zKG}pUI6;vzR@chmctg( zv6JQ^8=Pp?*Wn`FmHL7A)maI**uN#P*~GK34eQrk9eq&eHV6{oKvCe@ITD#)@1}L& z&7Z2@X4z;dN~ezLf~4fs_-(ts-TP>6dK|x;{3)jMgNhe`m`KOIL!Kab{_Wm{>S~pb zHGr8)=aCyy+OUZyk|sc!XAg*!@!-9qHH(*h+OoZ}U*39z)B**I8Ti~X`2~=s*0%@4 z_x-&e_M;?C@rf1$m>p9zsp+PNaRz9|(mVAb+xB`3cSpPEd1)IlR#hu=5mINy9lLhL z#gx1)M{&ePTh65Ni4Souz+wRPhjVinPB0BERe0dS2NdAPO-0#peU=8M<>VE_I=7YK?In>7Kj)!>mW0E8U>Xpk4$vewJSeVzCAhd0ut}GdC!ie;rV?MmHnn-&`Z>ODnF5`;;asN(D2ae&r-<}(qRfP^CNXH%GXUa81>9<|G3+uD6 z2rt+z;VsLhPa!3%Rq_(QZVYie9rC$i1RQ8NQZ^Xj;Zr?|Qjm#=0r!mMpF@KXY4Y&}EsD zcI-K=6H>QKT20PtGz3KLdiXq`^c)-$jO$k z`76^?36qglLw(bqKl$7SBY8Phvg6flYx`~!TM8|_U1x&c%A*U4WJ_Gp8dya>D(hOUmib$(g~ zITD#Hus<^V_&I$<>Nk&l0&>4hkH>J<+ct4*fj2#Idk-g5p+oX zLUq63(jr$(uacbSMCX8NVM++`n3a`gTiG*x{+tzriA)+Ew_J6H{Z7V1PrvD`eu)=4 zTM8Y{+*4qO(wzHSK&aYooPlX5g5BQ};Mjz>mgLUKyHrmX*(R4;mwJ- zh5}(~IFk=3*q$tvl_9Za=lhpB0tuQ>=|x>;$OvL;K%HXDov3}gRkju-z-15d+QRVs;x`b{QPe%+}@{D!yhY`k;`~8!ua@&i% zS~E~A84xMT-lN0A;-kq*i1HYGD()Xzr3&;a(-YrsG?VVpDRsB7W>(H;tSfTnsqyl4 z+TY#&l7L9Wm4Up;S5WccX0u(e_$0V0k0*Esl$Lg^Le7~k4(3%=^Dpj_)+=utd<+yN>%Vsh1>Ln zE`35WEw2{mxBqnUmczEhi4+2$9#69aE6vT#<))P5nWV=?P2}>!TNfCYXq4IgF)8z} zWhhuQ@0OnRTdcer@(&EXYg_2$U5-#{wS&eI)I+urL{v(`rBKJF_x4i_b_)^qSS} z6pfrl`2Qe7S)gmBpR`juGM~CwTAckXPIBA9(8-)9KP|w_=aDUG@%mF8L@T$Xi;0v& z!7knOyL(oPO$?J`-o!>no+4&1|9GQQ_>0T?&?BzOd=(Chm61t5o2nK}YCswOgZ9>0m68#5$zJPM=VxAyM@S_QZ{Nj8TY+uUUcr6of6kaPYFp_7CTD0j$5Rk(3@KUFJ(}jag&P zcFj9#1d4v=qLgw0GI{$brH2r>OzqW=n4C|1 zxuO4Ch4cS?vDUZqtGVUcn4vkXpzCvY%>D^3y_CfDO?rBj{7-gsjB5S&*t_y6Xrsad zX;z@2SR6b8eB9X8Gq09B{6=H+#lQh8H-S@4bVe9h za>idj(3Ma5ziW2RWEd59r2|(iaOMshZ*2E{*(7XSUs*Xb^*?RMzb3=#{ypMvUDrHq z<^~B~y&M}~FRT|)ui9Aw2&eAuQ7nn){TPPmSY)>pM`c^zH*^|8KB+)73`Q_6yMzLtM zURDe!gNdXR7Tk6ePGAd9_FqBFZu>XxJ_9tW96tGTL6lQ2y!&rYOcfr@LS?BwE5eocWesLH2|E-er?>-YN97Fo1+fdCq9#jX5A2}B0OGQ!h KWz`F-i2nhZQ^sNd literal 218215 zcmZ^~18`*Dwm2M{6Wg|Jn-kl%Z9ADb>1cw9olKI6ZQHi3FTZ==_x`u)&gu;H4x*m4Sf3J-)u=P#|A3h37JdKtOQ9mLejGQX(RRica=s zmNuq9K$7s4>h7v4!x(?6Ndpr?zC{G2Ol47aNM8m8sT$ISkR~V^L4l!pYr26i21S97 zMpZ?%oSeW6X(83tQ@SnIl|~2E(WBdxz8!?_~Ja*V$tUa#t+#ZYpS**b9 z-RS7R5-O;bM#MuMC`ilgviL#5Eir;f5`y)Bcq;1Xz`dD!I)iE!I?X5E+yl%$H}dJe zVL$=baL(lrQ6n7?y|DldBafhk0yUWRX^&B+sI;IM>VkiZxRQV|FJr_jgItdRwkfNV zia`X<^9C-h1p^U72ckf|ej_$Szv)FLIwM7qGKH!ZTGFRPCX922-&XSUoJC#`MY3V} zIG{MS*B~6ie4s)yf!__(D9z79r>09dgfT1A`vA!4Rejr+`97vNnnX#wJFSATZ>utK zX8}`>#7{Vyjuc&k&?E6vwGXuAdqIq>aC)6R26{-I4hZ7*BC(JKYmYb(r%FaLDr-a~ zI82D@gGWsRBef$E1x@iF;>%#YX)8+Uec;=cGwDt#kq$jQl#C$nOn|w$bZ;S&xO32X6{?IdrJiY_;($sf9r9*I8B~dpC_5NUH4xDc z{~jGmMPC_$YrD{-KFPkqTG1FuT8C zQh0<~`UAfIs-NJTsWSEU|L~sK_w4}n+vSONPg80gOpa)&T zl!43|(AI_|HlgP=IDR3)ijWf9q}hSd4p>#acn$2_TUd8pI7hvVF@d#A^kxAWMr!Gx z2SJ{O9~zkG!TrW%8r-R&!;EC?klHan4ELABZE)0L-$U@a<&6$&+0WP?fKGe#j0hY+ z%(s8HFxkb%cXwP7@?tCWm27Ex@ax4ebf@i}UQ_!p=tTqi7HE_6a=SVc?dl&e# z{K%C9$mxr3%5Eyy7d1~4OnYO&g#q^lZVyJ7LScr+d>eom&>MgrKof`D&B+YQEX$16 zYSGHfl(kLN3SGf(+_h4l%PqYxvo6yvdo3OO6+LHOR;{?1zmexB)gjUmUpC%g_`-ye zu{;(!c0cAjHapffrjCt`eSp1zjlx!h4a>S3Tfz zTbj289F9ds8R*uunY3QCa&&{*6SWn!Z{yPIBcswrjT0vo^V-%pr>Lj) z6{i(Ews*D#Hc_@#HWw>(wxBk*w&ga}%hTuWXHIAAtGZ2t=j-QW4?iB#a2+5KA*HB&>sCnH!*w)-;+vY>QK&~gjCb3MMPAn(37srx-F8o#K zHQ*VS#Z$$ViI#)Z!SrGn;~PUEz^kvzjuMa+C5PWlTPv z;549*8DgSy8_<8ex)7T6=$S1XAon?1$K;8R;Y^tBQBAjoB#etEP-O9~A<_hao$c zHfNh<+;VoPGyWKSW}7WOJA<*_tWWf_%z-;?ghq`wk+b1Mk3I$$9ku?Qm*dDz7}+80Kbr!pfpEk}e#>DX*-n z`n4p#sn_s+9k+*P?tk@??^UurQI`X78+L2FjN4q-IoH(f7_^v-!fqGWoNaW!*%%cF z`_Sl|blhxuZ+|z1+zPnqngz`G|8``2F_fmqu zb3W8UA_xBmKHMMceYC33OoAr<7*E?V+tX+G`U9n(!V70VcAcVLp5H2S!R2tSNM$jT zg1tF6j^Fn}_9;WfJa{6P^U(d`$P;iYLGPhk+IHkv+tlB&e^q_aiEnprQQ5KQS@u!) zxF6P3YdcsCro-jQc6X9tt+Yt52XN>(7oUmM=AhWL=_LC-@5%G{5WN}YciL9JMSpI0 z%vb4+_85Di{%-g9`r#J;V2mhC(7-F};c=LFHC-s}HNucp_?QZSidD0*JV}6FEtYkNIN9=Rdv>HzD-0S?|cNs_3<40CI zhp)ZNCRY3E`^M?}G9nd0?%nv)&{NAt@_I?wf1gC|9I0jxV>H`wLB~1zM2oAUn1ULyDsE-xIMragDy7{qfO%C2-=T85sK(^1>^H%{BZD ze;pi1%Z~Ud{zgDx!LsDNfKKfzsDO5m)N}>{LMQv@3oNBfdIbao46szyaM6&H+8$ByMBQZY=At52JlZhF(vZ(m~fPeksBermHao}cPaCdj7cW0rucQR*S z;^N|BU}R=sW~TdELFepg=VItVXXi}v?@j*yeMC*2jh!qVTrBPF2>;pF(8%7^g^!r{ zpM(DQ^KU#&JuLsNXp#O2}e?WN|{vqK%NcuOw{x$o>Fn$4f%tK zj0~1TmXa}_Lk<={B(Pj|WF+9f3`{tX+HOfv%ZLD|zYIZ1LE+#F(7(VT1qzWQlrodW z)|7rr35xu0=q?9xMJbC?ah+1~F^7Y4YmEOsJC;4G1yee~l@9 z5V8=65?WLqos)}%+e8sCQNh0qFKBpqenyFI1p_XEK$QPJ>T4#@2!sFzUS8CJ8Z^#^ zJxjhsr3}>@Baaj9`7eZj(E|j=kBSWkX%z$wbLQ>kl}e{gF8db*zfe{r3)~kDM#0D~ zWa&(l*7cJiPp;Ld7!gI2{R^g|1|Hv_LLn&VF+u#86+v3&nr}h4Zgdi9-3RJkf29)!bSrEh;y9J0gluyH7 zWcm-It)OTDg}`iq@VoYRoIv?lOe-ScM??Id#QGsQguzWz38BM-XGZ%){Y@aMQiM8M1tWPd(5AEQvkQejZXx&;9ibLTN#C zc;9HbCFVS4-qn@opJX3;t!LkHI81{Gw_kk+RiXa!2EjA~Q!5bz4U%vf5;x@#g0}Y- zu46jt#YNRr<-yWvUS@7@FWp)4%hxvyj2)%gm%@uS;DGERl9)i;kZRjJOKVnRC2I=oMO0A>Zy`B>I z&9VFHD|jcQHUF}^8dD(H_(RgF^!Q)})~tQ!T|t>ylbw+9j?Bw< z+dX$*^8B3icR5^VU%4T1ZFF5?QP3W>{>&%C{{<(s7Hw+%zT2i%ojzAFO!z9KA|&=-#vu$P zuqN%#UYHo@2aUbL&MU6(nmT3lc z$C1TNF!c2ni);6j1|#u7to&19^2br;((lGdrHoun#0~durf`18+xdK|K1f)RyGLrw zw!{gO&dxlPA4G}WP2I5O5rXGXWbso4hCh>(UjAY^Y978t0DK>oS+kDc++Qy)?ABsq zW6#u22BbLFd`6WBmJiM;@a5=j%jL+E_2^S@ZrtG0*-1t2yNw~jO=5t*<=2A7nIL8I zHuQB&GubPV^J-nb_C+0jy=^sYi8p4KZyz=<@R4cpKCjomMUvi%5Ob;e69q_`Gn`;mH4GJ6QkY|^j4wcSt zxO>>aV@sJ!Np6?Rqx5fw@p0DLZ}b$ouj`>$n$UFn3`xkNgSJ<)^h z9=xa+>1{-Fcu3ecx12UAr*g6(5D52vQ(YF(EdtC3@mEfCkp_0&1@E~6d|o4mW(wZ) z;(P7G;#g3l9df2m{n$EAN`nGi_1rbU3o zhBD!jNt?pOV_TN8wr@L2&M}P9dLZC{0f*6sXme$-h){r!G}g~Kq?b>4HjK`?9b(~^ z!E_Yo0|IcAi1C_zBsgE$S;0hKgykfk=SyHUzmGPckfP2LYdh0@#V^Gg!3r}W5f&po z;;;Lf4E(evj@s`mGhh1V4|0{q=W+OMdT%-zQ|@<- zlEX)jA$YxkN0zf-hdE(A^C4}D1jp12sB}1HRx8OvVPnWWh%-EF4O*1`Zdi0T6YGwr z($HJ|=WcaC`^#HO zNBm#d6%jzPRXwmzH!SL63j11%p{A2stuA1e`r5eK({@!z|2~y94<71`kFDhbV$-bC)C64PeL!t079Dz({ z0l#e!g~+ZpNSX+_sqb}E_tNxx#e+Q!{?ODQP%cb{(Frt^WDW-^x$-(rziBUOn540N zAM)z1`Y-6-5!aY$ur7hYZ1X}1jUF*Uz8$=TG2qP)V8S4_g=r{srNH~G+^SSSB72l& zFYbEsc}h@U{dR%^*+>yBWgx_=*(Zy17=aKCYPv8GrB^?ECULHv$YS^$e2n*Se{fNS zFkkEvg)2otWU%2evn;60XP=OTa}-d;s|#y8L}OG3vHes8x^iA741)(1ho=Myd~|7U zX1zKw%D)fMRJ%NiNc+#!!Lynucr1lcU6!mkZq`-yCK0r zD}=Dr+Tv+PtP{t7#-DCvkGoEfM}-~C&zt;c;+d1XF2Aj#sX}F@D!I+o=y8-{8*JL$((2GkGc?8E~=0{(dzR&hUd5!AUj=r6)mrIF&xd{m| zSQFefBeW6oEom^T@K2&@wglW%I8PSHw-HPBo4OA+%t8_)zc4YXrcak%=uTNaKbeE+toLR6ho=p0w%Rb(0ufFu6JCso zFSf%)ARM8yNg^^4XoCx6MN^>Eid}YmBv+fG)7kKz?d{e@TAs3!rGhVuS`2ML(l%#^ z6+?x>#1Q)od=HUIAP9+cFdAwuD zS$f%hkjyvaKi?(XHe~zOKw?OU%t$@A0xx}5Z@}DG{Je!hB;(H&|=@HA#5*21!&^`+7`+B{K6E#VlMAN_k2F5qJYO?rxV%| z&;N3yqQGYwILWyiqf{vV-9O&V)%E>=qjS4xR6(s@fb>KE^IktYv_@Uu_nzmnJ-D!` z>7k;2LYnj&+Kr}f6{_-od3pvK$do)TMvW?Y{WADNoA{reFYJdpHKl(nrKsCTcNxm; zNAOot632K+zjolZN7$x)@2BsU*^^h(4J+-UQ=eY&~Pk(y4QoSsTi zW!A0z#l!dtOrMs`0dluVQ+Au{>#=S5`Q4d0O9yI*vu-7zmJr6P-9^B*u>3Zqg&XmW&)h6Qq5*fjXfS88M@8RqkImaanikos)lgt1UwdPK76J-YlTHkPI8Ygdh?LwB|U1eSBJS^pgd+@a~f`q~K z3Bp5the&M{Y=+u#f9Gqt-&WFp5?unRHgXHIVShcVpG@*E=Iye5RHOdkb>6txv}~3I z!%IhMURGXyis2t^;q6kOBG2_R^BG$_2Gh_yQXH|GN#Tz$|Lc$D?QD7O9KWqV$TdEs z{TTMSnHgr&k(5Xj3N>Rf;WbDCpQ)lQXcTgr=5@FElpY-W_CP9l5cFn14jx_Pk+yxu zJp#{J0P;8^6<*ZV$Ll47CepO9qn)r_=lg^2L^-pdSN27IIR44mnT~mmtKvIEx)$4G zuNI!!N`s~SpJ#H88XEFLg8{5JiFo{ui{GbJXUf{;SP1jpaGIZ?7*Hy`jRBS_hkpi2Urakc66`m)pmky13nH84o$ke4-(_s zqI7Fli4KD96WjjA6XB<|jh~m7m&Kua6Kj5~=s2b|22H5bqOjbHz5i=gXud=9nv+bu zCl$s=$KyuZ%?J%v4+@>m8WgT9UeCi~=f^#){u79kNGobsk^qPt09*;ks^fma*hrOE z9X~uSF0S+Q{c;U|EcPZ+!N&+Z_vJ@gGaI9+&7YbMYRcjr!1(w$MXm=i^@cQE8(YF; zO^04?Ufad*(PAtR5axGtf!AfUyzZDG1%Y=~+153z zMK^y5IJJHDK0FTq?J*mri zqqo07x*K0kfv#cMjn>ZgRWz2#Dd$s1eMtz zg^z}`>E;;uOgcX2Ho_t9JxqGB@OsLyG+KIYPf#|p-8-xat^fe6ISxW4S)s?XnGvz#454s1oH!@LU)LpMnsSVxn&4oRBm8It75Eol zdd48`ra<)O0Q4l_@3^}CwQ0| zpvuI@etrgL+n0;jK%|S z3s|+&NN;cV)&9OKmL;)xJWWr(TgqypBVPor`%o}RTCzyJpqNe{J3l*ep!|c*TMu)q zmSuBd^mvNkVXwZlSRt1;!zrei@otTmbMs}P*}{(#4H0qy?DJ)@vqdBQbI|`Syu4DG z8`t$^i#o5?=)B;2LM_`Z>RQ#EiLz`eA8tF5rB%Vz%}iHv!P}s~M_~R`W`_U1l>_t@ zx@$F$@_}vSb+oGs;zgtyC)+RJ_VQtq&SE{S?OFJr5pdvq4@<_+0V{)X>SQkLIoo;QMd9s3NEf{a zT9j;HmIPAWr-sD68WkNZjl+Ht6JAFvkGVdqN|Uq*t#Irxo}t~cI7^35A`U~BDzTBB zKKpw%wlkN~iGD4ZfB*3f#Eo9=R*J(rReY{uSp z=(ogppS@h~3#M1kIOTV`SlXfP@=M+81ezXNPx$(#W>X#nZMcqSr zp@1KBtPU`lg-hqVlz-f;~NB53?qvC~_LM!*G5V9te6m`rZ({E)61Z z?Ii>Ov0~D*=KsbTXD^h+VsPH{eVnH3A+Y1v^kHd6;q_6yUz7QcR1ZAtHx6}Pa{kBhhH=yKkY&>?%)o380SrlqNw5D=& zE6hPsw3H(xNo3&>36=TJq$JdDICB4&E7SrZ+He;^fI6o>sPE8oD-pCMf0<}SEt`ke z<9qI^r9DIP&)^MfX%Cq+FT_7vx{nI>UNot*e!7)|eel_2=s1fyog1YG{27YJj)f7= ztXeqn4{$#WL~I`r$Yve!erUmeHU=h>>Nnz!i?!? zIL!4_pn1EarD$D$cR+>3PLZrY69f*M z9*;!6maS?FNz=ZcPsnzGK9Xn-$rl%7FaQsN8lowafw)H<^P>YOdUm_Dpn3Z)!Id5fQlGfJR@ z9h5#T6N|OgIfE{XM=Jru(upO3re{`NL3v*oLF98$-#@1(6mxu-;?M$3k zlO9^QhAB>>)wwbiITAs~E}xuV@y{$ve)Z$*y}H?%;5EK?rO#6^Y%-S?j8@d>GoClV zaRcYGwvP?b`0$fNVJ{eCG(wLIe78RFg&!s{Fx~2ZZqR0kLgW4J3hx*-Mp%T?g)?H=t3JUF6rw{a?dG!FeeOt=nja==JIlRb*RRl z6&sBRrk3VK^}d~m7AZ&5K|4!{tN7gvQ_#slz@9a&+8x9do7ue~(?*O+Sq>=9yS*i% z*|lGbj|}bneovuNq12zN+L>F>BXD=+9MJ_2ui@^d?c9r!$*8-4rn2`2>cL5PtrL8`hsH=&%@@4+@OyEB)yM`RKjv=wO41J(aKY6nEjSc zBuoW0d8kpFzt;r31Tbck|CJ_0Mfs|GT#=LAovav-epO5+CoErAcg#|KBf^vE7;}OM z4o5Qq9>8)$Zw#G81PSvFAcOmQaK%?v5(*Z2=0?3U>R1HGYVR5lEcQ^yFz{T};btBR zRpqrGhn-L3XR*z9D=Wc^8~1v8TOG3({mC0k+%L=#DFj;GhcqaoX2;V{cjQ?P#QbJ+ z=hO*~Kk$|Fq_%`-REyv3*_5m>=lIJCTMET_{DlZE)|3v>Do3W;^8o;U++WNgE`|sXnCE+$91@R=GIWmFpk~?qraJs-B|lJ6 zE1d@78$5mbQxoSsd)Mq;>?DF7pcCYJmQ3rGbi{^M)!(lL+MDosq(&dUhJ#AEt_cQ# z-R;nF>a+s9tJ3n0nKw~{HV0_IIsTbHye-@lt6amPtWfDwtsKgIJH~RJEBd3RT}j>8 ztI{GjC+jt}oAxW#Lm&RllG;|#*t4md$t8V&HS}e5sU97K4Eze>-ml~RO~`m{ly%lV z!rX_GKeCo$~JyyGH~sY;n+FY9d4Z+=D6cq)^RB~ z=OL(M!buepR?vwk zk1)P!l%Qo-VY+nQe<(`fSkN0c`AeFZG*M(scl838kr2%lp$@}Hfj*N3a7R1OpEWVl z&7lPjV0-L{7M5n1tyG;UPE`zs13quAG&cuRiI#J8xqt5mU4Gm=%L$wdaIris`q)dY zl*f8+^F|NBdKQN3q>`~1FUL%xrtp;wvz1NSwpjvXj}{BIB^cSc7c7vt!y4c$1C4tI ziT+s4smWNKJeQ8T)7#CVaGd*iDcL}T2Z=l+Yu)!_=D_Plg^9R@cq zdTTQ2o`cAXz&=wO(V`1uoygN3hS&u293Br1UqOJtRD%?G<>0Uatm3!T3qkWMAFS(* zXe2hm2A%0d0ri%4X?3l@V?})JD=t}ITeubZq;}=BvMGgonjQIBW>%k(D&u^9qwlx> z>>uSE$59upf|W4o0poFJAuGJ0$kGW>mvdiq&(WSUePdw9J)bkuRVW&Hdy>~GbGxL~ z!)r9;DQVx|iif~IFrtS5z9=KWj+ewrL<@(Kw?h~+ALB>@S+lHUXK&`&6%oMxxu2$g zp$d`C@)Shbuy@|t8=Ku%#FhTmvg~GIShsELSrceLPNt$!bdam>edw&5)gLxESO%9u z0_}t`-NW=A`-EGa4{Xh9AzNB!m}?0L<&&q^dxig zSFwJjfV61{$H)}1IQz1)5ecxmY(2ZS*OUl&TZziPeVIfoQ$Sh`*=Y153*b3sSxmgz z=&9wlD|ME}bKb*a+EJmvGHrN~Jlr-iCEw8CMGM~7d^L7X_4TxGz7={7_>M6RPL&(8x&FCRtsHwvORa~7?-Tal zF4<2_Sg_is7+ztt^x;e+Ij~7@)hqU;_SADf2Y=Qqd>i(3DNo>7Nk&f^jnqRHiKx^J z?T2h6kO?G@f`(Vkt)cAJT6(j5+p2L93@_Y@QO$yxIiw&9q8C5UC|r1y5e@59E^jJo z$zURCsxnwn<%J^EOK@VHw2oe_nmCWHhH9_cmKNFB3Px<|!m({Y{-Jb0UyCf*pqCCC z+}7N=t!tQA(W)zhIct5O)jXT8WCRET=XVFWvh14T!yl)+w>cTV5T@p7L8`b|Ol6Kb znAOD)MoeY9C~3S}SHNvgse$c6CbI!aN+e`!* zvvgW&A!|xm3D609zAf9P2 z5m%I2KjiewQmrzTe92Pyd3@(&OrAqt z8K`(D`7H@TI+Eag>%cT}F77Xv9RRHPs~?m0zGJzD=P0yjLyGq<ag&7I#G z56K)r!KPz!9?A~o&xG9RBL+Wvi5`U-Qf%!VmobXQ6vzW=RJ37u&#HV{lXX4f5Xx`p z$kF$Y*XYx0f2MenI;tQI?LV+y&-So5hQ+?`QIMD#;K0Wlh|(fxdaTjHhGxNgre#T7 z9v-bzHvdxrS0%48(_{_V)MxOpP%ECd*W|RFxNAnI*GiKAnS96Y>SFrlA@|jLmEK6w zgHViH40~)-wK};cPO5!+dcy2_UMR63yI~LvuCVE}*jR$h%(o)o7R>#khg08*!Lk0) zJudKh6H;j&mvM6M@?;w=8Ssp8-@1`=aPFzCovoITOb14 zPk;Pr;Te_ZNiQ`%WsBl@Q11I!ABNX65@E$5 zJcyf;MwPNyAwv#38uYhmFR15XiJT)F)wX3GZ16)}nz!3UeO}+N>zz zyoa+lXsJi)i^C;B(vUGhQE7OND73COezrEX&Tc$u#@}@-gLKx>Z|FSx)d((+?Hj@D+z%Y zlnKlaQ93_n~PvycQ|(n~r0C4R)&B@sLNK#k?PV>exMdgvzt<1=l%E!1=$ zL>I$8Aj8>qfrPS-X9t>8?n^y0Uku=jRG_suLxqKcsGKLn)u$0T7POT7fSjs~GsJE2 zv4bXs2uh5@xz|W&FZ-H>2}?($BG1@IJ4^s;tdSWE(CPEnrtl#+^-o6S&?Z zT)R=>KF~;H-gRv*!qL*E7}H_&+eot2S7@fa(-QYuUfyWjK8tTY_Z(X{*V`NSVQ6ZI z-5(12$yMDp2w+PtBe#?ar!fhG0X=}j3d{B8vt-a|pgm+;^di}<9ZqHjY*-cEYtdg^ zSY8{KVsGCe5zOsL>$Tya47o?ZP^v^MLzb=Oa#PY-;aHz5y9MST9y;Z!@)xMpO*6LW zGV)hZV*4oOC~$@U6ttzq?$XVv9CWc9|C`{hOc<8*nDBzrOoF$Dw_}k*L(`~XyIKCIus#qV*05y9|dk-`4Svq@ZlXaGpT( z@ncRG>ELy&rmHySV@M010FUCB+^qMI)x3c89|+Z)6s+lp-qF^(z_3Qk`5^PTj0cna z*c+bcMLIWG7a$TU4j0>sf8uy4ahW6tPj*LE;(jB1XoUOky$GGCw34lQQsic4 zF73$0dM`uknU4UTzdt zX5#TOQTcUvW16FqRTDmxF`^g<4d6cEerWyWG>H*MPsOcWkX`5){eIk#e}jD=|65n5 zzNJZIqCUHj6O;eqPpyVpV03PImz-|)@0{|()X?gh)v|dtY4+uG6(^ZWkbs3rq_^Minsb zP?e|!C>u6s&|?ZCDWQ)r#7?s_j4TSHIH5hNg=69evi4_`wiN8e&VzTTJg73xP-o{S zHuhj^_#ZZIu?eCfl+<{;CDo#ZQH;$Kt4spWCvZqXRV^fmHci7nQ|RAb6Ya z#No3w%iu!thjdru+><{?Rs>uak5DPoDiFj&AR>T=XMn;>dPfvoGJzFd`d)gc!%olQ z%K@;kmMGn28*8S~1zte8eV);gD4%V^}OoPRYJ=$70UFew839HF0SK(9m-5(=NoPivAK|=Hr*zkyt@%1}`Cc*R`;y}D& zZ80EWrIp3AE8&L$edCCy21`75K$VX40Sa?flL?Bp!~FcF>e!Yj7YcmzyRGW zx-e?Q49yUn0Mn;iE_xCG(@7;w7{6fX2&H&%!>o4)DZ{+qqnPBU6Rw;mookylheY>x zhs#>W2;ac{MA*n)vLwWf_$$#EagOHmjGr;^S$bRbO0pOCJqqaI&5aAv8#4VJIU>EO zL?bXRWYsy0DIk})ZN|^T;*#p(;I))$?kNds#NdWHI_@cc_7tpVUrA#2Ea*$7jlCg3 z4ax+N+y?WrN=)DQM5q-SH<1{_eBhXqky8W|xHR|yQ$kq>Xp4$=#9_3OE*qYgATs0? z$yBD5f$YWo@v6lKktg{8TNg4*5hc4xcl4DtOdrQ=U0*4q;z(`oxFVT_=Ow&v_YCrM zUJaZ+bC9h?g!@O+J1OUsuS7b3`oTsV1#_Lr;dseYm=%Z-<(d2kV;S906A^A=v}>=5 zwSUmUVx$%J&A+xPNGlorQ^|nS{^uVJ5m(H*Y~E+!7Q>r2&5IkE0NiO#<_sJ8zO0?= z-=es%=QWAL(R^9srWpZTi|dRN_L!?X@y8x^uQV7jzWt-59T!@YCECGkB#gdFPRtKK3I9|1HH>m^*EVr8&zf9rZ=m2}0{9N0NQG^XTTk(&-DKUE2RMU&Iisj8=0aD!BoK_9G#1kcT{tYoOJ5E(c(<2v2 zj_YKEzzMc5J1|cBGcgae)ADW~Ati~%!k~sN8vN(z>2sy%NJ%fO2sm_7p;^)_hs4Cs zB~7{)G1iF!ny%U@vUB5ej|sH~W43X@Jbkf(2X02lJZTZmKqKT76pX&>bAAC~z@3dj z7F|=y{YzO_)6QartBm%yL~vrp-)*Hs4v;Fz+2e>2r$1_y7_*}n634H<(gk<4U+IEr z%0K7^GjeitsWJnSt8g`ii=mw=R!1Wj_8#uyqo7pDAf{2lB?0V=rh{gWpzC(&Q?t`@ z2CP(ON{t7mtO65w=jXW0O>l0m10%@2+f^&Ah1G>4wJpPj{~vpA6;;==E$rg%?(Po3 z-Q5BN3+@)&-Q9w_1`CAX5+Jxc!Gl9^cjq6hwa-3Fa_`f(Y;S=Fn1|4LE3 z>Vh-WRm7?l{0pim%K;`eODD=z6Knh%W$>dxjt2&D-rA!LTvmzX@X4FB2>+~9103(_ zd~%?pcfdf>#JnFRh^Eqw5Z-=lzyqGsOKI_ak{BcpS8l#1b>3u?qoM0~f;Y}ZuVbiv z$8`WI{ppqiXQ;eQfR-#PYszgkh~)UuXf3xlj~~32cO72tu2v+_B)38I`PXm=Hp>O#xA5_d0#T06+zl^;b}Uh z1pf@-4Vo*gnHOeisFg|)zI>27@$%IBpex5V=*)L@CXRUGV1w&T4DQIhoM_}25?a0B z((r+xJaMUv z1|g6yRs;qaAvev6k|gTv1GV{z`tA)WJy#v%f2hGgK^W!r+L0ma=+9QL%r+ zTal4YI-u~i7*wFZ29~FgmAPvBZI23UK2#_a4Vu8-pD)!K4D%CDkMB856EBS{ijqht zFvwqAukSiiA%1OoeFG3hQeSwkoS z>~+f+s#k^c?i{4MAP{4=GOaYkE8}KzVD|8XZ}qe|o5|zGfqor1omdrNf(@DctH2;P z!r{)f@bCQ=bycUNzlIS6dnn294a3I{@ohuk>GkE}L0luf=aJnQehe6~zIMdn)2hvf zZe_2H;ND6%ht+|A8I=l9^Urzr$1|*g9bTtlxSyjZ zcluCPpHv~|(whbcm0AW#NMSiE@g}45dzc-?qaz-b95uh&wJ;R>K#lJyB(*-k_}UYd z46C*fTbVWBAFIuHuW$GLqK~iIgnN(xO^j{3kS)HEBy+uRVu6LUag1}6@qLV9G~pm! zGV>R=(Ut@Losx_;9l9i(G!xPa$3m*DUqRwa@2A%pwYS=CwA0>vf`05S{PoF~Z>IG?$-$v^HRSU%l6ig>o88Z|;C9hRlbFE~b!1c_Boi+FV2DZ4A5 zaotft-4krN4qolBBKy!W_S~p#)l? zCP|i1Fx|qP(lc`eO0pNyODkF=Hg#`FPx2ZUDp6U_fMQIz6{38`>BH*FO6=5vDVK*| zH#Ybu-AL*YT;k4^C3OgREQYGnR@Oshhs#oAMihp=LP4=U-A9^*qPH|l^JX0W$O@xF zsU!GiH_L8~ESHWg+vRr?%AN83OV0j3)2JRIk^^{HdUr7UgN&i{`CDD7l_gVRcD)dE zgF3{wTpiJBF`2w>vb5K~jr@3a=u>c2$yVWXYVG_lY6lpM3mI3#k-!+x_$ zagT?DQ)5$j()o;r{s#`Gf_d`3l^ogc4t|fD#D&Zrf_bfBjJy}kXc8yp(5gtUjr_Zb z!VCH6K79lgAhV1&`xTVcCUR`3Oa>98;5EWj=&_-tmT;0vwxzlytOxy?r9&qXqqtrt zKk--A%@4qtF~^GhhfwIyXb>$R?*zy-ex|s~O|4W&@xP2H>*v+j%8EzG`=QVusu*2{ zH0nw16bT5RYfFZ91({ZgSLo?^r&yr#GE=!ZA3Guy3vKBn*n3+?)3v2*v=&y<)`*^ZF*(RvuEHxcr(yeDdg}k=wjOnuE&4M z?&y14j`0mdtfLc?RVv$hQ1w{NXsN@iMOY1r;S?4Yjth~Z2qeo9!eU0*rNOz>1vNWf zo=U24n)i&RDT?{KyR7dh%y#IfT!JqpSw3*t)i^Gy#arv=!^|apk^yAjSfQZ>O4W@E zoj+UtOd392{H5eat5m*j^m38=LeMb69bO{|Dl*6f?SxEU_#Dcd2m1bVEhi6d3E#`( zg{4%!k$t0b#u#3Bt>$PG$^GeF#AslxpQ{F9Q7KlaWpZt&gJC2gitlMB^wG+;OMe3K zt^K$YTuH7<)7VmnN*>Gvp}g!`E5ClFxI4v%!-SA6Qb%e^Ezykmp=S4sH$LE@XrF>Q zhVZbk5?Hu*(f4d4E{?c@nxUPpRRumQuYU%aPMG6n~twL zKkjr#>ah21^UL$Y82VG6_WmgZt0b0GY7Opd56%?cro zDCatXdK9QeDV&)1>--^srJ)@eB<#V&ag94na((;V4#*Clf^_0#`wxOrWDu`5@f_<1 za}cDut|-0GPTH#18Kdh@**y3lt?pRq?me6!jB4^ZCiLqV2-Z&&{3srOZvD{X+$7aQ zVc?%Rw|j>}^4RGfp`1H_{Cr`pTBd(BLA>gfjk*1BPLQeapL^7M3mA7aX2e8%DZ*Kp&{`P0fW$ zwhK-UR2Zg~FVSfxdCF$S|Tqp=^BI!l?KdrRr_0swv;u@0S)Gs}^-V;_Lc}b--wgCmx zo8+jdAS~235Shrxh3`dzqN^pu5?GV;sqMv)5e>O+{6||mIL>ihMu!tp<#C^Y7R&ln zmTFDcFivWmQCp-X7D$b|VWdNhu7X-ilf_`$}7K`GAgX0pkoq#Z`;v&Msw5ZI6chHsxWre_A#2k>H zPP63XcCI|Z`w5ER4{wC|_{kh96K_Ms&a5@a`ojCzDYsiCG*%T~Q`w`0H951jPnVkuA+DrN z(F3sy<&%9kIyvf`n87-&vSbg-FFK*+xv&Cf#{@Z7NCIia>lJ%i^0@@unk{cxs$u$X zpI+i`oO>d_CP|G*mXp@JZ`}Ux-dz`FtFpw)x|TwQ4pSzBDY=p@UN1+RPY1!Q+MCS$(;Gh+@K>#K053tWPja zf+ZZl*0B#T?0LETq10c2mo~{lxjbP;#H=xCXb*ZYvkd5e&hi4vkYolVQ8p~p=4@}( z53h|mW!V@dGFMSNeGJsz`LKQcO&MA9JBg}k>r5rUl+zm>ofbO_*K3}3yxGlCw5AiWLKSeJyOClPWCm|x&%P6W_xvm&Cx!F9l zy1EIWPxaE#(7Ksx)7oP^SP>y^IKKS&w&p?WHr~Q6Bjm}{)xrj-m+UAilyRz(P1Ma| zw!(V;q>Ty1FT@@xxkkRS3as46dES@ogmxTb)3_?{j>Fk8rRDT`Q>JN z4AuHV$RZ^;N@Jk8>afZ7UMk9ojlzc@cSvK$9qW`e6oYjfiCKy6t#_|74MTmP z=CRHaCVz5|XARSZ6xyGZqL-S5RaK- z!QiyYWGc%mVkI8v$LASbSB@2g1Bb|WAo-}`IGHcpu-2L!`#am{JP)Qp)|WM~Gjzn73=hvT*7+`_$m$uhhwgQSdXp3YB^v80tY#i=#Rrj^8 zoYTCwDN&z&XkXQyA~-+(*cX)HzFNfW_aB}XU%qqxT55Rva_)o}j{LY&C_{o6Tnt+` zbN2>6KS)5dFEV zB`b?qXRDzHC@}0|i)cXg?nk*qH=!8VUO-H>hgRjFDCc`q$`lK4ik@pCz z=0pXwf9}_7lI6c&RZxEd5oQ zmmB*Ai!~)dyy4hFL7FO*WL%`gu4~|GL*X2BS^${=X(ma+kH=xLGO!n)%4A*j@{36A zpsou)-T0n*+sO66`PEm|ZhHG}5`CV@VCZ#L^W_=+u{rM3ffMq$56N=+f^ne$x65F| z*-**k%?Q%a?`Sd=gRj%{^85{S=c6bQ9hq}`t5=EwpM?Vlq0?GHTtG9SrXi#8y7Cc4 zs2(wgASdEFDJ#`*9AQK&)xq7+)2SZ3yuf_sDoo((VKP1U}5Hka2&F1vh<%yGWGU^(;q zmHSimo7bT@1HCy z9Pu02575FMZ#SCyjU;2q{p;lu^d?2x87lz^sy1sd6+(q>1Rwr0s{$pP7{M2kpq{md zGMm#YnPH@rZ`Y-WajnI_mIgnNvn@1ee*QUrdE3lB>{6iO9#g;09=~n?cb$`Z**crm z$iB@{BWN-b)qXPCHt6M!ebSrQdLOO7bI0&jV+Yy=)bH$mp5x-{^LE>+jkTSaCkrQi zcK7G}FSWm+4t2;C+mGXNsg(}Fis+Yd%er^6w%43mfk2d9Gs&F zQhwLXtf`FzQ&vqr($6p=WNX;2;Ipm}=G1W!+zVW;v#u#@y|lXxLG&Bj3hR-twYWn} zi@#;Z(4o^F0W zeF>rP+}(QSQONA-N+Q3d0R#D{I82Daj9C;s4v3Y*3hb{pRvX0(!Cq*a%qIpJp(Lmf zgCzZ12o{L-5zwj2ezyW)?RR7Eqx09qT~&_$4UhpAbO=?GJUCPt&VUSx%71J1u{G9g zGTx{SMyMktpZwVG^Y@=^d^tA`bCwdI$WAcl(>;lXwdOH>s~>oob<7 z<*du3*zmDGZ%gqda~l^nQsk@H8`inotC4=dR1k}PA;n-lPl2y(X+=^mmfphd8p(X4 z=}71ntl-1h-M zf%M6@PdQ$1z4_|>KW*b51xuyZg5~JmyIOt0H)1p}2Fe}NancCY$0Qav#^^ys?MpnG ze$L?KI(^<1gk6%p%>ridkBw)gGL110XAy)CJFd4Dl~h>cecf+y8IlL%;a#KC+_`>@ zniNQQyEY3PN91Z}gm;|GIC0w#1E<|xK6K*4q))WMqoR0dFBh*wvGs; zSq@CKP76k@pT@%!zdaJChRtFRpkgvIIII0`gLAGP_*f!l%cXM zW82MmcBB}q%bSKfRvTgn2naBbd+i0x5q-N#ey^{8ZN-}kyOG5W(=X2Mu@d~ah;Dl+ zUfQxHbr@jP)B&K=T4tEu+v7bBm}O zi~;YNr{tiDdf#!7VoH)M`)u5<}$(ICoAmkhzVxyiUmyr$Fq7Row7GJ%P#4D(!KEbBEH{zXl5 zPUSS0>+|-?&%=5+jPw8}Ww_snRuvtLT9rkfwV$b-8Sm-t;aGQ0snXnZAKSvqg2mb3 z&zQrx!#E!^wAg%m6h(*yFE}Yh;<4HeeHzvz`F=+nsNwM^gIOf^&>xZptMLlPlgexb z{J+}{95fm}+)wp`8rL}v=)&f?F}+h|Ywc7GzSa*|+Rr1y{x1gEgi%n%HU_m|NZ4~Q zh!~Ff4QpQ2O~g_C8xc6M8$KK0elQF9ok0u>9EF5q zuXYV@J;sHe5`WO7X32E;cN$9Ez!c?`jAEK4Z}}Tq3{b8A0EUJa>Uy`o|C={4NPflE zK`^lGdHfFl^Gd6{f_UUGWE*%#_hr}QH=m3Ij6Oz~IST4MMfcddHue1Y-~J7m0ff<8 z2U0roH6@te8*%YGPq1M|yl|9vxb||UzlEWq1U#edayXNH=1}|kj^=k5G{D#~P{$Ad zLZO}?>HH4p>t4gHfsfUiTDW9%GQSCF{?MP2YhtvZ>!8fn92^|nHXLv}mSMk_0F(e( zh%AYqTI=1ty}iRjaa#);8;alCeo)lX1RzvI!~#VRb?1-eOO`GAVX4Lb4`ckFABBZr z3?}_6fXe4cz|h3xWPqQlsw!JLJEEx9Z}bGPG^O$#ad3!loAb-dqgXM-%@82*&5$zi zYuIFePpCXq(4lEI-wj^ifJyufD+_oGH8F$c3E_zZZE$NPlXLt6F^)mn=->S*z>S@d zvj7p7X$Do|QS2}f}-Ns zLWo9mRxS`6MiD~T=X4{+|H&Eor_2IUk}o0VTpPk#D7fvajtm(l?@x0&t@OKv3Rss} z6N`eXTrGowyoQH-?oLele_*&Ofwcug!Rs>6a}%fPRLAjAg#GrjS5Yd9TKG$-Nrit4 zW2Dk;`rRK1b3n+#BVyI!MdW9&%xsDMF>Uj2>>)^itf<)KIR%ZlYwube+BqY^&0U1-&v)ALep}H=>P~m0Nf6nSuabm z1U&-9Qdf659F@E;mhk8NJdNycR0NAR%q1!NNG6wEXCNefD0#YkLtR}o?wjJWGRt0- zQYv*62vh`_JJu`!Mdx(VnLV90Bu zRRWP9GH&P}sc?f3*b0^WPRBuSxRPwwMPNiXXZV*o_bS#|x_OQvSsz@)lJCP2@DCGA z&BJ20LJP+TU=rcK3;jxf&>6B$G%omH1E8De0P|)KrX+ARO#&ASJ|3s?H&-!$RLOt? z2C?rFKn^|e>}07rqV?0|7pg7+1>9utLN@lQ=+0L zFuqnO_(yjuQ-Bn)Ptv+}P-d~pKe)C4>N}1Hk4}nc?QOBSwXJR6H^W|=G%cjf2eh@w*6IQn#I}dymWmLE%@{8A6zT~) z{|?c#M;!%acE-(O?-c zPIa2>Nqw&3_|FFb{tF(W^5J)Y{iYdkE~kI5a+WXX^9N+#j(0160zsp+wn z6`5q5V+5?r`$JNtfnBe7ojh$4zi?;q>@NPz(J zIe0O1y^eXdz0B7m9I*$_)AEDlk^=Ggvt01aRKWfe;@M%~>sTp7GM4$<65DdC!9{g*mUArzYc1ofHP+ z(>W5pJ)Wy?Dh~ON9n_S8P`8f_m8Viw2*ttZfP?pYI?%87r@G!~L$~pYS{{ ziy;RBV1<}12^>Z_=1Am|f=i{4#&~B8I72+VzFJhJi+_k{&d1>Lu-DF-SXt$&t zg~b@T9oAw!lEzw2FEq2GL;5$#e9aZ(jBv^~J8}DMOPRz(0G&z$cihnAsA-a<>GHg} zaIyVFUKQ?vk{R;vZK@$ho0)+LX<@FWLsh2QUY<7tcU{rrAoo(97L86aH9dL@f74xEQj4b5}6z*bf5s z+d=AUL4+AvZWEzqbAT_&A|93?rAzkOGYnUUP}476(AOu~74Ja=TB8x$_x<5`I3{L! z^fN0;8izF&%P`}xC{bEd@FNTMNXCV-+S*sKzX~$&{adhdHH)<#cA~V~JoibdZ|qv7 z1sOc3>0@hvjgHX^YPRgv3UUQBKAm7BO8|d>g2-9E5iwR~-h%pJ+&EYJ-ROp^-C#VW zK#5|uEP9(JrHIAax5%IFTfuLsd*5)^H8kWn4$+vfK(!iL6Ru$?m z?DpGUz!>9=cC39%*A7`Q^uVE~fM+5SZcyM{*a{PRB!%x+N77j@Mf%76EES$$<=Pf` z*!Laa$PCTr1V`Og!BDf6nXJZ=tYrc>OF#Il!TX&9r{TJmoPQ|PQ4Dj;6a=igZ^k4H zn;5W*%(=0f_$})Qc7ZC)28KHe8Z#>0Lq0!#x@b4aOi#bItp7%3?a+gHn%^m3hyj~- z-!!Y-FI!}CkmT#Gz4mgK`=XUa^2KQJX!UN)EfP5$dAKk|4VPQC(DQJ^L3NJ7 zXD`qW9lNmY{ImDl2QI{;3rrY(xH$l9(_-*DasLq(MsI_wzD&KRE|z>CvxoL39c zbRFnRHYif5(HSraVda)~w@{&U=@uL>2l>a*q?gQ#gWXqWm>jCMz)U)fm4vCfyPy+4 z+1;h#u?A6CvM96woZ4QPR&K%ykVpY|Zwpkwtx|=|xkk^=|MX&Yc9{6C-I3!|{a(rM zUiI6OTnPmg0G|3yJjT9JA? zrJjGVSG``@YbXQgoi!C%zwT~9CeN^+U% zW&4vrKt-nH!ApXrL6z6WWxa((JTLKTd?DP0fe6U4(npYv&OIrbiN8#~OoCRB(9y+k z7pG_%PV??%1G`-q=zL-f^fc-SBt#7RVulH^n}tp#AAUu9>y7GXAXa_1ohcPD2-rA~ zQXoQ$j@2zj3FbDC$;or*0EJfOLb@8Dj6Q&S%t>+7RNj`PiCe>GyQ4wXw1`Iehnc_B@bzt8R;LqduQGwSHj>c}$EBzdHkNO0DsuQUxM0sMQ2?)yn%8Z2>7^4Ai zr)>1m*^=wFt$Ke_IEZ6o_=5AO*%H8fPq6;13FEstWUadSnS@Fa>a*q|Wo_dk%l>Xf zeC-T4Z9h%b>9A3c1eiH`4S4Qo2u+yWg1b=zKnI$Jb7X$nl@c-v%9fiCN;)?TctPTv zcLJ`XY`R(Ts!fe&GqCQk;-oZ%p{?Ft(*Z|P9x)T2{}{u7-W@{|`QeVn6ea>RWKcLg znngs4oEG@04FBL|DH=vZ`P*Qd8XX(IY!BRmR746Kb@Zzh8ytn7SJ`H1-g6>%s7|M6 z?JR??DfyH{YC;VK3XN92H1!Mw3hIW!AKxc^{Obb53!|f2fQo=KcGi(L3djh-FFsob$Q z^{vJC+GMmp{$g$x#1T>%v7Y5LHuwQP${qr)nGTl0g6G}Ggu%>m#XxS58Gjo} zF$83GH`-y=NSdFiLlXQ_La$i@Q#6d$V7NX!>xz$=IO5!`BW>byzb0ZTq8TWPJ9N-t zFUh$t*&91lKGJ|Cc_eL#76wZtW_M}hS>yfbvITeJ|7wd@Aqbci%QKaymaX>#kfcS5O(yk0Y@(c-KtR$sL|1OIsa3w# z!s=B;eBcU!1q6c1oN{+KY80|xU=wk3{hsesR^So`YYefjyTg!`dJ82x$>JMMQ(Jp- zT#q9K(u<{W3c=-9YiTV-XJEfOy+^=fr@>U+Q@UiRL1x103KBy}p(ht>2M-BMW5e$#6X#4w zQ!^h>KqqgOt2~yGM*$;9j#QUcHx4g}p!p0ls1~rklg4UX!$xvwaScRZOP7Pvt-3lu zXjXQ}Z*798BU{YQ@E_Ir=ZUDJ1N$O3aS9CCBo_p6dd+6#^xPiN*FO`pnZUp|Pm6{y z5Y!K7b7^Gfl+#x>k>PEpfqb%3vDa@@GgT=v_1n%ct74O?707JTW1Sy2Uk0raXKsq` zwuAh6weXHTj4(+%V}s@+I^_fh%?bpXIp$&F0>F5E$9bf>YP9^gK<9|M$VSQiLIvGSRRB)|Jhnl9;rl$&65qyfcI;hB2u5eV$BXa&8yE@_$VF)H!HW$%r83yS*RHuFarJ@8t~@HkQ=5 zDUCG@uJDXtsMA-#mKeh&`F9mY389^uN9v_5(h+PyGDl13)n%$fU4_th;m2grg9XFv z$oz{5UM2#~-DUX8QcGRJz!D_BBv}m8r16teYOz0EcVw5}Rk^+?A$lL1F$m#%#<&56^@!Acii0hfFI^|Vd!Luhw62V(kG-gL4dntjamk|creJ(n|5`>s_{ zvnGQXD+LnEIilO0q)m#gg;R1oHRN|rLgah9ZbRu%=$|jkbZTQ13pOa3fUO(B)=%cR z7yV|8W^?r;+s9iHcpB67gf>ggP^pM@1i!D}@9D@eD=1#CGr0eTb~yV5|NZ5{`^JPc z$$CbWs}TXD&=RfvB56~lr(jpQ&|K$cJKu}UU)icXI!$dOE=ZACrAgPSBi658#b1mC zEf^z-MP6AJTl~+pB>_E#HXc3}E&*yW0vPcHxl-x8 z*&JOnA>d|$CQF*46O`dHTN$icE~mVtv~7TiT3$^RbIeo7x%egzst7wTu-p(gT`0{wPhpt0WVF&(bzQuH){p08m+HI;Ww>5(r9 zm0c~L*8g;ON9e`FspiXAah^vG2eX<7b;g3#n);gqAqFnVp{tsFSeS7I~}3+q`X^r$-3Cpkl= zV%jjAJCHdBgdl7RoA!}9^><4QG|E}_U9ef16Im*uOBr>a2V{?Y+-d0G>poLknEp)= zv0hywL%`i>{=^N*Qa(5ydj^aUW@moCmS$kC3CVL--lvGdw{6OI9S#Q~d}#Av;M~`y zNzRpPn%i?rq^pxJh`1_4a1R?JiHx{lXV4L&jswZ_>t#26erJ;HjxMsB>1ee|&UWkAMd`RMZY!T44%_C{3AR z8+V^!UCDDfAy#XJAr^`-_=PlCB#eLVJ2{t+Erkk{x_@(OhEPKP$F@B{n1FNIKdU)- zzVcip|4KKmZB*>FuiAL0p&Oz@f|+}-rDs0-+t z>lLQTciVX%k~pkIc~#5I#M4`pr-3Z59+k*PISVLE88f`UJ#n4{40~f<0ORZuIdzx1 z6CCJ)@N9b70Pl0pQP+I6apR&@xmtRg=hX(v_X=Ye*SW9l&z?Fs#ZwxW7k&746*NAH zRAW~`RX{0VC#V1I*9?R0)29=^b_2^->m7!O&uKA{r?N+T^g5e&JOhNyiD5HkswGS1 zq-^v*=@Oepbw%x!-)1}Mx=$T>pwrV)Q;fMJMr@!R6D@LG(`lwl{PpA>MF`iz;mr%Z z%zX7tjzVsP(YmsURU{DbbmkQJw)9G_cf^)h=;?{iX*<*DN6`Qn*;wywK00P}(Czwz zBXvj#H`K{7w>blO%dd;9M2gU|*?Aw^G*tTyD7tV1v>qT81Gi*2mQ_vz;hEC-Jsi(w z30>iv<)*8>oh)r65rLUC*AxEyZDU>|EUORI&`ySgbsktO(;%BY4kVF*% z0XIu#^h&Q5PBU7ijnO}jOM`UY9Ry;9E>9IpITcFj5pX|7*^J;Bpm~&-egIQ(8-VC| z9v7*6e64g;ay%Et31(O$1`tBG+-sQ!<6vaDZ$#MSn_Iu%pB;2h>a%8-kxo(fN;~>x zLza;y_-hj-Cy+Jg4H(11gjOSdGbOaikM`{)aI^_fg2?wJ!<3r~JWNniZ@V0Uc zBP{j+{{P9E3e>zHAe}TE7)ep<50Pe@oUa{@(9j=WK2Gssberhs*aTDL&C$gP0SKSk(_Lf@MKCJkblYod`%-2p<>ebdCZM z-|&0+-~#t>=JrQN+v+27H0~n3Z#LkpVp`%rsXR-ahec-oNPe zBXsE@08wU+&Z)Wjw|UMGo9F=ZdQGA^(R55o5of=Pmg?ozpd|9KN$F4CFdlv?*MG9I zWx{%C9=mp1Filmdgk#CO1x9SMT2jW^AV0bnzb2aHbQv+nsosdj9IgU2m%AQxrKF*E z{6wh8--s42fx^Ox*?3NNj8d<18z38en1f0bA~J+w@&5I{fQtf>4i?cDdQR87WjL*- zOPB8L_tK1G2X&#sx>R3&RVAp&A1k$b)wJYzl`V?{nEt!w{-$EQa0D8Sp&%XPm{7Hi zKp|A6SBm#S4}m71@8hkFH(e-_(|QPYG1*;2!&}#;0jf9ua7<{y-ofEHF1J$t^vC7& z>a`KCB^?CgbMwHRatiMTa%&kQa%mir#|w!%Mnbi`61GWH6QxSa;kg#uJNG3ftQWPp zPYjuYJ+Bv3^cJpX+I`oe6NxTi)X;J0ea$@LsPg+WfZrtg4L`nC6B*#62%a>b^%x*M zgcsO%8*8iVrs4Yl!Y==Z#X2=tlARpn$lMT}U#P)^?uIa4+mG1E&_*hz=ZT{pL{bfd z%^O51vkQ?$Lp{xZlxfsV9v)y>52K3}v$cyY`FCGYLCwsxoJj8mt~S~@mz{s;;Aa7W zYK3<)&3Y#^@u!Ra0;9Fps?9DVYv11loMPfLjM`pd0OQk)`g_JkN*g{QppXU06{(d} zLzyEG9M|ht+04DC^=)gVadMzix21+F(;B0LC-tz@T!Z;A`}H!Fq9v|a+&54Ph^=yY z^plNttA|8_G@1m7jZ@yyqb{0%r(M&9@OKbd8t`Nqv7LZT`Dbj1qaW zB4U+p_jHT)D;_&vs{8O3q8VtNuy>^4zB%tDDCko_@qo45GzW$Pck5W+U+!s%!PZm} zNN|wf^hDaLj(u5+>%~DsNvU`L+@0rt!3N|Uk?qAKwPD6mb^ z*PxZA)qH_i3v!bmb6xOzc{&4;jdEAnbIj|_J%ZM{B2Yp)c4dR{X zD76}_29iijh{>&R=YkcdPZOaw~!XITe50Y9sWqmspsRrpszFuBR1b!a8!(ZMUuYN$ZY ztS+EWufw23Xcs{D51R1~o39>6?RoeD*JZ8}z(6jzNMPFP~)iKnAK!_lWt`smpK4Oe>WXAhXr0h$3|6~HyG&oR-GYP-uNFGBt!^p;fapwkB36F8akh! zUT>N@ro^h=b==XT!x^N5RB~*N&<9!%*v9TAt+@fWOwI#$fRTj@Eg3Er-^67ZxR7lL zf)NuIw~q6+{2SQTC=tv(aMOyxOYVCrN%Vy`3W>A;_ZABW&HmVODh=`gpPe?rBPvWz zrbQI?sz!-cFZaH~q%QL?Y zxXbKks*?IXoiE&}FSZSE+i7Y*JVQQkPz}6*G+mw#id9;L*d5CX84p=fYne$}N7eH< z-skv=q)GA55RWk1-wG&jeP}H;JH>c_xm%Tm;g=o~usGy8?8c8pDg~Nv_f^hrRg_Rt z!+PP%m?&j=Vf=l8T9ovZ4Nm1@Ls;Et<)}Z@XhZmg*%boNxkMtCn&A9b2>q z#B7=JTDmyEo>@?_NNrAkdARTfOD-=7 z!L+O>MV2aR{aZzVu_bE)km0BRaOmw~Z=hcMngsb+ zBAj8P(TNn^uBmt?d?*T8JMqMZ-wqk5$yNZgJIFD=t)PmSr9w#$f%X8*!n{# z*5|io(S@M`Fh_tMmofBo;KD&(Ht90Wjg2C-K^VUU7dAW4Js$5Zz>Oi5Z&Bvy6a$53 zH$iXAN$Jb5pMc${{S+zHDzT{lrgEf7 z;)KbI(LJ=H6aQ;;+;-(^1mM>4-WVopZ(3S{%QsOV?xhiX)FeI zj4J+m0RHn#?@*K{lp)z`rSn`+Z-J@#ossm!FF*Q;{jy@%|L(8F3n!v!nj9}RD|mv# z19g>*fR3IB^i{xp-inDA1fmmrppQXO67TQcJH^B_us-Y@hQog?#r*mB&Lm(Niq3gq zA5;No+dm##K>=60W?`&UOu>hO(xwx{}Y;p)$3+vRQ;0s`z8Zn1TPA( zbcDNxKh^&6PC#Ku6n|6hAJ5dduAt}yb?;=ilM*S*Tf@DB_vrn0&FpN;#k z!xDfb4m>@y=RD`Tf4}1Y>w3W9@lJF%l>Pe^i>ZJQT?YI9`@i-u4;&SE`T(K^g?~#0 zI|uNg6%^*r|GmJ3;lR@)T~>Yg{O?yR1LVcRsw;*0Ukkh|4D9Cr*IO3Fb%n_AueH|} z3W!iOUlv%us%9QZyIice5qzw>(wN8FM$@z{>xHUZv~*u(Dui+)o~~83%_xr5`9nB; zkpCrV0kiUJ=MPl54x2r*I${Uvw4fA7VRp5%bDa9MQ_=M$8WYNVVsd}UI_}^W+$3SkVmwD=fRwI{@ zLi(Sf&xQNha(1$`CT~+|=>l$lu`-UJDJbkBH>&;LzsuuaOD}DAXH)a_I~2=Zpz5^z zc6yq~f5K3}i{yH|VdbJ$qlr=`HyytrAmkqGy7@nW;14hKrSKJgb{2kf+|+j(giLq8 z`u_v#o=Bh!^6wRc2zb2x4_L0@_3!>GP#~f_kbv-<%|a{npDX((@Vy}P9dG0I9R8cc zr@uhF;COUU`Fvbg&wnYI`Mj>Iv$oTA>t0Qc(X^^h6l(qk zwbybUO0gIl#Pnx@@y%nyvYtFghcC!}sHQ3gFIezo{*~ntXA0&!HPS zXL4Vd9CMCqT9f`KY%=2d8@gGknuastx@j}c%{BXngwqk=aNd=2z^S?VBX1bnebt($0*55<8h7}KQ?&l zalhe>tkdK5^K1wM478Ia^f*;d$$rA_tPfi=8IerSK)pZBr~zMkqz1G;M(>~KKOc(* z<}&Mi5Y-*hNk8p}L#V53=AEj$$12)=SjqyM{~oYcT#x-85P2E0ztNfUyIonowdkPo zZ$s=kPeDaLqoW}f@<%RagTi|Dx^6ug0`4*P!{xDe0cVO^ETB>`5P{BT_1=;IIUu!v zlj!yqLieAt>kVXG`j)q((PKGiXYg+zH?&4mdXD$02&vQn2DJB1CjvivBwaxM?7iqH ziwre+&HQd;_RGVL)udC{WA{)|)9tEeX8^dSZvNM+*k9eStq<93PLn^;c^Y1NR9x{E z>)#@B)<%xV$Q*CAK(5uUkw|(~<*3^m@*E$p_C{4r+Y!a@Y-T*k-PZT}r09A{nz*k6 zpBr4XDyNs$#W`+6PVR_3!2|5gm(AyjkvU#x%;tQtI1QsIfo;j7Cf*+3q-dI;lthi; zr08pxRju6C49usB_0lox3zSxpEH1*PVyr70lq z=PEJ1^i{`tg1B}2;%=ns-bwL%aZtux(c?wadl8r_eEna~h1QeNm+Qo-cL5gMO)M#R zIY>r!r3>O_?BA#E(cjq=9Sy)m!eOX>dZi+aH8$>)B@`C&LYlIm&40jS4lx z;pn)llVUK%^IB;k>r^YJ?Kt=jhtY$ccek9>{NgskOaE?Ioflp8w|Me-`!sYG+u7t& z=!n}~c`I3SXxF5b6dgFVtk#$Paw|pu*@+98QucCJJrzSgBKFY^f6+OMYE;Bh;gyr- z`6kzF)5x5xIAOti!wswRkm7*_N2C0#R%B=fv{UThvR^^o ziF8-xWh3ygf2@k@5_)Z2j$PG#T7DlZwX35N%btL!*u_@ZDwZ#5poMTTDvdpAWU|;L z-S6W@DAy9_Iyy9?>pBWncZcoZbDETil{wgoTTtt#6mYs^-JLqfJ1U;~=v!ZCr^$2_ zy}2)E)^q>R33`y&(YoV6WL%`a&evJmw7h<&RZQ`k+e}yT8sS--fUH2`52zKu5x76= z^6tL;a9gu~Fro*g8rmUn7g$)$O619nR~J2lD;ZBUV_xMUt_EubT1sTI6dI3VIC=RXL6#u44MHb(0^yU`z47;^rA+00J|QvlML{$zqghIwA<^F z5fhXXpELEztc>vQPa$mocFXtGiB!%nonyJL^6*=@U{~Jg%)c{8^2un6PSIoiDwY>< z>#CDiFsLeeuuoQ1Tdhd;D$`0zPMlu(d3_bjr1AdIi`_KrYFoz?q`>)kupmd&zTX~~ z#Bqmv0~T+%;OTqojMZd+lpdza-oU1%RacF=>bG<0Tx(h`b}FV4#nbhZGDPMz(Fxp^ z?=w=yGtzn|EVJ9`27u8yD0M&G=ve`JVHZ+leH#-vI=U(g{4Y@_oK}H{m3fJu*q{ ze~@VUlM(?M^rI^{-Gh=cV?V(o>hO9A#LEFv**0I6v8061G=VOyXt~h8=cSZ+ytUAm zN7{tk>xUNnJ+$t=EJH06w;+U*I~IHoSV?YP3+ckn2K@<4{OfjB&3_1d!Q3!+Su(7{P?EXhSvpSzTuf=iNYcj-N@=Sp}PCV5#(R8I#W79g3-Mc8tZD!zXeqJ<=7t6L^shT3GUbyRM+Cc@*;Bz2CO*V$$6`UcCmK;u8YD zRD0`>_NZ|0sDG^t66;ohQkCm?9kGW&(+BdIO3B{;+KhL3m~z#Fj7+mf`vOL~SWuEe zI@d5`@3PVE7mJGbyZ~?0Fk;iWO2Mf6F)c;0IvuC0le%G)zsbvW^fo#oDVi@nlFUsz zARNUFZ4(I%#$@R{GI|%jVtH{nbiuY?0)u~;DB^TY>fD<(bpXFqnOEyepb1biPj%6c zkJBb^kiag-@bD(ixcjBz++p$(QQ15w`DW83_QIlVq8@X!@^IDlUMOypo$#$5vGX}h z#?R^ESwHz@ykvkG{vM}hiScqBHv<6`@g|?k=tV{fDNIR!KaVwJeBOK>YimxhI#wf0 z9@D8MajYiGT*i7gmT$;HfWBvuAN4Rx=Is852$5@X=uA$0FDT6J;P0i~Vpc-B;MnKv zVhUC?Up4H~42nrA2w|FqNZB)~P3@4s?q`{gD%+s49qmnh`d&rm6`pqx6L5HJ87MGi zk@kc}%P8!6DEjewCnhBk77c0j3s1=)c)~qNyYGeSVd125iN2ohvu_Dwv3gAojhCco zSvp)MJQQN#*}4auUrVQPomFZ$9j_G>*onhMsuwDM8H4AWYC-CMvb;)8-E+#8jQh}V zc`7Q)naGN4)M^?RYiDop$p0y5ss|@g_cQeZu=6!}G`UjPE82fb*KA8_D-3Rbcqh}h zgEfn4FbR76U}gLbhG(o`A8Es4H1ldFBv3l4Tqi{@;z8K;o|ts%JCM(AGF!NIzCdm1 zz|%#8?af|lBc?X8t6$}F$()>AqqG{rW9n^j@a@CEnIjBFY8$NS7`e^d+RkzrR%JA% z`O4>+GT&T!oNsNvc~eH?x{mO!5BqA;6G)su+kKDZ4Xz~QRCApHekffty&%Hxjyan3 zB)VNx$cH!|vGeUH72j8y6>IarcOzx$R+=?}2Cn#h64~kLtVw7A|DdNIs4AE{Os({w z1W5P;#+`nD7cp>zy>)DCbJAI{yIY>rMtl(E=VE%UX~Ku>KvHb43v}^vgEuJ}mD_An z=2&D_CCg-b6YKBakuk1~_?>LOBw|7lihePYy5-@th@*>k-o2>dGq7$zwa(b3k;L3` z6j7*~h&9w;eWs}*T{xtbQp&Top;f)ni82@$ALD#kITgaN<&$)xs2HNohv$mtepy*# zH{{JtZC>(5Js4WmD{>SO< z*+LLVniBV;{~pfiM9mEJ#?iQ?b)K}|)?c|C+p)WK{?Ge*ay`%D6yE3HQ5zZl9}_5N zS_b1^-JL8ck;RV>Ler>z6m#RE>AmvV)|`RKBmtgtfOej0~3c z&6IYXPj5j^LuFdEr;=muTXa^b@l4mxblrB26n>HZ8}=f~hq}v^-vt3@^@rD&1Re+e zY^!RNIZe5c@l=>H5A_JE40t91^&@tA&kPn#I>nHqJHXKvc~OS<#Oey!=E?e(M9$9d8{$&q_5Kcyu(l2Y;-Rik=NvBAKbxGl7LQe0t!tlIT zg0nTYf&^h+QcYK`TnF%C%6t^;pXPZ6iN?c+kK9LmZFE-~ly7jvhY4PK`@w~^&7a2D zRiLUrzgJwkVSB)7d8*p;8?;v$8HHbV!<1)cRU^~QvcsR}>K9MNckj2(ShhF6W<+(b zMpWSAg-rVDp)Y~NZ^83Ypa<7MGDJ3H?CH})<*TTJur(szX8bT7P~A4` zF8XKd-=4Ih4RA*r&zCTUP76FW-0ynnr`cFp*Fk=Kc(O_nu~9nFb5C0YR+%qL?b!&C zT-w#OviU|=cW*P}T4_PKk-mpA$oa3H6jV>lf^;z~uXi?6MSNNvpA|IB_KyCx>pjm^-W`SJiO$3A4f!yyoj1q&eW)$*qESFq;eK$_1WYWykqPYMxln>*IT6V9ZgH*UDLbCAo{6>|*wD;~A! z)d;U9&&h7`RmWORK6wFB^GOcKD|y_sJ4<8n<1tOUg&3`*e_ywfE1`aUcnx%Zdzrr> z3_l>SBpTmE%<+B>jJ{#)5qYl#G2sqCD?BQRJmi@06@Ei7V(=-v+HTR4XjSH#=(42; zl8`7wj^8RSp$_a<)wNp9YS{kIaiuR?&kppH%dQ)Egxm)__v_x##LxBuQYaqsjwr~N zY~G)YYIj$wY{1w7Sg%j_V@g&!6pO`cJ{@1LFG{tJc4&dh+6)Yi$@D2o9=7i{iv~_b zTE#D~rw4CTktHRJX;e7qB3@Z8Q?&dmWVXEVzjc*IiQoliTR^LQ>5Q+n3;lLN{`R@k zrM&Yl^4JaJd@3|nSuy^Hq)=nU7<9{>oFNRxg7yNB$8o`>a({l+_sdK&xCdv_0&Ja+{0PqKd=Ta2ZIOy6_$3E&VP=5lt9|%Hbr4bA|{&=09G4tIzSEtF;#R z(#wFM7;?S>+%@}bW|xM#Ij_)lG{%i`T)7wEz{V+?x?+Q)Vv>$KWK*vU4Oje2ML`xs z*hV92w_DQ`EZvTJe#WkPz`eABG;S5sx95?$osK*y@$NJm zc(C&wbRPG|CH9JA?PY7u=s^@elQMTp_f651tV*y!LC8-FX*S=-Z@fn(*El$0vuBQP zzaE*tDh62{qng4XWJjgNF!RkXdGo!Ak!T#OS_2OrkEX>$1c_DTPNz!hMkA!`mD?;!p9-oxlU z50?1uG@wMtNMio*O-|S#YX%JY$KS3njZ{kii7>8)IB2)_&Wtfg@Sf}?-ZiFa3iiFt z^jVDLH~L-a+jVtgmj2}^IVf~k3-i%xjO26l>azDoLREk{E19sXUfGm=1I{5^cgx{O zLHj6h&E%Rdv+8I4(3eeoM1lUCk)EyC^{7@KCbFQPeRyw$y|8YnG1-4QSH+C5Ql=-~ z#o7DH{TA2@-SBwI>=stWjA4n66}KSA`V3X-yU>Rzi{3%tX1UnsZwrg-E8kEx(LDll^8 z$DxNYHMq2DvVbYXDlgCz=kMpBp;%N+XPmlh^!?s}VM2f1xSla71pc%WX6nejZU>xaQsCjVM-=htljFXBD;(j6st!WkW2Y%$#saL)wFz&-$`)6`IP2Wj+T`__dAO zpNMQ*f;rcB@yTfFDA99_QV4tBsq~2Z7#hglPb;*J_6z`$IdCYHN;*zTB{2~ocieMl z7L%A4GNr^8^T*;{U=)4&}*B*4Y4&PvD2jnp5i&-_P}61^>pWIst54xD7a}m zY`gi*g;`F7NKUFTn9~`Gd#_k3$OF}JC=fB3{Wo0#7s5wA1ag1rfAA_O0A_o@{| z4sCzt539U=2!w&Ke`E!?{4H#RQYW>re6-{bpPbW&*&NTc>zQ5?y%ebMB zGqXKta5%``|!`7{PMqSb?QY}TXBlK$Y?FvQ} zu$@jw(Z~4sO6rbvh@8UqhTxixrHoT4O1nH4Wsm_ECx3?fgxotfvoN`0JFVAs?qm}u zbl$J?M5)QmxvmM#e2`q;4Yy=j&yU)xBW*;(NMIx@1trw>7if6~cS#(Z4(%8Gio!Xq&YN3>O4Md2g(lL<43y=__AY(^^rAJQFyq6%&uC+OlDP%M zw{hG2-PsVI_VaF!9Cw;^nOHeq@3lrnsB%KVWQaKd|iDl6;26@ z8|n~DsTWPg0TJNMd0$h1%F$c_S4{CJ-SJ;8_z)ArfC$J^rc$#JO3+s5F8d?N>4=6rZ?AV*sFED zN8gDY5?^h-eAYr=jY};+ph=z6FvTE!l#seAgF&csV+WqojsNar?Eq;BrkYhv+4G0f z8#B_$DEj39?y%Y+jcq%C*Oo^Kib$=96dX|pw{_$9UX}T>ju?J`FbDke-Vv9fb=5R4MRYaKZA?CpjV-x3`g7a^EHjT(I&Jdf zP)vb9`90>WdWF<>9BoR)LK5#TSv(eWlU1kJvP~Vv8~|!b93aYuJwxNt;IOOv+i!9@ z*n$JzljJ@0yksQlnpWI3M?jUIElq9Y%_grP6H--+bcGR2((K9@DT&7YIB;^47#l>d zEhI;oUL+H0=eyfz2Yetz;V!rO9xy^oZ_UGPTPyT(UK%nY*@27SjVsq`)V{S|yFhOz z^+9~QNRIs;NCvF)Nq*3jsG0kgnUTh)iP)cpu{{>yVWSNQdd~52$*QUSg#fzSUM45{ zr-SmugImZc?(1D|Oz;szMyuBl`$7EC(USodtEYhlJ5iIoN>O<1_}L4K*`CMZRjUj` zT^Bf_OJ(DA=~B6y4t!$Oi(C3q^^FCV@88OaF(m39_JbOmSLTx;k&YwD9Blr?FFt+bx(CydI58Dy!bxRHf`FZk44h@9)O`%|?Tko{nsS zQLQJnYJpIJUIy{mN$6mqGUwN#+cM#2J!wx~-0+~76aDh*XZrxC@MsPgUr_c`tr{Hh zPnRSSvZR~rgNiGZN|?+r;dnJNk2A3xtgCsDV2M(fE00(ROK1wxUc#l(md#s+&AvJ1 zO(UGxk<-$A?&*lpw$8M(h<=L^f>jKW=kRQ~`7S8O@Uo*jJngxuH6~5qPGpqw7MKM= zS{@3PkBs)b+-I^+z=3nr*HL3)c2PkVR{q>v^m6i~G^w5_Lj6W6;!5Z zPVUIF#jjNH6dr%Iaf?OzYYxrqJI+olOk}-ImVAOgH`Jg^27$9%3`#$=iYO?9bkVDK zgIPj^zld|V-@NphIbo*=pQtEO*#xFlaJI8A^7ck_^#S_A1is;bl>J@x(1A22;_$RD# z9osR-{S8Dk)5*(L;1Wp1^Vp>w_GrdmO@tG}Lnbz!H8ZwGkG#mI_a$k0CrN0WtEFvl z4p*T<)<+t2`bDM>V=_+PB)*NbHmPzb*TMyW;z?&B*M4N(7lqPb!>^H?$BYTK2MBjB z^FI$AsHbDd{mw8(J=yc5)R;1^V#5ttaHIDPn$olWj;++IVI4(SBUxOy=@Ba`0zl8u zQDWOg#QuJFWn6~^0PP1ix}w=wUcWi`$zj6XCY(C^MQ~E1dt)e5w8`Z6nQzV;UdqGh z5%hRTCG(9_Zi<0!LY6obPD5?lLNPTbzab#YSNM%HMqUgg7^c#0?kot!JP5Owz?W$8 zX*tC8b@z!FxyhTx-LLYWyHN1w>KZLS7-3J2Ha@NA`KwF%+<7>yGaCu?6sqDdsuxPN zmCU^3GR8@e!N1yp(*B5-%Q(ZwyVeo^Fe6>*;T1=PSs=o2F!1@yci#2j{IL&B>zS8K z2Zef^Wl^eV#(0_LxZSJZEz~QzH4V2>3B50yL>#N6< z!Ypa?Sa53l8@GTuY*pj5*L}kUP42r)fjTWPu0Kk4<6zIAb!AQ2l5!d{DU{M9%5W|F z%=k)V*Ev63aLR6q9OB5E_3zTZ(*{5!7%=`E7w%hY*Dy`2lup*XBTb6N{CTP<++Sk0 z!Ev+nR-raJbMqVTF7;ln^A?`DX@3(c5IrDsJAhC*WC5Lwa|GJ9E}^swi>_AL6bkSY z48dyFsy;I8s6_G4CCOOqEac(JJ?|pkjiu!l#~|`B>x2Xu)Xw4gW>{Qzi)?H;o_jbS zkLPSA6qfo>h)n}M{7H-fX8=T-bDZFMr@-8f6sCb>6RHixp&8JdXLPKTEt_wQgV%D> zb;d{|a|7f;dCT!Yrlraj!icYP5x99CbWjRx$C;fDvcHCd=}-u4-%Do8;3K;uGRgG{ zWlCQswk}&mkg8iYPBUTyOewO;k(J!3Q0aQ$e48n!^7L%Tivm&j-n`DBi%N} zF?j-IOrkP?aKtJ?;^BLXsuh;AIaJ-1@BL$kPxKx^;)qmHnfFF+8rV>RE^I3i51!RQ z&LQYl^Xbq8@0+eYGkU_!%AL}zs7lJ(=M~HI=&5bF!pRhfJURKVj7J? zMtGAM@=H_72PLG3IW1e(j8ghlbLkeQk9-%XGv+&kyS=aTbHGQ%r+BAMPH{4Sfly&D zN~Wq`7+iAiFqukDInWLXs5EL2&p@_T#LMC#F6@fYVlrz^N1qa-qikm;k%3mRgxRIE}vI#`F%Pe%-YkggvVhA7$p|1 zztLZ6$#rvoZ#^$p+hsO`+lU5OuxxYoUSpPWq!gN(h>Yv>BjQmQKtlo$q;`3|r!u9# z)$L8xj8%Q>QmLWp*AQ+|Yp>NG%Kr*XGuau7_8Z1kG$sDkw8G#k{IqEt4YGIKARuHN z91$w4_?rcpTi%HBKPmUS2aS~|zwoztYggqGuBnX-&!aJ~7CiYYTeR};U1zjmbyBcK z6^4(X23lRRuq1cBPnlkBLX9D^$dm4ycSyovzumE2hTNa7ciu5^EFaN&-49?6s|M6d z>oU4}j6lQ5*S9O84oYCqlnx&P!=Lw_{diQ3Gm~P5ensOjd#@a$C*BG1G<=-( zralkp2mh3Jt;Aab?!R_R;U6F_aA zR-txrg=mv1EGTJIE_FE+rSn+lAIyV8DGRk=gcz*I;l!eC?%SbAL?n`}M6 zq#2BtNdnUCr>IiZJyRn~QQXe&n%YqsFyi~${}rKWlFl7ABU(X_+GrT3%{W#}&q!)T zp%RW9G>bW*RhlXrYG5*H>XS;=94pR{w|J!3*`4dL;;>`n=#~hY@fJxqVE%pOeVR3b z@IwuRI;qVidy?typ&zBu`;gh0zNZgzL*N5o3RWSkW2ZW)wDjgZHy+a*Mr>VISK?+H zu1T#|W;qiBY*fyb`Og2JP74gZKIoo(ysj4S5&|mf z)K5gGM{+2bvaXX|hLNh-w7$Thy??1)E#}949QdTB?N(-~{^6qT82$ny+VDADNSU36 zApN3hk1I2f+Y#~hJFxj2`+}?ngNFXHtYkHhOkgv2N-`~(=tHoa$D782^&3>i;J2t~ z5!tWSi0smqX)1TJVq2{T*t4otvj^(r*9Z$kwaL=w=^;esVO$CaYh%#W@&|j6M`w5* z-+N`!?_m{1y{SWxC=%;0|0aD6hkzGZ)NDk)Jev{bd}=bQZ(1ZYD&&OZo% z*Gj!h0oA(sJz@xE4F+=9(3%`3KPK28FPFEn>BTee$?Vpd?_7x;D3ca0-~Nbc!R&aE z-gIW_y(LL{M>aq=mX=Tsdiie$el?p=NfC!15H|Qsoy{PFtJ;1EEkfYp<|y%c4nC>P zuPe77b=hP5E87%IGfv|X8-P2VpSWd8P?;Zx;y2@A?D=7iZ!wpD!;eTtVf&3w3ujfxgEt!V(VO~QZ(M=5-y_2g9bEb#DN+D4fg4rf^ z#0k>i1D(C%x6fY^<@jI#qNqli)aK?UD(3fx#SvMwRo4^(0?VjJA0%ZyiQRJsmi1&TU>{hg_4~lIm4sU+~bq3>MCjcPE+p}TO@gQ2}Fc$$DJ_Wtr~T2SkeGKWlC z^uS=)k{Tla=I+WJ9zTMnMAF0<7j=oDaJJvwxo;&Af!{$pR;ty#)cZGKEF&-V%MY&8 z{fpXN@43d?17*)fP}N!qNTe1ozh!@S45& zu(JH-AT2BoLEi!Owu_q&%QcwWHHn^O!8L*r@q)hZTw;jlbdql^8HeOju-DZgZ#L+` zr}`z(vnmWie7g8xSj?Mxao~3VBmb*p50P`EhicJ%EyJCJ46%@K+S?#nGJ-sZ3%rGV z>#lmmz7*E?uLm%cF02{+HJmxruHn`E86|L9ymvB?lF3W(d(}Ss2MiaP&GbwSe=Fhl zO@Pmr!_2m6h_bP-M_@{)q~;~Z$)(4o*Q7rJH$lo@O~=%vYfH;V3ov`?4A*pgMJf#&8J!kmg9L8Edr_o5s0%&f z!XDo@t<%RB{M-N{t|%BnoOjPrVC=(1;fy$0!(q67Nak&7G#==Hcmc)w1X;&>(pVC= zETcLi|8jiKm*2FnLE-%&A8kw>B1%Qdt`#%+k{C|`gDQB zXZPIH&-nvUw$s;}p>)}#myzM7=z{n+{rLHh;+dk2_x|~S00_JZkc|dPD_)w(*Kxq8 zTgM*W+A%hR>*h-(c!KvOE14`F><5bvYZE8tnzKLj*^!MMnd?G;H=Q?un# z%)I!y?SbQX2a$=`xt5edF0d5qC}((!#^V*NL638uhP)4yOa7SKfPn#p^t}e7@wmK0WvecSl$9~bNHKC zLM%ZK_*Qz26PZ+Voj{(I|5uGHtF?BkM-pP*C@)7~vJAi^pltpDl zYj7E$u97phS;Q+oP&Dwz5ijyq?$%`%r#ZuXrqkD59`b&c6KYi`RY<8IX|^)3yA95w z1OQEa$?_BKTOaKHt+t_RxMo^SY; zwlev1C{5%QlSZ1Z>36v2a#_l~Tp)(&Y7`QUol-V|y-1%(m>1y#$M&B$(7PTaYqJ>u z(kBHir^3u`TQ8cpU(EU+Ao0lfyL#0hud{kWvhhJG$v-qy?9xvgW}DB$Z}w|SbA9Z1 z&8~2XCxf45CD6W-(+tJ>j}UlY88R z6X-qtbcNWSiYI>A(mW=eNL_MA7@pmLP{nq*Kj3)xz9&H?It7<1sW~8E0x+xY4Yc1v zrl1jsSim{4z7he3H;dP9X;98)#b7Aq2P{Op}vk-;ygXEv)~43>J(AP*+NQie#^SIbj`mp)&=ma7QHyKboY zn`@gRblKIK^6&<_;U>e)EufvDJW4F1$v1_wguxg;lf1w>d0v2<@Nnjv82g`?CWgQB z(XLpF$my){D83+3oXjn*ge_1Jkz($CgZ(sXbWPCrvVH^$9j^Je{;)y|NNjq}UU17#%8r+VekZ3r;DNQ`S$dasJ)tzgg(C;j!jBVGX>+olAW|4piWEBQl) z0p}i=H;UXa_M0N+KM{8O{dj-IhLyFYTaSjMGmtXy1Bcw$}znQS|c5Qo}GFw`X+$Uq|jae)mi=VhTV1 z1sM@sjD9|~gM|*!jgt1g**2LikbqN>KXbjSE8kw}L!jzeP_$kNSlbg#=x&tvevb;rTjHE8#!o~! z{iW0e)`?0UeLuGz2#_Xjr#`axXga319$`5(?7y)rzbgP9B$|rBs(8a?ne9UXR+z#0A&{PQiS1cL1mC^g#j8IGJkI>weH8cDqYgM&<8|<%6p+A~ zDt3P{$_O{o?*hwi_GD@-jv992Si=8U%V7>>po>Pl9S*`NF+W{dW9GDl~Qx`tp3 z+|l_w4O>vER{0@%Dd$PUoJp`YkT`&c{qPsesK*u1QmPOg5ak{2Udl;AI<6V8FzAKv z+cuQ`M8E_P**$7Ms&Fj5LvY1&p5<}jIc<8Vwgu#{wGk>~A2f2D=N83h6&@@$eTWJX zH!JW=o6{#s_mF&b-cqkojZKwj9K*ZEs3rq*F58%DX@84=u}JL)cjkeWrS>#Y{PC@} zMDeTfsQVxsmrr-c;RulL^@kyu?ynSM>ju!Y5GkIlcW!Q2s86Ni)65)qn6(ht$&%{O zDOJZ?rLpJ?T6(+1pKDd9n$Bm5b9kPYkJZuIiQf^}igDnvtO+rRBmjPqu@)1mz@nC) zqnRrBcIjF zP^r8|kY)n7U9MZ7=$r3%p^E!j!=ZeI1Li*8?l1O=XHwxY7wiCCoU#Fl2Mrqx-#n$w zg+E?YnI0-TX``^C0MGM#3b#Y|sHhU!c)87EZqatTAWN#~3=)=U;|6kk_Pn&gnP2Jn6Bos=0stU5&q;~)K6i4&AZgZSyTzKG#y$TfmQSxAC0tJEBt z?MKalHI%wc`g4GGhnbU7?^vzIXkS@dF&bS;41S$-(A%!5_nsY@}r3%T_OWQ z0_KW{lC4!N%(-O;5>@&dGC9#0j|u*CY}J4qEZYJud=e!*{ddF|^mS%;Brqc(C$wrl zev$cf-W+roEmAECOpN&mSlY#(W_h0}(s8iNShluXQO69z9Rlo+fTa||B)8L&D3d{BbGgafRJHzPA zmp({hjbDHV!nh^sE~dZecijvC;oZUR^UmXRXH##dI>2}yp#g))0pYn2$z1*H=YsAT zsga_=p^X_%O>dW9ORT%?>)zPVPOm?C_FL+?%gygIYe`)o$T;F)$?AK&_`w;@+z^>a zNzDt(vmPAb<8J$pLhE5 zQ8hE~--w|Sv-Tm3zy#>!X#UGFt5A$pBoT}ea9oC{L##bl6!v1M_mf4lRGm6O8*Ws+ zPqPGCE4+p?sX~yy`wwxGR{1k^Ajwq6_`v@+3!wRL@nGo5uAN=m)|GhZWb7^iP*R1A zy(?m>zbmt<_YE~E@Hn(kIeHX3d0*Ak*LZU{Q6|Ux$4WWsz%q&c)jI8Mw_TqnAaQ`X$wn%VJD1V zeq646LOr+vS=~g0ttps>O}1d|9HdG32Jgz=0o7Z}{z; z;luc2>O5wIS8*R9#5Bs6C&nwO+AUG?-F-|z&$rs4VnQE9uQ^hjSWX% zKr=74CU4OgD5+Jd&H1D5E@3wG)8ig3)ON!z8~#eDRl|>n7BEUNsG7uaNQJr$XFyD? zYcj=I6=P`36vr}A`In65J*0wfi-^4yX%B5!8xZ}w$JF<>=z;^MpX=N`r)}$%1*f}! zKkH~NTM62orSYAyP}t`z$tU#S2j5Sd%T|Z0vlz_Av+IRGOJ|;1%5|3J>g?_QCs!8$P1$jPbd-8A=k`zGL1Y|;aYcqW7GkxZ`!FV*N zR{7q}-d>m2_p5W4s*QcpKOlonMe;TE@AdXz?6a=7W*ePW(WGu!fB?W&VDqg{`?`;q zEj&|bhf;~JSX$3(JZ0#TiDNU8Mf>1=i5O&5u-+Jw0&lrL*eUI>WWepcd`ci@?>r3EDk^{_*|zDtFPW&jSar(R1%l9F02YL# zIA99BlZ);XbnXWEy@auQ*|+?Awul5G7aJ;{OSgfvq2UBJ$#%G7B0kjRq$yKEKL_}L z4<4Bx|HmjIO7(7gR2PIeIZH!le>`{|?TT38>hhxBE*G5yCPSmBHa3tDWTejjC3Tas z%Rs|fmpl^oOIEa68dWa(){g5kaz&13)zujl=UB1x+)C8=*kr+nEaqcY|A+W$rZui}Yj zW9H--i`o)4l#+YzQ6d>?MKa>GPM{rkhoK2S=6Zlaj2vuY&Nh{8Iw@LyBJ$RJHHK5A z-c_gWtrDu4#(3LGV+0=f37$-`oF=%VQt^sGr%LO0qU0^@H)WrzVCPt+Q`C3qNIPEmTh8Xb$oeZOYD=FBhFqnmp&sVSyDBx5m;Q4bJ?lJt_{55BE?8 z7KkN_Sv2wfn=$%Cw)>L${;bJcn@ZUeoh^SF*(d_AJXFjgX&mmiTiATkmZp3i4}c|k zE)W6ty6pnAz>t~U+nhtRu9_vEQp;NeTglly(M`pD);z7`D_LBN1`&iyRJbe3hKIc$ z%@CiEyFZ1=el>EfaGaKk;y**my)sB0Udm$ z=P0#JnQ@5)KU69ubE*Z!^}U||8KY5Zb)lJiqD--CN{kDuJI9AD)dPV{)8S((1moza zmN?Wd@>QBLp#va-7!u%6S`$r&w9Oznj>URHnj47KRU#q>9Jpc zo0+$h+{Kiv8m?rt(vtsF*X;k?;?V}~6$N)GiiPG6XUKQBtM~q3Bfa7=WCt={YJ_VQ zMq7T~!|Hjjyp%Du%>4M&DucH%k)~DwGTod{e;zz?y*=#5?M1Ns!W6<>muCp*+L87x-JG<9ljCiwmy)rb#Qu35*}?%Nw=R|%JT2bg|y)}>nrDx zYx{0R@~q#wi5vsOn4HK>_oQVxzEYh@(`pC>+6ryaVI=##pO~$GqZGciw8hx4H(jkD zO~+$Ukbq^=((>vO4w+B;E6o7M0(RLgJj|#-XMuR$;YW|dt-B{f-v#-faEGzRXOpIdhvsP z>r75bk%<+lyQ_oK8ZsE;%IrxK_>#={9wa(fPlDF)#eU&u)zw#2*!>F<* zWRnWOtzb3^wQy+L<1O(@AZglYpsvljsLh)?YoI*uvkCec_h#+pk+&rRPz36{?{s~eAP+06y$ENZ9 zJT6Me1KPXvw0XA#p<|?;w!A0Sk)o174e#XB?^tk+ODjla#M6STSY?4GbpLu)Y2Hp!6P`OZWB2*&-Ksz0Vr_I_>-Vm&;SL{ zEby69Un|Fj*>Tv8tp8a|1IgF*EJva*+4MrB_OmG!aGP3FAA?~{R$TGckp}<#BW}b$ zJ^v3aHWlD#M87+UE}znS97Z>U(BBFFcff-=+P4-n0QSw!+BV;XiEQ2wJ(k(C}76IgGcHifX2r|k4Q!+tNn=Cgr zq~;{ws@&Y^+M?(rvuvKg65&5o3!o6H1psY=`4x^YV*l|a1e*d7Lg*gzGXATb{!pM0 zym|cUj!#9K@{j-dUA^V;rYzWiGio6>bN@^91*q8n-^Kp*n*Oh5%teE2CF_(bzQFk6 z06l!0i^W-ed~bCAFUbs58H2Wu45}H^-ZtTa90C?FC^p4T8n6HLwp=J#o9w>6NH$X{ z15`5~e^pjPqrvvCU&+ZBlzeIAu2S}ytm&ONg+B`O81s0YPA?BA?!+g)d(EoDsRO${3o`?+O$DwA);D zXrPNoZz(0-ne`fJlSN{KsSr{DSMnGy;d|!s3{P~{#Hf})HK~omABLhkQ(Z`_> zIE4-3>z#JE61oQe%eRubxKPjWBOKV@{)cFhAT$ghFGl;MpYp#Phxs_|;?k)6rz8-i zqCYSeLC4AdH-)&z8To$~`)e%zUo-x5S`z%9f&5c!@P7vK|E_^V;>eiMKY9oUL2Zwu z+(+1A!|slex{=yogZY=3ft^K;=6dQ#FBm z(2STS3ng@%Qj&17=aHND?$ZZop>x`V&SOH-c%L<*ER)&*0a;go;8BpkRKJ-3aAZ5B z8Oo9(>`(+eU=9JME*^&&DGciQ6@dH&CZHQ;-G=Ir@0xJG{P1}=KmqPxgSRGG-T7et zx%OM0tYN+HeQfom$$FmOVuteZ<^d{lc`PyC3zzq%$(oNE`_f|9D$ru^NeK#H<{j(+ z%o+^aI&jwXpDRSDZK)6R?CTe0I5Kw=J{i}`R3=3oQShR z&^!r*PNiVK@ka}tU-LnET8-o%Mhck`1f1IDxe!{NR1nB|;qo0&Rrdl2A|`k2)v;Xjh`-*3)}t(7*=fB710Xs$0wVT4 zYTtJ~e1s6iJ`|lc1FQ34;U~kv0Lt|Q;EL`zzYRaVPv5Fp{!4-}AFy6YjAKAZKcL37#w*Z$ZMPXa**R>zTRL{o$P%5?XfUaxz z0=T8h=Ni6OTcM~#>A$rcGeitW*BP+yH;w1D+rJ!(Y69r#H=N#=YoWj+!@i`kunY)P zse9>mxY+FFwtZ+5JW{GzWj2x-zQrXcxPcjJpNCbX%q_;if4TZ}N)r&FzCZ3I2@KON zZTS8%`lHQ8dyGylYG!}q5c}ZS_vsStT3eUJ@GWLLfLaPD@#86#KK8`&CdU+J-S6%x z>qJi{jp#XydyunHT{gxQbJWx|O+UO7c~8eE}xJ z6D*sG6g^K%VnRPq1dbXtl}#ST36JpC4;l^nfURpxC-tj+qY6nH!@eIBq49D*9NU{J zR*4?H18Cp^88<&kXt~S-SUt3nKYOA&*6+X@Uh4}~}_6PqDSZB&F zWL2sIAi--4O?}7}ujKeKqKyhYd2pUQ=OaTNQ7s2B-RDGUSH4Qko0FT&X9oqViuoLZ z;n$S<)rvXd4p?+ea`~1v`tP`)hNb_|h}EKjM9}mOsgSs@LI7&BC%~(%m(zMv_T>?? zRaU#4{?^bInmkQ*^*&2V&GooV+N^fT)mT@F-Y(4C?*u?^_rz<{<4ukMxSismWci{8 zK<>G}gMxWTIru1X?g*ePGy;s^M;?fFzv?wjw)=WzFR_do?A!eh$y)UJgxt1^0BO?w z;qGT4`$?kI0e8!BG?D8pS!ATt(IYGfkk8+mNnG%GxLGL-d87^3%;I)dp93ssBVQ6@ z__*LG!?%|JGHoOJHs|5i{7{#){wpwS0_xZHlF4JDhA(<9k2k*Qs?z<$uD;d=_$ZIh z1rS}fs4a^7#NR9f_yM5vWXK1<`g9>pu9^F=0w|6*MVU%(Oo=zl07&E7>+dBgQtIE) zS&(cuD|p;b`e~{xcM0@w=D!{plOUQumt+};ATD|mIct4^+v$()0EfNB#*PJl z7xxapjAV(g^s4Iletvfk@QSk{3T25U=s!;>z!EOMe?*V%Z9y0@VM1B_t_HV4BP}Nj z*bYs_bAevR_>EuYALg4T>+Gp zYk*5O$jPP6jjqzX_*5fTF< z(+=c?78-Rh!m>)d+O!j8GiW0OV;Bi`2c@2R1cXI+elNB+62w$EW@UiI1dEBv(mEJk z`K4&RS@@Riak|`6d*fPsb)@O(*-%pAx5QCi((q9$M;v`fu%*PJZt+Bxc>BA$whb1Y zb!~`0nE^a5H1H?RUl2i{1R-`wBFP&-^)ff^vvYHh`AkUBR2t%fTfe-5o+vKN{#a`p z@L}`qYrStWj9RdjYd`!RULxTPkAsl50NZ7%dtq|k-{CfvE2PCE7ht2EItP$$F%ShV zW{vsh08{b-_?Of|r=9fN z=jOt6_(0&qY3zBX%Z#Qq^-b(Y0XVv9fMu0PqaZ)Y`JIf@LH5V#r~t_~a%CzC^`RnQ zUJP}&i;w899YUKoWv9WJEJUV~xrk(G(QraX4>O5+`2%M?jU{w260I~Q;Z0v0nFG0L zg>?GQ@p2x+jvydwm^UHksLWo5y!ptr72v_xn(5#Vk5HxPd|nG7fJJ&To)=1d4Unc_ za>#!w^Ib}^U)`R(g~v9Kwr;uFZkOPM>!Z7Rjo3@%MtI^KfSyZ_4FDsZE;(Y9W z*;L{9g!(1*P8RTr_;!9dn*uNPL^-e1_TzLNA3jV`ZTA&md3SuK1!(I+mybZqna}MKk{^N2gu+jb0e+-`pjzP_(Fsz*Tj9lF zqiy)O2z{lC)qgd;#Hj>*BoP@UhWBrlI%5M*R6x>|?Y938|k< z82Gn>7vRA)>vO~M#iqH;$|PHwBiAn%zytRO5BEO>x}vPVAdh(fSew*`RGLf9J3>qk z3vXpByrL1VoBEyXK~h+hra(KmFRW~WuLh+D1(|$EHmI9ci1cW!O>VunosJIdWlor& zYtJU5Ed9M>c5zK&4|ZSvXu_!MwGo~)2%{FZqd>T5*r-NUMYh7*<;mE5`04kMlI>@c z#LNmX$1@6e<3Z>t0*l@nWA5ZVh2!jkXjvTq%#f;CTt`CbEzlNXo?Uw+g`W{ldNev0 zuwLuLD-Q}Xgq0TUTuyIr5Pb91v;D>%;2_Nuc+~loST~9VmOVX;GSEALzisN=(BADWt<=!pJq$EfH2L1UoRCcRFGcm>ERr17Bwzc zuaZQ%>GM$*w~8kLrj4!#0kLR~0Utq#8=oP;t`##(wTtMBLIIcr;SQj{eDbuPOwHtJ z2(^qK?PUrA+o@!9#MF28`yZj#sRUhb5w?w$BA*5+2nyMWU4()%izf|e=~B~MIR;*-o=gf{ZB zA|~v(;1EEQ*<0*LH&Z7i|6Z1{N@V!4kvF{n$J_XPM(#*HZ`#r6vou9EnU3lCT-O## z8HPCGH-dLxCA+*Mz2EzrkhggO9v)4R8TmAwfcufbY~-eSQ+M4yz}=8qQ4|kQ_UDdg zHa~w#hIIuadL^b4UKety)~%W^vfIImg5{^v{e5)wh6UUQ@@glAb@7YuXgZ(G6J@XO zrot#>5?FCWh7xH*V+8?!NTDD@5wO_l5`+$-*Jaz1#*9DWif0?bH=PFWmjSN>e{sb) zI#PD89N8eGSv?1n7a&gI6L;KGV26d7^Oo+A=c5r<9K@RREKR)GRG1lafC;$THJueR z!C=4S3t>dC1T~7RdNdr z`WT0!c|_XIBvJKNmBrDGLmGMOKa4-m9f~MnV3DagRWHgTlDT^84zRL6zB7&6o=xdV zru;-<>uWDS0-@wZL-y)Wi=S|C63komYd5gIIjG%<{O(v}$m|#{GpTxew z9!R~vwyvn@d_`=m<$Tk$)O#Dvj8sfc8qCiYf$6r{$Ls8fN=o6OB&rPSB z&>^-v_K~c42B%y)g9p|b!*k9zYAtlIQx8(Om^-=-&q&5JbA@1;$CKi*N2eh>uw%i@ zPIWg4Oav_&(XHk{6;mPTL^Qw1c;}6czU;*DlD0t0?UdL`b?AY2Z7Esg5G0IKRX^2$Mh=pp!YsiL{vM!W{7wT2>K0sUd$EUT z5$cu-27KJ?^P=~lA<%h6Qm-NsOo_TP#{Oxqocz1!iBIKOd9Um^dl&1RDdj9;IU_{QMj{Qq zy##rZ_`s>?CqoSq98^#O2;<}A!CTn+n$EPJk|Eyd2GJRWnu)CUeXI?#uh)-4d`0b+ zKquN`stZgQQdJ{nv=ro(NkT86L6j4cSoKs9lxt*xvC3v0^gt!9nR4aFf*+rdNxR1o z9l8w0dT}*VB>2BTZ;>w8L+i)Y2@Wi*$(+nqt;P^zvIORHaxk0GBoMF!mz(C2Q?tU$ z7hNQUQM0}l_ny|4s^|-;l`L2kCRqYW+==O7+k&0JE&<3Osd&8VMYA!9C|t;vd_KYe z#$rrNS2N=-Xak>IK;F^7_3Hh5XSA9?D6~XU29)UD3}1m|U!DB% za(D{%?P&}sZ4D_TO(UPF{sT7R$|{Am)EF78m?{7KZkSL{d6*-Ib~%)DN~)tMzrObe zp-wnO?aih3`N3)SK8iE=qJ%&!B`!nA2h0h|O{C8JlU7!L|2KL^We|27wIvxOieQ6R zM84oLWJhcBO)P`^_fnMVwiRz=pVcAqf7DGonHw5p0Yg!T3m>@Uh!nKm9KICLcN#l} z359i0=-3f1(eC(tPJW!4$>AyK@HJGgyxzLQY_K)~;BZm+Z%l*jhMgf-$}eC7%#b## zz{AYR%Jvvx8j|d#Q*($;r_be{z+WJrb0xG<@+5RJHB}0wH#r+rW*lpHrCxSS=y5(9mU-Lm)4-r*c+s&Ph*>P~~OzI7BM=&BM z$US(bGhpHlQ__S2oOXWpE`SAJ>zz1v^(=YKIMsfFDJ!B%F4rj2G_?FV5?5K8cOpg$ zD`uxWKXV5IwLny4!T?u$X%<&rwIV%@zKM4EHnG z4^k)SY<&Ui!%RBZLUPz8)Cr2Q5de^div<*xD|Zm+imweWdy@TqOp$L_M?CU1P7VN^ z93kduJ~n*@&!BmXw~U5!2Z8v0zg(e5ggGcfpq~mH#aqr#D^49{#x9rrc@@~loWGyXFI zqpvS6Kid-;bP!^{@J3trM!5~Cus9+Rh6R)kaApX%$?P!lsNA0~B5CrGQ4(LEwru(* zlL^&htCHnytTJktQZqw&-B(O};TiQagNC+2)0-9di;$6S=(S+kV1ZAqENGO*I1ys{%-60*T5OxY|$nD3IJ zO|%N^z-NDSYahGNcXa4VHhg=vBU7(^m&Bi4nD`xEFteYci(`4>IJW{a&v4Zx`T;q=m(U40^uSVHIACr2ax|Yp!XKXK zakw*&;ncrv9SYjC@umwQfCe)!Ma&h9_fZlQmIBO7ZwQ3|N_7XbGkf1VP%*ieLyJMY z?a72gUt~FEF7A9urT}|8k&C#1kWGG6J9-c4y+rIvF^TUb843)QnrgW4E&8z*TbB0* zxMe}q8Uep6PZ-hnsSfxN*;~VUJcA)PZD+i$${r--~-qT@Lf*RqL4$8-c=p z-2o1^vbzu*H+fQNhC%sr3%P^jc5wP#NuSsvs8*JBIHDpV;T0Qd0?uN)Yrkm3hy@4w zYi=K3^&J|fM_DicA$sWQkOMTgQ z*ysiWwSgTDHCyd7U@(PKozu<;o*2>;9OD-i#0Q}-_{Thf8;2a>rd|WN&J{$?U2Q{9 zDlo#8>(W^U`K#PSDHu^Nq$V9+ejhWGzxKqd*_{mA?SSpvJPou?GRY zEZ^SN!%tMo{#xy|p(%noB49<6RugFN0y1DVrY$+r+hDXNeWCjaxpf6rM%Qv9N=2XN z1mpl}WecX(8kLa#Z?L|QQb+Qa5#RPc*9pbR>+{zpXy%0$d)S*3ygpiNz)Na8f|F`> z!%Tugd*YY~^|^!9{R!n~jK=G4-zxYBID_OB$Uadjzer$8X5a4yd$D{<%}|J{UKMwc zKBXeV$|SR8Na#Tj5c(W&0FeENi&H7E!Vm_&lcXT>OYmn!@!l`9sCi!OHHOve|WaN=Lxha?*7X7-dkR38}~vp9zIeykKS*b#UMQ>@fA%f=siR= z*!{WzVIv`1(oyQ&HH-^kyOwuk06*z|g!z`tyRN`w#iGiJpb7SV1XW3)!un19X_NHl z_tk=}q83SGgO|+|>je+&C&37?B4nJhI%8-#pSS)t@Pi-~d!qJIqH?m_bJEdqVT~xT zlo{-%h+ZQIFE4mtDok;3Otj}aGE$z7ek6~6uVSEb4Q#LlkjN{nBEOP9&g6A6)6sYA zli-(P;c7f{((Z1+WG~QMa&(sLG17Wc6wm9^kMhI`4Z0XT$=p7uU9@9GU72f9ly&Nr z%UrK~sxEWzZ2=p+zSHpYs&{D%XZ9HeA4x-vd4~7J?xFbPlOz#xWoU|+I+Dp8wIZ~; z6$+Dj(qxLz0|q@)sOe#u`-OPAULh)wEEfkIA=PmJcnOb#$1t1WS!#BFhbnmhM}{zCVSsjs$WA# zsSYMHO{SD@Cuk0vyCdPZ9{Fw+3{z^9+aYu-Fj*DkD)YmS)H!V2dM{(fkV=({9Mesy z%CGr`yB8iw9UJ%N%9@-=JlXk>%4!(!On8iblEsYoul-~+QPNV2Rf!jMR)!EPon4R= z97kT95@&Zn`s8>ZHP%=V+X&XEp>`|``8_Gft8*-AT0E}#l^MB*x)@f1guJ(M6m&{X zjt#0PdyH9}S!NWG8>4=lB7m zq3itX&%&cU>K)K~js~@{7_d~50j+I8hB)Ccy=d@Ew_AwDQ(!xy9QLVKl8Oc7%Qqt# z&_*C7E8b6s!Qn47mxMo?$9+u729W z5h=!@OHROJF#G%RVbbT!baz){ zeAc#ESvG(eQyTk|pg{q0ym-`NROl89GOGSw!TB;OOK7Mb1TWvU@1yOTWbFo!SiK$= z^B{BRYe!jXhzk#W5pdz^Y@9r{iAe7y_Br;D=i${<4h$4&O>TAgV(bT8+#tkjnV=Kf z^5%;Zyv~_^HLT-$G%B-v%8$)wY;NQ(2Q$B>4OWm~E-WSroud_%UvI)N%~HB@#kJheWQ1R~l6xaWbPq+Jfe zL%|}byp--kLOO3>i96WDmJIYgDJ?g}I{=X6sdGT~Q%SS~L#4)G-jk zqYDvdCrgseuUNu=jF9~9L!vznB^4HW?19uV%%(H0scJW7qqcbJ2#k%#74{r~md@Cq zGwA|qEQGa?EXtP=I&}dUZ;r-9p2=ZU3e4KZn;U7w!1&r@yo@=`H4D8fvi^GLaCZ7u zUnKzQaAnSxr|<{6_GA!(q)N8pm5w``{W7A%#%!H2zIiF`;tmZ*2@2oxXHJ&%LF5hr zU$mw?5S-Yzqs>GGYB)N3pdx!bWP})qsKtA`{_rUyKb(1v-X!AgOP<{MP zB&ARb4fdW7zlp5x1ww{6KoGz*0WB&HCv8&sc&nYmw#^@^J>d&qA{_)qT{s`PHLyJv zAsHNWdxrGYKpu;a1<=J8en4Mi3TEznAbJ?}yRQPAajD*8(Nh7OA&h-ZD>S?}^S@qu z(Np*?di((G;8b)pdV#uI>2R11jn5b?+VMg!d*JW1ZR>+cA1pxRSBFe5kOHcE5d`{_ zX2n1E_ahQa9!m~6fdZ|GJeLY0U&*ahy^RQG-*t<4u2XaIAWVtt6G!Alo!Yv6x**pl zMG-=!s&Go%=bav5i$_fp({3xTlV0rJgJ<5wCnic6MoLaT4(DD){t=23(*cWP`xFM= zDrKF1`?jU+v`2CebVRBLJF>5sx)F7L^3nKMVT_N(4a!?(j0zVpHK?8w>`|Lvg?r5W-p6 zyAO9h6BvTvgR{@En>XjobINLwgMRq?rzx}ch0P?09t8?^XJT1_uN`XPh51L@JL-gA zolmI{PU3pgr}4;va*Uis+#@&bQ`;k4s5ut09&+b{EyI&p;ba4>P+Nj58mg{sH^-dE z)r&*)$phvuBVc{+FZCP>7*#r6Tbx)9e=l8s7yK2M_o=s5pWL|dgkGT@PsCp3H7=)dHG`W+Xv zvAW;k+d{wK$?pX-uomQ+!(PS4a0*pzV5xJMw%qQdr-2xY%LeV0`SA_jF49zht2hv$ zv>pKG%>q1fJCfv&o2*~rj(81>cE|E$TY0E3%j-*EkcczGA0!orA}eZvTpkI>I#aD} z>(VuqQ>ThW)O3SODyq;?94;gWg-SoNqLv3tH6&K%K`mHT!-#^+QD}oBG{S8YTTdw7 zWGZ$}jbf0(_h`(LlG^DbnXb&8A!`!BCYnC)(_!vA3Bn)fx5U>}OufJQ9xDhP4W0o% zPA>r82lUKME=Mp$B215#y2bSq5I2YU85T71^x3PUwNR34&Poqrc9XCi1yo@%d$nN- zN=|v4pKmH{Y=yG@VYFj|D#KQL+I-HZU&qVZ&UsvLOm2c|Q;yTG04VgNY9aDaX97ok z%XTtd%4bit2*at)PmKyf>YsdlS!q7Jg9CHBx4k4I2aZIN5G=?f?BQ|RM9GIvYm7xpDg4GLNxbGQN^oh(nL0*PAK%YKj3~W+99)4&o`>i8Gr3i> zK*miIV~IM9hTeyy)3lQ!>)U!!emiN@c{TcXEzgKkcHx+@YyZ5(4_oJo40D}?XYCKaO16h57Aun@CfaNc&~H`RgO3Z zMe~t@uyf3^`VEV(fjklPD*?IKfKV7^w&+rVd=m4L*vsaJ`y03tQFeXbD^paWFq6aT zNqJH9VJ2(QAk_}0nUPe-3i;@y_5z|S0o#owrYPh<tyPp?G-*bDWe z)Skbqd+;^XbAAjuIQCQZ589?SU2SLheD+0c;Zi(37Xsq%$i$*M%56*dt@QFmE*w;4 z0oV!o3WWV}lGwgX0-=(KVF4{45c<&ON%25=A<2+9Jn%qW09;m6Tl8qkDC=~%efab< zeVH751n)A0h&CH}hjtEiB{SmN)UlBhIbkbVv-(Tn^v1b#Uy z8-Oz35)%L3C_aL~^JVlj`&y3#X{1Do|*}b-X^`3+>8?$8Z)DnzW6Q;mx2hJ~x2E zb^;>djSpdhZVfLF1f~gRTfP-!09IGG0Sl*#v`N)YIP3MJ*+*Fx*C?r=a`=8n-~lhm z&^z6krU*x6e7^i0ue~D>@!GR-gtp0QO;f-s9gZJP*h6O+|LKIPF0_s!4%^O03A}oj zJUUzO8IBuztFF9u3?`WPP(Swlpqwz)SADU55YN&$PHB`9G*{&&4Qod#^aS3Bzy%5N@26ONsQ0M zG@!nu0{d6aJZ@dotla&8U75)J5UbDE%1e7bE3+DjQy;e+*f1V6gLEeoRDwjT!T&T< z7i*kfk@~|#Z{a^&ej!rzasR9`Mpp`~*|>VZUMbSzwbgHdq%|eVMC}o!(-hHCf8Ow6 zeE}ukP$Jje`7b47hE?DllN@SZTrRlI4mdG>P$5r=F_k>Y1|{drx>jio&5;Vv-PvT~ za~2?h(q<9Xpj@wc;<6b|N%$R-?eypUmphqG37P?<0S$K6v2A#&#xQtAM%$0+EtrGH zOh(7)vbR?|ISQ5|Pl&ed6~mLqwADTM{Cl=(k%ksxD=I2AKV?vr3~4jdLi8IuOH#o^T7A`p(JK7pRs`LfF?Rf84Bc|rV8EG_}fmkqX_V3p708j zSIYkU_BlatJ9goV0bf_Ul2G}!aSU${j@;Y_1lu%n?)BIEwqocLbn?5VehYTsP^27> z(ilUQ5vBkt>Hw87YE8*2=)e_GrU9W_o|8U`jpU_Eb^C$WW1j=Z=bT3QN+wjSeFD*w zDmM4P`Aqw|mBk2%-+?3uj}YVOzj!*Qj`<0%lTEObgZ+N%p#SQp=5n2S+Jh z&Kk##ODlfN$7V-=xdx>E>c@$&2Yw`MN?ae%_t<*t9$XO`ehhZ0cMk+Yp`N4^-P0$h z9zFpX4N0$#yD$PFb3yQ9e89hi&PhB$Y}fE1Z}L=t5U%$!WoSGKy@m(U(N#oZu7Taz z?l}B+eog+omK-EypGA-4UShA(z5Y?SxH1<#?i}_X(6QuBPwwv$HY+R%yCe|bpXa*W z)2L>{etFieZY5TLOqC55vb{PXbeU3ez;6KjpjQ@#n3P-Q><=EXx89DauZm6s1{|_G zo{@ZCE#zs2C$i?=k(~~{!bB&&3Z^O?YZc{by38Qt=VNR}CT8{?F0IFBS1tiw{$1LUTM;P%==QRwng6KEq{g z`}x-3tvH4dUA00E|0fDgnTFg2-Y(5a1LN3{cfbZ&SV7lrfCqFk!CiLWxM{1a^V$z`({0gg$vQti5h7;O)oD)I z5UjL?NdT=BgY}#D{vw4vFT*zbxhPI3t-^wq**Xzk_|H5uLkmt>D!nP6hs*T1q_QR z+Gzi_)tBK5d1#(!C6!3H2>%KSKj|&5VSLijA?phVtIuc6+hG>#pewLmZkqzrX(odI zA#xSI26+Jj|4WcV*ra{lw+-i^voai~{(yV*V-L!vZXoS20lYi@UP291s=w=?_*W`U zMJZWazQq%yO({&c_5&hN;LBt^OVW+rjOg7yaX5CH#Tw%4$R*c>;X-7XTj4qN6B4rL zYXR+_^*+csyo|+o5{F*z{i2_jmQK`JNM3|lnpr0@+vtR-`Wn?03XJqc{zyqZZJ(X2 zvF97=XQOW)OFl_g8NS8x>JJ7_TO?hQwi`*Z%TBGnx8^`F14qR}5ov{=(n#CFP{H7s zK3O^EmFScBh%7DC5AujW4(UUY>8DcCht?W{%8s}k?G>YfPY@%c!Y1i6Zn?n%+c?eU zbkRP6fP^?_$@QOnTBQ)c;DB;YkM~CAUWdhWO6`+S{!X6WBUN~*pPMpXQECCjGsq#I z3%#7%LEe$)#~@*BpaG#Jc4j&*!Ss_?7m;$qC0Ze8S+G4h1*Hx_a!0`DDq{lVvxxey zj0nE{(5;Xy8yxyxl{3wvG-lb|9r~?#Bj#lTZ!*_kJ?FF%f(#x>#XBEqQw@$w@`wv@ngqn~@r;u@6)b_}EKP zQqm9A$7E(gdOeW}h(;U%*j&)<(Yu)_OAfArZ_*aGq7=e$JI=!TAQmF)qCT0_vxle& z!SKoYMj~K3{|+1%rV+;U*pm+tGvd+%3Ze}gI7KDAMLp}{Gz-keexf>oxi4ism2t_z zK~yLA1?75h(q=a@LFY!FKS6%z4X|hs363`Wk^5-?ulf`k4->`SUo_ev?S#}b$5{Po z5j}i1jzRAx2yp5twaY^+4lL=R!Fss$;W6+W zLf<&^bT6!VOV}+qF<(b=2fGpCQuUESRgc zxMER9IoMjsVTxa&1pC7v>aS+nh`1L!xEI&NaRw7?eZK{WA%TTb!um%UMNe&o5QwTJ z{&e-9Q^aYGKM^U_nOi|3ri>qACV=mfI~s*vh3~8@TjeQ4s3lW>ni;seZGk^%bB5W_ z+B+XWYcV%}MKJ}gel|M0DC%HJZ5)s_8)b^9^du=xdlqD7`8J#1%i{8aruQfo$t-yc za7=zO4wPDp8HHE^i^wTCQazMd9zO7vjQUL?PiWPHLa)kaiCI5tlM~lW1w_ph_~7<3 zUm#}Ec@r$f1uH?=zPf#5={cN+*+j?g>Yy&^D@JL5#@O`n{yirzEf9>#VgtDS+Xs07S+~`!zBi(! zed;mYvEi1lfz3g9v-{t@?3ro*R=R-6I~7iPF6NgG;T z)B6UXXm>q(w*q4W4!=P|&hlRj2)4&}2qq$1WVm(hZ&zA$c^BfOZpPR+*=Lz#_!0*P zdHn;K?V1@0jUc+Wy!JEp{YX45Jq+{&IXfyyT--v7w0hxKJj4BpqllXTx>HQjCuG6L zOHb5{Y$&a78XJ^|psBWYb}8h>G6njJacQ8ey6b|Tx#&op&<%A7LiLiNB6M$p1qgwa zMZd*xC{wHgJqo6(=vvG*oxC3=8Pe!3bt;o(OHq7-C&8`gscK+{RsW`FN7I0nDB_t6 zrb%{=G=s~;LoR;Y37_w>XVvLY01ap|q z@7Q)W`NiwE!{Uv{z--*fTS*3_e$u>v9_`@DBSBm1LSH!GnOW`fA#HeR*x)`uPhv=L zm27(*yc8jpK>hKRvStw8OcA^HgLhH#&W-66>WBF zkBo4jdNZ;&1Nx(8xW*iNoVG{Jv0*Ag|I6o2+p@D}(76vVEZCP>^i?)j9zT+nx?YF&{p+~|^Oy%1!BPkN&(zBUqebCqoTS73sdA};bB z*-lTX(K$t&qsQyJD9uP0WJq0_dddi0k+F#S>&8UoamCk-`k8kTH~gZi%P^+IYVu>+ z6=_f8ycBAf}Lbn$%#tZ1Rxel6D_jY^(6HX#?r(Dj!M|01)33I5O++*98mZI z3`qTMc9p6-Q{wtkLTOPB;<_(>ztdbn_*FLMoZhR0Ts-Srn1` zt2ghW!DnZt<1lYwhBw903zP5f!;I>SejRIcvwO!>K~4k94`<`JmLuHdEBwXjm5`49 z^~HfUO_{*@^_#25cZUoQZ2SVX4Cx^l-i0gma)XEvYjD+xSo9O~A6X7P zcq@JnK4f)W^|`1rBZT{Izy_keeLIZMu14cC;#WMEw5cAym#y`*OZ=WwC&)%>KR{PF zwWi?7Mr!@)j}R{7An;lYHHq&U=6*&e>$z0f!5mHlyJRsjWH7?qsUSMQi@K)7Qox?hg}Ky3rZK;!_bCG>G`v!WBE&UNLwa+kF(V zL)eJ#UPa+QM|6UnWH&*|dOJ!1?S00B&`qO}t73N-!q_X3TjU#lsqEmZsLNL}3-?!o zHw0fOw+l3a-`+frPqlvmR3QBz;hQa1p&5Br&c#)lHy_pp?16GQtam*yHy0@65bs0B z6Y+F90@YSaz?w*_+5*gIoCe%C#SxH}WWVdbKpSufl#Px$yKpe++Q6&su#DsAdpq=z zg@PJufO@a<{W+&V6fSuFFMpeS`zW9bEcz%?#Rt%es^BZcUb!9W+z*61%rbJmnJM)+ z-ImMXDy#CTiNOP!=s?te6tfV_qER7eqkX9i9{yU2YqjH!|Md@V=?gm0OQ!}5ISPL5 z%l(n)h!beDu#d1U^+DlcOneG6vFWr2JnOC1n9db}|-LeDMGEo@gb2 zfq?7TKO6S1Po#E_C$Lk*C;Punz$z9v{Lj(nZ=d=I9!EWbB=Nrg-gvCQAoBaX@z*8) z7=a%DoP_w(;h(^M0RNfL`0t;oaLy9`ecbxn!N`y%GGM}N60!{X_fda73-IW0|JRrQ zZ$G+VXOj}U@Epa_lnM&9+(&(7WxM4jPYL3BizSy76NOGITxb^zNXANo$9hya6&00} z%uaUe(p8rF4^{3P%jSPK*>^(d=|YQ3&x@}rep&NHz7*{h4uL%x+cMgZ>B#^1KJI$O;Pyf z|1{()k=z%UDAqE>H$#<)e%bqhv6JWr>WhD@23nmnMqq9WEB_V7`Nb@JxM zlY62wTD!4LowlbI{2mH+@j3O;vtFhEeqF0jpkxh@Ofntab`qUZen`D1?FEGMw#;*U zW>!bbkJq~+7pg6*+F=Pe(Z!5iT(FrsAM3y00EJ&&2sD{-pXaD#l4Vm_*B%uFfHLml z&p^5L51?vUx53$Rrc_fJQN7e1?Xf((_2J%Y3y$MMGLv=;+UM z>-pWC0}+d_<@yihes>I2ZomScTDRBJYU$67=yf#Ox_4_SSFQVOMzL3s+r173SHXe? z{Ry5s*$eKMM{So8mQUFwPq#IgMX^w9<{6b1LWNn)IGw}|*1N8s!f{!RzsqbZ$Vkre zFe{E$9WlU`>JE;?xo~gljpF54HJAUEX}A<(Q+bW8wsQc~qc$G*5Cl4p@n^cuTLOv? zf}$v3#>^Dix25S^Ht7kgz>ppyy$%!*PoL;}A9n*q>Tu79ALT)Sf^@ypJ#lCqpGg2!_J&Jde0x2a)q1}Te~J|Fm^$_M`#7VpUy#pbzH`%R z&iou^V;Do479=iDkVt;;x;wwF z>zq2+`x)H&25vBUYIu1>o71kt4TC@XobBO=G2^*gvTHqkcuzDzuUhXXL%_F{d>Q8F zU~0wHT3WgfyztjdPnCD?Z+@^alGSH9Pe=i)O_^cG)Soj-rQWy4y|{UgsQ?}{ddQ$k zefZDEKxpD@@9PghR={{H5XQ=w`t)>4ML~_JlmBMFYFb@eN5ka02jJQ|xB?ryDf)UdOn9P^(h^5Gh0O++wOpiHr~K z-l977p>-`J_$i!vkW^gOcW;M1=I+-TdCRG6r!Sl}K3gbxNI~r_lzj2lrajO$VHa+IeiRn$K?H=xH;3{fNz)g%Jr8t>QGT_S)t9 zRCt-=J`UPt<;&7c9#V-JK!dIfZEtfpwvH+e=|B9CU86M#z{=AJL?F2#~S?=SuLyW z`DD`+ioh21Z9an!9El%<(E_hG6WS`*Jk&ZskM)3dS=C?xSVLKz z_&|oW@=WDM0=3-H+oCacNs&#(DA(5-WGT&ZFJy4T$SeK(Ab3MsFr85qGTeCTTKyN& z#MDK@S=wyoxjEj&bco%G#2=i`hoSHcjoSlGj)R+nBB7yW$ZB@AI4#G`uDDKOXDbn` zk0kjqwZv`1{~yNQDypun&DzD?-Q5We!Cgae3&GvpA-KB}++BmaOK{iV?(WW+`D^d0 zuj=Zwwr=18R$Ft*?@y;Gy{(i0yf7^4_}nO@PqT!r@Vi!n1#Ml6>6w zv}jZ*?G}!{LP#Ot?`IF37S5I%PF9*O0JsDtW*1<5AqjfAIhg$Jk(&jB6>7w&UcCt< z$qS({$z!%{;@C`?I$pt+zDE-AYh3?$IA_p+vWZ)W{Yv=(Ko`D=Xo?Vcdv1G2JH*%4 zN&Cr~f3`KJoL~+F`>L2BimpM=n5)Rfj`~gMMzI-$l*_lvq)4V)No^-7-YU)DFQ}w1 z7%??=#j0)`xVmynNTtpjUJrW%3gWFr8NO%)SOCud$lVw`4mz7<1rQ>f?>Gs)!#;%+LFh10w9M9IUNIP>7GP{eO8jpS!HyCk z6DBXm(n$P!rV5ztpj5G%ficuyL6&2w$ZtL!al0@I}U|sLdnkd1c z*e&i35SPE#!IUaIH+4bolBvD8?N6qE>s%b5P5GwNayfvFujQm@7`5Xzrx*c+px(S} zu_x;oJb-UIr)xjpk?pO~!gl}uayKd1Rb!Pjt?=<)cv69GRuX^LKQ}43*!J-*@%HlV zcgm_2n@r|c;5)r0v{U^HN}$7XJsMiuY3*Y-c%Mn$uTBl0ei*e3%*>H-OAd0j8BFH( z$L-&j+<6eS(=`@?2ibF*1c1Xk%vl%0( zK1xUO+3oKA(tbbPHEr38>a?C*@vvTvaB^o4r1r=$N?d%c>bPj#ZlK={=$=hVuX^o% zM?PI<+$8!K+cll=dGdTiQEVRN(XLwAYem$wmRjzZ@#)WcJsmFB?edu4FTYNDvDPoF zv7apVu&I#{kA#}bA#aPZYSCKSU$y4+N{bl+h$*D;uZgtE^<-aNdha@}N9X~MTSc1Z z4MtN?01}VcC_~d+ybYM8&u1$Zp{X%|_ETuNSOrbwwh3Tx3RBaB_E>;((k&#h*KM3p z6|fdt*|!Cu-UO0R;rw;ZEKt?}hMB6yWs5*Iy*8XdxZ`-(gKxTC9NTd6oa=6eS|S2Z zyXd72fTdI$m=sx7)C7MOn~^g9c)ReiZP~+ep3314To*;=$B2s5&2sYwkXztS#TgE8 z`jD~V*>2mgv=ZVNWT(FqNTBaSKOTUj?K^-dQReCPe~<*6k*IYX%3R5sv563SN&k*S)v&p{gkMbXExH|{x;Bzziv`k3eZ2m1Sos>H#tfl z63jOLk9!O_qvIfHe?1x*sYlaVMO0)kmRYmuyaB3ot31jtC|=;T6ziThvQVzva&8Wf z!xmt(@N?F#z#hD_Qe7V946K=~P8=r(8;2zqsoOqac+8_BDI^T11j@d&Q~c*Zhgw^#ffM$S0E zU4cY(>#roW`oRA>{Q;vpUv``%xa!jYHvfc!G2%8K!t>M3R`r(xvSPVvi4F55Igd2L zGEmnyhWVKFU9*0uFB{KYnlBxzlhj`)ho^fFuwp$me>(^hGO+3G(y z%YJ#g$w5SdLz9^hE4Y{|^&CKPjlyN~c{!sGT4L&?`D2tI^z=I3nMm3Lw>)c00h#p!H^T>(=V5F5r#IKN(320J^>M^1{KXx8e0Ev>CnvzjFR z^vj?17 W;63G31<0*jvvX3vNuCCC+BO@lQrc(>5@>1ZqB>xuG?WPrP^-`R98^ zWScZc++ZX^X(sj?0I?+TX-<=~2j^)Wx>_t{xW;hMK)73`85qj0(iOz;T@Q)8&McLyrq(D~7(< zlhbW&-1d?%cM^Y-tcBT3WLYfhD-mgRp!KxBy6G@zsAP!u zp7P&}QJg2V6*2s#oZ+PkofW>+b_u%~>3Vrkgn>kEyF>i{w4{=JGRcSSw?R$lNeFbd=)8;0*un#5H45q!5QCz*>1!HUH3 zCq?=&AatUZ5@mw*l2+X<=U9c0oB^n--cGWYJ_y!bJs5~bBSY`B?p@FyaB-YwhZ`U7 zoatvGvXI=L3&~BEoIpPZQgYOpek`o@8e=~Ls25V`B#3yN{z;yb3cyB&fQ^(#%lLC) zJeiR-QpD+p9HKANaLbwhC4mZ!HP)N7uI3F1@-c*RiAq~TW4TO~-a3uIqd)@+Pb69# zHNYrhb6mN=Ne16D_bW|x#DqADo8d#s0(V+qPo6WY<)xB1j*7*xtlB$+W)q z-U@Jh+&>4!#d)a)R0(~FqSGM8l+mqlTg1j0_3$1;TZdxZxT!z6C7z-QdBX`2pUFjt zs#^BbMkrv%Saz{k)Pq$`|RJlB)U415vnRSp;%jBFBm@P zEvkwna*#Jf8$M~3PY^nuDg(H%d^O2bSTW_SydKp7pjC_^1R^3JGVl4qNI;i;(gTyc zFJJ0cVhHerc4M;Xd@?zj03Jc=5fbCz!0o;W@+W-jdemV2&F<=Jha>F1O5=aD0Y5U~ z9FlOTRpN=&7(7aRTdl0gh7r22RV^l@_bsQd!0tFZa(y^#WWm+T zqhfj$sGY&ZLDD?SMb^7dk9Uh`yG9s@)pRnozxRoAylxePYCsxsCZaPpthlK8+2Wr4 zNnPGeGZZ~`qM~#>O8woTELWSMpa~Q@`-wTq{47W9jtUFc+L(GPH|LboOU#Ge4>-=3 zoplsK;_XxV%hQDUc4NiytrUVM40`g8w|>e z^myN7d*S&aH_mr309ZRt-chZK1Y{pWops7@!qT&csLl3QQ^^I&tc(*>IsX`EY9F?t ze6O)@`R<@v^_3BJNmg$&O5Go&F!4{6sb>0RY!xai+9NP?`8#S;Iz1}RmS9$Zm338) z3;*MY#S>}-;oFTsv?WU`iIy_*a6BI&^LdFjCPxI1JC#r)ZZ^^3TFoc0VwH?$hKa?W zgoJpTeT+n_1DKClB*cV-n&G09aK&B#{KJU3HNfr5)Otu}#XmJlf;4@lhi;^V8mBXP zng1e{KBU1Xk|8?PE9bXni^)1IsVBrZ){XpQvo=ZAM{!Du9zc$vi-YH}=A$gKcfMT@ zozYg1wUg!oXr#+6sb$za>}%#CdZo>e_lplbDuYbaMJJA3M}OKpbFiCXKIU>owA{)^ zV74ZfCSKV`@H;CA_P3J=o@)d1n5iG$8~l!3TZ;E)xyY+Z#g(E=4%ZH$u)Xg@=(vlw z4ea6JawPL4Wj$T~q$k$B!r2#*OuYC%F4?Vpc$|Kz)T~BsG~IL^!C;OE8_%_CaGz_YEtJSc&Nk13A_T-GXPH4#4Ve@gFcDs zpy1eD=_!u`bIW^3D-?N&#n`Ly4c#nmE-9D^7Chu6M4;i{jcX0*CG|9m?pRHXAqs>RoV}ZLr%6WJwJQ;jGo6pekzxUnC zUz*Ug!Ik<2*Ix!zwW}Ofcz|MA*OyTnF2k>P;5-D?mTlUwha}tm4v5LKg}IvahIo_r zHHHUm@kFYZDn1`?GxWbIQ0|Il%e$_AW43LGnR4CD!IeW1e4R%ZFcif=Vd#Tr8XJ@s z9s!P3WDtUP;Tz720}gO6bEQe)!ByJW59i}QP13LoMyut$C;2hga$}c|0g&Z(ACl$u z7#rsHbG3ey%es4y$BAln$nE<$=N)izuh8xKJu}<&?mQd_d!xYqdMOGidQzdR$kk|l zA*?pmymF@iCOY-waSQSgrg6i&hvj*S+*IhXx=xAbM>J0M0Jy;zU%HjCGcbhA|}$|$(=i0 z>zteqB!^r^0X(BeflH_D8gv$+#~&eGA7@ovN7Q?{F+?KvVNZR|wkoIE%2`zFM{ja3 zNy`cA`jvz#-(lYti#mc)#j2A6W0s<=U3ZBau$aETjvY*vfMVwoPlZ60?s3jKfAM7n z!z3%&fUxu+B7G(A)+;W8yj8~ivsvhyo!@{GuEI)+4jT)9h1>>mPc1%}>wx!?*EtqO zQa1Mr(TK3jga(Y9bFI!gFMHv+!pY~SAfqaMwSaYAN|C4q12&5N$HY+7@}QI`Oih{b zCh9ZTyn$VL&F3`@6v=i;2{PrT+BL9cCfR0r$|&{G&YA*!)gN%#Zbc-yED|Z@k8S7D za4YB-{5D;y(PlLka@CVe4E_ko_w<^d6*gdK8*~8A5~5g&`0k4b{jPN?ni7#QXgu3q z60tFeCJ_eWs=)1(=om`FcarO%!vPSrW7Zxc!|dLkA#5jw;pebObR0pjnjAmPPfTK? zBuxyZ;H%5UuxaK z1B(O=$v_JWW7B^8o2(<$@7AQ; znkpO$sXhin&i$($C?6yYRG58_rfpJ0jzhWIJdrC-2E;G~9OgDtlXwtb(?)nsm6TUk0 z90-ePR?RsPN6_CmH~yptBhzFqn2E{eeG6>`(U0aQ%E{Oox*35$DLE9BpZcnV#?P?%69K6A4g4u}8^k=>0 zH`?W^In3Ja-}T``Q*Z$h1LMRIKCC11%~Mhdc7+H`^L89<_sc<=wjC*#c|xri$AVW{ zI)cStUGHWAi*B_=ohnE5DBZZ)mMk^a5G$~o1PxjorvtN*y8+70?cAFsmIVS+V2zeZ?mS< z2SpG?I4*HXMe@*K2bIY_thmI9CYY8Le-s17K7!z$w8583rHTZNs0U+7T%Me`*6$PV zhX`~SHZdXwVyHizMl89!_6nSJGNY765Wg1$z7@;~18jj=D2ewDo5%yI70S&eb}TD| zWxrbw<|%9%NzBqg@nUCMKyhedm50d4i%(dIe2#GA!cj8J{nTy5-cM0^az2u>8ivAMOtpEHw2+{!8yGvn7WQyS(e9D^xkN-4bc zz-E!H&kW`=AzBOpx{8(ibfqP?9tUti9B{;V-9bcnjDviH*>VI`}+&O|I2PRGOf{jsGnp7_c%516fr zhJ);rr5I!1&F$d%4e#jE*o25-l{HBljU%u;BvF2`t^tKq8P+VdDB|EN@Ax+AMOiQ~ zXG^YT3b11ri|XVg!zxP1g|!A$A|N%m!5ofncl5I#FS=evQ5T)`_A#kTwp~j-Z%+r( zGKo19@gYYGn)n9WsHJEvlvy4W)DHLrvt=aYx7=RjTM#HJ6f})5nEqtTDU@GETSJAxUF7?+rn zscWVI4#}of41z>2c3z;I!fm`HUEpDEIu{oKAP>>8OV_xZEq@^+ql}0Bx{OU5P!+hQ z;SqT2e+&^!n`1oHBE|B-v`z5l&=k>tV6r7G%(ez$fJSX!a^&|SHB+W^MND~6?auge z;4L5Lz>+074H5)nAdP-i%M~@IIM_eVV=Pb}=vGuv0p@fX-X`hxh)LTQMoNR{b_*c4 zB{2|F*4qI>@NUinEDwD34<<@&f1Xl@tZr$Gze-7u_m0-A?qzFW-P=R zf*v(hCUWlfGQFVa$dw^-+HT$8c~xbL&VL;%;kbTes_a%PR14w@YsZl(QvKXtv28#) zpUWunfBG$;Q_NJjiGH%3r3!dW#q7BaV%b^jmD-8Mb#AE)#IK4sz!oo_+Vk?IDijwkcJBd%`<4Has zQUeM{-9l7x*i`ywnSUgE%}zAtU?P+zLS#h|`^bB#O6re*s?+~1LNc#qiWMzH$#&=8%N;Ys_Gz2DFecz?36&BJl(#-~gFU>G zLt=56X9i|FVHHg$I`I7dE7ih&pdC4LZfK{Gl8iLj6q*B?t}E@)d(?`)8UuKcDg&Ci z;$btT_DI2-ww9QU2lWB=^Gt;Lgkt2g+^X220D zloV9fdwC9J>#8X<%KUUNoHkqHUC7j7BJH07AD1Dr+WBlDZUsi#Im#Hpy@I@joBoPQ zj)#PUGO9B%#%>X0Q~D2j;PH| z{sIrb(6UgyC_oAjwHsY5KHaCd_~{rL020v{qzv_^V4mu7j_c)AaDdrmxadnc z`^iJ%FES{XE`*y-nlNx7$O`{mX4Y?(p_lD8ig!=mpo=D zdWj+e;~~Wj;;wK(9$0i7FExiLv1|cunNhALJX+l;*b1iniCvNe`_#izRylyl`qTEG z0iH6<>ld&JTB@0*2!v|Qwzc|K-i@vuJg&&}&vL|DXQv-CC$pZEtDEpCiGo)UJ&@Rj zI??&OHLw<=cOe^Q++DN9$$=TQqvRjvu68W3j_4F2-3e&Kitw>hL-5D~OP6D8qm?Jz zMt++x6v0T=Zf|#(bT|`3m(Eyqrpa}Fgp@u$uhWGZaT7mrlObwZBpu8q>dlkP?O{pD zQfy$hOaT@9#{1d)JVx;Sh#25}kMVB>Ry3W%f?VCNB$YwPoj1Qf2Laxd?Ae}-X^_;B ztaQ7{>d0AWRY=^X%3b!hO|j4nR^#(}b4Oi|Fzr>rt;^?=Btn@YWo3C5&6LMAF>9}x ziZ5xuV?hm}t@2^^^HRm~=Gloe4yck>N=7MJ2;ZAWpMCs=TXm}|PE2({r#Z>2^x7DE zvmoGA7{KdM%P$wGwrrc?!?M7>r&T)!a_^S3yuZD`^WsS7_{KUl`l!N?mVl~(*f>Kd zHpWqgbfce>&pb+s2|QZSI(iVt#6pP=`GH|`1G0^IwIpiVyLAC<6G#lepC@!c3AvQ- zNfvHEFe+PX)}$&!;4pDaz?Q*+WP_YI-ou2CpVt&;AcZ*AN?170F(q{zB?re0p|^i91`eyiD@>B5m@d{ zdL@@FXg8iP8)oYIG%badc0{$ROc$v7P+CwP8Yw?D`P2hjy;Mtifbnz9ZKt>GoUV!N zXM}H}d)M_jrt=y8VxEn!JUe7YWH~~){|rZ%L(Ifp0NZ=|m4W7%KyZh|173-7dV2XP48o_}$_T}G|86GquF;mM4x=B~}y1QnBPuf3kV|sYK z*`qca5{w(>R?9J+9Emz1;!k^h*+69GY`$NX2!Q_P5lY~yRoX0)OhJV#VVmp?!#B3c1ob2Z9+=dW(Vwoenyr zuBqa{-A;t zFjXxiWb;1(lKNA~d^a!K$V14U$9cqogcnQM_B8gSKFlRW6~*ztEP%lTf{zckg+XWm z@68`<8@#u}q8r)YX{FEV&IM>_YBG-7taT~JS$^^&toYG=JY5TF6P@Ngd4_Tkf3tOz zs%HF#j8sPabQ{+s+b-KWtg9HbE2LIl(o(xyUFn6*lUs<#x^RnH^ONiTJc}A&7l4)) ztU~x#F_n;8<-prT?8Qo0&Gy{`!Z{B=1M>D?oysn{#+Rvh8taNsH6^z__{VOAG)rqR zS2SrarXMr81Ig2&)?Y&(AT{iDb2B)slVY-fPqQHfA*J6BQxREw{@9LSDqE-S1qV|H zu(HEOk$`?y#t(c~g64Uctu$7an3Flzb*KjTThW_*`h||xtOGa_0m`{XnO>g%W92-{ zM(iP}JpWxHuMEyaNqG=fRTjiAi`_9PCmsl)sQPF>@S6mg{bPVK_Jy+$XXW8B!N0(bk==)Qm`Ti`D8s{CZK@g#sAkZi! zUmk$|eiPwbUZ`uu)k!&MLF`tv z{p_|At<|}fztLz@Z+g%Jb&sA@)CJvM`d6N)5p!W4>*b#&Trin15?>v|P;zn~iaIDm zkcW!OZNjNUZ0vt69lsx@o*me_dXMgDv|`lC=1;5t7-$$=#Am{MO3QqhagYCLGeX<` zCw~4K;QxS3Jdst^ZN(rr3iLh;8C*1&&q{!;p^l?c!ywg5Mp9q*0zD>ZO_t&86Fonz z*3qJTfF`INJ7||IV)tEC?n~sjly2{3g+|y< zc;XB$>2a{A0|Qjq@fNoe%6!LrB>bU&OO$}Jmc0~&>#+B3ZtQyhSc!hP#^~X(<2*$? zgSK=HKz}8Qrq~3(4lKuEF%V;=El}QqLxd##e3WCQ<+v0gF&OCGWV4n7Me8xLE++NS9_dt1d{4%8VO4C4Hsi-;=^tgqr^I*VbX)3e37vxIyf> zbplg;?FWJ3zTBp2qVm#(-n>Z0x5F3$Zl(R0uY88Bbm-~~kucgl8OhC$h!OlM|h6##GZ`t=4{VlD!X;DXYX2s+*AM>9vSBVJ(YXs-~Z-riaCsZQFrFZ`r z@5j*bNvY1&TDy;LF+QYh5ZU{jbzhTN-=hdM|14y>QVFMtZi>p_V=*z7lCAhymG{SU zpViL{635KPNO9zW0Hbj*-7XS&#V>qMyPkJRLfhh-1On}O;*nI<_CjOp)=dLtW}667 zi*CZHH(r#taa6Si5|%w#c;C&6D4-BEnin)M+vF=X7x4wXy$>ezlN1W_R*Q&>$fMPr zPAZf1F%(alHG+qPb8R%$eGmxLJ+20l@KC&3=a}2K899v-RUG_n96n2mqy`{qWYp!6 z(tp6L(FNkdI>xdtf$9b~cf%E1yDvjkevVzw1#^4ODQ;@_`8tN{|8~~t?aH1NCwU}j zpmEhZfgSy7WHcXzotTp@bdEW_I8MRPcoiR3?F2YcM$FwaZjwFcvQ>nsNd4YEC+{}< z#33jX5$CL2pPP|6=(gVaQ)R6{Jh_JTar?ck5zNrNla|@H$fG`ylyFP&Hm!TmOJntm zJHgpg;cEz#LtH17Smjip($6UPmIJKoQIyTHoOL*Lnj98ISb{r5f64VjQwu=LBiq~Ddtt?Qb&of zHe>ejSt-Av%AYF!Y3N_{-24^6L&~JQQLKsdg#(bjs))!lc4+Yc4RWdd%XVylWV-9J zDS|KV9J{6Mcw{6){lIbnJ9e2%j-r~gV$7~v#XAqUdISC>mhiGDY&+nZ*p&D7j>q6L zHANQwlfDQ?_mhTg5qjmQ(rr=Ps27gk-G?%lN4kkdV1D)8YPxW4BM#b{V~~H_d_;Ci zN0qc>AZJhKu%P0{G{O6+atN8mX^{mEmVp7_Fm9My3iy8K{y+p=EW{rrPEAPRvo=ps zX|jf@U#7(BOfbQ!2_jHv3EG{#?&7MFM<9FrF;i1kyB^k8>3@9^*WJ{K!vcCsrGmz`hv0nL$Q>_UBwrSuFon z=n6G|v~L*0aJbT>!f!)1^tEjhm;>I(3wlj2XnyM*B0rT}rPnk{Q*B4zV<#)nsx7cb z0Y20jsS~AOMxei@x9vg)g#T@ zv;=u+Dh1=l(RF#N*53Y9{{gjE^nK4b1tPfXxz|C!8qsY-DsWRv*S$%fb#WcR^KmSg z*F>Ad`!?SXjmsyfO|vVdIP;x&e_oOocrt225a|WNvP6F%3oiknC2A^3??UTW0Avb~ zb?VZ|&CB$Ve7Uv+X82v4QKpI+V6SV~J@Q5oxKp6)=)@>iL5yR6J>MRYTOd)2fK`IF zjlJbBt7 z-=6RKdTFkuo`*J?0XfGpt2=h!IM7y#Lb|+ta|Gd;eB^OxXh?!jc~zqzpfA_>lm0)) z^9-28M?9N}Zz10hY$6g2lj-Gh`|O_&+y!rV7Yl?FqAPh@;k}xSa%UlQH($XJpw-Lm zJcas4I;JYW0aQdbX_JXhrMg{O%TW^ik7_faxrRbc#5iUTVg@W9~4Q+ zMwSpKq6NCm`uc1M8;&0q3j0YWK*WhO+Ry3+7OQQY@Wj515_1|uM!=la1A>5zS-m?= zuWtUcrg6)4?0b*pz5k!=*T>;@@f$bkM+vwWx?sJ5<)kpTFJe6JwMCMwVvK;QF>7|3 z-BRtd4BG+;sz>qe$rX|SZu zGt?5D(akuZ4Q7unD687HS5?~YL(WjfsjX(n^B!q65BKBix!G)g4f*v2onoCYc&hEJ zaplq<^)m{5o6}Kza3NR{sSr3d2MK9DwjP@<#6DO%7EhcoF=chFDpsa4Ue&&k)l+EpVDo=2pP#9L1f5K`C&?r*1 zH%#y6HO^tg&McQ$#FgWuyRD9 zM~rBGW@$2n5df_J_hC9Ov_E7!3$k*C5PrGiGbr!FkP)}EE{IF;Rsflv{k3w*XrOY% zDznI_lCcjk4~-qQ`7+CY3LxmkEwBPBMK}SkV&6kOn35#1MQ0rxRMJj3oc@%58(3ipalIyVeP zU!uTgQzUd8=)zD);WV8BTIEC7u^(yJUsKt=YFPhgj(DcVK1X7=Q2%cnPazov$X63D zQ6v+gu9N+tC}4Ev%Vj!kHFWjeLmHF_RUQWXO%I{P5+cq&jQ;D9(1ZC@_cv^vO6YOY z{Qv%rGX&^W-!Jma(iR)NDaI(niE3b&J5N9^KF7KxzE=UU1^W;MID=^}UjIN$@D@l+ z{a2DM=l9=$ZWOW~i-H}*Ws;NteeNcKIygrZh#wBjQJz)-U!KUQW*!%+UQ=o_!X*$d zuy;^zYhDJrLb(Z0q2YFQK7CkX-#~8vj3t)FWdH{yRBpX7s|*a3-Gs0^jr(V6VlA2D z0v%9sc8LEmAKVR1oH#_AE4H((E*rt8YCVMgMbr#SC-r$Gk8&@>3neob@J8=egp=t{ zE)5|9&z4LYJFIz>o?pg$3O;JNF$@KXke2-F7lk$y3l_t`X$YPk$J0I&ofMqT|V zjbN}i%YxZl$*UKABK_ON5IJ^b+j)H2?(b!tebnZs%5>_96m=yf)+VR(rfwa{p*mnIFUd9^)jxS5owPhEBwbhYU) zpQW?Y(hu^mUfu;$neLrvLHH+vXKd$TyWHjy9dG}S)$zYcL^xm^kwVTicuVB`_o;@| zAAEn#Ec>1j1o^bvL!b--Rwi2vd{&eR9UjS*#((MYT|L)yklu5j;AWye!QiV z-CA@~@VGMKb69u>K*d?LF-UG6731HYuZdL;A(oJZcs>0dU6DEn?*3c$2~_!7YFCH= z7_>h?QVswESej^U1Ep%uz>4c8U0J2w_F2mF%^OHp0lw06p!sr3W)NUQIW==M=AQ0u8@UDH2hDz?b+#n(rrLD%Kl`w4pOo95jfMYAxuJByqD5d!0+Z9fF}LhiNjlSYPqob-TjF09$uJ75U`nadA|V`^G4R!WRb;s z^I72V-)wJ~1>JzTn?KqNz!+3Uedqhu0gfiX>Y^J4l8TngX#eMV5*jB%S-P5F5@qvd z+TczscdlFbMQ|luZc-_Yo&4Eis>D5C!~^Md+KlDHrwBMlzf*MhYX6# zSNm>*c}i;*uX$QhDrGdHj#TT;3|x4yyVwKN&1Bt9tzh}i*eCH*c}6TxI^JWP8)!}8 zoO#Ox*%2O>Pp+YM3?J@gP4pG95gd@2ls|WsbV01kOwfdK;V`&JxFAy~KqADTZdCN) z|3Ho$WKF-mJvdk#<(t0>{x|?7QoU-pvhomtMFGooEqI_KAw(5!CDRt71;ct*Qboad z!vKXu{Z!&qKrdV1>L-Q^M!=}&&Ar9iHk#%(JcIh*L7eEo9Xh!`TkbMj|tpq zNOc7vrkmmf>C+`b0c{+JEZP)GYbGX6rq?CzqDPy3$4Tnlfy(5zWNQwhciG;7FD~_; zwe5_UJG0ypkcW4@MXg`g6h6rGk9VW3tQ%^HTF^2wGl3}V3b5bcuv{Q1Fc^-fGz$_0 z^F+DcW7|f%wsU+0I)oL1jX{6HGz$Xd4^U5d?HwI^zBEDz0;qZW4U0PXECvwo+hLkx z^S})Z40B6y$>usgC4j)j{`X7hH@cKGrrdOR4$a;SIDcVWQzDfx1VJA!qlt zyiquFJfYt&$gV!k9}Dm8Htt(vuc9HuK=;NC*AJXeFpA>0d8yQrnohE=q#c( z$J2HEAI7qK{z-z?@8&PHXU5~>vFRP0w-Y;Q&@I#vd$f&*w`AlaZ z$a>yS8|#AZgzp>bKq~L1NC&$DE)>Y60)H+PIV`1|sZg$J`I60=U0QIE<%YoE-e_(NE+=GL+D#X_Iyau&s<)X?;>O@jF{=lwanDg(bgI^{4uHP!+WR z@nrettjXd0F(><;ZWqBc?aJi6*yZ7qi1-I(!6hVD!Ltsly-!& z^JJz{aX}0JjWc?KL&y7JdO1%e;+KY(YM~xxRKc4M=Yp=*$5YQF{2pDP;HO;0T!#z+ z{O36agRyCl(VeoYayU12B>z}#D{MX7I54RNE~BAX?KT%k2#9Wd&@aRs<`_0>tD+1jxf}J-O(@x?}7m;CM@EQyLB;*=*pnaLt!px(huA7B*~Ko?FBVesHyN-mPKuK zYD7fGm~p};%M2XpcY%jZf|>RP`dJ(2hj}tkF(ETw0(65E49jl%@riPtQ5~9B&+7vd z%Cx(RRprY1I)t^my=2S}zd#!W-rT4i=;QO7$@E+G_OBuSnIy)~uKl4n5rdk&mU=r9 z2aX4k!QehVgKKR&n9qS#3O4$dG{8*>DMNJKJ{IT#U14;tH4C^R^1Z(fQ^1=}9O8Yx zoeIt5+FB}4*4+7RU;a%p^r7MX{Wk*Y!}(FG6AK;*dxy>1jQ(E9CzT=n5sW&+&m1#h zwh!kVgzy@bX>n(_M~shg{>+E%Pem*H)pE2XaFz>PTL&EvdBt*3SixwD zf~L`I6>Lg42yM=@_eqsBzB0vfL}i#DNr!h^A&fu)Fe5&v#9X=QGeQaB8Lei2bo5FS z?-n%`mBk#5ryEeF)F$Bkk?3xkg2tXScuv5J?gkYc+y@$BjNX}o=wnS|tFMOBV6Bp{ zEjzNuWI7aV_Y(TbtFZL~r-Trc6Poi)t>fA-g!sp|Glr9CugjvIuIv}>h0xyfleUu& zrH_c8x{+m^(z8Yf8&RXe$u({vLbA5dkFq5`iFw=c4~lb0rY^{u;GQUdkAzr{g!LBl z(ufhhvck6|z^-UUDE#_-tu39?wn_sXg@80oAsc$1UWY;m?0;Ud4X_kS-)AjK-HM^T z1X}V-B~Q#N5DaMgxkABiQ5@+5GHh1nt@E3ru$u6?%xdULpN1G|PCIKfI~uw^-Kkq@ zU#ju(EYHCp2N`Kkd1q$GTu5=KL9rx`U6@S{9UGj+C9_uJrD)O5wg5t^&A zRw-lBcBseVK0h$Mefjl`>2nLy>%0&YMM|vY6b3Ud0cbxYG#v216SFx+#Ty0)_$@9$ z<0|tMT56AI&C;rLIb)~QaSIT}5D9r(`$#A)<5fXsTsInwcVcUBhVj(k`YwciT4Re% z`3?MvSw`KcMSnmzVCr5n<&6Ym?Q259h87A%U0;6v*^(rTgv1%baj-a~saSn;A>ov9 zO*w}UH_^mlF^9tDMiqS^ECj*Ingja>i%zA)HwFHYa`S#FS5zLQCc$oNvb0{ZTP_!m z1WoLb=|9#L5LhDyFtw!mxlia@$QKJfu1;^$#O%*GX6JbhG>M8ga&NAeY_;c7_|uzq z{6b7H2&EiGy*KEA@DH;zqG_!|OG6J$T}*>w`bHgbv5KOfyH6hBjslg;T5z6vITx~Ao=tZq}zcPoSj5ZewC-AmCSfFkFqVlzi|YB;6W$)8@;VmfzOT8;T40(y?c?`F zJ_=hN&Cq%_yx&X)hmie(MbArSi16IQR5t4O1CO3MAXR9rP^3miTIdM?$40}-@g??y z@A`O|eEeg%#6005<)1O#4nksw18DgPS5QB+TIW(Qk<&iL(}TZvL-$ZwxPm-g4Gysy z%QOZY&1zq)vIrC<(W9R^Jzk~|dx!cfhuvl=gowCL5F}7T+W0)vUUz-;?y)L>YxcQr zw9p9LrGZ+l_1=L?5Rm#wwm4gg2?;Hot>{a}1p7P{_DJ~prp;-E2H>bTqIAYT_J&_% z^5;ic@}2J#5q}-#>2&tKuy7FN@?6=yCNUDb6}#daKo!b;m+elr+53|uvftYWKRU^n zY-qqTnG15Md|0OLQ_zhYI7QEW&Y!F z6axDmB}5iPSBV8(xSd}_95!t?1Hnog%4&dK#GY<2`AI?WQrCw<3*~t@L{>md0xCm1 zq!XXf#&@=X!03uTqK$tMvB|DG*s)=V4I271h#FhYmPI=}_tHDyVI|QVeANGNy~V(^ zKZHCtV9J^=gJ0>stXhHe4G7)>tINzf>SrGy8*G}5dnaGgHeZg9>0o4v!#z`ilg)op zvJbED&9Vl;3v==Q3^i041q>Yew-9_1>2sy#U}JQQ`ylWApSq~JCM#TCH#-@TYpw`z zYHe88U}7Kk(phG81VW1?@nX4}UtQ;o_W9HN`d0?Y`{U%J$()SHW$%@m(N8ynq>g?| z=0R}JB&;ArcRi)F`g0cXG0% zJYNE;uHJ@ly0xKg$OW{~=WP!nONyc8N0U7-iiFBQz4>&`w{kVB0I20^RG4m*9D)A6 zkiPP7oJ&m7$z^sJHqnQW@(>(HVGX%ClgPi=2y7&!#z~AM=5VJwwSIX10q-00 zN8hHrOcE>KAQ9vuaVlLZobi{w)@(oRGOot#ecLMU(;dcT8Mx{h9NnjnNGlphCp(BR z>wSE&J-1bDO-fXO2ktkL&v1e$(er~g?e&V!GNgClGO@=b<~lB4FS-37L6d)ofX&pI zPLh#>7m02|?K+8pK9SoCK*9kCmI~$KXsxj0O%i(hu|SUk%6CspdNqUvINENo|NV`H z1)6Z5BuB5n|8u_f?5ONXrEgS5-xwu+-m7e93NA6uMfhIafoloApTP;OS#XHTs4PH& z)K(l6T#ZRZJJHMWDNmu?j=?^{%|TJ>{NnYA5QWy)DE6ZBBwUniFQ%SVuN$9JsSvL0 zYZ@n$v<9vizfA-dM@K4lQm?&GVy?(wp;j`d9MU%%Ts&XCh+mx+^+`hIk`rI^V*~oC zwB>qwkOTxgjz0Nnj(P8fUL`;gLlbns(g%$3=(9i;5lp#&vS8SumuUT|=clnBqA#`Q zkCi}-T&Oes6YkhUl5=7CN8Mx z#a0FkVq zSQFK`A3yNq9E3lsMWuMaljp-YcH7QxhUSt4|A{*n9_eclQpZ&YxZq+H7%F1;);LsK zq_=YsoQM!3cPpDb=q~NY&a_zo8z>&CU&w(Diu!VYhMqK2q)?^BZEr>yofq4Q%+-z7 zLNWf$88{Jvj?8!Y`22tDeN|MRYtkhj?(XjH?ry;$KybI1F20o@k5|>MUAuC%jG2e=|HAUX_N>SjMjkwV@JmC_uYIaDRt16G|pxwEdJNm7)BdqI{;wog;wr3c+JvpKfja z*#E+KfrTx@G|G~3e`XYJrS++(#zE_=diUks*k0raE6+OoXDTSiuc3@3b=jV{P$k~!0j{SK`@Am=5B1EzxiA}EG3@uFtZB#UMm4HAdkPDhR;Yv zMUGS9%D5;!M}Q?W=7Wr4z96+90epM5Js2ML?wm&UZ6wdV+sWYBhK-7kBREttdzCzY zz^1B>0V+b_Dx-P>Cu4QBhRHq>$MWQXc4MjVmnXD?@to1po$Lb@t)0`ABNoj{r1OYr>!#HZB zi8)kTY_+{CARAd>zeDtsZ}$WaCTgLmu@VZnD=;ELc_D{cNqvs2M+FB$j%Pr=2rZ)l zP^8)l;Gr5tt$kAb!$M`rlE&cU&nsNg6KUbiGbS*{YeFO^xrqJ9R66i`ROoxkEY_&j zBIJZUz(allR4^~is>kIoyw}%f4Qe&q-rtDAxFxkZKgYvA6`!IFtZ0rO23$X{om6Oj) z4nB?`$Gt+k${JEk9o&~>{u=xB<$LKvmudOoT(vMj&?V(CUml29E!f_!FIOHonP+1g zQ^W zm0Y2-@pv;K@=Ke)FK>u`eMRP zc9)iUzd!`T>TAF)<;j=W0SLY;JLy2*|du47eNt?6rgZ9l$*?O-O|h+h@lC1EHQ?gxwRJj(Mt)(;Ci&XtRvndj%7Cy~j}Rn!Kp6oJ<;X3zUZbsw zfQ^gHh>D};R(79u7$j>*{DOW6U4xMxtSw{l1xh}r@FWZpm31aZ4+wuNd#`^d@Zfyso&g+!DqJAfye9jVgv? zhQQu{pD&uNkM)hP<{oLW6Kb(@Nshtv_ODcX5cX4GQR?*Yc)U`kX(RBx|1kbFi~7cA zG|>vp5DWv6ovvR-RBw9Oi5t>aXumOu3&R37hK9O=3{hooLZ5h_)E)a|F)pQC#OMu^ zt~Y4}d6(*@_qd`(<)Dh)8vZG%mc~Rj_0@>3L%aR*w(Cx?{WK{LS5rwYzL{C0G}~P? zmJ%-#<`jNGgnlO>oA^XFabamg=5h#8Q?FGR6bjKvv~oNrZD<(Zwyk=5IHqnTaj1}1 zfYUSetKwbIQtw6Q$U$Zfi^Ha~S4_k4rNtG8UTJAwSQE;myYAeN{pXa=i@*5YwkpI+ z93q^Fj4(o*W?|xl6D4+V$ID`JipLG#y;wJthSiLCkjTuAtzlvoo{YmHyZ-zo)ARGG z8)I$I!5IEcUwvbYpb>gEB2c@J&+AxdB|bbd`$0aYQDHDJgel1kbb)rM_KdR)Pxge< z)z!uC&Y9aDVgqy~9A%~(+ievr5U(}M`nS~rEEs!I;IY0dH8Z*PTZuQgbBE zO!u#RIxn!z*Vjh}t^O#_O6@{@zoKP|(bsjpy$Ddu&(#IR>r{rI77=gP>)8)wC<=83 z(s+C@srMNM><^4>?RfKUc;#R}h%})+v3nm#8Lkc&#?6qNQ5E~1C}HCQg<8F?_!Y7- zxy&M+`j)q^rfdrO13_%$>ktAzwnQwRy;C5>e{QbkRi*UTUmlmMUN-bEvU=7-dAY98OG+apL|*;W;On^mK$#VL2laG`tBhXIVN-A!)5w0g-WUR1D4P0?LLpyJ+bm4740_)X_2$w@-VJg; zFw|FFeICe-i90E->4HDk^~uS$dz|taP0BX|Pm-YDuy2+j>7Ts-0*o&F8N(e$NlEcnlyn z{RRJvTVi2k&x_5nxEb`Ic1QK{lP*%rrLOyon?|FEz;vxTg9Z#5e0E8#u0-vL?nUb${3o|KJ;PvLZ%w?a`Ll7 zWK=cZPg{m zQlC3hZn+vejJZs^e&k-M$fmm+@Yg*Hy4d{&WXiV$nYO&hPjts3*@W~3he}P@Ci(Xm z9`u;o=ZBWHhrJ+e7A1Y38Z<-l_&BV0#8VF6tLC+^L2Vkzr$CmSUvo+mwzXx7qu9F2 zq-3js#(LC?YHxyHU%#Xn7xTEW;Vccxed$-4pBxI%lGNV?Ee%VWq zb@3VF*^;a^Ai}Hb!=nLRZTZLe9ud8*fW@a)h%{)Qy;f*|=3~(@3=p~n5SATyV?y!^ z5ek`M-wUbKdy5}^@uuvX0U{k@5Ewjs+}lPMB2TFAg&R}fq=0uz-aU+7(HZkP&f*CQ zpDSYS={`AssYKfiO^m4h+|mzA%b=S1&_937R94NExjTC)!uDq8SBqN2fp{0l_rh`wC_Hy1L+=NAmev1o-wdU zs%yS2u{Rv&qWv%z7@7oIiFzpuM<=i=I1ae-9zQqr7e|4|17ceg1kxV(OPEUz>N4k| zb2B0G_YdDcPh%i3pcRDuIApJ6zu;R{sUwrTF{c6sy617gH_(=7 z8cM>bTFee_GRrGE+g#Y_sdI7V^!2RYLX5eT6#{umkP1NPtr}DJ4$y0>{WN|io!4!3 zJPkv(K-f<{XUqSZIf$Offj)`A#ujdXyD~11f~1}um;BAP3C{~JIK&NhA~Jdk9F75E zwEW8`ce$b$#bs0aYqy|M^B`(H<{iqf2qF*&ZL}vaFia=R#>>Z3mq3V;egA3TYKLtH zRrEm`HYW$S=4RsJrl_b01OlBG1->B%)PBR@vBaYz2zzYo6aT|9kP@JcZia{3LT1ZT z1zE-g4o^EeEf4j#!;9FUwi*5rSb9RkmHK=edRbcJ1B)CXU_uOZ_9bO$4RCX}lsQ5e zV7@!N)L?J{_OpNIL&kcY0av!_?O9xfE)Vm_DKdfT4Wo=qgzf%h;83SP=eXrgh3R!- zXi`-_K+&+6$POdx1`_V8%{JxMHH<4>-xq65LV$v{N}w(xU5^4s>W^j74i@boz$caw zv~TUwmsF5UU$%#4CV$8LSt7Vc`sC(ZDkitp!6R6X1pnmxFo(^NTy~%q5}H4j@c3}Y z7#AHol+T4+YGsnn`JnM_q6wbd_TFa_hwd&sB?;w{g6^g5&pbAtvp&vsc63Bv5M8h- zZhTn!j>BT$d%DcVi>#2x&jma-l?2Lbk3hROKo&K^p?@$6V?3Ry+OEZ!g~#ec-ojr_ zqx5h5W!cp$MT!AplTzTgF`?DZBESfl$XTyxII>#5^Amu9Stbx4bK7$w8&o7htg|JX zTebP=mPe>kJ?c%H9+(tJ3#A?8s!t1i^U&XNJL}_XsJud%vaH*68lt)Ta!0HHHg%j$V8j7NJJ-%lk_!Bpl`DL402ok_Gu)MR;Fu>5gaJ^Mn zeyjAjG{CnhL*=gg{+KjzE43g|jls8ZaCVFq)>g8RWFPb!$k)daGVnJ#%%q4s&JHLJMIJlc^xW`7Nm=a1 z{i&}p^9i2!Cx?fmu|O00H%IM~INwfy*YnNiE?dP~>q|K1la~D%Y`pj9ilXz-W%WXB z!_$FG%Pv^7MhU56)ZB|Z$8TX61%eccbz@3GLvdqqNGZH@Fy*zWgj1wkaGIOy?jZlo zO!y&K=12=04;vpJlTN~IGY2IDy(a-ngP@SKK2`L?AtI&Zi>WNNadS&7<~#)-xxpnx zN8j8WhG8$IOeFAUNTv@oHP7_LEP56jY==??8$2l~DHEI_h$OVrI*a2wi>0#WtX#s~ zxGgIJ7SR8Z>q+X8bG5#|zkfR;iAI^Yo$gAQjEqd_!q8d3I6I;DFW??Ob~$3|BJ?s^89Hhb%RsaiBxf&e~IU@e-=eqvUiOmTb z0-aLC;o^SnW|Nyo!S;HK*MG_L9xlQ$MvbZ4`bHxySVKjVWdf7Cn-rZeMSsxS;8?(B5@Zm!ffP z!X5zMsP3%8h9gCWA+-PewUp0Dts-EWY+~uwX2j4(^X$Eo=vo>G)KR7=6nbc@=)Bb( z3FL#ym1?whI6dnZGCob8lj*h9?-c2^6;{6;*?+d5+D;5A-t25WxFFz+)NKF{A&|ir zUdQ62u0>)v5pa#vZy;C8rrGqFx``n9q|eFz4OTf0$KagTW75j9QiI`a5N;d}yj&qF zTvhrj6o$glSyW!kBC%1 z+jWzk!^I$RFy>up8@^I>@S4|~ zbA9M+I&Isiq(PJsP+-7jIld&-+1c3Mht!Q~sN-UQ2z`B7gCoX>+~^(Uu0c!$^m~_l z@2^>{we&=z88l#E*dk$X8-V?=?6W)`{EWfMS*Z5>t%|*|>+*DLiN1&0?Xi`+$9!Te zklE2i)J*1=+=jUa<~*HkfWW-L;#g8c(qgvl(#X(|A3y*XFA47#v|s1-L}Ju#+y@8- zEdu>vMSw_qcXxMW{%>5*Nu6^4U5!?d3waLbd-8{3LGNp;P>yXB*eBS0K{hS=$6r5x z?(I=PqSmNZsne-q_Za6SYK8;NCEQQu;rxK=)1|p`WeKDHP&oMvW-LG*3;~-EV91Hn zZ?l6LA|@fC>le<(6m3@-+3QOgdQuscPv5uNt~lRcn!CAchyMz46%7;tC+9t!%wy1D zP_JMG=r4C%y@ifHR=If1J$-yi;Jj#!=TRveC~@_S!g+mePd&EqY%75d<17m3j$4Uu z>;IE=A_~3|DcXUHkNR?M|Lief(!ZX&o}RDm{rJUB4D#e{fwdp1R*-Jj#*K4RcDiUA zu_)M>h??wKclV=8FPW1lm}@D(kf!wS3|d z=Hc2<$uCxI*kBm~FA4D~b$d3CLr`M9$Te%7r8y<@1(Czxk1Z!-(UUMKWbk!oa12mL z0&EqA6D04b^D&#$AlEmu! zbd^wYvI=m+U6aU%*$^Ko1M9~N-cbq@@lT5vwH)0Sckh-y)7ztL3D>Z}6dL%3T71gCSW4?@(c2`C zcs~5b^JA=KVC3ar1c;l&f^}PGQLEmkfTQs-Hi2Q+s_8eT)G4)qsrqo0CS4^zgT^t& zT;WyzBCuN>;V|t2m7k*cg9ct zuhCnv158<#8MA@*0VI;aM#W!JXhLSMkLMzgiK1`TJKW@P@wsi6Nox=zH$`1^-#Dag&o@3#4@1=d1DN<*MGFz)BgIn+WEqA6IKiQ zc#NRToV#h#!N2r~wCV7+Ows?>Q?bucbe?X8eOxHA`1cTUdi}NqgO>4;_^cX4R-~tA z58hd->l3UPb=N~OXcJo66B=dw!PKOif$*Uk{qM0HjH-C(#Xff`88ivRRV&j)!$u%2 zPK!xSd^Bo+p1HOpj@p+mq+~gXy99CbGF{}z2)9(B%2uz9h`q1QOr{(r2aiKUx4cWX<8%8`B=o zLpS2+>Fks;qFZ&y0aTAB(R(2Y855S%A9JuB=xdE|170VdYvo##ldb!PTL*YDSZoQ! zabP1vO*=Od6xc@GGnfA|hepQt2nj3MfX>iU_b=qR=!k#F$f*^QcUq5$kQY z!eoT7PI?+P0)SL0+6XJU*+d+Mlfdq6)SU)@@~X#) z)U~ov`H|QuwoVS{fhQ&x2ZE|`s+g1a#i%6AYx|o+i^v42^(btoe(Ve=so97QrEm5jPCSLWft-&@kkM$5 zR!6-_AVMZVAtpius#w)A5DYqWKx(1tfx|M^v_Pb;+Wro+^rZ3C1-4p@Y)YJ+Cb+eH zrwe_b5UI;}VR{;1HC%{=U4gHY0LM{c2WABufKC#t13J&v`o{wCeUs55gKXoA-Nw_H z77>wvXUf@3q13;$WVgoG$_z#Te!3t@6D-(Mm1U1eQsmo$CxHV~AUIJH37QiiQVyF& zz!lkFZ>m&Hjteq?D8AqB3yBm+EY~SSXFx5iST9y&b%}Q(|nl%U3$0Y4~!r&4;F~Z$;?^*P0^S|`JjaLU8kX%le0o6 zonO{A_d?`ad@XiH*AYqOz%Ii)T{5!S_PtuJuip)?l=uTqMab8@?}GNZbjad%$p`E0 z7{@z}o6;bMFcd=84h~U7>e$&2adb?apB=CBvlqqYvNv0KkI*v?iEi@*_-`_!;)ynD znJ8#S%@XRVQ~YmK?Ah9$V6}6Gl{?j3h z7RBb`j{!H@9fA`oXsYdwoL@p@|zUnTz_px#-qi1|ZO1HVMuXtzKb! zBAcZmO(_-M?uc5pj?m9MKNu$)55@yiii{>Eii!acBUmm~#&m6<2)2-I^k-W(cVoO` zs3CY?tEmH>55~ymg0v7XiHgtFwer|8>4ysJx9PJQ$BIgd!trZInK_vv|Mm8_d6W-?i~f_(B1_7E%F4H98MZ*#4&8-n|F$CzQ^5ac+sStQuS0{CYtDztL$?N7 zxoV)G_T;-(?gpvMaG&q5pE>qYen8M&%aMf+WDfG#D5~D1v^ARF$o5rOXeI7N23wFT zR(DA~KxpR+@;!V>sgFQu79k@u&R~h4qXEP9HtWbw@b$XfV4A(v?sriDwj@94c{FJ^kX%=5MM2c~J;sc}?Oz zTpk^+rUhXnW_dX!V$jc3z@B5A#lCK~yTBY1JIz&TRFg@B%Q8S%JF1w=%EH1|zL5wN z8&u&P1BKbTzKNw#%1u15{p8R5C)39kW14=T9^U8FUALx*C2ZoXi+lWwn*C--wqzNT$+z#&2{1R*LhN8>5dDX>*)>Y@wFcxZU z&|A{!dKB)d0&1N+!pOAz~`zJWzFB#;-7-efoW!D1AfZSA9i|R zxMyoek%WoaX%sA*KYIkU(9>SI*YMrG|9O_q}6VkRj15` zXzqJ!A=KIcfIwSG3`E_26eGeS5H3}&sM7}y1#hH4Da`Bt)J8}MwM>*4BUUS>ZDBl! zmYgjKFd@z>C(gHP-D%Xah$g!1c{&K#jFEykxP;-*S0((j z@70jy|G21l(xTNa7R!Tl`4zuu1gS(0emTt69ebf9N@a3X^Fh{5=1`(`m|SZ(SV+ea*D0))9Nz+x)xqYh2C~k6mYn-IaOD9$2o4 zN{IJUuk<h0U_bY9h^s4bRTP5!{l|M?pLQkOgDsEq5zK&TQCqSlfU!Vlg7{C(_6 z1ni9H5Ul-5cvM&WNngApfNa~+_6Xy$Jhwu}j) z0YTxr?|>Eg2$MlsQfiB`za0qm>AmUsSMY%_1e_{I$#=Zor>o8KgqvCOMz%D6xVDTk zq<&lX6_vte|2}w#3C>?4cvBAYsav&TU17foNspmB& z`3p4!xGT%&R3i};=?rctZJ1w>u`E-ov0Gs?@nCyH=}c(5j&V~W9t-=Ks1!zzwK`eU z*}_8B-&9f-L?=x;r+WgfM@Vwm?G^G*gGtu(PaTlrVb7Y1lqD5Hy>N2f=;H;j6vsOm zc{1MEBh_HlW;8Ee_rFQs4aM|jx52LRy6k3~29r^pfBn!cjeBH*ea?D^TzW{KUY>c1 zoU*?3>dWNNpm^hos9j)I1lR~*;fBdFcm(ps71Wy>fQlrhm<_T2A z#b9xW4Y;zp!2W{K!u!ql@h0MPhGf8e!9P1Y!=hJLfoUy|Jp<$yLfoMHwM2@Qq!Wna zR@kVOy4L{$0${!ClYrtlxDgni;ZnpuV0<$oaP84)qsMtstd-CsnkSsQt%sQJ3v$}h ztFLYwOFQ+V+A01}%g7d!vDTaGr54pL3sqPV8L!rgTit9fM;)UT@R;hqF&Sz*!yjSn zCY%PIWXQzu+1~3_b?+jOV&GsMfiPJ&flIhvXoi)QQy~|(*`qE@pR__Odh8ptwrK;{ zYh)X5#Pag~J=fjcw&!FdwIX9p-!9{qDoND1JSR>dQ9D~l{5BGi)S5WaY=R-I8nb4$ zRmac?p*&o~`$hFth8#48}|xRoZ$-^ISPd${8sn_e<@=6{lS8o59S(Kf4RBMUQ_ zlG|ry@^R|Q+4V$xC>nO}k3L+jPH`*|%PbOjJxt79+Oh8Le2B%Jn;Uv-tHL;}Z8`tN zrF~y+W%&C-%-UxG@k91OQrZE8LZ4q+@68iw4Ev9_&v(vWo-sb3k07irR8xrDk?K5P z0t|(Dp9cD6@RuhJlZ_5pu0c`G%5dmNzZb80fctNK3ER%X2*Aw0v3>J6p#M0w$By{j zeB}%he&#OAbk(8Z`StuqIrn!6yyh74$ztTY(~67=dQb@X&`CuXCrZW{?JhKk4C*o| z2LLq$hcQxtmq1l>O}E!_m{`?n8%hWP{>^e#C`$0D%>_j%-fgQ=&rQX=MC~ciHL?!` zb8v*J_zyNg$w{2rkV^R1r;I}uFY~DOtgkN)5rh~~G%SdntmPZI_+i$ga;o*u&$F;L z_7~s*02*BnUU5?glw1Rt8jH0fVyJDfun5o(ZNkQj7Zc!@U{1oKHhI!gpEyMi0+9K6 zK8ZAl4EMJuFmbNuyg)CeS&2lHal;R>Iwx?G=kKb*(eY(X?7pTPfs9G}I=CpTt zI8m+n7vj%KSm5MSSk7n-G|6#I4-6r8f0(&pyx#IC5D+my5#9iowFo3WgGe5-E^iS5x^^ZOBFZrFw9ymzu!9r>Lur_MIZ*Sv;_>z|Z^ zT{C#XP;G$j?}8E1+?}PH&;w98*F7Ju=^6ze9v-S_#B8h)~+M@?IzW;}!c?wXY~SI437%e;>gg;q<>4 z0YUSCMjIK|^>PLzBINuP1-U5&T-blGCH_Hd$TtUvivS6ZBer><%R+&2-!z;bgz-wD zSHhH-l;C68rAGHH|LIEt9(SV<2p_-knPMNBJumS7J}Il4lKH=A66`Tx5;W{Ff=5*U zX@LLvIfzgeASe*tskBr5&B#X?fmxYhdrki7k^M93gTI?f(C+~Sp`_`O|II3VB?qV;7D-24e>W9~-y{Uz(=XltUWn6kn#xM7c1rbuTw1Q{rfwFj>~SA0XCV*Xyx1T7ONTZ+K401zq^Av z;ITiHnfegFC!j_Fi6&G2ckh<6xoj+!yK*D{=5C$Rh)RS!*RUJT4h%>zikHuszgV|{ z|IM@Tn}}3|VQg$nZd_DeZt2WrHna5Q%a@If4cm+G=j5aX=j&vtHXd;2SzaIyn3UGH~S876uE1`GX9fZqdDlIYwauZv=S8??96ffvpK%e-Z zo1XA+X15K{%?d^dzSZ}gSAjIH9`;b-y=&JmV zFTNN75_Yq0;x4;;03ZtdI}le8@wFep) zdPGGM1;hp`{-G$v+i$>ac znPN`AXubOnoi^^?%=0mZy5(iouP`CdNazM|Lf+SH&IfWTDyaKKrgb{aa;}o|5W94t z+C^#~lQ2(4StTNMJ(&@S1+`MEHLAmTkwYy=%|qyOYc7FJH`+*qMz--YCXb4b zZ&|dqdaYY23{js2U0%^vuBKdVLzvynu%q=TVx@l{^$E=(1xy%BmKw2(>> zAhO5OnHV4wgLeR6nRX8rT8q5&N_?hRYR3HbbmfG%m~n&-AYPO$bV&oTuBL0oo-Rf> zSQkf4^J+x|%P?>ajWh$5f9o5C?E_hLz!kMC!O8i1J6)PlBz6i@_wfAu?6BSz6g>UO zZViVLs*OWyZZpY3%@dUq$muKR3pOIa1NH7I4(51C2*t6V&_+j9I^FGTEVV|NZ!zop zua(XMw~@^TQ)7bs0If2AqZ2u}P^ka9tUxl1^fMw7EYRxNcmTLVwRX$lS0Z{?zP`RS zZmGH*{*k*74I9Xs9}k;% z)){z09FasBCO1Ms#QE5qLn zD=wiIm2oWqe2f6k($&E@jST7x>gA$z-^2uxjKgN9 z2TD7ZT_n{CTD zo&NOxZP#){EvOPcaa~oC84?mws|nvoXmo=VWfT)3kO$HiPpt{J4eR>nmIxO`{oi`9 ztiN|OaYW(e>1wJT-1XZuXdV#hx(2ky%7{tkAA!(63Kl3dYau=UZA4M~Sxtu#ZN%_^ zQtQFKbXnPzW*a~ZXPjaSq_hHLLJo%G2pWj*@U3i7D-F#$;(Iw0kE1g}{*WR1hm|f3 z|ILSRydziRu-)juVi?FmZadX0{@P=t+YP)`rX%rK7BN8hXy=tQvn}~#&U`;9@N-)W z{_25IH)Mb-(Ug3gIRSyNxq&L>l(Q3Bz4ZmVu`+!M#BS6!aEdTG{ni!WNW&x@C zS@`<;y`MsCh*CzmG*Vv5TWN)Cro&M+v)KSPp!RqqG@QSq!#kBZ4n-ncq_{0IvK)uy z#ME?Xbke{e-+X*byO>ze)A`|37Z;{$$J0c%`2!65*DVsQ>zSE+Ec&+nsRFC8F0wx_ zo2MoahZ=qVA`!*U;b^3%j(OejAmGvQaJQW7pT63NndNF2YY^amcV0i9QMA0Y)-5z4 z0{v^T=AuAQ`}wZT+h=L5LE^~v$$L|``DEB*s}lV96S`Y;>A3fE7@KJzRC1cd)UEv- z^)Ykmzi^BH`Td^Iu&l2AixT1d?ZP>*MMJ(s1++81il+odOt#H zHANhk2EyR;_N?&7uNTrUu|#VA`o86$``BPCr$U!ZT{Ri7a(G0gMyaE4M-l45%T?pOY&emt(1cCfJQ(Z zj~MVRnPOpext>)bSQ!R}#>2zM)d#|^^hj1UJU%lKl>lRCXi##!%^0`GNGvM})cW3S zhfF#m6jwOf`}5n6dhfXyNB*+iq#y!d1oa>qJVg~8wD%;w*A=IGnxCvzqA(CXS&>;6 zlL+zux?Q8Curz4s3;B>)R+tdY576OeP)NK}Sqyhll*#Qk_+NWo#Tb}xhY9jO$Qu%h z2tPfz%?vFpwT^XpwbN_P#|G*mI-9;c1r`LPm*e)L^jv4MNj=`WIetn^qj20CUX#xt zN0C|}@rUj4UI-Y&JkCXtkjQEPw2U9s*YIWPsi6={AK-uFKJIh%`%V z5CzR7A~B;aQvKz=_gTAh3bnSc)>vpuH17UtES|@^kE!CGsmpZ2vL?%^exz@kDBIDo z|KiFza|2k)?DoCzYV-H?3f1f&N7CK)B#IbhLaoDqR|yPP+5MvkO*od2VhCt0AHTQ| z#3B?44D2j0(x>ZFZPy-7<6w)YicSw@*PjM93srgmKC4(Snd^JAuw1UZAcyFAGtTm} zR$uq~udRu#9hr@uggFc}97m6#XqzUBN}?T6C%;?U0;RkfCM~QM&2G2v=>-J^R<-Jt zYWU@R_1reQ57Qi$lX1j(ZIh0__Iq9@dwv-dDdo{C6nu1gfE9LUyqhL#MmXgHvzeF% z8!e(y-bM4#tnsWe6iHp^_N*x?SBfgZ3wHmB9l=!rUX@0Fx;xZ}LEU>@0>5lHZ28kO zgSkJH%x6b+cSm@+rFJ-(%5#zKQFQDt#f!gR0-N8*QJT6)nC!lQ>t@+V1>i1-287^* zlypAkQOOtfnMAJjykxg(GF4=F7M!YCg-aZpBMJFn*xdv7BcOf-$8P z67|ZqZ&q3c#3a0dH@$Ox)lPfjbzQ|DkrGI`Ip@)ck-VSt4#ttnbNh|VgusW4a3URs zoYG7PzwXX8P!M};FV;F5u;SaecA!&8_lEU-S-_e?=%x%e-Xlpjeez0gP*o+Sk}sCl zk9{`x^;hbC>6jTSoJ;fnrQfiQ+@(?g=yMkU`n3zyIN9Jl4&mttP+A;qT~Z@e2smSqsE(DM z^y?kl-UvE#2Zad4EgKEr?=403xm7Jrp+)$PrPoAYF_+vI_W z>ZUdPRSR9fe`mE8g(PXu?s<);MNDPW-&}1TQfe^QD(odoBa!u`t9pK55cbCCBZBlJC~G6@s>WjLlisPh36UGy0zHP$mq*RE)oE z{g{X-dH2%V^DP zzOQ804g)Yk!jEq@W0?XOGm~$`_JKjf1@qHMbT{G*Kwc z;`9}`JjD7)Se)GRm)I%a3H;hhHU|Z)pX!pb>lvFu{ANP8^`mTCpn6jN!T26bwxHm7 z$n#Snr18D4jIpOgq?_hS0srLLZ&T17eM@4-z z@q}TI{p*XD^G+~?!G^9JXJ)4k7smx>#VAg|hd$9t0`NYB7ZN{8Ted?AymjT8BSY4V zR#gZ~3>!EKyKA`=YRY`n_tY0j14oJ+)R%87xGL{OtM=;ulw<>51A5>AP{;qmOraxo zog{qT67J~wb*b%QgMjCpe*^Ur6%!K!oqQiD9z|7E*%&;s$EavJmV$hTD=l0%X7hzV zkNQ&FJv94=W@%)XpF8}orCN#9=yjI(tW;rOeZ@Zafh9?QM2hlN-cQ9-i;Rio^v7iK zXXUh!9fDEgKzI1dS&+pa@m9&CL)x9{xlTpPz zMNS|llh{w;up|vJIDgMWGVy{>SU@*9|EVX&*Z>FTiW2+_KG`@tMd@WuU#AqEvkQgCC*o;;op0is>?Do)bYTb#$&ZbgMYR&?1DjRMFquwwIEfIOo&G~U z1g<@;uCTSP3Y3d04#U_X<|z7!#exbhEFM|td)gg%WMriwS-R6`vt06iDC)q=w`BmW zD_p`)&yyG4{l0XWr$^%##$(4C)xoz4o+kW43UO}dca`MQiFo|`fxLnm=~86+B8ew@ zvTji_EFTjgWr#Hb&Hse2f(Ru7iIFP3Q^Q^0cb|VG8GxAK+UeC32gD{2R-0tj`DrA_ zJMSNMSz+&3)?aSkarS*P;vJryVLML{vHzMy5&HRLF>OR68ebrQ;8K#8ZwE_=T}~Pb zsxm@mnNOTQ$_{`xfm#}qb95>>Mx9TzAy;B@6Cyatn3PINK^USn=613)_?7sY<>RkF zZ*eOOf#@#3wOtBqOUrH3^SkLHNgBg%H$enE(JM&|l9;nM5S3@w0LlP>oa~clCmq%c zRZnn+@EWoIH7(J9Oe=D--31{4GLzt=-*fYcjHBg+Nm4Lq)#ywvZIH8`biekUvQktV zTn12YKVMVnz3Rhd&j^*Q;mD+gs!cfqrSmOqN;ssd^4OmD)%oUx)7PyIZ)>$SCML%A z^5P=tAeKdNv&WDGgW@P`=x8?-?vnc$JQg#{(=1%eP@Fufb39#Jt6u3{n>>zcz`@nj z!AyAt9H0MsX7>D-O3`Xfaq8dOb2*qr(I{>Cg-D{M@y}8b%Ea2=8NUC;-dlK8xwT#6 zQqtYMX^`G@r+|QfMF>cj)Fz~B(+z@jqY{F&gmgEEDBX>8H+;+Yd!KWT&+q&LzcIcs z_Hf(|-TRJpueGjgUh|ssYj^%AvAUZ&Ce!J_LaR~}-kY=CFXVhBgQ-lxT%tg{NKPhS zPEtwEyCL^Q_8aT=&3kimyPz$D5P8G-`-#LP`{<@}3^{|x;*S9n?xu6gGdqs^I7h7M zzI%^>zYWzk2{G<6ad&x#;biwS9=^9$=QH8bT9~>Bh zXu==FC1EZ50Hvo_XWk8p`Wv*KIPdZS5b8p0E^=+2-brIolf> zqvgv{bxzn1g@gh|TBB9mdgSm%_MTGdFtQJ)INAMR{n63^vs+P`G*y?O?xVTbrbl9z z>#dn@0?PW@Jeeb5CHnDtB91O|=euG~yg;GCjOkE0F-!Et90f*SrP6czRfXWWiFwkP zTFd#Snj{VxQe?#iR1~)kBCk!R@$<+J6E+`pDsL%8iB9${h529*j6>PW@8Ma#C3%gi zMYtc*+l&`M?c}w$+RcYT&;OFH$<+D1AqFTKIIi@pH{HI>l%`YU5HXfN3Bm+kKGqu2 z%qR|JxgfJu>SReh6tXtw;J5u0&7e<5<+QGeDl^$RJ*|Ocaoh0-6{E^V$9}rn3!C!w zibDkH)yMAW&W2QGlZD{-CZZx$o^EbEKLB5f*+98t>BVIo~JYHea%=4|fL#8yhR);xYz1Lo0DiS5mPQ&0jS|s$%cQ z`H^6IKMlk+U7@I$(hFsF^EfcJaT6{U5uQR)t(d|Xb&&{`&h$K^GW-=Db;6;mX=1E2 zU1t+XLP|QGub$=ScdrM6yAj8v5>Cu+E|>ZEzarqtRO#HLok}EMKh(YI;eyi1oS-u>UGvuGsu06l|~e&%TvjBNX<&tkRZ0+noh4d+G(| z62>1t>WBP2{+<*62NwNkAXw*zq>-D8Cjlqq0{qpDr6pO$|GFiSH1MDQ`f2I4$7^Oi z*WE3|e_VmI!tYt9@ODl7zrOwZulrblH?m0X)-&^4-{QYNrbHlSfQIw0|9^l3fPq-} z{g%yWky|7AuWj`A9YNls@sOMT?@1@q?+tA>z!ni5bzL^TwN9q^idSANS9xB527F1k zeMrYNfY`-c?dq?d{6pa*Qy=lVxnj_wbUY+e@^eg>!?(@D9|bv1jumW!3Rh?wC+fpl zF;StglT#ONo$pPODQv`Rd$fcUpJJj**1kLcjD=g*hpeP+pB?4u!Pw@O3 zmWJ~KCPx5gtOoZm78MyOF^ZhrO@^4hx0lVM%0WrQY>!Xsv>HJTvB-okS$|5XGhIw>wfIt<&K9H<@MfwbwmH=`0&p_@SRBXYYt2&>_kNku&|MRv; zS2KeL^0^yyOJzhv$UT2IG%|APdt)1AT>iMU`LdHV2A9?a=J@HhpBrG4Bp;y#P!m#y zLeX}ymONRr;nVg~eK9d(y69iJxPj$PgeNj{r9?rMd7v}_z(}nS0r)9kF#hzlz@-c?d=rnnW zbd~DX*a75!faR;ha@Y1`CH#r42dMMjo)-aqNMMEZJWYdSAcj_|Wb6*~?*Urt4(ByqQo^Gr^II3tGtozl4R& z%+5;P9FGoExH7u(0x%)$QqMISZ8T2b!Q} zybk#Ih4Ze?8X6r@lw>U1eF;BePB~&0_V#>1$(*%hWMsr@2tjiF{|aa1+ZRr1tDtV~?lP}b@yW=} zLDvPHs$2#db1*1z>qiT@72e~#)Xe@~ZS=#XNYv+7e@g{;*FcTP5{s6GhNh5KypG|6 z8!sRg*ni7zeGA-nLEY7`#Scqxl+iWq-9K(l74RbD@}v;CATN-q799zE%|7hD`$Q}i zpYd$X2|nZLrXRziTcDAn+7YZ2Xx5)-gaA7(c=A5DGXbK3c!Gv%IbK3dj|lRpos+F` z-^)!FBS~p8rQcFXV1N6{(7XUgLY1!rH7xJTV;ORf;{7zg&j_V6X#&>F$orIuI6!mV z=8L9Zp)shgr)oZ58*X{dFws<(fS{s`>zQbtzgENc3QMvD=ni%v`5#~o=YnS9wSdZh zCVA2XN}s#GP=%u4`62oK2tehfqf?T**KIkiiWg0XOf@@es}epou@7tSp`$A|GCJ)k zf@*6YzgOI_wtdm7A1PWZv04yBZR6Ch?kboQ5YMJX{U-HFV6WA2{#zBE}JsAd=B;$s~3#b|-PqaVj8dr{Qon zMi8<2tHwIojxm-ies~!BnbpKd_97`E6)}-|nAs|M{{#&UP1?1V5v*s5e9^U?qVsOc z*vP>xuK|WopcpswR+fPy8AhO1$7Ug$}XUj9^1=I&pb<`53b3!vDu|I0}8zZrT zS#VfGeY(Kb;^|J_BFKv`Pa=+YKj-0k)!I&js81h{nwpf<>zFn0q`W-elL+`{z$|0^PeVeZ>^`f)F6eeWdG=N_&}ZOjoetDgIw4k4+2qPK467g^_c zf-K`tpY2zWZK#QS^ocCi@)f@CLmq~TTr%lazg*VfpJw9%e#ORWq2UNbLYDx zn)$lf!l#WqWQ8Pl#&L%=YIX}hJ=<%P7)}Rq2=Vg@B#w1O8uvg$&r#F>i;hryAIrv4 z7L|GX<{QC-Un+?l4kxB0knJoogl6JGLhYhbWrA?}hq3D{nAMD8#V?8y1sH`|KPJ7R z$b13!#sgOj8-unfQonf)ME$PC-Q}+UtzcHcWUN?cY!Sd@kMIo5n1eReZV~INvP?zO zg&lHB8CB*d_QA?|(ph%A3)(F`T0ViAz~05JLsC2B+hU@woe?ftQcD)URa{zxK>lJs zcXQ;Soxy#c5G-<8Mp=H>;1dNWDECX2MXAXvJl+WeDa7bFAc{8HVN2PHv?TbrW$HBM+YOiscuP~FQnbx1W=oqUZ3`|=#p?1 zE8|>j2N1&$Y%s3GJp0w94u%EpOjecGEe(5bhE2Fslfm`Mcs=oTKb{GB~;pu@+Hjnl?cA!W9wTE z3)6x6C5-17GgCm4H+((I;9}U9<{X%zpU%(kI`XdtPgZ6ewVjj!!8`pUsYH(6i7)*R zUu(oLEcf#be~@F5Lx^ctf3LXQC$e#|E;m_6>NHSQM5r)vNyi$UAt9M-;*u{BpMD;COF3?z|OW4@nnq(xp{5e!1sh2 zTf1(h9hmPnaLbWPp=9xzbOGi-tDO;x7=>E$V+20)0a#KLr4WuV1zEI`CyOBpl(?^Jxf+7*L_Xauz9dnBgAK32JTO=o=Nv+_aWtkX4z?xrnwixjUhtz3Kh_J1UwIu$ z)`lf&`7N6av9}?C)@)yH(W-4k8is+-<;c%?)M~}?_b@AhVNR#fafy}kEmS1xoXLmw8+cz2H1P<@xC#Hhj+WYPMi-Wsg z`+Ki~B*X=7qskl&AI*>_8u`~1v@CSw8+?K+w zJKTN~NV2>RAFYJRL#+KU4b}&G*h^{=-Cx-+p$$lk>E|-AO=!avn?_>kMnt(aKRw^!8!mQe zLtkw`T1S7oK&42s&f7d$Tn^+)2@=>9)9&BYxc@3V{QT11W9Z^PS^z06J`4&RzPIiT z#CmI*DVB2bCBr@1P$U0Y<2|{`C+`il>z=Vxt9iZIt+QKi+pW&QGj_v(x~7_Dx$9h% zwI^?GFUN$KG7r=_u~%Cw5;A`oen_gLYb-P{qEYE;PcPu&T5Hk!(B+-&_YWs4y)K!o zH{-&q?@EUxDYe>uX&HQ$a)YJpf8NW{C}PRvxqDdEUA3-(;&rkdBbACBV|xu0n}jUC z`5}OZ+i4U&alIHo=Q5^a@n&~sGUZDTpypKr4?t1_FW#cF;tN7rBTrFnHCOguk+Q`I zYsd4qr$R|)kZ)27^+(VdmuXwDk9|Nd+aKCYuI1UtqTXKXF;s8rm-Jg+h!)$GP96%A z!gby}Q)h|Gu((JAp>8{*thvMR*-9to~kyU3q+);6?pC zj6=++Q;c`5Oh!4_C`$#7eHtw$vd=MV5uY$NMfUNc^(Yb|Mcak>p@shMM}1+~E6^vQ zPZGgKC9%&$J2?d$=ZcuWDQ8-1Vj+%RBakbtR8CT>RbdDAD?7jlIDU z6c&cw_c5gpE?KV6yXMUufc|al8gXK7(Nk38XX4~Vm~^(fYp)EtOA_I2dJ?>T!)eZa zU5Dqw_x6a^7*&suy>`W2Broci@~ftgS$5?3J%f7~+NlcH9OlEQPhVENK)^+_#fMC^ zDt6~#>7H8bl3l}ZA3d-dEg!^z*Ozl|VoSa4g_R(y;j!0iV|n5o zcF_#Y2BkH6%IC$3_z!Y$jUz!z8xF(!MHf>G@Nf~%P*`p|o460E;v)YVCbVl5bkSfcp|QnwU5cw*eW*PKSA zWwhk085>n0>%t+&pMv}|{_FQPk_XftmXL+qDc)`l&tHX>%im$k>sj_XRjf|KXwNW% z1c%c|9}5q8xF}4VqCS6lGi7bV$vclX==5Z?l;5sBrN^)8es*psmA)Wq%8 zH65bar86CiB9()@A!<0`nhxHHVaG(3e^$>NEiwuBtL+ZQ7cEh37ZJ6FUcMMZC@vNg z#gn~l{YZg&Z`h3#A=eTPHP8PyminJn($9|y`@$PrS@ZKt^Lg)H=9akgxDc8pT1DCC zucxbJr>l#H<8|q>kjexWLX6s>SHm~?RAS+kxFZAJTV7va?(}^E)`^?k`;xWhN`~Z_ z=Z05b-$>2*h~+On6%nJR8=XC~%2>dZUg1deflq#N%W=YL`m1pM;| z!r1q8^W675&siQ{x-7Ir8TeRvzeSHTbGp3|c3knPnR{8}x+fmf`t+JKKVuE-Vt!O1 zE_H7XjB7adkP%V(JMjus4_EgLe5uLG$rNbgM9tx42pQSS6Xq$ab96X=5k?!te%SVUBoY4E*UO zGgu)iJ;wH|a-{83Lxmp*!UB!YopyS=DBTd6qGzxUHB z7Kox8g19kMxGDu2#6UdOcqbW#KGt|uvO|%*g~Ff^>r}2cn&(c=%mB1CKGSKsR{CBu z@QFgMDi_w$w3^P1PhYIuiD=UuMlp^ho_*JzDog2QYm458g!sFYPY^q{)H)=Hxj2`T zQO)u0{VKwgU;Ro7wD?up3?SZl&soS~8O`_;v|DLJoh=(ZPSS&8Bu9e7ZB^iAN|f8GXo13q{c>}Tk! z9*pOr+(GS4O&6>e22`f0%RHv49NnRIR(4MaR`4wWfaJ;eVW6j_6kjGx`bDw?ZEaRy z|Bm(eeTB(OV0&!t>aB3!!pQtpuq?2)_?9=gtD7!S2g~tI34!M8zbwM8v{-j?lHVRF zKN3lTXE6t7kZbov&x$YU*Mz<%DU3II$6Xm>o0n_lT9L*EwWWA8kQ}a>_ zZNeN5jiMl)=Tq{TP8D$PpISKEOLBVNMH-zyV3ftIFchTkNmEq5cr+X!5h&|Y!l*;0 zJYL$|J*!`zOMTzScqmMowSxw`G2;8luSvT1($m#;8Ls($5dgV+2KrwnV~5uV_=Y<+ zWA6}zzXqz6>8}r@ET09d!$+xZDEg8|%~#_{A+T-4zGPhbZyoxg&t`riV`yh~%;}nK zmmp%-by!yQU3Gbl*y~ zeRNcZlq@^e{OYjeR%$>{rO3=`7RuO@%3{lOC{@Yg{LN}%eviW9IC9@^ai73wzT(j) z?9g&`D8nkWCB}6IKbq_4x}Vfw=F1!F>?5yNqFcPfH{aTU6sPUrhJjr`D>hkCe>>mdtV2GH^}?$w(^QB_@yyfBEn}h)z)Y#B;d8{y?@$2Eoq(t-e^is!_xAU`+j&blvyv0Cn=%!&6O?#O7%_c%o|E~WTnV;6ouBpe;$i8lL4_Eu`3k0W zJuWgy@w#&BOLpee7yH zYy*!?rq?~&JccdzKp6CZ>j~CSu`yjzkX;8)4?w*=dzq398%+WU8TL=c4W!OyoH%;o z!=YYUD7_4}#h}rsLTq8b9d|u$h?3v3Y*_M2SfHSF(5Q=t`4|`?nbio(AYs?_GpXD> zcFH;qT>I@@K1c2$?dy`E=g>(Usg{+FcK`}OPDD(E?f_UB$X|G#>xUUEN0qJ2PLJpn zlrh0x+EC$j;pm(vw-ArXuuus>3=$?)3?`HVEnNys@k~xDD5QG#Oc%7Jk!MdgM%v%l zq3|Pf$0RdHngK$>RdI%R8?1rGxos8{gYX+P-YZ~dQcpn{;xJ+Kh47DyVJw+sBe(=* zJ4hHS7jZ&(s2Hh7M8Z35mvnmml}`%fk3iwUd25XGo3AokfL*PhEVKGiOGaif*lC_FXKx5&T>G-Z}&TQ@yjoL&5;VN)!__U?Mw%0 z+GeYL3GI7ee{|~}z*e-xvU>}&y`iW*@UKhtZ`%-kE`uyD{lA~5S<>e3zDQoswS;^8 zEYZK?Piu@WuyQ{6$w}+|BO`NRc7THbdE|CbXbYQ6!+2P#0^JlAx2pT)wS+q~|K(Q- zjN_T5+ba=azRC*ACsMXcW0d#q=C{5973 z^?^>k%i3vdF)lxj&uT*Wt^pZ@l+7krFBJXqF73bKFxmpk|o&) zaImiwTCAhB4)1g1R7}m}Um`;j-z6j=jlvv(iVGph4$vVn?23tZYwbF zo3^dd&`idRM+vauUi_WfOXia)Z*E(5y_=m<`4lgWFhwbmy}XOpD~Bc1ejZ9oBxOS^ zTEW^r)!^=QSF$q>6M&cNH5$NJG+u7N#g<2LT(iIXAL%@((Y%#2YDaWHSZqa9GM0Gd zI{RkiJ1r<{eupySIWM?~p1Vo9q-1q^Qbe{`HIae<=xX@Kge`ppl6Tod#@mU za?+Tioc)&yPFuq6#D9&TJmdx~u#HrgWBx_7&X>`chmsamwm(1ZSdZDv*S_`Hr#7Xr zO0j*oUTKkdBgCmK?SeEAK=89DsWY6lAoHk5M{lR*^Q*>-ZrbIJHk?pSl|+c@t;+VD zUp!OEfg^!Gqo?}5b6GE94Ngg}d}LfaNnO*hR)U$)kR;=W22g1(kCG0Y#BU)G-NWkQ zJ4B%gwUdiYxL1!^l$r3V;rO^Pesb-0r(qI;lu69Rq(xSW1!_z$ zD3Q|Y3g6ZnGBUy?h-x3ejP@gha(`0$hA=;>A?q@|_ffr1oYYGlR0yI*aQTNoS8*96 zz#S|A>ZrL7jH@{Jbqn*bJazY}7xWlP=(@F5L1U!x-CJHd;UESz%>J|7maO-pME}Vg z&ie_zjfEk49y3{wtr)=Hth*9nn0@6uowHa6s!9ax^5Y@4m_Gb6y%YlHr21YlV#s3q z_A#IQn;?z|X|>s&kKKGH9vslh`SEBP%VKvjV!aj4TY;i=Xs!s8=nxD0_}k!{b{j^C zX5s}?UCG~zT=9-ngla*ZRk&~^4{D65=Nhp17drw2o5V$4Cp_)*0urwzIRNJ_qiV!b z9n9$UZw~HZ$b1-ml<+q?g!X|PfQR3=;mn4j+U*)6f!kLkwxq-Iy3U%L)1+D zErkJZ!o&Zp1oDs+r;nnFFV~b`wb5&^qAeK1)}2W(f}B3C%-C4a zr*NCtSMRNjn~Z2UEiGL z#4#{sh;`x7hwPyg+N@(?3rf>mJMX*tpsg!y>Kb>7<*^vl)Ed4SbjpHLA0 z4aa}-n7>X14$98Z8T&?H9r%iPLuH`m-A&^OIwaqyIu9iC0s;a^1CHmis`&rq{=wtH zg3GU37Q84TnPyTGS$(&j=@|{}TXtTEPu3~W`j_X42Y&~NWmODF(<^-X@nyA*RZ@;d z&(ETNUe900u|CKIyOv5)rm1iLcGkc+QW`yLJl zlnR@6P-)`>8+B-2_^-d`!#_@sAqHQ1lq?O{GDXh|D*$^1^jVe}H2VN&fntpu*%YW5 zP;viw#aAom|k_a6^H`a4=66beNQ7|fJNO9fh2$iI%9dRalhw%v{^wJlm`(0mJO z>=DH5Yp?$R#r_&W)@|fC{1Lw-gse7`7zy(-&=)=iddh*|0*Q;ucY}X^A2_hU$Gq-iWj4tcZ#P;nEJ=p^PS|8;uM6a0Q+-h>9;Ckaur zIdUPP$^}pV_0$q*j58zm@<2uPFy4gu^^bzLG(Y(NZL;FQLy_YaEm>dkJG}!VyF2xT zJ(u9qU#sggt8uX7r`fI(FmPhPl7+Y&T1Y^^%7)%MMT;PQg-1^N}0<5n--{D{X2%!Fc zr66X%XJr`>qpOfQEKd=#`fdug|3qS_nT`_gzdm^{0;PBF7eg7M!E_uu-I)evI@4fO zflA2QvnK+7qh0^FA0= zRj$} zsL|Y;yPEZ|<}Ytny6GxyzIEAOXfaKTq1OHEc_wWp$?a-4S#jHV=3+OiEJZw3#YTAK z30F;zLFcAy9)t1QOjqxechl4ZRbb4ZdCn}JpL&068;HLj3T(|82#+9rx+dlVvP>p? z&yeCfy?I~@EkGf1_Dtp+N_G^_qTc0K-Ur-~&H!{py5Iks90pWGq6?QF_yWrp5LCf{ z8w-G?o0h`aXP&_jiL{G+PbV$dsz5mWD4zNDGgUv}mkOk`8Ec9l*1aIJ2p{}$63dW2 zf%vep8LhMLT^Q$uR;_M>Y>-B|v8Oyg5!+M#G&;AW|L4E>Iv&cI_PDUkAP+P=Bd0>69_pb^&ZyZ2q? zZM4bQsT(c%F_u4kmtL>9+8Mv-=f=Idw7wOn-LCM@z<+`zHSc?;2!zkRuG6fW8HW15 zIav7mYqFAM_V8oZtVY>04f{VfW{{sTisj~LNI&wAvq*QiPlpBg?-ty>fMuapix5EI zDqYY9P$X)a@s=iZL74?^?M4p)Pr`|OX-Sl7edOm%T5SU3dL{|r{i7Cm{Z52=Bhi+0 z#Q^eHKRZl6u5uy8A~Q#|4N*`>{@tVeKETx|??-Z#%bD1CtG2S7r84sL;}U7KqT^_T zy9$dM$ab@!UMFa#=U`x8`CP0z4T-R5J(4~YsY;FbE5rf^(hu;I+~kq^JwTR3dCHl` zK{&+wl-fP;qAZve7qaDLJ@#tLZRMGK9?&ECK)|6}y$y=7zT^6?vw-T6HvxFp&A>Vl z#x(D_C-ETcS&M^Tw-Jb+gmKzBUjdZ2sAGsuzO4aEQ_e?GbxVb-j&X?rXlx<4;K$9@ zyOx5W-EA07k%^NTEu4jK>n?q&y(55jOj2v=a;i;8YA1akgWv>zHFKFHlaZA$*=JiN zakb{NPNnV6RK|g~jPV&sZN15I7d55eeyP&CGExJ{XO7DXT5{x_;qvHJyyaLlcfr`9 zW^Pv=dbk%H9qM_2VnIFUReA6&=Yop&rPEvqp}P3n`arb7G$HQPGg#k`Or@k&Y~s#OvRdRRWqz;nA_moZnJf0N*SLBzfx|KcZs17Tb;jtJ3 zRiD<1hOLIDAvjU?wK<@)TVg)_>*u__i_Sqfv8(Ou)HNJcd7J#G;x+&mP%EERsir*r zVl7m^#Z^J~4F<>n))oWWH(7T!*T0HpDKTVo9w<_c?0;A3r-u1$QSG(VB)w9O4lp+B zw*#UZS5seRJ~^H*J`TpcbNQA%h)L@BNHVDU?lMVC(q!j&Q-#K-)^f5Us-(_*u&9)4 zSQ#;H(unnk;kWEdBvg`KHq_U7FtseVo^Ogk!OJ?+%kie4i#;Pj$mq<}>6aAl8Ls%g zv~`ec%-(Yuv*Vrs=GXLp#LK-bAmK)m*>pqTf!?qgRUaH!7e}hJMRTIfyaWpkcy(ly zq|z`cw%reD*J!QoyxrBb7fb}C?yirl@-wPQZDkG0h5Ln-2S)tVjrJE>Z>!O&wH88i zXvHx(Usf5iWvYDVhpc%L+IXTE2fik3pkA*HZJoas!?Okkd6B{Aq+E^1PYn!#+L4O~ z+7LvX`j%78rS{jSqH7BFO7*~DfZauPcT8T-w4uO89+Dn>(aibzHbVArCPn%zJ!)ANR6kdsI3Hl8JQ)8a;+fQpmPNYVJMN!N3! zQ|~C%ddBxw->ejRN zOreXgKruGc8AzW%q zhtKy{Vg3v}BDsF3-*3koeAOhQVqn*%NBmAl%KOsUu`{X9%1L{w`U*z!57}4Y*&+=;N~h zw=L3qa@SsFM2sW;O%!*rvWLXZt&`a1g?8$W5XXxeyLhAaHC*bxu6#he`kqLUikgo0 zWmx$ARCzbST1y@7%)@r34$4l<2nQm`^JaxH4*gmb7KM-t= zB#U3|FZvU6+mwgUMc+3b%20=IBax**OxQB{5o%@}RfHuYC1fwr^DRt*_^u};rJL?0 z)?S!o;3^0VT17|%-&kdwhX^cX8g2znKf;$(78giHCL8v0Bx076iH6n@wA8N35za9A z&ixLmsjiPSaET>^1=6BVie;EWK?C8f=L;Z#|JZ6)L_GX&1pQSPGSBPks!&B?c^Bf-)L!Ue36CN3{%d9HrxJc)rO1k)9Ssh-zUUI!yv;@K>Dl5cZGv)DZ` zPqibz`!fEjvc8scpudG_4+)=ZQ6kyY$0sg>PT@#s&aHhnH*&f%0IC<91&JMY4d%T6 z;rW6At1@2SFRfSQHi%!;q-Nd8p+7N$D6px+m;IJf$@!GN7>XXn%h8|pLzmfnT{k~k zzw*tl?Kk`V`HITuKeG%IT#}mFJ~uIc{~1 zP00paTpWMmAGtzVQ(yOy|nZp+0yj| z_aDPx^$0?TkG+e#MW`m+DuUReSO;jn)tXdqnb|L>^2$K{JCfT;w5*ycwu7;nlVG9< zeuRJM%e9QJlhqR1TW8ugQMH(Z;AX3e@SP-;9ig34?hu68`iUnFiLj0g%yS&2pan5f zAOvOMBQ8lSiz{ z+0&{H&Rw8|`zxNGYNRFLRL# z$@$>hQ=M=eN%G>$H+zkWN{F7UYq;dbHK~zgaNIUhCo?Rcrw{CI7f~_R4zqLWDLIg5 z`0HJQa`&#vs@)#xRfi6w=03Decm59gJ3iE&tY@)5?8ADE{r+}@KeIH(^g)t=ATzTrvS$4WKn~EfUsdiVR-r5A}P&vXAYXGZ@i^3|iorUp&s{I2V-3 z$!!LC(&m5eJn5N>t5jqn)nR!S8w=H@-JDt?y%qyq7B&3?gQV`4D{oaZyZW3Myexp} z`ivg-A5SEkhd?RxeJgz5YVWgEIqqTU=RW3usondcgu3qH&`V(%y5~25IAgqpEa)qb zQ$kneU@QO5Apw}5%6$Q_#RK>1)|)(~?_UWW8@=Ob)P8gA&o>ASabB`?PH6Jbu$-=ONcoK@28K z$Q$-QW*D0iq&zxU_%Ygm`4O~C(C2K{65vH#{0+|Q@?gr9S~!qArP<0Fq_D_z&x^p3 zP7Y8aV;A@HXW5`>OtX>a4z8~s$Lw{tkY!S?$`JhQ7KCvyob(wFkk^aclk~@o#)_qJ z(Ru~WwdDZaG_@>dtVj7Q{6@6fqVbydW-z^gnJ- z>c95;lezu7$cPFLWDIa-pIdCbCdpi=lye(O7x}TjF1)T&fjTE7AQCi3nzwE9a)=Fv zl!Dbuo%?VTRW!`SDo;{Db{lsP^ZEz6-UBSS9d(F65)XR?D!PxUSR2!JF zlSXsIT0BC{-)+4XYWuvoYFpgNV=qir`w)+z+huwLa}zEY_lY0&C!Kv_AbQk6mXlaj z=uS^qUW8b>^0+#dT`(d@><5OI@S8tN2W$_2$2n&q0+UzX@83ZMVTQLHF;0PW2^=>L zQvtrDp4``1vSZVnONKzU1;J?w@$iPmQMZ~9`5Vh=2m=e^r!|*sKXK+WP=0@%kX~un zN_r+Q_7DWMPngmg3LQ+qR*hD;V4&x4%(xC5i z`G7_?h_V*kvL`);?rIg((ZrXd(OGS4oBhSUk(f;MmzhGBYm&i`Bt7z{lZ_FpQ=4fg zxGsE|@kaAwMY97jj^spX{y+*2JoQ=+GzwwokL=Pwp<`yvxx~1lmadIjpMK;7=s#3j ziWP#sxg#z#xWcFNOxx*nTiM6Ox>bC07u%e1TVmk2M?y;mx_eH00Z;vDH91Sb_j3n~{+_6_Y3%XS)AZ8qjP2_`XQ?Gq6=3FySx1*Bb!9W_G>k2E%ZF=v4R=9RmhX4jR9Vj(@Vrv4 zPV1`McN~SA{<%-U2*Ex9PxtMna>B}^!2E%9AFCE1xswy>Ul9g2VJ|1%$d6BdzX8xI z5Q6pjCWHExCTB2rCHzTe_F(4Uq);Qd_wwSDzjy7?=pPYpK>lAYlj9ToP2| zl5~I%&rcbn)T!yU*4a-`NCzS!Q12$PX9tT=w5Y9upI03Q@aR4;o==Zthb^F`Ft~{Z zru=Su`cP%~>Nk)YiqAkViX$wC93T0f-7OS2kvAUpu{*r`#^=CZGJH*x3>1v!*yn@{ zzqVk%5>MygLI=9uGqxkSD4Eo1GmNIeR&nybEq=udQEA*DQlcxLZ2`LRiP{M(`ko+k zU}qFHzMceKSUF*;+~>xg*88}BrFVL~WK|e_oq1yaVqGeR+S=rc#}fK9Hauv4p`w4= zGR*$+#2V;%CKTwAr}TwTYcjGYbRAk`F5-m%GQFRx#o$&cci&LyTcHewsan&JG+v3QC|z7;0y1amPNVY8u~lxYhfNw8C3Ytx=852~96#4tSqppKjuY1PNl z&0NR~D)I&arNpN@0H)KEMx|`fKMrBi*bibeS#jN&`bjT2!D!}9LQ`?V6iI$m~(jtaJC=cJ!?aVnh)$(8E;UhFY$o$WQB4hr=~ zN>>D5Xc^2Njwl|~8Wt^Ew_Gbk)}mu(z#h{R(YqJoISot5#&17JXE-{$ej(lRFh0$$ zuZi|rijT@2Y**5 z->8Uh>6sl)2TxawL<-TVNk*53@=ev4%oxGkcPH{268Ba$Fw-_X!g7jv^&pyBu0CVAUSntI-7`WM-3KCz|<*{up1Bb=5HWF&d~rM zSOZ_}L!~E+f$;XpbYVzAtKrc+ISZi$d~FvtX!%&B9GU$WSF1! zm)FEl?S8Z)zG`t-okGE9B!sS@Ot$?gJI(}$zNB}3LRS>MEKhk!;avKDoM|Z*+38~Z zn~*R9LI+S748hdK7Qr9sqn3i=#`Ff>1+h` z6tdXsa|*q?1JZLB4xm@AgndjmZXTx?lWn2Um4|{^eMAj6uXAK7(&7%YuvMqk3)FOY zj*n69zStJ2UmuRp_)5Yf`u&q3`exNsc}=vHmy(A{t7;k594|YJNLH>n0AF>|`)shz z`D?c)Do?=8cs?ACu#=|)OQ1eES>q@_zxLb|)VnIMgTbhjWW z9fEYHASoRZ(kaazbFKfo_shH1{&YT_bDeKo3Uki!JkK2CzVBZldfPPGFnc@z38mp+ zQD*s9oD5#4M(g3`4Z%9UfPnZ$BN;w#3^ zm7_`u;n7Q_4wcHMAp9_JWr1gTGZ3!VS5CWp-(MyYZ9>eoss{t6<5ZFIPzJ2ux1?h++;_3vbqhJ(tsNSWa(pTYT9qoFi$%xJX-ZfKAGsdhx|Dne0= z3zcDBwPt3+4$XS>kp*a_2vOARdyc5WTXQHM9M9w#$aqu2L2N=G!ob9Jta@OW4{cq7 za&eIq9RK0~_nsR5<18&j^SYNaFVFK=RrLG#th} zLp8FrE<-Z_MQN3CxmTlmR>JV%vug^ET9OAz5ck5_U3c4{<=Tzd5b5RglJ;xgj`-C*r)_L>c1!0@2cpB{|g4D}$ACieg z_;ID(+55WMDV!+V&s*=V7|igiEQePjH1hEsvl>Jvoy1UL{0snSN(q0vSHI+5N||Q- z^p!1#0_7WVmrMp&KLC1dUmsYgRO&RXKA~SPE_?srQi;H$(m}*_&1v(yKGi*Z?|%Vp z`D_qj9`gJnbS|z3clv)eTu44tKxXloE9SrbtuU1Acu!;%{NR%Po9Gt)FS>L zAy#J z09GO+UIzIuF+f zMnUq&kCnjv3}jQO0|zaD7Z(ZF3Ih%zpv`_%{`{uw-KVFVp`Q;6$b2v0hz?Yn9e;%* zJ(FPg!b}xGMD)~jbg|MQ?~C?!FvcL@oao_JHhZ+g5}R5t>DF2Nz5>y&ZIK0fFQ{(`7j!syFr{^?<(J68-a^K1 ze9{qgR~9>2T${zm@kVem>se-i-PNC^TAQ>l-V-TTVnjqIUmAN%7oe)6g{EtsxBtg| z`Yh70%jZ z_-ha;KAKHcuS#OF*j2N}Sf*9iMOitIQVU(Zv|MTT)Uc{(jV_2LScTnRR(nuR1 z8zZX68nM}q_4>Q1{|!25V0JPKhv|^VDW=mFL(xCZXb}#jkEfI zbasZVz0i4z`#qqsc5mIVxG(_MyS<%hvv(Ic$_{m`c<*Ss^?nywRQ+>rR-QotY+fiw zt1#2^H|Y8u^8x+X6-qtWosEM9D(PKl1MNV&?0*5}Xxv?z(L)(wfV8mSILHJtR}hi_ zy9*dNsVtL+0Y{g0kOST0XYdmAkYfbe_M$R2+bk-^Xjli|-1gG!h*O5La+E(|KKp&M zXD3_QkQOHU&rd`Si$WfwA=O!`SsnbCj8ysQbSJ>X*?^u(fY1=|zO#JJX90feU8!Cz zRE}uZ0XVhP^2s9W0dE0`D&o(QUn_35NQn1MYOAToEfSgil(_vq5mBqpaerbET8Zmk z>Q(Ps_=M@&|Jb2LR-VU*NOek)`K&ji!xN(uu^WT;x&$-3Meh@Z0BpchGmD?WU8#FT zUJV2*(5}Rm%jV}PNW`Tnhgs?ZQ8H6s^z1~j0C&zsHy9AfGhC)L0bgouYb@_18fumX ze32^h7z|JA0I$Er=bF^M4TjBlweeS!?@(V~sFaWt5q~?>p`Ajwdkr~(-4*vpL#i%BT zHhN1g$h-p$Rov%G;LZZoFb5C-Vo3og1|v!M9NYY1;Dvy`08a{z`x1?%Ss%lJ4dlcn z`&L|P)miu3qIKN}Icb6T911PIi}jh9c`Uwxbz0>wc1!%5>4Ahq)Q)&jQu@7it+Oe3 zBuuk+A9%pzSED4gF!Sr}v>uP|*;`Y|{)zkdVr7}`FO05TIhk~G#D4EEJe>9ai$_ZW zk>DdnHq!ugedn9vH`^exkc%bo>Dh8=bW=@-DA=gX;e6ktYSvMSVgW+lh8lT`KJb?% z^FD+*1Ho(CKMJEy)HrSDFFMdz07hz5B~;!V6~YjBGTWnpwP+bkS*F~F!KCWiFWLQ{&+qr66w~49ChC$$e>>9`rTA0Wz3kUbFo&0|f4Wxx(QNqk0sc!G?2Aepc){Szg zj^rcPn3TNgTEK^?Buq-@KLSt2N=4}NPNs9)78a>wZl+ek9>*kZR&+FuzI^N{-M?*99lb{?9Gwkbnj{ED|8b6g+GL>wDG^ z7Gb_zGuYNiaceB}M4RvF$|dfud`T*o>z+fOmR&mrEC>D?iVDkBA)9-Nu( zPsqoTLIs8-pF5MN3`z;|LWwjKqBj};SpZW;h~(BO+Vg%=SOM$+$}%Gu#olLPMie#Afyl5LV0rL=x#5`5 z?>+sS&)-}9?!ERKLoDX~?r$#;z}Bf+_O5ox_j(Tur-?y=V&Q8bWgbQl>Jru(f0+r6 z1ZU|uz*<|i-CF#14Kx*NlY(f0{d|MU72gSPl#<~{4?E8dE^V7S8#A97Ig?EK8p~+cOz=t1& zq6PRi-}%FIGpI{m0!Oa5mP@{0KIe=6XRClC3f+>O_hzd%!Kq!Km?3*vR;pBkA~Cq! z$CiSf1}L>bB8Y=95Q4yHmP8|ZDZK{!1gmt%fvUo+#dw$b9$+2^W?^b2YHROt#k$>u zWmk#uqo4M+w(ZipY$>PchkQFm2_k+>RJFfX@4&olr6K6sZ1K9p&5n=wd;o4N`=9qb zh8)%leuiL|&^m%p(ESjk69T~#&Q>EJ-$LhOE9e`oi@tDMY49{(IsGr$w(#BSRZstX01dNi-S7*FzT3fOxO+U&^fxMRGh+E>-~s-qxTD zO=XdjHNjA5uB&L9n=%mF4IminIv#d`Fn}>^npfZ-X5PLQxx&A)tpm@8&v5GAQy%s> z;F)GdYC*>vbXR2gA6=RHsgPK~wqvu0K^4TzdKW-%EuU<0bUr$!nmnBA-MC1hti~JLT7anGXzkgY%wGPQ(fQ{kr-&fQ!BU2f-cio({hK z(e;FmX`l(4bNjl0Xl8ZS2KSf**a9@#WGY<(#=H(-BtsOBxj%}~_+<=6Mn-~SkSye5 z8hgYLAaZ{Or4W$L>zJfIu`gEuCIJ8-2JwTPOq0zXC;JOcMyCa{oT{$;|6=F+dji6h zLQ5dKN#z517XlO5SYCxQj+u!xsZP=)fav%RW zA#DYamidJ?D|8(^7dG(^^Z~C8Trqo4RBFtN3$bUx8-v{B;ooTTH`#%hUjd?%21ekY zS$30vbfEcHqSlQjaKRt|X0Ce+O@E%KYk$0o9-0>HWsk~-R_g$<>db5f@atd_OO{cb z1X?MO8Up&JNjxa$KP}BAn0x;IyLylu1zu$#Tn;wCT)cv26|ibs65kYbT3KOr9xm^wDUfvRNfJ`Vo^twhIyvh?N?@jV1I{6pH-tKTJ{7wyH8`&kB* zu+t=3ZJTFuKO6or^jW3M4h{aCJ>S7-ftHK<>$=CG_t`XH&INx4rWY*&gX&0LJTdL+ z3m^Y5sJ2E<-hw)Iy7(;`1#s>V2G-s++mV`zuu&k}r{E6Hal#%5ZEbY{_x;;J%eBS| z&7mgdP8q`k`~F4{s6=amywkXCj;F%jJ(wqnh@k)u{t5t7 z$>}rAPXaQYJcA}^o@)Cq@OREVS=|*>AE@vEDQNtJxknn)#{Nc8-mKiKZ>U8L&q>CN z!9#Y0F4`jFCDIo5tZ>;pi9ww?hdI-IVx&wSG{#}JZ9_t!CcgtAwpA`2p6=@(rEVhA z8uzC9;pXr6XVQX?4&ZNKDR2_jgvnD+P%qgVneNlia-Qfu5yY&E_V48cUZ-%Fy#AiB zDnbAwF<;2V;XkDAI)S}Dkgvhan-r%$A4I{2zv#*fyS9}?(?L-WQZX>=8cZNLHtFtB=%mhxKVG3#1 z9_l(SlK@2aScwwqSb0n$7#*A@rN$FoHPL8o~s`;)1l zA36kxxp(8!0i?hPyIk<~j(zvF0(^r{EB~yk^7%9Co89zTiot)+~`u*jDWZ z`xWSe4w&n9K+9C_J0Ui=1!6S(z0o=6k7_ZzPY}&Ja zm}KvP=k0K!`!vlu(A0QVTkEJ7X}2La11|mZz*uMoFN-CJ7gR9Nwcrju?9g@^<^r{Y zYqkDr^H6fC;4EO{$9j4_zqE{J(n{13Oul^MIPl1h9D;+jhC|)_R6<30L#szUEVEyn z4@98@BkJ>?R!6W{Zu&{UVHq#{Iox@F9L-7Q$<^)B{jHw6JVqF)6mZfkS^WH(!fO+% z!jW9*Ts!H+kKqGPE|6rI+Jb#ZYRgU!D;<-uHBa!%AD|eLY8E$Ld-2M>bPMn|3xoZl z8wLdqyKt2bN6h+7@5nu2JvJL0&hV1E0rPj*H4PxdIjQl&;~G;4stl`w?I z4*K1&3%RgOF4G~xuB~vrFsO-I8W#pACFBAK|2(O9zMG@E!Q(%UC^18cl#hwn<2P6lmeh6sNE;&+Lm)XQBEc@R1wIdK*L{^l}#K>b+o#&sOGZB)kJU#p(v z(wzV~gd+p_5V1QTw4bBP`{Ty+&ILFGeR%UpHleKyUZ-l{cjQ|?H|`AlurJp(S1crzIYzt)G_s^utEjYFDMS$bg$)}Duu1UP-5Gg>mV z8Hd-xqe#5O!FxoKMW{q1jLGheJv7#)9?1C~Qh0AU)%#&&haVh^D7e)A$Vu-{5O^Ve zH)~|&@-W55Z`&M1C}GJsb8yaB>wMS-(hCgSDp!t>v-zTg<19e$D>4)5L7&26nJi}x zc9rwDwRV+OuvprT7kWUMv9bu$G#K##ZCvIZG}63HukFtWVerzcQ9u{Oi-v2bxYN-s z26*S7^IBxWVJ3C~Qx?=XP+DIyNGm;Zz7`k2sx?{j87)~zH@fX1&H_F2b>_glbIuPR((SB{PA#Hn2!u;)>u0# zvGuF1CO>FKH_LFrmqdesB^lkgY7buN{B-MlhNn&L6DiC19t2V^7rTQ=47L#)uGS9d zSWeQqG^sgLU%f6B;B=F%3Jxn8UAlD+DANKJq5Vsp`y*QMkYtZ!RCn=1Uj5Ha8xUAm zgr49OvYAhWVvKuLb@xCswIYv#Oml@pea>59U*UHVc9$Hap^W~4F7y-p%R8zPx4a z!r+;2J4b3|7dF|LEYsqNSarb75!bexVuJO(-~Z-#HqkKfTs-)23k)kntqdb>un_5} zIWg(P(9>B@Ly_8Z*1V11KR)DYrxJB)A%_~|+L5pPk$hUH1Z0HXRI?uqEC^zH(M$vR zhdVP2!5HAtz~Pw#^Yg-Ju1&A^R*^ACijCOVt(4mO7) zhj&IKAcv(6xH?x#j5tv_#y=jnhT6GvAGVVD4apW+PP5RQa z?LBWc5QelT)vyO1zseg5tT2zTCwg$SgtqC@fI(lN@XNd17C=AEcoOnynBK4fW0@zc zP=1+h12^_&{lGU-hVEojs1nCxi$px4!TD*@0?Y=+%6B0%^KLadMH7GjXNEc;*JK;` z0EbR7Nk+{+nNfV7-L&jF)ela3?cLADwp1cWHxm9-F^hEbbv9b^6s2lIa`-=IFIV3T z%)?|Qa0T)i2J%rP-m)_)to1sm;eRu@*36pu-0-|YovohL$A^f=+!n1=Fpo5A2E)T3 zJ%~h${jNz+6qlMi#F2DS#tM_n3#IatCS0#n+w<(*)fMAm7&t^$F16*`Z|HDHnCmjw zZf*7G3`tTVoT#Q8O9Gh)Q7Exo6>k)Mu_7AVS~>jgyoHUZ*y~jDfV(s=(rdWeFlAPs z(A!F}t}N_Ak2#xq&1A%DUiTpDECk)+%$HuNkCCT?ef+j@li&|dHgUf4pF@9;dW%?WY z1V;f^?;peR83y>+cyye#xGhg0Y%GrNXVLfL-1OK08$wNMb?We@=`TYaiNP3xdySC| z=b#yzlLGXTEw-;FFt~6yFuNE_{JDr)C18HdS3cgmC0Jiih%G+&hh5`wauf5&l3A_k zFtHX=Pb!8+@H$JIIQAGxZ6CI0kIYKx>3lJ9j9K9$w6O*Y#~b+FuOh-84P71eYVY5y zZ)S8U^(G-0CW1xucZiIRjpNptj|DV^raXc?I@fU;Tt>{wV$NRvz_WNr6PjR^WL?A> znB|T-C@fb0{^>;rU!sRP)AIVUBG{k`(x$i3qzWJ(3jqj$K;11PO0mBcY#q%n}}|y z9B_3TqQ9H}ljMqK-!P*?B}SB8nH+iBncwH%b{8;D>|W_IGN-fpmF*lEw{7{ouxmz& z#_5JIW~t|m(EJP9wXFLankd5R4=3E_nOZqGgKHkPnWMp7o14{0R{LJ|pWC-S?A0Oj zRv6BF>O#}^I*D2_*WX*n#1;xt*Yya3xWlm>DdTY6gLQ8nDaUjt8C1^7DSd)w7w$a^>;wpx>kMg}=u*wQFRLf_uR= z^Q9}zLYL6eu+!Pb2#>_x_wfbCr!c4BSk?fdpiWUl0vSO*0&8Br9Hfbj+2QRBK8*7T(FdXAeyDUk-?L= z=ooao{VtfqkC(}h&Piv2zpvSiOc#@#5w?r>>g~JQ<~&`% zUnz&Dk$f&#O0K;a^ehPN`b**xlSvC2+dRyQ&gHPjW1o8jTbBCDp!5Rice2>6GB+l$ z-txTO1?6#st*G-}tM4iB=iyOo+M2LmOD6oi?|)G-jY8Qu)+S4lwr(V!(D3Ufgt>RC zRe7nSrY%PVQ9wmFRvyy`C2+}9;>F7VVq_2L4){vTc|L-5G(vvo&s4}YIY)srv$!w{ z8gAWVZ8)5pu(^Wmm&}LPJx}e{XIf`iB(v+Ab#nQd6u*P)R`kzk=MyM+y0w-GMSE#M zMdx)VwEf9#l8B3J<1ddVcsmhl`T8eEiA6sAn76U4m_5Q}Sg2D}KBw6MrpvC9@Rsvb z2k4co`|KmI7IcQCC)F#fK!jUaewdWuv5~^85-4-@WDO&1Oi=3CjghPq9wtG>FvZFl zgt$XeIW`o%Cg@y`l9Be&-+gVtbXJL+sGq#kCiW45+R3ic&CN{?*%rqzv*wx!0gvV{ z){)B!uMX-yB6q&kVWx1ARY|K6+xXdPGu680%$6jW5_U;np;W93djRC=dF|Ea??Wv# z(_Pj_V28?r8xuC)BTS_nIjmoVfw6IY;EQOcVNt}X@2~vqx7pbdubg)#Ay*Q5s{tz- zYLY8DtMRDKV*abK;*zi|f!D!R^)IrY6ycXX;lLPOZnKT>7c8inY!G_ptw@Q2-0MU{ zq3geqUr9+E8bB54tcz&*{R7W?v0YNE(DRk{#s!j<>pE@3Ui0Ci=w7ZsLQytutk1LG zaPq|e@vZ(Q4J?j=NeV+ej0V$6)W4kHFBc4D{w7JIk#1vJM6@o0#~|>lmw?d*;|aEN zuim8u7K=?hW$_n?t&0QSSvg{d*CEz;B)eupp@(V^;_6=8#$Wg)(S*h$BHmc`=VsY) zg4(P22AhKOhJJk%BBu(ZU}AYwpELkmN9Q87Nby71`Po7>`Sjy|rl2V@rXb|L8G@0( zi~AKkQo$-ZwupqV&$pDy%rMLirN(M{t4YrUER7vYvg?O(8SvIo+<+N^9Ebj6(Y929|lX$bSu2CV$DY!@(X0F4|J_5qaLKF(>Fl#UH&2`^ElZnV_Wd1#3915vg_0YPHHXVUUxfC{mn=1vzns-7YM{E^>05m`uXX>>L6Xo4VHi9bBX=XuleAtKP^<<2MA&IWR zMnKSf@L1#om}WF7YD4t#qx!9X))Y-*RqN~7R5bj+2gU&1Ya8;oH)_6{<+!#V*8=S` zZ<2S?#rvmce+@no5a61}!baK# zms93K4v+^bzI_66W!o2(THEChx=qSB5m^aNeQsW~b%3|58w*T9D&?>K9AwGkLn<}k zco>opMoY@S{su!zv*AP~p`;8=&F=R-K5Zc4tyC9xe_;`-Aa|agZsOK8KY<_ z6xg+Zpik=Koc7wEYkz~B9Z^l2DQY^2l6wH#H2_xAIfQ&fX&g`9r z+h0x;L52Xl3?M+L3q_jLMl-rGe_Perne$Vno@laY;ecLQM|d10jf+5*lp*#__N&g1 z_>}XRX1y>yWnoaeh4gpfBgy;wB?HDNM2FXv?AWNy=z_9wh1O^3mMiC)%T`UmuS0uN zqUA8)qp<^Ev^AelNJcK}yjTc62@Q(I$qP1zl@pd(l;Yg{MOg&@`ib@9c0cUA=UDS~ z$SQI2DeT?4(&xz+%p6zs_K!_HfeY`U9Qv?B?;+-#5mhnODjJ#5R%rd9=Q7-CBGp7F zZJz9d!TU@Td@SDB@@jbAaN^_yG|s9iIVpK+tjloP2(mtm95%ybsXe}_I=7rHIBA9z@Ptk&r*6BN4U zzd`7pj%silBlZOhK+MIo{maCv7Z!-ZA(b6uoFhSCj>1FUL#6d=!4%`38O}fnz^nJ4 zVw2i0R85}{&;&)~AExatRNZHPi2m~UhjCU!gmZ-!<3e>$qoKJrmn*=GenINFAO#1Pnym5or zO$PGX-r|xg`Mb9OI`T&e@0+Hv0SKQY9);$bKMdTi%PL!ta8YPkX{-$kFWrzHt3d}m zO}~2JGG5dAw{KwR%68{~a2$kjPX&BQKYLS0I>TkS{WI=OE%_olqRf$zP2Ux=%{DIwcu2YKP-3744liHYl8{ad`gtFZNm9+(+EbC_(X9*xR|qq@n=b#y=TbZ^bcR!Pal)6A9St<9h*trNx+cN2nAx78)%((SS_AA1;(2_ zWbwzgO?#5^sD2>E!X&dbriHcLBykkhjqt|&MvMNT7mERT4DRAgO_wj5TN=kpBi9PE zq2%^EN?AN&x#gv!6w+_3f0R@Z#U9(AZ+PTrA25jmVBw+%d?c>1HVJVB8VhORsd|Eh|9dO5576{aY5o)$LJs6@j zSlQyb>5D-NZatNxM8;Mbo0E)R59(bDOp(&I=(6=;3lIW(s5r(QJ0uzCW*!2Fv{Gp-|u|ww)Y@Zd~bPp81gsE6K|0Ym&Y)q-t!+Ev9y+ zpN4eHaU#_A5gEEd5Lq2Db|n*`w0LApb_O^iGL*bNC-YtZu}wQE_0@(hn$b*!M{5XK zp&eu07azlf_O=$TVl~}YWd8VXVu#7~amv^t&l>yOJ-7*YypRqPb7>aLx(KK*rm99xTuUHFKrS0tkefy7-y!aM129C+~%%uHW+RC8Fv~NWFmg+6kyT95WW;( zcoW$n*D3pI97#K!HF{tGYa@^NZ+j&qfJ!+X17pnXVvp$X%XxqgAzifMJKi}q@Gj0$ z(OW*URejjYT+5tDn9BAXz3M^pA^R#pVsejg15M^>Yd9vl{*Lp>GD23Z&B;H)$RJ&A zy2lqrO(QhvJ0R%;`x?7g&I2d|ie-F9n9U`pgDJS%>MGv(igqjV)?V*38_3v2Co>*M&>I#T~aJ32kgGNpbno* zQu6%yaE>iV2brV^<&N<|VnfU%j#I)DbD!ZV6ogIK_m9|SBI9iTBv()Fan zh#MNtaJ~G+;CmH@vdijjzbF3q18p18wD-sybNjohD}J{bS>OM&N-E2vfQ{%qPz_=5a|F+7gt#{B`+8)7Ia zl7s6&M%#1>KuBK@x&dI2C+f5(Jm5HMCnqGf=j=N**FaTynyXg!Zi-bWj93e zt~SWH_O0!Hq-Y>Dsn>27v(D7*rCD>1lWf8OomOI#2`dNHJ*M@T;36SW3Cr6Yv2FUd zunS4$8f?nICnN{}&H>S*o&Y{lLqK#?}v;<=urMC;#AqK#dV`Y|2C69ov1Z(AEJ& zO@x2cwF~5WK&dD&N@X;L&^TbCTDaN(NfOl`72ZhaE`9f;T4+@Eg>sB$y}LtZtOY@v zV$EnW+ti*OF15GUl%@sQiy%z(kxT_zc#LU$_&7Z3QtAf}sB#T*-*>D(9trek^-WS= zUANbw-k940N3tT|3s+hXvTlP+RZ^u}kmJGaIs^S~xu)t@%Spyw1^5je_jFa%MDa%v z%n#Tq*%#O`InVbwn40?aGUm#$)SXX004KIGz1GYUK(vsODQNIKvrHwIF_s7#ZkbIJ zVDP^$YuLs$S$+bLV6FP76!qdabm89ICR4X(1f4LPG(H9dl&bR=WMMd=+cVaXf&1K0 zs|v+*Gv9cjxD5Fqo4T-ZCiWfma_v@cK71H<(Rk)9nkKd&V_W8GdevIX_+;#L50Os; ziNhyjO@ajYaX8pf4>l+8WqW}WlVgCUOgl=cS3@6>mVEpb;ATt!H+2mw!2@%42h=@k_cbj(W3L%(n`|O>$~h=t6I8EHHJj`WCSRV<6lm3N!yz)> zZBbZtm|nQnFX?WEQfNQ!$Ct)B9HfYjFVpcV!FYOib1aw2u_9BLq1GY_w_~}N`-U1jA}$_}XJxWXiA}M;;L~LioDW`SQ3NcjUm_^QilX^PDQnha2>?NlF>N z*Byc$>E}6(o;fr&5z%7}R@kriA=ounI*(KSAxUu>4#c$l+2r;s!!?}E)^@*HT{p#L z$;aNw&10+fF<<`J&;%G73k1BoBt73oP|AaI6Kx(<&(Kh0uHKW8f=-KQ%U~?`)d2{i)ep^!;-@dj zfw>ktSp4R*0_>2AK!-SCKLh0!-D+vl=0z~yZaf{rNT{2$9ZxY+IY-g~!nLguS91jV znuS;Df4DZ!faZ*dCyJD}nboZm(TOb^M<;&Z69_GF+fr^-l3bTS_eTBw%qdcj?3=tw z@Gs#TtQb8$BgFg0fytANAPfv+W=g-DxEw4u_QuAO`-A)48BChdYy}WN#xk>lsfGSAEg~o6{v8GhVONVm=QK^$?wnNV zO6ui_pV!?bF%pUL&T?QGB1hM;0-gFzCL8O~DF{j-R9E|l!!fxDEIX$)Bq}BZh)R)r z;pO4n*_r|AYrb5sRcN5KTcTo4uTb&X=EyQnPf!lf9)p6AwG+PX(Y~STa(W-&9<<4T zMt)k8h~IU0rqRi)8K;lu5>OeJ^Zh?{%JeWmL35y|3|V*vXlGy?K%~Sj`?91{6fPQZ zmAryipEq>c5}@NvdjG~A*T7=TJ5yjf{z<%*c;qnQGx zM^k=N|zZ}3;|Nf#cEK0Hpg-SI z`FK?DMBXYuxSOed9_Py?7(Wb8=Ya#`vCu_!<7+`+R9701g7j{M7V<4vDO<=sOcZHO zAsc6|omolP#*E53ay0tpxiajBg|0LmpEI(97A@cEbJE6WKltGRerzKZxh>N6VZm-_2%$J=%fhMTi)0y&|3lECgdK#6ky)UQ~uI3{&M=G49e zgV{jSl{THaB~Byvo`~j{>by=4{`2L5r0r^!I0l;^CQeWxog}^=qp>s zbNS-Q(EPL^NE1@#>qjsV^x7I%c8s9h;n}-7X)}{$ocd58KpsQM|K#r{+U(nKHemV% zBII+`ppP-n2KAEY5QoLEVY01+H0|TSY$C1D)5Vu-I56DFZ3G7VE=8YW7EZyN@0$b8 zxUsT0hkC7YNsXf~LwcSfx7#1f;JGt#ThGw88i+GzGGfcq)5%kRl@seW9fpnT?Zf%p ze@cajd50Di8yEOWKQ9LYN5?a_%{{Qb@h%oEyd|x(M@!4*ZJv!5!~1)Fr4WoU60j}fTw9T#2!65e|I%-jWy=u6jL3YLuM#dCS^}bTcPuDW zO}D3oYU@zz_-g=gF&1N`3{Bj@=sL}-sa7l0bOUzjeJgjyMq7IQoUMRg3}_|s?J&_m z?A8#{Yj%Z;jP>eBVZo=zm%EhnBq5>xRXXLqw-p+$mh7BFCnN?a)rgOKVpU4Sk;X+# z&0ZZzw)2Wj^n{c7=JV01idhU+X&E0K`l zR_?G2;W}TKK6dNcljZrxxymk{{l@ewljt{JaEw6@$ZB?dsWx4GvOFx_h#bqwVqiQ# z_WDWG*n1RQ5b(SW^eM4h2Isr8_k@xcA|3iIoDi@fccbR94>sicda>BH(00;A=G&zyRi`)(O%E~ z#&Qg{B;@vw+oqQ3>F1VFp)&3)sixQZs;u4Vg;{syW}{)f&+7L_g--;iYX9MtFP$k; z$I}y+WsqYldIb$$U*D< zU7B6X3Sif?^}?XR)*D`N#hm$fQkR36#w!qBE!;n5KWmcoD+T^JtMeU8SQ_%<0mIwO zN{JYzSPw*RGH)V=A8&_LX*y!Vtat(6!g;%k{p-`{M z4$00}+tM;UD-h8M+ul@=>-#TD47JttCHKiBlF;`TVQFwSTtjse zJPwELNJx|%&S7-zr6_3ljL7bstoMK}$IZ%Y- zD?AsQlhqL8=yX2GE&vW!^x%hHgGINco0U$wdk_{x8O^9w7!_u+$D-%)M-7)R;Z~%u zwCv48Rj-IVs8YJRsw&4%Em2WU;S1%!2R&~|fT#g7TwX-s%#6;><9>?A0=pG}MFDDM zqoYrCRU~l=shrQNJ8jZ1H!0ZQ`>}D--+s)lH?~_bHs}?%|4kF3xf;LXc>(783n>}p zRbVh1Z39R1(LQ_9_b&C$+bY_R4P8HoLP{|p92ClmhbEzPWaYc_jU1gHNMEML zk(W0g4*;Jo>ZAFNmt7?ex9^R3KJ$T#G>pe?J;NIL%v8f*W7J`OEs^X7Z-jrRlxYX` zB0N5XJCILFu)Q+LZ|)hlbuu91BJfsMrKz}7sJE_1uvm)=?Zx%a?h5(Gjyj?Ii5632 zhU?w5gUrv*AiaD)N27N#{T_wEbm%i4Keq{(O1v|JSC08H<|0qqd%)7N#e55^lVJDV z|AhBu00You_3DN!4VfFQhf`huM1u_b+477fZYGl@S9~W}3!+yUe`4XMa+*C5VF6lnQJru!%c-Kkp7Ag-x3%;;sq+(=E$Q7(z#f;2LS$!|x9`MA zcOt6ZQ4+q54`liS8b=_8%fuxC!pAz}{#eiu9t|sa)qSRgc`DI>`z8q-yxwP%>gW)^ z_L%45gglJ+pAe%Xf%VnTau&5Fq2JDFZSRk)%4Oe5HZjsbcM%p_n&-8*#% zjG2MDj(tD_fk8FrHAPh0e1$0n+l{K{A6wV%CyiE9@?@;~nzUOKUMH0A3|em)8QLGK z>o!^+uNS9@nIgljOh^`@7qXI#WKC43*cc>8p~?or~+!H=Fijjib{&A)DdG1}AO%M<*x0-%DBD<2HeD zipBHAZ;NCT+*>#hF*zAfBIv7U3zqB4s`xl9?w;CV(Wq?pQ8c2tI=+Qh*#69#>F@}klWJdo7r!TEo$X)WP~7=} zbT)T(?j~$C?dv!8s{Ax;dGz@H{#V#-oMWS7`&(zIvO+io29t3NQYBJBh z{i%cTZgh-<`Z-Lk&R8X(9N&GX(W5LB$_uSxF@*hNkxLfFshgixI(?CT^(xh-w%I`k zt)qUfNhvkW(7?nXc0Cx4ufYh1AMA}lToY|{CCOi20Dm7BAxLDuf%%}t8N4@9ge!3V zjNkdN&}tS*8-pGA4XKh7!Gvf-!K}UUPiThjBD@Bul>XKrA5qW9B-P#@!2lZq!T=|8 z5t}62keYzzmH9j^m|06A+hbL3saGJ=_}lQu6J+~Gg`YwWgZPr;;u9(J9|fSb93rr5 zE#}iSJ%ThC{aUx?u_q7z#-dy<8mN=$iQ4ORy2w6R(o_m#UAP%f7nrYSfw91cOuhUk zQLIsZve;(PuXW60GhxA7qkk#SohBilH6@(s-8vu93p)8=+!rOaw$Bn-)W4ASgxUc6 zd2fba#L5R;j>dbPkNyedTr=yr`>zDcqf)zkL!b4JiMQ8CKNA64bN+1=m$BTh<|kI&)7)0u@8U8F*B=ba z$erLFJD%FE^+uaDgBqR9DG$FjQzl6S56*9_#({@?_r(!$B^nl6d&uso237_2#g+GJ z9fEI?5w-FsAP=6@h=kMso^a1B;E5UGf_~4)bH;EJctex$VCdo)_Od+ zW#$ch_kBU80`Hy1k9?nxSw<^$-m3v5_avX&5FDq=w3>`%PA%fg)R+pj8@G%1v$7S|Uzl(D{=Kt(pm@$5s9}}7u&!!flmF4>G#Hu{ZrI2@erHIqjsV)>CUy^FU zt2Y(3+SXg}>j?PRuTPRh6cs`9_wK0kF}4vruX6S8@}{1>GW(_Fp{NfQbc0FJ?8EbuB_*{M zxUZ^``|W4;4p%yv)a5e6-i6QptR?x|hljzo2cNz@NBJKknSY6(CCFRh@%TUL1z0|` z*!`c+x(o)v%gUes0yG}3Ko3+4>q*%^A3%YBnUO$C30^$<2gm7O|F@px51av6L~98k zv6BAPuYoZ176|<9zMV}0c*jmgt7Yn;9qEghWbizNU0OT10j@EapE3b47!W$;kx@3j zBB*5MMLz)DhNwq%8W3tr`bAJB-|;JU-Rw6VfUm>BS5qAz8k<;2!VE=aLjff`4U`t$ z==cjNhXKDJTo6hB;QJ>6FG~&c%#P@+(FnO=Jys5=6RqM)04`BvI-L4ii#q=4OV3k4 z!CsDl%D;gciU|b8=%wTwpKkwXayua1g1)mY_ZvR)28HnEw7^=b(N85Pcy~7^O2)ZHHPimoC|Bs{*tfAGHXR-;R~6TX-E-yXfB*11gC*5SSoG zlL_Ayq}f*4qr^*M`n0vT*Iog=v^1eiu%BXqeI7>^@T&MI=nWx9^8{+dBy3>Qf0D{+ zvAUkJ`0dYPgqUTx$MIqGk)&`@+SGARccycP>AQp)ySHTgf@IH{N8vb@;d}n zr>CGUy#v;prNOPdo=}+~01v%~cGJ*3!RDSVP3F0k15R{sj;Y^BiQvHSUSNR1#NboA z<}sUAGGMxDoC=jrZjNSwWzS_fkIS?~BR)nuF8!F)K8$n>)OY}blM6^{N9kq2^6HQm z0?P*0(v1YcQvGl*f5ofT0e+CZIQ0ntQxq}<8^HrjpyP~<0Xk`4o=VVRt*u6|Yz#YK z0Hu@~$^i#4CZX9VqY7(2u~SN=1NFe)r8VB*E%&y9LM8DVJoD84S||9i&F=iSXHk zYNeHR{etV~RhX0aF4q7S%1Kqhtf_4ZGc#nr=BP!+o+}^U?CF#7w4}?`?xoj#L z->!GEvLQsURuN54XyN;y!c{;PJAGoOm+AYi-ga-A@@)?_bU|HKjyq-b`(V-lX*z6| z2v{+KVyIwTMG8`){2#$<=fdw|16hQDNSjV@v;wqN{TF7jOyGIC1(_ZEax~a{q9Iw4 zaJZG_u|f1F^P2quj{xl0@tm;7=Ey=vRC0;S&HA+<8LQAb?Q+AXZK;h7V^0V4>nqifCF1VE)h|J$6bqlh>MS9+)#MZg)y+d>n=pLo!_N#Ng8UFa>t~2it%SSlPd_nw9ynA%N9j-ny<8b5lhw_y?#ei&##v)%#RmLAbpzV3cbtCoaJMrc zZRvCcKN~92?YRXLJE(E%G|dW5vUDkv^!1+urA{D*n&Geaq#2?zljLO(2NFXEIOw5o zA@p3N9$r-=F4?c2UInz6Bn%a`?elJ=VRT;}64bCDu_7YD!ONugVovVYlXlRM0iyiadYP_SN!rkU*`}PrD4VEl zm4~6O>93KSFA@9|m*wlrm6G_YE|V8~>*HoU1jKMrTp!7;3vyQ?93it7q!miFrPAsl z1~nwX^apdu2YWTX8*0h$(XMQnEqS$Y!ct6@csZ;vOb)r55Mqs%sE*}0-F8zF2epTO zM(T|a7NiD+>rPN{-YIv2D~E`VeM69~@6E@{LghZdcX;`lMh=rFFyoa zvgBaUSp`$_z%r^k8f)SW@Z~N`=CIA@MB1_4%G`V^&_ zG{}^nQ(Uo>c}KvTf3FmWB}}IK2s$qy-S?3MV>=J2%}1Kkj&zpsQx@rKJTB^Qg^rXgMy2_S z+-vanNjleFaK&&f4XYx+LJvzr%GqB&8YINSU{R#f#CeNCL*P}32*LSI#<&O%u($5& zTr+=7%%nrNi0ng}p%Rj)i1ZOkWUi%>je1H}Wvr#lOWaJ$X-dw1 zP5B@7+5gy&14&eEreP$r1vY@d4d}9w=fB0rU>wMd9%=dVM>F-Hb#I`jj@_@b&Bfc0AB&r`}dcId2&Cs$t=v3(`$m^-s(GG9L=C1&t z&S0vzf@!%fGpWwmK7Vi{UUm;OEyM4?HxLpFaQXT(@TNRiKs}BBwKp-{OO#9zcKTi1 z63P+|)AC zQv|wwJCl{&E&xP3OF=rpDv3vx-d4sdL~Qm@K0wRn;^Da=$M3(kq-+b&WE{8X&1eL$ zZS@7)Tfk+AR*A`p{_A&L7B%L+w;bqJ+D%sSI!=d0W?u6(FkYcgi%^O1GA7Irlo}#= zHYn5B^Io@+wQXGMltR+pl&w;nM{eU5xhTvop7~`j1i*F@M_e~i272UlZfX9@ZuSco zx?VzapjkMg67lEC%#t?IRI21XuN8b;t`}|N?{vzD-9i&@zv)wySh09mc&+}9-A-8w z(SOI4YL^pe2-QW2!O(zdErqt$wj+~f&$Zg;0L|4`%nCIe0%UWqUI#~#q1*`DPM5a7 z=eA@5FwQv~=1rssH$-U>wZ298b>aThWoy-CCI zh3Un_@`hTlF1!)N>Zyw-)+T@A?mJ>CI*WD?OJv(r1}7{Oe{=BYH>F8Bhyz=>m0d5Z zSx^l_eZVP;71QvuzCX>wdP6Q13$MpRki?t8so4Lksp7_<(Bdab9NZhO|5}A$VA=}O zYz6^;Z~Tb7hrfdClEOxbqaMcdByhGgo=6EMG|ulaZ%xgRl8+uo-NvuJ1b4!u5Q|WwkL^qzQ9z1ikh0^;Qhw&F$b>jvJQB z-7A>@_-r)${*!M$zb}7n)5fNi0YkKzuNi#@-bfiwFHKVtQzTvD7tuuOgy5F67(GRM z^al0BMKo!6e3#&R`P`#60Y9dL%^j9^a#70(dh{*gbv*0&$`WnOcns`50=k)CWYF zF6SRUJoyWE$qyk01lu`8z+sBOc=D5EvbJOs)CtuA2-xaXsW#B@X=0kxaqH)p(~MS} zn7Q-2j#cP_ePfDS#w{&qxoek#(%WvV4l3w&Y15!g?Ma#ryDeG4b0PNwNZr6BHtUkz zM$RihH(jWNFE-@}!dtVYg@OLxb6^5xki-WjqQlHCorjIXR{MO5AlKg&&*DkTqEP{_lKnDbrkw~7T*&hZ&Z zpe*+UI<|eUg(y8F&_PjqlH>p%;`3QzC$@_?gVIpl|C-)k`RUb-IEqAVKahl0a=+Pm z>Y_(?x>>s#R~UxABcIh+KfP<1h*$nF zN@@uf)c15Z@>W@zC_WIR8Xcx#kc9Z;18N(ir^wIV85=*XJ7idy@`a~PRmu?_@nx&2 zZeKLqIK~ebnHFkc6_LKwuIgiGw4*xtI6qJfWrOJ6VNmN1N$rORe{ISV)ey?sJjr_f zs=JfI8QpVe@nVXf?y~voiJr?7FS2!Ve{~#Ja!yoKd9)Sbe}nUMf>=oO=UrrcJTDGo z)U<&_JYyUMH3`}u+S*e|ndV^&qyczew2$Kv%(E`q&#q6R^ud+E@r&Xn8Q~8K2!0Zw z$->6HE#Cjd=d)!X#MT+Z62Tu>A{k|_2sO2>XoSfU9x=Sh&Fj)%5teSm^uOtlPbed{ z&dTJo>>{8xb6>Jb5WssuWa~Ur=zwlAneRiqG|&$&rwfZU5NZNT z_U)Ff-rD?U5S0jn16K7nm>sQfBz7y>J9Ekoi8JlesTIFa*hkOI1IFKWdUrN&N zaQo9yR+D85?yf&wkF!xBy+s$LNHaNoM_;OyKh+nfsw1dUk;PL!$+-HUr_~;J_JB#YEiDDY#}# zXYl_-IafF9Y2jlUKF^8KcxR?`mX^9(6TFK9Fe71CVyn;&1-pj5FUK!*^#78HNz<){ zg#l!)nz)CC?&sq5;PEcfTI8N>s&ZlzH}Cub^UZ)qG+q*BfWw!d$Pt^7h~K~fVX7=@ zPJ|o6y8ASdd6iO;y;gm4i+1%KU*7+u`Aq2Y!lzK2z(72d`!0sj3jnprlg5K$nWGG0f zV!1-(aY;my7uxlg3pF2rX{oaC24E6X$P}I*!0T>p?FXQ$Gh=uh)_Fjnv()WHyi^9@ zLzanXQ;S&GVSFzl^fB83tX>VY_Hzom_zR ze6N;H_32}O>i{m4#u2QtixpzXLCthi;Qu?3@lWW6Ih(LpbEXRCk~7_ zIbcZDSX4y%f%6d=b!!h~6vDs2AADL7`&rvQ@%=TB5h?;8vCzo^+;i~S z8Ox@;#lU$q1|Z8cbfuvo1kP_GyAFO&wcTg|_@Rg$t{)I*E%^T6x|^vFRMd9BWr8=r zbjjQjj(Zo8MwfXIcbJB0>a(8m(=HJ}yeD9D57bO>NqvkOgtP_G4eG?4j$;bN?sLgG z5Fqwr?}?BZZtT*?b*5ET+3e+aOUd9aI*>gQJ#>5tBJO%7x88KudJdgxn=-4cm_-$pW6b}Wv6^_|4=>e!06!}gnRy^prs6; z@Bc`dvo4Wta*H6p$9(ykN9yTV_fuiK1F#W~C^)&4(98IJ5O907IApa1t&l|YSZz%K zaj;p-3kdE6UW+@}BO4XpNn{|l%Q`1w-UG`Heh)#0l-ru}lj%}@{>704FuxW)Y-rGJ z`40J-ISlql^`HUj%Il#AUq0x7nfh2HZr&CG0@C(@S)Wf`wu!gD8L*R;(hnnZsVZPv zk@Ud>s3aqwTxkzx!}?FctfMaIV!k<%VzHir;66(|`>{8;f{jF;XJCOkU1Uza2CpID zY#rwuoY5BX+||3MaubgBL$s)+te>Zu%22M4l^VP^J_6GJ?5arI{u@&>+Fhlokw89- z!Ym$AXPDn>fJ=XEA&Wxoy3%e08(>6a&LY2c)62%GA~z2`2qdFc?KE@^wMB7tR>?uv z&+uZm>FOt_vHX%r(nNg&;nO>qF^hwXG>O9m&Rrh;k!}x1f~!$bB4ipO94Jv(m-iat zxVenXh>T{W=0?R478YvHDR{s{Aw|J;{=qTN72vjTsYS;{!~ybGGt^fBEQc<<$f~0V zwaus8-_%D3dXdi#f-785JdOf>|1iC_HX^aVzii(lDSkBz7@k)@LV5xCF4iIpA?QR* z*kW*LdVLst1C#qOVN+Q-hR&>mz>*t94buzdS?KXCNM71HTgsNyW4Ca4PaGAbkIsx+ z33}~F?~Lz#Dr~Yi-o+a-`^vblXe-^Ec#dXzo_CtQjgdk<_&$mX88 zjuFx?2OiLTl6WSB>_>1=CjFtKv5jLbM*UD$*$96kKGq_IT-||%0EyE%?Lv~SJ*y6A z=IhB|8*?j<3x2zOgC4_Kb#eIZ5y9)+cTwM@0;dmJCaWL?4M+_%KBTcAxnvEc|9XLa zB9(7_4v|bRFJe-YeCabVlRR|M{A~{eTGhJ94nSA$>b8z& zwfgex@{D#M&5fXqUCZ;fm+XENe?B_W&=_e&9NgL}v8*;=&OR`=;eBLu`Uo%$w2JYb zaLE``q4%%G1kPPrE$YM-SqkQ$DD(c?Y$r36=V0&d?~q!Ku2DTplzj1BNCw# z%aCGV+2+ZZ<}F}JVpJL{D`3K`WK4XxOWCxfo|nAbC_`!lq|y4TCij09)DssieL71& zisS(F#_!NTl4|y)D){4Q-(+{mCR_QH^6Q4$d&17BrH{zvE9wr?iENovJYkJ3e&lK4 z=OE+JEb@4BDQ%vMjvlu?OiZ{kEU7$FY0WDp;ocycHY6bvI_C_R$pK z;PIasydUZu{_Gj*hD~S@(Nn1CUz>_#D5MxxseIj{&eP^HQf@3J1f2-+et2dlSX(Q` z5iDr#PBXvI=WuKQC&96D&yVr#GJcx|MDPT1!M$snJolIl@|>+J7Dnz3S_ECRP%Ca} zq76bUh1&SKY06U!1M;?|f*nvlxs_acMuqnA+MPl9Qv8 zhuD1y_!;tB7R1c*X`k%PPG%cbz|6v zfDiYQom?*7rP7NUp}E);S!9+k>qCnb>i2BI+10wfe&7=;`sTV|ljR73jSKIIx^T#) zseDa9@>r%fCqvnN39lI+6j`Zh?=@_tA+X|{l?*k1(nMQB??h6BR-8nS^Qh@%t6+Ck@IqV=E;`X; zf;FOvpV<|r(s!B2f_>!j8xr3!w)ByChM_gXhGEo47)K-zbdhMY11}v_sFL~M#(l~uFWUDRlKPLkM zIFT|PV;I_*^UFpYh}vr1VPRzwC20CZ_VL>GOflZ{#uW}->^MmFe7#$4j)nTj=^JT<8}#g-LOOzy4WE+(jT`e9`oGnul#+p?CE6VZYt>CH@_9tctMlL-=r zdQKZN_tthJ#7YyiHu) z!87Dm>%cTnpq$y{3AlzHD;kgJqp|SR^-wCMj$m4vrMh9c#y}^rGHCkiul*`PPsHoi zSI@JG%UNu^`G5WfNrwz{a~&h3%IUV&I0SDB!y2%zLJCLRY(nmGe1?oIor_MA8qZj7 zw70eb3Idr!E$cp)R`9})0W7$oQCJ#rl3J}fgxDF$ONk=DRgD%If? z0{j=GZ_KkKDN_rP!W1-epR~ymp^JX%#~EdOt2nTXRRohuYURSH zo&n(EPNLw!*53ANd(f2%gjm~G1Z+kUGeWD)1|-|8&QX^R2qlw!GJ3@`TfA7NdmHv; zr&QA&o+SQ(3BXs>>8iuRy!)&I*uGAAP*!` zJ;G|0+>{d0pVttQI~qgB!PgHf_I4v(q_Ud{N|?^*^Q4O#VwAgjZ3_;-t9&WNOu`f- zwT93L=lOVOd=m&!fjQz}Are>L(Sk7Nn}Ak8>7ey>wrC?xI3wEc0twcIeQI!0Qg z`eMzz@h7}q3sHnT#AhpHa~m+ByU4btJ;@6nTR?>V7@AtL*2x8tykv4hWJM1;=MAcS z8+w)fZ9fARevwhorpP8%OfH#?Wcj)m1WV9okJD}N#j3o*#Cdi4fD?6M6U0i(z%-fz zc*DwYG>~}khh)~|9*Kvcjn@G!h-=!IbO{c}HAt=Aa3ScitgQoa-mieMQG6C<8^1JQ zD3b0M%88cPq3v=7l8{r7?35f+ZpCd|M-n^ymsx8|ceq2EA2#5H+X+G02^1~nxvYf20iLpk)Z;&_L|H##G=}zo97PASZ8fP7j7_+07OU|&jK{iWam%G}IlWD#6fVX$3M`3cXR|5>aBsKhE?U8-BnAlqla3;n zDdlQdLfQ{3IG6Y>yKsp3Y7NUQq(!TJ9dP+qWIU1*CXgVqYgb4p0Ez2g!a5>%*q@p!z14KjYB-TgdhF?HU zcL5U*1?i>dg1TssZuc;H7hUBQKP-)!a3FRvb-HB2!{*B)gK@tPYRNbd(EhM24tW;w z1ltCWd*6s? z*a_XPY#Ox#`1KV!&Exj~(l-BT2ibMY+LvgU1H_s{FQGEN$}!MtNWSp{dqedP0h>8q zTKx4M+d<_I+`=pHsGM?Z!@e3dc5G99K*vON0)3EelcNF3naVw0wOqQGES9Pm8JiD|Pl$)=hsO8yQ#GMDZsk7Xcs6|;AefIKdkHDU;f zt`fAyQVfqtj@&!621P!+{)d4s56t1`Q^b`HRYc~b3qGx2 zPsIRqCFg*#_F!RphU5T!^24Fm1J#a7jyC3X4iQZ1_l6O7cTAQVpXCgEpoUxxS*#kZ{yO7k=AGv)>IKD{1Q zqTPE^kb8LetTg5_+`R!;iAdGMAh(G}vFasD&4Gf*ow6s2ced&Fvh_iMesx$ z+hG;mQGT_UZ32<+J~a3jkdbeJp6(8H%vtoslh$B#L&F6XegUQ{*>6;`>Q4cvpdj)NhYe;)Kma#(ROZJ>!AR!4f|XR`Y2ebRf}QQ?^mV z(4Y0scf5frzOIuRc5xoB(EBptqjKG_>tHz~-EU3FErSGc+VC$H^f_`KacM zV1IRN0^e^BPuvUbl|PhmkxM|T0+_}!1q9tPkFyk`R>Hf5;Wz^O)77kTn9DbXlYyZN5k1lw zGK38AIqaV>eOs327=}kbh>-AWUKI5(kqh0{xRiB&&t@hrl8%5oqeM-M=p6{^HTj9C zyXXS&*F2f^PW1<9%1_>hF6%Sbcc;KBTCgbnowQw+@OO8EWn^@!7T0tiN~eNq z)k3RYAGMGHf+GTI>Wwd-gkQ?j^2sGa{D{c#ZwdmpXVuRkeq^&U)4w?+q!hSMLmD7@Gl`eY* zVjAKLR;>aW;yU_OYs^@cM6&__1$H(*{&a6JHDr{D3i;6|(t_##Jh}+!2@ESH3M*2Z zVZ}3`y9YF*suo%=s>=5;#iy%@9&`!j^w`HLR|LXv*6+4eM=H-vw7}Bf@(zmi6fd${ zo%BW3QKZuF#=MnnoPX}%wzU)Iu8|*Xvf1}fP-aCbzl4%9n^A&h-v5gq6>=OPaSCp- z4@T)Y5oI1#C&WJ7=!_4NuLFEe8R113$tB(LM;dYQktJ@H*2+&3(~5E<{;Tp81|x# zbJv4s8_V!NSRm#gzydL^@I8C=$Hw-bc7+6Lz#mzKS1bNuZ~dpN98M$y#9~B;>7SMV zBrN~&*FAXP3str27yb|Tf^;MuP&Y+-4;U2vj{~&|2VZD}fI3RFks*kCQM>55AD6@H3MCaU^Vn2;P%0UO)Pe z`So8XVHbQMda-Q5sQ+;!$r#}5B}a6ZD*SQh|G&%m?`!w}E+@FB!PD7iWlR3ghm)P) z93bCs)w2KvQU}6X(%1_ZKadc-kNo5IvdRj)Se(4}svb&NlM)Yl2p}Wl#u<1KMLv!G;`UD+xTC zz?qE-;22vUry*u3LIjwsbVNpz@st~U-T{Jh7eM_KoY?pRb2SDcx<-$J4MC#7+PBO8 z?^+3-EO>z|c$8~3v6zoR7PL`~(>yRJ_<|Q@M^gK{S*H{(!~3~vLB5mXaJ7exDOp)! zKQqQIgL~Q2t*C=p8&yjsOx*_jr_lhH^W%@%z^#)Z(HMWZ-A|ood--v1;k+{{jP-B(M$0iuq`x0JvT;3Sm7N zAyaQ4i~2!dkxJ4;xp9^otGxrF5gL=xIcd!5Ij}a%%qVdG{;u)_`lolXMMOi@^b4s8-m~A0HVw=(3|X_@M~K4Nw!jcc`1g0I?Wq8!ge6O7UAl z1+Es;TR`Ns^Lq&n5jy_bVKSVx?Pu7Z~OdRQJ`6zZ5(y^q6S24?MG1U>PZN9LkfT&AqRgXyFI!Pr?I9 z>@9;-NgZIS0jE5w z5D54MJgu?Es6wN?6~5qODvHd5nHq=bZ)*XU+pKNcpalx$TFD5%NQ4dz8K3etwACnR zRT|Jw1ai5}jh0p>pee0M&?@of$?(0580JZj{vyquY(N_EJ?Znh1pzw+T*il;0lG1AGpUq)i{$+sFdW2y$F%qa43$jro4OaX{aR-A?v1xqPUV&AO z78PaQ2Bh3FowDWyfc3`D(}AndN4x#z?T0*x7pLckt99bQAPe}=1)ojopaDL)d`&bb zg*y2MR&5$Dt5~+$8I|HZ1}no(cBjWc=Gp9gxkIRTlKI^{L*(UuWtgesv$SCLSUSq= zA;5f#r%TAoL0@smt@Unz6w5)?>1eAQWH?6a@XUw!gpPDQ42#rtVZm(3qRbGdCFXB! zw3geg>gRT1(RJw)XW>_~+}LgDxBn7Zk%dAjxT%{kUIDVy{GW(R;yC)_t^=;0@CC9n z8Rd(m)cN)G4#nWkWw5B6p)Q?*rD|r*g}B9l&^w&4>(T{y!uaw)PjUrtgH2G0<-AO} zUN}d)0Mc~Z)j}Fes%Z{{A;&aqALE%H&l+JejsU-0;Aucg(4SyT|KWSVF#WHvjkPoW{QPV&Xdm0)Q}bP_+RWV_&W z^1Dt1#TSB--I>oEn`CD1$tC@S)u%QTeuymvqFnnkq`FetHjuZY^Lr&(d67M)jg8Q7 zkthMpd)~R_w`@tT4qp?(#7Jm1D zrVp^J(;+psBp=5DYC=#?KGyvh%4GIpKa zqlK{t)Vy2i28g#OKjt}0^|5Aj#P`~?S+%{P)wW++4ZH|uuPcf~76xuVvlq0~G-H@K zya8I(Pg(Cj-+2V`qO5BH=e$iE4w>Clx|7(=;PCG(fbJ{0Ylxxyf}DjUcj zNP^+FHFCAV{tU4JStg&LuxEA$X|b{hRcxVz0dwa4*=h~3Bj%)gtBHTcZGacO27GIv44Ruu{2&c zki0JfTJT1B9hD?(W;;^sZpb++7@0^{x$n-XF=|$NL)g~NyTbRY_%%VHgR+m_)+TF? z2azpBJ$g8|YJ}vDKo_GVc-inv@=QGT?_u&+JGlmo^(>orZHVp9r*q=QxTtFOJZe+0Dyze=QrLxx8Suk#c2r@ z!DN8zG&|IX1bnSzyw4e|8N#XDFuF-=H>kxt1%6?E#eP2Y{jILc)%Ws*+Pra=B)#vp z&&W_yTt_?n7TunX$d2}+3{gkgZY?S(bvh8Pw^zpT2|jJEA?ef8%T8&{Op*+IOF@p! z8X%xY92j<&%^Kz-$+F9nYK^kN{TO~%$Q7B3(hI2u%86ZaGSOkpy1f4Bgw}3!sLu>m z@1}sD!*0p1t6%AH2j#$ds>G9CWMMWPe1mE9?G>6*Bbw zIa}l0*kE6(MBDVU{>Ol75K(lTIu!bzGXs0X;4s_k0^*=G6ffPa_+1J0b9l53O|E92 z)sdfCmgkrd5ZCRq)=wO^aaSjIWjt~T)bM#=B z^3AdiCRWW9bc1V-;(Bty7yFi(7dcV5i1+bYM%gq*Ld-YuIeNh%ZnUnzl#z^UW%Aaj zHW$0NNG8=GR18*L&yMW$zfBcb_;5}N$_SC+j?C4Z_Oz*031a@&!)j`1#&N$*{ARZE zko+a5w5HT-__H0Z;cF2T22Ug#6#AJ*+U<88W{7EJOprXx?w2q69jlAqI%14JfJrui z!qP*lK#VX@l1>z6VlB7xKzfGmxEpYpkEDxf>-5fc4jPc8_{DZ z$<;}sYdq!ysNAf6Je=_NV=cTXk`l}YdIEY`Bc%JG6_>FFG(FW@mz$C31(8qDs z=39APEAc_D-6DeL>IG?s@x04WE2>_X8bZdGRC+WL6R|E*|Fs+rK&#B{%uX(?2;Cv}{ z9CGT-qCUEewE<=3%Hw^PP!t^%d4tcjnu0`rL!MY4H|O~ zci4k_mv;nKsCy#W# zCo(a2%?vAJeK&u#T)abGW=k^< z4qc4Oe|@;xt%G9n?c=2`8r|yX5;m)$&CSpt6IIR{OqgZszC>`|h>AwEBsv8pY$k zpcE1V$TY9OTAjC!FQH+D8T@0E)(w~qjvaF8k!#AR7v2nwL=ZM~N~{4n2VTR9l7o}Q zmfKnim>224GN04Qol1@+B<(Dco#ws!o^&)a?i{RkI{-IyuqNKwuTe1Th#>S=4}#^a z<>4fWoG%4CCIbl?bXWLiC{w@Xc_-cb&S}C@^t8*-Vpf-BqU<4pE%DZG3=5F7X&G|v zgjfh;#5CQ?^L3MRZ^(FqlKpGuCMFd&2RoDZeoedrtRKUl=W4D-NXh*zPpXomyZf5h zryLp^p}sp>s8szz#BoL-%|G5vFR!<{#sjm~aSz!1b3hP>c^QlgbjetTeJq@Oh3OVf z?J13N7QSj1=7%=Bs^ijm=gIj?o01p>#{5Gf2AT0d+)0pdm`$el<^qpkL`D?t$f&+I zp`#Tq*h5gahl=7-XsKk-uC5WzEsT2A17x3$#8UtZC|kN+f87Ygec3%`DINOahaJSa zo=r#%yH(g5cjH2&_s+z;e!KhtG*z2DGwO5dbiXYuyIS(vd=wW6U>35$a6v)E`bS0) z1%n4~A@t{rHF&L-9RQ7X;`kV|Pv`Io&=S@dE>($i?9y8>*K$T~fZ}tmXr+!YqQ460 z*<2u@Z6krQVD96l{kjkgh=T9l}b$F(j`g(4ZxYC)vU4~zR~NawLH>`j!S7(Cag)Q zzR(wX%#(zmTj4f$H^}3!U;g4Yu1^;Qnqx2?>Ot&L%R%q#5jMeeEBjsfs1BniioN3{ z4^h@n0mxuwXCgZwdM-?`y@VHOH&@H|(QY8}(5`~#lk;M(uDDpD7em;G(4_7Hc?Pu~ zQ9~ljiQ&--fp-SMJOENi{0O%Kz-p_$+-osbsg`QCsWuHfV94lLoMecp{jEk?l}^RMee;A>ZwHJFz@-(>*b;VcM<3D)tB__d$U_W)MCV{)+;|7=${HBR;8*!fkc-ADc7$kkVP9r@zIK59 z_E7Gf7}$@t8{bK!JmU<*r=@XMW&6Y}X4c@j*Ym}!ThYQU0$AAIM5{G+cNE(4%pd6# z6d$`5T7qg5P(H~KWjZ_?-5XfQd{OIba>v28N&KF`KGHN>LaS+5Jh6tBi#Tyl;5snV z%StJ20NH87QV_NUR`<1}#Y}~gG_iwR`7)5q&%aT>+fZ~1WL-Gxd1|qZsmmXCfPO7= zIOLU6K42E%p}bgrddXSwf7Y(hK|-^oZQKRY?+=;kU=vrEeWaOZA)ssNa=yL!U6%6k z?eeK}3fu+7^QsJelP}k-gRMsN`OXOt6oWzs>?_RE5=~E-#=<_omZuMv-Vm6ae-7Cl zbRy;;n9K&hPVl1nO#dN?1_Ps()*>Ll;=Z{A6P3WMdJ>SH{yhCTKdY~IUBjji!+X); z71ryn7%U~1b+4S1>IZ>)Fg1r9Ajn+@U7`zGq6g~-q&mM>6EOKK8*jlI6yPH$RM56m z&ee>isRuubfL3{cDha;GT8oBH;|DhPIu!d?J&fe{9mq}VTdp=n^uzt?T=+#mE!6c5 zJ3?f39(W+6wQqW9v`a+y$_i+5Yjem38CP=3p$ouX4A*0^MMmBGH z^H*k6h|}vw7Pwx?nAxv62?PJ#yN=*Mg1eqqk-BtSNc;qw@hlp8ExxW2(ifXKA9c$p ze_!4I+#)^7l(4)O2-;q+Wzau-%cwdDxtl-X%lDlO) z_A3}>8ndeK>^u>R5Rnse3z7(s34 zEiJ)5agLx${7u-yvOgXrgbmDP)EO7;+&>7j5OoH0U!lBd2`9)( zG%F^ygOc+2=3CMJCkOyML_B{$Of$95Zk6DbL#kWdZ^D}&S`4SDco zbw8R&fOc*)=3kcDm@HM|ntwG|fd8&frUiTmSz2$7NKeMP!4;7yJGXUlg23QyP?;TygL^FEoO} zGgvX|`Z=UWc}IzwWgC+cMQV%X>*dzvq=n8~z=DN5_ZAc{*la zYN)HL10uytUULXb7{Itc+k`#-Nyw-O>^2gm7hu&#pf`-%xhe~8{ahdt>Nz-g6W{G( z0X3MA*UWm_$MbDgmWuYJ*3bA70xN2&LFDhxhieutucCP_qstbHAM|HA9X*S(^VweO z)!}aB<-I64!y`Gtj!Ah9#%(hMUIq}UJ4+es{j!1?oFBis;GH3`r~PY?v6Pj%>b3!3 z)m?O2t4h>kx#TRdUn|iUx6#?(if0>d{p&y}0RE{UHtF@n#pbms)9G2}-fY@C1Hn;% z?0GSc&UtA53l&df1$T&%&F;5F@6}j`?i}F103)-qQ60`=1+EGXuKgj2vba7cY9#8) zeA(j)v2J$g&x{9fU!|3;1%S}9Ix$)ixL<>zJDBE-cKUi0;e^9@2< zGlHC27|+hbg=gAGX>d!+F`7_NuOMkNjqeQUC!$}iZ<>^_B&~lY8(N;*Z+szek^T1S zH#^AKl59{dtGlW%{&5?qofWo-#e*NilWt~YO%nXhQ2F8Ys5{5wf18>S6*sPnPzdQ8 zwT!ksQ&%i5iafz*6Ui~5q~9Nxn>P6^Jo`AkYiQ(U%;hH$k=rvEOQB!kdARbp+Lo~q zNaOO0=xYi?^av#K2841eEHvklpp^Qpsc0Ha1NCp2CtNk0te9VH{zZk zo-RI7%@r(20y2p`5p7FrMy_MF$4`lV??>I?z_@3=k#g^`D$^5!vw!rHKtMkU_ZbKT zTsU9M6kG=lCA(?4shzKmzd4}aZTC9vE35}C5@d+UgudfPSJAjdma8z!@OC8Ly$k=4udwj&y>)?}lSK_O?8^h&`kEV5r zP>Q<~<@4uwYrl$Qy|HR7fcb8Ku5VwP)W2hQGWg;Vu>I^A72K8_VRxx<#?cMunbxoC z6sz9hmp&H5*Y>aBcv4Q~jyc!avguDlVF#*10KxJ6^oKNusIBB9mS4B16I{*hhi}Xa z6V}O`1+VJ3acq>DRF|qhnsnqv$C^%k|8QX+b<&=jb#+G8s9h|6rMct9crb2Za`U@A zD`~7mDPD0iN2p9Quah=_N=G~8Uu&~gBWBM?Md$H)p6}lkD;H0^!A5rov}>R3D`cu= z3Ty-Cm@i1ax*iDzd#U}RB#ozIZw5(G9Gdcv0l&5batNl3Dj$RIF6OZ{0#-(V-q-a( zQ5BiI7;h7d8@&$!YI=Mf?$%gN%5BsGxtJ^?UZU;T%y!oR+8^Us({z4Yymo{YMR)q2 z;ceEk<_GApPZWRTf=0m~XGG8Ta z18Khtzmp~S?0a*walNau!Rc|n4EExS;qS>n2-Nv`GO=ag z`?XHbU=H|#0qWU$l`6z`0|Cc_8D4WT6JEq6;{~WwGH3CAiVy1Qnk8<<-+`jJYY7bv zjrqcOC;#t{tu3}ILz2{*54)tAufM5BrQC1c$TUEw-F$jl4U){{-tdpPW zx1dX54$9m5seV1xfLhFn-jt08L|%Z91fJb0>wbqIfWERw6g_xvkurTjxZ3@6qEx>I zsHlMO&35(Z2OpS>BAUH@AU9wMK4(7;c(i@nL97WV4@S~Q%_0WBE6m%&0HJgb%w_%Y zG%j=aVl`c{cM(?0*LlZRUBhC6ddy&4YeKypUZ`s^6!8dX3?_M1RH97Aa8EvDXHand z(Hc>)_G{(Qy*nQp0*LAs-J+Eru2Cv5tQ$l{g_IXzv)^ONxjoA+wjL56Uf7(z+I=~+ zo&8x#vRj=0*~GxLP42e-vk3!`Zw8D!9M8e@_6@wK$hM7d_N0$&(@#QP$s)PMqez-dE2~~Mce@hMj9OD8DJ9ioBZl*RBI}E+-s~=BeAZ5RVHcrttKtiCB{0K zU!+va085=f@knSS579RS=wJfaWdI2M@^Sb)@26a0&w1H=J6mf8|MIzeRB5vIatHSL zsl7&=yv@eW6Vvs*vAxi5^E=}*VZ&HN**cs38w1T4i;J%4vxPrDTzT%Be04UBhkb~I zp2bpqjk>CM@cHT0udUVqv4X>a0g~@7bCkMv803(!KzcI_5_PEBXo9851=9Mewp3qn zO#6o#s(^lw#d5keD)#||CV_~<5heAOa8n?T0x!~aYKB;9u~T5ee59&EcK{*|{Xyso zptGZ&kod`W1|9%inPTDuti>EWz|M__VUzOUTFTwQ-6G?bi+k?x$M!()1$S10j?P=& zJ(VVwfU9fwrZrEAt$ZHt+Q3hV0&Eh#gAhiNS~iQCT3ZSFF@|>Yy3I?%480YcR}!D zJ=&#FCXtOJg)aF<1-6D`1|> zGM)OSoPsa_PJW0aW^DosVxdyJXXpI&bO*uc*P)6R2rC?5LYdAueHAbzs_+Qk=JnR$ zEhF8|IyKv6SB|6w^X&H5EW> zcMC|Tgi4o4cXxv{N=Y};9nv5uARPiq3y5+CJKpo2?f$OsmwKs-HRl|2jOV$ZJB|`( ze3R(!Vl_LYr%I9y{Ix)~gqCUnMgimv!cSR{U8*>$S7lIWh!X!<=xh@y zcB>rZ0-2WtzLjCIVth!i1F~WhWuUY84mioYL^i3u^21r=)Lp)XXH8>*j#Y*$UGb;E zsoF7(6q};Z!iQiQHwH<_j%yj`kInVrEKSs8+Yi>nb--?dR6x1|5_as3v5x#i)a z5s2_T-_6NnFdyZjjNdt_q2Pz_EU9Ztb963vdOXI~0*EoQAcuw2|KMbU(5UhHlc3b+ z$Uj~=8)8=ouLT@JP2fsVI6Lz z%4t*LMvy;iAKHM8nGzWnT;zc@N_!xn9={qRI6IISexCU|W$3J1cV~>^H=0<0R(V;v zl)PcP>rNV17i6Z5*it~iGX zv}pXg+KHFQi`mbxRrwSch`m`Bh3|n?ATpE0>t(2?|Rr!XCyw{#JW&sH#Oe(m<&OVwq5p2IQOO~=lyuME`^>;}z zNnmf!NFix7pIhJ;qa8IbQCO2(w^++qefo$&fFp5t0>G%yev1goa0^g>yXCV6B94)- zA3QnqgkelTHwYUUut%X)6pvTw;%6 zfP^0dCO$0=i6eVg%lb_tGw`+8&FdD&&XbDDMQidQu7H!?Gv>T~2a6Iq;RxV4p(?nN zN@bfX7B4a;#zJ^oLVO~zpmQtH%oY1>IRFGQ_KWv7_s6sWp9Mj1+>3gTe;k*4V<82%E$8X>lCffoB%FAjnj-%U?d;`HC^% zAFXNYtj3FF5#kAAx8cOesI~Z~QCB#D*y9?otDawi_~Cre)pCd8J%=<>2^rg!GXdaK zz2((5*A9ZG}@9aGbe;e>IK5&b0V?^`j z!}$n9V6Z+z!iS1+h1kT4?izsF_s>#yU|v%B+9q@k8(ZuIE889p3KYvk6T(nWA5y;vl0=`5nO( zXTrPqk0J-0D3Vmr5s~ypGZ+FZ+;h6rSPgk3k01R6%2njs&y)EWf95J^j0%~ZktFxe zWM!}<_`Y}Ld}oM4#X~yMk>2~9Z8CyM?%VzsU@AS*!E{0yYCp& zb}P5~+$@q{Z&uj8OZ>?=&9ZND`ZANUaA*`GPWz=^TAj-$VjN_5_*Qdl3TVaF8IIRZM-u+k=HU-u#K9^S_H`WmhCZqXlzB=*R_AdE+lfWkgqPzMw>}lt* zz;soPl$o!VowNFlzVhfu*{hF_EI$O!4gZKCW}{ZqJ(%4wE7g{lzmy8uek;;QGBByj zE0{EZ;92>O_2YP0Nrj>MnsU%iR6BQR5tzi(8?cBWp_3(a__3NN%R#NlAta*P~GnqXU_MnC;yE) zE~F)2U@ZBYRuwq)FrX3iifH=`T4}oVntqh9X#2hpdUaCtS`q)lpL&x+J(inWuaD1b zXphe$1^9+J-~1LxV`vAfO``LXs0&XU_|C>sGN|cNF7PU1f8J|P5H}>EG+WnyuR%S- zYn^sfi5hyBJ0gnOX3mvjeserE4h&{;bjW!=*@|+%CpSGErelP7-@U~ zHw6p|-8U-&x|!XF)g84o>vzgwo*5G=Rt?cj4Fke5<0qg2c;LJhD;?73e{fBg%fF=2 zHQbiTO34lib*vlQoisYVXxUVL(pg^Y2X4kbF@O4+A-z$K9S13(ZA7#)o*;w4mT5(WW)o-%a zc}GM1)l*NBRI|N_poxW$b^(4I=I%f^PFq3eC^Ti4cjhOMCw#NWB_8dC8?7IY|Yt2 z_1C#nYt5V$$g-bN<69nK8zYE>aOAwGbsr_%a`}GZ>8f$fyk0au#)hTm6~!1=5q6zq z5EO(XX2DL@8Z=BJ#zg2Sejj|2fc-?a2}?|e#gwX6+ZKHz0%NH15tMq6nqKSVW5K`+kYj~n**Shrp}jN-1X zxmJ&Z8=n6IpYNb7_AG+GB=df>!9}=?_kb2ftLUi^wE&Os5{?{_y(quLC)JB)Gh4In z;7BTdw)2XPfwg8#8xUeE>hmM%x)tF~2K6oE2yh3Ca1AT62D}*LpB3fUP76tswA1=q zXXw2~f0-0d=`pXBS0tq}mpbi@{(N`u2U?R=!Sfc% zS~7dKXScCxaP=w(CyfFWfOwFguQKPSg7~2@r`yfR4q4-%2ih6!aR+TlnbHd^?SWB? zl&l*K_VAalLP-cLGBGVlpV=s!#4hPQesr(EjuU;@=QAFT^;Ddl$DdVt0H$HpERwm8 zYg5bfMyfhGQTrnrUc{a=vCQ?{cWpyFP$F!v7lX$v#n>b~=sSSaFM;eFzgI=bd@B$os7SK|<*pKq zOv8ZJ8)UNPWgb7cgLxTez3Y{&q!6Vq&HKqQ&ir_W?%PH(Oq>ITC1Rg>SU zelO!|)GLayJ^azlyn1|BMZ)j58A)!!Y^c7Paps1vaX72AYfDYSkf+nWA2~3a1Z$9EI1H3WCFv7(f{sLfk%~wf*5|`$vB)& zBMRO;jrW)=3xYrw9ujK@P^7o%$6jO1;_yRYG{JtG_=$EKzER)aLjls0>;MbCV40Fi zqxp;^TGUxAapb}Hq3sLIY7dU@S1M12#E7ut;P&i2iugp6f98~1b(*$fPv>c0T&vA5 z&x0aE6yAH|Cf58W%uR^eoH1e20119GwqjA-i@mT!A5u~>I!Nz@Z>Lo zL}I*+dPHqHHY#iQDOF#LUbS5dO7Nvr%IwU%=?Xx@Q#utmT^#4)FA1QV@HOm=*TMj^ zE)Jvg? z)-u%EjaVJloAeKaAn)3>i0xW#Cy%SYpYFgG{yIiYyR?} z&znO1564<}t}a!HBiPpXBy`?2+X80<_tf+l!ap__HEOiPr{&yTa<)6N6n70Q2JSbm zs_Ir&zp-c>CN+`mDC4j2(p3LqapEEB*M9wjL#~7RRf&Sk3r{X)t^_3Aly4LaL!14p zm)f!nSH6ob1K*N&txD3oH0X0m$c0Gdv_AM92%8P5AT0i1Yng zqji*uhH<-D^HaiOg2*EX_4gsSgQVxbf-6=? z?!5ICd-+S+dArA!62rC+Bs){=b+Zv=C*rBl3dOAJq3m5}wzjt`Uzon&&*9kE z`pZL%+`~N!e81Sk6O=^C&+iA;ND?U{+#yly7#g*>~yj zLX6m7Z2jaQ$~W&iL%VfM?7uuq5Iuu5<$)Z|RLUZQ!o3yIA9a1E#UO{q^jWXCD`JuR zdc-glJ@1W#PsDp8VnDDH_hKJE6#5l`I7o#>W_P$ES$Ed9cfs=`>P@qW;K|(f=)&z4 z))PVJ8Ky2ABmduo5!i+gt($2_HfzY(**dF}UC~V) z&QVX}Tbo;VgbxQMc>Bo=`CZ2!n=+Q)etLY|Y}7Us*O05n%a?&kZ5r{Ir8&#`{`z8$ z29D=b#`k8m^Ue6@qm*8teXO^>Vhq~`Kk${7ID0jTK={hPdV6 z3Gjnf`Z)mYId<@Ov~=uvrG!YV|4eyeFo|AaGi2T5Hs&?iuM$imOWpP&%?JLszKWeQ z=>-=)vw8`0nGV`mP6nr3)%b_;d8@u~u^9P!YXzHlSybGlCyB91uEC1Fl(uKD)f6#n{I*$dyK-ky~14jKkCqj9S|K2 z;n@t-jZSELnfVu|%nSU;RUzPBfgOC^R*i)hK|v(zHh>mf?4Jq&cI2YKlrKbne0CNP z?t4C7f!u^F5f9cycwbAr@xXPADI1I|Uu++dNb~#al6^ok_oeF(nmVLW8=%1E1ICxb z8&ncqBi+}V80Crt?o^EV8!AlSX_I?$h>grpBFT~U2o-s8bJ!$#zmq|`%^1FMaREcq z7;+Fea~@B5=|AjyNd9$rb}?Z}yu*7R5GBz*p7i5uOmwaX&tC=6s=y&%PXXJ_1puke zl_=fqK?@c?U-#HgD7HRy3mg z94S1&C#U^>6$D)Zk@Ciq9Y`vayPgZTR&VeCVR=AXu!}tguB{U9M-<4WNg#U^&ExuH z8;Cue1v>y+Y~?X!E1I;dkH8wR?Zf{}G?#%0wg8)vqX{pzbp36TzVsLxx zW$JtQK8J{7Q*MnrK-}qqSuJPs$BflSwwhMT@|E%wV}^k*g6EBM)9kegT046!U;liN ziTL5z#`v}{Wt@@y%~BWc5bdWAKdAju18j(}DXuRA;lHu{^tk`CeezYm^R?5{ZgG94 zQ?%Jmrvax`Wh{7^hgFCf1cPMC$*86Z_3s_J0nc zux-x4G<<|0{3!W!Y?6>QRpNkw0;xc1GFQy6 zUF9wzs9!+O;aR?11IIbF&ySfMJdTfog6$}ELsNKph*vZ-OWW#UnI=uPQ-jOrVG$#O zz#P>>sKtrfCVFkF)vLoZhloplPU_%RLtM&o>dm9Fl3}W)m)U1A)Gv4@MQs& ze2cvxWqVs2lGf}|*@HDng%PmlGq{?1zb`#zJ`-2u92B@t5Gvr#GP7n1LZy29Ep`eh z(+y9l(;DFS8JeHD{BNeIH1?a>E>qLdZ$xwZx3$nF6CqN`@TadOZ zD~n$=y}V9v&>)YP5c~QZIEOc5Qaucaqa31~glWnyHfKssdA?ZlU(kps6qAyoVv%YW zzH$@9cUx)K5S21_6Sr=lz-oS)Z+XP(+M2^c_{8~%)T#{$%bO&Z{ogRB)+9i5Z7hZfi%cq=nRuLhtcd*hnX_Y`L-BJ2fe&maMc3OUtR;O9FNP@)VM>o!_ zlJQ3`yw^VpqN1{iCD9L?zt`PKmDmXFAC6y;#Oqsp>An|DsFj^MbrUY#tP)3dY4!?P zu`}=JYsb&t{)pl}V9watxQ4HR6cLpk)K<3EeJ@c__Hv=Z9f)`aAY5)n7MAzJipp%} z{M#t03wN^kXJh~@E4%xU%ND2OnESr<@zBnwd`6d=>Q5vO`MI|qZaVMyIFw=Vwwx$b z+~$ETIQ>Y9MmzJU3zhmf_&8SNuKU0-qt++&{&kBP5$6?sSW;5XEScvi%?b$4H9KrC)< znzDc5;=8X;QTwmIPIXeV%-MssollD4Vg~?Q=Yx(PLNQ-(ajii+-lkaMX=ulzKJ=JI zrHkcBw9}4u8Y?nAeM#wtDBtP_pGVcui(HaRJ8e(E$ih}B`s!p*yZ+;=f5@{46HgYM z%Efr$oy4m4tSq5q^XtHi;Kl~4gV~e&7)|V~EE{amTI*=0n2>!iNLO}$qRG61#-*E^ z$E+WU2X#Vd_8~F^d=)EF6XGDXSU*DUHqszJk}c%a64oo!!VtV}G}!VO$W%E(+m^cJ z-{IzB)2k|&x|7ia5^Lo&*1_z8Z4tP8z~-`WL@~;&OlJ^wWGPF_YD>~blc6f$(h2u- zvNa~yLSK_Lx$&DR>JGT4B;+3ZYVv$uaiJ@9x+^KT?p>tZpCfXXZ&h_X54cwk6xIbW zlfuXeu?^;s#fG{eCbPymWey1u1m9mw?)J!z7^f2D#eyayEcdrN#7DA4p1kz*7+KPn zZUk5g8d=G#AA<#Dx9@@Ltr717aH#N+>M75wMzyL;)kZA71jtfXk^%S9KPx zt_hLj#;R$w8Gi$;5{=A+#jxPQ$gBGgwO;YaB-5LVKMCUrf7Q56Cquo4@CxUi=h|{grldkv-y8X=RmtG^yXCL z^u$g5+~TBByATRxvm5{)TZh)PQGx^iMQ}Fay^Fv<@0ublsL>LoxiMl1>!v}-w(?y{ zaYW|X_hwjawbLgoTWhGX^ITsTE@e2Mpwn8XY3H%e0Pf@Nx$hnNAolq%KS2Hxe{=-) zwAxs9_eNeGie=B{r<@4KAt8@rr`y#id7magsMkj+EW zAc-r}W64kO@WmefbkD0AxoorA1LG(`Jmx6!S0{uRhQAcGVTtbD>uvv_S1zq-Cp!$WW!$dw2X z7hGZ!9P(HXLIngw^hic&Trhqddhn433gid>b08c>l}j8YjM(k86`1*`mPEfmq1SL17GDq~y=3@+>|&28g%IguLgxe!R(?4xb|i+ivlT6NrdBHV>sK(_Ty&ct^e7~ z{%h#}@XIJIK-5P!(t{0QAdr0tj=sc?0P8gfIt78HK{K-chWxGlBMXUAQC>?M`8Fx6 z2LdZoN7glO@IcWIxyeyAuU1YRA?lJ^M> z!;vwFkUDII|Kr^Wq}3FCXy;E}-2+|}AVBeYeK%$i2WsPoWLxlf1ajMD(PT9zJrrU| zdI|BQWsX9?^comlOn}$~RdLYGfehI-kR1uE8-4-rN=eb@i&Z&}?Mf?*P_niQctq6q zmq2iU%bg6lUr{Z5KbPMyDH0>(o0o!|JDfm`vkbh)>VdF^O>DJeFYh%J62-p^cpf&@ zNNr5(&Va0ko^`F3=)aw$0AXgyy2vHsDbxsVbEdiI2My$d0rym{u#W3>OyMw46ZnH5 zgHZXeB{SfCqm;T$)lnA)N*y!r4P-BL2pt@@m~De>PZ?UXudYu&+$!(@=aQ5ZW^HZYV1Fl)Pu57#)Mmy5#Q%~+uj5J z+W=T`0>LpbI{-iPH*nNc8wATyp1v4v7O zO$?)yxZi02ArY7@6=l)jl*FbIJ(^RY@W0+jmhql{V22!(|6KZ9{Qqoq4;$Z$Pmrbe z__T8o-k!@g8DRveu%P*|8@@hYKwnOk45kN}v{Tu|XeBk^9S> z2E#ahNMqPVWpJHB?D6db-9|Z<%od(SH%!ujHaZ_T%I`nI7TKqfbB=tOm%nza_K94`rC>o(6tedakQPtm- zw%o<-jKj%numq%u`vgPN_(K(yGHNX!^25tj^pm7d-D%XO*mTPh`2&QUXD7|3C7HE5 zJYTA!q;r}?!^P%>d1gg|Rr?v=ra?Z9k~^60+gYW~J8V4$RrD5GO_L|Aums(4=Lw!+ zZVbpM$@?;?pc7oCyEi;m;SQuETi~{R<<&b+e$69mGj zdE{A;&6^bczzyTh@MYXhdQ1<#1;_-kK&c0#MZ8^nB=Z=)FkQ6YD!NI4x@>6>RE;%Z zvX!LiGtU3da^rsvam=92KybF+J>8u)V22pL4g#kc4P!-VyAf)VQ6pw%4Snk-zEn{S zYR$jB0PKNdFo+@!o(-a#pmuiCY{N}TqvnTH=Dwi+E*<+kka=GjACi(w;?{MXJ?ED0 z|E>I;CqQOsvjXp1r4@@EYqay^`!WtOH)8Q4U}UwGPGBi0f0n)&)TBO-15OmmgXW37 z%m^-I+c&g(#RE}uD6QA5~hV(*L>yDpU4}ma{U?ad~;fX#^RJCOpFr+P$=6+^jQJ`uE zvRv53w~JvG@R4L;3&_0n5Ej51%Z`myj01)l1R=!pq$`T;&KrF+8chRI`F(W6!2rG8 z9ERU@L6DSGLUk4nibd+CaU1_v_4~hWtYzfG;DP5LGKOSHMX%fc_dHt=-x>&*3lLUp zpypmBMk9)YQy8x9F#{}ZOMc0)6%zgfW@>ymi%5NhJ{g{(MG2n&l!)SwAdI$T;|69XC?w2x!$+J_^! zaKNnqBMTPIiWhdD%-xTPdZ(Ao5exiRe5dxE6&wqfh8VWBEA*dn-s%zD}7PaB@8 zRcVh4{ zx!*aLW!}lG0Ve zuoJS2!5EffW&d{}+6);;U5VzP<2?bkL7&}hC7a#F=tX^@jVne}E840!+?8H#Mek1i zf6SMs%(`vS?9HBy)OC|Rg7seI&=A4#JR+^Z>>b2MMzY`<{nYSto!c#L5cVXvi^`y2 zy&cb5@fzj(I2kR{4x2`3g);vX_Jbz(U zrgy<;`(`v7zX76OqGQ4Qx=+ly4fSWzp}wP&5#KtYM_=*&u8DGSLBV!?@RK%3QpXPL&7kwEY+gM}u>H7!t<0_q5Ims(fEk|@sm^kOiuWygWl?VViw+>@(3 zVK=5z-c2tO4xD;NUoz#8z4Liiowu<gb;eU)2vv-{GI-aKo--*O?s{n zT%@kTYq<=F>kWvv8KorZ!zV}OR4s%>=0Le&A0!m{?qJs{9KW9bby~`i!(*UiS@~-* zMhXOtc;?9={=9RMv&8;}#~7Na8mgi8nqNs`wg}QPXV$O*Tk|@t#xWhH!lk|3YL4^mStC z80?W|fekVsz`B%R4h2hzM&UMcK(+Rq1vI1F6Wv<>}skVM(qLr22mv4ZY_dS$bl>karCq8RrAJ)NaY0s0o^k1H+ z>OWY$QM=oE@o$Aa_4xb?4cc5wKc_!`gHFf_5sMB_eauV46TK8gu-AqHeWZ!A)WpYd zp}UN)fcq}wjTc+;Zke%bB>D#7s|32S0BM7Ad+v@GtA58FM8yC|UHj)Qzh9Nr?a_L9zGq6gEYhU@ zV0tbT4tEXGpM#3fLm)>snz~VV@O!u#sD-N2va0#wZxj%+oo_04jr#tLb^cG)5iN}H z*6xH@5-8OYJ>a!R5W z6ghk?4ie_rQkRY~iHKT%TP4HU)SjQFQ3AKv9L?J)$%ot~`*C@nSslEZzb;o+~ojhO37u@4OPbn4LPy*ED3qNm{DzEqHxv(*CPiKOZ+-2p^PfHJ!ylB4d+ z$Xy66VAMj2QGC=<1FHeP#sWW0jvd4|(2Dqq%KWq0GyWf#gJJ zI-CAvvC^%U9!APIzXe~i^yFtoQkLf=pK$*0Y7?RO@N7Qv7Jyujtn>6HB-I=8yO-9V z;brm_aMI$lku<8CmW%z3*|^B}7&BW=iq}6aLj6RN6+sm;{DR&PKA1Wi+ka-Hp-iYW zBjk4fiMC=CgmPvwUtaEY(R-_|ji=6bS`~O?6L{g@WN=ycicORuSD`+cZx=SNq~uwz zaDf@9{J5lj*&-TQD&^hEx4KzX&LC;nAXcb(S=|Wx*S}vmX7I|v^3L;ywD1RT-~Emh zWSPZ@#elGaPk&dNL4v*R6#Pa)99Q;Gc8SUMbECaCsv>nODMQ#tLUy)IA0T@y2R^BG zW3uCt({Vtqpi%=_JILZ=_-usfaI)Wf1#gpkqLCUh^B1}&i7O9SH*(&t7|-l7_vwR~ z<^^~Iq%THsi+|Ci4gqVFxXh1DxGUM^`sz>~+SO+SqnE;MhTf{qamW%C_zpqcK08SM z_#m`I*Y(A!VfB;99H+mlMQ$JuFn>-n2%!l!1x~2fcM;k95duVqrXiY}C47%+kt?7- z!;yZP05}L3fKH-_xu;e!et^6x~kUhH9#Q8V&S}5HUEJajtP&-*d3{R)V zY1ABh9ZxRsqUR{MQnif;DIofy*$}#dmO@L<*hO!l7KJvZh{63rLCW0x<=<$3XM_0H@6Ss2zFJq_vF&|8^HxY_7GsSb=OO zaJ89&&-A|hy`snvZr)J}{oKt74dvR(HKbV%86x+5;>ng1{+m1Y{2YFaqojl}0Tt;K%`Dx;E2^Dlb5w z+N8)~)wVfOnW$a@@+G)YV5lr<4`4bFj{V|_sV=bI_di;f0uu0Lc*r~fx=RG!7?Z{g z`3B7jU6!H9oq*J&w?tV0J8T3>8Zn;>gGM{8)Wtd36exlt_3l)DLV6Iz1Zi|dI(!Z> zzb4?c0E0woA0)49$Lz2q{@9aID~}dtN3t)`08mXn2Pv+F?#?m)4ZWfWy3^w)kZBdx zp1jwp0g10UPF+u1oi_lqn128MH3w5P9NsnX-7ycg`J=@AX4{2LIU%zS7%J=m?Q%~q z!zUL;^kUutwup=DN6-10f1s(70CQZ7d=Osx8aNTZvbCf5Yl#+&&^%msG^TTyFO37W z*2pi?&z)d2`yeYNcwpS(9!p8z7%)N1MM77RqA@XLTra;WLFxeReGK%l01@p6*f$7F z`%pyb%1CP+yYwF`4z>#^SuF}eoB~;+z%vHO`^?`|atU5Q&tts(*(T6X6XT<{VFn`j zN%x>*MDN7Ix1&+7!INXct~8%ffBd^xn)PYDbuB+XYL87V2|*~HcNVWf#mIzD>8n@Z zhvSs`0G(&bbmj<5`wLqzOJDjRQGhx=WmCL*EdmQ+nH^zLO`K` z6&M!>Gy7A6mSZHX4fp>}H2yVMlH`%1i6g=qsRTTrG2qTmI0nWBk$et*?)XSU_v;}z zYrqpKi_g|UTG?jRDkCk`++ zu2TSpyA-Kw`nA!>D1&rgILkBKTh0rEYw#c1Hp0nQCPBa~@_0Z(gD6v|DlB4JmHWe1 z{Nu574wVBC-pI^vyy6U?Wks?leJt;C5i|`chtbV*0fnkwY_oEkPcr<~KY?izmPe-# zZ|?PG3;>#ReWK$<&qjdH#~ZMb7OvZwC~@Bb$vgTKHq}o6T&r-l$?m*#P27Hp!XOFI zQ6Fu_MKgDi2mhehLm-o8c(D!invGUSxPfi#c@J|OY6&Xw?N)tr0NDcAK2ylj6XeJ| z@sZvI(W_11x4`$O(Qan54nV1(<|@Ux_y(4}7;rwa@?iPo3m0&;Y&)L8g~Rwfw%gr% zxU$eg&pF z=S?I%^NH%mkuY|3_OQTeJ>S>*FMv7dm1axrB#mV;3O1|ZZtbqPAe_m2ASL#mmiUz% zFwr~*_dZHD9I}lbkw~jeN8Ns*Qy-Ar?-2;Fz6839b_#}%y$?Rt)vSn&lD{7OVri`; zK;pY8g9^hEcqsZTO092P&Xt%$WAz39&>MyTRA8yNmoMMhFZ(G7Is{T;FfXLndS1@J z-6UDPD+p$5K)AQYAvY}-1l-!XTeC_efWHsmr2wx^mqJ$BjLeO%ulC;96^zJYUNvG` z7md)k!lN;~bVTtuM45Az;gbuXk+&np3QNX)3^%9@Tuw$TloSsDQ+unLG4u`uhxTrO zcn3>6?Tm>BcGoClJYZTz@-j$i0jH*!M+|;jyARPg&llMGVrC_VZ z^#*S?!m>qQ)&G>{=`r9M1U_V7(jt|?XTK&+bOnZCg|(o#)hpKyQz{|HjBW!58T6x`ej8%a$m(?NOjiAbNI>o)c^*JZkkD)I zftm^OE*(hYg?C9fnIvIkyw;G^J_#-a98c(`W1wohjZlC+u~fEXdsOkW^04ARv=J1WuA0R- z^5~=y-wY;;7o{o@mLGjG4uMoKA4k^g`++hEA0W_uD%3f_HC?yS%Pz}9%Jeb3Y9oMT zwl)75JZ>?|Bi<4;E{ThzsPwi)Z1a%7m&hW-^K9(2PXq>CX|=HX6bzG8Q<-M*p>PIz z8B5}l{bN8MUC22y-8DEifF78`w2$LmGzd-8+napzhzz+R9l(UI7Vy7e&W)|67~ssH z-{k|7;np@p_bfNtPzO>-XB6oO@OzHlgk!|0sYOb5KUn2=fV8V+qCCTzaC^en=E3pz zEJThls1ZiSZKMCD4?x^2jrKb0G6OrRmjGVEl#VhE6^%6!EJ}**%KYd?;kOY+OgD5Q z-4q9ue&05|{Hj9Uh0_jibWC z0#|sqlgF4%P=$m0!U(G9)|mWAR`JF`*`%3hEj*zOdQ)^p1Oa)v65z7!OQB@>WSDvz zWJ+1uRZ~2aDhv)U1i4@K$QFUdL;Fo{^cg}VH=hhLL8jfUIwx#w;i^qUQ? zDm+g#pCdQw!epo0xYZ(ma%N?yH}bKY`*U010nZ#Peitekzj-=>VB;v(8(`2h=okJ| z9Wfr(nPVPoV3L@$zDM-*^M9Oa+(^+J2=-_wQIxGCaxiQ#uqawDQj6U2NYD+5>Tn<} zL60~exk}#6)W-`or^!aNAK+7d!Z{dCE*CPry~9PvsGVPF19enPE8>yzE6BiYeFiiswU1tm1xy{;TdD{TY($u zf-AfX`KG&M!UZRrsQ>D9sW}4m2GP7y*rHkyU%Q?RX>z%cTUe-q40Z*Ki|-<-De3be zpN!FeVI3@SLKS1=LZT@kbrlde>{*IDw`g(unfF*e|O6r7DY;r5qJ{P*4$`tZx^Y5-Q_o(-^?jkqnR_ET<__Qt3HzJmGK6 zYI|!jLa?oz4uhBH&%5T3qQKut!Mo-6koVA*J?tSw*rMb@QW}`ta>q}a9r?Q=vZGHV zk_%xauH`*hkFgm_7*Mx^qr>pNxGj4(HbzkiPk0>``}l4?&4i69NphR%D40A*f+$7k zO>YCu5@~5dylT~3o#d&)Td=jOx(^ww7+ax=)cDX*uw7fE1q#9=aQ{Jexq}F`5Oahy z7U__FhROdbygB7N2q{zCOkp&P{?lh<73Hc(>z6GO@-diQw+asuE*+7j=I+^gpXf|1 znOj#7E~A7S(G&gmcfY22`0jo0B1*G$K=rEj9O3*AC=N+9oJOVWW}csGN=C+#Sray* zDHQBXLTqx3gB{(!%_|+!sXVt9$E27qlR{&7sP$h9k91b(okV+MVlxwD0h!otflZ9pDnHd9r1|M@V{XdFSRyCxSIAOE$*O|NGnR+LkDOKa-Is!a#1+IkN2E7u>K6qJoFPxpI z^^%{&G#)r){39M`3O(r^%)uUQ0lni4SF{pbI@I)Z#TN&P>}FjQl}H=Dz$1Lir$FF* zBsbI^LZ{TC1#5I%&7MvG<>7jt;uD-8Ogt7QoI;bApQ@8;L zyg1Eak+7w@%;*PIejQwv&gd%$LfNbVnwkF%BXJ~mgO3@e0Ct9nC#hVomzuMczz(PO zyhC{Rd&a*m5sEZh1nLyx@4b8mpF&OTXn=!;fISjn_40Y}l+uM%VQM#PC)d;Z(Dv#9 zb>|2n+FQuVHE=VFS`|mV9FSrVDjm<_&$t1h_AyA$zJbsR@@O2!0uFYviy0N95VBCy zxlAapDLzYptPYJRDkg~}w|a&IH%b$xa?kRkpEPG%=~x6ML zVZCF)V}=JbdcY0KKoKa9DA%-F9p@<>}c^~mQ1+o?4m8iYYW!5 z6dG05fd-V=-<%@HeVZNOl89;~8!XTleFCB1o>59$8k}p_KzeezITh8}qEM3%>cP;+Foxg{ zuoWg~2$6j-kz9|w^I6rY79-N*8h&r6ck-+0tYAGnY5CV_veFg_^!gzs&e$lEJ-W1( z#S@hN(0!ZD>Eg$Uj^;lCnGlp!2{lDR9Z;tgoP&`0Dc-q-SyMEl5tiTf>-ZPuYG(w6 zlDrKyXnMx7k8&y88EaGo1`}(vb9GbfKj`THi<<=w3jqXsEqQuj z>iN_ufUx|yOh}R;55(8N=nZ_Xuv3^dcKH`_;QIauI<91@BK-p6dqufN@rxlXqOBea zlrp?BjLvqUv7ROzC~rzQG}7Rx`!vSF-b5M!nO?lWu+|+Y&@Z&`(P)dCk`?W&yFl4D zToJzzNZ<#&TRE{BxIu2cpu%im&tkfXoIGL-9-KvX7`fOM@h0AZT$#}3+)VOOsAD)cM1Ah``Nantz6~KR@JMQ z)Sr>~$qEtWWMFYNe*qklJhC(#(I_P=p!*17!lV32<`4WbARNGF&}E;Lr7t1ZRc4As z@XrR6{*Gn|hm@B=h;iC$e_HD7;c%|-1$gy7S0p7gH7I0ZYtK2IO>fgq3b_D7C~eLcS}m(2wKT#5r4yMRwCC(P{xi-Z7wh{DVkY|xGB}uw6VGG7tp?K47?nst7fm(i`&yyY%8ej08V1qc*I6#UB zPv+?GHg^hzqv(nW@zu;73OjvRelb(DYRW8#g{o_kaT2c+cN2=>z>m>yil3cF;%||hMh4`z5Mc#lP%G?H z{EE@sL%E@Mg(o`dosO0LpdX|+K)W+Ehzh!)ZgRCm8c&Kqd<7_3)j*myJ{}71V)3q< z|9Dg<^X=>&06UE0Hkf%^x2ItGAjkrt2`>FRJFagODj{1l! z81o=xzLdX>9DaaytmLUm_s1?@_4!-Hday^(1;ROj`v*j{z2M5BnWvlSGm1;n; zgpPMRA#6UzXe7KXB%7A&-vXtx5HXrj$ORos&?lKdmP8bBCjUD~P27S8IFb%{WVW$77Fnf9 zk8KL8W%8M)1iPaDAU`70wp>Z-Q{vJ4`z5UXKfJwlRFz%V_Dy$pcY~C)bVzqgm$amG zZ9+i0q`SLR8YwA}mKI4VK{}+~#r@vz^IYon{rio<7zP{mxzBUOTyq}tcOW^W0?yB* zi)GjI%=jdbbrFN%69gIBq)QAU0l2Sss05qR2G#;(cw=c?v_22|K#l;mknJiIOnNhT zXS*&N8gc?$Z_4K`ISg!(b^c6mHIpuec>kv*`@c!4AnJIYha*S2yHkPgMxJj^DDqND zkS0PmJTh^FG|Lxcp%oduh|%|lY79P$k_D$>)_j$e!Jl z@ao9;akKuF%7&Onbe&h&s7d*-OE?#9f&=w!)Z(?Yc_GC1SK!?VC9+t;p0iqmbmV<1 zDuNRoOSJHx2T%kDA=l{nnD#%h!(WdD3Y1Oito|{= ze}4tD6ew;mhfEUvf8^U=b$O5#b~B8DbKh%W7_gh%g1lZ!Krtw>KH0at)=tPh>=Eb! zI_Zb~>9h)@ZU~&3IY;BqhW_`Yh(ZZ-zM3;Dcmz!3G)1$(O;KUD*J^80fq3K`fP@DQ(o{n(U};_u_IPqK@R#GG0a z=Hwb+(Gu6d(-vh=&q1xT9(^3wGYaLq{-F0ZhOssKX-|DXsb~e-0YDK06>&NXhj1;Y zq{JTlL}%LL2d}RyKcw6YXzBGtF{gptz&m+Qz_rA!)+{fQ{N|~Dj#&a$&xb6V6?$ny zOrJM>YsXKJwH+XH8v(*%ZkM$_KI)~$>)reP|r2vlFZ?Oi_^)wMm z-4zZtB9#54RyA))v70=%7vDQZ6+rt_vebdtL&o-_%|8#ROz)94nhGExywXUW`$&p! zB*z8UHy@L59B^9=7rm}-m9(Lu*$NfvEfxC^krf%k=nC^{oY_;~+AhKy!5~ANd+41a zT2gsvbb0hQ;z!DOLx$H>aFe*tBAd&Qx`yn!ZNpwNF>GR!zE&w&Ne1fmfa>i!Rv)(G zh{=Rmo~@w`;N772%0e0{QGHo0LZ6c}T5^Q*DxVh%I|Pz8qMk%H0LlXPJ*MWlWakDxmvzt-(9nPP&IQ)3PF9f;gFu` zDPn^_)QlAkZeRPBLakF0gP~~t4(NHTx6QMmx>N+i?d6(eXpfFI~gjwp$$WG93vN62a>0|`nWPcIYt3wIk}m0 z%Qy*&y8$4j4u(IENfIbPb=N|to`DEk{q?iAHWJU-c4TbUZi*Q}t3@SN#5cGdSqg8s zdRgwBAs`hFoRA@G$xfzuHRScsCSv&H;Vix=#rkpc6d(P@e>)p`|>Y%6eG+tpx=en1}}lxhw^! z^zu^z##?Ktm0;N2;pdXU^(O?Z{L4P)Sdl4Z*O(4R{iP^f=K%VB_mE;iq53KcK*PbA zLC4X#a=-SW#7}ucYa%Aofdo#upuo-=oDoKG&nqlALq{)`C|ss# z)9N?wx`z1p$kD2Q$OFXdC8;`IBW%!gUJDYe+r0u|d&?k=1JwjMC`J2yI_4Gh|K(2R z+0ut`p_}TkGg%zLv(bET3(9^{Gc>qYGxQ9%AOn4v@S23CWt~BEZKuxb6{|JeAQ+p9 zloPM8X4NQpCms;oiZnXbpsPeNL3RXNzpMZn8A}Ojpip)(*!kl?6YGLEk2*zyMM8&& z0t3eFD5y)UDkHA3dJ2`TiQ7TO9tHzwgrF@4PO|#k>9r3f8H-OvZE`bZ>a-lM5jro` z#afj8z=jOEBDe%iX-Y3%Fh}Gsy1K~~>WfEP9)Lk7%uFLdiLAo0&P1XI;cf-3-;&YA zk;1JFP0odiz>E`m(3v%VWANK{T_ctDt`0^`O(;K-ObUT~uuX#dE2|J640G?uWCSs{ zx56xqA0-j6vx}Eo+^nl3q#09vY+~v3IHR$x5qWf2dOz*`)Hp7|HU0A0Y(F1sYm=(bKWL{}J$Lp$nA+*P+ z+1V^$S-akHC~?JCVIU1PfQiT3z-Uf?*HB??SKbP*-RGuxSoTYV_x$o|q%rb4!bOa_ zya`U3kfvup{@b(oe-j>rkW&_t4Yzn$?=RfWK11EK2r!0*rMD}vv6OMgx+n6X3-5f= z{d_1uLZXa?K`BifUlcmw7l^qxjw93hSKdge!(sZ=WN$>1cxT2brcq6pg_8X!J&v0s zCNjtE-V$BF;=%Akw|~h2c^h=Y@C*Ut@m0_FEMnVxP$iU1U&CfRVtUS?k&xt#(Rw7> zC1=?mO>BdE$?oZWEOGzg{#I3NyBlr}yGa1OBz%NhQdD-0`$=BtQgE2y*Y5&G!4j@- zYY?ZYyn)MZ^9`;-VPLnHL`4vztzZpa)umz^yHu!git)Ig(aSH|r!l<-(u9LRFi9OG zvm`Sck{G-zXEWkeCWmkfC<~!2OtkH1LFeIa(;uSur# zG_N(__7~W5(HI(Vv{L%JLPTlZ<$a3xihu+nt&3`nhL6^DZ=5KK1~zeo>H`dwoOYB|${%6{7cc2~MfdW_z5x88z|q@M3b_Rxl|;M2h^gwv3DY>{PktqZuqx z4Z%M}hEfk}OB0tvniEJkzB8AZeA=%`+*Pc^B;t4J+-FpAm?~ubUb@aw|g`JRZ>^k!&pU1C7dg_z_w?$x4I(0ciTAaiMV7!Jlc_i?bniqN{h@5USIjwk9zOw=WCz#wtF^v|T=1iT? z9yJTk964qImmhMCz@524^^{xR^#j!~t+y4EIr@qK6$;WC2hFFmpWkx*rlF%{gvOqP zy+?v3Mb`Po@hik1u0dq*M5=#%W66-l_FpO<%S+*-$ijK9ks^cAPi)?HmcYM6%3Y-|WRq zh(1yJ!aYu7yN0?wVD9SQb+{~ox~XIg=TwB+m$u<$_h>^%-d-|F5_jn3tMzTQ7l}g? zHHI~j1>LtlN|NKA3)Vzyip+$IFbOq6C_T){0X!%86$|n`s$Q2t(K%_|MR2IFMLl zHI9(FQl(X7&)UhM7jZ-M11gO3+0QY|kGteM_Z}v@Cz}snloAo#eDUez;8lfHJWW57 z8LgKlR{x-Mk?y0A$vkt~m;6=d&L8d=;*$&B8c_Z~i2#5(N3#@{I9-$)ZFx4wXYSa4!6gbJ^ zWDmVW>}8jq_K4U7Z%mzMFJ*I40-UE3^ZN1-921(pc8BvSVI%X%w%*B?`9_Q;@C`2# zZqOFLj%0YNlNEZ`7~GW58RxQH{a@hZf0lTNd6c-QK96%2zyYGvf47;wg?YNrE%1ip z`4xq-nUuO-apB{jSK@==ydi5#6c%SI1MbP49}Co}!%efup3|t8VI62o`1A!ya&whE z#d8-M-xNsJP(Nx;%o$*$k0WEiC4kmBod-=k5i)NhUPhpIRMJ3THP}lF``Cg?;kbIYoqqk9a6}ky@oYDl!d0g;hH>pI?Tn!LQP&cI4O4=-*Rl=K9{0BiR@0TaY$NW>n{}B=5r@_-~z0M4$D=Hp> z*0BO35$wKERQTAB%OdlU=qeH3p;UOVX|%g?9V~yIkt?mau{4`(9F-1o-A4xsT}zxA z)hB>}P`%QJT`4%~Wk&BoUvG1qQ=5&9f?xK$@1olgzR;zf)Vf^=g<4##n-f|FHm-n| zYOWN?D0BeU&(b!MEvPG6FSbB}-tOhx%=@(nYnCo4GIEd2Ecf8zwOvtR(bovUB6l3d z<)Splt3(`eHqdoH)ljjECqgSVCX{gcKA4^jOE+J2^K9)lu!ZzvNSJ>%+x?WU1( zI4$p!mBVs}d?YVx&1oS1;Z^e}3;hO)*c+xNP+S67>lVcDbkqBx5l1E6W9)BQ$2-0v zy!YWk(LpdD3*LwQl;c;>RU}Uw+R3e)g$pwr@z2aI(_5sCqyp;P0h6-W%;ULq(AjM+ z+LhmG8j;ms{W7t?oMON$1wRRj7&>GRfvaEVC@$`0K%JyRufY?o)NVq*s;5J7^wta$ z9fp;DK$w&>Jun-n_Bp@*M*TZXDaF$mkPad(FdTp32dXA|h!OZulTTQXA0>unuy|e7 zynz&Km|l22^@}dPKdvZ*IXw8N2J1-x~%<-jT)HH&g(r|h6Se{iB+hi@8Qd3zB7`Q>(cwjopyim&OcVe#hK5T4p8_N z#0U35_-uL2s237NS^PuGFpzW!EAgN`L+h}iZvi&x`3wBt?uY`?MtA{q0xuCv7kol3 z|80=l$s=@_K`17?x}gC#h}xE4SY_HkabiT@P*|a+!F@~wv*9Aw3 z&i_4PjhHKe=?C^a!jKxA7cAOjd`Nt`C~ZDmqCA$gen9(l-*Rt>lrD@9@tRd%IcWzH zJ;vjY5WY8f+axqOWo~}tLLVj;AB`r950F4P(-=040j;uphi-(gWZIWYU$2V1UimE$ zh2yP%+l~V^DiO z4G8rk8O}3_y?Bp0yv!c$7e>p4Q;^$+-2C(aG@Ybq#5N=}(W4`p^^|(xwLzr@4o@wl z1GgeC#$~V15j0(n$$24Aaw^-%Ejdee_(zfA{L%SwECQ3lO+5won9PxF*AK%8)cgj{T(Lmw>{NO*TMdB zsW|2Q-9f*H%Z3$Hpmo*>KaVKFxM;b{SEU>BQel(QxR?G`gL(QXVO5UjRFAtGo>IJ85y)7gdpI~cQCVEA#GJ~R; z7v}j=D(1dhxs3CyoJ_PMpjylJJMTjrOj08B7W2>HRQ&LR-L8ORf~?r@Lcq7Hcf9qC z-ycr8P8MPPQzE8*554QVA~{jNsy%&uoNC(mb(lwqq(J?J)Dx-)wL(RLK7t4p>KAKV z+q%or2ep)P)mjX6hDg2@b5W?W1f)Q#cgtn@Z5mpQlYHA;2#mP5y*n3jgwOP!5?h z;q5yAn=cPEOTT4MwZ2F;0eWlW(pGO&8zVO0^^B53Gg2s)~dD5Bz*MGwokc z%sJTOg)ZW+s1HSXqG6H{lg&5k`1%NY9qE09sOM51b(ohxQD)Vn0R&IbDde5>O7zAX zXqHS-|6g)HM2z``5HM`?^t^!V0YPb5L5p*Z5hl9p!QW`I7&t$g9Y_SGsRi|9%U-7^ zN`p)|$P`B(zz+CUX(bvnzOdFyAIXbo29n9%ksPU)OHr26>{bCKQ%4@-fd_%B+9~v( z<7KRZxRHeElL+H=jp3dXXoGZ|-j@+k7K#~s`{?okU&%aENHHAl8!QCPhA$Rm(j)ME z(8)0Fuu;?sXLdOA_ui^UIaGcl06arj@yzOIO5QN5*{B&bb606EtF?PM?_(P6Ib<>}*@BB(m+)n6YCI@bxBE~-j-3)= zvD4qTB=VzDsv1gjv9S;aWh%}h^vo(yL=2(T5UbO+o(}6=|9n%zLVmO_SN7Q53)I%T zQ%O?jIo_RHQRv5YBZcBNisg#Z9i6eil#QnM~laHXPgPj?&2>k~t2o77Wb}!U#^+=bl()1*w zSN|4dSKFQZ>7-ubcx(z?H59Q^Mh!#YtQUHyncQ0}C#E9kv%I3dbj_<UvBzT0<`oEb9g|?li_U2)utlD!7>s=PTPw*~L}Ma6e6JyTJ`JYRP9e zn@sn)P=+*mh%?I%k{~^D9U{^Nrd3|or#4?7ZQi6u!M-~|!;*#l855L)s-R^vooYmo z^8|a6KIV{dW&3OTi?$*O`7C|TPzlL%g|D*dd3t`yiLv<_X2doa$WLQz(PmTggpSV9 zcS;un%gf>sN*6cDxRh%P+Q}n5`rpaC{(}28N{!-(uO&*MBA&K{^&kKz%An6r?<)XY z|JZ-w*1FyuYLwd1^N2plmPEvd=ap8`*GL2rt?t*$VY``j0kh3K-(Nw#9`})vMp?3IpIo+%u3C%^jgD+$Nk6Sf*A?=0fsC|d?`usA&g!V z9V!Ka)3l??DQmxG;IH!(e_J~j|ATq5#+`g(CT};KR4GsOIgdCYSYn06Mt8y7Aa*48wmsL7xg?f`tRV;5lWYctgii$+R7RNd?2Bptn z4djoqwR(?M)>;a2KEk$9yn*g^alVcCKv6aQzhz=j9)p5=-B19z#%uN1C-Kjy|3J^3 z^(<1h)+X6oBB(0~$Bvo?7slGTl_+6u@VzHo-E~)dDJLs*)EEr=`zICKiFJP6Bi6ow z`BnG-Eb(n%!d&5@aR(x-zbjasYQNEo8+=^Z#v4i~J3T0peF8dzE**uSXD=y8m7M|Yg(fLT!7GRI9 zKrM>FP%_zLf*VVOP9>;`Nh!J?3qa>ST0<|!X7pHDRQ*;2TtLFJl98Io5hTt2XW03_ zSFeVIeR+Y?Cpjm$m-z3m03ak209Z4-&fwUq{`Z%tl|5h_@^dKmvi|)Ab^g?#v2WkL zv#RNT-};}D0rC8&c}ylw`el%@gQbE}k7u9=i$hF*ch@0ZofEjPZf(E%(=B0V6(FR( zftH|=k&z@7p>n^gV{b}At}8~>oOZAhA)u3J0aDaYw3v{8xr{|xn5$A7z4N_>;=R;2 zOZi1zZ%<;M%+_j&!BV%4-viyiJ0LCqT{S7!9Gdg7=#I zNDiQTZm!NO*Y9`;9GLvpA{qOOmGchEOLKk!KtE{BQ=nPvs6ck+*ZAka{_hicj|g`x zHNk1WS@oj!<@I^0laaW`f>~*;Z-7y`5!DpFxB2o^(}RUPDn$KR;QfdBgm zYVg3O@Cv}RK#68rjr(tA(0}Yr;yq+~Tg7I0NvD3qJqmOv6*#Et#Q?uXazd@%{Nuhm z(u^Y}^awGiCa80J&RY@QFtlrUTlNDrLa%^SL1Ve^ zjWWh3B}T^~cbYn&F=%nyn(P7OF=L=~G1au8Iq1B8@r66@NtsdGdy+=e>O%l&g11l5 zY<1rO70~!DB=gZM!G9HCy%L{zO0`db&dOh<1;={u?5IQKr}o^t9+SX(ZP$TBD}^@` z1(KQK?tT9$yRV%Z$b6nY5zm0M(WX5l2GMAJ^KvQqaiu|{*;(7RBm zjF|gx%iclO+Lr!Vbn)VH5!2%m6{m9Vf=tELbiQgAHRnZ?1>I`ahXk>*mz7lxniob6 zuhB5QZoq+-!78HFj%UE~Q7yX)FmrBRYT8gfW}sWW7Zr9n09}%gKlonwb^<+NI<9Ox zidVI;*N>G^+9x}k@4o0z2n7sAz~51%v()AU4yYD)hv8y9em-Jam6n#qqFpg3ksStF zl!k<7$~6bPh`su%$3cy9bV3c7nHqlx=xB_31bRln`n@2`ZrIGpT44G^oaMi6ZPmhgG$k5f2X~*Z2{|@856o^m&%S>6VffyA|>gO_)BWf>5g7u_PI{eAIJ5YftjyqyQUAD{M@fZV}1$GZA;+z zEhwOv+@62QqeRU2*g&TZ{Cc~WgsgfXFGR{Pem?oW@C8}ehaqNFV%5G5lW$Q0nsGf% z3!V4|0_Xc-PHqxItVgxeY{pabVNS1x#BOh=nu(M|YCC0bDMD(9H_DfgoL}^0MN&Np znT>~Y6$fwy8_`1E1r;U9H}Z`M`Q6khQez_` zlj17T^U;yc&P>!@Zyyk>KXZmJY5cU4JoS^qJMgq&Li35Tqk4W=BXX~+%bL07%&G5r-@6zve}_sK+u649HK6gAUTdP`UalB& zg3Zyp)MCBmf2c13;6yy7(t68wC}gO39};qI=Pn%aFX`6cGQ#ijQS z{r#(8UbVuWLa$RHGT)U^c1UVHh5XJe2uXnE&2w+&vpd?1*|j^lH8Y>SbxQ~c`sQv12lg8A=X;th~2<)x((B530Hyid1> z3bDc2(Q6Yv3;Ay^^fFFUNY@lOzHQt07 z^NjCx!=9{CsNgrqC)@sS);w%q$C2{tJ`Xs)**EM@&)}H#pRd=E)sK1-6uPq0RyVpgGiZKSHs3lqb4A>QUqNb|cBS#Ar*Gnnx}7!Cb39kuM`1@Vh8&PJ_JriYS-t>Kq%weCYRhYsdY* zm!w~k#QLJnbwRr;em1BQoIWLY)~Z_Cl2cYmsLnCISwy-C1a1M678WW=*3codOBMi?>~^S{P$A(CZeVRuUp zZs{2D?r1|w_rywj3zQ}SM~JQe`L6W<8)eAVFs*r%2b-O4arXW;o8NJhYE>%4$Klg5 zy6+QsSM&KTutp6%HOj!zXMd-vm+zN=uS#j?KO==rkgg)L{H1h8)PHkIg(HMRg4@OQ zIYbN>Q=Vfs&4W zzk2;Ia!@4+PWx1eZ;C$9_=3S-MzQY_$a_Mpee__5rc}3+Ym;_I%u%d{0 zMeghW^%KceXP@C2gHv#2qU`lDb9h)uO^mKS%W*p{x);+L>p)v4^oxeNoTxY*zdkY5 z&FPh(uALR(VQgYz0D%qxWCNd+rWCT>qu`Ag=fie(^)V7X4t+WHUlldo4^lNtQm5 z@WRfMuQ*CuJKZ+0|6SNYoCr?9yevBq7Q$w!bQ||reBCW>nu4ku1;RcNLhH{ zZETM6G*Gu$0G7qM0upO4dU96j%C3S~Cqe;*QjXjU7G|{-2d~&Kd4JzlM;OB83GwkZ zK{99?HYvW!Da9;{D|aO)j-fUr)1+Bu{Q+%TN*5>*{yo8y2uHE+{ah-m;U!|HtCN`3 z&aY70Wyf^&-xKB zTo+s{!*8L=Bn0(|424vSv$zvDKI~sR1|Hvat?N5ePFHAuvM~S@T2SsX$}oxp5+%?; z)zNso=rpz*+Ea|*8B78uwsmXY$B$uQS8iFW@9PAq1ZULiOeMDTSfc(XnEk(h5r07k z((lm5x%aL0)cJxQS9-o*p-M02<;G$nV6)t8yvALH5J2IfuWtV{b*i0AWs56&JPKrw zZmRe*{m49G(kEPNM2@fXtsrDdY10+5XIf9E2}Fb+wZCatC)BOH9{`GYVhGS1m+&`Q zXlP6V;GEJ0@Zaj<(#&;81x1Hpv7s0q2!(cQ-NQzRNwg?pjHI%`D4@ArSQC zm(7&yqe5o;(xvOO4g-uA#Pv4P-$Fazi?Z|8__=#ur#Zs6(LTuHOrFd^RKL$H`+?y2 z&E;)+;NxD5D&YY!m*;ocjibE9MK2Dw@^U|a7}Umi5_jPC`@*u1-GQf1Bnm}t77 z1*WGb3>7ffef3|J&H+oP=JZMZxYShub`b*r{LvM_oq+A{y8@hNrkCEm9$B1UnBc7t5L zv71wSfO68m0uU+mzAGg-_DL~E+cpDVF(K&A?wZ>L1MRts-vmU(?xRVLwcg&DXh;u{ z;jOC)v>$F3bL4h-^9>ke#M%k{>YosHbehjK?^B7F7OlI7ouJ`PeMf(DM>AU+2txz$H*itzKqd``>4Kz2$B?E>f=`k2Z4 z<|vh5@oKcjw9*hB^jrxpl>L1lheScdW1V@DBiOh$-!oT>z1WYstmEs=(6u?JT~3BQ49xtrqIQ5IP#>EYR1qS@dQ z5qaMi8D2@HXuJVDdfu64)Y_u-^d@q4sJ}jJU&?@p`g?3-w;1~SV`EehrZu*HZngWN zeOY?QES!Ewci*nyqx0To!W4G)+t$%{rC`c;g@R@1aGLr>l0HuKxtJp&zwoeYEldYo2< z)7JLX%1-gQ76aIpmpxxHh6cb9x2U|E_%x@)*#|zt25BglWE6|Qo4t&hXU$Ns>afLC z4&t_Xieb5N5?=Hx6N%{PrD_qjyV@@N-jO^rk zO`h2;(9pdNPwPqaMT-#RFrm!}1X#Nv@a;I3mQ}`8KI3t3h=q>tI4?*vMpfa{AZ}B)RB6StJj@-6X7BAW6s5$hTk|fU%Gb7I{%iXkKxUGaq6qAMmhBG0Icdgk^JnsO~ zikX@D=;-LW5p9v<4)cFcrNx9oyGsukJ6T(LkP(3g^w9Ocb2Apw0* ziGY=r7j$?Pg96)gqa+J6ARZMF1~i8(=<< zIORT%b@m`auKNbs46ZMJUKHd19J7cMdAbMqXPp2=W@27J_{Ts5zn%}=z#@nc4-XFw z4NN}h>nfqnu+P*yEUi8vhe_oimueh9wzekm|3L2H9-0~&E1o*KBvCbXBa-9)Q zd2A>9njhVr)dsq}qQUx{i2e6l2tB;T=W~z^LjuH0R>uiL)_NY7orkqz1fSGYd?P|=cjs|#jlCG}YwuJ008;K_i!W5uW zhYN_DD;QO>Kr&w{SWQ@3i~9c^^#Aw7uMD?gBS-2Ui93#AD79mv*wUdjO< zRQCP~rs^>4QPsDm&Tl^gG-p)mr;yN*LAFcg=c^8& zIm#lPk@f>72JEd_3G@E$7mNGLFSJ?{_9a&bBWaBoJO$l{u?`zUGwV=k*RF=cnEj zPWzK8ZC{;twF{zw+w-X^8n5u%bT*!do%Lmi*k4-b1B?M?Y9VxP&yC{=CZ(8Rsbw}F zWz&`S;+RfaPZMdUQJp_zDqCU`P+nt{Kx!)bA1ylqH!?81Iu36A>^eMs88> zdTE-4AdOH^2~0L#Zfg&RGl+W zf{aya3td?>Jz6~Omk3M07Efa6qiGK9Z(ms1=ti=7h&mkJ+E(bieU`dlbk6Ye_UG2abhED}JOBy-P($|T8wc79eeBd_JbJ}zOqbd`Io^2qxaX#9 zx(^oDp*mfweo$X5Oxs+T!E%ZXq<^YUpHYQ`D6fe-NJ>fu>vD}LG>A#d z$|7#^db9j}u@T}c5A!bs|3bNl8r%8t;t#zAhyV!yOcD(8b#4ukUO4eUkn6)Ns#Cio zQX<&8Tad@tr;9ceR~_Kt?u7KxbFuiyQ0+~?v zB!kG`yV4Fl*p+-DH*LHpc~thetM%ufd%Up>&T6(>TYpSZ`%C3Z{ilJ2Un7JOHgdH% z1*DUU^y1#5R`FK2X6%U6Vb(z%bPRtjfmrTfBf@qs62;FjU~DGtF73-jr&mY@zN-U| zo%i5S2J7O-dChi4i@)98k$bATexoDQAo=JIkSNVc>kIUD_kt=VcZ&s*s4w!6q-7i& zc8|~rI;}ntXkrxUwpxPY`NjUCY-&VAg!{|nx<5y2gt#;E-~^Uo&fQ!1fIYY!w8!Z- zc@pIaHx_(!K3Sb+zp_o1y)V6XwG$|-FC)-m@17Golj>R^&?xGB0+X9~=$-v6*vJLq zjdxO3U!zRXv$Lys8kD5wZ2T3-M1W_FL^HypWd_xl8;96^AA@742uTY}=XvVN>SOG_{z%d!qMg`|z z_H~knU9tPivyY%OI9hb1n%D#SIcZ{!xM%$i=epq+2@MT)7QpmvaQY#X(cghtjFM_@ ziJ?ETEZk;M9h;sR*5(Q1Wvtp2z%i)={Ei3a7>L+kAb%S>Y^v=_#Ve--)kqn8EiEnJ zfaV`rZv6MSCPf;YSb;wVa&CbW5S6fG^@d}*#Y1|)jx-<7p8`i-r$|XozF&L-U;V7b zck?r@6N7TER1Bq^L9yf2_OU!;Qa>B%WA|mX$F-ytODlpB1r%$n9|JqTyJa*Nn6Ji$ zDSzTC_FJ2dHO7SE`t_1d~S}LM#xQ zkxs^sv{GN6Tnv2^ZE5-Sd6dLeM2lQK<--Mk>pfV;vXldc;kZ}qyX>bxv;tZ{KikLm z05ccRds=&~E~(k{;_Cr`#+Rb~vQlEw<)X zE40cJzb^?OCX^_hqiS^)xn5W7pMzOO(g3XJpZ@}!KkG@{zGQH4_hO6H$SEw~)8@G$fd zp3d)f)&K2MmSZa2$3_iqD`)iYV*g(~!r937_JL`v~QzD~c?DgyE?{`xw zJi9y4sz9*3_5j3$hrtTCzrDz-qe^*o#J$oJpGwUnS1kSKb#lv#W22|Tl^yKRIM3*kX$Leq%|{ZjQ;4l+U<0jU}D#9BP8^!DT**QCRAGw~=-Z=5N+ZP**dj)~?j~Xp4h`v*s|odknB# z7hv){9tQJ?g#QKEit{ByiE3^q7#DzIBqJ;P5TSvZaQ8NPU0WZwyeU9XcoC~Tbgr{x zQ73VJG5YO78e%}4EUIo{`JkM6&;fxCbwI1$*JO_(1@S)4(=O4m=bwa=KJ~~P9}Q5B z<%)f)q@Y=K$$L1&*xT!9aTll?4>FRB(&Nm- z&i)wE3xF9k9RsL4CKi_A7T{{I42)C({h?llWtFaf?%_5-SqxeaK~p zTLheGS8y5vmv=^6!e9b-LsSWz`9L%Gfc0qBPUTPdT1MWWGwzk5=Kc4J0&Zr{Wo(cS zUZINi=Bcg#)6IZ1`cjA|89Hc}=C{3@@b$w@8Cra_w}h{- z;wC~^2F-d4BG?R%@BA-!r?F|DPFL!VjfCYPSjZdam0yvagScEO9E0~W2$yq#y%{wc zoDHz)n%4rTgzy?oYSEYF)KYK}bT9%&F1lZ8O~j;xrvHeUR(h~N+|1}nYT<+`Ds%53 zr=lY2`aL}T)H?gaJ;f7`AHcEsG_O#s#c){%WSe@d1MolydNRyfi^pM{#fv>#=lRok z>|?@kx2dqtX_P^vLePf!WaxZuz{MnqW(Z>qb%~+q|Ov# zEo;zgsYWqqZ&m-C1i~>BD@k}yiAkB(jWeiJW&MbTrUzs7gXzp5kAU~8$>f*eibx{! z7o9>l#$Y%Ejz<*{Sq?bAoc-g>BCf)aM~1%DK~4te(3r_Gs+`sXs8mt&7Q;jifJM`O zvFux|{gklav-Rf;Ks7Wm3w-uc*Vx+%`0e;_l1mG=t&QR#%q@cr!@0 zv3#j)jYAOWX&lAp8%OjBTwqV9W@wmIT@R1u3L~XcrPsZ61jyVCpu&>|29)OqC!CivdldRY8^0&n_EOw^yT7yEWYH>f22_Z-0g%-c1&#gZ69nVE zwICRA2x-GGaqhVXcC7V@CE`qX0D@0|i#r8`47y2MwCo63s3jC*au^rfdZp3noa$&m z<-ljPC)(xV50=g)Z9i*)Z)GTeMax zFIQM-@C#aIgVLQphbEL8G}S21k(>xb&JFQ*V>pbGehalugC*d-(21Xp=ik%%NGBqe z3COss94mW0=AI~-+|_>Smo|q24+BmGy4fFWna^K@BCk|C`}ZQP`x60f^9ix2U-PZ$ zpjdq2UjLqU+i(VJebz-V5{Y$NnzE50tTdN%8^z}z2j=7E z*y;?b#vnst_@+@EVzmgOH((n}Isn%RwR^7-2+_d<(Z>&6LL37&rIw)SNYQdZ@ zX9>D-E+4w<$(z8B2!dlI>ys1W#bbH`C|&r>Lw3dpd$lSBozSzrmqYAkCUFjzy>NR8 ztDqcVrTN&|z!L0LjvO@POm5F0w~iSC*4@YmdS;JLp62-MGVh^^qd5|l^;;IB9>DfM zNAOFCnd_9+!eCGxv|oZ54=ChrmPqJbnx-a6>Bwh5HTu$)NxK4%3!_;qEm1jRJ6OsR zW%0UQ(~!~K`Y!MJ7aa#+nKJXKYxX6$_k~G8FpXnz%qpjl?C9*Eol)T{^`Q#i)9r4; zHuoKkPO@tX=CoeXv~8lpn~#uS6Iep1jzgWi+_0@xw{C|8#`>g{_bTFBC7LbPE8?i;YyC=^cO4)I}kz;&7NiWFLtcV>wB{d}8-4_hf z5Q{HG1;m8(-DO*o`=BIWEoS&zwt)Y$Czy8u;wl*arcSwq=0zjG%%725 zfOSI+8_Cdk(M6xMN4-|RB1M6+eC~tlRw-QUZSByvQc0eaW7kYKHDuD)ZjYcP!@$ef z3)mJxfznZ}oH#o`hvw46NQ~X1lA3^d(@Y+&KRHtClU7Tq)ILiaDs^KE*tX1B=Q0?C z8o+Bl5Df1A?6d2W+ebhj(9BQ5*wJt`=Uu+&y&3w_$x##BuhSGQNep5{y|nds?C%4c z4T-OI3KrQz-S%x|Ej^o|7qkmrdAz?7bo;frF{91+4&J&sPiX3U;CX+i8I{j=)C#%e zMD;kn$_gJ}Gx0>{muEQD#+O^Z=^D|GS`~BAk93ydA3ha#tlx@!u^b*7^0KDYlCB@H z$7*b@EH@oYJlr4Kl8Ln)ISeZ2jOR;~5%;C-FR?M%r-r$bCr&>q-&5i2%sXn1sj~EjYx?tXkK!b**0n4rNWhSTEMdlpO4QDbQvT z&54X4G{ROD<__T0vFBY9c5XN_SdNk@MG`>&*D>Z&G|r3lL#XnV7mXh&w@a{#Ed9of zq~DNhmJ{&;<7pIU`(ijNLx4y53u!VzcJ7Q^Mr?(XRPzI2@9C=Gwbv{%rfvFNGyyv*4Wr9KOL9xW7yqi5M6u9gYD`yWX8zc=FK zk8usmAKljtPY#HDO1|IU;Hzwmdmt9dt-DN}YGE=6_DECM1TMTZ#J_DdMa4M=IVuL< zVU?C=yJ46*xUrpE2X7XDwZ#jyUkSahtR}{Fup8E z^*AjXSuQ+}U>@s55ug;1l|Z1H{Rf!|v4AvpsQK2e!;s1tD|1^WqYE8Lq}`gF zSiE^WidbD*6gCq@NmpXTyID3yNtg9LKO&FzXn*t*jxR>K>(Mu&m~4mnk9j!FYe#>o zIIp@}IKAOw>mjKb>vf4#@f+<2p8AU-wobprPF$<&!nI4vEqyMSmN-62kKScn{R?p- zI8IDK2srf=Q3F^V*~1FzwiqELp|qU?&MpJy$uG8SKhDz=c!U>|(k^N(2AA<73?fTW zMsr-X^=Gy+Ui<~tO0j|z1_@klF2pdIo%h{kM=N)ZqKd)^OhZ&zwHHm&rf8gep>6q! zzYeOd?fVOGPE2k8;KI{ogm)Kczj5V+@KI!m1k+_LD7(~2_P_qtUvGOTKB?d0W9ur( z(C~Q0#SNEBm@3y{w1C~FniGs^YRVXOC4J+kckElePlCPq0R{-H>@==hX z8yGj8KWF7PrHJejPQ@Blm&*E$r|wK8e`$20$zvUHPnYB-hHwlvBL?lQ{2aNUeSR1KiqPkR zWrE&Avw;6uc$-+IhgHDI2$A#i@~spQBB%}&G3x$1bb0Mr0bX^Ne?(70S4PD)KcF&Y zISi<2qBb@C(`Tj-*FfUC_5v}9za{VKLb0-QzLWHKf@UP>aS|v*EEo!6>VjHF@@fdG ziwVIhB zQH%NC!rpWcay0gN5H|cbZ(sB%JdK8zGEsa$ec}_=&el8(f6v1tFZ36wYbQlVlHH%2 zjfYH({R6 z+|m9nUlLg-(1!$2uE+7~;q0hkYLjk6U)HuPvo-O57(}Yxgdl zEB!zCRA)Ir6*u-CtX5YqDpmb-r0CRO&Sv|$F6oB;=|R`FT=}Q3h1~Mv3SNlAK6l#5 zHz&D`#g}@-1#Q0rah`3z zWgP-27`#Sy#d9%{%>8*Gk7NPL%T+-Ay^-q`sjJ%5E8lNcRJrLf;~bLe z=)*RGMl)H*Jl&xULLnNUQOTNh>BBw0ZRoDvvT?Y$V9qQ{1(Ikmqxl4j);8zOIi7MP zfKv@NAl=I46c`k+lg`M^q(JaAM)(grHHkYmdT*kYG(m|anB?e-wo5wqCLd$a7PEmSND&&_Gc7v%%-;|X_9QG3$yuf_Q zs>_zo#7Fn(Gxf1_6>x7TgpF%50S5uxbi3{Ecj?@W>!8ND&R_A$)JkOpMuSH*$=q$F zB$XfC)`VUW9?K?OkokA#9;f1iqym$^w=I5Pa{O|M|G}R$z5wCcsKM5{?GYPmCasrR zSU6F2_A=Beu0=^C9y3~HKd$-r_t0?`;X0WohT=k7Y+U{;zx8Q@tPGL&GaW_LTFE@9 zmRdzfqlHwfwpQOi+>fsV18!aF#s|ll>&F>7U{rhd0^w3dq9W#&Rlpe}Rv-Jwa1S*3 z@gxkDs6G~GF#gIC>dNHfb|L+=gdtg8#xiuc7)3DOk0+#f!vmU#4Z31UjHOKIf&W%j zf2P`K04$t&8WY44?FBlv4Ow}gDOvC)ijwn{?!$%gi*188=+Rs-qV-#V9JJYLgOj&^ zo#H__;=%;Df?GRatK*dgvX4?T_s(C7=_z;Y;1W!k@&}l*oZPC}*4Ep#e(DRVQO11- zhsA1t?l9<%c0F-JR){~GFT>E%X=Aw(j#wcqXIJ``nT=Zdz-+75pLZVF{A(&H z?ih2W@`vqor5{b+v_#e!Ytv|@3N5UeDt{-v`@P>O^iv%)e}OxLyp)xGXz-G@zc;;tiVM^z}GO0iZd;Uca0@8FkJ4=i;pdW68UPFHa54dWN+*)Y8|C@Vd8yrfcl_} z#Q?<|?ai2XT8G7B44~9QyIxRm^((0d5zY8r!!?%UTVftjZLnSM%aWPghSsi$3(+`% zttXStG)>Ct5l=KyF)hqD#@zANq(%5xollGUAl@aq48qY!=q9vX3`gw)2*K+vCR2#U z9NLR~m7naUQq0$aAS0~4o}82{=B2|V#?cNYzeQbT6(rW<(Xc05YpdcEg6*W|$;iHe zVV`7*tbY=(wGNMcs=W+`w;c53O*C|5i;IAjr$r2eUgwPCG_MA-+WU8y0#DW~XGFn@ zkGHRavw#D(j`l9ua2U;X+xwo0k*%M^~{F$0XRFN^lc^WbKCRp%P22@ZHCjE z6>kXpAj8HE6&l8xP-kVldQI6Ht&9DLf4m+G`B4-I6$*uiq4`&LeAdvGF_mai<{=w< znVl}P%;9F(S@G+XfBVL0Mzv8|y6F29&r0d5fI4$W>qWDo4I%~6MkDX~W(u|JeZ{K0 z=j>{4K56$unzUT*)sRydWCaSKf#YP>dL%E%2wr;vLbXS;=}eSbnygtr*xT|Nj}tFl z37I1!qrxEcb#8t>XT#ct$}>6vv+BblDU)*fBW2!z`(ap0Xa{VGVpc*wptGCBq+(F` zM1^OK0F_U1p5%9MV$%e}qWMKcM0`|-Pa??rFdJI2^G?-sBlZCaVIP?V;BYv&YDAwv zu7R)w)l3VB3ZzBCS@Lzt+RY?xnfrpkrp}`z zL+PQa*PK2;?3HXg;;>l%gWb)}&d#|n(KJjF!oto#PGKUas;a7_q_pu;Q~1Vpri*v7 zyni`0#l`*D-_~nw4K{k$B8_sKtQPkCPS+keHWqG;%9Re>;B&S5A*HF*5d!S=DOEjo>-VbvbqE&gLiz zDJTdDwWVU0hlE&emF2|KO5PMRon?S-^kVG-P0!~PNS*k82+zRAf6(0Z#k37mYE$!) ziU*`nhD)OhQSR8!VE^ZI27{UBxg~={(#q@JT3WEEgzSH~*1hz+ctASy#4923#{x3X zroaZ94CU0t@^?T`;jN0kNe}cDUkBy6cm<0DYR(jb#K^YiY+lOXu{9++TPG?a6wmSc z=h?d@r}6e+ufJ&=;@Y}Pf$vCZ8lMo(c7U~zKs2U;h=w8aDiXzzbV&|S>d>o|Y4F4j z4IPe#%75(t6tk%%Cnb5$QFi_F3n(PK&JG&nEyQ%DN>6#Fo3#CUlSh)i-o_BJyfJ&D ztLtuj{K1yud6D4dPF9hgC2B2)rh1y1P_T>Bb4ZcUofw z2^)A}1K_(&0C{%byM?g1Y-9w&xdwq^rdZvS5oDqQcp7vQkmn-JnX)cDoIVw~U`P#I zgTEpVz5rS`^@4Tldv@Qzo;t^lmrXpvV4MTEuaLn7U#N?9LVm1zY&+@Q9UXo<*&ZvJ zte5FGW4aHVHmMm@Dgvv{a~OVpb4Am7ImO1`1Rff^?HHS>8u;Pr{8#hrCz>T>` zpVj9qc0)Sj^neI{nSmDm!HIUebNJtl&+oxetK7f-&g$~z_sTOD*!iARcV{2LYyCGP zQtc|(@>TN@iN@I>)yVDTCs9s+o;(3a1+Z@4ST3!dqr3Bx+9Q1zK;}E^GoGJVUfTm8 q^f>QwHLIkZ?SD(>P>sffBl6Dck35(n<0HwyAL5dkeu=Jq_?)SZO zf8Mj!KD&2!Nmq64>gw*=QOb%^$Or@oU|?X#GScFzU|`Uj} zti;5WWyHkDl${;St!&M}z@($nGGNtHMsWfUnPsGFC{fAx-2{}CLQ2!$bw z1TQ)m&q&{bqb^=UQBKj+2v+w!vXe2mQ<(8P?TK-3_^%wQz^eR*4X=mXr*~$L^R?Co z^EY00u z>K?H9v#NnF@8#cJyf9F}q#5^It3p!{pUA*k2&JedVFRZT?@H%xBod=oWDDrR!M|{& zIF$wqn&wt*Nw+F|(L>wg089OD+r)zjhOMnP4x#wzwzv+fE+mlw?TdZT>DYFCYof3i zNnQ>=a@|jC)AHL3km*~$#dD+*7r#p=cI$`)ofaATevQQ&aQ|^LIfs%szaNW>-hhaN zW=vb7`P2%Xt@Mi!gM31KPzCtY&p?Lej4RA84gsf;H%KWAC>t0&umW`^ejbQAAyZ5v zMc2K=4Q?^CU%^N=37_Duwc(>{X*C5t+m8tIh0wr9k(xJY31y6zs%PinvFy1}0*49i zm-R-Jq%{Vj+_B)2i^$7Gx)?^|6Y%Q103)c;Bu(gYyQZxD??jOJJJUR2$mZz`_>+h? zJBV5{VnkU}N^qL7gQbp3*GsNs7FSf56cM|7*TG^~UPclEMmx$NtyWrL7Ws{%9-bQCNV3}$~+p}376 z>ce2?mGt9n#0309k6^^^v$>Mzmm?|xDm_Y9bbAGCtf+v?So{ zX?1ahTg0Rcry#$=`DZ5xa+FnAV?=mIkVH(0`H=y+DGlyxp%gGDun^gvp2opibZ;^l z$%p+p=0M1&=N!MH&ZnIa=p@-(Sv^l+q!UVo zlEpCtFG;S8aE^A)xR%t`UKU?g^XjM*@#=Cexfa^5|Am?=tjhVFYo6>6)z8$n*tWbC z*-i;RvEms<%eMw~9U(jX%6Ru+ci_EfSe0>gpX&C{akU>BR?4jv*Q#k1)>Q-rd=iby zjbH8p|4{!1aMopq=_C^9+EwW^t{62ETEG5rb(6ZoyVJWX`NRKv>WAYGV550$8Q{I)#stCVGPFbh8Pr|*~*#lp(E6B~0uY}LXt?Jl!*6dICk@-p0 zyxn|fS;WcmQQTrqRkO-PD&T^4LS>_vr&w0%jMPn+^mp@2!W^)q`|Ep=e|J!!XlBre zVO}qM2tX{nu-hno+q%CvTu{=L(tt8(#=6)js^{ ztWM5^&g2(|728N|))cQmFJUkJUKOmdSdv=7UFKftCNkmrAPFY z?)Pn;$=Z?H<*UY&ILzT6E5|?CCwS+*@?BDH7;trQ^T#AJa;jVxX%{7`?5Ygb*478? zeb&6!Cf4TG7TY&3r*9+gwEj3Pq-^0I@lMqb-V|r#Xm&07S9f=wb<}rycncHS@TPM& zxt~&!Vv;sZ;Q*NoN*%H<%r}{YO@x~a;x@$s)B>vBx!%jb7r?c_+aUljd{EJlsL)-g z*9h5|^8~Tb5zrYh%@LNNRN?AjYp9&?6G)Y)XNWGyIoJyXif{>V2KfFVW<5JKf3*B| z*M=wXJ=lT?&`DLWuUKY3sS+&{g-OQ6=wQ2XuL$Vit>Zx&Yq6FapAYH`9Spv)=JN=- z``$g$IK?~F?@kWP?UoH)#0F%bW!4C6D;9k%;;8*B!)c|esz3X(x3zb@R|RkKX*|o3 zT!L95aW+*}RzY?>eMOknbSFD%Jw=ZuO}nVYI03U;&pEoH@WPSQg@X3_^#M$=KGSH*D?R)K4j-Fsj zMK;Tzjhf1+b$t>rHAvNw2}ly6z5M*0>5)ay>NyDi4|^y((_}+dtLed9-*n%m`F1@| z{far^6oBQWH#d-1_khA>?yzZsJx^1h+Q)whM2tv_ScePJ-cwcw+`1cG_Z=9A>^u!7 z_`|=HbcvUU>p7bwJrpU#Tnz>fp2_8AROnters{fkjTcrFo!+QU&fawMz3DuiY}Lh@ zosF%hJLrk(k?T46m2CbZ-v6^7wx6IOQE9C;p{L*We3t#gX7X!`x~-0!HiQ1^SA6Z| zhDqD?Xg6T_%T3IyVXN)#17|$ihXBoHwSdEG?*@R*X1&d9Rs4Eyd(QpR z*>YV*ZL=-u63zMp=%Hb=`XXUz)4gZ6CqLu>c9FD*l-^E+z?|!r^O~?dTQ!>xHGzL^ z;dI^i2=zs{sTr?Xopgq;4n(_ec{S5(5;pv-D^b}w%$^`@owMS8I1q}dsT41~>`%Wl zzdTo1rj!fM<8oRtwVdt(I79(H@N9bao_*<;Sd58ybA7#0ZrAncr+$!AAkHVA$TQ(j z0Nl6xf@Ck#<}|f6ne}427Q6(no+k)X>|NLQHhkR~4j0N-p_^gP$GhcPbiI|%0E2$7 zFP!gnZ;KmD%~h1B!K8Y=c<+~!l#|7{3DgO~9=O7Vff2PBK}9!5K-{CZ2aAWJ@&Hc#z7#kT7Y8L|;NmKHK}k=&7%W(7 zRCIfm9@?Ovs^XS#?atdLTyC(Y7sx~&(m-iX$z`}4UQ>GCvG^_cH!wGM5-Tx^QAc;x z$D%$$#O?mAEkZ~z=?O4jbW{`+TRb9=Dq=_{w1`dqjv_7QGO=rYXDnZvK``n5W)8iR zV$sa_^vA*h-cefH1q=*_`mYOIMwRLk3=BNqN?prUOF^E`#KDfq$kf5ujLFl^@dFwR zOu&=xqiAR5YDDH~XKU}m=P5|>4+y@G@?X`=6lDJZakUYo&{9w)6LWAjBjaS^U}B*V zLLeg}6L2;)=TjAz{3rUylOTnqtE(d)GqZ<>2a^XolY_GbGb=AIFEa}pGaK9I4~Wk$ zUiPj=o}cYqDE}t%FFN98E+)=aj;>Y?_GEw2H8OT^a}}hZ_>0lMe}DJW%+u+gGng42GW@Tbw{x|IpRDr*0`IN0Z&1|*At?bO~T|RgSv2wC;2>b)#|JC$AEdK*l z`+rb5Il2B5^FLbti7CMRmjwSI(ckL&N9~8cgb)On|E+r=gvG{U+mGQOvJzKP|F}c^ zH8vk3hx74A`}h5$3?2-*U4R+^1||X~BQC1$34WRf>x(^jKO8_u)iC`9nF5^(E|jA4 zVMDzq92yN8(U5sdZQuddSBO!mCJ>pBO!b@eSCOC6YF4w5w4D)P;K`r>i}SpkPHt{A zvP85=U$S|LGy|2b_F12fB)i z%19amZg6lgT0yXXX`661{{MpXPjcZR2%@Br;ol>=a8Uj`bXIJSPne6R&rx{+aP&OU zT6~Ry$cF9PSxhMn)Q;;4wcnGLT9UsuGm`o{=0JoOF|}>*V~3Kr&8rrW#-L^1m9x zKeRN`05z*Qt%MbnZ|tW$U82td?GFV7nD27#3QX5W%VbA7{FKo4-OtVBzjtgAA;=Z` z7IG?Iv_xV~#r>UW?-xmQO`tebO6bVT!h=xY+vAlCxK@55GL7@W(2q4QC!V5jPryXM zva$X*3W2*2Yi_YGHv}V1z6_HSM8m}~W$%r_1~)(cuh#spVJ5SKn*BPRp>`nTjh1KO z7c(?`Pr;oA>TqpAE$Oir_d{R7N2kf0%dGDSRuh3Z+@-y|7$XV)e3xUQ+e%QCD|eKu z9_F_g1X2E5%DDKsrlht}O>A;YPug)R68?AJwa+7)YrE+*xJt1l-zEC_3dpA`@&y$lAg21aA9a z;M;6+LnY{rCsYLuXiNkyxrkI5LfNf_haO|yQipcg7RmhAw?_nrBC#vDE6{1U7uEPN z&;F)csw~eYiJu3~3*VoHU#bTg1xbYLs|S1><+f+~v|NCaF$PA#GW|W?lldJj>;zW0j$JV$uGLmZ&$Ic%#xwGqD-RiQrNfk%FFX|D*r_z;IKir zxVQUTTzZPe5a;?V@cPY_;v^`kggBXzxmb50c(TKA&sYx$K)C_q=w}%{|KKZ+9oo}o z`t(3-=;$5cujfPgHAPG+1~4o8P9gU6F`@U(>>3FzJMC)mB2W%X;VwIc)&F0VLgi=% z1g9g z7QH9i(>9%pr}G1G0gFj0;Ef7u!x0n1{-TdEx<%>2mjG(qs}A-@0rNN~-9H?z$S1Nb z79NY>HA|fTaK4F8aF)4i;~q*JQh&Q<$4ayo=in|hG!+93!E0S6U`WJ!uFZ$-AEf~^ ztHB_CHW`lRjl%J`BQOL*@_O1Q$2!Ewwx&3WX_3XNGzb%6grdqe_l|R|qfOUBy7|KV z;T`^E05hKE0LBX|wT`@j7veJ4#a`BK9e{^2LZu{KQ0{InnDqB#U`h*)ea4Q}ov-(A zYZ~c;Qa_#QlGwSvF0Iv#Q)ti? zt01hwT8{S|4 zR(PTi6b44%u2c{^(cagCJei-xmtRDrzq?q4PZ(f}8dOb2J|m^68QoPMsg%-R3*I73 z62(Y05fzR|*$Lk7;?@|?2P83Tl37Pk7b0-pr4TW4=i5Khgg_?ZadlI_DjpOI)rX8|p?%W%y91)41Q&=2_lpY^;(g3TC3~C4|q^H3O zFQ`SJnyJo(r#H`b8GV7}2BF}iEaqWCwg7m5r55bf;bh?-SOdekLdvdo)40u9!Y>Go z7u;!hQUl66%t=vRa9`?L<5>K;q7z^zue*tOGKYWRBtKt)PV>oQPTjO|C?;i;;%qx6 z%hP;s3>{+EV&>1KbZ$O9h{u{0`l{sZ$}tjN$dM(C^;S)|{SwT$k;`hyUd&wpl|8sS zuRB*xQgw!g);rFm3PCX*HmE1gipl;mgxkc$%^C0F7%$xyP@Ve5sXtI^CAbIOoMOW} zW>b4XQ4bj~!fu1Ki7U`|S07|FXrcWF-=MfFRnV(6qI;5pV$Xd-XOa{nYoAYUB>~ij;(3tB zU7wcAWSTg(&liP{+|QchY~Z#5d;QOwCyMP{WLH-o@(J`Ky;3?#Ei+aiM^dvKo7PKS z-NM^ah;%v9PM15Y7h6w$XP4c$KVDHD={JA+h072e-0I?;r)__AdhZazdDRen9bATB#D;%GA(P32L0ZO%*LYymMl$QaF*;2r)^Y=V zjx)j}TIpsM3HR5Bw^MdV}YR1WDdrPK?YYVMetT6^_XPxnr z#3adSk})}eXQ25Pc8norEuw}4*(75T?Jnv!xKS97U`>qX&A7moOcoC^|#45`42khgs~a$Jdf@&Nliu;bB+SK=0xnP5x2d|a^w zyCGhbQh(MTDoMeX04m`Py=G{qN$`bEl)kpTFLb{ZRP~(-DFL_7;T`E*g9tme5SujT zuIU%Sq;Gl&oL8Wli8QaxVydy$(e@>Z>pE0HPb(zx!H#D>+Oyr-)F4Q&1AOH)^T zkfgrt_FMm3wN_P`I{aFqFz`g{gc}n27+vwK&eoDe)^SZ`(=|Ex8ADQ3#z?P_-Tt_~ zY?|pAXHJ9Tfj#N$;FL*UedSXL=48S0XKqSlt@?k)Z3G&N$xoWv7D#zx-`vWJBu6nudT)rM_@nSo z-uf0--^^b)n_ZgA`3wZ!s7hVC37?$X@Sk{nfV?p*l;)bU8~#n5voTfK1=;x`=ddfu+Uc8p%7x;ZjwkdR$$-77EX*^ZMA8&6}HBoQ~Rt8B6rT1h9^eN>`g+v=wu#Hyx_#insIoS z-|+7&P1Uy&Ozvveo8%@vW9&uL*h>D%I=29V0!BRj$*&d$3>Q zbtznoX*Am)@_dhP8u^s|1=!#?=I;9LC9_i5B=ll%4BTDi8mb&bDC3E{UBc`3Mh58I zmu=z}x@i={bipWpWZQ(nSQ5IFCMDR z60k_Z(N>68Y9H5ogP5`V@ZvDsg-J7GyTb_b^pQ*5=A6f8hmPHyMPz6nt&0Yd03rtaXK@2{oM}n0~FD0WvuwPUH9A5=kw;-p$xM|d7u=1#&AkQ z;yd2Wlvk5{>z_28uhW8jzfSq>_=O2nzonAO<|}}zuCwb)Z@w~Y z8985OXv{7>kBmP>%Bd+q1xSs28G8ih?KDA5Wzqs0rmf<}=f-wAcTn&e(!9ja@X#$M zH8)6#dHb&NS)EoLtXbg+U+UZW;cyN*{}; zS?NH&p*b_Tov7!=r3wZArr4^>8`WYtH+r0SZe4EHB>Gbo{gZ%wZ)?7tjxS}+&DjeQ zhq??!n^Q4@XAOpYL<*gi&_&V8x$b54^DWN`z?kJ1wJ1l?h^c044e`TY*r!aXW677j zJDZp12CR{A(gOqSdqlp?Zk(-XGDsBa#?b?Gr&em<6PR_(HY!OMs>3y!Q?DvAc6}SKQ9IsVhOkzK zVtyCRl*@Ygb{ioAboyS2Wn6#Ay0X2bpO93b?Q87{`oy2wB;*ILl_?qbbv9GxD$XXG zdeyW?^!~~V*sUTxG*Fgf=LZA{zn6c7KeGbk?Q`ZxQXF-fed$RW&|*Dc=1ZAF2Wb&C z0^qxMJpn{HfVAcEPe+H}u+OTVQ41Wk{Gq08SaQ}_tO<}xLu{cEb5L*Mp2TWwmATao zK-fRAk^Arw?kud1d54pn_l=GO{n5EnE8_#d@+US}5`L$+mb@8s+TnpvvX?L>A(?Y` zseRPqX%bjREJ4H_h>g=#30waZy+0a9x>zCgQmb~VVfmF01L$ELQi+r?cX?C$*j;)z zoxpmX;Skd+s`I>}@gwrI0U}z{9-w0*J}=};fg+`t-&bFMVASWA*9q+x&gfO9t-q#Q z_NgCk-xM6V2>Y7xt)R#G0k`;yJC4BM+Lw4Nt@mf%d%^M|raZt}3aj_g=R)K@(Xiun zOchCc2Lbuk_e^&NC>gbTWijnM?}3l8K+`4ykl&N@Jgn8aE%^GAM{vG>`SZcYIh#t% zPE&_KPSP=Yv-CBUAj!+`D(k8fB+}4QEdPgGCqXo$+adjhSH;kp!@xov>>Y_6aIm8>iThJ~d1mu^5qK^GI%5dcrhVF=`4{lmLh!BM zUL1hPGJoq;IXnvBUQUGCEmI|?3SPq3HfW>0hw1SrtrokCa^n(k1lZ$7Qc1`(ydMLJ zSYrZLC3gOYB}tK92nW~Hs2-fmC2dHgVG!GRBE61Jy_A_^sIQw^y{9Iv4(v}f0$Qb1 z6#dRdmluSN=aG0hkBrxjJOei)e#s|EtaMbPUOUECqKEFz!h_aKi`H>^Rh4Y}2Iw#8 zYrj!=y$sqX^bGU;S~u@J-+hwYD5{I!maKRNJiz0LjF+k8=&GIl*h<6onrqToB%R;Q zI^im2i@$hVVi{5!3L;sqb8o!0dL>Zh?XZ@dN=uNFN;~U$W8py>;w9{|LbZ)$U1HyJ z2oT)kYqhp8k(~u}TWP?$QOIQyyAufomtvZ?Rt=KunC6}klM(e_@6nM#1qP7k8lC>y8;gJ) z%NUCD>Z>^b3No4eOhsHU;ORX{tE11yH6}{8X_VoqjVYunS8Z}miTZ5PXRtxo2kdN2 zTvd5HearIH5n`knTA#_Trqlf1A$pTxY&&kBV&XM~ySbh(IAHAbauii{DBm|Q%6@Q* z#!D*v1Gtx!@sO`@-O+};VBlsM{tJCjIU^p-iTBC(#j9*}-Syu~4LJyr3iS3GHjkY&^SX~DR1XODoSLz>n2>?@P z9r+7M+y#;|UtE0^S?c7#yWY+#JQ43_+dE&sE%Q)J4$S74EMQ3HopbY<HHe zKB^1AEVN=jc?nC<@Hs!n&fLe(8eZZ|Q9mDyHPTZ?&Pi^*Mv8U3Pu{wwIUlK2QgmFO zZx}K*vZCKoqlUTcvt+0R+f|bzTkimOX)b#zpe6ZJUF-#Y@ovs-Bj6-M5>I@C&1mA2 z6s@n?HcBDYS*>FYUTaB0v?SRmKA+C6>?y278`zbkAIKMNO8X@IFc_p=>#^A0=e=>_ z5^f~0vat|pyAJL`XG?a}tn^W4z5ax7Ap>W&y9}tpn;b9^x?RJ^n#@6l?IsLLHn<;* z+wPSv2%~?r#ptI4tM0l-_peyS2Jfzrnk7L%oR!9jyOCyucsLgq*`Sm*#KltcCZbr- zH|JPZ6HC^>s4$?@O|5>D`Sg0$bDoD?xcf!la)9UE=Kfr&1aaW;A(tc1qN1QX4I=YG z&I75^nUgNruil*!hUujr>y|f9XD8g(E6yBF^p zAwBrk_8Kryvz?+k)^DL;Q;Z&_a;0ZKiBQRB5Z|46qyL-U=p@O+^QO!ov+NUX{uFf+ z=|Me<>-iV?QZrxo4EGjthsIaNlVN3OtJ|hy>ELKwC znj%30+~Ef;GI}=OWc4jV?W?sZH{&fj$=$)S+xX9NF&$bUvx;bCp?+A;bq_VFsqs;| zBW^tQv@~G*Dp@-QQ_BSkmhocSGpN8)y#ftj1TAF#bkS1gjlJ7CeLZ$@<@DTM)U9x*aY(Y@SDT>w)ZS)K0r0S3Y-$L1rtf_>mQ21ZIqXKvcJ6AUM<{!C!l>ej zm4EgrV0XVF_KXbgCGgD<;BZ61VVh5pL4v%KC=dUx_Ve4eXXM2+vd?`X;8>dS$io;f zEgIygGiDD|D3Z<5ZzACdmC!YDbcoYMmo-TdNCASH5H4P^L-IK~1b-#`IWMs&S(LJcodChYEQKg`2AMj@_93sG7K*)j|$9^-c5Oice?o9 z7cl*;>52edto-uX?FB`EHNhNv;jIRC=YCzscmK9mSx4{*A3Bm>Sr2CXU`@UzhP6tAM zWbd_|dRw?58Mw)52k%@SY@OroltypGmDuB69(po8`&9qD>FV0q44UwaJM;;+J5I5f zEE00VmAQWH+?s3}>388S=wC;#x=}dx8dr|amb*B%tSUsAqM=Kj6dP-I|5k4!Kp@?< zv;bEu_3Y4jG3?J2Q(!3Q->*S=6mk8Vvt5m0?#*@z&{0H*UDiYfINjmNHOMveaT`S8 zBW^Sdg|a{>)o8oyT16jf0>oUg*G8P}X8c?WhW)|p-)T?DMVinb@SWSQ+ln7hBr0jV z8nE7$fvaCWI0LJCZjpjzv5pV2DARq(Vni*Vj<=>85!}60F z0#8Y1fF9~fm~hl^Tg07fXzDf2W%-DcmUWca=_;&*80o>m-G}aKA!2BUGi}?+RAiOB z{?Md{xtn0`%o0AOhW8E1&nhs%;04!Q)o^np_+d`WV-AuJ3o<|_aMU(4qd@BfsZ}jI z_-LNwA=AayoMct)G9$=YxV0GBTU8xf)MaP|8BbC!YfqxW2-S}py7-Z0qCjM+%~w06 zOc3k-SJwmGVM@(@)&}kPO-K6?w*WikQxFRwphWn-@RsIiGeC|G>8IwpNp1O8GSsfF zO6mntzGzr;Q1JeP+kj}a+uc(g&eSuhCD?b7!Y6VK7({KlOsWN$t|h@JlE&5}m9AW7 zdh|?cXB2k=aw2?&<@=nwZev*nYn11FHX>A! z7_Rm}23KUrI=_!)m}H=Srma;W_>@cIeY2~;6Q7SUog=A(3<|*W zH(vVV=Gp9IC;w^=ShZYp4d1j1f~e@&=edT6H)YO*(~#VpZhOAi z512(*UC`p&&wa5HPaz>@yk8nP>|f+Dt!A#5-|rU@xV@>Z11+Xz4Oxptu zG~IpE_6#p_bU-Li-s5acn{t4&uxCu1ikxSyz$%{(5$SjlezoKD7xadp)SY&epj;=O z3lN>E<2Dj`B`{}*IZAq8?QGsAO1K(}515XkxU~<5S{-Wf{py7(AplDCji}EU zVIu&2(F&^IcU=;oq+U96Tudsq6mmL{7|!6vbxeEj=s6GUTqeF`_x7=+nR+k26aX6g zPCa48dc6X8CVthlTK{femFo!K!b=crYq)kE2@|o5W zxb0&I>9dx4;KRW^HhmFr(Fy8IOZR|L<8c6_a4-S^wZkl({~ zUQM1r2~o`B>wb1OOX>zomS9*N&6b|~Fm{MZh|?U)ZoEcm?M!@lOSf0RKK^^Fn#~Og zrVxYnVk$N1m2~pq4IujYV*W=qeK(1($6McS#3f`Gwof`y>9%5gYvj7aL;XgK31o?D z7YiGrxwgRIc@FDipkNNi--maB7$YlCzM<~bz;n`kGb_;N9)TC0<0t*2HIc)Q90MLy z@TdDT9~-Zsg3Z`{5KcN^8^~Q9A?cd_oEYyHHP7;G;ug949ro?Mj5nEwUGW>-Svjhp zoN{B8yGtmpC#DzDdx~ic6Kpy*w1FZG(<9+WG7Y zO}50NuQA@1QkLxf$|W1%C8#>7!$Fb}I;EucHu2H(L7(j6dtF>Reu`F_bLOql!@?Gb z%`3EIC`Ocdk?Jbt+O5#f9m~_!-V9ei&mGwjGki7lVMb9Hnvym$Y*iQCY_{zfWSh>i zdqDlp-kAbVoZEEYF;?^8nyP&glhktsA1Bg#3~ZV2iMXMCTw00D*JG-G?L~j=bCPyT zZeP;^i$8HyuTDtAEaRUGB|^A!Mdw^e3&D&15r`Mi`y^6vk2|` zhQ2w1G%9d*AYI6}riG|xV&-B2pY@4@={7Vw&_xS}lUlr~lfOA6DxEQfY}Kl6a|(knPM*+~Z*RmN3g|jz6IdHWZ{LOblch+-;Sppv=X_q7 zyywq-o485uLFO*#QiK7Zv|1L$gC3;ocCLWxmCh-W^Ph@mh+_|hsq*S0K-d&Kwe_yt z@Wt}z1+2jQA~PF;N(~ds@$bD=(teLVB!fjI4=n?z=!*l<88N$5AZf@zt&Z54@Wr;E zojpfyrhxTQ)_Yl^x^1MdJ&6RQ9@Y^#k6*AZUdL&V0@jO2eyWIH@Ly>9j?~DpFAJWH zTDfS^<@b4pnq^NnXK1&~U=NKY$bWWN)BM;U(H&|BHH z-!n8@HXJvN*GKzRH$UY1ZYxfiAG=7Fk2shnTI2_jv zG^#Q%en~6C?H2ISn9lGsV?;!M7?j&;`XQWUC%o@TkG!OIw5{E3Sonk{|AmGCYr~XV z^~QfkgRSumpPgrRhZ#E|QjmjVIm(M^JNvURB>S*umTsY}( zRR#C_d-k^|L78yy%)#r7jEl8lZN;}?RClLXE4kfyBZ6^>Q=pvKnmYln>7~Ol_|@@_ z83$RVWE6$O<=3(%JO$KzC7!_AZ}F!n&kpHE0W#i?9e*%wW;We5t(WttW3~WN!;2ep}x9hKn-)&OJ!}mC4n<;N~Oj*NY6~8D-Nu)pbSs<7-2mP&M7DN7uwaZ^SiR z=o`xy@y+m$%T8cZ!e+Z7jUakJiQhift=Mrz({)nSfzNVQn5H=>cIJILkc?cJx}!`% zB4*bmesd|($b6t5fLNr zTK@0Xb-QAB?lchriF=^sQqA!oL7Kd!!P94gEitSZ++TP_=jy>pGUoLbB8>go+=Zov zoVC{Th_Y(BdIR_G_8XWov!&WHu-fp5pJn=LHx%=vI6_7_hQ#Ze+t??bY}iTkMRtRT z$L2cOI-y?C=gYFMvhySf4xbb}&96Y;C^aO%-M~M)=XLu=?Vjq-9lq$gNo45bi*8m{ z9wto1k6?bq@8Usq-lvZwinSn0#1dC|r>bKGOhy^&)|5PH4WzI2Wp3VAPK0{C$gz@d^`M6@Z$m0>!*XL86PbDga2Ns$QihXmk1Yvc2Ka$AX@nQjV)(G{ z%jU@A$!b_pRPquO*fAh1!9h6y2?{@ty>;*#Q#fs_(~(quajC@gdvNX5iK_gn?-p56 zY(%GphrqzcNu@GeRGFsx9jerJ+NV25R9KF+Nftq5DgTg4=8w;~_8d&K2=Oy-CxE!w z5!Ff|D@!6%bJxa!BL%KPqTOnyP;aXhU02fqalyOQv5RRDPVH2u(T>bAloEjGxW-dH20EYf<%U?vnhZ%u?*Ma)!TD-?H+75esYS8ez!ZpY=q_}Bu*zU- zvBwmUI5qMI6$eY&{66HSyG~O%vP+&;P^wK{mnUC7P;2}8I?^lLO}VS9-OAIhvTSdJ zu_^D#-+9R_Ih~OYw;)1^JkNn)#47B3Wk`3CDZziJT#iE@VtA18#ug7dWZyaV0JY$Wh`vF`3zCt% zIJ~tH0H-ZY=Jrq&mKi8W`)@&!qOm56+1M+c!sC6@pWGBYYbkt%Kkt1%CLAqQiu+Ys zCZR$gXKD*CGixHffopydE+l>`0BHO8PVbFzSqN>3WLp{ER(v?Fq z4JT+%6DQ$Yy*eLMm@sfNlvP@Xc-~XO6w>1I^fl*@(|5E^H7R`F50uX{lCZi~VIX@j zww#IPE z#@p?(akI*8a{lUzd-B(|2$OtZ{*D>u)J{Ls89F`yjAp6pR5r<(xck+>5N~6{c23Fg z=Kr4C}%36qaYX{Pjn<|x5Rd-F(l^}C91Z<7<`(6oa}9ZzE~l(Z?N(-4A1&SI%k zY$*DuB{Cg)E5xp!Fe^pzWR^DCI8X*y+k(G&oNQPR?SP_*U`L%m)H90s zO*lkywdg)#YkfYyQv_+Kh)!smkoPz4e`pWl*$P?kP)3E4o0xH&e>MPC7R(HE5dw_GTwdBNy&(lu!2f|eV+4PeAuDM^(ybC)*nNMbMQ6d;oG6gr@BE)D&5pZ<}p#YhT%Z%QB0%OzsYJA*h1 zt0+;B#0c7jylLDVq+`J{Io(fE_Jy`)&KN5IOC}~bLkJ<$_JCW!o$_Ttr1M)81^Hw#~H%}s4TZC*O58~P{z9crd%%R(f;r?bRdcSTxjK(vFD^)y`!kT+k zVgmkZ)h(Z8UNc7r>c1sKvT1Pi|ie5xk{H!y3mubZA(KHWL(t_Orpu zoRC%O2#uKb0nNp+9^_KvTU+9^7d;_7*Ui%LSu|uaKh*|z%^8x9QFltubm4{>JQXA= z6YNA?_A!sKBz3Z#MvTh|f9`wC^U>qKywJMe`YwK@9lF1Q6v9?vMJ#U)SH;yVZUj2O zyPu~Wkv0Zj!X=EUJ2j@+PT%1%s5JB!(C)_hHRnhcvMik~hZ?o^5pm)UDH53KeHHK# zy2?tmPcmRJ*yS3`v^xWi4|$09|FN7@X1*sTe`>hNu^LEaVf#o7v5qAYq|R^WkK4jS znUKx6#kf7Yu|!tcym^Lg88khN2+xS<(w)u#_lLP8X3%+2yQTNaeQ0DBe6;Wy8cFnq z0kyQDJFs+Q!aWOL`i}-~F!^mXZh*EbY^=>~(_2RlJ+8Cm@_kZC%>}IjGsgiC?^4BC zG_0U>Jfs*GqLd_#Z-V0qo4{t4?y*HrF0WLhe3G5|C-7k>&se}4qE(0I26JGSr7haYS0^a8^tlCMNl=i>}`3W&-%8@vnrg zQ7GGxX_6k0`RY2yCbKd9%c`Hia2>n5HCK)BUq_HV5@?p1WZWCaO9Z|14C!q2;uk<1 zpN?)H$&A8~<;ur+au1jmv2*nTF7_zaE$&(T-}eI-MUL5_v=fJwl0sk#^ZjRjl;Rj8zd!l%C}M zD5F=nj>^lyt`M$#Rx4*`4ex5f@U-*H_nL^5I>DW!pA<6S6 z;)sQ>gKf5F(xle%Al}ii`p{ZIF!=4ZHM1*1P>M)~!a7n^Ts3DWY~s<@xWGHo_>F0% zgq^IUyMWmn8qr!>kHVf$9rNir&8%=Ns=!ue#7&IFS#odSSZ|oM`MRD5TlS)h9jg># z$_X~!QSOQeak+6v1*J|jX(p1YsV{>@H~ne}{15}3$C?qteU|UjiCLbKGfMaiiQrI% zv+jvD!_}feiG{D7w+BfEI9^0RJX@^H8FP47b%vh~)T`*NdUu&KG|?8M-t)u*`WP#u zd{L4SM_8B-U9>b${?GpI9+Hj6IHd=+jn&mh!?c5tOFA)6Ux^?yk}xBD5zU?9!_u^> zjmYXNhDsm&v{SLxBqMb`-}mAe{;)emB#X;~GdnK^h-t5DB2W%wzIp-L@-VUg;K2i7 z^r#!HssD)9sgxJx;I0--i{0C@{!6QG%>b8=LrPyon>#zRPc|?}6}PSIfVI zSu-gkLehVe9zlDvkLi3K@@l6G5Tb1HlS4eV@ipp_p3;ENOIo>i5|U)dfz?%cZg($> zq+`~^hbv*K+$Lqxi*9~jGxc;guFh!YYcQ71Ehd`2Ds#P=ZVrA^*O`+z)oHzPSR1@1 z->6Yi2|Zn{Y0?}aws@0pO~*Bt3|n?O81xSu>G|o_{g_Y6Y0nxzRUc~DnxcQyWmMvk zdVyiF*f&g9=Ut_Xq~<#uR*S^w(?K|St-p}TdZ&2(|4{aoVNq^jyHb+U-QC^YDXE}< zbT>nHgLKz`z|f%}NGRRi9n#&+&~=8bdw<`#uJh;o5@+66&;9g@du|g>)$8$>rF$-S z`^V@ogEn|SZPGmUle}JkxoT4=mw-U_o%`(SCi}?k;_JUccuL!sD;EqzI^4D*#p*>P^X}D6YEfP zpJ{hcy#8@JGKZ}P|EvEdJ*U|y>vI0SPwcZ>;SdUJjkQ=Q^Me zo?h1j6$)@VczFYBu5;CnYPTA+S^Y#Q^4`9bcvvEHLm5Seph9B5aY6Puz!`;w>LkAs zg~6FhWO_xr9#?gO-)zR8@3YB72xUl5|8KO#{tifcjpL1c=Vy92S{kzCFAss;GIf!de%|&6) z(K(~*sv7&;Aw9&Vc-#v8{$)#H_x@KusGx2+4%MQix#JP>4zg#R?NpGerJdpyw}_NC z0hK2u5|2w{!hiACptGQm$--pPlUCS{^n)4EJj&tQ9c==(A6z66dFd4m*>TfVUEl-YmR31)2O~8V7s(EX-=#A-?ej~wrieC3aboMi@0rG;(Q0*tRYh8u&`7U zz2TZe&{dDt`NWj=4alUfVV_10wE(pKQ%qs0oK;FRV*G%9VVNMMJRoN@m6?t6`kkAV z*mQv2n1F|oH0fru0E&WI7iMFEjo{9cilsg)k{o_%3#Q0zx8g`Tsf7fItb|fi zb2pbC#fDNLFlQyx@yAi6MTK_UM=~FCeUBxE^EjVfQPT|`<{j}69%_<_Z+Nl zIE4w3l7$S*++~~qX-MOrMy)8m3F-M@dUdy+@FaQ zFoBQ3IR3`a219cA!N0F*=D(tt6sW&iQV7gv*g#R;a@!)X-oZDpRDaU#bwHrw@3Yia zC9b&Sc4+FtyQX&ANK~U-`ThVCX0=+^_{j_#bueo`>e>1fgF}z`3nr!Xd2*8R_pTm1 zHhaXt$aWqsz5;Y`x!19B)EDS|L(#12<5*UR!ih zVQ;@*c*>9A&*YGi%5H5?Cgr(7+gR)Gd6QW%HN5~ z70c+zK|$CmM23{<}J-}A# zJ-36Y42|b`$Uz9sonN zj|4PWa%OJ4BfwK#9>-0!o^fQV-WWstF40>@2vAhs?RNFPNMvv&+8*2jbl}v{Ry8%L zXZpatn9SAR0T0f@?{yF+<2Z~C;awPR|Ke|ymh&ZnhWzAMKj5t=Df)rdA2Ig|89t0nHQ26sA|X8YTAD9%I-LhD>9+1ldAOP>SnbGI z6(*xAoHt$1^p>&f5ryZb^w0%=MIJF_k@iL4@M6A__TUs+O z*X}k}&s==p4QG%Djd<_#RE+*J*%{0Sui3JqcYd^;@xu=pWG7$#Jq2gV1O2|zEAc)o z{bneNo>HL*7ildq+3ccVz&#&p(D2Lt~{6 zt`9rWupb*a{aR`adMY0mvqdZ)3&En>4p$v1!r`G8Tnpbx=3djk9oOraYpOhSuV79# z#ovSM|5Eb8YZ`D|dCLxafuf(ii_AGPLvO8QoDALmT{z|A{ZIxzR?8N?<-~y`F@{-_OvDZwVJQC`X))BAw_wh8 z?#y_^wR)z@QC;}RP}DDqFDj(_`cB`)9Bnxj+?{ju5WhDo5n~jO@(pgrAzWj=Uw{DN z`ioiK(tez`Am;Z2uCHRQH)J{#) z+~IR;>wktwRTYUp^>lE?z>BGR|9w>SZM$$WBE|(cx<2h;H4CzfPW|6qbka+N(ZT-V zj;CU0hivyp9#0(d?zw&_{#=>M+q}AQ*e&~H(VW4nPs=+L8bz<@_V{CvDvQ@K+S~EA zpacfzG;4jiNV(`iZyUn&Zd>cCgE9DJ7GwRMxl97I=I{NkDC^p7FK#2n#+(yv9OSCW zVV;sr>hLjUW!S3}?AwjJ0#tIush8u}*tZ=3k#i;lvfE~IhxWghwb-rFQ|whK1D+Jl z6q-&B)bVRq9IKYdqBoe?qQyUw+TpLvQ8fH1=S;Q|nGutP%@*cm-^q?q{; z{q~AxQ@9nyyb7d$d=8DBpj2*zYcNJ*=@U=QHf)F8W^W#jh8Q;yWPG3z+7)hGA73?@ zOgdmP&!^yePwUe!l(d^iarg^u?jz;=yut_lbIDS`?eTdqX#I^S~yQ z$TO?s$08L$wK(HPBR1$9t#vxinm#7%2mcN7mit`{)`P=JqUwdkYmzUyY9oiBi#|uw z1$=#r;{NJ)L2Gn9Rnd4`7qnJ*Ks*yTTD|1?x{cv+q5vZ>DjFlC`!UGdZmjjiXw8-9^dysmkckGSxoIF9;tkz+SZZ=jqE zzMO*;YejDz?!M&tQ8yRZCH_!?xyhx3*%ekkS?#?ctsD(+G7&XjJh|pv>pSCp=6sAb zwwdl&^AjU1yVQS8U@vnLe6qmDMf#>H+hK9oV+W(&iC`IXp@o1+e^7Kjd0~6I*;H)X z)-%OI;&mYJb>#A6w>Itlg^8@_LkL}5Y%%?Il-yP41HT?MS+SEOQ{Q0IBOvvxR$~)v zil>Uj<eJX zeV)76coU1J7BX6>oS{YAJqFC^>6sGb3U zu&{62P`{#SqlZiPT~s0lN_r;%wFvzT(=xPy1&vs)|1h78DWr-oovqR~eY&y1 zQ_S!Z<_V6(F0)Gn9O2DPP79Ei=~)!3s?~ZV+Hoz6q^NOBm?tgQcF*zN2EmM_=7Nlq zLXez({XBBo;TDNw_%dpG1$Nywf)iOixk4OWJibH`pF4 zK~5O7IsCiyH>pmM)AP!jD1^+ew_Inq5~YEII-Vp`GZk3yn)cJ2fy;JtR%~9&9+m8p zW&2g5Iy`pg=fwq27r%iLY~>?c=)te}3GWr-l(WNrPn1$%6+n)#%Um16W%&E!zx~h* z4@cFZ91e-I`1J5+P;!-PMYWDU5CGQ3r1=SHkpj`ncL&cHm7V_6b zooL_@p0E?yK9G4$Pu{BAGclK==e=#ULTvqB`ZDcd?4Uc=ms3P)M=qgTcr0n=^j!Q3G5gY~Ru<$}S!tD+&L(SLSCZG0`Fj4No9t=cEk=B$DDM`VCX#L<62!uiXghc2ePIHe>I=tph2UAj!IZ}n!k6l`!#$&yJiCPWflNg!cN$x@Yb|a zU#m-o75!RvC#&}_^lZ*Md%K%6<-)S^=;KdB!hZf5t6w6Pg5oo$6z02Uf}Y!oHY*81 zC!r*8k%wW=19!F4u{X{3q`?>;k1b{-u5~!kz>LfYmIX!!{`tg*9pdIQEX*LKrlKQ^ z&dQ1#`I&CYef64AAlY0OG_NlPoY=>EM{01qw)Z$^~JJvK+pS?$lC`Jcj#y96{P2u@5;Ew z`M|vYA?FAzm0K-DR6onscK&T+AnnVWIqYxn&LCA0^q{m9mE>&8tMv5M`QM(r3O|sE ze76w+hh-kS+x}aiZZRhR+n)$`3d9JBCIn7ig4_uAvqO+2d8+s+L!QJg!t7Fh{1Aj2 zBT@K(RgNI-KwvK36T!!x?qk!H>(+o^Vl9o-Y-RDWaxa%f0N}McH>Dyi-et8`{rKQJ zXgnEl=6$y5GTR1yg+5wDJg!U_<+mk-%5TX{5PnEj^>YmpaGij>V0~`wo^?2~)kEj3 z1~}CH{I(svsk0dWH=&L$bYRajNvJ`ATti5k;-=&%hH%7=bn0I|UD0t=lLq&suak^lWda zDJJouX7$XqwOQoPMY_7wSHyS{zQV6fG-XuEsfRcTB@j)@S$CR zmul@$ni`HwDzR%A@9c^8srVzncz2Lvvy~Z`c1Hy((o6LL5d{q&6(BN+RZ#DkgMEDRtJ`L;56wpf(C3MJ)wK zxZdS|*-c(Te4l{!x@)Mf3E{SC4C(Hv%O@5-So)(N<$ktMX;6#9)W3&~UkHd?$PHm1 z4P#?0x_lzb-1+{U`6%ATe}TSzYbcYdMRrY1I1p z8J>G-@6hQ#3#!A0{-wXxVm|Y@fN)b4-6~?-G9)DJdp2~9aK~z7V6tm4ATs9rU-W~b ziBAl`0DpIHQHseqA@Y-}fhxZUzY~s?V~Tej+3~!_&_1eo|s69itqvY)a)H?|ML?4p8m-?TRDJX$Vw7V7r zl3o;la%TKz1pLcoR}oSQ1HbdIiX1l5+p3|Sp<3gXbf>Jt%iru=(_;T-*MB4i%0V2_ z2>}CaS4Se0Vn$kN5aQkI9~WN{^t81EwWRY$f^r;a#$5v@_qYENZS9GZzk#e?3$64F z@~IO!^@^BDy@z{rsiW~i_!k<;YvLg|cBglQ&R42qg@)~|CX=#-obOHD$IYlUPuQPbgu&{!Qv;7ELpPz(GMXA(HO{u>@ zDB0ateDOH{Bb5@D#bgg^@v(q|YYg#W+EzsIrP@`TBT2_w(+}}YpMOqtPIuQ2oJHF~ z%KCTQ4$6?hJb!8N{qom_>iD7YD0VaO@BF%a4!)1d79ri?$vu0me1d(@0W{;lz@v5@ z)vkcnC>u5{rb^!3tUN!3O_yp~c=(lHpXbh2yrk{AWkhP^JM3dSV&CEt2zJ_Z9nBED zygkPc{&}zNC@jQ*KqoEwjV}nSHc&Yv1hvqGws>qUJcCDox2uLY;EKj`=RLg4E>d#$ zMMro-KKtlmkk|eMnhV^O&(Y6WrMN(aB--)ucg+M9uZ%?s{z4fDQNn1wmqgL&1Rv?$ zdKSc>3Dx##|5{@=gsv~U8}LIP^yhMZ#2YLyO%rM@2Zj4?0-JhVJlmRkS^Vf%#8yb4 zO)42FjuEFERPpfO{{>iy^|?JSv@0&HN%7d{vzma_skJwghKZ#4FKnB>3d9a-nNnEK z1vG<2@)ULxjOK9>+pl$CsV;lK%xG-APyoMaEGh|5U8a-8}cpCeMF-^8pKe zd^w39;5RTe>W^_I61?fujB`&g%45~83d-M&#^fgpbO!SerLqu3r7_s2YwS(4<&gQx_2%%a*k@4PE$Z2F_&tFGl-JvT zeM2*%&3pKOijfhi-f1guNeBUo%DbF+hoaD#E;wDE&=3*Z1E65aT~aS^v_k@r?zA^$_s#wk6>T_F>>j6vnF7u| z_`vQq+vg}&ii?RTrEN3o=O~H!+m&>IH&i%7y@h1Ar8ip>_YU{uR(|?gxvI> za{%2w>35O0l>0*6J_b-L*VmUrA1n0!h7@1uk4fkf24EyMiD!s~t0c0nQSL4boXxkI zHbegl6z02Lp~Z+vA%pG0_ciKYnSEt^G0f~}7=xaO08P1CDdJxsAtaktso^hunQkwv zzTd4`Tn^x~&w-_7w~r9ccS*R%gSsEPg^+c-9N2O$=#P!@ZiOCuioUxvyREm|844qB z&jmN~fu(;cVC}R3ji3=zCK>H*>K^lH^2pLwZ!**!E8$4AqzOT zZsKk-O5wiuhONgv_1P(X=I9}d_WOO2tC0WBnAuzMW<=rp!)gXdwxK=r$2c`xX4oF& zRCvn7FiZch9nk(9u!}vj$L@V~xE%C!P|*saPRaY}b{|Qm4Hu*z_Q$Jr{b79$xNq%% z+7bqSgH7M*k2FGjAdO&kXBb-R6Zrs6Z|~|)FFp2Q=SyEmH|^d9$c(d6G*K%vR;>Ak zyQb~kZLvO^W7AAJ*M)U?YfRdpIV8Uk^j!MWtS6Q4S;uVgJ%$LUkJ{oKkH>d+G^He> zJ$Y9UL13<4sVD6H&647&Zm*x#{@m;Eo-qchF@DNIc0w{PoXZB3^_h1!#%Goq;NR5d zGwd#&w~7_{{c$RQ=-z;j2@I+CI`K2Yw_=+7ul+bkNhY-H*-mu(XMZ}nXf9sIqS^Z6 z^_pGP&o`T53tnA#p2%FcCu1zFNoT*{$ohg~BV;Fd>&5w21Wi6js&!4Yuy)}df4sK5 zIJ($Z$ls!#aOc!rU#|$3HCsGe)K}Ow0q5*!D)Lt4dk50JlR2a%%x)!>_*){dL4DoH z{3HVs_H2TCoYnUp-e{pGviY}(iT{9b6Z!Rgf%SrJSBGOo#TV`|PV9-lG^f7gAbf?N z@bcKZ_YjK|C&u{QwQ-kC499mH5tLj}{L7(Q{H33VG9(vOys<0cEtkh!&?j6gBT|qV zJGn7~MmH8xuU!fg!v=IJ3<1&S2WM$HIRnc{?1QZgHWRqVC4AjxD%u8TTvD9RQW=Te z+o=Gh_SKIVx{4<6FHKF}duZ_2I(tXu)AX>BM$1#p^-dyx0c|o_YaAV@8oaP~%5-17>E2xx%(2Kw=CT@8KO8W!oDeEpYjbU)k51zw zoMzQ>nRF|!(beqc8OKLe6s(zibF<8A++SxvFz?vaWLKq)f|3=LQ&AQhzhY>L)tGpUCwuxyTdwuA=4co7`CI44cNqKZzHT zOxwtyEMzQYe1(2?wEWSp8{@`sG_k00@~?P(-;Fu5&-YOGX~jcON?HiS;FFAGPUBJUs?Lb}yFCf!T; z(_fG~qtmP91k~m{4=`0gLaxAZ%(mp;)Pf#3hyj*atzAV2>|3)}A`vE59MzD@ zwm2{x{z2j4nOCwlx^dJ)#BMW5ApP?&_&A2awqf#^28-QieqeJ8Ueowg2+M)CLBxv( zwYv`rhTZOaN%mUFiSCN8g&|7 z_RMPL#~Mm0*BeE?{&#idA{#rJb|__p-A(DAH6#vl)Uh|>`Hs6E$KG#N#|K58)oL?ya74hz@k;7(4 z-iKk9N-IS1-hOwwl#5YtBiKRt;W!1K%L%(jNoW5&?MvvyOT4s=}vfojAR5W zw4X{m%)N6++gC(3&lF0h>jop7Km)C(UBHVRH*VihxTDZ#czk8|gB~ z^`ZhrVNPp8(37w1*h$ZicfQ&^G~Ff|mtOcI^~h0&!q)-Zn^u<8Ue$OJeOoMh5h8OK z0~YYTrNTEnnDN5bL`$@V>52wM}jp9;>t0 zuJGPF{4ka+A|@yo`b95EN)l>np4IYvYkM zSKm7+L|r53IN|f~Ua1wN8%PgjwCm;Zw@RW#%v_r@SRYEEl0yY1=s@VI&qb|^{#f@a zB?7<9hjjO+*7z{L2E;q;+{Q*Y(TcLL>(0rwlMOxVjpgO`%Cxbc5t{Nu7P~|hCUa!i z2s`h{T_Td*jI7loKUcVR4NlTC-zgr28OuC7)EJlDVe7c;LMxGE8?$LXQL+C7VBN$W zj$^J&Ff}l9)g>&qUlW`do^;O5EiEhh`rTrIO*Heola^)0FMrQJ9I{bJZ+0nFGVQG` zEo`mX;d(rNl^Q)V>TwqODNGAkjv)O7vul*WsTZ5$zutwo=sYSi*XRxyfN2C`vwXrf z>5Y|v{4$mL(zXxNdpt0jF}Fzg<07GRo#10@VVzFu#Tj#UdA#D+tls`b?+g#6Lb*rs zca0>uI;I(>3H2hsuX^8JyR}F*_L8sScDFkSS!}+dKtbjzV)H@+1{Y(0b0014eRiI# zD@T=8B8ar+26s)W@s_v(sNU0s-6aIyD)xJd0YI|S-ms4rmIeF!?C)u=J-$lice7W- zi!WO!ZPeckku`)Iyfx2C#Ded02>I!>oqfHuL`3<;%t*t#z)mPNeG~ubxblsI9jMbY zPWWW=)Pf!GQM1Aw;4=gpyTu-oaewMM1!vr5OD4#oywGpL_hmGb;B~uiOT3F%*x>GJ zlPshO)J%lb>6(e2iT-yE zM%<==l1&=kw zVX{zCHdPn1BASih$tbV5-r)48&IOY$yirO)CH#aijA&0-44#;_qxtElW?~70*h=eZ zM$UsHf{S)i%_7R4ID8K152YVnSzAhOr(hrPms?qn(^kfpmfKd3V=!oD7N+GL@waq`XbGoaJ!2Bg@7QFn77NLFlm za!%hQ=FGiMn;%A}ffmEzd=H~MoCdqIqOyL}>+yQa74hy}Seg6n?e(zj&X27RNTEAf zhrG-Jvz}fVH(<$Jg@jQVEUKYdBet+73Stkfj<oOma`c9N;8xoDg@f>|BL?FQ-+E+Fb$`{r=oZ+d4F)Q&HpZ7R6KM zK5&nHOQ2;m{WXa?|Eg2u@M0GW*_v|q6?A7)3lC)NRSd?Z&;)i>(oP2J5vTAkO(VyL zB9*B^95eUAJzi*Ba!OCm^+7GfR&zg$Zq5gjgTJ|%Smw0&-qU6uVs1Ym-}NugmUN=? zMML&`(3{NufR{#_gz1>ZqTAxl;VAN@w*czWHtBGYVn zHhhbrB9_y3lqxs%wX2Kg#)4V@q-n)tFb4;JLQ}`k=MX$8Y5C~lGCg`#3)x?i0H9Oc zt_sX}trYA`G;zRM&kb#ui-^gZH|w+RyRZp2y=}=w|B*o}s^-CA=foj=xF)lJ(oaVwr%3$^;|3(hYv3kY#-PU6#mYHA-^>0$cxg= zuzzz9KXkhC*a_E5&s9AUS6i1PEN?!rMS9l961hN6+CP{ibHj~cJan4^VOr~-xYZgU zzs(Q^4OF?GS7t(kP&D>2J=M2WJ6!lXks_|EmHYL_SWTq6Q%398-NUYpN+OT-CDWGZ zroYDJtOz7?lr*AnW-J5>irfpHQ-=nlHp17#-`K^Sf#>4Dw!5dlT#*OXs+VNpuU~y9 zbnTl<#`w*BBWD0Hypj2&e#6ONP801tET&x-kPbI0YzbN$Zlj7*^M{u+# zb!#5GLzh}jbWvDKHC-4(+_mwy%McaOGLB_tQoT1)_&R=sz1@4%J| zevs$c7+D|>$z#_upBd}-o&7i(l1V4M!yoa4A|LCfzZ@$assJ< z6W$BA4WUVFI%n1TXk;`JBn!4wHZ{lLcy7;SgV^h)UbSzecc9|87Ue82ldU?}iq3y$ zBnadno*QjaVe#iQNgxVk{BKy_leGD-h&c?vV%o*xUk^=nXywwHJ#Vh%yNP#ys{FJJ zIRW(rx|xj*;($9ba8j6esi?aU48ukq#JqQ)$v${)KN}#x(o%2Al2ZhRh?CPxV#gaY zqAPy>luQ4MJe8*m&p}i7=co3;$mhs)>*=_89BVzPueuGNv>jN=YM|!jA?tYWI=bT> z6cMB`ow7|=pPP34RLp6U*prk(cYH+IEk-;?y>CTN2*B5(Cl}Nkw`i&Rq<6Fx{%7CFSiV3ToN?^zI#o-&vG_c$X3iMX3M723bO4BBj0F_}bes zO+3?M9~&+ZZ;v)d&rmVCvZo^5#&c#*EkFRE=a#%}$U7`y)$lGiO$WFdykkLtDb!d# zNGP>JkK2g-?TvH0V2gE?MXLE|3P0zHg5N8Kp?+ptXF+@U2&)s!1zx3#LX5E1xzOkQ z&(%mmN1^qug2s{lVRHH_3RJiP_Hny6GHn!U=YVr?JR zeOo)yTV28+Ba_g4$r#m>uKTRFjbciYfqJiM7EHM>ErHQ#tX5?+&2S83mq7zjFCsR>JOGN!7_HLqPNse&dv!b zDW5eBd_UN)B0ak??h+Y$W62M=vQLk_y^n1QIg+@!OGsFhBs5TopXejT}AIf0sUR;NDE#p+KO!p`*kQxWUK-T zozw1eX>Nw`OSlP3reL!1dsLRuzn2h_F-7PBE@$$4ZMvbICq5zdl3xq z&w`T8geK2IV!5YP0KNT9whVftk=E7tTcs3Sq!E+J$;s28g5~A6`u%L}$I24^{P9w` z8p{~N6OvszL@PbYFqVNGoS>9?0eqPG^qlbL9NpJszVR$)<*du0aPAC<+9@7$^La4S zgw6*SH>bCI=(k>@hmZJN^k=0mi%+;TfMx|zzK0{olh4+>=Nep@ij>lWwg!PjpRAt0 zI!lnvWk@08oDe54wp@RYfz1BA60GR?1;kxNq+u#2hMa~&t2c>EIZ4m)513D{sNpi| zf6JVXF4c$*K8x>(f$B^ezv)&?y9)2y6Ms6a@C_-Zqnm6z<83A~c84t#^s}Q%Rl)V0 zLYtMmB*4tJkY97^YP?#LYc23peO>5zjyde@>4D7}wrx?RuNPH)L3+s`Yv7WDuFb^P z^14#r(Qpzujja{oN8||$VIDH*Cyn&wued=Tx_UYT=B!9L(Ug^~*jE6LJ zUy+m^`bCZI8SUFT<{lNqBC&b-wQ_{!;!&?pc2&^NxJR%M$hxOmm zKaImjS?Z4i4t*~9J}OKpBj7}hjJ9JKtL z3AtoU$>b&Zyu(*ZQSCA(^f|0txh(r9U*OY{VnHVWnJZRz|KOucyX>` z(OqC;-C?A6Iu$j*QFVF3VtmC^+wd(Xt_Nl_P{iQ8ZAQ+De^Pk;GVKoH=8B$_y9c9$ z*1s}))O>DP>dJRK&!MwvkM4e8=TuOd4o?>SP{@n6uT2$VD!W+zR9p=QZ9|awTVy}U z(Oo`D&=Ovn^eBK_xiO`)`qw-cD^)Stg;UP<=9r)~MMBYgdFoEp1h>It@O-CJ&sB|d z52+|J+Mug65h1MLIIgt_4Y4r`U(qy8k%_*K9R?h`A<+@#ejN72Vbh;C`vE7vuwuVOY;0; z)(fzx>B{M#dN2~kZ+*j-#CO<7L-t-iV~v{hYcAnXgci@c{zT?kkxW^igbP?w&t7`l z-qn#XKyQNkM+V2?u!NF^*BstBXDjQU=RM5XKxwV7HrW|TK8fw!4XQE+81cLK#5B!3qi%!Fxzvcy5?fe>V zmr7WdgJiI)-d^y;QeN+TTZt#3PM8ZCR`8bui%> z(a@k077l$vkolP+6$!<<9+fc|;ioZ4Mi{SOTQ464u-o^JD&9i2CZT-YfL~mw6IyI} zxD%1A5kP6I)AKDN*!sb3-AO;Xw{Ji#<7{dn$WL3FVA*qrB;@mFf()zcdL0RauRjxd z6elsTdG)^}eZ2qGc;+Tr(n`hw_N)8#t$12<@}4%DZCk;6g>UqZdOLlHQ@Dt2e!+Eg zo^ovgd4rRS8=SpNci^1|ysXK53h~?Cs|KY`+RTg!`_4={c@2LDjRQ74@3MZX8=KZT zw@o^Y5Q4$ls)GVH!4(`(Y>-H_0y%sr_}4wOUfiS1%G2<#dqj$BLVoJ`vQ>GO!E`_a zDSe$;DYO!l;_1!4J;3wMBt_Hj*ur-f?vbW%Fn!D1C}(S|nPJ!eW-SbT`F0ueLIwcI zWje8u5J&v;f>Tl`TyS%5qRpQ?8Q4Y(iHwE|K)K#OI#WT={SYL2OO!1p14!3Z=nZOND{;#D zGx#xFh%mLEae-$4X4jxl_Nx6y=_qkq!h57ZY%Y{p znpzfEOyoOJb_C7-|FTImvcI+|;kkA?usjs^s`b+5kQT z$mee&b>Uyiy#PerH;@9bm9T8c#Pm;*fCa%qng2!+9vzYnV91@Z{*Jp4^1PM8;odBZ zKLbalhJXa$v0e1Ha1?uaE}2{17q9l9r)g6=Ql>vIs0o7zc$C{S`UmZ{X~;MqOUL(x z&0oN8`=IjjUms|fj#KTi6WHBF>OZ%+FhFo}XK;43_#;ma*30`NPV24y=xmqx<$KiY z()te+yVpkxc-`II;nC4xChR=bBs$feN|WBPW^eb_eIob2o>oo@xx+x@*LK36@~ABZ z!%*UR;O1=duY_273(eQZ2#K;GNi(`s<}Y7ip}F3!n$Mdk~}K)&-cBREER z8RP0cWgA3NGXN>eUrDoHB(>;O{ab$F(LfNwFC_|!$T~dzI`Y??I8lJ4zjg&jpa&>f7hv1Il|%_`J-h`Hi$YZ^cODr{=#4fQ5S%S zjrHGfA`XG?6}vGWQBaz5hvIYNNlWK?cN9(BO=4oA3W?17_lY$ohPUTi_?)KUn^|Po z*i&D9*L(gvY;_+p!l!8)Z&HE362w3xu-oV^{PR8eD~L|XUdbK?NvX*^78TJV#QB~T zbD^!puswly$Ah~cMz~>X|BeI`GEh%Sh>{(6FzMOYfakW;Gc)=zovr&li;ziNt75lS zzsGXRLx>JAME%nKM85_~$X$(j)6n^&#>Ww2SuES7+@EvUn$CZ78PIfO|5c1OMCKFb z^F>z3D6?d8%F0l}P)YX7w(FNvsM(FX5W2d%e>TEH+=_@qtnUwQdMOY|^roE_0e{Q| zqAzkahV_ZJ-$-Ey9rzXkxcE=zxG0DOny|m@7HE%_U0t&);8~$6rM%8J`;SP;A!Z=ki}u$H zG+)d>^C$6j#vhqq621n+6yd*_dFnzWtfP$2$T--?HE2abFf}v#>7D%lDG5Z<@;+UX zS?#e_!NZ~azkbXHvF;T`@_NGm00=>RXHbEWx5Q1 zYyg%Kq7s#?YA%WU8jKp6e~04a?z^zex<4ut zBR!MGuNFfu6dx6Aaj7#hQgdZ97~go2Q%z}Cha1O{-idB@Kq0na+3|UTYEDOMzb3YZ z0}GvOI^&KJ6@H&BnGyiVovaear&Y?`tg*N<-l@gYO@^8Xx1by`K2Vuu?wp_{r^1+P zbfwdsE#PYpY8eR{{S$ox5Nlra@)5)FKSe>1b)kodB81Jy*8`gtK!(1HF0sNCB-p+$ zE#Yr!j=OGKC0vm)9C$8OKj;6cm2zZV&U}5IKjW6tx0FpRL|Ad{ody7f(4l{ZXz9zo z<5wMoZ?T(sZE6-BB}yn!(1$TsN7lg$jfmot7IMCVoWyFF|s<4kYxwAR`(g5%w zDmULue%U6u3ejRKR%nbgv1h(}>s0joJR`QhHwq6o=aN(lFUXZgp8+0 z9BT|@p}Div`I@zgCG=0Y?`9c9@NcaN;2=u@Q%K-S#sXRB9C)*3^@_T$d>LWOcIBQzBBoVH*!B3K&uWp4S5KS zSHKb+3T7*6+2XE_B0Nc79>?E5SA=no5N|`^e38+g2LAZFQX2_G*GuDT-hX1oKXvu4 z$4ib7cD7G|WY!Uyu`fAdwnM}C*wd2}3g=4Cp~91jo$S^f{9vqsUuOuSd;()1MW z(X|E6ALe2SLx;c;)RGyB8rcE?dMh9r)oaAfz2yZT^>T5f`e5wyFL-vdKqDK{&r{>V zHxk#i>zGSG$tU`trym^Y+_+IYK3EZl3+@rq@Vb2BHPf7#s{Tj%RFSn$B)ugXnA0hY z`+1cRW*BSA=3F$q*liO4;&)x@XvqGMP2-aS`)`QpHT*f84%^*qp=B&c;h&N}Lpv^c zP#0JjOBRU|lIpHG$bO`7F4;oiZq1+T16+8fUtIZ{1%_EYK6vqWvZwy@;2*kRsIOcX59v z@%wIX!iCUvQ8k6BJ#X!N4{B!J6{`2o6RMK!TYmhxqv)7N3o8YG6mkpc)Xtb&XUjgwZ;NgO^ z-mTgEYbCmnDrW@_zh&sJ8NJkCDv-SHM;Ewb4#|SM+|b8xz9K%+yE(y_?ej#0YI+|+ z^8kwRN~AZ5VLp>^n}8Lj&*ch)>9H*$!TnYs*nNq;$gn2NRN83CS<~<_9?3aY=^LJN zV<>tzA6F@^F%j&w#8SZ}F-O$O;7!=NbNg5O;{I+zb=RKKT7eG{wffvK;h{WsGN)dRL_i84? zcxtED&8#NF2Ndr644Wz-Y$rKnfE493ALNgV)LC2-dcb#FCpx_a5;Zq+yu{g-N}?=} zksv>z)!Ec+ac$S61-fAG5b_##GU;Bmj@Pv`(;2S(+8RjrW?SBB2uE}H;cAz^7Dfk4 z1T`ZgzkQm^AW2PZ-yJWd%QJx||5j@uQmN5qP!ST)3=Qw@|9|Lu%dj@MrE9n?P~5$^ zLve~04-Um?aW91yhvHV;p}4z4aW7Wf-8IFXAb~(2@P%{V=REJ9?ui6M2c44<_27zzR| zHU>*d6j6<30v6F-^1|ZlyEMigL^6vD8@3rOr~|Y->;HnJr^L_MJ&m2Sw6tM&Qi)cd zq`hM0pe>RI%ayn#D*Wb4d_81TrR?{wg|U-qZI%Q-V?^Y#)|Ip|9%y2~3~;5yyV)|W zA;Sn=xe+}G3fz~+j$zUsWsMEr;eRLHF<(;C+!F$p#^oqQwFrf5H%$GR>!qI2&RK~x z49<0oi5pR|VixeG>b z`9Azs@t#tx*{+*pu1kE+VtwtKIU$3bW7Wp2*y{qfu@@LoF3A`G`|`UURB~bc664(Z zsU-r82vlxE>x-gRG=@Q?O$9g4paE(NHMS+oE+m;8LkBH#>4^|SMObZWwkQJZy`rXD z;&wu&@~zQVpNBCIlI0ceOH$N*LT&Q3FwC|I1%ZOW$vz4?^GxEM3T^2>k{2tNZnI6T zKPPl_e&<)+@AkrZV`KMCq%LwQw}}54zR4@Y=s#DJ;<0dayOOt&Eeh<$J57>)g(SglR{97Igf!e`B>V?4DCZDg%-99YiU=|2;@Ni8(O=%*=S zWao^Jurdg?DFywkSWg1S6uwqo?4E|xf0v7!QkK*75&eqEUuGA7elfm)Kq3)|Fk;KQ zAdugtyv_i}jpM_*J1(4&=2&n98B6J8{41afrOf$Hi~?!k4#i7?_odn7pKSSCZEH=` zf8w)xT zSRRWIf_KXSw6CcdI%7vYM>|=ii{=lzA& zdQ*akPAm0}ibC>_x^7rz>{Vj18FvCAEENkM%vvL48Q9>-^8RmZfUm{o7w=3ry)AsD zfJY1>a8m2mI<_3rcrPYY_i4imbl30R@^3c!Pk28iRF2UWAhM{@@< zCpk1x{E)cqLFx$-v``|f5j4o~ut7lx%{w2Q!WyW?ys`r#`m|BFjJ;~__+hvnkX$*s zWv-Hm9Zg5%mc`%xY{LF4TLBlNj7m@iR&nib48Ick22ULd&t$*@IRTAtl-!cq2a<%S zjmtMbzL(s1bGs4=;vW?J6f-4&$83T5?7_z@B#vGvNygETa~BRcSnzUd8U!8TtLb{d$;DMAR#JRR$>`S2fSkY};%qYrr8*3p_fmTv`wu z{`>dq^x{1_v%d~~5ZAgbU>QzP-G65R82A?qPBb#B9?KG>yd|tv$pU9MLYKI;G|x~^ zQI7l%THO0&gA^lQq+}dBbT(iRHa2)$2tqMPVqW@Wf_v3MR6C^L^a6xjcgk&YP~T11 zh0@a+$5$)%0?6WM_uzau!*JMkOTY>}DiPG~5)cqbKmcIF`2acD-72^j{LgX0uUa}h z{V-hdUh{zeW<@fWPtv%Ohpk+^$IGR+DCApahsljRu?2K+P%~XO_xD_;+9#ilQ%fC! ztzHG~j=3K*^`98B0Q;Kj9n7QfH}iB$=7%7M%QMN7@4cwY#pmU+Y)d(U6Y>PTq}z%` zm9}jf`OJtoxmg*$=n?w5Y{6MJ9FgS8d;44g3(B)x^X{P9u##{K(c~(lK+C;{4=W*- z9VNa3sL<{_c?F7~S5ZK zi>SYpGZE3zlVM+R74`G=#LS7nmT{rJ^z3N&zGBPA)@qB))gj@npRIc{ZF4HgORLLL z_Nr!u4lvrB#s%Ga>3Tn`_1{xJqWJ4@C2yc^uYEeUN(k+&t*$%G9JkdrbX_#op8Vc; z3t5hU&MBY--OZxr!@sj=5 z(ichbFhtnq1fq|6BL!0ok!0)yew5B-i$Ey1eId_aBTqNMz8GFosTvhpo$2{GJ@aOE z;-k?(8VGAUzv(qjP?+Qn48#*?_pZ_LgLPoOG7ST}f?kkS)%(DBXZjl@toxL&hSWsE z2zngU4KElltMQPpoN6oNqi3p;>u7AvhjoDRi8iTPTLmkt72sn?rdfv4Q^N@`%MoOR7w90R2JR1Yt5Yy5 zj?0KK?7?E?ZdINHHKP`UoL!3fX%AxaeM1Ego7wp>b(=p^Z=`cAst4v*Kl-o1 z(lO5GY>rI#cWVkoX?C7@a?%RBU!IU$vLfA3Do)UgsX9%rpLp3 zboX_)3~;g(Cnv=9YjWx5dog@cx?KpMbJe%)xN+6-$V9B^q%J0WBcLI?g@KOnz3bIl zmY_4gP06D_yO-7@|427iiW8Thnh}_=Y41!f(p0T?8r7WpB6)fS6je}+a=f_zo}CHq z%oI#xX6D+FoQ&PnjP)1DA}=%JNtLBZ%%3TJlr<~!oX`%Q$I zpAL8QXqX$2j_pu06KS_f3=Ykchu)B-yY*h~?^dtw^xMSJk2&%Od=)q(Oao{WYeZZX z2lPM750FXcgU{(d{&hB_I#Q~29Jl;AoA#SIC0<4$M6(dyoO>tvYom}cwCJ)HKTeeH zBc7|Z%C0GlNRh9Symq@Jp~O}EFylH~ZvZOf`<}u3l<5-+IrCa-phz=Cz1Zo4*Mg0n z*3Y?xruj6u9u{Bo^X6<;uCeiNY-_RA?RLlAZzFxWLyCM6rFQV=0Q3-juEvvrQIX6N zn6B_~UoiG0!>hpBv{V5)<>kDs<#*HO_PrDZyX)Mtl0XB;QTx5e#&*zS1VV|ceP1#R znjbfN*`%H->Xk`b*9<&!TE7;&9e#7-wE7us_v4Vp<=cjZ?4j;+{JH(=!Q`F=jD#PD zSH(2NE2)XCyJ6STeV^M@hc@i)6_wyUIQ$tzJv&IGs?b$`Av!&*NlR^=DF~@v)xUh* zfAY%H=lrOx=0Xa7gx^h{Q*6_1kGx|;xheZ5Iu6elTTnG31TQ2smAKsbZ>2ceK|@mF z1#}|dGM}U4X8q`VZ|o+5ke+#0xHthINr*v7)D9j_?$w1PMiQz@U1 zxR+OGE~;jFB16Uf2yrIc2Lt3MQyDbWwL_UaShz7xJ9}r-IqGT*KTGZ~FZjLvz6)HU zT_d~H&gy1K^pu8^WD$fwFAIkFJWN)(EY1w*Mg9>kgHwJ6-!Ps)ax*{>k6tUA~+$dGmJ>3r?y;$g3vF{{}-5*Qh zM9WM4Jmg%wmNZJ>f`UGNa~$ls+5G7@z4>vxOm%h=LE|y`>w6c}E@)>NsnV*Az_a}E znf08%?*ajo8OaK5__QXRoYH66wCd7g4gKtmqRSdv#hyeI6w#nACTv(+i0Q0c>qua} z1RBq~87p!uEkcUIyNXpd88)`-iorb4TWXPO2+1D5t(?S{1_T;zDP~&kJ$Kw*$AsLe#pvV2Sg{O}i*QI@OqK5loH1>thUEMYr=##b zGp6Y>9wKm~Ymoh0QQZmn`0=B4|E=!}ys)saKbQh*Q)hA4hqK6KWn~BJZ2|#-fw*9} zgxJtd)wXhA3R}6SN54uYQ~i(IT>jrt_9ZpNSAOs((+I$oTW#EokGpR#8#sgC2n~{~ zZm@yfGfn2kpq0iRg!RdgL9tJmf-h>d z6LW{CCqkq#sQs&Bjn;HUV?yTfb}y-kALdFu{gm228(NFdp8y}nGUL?q#CpP399HrybkPs-~308KO2T+gDJv~94$uHELoXQFGY5!nk)MjZQX|0eig zCKgSPcQy9|xzFPaY^1>bX`4b(a@3hEE`Xy?rh+vE~hwzQfYJh52i5idVr4x3s)?i`_KCSpWhSS7OVG6_j zXB1g0PeC8rz#Y1c3ZU=rj@8`$yO=GxsUS+AD!)Tr$ShE4>S;rNz?6pQ6S3}>Y6eMg z1wEhuK8!f@DUu;^Xaba7LM6BuGHhEBO{xO7zU;>@RapH&CBXZl7x|Ox%cN&QXsgTt z?}BJ@-_EqBiBFvc<_PO?;XExOL>hwuaUM%CQ_u6j{?;ZXzcV2>M>(K#UP|2aMO=6ts`7!hqwWvR32%rmgX`1; z0$=&>@L0V(nIE<(CG|2>Zg#A~H`BC{zngN;6idsPOES^B`zQ}cDFeBDG*PA3p1OS6 zWmbjq(4fUgL~laF#&ftiFkxo2haNt-l4JbW846R)9Iw<`-{?8$RIBih54@$v7XNB` z;3I%8<;0#e#E?;{MAp+L%F3|DR@2;lc9TyrXW!qGj@((>wI{>?`dr(9yls~H@T;R3 zFUiIAFbn03G)6wqlpOvh5p>mZsc6S_YM+x`S7&b%d_qbPXeDCDB*sPBjSO#KT$@R zXItq%>c;S@wzigp*9s?y+K)jl6h4~#wmX)q$e5GhY(2<@A>>TfAhcingDeES-~B>D zfNdg5ioqg0I`W8}@LP)mo=__i^E;9J9M{5Jzn5~cSqxMrcT*jQ|Kwx>6&+}@?jLwP zcxSf|+2y}WQA~WCD;H3Bc^aKphOjGmH@^4YmYX0&{8Q|xs-mDGS|qQ%RR`Ynn68u2 zwUKqsLd!CL-$01Mb^=)PYwo^-ii>jb(Pu|32wjsWQzVf&OBED@p~^Fo62-JD{}nJL zrJ$jKAPJb$3(;$#Dr#W7Ne`YOibTZHvl&wFy$ws<*&_kU`s)%)kGs6-MIpWaMx=~i zYKJFVfj;akuF6u}5Q=6qwwxQvT`NPk@RyGhP59fGxt8ck^l77~9WFSFgItYafSb&j z3}x~%?&TA5fkbV{RO?}}nBnC4hjQBFM}QD3 zQ05&&-cNM}W}cwPIu2PE2(&sSq2;0ma-gQ+f;J$hYc5@8N?_tX0&sbxeSh@* z!e5WSL*R$al?gtT8;tKp6lX$r3LBC2lH~vVhQu@O`G3r$lYVMxDGa{YB^!QV@LH?& zh%L!-2^JO1gA6NOb{2WLSrx#8)Tk3@9DC0cQCNdTpjnz0a;{S-bMvR$FlUpl-+vX% zj8t)0N&+2EVsiZTfq0Maq9d+nk9@IrFZF)^LBH=0Z(rm!ni2_BkTfkj!vt8({8WZl zk#eE12HAzF@|t&Ip?nlgoAE%u_h;%G*Grk$*2=Z-@Yx4gJ_Vn)sH-89EWFX0bmhT2 zs>&yPk)UnMFPefRhO&HvN%lFp)u>B*lxp+$@`9gw@A?4ll7lTcV1a~OP`{`f_wRt7Cz~k)aMF|vU(r(g)y`(_=i$2?KUwaZXm*{6 zfse%5Wr4MM1g~8(SIoznz>Cx=hjhFLU$P5?b#q^yaqnU12QeiV8h}M+7ke#JM zZ6zoPrT#OV&4(<`l)bovBZeO&@uA)ScY`sWZ}32(2+lwB!#TG<7B1K_J~wBh1UKjL zu3D`~!^NQ4ay_J0-DRm}o7Rp=vg>lksJ;Fk1Bd&~- z)%NC|5d{|^G>(}{+DAB~PH$WkI3=)H;u+sjNbc3`@^|Nw2{%6% zrGJ5pAi@Bbw4W$E- z7w3)GppppNkwAlkyNM{nvt8tBzRYrvrZV2V$EU=+Fk_=_p4QaW+A&z;T|8XTyl3{< zu5cV#M~BK{!xG2X3gbgNNlm$AqT?+rk=e(eHTPo{+PaxG(rGC(r)Y3~0qsS7Yg_r!NoKRr z0{wtB{{qO%Y0nEAmh38k4lPUN;gj(t#h}2#Dt43hPza#ApKj+z-BhG^LNi{Urnp-o@_M_$trZTj5x3-F&G4Kz4$@kjEx;(4fGt>4O2j$JTT*g5OX`b?}l6Uh8;Iap`jFADb$Cj`sHG$#PVK ztbwUe?vq#)W3y!$YRu`Fh#54rYVpzBe&#p_&Vxl=@KSQll_JqbO zoPNNCmr6_3wctf1)&#w@HUB4&=j~x73Dma6D~pCWuw8z}N&81Z@vGhW_~5V86;w9e zdmhD3^KxH}FqBBdDQ0EUfga|R$aJdsa*??p_y^hVD+a1R*6VZL@)R8(7nIo8MB35+X}t zH=`)6a`2)@TwXLX=gYjD-!~$_YLd~wC=^UM!1N+iR8&;AKZ@Wu5YWyZxTv2a9-+gC z00Jwu{V+&N2iu6XD?@c(L=DU>yHQW$koKu?!y!KRsDGI=NGf019hFQpO)^~oHU45( zr(W`+ju=xRPcHtTkIk+rYSJ(^kZjS)Xv_1&?aG;5S~g*59ZrRLrtn+v^F(tyd{aCh zhE@{1P!zK%u)^x>s<93I`fF?2 z>hpNEKY>*HV|u*E3X*#1yzo&|YfJYhvxs}Y)ppY2zV4fuKpj|`-MjgBZQmhpyqk>G z?+OMJOvGn+iAUZ10+jCw0TF^m>$m}8GAY20FPL*x-Lmcb=^i=xown-tvV|oRSxU|e zA}NUUs;4T;-pb+ba?d%&KuVvwogS}3)_eGf8>tU8Zflcl* zW>LZlW2lkrR~x=))}QRB&CXY6icMU<#M7{;W4ij)QNE!}kFEbI7BNOTaIVh|tr#!= zBWx-uer_{D_pe?Z?zn&R!@YQ06RZXDk5ia(M0_tjh5UMJGpD$K)1x~3tynT#y)^w!z+SL5E;x7sD)rs?~4aS20v@{15ZZ@^@%1%U5`s|W9 z$%ttL=*VBwNae$u@;V=wo!m%!cIZFTcFf@I^wX7#~^IdrE;q4d4-L4X0lULzL31Iy*8Z1_Sox)IbVF3A5^;MlnbRmGVJ`Domid-t`fXDL%M4$2nBge${ z^Lh7+suhLqF%MmWmC(L%isLwI9E^7?V;|C!e4R)(q zEe1(CxEdK`5mynZflEF-Byx$KqKHWo&IFij$m$7xD{^k)9ZSzr*!q1C$f46be>Q`1 zvO8qhF!oT;ZPUuHV}~^_jq%GoRU= zT~`d*DR_T)iHw)M(rOBaVNadQ6DAnT?Ow+K_);DNB$>U>$1M19v+37;hS7(Jv@UNDW6F6f}X-je$ZU_WV#ge^T{c*Ts|d70;~r# z-;!qV<2HXT1W3C%s@=wAUjOEz629RdBcZS_pUiAY^nRRW{;d2(#N&Z@^Gdl@j|BIW z=F%!h&=GOO&3=BnPE($gCUwzU#jIh=y(%PdI{O_e6g83j`|jBolrrWM6#k zd1plIDkIuM=OgWa+H_VIanYbjV~xh632xadu1(Mp@v1`km`}x5BzMugMv=V!dP{7c zj~kt|>>^}}3F=IlHC+`((ZC8=;WSn1pJP#A9$7^Tyz0_WW~(cHLScag*;G>Qv}Qwu zoY*&Rx9oaK*`lIY`jZXTw?)&PSk|~TPELM`Za719mWQqxqte5uelB*txgKOz6J8Df zTguCJR5)c9%eHb#qdi~M#j1p5Huz~}eRJn9@#3?vB3A_`Ndxdq@0vc5YWxY=mEesV zV93q32mLV_qpyO?Dx0(esy}8}G;>+tvD6@CS>0rzL=qfMM_21LinV?Zgbz4tK#>E@ z>{Ws}NTt1;jgW*DvG(X$x&VIhTP$04d*P6s$89V;DAFTSi@p(mygY-{iZ%kC76BmNGC zNOl!|ewYmP>x(X^a)yd@fmCY0w?;mw+)aGWZkOw{&a%Eih7*+cfrKsC{{|{Tl#`a^_V{WmI)oM?uWFkgtw=)8=yQZ)6HLW6c$^zqaqvIoLcBb#37&U#8 zs0s;C@;C`%Joh2{G~#DO+`PW+mgDbyV85oUzbc2XpaKDM=(Rx@oxYb=&Yb6pfWHZmW} zv?)0U3o0@ey9Mnz3=xihL_V@EpJqNv*bpjL<$KZiQGvgJ3>j0cf4ji9S@`}YzJJ8CR!(v4hFby6a?mK(8v$%3}ikorCFjN(-i+1Gde z6GUK{d#QIk>IT_K{EMRfB+!11TTbLS9+9#yyCR8@nwugw}h) z+u-}TG&tn&UNBw#>QK(AyxnAL4AdrY0M({=J}%0M4LV@s0N?#{F9K}%%^LB)DZJ$? zBSNsY{k^b?mRipUPOQPt>P&Q4>;V`YM*@J)^oL=W;amyqfCd=Q2D|wxYL!sgC`FUE`eXNMUKs+bDx-g#ZoJ8>xc0R~htiH|$5M(;*>1ct$Vd7n zminU_6=hO;km^%zSy_0#_ps@MJEdpNd*>T5O~1-G+f(6DLG$Cbg_bXVa9XHSTsQFk zU-t0v!yjw7)$3TmYHjB94MbGDfj}KqHi8Vjfg(dF2J_Sgew)n5_S~!r}d( z=7p+*$des?vT*5df*<++@%%OhyB z_^fDl>QjNb;8w%uW0Y8=hSO+^HTaO3QAD;^q;Ns{Ud5-u%#V2fPs2?&S<&n^K8rYu zC#rA}9fZl+!T6ty@lOt_yWtdMzuO7;-xLxB{*A9hTHZb{j0o(*i=J8^`v+>;Xk!f| z?eAG|!*m>fl!mKT{OVpqg8!)`Jv?{YJlas4k^Pf%-Uh*c7V92Kf(pNklo$x;jKvtp z^g&#}uCRb_E%X2H*20%&dF=4-O8ziCFBSDu)g18OrF`bvJE5!b)nQ*j!T#>J--xy@!j_pTVdY&6EX_k6gVMW%v3z(9K1A?)b1?gQ_&k zL^}!TkBeod=Lwv-Cx^=-UWbMKy!@abRn ziHW^DTrIEt&-xo~%uROo`U8)zw_3GrPGf2}jpW8){NtEZ@V5aL1}hz}eF5-266Z#A z3;cJFvf+E;YfpPd{m&k`XGo+FfRWe38k=M236xbF!Dy&GA*xS5J~Cx)xcaEidEGk@ z*ce7@$Fy)T0u}m3vDYFRM)zyM5VG3p#w|Ngg4#z5WLm&_^3iRES(GlXY_8g`#F7iICL@LI`J@9CKQy=~<(V&nGAI>kbh5&># zRpBu8-er5Pr&!^a`l_dO0;;l>6FpnFE;^6*T?U(LSHHC5DNaHiAcJk9UJ~rPgcH2j zpb`&xm+~vMN+T$Nvy(9g<*2a5?=@(Obi3nI<$;T*FeP-G^Ze$E;<)irma zjyjOlLGG?fA6MgR!i|rE2Ze9$k$&BuQ>?)RB9Hb8t@H<*Z?2P4@x8KJ!~(x|wA4)e6LjdcBef&|~l-F@Jnt-L(u$H9A&_Ddy%2=|@NHiCrS^z(j!k1dBQ zol6qVPguqhoAC_?zuds|b)-&L!Hd%Bgx);Fgp}_j9=+u~I2!Pef z{NMdFNAn^L!%7{U*^g?gw3$)kj6i2zGjKbx`uVo64n&9lOsv5DG{ONpzx|-1w!KvQ z@~miI5@MZrZ)!TnKVL!VdZda8!<9qvn8|2AEIWHsWEYM(zsnmHCN|JtT_x%JnX2y^ zrN1F6JoVi-YR~SNOk&EO2Xn=?eB-0rxyJ@`Z^+|G4)?p_S4!Fs*pv+2BEoq9;`sF# zCCVx}zkhBGl{^fAzuGu>cb{f_m?y~Wf{5 z3EDUe$e}@0wYJm{azeY8VK^pU&YHfLibAz*hedM#hA7<4*pC=J+VvcThwu__U*Rjb zQ!bwAF)dQ(mNT+YCeb!yqgP`km~I2o(~hvj)WONi!JT3o0`cF=ec%Z9+VWu{nrQ~kKz^@KUg za{^;>WxBLClz&&j#ME2{ev710nML(XEC6|3P$o8%ud}BY23+Uwvwb~#7aDZ*RU;ca zILHB0boNK=*cqYFecvBONyu@JCHOk;-@hi~e@yAr>^Zy*`%0+nUchcW^oeGmmF7gb zYjpwWwhMff$v=kKAuy6wQ*qSI>Z7e0<@4u;W%!t2x-g1 zR}m1<>scAE+otDi7HC)7WMzDAAk;=#$;oGW~G(tPHpW2}qly4K? zYCiHb3*P~aC0!qoY&eem(+cp4crf`&YPJL>x4H4!<$yfcWWKI+@x7nz?HoBRck7w6 z!92+^(J9aw37;~Ndi;w?*h};?AB<5ZEUC|7dY$CQk2Q!_ZeNBrItXTpat9kzMbtWt zJua=;yuTqWM9M@$$$s9i8k;yhZ8(XaG!B(;0d%dqib3P4J`i&qZkfte6`M;g_q+E# z-BO;44;GZ$o~Q5GOGVDvnv}`ILS{YaLmpp}cranXSc7HM2edtIs40DRF#AOB%*X~8 zDeT#NanbLSO;_+i)T_(~ker|_)kgx-V#nR7mK!q9z0EQi>NX8YW8#IRo6wC&=)OH* zw>bm%>5}Pl!t{{Owq2ceqL##x-#RF#|Di+l!j9D$`KD9n{<7<6!!QHa$NV1!wvU-W62E&|FL!VWLg?w}B8Wcq zZ6$?<6`2`r+1Sjm{hFolED_gvpx!HF00af$h4Ma8dXZHGz+0$clV%q|-C&MzVQ|c} zJCWE0|3at^A`EJM(9xQ+k%6;g?~9HvCPGU&g=OBx8Zujz9zMO)1mR^=B`JGIo-?;oF#CVa`di-o*KeT{T z>=xozaG~!zn>=fuekAy&E@)_4v+uHOMS7ZML2P@7^ zKNVh0yPyWmJh?N-%55=<-eS9m{m1e9OH$(?!Bl;(_`lqsOYXKNFqgiXLB^z4 z@8<~5K}zE^^YCPQ^LA9tp%eA%jA-uP>aOnVJAqM3=;{*!9eN334j1$qspom>vUx7aZ)pJVWdJWFVgm@O<$S93U_r`<-TZFPgE za{B(bAmhR*g0B220s{L71JcW{52RAt@(zVdKoDooOm?K&d1M?f(R2!$XR<4=nqj{& zOyG4_?ffZ;GTAJoxAPUL?QfduP!+df11 zhRk!3LWphthP{d4%pIF<)1k(Xjed@$XAX>&#ngv%N~0(c`4SF1VkpyXiR1`UO2ptS4vhop2q&DoUVU zVj&Xsc(~f4L>*XB=w;R!>4n+9YU*_sww0$EsS*36OcdY$@V<%Sg7X(QKT8(b>N=Mb z%Yt%5N|ituFzvmC*vjNo{vf*iV^Jh9t;1{DdI*JHE(C9uF>%`4{c7BR&EQ6L1|-D| z6HCmCYLhZd59S1nrFYBx#++qq&U|_+%ja7Qs|8wVF{gC*J3ay*U{&Y=fgWV+q4W~$;gkjrcNYmKL zKlD+6Ot69h!@zS;YRwVJPw0!7E@>0_w`(+EC*U`uq zH;x11GkRS0h}|rC8V}}QuT@MYye#ABg5w0khP&J;SK(tC$T+(~PKMCNydPY&Hrlk% za^kPXXvdgLJ#P>=P1dE+SZcRkuGJ<1c;^z%!CkR*y13vXHSxJ-mo6V2tY#i7j-wQ1 z+K*z`r1$1(Ty&V2mae@wbD-U-#|KA)Y_ZI~-B@-0FGP$9kmC$DkspJFZFD>5REh4N zP~sS;H=;wuxS|FiN`;y&@w-WjW*3beaa~_g86UL6wT|1mXjE4cDRm(Hs1tN6JYt4Z zIq&RQK67gZtt|%>fh;wxiqPGiFJSH`4$QLRH3WB>p631N4e0rrNItqzDW=DM0*O=- z1DwE@zOuCR%Y~Qc#yE}cjanu%0TxUAwp_sxy8aJ4e#wYFCC>Z7y*jlga$yZAH20nR zNd}nsjt?*|D_=CnbHY9g@%@$J>KPRrG&%+%u?5%}_Zkc$y%UezP$6_ax98c;V;<)5 zxORziv|lp&D!~s8O4SY!7J^2IpexmkNMi9yJt19>`cVf9>%HNPcEaq-vw12BGX~K5 zEUE2O8AR=9tkho3DCka=tkWaW^jhu9p+sQY79Teh^bBu)m=fr;w10q7h@^Xy3XKxw zEqV)dz4I$i@Fkm>U}@7&cn&` zzPoTvta_xOJrwvmj=tF9A+Sj8UN{Klm(nojV8B4-aimHy#Siu zz<%c%i`=;gYreFtj;7v_Cs8nh^j4Z&U)III-UVNf#*V=JWFIQ@$B5im z4=O-v_`xflB;t=+zTer$r7rB1x);RZLjoACVt5{8XYI>SK#9d{Z1uVJ1LMHV=XoMR z#GU`^6;}LH5%4%rePa$(u*fNGWO&>tLXDzK4I>qZI+`*6JMo`(f2~*kOb4?=N4@8l zfC$rUS)(`@Ed$7H{+(0*&k}ig<+xB)Grx=F*`=mygt4awHit@bsjd{Xl0E7GO7B$# z<~Qnp(sHqX(Q$+97%~ISzIMJw*M5laksl%ym`f};KNoNZ2Y+0moSvmuQvhK1_fm>_ zzD|C;^(@3biEUxt99P8$}O zR4Gmp^&M4FP}&v_jY5Pfh2#XtsGqe&e?T||t@VC%>%!B55^5$ezs8;5}{hh9;Q}{5%;MK3l10aYz661PTA`zCwq+@f}yY6H) zakB~O!tFq>x}V(iaeFF--Rw27C z$XsPNw$|3^S0~rE&-R1GIjN~YgDnyb(_Qy5wlff zY8X@g&)kW2RqAs@7}>>j8-ykc5Q3t*xHZFr=J)Pr56~bvII&&U~FVBxw`j?|>+u zrTIi5`k*1z-NBGLagw;uv) zKi?@=>@@uW@Ha}zel#!2jI6r87WEPR`iY-kEG7%2+0?rV33d%=>Db3*qZB6j>cg@Y zL*KVjYkSH%1pO7QLI4LlBZxjfiQC)T05Xu59@dsjklar76Avr0u8^4?U)FcgD5ccvs{r=9+l^=Fujg(BrJC|=tyEXVWUscSJ#ROeUnFpW(F%Y*Xk zVDorHnA?*Iy0>}l4bYyc)&Q%SPEy3{#i8nsdDaf!deD}$@{HHA#KpX=>d?hJ-_l69 zrlc|38)npoXu#F_nXhGXPO{tU(qVV|%uw`5UuyREVL}=@ig`7!r$Iq=Keld~El9_) z>+E+Isc0UrCmT6pRSsesvQ1NO{ua!fVf%C|5KIHd+XSdC;Fb+`xapeWU|7*ruen1= zjLuQrINPsv*mySsN{DktK!ws8B#mBdVi{zM$GbA^(m0thzIF2s>rsc8eAyOQGEwax zd$b((dXo8xXfGy$3+6nLnZoay%^^!xrhLM;vJ9g=Sfm8JXSi7}Npug~J1hv)om%w2 z(6xrc7yD=&;6KxZUNTZY6UI-+Pj^wRuetp9eLeMG#KwfexzRuK($s$uVHlTnOfR*f zGuYa@nb_Ucv0AT=WtiKpWY8uZ(bQwxr6_-|`;+bGepe@=4c5t)TA>Oz?>Bx3k^XJ% zXnM_yxBM;c>VYoRSlU84RQJS{^Ebu7CDUM@!ztjw9o-{UGk?M@#|s^uLDCiXvptYx z=ix|ljPn$AxuOYGsH@mjrXdHaE(o6PqS1{Q^pn4N%kQCdOxd*z#H76t#7YV#!@Yib zl-n@;dB?@U*-n`h>YD2?CB4F;7IV_(8~l-f1XapAJKr4`J7D-Hvmc}r#O@MBP)B>e zcqxh;{30SNK&tu)aEUev1e5K=dZNTK2Af@FT%B)o1|y4lkgNKT6xBoN_IHUJkkBUw zn)M>{IEetCt7yR}r%t!yfWEA?Fx;+X#qFb*Kh-g?)?5j#l4x>rK@}*MTQjmvOk#C?DS2U3A{diR9_%! z><6S%3DJsRo-!(S#kThs-8s_5RCi$yH-=v9-TB*Xj5iM53Osq0(4Q;D#K^CN&*A`N zS?*pnS!ydZM+NvUpEg(zj@wVh`OcqOC3H@}9>zgMC=-dClTWkldgUYc zzSF29tjZC?CyODcQ5G(YTR^$+`L7m&2VMsAV08Zg=jmhB9iN92J!DEyqe7Pve?cO~mY2Hmi@=1amh74dLPvda2}lUwSd? z1itMKZ!a0hQ}!=zu=ksBY&>DxFuy8>g^i#gf4*m7^V{W2y(0+!se%aOjo_wexTO#7 zX~iE-^saTVr{vGX?9XJ`F7keH>7NE?=IOjarA0^ z^9Khu{N4(r+z13&RRt1sMaE*6NJ@hqh?wTSWCP0`NQqvg?lhq|D3l4xyQ~0vu^e^qN}-@{K~Is@Lft zGFO7!I=4EfuBg2^#*xwU7x5oeJ!~mcG!L@&nc<^+#e7WM2L;w7+P*Okq4Q6OfXSG( zLq~yWC1^WS+ThL|;Dj@_YD`8+Rau@`7=e7603Q!_t+D(al>KHa-_ckjJ= zt?qnd4lg0zW5ID=-KWa$<;ZPoi-;o0KXg$XP4*8#UySue<51%|8BERTQh877R8TzL%W}&?`ZQx``Y=!^3kE`vK_?;Woam)Zc&*ZLt2ErA~)op zqQ#t=7{pwD4Z5s#+J0EHZZfXOK@rOS0Gl;JL-bzOTn*VT*LgS5ikf-)J&woFMy*Qabw$IIEce_F%owxi)p%PD`nm9i9%)~N^Yf3y{GNLDzHe1Y`doN4AlWsi z=9tF`JWH_EWtfbv&=A_dvnhjx?YA+Pq`lbIe{Sq58TW|h#f8ZPY%vY%Ty>p!BMy)^ z8l-yo>~Rn>U6;NO&x~uR6=M06wvsx1t{S&+Dwcjwue?M>VFBbshZxJ}FHthat z>+hDOpyJbU$Dhx5Eu>q}EHegaybG*gyen-mxbtcG78;tzEK6%@W|3{n-0%luN;_H+)ZuJQ~)f?MR?wB*k`S7FFbx({Y0GSvoYHT>C_LXTmo_D=FE%F= z`xTWH?s3YxWh{q%2OoC7(lJwuXm+x%A#PKqyO6FaF~mQ@0;7DuY=_!!QH~8Pv0qub z5!1<)?qDVX)05ZdQ_%&nQ=+MAsR;bp#Y(CGpEpKEg0p%J*Kd3b|=T1 zsrxiF23H)rQ<|vYA;b$EXd<8W^c735siIgq+35{N?*Lh1|6^H2H8U0 zK_%?xBTUf3d|jnrz_cXmTnw3I{a*Aoi;H2idtDiaW&T56INan6RcTIJ-Tf}hD5&i% zytt+*;dIz_a-XBM60zt?e4~a-%rQRp_B{W)F(Qm`{)hCKihDNy!GB0sx?`SoIZs?KftcgcNEcU8RQd%3HTtQGY;Sm|M?bb2LN<4FQi*Mx{I^ql>`1=2dA z>vSNcP445335xLBw5QUi)JGlQ`7mDZNWR+6w&TrN#UWHz8@OIvNSpBlJ5Q9v(P5OdriB)Fc|9g;?Bkc{GGE`TfbR-N?z#u)M9I0O7Z@p6(sta zsfXv9`5qfN>S|9)jSLQ&1ONX3DK8+1y2#AS2q4kV2I~|AP&7EcU||8O!_M-L9c0l} zStsO1?@Wk&F1x?ZO}DXy29%xPt*1P`RQU86=@{V^F2VK!$eH8uoaO}n4wEbr0(+5p zSzx>3n7aYf{O%P)5kB~Tf}wcO9G4G^5;ZVO?iq7F9utCSv~EKK4ejGq7Pf$2$s|T& z&B1i%-jQ(8&_pYq&n5{8q2Js+K)GwW&$IV=;R)bKueHj~@LV7u+@(urE*!IiWBG&w z6sQ`$-1o4h!~n+aDqWG`e+MSHDa%@^c5~u$xc^i-v_gKC8q11R;oh|MTzyS-s)#lB zbj?h!F^Fs+?JKR`l|TcD<0mB)LI(DJ_YF@d{I0kM^L}e^X4uB?z_UuD+Lnhy5^54^ z8a>Mg0u5+5a>y7O3U4?qa0(v^1|J5BJkjr?UC)!%-a1G4lM#)Dz?thp9Cywe$B?>yWe$3f${-pxUw zAuGw=m#~wIz8e|VifXnHqaZ>V3z2cBOHkL5Kdp0WPE|JBz*O=e1))RQR&@eM6>-nw#lvNRQGmSb zmv*=@P-?+XxHsQPRMhYd%MIQNPnQ5Eq_O)xCnpcS?P<9|YLC#8oWC2g^kNXt_bHIA zvEc*JUEcz^0fennwBX0PJx)cPge9Tey=tLYbZ}qa)kD*?{dBMCI9>7geB*-Sh2n;_ zTfIg8yx?gyG|jRzH}jF4b_hm{HXXAF_kt&pzNxXXb8+}U_DEIhZBl9Cf{op!XjYW7 zCU4#QPYEd2oa;wRR}l`JHf6r^sk!q%tssy+RyTI!>*_t4d&^NQlQOKCzZP0ixu`Qx zmr#AuEfL@U;(`^QqWCdOkj%YBA+M1VX*wjC(-7VFDf#@vV$q-V!EA%ToVHKQ4#o@6 z1Vx@^Sry%>QrFv-pu)BuXEqd&`B1UW!d3m}j>%x=;)sLQ)Vgj>zeW z&+BWw;w}>4X4A5sO9pHx!U?D7s^icL&+M%{>1X`c@->W559iOn2kt99SpOPpm_45) zB*rDgk7^{=(ud*^-KvXKM&J@IBkqud>$p4$wxZT$J{<`R7{mSn?JA&2t=n35BFK~y0#Lm zVCw$XxRn@&b*9nHZ1QvE%{P%>lUH$t33t4AgE1UD)z8hf^oslAK)xJfd2U5B((Q(A z{*j7@kp#uo$$`%+HUyWrl%`{QQFAz$C{pM*Q!!GzjHR=LIt3N9gF8w$*X{A0wt`9o z^QbP%z|-2e56%ZadVz8#XaJB0I*_JmSJEL5SpNOC{f|P5%;kigeg(II(~p&4j)VGJ zR~LENnNMC>%)2v*Y5g!{UTA0evMn;qXg9rXvGlrl?{V_B>3uzsAmS15#zsh^Q|HjSB0In)%)e2ilD%yN&|;*gudgDbDul|(m%4Y_murp+EU65eu)K$B*}8mAZ^m^U{Zb2aah(ki~HkDyZ*!y zD;4*}#WEWngrM?tN&j>P`N7sN+JzSaY0F}cS**WdkIFCL9OS!U;?|jITyi@jlUVF) zDpyl6Mor@OQwM%QkMPV9$YP>XUrO&1vv+t=B*^a6A(iywEEm{NV_9%J9eNu!cO}EB zT%|HD7s_GukCS|0zr}A@vqz8H7q7Adxk3Y};m>@8%h)OVuFft^((4AI8;qOACWZ%% z?}~gqMK2lI8(aoR`F5Yeu;uhOLX72&r>27n1pK9+m}o|y;k-saLBHuTtg<73o9%ki z+pl7*C+35$=<(mRD9N9Lb~CLL?k&<=GdsU(k8|@s&pvRMFqoUPX;^9s zzs7{FE)WPl6OiRft9$GAK?Q;5%8FL_882W~Lo(ZOz8dbgyz^Dc%D&BA;>AtDAi8x# zER>F{0h6OzV|0Tj{yl<}_VEI-Jo1HvuoajR-9F{m(^~m)5(gi+PiUftXfGu}i$Vyy zS(7)p@zp_;Z8nwv64iN+pj^hyC~$N%4jwHv-+XK+|K;P~9b?n%ihOV0!s2!#wC66q z26fn~8&M4^ZjVJk>*_+;J_+|dwC9@GjG&ebkeV@rA2YC*M&Ay{i+r-8(B()VXS{$< z^Mf`pG^Or}*e$IdjieP8yqtJXPza^i>-aW6%B5#m6ZS`?fZG5N(Ti8m()G`Twn=rg}%rUMJ`_T#OBhnhkp}>fS1b@&5&d%2;g2*pZ;V4*d}$xhEDKj3wa*ZJXT@gFb^wfGgd_v8A9E`DZ6AMs*>tMxO4T z3H5{X7Ej+bLRO-n zz^`G6_q}6k^`s!}=e|YzC3W8l&+J=H_Sv(~pQGJnbZ~!;>UEh`tmLx<=<@*ABu3b5NyK+mT@JqCu79gdZV1*E>sz>y}p{zWAwyHt`5IhZ_S zliIa6jJqyR<_S@r#v7C);KACkM%Bht(e2SGe9Ay8fjPC6?PZ}DyLZ}5S1|o2%jZT= zw#C^9Jb{X&<*auI;K~>Jta`ELSoFQ2+G*R}$6EERn6yf*#U8%WFf4 zG;(<|r=t@UY_vQH?Hvy36Vb%;7r9M7iun+nB-NGPwmYSvY-H_{%-ffP*2}yVk|)<^ z^KE;>%g`X`vab1P0tfvbrq0exM!22*8qvj~q$>#`$0%#S<`?{q{q zbPHMSfjXeqwxrG9VGff5n#ZuNotHg+YF2)JBV2gXQZQ={LGQf?=1F37Fn!RaV%Jf$ z)Xmi{inu-&sXp1bf-}~$M=R0e-jJ!b9(5X1MU4NWNsdJ4(FCpw%Ia~Jkpzq#x{88e8JnGqnSSqza^UZ*0jj@(RR#ej(&gi2 zXU{mgK2AUL80!n#tz`*@kVv6DIJE2Ed{SbeymG|+QS3iW;3Akk;vHmvNZQ`pm~al` z?1wDP6&z(A6?DpP)!X<((*-Kyq$svMvV2v5;D|NlcbB5*yPYA4fDXA*)X~D zsoP*LI)4SV-~fIB&e-!Ll#^4fa=If;wL zdwvKr?HpNTZrSoq{OGhTg){7yP@)sVG}$z&7jVuP0)Awl@R7=?@ab@Tp}%LFWaa|w1roa%sQdfT_k2m zf3dEq5p-ns)uYp^1#8X{0>YZprK4tE=b*K%1QgJY02g|$v>(4X_d3*-4i{VH6vips$FK9!LR)x5Sj6 zqwqM}>(FBrvPyIvzLcps&RIf|k?tey2RP3#s9Pp+0c5WWou8;suj z6x5ILZiO+c#FD=4bZBFF&bu2lL``%o?G91Qk+9j*Q~1_VNU@C84Bg9m0eq^eFpR3rp^rm5X8&ArxIBMFtnY#lR41#x>u+RaY8M#GM|Whh&!`<4&I; z4TA*z@%^I{G5hgsTO2lya^E7**Z!t>+OGu!gKmx2QM~U*2^OL5q!YHb{N${ZOla^? zx{O}&WJ1XI2}-FI-uhTdoA(RXfhZa%DqS*xa}FKskc3qrOl*VFNJ8qW?&~wlSHG|G zUk3=!MX-IJXnvBftAYs>2xlfwX7tfQNFh;Dh+%&+GVBogcu^eJ#VsD&e`nEaq+`E^ zC{QrS^-P5Xy@#D+ek}`kcZcQtz3}OPRWbd1d=R>h@TC-Q2s&oM&Wz z?_#STNwlr?WV?ba`frZU9VvC%v3BIKN!qPZLY0m`j;mR&I{n?s{S)E`IfA3F7m#*W zz177LkUH;hvKwFBss2(#LLJ(_>&Aj`*V-CjbNd&}CPewNh~$eL50im;@WJhl_&y0S zq~QALCsKv>eiW$KBt7^@bg^P37$z+;j=g++w|pT$>dgL?1v)nAP{!6oI*JU&{T6Rd zbVY~R(uXyiZ>L|WA!6vJZc#EM+bi5~cZk$6Fk0=oi)F*@(?iGRJy8z?1vvK3aZl!u z(OENGXHnTe|NXr@gsZP-)d>z>k&V*&NBZkOu;EEVBv@
    Z*(LJ9RxoPC8+p`w~G4jpYZK^YW|y&RlVt zOTu7k7`2ThWfkYe6}Xi+-soWBK)ylpG^|iUV?!h`5Bnxl!OM3p`G1;V1U$4^?#nv| znA+zI6&K}zMfROSYOCWVG?QvfkOR_)dX{Ln^thK@Ql{0FZv(^p6lS*{IE%YC0a`J8 zCgh`?g9NmxM7L?mZ<2D0##*Y~8QuIjj6{6t@hDT}acok9Z@$$j5gPY?n*N?X8CinC z8P|d;Tzg0~A&};S^!(T+;q&G$@`bk*;AzMzh>Lv;1ychfqyCw1ga*K<+VHF%OHVbK{g zJBW;1f9qaHg~$uA)fi7iIVb)fcBvlMfr17?FdZq&W|Au^T(R;XAlwCfQ->WW4>!TU zW-FQ(YilIr^_IyaS|zTq9!X74PQzD%`%rW0N~(yK&SpT=Cesx&+DR>RHH;PI4}p8yl-7%)7B2Pi7J^an%-4F2Z{0 zm?T|W1Vsp4fV5r8{gB+l2~?Ag{CK;VuO}khT`fjtyUUyyUS8Hv=on!)huE5v(>1y6 z^Seb6=$HWq@chSqLop*-yA;!iH*X_0TfG#2{;>CApxs44@XW$l2->Br;XXsdH`|+N z(<4@+?W=o>)KSxxj0Wl@E4CZJXI;$>bw!O+^$!~mwut<`x^p0(%@#bz8G)R%E$hcG zf?bfv<`IH{V2d0Tx8V0JY?+w5^sS3j6s~_Yb(qaO+Z&l2sl!~^o$LzuJLJ)%Y);Xh zkwMf^j|UbyhBq@m=B7E>*;|TOXQT z)et)#u9@eWEKC$F9{LIvu+TyHznH(9GBOwN{$7fJbO}|KKS16M+fJFFr=ckp!74K# z?eRo?w^!>Hn5_zXhhaz?$yX>-Qp0ty8u?5hxje^Xp6WDNH$EZy%mHs|JHh;wHn2@v zxcvkCeJr&xqL{(}AsUE04i}f~vhFbP!ZLEpie`5w=F!6xg<5IM`0LEC@HZ;sGxd$) zt|Q`y6Rk|Bpo5`G!2`O9qR&bLWW=1432FxQ%T2gQDwG4wju0mFW-%7UY_=V2isYS~;O%MrLrH;a@_GvpNEw09z<*}R z8_@8lZQ%YJq( z9{j?BGj13G$F{=9tZ4UBwxI^-XT;rn(YHo)Z54>w@4*b?-h<)UhC&lEN$>}2Y(6sA z&D)*d2@2^xJ5_1`@x21meSA|?@>Mo?TQaN>cbac+0-cr9EUpE}h(0;em~NKsu1 z?0D?6V;3Z2g@MSB4+F@dLq4U-y|8&uU(J~}l@485-=NTrIHR~s>yC6+eRb*)M`pJK z@7?gcw!m}=<0T8j`Z{%;F%5TjeKQW;q-LWie~2BTBjA)!5CLs8`Gz=!KRW1|Xz;5S6MoPZ z3Y{E=h_U}fV0<)h8wAZ}Bt&F7gD^|)D8VMJmV&VETxJ}*6w$t1jcF}ThfC@-!}xd# z6q$PjtP0x(Q#aS~x}3^*VmCp^8j(v$YeLX7(~oRca7`G4!#6GxLao%x1px{9WeL+7 zTqUI zB1goO=5Q-mA9$J(DeaHZUrCx~%JF(@Jw!qt3?d@n@EEm(C#t95PGOAr>-|*GkFtCy zyMAE?Z+ea@h2uJq(J7U@%-@tj-nYfc%t6~fgMV~%OY3elS(#KPyE$o4VrsH-C}5;X zzYQbYAx&{cyVRojJi^2@F!1%@$GBa=zFX@1K~uYi_5drUXWS|@Xf!&a`jPKp-7 zdsEb#ZT%E*vZu>kATUQCfRTgq&}~|2P}7oCI(ne@E8pI$6d(vnC<-%%&z^M_mc7LL zb8>_N3i)kDJIcY*tmh@B&=cXCV}27@z!ThS1~auOiZ<1o*v}F=^GUFNg4<-k zlPN=C(w&-(znvn~!cd{SbeQ|qb)Nd2qA@hau=ZXNR7l>(@62udh_2r<(v9L#VK7lFab2O`Q)Dgmzi%n+w~T$f;F=C8_pp!@U=0xW+&4 zkk!^>4&TT2e_F0jfq`_ zKiis_)X7n&C7kEnLako^EgnN|v?dj2XfiVvp{j`q5R}PF|JpKGbLhM))34_*+ZZPS+PAQxWhM6;@q09l}W;BQXg? zSLkNB6<>psqTn3VyCsO?H|d-vl==s2xf9tgv3(nUI{XHlZ8W?`WSqlDiHESkLjLlI*Cg z0hFrPUOd7L$c7^@JUu)h#*+%l<4@Q+oiyY`sV}XeDwxak0oFyn{hhBsz-b6BCvLT= z;>`BDn1)O@aD4dz-HA?$)pjv(&x@Rv7>pG`__iM%6`yD3IL+biWHBV4$2E%crY;(09kMsa4f$C{t89lH`bfVLn#U{Cp`- zsi~9jsIzqT`4gn6lK))Ux(KoWt-1G;Vg9;d_X91FyQ2VyGOrRR>9IUMhk8|nNNZYw z0`vQkvpe&Oj9m3&&EbM*rr*C15iXaHUF#FFDpKbZ)O;VQE2Irn^4{?@>KDjBIR5d4 zVS|Lh>zY^0BK@tkL=bnEyCcgGTB5fyl~~E$4DK?jThrHxatVFg>T7Ji`77HaMV?DP zrx4~Cj@Y+D;MW@=OV+Clx_D^K9tTvba*Pa_FwOIaOu~7ZfJNN$_DU=KD|t{wXBB#4 zRqDK+m$fD~>BJxqihysd*bTSq@jYb`yZI9bskwQ%bfMA&Mr<3xusbt5lGD&+ zSZh)k0^Mk8EEDkjh7Xr%TDqujP8s_wzUC%CssokHjK$bV1>(*PZAgLSNSpTRfPZ11 z4O-J&h#~aex&|u{1=Ks7M&2Jve2IyoT0ZNCH$TE1l1(LHBiUR$@0I)t|!`L8wbJFt(_Es~_$ecC|1Csb7U79mP6~ zp*%Vj7~)dxYt_4o+gDKAa%uBIVTH>Z#dWU}t?CY@^)uUS?9W6Pdk?W1-{KO8ge_x% z#hd}OX>tf-$9?ro5U8S>TDCs!vdw691L7X6u9b%fp8HDn4d`JMOA=($Xdo{-rp`8* znY=|Eny1lF+p-|-uieAb@-s8SrA`Nn{bJg81<>pYuq0eO%7`9I3PJ~awbvHpTMI(E ztE0a}hK^dtB+9F6vaIXWF%mN|J_)UR;UhU{b~mA2!f38^Ok5g`W=X6y%3pESR`b_< zX>W9&+)%ph!9%8+FLaGSYSo)q?AqZr?#hfw_t4QkJ^9_uXjG-|mI}R;hQWb>-rgCK zLH(XWg`B^=COD;hZYz{pc$ZdXpc3#~{pN64~b?+z;k&MixPaob*X`>&R!= zz4rdF`69nb0wn1(@xlMcH&}!LZ`G*~TOj@Y=Pc7prlsD>Vicgwuq^*DO9C{4M;btJ z$oXz8{7v%$DjvWxPYEEGwBQ)1hyGGj@J9j#@t1!nQzO9wmGvhm12TqL<}b*Y+7@ok z|M4X?FCxcvuxBqiFaMPHf?#c=tVsplKa%Lh25uLST^cy4aT603Z z&JIB6p`%8O5751N6M<==fUFlAvQ1D={t9!x%%_hXPjEN4w+>AxfQkVg;mWIeXlpo7 zVryyFNnQiP?g|`#TO!aMT4eeNK&0;M-%R*FT{OP*(w<|yRe)LKpYn785Rf(V`!5iV zfPe)=ds7Pk_G!kzqAIGiTYKz^~Gj=+00H4vPbh54Hob3zuLxu@f^i#h9>x2K?JojSc@$b>$t-|qzHHP;G14sN{*W2_0#aar@WP1gF z9ltQxI4SJ#{T=habLE9KWqa-v_us1T0K=Fm?7aWmFXQwtO{4#jjPQDT1O7dW3bf>? z=;!RGR})YLn&L$7Ir@rb`)9T<38!DY_T$S_-Y&p1sV3bf|IV6SptJQdp-=uQ)CyI+ zG_9h@nCVqU`NjGqUuIhFFGJ zRhG<>2fD-qujQ3FvDNowEOJsBqyK*1OUnpf3_6l7O!igT+UfA}nM$PVDvEN;JPP@x1zp zmr14bqEOA`Kcev(^RYy!gc8;ZzEwVc#N;#df@)tQUt1&A$G4d`=q)>c7njZzIgj4H zx+*VNk8&AM={ClHy_#&2(TTv^;{Eh^Ur(;m){_9I-Ry{TFjLV#U8WV2oE+l)ru*ub zgVj*&j65WU>7Ih#4E28wDmqG4F3@XQAv`3F~bN z+E^g|aDP~PM+cqKq{+_CF(>=T#mn8*zeHFc`GlA`uhn2jhK`Jk%yW;;pzh6T)Th5U zk>6z0@`nx99S}oru-k&?u$a|XASt(pAZUGpsE^fMft-ix(B?Ip8F5#PdJ*`JDph5A zT@TfLfM%$A|C0VYQM@hDfo8N97sHJn6P;!}6Ijj1L|bc@c;MsP9Apj_H7g=ZN=h#6 zfk8l%Ffx_}VAmgmFIQ=S1Gz`%RUa6IPoF+3K>xqgoShBE#KdItFcM};X6@m&qY$a{MdX&X5=H@7r1>ZrVzo;w!vnOEc;)74bs z*NmItSLQ2VJ)>UT-=8-_XQbP8nul)#G;;rDaIw)$E3&K^)=M%~gfsX-#S?(Y9)ec6 z<>xOSywTU;f;4_c+*M)H0~LP&E`bvQ6^<~%>GRC<-Q}m&3`vy z{J^^hovZ)whXIY57=-e$R-cm^7!UD>s=@N_8oXM9kRdvuTK!Fc%gi$D6wN=`MZqF)@H5D>NkO>r^yo6Unv6HBiY&Q z2~Nt6sw`{!smUW~V2)qWRU$wDO>1UJi>QGI%%IFS9{tr>6215wBK2bJSGro$bnhkj zyKMBzFVm(%TGpRBTnV@yXa$DgA=AR;H12bIfdrl%oBbLdLCCm1v$>XKC+JIa?Nc@m z?!I+&c>x!8fTq`8!&6(QI7m2ch;h1WN`*TYO8fB<6%P;bGWv{v>2@ye%?!8jisya{ z*EM2M=*Jlt!s{r4kWu!9^~H~-kCJ4Q?dk8!rdEfOKVo3&_(>eVKS#s%$l<_gxf?Rf z$i3+-#_0?WK|CHUSTSBsSr>6lct0}svy%`MelJP(6CEkP_Nki$;Y?ULoUO5UQhkbp z(t&0&)4xlGe{#Lxm{)4auZ2_BQA_)6bu`OXHEXK8(*~bS?(zL1B6LT3$$M4yq4Pmker0aa&vD`xmZX zcvzEvVh&qmv_D%9q_9}wB<^*+?^`#wI!9}1;E+F&bKr!h7rcd7m&8NHC^Bh2SOU6) zR(Qo~U)zuI&yKz*eFwHj9pMFegn3;&ex9->g}fJzdGW}JxqRx(XT#n!&S=u514%Dy z8r36c^NXMP9@gi#KVDuLmpwo-_eZhwv*bJJs2!(#rPfIn z-1(yQV)__d8mnA05l^t0po zq^!&^H6l+`^*dYv+0YuL5iK4KMkQ6g@Ts3&p)SN${p(4_jb%1FXnggws4JE-VpYp+ zRk0RpqiUy%c&ptNf?ZO3j*Pzs&Ao^F<#60Y5IWZMAH@voCza@KXX#QF)EM11EZ`6ORvYgOT;L&f-+1wh#^ZE8Z ziGZ%YdVM(Ec|YR)I9x9fwR~($zp+^1sX3tC-@QDdPx=-QbUNG}ZN<2KfllDbH0698 z&Js9GEqY^qgNSI?9ePE1zUP_@LxC{&9#?JbDBs|nhqTtUd9fUE0rHpML3&044m4zX zmM*Llb){Wj_Z18iy&|vCK|w@XaXB(?lYk-l`UcR`co~x7eSM?XAqjjLlJ5BKg4ZE& zfSqm4^5P~urhm$TDFoAcS`jqRl|ov6uBc$S6wDA0H%G;4tHJY1EXNC+e12Bx`Sb+Y z&g@h2?7|!>pAoo_)XXZP6jU?7UH4YeylqORI=vYS3sg(7Te4n2#feE=@kEZi8Pjjx zgUCS5ygj+8&G5MUp5(}=RE{`A+!n9Y)ue2*gy6jcu4-Fyu)g>i-N%)XT6_*t(Zv(& z0C8sK?gwX$?P2Y9@Zy*;5+XBaqbXH9$=S%fal69@XU6G6O!!vf0w&GCnQeKF6IAVt z$UXTM@vga@)0I;KRU_`Udv;;!*}X8EBUF&!{jsvxeu=}pc4?MyWH`-x9i$1T^8Wqz z&2i^Uz2`IENosV~%pUI!27~27g{A{_1`5!`dPch^wkEOSY?!u8l+OBVd)Ty2gFC`+ z!%j6PAk$lqN^ODF?pSQRzJ!NbYX42Rrw1m{hrW5{IvUG}c{wHf1=g`8V5hx4p76Gk z>K0G;2}Isgf~4^S2kCp_aOeEF_-vbHn{MhP+2zKJEsY(mF7jY^sERoWKNBiya7sH` zMQ@OZvUBkMBf{YbO101f%KP^r^GNwI&3`s8=exeS9&_khbLcQI8q@$Uw3EUxBafAsedtElVU%{fUa z5n0$fkVcHfWp*YAmaF^{yJRxL1>IMX7WXZr?>&)Vos; zx}QN%Gu%LT>Xy|CcVRy4IeGetLvDkKpm#&aA*~i6d9|{Og9OQCLDBDiexBHMWh!IK zRRK*-CbD|)qj%a#ulK>e>GMLp5v8!6R$6FKCK>C9HC;kGAMRGrlavTH+;56y50S2V zwb0YA*oH(b*0&$TzBjae?ba1zxFp0OL&68*zI`!QQ+cCyQTI3+Hi+a zyv2)DwIcugcBFe|0jpc)>=)+sht_NXZG>~c#sdKL9A%F}Q5y;C#Wn_XF;pm4h_*$kN%zV$@m;BUa1tg1M)116Y|aRk#n)9p2ac2Zi*lY*#>6ySOz!U zNEZ(hR;bNKjCX_DI;Aq_{darv3eV(k<(z3R#op4ixd|@o2g`*pF#)errm)pX;EBecP0~Wgw)dz;@xNwoJ6IU%uEPn5nmcC;lwc6uW<_NGUhx+7(ejgAn#> z2+xve54o9l%%24?LWhZ`H;$1cc~!Gj(Lij{EuKbdl@xptpNvrl$lNhp5T z(I{p>>B9-PDb4vi9k4LNosbs0L#SGgbd&uMytp*r z5)~~gD!x!PJgYuCAI6~cAKk&QlN|W*vmw>@ZM_#fH(ne;l)#s^cj$e)ccK{6JL#=# zrtpqN9<_ zY)Vdz6#HP|ZENMQ-|c-ohlkIN^NGh;rXxG3S6X@JaBJes>yz>2%bSkH>O$9*lFRHi ze1wCtY4IG@uWVb@p7LampQZDd<#eJhdw)CIKS82_iixE{`hzZho%Qij+%w;y5RXKe zk#2{zL|mvG#f*h`k|p2O22%Nxv0#?MOn;Y>(8Q@7lo+S8y_=b@2an)FsLch}9( z45!Y<2{5G|&k3{>*bmno@SezTBHmhCzZ}K?20;G_T77%5*YxbZ>%T`fZyj{NA?vvz zV9Qv8G17Z$H_8Q%9o6>%gg$F7$Ou***DKhM!bVV$y&+; zBSxsQBMl3^B&+^77`ScrYAaLvjA9Z;14Q@UBx9Sn7ia^1yhN+6tF&Xhb;UL6A9unpsB(?bP8=o_p@xaApHT0y^qLN)_M3l9| z7Y{A+C@Kn*smc7&kaR6`S$HNsBIP;?u{pi_Qymjy3ya%~EM>{_p$0c}fjO*w{}KaS zD!CC2Qzl!L{l!;a!wFit7>DyU82>6r=x3Og3kMXx%ux4WTZLhMr5b}iQupfWjq6FQ zyXXA8^4FAOZKjrwV_oMzbW=wC5wN$(FPLcjZn#i5Y1V>Oy(r()kWkEJ%7@i{ECY{h zILTYuV97$qpX^cxkBgh65DgGtM?*tL!A7SGxe73RqpVZv*RreT*(l3-I$Q1@ANDIi z?FrxI#^U~6X)-zADLD$h%-NF}Hzsq1;Sck#GVHMi_l1f|#CR3hOi~mvjTz9I%G=7s z_+f&YRf4Qc=w8SCfcHD$cNn33J)H8AukP)I3=Bv)J3Ds_4#FW@w?AGT@ydKZ0TKXz zqn`_qhdSe&C>v1wsAJzryqFiQ()_!h?bXlwut}X#_(8-y0qrB=mBFp;idW-7o&=s> zs&rD(LAg@jlv_oMlPnSLev#Fr=AQBdgazk{f880f8ua}x*YZicRrV}XKlQY{P!`Lq zn$=;Mua_+u+4VNI;UsjoM+|5G7NcGpb>-o~8=njf#j-|*)XMbDDmr|XaG2t9V;1um z_$)nST|C8ZER1K*BtZ2^6|XYivMTTD{=6&pF}f=}4+qgkA*9NliLvR2tI?mG#bNfm zf;|0_8aeOaStEhG=n_44!=B0f-oM_Ybg9l zB@5dUxCcQx^KHIhRMUTu9WU--^lQ-pR*^_OA4(ql9qOk@b?2A?1X9pMR#b_(7g~~* zhP=qH&eN#g`H+|_W|xu*RNpo}8!9`M?)ShApF0TKk*$YMDe2X;8S||ICSSv3ipEf? z=?ts&3VTb@P0PSt`U2>poY%kp%LOo;>UKqi?}*F;yYKOX&~hrEp^A$W9k-Zvw$Y)2 z{gh1PtA{jcZH*xo?1)fOl_^muhlRLf&%`z^X%oQ~;fDLA{16+9X+kXBt};!5OVtR` z<D2&rcinJXO>iD5}ir#cTL`=;mS4XxkUzaB}ACw94i z4nF0wmdO_N=SmmJ7F0DUB6Si&^e}qsp)wxTXFce~E-IJ?X&{woI1UrL`xry9fArB* z*ISKzYHJGR9BCfbSK-y=CnvnV{EpTW&IX?fNxh zz;6c<0&7aAs4&4wYYhtx<8S7|OvGl|GA`O{XaKsY1L zWUH3yQ?^!{E^g7UciPVNd~CR>p=76aIeRF~uBM2NjDQz|^61s_*h*a4!$MBzBEk(= zR1ElL5T_}pXd>QDDevsp7C_|iKK$*J#y6}>#?(*0n_Mb^H51#Y(%F^`+{mDHswoS~ zx&FZsAt#}|7y*LfEyk8K8&r#u*os7U{jLV1T9@D8flp1iTE%vWps27q<^EX+r>W}i zPplWT7$*}vkQx#YuE~SMxip3XWvy9es3PG?(HjqvAz%`M`}k!une57&t;;WTIm@aW z+Pbg#3y^Fz2c&k1~>h(8MaAg_XGUh`(oP2}F;Xc3k7abi0}58ztDp@vo|0|}n^S#Khxe-=%IVVw1V0^h_So2F)Xjm9`kNM(h>;)R8# z|8?tiCZ$0uPO554Kr3g8=xz~`E|$XE)oToEJ?5OKZ<>)1C-M*w2dkMs5Lql_*io)8 zd(~}Y=_E!q>(Ht;-SsM_Ru$ec3em1V?oNX#_;2tDGI*SL^u$CaR+&LfN56S+t3s)e za5nw}3*!<}@nFN9^Uy1jWv!2$PkNY^6+b39tS%9j+t(Ejxh_k;+zG}IFD7D?>aVlS z0XVu)Da*GG}z{|I|3BH0)W$ zVn-J6zbJbPpg6iMU=X(uoIr338Z2ml!JQz%B}gE6@Zb#Y?j-0yaCf)hkig*X?yds^ zJG?L7d;e~2)z(r4T{U#yIeqT2dvw@g*e&DLc;4(oN5K`31xUt*D^xd}T_opMhIsdZ zgr$*ekMvDiGv*0D@YM%tY1wQkegT%x=&C~0n`)B@Ajk9Q;6 zJDT2Xx<1$RX<@o~7zf4Vu%K-|+j2~A@q6&cC$0cU)*bh4j@W$uit!P=4=mj=4rK*z zO=^OE^GY-k`D@g<)+1-Vl$4>!B5EMFBqSuc_2C66-wq#f zU)XPS9EU6r;%%?c!OPp-dl81+erDsZywHEk0z^a2j0mc5Y_D%7sLLcU?^s1`tg!Bk zdg5B*h{EMoIW?>rn3IWEw-qW*u@ZD)C6vfQ`c^eisvY~W=lYeN!?x1Hbrg^MZkYC4 znj@R;X-B!$bLF2>Uk6!quH%!B5(B3b>2yVTGZD688QZr^T5HyPz2YDH;u||09fZ)0 zt3?Y>vf0_jRovnU2{I)yNoCV;ZbX3i_?SZWNtJYTY_?D8q1M4^BzAgkQ{b7>oL<$z zl4Q;AiCuw=FmG>@fG8*{p0amHS=IyP0-v6HC!47ULReu^68eM9Mtt!JO%7k25c><> ztntrShF&2eB1xmU5Fb(F$hR-E%JjNedwgPoMi<~50ox?@8 zb{3y{%xvbbUlC~y9wo9x<&5%D&yiN9pcukvk@S<=u!Bs5U*7rUmD_q*S*SkDClSMU z-9Al|Ayl}c%Tf^t(k0|fLTOw65Dz$IlDaoSy>@tZc6Niu$CZAt2st35GyT1jhX-zxSDSC=_0fW$zC26~ zM3tMbFiMu3pF1`-HOU#~_Yg5IfA=lJ$2qDmFIVzFMnbAAv^yX)>k|GLVrpuN^YlK9 z(rJ3%bCS}h3+XxJ#~TV>*KbvaH*7uV=)tb*c)Y5hmmj;%R(b)kVX(39h0>x-m}5W0 zI*qJ0N{ML0+UV96^K%he&OF3^A0^wS&3F|Gv?dBdgfWN@iGl57f~LZG+w7;kC$5W> zy{xfufHA(YKdWt*@aLQoq9i#^l~>qmjFPxPN7)G^0w+>CiZ-sg!oSh$f~V(vG9o-a zF1()d3l*u*LUkQ#`;jcGXY-TbV(}XfpJDl7*uxZWS>C9|+0k`_)K24=5@tVm6;28t$ifhhSbr9oE3^M{IjPqqO15S=;+ENG zms}XNR^Zl?>=`Bc{IjR4OSg&h0742)Jef35JCYOuE}!Tl8Mv(>4`xevJ>gfRrjmr7 zTo%W&FZa3xysFq+XkcGlTFN_#SpSZ5ra^58BzT2%Qo7xg0w2-LA)a)>D%Ka+$Z?Sm`j_tvl)^(gl2B{D;?y{Q$U$zOnV@N_g4IhH z9PHnz$`W2ZeK{ZqR;yz`fMrmmZ)UTT{Z>lnrykA>3k{8JpLnrm(Awkr;0=Yyt+$uG zfONr2K9q0&fVBa~nA7xxkr8xv*8-MMlh-w?goq%HJuoz6EVkl{;Zup`XFXFcw~Vu2 zD;+^M6vpAlMnlyuRASQ@QN8)~t)yrfAB=d4-C%J?Wag1@*x%9(x>C{3RPi#xKU;Jy zVQ>uzwg`6Sb1-kAizX89j6V|0GNzuzCguHrcN+AaT{xn$5)lDmr;e$Aa(JgR*mx20)o zxrD{Ek0CqbTz=8dNnEtGx4L@I2@Pdave?xHC6aX%fsql$63=0(?4ex{lm65V?DX|3 z-15lr)urCg30m-v=X8RTI1Qjo(7KDG%&o`pFf3c4f=LM0Zxm)r>rt&C!VYHNp98b6 zIlN}sV5c`bqNy)g(k@AbW?A26t@NIAtjcC$-S7W_-HZI&Eha2eQQGW;@1I-yZw6*0mO<^US(8x(;+T@TOz_Xr#bF+X#HB0 z?#uEN4R|G`;eN}Nx)@Ziw#Bp<=Z#EFwP0p9+Rg~cEVjjFdG&)X8VAj`P+xbk4qU>S z3W*)^CzgJ74xo52P0#Cvpz0UPF*DMTta)P_AdxKlM(XlQiCA_ra}{ zAE{wtT5d^bT>+L@cO!T`R%KE0l`{|`i4g;_5nEgJs7*`?2Dm8ic6`$lW?fgQ@|7Za zv?ZYe0CSVR$X{Gwp#UO7FI+J(FwG!@25^fY;lz=Nogt zdgW?1E=T;s2Se-LpQbu#XGddI+P7W1yi$GP58Sy1B))TMT??)CtO+gh>ej!pguzGx z5}l=>6*uyy*+%@W%P%6Q2gqm_R!VJEZzCW9KV?7NB0zL>#|pgo)Y^aI`2_5GFVW`m zX>J;sIHsx_-JFGK<1AG)o$}VsXaJR6VZ#J{&a!T?_himU@mduyUW{pLSW57WH8UwK z3!`DY>|K#vZS(1J8x%C`QJGl@G`e`W0Mh~hkT#Q}>-9vaTB&6P)%3>ix4cD2u~1s9yJs?K1;ji0mn@*3ZB!Fsicbp(j& zp+UksdAzp~RGN!8wnRcFFPtdU} zwY}*^5O=C}iVxPe06m(GV)tM!KC3=?WQNy|&DYK_oZ80NI@h|GWNCF_{ZU$drBoW= z-q7sk);&=N9I`$dPAkmw{4g)+>|L|}YWMA8>wQ-lP%dYRdrVBQuCCF&M6>!=cLRH5 zQC)}*mo&3-5F>VI96P6#Fzr!YT*}y#h4S+D!EgS$^1H7FqwL+8Q7hu%4`kf99X|lN z{kpGOYtDAN`f)}_ct?-Nxs%64yBb2}bwhF6la*S5n(CW}qK>J2-cWI&%RbI|S0^F^ zX9l3M0#}mxuQ6PT%*#)E*&RzNsAZFTh~l4845CNuU1D#fNL3A^qN2vz-LwqXMU04u zhz3oXOxuqHbHX98r0?d|c~|or!!6kly$T%mE-t}U-&dfos*(S3R6VCXw&CvLiFIUS zXtC)_=t#XxUtC%{Qsi>IN6v|tc;AP60M4LSR&x* z+1anck3XqU(po%PzU|?`ESbEAg)~ZplCy$pyJbGojNGAq{(4Bj zT%y-4p{+RNPiS3o{H2YF0RgBJUl#m>X>5n+0_C!K zYx#Zbv8y>rV!og5o}?G%_cjFx#yKm>4h~H-U}6iugD(ztzni6aNL0=ynT#4?7`hNpgrHZt!{w^PK6Ccg)(b4mi$Ve{QEJjmPl zFwt_7DAr5K{}BuCk~Ab*O0?HKyAng5X{_n}OfG07Uw#G4> zn38SPSJy;tCPON#t0U?vJArdQqZR;5%24&-apk*-3kq^R^fhjQIG@ujey<`}TmECT zoX5|#<09NMVJJF+SijQBWk$quw;O5Bsy_3#?hr_W6YudHY<>n0wf%%s^9CR@%69)! zfZ~v9hD@HsF!J_)o=?O*#REGHJAWY;j{JG|=F;yxuUcU%+iAY~Vm3j%EyiG;23X}> zeiC;w^;E(t7?N>clk3NN3Y2A@{$pa(`Tg}sG171$3u)d(|~xDv|>JJm5$V5S1+l0!zCd$^q~_x6@T(J z@;c!?8n>hKm()57^2>P~?iE<%Mhi9;G+|>+Yf0~Y>D>5oL5o8I%ol~-MU%8R zs>J^5mG;+Y+y=v4D=w5lKDTa?3Dg?gvG&{@>9uXhr)?f&1tUALqsjp&5+TOBfSXu* zH{vuk$5mmAx5l)KaX^*ao#;U17aX%1Sn#SFN(MtOQri+W(gbLWm;MO)CT_eTCI*Ta zhur4u56K;Z2SQ9{fwHMMm`7Lmvya_6ZV4*yd4h$BvZ$%=g&=_|F>@QTjTPL1Y_IUn67Px(pi2UBjjXon^zegB5LVGA4?CZl( zWb!}%p;Asv7YfZ9(V}^Hl#D#k3*O8S$sQCWo5+v$<9pfu<@g>R*;zPoJ8~p%4qRp8 zVpRci_!pI`+KeWziVi6kej^L-$tC8m@BL85v6n%7L@li$9166@K=GJd|KsPg=4DE` zC==;F=3!uqVi~tPuen?{o$}FG|Ka09!KY83Fwe5mf90bcl&j%!LNrNKqo)VJh=847 zf7och1pCMD%;_u!*AkW>_8}%iqIT)a7Xc3Fs4>g|@5D@SMdnF&unN{36>;1j!1-Bd z!(50DAxe^$M&<;gcV3W}oI#NAZ0>buCL1nQN2}ettWdNYc`_k!VBWpOy-ocm zo773t(2o+4F`ls~1Z|k%n)dcw?IMP>SJ=jf-1k&3-;ERIu%A~51#OTBWF zCPgrl)m`792Svmq`lCe2ECveSUQ=3nAg1}_1~DlJvtI$rcY^vmy!vOE&d?g2(Tilj z-6Cx0F;Tk`z!kIodw&j2Z*GV>;eZ!PugRLkT4Ahs*pyLg7>;Ddo+qZLET!#w=L{3$ z9v9Aya#S9CFLU~bg%~f7SI3I!QDU}H155WWz4x5>%pzZO9iglxt%NSH9oP2xIi+Ag zSZ#J_|En@#8N3-trOdF29-$r36N5=4$`jsXql7ZzM#$*|zGGD{uqMT$(*CO}&Lv$2BAuoQ6)Nh(rGS8=0f>gvgx?1uv zs>hiQ=|;L(4Js2bJ28hlhS%@$)0-3UL=(|&db(8aojM(5X}pv$)LUGl2H`O?926F| z>0S~#J6|$;*098Q#-KMK(ZOtv9bbdG!571X(29rs2`$o?UMvV__FXWOi*>AY0Q&p2 zGdF7CPb29_2a{2Qg({NrWtwhuja?cdu+d&Q8#yFlgM)PRe6XJU494HMa7K>qNOY#Y zMfU84{~^C!^Mb?63=KGBSDN~s8A1EJ$l`p~?eX-_>+A^8uK!zwb4g&R0+LB!+c%NhkY@&Wl zT#=F8p3!s_D0g$jU>t&e!5f~7XXdz3PZXlZH#~*LdYaSpZF!ZrzjQ2lvvY{Z{mYcm zT}Ni|GmxpVWv`4?jFL`Zg823I!<*MB0r{i8ubbKxs2dYw1?f5!D6VYKAj1@oQ3H4V z(z#QS1@P4tMbLvAd{}FlJxggiBe6xJAK}b76I{(4)99p6kAPv~Tb^O!SiuPC^Uam- zUa?45LByZV+3~>av|d9$Ln*Bee<%;e?K~LndOw{FLkXe=RGzBv--L%6NI<(+oS^|9 zdt+H08FTFE9_bB8{-~N9^MTdhkzLvgC_{1Q2xOpOv^U&3kC`1*8VW*`XW8e+lEa~> zrw5H3mb{VlS!USkFFO-=f7$17Xz}Ro0~hu60`U&qR!r1iK?+%a?pQs!lQgCUIP5!> z3N4&hd5~r6h}Pkot+wOX$#oJHg}VGa;e`1tQg{HDdb(hlfAUuUsb&*I*H#;47pTNq z$v6mk4;Bu81Y+Jz$I;H`>x*nWGGz7JXX{OBto{Lutj#tljAPz<+i5!_OFD`Rsc2GIahAT)){3T0?W`#*G+rEyTp&y&ukGnCN0o z*C{!T18w0%0|#*T5|fhTr0kuXx^v|cIKIwuSXPSWa(Ug_GbvUbPtwK+`2#$!L4}2~ z&CA{0i1-XjbTTd3D#%bjL{xdKu=NIWm@?QrUbKldjxspYoxhz1QPW2%O7NOX-q1p5BV9$g16&;%uWY~=fRZcYCji^G7_g8UzPxKw-_DHy(%GhCtlpaP%`V2`-AED?=iaLEWR6*d~!O(}cu~Yy?tx z%aznS2RoD3tlD=rBwKc(qjOn8lKtO3N3 zVr?`ZOp;AWY`!q_a*|K&A83SM;yeVSphZfQybzo$lR~zxjky!y-#SmDc12+S|5}G9Eq-Q5V-WF+Y`F<>#8N^cg&+$tGfAI z=n8OOd<pL%OgVk4TEy8$hWmzwgzTseO!^Te1d0Iww_JqnkcIf+Fj`od` ztEyq&Zu5D~t#UaJHdI=wu%d|>T=_b+5>CyFJ&w^^o}^F>+3iYT(1iqNK@8D&&4U6m zyw*%0=5zX?74)Z_bL)&!Y5l>X4GyZs#cOioe%_^Q-)H9WC`R3a^3gndY8-#u?e&{U zo!f-6vk^JfCkdm1yBNeP5sbxUJP&BCp`LTGnr+5NC$l8!=BQdCjmdjk?{ zwl^T$!CE!cc@wJSM%KM(0VnY%GMhBWmz`H8tHpi=$a(5?+)NBn?R$WAegBU&{hvBj zQ5qwFSVy!suT2;Dc;fWokL;lGsL(m{L!<4BpnPN+`VW0@=Sw1q-b!9w31H=!JEqi* zl~b5cFIeanAHR~|nZI{NG6GH=iS)F!iI1BP*a8CsiQQUK5)(y3cf^s1X3+G-dgf!I zqx%BU@lSh+3_-AfOS4uel;%rxqsv-Xkm`I>QRagEW& zx4(^hJ{mNVu8017c$q{D=)blSNpp-pVfXT~4RJI8(5g6?Kx4d=_DwQ&j_A{ZG1S1c zP}32)XoY?i3Ag4xSUBecl84uhZm#}*C}VUVUxrf9jJZ5KkO$)t6CTPb0J3YnrIq@g z_EoWVy&j0B9JrJRtoCMrYjI+AW=x!CU;tnVDt+ssl}Iy|=z<6J0*T+hMN>pa$hTwg z&BWGTfbLee>1U1seB~3>0wS>i=Saun9`Dh*`ajA@qE;hYIey=!YfLhmgm;djr?OqF z<#fgTl(Hwz+sK{frZb?`H?#k_Ee0L?FLw|>zX*O+P?1@-5Y4h>i_ii@eb-TAWqnA@pwJvaN50!*hsUi8yXKs4EAM3t&N&A92OCswimrK>23W3|2T z22ylxmWsf;IdKaz9W1fUP~l0C*<$Ivx#3nlr@wQIr4&SrYy~7qU{(I(X>%Z|$c&-E|z3 zv)#5o_>6?m@C?1{wdRn$elFxrXL4>#n{yYF-sz29K?wHlTzy#cK~VQ*TL-Bhwn*5o z3>zdMljq!{jBmS*{+BXF6%FJ=c!-x${o-ei5uXjqBApsDv==WBB^-k9QrHbq7EbKx zgbJogv{+Pp4ri-|9Ko4@u>xgEQ&O1&b2Bq$(WjP^cD1(-0?zw^pSy#ZGTs^SbG7Re+U~t(D=45FktFyCO>R`t+a<{O^jt|B$}CZ2I1g zsp6gH%`!748+z#ZYgN=H*!ExPL8`XxJ$c*Z5Sv zUHc_0a);4n0!+V2z9JuchXM}_SDDcqZ$C`2L%vju+twnj7d?T87Vf=pSce1rVOtXI zhyVD6!Rb?&5}#J~7{KjIBBcus@y_lyg?>22#TIA8aSY4*?-Y%->{u&VTk*ODufE)H z?Uo-7?N|z!yKf!-o+)LUBgs~YcO|p#P*kJ2BvlI(j-hXCGs~?UyjUJ$3#F-Wjc3>- zbP8<0vOxRYB(I05QPuX%xiqRb91+M&?gw#3+ILFwUzWPLvJdCiI>{31#WrX=(fD5G z0 zV&va<_5_KHV54>@hMaWTn^NG7uq*Gy{$X-Wgg@ynVg^(2pe}VxR=IhBm2#o-H*Pg- zp=8Tyqtp z)_M%nrt%>OS$bKuHk?S8?vrrJr@ya-5m7KXPWq+=Fdv&R(b3U|Ec^!(7*h7gI4G%J z)aCeEBS6`~_+BS+R9Qxk@fn@BGB9-l9woA+3_mr`f&x&CWGN9KNT)YAlP4?Dox)pA zu?9f@LR^asa?uq*M$EQ_IP&PC0Jeiw3bHnJUp9$8>Q;SRv5TvLJUN=55ah05`%K&L zFX!Vw)uPElqVKE-yi+P=!vpQ)mWRy;l&#r?=Ei9oCC}p8H(&b>3>a9*`5wV7sW4#p za~%KRU;%+U@2TML+qjF45G8^^`0m>2b8Qr!a~d-6j{0c32{Nw7`k|rwsCIma<4LU+ zTz6(MIaonL(br1~F2cCk+^5;(^3gRNB2hSQRH{F6c~2hT>FBaViAVrAO5<L=?yh&M2z0iqPccF6)RCFCdwMkXdJHUZLm{>!TVINO_AJ=c|D!2{cD{ z;Jfr50S*$y9I3%1ZiBm1yyFU24>rm#+v(T?Tj& zN@GiI-(e;A{acu-wH!bNbPU{dC=khppt)I+ju)5Q;{D5r4nClY zk>diSUOrOhEFQ4eCV6+NjmU;AiP$Ugi~)qpx>s?o|EHp~lJ>Jgg!G_Cmpu`z=pRKC zFkDH#W_ehD5eof=*o-%7=i@-Dd4kn50p7o1>06sscE~M(gQhm&ZNAOvJR+<_-SxRY zYLM@f2*IiofcG4II_<^|&%^5WK!F_kfPGjT?THXQU0q#!8@z{8d4>d{_d6oq8qASq zl{pfpa*fV(->MZTeZIdwGV&xJQC1-HTLFn7dObmJL(s%gTd*2N;`ks&uA}I7zPKZXB!Osfcr{f*q5EXZF>Y` z3iw9C63Ie?E6hepzqP02R2v9 z&bZLGq*H<4lCkcVeF)c!Pw-Pbz*mgxK{^kBfVO&}{9-uvU;aqmzDn=U*FYa?f-p|M z7L^{tq8s;u(=~3ST+U#~Q~$j5W+D)jr(lAu-je(ya@f;HDh;RFbDw)zoSieu*_o96sF0^)`d!YBNzxR)B(`AY+-!CsYNMh7&fbv0)# zQH)N-H<}2GL0*4=wzG07^+%(eyF&qJw`d9aN)~jAwbr@?wquWN&Y9f{s_G(l=UTPt zU#+W?>DQ)|my7neW51mb6c(DP3zyl#t2f+hKFbc8Nx8leG0{NH6#{d86rqYVu2k>40ocG2n3v0T_HEBm$>gS(ZnZ+T#g~ zH*=B#_fllaD%%C_en)G}+ER)Ys`JhlXW9~4&X7hziuHbd%WH6R`14MPB9u)2>MRHH zoqqBSYrwZxl3oU$nE+jP))TG7kyb;}`d^)e_Dj2z&M^@p=ZpBl#Mif?++0$-hJ&*~ zWvFap=_E0+&?L!Sz1qY#{|{c`=(^?fs!)lVdNlxG*cLV%5;+tosZ>SL4-AKlv|0l;D0?BA^aDFdAe| zT=B~}I%mg(-$7l(7315%v|`xt4x_iHuW$_JDuml+o&cZzgUb1~&+JHu4X`n~b+Ou1 zp=K$@ok6G28Tk|1?!suXnyapy$|I)OV>n%h(tb|^Dz@&4ewdg1_DG#@eRWo$d3j)P zE8}5!a#Zc+=X~p0#jw+YqiyG8e2d3Vc_DwauPfgsB@#2S4=uzunf|Qq*%=>^*rMv_ zIKRn5Y%INWbh@b2saO5MLdk+XrHrnc_ z0~Wf!0|`81^r`d?vWd-j;B9n3;7ZlAZkZQ>Dw`cE4qZMQ6Q%}aNzmY9oVuYUBhA?U*z4P3fE5Z*~e^WMz(1Z z2fnR)wEeZ%zP_i!nzXf$kPFV-z~c5NDm0QbU&FLiY4*kK#a9|%Yg0^3%HAfuxIAKf zxn%PV+wvxj{B`2{Ku2!X%*}rDjgMjtuy~-u&-<0>>vwZWy4e8hTeF6pTh+ND%BBx4 z+XYDlG+<$2ovkc@sY0nUT>Y;24PfpQVfC{7xvw2+c;vt%?pe#b#4vlxee%$>j*On; zETxDLZyhGnC`KxbP=p-c|6F+TMV%3VIp?xbP`rCb3*Gj1?4Nc8yxcWQ8tDp9coDLK zq(q2X;A0!>#ekg;qGHfNM2O5y!N+U&*#-yHk#5wVS<%u9@7ACV$uho=Wmj=!IlgTe zvxt}2c*>iFZjTFze$5pX74NFvB6^yk3E{HJv~$&(uY#Aag|bv!ZF-4Q+oU#ACSQVJe<8oz` z>3XsfFghKCM-(KW)(n?2i zc-~uQ9oZ1QFE_?$``hW9lC6W?nGs}(FVqjNiX z8%rdy@37ysUNzZyXSh4v%qW5NwKZTF{oCp2n${Ho@KEx^cUKI-c7Uh|gJRJ~JxnqR zvuNh_B(wK>dQ|~tXFRS}#7>oV{aATFC5Fpz#Zjo~DP-uE*6o*9_mN57*R!z-XkTjeB1XG94w%6Tem1TZyjREzi({K54 zFM$v^V9jVX6$ZaLJWTEo89hBhF+e*VceOqEL+5mL`YkRRnpIG&ZWmV0Mf2TZ6~i>{ zl!huBtZrcRsCnVYX}Vv_a!n@fIiKxf;fHS}TmAU^jyt|Cnd7*SuNx-Ts-Kp#0Ovd) zu=8xSX#~PqU29W+P6*&pvS~fehYeiR`N$2^`UPFFdlt@S+xnl$P5p@p zv=@`o((2sGb(mXGe+tvhtki1-n7fn78icj>XqjD)$7N>v%;6z*+07;+I&EMIh1YeI z_I!5dj0Gr2JK3V124MD=+tyTuo-?3!IABC_HQoPsGRK#epwJs+$NUD_jNoj==d$7R zPw6cQkH@zDuwUQXQQi1OTJT$mO>*ud36D0-$qFs7LafgjThK%a3Bie~^ zR|?x8mZXG)X?J9#-4#nCtM6izQ&oO=TBv>Zm<` z$dq@m=FVo=jtQ&BX>4hk>7G|e($|Og=HK4jTpbZS`F2%2*>Y~!{0+TKuOT=sO|Nbm zMi_NNEImdNsmKx}MmBoq|pjy+P|uc0U}NvPP_vD@3*hwP*t{~WwOd)HBG zYMPz=#Kgn||D=C(bY#P3&PpVw#Evof8V3B#kNnaULR^p`ds>?rGjMB&5P6%=db-q1 ziP0PvnXIrA=7Rer0@{K&g9>>PHN3ss&krq^Tixq2KmNZ0pNDb40k7_#+q zvp-dR$86zG1<57Q!`Lo96hZ52kB7Ed@GVLJd{9sj9{NVk5!k2aY&Se#F^w6VwI4o7 z1Uktf3MJ6{A9?MUB9M%l8-YLGVxGTm&X9@X!r38u(%f9-dL&KqVYi|k%0S<}SfN`y zvp<^Wsr@>q%0!!@91qjm{7TQ;KcVVa{Jg_6t=m`ok(jOgy2(E=ZbK62K6cI#u2SAT zpu#=frce%#EbDHi#ANa5O5#6>sguhgLed-RO94B#88c@QDRo3CK`C{vkBIMhdFutj z8QjDF%UQ5JJRt~;jz*EddW@PX{774CF(zI$y#|L3Ll&~fsa;i04vnedk!$*6Lx*=0 zujHoVvO_=l>E$K2tMsJw`_1$mQGHEczjhczQ-rgXp97XOR+WxMG6d5QsY(B22;Sr` zH4Xaxm#|+jf}m&pQd6sK*X8g)KsenvwF!bHBJ zn*U&Js29q{yaFQRp&V*9Vbud?h<$eTdtR95t~@E^_sCY#f2AwG?la~z)z9(&kxuro z+L>KJAif;@{~TGMOeU3&pbB`WWEs6_2`{Llja*?;N&N`>4&gr!6)rq;$-)s!-QQIz zvwh)UZs~#ln_JrIqu&r5FxqPd)O1}=xb)h$UGtSY(Z53Rzbwb}kKkXOgsthiza7^V z_Fw#q0-hxs;On=d>c2LdKJ2Cj1>X0v|4>n95=;P}43#mM{&!tTzve<;IB4^1u`0tq z%0Z!h=CB@U3$GIGj?q{%crktB}^)=I)`lB>W%b zOCq&rTvvcvIx{fGNr}yQ_BFw1yNrR?nUp0ZDzC)n&)MT+Ehha~x%v5B{RPKfd;%)@ z0%=KbP=SS7RnEV_qCV@=U;gqge|LYR!WYH3F}JnUXQ|Oa;_lDMn>TMpm)&*lVF_fB zg2#=B5k&0uCDmT-(QU^vN)y|syUQ+V- z@cEf znuk;=FQ$A9?V(pIgb#0bqU9{JhV)In9fU6#$f-4 zr&I!-sYI-5OZ{KeXZupnTi8o`R-l6_q;C1rxyktoxw6aYqbJ!PA4G1UH!Ew$$F}A4 z-TlpG%Cw?}zU_AiuBQlwdSrQr+uNFK&sxQ3%z17O(!P3v?tNBRR-{(2To3#3!_N_4PBd;jhWc(|PHZgYn{87&9 zT!YbDOMj6ZEyX1Zl8p9KW;#3Ok*Tz@i@XAzz`fR|ez7-Mn5k*3-*LPC0(qBfa;8&Q z!s|!_V4nPm2f6kH73vyrox$OL{>73~u~5GKzPq3>Q*pM@ zuW+xf>aD1zXgICrd3N(N|42<4U^?9qUl6;bukQE#J8>SD%!+%F;r33J86dySnmi4% z1M;4e%8NDrYvoKLA`1x#9hT=w__Z(P#UlPI82w1pLEIl!!SfN>2<#VvGU!$(x|>u! z)4r{33F}t<3W44)c|CZt5bzfxH4$ z0QpW`;StTTzO2v8R+-o>2t&rFZtBNM!PCB35*T`kg%eUxF*o^`&66;1GiyJEESq7arnLU#WdO%andVPM<+CHDOkI z&=*BI*63iuZ8a(TBSt)8a)w5|Wam$ONj{v92n?DIVt%KBw}|GMjn0?5W8=c9DlV6XDWY z}l-)JHvc2>u6qgnjK~mMzTG97yFKGZJ8vgzA{AWX3P5!t0zh5rx zH_jo3{gq!gR^&oZ0_8{_5$i#x<`3@g3=HlfI4DM_I-d*7>I57OG}F`5Gvh)OQ@J@8 zj$tHA(WbCd{N%lA-eK0@1cf*w3(H7KF4hAt-NUz{ugG{yqpsVE=ZLH7F)u?fU(<*h z`kYM$n8HJ@Kyi8De|G`vBkb5A@g?!YUvNIj`3b;ZDujyxu;}NO;ILf}=vR+y^QLu9 zK5}Kj)L!I>lKm@~fj)s&YOSh@N4#Bggw-+Y7~dz_qdh1)t75&w|qR!6(wV5#a;V=27 zE5WCrLQvRSIWMq@DMm?H0lr<)!#|FUN?SSvwxe8B=(hyJ8Z@i71C2V1YSlPWv&`q`+D3&(UoV9I z_Jp?)NBF&1pgVX3H2+>9wOdnzP${C-Ep5g>$YN8nk5WYovU|FwK4%VE;cTFd8s z%U_wq$_PS5g6T(2?NzXD59qr57uic84wVwZ-x<7d{O27exVa8C8L(3DFbf{@H#par zw!$k!=Iw^zEg}zM4Kl261<-a$mHUi)F0sFAQrTaMoGmLrKTF`H-7o0=_|b@SaedX} zSs^`}6a(P6*L3c>b*%!0rD>V?Lb(Y_2RM|KU* zxX9~!{(K*tsFw>2>bNu{aktYiW@lWn45D^b>g1=p#!9VWJ~M>NIT;u;0nm^RXfG8O z8c*wZ&XpEK|iXX#T5OVAwuKiC_O~*};yo!DzbUzW&!EqroXWu@!BCsH50D z*pwsUBI`ot(Oxm%pP*zbAEm@0fUrEyfUz2;Sjdv`pipEwQt&%ylu1SOEpEphpHduDA^{lwF0z6nbL8S*992B+~5%?JHc9(o*= z95)lVjO4Iv?(YF!ZJ+=a2=C{O5T0zr;;oFfbj&CrP7Vygm1(IY*Nvymcggfy$y&3LA z5fm-x4f^}Z`|E(=bZQZ?)`hKc2PxjS5d`Yd+oW#c;=Y6A*4>*Fv!-MvOL98j>PofFw!Iv zgyYnPKNj^l{g1ismbQ`C6YY*@-vHjYIh$|riXW!%bwFv!U1DR<@^47Zt`8`x(^|nh zMj;+N(Bz{fQXs&9+uNP|lMn|g0oueGQ50&Ic3_fPH)N%*vGT2a2YX&O)POT8_0AjW z#^3}+2P0Z96}W9}+Rusm-EB;-0kb;~8w&c3#du;hy!88_u!fPlR~97vjZQqE(rGkM0{`;CE2?K685vppGK9xd@Dj_=p9g2!L2txiY$GNWAU(hkx$_`x z5a>1RpN5Sv26Q5pX79i~3-h~*Vg*i1{uV;xxBO(@iKXB!)NHGf@ufUJaKZmM`c9aH zognZ82ZJcQf%1z}h&wVeQ5=4Kwxp87oj3Qd7O1EV092z_Fstz~gZKjsP8UDwu$1C9 zm-UVCh47fyg~p!0;FpLjY6_@fPqPi#yXT{yQI9`JjHVZ`MzX51%2I5?ynAc-+*!M{ zYgMK@G|G*{Co_K_yzvD{$#(EFV<%eN_N{0g1jeC!Co1_tYB%lc`_nm+NrG;&f+zg^ zq@>#9M~cqkMfXQn^gfCdDAl6XlpXSo?Exu%H)NTDMTioM#Lf%TmmZ((VcM@2Mz1VW z-5EK97k`reL7o$*`_jYcwVS0#P9M?~b_P zJeCjtg&{^BVL)pY07akL%1nH7$gEm@vSpmcbK-ToXs_HWQ=#1gFXw(uG>w?on>`Sn znh)|4DIhMb)Wj=rD~OF=k~+cgnaGc)Bi6q9Oa3^X3E4L}*b!qmawjYxXm*j;HFsq? z*0+-?BsG+oYNXHlg}#$&_sq&~JS&I#2%lG|QD`GWjv~`?^ZJ5<&b+K}aY54p!68*0 z`0qghuIO6R{a3GZfDfHo3PL=(|D82pv8SM+3>UgH@j>0si(a1J3OVk5>mgA6-8y{t zgR`e0-@Zxv9(J>aYx{2(*%H33V+yP`<(e{^-1oH(`B)LZW3Ht8J~C5q+#tU4Hb|8e zdu-7x-@$yiG9wPPmDC=|{}>q6e*XCQJW0g(F=y7qjR#5%-CTab*)3aCC;eg-K1fl` zK$gafjYvf?0+(JG%eBm@jdG^lg zd%4k4r52|Vq+mI7^JLBTG@}fXnj{T4{jpAe5S&izB4~&xw&vXDuqJy?^#@OYgFs1% zG%5*KvWL8%bnI0NIhG$gI2lG_aI&y1CwdU>OWaQ^q#AQwrNSytoN5LhiJw3m(2Qt) z0x&n<=FnDd+0zGb+WSh$H*XCSH%NHQF5?MASM^7tvV6{lnx2orerdKA?mis~JMC+} zoa=Boc1v#0;O%=Cf!#hKBbaOdL8*5{fX@!6cMwYczASYa2*=nxsHy(T)9(`N^lQw< zqxa`$jAxOj+RylAH-C?1;uq^()Xn3{C{;U({Bi97{()GL1yrq1n~IK%vwK6r2QnsZ1T4bT%`E`tFS_V>0{D@Q3emF%&Qx$P6TnVJ)5*Odb zh}aM5$IN6a_e6LVeko;<7DW%3q6tS(lOri0_x6#IHSJ&1c5& z02rwshmzvSBQ4a#ug0{JCw_%kz)^4>fAb|QqWJ*>^?VayK9O%jkx=1Z=YhHRKB&p- z-wt)!TKo&4+H~X z^hTs+$pYt*?mDC))q81Ju=$w2@<*=H;u6XhxFHAImsel2JuW>@H^A;~j&mr6wv@ao zg_T9Gw(60|GvtXLz!y2<( zzy8}s(aZXtO|{|R;n?D-Z@~&{?mgV^MKj+$DOVDR<0rW7_0KMU!g8XECVpL?lO3O$ z=1xw|SXN`1sn2=V31vv)#&`>im-tv91t~Fg^!t5p|0Y+>2<->Pb5Z?j*Jypr=E69@ zYgLS1j_AL&O88Hbl`5t2`cDdMNrFN~Yv>z&PW=Y`h6(_L3%NNDj(8w|E@rKBJ70E> z!hCYCl3jhj<5|T{eK?4l?2EmNlg7}9L(fil(A@L>=q~QLDe_=BzY^L}de{uk<*eM6 zntsYmg7?Fmg1dM)#+a>5_H|*Ia*()Z^T+yARm2$>@99mTBcH}#621IuDK#Z1$&Yra zMXldV_Lgh8tV4p-w%nO?IWoFL`gI8Y_B2;jo?f}i%rN5NchBjx%SL~QQ95o9Vcx|L z6#tC_ct4w5B_hPTG1f~wdfdv>E6X+iVXi`QjqB5U?1I5{k1J~3GnrVk0GnxlzX zA>Wjt*0I(nWj1S7SA~VRlJmO&JVnBr6#S}Wm!EBcjn%4(`y^s?#K1jld1Y{OqP=j(tih}B z5unV1l5ta&rRdy5&FLvP?fposDCagDN*MotRtRw!_nX@I=W*YB1P9kIfDVRKo{d@bssXfe3Qm1LiFmmYf48& z9P+F0WcjXwc=hq=nL!~To{1#<%iU_?AMDqm+u>f6Y|AjWPdA6?7YP#Z3evJN^Uv^~ zVo0hVQ|ReC!GFMq^eY|IA&y{>7oU~gC8>POF>3dk_E>=+6I0x$TaL@gt+{94v8GBs zK+-GyV_X5ntw(pi%F`PD6(=4DNc=~W)_O2lwNUlaoz*S9a9lUwDh%)kH_M7`{l0dN zY%44d$J6MUp4R$SP(HEa7=l&I1_|b1f_%}A1|Fno%0YC3uGlWd&T`m6{TPeJ#*?VMQ~t$>6rzYS05%A7){wMkV$dyBa!&CK(X(> za_ug2Ck58P?n)|)ivN3|YW%aC=Wi1W{@cX#l_3R4x6Ix~e+3F!?yN$kKQ^&GZFWB` z(R+?3md0f8p=X|W?mAo;xyNa8CwAd@3^s#zHz5Oj`Fq|or-Lw5$A@;InfEy(SQYL% zSJH{E=Mp7D+wgz!kt4mJH}mj7?69yOaw&X#AHiFD8wb8YSup=r))CUAFxnkD!!=d+ z1=sRLM#8W&b!2`Ng~0uGWk2ra>zXCzYd(VDCP5&u@aN`jx@BFn@lxeq)BUBH<;??^ ze+M!*)x2fc$R0||icA2kGrhw53c(bY-gz3i?VTgP8<`t~?bk=^PtPg5*66LfxVEGx z$e)kRDDEq<;eDUIj?v08G48E|RsQT9Ves!7q~~h=IQc!d{Choo2afo26E1-WCgRvv z`dk{uzV(_o@?k(hgn-cBOifyCUnV>k4i*I?SF*rqiX`SuvfvH6RlK%Ocdgv9&Pq}%a;ejG%46C3u zFWT=qMN!ui98XO)gVvf1x%fy}^i)EzTfi9aUP}e|!`U2I74qAdeTX-fI!fpL`|XSH zV~A%>ZzpIctWGxX5Wh>YBjU6mcD0g%x?k@yF$Uo-ftYB{_VFfpMa>bae3zODW|KJj zJ~=D*YYfM)QR+EiD{rE9x5SWsbB6)G=U=rPr1W9l2Wcv&v1jAGF!+zL`vVM6eg_7~l!j@ad{5{=ijb^^HK^{2BI+A35a#rdaBDc(5aR zE+&L?r8qUm{6FgrXOX8-+4v?h=n#{U5^~#8nJd8mfP);GwD^GLV-5DbDToK35TB%= zza7X^`b2e^BjrRIZruN|#a2y)yKRJjHvaYz0YVeX)Mfp0lZ-emo+`e_uQ(IZtZte8 zm6MJg>*2W!hiF-SkShn}2iciR2OEH>wg>aXn-(1`sVh(S)jcjyl~gJN9QgoYowpcT&feweosao&8Ehd+bp9sd6MMnck~2RTNBmD>&9%MvDO zA98@*8O7>FtdBwnN&xok zhBP#~tjLarK`N>JJL(u`y0F!g8yo5N9p2dHFepuRizV72Cn$L_TT6=Ur@%mDDxfEU zq8lBkab~JGBVILoc%klY;mGIT!KGPcE4zT_vK$;t^_l9!?|PL5fcPn!|K2R<%x>!8bL!22bS`$DhJ2Fitc8VG*zrn+1`QPg z(^|Ix;m5=7Ut7VMVIE_vrFI1?sOf<`)sh9!8k|0xi2Qd1fz~r2dAGEYK}}H44gJDr ztQnF>X)Z=BAKBi1nP2%zjv})Xy$`0p4eSoV(I4q%L{@iS_hRmCI#>-}2l!i!l5$k9 zzKynNF3d=@H&tBQV~2~xPkVZc7HWiPP)V3~zs6C@!ogl_MsB@lI4I687fe}33|cvB z3N=zh3_2Gw`Qubq+_FIW3(|dYn1-q1*Riq6t&aYAWaM~s@5Dc)Mcv#fTp+g*#pV`J z6P?87<$Uo(T~=WI`S~T@KI+hTLw5AG75M=Wb6&oN8`_KD`PBb(U_mTIfz$DXq2MTw z+qunR80UqaGC1hjprjLFGdU}r$Wp+TAH=7ZKQ74K+!7|ABK!RIxr}~j!Uo|MGsfk~ z!HwkVe&)#ipS2)ILZ5Sz4R#U?>J{Sl346N$58Lt^_^GABB`H~G8)Sq;daex39fJhQ zSp=$CAR!BdEaAHrk-HWhe*pqQzR=JY%bm(^jmmL~S&kI^gj{?Z`M$6cmXl&PvBEhi?)xCl5wWd<>IXQ8PM5_fdm`d!{ z-`^9bE0ZHFGSyqNQdoV zBZ17J+s7Q4JWe?8c5hQ2!f-^c4PU@ zdv^Mx&M+kSh=HC9RlD=SZ8Kk6{4WbGRysyYuBNkho5yX8x&=Yv-6E5xD@ymg&vPhb zMCqiZ!)_J4-ozkbFAv+;Nvm!;YBp-!!5L!Z!wHUbs>Fs@V%Bu?oVzK7D!ta% z(Q`YlF{TWSWsX}el=<*u(=xH;EZNFDUIMF+Na8=~H4Mat*h0l~__W%2X36iCECiqD zG+*Je;5wG5^*x}lLDZ%9V(Q8p<545JoMm@g{B#tlf;oDIB;E$Cg!-C#{Zkg&{boJ^ z{`*R0N0KAU^!h&(e`gIgu7q_{5EA48$SPg=Xczf~pg~^Q!F2rCuYgs= zZ?bNMSje9cmr1{dDHjuHQo1=UIfxqU`{~(@(kkziDeQ_Ua*+dqK{qOfTGh&{#${R`rCz_Z~f87RZ`- z`lGlaH~f;LW%1LLtFK)q^=IEw8BvlC2i&w2iS1IR7{xM0KdM@AJ zk9*6LMWsGT?hxnrOy#RQ#WzhTx4-857Mhok(|(hoo!BEztR%nVVFS8}sb$%#{?Ek= z$h!WbECA{*3@!Fv%k>74-z5*d&RYxEOG@q>2t1&LRJD4u=aK>ni}Q^Wbi2MJQtL?^ zb8QX1j#}?Mj%$v?%W@4UG>1+&Ml3Oi1t)xAhdJXVFl;Th?cP4I!!D~9Klmc=%|+IE zLE8C6Gj8Lv{m&+eLc2VfRrA71Ce<8@8&q)nV$wGj7BoRYK`hJlA@fda7c&(V^t$?b za6CX6R)JI;v1a`9^N{b`=5eErxz9wUo3v#CGW&v&l3`IC_%12Q$^GcgVb*Q@osHD& zx^3`u;kIbHz$cFGkP`o& zJYqu+t_q(?;5#b(hX36h+*}8X45X=MHSR^e3f(|Kg~ykNe#x<4Un1&QRjNuf%w)l+ z1)H_TWY_CDxZree0Mjs9N(f)IEL6b);kU+tv(ambXI$mSm1k=K|7i^Q2I>bFcDT%g>|a5_C~6?{%u03435>P2f~G~&Y#Thq5$-IEzml~ z$2TnM5K(G`Ll}x2sc=kn#%szTCiW-)G2Nq!fAtr%z{4 z+=H-CRaphCJFeUY8%w}?%o|aOfx?yLkSvqNUvsX@ELIrMz+>#j}VBr5gN@azyDX?xzM$%1kKBJKe~k+R;HDYxX!0buMhC48>ojn;o>9@ZSz6G~ z-rW9C44Ed0m28V(+CY@AO%Ww-Y7zR)8ugdl#e(=!6emKr{sL{K0 zJvUSv>Sg~-r?{&EbSq+h|AK~tQ|v3WNG)*eV*m?kU%)m(3U62aY@57^Ls;F!o2=h= zU9`&b)tub7yl2rac^OCi&>L@p%5=V|;PMpnM+6xb5rFq-WY6h45r*8~D{bli15$=z zzVM=m^KdVQq2KL2nmV>3S2GgZld{#<-zVD=2rKl` zqc77~FQ*&hD(hCEPO$mRASW#LdZ=@W`03pP<_()BY>;C0h}1JkMyBU9e-VA|9xZQT zJ>;+33XT74vm&7f!SP5@+4vhbf=Ta9J00Q{f?l1a|&#c8e-D7va$;JvU z{0*%bVW08iH7StoQEwM9rY=H%&7QNy$95s#V2E~-qSG6l)`7Q^A5qI^h z0@`3lI@v{~85CF3Yj`x{yC=iy6`MncTvR;!%Y}O`c`;g!?sEw9dPfFaSxkuaf8#`P zixF`R=S#qG5Vp_2;r`{H;0_9zR--`Q8>FC$bbNk+={w)Lt7IiNXJ~7*2}$JhhvMZN ze4ytf3*nVHXX1^%8*LJ4Wk1Ic5fWSoK5mDqeNi9pwStk{Y>&J&eWjuks;-#~C7R+BrCkpMl&#BgoA|zrr}b`HheL1FLV{`{y)pS zix=OC^@Ps%)vvHt`xR{}M_VK}uA|`b zYZD&pq;{`|o>?tyzumkZn=*WoVbDvW<^dzLV=ZkRPeJ6+vV-6S3 zF(CnUnZp*4vlnBAl#)jxvTl?^{t8BqhKRU{p!G6KEI$U{RJF3h~pAzO<959GZ+~JW@4gZjQ%#d`o z1Z90RW5rncrM|*2#PUUu#VLq73cC>M7?o#M*7`u6DXAQHu6R=jx#0`0Z$w;sqcKL- z)ou%wZ_BxzY^pPQ^lY)>5sKgQ^U4u`8_|A7EqU>=8r|dRKc5m|3Ef+Sqy=;}2|k>7 z)YKl)@N==DX>mLI%L(QKJTcDJU22V|5nX+@CqeqYk-VQc(JgagCk(Y};uip3k229d zL#VAn1KZ`hbMY(3!V-qF?3f+Vo1_r$3 zXuE6i7MqEx-V$P!Qi=YmVRc@lG$8}@K~cDa%3&yh4;~*k(3LsDS=8zWYFSsHxpzyT z1^R}c3#`uL>n{dhklf+P!+dOzz3@!Nnj|ZHYdLLf*OOqEpCxiO(nSac@8IC5`wS~{ zZ}(-IYIA#7tcW;Q-QlzqlQ6(1X{en>xf|OkAeRzMHCQ8peAw^)j!O+f6F;9@R;s)0 ztvh0WZgvn=v5!p-qkxvpu8E))`b{rm#7Rn^J~{snxOJ_Rukr@{s|QEc{Q;d#-nqc{ zS1*>exBXOreNM!wiUk;|0Iz*3r10Lfuq2|4_;)!-je#%56k!xYo6z_X8Ok#@SZxho zA@mZ$`ZR>{7F*yOThecGVgJ4(a!(;_%1#|m!6+jda{y;ZW&^3J-vBkbZ|eL#2?(k9)lcl{ zV8+QK52;`G`Qgj&W6L)1g9v_&3FUMUB1p9w1O!}KhBQwMHfJ=KJuBCkiQhWHu(#VY zsUb*!Y=k${5(6C;s6~?nkWK#vZ|!RaRTK;_W%>p=-wqgTw5s2;v{T&Pu&LJYux44y zx4Iz=US2X?Zj}N*gT74cW@yMT3Mpd%iuEzjrFv{!ysZS@hH-Uwm}E z{rixTe4iHqI{NY{!-CoyH>~nYsE0$Sz^Imp9fpH=^cPq{X~>l)BPA0yU%jTF&D?3w zx!9tB+ram){bck;G=Eo|KF_JEuTYnMF@dtlo-UA`4K-m1Cm?4?ip7@q~hSOh%wFd@c5jc*sytPk;G zf87k;VDX(hrwMxXzI-xaK?EI-_#i52<}B$tX8aI;BBe9*&cK2R{`P_FnzfA^Yw||) z`)Nc$n88;=@47E*h}C3VABfj3%-fIIc8|YEZvi>BQn?=#r-k7sWE4KRVtT;GXZ9Z^ zt@SWzZ`NEz63(#|{U)KeFNpIy>hJt_66E=R%DcI>oWtnMT9#Yk=+!Pr6wjFrg=&W@^g*|N8kei zKD*TdH)Rz0@Ib%DW}Xp4Fp;cgAiRlq3%1p|7h89$vGymAKqhqv4^dP zi;Z`bah=ij9y2BUyaxhC2bmMx!f4&oIfjzqi}il6QnRbqh{SBc}uEjKhNH z#~DAuq3#SJKY>BdUSGp1+T6{fcjf14IiYXfYt`6A=ba@V;rA_p(IgK#;1H)$8rO>r zqI!$Ts&8&MX<(zYjmf;LsR7J*+f0|0Ac|B|&Hhofse;PYK z#6Sgm#Y$$B(`3l**mV(2L}mzDXkz49@2AMbBo#!Wk@b#2nkVWY3DfQ@2Y~mkz2EOo1ROqC4e_NykYOMWm_E`zXqjkb5 zdcLsPV^me^hTfo0gURY>hM?UVo^q~id_|b0r2i>?53dIp4dUcB1X$R}@dPO?YHS6@ zRCamb5Ry{GP4oxlU~&?#dWM{2(T)R)%9YW0(6`?glXtD4L*ZVwb^NNZjCw0GyR;`KORbUn!c*~b zPmoU)ipgDqLf#gzCaEayFv-9Ww=PBy8wmWS^u3Q!!kNjfDjHcTaWrD-Ru0~v9&!#k zUcyYSP}<+dr#KSw4XV($j*vPi^rz&R{KKru9j~=lvTaVn(7g=N59I_gA{t*irRzU* zU+muHh2loe03r(X$hX)EqZ`!3uBi5k4pPteVx##Cw)uzQaYri)=N=i8#cBR6X$_fG zhh%uT$Eb0cSjvs8OjOalJ51#EI{Z-irfA{ynrE)Q$uWn^g)FuOZ*qWMbjUUoygHM; zz0wGO8&c8D6=%4)l7|Dm&8@iVXo*h}%OpqNV1nD5#b)!xv^&;Uzssma+wi3R{N;u5 zrKvdgBn>b6YMh)!5h6W6@Y_yi&LVl0Ct9{`dcO8~hja>`0=Cs0(3xfjSUc=I2(_Ug z{83`2?P~^K?DY}Q|LehM$FCUv&yFZot4}YV1+}UlIcu6E%1;WRdu2GWW#^4x3Ygod zQOM`W#wi2>r>P6Xlp=4_kuCSOnX;&Rh&S@%D=!Znrgv@&mO8wO42oOk?I^Eyct<^{ z^Ku*kwz@yM%I}{@=Ct%&|)xc3pGy2Kh*Z1d| zF2#PBxy?U$KFiK<$237ardE5<)V$1)Z<%RFm&@j|9~S99__YFWxN<-3Dbieymup{9TYO zw&2fn@KU=6^ZA7n2_C=o5jbrOG+&OJ?Q_l}pUI65?&zE1Y&`|{I)RT4ZdYK!&}z0A z44WFzrRG!<^QL}!wr%Ydyg0nH6dVWG+`}XY$Tt%7e4#Y?+DD-D}I5)%> zKL6{UT(q{*6V|`6d_VyO{$x7aIypgNHhF}zgYt7$%T;WLLfk;&_%)RJ^9~2zvx;t? zagxZN*j@`E02BW6aWF)aE~~i_vqVRnuPr>rqMcZv2Lv(v5fqS@TCX-!!B9fD2?A1p zSUA-ki{|)5%uAcSNyA^yQU5GJJI+5b8}c)!vfp}RsnbYor4?r)$9PMuPKu?oA<&h` zq&jr_*M>b}3o5nch~epPFy%|t-_Tn;bE#k3tWxPzg+fL{)J;ek2iDn!`vi6~lpGtm z+@c>FabmD5#0WHVmwgb6Qt8_?qPoyD5U{62qnky3>cfu~>$|92^MgYU<$y;MP99mF|Yxc3OG8 z8FJlvkqSo;2q^(c-Y~?okYI*d)ucK(+2smlzDI~tPKmslTXg@quK#oJoH443K1wv( z0gql7-|hTXP7uf&Dhdy>$&&R&vc{20B!+dOk&gB2}p)-eX(q^d5ygEt6b@NuPE0^5!Gc~yU>61Rc58;c_tI7G}RCD6l;j(gO zu42c`V$FH{Ze-$)eZGuSj@0UaJq<0B$oo;&BYPEkwolQQg@i`MJ%C}cUH{OdJ**}m zH~gNJ@$%wBu5YhfKL3t%)2FWPsg>D0mh2^!O&ADKd}|$|z)pMKAIMH?FO-gzN^rdL z!_?E`hzgnPB~#FD-7$pG!&1E`%CtSt%899oSv&IYV6mz6F0^(FlzIxJ<+h^P(mRUk zs$zf7->e~9z69GqcwStK{4b{VnVM;k<=fL$V_J0{os*1$MU1>whM%ne6D?qIsQTZ zo!)fQPel~r_{SellR(W154BrBte%3=3bHH$=wxxW7M7i@q%a=cX*6;8w+`$Ua{i5T zRlbZI#h*S;*NE`1<=Ggza(dB}meW-few;IcI^00=7*c(vzwnx$&|#Qa_Q>c@_))0N zO5lB<7TAr;L;u?S6ZqhgJ=SLm@MI}I!?Tu8CFXlJM(FZfa|L>;S>-x}#v!_D7HnX{ z|CSILd`yiAu z6aK&;Cm%K`6<5h3j~1AEw8m({gRe`ZL@4^mb}eMlb9(MGRepf)pzk*o8#i-mG3wFEfY{hXazHRkfxd_dQA% zlwM#OC(a0ujZ{KfK2`2}fBH%DNE|NZH~rbWe?%~20AfWN ztL*{VB5^3_5In#nEYHA3i7+N6OzT%Hs(ncjXSN0V2&7r{HmiuB?-T9t_#OEzmz5b; zr$=v}&}Ydo97^swKMu~*?!pRG&7;IWMzAB7`Q&8o(~4kY&y(7joHcB0WbKD-BEgr)u{Gin-2&m$$_omm$j?e6-sHN z7ZzDa*H|eKp*_!0INw=Zt)Kb}v0M4`#aY06vT6I})go=17%$mQ1Z2wxF-Nt{;p`RJ zoxaQU=hKj=l61QG7hD8y`X+YWhBw~GJ}s)yA{y^mPD;@ay}F$|1Q#6&yG|a_O3i+i%kwG-V{G0qV-pV!Q9y?NJRvN&2GvA=miVr+=i`dc3Gju$98|6ro={2LkL6fu z)G*gmxk5bE6#gn@ei-kDC}LE4|Is^iV8$3KGawndI>(hocT5h>R=>chi*3p|g^P-EkP? zI0_;paTxS&%UY;Q?x+&l;>NH-ezy>pXZ9sr#!+#Y{^PdU;+wV=cD zR-T`(0|?8(sAqm$Q3KgMPQ=p&@7JZ|i^3=-IsuI*^$Q`l^ug5^7^}O z(Gl68z4)L(hq}`Dz9Y41?Fk400rBynrUGgAf8rTVJCTGn%-TLaB(Nl8VAX1q)EoNP zkAkdp|1L?#<}0`d@=|6ik2Z8a=2x(>neRwUd_|$W9EoXEZUyOUqo3C$--uP!XN_1b`2#!+o@I3Jnu64aX+la5N zLU-Nym2ZfupeuKG8E$woXvt=6v?!F3R?4^>+&2(NJQ`oEg3G7rP6@v^|6eJEhAMvL z_-6LyNRp<}gJy{IUkbr)o*lD(tI6L79-*B-a3l>IOV5*)ApglM%c+{cJ6#PFI*t;$ z&+59x@IpnMF-WY;c@~bnY?M*zhjq(r%sr7wjN!5bLAi+G^w|^`2Wy>bDk0G;WdMMXs!FPEE+wm#Qa ziv)j?mXj;T5ti%zjq0#^JYNn6&TKl{wKLuIln#LT-z!teXm2RK#ry8}y@TA{13Ul# zI2m-de|q_0=J3=Wx}wLqTf0y2FhR|7cT%QXFpNr-%I<%Jx4`^DC^fF7yK%?zNn$Nj zZqR*;BnMb-aEFTcw+IS|90i0Kk5V!{8Kkd$Nif`Ac=-s}^souusfW864AUg?e_u55{=cn4Fv^BIQt$&~BSsu8Ky^iJsVOvY`8Ien1wQHUM68}%9X2*4r! zO2Q{bYf=Jy#RAO7{UzbU-JQL)6)(FQiP@PII^GY3F?M^_1xf{x!DQ3bdAGN zt;cImErJGyOQjxAof>}7HW6bBzcWr>&&aIZgX%%kqRrxn1^gL)CC9roik#8CYv;+9 z8YD{@Ppn}m$hr1N{$tqQam5aM3iYz zEvg3wvT&G(D{}?e-kDY&8}qLV`1O3_2p8>fn{Di9LrsOM1`jo(?u7oBmY*S?>R*49jd+yI8%vZ)#wDmDynohlC(Kz8CC zUBcBXe8EK$^C2xv*mqR2_xOp&N~oMwtuAZ%(R!V?r)ti-3x`p6%`UZleAA11U?>7bLVkthD z^G_uj-qIMlp1yhs{H7A{OaB$~Zr4Kc$RN{k_pAA2CQn!*wbJf{{dy&A*XEUg?x)RY zoXS1DJrq3^NKHk#!L9{V)3u>?WzC7|o}v82#~iDR-Bw!6=@Gx5yK`5w*K)_m-mVua zf!iJOP+yg`;9H_{0a~6encye&zsr#V3hUNEe5mt0l{#FHVGf3#gUqxw_e{ev8ot9k zZ^Ba!w`C3{_UgCJBi*bb?4rhH%&oTJm;F1P?6>`h-6ovm%w(Z~gqI0!-X^{;KgRz~ z=}>(O5ViHYAc1$(o?^C%AIEb&Ok(`rsT%k}w`E_ZsUZSj28fWP49H%XiGtLAVMSm} zceZ+VFI`qxV$%IYNVa#DvSoNdR-e<7SiGAkrvs&OYVa*;Cw~ypGJmjyais^ zSCYoIwWBmOrkd5Kc!s`DlpM7@{xmiz6(!N^`MYsF!iDYokZh-ceapC8YFRBXx&Ml9 z#Nco<_QgRvE~e@CZXK`~BoVV;k-VRqe?sZMc*efqbh-qdId-H*UHj1^vNo?^K6x53 z<1uzmw>XnOFd|kw|UZSuR==+h=D9!y=r!h|p2FA0;OEiLY9# z8ApBg9o^>TA%x{qRuLf%%W#t@M%}v4cC4kEIgH?9$QjQ!z1|UsDDI7Rj|)WhR+Cdt zW}@OW3-r87Vwzdk7{eCleZ(e*&8j;qeGQ{u0Zw4Wz95*cO8 zJ7^0sliLgN%)y4<_~YGjsV39vt40lqJ>B?3e^ja?=#trT{@A?1VY3$b#~a+e%wE^! ziSHLEd+QxdObWB;y8t8!ttjQkRan9mAG~JpD8}Z-&T;69;t24Puc8tb*_;nq7DV_7 z@!z#16%f?r*J0g$JCh~n^tfisk9CdqyuMJ~G+tZ&F&@UXTi#LaC>6zb`F)YP(k=CQ zByF!dtZF!|s(m;uQh}z!RV4wROY#On&(E$tF1jE)dbcflS9h4(H@&boP!GhsP z>$6F6klxI;u7f8iPy#%eA*6ohjMrdF%XTuG@uv{qwd4D%199Yvn>m-AccO2Z=vOv? zxiC>`P7t?KU=^U;lx1(lWzlUCl20erko~?b(s}3FY>M^EW;^Lyq zvc2TVK@TlVrfui`rmFEze7@fa$w4-|EmXTLhHNM~0zS$YVt$0OQ{5W{Vep3>TqxMT zF6vF;l5(q?Y$gT5PChv;AI@OU@wsWXm%zN8D7-qYYsDEZFtshCg1_mqjqyLI&|Hn0p#o%?F*k{+-`?JqU6w?ZH%Sx_6yO_8Ra1))_*NizAV=`g$ zZ+_dW+*<7&viB-eECM(I*7`IK46S#uL9Z-vM6@e5RK8M;Iq!_3CfPYF`#opiHW&Ez zv*j3_Z(-^tbT{FZ^5j~d*q9(Y^>_*I6J~w74PmE!+pvF}9%XDH8G2D*74OS9-IV=t_hVDG z21-Kkc}itNHW5O?x6_o$+~**T%9Ix7O|5*8TfvqRw^2lhe{+L-g6q8ijFAGp@O@69 ziup$#q9x+dREJ9OnQGX+j_fj<810CfsN0+}lvn=2h=HoUKq{Nf9WLY7ZbzY;+5Gzm!Q zjudCM>8;&13%Wd}8k}Q;uWa5i`qJ`1oPAh#BJoeg=M4miVoEjfNa=wmH#xGKD>}S7 z3V_oqH^vHLzc$zxHNmFfp7KX)Uy&z&>19s`NOo#tclp19?T!RYZGJ&E((jJQU?%vq z(0jH&-FM;?+&Ic_k-bt4<;_+VTzrA;e@`4#IB^iVw(q~TG@;dg0{g7>k)8TW z{nre@DaeRONL26r{F~c0v}5jHjRb~b)JZw8_~owm2fH*Iqd5OHEJQ|tP>E#=cz+e{ zn-_r1vv78gx?cx2xRy|;aa8^HmB)ZlY1f05CjfD)Q`p;qEq3 zBIy!dULv8Pp+Kgk5CrV9ES-u5#yf$3^;(??Q5IL5X%v6gqE;9Ih_nVNJAVg@yKKG* z!ZMNlD-Qe55PyXrysAh)xnEW|=lBZafpPa#=D{5IYgNJ&otC!!fkcG_MnFJ7#s8V; z<^3)WkTudkcNg2MZ_HjU??VOzUaV3?k!2YXWSCO_yUZ?u59ncAq4`28_W}eQY556| z5Tw5YDEkCRoUm~@^WUKY0~mV%=Y$8z2KQ`S3+LhYi>Tz1cQU!~5-yNmYOL7)c|*bg zE|X8)lg$|JD5wfNa3u7Ph7|@FuZDm$?4HZ+<`)M1$&K5X|J8ki{J`@K5o}gycMbaH z10uM`l98SM-%~)ju3xf_jJ@OK{}%AFSorY$$_4)!%meh=B;ZdZNNnUu-!QdpW935@OKS1w`Vc@F%orf0SpTyxnG^e zXt*kKW(|_LpQSD$AhCO5A8_5~$EmOyoTYm&cja{78Kb>{L!6zW#wi==VE+L$pJ#CL&>rY+Yo~p>OsvoswLdwsov<}v{ z`%v4_4IE$?laTN^Jp9qh$_iLEjQG6?Nd%1_1)w>6=%pI}lA8m*eqm39u+>z)5GvUUiN^8|q}!x#HQ`nRa=5 zdpj7g(C!BvB*haV*S^_o5~w8}q21$r`su@);bgq<5+l>$Y}o;;NV{m?Uigv=nY!L< zlLaRDl84Y+q24d~I_@_+$=j6RQ>^#zLYbn(KS94OPfULrC7e*JeEWumMgB~p$x?1PMP|#XkP4CYHN^c-0}vNs-JB@3=aZ7% z^3($kkSrmD730*#;+)wEk$sCzRlniB8E|quvW-MeQvmo&-QfrKy+8@TUEhdw>Ct&J7wd~1KjKc@8`oq;OpE^%*uPC+mZv=l~ZQ7 zy5HIu7bUA(FN)ROT_bqA;EOTNiUP8XF&E~O1nhyb>f67XVOIhewy?V&|6j2n1cdzY z>eC5wMM-Ft*W5Q3jKLBI)Vli5g%!HF;63&)w)y|dvQxn=`$Ntl)cc7?!Hr+e?5eIA z%SJ;&DMgja;uPNijn>2m7tKOKSi0<&oiSv%Vv$S+-D!b!JVBouk>;yIMM`n+#@|qv zyqovqfTMS{>jseqj&P>>{8AEX;#?$uw|=Txz;c!rW^h{062f}@meLcMMH+sUu56N* z47`g6ygK&dnbARERr)pn%7 z)x2%lVkp15_0qu`A(D@o%=Ntv4wo%rICcbacVevaC{&gz7L%sVKu8 z1cJD!hj2D>C-#eH)5->}J_7RkiF#xi@$rGcE#8a6a?~zePcbgAP0SeYr~~XkTXImY zm+0nPnvZ+k79(~OjS5%3e>YN_Va)B5dtF`ICwncWG0k;iAyNYFR^SxY&Gh%q)o%L3 z6lgi&9;}Gy` zzS-C}wf%#G@)>@N3s-tGl)vLpX)!~&j?T~)15=Qn8+7!H@0oRT=8BSq>Y!f_&6lFF zO+4-R(l~Nz?>w)$i6fDdlX|dJe=WcozUa_)V70)f&p$(X28pTBfLPL%`Tklzrv%kQ8U_YZ zMlB9E8`r(04P95I`f;W+6lbaYXx|hn89q;^U`Z1(#}lG#b@wL+Uw1T3)Eaz$$-FTW5l(b3qnpnWqKIIgqEU*R#{r?rUAOPir~5AQ zi`gDF`)|E+8!xfwqll=rtFE#4a;0{JQinW!Qr%`b;wix1>@`i>?9K8~$lQA~VwoK3_)`bCMVjOV7G;oQz9gbqx%-rS zKwQ*!k-{E*-c<|J?e-mc-GmKQVeF8S_y}}#?JAa%@Gd*ES{9s&3>C?*drcGS_@CXHKrHi7b~xb`l?u# z(6rth+6H2&v>GwV{7lb#DKaPi3+ll?u#ApgO9r>_Yq%-Wa8av*2TMAztwz40!*8kt zPJV5hm>?1B?(Uvodun)|4jd*-$IV@SDAlNVCz093C5m8B1^I+-uOyWvUR4%UNlMrmclZi)FRbhWRbPPkB&f?rJL zeH50&Fh-7nT*bq>>_nIiyX5krnqIj+?>VA(M@t=ae{dFuc6iNeg=T_63z4#P+Na-{+dm~?M2&$t`|$*;@OlLk4icJfK;CxGFP#W0T*D+ zCn@Oe^A^2eu6rla)AsF#9=FMz>a)jpFz4*U$fO8vx(ysQ#*Ou2vOqjV)3V*=q<3Na z1Gc>zekp>kN<*BNw#iB;)29}%H+5-Dd44O0mGEm>MnFfe&92hd*_mvEko5tj-puyx zhw>fA>mzPLq2WWEP#-`=uD2H6J3xl;)gkTh3X8U%YN!Hph8|p0I*^{lPpNfCf3TgR zsrY6H-}Va-lc9r#I?{eE{rH1eZKoUg&0@;kMgij&)De|Tn1ZI-=;6Y|wV<<;RY;k4 z0UOUcBh8O0>gr-2iYD>0{wy;*81D^h?8Am$4`U_Km~-){EG^({eht6ZbvQd$7*sbW zF~R!|f@e=chKz2XW{bUh$F^a#>DQ;@5FGdMU4#A^CJ!;ANjoI|n-59|aLSd!9=26U zzd@I~Yo^4R0o5#UVf`wJQxF;BsfZGuT7yIhXWUkxhaQ*x&1YSMrh*jG6~lLx+38sqxFW1EK3th zDXkWbSN9{e1njwhz`71C+qZfV>TOc z3~E;hLP>|y*dhW#his$Nlq z*M3b#>oxL=EB={8iS;#7(eFgRU<7*Sli1hbcR{R?^q$6@+i&_3E!EIRWtj`%4T3|0j&!j?zztkmw=}`sfJt#e7~9)6;lB*7@L%djuidMvMh4gi+3-qHKZ#s)EabOx zrXjc~wX`z)_|BP!=c2k}w=I8YwK{JN?OIIm?8eC|s!-7R4DE23aIt2Ned&U`UF;I| zan;LYdJD(X1xG6ZJXlSu&r;&q+MGEIUs==#`nJ-;Nhdk~C*6i6=!TXj5~;tRalr6` z2(x)A>Xl{R&z$0`4<{>lD`}rr$nBrdHG2G5ENY&nF;a}BXK^dSy8w!OcLZ=*t_{VK zqvo`(@w^pztmLq**Ff>6wF#bkq!rW8J(OY`O%FAv# zCAnVnj^$U4p}Z!axOI?M{5g6pmfk{Y-Dn@PX-B-`FOog+COh)AlEI;l1aF6sit1G{ zr3T-kFL!3`A0HfgA=9}whJ&EweMYHl%x@Q#V-~w~#4uuscP5f#Lzpt_L^PO)q_{29 zi#EC@w8*~Y5sqd|mWaA-Qks;@lM{zyncAErbFpg+kfd+F38z|ohX=FEK)x=O_bFJbJSFLpnxG(yQNc zNkhEy^)cnWyz#J~&k6G$1w=kWjE&7!D>Lxg@9+jKfr+n2i5Le}jLi`ZH+6zN3dp9B z6WBFyO~lF1Db{GE18%r6&8x!Fv9rOPQJ4Zzw18V}rY|cE-&KNNpndc4;gy1Tjr}hT zzl?G%J*EAJBvL!1B=gXgCocz!2a1O`yN8ws0vXV?esYmCI!rVAB655VZGNJ5po3rh z5w90&7&+;h&j;8p4vR>%-JlA*q)K^_r$Vs!GVd$7m9x1>L;PePW;fC2SEu=_Vp#QH zqsR+49ttr6r~RhN@zq(ked^06C4UA^ zbdV2^{ig5Kliu;gv9o#7tJpJ0eRsobWQOY|t^7SfeM+FI*>iG(2Z>`63i2ntZQ4#f zR2(DIUt06&B%!a6K`-rdMv9BRIJ3QbodZ$&EW-ryHP3zB`?g2@kvbJI|BkY-c_}zk z{EWj(Br89%M`V7si=il#twpuFL;S2Chbya8EP&nYj}VB<|7+Gwir3tE>vQ<+T%6H$W8C)n+wV!UNsPxaTouHHbw=d|tH z`CVT-TGd2FnTp65mt6fCA1<0*i=tgLAi@q^H#@M!^>jm7mn*>P>M98^4SHP0SP{_S zy7jV3(gEI5>v7RV;15%ajoPV0efAfoRwXAQ8^c#9Nl%v`UoMdz_R4eyeAp)V=>xk$ z^QyEK{LzUm^X`2BSMuA8$|0>mLKVaCz8t$TZ|iYuZu6V` z`X0Czk9r~ee5_sT&)6$sTcRD74K10^6LT>@$ioeWVMhmSr~Ibr7uY3l#|qTD1_wKV z3zVx`nk00tN%8UhQ|+6{v`P&-s_ka>7e3)ki>fvgPP{iS`X|&FNwDuOd22=d(GFd5 zK%FY)$j5$R*KW8rZk8g^NnCOzYrczYvXwH6Q)n$U(CmO$ZG&inTR=qp9R@TwJ%d1nJY{@M#TDXKK3f9 z>_o5IiQ2cu&>cU@`A_gkaf+gIbLiKah#^jnjZW(KLE#Ul<+%x-w#nJ3RTXSE+p3_C zt$vrGgG9dr`A9&{#OVutFEBZ$!r={FTcZC7CMc8)939hTi3 z5t=^w9o!;5^Hp}zExyw_Gp#WAdZ>)c&O*vW!WZ22JA=A6K*8G(+tqyC)YCPuIz_s5 zPqr}wGYo96Yyvod^ly!0Wljr=pARO|>$3rrn~c2qks=vUD7Q;U=Ef{G`RwO@WTX_k zj~6q~xXfhP1Lbz{>=7y1)vVj0M@kv9*aZcWhd-7UyW-F*7#*DU%fB{EgONe8UmbL< zJl6ftDa=jjIX)tYB{;trwdocomW0;U%boh*BnNYq%5eL*jWIk#0qw-B;&3toN>h=l z*90ER_WgwXoo%C#=ohpLqH(%CrCx$UCG6TBh)MDRxH?Sad?j`@pbUU1f0V#J+7TZVeZxW2j*8U?0Lr^B%$j*|C=gHhqwAh97F@F znCoSilh)60@Ei*~ zCpqHXEZJ^zL3g;fUv+s@#_FM6yigl>*LW_m%0n$Eb(qylEF_2klFcGjdE}?SnG>>}(BH+i(uJagot^I!F^Mxl{^w*Onybf{)b@tu7@ z8y+e#g+lxO7sJYpz16kErO<`%5mjTDzIHvq(aoHeY&-(C5}y+u8d>0ssISEl!NxJ& zav1#$M>t<5*0-dFbxlsX!H_Hrq|&&px(HzRsXC@D*H})B|773Tohz@F7Gw6 zYM9dEe`AD)-OxoE7~ugT0Y+6SNu!YSOs_-qflTqJg$3gs1M`~sk8D?0S75pm$nA|M zrT9}1GYfSeqik?y(neksLO)y?!awPMBY_u+pPp-Z_IB<#ZcxuX^}q3Se_}ywSX*2! z_6^e$_oh|J`K3D&*;`4U)j3D@;a#EI#L$LB1?`Fi$92(m)C{SWBYQeS#k!n)=y}lZ zP@N(1Ix4Xd@3jU@h9<0NdKT7QLs!+k&izr=jltN&rFVEYpHrI{t?Lt9y z{PivkjyfYsRA`E};P&qYsykZ3{TFK|jeE7gC=*{F*r8|K5A;jY#Yd>DOgmwki`0ff$Zo z`{{#Y6R{xtA1M79T>75n>i6d54|O|xN<#X~W5TVs31Xwe5b)zO(MC}{B;UWQPCydx zS=vMN;OrnMrJua|@fK8cw7P>h1l8bXm-jkC{{6Wp@cN_h&qISkrJ>B3C)9A2MhF`x z=#4;j2Dt8=;XX1;ivC7!2-@gTL7ZHgz3+3{0L?ON`uICfBPD%M{B{`7 zt2S5pMxX%_qYpcR{8XoxZTFFn8Zz=2OzqmBVpls6r@v19v_Ut#6SE_eMdDy{-80n{ zMH+Rc*Nbtn1Hc#vP`OSgKRirH-Z8FD zw$oKyWmSi$mbVS6y{X>}9c`z)yu6IA%oYfW{+Wen-5NL2c{ z&AC!sU}Coavl8v=BPv=JPt=8CYllPsK`C2AE$~|jH$Vp(7EyOHr~Bl zRR10oszQZVVK2y35u7e=ERu&6C9xREm=yQ@OHV*Zzf+dsNkMjKhgAr9l*C}k&$Zbd zo?pprjGi36*8cgNhnhGhIv_wS3!lKq7povPns%?KYVDMSIN=nlovB|!NgLO3wWE^daA_KW z)!sXJ#VYX>DFCCe+W~Js_E0geF_Jwzrt!QjI)C+JP$RAo0v;X@UV=5-ep{vDVNX=- zI{oX6sKvv_fnEi-tj&VFq|`-Wc#=NPPUIMj2S~erW@JXNln(r^W$HV7(1!TN`)q3Z zYe+^(2Z#>lB>j;7PaT>^=dlLi?u`EwnNA9{Na%=TIl+?E`j>lU8ypK+-{|*7$-Pi zk$*)sJ5bE8z6+!hM|IfmQCth>$Re7+T_xm}H`6!~*!G`&MtB_BcYs~`%Z>3B%D1;3 zJWMMH*GsVsnT;P=N_xU>ahrWkUj_)M)oqPyeo=S`5Exdt7!1)tzFs#9;+BG3vm|rL z$Dt!l;om|arn-~;wYEFz4Mo>OT4>L>Ru(wxo+t<>Ke)4J3KEAo%6?UfPFryJW#yoF9(sx$GIi+qnlVn3 zKM!o48# zXN}Lab3@9Tld8CItks!9-QWA=wO=Q<%0$TsxasGnOFw$k+iKELMA%0nPFC|Xc6}jt zql9k1kmvH58o$6ZM8C@lK5hepNJ?>}5bj0YWCMz~Ci*t*|b|lx{nGgbu2!@ zOkzjh*DjGgh%dGiyI_2WU-WI7-IC{{QcZ6o3z8LQGDJ1jrO}6^|!<&^2yZx#^C|lk~K@V-n;{eZc@UC z_0OGI`6jJBaWyEV%3f4c333ncgeu8;60ka$Fz3C zjSgG`yEmG>!q8GJd*s^-tQ%T_2T!42_a6UvRCeBxS*~-jHM}YoAdtARWCxio+wyMO zmG(>y6NDYt{3cn@Wec2*T&$^TS&ik1ocmpPLM^B`R8wIYKdQEB|0cCSH4+JCwljNm zsM8B7R^{^%WT>AW&7ZXWau64uec1C_7xn3Cp|<3~Yi+`OR!tT&2^X>?rNBc|LvAT{ zG&GE|Drsb`D)OIS47=WFqjiPZ^j2s@$l66-)Qnub5btux`cWl9{|qzjD5qY(@SSY# za>+Y-4Q8!5Z`NWXMR=OD5D}4>)=!rn$j$v8sxX6_q7RCQkr)@r$;m)ij9y{n-v*BeVmBuQgV_IHM26rY02LBMj5e+VMA;5T|?nS+{U{a z6F%*qxsUq%z2js8I<4^*`2@>!cYAX6Z9Ht}hnnt!hyvjuX8VSKMQxg>p(KMl_<~Y^ z#(`!uzxh6nBYvgYD@RR4WbsM^P%fAxn168!631wy9auR@_IQUJo^XWNHeMCWhG#%( zn#otiK+Bi3u}qwNk@8;>$qOHGx>z}}OgwKN){q8LrK~?}C*Yu^27lI>J3z!Bqbh-a zb5B0RWlK|JzXfUIcX6sg8IXYr3=8}R$?*|*K`Sa#YdjCUr?FdP%SD*Mma}{&>gbYY zpgi{VF69_h0nfMs8Njv>*l*l0(`Lk$}%4C7@lSCu$5Tu9L z3&5eB5)kie_L}dF`Cuc9H|+P>TM5MRC&2n&JZj2A0RU0UJrr00tw#}PLav$mRLh#` zRX|3Q(E4Qg5wJ|@k}-C73F1@S4+(eBzgf`foGvne(rF#ekt_GP_2%G#mrCn8H)m#M z&Mo|RE>FJ20ce8Ca@~awSgmwLHgep+_0X<2d(Z1MqcRI^)Q8;oOse;k;QhRewY%FN zt`mGK0q^bYMFFWUnAa9zPZj0BNn##g7w@pHi}$~ zB|JPn4t(Y4cz7HdrsZw9IaVOAq*T<1`;V@b;2;2?9!AGTZ{7sh`&^+*XL_UNV?S;= zW=ph5R+#%UpAXz&jj~}K`@>VROzgSp7WWZc>W7U$Qfd)r4HO(jJ0<^H+L4GCBsc^?0=B7@-1dSBIZ zQPfc{BFO8$;>b~mdY-TYNIP!F75SZO+mmWu(1kO@lP5EJ{kj?c%|s|T_uRLM0d(im zF-rO5-?sk;YD|?EISvd^{2lx{_n4R%zENu;&+eX{)^f8>F7qy2026Bd%k&>p!vPGm zFfc0qKlEt%1JFa5iiP4%e@8KaZ{|wq789QugaTVHS`iV-{#1d|ac&k*M@PppyBU5F z0Hu{83LMw!I@68@YKSREyc?H<0bJ>oh$B#n|2xhGIHL6Z<1A5l@2Z!`Z{lw1oYu=c zj@|+&7=3~>n^9K2tRVmf1KXOw=}+NvVt#Hk^-2=L-_+H7KR}T$K$JDgnIhsIv;v2^ zn8HatSw~3@@8-=XfOmm}v{v_d_CO#zTRkUaScYWT|Tiy#tnL#f!c{JmYC5CO{S(-Ui?yV>~4!Kb3O zn_d2wkU(9UFOaHxzF>}hk6D$VSG7dGriV~T&Yz1=LOxJxg_aIozeOx2uaVkf0*YbF zc*!sL8A$GU*N+vBtU-}y{}=izfe08Le{ZbTw_5i-R4KGF;>X-YPD zuJ@Dkw}FQwt=!aw-2~!K?l`!Z0BCCKDH!43kueyX3g^B%KxEPRK*~E9VYCDhx_cEEvYN;- z`O&%Xvs7;EpZPcc1LzBzOTN z@Tmr#oH<;$bLj3Db&Ma-3ROD@C-*={RdwJAz^lO72m)}@f<+)2f^u(1!*9q!04cGt=xJ5Ip_|Irzwj*(yWs$9eDWeGjSz z5Npy@auoo9-lYMeKrT0c{U2h1;Tn;xqb>D*0)N5i74_19w`Jg81dJM}9RsA1En`ja z-?u~p0cn4AK)7>{t^M{CnBzF(hU_um-o)?<5dwrvgsaupjkf>z0{|Jf;V8s^Gb#&N z@JW4J);Hk?oNQGK_yFbs(P|d>7;pBk;P0CQ!xNqItW3Y3)MMc5PS)zG1<+I7Cm|7_ zr+m1?KZ~;szI)#73w|rNodbrq7z|osk=c>vl|((3a#x@@F6_FxbQ-|Zx8+x!07{yH>b}K`TZm( z05aH{d!Cbcot&e~!0dw>mG{i(QBB^Qc@$j`lOCEQ>n8OYc3om;?%0KPUew6VuMNgb zup&~0xtdO`i@yv>*f(beQ!-AB|lNQq%%=FjZ7sb|dOP13mQQL&@j z3rU~WIf8}tb*=Y8hH#nitB1RT20n;_-uJ5m{B(o6i*q7Up7(30FAJ{s_l{*!9x70O z^6dRAX>)V){!0~==G|}QfRin2YMPxhZntT7Zh=|-c<#HwmZyz9%x-38&D-ONrX*yB zMv2rpheWps%yigPLwBWOxHQoOXUQ`&h~I zl(%rfC}VDOdS02gd`S=Q25b22!rdEx=5fU|P;~Ax$Gic(oN3Zk2>D&Cmjk*yv6`;5 zw$SjnEAiMQ{Qm*A>*sqdbbmUw9z!dZ5;j}zc{Ns%y**=y;!%^3| zkG-5X0(P_4jq2z^97DZZF?TJo|2Yd_@S$I%PjF}K+%f+nrc|7M;M|8{)6VqYXxC2S z_ASDhhULlO%z8B|p=v{~8pZJ-*C!O-*T-6b`$Q=#E4wkA^H8_4jNvhBK_@+&A->rl z&q@wKK?^jyjfKMGu75r!oX$nP5a27cdA~n-V0g1bsbX&5HlFEJzMh|nOFlC2mR+F_ zvYz2tNfQJ(>9hp5EAKwFaj1+zs76LcqGMwG z($XkzE`M?~z7_}v{11VvKDB#e{(umM^~#SidoC$EPek%ot6LcYlX!(U$VLqF!Iivy zSphaO$6Z#;zT_ZyfO4ifa`+7m6b6PKG(Q|JPcEJMabd+=?_Hs{DC$Z;IF(F?Z<=8w?pemU`5ma&RyZVw4&o=2Vj+T6nBYntDtE$agA->f7 zds1PZ^dA&`e@?y*T6bh5_c=yM^*XQ1>~H^$Sxx6t@Qyz}9{nQ0|J~Fr;E+vM?8f|eZk4MuGEF;SbZ6qMD{ed(V^ib_&-CHz-EWi^ zydwx9Lq2_JPxF7EUGMUkfk%#uo=@U`T0+grF`w+dteJ<7mMX~v_tY5;Ykp%w5<{pL z4ZqzUCr*5I6<|?VF?=0`PU`GAehAf)9jbA@y*`C5_a*b9;ot=I#NO{tH}T;!*C4M% z#UcmKSL8S-XCK#jFd+~X7&XES^qW2s9dGF|9cKNfp26UlW3^(H^~Zp<)_Lcd@TgGn z4s!jk6T$s4i0T zYsuQKv!p1c>qFv01_3ZGBXeaVUqm2QdboE>bADP(@PJVW)=*fpOH8ydGoER(Z0O_z z#CKJlaq7yMeRf;i+}Cs1`<0G!%t~q9i!yI`47WNIu($DG8oaEXH$cIQLoY{-Rh>7R zbxO^hMJ@!QbLjBOrwtEyc<_~j1NWF*x?W_wZ)!%yE-pzJIO>*kRTZS^NRN2i5bfVg zQP&oqr|RyCW$-`X{2FkF3xhG?@17MEu#j`(VfY{&hH4%9lhCIN?SH~n_4FXzx>eCN z8|3`OC5w%$K_@`z2|ql_`|XVg5b}|hkYM!iV<)P;a}aTWO&_$4YC~lW2sS;juN@-> zN-2IS{*P)7`2JNRtDWcG-!WTdjMRP zSnja=f}`{XK>8cn=}q2AS}iZo%lKI*q#m?5FqC;m%_)gjaP>)K#9K$^e~BPJ9>yDkhN9{zth2 z5kS*IsSt~M>(S^3_-FR}*|EPC9!VM{JHj^9ysnQYNV$w37$t2nK+DJgrTw?6b8+}L z;Qw2lPML`WT6$V(G_ludrfb;60pU=Jv90%+gs`oWJt4=xS{?v-)L0E8vbHGzmL7ik z;foE=eS#HsNh4Z#IS;?>q_0Z)^DM>0r;n8a?}{EJbbz7t@+=ZP1ojzjxM}Mez*Pd| znK5u3SEzw;F+kVt`W-I(8qDQ?%Z3C1)wYLa=sZH2#4U zZzsXNl_W&FiQj-9dU4SV8>E=yG-!s^!Am`Idvymi(w9uHm9mkxOMok{2u=Zyd)Nkh zN@;P5)>fcMrbIyIO(6=fZDGB6J|P6`rPm#?lkOB7JKSmheQfb?7Xyf8Ij(f!c^&l& zxUa;>4FN%pA5=#eaEWy%g_kl&e>6S!eQ&kmM#QECiDHDrp;G_f$~)LkWdU@I?Z<4K z`brELdi~BR-k|8_4+z9*Hu`}VhN|s4{@x~Ht~El&V``N_>Of z@NSVuv5yorAAGh#Dmu>6n0g>bwzAD8lfSRRywlNo1)8+PkaCw-jknwXNUg5IukXiW}=N&jq&H8)vN@-CvEv!830? zFMnB{(w<(&Y&e<{1UQA$0^7t#rqUdK;)D${xw2!Vbb;5*X>){XSFT8L&|hn64X-%3tdgcfmHN z_0<|84NI{<1)vn-0{4TAKx>v9!ZXAl74mn)<6Z7l3lO99d#gfiSsroAK$)RiPL-qGVpM0?erfOMKeB9%?x9 zif$dLZ}_Ogv7dp(@ruD<;JzSZ=>J=g@kq3wt-8xgfGkG2gwAI4VX(sOi-z0ZX956( z%D9Dx<-^PAK**In?&&z6k@X-Qd!ZP{Y7T!6Xj4D;sO}B3YtMX5RwcdNIob4UGv>IH zcA4T5NVxLnuXapz`4=O*1@$AA8oSAbTGs3AlH&GQs6a7A02wuHiH)zj>StS4iIYRZ zujVVJS4%Fz4l*G%kEEve#7(h?ZFPfH5^P+zsQ{iOQ-7K0%@~HK@O1jI#^~FbU*CN+ zsZh;ihNv|ah6eR`G3XQ8jpsVKf0z23X{MB!j7fZz<^0vzwiygI8mO>f?dsG#Z~j^` z7Zuzc#?3Je(iR+?h!mwAFOZdlJcnsdz%x!ejK2==ReXj*iw(*(0m&=l1Dr5E$TOSC zN1UI<_X@a>z{*b17jruj?geU5(q)? z?J@)V0f~!YZ6J=C(RU9Fk*Ss2ofA67s`4a;Wr@_mayz-I!W$PF}8_K4aTHTQryS>GOA?yV8C8 zygl+t3FgCuLM$n0-;2>_R8Ki|IrzZOFPM`5F~QQsCf+E$ygeUe``RDEg9Tg+P6cQ=--ST%6K5Z9Y{Ae;ERSZVwGCBwi=mY9O$y7 z8ez;ug%D^NPxd7T^SpYW^tV2Qoek&8F@fH~hKorKsbKjv%@CKHHdl4&OX} zcE|8v4DI~d%?jA1*hpcw;Z$U@hFusKeASQJq{_{Pb(wKKzi2n@Ge5sU{xjIt+_#z} zYLDj_za`yco7`?VWfx_|n*z^m9aJcx^MSDjOqH-pWE9_u`rPYm7HL zsvQpIINM3-q32VVt7?RvOixU-UtPbVSr)W+))y_wZRR|*gnnj0ib29S8|b$elY{%)BY)x(FAHh z2%^(@Fbz5xz+CjQzlzduKXbq2S^EI$Luug2&GhIRp$b57Ku_W-dg5A~92z&Jb9jZ) zbIM8T@RAp=c9Aa%p1EC%xaz)SP^E(V57JRC_aQNn5Z?)miym}^A+6Tt>H3;R0-PY3 zY?JHH(?ULRxRai5#l?in9w&yuHhS|L#3Eb;s@PZ-{BwY)=)%#Uc&9G{5~dk@axhD@WNB!TzDrI$0P0$SHrt7NxlQ5j z3@;aahZmKrwEI{edxy#YM)T~|6+ToR*}GI?IPTQ$MXEqze7VaxtS-1o_7|2 zTWi^c=iIY3ri3pzqh4H?LwNY*9$MNIl7=tUvb=gt6fp_J%L1UE%Yo{Umy16_YSX!H zIMMnahiUy+LCTG89Z48ak5YhM{c-X8brCjFPJhC@EeZ$}__ULY>*dshY z`ZDc+KfVUm?xtVd-S$cGqLIohJXqV0<#m%HTgQ9L-tLg#?wNjQ<$4sg^{LqGVK%ZX z!>?P$$9044y+?S#lLtH}AR8_pyAp7cUzFTuddqWWCEZFRnVartIm~3;%PjH%xCw4}x)j49$kT$`~H5&|CiG z;vi2ZsGDjP6NLWqlXMx#nzeb-{;|`dtH@1zp&$W2X{<#;u5sAST)4b$a3_Z5ax9jO z1-=TC8mbKaM|a9d;Bq|XTT&lO&GAxpImAe~2aLw|`|plP2GNjXgCYmUFAHb>$`eF6Oq{m@YrbbFOAqpc8NtgW47BW$__H;D@}h%X#knb!)Jbho+><5a5E)e0@M2v+O4P@um4q*6i`dJ~UHP0#dWGdqYoW zZ~fNkcKhIy`{`{{h*lKrCva?dP$4=p9_$zm3t%G6e#+EsL?z3pI*+?rsBk7nAy%$%AE zPHTM7J^V#mo{29mi6R+>@FiY>BbCMelhT8)Jq+@Hy58$y_6ir?iAQyWS^N~1-4D}! zJ!)frj~^Wj*_SQD6&>EmxGeJVUYF{FMBDSeVq?47KdXmZ z|IoRE$Rqg)|X@4EoDW$tRrKN_5 zib^wJ^rS~iY{160?+yBSj_3LP`}>0($HsPF_kCVxzRtLo|8si=wEiW`9xXgM)xI9| zrui{$bznq2%AMQF20;$y;FX=o@+8Vzy=uAQ&GcZqJaw-^*s+tY{g+kkN^uukVeVF} zTF-QTs`NE>Dc(eCemaY|yjm54;O+mlwQxG(|C@DB98~jr9<`HNwyJ_#}J!;=SC*1y?lpDD%TmPX`j@7lnc%MG`sp^X3k@D&oRQ>0u;H}#t zj3dG$D0=qWMx>fQOkdXthRpDPs$i_-YO6?1`u^r;@I|0y^5SJmntD~#%oXhI5W$f* zdREJVbtm6}3W#G#XM+{KMdaP9!Zl@b-t?#(w%AtEgjG;t z@7j(J@O|AUD1Rt7q(Do*h9(4Q%-D_cNSNR5EW~oEX!RCSZMhZ**oshnFWGz2qCYfL ztCq2L<=t5aOn%k-2lkFy#$78XbR!Bv6teX`b#k)shaZa6axKX43FBN2N(zs{J)(BZ zEc}2e6+Qw#((hI>V_fMOT3w1Cp)4koCXE`5a0Y2%y^*?)vas`H%T9^%E^grclBK8^ z-L#u+B;_S{+%m3mCfXI!$*F<6_QjPz)(+5CpgdQ?sa1l+U^?9VVoBc~lQ~7g1dr({ zJ=>cpIyv5TXtDnj%t=sN5F`~j*Q;9?sII2b4YX|N zaaoV7>`%B3eH`**G zDRWcTo_de~nGsN5gcR$83o~|TC9bD_UZ|{-H*JO+_3JWTFNB_hKyTTO6r%j%=fkYs z&DGo3G^p>)eFr{yRdMmpNCSEJ%xjL&gj1;kYyi)n|ZMyzzLc%=_emm>HyzU3v@Hav3xBpA1aJlENodcb; zO8nTE*rDArcP0JmqiG$omb;V-#5{E0hZ>fCO8!z0# z>3&8F&v`wi#Nw3S@96`jyw>~iPuM*DGPGXh>FLxn7$gUO0EC5Bp*LnCf z=kCR=^(RoiYPtQ&$XrFuR#>cWxc=tSFDmlyT6}po`mS9`P6eYXFSB=D(11>< z%U+AHi+>u=lNmPI(&{#BpBP+UwlSTi`>J#Kn`=TF zuT%ZosgX9G!|O8X9zrPAT=H*CoN7^f2Y0o`GbfW~>C^XL&nCHPJZ<8g%5bT$X;=Bv z)i!~>xovSPaQ|*L-Eu6sIbVqlLpQOaY|pzm(xr5lh@8)UrDnoLAQCS?IWLlgTjgCO zd3zOe?;hj5q)F9}i=!@~_ZpwLx>pwvg(fFzF^(2Zp^q!)s^l3qL5)v7=9N9sFEiJd z0LEeTzWTw4;e;v@*Jjf*XSKO!?6}ic1Yu0iOWR`6ATO9|*iPa|TD7^<PeI@|NVX}vM?s$&wZH1+T<__ReK8Zcb}(Tt^W2(1 z5OEfvjbsj}8y%1&vlcm66=Eg0;sX^evSsRwggWbVbwt%RWSJTzX@|wPa|toCk^AgZ zO-c=BtIorJgmrOoX@Nvyy3B-SOWhVMbz-^Pg!Z5xdeJKD1Pc;rz?wDI+6fZtb*f)h z50Ze1VmvbqhtWKtML9a!uM`g0JN+hO5V@E#2dg<%xcldspbqvBRFI#m_oH2fqb}@H zy#!1VDgSVyf6txclgurCEv7b~W8>L+k>E#E4d7dTkD)Nx*N{`TuCZ}aSr=TiDB7Y_ zDZr_FcfeS9hGfId!+bKZ|0u3YnRYIzn4UwSv>mdE`~DjI(Zk0v9LAb4y@B(5ofi3l zfa;F)E&o|zCMRhWpXqwKzf#Pyo=IJ-Ez55~`R((>vmEfBiq_l|@olCIe`h@((be~i zM2ukP^25w=$*wNfXIi5c;@D4bXS{JZTbPkd9`!bfqh4I{NalKDI4tuw#q}M zZX1l>1&125I(~gAiL~5hsRgBZde~VdTu7_SFU9Z z4Kb`jYw5M~LZh1N8#B9hUVn$iAXcFDbFvF#D2{zkFYXI~uziKCGF9O?)}lB`_q zCuKw1_IZRArjnvs?tr+4^Jse# zj3mv#OQvXR+?5PFg~F5`NIIMviJ!Dg@dj{<)d9n?a#kBo6cCQ(QBy~={kTSL1%O>Z z)gic#jz+N@_~#k*^_SLd5Bgt^dK#;Q5?>N?on88O!qSRtiUehT8(o>-QN>SmL zO!&qz*;WU(R5yF{#|H;vdLgo_ik!d5Iu=y4$+2@D!_R>3({;x_zj>k68{_lldyb+c z@+)QR#1kTFFvNK*3FMT1pl>NFL@;aG8A3Sgzw>NkF8AeQOV7Pm*cW&@`48k!yB+J?diSq zM;qm`?D(#X2S4P8kKWG*ZiS*N5wqYDbYW-_D9yRNEuH9(fgBgWg5xPGVxy>wTQbWX zevy557@~x8hz_)nYifR6Anbx^$LAbMeA>RH5B>~T;fEZuxnq~KRvRRAgt~?e5E-7T zNf_J5P{{8jC0Kn0R`R;pQG-dz)`Xn649;cF7wrF2lxK@@dSpFzh|785p+AY9;GPXd zq!k*N}T`KPC8 zko_6NRyWK{ZwVI)hLs*c%GQycy>+>mH!{*Bvi@7cKoWgGxx0S+<1=OBtBMz(WpsZc=!F?jk@uuNS}--aIFvR&t&Z1W zE{urR@0H^}4Dcc4ykG0l>1U*>cAx_lm8o}3B@otkyuZb@rr`Iv@h2r#F7?40q}@krTdp2q zDf@f&7bfssnXj-m6jF@1*K*L9K2;VKGBhaX&W>>@QTDXvhbxH6(rwBOf)4F z$hBLxT$$dMu5XS`#+nxX(HyERJu}1&EIPlhl09PN9Hvc=qdzC|^D|xYUFQBsMKF+( z?ZiyGww>esC8#} zKa{|nRKtnNOsZ0fwHle5i<9bY6OZvrc$VhS4&qSE5oQ#IW#AmH1%=kcT_Cy4sUJW> zhV^qCNOrmP2fJCD8JH}jAI~o5VsNGO_Nxmd3uhZkhFuWWnKZn7NMEwa#r$xFf3Dk_I+h>lybVKDhBfA$RB`x7eymhTWTO zZ=?p?6vfZ{`ZJ2eKMOte3bCw`{w30$Ub~$~1Vk`2?WwShD7{{q% zEQ|OxuEV7TBRW!dDWBCCYcSfx-rm046;Y!btCuCE^qGSctj0k1mf|7_{w3<$J!euV zB*|B0xl-xRO=eq5t|gWlL(0RRyBgBtbikg*UF+op_U!W4FyL_?9zG{&S;!8Ue*wEa z3dpJZRQ+S1mO;~oIs@2vzq?ndp%CxN37LSCmZF1^*5&2pcW0i1p~EI$I7M=wpRF5bc(gOO8B&M=@*uQoFOr~&$-pz3TzhfG zQ?C@W0Nv56p~mekO2F@iZZ-_b`&<@mUA~6iniBmW?IcTbc?_y3MuzQFgO~(54z71G z(Nu*o`b4Q;7W-jUnZvv04Bv0*Bg4{jYwHsUcyXWAes#-c>tA<#?+(Y0QN$0K7TLLi zc05CI7E=#GF)`cIXL zwg3%ccG+)!lW0&Uj^#h4@bj2|%3Lk_n&Yb}+0`=rJQyh+=$UsO;K>v_U5xgtQS)4X zO)!KX@Kp)P7HH@F32zK6WLeoA?sY{}cn-1GBgK3`K)cZx(8J`nS>3V`!Rn#k;xRXC zC5+QWY36a1(+cHQtN)C!uA$z`4cg!ZIz;q1z=t2Eo84K1o@GTTQj=Xibba+LAE5OG z5@CU>M^G5`ZYreNxRt`&wyp5C)ydeh@ycPp0fG^Bf>Q3B$C7R~g&@4MANq)l-Il72 zBcB#?FWlOh+L#c*kQU06EO}Gj*GEEOov_aQ)}S6$tH7QBKd&_!d<)8$JorxYd<`!WBlifBi>Gch+8( z>&AddvB!%u>2PGz^a0euMKZqYZ*hvDn24#tvePT%lLo3bhPxVrPIwX4%GUblLs#&@ zh_1>&vB<}npV8o2p0W65-F$fi{x+B=lR>F&U{W~c7j=X#cS|8xt;H0GZ?=-Q>xq#Fxyb?dfZxHZ%<&?F#Gqg)nkpZxNna`Mu{bG7h<{_Tt+Qn%^gc%xSfyELH$Ps*6B2e_`DTf z@I=o{c5G@+Xt}CC^Ha<#zKQwCy@)MeSmvrokFqReak=s}juXV{J7ZlP zxD}Sk911s8Wu$&2it%lVqC#788QU2>{vXAEz9AcCoF1fL+|^m>-2NB7!A(2mmBFB= zU84pN^r>yTxFg)MG{s-gPZqDKV?AFU>%n|1=V%Af8C;vh(jWhh9uML{N}7!3VZyRb z4mWEhESaH`R{`}N3n@tXx8D0REg1zzG+qbxN8K0tiTbb;z8&*0G@SW`{Ou83+bD*K zfoZ)?Jk%?$4!(2hi_+tlllNNd;dp3S@4WgAu9foiEd(lbVkHQya75b_tqXjP+G28f^Y3Az_!aYVKY{i7 z051WmLJzFl$X;92Nrp}S{dG@}1TVE?nZYOw{OIk7;e-F}@6`lgZFRS-@N)`;0Td9P zbpB8w0)-UW6}DRfE9`K%Fl7Uuk4y=TUHNy1*UW$$e1XeJ;R2W>3Xs`cCpl}4V9G=O z?x-azfQqxNVgk>BAJG*S+x&YdMb10G+V3Z&D9zAlSd5`E9J+0R0=4tkI5+x1C{bK z0#F5*at-!$FCoTC8{{*eZ2yXbZ+oo%d?|GsO5h zRU^i+HrF|Z8{*|_m5e+PziGtR^Iejym%;#zt^J+y@>|7R+Kc~ZldR6@Kh80B>6~=l zz`G9()k_J{NnN~pvJ&93WD8pLPH*3%B6;yEg{~~98O2JVj9liNaWhsLTQmx}GXU20 zo5f6nc2Fg6u<$P_*q<>Yn!?}M=EOju=2Lp2rI>Q?!iuI;(FyI4X{~b$zi6yt*G*d1 zt8Xtty~wd$Cj*|(^O}t5{;|MSK+1CQTaJ8!f~JM4v01qh)T}8_n>R7N1nN!LjQvrs zd%o4$U)}^%{Z&+O@YGKYFEuE=IMq(hqaG2(kafVbOCv@^h0{Kb{a&Re@)P6#zd|T!%Ts+g3 z(+@&uvX zq0hj4&n1_!V$!6Ev0KS}bDr|H2dQNrE!rAw3r*9T{ZR8mH)4&EG?)pMaUoTdnfq6| z%Vbx=`Gr59zS*tnG&@0>N5lFP*y?P0s(K=6U4cS=tY;5#BRWPX%eI08 z#pG%fmAxk{DWe+iRNFNa*zolW;}gq|Bie5orjGRtGYgC1i*|7X2FR4h=J;RNwox9i zSTCy;r0Tut7VG!{>jL%?jtI0%d7*`lLiM0e&3N@6di?sgyQ?nDEb<)<8Ku`NVq@HvST&GK5R0O`e#EmR@woN5Y9GR-eyMa1!@$kv4 zi<;=_OD70c6qEBxkDhb7G@olooQ!K1V@K1kD(aEy)ftlVRR^UN9Jmm_-4y)a(8R|Y zjp^2q52o9xMA0W`p3khP7PnbLgm0%RFj`w?x8uN$*s}`A=n9~8`!YYCZ}yo1D$@g0 zHFj)m`kWd3c!7ut>ClGZfsx<|(ru!p@$`92Y2{Dc@`-fp_3|eauPl1gc43f(os;I$LabVzh8w2eY(;h=4hXip0Fn3sL39jTpjJ;Y@OekKCBx(@D^uD|0Q&{X5_9Ur)#oGom$>!D-7S_PW76_h! z*E8QE=WT9CXaI%G94{ zzPL+!7~RSBb>PQAtDciRk~bq`o$7EU90%hZGqYxzlkQ=A0g}lL0=XZ_r8Nadi0I^i zjO|F~7h|XE;W046>UFEsr=1VPZ$b$%(UpXf@N93#p9yzeX7Z~iN^1h-!^GM1ck8Ga zQGMa3v~org!$)9TH@+?0vx}{J7ZO%bK)mrz@8)QZ7078QXU@ENS~tEZf%Pub!5*R2 zr8C?fsxTwGS`v}e{w$OCR!{gb7dnS>IJ$C9tr!Yf!`EiAs%H1czOG7OB`<+D?M(4B z`b}Ps)d{K{#UoA>Y@r8xyC8fAAw%}-tgpAHjnn`&HePbr6W$o>!T!?c=L?KO95vb< zG{Yh5(gmDcqN%56hQn-Y#o1o`pZiqW`Ol-cx_xGW=at83&-I}t@Qz(0KtE$#W`i(z-)z){hj=-XmxfKc-5^xGTNz!evc!bXc`uZWssu+Gd%}-{AG=0M7?@;V-F2w04VrK2d#JyCctv zV`j;0tQy5*L`@y!A)!lP_@gP$55{r-S4_to;8$C*MymIOu=psN01}VZg?F$@M?T#zA zGrf7>fW;ihex)cHFb6Pe@95KvOrYR(ne@~MlLo~GOSsI9NU+RHN;{S9V3R_HI-}t3 zL_@suf~pmG#ux34Lp;oy8bsu}z@~&vEBWSjbvKvsjBm@+cSX$yD7X-JhL4(8mVYKr zxA|2b*p@D~{(en0*6QQ9d+R>e|Kba=|Ngm9&y4|ZR#i66<=pI!J$a>wRA#(Ce57+y zuinFBUD}TsXbkPSE$}2z(ff-bJt}kEkFo4EZ@|_~tOFlodifLXlhVEyVT+?5;Kd6b zC6smBzH(Ikz>MLVAp_b(pS9XOtKEoPT{B79HHf(NO07X`V*f2PAo`yC{gPaB@iapt zU-PYu?bsijw!DEEpfUMhca&eu8K9d@JaPlOOblR61jSHC1uIGWaOR((+MA4nw?%Fn04Fr5B9=O3UMtO>dn!I7v(ZAEq6DSr`~}`ZKz2@vhN%-BRF* z`=b!tMo#ey=Rk9BmvGVPnZPF-uze^YuHSjJyE?XUYoXM)OgqoBC zl@;LDV-JUY(O({$*N=zFO?eSD?fqTX+aclvzYs<;r@W6a7qAQcERdbj{!;Ad-KoZhA?`?Mjvgfb0C4~U1aHA3Kof~Xd@RxMUIr4 z1|yHAmZIZ#CKgY8l3hI>99yq`?`a&_Sjc@|zK@61H~a3E(#T4>V!H1I-(K8y@QEqH{rZ5$|XX1|Nu0YYTvc*-!|m9y&1I8C_uS&=_(O zd4s7u@%!q3?|;ts9Ec!MX|5%H8z$Qn`zc*+pGfc~!Dj3Dx!p=s)rt0gg(UPDTjYAZ zU}`xKrlCzERXzbo^X)?s_y1|pe-L;<&0j5ghT+CqXJxa(p^z-}V|1piCwoYhhpmti z&PCq#*2gU`GA!{IE_GtZr&RMw2g#e0DgsuDfVv6{)0 zEE+gE>Zf4u62F|BRoEr!QQLSq?sm&e0Gx6^lwvyfcT&g}C5Dk+USW*xpyk8I4l|Hn z>*;J-c*8ZnxzPPrgbZAV&zy%q`#KheK$OvRSsch_^U1Gd9VFgTFxFgh5SPgf+tlME zP6`f89o8FNl+Bol+o;)zR_tms^b5ENh0HynM>&ba_27(3Ad zGuiwoYtH5_3;6ogNc>I!AkbFT(;9gk)t%a%=Nt7!;A#t$KuE-)-D*6?UbuKyc(5Jf zy=K0d1Xdo^j3*4GNV{(!o4Z;sCKwfB0y$!2_D|_prtDQ>2iJaMtCOlPUAQ1ctD&M~ zO#d?ADD;r9#-Dw~(wlI%t*Xy{yyibBq^W#kOOXWMmd`z0%j#J3`tJ_jzA}jvt!dlf zi)EKH514(S_UP(tC=ov>Nq08vKpZ$iuSkL(75VkM#<}nc-ve$FwoS!~nz2E`L}t+N zPacKISxH+}2YTh70ra-XsW;Oh9Q>hSFM`TbAHis)MO*fLG_FwgaK990Fpv4OwPP#( z3IEw8>*p;r???O(8i7enn{KXLby0*yTj1{jm#HHArb?AOkb8xU?0dy7k21=`&F|@z zf8pHsZ%Nh$vyIHnGyHdE%T24z?~*~;$`u^?Z@432R@AH#|CJB?Kgiivzr05S)+d2sIBHo`>1MYXxtJ{{l5`W=qH*(_t44K!_;K~;qGG3!@&PA2H7U9W}DW7 z#QbSNcb1a(4&gXUgn zn4b@EeP);Xp&km6vUbH(k8E^S`Yn%r@Vc(XOm`jiepIP&z}R?Q0IuA276j}3054N2G#AfR>@p3Z zlLo-!+f^%9?hXs7eYx~6TD+Wi0V)rLq{(Yu4?B++OGzl1P`%Cy+-?o9<3*}Y+)n#o zSmN1vd0^ZO(iBJxUYjVaVP#K*LI7*dao;dA&K)SXC3oyPS=H=K8kWu$8O#u=Aii1{ z7BaoAQFT`4g33 z%+|=K7fZg_Lubb#0x-KT>+N;)#A@2*!}w+A`M!2BhYfodTY|(r30e=x=}S=QNq+_4s$!nXnFMw%_k1S#>zOG5c0MfcBd-iO4TpRc!a7*z zEc07wT3yhPKxta}o&>+Bj9pW1uwQb;{kri$7u0#@X{dPNYA5{namn{7x034gqM`vR zG{P*zF{afe!BB5|Q~~*`xbiC zj3_n~D3CX2a0a>(8XFwrc0R%4m$iQ`weL*P<|3#(Ifl)6mdmUXq<0v6vLa9vT{o(K z6`+AU%HG9+;N?Ig!d0sm25ojFBZ1qoO&jvvV_=(i6Q>)rL2As{?@gp*KW4Q*;>2tU zJlI|~Th0j`YudL6ga zm&}YTNrlMy4XujHWKV$B-4hEhwVmexR$~ryw)FYFUB>H=Nl~1eb4I|Wv8_W~`#3n$ z;)z(x4Xvq#perdSu_$+MbtsHxeE>8tafv<#Ddp2Bs@Ew?tvdze9AlR%y992^2(Iu?F0Gfl?(5dubM1d!?PC_0ZF5mfM zK1QlCkXEktORZHYtI+!5b>mOqYWatw*0vED=vQs2U4|zDWY`s*+jT&}#1|paSM%!=0>R(n2>CobzM~zig5w?h*27t>SQTn9C}hl+oZ;LB zd+QA-zxfD{HhJIW5U?pdBBcnIJsg@i+FKCy3^&)bG@ZD=VUF49LP9X-uyt_GD*&$vq# zZO#Pf86rv;0yL)fo73sARM}yX5cjOnA-##-At62=<-$LCl(-g8`1jCi?=KL)8}2ZM z2Vc|Gkoc6>Gdu5UH)fRcK*{N#v)LltZmZHd!9Wl*iwix;laMO4&JW({(kg^oBzZnzumA$c3(qD0swv1t60rOg z175Gw;}-(vjq0}FV+OGVEpi3765Ei#XTf$^KB{@$TB@X2}leTpPs+=Sx!X z6f13QvoKQd#x~I(8u}|+Yb!}s1N(BJ*F}LK;xI4F5fduCl&A$aAmpN_70GKAe)&P@cA$EwMLT7FbCO=_JX3p<&obnXg+c`1Z z-D_LJ=Ksn*%Ci=o!XJZ(JxOp&J(_mQq4KU0A~vY$_<1Yj1lLkT*ENCDbJ#o%g%tLA z=GHdSNv`)U!Nqn{8~VpGoP!nzzO{WN27~W1@HPNZz<(5H6(NB18s7r~#aq;}8(P1C zY9M%<-)2%`Vj&Q_Bh17Wa9cw@F@EXh1LdjN}E zyu_N1EM+m;aTDWOX4)*Er1$GKYpc+yqg?5edxek$g#uN*r#>i8Jp@CdkP)KMj!ooX zhKL_;vS*)XjVTYw%Pp}vh;QJbbEh6uOaTUjE;S*Rme$+#aUo;(mONH32~$GgO}xUoha~yB^@U zo)g+pTRFFIV6W+K)&Yf3AgDoy>$9wE;v6 zwV5Z=U^6sm&+BuNtOE^RPjGAw;b3fldjvk+55c=0Tx=qKx0{pbHPWS(Y5}=0A_oVm^ zGrx85uCYU=`4MXHFTFq`-_yLfOtr;k@pjGC;V$zG@AZIW!jDID3T?i|#J$-m7E!c- z9FEX1ebA?c_7S~@gsH}KX_#5cDHTSR@V7cO`0QLlPrRkmv4h!FGYY_l-!VAsWX~U= zvZc*3D@}-x^FUz!xN|&4-3=N}c^JTkEIze8nKt{R_t-J|T026n+7nsTDO2RDUpvv6 zG+VYY(v@RW+42KO#Gd~v6(%N*c1q$0f*)(V9Nu^BHS;jL?j$IH{rSI@aF} zCBT4y*6JWpn(H=+K%#}qmf81;fX(?PK?N2PysZOERtth%A1-oK*snPv+|=$=6(Pel z=Q=l5PoCC#h&)9&kUwTsMlbt9EE zXfx`KkFUo1tKh^RLPnUc?RR;$c6XWcbE|UGc)d+umJSG`h{8swS^2KSz>;yi`QE|Q zuZ~>^HF_v2x8)dOb#NfSy+w?*7F(;K`~>~zFzA_t=zS>UQr1FZa&!j@8CSgtJ@-#u zG5nvZwve{=AdNojo>C{!A!=zx*jys-p%uk6vJrfpij#jP0-~dCV`73fSabXU2;EWs* z+Df+R3E4;phr5K&T4=z;C3M$1qRsJ9+kOh(T7ihaB6=Aab4XOmNH<+$aaV7Dr1wzx z@%qo+rY)fW*9i;fK@^_*AY?5rmkUMIlKJNFXTs;^c?!EQ! zBEc=0pDqs_9;}%rZ8a?rz>=1Rgube~YGfdwHhbp_P~v%3yG23{AZ0B^+Ye#by#&m> zj8g`sphZemgTATFdfXl!iZ>gNsZ>8|2{a;3t^QtOz`V!W|HilI{LOS$S}0EzFv#PR<$!USGcr4ZCFs_w0JxpUz?2w5y1=5(qYbu;PQGb{u0d9x zWa~2IdebaB00$EJ1%6%YJwl!6S$hjkHJIpG!Phe03+d%?h**8qy)ztt!#({qQ>3@7qzD32e0>fQjS)h>QnQB)2+pwK?mh|u5MITOP zC`d7V{9Ey-44d48_o93KLVX8YXl?KxPW;JgQFO-EFhwo=cr{1HB4an7r-0Z zOm*~>Ky{c!i46%}i?}^nwRoIZr9_MtjB5&{uc6a zW&qzVYz^4>5w zLd&jQ0cwGL%_gjFfwt90=;Y8NMuR8+uv!-Hr&%}Hp@^1_d_9b(_m#}|!b(gfRyT59 z^A^VdI0-Y2>0cRFbKLmmSw-8k(unChBzXOm=bIfs@}qf$HuR4d z<>xuB($xD6`U)gMArI9+H<6o`r{4*Z6-Jt+*mCtO5P(1r%BHRUbx!o-&o7KV-dh?y z*U-Y802)D>ZER+82N(*yj_uU_M2eqcbvUbxE;S%J4hM|a1x8}sZAd`i-6$}YC?Axs z^1CK_$Ivirn83-$8~!E_xTMn4$+!Pvv1dRyMZ0}uRx(T36B&Q|{E41F0@88ZZRH+( z8olh0fr>O&7plci0G${~Nh*09 zt$Iv%&qeEhmG)`VKp{;iFamKJ25yhAlu!|J+ta}T`G%4~N?YL%mk+6iu5Tm@)S?dP zb0h0Yt~0z->|0K0t3a4+q>TzE!KF(x0OF2^FFyDOU6s|nqCTu`U1gzW5mP@xhK9xt z#ebuBlU2Z?xbRCHL?Dv=MpHC+e!G5rfNNbkI|8l)+)-dQ5ei8pw1@`Tp@#GMj_VbN zszSdz--kjD(SdhJz%9vVR(N6;RBKxr-@Plcf@X*hE$VahRgynYPz5YG=fd3T@_9?2f04jcvHtGXt62HO9+_J4!+fKY^G zn?6mB)?=vnh^k6R$mA_{9$00s(#RvOq=IC+o=msJ+z>9%2HLh!OiV7J-A|i#hwJ;B z7M+Vf{(hx5`Yz(?E4sFZ%P}o z$)@C%%3cEy-R>Z{{r+p<-I{e5j)%K~?g26P+2-mrqXU|NJk+|TVV!p__)&CHb$cmB zbQ4>=+-NknxLE>f)wlv!=i zBXt8Ra~;TDD!U2(D-3-Nq)#Nb;&HNm9U_xm3|nQV`f97~f@IhlvfXpW+B*YhdE_iV z*{0`&1Xinp#Gm#AHBOAk<)YS^Qp>_9UzM7QUZ~+4M%LLz$`g)l(Wnt+;8dV0eL)c@ z!GRv-IW`cWV`zTT*#4zW-DR6Iaq$oqM zE<_8cnF8twoR44uSxB@ z5K}id24@;;oL+@q@2ssp;b(GD%N${uYftw#o+hri&kEoq1>gm5Ty=lgu`@BQ4% z=lz)p8x~}pm^c6R;hYx_DVvy9+=X%y}oUt?3gv_5O^mh zXKi2csqzEY)L^Z>L2I#(&#MMwg>AZNKGL7!^U_Tjibg54FDB|)KG_xyCPr}*cq~%;yKc`FYAc5!m+t*@#AxO7=_qCR!r3C?|+!gC%%qV z8!NeU6<>i~4iVXAY+%qIQmRr~ESdj8)!E8!H$AACgS9`$mo8*cp$Qv2w8ao2+Tvha zoPwj?3ps@jm}5E4cg@we`5Kq-#4e(cxmK^Qs;6cLo`B)52o%o_q!%@`Nv<Q3J9$AxS?{HI>oUz2utW5SGh0K*rL}15Y z-zdt7g#)ZMM|;g3csovi*d{@94IyBcOoPaEb$s4qyj$YD*-PdU3cqE}TiZ zJUgB#Z}I7_qcUgTnehtRn)wh<_xk1^>$r!tjxFDN>}Sh zM!rj&tbm&`F-|MXwidtNp(Q^wQqIitLoeq)EcbP~w~cS-P~5DQLPw}({Od;KdoO_b z`_sJVi_rnIRvNVXWp(9rkv|)e8H43Ym897* zhvrktDJ7rB2YPK!q_scp$YN!Dxu{OdP4IJ#ERZ7K)GlBm(A%j>BaK$C-(;uVcA`EK z&$7g?=%!ao1Gv^k14G>oGX?z=Z7c9QoQoK-<5LT*b*Jspdle;V!GTc?1Wrb|f78jt z*ms7Vx5r(`s_OixH|=#PQ^%3{Oug_&3R8kKLSE6HR{KBZBrG9)L~AFwjdS#r;L9jE z%acMGS*6N!i6=1}#|3GeHGYusZKG1dx9*#)B^^P~-?@M>LSM&2hOktCV~+`}kDGV{ z3~6nrpa<#8EYfruO6ICN^gwPijZiUyMpG|tTewW1aWA>O^S3UYkj;Bcc{vbPsyP(0 zjGKHA09T!R|H7FsM*8=4#_AYgtlrR0WGti!K9LoA0QY}<$qQE%(y89yMjx{DYc1>& zZTA9%+ub{xGCuCE7TPS`jCv$vQjN#;6ix_yqvoe{5Pch2mv-F`Dw3;1&*1PZ8}nyH==Ga%d@neF;L|DeuXCSa3C_A$((^s zLE;Ox=lrlarKMfFQ}vUqIi0qwmMYmvXO{3Z&k~(+1k%GRKnuRRNXhVH;2!C|g{-FI zXdkwgPcu14p*MZve@kth;9?`X^K(nYu*a7kP{m(~A84Kr?Wc1ueMmTpcY5Hc5TN5u zC_04%qasEbyO6OyuT^F{k9=-0mrq4fQV_TCzm6oxM4R^}V$A|UCmLf<&WlR5*W+q~ z^)F$_kbrpJrJEtk$CS+X z)@QDrj zy-*kw6nvruK{;MPpdmCoo*04kh;-+y1xm3^>A2BY=r8wEdo@9ANT-Y43>3+|giKVI zAq&z;P;$Im?K1jvg!;ZO?yLPltJwlOlm`-Isz+AMnEOTPTbTPBg=Sa_DXdP&IVmyL=|X}++Xl_vQ;`1IPGrJ81cYDOb5kK7ZazvH zyZK1Mx|qKsHZ!zgyRs7D`+Oh%cKI!wiAa%BYyCnx=P{vb*y5;C-OBLqc2YU*i6V%$ zc1-Ld4W=-cPrMbVYMPngd@w5o~Rn6{&paaylsz?YwgL-nX zY+eDrfKKmPyVxpVFC)BblviL*c27K&XOZHYy7PwAnr)pw0!^Zh1=*l;MRzYhiRHPq z@O~nq8y`z|Hhax2E3}Y)M-h$RY+kRS-0tL__S=}FHmDtqxV^Ks{$zzTF2Q+?@v$zs z*H(ocu(lD4h;#Q3^NBH%x)gM{Xc2nr;*6-BYot&nnbx4eki%Tmq zV#;XKv07T^k`mPdo@vt^Qc|#|J6_(+zSMrqV-P-!ey^3FM#J=s>~V5m`rW0oL*`^e zx|V{x$s<>MV}pVw`P6fp$7p`ib(I}@_8fr z!tZhcjyby@o;H7u6+b4qK*B55=3aZ_U5|+@N~z6vvI`%VVtyxoO~Dp>ErZsneiGxD z>dCZFlW-Y?6W7!V2=r(lMvCjKO~5aTR%3@PA?fnRS&m})mfkC-9kARE^)u=vbwI!0ONa#d;1mwFgW^ zhj=v0JRDg(OR4kutxYjeOcgU;+kWdgv7sZx`?^uPxG93l%HKpL)!!7NYt<$~+tSXx z;LqDei;z(f5!q|-ih1>6kgZWmk92CXHuql3v8-0@L8IUTkmv5Olj3L>Vf%ms&QGYY zl6`m>5&(d$wH&A->PvACE_c~xlP=zy`43cZy&$Y4~tSeZAt1?Tb7kRxD|A5O1`=n z`|z-VVUjZjD>;u#nVTvQoF$={_XoLmUP}T$^>SdpI9z*!KKky^bn`i(+aOzRl;Bsb zK#}2I)a5R6WySRXnKD|_qCXHn}XZ4*baaEv-Cx6f-Z^w}74l4{y_lng7m zt^ukiNouRr6A1@!fs{1a@yh*xgG%Q0G~@&|1%u%|4<-5mIFW-?>k5bT`jT-2coJge z)83;tK_A0JCn}`wh;Gn%*=*Q+1nDG>MJM*;Uk{#>(?M+yut}nlDEF4oR0M1ixz0g5 zj|G`!5yL>VBOF?}4~IQ9KxEP(}+J^BgNNy?gK#>iHKnF^pJ$?nIz;7)CGCGpWFc2y+EUr4gnpT4F- zVaH-6Pmx$=tzADBG73f`)%a7@`s|#$cU5$u^NP_Rc>#dGEP!_j&vDbU}F~O_ux$EPcAF8`mmu_SM1p&aSTkiX8cT*U^9-2>}C$$ zV<2B(uYJKXb~*2+6u^^V^glI_ZP?I3f?ZSfOv*h91q&G>dZNmbQB+0BVjrW*QElgl z_!=~ld<_7Kb?O=N**kNgE6thbe3k~&lp`g!Df@Sk2aIT7{MGzwEn zA>)+}OntFtYxhKyOiPZPDvZonvbZ`vAS8xt)JL?PM{Qxc=G1*1n&$Unmirgd3&CiBg;P^M#{cSPEeq1=fN?Ghu-G2d;c(`KU0=(}k`)H}afF zqSl@bY_k+KrNXigCd#?zk}KhX?wkkGO+3ACvXlT^SS4-iFuY>OyKV2>qb+Dv)UvMl zRFkL7^KN+~^5hI*G-FpWyT!X=pbR@HIKUZAn(kv#F=X(UHEV+X`URNaL?*%c_~S7i$4`9bf^&9O!*ml*Z8r6sIqPM9H)7*68dw z>{2#dJoL+&zJUxA$<^yZ66}W)NOla3qs>CS6cUwENJW@e_r=F=C_;q527fkKRQhrT z)j?;m78#ivlL{FBw?S1is#|2d?xp^)edkQetflhpfsmzCTZyxBv2zYGE+!*ImD53u zFV&6Q#Z0I*0tPYgMB9o19BF4Mg>*9<)dY?r6&btAgJ(yz z!r>g%cTm(4o9q7lDp-Uy+ngaW>bwi#uF>A>bg~|9F!KVlfRtw)^wu@@=ullc)uaCU z^h76^dvYr)en@*@QKaUfk`%2j%CuMW(i1{i~wDm%*(UqruWuP4Z3)+{;N>sOaxUyKH&9P-h01ebi295p7VFujLz za^v|HLVJ0aL{{v-7|OAnGIxh2=;w+Oyg;haHu=R8emu@UV?~dr8MOkfmZ5eUn ziJJfyW}BLxu^OjC+XTGMF>LCA6~?7eOJ*o)hC*k!f9%qNpG%;R1aJ2iI`z?{mLyVo ztv<``u0 zL>V8UBa&U^q>n#L&?@6DIcyc0Q<)|R?bE3uWNw@jqu2X_RL$U9!5DG5H~ zq)l8^y04bNm*GMC&Z@ILkA~Wz{p^@d?b$uz+=LYVb@(y}dY=2vU0R=jNl9gaf8Q~) zwO~o&?+>>@z*y>KUZbL-&>7D4;9wKlg3nm$HTbYNgJX4g7dXDo0R5cTZo0dAS6j`4 z`cFUgO)S|+Kn`&6&9DQu)k}Y2Pw4s4AG7sJP>|+6Y`pfycjdVFrXA6>nQP)__W>2c zE__&9_iE6zrlD^FaJs9;X@w(%Hbbs05?%#hkzfnObAwJB4`(Cx+t~JhxW>-|T4%tO zE6;prqpxlf;xY|Wv>jGwd#Sd?ZA|vcNOu?8@GGmA`P+*tdqeC9^opDeLfmIW{-K^;d*TCy5zww7nuecK>oW3?|_h=*Y3LHf0xjggFr$s)*1Lua{0?! zK2S&$kGLE9&n^5Vi&-j!@q>3tW6R5H*@h2SeryMmY(B=JN?I2{@x@1Z?|+H3Yga$^Qt*N;PAiL zs@45GFa%VPT~jBrLN^T5&+7nzfIe4Z2|U{h=acC@H|)>_Ce{Uthfm!mG(m?pfxq9h ME}zdmYjfxS079zF+yDRo literal 238360 zcmagF1ymf_);5eKXmEFT>Bc<-hsGfUhsNFANpN?!;O=h0CAhn5a0?dXPbPEk%zJ0n z_xGyqRdsyp+2^UfRKDs{=r2>c?hSYlF~2QA3fMNej<&wUHa_fM-?!WKV`InJ{$~| zhr08t`H!KmhC?etn+^{V1~pKtYiXV5Yl^}?)-YpR7Cr6-Y-I|xzC7)vr`)#NoUYui z@?GtXK$tEe>|APTA(G0g76nJa?a52aZnJsAA}s)*BuQbqq1+U-w2&T6+#I0QbM0qi zF283PzpQ;`LdSuFtm2-@Afrd!BYR|n7()As6#!9Z+^ac48?W4i0oHzn9(*o=U{VYq zDu!JRhqNrNkqSqF%=UmRs)m6Q!-k;2ym%xBV_){5ksVWFNEyLZzFW|xMI()Lc(bYK z<~EHsCyHvx_Pj@PWTQqph_&kzDh52)BBuo8R6s_8ysx^Y%M61&3W>7rtIC~ z5ZuZsahUAE<*yL@Rc>6X>Hze1s5A@(11JvzwMNYtMc@4%HykLpipaE>nc+Y}{8PRr zCel5*sNxO*XUvI|1*GAQTi?Z|cHrS0P_3Bkl zRAXI|Mv-|eBHPu$-vtws3MoQBxkxh4HJ)oryg`q!UMI$QtGEpi`4v8YpQu*|&L4^- zzR0-KaqzX7g^M){V($j`AX0>+;`c#?G>E=)n!#w((uixS@So%fNP0Lvq8qMbA5Xcx;ZPe^s>E1yn!7=pcp z8vx4@R9X|q@mTNrUfe+eX^M=X5WkSQFiOOxry2oOgf4pzqWnepVnn#^nbfewGf_B| zif#w?v$onuR4lwi)K)7uPb;*72d@$Id1&EMDR3=-Ma!GQ*M)8gy67ZcUL6vS5h9ZR z^+v~iw|blCml1D*! zlZ1$s1fcgig=zg*)!;3_6qW%eRRrGtpA=vvHYwBn!ff#>dWh;cc{lK`I|&YY0O5Ys zKxSj#6n)iX=)I3_;Ab^l>`#3L#cI5X0Yd;PPFT(G1n};HxFvyV#FIZ!mz@57HRmzs z4a8Bmjy{Pkl*wjG6VN&$s;m8+lpkNdw{SzQ9qXDzOX|v4ltJQ4wg_3a9G(eJQ^h)|I>sd=4fyhp4ROxN&iGG!M0H$_ zycrUWpFMJX*b*{jvt)I}*Jak_ZSoo?NhUq;kOCol{5A)Ijo@$t!qNL*_v`e-_hX5} zZfB$hrWU7$X*6l1rpj2wXap<~*KeDDn8_^qUTjgUS^T4Dpd@U@q_|RH{qtJ3w^X}G zdsOjg9ryu=k+L`vF!FuGYh-$)b>ss+KK>s58a@U`9zG)boCVJ8_^P8J&uboRo;hwo+))A|HuuScmdWt}wMZ#~TJw=b zU{=OThJA)J_p05T9r1qKU~|`fepq|hME=xnD#J3zic^P0`w=1+ELZh1cBAI0*6qdV zdJ|W(&*k~Gg6r|s!DZdG%8CHZHYfl@R2Y?4{77|hn6FNeJkbqt;(waL zfm@Yw75Sa*`7&x&bTe7RXHip8Q!`VGWH~jZh5o7^n|qt;n;e@0Xs2kk6!;WoF_ST+ z$PT(0k=>cKb}ODwQ-%0SB>KvAWdY ztZ%V*ci@RNMi6MNzuP--F?QjwHBN1dQ9yMp;8-f0+VwG4%Cxu{%xGBtGE*U(ZT871cUR}CLVzFFDB zx`ng}tkd{dc}J0JRM$cscP zY!?I0(!XfFXun*J?3Od|t6Hi{>C$Ps8NBZ-S>0XQ?c0q~jndU_4g6BzIC@k+J(pQ& zTe{E^*(}#w;pH{*@bD&)K-K!LNwMa_-N<(LSyntP(b~OLz3%>NVkgs&27DW)<%dm= zZ?}yNVwR=lH5DZba_c&EPZyCpL?%Ax51-u&H^*u+vYdyU>(3(BSG7*mwc7_w$3yYk z#MP(kT`t##-v&ObeHph~Z+L2ZGJ@Ukz3iOMn(}F}13c(WlU0iR@NRf;n+Bh5PHB`m zt6Irt39?5gZDICk!vi$5cQ2n(=Of3bLiq3qmRa&U^ilwrdCeS zJax`-eEF*8!o>s010McL%jSSI(r%jtnz)|CXat9DWAfn9Y&U;FNP<>?n< z>+h!J?K^J8&o#HZfeqDG1C=mZJZ>D>PDtfFev%$1mgRx*u~sS^E~@`EH?l2l&IT z4`Z5UHiJ1sqP4q~`R&u8*#O&pZO3Ws+xnN4>y?}Pai3Suvr}xvh1&sJVlTr+l}I`# z?k6`b#atD)&uML3UN)e0ytd`1wWFs+6gra3>(RTxyQb&3)xyB>$*i-R{qMKKuMqqW z8HKX3$46jXn-P68LdTGkAd}Pm_>#Ut5j4?|hrdw`eTF4&N>U^`K!R+Af{ew6=w*kp zd^d{##&%a4IO`Sa)YcAS&@WPo{&tuHg$weFq&|E2p^FC%z=sf|d-(B#^Y+$f$#3ki z5fc9j_S8L|!zt*Ic=Z*8hBf(J)aBc^b7qB4IZUcQ4;ApXlIji+5ZF||zK~K%l;;o- zkXdFbYL05MGQ0*hR?J{S8+{{YS1a3}tsx-zU3q`rS{XTlNnNchtsQt>1<3zs!Ta<6 zS1}7Y=^srTEdjXv@pO z;^N}M?83%uV{gI&LvNHW_!Q|j(?Fe>dvUZ^OJ;;B?5jAo!us5@HG_$cL z{WUIF-^R&NfSml-MF0K#UZ;_(*I(cvFee@*$fuIhhv1p+w$fA9LYsz1B(v-~3AZzTQZ*B_-n z877Fp&+=d93nExZ_Rv8<2ti1RzEg39JkEem*EUgq8E+nDOreZpx0BfC#hqH_n!@aRHNB7IjTmG{fuW{$~b+z*^#lO-83uOSN*IVqGwk`Up5L*r#hGhDFm|6YVrQ1x$fEDSV?uS={2J%4s}t^kPDA43|91a5AUn-QXwpmYSnr+jm8YkT7jVRZi*LCs55%L* z-~Ma^Z!Zo?5%FD}tRsi(T$Mbdvdo+-g^m`1w%ERgP3yj!d*`#{t39uTQaacW)=~980f!N}P zy1FF9W-0AUn2>0w|G$@?8Kv)*(KO{&E(a^?M}>ewRq=dtZIoe-l9XcoLF?Sy+y#H> zWHW1xus_%%gX1Tjq~G`Z!I8x8TZ;=V$-hiD^S(sK<{UOTE8F)wA`%L%GU*G*9?2RJS35@TGT@}LrvVEo+=-H7_K<>=k=5$!(Il)KO_#&hM{iP8U8JjYzsJ< z`F4@}e=1+F&^!j<8)79>;HwcfMnaORy&#}d&p3LIFe1q2J{4EoP1Ky}_kbrb-NTjL zzqM&qR!Dy4_wE1uo}h1h0wduh!{V4N6Y&;N1O_y#C4<+^Nbao~Giq>t%EgbFdg8QR z^X7L)ha}(3(tlF=QyX|mI~5>AdId>Q0|janvwx~iL8cN|!qQ&W`>?7x? zP_i$Rzl6BB8VV9pa>Sf?HojQC{jOedU0tJu%>UaR{jN2R@0^%Gy}$kby*iQxn<1ur zPi&a2j3a{3MAkkP-4k4!tJ$V;j|1VViA8qx8s}sov@%L!>AmYx4_>cSJ zyn^>K3({O}DDbx;!Ta&3Q1_)$Ej-R#X3KrvnN0fU`~Z1)TcX_EKb_qOEY#JtYN7GJ zS%M)vhV zA)<;GHd!UfUtCzC)cYV@{@eccp#3;)&{%)DgnwMg^nST4fhViA9#4%n463=pg!^0O zf}xkea)0>8 z_!oV%?S&`w0$SP*O^JO1&G~E}A?6g(L_Ta4;ZlT?g;1?;byp8}42pI715l}yLlz(X zXHDhlmUJT&#^`5@!mX;S9Cu6N#<>mdrAm{CU9`w_lOKXV-TOzxf7RtxSX-LUVxU4Z z?)hb7b@pnP->4|n&}1sOY> zXVif#NBgs_u2oIW@2d0y_geIUeSO^bJ=$f_pn2?@LC0De9F)D3Du5lCh$MsftZ*}b zgUgvR0heXdZDwJE)e(Ujz;%Sx+_Z;*NQFajW6Ly8N-&Eiz)Fv#@w;E$hGb7a3kUpTy}YlN(5{ z%6UmP$XYv>tX84V@g}p_B*D(t(XWdTV)fGB0vgL1z!_U!2GK0$aP!9V22s&#FsE2o zClBw^?{2PoGzivsUu@*`4P*i6f;HH&p}*MYU#|ZcIAyxoKNmE(>^BTa?zb~CdPE8V zM|A!Qu(D;LoKvPW{_x#;f~clf*klg>%Htw}UI2gN>nMSWv%6i72J4}yNbJjI!m$s< z5dG{SA*TsrNx}5};lbE9lGjz<*HzN!HmP{)Yf_Ix_RUkSZnp(N9=VeN402<;mnRQF zo8R9Tt8}#Z97lFTUGAuXo*Su1DjAa~17$ZRzWkk&OlW}mN-AZ%e+klj;bc$`UIWA5 zLB^Me4G=E7beCvra_)+Y+uMaB8BgJ~;A|xq>r3tUOZMxe*QwoNV2S0<_IMkF7$-`9 zxYy$E7LDR-Pz`jxt<=(Hyis9Dz=kCqglMeNyp39mJ<4U>$&!gmey*0{M(cUtklpG9 z7y0~sgVi9N*s3)}NYP|@Ka~y-5bIW`4BJ%@ z_8G0zu8D9BZz}cdB;-OaV!z@4173y81TEDetPeN^W*@ybPMFB~I5bNwQWN zw?nw?c&?7|8y>6B8*Fw8=t(F*{wAoTuG2oG&NVTR8pz&Qi~%WZ0wC3reVb;Pw}xSqlxJ) z*|j%v<)5#;8nWW@Hs3FR#jvR<06myh9h`om1-C0^4&jty2!ew!4!zb29_y6w)xL@O(4HOXakl&te{ zUuNa&^Egp}oNJ@M`Ew;xkuk@q0)J`R1P~J49PJMeoJ<3Ajr6aZ^#rqDF%o|7(Oh}$ zwJJRrssWW44-YH=0O)Z_SjxUS@4y#yJcwcOsc3OCuo3(aUeafd<%L+ul7@q3_En%w zrjneP%F==kU;Zs;U|GOI-?}RY{rj9*Xhje^bz4u(Fh4hEt$ zL8K7@N`$v@9YBMYa+;!sVY2h?EfeJ`^*bsOZFbJbP%5tj^Bb}15dQ1RUy`qr?PTV( zqEUwW9Rl!33OOK+bjc|v{9S2(^FC70@AGFe`pNnEfmD+D2DN%PuF7>bUIh$3f_#Bk z3wjYducs#yt(U{C#c@2TeGvN#V28Pb+?WKNLMxPfPf>=vG~__cBWXwJtXicd$L;X~ z3)mnIQk1Vxy$gE3eevULN|!=TePcJ86_Ga`dTgM-%7vwL?z`+UNh+LjR$zln{sGqgv%X7 z^lz)Y9p0y({W`b-q-j5Y|7lpB@-vU?&*0!GDQ4w4GqWi@K7ueYUf)Pv$1*2aQIf2~ z)$(rl5(xxh_ES}o=^ix4K!n9mk_l? z%0vQ#LLlh;xK9KxDl5d2=Uo-*`U)@xB2%O2HhsNu8$`#KkA>QU*R!K7M#D}6COA3v|QF~=Nd6=DYH~qdf8t4 zrIm^VVejS^N{75olqpJ<7KRgf+)a^;k#){Cq0&VR!x5c)Uhe2{xE;$_f3c^mP%*5f z48p3MVK071?n|Yyu<3&~gw9${2Uhbw^H^VQw8^@iGfL79yqobq*&9u3v|2X4`JiR@ ziX^}y`!^KMt?ahDNW5|NjR?!b^jSQnQ{`%G_9@_LFZTLv>fJL zbUe3fHpYW??J<)O1nxnVQAw4|5!Kq|H_X9_S@D{7?%z7cOkdW>1W^KgjoeYkz;<0uuT`Ca$A zJFo?M&fz_-CncsCkt=;|kE;hbKR23;6Jv~5@qEjR3+>ZCS}B@MWYVlc0a+5?C;>$L zdw9ZyY;4YkXtHFor35r96Mn%c_-Lly?qgZ)=P7R^-h+M?>@oR?-$MNfTl@;XI#;MV zb2IuQ|J;Q13g9BaqZ-^m}d$B4+84xqEts>u}bYhk&71#{Yu4J zZP!Nk%9+ael>j^a%K@c|enjkDIo?tQJc7lGyLWq%GS?NNL)~JQX21)i+EK#{daYyf zirBCu{`R9gE0i-q>)rEMs*rGF#5gLsiMo5VkP7l*s|Dw+X+Akkw-s>h(DS=g;T42( zWFlVwv6;Le=d-mF{e=*kOj9Dly$Q;d6x_Xh{)=7=>cHJAH^(OG1WP5@lV7Iow+C~V zGQT3^@lRL3hXDB_WZxo!dNd(YS>RW_9x_^TDDK=8d&o+=@pu~DAV0$xFHTauEJ2pO zPH#2(`g3;zgWYCMW=Pq4dYsF40i1SAyZB{?@e_ky>e(pNYtIENHwLu6DXVT;-2ii^ zHZGkSi}{3?dzSV4j-!S$0WSJ99t#IV5Z49L_Lm^yFyfU3pS=?7AqA5iDg?uhciqmG z3B$XES>EpVm%9-QfkC9wYGFp|)?f|7ADJ(Y`!sOWn&;4bNDNh!f#nU-HZ$aX)87cV ztgj77I|MiCg$~7BLS6h=()xO=Rb4T{P zFID~QH5MnBlS060S+tA>E_B8Hk-+dl?jhp~-BhjM?nglYz?bzT-$flldU`4+gU`jD z-vMHNPn;6uM|Y`}p?g@`;cq+W+r?L^fCQSKpL!TbJZy*Q_6zRw z0oAp2eOm0S*aCG_OD(k;2j9*>#!sr(Ar+`8T$AG`f*bU&5{^hv|N2_ds;%^&VyhNLW#Dy+==a^TYT`e;vR zUX34iwm>$mtnh#|lHrQTGTz+O^q_V9iFw*7FawlqfDS8~C_(JDGr~45if$1v!qrDX zF1Lh+r$8IYC^|+6y8u>{|^TzI^MppE3UqIU?3|Ed6pn%g397 zt&b0qW)D~oId;J|#{Frr(86VO3AeSz>^H#o2Mpm5<13)=q(`ykT;-5!CiJvQLDIxmdBNs-aQ4a{MN$X98x=Ly{lv` z>lSk09SHI*fH5QgalH6&bvt;RZ+2xQuvci%0+5lS1gcnc$8Z>oI<69=HA7V5pT~(f zo^T2)doM~=PBoiM7kOA*9}`)2ClQrC09qGw{D=~J?*=YquV}-?H061AFE~s@w`swN zZXEYxYW3(=Ys6NoL=oX1JUntPEBGLcg}-M~tMV-l)jC8> z!)Tzw)LVHi&ACkiBrV~)-0cQ0*p7p(Umc7hGBvULdugj8J%F4tH!*@elk(t^#!?+!SFXf)9JK9!K)g3? zOAiJQ>M_Cw<_tKC^99Aqc9gtlet(ZI3;dE@Naa4LEeHrg689c4y*=+iSyqPfn6J>* zbeReK>=qxA=VbTz5vQ$_BaiQ_ZCaG|(rwHkvj;$8BjSO!KmrLwdLc_3`GhQ13s4QX zC0Ng{k=4Aa;l#L5zjGV+&^h_w39;27>-l_`?l1-)hhePnBW?Y$Kun{D?X`n7#w_Y0 zOBtuZ18uJ7&&*01;x-ii^G>QHE%YTd*B>`0bKB8|6Is>XbKD{`@5#?v&jiot3U+Ez zcm(8EERthm2A8c{M=wQ-#b^&7B&Ia%h9!zVAM)mZQp#q6@*d=no*m3q^t2MRmKvBf z?>BAHk~(eLMowdnQZ^n<<Li**3b@~L)s*+x%?0J!=+I52#!~e0reg^=5$q2AR(HK-pZTTL=w0%mRPiavQ!Y@o877<5 z$lTvgIjPVxYdbvq4K!j-bWh=?-5;n+t8vipZ*NID4XYOlxJ}V{iqq+P=Xnyoi4c!E z3&~tb8|eua!9{G1h1w1f=o5PyS7MF5OATltbx~(`ahWE=ZTbF=S4YF7&U&%Nh(n6P zMFyPo7B8C2JT3&h^9j8lLcwY&CF~u`{Y~Q+B01p%I>bV%860|S1dTy{=D?GcW;Tkz zNm_!Mx-#5x>4)ew1Z>vp^6>QS3kL=vf+JO20i;1rMDmI3u1R!_Slgc`e(at`Tiko2 z>cQG^LB2Xl&y7w<8u_t^iO)~z9;eOgKh|y~KUVa&-r#Y-0jE^B9s;0Is6w^@%Bu60 zsk94iJP`Yg5QzOvE)bL=jB#yP9+v!AV1*y45ZXh#ABUlK=!SQ)CX8?V%>w56l^JY} z^s6il*J;=6ZB=)qs2Nkzfg4ZI{?1TZNoA(BkQ9}qRcgW&Z>xR=K>j8Bp9I(z(dCg^ zIJ8G%&O8K2Sh-#@s~m?DwK389F$zpIr=v0n+&ObHlKM(f9IMYpcZAwxt3muJ=X8AZ z3Knc{4Q$a__cb06{a{`+Y$w00uz%|im#Nx(?0yYBA{2Vi4Jm|DQ#FrF90s;hc2q*` zi}BYf;6^4MUtg1}s!ed^jhFGPG#cdPw`X*)RHKb&E)ZO>Jtlkcmq2`-veF{hzv25T z&;MlPyDEk}ZWOTB#)EtFwfmG5&7`cgLGJ=+e&L&Z;ldFGURrcH5!PIkY()#@x8nzy z(fYq&(@rg#&gpnAzkPC)`3Mw>)mUGFPXR!LKcV?wcmRPuZlkhwCCXcYM-{j1=mYH# zO`NR{tg_@F-uSN%=i-<0tW^4}g7|D7^y^<1-Ye?L3*v)fsazJ>y;7MM4PB{l_P@@0 zfs)p~KjM5qK)U#l8>4hR(`7nbaiB+c_%q8bq~BQ$p2DfV#S(0jHDn;$8gPLot~27# zHH~*KO^-7$V!D5clFRATSJi?S6orhG`1zyZi08=QHXi5YIVLpf?zA%JRkBGR9BZyq z9ea48{>)C>)fihXh8(vaSN%yBoO;-YhMqtG%*YlVE0h{TKVZ-FHKip{(mqckRFSHT z_86mA#Wtm6MgHE$`K#gi|(HEoKQ6!8<+LPE9JF)iK^9#yGMeY=SG3e4OOM$s=EJ`=j#4rTk?6 zQL=>_d%K=jaR834^&|!w;Ldg{gXt4^1w!v}!{Ew> zdx{608IF~v6LYC|j*aboTN052wsp3j5LexpFc^8(nF!~6IsypLs$5aMZd=!>Mk2yZ zUPnUif3l%%X@bu;9!Widj-0&a*w58Omrsr?@L^)D+kX{*9$wHoG`H?%uAefrE^X)W zw`2f4X7nl&yA%oPusj~(GaxZuziZEE$0>88;(y@b0!cmOU~sU4sgC?H$twGb=7?Cc z%C+$ESE`RV-MGlQ#WxADl1|K=al2JY{ajxjaBi4!86!{Gqky%Xc;O=;NwHn?Vzs7S zSl#NgVt2eCT~7Xc2)Pjwakw4a81>1=`Q!{ic)eL1wvaoG+KL?I-?MWbGEWg$pv&td1&9o=nHf6*poe zh~cqVqz#)AHz{iiF<(QjnuS->^oO6r8?cCssEoj1IrzdYhaD*`Fn9L?u#ac6;U!rc zLO$S~4gRR8K(e)*_{=$`Q!3ot@ydr0EnlGzcYE7*soEeYOfA}vpj|IGnZNeFT3Yod z(h)!uCzpI78vm&!n#Ve&>GCT>B_&wHyQ@@ozmiC$6w-$r^TF|4X zm~|N-THI#h%XgwsgF{--?NMzKzQ-N4WImV!wsC^2nb>YGPq#rHOe6KFAab5s!<#9K zYa$|1{(IM`)}EXJ($X1vhP<3-LyOcS{$urhXqfK$CsUc@3y2cb?Vawat;y0yB~<4( z^YSxBg3VuP_5i>TOm#0@nM0kCO`^?3Ia-L{Au%f>%ceA=P}JdE zSBk=e&dT;MjJM{%c#J?v@mzH1-rCHLj^~cGiy};X;4b(OBa!93#E%~x_fPYJZX>Ct z1aaiV!Q%8ogdqa7%#`vH^(9y<-MC%)+Du7FJHv%p3MTqJG6MkQkKEvxL9AeG>rQHS zT#VP~J)@c6R1S-|yESURrGZ!CZ|}R~c-UuT2M1YBwVYhsD7U{Mo!%L*LX5(%Di_&u zAIPJxb?uja4`2PUK%N^d_&AWA_l9Lk2dVr1VFdb9L3J1;-RmmSPgC2odXzEmXqS=t zh`A0rAUtJ|MD&qJVEwiq65F$oxssEG`+0Xngt0gDFfJ|qhdBUjq)hf<0X!oIc3M>2 z>D)*P-NB79y(8cW+N;xG6fZ&U4LiDx8H0rx0noXxVK@6(K*MoQ?Q(ZIKC?Gz=ft)a zFK%c)W+Y6sedy=<2|HsS%y~j4ClJc^iD}-Fn#RJ7d$M*t*uu<;ABa7aKLoXjZ*8iL zvSVB0kOiPP`*n=w@MD3CO$%ri z)RWin$FQKl(u2v0g3{L$8|mS2QR35RQu@px3^8_hiwaM2%7-V<*D|hKn|<5dI~hWV zwNfIZ09=xriqZ^|a0eEWKTNjtd&Dt7AnxYWFVKx7q}LQdG`k5#*zfu71m|O^5ReVv zi|Hzh$T`b%_&h9y7`ao5_{wIM$=tFYR|%gxdrfvt>8=bOYhyeF@MXxdh6}WtjT`Uu zD|Stz)o`s!d{|O^XC2>-u9MZQYAe0;Jq5jdnER&bYAK>#u~I#(?yp1-KOPC+0RrNw zR24o?e>RW@`K3yPy(#8;9^)T)*)PZp8-Bzb7!DLUE$=bN|K7Uo`MJ!-5NVN#~vnZGamcA_AD}#^yQoe)CU>vCeW(E_&!Ug z<@m;-!n4i9>U!hG&{_17krjs`B}S2Ih+(LuyVFO7Zot4GVd@TRzg(m3{JhX8)mOZ( z3KGq7B!q^&UvUIhYXV>?!^B=hAg6)J$S)3<{!q@PrZ=TDp+%LN#%jMa3f^d%od4$5 z^N^@%4FrkjTHZyalWm+Wv6UU`<1*E3N)$2AEaq$MRmJ_s2*_r2zWTNa(b>VM3qT`@PMXWge9cc|k zXuQf30D(D0&B;jkBD77p8{3B`fkXqfy$4g(_{o0@&f)dsyY%uo@nx}OxjfQH)Zwh7 zw_!votIsySIXogDEG6!z;I~hba73JP_+77gp`D1-x;gz%7-Q|P>yv~=#vqJ z2K^DONc%@QhE`tAFlr-QtzqZXC%EOCyRM-=_(8hxc8@T7Z6kZoTYBzy3SUpq0w{vK zf$b6L&27oZvdIP^+d)yYF<7(Pn8;DQ_7;uv$>toz?bv@E(YZgu9HW^|L7jlhDpCY$ zmY7jA8^HA43p3a_OEaUoF@K^p6L@~=Oo3#eh@`J5XBDD-9k23DEgj>l*FVQpi}Y;s z^f%p^d*!%N>7taXP~??J=jgqEjeULd{d>0FcdW~HuRg8(hlK}^+p{$RT#1p^Bm`r{ zNXFq=0Yn@D4%aQBLM@*3vk;e4dF$?AmYqIQ>c*X&iu6OU)ah=KR%oCEJ>ris&F1Tm z^$b@3Z=Mn`rr`tnmv@JSxESC)un$5k#mVb7P7C=%% z1`F2nDB}sf^rvbw!f>k$`YqKAIVHOg4f?sx{WO6t$t(5cEtqhTCb^>g#0GKty?9D2 z&H_tyjT`hHtFd+Z4I`pVAz=?uS@gi0wrf2$Lwn{pCZ*DZ?(d7tvO7{p+JOcnU%6lj zf}{%?tiI#FuOUZn)Oz{b84`_bVh7$IHnR#r%~-#?-Twr#Sg|vlW|T6n@XvH*(QE^# zS`DOa;#MsnMz+ z7(991Q-$G>xIX#?oi(Gi+WRgZd-;6VpQooHKjC%!Le;%QM& za_&GF$LxLz9MF->^4wh=3a z9@i1P4<+{=e&Ib)+a6&!LCUVLk`Yz`tQ9p^swJ?jnbZE!)=Z3tg5)tpP0}>cluXrR z$>?MHeCRez#K7eyd1v)I`*o1-5&bBN0V^1Zb{OI7ajHm~9uSTl_7Xf=J|Fy!OuW8E zyKesYTT^&Do)G#|H~^9lm%Eh--YdRqw{?)oVJA8yUl>@Q8!sDnkuDGj9km_Tcc^E< zIhv_-Tn2wT7WDk)-j~b4KZw1JBAlr|Ffwjzy#)Tm4XO5d>v{)&%xdhIfC$TZmdZg+ ziY1Kl<-6{Sm_&5w4~lUQMy*o1K^h1+M5-12N~6ep8S1}ffBWH)PwdzYgGLq>3fe~o zrtSCgt!|bo*$|VfpDt^Vda4Fc0U#EY|KNVf2(8(x$aL*+|y^%5TF`(k3mJdm0B0iNAs((4K{^EmHs#cnCL zW*_Qt`0e{ZxdI{=2d0M+@5PKmZce3)>rv1Wckny)hjihG%TmOVD+#E3nBT>a@6j>b`FSw!cEIm;R=yp)whtR zB%bEqFbXgRHybV#WhVNAN#zM56u6as4}#i7E<6%l%-!=L=U?n+*Qk`GBn_8CWf}YV?tRtxlQL9TfE37P>fL8OgVq)&D_k=oHQa%+Wf{y@k($nS z6FI6M+nGGh6hLAjo>ie_9|-VF;`|f|2*iGT4%-}uV<}#E-)dbEBpZgV+Nec;c7bhd z1*NpgfsfKu_H1p#v(WSw9EIX=Zh@GDOz^p%QH%R$RvBt(us_*A!zhgz9y+|hoQ~%F zIPSN=ssTPtsK+9aE`6NErG)4(G7U_uZ9+r6RU@P^G-=9MKl{>#vzjFMLH|$wl29t2 zreX|b5;j%xtxXMX2x51<-wo1&DUonmwNSU6s2z|$*xSnQPGkbUi%Qe@8YP=s2W)P; z%6hVv1b*DpENyNUn(g);g>hPKn3nbSU=_?PTy^VF(Sb?o1!Mxk5==S4;eA2C>!!t^ zM+*nIU;6C?g{?<~4%f4WW^6Y0(RPnskt<;xn}=BP=tF|IM8Mm}=(P4HJ$jBbt;dsx zs5e93aG2qoclGIg{6A$L|K`aPJBdjCj=4e&-wiOxYoWs#Qd))!c||@8Md%x;_=y_FO0}bg^Up z(AyrGf)=-I>0Y^OJ^Ur)?T7kwnBjoU-6#^Rr-i%jPm1GeD&kRK{J!dgM=?b=Cnr}i z$04KLt`1Xm!uQx>+q z>4Y?Z;b*fl#pXleP-DGO`LgBe#&jYgvy)mPQa6_$d&&HAj_%=#d`!qxI)=LZVm z3wN4gihRU(N@^snt0We&LZ;x?EMe|R8=0uzzZ3Vfb8k{9tD`4e(1Qed6OX^{<_HVY z3yK8t-_b9E#0TbXzEf>tU(;m0DK2N;9Q7i6mAX%I`h9b+LHgTBvb`I^McrtYURlk7xuiu{@4ZadsK1CGj-&bXwM_pQY zcoz^EtoG3Y5hbH7M=p8C^9#ae2@it=Q7X^@(>}MVq;+v6Vvwx;)>Q$g_S;FPMc+c#Ost z2cKa-%+%P4Vc;34fa4p8t$$VWIn~y8u}n6_zX9Hazo%Q<(8$`;@r^yQoFpTzLgO0( zPBkxD!~i4#3Ymp+urhY}P`zAwZ)>V)_EJ3!tb|1J>D8yKvI9Qsw8_YiciQiw45F;v z`|(WFgonv$k)L`BhH1jLjz>o0#rT1v6pNtIVotvgZChBC9;-)Xmw*0i;jei2=f6Q9 zAp2d$Y2m z&@C=5PU7__)Di7T$=FZBm=xt8x{Z-&P}HPEe^p(%C1j*0X4iEo7SPu`Ab)q$^-L4% zhK46C-XaVEsw)#^2AL0=bOs`e=5L0i&L5~J8$JJjY`q0klwJ4#4Ty9}cStEYbcb|z z_W;r*-Q67mLyOXlbi>e%G}0j<9n$d*dV4?5`+NRtxm?3ynCo0;&faIAy}$c&0eY($ zlEDM}BkSvUI8o9hG+!+qnJW|?PcdtW;VKEh+azht$U)n=w==snOrx^=FkT)KU7`a> zpEH*1i|xMU;^Nn%nJjhO6AUaxGr+gUzk3y|ja4tjpNW4_$j$j`u+!pnxh?dtFNUZ* z#&rur0y`j7Je^nt8M>{99p9BAG)<#x+DUMDL*H%reL|gOfHx%bYJLgxwC&t<+Q=UJ$UyHovQXYmO^RBG2J(E2=D>t1U> zq=rLO?lj5tKTbD3{|vOG+_S6_trWbV(2URXRw zH$~vYVbbc<=jVt6^nRSscpq9D>SiPx{~xhA z8>piCmct$kg$_GSebylsGM*gzfP@K+W$;vFk-# z3gGr?%FToHZ6)Kofz&ACC^=Q)CYy7~vQavZYWO^kl;~|OdYMIxBPJlRSNc5?wc4Bb zJ$ixdZkMq5a^_+ywY;V{>=_U#2|1KGCD^LqSE-Hx6>7^vsMpGNkJakK(c)=&d#=EO+p7mbA5q~jk_6?pV!8Y*ygb{`v zBU!9ylBq_e!Kmhv4U(8l)5GQBV2UNugiU7@>B7J`It3PEj9@;D+tLU4FlxlXoTY(G z_s9ZPf;P4?#ng}wv0@)N^vcpcV%8f41K$K}a+>>0rwGJbe^4e)1e{`r#Si|1^@#rR zl47A(R>sm%YI>W;i3NzoF!CK2DGNd4l?LK@R#jT*|EoC~m~Fh?8R# zLRNK8RokpR?>C8gyuP61G4n?>_h}g6*+PyyTCUn@zd;LFiQNhH}9c-!(Ue-pg)4R72|qoV+Q)Znm+mLbxO~aKISods9Q>#B}Nw z+b&IpL5eoUekmko`c!-`8afj$py7_8M3Fp|5t(W}3%wD^7jwy6u1{fUf<2^WMxxhg zAq{>dJy+up6&gjUAPm&kp{AW|Yb**@VMQ!b?~`{hX8sR*g*lH0^)$Mis-KaCE&z^! zd=x+^F16QswhWALHn&tUxIB_`XkoYK3sfo1sP7y$N{s{|UjY>%YW!>IS$X3GGb2X^ ztXJ_vf`<}ZT@!_sVumac)FY9#<~cr>x4-t=5_>+meYQ&{a7=$|1uTI`A!mPP<-SV| z85er;=I^u_lKI#R5#3Qu3ExS4lB~BrMr8EA4M8!>61P7qu8)QeL(0mC$lQ}x7_17H z$mm%NJQb@#Vh0niKoo#hpa}54g06pGEKB+~SzSX9>`&k0U*H82*sg`;a-;yDIT?)- zM!8zz5_PD4$7~L!&)-rs=a&Rx?%?#q73^1Fy8Mkmk*BXKdWO|1z>Jpndcf@-M{gse zbFK00qClI_QqZPMeyMY^oV(N~wb-+={?E2<60!<1=I_LoW!wakBDYdPGZUY*Du_u0 zYUHV-)pQ%|ZI+iU^SaHtJ?(Y_swa&b_A5Z75rG-SlEQ1tIs|b^jk&;^_e|B;CZg zNAiYjHh9gP=*|uXWGzRxrzMr@eTT^k@29Sk4jEVeb4}Cy2;C#DjVIUi4g-} zxk;SB_HFe-JloJ4>d#@?F9~OE26}W!yh$`7svs0CCEd9)Z6{N!e3k!oY_Uwx1jpm^ ze^~$pYE5@XEjKLq?Qo-tI;A)rxQO%2)`n!uNTtrd!!<|~?1aMzmkq#;rh{VC+1=E} z+q6tCCK89L!lLzQKBO%Oxsg64B5G5OTDC+fQt}GjMvAxzt%btFXR5{WtBeq03Z%tJ ztxqnJZ%(p>OT0fjCEQ#tOiE|21E6R;$#MJzu;Q~jarNthifc>EM;5L{m!i{)se;|e zzJ{beF$LPWZ*?>u#S5F`BcgrcONN|U3F$T)ibP@CWtvOcC*y7v>!~+1a@K>(iSnV_ z2@?0J20egU{nB7%zmGK%KSXG-yOoStQwUhfToTi~TeoYdNE0?oN-JP==%eNc$kHbu zZ65peXwspI(>jk@m;f)BgXZ1OQQRXKdV8$Av9|zK=+lCBBacDqRCk?WvKOzFjX|mj z`rBNgG&_;X=eHxdZ5Cb(v);2YR$~=OQ!hiWFDbZ0;4CKsOegCt;s|*3e16RsFtIRP zz;qBR#l3mC1MuaazjLOE(jiOxp{%XCDAzzew|urI`QEb9g#f8^W{4SVqty)I@qxhw%&TfJFyOlrT(5345Z zU)Xmk%s;go{c~qc#W--9l&TYtJ=GoW@N5VVS4q`z0o?XZL^MC=)_dX1X?v<1sfVcR z;s%_i{3N1$`G4x;st+qCS0>~?0xgWG&+!n~R8 zCpFS+Qgobe1&BSparrWWW7OIKGIl%|-D`ZtaL{M8lN;7^*(t!=?``tN{)d+32%MuU z8w3M=)FA|mR&5~w;RUd6ujEewTdmhRcyWS#k?uVp2WDV_it~&6U0a&IX!+$#^Ntu6 zoA)I3)V1iF*4;4-0@)^rr^&r0{6CY&plutmm>L% zHbFM&%m^lp6J>3hqhcem%kYA3Q+Mh<@Q?q?q*)bd4n7Y-7huse6we5IhUm$&55)Frk zq*Tz`;_PIUHHhbBs9u7w-LB^LqQkuRNC~U(|6F$9yn>sq@mU6#Y^gi3h12lk>mb7Q+V{ zD57H_;PG{J<2xb#3P$?x^VOf%kk$ZSOq~~p`c|`VfTR*Xz5>iReW{mAgFf&V7S=u+ z`NT48fn&d%IZP%)5T%nqKJJC;+f}TbM&6%$-%NnKi2vqMIDnEx5TXnC$B0fTeSXH^MYYOLx5X#)P2GLZc z4?Slc54{BJ_G{?aOP=QWAF)GTokSwh;yV5u9S#hAwtr>3|Gft6U@m3jIYv!K?Ol=2 z&jI%jV+K(y86ztbDt}S{0UokObZ#S%hE7UR&b0FQ7NcDJA6^I!LR&NuMIWUMBO;S` zRc!KeP^0K)^wQMp{MH|j4O{O5@2{n>WBQpQ(r_ZAO>-V!vjFeA`xu?czn@ip+xCkK z&6e>QSCO{a*Zk6ku6Bc_Nl}7L*KpAK!gsUh-&(vE)%|NnhAGB6dv-*`2_{0A)Z5>R>Zry0$NWQRDAw*+{IK<;Dk zUR_gvhDv}e#F)8eoufK!MgLB62%SudMslQA{ZSehN$OFNzgtB;`kT6Ql^6wm|0|Ul zjv(61mBkL=R<)yGng)wUs;`+qLW07bvsDU@SW6cl38t(x3+IBQX0%5sc&`5%$*uD@`R)`yy^-ZP3N{%(j_ z4)mU-H`R@sySoSD3?eN@zS69HJIyRcm(nv%eu}?Y11t#E|7v=Xxp_WL;5}A)Yfx)i zTc|=1ku-=oT#QT*@wuolo;#HqRqDwBji^}S%ZHw@K2N2#_@SdX&Ig~`hhgR1L<_S$ z!#X8u47?35w55k+$qrUt&Yy40WBs229iVa8$^B_!-F(d-d1HgWJirm0D0dZdTg29d zCG>xjf4>_vep}UB=({>8tEYtldp~e}y*ql%o)=@;UoHeUGzbrK4ZYVwoSCc5>Q`0d(j4x_L*Y{+y{>Bdhc&z; zrr4A(`gdqvp+KvjI)rJ!e`@moWCQ}tv96~J1;xdpvLQKh z0lvEWf5N&o0vNUJ`ELYU?5}U=9gbyE z)aGO^s^*xyW;`|vHR!(#()PrU4Q+l4%r2^=rOQ|kp5@Sq={pOXVRjUUW1~hP7wn@J$zUmfpz14i~*xu#{_%8cd+EDn^Kdc+%Auk{Ee<>9K z9-^JTh@li>8zqQMJq?fq4 zuDbofiVuxK!})3rQ=lwNNg-TJ`BmUqQ2e?l_0cSf@jzzDr$Z$^|l zI#dPs2x?X>J@HAx@!nEoO3iXQY$SJk0qb|@CVYkiPn};b&e*8e0I40mWVaj8Kkz+Rv=9D5a+I^ZQdPsn`1ISOjJ1 z+`-j6r2YLGk_N4GIup94HG{@X#-(^!7tH~wWdsHq`b&7s2H2@e4Kx>k$r|+v39lh# z4!j>QXvONrmd5zdt_6u&kdZoim3mEdllpjya!Z>SSNYzsg?C74jnm+5Z{P(lBj zgZpcG9%7#^H7QtTeUY!X`2#-}{J{Vbe}D-!sVn{Um3stK$f{U%z|zokhs+Q72vk)0Gw@WkTkN z4qh)JzRv$o&*0xp3*-blX&AO(80bQ|qc^1n{Bw@u5i`P#09M$4@Y-Vjg&3s_c37tC zZ3_4~FRQ7Z!X)Jlz8=twLe^2QR&uvI=3zE)uzZTd`oKP{?i8_#?+KDNHm$3~M9sfR zdy#&K)U81=TTGQ=nlK#G_r-AOBEKc4QSuSUcp38Cx|NPq6*ah|qZ`#ab`PSkv4$wD z{3}Pl^1%PzHUAFt&lK+p!rTK;@a)MGf^D-RF695}q>X;UnkC?;EZx$iYzg8^=36Cq zoW7Y;CH32jocw_76>hJMAba-p2J=-l6Mgo*=x`;0R#j~vc_L}MT8(M;gVv-@`BzU@awX4`WW(`l`{`Z``hok& z-nXm>n!Zef1LBy^0?M^-%EhirBGreDZETIR{1_Q3d`*Vs;U&=@*A|$^cFq{NG((>+ zd4dw4q?j%WR0WHa51W+X)K+nBr-cX~4p3y^({dn|dFnb-9qdb6M;m>{Z0s7z;c=bS#ZV(a)!l}QNTXf&R? z>=kx(NVX`%D#)rqNHm^0^s+!K54ZT_{ZG8TVq&0fA*&W9gL)z_txNe4(aPQY*k~b> zP`>yP>vK315byth4;rOaRy?f`A7(6ssBMRi_3iYZKucE7!Wo8h~HU94?+>a}d7 zV5ggFMiZm%muevm}wTq%1w#U=f80z0b>aB=s^{FQ*SP+ z#;V>V|L?gqlV;5d7dnM!8CU%eD$(c&6XddrBFkTa0UnwehL>d0=gE=*xnO#N80*tk zf~Rm^Qhqp1?WQh&G_-r2?WdYSyididk94D*$1t``sltH=*?Rnu5p8qmt@jHg>QMPu zY7sa8mL&emxH#A zZZpF4BWFVANJD8t!p83|^lil84%lrAkX0B5ml^41K$>|mCd9$~pMPd}(09`Kvu~pj zyK`gy4Keh8JCHFBP%Y#Pn2VRCW<=O+`LReQKaplHy&_2Bne#US3$6O2#gas2k`JB& z^`BJAe?Hx41vCDL!@kAf_eLA=Mb>C`^q|3*yu%8L!X{mvH>R6(c~kD0g(oM|>5J~X zkiH$vn2sHY@}H0RpTUY5eMcPOWhpBJjIx3#hIh`V#uNztEbCe+;Y6*dPo^~gcdyi! zfPDJo8+2#w7c<2Fxbt^L5+4hBzB#I^{GJXYLPV*fbP3Rq%>SL8ZFJ~|%uFGl@|5SR zbShJkvmCquYkqq+#fduDi$23S*K)%Dx#qvl!zo5s)>!L0F!Z09g+xCU%YxsFD%Y(j z1LXb}CeR$^{{Sdcqhq(G@l~Z){4?AeVI#V$KKJEjmy{K$1=NTt+t}!6jH%UCd(OW& zHCX7Dtue8&spP`B37B>oBV-{M&E(OitoI~nZ1meBs5RDdXKXjs@_Q)Ok`ub#u#O{((+ zFA2+uJVO^o`x05&rG@`UK?2e!Cw5uO|1Q}74hum0ux3Ydwf_JEK|-+icKAqIqzn8L zN+`%l3C!$ue-!2rEy+5yCh&hYkl*z^Le0NLRZ9v$a)ybl@qF#HW2i*DI@j|b?83mP zHpSBaNs|1xM}e$ZGZ;dg|KA_$=O*_|sSGpWm*f71wt#qG%JpL2PX$A{rP(glSKAp+ zdE=?hlO@}mnpr_m230l-U$@n5KiJcl$F4v)2dwY>q6Xj7z(^)^uCzgvxvTa#6S-d4 ze4p*Uu}$+qm1WX9*-s|8HMow>MXXpM{68`DamlxUDq@*X!d&aE2vo3OV;@7NL0W`4OSv&;V z=eAX`b16appc&88gp<5R^IA4s_Gi;xHM~aG0&&E}iCtb^g6A&(GB(i>gRcLDN)$XZ#GZPKeh8O!2BTyo!xVS{#n&;*+ID@-QxHb|BuC`-9>coN z4cOKP&lr)nn8)V2bESfkiBPn?a~+Sr;(RZ@wC-c2cn!TRY<>Jy>9;3q7&(H~=Ozy_ zZ({a6jUv07?`K7KVZR@>_dTD{G+f>qf+~&kyWcc?JWCZN^!J==S@|`j+5T1BID^an zaUx8x)^rdPPV`M||1h9I?s_6ul(;sn!9Aet3yHR2@%EBJehZ9X zBh0RCW7BD7#S3VLby)T2CR*`tF+E&~A>#5~dqKA5643Fu-_gDnh-`(r?K~oIwRFxV zbk63IA>94ka+~eG5%F{#_r%T3tz~^4!tA|%<$HW(wb|4auun;LGf*h_s~Cd+m?1PZ zetcMcf|NJy8wP5|cGUGuSJZM@o?bmcRAxTaTF7Z&bF|& ze_dX{$Pdj75{xywnRj&5JYZP&u-l}Z`@G?lBs4aX^`E1q8`-|MmKOkP*0Vkc->228 zop3Q6imnbF)?~`Us9E{UGGa%Bh4H2~anQ!Xp8nUz(R+%{+VJNT>_m4uJ3&C0(>Yeo zb9qQ}PW@OI3tJt)lLoOhEXByKLX{UTpH3O6K$YuQ;COwI6HTf#`MQlyheybZN4a>7+n2!~D3x{CV5#_0BGhZ_%;uRSR3P-?PG{Tj;)Kr&J^(~eW3j#o5k?9W0FMJp3H zlB;Po9WI;_#(G(^*Aut_Y1bs>{QuNm&>`4mtZs|1bRSPV93?b= z3|P_Nbm`2^@5&9*cUyt;+`i8?L)tNF>$KYLbG71itbDVv$IiL%)tFHDija2uuKoD# zwj_PG z?$5`~@sKatf0f&xD(fx*bKEx&qilRe9Y(I&uLK&NB66+~w;CjvHC;IG_SJMGdwC0f z#;8E>uBlLAwdg#a^Tveb#ZW2hydxCR&&dKvkOV;jpX|Pm^BzzyQe&URJwnFn8hk3J z4E11g?EvC`W9LGm91(%GO=j_6G^O|_K2oLWyl1OZop4|tZdDp7YId%C+pPhoFGcR1 zo{2iD+w8n&Q`E;_CQ3hkvpmgLUh76m3TJhuOwlPiyd!wkRApgGstFG77mrp92}w$Q z-RB&D-!~H9TDpb0GZ@2+cX@TCi$c36YjB47u|w!_Q>Y$4l;#N1lx!OC7 zxVA{PrXbq4iz!9>$1X$(G6-EKVKjq_tp=5u(n};%igidYio?7%q&MbWq|+jHK5}#D zE8O9v#Wqx1RVRalZ4N7%bBeQPrwCDuHF*BAVs7SyqV7wqw_8Ta|MpFp&>f6Ody?!< zxG$=lV)gX+z`X?x%aDyujrqdBc;YQv%`JX2oq) z^V*yF)7g{R7QTfqPke0 z9$2@vY2?q2f0dB8pwAAf`lX9ln}!}!hqGQNQP2|)oX>SRM#$9gwjEH!s4z;^bBx%> z+8QD7fe@J-BC;=1l)Xfhv6zq{R-dDft5KmRy=^^Pc8WBGd%1xS0_`R+)`iU68n27l zpw>`~TI;oG=Pb~4vz^f-sKgZF*!DHB(R(I4&min14`HGC9X+(pL+1Ql4F>?Nhy{#>PGY9N1OW4vQcMs0=_ z+iJB5Je0sAgb-xyo~L~e$pfIjMs1G6BQZVIQ{dqeIAnf0V>Txh+U>$?U3)1g-T8Ds z{?xi7a9myLdEPyEZi7AF?$e@~p*w!|W0*+Wl~^cwRzAe4A4fyvm4%r9KFT8@kk!RkxFDFu>*m6gB`KjI<8td=Ka1nSK`h z!2k4pUH7M*M3&!*FczNHO_F1y#wjdL^b;G2D-NJI`r}aL zJ<{vfHz)oO=iP?Rn#TWbYj@?kkE{LR(B3U*nk#*nP6^%wzCrzDtNdt6S$o`@}C_+)p>JHYq!_9(Us&-;AOsK?RSw zkV86GZ=YAnVJk8tEo#_^sL5G_{PCv=)`Kj7x-KIs7AKNZCfhN=m-$?1LH1mo)!7BS z`qv{h^=${X`wYlV`wUB6Q9O7Rs<2Fh7of-^iwSVUiyeR>9%q0e*kVA$)V|3Iv>oOmTY`_&-+`70v zxTeFY<21(p+Q{O|_s;_N{eJiT9S@5gX;=|yEV!4=b8TZ`mOl&LU*xEvRB(8-@XHN^ zt4kn?Qr_HkJbCp6iPWHhFPDrEEnjv=x%3OL7-c>rS9(*6<^xR#_RR9WN;h2-0B}PI zR`1u=12%Ibst?cTST_2?shBgkq`^?QIG+lQRt8YMdt2on8Sw5uUtZ|(H13Z6$xg!S z+qicPr+*<#7Pwy)-*fEXJ=7KCwB|!I$yLk-vo6oId=Y@|-AP(xgBQz9;6j%PGn_Ps z`iT%sN6b*l@kN{g<@gLN=tV1l+Wn<;b1ERPh=_^@&Z<-lr8~j9k$MTov+BcraLfHX zIjb|Cu%zjIbApZZ@su{%!>*+0_bvZBlcLLzm#-u;oqWbz<-fUfvaVxaXLQ^UDXhqS zSf??Wk^;|bDdwg--2!G0IKxBc^ql0m<>WAV=TQB-zx9U9ndqRKg63#0(1x@^+eAyy zbaZqzXhMT7+m3zgi-IX+pJO=dqTo8O%lA47ZxiJ0?cvR}ZSBhYon7hI$8N1CPXvW1sWu-Bjeqv^$4u&AgV2?iVR#u zZc}{hBZ(_K&zNCBE(V2QKv-<0+WIe8gz%7&+#MdQAi*6?{EBr!j+{j;|56$J?kMK0 ztk7xrC)x`@q7&78KKV!*07h;X*NmCh)F$0#GIP5De<`BTxdWn27rsW}i{6Yg% z&P;5*1uZ9P3MIGol3_3t@_afb;x#XdvmOEUpX3Ti6!w5gIHY^=9ll*Dqc%%)G;6my zE<`Hkm|Fekn8#1C_MCy3Up6(~KS9UmH0QwxRi#x2mm#9Cv_gcqwRkfTz|O#~7Kf!= zW!Y+=T7#>9`)qa>_Q&j#8TW06P1_@hFPU{bwfI3sos&KI%YkuLjj~840x=SFY-s?x zp%ERSU#dbJdyQ;}haliXnBR5SXflBLWh&)#*Y~+j7NhB}37Ff1r*FVd(K$azgB+*y z5S?VzkV45l9#&0GWGy}iH!Wy(+^YA5``tD8W#ruA5}+|RYezw6vZd1lv(eEM-=r5+ zt-t!=)eU8wXY1QR@c4_!dzL7 zmf%nZ1I};y%RI9kBB94VZOedujHiJcY~ve8?zYZ`na$NmwC7}ZV8n&epP(OYr3~An zI>UtS*PHF=bLGDPe7;$v_OgksEOlPG&Q8+s%&CKwR5i+VeiZX^-|5)fx0q{RulUbZ z^>)fF$$&pUd=)iaW{W*f$n{PEsMFjZxpnYiB`ug*jY9Utr{P_7JJD4N7k1uEr3|g5 zMmwm%$fWPR^~HL{;QNHqZh7QQVaOiB(mEX2t-AeLzB0r>V}#4S>m}Z_*^CSdNigwz zPh!oZt-Y77Mdy9Ry4gA4DhhH(v^%`r<*Nqfrvm`M-piPs%+kvCX7O?+Z^12Agvlxf zWFZJsMxtJ7Dt?aZ%-DG;m ziO^o+BcE)Aik?W*W_^sXfA17oz7MjZcS(vtaF*!wBbtR9sN#em{v2%0vXU#*{$ybx zyvgO4W!q_dfn2n;rvuFa<`RD1ToFF7Y$)ypVxIG=Z~v4{06B>BK3#Sg!TU@#Iq1Kh zb|Cw0Rd#eJ8|Fpk4nO=u#0OEh*)+feDn8TJ9RcUjpw|*ZdzAJye$BQj8CB2mj^ERj z-*XVQbd`wyf@KuGN)8dZ-v?BKa7e3W$KSn&^yvl5NSFb*klUS_pBVi;ECwF2wT;C3 znytn3OS7fVs|2MYIh}W-rNIUR?4Hr0Ebx?~5LeHl{;4z+JIo6s7Oxf2E{(0E_u_qa zCX{M7P!tWs;yXmFkxP%V`l6IR)B~+`C6-V!&L7(&I`o<;eNfVxpGOcYTIXO~aeEA% z0PWpOXoH^)y*Qe}iQUQ2?~z=l)n3@1%~bkwyk<%umYR;dx-GXrbGZ#k8?`_nQ=@dG zD||87aWiUf)(6XjqIK;}XpSn*#%5X+>>|-guQ<21B}w0Yw|qRwoI4iqL?&nB{;^ti z&bhmm_yuuWG^pN1RTVg-|;Iut;wt5sa-teAv zX6M1C;_hn~TD~8g{Y(DDMVWk=ZmqQI`4f_IX7juL>oQ|Dk$M!CMu&%etXIKG|E9>OE0BneJOcq%L`!6z0$M~$gi{1}JA+t2zzKq-^vc;0JiZfDTseN& zI(Iy3GJnW+I`-l2xO*bYu9dsf;x;B?W7Lx8@K)(%MTQo8e?*fQ|F7!I_oS9;Td zVB%j|UBFJgek(+bOdya9c^?3bSnFAF`6^{1SyXz#i6f-( zX>)R06Xj_2dgl*p?YmRGG%1si1gK)~ArPU8sn6I`kHGXWTb&>D2g5~__pZbz=O^9`q#-BMWBvCDhHsu2{onWB7856Bxpx#aU%6^%V_` z@C34H^hetygnY+VD87IsSvHfr>(L?+wMtB+nQoWqw9HQ4`82OeCoKlN>Joc9D@*>1 zPI~c#E#e2Pl@b!ayrIyDPh{BmZdjyc=v8ari#{zv7$C1)h*xH=7*nqSAQ|6eCHBP< z_Xp(9N>aRxxd9B3FY#?-*+B|YH9&3<6~drpoDvPbUXOn*Gf2uJWXa|fG1Jw?e!`!G*7NG1>1Bd@ft7nZHT){}Ac zD$vXbyYYQJQ)m*<`uOWH!bl@_zA+@f zT8YMXd&<2PUjMu%g~X^vi+}RTvo4R&1D$ zab559u)A>?pX4!BjLSl^Q_akE&T^gULdCmTQ#@IWqaun46+`#D*D%#hOj8JpjlzNh zMGWMuj14qegeCP^LZOh`8TOrmr5K2+cn~Bk@!P89a3PH`#Q9mZ!TfUP7Up_BW`$Tt z(wghqxqTRPA5pAA-?r)alM!LiSvm5N`$>o*$(VFuAB0@i#72C@r`1}wdYDr5JxhA2 z7tI&y_Q4~)kDP$jC5<-K(APtU)!-Y_6}Kn~%h{~qh8!{js?gK@`ChhzGx3Mocubr$ z9xNO&QU)j#Ui!BQwpEKJB7J%q~sw_anRUtDS_}qrXnZPuh?-(XT4caWI_f5 z-3c(cTJPTEO>}`RyIUA8i_AXhRwky;)XDV0!@nblLBtp~0YNG9j6_W0M+-j3Y66wsnCW>a*@I2tr!pX2JvDKE7HVjDbUwN-1L>E%-Kl|nsY&| z2avJ3VuZKjni!z$RVACEosfnByB45^j4#SbYSiH2D&1z(N_BszBRb3Z^Hm20{T#q~ zDhPf-L1-JeYNy`1*PbRc?9Jl+fW`Kw@a%#!nRnuDG|1b5g(GQ=~{PsIza* z^1}#2xES0+RS}G6ZAoFHu(HN0Km7*xqcdl{0?6f9b%yFHQ?6f6P5T0fLArxN~d*Ep%r zLG|SPOBK7Eu^3D{2J;r3KyQbhV0f8IK2catw0pSBp^a7~>X4xv2F(2G{p0bIFFqOr zCK!`K@Tn1+J%d|ssU5-rZ}!@ z<=A@zSL-i+#)q^_PEF_>N%AIN;w>ch+pkN@ZiKq6`a8h=Vj8w)cCOmgls0gDz!!_X zcS5Dny4^BmE}FE|E5tpjoJL4ySju%TkA^`(?c}k|`YbiIC7rApFrudfEgfVW?4o8v z3fI-j0HL`KgkvH{RL2gj@_C5c_V(mZhv*I5EK#mPl3U=628TL_$bPeQLQ0Rd9;O~C znCO?6J=HZmpNNtpfo}qD&_cW?f+EStIzkQ`CnK#wJxn&xaYg6Z1(+AE%b^^VZ!wE` z7(Zkz-8Z=9s)Kq9X@c&NHUu|bq|ZA_30|)St#48kUUKBxk;)du$~uUd}qBFx!?5;i_7hz3JP-C+u9Sh z>Kdz`?7i&2{p~~tEO?RPks1Td6Z-|l(x#6o0d++l`MD7Zn2AJ6Rf^;wBCzzc^C;X*5<>=tgRZmU*5v*mVsng$7V zHj4|hb)BSWBUrbWOMM}tYyjnNn{3*(ha`O?Ixsu{NyXy8w zvy+PfSquE`IlB8`Va&}@FdBp&$JW6MAVSqe(U?mdcf6i?vllPgMkqD z>apHS2~B%rm0hU_toqxL;q$-(m+@I z8-DA<$t&mVD8+3a5&p#a=R^PG`(l0*3bgZ$eY=?_$dvO zph&HoGBJ3?o%1uK(%BnakLg z=t=*|szz9{AyZr&J(|AeP{E1v?38-qZf;Tpetu+4_`4&bz!rwXZ)8_a9MYc1g*Yykc`~x((O#l9R>%qK?B2te*8Qrl4aGl=uUnyamw`eW zNZYSBEE{0>AsN{$BALxAs9P?{kFMY-$4)ZDkba$BeyI$N=MxOMn=|5sK%-(LD78j0s{4lefsJCJsE1%!$A#C-1;2}JiRZq>TR`x zj@kt{OM>%2UssLK8u(qO`%gRujVNUVK!7bT*-YGT9xOQlCBwdw1F` zLfKZfGOcRh9};^|r&2?;@%cVeO80#t zS=Qcrljz2#$iYTs->1mxJEX-hVk-I_Qj$uTed5JGqf!3j2M8pt7GMBg4a68K<;pA@^_jV` z#E+q-fTr|Mzn6!Mjw5Mnne=j*Q8+d#w9HRUGOYnPOJ1>bVJ!Isd8YQp_H>fB$r#@( z8homuRUgaUaDpx%=mWAKizqw9L=8Irq)g?~WFqXu&T2NyT!rMG>WinCK7(py<74r7(&Y5|889gNwuZx?7(A=KT+VPXTDes%xl?S z_S`Rb(IFb%Gd}+QnUKinZvYI5Y+(YNt80N?aP=s|^t%Z-F|EM9-$8{c%pUkdjw^7{ zg3}E3xkR)H-MGFBx&s+ik860$uRVh)qh;k@&X~XihRx1z?*QwK5vZjbwBwd+^V6_c znYVUJ@viegLB;@psDET994(08G<7uC(JwKC;V`@J_-If{5>g)229B$}A+}_@%=*Du z?^aT?8|sxW!jqM?;S|jikp})O9)it<74~^LMT*(P1E*94AwX2av;gN6? zUi^a$@N73KiND-49{h3`bK91_rnerK9^S_GyJXpvj4cBkLon>F@PTG(;g&R*kY%Oz zOjeY5syw^TEXVKhrU(t9Nw5!JXWE30LbWTgMjKt`l5*2Nu%wtM*Obnkw>_)fq*Yh= za{Jm>CmPV27hWFzO!8>LEqFE41X$UYyD7o?qp8qwxBbMCW*?byPOk8 zcCnL>J=>9dN4N8W8B;On8KME z5A{DsASIbkSS)3=@t0m_HEqHqGQz5`5PRFZ&k11ZF#&dbj}ER;rULZn9s zQP$5-3)+AGoN>!B6H9V`x?44D_I)^w^UECNkLCUOhBGEk`|*tVDG>(#A_#j5UE8QH zn&69BC|rir3k(r#4q3HuxCGrXa&l}SSl%PII5(7 z1#*)h*J4B$$ynB$jm>_@WiF%r`U3}6#ZPtX{Z1i|(>A+}V#|UyE1&6pS1Z^ApEKNT z*#!>{xc5;)6C~vK_)U^bzwm^nLDWjUGmy{#OSTs7l9H4zX{1Y#lvI!|k^Ifko1e#f@AvtS*Wtv>duDd5z1E&nMg|s+=KIok zN4h5QcU84B)~q2e&l|3<&P7;TWz;G_z=-wP;VcxB|ELxr>6BGfbf#&DY-ZqOshjv-oW=NRK@6Z}-{r~sBvuY|il z8YYsvc7C7}_Iknm;^q8eg)XuC{<~goGnwe7+Y9#`h3elYt6umRYJ7LsWHAgPJ8c+i z#C-k1-f+Rx7?$1sh~0$P@D$L$u8mgM{;SG*wRtZL3qUH41{xv6o*FpJ;yL$7@A2~iJ(=Tg#LRk0JYwJ`s6S}p-;uvJUgFA z92#Ra6fHk}G|%e7XYRclfkUojCdUlXYfgtn0Np2156Ek!FK3a9e67@6-w+n0 z?%sSLC5!gR?qYE9VsK16zc1$N@{nx=+OqP6CT<*5e$R-kY9yzIpQ*_G8 zM;-D?w-6F!KkF2?7b|<=n3AN1V>jIxRhU0~7`E;%`5S5ioCeX&uJS7qHf^RKo5x>* z^6mZhmkOuMwF--5u5*eZO9{?|G{JSfy}W2hB895RmLrofAx0#G_0xT~3%mozUR`uQ zL6vSRiGCc9@}c0_JIy;*wSU?0tk(KAXD@xkg- zrbuHh0@vu0Q^j?0d8XWU?8HIL4SdzU0jrw=h6J6S(f6|QaMHl?U|Gf>f|jUgcTHQF z&w~3KmfE%O?iJReo8b-XuwvBelFAGf#v#RXIye!R5$XhK51Sy8`l{?66V{SyJ;yEr zM-rzzTSPpBfnz{ds^*z{d27qr!I+?Uh1id8#I3<$sU1}7mG=|O(bhvPj5pe$BrJH8 zg9PLE(eYEY6txaByW2JEqHAxEbbzZ830VXobugVPze_v5t=qRQ665TVMb-t=iZC@_ zy0SRGCF)%gfl`TGBgQpa+}-zTwqF`gtE0EU7TVt0@1YoOCf4(|P;B~UjEb_IrzIhy zz!4K>X!35DJ-V=+Bg`$g#EE`<7fVYu;B~h;$!I0Uxq5Z_RP&B=;E{&l)2Df{)4F*D z{`h8a#)MgfQMdKZxMWi40?^o!ug|@&?QfE>ErRfp&TJJ!^$ebk?ls=02eZYMX4srd4#u&Zwq+KxZ0&|o+(@IJen#5JCfE>0WamZbRPC9SMHWE{lj9fC5{ zj!7W}jhtp*I;NZR7Im%x4{`ue7@=yS4_BPLQ~=tYgA>kGK@=!I{}Hti+utrAQZZq< zw>M5rhZCF~Yy=`CxHH4HWoVAu>>xQkU=JE%~6!;O4x%DfsI_?t}k1!U8}|M-%hkgz;3W|0ZQ--G$7$==# zSD;7nLU0K_HKkZCTXHv5p>&*k{P*oc>Dxk`ZK|<|@d6_W)Ep>%A5pb2(zr;JybKr{ z6?OF>|I+7?P2yOGe8YR%*IC)#kC7_IV21!zP10MeD}z&bpF+IdMBVHB`8)$zvHzzCJ*%HxweU(9+!`|T8btm>?Mx48+vuz6h}keqZy=^de;W^i!auOU<>0{7 zcTMZ8TJxv4pnThDf%rAvFw-w5V@(2TF`8 zDvO$ll(3Yz_DFq29^*^UK~bDD&0ZnDfQ2C*4c&2RN1TL8Bs`yd;0=2HY4y4^m!Unk zJ7lQy`{o~?Y+i9U^&9zHBk_6xK%r?P{(CJb_xpC+EaCXF6RyS^Hl17HS}+ke+a2Ue z)rOTQvrQTnXt?17L=@iBEqJYFiS6nhRKOQ>uyWHfSR!*FTHx%9I=G(KZ`tp={SnDb z(KPmOW5go4<5y)ffoRVbX&%6`{z6j5?{s#`kE-bguQqouIr-K(u~s!Tv%B2hh$KGP zUG6~iPG%M@#noQ$cwhAXm(tpWutv*;c^grZ#W)&qjwt3Hh@An=$ z)7FZ~OMEypwnnA2tKh~rE_XF#VAQP^zP~I~@a`gTH=j6ArB$=q0>_0N`kY)yTEuD9 z`s_%0k6$wbGBCyd#wb26CFQL!A@YhPNl2<_NUnZhQ%Or+RzQLK2Awd9w56p9vLT^R zECP>_9;0b{=eY2V_Kj4_rRUxUSBHYv9GdJ3EB2tAwf$j4ZsZzUYMdKk(y%U}d;~W& zfG3jw1-DIkT7aNT1d>KpWa5+a>SqGkW@7fi_ZQr#;~WOMZ2j(5oPIH2$QrC3b3O!2 zC+|+-@J0LvZzaaYCZ4NdmfUNXOzsWNZ#A@lTqK{X&4r%T$)#(Ukq~n z=yS}v0et0rgTT_0Ip;8!_4Y^y~qlAyL?ao zTfsha;gn-iN^(S^q?TNdp<4Fo1n;zo3X$Kt4{OQ#nKNw_kl;ZDOIFAef?8*1Db@s^ zdJ(GpDrL1-A3s^OHKpXjqCHu|N2O*9pz^r~xFt_IYyzr#u;5&`?ePl&Qh&ux7Q#0>X7*}G#LG(O8Y*;oO zhE$q7ug^+FXNEP+?q9VR zQwySpeoCsjbu;tQO;{$r(Z219yS{%6X$sZ#g7Kxpy7=*J z;~Il1rE;dC@uGFhTkEcaXc4rW9!jdi4I0wudksC*M*J8z$x8#L9#K8gE@(n_Z)N|A zG|=T^^m*pPY=ER+O@NdC<7wd2`FJ@J$E{~mJKAR#a%8P_5{Tb%6!NOcJK5fTszz;us%lmT5TbGeNCUC~Fr~}Tf=tad=qTOoLBXK>RaLSEa zJc~{R-FMv2utoZ|EOE;25K+l?jjOzZU`zKBaK>RTfYxX*Y6cKxQONd(2I6J&GAC&i znUTKa%w#7I(}=MWLtR53ul2LRlQtN>`{ByrVZu&P`=r4~{~;V@TnWymR;wJDJB)l04!9>Q-RXT3Al>f}r2VP~ zI0s%aQe4HCC@>%2QC*)gv5}9TrQ*DSW!4q|A>HmQ^s^VDk>9#Logvd#vQ#$aX=dn! zq>1_+DJ7U2b@Ru_+e?XXwC&PT2TDHfZQo?a9HH#9%{1=|-=;)DjFQ^lP){WaB*4K` zM&HrF&1N4>neg%N^ImZOOmJ{7AJJd1DLs#_DDa3&zJj~Mfdx~ON}U%Wa1@nccGHsi zwqP-o-*%=N66I^)Ln@EDM-wU_#@Bc*Y&m&p`y@H9w3K*fOBMH4Fu&c=t>_*H)J9^O zq@zc%!g4(gLl^S)ePdn(JA+52oCXWL7!p}#vY9FyCal%<)%v1@>07?6c{7aa+WVAe zz_dN>E}jg>^=B!`KIKw6OMBVCn(5rs0)5m51PKxE;iTZ^sfaaVLLppHBRYNorJ1DhH) z*S23`uA{n1~{LkrdkiAzV@)Y9HCM3!!g%U z)86}V8~!AmQ9@okl_(z@@5WuN>MFT~GZyAt9dJ7chS}zr0plyB?1~k|LOb=|{=HGN zARnB8@%v3=x176_#R;iC11TF8bjzHh>3pdxr~)d_*y9C+iB@0S*pqIBGk>xaLJ=(*maH% zZmo~LABQYGxP-j&PdXAydh|?*;B06LQfMx$67%UzRZNL?!9~m`+jf2Ol)0b?y4^+> zS3c`WNXWzUt*QYlTnE>~xMi~BRyZxw-N94#>+hF0pjsgtcN=T-wazKI(kNKk(91q4 zuqTzr|t5rFc2<2TGb9N|jPURalT>eU}8kq?46-A!~f zcD4{B$insBZV#a;1IAXQz~PJGT&PD=Rwkx6&pZoKJoB9Ha(Tt4=3DBo6;X3Az!Stm za)FR8$r_Hn5?h*GO||V{joA)-@;HI3$~7sX>lwH4IUw%)S{i86wowe*Ad=W78Y~NC zN1KIr`zT13XwSCV$EiLjHR*m%J(I52!a9SWF;Ms|EeAN15lx-4JKFOrqbw-ms zP7t(FY1aAx+ZWSih|#KFBmsbR-rss5?NS>iI!*GqdR#6XgX0Rt@3};9F<9KoT5~2ENaLik;ixrBVh~uulct^e_ zaiOymMW`~`OuASPLV=q7^%^J%7|zj_i?N@MLF9crg&cKLx&z9;84Ug9zV`eC z^EH$H_xFauWNI9a@g^G17yA^~j)S-k8;1amy=@!)nUv7j&^#m6U8GX8%B&)2K`m)L zl!#fw2kpH}zoEH6Q}gIY7mm~xq3l*O>6iPN>j;FbI=$n}(dIc)2**lzGQ00?CqqG9 z7v*ktS|^Uc&8-&L{v@XY=~py*z#xno_7gvx#k`r22op(|k(LKjc-1wiuk=Ls=q^zO z-&V>f(N-T8IJZG}-E-$L@U{0opA|f*h(>fov95`iK{?(2j4f1i>kkXczKl{XGqA3mbw2kQ zQ}AMMY-lBY4@gp1VHgS(=-9!*TKCBhblrVUaoUFL@Iqsz{ORz+$ADyc{5mI!vGY0OZvg4fxgOmEi*<7FWttn*s&cX0@|2=oxoX%E zmi6J5LNIY3L<-=dexa-AdHi8r(!c_xM7uU>zRH24&afj$UC-TguFkQ9PCiS()55Ck zb>O@Ykf!5YP`rwi&E$8V0XGgk{>t~#??o~VdnJkucyC`EEq|w*hv7+&zx9-^D=m8^ z{j@f0&Vwl6M~DTV{&*@vt|+IiXY)%I41YW!A-GbMPiT%OTOUi`U>aXcl=iT5w-wQX zwY%%X+X7UUCvMcXnZK{z&-j!Qwhcpi5&rTezy9-5N(fq_|9HsduWRy`<@xhZbO|A- zjgTna?+f?mf5066`sH8L!5O~feNvT6AHUHNi`o%X2!NG%fl48fA{$W7=@-=_^^DNQFNTW3>eftOS@VAZeKNEt)_!v$5 zgvk72V1N5YnvpUD{Ejo{Rh!!Xnoye$cqfvm3EUr>{vW>F*9(?vmQkBG{#Mob=Ldj9 zlMUgfyz`U3@%MrK{mYM-KrHM<_F$If-)0Sax_}=t3ZKvZ+xldCquPLt4S*QXBY>RC z3f~*VE>oj9jo){kgVqLIe!Pa8;SsTdVH}Yj|`r8dYY4LwvjZrtcH@ zWs6hj+_e*lm(vpFXzT@RiW&y>V06>J4-vKkkb7|RI@q@>#kFI9{u^u;2{xEWr1i9m zjZo=Fh&}v#fx~%oLfB(tgQ=|Pz})-99K}+Hw*N6uFp+@qtB{8YG3azR)zaGZ|zexJ65 z_)@iGpX>o)>Qdpa#u)?3Ez?&(;V#NnJ6r=JBtlRTQl|+pFtVoGu>sRC06vH5hwC>; zA%JIV+JmzMI@1O!i$gy|v;_disHq&m^wM`ToRq};95w*8;rs5z&~CsUu($_nY~J;h zK`)b^ZFNxFDB^D;S9tUZ$frNY3p6?H2`YP7^y0uTB^3WKeAzN^`}QQSUk^$D z+-bf_a680mVO>ADXJ3rrVAY_vxZ;ZfM=!DFy{|7PGduu+0E$>F*P)8gdUJe2(nslZ zL*BbV7&RxR5m0ENfspQiByF|SP8&9u?H)DN+KOF8ZyT5ncjk zqtS$60wC;Q16E&{<{89AH?Y^?3~+TsF6NWOu0Fc}6L!fp~0SIUci6NR5Mu>3J07}F!^V%||cn-R35Xd^l@VwRt{}-8 z&3R1Sf7xw+(G6T4QQ6hh^_l+SNlh(c8Rg>j)wk#(U|o0IjAMj6f4m(oAtqOeF}W9$`b=9 zYr@;2gmI2xxZRV;aMSAb86kOw->0w>mh05|AUgpr5FCwfN)A zEu`vqaWm9|mnL_;e{K2BB7ptQ!blnXVLg0r!J%lO5wS*~Ud^|jyMpEckNIhR$WJJ) zSG*&jOzLk;{3{%0MrI9}tEz4cVHt}L^j=-IUVj5Dv6vnu=j|X;yEc3_9jmxn1?k_9 zyCRP*?70P+x&1wE_}x8>up!iVMDC1=!FiNMqtRU%Ww2XK5SGTuUuk}rvnwJke!`-4?;5UA55EK44AfUW0?Ox~LZ zdvdc?rMFfuf7bH;_lSX7%?x(Ow7BW<=48e(wq^)o<2%BDQ*m+%<$dYz)rBgNTtXz{ zUF=^AfRSp?i53VmFAaOEGE)vBzOA=YdH}ChfqcE=9Ucev4>l^ZMmc4W``8Y^qxLzG z+)#Ea`&mPTsLU3c+|<;$cBgo<`le1RYp_fZBNB(XG88&*f0S(g%`p z9hCwj8WvHSa3hGlNdx=ylPx-ThTGb_E}G$Qu|aGnfGzjKO$^sC=9y^xGMvbcMyaL> zdU$y>UikFo5N{}@Q`=4CfegN6`-K%X`V4}^&qiWwZ*U3U$Yt_>2Y}CG_DMz^t0!SN z(?CR?VmGIxI0G(z+Fg7WvtHhTX)Md(?4}w!`qTI+1mB<#nF9$6ly;FKLBQk*{*L+ty?Sf`KJwgAjkq;Z=*s$!6fVCamF8>KKTuUUK%qoU%9hW@D z$UB|8O6UH(Mir;!*Iw86>dr_IM6E(kt#ueh*u#JLk>y>`JLVBG8K>Dm%Gt+aWdX1+ zN^#4X(BEX`%cgx`eAKwlTz|)Z9+a`S&CoZ1%zUe&2XrbJ)x9~Z$uU3+e#d~6n%(Q- zbedYg=k?*F*i>tL2*zs)`X-c_XB^yp+Vkx{_DpnXkzJMgua8+!a$AyYY0%oI_!A(! zJBB4G%JL5<6Oqa#2igPCn8J!$v|^#H0gt9(9f&J%3DKIqzG-X$Z9>?zYnZ!1>;$Ze zuN{uX@lAwVn$vhpDgocsYpu4_9uyXLSIaz0m}y9)V~Wg;QFCW%I?q?dv#VX@eBS8( zyh_zF8vWN%=@E`EH**J4j=EbuOv)bpU>$PuxbX1OON^9|O|Qvrz99?&@U)#m5l4Bje2`A6NiR;m$Y?9CYhp3h{uU80)= zlwuk5NGgRksc}|n04vS{J<W*W@==i)p2kwwk{!O!gI(vFf*6fI4A` z-1xgmSPuItoOK42k$HQqf3hP{5Kh~cVJI-Cm6x$EX0_u=cZ#(JT%nUFyj2T^|G^Xu zo3m^x=LUciShk&i_}q_biAwkWL_>eKV6dPo`y1*}00b!?q2yTNL6QH$7C14T|8Qcz$B%$H`o#}&5_G@m=F{SK z^J8X#N(4z7rIB#9mfo?|u{`PGjX7{9r*gi`BgrB;wP#$co=JL$@dzp|xZw;aGx%sm&{wvyB2*I^=vY;-3iw z3O@)@rZGO#hnVztLt0(7XCUNngkEM8R`pa%Hg&Np$Qh%6Xp+9jVFtEU&<5$#e|R`y z4*p-qzh9`~bb-0nL122V=DEJ9_k6p{OxGi5m~mn7{-RFB)+xY&Y=B#_P%jbAZr>?V zg&%V}s_y|Ut$L}WWvH!#J`bj;JDlNjyt>E$OgDJDvp}NG+NL#@a6%T)mYCFNfwLSC z8ALE2m;lHZt!U1Gm*wwewM->PIr0Q>H$G-6yFX6IV!2gfy502 zsN>`ND?LwV!0@&H@52W>>4RFZrh{lsk%B*ay44zaQT3f%>J!p_wrxOfc}(W4LVgX7 zYB`ZKxjv2vfrUybu8zYy#PNJtYv;F|oe-S$3_N&neKp4+pF{T{5wmyrApkRVvwjDgCJ!kM8cW6=H}vOrkh7>#{N z1?LEC`$&`GQU7p_Ha`F0PFkqhCyX?11ds&izf;nH1iuVU)MJe7JYS}c=8w~FzZDtS z?qmzicLyg>iip_QPxyv&7&8d0fn_S#shPFQOD>`Yg{L+^UgMDWpPX6l3O_v63o*1P zZ|lIs>WcDrsk|n(e!u1Np=;fCy4?4kW-I2~J)-@%XVxT?E+e>f@N454BMe1381PB) z;|B6PY74WeTJRt4xuMoMPP@~eq~6%>2{PN`!P3)b)`Vm5^~dC;e|Xu&<`tV6#J+ex z7we4@$1?v^cDI3v$3frS9!3X-bZ!1ai!-3SCLP~&fP~U&4j@TixkJYJ9}Z< zk@T;W*uNRNU6h0^fqR^uDWFZ@SG{iyvv!RvWth9FHeb8KRJ|CY-NDSC)CTjUr9zF7 z)GSKE)?ExdG`+_p?;`Fhx%;}y6i$m*_Ql`Lv0U5u+Bw2~b_yC9eGawUyE@+$+!AOs z>e##6ZD!x=3ybw4m=W^jGNjS~lCRINK)^c5Z~E~YXx^7gNb^EXbJ_#*(?I&f3ytE= zPU>xTmkm*)A_GDn>FOA?rC@~Gg=f|c%tN##mqw|=-z{`|eId39Gt_Y@n^6x>rs?Q! zak+fM56+5M0U?Y;wrU?|=4imvFjm=tgzSJZhA?-G`}02H++U*llUSG%hQAn?Xfom! zyd_N%;<_nNV{me>N4OHSp+oa{4z(Q1O+k~~0G-Qp_#;nLp}=6zDcntWXirqvBer_1 zc|8S6VXnT36@;cnrjm* zk^T~yffZ3Z=z@Gv4gDkQZt)`QyWJDTc>%beKJgEX!jnrbv zlm-~=dGOUI(7^cUuoU?jK{rNk0K8c!YRtP4QKp2NlsLtePxx78C6nEGvnkow23+4k znSxC;?mmhxk7P%;A7>y!dJTzM(o63>e~JXN&_MR=n}~6^Pe8LIyyd(ljBKxYaH--= z17~T4@CQC^Ctk?69faik-NOwvyI7x1-X~hsTw}BC0Ip8s*>Dmaw-Q_)LP`aZzQ;bS z)1N~k=?Pm6`6G|DiWAV-Ngy!+#$=qh=6+S(Z6`+lbO1{Kbd++-{%YMmCMl;?f9E}{ zOJfCV3#tT5U+U`G(_k#mFCrBtaInUWSO6Tw{uy{wsWQ<1$-MuYA@&qC;OSz&;3M;wx7`IuSKI8QJzMH7eL#DO_)|*rsVSq zeFKfx6iSkB^|7l9e`DnG61HGaH7ZwXA5_oS!QU+65i=I4cwIiuIM}HA=GiTrkiZF@ z;r=Shn#3~ysX1-RTZDX)V3m& z17(`8h+Li^I^&gVv>HyOr0I^a%{6YfobQy{AiU^+<+{?1SoEAjW76VM z*(`7}GcZhKe|s;7;xfei3eYZvsTQwJYC%`HPz@6K8Q0eH`F$rh_&KiY4Z+L135_G@ zc1c)Xd;Ty8`y&MXc*k1*Tx+HYRMt41$7X5l--*X-xgmEoq{3h-`x-_g= zg}@T&7&m=%PvL;KIPitfVof+}!)u2s_DayQ>;!xq1L>8*Jt$DfV8L*N#D&)C+zAot z%#TAnD-mPNfd>y00bJlRV_9(h9cXS~8?!qhL=txS&ax%Cfpy^52{+!Yo19m$j)9Vc zvF`YvONFz%U`+B5dyq}5{GYRFA_Q>^dnQMM-XhZlq)x^f=f0J3NN(zwXBsy77_D=J z$^q5@1G4su>gq9eVPPgz6OcGF4c=ppg|DJFdZ8Ew%cr9pq2We=tdO7aVA*!vs= z(F{Q^m}|8uPSO`%WhH2p{0^kYx zjl&pVe2=x=LsOi}ai4k!>ji1xtLeq!Ni$~+WFz0?{Ks-2RDcII}y!D=C%^Wl1ugj!N(vbY;-< z3WeOh>$UrQ(Rxk1uBz$#Rk%9^A5*<#^s_3}Jb$Cyzkyi4pH89O-@w>GH36Uo+@eaj z>vfp6sKaQn#&8Hpn$nyy?h|Nr0?}mUR!UEBia}Y&8?itHo~HMgRTUyK8;lnqv(he# z@4BZs2y1i+HhIDa+%ilhA6cCiK#5Jh^`KXb`MlC%wrMkLFmHNn12k}YDE-Qk<%>A* z`w$~}K)E?Rc-F(WplAWf{Kg8vvy^;n9sv1Z^6v?yujlQX7#=d38GyHy2EoUcOmt`r zK;Q>BF+X^N8}s~)Wg^7bpD%BMOvSMq5}$WdM`zi$>Qi=H)4*q)99rY)RC@gd+5RoN zI-_>>I{+P`Bd0?X@rUowwyV!O^2)W%`-b+`A(~ytePw*~bCBsGUA|Oi$>iP|L+y5o zj`h~ttM1>32>ci*oj}F;P8|{2_E0z#WbJnJ&x7#?$-S1oB4@IIY(~C=j3PLH-y=b& zDw$amx2PlL%m*Ix(_s|vTad}87gc%UO!3Gi# z4;k%)BPcOi9W))OXf`Q-tBk!JDM%sDR{Z>3+nrsWqdiC;7KuyJ`;5?1o%(x=X{Cv- zqF$&}8&r|27(%H=$oIPUK~p?QFM{(bVEU3?4zOWBo`Pb)Cv^^29&z!cEdXzPXhLWh z@ZxZK&?>r`m#n9?sTRA@L+C0}?>@Z}EvUzs@q{f~K`w=z%3iSBmR=85h}Z!>>4I_l zF*eJ<&DG|^A-*ydCzB)MuQ^e?VXu0%ivpluOv>tz56vLd;P&FZq$P{{Q{@Gp)z(5s zKZ*d*b2I8bEPVB{QFTsh{x#6Zo`4Y6Fskr@Y}4vEjLs|9I58&5NAUyXD2nG*wqLF)G7fEI=#pQ7 z=zIG$%8Eo>!ti!RxbbSYRHSLbVtlRCd~7p6Ls3}9l+0*IlfRbNf>EK#tlBR89Y4H* z15|K?ozs^q%IC$`1A*_l!&%A?T^<)B(p4!62<(hFaO@#dXOz4n`1#r0dH_-d<#aW% z8jYwail~=rsTLW<-Jot7(}kCo=J+)j@mt^mE>AbCK`*v|n$eclh#88J=e*%(Tf9nU zEvagWK33W>!FqQ?kwxRpYMyH6C#5p2Vl?+;Hhox=$KqG%YROW%qGM<-Ztn>gED}x{ zRF_JZab|Z~udgl`mF|QA(R;|dZ3r%oz{surK-nn=O-;tv=^*RLgOhp5$$Xs&l8rff zZ113YR4pp|rRqdybqS;vXy7NIN^pk>xW!pVigr`!aSj#V9qYnpy7Y5D6|ldKlVgPN z;2yFBz1@U_2^WeHJ?D8l_Ig?s?dcm2jsgHi6771Y^e21>9K=?_&eshr&U+$9g;7xN zkXXPF$uJlq$a^x*D7%;_@yUQVi3sordZNkrxTD!RN5IXt>#IhPt9=+-m7SGdCY|u! zlUpVe3H$=lFA1eUvjHXC&`_n!JkB6FRSnnJz(>P*Wb3GBvX;Jqy@kpJMiD8=Lf?YL z(V3hdzPFR)dLR-vM7aK-Kw=`#2(sdr9$IMX6;9CGsqBK^@z|7~>vxmMe z*US|3wzBn+g8S)G@YCBR^Ty?gJ;4x@8v{j8`tLI9j@VB}6me2Bo!8@vzxR_+NUhzy z1EqO?jyMUT)(>0#06F6Uh_Om^=LseiOZSaKaA4m{ix>)T_OD1Ak61bghmdJ3F&b&S zW$Q`Fl*6dQz$%g#IMQgHJc) zl}*j1uDF&^eTji7dvEy?5Syd!zyKoT*WoJ4;1lR+%*aMjLL>fDv#c>=et># zxJ=iX4W{ZC!WnNw*#)+*F!=jH7Xk2d+gSL<^LL>4_^W4x%ZJLI!8q*rxc-sy%Ra!@w5sup?S2F)I8Q=nmC)Bn~mHSaQO8+dvF10`;cXd7qt z|H)nZ#fk|bBLr!pi$L72zJH#zx{oAOe-n?^;8=-8*7o3h2GvY20HA9a>-LZyKS?i9U4=bm-|%#a_Y-4ujN3QX}tYD#It=ZEsZ#w}6P@30|858po}#6>4sc)3`s1CI2{1 z1h@{iphVWx9F2r$im>Y{^B?5-2iJCto$|(hn|nG8kcF&pZRR<7;BxNMiKyg!gqK1) zg?Io;(^S#TFE{ZIw(Y(7?k`&=ntoYnES__c)=J}Xyu)kitL>Rn?2#QzzEwUkEhV~T zXUtwHs+B3|?PXbE!;+7yCe{73qWfzT1WsoTQsG=`*Rm95aS1;Z7e9R6-Q#IvdXtRN zo{oBhF=6qE4PK%-YPX5I2NlvNVsKa~=D^Em`VjiJYm81+SC+LZ^wncZkMhi{gT?WR zqD?K7;3ADtHkZ|`@&Xk3zRiF;?zs>K%l~Bk{Nfo2QG!*Rj-!>9dTaz(1@$w&4?n;l zvD6lK8a={8VXaWZTpe(RpB5LwioKU-$L`I|c;mX?O+deC_oIcU{zNF4S;#`IGDaD|Bt< z4tu&CE_B3_)@~a_i}jG8Cu(~UQ)=qPc!HqmM$hf1GOY2G{fTBKTojV|6UoMwXJZlx z>GMjiij3}$HA(~TgcK8YkD{9Rk<6}Cg^4+8JqmCu9~Y}QE=UoP?k4GlPI9#o#%1$pZhwu$W1p^omz z-*r-$OYhs{B{8xp2=9fY6>oz|pSyT<9xrKm9x-t2iJ3J7`LWJpJ=_JMFHmhl61$HS zQ=&Rb^RGIyji2|_^}NLLqt$+^>O`De@NLoC8^&G7X$5Vchk3u2Xm~Z5{0*=8|CB9t zVX#rfj2F0jCCSAM0BXau$Sy?B0a|x(l)?&Y6kjs&Ia$hBqSbAGc`Wk1M$aNBn<%ja z<7$)$g2>1Fw1Z+jZyvZ#WD1YH&QQEVX=fvBQY%JeXDREHAiGMw8BpxhqW=sq8zIp` zOWETQ5iFR@J`k6^^f}w_CdL?lnsOM0CSDuDeffW{hZv(yk{m0+(?%IBo?h?)G?wrA4-SimII$*I{dt3 zDe4zC7Q@Fq&DGey>$d-Q{yz@}JT8PBiY~TPAb4qMXGf6weA+OW0H0ww3xq3ULk4Mq z8mi9su2#51a-1NPiq05*Fq%rx!sn*r7fq9(mOM*=kO z9_kYmJM_^$BpXy?EvwIu7kZHcgQ#6|f}^GQvAazsonP)J$RU-WS|=&!dev!dlH*y% z*e-k#&cMaC(D(YjHgSL|DwAu+;6h+)K*}x7q3*Hd@yQqY!^;2+%{Kv5CLmaTS*_O) zWT3;F1Ftqx#@A*)nz%1OtchceQ7)O!r>R{=zK&GPjcsDk#d#1gU#^sj`z%Y_kY9!Lpo zOomAWw7DOytI|NuqAEp?hap!dWnW&!EKXPeqt#N zyGr6XsH9Y$0%tv_c~R6isO&FC0X+ZfY1)+O-+4IyWAn1L5rVYPMG($rrdnz&MqjCs z@oePAA8e;;boTVp4d3%TJHl+jzeDh8ft@W9!cUQOivC7u|JH!WwI zV@N8Zc5`+6QI#tI$bSwzH`BcjO#!$;?rND3SC{F5wf(>pC8yW=i2fY8vBKRBXP|#? zG-i^X6!pgr^kZlwsv${Mk)6Ari!nVmT01|H7bPI5Wc3gked*t>UDOL%#cwYx)yT}& zhVoo6vuv6Ro`3XN#>;;WT$X$$Sy@n?n&~vAl1*l2C&;w8ym0)isC!xm*m@$ODQcVITs;E(jR9M2~Z6|IKyuXG_m zZ@6D=9{=P)Mj_csm!Fm^D z>%w7|Sl;-CZq-autYr||xRN9U%AAA;KQ}LP1#&Juz-D{w&+haUCfD7Vgmn8G6{>2= z0H)iot-z00M*@V1PS8aVL!}o2^Rx8~-uo{}P_-Q47k&aDYJV3Cx}%gQV_b_PKCZ!2Za|>p=c5NilNSI)pj`|r ztZw6vyLV_LQmMxGU@cIkCtJl(KamMff*QElwY0^Hb)6nzzIRSrfHgzU@`7ys7Q7S@ zEU5ryms@_=0sGpz^~5@F{$2iFs`VQN0?QQCymnPRF>NTldq{qS78TL6F~AmZ1~FZd z7OfqKQ>%*9l2r3_nG*PnBWHi_1+bW?%tHYX@$WT;_(K@seu12bwZggsW3X=4c|kq{ zk)Rt6;TXpmq%F1>dzfJZt^H&vm|`g3G|!79cAOQozo{_pSzm)t|I1#(wU7XtO=Ckq zn~W2!6r{Hd?&2j;QiwDY%pkS(%z)&=sT3vEr1&kW@g|R4?wnQJ-xc7pJ>S&p0gn9$Yk39FwV~&Ch6w&QJ$KgXk=T&@$vM;1!sN6sd5?PP^^Se({mVzfs+C6z5a?>?Tk zFkxk1IyT05$Fk!&xSHgC(-H=OWzf}_;$yPF@9G`T_FXhyUK;cs!!G^$&Zp^CG@$v#V6LvNt18_4DlQAJ+;l-w)K3SHJe7aD`J>XhF?!??WWRd=G6-090u5 z<3|TH9Q>tRUd=95{Kj``?XR03U`SMYs*61aMTRQ0Au>LE(nSp=Sv`in2xh;yXPC>re@_4dS}+E z3M$^({>Vs+L5?~$o`_W^MLcK^_!`fVN62rL3J;I|J;EzOwYd(J_jtt{sQwjO=z(!# z4v;ipE;`-@)$8zjjSoF1Xc<1GaoBKAqn|=e*^t%B^dcuQ7pV8C>dgj8@Noi|I^zI1 z1(F%Om-LC>c)y~Uw#R5?6FDguwTZFN7xrd-v3>X|wnVov$>m8l4ld?)u5ER^_=y~! zG&0;|l4I_^gBiX54N0O;w}YO5ER)PSGLMTyGx&yx!rhPT`Jun5cihl%VN`B^1|oQV za>=KF_>N)xxJ2uJ{d7=V+xO2|!?o?qTjeutuAl!R6C;tNypn|%`RMia2Xx_U>n}u_ zWH>7xj9f_6RHfL&!)`^K%NSIvQOz0!kUDfdj#E$nlupI*{^d_^P1@o{n@?t*p{GvByJ(axg$zmMpz zV;+e{aEhOf&M`|nPiq5sF;XFXS}nQ1N?q5#|86Y!UBReS+Z}s2|GPoQ_ldOGel@<| ze+q&CFz&fgykSb0WN`XJTi{a62jrTrLykRVJ>_<6Ky`#;S zt#W+4_tEV?=fenOU8+t^bpLiFVNY&k3Z|EOc52h~pYtIAk|jo7z4(6)N`L;?7i;j_ zo)0~^_5O1{VZeqEkd^(>LHwUad&K2C>N)?6lhvN(*FS>wA;+^qjck@wdq&DZNLSnpkGp0QcM>q3HBfrDj<0)y-G zup4pPaRg<1_E+OfQE7aad(HVrs)x2E|M+lx1*n0RQ-G8`Ee-edsySS>U5PU2Jp4ux z1TV#MPJgiGXSa&K{`Mb}JPQZY_l`_1y}S?i?cv*tP@<`;*?KC}qq{frq4D*Cvv$S* zyc>^hg5?l^(4?gRW^ef{xn@!iDYdT`jlz)>%Ew?!?_MCZ=_RlJ~>>k*bKfUp>=?Pq2B2%kD3Bp-? zTX7oqH}j7F(=h*HXn*}DFYM$)c{%>o+rd71#0#F89tG9x-9J|F*WXAp0tN4k8h!EK z_IC<4yEBxJ0{?lSqrf|zBZeaW)0i~N0pmFJxcB5gPnDC5fm6?e!QUd2FPd=jP34j&e}9IaLE zJKA*=Jq>bcd?{G8f)f#cjaS?N?{@0@eKwMW>-lxZ$@J6ReZFl@tBKd! z>>DqguM&l9;c~=x*}dlE#?7`GR!$|^%G~x7RhiaVKAx`QeXbenNPkYfeYI&5#H@9C zL%yIHli%o@W|b#zWHlK0<+e@cIpI!E#hdYUm(L>Stt$gmH&Jc+5m;L+g8W6ZL^bfd}7^G|Mp){6Cf-wR^78>Ui#0=XQ&5NAgS!!3rn|Y2{Jdi@;FYU zoYT>HIz3hU_Hf71CJ=s3v-#nP+wj|+r!QcNhF!ul-(I&bt}cK6@wU z&X-X#ePiDe- zLo;#V-1bsX4|LOLGB%F$lH~8wE@_LMXsEY#op9;SF;x8<|4rye~9!kpermmq3R7 zn-*3Rv88S2r*mdMr;0WCvN`8?Vg= z7L)n8YqlI;!ds%ihg)Z+%?{G_TE079fsN#UFj|lCfIHeumV;%}bu~UtBgz-+##xMd zqnXKLaOwhPwXeoA;@09vxTAizI1A@~6~+azWPO{}0C8XT0prMSHmer)C99Xdmqp&Z z@AFtAs<+vl?QhiknV~M zP0w+BwXfYj3sr$hGqiA=fUT2!m*vmJAbNPU@(MLzWC^Hu^X%#iMassl%`W6)U?68n z$s_ghY6qMCO*>YT;qPy5Of7rW`Ug&ya87w4N5^`!A9|WT46^)(pCc@vgJJd8`?dRy zU+3DhEU?GuqCDN&+w{aLh6c}FY>~oGLhrSphC2$+YcsCS8TUi_*JIA_gy<}0rxZy8 z18flFV;5j))rz*89t>~CFNb#z)0?ph^9S#k=?)Gd?xvRxuHzyvTzBBE4DAt_F@xaF zUyIA!ptmovLzbJb92c2vYsgUb(uCOyZZd5~X`Mb$`}qX=@BiM8=(qWE+94UR9LU>l z_={ToE?XyTtY7#sb)CxZ1{qTYn7J`rXO8UB^>L?7m*3{*QhBIrr(3(!?p0)q^%YeSZ+kIuV4q=OS@c#RmoG;8=dC)u_~zRYjbS+%>TXoQ0?E0SBjdG;#Ct9~m8Zb<*GRSJEt z=%t2v;Hzp;~$mG1Lz!?F}AR=+Q7nj@0JR&ivIJ>W( z^kn#-abw@kv3rPs&F|oj`qu@2m;MY?4_0lDr#cVW?CLM;1hQzH<`3${POfk7QLtuM zdd%Qj&taW!Zs@96*xT&b!1Ucq{_iqi{;~cvwPlVh(rQKbzNZb}$LwQJHk3_#>JOXo zeTgnlE3BPel{VRV$5>#l%}b-=UUI|zMU(Ko|9*n>63^{>nYQyMJ4|&;o19D=r%4Kw zOx=gETt?%9)Oq+?mQ2+mEJ-6Cx6A7I$lhSo3Xye8li8g`mF051>q~1ofjgj&Y;_(5 z0*=QeqHUPg|J5p{UE*%lYBA13o>Q+$QlO~#iN6Ec#q+OJP9-}YtHnLM=4tEE$U(wj z)Zrs}`*F*!yDY64s+5~dB=x4w5)a0sRKLB!`lp+IkngQVccb7elo?;PmNP|cHaZ>l zPL9UQzwb`>SM9GXH)zMd9}zLo5q{gYW86`rS{k$A8ji9@B8qL+k&KwVOy-N;|4cCb zhjRt*m%KY#&2lsF>+Ed)Z9QD=+2tx>Y>LD2&{t}z%n66Uwef)1LkBOSMEsK1>D`%^ zC^%{bFy9xUyGcA^*@Rct7xs1$&(6_Lqg#j1NE7&!+K6I4`d+LSYd3ROzWEmHb!y@F zca)&!CL&9-V{2Rzw%@x=c2^fHB;oO^X$C-6rQViHY*$aOOB1Jb~u3>WJ&h0r3hbhUQ~;;p=eYQGUT2BPbm z?V`ADfBxMuo&5JV4;Sh6eJv}92KAv)zTjcp0wV@4FU(YUDe~j#;6&=4o+1i!UN~OE z)BF+v2gx9;4WZodq@@=+-EXkV0n=`%6Y^{??R&KqH!n{S`;WFBu< z%$|Ue{Da!>61jvqs!cP$@BstJMB*Qj&qbu~T|2Y&nt3JHUC)25HTmyQ=^JYLY_JC1 zx!wXBjM~v|h)~vRZBK%R&XEjmyRDT6wHo90s`wSBJiX$C_%9Ul!N<=CuO zYb{6p__@~SF6LKp02GE4U--2{lT$RB(oq&VE6=y9aLQ!osY~#}JkLSTUM`s;CMS^g z%f}Z@IBg3N+F1jsX`rbFFtPA(`+lEGpJnctl8jE%jM5I*l>OIj~ipOBHx(o&?F;_tuxCl_e z@DA8f^l~C=#P+aRZ?0c~HtsxTdmoR(;jr8BTW&@cwH>Ix-nO<8;6gVgYdW_GAQ(75 zs{wD2C>&WkT}JJ{O=2;{i~H=~Z@BT<54Ot5pN&KO&n(yP$Rx)F61EGI?JDdblM(>5 z>p;B-=2ecSjJv)wPTMic#1%(myqI{y9Yc_ybJ1{}HAkjb+Gtk}2mdH`Zj#(&JCLRp z&^S~W@VHD&yWM^XVsf5d%u>V$~LOz z`U=Sf$V_VDdjopDu9oj|2g;vA^n&}u;)&Lo;2Pl=&wEUC2VeHlg$FO1`VP##2V)~+ zIg{mISyCw0+RC@|wh096DmK2ekjZ0aCE(__q zYEP4B{fC|zxzB6wnxUmD8j**@qBa3<9VRl6$zrZjJjRzSi2<&xpm})6xOulBVxfV- z=P9IHwQ|2UEN`tFO1sDOYv|q|gOFiT^wHB)npI4#uCnt+Gwip&vlUJy@zI+A)V0YV zYkkM(^V^2+eW|zxGi7_?=bWU3Mq15y&{FE-PJJI!^63n0%ON|8u|Cv34YVs+b1uE= zRL!uYK#$Rw983nvG5+Egyg{Yh#`hFy_osuh)Qgs`wsv;(9lw=_m@klYD8sKauuxJL zDlRIuyG~{>R7Is5mJ5V@?>%Jj^bOs4%ST=vJc9l}LUIQ$9-bEL)S*~)4X>NBkZm%R=@tD!4KB){fEo5?@ zFI|5dijk0}phdp+Z{&{a@1L2IZQMZCwCTy}o-mZiFW2+JncV z2K=QGN_f8Houbp}jP9lP`p9_8(zv-39qTQ7F@w0m?q`@?e|VO5;r~^m=FXZ&XAyZ1 zAYsZp-`48Dp7#VqB#9M&S06w?1X=i5(oF6Qp@2dU-%Jl}89F9WvW&#giaqAdFOTqI z6LK<9aK#$N<9SRres(NI%!@asM&0pF|6peH1k7mS05rI9fB`cm_cj`NGU$v5fMC_J zJ^c4}f1w}r$!yq(J~yKjAg&jeb2dq`B2Q5^3c#@bC=Xj_2ahr&Vd-0(srO}&*ZXvN z_F*K0ob%l`XfG6q1(X&eN7PI_iD79@ZY??918>|#|Cr3MhI2}nnF=%j4tMU5eL)AX zj0;n*nCHH*G&)!LttWHkB?m<4|0=ckcoNX3H z{#c5YcDtv~>~!GmavWUE%xcX?$O!(9Ubi7_K{Vm|! z%v=^o9Jj-Ek-lfjTw?)|vh^TjUDc<{du8eQXgE~2S1x!KB&3qFMUhkpc~op6eKE3yMm5*71~2I$?#~S5E{Ls(9i=uP49Z zS`iNGcSp_5ikxLw-(FC;!AV5Psjv*GT$s&}F%2puLn>l9_**z3RU1z9`nngdNWpm< zz0ky^*;W<{b7{E}tyV)Xh2;2LLLNO zzJks&@ThFO7LW`!JOSdT43aOjKko{@d%vu=ifG7TU?*E=wu}5ywiuU>ZbcWBFN2d7 z@t%~uwm->9kTIwXx~^zY>5-N9YLRz@eQSX&yNO`}Y0K*pJYM+cDw@w0`=6rvH^~W1 z32tMs1q0r27^k(alG=X2sHU&OI`D8|80uj3paqLc2aFIAi@=}ZHCu81?xO*Cp$OIJ zB6MXaqtP1q3qrPi@#9xzar1;Vo*~r;D+0*6|0vA|4Dl%uN5nzmh&2yH3TpLS_Z;%l zMX(OZN=_@YxTrE9qb@Zu8({hmDYA4Ucp_~_WA&}Ff;S*;z!yQ{#1Mz@o}&FZvPh(g z0!&{2#Efc3#8geTDz*R(NGxAHXQ5fHgf`d4ilV?c^(H^0h^+Z30FS_2mr*U-3tT*l zMRp%!l+3W~V%Lq&Q!WH-M5)*_Zwgv&B4@szoQyq)s3A=8*TE6T}nL=qj>75q+ zdEd-OB8*5xn1p3EUE|#)`33s&LQ-2&{rGxs&E$m1N@e?)LiK(C9aErx>v-INC)N?r ziid~nqs7GE(mQ47=KQSUO%@Qam>HQkV#;vJnv((!`X>2K36U=c_w&8LzLcjMChM~4 zGIbWO@G0&x?CmVC*sy*d=o3TiOsoD3lR(g97!wO_nzZyh1(T%d;n9EC4T%-h0gHL{ zgRytj4{X%SJ%)nY5qmg5O{2gsm(H;dOrC9vz9eIFWYA)T*tX_A!wif?oC@JKLCT60 zX~N~zkuY)EAuN!OvG7)azjR!b7K6);$Q+CHg2M(aqY#K_e}XOUy@TW8|5U!1e<~kP zNEnFzjzQp^2hTuYpa@PJF3KupYI?EEdTxlt^!m;SM{*c4J5|H@CyR%=zH!Ei9W(^l zjzIq3VhPBG*;i+doNm>9q#76iqDL%tpG1Mk7?yNS$bi2aB%EBBUfauK1y4Na%xMWD zY^5HrO&Jj^e+8E`;UgY# zP&n^kY6w=eQs0dMTMw*aO^IB~i8|{+HF+1H=UC;m2wW4};EV<+h29)CU<7(IN?dwE zmBKEve^vE65;vVPQ87k!Fy&TTc&%fAqg*-94f}Ir7|-<0A#}Rg`N=}ct%q>JK@_42 z^fQ!g4x*t=^OvYR`^lF{iZ`r+5ss_E(U_uPGsWeS*3sbs2x4a=q6eLV1h z%0l#bie7_i%H!CgF=uXAE-MGx(zA7i63B>n@L+?rc4=!m!{hE!dodvP(s=|nwyHs3 zF?uUyxYJQ#gMA{TSO}6li_)Rc=dplqjNXWwZfm_Psd353Twp3m5|d4jxXH_>{3d4y z^-CBrnRwC}^CkV>rH2bYA-N&xlA<#{yKHhI+HX(A)}J?f_-iPe!!>=NoysbBxwnT@ zD7aCk)&SUmj0Jm<{@^W>W(}NpiYVeF%a0zV?7;#5^wt?(H*k6rdOFBE;Y>{D4FbtW ze*YLwmxT8qXT)NS`sIiYI#8sqQ%zn(L(%*DZ$!4ue4{~8j`q%yP)=W`mS<_}N*?!X zx7q(kt$&$9UI;(kO-%5nbZYS;&xWe`o!=sKvAx!;8#B&=b05U&y_GoGVjEf%?Cz%M zvkVH1(wMZ^si^REh{UA&bJ2s$_o2%{^~=#A3nGPmk~;ButB*)n&{D8#>gj2qbO9M3 zG&it#w>J{;fy0EU0Nado4qSbsI*sv`vfp63jOMRgHepu~)M=?jYQ;NuAq-(UYmYy` zJmV3`W;XjtgGp(IEvb87+}M-}>3FtJzZ*-tE2bCs^i? zlfku*Z6KD*ps~SUh1emMED&)`s-gQKjGjH+psCIuvD4AqXU$OUhEH?wWP!2N|1*hjI?^-e^IPi*i5R zG3$Nr!|gXb-HpDOw<<3Cy6Y>oZah;sG-9r+!sLWdAfPJo^4gFLVbN}Uo3&DgAQi{v zMg(nE)~y}|Fw&P=zu#=PIJL>WQl{_k*xlwZTxkf>N`?q-D8RcJ<|>BAQd~W+a~TYK z=8oaOqx;l<@HB&s8WHe$ka#oCHe-}xJXC69G^{H+Wgp`w0aaFBy#oO1gj(L* zQ7P&;3mRTJf&;p0pYyrO)Okt;Of4`k^uYQmhr@c`8+gVH#t@$0-t@?HLh-c;Iq8dn z+da9996*gL)d{o;Sl#Cc)Gn(m7PKIFwwhh!w)!O+6l_?Ug6>4S($J2 z5u7hz5*oWK*)HKaj*_X9ZKL9#CIpL)!Tiy_`$9|6Vd3p^THU4=pI|(8y4KQZd0Bql zgPTi#Pix_`ETd!Uuur)mle=WZvzX09y&JUKhS~@rNOZ{=H-4x@kSOH<}z{v;gE1J zPUI@svm-W+UL|0VfH%ls81EX4HG_|6BGz_&CWHhFGB5Gtt8V8cE)f$Q*!(&)2C`pJ zTp!SHHq}2FIT%_n2-0FE+QC4{s-U=GTSMFSlc zhp`XIZl_Bp=Pn|KI@#9SXzW6_Y_UVwKvabs_oHp!U<>`+5dY7vh656dnW6+KYL@CMe3TKXpDeax34`v{|A=T>4eGQ%GmUo{SRqDC!BNt zSX_5DLQ2@v)id`U89iljh!um=O5RaoCd*DW<<2$D};*`xB9 zY7$(zBmvUX94%q^C&_ONld7Ix%UJz75|2J)h8UFZWSety)I>H;>mZP?m?4l8`dlV>L z@?1AN%bN)u$}HymR{-)a9p@zgDn3kzpmBE$_0Q}*FrF*fB{OmvX3`(IM)l6ca|_sE1Kb0Uc4T8X0U%C0u}|*G=eJ2TBFB~i~5^7=&*G-LNPJ! z`omlpHa1{33nv&kmYAT}zGs@k8D7{?B;$>=(jN&SRxmA*HA9OFbXNUwvt?e%T9mTq^M887+I2>)bptb%+ydn9a11Hq1!C zvWP?ssWTIzf>`L>v9CeZ#OsS8ToD@z0ofzgN?7VDO|cp|Lv1h4)<%a1G%)NepV^X* zH!m;(bjnPqmjL}+A$#hd=3_NamE%3>;wc>V=VT03Y}lTkxA-?x*A)Zn70)euulY&g zuZ>+k;vdUAd%ox^b}6py3Ro%{$nHqg&|LnRUh;*fd>t_B>WWXANDd6U9PI3sIX_-_ z)#PJG3~jz_;iHlC;W79p3@@WA-E*C?9`A*QPX^UzN`mpHD8)M*&Q?Z6{JA+YDsgXa zT&-7Q;JvSBgRiLvI_EQSZ%!SNSRfEysRthr;C;yp_ifG2;_`xF0wcM-uAEq8O0|A^ z`X8-&9bIL!vnW1eEwYo6(Q@TftmsW)F(MZmrqcpkSt?e&FIz|UfdXM-5IkOW8A2vd zhgZ%9H?XWx&ApHQJ^cNL_T8g{x`#8u9-?z_I-Lmi>IF@J{aRjvN$i#r))FaZ9znd7 zD?fARN4nL^2E%8&L~&UG!({IDH@s%-H710o4LFPE5F>WJ)3&{qupEWp(_zTy4g|r$ z{4@^u@J(~rRzrnj27@dAwUsqTUfYjMV&_Y$k_jW}E{YUg-78LX7tn_&Lv}4M6@A-n z25#Gv48koXoQ^(7t55FeO~ge^3t9lV{*x^@iM2%alBTgVysi@$yRSM|_~Wks9W~DF+Qz9r zNS^zw!G+fJG_i5{UrQxr9K%FC<>H}qBv?(BTn#MG0z&Y1+8t6cC$v9(qvdy!=n^eHJ_jr5>MJkQo>=EQ*~5iOK33 zC1m{Mu}&8}F_hyIZLKjjs~v@n`A++_-|CA{Y6(9Dh%V9*;}Ydz7geQ z5%8&|ogO-?h_ceU0f9k6)k13{$}IU$m4(N>r0 zesy-W=lwLZxcnxS;VgZefPTF{5NvW(HX??BKdUmE?L0josSxghjW+xt_IqVS#zmZ6 zHhv6nB#G$C`0t<0hE3x60VSY=9RbVdIY;}BT;*AF^-*Llm=8Hm2_pmX0fQnBh^GN= zqLb2f2YvSKu?Y(eLqnp`M$K9u7BOKiR)5#s$ynJE--l6E$=3~pKx|UEd^@pkH~jLx zA&N;kP-*-offTNRmZ~-$5<8Sra!_uTXkU+pAq6iIENP--2q6icOA{{&drq0_z1Px7 zcZ$-;WW1k((F@U#E!Bh-_~=FW&%~A`!+AlUx+bndnj<(#B)}BCk^K&O?hnokW33p} z8NVu8t(a3ynWOpAfmYo~#7COV9Ld@1q2WT|{h^zm5%8XzVQ_cm;xCg>x5O>C3S1wQ zm>gMM9`iE+?}=Xq%rj!PzPq>Q`)F-a=Pd)^l5bB$S#!;^DBk(YTC)!VI&{X*IL1w68js)!Ap)!@*q)YALHoJ z@Zg7D3Jy_5$%5o-JAW(TLq5hvVoa0}sKmYx3mZWYDLca7*6yegam(c3kMKSr4W>on z`pS|&?7!1r=Zw$NWi{I38@+`3`i6w9=y*~*FWb}sUVr1kcLjn!#!=;I#OMkxh(x&p zO3Qu+Ah1l37^Kc`eHpq%EZdEW#vq!b1k^F*2u zrO^NqL!vnEe$1cOsCyeNF0G4PAZ0KY;9{h2V<4uhJsZ8CbgI;i8I5L0_f2vH@7D_? zC10juO7~Y7QC?>u4*F(MY0jCoJ%%iyXH>m=hJLv$5b*-|`OZJApp6lJX(Mq!h>%I4 z2__DfEFEi}f$n8jKmHz+K*uFQi2mtJLv+-&h39QCz*T^T1Xb&+ms?o~@|-fAD!!6q zJ=d@>r*RICi<&$0X*%DlOH-F(ruZh8fL`i3S6TSyRkfjmEf1s0%j=zaTwASRz0xaIZ_9 z&rzVtoUueh9exKF6Z>Zb$dBYPE-{LrAd#&_!|8gW33JH+CHKpJk5-J_46>CRve+4CtOx1E#*F51w>ny z{5B+0`{#z3ebX#IHeD*Qby(rw^qH{XPE~2R*$UwqSxncQXdBoPG3iBJE4pa9kctw)jN9d_ z(pGe|$|c5fuxOy825U?UTL}jdcuNV&?f8$Dnoqusp#mw6s7whODzKd`(6Y_uiHfRM zppL~&i_;&;h0NsG`*vNV*_@cAl+#^MR#=D_%Sa-9N*Jh)3gx{6q%Sk*2Z#1pz$PCy z#N9s^F5lhqnTI2kyyMMUPR5!rklQ?tI<=Vj5`q#ooNIv#zdk;PHK{vAyHEjkwK)om zUMa4#U|JI|CLR@Lpr7$WFSmRLQKO% zQ&}nL6{CIZPfu(mkXK;_1$=}<|59Not9Hx8#XIkm^tHSGB}B%Wp7}Z$LM)NPt7&<* zXTvre#;>_f-gsn?oq$Hr+Eop-SbFG;2YigC;d}&99Jq$Ew(m+b(p|uS+1Cxxsau^- z2t*!sEx_p>QtiPIyt7eUN(w0s;Zn+$(m^6@T#;jPBF}r$RGm+zpa;mfPSq{ow5?rAb)O694xz&O zex8nz^dU|N9{wB&r)+>_L$dDy8uACP`rdY4e_FubF;-1%tKiwh#U^eM{4tourAf^q z*z*;RYXsRG&gE1_O~npv17C=r6-L(u{h>MgTbl`pUMN8@ECe(DT?tMfU$v9QE=ywnhhQ!bz|XlE!hp!g8rjp^F46&b0sFE8V?sSNko3j52kzj-gVc=h$)kb zuJAs98JPiE;8sKt?HIu4HxeEVgJsmAx+dYAfuYUE)1(W1pR8z(&Fu!o{6{wV<3Ln7 zYdfYt;ivC037eg)kBt9M*x@SRw+rp%f zquiW|5!3M?f@C(Qv7U@8~mn$4QvCZm?0f!1yy37?v zo3iwIt*ow2P1OQ`;W&*N5RlF4JRSN;SreD5KwGGxHaFjB{Klv!fhrswI{+$-ISzX< z0WybDnU;u0fJT-g#i$80S`&?DW>bQwT+0V72SA6L?@s)+LC-0MM3_S557L(>pE%Fm zmtdCzMdh~D5BWQiUo~v+sC;!{^@j5g#}Wh2PsA~Rlfl{0)>XN6nLn5`6++Iv%+ZxL z10^#hI9Jsi4%L}vjhG*Z`(MbYUS4WOrP5geoVi=a14iH;VKX^)UT1g+G&iNGz1GNJx8i!ajkms z%ZgSJO^e%iB<|5G_S9=J2 zUfqb$_Rr7j)StAyQ#di_l)}u{^{g)=($~=8MzUf7&k-Yr&|&Zhyc>jyW=LZ-5ycE5 zKkYR*I11UFV$r5bhQiqcDqIiaHhfwg`?8c3FilD3d~6dOF&7N&F1w|;jV0W3w7?OD znMqj*yt`sL+Zdc&RrBRk)nh_Sqe<_Hw{n56W&T&0D@HU20;|s#Nk;2~CihVfwoVX^ zd&$ygz34d4YhY^+qjPH{uIhLT!cxtbJut11DFcK-)U7-*+H3%+)C>UQ6N?C`2v zV|1`QZt8r4;ABuBZzJRlxwgj!XJWgJP^xf3Brl^3P4#rQG?b|8gVW-4|CDpeasm-V zss|{Ozqu0Mb=sCGj@yZAf3F&hM|D z1a?{MWXT6972(_GC}y2E!@DLor|SyBNram!D*9ccVAU>E2EUhf&{C zoBW7!P*Y}|89D1GoL0Pu`Q2ZEYPo5&%AZhqzE*UKy)f+T#Oj=Fc6x8T#2-nCiu3w} zD{<2w4leIZk>DYK$R0+IY;ZVp5ISqm4mk}b)!2Bh2Qdo&($40DUVXM!*eIb=2YLM7 z{NCKUL=wc>)bT(As>j&#SG`wf{gM zkxz_U@?Py6IQB4>-35Wpu}C5NzB7omb_;#!b7$e>Muo3A`Ff_SgoM)oUO68K6P$*O|d4Jo^!0;Kg1&;=(?=zed##l2*s#77-p;D-vn^wZYyX?3YZ?2XF(QaJMyRaUj(e=Yv{ zKtxT@k4Kz(YoK4pBEZULm;-@Uk3X(Q7_`SO5wa*w$k@?JaCQsMRQTeA6lbkdWdazwDIiD;naaQgLi zHgk3ui6bI?li)N%eP*7f5-AqRR>W@5d#l@Npdsgud%JYWk%f%G=&pdBF9;maY4-I; zd}{@?nC?2z7+WRk>JuSM?dRT84N|urBwT+qZ_O_+pGXF~QS@rKnW6aN)AJ4 z2A7%)SY$ekA0H89K21ma@!wTAF>Oh!u?6}{hs8XMm)&63wEx5Pv_;dbI+Fg1kl^Uw z4GuCWj81X7{q$=a_q#Jj0fxu*cg06?S;2efO(s#7MXZ8HNA2MgeNQ%3p$7LVB8JFQ3>BPe_J30`hlR3zI&0hB3z7pEPN_C{yA%NPxlO({ziDl3 zzPodb`~85|i4)rAHR<1d_X2yMDbjb1eJ_&4dQ`*`Hxiq`PY$Hjw*QRwz<4h|c1HND zVNB79OCN_vHXs5W&?5Zyj$#h2OWB=x-?`HyF(B6A6tZ{w2HBQjY3rTKT4r zK4o3tj^0e$XH97llDyIobupZ^F)j_iYtG2`KTZqlUHg14j#0|HPE~mCJerZbt*Q4? z9QB+)QD?P)T|pKOh;nFUAR(}klA)5*ZrHAD^V2(k&3xhO%4puZULG&R(4<|@cBpan z^D}at{|!N>{lny&cY0VRz0_ne_~doaS$;taIOaActq4Y-$0QloQKRm9YZWJQ@^Z=6 zOiBDsSoD?-n8K4pON0^sCGpZ{M9!trj7d?nwmj(H3pi|!IeMYG({--yh&l;bq8Z2L zW@2U|p0@<}W7r#OF(i1N#{O)z-|gu%wgDP5aT4X!U(T2SbLbSzZ#B4tr0M$p#!+DE zJLRTH@0&?ZF)N${1e@C-ba~I+!{7h-euI@Rvqe)LbV+R-xmezN`OLOx*PM zRlbVdob`*KCqEZh)+xRL*~xOMEVELu=V;tU>P2c~ynPDV{wu1z&O_{6OwS=`<8$j*3JMhLe>BIu1&RM9-+u1t~>ui)wWPF0EBC5d;~_vm$GGMG*!DWH)5}iR(fV zi4q0%v!ymGQ{nGI3)4BGb|@7X1Mxiy|FAw;+>miQch!)v46h)&c28U2_R&OaUQ0d1 z$dSl!hW3eZ$>juHdSl@7wQu5B-y8{GO?WO0$?v)?57ZWhTV?(y*&~3ftB97CO?_QF zHDhB3Ar71@Bm@lk30r1rt;l#ckvkL{1sxGXtR2+0=iaHH|MU_ZCtbrkovGqR%V^PXY`5aa?jFf-8(5C# zYmpgyM~mVy7-|2;6liw!U|Ea7YsY6da6gbRy8Fw+<$G7f>YAtz7J|kQ6rZMiBES6v zWU81z61uz`I^*f8TDvWdkc}evK*L{;DOIo$BLfIBGEt;2*@29iwMnZ-Jbt}zp} zI1yz!Pi1A$!kgV+QZ1e$S8PEh6nMd{35LzF{jt|mZeJU3dLUM4Jas~4x>xcE1)>}( zj8^Swc~Tz`q`S{f3|VBwKlzJA&}PS++d@Q4pJLIpaF!_Pp;x@Zng0t)Kj8iZw?$>`cMzf%vrN#hs$o$v=y${W0wFX&^j7|$^R z`$g@XgzP>aFQI29%p$4*X3WrkV`2Jh2ox*3%YtuX3T)wKw>5v@AxyCc1)Q|c2EQND z`Dtqio$q-QUL^2NMb5*rV3t@BM?xW7E31R(wwbgVE&i#^F4=Erzdzd)bGI(?hG zqnsGc#MDJf20nqR(%S2FZ_Nzgkdj`e5*yTKOe%7CBD_hiaz}JcUIj6RrAYJ>nS=&l z8L#1F9aXQ(=?q`DG5F18s3g6Rp7qrFjHuVcbg)IszHZ>g!fpw78z_5#b z5nmZc28R5eAmfjOh3_px?-^@7sc29%PAcZlii9x6ofukt-MMvm$(#_Z8$f6;^lPxH zY`#*|oDR|_9f!i>u#@sN3gioVsCnGDT*QQAOpOpKhN@D?+?A=tV%u2@GY#qer3INH zSxG<;ssm6LFB!blqy(fm{A$pIt4U8`Cnk0zuWh#64pC2$~X)$O^eGPa9 z;T#EiFgeMSxDPXr`bqG29Z!D)4<4*Yo&J8r6VIf4SF!hc_a83A%B0mViVjmhz!;CW z2)a4l4wE4I52c6g2&P!yps!LhK{A1?mF<;ouyL&{EWcdq7@>~T$Ba^picxLo3tZ(U z<#M@{SC+k-eKYKjKuu<3{A{(OM4j!24cKK&NFvOdk7Us9$(w9E-ErNnKrgh|z4k|T zR~N+1ZF`UR{c2pad6E9yb7{^5z98clq{e3oWmg{pE7hGXW+x~O*Y4E(Q30~n;O zK`MJdV2Oq91@*v1^vtA&L&~?*cV9iu$ENsUJP3Xy|UJ4P9Z*k%2EYIC1jM~M|g>OyZWJ%JPQ`$`{ z_{cz;6A8dSe%8w_z>p(&pKO+={@U({8v1PDi)lPipjy>kMF$WUWGKcP3MU7IyO*x*br<}dBjrL)5Rc|n6) z>a5MGr`z@on4v)NWxkryo%3YzyDMPpbQI2Z1;X{|0r}-9gqe`M_{~(v#+G)DwPU4T zQ1-q{runH<2ATLQc@#bRepXxR3T+UDy}}MyyRi=!V85!TU2-ugagURqgw4;y80WD) z@?Sz_y2b$!Ccr|^K*VF?=(CjfSJ8}4MqjGWi5Wrin-hmCu+O5hQomKhMv92+<#h<* zw_Siduk>FKh8g0$zI3*8VOagr@Y(;qgA><|&j&wQdMz9zp)6ppgfKL?G|!`VJ4Xlf z{;{SEPlE?RuIOfdr;lZvqBMhD#l~Y+mdD2ygOg91ar>tBPi-^2-PAa@xJ>lB%RYx{ zwTM~G!$^(Jj_LWC6LCr^p9bo$fnPsw$uU3AE`Pfw1%v{aJ`U!1b+5R@U$#3Rr(oWo zJq{8=9`F(qU5Z*alsP5oTe-T5rLk8nmYyz~b`t}9<6E4Ftb{pReVESE2nC4HqkJm4 z{qc!f33;9`vsm|YgHRX{F|}R;WouH|M)q5;u5PPM`Tz9lQf2cJe=Km*Bh7`qiLfxLuu@TcDhhLj@N37B!tSfdXKmb`>%}iG>R7D702Y&@ zBJbAnz{PigZ(Uu%5pPRMmWH(qSVvPyX)IaZ`omn<1>YHaKS)N2Exy{$q|OG zLw^=y=uO&ZQwzT99w=Mk*$JloOmZKl=3@-02@IAB>QV8k^gvyh#LwdPS@Ex?q=(C! z0R!tlY@qtvb?4zVx&&GvL|1A2!IbL7x?p5;OY+)LPY5(tZSm7Pp>4-V$5@(K|Iugg zPof1h=s16^M^FWZkVH*HB(IN2*Y(ZQqto6jQp=>b@^_c3O)7mn&!|SL>Fq+o6k? z)JdB%iF{s(hq9LQahLm4&uJ#BtOI&RI*A`Yu~+N2a4EQz)&p4}w-*fMjT7qJeX!6l zU+U3iRancH@TrPBt%rL&o+=b6FmZDyRU2!SMp3L58_ZP+s+s!1JD^Sa9G!Z6hx`Sc zMWrXt{4FCZI%WG{RcKJ4qBhrY&S9!OIc&UY%Q2m46th|ofi3x!>&UzOkmf`L;q;@@ zIDhD{EMAcU2XiudKSu>zH$L&jf2nA*;SC(7{3i_*_O^VMCYC9g34Sl)K2)Q^!5hB&;x{ zJByVF`h{A~z;FnmtA3&kI#l$&8kP#}cNI`poALQr)BH4zcDuT!4ib>GsDwwV;!OjV zC`FV2HzmbVzhH4X*6=`j=Dq22Z3*_H-{aL}3}=AJk=!b7c%f@!$3{1$Au%(_@vOZ@ z+Bmu$a44~kAwQPM=>EgAYnjCR<1#_svpLD$_5Bsf5eCgJsg;Gk9ZO%xXSyRhCP`^`z8P{KVJZ?UBTm5KHryOcI`vnI&Zj=X222rX zvvAg0kx3s3gxT^ymY2&$%N%(sPY680=Nfr>T#S;4E6zqt24Ik`V#3DT;dtQJ;hf8E zAIK1#WN~466aI(cMNA|@{Y~Q$U*Hp_)XYu!;M0?k+@i&sqAh<2)D`g#AA1ZXca!-R z37g+M*9ZGr3`ufPYo&^&ZusxIw_&qK77^f-MLlm0efxt)>Dh+)6Kv}FhJh_B?rt(Q zZzB?`pa!j9%5ASn9;u|T)XPazRp>MTRbIRQ$KF?l)v+w?CIoj0?hxGFf&~p8+}+*X z-6as*J-EBOy9IZ52)2;B*pj`^clJ5={=U!mhi918(^Fkl{Z@5#SIshA-zQNSth&`r z{>Qy<&Bkii&*N4ITX9?yLyu2ExiQlf!F4KSAp+H~Cz8p1V%KF^McYd&1xISLcd4cH z<>2gTW1px(;1G0ERl~_*f`{B$?rrHjNPYxV+yNk@_h&pP=L5p|=wipdhIX1yig#;X%Yy z+3IjRmT4EW?@6T#uXW6xGl@Nk!&m?}?yZm>KZc*ipO#pu?(THnQtN~^xDdNY{Lv|+ zEEo2xZWqLdpqx^IB(3dw*2WrFz~kWB zHHB86FnG!a3Hmgq2gLJP6AHe1KPP|Lb-K4u-Z(3BGrF!Ge4OHU85a8YAzjW>4GV2bqVT z&y1ioBcTWZ?W*>{AiKp|1~hT7@ui)E`cYN+s7PRgwC`Fo2i2tKH~sE1An$|GJCIcO z{1x87qzPW$<6ZH5XBqzLtCg%QCGI%?@@$_Cjz4b?fWbbmEa3N=$1CFF@x{@ViS|xI z02jxanMudHAM;mtN<804utM_2KHB!NS(vz>QFF80?twXG?v&X)YBs(?dBm-$x~~2C zr1!pGDnn0}(3Hs0&?p>_y(Qy^0*FwdTTu<3;TzMDMAo7gAB1P?mI_-D&Cb0Pgwhum zULMb9St0rC_V(24t|{vb|3WrA)gB3ga_X0hYVJJGXU2M0)0oc~Z=8?W`wILPwtisT zgHb%@bdQufrBW9)8JuExb-IKxP8K&|a zpDS2x4qTb*B-uD$^{IdFj>AFu)XqYsaK7An5^D1)=6-G_6tM^Q`MGBjL%3vG3tvAE zGVks}nrvYKPBbJFjfWfFrlhETvO6%|F;5i*OK|zS2!h0_Tiv1@M=K8Tm$lB$v|(_a zj2g2!{EI2EsX9A%z+e>#omY-YDX*OV>QPX~6)7wxH@U}ik9l1YB2>sxu!F4!FtcM2 z6^jHR6mc7ZyODl^ZrOt*k}+G$EM*bx$tB<;`&229Y>s;~FG6Fvi5mVy) zzWbzqIKJgFBERujqLL~Y)YTXHgktf;_7qksBo`WX^CtjWIjlxwp%~B8N!-?+oH}Oh zcnkqLx~csz5PU#QVbPi}{*Agbg8TVhNOlo-e;6x1yPwfeYS3z6bZ9WB2_IsLr0$1S z(P>TKYFlx<7<&l`Z~D0OTMQyPX(`zmBc>6gl{O!8;=|l2Ma4}oBQv@02fU4t0t|Dj zdZ}^YkFYJ{KGwl|MPoP8a0L*7gr^L6Z1kB#=NI1F!FIDlRKf;RIhYK=YEj=*47^oC zm=#6U-X{kOJv+5r!qiqFcj?P+#()Q?RI{z&jeC{zGB$aEXomT8bzdM+eaHplw!WW> z#GB;5qQVMNOe(HL$7=Kl=HaxZ--8aKwS$NN-~=)r1@Jh`Zewy?mVOsHj(#U zHcdb)bz|n5-uDr2(l$B-W$uYirD>`^Hl1fa+{iR8*n?uir&_`GS01-DYioD+D?P3s z%oZ}Ad4{?WdCfOURZFOcUMGn`5JFoRyY9}Tp!sQyzmF1k$U^E^jXsC3S<1Zn4>RKkHC?h3_tUNocz%BJ}RBKL@znflQn8tNE`>2)l zG9%;SUH18fVF&;~K{9>PH7d};?hAT4c>UfBjCy))NeCNeK*GVE1NR)6JAbe*!sun_ z`Bg|S)(#Mx!pZzNVBKuQ<#sjaQ++JHrSmS=vBf#gN~16M2{rNoap!bAqh;~O_QU-| zBb(9_C>1HS#O4=25?E7lyX-FC0t2$s0V{rYp)h|bqf}A*8cR(!Wl|~es!llMDp|3Y z$*BIl$dLiG?4>Mn4pxv1DpLLShYsJNrroCkam`0jWUXi{(+mn|Q@ zJD&1KR6<&q;iGfkUbdyk2O$zd$872A(PfK-mxXYp+ocuz1`M)#9~!jK0}cwUQ>rky z_?t65cn#+UL?i+R>JQJ9jbZu%Z3WIm61p}PFd*W-OAS7QXBIq6EL;IRXwq?WTQbrZ z-RD0qQ)lcH3dmp?+8Lco8p>i};nV4eDx>wMI5V%sBca~0smmnNxRBhWZX`fCEey!u zgoRygY|gqmQ+ z_Z5Dr3THx@3KkXUt2+7InByCe*Px*5-LB);o?{;xE!~s5iOl1|hr;_Hpb#s#keN>^ z=IFjc6@Z)PYX&zraNRuGlu4*pyd+rh#A8Uh(`u;KoK{oSDY4e9u+wmnhNsGQ)24z2 zjPDQw%9owj%e8PXl|ZK=w9J(VKPD{5g30w0hmk$nX{0mL&uHQ;;fTMphncUF=Ythu zs@fYo znT!P~`0Q3Q#_i=B@j8~b+1P#?RozZ^)(2X~1=;dpFN!9ASUe>I_R(R(Q)LX9rfc@A zXKK}B`e>0XIPQ^X@SNFkjJ$&jBs4sYM;I#fS^TcK4;->GG`qZo2*=i&G?oR+`i$Sv zZ6CB-mQlvR8Gu%8!vt5-H*;m*9;2oLmL`!?9p`cpPozr{#HeJhaoqi4%+D@WvMeA` zovyrS3UM5p0R{XW`1O7Bu5{B`=^N#1!l)w_m60wYj$t!%1+@p`q~YuzZyDo01(TGQ zxETD{sTMXQ>R~Gm`$kB^fQ66Im+s>9lph=6DO;S6#E7IQoc2(=MypPr2NP@@Rs!X; zJ618`Y+6E8{jFhRVGNhq!q_VQ4lpQ?ZC5?Tt?Dp;%%L>D{M_>u3iAR9){EK4iCP#= ztlMaEr*U}o4o#%ZfdRU#e?WGK^OQ)3OEbPnbFX|Q*6XZggV4uzoy|?YecTCMgQ1Bx zA|}|0Wd4L|zegqa>%i?MMctVMNx05F!LbFd>q)R@h^TLu0P#YZn*UNx8$4FcRqNgut&J-HWYvSmI*=R&23k8>-zR|b8PWK#CV$S#hr`k zhMdI}BUW3RyvR~`A)D>A1y^m{U=zwI9v_Qs!HP#8V z{(@Pv%UEKz>u5r==rx(EjvhL#fiy7v=}Bal5H5LXH$niX!ZvSJn)E1p-Vm>gSL&Rh zhGi^;KJDZi8zl?Gn1YGD>oG^ywhMWCmBtePW31@fAP?>-r(c#p0skUFi~?JIih;I; ziVSr8=XQ5T6*6ne3SFU2j-gD>DcxyuZ0 zcZqL5CqW^FOlCozib2Kd5IYg{?&;M5*E$_(k-vh#s@S6d^=#lFKa z&a!4pibS2INoGcjgOr%p@y6_NjHmeE8l&(vZgYf{QbI4~P*g`l=1t8EHI!H9I8crhQSB4OI}V4 zls$;4hEsSq ztZ_B|>|kM%CQDu2(XqVQ4bxB4RKg_aP^xS@9>+$x#8SRy;S;PwCAu-T?6Z?D-!-)6 zQ@hr=%uu+*8Lmp5^HnB9q}WnHE~T#WvH+Jv?G_i8x#DGZ>tW|cQZx!73JQj(0c!p( zbwueb>FRoDMI*u&%tft+YcHj->9L;$VqCnkDvkU*K3nuh0f_v5h`R&B$m1|Ff;L4~ z@{L&2VnPW{-tqHX=}Ltq={4tjM8P_Lej8_3_OTrh-n7o(*d|%MxcU(S9U+M8kT7`q zu6AUhQL&TV?%p~^dTyO316e{HEy^JiMWSPxJXQze*_%|IB{YUBF1fikIlx2Ij6oX~ zXmi;e)B4OGgp#^(V^!0{HN>YJzKg=AZ~vQj=nmSzP_h?=ln_m9&45^OjHeY|=ZuZx z0~7jYwY`rPS+ni-USq+PZP;>cXuFfb=1W$M^ZfTqz;k1axd0T`GcM=1@7;&;p-mpK zIa3fqz8g0!SIv>=XPhch6lCyHp7xUqoxTk3xIAL30Pc-c8OOp<=$bED*IjJEDK&=D z&$Y1-L2H*<|EW>e`sT^ zFc339qcJWJTRe!9a}iG%z@zs6O#4|cZmChX+Sso6!GfJw8I2{dS0o_6H}KnBZPf*3 zxk<~}$=Ob4FtW*!?E{lhCS4@OlJ%89B;AKCY)*+|G)8FO3s z)7Fe(Wh=-tXV&73LB2uo7{wt#Yu=wXPqBV)m*~3_<%E@W=r$mJs583;h{;>yT*yn= zl($N$Y1{uShwIX;?~SgsCGf~kpmB{el#O&TEv|W3Ypplvhq_9BS04|Rq-dll8oZIh z?HHZGUn*~M`<_|bP{;F24Su`hGF*IZ>Ic|i8CpF$tGfY>PfxU{tR--hDWK{ytkaZvL{+D z`4zhWuF*)7c5{(GF`A&ar}KNBLqPx!XNPGvWOo1RimyTH4HjC-XOB|EFHk~^-U{e; zd-70~3ZhSs^iF;H!trwnfcx&j!4_!DnIeVpRpt^cXwIwd&MP|4x|GpPS|yDTd7EB; z!gKMyiwL98;?*M4Y(_oS)~e&GgYM}`=(3onSob`)_w4gdTD2r|N%#opz^zV5H)%o9 ziILRY=kL21Ar-jLcWN2N`s+04ANaTVZZC4pu#S6_#Q~&3x~)x~Vyr}Xep-@pZaM~X z4mnkM8rZmK7`SNeYYKXWQ1|+uxuukI2f_?aQ_(&P^;DNxNuXeOC=ges>sG==57Ul+ ze%#*!QQ;$?V!`~W0%d!M0l`=^wtvCdH4r|aC-l?2uQNzuO)(?Q#04D!&Pj9cMD3Qm z-UkXOK!oWX~6Q{A5=NL*c@DL1o+>x#E!l2Vv?9wf;K?b1JcA zX&)iAY?@}nA=VWAg#n*GzbV9ZxmUZzTYWBvK+zjmrK&r(jxp_;&7bQ+4ouzS{lQ@= zW1F!+ufL~H!s~jH>s2;>r3C8tP7HT-l=?hsu+XB^h*=P?6)WOHuYdc^QtwE$(eNZP z!H=>@1r~B*U!9%HW}BOZ+qu(YNZ15$U84znG|057i!@&9B!*=B5A9VZlQ6}-Q(w!9 zlHYuj$4kTsJ5B7o-&)eL$9RvN%45KY$!x*z=?V^CgyDeo{nLB4uMwn7Q8AK&auUw# zRJS93rSJ!2FIJ<6X-4||eD8E0TJJ9U9u^Np62|=H``~-LhzN+i?Z{`#ONzGl@qqmE-nh7fX{0I{jf`VQ)HuEu$BcBB*$3yk?D zm9_v+W1=i?+Lq^sy$lvkKy?`B0C*`U!^}$#?;1okNpQx!^U`h$@?19DSD^1w~zgcI|N)sTD2EyY*Uz-HWsiD>#>Es#3lr7`&^z2*T#_^S*AevhOl_*XZzqi1xd$fVl_tg@?vBF&MPWaEvdvBFW4^fdS!B~x_s z-0NfzQ3oz)d?0NV#za46`@j@a+6s&`pSgQ&*K!qf3g%Ajmw7(&8jP9(!#w%5vIUJV z5#zjf-J74gAF}J6<1*(zt+^Ga6QgbVDMf`s5|*nkw;NMxg7UZ1Pq#0+^yO>KO`<9i zMgdJY1MKv3^*8!JpV<)xzkQbo=e{fxYrizuKKcOlP$EQa?Rq@A+L*UE$F?Q58{BU$ z|M}qmpA{W@+Y#78$ zo>~$u?Du7*%KX{6N0XK9Y=1tC*yLmfm;-i#DsA&ufvw;3+g;y*d=iXXB_L0T0YXDV zCM2+Ihl*XrJ9 zbg)5qbEj0rj@T9WRO-WJsS{))QHizcH_6Dv6RBTv{5AYnx$F;t=Pq9En@N_Q0bCEy zTmx83%w7Oj-krL8QKZrZGQvZ@0xOn6wb}@U#e6XfyU`h)B;f9DPM(27X&fGQ8#Bbx zk*fg{d{<`@)k3HKy2QZec~%9pl#bk?LKCl-12s%`tXi?PRl&Q8H0!K0w6Qpb#$p!N zhPQ-18Z7{HEwp_jf{$RZlbh?XS8C_cK25E%@GFgKOOEM>@LqtXO~+$#ld+IZRg6dS zJTC~~XU@-}_TluQLCN)Bp0*_UpM~F^h=pUxG+8`@iiH!>mRaVTIM!SBs(%OvQ73{= zEzRWsW`#Uo1?9OKpmermfrQL^Hba)tgrg#_*Imeb(Wr(tft|*;Oy@Q*CUt+$47XDV zw&Hx>W>)wPe=`b~?t4zD9Q?UPCeA+Y)RbJ$IV$!<)CxTcjcyxX0KJo^*EBW85@g-^Gmo(vCS~@p>k9a*55&}Os*H7?oSfnLJoJNoWS2svMpcXJLvfsl{X4aqjtfaaXkoYkvvLF#Onx;b&){G;P7NsM4F~)C3qp^mKu?NMkYczsD{JQi870xR^+l6kueTA3TI)jvAOJnnWr3yjF5Lv zJFhLS1QmHeqs><5rw_|o4rW;2x{9P*{4-jw#Qcf^O2$X^eZY*orLj?YxuW&xN&y^k zSx3!)<>dl%CA06cm^K(XvalLI#Ld*A(B>AdgQ-E|!!DihrAZOYt?22pXg)@Qsm17@ zB|)o-sRDjIr!p0sqn%`*O@ zu21Py%X=arPQk`uxT*C%vp|**CG2V`N;3*G$cT9}tt;TcN!;j@s#sJrmu-`D_rfJ8 zu+rV{6Y-DDp-d%$4(@K(UV0nCKNh?l7N8Y-eytTSHt1p6{V$?AEOlRXE;~tA%n)ds zEO+-W7yEh=gLR%{1^foMj<1vHnXgJk(7funUv84Rty>`K3M*DVT77c#RH5 zy_U>E1F;9$i1Kf%u%I?;ufwNXvA=sn)f-TISrC0%>JCHa1%!A7UsbzMgItS8469w# z>|isjbeeXx5oOI`r9Yk<-C@eWz5t$Lgkr%-`lBJEV!}FhPQCUx5}PR zUbp%-<9qJ2nX8M4lIJF_3$r;bPvTN}3L`XT{P4M5w$)W^3W?ZcAd;wt^P>uHyv*U_@sfiP5W9^v5W20tZ28FbE_9BctU+{J7zD(*%t&aR ze0*9e79oC@^7D+>?kMSN@>T$<4*y%dY9>9RPsZN_z-ITTQheX3Bnm=o{hTV&I7)9$ zmr4SZw;m5s2t=XF6TDe0gDV;TtBV>u_j3m%ws!$oGf5VFZ_9xz|j|9ij`R>Mk^}~&`b$HGk*NQ6`8Ox!?(>p z+U9znjuP;VWH!RsOk4HSS{!?mL}vx?P)hP;{}7BWb$ffW$=v8w-^S5eJBRX8c4*WG zP7ZQzoBrk~u7LdwGpF=2+7e5KFmJErL@s>uj}KTJiEtJVD~wjl+Z!GTsgzLAanhqF z((sQcw5gAv_{7$BUbEba>S{~NQuBKhjZZg^X9#5DN|dvU0~6X91(PqYd zmb4QI+Kui6H{)?t?tdKpxAS%vkoJo&hZj?}Z+``|Ccr@2xDou)ruge(5r0TGJX)W| ztbE5_iu~5a6^gg+e==4nFm+Lxz(UCBezP#)#ESQ#KOC}TgpdQOLQ#3QfVhXM*#07b zx0p8>Ed$M!(y-FM1~AcI^Y1a%&F{*;iLD=Vc^YobL?1Sn1f&E&FnDCF7hHyDLvbRFkBDR6=moHE~<_S;IllnXBM;A z9y;y!-m>%J6zuXbZ-~&dWt4WQzPFWVcXPUWXs6Bs0N6u zN%ro;OT+xj7kyLriD$y=446(gH9?0)dABodnS>GUZ?rkYyspOKv|bJtfzg-3vgqMH z1EWo1G`E+=p%*nT$Su@`L3mTQQ=~-}7`|J3t`8VH_47C#m5%oA8&19&N(@HgSX+ka z#1-NOBtl<`Y5Z|gV}hds!XIHo%7R|1Ln0|4{fQWVEHq3%Z@$l{pF8$4pJZQTrp~+z zz6X90U{BG!ptdDiF1*Oeq>_GYif0D#0+ysghi=uKkSEh&uA$b(iGuKQslKW7FWT5Tq_ethX-WbeaJ@lGmz z*ZWJN&virZWpO;GX{PdaRD_xAU|BR2o^r;ntF&CH!R$ zm-)Sq*C7kK?w&18@Rm;D)f!KG%cT?Mp*Ot&4TD$HkwogFJpT(UCn{%&iy0@&}*X?MHr9|F&79JxgJ zgzH(m#ay>y0-h7}F{-`(UTY0cuuuS|^j-^gc}|anDgDkr-5)~Ckwb7dM^fbMWe*&9 z5^ca3q^mm?s@iIAka4aI-L|R;UhS-EMwl{4t99$xkcJ5h83o?II^o7I!sP`(h#*qV zFkuX0_cBX&qrJ%J!vLTsx_YUBj)g%ph##lawOzTIbCRK7geI?o!j?BbFUGSV+ zX?syz$WAJOrJ^=qvcuhS_gM60v`bXt8=OVW%F;_6eBfKRGzbHW>cdj2o7TGnDbkJM zVn^S0=L7FG40~|AGI&DYB%S^c#_Es0!{31+oo@=w+6}(+;Zcp~q=T!|sLFc1X61n5 z(>gtpM09JPUcrjS34ZJJmxA%mh@WYM#~P8ZIhU^ejl-(SR|vaGm1q}y6z|6_st#hh z$?^ftjF!AO)cRPq`7J&>dXA~(jQxqdm(3jIh2n!I8dx52gJ;w}!WVC)aps+_ip_wV z=_ev+R28GK^#TPC>%(X0ycb?ZLx)KV#vr^(=Zr%^YbSZ5KVCw$pKPnqS+B=jJFkA3 zgg<>gZj`s}zz=(t8Hx4fZ9+Eit*2!`tH)guMe@-JcGv?q_PABd&AntUjD1&PGSP`F z2|G_kCx(rC@Q53u3rkDEMsL@=&aTpZ8#}cJ8h6UqOM*oaH8ohRGBqEsy=M|th?&Gq z)c_EZE_vz$_)PQ$%A_DJ_b9iJyzP1cvs_Z~ z)l*?~?MG-<$vn(7JUE`bSfv{sjYGJQha}fsqX6%?LKR{_{0!@@c%-GVLh0QB=n2Io zx-=fF*}!DBs&d8hcAtzYyNislyI|`&vv=z*@GOm@&h(mZMZq%8)KPMAmIj&PN5a^_ zTxY8_+n?ks)d}vMx%?_2G3Rgml8Rc|9J-%D+_27UGwXo~BApq+F7F;M0-1u5Jmw{^jOo&z zZ*yzwLIBI_Ze17Z0X0t)xJP^q0`p-(BelYIkK$L3dqdD0w!p?9l!8MYVD5l20yZM8 z)GZP)N@%G=&MFEjNZL@panOB2FrOmSay{?uU6u#VdANwt=P*8xeb)Nft|3r;D97vI zI_=119OpLVG?-C$6A zOzMF^z30@$5HK~Yqx}6_OIW}+c-^{|k0pvMTx9bmY8=F6uMl2xg2=DgbV(bj= z(umi&K-JA~X?B{ND8{+U2oo;%wKzTYTvwU;v3Tz6jd#PxUA6;NytIB2yZ4?`8H-uI z)Z;b#sCFU#yw1VDatEZ^ir}-u{jxxRF39!>IwEwqji?(w{E@G=4g-oP)E!jLeLIrS z(I}6O|JKPq_Gq;iKOudZXg)ff2~GVgOwgP8E0`b)Humx*H%nzj5Z|!D+-C&ZB+q$Y z3Ww35%r`FT`sCP^D`dk`2{ZQ#6zMj5P`@PbM;UZJE543)HpoGPU)wO=&5-NsjM0}| zA|3UR4LM$I7BO~58j#j-2O3+wc|YObmhG)>IA)o7)a~58wDFqz8?{(>!p6rVlGOQI zaH~G1)X(Qk)PHUG4+YoP)Bkm#??C;PiY0;NtIlaaP7Cv`)V7yh(LvM&T<4u_Oj-rJ zj~bmCH2hVqN>>4^w?Uc>9ByY3zks27Drd2eU7Bs` ze+Kx0p6YxsEI(-lJf3j;$8iBYkuhcK&wq%t106DP+#;JS=mN+`5g;E^)Ng4GK>yO) z`#h$()9aakEO?6N2b>*06SLNR|B#Y59pd?k3q~roh&R!Cq@``i|8rOAzjc*bx&HOf zt`HC#5DRpz(}A_}U$LZ249SaIh_um!#$*jmX>@G=Hf7Sx{ zLvgQhK<|?Nrx@=VC2tB`&r?r{*CS1BSB`kbS`*&QN3U`@s@npmw(BX6Vgg9sf0EHD z=A2!huXxuVYn?W}jx;%1;7sWN)`y>zlyt-iWPeG2{SZO=M^1HCEy9kU%SxipoOl1MKzgrRBzR2k)e)r;MI)gv z{ZprHL0vgPmfMcHk%;zKKTG^MMGF6Miae>BHU6g=*H`h17sv>R1HmvmG05BU7eI#+2R zs=B;S{QFGI=|nX~vy+7HgL0dR^c@owHhu{@fFeFU&Qz0LZWPBV zosDy78Mi42vj!IZsPl~N1V1(&?_K>7yqaA8^J+v(b8a!(mryc&hh_`?mQK!;a}g8at)Y`QNaJ7ArpT^_F>ja_vrDiobK)E}Xuw0eA zav&N#*c;Ri0vsGt1IMESK4I1V5C_PK z(7UjWKqM$CUdC|Ag_}cb%Qb~VOTuPLrY#sF+}^Cg967+;^TMFZ%E_|TMx^b3ycUm9oZ^wv-3MAL6q>s1G}WBL4PH{qu3nQw1zujcTKQbbEZYVZ|9_R-5P(3!+Z z4VI_|r#hJ|2yxXL(P*F+#!v!KAZHfw1m%A9D3du03L99f8j*L%DX9fUs5LFzc1k(d{p6|;%wpWm*rmc>N7Qexc;w?%C%8 zwJWkj5hm{s1pV7@e`_y;NRMI2_1B#Er>lKspxAKLYSASAPR1V~@|P$&v=c4<`tP5v zGWEb<10Bp~Rq20e-#_-~r&ITj_kTU*6Nw7c*-8bo&VQrgU)~EF_yhdEx8PLl&{pnO zECu_UPW?+4zdsIS681l_=C6DI%cS@J%cTE*HEEw#tIFYM?weMJ^lxUO&$pwagYP{P z=@b>pkM`$I-rXL-mYbhHdj+km=uq^0h8h1dbicukKcBSpxIYvMtNB6)m6wKw1_M(0 zR)3T^?N%kq6+gLRe`yBME~;E4rP=?7PC2Js;V7@SIohX1BryimYcQE`J0pS z7sC8;gz~WC4lhD~16Y-Je*D{}VHhtzq1TTU`-RylTa)tyejxb?LUZDbLrin~XI9JQ2G6GlIBZsW273C5{cTAgGzqI{Ki*xSkV&O}bh(V9 zkW0kaS+;-J1w1^k$Cs(qM{ZG00g=t=j``EL&Xs2tjH5jV083xO=LDf&Jk{EY9f_5r;VXmq#cb1aezV0)d@z<;SC8! zdfaumIsV~Lup3GOw!RZA23rEOnHx?x;`e$InplX!G`S{4e)Sjqp(Auj`Aq)-BYlvN zF{nD}`+^ana?0a%h(Nn1^j-ZWB!MP5s~cmmhaYG!1RhBufu375_mLQ`T zViZH7g!jL}JuT&VJr3P^a(+693ebnbr&YZhxJBo&9sD-mx^=z9<2*#DAk2VHqk6sW zhY$%yH2k{-;*$hXC+IpOf4(bAWwTm^=Q@^|KBWCE2Jt7XcfWsSs%y8-W~3$(RG8%t zO2E&+gQG*BX1e6NWY6c3d`c!Pggm7dSJTvMn+^mFc5*%_gNai z=FcrHiGQC?tUxq`zbHVPBp@?Ndnx|(!Mw{K*Ky5iu7qO?qe=J_x?Hf!y;L#NZA$Rk za5TNzeDJFf!YUj^g_hgd@Q}L?t$M?RsB+jbq@lno*bU(B*jgP)&7qIq0OXhad)@~QUzngYw2R!8koj4BZTovAK-U!UE`hvlP?s16#xn|7e!#8sB(u-OcEs;zHahaX%O zOV55J7thxxR(tMz4J&Uv+7@u4RRM&d(^(qI*{cEZu>%syIykI7j>l@c1_c^c&U?29 zogt*Bw4hOoNGlV(JMjA!uyp1Xt$iSCswf6{G#SuPM z1>=mRH~*BQnzK4SF=AhM<8tGG0fG(D$J^+Bl3H5lVZZYg0?&oH>D(61uU>sa^W!wf zYd=d+`t4h`Un=XU!Z0l%BCX#OALu5X|MZ6I0Yuij*| zOMuZZY$_HD4xl;lfkz8|&Lhtphr)*Pk*xEZz+H}}P6mMvxAA!oB~IL}D^)?y-RD8= z`4hxlCqF?h0z4p-Fc_EWSDN7b~KF0Jv=#?c37<4Y`TA>Xr|7O{Y&49zT(C{$Sd>t+U9sR^O zH*){|NJSs?_BW4mAjj>q`JlZSkls1z!W%>^2$P{LGDc+b3AR)bRyseBme&;`*|VAa z&1n~da<&VUr5yWh?O?3t)1^@YJq~+S8v|mogfr3rWSvY3pFjDO85W768YOLMN9G&2 z5MI?UrlJgFhM~EC85D1j(f8K+%1pH@@vkb_5e#$A4*!toZc)+^td#Ow&`MQ+pqEmY zh_Tn+Y*TN>35nFV5 z)#d^y!5*U4T~f}{n)m_=5Q=LdJ29Ff3N=S!;IF@EI>!NWYkcDz=(Yf#X@)r;0dC2> z4$cSUWR-HH{dI5|-fzPnmer|1t70lPO}_oX&U7YfKE{FNk0DjRkD3Oru%56C{0y-s9mK*J-(ug~_DM6cNK+b)f-xsH=@tA~O)$ z`pawM6EIm=YBA(@u8VE*Xqg8fHL364nLYHaWWJL3C($28GHvNg%FEC4Q!Eb;4qV1u z^Hz`BU!LilJ7GvAljRQu_FmP7hO)wPr3u@z!DnPNkzp&GX^en!E*le9vre&8wHD~G zO}(z@fev6`g0gt|%7T$Wk}714{kIg3-(|0=K(MJ3(+pXKjAu4Gl1%qEjo?%PQV^bT zyt1+~3)+3!bW{d`=e$0`G_HRy#p-7HW3EI+0NoB4UBTT?v1;QW7DseDB^qzj^Ii!> zrP_BfLL0WK>_wY}Ezm0k2L4m$8}0Uh{?yMO+l+Gh1XrgVvA~#%LDhAtPzE5-y$tAp z8k1KaaAt7W872|D4XdOtwsvH**%S)i3or}*YtY&nd{yKMP}aq zbVFPj!f3*Cv!Hi0U#`{W&e31V7{;$Z8V zS2Ro>O)7nFJfpoI&+UDN5DG=hN!^mI5(qLZ*X00l2vgqsP_lSx736M3&FgIRZ^vDp z=baEQcm2F*-Jiitr)M=Caqi-n#XAqp2Tagq@eTw zy{(Eg$3+S*rhIJKsZD(~yRp6qY&9l5xIqPIgo2R34{8QPQ9KXi2_T4Nc!r5B$)iA& zd24CDykC&706ZQ2hsw%)A#h?;uyH~E8;J7x1VYX@ve5@}BQ1$~UBoB$n`vGpUuWX_ z5`hk4eQi{f^8W&_q~+m2i!2-%0=wb~IDu^+0BeQxTX8)qZA_w;X5TImrBzK{tp zK%I+_4#Gr0HT{^C4V1gz0IQt$i@R?;6OB;Z0Et7~ZGr zX`uC!qEP=UQ8$x@8JGh+0-(A<79TWCfb`2Mm&5hDE=E7l+hw%puC zW)Ch_Fj>&De2;3WV##q>sv2|MPjK!hXJ;oG)wx%_w=S`jm*%8I@*B0tp zwp1S6W^VJPSclVnyB0li#np`Je{%p|f#i4V=rf64sH2aOG%CzrjEduNEk61JQMirpH+?b^$`n90XYGm7$p; zbNfhPF_y{!>)Nmt$rWr(AKQ$@sA_h*;uHl-kGA&=IEykN3LbY`y&^}_JAh09JE)!4 z94>60QNa0O00Hr{5)aAJZs;-j_uFZ7Up<9MZnvJK8nc&p2Rr?d9 z9i#fLwATYlo;LiFD^SZER~+UCpl0g~1u3%ZB2qEgkL& zn+>)^Qwk=LSudq5BzDTNKjP?2rP4XmYs)`GuG!EkWw*d#$xI4R2VwWOrP80!474E@ zofV9lg6f;o_8AO;Xd1SIBqO=wY3I_GEA7ue%VBwSNppsrTTs#}+V6Vl5Y|Ct{p?5f-83#Ok~I$<+%az`qiKEk}w z1cZrKgLhMrza@Y(8t_s$vFu?!b~FDkPM`W0;D)AlG6{Gt|QK7pz3m*MUk` zg{*TCF*;0!BLo<%BULC^8+6DVhlM$X$%&qW;+mfZwqr!_HfVmNuvy{d5g8c?(QDkZR5e+;a$n;)BT+O z`uoS&3^uiDRn@e6US*ta<^6J`Q_HYKQs^h0uk>19SRRia8`QMDQDZdvLv*A_=W-^E zP2IvOX_c5+%qUM;2co^WRi*v+S7F0pn3JTD$3r#{c!TzQ6OL@q(IB_qQ5!Xb=0Bupn~^mjCp3LwyqyNm|EL{bnlndmrW|g zLJuxty^ACzwU)MK_K*9Gj=X3iKa~7Yr>w@oF3cJ%qQwGH1j4>3J8Bnc$i#U&N~^^C z;4k7!O~dJoU+yOBNo&w_4RiXK=`s2I`U1tc6K11y=q6V0#RuX< zNh7pEKe^|Ndc4k6diM{Ff)dOaNIkBn#P*$8k|GKdH45(ErE7d#pQcL5@lsqP!!n&o z1%G*|V)@NVtT#wlcU820@l$Q4XivK3b3`_?L2WW&+tU=(iT9zCi}wMwifl^#OF)Hn zgZFF)H-6nyLj>&V;zH-jypQ)ejA$Y&jY=ey*3ID#EZ?-vgLm$@$|~$eGd$l!?KP@+&?&{tzfrLIGpX&GvsHvr zdQ;e>eqBBy@iw;Q#oYLQ38ZHGqsj*guW%3*`*lvp)VAn_?nevRFU59H))#(iUy3ALo;sz zw0yMQW!2=@4o%pdY>RU@NbGPuS+x-P0j9o|Flt_>%A&LpvJ?-Eqj|MV4vLdc2u237 z7vh<#-|YW*DCPAMj}cno+##pcgM`AuLtTKkVvHltQe~OKEMPiZ$Bs+of#U;VXF+R* z*xPh^PMx1`iIgY z@kLqglRi_amV2{lmQRWR&LN3(NVOuYZPs2r=zUoLf2t5mf;3aM@DSlucs%iyS^sTj z&b2Gs?|Pr!-wU0F8`14IyM66(5^^?gznyrMK0C>!w;fs}jAuTAz)fVbohLr1L}s&v zH(Ib;(g@=XMz1WcEUbumR>U@&dz+UWkEc?+%2GCK{eXr@=LcHvir|bS>Db0nS;#>f zt1nIZ$_6QT=Ci-Q{+?^+{PBvVY5TsEc`?&&z^Q0viQ0a4R|U2+h8mN(e*L<)5=7Oj zmUFUvWedyw`{>&Yd=tkW-o`y-XTc=b5luxr-VF~s4|oQOrsI^zF`Oo;1(uU%=Z)HY9|XCtNvh!Y?U(fQsxg^Kin~@+X^8dOb{~&GuawqF z*rUu{;jgFSF~K$$STL6(<(Hq5kyJ&fw!4o5=4(-!7{8O&kD0xRsGav`DNn3xh5`6nHdeLiM zY~9{7EP%Jg@TSq*>+G(g)n-*U=jfcb(QTm=F@xX6dKPB8cY^QrBD~`1sOkOh;&Yj9 zBL7}k9cYo}-)AS=bj{l-IS%+9ExvH+4<^jEywjE zwR9WLO88322&C2y`ntF~3a>(?%{MMF*sktlkdK~$t>w#RHJKOAHdcOR@z#FkMem`R zL2y1o?}KQxpy?n3k7iQ0YXUgh(VjabnHM4%yPyyh-z`vl__4n1-{jAK2el-$@T6oIwQjse(af`pRr0_oNNs;E7+E65<|N463;#xzO#4Cde7GZ$l^y?C6mctORA0 zd0WBn9HcxX-A(cD;KIZ{T_$vk2TWK*`Kqz}+`He7ew%RV=#{>8so)<9s;QAm%9a&N zDF)Rs6oi9KDX%dk634ayZ~afyAvB9ZowhV*j|5Hhf%|1qS_g< zn}M>R4=TEjA2@7{Y!qdTnbctW~8e3LunfD3V6r|~j#v$|32q=`VpC)|K~+<49F6Hi_^e6|Y7v@!b|FnnBX!tquw-&J@l9 z#`=7-$ie|&3JcmEwEI$qFr{91YwVW;4?fZn!W#%5Gl)bVGjQy=WX|60yBY?0KX5E< zry9bSkKTUnyX>!As=YpTv%H8i+KYk4IviYERE;2f&eacY7RIf)(M-eGm~WX(x=Me|@FMEfkrbkM%jpu+o#tbgRx$tbUrs=6K!dH4{F^C z!gK9YaWD-mCS2$y5I>2jnn!M z#@`Oa_?K}A@>tliRQleBMQuiV9UC;i1JBye_TlTib_U9EcWwK;3tBD&E><$%`H7yl zPf|^?9)QoHk9_HxcTm?~UpC%&!3SsbrK=Kko%;u zmQDl%T<2BhS;UtUe6!}z)vqdAi85j6>(6`aDsG8#&i!m@d(=EpKlseRwsL zG!^kfZ)TN7Gg$M4Dy zX{8q9i|}us*L@#FLVYJH;-~Z^trU&fS0Gb5K{vo+LJV)cJ%}{J1NPAQdek`$hA;8D zMfB?1m{C;)FOrvemGpBzjB(iwo%zX>kq+Z=Sk60+n$HE zNL~j6cs9a(yD>1z%^#ro&yfY_pHR37@?uIKFfz49|5>E~Wcz=tm|>ILzKO}qKeURepH{BDTeG9obUY)+WC{5sN~fd4uy@3-TDZV?~+xg;ea}Z zmxLp1-yx34(5l%95ZtmR3U;EJcwIA2RSp(8Ht_@5W}28$nB%gzbhSYZ+;?)~VG}gp zOr+s`C^KW%-Isq2$>gJL#uks~)~qc?iiS8ATNOM+u9-?UFn?~^*W>B0N(0$?xm+)x zG3Uv}obtQ3{&>ogx5KAclrX5wb)G(rm&!2Gm;IRrm?#;re!2ahXQ66JEv>ifW~1WH zTQj8_wfd`MBhA+|kQUOn!CTC`h;X7{f{QQ@HtoGP|6)+XdEV>Dkusb$Ov9$(k$e zCB%&Ia4^OQOp7sFN|e|>UL!&n8s?YOXlC|d8vW^riQ9n5`S=P>H$#|G;Ya*f{Z%Ud zHxAhTUPK4|w%wX{+M`67_YbieDB64Hd9hPF8!x9WGevf~!mk@|FB_b?g%6Bhl0uTt zNrHbFpXNom!Lgpibdjpa_VuGR|FD}w>Et;akxS-6=;TF%x6q9|hhMNlx9kRANn4mp47v@{xv7HKWJlsM5;zS>C~il=vv!wd@PzycOi2JWwx~2GODKtT*c|j6st$daSoB5gOVLsm(#^w z2n9N>s<5Z|lGBH)edY{50jvdK=G9NmXnREj0R#-Ue2c-$2uk!{(_2_sSA+vY2oMqd zaVMxb1-1|Ra-N~j;d!1TDhlyhsG{03F`FO@i%|hL-n4P|IPlo+gl@4TTmBSRozr6qT^jL=R!f1^C+_EmC{R%T|sKzj;Ai;5INuLDWkXT;)( zP#})2ayR(PFo?r;>y0FuM-UbQzDYf>J&|G=Lj^!WFvUHI(7U6AR>9R89_MNc*?=#VCb1}nGeL$#_rT!9I?=OQm1?Tq}+@A3_Yaa#bXodt3=i~cgpO~ zOP8KQBY@H;5)>px8am-_Je}y%{Q*l@yhP!oS%I}@B92(kQ)=3>$>aTdq+7MuA)ZF4 zeK^K7t`yG0=&ms!4=o{k?<7+d{4ngR^YgfzQ5zYG53~o>R~~Ldm7-{o2jfjB%n$*M ze3GqWfle#ZqlowMha5B&w%T{SOrNJlpV6LGr_^@;%S-NTQ<>+*?LJz>&||f};t=ot z6e&ytG`n=Q+e$6xr2=f&-&jC4zV9ceE0ZxbVuP|I&ve9Zsn{xx#Y^xY_kBq}YzryZ zC&2OJ5QCghTEs+Ri$dr4Q9}wf=V+-ngQexDF!HW3s3|Z@G z3&{sNeG@r$pyjL2xg4G(+D=R_uJ2dHC~Va>;;RAnj@TjKreNSKd5f%kryujjxQ%KG z#KbuX{!dW|Q18m;H0UnMf7px&T=D6b05hR(~@?{|Ry}kY;@trMkXgY-93} zagij~%Exc_#`l>P>qjG1gaH$u!=Oc z3eLxFKe?jG%eHa{M!)^c^TGR!Gqt;8TQ%!V!i{0rMCV)O#(e>`b?g1YUH-LF1*}6@ zmfN@S3=<`rQ)M25&y6x%5-J8Djpe5^J>>S1L~_033$$oZQs{5oHG_( z4pt*R>6G2iH+HacMLpaI$}z5WT-?h=g$dVedmA}IT_qvP>+>6d9GX3&9yP4}G1!<2m?cMBO@A_~E&rtY$ZHEu=Ocl6k`M)@ZR~rS zhwatik4Dn9Z_^xaH#%>uxTL9{>|Qz0wp5U9IHi6_2){Q-Jf@ZjY*Oda(36Y4$GK4G z*^0ILomS&>q_F`qoh+j1Wo$0=_i+xdD|MLQB>!N<3E{QJoE=&TDM^rbs+whH{qk5M z=M_eK$z2{F^udJnKjcR=&xg?hHvIa9QHM zyV%br%Ttkr``OJhMI+YLTi=3e0vG9uRER#79OOGHa_ zbCP)l3m;qgA=Q6gG_BTuA}g=wQulFtO#A^J(w_K*oO1wuaRMWN1YG4B4Q3lrDuA+NWqwE2+KMd7YK&42%JVsVQF`1NlDC*j@)2}mLSwW(zB^@ z6j<_A)s?LlU|Yo+XFhHEY5v_c-oN!{s_Mq9Pz~1Ju9kbrov}_8Tg^{`I$y3E*=V+b zySpKcybeKPx2xt4R*Ig02zP`lI2zA*7sF=oU^z6>3)=sHT1_Ql{hArEXPqFgI3sDKrUzIE% z)!p3u>M2n%A4oBKR585+FOy=q%*v2#%o-8QCVR_buM-pWrzKKiRMRG8np0`2ur ze&5$ls65eW6P}?xa+h-{+(AJywgQHLs$|u(<_kG==9i2>doC4EFZQcExaj!agEoU} z33!2FG7K9vjz{{GoI(ENkIXavP>NPe>STatwz&y=;pnc}78+#J?uzC#R#Gxl@@VSa zTQa_9!?K$Jr@r*0h$Ip+rTn8cAfjK)>_IQ7E(#Z?|LY=Do_WxS&||`yZ?Ad-S`wcc zB3!k8!$J^eS%3ZX^*U&bPwEcV!^`>}>xI6Wx;mNh%f9k5=2uK4BT`Ct@NM06z5+jv z2-|Wm%e08EJY55f zc-hYc-ZB@-@fyF%^0)8iE_mHe-;82yJe?PkcT_P<)$E_OU%}9?M!(maER(8O{8hWs za(gO*#MjU!%{Q8?Uo4lsq-pUYMMSZ1BbH}W}{Be>R&NV(*zEX@rbJ zL>S66{^>oXAcAqYK)-?*3Q4XIj|aZBeM5Q`CoU{j#%E9c2&oF;Csd z>6?4;DDWhf1Bg8n6HD?CF^k+F#2<{e;W~RZ-dE{v`Is<;9BS|mCbar7%{& zv7eIUUg2e)&V%`-9g}isxM@~Esv|mdFp%kMQLR!74+sK%%Wlr}Nym2OCz!*N0}F+z{0#&P(P zrwMy~CeZyr-p$PAVxQb+kBBqsuVjIunw6ux%7b9yzT%Kx7!U1f9Or9}YJyYoLfSGw z<+$oVn4%SqHi6ADAl;&+8YEay`mOm2X8X+^r})sST%(fmA{Vp5DG#A!Tw#gAgW{!M znU43faIC`%dfkr$hg^O?{w5&6$z9ZN=F1dzq%X`p+sv%HKLu8tY1F{wO${`nJkMcn?QS58j-}&8mL2;OJ9wAIP2?$mbN!x z%d$u3W+IyE6@StrcKOZ=$U``Skh7^BzRR=^|52@QXxiTY@QD5P1*F+0Qz7o0tmV8_ zzxIawqe=kb>ZU_1+4}g%#1Q)<_e7@X52NXPi8kAflOnl8?b_VCz2F6Ob4tAL6lL7j zkFz~iNDt|qPq)uRGyOn|Wn?y=)pV0LA)@2pI=9u;gG+YZMg>q~BV%MDtWm7H*Bvp_ z5az9Dwb09hsM)1mwv9Iy7TRTP~D^LeLvS%K0IQW zl4qAH$Gu6itoKMbj2Ubbd@rk~^D$=y*Z|MCpH@6~FSG`*h`9}h?NXOe8OM{2t!3Tu zI;E#qqB26V_0VY{HBk}5^zCz~mmoPlgs^q7GJ5)7QTwIHLqs>{2R-4}Y&*b{qFCFD z2M+Yu>Bux-Os@B4JaA5$gN;3Ve>BE)Ng%7we%>LEh3+KgAoz$rXcMPdW3dZ)xzElSF0R1OrS8JRBwh5bfT&zHno|t>` z$jQ1`$>CWg*>BJLnnf90F7ieqgymRp$!*xF2Uss^cGYZJT`F2mU+IhpZ(^-=Bm#|* zs-~i|HTdq>)~+JK?eX!8t^M>!`*p0hVm;Ck*GL*+vcC6!J?iz*YBm;+1uEucVugQF?jh!ziU)k^kNqEG9 zHyv42f%!^c*nPU6LKVvxvYSQk>R9psSx7Ghr0eJs%(_z=QoIDGy_uu+VeZ>7iD z(LG~tB|$@C(#|=anC>aa70}Jf$6PC~UmF-hfp($8dpVX|4=|(>m1paqP%2WhL`P@|MMJ8He45v{%!Bu$!U6 zp$!y9jm${(1aLS(r%6x(9!*Uk!)}Cx{)j_{zdAqTlsX)fo#TLONMtytjK7=NjqYj=Jn~2n>YZB*I9A0G<4Ny48J~R`=-^nVs`Z!c=R}kBuyckD&~Ardy_Eiug7fPw{|?8jYyq-la~AV&M@xf_{kkQ})7>KE$P3aALA2)?afLN3E`FmHlHS9Z z`+fvl&sURjYc;D4$=Sq=u724R8*i_7O0)dsqse<78&BFDYxgzVc#-~kn%%u%5ykg^ zisjpB_T2XB3!$-So9y6UMbFsKiZY$m?Sh?@VK+1n5vxg(Q@4u7?CMX2s85;=pXk{> zWs@vqS^Pv{8nanLhcSpj-q3+CE~e>ty$^d{zw2Q4oI{lI-7xmN5z7S$vUdzx(2{f# z--k|Ax}-s=+@6Df^!Bm8g5>)_%+bdfOrouMCsZ48Rlgb}$=1Be)J7&lH?kG2Lz6LU z3%yG&j~g*ci{HE)q{G@zjX6ol=P$kheYqX>$rOLx!J@$bU1#8Lj{AMZtMY$lN+VEw zN66JPDBHIXY>J1z^zGWb?%9N4ZNK7aYK*t@aTtM=CMxxRVb7d-?sM0UGS`_FPl;q8 z+t)rF_eeV^4(eDNqpgdQJ6Tddh9s;~6xAH=MLH5ABZapX+w}~XnLA?`HhI=`L;*Ky zY^QgLW;CI?Y|*E4?1#bYl=9gz^$I~(v|Wxg{i@;k(xMKU`d|q#)U)FknFvHzI z4}UlEDH9(;@X5f}8>Z^FNd6E^=FLt3rU=`;RGvsPySnoc#t*1TN~rMUMcp zXwX<;T}Fg(Ok-_!kZN-nzgaa?ZzK4)wV3qG zI>;?MZZFm2v3YM&FRY&Li&T@gXV5RxdZ?5>Gxx%7)J39+lSO7UUQ+9UzYcwpdGsfG z*1aw4t7}*mkG7kw9GBXBAs$pt@~v%uk@Y`F_UBDWzUp^p97 z=~lzcIjAdw*$kP( z)_iUaLht|D2qSMxV^WWR7T;5y#=rs*SUvuazh2^^mMM2@m*DdLQkvqd$a!GsOp z6!N-71!7CJyNcGmM%=1cTval_O2NM^iAOlUU)049pN$s$njCyRW}))+`8E;|hvi+T zks$Hx4z(}Ps?D|oi+m=;+vYA&L>!QI0ej?zU{KK8WibrRu8g z>}Lh0pOQKCy|$jR7zj>|6uGmPk!7xDIpL}BzOWA-dvv?maJ+$6ww^w_oYudTSs7d5 zHoDxLojr^bof>O^d-38eRIFC*R9?BP8P8Nz*^OyTN?kPIlEqA$so%`C=Q?W8M%gck zW5ZnbX9jtSJZdIh#sXrkUZPjqFrhwAE^VAJWfv&BwN#>hj0f51Y~<@Wv0!^P=DT$d zn_H`gLza|0j#CjEr)vO9{fTcEk0)pvRVXZ&aCcsR6J_oWD3%tx3N@37UC?e+l@MIA zjO2RrqW%u4!VG?_^BiGQh>ZK%rCtrMLs7;tebQWr`$-zrEsuAnLVepcZUs$AV(=0I zC`LW4Y#_e(ZgHSQo$<^aY~=T^kZf=W((b2vRu9qHYuhblgYbLeJ_#yWQGg#&9i{8^I_bk2FV)`)JkDN_NzZnNULBK)Mh2<_U%uvFrncG zYzqb)Rd!}spxaBK8o%~nbqGN6Jx)c}M9(^&Pm=BaN9+jTPZGs~A}OQHVWqlqIE>}T z!y>j^$1SvbIq_a(9H|+)&9wWitrhp`*d{E4mthI$0!pw|ut?4iZ1oWLdktt)+lZtte%D*qjVzN&L8OMU zV>yT#G4JztINUT{P|w6v zuy_7GM|v5jCgu{^>hL+?rO{jjIlQF^nV#hL-lZ(^`q~-4z4%Oej=w!zreVm5aHY#+ z-Or;%g|H@7sJ|@~sgM=&x-6S-|13xkijE}mS)PJ=IT;okh=XUV-Aj5(7dy5s5638 zFv7B^JiJy4c+V@GJMF^jb-g;(BCQ6(5>)?)u(ugx<4I6t_|Pr1;HF*%g22%gkl;Bz zIMR9J+J3Itl7N(&;+G}P&#pKrPBFg167b)5?Z1l0=AE25Y?`J>(;0!kYQ{5ZTLT)k z9aI$hJTW&XT2^d2z0|{>rnI~tb}w!LeY&m?$=V;ULbONZ+AocmA*mhJFI;fCmaUds zi%Q)@E}wuD1C2tUY0WOJck`!egpP%>HAEd<+j$R?=O*})DkZy@OxzN;fV@%$qAhn{ zPsl_fwM3FL#%vw0ete%yp3EhRZ)4q>I=N|l-wFrV^KE0N*Tbkz79wtb{6IJJ0BSl=bSykjw0c4=MDEm_}5xAi%Oz2|L_&ex6TtcLBRKs)u--t&x^ zXuhl6;Pv*+Y7sfwY*w$|9j%s)+wNQ=wa9D3|{H^s2q-9n8(!P-UH16)ZXQy7* z#V;;9{%$K1554UCo-1B(MBbjm4GzZmEgNl=@vvWTnY@=|?#i_`u4cpPHO5yXI6cnV z#+`IHqsd@;&kt&}8q(9JGuhlgSnDp2H+MW2UE8UI12P&mDP5n3>*l^?ZVofX)^ndW z`F~`YgeR%%`c4YDoA3Lz~KKO#tv~FJqq``xz)f$<^@hnZY12 z;&J;wG!-8tcRm|I=vTKt?4>v3yc>@`^I8pY5bmP47Z=XuQ0)=>#Rwe);JokJG~1C$ zB63xL1hx@f_$R!%1?aD$woyUCQuq6SAZ7t*JYs<#@z{=JN9B{4`?8am!-^G>&zZ^l zh1VNcRkC_o38vVGFd7QZr#2a00Sqo;q!nXfG-HTP&liJAYkgR?w87uMt+w+V_Um@n z4HjsKov-*yN;jJCw`6b}6kwEQf23@{o8nOCVAS&4$YA=#XEtp%gqvPiB35GuZB(|u z&~eL3+=`#UIgzkIY$mFRUx4E`@F8&ZFc#!+*RKpKy(kQ1u7gGrH!8CKNzVZK^nw2L ze-Wb?M!=e%^cgqf!b`+BV>xGRO3H86E$qmJ!h*~uF~G|I(P5- zkOpRN)aY(x?#b$g1^13;cE2t4>!%S)eAB75d-j(z^wWcwU4Bc|n{rUs++lJ!Kq4bW z_9r}u*{$5^`4g7u+}0$c#BZwQ+gUKVN76U%#I5%9X{}!-H44koO~6~VT}^PS1$n%@ zJT7$Z%`oM8J4mp;S(Upx7Ls!bhCU}wG7~%s=jyLYTuY)|6Jv`)vC{|>f&cD&d+Ho5 zQrc7M5j=BWXno!0e|{L`&@yVT zlR$=Ly^#trG4p!gx2ZE+gG}!UnBc6QX`le}n040{-2QQ$={pT~P(azI!xIV9Itd6E$@ug)9;Twb6EuRL?Ra7<3jO&95z>t4C^!DOD!Q;yOsAQ9 z2SN(_{{Xpaz#Nesf`9Pdt;WBmsdPThMW7CO|GtKXLDyG6a5q-XuQm3Xl*qpCzVmjt z8>vsv3XC6p2xG--`T)`;J?5OG3h#%GzR($&D|z!Ap0swe^^)5{tXt}wJ94M%*sExK zdD^YsVfS_$RHQCRzWTsygUkrCUA{XD|*(gR$fKo^}93XzI7d1&~30icod1>yExE~%jEHK{s> zKyuRyAuFbliJcoNVXqyk-%ucSV{;9XtdyiUI{W-q%GMV^y~2@=m6WO51{`h+cR}pz zLT#skn)6@v;^b?s(NvpCo%~=%oTp5?)XW;z!EZl#k+*x#@7B_E(f2f@w~Jk{GaJhS_+0&wCsCfIOEc4HTUD{LGc&|o(ZdwjCB_0yuw0& zuRzzeyZGGiPbTTi9O;gz1az{HjGNWN*bfox4`D>pNK`+HA}lWikm|FAs9+(?`oFhZpr!9IGSFo_9bK7M(2FT z$>-6+@~C`pa>IJU5#)gN@_bw0H-@V+5NC9#_6pJ9;`Q`4ovyZOmwFmLw`g8Y&w85I zKZAV8YAlc9PpTM9bA^*5w0DRyf$96eLmI&ekCo%NdQAOf=H+HOQ}0IyN%~&Zr7rXZ z#r)hYfwFyFsNeUeGBNAVI7vM}e|VG}N9dnSftYIWXr~ck%;7a%Jt0YPv8YK8$0qzz zC7~QlY$3ZPxD2Us1;}5iW`xs!WtEo2szyKxOi&uPT(MnkS1Z<4*LgIE6T~9eOMwfe z6id{xd5G?s2%X}SV#Qs8-j@UaWx%VKpu))m1kN1Ph@(535Ep`v7-Wr@>xmaWhZRxye<}?G07LdR4TX$Zv`H3_V{%AtxaMdRlQ6bL-H@kB&3(ZSNeAdYq)f^Y<|&7@T>X{C8xoP7BmQsa z%*L?)kxSJE@OsOo9bcO(V zyu;Z~U0N)#LKU7?#R3EvTT}qglIOi>3q1Y%g@52${;M@z2KaZD{~7@SS(pDPMRa8( z#zFCCeLZwjtM^%?eL`_VB$EQ>CmCtn|BUkgw6-sccCC;$$b3!x1G)IaNs#1oEA^F| z#7#d``3nm8h5SA%<{D6KHdI#H02w_Hj0S?ljPSqv;{RQ;!`=o4G2jYbbq~x0pp3l- zcmqKLtdAvTne&n~(O|SR0`Dw=CPvzfFSWqN{^xY%2t2{KruW^JGCTa`N6E|n3$ns# z$Nzsa5pxRua3&pZQvT8~eAPeTK9$iZ!TG9NbnpIyw(>tr1Ax#C&>xks_Jse8Hs?K< z!Bwg@iPmpO^z$rr5g7e|RLsXLND={dWy!aWOC&g4Uym12}uuGqQH z_F+Jn!IH-%8karl0-p7%$q%3=HVYiz+aa4?{fO|sCBbg6+a8#jlzhPNn_6EOfK6b( zQ~E!?36Q0!>PkcOewNJ;$EfQz4Z^ghFqZLU!w4bZM+CD{511czc5&I7Xu)H%JUBYC z$ka?63e9U070iLCtNs5D3>d5_k}o3)d^&)9tFH#Ab+^T64kh2gBGLYSVWgB(MB(h* zT>mBmDBNPc3}nyBtT1deBm^U3nJj<)aI+lo&eY~OOEOV9s+T>iBU z{yC@Wntz_ZNoX0s+?9mxjwa7=u7ken-vV#{6WH6sNOo}eT#@NQFhJ=oBw zA>Y(b)FK$w%R3vB@e$$q7)KX^pe-d04~&I-u^)x_{|mL{Yx1E+#pLnBV_CS;Kh{h{ zRhQ~v`!XQ?ZEnRPCzML2(`vB9Y2PWI59aQ{u)2hez9|U+GPjW}r+TQt zpaXXd2&}*w8emvyKr^rZ219|of~%W58vX|<)iFlE1uC&XP~o6mtKFUTrQrzjjf&=E z5LlvnV5eqj2%Lq8{eqlw)8p! zN4mMQK2DqM*ZsVAzl`kr`r4!0s^zk39OLF`hXBwl;zww+&hY>BHU^sBb%w_YJd1`k z+HHV!2+S1y2UDpl1U4cuR?k$fQAme7$I3w-}%gM!*#kT7(63DbWt_H$7n%Q#1O)jz!*!dJyUeyxPCE@5 zG?IJuhyRoYQS$}v+t>iJx#VErV3R;y2I#i;Z0_{bx`XQhj9^W9-}UyA6+xy$5r19u1Lr@_32J^ADM2^FDo zavp~8(zV)UYn(O}bv6bY4!SV%MfkXSr&Kz@!e67Q9e=`fJ@9?%g0ml%lE4n+v{`Qh zQo=0xY%+h&BcKoAei`(OU2sFETuIS^Nf2zxzNV6e9CE3avx94Uc%G!rEdDRp_unLv zvaS@kAS1Z5zxN;rm_YxkMEAkU^5_X)zdA^pqlS8PhkR@WC<@ybdT6dUl=`gD=#R&+ z-Q#=%0VpslmRGcNo^LRd*vW({?LapY9l#zmxp5K=uR;F{O;+slBhi*F3ay6GFTo>@ zu~O0e)>AJ`I?DU}@Y_=S@$+9p{cqsW901%Z6&*JEy9d>^zQ2wf=(fB**LJ!-WNx%Q zvxB$eLAgH--PIx;ncfK2X}%7)$a#*2iglj#fg7NI95SkL?RpPIquF%66-;DIqnq0Q zEdFOs)ZP=ZF4ugWk-*vBhc)Pl2yjx&RR8M}{N3fg4S%vFm~)1o|DN|$(16%Du<12z zb9@Tw*!EE7?(KCFiT`3JUbvsC?09ez<0E9l_&nf3oV6)MEgHPGEeZGhgb9j;)baoT zi<)h&jCpENlot@|>(9`Dxt34Kf5BS+ z`_#*U0Ss{3_v6oh-$fcJ@VJH8AnprG(#v-dD(7feO`Gt4&J=7!UDu@dZEfs&z~SF9M^O4cg1-GhUPt!z z{4*H+UtY%lWADA=sqX*(@ls?)=0S;2GIA)R5RolA8Hr=>j1mz}R`$v&D|l~+`#wbawoQ{cSvEXHsA==3ix6IPL05J;$wDa4>} z%wE2-HZ?iv@8>j%d2N1#VKeT^Im{XGXwP&vKiTo)h`Hu~DadVtl*9F7Lmr`++fNLc zKWxx9nRrtPr_oa~+w%0!@BWYEfJ^C#q4L<&{4SEen?N{`TZo@@^+T+~A8q8n`;H}q zXR_~F%m3ex+#m$f>w&V9mZ!vPfF$lRYlI@aI6sSY|rV|PQy|9)i8C19^ZhHm`vl6~LibVhh4PUxR~ z`R|V`%mp&*9o|N|6p9sWpb5A4eaX**pVO8)qNzh9p>EO0R{ z+;%+q3Y#Da5(@et#= z3(8{Mmk+$zr~U<_Z0BrZ6t|Ya7LxobeM0Y09~td|^OPN!eOh=jnrCo=n6aUFpBS%s zT=UI-D8v%KcZ>V9eUAFYOPS@9{f|d;OcU{KJ~w)yO9Xx!+Ld=_J3P9 za&k1WJbusHUvK5lOJm{IM@2n*YWlxtYRJe`p*fk%lwj7||5#e+GY^dB!dSxp9v$+3 z8}@w@|8K*-3;+MCgjMB@D1G`S`I*XtkIQFojz(UZoSMqY`|OCYzW;hX;Q@FmRcI@+ zd66QJU1dp}6zPyFMHUyOrKNdq^_SWW-vj#HT)lx0A1ZEbEE_3KYBj0rPrP|>{rdIc z`k?!Qc1Hv(5-R2xnkkwLGH?uE1|K?YKT>G^Rw{@tvTVTmwmo53rsuNj!y$gq}n6J52M(A!*Z4VF`GH)#$y z{9m;xea=59SQ+`KYVoz`>(Y0p9mYnq`VD^n@jGyJnQk)dbgbt^>>8L^{_-VTw>)ZB zpzHw&;^oP#+*x(DS2P?N*=T|QictTAxYU1!z1BC z{Aqf+<)Yz>&-p%Y-ztgf+VW{6RoT2gyTK`c$N2H#F_E!WmCM-5*}{(rbv?hGc@?_F z<(qEi#)i6uxrCW}AJ8AlW7{=6IyY3~N2bXTxNmpYn+Fmwe50YmA$QuT4{~|V@#ab1 zkQoqpBO#)Qjo({p{xqJq2K_ldKE_ogPfYKI_6sTV9TvLg%kJR51-Wd)Q?}OK0jPrKETg2b!9kt35)zotc|^Z2sfv z7RHya3Z-p|TI~3rHm|JcMsZAiW{SF$#Es1^QR|Gq^={A&cj3TXT9b6+zl^}&--D7( zAEm*V=!HT?PveE4O9C?%`V*=&ItNHXbeg_w(ri;j-YDVig+U4gVG>FDUH zZ6AkXsk$0rekj_`02}FqP?>*U+Yg`KaKk3J*=Zq>aQZ1kh|rYat{P*`obTMlyvx$U z!rw)%M$Vy7^v~Z{&IB8rdehh{8XiDaAf`=RFQ-q83)2uxS{@bM{Ou-xQzeWl@YM59 zMPlJg%7i$vJYCb>1vTONsc(dP|FlZv?B`C=gfuJZdf1gpmRF%Bn=$Mt~;*|`IeWJnHBM9YU!S#{|Qd=-#hN1K@-!L#!Dl= zI(nq)*&SemHq+6eCwZKcV{K#OXq{*O)UjCY&&rXI5>3oO_NE4W39KrO=2IXis|#?s zdDDz7;s`$tw75TJah2-+>+|6{``^_}z3j8-t#W)YaQ1?>!E z`QJ+Iq(*)?#eetA4!&8(YFPpAQ8h@ICyv@v-I=2!eG)~+Z~7|kg>{JO-=cCn5GX6< zE|Co67;{0Ogbb^TZ9YtT$2VOL>-qQX ze*EO&fr%+W6o)bs4#%a(ECdb~wS-UO9RFAu+KuZE3PpoOCJWJ`co#h+tEsuKC{Vyf z8GD`hx#N!AWfGr0F0Cgc2#dDEYZ02s0RWcWzjtqc&wP4TR>1JXJz0yX(N}H)>8jqX zrFWm+K2nN5Q}jI2b`ZLykU6bbD1!*jRqb);$TMlZ2RotSL19-$#i1d**m9CtIymm6 z>3(KPhl3M3TDVOU{LMXE!Z*MgaO#%bbyrq8yJjTFvT12yW0L@eGhHP`b*j7amQVm= z=4FDwu&|~Xq_pUi^%^xCNehx&!I(hoL^(G!g(%y|$h;asLYr9E!8OL216lPCY zOOQan3`s3HkTbajYi6aUjwZOwJ{Y0zpYs88TAq?JygrpXKDs(T9G_0%yLmX(`M${Z z=3@5whMdaJl^gr%mqVV@nGeA-vJHf}E2bLCC z^w&nC_4+MyEAAxt?mt9VLA8Q+?ADMsjoD%hk0G%BQ?7i<{-Mp`X>w%uTEjxpS9v+ zZAypmwuowszcyW}3_r6OMymw^qvzu>iL{O+?56U2pX>hB(uYgx!!e_eqL;6213fmsrQF(^w|xu%F^3$jvLfsuIcmA1MytNL~?-OdxRykJ)MU-YPIHB1%F0*yFUfb05CKMl!HsSc0#*^!B0_DT(h8 zq6BJaRye!r8B8**tgI9cwW(IB?{(XE&{iy_I#`QrgRo{oP>r{}_1--?sr~uq*vJj? zgx`iOu(BI8%dhd%i`=}{nP49&Ay$_YUL-cAxK$BKD=mw9%Tzfo^4ZTvXMPseEs*@g zR{DcuueHMZJs1C1EozjvlzqOnVYQ)XPNG_J0j2M z1W!#uGG9;({(Og;QcyUWuF8Yz=pA4>*$}8#@m@v;`p8Ep^csi|(TbfK9ZTH$^2n?~ zu}4;G@%VJC&H#u|g^o}UAme5W$@`UqUw!_no2Zjq3@_NNBzL}B% zw?NT=**K83s76$U#P1pyTMa1_jnW5v$Z}UWah}zwtZ*kkfBKa=6z0i)o-9+D$zK1aZ408>|ZVbw^LI7Ox!JH_01i8 zQKk1d=ZnnW9+j#`SNhBN`3)z}v$fI2Mh4HIg;Qa|*8%LQ5QDxwcSHP_iq1I)eU<`x2Kq{QKYNy)P2&1#ZYBR_Q=@*;ZHdW211z^be25boJ%rx20lhRqO((lZCe zX>Fc@>apFJ=UuGdMQza{@%13;!)vQI({r6;t@n>tmB&UOJ%~y;{l40psD|ILT2XMw zt8fa{|Iv-ne(MFPTa~?_I3KkL2;Ui>#_YN=DN$NKZ*%*0`CwTVSTA@i#Wuxn`a%*+fE@i4~{ z!k;#cboB^1FYK^fVGh0n&ylKA^N3N|Xp~O1%7cWtmQQB^9dOWf?!^$R-V5mQdhQN< z|Ev7liCdo%W9TEgM^u{&JZRU>-MV#a@>O0doqWsuDngXywJ+oIlj3fe;q)Z!fC`W5 zo}Re|=8(K`pmA;{u1P*k{%Py%#x%wLhao`Oj@yg$%c_zxT`k?1EjM>n7o|5B8LhAU zly<*M>4wbVlyDYW8K3UOvulk?voQ6fP3K|A0MOPsG#Zhc3UZLEB6xH zUQ67Tm7OJI9|`sk{)o2$K!7Uwqxd2-g>To(GVsr(5RhxRa>ykl-%KXyX&KCW6HWG9 z@@m-30X5z0?=G5e??u%)H64HSuufg!^BO2D{z-B5#<$LQhl) z5`5fij~kJ9?ViE&qqhwf%9N)b_%g`hg$@(%-SrS3=@n(8xYhhRFVYtOTn9qgREaTF zYT%Zw#&_a+nB zbes646^aavUVpDEEre{h=`DfjhiWa(sA{x{obYVvFg~$rb3~|j^oWI#3Px!1Ku!>0 z!6HH$C_|kRZ{?Lxx$)ui*|8WyC)!cxhexlv4%0KsE`)@7srbA~^T7~n5t#;#nJeTx z|58Hqymf-P@eR_$njO!|f{-Y!rgGmh2j?dP*U{OKtV}wZcS&s4qW)~!tY2}W^B8Tt zisU7(p2Pgf`hIs#P3h;R3zwW((^vZ%ed|+YIQTF@;b) zXP2JDeJ@t$rN*r4PbmEGgF3y#hkYxcrK5sk2Y+uT1Jn}SzxiN4y znRuWrLe=XlxEcdMla5Np&6JCO9Q2f@aa{1w?mbIhO|;n#z{hPI&l@&zR}DSrQ#IJc zkxTr7+_QbZ=;j<-t-p3mSw8p=yt@yXpBq!~Gtx!4;UzFWJ?Ev3#oO$AsDtti!F*55q8zH0x_o^J2s^$s2)$A5Y)gIPsACT2Peed@Tj>oj`J$C;7IMY=VK> zswe1i`%SzA5VVZAX$n#6tK#Bfq_cV2^nTX@*HvfvqcZAe7wh#0JHTqo*vjvxv+onA z_;i8T_91_LEKgxT^Ud-28w=qU#~VM$47Fq>I;;B7OGR@pelSxC=G}A<{PZwl^_YYa z2+|GjXHi?c)F*4LP6-n8Jmj}g)K;W!uvRds_l)PRTs^XnlcSBg0ims8=# zt5QD%J$qc+R1^J2Z4za4W%on+&V-@b?u5y4ltY}8FMg~#I@l8OCV zd|q<9%@dt^(l7t?DzQCdJ&rx+`(P^qOmdQ$s>TV5m0k#|p0Ztas-Z{t$oqIuA6OGM|-{dz4u&PQ_lNU2KW11DJ|?heLVCMi!2)To|Lbcuycd zwx5xMaPeW-C1W3v+LLX9;f2fR2uM^VTWfMoPglhu~{%Qa@(efLoPE~&#^5wY`4&?o|H6&iN*FuHZJ0g~1 zy_gAWP)v8vuyAN^`*|@IZ57^Z$P!iqbDol{rCOjIpC+J1&fvZei zNOQC}O3LhoqdHfsNs=ah4b1)~!{frJ24{JsD67I7< zWNtB;QBZ}3*d|OkH zXj|AJE`aJ&`xxD*8_{S1p8x601+L#R0Bpx0_75*R$d7bAG#t?I+~75IykGQBmBPPlM*Y%Yp{g<*-BiGSsqqYY5(yC8kb82Q8QW#8$Lp38|Pvx0cS;U26$Oba5n;J~_YU`!F-6 z?A`Uzs~CkV7V>u-HSe%lhoS2gRbu3$w8LleC^dQ%D>KH4)*;G7lN@4B)FkAPBWjnE z;$CY{v@>jLHuGfkYVf~%XZM+w=omVme9^>Oj!bq3&%WT)I_eXb8+16EDXa6$g3qUo zHH#DndK_z!mR{TeU=(bbk#d>de_*IUDJ$%AnzWwZyEdRQ;^G|j5~tP^ued-rp3FcA zr!3c#fysR)ee!+R`(zYIW%?t|gr4|#p_gs5UM35@B3~xkLK}65$aBp;dOTt1)lfQH zDaokpy{AkLhw1z4y!O&ih{c58rK)yxR!<0soq57Q^vxH5E%?_z5F;Q&`9uy%=F7t0 zREh4@jP#Z9@T%%Qu#F9Gpgu=~-MtzYG^AlZZq&-ckRjA)k6)v@a=v%t^Kl+q zkk#{})&p!?de;d7GxCt^y7sigflZ<6VQlBQ_Z7cT@M9vjw6->;lUffSKj;@1j1hBQ z<_Qc)E;736h1u&ucEE?nAn+zKmk7sX1VbwFg^b2K&hv7paf?#`UsjB!y}mB}mz2^Z z?*Ws`(OJ2bWqP-cHwCrjo|GzKvpbry$6+99Wx#zK38!KF)Aoj%^u_k+7I(SW*1@r% zIdB8H7jq7}$T&JYn?lBH{`cm{C*@9Xii~fqSv>=jT)InMjK}^?ZnGBUF>9~$-Yd`c zmc5K#3N$9jcny)wS(oMMzJcmRDtA4J^6EF~9cOc{RIDR&54viqj)p5rUFMneCLlQ7 zj7xJl^m^L)k4G4^KJOhvE98jwyb`|tMPrLzAwwGf++NAUw4F7X@`&L*9T)Y8^>*xa z#TYbGOK?Md#-ju|xiF2R>Fuw~ZZ>{h?&ZGg3^=OT?NASTHQCt`R!RUys-9KLt+9(DC!!3L-R}8BIN;mF~|0wp36V zdENAjuIS>{EucVGieC#FEq%gJttg%pL@!{HhF>f0l}5=(LC0sIuIEeYB>I3R#Q`B- zR+Wgk&HYfBxUtDfp+_q@#-A-JIv0G_S}sG&oh` zm^`CTyU00pn%q-@sf6|>>Dg8N^zkmpy9{|z9I!^zUNhjZ=OJlQ{DP5utS)FIJ2K zxnZ?JHQv-sp1uBIvP08AYory!%VTMXil^w5XacRNh!dE%pL%0p>G2IoCS^9MzR2sE znoiN?C*N8OvraMb;N2m?3*yhdm#7dMxB_w(`dt0su?C^*G* zqVVS9ZuC3rt+z*O_8)&H84syE!~1JgpG$}LKcOJ!PLnkv?Y6zcBI_|*v2tY@eKW*~ z^PpzhYwtdjK+4Nc?O6}`PNb&<%atFDIi??eJ>6i;Id(Lh{^;~E!&%D%+*{YtWDx<1 zvRiKjJkXqUBK4aQ=DYbLeY*q;oM(Tpku*`GiL<7sEI&hTj1za&QOJk^(x|3BWlyFh zBv0KtKHb0e>W0tbxf$UCnkW7jwO_B^jK7$D;`yfiGKI{;ir)9lW-lHV68gFCgbpeA zq{#*lw4l-wDiDtc{hop^@#4_rMTNPO(0-Z|8~l=TpoPk^`jFkt#y4_Z^d zQ=_K8W>Hn5b=M7pYhvL}aM>NKy06IEKx_uzLu&_rh*ylq{BLRn=QGd=Wxxys$%fqN2XY53le{BD%Fv98aH zIdZRbYs3Fba>bP)qUO>*zoGSz-Sx9x<*f{lsFVDoK)v<_S_eatgeS)yq(Ec>F zyT9X;`TGN#K6QotU(y7^qIm92!o*KPy|)no&%lMbz@=4d&2D<7-170$qaBU3=aP1> z#ofMMhu^MgMVx>2)|Af6LpKh(co^{H_1ctkb%-g1pQ#Gaw}RQEm~7NDxc5esy6C=R zXO8D&2D&rvsch9}I&Rc*WWGJbebca=*;U4))IoLM0s9)|z|*Ih%QgkX@#?9z0iIVY zCflwzW=Py^)n*S*^xeJhA?|6J?Mnw#pNj<1zRhd8Pf~hPHK01+Q5k@{C zY}brTpb0op#%82=UQn2~_d1X!78VuFlemoO#JXs{ymR$k&B~ThBGZ-vNByVAr62u~ z$diefd6l4hPo1Vxu(YE_)i@WJ`+J2_99WQSjhci&q3TEUZ^=)$Ma?Y{>3BUzr1(_^St)psKcPxtwHDF~-2#+Nkpw5p}{6nInchvyt8dvE3maTYuYs0ttuB@)0 zKf{67UL|9%;_s(;nvb)8jowqT2$&wc7)mgNR#UYhN?H!e}43cHK$nmbpdN zHK(uCh5x!mlA2$$+LxprcWJ6)jiwTIbAekpFZ_#lVYnYj_n?p+FH@!aYhwF_TvwQJ zPrPr$9*_XBX6%{%VQkOpn$ZiNAemr7!jKSIqWEkJ5^d3}mj#`9!_7 z;xAQdFiEy$t&G@TYJ#Vmdgy+YNImTf+A3>mDQ-K#?IszIBbr+uc$z|ckNSJJpW)Nw zRx5AhzajLAvgCYzk3@XTT(0N!rPX897X+HN@+#>Vna*=|((b#gG#1Z4m?@0pvv&lK z?c8p+2X%UnHuGPeBt5&|D`>9eYIrh9vD)D;-qCFilTrkZZ;$R&@mLtELN1?{pHRrX_iu6ne@7_|s!cGs6RBSlYw8bGdvluC6zWPt z(aM8kzN(4Z32H(d<(|um|L*9I_Zhpo_wq^3vpSYY!>=P(o>61?tp7esXRaJq@h-Wh%2A{>SIPvlEUKn+8a(;u_ zJ2;k$bA;4_gC2X>OK&*-d8(}@Zro9xe>zjvDfKhZM`cE4#>U2utEGL>dL{7FX8rgo zgvh?_dM$yJh;jD96sP4JosuW&D2(RrcTPyL`j<%H+aJYFK|gv_8TlPjExmB5w-U1M z1o(OCn)32QBpAoq#)xClVgXhFtNZcSVJ+YOsN4w8iFCa_11a3QNxNhs0%J~K+b@%kh?UAjXoSc*^P38aNLBZaqAB4?odK$G1s3Ez_h|EE^kF}YZnUxji zd|Hq2Z%)xSkxMdwJ{$Hi)t&^N}Oe^A_ z%x0gV6S}Ov`)X#gy*AR^@O{n5r*7QMesp<&3v%X12$4o0bReN@wFUH8Xck;dB@|pU z%Jk{QB{V{QiF58e!nG@XmcQ{;JTwk7NWS9dI?jp$n zhw+y|^fZgK*_Hl#pY$tg-OxDTuN!o(>OMQ3de*P^o>3S2s&s4LBcfr4n@m7{|Ni{| zor0oNf@v^jih@xlADgiRw#;@y#tc&;QRW{L#P_b3y(P?eYq(Q%{ZYIb)!?_@)`}p% z)wVZ|G2iU2v-+ur;yR* z`46spk+DF!{pM>OhCgw7?TABzw7%w9Z_b^b%{O{j0K-+IgSEknxNzZuJ1KYfg06^e zNUmGlIEfCw)%1tb-Yg;QCeZgzvCErHHswptu5gac@;I78!awU$?v>|mA~YW(FA0{A z;;vIN?g2eDoPd$d%PwFxVfeJO;jViDxLMQ6e|@q;B}Hmft^c=9BQ|A95bR{MwjbO>>39kLL`iVTrFlUIMHV$DD-!?Psr{n^O5tt}`E(*DI?eK? zx0n8?Oq{^gE4T`wrOv%$Sv`-n-xXLqQe6LO3d9d`#6|%x!~0a3A9`z=q&*18OKd(g zg^LFT1&zL)Jgal_eZh^Koclkx2N-j=C zmJGo1g|SfYNk~Wl_-X7(Q|ep|OVxjO{RjEm**Uh`u;K$`IUuB&T1n;QE{)-=_-M}T zdx!qokfeMMG)=0ru2m%1vLG?AnAY(;hV@Z$l5Nw(@7MAt&>C?2dG>Ze%f9vx5vABA- zaF@LI(w8rRheh2wv$a@1DAWHkpYpplR?mlRpwe?KhjH+@_+OTnPw2pKXi)9DBuRfn z)P7$X?56=V&IQDGQGgYYBrGh39E`L84KMGr(`+k^qWa*_K@d;9Hk4h{~0H-2Y`|NRYqk?OmJ@EBP= z2IP%d+1_+_ckf8z9PA#sRle-Q>TtQTBza=evmAma5n*4EbG)#L!x1b8*WHeJ{L^0lOU;6h}6vgo}8 zszD49acKHff791@S&WYX?&+VG7@7|Im)w`20Ltl-K3oc7B1x94{%3HgGAi>Q?n{eexG|ru-Bqm^iD?((*NYS06h` zOYOMy?vlW%Qwj;fvEwlYqIg{&An_(5B7&<&&GBuuziYU4C?0o`at45JUqk>Gw0+b9 zMXjNuQ||iGbBmv!pWM$n=Yw`_GqmU_abGUYyF;Z7D(y!({VpA$k3bv}l&RGqqe1}z zt6quCq2nrW3+rxFK7Jhh%E!iW+Ud(|Te*&e-TH^#-i^5$Qs1)WZqvk%>K0so{>>yY zH=M_`QHc?$@RIAu2E7aQva^;BK1e%2)ejhf`XKs@L;@6K&IeZ&javWqD`^0e12+?K z)TZn6&>AHMnq4iiCpkhqi{6x#mw#fUC+_vH_p1BwmPx|L>VtMmH_fn3-qN3CSPU1; z;KeG*LL#_iu2@2P>SD;fKYn~dJQs|rNl>L!2*V+~tCDwXe5hvJ8Bvb)hLHBwDYkkF z1)dbOKNS4FEt`c=x$yHD3bU+7$Z3TiNDstdSd=&%g;86P{0x?0_c_Ud!k>Tw^_Ac5 z<9YpD$l%?HE=fCl=4;gWRljA*C+Eod-M3r~^TdfoG=5aW|H&>Mr#^~b{|qUlz5J=z zRxp@DhYlePF!%wF$^71rG)7EKP02nL|3XzkWj&{XI@Q_HQ7dQ8p?8J0;^JyOw6iHe zE_ok=lgDlg@AoDof7|aZxUg$FT8KX-p5*H5%|kg2T@;$dR$8a_XJcMMZ@m`T4d;0j zM?Jd%@sNd;m21vibGdM`J1%}N()*Ea{Z*)|{V%^>C58m)?BjN@{SRdvwTjgZkGxI7m8=eEaxC#CqOr%v^g>VS`t@w@Ki_T$ z=3%9Td%xI%ZDzwR&{0cWllF}54YjN)JzzM}phBIbE#l9^sTfUoG_l zxLJI9w>H4?3K86G%gcl2JmJl`8nAEIJlgaf#$Uw?+I>`vKb}hCGE;QP>{U}F2gDP) zau*8zafJS#E_hlTVj^Y>gV`6*@e(Mo&-o?5q-3g5MN*5wqFbwe6vy`4 zdkR;{EBIt z<0wXGl1IOgg6#?i`2(XwYs%@3!9ag3i^C}sm}~edIA+9p@ac~$NaEZJm#f(PZQ7C$ z!)AIuXV2xPpb_7gVMk%$?g+uuY#QX<&5g9);d)_9^Vq(BN#Vkjsu=bOnf~osG^Z8# z`<}h};dzKmA-C)`w29i%4|o5hHnHL4PT;|SbDZG1&=_uU@u`4MNDcR_Q>Z`EstuPE zU=PPv6uaHNor>defQ88)fi4lTR05$6Of%+8_z@q5-qVB#lE{Obb+m$ci)aud;*su`wLeh)2 z0?B^uHWfW;YK}vUZ|GPH@gNz`*34D+J+VcFp0<1+43;kot z_I~g&b$zliY=MyIOT%k)kVGBB&B)COpTbGJXkaq zx&YBK_F$p_(o=5rIr>ed#PS&yi`HaQ>eGV1rUa0OYC+m;Y4NGwI9WTIH_$pUfK7~0 z##$i?f7*wHp18(#Zbx}2zW{|vINv*^3YMAdOij1)eEi9(agWik3I^q z+|d^&Nk`PZFns0P6=4;9EH%S`0{VCdx(z+Y*$+PT7UQij zaYan#x&0W_cNS;!fyk2R>1n{P%98n}mpNaQRO@6v*_cov{Kjbl> zryfghZ}GMyzxb9BZ6V$k$)yRHwEtO{We(`us-8OQmoJCWcQm}Wx_(^SnN#gZ43Hu`Td#%EM z+*0*CF`VIK84qE3+yU`_>N;2ijGv5TpV&2$Dk|btw2V|dc}1C~yg*D0R_BH|srWUU zBdOfX2l$3{A02Y)>1b0U%59?iQEQ4}UH4X#@gxpYH;ij*?yx$EVeM(4vS$DMb}>Mh z9J#~}NBsoSTI^O( z=1}kG>l0@4Z1;cT8^}IsIkonDE%(*xA~A0h_X*WVIrUfGa_Q0praOAti)BBb zM#GV+V?@jA2TR1LZ{GM8MJwAOM#V0=V?RjSD;$iz#)2#=hmpYWX8rp5&FZ{(`caN= zhxPC7V0DRV=@0PF~~J5LMD9{;;GIlIk({R;-$70M`c>N zr3QW}SU|g|YGmL~3xO+sLv{w(<{sjEVK9&;hT7Kga>Iek<&M^c>YCM$f-9~j{2T?# z$T7yAYA9g2E+X=e!{-Sw6MnNV34ntANUh)a7zhs?qP$Fx2SyPZ3ToGJQc4k3dJL#<$XBu8bEx|arx%ZjlTIuqF?=Hb_trkI(WAA5Mc9KxWgUmA?7&&EMo@H z4b<0_Z}_%9D8*a0Q|rY_i0-%NGlHiYzy{fNY|{ZVOUsm#Km?GTks+IQ3F;9>jpVQD zOdr)M-cxul9Ps0YvZ2{CBqmy-)PO(^-tq^qCfmu~XWP9_ zlLN|>-124&qaDH;vu`9I6n_7iTJTrf(>?}=m2It0al@E}bZ!aX{kInH4+xvI zpl;{UTybKW-u?g>pIQo~O&DlA_`!A=3>5P-K7(3(e*gZ=kIVVbizMlItUY8~BaU9^ z0e~w4(w9;=_5|tng3DnK>5b8B7O-{k!$c5q~Wgt^jb%w=emQIh!usAHBX@LYWM+(3hl@Iyy_wQfca-8~B$zz0dIUn0`_XCcDb*4du}zbi6J>VYXB9IL=qpRpflnB$G}K28Rxhug?e z!btN28@XkZnKzzqe87CoI~8)II5W5N=u^?08M?s zrlEN1RR@SE?*rLthNi)DQ?8r)a&}BTc*?{zO&@u?WuTk=k_Sunqes5yv|SAvS7NH0 zya~t~`^kk2F*8eMKOU)alJH)Cp(~SssU0c_{1S(i<0O_MM?CJ<`vSfNd5Oo#w07Ua zt-138RaXcM8ZQ}KI2|TbO|l^FZ13)tt*_B>rDOdt6iawg{)w;Hor8D6R09#P=BeQn zGuh7Y4zX4MML#qt)^fD`T(Pp>uw$KFH1DfA!-^+v7m9@6xQVhy2`$A^?#j;*YFo}%W)SZ2OaSEgW5u*??;txwVgs#_)AT4q8qQ?igV&*>CdP{k%LKC6BNeLm=Mb0%#A`p4W{A&w z=tTi3$h_|oWCQovyNZVRLA@>meKz9DqFKO?@dJEygdBf3&OWkXqXGhKfI=sHbaix@ z(iv#-K!427mW~jQzqQ=`Q0FRdN@UwGL(#?j9vpqXa%@Lu0UrxaUvluicpGv)#<)QWl2diIvfO%nAn-3CZ<{^1wqvQM+Yt#an=CZ;3|PXB0j# z%)WST$4pyrN=uE`3&wZdEk!<3X(C3GE#m8M?3_C>GtK-Owh>0Xm{C+ps3h0XeN~zz`1A+9a zib;B&K?1GJOF=X>3VH~=!u+vfsK&i$%zhM}|AqVfL9YkRQzqD@s&)Vpgtg%AQgh@+ zg6m+9<*8s{g6c%nOWko)?{7#wMfHtD7zwaXqy( z(Wf4wHEu}dRXP-W3$(aQUUfNWt$EHf`1K`*$Qnxuz*Ym9Q<@5FQjg4LS!|iN~ z0QHWe#gV{_^&Xoz1DJQ`wTYAvWIBAHqziyXt4T%tToIi3lT-bGgS$m4x@ntc1bUmu zxiO|dp1a}{oh-FQR*dP9s)T^4Lo9 z9E_)21ZeppZA2q|w?u2;E+W2&{4g(-a22zbBMn~I!|QK+@rRty7XECH>5d*}n*GRQ z*~xil^#VHv^_s;<_zrQbF;7d9CJE)x9T30w>XqV=gPq3^n(ch*HImO;i=x|$fZlGM$p5-Bka$25SSi*J(5qYK zhdj(4J_kzfM~d-s__a>F7K6r43UPvAOa@+btm?$!>Yyqur2fp>8t0Wpd(W z_C^-wSb}g&hVe{K(GbO%bIkK7(lVu58LOz4LY{X?)`uY(gY`cEIoB9@b7{(S3oOee zA|eVFUU^T;WsM~f)1OLMj3(G4!Nf!ySKIp0ofabvVW)ex5Raqz`6dx=OxYaVZ6cnF zY)TPMWLonZGRx46?U#RfS~e?4n6pPBu3p#|G09vdefEv4(Rsw0U#Ro@ii8-5P1Zjt zxi_VAx5q;WkS-QX!duX?IL4+Ic6BC3V!lY0@*Gi3T|*6ieN3yjZ-%Dim1{dyWMWMq zEUCK20J+B0BUO48Lo0d&3fhQI=l11g?xyBMsrvd=q5tmxH$R6EeI4VJn1R0yLuz=F zIi%+SxX=Df-Ye?55vs_0UWbwy)yyfePW4VHs-7jt$fTukJ2KbRTlT=(u4WcrWPaCd z<;JXXpq!>l;JfNHg;;SfiMRp)Z6>XBkydKNTbHo{2-TG;$Nr!-AkrRbRTKb}DR9lJuYX@!Ae}L z@gZ>;rXPqo9nWlPDoD4H;SBDO`S=nmXLdrBXUNP5W+XT^17#WX^6LziW~Y}YX)o@T z`ypLO@1XLCA$l(-Pu(vHtwL7zKv*nG7-w!3loZ1k@cHcFL2iNov4=`#QrE(Mdh zk+}v|!xf5z&e2tV=s+vZ0v6eKNE5!>+0cIAn25z?xH-W@{nbG90g)On(vW{%?^v8| zwIq#ht+t+#zCx%>wj_EPApF}+;Yv_0m+Vdlf#;UL#64%ffTu6<&ftXk2q$fz-xBLD3{^L+e zaTOmd*!}hb`jS(2#prnm^IUl`Vf~s)>K`L-EL{_+T(?UQu@I*t)0 zJN*{i^_ze=<|PlH^p6>-r@u;k7=4<_QWth#=s*ePYSUf=Q zW@z02J?h$p@tvQo0V9n#nH?=oPax;b6XEXk$>A&x%7=jcDWR4ck5tHkz)`hGJF`)zg8Q#5}17i^U;&#veyv=7M@1`fN~?YFf=62{Bt`C zgBON#PkjtJZIJm*GI0dQf()0!7jZ{P*0_Li1XqYCGF-u^J!f!7SAtmg=49(S>Sar_Xr=|;5nRW1v zSawQ%-pq}}=g)^zgQJU|srSv6n`qjJahOTQtM z33DuwA~t;vc!dyn)s0|-W0ry5C0+W{G?somP5~#MI6zI^s$l$Kgd(A_YNFPzX=wKs z29$qeNtaS9=-+V?fxfCYg&hV~4oAimkulj_`hA)`<(1R9y8>MkZCiL==2`WEcenPH zgL!c^0V^~Dpd$lj;v>^ni@x=aqGERGiT5u}>(YqY&$rV=Q;QI!$Vwnsk5Mx%r|9hm zt=~Wm*mDO4b5)}%MmrQ7!j=pb^0n)l$`o{7DAiDNYAR=+X-;RT_u|s6ICcF>ZFhvp zVf4I*`EXhg>Wsn5*GKeRK6)BYp5(6a)Z>%d=~Qe%f&|{l$_qbsRbg3htCO;TRl~kr za4g$!?L@t~Hv><40O@h2Tm#Z9hkQqMP02tO&YssgwJa?b4fRmZ1-#N{QFT)u^a9#( zfyY;+O^Ya4y2RK?#m#&-`<{JDCE`im04i%1cC}=R^9` z?Ahwa0OCs@b+eO}+iXYe%n|UoToP!Ia%~0anO?q|VqL}6-&XnwLrr7_u7`fxc-iulM zIX#l80asi^Z)Co^M4}kc1#P@gO$7d`#k)rrE}3V`q7s8;s#m8~;%llIbeAS9WxJ+A zWQ!scg(eE^m|xB}2HhH3TN3!X{o`Fq63ZY8wLic-e>{B9*UC>?JAgJpvQLsQvRHJH zb-02J$(XdqS1k8%6zzIJbu3w|FhV^re#ACVN{*vt9|tf0QJFx(h~&0ibh54HTrJO? zRjh4BxS`Kpcy(1tOoqL_WVi4lqXg?c0$qWI~ortjo$ z1ouXNW>OP!d2O@uuyXo+F-G%wDJO)YFi~$zX5^A~esM->s`t7Ud!Q0$enR?Qj&XP2 z+1XMd3zTeEtivm#Q<{Y@NxS(6p%(*LP|+dQIJvriSLXze%^b( ze_XScYw^DCIs5Fh<9VLln}Im1YL_ydH_+9g$k0t*Zdx5Ns%0}yFEhsEH~oMmaH~pawmxii0$$DrW0Lw(n_$B=Yt8ge93NQLo)?FtA9(wU^2eoU zy^~jjUW+g{_A7tAAYX=#!>##ngb;RPx5m)S`Hwkmptc}4UFNCo%uu$6)7T9ssj8 zZk44KKDzZM_%*3=ucrK;BB$!q#2mRn@n)=XZ__d$VU|InZFj#hrpurn$fMviv4PzW z2EPA7rx$Z#fs?$nHdIqJrsFOx6<|7dwxe<>;2I!6aI&>&6S2W#zlrE1A%mZq0OF0f zN+6Y-Al4hHhFMEW@#Y@p!>Ue0f8Xa~VZs}%(qFviAQQeOv?FH0N&R<4xDdRmNV?N8 zpb)C`#Yd$IVlD^Jq9rB^PhCzJuO#*^2^O&g1q9JW*lbN-1CkY;Tohb3TS690$2A3~ z4z~p=+>>vLtIo!+P4|Q`@8Zw+Bc34cA%oEIEQ_l#OCIYR25^@%hdGQ9cBc^vZlHeI!*_*E$HXEmRRk>Rpp-4uh6 z2V3>i(~sH$Xv!CyuX|=pb&V2mO$velvlkmap$Q7;+=Qvym>EKTN8vl^I+Uqx?|aP2 zNMkjo{t&SH>$rm+(m-(%CBB2i9YR=GW%F&zKu0RvgEsv}1Z@|B{pO(9o62Iehlj`U zH}tO~kkbI3Ay83N3jm4FA%`_F)SVLF#6!~YU~hx;u&zM1eH_Em7*s(nrBRAWDOWJ_7Lv<^R&!%&~&> zXy;d=( zZzZ+0DOOHfJ;fo>-JGtq|GYnrA$EoUbg(7mjy-%D#YIZ9FngN$kRZ*;DSbaeY}7xj z&uJ+WQq(3Mi1(0JQJ|&>?w1)&vggF+7D)OlTrL@l>Ckn8(RI#1D6WeeDH3wH2JT;GlickaFc80C-^Exh&HQ&66WBZ zVR~I4=zUhu88e#98vmoIw zzjnIUAe2p0wq30nl*YZ2HPB@cLWWJ&M<48;p+|{&JKA|ssTAJH=u!V)^>_u)9L7`K zuAYY2q=o`qx8%&LN+kD32Q*syThFM&=YMt-`O%?|6q5!4nk&560iX!03NIs#-HwQN zLK!znTC!dhyU3i59$axD39jy!pw)heP>mM5c(Jp_L78E3$`Eg13ZUu=A(R6%MKJUq zH#RJ*FhoU*3GI?PlYo>~3n$$X>HqI1S%q3VfKCaEm^00?Z4^}8eXh5PjeJ>1U*J(J z7oIfZ_xSX=o6bWgnx+@z3Awu#?=ePi{j_MQUEp~h>_y4aEQVi9pjdBo$W?b2uKvp#BIvngNo5Np}YMx-vzDvZM|J_Eg+w`mnumsQ|L7AR&8U&?CWrX;R74=DSj>;!9 zdgkV{xw}UxCYE^VQKbz8wQ3fS^^Q33^#yh11D<5MXp}Kt8dw#>J~1ewx)_m6`MpK6DTIRPnJWHBcs-81{G2!Y8o-+y0>Fy4$iqS7oNK>K z2gS-!zxVU$$J^Wvq$xcY32jH2T=Dvt;0F2gAFx7UNSylxikM;tpbUW5_O6}<2oca{ zG2VKc(0td<8c2`7HxL*7T6KlW0YO*qzxb_41ta*QjTgmUl^!f%?svBwfK_M!ofoXZ zb6#-`LZV|f5X9j=(d&|t{ybCjPt6tuCt_tAgx=X%S#=lnuI>aDE5@^eKaHUi$U|XH z8(|>YMK7j|Ijl(Y5RHm7yLN5Lp(Il>Sk9b|RUGfhKc;~N{UcQTMS^$=5Gn@Us2j0X zdVm`<+W!gkbN}|TAWb`js=GuvDw@_Yuy<>kJcq|Ryf=v~ct*O=$HaEDBqiHE^yN%L zeJ6A;B1|o+6CgBzmR&rczkum!VBBB|TFVXJaK3sCII#j>Iah$4%O^pqv5PQ#vobKa z-_*Lf4mb#)9oR}w`gHK0NC(hk5TMwu+;l}4E6Mogq=>4IcWj-Ei-llTe zf9g1^P)6Ng1f@@;UJ=#I{)xX0mzK9ZtHOW-#72AJtVp{cn-;7-8!h20r=%We6-qG)fS+ z??K$*x5|`YTJ5kOr`BbTkpXn&E~>qshy7h8Ljhnfuo&2;#5p6>W-FRj^_!&N9|)3v zY!a|si_mFG)6+}I20At*=eXP1sG3(FxU|nuzkeq-izi6a;JHocq)C8P@1n8ZG$!uY z^D;nv2Kp;rVn3~BL0WeZk_hU{|Fi60zuJVMQJr|m)N;b5mb$vs#+yL0!{_IEJ!tZO zT|&;|@t-A1ZBA6B%hI!2M6a&kaV^}{*U+PB!QgLC1}=i^cd6bs^u9b^-^V2ba_d`^Cf=?>%!e$9_D^KN|8eH|GrLKpCohD70;{4Ac1(A1TmhVRT2yMP`dpH9Iq zCi*{@_%`Wg^&6l>O&PkiF3PeGj%#{Eq#SDS%-AiZAbkJX?@B0x`F>KOSE=ltd+Lew zmU>b*Qc%Nf7_glDVTqkr;~&9RjHJIRF1FUuI0lD4Coeg?18~SbNdy;KD%^of0@#4@ zVU{JJSLzwW^UUG*AnMmaSEpq?z_3+;<|9S)TOg+&^i!L&n`d_%bX*Xr4{IGkybuJ9D%+!uO9BDIe)4cDuA_Z#9+CP5vp_#eKO`%vBdps77!NDL z3?$0KuKDuqRbTo}aE-bjn*X!kiA4kkUpK8=l$i$%yXi%MJLWr}7x5SG#sH_`23ZX3 zO|uG?DqsGhJ4^jptD_R_Uk|b)fL$ouR5$xp=_urEQtuGyoI8M46~i? z4?yLtez3+FIG8>ihx|d9P{koopivKQc#r94?>Cyi>d3{;{`Y-w1Xj_&9qm)?SA-HW z1f5zgi|*?Mex*c>^^oKdt>+Z+Y{5?dUheHTrgCXx*?{3v202uSK8QX3oDbMCs$bN0 zYeAoRfIv_gH;|~Xt1{WEx>iQ~#0CwZI?G?WPn=F3<(hv6JlnxXm*x>=yy?bxfr8)7 zqAiqYPn^|lQn>}o#;J7_Y~7X;m;Gcb(0BjycI`Z!ge)7_E~*7H z`W9Z`BnKY!dL!74=bmC>CJgb2)+At5Bx4nYr>)t3EVUvR>-V6Wvbn%ABCTEF!Obb|Zh6niu3X56Z27Bs!NNoYLopp_4=t z&vstY$_nC6)?otk5zISxNx3ChHP201RSev(tTsQf-1^yKVs%?gw&;8#fKmXA#S4J% z`0^p-{Mfs1{}CJ}fyWSle9ny^wruB{V)}bmFqu`wstI$oX;lMwxsMPWJJ;#h!rM!w z>MfD**52V0bIz?!>#m~%iQ#Idi1#piz&44p2tCQ(9t%ZQTT$v7-vh)F4D$HL-7`ur zV?G*pmZ0N)q4NBIfOI^AXknC?*}n)@3Z}%HV;ao%qvQOpE1-PrF!XTx^=n1u$(e(b z>RYRN_YL14sX?dhC`?bYXOi+lX=vDS2o=L|NxhfKtQ(AJAEh*g`ZTspbdImYZ>@Z| zabs?Y?yT^sZbBb!{$j~o+y&@43Qt01hbhO?(-ZTKtJuigrupFJ($bRB=1W8$vdoMC z>H6oBk_xloAM(R5pg*;}ZERO5o?c09gHy0Rc+=$j`86$EhO=bfny0d*yqhg0Z|y8> z@`v(m3I2$0T}!Gkb!!Op;|(w4CM&V*!0&VI*Y%Wl$uyq_`o$W06P4cES;)N!6li+e zV(+s#+!bv$flwueLOEFpVd@Hh{=}xjtTW|7cQf&2VyEX%Fhrq17$al_(dEIqQ}MpC zrc5DOcC#0Ad8V{a+tK8oU@8C`I8S&nI~@u4E$Z$bPB%kbKva#LEV3^gS4780Lpa@ zcVLd_W-JG6n{GoE+3AdlEjnjDd}ZJoA%bwpaHdZ51^7&KDWl4SlTtYdVK@~6df|9A zI8$k|8cWc6?VAgJHlJP>7#sy!;9?s<2XX1_WcVtL?^88{`6jx+{eG%I2J`Q0KyCBP zr9HwS8#uo|&I&QS!ZkpM8+bMfo| zdjvLIuaS23jB{}Sod6;1qB4G(Pz?G>@+GEs)+$$HY*ZKv!AkW}Gys$ckc5i|z`ZI< zM>A=+nxc0Nu|lW#$vUlRPHXkkB2Ikvzul2QSU9Q@y)NF0t%d&2HvTY^kMdXm`PJ*g zYac=@4;2L`pMUMfoF1cQ3k~b@GglR|W$3xC)Ije@nb{k@nNM{+j-YV7vJjImw-m)u zI7%j!o`RXOmfuJTh0d^2aCEI10VLw1nngruwJVwh2-$1U}Mj&jRvyqpI)IZ23MY&T37YyG1E9L@l1{ zE?5z<+U3de(eImh`q;p%49Ypeh+XmwIwYZJcicvqz+Kkkx0R_9+wMRqDyB@HlA6Hn z(qFT|8(wv_ff<6nXFk^{p@T96OA<#@&)=Lnc+IB0Gz?T4i)7#P6h&W2xkNZGW=-Ytm1GjLL-DGF<(jLjfayi~krd%pG_ z3U)D0rW?acDGOCJ6d*>#BT^0yA`TrTsPDUm&Eb0fr_G4&Z%Bp-`8e%g<$Gn_*z}Eb zb&)GRt`-k?0bQDt?;a=L!AbFZeBrI|Z-@HVPf>*htCMe5`&68Eq#;f^L9`C+5~F2aYTTJwBCiQVd#s|k359DJ)FGm6 za{ndhxvnncj(}IGwKjp566J-1SD}T?0bmzf#DruuJz-gqR~_3`n6LllIQ(61l_qGLoDJ?`d9q9es|e~uyL)>-jhd%D?Aze zI1hygMLI`Pf$XRkT}ESLDhpcx1IkzQkjkCwu&eGa-`%*e2@)3p>^^7PsaUD_eN&|3 zP^NbgW-932PTe(#1sg>>ZFS|O5!=IEh6iEZ6NY}qDy2JIVI~(|^fCIrY$_(RBAXW_ zw%$t1Q}0aTs?i0~_w4&Auzz_EgUfiwdfhrG z!7AWWK?BJm#dn~DEuK;t*-FatO=KPOV)qt5qYKx40;?^6Nj!U!1hB>!qaItq9}mEl zA+5fUq$R9jD>(eckcmxT!3d1m9aQ3E#Rd?M=*9a*h<#6GpA?e+IkxRGbfyE~M+suc zI<<k26I^f{cn|5z<2G;oV2>oR9E^@x@eD^GHpa!mnuII?u(U;zuY1?P$?y+3b+?rNom$yL-&_7_fV|v|ztt92uVritv`G*DSm9ee)di^5xZfOA}?TKI|G3{AeHHr`+_7Zgp-O zNEv%VxH!`xBD zYZF9HVbN3}mAm=y~@g7HLnwE&S&dmO#PE z&r16FwwUPiR}|ffY#h484h_jD_2gl8?m3L@;qCm+^*Vu1cGn%Mh-%By%d>ofqg3v+ z-lmt1)Js+@KlP9RX6l3WD-T_I!lU;@?)(eGsq0K z6*bYWB5O7@>0J)&v+tU@Ze`#@NTfV=on$-6?AiHsU9H) zKw)J3UrTGU-EFj6jDUD6*^-=c)H|7t23zrDWQ&<%Y>Y1J>wBRN`O6-zD;|Ph;-HU| z)UEc@1_ZloW6hRGR3g+r%(pOkKmFx5mnr;`9jCqrM8tKxw*AZHe^d%3&-LW~?9112 zAO00({wl`V)Y&0wam51phQOM&IL9Nlrs$6R{X6o16D7Fcu{>AZB_ zQm&GyOTI~4ov|TL=y1n+!6QuDyb9BWi#xR;AL7DGNNmj)=jbUenCNR^&{f%_ek~p> zG6d_1-zk@%s|Oip5uOr?M-Td>wKNe{+$v#WUa7OfkRb@I3OZ`E6lLFD><=_* z(1b_psd@wt!%NzI%hj*I+g;XsF=JMB`0nEQ$8!j$V>Kc08XXy4)oR7M_%KWn>HgybJQaF=ov3_Q;I@cJf-oB48lYnWbwt@2c$YZ_sRAGx4s#PVE z=jZ+k`NfGh^R(3-Vx2&N(n{z92bu3-=dGK^6-L|62qiZq6Zk}`GF~hXfdKc z+Su{#hpAu>2ch<=(j+2*BJne5dRX+hICr z;3!Sq*x$KJr9jZZK0N+OdG|8;n1+3TNP*ZR5(vtc#i;AlqW^HYKNdat#Z$ScL3) zh3@fwvawUw!1)N$9DCllhlsWlFz&0td&C*ood0|lUxZw6Q4bS$)y;N%l=dqSR3L@2 zUt9tsi;QIH?`l%^Es4n6f#IjRdZNt$FBzM$rKm(@_RJ0@Zx>}(wjE7RI?b#jzwPnL z+VzzWO?pO5X+}IWq%YNp+Tcp4)CEszW%3T>vm5U06x*AafeLiYxQUO`nlU$=EuHFlUbe% zu-hI)@_LHNuaq4Gf?wE4e?!KDY)bKTe{Sx|o>o;xuR%I~k4Da6g?_j-ZdEWDz-6ra z2oLXw-ICTy{oC`qQMG-JE%<@srF)#nmcW*TI>jJ*Km3xZ9vi6}+#mPUqP=DzS(N}G z|6Wsz-=<)_>Vc!9*z1g53vP!{f?LFF)To3xFfIcxtOtlPW7*$p7_O&XNmRH+9JYV& zK>O{E(Zx&eYTg$f-I{NeDv7Zftg5vGmm4`@a^*0*#-VYhZ50&97ye(+V8^uS$kFhFF8Qd@R3c_WpxTy9e;@2$( z*MFJmxYmLiQ*g586w8<6GcsQrykERGd+^r)FXo+%R17K3;p4pSmAYf2Y znhVFz?5KOA#UuD5I2cK1(4$z~i>ky&%I8wdVwkGiwC#@PFqk!Bq!*>U&6Fn^kk_ld zmQ`&VuDn&LY0u(9zR*Odr)@OVNYYuM%Bt%#XzzzYRk_<(S>vB*HMa{17P5R^P0W2_ z+}@zAw(dQyB4O!s;_;Q|_Wa!+x7VKOzSuYQ=(ZJnD$T-KY}}~HUcY-}tZ^3i@V0$r z?rAzM$Nh|`u@vK2U!)K|B&CDQHy+DKxVEr$Rd)Q4e#laBO|y>T?=TxF1l{g;2knHy zp`Tw*dP9$*%A&+0PBDa-QuMK7hwh!#Ho`sWMbI#9BLexPNQae~C-8S^x-(?J6RY{k zoN6HFGFoQ^rTpxP`@NL3woX|7{yn~fCMVYOxQ`=pA_{0)!}U8D&IcNQc^4|^6QSZezX*k-|PBt za(Gz0+ubal8~I`*>dks%e|CYx=HolkDIwFD7^6>24xy*J*{W6+%N(SK5N7P_R`?G* zOWE+_WuVCMMAxO}Wye`L!C+JzCa+_R(K)K?@7xl*1*2*Gbm0m_sJ33xZhq|s3A_d*H+@KkAdBSNC~26I|GO8eg@=%K2-2V8l^FChgM~mwmMJnD;pdy zv4|Il-)Cr16?cmv&Zko4e9d<|nJjS0MvJ-LZs#cTjMX$g36i(vPjQ3p5 z(BK#E-?TRD(3{NP{C2AKHtX&7o3l3c^yd%R-?o>qedGPhC<@D%8`CpEZFg zPp>VhsuIbJ;5J`U&D`0Zk-`dDY#Z@R8){EU5+?V(xGkcrSz8Y$G4X?2B{0uMz`VSe zjBIz^PD6DXxJhNKx!Alfde8>vwRw5cc}i!XYVXZk|F=>mSv${)TFPK2$gavKEsN#P zp@q0Me3BbSF1y)wZS8ykzPVJZ?ow;WN5YmWz4cT>_ib{@ZMKI&n|(}Q-=&WR_U(i) z9VeXpv=dg<9`NC#EWItEk8ujzI@eg*F#5}1M-a=}WlXzAuHTZ}@8kKp-C)5eo$%Z4 zKViLUn6LQ>J=n?_^9g`B=}a`{K?mn?4C;Bx9;(;Y)kcGCQ6j?f&O9Qd!xv1QG6 z8=iL2e!tT9vp@S>%^A&5v{3vYIKLoIs&vaomCA$Y!+wv(!w4a>50f^Ht{wJF8#AHI z-&f=woikfIC`B04YxLg>h0?KwDl3Q73#(N`HK;rJUb$4Kz8?ARiYUXR@=bp6IBY;5 zlV)XSJ@dE8o#3062a(@a{jZbOEhm>9dM7abY@g9X2DZ`mt93pLeOOV^i7`FQcko;|OQMyiZU3;(9*gAaLsO}4P zHcCQ`a9k@W@umR+!d)bWbBWS#+CB}HpFSJPkl>JGA$}PmM3K(LCFq$IF!kh7>-LTm z-J>bPM=f8*0>3=(VbhgdfyrEYESw)><9n~>9Eg-CAtfE^jZx=Z>R3gOe~DM*5?*-13f#`txfqR=X@Rgt^f68eC-(qh^9;&k|b#-{P;? z9AilHdc$nh&+}Sewy$Z`BLA{jPGlEUaV+MvopgA`T((&N=uXPEkP(XAR7DFWGHDb-b&mzd-)k#6!kdZDcm}k6I@!6pmmc}= zWvIyEUVw}~EDepCO?T?5L=grwfcRFBuyB-JfPb{|jNrqB4Lx<2-3+>+iSNwL;kpPp z--wbyyRYjxTxYWUJ2RLc!dhO#6U zwf7Cs+8k0m+*VMX$m3_5qc5&78ZZf5$lSQtQMg5J>xwF5_PJU}@p-)A!>0(Bb;tNN zZGIcC6h-ppkKM7sHH&IzEhM@sBigQkO6Mjo&fj1<)Dwa#)9OiiKThj=M7cqL-4LnrJ}80Ciz#q zhmVfZ&tVPQHKW7xMVzIrP3kzb`az7<6klbJaD!hGZ5iAzE!4j=J#8dRS4X3Y$n)7w zt9esVP2V4LC9w@IU~`#P?n~F1*1d+K_-4Bft~YbRgR4I!-e{Sj$}F=}xq%Pmp@=v4 zZ458BYRw)S*S~hM7T=Om8CP>pYG4zVd7qhyg`dret9*_Mc*r=Rst42RofS)Tue$~^ z*r>Y(Tm>s0{4c!umIX?&i!X4crM#xAs%Z6^Z%V z-W%*&a>u1jyb(rRAH2XVI08k~(~8bdUa&z+W{ln|P?PbYeK4pF?M?51TOCV%E!}EN z(lp9nKE-{pJn30VslVa8vgLD3U*jk~N&h=z!2hgKV3ps9;dv$W0`{z{`*p=@=j2qT z*l_ZUjfw7bC&5*wj-IIW7@V1X^y~JdoUy4y*V$p!y+@nm%OnvYQx9j+Gp*3dgvpkwr4`{$Ru-Ls`1WVWcb??Fzl}DwSIrY zT85o{-fCo95Wme8b1S)y2RY6e;7{ifcd`n0R;sy`*W9U4JxP%lu$a{(8V{n-@o3l- zJB07V)a;@6oENoq9 z0vRywN@4vhy|Rok!TFH11R*Z3qy@t5)07RLE1Eq$iX2U~=DC88Y{TBa?YWQSpdxZ27(u7aUe3 zJrqr)bA;zRok-+}C}o2S8pXA()wF7dKtxZZE@55=NSv1k~Yob0~ znXh`)`V|8ZMFST5ZU*{Y?7AC+*Mv(_?R9?D9e&^T5i;yM4~e1KX0LQ1)5UiKjisNU z-Vu}o+)LB>$cP?W?Tu)ujRo>&Szl$#;bJd-04x|T^(fKI`tz<1^456CRWMy}M z!)|2(`kgd;<3;kmOYZ31h*mKBYw@wJeGuEXcOQeg^IME&GOZ-L;EneNNZmwImP`J9 zN_=7zw)!j3RpE7x*(#!TL*J>VSkepG9Guk?s)!4skkm8qb_4@^{8HyS?K+3^2i%k; z#rcd{x%AS=+PiP7wPOf0r%H@U+c(uKM>LFVxra*;;`h@gR5fy_p%ihsnN;U{yw;TU zDBV%3jpvJsw-UN|aFjNk<=?|=C$J3-QOX?pXd2{ za$Bfw-E05IJ}K>N{VHm6p#YpsiB1BleGwU{sGS%3nBY_|@N(CQGiILBRQL_(V-<6l zjb+S8zt9kB=R65}@^doSiKfF@%^3^aj1p9_1>-lqclp#u;|r|0skzsSl%aVy=*0i(8hqN#IgbL0;oPr$Ur*DZ?w9L^C6YzvFK zQnt%O+elfJ{^`0`7~LTD?<8eYpyq^I6pql~EMv!gj+2bBb7^$1z9o$)i5z@qU#oJN zH5m!@O-62bS6#de5*t`aDjy-tRqf(Bn9K&j93rB5ZFV_q%_pFJ+dAy>VzkyIHCs$# zFC(;9#`j+uT%LYq%cJkD%l{9T3k=w~iMOa#LN$5Ot*f#SX4V2-O;%7om18U6@Dp|+^e$55Zjs!&lP#Ut(4v&l zlm<`MMI9+-GtQBpPWiINE4KeS%WX9jACFoxQy)lN8@9ZKD&s;+c{18Aj#DaAF8A$K zmrKkfR?t>3Pu2=xZg;a)1|J<5wl~!G@V&A(7O`iQ26s&W#$x zN?u$PP|bF7w-59^lYDHmzW7}`j?((@Xr|VMkGV82A;dPFg&&P7b81f&!TNJjxZ5lX z*!WP`7SoLRTuJ1pcm+~80yY+&&siW}$o|&{@uTFO4qT76 zP=pi_b(q7Jic@K%P)slUJWt!+Pavs!wGTuXTbUzUGeZjBXgVAg*k_An76&F)TcD@g-9P#swB@He9R~Iz)s6*_6>Se5dYp z6y*+yUe)NTWN3&2%k-9=0(-XS$GQrPLz2zAFP^W&sIbH36G)X7HXY?Yo8B)`djBdg z=ifdT?5BYm!Ov5%B~$A3WS9Y#n}A}O%W>i7j2%xl%ac`yY+XqDxh)vS1^)U{O?2ju zPz*Y=MlKDydWWDR?K%&0i*22_`7`d1fV6h;>M}(~w`*+VUwy3KgCOi8%$2%}&aew{ z2e%mhsV73jv(EcYtZ7<$uoHlFhJ9xUiTduk{LJ@0SEBbb)+NU1l0lcph)Hg@VJ_@6 z55yW|p6Ol-l znK43I2qFNK{sTxga$KNS+dXYMS3W)zT(@HEs{Ec22y(Xz*r)BiX8u9fi=SdbWtT=_5=Xy2%@U4^jQ_ z)qwZuehs6X8-sFJ|ND8^X;3`66Yk%oNZUcvk%*y#^A%qvw%u{HizE=|x{iD$wbC)& z!r=OrhO%ve!96x?cg~~rsIYid)WE2#GNUnBTp4nor|siuepE|fF#KN5&kK|iC!i-9 zN#hvW**C90`DZioADb8q#W)D!c*5#0(<&V|g2<|}xDxrflTz&*mS3E*S+n(3U6ark zDFRa<)Z=|&u=?T@|1DkdM1o;H2~JZ#q`&K#RbNK~C{4!$c9%w~bS22H=yfs7l%!kF z3&B?mmnJdc&Pdg}m9yv6ho6iJFTWsv7ub=?RME1$cIAg#lg-Xq+LAT4KT|B$N>GR2 zs2XEFBP4|{7ILxuc3oo|+1n1u)U5=kE>;$^;!vbA!Fqz;O!9hit`*GK4GB7z4*+Na zW>}G1i}`LIL7%2!rl5;~&aeR*!qbvSD4+Ruj}w9;UGm|^j57D<7=q}Jp3rlMBHB~& zDyPkwz1C+kC*S>;{dwBXJME7%WvvbY>6z$>m3NQW{PT@WwGHo;XkBSN4Oev^2T-H^ zGd8SNX3d4D?kO8q!5>BCA+mYzKdaG>K_;ClYNIwr*91?4ZP)=6N8BuXzIV*4TJmVvW{Abf)=Z|0T1%odxL+^s- zS0i54qkkD}x;MmR?nakby_NU&OCFZmVLk8o_SL`7D7KNS(pR(UE?GN6_`V>*8<&Tm zBd2?=Us+%2qSO1(=d}j3uw%Sq9-zO5HA>wa-Vn3)bcYGfCB{6v|3W&(=KlMy(_Bay zD5kvI1F?{V|j))77Tv_D=zP;((ZaAWU%R;ziv&0F8hivEH5=B>cz*LTUa zSbvrKe_efTb5w03FR2!(QO|SW;d#KJkTyG{!voNJk!R^>9(`E`G#5jLif$tse;Jo?@zOlK2&S&wpcr@SBS z&ZV&&Y?By$`?Jih5Z#vD4B*>6UqS{_>dp64N?VeiGFi$2tMA?K&}=aOUsHWYgMcq8 zqXYd~&$YCAYB)FowF1gJ+;`X+9FpMd(10l*5P-6zj3tJMPHQtwh(Y|wkVc4Fj5yub zrg-zLbs2r~QZ_=Ewix#F)7I<*(d_FFjYm!`dzydi;mdEY8Yp6(aU0Q!`+s>IzV)HVBqfLlcqs$ zd$5T6YqK|D<4Ocvg4_p;J&S3_#!)uwJ?gkLgS>l$!cAM0W-HaH^06;}e#@z+`Qy25 zc`%BA9}~lVzV6MSp^z>sXmaI$F9cI(B}OtLn2rDhCPK5u=aW8MCf$Arh>;kT8V}~1 zhq>sWA9_x{wC|Z!fW%R?O{v5AWht|sROk)m0a&OwKWYb@C3}c=%d0}v+jjO5cF<1C zG^*I`_`Db9HrH@(3O5)Y=TKKU_`@xp`ogsnd)>h44j!On^XnZHj-Z?5GJXkntLI8R zTBXPSXKKJMud;_~`n&=t1;EFcvO^|VbF$^K#$!-+jmiEcCRK+cs&<#4m(JZPY!WhV z>4yUOH7Z5zQ=;ua_6ttJONPw#I2x0E_GOvGZcDcD;f;i;=qMm|7=aMi8jBuPm|M8|lh{U$wOIw5ei_0+$qM1=RwZLQ z;1!*S^C`EaC`0c;7H1BbuWB~&1tI(9bO$RKlMw)LUSOZSco>)gS+Nb{Rf$?CBbhZH9< zXC;KN0JQOVr3Eu32e^5%4%4tXLB}MfaTbtR)C_J7lt#!Y%0CQKCe!+@LB#?$3mfA zg4mz?`_G4_>q1^6Azq#~DMM4{6%@t8?&fER!#^9NV+t#djT=aR4Rpk(7x(eVkK#;= z;v`>mR44|Y4FEi>pdzbJnO8Eh`z_Mv&RESSI@QolPcv1xS$_D&zstpcI7q*X*u>5u z$z>c!S19S@GZvyLBRR!=Eio6NayYMdQT|i0%x^W{a8^nZniCN-Xv({03h5G7`@J6+ zRRV$JnP?`E87&ek(>p#yL?=^F->MaNzhg2EfNoBY2&p2QdslzEZ2wAK6(C12>mwdd zj3&k}`pR2#OnQF1Y8XO%qB`eWtD?JG5<%W!bQMG0mAUo@vMKhLlnE|9_ds7UxMDaEMn5{dw*)uY+nba>*^;Ai0MQw#i9$wKaX^D`3*` zud?u!BgCnMMS;&Jr;I{GH=vEUscy`G2w1N%>>gRCx0HiYLCCUzGpCQk|GUi{6m4tm zD;i-CO%2?>7$SM@_AP-sTovYxR06WA9BO*x-sK;p$dwKOIFXdlJ#&&fB*SQ7%C+M& zX$fc4CnHzBHTtk+56R4KxMY`XM-gpDgrmIRvURn9>i<7xpF2l zR6(0o^|X)G*ZYJyr@ypcz@6~EHc=)MShdq@q2~zugszi0oAAK7$p&)AXO@5$iW3)ed-&ClTIb? z+71T%^QEnhxrSd!X@OV*n@5S{Cxrvh!Yt9=sn8p8o_-reFqI z2IZz}#8o#w3`tAs9NwT%rIPZ`JT;Xk+on&j z{PIz9Gr;F|dXNKjCm%;!g1i{BBQg2QG(N&h<#gAEzIrr-`vOr_QeN5czfzIk&t^Ie zq2PjGFH9W+)i48UxS)m^)mXS~{W*g1PVeH1&DpA~HB3gBJy7$%h|RpIe@!QiFgG~- zbi@bq$!byv+BhQYPZ%la*gjsr|780x@wwFAm6amThkQMBPwZ_Z&X*A{&klVeaTEvD zz2E)8pKcel52qZ_$T!Hls_8>AlJ^k$k@q9z)}9R>k&rgR&3%bs?0A#8=T2l3eb7nlPZ;U<(X@r@ps#dJy+zd*soGi+Wh~N zSrQe@py_>g(v{RPoh(Zt8<=!=e)8w;xsy`)2!%D2Ga|V20@3aOPs> z=dZR_QqgC%rR$&hM#_L|&3f@H#wP2UN_D)huU zcOM75M$$B0XnA1?<0AU^V7L^8KgvL( zPj=ImASsqFk0l-g{A0S;$CFzxD|OC~bfnM1QE;BR8TPr|&9sqi zf(-70q!?-2jEhx7mOT!BYj{)n?meX<}@Glo3o+B|rAIL5+Nvq0A^BKCrJ zZ6GoCRqgtMQIUn!&zUy}WLN(xkdNE>KI<|6Qd6f+6Oc{2b9B#t%-;a8FVhUXfEbX| zvq@>`hw(g2oQ>4JsCf)^e7$S(32 zMn*k|BAE!cb{QuWpsQuqWX^pnn33+VceA9*sq+-B!C-(*!o-=^V3>j#uH?=~!9M)S zkhCMlMoj)Z9T%2?hk}x9zpr#s=QQO)*4={I@OSXHG(+J;&$IgaN(i5L(yW>IJl}+O z809L$L_>N-LiYQCczLEn%etG#=fI~Ml}ah}R(UJYL|gYC9p<~Rch=1FU(aO$n|HlS z%j)1Y2srG${XCbLr~E%;-HxInh23GIF_B5ya446C^f>>1Sgliutv9R{L|L{e#VA7r zU#nxN-#jX&l}D*V3~>hoRNk`}(g{xG>9u=CVtly0d(RHZG6gzaV4MPBNGu~lmz{KV zjt+8LEUqf%O64bM(-zPhz6`+;FWjPNwzK!=3yLxGB+V;_ea?&*VeIt=HK0*m`nV)_5-3BN)dYyiyK zvxkthlx)=xw+%Y%k_R92%#d7X-H8|xTTBcb@tjCsuf@|6gD*}IhY81uIYpesjUJA) zdD<>Kjuk@dw8L;}H?*~82-?@b>cd?gH997@8B#5rxffs({GQ`#7u+(Xwr25(N~p4` zTg}29@u2bnDN_h93tl`~Zd%&!yD)`<@a-O%ccbAC&}aVb_&#cdI(1a8S-*REao9>* zzgLg?+^BkxZ0<_AMKj%{#|&eq54LY{Kof26T^bE*TF9Q7qg`Cg&3JiVV zeceU{dab!gkY&T!DS|By8OdJNY_>?A6Dt2?rEmUbpXh^y*iN4KK|8l3CuueNBHi8A z(OkJMptp<%snOJq>Bz_}!gPrp8CRdOQ~n=SXB`z~*R^qCK#;B>r8{KkZWuZxC5J9) zq)TAvZs|@5kra^b1{D#IE~Q%#zMCiB_x;wIKWDA`tkY-jef_SzE*ny7I;nl;ET8&m z$_n%|{@#~9uhnx8sKyfDOGtt7 z)INsE;D`mhh~IDS97WEf7eZS`^7tiXy)Y&G%7m8#4fBvHV^+(;v+dXM<|AdU!gqo< z%2Z<6>ei#>S=v#&bDReNbp~Sq-7=cbiC(FI5n=7=smj)#QWPwjV_91pIfP?$O1W6!4#K89enoGaLm_AQ!U2c0Rml-%RcL*8KbhSt=j zl+=!CG15}Rusdz|U1-ocxKok2{?xMs$tbsi-*fVNoku=X2rMsYSah~HV=Mc=H)~)E z_=UInaT|ac_G%^E_wVil`XaS2aOii)H20|8CWTMDB_+0djS+va28L2u z)pL0NuI(}ByT7OlQk8sVJM z!O^>wt1ja29!DWDPB#ObfhV8q0-N-?Bkp>r?s~=mkUs8*hHKN=-@aPq!VJs?_bJpX<#U+WcR$k~)c{=q;@ zWZb>H#P6A}0MK#0!Lp!N-DmVgGT`W8gI|xr%126O&Y|=D zwzCFzH<5rnX3YkB(7AjNC_VY9^qR`=fj=6K!$3Xf<)??+PkP3lIt@-Gb^~lCC(#Io z9kjpRKoNvCVtMlJ&&xx9669ZlrZ8%#bwyd;Z5OUe6Jk8{N!-aCh5@r*Kwx`fFzzv! zBbmnX!;KUD_ozP^2YoY<`=u(~2P2OexgUQHy_Ee}&cX=Pn2nKG`@~RFw5<2LlWA2C z5Bv;INw^zdI&TYB9nlPk2w_mM^pxHW3Uq0<&)F`sncgUMgX=H`-sfUm{W%H>`dppVbWE`^1Ok0KVvw%TJ^Dy4J-l`L3{UViSW(bUCoA>xA@e3TcV;?cvJ`&rGh3?*u&7d@{Sz4C?+O zRM-A+i;}GvK~@=Dg{~Sq>%B4GbsfMmFB*mw(xm?wS^rJ(q1-L3kHoecH?-`@i+)f- z=A>&yekDe|zC|Lyq2I|_F#@V(=YZx3soXG=%5AvSsb*RDd<(j4%x7qr62A@msP7C+ z)*k2`FwZ*$$ob6)BLPkG=h)36#(AdxdbZlubGyQG&#knIv8ODJL=2tgBce)=)bka<36jdC8*w+j zGK}aou0cu)USUS++wh`vjU6q3AF--nHH#g6?j zEH929k$PX$5&+P0{Q+&@#0J$Ty;97YqNHPDqnVvzvL*5pqk6m^c6dMMUX*XLOX~ePX)ybf!NRupkzbT7btv{!Dl8xU?TvUf8?<6Pdn=`9QraOd1Ja zKM}zme7ozzo@ud)?TfxaAzuGW{lh_=bhyU-T)BVlocYup9MfN7dYl8}4-SPX5MjUjd{dQdVbo{aVm#2ioi$h}sXB%Ezc(kNeu z=cAH6zhFsOb1sn~AnVe{#`Hlz0_e#PBdi$txYOJ1#fE1Kr}G7{RR4Lcxu=LkL)5+I z{cEoI8&BZ&Xzi^@%wh4*vhr`nZPu}(st6|vKVLh~Bw>AO=$F&f&pbZbzq<;tF+h^Xm>% zI=z-9|2DvbZ{o4UWU0pRYcoY1s#k_BiUb(RP(~nwe6q;}`+3k@Hsnw$c*0#~SU|Qn z3+Yt?o4KO(c_f_7u}>TSgV!spA{E@5cg) z4dV)&zR%jeX^qB>-whL18?(A!+l1xxN%)pFVtyKBOSZ?!|B>^*Z{S@*#Bp~2sngHF zreh%E{^{;(+|{3%3B+BzBH_cM^U}c|F+Soplhx2R5ej~j22>5)B&K!qVXs|rEEen* zjbZQKQM$Yys7G{J0tCB`3_uwPmx1nWp(Q@2<9iAd;7v9>CYcdw`Wo}kT@8+p!ofF4 ze*{;CuKb!|cF?MGGn;y*&i5>?d-J-4b;x?T2nsOqHrBnqV4kqmRNAiwKYzoH_@tNUeTJxlDa}%m_RMP# z&t}TnvtA{gQ5`|t)91Mn%%sD=N|7Fq^uB>;-gPXo>*=Y)^VKP$4nWjYoHM5`A-3GC zL2$D#=>;PeyT|R*FoLr;dpzSBBB;gf4GI4w+Ljt?GYdRgHW!9jdy$>==qv0M#+aFD z_D+<0FGkIiTJ@!kqPmQCX?J^QAd<*Fu|?pQtpukutiMN2a0@^BpTwAMLbvp51@%um zG|5tNN8G51o)TO^1<^GTSwBrH-a19tkESnaM)4z^=Edn7c~;4GctHCZ;58w zr$8`+C%e?WWX+3J@0vXmtuZNh$@y@~$pd0OpT?HB?!md#wYwH+yIMdrTt{a>j$0q1 zA$pt9vB_{39O2B2c{9Q~^PE2g%4MLplzwMFAV+o2$fDz%`7?4eT`xkOg(}g!DP{os zAvX4Wmq?vUi%i$Z_fUtD`968OOX-@9$jbVEUkoZ?2+MlnpzHX|nivw(-?3raJpE=- z1Z`MkVl0ygc;i&1?o|>QkR}*cB*IH6Ok|WxidelD%e5zU_m>;QFhH@|?MM*$RdVKw zgqB}lXR#dv`UKhLZxS!!M)G-rwCOG!(7|AVuhiP1x<;pkr3*y2h-;4*Nbm((eGfNI|5X}nCQAd{HLv@O|3By4|O8Vd!M=&ei zohF`}NR(E!>s%qnk{V)_TSMyO{iuQ2xAX<|>`PQXdU6^pnT%36g!9WgJvv3>@=T5+ zPL<62pQ=&Ex&LX{+UCk+|R6mo{m#IdHc>Mg^=y)iB238&S`xE*ye6T!? zE0-UsU7l6?I)h|4-M)2$%ORI_XbcD1cFAOyf; z;HshOsKN~%R>XgHt6RAJ2f_w|CwT@dd@))+pyRLF4Cq2*@G5}I(T{@|FYAmsl&35h@r zHO|XnK#U6Z92hBe(PRyW$CCT7W%q(rl+=_s~GSt=-b>V+uCy zs6lhg3GZv;D4bX_Uf6BqWlf15ixXIh{PKpM6d#)yAw`r;tlI z+a8y0HBrc169%W`Z#{Pj|N7F2CaEi+Q;w+dqAW>z!(5X+&%G2g6>cW8lLnd|rkLW+ zQs!2nh}4jUDAL;phK-EVk-nt6iCKJ=7tZ5W8k}WF)}8_*!9nY_uiKtDLMFIy&Cpt- zC)FACqjOMz?+zi;+vC6qFbneHFQabAR|^76<6W`A#+2 z+A%yqIi(X?`m{{j#S#Plr~&^lc?Ca>k(osaBQZrY%gSG{?X=R;866?EafTxr&c+p& zyxyc?&|ztw3+*rDC};1TjwMNt;LYe>RFjW5ZEK0KSIxtlDQ)Ss)w7ioY9q`-7kknG z=^tuP%oOKR1}|hQ0OiqDuX+p_2Hr7;yr2$CEK9ftsxqA=8?>`^A&Q>?Uyik5XpS&= z*54YkkJSLeqPCn&isr~mRnt$>`6cZki%F-7ASrBbSuk1hP1-$Cla4&7jXCG-&a-cr zSvwE+M-Og#TOcoK%&iTz-CGR}H>aH$W^vm))^t$z=;zh$yS;Ar+N%{hfMnXXMnfOy)RYgN^`=8@bveo^BRQj5i&pBwT1U8&ZP^E@qc!# zo~2UW4)X4+e@K^Y5{+$lEp%_n+)exD#u?#aR{Kai`8PY0NriOoxEC=rzacAbMsF1s z=^TNOKN9J39*vd0fPGvx^(KSHwBP@oWkEsNl)m1L|1>N>4mAcA@84kVm(|)LY5cCb zlVYHLG3jLi$T}JUF1na9Nm|^@u_VGuCBC^Q?IwwCI2_M!DYRiz%xJ5oe>V4kT4;64 z

    yeY}onU-c(T8r~syfEtn<2anwig(U#I)x{r9`i+~KvDhd2+*O%k2A>EZ>gK2cA z7zG1nw&M(FoTxk{q&ra#CNg=xDqDpV+KBZjgEM57PEx&s*%_0W_1AJmK+u&}kQcy9 zG6mk-Sy_QP<2>@^5XSdpO$+Q*6^>g_H8OFPej3?esl^^0wpz83nYhDc;;!Rx(fV{y zi_>m6FDriMz;>UO)+um*diG1Kq!)~Y3^%0guGHBxDn+WaN3)=@NWHwQMdEKCz}VXH zV2CR(6v>jT?EGx#fw6C98d87N{ZwEC^40a)PrQoP|2~PbbLBGr6GK`_a^6=%?)R?D zLk%$=OWIikg~D?fTQCv%H0;>%f&4F8LK`)w;OBoJ=}P3&F4(_BUcSL^z8Mhf(oU`Z z&+adY#6eKd2e$yb5r*pCfD7cKWAuxQ8yZ7$$UcW9<${#^*&(e{S@K!CjEFTEtvV-~ z<9~SMGLjSCqj^yMNF#X#iE|TpP2eVPUrO>N2?NGQKdlLr2{iB~`VY6ev7>SZ#yrt~ z3um{N9Dvs&@nDX|Rzi6^U$|oW4;&&Z(#e?9-dee<@yq%XHD3C^zp4cP} zQ&pE1C)xm(C8lZo*o{i6kzf*rayQ6$NSN=VK~1KRrqeOLg<5fslH_(>PC-oQ6vZO4 zIn?wjHC$=XNO@=-s2Gyxik|j)bawUxnWnQ_`APhT=!HHV1a2(T^ZW8D3`MU1^-+etvZ=Esjg~y};`8k3 z>D-(`Jo#C?2<`alW%@oN`RA$#^pxy1zMQzkwF+ImJlx~O8J~Ct+?&!(wC_o0h5V{Z zzBpTl7ZJa#)a>Lov@WQ?zgr@Tl6wN0XY0KXZn$u>P`=-UGLRoIrUkeDw+h}m0Rc)P zti52ld;1Q^F2_~9C34W_&S3~;Fux1dArj0Gnw-1Kn@qjY<1J-Vf6c}Qq6j!60KLK~W zcktRHmfub)bC&=eD?)lYn~4d%I0LRKL03iNOY|peU2;jD^;&tLVQ*CqU4gUoZ2@|r zrI{==8&NYedB!Y~6)*?d*q>GAwBjy=I+IT-pqO_^l)Y*n>l8y3mI1Oh*xp&9HMVP| z5w3t&q1lj4Aj_fHChxC$LAB1iEzSgJX%y9bCxsxNbpW_OHk>{kzui^$jF7&<+*@e#7OEDz(0#QzN9$xje^P!H!H#d4M_6K;icb>ppkAIP`vqeeXd(4(FNY#V92Z zAN=gqb@t2&V%^0MO#d~qk$@}tI)b53_WUi|zaJ#(|%XJV{(;{k#pS&>v%+fg@R22_i$|+@?x_L!HYw@- z%m&NzJS(NSC1f9R!feRu4eLD`F%Bu?M8b_(CyxZRpYnzz_R;cWdbn)s%B*k8IY)hK zLq$cJ3y~KVwJ!U_e-&z(PmpX${8?%3AW%|gjxIJXg3_K6AtXHZBz2Om@_e z=_>j82^x;NFSt6vO)3H94vnCsvmGszzrv7dt&ho=-dq+3Leh_$VFoult!4=5RqB zt)K181TEHtUjqdO>9uHXt3UwHKs0~&GViLRy#JGT(aYIg1~fP3*XLuJs=ge1OZ=`8 z9Y2kaXwM?V&uJ#d#$j@vSqGBcv=lw~ZSU!bzK=Dx*lEb!%eu)>!_ei@KQa@clq5lz z>7-Yox>_1qCkGrpnH;u(NpTTxOp~?NA}vK%lVHON0ev|i1aNlZ%Hce2osUsu;xm_> z0)lc~Rkc79=@>aFc9S~6r=FrtW=`69s?QC!ZHhz!*`mW!Q2jaF_+_;WTnvXrHy_gzBzfIMoa?Z_Vz`8H z9f4t;;~+TQJd2^HruC%?F%C=V3wpL_n>IW&4*fSuI&M1ud0Bi=3C6~mB{VKb?7xOv zdCsh?W~ObHDB$@IxM&HdziuDg{C2H@WdI5PU*%iqyB(k&D~ zO1#Igg1>lWnCqLeJEaDLnm4@C5&42E%6P&Z2#5HuKE`seOmcq{(1n7n3AJ7~zO2Sy zF=M9C@AUC`bl$7;nQ*AD9z2M-xC3H)ul1)I*f(tYE+2z+ToMGRUP$=QO}j)nCXEj3 z0uCH#Hf7Q2{*>Mcf~TT@BCSVnQx?|as%;}pvLa1K`RtLnI^WOL6{yax;98ce6(CX^ zCz|d_Udfmw^!o`iw&>|F9VsUi)jr01Bst4Y8co8H1zlmDqp=~T%afS_%{$T%&2feZ z3fu3brHI_b9AW*Q$UZ3kplGaf_9?o5^=02%5xU{^QA+R5C^ z#8qF=Pw4K(*+Y**QMn*$FZl*3skQZdm+LX+R-`0%>N-`Qt5j4_5}v`~T~!(cd&)83 zUYJCv=Qrp5$#tjkPc&QYDk4jJE}jsslPg-Z*{F4Acjuvq4cL(| z_wlOFXKr#4(Od;B#(==6{6KVkUZmy7=FL=4;2oOHeVzXn+a_l_hxsc`C|g`gygQRk zlOF+E{+;M);c&6+#KYaxgXj;uC2NIYQ9+7}SQ}9b?Mm;Q?2?weQVzoVr5$mBVC2LG zl)b1iv5^g`&y&T&Kl&eM*)Ds?%aW#|BB)-uw-=EYk>`6An<7ryzyphETYGEz1qExA z#;*RQfAKfn^LUap1r~i?!loI3h-UpK-QYX^uefh`%~J}=2VD-nI7&_M+0-gQ4c8Z8 zIeVwtFn)W^ zvL3J8+Dm)7~ z!bO@0*UY=~b|{odOwpx&TBanA9*{6$e|JLlz$Jz+{EZhEi{%Z&`fGwiU01nQflLix z8_6lmalHmA%(eDogkSVNrcCyO>EYkRh~9|mHGy0ordir3mhc~WgHb+_vT=vK?_vn? ztlWexMXT_HD$xU3mzeZDN209ZvQp@lgVxcRQs&p@8-y|Ua@lK_3On?IB|j;Lw6KKz znLnCsIx(qHSD%Q4u4jjzHTGGRGN0~1yMpQ&H|E2qb#lVkd5#I5q_^%`mf<>~z8u}? zvtMer+;-XoPZYmZpzz%-EM#_{ee~HC#Wuq07p45{A-ZmWP0GiK$On7gl35%NcSjyN z7necOLe@No~gcwK>D&&2LWK-s$Q{OwP@N(Nb4%cn_vHq2O{^981k3=A% zEg!+B6VSk`!4OejjAvcbmzvJW;iCq&BIo9th@|4bXA;~}f>X;wMIg?}>kU33`vK?C zh+v{LlXOa!$2B4 zh3H(*HNy5!wPGVkgU;5Fj}Jpc#G3dQOK5nQ>jG>&YK>g=$_`aKnb^jb9F2UU$wLw_ z8p(b4we!$(e0YErTI*psg1m>qeWFuqJ_~GarsOXPZ%*8XjV%~Sa4w37J9iC#{048C zCE2hLdH>!kw-}Q5=+UQo|KyA1HrA-)xx}zL;1$_`B8O#@QS_i>O`a(EI=YjWqvNA3 zr(XHjJywtY)+E6&Wl}vJj3FcichsL$yOFI;;jNiH_;v##KW-)y{5R#<=N1 z@-)vkq^7Z6Av8pwkixH{OJ!K=w1RS+ag}?|vxH5-k)k75ZN=rl}*Ph@h0H;$eOVh-Gb46M{}Gu>U27_dSh3QQh6ak9&^NwQqy!8XttC z<(LlH;I8bam`R73NiiNj46m@9y=6dT49Eg5M+byNBxju4){LXl2OG_W720f#`t(Kg zfHspyFWnE0DR>i}Z)&}*)VZhXtn|Rz|5f6)QDVactA`vtf`R8sHP)BuilfAfZv86D zChrE=5c7;b@aA2a^9L44Q$O+4KaTOcEgL~kOmf1{v6XL+M<`iD7|&*P{@SHOc27#P zbDUuE!@bRQ68(5?<|?ICPpJ4uuR3$977txdFZlJqM!K-UIkK{&@}fSoO^0Qr3`_Zc zpJ`@qK)X2wF4KBam@;Yw)pHd;ah7S?O=2P2b9pm?1dgrcl*tI4>kv}dWj};(Q}dlw zvG1rE)=VHf^34;%31;I<`~!PQN-rSr*C-#HzwF{&bft0LK=Qsc6AdWUGHFMeP?XI< zVq*;%JH8}&uX|yQjCV*lqY`_K{PfdZ`j`NK#^R>v{4wKq_im3!4!qjt%{JJ{Sy8PBR8}=>epya5>;cMBfT!nv9LU{9Ic3q-|CmCEqX=lS}=F3 z(m-bpiVYfA_^qG}p_eONp&Lxrg6=Hh&(y;m4!YWJ;-C{^np#V1Q9oO=g{!FGA!F?- z|LtGd1>0eqGIlXfH3XG#Or#dBpMSb@yT=dY*vNKu-nf;1XX`eztu}CcB7IGRXJZg4 z>X+4wbqeF%>>yMKca5?bWy5E%pe=_cGTW!5c3Zq5CVUrP&Chy==ovHW{!EiXDnx_O zaGld|x2$-4iwB#zRsxl#Hq~XnvK)LH*+(4n8-H%52gF^P>`sIRR1GBPW07(@ z5Ru}xwB|?gjK~am!o)?Jhtk%+Aiew$@5l_LP-d&(Xputoj#d6aOsx0R=e_q&Bqk?x zC7J9IKX{7PDd5Y=me(|EL2_aFg{^FYE)GHvt>#E|%85l|PVYD;&x&0lu3)NNb}dM~ z<6?Y@`rKSa) zTdsPA7GIA!bFN;GeUHA)kH}zv;^10<=WsZ&16vn{s}sXb?M&h+L_W2?YWp{Gl)@;P zA7O&^8b_GPUwO90uGYP`O;FCC(WOXHVEK$wI=G=;Kj-=_l);$Z;p4{!44O*d@%^LJ zbV7hL*Zr3r$=lt&7Q-a9d=6+Cwr&&HzT4s+NL%_81~?{kJ{;pbK&TTw0=2T==z)U083s&8dXZKIg4s5+0Jy z$i&na9qE&P)|%n=ynTfg%hd6xfvCAZExP@LE>V@*?Ad2ZDmSCL`a3Og2`eI2p?jJS z;Wk2t6s(Q5Ljnyh)LBn{bQ=hMqaJ*38&2MyVn^NG_ObOx1=JWvVRlvAZk-X4n6KOg zVX!zv&wTvjWxQ8MF>@~J!dgDw^NHhxl_I@^e9pU_@iUyMCZ2qViZ+uZRRS?X0A?OH zevKA*8IH(s$`<`9@(0)b>Cb+;5z}f8V3K@b6ZYtC z9-vZxn>|*zspSRzToZ4;Gf3grV%j=X4t0z~8B@cex$KuJ- zNUI`HouV3ZuOW+aEUo+}nd8o~A+h3W#xQbFP(V@fFxpRe-c-deRhp%N%`z8%Aqbo4^XHlNa+&(1^%N@LeyP#WN$XnimRf`&EH;<^ytfa50NanBo^%8%6 zcUlIA!NioTEQ;hZYs#N-U$KB6Kh|!PizO5=1s(MIbh{kzt~(Tmq9a3X|FL$c$H(pH z`^sgr{pqXp=_Jr0BLUM7Ji3QZeW^ECVP6D2i3NgwOiDcXZ?E+Q@xJXw zLuQm3&#<245b(tM(2EbKx}=4_^|deP+Pqh=2$rPTAWr6itEHDiN372st`leG=q8f= z{;SmkR8+X#yT;UT*i@mw>@=ZyW|VG?e$Q7#yb`v|dDfOmbfprR%ikiCz~v{bqbl=}gf6xv9bYlvSCv1tq`-guCC*YsBAxpRo#iM#Apmc&CV` z2$dCSls|FhNwt^poVDklk@x*8aqnHKRqX^5n;7Irg$E4wy3=+!Tvg3xur z-Or|rW!sS^?g_fo=a+(-U)8u=PmPheOE0og$Vtw4$5Ks-*QzHz`;Qr}oPL`MyadXx z*O=i}{ll@$A$3b%Kiyx%|Dnh$7h=Gri&k20OO_sF&8=NwjuA|B6~vGbU#M3Y{x1#| zQzqC6nE+Z4#`V-@YcwrPy{40{r(USWJ~bf#)5jw7Wmfq!q-(R82Kvdl{!smO!AKt- zONj6XX7^GPR}snWGxliMna}d8CJ>lj5=6f8t%IDE7&G$ESkNNTsz9c-)eSIsWjJZ2+FiK@&A*k0yVGH;EEi3XXN1mt<{%IFG3m$Px+DSpa@3eN&8cY>x)_S3qSWzzpsawTyW7nnXr2x-oJB5i^@P1d0~MS7nJ& z;HQX_NXfXU@9{#hQPz95LcYqJp1I^wb0zM>7wLbc44ELr!G-}s^MwqJ0Z78U_>(7A z+*gRR82RY0l^I==ZgWzeGw6jq!;%~R!U%ZNnG+7iEi`zNt` zoB%DYnk0g{bBMdg`#XsnIj=On@~fKO#Ja3<`+vzBXHW#O*cqNk7`q4gK9*G2e2>0+ zbgBES$a|X=4fT#r0K-y(q_>_5%?v;aw)jl1K>&a8u{fojjj)eVZskm=AO=KxjVUo9 zciWptghdqB!-@5$AVP%aC>B0SH9fwr4SQ{l8tnEDC;&*-+vZEYp6czEnu>#|WaeL! z=YDs27Bk8ExlbZbMw)!g;(J}Q4GJV%wgp_4wd|AzTG)ehS29OHsL;l9@}tnFyvG8J&I6iihvuY$$qK!G0K4wNHy`}=zZf?GSf%m+S3asiHg+e zShdJJsAnQ)Fb!jtjj5`B!F@MkN2brn7iM|w`}V)fU!j0AW#_ki|Bh_| z65`4!K>w3RuhzpU#^`Zz_n#zLGd{|(Bw7@Vh)@CA1fRE1u=IrlTQJI@ySwO=ABqjs zs~Kan(S<1DN9uDCrPiYfhDN9Jhm&Q8P^3A*_`o8TnkVZeN3*hX@S4zX|7j6mwLlkK z7|j6vHG76x&PB*g&-J0<)d^t>s&+{oxlo^7P1|q%_y10S0|Lp0ZVC4DdLlO6#9Au~ zKpBVEx2p!JMtc@@`JJ!Dre7eYi{ekq7vK^i9F3SVhUHF+<+A6{7QTJ?lz$!|o&Uvz zEK5KLC9d!Q#&it~kN(nc_T;IJD2j8^Q`Y6ouL$a|O>$zhXGJ zIM%l5#>!>+(>~uer)6Qx%wgw2xtbHLFD9wx`UeUe8b~h`sk@Hx0$FXS~5;s0JpWD4A3Wmf>N4r(LJ^f!yHtEMVz9md(eWv=RasF61Zgl6QCtBW)@*r0F1%~+kv@a783V1 zU-X?S5i|lf0`i>Ak*+3{vH$Rgbu?`G6ZDG)(kom*(i#S#ys#`R;#&g`9)D-ffqX3f zVn2S@F;eePaQxYAA0Se0z&Q+MP#c$SS38rA>;4`PpzsL5p{u)x8v>IkSO>!bjKQqb zs}I*27aB|)!krBRK%0LFiMk2#D)ewHAXruCaK`&)s^7NJ^$Dh*GfzswD|<=s1|YuM z8KsjXa&0y6G_r@&CXF}d`WT5~;h=UkU>}PYenZsxRNi@o3x-kAlFQYM6i{n*hIrH{ z=;;wK=se%`@i05w$n&R8#F4xAe~KD-rB*Y%?zv;cT&|FLUIF#R6}gg{Bx4%U=S*Dj zVGw9VD4JaSM}2~T?L|XNfXw#yMkrZe0&HYofOikHN=i_Z&u$mh!Df&ERT{=G`?Z7! zvv^0G^z++AjWdqdB*0eDr@@^7vd)nNB!DzzCpxCLF%%d`P0@Bt-%F<0IFRVM%Zh6= zzDX%J$CJfxRei5D|8Nh$<4=!K|MkU~S6^Kx=bV(rSw5RqdDM}RF=$|xlJt`J>Z4ic ztMuC9yApa5JDf}F1031MNe94A`1WC->5T*RpZt}UbmS5mOAJW0G)@3sCCXbfV ze~8mIG0S%pFKOV(cCCU8TeI#Whw=7Cx;l%$O(deBjMQVz{wz z*u@qP5jjy`**dp@B08_a8x{qcA73;@IbQw>#MDW|D69!)x;eQE5_&f1hS4p zsDr)bttuPH2I?bY+g9DXOlapLl6`vSNh(}2$=21M#XCj1^leh{P1!}{Ugl>!?ELxE zDS>_0Qy1v+`JG;_OE)^e5c6af)U4{Mc#l5LhO%Cq!fpMwqtmw>ehNr-@NgfGrA9pKC{Um9T z3ZT2O!RBdtkD6=3dT8z{i`{wfqiJ|7;ziGnc_f0ic3$8B@~5&o)7uLg9D(eAhMgX1 z%5V1USebGU_fhCiK+}zDjcU;#i#w3mPeK8$2Z4q6ucNtKUdXcsa2#qAUcLy-vz@2-=A`y+9O%|XXPAI zk;VZ=#N(zKh&2}MB&L}!Ky2!YM8#k2K?8QJv0pU*8g}bM z1TqpO67i+93&>#5!!H{Nx@$@|<*|??>pkBBC0c`-A^^Ej3%#X3spf25MXKdwpaMLq zI5MBw>=k)@;`sM8Q^iA5jX_6;VzuLk(m~Uv!LOj#QFKa8dtlTs5;^>p!x-K7>6n=; zHxK*Y>$}W0br#Qm3^P-v>MV56VP37 zseQ4PD=C&Ejxj$O9{TR6?PnJb_f{AK=r}J?5BKP&X@HU|S8Y|cyB_pR!;LXi=8nN# zP@=87hKd6s{;kPHU!FI~!`1TE|Kp?(qMOA7sOy*pqq_;8fn)r>xY$oG=c4-`Rq>Ap z8W}CF??N;Y+q$t4%!7~4*xS`2L&mq##arg!ZIO8tJBmLl>I`31>7qVb8yHixh?l*T z(4nOYTyM+9pixpt(p?d`i_|>s3s-` z0Ac^xkbr@gG6VYN6&~Tc$>)B{4OX4?}-kk z`g?HI8p}WbJ5vn4a_m>`*U*KH;*7g?)Ci@AET;10W3qi9TYG9w5gDnktQQfe9h)`; zjc(Uf&LUdeqB+3Tz;uXcv`LcOK|5|z$@M;p*XDV%lbr8$r+~j$R2~`JpjO zApW~=&Gj)~U0M28b>g>{J6$&h0o&0J*0|(GrH4nIt8Xw!mejA{YLnMZ3`?{aTN-A@ zfz}&eNAcg%6_9+PFjP*LEJr_XFAXe4)m$rnTfFayvGGx?V`4FfBUaB?)~aS}`P&}b ztF8a(we4U8y`J@P_7b~FA}KP>qjB#j`0ZIL5T zx0tlj9BaEa(5V{Vro!cz$@avuT>voXF$`}_Ej=Y=NT`M5k6cTK{p>j-C3v+4O$hMf?hj@at7=v>-28#m}p^U=R3SoW;jL* zvFF0?3v$pdKa!tZs1H{%A^4VL!5JQ`=GhjjxGn+)J!tjPB0J7 z_3*FO+Jil~OaE-Ye=SIWNe{V;VNrA;^jpI)jm(X2oUS$nP#?gapb%?W?@$4M^b>j9 ztpW3~U05U=rkIp1X{8XRNskcbi_d_5UtZXymU~DW*4mL^H_^umXdH=Y4b7vCy8Lf4 zDXrZ-(XERcqG64$&t8Rtpwt>10FA=o?2l2g01=(miKay+>yI5eUbEPmJuR_7mo+Z? zwZi{h3AWh?g_G!PVZ|i6C6IP5-)D-kGj_5LwjcQc2I*gagcyPnbzFwA^&D5~>G+<< zYVY!5Ywe9;apTP<=BfO^>~jp`_!?d`JWdZILyZXG_wxaYLXO@p(gHB1f|s$fj3fn1 zzRSJqShB6XUr^0uy)6AR_T_Lt0;Z_{-Li7&yNCIO`#>n8qyxJI+X!rOR4UrHg@slXLKehlq6&6H(O6W?M&bHhX z(~sMLmN*QKjYwm@%0o-;4ASV8lBj9ds_9@geKTx=~)j%i`!XXQW%*YNMnq{sXrk_r)=66GZq1orO* z1i{1PZcO|NV8KtV0xsUj(k+f(57ARRcGdak!q?gBhbi%etJBq9qiFfj!QqwH$`!6X zJ`ZDxCq5_9Gmt`X*cO6qMMMiPbY9_{F$i_h@{YBi6e9rG8%ti*@7M%fVcAcgZT3XVp0W+ZZ2w zGga&|GXjw$J%lm~x+sp9A0lyOWW9zPU#4;V)8@}K!HCh>EG?S;*+nsiAl`MtSZj{J zsexQGovjurU?fZ&OtET$+gX|69EC~OE)lB1T2l|AeN?Z~w%xg1pv z4VF~Gh3vi^q>Q|a(p#+gig!Sah==>1Uq}_c**!(dhzsFH+cKtma(|WZm;DU1$Y8Wc zNlH*_%1eI<>ny|h(&r9R(mY#??GgqVR_R>G=`uO{nG^YDm5@}3^>IPi$vL;rsb$B? zn1bZ1$*AD|`kd}6eLQ9gL?={6_&{=#BbglB3)|_Ttkz238qJ|sn!6Bwn@>?B!@QUH z*T;bG{q>s%0wCC2D#`k)f{QHw^u`(}GhH=S{B)H9?8*Rf`q&(3Bm0NF1JOz}nq}&Z zHbdmGtOs){#E>;yt~XwIzk(ZIBm0bwnu}ecnbiOkE!VA!5ko|W_h-Dn9@B!Jf0ko> zsfZMnF_~G?yy3Zg6BHyX`R}hl6ht4DW>Yh-^;aK-nB#TNVN{dhJ%Bk)VslxQHrUhg zQ9nf96sH=bi&hwH6GcRl=#5G{<%oFIwxz0OaWfq5SxJ?=E~5cHk0qANhGa#XNDVDg zm8R)7tKl1zF5q3@i1iT=eT7K^Dc~}(rFr(1=6{9*5sz5}QPJFrN~+29uQnC1fR0_v zorGPJr~8U{5!k@Z?5EkG+Nm#CQ-j?c2WT(tMz_w} z{bF9c1j!aufijhoffg)#DDQ&@6N*vLbc6e=Og{fJ$ba{{JA*M{-{H#~gO zucZJGTb#u)>k%0->><5ni3){ z3C#2XcCSO)_)ZzdP{_LVs4M}&)JxfAsk{18on2@4Kk7>QsciKIJ~`6^1CFm;STnv8 zbiT^0j0(Kj&US`M3}n18&%}uokCx;^r;kjsb%GhMUt^Z&o$!VTF4btIFNF>fEd%^;cjKd!z4D6Xd47RUs*!QCymySqzBfCP7Uhd^+5 z_Ygd|yF-vLcyMB424spIy#6Qzo4+9-_ zw>20$=#TJ-XW4kB=gg7KVU8*m%WC`ncV)!FgpurkE2q0W9&K$f<;)}mfz@G17i&a< z%<>it$AVl3F6;BPzM%cQa+|lS%;0c;u>)2>HA}?9G0796Q00Po!uTE?KlEl-F+{+P zhF7)N=DreE5o-wJs{xQW*&PXfQ1HP#{9Pczty+7v%;ps=W zi5bVgiyZ_?bBy6Ze!%G+PkN)S2vwqqdaG(zWFjq?$?Is;VKXytuMqs<=gz)RK$Oys zj48y5`}5VJHZAQB9KuE;Bu$Gky`yjncRr;N{2+W&iM<=ngo6HR=_w6eiU}f^mxvu< zshQs1dEq>)&6aHsAwqE$lNhl6QDx;kpw;kq&i+P$Fwr`Mv-)uf-UPgqH&O2G>ge_D zg*XD1H}ECCf5)1K*}aCu1g!=w6xAKRF>0E?!=p~qjQOFmmz&upzDIt zXwf%@8=7iY`9)6Tj>K49!VlAho2HvMB)T*wi!w*U>1j#%JiNG_4JJ(C|8Fi!#h?V>Qv?$GS=G|3w6wJey-O#>uEqbI@3$DD zuQG)IZ)qdNUDR01e%)kC>Se`+Qu+|`Zbt1!M_z_jrZ+ybrD&Bk^y+Fl%Oo&_aYel*SpTe3A;=5=h$38f2PhB3khm`P)0 z6ioaOWPFTgRDur5AL*F>F+RkHZk5_?&c~h4kB!HxDe6r08bHAE0zF*qPdCY|L5HI8 zUhk{JT|Lkpd0e-CBMAp~m8d5IYiM@#EjVW?yymhWQw5zb^bX~S(Z2Yn)4yLyRs{+n zblq10&8}eKS8}-1fDHz_6Guf(O{6lI4I|&Q|4cc|JSbIi%n-IKsd}C373rk|nLljE zxM&Ais9-{RF)mfhll-q;Y^sSJl5j&^inp*}Iv)RKHjMig&S%C%S(>B)5DvU_ds(5O zQXNx2s|teUkI5DwjX-WRxlFh@K1L~1!v+TXE?ji4Q~9^;(Q>oRNG-WfNS6Dur^MsX zMy@<%c_*;a-vvRX2sCQydA<(Boh_4!#9?!froGvBzNUu}U-{1&jRw7&>#R*xaMl}) zRIHSVgRvHt;th8TPY2eVAOIUu{O5#)@G31asYuuap(6WTe>7|U-p zox1qEUV_GluGQkR6QGH^ty&#{1r4^4f{``4bzAke`lHT5;H5TxBBTxWe_(uBa^GqE z$kF7YnqDD$wtuZ=yN!{#)!-h)^6vl1|98VKBap$QlcEv4gck8%#ux~wv$96ZDrJs` zl3g`W;Vw-5Y-4Qk7_f+B4n4VF#m6g&1_!Ajd6AYs?|oY&5}(SQEUdYPi zkumx{KEbHoGAR4ur(UtP)_zATYax_r!FC<4G_*71%xtw#8FH3Lt#}_20`8`T6bGRk z3Y6O#ES*t?-U^iK_ho#I0Me2l$_C?)-g=++uDJwe%ja8ruY(KW&iS+Xol2MTgZ`d> zU*Qe9H@^}$u|{TCAcfZyyCQ1`6QFYaO>1&nKxuCRNVx4+1&hLS&4bdmCc5FdAO+6l zpG>x?OEpBzV{5FZqeM<$G+^^bY;v{ft!iX{>&3tdLNlzmQpfpx3j5bp_j3X9cDz4ZZ+8_8};ggmJmQX|teKc_=Un|Jh4U5_YG`g}Cf@(Q6B10l+j z0q)z|3=meXXDfr4B$=K!~D5U`2@a^v1jPa9?Lx#6t>|{4zW@R8-3R z?;0D5h2oJ^)Lupb`xYr>YN$~3o(1C#XsjNr&yyg;!OK0oRk?A_DysIg#0_;1vlLJu&;9t}5an-%z< zWkZJr-3vRGEs*++9Ix${60?j%Z~UV8ujDb?FhU_Fk*f)b14~}d;vmD1<)IYqL7PNe zNK7zkeB{j`a!}561nF^jv$Bh&3LpZRmj&~wH1` z+=nD(dt~S;ZLOv5#({UGAAza-7}`BAW=!h2177akzi3=pI>R_z%C|ps(HFu7#^zs( zi6`X6>6n*xJhl{+N0Z{iIrDFp(+qMz5U1|thc++qIQ?FOT8u<3NXhu zM0=P8;5!SsRE1!}6ArlAw+x0?xDpIMsDBl0slZDfRbD_jixPYR#_q9Osst<^uNW2xt}l;<%CC$Qv-6+L`5K9HkM$%0 zo{8&q-TE7)wLtKA3L+)MxLJZQJMWgWxj-I|0PiCWGa^8|&*@W^WB{>7Rk9oK(}ZIZ z^7Bv$r(#lv6n-ssmdecqVf4#xn@N~A(-t1R?OX0uJ#DU9SBFHUCE3m%{4GzC$=5Oa z=%-m+|4Dj3MnIXd+y9g_l5S{dDE$21@=eHp>IFH3TzhgC!|Prto5gxduD$a7(64xG z_#KQ!c+(4`VYnnq-`eH1RA2QWMD5`-9VHCUp+iRnYtD9bjP-UGyiqsOBJkqZ{5Nvg zahJCTSHE-qAy=-Yk=*Nq>pY;gHrJN#jH`qa(b#GSw1ldnVh8vi{ED|GXK|LUz0Ky$Mq=1w#z*1A|osWGU)hXxY2TpH@7lc5@ zfu4+$(c?~eg9{tuCQhI&9W$~#D18)jto)N%Ry>s_qui;cYPTALc%p`#{(*Y0GO5GE zLf|;bPjB?mw&Sl$7=TT)mXQm;+P|w8b9`!0)+S0D?-mdd;C@q-`zqJ$|C{PhI@zk7oS?d`2E$6$KTJ7k>7$G zGaK)3gFcn>WOo2F+;P?#Kz?K>CiMyl;eKCQKHLO!bFy|A9TUbgk7CP{hMZC^dinvQ zubndq$DbaZ)sbOy8N|-<&Hu|q(rH|@Bhpf>Y2+x*d5A(~Z`!y21E~RH0mB<88N2aK z*RXvt1OXN6Z$l?=q^RXS zJf2U_58TP9gF|~?(!ffCJO3JNCPY(TP=*$}{CsEEaUHA)(6uY-ucnD9FU9dcLLjqa zBQ4PH@&%jIr8T~zXkv|X)Q_ESJGw8G{P%hOySe<$FykV%>SRQ(|AvHZLcJTe?@W1j3C=-HkWRX!z5B^f@f0!*T^7K zxsOcS;iD<7PD-*$TJjqh3P^l=HMwFyoR*HxTc!W-sZt!HZMOlO#9no3D(P*S0 zh3@%1$^SQ|T?}AM&i7?*|NP#+9U+(2cBPd|3rXNbk-#v_m>LYKGYBaf9Nl4< z$8#p=Ba;i+q%$;Ix~fE^(X3vd=Q0>?%|6s&dgXXpyhR-W)?p0MxW!zQ2R=RHI!;P1gmeP4oVmggt8;@3|j!Bnd+VBWSG&&``NG}e?KV`snEiIVfHrXyU+OD>H zKHeOM)a?VMDRF24r^4U$_V=A$x_gg;+x)hV(H{GagEvd#6JkIS8wkur z-|TAGL=Xc!p=*J=D($BG%suy|8Bm7G%UZ#i`2-xgVf<-t^{o7~@szkuE%mH3RX#BN z&w=pw2Rb}xXRt9M@gz$Mv+j?>SLD=hVY}(HaA$~Kz6l*1v`IB}W~$B6vR2G)X)=W> zEKK$z|58)arM>JIa$J;-useTK3s>;c-8luf{L7gmD+%D!{YmOHgzGqFUxPM!-Wno$ zB<#0N$o)@@eEC7u2Syf%gqSpnWR+keJOD>2E%R-fk@)9RG5k%nQ&t|QI^On1aMD;Q z6vChFwAMY3B`;}elxm_JMG5v1z-zw`7tH|~eV3IqbV3R#w0!q!xmVBYfLicHPMgQ; zYM)bSrse;9=1b+l>astP4`9d##a~tdAAvln$TYP&IzCh6*ye64& zIuI5O8zS`%*dGHu41mGi8CtJ^YuWTlgC0%w;uqdECepO;qX@X@cY4h%TfaH~P5=LA z9m~r=g;ozI($+~|0NFD_x`n-H;bAbrKSh5bS+)D0Rp2;3$%x>YpBuk>*pv2AmQORz z3xY2o@74@w`3adezu0lwIV;>M+Y_b-2Lr7Ex}+o-P=qqD4Evkbd^5WP+i@_&sT>Ch z)^S1}32KzW?y9N3F_VeF z;O*$*@kCt%#TpqtqsStCiz!0XK2HEk>U7`TRE@m;$dv4?W4PgP^8q864#!^$NR8f_ z5Ku>r?H%I`Sks@Dy_58lOzCCDvHL$wftP$28RU`73wV1B#*&T`3Fu{*jpuX#+VAX~ zI<_xmss9vuH%}oDo2^fP*i1tWnN$@A)lzq-S_8#yr6Q2&Nc9KxSBK>$DS;4`$h@K? zC22+!k9zubU$&nS;Uy0oGM+;(0xgwYv2r;pbUUHGVDrX-)`s2gWCL)oql!VZ^2J>v z0ef{}!E(g@XsLd1(1vmzNYP4O_*sv?L{&z*I^24;NuT#uc>5J&G0hyW! z=A--^(%&32Le2oFx+@L;6SYeK96@v!G=}elS{D4FhBvgr$J=)I{kvw-a6@bo9oLHY z@LExly(H2f+JGy}>{7!zvAG>W81__3rf@^|YlOz@Jta=1; zW0Golc^Oo)C zYq;XRR=6QoH`5WY-HSZizhUf%I10ELH6VO?+9F-)YIQ%=JIZvd56X?Wvd`|;fkmQ)@pYr3v-xHf&D*(Ul0rB6sHWR>+uuT*u13`5tyc@uP+7g;N ziibE+rRNX;2$JJ1DwvKJ8p#_^YE67_vBjD8)YsAo4Z|3P?7k(qLa!r3dUHR8CeFAg zRB;s39~OOG;ay!yADRf!+5`;q_t0+3ieJ;kDxS$yz6Nk81WAc=Hlxr?zsIbiY|i=0 zg8wcWFak0WuLDpL38H=&lFVepJJ&J&@8Cqo0S z(;hnH#cYA>B2BETQg6O^W>{jiT|^sUYey+<077Li*B{fom+pSoxTu?(@2z%te`^IA zrp*UI+B`#Rlan>}Yr;BOr~*fbCI)Lvr_lq=b>SBJQN!-Jvc4$SDZ7 zMazy{U;_pcV`i*bc&1p3#jjlX1DbGVpx06dg)cHCQokj|>M-@NIhjih%hfGd#n9|H zupO;-EYb4=3M#gepZa)^*IlyTa>5|f5sG7lgk)8w8>F@jy$I0Akox>| z4~vXovM?OH>jZ33UL)anT)r?50gDwbj$oj@&hNB61b8Q|njqNcFi*?APC8o;M0;?G zAr`m+oOoH2VRjAxZF;a3oM&bGg**{ON4gs39wfbwq~2fKz5_dZz`I^AW9mhlpF&v+ zR@nE{dyVpWSjG2(aP1Cujb#?Yc&m6FX!hM$1%=T8;U$gbEjm7pN{*lz1n&zGMtL?} zxjP_AeRI6Tx6P2n#pes$ECj;S;c6YgnJ7oVq{(flb_UucU3cFB*Va}J1Nf5+A&0*sLCL}2DQ#sy!YG!0@DXF2_#PSx3bpFpuI ziPQRgv$W>1g(7`lT2^@>zBTxgi`wawz&GmO_2M?eo^<{34Y>m@`1!-)o&jq6&aAyz z_=gVK4OYobo;m|*zq00U0IQZiXQ3igeSkH<)l}ZevW~lv8NH9nn?`U`L_wEKhWO{_GAv{bDAgU4RSBWEj54{jiGc<&JpG znLI&f*0pe6jT$h}E+WWTY9Jv!%o=5XETGqa_cfG+V(IDr63h|9w~HrWIv7J-6m)*R zXk8Uu5)_Z`_?;nC%kt^_ama6q5V7!BizPaq6FPb9{biX{GQv?0;JmHI45ka9qi|n# z44sL@aGO@*;Y&wh8VthJETRS^r$oMf>xK@-xc}k)Q`l~?9wCAJ8g|_4a+K#qb^rBk z-QJ9*eqp?<-wje=Cm`VWjQEu~EuH&SkY6aTlq08?@<(gbx1!R!j+&?mOF(n3$)8J+KXq;O z`!r58j~xr;=$zbEPr#^bLv^}dw5=Z4wSBj>8hszy?OS8hs#scND)k0NiCP~ z7&S2Tu$i07y-pvQkVy#`4=m95l1Y!^fxa))9Gb-PaRJexWM~?fb&7f~bL)OyqKYuS zm=O><_7&envHIQk;TWyT9vR|PtM@XBINrx8rs3M4>|lM&$(>XYx=vE>b=c_{3Eg*_ z*C6rS2CGNK{eZBQRaDpltgYSH@5;`r;tftPQ^B};cnWB_6j>1T4)i||{KKcq@j)hJ zVI0?qH~5wM{r%4Jcxz%uVvPlY5+PUO+*bmh9+icz-?>Ndh{d9d7i@1urip?HT$Ul`15`GK$5VdDu)GhA|&av;6|>NSqw4HX+WVQMx^k>ive!*)`i9NJRxKLc9T>u7@>)EgwDhmaFreG(Byq; zSyU=4Iz^6nV|YR!>e^?Y@T&x#QAcXOp-2AUC%$=>n1dP?W9~w>$AoTMeNF@@W1W7y1dXYsyLRk8y#TV!qn&RYeqr-+1tBKit-a6Yp-}fJ`N}k@0UN}2~Uvx^2C)oAZTbF^5!sukWJ)e zOUbrw^o34iAU_*O^5wpz!gL{nuY=>s0MIC$W2@l=>Z0jfrti;hS=$!)!>XY%sStw_ z9r${Fq7ob=*1}^CX|4t_1gLB_;JZxau>i73u@!-mxKDsK*B-^)*DA8#3*=c#St+oF zMkgN>rGhA=QBZ@vOZ*V`5|bAb5WIwN9QSK|R=ZXlGwdfn{TN9#U>5MDO)?wl=ZlUb zvN5(}1FJNSN8A^sj36&8A@k~nB)?s&N0zpuTEJs!X-WqfVw@|-J<)@?&^N!4y(?&n zKnVX9Hb6qBLyQgC;+XKTtn z^$E9`UXkq6)A`_eh0D8R$*Ms-l-)APZ533u@{&ws>!G0wr5f?D^kGwbOX&#m?ohO& zPjAq-AtK{31J7-)BeN^^$1&b1EFx<#Yb2*K9wBR5J$O0(Q>~2s+WAv|^8h~Z`n^A? zNe)^IG+TTy9DT4N1fe0~rFrr9`1xVI#9Thj8lf+=^v4f9#V%31 zZ7+vVx`pMVWs>&6W~Fs?c+r8$cGkn_@@b0}_zuc3-`>#8hJgc(00t2|7~;eJ9=N2j zz%B>UH=ldNdp{pe5uwZ`o1t|fbSmWOS$>LdNjM}%IVqm^ItlNAG={3r|04d|Xrg-q z?JMY+!}f=jguoM2%O_2J9Fy@MF5=4ySf?gqgD6N?g6S}!oo^yAD8WcHMbbw);e|yy zE4{2Z#%zNR-!w-psXXEq6vp^ znd^VF0M6u5!w~PT(nyov7@CM8$!%dyn+hx(Ql_1LN@yL4zW-!>5Doe z?(l5*1kUG7booK7Cl)1ME|_a#jq3-Fw+?dxK4%Ax;}<)7G&@UM&+zTLJu!dm;Xa(qNRF$yzGOE|Y9M~fg^sBW)&eOXtRBM*igtprANYlaAE>2}om zj|6(Cs@ID$ilI3$MA!6c7*kyGMk79t)7ofd9$FGMZ`M#GrSF~-ZoxPwglEh82`E(K zWE}dkZcGmc;2Rx(Mbbeo!5!j8Zc0212wp!6$}!i6%Zi(4M0@3rEKwjlcfQXT*g(d( zp!@_0keN)6v$Xc5KF-7wELT~LSU0bu}I(RWL6bI2I z$vf5AgS64o;wB*d8ajsRNh2@0UpcCshj0XqQzD3D*c$gfFSWa#H*7K-tYGHE=+!cW zsvjMm-bgC`Y+omIu^776JP&672ZS|O70aU7EK!S+K=imsax{0g-+lBg%wH3RKv|& zef_pTatxST3qcvveO~w!gJjmeg|bFXk&;eENPrg@>B`d6GN~elz!BMnpr)H+;f1ge z7Z5f+Yx;X1pKhUdQ%dB!Q=W+jaUDdUIHw)lbQbpnVB|=A1WsX*p$0xF&`15gCLMIN z#pH-UvioNHDhu`88@sZj#cQ$^kmxM}yoc+GQs;{P6bBM6bOLqKM~j7hKHZHJsDR6{ zaPM?Sw*H8?14$))!n{ufkKka;H5NIg4LTAouSPVOIYn$I7k;Qz-mPY=)5Bt<`^jfs+-tj|QKB}Ov^KMWMMX92 z+H>W5tHGz>!+iKKD3HfE#Td6dp1a(1AL8S5>G^+mJ;THS(lnO*4}geDiLSd=8yxRI z0H5AMG$mxSY?cI(DDRyz31$#A8ba^qH;K`Q10+k(yQ}lhg~vk*qq;m5)Mw3tK+KsY zxO(M!iTKnI!9I$duv%&PjYLVN z-IfS`ToSyq4CD*G#ogZ%^h%H(Pa)|XjvoI=oys700zTvckIa+?6TUy~P-{TA!Y0CF zCal6Gk{@*AJly%HqtZfwEQecDU{$JIAoz3}u2eijmW*zLvxe=UY`2l{!gfbh zs4!h3i_k_?2^_-$pUL8^Ln`nkZ#VHryZFbx9L!HbQ!-#WGH4RDWeg(#PXnnm+Pt8uT1K;!Usa zm{{0du9OXXIIYJeJgFaZ0y_Mkye}TZK&Cl{pg~H0{RxxUUuQtwKLv$K{wMb>9%h(f zj3n|IkaXwc7M)rQ$;dYjX=Z0<&x;1manA?=h3~wEmLOz7edsB3?P(ywl=zMHFTmGT zWf9k#wcwXY=yHQtob)F2l&?M?ud<^nxvA0@VdD40#Mx}duwrlShYAh=`dFgDInzzE zho7=;KGOdAiDc^fVeEcFGKO>w0OZ#9)@XkXdi}UrN@I9q%}mJ2$Be)YBlY5ZM#l!o z+f+phE8_L+^L6ZKFWxJwm2X_^_WWTRpAi6d41|zVwEa>84D8_`q%%9bL$M`w8!N zZ%WGJXFS0b%}$@~m|fq(4vKVd&~>;K7t*9Mn2rm&6UGjKji&i*)aY02qh3<0wN?B7 zHsLu_q0lrzNmkp5$^F&4G0j%E{!}2=BIB{{hqc0BYOgV`ByZ2U(TGV`KCDXUjW;r1a};N_z+P4~Eg^<82TROBQA_6ffowdD zQ}}6w;nu)8v&(_1(As7HRlte90a|O_D6fRd4HMG(tKaJ2Srx!Py@IYQa0j~fF)hK_ zKr7;pgYPz4kXFBHKrIGmvm|rAn>2(uErc&*W5AlEB3s*A2qbDwkfsak22x1mn{OyD zL~!Oqr5@sg>JdTMqpHG1)$n!J-Q2(5`(mF;Bpoe7;u$0aqB5DGFqi0;{sZ9{SHMuI}kjY4d9#h8y2Rd+1q}LCh{0t9)L5QBNh^p8YY`q0x5Ao zY-JEA{A{N5EKFQr`*Eo7b?b{AmKFIxZz^XBq zz$UsvpidCxhgouD+%F>#^?RB42&+nVw|xSAez{}njX8NO{pC$z$mUh!TRc2f^|vC#C9M3JS1f%N=`w6k%a9FfUxP9 zc>E}(;e510_uGjt7K0s;t5751L81YA=zj6AWZ~I>u`1Vb)De#(!84&!WT~2kmNb&w z_&fd|Y(8argiw(rA=-HO4%QOdd7z&$0C>9$NO&I=#G*@|8HjOZ1Pu(x+!#Y$S4Nt( z5~uP@ZD_%UX2cWnu{E>J!W#@JKV7Z(J{~91s_ar)^1qfy{Su3e9;3=ygC4BI#0!Wc z^LpQ&?92@_tx6+c5waDR^=67bWN=z0h>2djh~PvzkuyyBJo-08j(W?hPn{eF>RrD$ z6{hOC;nRpbjA=C=CLZ>C3JutrjK=RES?`>6xyavU7vd`mM`?e%0N5f0 zwCfGm-q&>*XrapF&b13-_VurzN$$D0zeU|6(j*7Nn{j9QubGr(Ya?6UviTIh3xKDh z>E;TD9q6ZFHh9Ve(CIH8Q-+(|`~SsBDWQUnGP60&zSC=p-J2H)H{|PmKS>b>2Ms~g z!mLhnKfn*MoqTFdctf2QRrA3(<7!cu{Q#`X%HY77qD!-*S&r^tm~GCP%^esYc7UM+ zg%?6UyRg5)w$#caGKCcRk=!P&*PkV_rLI{}2f|@KcGb#5t}ZrH3!Oe{s+^chy-zw1 z8ugCF@3&cnsX46S#$Gt{a)icM2t3E>V5e!a$i^K|0g)dqA9w|wJUWED4I;0aeq zdF-+n)-t<;BVTvJ4-YB(K{B{TypygP=JXUu)|A&|kn=xPDG7StZG?&`oeBL!-CX%V zdR#N$jw7=YV7o0N#o;*QOQF>dAc!ccoprx_3IcamgJx#@uT5zpXhiG)Yq~!H^??+o z#z7xpo{aaMosZ*|KRp)&f+Y%OT`DQ2527ox)~0 zZlx6huZ>A@3w%so3pa_#=dBKj!?yffro+eO*0&6M?!G<#DFYN*y%-KEQOsZ)`mS+V z)1$rf0TP?6%k;#FEM+s1$xd-a`mLOyfk5=8ckE3X}*D$WUZn2kNTvCFh9ZQNt8DH&eZ*ET6JpZ~g@RO*3>vrU5sD(y@OGaThjb6e zejR+64&eNxD~$#V@5Wg*2QQ8bUe)?`I4MKAgFcHGWtSxjWB97DzbvSR6dKBTU?`q) zvd>|mhQ7LX<2|g^^h^T&oCS%3c}q(3fI0y`{;Klbay8~*m!$~a^S7U-bs#V?0;??S zL%=|vjDE;j?A0Lv;+soxW{nS7aY!saGqR5ZWcJSZ*__d^0(VNnSBb06g$sI_~8846hN0uHKfBiBcW&e8Ss0T*6O zXeNZ!eI|eUyNrp~nlbi2+tYPh&Bp=39-nfa+M_{**pWW-i=ww7evyOmj#6)cmK+eI zpPF6o@61We#?m3H!`U8Y<5DLFLkfXkP}PSN2{}@0=|n!PiUZs@I#Alw;Oh{)fh~Sq zT5v_5{j%DRcmKujk`?uX+iFW)$zJcD72GjyT5;2s9Y5Z0>Uuto6XXZ-?Fm^J$&$g7 zaA-Ciw5jnQw+ETf1|XNIVFZzjdH z*k&y_u&Z}J?WwFeLhLggG8OjXMOQ^-dd6W{a1@QlV)Uz+VW>7u`rbg%q80H>y8_L+ zA4$wnvlxjx6RfQ^aveC_W+Iv^K!-ezEiEcAC!Uv4uezbRM~yx1UaFQUjkZk4?y3dq zj@UVZfI-bs2=pk`*9?V6ARvGFtAWaEEQ5N&hki5ey_l=WpZjr7C#k6Zs>z1B-4>K3 zIZEg-M5#-tutU991Oq!g)Pqg2_J|HsIv$bw+E3dLu`tFt-peg}D8kjKUw~VD{y)%J zzeO!^@-X_c$xd)R;!Vk1(sB@wN!BuRwZu)i$bzDva94r0Obb>^$wVKgz7lQ-Kq?r) zXg0qVB6PQ{KN%PF*$wlhERBHC9z9uUyYIPW5+JVW0u|EI2&PNSZpUaP)i&_@&$)HF z%|)~!_tLt3-b^GEv9$D_xb6x6y{vudqHnq2t5a_gU)9|d+Szeh?bBGddK&&lW1}>s zO6&IgG|_E^`Gq%(r~s8tWfUcebQcnaVKjqsV5V$mIM2vvNBB#;HYB*d1tzrm^vvSB zna>7|Nu_~NvVlXqXzzHhub~sJf=M=D{G-Q$qul>(M44w0i$qHez87hmZWCHh6*-Hi zxPdC#@ynD}4-#Bem>zpGC$$EPD&>3$n`vMNh%q`7SKHz9U5?52VYKgBr;oM9;3i;w z4d*UBZn;O{^9*=UiJtV41M8&cWwdAq^!ZMi}eT_I6rW(s+dIjWjPooG74#u)G#OiD#>lJtv{K?5<^r`>J{ZCn_}iD+?;cs#DoILal{|SiuWfyLz3(?ms&$pNGYj%4367kjEmqBeQbqN->$)?}BQe zw6MlmuaE`%E=0L)=2>x?Bi^#JK`rEMGy0O~VP_}{Q&YD(+$@;CT+B%7BkI1t+^bW@ zn`BRSUQhYf7o2*o^dVmP?@jg+8NWgLDv@+;^f^iP-%e{hsq9Ei`>r(3rWxYWeC>J0 z1RUQW#P9Qx5gh!%lwhH#Vy{g|ptzS7gFf1?Hei;x;2o-}7Qj!HX>@&k5~KM_%i(VT z8DLQsC7W=7>e~#;LdLo6Hvv`#^;S`xWG({8L!l}4@@%HpBHhwV${A@3JHBzU)C5Bp zGi*<%P!o1b4ry9(vpR*ZsXpU7r4E=3T(JqXDRVGzl;pf9Y(Cv;MhfGHO(sm+qs>Sbw<_jU3Hg7y!}8tXCY>_YRBXPL zj>KCmmO0(&Jeco>Sh>#?txqhR$VOZX}v%w{kN z#;wUU0wD@xrNPmRv~+>ggyqtWEeWn)gT97>CHx9AS3S0(JAkZsp$@Kzes|ke|&z|dMZexvIQf1mg~(++xr%lN&0yUAu_PRdi>mrMiz>Seg7->;h!t7 zC^VujIIB4xD#d!UcmoV0xS;R7O=qc^j|NL=2}1=EtL;YLsR}r0#+*{Hhg#P)A~U-z zzPHA%dezXPSH`LH{`m1aC_Nk-S)@R~VZOtg$JNoCd?IZ8ITG{R&yPk-#qq*4_0hvZ zf*ljw21X}m*qg_){Vu7dA7jNsy5XiT)Y~=+drsRgQ7xt8uDd!IZ2Qs=AD55bVSE<-ICHspZ8^Cu?ngHn$DUR5hF<0e2fV7oWkv} zqRGoODK15wz?!aEn~${Xw&OIOG_r@M^Bm4^VDu-jDB%Wk05NJ-u!~TZ|%;=_u9#TrbN_pnrK@?y^n)^bTtl9xDj^eFc0b zQCeKrW(30!8x67MOru1|kjd)no>KV-_f0;rF+H9Q{$Tz~y!X2D)Ncu4aD zn6sb>mk4-Hf2h+X?#rK#7HOF}7)m9vtOz zMUY&F1;SJ3WJNaDfvqzir>Rh5{@(rVC6Ih4_wybF=r~KM`o4VgR7n`zNf;*u7RaEi zrk!*I`SB^(K_cg<>{{bNRQ2Aw7aol?P$Ojx8E(?pEh513?}bYN*9L>) z)<2l!R&kcJ^BJFHjwxY~ z5jek+{<$0E;>i8w$XwQ)}SN>2un@%scAWL%##U2VmRph8}VW*+c- zan6&8ypn~p`N?oi2D<<(X8VyUjX!J7^y|GnPE^U){Uq_ z#uwKsGWE4~30%IVzh1JnGh7x4=o;&ROx{jVP49iWbjeFqy~*;ELgf^Imk4YrNS5lU zjV)Sa`CmZ%8a|DCH-`U|4ML~*uxlB2ZwS}V7At60X}nK8-&K&o>r$wkMiTNFZ);3n zBlTPtuka+B;Gh5Z#Pk4sTCy;thdyAT=21Ew_Nxq;AQZ4WL#(W=`y78TDj#v~F}-8B zl+Me>)${mji49&-l;TY{#L@2j)N)Uf{rUHOb1=E9R{8f_9kDDS;9lgJ*zM-n*f9x1 zoRqA1hjAl6_`(Vf}w{&c#GYF$OL|{fmyrn@QLc0NhfiQp8avDqr}flD2PX3(+tza(q=#0 z;==Jxkh7B`X26<#KG*_SZm-M(sGWwSBXVjNgNH3EK^P{T55LO5@@&fNa^F=mK~ft! zijKydFC&4Bdux2$`ug zuLOO%Zl9twHe}i$kL#^?`#hQnY`1oVvQOY4Kn)l#{02Qv6d^wuxI))+QfGP9^Y#pN z_p@Ck)16oBOkgo-ox8(n5^~ChYhm6?X8QM)yAEn!HJ*J(g1X?r+fUIKA{D}!c^FpD z7Mk}HA%w)6js-@sm6jLVv4j)9^5?740pN(|jDpIGIpF+hv2j#xB~F>gpG&xr5+r;M zG%z+1LtuJhndD);9XiR6NEWfd$43JZ@ct<{WjnmKOPqE%-qJ?odTW3k<3esHkURwU z2|pB+D=@bNdAIVF0FwF6X^ZXjVcDCmH57TjUtG_cHnCKYn%m-|34dsIUSLPms-wF` zy&^q$BcJwN+{Bh?x`Re3qhEO>iYg>As}O+8y;_YA!~$X26aYQbsOXx0*7^a+NWF&V zp9imJzz6C#@Y(4HZ}rBm_L21O0;EPvU=ut~Jx9b$9v=Z^F5uO1vaLgSo*h&;Z zq?Vw&A#$=c0BS4v_WC%nCy$e8az2nNR-8!4CNmHY?KhAj^r~ku!Py0C)!d|dKcQ zVxw(h4LG~N>dg;~BuPYx?@jI1B4f_rmb>*lF;sQ2XLxG<=KnVjd3lecgf6$S8B+

    A?1aDxmb^nD2CP2p!N#QIeMI!%Q!Ap3>%JBQ3X0iFf zJo|gE@@gZ-aopN_ZCT`FIJT(#@?pt((L^@*{Lof|N^W>OkSwP=vpspul!wvlYixUj zA#(0Nnc19u=9fKc2yCzouMmRT{CGZ6=HVtJAQ{M-JMLYSn#Q~mHdICc$^s*Ybc(kW z4fc!m!1cx}LV+Lo!A-%;C~UNQ)tJR}v&i}Gmb+HlYDHXl1gm=aX8(__vkr=D>DE3> z26uONcY+6p;3UCy@Sq_O9D=*MySsaE*Wm6F2nkM*Am1kEoLBDsZq@vgA~m(AclYXE ztJm{;wjKfthg3aU+S4)8*_&HbKRh&mt-Mn(&B!nc=o7!O%JKdFQTK%s7nJi_efRSC z8^cI z>F)qCG9HzG)(ep2ih0XO`qyOh4tRXn$wMREn+CZtlme~+dlz>e}*#w= zm#;o;eGFnBHpV}OQ&6gPAs+ngJ!ZyBX`@S}+tQgqFxb)n z3juMO0a&>}quRbX+m|gaQJY(6TIbm`QH*IJcaGk=WI-d@M$I6X_f4^7;eP%IsZ^t> zUi_p0B)s$GejQ9wD1r{Bg7?W2f7JZq7>3JmVV`I$Fna&@vrSiq=Nz=uppd0rkzr4^=KDIxIh3d3BpsjE9g>8wj!O>2Vunp{|=!u`#t zOnC^PC?U%-flz9-`tO&s;DDxVR~5As%4ixu{xX!Ge9DmZi%a{${AD{&yG9!RfKb3` zdQGIZx8#U*0NzI^&d$JZIOtoW%2j?ZJ2fxkwf+Xe3cmO2Az!<`_EwjiX}D?H2THR500w$gXj{n5*KEwK%H8-J`f;k&^jje< zGhcK}u8@MX-1IcssJENjcN+FPnfwm4Q>f2DW(Eo{90R%_(&SKs0PHC)Dm~5fArX{xhHy0C6pbyKxdJxmXy6GvG0vF%Fwi2y;*j1dx@#b2kDw zWw=O5oMEC^@QagHN!@6QT=L`=OO}xMtQ@ai1-R7lhc>hLZ?vLg6>|A~MOj|U1G4rQ z4!z~R&vGAfMxQUgpF{*}?V>tVmEVf|Q7#feJHXVbr_JCi=hK^D4j>A~9zcMMfJ@l= zc(o;YP8%|B2nmuv0aQ0Yrv$MDvw)GgIo(^hbh~;h#*0^^0MR{MgSmZoOwr_V+|J9` z6{tf?Uh+?0Z+w1r*es%4nH!9dk1-n3!|n8AF`JDLRTEzC_w`AZh-a(DMKXu+KrjXs zU()ZMA4_$w$Fg2!2{+>7@A17Lsd-c}^8G2MNI_)Qv`Ntf_rzG*Xbncr|Ii3<+XHCk zla=kR#L?Ke=k*-S1L%wr4rfl=X&fwoCtnwql5g7=3#U0~$H2=WY-&J009ui|N1SvQ zCFOR9Bj9)u-+P-7HI*bx0t~BjiE`A$Oz#riEspD<-bwnGL(OdOCRoOVrSre_vn;Ll zB+qRhZybiP7$OJ>GaGae2eD_qIw*IpwEgG)T9gTW@%$KfLGdqk06A!m)|(>|PSqW^ zf-YRlJ4L7cEq%NZ78|U7q{kz~-e4nj(M9b~CnE~E_e#XvIZ#i5?XL#r(Oe=10hd|0 zDb?yK&<@Y+*$sooK55wP3$st-G%0-Tu|u>0pA`1y*O044jV^(gM#ra#;Q3%Q+yR04 znDbjZejF4Qa8~t#!RL%bA)rRZs!4fmg>03M6a+Y44K65=uL8%KAMN|8R?ExF9x`h( z^}Ww|v4_bX`!}Zn8AlSRkF&lst@_Pa5|}k3eF-C>MzyFNw(CZ!o3R0Xm$2+7A87_cp%v8_oOrLkKeB35H9^nYLAnou=6oGvgs{MfLqY`l%) zo=%5l8OQK?^$=*y5JJrfTGETUUk|9&<*&iq3e&)#osy(IE^2miJt-}AET1)9fAwrA z97QT@IUlwHRD^$`Vkq^@seIr>h`p+0v*6O^01Es$X^|>EAejnFKTm?3mr6E9DPPOBAym~lP z-ragXPEqO_>a;H=n)!FoB!Eu4z$B%LS*b7$D;wMe4mL;W75e)bpJE9jjj;Wf2c>mB z^zY2$q?f-7H^Z&Lk@$_g*5TgPV0}Bkmv?90qM(N%Sk4R7Ot=%f4A$>QJ09R@muhl4 zM!CjR4yyiKE1bH8pN(|2Tdxx`$5Y};b8#AR?7qh>%guaGQx}_sX!qJ$87d9_&7`xC2~hk2Q+A8@CD@*L;QO5LNff9 zjbB~c`qJOY?Kk>XM41loNvie+p%oE0g7Ioz$I;qX$#_C|nEeH;NKIQ4COxF)?t_zb zE;)hvq2h@JOB7jitsbBtEtU}q`YIcTV5P1aoe}Qa;6Icu8-i~ZGZ>t4z{(%FT{_gf zbBQa3v*TE4!l>HWRB?VRq-2eEra$n!L=)0rS<-gx-uE+fAydcz&+njvRbIGHiI_5aQ*B{41bON)8UK3MurX91j z;xn{BwRy>oT*!CYpRu4>rTGWH>hlImlwGv0W{JoV{K|~iE(%?Pwf$7Qb}M@aL|r>b z6o3XDzM$N$_|f`bsa32!1`9cR>KhLV<*{zZQJd=*Yo z+xM@l@NjvB7T!y9GMVWy@8?$Qt%Y4qwF4(1nKE zrhk0)o2BEVgI)Mwhma^8lvlj@=;lsN_C2*UNZ zFiq^#EC$!z&-a_SLQ}ib&yMKKDtJPKt6VOphTM%mSJ1{V=qxGoaL4k5U9nV>`~D=< zvP$H@Jyy~C{?ZVloDvIyN?d85RX5j|iRv-Uj1eBLbcBeZI(Rfhp>&QM2{}#O&1aqR z^zW0S+6ir*qj5dB`Nlcg=NbtELbm@}%l1U%7ZUXeI@H^~jT%1J++ zw_->^or{H)J?JAL(o$E_Z)m_zn1)AA(*5)_T)H`facJslG$Sl^Ul9i1-_oyS~7X3#EYVh-FL4g0UT8CxtdDNz}*7A+i`PkJDt(+hm zNIag;OO(*!zwIREb0i)7Z>E9ahea4 z-`#nSmo03PUGK(_4NhZf(UGf%RrqYVCooD|9x5~hVQYlle~}OTSZl-c*#!#2RY_dO zUZJKt%0|%ztrBVm7j0JejVXkvj1}QN2@Gqqk z=_o5D24?M7fJ=d(8~n9yBve4)1?NTwqC*Ya8qwlIq-0rZ*k&E=&A%YXGxz_-HZ?`x zNmr;DP^Nv=TFxGFHDu8V&zKht1#`Q($(`(0$UGl+!W`bSqy?;YxqbMohb zw9Ut-N()Oi5ekAQnh{N5qJ^gPBC;4u<9z*z>l_q!2=lGHjZ zx;A@FC#o9)Q9*%qb(3(Sw6qb@aa2dpvET;nhSp-+sZ}r`f?uPgb zXE*aPwm3C6R;4BmIaYA}yb1q4m)RRbF~qHi*}H%t8ES&HGEM?Kqa>Sy)vq0SYYB~y z8CUlXWY6k(be}%(9ohQR$WE1lN2}6bs#R|&HgapB#f#{uHhd%eoF0yiWSAg0V+Xvj zHgMSD{BwYwAt?x&dr|NJ{BX)>Z>U!lXd*X;eOMupiEo@jHC63v?ZcjcBR5u!aCsDt zi0LaMewKuY5~(4x=9uxUR~rIK_1_aIRf;F%keOyb6mw9XI)i>uVjlddV(PLscx*2Wr&PSv#!k*25F5y4Rp{WuoQs5Wxde*g zNw}?fOI!voxL=I*QYbi=2qo{31(G(J%wSf9X_DfUC;)Fnom@2CQJm_$Y#K2`jqEu& z%02_@qD$@1Z7~Hh4y+(v47iw?#W5xh2cJ%Nf1H@^<@6~Le+~*=7@M^H`yz~60HU3^ zo5EA@ibDdlPX87R)du8!d9qOiyrNRB0FC`(T16WiDh|~|-EPrqOwB1gLtXsgq{c9b z`Pe`gfkiX&=ay$Op5b(#>e*ulK&(A|m?wElph{UbRno(0{=lx=S8Y(`>HafNj|iYK zSZ{p!QeC0e3p_%oboM0O-6_{cg(~9#Vqr7UDrhOI%)f-Q3DjLLUfGdZ`=3joA(a|6 z$Gq38>62g+X66vc_(u@kPwfr;>mV=6Y_1+a1Vj;V70zvcz_n4K4#csMoF&EJ6D0>E z^+|Mz;DE`vVe9s8vmUqpP@H{pvzTKphB~c)Y&KT?_GE@imlg^?wQQMjN`f+d>!mD$ zOM=IK0r`~Y(fsF=D!-Xc|I1k|d;yCzKZIMU>}xL{qZ)HJlUH~jVA zLnRD%yDRv9GN{S}6s|N^E1E#l$agTIqOf3Qz!)Qn!j!_wo?9r==+v>E$;k|75*>~A zf=faGabB%bnEl(_+PtB0Zkae$qSei;_H%Cg_s7Q!gIz8BCFlaDuAu)$-dzDF?DxcU zc_eneeo+txynk^>pHeu_Lk;;C6|R3hxFkTmn-^JZKI-D4UP&dflD;A{JuH8eLmmyy zAHnb$d*+X%$BEAu>HiFQ|eYR_G3p+flX6ZHhdskl?37e2+af3bmOBmMao z@bRAqLI(+nUnF)*Z`x-u{#tdIq)1lVtf0vMy+r!a4S&LEy0_M}Z2Di1f+)Y)=S3&- zIG91)qf8QR@NgN~b$B&QM(W7G_Zy=`wBFLsuvvh<#Z&YX8H; zK?KCdU(+JhpS);(6nZvq9BD*+m601I?d=-WUb;z8=n~RO{jb-k?b9`sCyjb2+*d)| zXfzu7<@dGzuaZ&DkR*DJavWBQs~QEc!U3Xc&%u~H+n2r#^o1J3AbTXPC%Qrt*rdHx$(#;D-EK+OBL-z7(MbBm2=Lir~8%=hhj zGS+PbP*lBNEH(Mp3k1Aj1U`NzVLGWc4BqVynIa+Nz8E??DL+G`ER#?C9DHZ z>0Aeru)SE*9yGh;ly-e>@4nbdZyRXRh&o(^(MLq}kbh4kU`CMlFUG> z_GfzCy+?}b_U)?y8TzZM#rRG4GIBhS$qoV?tP@&78m63$6x^iT^?=>fT4yW5nE#P%zI+s4?q8PCr=({V4Ks5j_hi|v6 zOop3_jFK#&||JDSzP9k!Zl5_ac*CDg%ys1=63S_ z?6QAWgBm4!_JTLj;HqqUpS2*(WVN+xDL00!R2K^@?sjLtoW?8$({hfh7yxWHvsL5v zUZkanP>`Z%ic4`Fac^M39MEY}ryYu@N5uVC1bRLL6oXwV*kw2hVL30Nu1WUSYo2ev zethefi8KnFzGSZ;!@cTx^QR>DyuN%mvASMa4;&Bx)KTW8`aTwl#99d;l>p5G-B~oy z`nU9~CL92YbLL|%5`X)fRD4|dS7i%K>q;Ad6Lq1@xxO}=GRO7B8%0N%gQ=SkEV@Lc zVJ2F&2DyJ0$DbuqOX+>kA1}yDy5@TFfhqJ1YO+nCd=X!Zr^I4U7L92+xL6o?u_7S% z>yxXWQ_sD^V+byt7Q(I9L+$tCID7z(j=Bbbrdw_3x7l3tViL94D6EQyKtiStB{7DQ zp_DX;UXtE$2DR^}lwUmqzTDOtwegGclRkf)E9cf%$$Oa%&;P!$ff)71A1CLvrQbhO z;_r!}2->ILSH~7SplfWYB?>gj43T`LQ!*lYvzrjdT$fV51Du8<2Ih z)N-t1{)U@4e5cQBshE|LK(NNfVk{jJnzvNwUxV!vUo8p$DfYh}j62m^N7Wj*ccO=Y zi^->Zf!d{O`!}AVsS1WSo`pg0AF3%MLY65(EQ-&8yY;ck)oMKR`noyl85C@yWcV#J zgQ>CdLw}@@WxhPq6b(=Y|3h@QK!703lVfU9uOCZ@H`(TH0!Q^Zd`AUq9s-$Syscx2YX*L@#bHn>lVt* z-W0x6Gt!(15=BAxc>zve(2xz6+}@Al0)Sy7+;(-B=QvUvC~?aSjIG$t*~a55$*h)P z@_GJxZifXFhG`4ZbAQsKe+LNkkghJD7p|pQ0-qS!B!#xPRPCm6rT9X%sBdMZ-Z8}; zLT<)tnhDuN+H2LyK7Wq@j-4s+e**dHKY?84IfdY)22f@Xf=7BjK>O@uLBhYalPy^6 zbD^?$SNXJ*`&jt?O&tfhsM`csbSWY!>n!29#Tok?OkB>mTJNA(J$hVN^2ZgZSTuA8 znW2i2|A~!%mToaQw7yNS#{`y0-6gXVr7BiTA8^&-Yl+Sl1yt>KX;j@bVX8iTdJQ<- zc#GP&6gSgWE!J+Y|8hsz-~aUtHHwafGAI^I+~`gkP3TRE#QD7{Lz9bsKuQz$Oo!c7 z^bU!5hp~NSY73cq@o>i2>~>{2qHuy67?u#Ab3bM-qFu~xN60Wrr$4gYABF)^?MW8a zRyI)33j4dw@b8@%;5Q;dXP&x_L#Sef*yLoz+N)3Rdi%tj3bT+FKbk(S3lcsakTPxGk3l2#T3FZSC29kNevraM0;@NPF8MO|eQ+F(4bNkjOT_ z&P?9_R-UnQ$$n=Wt9c}VI{P@ol7M*&8t97VN9Pl==fi)x+08iCDB%zQ*Yes{$T-F88Z-; zONWXQAf$gi0W&~C;FSIb+;UQN5lfFC0t=iVw=E#Ye+kooVdEjV!D1P+DyAQecZgZm z{hH{Nm(Ya-7ZEAWpK9vA*Nut`s4*Sij&&@0Cm@)?6JY09lDt-M)7+|30WPeyeW5BD zxuho-zqUYz3u$8ew2g-5hG8)cW2!G;vx+|TOp_`wz;R1m=Z6D8#Tm ze^%3URL@b z{J7zKhq?Ch3lEwD{ZxgfX&_^urzL(Av|eL3k95d9MYNo=4T-|{65tA^>zIQ=-{3y= z?Qg)IjIyzNwdZCkD!Egiw=ICYry#sRv+7>qSDrtMuFIy#_!o6KOcbW3!#I6|Y(< zMo7AqE#y!GsqQuLsyN|?&g)P7+cIA{O5Vz8O6rdMRZsbI3pV>+4r9W=HUfw{WmO3E z$oDDejIzSFe@J;zi_TO6Eu2ZiC}9V|^$>tY&P66YFAx<3y7y-Phx`!N zD#=YS{=OhbXVhxiGZA;>)vrb4XBJ%<{ar0X3O6#%p)uL-WSpzxwi8-SC!{OXSVwVz&+v{-7DK-Ft>F7%;b?AE@|E%4w4Ub>eQUq;rVDg8q_eur^ zch0W%clSN4thHmQ?5LY}j5#sMsfe!^@zI2@;72ITEVZX;|2h}^^{ITN#TI~fVy-w1 zM&j!IIxhQ@BS86x2;Ws-UVHb#rMq3005<{XSbw$;064UGw?6-i`yav6n*Bv9RM=0%b*WV=0@1}&*98X z4fc$hAWiP5ab%z~Yp_6~4rvVF3@VA+GhDQY(zvolR-1c63}6#Oxo|CCk79_}sMS30>?9;&YW>|^1>)_w{) zgC;>-xJLN+M=8RddUcUhfjN3pXXW8u`|EJZgnoTP2L2rJ6T$oH22@uyKfu(137+7& zM@+0uQiY?V3b_5g3O51|;cpA8a`HP2QgO*NJ|anT*vJM{Z@nC6O!EB;s0qC8Mt(B6 z8m(V9d2TH^zI&dn+7VE&0q%k){n;P(&NLdeH-LHz@T)9@fKG~Qk&Z2U#YsycSD+<| zQiTR|A9{Wgx>JQe^I0L`4yTE&#-zGAplIv_L=O3Jey7SKUjdQA^r$_a3PIE9%bVrB$cgFT9^$ooA)hh%VMjqqQ^m3XelCa~fGIl>+xTSMa; z@QrEFt1GkAMb-Ff`p)0<=7}5b3kpzSY0%~v7E^*))MEUu>6A3gXrt~_HM*#eTl;P=W;?Jy+XXbAI%$*Ko#p)Bs^A=T9MCeUoPJ) zWO7H;IK*@;T2pX*Y%_wM4DZSumV&^Qv zsG;|-jsOg7v zPU->@tM?^rH9lb zw!m*NU?#J!UB!|61WOfd!^-DV_Auu9)W{;5a4z)HlRa_qmS=0rGIv)eK46DHo*ffA z%fZRSVlzh!5Mmy^P(hU>;obG;Bt*O!PDR81xTA6hXr7dkT=Sv<$jkx|1zZQyUwrw4 zJah&?K1=_%w~vJwC|UQGW_QDX3(Krwa`|rEXFqPH&s-7X`Y#~6N!2BC?Jf^ zKQY{&2Y-r3kHpZpFV$jJ9|w11GK9(%DWn+5Z2{TsR$L(g&&t3%_vL{%HjD2R@vW?^ zjFrd7mbC~#WK{~vvltlkc+hcGu*mo-bxSv(x-3^jo~S#g+G=Wd_@?9mmT-g54kMA2 zowtP1M=}hMmhCIt$f<5mNrbkXpRf#n-^EdSi}{dABiEwBj51;Nk9xlQ)A}Jvjs?1} zXI-fs!;#7cd3hE1^OHa4(ZQmQAga5FO}cnI59g74jc;v^G+0B1^z<%NTPu>>wBV@s zGlgd_dsW=E>;V_wI~l5!1pVTDSDOTej9x<+4BcgG)nSz-NtkUv2IuwJf`to z7_S42Z_UtkjwU&qAIY>^9T^m3l{MFQe04N`M>=UBi9r0^&_4ck83c{t4W)0SZv}F& z4q;{D{Oe@0gei*PpoHAawux2P9K3|7{{DIUm-5zd_mb61I&1rsgSt9lNi9e12c5pD zU=}>u`;B4G+bw~=PQQ@hE0|7B6giB^!k+aJqA8<@RtIA`&0VM!oi`>IynP7dMNMGNqO${WhumMlRkcjX zR0G!u*uiA%P*F~Ey_n(i&lM{@7KKj0=ppZ~W0MPlVf2?D{-dj;C)}W>w|^!bx(_#a z{-^FYB|6E9cy<|yzYMCRp`NOr9g_r|_aaLbzcD5u4~UX~XV!7*f=gvMOp$qVc#*-k zW%GvC_6;s`%rCn?W9UcEe%^^)4H)gP(;{i#fKX0ZNa{ep)T<6 z-%jMKTR8XtAnUP`^05BIf^)Ev$eqc;piNuTme!F7<6l%_AcucddS9VEwpkSiO*`!4z%O0s^(JORtHUr&4`L%@UQr_M<~V-H%@(Di{X&WBtOk-C zK5Bd4q~Z@;Y2zQ}8B6p$YqK6ay!(0a)dL@X&DYv5Y$VZ>dF_5Up?HRL0AZjVe5%^z z=j1u)wi2?|h=WUUjR{;q0e?8B+RT8k16AEywfxvWzqJ6~1rS5`z zphRfOz~lGVSj`#Br*{Di<|i`uZ=@bl_JyBqVZUO2Pz%NrjmYM(Y$s<7Qv0 z(wlz?^`|$}u$ta-cNKvg79(N5LrYh7ca`xhH2|)-b9)cz=;N7jnqI z(va6lLD?%%v+#O{($jPV8BpC_^X;*9=5Q#+d?9&fp=!W zVhMQN5k99;D+%5Wqs73LLWHIu2NU~1Kft;q_j?E7po7Fk;o*INI}n0-a&I?Xcz9@v zuUGJn^eED8qFW#EakTk7#KbD!wG{A4^$)T8gHPYoJM1%KG|;)dQUWolc;=x^nWQq6 z6xtraK}R~|9m|H1^t8KP@a4dYS7Os6&VKnB^+B6gq(HpsjSXH&G8AQR=vHBozoJ5{Rt=#z5gO=FYZh0Pn?-GlMQa%S-b!YLq>zmCGB<)@ z5Nlyia?#M=vW+uzhCsun7_)qmB`M&M^#6 zX=fzmnFGO%L|t~KHF&(FWs94T2F~8v?;d^q{s|i^@pOM` zGPV=zx_|W`jJzl#)4Us)hP^16-J$GdoFm38{Ee|0_I^ba^Cd>|lqPN+`#!Yw>#ih2 zNFDnxp6C~SI*HrbU)ha%w4rK4|NSWc^<4g8ud*3dVOXW@ z;;|(o<9HwC?Y>;8x8(LP@*FT@xQcQbk>QT$R(53)1)^v+63_(tK(ipx1e!H4p<(vY zGMn_=UH|+%n<+zbekS2WcoQ?1O*OOKW!j{#-Ygjd3CNl86VHYRc^tbk#~jz{0Nr=W z+=NI-#%}{t0&sZ0`^xujPo zzDgiKI&>DVNulxV3(UNHOfJzaLGBlz0Nywo#1XMOR%bHTTqijS_ipJ3tw%FNOb+K) zW<(M8$zc<5xuP3;r%ReT8*(0Efxd+@Rm|;ci_=0KF$J0nSU?7 zho7iwVCRz9uhoix_$u55jbf5td}9gq)Pm7nozns_HRtzwPikDO#DXvJ(*M7C@x0jd!bOv& z4C}GMu~S@t+_j|asLW+FQ7KS$<4rmZp{=GV;AWuo%<%pYUskbtr_A!TN<2jD$5fF0 z-CWC~&=)M$r_McK-8W^q$owEoV=s#tVz;w^@fV%JV3!FFp-xP(q&YXisJZFEl%NgS zH;w5#%d(c|zzg%J+Huw)prS2Luqb2xpG*IrS?~@K`u-#beP@$~n1LdhU+RbB_u$Yj z(yitwsPBS{F>;VhRs8U#RZiqUhMl)daS40OQ*zuVTG0?kIsr8%mil7&ezB08Z(xqu z#$tUm{@rNFD?2BT<>Zh31{yAh=~(k5>lk08DSL?#fhTsrFeKkyR35mJbZ6cfzHux> z>V{hz>j_oW60+6RW|9f$BMd>1_m_(Kzux(Q8<>(CU+MO1i0KlQzNbm?@@k7i&ULl~ zyLM?uR|uvpz(gU9iep4e!lxhA$_)Vrg)Tu2RvA~nDjoTk`~baz;M3~`}rG9Di^O?wmT?>q5=wwV4hN8DW4tm0Zizb z_MN=<%k{eawRE`A#FtcRlQrv?Ue!}GKN@#zNsl8%vyO`cEh)gIQQk{lC}bi+tfd(= ziT%vsHZVf^W7COCfoTE#rWRV^91!e-AUFGBz^o(Y8*(-qw{{yh#)B6xl^hH6b>ez{J3hpxM`rkVpLuLjv}@S%P>YeH>PqgeL>KWr zgU+migidK{GG#IKaUsObAztk5;Yw5eap?6AsIT zg5O#KVCh>%E!dZX1;o9wAJW(%;!K{GJbCW;1dXm2W~1$|P4H9lcDXO0W^{P<@uG{Y zrUuD%?K&Tve$*3Aa@nB{K@sqpj$oG&u6Y48LUgaWqCQ}j40%(yGU-2DevH$6;EnZP z^Y%^&pqYMU%b~ttHZQW#UmnQNKd{5QY+*ca7Dt*~{p;@ik3iM}f&b68@z-Qoh8Okg zhHlOhrboQg{={QASNXQqkr+2%9+)t+g@TODy{LlKbsLuQNVqGhS;+IR2BvAg>Ll(l z$Fz@chIcOFFl$zOr?2FqN45$Z75@4XUgBs+%55wal z#2-+_oNx|NuOnjkA#g?Y!VHl9A~{{6c`1!2(g$f|L85vB%(3W|6f{TKHU7^@Vu1uI z?ooN@l~57Do`QxWhJyP_!mLWHT(4iaaa719X2DF?n`%w~n>?Ur--GLjfjC_&q%9`7 z#fHHdUMS|3TF4@&YS{PYs}=-#6Ch&vT8GeLomCbj16JPz>a3X~ia^Us7hF8jO#EX>oA?bR5k3uvENkJn#(y+ z4SN=m_cA{=#+2s!>f(1nD!a-!0K>+Kv0e+Gp<2c=k6LnaPG%%IIVv!rco#7Rt=F3M zitJiUEb)!XGMKW8{?tZ6%%zkCw>zF_0^$2dFITIg%l~`o+Q>k`#B2H4t7G+K_lqxV zT!oSY+LM7<(*%@T`MOgpOERW4t=!j01h8$--gBi;WPvg2xq*!(Y8PWuP!2f^1(TxO zM#w=l#E2`l4at|G8g%5Gfq;N1|cLh%})uAg)t(;fj zVX0>iUk3kQM+5;FXx%O`Kl_xfT4b;OS=;`4r+_fqZoeD+{CEJ& zV7=a+0+qlr>I2C=Rs5JJtEHkyyc;-9@C$OP9IgC`rzQk55eM*8U-7BMJA#qi7Ppaq z!k|&d?P0hLMwW+`Y4R6EfHx$akckN2j)Q8tmLro-ej{2$-fN%J8z2?joEullopki-1G8l&n+`iHaNL=(4c5+BZf5sC$wj*^L z(fW^F{S$_${r$IBPXsz(?sjV>si~zl=aW`t1|ReTX0YaComY+CrtHKvC^aXj81(f9 zVh6?=_~m?m@=3kdG-jT;Hyye6ylRPMv?%t`?^m|2dC0rORzs`-0&d-8lv7XXx0X0# zAgROm^r>P#Ba~z6gtu&;#pK5hdHd>BOQ*}Ux{;*<)v8sg-)K-6HX4M7o!OA^XUt-Wf7O$ z9t?0@P%J1cF!x<44*T!605}H4gA|@#JwWH^BU6ZaPkJ7VXw@^F@tR%de!5?_{dYGU z+3M$~2YS#V?gu~>5~FNX8$VR-L?Ri8Q_RPcA?(hpqL3-%`eU(%A(`7H?9mKH??X4t zviCfB@?RO=-?;|{IFZdj#Aca0$|P$B_)=!hF=*9n;4Bjr{MFEUKabVAN7`mCuW_l{ zAA%))#gX-0jyBwk2k&h$TY1}ZXvdNs9t$O*)QC8VT=VMoQ1mZxEyP?#v=)g&&Y ztXlRB6=~CpDdNtRS$t{iovpEnrV>+zi&BbBA(>7uLZeusAipM|;)^C#Z+SIZbJ$U&__UID zUd`95hU{S|EUd~EyH1h{+U414o2i2`YuJW;zX|^^IgPQ#?2#cqd<&HlPC3Izajs`G zv%dQ5%S?5D?gZhAezlE6%1seW8!jo$MxF42lfiFDLvLZsaInX1L798*go!J@?0I?a z3bvkyEXCPCc7{UPWl6ySr-_~>AxILNTz(b9x|l8Ihseb}mU9-9{Mc{hIj5~+6jkAZ zRIP^bAEB+77Gy0il@nrZ@ex>~y1W!yRyme&_m<*xC5&OBeb~tSQ?m~JS9%E*;vv3b z1&cY<7GGME5ngaK)BjGzutgy-3rMZO=F9px!a&2DZYouf%|8B$A}xU-tnBj5x!F)z zAIy?a441GU`@3$b>)*O8OrSd%Ro<$b*Y$wPZZ~d zql3;^68Bi$5`+UXF0Fa%%mD%%>z+Q}4}Nwuh0%I4@Hp$&(`vYsyIKq_>dkHYkKc1( z3tlb5U#}b^Hc-%XZgO#6m{8(ERP5$M%A?D&HlaprBc*sJ*Q8UbZGc!J;t;Y#h@k4v zJE=vfmaUJTLT`=7q7A8;v311wnL6%^I3c~E2_1PKME?cjQG&O$ZjA~aDT#TkCTOEqB zh!*K!Y=m)=Vu}-!ISI^<2jG&gmM7M9D^l<{c%wc3k`G_2$#xBJ#nlqtRz)ABmLEHh z#jlJvpRuEW;_qMDE?!5cxQex}$r zY>}}b0a6MM6(~JcDwa|H8UcmsX6A2z$<(0zrdC{Bpr}hvi=|D%9s}zPj7vC7`Q1=8 z0Dd;&bVA0|xu9S-5)4c<-(`#S%WQG3j>J|*1Hrz($BlIlJZtk};;61ze^Rkm9$;}N z8H!Vqy`D&GF~>nu(Q@Pa%;N9^g8+MbSZKf>`VB}FoZldTj{rz%se@zakYJ}}IO!tx zta(76{cA)5sTuqT-=BA!=YNY6x0eM4V!Xy=k|mQsUU%y(`?~_-NCS#Fqkl7~?Mxs{ zkWJ7oo5d#|zWdghy4Ket-h6Qz!|P79oH|>QC}5N@zjjv(3A)U1xANg~iH+VKm9-9r zv1f?XP2^O|YCFHO+@l?5sQP%!zi*2>O-8eIy3e8}go6q+>??^5VYYE5Psrw9=%Yr~|?V-U*O#%sW zS)cf-;=f`_!#g5^_K`S_J?3F2Ts$<{?upOms0x#{@X^i{l~wryO%pK9q%GDlIGm09 zfD(@BG&R_Q*(s?yEo5A`Vs%n#(wwokl)QYQNaU&nP=q@je9eJSLujvmd~E?}O1(F0 z&KnOvmTc^Y?6H0@l{X0(w`nu^pO(lb_ha&}0y?gk8woUF?AL+F1mZ$RiO)(X+V6m- z7JDn<39!uqQkgIl=HHzc87KZ_d9s;X${&9NRFesz0m@s~+r1>LXPIN?S%h76q;`!_ z|Fa*NQOE7L-7%oAeEb2p(0zJ(xL$GYrJDMk1u&@LNE}swxboI_7qEmnyV;4-QAp;; z>HwI@R(m4}naN!-01|uEez1m@_8tJA8o2*#@WMlQw)Be2+8KyQ1AxMV3n=V%kNs?3 zXCq8iQ0D*%HFS)*xUvOc3emswd2kHw=C_21ct|IQ4B)wNlXZY7$KeRGey@J}K#nf+ zV!|3QrTl!;OV%^?I_f@%#3}ijOdn2#62%&E9AlcAUe1C~a*ws=;$5{Y+{48sJBfLBIinmEt^%X)+V?DVqsqcw534q1h!pW#1b9-s zst;^lf``0^#x->6f0xSyP@%J|sg(#*&5a3U?vrGDxZtwxCD?({l;(dYsWMI zd9P#|O<+@Pxa6t+)0M$f|62w2t>q zk!>@Dzi$JbpI{!ppO?w{r&(1Q5A071bn6S$K%JLmn(x1Vfm zJ<{hG|Nf08RS&r<0BaBdC@KDIExVX>Y*fRPufF)=PJSFv=Y}X7tB0F$aUz(TfmHIO zBhHaiUh98;xGK&`@#PFN>#)?~P!3BM{t=ZAt}SQi8GcpAhii~YVZp9r<|B=aphMVu%XST1&43`3+@N_?jt)@h? zy-RstS3BB_jZzjz+Sn^XE+meMd_-^w>(Ov8r8Jozc9L5tfhPu|g>fDI$^YjkQ3Hrc za!hx{dm{@*?pzDIF)gFGRIsb*g>8g3%R}s8jCqPy{C%jt>wdK&15Yxt5qBS?dJxlP5I5w zdN*9-iJrDX5WF$GA8bT_5s&XcDvwnqK%kN}lDE?t>$sb%T8_W#dVyx5Chpv z)juc2nu2h6H6EMZqTDYgmghz4xGbmFwD7Yb=Ev~Cb@vmJ&NBSJ%6kWnV2|1uzV{{| zGe`L0@#U=qYTxw_LpKcF-8FP~N=QoAkkZ{CH4Ke}AfbdHsB}n6 zcT0yf(nw1P_}jj|-`_cBt#kflEf(zA^XzBe&wXFl=L)sg)EFXDr~#UUM~tZxhh%Jv zKeuyyauqVARqbeR&NsJYna8P&VEpd3{Dk>1F0&?lmr$=~2GTQvzeanPD8$o^Jw;c} z3&%I^rG>y`49lH*^n+H zX}58;7U)RzBmFtIO?2a$Xt1wU=+pmu&0g@k1Z zosn>{jyfMI%iv&9zRCwNi#l%GK0{a635`%sg-qSeJ*W+dsk6wA@#nh@y{-$SBECTO zOI|X>wL|9x$uJU6Fu0~qA%G6}krm{1tdpX?*_oW`RMrwTmygcI&102@hC34lg zE%Ps+X17qExp7!c*%@G39XN+y(Ew_z2Ko=(Ut*pgzs0a&q=|8aW=xzT2{mF2sKd5V zkV66F`CIbHCcBl$bg3P}qet&28Y(Zd30gz5J+v`5Ylv*$+I&mP7#SZk6ZgUF4uWgY zF=^m+Tj&d;*A2h-+a|Zvtp)kBB)&w|z*A?`=V;sUP?7`%58NR$3@FCmGk-sMaV?>c zHZURcJET$EdcITu>f&VcbtHFWcsM{>6gzY8=#AZ?R-r0-piP~{&wA?$rOl@L)Ty3) zCcho7Gdcd=0tX;*)e3P4ge0Fl@R52OlQt8=iB+fVKrn7d^l{Imxc<^3v%T!Ki727{ ztz3c|mcB-BE)17NNrl7+iDT zWH(sCWj8d)*6Zvt#uJOk0!jd`6MXYI`Y0yh{B~R4QQr>9m#DadalBS!0hr)=1Z zW%SdNl|bliaSCNT(p}mW3;=+vWJ7voUBn^pi$VmG(}w{`-TRqnG+&H=y1uf83@M_H zm)UWKi#{6nwo9F`ffDA?WHRV4Wi&X?4K~ju42^^uA z6$v3~Ad3mR+|0AP>B&^vHli6FDkEf)RggDaac!{dV}k9hu}l)-c`E3wZbI41Ra7PX z9P%K0y|>8czRY0;C2d4T3A%1(Maz{9Huel^ocvk=@k?()u@`w)B0YsihFCjJ!M*;nLG|nJN3 zTpBUkh+wish^{4qp=~O$z>_ME9HU#UjjcbNJM9WZ+JYTTv#Z^0HhMWoOIXcO+^O*7 zb>A#Q&eE0G*U%U%jl={T6RIW_BNE1=YgM(30T3g7SVcS6yKh%$WyZTxvSdGCrIRSK zCZ{zZG(nW|)vq@NNRbjgGV+2ejCnHqA9t$49-M#FW+|fB#N%xiK1=+>7YU31wF>1*I_LRC z*cG95^%Px1p|sXkAGyByyQJ~v)_!W4K}9VITEw5=ZZgZhs zFW^l`&3K%3o_^pB6|%M^3-bQCU>tP!k~g6Myz)Lmv%BVje_|?yQ{#lOJo7y?abL%(%J{!kk`gvk(!bzU@=*{Tni*^hBmpOscOi+Uk$vbZjCEBj$yDdoff^Sv!~Jefl%uj*U7>X;Tr6 zLi@g{;hIHZO>MnJ*sN;-D&b^%b+Ir~csM}U70BngiV{3C^Fg>AdC{kC>o-FD z(%mUBTTB;OA5*=lnyX~x4Wd1tk?(p$O7+`J08J!yDVw!AeuH7-6@84dK_G}b4l~AG zD8EVQdAGvAXBh>mfvD>RNEGa_Qs=!7u6)SKd~)-;4^mbFVFB+@S_?nvWmL8uA9yOO z$5zdq`Xum%>jO9{pLTjb1-nkzp*@X$;(a zCkT~{^ssPGTE*fI*e#NDdHXinQX(j86j4Okztzms~FFK?}Kt2MVd4 ztv1UP=1h2H)^(H%UjOgfu5TPlt(`ppZ^07kjM-5*)6I1-n7&dDd=7%W^YV(16oy7T zSi$pOjU*?Q{A8ph1cS1&l8FxDnxYrxhG+=)C(CMf5)qOR{Q-ekezFo*)HG)% zz`$CK{W)iXK9^5bh;gDC6Sk;A{FE*mqbVAbtqUnI$gM_wC`=jjemmXYUOs5AB}J(- zHff)^wu>x!n&tzj?ibXHEc=NvN7~Kw?(OKU6k@su&kF|S5P7Vi(cz{ML3l>0j2+^- zyppNal)PKuJdq25f+Q#DM3Nbc@aAniJA=jD1hBY22zBo>w@baDa4?~)9DI%?IszDw z7r1*!V}#mypRtYXG$Jn)k{RPOtEGX5y3yP+4uPn$E=F`>hsS(>5kIxZ6(1}q8VWKC zW&^&Ep4N{}sUn*MQOPlSv^AQN>#hC1y z(z?GQ`xO;|La&4Hj<`6%Ky!SKF}+677;cBGHUyKEMt0(}D0y!O&gRco<{o6tBM)ps zVpBg89{BE@l@^#8Qd`i~zASF`D4xsg=g#a_F-rOapo41stWI?; z-W7_I=`&d|I4=*p=W*v$m#ZuCGQc}8LGfcMfF9$zlkz5u;KnDsC%7ZBw8&XVF*>sG zB{M$lIS5QbZ;Fq3v+uY8ev5jJhaOLJ@15{_@P$B!)~jxDLpBqg(NK6kkA4JXk6KkO@i_29NKT8i|#3bgNHh%$;GO$tP~ z(*l`5#(UwS)zEVw$!~-37HbU?jKprQm0J3N;vsX1AU1#>_CC4%{g&e(PJ&b=gqe9t zo&4n6mUglBi+(hIig|HjYXE^sb&yykndcV}udyeFj?aOPKQU{1LxCygUXI{7Iyes> zV|&hqA7#GUEF3i8T>76g2n%uuEym9Q4UVj)+TT%u1W^eR!qs;qlyPLCWZr}X&i!<= ziISQDETh7a1Bc+n0p&;&_U2hD={|dwBae8E#M_}tI+6w>Ub)-OB;zlm+%!;O^Y=5| zV2ArdR9#W;IU{hr-RI0zIn!)fr(X>VpwC0 zTM_75BwE^P*yc+gtGLMFKJHno!B{5P*x^P$m(BN56fXm!${AB>c19Z)!7rJ~{e1HR zw?PN}@Jv7ZS=_ zu+A!ws6G??DSgLkeTbF6k#T(~>6T2pC1#49$P)K;_-uR$(WY1Q>)<&mHB~v(y72S4 ze%s%{rVk&;gJuDhZ&XKxXwuBUxFpgg&uXyxqv2}mAc2i)Joe-mS#eHksbLpL+V^Ge zg}{7qWmgF1s74CJ6V&fVC7AcEZ;=rPr>Coa*a*!4hxs5RG*V5V?<*r}$0EtZK~(at zV^*c8@nI_YpT!v7Np%C11t{J!oaiS6XAgnoc6+bfntl0m1gSqeu~f1%pud{s&aStK zfF0!TbC!uM^TAc{P4{XfQj{!vGs#Gt*rTEVlCz_d1>!+_IA9IJRB_Mg6(rlmK*XuULAZ+ z`EBC8AsU){atU*4P!8``+wXY)`vabG(xRaKLZ)k{P^RfdX<<8A*^zq|Mmjg#VuuRb z2Dz!bH7SOipI)^oM=D0v((0*v)^gp-dv>Mxu{fr z(F}vWGn3{v>mhTcs_nMJs$t%x+a~!C$N6-o(|Y(XQX7xvO!Qx+hBJQnVRs(QMNlF_mUiHmQ~A+8EnE*fJY!%DUGQc0g@K>{v%4u|W4snp za);!D1PDnC;?l*1RW~rg13}{$i-O9v6%v}3A*sXG-5CiR30B09wm~FMmAXyOAkOt3 zUfnOcy2~1@oiP=yNk~a3+KD_PrftOEn+mssxNA&XVvvZ=jy5ChFCNo3Lj}Lk`-t#@ zPI;QT))~V!jcil1M9O$nPbfJlZz%*Lk!;BliLj-3l}7HL>9W}%OGS8jW}^?mC@NA- z1hOu_ek$tFT1Xk7ODRkeruC}D1+V2mfYEY z<9_p5@9of{nk-B1Zz%T~is>UpCMzNe>v3`2DtvG~&`7Y8)QjL^YCQ+=rfEv+yz~|{1=@=Zj8nH0zY;*kezg3 zpTQYXHG5IZ2!S^F703OdlV@s-!hm^}^i%3OlWB&*32Y3gC_y(|9Yhr?sOpJZ?+=hOvh`ulk`jvlJ=a_sA!n6)X`nwPShNO3cXnO_ zg93pNGP({RyEhwLK*R|Frz&v&C?${57dS81uVNDx`1lNk1v{Xx-ci6=P(Q}+0MVO6 zhWFE4-Qreuo?Pq_Q`-+cuE4CxpxQP<`>+(*4Iz|rk0!G(Kr~Wy8P)_0OMM7z(5UeS zG9QUMA8TJZ?BoNa5|`W7m@)AQM;?WiU%lD{G`(nrKiNBJK3uL0F%K_BmXqDE^~tFD zUi%Ac6_ZG)*T)wWb}J@EIqu=z+hxfnmV7#jM;y$GCS>Gk;R!ZM60)6?``yzt*YnE* z-EnrP8t03!#)-_sm9!~d%;aBrG%Dy6TK214dh0T>*jne6-??OlCxrL_38!*qJ-Fz3G_SI3`m|!KpOApEjGkLFiA4!FrobE# z?Ud4Vz5ym0N0f%8nu`7(X<3X#+(#saDaIJ(_K9*`3DQ+iupo3I z9daZm4qc&gVn%p&ctCC$I3)ppZkxiICxFSA_BG^N?YIV-Y{wI8o_6`8^Ah95MF*bh z_5Ogvt+z~hb=LnL9is59x~=Zh@u&(5#HCb?U+St*9gtVg1YJ>G0B%J7BW`Q(Q2 zb7HX`8!IzR$j|`$Q-;(;+fDCxcN>j;0YaZ>4e)b2?1AfLcbTH{^~ya|W`pTg>x#ev zt>=rdYo3vmybk@u{L#*lh@_!Aa*xiLYf=Kwsl&0R^Ih}%LT{WYY(WQ*%7 z2Ri7(SM5AmxK1cZ+JbY-+=o888SZhmlaXMV|F8h~d|ql>mv3&IHci6m1)iM(n=Qbt z4c3mEoZV`a)D%JC(1bJdL*HE18CHFPU*1iodx;3a4;YYPvd;MH%eA%hYPfWlo2MO^ zC(^!KGgG$^lG#Kl<)}bUuKKK}eCW~6jvdRPh%$KLfk=nqC09^t-D`-4^ie$aj6}2>_I8@1a=>1b(}3{c|$N)a4pbNsTb= zU(b<>aRnQZQKs*{+={dT$Q+cRmSH;(WNCJQUvxX^SQ)V&+}{gx{n{5GGI+=Nq?!JTi{6n|)l zq+H8`aM(Jo>W6>(#anz=3$~W)@2@SNs%?J3Q}mkF`gi_dhFtJ@6qW!@6$=!_gmBe9 zV=Je?5RFf(0K3rS@Y|`0@_4*qH|*+Nr>GaPtAk~ z*^a{Wu9#ovYo<}irg_!V71{Sg)eiD*GDU##8v zVc}nO@MTzcQz+FPH^hh?aNvqTe5!=E_5G^Lu}o@^sZN7|@zNO=mio_$+Sg-fl7sL+ zkvg@}L3~Sz#~l4@1+5M;EL%0qR>LS^2m-dt!dU20eoJH=b2mv68qAQHENb1* zV|5Bwm^G(CT+t8}5TvnXh&32hCm1sGGiw(tY4-_fW16hpVy5zc_yEbsA!iAeK0KIg zBs^x%C0_B}VrEF#65A(~(V`>GU1SdZiN|O|e1n+zNr&{!Z$AvU9~=%G=c2%?mc{6*$UZp7|^(59Uwv>(=tc2W_Lm~_*m ztZz0Hs@SB;zP5e7Q%_hCrTMDkNG(G=w_2m*GH_9$NLiQO4RyB~xGuCIr*b8Q%Zh^! zXqsr=p23AFnH&;BiWc>sU0L1>h@4|s$5Z`8?AG|0q@;|XN3cUIb;8+ky-NEOWw9Hc z+-dR+)9yv$Rc`c0X>Rq<+o6%}>s~Gvb4jXkqXkKV&y6;%`j~%3rLxB8r_jblkPVN* zvk#NKcW6QeM)*Z!LF79DqlSgDA>}oKZdE;H#m5OGb`*|FV5olGOlW>s7(>GkPBypPxGa$)Kd)p@ zK`ufweGYl6489}w-g2(`Lh>Z*=IigC#*dW&2bs~Vqds^BPCmo}M+i<0txCcubA6T= z=M3RJWTr~*cm`)@FGyyL_l+r*9lH9vgwvqw{k5;pNRwkSd{qzX5p?@CSJj5U+YG*m zm?s4M>E$MLva1a1Tm)HI(uuqnOQ^r{CJPchCy(zxXDUrc{)}r4rfG6m)jcRtiRgF3 z%i+?%$P%r9p4w&6Jpvm-Lj)YsLaMBxrBS889Tk~Owp@SQh>8439nO7H7oxOAZ1Mh~ z;RuaTxcXZ5rOrR%001N2dbN6dZ?U`f`FrI+kc|CpXFKmfh%9JlJ2bd+zH!U)Ls!?T}G|ve)u~jPnLwuoq}G z@+&U&HRDUk$OK8;T57CblDad{sx z$d@^eoGb}c@+f_Jf07JRsW6fc(3J7th+V`H_Qe(_(_ueBFodakxZE@4p?WpO-wQ@2 zwTA+^X|;O9RA4EQv|1F>6Vm4%v3!c?j|hcb*v#>$eEt;}1hGn_8S*GU*MN}e8?oXV zk15cy!=tjQ7WXKWE=k_qR$QHlI3}+qz5tgoOpS@wZ!ph)HE`VAII=Aak#>s%iI#6K z_|(-$3|1TyT(>heH?=huN7L;bSub-tq%}zXfO3M1a@*|x%I&R~A;EZADjr$wYw~4@ z@wIoHOQuiU^doTle+Fv3sflf?lFTP+VDi=`zl~z*hiZ3U8d24N9g_wPNe|v4(^*-c zCQnDy^fNL)kh=@C-{zpmMvXZs?N$9feCy*N&KCm_XdO`TctB9$?GZuWla)bEd{$=p zfcY9h<030Bm1ZRguSnQ+PzW4Ia3(8&`q3=Vet?xHB-kdTR%de*)^GP{btqaQa*1f) z_T3;oaJ~3R`Y4Rrl$PZMpeC`;rATKhVQy!PUo^3;GL5p{Hql#m>+;j)Q?S{k<<}FR z{^St>kHOCOa!A!&sXc{>S);+nS4^0&j@0xJ5VIf#(dGVRyxOEhgN3HF&~=-nxS)tx zX5a1LS!)sw{IyEHitgME3Ulkp#L@cs=;6R-yO5=6gGm;{ zX9bBbaI{;pt#!5QgyX3UEZs#kb9uynC(L0E#O3OQwDKk3z%6(>>=3a-Kgu~MqKW{g~sCV6jqprycvtmNK`59ZQxFJam}FOb~B32 zwzDBNe4S(ci&5$(if}W*$8YzwjmSc!Q2Yi)@S_KT1T%CuWc0w98eM!5+9(|B1XC&P? z98ZY$gKF`Mf9$M)<&Un@4Pz(2L95$ediE1V529(LOXgr6Yo)5%0>REtlI&2Xcw6wJ z3s7-$PMF2 z(vS#+{nIwc`3aOL_hi&DY%TQ))*Hk)zVoBu?#o)>vy3vyy37%ciiLKY84dyhBl}|V z-O7C-SLH@qrJYu>Le_nmef~W+M)ZOl6&YEFa0*gF5;^dDv?>`&2B=<>T^OUI$!)rp zV7{~ByCDjn1w9Ry%+H7!pE&i?gEB7&E+F2We&wn>qmQwK!Vu;H9@PS;KTiNr5|&p8(g%> zr!`)`-1ox_s_IO5qtpN!C18>A2s)WQ>vqcMGv3q-%^$Y#v`2UzQ$xzPh7vNFji}%v z%LJh*a;mzK&H7TZsAuidjjnR%;R|c0RKs|AAsyN%$Ye7%w4s6h0!mQqK&^V`!AYgS z5`*yYhqSJv*H^97b}=E7A7X~U`T84V@^P-Q-EKFtkWb_XTEtci7}^Vez`?5}D(}OD z&Bnn#64^*nW%Sh%ilx!rKiT|~fv927t=g?k@y3#x!kwAHU8}}VXfkW9wUevv&k?=| zIc^(!q;Fp>l_cUR9U)r9DY^Qp6TS;Pd;oCJ=X2e1S*>7nZfMfe8SBLC6JQ@o-8aI% zBqIw>Uw8Ztg*Ep2&|?|h@d9Zzv2xhoN+pJv2ptf)`F@R0q8O2VA`jd-ep@az$V)NCCMfZy~vok1_ps>8VTrGXZMYCVYCH29)xT$}a!+4I!L%w0-Ef3aVr}g550BciXYn<}H185T zzrlJ3m;F*8KJxy?O!|wTo|TbhxwYTgC&+!=>+uOXo;vy2epEZ6tGlp5d%upx?wI3w z_;~yytZi*8^CHzWx!0$N{5{*>GZB4tz{sX)u>0q!b&8h2y<5tOMvq}jVQt9ZEk(GH z0HsoP>5dRsjs0ClOy)P%xuZ6*t31J zvvKPYB|N9{H9N5C!a$hZ!0n~EJI z8OXQTFL-EAz|o81xOS9rVA{q(Fd92Ab8y~ZJ{B$SyWCRio)@A(Gfd zf;T|DX88!lAhZuwltk34Evsz7WQ1%gl*Ur6Lg((=;@C_)4+a@M99?X##?bnW-q)E} zXftW+h{RQ0zgR(8=`i$}Zdud)r##li1cf-b9beIt_YXnY1J7&hTVuo)KHHK_p6d9$ zW@n!`$fQT0?^0@-3!2#H@%!jut*7|el?E30O8z1Kb2tjHg8a zU1;Oxqj;|!3#u#1u4DAqNW~Ewjz>byiYIJ5W<)R{X>-?W%N|8EEYyZQ`_`48+$y|> zzF{{%t+ccz+>oA8I(1=e>vl&POFJYZFm!5#(- z*kd`{5V*>mT>Ufmh@ts7f*h~tdAagcLR4xD^E8QPPoDPc8!M%d;OrIUVRO|UX`%1 zd!_MDA00a);tSd)yQley@6Q{!1?A=doNo$GAc3dfZO!zUw`BCH z{I))@>>v>KOZ2HA@E08)J$(mz7At5wS81LA2|>I`;MONzkL-8@Afm<8U4F6nB6oT+ zltK$7w<9UH@Z$$Dh@L$k(?(rN&40eC?Vj_YY`zMYP!MdNn=F-l$8&~7;PaMiho8{<#Qlx>#bqE8F>{Y-1FO5{Uq}aVbW{k!F`hqE;K$yoCdidJC+qBu9YeQ2 z(IEJcTi8XmLBalXqJ%EI4@2U4mMrAtH=z1YrVKSG#(v{cb5`SVe{)u@vzXkAXl2aR zL#9c2`O8w>YSdHI;@#W|A(+I7bsq}Q{GgjNL6)fkiY}$;k*7ejn0`4cgn+d+$ zj-b=1;qn^HSZMDsnzcc0EfJfy6vm;=0QwN*(ul~)`Yd9Jy~gRgJwQSsC+{DqJKiSr z?PbEcA{YnL3mFr{cx5KY_J6|J@}3@yOv$sGaY`eqDwWFKQ7tlG$0oQTTZmC%uv5YHW-Gmf=N1R+|IM&EiZe| z=qSaxH!o${uTE9p;Cn@X7QLF$!;`wywze8H!h|=P(AwK|&)#`0n_3g6xMP91D@FZa zp1e}79|QET#vX@7o?+YZ{P`z9B~g%regq_O%@UPM6rYD48oUK!cLo|KKL?;1kuGp> z$tZz$u-F4tLgfxO5>+|2BnhGX_wcFWS3rU)|94a->amEo>*b7zGrm!@y+);x!+o*2bF8gG7tj zSLh=Ax+te!3uP!Ki9&-Oo~DLYRJGJP*aet3USotVEo$v1wJwz z9*F9#FW~-g{nkWKizxGy`oKqAD=DJ^37<^dhwW!UfLgB(vYVrA%}k8)T}=|h zQ=!~E`h6G?KFHdN9`8%Pt|;Soe!}>SC${k9G|6_>J1_cGbfX6$txV55Omtb^t{Bfw z$xNeDt?{mUb75qKkFmyNk#g`eQfn8$Alh4R-avv5M8GqOiFxpB>c?@P+%)!*8t!eA zrx9CB+iod>O)herDwgN!fhS=Nz1a{^lo;F$;nI!XqWX173B-bfX;3Nm6wk|RF-B2V|K`aUVh8vlscZ|sqCg_N)xaZ zsUzqlMfF1XdN%<3cVL2XjWvqHe6;@s{MYwT+5NxbfPacLy8nG0`XGae{WV7AQh5QstKf7(e@8N81tKfpjUvr=m}@+a^CorY(;hH)xj zU`pM^1}<^ZCeUvI{O)=1$(Q*mOPM#WxMUDIpW$c~MX<*#*bA^rv;qecr%hdm$Tj+xdfpf#`+Ntk(AV}7wyy+_e$K}Azu$`g zdOlae5oSPCNbvzK-ILw^3K>iKXbSGY6#msqJG`*6A>rt8z_K~tXF9=FBQP$CGy?hI zC&Elp^`erDio(|^D*E)7dEK8Y&~fYuDnbIqUV9=iPC>ww55!vt7~hA`}iOJ0VV-lZuy6! z@E4|1us!fU7XqNx(TA!%cIDW!ISKz>EuiXyX)FNm=Vfn>)uWu8+}^QV5j|}??w%AJ zg-)%oyJ_Up^hKcN=z$BUjgEt(z+11lih4&!`7}OySRgC@^^g78Yt3UjPpM;qD-0^l z_n|0%zW};O1PRpc(9ead!fCqyDro3ej9COSdi&U!+#&wc=}G2+0XTq zBCcj__MSB%m#_XlF)(zlK-ak6M?ti45W%L5U%%FVKgMZ=bh|{g{lFBAMKDm?wi7TJbxf3>*x>*F4D9HM ztNTTP`)$uIw!2!1^Z}F>K1hl@g7opOyUtzu5Z&h*TiCYyXojt-36#Vbej>-eKkdL- z2(juk@INEowK)}8XdVYM9!DoJ8dz&vXSK>H$ix4TeoXnOSxFE%9!dwReL{Q;|0xp~ z@vR7dM6^=vJ^S)~(R8luOMe*0zgI^WF8}k6wQ4}OYx2fu?Im@6*&xIh=g+c|9YRmM zy&;pX&fIUoIkdkIgx^%B(Hr;yuf9trWTT}E`YVc)*6H>hKe8Q}R<6#k+TOe4(X5>> z^Rue$1$hkAY@Ne^xM%sSt@0cl?YKZ1k50r(v>=qwqF$Hf4B}S{?4@t`9jBPEX;Tqj z$5F8g1ECzZPe&IR7<%|YpJ%ZSfO(G>@Zwkjyq4;jTs5T9hdpYq$q^H!pbgFFwl6-^ z!-!e{QV1|dJOc(+79B6Xm^2I8X6UfeL0(LpRr<@$CUhJAgub10OVa@cvLy7U^Vg5V zF=wAULhBbKpxYQocdjeAAO9`-)fB_E4hd*wx&?Y25%VpjyJ?F4Ek0}&5pt_NcuBSR zqQ-7q2AHETdL$Tfq>k)8lIK0o{b8N<ff)%LJM)vxpKBS3Yxv#*jw;d|PJ1|a;G zZHNXFIsu3OTF8ZF-V$KtCx3r`cD)+x4nXGb?_Y=AzAkzxy|k4}9exB;4cp0n`4q8G zQL;w90iYl}j`5h^VnuM!g?o}sZ%X8MOv57ZTF)j#lBxL5CtuQvdz}uxMSTIVB;-tn z{#gajaX?v&RqU=%5TY`NCle&S z%jSvpHEbe6o_v)y0mq@dNx*#FK`jUWN?VW&%=%-l`hfK7)3-*L6s{A!Z)Ce;{$v6H zFMksP8TeAb<4?*|%9KIlUUg(^!FTgm zFi)%FMaAJ8fWJI75%ze|gzdr#)GMg@BS6t4_sxNdw$Fza#O62eYIgnWb=b)>n**wB zsbAcGw>&A52w)HM-5iXQYPkE<^v_*DL4Q*}@nkwvGk9x7q>W3MS@9?H&A-1dU%&%e z`SKbIFTlF{;AccXAz2dg6*{NA^p{U3EMJ{)&`@PyymBp=nRrPGIv=S{-wyU|K=mhi zm(WR50r1}m#d#IcsOf;nD+V`H{*9Fe(%e<=AI-b+Qvio&F&Wb(L2^as15zxvgAwCX zOIt!T%h^oVO)Ewu`(l+<|9X0?$J-!SJr4As#ZU3E^0yq;5i z_x#V78je30k01;+Kbl48qs}oUN_25>&lcz7yj;l`|Hiawk$`}C0nmE3?nYi1bi7lc z@W?JBYhTy*JF@~dbibprHsUYFAi~sn`c|t8a>#r)H|j#M`{Xr zGqZXf$dnU_5rmgN{_yX}y(omIRTqn=e6a;)Fc1%2_tA;B7~6T z;$`LV^M;-slLkxDfc`e-kP(Hw7W3vA@cg?4AkYtw(C*XZ=lYtgsw*D>3H=Sw=BJ)Q z0fD8&Y{~M#5}-iu%-C8|%g9p&YI^U8z zeEWyHtKJlcZ34`ojc8IXK4EA-Y*DLa%t88sJhA6FSNlmP`ix&f2T=sm%nZ{;6!>FN z=_t)$nnSTu>L{cCajCyo%N(V9o7|W1Xz8fYN6{TmeYEW@{L~Tvq9Mv(ti$#SyicE~ zY^B~wXjEX(SWG}@=@dL%{N25RhX|135)yQZV$UK8?93h-^8P;b3SosyJAr9+ivM}s zI0?uuYve+}^qtDbf=&Q^vZXfp(?9R@f?@Pi6zN~#b(X=qpG#Lm4zBzy94uNMA1V?g z%>P}=ScoA(?r*z(h+1FDB(Z378k6=WU^s^|TC0bb5M@v+6-_F13`{(&5s&PN!r2Vq z_Rf6%&Ek;HMKewilk}&D-#apHrM_lhi-HfK`B+(B^;G0nwO=?nE4tdTn?HqfTeRKG znYyD)0tqBkK9h3ET<%X8Ysed~%E{TJ1l>&zLs|t7W4$#FV;4A*Ufft5e*ByT5o&Sr9 z=U2C?>I!g_m34%rUakNh|q?*n=Ti>ats_E5NSU2F&YO zN(m$c1+}W+p>KTHoaev={{b-Um_uIK0)hxc!LkuTElD|o7xOxA#eENemrRV~afYGA z(28n0LC@izI`-n=t_~R>d1z4wAJr(t;9jY+DB8y$ zWFga&KWqaUEQ-1b=rfACc0nM-w6l4hruTmW+o1#1H!FV2IInJo?|;L@9`pIMswku> z|L{N1(Vlq!VM(lSJrMS~wd}KOcW&aKrW)Ydbx)lE$H*ALP1QElHHm`{MaeRc0l!%? zGw|N)QSGe&w2(q366yWo^gXZDF(+SjlW!ES`lasniWqOhx06IxZ;t9Fz+pqouTMi? zGEGjHp+g+)Io(Kq=?+H2!T$sKSdWF+!1!E6zO|EAo98e{aHhOu@qF-r^1DrhMz_kO z$vb^Hq!Io}(oaxO^q+1(1iB=^rZxJa?b$A{;`Cjw%l`ArYW8|f%VCj@NUu3CyHBQS zlR<4|Tjck0v+p~xZ)4BB5q4JK$TC~w4%~aZ;qh;IvuK!QJx9_ zwO*`8pta;tFNGU4n*Iq7P=$XLNcYr^S!6SE!8Ze|TFk3lfR*rdbCR$MR8_z>!+)5>W2scu!YBvlAZ zK){|yLxF6u$-@!7#=Y^mopV!ZgL&%Zy{#^;0J6vvH7`^thMGgnvF;;wbb$q;6As1h z&rK(a>RVwSAAu36`qPYM;K(b*uTB@7XCXu}B2h;QMGd^aG^Gh&yRxORnnKvoUlVT| zAT(<7pf9hM>z$}QFD#j~N`XR+Y1+4vCHCT1W1M|Q=oR1#jQ9a54g|sci4*{7;{7V% z=#ujO_sH@;&hQ07=mi<@&(`}mQGGyZQN+8DfYF&b2S5R1-g7P|V33X|Y5`X8NlkRC zBkQ1UAQ4n(yvd)Cf2=Y=OjWSrt=Ojg*y@q|$XLE7vNu*1eSJH-zIJnTgOmu|o?|=~ zn-=ax<0}%@oJicmGO)&|p}rz9GyIuTrui_2QW#vbn)Yj6|UO=S|EoI zCiqWrABijkHa7c!1%d1oAg8v=RKQrQB`e# z)j6nJ7m2eG@eG{DICqsh0WSZ}F18o?>?kE@vPYEujAv{T>#ZIGXHGyh!zFmsGbC`YvzG%7yz4#GOJqH**Qa8lF< zc!}WwN&n8o5&xDoe;W-EU;?X(u{@JY3^Oi)MZZK)@}RZSX~7wkjbRd2@R2R^Wgc+u zE|9>N?TY=B(gJAeH%$HYKj}F^#kI<@&BE>KxqrZ-^xUD$Y9g(N^S#(l=i5?058zdnFJ)VN^(mAb+9?zdhwg+$!WB znAc0ryy>uxQL`seIuuMWrE|hetU69mtWz5wiP(-ie$%-0FPB~Vf~F=9q9@kB6gRO_ zA$~5(@CejPGQQ6|OSn@bKe`vG{1%`jJ0sCLOMhg`#GD5B0`9AAO}ny71Zj!QwNjCk z3ONgAAO?hP!u)^(va=AP(E%3d(u$LbH*&yag#bz<40k} ziGXIH1?@4eQwu8XL|ntXQjbIj>7Aa3zFrqVU45bjD5^HMDgE<)voFD5S-)%#|H zHOqsoYrgm7Y1Hk`;vb;_V1_0~*XN-~vZFO1$g8mH9>{fiX1g346Da93VtP|35+_Ir z$_zsQi*Dz=6T6Ux6?i; zbv|=%4y3YS`R7bd?Ah!d(y-jH+5RZKCv@N4&SNs&@N_FZeX_X!=GTF9gSyWnsyDv? zbjZ@yvn|m@-w8j6NLEx4O7e|o7-REL(9P2-G)GLyWFdIvT^pNj_5H?`m>0$2s9e*f z-Y^b=8gdG!bE{l5$4VMNE3fd;a|&H1qoG5feHT~d9zn4>wtncCGczhrEBsKOFmbKJ zXjMJ6r_rog(`G{w68)|#(etDxo2o<>;A@%*a+!TrWI&JwZKapGV{0KsY7?)YWY9tI zNz$4Pws&aqY?l?RS<7CzM-;rpZHnawdF=BU56J81H8``48p>qE+rH-&XV48|PBsH80O(Q<@qTu0j*L$>fX!?-f(8q`;o_!M$LI_w!QG|L4`* z7B~bk&C8ZL^gF9eGk`J*-b9z;TXLUEFS>bafE$1UMg(MD#@sPIe+Y;u61=OZrf=Un zK82yrR6Z(W?odAo>Erct|1L@mBv`5svC6&n1IkpAnD4_W3f0pBZSR}%^zQtaZkKVH zhdo@GvUPq#bdsIPFf(h`{Hsfwr40N^c2vY8tfkxrc?!U5)__EO-o(j^)&BS;1X5fc z`7zL~v(sCYE#fMU1I(66-dI)npzW@&Zt6X^BaKdh939Mztf71DblX^Sl#U4(*s^Sg z6WPN_PRhZsR~>#{(wIz5GXp&fJZQcEbJ^=uIAsX?i@wDcF`27nBIIm5S{`T_T?{4U zJTnTRm2h|Lr!!#-=k4*&Gr_6NiAj{`$Cg~ zo)p*>u2f2}<%l&($_BuMj)es_N9(p7q+}B7bydfOQ;CwjjNu#s&kDcHAk5djbEz0& zJk%Dj9MANBAMzg_cSRdm0OTzlEwbK(f9Ku)rxf!idcvBCVJ02*OP^kBRPQ8OlF_(2 zgdAhCW2h;+u|A#{s4p?)22xJ}6s!pM#1)Jqi8lxQlRxz#oFam#Hssar z1K&7CX%s=s=Y7Lh;^h?Vca=!e5UmmM??&Z;~tS~M2z+BE3^6M5A2uR8RM-`%+_Hs9>@Z!~ER+C=Hs&d)4wB6_(zGiO#YhY{cRDowBb2nD=rtwzfoMhvvNUroKW;Z8fEr=L&tR(NIs@Jh7Hj(c_!z!)~Ei z4Z9$TBjJah0n5lx`8zY%k@k7>-XF(4Jigip&!37-Z*y!)xfvS_rD&^l0VvfMefk&V zu?dbYV(?$eP-A;xXaywGdHA-Z#lbc2*OJD}LVfc30H)k951>7~(E4hXtvK#CXNi_q zCrfkkMT{-#+c&nZ;SQxkKj{;c(9Kf16Q#_j$Wd5Ce@2-txu6k4_!gDE?DOSN;`BR- zh{$ZVFsi121z;b)Nt6sh5D1AT3CH~^iCW#=$75AzaEewNQ2uTxGhnsc_2ymMI_`r2 z0>2@DLj=o%Rf6+1fVpqkXv?EYV)Z*my#>JliHf9w#jV-#+yJwC`HQ3yU2oLSJ+Eot z_4yn=?>18j7cu_KC-BjVs&LLI5>R>n{E4b4_ zpX^tiVt$h$PlOjhC+Bou-N!) zC2{X?w$5SMBvs9or-#y*m5)+}Zm90+!$EHK?#Z$uQqRG-NWmC(M9&EWTTbG;fE6x2JB+RH-MDdt+f5*>*9F#)xloTJ@?& zVAMkUn}NgJikNLUI_3*$HC>7@8Se7b zYY%o{uFiMiMoNWXI-Yf1)dU0UL9&}sGB*DUfV!DbFmx3T!O=a`hrM2W{Ufc_M9ItU zV5IO)-lTin_!> zf&^vFz8Y_+7;1N916t;bZ2;aj*|Jm)+(m7yv35U&P7t;c8%^oTEM_L6Rq( zMHB`S2rPibSA^7}vJVsOMm8&Bx=sg4Wq^Y}MXv_BI+fecG9}O|k}{d@0X^RDnI79JF4> z6I`ZAJ&q8W<6Vu~SP3oZan@lDp%9Kk-vfwsXarXodF>t2t_zeO>9O&I7&WZ{r~Q-} zK5d3+zW>|{|GhD`lLA$m0X1I%>8SQp_fI3t%}^_ghg3O4%&4SmgIfyZ9H3vZNP@pw zGqck-61PMdnN~bSgB8umEghwP*r%&5Fn6xXEFuEmBo~WQEYea+w*)dc{9>mGMGKzx zH)X7hpAe|j?PDkHFzlc@PS&A=sE5Wk9orrAMO^T^~E#xQH zvm8w9M~L$p5Z(w+W$E<2`Z?X+OAu+b2s9%A*$jgRwB3yt-y`LlVv26F1PNjsi9AN0 zbxk@!r-C9!Z=5K*nPchvE|y1rbXXyn;c)@Z!j>8HIxKHEsj>E zBE%YkS?>3}Uc#>DDtjDRc@`DR`sHfGqm5LqFV44@+Sb_ba_wpGMOYkH-B1$8d{SW% z(j33Xk%lUm6{6Po8>4VAm(9l5_i4JBDwbU0|K|6ai(;#;{4WXgyp z4zs>@$F%2u#3)6d&;O0Mx!Rj(ew^BNbQswxE#AcG{9G!K0E4WiIZub;o+-iaZnYf+ z&=sCgr4x>`lTM%Lz(WfQ*88Jr=vpIb#pcTd$Yju*;ddo)w#kCK@AP9!<< zlw)1>H#6?*)teftqS@m_;P-~bg7z_0aA~dj%D!rkTvu|TZQ}<>iPGVIADZ)P8c1Jc zCvM|_w}iSfdEK_ek#=FkrsRI5cAnJ`owwpzmdF%2vE9Lj)_MaSy(&u4{EbV2WOt*4 zpH4{ssHLRGRq48#E;F)~iN~`a6u_P6Mt(Z3JpiYrCh0W#5bkgaMMqF-)lt<)1YZyr zs?jE?5#gEzG$l-I^1ibp^t4!Th}(Eo{wopkKgH(%I4Z#ANMOrh=ZO3Cv)C_bt`suj z)wmg!WzX01_*3#EDPOAc-s*LvRu0z3zb<*kG;^=5W4#A&&)6DkgoWz4kL$t^Hoow} zIR9OikR+<#5N886RzX9vCQK9yU4Y{A{W`D1M?qWMQ5RRk5@Q?`Kk~(7RHH^Amn3Hi z?5{QoJGb#4+EeLiVG%eyaOxj;ANMI&N-m}d>uk-1s~?Kj)xa=GStI(6@%Vng9D*5& zUU~h{umQU!5$L56kwJSYxJhTn4YAy}VKX~-6q+ItAz|oOT_yo@@48(XJRmUd7RDLy z2l@fu0erYuY*C-T0Q|eF0~g`JR^-(dC<8B@83nY@g|R)8upMK)p3A&V0b_VE~!4%-D7}AlimMEPMBoNj7zX`*F(nj z@!9xOUx8MSo|=fh)xnA8PG`RpTJugFrWk?S99Go}=ByV57AOkR_?yE}nM2w4k>i^# z1t;snD@npy%$J*&r^JunnPN3j-z%+SN}~hPk|WebG9Q}%`&f7DEAE&x-@|Q!lB9>k zZ>ncTcMiuUQ)WjN&kGaL8owpOjI7=WU(0)u ztwdaC74xezvH7cWQUVd9#EW{BPcO-L_*|)NMzf@V3NCo*qRf+^mEK~mAG$xTxblpQ z74!eD;Yo^UnXLSzVx(Y=v?KVZ`qi2uYUwB(Ewak($>PfDJA9qNFn7%EVC%l(R<8L2 z8jF|W)Jo)3vSUy4Z4|WcV9l92O0>Sj_TZ;Y95zPXJYVUVRf$U4t*RQ_TbtQ6o_?L^mcAp)QG(+TnA@7*V%f6Ih zKqaD2SH0H%V*0x)ScS$f#11Lv%0CNKB6r+50&QU9WLk*9~xqRV*!GA~+QQD*SLI zS8maXMq|vZOzc{Ia*PfOexILCHKi2nwYAyQ0$M0^l@nKV%a2fjMe3hvLw1t?WwQ#j z?A4#-xm%gUl9HTftxJIr%O|*@^ zA@2f}nW*u(B#t6m*Pkh$CIK$>%}^){OUx$y)x1K%7#iJ%h=)$rR&j(Fko=ei>sM@Z@ zjiXnmETku%n(kg9BFnqXvZsiDpU*jKlk~Uje(?f|D0_4gG2d1!XO+pz=iKIt7fN_T zKhhS(%#^P5(FAD!1@ZXDq&{XwoREY0vI3|izK5~;)}^vjlBF1zj%hU{Rp$z+VB+WZIYO(Z5>PO#c0;|yPtW0Mb(eWzK=KUiD2+A z+38VqWv*vkMq`H5P8L*vwZ;N3S-Jmz# z3juz&$OgG(hJ*xLh28BXVv^GR$4%w0LV(nIaCrErL@%sS8ju~glB-aYbm2#t!T)8a z_(2eKp)I#zUF>omvSPqi4k}@n$zTY!hRFV(8<(ur$3)_zN%lIfi|b9S=^1-<;9$oHyktF6<^lA z00hHlWDysJC}p+=m^-;-z5x735ghaGDT~Gy%DOZz=Syd`553=ES_Bf3Iy4 zfcOunXJxuux~CCmf|lsUW`iFX4?ky;soBt6ON_d!gt|J4^%oXx(MhhCu*zq3H8430 zT<@krimRHZ>hU)l4H0s%65b+O&a?!tZZzIKa0Q_1jEY|f*W502C1~fj*nknI&l{FH z{Q;MgDSbRF>iW0UEEk+Zh^)o0dyszboTcX5iDk`%L-%vn2T)F728AH%a?-t*L8pDM zZnG{TEWm)Hmnb=BJ@IEMT-pI59XJ{Y=N3L|P9Wra;o|;`451uLriA!Rhx&xFi}J%_ z56Mlv?s;D7G9t+-SPJ&O1*XOKovO#m!ld4R)QH9yhyDg2-ZfqQDze2q7pDynu4n0x~_WQ~dxs4Ik`cx%u_B<~M?+kbobsuLPWw`|2t# zE7w*WU-kgy)(?Td>UJ8pyRky{^6lv?RW{h(&D%Ol^7H^mt)&;8(FW3|O+ep#y-U{r zVqNWsK%Ok?VI$GBq85o#fb`F~UF}1{B<_EC+Zys{ifjvUOvs!zO}|SOa|)y?@t%H* z3yKVY+B5+0(1hI1VBDamtklvea`+(^K@mW!MGhZ04A%jF_YQ@9w%OIRf%)ESoo~6o z*Um7GqoHr+u1;R>EO~Q(I7(n)Xc}2*=DK2{xCI!p1qvVczWfWL{{akA3K1Z@rLwo^ z;vVOO$p0VotEM<8)xGLezAvKTDow~%E*ySr9ok6Slcdk``1Nb*KMXLTE1565%{uJQDfm-(2RR0DuEi2)9uT-RXBTwY6>IR zN!u$o{71#}Keo9!+u$5VnYBdi+j@8P84wlw7KYet8rrzvq&x^60|2`q>NLCRQAKd_ z2MYNV9o@F8a-ED+_YK7WJI0Np%BaVTzJM~TpZnK(jBUF|-pokhu+?pbBn@7{(rOLX z&VR(s|M)OAK?0)>|I%;jvqugJz^34-g7zX)@4_+^A7$(dl9%b}0^7J;0Na>Ke=wa( zzEx#G>mU9l@ltUYz`_lUStr9M!M~f_`Dvs?tZ|fL_ps^pT;t*N(^inbyOjRpUQB;Y z2q`f7)3}}KirA+QfZ|g)i@LtC^?-_qaXj5Eq&}dY32+_0$`qpy1|ji9RTded&Ue2h zWgGNEOBWxIMEe=>hYXgTfx`$Kq}I6sR2lle1V!NFz{CUkz_fym12uK`(@(`GBo~A7 zMjcf~{iU8rOF~RKrF5vqn;y&eX<2d%K+y~&XV3pk%Y?$a+~=G^eHH*#oN@(%B%K|S z&FP;kI(dJs>o3tk6|E<;063@ZeBYs>{p4pak%&7W3yK8{!mN;%5B+ZPo$q~>p6@Q}a3fD%fAGbj-9N07QMBFbMTMy4(PmQu zkkYp{tkAOv2Lh%zW`OK9j(|0N{dFaVhnUL@p^;bEqn7UAR~97&{is4rArjZ zsA&9o4-j2BQ|yE9VOaPSF3;KPcG~a?gKh}ayaHw>(eP)Rn^}JuI2i0WQwyIuHke4!0Q#)E{vhB? zg3!1p7u!#N?Iho{pO36_m0Lt3g-9^SQkc@A@As|QM2#^LxhjJParnnn*$sn_S$Rwp z^_rgiru9gqeHFz2q@ag|k|M68hgZ%7Uzr2~Qep4ze<-~KT0vb$JEgxB*d|C5n%F?e zKN;IViOnDUI?YYnGHZ7x;=H&o{Ad}B$%Ga#Myg3S)8UWP)_TmRs=xV*28S69T^^Q@ zDxK<(WRF1P1Y>=uaTC@_!+2F-(PeLM{}j4d*Pt8IfU_@%6*}+`*Drzy|J@OlBbX`V zhz*7YB1XU-VZIOEny(ieU440pZ`ks>v1!oGwB}z^al-e1d?w@-{t99b;fl<8UQt>)>9NOBA5;u-jfBYApFq>Z z@87>KBf|*J5D<(+pg;+8Rdy3KpB7`z^iH< zrBlb)!CO+>pZ)2a{0h^4e7Njd^#;Q_TtPcVJKjU$uWh|8T)WbXiMBR2Mt@w_jApr3 zA|UJLMP%J<&&&tuI0e*;Q}G-e9Ex>Au|9qqR$YCjr>DmfKc-!F6=%~fEUM8{+y`+H z7UWHyUBEfE|G90cvB7rxZsbVxA6HZ;`DdW%Wa|9vo6g5ns)+SYf({e<&zvpO{`L+s z2ze;xiir-!Oz60oz3xkodF{PB!Rjw#K|eNGEcpDZJRUIu4U=h!DWJl(?VKA z#_xDLK+h|7+uRlo!DKoN#w^Y*e?B*T?mjoi$ap!^Y_4_$pA|mB<@s^oo*&GWy|6V= zRJW7he@E-Z5pF61x~3?M3(@|V=m>UASCDUPZIz;kbaHaKLqtTGEygF!jQ+5vGUU`< zzxSopPu$E^cQ9aW|3@7XBfDyQzfK~D4t3tB+J&0#T;y&LzMj0+di|`IF=)3p+PYxM8*WXoZtk6bX4}9XmmNqfa#of2E;!FJZ1}u&Kdtk^2Qx zBN1@p)DE>@W>3rmUb&q1e&Ex-wJ{oc5!f@-u54;N5W5J4q`JM(4N2VLJt_5YxCZIZ zj{PMFewEBt zWd4HtSSOdRkm2@n2AFLs{+2J;?2%o_cE`vg1HI(EWz=!h+4WiSw1Kn3GfiM1aw;&4 z%uIvEW>=4+P7BWYZm!#(2Hee|0z2!chU~+dgN|+|#W8yh(R%A}>v=ij!Ij+*`#oiV zxt1n9GgIf3Kmg8uK=_*giVoMs&|dE-E=JX}4h=06IeZR*)6=KjPlUWa>-dF}mX5F1 zp(UXH=-kQa;JT5@Y_&wS@fN+-O;(klgoRnrr%a?LeHcy+zFU*2=F)cMLK1oE8^?U+0yy0rUaAoxIkue zQnu~N%QTxxV2A*u6=W)){_AdX(9i`KcrGS=%$A@RD~mKi){;|lw_gCG^DTR~?Ryy| zokETHbvilWcm#5|JUWzQpG-c=+G~9TX#{q^6fZ*xk|0+nR!%yz%FO4nwn8D-6$F4&=Ajs9syM{9=#L@UOCr2au;tC2z_VM1=% zyk5GlF2BY3*|cw2R(YLyIk)bR0n<^u+a7v(JklU7+4uO?zQmjY=m$$EpGk-Z_OLwg z`63CS)5R92%m&>CJ9cSwV1|C&(eU7StFo07K|Vo!@}zM-Wm-x~ij|c$TtP0JRAZsv zrJ3&K!8;11FwgZuuaBvedgFcnRvLp=vQK0$PiFvkXq1O#khRrZ#<=sm>|^5%_~Q+K z?heeen=phP!AZIKMgEQ$@Ld>FB4%RS+U)lIm2}|MC}8(7n56zWyt(6QTD=2meGuz)ZJR`O9H}!BB!Y;h!|| z*6v4OyfqUMRL3Hv*O5+xtoeC(G}OXGi152f_nW9}nA#G26%%HL22#jdNx-F4xWGd{ zhC!OGDr}Tb=7U)rt%4vFWS=S@S%m*Z5Ra7QV3%lo%FI^wRmM^Wl7CEtv4gX!(|I5b$Z@r z-i}`G?&mi_zjSnTyi+uf5)aygs;_=_y{hB1JeO_|oucBA%a48aE%WW2kJyTtz1r+U z3`>M~|MePg{StUK3D%sCzkvqFP19l5XD9D7b-GMC+0Cm#3+>ryuM!CgH$Ox~pq>_q zks7xj*BNJoa!ISVE7NA5&1gj3QVUhl0sL=UJ(%vJr4`Z6+Qb5nb;9nRLE1(22Q47OKD{yMDzB*(sH zUhPMPS!_%@UZ8Y_~q;~k(SC+*7u$K$Np8e4-DSN*$~D65Z!I%U{4g;GPB z1QWX@$8O<4eiAd1sNoGp0o1TxXEB5rT4jcpJoxng(FQwuDVwI%J7!?&obynWqo{gx4NU941KKEbF1&l}TA&k^mUS2-p_2;Pe zV}>;IoH4I(c`sthL^=~Eo*b6OXoLy_-KSbk%p0QYX&GYd;7J`c P;7>*IkwUqgdEoy8fPVOf diff --git a/docs/img/dags.png b/docs/img/dags.png index a551f02b1d2c2c4dc0220a904236a995da5c4dbd..04c82131f945cdea4171d90355c29e18a3b6bb10 100644 GIT binary patch literal 217069 zcmb5VbC_*ElPKIgZQI?uZQHhOTc>T?wr$%wtesj4KkA{6Ar;h?dhfq;PEBqc(zvhj*d ztIC}`kl}$OXbzlfgOgxh34z*i#7U+h{AXbvOBe1%<0BZP@+m`sh1ilDN&|UKvTJuF zI%I`(5D!>@lFMycIZ%MmG<7CH;f((cAGll`GxUwvbf zh4`pt;$j1;fnO{AsoTW;9LByt#gW0Re_#UTtI%n!Z`Ge3yQTj%Mk)+mP(B^4RgN-qcJCpn=vB~ zsUAIC`g8Se)tS)zmIMV5wtsLJD2(c1D9U5Fr|_ZCLBY>pH`c3`jKXaa*K*@HXyS=7 zc9|IXvfJSE`m=%ZZa9E~pG^=6r#534AEPQ9xD7ag5E2ZOEPgN*zBGV_(Bj(mBBT$M zx4<9n7a_`x|7N_IyGUcVIAq>!KH>*>1Oy53KFDDYj{?x6zF2c)8Qxg@Gy0ggNf3Fw zlOWhwKwrt7;$XsBV!v@Y(gzDS#7HJ+R`oSGp0bOEI4sc*l%a?M5o;ZU=i#1PiI<14 zDX8au-mv{QGkJHR6A~U09b#uBJ6SZ;2*2=bs6qjZXhvs>u7~TUA=-R(bsog$fUg4k z?0|D%?hWv1RR8yZ4&8#Mca0KaEWfJ5c=|?AEK=*F&v7+=v3#SVJ+3`bwTf%&f^ce| zVl6>pUj8Gb2i?#18@$^h(pF4v|D1tXPyK9h;X}`=0J3|KhDjMFG8?rP)0)FVt>l|Y43KqVFRvoICOtxg&%7EtohJ3 zAld`)^*9$0G5O%Q5M=}4v%s|h+&^oagD~Lgjxw*#;(T<4>%m*PetsA5aRIIWj8I09OmT9LU_|_W?g2^lXUZKq}jp*?_NI`N@NHyyNkr<4vg-a@~(}lk|f2 z0lXFFM>-4n2BsZ|5g?i_Fh(Lno{UHZxegfyS+>)*AnPtpf{Z01UC90Mq!h0COY>LB zFSuVlztoJ#Z;=}EL;$CWHj;eBn8;bl@>E1M#Fqd(VqG$qBt8mCYAi}l$`Gm&DyWpm zWU6FWDm>NB>IW4||7tNda<2sAF`YY^J4Z*_jtuXJ4O#sBa%CDyzGN^dbW`Ax#Kthk zNXN80F-^@iku_zH?nVKRUdNIy6iw<`=zQCb818YGi3*nP73VZnb{0w6y53Ahn#Dtz70RTpI4jM2AHu4r|D6 z$aBcKm0TC#7RV~=5%G$9GQE7}DsujCapx-G@^Yy?^`1993q3JAuU)iV>?sR7Upt9e z$*OHryiV4;rkGOPD&i=T62ByH(I!}Gn~Ph>E9q1HF7)jSC=g5!7&FKjfC|zR4lU?2 zOx?8_DhlNlb0*d&4w$nlG8UW3(3%7?gKtXoNL(|ow#T)jVcjy?u;Vc>vP`s4GHF~n z$fX=!9AX)AZn4a=%*#m7OkT5T@@!f^GI2t1>V4I^XFj^7rJ+rvMb~7};%QoJFwsQU zTGrlbDb#$`Jk(s$&}|{p^3pKZwy(dc7qr!}ePLT-OS4hbX4ek5{INbg7d)3+6jEd@ zwq0NJ2YeN6b!nZq-h5U355^k%pFUh;HZqsV-;CxX-Y!2~#vICB+V{(MIi?%N8rE)G zl44LsEB>5TGEZ?XdgMAK-BV*|W8_YVrDfGRuTZRr*4ozUZ)|M-vh&*T+?d)}*jVY> zzL|Xpf7Cd0SWeo(I^mpY8on<|%Tn)M@vZCYx$JK0ar5NIvF1!=Z*{#OCO{!*ok7o| z(=WBpyf)jW<2UAS)sNX0_EYw&{bu_v16~H!1nvUS`@sbk35o#Ti*N^>iL!_t4IT!b z_M18$2H_IM2`&q58CwoA4pJY>H^{Vqul`KKXMbaK3d@Zt5F3d= z5$%>?o=gdM4L3wACQ1v@)k2mAT1i$*B_-YY_k*2-yMtONW3tJNpG2bcqVe;|Qc|)~ zo2h^J8BO*wBQ}$C$WvrG`PLloqQ{R*Ph`h498D_ERGt<=ErXj6nsY4k))+6aUH!PI zeSAC#$itgb?*q?gbf+$NUvGYU$TNF0{btTJeKV>3eX;v~Oa0OlNGO=HqpIeS0n>=_p?RR7qW9oxcsF=t6twp`9On!5UeYU4BBJAH zobX&I8+AJzID9FcomQ=V*OIL5**jTKU3hV?G(CUc$MvQ4dcM;bZF)JenQE^is6(XV z;8U{w8}IP!Fyt^!O|-^JV@gN2^X)RT!g^Y@UByOAT9aCLT@_1nt$EsJGtwom>P2;> z613{Mc16F>b4+(3D)R6Ywl%ia!ZeO|#>QMn=}++==01-P zwKiFM?X_0=^^o;0*I$<$n#X$TZOVSfcb?69TH8(5^R=;?16^59tCwqyX$@^Q1gqqm z&mYgt+jZA*tJ|*q`~A5=M-VFng#=W#0@!A353F}MU71RmTnKU88_O4)-X{p}{H<-6 zZ7KwFT#X+ThZeVU1I8hvZ`z_YJ)_KV{8m|iT#tVRqo~Wr@~`<)?Jce?6qLzlLvc7= zRL`uXI_Vw5=RI?5yANCn4T-Kqg?%}H-Yaxzdkv93OUvTr;!WikbI0jDb$NeC-J~q2 zYpT=hMD;Fv@ZP>n;Uw8PZys!UyHX!7SFMA$L0nDtNw;fz%3taY`+UB$em8!tY|*vV z5+ei>=y+p(-%Jxv7h%Ma#__vh@E7n_xLZ9mUu7&brRL7&#vY^Ox%DCTV4l#vN`KZp z(%0zWf2@CBNBN+2ARcaLghUXy94zF5NDGV6@>8kWnIEVwYJG8Td z0}3QD1>}u{01t12iTj}h6VwAPV4b@sM}e}2=iJm2&DE(NNN~8FMdct@I5#=_cjEx{ zQ$o`T2ne0@Uk|XP63Gn^5OA)giiWd>tPGd2y)B)giM^32oxAPNzpa6Qc-*=EW^GNK z4GGK@$=Z@;I89 zaVd$2{TKM(H(r2+v-3|bdU`iEH##?FI(tWRdPYu8PI?9=dL|~?zb$B;JnWne-D&Nd zi2t*b|F0hrQzv6b%b(7c_I8B->etZ7-o=?00QeW8|Ni~wI8EIx|1Tyxr~e}BFM;&` zs-b73W1#=&@|2OaXpmFMDB>##7*HT1YnBt-;O+<`B2Aq3Dym-$!(Gs7c$p0|eSTpP%iEuu@U z7De`%my*9p>3cFTizHd5u)RcwYgcDiy{>!m0aC}`(;M7rhmHYJAkeu*8@5eNUE3Fv z)7kdZobcw;*|LO4f&z$$U|_)iG+_b?&}=eOdc8Xw7YrYR6;N*c{{syM9O54mUOcBC zE?V4;8V4WnNF9Oua`&Pr?L)o1e#G)t?~rl8vmbz4pKEm4%$3gu_x`{}uiZQaEHFk4 z2@>c}NC^4ABm%C2J6YY+WKj}CfF|jL!`b(Zevx7`(k=duEH|uFe|m|2FpjB2Qq1m{7!I#F&bc4-$l^YKl%1 z&ZKyc5N_S_@LdEY0XCM5P+5b`ItM+_QRg2DF923p;IFuXl$bmzz}N!ARnQZD?9EBg zV6VIiEw3ZE+}e0Hwxkq0XxC&xN%m~1gpb*1%{#9D6J`jsiWrFeGNn>11jEdq)Aw&d zL;}M{0>N2H9}^|Rq)m;t8vb!w{>)$K<%3IST~U*YwxDatq?8oroxbu~XIVA6JAH`% z&nDmwl;B?~$cys{-bXi@mt{oma+t6zzm(ah&)=Y;&pq?>w2)8nI4#Db3*i*IrbZL1 zjfXO$ZRY32el$w|Q%+3^?m+Zys1yi+H-t~(x|6DRA+2^Vi|~;V5wV_@r)6h3Y;w8b zPBa*i;nd{OL%ohcWcV*i>m9@0H#$i35d!|HkW&f&3OTi(2161hVCo#XhEVlGoO#zm zN1i2$MMJ`I+6?3_VuuKAVhq8S3|F>_tZ}Yexxa^GIGryUxQqT0QpC#e?ir=?f{15@ioEf>u)7@ZBq9Ug;oVLW zqQr<4@1ak6P}XRABsmqR3wUn~!q!Zj)TXSg9jG6MQh zWJQ?D4mGP`L-P1DuvVpVPX!Ime^7%YN{`A2SOqH5yY!B?J_g{ngQca5pq^T+7Pb9V zG{mtQ@q`Bx!Ei}> zfH*K<7^HD9lHWw*1ltK6Htv1s`<+|Q#WYDs@gF;x{w(ovk-;n|i9#ZLl=fjf|A^~1 z8ier`A|~UX>*aU7H3vtjSv-Od9o%FJbT<-mJCdj~V36tG#<3CmEL;IB9k|yL?_(Lb z8$6VQ%m|_fw*}myQuDgoff4?U9e0Ezs7@LjOC46ak#H7OcE4OOP!gn3?o(UDgZu#i zK_RZfc$Kxxv@_A&z#z?f#n0eQx8HX$akP+2T;o5_B6T!99;Y9C>laX@i%#| zLZUKa#CrRkULw0YF#Gk_e6gX!oc+*DvEG%b#eHYb`PBH}7xmFBhi58$;lAt84S6NOiHtL=$Orc3g2v7j?;hEJi z|IBkCBaVL+A;A~uQmk>X!GK$1NQ)%-l~4v#O`KwC0@A^N72)DK{ChAP3Vs2t-Fy>w zSOlQRLIb!C$nM5G7eMRYs}O*qf2;JQ=hDdMxsz7_D;>6y%NgJ&L!R&WNf~*)k+f%+ zP_d!nh%3|^9)=P=)`Lono`C)fe@%)BfNWZ_cDhId$Rj~k!hrAV5}RAMiZK3bqdDn{ zweGmSg2Pz~$h;L~s*V*t1wvZtq)}1mbzV+aYJ-~0b z;03;$1K%)2?J@^%Lx##4cLm=+ZLLCyshbZ_OR=it5ch#o#y3N5G+>OpUJTmXbj+e5 zS|gxA(RQeB3br(((-0A>Ly3|gzKS!^4VQQ~R1Yp9>ey0~Cc|@xUI__j@h7SJhvQv> zG{Hg|M<|UL+)I;Nm}O-y*E+!a4aYPOvm9?{#$FR^pxDm}Xh1m`agPLlZGYvHkhpLJ z)5HSxJ>;SxwSM7HfVu{Qc9P21B*mU>mcpYPSxdp)b!FX3u%+I8T3U(Q>wGK)Ly#}T zM2xdRvfLw|Vgv|^*rj16z&;aWz#v(+2Afo=+Co`2M=!F!k!T5>nXgF83RKz>16QO= zBt6z8#G^(@#AMTe|07485C(Bf36PjK3M^t6SVVl-=aHeO%^730-6PkLlRdUTl&d3v z&*g6r3E_bp_Y$8RHMTjGASX^b8*l=Krk!}-jv}ps{3CYB2k(SztdgV5T`R4PB&KdH z%dYd?NFIdq?L|o_&KLFibl6unWBAvw;4B3IS=x+yQ&8jz2&u!a!h`>q?oI*`5y13W z4}+B9J9z;64_7V|SYvkh0r`SSvHA-wcDE5UD|D9x9U<{%KYUDQ0YP3^y5ix%6gU?c~bG4SSoa4IcJN>1f#6T;Nf{IFKUw^=N0rjs`R*h@bB&hJiV}e9=O$a$-hpy>JJ4dEag{YN7URPy8yqg<7Xkgx3EgoJA;iz+ z-6dTt<=|f5$iZ{vLk1)U%-Lye03e%VT z63(;fPv#$ga>1`u^2w+TRwhJ@9eM0lqN&9uL`IuTjDdAxUkddhd?yF$`$V0PLBrjW z%HG~pkM(OH`fk;SZ+L_ks?)JJ#8W2IZ`;2siJr#g>Z+*8)z^M_Zjxbxw$-*CEU3v^ zD12(6LKL3V99mnRZ3(v~l5U4R%%o8Z5;$9lliml3gy_KaNzY8{$t-xf1XO7i60g`R z_2)4!HQ+ubDThmtL9hyr(N_Zcm@A4OTvVRCb*nXM)&6&% zoKPCcV4tyu*o`5~dtx4EEI&$=m|?djD_kB^l#gM4A_wLm4Xy*WlbT2$7)KocTQ@E` z+{A(f>l777mUm^Q$kXd7-m8x2fZ3a8cr%U+;#lWk4;Y~)%uRVQE{Nh``|{vw7ieM} zj4HU%A_qYSCOp@o+lFz+`^q@SSaL9mdg_Efn=I}nbZ@U8d3zM5B3M$gnn7?M)>>>7 z%<;pKgV83{`58mtlpb)8IB+M5%sPYmwlWivk3)#mgQV3!QV? zVfUgMHu`N5Tphg1asijFvkopVt0_D!{%lmXfu_gDA2_P-ArT*rV;bh5aFZ&fow6|j zyZB~sYVDjO$6*5R?@}>TN3K?zL$@J32Qk6qGKy@rJ3duP4S%hrnjjmzlab4H7N054 zY#11|UOllWI}_F3c5&YjkGwx|re-J!4Uo&8pIU%gZ(ZN%bc+k&p8aFjAcR8*v`^d* zq;U!;;h^bfbKNU%T3`O|Zsy)Fbw-%g${Tp-RY60uxf(yh_zUeoDLVzxTtIA zfWU~$SYDu?pGR-QE>jqIVPR;~*GDcJ&|4MJW7me)$BKC~F0tz@#@bWc>dv{8?7hHO zCu6tlf+Y7y5>>f3K$m}17uqs-U3sW>-x+%8 z{QlytFH=g0Q{rYG=Szo)gB`N4IoiUJR zbm5109?!JtXQ1M|a_Go?^~!69ozU~ZYKe=zH{JQb)nF4Kt%h}Z*#oC-p);2E1^a&C z+#CNqQWN8MCysTtb0-52aT>z8sOGa>jq>biwDdX8E8bw2?kGC~+W5Y#BE>mqTJ7wx zJ1T0NTT{{%f~f{;8gb7On|~!5rG$rOS#F@rc1s*zu%WJNvbdB8ReyV-qwL6culck> z{|*5KG?qRBia>ux|d4&~ccU zn+!L-qHg>MdO=w1kE8wX!kRS2j%pK~v{sgCL*16=$6;#Ajb%JFN*jCM^?^mxoq2U-|P0qmJ zc7zfO9zMqo#;Tp2Q|5<-3lA1kWs69WnK0hYxvkuMf%jUwePn5%5d5zTX3O4zwW4~-xqypUV~~|Z;^IP ze`t%qSLr3_RVk+wDynN{E6Uq#b;*0oYDWBm@Eq-cVeuS?hQ8m6Go&J^<>G#o=Ue>z zn0r1AN8!6WeN88uk-Bsc#c8Nj7r%3LQnkyC8GqR#Ha4;x{}O)i({@F2uvvpmg!k=z zkgOU>p4ssIXgEe>9ViZB>g8kco-)${p19uDW-ml1jhaLFiN?3}%k4jU= zeDdU|!WM0uv?oddZE8ttQ-B{TcgxTHH2W(FgXtiru{YGl93R9Od%3t@>TouhboSkv zeWAN4?Xp+HA#S7jrNZ?Wir1eR@hsGkfOTQWkd zPTet&xq}ejDwM4&^_uHD+0O4>hhC0)mWWVyp_oSTdC4FPJE!3v%qB)u$o%-S_wsWM zUk8XErmlt}g0V*q&`^v{KG~)crdRH&iALq;{Ms7n;75R;!g*IfZ{J)#tyG{p*_U_M zkxQXNLq4RDLK|!noe5t<)u?23E7c3UbMc|Cnb`+VW@ho4FUX?xn`@bQP16rR#@VU5 zy5<+{E>C7-vKIY@OAnx;y#x}!<{F8XXceQ)aJl64uXqf2GS56KR~jI;Q5}yNj~`-z zlVL{iiI?pZx#Z7_^*CB9)d?R)Yaf2hcWc*)G35k4a(3syh?%DHTyGV~}bc6;X4o;3yO&CW_SJEg~pGQOQw<|Hoqv4AgACmijTxO*m&V~Njb@D(a;MRKE#W> z1!s&HVJ{hZ93p!>Ns6^iMB8b$?JEBs+%b7S_wO(RFu})cwGkCKqsAo5?piZt#ep|H z>~po28p@tk74zF{h!LmND4zFJsTQC5be719IO@spIfQ;xQ%-Rybnuz+W66j-+0BQS z_C5ZH-bnNdj*;(m9i_8+{2GlQK=45{ZTc>=udq5*vYNTDaQQ%$AVj|kE) zPU~`=HxEpY0^h}wx3fVqO65jETD173?R+EUxTjSn`JA{-5M`i?#!J5xd8)hPk{dHP z0ZRrDU#^s?TdJ|^0M(|xZNHkqX`Mw7pDvyWgvyv22&-~6 zSRUAa*h-Y#H5;eJj57I+^ehSC%Fl#U}>S zL18G^@4!C@2tb*DhmG`v2rGV>YVCFT4lIASkWUY{VYH^)+<_kymJqg+=4vh0Xug;I z*1Vg2<^5H|bEljUzA<0M{Mu{}3f||OeSzB>H8I}kzkms!k72NH%vt_QP zBk2KsZ`}&mGf}9aY6ji@vo*oC>XulH{BK7ljuaq|J&p?pR{@d9O}--VD@$f0fJg%b zWnEabZ!ye!ZCc zoYsJ_ruLZe65DT05q6E!+)!tr2&PtLoUFIj*@ffs?Z_A@^3|T7d~c1y(pjhQgP3I# zg{rf0%qa1lt4=+kqA>F+(d!wgvXZOyg{PCKBM<%WZt>p0hZ7{+Ige8N?6yVM2zhFr zP+b!YnsP^YHZ9|(q2u?;0@&FoLan1_zj;n{2#&-T5`T0)ZmPY>@OX;Z07|v>o^F!uWr@k=F%Fi z*^^=)Gex9;TM)+GldB7ez(VYI;cA^+^DrrFi_0IYidg#vs z@XEZI0Ip+c1^bcK+(;ph{?yM``b%bpT(3i==iQ03u(Jxz3tZ_n8Bt9!lGnbTY(puGrHzmxP9^ z=5KWU<*#55egbntjD%XdQkPG!%f|7^T-l+uZZ2ZdpolTKN2`TFge_6+eF0;1no>~n z8EU0@xKtN7s^p^Dug9@sfJ_CEI+n)w%`5rQ0qqEL2*WY6{@F92RG$SB0}D1ey7^M1M*jykDI=laFF zm(EWGmH=mZQ0v8*qs>~|Mc&x;q9};nzRIgN&im};u;{olxy-vmcj+@$qaz`3T36_+ z{Ra&DS@*O{*0nPG9ZywcUQola7Xu%tD`%1J+I$<%A^0hW?~3mmJ^{4(#&PhnECv6Q z4}ND>tA0mrbhvk!FB~+5UOGB1zi{7`sMoOjZPVTR*i5XOY(%FSVAuAK<<}wnk8J`B z7`FosTKggy$4Y4)S&>}mE?N6f8+A-(?ArbsD6RtD-)R8zou6nZFqu=(Oq!s)CRpSi z;*+!5V}7hfKiaM5l0e+K$GL+YchkbpfxF<^4jbbq@^k}ly84f`daU1&IUP8g9kH;N z+H()*+#AKYVbP~WGtUDK!#zZM-rc{a+~10vxH^PkB7c4PuMrNd0um>3*5T!b9wR0~ zSFqH#W;mRDjz85NtJ~8cVk~#%6&-F_ESAI~4PbO0@ppOTs&prBxZ0$Q8!=W1-As8U8v41J-Qxz;CjV_l=)b<8C1^s;UnAHSR&?W|q= zqR6q@h4!AbsPTPf7%s%~c za$*4?_&WUsmipYdac`Ziqpu)7(SN^?=|>3wdYh&0YAa%T^+wC!08d(QGe$m+TB1^L z(SU8N=J3r7?SHHATibhZ5Fl>3rf|mRAAE~((VS7Q$$!uxXWX*l_A5ElcB4}AKRbF< zN_AYCG=V$6%AA6gGD6}k(N(sBty8LQgW+=uK*!~l#goD)tHm5;tYH40W%P7nra@Ci ziOyi}f7eiKA2AVfM&|Lt^B0`9K6y@?$r3;Od^(dwb$$w*k2?&Yqe0&1T6W24cL~!A zXlpqGmzjARI`TU$6DC*g%ET^Ocj}>Cv7Zx%v!-;(VI1a@*R|*mBg08ch%GVB8 z72_RI5&4wDD3aG=8&nt$klTVMC-b8k=)CKoI!un0JSAmVd(wH~?V3=~6+F*(hc&@} z?^UT}NO^iHMxPaJic(B%4j6H@`;X84;pqcS7r>lH#{+ctETZZTh zb=0>b&T+}anZ2|C+~Gg44_4pR7Xdu{2o0%PGJ%~-At8Tgu@fw#0q%x1#{t{J-tc`QzPJ$ zYwIy&wUy>iRQnOP^HM|%&}0NArKJ!)Q)%N?rPGDFQrREFLQ|}`Xy+UJov<%4C`#8s z+<`!)sYec>#n%p>!rzzHiSCl`{*q$UB8fXMEGqh}lRdU8hx@7BMh2JH8Dg|H!=f#W z4_txh7<>Y2)Y4F*WkbSAsF!VATZMWYAvN(l6yu-5LZjl(2J558s^B)Kkph0(ZvP_9 zfW?+!f{=$HvhNxg!ZyZ{-i%B)^Fa2UzIX? z1b;sNz!~7N|O6>Nlq1HoGosxp%&T?SJh(kdcX3XO`>rpqm|7nQ3o2Abi z#^3HgW00tajcFSk6-v48I^TIC5qTJj7VM$xhGC^tii$S;`oY&|9o$y{RM!1!vuxZ; zdybIvI2u15g0{&`q3rkN@7Hb$(emt|E_I*plvW`OOkky4!Bf-g7?jE=36*!rJCRnX zH5^&#vSwP`MZ;uM5;CW1Ax%e~iK~)G zrIbPNaX1p7wBs}4QA>Ki$>>|vUB^OGJ{cXsvPHO@N$C@?@8(1O$f@HrjVsQ2o+aV} zg!*P{oXV55JlB9``m~j-?i;*JSflU#9Ec=7R<~xPsGSt9r3#)OCL3(Fd>)~(q9gN> z0X@hO67lpocb>BuUW(3`wxirJXNU-)9G`jIPxBpMwu4&o>wh79A(haWH4RH~Hr$O&1(~8GIIzYu;n@u%j#Zyn;k-g3j3FV5B|g zHa(^!0P{jP{HfZS8WU);6qe93Nc;`oW?yPqeuwE~kZTthefvkft|7QjSs6s_#%sdM zIN`E5V>=7OKp5LU{%0xWRAki!DoVvW3QNfI9QG6WxKM&1C> z?q4+^D=2L<6^z|iI%DdUWhg448v4(ZMqExbUhH&lnF=BokOZrPlBS2qQZ56 z>~)rQW2}iK1_P+sH4|q}At$7xd(u%)>j^9PmJ-Lhm6k@Mj`s94pP#uU#%7Ql!-vt5 z#D(8oRQI1<%BJVCjmu8TTBjBp;&ww*B-PRJm21HhG{HQdNIHdGSYG3LR=g~yQV+;M zi`uGEO9p1wnnsIV(FjE_!nC9zytYGsKr^pDBXPo;|=>;By+{_rdl7ZYiW4SY@Xho-F% zl=X}3!;|h{K&Ore5HJ{KHtcW5F)vfp5SU_hk{^h=zo@%}-xlKad4 zE->)qSNVWSzf2)KXav`QOyQfLrKkG=p)~HK@`}Ae#O!3?)KUsWT8$0c4V@#}mr;NSp zHT0$v4=dIqzAdj^X3j#fk%!jv(s%OGfwzXL>-UGYOYfunb!m*n;H)4+q(lC*0LxdS zGx#Ul3Aks6w0MaEYiS&i$I`i<{yY_WB!nf<>r)Fv&}Alc zGxZ<+b+!Vw5^0hSB(+}%T3ABcEY|tn1=j4~nF0%V5SzPmk)Qlp?4{@@7q}rM%R z>8V;Tnb%--MIIP3c7EFZF^{WdAmb<~@Xa?aoWA7zINRS@IEJ)2prZTbT+|zyBTm|2 z4&!{53Q4&&>PdD*MPuj*OLGUHaYwQ0@K;_g zs>XS54O)(o0d8rv`9bS+8KeDmQ<vpYI<7O0TAqu#XTGF1Js3s-fuT z)X6S{DBrq&0;-2afP%6IC)*QG!ndM+yn;lH^`c2FP;|q%35Yo92**C=hdAg$+|{#^ z;oZP0(CW;D(rSuQ3!K~=BC@r~`swjR)nT+3%KIp3nhxrBQA7U5J+_S)Ygp66XCVRKuWR+}F+s`3PPQv> zL0guV|AGkz9^PV2{ODVMuJ^#`C**hU(wwoJbOj%%auuk-3i$Vnkfcx`6tepQa)TVW zF4-Qip48MhEA@ha6aFUOqKUnw<^pr^lV7|R;|&WeAPvMV$q#qMJNTC=&t*tAt0>M- z$UMntCA}%XDWv)BW!|bGgn_liz?`^v3&;NeH0Vj6{>(^{6MJt6`P;_!%*Yy*N2)pu6cU;^KnvFq>RI22D zMTm04vu6O?2qvvJkLBJ1-lE1gb$l<1V0I~q6IYwiSn)?Qi18}X55NRkl7R3#e+?}B z>fSXE+1t8u=!IDeQCJM{(eFPGV^AkYG6m_r!7!^l=|Ay6dz?t!os5;1(V@z&&@B0? zH3M?(6(C5?bh%)v=E;VmG=z8^GQ&Eihju1bmW`*whyX4BQq$eKRmgC)+QO&1dLm=F zh~4U~2khNmss;m{ke!=#aA18k&2gUFi_0rDhX>~nb0mRPsRr4Xc0 zMM)X$P6ZMg1{dfW9aRe8nRd3ix;}V*H>lL?M+!B)tvC5W6yZFrc#8^bv=c=CRSf6S zl`xbEn*8&N4qK5TK$9}hpR`LM>X%IzH&XKXG-V`RG85D(IEKflc#o9tmfl$rgUuXt zrhj{U4wORuSaS^IwcoepJ{^YZ%ih`Dzg6uuO z!Rg6Q7@*AqtFh9ERgG>b0A#OMba_z-jKTGX^1cgmj?zP`96r_1CT@w|j5f`%dmMwb zX6p0A1wD%b88Em&Mu*1JW!bMIL4TOzFZT=j(wmxwK4<75!$S8^>s>{f0|fX;AjZB=P1T(_V2>{!po+He6*~9yd@)j0+}CJITh}SqLY%`|h1^TmNz>dgz=q!qx0V6<|8Ii`Kg2LHA{xCCOp+Opf zZ+f{c6LfB6r`s{pVk6z%6P#c-@Fc$JV1#=FxSF${qtC10MCMu8T&oY|uReL63tIc=w1vGeU=FeZCgD@%F25dX=Ft5H}#3=ZA(YT#C|A?X-4 zOq};TKi|@fza7$1PgbQGty@CUYu&klHh@rgpPg|sX?Zw4!(6IJd!|(qXu5p7a40o5 zO2~s*w31p=a+oIb*v>(Q1^#?$`vX|Wa8qm%p?%o_7 zGM!qt+uoH?gN$|Ukjv0nej()J=ub(@Zg%`OWFzo<8s893U+w&kk&gX!kS%|%- z>kiX%C$)gFg@2EDcLm%i4*fZZaM(YQz483z$&>1Zgvyj3=D0Afrmu@$tMX~3|FHJG z>Xy#8-nx2nql(aF8atO$1%7 z1;q$0T}fi>hF9Fs1%GnDmX^sfkJ8&xLPoxvYI_l5^>y3yz(a<<9rjf{yy)Q!iXYPK zy;4uctJ+cs0Mu-bp0RjZUO#FLQD@<&-E>Uvpe_d*n68vT)P+&Xxa78`;yPB+JNre+ z17XA`NaZJaH>e*~TY;V-_u9buV6|sL> zmq>`bd3DYmNM~*em#BH>2IZBbhW|_i z4W`Ax$>dcj6%kn$>+oAC&6uY=6MqsN5(O9)pPae{%qLX~hiC1i6#{3J(s^qEkq|$YqVI1I@{O z$4JTg068=bBDk0srXfiUi%KrD_w!Tm)v5gcE`;{3|h zGR>dqd=RSBWw!M0T-fuoO&d2V>Sf^$380Ax;x|V45uI0{g^{ws+-}J-x%m| zjO^_55#cPMbtuJ(w_SQ4nHy>n%y{hyHo5+U=4t4n@lxHMOmVU_?Qd{TbsXDeK32Vz z2^R3-_0P%peb4cxR9-d(ALGay!X)+8%Z-S;Lg3@Q9XR5+?zwqOq z7~Anc$xH5&?$-WpsWm#FWdlBJH1RyMWVSv}l#lSYUM`R*s?Fz#w=ekK2<+Q5DT1q$ zp(#Fg-PVGHlVMrra}`0HC!=Urw(NWhDc*+TT_D$Y@v2z{s>dSQhLUG7z5`?$?+6+# zqCS12EH%=))Tyf1e4`NQZWql$g-pNA*#0URwLhNg5+aVBZIexsqXU85i0B*3F7N6Z zByEOKCWsq6QHpJ(o`mL;aS)XD1`LBrs$HOhJqyb-X9DnVjPRby*4oaSCa|M}Om;Ln zcQ-rMQ)^KV&}8QIKlL_3Q$@F>wC4~;2&?T|q$+h{Yzex)*gE6~Wsuna;-5{(qx=OnP4TPiDoDx}6xnjx#m<@z2MiQ&WVglM5AZC#mpd|FpQ zwNuaNvL9oYgbKDA&7a*A)wV1{MP9S>C8Nlq+d#Ed@(>rN!XC$ zRH*0kC~QUIP@be(jHlS-qN zKzd(Z=ynjU2z$3ZQya|MGZ_Ba?Cztb-wRh$I~#j-ct;l2#ZC#Fg5#@g_!-VW*-6&E zSM*Lugi<(OU`OfL=^Jg7al*Lyf7pAg=s1>TTT~2|WU*wi)M934i`f=4vn*yNi;(5gT0bM z`|oTSBK}ce^+RpwT{^GaC~{|=(UQDko~hoEa$bpDhUv?cd4K)a<~2Off>Ga#w03*M zwPsszyjm1Rf?Cm*xv~;2*@3!{#pL;$!7mzqsssm?fmTf0>7HQTw*xoc=lqgcU8(*? zALHxmbIR8IT^!evNR)Cr5ZE*PEmHN7*q%*b1N=$m<_S}Sr!s`|^yi<{6qb#{rW>x6 ze>`tym6Rz$!Ir|tgp^}%uYZ{S*z|USS2rIUsp2P?oB&Q~5b-tYs^Hi0K0<7M{d%(d z4O%;b-GMD;1kM6$!zb2IoMjf;1D~B)o7t~uiFK*pyr!7&zhB`)z^$(&M^C{kTUYv; z5`33^P*Qb8w{gzfA=KvDRPSss6|mGDFB&x#9RxA|vM9HPGRUFjd;(F<;SwL&J;UCa zMS$mfyUISSFhUhBr;GAhK!y;XwjRgPNc-H!-pI~b%=4U+cgyck*Tl1t zNg#Z@Md@SvW+6bZN2NBNpWYE|m0gywM{%^L9gQQW5rdQE_x&8M)SLImbYIT;Ja{!S z#pu?LUTF+=(bqX<&RefFzQYE=1#Sbcfr+N_mWqw;ba~k)ECuYFqk}oeI7ZFMU1G8? zi>ZtjfKM7fIyNnA48|*5nq^ktW=_n&%?2Jj;4uXpU#_O9b{F7IRn>1LHsI*(KfJxr z0Mp^q+TK3EMw=nShcy?&DW|o-+9djp?q)DkD47_|#FTqHN`k1R=199hG7V(>- z=gh*wXu2ZXQ;H$wza1;d4-we)G~YsaHE0x|g7Jqh!oRGGw31`1$y(K6y>c3dS@c#ZT_(x7w)*%%7A8!?sGkt{pqE` z%~gx#&m0@CBT!G=0T z<~!h7yaqAydq&ccM^rLKcx{}J0>rw40YjnDa|;rIpmh0J3-gRSoBLKt%!;u$Gm9be*SUy6LQ!gH)@pG1P^(b7Mm(K%EHS0Y(2V%j$@{PdKw0F7tVlz zUD}U<)dAKNcY8aJl86eFj^2Qx>_z(nbz@e~XgUd%??~q-!%2-CVIi@&XJ(;cP-YMS zAY;oMJ<;TA@FWDJJy$E2z8LL}LQFdZdS4hu27^_sPo*A{0lS>AHKPSfm8byTP=F558!R||Gn4M4aO!gs)9gnjt{ht*)JDjD`X06*ll%gUo!=scjtYRewmX6sIs!DZzaM0A1!k#b7k$m4N zJiQs1abQ`TXOEw`1J<9fJiLdC`L??e2bp*^SQs$|za|%;rms&|3LTQn-|vLPespV` z&sJ6<6bj-k&v-UrSUl~UZmXu#I9?#X)_Lpfxg|6%QPn8Cwv@mJMr4VjCRXNe%JF48 z)mvO`k=R}Jh&+9}VtFz^UAKMRlZ)-C>UJCfs2<3y7u^{=enN_&ef2nGR}Eq$<<_U= zqpRA*QO<$jRj9-$$xC|j(ZC0#9zYCvYa$_NO4&xrrxWCgB_mC z%bK1aqu`JB%VIB=qYcAUj(XeM$E$lNPZjb|JL-*kcJDLtoz2z&>zQab9T9e|*? z(g6$UsXa~Yeqx95?R(ukc&ApqmxZX-BB-b(Z?GrayyV0ba%~0{xR?mY!9_tASE(la0NKbFN1?;HM!^hZ(>AMele1&gW}H;t8;_z=)Yn zeY2-eMwQ&7sT@C7TCU5uTj$TC)+>**s@E?!$qHV+%{lIP?tQnLnM|fa4yjSJRS=}W zhqU#=*j2%iw%Jz2pe@%d{&f9pa87gtRrxW!v7AjtZ&>*<`OrkpvnDbj8ujAfyuJN) zxAk=})`K}5+MhnqXL8j{LG`Z`@tis0eb$6HaeDY@R@1C7ATw~F&SD!gw- zN#)(V;0cZ!U%y-Y3c6tMD1#Z$sohli!4O$nl&W0oX9~7`swBn%ut=|j;)B?jVJy*=QlY?Yl3as<)eXh*TfFx>9aJ%He0veCXGGZQx zq-F_{2w;+9OF*!A^_N|4cVK)fF(R(#gb$SG*4?xPU&q0O#}@~t12y1d$B5)kv{f<2 zu$*|lg?Xle<8WbkQTI`W9g8uYPU}0Q7fsjQE*$TL9j|4T(deCFVuG#~{hU zQAE^{a)DcNQ`^=H>624dNage(eq(`_FbSECcX}W)>;khBW*c@~J%P$F7XSr@!|xNN zeJzFl=R{D69g+1+FRHW5JYs#YoP@ol1;*+a!gsr`>F17I%i9Dk+IVDpbc84d0#rSM zrT*YB0}2=Yn)l_x(t=!Ee!{^)X(}yXI!z~tm@D6#+!bl)J$Cj310fImlO13KCM_T4 zWySq>AL|XIuz&XPUt*h;mGis#fxTsx%P#@YZuw*xP^}cRs_pVdo*l8>gT?bi3}Iee z*t1wNYQ@P!0rGKad8usK)?9UpBvXZ`MTZ%ij&*Nyppr3fw}It~cpt6=`uulO=<2=F z#eqjBRi zgPMz*RQ>B=k-mQQ){yfL<>nfe%UhKMowE)9#Mx0)vLzBk&iP*{!>Gg1eMJC35|lqh zRIc*v(bR>N7MiWvr@qP#Wgg=dhqv>CI(Q8K;t%)IG=7fjK#t~5j7kmuetxq7{z94U zfhARB@JYYcw#qV*Vsq5CJ2GgEsB^XAC&;|C`t;-=Yq22_=A|awlwNV90}U>BtU)WD z2<{w)2pkod=R8qUj=&4AhW^W^JJwc-`UKhY*~5t1fhPY(47_>(;qigNc6Zu5^Z4e7 z6@IYf-ttFfH8BQ80CmVvzg#_O!}rX9WYBXgeW(lOanptG*xC3R7e5irwU^~zjB#d0 zUf`cH7(2uU23ee@u!79?IkFKI2A~DX%Fd>JCA%|rS?W$L1=4rLuoxJ55aw$GQaprP zxk@BFLxt+U!EK8`W}B!G{~)G+UdU;%c~)bK!ra?a9WaLFw8xk2$D;rd+;5K9K5oKZ zCUzpl!NDxuZe4OasC9S6aJJ7u_iXtwEKni=PWMzhUajtWM>>iH@3hVtD^o^1?<|Ls_fbeB=EI z;>@?LTeXMR+EU-JZ={;O73mDOdRfDLOt2++3+!m3Aeet&at>bKXqfcDRc55Fx3j|J z+E0eVZ((F+k9fLyDH)fT98`81Rj18jFjKl7t~Sr+uR{je}MgXFd~Jk@E-fpB-G)EJ$GEmej8ISlU1 zYbNHlz3-|CMI_|@J(7R6hZlpJ`{l3>Te(;}RU+e|mI-ZPyR%-nF)c+a zWQMA@X0WWF^(t9_dp+OjSIZS4VP&7jsdu&#p=NluZaaq9OQvIF>n``jQEE2za0~po z+I+{gG9o~8L+<5O%bvJb<8jZN5z)x0dChkRJ3HSqijO3Q@SM$JaVmoJp|oys4Qwyz zR?l5MC-`W!(mxnRx8Zx3aOWk9j#2BMP@ZL&x4Iz;-qQDvxe0haUn=Z>F;9 z@I13k7^Ir~HL|zNJ?}Q2Z-W@~3>F2nw28}re0o}Aiw#yphRJSuDVcZOeIB4K1K^6_ z72d53Mr4Ee@GUU~c(jE1I+1i(%KDpo0fA;ks=l^am^c|#B+-5zyY{?A)y<3jimY3! zqrFmy#yg*XY9fv3yTytoWhcLON1!MaOQ1d#J6`WM{|^RUvuY+MOtm3HM#-c(kmqSx zrEqvDSJ00>UqXFWdbx<#3k%UlXZzR#DfMl~{77%IHs0q=e(Y(iwa%za-m4(&5e%J2 z@E1*8ZVbc_&=?R4P_cY^3Z!%@oKL(~BkaTUykK)QTUOl~7S1QP@h+B;Qtx!9M|eAx z8NR)}}~Z=+=Aujj>1S-MDx!@yy_tQdanzx0*PXpRd8@#kC*7bVoWAsvcQaQU8>ixZF0n-f^j9ngSELlRG zyYF^zSW6F&h6NLsc9v-_!Uv@U3_A69PMi4k4RgC3^H+hE>xAVp+#OD?_L%(b9cBSK zMl-h$w>jg|O`oSaZUjGgh7eA-o7;{JsG0{FY*D&3=WRR-cy^fLoLmJfy8ULEK z{P-xP+B1hB9$urd1Dls|ldY#~s$~T-!zS=47(5$1jSB$R94^u;2#swkoqEA)iL}6% z2cBRJ4iXo$)#);IT$ShOuI=wnjgo3SqB zer4W)$;b^P+(`#Tw5s(_hAFcHPWg&rpx{ zI2;tgOBJi4NhpznT4ZY5`1j$|&l|09U^z8;rGt+a9 z@jB|!kY(-X3=07zWgJdGIjii)cH@N>%c8LysbJ6whz-9kFuogwg*)>&ldE^X@zab{aa>Y_XEe0$Ze;TRs{HHdFeHs2(<`!)vS3VGH9Ba9Y5*nv1C z588MSli(Ts_`nHfz%W1C;_eqOWAzd{<6>sGX3m0UP++oDZDncrp!*|Br zZFIS_e!Ccrj#ab+Y@y@Q4mmx(AOYj9lO@pF`es@~rr6gfyPZbx#7O_?Y#W4LZ%0ky z^84Zp4g|DQC2N6p8LJy6;#O3?ta)8uGjim z!>?6lU%buXdt%DpBDFds6Z8h~0t4$w^tF`x$C40&u@3#9Y&V+W6d?zj{N!NPB!n=r zV0f<`nG9n>gm+taybDOv8lrJt(wQ$8J3YCNzyVWh(uCvdO#yD+x4f1QC&cE;c`X6S zJZ_irv0~SLj$I)kJ3j<6c^nR(5U7}JkUZHg)UelDTP2;ZC-AUYop@gBp&)Rm?~w8w zhB4psWM!lY3`H3HgUdeXyQmg@ERkM+cSP}Iw4j@C&eo22HCy)P|B@Kf>kWyFb<>0z zLuT(;ApJ%elT8=HPS2~+$p*Qd*Hc^6BpIpN7~r>ct)Zsj0}tn$^>}c}cSYj&bn*Ij zVDCx6eiQ>hE~NiLfGk4=*^OfS&_=qe)5<2}>kcz)Gv*sh1E%7JzFgs7AI(0{o=e)HChN^n1PJcp~78=!Jn4&Tg z0H-q1d-d2_ebV|zu)_eb3UWiD`A>#O z8|jrL%(a-^j6OK7eZtdETlBr=@OWlCy*_)C&iS;Ss-&gjFs#hD7Oh-P%A_hdc{xk<^dGs#(~ zyRYZBTG8>+rV%~g#yTj3D){5rEc56alWEq)<|_qz)svqDh!=up1|!Ui)Uv>)$%;sp zkl&$;$#;<0zlVrNCXAtybG=`&uh}?|nMlv5zUb?)!ley+H~cbAlDoww6D2TR?^;(xul>^e0_@bSEhWetW5)0d@)JQJzgeWnK2c?3!xwUb{kNPu zjAW`xNG~JGK|F6FvwB7ES#NAG!;-8j#dmTzxxxgR`aGZ2Yr2bgbDEJOq{`|7fIP)>B#WD^OEsll(Wgo`o*G)E0V&R1sx?$EYDz zXH|Og3EM*ciIP}TJpw|@SX9RV^I5e-G8ulf=JqER_RS<%sEyzop^Jv!58T;WIC#cY zhH744H#+UtMGaUlbpp32?1Z^%k7hew8N*Uz%(!m9WwY1b92lZmZkC{HITP$UJK{CJ z7-6Sq8e%{OBX`SCv)%5F76L>x{c7Fk%NTB7p0MUv_pgXZ(X#P21ea!#@8Twsv>@-v z(743n;>Ot1rj!7b>X%;|JHn$;NbQXTwgggJ3iylq-7~7%|1t=rnfj{jCASSXdVDhpxb&XZPjrrTAmIa zGW7szK5ZXoxLhlRNt~S@vrmu032HK`jUvLpweNkQs;|@oNLO@QJ1kjy=h%$#=si{$ zp+4oy$ip?C9$b{!w2$P{d>$VWS~tEv1JNBi17j9p3JG@Tfo5cd<+TgP26PxOMw^I{ zzd=Z8si=3JgFuGARgTYMcwT&TysR{#TFLS*fbA|d8x}h+`4s&GK}M2yxiyLSQrD`) zH-NyPAq>^Kca)>!Ro6#=(EE%PghiX48BmUt`E+8T-iGj9-Mr_#No4H2)$QDs|E-Bg zWhKP``H{_d2VS<97DUGjW%WDLVJ}mj0SGI_NzuIZ4)Mg9_jPYZ?$^%(s(hVp%;TVk zsyO)^aGl;df}p%G5W0LwJv1k9Yp}N~&{l+X3EP`3xaa1*3uDj%dQa?BE6zvf>C~wG#fmnEIJE@xMT$-(wvJzb}7?oA_`%Z?r zx&Cmg*b%@OybC?U`?8RMq(Kz9Jgy;oFg2}@1>V@NwKV|jPz%{GX;5_>{0^HaVHAg?vhKsCRJqZL9a_jz`D-!d&2J)Kn>sUUoX6+TBDYdw}wS9Y& zE#@usIp3tQ(Uu3eG4wQhaDuL!r#$&ybQUlJR{CYXQ&%Mb09a6O#@c@UtgUeqM)+f8 z9QM9}Wy|?rY8ulQ*Q_W;hS4qY4u(m{TNmbB?KEzo%Jlp%JiVbwYBCvmF=0QO`fm}L z-8lbT8;bfj<+E!wF{yxn^q*Si~NIi>KR0 zAS-E<&e9bl_G9h(kT})bav^NJAjc<+jyI+7#<~vQds<+86_-S25M~nOH&|+~9m}^>36|Z;Ha<3dbXrS2^>+JT6tkUm365+X5kmq^swn-TiOrff1L$|ciUz5jZ* zjrIvzT2{`Isk{Tx;+5#!d+|6QTE~-Mz)0#>l~+Ki7MZtNQ(5I&dufl)#dkLhcx2>C z`?PVF3=H^emHNG<;Bq?l?xPvF-9}3K*-!C~)?D7}{8&$uQMJBr*O1@yA3>~C)abev zPkQ^iN2ueM^>z-JQbWIcK>h$8(X6|jQTom|yWh`-!f)KllohK4nf9I)&@-n8mkGh~dx+^7d)9t^@X>6tjWZ7TR(MY)rxL_}GV9Oh z)exml5U3i38eXA^faUO6Q!JyhA%C}bqmw0C+k7FkZ5*M5FMxm7p3PDTG&9R$DcxaR z2yA7#X!4d+N7WR2k1XDqU@}?60Fyj1;nF46@wVfbS5vO)Z!r$$l;9KX3KCDqJ$i3} zh@PYAgH&S@1(wN%ctrP;KJG<$X*h*}C!Uwel3f`0 z{pqHpFlwZQ8>GAhQc&v4&qG&ESP}yeC3men&&x=K*nWieZ`CtO36h{G5KzQ6 z;)fy)FU1#M(Z~vSH9P<|@sn@7wI(&K+q83lCQZ9xQ0{a+CuZ@9Rc<|-qPdTTtUh$hZNg%|Z>A{^Nt3OZ<3ubZIsTjG7-110`xfRzc{cUe%w*f` zwvwCE5?B2a*pt7t?$;vnU)K}yPnSm!FI9Uxn!hT~^vaAz_3b>42YzB0=6`J*G-!q- zpYWkruBer;=R}T^(ZHaO74Ed%@j^q;X~i-eW*vQO7_c-ED7jN_8(?avC6P7iUq5bF zEC;&@uK)RBIU(m+z`psn;QEgPqA09pu4M44 zC41L0%6w(?X;lYbLgSV#_wFg~_qzLz(Wm$iy;TSq-5lfrUh`#&%3B3YCiYVM!| zdlzF~$t#y5#2fxnWy|iwl~&9xP6>x_FII=jXGiV+_yvq>93fZLNo-_LQM)*4rm+eI z7t%s7kwFzb@x{%rKXht7McYM9mN+xN0ZP^NAz7XS(c)W8>Z~nRoT~7dNcd2SO|=Or z7XoVba%^neO@($y;XPabB2e$eJb-|#g!ysVJ+M=}K+i8L_2e^82N(Z)3eE5x5qL2^ zpFYj146M_G4t{lSj~=fj9^wR7*i%)!A=5RPv7Du7QFy@hSkg+u9g%S+FY}MG zYmaElj)PQ452c_px-<7qCEQl%;cLB#;$M5R#HA|PcvZWZF55gxbiay*(!&-irc=G- zzDk_=_)U4P43w1yHCxH@u~UyEd#={n3}%7XYqS%O7+qwi4Gz3*wH>NvqJ3ig5iM`2qpo}VJs{J4BZiENK*^c^mH#?^y{?E_2 z-qGzdWfDf3iRxZFSdF~;qBUfXTtj5Upk_Q#v#j*$;+J3^ymup4g!<>V_$1bxgNB-V za;}AeN?PA9e}Lmvz`@IDewh!_)xFBzvpnW1tF?gCN}*3IqaXeCVyVT8J*ewZMZ7bd3cX6YQjA;F~bafRbPd7nZ^bd-Fv>{`901s!{{F?AUsyrqiBdS zZ3SD{jin)=QtEBFHhCHyQ_p(lLp~LDPkuF+z^u8LVicJuaGc0JaFb7x+*7Pk&}%du z$n?L_4jV9cU(D4_t+Gt2NyVM7s#s<>o^^VfoS7T0F&dSW?TxCK+z~bsU^yeWWZDZn zw53=rVqEfM)Bo>w*YFVWQOi-uKtQ(E`lVrDNZRMaQY9Q1T5_4Ajj+@N>IfBZ<@Mxg zwu8k(`1T4zvVZH=`S+*Xz|%v~UJxI8H;p0M;FbFOo|s(Q(5uai z0Ah!c#vCppZ*pQ;pW@gxEt&`VDDS_+5@Mrn<57HECBk7O)7^9E=0DUd-k=1f;q3QO z-7P>>JdA|lI{LVv(`^$sB21z$xef6^24SX3Kqn;Sz_vvI{doqKu_2O2P3pP^dkHS zhkDuyQ!3tMVg!g~*3GR3*irjL*dWDzU4&wnAY@Vcsd4v=9m3C$5y{>@OjJgo*`0{S zgo4DY{C8{tv36rXX7V5K@gLqmAdpBD?I@h@jBr&7A2u|ksw&JyByRwgo}Vf{B+g?6 zu?icAry;24?37BdzWaHOkuDLJ`de-jSrJ8d>!)eF2g3|J_^<(0RUQE4v;8>2el-Rg z2cIDg4c^lB*X9BU(BV;2n!E=@!(B^fySDW(eLSiy!&%lb%=rF1VW1%XO&&cE4vcZk z|8^q#AGBf#N0A+7G%K~6rFRE{V)&U;3Xp_q;7wrS z#P=~D6}1O;TyWVqGD$QnLjAeSteko`S*{0^viegaCUYQIZ}uc{Cc#^JdQ3a@4fL{oC94uKZkDTY+r|F}u4q zVc50FubPXbxgsmUZ7G(0I$#q9J=GY=67;GfqTTB9G0}y+M@?kJ=fqlQ+X|vx?#K0X z7mLd<2ljl?@fFrwRGldye?wBGv++o}0$O$JFG!A1B{w#|=^k#-ye>rlFHJ6hfe>My3`G>k2GFZg+#vp%CjOHE=IhI&{rAVoyX&Mu5>a9)Mt z?gCfx!TatEnk)2}=j9E&!H68axOG?NOPuZu@o)#n!|X}t&>N05({ul$zXF7EoLS^SYoC_DH*u_2j)!kAd~ z63cQ+mn`nqT5e|UXSrpq*XJ|AlasLwZsx3%c5dd$Rrgjmkt>!ieE>tD+g?z8$b1m% z6o(pk%z1#Qn7L@GZUK7u&&xvN{5P2o<}#z1k5}ENkjiBW~`Yl~+0Pqg1~Xi5z$gHdOrmS*z6g(G&aQp?CVsNF>()giv^-r*=yf>Oym1;@Px5Bf9fM^w3%NF=y;p>yF;Bvw)cKP9`Sg6sx4W@A^Qvj ziHvUD{nv=EZ=IJGhN(O*Mk+qQ0d+tCX4wrsLLdo%fd1!?p%j5T@R&lf?R@U5FoMT$ zkKJy&=Hra))i;~WwH$HS<1Z7U5y@ilFk#N|<8bfJ**nD;!v{&%_S2gJusp{*Zz z#juawhZ`)f%-!PGOtr~LAqLBW5^=H`wh7Lkcg~;8idQ>;EXwRHD0VT@4B|Ufm=<<(j+S2~kzTIMf&Sf1*};Iobe*WQn@p8nRo5``E&zm8R9<&GfcRFFD?Xt6(qMi_xX%68gc!d z!1a7WqxXQN9#2MhkItO#BY}PVF3_g8czThZ2cvJ_+K*Y7m*7!@9jF+?|1(7XjtEYS zAuqz2RzWiH?c$&NGO;rzsHQ2w^;_~iSpFgUU@=?j(er$=Ux;VE9SL)NJju?p$6VPd zb4K+EzZQ!SFT(EqU0fOYu0jK9+XM~yMIi%GgX`E`d7w#7b@STGC!F1}eZ`AD{O<1z zJzx+3F&F5nhbkXf;nb3vI6PLOow>kpOq+0aFCM~`Zqf6G#T~8lOzfD(+gh{A1nd}X zUOwV~$OyoJ3SdCO2pc0OmCYhGU3f+a$woo}WWLNmV4U*DB`{C@!j&~(+@UJQ^I*X- zLQHdho`fJ6uL9os4RFB2R>77PA^1`rWGKxb2!fmsdZ8&oPoe+{bUABKztr3 zSfpmX<$P{qdery0@sTH$HWEI~$KatBFT8G@6~=5GR#-ND$DdZ@A1|$o27?rg3(4)P z^u<@4aHay%3@nywz2xx|zKKwODFT*t8}EjyR}=&eFU}UOHA0$hK3+6DG-3H91c2Ah zL--%B3*t)yhZH=<#IRIEfduqUZuInW3>l7fHsij1=qPsBFiYewH%{k6g>+ zHOpD!+^hht?;ih{;DBa0`2f6T1$}c7)y2r@GJpQO-_0F@*>=ob(?#~}61=<4dRM;l zlj_?yXl#Q9BF*RN@Wz?%{M_KtWE8F%7Giu)E@P|VzqtV@L=7o0&NXUw(=z`G3xVAG z$q%1X%f{+1S~F^Im)|$D)5mfAXjr$wl@y5YH z3PjsKGYQbp=`h0JF;Xh3dg19b2?77aga3kOR}+S*HiJrNSeVhw-e}S}JUl#*_ByD$ zb-%28zk8MoI}SeKEP#E({Y@77`yTLWy0K1;CU0jQ&sM{uqjAq?t#Vh_*9Ccb#Av}D zr}YbGZLfDLOdK4hkO2ONP4H&VXExsf@ z;H~Oqmj6|O0_pMTxN?KBo@jgn_&PFjG9)wYLS+*A|ID-hBYe8#x*!1KqY!}e<^%jv z(fnfcN5uCF#2f)b6O#ix6H9T_U=o-^v0;usUCHJ{vTi=CFqef$$HR75j=-+>%b@ov zC{!}B!n%^ebznu^Qkv?9V+5s5faPE9w`$AaL*mxfaE5PL+{WDm)ZPO-+q~M#P7Q1Z=n+(?)QwSEm}Zdp<+DPmx9v2OgKi<#%2PeUhpjLs$I-L$}JZ5afM0wqSrreyhp08J%kinQq zT}1wdY5q1s@rT83@wm4}@8?EVBsH2izwT^wpaW8>#qkO&~={gd(fFjX(z5w6^+@rjYJvVtl91knCDqwn12$48x z_>6x;TmLmCexQ}SsKSN*lkfh=cfNYCz(N9pfcZCs_4k;3vOs*r`Oyc6{xy<+2-L1B zplLpaMWy^HfV$*-1oE>!?EJ~!x~f3a1oaP&{lg9a+hUvK0)#Vi0k(ho9Rm%}G`-sf z#(xTo3sgd+!=j!)(!i%gAQbd28JYit^#`D>Zxr`Cn>H4A)JJ^@%Lv+o@jw&wapv?zVbG0LbJFiqLyz-RomOT zXtl{ss6w-yguTL9%G-suNIHcIQba^vQIkgGiO=9D{^vqg{2%TTZs%JD4S?pB8*WqL z4jMv=o>{WbZ)?-sAEQk0qr`%d{-~XF>{#dkhnePVRPy8TI0imw0p07&1C-VWy-?^rGs(WGwJNVm(_>nfc{75Z4R&ZB7MeYjnl!H-z-^IY$`2WrKd zbA+B)iy@BC%SMmVsTSiPq-wvfsKR}^KEVrP7x+LCM->o+pLnU^m#-#9yN<8N$unD+ z_po@O${^WgLJ^^M))gDpr5MNsrT#+2NVxRK1zDu(>{7>#GQCBPH;ia4^=C$ex{kR4QB}c7X<&~ zycx(KL}`%JoSTFa$<4s)8?R&2&DZIkwIgO15B)@4jRt$ra$1cB^Aha1%#M6`?62a- zB%0-Sv(wl3k<5c7gW4lnTB5&SN_b!=6<7SuTPE@-v6j+RDxMAk3H89h1 zP~hlV&25=3cbR6GQO?SirchCh#h#TrKC|8+FCQ0L8yo(IiiL0x%m9F7Ce0;hnWV&R zP>P3P$w6qT#2i=kxS+0CbDaMZ-ENfKx1(+6>(V|darvyk@f8|) z`CU&XL1jX{Oyx2`?|Fe;cAkNECES8j+6R_i3nxguxgrCO4-(@k+#wp)g^d6q{u=lf(4aN{~U!yy~}E?63^^ z=U-Hpf$#Dq)rV=7XT?hqGt+Gh?OtT#AJ6!xqmEuZzl z>ljm#kqbr*&m+}Sl%gq04!OyRe?Jr?aVcHkVoxUBs#S5oVIa9pZcsMEI>EVP=@y%Y zrqM3|Gm|cfmfj7JF6A^#y+R-ty!2m??@wmIxijCiNz~b7AS*Tz?$qRR7j(P6BBAFY z{RBS(UW*^6Z_OX)pHXA5UV`@2c0==--)hlaet~{Zmcf8o7yb`Ol7JXcB$Rif7k`|g z-G;?7*QNcsS*f#JN4Cc|DLtCWNhUW=S>*y%32evBR7w-JTp;_3+J{PCuQ)v5fA&Ty zf-$reiN{1-n(*;?&t(o^c16cEJP|4TQGBP`pe8ys<||6z@2QiCilG7MqBE$8Pj7+n2FU(ffjge#zOcwk&JUF3lNjaz|+b@wSlW~QmnUs;K ziJ6pG&%ZilQa!U3*%ZCM8e{MhKDvU>c8QuRWEr96#%Q@KwQlKxGUHbweM) z4<#p~Dp2RHWtBaZc86@G&q%0Ua3_8hSl-#`kfTi8+p8^mqmfBH7ArA2tvw^f{4b=w z*8ciC%3TW4hu81eE*V^AE|7n}K=RFj4G$B@N3%@4Ye-;Z#dN3Vp_OWMIi?vn?w#%o zG!XDlFY@qDUtk?-W+NL)VL>Ve9iz&iFF(TrZ}BY3U0HB17D=??`L(nIx>RLRsmROM z%glFUYhyFT{HEQUv>c6($_My|n(V*m+fY(h6S(rS0DrpQ`lNanp6+HIS-F8;ZuXF0 zRG2gLjk8Q9c|b7~*E~WzW0KzIu=VrZP%hV@sRn*AGUuYJ=5zbEnxdAC`m7;Viw+8V zV|0nHg1*uzpX;U(`pAE~!0^?C`D^ahw0-MI1s;cNmv`Z+_Z~qsQ45T@)~JPm(Yws} zB4bx~os)#bs{*^JT96G#+p}Q~P-&CcWIZ`T`FtSD(#94lmMB4m8g?7N?N$fa8!AtO zGbMBx!qXG~0V(u!jw!C_e^SEnS)xq&tOU zt?LRnW$=(jxl@T~o0Z0r-@BHQP9z@GQ!cn9(8ZOD-7n$EFKw#XcM9Yue64Kr$X9FN zO|#;VHyp1pkiT{jteMl}`YlvIz=TjfDfkXe?9|tk94bu)V*C}zDSN>}cRHMauuPBn zI4c(dspHI~`Mv%M=_mU7q1*^)t(~zTqe&o^?0LS9CM&)Xb495fvuQiZh2sRnljEEu zooTS|&J@(zz1-aI9U;iMSD&o{vak`IhkBAH=!xU7{;-9jI|ODxfrM3=xP#4kCmF$U ze3!BI_Jbu~M}h%%n@MXzp<-`n6TLX|KHplQ>@PQxy$7E}|# zfE%6I5)mAOCpQ{(W%~L(>eMx4X4!b1R{_x2JaYR1VKC{R{tEbf<1mj`o!^_oEDG#|b~ zB%C=zt)_p>F$n`L8USFKL+Vj<-B4DfX5rjpwAHa0l+PP`u#!2N3QzkX$k*;OQhH_! z6t3b_5OH$+c;#4+mMYST&Vcqj78JF&hXmAfs#+F0<;gMEk76T=pZ26|m-}-rx{IVX z@=(&dX*6g}QQ&c74CF`AB@_K&3V~z{`o2Rszmx3lT0tKe@hLLKeqldVUcpY+uB*Lc z!+w>lPjJ3NJ~X8pM67$Y`WD0`Vr_+rNvFJ&a^sS#Vg@`DZ&H>uraK+U!_}S&T(R&@ zqn8&G*1tI_vq^{05fR%?j0W)_M(Q>dkrH8nEV@hSu0=1AlJ2fE@!98n+^>7T&;P^uaDL}}(;suqImSKiyzXm^5z8qm zV0TnXU+p_3vov@+r0jNW7PT#$B?bNlRLih~q@-acl%3)Bt=aXJS(}_h)JwPRy%ri| z!;IDM%C57fo}^C;l=+lZJ?jUU-rDL-#lMsMCoN;&Pj*y=*tM5#BT_}QzB3nzXb;+G z@suuuZKmw*nrx6f{=&x~u0e^&O&<60BUL&l@54~Z%1kF?IrZA(Q)Frz?LxI=Tf_zy zQ71Q2ikLFdk?LXm?T({zFVe$)^Kw8eV#>|D5gut@J1Nz)u?$PjCYYh9rRQ;*6yIV=D#^XvM2+cmC&u35Y(AUjg{@ z(+`~J|0H*sM*u!H9Z~!b$Q30A0sy#416SJpqZ+XQno)tR{SSUf!VKje!w>0V`KK?( z=>e>%mlTEnIccX1X~VhiGQhpp|HCjSfpcS9?^=$Jj}=T!O}&U@c7OY(idm=xB_$#v zq8rrM+fp`E!a_%{m*9^8{)TD)DCf>7=z5u=zgz5-vmWR>TOO|TN)Nuf`2B;FF>O0@ zwU#a}8bODNIX(1Z2dMRTw^z&gq?a~;0BgUh&1-MoEIqC|J%9g~k&Ajk$zN7Lq?k>vr!x zttVJee)|j1;Gsc=|HEEYPyoZjf5~4N|F>N76aaU@FiTweC#Y`$w9qV09eRK76a8;} zzff18J?1vQqs904ILwj&4@0^t{C4s;)%z8i%s#<~-Q3_+Sq&vJYVVTS@hw}4!N)IK z=|Z0VJu-6{U}TSHG)w9JeImuE$Nl~C83h|N<74X?Pbty)CX@z=h?LqNzg6_gUU*6Z za_)Jy@P>U?lya5(P3sr}l`%2`#c!t7(;&h3FrJXmGzsSS12vrh18L79P+9MHh5j`T ze3K7`27Sz{AB5pIH1(+>glm+Me)weva9>p%`BazMHP~4;{6aHDET_?d1#*z$`mNfc zWDVU$qRNR|HPr_U0W2!@^55yRdXGm;s~O7tC>dgdZgQqK;~1^ zc%P&{2M4mUxnw;>h8&s)e!>*EK3}tpwmghGFn}evx6}Hbe$&7&`3oWVkU3i+>y_>GY!t(JaT{~EXZ*Q) z>4_T>Rg~3m%A&5R=b5j5gT7i#S=D8nM`7|W@WyEscZly$j*?5N{qLoE056E!V?}w0 zeYZ`Vel`OT9R+6+SrC4+T0lDh$NX0=j){2V5G^t4z-X$)=Ds$Nqy?6JjHUdB?#v|N zuz+L%PRw6!D=?kZf0dEb6Wet0;FE(eS=M6)ri2!fA+sOwg4cam>6|{scI)6|-G>=d z_?7c6-}4Eo*xpafMz1|0W10<+nAI5~&~@Pyn&GYbaFDjGr}tgW@N;O74tzWt!|!DY zhZOt_hRm(Ldu4!5xiTGMSi3NFRUpfw|32~up^?-dB-?hLAOtYBKN?>AqU`ld)QPuX z6N4?K%r?oo1)@=iAY9sZt{@#!wwwhSzx3E9c1Mxr8ykwnQ<5ria zzv599s}Qec8*ppuTl&G&#HAFpaJ_v({rpMDd+4@pT4zj+AJ3`_C4>BSrtVUI<8g*VWj zx;u^u8QfKeC|o5v7&6Xs=1+<*fB*hLC0*$GtLA36txa@@Wf7SKOY%BELE1)@Df3-R zn;1YnS)cZ8GSCU1i(xg$3)GvM4%EZy1036G0m`PRU8 zXpYHzowH?wU(`l}n9JukdBH$(iWS3d`|2g4GZqt$xnk&$6!qp{ok+QKTX}8zr1vd8U+HvsERE zUP~KHGlws2MES&FPnn~PF~7;VaXlO=)ZeN?T65EWsBH?s8n3MMz+bcPP-Ofh&VOr0 zNu=EKV2RVV|Bi%z`Qx5Qi95n#<710D=Pr$-!lmh~>hs&7N{H)o>HL(HwTD+L1@!zf zb}~px*22yU=G)gA8JXbq0P3-J&MwR>LW>-0Z(k8zfv?aW%Dd%X%~uIZ;c=M&4hbedpV z0PVrTq|3%C?^=5Zc3VX2w4)Jyi8-D|GY11M4 zyZk!JC*LvH+x)0;HYUs-|816`+xKMK7Ea~gV$!ehONk7>Fs5a~h%8Nr1zsf_G@&zP zfd)NBcIrV_MW(KPS}m4GI&w=IHn<8o__mKx(7!^w13G})NU6ypS^eg+9$!fk+dGck zw2dZ1Zd0C`?kidE5zy!ma5}kPLyIkg<&`~lOXe-dk}|QEyr~aYKQc?HTGXl9jl`#L zRm`-ZnkTmcrhX1N)%Onq)B5(crU1H&BKq(*JzW7}zlB>%=U zw|tCKe|t7|NQ`R7m2Tm?r?zk=kN=0#^oG9y%tbp< zlqu&{enAaF-19oK2K>S_$Pc=5cH`LTYL zStJ5mR*G(CygIYlxr8?lj$osvPzI6Tr3+~ivvn)0Ixi{4D&Ytr&k+%8T-T+L{^MQr zG2*wS;`2{ey&_VF%p417C|-zZ@cr-%^Zw+^M6G-9GNvmLdRb_KEZN{RPrQKz`&Qy!Hn< zJd~3L$Q&_tSab;grMhND_xy9B#`yRDuDTCGEr41t5tWndUk^VB9k?fem(th&fgrRo z0CYn^C2x$9zl#+PC%_ZM%o+E;`Qz~rZGbbT9roh4G!4QIWY^^b(vFl$$`mYz6SpW_tCR$ke4<0_ic>pHFDG>E{ z-ki1%Oq>4t>4V=BieCnUFAb#0U#Ev1X#Zs^Z9f5~ilE;qqW!Bs9@hs7iF8>b$Zv^N zn>;XWY_z=J66C+cZgzdo-V44;V*R@Ydb|J(ME>md#m*m(Q%VDW;zm&9`^}K^836go zG9>&V>$YhCEfB0Zqp<&*-o$ZwFOlZJt|z~RvAPc1CS&}p%Wk(QQrV<3aaQCT^PfQYC+-;BeD-qsH9e?VLH3FwKy?o668npbLeZh7%C^%F8V3o-oWmR#V&f)`)cEPGC_MUYMgIC> zoIaepbbe8h{9@qQ!hn^E)9+8<;Bo|3s)P7c0hT@WGM6B*g#Tk1|NC!%WdeAL3Rnog zP4f>8xsN_Qb%5POCnh}om%9F;qkp{@=k3blKg|DM*7GZnJqV4yw-fN$lKJri%6yO(XIsqrBz#_NNVE>bhP*7P+b?Hg6 z^^z_mRzS`zk(~b|tZ3X&^}tPSV&n002m6HQ=vJ2WI2RSMfh>8QtHc(we3f6gbv@3JEH?z%~aV5;CP&IVa^P2~2V&vdj!tz|p zzo>N*ey<}hqdcPKs6kXAj@yt}p$ImQ9k`p91|V5M9){@g(RYR4Snb+zfPu!@!}WbP zii+J^brpHPcVq8Nb~Tgo{oY|eZiY{q%=3nQu>!o--gm%j;X5)5meJ3WTl?d>pr>(B zj)(UA3u$gB#t;6;p|*|h%;a=n@%Y|PSeS-z30!sI+oF=H zsd~-V^B`;S*u9(nsiz@r=Qtlq$tkYl!H*l^1CYCSTg(TS(OrzXYhR$j<4d+B!2y1{ zJ-d*Dd{(W^(Z*@uJ_K4w0rJCtxAFqrMPvqV`6}y1q~(m=5a5L`ONT)G0=DIAUT&l9 zlaV%F#qh?mx4DwR-Y}x5Ai!xO0g(XFKcmCPMi{O(x1N*eUUH~_`7 zyibqL7h4_3;8CKn;rccYDkx^&RWP@CyKL_YPl>cV==KaCDWv4UMNlCIq9MVyE!uMK z4#NNY1ecB_!^h9Btf?8Va?f+*iacKfIrjljZH0=Jpxxm|K%`~={}8FX^o#4toEsG( zVc~)gK-X0j#E14Qp}fP!MY$y<|K7)S6+N)rKc6qBu6|CcPEDILw+hK|-x8flJzb`_ z(;MJC+E5QR*loSLt!NP}6^A#M1Jvwz`?L3=ul#+SZrw+31)!q+kaVcB|E=Kx*PXo%Jp=$WWqb&v2Cte$wZadugU_V}1J1wP2hYQtyNev$NYm?Y zrb);Y0UAt9OcuNI_3;OJj%NAua-fG>_ig>?*T5hZKE=W;0TYJ>Zq0L@RswmRcxa{d zaG&_?vUuOg7IYO{yZDZV6I)sActjdvHTdKr+4Lg0#|H`Me;omSE1h4Jz8k={eRu2E zSE`{ViTB8Pmauo}%wFT37$XZy)$VAiUWF8}O>Nf$Lc8-A@4K+T8!sK7=NF4~Fl$n1 z?|X&NHlIiTW#D4UsD+0az95d7@&+$s1 z;@%a}ATor>YXx7kf|hNsceGnsUy5H9a}1%Rk@&!_*Z5bF#m^r&#x z1zw%@c8lM-`cHiF87`RXadgo@LVA_^ZGBa-<@Vq{dz1@nQ%GuW4~0WberrI!rcEC6 zW|>lZivzDav3ZwNw?^g^ChsP09lWL2p@F#Qvy3^@}J>7ct-qszZ2qr=JYC_jZlm-ju>z@0|JGhe*TsNOaRkJvgbQ8^Ko^p;N3mm7}xO@ z@`67ylQTEZ3%EVXsg*wvQ$vuu@{V&~{~6kK#%CArU{gf_?VZZm0DQE_U#>3uh1}YZ z<%?^ubD(m|S@YhvQg46%#+C3eBh$;V)UU_AKLT#LoUh-cQB02>_az%b>y5P<*9%Xx zHd;2ogvzw+9D{aGEO)Q#xkN`*4?^w7^q#XD7v2qKlQYiDQW>sy?L~1dD`EtJ<=ChFz)IGVDIHHV@OG%j#CCkp>smYn~Q`YyfEYS{KC_?p?Ugn zP=@8)^@g-b@~D9MZcNes6!O)|b>#X=y(qoZ70oNWxNm+&V3HrMguQRzOhYZMX2gb$ zu3IMjeh^N$Pj<*Z>VHQVRlpH9#NXrXDd&)Hn?-ZnH-5C|Q-Yi1-e@PIE?FHN1?tHV zoW>t;5qxA^qmW)zRZ5{HNaEa|xeNm~eHIrNS7f~OjA3VF99^x^%NuY#3TF>!7o2%g zR#uimpY20sB|;5oyAOPC-{w!jXE`dSFc!=x_dvCet;q2&pY@N*fJ)X)8&9sc3>JA{ z9&|Zv$MX3vPtej0ZHCa~*F0~peT&P>WA%NswG)$)ll7N@-PCeIrfa#bnI|>YB&;xb|G0{3QtJb@#crm2RxeOK@+p*hritA7o zrxwlyBec&yb_s@1(QalWqpUhg=XOeV^u0OG~T&}kv4?O{n3%7&eSGV-b7eKt! zdBDROOPh4I`lgTWXhyBc<$mQ21U;(6^VC<&x_wW*5ymQsqYA*H3yCJU)-x@|C;6oe zDt;T=`c4@XVf+PFn$>!(bm4}p7WUzL&Js;VVaD_pnv+iUZoP#%w)&cn**DTib3n_C zD&yYzn@slKZa&4Pf@Lydc$CvkB?drA!;I)ObHhW+BKn1xjyqi9%_R#K`fC<``)iu| z&&b4vndkE@+usRTN@}Q0up|(uNrF#?90b>lYLF1-4D)H0vTk$7Zw6u4@2hHRx-8<9 zD90v{X3Md6N3AU|D5!BBr$|5`v%Xlocux!L&DbVBrQ*}=4$`lZk# z>Vpr^;JdS6LshlAZENu;{Z5Ne1@5q7-VLI!(Y?{Q=5mxX-Wpn>W$3!fATgB{?*@Wx zLHZ~iJ|PvJI2Xus@`FYirPX=CF9tEB*n1C_yLV!#Z?l0r@+Ptn)6e#!w-!Y6C3GR0Gpb;uul#J{>_de?9h z?+p};A%Cxa*{btoL?siRNam`@9T(qaBi@DOD2(uoUHP=HQKqo~3|4J9%Gx`pQv{^# zj+EGayis44zG?LE4L2iT^g!D@e(RkDddv~d_3F)}mVb|{WWmJ%)GR=Hpz{PU^P-LV zBi=J_8`ne+2j-|&xy2Lma9H;_^Imqx+0;&aUr~OyCsTSH-4=WqZ2C2}1oG;STfdoL zUkMsL2257N@g}FC!4tWuc0Gwc_(BW-Y2ng6#asKx(c-=y)1JwK@S$g5BcacU_DZGeZn=2PEAzP=4nT9bjU zuydw;^>Ksj*yHAPHbYRXLihs4mPG1yIXI!G{$)qe&xmSJ`hB}rfhq2WYucF3uYVxQ z@4(2K=q()hxyOKU3Bxp$n!nNSt4Ad$m0|?MJx=r8MruyK#(6U1IpO%_$}F#Hxgq(8 zRV_Mi;5IWupt12-0e@N5615}5zd1|YrVL=NyPejKGMJ7Z0pe^cw`u(I~tS33M9bZa99tIqrLOSk!ymvGK#{J&#}Jimf=TfnZ~!fzfZ;-mIENjkPg; zVdIy&LcO*{B^y2%i5Zj+uFlabRj4o0@kcoYL%2HPxi$bAMgYd~d=f}>?bIt9p^$L% z47S+b5L>kxm0W!#JwsHMNM9$VcRgA-eORTk zF~;`b61b3=UHdfMjmeB%ETMust1Pb`_tOHhOjOS~KhnqU8dUPl1JZ0pfswps?4PPP z8oBnXf~x)9G6(%o(_rJ4%gORtd*1xBSFhlKM0Y=K&5{lQXP-zxM@M)#_vNtS^^r$1 zZsEYVn{`IEy>Ar#O}j^ux!7u=T5CX+n%!B1z078(&ES>_rPk50eb+4E`CCk$H+?Ij z@yRt5YEda$g)F=U9_On9-rV#~R_UWvxe$?C=a8((tDD%{pRp^+x+bo%Do5irj&l|V zKfc)x$pSZT0_O7*OD09(sKSbo)08goOHuhcL_}AY-I`SM^j#q}WKv{-+}Bev&m>h~ zb{kFg0M&p_%zKKB=M<`VlRI+8X$$_Oc@fC7KD;*HYXsX!e=kTsW-qz!Y5sbUv+Zmu zH2#O#*s{>K0EHCeP>-PHuuITmzif`+6e2ZUm18{l zxY^d$v}u|zNIfIG&{j1H3vBy>ep`?|nR$y^J&Dz6hqbbO(hC0X z+oeAD+xa1^_og-TECLC+pHIUwwRn|L=T;9Q>a@t`diday%BkVKj0jPwNo;F)lk={wJAlm_LJrYEXJk#nF>sp zWXsx1w`Hq-<_wA18#$ok*4;uX)LBKdHF}1>8>MA$`p9l%&fTZXS&Xivy2kluLHEe0 zu~sep+G*S~o+>-;{oD+Sq2+vs6Phvu>|pz8ZE`xyfqaXbxNLt02Txn^meAd^Qp^UICw@jK&WPsNy|dYWBo@vuB%` zuf+IoGZUpdhoU^=Rlz&6`Bj1TMqyR-Rq96z72VP4-Yuu9iyFDuV;dz~^G>49?*q_F zU32f?A;tBRpFPVAcR%CW@(=o{8lo1vR^dSmV&_g`S9dD+os(Xqw_o_VS4mPu z)L3su;?rB4?EzOU$66IKQqO!#HglNKSO7~*>k7$MUaA@5jJ;L9RT`^$^rV3S9z1{R1! zz{x2Ns&`|lEi`MdV0)IY(oJs21B}?_2jpnkUoWU9I*x&QnLCKI3-J%+n)T2Y z9-%2n8&=NqR6ps2zSsixR!lAx8X5D$-b}Q34SB<%%RC50Wr>Rd3=BC=m5hOw9+z_~iP*di=XKe}tusm^W))q5n%EwmDQ3vFrchkea_@XqeJ&h&o;f-^Q&gf@r`u?(WY5-%CBqUEkV6+q2_1r@G!@i@5Z04=-SpC z13T{8SZS_q^lqvDgGFjrropCa3CmCqB@Cw$hOwsN2x_GK(e~PU9 z>j<%fNOc8I&+35F%A9JuG|*^aYtML*dUrlAwMv6^PT-XH6Y6eY7<*=i68XMtWtuv( z;EUToBQVN|`Zz>bmXO6}ZjmN-;Eefvt7E>)vdX#99& ztmUKw9AN;rwlT|6PTswwsil?M{jIQ`%jH0@__H{HN$CD1hQQUksbvUQaZPjMY zXWP4t9{|TX?sVILHo6s@P2{hjY7Qwma>S$6!hT002WMj6TTqSb3=BXz{XC+Jo1Zs3 zyFsY);pAdljb6`~?6=*NK*!Edx|;>{XJ2P@caH{tcB1c+ytCXKvoM<2u{qH~)orM; zyLh&ez6h%0P0IocG`%(Jxp zY52S_u$ostu==DY2pmW%)Ud#R08a0?yV-%=f=LBj+60-njI=(o23J+-_E>ONIifLTvp6jRNoDTACXMF^3s60ucS`Lp7mL@NXlM^jD_oi9I5tJV zeyC^g>Vh}yL=7Sa`fi?HGylRQ64<0p#Fiv}ktkjI)0PzUeYl@iH&YL~(-lD1y*M@5r4>I106y>Qm z+o>>)0d@{dNu82w$8Jv*)~2R( z0+kp{p01?2pLVTw43-8SN=@tg0OZ4666yF1-ZB!a(ZqUk#~+>r3tvkXePYBa)>?X( z3Eu01I@900xQE+F?%_7tY5H|_5&GdB!KEvNrfSEf2q8aG{;%lO!yl$jyxfH(1%o|e zGaHL_yW2M)PIGq6{p6T3a(&i=1B57?mafW?gfeIgR!}o3?UDgZ1lmu5LZ>#Exqgi~ zN;|b@deSY=co5*E2@DG0R<8t1fiA+=e| z*b29dDtLKdlwNkjtjatgmSS`R6LQD90Q&IOoY;bw&03f;YCv23_IB@XksTd4f}jlt z@jg{^g9ZuEFMdR-f*BkML~a~S9XCIk2%2s<&iv$*dK03@Fv8!wW%F(|v2mV#6q(YX z6q)izY3Mq?s2Y>Khq8P`O_-U<5En(n0^))&wD_m`#Lz~pcm1dbs8W=rxSa+ znS5eZSogaRd8mq&Zs8E0!0>@dv6&*<)|tW_dsvXQs#f4ZGp1nvQ7}*qWeq5RctaC$Ln@^ja4#Tjid$n84{r+LjlC(`hTI-Gl6ETqPpP+d$+rLeI(9JOS3;>0Y<|i z(Dv23J!HSrx<5M4eiqt#>v;jtCm5`nGuhqEJ9rRZ7P+gDL@pWC1KFCWdgh#J-t{e)7Y@Z z`^h`jOlNkO^9XpBN?u)5*pq}md>j8xO;SD>b;-696FB>BhSq?6J&e_L{QdsxOj32f zqs|$gEu9_na;mXz6N0uY1m91@{Ogz*T()-lERA+gz_9ZUsU7^0&zhP~2M5Eo3>$@$ zW?o{dIDxiWHnbqBv!071u1ifm=)Bb%XfkrTj8+W5$$sq2bgMs~!~-QuORxbo#Nn1Y zJ|oLGb-3PMW_O~cYF*78nF`;bj1v?jb>*yD5Up?`mmZX=Uc=^vK-FKx2&1dOX0{eI zhn0Fe=GA5@e7W5vS>7$wS6yyekKKGo5^)^*{-^CnX8x^ASHGNX6MfovHVtFOsz+f{ z95A~{4sQz3kj3tUva;l5!@Rsa=BCcgCn?XKJ^QM1zG`}F zSyKF5Si$zPjGs5Kp`pRTtnD?=$8yuFYSTpLpd-*IE(Tx(U+#?{$kK2=9BV!)&;NAm zWbfNg&kAPO^9J1pEtVXSPz8B=nKFkR?Cis)e1{elbYv|gV3<)uQrw4zn=B8PjZ*Dz zbKMs7skttkLz)eS&YT0c3=f<40*3|r8rHRpj2sY!2f~QgEbW4GME!Mu79n_e=x*SK z)Ed?>WQ!3i8fpQP^p+S7jGi?VcG-y&L#P)&)}%E7B-X%vA`tk=6A2he@6b_k5$s|{>1FOS$-NQCtSQ55xGEbwaW&2?Az4ZvtH z=YG^mbe&CATWep&9oMNb65&}IQl7wkfY@P8$<^!W{87S!uACpTBYRs6a0>ubcGE;( zIlZLgJeSq^E*iQ;GX{Abou{(> z%E#iRNg_M7=Z&jTO-NM(rITFaEf>JS4lz243+W@qKDxopLaDy+t-}+K#&9$wa$Hme zQ9kif+4JHMmu=I8CU7}kj+wj<*s1LyWsSZn4#ElD@1h+QeGfX zEwpQBU$M=bY6>_)p2SB^+0R{S;D0|gi0wKnrBI)eYYdidCK9OW=f$*vPVXEk9T)ys zT}am2djH*~ato{)J{zljd^R^^UCsBce)0I>PSeH@A+}5+=b%vFsLF92h7MOE?ZmG_ zvzM>9yvO;{w>}lkG@P*rWv8SlU_2d|+SQyQw*GUo~2>H8~rUFyV5;{lCuwXmwYB@@*c(U53F|bSwWP z9a(^nh-aE``)pCRs5fC%ymMu*L&;_2b28ubiFa_gCmsLEJTJF^3GLjmoPPFxWgCu*O~H zy4IO3>c8@HG_UNyDrg)g3Y_lQlX(v}mpdF8H}IL8)_!6d_mZnLw_S1@^#)I2gpeI3Hd!d-J79ri`8x`WV$0uo5jn@2R7>F4_29G`t{s`K2@059@_SBEeEI zn_*?pm+J3Gv2hyO&lej+wJwqX+n~iCvoNNdgt56})A%*xutj77*&wtT$oZ_OxqX}C zJ$W9*CH(5K7s@0r$ka1M4@T}lTTXfF{Z8yG!qCvrR@#v%q62D>7Xyw|?_0qZ!AbXY zxkF&Eia{-l8YHykM)mdROOI^zas8+?=ThUd?bq{Lyi%9x)IPxqZ7)UAt`sZ3+T;#x zg_X4g42;_f&t;J9Y$31CMjC77?d;F7udCAflkVM9XxI=SCP?;}9T~E`@ejStLN^DB%j9=@G= zTjl&w@KU$pkSwl$o!!v&qwVb17rBdeB@-Evxg3#!a1tL+`bY1OoJ>;s zMV}-OfC_J9Z9^c;vnx6sATckbyzinjL8n1?O#nvCw_e-h^h032)He2lt;lB0p+)co zu`5I$i&gc&zxd(5|lvrup$=U_wjoHl!72iWuLq8_%5X#s0O5uBee zJ6M?5&P#xuA+>sA+g+4u9DThX4PO8X!h1NIQ&eecWKF!aP?K|eTy(g~9;^>=3aAr< zPIQBxi`^c?%8SiC1U9#29G{-fQml^3QzVPu9f`*VSOM567j;}|saL9F%$E7^kFZ&G z=PE&)#(9p&%yr9LSpRiDJrJ1C7A)-{$gY{K?J;YN)Fqoju&Ut3g;j86{iOT*e~C-(=onXPLwR8#MSLH8?UFB;pG> ziF;#g&&3rQ%E@{UWMj(IWI9w_^7D{520@3n_Wbhl2+Q8lHODZ4=u*#gmB9;O8N8=g zvotgH?yzcnE#pdm6ke)kD7)2a7Bu+uj8PcYy;dJMoAA}waV#hU=e5ERy3uQAmKNCmqiCH{S~Qpps>D&N(L6=XZq(-=ch-(N zog-*rwzM3}3++per{moAIu7GMDg0;*qHE$9hpMI59C*%%&XoIksQKKQSHVb#xvu`E4dSuSsYgz% z*&M6Me*2t#hF9|h^ugq9PtfF9g8rE`=}nHnEiFb%vfSJmx{bEX#W>_5z0?HbtpeTB ziK%)SOxJ6v9G&KY88P>k`&YRB;XsE)**bYDEmLGJV56;pl-**dE!9upT>2zL|?}?YJC6eCYPN?p`PH zwJ5(Bu66BDd%oc3MY%$KLf4o5MzM+fp-)e9d#656BUH1$_}&)kvl1^Kph)Ek5Z;}K zKQQBNfF)1&Xj3b&FzDReXaesx!_YO02kJ9|g!$XT_jJv`b9Bkw734C*%l5TF<@iaI zwdhCmA9?ki1(ak~Mift-ox(%OX%;F(YK1shNd3s5j}GG)5Mx7b7c5PlwC86U>cLe`B2!O}D2)vtZLFeeK-t0pVDwV$)L$4iwR#G{?@ zK}Yk_JoU8qcUMWY;W8u%rUk?b@?s2vMqGacsBGJCSFJ)q_!K^_sos#5mbrQmh2qFQ zVdSnrx|m^OboBY5plDuFG&ao@igq9MBKB$~H2KOKix8zS4po z-f|INln3AR%W`u$v&<3JtQ~Gw4u3Rh_OsGlX#g&WU3m!S>|T9K(%1Aj16!Od64hjl zvdT!0Ux60y^RJ?Hf!!N9M1~U`0p#Y*2AuYy zL1HGGR)*#c{IXg7?j>>iw_Bb@{=-r)3E<&VU8t?w35@Ro?x;5D;u*<-$H?IN`dMyOIU6Y(PZW>k1MDC%p0f1WwX zhm!>|;6zX=Ct7W+8W%k79vRS>p+dlhHdk1?+R%K5{;XN6diwsu<6Cz&(i^nO!ofB) z{tYf3@0WiLXJ^ zN|v_Il0?}!A7F`hyzh?lEtla7RcMd!o54d^he?sBf4(4!d3jK3n^xUXMpjrh?z|k6 z$6@u1>Pej**B9A3iHu0moM(lGo-%vA)iN2(ZsUq^w#COFHa}HGF6*Pos!%NV)JyNn z(Q0&_LA`=NE9PcSxn>^no1WT7$TwE;$?x$D$z`vghT@GjB2Us;X>EhN69@WqYX}D% zz_onxHm{b&R4=CCXURNj_B)ZgqI0I4273! zn={&R4cz{e=(zhy0^T5Yls`VG%XXxKJL3oPF)E4x!n+rir#mESxxuM zJRi>w!N|e^5}!fW{kJwGHtVQ^X+RPp>#uK7uUs$VebK^md^Ca8yyMwcdA|}nH8wfiCmdFY`-EgYQ0*mB*o=j6G!)72X9E}7HavN)`gJ@pgQ97` zS8PC?#cc4Y#?dEX7LoEpqh6R>_yu{3*Upn^FxGJR)!Cl>#o=nTZ=7Ne)AP@Dy4pjX z7@=+AVQ>gIQ8*-L89|S1d+=0u_BuB=%h_9fi zuRy_dicG&Q)P)ZX8E6lc&^**Y>G9E0czsrW?Cx_`UZnf)e=QSJrA;udgnBw3dZk(%Q%=>v^st1{v8PAhY@xLeiqF#||GF1F6qxyUi4P3E z35=s%Nhz62t`G~ZQ$$v4qxQkba8lHJ)z+9s=^-_mejDNK%az^H82o%IKZg+S zkoi~kvQI_n?zw;ZexlTe5|H`9&-EWuO2p7_;O>IvS0T*zQ^|ZwdPpFLu}LG7&peSm z47@)&cE-4U5QDg$?bkcnsev_ZH#f>uiaU5F;%9hS?{i#Pr3FY|iUg3pbmLwAX>RPj z^kETWubNPL>V~2sjQ~mE|0St_{f*+sBL_N9sRD@=fW>{4!d>=#?W zgCu+eRAZ^3G>H}LHiFA&PHHSG3cpmoLk?Kn4mpvg{zL$7xG6E9zbTE2GySEyuZREv zh1ELeTYY;7=Zh32FHTge(^tplB+9l!h+qVGxcm3p%gEfnoY2PEnv2+9y63#*yH#|Q z(O;3A11C;L=h43yRLQ0}5QHrW5xFDyqF}T3< zULxf85+T1~0HYY+OJt!XF#|+RxjaisQcsOSfQ;aSg+s-K;%7i-ekUa9a)iyV_*a?? z>?jZvz+q4#M}s3)M~mFQO%B4^1CF=_z9<2xp1Y^dab6Xb2X34-hsTwgol{Imw7t8- zN(`k2>94G*hw`WGz1blAKV~-n7kh6N)d<^k3%)HhO=FF_CAc=;xCM82Xx!bsu_idd z-QC^YwQ+ZMcRzW*|C`G>YtG%QS#v`!NV1YjJ-c>Q?WgJ=5`ceZff3VP1=?r!ciR{x z=+`5Lo_nY>eDclS$@(63nH%->T=#JQ;!M{p4f?+T*pE!AL85#TG=f;rUrVUrJa=yo z>dtHfB4<#8e)z9J)8qf&Z_t0;3VL~DvQQovJ2*7-$P-0O$mL8MwJ5IhA2a+v+cOFU zm)~E%qNxAgkeK^Fcp1N)Ff?#)EnQDp|K}9^=bS^M!~e~G(Eo1W%=`!CD;40XAJ2xC zpE890Sl9azW^<|D52S2MBj!kE^PjikhGJ2riRVta4Kx;9Ay&wR)4k#s^HAJkLIccJ z0Q5lg6*c6Qb%CL9)U`mPQ1AjE-sbH1D&a0bT>D=_aUk^nJKFyjPVPGYy@QC#7w?B# zSgeU;?R5l3c}DvA9nin1{6V0T^P_KiNm&9dbi86={c$v^wU!R{_TO=EiU<;D)lrRxV+9VU@;<5O?zT44tE;PP zc$QPVt2)sr7sskJfWco!9$~1r>MZ72M(rIO>S$*xx`Tai*!wFg{cj4+!45uG52^;z zPtz_4p`60r`$yU39kC2w12{iEdVQI_pZWvDt2A6kjkK*TnMJzqU&TXTmwZ9YZWqeV z&cFySpw|Z293lrdF|&gX1nf7JcElybm81WmtU#O)nZfAFD;q?06EnY2>p5 zB;#Cxp`XyqTf0=U1YyE|IRv9`#7HL4o89Uwh*=dc&XcL)kzn+xI6f zRalpjTxUQ21LBQy=E2W9JJK7AfTYMvHFEaz0z(2;WisO!!VM{YjbVm7zesmux=fxbGCM zmK@NKcHHj^r@r>&o5+SOPAc$Kwtl3I~CPy5fS2Cq_U4a-zD~Sy&mnW8NqUP z$l|l9CIFk}{+tZ+Mqd)`jAUY6lKZ^GCGbgS>z=QJ zdDQPoN8$V(`ghBw>gLAr)FZfKJvJ(_z=lJn=T!cq>C4*oIjZcsn(-;E%)A%Dd>Q#;2`150~1J=2+A6FgV6r@N#b$76dY2-D=0$ zoQjd3xNcMP6aCEF4CoO3AY86+jGgr>IwynU*D7lmH!Ob%841d0c%Km3- zf9LUqGk%*bX@@egC6S^A;_sthAFdFtTu^-&E?r{2T`F(aL9Mn_ebv^xclPKZ&d}Ru zd zuy64OR^sJY^Rj-7AKSw$w()n0|K%)9|Kow5c@i^*5_O%s2v^1wV2axxyXfScqCEc- zb1BRJ%@DLa+J7z4Z5RErMXc8s2X~N3E?)BM7GT5lUx7uwn1=WRvK~q%O+?>77Fw) zLcjN+gz&Y8KC{R(EiB1%<>i_e!TXW*MTFs&hLixkGz;CPaH7A@&rUXbY%KihVR%Tk z&8v@&&q!S3za&N#N}HYgg#KOy{SGU5*~FqQD!XB}@pF31tjQ4H`q}g7`dgp@je`?qtf-yhmPwa+4#-l=0twdr2d4DJ*Ly#C%d_?yz#qb;sr+mx##nk$BH?~4j`D!PS8-8czB*b$nf zVTwpP@uE+JhUdK@Z`?U%eh=A))USun*N2Bq-{w5l_Ktp)HOxqU|BCgwmB*h}Yxe1M z4(a-qwze-G_z{QPCOM2sS9-GgM%h);i}?qR>i2w!>+97XL0zbhc(T3bqu$yqfdg4% zb}j$vIo7^|&Gr(JB~toGF9Q~RmRTIOQt;JkZMl3>&|hu-;il_C`EuGVs`cod6mC8e z9xm9p5m2)sL||TjrMwA5*&Q>$X8CD;oQHe87{|IuYx1>v_xjwMClvWUFS{pvcCt(* z+;iK!X~R=w#rX+AL-|(QmLa20L*noDyV#!aOa*$l#^mlMuYxLpw9v7+vL~Jei)j2p z3s%Q`8T?GSRS}^I@Dkb;A#F%F|Q>KYt0Gh6yvs6juR$BZl|+VQyU&5c47lv0>; zjpvyZ_86#nzd2x=4-(&r2M^jz`bx@@8Xh;-UFb~xv$9espYBQFQ~kMy?-f62J^8eu ziRxa;t80v>EcIW1fkMy@@(11h4IYoXU)%lRVv>?0x?Lo(%_8Br2Qx)zxNMf#f_3!C z#ZR2+A^GtN=H4h5GvlpA{d;|}kvZ~oqGriak95!OATBXxdjN{WF+ zh_t%7hl@K@DOlH{%=~c{yQ*d))l@Pxyui5ld~HcQqEVL!HkvFRg|xAV+@!v zRZ|%vz~P5vK4`l_rps$~T8m(XHDL+v-#SZ!NtC$QZJW|}(p*6ZE6~I$BQNdh9yO)T zL}5yC;~a(j*%L$SDz;#f$9w;}IeKj3+GAAWLe+LiV(sZ?;B9yo12cL=TewAum0BYo z#(6=mJ}io+iBo|3Y6!-})sxywoiG|paG4=1muPDa=-|bzN2T<2H+EY?ahRe>5s&2# z$QLJ`v5R-j>t+A>##}xGFH2_5($xh zYzmqUCYY=@R_+*n;5=1zeA)7epnX4&&}7`3OCi`dhD{JN-LO>AQJCT3-Bu^h&VAnX z%c%nCFfvQQ8W2!Z5vz4HVjgGYDYx!;m@RI<36BoFKr zonG3UxUI>%UHc$t*p){#Cj>KS>{AOD=n|Ie5#YmV`Ky-m-#KimNRPMqoV`DZl?Cz!zdWkhfaw_ zIu-ej$<94kPx6wmXa@6Z-Agc>GuFy57%TRH3zTZfu90#u=xpKA^@J8rWv#D)e(W1MOhur1G)VIbb9+OzAA=<+-Bjs_Vr z4^KN4*moT^jIDNFFC}T=CWDHZ$XI9E?H_2wj>Cn7rb0B=ThPjBQuxLus6eqY`R`}G z!GjW@XsdiS?j@FrkI^2Uua7+z9evNB^rDGO1(*{OzfsM?louEoxegEbGI0aDX)l?1 z!nG~UrjCfOhoJ8J#brE;?IFeFhC9Mu0~WSxJl7LK=;~EvfXZr5aA?9s0^)MjOX2<{ zkpOOM+wmIMNHoGC;Hrxxcn?Gh7Xgw=4gjm_!YsBpGfeHiEOq)ec3y&Bp@ zQzPCO!RK~4)57O+E>+vmpAmaoz-81An51hm7VS*9B+U$@|7iR=x)f#i9`Wg2g<28r znQ&P(GsQe?1L7dsX}HsL;-I0PE>+8<-h=wqnxv%GC6Q#_6De(RMXsgyxx|P%{3|(0 z5V3av4fLoouIj9D$0gejZhGh z*|~MSRCli$B}|RU#40I`55sIZy4tkc*-yKqaAS6RPlhTCxEiJ;PlIDPm4LXFk<0rP z^CVh1V_~(Sb$9GMnh>V&BNW>C(f+!)r_-$0XTB&A0*l{Ok3)QO#ld_iNdvf{Rff@V zV>RV`-V!xBjgh6;_NJOS#rfQZfnlBoF5mjC){63&pO~A~fl?$$zu|;!HfOc@wy)w! zu;n`9MFRTi)bh!SWN}jl3e!1ieb)SHfgOM?%kALlhlfsigqEgsW5H?7wWnvk)FFOJ zfb6ZF%5psRrtmZq$L$nj-u}u%Vy=kqQK$LdHaqQt3*j)AH<` zF*vBBYu#CFz}6jmZCKW!`#z(Z(Y9%q#LmakdgcvbeM_{3SNdhC?tkqmLywZrGx1wY zN46`h^)|NugwR5_Ii>TK#f8l-;;Aw%JKOF8dC z^KZzcAQ_+BFxboJ86~>gL@!#A5jQ=fgO|K7lj)ZNRrku?c;%wtNP$Q#Vy50t4G0C` z6&6|Hr9j5s&2lkwn1CsSJWri*-rou9ixG6n$NHCEaF3(3HdixFk|+t${>onNIbN*i z3>&Q+4=!W|_vVgko?mj79eXDpir!r-8^rD-E~|vR)r7?i{FA zt890B9_*ELEzOt^9n#HU8HT({G#JCI=EmI>i>A4x2M$zn;9&^bF!PgQJEqcUDEUN< z5$hDnXUEuUm|oicri5wLQz%xpK4RkFm@R50cR1O}x*W|=jS=z0Kt#H}m5Ur+d5lRk zO=Y;rBkjJ8E+9eT@$-w94_o4Wvrs8{EbippPx-l={!e=;uiMKLk(l{GW>z6N} zH$btHH)On@I!|zFuU4Zo?8l~#>VAw#b!lK&GSc*{AFB*2&jG(_a>>JqVZ_gV^I@bd zbu`{y(tNZ$t3EX;8ALZ~77=V;tA}}8$<5MZVxwB}*(EQnwSbcWFvpcn?$L2qK3UN6 z9;n+UOZ!BadBc-OCn?K%1LxoDQGK-Pb!e*Dnj8#>yw9>&%*=H}JwBWD(rL-!h82mo z(OK9Ox7MdVg~yi86)K+I@3J*)+Svp5sT@gYBJsfgI3bVPg|(bw4j8#GDaeBCNp%Os ze1ZotiQea;-M$yt1cm|9f;#$Hc?C1lg?j7)LnCV4<{&{RcTeb5JcwjwEBP^wB-s(m0q>#Ijf0UDU{6VZV{M2m$tVw zE}~J$Ak@q~uce9`UJ3efK(k(oYpt*|!|fbcu8rGQL`IWU57Q%vIJbYo(oOe%geB_5 z*_7y|cZHtXv@O0$`#fyA5eWg@Zs+F@HQwcS=B(0q)zDmb;TeHs~wTtbiqUtfW*H*n^ z8-vWqwm7=iB&q4xHK9lLLb!fn`^9m|D}5a6?+Jqn->q{*T_va+HFsiYjZ_j zhCJmjG7lyPTTW(GmWm$OnLbqf`8U%x7T%gslp%7bMuMkGE}0-DvTHxnO5CBi{Vlw}<_Rw!}uK231sr-5dyn1{_9;$j#>T zXw^}k&kJ01*$je>*tizz`0Yif0XKctiXqF|vB+|ytT0m~N-iv`Qysq!1Qjzi9a&IP z0;21l7i#HXd2ojE)t4cWVm^Kypj4DWcu z4t^kbOyFf$dVgX4Nz;6XuDv{Eh(9-W3dDuDZQdS+$n)87H>0HSRX*Ne&)TEw9KM9e zy6v%7>)=ff_9jtu({&hmNLGkp0|g_8Son9^Z&&fleF5o7 z9h8T%+vy2!Ed71c%Ch4H1&&#^iFb`iZ@z@9qp z|4m!y&EIjzAlg4kMPHy^v?9Ez@EkzOX3y%c78{UK&H2kP zS$nl+%cX_*f=0rxo_HxHgFC#xuHnUAtmpEn0Sg%FozZ@4{IVUPNg#d~12^Ymx$Zv? z?p4G@n1$9dHRzry_O7e=lEl0^9Ppe*E}wO;>Ri^Z!fpTJM`#ivn?vHa-(cNw`Z-eK z!nHe$OnVRKrcDsG8+AQX(bK=M7neZt+8?rRKp^92XK=zL`wXQj z<}2Pzb!_T@)HuVu#hyKffcTRS<)O|D<@qY=Y6VEYCRm$1>l&>%N@V9#_|ebm3zjzm z$ieSAa^01BJat1{t~36#rMhHt!{CZ`!J_>PQF?Vu z;$YO@QVsF$8y~cem$BymAWT-%37`Fu_?scC!dXJHe<1HAA0Od{MClCLQ8ga_IA0(z35-BC zU792sWq*UvHu}5p02f?RV#U5>JIosyCYj5rOZ|m$@(FIAx_XKVu^dpWr3{LzVs29X zyG>eDmjR6Cg3mkDaT~6vM{e6jKXWoICN3?c z!nUssQ-?R>91V{3=})5VRRRv7g(OH^{*~vped3JO=_oq#1&?yt{X^Ap`I*pHV%9uM zhIJjzDGw!SrH<>1lBDRc9vXsE*|-FsUyKR;)=HjMu?0G{a#`R}+-;+RVSiL^oJ`e_ zAJeWpT{Cs@yyKi~T#8Z$*`8B*$jqk>IdAm7tb!L-VAIuuS`* zzfMjGyg_p=>Ur69RZ#-ig!0%rqS!}A2MXy0mw_X(CgkC9l)XyjUvr82%V!*wEGANi z>%*xc6SbSd!VI`3-FD?qMBH*$eAtVp)xgvyM3Gu3Hm6hlGs3vfED==N{gdeU91a+D zDXbz0k)m*ZzhV9jE!Gi(mZSs*BJv2!RHKsV>4#-(LAD<{@J|QoU3VBpoh88Ta zPB_R>fK1N%x{cP8j5TtEWgX4&!`mz{w^~@HO-&~qzT=9T=#O<9U3?#d_e!i80P0~TJGVblE7L4QZ*ar_0H~AbR9wvAqb_+Q z%_2g*`h5L21c5w=M12%B?l~U^g1Pl2K*owm6p^{|O&nlSxaXx=ia5 z3)$q)jS9xOxG`RqS0z7QRgM^y_xi)c6v&^4jcpK4*eMssnWVM|0p4ZWh? zW^y*&X`z`&&@6M!{_sUQheNS+ zYSk%o9USP8x%@E;6;b_V6~Z}^getvB`%#c`>v~7o=F<62`j}eveHfkW626sA_?4xQ z+fh60l~V6(cOG!z;#OA8i8FN0yq=A8Z4MvKX_b|q9%!_WSt>CEI#Sl?5RDb-mUf!l zE${#1qfu5O)aIgHPR ztW~w}QEe6dxsNf=bp9-DLpe-o3&mR|+CoHE9Ol5eh z{1~x${FxiZdKXv?>W=;%b09;TUEk`CT4YWIj~g^CjGEyi7~w{*_2eO5T2wVQ$b@(= zV}|*3>J!9YdKyFp8Ya!(M4*IoiJ&9BFD;`~kjmZo9_7*H)8j`a0Ter@q5CaZ5KA?V zv{$0E^=$f+JM6>TxWC7djoCn&_#xPfzTiZzNxK?JAirQWF3DtSagFhrS5#j`$x6 zJfw86m@MgzwY>(eE?Ix8qF1Yd6VqKVc(myg9~oaj3b2W(NsPx1wP=5^R4F_~47Ypl zm}z@|wvgMYDzc6>n2-rXN~a%BE1H1TWQ|A_xq71znq#63(V=~>y!HKHYQ2uk-DF2) zeomGh)?Li|%1RWB<1v?goj?5rZ&!Z7-G_l?Kd@`_ma-((yOM}cGulxMqD+a)qL9M zH2uyVRV2&sfpYz}$GboKza5z(UAq5kPkC z<@;vs75sJTH-X;t@46{^)_nKb)(=88(-ZHbtYuD`K}aBneU~(W(A{X_4N&J%OQ@n6 z5TLnniS?9|ADY^2bnRN!AU=H1$1~vzcJ2 zEp*2X)p<{)Mb_$fDNkz3&tfJ4T=Q?)cO_izh$TJ($&qk_SxpF3PhhKDt^W$%m<}`8 zGL0?8NY$iU`MJ!G2?Jliuv}ri-T*uk1o56ktipH76=8c4-HdspG#`C&c>Bdu>~-U) z6cIzVc4@*$#`m?YFbh6+$9@X2X8-je&@X#V&g6PW<|XAS;lUS19Bt9J6d5h)zVyXQ zRQK@gX3|!$Q-w2<=9pxT_t}j%g(YXbI5V@ed)Rr(2X!jEi`{H3#X5vt<*H2;)6G-O(SrxqhgHk%yrS{%9)M1g?>85x-`J{)Q|U5L~Mw`cStE}8lVKk_@&`W zt6GaNkGmDFOoec~MJHcVvUxqUyiSWZT|2Z|>#v#T%bf-(P94qOmsC7cseib>(UjWb z!z>}CdtQlKt~Sq}Qg4KVcRou=kx@`g6PD~F9&smOb;IPCRIYD%t2M8)2`;jjOU33v z_6kq`aEz0<9aEihe=1|NSQC3NcWggGSaKQUP1yGkHv=>0lMSI8%YS~;G3dxV6BO%s zy~9(ZnR<2Cn}diieZ!!}POf1Xbw_tn{}+VQ_Rctd%cs->Bfv?kpos(jbg%(hX({&& zRvK3wkIDF^^A+PDiIo}=R)>e@m@c3i4Mj;8_q!67Lj|$fn6anZ&S#TzMKE!;6X&Cg zrZ0JOAwBuQn&+ZGia2YYa7oojG$v5-y%cCty_BETuM$!KIe$uU;5P$vXC{_0AV zQqaH#XaIGN79YCTA(YUBtKk1dh#rFNCLSX=KtiuBUzb7qK!1cW-**TRnU;)5RU$UF(yZ9kpm4NekIxZ@xw~C18?9+m?E{!Gz?aM5hpVW*@g>KxDCyIk2^C|Y!NBf_2 z#Abi(^|etIt(%c9Sz?`uc*5augGETs=Xu;iylD}4J zD|CMY!8`T^Gesc};Lt>;2^DKk40BPRYnssJ`E3*g)cfcT1}MBdHugPT=sZiK24jkzI>j*Zig*UUub!w*|>P6H`U3 zprd+9)Y#!U=C6K$DkOY>Tvmu~*-9oWa3pAMyN`orhRXcL6JK^knn;;`lW8u}2j#CC z|M&pK_F1xZ(k@~5!~T9_t|K1`V2S``-T_*cq%$Ia^gfYjCps+E`$$MIk;bO@SN{>Z zvj+3YS>yywt?9)3>M${!LDaExhDUl(bVrLNaCN4*{b7L|`<5T?rS%T2bg?JcmwdG) z$N5wSn?3VOzl$vyz-83?d(PUvAIBORc9h%)P2Tj7U5{ExSI0RC(pAFXS+wUZ7=7`U zjt4k$$9gz~(`VDqf7+}=A9`D8?0kUk=kgEE60%|H-k3s^rv?qn<%{<46&)h4qf@uE zM&~jgHO$@SFl&ovkbrFYQF(uW2l(?aIW8GMzPWZQ>i9>#<&p6=q!R~98k!O@_KMM? zP7UHxCpGjp66N)ar&9gC5PB!M6o{}96Lj+9(MJ0OYmY)*!4L1TBjQ)H2ye=}0yhi6 z?TAS&E1m0-+_J{6+J9}srqpJO$dae+gX@4T_%&sR_{aNbDRRa`c6253xdIQKEh2-Q zmq3v+E^AH2<@s@ZlF-tAd=9U4+D&|6Kv_lIFA#`OM^dn(pl&(mXl9`CGXP=yq-WGA zh%s$$*j(Crr1ONKZu675e?1~C$k;*4qGn4n{lypC8*y~EQ5$Qt|NM&B&zBb}rLI+%ohI37QP3uJR0 zZcO`GBd?|X6C{+JZ0}155d3&wyp6grTxu@5h#UKyes#A{b}V}0|A0YVHP!NgX4{rE09cm{G~7CjIQR^{9P@#*__M| z8$Oy{RmE!wvZ{U%hN%@lrZR>@xWob*EF~USwx7Ojw7RRRO(rg&e`p1 zH$%kdod#qdr9thj*A!9<)gyIV&CF~m&&~r}zl@`usUaRxl+SnzrPR-EHz4P?P!^Mc zsJ+sL&OM*``?G-ls<!sxvoux}OvwmDS;oV?s_f5_zrZhWM_*lTH8dR9=`E^u? zmCUHIqs~-6`R~D(lwhOv0{pXbJD%dWnzBeq?<+V7L_(?OLC7FUnV8pvr>L`|x1!eL ze21;ogRK{Oto14`ne%D)EF79`oK({jswGjPNb|O%kzh+gsBg~inW$DNrt=^|_qRbwT_1&?~R|!f}P`-^&NhFea$|8NV z|03+g71uJO?-EO!v}oAgUB=QMoeYAR!Mdtj8~#&+0Y=WCL0kg6ggO%#YAXO!C?;qz zH$2}@XnFK(Tu%0B)M^<;C}rWo$@?Usx45_6RCl`nfqG1G7g;`ow@FA90jvoB+tKrd z(8`&q3QIe5u#(LUJXo5ho4tgYu9o^aOGP$F$w=I?)#o2nrTAhL@(vlmlyvjWlnytK z54*8^b5-~`Y4Bu7;yXD?repd_@Mt4jq3O818FoDR;Y>TYN9QKTQPOO(`I&^XA?6oO z@tSk=xVp936w9e(a{k?k7-xMLIe}&%%SSHFmnib=M`=JZRVY2QHFW3&cQ^VwM%PCi z{zreP-JxWomLrs@y*3_xC|DmWMUNiUHqpGH7e9$CH*!(K)3xhZd~r3;i>IK2hB@J3 z$8V5+l^;MsELCu4n#ncFp_Z$Ljp{Wso~1%xY+U~AVkD_c;O`lC(3_g*zbSyjjSPtV z*XRy!p~TDDE}HIUAA9_VRT2Fqp4-}Q1loMCHL5>Z(KHd<<7-LW0F7Zbx;`iw1+$%l zSn2#c-m&yV-Ny`nv*>V53g@%&-0#~(M=~mOc&p$?;>!kHf*h;NXM`GL+Uz2|BKBHc z%5?wS(zH7am58)q?pRc^C)*lXCF#9Z8>wxUbC|UpKhS`3KYi=@1PvDP0(X%I;>qDA zdh?D-4%piBEQAES1ZkjLX`q5AsNZ=Y^}c^Hs7=ChUf=Qx|IzC%hRSZ@w9yJygeXl?TDT=2#8(LYGV+Be5T_bR@XtJza{5-zJ^2AMG^(Hg6f> zAqSJowuUy{vuD?uB~Ilt#J4mA)*#mWni)4H&5X-0@LoZ-6YVGQ(DE}brav~MuGT-w zY_{RfH82Pt>#lZF!x+syo_+0pU|m)-K#i&+QSWXq^qcHY#xYY$bW$>&SgB*pPPfo>B?#|=X+`E z92d>sA+@Q9^I7O-*vcfTNPQgG;TATot+fse=@j=g4Z6BG>o=!IsHK|O!M9OuLSVRAm}?I5BFhEV>T(Vm%2pQYXunUeO^5K|-D#ci&P1wssJLc~ zDRh{~vsN;PiU)8}%Hr&3mMsHvX8}RP`?y1NHJoCzwIqbRSfTN}$Qzlzoqj`J;!-(DTqBvM|;deY4r;Fv!oL#4w&xigmf?U~q9DpCjcs zFZdhmNGmd|A|TNq0v@iHW!-7`VYK~4Iz*9yF-ag-i8Va7x`7#hD=Km`fhn>SCut@| zLWCVx%u2lO9f0>3GWN8e}Kkxs#rR5vu2R5y8K8^XfM zXCN&N_(BAzVFsNb`k1gsQ;j>2;<)wFS0y2LaqD6$nbS*w)3(_zu`q*$7S@9?n-OR9UyE5s z6TwOvad+NxOQ%9DUqF@(Xpr*Hm1Wvjs^2UB{GN5e+TV)(4b{=jJ*k{ITllJ`WgGdmtcK?9OD2nbb0%i`4%cTGLgv9O;SaQz}~JFr;;K%{=H@ zOixOxXP-e%nU+$Ll+bBp!l_M5%EN^tO`Y&3WUn-oHy$AEFm&N&_Eaz^=at$zAPCy) z1NrnNKvzX2gabuoqDggW&F6{R1}v`z#Ee0AKF=uqtuGvA;?e4}_fOv+O=Lj44YmaC zGjna99doybo4iadBK^*mIGf{Z&Vh~ucgc$GeNQ;s{phGE=1)0m z80<~p>*1H^<)WJcJYr0lv2SCRi10!Qh zfR8Bkrpe#L+P8G_vJUkdz+y&waNRL>__opj^~Z z<#~*zLlO5So0d1qG70-MJ97|wf{WM@CqP!Q+S1|~=?8lDEK>`pcf*tXi<6ZtU7wqA zT`Qqo#O>Z_{pwIStd*QD39Bqpo)|c3Zb6;Y7GJr)Hz01-o>O>XZiY0hgoV#4aALj3 znVmVJ4FQ$bJ4P#EqRe&b*J}UX1bi;OD54ayt5+3tD^kLil13uH2@XPO+l?%tq1#Z^}J#rGXjfFyfVNz)X|0MA4r8)0)EuhR3a8|){Pp*OOi z*K<5i_k%`*4mN`KTn@q^dcMIMbNa{MFRu|?wU&}G!iA{sK;E#?Djly&VUQGs|8s3e zHUy$szwlL(Gt$r^&+v@XWtr`Tkmy`8uG4`(Gk7p)pd&yaP3)LeNxG(*S#$2siS%8} zd8rJ60kVozb8%z)n?_n#87kydlsoA5)ErX3tup28Ig-oivQ^OVXVW;ko5+*`V3TK4 zr}Xrz;`z!%BRTLrHPF}4`)rx4V&kvyv(#0UF46~p0z^F>PY^XGl5(thO*RJco=%n)ZQ;&lUlpBB?$(=y#QHz<3#TilHEt{6>rPZWI(IL`_ zt8GW-KTpk?gme8cPihE0;ivG5AnQ&_ z^3bC38;#6*!?_>KqyR6|`I(f6C9qfn<6+g2z2bLUp?~RXj2ju?_;z3iLU&L!*jLUm zZa~5u3Le;wf%PxTY(|5$4RIe4I0El)Tz1gxQA9=B-VA<3dvYL}-d1T}j z&W_Z#y%<2Fb|X?P@~64qo(Fne2m|^4U-=c#Y7eFKjT`6OrxyQtoU?ohsns0Hu{WDl zv-U|=#YMxj9f5%H@^rs;23paRC=RtdHN^%&j6-pFc>o7jOzYpH0?rMXCu^bxTjr+? z=31N^rX*c$aK6Qx-(|-9#EXiEO{$n%xq!`7rDuPZ-eJ5`Vk+*fgE!lpCCCupIU^;c zu`niax|mHRT(WEHgOjJA6bht{danDe<0qMme^fCI^st>2}jqEciD4% zD$-82t<-K*r^xABib&q5FgQ(40MjPgOX23Va3@NQHpeNt2;cFBDIIl{9GR-5;)cCsmJ&E&6HW%ueuTA=S}hjK>8FjrJ=V z4*%>gh%Y`TlR>uxyTL3CSS2(G3ule^#Q~o#V z)lVs1E8+oj_Z@lD?+7mmQZ;Xa?U-c&T!W()%gOWGbH~)%cgOnj717#4W9bi}JsrO~ zhb!>a85f)1eRUpox5#RrN0qU23WLSif<~Y%KuUbVW~g!#*J+}-qUuO3IR{aSHUnUv z_{!8?>BOGcT1uczDCtq_GWqRxUexdo_=&*i(S4(wj4S_f?1Ar!?}+O2l<3a=&bJH9 zmPj%BYi_+K#iu6jK2>_fFX0IN*&zpJFA3I*sAS#`ALa=?zpL(zPDX=TFFpcv-1;`r z6duD`MV4Yzc^`h?JpsR8E?IzG@6?%szr#qUM0<%peRoCr%7XIPkYJLvZN(1DfKoK? zdLvvTFAHJ=IGquR@o>mNb!(PMV0;nK4pA!#q%0WhxeyK}Jsr&L*yi!*IU{;=@my&C z;=T4lZ~b`Bo}PmD5ws&7ZColn6=vz|U`#NTJY-jGAWv$BgJFjeXKfx*E^WTBtAYUnW)B{xAg+k5!o^4+nX^r$(+ zq~%EKYteCptFuoqb^4toC{;RV0t{}A@znOrqI+9lIbKQ0juOUWrUpS`f1#)K*R@Vt z<0`E(zAKZaO@5rrvMa8uq2f(%nLA5~#tc->o($9}wgcgh>bl6qI6J9l8fAzKuc2F# zOxaH+h02T;t%(jom&dX0g!+r|1_tN+@kW7=&Nijiny!p3FEA@x)h4YMoj|}!yn>B7 zHN&ShZ0)$Y$AkI>MXK2vgBbH74mW#6J#{zJezTbHCxne_`jQ8*VW$VBUPzY7X z^%-U352;lUP-yJ)-M<9^&DS!F7X%?scD%bi(^;W#h95ykt@z@5pX&uQ^cDJ+NfA*0EWf z;EhXH!v3tw2rvLx2HKwCs9VR|N%8dOhVQ@kpOLVZhCMZHcHnLuJmz>?mFA6_1(L|0 z8{nn<$c%boeB|V1g;|8&^y&$kFm(pvbGVSo5+oXaRUv{%)y6UCwu`+Y;>-%5K%`Qu z+05iW*KmsYxSt^&%8ox^uOj|oG*YT~(Sc&2*O&uVlAhI=u^JVvcq^8+#UAIpvL((y zV4(vl#9;_yNG^y5)uTwng6Ns*8yea~0nzU9M)2z*MCxQS5`a z=TzVJY1N>p*@Mo+)F~yi6cSb3oxYSeI3M<@#db#lDO(?xxLm~uR>YTnppEBjUzX<< zy}mO3kARqjM0ZXkZ}I9S2viMsKF;^PLL8LcW_W`@jrxN;gvBL;b5{|Dd9Z?oL)Ouh z-+Wy*G03X+?TW0BQwb`KFBn(=e~}q?AbAdM-TZ^+IYggnH+J-k)DUgLkIoPi!KqSi z>|>-pOL-Z(`??SN)(DV7uFMi$h1}UOibd*;5i;2Rywc+{SV8)nU??q2P7TXmyiM3= z*B(09GnrCvK`yo&&CrIi6CZxPHxcJaYR%_`IUke1#1|ZPq_V91V*(Pq+y&KM^hW4T zua@Ix$Vi(;R5h&Xd&ZQv3mC5cylN>$#vAHG3;G2Fz@q9s;h+>K7gfATYXtKgFYE==Ec8>n2_X@+pD$mhKx%(#4HKr6_Xk*$ zc3-0Km(Y+osL$A@Qm4IMmh;G!wUo;h6(_z_HlN9SY0_2Q7s)rp&sP9fN|JP!|5a`i zAx$#Y)&pMjvsuXIE1iBWDRVVES(IY%!)K8-4Q(8rV+%nSPXOf}v|2Z_DW(rwA}&-I zz*ZMG@BGxJlZ9toLU*KP5g)m+nqnHOVwhlpyQeFVFH@JSt&rTQ_UfxGGc8QED2~=dTT0t23zNAIuD&V3Gj2Ya$<4WBYqz@d!vBR_ zWF_U;wD)9A-1|bc%GTc;8cH{T4sJ2llT$LGNH#b| zzZiAPk8pmX0%v{Ko$nBrq+}O zah@!eU)l%1_AT*fhhDtG7I0)|MBsyDGlCwOW4W-nkQH>I6Zjj4{dn|ebBC*0GzxCn znuo;Q_O)hLwgHeo+N^MX;mNFS=k#SWorB!Tc`O*m~o!puv@>kgLT znT#v@4A31JbfjNFEkv;y*&PIw%`R}>@Ih&p zxVxf-kN@1et<&_jlTK>LyB!XCc=zFzE!MJuX%;qh%s9y>zfRkCzVJ1Wk(s?z9yKZA zcY=71G*OhsVV}vn@<30ZREldz47-#|Ml$y39{jJU!13dSt2qW}nKmt4##XGUMOgGh zS+j#%lr{f*FyVwpea~2+&$tFZr%TSPfTU&HsWAKh+GmtI1{Ndh;8Y_N^-f-N-xuvw zxmmUQk8gHs1h^yy#1FU9Nx0X@F4)lI=&50a1*q-|+38Ayi-#TfOKh?OFD=}LFtYgc zo1q=xmi)7U9L_7^qhQ>=6 zU0hHq3f9SL{=pC)jfy04ty)p$(@9zgJNMc(tqaNz+Hnlpaq3n=Z-AsCk5@7&)vHfZ zI3;w?qp-g1h+;BemgavBm~8&-z#PG)k~~f_xEwx~Xrj{Gyk2-(E)-acYCWzs4L_yyMgi`kxQy!n>z;_o z+t7~7097JVtErcRy#JFj?}n>eFWE{&I5T{n85tZ(m5LvkeHCE@1&^iCA~AxrLc(M- zNn{+39tIf|I^JD$mX|FazJE3NoGe42s!Xr)Xp)nRN!En6>iHo>&=HQP#7Hi)TS9R~ z>>fwBFKDtFO8id8==R&(l!vdlf&@F6g`kPiz}V}wX+J%a>`SSn=Br%3`1YopQ0LDf z0xLqPCczY%w_l;PZHl9-7X9@M6 z&U)rw7lq)fjTeuLT%r{+HwS2EGNNeueC}Bj#~2z^bvRthWKZq~SM~CFf&z#r*WIlj zoVePvn$Kv;z?Y3%+RqNyRW&w2L<=za0-q)nEF@)hp10hE&u)pf2Ie6S>O%xt2Bh-5U1OllJ z?d~{+vb0$ilme~x9I=_T=rrymcn=aNoftzzbixo$qJS2Uu05HvBckoFuf1>zJSpV; zc>p*nGZj`7S_1lSdP~PA<~+>l@afiO?c{q^lpQ07*FpaguqviubU4?#ilkJT)`5<xATrP`49R=mJ-B5z98R?S~UZzvsU$B2hGKf#4+c)7B_y7 z+f#4KM~i3JkIryR-y5U*bR@-{WN#yfRtX=ZDYI!aZy~4?_JzXs ztD2U-j21cFQuw7!rTUpNCygGw*vT!}O8V_tcH{|)3s=70KQ7)+#!~tW?CkO%E9V@w zx?|q|E9GL?2G!SdvR@qsA++)t!~A%?o!lM~lo^3V{CsTZJIhJ-D0Plw@aNukf3S_i zeuLt_&cnwkB2Tt>YqV#K(b-6wgIC|Vx!_k>yCk5o#yUF;=1p)nZyhG!i6SUw!QwXVC1m2(&LhpysQhQN>D0ryeuXbIj zY^mRX24oRgxu+3v<=DuNUjlwnM4BHcb_QR{8GlvK%A%-J!7+#jP4lof(j{y@N>SMI zn$;ZA4uTdYUl=pYoH;zl+})8vN1~2K6Jux#j7zhSG8Y&OoSx9_nIqJ8DI;`_W+cb! zdk}X<7^gmUXc05)R4Abs4J1^>2-P20<24YMDrMAY6WurQzt!pw9OZ9}V^h9@$fQAj=%RR+l67crI)%OadcX=1hOUzVK{m zxcKa_7&P*nDGgTMC1tND4_=}^RPrI}DXO1Zk@va8xI)z$kcz%4;_NVx6LWN>P!ZwOyC*u^&7|c)1C6#z>;V|YFYR7{!Cj*P#P(dxR2`E zh)#9nO^H3wDTcqZeN$4RX8;k>4;b6nMS*tIe&Xl zB1*jqJ%q!nv%@{faEpuP_R~u@>Au#}{e~lUgXnpCs%S3+eIeksb;>$HC8jjce7@)J z!K4^bQ*>}}=;fBYHGT55zCp76uSM*z(iHI8O}hwJe%zf{A*8k0P!Cn(RbX)e$ODvI z@(y$h;LaBLASeN&fb`7^DHDS)Q}e0JdQiSNupZZ45L=A3l|Zu=OE4QAZ%P>XLey?= zkRmksjqy?D#Zqu2kJ~v^^^#(FGby_@^J9`V`bo!vx=%QGmNF^75mIZmW>hCKJg+?j zwPPcC3iUMr_E~khWkXR@gj$%l7F&-R;epNG5_2t_Hbyl~!o=G_`M}N)qor#^qvQ&8 zISss!8y07hpT>o=h!L+{9%qeisYF>P$Z-E zpRikP{yFz|i&iv%Pdo2^`vMu?uZj3*F9Q8#l9F8ok?bH)9z$gN3R&}cj4s!32a>HqPhLP~QXLwvQfa6gE#8DK!%IZSFSE&=5w2`FCa|rf z^h(X%kt=ghbSQqJRA5WcQcgaCU+fx(_jK#;EvW2?F_@BTIH7Q#-r3)0XG|)wY&vF_ zfo+ydYw4s|r?yqv(VUUJ=87|&ajU9a|Fot~9CM##L(I%PNdZwhx=G|iJ)hvRkP+1Z(xgeVXM!eP^n56dD2)U%6CEKJeS_f{GC;Sef< z-+M5=PJH4j^8~D2(mI1Y_j1|XiZPfVUb*XF_Ybo+7rsm`scGeDtD%jEXOG!yZTMZy zS@vloTQag-6%=#<*bW&BWU?hFR8=zN*W*;e#;2x&F%su0o+UIcvz%yZZReW2*hPiQ zZs($lzmjD!2vZwkZ2NYTfx15IfyvMKj+yH*KQkUlyN`9(lxi>nk<0KWMy8U@!{H7jL=+#3>xB2Z(Xr3v9sp*6^-Cxf~*W5Bq zK9v;y;eXQBmO#s;rQz=#U$N_qN!0bVF`8PlNdnewEGtT+{#=hw(v9MVj5sCtm3>jW z$lY9pamI+CBfRhb%05f1PanS1OLcWirP?zRd)q^ir^fc3=o~tnViaO66A*AkOYnoV z_TeqaQTiRk$*Hi#GaOoB&6nlvgbm`sm+sG$)# zXy4<`-(tQJE!~iZ=9A95B_<<~Q{>89&PG$cM+Q)u*2i*@t#hH`GBh2jV^WsQ2pIMl zC`h%#9Vxp?SRc&f4z;?qA0Iv5Br#*2<4w%x99Jz^X3S@y+>^4!=wQjA8go z-nHTN{sb=nf>cadh0RK%=+~r^p-(*=fLyeiNbrl`_PF_ugwH9H{b9alnL`s0&7LN? zvz5{SV7X8avI%<9$Y+$(f&_21V`v=;!`VYgqq4qLSGdGNECzLnFPfILqZcUN?=S;( z%dK}sw${bYb3@KY|bZjAbS0qc)x&(S-6RV}Wo-%>}0L)fOjOMS}BEf}c; z5yA0kW=DBbkhi6{8m|k({mIEVM_+4ucW;QXx^`;#8SAq^J###zPZ{l?#3~hkjjj}S<~vnBYdMky{XPXPxeA5w zGRsbplZnYe0w>Kp|4N(8Uu~F0I*vrn52X`(5$n z=`?;7W=C;s75th`b`P(f&Zm>x`nAj!qwq0d#pf&K)***&bGozN&i4ufC3dE_4}zNeP~YC32@L{<8gk7FEJs+qTr0IplD z)4NL5k(72gY;ZWx`n^7$Mz3mPi`DF|zQL)M;VorCvKUKs>R#EANvh_|QfjalI30Dm z(_T_kWI^DQ{PLG1A=krmK;nsmr5!-KV7qkrHF|x-6l>UR_eq0b;St?pe8N1M*HCuR zbGoVe@riS6>%5TD!tq85O6P`*-F`20EFO%lH>OTwC3yeaU#C_?A^=8JIesh}_xXv@ zbX{ezq!zPhg44<|c9lnoqom7nVR&1DT}_JY7Tbn+d7<9wSDl=ihgRV{uN|mT`z-v*0an zQ8Lz-@6D1ADeYM7#~0Gdp2d$}X1wK{I$2|IS^AhN=JVO&3LoBL@ww;=_!c7zo-C&| zaW$VuX3O-krYWCRQhqh0A{bAxk(mlacZ?!CXb$p7UECj#LKz!>eNk&?5NeZkI{ zR3C4RFLGWT8(jx57?(p$xxK!0^L%{xhdnw7!>>XxvP93*rX(>sO?N;4RtFD5uZt`J`;;FyRclqQ22{10#xgd2)NsHq))f$ zQK5+y-QPErc0ba`s2Qy5ydkt?4Gnme$)5T-3*8N}ag9Es`S_ltn7+S+e1GP$jUpK2D8(?VBp$n#2TISU($GK_!OON!zqtjzNdZQz*}-ka8r2=5skyvq*=h zjnEt*2s`%}`Z40Ik)c`uk#DQs!YbA|br$hR;77@nyUy`OziCvUvuw=RfPUNoBMI3h zp(*m0r^`nhv3<%~#p>8@2i-|g)8r>d%_8!`g|X9mwO1jb1N4O`o4>$p)I#CrvU$TL z=quqlA$3j4IOg#*HgRAelF&Dn9Vh$e*)Bsl;b4WicnOp5 zv?nC49GIXsY8Y~rNx}p@xXfAbrm~d2O1iad8nxs;3HO*kDk>g!?t&?fjFx+F60(zJ zQMvE;Y)1-t@)>+>p%Gq~ET0Ytq;MG!di-5YSmvEcE(S0e9Fku!vx>ay0_&*rZoiAH zs*=pvHeG-3!mM9G2f{dk0V9QoRN6=@Zc^;4PLvTUGXo-QwB2WEg$E-szU*MyTX$mn z<7)wIrKz>fT&3$MZGnz%KD7mE(>=}Z5uYz7&>B8+-D;i|t}}`JCd*TAsy1UrlXM2^ zUY7MX zia)_ND~(xr`xbb4>&bQ1VvQ@#?y43v6;(WWzpSyC#>^NTH`Dc{02MEnBh5CeyO+Zx zsi?v)wTqk&*nQDfHa0djQw2`5=R};b2A%&-soCrTq`nZx$K6E=@5yQ2k6cb{)r-}U zd=Jl*sV@?6sp7w)!*m*VR427`i$Lz%;X)+wHXaC%=Kr0-PsmTg1jcE%k%JnPUddR} zN>viEHQzl^U-{;jWWcW9owd}#8gd&}1ZKMGNcCm7HIFczR2&f&`KCqmQ4t180)S># zAWx7mX)it~7AzWzoGOU)%g(7A^}V{MA@C&m`9Zktj7CL64@b^)JkfHA0392)%#)}H z8>zoWt*z2ujq*+kiNR0>8QT(`mVv0PR2BrrZ2q;c)no^6GLVy$W}<+jt*s5SYe=n| zU=&c-*0x;B_amR=7-O~fbMDc~k7LJ{3vU4Woy0VA-$R!o%iJL10ymOa4AC<;b%e?2O+#`=}TNW?Qv}RfSI7o7j%mbnpy0&Ydme+5e!iyQ>4xC^}s+X8kY;w zyM+*GxKpffwBkthhT)G%rU!yhQ=mqIx!PfQTEuTiXg)3*6lkbqiLc!7T8-kTY8f%4 zXDQ6wZ!K1fZT=8evM>$=kT;N`81Vo?c5-<+{fH{b%f{WrP1M6im*?JgB+Ffv&?o}H zfALi0$tSS4YN|r3m3N=nzZc4xE9t;K+Iw}<5aTp1IjJZhqbeeWT8?jk-T17DsS&Yi<-bR_q+S5uOBP>!>t6}Ftw?7!C|h57(x`}D^=$6cy6 zk5&WvgAO7N^Z5KRe|byh-7c-TR0JiI(v7tX?+12tTkQjdGz=NhzA*+cCdM6$Ouvr)kijH)1G`AMEs4fRSo5~%X1rOu021Jr=bQmBoHcVD$pwBR;ize9x z+yXFsfo#3qPcjz)^&^jQ=dO#wHyVN*BVGTCo=jL7KIJG2!>ZCQ{=#d4pR>HY#~y)y zos5f)D5tC=b_(^(RlT)v$VFfhKq4d}a?iX00^p?4{2$2v+^hy?aDDnT8iB>I ztuf+9+5LRpVG8I5$i0;8}hE1(>h`lX3R7C2&nKorbN*Ufh zOCN<&BV$FlmPAWFBI;BW5+O#O4|^`-0`{<0;x#FOX-UxO;RwRJ7=yd`YZbuQiE&cB zuDCdSZb1v&-R>`?40k+?bae4EGb(yX6%@DW7av)O(x2vJOb(_k46?JE%&WpgHYxtx z5OK0>1k7o+2E*$549P3oxXTf*K4?M(IXwgQuEkkt!=FT%B*b!Rn_<$!E@8kDETWzX zz5VRM-v}I+_XPs=E6tc`CJ&TWowH74mct=HwNn3)nUxp&-Jpdz5_x@Bmmc%`ahF4F zmuev`;k{H2ef0;J#ptrQgZ*_fjjtN_;Oz_bSF!H<>-no;Up?yJ;9#*xFfrQY2M{Tg z1gSba&#TLkoXu*p;_=Ce>HFL3lllTl9eQ(fvwC&=ad~A;#pcI$8)cudZ%^H#z0ovb03;eY zAC=+xRU6rm97%!>3LDNNyAWD*vG9`+bQ@k$a#6EB%=E|cPW0*})Q`*tuSyb@T`IZw z2(Ho%LTqHGO=tTFLhEE}u664-^WHR6^3(v2`enMZ&aXP{n0{ z$CQ5~YOtpCw!-Y@rux~T3{j=wARCjVxs%QO=?jdORMg+?$A$Wa&KMQ22p3naKOCe# z8mo%Wea5GBn_meK7tjB&vAJe2{+J?gzE`)du3((*$Ceqj;!hG6B`(rak$x7GmE=?L z@v-ZlREN~nW%g=+Ldk>QTYL&$);Zppd)^i{N6l*pt$;yL5XT={NJqaB|07#S@*l*I zcYOzSf6{$nmU3Rc;f4~f9lPI7&e#9>1?#ZAgCRR#QBj?!GqgdTT$HTxC4VUEILz-g zDq}SxdM9L^0On`{(q3ai8*8aMabK2Aq1HdiNo>gQ3XzgpMn<8OjEK4~AfuICIl`zP z6~mYGHA~^me{H}7|IJCe#-td}(;H<50V5nxwL?Mcd7^=YZB~-maoB}SF_My!A@e1P zd`I=OL}gP>@V8z$<;sm%fIzQiI;a=&%*XBGw|zc7ocBWq`MXj4xg#2UFkdLwsxNU@ zR{6!IpP|h=k9(hlnM^q(+@XB>X?y#jKL*DmDpl?|6+f~$PADv|rv~@kaD2zCv#?D#?zZl7@*V< z_&iinP0Ve5Wd2(H8uHGm6QhZv0dyRWa8F1bQWZNH+^g>CI z|IQpu17-g+#$rH^ zfx1TI$wCZWT0x!65FHuAvUSJ}Hz=Cq#LvlD)P1EKWKR}JjrB#yWWIiDX=TOo^dRUt zEowl*NE4E$x2c?&1UPye(BEfB(f1)p3xsWK$V-tGkEUWy_PWQ|r$)%(eb7xo)z|6f z-wS_3nh zO(U4W`EM{$dhBfaGdrYUWBuM>D<;Gk+yN1FubtXFdhjn3slA39;hHl(`7OhyFp){J z|1pQ=&WkyQ$}POoE~jTnEY>0|d&|iMr~P9&Cj)VXx#asX)%UWOL%bl$wOFRjcHL&;m1CJkx>N-=Mw&Wv6*(wUo} z4C>_{gF)Jw658boPxulb>Lrn5vO|iY{S4Zfg>c*kb9x98a4>@O>+=U`7=@QDSLj`6 z$fLm-W|_7!<@@i&#~3}Tx4TTYN5oUza$6-e=M3pu%pfo|%u8pWM<Ywi9bXsR+^tf>Qmi#5?GM`ZJrpCg#K}TOF|%zSPh%Qc-&%C{TanbOS}0ZNsZLPM5I9Q$3%l}2$3y+s)@;*J2~sT^5@j&96FP}obB5h zi8h_ANyc%}KwbVP+p}9xmd9_MZZVdNMRW zRYVw^Ah1=C{AXvuLjjUWG!HDm31zK}AATyNGPPV~un+^hz^RCzx3kM(d{Y<;lo#Ai z1;3naG{kBX*C#JA7zfDzJw*$bp^YSwwa%@fum8O@DH=dRFcIV86uoza@!)fc+pP5` z(=*_kz$h?F8s?$IT9FUry{@d??FG9F@U?n6-v($TNOAsmdl_c`?{Cj+v6>wM0*_Z# zLiFqZ$G7_bkL{r`kh9|?_4A=a{*yZY==vY&{NMlLZ!JQqE@AS!UrN&7^Zpz;3lt-_ z-k%iM8XV4iP5tkmmnZ-xB_{-G=F$GCVgsKjBf02bmGh$k`$&%eA@P*MZ|56$0W5(3N^@bA+8 zU_jo@3qhJV{9APg0;?+-@?IAr*{095IQBtN7@bJX(Y8p zLi8g+5m}9>OPO!SwF^xI;~9Oja=CA^Ub4!nniU}i>hs~hfpjx920cRb>+MT1F(@Nr zV~H;>UMDi%Hh(EX?`&Q{ANM7S;$b3%AJk z{SA)-K&j6b^44BiT}4)@3N<_)@`HpftgZe00hKt5Z5tC8k^{~A0Xn1%Qi86Qo|*a6 z+sBXnR12iTDNH(rKYxmfNbn)g&(E{FoNA`C8U@T!R__qm*rayR=t`Vlq3Z9?}$u z@X$C~?T5(88F$`@Z6Mb3`8HwHMe%6%sx4~Kpt=fW#b2U#H{r4vdCjc4A#8d?$^Q+# z@WsSYNch((+I@1qk_^!1aftqjKr&rV(Gi ze$^bZf6goFQgtB|u95*?W2cw7W9}c|wXE2{kIH|zM-7gP$`?|-4Bu`_?dFTP%~Oyd zt%i4VhvR|_m()@J+2~vL$4>d0xPtZr+2h&KCW;fJhPO|bKuQhxOD3!imL66;&6GBC zRBHG1Bl5RS%o`|Xui?60xJ{idF#M)>#U=Xo4l3jh$c+r%lP#gB-}t6iz9Z(#EiFC3 zPZs~ATxq6vF5}B6Cwja^t+EGU2zkOs@C@R1N>PMq6L4!`uz+Cx_Oo~ie&i05KcF?wKTYXIJxi&+$$Er(IzgVeR(DS-sDB`_PK~KLI_9P$x zR#5$BAb9Eam_+^j8?3LT2wz-k?T6mNtb8SmFNS1~*24_}1)C5KrvPC0P!K-#G^^-- z65(ti3AnYonwgG{j_vJbv~V_16NWdUlcn_!)WGQU(Js-k^VO~_uk&#d+ZMM87%i}O z&i$D;e*=tj8zRs<9$f@|3~P*qkthSVoV75fbPfPmj$Tg*=Sql(rLK&E3d!Mr&-pod z-kq{l(x{G%08y$o;38EMfj-^ZHnk022YNa^d(rMBRQiNz9Hcn^(a}0?dFfUg!OAxj z`je5A7?YDVWfT%dQmWUG(&JgXF81UN8-r}RHcb=|7z<{tJCKxwMDt!GVHloqCLnnS zC~j?)Ao4`~vHqkx;0RTNQC5APn{`>}^{V|IS;(p%un6ziOU~pLTzXb{=DZeoi9RH| z;xQwj4~6guB%Qm8Ia0TF2_w~5sr$GIH`Sxu|027hkMB_J2C^X!p)|%dN&&87^JKX4 zziXp_GP;4h#`cx(_S&J5m1ws>^TG5dx%`51qZM!<>3so)4% ze4?(^VyhPrG)Q^0JCaz`ys*v>!4M0KSj6{~JH8@Y)pK2+X8!Ad9jatbU5_$ zanRDLGrW-IhAiCW4^^fDC8VYx#a#|C|439+St~u}kOhYsv>sJZ=wn4l!x)DH;glaE zR`R>kPtq8-Fp!TjXB5>qdj3%oLOjP1cIcYDdsK#599=&=X50`En{!UR>KA`$yO;`O zynm0qoWUSc2Cq%Fxr|un8rfrDpBZ*L08`ClaI9}r=!QONLWID|{8d656f4{q4SyB{ zVlC&+MK8b&+(-cS%b&jC_b=7Cnu_H0G;0g4YxGdNJH_`dqZ)4)!lR(uIx7v5-=!g9 zdy|G~4!Jjha2JVFQL4xkA}TXXwqJgQIe`8djDV1R{=;L*)Xv@@D5=3MDz`5&we)WP zbZxmUS2ZK;hsdT->MwAEkpcv-THWT|-O_cfz*el(^Q>(leps)Y7Z2SvYL8$mpFS_G z#2%?gGfgRvbX{nRu;f?Lrw|mP4Di$x0`+mDBoXyuCz=&0m*}LCuUwBrZX0vj19V&X zlZP`!1r2#%mVkse8yUi59vt}KlV6b6-GKy$ zx0lBdel1;o1Yo1H-P9JgCmS3Xwi-3K>@p(6PY7iT$+Zzd&V{RJYToTt(gwq>GaC0z zQH;6pCLB;on|*VVI+`X}fHN6Hsk=b|*f6v;Yqt@GB(f!tkxhEPV=ACec1B56c=`mPVN<4Tq=$*~#(1Q?s=GjGYDB=QLU>8fEM zU>~lgMoGTT$<7@Cf$;d`$-jH;`x+P3e(He7^CM0>n=t0r&rH6B0HOy@5)l00*Dse2wnsjimozd=(_tp6$A+ll`l`#wMvU_4eH_EcLS4(3om z#WUUD4g{}v+m}zGWNS&$`sv?SeH?U*9gvxa4NZ?I4WTOA4Vs>MGJ`75%9GX3Dw2>3 zuETjd9L}eD407n*=s~8A&t@7hCDdeH=my~dp1hECA1WXf>Z9?wMsMZbU!U&@EC$lj zW(P4UVM9?#HX?5pTE;E>GYulA<`VwwW2qO}Iw#191^5L~nV>;F8&v7Gv5PWyG(~ah zhy?__oPIZr*ZztY^xdJcNpC~ohuDpN!Z*w|-o9W{ldrdU@H}{vgR%zMM)8`2ATZs( zu=(FR2%I~3T$a}x7s;Ep8+;>Jdh9?&9bP6jeTN1v~8DoSjdDk}q=B{zf|%39$T z6x$IQMX~-&s7`tqyc;n}w-Weo;~0gN-tno4F6%fBbqWBTvj+l(T(63xgnE3KQ;7(7 zL~p3g_R-)_OWT&P-55(MG>ciM zc&AF7j6` z&8~=u|Ea`!9d|*Z-xDeFF{A@mB?O{c+#u~hIG4q7PTbNM$dm5_k%&~k7&_#$;$#LF zgAIs3COaitkx5G66qffk0kO!S7tz?r;<3HHb~zazp+R(C^W(XMg>(fZbh!HOI^=Vupfu<^4RdWbbOG zPle4bt}vslYQ4drnt|lFx~lAq{%XTpqn)|yYhiu8bh?ji&@|j&LYP2>PZ%-ZEPdBm z^OEu_4-wlBkpy;vaa;$b*|v6~`TK zUpgSuQn{!m*%3pST<(ro0Cr)B(&qOrCJX2SlR8*=9!=(@?AuZ?}%_E zrhV&-)Rfb3p@OpT3AmopF~n8WdUJO29e;0Tl@afpB;aSz*_+0{j)AAV&mEgRmHV|p z%vzv;<{*XS30>?Q#R^>VU%_{+l4`kS&XHqHdIm(Pe4mxb(ad2^H~Q9TT;%6x4}7_M z9VI@e#FeELF>lCeVBb0Ds?-jGslUsEzb=ob#A`OrFKwjZQyt+Ws^6WhnxDcREd+|G zNTD!~N4E6qp6-ey$w<~&E!x7oitaFL)xCi6uIyB-wkLA;tPE#B9LZunM~*Lr<`R`jdC)RQqTzrg+Cz#P-4d6LK{6s8H) z*50$0E{WmZ;+pDvf1-HQ+)Kq@Sif>lXZHvyVb|aE)wF&1)h?0Z`+lT6x?M3$ zRb8btNsF?pX{D^f$@TVk8*;{gDFKo1rPjXX<@I7SK7`*+F7wu(gDE`N$$~;hf|l;M zL?HU6!EMF+*g<{a&c;b&+7NOE{Mfr=r;i z=c?H{K7b|%jc{J>*Y%hAw>G^*YT4lc*cs{v{BFa}G4V^wo1MP-7py&!&g6)#?Zb^Uta5 zPRCqN$w*MG>O|t;F}PHHGMmg~Q&oE3mi_?38`O_ivLBOq)q)4idI{2Dv>&r@+(2Rw z77lEG#?Vp82yx$FHw(B-{bA7sk*JDp;&9w4xY&59%HC+!kE&` za(F&#(UC;n6}o{RlR3n!_dc^SXjlt$n0>==Xxyv(a=sf+#qMBuHTnh!eN*ENj(Xi> z=WO~}dI_~$^{b(+Zh(~G9_c7Ry`9T&BwmI$c8AK-!@ua3x&p5kOJU`G4>xGo!UV}? zx9U9(EYq9HS1o-ki?m6L?jlc_()57VTb`dg8=N7>1 z6&I(*CKZK0tLm%rbQ$|-(gzJRiWYi2E4P3JZ!`CP>5(i;v8Y6L6RVi)LTeaUFy_=p zdz~9qXfL_0E9+UMay+PH>U_CYQh&u} z^9?Zx_M$3PpOK~9jWkI(%`uV-ROSPXDezRfD_qu0|K-9@@38Y)>FBfV=VabCJ4N!F~!uz!%@)4U&mG9xZs+%Nw=4W5*mD> z%$Sqg!>=Gx@=LakKk}pG?3NOR+Io$9Z#_w}5p*pdNy(L8UqVB4$kF4~X)lHTBXQz? zDchf)rUb4P`eAhiEK}X-80hIuw)%t9S$N2)+AWkppmbw5kzTo}eOczp22AHEWyPqpHG6+vX%hTFaV30=V$q^E+X$AS2Mx;Ky2N<5=#j zgj^Eun|L-l8|jZ8n0@T&)MFwEcg^21V2)jTp7jD{VEa6$1n*5uwmZAK_4Pf^xWsK4 zD!|a@D<;?e=izNXp`S*~r9557Oc6e_pnf`2-8Hx3etg7h!rA)`8Qe999-c4ciNz_! zUG;Q~tLC4-PYR`(6!;4(^OFraW56I6)YBTFktcvIs z=+>)pr3s_0rR?#UAX$=#vGGh(T0`0yXVonRt@CdaJqK`?Toefh)dg=#_C8 ztLYs4fE#=c;Y{!ar*$eE_iKr1qaAH*m^>yNgczHi+tt~9VNsX(`GW*tey9ug49Q#} z>{Vw2|A4QQchTYo!Yy!~jDrG9SrB@nDRKsM~Re z+R{V9nVS$@&ctN1L`Fi_KSRhE4(2c~tZi1mYPRLdcU8=y&5p#DJ;;3tFICIV|CE&a z(z2Q2#g&?Re_XMy6+=j8e%T>9q34d^^8`_G1)$ZuLqb5jze7ED75ra-%BS84UYKfk zwOuC6|>K6m<5%p$C!RJE3kW+q!Jx=3m zR4`}@<(d_}@B6ClMrSyHCb!ePr3^8ljX%rZ5t%W22=&071rJdYOiDCD84dDZC8yX#Z>*6`z(5)m=5~V^JiyH$r6SAG^QQ_$H zLc>@fg41^GFUMWonPYP1d1l&1ub(a!cV&$E=q#V|KAWC|1-bVol8rC4FE>fHw@MQ4 zbWR^})K~~%olB;}osyM{TLpq2DM$**71VK%*7?+PpprzG4*;h&dQAY{pwS_!=B=vujg?w;(w38%1k*k7B#bnhpS&cBDUcL zV3^V{o@?y?g2_eDJnPb?H#U8gi|SyTEN|`X6vznJlxQ%Us61CQgQ$@P+RABUbvAn#>UG)3B`!77s zxvNe$dq_q`MpXVnqYC(Sdp}zid>!TT%>+hThX({e(Y@nwn8`SQ#YwpuGsR3wkr#ax z$1kX^X?b!f$~ItR9&kOlDtgJBzaH+IAka)!Q_U?=leE(Et8#T_vI;3_e-{`00mNq* zbz9N2w6t7wCN4I$8qCPp*=KA!kGm~SxF426hHJSTE53FSeRIjPlbQEoP%`-W`1|?F zi?&mX=T3RaZd&pYEKSp+bi+-8d}OeHs{k|IQy#b2X)qv`8?5pr(vESc@ybH|*OhB- zkL$}>joj8F+Ei*lK){bxn-(?GX|8%kot>30ren5igV34XgFg*e0$meQZ_dIuUE_xf zKH11VA`jYZ2D3Cc7V6Zj6NCug9p*=yIlr3B5L>@IMya*(<;BEun{9aG?B5>KZ^0aU z9$(k^^2Yk>^u!0Oz$^?O^;aCP&a~yW;AG5Xu^woHY-@DjN`{d`S%P^fo*h{J^eJAg zeJS*Oe)Ie*=4NhR<{7neg{3mNr?vhqYA5;Jm2L|W)sGm;e185&F+Gd+MXt152dF8oL+tlz%Py3re#B7=LHbN__r zq3ffMI^KDYxX*Vk*wW+(|IvABOP+!g4@UVjGZB`2% z&EH#}P-_1V0Odd$zgN!R{`NO??AQ@rH5G{DA<=ln23##GliIN$A`H{RI@5wmzq2_~ z`sFJaknjy=-g76#4VBWl^N-O7;Svy8dJVX}9e&@V6WY>uFDo|>|42Fj+!ja!%uTJ} zAKHx@I52av2Z(f2m%5zIwPT3;C>04lO;9i?8UZe6q@OVTi;W1&x{j3(4aB(4jr1ec z(9q)aBsBQuB8~gX=^J&ki_^cNvhonyIvg8g%pY=f9M8G&9!O85ad+lAHVhnwq3!A# zPgs>P>9;qNetYs~+_K>4mK~5dw3lVzEpTP@+r>?tJ9q9a(BjH(!UEIeuJTK&c_lbA zo}8>Vj$v4qd){*(Mlh(GWLZIO>;QfBhBS z-rl9{K$*q)V4Bc;t~s zq|3vH4`bJ^UE-E>96x?M{_uxCScKK23tPWI|NkT9T#~ThFdUdj`+8Lvj| z(6^=hTXVLqt^$$HmQ&IA`6Z?3_gONIU6vK}7WL^~DL9Zb^)x#Ex1T;oAmncF_*aV_ z5b5>pxA5g}Bz-p>fsi6EZ&LR0dF?gnvi!pjabv;+am(=0)pvls0%|-EnN7%chcyXO z8(8b$%=qDUH5nL$j86Dkd4cSU8;E;ucP{{A^g%ccME=$l-z<3+{aR}^l}mB>`$XKc z^h@`Eksct@-fyC6@0A?Ggny=DwSO=QCOqN+B2~R#oJru|4$}JZp}v{q{=QKa1yPm< zh*bBA>*rb z#$*|6+!(hJv7KU-3?K6$%LDJN(skbTyeiY~7ZvE~OCYINnp7^AuNGkm#-J7hYgH}w zKs5%3SQsR&4q@v9e}8`~uu*`Nwk$a-yRO1gR0|%nmBu^oydxFQ8oeV&j;!peSmb}@ z$`!o-{`*o`w{G1qWXKTZ{+YrO9R9UXy1x7NK(r3D_Yzf-x$WdsK{0;3@SBuhFS3Yi zPL`2EYU=Y8^{dTCy?3?4_LN;l>bHmR<+%dnk^yZ1F@r>R?tsWH?X8lYPv47#v^-=; zmbN|!Z`lH|eS0Cu*D9(hpa?1Jw~@kMFxHIftGmWMfAm`<9K9&#NgDm(%_z@4l&3R8 z_S$J;H#vr+++vB{gxe6?Jsc73v{vUuNI$d>Nf`w=O%_i9K7P2jb$bkW;4a;|*=iLG zmX}Zcm2~DRk;7POl9y1~p<(w^TOF)143+~FAbrn%B%ColLvB}SD28^|w>+|FS6MDn zcKwK4(ssA(@^X}rBJillyDg#@XCFku{{Q0W8kQll?q>a?@aQCpBS!C{p3%%c9!f^uoHp*b% zoCoG$Uf+3cD9Lyh_U3D97Kh9-Vz#*1#(2hVh-q)WR#H+TDf(0g69PA_y60f$&PELp71BZiayf5D7LkF=Kf#M+Rk^^# z0y+D;|Ki=$bTDB0yM|`V-nyiaIH^kqL)CQ^i2Q=UHGa;nO3V87FtS64q2xSf1}A=l zT_$T$eiEm~4MU6i4!*Gz?^j~YTz?`PWh#XjLxpN0HLV+fr#iJ&LhX%zB@&&m)#UvB zX(%p}zp3J$AC0GZ1S$t;<`2FD2Egl^Z*a4BZ219<=F=UK@Q7P63gnz46oPXV6(%t6fLdx%0@navjZEv_|J3 zKdEZ@>B(sH`bMMGv>%UtY_*D-wcs?;&s-o)0b+X$Y=O@0R8+Sm1<1cf3tpc_2x_2y z;gqBJ@&#Iy&|;%rGX&HzEd0YW=;L;x?iN^+5YDhDOB7^_paDcW{yV_w8i{`nV+5X7`w6fcY7UExI=r*B8; z)iZrNKxFaROk`Z-7q|}^Hfw{n&723+1t7F$;9!hucA4sIqYtT+;(}b7Echd+5tYO3 zJMjxj0^1`z#8mFJT%6C%M{XhYwSQeSqvz1Pm8wprb_YH(M^3pF>Y`2SQ2O1d`jXpISc-rq^c;OD zZe|9J{{{#bn-Y$GKQvBK62C?i&EXby>q**3O%c#QHBR8TPaTd$r7hMc7HKVRi}wLN ztH1)hy5a!dru$n5kHUZu(f%TAS+f}^hlu^JR%-#!tdNoKn25i=Th1r}n*+C*q`R#Mn4 zUx|elJz~)Ru0R~yu@+X z)BD458^`fKZ4-zrp)zw$=TNz&@Ne80ZCbRoEVr-_^p!!Az(7^qEz39G5aFPDspFet z+zMxU?+vwveclWL`vQWTYHIqIOE6~0%RuxryuN5I26T2VJyXImH7v{6Zey`($SS3L ze?J-Y=zFs-c9iWyGx}ojDEl-?dU>kp#&(nw1>23jO0?<*`vj=Qzc+Tf zpqA|HYzYL^qit2ktE_VdNf}(zQm_>j6@mM$y1-j*FN36Y=w;NH-WfvD($bKam?&LF z5#3R*Uh;Sp_l+I%5kKK=+`H;^I6u;WWy_XH^sG!B%NChJemGBn#5I0r0t4GN0aJ-N zBPq*YMf|?=Qj#l(z_L}7(AhpA>c{Mhw-n-gvWv^x6 z)sGHIb75_~u%Hke7;WAFP9>(3Uc_`_E%kBy*;ri)b!8o=t58ZH<);aV%G29Q4nP_y zo_DrpB(pA_+<3&18;=C*JD7D>PrM!}z_@dWUzLK1^3TFDHmx)BXq{>3D`dDlAa%

    e4Uce7O{9tw45fI4<_*<}g^=d0RYOBnuAwcgp zI3sBWmK|6o)rP52nIhGZoKK0ztMMsPv@2JFFm2glM5FX!Z?{*ee8JMHw7n9}T zJF+#DmcfE864^4*H5&VTe7R&MhP=F*rgF=$XznE1vvkxl%lq-(q}1kPQfkxC*@UNAMVp0Hc?Hb&M{}9l~ zG@5Gp;0P&{?l}m)pYhZX^)2bVKT(tl(8=nioq&48sCfCgQ3|lfQ5gt&GWlDwtUO~U zrTtqx$9g~VUOe5|T7gF|rD(tUo}gxBgQ}fpX&>4oOI38XMu12LjU=ZlLIN9x8^2iE_`I$Wo|t%&^c^l$_dz-Ldm zgr_w0ypP<`9A6B1$X+t#p0HX}0xEEFY;+7d1j+MVd*k2bBFLD`uW3`zrU=SE{unn* z3Xpzx-UYD}}xvv^J1H$A%5z_2ZB5?cQA~`ud3zK!`DrSJc&& zcYsK1>saG~NPfxi%Z9}s5NQvDOuLwmuqIA^8Ey3#g&Tbk&H$0`&VL6lwkJ!-(hK<8 z{8#Z(wz&e2KRp}k9+6*cJOA@nJoE)^$rjO0Ie|fhPyknCtylQdjJ+=X7kldFEUeFJ z4G>8Sl`Y#4bxynZsgJa$V!_5%0&D}kvHz*Z5l&$&=V`(mVwx}~j~FHSNX$R_75eYW zQ#QoqnEW>Y`=5wKcq8h(Q#%QiqCT!0h(##G%Y@gMr9P6o)yLaM@-&lns?T_;4|Tsn zDN2dL=zp3yv2?bkPb5Z#wUk5kfXI@Qh+n%86D`W6yn7D}cv&3&V(gmg)5 z9zkuEd`A`_{=;pU2tRC%Xo{u#a-{Z!wY(DthIT{h7YXQ~%u)pT8d#1^eLTykZM#*5d4MF=bn}rX5sR-5_L-eiY564RRWPiJuS+ss2J!8dpjm zJ5kYVnOMH|0H#_kEd7u?I2w^5@*sC=ta0(#dof|#DXTWk>JX0CqwbO_DB82ETRRjV zJB62}ewP=c#g2YIg}dRujea{CS<|0@@Qw{&tWWKY#k(~*Yc5MN5ckhR(uRci%_}jo zcTwHW(q#a$p9;m)kBJhQeoc9_$Toi+3b6j)+c1WHdn;*^V4S^80wv?xb7J({#Vs-d zM1~Pn`maS@@WPYfxSYBH@kc3P+Cc?0Y(H!|CiSLwT~RjPoBx3+d?5bvMnBZWb)^1h zF4k&akX$%{i#>1ICo@ZM%J(<#EiG|RE&sC>=}XipU6MY!I6G@k4cjMNRw z$$E~fc~ZJ7{o!;J>B1LrN=m8>ID}7<;r0_?~ed0On zxTKBU7;{#v!{DwwfhTI#!9>k^sS7^Zxe}f5BgXZ64w(ctzB@JtFU_OzK+=a^gWncN zr{+J6$qI7YJ(zvpXpKc=3HHX%#!DaWGsiWZO@4P1#`Wfnjx4;$>FDf@#P1Q=M2=AS zV?3gt`;aaysbVi4dJ{2Is4fw&VdvkX2sXVnMen`AJ{s9qqboFH4DvPTm4M{5m1uS) z0bvg&AgwgfX&^>Wow2Va_PNBOrbeApPL>BZ<~jFJ^3$;;Aps*N&O!7_0-DA-4-f6~ zCLfkQ{iSlX)UZtB$$%D9ja8TE0geKoWC}8_Uv-GEehASlj+qvfEfcf=WmVQukD`q! z1NV>?GRqj!IM&wNQT@{QZ9&4@BQYl-7W)e}BHaF7i*9yXw{FFeBS)lrfBy5Il{nv> zyMl|w*U6WU4c&6V(Uo346-O964SfW9PTX5pA&fg z3|iy0!`!~1C_Z-tQ@=SY&9|dzt(Z9UZs~sU<&)%AhAa-r!e9tl14^O?M4A*p`Tn%F z9WbkR2jD^mmVR>zZ_~O%ou{0M-d}3N8ft^I4HAIlff0xZDn`oJhcG0M_YKVZ0>Whk zhixE>%K>fcBP;C?!j9xi8Be<_0*fOZZ6X^lkF6BfucGw?t>f0vhHx0|53n+|ETw4H zx*M9Sl|aQU>1e#1^LuEoW8{C7Wk9h3KlmeE>flD)MIu@CtZ`%oAu z1Fz}Xf)uPpMd-x85yDB)S`uXq1Kt4D2g zN7Sq$KKqLtF=B+ctT9Ku-=LT;aV51>^7<7g3!Rt-NPTWUo(ru5GSos)XlFDRFX8m{ z@DCf~)Eq_7*{kPET((f$(xZ~subVkEb$L(pN|=PSm9fZ8nu@3;qfl^QlvCdd^?XS? zXXE)2Ge=YxO9Qu*m5`W5zdGk^<{q!UGre)`s5UE(p1_n{`_Z{k1Ef7M*dn?bh+_)Y zH_1QZly04rI7i91;63DHkmdX{BP{&xahR(>WPbvvKGy-Lu3j(0*d5;?F-!i=aT!L% zhk(|_L<9P~s{W4Bs@pH(3Lw(|!?hwvIzHCEHP#MzKst8>M4tWa8U*y$lB@x@QF|%^ zM6Ss=h2QVoX9Xy->$|t9&GuB>wC%8Hl);Nr zrKUxQLi=b1L|#Vx^6i)?#q-0m4owkstpH=r$z_BEcff(idP(IO`C|!0a-Ew|oqxlM ztU^;Ck{080q{-@QUNEWiT}axb)0>_`gIyP;;IZT^KDL=Zu8@=Ei5F#XSS)h1Wi4#m z4B@`NVd)9Etgt3j_fcH;b6E0W3SO3Sq&iW&P=2l%7o}%Ix;D~t4G`34#p^2 zBqwEveiYB&6v5Gf#)LnO2}yhcu%-j;G8g76eaqnrHS6X5h*@{S)JN9Of&EtIkxx(F zO4PR7b{|V68~8)xnfhm{DW(|rt7+UPNMYuMwNqnFHU8mWA8Xn*Kt4_Kr{>Ebs`#B) zwnwf+Y?D@4Gq3^1IQsqEa}}==(9hiZt7!~JlNM7rX`l7&(+>lJG7R>q?XmcZqSJp`*Cq7wr3Hze(wV`uGV_YH)V{&Q`T*NVw5& z7gyII5ZNLI3to7Dwy!KDd*^%jfE77No|(M>p^f~IM3HFRmZn7FP=uX-oR96|l8(+IRHV5>*<#+m>DtJCUG)Im$b#=sTdt4=>ndUa3(k*@fMOx>~`86~vgJSG8i zT?8W4b(k_`3L28uGt2g@i_*rC76ep}Q=8}Kzl48G2D0}|LO?NXfCq`)Z<{G}Z|pdI z2}PutudRb#U+QL1iGekt4wcb7y?3?Ea;_ai9QjG-b%oZS`@T;G z++j`H(b`+?Uzst6)G_NcAf_l*W;ZKsFX}@UnqjsVJ;I$uZ0kF*G3IWioZ_@Z8Uuw? zj#%R>w``vLhv?&$F6F1@x(psX7;}hx_UM;0A`Je7+laI_c& zrS#o!oY*3U07JEPs0oT&HK7g`omEX2dgIzrRi^JcfWIHjL{#hM*cM}Hou&uGz5759 z1&q8x^rW}1l34(gV;@FjD;aPRWv)8*GbX3}ATA9ZZzD@SwvvlzMw^K`a)ZFC$G+H& zZAAO(jI4a9H-TL3oJj(`_E!Xme1a54V%y$fQHMv{w8YoHoOcRV}{6+~Sp*iW_t1qMZIaBd)vy9OYl@fk;NW zJFQ6|>O`JIk`}wl2`z(=aocxENV+{rS1F+re~dl|!$4%8-=pXc-4Ss0XWadVx1jrz z?>!MkQ+gqwGzU*lNW>pkJ!hhEB@4W(S7B*OAiFa4T_29njD znAl$?Ypd;&wG{$GP&%d$sYhD+h%7#oh~RyMqR^sF1_-4h$Vzh;dTu{Uec_GW1dv3C zEKRp<9R&EoR$vx1H?#nqNefV32F$waexj~vq-bt(UTgZ2)n*rY^Ij|}quX{z;-hA5 zN1j;JdU3; z3oXzl(86;yryoz{kd~=qDaSt2{FQtHYAJX~%d;n0%JDc8&oGr1;k}qH$R`ll&&Hs+ z1XfW50yAR>%uJ$Fv6wZt+fg6W!qkA|D`SAiSuBk+d9jGHlUTB1p9J8FajhP!j>ekU zSkTffnW=VU>%jA0Y(!X=3^Gsaa3>Z=$qP=SzaMU5Rx(+~p16)R{b^&Mqg;REeR`YC zv1p+CplohfSUb|D#D}ygK^bg~B`Q|aZ&%Kb=vYk)d69o~hX^D_Ins<4v1`xK=(meo zrb`-|J{J`wjCuLFzKwKxB*d zJ<~3@>dr*$+-|nqTcjDMaE*n{{OU_`GEO$Yqn2Qw#G;}MfY z%4O3XQ>bb4wYRy))`(hPgWB|!eXX~#CO|-Et@_+qS3!fOwaR2#R=!u=KqOPgGEggI z!-fq+5zTKEKy8gE?i)R>kBLJlZ7#TC5vc|u)mE9dtPzdWsW^2BSuD${f>Qk0mB`yT zPQ@HyBo{zfWin+M)58j>sn1i?uQng`-qkkSL+j6&f2PuU#u|VV^L=?ubO+6mw@F~v zGFppG5^FKen83D~ zRY}dK!m84T)-RLP>?X7h`I^Af3=`AM!!Uuaen@*Z4xPpHfpfZc5IsLSPHR8|Yf7W# zky0{D3a#gxgkaBzZlw5mnG_!b(45>}$O@3=W>}pWEhoyyv>rAqs497#+9|U;DIl<5 zn<0iJiS0;U+X?#E0T-KPQ-fytu z`yFxm%4rLwDZ3RePDRfKy`+5!IbJMw=WQfwUm#_sKN4cIF)ep0;_?^bfI(#`F<;{R zgjJ?`zN7^K?bgrYCeyRNP4uj9BW&4zs*m$?CILih>GbuKrYwrCf(xE6u~g^$qY9eo zun4_z?I_WkhmMlJ#WZxky%F}t4YY`^1>zzJY*`c4M*&<;75s+KkGD!`etI&7SP`w2 zJzpm)w)q5h85Oo#oJN*>wj@6N6oH%8)_=C*i}3bDAG>)_U$Xp^6~RQfy`#4M9JeAs zq+VIvZ^sA-bk-u00oj=Z?!8a+xQ?VB0mK^yK4|6V(@{Ei+~+ZJCyf!=oWgdFObwhc zxF7lxAYNO<*7*0@yBEHF`%2BY&eo5l*eVJ8gd93lrehV2WSZ9GWZ7q=Da?=93_$D9 za{8{`c;*=q)6mhCcYvdeYdsM8UCyuQw^^Rds6ogKa&XIzZrMSuJxW(c9i(^P=z}l{ zL`FXMDt7h_A#2AU$l7t0UXc`bAAVZ(G*MUO;0Y5Dc{k1Rv(mPs-P@l@Y1Y2E3{Q|1 z$=-y;xNn1lmq#rCA_YK*x=nDL0I-0Pa=CBn1R#TWkJ$&EuwvvJ`FjZ9I>G>#Nf<<$ zL?Dv5zIG1jXA8-9V+W!`VV{T>$d_XdcI`ibF=X8*Kqv*=Q6}%E3jTs!XLexf+LM*;9oBkk73D(zNI zRGj`58hn!{f$0SdrqkGKK$JRgIM(_h?IQxg=_j-@q9<9k`p~_gN?>fcuqkl!% z``&zProAn-*ehQhZ!B46b|K45Q#EmJ&vGnDJgJVtjlqK91geTiL?^qP=wvmqEYr}H z2Z(HK)hD(CN|2RB53x=Yv~59lk0s=f@zOhwYj$A8Wr~fVKpZ z@^wx`f3(Qlful{xYBQd!Hgl43--vnod5XH-B1gUVh5#b9{)U`sa#rd-WMm{@$e6dW zHMbPv<{kr)TAU1Usj|p2rxP0P&ciFb3+{vnAd-^|X>YbbsHxA>iKf-qsCw^e+m$WH`J;!B zQ67ZIFta5cGlYCebf-gTIW1d6n&;$e29lcvVpB1TSdWnctH}j}`5v7SP&t8s%GIXz zpM|BAig{*i%T}1%lQzimk7L2E9LaseETT$%-2qjqvR>B%EybFh)-na7T40Q1{iy>s zUfxSe&!;3H($SjIaCy9T66-aK4v{aG5ryic2SRo3$U%n78BOgXFgZc&DX6AKYNcHn zeANRY&CgF;nJ{ugvS!4ik_#LNMPdIS><;zEa&k|z+LR!S73hf2?G7Vh~620Hx?Tc?Git8)J zfk>i=PuW+3hymgBWQ+0T%mx_JY&Y^EQxLqMElLwdAuB!xam9-a1CbN9PQdCT@>6p+ z5Sf~iij1VUG3M=X94~zxZJm4`V!osWB5MMJgdQ!slg+y&qU()oM-AISegy}Se?g|n zWM6*)K0Znk0mOMz$e+ksY!3Zric=v8b;oX700(iC~Lk3i(ltB7cfTWyym* z@rSPJZ=IZyjhg2Fa0LH%@JFLrS;gaet|$6bt>q^BpL~tLQug`iNAIjLp>6TZoxgLt zMWj`QD8LmUlKo3Q{nhvQ^k;c)%OI-{DU)U9oA=iPe7w;sxG8q$(EcMyx5h!(8WV|@ z_3KI3;?LxrKY*m;6mphl}&? z&tJfw_a3gWMda}PN)#F^0;q1}o_%AAgAF&fM^=cWphq&2WW;^hG^igtX5z69GA=_9K4p zX)Mn-0}dir0W2vC0L{WlgvYiG$HqtR63G-QR&DxNvLM{3vLKWa6zqrp$0z92v>5VH z1ChOuVDcxSWUh0f)@0PVZ4o5;P;Y@)oR&azDx|3@0j}ncy>>(FkjNXy0BMdp{i zC`f=!qAbx5;=ZNTr@eBN6w-<4Z);6jv2bz%h!l$`*5({f+MJf(N(fL^s{24c6P=EZ zhAG!fEN%l^kyUG71$49SHyUpief$V5C>%)dc91+WUmI?hOiS!l11cL8o#SFL5vMEMUjy{y;}hy_4?Fa z{lz^&NhCn^bh6M4BMVK(K45aOe$Cey!}Bf8?_cCyn~z4nUEIdGbLTp4YFG@WKiVGO zzPSW<&6oik+>F&JCn;TfOkXew%&vNps9Kkss9L2ATacyX3v4NA4~PU}o|%sJ#ouDd z$uP{0D8q&&tF1vno8?k2uL{+JNt zye7Q>BGq+JYxpm|u^7|Jr@8bLDgs2R>!1fjHg4QlqGxrfVs1#J2SnBci%3xiEl_)j z7SGp7L8Lk@67;A%lai8ffEM#CH5G`|(xMs}KKbX*1I?Q&7Le)W2aL#*owjrlK%}+~ zHT8KK>6@A!NAF$jY@JhyhP4;du##+zB{rf~*G+SBy)~CukC~NHEy~ld{%F@g=f7S{ zw5S#-Oc#N)wvTbRW)kZ}?Yf*Z5%=UZZg;FVxjl&{7bl4-7bjx9Ae1rfDYuF|9hdir z#0je>v(tYC5ed15+ij^-xa z51x-S2=l>*m`wjjL4Y=plhV?pe8#RxBbxip4)YXs;D`c=Z;lCJ5$SO8qe*q>rY2Gh@$f@O1@y?BRl%nYj|)|vJ%cG)ol^UbhwXw zu?vi$uslOP6&Dj-?h{o1q%%2E82@2lIgh|$_RIK}AJWKX<_6jX)+hGDNDHe_SA9g5 zP9Ck4cPqwQ4B&PPHZW#=mcm z9`GXH@HfVfNBN2sQWidc`3t-Wd=x$*h0n;F{raKo*#rR> zPvJpYbO``WECA%iiy1r;;Ub=#k$JHr!j*V>KqM^;;w0@wJs|Ra$-fX=Y zsS;o#z$Q_a=m(`@VlgBEJ1QXZA=0evtbGYn18L%dI(0RRNb_@G3%2zVU~64hrA+OK zve?oG!o*mU>aGSNbwEq;e1+c)(Qe0mtZsw0g5uf?MCw|;D&vRzv5;@iAfoSKi%4A{ zQi)4rZA>`g*D!q&QSt-@VLB~H0g)@|`HWLNAEUn? z&QbJ<7#kKqq{BYF%lYTag{p<`KiYM}sP2}Q z!XRaV7*ZBU(pp5T^RK1nM!#L$ytzhzNL&37@eElPhLUw*9yZK<2M5*h+K(p6)^LYF zq&i7WKKB5Tt&Kk(6-6qUKVs0>LFnDF6WTP%!T2c)$$2ALM3M!gO^Zn7zN$zK=x7g! z>^S)&#E)ZIRkf55^=dz&UTuswHf=*>3-aT>_f5pSv>6d|KEjLm0HYV~!@Eh}qvPHM zh#^4pm5(;zw$Rz>Xp7N?cqKdboVDSJ1kmmo) z;4lOv=aKb`n624MZN8@mhS~!G%_$w9xeJKYf}u5GV~wr&v|y+$5YTZ?xi5yOgVrij z4+L~ysm5~B1J7*Hq8iIl#HyG@q%APi76>RxwRcQi;*VZQYmi4)H$nNShzah6acKoK zU)RD&_HrQ*>*NI~f)zE=PYeHo`<~$HtoVgPZ?8s4ANMh@f<{E)y0;r1)ZbyJ&Zz5F5K*~`bOSDc=#0TNz z>nMnnG5u)`Bz##ax9LQz)r3-EehT@5JVwg5B+)@lFe|h^GLHR%sf9wp*0iRi^`bk< zW8agV$oJ&JySn3*$WWFI z_NLjkO-oN|>#LOhOaDNu85tv6KT-vX$sJ4_xr0ffb5`Jdz1b<5jYq0neR8W3MXnZT zAEM+m;@5tMiHrj{v1a=F-~Wz}J`!s5Ql`ouqd;V2WF+2r;|(QUO*zr~4T|{^*HM6g z_UbFsWA8#5(Xn<8$|tM&V5}cU{vczUlW%THNw4C4uLhHqWHRzziB$Hv?Byxy5JvN@ z0~2Kh8jf!Z1hkizd@rA@<%3bY^*9y}Xp8*x^#~3dgXFwYI5U;A2Zq)J5U?i9*&IKM z%8#Kpu8ovBf`F~Wtg~oF)?#0huSWsWF!;wnBKz*s%0I}jB>R_SOGO4MpX}6D8%4QJ zCGc?SuKfh|6reBdO@49u95`>50U<4LAny`_sq8lij2hTRU-3i9%yiOw<8~Y(K+=f3 z9ubaN-S4s)^*sXXW_|Zxi~GArL`l~*4TxmGaT5Wj++QC?_QH^McS!Y&N!)1#L|&K> zC*5O<$MMPE;|LuYjAU1M!bf7ziuJfAQFvQ4-|j^{UmxT@H_C1tdpEnS-G+?(0x8OH zGE!Sh7K+{~OIy9Pi~su*{zMj$?vUe84MAJ7A{DZfd{{Egty=05DEJvs`94e_yPh~2 z|K5N58+=}Qg}%8yD36cFjgceeY)+pB+Nkz4&z%EEVTWh~;Pt}~@EtUWrgH@nB`Z_2 zIxDWe1MHPk54dQ05eATzEH7rvT8rgKS-T!lghNgX z?T*C{i$wuNGEMkJTv4ueQZ2%@K$|vPT#Lnz#nS!MPT6MzTeK07)V5r05lI@9CjWf1 zNK^84jizM&(QlEIO^)Y#KS&xU>IIq9Bf5VCAf#vACj4hWvg{L9Xqp9!iso`d|A82) zqFxbsuQ36N;#%83mKKrJMic$4RXi0P5QA{z{12I^S&3GMXh_Zd=mHR_S}ZGmMU@^i zj}5s_W*QmWh8P%S7_OxQ_fU zn)OnN9`!z=M^%7EVWlYOQcXT0mHVn*GXO*)>ofEtt44I3Oth-wROM1Wg?lkL`X$M4 z<%*pv(UqrP0x<0|2!qHA+GWr%;PcH?UzJ}^^>OT6)<1`}ZT6{jZS3#l@jhC#{l$ks4MEh_poyD}YF@ftnW8 zNR60|b$MJTOf?}wV^{9Rs&NsixPt%Heov627Qr6-MNXKp%k`R?Ato`I1!)VjBZBa4p z8OUo6B)h9TZH-OWDu8f5xfjtg^l&UucZcm7e}_!Xs^cS5S$k8vY?Wi9hElLt$4P75 zhfB%F=1{a7TmWOhJ z^IIlh;+0uQ9M@Sl2W=qFmw4W!RUQ-bB`qzgHoe|;an_d@(`ksJJg_MFIHpIo)r)E~ zEWtYgaXvDrp;g?f3ACt84;w@Z+_wo|Y&KhRYv0ft*G5`=jx7J~`Q!^o){z=8QZKz4 zgc;nXl?2%6#j_O_xcn2zLXYp;@-1mAEr%Z`5UVSJSw;aJ+r<5!r|GGa7?+~Du`ov(x(vPi6f4*YUY!aC{II=I$d&sA#E5hYP z5Qxmu^Y+HrG3*i4Ghp?{fcOrw_+%R0ipcGC>sV3qYM;L|Af(OlPpp={Wa-HMBCWtk zod&2uNDwva^}~mOhQ|GwiY#4y2Z;1~#~Kera;1Y;ZA3nqwz5p=)(OuOzjIwX1M4I? zFMVnEK?$AbG&9I7m;Fb&ON>4U<3J?w7cYDle`rHyN~OQxjTx`tPrv$`s#}i7$Ir&U z2a&}jh3x!f0UkjLyFIf>;T?IUo6tkRlvpbu9s(k8TAq$$7>P3V&cM_lVu=MP_utAs_hVd`X~A-KKPDuc zkf?F2Ky!-gXsjvqB;TpW$Hbs5y>3|Y(~{_UaGOXwmwr3>CNSI*h@?Izt3o5cAI?*h zwX9DhCYt%Bx&Qz`07*naRE5dy7GV-CvE4rP(I1PPu@fS!TtyNwPNi$|8QC*|eYLXB zOzLA{6?yF>Y4`4v-YvR{1R}FqdNlg&;^w&yL`Got8^b}~A(8r@xme4ctQHRtsqT$h zeeTS7;)WhyHxc9CJBr4G-oU20k!1PEQizl{nesFfXiPxiHUvdXS%dCccAZ1vtM70cywJ%lg_!xmw8+#zcki}0#C3b2pMXJAb&IXO9~Q>RX~ zQIP%m>#tIE7z|ZQIk7dIJxdz(&6-K|Q{Stqa|US{Jg#4#6hckH;N+@RGSJ|D zGDO^}l4Ef((T%_Jj#L(J;Exnq(G-$#=_+lgv6&Lki+n^jCts1yOED!eZ$>$j{!C#B z4*x1|K;3b;FB-U}Xwu~j^$X22(KAL^X^ExD!6n2x&5uKfjrC)cHS5`!JB5u+ImPP>RDh?U1!<~%S5^ZL$HZrYo#D3L1hWP%U*uJm3lm9W2TKf-T@ zE2Y_+uPBi!@q{Vid0y0tHt&5@{tYX!$Qqlq-ncee*)RcR4yEKY^gA)4D`qhn}ZXPyu9y|jz@xRdBc*%v0uHS%M4shBqF zDh&eB@3 z6j^7^g4`G)pnfnqHwMZ+#mNnZj z(xgDbRH)%~O7S)i1{Rf7mmFaKuNnN2<4O7XQhrLi18IR0LkkqKDAX=yM7W%OEWO-= zXmPdF!dE1{JfNuJ`j|iom}wa5y4QH^j4F5_VE+X!uJ&eW8ainQLls|2Pru5 zMp)3q7HDyF<21pQLdrMv~nmRvHGF z@ur>y^&$}zyyhw~1%$5!mt=dN-(g?awf-U^`&Vm|x%~lSSSPq1omqIT{9a@XCczv#MtaV^2ib3)nL;>UPDFy8)p6G@_hOn22C zOge`f+godp3O8;0g6$vDZ)00{f`0@K=5()^8zb;Us8zfY408Usl{X@gtqL0&6rwchtx6vNW+$XBca?r>xVV5t}yGmno z(PKMa?vAwatHsa?X#dfZi8)1huXWP3k_tBHJ=DIs^UE#Q2wjx2d4**_67%PHhY9pu zX{e97Ka0X~-$y_SXZVh>3m_7Y+w-{3-%w@Le`>qo%+cYoXdh|e!xTXHiNtNb4!((M zRUQhV6!2y6=;OkZR&?{Ex^0MSR7gQF4*&RWZ|GkpaKiH)=MlB&6`68Odz^QC{Ji@a zjniThr%if9(C0~$A@f3iC0oF&(7q&$-*$!TJs4+^`>^z$nBND&#n;WsIBQKvQl5Xt znegq)cj0X8x>(seH3rkNDdk@`DLDgUPRd8^>|g#xN~z75Pp2+KlssY%`6s~=>4AND z?P)}5;Q>8==egGH7^W; z<^*q9*E*=PWyY3#VN)`#5oBxO<{RryCm1MIo?5$Jm?iHk{1-Y-3TvUqW zi<{M~Wlc#)h_PeN*@|JLb=j9oov&ZDBXFQ8J3fA7^w$=jYtwW&v5^1rC~^?vEb<21 zn@z%_N+^`2&F^=oH1;}bL&;=AxEzp;dcUb<{P!2NoUX-4&*Izb#ZPIZHlbH)$b04A zZOZ{%74E1~5kK`SJH@Q|WWeWM%QHo-yx&W`Py06`x?yf2DshGPTs)yrmYcB+{w7o` z!e<)sQ*6*(?_BQL%>Oh7$RQBKolwsQc0Q!#7cFR!Bg%#V|UqQ*qe{hbb&_T!s= z46fh5vZ=?o4;DVkZY{1%5G2&Vyv5OM$kz|*lz*VAw8Oxg?O+)bhYaBG`iyEIz;vwu z)N}bR;l{U?*h5y!yf(`>q>}ob)nmz(Pb;pyYf@e^d`tP{Rm`)4&MVI+yCf4O9QMyu-z3$}WLxj(A zIHKW=?7qV{Q0z1jMHbCrTs6Vm1%If-Oro#1?&wI=AH}zY?U|5Jhi*b4n=zVcNRf&$ zl<%LR*)N`X*^;hC1;WxZV;D^&p}0|R%YN3;9~uAo2LHVv=yP0CgGb_*FZ%vI1}B3! zo@Z9vsce6rV8Mc6VArFcNMdjc57LvJK?m*96MU|B#D=cbopY$}7N;V|f`DUR@KKf% z%OX^C!j^ORwk|5~VINvP6wUtF)ON9XhqYl+G%?H?r?%&C;u+73r66qjlc`>uBznpv z0L=93l!xdu2%LXuAEb~H^Q2nM>$PZfCwH#(E+_gkTK^i4frvdWB7@qO-wg?@<8Iiy z(;eZukgO!0aDCT?y(y0H7p?a0!NltdR6kpR1GI5IiVg~55h#&QZ!eCAnz*Oi3IgyV z>E`ms=V9X-&*7Db0u_w5O0)G=mc!z{^Kkta#T&d=gvOEP^8SVMMjrG`meTeNQ$@O6!s>7$b7 zgd7Zb^&n=3T3;C)SFx+ZMmuCDpWlj^z>LE8+Qj?O9Wyp zDEEH!!43{JpZG_MK|BN-?)ZfT(LC%}OU2>Rh!&_}y(DkrZGB`8ZCOi!JY*tp<%OQn_W%EXXf4Wdz*K@&e(?5L~3KuI9lA<*@?@-|MP+L< zs52CZ2phaPW3HQmJ-N!@x9@bD z<7=j0dSIn8E|Yiqqe+xlhq@ll#^v|2#-eT&$Mf2Jr){*g#g$3MLfPJPypQD3`yg%` z=DRBaHwIt$+nr@3Io`}jfKE%vTfBR-1wZ>(G&31D&TEt?&PTPIU_&dP!YAn}7R@qd zdA);ROzoZ05x5C1?B$0;DwIx7ihonfS&!6F=)ZE2YuFXiuJ3Ty=chk}rmyR1-pH$) zT*k{BxG$_3x6hpS82PrXZD=wn4ZpZ#ak%a(K;6^J7hQFa`$p6LmmN><_G57sAIvD4Z>^TRTIqJd0$`#j>~~Ssv&k z?gGzrV}}QBxwx4R%q5nFT@S1*t@1w7E@g{zkFF7}b4s^DT64Im_$s~&W{2~mAEL4) z1O<>&{DrLwEGeyUr}W>ojBfaJAB?-GTRLQ{K;IJ7CL_eAIteLzaWtu@NI{7s*{Wvx z=!)Jl)v1LcqSum8%`_A{$}D(pT^0~#vcz&v`#w;GF=1q3xxM(3+?3h*=pG8KM{xxl zZKiwLSm&=JrT)9|Q`ZOp-x*JsF+by@X6`lr zmeSZ*nZJhC4V&O;CYP$+&Wbs*KH?2}KHife_Pwd0d^NEKf8<%!#N9>w-XObG{I1#E zRJIeS7FbyuG;3E<=SM+Hd9`mZ`xd%<@xwq^*neU|QsiJc4R1#6utLVap_z%JTDL9m ze$&su%0Nw9fl>Gu>o3Dt>a_Z||=We2FWmPEwak= z8J?DjN(s`fmt7xJFJoG9ElZA+318eZ9$0Jn*Gm|&$MHi&Dlu7^Grv+#&q{eo{hqun zESwUbU4|rng)@4Cp~Mn}T6)oS*#EG5BLJ6Qy;}FZdbyzlp%fM=X*57EsL7!BNme8I z-u4xK{-)BTBmVCZWkq?Rw&cF$tIxAZ>LN>-pxP*ivXZjRQ0JOUAOE7WupGCr+&WvG zCa6C|)F#>*zdoT7gubuW-)F!IG`ma8Wc$Q!4JY7c3I)VhkG+G=@1|r^Mx5=<_V8?N z{eIXA^hI{>@hE|5R=_66ABPE}!a4%LMmZ&0*RuyCc&B354?tDKbi_9WJ=|txvz(uP zN7Hc8$qL{4JO1c(QFDtyuaVi#&Y0i*N5O|>sHK#?*uc1db7anw>4yF{c=hE)7do5g zEi2k_bdv!u+!eXrxq;z!I<^2o=Vrdg_eskb;{6b_FDu*Z`*I#KE=_klV=xNMQz=8G z>i&W)QK4b=p%aZPc(Kl;b-7!>E}?FD!t0Fk+T`BQ=*$}-Y_e2MxUlrQ3wg94Vrl@m z{F`AF45%VsWliM=G+LcO2!VcYn?^)zg85OkcW63`*Ecp=7uTur5hJ5{yloE40lU<1 zmM0o{?~!PwGP#sbN50=xzXRb`2oVz&+G|C!jW{FZUlK+;1YQ3aBeVXPq0HQuJnc6> z6KBOV2jr9$e?DMnMNd?uL!9$Ou{Ddwpy!ZdFXS6WVeH_?!-mR^XHf1EGQZ?bR3R$i@+#LhqU_XE1pV<#+EV7_b{EDN5?;{yf^y-Ctje^*ox*5l z0~3lU2SFKbv-BFkX_t>?VlCY)Kc`%1HLH4~t?*ta^&`%6OI=lfo=T2|#B(m+?5d~6 z@%$(j{cNNN6So_toV=;mBQ1M56QyK&>UG!Lt&$s8FJ2t#S=^bolp~ZTInq6KFI3Vr! zaNnRof4fjk{yb{2;bX2&JTX7BOAdVHsxm{|M>{py)~WyG#foPm6w<<+ok~9aB={|H zL8EbKRc%#TCktUXa+u=M&fH^XnBbFNm}!KVqQEO{Efikn); zm?J1eUAZgFYEz?`)*Mpyt6ts~r^TRx!HASyL$xU|$uPE53ZAm^We>A=qxJ44wgiD+ zz~T5_*sK!H37yP$^>H{Uxpmk8wM#<{7|amuD9Uo^HR#YzwqBS#x8eM z_r4yQDB!5V!~odi&ec-WhO^k{18xeJ3V>tozhCLHuDX*ho|H4XzoMg@MZ+><6B@Wp_gz*l8kYK-erY|B znzq+H-7;<}M?-NppKr5JuZPginav;cDS>*+Nsen1<&y zN0?SjmSlU5u{a!h+K)0|V`Imsq|oYpKVCo;`+-uI4k-&XesSOCITV~N+}PwOouStz zd@$`vlAjne9w$<6m+|`_nz?8xuXC%GrV>!phvp9=9(bvvJhw~UHZ%<=bU(=I+2S;V zk1&p7GvG!*AWkM+X(lmK>X=+}@$Kyd0%mkoeLFaUpStuTTJW@XJ^f$B<#_|<=!ovf zUGN>fw&Vn-$GpRaMtqQ_0BScnHw__50Rg@If$rls7+d#ed(UQieM&E3mwDiJ2hpO7vICBpt(BZ9<& z_si-UabqBa*0^W=e~VB6vR*HLX%nge7;(>kzeRV$PB$B0p9P8VC5biJlN$ntC<>QU zIYn)UGxbm;_4E9gI_rjsZO-sci!-L32F>wd^nAq`RJ8!>L&11&{aK(V?5wJ5}G1@AADl>)+)pFR#{zls@o5j-P&hM%6cg#>08Y93_(V9qK4#85qJ z3kWkj6{8p}?HAbDzWP6pGYcuBNmrz=VcY#&2KA6#?aaw#Ywi$%5^im@*|7$DBdG|H z$ptCmp7OEj6rI@`(&*z1eLuxleu1sSfQZ}wHOi^MKS8B+!qn3!GJ+tBFOujx=*=|x z(_$B^7RQCCSZ?3)2~T-+ZZCtq{w?O8YW$A{wtjtHf)OV=wm`=?gaSoJC#?{VPkRM% zpA$y?VgaWRjT-LdzeTPUA5J~SzI>-v{!$|ScwM{_!~PisQq!!uHz+A0T1Elm*GN&= zkdlh>P^V^S77*WGKBXlGZB(YIwGav&Y&xcC!OW^>n;0;{AmP|_Y89dGo6XM7Zr2pW z@38eQA;A>g7I5P#N)HF z%zs|*>m^xfl$$@mN4wod;DZ4jhC*^lZMX5|i)@!JeI-XisXw^myqj21P*A-ntJ?vS zbXYRTg=ldmBP+(z%0Ytg2&S1z1QD|oF8#7Qi)AGu_Z zH`Tohwl6m>-Iou{LiBCG(UcF3FDeD2MebI@XLqai2r%+zSR0MI(cBgy(n{U$_S~a; z{Zm_7T3XfesDfhUamS@JpfsDK49DRBzjiF`_WphmGM-xGf-Zz^vD)bbku9^;C2v-Z zij17S-TY(YGylCZD@A0oww7%GVD#T>Ik=+ni_ZdU)&5o&$vXGDM#71dGq{pi7AmZp zy{Q3Cw<}g|3aC6+)zxJ#_jBWb!bH&t9#J_67+$QMGd;Qo<*~dg!yLM;_|$+%Luc=1d|JOd87C;s%q7fST@+*Whos>VAw@x@=hk;}2oBdmqD zyKEfQRVp5$|CxhM5yFRE(k&>E9swzEkMR!V$74z4v~2OC=lLUnZfP^`dzknezx$}s zKTjQZ?KhIwJa)sOIW$`b5IF(i!nOl`i~m)b?SCqR;pF7RBxjv{np=U_a2leMPxEiK zT~&8t;O6nN7*?6E4Jk2?@$CKMY+id&hZ@U}5`Xgi?Wt>){UCrhCZe`|ir8u#jcFi( zJjH)XYYFRWHa;iNhmn_YWuoaKa~q3b`Cih%qM;w40Z%&=z_zhv7}T2ZjgN^%k>q$7 zq9CyMzm(LJ2!;}br<9YEqrph1BGSW!r()G@rI@d?oGr$MmrVyyIyk|8Yc39mMuyu1nDCqBl}X~UtV$g{kMbA1;c)z5=T%Bi~yXeh5(nZ zp->VNimH@^1TUOuL_Y!73*|wCG2x+9uF<#Mr*DhBf=w0DHG=iH)XzEm>3*o*ll@!! z0P}Z#b&a*Zi-LFd(Z|If&IRX3E5Gp+Nd*MH5_|Rjv5)i7yY`mRl#FXf1|JMekC#yNRSjrWwSD0=)ap(L1eV8 z`e0zT7~33H;f5otUKuucO=E%kW6tBM#XwWLy}Yq#@?2ejhp*ax{?2O!av1RU_J6I_ zNR`kB14TR~IoW7x`a4P%Cpk|L5tQ%V+G+)Sn&eFA{YEq8!?%~+KWk-)GsrgYAJ2bw zu=@@5)cR~2TnML5UfkZgL?${EycVktJMSDqDWd=9!|jftEH?Y3V#y?AWsv|D_BHiT z9ZIUZOguuua2CDxHKYm0Hq^-&x>V#V>dFN&hkahoZ>7bgyAhlY52Ay}?zOi;Xgos% zzf5~1D`G3v-$_rJNK37>5{VLPz}mHcsEq%cHf1ulPjrCo!Kr*jJW6aLax;}ui}%;= ze0^!BX~RU-C1~n8wVwn6^&UFm(85l^=C?02IdNGU5{1AWNgq91kFl$p z-oLWwpRS4b9Fe(Hn_h;7Dq);Ax1i{|(jrU9 zGrs|mr!&2K_uy8c!17TS?&m{tPUX^&GX3f*F_P{ z6~)DF=DLe>AKi~2|1{aU0h4@*1{`2n_|YxMU}54rYoG2)Q1~16#B>W|X+!5#jegg? zx5ap|elopj<5XQ4++uSJ<9ywZx#IWu4+j4^i8ciOb3dJ^w2A*obE;uh9p8+kQ(7i~LqzHL7r<>1JC|TBxGcocZpMk z(z~c)ZZlI6PH^wW7qk4jJa4IgtD>`+878kfovv_~Z9Zf`$ya9rjZ#W<@YvW`ScTDF zFo&h!otKG{j+Z-m+fFIAEx`UCV4CCwN|No?+htPXvKC1H)kShbv5#ilYZ2&FpbBtHH|gAAZ#4)3YWU!2l8ha>b=I}(goK=1D~Q8 z^);x;;{pTK=BW0}!(fGnH^Oj5a0SbgYaUi7t});4u3Oa3btW$#b;dV+vteq(?EOL* zog*!_##+0FItFL?713W5|L+r&7&slFk6Zvs^6CeaLBoQr0`3CRPAm`WhKF;2h;&zM z8{@58@uu$RX5#SwIY5;l*xdUcklJ+E^Sfo8Lru9mbx&EDAZg@q21RtX4dYZyMl5|b zv09AK6j}L@T17<|w+3k%SI5cR2JDr~?BsJ<{odaOH<~Ra9LwjBxF9MlVZ_`uDOHLv z5)s|W5Pw4uIFDXLH&_jS0as*w$OQLd5w|A9MNe2SJR;EDfnfv*R-#P-hifeq11kg; z6o=9K_t~@9)01zVbKRMmGw;YF;jkd4X5*;ru>AhBdsiOavqt(9AYEPCnwD8tWKj1CIr z{O@fb-*}SepRxY#Uw=+)xu;1nz(_H>>^8-(`8sia>`Yqz8t)_nOAUR8fjCS;5iadz zlDpT;i6PGy5#rH`8Q;lvfQm{>2NQp?g%@^bN{R&)?k_G3+7cC>>|LI3DwECZGMX73Ng7JhH-y_Kl8e)5AKaFMjGNpcP-Rsn*k)$jr z;R|$8G#uhUQsnBq1cyVAJ{nb=9^dg{OAeatpn_hUSTGp=r)iTOo^8HY$f zg%NgcIJ}MGeYH!srKor=XT$7a-Ct#P*FC5nLqeRnnf`F@K(o8a?rX5)M&oPo;rO1$ zmmJfy@T)XZ@;%Ki9W8$~{7xHMY_==|4Kx-M6>K0of;4Cb(UNuX1PwMZJ}?evL_dSR zWmTRJsY2S3gx=_62d~uCOyK{>_Z^s5QxGmdTv$*sK%DdYG19^(PKMEy=Mz_^IjqEl zll*y=HSpI&tubt0#9B3m4pX!7k*{(j=m{o*^&RTz{V)NB6Z6~CUJOOzxi6TA3vO{H z!u9ioEv@m8fOjK_duj1kK4tvBSu;Nd-Rh=B-_+v*YV9@nKg$eNy0oB33q!fZp+}bX z5qj-YFi7Q3MU)0$f(7M;2;m-C4e2#Xxr=U{@W1-0qmYGDFKNYGZvHqX56NTL3|=alo0AsH}vX{^fEPc zpA-dwb8j4b?ADvoXhK*cr*|I#;^)&)YyEJY)|jj$9bgxwh=4{~K~ctkZbSqX^EmPP zlEh=2*`||S{;dEjpqBzrN$(cDbv)uj1fY|rh%6@jF~Dy5lBA@@XfY85q5INch6V4? zqfWens6nSF25j*CU!J!!GOd~Bb_w5g%HK@>ef-5|R05Dw_8HZ;g|e7y{8U@V|`|DG;3Vjn(Ts&MJg z1Ncz^wX?ph`^l{#$6%Q@z)S)n{Qa|n82jmed?DYv#^E!;l$Y82gn+H5)$*UZ5K#dA zqoKsqwo>v;e}q zzy;KPgUhSMl7tyI9vEfKbjP0=kQoq9&(b5NDY#?3$J{^ofAjV7h z{1Fk0XVW3E=${dX3sd|P1I5=Q7ev5}spCqQa-6al?;gfB{sjYE7${y1jVQ;YE<}TW z&-EVpB5M}|2zsI}-i;bM6$Y9gaAzE_lG+z~e|r5OK^%p`jl6 z-&23MDd0P#3PSZKhI5TGTrLWKc2K!-=XGK{nt$JU(u{HIfPU^6a51X9b9 z7ZSq02m@{>g?sw|!dFO$J)}SmycE68{?s5b!$OGZ$ts2Pn)iT0O2YrB8(2|afUwd6 zB}Uv-qHw;k-|B6u><7*oT$wK;D3L_Fa&EW@zID#QSkXaw8s57wvz= zYx)1bcwsZbydlXNzsuDaRr&nhUhD&4FM$Evtxrc4Fj{8dIT|{X%^#vW`^0wjN&jOV zuz-c4!A8`h1^9bs!(!n+a}w-!ZiC@@ez)@+ow>3o{-=1r&65H4UU4cz;VcKsVh`aREn!xc7&I=e@yuw)V3$;jAEWq~QYyO|L+_UTbXO5C-y#ePK zy=nwF3GfFdukhiv(5@lT(q_)t?6BKY*z0$?V;07RCnR7MX$!`Ng?){wP%R4tzOqAD z<7~C7BF#RqunZT;r6z&AC$AiKrbL;~&gh-qJkKK!i0T2=ix~s3$6jV$dseR^~?96jLulI^~ z{4_w9JF`->9e$<=Ya0~JQ~D)Guqw{G({U(VhD)UI@(~y`R-BON{gY?OYOXZT`|8z@ zhL+6l^;RQf4;z-+K3}a|RmL`fXQlDJT&?2A(R@Y6XpK4AZ>ScxyAv&mYUhK=vL+6~ zas_$waMdO&wOX4O{9n`~vx4pX+49uaX;vdDe=A2X8)I!fk}h1TSN?OxC zK@zW>jjcy3JDDJw&-TU#;qEVPu@#R^_ujnbS2RLG2FqVfoRl!kI@!%q+;MbYijb;CIE`wVcX$}az`5RQr>|10NICS1 z(h^8rb?KP8h5U#;sR7(*FlvwnBLDYfU~%M-c6W0V^@qt))lDk(~D9?4ZCa zo&8uH@uCGscHj1F==|#G?yAZ3ZwX)PXXL9Pa^)zVdL$&IB-D@1F?$M?!qgjIO&-(y zP5URY*{v4H>up{}Um>UxoL5Q)>RH!6$W4Y{cNO|QZ;zI% z#pa4$lXlA_23G1^`IkY#;nuv_3$@6%x~}_2U(X|x6hRmdc5i$JV$@g??=+rm8$Lf2%EnNQ49Tf(B5g9bSnD~m}36}Omu^B?q^QI>sY-~ zx~k(6;jtrhyMQFI1)BdtIg1n!!@PN8JTIP@fU9jIVp(n-QVZ8UKJQBhL}aOruU>VV zk>!#->wUyCS=BU`WuFR=6QU}`^ODa*I0W?IUob2)k8dWizoZ~u3DL@0I>I^pPT`_OU_{(l5TwS9>x;I|-v=zB$)C@0Amek&=wldpJV0Ju0SL zN=KO?@qn*`4*H)EjFRFzZ|ufle&x6u+u?YCT(gBgv(+Z7J(QPG9%nwCd^7O2k<^pB z%3Za-GyTBWptts7tC;e?IRZWqid>B-V_&3zVEtx7fX1y!XtAMsd}WT`^qHYbImumO zgNb4uniio`hUa9!sJywS#vV*I)H-xws)#qmq3Q|ilSBKmyAMrNG&2OcUx z!Hs5P>{M)7owpAh@6!h@YJJHiUbo*YA#G^)g0heQGCF5iceFV1LS8{L*O5unU(|aV zLZ$`h1&X;M*LaPTUw<|hGbX23W)m4NujtLcL%%%@QID>X^qydh({Ugow_Aw{XPxNJ zI4IdJw#J5*lw9eXNL;(h{x6WnAudS)5#^3Ok>(e0?o8zUh!lbO1seWDK1)tAtT6rR)IX0O+$TNv4!@VCqTrOVwBr6O6}kSLtRE}sYm z?#Tn>@WU$KZq|>BCs?(eMqr<-X8YJ>%HA>8NtVcoJ>LHqj9dEl2Jt>SZM%cZ^eI8QoUO5#ZvH}YG?+H>twrVj zO{E9)JZH&Os*W43YHt>yaHcrMDj2Ihtz@T$^SkNOI(G{S#Qm!GiP`Y+<}iHn)WNoj z@g|iy$Nu?|dSlZEqpPMm7Al(fA-V$cqfki_~#Pzjb=>>}RT$IG_ z($R0BvI?|=Tsd!2uPLGlgJjp!e*1|GD1{8ogaIqse^FlMTXL=BuVoMcFK=|EF5zwY zP%G-bKGC1Wp3NWxZ0z4*@GG09799KJYvv~8z2UOSO3DoVo5e61+0tN^38LN8M$Fb? zmt*DIFFKvDH9j}*(K;_Prp}6Ql2?E#+Lf=nw>GG@Zjy4bX+o0Pz= zGTIe<;M$g*J;j8=@r1@z1Fuqnn=-RD%Cvtp*utQZyP6fke z;z!VJ7T@!DQtqlX!GFkG@q5zHr2tJZMRzsB=^rm^wfO|o77NOCzf=6$RTD9Cz|b`j zq1A0?!C}5A#{DCzP|ejxY-i4?AYfAZ!PfncV599ciKC-qk(JQH`1{6oFOo0Jo>J3S zEXjSL$gU38d7fJ0|K&^4u>2H;7HAs{7U)11u)v%;yIoOHkqpPoNM7SL|Kn0H6CO08 zJCQ}zzR$hndY$=PMy|}SLW>@^XG%s^IZG4ZG{`@!f)%W-W!aMR*$qveg~Q{3d9q;X z1SuuC*cW4vmWm0*Lsiw=c+9bkc|n5W2g4&G+eu&g{a@lYVvU++}SA}Ie?~}YG z)xnL28(t>mgLD5}ezCp0xLwSTkU$TVDCA!>n?3-u&@-vIbgVcoec~ci62#KG-F!l1 z$MK{zkv8yld9jFk+~jG!Z^wJkKu4JRS}sD2Z>LvBmXkP7T%vA@+hXTnpsO`6H2(Wg zJn2h;kdHB_o;Kc;xD5Tih1u4>aH>Xt{CSASJrv9tuf0$i@KZd_LvCV5Xo#K2&Atp- zi^;CFDyPM!`K*RUlrI?|Ui)#JbTF#2&EtAH0CQvWl=-l{o|+QjDPDOrTgPc}S#qmW zw^Ms?AQZ^tj=%59h6wQ~P@V{vRwfAtct6EEUpRy}p%f2iNZq!e4=PS6<^&+z z9V5=YW-{X03~l~oOUz8PoY`!@P4c9n-QxIBh>f)#?OZVZg z$?3s706ynVToTJE?ztoGmwUl8P)U1HeCddx%&`)fEoyf4^~&R@t{ za&hns0hmYKdaq&ZYO(OdJp#&TdTuzM`e>UtO0?0tGf`MCtG}w_w=d>x230zoT zd^@E0_)@983J0wrg5g1BR%J+x=+Wq+X{ln!I5Hcc+ZtQq`Z933WK)gS;w$aJY5?oL zY4i8(b}2JM;-3<+;l?l+a@M z%AsR>y;Nrs77>Be9tv;U)2h(CbW==G#HuiNiKuVWnv|m08#=`}mLhnWQqq&xjeZ;Pl~g^{J50NpWO!!+QS6Ic8oJ*+ zD|Ed6c3{K(5#KBOIB~Y|ez#LoyV!zUc!BkC#dz#;LV+kBy-fi!C{B%?u+kWTQfHO$ zdASf>V<&lY-7qprUaK3Rs&3^sshi)N@*^m(+PL`ibbF6lbIe*lM2>A2AREQ3bcBF#gS$7du9ghPly#YS(B{Mk0y&zY?}ccn^|w=1D*x>pY=dRC`1p#^E}(%pTm4@02=B~-~A z+8-A_|Bjb;1@fr)5OU_P#z(+Mbrc$*8vOGiWP978V$X?-ZCe>%b@ld#z9zZ&I3Kc# zA1IPj?&m_iMb`8qjE}31=Lj7El2z>XzfmFd(umx8@);7CY5^Z8sS|N%Y%pq{R)cg|1wt1*7(BKf^b2{NZ)*xia#BT>mNoV`{M|g={-@&a zt?FHsbW}~^)E4z(l25YYuDTEDG*tC((L{!$x!L6>ZzQ#Jzp`^)6e(}g*vzdp?GFch zLLKijY6Lu2Z`2VsdlIB?&(8?(u_W;SmkXdY?ikeb^T2O3Am`8HUu7IM!HF<%SSY#V z;K{Nq+0e@61kWSFY*po_@H9bE%-LdwlAhb+&;&CyC%v_nr%?dYQsP)haKL@e`UTpnq0=8KzVaPmmm02t7{~H_atS>$v^7 z9%6LeP|KB%_63-bEqwfH=GRRSjvxn?Z55KFo2zHkYujfX(~r)uR{T9kL}D}6qnuVokeTh zH}+>W*%wrew4K4eO*6ygl%i5^_sE@(d}c7IEuP1z^!7sJKNc;Ag9N;XHs2-r>(ptC zy8I9^ymU<$0{5|uvNp9z`^`qFHwEr#ko{(fqk^f5G6cm&VnjBqGr%kZ3Y))$8Ky6{gv8rxA`al`8O2;TAAt{*>X=d!vd`M#yigV zp5aId44w*)qDJO-b$OhqeJf_C;&t7A_CnuId=&~XVb+$JT6Y&9v+|3!T!=I+oz2pX zAQQ5#>V6JWHFKJ`L&C!xUGOHK=*!SDS{{6qb&-ENF(zyl+ge6ZyV7Ipn6)^V4mpU_#j3 zmTu!c@|>=iwh(6UL~E}}f#nBzwD2(R#4zx6JeWuT=ly)YxDA6jn8NV3B@!-0+!TzW z14c4F@Tgx6Und-|W1vIwLAV!#Q4}}5w2L|xfd9-+5XmDL2)~A*=M`3MYsCp?eSj8^ z7w_7)2;m_*h#DZ2?UbD4%bSJmTsoJ~E8EJB(c{DWmx#}V#Yb`~!YVk>1c~}PRZ3Hd zDh^6~U}g4Fq?Y$c@ZqPG|GiMDr9x?WYeGGu5*xOw8&ks0l3k6oSSPuus0iub&YspK zMRSrPWL1O(ss!`g4UEP=#jiA425tU*yB|9Q3UO)!^hPabfI@Fa^0r*$dD9!xx%MQU z23Ztyzs3zl_01+C0iuq?Hi%5?dq$T6H<06$?Aoc})(G`sI;X{$L>@E2>t)H8f`b(u zpM%l+$LN1tEY z!*z}hEO4yPn{f#pCP7=PrH9MQx4)RRZX?z^)xtf)s5qe1)jM-ckD;{?_qfUrCA?k2g0d{u29%3U z@@dVvOEI9j%#}<1wcd!%Zmf8~f_VUH%VPNXa%1=LVIU{wbRNHQiRjQ}`I`QASF#=f zKR+6om3qS&zjR`pD)Y@xnY2Evv%*EC2}yCPCnFqf$?EEmS&JT&+Pi`yqu#0LYNs() zYC&q0wk}>7k#f=`B+GK{k zG%+u{U8Db*FKw@$zkawoHv7Y7ZMFrLrUea*a$9dxnvx5O2$>7Z+rUtBsWcOle!|U> z?9<}-S%VeYs(Bjhj4AP`3XhMeQblH*|5N@fWcJe+^V5>BJr+^w^@=aI+|!1Wkg#_Z z3;ttS1AQmtaXRv+k_G|6c+8jscI=VYaXxGa5<|Lfq+FKEgAyR+S2(%|41C25x}o$Gj?URW`!qEfb4 z+CJ&Q`Fz(_KQBK~B^%C0`xBdc8?u;#eX7)T9Ptr~?BP6Lv1s{8-;u99CK1kYZINTM z=z6+M+QEvP_<}@*$fdEN;^ezWc?_>h5rtUs;xl;IGCB?vtJ@&Te7+1WYnj`pv0pg_ z?P#CZnK7C*u$otVYu{O1D+!xa8^$Y6ZLmA91$WX*I($buS~B0nHGL4&R^hD?Sk+IXgMjU%-BeL#QUIbNQX?3o~ftOS~)^;roqV z5;>Jg(umSb9LJbJ#nA|@?R-qcCP=+BS|E+l9T4*GcA_{mzD+5v6ZY%F$rLRcglCss?L?Q^4NLz8jRa(!t9_=xt6oILF} zg8HR)z1RC6tYp=uiya}%_VuwL98~jTnJ3uPC_3b-w6@N+#`QH;^W~*O=l+nmG}YY) z!^=kL1zO&0ZLYOghNdQE%@*xLCP zndU_iWK$c#G+K><{8Pl{nBO*Q1(iG8BuQ!_QD=E84H}^vzM`L_oZGdc@pn5@rJV3> zmnJSmo$i)R+NqZ8`}EfSUu?Z)Sd;%7|NT`I5EK~QF}kH21V(o^NJ@9760*^wC8edi zOL~ML-5@Y>gmlAy{{HuUKf0gn*yCN-aqjbZ$Lq8|8mNz~c`B!>VW|esUB-zQdfUbq z+W2E8c)-?b-;~#3x+Sdm41qwvQVkxub;vvL!dphiASSAsw8 zJt8jLkZ`Uhby+C(u)4`&7_dw1-i?c!u#{Jnxlqh#JsT(cA49k7Br0?J&za(e2HusB z=f{zv35n;9nR}r~4F6a*;Uz8Q3Uy`Kb;-(r8NcQ+X~XsgZUQkL0WFByaO?rWQ2MK# z+nNrTcXAfle{8f~9!Ah7AG}*1f`OO# z3OH`Rn65V4LA-TjDo*~fy=7t-v)+FnsZKg%0+fV~}C)16L#^`sl-*xBf6Cdku4*YtD=Lnd%`X}=_(ftCAs+{DA?bp{Fn4-iFz^i`*` z^`W9@ulzZJk(Th!+V8AT3Ge3|d0%o#clmzc0Zc=4zQCJv@jujXd!L13A6>K{2Dcv z22BBNhB4@*jac;Ec%<;nny+&nrcY7?O+8wXrs}kH?04RO1ce4e z_jL_Qlk)q>xXcI4vmKKw?a8rLx&FJ%##}$##)SI9HA~OpFgZwMzHXUS$|jwtg0*%( zSr=X(ZzoSm1Y%|_v={|%b)^TIiU8?hX6Axa0;GQBLcMlP@Mmv8MyCH}xOE(zU#`PaRB zYWL{gR|K7B66^UMEJn zP=-quQ*G(OTutY1tQ4neE*QYz_h8C-Ce6S$P! zIUCq>2XzzfF3&O>gQCn^(7h-=?Ja(tyQX57anG4FWc@lf6ISULJiH!fPIdUT&nK0o z%KiOqn`-)`bb^~u`ty!|jAb@M5HMoiSkt$$p{r?ylF$7kig2~4p*_{-Of&Akx)&~o zfo^l7eY>g4!?Z3!W(m#YVTzkoy`>^CZ$9$*7&I2}?+M;v4PKYx)hsRe2!=0%{oAL)W02ug3{jm99fImV7*o1P3zS|95QE-HM+F%R4cpo-avRq4ScMyzYCf zYOH@O&|F1#EYOTk0M3?MU6!iofn@S^J(HtedvM#m`SlmyfcOBfz1~`&>5KfgX6$~x zMq7{2#FE|h`6ia5wOE*UoH)~~0xh@tspfUy*-qw{q=r__2;M%yLYkui6aAY1(lj+^ zcJdbt6H~+`ZvIO{#n!Dh68a-V{Tp$BAqnya(&#}A{uY>>{KWEq|1t%6N%scB(sF~} zgxIO3ChUgvu&SvINRvh{E)R)DU5Bh21432+++jDR2{9-)uTvLDxLx!#@oPWo z?8nQKRWmDt<&SzlaRZDMGI@XHjn;?7x}fb;ju3dQXCujN@dKPUpA zx9Ji-^n=W|K(2BbivCIc5(nV-Ve_alDmySSuQ4Hu2QUGgHG4 zx&UX1Rn~0P=oW|<*7u#Scq6gAb<_&6@gT|LH!-wFs;c#E*No>+hNM#I)w+Wh#g_@h zZm9LwLQ9Gj(skIM7Cqi0E-%x^7Ma$SN9KhyHuY_CT{-9|wn*#e_cL^c<)ZPH1j&Nw zH`3W%@wZOK{v6SUU}}ml>>m2qISLV4a}4am=K~NefHy5k76!tn3Xt-!NzkEler--^ zjJY+iCldEf@Zi4gtciU2O);TkFDB^K2?v`dT#a@fmk+|5Bk1^FMo!}r=Q&qSKF7@J zcG4#T%opWIbb^Be$O@N1g)G#+Uu5TKqQB`duX;!1#(B zEI*M&0fdO0Har{42ui)llv~Qlf*&ln<__V#A2Pq7OcpAbDU(e87k}x^2yE{57GUV; zd*{)G#sw@zxnlfTCb6pa6M4JPo zC<(uMbH%3lBX957!xn4b`lFAbjEu~7o1ZlUiqV&vS(ndZqZzsPI$2r%j>lb z%~*%QV9~#?nICtUn;ChXzA!6*h;tTV4IHqU5RA19KH49za_|0fTuLsYLTFsL6y&2= zd+!>Tp&3IC+G(Ng{D4vj2Go!w8n(&ZYT!javJ}^UfBJZezYMdJ7}ng?-2{v4HD)$Q>|wAlnr)3f;UCXRit zZnRc_Zf-B%=6a&eeyDTS#culIMcu-C#WmylDDEUiwLz!mRR&e*o#{Q1z4%RC$sM+S z9|M_Gn8)PB4K5vnT7RPSz@CeoNAzz1cg;1%l?9C^6Vu(_6IP>Z#Fl+z=qBOl&u|~V zu6jr<%Y+(IqsSD+c3${8?{h5KEqht(sLt7ZC<|Iq0y)0nPFzi#3-GAbw9!#?x#$?l z^JQZ3vz!+fO{kU0GQ6vhOhvY+)9}?YA*lPfP#tNB<8>R&s z&yO^~6WiHWr7ElMexfF%wKmFm1Tg^37EVzp*OT`-T6SVqce)l!SjUW~#;M)o-&mXB zwd-sP#^=DsL!jG-bah#msSA%MZ9PG5CPz~a)hB9%f2fv2s7Li_i;y+N_G2>z-D1ww z8GsH?`!%+YV{TiQu zaSQpsRXl2ZIZb^>H}c(Wn#I#$j}!FVJC6?{7|h-ralO#1&c0-mC3=FVz@;EE}|A|!2d8ZpY32v_PBt9YM3|1%23f~>K+Tk(j;sFnz}V6-JRSN z;v6q-$q<2+K}X_?y}<#^iQE!~V2%nuGC;KS^;@YRO=1mX@RS*3Se=hoyXSp5K<2E$ zQY!q7u$E^==FqMp@Y$n~&^Z574Igxf{c$iY1&gq&6$YR^Gp!5v9C7J1n^m!P1=2{c z2>w$nh*4QV@Xu&DQkf8(7^#?~|9W|M;%0ICbnwX_G9Q zk#gew+=E$pf;A2mCXoBh7UQKQqLyfVeuaTCWMnBIQ+dFuFePm3)({o>!nv2I~nu})x7$`hhiZTRA`Pi?=7mZHgG(PfO7Ll)*AJUb6cnJQ&Ss3m*G-YcnL`Ve@sYd^n~NL72SiV#ofynv z(et1Wy%F25@cTnV{&&&N)!20RB!^0dIlug#Ee8l}bVrO1bEGnL*&+00ZO;64*qWSh zcYEd)zi7j!)@*T?d^h*?D zB!vtZFy-tmgcvLck+z|$6MIhSxe09Rl-Yl?3O%+r2TDtC)fD? zU`hk_M3&TxvbkSbu9y7C@WE#wTH;qylU?rfKF`kSR1IoXSqkip(O-w2L+T=+6lJ zkR22U=>oS>iNN$GmSXekm%)oCC`}7Ek+ORSFez6qMKPK9u#tO z-XJoWuu>LWk9gVER5iTT_B#djIqVDx{2@ zoi18VSawR*Zzj?KU*lehBgNskjA*yL-)>Xz@>qs`CL$s&QCWA|y4_6d_xllQFvY}` zvM#Z3n1lz={UEfFcm|Y^pk4me(^$v{-RY>+%t&c|{N(As$Dt1UL#dRJ_pXeq zk3#^1HoTzPk6S#pQgqC@yL6W_VPXz7^;iZ@T`IbnLgkZAds>5D!oo)8Ph_$65+Eyp z-OcpB;wrA z%4qQp)gNj{WJ?;f zCZEnMMdIi3cWwIT&iiN+iUnP%ZvKA<+!+U^sz2Yi5kH-|h33rOMzG=_mEx5BUP~+X zE8mi)k3=Ovb}?W4KBgv`_Ic6Z_}cAr|GF!sV&5p;>GEy__OZOyCvp20T7+PpF{bIm@S?rZT)}B>9^!#u*^Ze3mTwkK3*zcOQ`>%&6h-T#a&ZEL6?Rf_= z5hIRXH)195G>#qBu-^pHI#_G_oG)rs=&Fs-dA@UUPv|4^pE?8W9I3|IWp%MK3qp`s8b5(O9n z>yi|8OH;_me_*1A1>7mUlx})YKB*ZgGp9LaWt|_hi5lb4Xd-+fL%ve`!lNFxx+gSm z#kn+v2Yo5$w!V5WttDdrX8Xe*L;n{D6hLVkj7z`5%^}n(t}}{Fr@eeGyQr4hLO0Ba zgWY*+acSBzpI;99!__?&u+B4k?(Zxvl;wp(x!I8pu&&`Pr3kf)F3$gEqfaFcG%1~! z4(J{d%+AaB)%CeeZ}{vzTJwV+X|P-eJgvGbVh1}$ZuLRgk;OiCo%z&Nk!js1xtinY zVFpLeQ6U0s9A>KaPENY+LafOF@YG5j4eKujG2k|Wu89BuKRmqLek<<9Jz0?W>6_r| z8DFMZt%t=Sn?O~%{V>I~M9byB{Srie^iKH~ac#29DG z<4~#zp$hsnSJt+E^6Y@&!U8D`dzeC(xBI2%+aUYb_VF2E*4PcRd@S;6hG5*`#rO*^ zU5ObowaI|{){cWo+PkE088AoXv@w-TW(nXk zwpovh`mC_E@Uc*2Xvtm?ouYqc?^VAwJkdcsK*T|+ultlBwsu#In6*DM;L~`P zq;%X+o&pjwv8z z`3hej9F^ByvqaZwnkg&ONVZIWAV9QJniYt$DN-NbpcqZjW8#F^+f3N~)QbqP_Pp-k zuU4sQUxyRt+PT>N#6ncq(kUMEanz*;-A@~IkA(Wn!U5p1MM;z1}~Co zHSl}kuFIPI*{+)gL8-YE%b@~7EuolpE)_VYn|DXUrqrIeMRq}P95>Pv68>W#5#j>a zT?LQ#ZzHQA_zryZ9^eO_J1pmuc4Yck%y`_7J{CGTi6utd9Wl^2mEdBSm zMCg3LDP;f)zHx43C@^_j63asNUtq!Gc3@ZIxiUW4(d9z5mnmwJacZsuk66lCeH}QT zBb#b3t!u@^p&6paMX{J&rg4|DDclqgA{3WRljOTQ)|PY)z+EmxoKxf_uR!W@S2Di*37A_z#{t zZ2p0VRWTZC{|+eDXK6zCs@KNT#%{5~?z4LOUUYItD0ri-AS@5#!4m$YMt*3T3X|2$ zl%&{dg;Y>1z#wp)!rsUm`$D%487K>TF1RVk(WJHn1uNzKSM4|+?e*GFIgrGIWcGl+HSNB9%?f) zTVr8mar`!*C>r)2Vquj%Puis+qvifHd612kV_zeqC&`bApwv2Taf*m_DNdBRu2M8 z2IB?&!O9Tp8DmL8@nnYS`s5F1=Xm9G85djn_vMBZG1O_5m;}-C>!*B&&6U9>4lH3f z2En?H|0XCG#u@|`hfn1$h}5QG?dm9(WwI8#0r50q4*yXGRT0@|FUQN|rZJ^P`7N*Y zL1p_Up|Fv}hiuQf!Snw8@}BD`oth%eN94^M!@;;P@s<`f zjucGijyF|^)j6u1F4;pi4Ai8-do8r*xxr!qD!b;(-;*D*?J8<#0K;^zpNU*b8t3qJ zxswpHYjFQY{v4x@w_-5bTZqfkmt=)pYP*AwQ=$hl+E0$XN~{9;vjWplsqmfTXAap| zAHh^B)-lsteim0>`PZxkZqAq0x`KgsHELuIukjh7p)X43SQkBnE%A5EM}6yl*FX)s z=SU+}DOl*GnQU(|hkt^ME)VdxTFEUzq{GxmLtAw*qd_$B+gls`)S?(0ZenwTRChNc zS?+Y?m%4AmAHP}~j1XdQzbfdK^=Mr6<{p0MK;+%0Xs2*cxwl@^-?!8w1Bz<=ZWPe4 zGwc88Mhj%#I&D0e7GAFmKc`DgjzRPD9#s??SW>B`9!miSU|BC?=?fw)g-KkVR#Qy2 z&R^DUG{)(5p*1?dDa8P#wL*2ahAciM{}aKbKq>F)qnN7rH>A~y&Nmo`5t|--XTl8v zc#-2da>~X^bi7qX7WjMK4MAMO!Fzd;HIPc)x0;5Aq{rRQ!P{|+z(f<@kh5V%RKDaB z`ht`1)GxLyVD8s>#+fe(KR(s?#)$AzWa_C&1_az0+8arI&6wna3Jgo~{#D7r6jFuj z=$RDlup>}-NMe#5M+}?P6Y?tV9cOYTJ)Dx&LXMgyxN9DMbM@=R_KeJkvncezsqi4= z?P|GC(r)xZU35k)+!}tL5XEo$Y27@ekd6krua8MaO3$GoRTTPg%_X>yQy- z6@Ae{^D$U6RumP!F0vvTmL_sU$ToDW)gYu|=UQK0`?*4!)q%|^HmdvLDEiAscZVp*Y%jou9?A|cpHrP4l!JZYfC%G`zgNpGyep9T zJZX4WLPM!c{w0H*cOubOmLCm%Jiy^>q1FQp z-GNLdR9%Ty(=%sWSgWP-i^XfEifvEKT1>R}UTX|vbV7ueNAWYM9g`>W`Ny7&c}(M% zH!Vh>t1qq+Ml89=4AbgFT-+K`A{!yh4C2Mqto@6mhQH!@z_9!&#rSE zm-+w`Ev7_;-;F(PPu~j<`LpG<#25A+QRD2l&?UxAPz+|3m`M*QOp}AGOc`=~?Y_LU z*@VeKz=at2`EF#brkw_Eog)IRRr#1YI7+Wdw!{sT{i%WVq;Chcu)T}snD1M0K`h&G zP^I~h)Vg3j()ouZP$BADvPkb7#wvcsFg83Y+EGwccgvq|!B_%iiUDxSMzU5p9mwEM zT&Um$mw%EDt>LPgv(7Z{k>*y!`k&ONq~w+qkGXLfBOC75=1VrzsG9gN0ueOJ(?Pl4 zKlkGrfBo$F8Kov~s$#5IL`86(J1H2gQpv!V`SBX1-N{STLzlO$LQ(E`3IJw>ffNIY z3azt&19Z(!+SOhe2G(cAsZhTBc?^{Fp0OxSeE!Cm3hrMO02Fk{eAz3vBD1}JA%bjj zwTTO z`JIU0LBHHCAK4<-7xz|U=Z~`z5qz}0?-gnjjk}|hZSqX6b!G2WvjIMEOi7lYw(%t)bObw*{Xx{@6`5 zm3-h2#m5Y>EUI@n)HZ zMkP0dn<4sem5iJ-k)2`{CXcewNJnRv_2&T#I2feXdO_*_wf&##eANQeg) z2CN}v8U?<5tA=kom-K8ByvG>SYGS_IIwozEitGxb{L2NTLgd|_Qj)|!Tae@o?N-MR z{W3dMP3JfH`K)rwNU2nT_3$9ZqH;$n!hr(B{RAm2Ao6R<{Rcca8lo4a?; zHSD!Y&f;8Tf(wNi@%DwJ6LJR7-hX;xP^KfnMKpY;sv56~2%Ub%){d^TQv3U*_3Y2! zy~PDPu5{!kckzWvw%@u%SaK6zmtK?bStY*88`Ld|zPs`OK!#xco{&C&Phn_|mUEf2 za`GhcTMv$6-FF=+@C~QnDPFOf7kzq^Iu&Z4G{+Jk_&lHndYGsVT`jTu6?7}4pVd-} zMClIuLr6&`)qMU;?ShKnPCpQb5K0%5pl0y!j^u}73gvn&sS5uoFfAu#?a{7QUnyyx z#W_yq9B9ZZ65Pf)Ws%en)Y&CLy)37(%El z+=>kA4hDkflfU&DOjvndw6q#Pzo4715D+n`caM>e2hq(QQp>i!lM`+Zp~%TOZx!Tw zOa>J)^HRDXyAv(>t5h+4>3Tl-S3h@aZcZtY>=11ZI3K zW2eJa@%~zL6U|OpSXPQw-Y+P|u2#73camf=h{D0f zl(X39Bo5HKoS?{mwq*ILQmlcmEm?H8?q%s_ij;?0^OJ{@Q%*rkQZ9=dBR~f&^7tU~ zZe$LvGFz9^%g{<#%C0|Q)(Xgcg-Fsh=WC^@T-S1O9nD9|DGENM2noYjxad%DA}mXu ztV(5^VDfG$uMoob#WvogUYHFyZRpwC5p3}sYV?bCV4o9zz)pIMDKcTzps3=*`4S7VEU8YS&iCV4rBPv^t_~n#aTx<)x);j!dPKT)6 zApB;Hb`LkF#5^`JSDNzVWI<>pPU&Cw%~bW^|H||Y&%f95D`fNqD;TC;iJY*dIBU5@ zi^14*j$^J~qDs-6I%4z&k}DL<$>QR3=lnf+ z8iE{=I>|nD*dGrKheD% zBU8Y$pZh#F$Xc}_G9l_LfdROP+@xri6#RhDjU6;Y{CQ3Ff6%VsGHB#*Tjhh(S`An5d?{aXV9EZN5(kbb3C%Qx47E)Rnbb(zzL z0h;O^bLLJk9r|f(bBmPdbc4MvYV$;Qi-#A|N}{K|Rg6m)^8-T71$3z*{4dKfqOvk_ zQ)=yD>K8GL+_f{nBi_`G_LqH_6f%nh_7)d@V}Yo1)Kh^%V-L5vtH`t z-OU6K0n=92`%5uZIvcYbpu>EHlW^1XUZFAXXaa>tlJwBKc3q~qEdRh6@v?9W%AJWv zDK}`JSFw8b8<6T4zG10XMxL*4VjIphUN48#IC8_S1FynB0M}(@Wt11=?jP?={M;VL zQ?gQ=udB(ub8ZEPXyGZWM3phaUS+`MKX0cXt~(82^p%-Go2uA~t>q+yYvR7*K!fkZ zqqIc7!Eu_clEk2*)7iT9=YOZkue}(6;agORzxZu+p)c8O&V$86AKUBQ@^oJ=H8_VG z7H*hIafPh;Kd~RGk;dCHcu>mM4_qJkvLN@xGH)8ZuS??l6$kqR_*8XYl-{rnk?1A5IBb2e5%t5QH0mp| zPimjD-m|y*^{1jfKae=qcok?7O^Ep6WYHZlBQ7f7*C`e$Pn+wy8|Q4u=p zQz;uicx*WBNI#fqI9oOu;6=h{Zek0%dZQV;`rR$2yiPTT|0J`FgG6&uZ@i;m9t%l%k;gUAh5E_59tjEz_R$4 zVU0HqcDG<{lUjQIxH-eL4`CYDLU!OVG1PV)EQp%Jm~~6Q*nSkyN(mgAVGx%QM-?rQ zrz=CLQ+{2Q=giiQKDeYS>nZRh$@fm*>34Ud&bw?{(HHdiNvEQ7fH#Xx-dW2*YK$(2 zOOa^YO}--*`=l&k!Aw1LkZnDb2M3x4FH?)YP31KQP*4x1FD zXs;vr^XIl)wDg2E*4z|3U|g20&N1>_Hj9O&Lf?4#^dOiQ-}L!XjbpE9D(j?l>faYl zHr}`<$d1OWl#42JDEXGNL~QY0#q~U33qY4i@}_$U20v=F3+tb)Y7Crl*KRZ}xURlI z1MD~a!PBasCXT?=mwjF~9%}XxJp?bz^8ZM7xkC&{U&iH=%^u8Ld8do0KDt#wNl(k!P78Ips@UbRc zsOdE}=VHkprVND&(#UVT^YkKowgx9fr%3!poMz~A>o(-$lEGzhQ|rl*}1Q(fugi9VF9M^zFvkC z%bE8pH&^QZE5|#2|7IUUUG7q3vORMB4SyLgEpT`(c2f)Mi^mhupb=D$~RD1T?CauoHFQAFY@OA*#q+H;w@? zt{>xF?GHu0Ky9We2HH}L!gW#&a0*{MPM><7`O6@1UxgapOa zUNP2zxWZ#eAHihm;eRb*}w5-^E+q3>Yz>ArN`HeM#vh70ul8Z0fSB7O9AH#PU^sKD_3L`Ct z6vHv#@NIAV(g5oHAeRAfYSr~_D8(qC8`6;B6+prvwD+T8$<1NLt@kq7V zc~!BMVtak8y6aA{b_MSnqiPlyN-GyNop8r{k2Mznx%v>L#=J&**j9ht9Htz~O)>(p z)ON2>bh2=Hbx7JND$3^@aKfaf_HO%jEsr8sU64PZ>IgRIY{+QksAGW)Z&}vuxKJ!_ zZXl|4w%(F%^RzcT*>X6WHmSw}gxUibV;rP2Q-TZW>QLdtDcB4QOdRrPnvcV=h%G4A zyg```j>gj1h7_oZe$D!4T_Z;AAQwMLWOe*=s06^S`5ewHx>`g*NsAD#$A7%Lw`q1X znaAz6c`ZObD*asKaLP4cdZ@+`bVSz_nk^ghMSguvTs<~y5%^Qii=kMnu+N&1O-s|ZM{i@_Tk9pWQj>$GY)oV|Dg&&n(n;|s z7b{hCzB@r?ZXqY$+);tVv(M_ao19<6kEpJQ{Nr$4&EcE&xsDhuJ#qZ69y&+`h ztIi6q3~wZnhPH30#aDnAF-0^n+0I;sB6;LMiH2trx{Qtm4iRETla93;=>4n&GFRp` zV7Y_7`cj;bOP+S>D2@@OB%hfCFIMpxg;=w&gM(Jozja9%%l2k6@E7foE!iBG+V`B(;W1LZ(&C` z(3ihHJ(Ddh?%&b%Yx!i2c-eHw1H9K)Ut4MWz!5#-KdsjPu|lsp2bfFAyY~0j z@bHUvo63FZ2b`cF@A1Wh&)d`O)&yR#Q+)Xrab&f#rp5z)bZ=Kur9SGwuSYiGm&&%Y zQ$Ik?q&!%O&(IebG(y8izM%>n&aix+xPX+?A*}0Wk{A`9`i<6in#OGWZyHtfNX4!V8BQT99>0wv7vjQ;e^jE_)YQB7s{Mp{i82wytAx>QLh zg?N!#_PGK(1R4F3{l189i>gKznf!1^4XRr+?a@;LJX2hUCCk34zrwJA??Wtf7?^5V z_(lrvYLT}doqwNV_RhJ!~3|xEMc#%J`{Dsm#trqRuegLq<5A$dseJ^ zy|RDT(|5BPov0OWdnx@F08OgZc@|P!2Exf3Ee^}LtE9FVFNRK<@RY1>0gzr;=@!?wM zj>9Z)QcEP~*hGOi$Y#jej#ffq{7S0V{T1mx!Fd~dA4x)6M*Xtvid`Vt{ck&SY(2-@mAW6EkKKG8QIYV z{*Fv|WXN!M4z{=cKFyL4J=sa(-&3TY`}*k?@nGcJR0O_QlWBpPoAJ$T%7no>{H<@ z9GlK3zVOGT@RkF?0UD+(gXw_lA#~bN86YsSGbN(P-WE%5RZ!WSU6)*slgW#)t=Et) zDKfvPl$2A>UiBJG%=w3E9`9FQ18DWJyWtGBf0q<4CJ9bVON?+xv2@#vQ6| z@u5r3twJ9#Xr!pBIw*}Tro9uzy#}9J6F|QHTfFYdM|E{kGltUsn#5azv5ubVsV7Rh z_%S0&l2HVMH@Cc6aZ_=(+pdmR#%5SSX+gC`dc&juPK|IYs2^abkkRdAgT0asP9NWPJh9ckk0T{}ZS(j9}|J zhql_h;6G{>Cq_Q|xD@^pp2F zTXiLe;4m`~CJmaif8J-W8b}UaQ;?rD_8xSPS<=~~+alLX8Xg^AF$X??w+tn(P?b#) z=mWqJAPb>^&s(X`cYT2Jqm9EC7-)4eM7XHw-x9!(&Q4YqBc~GE6I5mW0t8WHD9H?w zf_)1L5V&sog_l$!2tDOzQvuN7_T%n&wl5L^<@V?2i#~2g_918EfQvnIUx{Q6|2r)E zO5i9^w{R{X9Qk|vRAqXFgdfND?CjcqbybqeX7@F3PRWx@bBe^*_2=!DW zBxfH9F-Jv$q+hCl)*k=9joI_){Kwjd&JOG2LXUqK4*crWCeG1N9xxyo0<8aFmtC={ z6U6LuHcy$$-SWm;&2FcdNGh`@0d3TVaMob#lb5? zcj^7sN_RfPDRQmIkkl`yjjcgg!2+e{T;M~%8b(-68SB6RaU=UecU*$`z zOLtN(jSnwQYurw|u6pbPW?IZ1>(^8@%yVeNV^`JRV~8U~+~tog+3&e7k(3tM(r$D%s>Hov||4sXki!-VK z)Qry&ou@^HN2bzJb~nO4H@nUKAI9D?D6TJP9}I4R;I4y1fZ*;9!QC~%f=h6BcXxMp z4LZ0>kl;=T?zWfT`~G*U_QTeGqH4}?Yi^&Gr@PPT2R+c9*-^C**EhB9A zF20w*EfX7AFK~XbOuvG*0{OFW?DF!j23Z=n(@J4jVORCM-?Z7f^~Vv2{?i#SMctWM zj$6qSC|k7PIW#bE&mxN&`F!umq9$m1_|&=@1(~)86PV^5;oEYSn@##L_%iv`YXoJ> zB82qX88hu1lksmAdiFPA)Z~SypqkbsGa$2cb{nNyeJ48bkNKb1&GFj9YP4ODL636@ zo;o%qJQmx%rS%apHqN&o%m60;(L9^`SXYQYI)%KUk zK_<1Gp(vys;>4UoVDjH(f^zd;CP?+6hyPP8Q~4?}NyN5DdgqFRD389<;DBs7bfxJs0v@X*gbPk&l9n|Kt* z^#zQOCeY)qTimZ{*gDD?(J?T(>Q{dLu+#mm4Q8{1r&0{=6gw^z8?A`JYewiHoCm3d#|$#kJB@EvEPJ9C9;Mr0AcTK$Lh`}Fqa6_xgz`1<;Ig8n<>bUvNC-rd zKj8SfzTGN-lp!Vqsi2n@<%h0~e #!0Imx2--rd=k%cVC}cycLr|c{>7#~)kdnf~ zLox^XF4(4&&o5T#)LAaj6_%9H*MP3ZT%>uN4tA)G5H9x(HI?29+X(5%%jQ3i5WspmwzBK!3=83?3LBpmw0$UDhdRdfmRL@J` zPBlH)8Rlj1FlNurxg_g$yT~!$D9n=(3(oI!u|pE~jt~A% z-2OrkkSwq`hGZJOe`H?2Hogqv zoWF1Pe7Mwp78?0Fb$L~ci{Y&+uztQ1)e{&9ElXh1$qXj2B;?d&nQPGXtU1og$f!We z>}0l3)1$}eDUr{bo!y)eNPcYeZ&Biod@kgVQoJ}Vu~o1E*MA$6@pugWOqIPtRAG{@ ztSlu=8jM=|gi_BBe!Y6MawPR)UxsZ?XQE6g;pa)`tlR0=>LGj?WH&I*NO*+O;WOCo zH~4IXhX~%#5C>iU%=u)e_27NwNkvT!x4gW(`G5faCOxbOu=MHo9#uS`lf>7HBs;jx zcLp*e29JM)8LEp7{^L!A5t4=3m;Fz5Ao7r|7slv0DyR*~ z3X>!7D4SC1>tO&RI*V!oEN`ngy?`>Kun@4r##j(|0VYj*`3-BcIr$BGB6{UQD`}iV znniSPM+nlq-*%AMlTkK%gcg$0_P~yg>jTD@(KT*Y;Rtw5VS~`52{zA>M7Bd0`&d&U z044V<#P@H?ukK|Mw>xrz>jNfWN2k{vWC0glDyk$dl$0z4%5_zwl${CQWJ^`e^C6Oi zh`;mu6oMw;dwW+G!eC}3)pTXq0kEKj0?FpR>%fb)HKyanKF99sg#5}JoSo7mO>7;H zOa*8V9oiV(PiVbYaXbb0bF%+OuxHrZejB3>|5Bd7#QZnZ4KK&(s;X>N-mGmkY_U*!%X1?CR>vR*l>Rzf1aOOg&p=!{kAEOtN zBPJbWeZnVQ5S+{WEzFW#GV8ttQ|+}(tuEo#2~3xM=8soV(5A1hpMQ!9eiRVla_%UyJGUHT?AX8oVuRKeiq5-F+f zXmv0FV|b+MQXXA}-kL~K4G>WOi&Uf`!iAp#u^}FcL$IgJmA41?JlH}t6t&5poy zrUm)RszCwf4J`&Q@UaoX$sUSj(*e$!> zP3-7PsdONClEUlCzz^ek(l)u=pU}O~KI2M?qRdXtOd~48{%0kg+lw zVKBylaDy{xIen))lIj6yiL&7#Qy~8fe0VSn3<36rQKYRf!5IVZFDgum3A_a%A|@qr zknc5hkl4&?k_+R?DSZU?r?FITMu>KbIfoH4?G$N~{iYZRF9_+SyC#{`jl^EVk@oYUK+6<}3pMo4JrIDaNu zIpViCV2!`6|07VEtpRwXsQrHx`F@Cj5D}9QmK`%B)Fg+u@!O)R`)@VJ$^*YC_!VWf7nr2hpvrRiA~T)0Jee;vbC>CgXdy$B$--aH|t zSw(m$A7v@bQv-{G{6F2wyud0AKK!tz9@~nNC0YmkFfVS0H8mxtq|jp%H^-$IYAYt{ zCbLlJFpaqdu=!Tdy$oO0D3bq+etCe{HWhPeENGw$1!*b5Z)+MIXKW10vVc`gWN6b$ zM*LHnrcgaV*CK%%!0#MdFYjDeAb4HG|FJgA#>S6Be(e66*tj<6vVhZq_EQAJW=Er# z$c70yn}dli{-m+eX5LaWFAuDS!$PvC<3zb6Y7a0%UZM=ytV)kepv8JyM@R9YK{LVt z@gtNLbg68PSyu2r@%h~<{>NVbKmGui4)`~g`Jkv|Sb(WqXc%wHyiA8{jI1>XLr+@g z{bxaJ06O`XfW*!66QJm~=iv*}vy*iz4$3xw09zA*U3pZtv@Ww^d*5od=I}&XO+YE( zB60_Cpp6Oq>>?rlZ>wr+Yr_#mP~bu&7`=o^P*0(AB0dz`)l<*zs zKwRhAv59>c3~i(F*BNI|!_<729qKz$$s3CFFPHKO_m>D2LMzN7|4?sA_AVKrSU?Ae zD16cNP~p)5V-0{-%$KADo?K9`RjBAMG=K^Zes_-_CUByI;d61Bam4K++NcsBfnO|y zCIaXIXg*a*5$i}nX&N4l#=p@KS}6JR{}>56;0E5GhXg77E}hs9GS{VILXXa>91)l@ zsG+!!Tev`+`k4FBH-id+Nxq*CZ^OJ~s%-eKnj(PMbp!EP5z9d@w21g*km9i}(I1B6 zgZ|rB;CUv_!1J7hNu}N$8oE#zaT4HRqDQ~Si6B6tiTW;sq%8h%|uV;%^P zNv3+5=}v)VQ0!HfU zum}nS{DXKYb3ra00HAWwK#GBe0DzjKs>%rr;>EEfuv8D%sSsf}mso8j81k#>KXOet zUTbsYa@_w2%>Ir`NjXT;)72$>2jRtE@hB)Le9y?YCUXR18qB6FeRMgx@0`l{bem_3 zWIY5>R{3|Fj^}B24-TwanWNpo6pSWAdpizEpS6_VxPC0nX8Q@#pW@-#%wW@}*72+y zgZ9v%jnVuJ*L%IF?~JymFq$6F4Rq{1eJg$jtmCG$U;+?ZGYnl7mz{dc)<<4GT0}%ldPcy3SRK zg`wLN;@LJcOZWS9LV%-3nh>$*d^zj;wo-BV-7X88N5Fx^2mO0jYg{`l_7APPo)iD- z#~4M8jg5bE{C<7DQb6XUdUg!}5-XO=8i(+Yrjc$Yu|C|NQ!0Ke*VAcA`4sU(1+en+ z>X0X@5tQA|-?3Xv{ijqzO?FT-BV&huh!R9aCNXw}R945>7Zh&wv z;r2S1cHi@F^a8VnFl{X<>K^=WFF3dE0SbyKzr-CNRSi0vtBv6>*yfHc*?E}tKSg3Y zl78_F>z6y_W~aEKl#l57^9-&w9t>}86glS$`{a4Sgb`1;aoj09HVqa8=&C(K4zOGf zWC>Gx{B$|8d71G>LR!U$dbl@To|@cbfUpbTceMAgAtDAljp1(ruYWDxu``io*J@~o zBBl{1-xSzX7f#!z~B4zqaM)HEozh@;gxpbw4-J8jpoh^<} z6|279%iX&eW7&3td&uoV982>G7nyT7LTQG)h51!htX- zOurC)jQaHpT-?1!e|+p`n30V-Mbr>og>0sv_U`g^@hyN^j1kc}iRT>^asH3lUJotT z#AF2uL)CPW*Sfe{q>Kyx3 zPk5htm=QL*Ss+dcI6vayS-^1yH(H0EkZ0|JUwiee8a@di{X^Y4G7#OKaA^(NI@v;7} zCe2kDJ(t~Tv9ZetOyCHBD6&7Hs(U65SHPt7bcz5ypmcY|SIDoXaQQ|s{&XaGbBJj4 ztuneAUDg7~8iIV2e#VZ7BnO*}dME%5$<5bsJPOz6eNB(UwWOEYoI7#tV zVxb`P8t(JaG0K`>X3{3g`zTT;?po1|X{K6e9W4cE35>iW!oCP0t|0)xLz%PkBKMLs zuiQoTT^J9>X?$k%UF*ZyTj3c=6>$B4&u+e%G-74I*n8qAFu49{y`{@-D=BrZAI0Xo zMQG3eipua^Q~&}V!v#b&)QZV&;B#7;qP;8EgiOwns~)6H7S!wLI8lyFAwFbp1aNSo z&9WR}nAGo#r?ECm@lx%i(tLzNC>t%V>B*84r#!-&MCPW#oSU0tPqO_{FcLnuB)Kxz zj?ZnO@N`y^^onb3^c=M?qpnXIL#jBBdVYs;Y;7L$IGx-Ve=66Uy|PlYvR>X;^!j9J z)}4FV^jOg-q*|rDP_M1#{2^?ic7?ywv`LpR`hX%5YKlpO%-Vwb%pB#U#sE(Fc0(b% zkHjZixM4J`G*gi-mZEd}ufPL!vssqdME$o|kt6I+vum3BZMv$TvCfheupcPh@7rK# z1~QcBy#>^hYvEo%p6 z-!RDjx{y|OyxH*)ZKyc5lt|~FTlN5|*Pl8_mahav!q6%CNZ_1o;aYrIVx<5;@t$!M z!~Z4<01ilnC(&zIBC*binLI-{u9W&gR~HCWnA8gwG+}UUU4uMSqU`H4QsQ}P8>ELH zptt|-Q=ZbE;zeL|iJl^F(^Xt; z)r`a1a@I=)YL-8QZ{XQu5WrjuutF;hRd6-Kl3 z>d_9ew@%NaOp5vwc+-PArHD_SvFNymF)lu|Ot_oX$EO$;Q8!Kn$g_>*6T+_-96x)|S_k%5)5XTb6 z7d=N_30iDpuj&N;3-vs}VdeXg!ejHpo4fazWy zfVnCo2FM%#%bw09eof+LCMO5uv3_@@Ru=OLMj@gzEu3g>n#=h?#n4*-4`Fax*@9%D zCc~UsepJNt-jyh0W0h5i)9p;d42=?NaGH2tfW=OTc%2VsrtPy-VCs2xXxJ4wt|?@4 z!-!FvPv>sh0+&x!%Fw14eOt)kq;Pa_cs3?G)$%QEG=soauld`VqdpheYzL^7IekT< zBCEj7WeX~~<75WY}+Cf9QQOnb(sgoMY-dj-0 zpRRE>?ure}IcP}NUx6e@yytn;2J#k}wGP8$8a(PbwARRerK>IJl9fSM3pG{qlrPs> ze65!J*r)u5tgd`$zf8g+JulO#XpCkOwNQxucDVdQ|1mXwE7RdtB=$W2rak~C<)rqs z$x4B$)ZK1i_H1`g6U^|2K%TA}bjQQv3S0Bh+V%RulCqnsFHA3B;8N3mm7!F5#nan{ z^5{Y=F?!>?LPL=X1Wf0-C0{U7Wwn(H)kkw8zVmv*_>SgRtot(H@SCT!q@Dexir)kR zMXBS-aVdUaJAJj|9_jnZ!=0GaopvXri8corsI6_Pb*}j(T8ZB07H?=&^j}9u$gMCs zT1EE1=_6KGyFJ*?>jxcgK>OT5CyTx|w2wKj(?g2H5fqQoJ=Ap$8QvpYEOSV7zQ?%c zMm^bcgKchqOrJVs)*B5W5q5X#fz!U>_$hqG|99pxYxohEDKUsYD+i+I*6n8A>4K^ZJ`~wvqjP{_82c_+K3kCv~AXZ z*pr9Fy!SmiY*_GMYKFruFqBPH`WeV&fqF19bftGkD~C$)*Sr9=_5`&$r9ofWw9L%N zr?Vcx&HEH%G2WLW<$;pY`Q^&28}x9`@*IUV$-cVLItDzQ#!AnuOF`H3+~zPmd3OEm z8#PlrjL7dL&qyZyF=Hc^hHi{GiHW;fKbE}I)v-0LwF^0(xlAbQj$)n@Z=Ki^T*yw} zkLK{w198hKpFMI}lc);d;6Z?l{6I3A#%_h}{d^&LPA1iD2Huzkcx(&4jV0p>cyiIH z!s|BWCL2P(rga5;&KsVM{Tn|7O{Pt+q0zCmheq6vihl zG*olBLccw-BlAi{kaZAPQdsnfst*shsI=^qk-Sw5vOeCyTZm-}5q`i3e_bQU@abQE zzG^MdXNZEgKA%a^PINC6NC--Iuz2gll9qfG8dxqD#k&tI{P&L%f<`^h0{r5@hg2o- zvcfRhLA0{zEf}B7DxGdKR`<2V)wBOt*|t5ueShkmP|dw161t8qmPd=~$3}Nv5(7*9 zK|YqIJL#V!LfIE0f`43_o0n{@p*}v%el;yPvd_$0*mU6E_0VQ6HpFPf{E9hJ`UaDc zoi|wEzsvi)?HG7w?BX0=4i@!9aYT}aY+Rhtpki|rig`NAt*G>gLT;5zw==j zBk_Zfmh|9w77(Ug^*yh?W0rJv8gKZVN!7bM?^bDvlH{6tVJFuF;6aVE9If6FU%#Df z9#VmF9xPa04bKm*DR!lgJ;$FLGFrdEX+mFS+=O1tgIyK z)s(5^BY4jnL%~YxNA%*zBY44? zQ>;_;kJgKmon=EAR(}^i8N^^Oh_Hfh;p(zKoT~NU2oRj7X(aYh5I&=Q531V&e<)LV z{Z6N`5NyC@t?tgAI46ph3($@S8KGJz*lHH?cQMy(@0`6f#!EB?GZR!4G<5=vD_=RG zuc==kJnsJ8Ai!cMDe&K5^0@pxWYskiWb=Tt&GF#v{^}Tu-;JUe)c#zF()`+S(`4!v zJ>1aW9f?uy;R9!h7=fw^S4gK`8Cw4>fb#^4t`m`-sKNWRhOhG9<>OQjnObEU zJ_2{0sMT{*S%IMY&^LtIS6THfg8C@%zaT4>^RE@f*}a|r&a~TMue=2}_iFA87jhJI z|3FX0rK=g-BKJJ)d};SSBDJxMkL%dPWL{`T6v#Hn9e}#okfsj%#}1TjCW3`HdJwNS&wztc+Om^De!t zh^bzJm9bIGbmU1TLPP)a9)2D;kXg3>(ecWABPMUm`?}0}&H(?p6EW_%2mNAXx81?z z@eQZrQ!I}<2+?^mGBna-8v!$iO&Y08n%DcDEN5+I@N?^D0dKzT!dopW7u@8@gwAT?&y@7lQ5WSYHMb%8K~4{N8IEkAADJ}YnX}b z0T@4DCEgRxmhs_Vr1jSXYD`#ZPl7Nm9I9gKUZWh|AX?P?CPH$z$8BA z<-?jmR)s{ik9Blxmsh*ETQ(as3k4Hh&xc2ML9fAxATtRQ=g)~cqplX9 zbAY+SB81rOS)<`gm zpg-6Tzp@mOJC~3v(cF~w*MxSyuQtr}_OI52Hxtl|o7_Kg?H;h))-n+(B--gK_a~>N z#6RNEo|vjMc#3$N9jbEucfjB+EtQ0l5C@f9ITZgA@~fP23Hi{{k~Q8Mr+#7trsHiI zI$9OK+uNZ1pJinC%3Al9+X{J&D4qDHoRN#9$dl!b|47iPk#NvPX1!#v%CaL(7b=N{ zB}_>f6+tYqo&I47;^-||`%L{DCE@E!D=ie0G}nBwSuEP8Z=+O&C0_3&Xum%R=0f|B zA9WnX1gd^xkPq?FjH2(jI_rr+Dm404jpX$4JZb9OOusi*M_YmkZNzx`<~zgu@vjmT zl#eGTR}@eQv3l6v2lzsZH)r8i<6+i%hEjEbdhmdR^kY5&7X^lWv_;zWd$42+w>m@MhPR!hdvXN_c+Vt4273@iGyY_bTjTG zPWep6K1OP?iOABTHZZj9NTcf>CtF_lsD6UVbC(|9@kQ%F0CddH* zvC)Vk`_t5fq|EbcD1De!O=W~mjjgdJN6aux3-;9*s56V(AVcyOa6Y96~VoDn~3hPSk3f$R3M&b7MV5sjg}=Rw;j-s>ZD1_X0@ z4Xw1CLHWOa1XH7_r9Lgr4hw=CD<6wp^fJwFE4sVWX9-JeLvPI;D_GyC%M4#OkaaFN z;#_3{^4y0_zh<-K;EYi0wOZNP_J5&v zm?XQ=#2%`W9h|}gBlo+y3&@Bq=^-DRo0bvrCU%Dos}wF=IL1et7+2X3A|7Vyijq5U z+xno#?NRyogQdzHI#rN%ah0{6sLHCp^<9@E!kBtn@WgV>#^7tJEeVB5HT#U55|&nI zHGhPiZ*ch%|MWFRe8cM4fm={uw$|*>x>RX#iZi*7h_Wgz=if?<=N~^@`Tpt(k!G?p z8^u+u35~a(F!@Ny>7L#%*t@YZhF$d1%AHejUa|^FeUvs`YAP-6b$YzJ4J&Ht2AJeY z74{}z(A!Roa5K9QiF#u0*-6lbrcpWiQ8{TJ*D}A=` zgYm1{yJ@4p{AnS_?A2lgVji1nc-I?^)(5 z5xI{f@}@*pvii(MXrhJf-9Syc?V1u7h$Ap?jH-%?lyON$KK~#TaDvtnbI1|plS!hL zJz7vMn_&X;LME0;+fN#w-CA>r!2QG69xE{7XUUvIs&rw9o z1iM=R3A`Vg93FTH*303a>HHeINxnqQqBU@9_D8^F*4oOZ6rg^SrUOR?1>$ zr792eNO&`jr*`yo-8jeVX{NmBrUoeHsqEdzQyfh>U23B*uk7cErT?AwqOE=53Ixd1 z4;j6%*H#!4R<2Q7Xcw63VE6Xg4B1Atp5J?6itiZ!msf|F>1uYsbiNI;u6la6hMQC0 z%`f@S)H={de%Q^Z+hJmb{L`n`)KiHHXh(L>^Y)p!wkO|G)E4E9Q|7z~+@SosZ)u7i zvii$ZfGagidoxr^OBT>TNmD+jICaj?w|j3w?_64oUu}Zrj)Xeo@Qw` zbO#bMbt#~S4E+}h}(Nqx3sVoKA?U*VV}5J+d%Np`1M8qOPP8&4^svA0O&u8&b0-w&K(x0>0?rdu{Mf`s*YTeCly|trAs;%*L2$=ODGrU|tIli-Di>S&J zqM~j}dhi_X=jI;^D5Tqr&x`Ez)JUJ>MkZAPf|M_;*{Ew%J*RueQG~O&>>;0D?lLi; zv+@R^5uZ>K2aP2tQk8mcQ!a`NsHbh#IcbEON9IDGJ2HPj(3OO?0!5>KFhGxnFW2R9 zhQp)S{+;~(A-bdM?$wqItLR6;Of|hIJ{Id4$sy>LO8_88V|enTpg&DurSnl=db9ys z*(;ErLGjT=zG^HOoRz3H)C*y)j;q?l%V$+#f#RjxmF!!+Nmk$Onf}5YEQY%f9=yK)IGT#o#b#XV%L*k;PPY(Zobwo*bA7X7{=6QNP`h`@iyUs+ zHpSzxx!d!0fc7g!`WuXbfda*kTa-=~GmIhSpAJ6*Wd-|!YUkF{*uG|WL40y!aFsFE zieWrV{rkb^?B~_Ap(JteJV0uui2eN-U5?EE>V~dl#x145dY-&`1HU}yY$^(oq30bo z;XqVuFv>5nplKD56XPu-vp+bBMdLJ8Bme7Ek+7#eNc2oZ4+nhrg-u@zhChHuQ(5qZALZkBuL8 z3BQ+(_wcP$3A(f}V>hfxHNh?!nj4ChS!D0dSDpT;*Dz-cLAwy7Eh4soeOE^bNNS+D-X$9Exy+5R4EEW)gk7> zCjEPb@)8dmNOeEPR8@J~P-!_o4tGN&WlK+*X*0msgZ6xxMtY z2&ZD!)IVM=3M-ivM2M9)NH^T-8MKvn9Z+JaX`!%D1yoxIND4~6pOq;6*1TlJub3F~ z$u7ha6gRVOld)>mS;>VtwXJh9J%B=Ka;j-6deD>At|2aT-Vp(}Uu@Bz80v-u4|885 z5kJ1>N_=oBTgHue)UEoKl+czjJ{mhfCXIUyvJjfULyPn1E^2BvA%ECfxj9$uaQ{_D zlzRO_+$m$FzAX~SBi=Hs20->Fz(vALL~{WxXVn2uG- z;5tJ_?4@jS@^io3%Jxn4D@PmxYs5oc;&#a#wiWc&=M_}>ycY<8ucvlxr7qtbRhev? zjZFA+`mM}sc5vV(RWe!%UxLK`-k-TQF7N*)a1Vi>iYpE@E4);?ae1yERinH)e1$$Q zNmmOp^N4$SyisXUy0W@PDStr6nUzZFJnJ(v`h$aXaqNqx`THAoJ*2#FlR-1XfWF`K3lK;D>F-`T045A zAkaJ5?WOms&IpMI^<4Bs4BQ7Z&I-L27h$9fXi<1;MabZX74rFY?{3pXR*4bk6)Vud zTdq<}FYrun^E}CW$N6?aMO>i)%Z-=Olj=aBrh8 zYL3&Ldh-Z_T&(aFdS69@YHLNgEw!G0Od2v-bf`32Y@DcXcZB`|RpC4-d@evR_onX! zAK_42>YJ^^Ta35k=X5FSv1!v4c%FnH4fDJK(>(3#Lj=CyQ`*@wFY9NE$c+0?M!yt^h3A0SUsUnYTr*o(EDzFU24Dl|O)og*7wTvH|XNL~d>Pxr}H|cIl(r;*O zbokYm2COA#UXt=R7e!zaNy6|rivLhlxMOVl9rx}2cNt@7F?6^Ymp@y+A_vYYkH)LXag$7#~D(T<{uOT zk@rdf=ys|!E2)coN{)eEZR7RGF8&u@hy@f-9XmBA62zUW=gbY7P?5S>FAD)q?jwgu z6#!rShNOnu5XG+UH`E-B=%{kC*&9Hk(_E>8hl!x6NoRuhQw>&u{l|CB;PR@1+KZS)#TyohuF)o)81@Unb#oUY(}vvQpvxpt%PDoj8DR-}JhUPjIb1N8 z#bmIB=`h$W6-a)Hq*Qa7*^gWo78g^eQG<}ou`3m7(f`+fZ#<-&Df`>6mlYo!jeB0- zz~Ou0Xl!b1Xyi;Z)Q9KL9_z~OjGmXEz41|0WVn|6ypKT(jE|mCk4>*_GdJWF9GeC@ zbJGu_$C_(%dOQGQtH;%&IwQ*^MHzJ!&qN-Koz_{#^o0Z5R{mvL^Xbn>?T3c;rfUy8 z>C3uA-BnAc?V(?Kr1i(QH5w+&WnN7?#p}PLZy8U1hd~xR!ikElOQ%1$FS+=HbS#vI z0bNhRFK7>=#n!m8mX;HQ6%#M#`zMCQ@h*^Po?mI;jxFP%8CKn{#a+4^zQIY%ZPEHf zFT7P#s4&T8QsfCd*HG7=Uk+K5DNbl3+UjrMQ%dn9|M6))8{#a?ZpK6C=U;746Oip$ z`cdKZdIr~M$B7!FSQTmFr~6}_FR}H6ahrELPI9Mt-J+S?_Ot8MB*w^3zxldjy)X`I z#n0?mI73nWZjTK+|G86=D=~_OSLET~T><5W5&4?vBN=k+0$CAh4MVCLwwzs&g4q^J(ilwk9Bg-K-|2vf+P+F=B9ID0-L z$`fIO>(^h73J$qON-3D4wx`1W#HpZMYmWQKI@bo(O)=+d$Yb4l}T~zLKp~xZJ5B@#ObnlflTlS{A*Ce^)uEaROTx?&c+e7I9NA;lvcn zu8{K5l9aCEujvB?W5p&AXh*T3^JsBN?Ixpkt8FUv~;(pAhCgv0$o&J{-7vY&k?_qn;jKV3neZ1;`5#>;+3RR zFTH$U{ZYXj^NqYfQ%3Z~(oMfcw)2*0L@KML8LNdDFW{30O(TiUQLO_N#9gXiCfMH{ z@6{k(6Q9D;Dy>ef(FDh$ru)ROi)rn{a+qynU9nU_Ut+kTT;7NdB2f;lR@egWy3?r zOdhRADhx?_Pt9LMjGRp?G*3i{mlcC@CfgO6TkMwHZwuj?t2u>DPLm0tCsL+km6NA> zfn(W!1!v9+n3`~V{wyRkg4{`NJ>v?0XH(aYm7A0ehDgUxjZggU4jGoSJrZm>a>x*II4{@$+n9g9c0n({6wn6-)U*vr?P=Qu8qmrP(stTIBX< zM-g03H)5%s!r=Hj(&;g&QX?kna+}XfgK5p92GQb(GnV<6K_9b%*lVgcW)?OV6{|bC z1KjeEiD=6Y(xns`4WAEq{4Fb}01-{}onQnOQM~lZg;~5y=J!O(2fdJu;y~mQPn7wE zGV1}%SV+Vqhl*_qhHo-3-_habtdHQEimmDkC>eG!;a3h3i6D)i9^=W5Ry+A=f$pX+ zmTz_W@9USS`Qh3sByGjLcm{+N$48Gr>L>g*o+o_mdvc1h2X0x*_?L*M;a8aE84w7I zH7laAOMZb_w7N0`XEnC4qG1xDXwu~J0OfL_B^^tn43ROqk}}>UMn35?TG&gdlykX` z+OcKuyp?nb?H|Eut)~_;XCDqrcI$vll6@;nYvN)Rpjjoa=ODqoI)XBM7Z`=n&?WEO zhKQk-dO9Y>KFO%Od~ENa@_>X^78`FMOnUtvyqtUu8^+{gd1agzgI%;4lkC7Q>_#%H z^RVKf<1hrp?(@(euFiKe#lL+WWMw)V!i~>1^-=g49y*FcO#(U^?O;2SnL5b)m)?>C z&QBNp;w_{SHQ@9qA*9OD!ld9>if?5^s7ZaD%#+V1-=%brKl>>QHSVA5eMiGw4xvR> z|9V%WuvR&XIFKcr{1@aLb~cWvRmvm4I|FPtVSqirCm>q=&04n z(qKiIvgArf*t?DQAa|^`B}%PS`S*Lnr2>R0jr%Ni3OAZAQcVurYgMU+lRUAv9;i0Bn zHw0EQwe|r03~-n7oZ>b$^qOJ?^oJ-iw8C6*YKRMWi~RJcBSgyMaR_`^w)|v)$&vgF z0L!b(9Pn0EM(vI#h^Kgd35K4F@HjTKGhb-Xe2JLCxissubMR6bNfA^9)X7PX5-Nx; z1VgE8vDHT8Og`U+7KT9i$|2H9j=)rlm)Tpb{rlt&t#Z<1K8acxcjN2g4@LaAWXlQt z^uH=7m43bH*Ci0kscp$9{AV+vQX%`KM3}@W$ydOM7x?-1-{i>MULmTb(VgpFT~(jH zbLR-FGOdVemfm7IsgNhTIK?K0d)`oi`V(ThsfLyUV3G6NHuYOV2F)8%DG3QUe!x@V zEi-7a|F>0A~EwMicZmCy;EA_92y;|sPTQ-tRCF@w1RT__})~E4(tcu?jxsB@; z=dJ|b{NMiES1c5E_Stg_Qr?th$7lTTxr719F7KO2<2S%^Rz`Uc7+^U76GI{6BuT|6 zR#Q3;R&&mm&rrgssI+#Z`j{9h41M^?s<~Du)OwGdR_-X5I+a1)f0xnN$C79ME%dw} zjg%!=v7ex+y5rTe;znFG8Yh<*ZQD?IElV=^b9MG)+TrMm0p=7xtEu+k@%;GK;Nmz9 zaN>?6Q?BwZ+&LzN{4tn&Rp4-xRj;lr)1NSCLd02Znu}+gsUisXda5dpzTxv?-O=}$ z$a^oKI-d{*Qlh0$GWBHc`b?7h^uy|Ym9A5(6dL;AgEzp3HZt@>3;&X1Hpf%%22#nY zqM7mq!u0*Lh9(v0j0|-7-A8PNR8!E-479@E9@t)JwcvM}l^N!R+Gb3F*skyu-3v{&VB-r+gZ-Dfe;Bj!y{p3q%OS%x2 z{6i?X&rYHUl~cejyjaC_cnif1QZOe%S%~U;Ok#y1GTtzL??W|K2f1Si4ivtLh6iu~ zv|2yEiv!2jDm(r!_TI9ss;Fxl7DPblmWEBYv^2;DHZ9#D-AadqAky89z^1#qyBp~a z>F#T;rTWY@}7-gDXC)b zlppHJk4n`LDxqY0GgUuo=(PRn!O)1IjA(mqDcQEFXIj7gwn?e+bS*f#ym6!Xy}aQE z8dtMJBN^?oV`Y*ZLG z%?qX=V7MC0xmr`7wiF>uKEWw@#JA0@U;LDh1Gd(`uDDba*aM$q*&xr-b4a zs4_l(peB%<9FzRtj;?V_o{F=`S{;RwWt&`ynNrO-veeMEWdj3nGn`g&4D+8RW|K0fib&>>$xo z?iKK20xL9G049kedsvj;I|Rn$Qt3%S9=u0IatPtt3+(d?+&Ud3bEc?&_e0pTn9ElM zgvl)sQhCiK+L6BS?J^HI>MM1>qm2l0qq9lvo zEf~7za#`!|q&ah{!REV52JnTy>mUBcFh@R_lDkQY5r|J&pYS&%R;)lHTAh=ju`bG` z*hEZ`N*(`g@AWFYrUTuC4AZL^hxDXb1}~hE$j|Kt61R;VK-RqK!{5?WT@DcF*(X-6 zB3z|Xu9?l7(#_E^M})!KhOp)qNmDs%$!XArlv?fyL%ty@vq@)95eYXkZSyyUYY-Ug zEvAAkMx-BRVu@R1D!L+FayQZkQZRJKM%; zuRkEhO4OoN$755Q0oY(gWP#&|U1aBi+w*q})TQ~#Zd^DZ*> zm-7gM&*FzgT-E%N!U7d0e;Cq3C3aF+fT}<-q;EcqCQ31)jPWN_Z^~+zF{iN=AD;k) z@+ZD4E()QgxKSSd+rMEhF2{f`O3K>f}egI;%{n!m0CgZP@?(?W=07JY^ziuYq!!ydy&H z-t?$#ptzS-+?xYMua-firoCNT>L7!c7lBO3lz~m&v$=XKX^3oiXT6X0kDAOh7wH|6VgYK zS*s&|iGleg=Cm}pDi!U}0ByV~iy7>suh#F&+heN%A-zNQMtbTYjLaAph86EjVS<%i zx=fholl3f4T8lTi;dk+I%5;gGnbmEX2HHt zoNI#xT$ci4?{7h-DYbz`-}ic-8(d3|{%ACYbAh7X_KpUAG;Ufxo`rAT)6R&k1iEW= z!JFl4kJYA(rys56RNOkTH#4#jkNY;9dTro_O1S9^Y>n^HC>KQ?vywnfA_yy-S0hV@~`mVf;VbKE}EPlr)S9CkQK6G$qh|+)~p| zKUwo*I=ur9W?}CXEL$AaEkZ_&d|aWRK4?lGN$0z+=&p!C_)XVlFlFj*hXM|dAUUa) zzqX2OmiuVx4e7|EwT<3#DSz>dGzW5?6@~8Mtl7-QMQPa5g9h6RyBPKcbCfITLDgg# zu$TNYf=<8R024Gi7F=zag2nHA##Q1ftas%(3oAX-kW|S*ORF)2B4mvJuA|#efe^)9 zPCqJa^Yno2eH=-}$U~ke1qi~e#2;6x-K2otelULxfdENyf*4db+f?^yOrf>m=Z7@t zZ~nl(kg4#~o&%;Xwg#b65pXs|U}<2Uc7C=3MK+G4NL~xNe;*WQ)1WAjx)$A0kSvRAkKXbryrtGOFJGG+RSir=pnZ_%sK{K7qr$$7qi?*zKyPoJ=?DAjd zFPAcP7D@2l#d?l!B*jMmf^wUAifTuW*qGv6k?8Cp_^MmJ_pt^c-}zv)V3WSm zBK6ZZ6Q}p}v@&KmIgbOAx$-}!f4gEvPNZ!Ui@mf8>d6l+%j?-1tR;@CTW2qR|KlDC z-)hA36Subrm0SNe@8&*;Fxc=i)%iEm@a?;lumGsz`eznvo+HaepUV@C3ch;bO1{9G zfmxwCqykGV(w)c#s|+Pj7m&3^5+gs%)|t(l1jrCRX{#>FnSvrS;Rv} z$RMN@G=e|UpTXXaJ?~KL!R3F2Oyo-(_lsU=6qE1R5iVn}BuC=2ic{+)fzvNh&O+}{ z8L)yN$#iEB4#UZH>+{ki;iY8gqIK<(pXcM9`RGiPKoK3~X}QO})9_KHU6`@2p1*TJ_s| zKPY)wY?Hc&%_!+#mIVd|{wnD1a5L}3o5<7C>OVx6e9}4Z|1$=q5a@$s-MC@qNE8op zcJ$DuB>4KV|OvmeTm%}>-pAPu)mc#!=H1!hz~ zWXl~Rl88E<67*c}iL^}iiRE&HgjrH3j`{HsdTtagH8jey0Nd3TY;mG@u z<_ni^GQ7})pZB?)4EnWHxc8M_K0pG~|4TlJ3r&M^e&DEm8|CEHt~2vtmf<1wMs|4V za$fm4kts*uR>foAzb2T}0LsU1tmUp1i`b0XTeK#QJM)J}pi!xA$F{eO^+w(~DuFT` zCq1M$G`K(y6HGZW2J2_VI|OxWooV7Esc&h)iWW^^wWbnB&$v=PZ$f2f+K*e3?V|^643pen7~`> zB3MA4UkU8ik_Q>JZ)A17YY;OQTa~ zvD`f=PswCKm{J)ET0sSj(!%o_QKghzX4I12lV6QhPKrh)AY6fd*_X@c4CR?TdI0T9 zK|!0A0c;dnB9|P}b!U!E8BA(15%4A|0RLX8-0sP0>hC?8_NbduAf*v!;Sl?kyYR6Z z8Af6xZ+t*sC<%DeKX29|``!2F-V3ibyE(770h5SU=EX zC9M2&N{((%1BUW&iOU~c(1f+xk02s{|IR?~==C-|e811Y2TC3oyk z>4zo!8qS2|lPFP1pjflh1sTe85-j!tA!Bc%-DJ4pn4}R99m6E6qiN(PG*9eOziu+e zG6B`x-ir$dl+q4G5vPm43Mh#3XR7m)O<=mT!FWdoQKkN(p3kWoD7+ee_(=mw6j4#@--~RKYPgUPhm4!e zt@pTy3OIOLOa96tIK`8I95Wk?k9i0-2y-6Zps=fDJzQ7yIFk4zV9hb9(T?74ngPufdf13>NcD7u(O`6o zg+sxcX$O<<>pSWbrq|-!HSml6E@u0~OlJ$j2UwBXbn)8;pwzoN65U9L@TLBcaU)O4izHN7J6Zj4q_ zZ^h*8h9~uw8l$%C;)6T6HF)H3IJyI|55mKy%~ZCkA%i`5mAHmK&Rf@zV4~&K2vEd6x^32(xoR$1zl^M~ZG(hegNWosv*=7;iq z+1DQVuPLr$eY2z!@Oxung1z=PF!l@&6or9Jg9yfb>Gxxf z1zJNpm&A<>_=Z0ilCXu5v>m1LMd+22Cvnwb;z&-bl1K`BLNFs5_oI?jDt7)RWMbG# z4jG%Pc8f8GTS3CUyE2gAAAcUUkO7Bv&iI9|mZ)-8e@ye;3A-KDcaCQLyIm!kD1pBj zPm3af`(`tjqT>rZ$o&jPb;3u`@-HZ^j;qE3(PYT?2yorQ1Pv z*v>Iku*&~_Ybb@gMk=0Jht|o%JyO)dF!+$Qd?^FMr-un&R1!6)t4^8E(*N3Aej^QZ zXdV*Gq6hTZmb#Rm44D;Osn}Fg8)DJge?9OO3{~%ps8vdGys(sHQr_0AN~`prsR0)( z{UHr~CzufiZknm1wf~lc|0a-+f zjZ`^~h2F7xMQt7IRID+mi#203yHi1M6qh*#b$fp-JX59D=jirN=+_!NXNk>3c7}Aq zKG?F4XL{wpY7FOUfy{s2$xXs@?#C^BDD!OZDU?=<^w!jlMFGWzGj#d{lX^X1$n&3q zy%BV(zpgCrpP(OS7i(1d0bci>6HVx$Ox&nJEG&vJ%lR8n3e9`A@hJ99BOn4P9E?j# zY^0(bn`xAXO4X?3JO9ehZ~v15iG=>j&d(^@0pD@aA6RC2TIb5hh*tzj#f2F4`pOD` zF6hUJcT1jznl3%;O+;S2{CQhXYBB&neWmBc$rr!+JU%D~Q zJ|YOpEd(!UY|$)L8qzf=ad5HPT5R$M(xPHXf}*I9Upp49*Nm>mGFKPL1{fLP)XBV` zrn<~x2<$6T7594R708{3j!y(V%3Z53JEP@tbnPE%xe1Nkni(gZzNxUcFOP!snZz!l zasKsj(~(=a;u9}^m>XOvk>JND1Bx}am!x5(^Xb+xY-z4?`OE@Jy zmbK4#TCF5h;$z;6E1ve)v8aEwizH|YZ+B1>`>?tA#~-w-#PZ}Budt^pQ_Qx z)DX`81d`WCdX{w@_IZ}x93x6|tIFbq?Ts5>eu!(7au4IQ0EA8_3>TMs(ujV6^X;*R zvEk5uezZFkADVo`p|2!%DK`@s=-~K)4@`7FB01*#!jfrQ+>R|jiT_FPC0?bO(PsepHzv;!~wIRK^?iTW;OMyY!L zi*_wn5=RMzwW4)p0GAcoM+7cYzhpAIK?AW!?*lDj%X|cfgw#B5Wk(p${2|ttU5V0FAqe~*;ESU!8)aN) zTWw*MlN%MjAMFFJA}wR5o+i0MJO=5VIXqY=s|$mUfhrw&cs=BvryDNGXuuq73>0*4 zFm1>f`IizTeMlwQC2}(m5wI9+n+pHMRV&KOz0hB2Vq87e@@}5z{rSy;O_yo*-BKW& z7r~(eB~dF+|+9lcV)P(}FH(*MsNdm`{VF=@zOHHa3 zL(ah}XO1YRB~rR{2gRH3uQ>O==#uuXC8P)xetV4iIls1|p`K6|zrYV93RBHFplCza47fMU)UgFNPjdHN6EirkQW(9Vdu>!|YNOq_B zMFstscF)tbD3;&TxL)~#k~G}UZtmiA2M_ub7+?*#YzW+U#LZtR?rBgG?xeQvD?%BK z?u-oQwlERIqLyh}?+zX&A-v@}<0nX+hzb8Egx zZ-I`q(A*=yN~mHOLeXo4A{{K?DvK4LDw6fh#FGJyMC1=ktMgodd#xzp;z~?FBH3a! zB}yL`J*nk5L`h^mbmBNln-~CDsE#QBWnOq2{t9?VbBWh#eUPJ=(Y+nyBtR7+7W6|1 zs9JDar%Y?D;dzofG~QlcNw>c0i)&50;S8?3`wZW}_<)MkceP&JKk|Ct#9QD&&7K_3FmlXWjXJK=Xysi4xCOa(Q{Ht_1^8*q0nd8z2}_~@}pmPZB7(AT!9y2 zG*89Lz)*O|nOd(fq(% z&s*FY#&gr9vrm*$(AX9ElxU&MY>Id#v>~VH*1h#{>oWONoE-T9F`2Q$Pv;ZQz-0Z0 zi+B04-Y1}Z#c$0%T|*2Wr!Rn<5&ue^_zUD96_Nd@=AHtaeu(o(`fRIBR`3ty(dkm$ zFu#tTBbAunwiZj`y&Wr3d!N9XBKc$?sZYx2 z!KEW>Q*^!gvp05IM4ROI6TV3vgTFLVx9gRvup|fxyKpkJc`mXk5T}Ei#`W`PPB3|Ii(P~|18PKvv*eCXSuoHI}^o-BPLLANuWCHjBx|j2->bbrs z4EK`{bp2nv*ufh2vVPnT0MVojH(|CwGkMGvP*F$^kG%%w%f}xUt%|tlE2WuougX3W zznLVRaO^TuXnaF_XZ?x${=?}b=_!;w!yju9)7Dm#uWe8_&0jjTodcv8{d7L|0V=T4 z>Vry#BlD`7LtY_C?vP*Sul`Ao2_eoUO2xG%2pO$aKX16sG6e{spqE8>^c|+dnBN zWAAvRvn$Sv;&mSxB3tFKINhv7)5BHKrJrxwMyB$WPRAukUFUvqNd1#*tXd zrux$in`Xk(B4Wbl_4byBikD8umIG$7;Z0gtA`zG%fa_UEo zmKPrgnJJjqz0pVi2PD8jlJah-ueX!lcf9ncTiut-wXdy~nmjMq7!1_XLQmJ~)HTr# zy9!ZlviaH=BG+DuBiW%FBwFyvEQ?bfeSw2 zU6oGzAtnTbfZrH~1%$xSX7qQ8fpzD6C^=kV@HFEqV-NI72F{ZI@)->pLLqo}-IA~X zw(+}{giHv~?qO84K?{a9<5o?kA7nFTWW9x{blQ%-DzZ7)tdGWs4w@?Tm1Ry3(Lp<8 z`!(=R!H|Hb4FMk?f3;rdYtpUVQvkP~?Q2&P$E--S*B|il`(=d+2|Xv$-Y~HM=sdqt z848bcQ#rEeissa;oIH(HtW1-#>cF*hPiY-}$n-2lEo!b@bNh5^ z4b#wAz1GjV`1E;cfXf8+{rHP19{Gz*-lTLbpuodr|ZEi2BT>L zezpb5`93u1?wG`UP8{wevS}^LBH3a-gO*Sb=hWp%jGvL@P#2sbfVeDHLG2Hj6E)hs zAetBEj`mY_0Rt*~*AhqW9nn#dSiDEV)Y@|ASQVo%@B%@(_FpL2s`zO?0Arq5yRfdl zH>jD+l-+*%P=56()TJ^KfoO9JcEqfZVB*1h*{5O=y$ z$~%v{0Qm!`U1?RGm}%-)zN+tQ_^J+$jB^Ne+T#+qr+K}m@PQjaGLVk)%R%OpRkfxr zFYbfnT`Kw};b6+Ao`uI8W@Q<%_-4yLSl_TF&@+C8_-Q?+Mg^2rtkE25j7_T$N`A-H zUq2ogx!+|eHuE^LEJD>tOx5&+6u-h`P(_xlh%u3Cp~(MHb!(;oAQVf!H`DZ$QX!Z9 zR3tdlSMHZ>B|C`zHjtuR^S5JJ5-Hu=enCuM@!VHu<%QmoKj{>2-&Y5A9-pjr8XfI% z186Zi_qv=mi|H2x|;eplGcI{+3NP(oEf*pt3$0C!VAf;p-8SS%ey*i==HGcc;5b7;6DpKkPby)Az#MHshF)Hb3hWfuQ zVy@tz0*BWv!prja_1K=sUWSfq@t>stG;3?PhvuJd(`Uq%j#H>#f4z>x2p0b_-Cn_7 znSGeZ5+40QLCp;f^l>cfT=|G z@L#}Lr^s9Z90h;_W0)Yd+75Eu{q+xq2{LOUQ#L|x)4ok{`VWf<9j*zZ zj2#{cV>dMYrj9~CSE#->rcwP#n#}dp{9FLwtl8DOQh(H*bfSDs9^1ko{Fgmr>j-{cAtb;pfwk zGsG0U(xcZ9C`Y}%j;$su+hic+ulPPjx9DH1ywvIOZ5ZoZz3%!?sbL1HqBsC9g-G;Y zj8hU-!cQ2HTX##pf>rt<<|o>^ud+D z=;r_My|m5E%@bHVGiGQewirVm{Lo0l05~l?@fp@+07417GB(Di(}M4NRHS>yb3N+l zTE-08oOmSn)biX3D)kdThWl_B6?yJulF4@^S>qC}KQZ&=`hlFPSO5~C&r;p`aP`oT zOeif%x1>8YXJVdVas*DlA7MPp=Uzfj;Jnjxpr(+HeP=`5i5bxp*+&>@H!G5-F<#5-&dsf&9a}KMr9E5{RM`0FDiVr!|B+PWBJuQXew@g$wi)>cCq}G66Gw%p zw;!mrq@|>Yo^sta__6HUe$p$>;57xn=$1{CidLg4G6OtmZQuX99am)& z=rZ85;D^Taiqkfs=TNMRiOugB1PKmB@_czK9tRkUF0wwEDE-$D7)7hw;b%r9FA@g} zgI2-9VeN8^n~BGWeNUTwerpjwr=7ODIcJPgs$Jtcl?(G|qNA7$v}or;9WBSU`QN?i zeiNkH*52`xIoSLm#Q9SGFR#dW(Wa#robHeHZW`*tk7cUr!%3VtK0{`ap?H%cX#$1l z=1sqQf^}s+5p-Xx%8eukf_04y?E(UYPIKdFdQI)ZjScZNNW7x5_SoL+_8POb<6~iE z>6?O`&T7#HyByc-dlK=GJ*GR92D^4D1gZlP8fPr!ZKSiVA`i_`|D1&wI~Fm^YZ%V~ zglz#oxLZYLj4pQE134Kyma4LC2u1Er8{>8;`1OLq!nhXOI*vU~X||=8ty~lae>;zy zAzESrGtP-!+9R}rgWcy;p3k=WKC#1>_urXw)jU70oagBI+k0dmyaL$2|CDoGuna&x~^fy6A`}Hy*8aCgsyeJINR2_WmXYNO^7E zE`zT2Kt5xR`6+H-!O7&)S%0iN$l5f?I1&zK_xKHGH5orC9vlgaTwt5ME|)}47t-j0 z*eVYzO9v~Asmf28%`aDd#&|e5biu~%%svUp?6-NS2DTC5Pf3-9=W6M}zxV~&{6L7G z6d71pP|pm~G1{pn?%iSzcZq!}1e!aUu$5f-RQ(NfpfbQGf!h{VemA0Mn|y;D#ssO| zHLt6|D(Wz|>UB5zNb+Acrmvj30n*qxcb6POnYz6_PuVdC1a(_=j>eokJXN^0H6kbJQ))RCoDm|13r$jtQV*yg~C84uOdZxXAP<;(&_|dpO&OH>$&kY+7w+F{7q= z{6DK;>ERHP6@ESb4gEbs(XdAF;$(fpw2f*bzF!xb$T4nnY?cl5)zYSGgF2A=VZ?AD z58Y{RqGK7`%%WBLzWEhld5o5eGiawNqKAA8E}d5XxG&-Dfp0ZlPY~Yo`&wZZ;l2K@ zYhM|6=7ng54|qo3lRD{?4R3i>{VkfI+JJ0C+DKd5FP3)I?Z2Vi{IzMBDs*L8j01?# z832&&k9qf{6cChhK;64$03qQhdxzklGG-pcV<$3Ci z+eTjk&Kmo_KfJsqE&2fif3IDSAY$I_&L^3R#r0*MUVd)|J;Jbzv&qQn`o;c9(g7Bx z8hbHwNewi>J^ThykoE2bEjCoxto``$fZ`|0AcdHUE*vNJpLLRAL>N|w zzZ%%YgaIB~MMz;{`b9Be=1N6h$U88-f0LsJunGe#aU6PN1RY4M9n67UOb0JK7{I41 zlfiAi*Y+l0hbsWxDk<1OI@hHxT3B}T*}FJ-dWJ1PuBg8%UnV&`u% z$(ztA4x%879VPwHHsITY-+Tf#7$=Wa^f!Hl&@W*MI1#wH5s_CKxa9wK#2xU*(}J+H z(b90h;}g6G+-GVuNjRfc)P8^+L=w)&PnfLdl_c10-2MNF{C|lYRtgM{g0BJW^ZyWO z{wJ>ZKmKzS3syFn>yoaS%T_qBuduX5Q%m<(-P2aQ-e-k<+&=iIx;L0QiM9XU^DX=b z(a)s~igZm;;VT+TmtU4vb0dIfSG0Mt7dCRhUIwP-D@rl~UhKu75R&LFCVXY7hY_<& z1-ljsa56;&AbWGg{4944C9^$8#)u%%z)ehQ)xeWp?YG4t!ob7#_O?bq(_^8qxpf$VUTg=Ku}cm;@GZGH{qU_p}T32DPO{slW<9vjSG@ z*li}E5qu&B=wIgx|5XoSpQm+5s5=NF4;i39TJ-|iVfzU<)KY?q9msgzYl#fvtF3Q$ zeMb5RUo3zJ z6xA@N607Q%$A*gPx}s8W8xBB=_3*;u3~P9N#1IHk%UjaxZ=V$q6ql#65%24}|Bo{~ zLF{Awc=BE?2h+%_fT);hq28{lvM3u^4Ei!+Upa5ng1i*z6Jpp%ag18fa(gEa+JA<@fEY?t)Dw(Ot8m*fArVQ&e*D|q+TvBi?zD{Yw_3Rl^0kISS8I%~p| zIT=|~c2lXKelTXDdI^+;f1o3FsS}ea7}sEWzY7#5^ZaSoqDrLIDc{>WSODxyOQ zj-KZ~I^lkL`6qI!rWn|&3#R{@ttvfCv8PAaM=kDDo6{rraBk?U8Diz)#qA%CveoLa zuB#p{UftLK1<;Q8JWEMQ4Pa|DyEy{!Jc>JG__wDA*OQfBNO%L-q{PIAVPYX%-xxt{4B5^Ey~kQ$-{C zBEYsh!6Qk(D>pjEG;WEBOonOUdMCQ>PkWvXg!-ATsoUb?51x8W3>Vwd1|`bCG3@Do znj%REaKm6=WIULs_gh{g-prRS?J%dmXkJDbV0%y0AUzsT#fvmJn~6e-*QgAn7vV6^ zX#e?MpBLNy8SW~^tZPys?5R-S&x)wXl0xO{-~H!21@bZ70ab?a3`vHQ!} z>iZT@9SDdGjzAih#>pibmmO$))mUqkH&hZ@yut#t(&MX$t(sEVxtWxIc4w55mQYU?e1cQ{RVVa#V$s zAZ#%A5hYrjm6dg1J-zjG5mO`ORj$d+ z+t&=y;bLN9tHa{!OQRB5&O)P+H%lxK=`gKaIbz`L4@dO0w4b@gPlzO&dijUrUCvQP zlRpYzWiuu#=nSnZk8fxk{=p$e(+~<2HWmfA@g%{1!i9tFvyI{rb*}!&GF1N*>%kjP ztVi$6AyG22`)J`z1JdFu_IoyhMhC_lTG(V1!c-0dI%_3<=CE?rS6ES_ru^YR#0x^| z_TLQpuIeIl4AKAAv5CZ@ZL1F&Cu9=P)!*QEDG$Xf|48ZuHxi9^q~7`CEuU^vB>`Of zz-~#?nN;c7SX_UwV{aI-(%ztptw%BQye)~00o#){OWd#piDvzcW=$>${n4>)tAOpg z??hS=Mn_lmbWL8!;V#Man2QB)mqiKuM}ZBHkgSkJM?4B09JA?J05N!)tf%T+*;fA| zeXd@UDeJcBh}zH!66M^^)ndE7$5r#Ned92kurDPdzV0Cx{e6TmVZ59$v!#)=9co=ZK&P2w#+ zFa;0E|2rj>xn9D)IZghu%h*S{vKv3CBa?;1w_O`I7ZmWNr z$hE|7Wi-pJnr8p}7I~e{Dg5Sb51XCm5eJ?xvbe-tnt0#)m3W}bNLI|nz^SMN(J|H$ zuow`miPz62jbI=fiN!M&Snw0LPQiMO#N(ps<)J4nuXddyESIg9-eOvkrSZEeCW>-c z&5q^XMkTDAHCZeQ~B6nCFLpsEIAMHg>o>k58X_%@F#0@ z_`cV2=yzRD;$CyxRalQ%e~Ucv9F!6$-o1bHvoGO%H-pLh2dYAq#Rrv=p_jjZ3G!d@ zzz^!K%EK9XozWy^B*!N@ZHrag96Q)6`&5(eEb)$-qKNB4_pe2S{QFhE_Cs_(pChpD z9Qe-+(u9< z`MI=Ap?bc%yL*RtXIepac2)n(MMdgBVKI@mp|78%KUKV)iM=kl7<4xuiee3lgSOdE z{!C{4x{P_OM!l7ga;~;|ny~Unw7$O1_|1;S0@`M0kyszb+Tj^or<*cN8(>>Xp_7dXjWX>ymwPJVg zhUFA5SRF#ml{!3H*Qr~(=TSP|pEHgIV~jbtRRWG;HY5$*)3q~y`g~+#bsE3b8%zIo zwRf}?Og{(o9pZD!e;mN01uH<2J6)(M)|oWU99kbwa92!vy?WXO$C2%K#R+Wr?XQKi z3<=vzDV3B`%r(pLCuw;~quRLP4?>QV^WDAi{myr$yDsPFEViqM+|69+zlW#Z+00j+ zY5tFCxC5E^_bIWlzd}NgH@CKKRlg-bZ5HcT@2(DmYpfUi0=u>ta;Be@TP6l3n#J05 z_z(;gWc!M#DdzYYSV~;qdFv`;ZG9KLKm;*=4z~EjRoF%BReO@a;?5J~{@`(8BBEGB zgUxPNv}+Nk4tNbMeG}OvVa)xfCl8nC^v3e}`>FwD?He>>P8aGansvf+TRPM}c-%m)RqTo_@ZU`# z0v5+cF*SSr%ug$qeKL_vF6rplzWb^;anKrJF z-8Y*|<`X>_hfNdvdEi}iH;C@X(2g!%Am?1ASiD~vXq^3kg@opNm4R=G;zP!X^~VR% zPa*Nbh;&spiQ{d+cJ`Nc@`pB^RxS#srD zRA4J-tsBk?q~HOgrF>n+8MNbY2uomz__1f0*;!;7YY|HepB?h`^ViosRbNBBs1{uO z*xqQ?*)CL6-y~Hnf2^V^uHzKFUmqZ7G4~rG%XHvw%CGY40HPm=RY3+V_ zDY|0S1`8Smn+A8nzeN7mJA%67BWYqEKX8#n=Nh}|X>YYw8RH+1=WsHV&CX4ENZ**=*ne8Ge z&I+vky@PWe^60axP)~9BVUlOLL=;;Y59qtAfzt<*$y_eb88e&Ta@XJlY)l9)evPaf zHZWb~$1+F%sC2S`M+%J0;E<{;)Z?1iB7F`8u7NMP{NIWc`o_pYObr`F+rG*D^|9LWl2xiNcfPOEw`;=9dZf`WA5Yx- zi|UoX?vL7YFrO>$(iz*tJwj^Mh(eiYKHQp^R7TaFnN21#Mej&k-91T7oV->2=zdd{ zS&mOTdC^|r-#TFXr&J&|Qf&Xal_r#T=9S@jbS;Oa4?^IOVQ9CTQB>@3Wy$Sjokh)`Eiq*B)ATT=mM+-!u6j#z6Bq}$k zUUcbv(7v{J;5eGKe5Zchaz$+k@Y`&cw#`<+@9f&|1b!e6;R2sEAc=1lH*$!Ns^2SG z!v4{DHmz(e(!XXhPi;v{4Al!PQ$`R4ogHW>A5SBxI30Y>)RU9y==LY=*&?)QGk)66 zM#)z3x`Dws-_97=o33Fr-`KBrtYXd9V0XmG8FfX@ive#=t~|dYdC&ib^Q~(TFDd&o zk;&m=Xcp9tFY9|$E~AV9H{W}9jD9{(0~?yqzR$pPUUJM+jg0&IR7vAMv>z}uE^Z|T zY!1)JLGl!j8>Ot7dAN2Y??c3=Lnt}x^rRX;WLmuEznDDKkNKiHHPP&HyTmj*TSkSu z1SkhR{!4J5M~Rr=;mk_(!iTfoS{Nm~p(v|+bF;7ayk{q`8za^xW8?xUVaRMUZwiz-^zWOr`R=uz_y3XgR$*;^U-aj|EKn@CQ{17rLxBRp-L1I06J*lge`YRbZt~^0lX!<1aV>qW(= zE4(N3G@M}Xajv|zUzL~Y;@uUq3wL&QerB506g#qrAF_O{$)3`s#hzJpi}>6-9h2>G zMW;eT#QBG%+N#dst-Ehu3JbQCG8a@(zK_-7^E>N8*6j=sohLyvafMKB{FM1Y&~(yr zcs4GL*}MY8W%@+)^ls)dN#bsTue7tpkUB8Iil2!%asAkzdpjbEwQE*|rOKtfmuq}K z@}a)SWntNJj9ojHaqx9WuwS0F^<$2-!3x+x-R;kXyYA;X9KWn#%^KPCqA#a|r@w5n z;Zai#53c5uyk1Ge+__lRfzm!UXb1?b2?CMo=ED)b<~#GAGPcMH35{3{O;W+4V| zMLXlgqSuuGqh7OqtoMEB;Tcgg)RpidKC!k6Rd6fh? zdfX3GW`v^KEE4Sp`3iz>w}V;DZXPZIQUB%Q*!TFL{ zVeTf+!`#o8kD&cu736?5uVx=E3D!Rx$8K=6`vU0P6eObT9!y;`j=HCaVmnAwCo$tZ zCGUh&$RefFAo*lu23dYITj4O(HNDVj*O}`5j&+cW6JVh}b?l^@?DX{a21(Q>p_ax& z?m(lrc^{Z-P`v>Mo73|F@2f3@qy%}p7*t~uj~nI zkkV(F*w`>i`SN4G)$e)X0_^YhLremn@UwVuuP5&}$>VE}Uc!gw-sPYZAH5Y%>d{z0 z=%y8xtx3lvF+W;cvjVbDCPN>%^QwiWx05ckjNi-?KB2Lo&ut_R-yp~AY>G2C z{HHUHlg4>#SHWH_3r@j?3PT`DzodXkRu;ppOHhIZtRnI@L5_Swj=^7(H3hB3qr$HZ zteG!hP5%bGAgX`oOG*)-jswjSs!Dv;3D$6Q8a{QC92y0M2;YIqxL#b)`BheL{##Y&m$#r#x=|qG+Ki*ApG!tub!K|_`I7W z%=F}h5J#-;j*8{Z`I}3L^+qINEaIlB#YTds^pzYDOg+~bd-h{%;6q7qQD@8vDCL3% z-jt;@UbCPp8i(j50}v3(|4!1$zmpWw!`)sj^l>YROIzhl601R>u8G-*05-z4@pbLx z`F3#o=6I<(T6v(P+y8bYENtM7QSrO>wV#XmZBT7<%EP<{mA?*aXQ#iJRa!>t?y^uC86rs^1uU8bqv%jd>q}Un`T{byKf*km&hiK~9bHr?ctE6k) z(2&;WQwWg0ocDfthm3(L{`bMB%sQ{^!sE2PiN{ZXEsCT7Y!GFLq2>SLUo%M@hKM0P$W_X`B>J=ufjM1X z3{>)bQ~DMt(Yh=kHxn0|IPH434ySGUJnQlT zg(?9G`Fv!KCHLS-p7WE-zf%xW!i^x3OmQ6cW zUu0^?VfAItaz95CN5*r*Lj}YQ{=4b?NGf+5bJE+_^OlD}1|ZZxg|oN4^_~QP^Y)&B z3YD+nv!fi|(tu0?-lvs)KtrWxs|647pdzA4ru=}#0YBL&^^ykCd7MU*F!c3B@ugq4 z#MT`|W3L!|V|64v9_z+&G*lhqP4(z>XVC8*Z}Q7|uZNaAo>GMoP9%_vbf9iV7;&?? zL`w=S3OL#^v;(%h>VJkQ1WxW~*BvW=AnN?@I#evIzv>QeiU6DgPl!2N{JX=Zrl%3% zP##}Juyj0`3wV?*y0(B+v#?5Gq@D{AuMj!fZJTYJox|+l@5RzZ@pLG0IH7`}K(Oru z%Ey6$9u%%u1oFbi>HK!lM4VwiAB_g9@kcz>H2=``FgUq``~AoL(c=$5q#g7=$49ni z%;WXSOR1rpihT^d!$-onxGgGQ@bp%}6lmta(!|^DBuX<~_)n%|J{@E_`1k!rbtArP z?vlRet~x6-OJ6&vi}BEno4MKc@ZPO0ZOo_0GJb8}giEPcT(;FNz3Wo~-D`h%?Q#O8 zNUsjYYm_Fb7f7KU)D#Q5UR^A&<&{ayKx37VZIsN<41J3sLmX)6)uuqz73D6unptZ9 zITVAeI)OAiIqy{o@6y%&7QHg3tdXw8(}3=XN1W}Yq(hFa78*CFlBYga*_)dvU8qrT z5V%8FC|9Uu>Ot~_bm?%gkga=r8R$*x%TN6zft_2{99;{@jzfdyuXrL=s6vEKqO+V? z23A6^SUb%vsvSAGbJn$p7W$2riU+yXC=8CKV9%e?CNUS_sGG^L61t6QmR2VT1668u z5`%OcGgQ31_%R|E5SPQls?Jakwo$saHJYesnu~D!2+31^0sBi9C4F9bY|qt-%h*Lm z-s0TWC4oK~j+n4*nHx?lSyxOm@|m?CYgS#DA`sr4w6HXuE1LZ*{qMfS_9euG;>1WpgF@6M<| z#Hq+Pi^pq$=%|xFKH~St_h^#pHXkNW^wc%FSEc?n@ZN)L>M32@2|SOJ{hQn{@A!Sc z46a~bAK%Nd(d}uE5@$0@=o2|T9WdFcG4M@i(zjtvi#VcEpEcTw=bCN6_*?nn3+ub% zgcd;2NycC6W3#d7epRHFWpJimYDB&{?%C#Z{IXtF_33y*PR;UIKW$uQ2_(0;rMKNr zp|)}oclxm+Q}-Hw01+|MZyu1`-`}5yx?pizQDj3YjBx!`B8;>nMu*8Furt%1E;8)> z=-_2Z6-C^`etq_3$`C}6!ol{gL=I5!{qGpiJbnrMO4ik-0bVR|)>uS0yL8vPyHuF= zH-moDByb92#@34srx(9RsdX<+pS`q-Ji-dPx;HU<(;Vi8f}Kw+n8ZtM%az^n;R!R_ z)B6K}#dBPdtE(e4rQL@Ur*>%Zt*YZ9$>lrGt52bH_CZ5&%hyDmEQGd;yS(O5*EtDz zzkGFGd2@vYhX(~`+`*?js_5trVue#}Txbsn_isFs!m#q~Y6#8sqD_|-(7z@oORGN` z88kILEf?GJ$(!w;_D22MYR$-yzYGYjLeA=DMQID)5aANfh=?&HmV8?|>M0(IEnlah zh(Ee(ZGKQjk?K>aAX}7qK3|7zvr)59L8NCgEVK^Lc|$Xzzykg^gyk<(unbC4`+;5i(? z@N&8~{39n=m7Ok%gVv{tMEa&cIi6C7tTKn(R+6omTs3~SpiCCb$d@k~75>7ughuXt z&7FVqcSICu>LMiMYkTfPv-FnfffF66?hBvW3p~d&kO5I!trtpAYM_*CIlyDcCgW~E1h0?7pr+$6(;X4l3%FAe}cd&}A zhd(}bfeS*9E@Ph%Fh-Wp*%J%~nyx5Y_)-{+_t*7V%)%s6?fXh&vbCG|?+t+(etkED z7w#+Q_D}Iji zZffP`Oy?MCy|F>v$Y3MOIhN)6C}}*6>-G8jyQ@qmm{Bx!!6nTNQ$~wowvC6Zizs|5 z`#Bh4U1?s;XI2H$(Ep9%K^UUz`VIoVwJ7aKHs#!swpmP&>9+OHN-yx(In?cDX{WS* zjF&~v;mv(c5e(upKnH!wHxaf*o#Z3a8bzBxuicFb>cIAz8|3p#ob6IjO-yA+sonEQ zk2iFEzl=XNH_u+|fs131WLg0_e)=u1&Sy<$J2aFKpl3UH8g1NpLo>2m-cHo{84U-L zvLN{;N~kzofUj$CP&!bvSz}1Z8+9%|MiRsO7Sr{-wg0w_768YQtk(?tDdhV6T3x52 ze=?&dqrCu25_qU|ae7y*5THt2e8|3m3g9}Zi0NhB7HHGb@SA^;-CjmuIfqWxRPQtC z^3j_j4>r-TnkD@Ac=*9}WDpN#EoSk*cL2?~F${uo&02e;Kd(#dhygzdtuU z!j}VYZIMDOZwA5e^9dfOZOBMKKP_Wh!3)rM*XVo~0*bKCPWo=DDQO8ez= z(b@T_0V6^#JagK+G|>mMvgIn@^X`!!`%zze&;n=Q*o|i^oR8f|>;~%MK<~W*Pxy(3 z(w}SySIY^mZ72KwRy@R} zP1jMi$Yu2``-kY}NZ*L?DZ z1@`QPc0xKsxgSM#fQv3RP-9X0bce7%M9{N=eBcge#4`~~aE&Bg&JCY<=%26VQx`U( zO*1EA!G^Z?nLV&p*tl(&N&xxDBWz=Ce_;ph{!nwvAu+d`SKo4a!m0{b8hpA8sT|%J z?=Q>=%M4**W00zE;cU1Vqv#9MNd9?yG}T7CWH%wcih=rU)qo-h7t3B=;w;yxo@8#B4F3%d26=t%?~(AO99=` z&omHye2#uDzL-KSAIwD&2*C(?m*aieks?yVnRT+M0Q~Ih%G1Yg`VhA%8Vy~k`rW5$ z@|z1FBPhb`Gd zlA#Aq9>Z~-{!)#?;MsH0+-Ub2H0ceG##HyP9^|m$JCFTNZZFi0<|g~=RfhTpj@q`P**&Fo*ATZLe`PkLcA|gLEUOxnl@e1VcV7Vj5Ai z1kn;EThIhKsBUV%@4w-xRQ3{?AT5ic?F21GpiR8I=`W7P_iv>v5yLx%J%{nEC5O#X zF+J(fmYwHI3h)Y&(}o59xtr-^dSm)RH)nc|l^&`u-}P#1E4xpU%bjm4_O1{={hQWg zaYucBqfOMnN;ro|HEFFj#c!pDQJviqg#y6>AwLQCPf3!w)#ehOZZWJ~i!)0cG;0m- ze(f6r;CoslYcJubPiffQgxH-6#4U(*bHc`uRlr@J`X$oy8lL?OhFrk-u`!h zrwqhxJ4v$~06*T~0!RCFN)C}WDsWBItMEq}RU=8KQFPq}KNHn#;+-hMy2#Y>zN{6# z1?Be^YFu$EgA_OlMwr2%2*vI~WPZUj)Q~8##{kRYfC`i2PxDQ^Tpa5aS?q$p%YgJg zPm3DY6ef-MPL{`#orO`qz+aq__k4iMWk3fMuDFpnKEGXV+L4F^u|4<1s4v*5ZSD3H z#nNFYy1lGF9 z*B?tziE4^8V7Gc;662!`>}Y$U7pdi(`X%JRcbl2+`x{1cj0Qa6%sFy2tt~y;ogiDd zXP?Z-1f|txy-t5I;;v&~oMVO`UvCabJz6ZO<`b7SG7f5tj_&HjCGD5-d&u%mMsJ_3 zj={S`H09Jo;r6>2qY%2$c_kLO*!CBV&3BM8TMNR{fE(h{6<5r25~zTjKJ?({U3A;M zReuHA%L5@>6!B)$PW-`2OOApS9h=zY2evpy9p-QS{~h0^CSLhC_hdo9v*O?UKs&MQ z@)&lm<;!=}NmUvtZk}{TiNOTVsbrBO2&R{c-7&T%&vv9WP=X@;b@2~im9j5-pq&OT z?fufH&4G%$=ZXoNYA4)TN^(!yMdg?b;UHdOil1YT6>PTd?A=hv8|C5EmvE(1Tb_Fb zZs-MDDp(C3)FvSY*j?}gCQY)*(PG`tCBnpwM^z}?NZ;{T0FtHljoThi2t=fdxSg7a zE$pB7=DYM1^b-IMkB=5KxZ;Awe&u_KQ~+NAtAJ<5mUFgo2+@`Z$NW8&iv<;Rrz?@n~u0thfaS5X0+WCv~vAWNX2gdvsb*M36D1p52b z>=mb!kq#P5LHQTe|B9>H|L(vKY4iZ(jgn-ap618DG=Q+^bVzW^j9Nx^$f<%*EFVoj zQ(q{5R!_2b6zAu~O`G1HFPzG=9&<8*{9|jn>R(&oTnqC0F=+gN=Wk{?j_1@t)u({UEOGe!_LBMGf@do(R%#>8gk8N(vB`0+2w*rA{obpnj%~|N z_M>CW?E3PFp%dW8>G2=Pf!59IQFOSSJ8Sf9I1{)|vqp9Wk}L)9+e3J%CmD3P zoCUi-q-h&yG!((LV-bETiuNMePSy6NbvIR~a=wI@Bev!9XN(Y#p}=3h54N^1bcJt7 z_eptEJJfn_x4RNe0A>``sfuJT=kFXp2xK0RA`_Pd^FI_2n<8sO+_hr&XAsS1u)!?wPRZ+ z@6-EheGlKd&2^b2nv`&I@l99anIMOQfW&s}uzksp2|pMuDTl3 zJklzx65iFQ*$ucj1*J=E`AxQ3z(xQw7X^I#CwTIar?arMLVR}{4Vw|HMnT+|jjCcm z2S|L57IxHzIjTGIV6>kOpZx$7Be& z^8scrgxOO(BTqOA<@6Kby;L*jj%}|$hb8^pZ|q+jX?jg-t&)SqJ5fx)-kQ~Aph33* zHOv&5ZbVeS$b7OSWZUZ|E9rC!nVU?n$2VM_Pr=W~;SlY>y(Syv8hPz*dB5f9;+Ugv z=Mc7fOe3`k*r;yV5$ZNs3ffZkFKlQ6UA627E8{sFdDbdf7C`}7#u7T&1@T$8!B5f- ze#R1MnM3#^Ds!0h1`$V9;ie*9Sz=5$`*|C7$k>9(9;B;9tBy^{Jp2O)>A_eJ!1d5= zEF$RiB;R4BssgQO2r$U5!hBG2@EKK%jWTj3>n z*hMs%++iY*L3I?J`J^#pR2ALP4KYKL8SLY7oMQ@$F+=ijwPr^pVA45#34knlj7-)^ z0JYS&Y=5at&VRle%7kwSg6X^glDxDRqvXj z|Ki>gK<7mJnMh1oT-Cau=Rv6|)1L}3W10Xi%2U~RUoM~@VQik9DD*wu6;aHL1vWlq z?X-D0pI+@xS&A(VE*a8o<~BAFq5^4#3d3R?RAu`FU^hH{eS6^(r&A08P0MqnW$2bg zPV)(#p5uLUHETh%dO(g!W;egE!ZDM=CXrN`boc)7k+F>GP2pOhRAREv+}!%V2<<&5 z8-0K@*iQF*oiQC_Go&HMuLU8n6f1`I+jn6p7%m#pK0<(E)nIK)Ad4P0$=-bNkQzoM z-33qUAMG%)U-6Y`QNth4kN6h1+6)%9j*5}i=FqH$Hb=TFzfYCMM}6d_WR?-q5ZL*< z-PnQ1xCifCznXlE%>6?M@wc4PEw}Z);W1PFvFWEmkQCfJ57a-LA0{ccI-KHvZ2>v6 ze63}Z;u5VO@GG<&MCNQ1ggy2l`q97IZiXqj=%_XYMS;(V?KK?oI2QO!&1Mu`ArFrq zsSVX1hok!lo0wmgPZfRWI_eFdceCTZ-qkCfirRPc1*QJT`*GsBf4c#_xf|aI`}Mc~ z?Vuwow%#fMP2>)R&n#)(HR@G8^z=X?nbhw75K*PLDgm9}gOV#>r6U440w=aACEiH> z(JCNSjc=IeHWK=-oDqRYvnE}Y>_A^=+kqlq@n+A(HJAim6xr~#Icr#pEJGng%1ugh zV~&gAFH2Tywx!;mcjC4m^IxJYdH5Gzwl0B5m9#ppGq#bf{2hYiO@+L`ry1|C*VqHk zw5Km?w|9OoR5CntdU{<>rzLA2b7kc;W#&S4%#DmYT>Mb@MS*WaIUD$(--DChF6> z7DKhAp5pWw4>x6_1c6oS!=`S<&^vS7Ca zb19vbv6*dMLH?tcd_nR7HgVY1f%nBL?`Ann+NA?9O6lMW1e4meoG{xL1Dku`uUM+B z77P|e>KIIGBv-QpN=}LLp%5hH`Ylgd_TwN^IUp(Ufae#tj3( zG(@|HgSbVibfs-s&|5`Ogkxog+#=iJ@e1Zz)ZC8=ut98DR_uZ*JxIQXo^hDUM`2HERg5fZ+AQ(%S zYh%H<=A&lM%-u7KnyP@aM3nO8Xmgh9NdfsysuKn)r6L_)gO7`)NY2?aw1Qb z7QIfEN`18nyG$Qw(TBu^d02S$!tM)@RaY6N`rHdvV~VE-a)7(%_%(3#KdrkW)jP_h#DP(uzVD z)leWK0t!}9qNf`;h#&-RLkK7fbGlTk%y1+(v|EeCwSRF+SgT~CrF1D$%HsuN=>hGj$*DSn7oW(NbHR23(9M6ubM%hblw4aXp89KCJ=9p(2UkqlL%SVR?Nhb_3??Z%#y5 z<)uL)>D^B<^RtesOg%XGl!2?Nz^zkJZ0HIE;s=rj#(97b);QLTtA+a?(J5~2BD3c@ z!|b1*+TjPG)QA-br0Qbxj`Wme=IT8ydn!LhP^!T`U{Un6gMoGod9Pp(IliJ5=n=n* zHdN*lt9ru{i9P~ci?muUImGXP(Y z`a-0pGbO?^oaJ=}NlKAQoGe&WpGDZ@D|_KH+2AHn!8L|exsV1x%#j_G_zY$)Pw->M zX|{g3Db*R<(OJ2PWP{n~4(x#x5$l1p4&Ir3pdbP4sqU9?WD~> z##G~D=|Y{u^@6b^%GAPF^n8yN~X-vdRj`|QfL!Ewa1pB+LIr5>#)%?Li?p|aJ^@=bi4-vrejgy@g*=W2h|5IjJL!efM^Gf?E#LmUzc9e zzZD3EVe3$n423NpQQD?FS^%K_x3)cjuqqmOGFVxLj_EYqDJN>9X_V&TOLq%v1j=OA zmg1?s%TS-UE0Kg4PUBWK5NT2qE(}#eN^kxA!3~@%c)wyjRDG=bZKv{i8*L??W531z ziehpl5J!*S`^h7Dm$>JFX)Z?=xGEF8PYBH5%Wrt#x20WO?B=7(XE8Deg^1kz{ZfEY ze(v>Biaswdhu{zJv(BKpo~u$BT0B=qrKdN!R`u`kB8k`eqFci<#q=Xqifjn+Lvp|) zXVjDG6{VAPGm7v7Oz};2(tvSa@V)VkXk+eMTWADFI^NFwaD~wHPp9%Rr~UBsaPS;j z)1IV7bHi@^ur(F~<)20Fb3 zp0fYjHg#!}Ed8Ud3a?!ESn#$ZCsajk<|`Ww{eGTyh0f>dTI~k(|5Mf!fg3w03;{o> za|4$`>?ypZ*Ldt3;c6o5o)}Z*fq<#_8Oa~3-d zU=m6S{W%3{g}O)w?1Xt;Q!mox(#eL%(%7pm%FBh_6dYtp3uNNv8O;l+F(0!sKgquu zSkIe~ziIyJX^!++Ikk8kssUnl<`^k*6r*~3UQA1;Rl=K}M~;nR_FT`3oq3tMrx^lZ z^8GJaxzJ1}-d1;10=8s^_0Ehq$r8QS`tT*P&%?Ex>>c{sB>!-hdJ_e$B>Zj7s?6S8 z?MtDNYJ~4Qsnq9eN;vk+CGg{lh>G4Pi%E_&4;y`SS^dswoR1!Y=$U?|uvh!p zIRdG@(dJYn4PS7;**~A8NDz zB|YUN3~h!F#N;LZE;Oy8&dyzZ!&Ze1g?;?T5G6LQM5fWzJ*Qy3tg*|%c*>%F-xNBi zxLUe_iKp5YZyKefNNxhv>B?4%Fy~EZ?1frlp+end@}NcM@2*SXM7QQ5n+Oj6G@O6U z5BL21yx8SrS#WK9PBTM1veHVpF5o?4`cQb$`~U0$@C%hvcicW>SnznJ7EE)u4ont) z`s38dcqCHf{e9YHAu#`c3Mvn_ylm&iUE`@jn| zuOm6l)H~4z9Vq}#*?{K;l?#M?DH8IB^ym)Hw8C6AQJlu6Xraf00^8iY{Dw4VQBa^( zgHWjnI{ce~3iGrRvHyPz%1Gp57ih9hQ1CHUFgXuwOgl?HbUr~tckN?2zv&|+F!EKFpXuH^V?Q!aVQz3hZ=974K6rZs8U;Ho%KOU{Ya>d9;Y z$I(G+XZ|v25*$w;Pn@8px(Kj6r?z@)j0};&DT}RVdx<2^Mn;hxm3-1h&ezR_nH@Ni#%BD;IzQtpmr zBeR$D0xBX7wiJP60Z-RDy>Axrbp$*wOS6$5+J63&uobQlG+4;$gAM(gT&>{dtrfNM zte+_%2JqCkU8fzg-J?>z)1{!p1zhS7cgp@yZBayaGoXfE?|A>O&?>P329xZ=V-;R! zJfu7(!xx?dz2NGA3@83LCQ0QB44%m6O$Cv{>&*y~3G0ojKTLPZix6NouhK>YLEK`k zi>hy2U0UOu3=scMT-&aCZ)X(jpL~cT_^6eII9Ii+`ak5|_OEXnb)CjP`-!E?cd!Sl z!I>kEff7Q#>i4*W|AaqD|3o zKB_Mknof}Y_{g~2F8vYu+OHePkiJuaV7V3Lx7$&|;UC9-3@&AKpfPN_Zo##l6oOCW+4Bptut^(4e?>~Nj zq+VTUXg0UfOwqD5Niu(9ju!Q@KZBlqGSs^Z$5;<+d?G*A+!2`BunlJjAW({eAr+1N z#kA)ykx?H_H&Vkb3tQyj?v=i2^b*DPI`|~Th8xgmBj|yj;jtVI;BM3d3h>dQgWTtx zb3aXZS(>DL9G!d^AHdN!yrt62sZ$1Ni}|3cLU;BL^S>=dZt2VE7e(1E&d!P!Z0^d9 z62O&!H3Rz38pw`t{$lKqPQx|HbG?$Cn(~_$rdH29pSjAq%~Bv0HTyqZGMWsH$L23f z)^d>M)0wm;{Llnn85Hp4xvc7o_fJme6D7l&NCdLPa>cNgjD&v)4DA*EIs~VrP<_c@(-ei>jxamnEc7i%kK6K0$IeJl;4Ae9$0KO z8N*>aVUC3$d{aQFg}Txk54@zNyNDx`PGW+Xrf;(1lX04llJllY1u+TUSI~|5L|CI> zjPBrOFpQdH!6G!b7v+NR{4M;)-%t3P7PgektbIr3aApW(T%Aq|eehG9_k)f?0AS5h z9NG{o6=Zce$(_R5XRj8ziG($N!iP_Rb>atPYK~`|ryr2(ci7G)aOuncsmANub@7ix z*ChFO;UmI|0p=6b_iY88JEvpzRX8d2cDDhQXegjy$3&KVt2n{gC$ED7vvC#;#^$4! zQ__^$hSa59(zInq?oGQno8{~)yii-hb^(3>T{fLLS0bbzLrKKtuRA^5cH;R3hqGC` z`S>K82C$N2;cHbO4$aYM^R`DNh`rYGe8vkA)XYU4K;Be@k&NxAXOckqE=xA_Y4a$F zmsIE+`_1?q<;YkEK3?E@^QUTQNws4T0RzbRSqx|+yEmH0VB%nlU8`RrttRr(RY}5| z3kK&`OnkHkF~jw2l9+_o3bePVm#S5nQyzt82*6!|%=#uX9L0$KO?}5fNP9@nK(^i^ zR`P{6Hx0YpO!+>C{rZJSEFog!#oLlTtMU1MtzSs#rozgF-gRJ#p{^g_J`O6vzoJYs zT%_mHI_c3ZWJO9XP%hQ7&}E%8!To&{NDsykH@jOdLR`Vd6qw-Bt)oJZY>H@8V+#Z06`)}9Jtn8lXxk+R>7q(IBukP#C)fdd`b?3uX-_zb|i~f(CWfML| zva+F@`u<#LerCrDwf)Y_TwFK^vs-`(o;%*(p72#C4r+d+C$YVvt+eaB4S_BifI+;kkP}jR z6PZGXP)H!`2H5Zn45q>DEpKiLg$y^P4Xf;hC4naLCG83D4r%0izj0>HkNra`Taj@xv+OI`(f0Fr2R_ROlD8{Snb9yT|&; z<-`l2i=pQhf%(4jxN~+dRW;t+rBTE|Om%814=8G{UdE`ts@REkct=wzypS z6d9$MOnr}~`a8o+8=eFa`|{iB`;WfQ(2RlK37g#5250pFAD6lgxr0Ou^#v^N|*|M7e=CwPsi&R zRT9*?IP~R&uih3UlzjdcJN19YpJWN}KAQ_s)oG8}y)cEv`*t{-zd6(+%4adW2uC*S zAMb_#?ti-BFL!_oL!s!XKBW7j9e|t~P=kUC%|92oo{1T6g0Fa;&C>jtmzT79e_5-& zKEp%Gxp(5d@&1B-A=P0;_`)f&mBJTGUO6NF=(0Dg+HO?5nlHwFoGeDU6<_&CKh3X5 zKpf50See?NZ%h)w{czSyX2&LfGYAt`4yM43tfCSdqzsfKpWN(UGl*n<&XgjF`|d?U z5zh^y->m6N=r^1bf#EIG`k(Zbo@o4fsPFPn?qBiB9ebLeH{;SCBeQCiyWerVd!wrA zhr}b}8SLYTyq=m&mlqZhtVxb~S#I+~hj2^|6CQfp78z)IH)Rs{jutyRp|O@wsHU6) zffX3)szaLQHWGyB_!avGeRbZ|cXSV@%&tMPaz*5MjltWQk3M3_%d9OZ3rODJniyWH z`ZQ!iD{rR+l(ALRcEYA?jz-xux4f(tf%HGqsi zqgu1i`i#f`k()#T$wHluIv=P3lQ&G&yG*c=8qzN^)zKEOY@htQ^#dn>f6(2jHu%^c zi3a|t6*l~I+WHn%&F9ZLIi}t3kMu6?UR@63w{g0+PS_IuOdeNJ;ZUBFi|-fHqj^)7 zEL{PTb+7gR3Wt}99qd#nxV}Gnmk!TOF4JxYt$gMMn6-$xbM96+aeM=+9Qu;VHaOd* z36!%XQl+oVIXyjGr|^Vd12fk{Np1XPEDAH%QrG-%$x;$<%({b7n&2m`y_O)dsDL1J z6Gw@g6Xzo+0`=V#3dcLMf7QY_lT_AR4^uF1r9PA9NmA$)kKUUsmf-tymeW3#A;GS}Q8L$h{wV+6k?72nY zQ8B+-_rzVZEj2p6HhsZwKYt4_S$$38FT+y#Rv(gK&-aF4`d!!;bwN?($b02}waT>;d#SaVh=>b(60 z!l!>i=8W_Fr>+OZ8oz}7uK_}h+<9rE$o{lxNtmTQZ{cj>55A~&bod|y9ob>zN$H(x z?hHaWW=XII)SofRUc_+P(``JzWcOnlyo4PZzjPZXvUx>-vO&M%y}o4r{|%HNUlTbI zY?bY+UqGMJGcMVa*@~9@pGCWDGQ29lU8@8GXSRkDn5*|6!YEwfo*OS_H}d3g_Rs&X z8W-ux*hya?BlF)eWj=ZWBs`o+TvDnHoadnp7d1yP=L4p}h~1?`=L_!(7%EaZ)jB9= z275%9?FoNbWs@)TUm#2^Z@KjFj%38wgR0GXLL>ZMGdpj+U2AI-`DHt63YRvoEnl|s z*Re0DZso{U;YwG9R`Vt!f7vr6g1?UMLHF1@(}81Ua%Ey-#X6uBdd0 zx`Ahoa(fm*#YnQXbhmud=-%`Xzt4OPD-928SaK3rl}CXWd;YKGw3Y<;%n7MouNzHO zSy7H#f9jB0S&Yolb7#W#dh!Jocl2DzDLti@jq7rJ2H|{%DkUhvCKRf7&DxJ4gvP%o z3lxo8^f8Zp2;O~&~v-V*Mruq?y^#nO}8CB zCDzNdJE9}Z@BQnBI=$;DwN5ccBk%yYm@al3$Tgu)%Z;mLV^ndVD!%SgrrqTBblv2) z;+`K+-eA67`R*k9oaj?YIOz85&1nhZM=XSGTP0oV^+;7ss~;<0(!CjWcI(t|*XgpZ z1bm=C7vI*Epa2dfk_DY`Ke-lNKW@3r1UF6y=a##gEU-MIe|ixFtLR2>LLO8>&G{}=Vm zYB|aPiH;JNZ0tT+`zK>G!?wTuYqZYJYzJ~|0MvU@{?8xlLX2y@U^i~IOo*4>p5es3 zH3aKwb%8P?AZt;d)rL`A!cV^~D<=?8%%Wp-ukc%PLdgQr~ zEiips-395ls2H=OlJ<6tWnzD=IW0VmIM_ndTL~BczLam0|67Fj?wx^RGs1n*P6uF> z-?RkF1AS03TET>BDZXD};-1c+TCN5v5@G>fkL=0CsW^wlu9mdM(#7wW7Qkhw+`ok; zT>rzMQ$T==`@w16w$if1=Pj*QTxuZ7zm=IY`p~ZV?W!Ae!O~5Jw#D4{p$2!hf(NDZO)UPVBfG?5O{ zgb;e@1OWv^=@4ot(z}$<0_3~(d7f{b_pBwq_xyLhb6)<)TELxs&s?)-&z`;aT*GRw zc=h9_YZ&Cn{^*0BC;Q!WADfxCloq4`HXQeftmj-mvLX>ijC8G5mIne^O_loTo_E2* zoe%Y6C)xWpl(w1{!WQ4x(#uVJK%aQ1aCk^q|Kx=E>O?p9F)Ex#^gpZKU2K|qG$myY z1h#!t^fIF+(sq_dWT%1aIR==4ZW;M;`=yUF+rf0Q{WNGwTZD@(X9TfqX;wyi3b?^8yvR-5G5~B*p zjVVA6Cw; zs`tyA<(WOjJGroUVHEKP@gFSCa#!82^5+8aPV(o3y0oG9DIP{Ds`*bT2ks@M_%SvT z)$@e73>qX49ME|UAIV8^Q zD5qfL_#&P>GVAd(9^wQ}8n$n`zDd1RJ}ioI^xqFH1o^7O)O#G4si#VLdFxG$QqJFEjjA)`-R`L? z`nsDh;232+o4?xf;9(CtOx**0GkBE-3gpXv614NqE>9NHHPlA#W(AFew6zXq2VT%h z#)E}tGF*BE#&V3BJbHc>NgCpUzmt4{0|ZMDIS>CP=uggs?>82gxwodH&tI~AZD2nN zi*yTMq6a@XC3(!b;ItbMJG6dYi8%h zr9EBUz7Uc!I=S~zA4`VGtN3EIWH$dRjOM1~7jsaX{^wVuT{rgz%`C5~*)UfeO6Sy& zg(0t|N4i@lXp!9GRV2Pu9jyQE;LWe3E$}c){8sT+<#u_?iGO4NqbJ!u0B@a!?Zaf* zoS(mW>pmSiGnAerP_`$k)>M&gNZB$~y;)crE=neSWwk>6>e_QRx0+n6y@~IOie+j< zj~?TiIC^96oznY|eByV$Gz_0KSEA5sC);J*M~%A8b~;3YD?9Z~I@wS8?_* zi4OkwX+_xV_b<-FdpO4bVMDsXAhPJLH=ps2UEjPbmUM*!SI&d;{NV$fZ%=`nL{dEh zT$U0)v{=@@l;ADvWe^*?{lZ4xRABJi?!SXKACyYIP>4>aHWPhu@L>}kAEQ`dpro|a z=ZSsbcMmTldeMhT)I%k{E8T}A1`j!2qtC0x?9`IFd0_S8MQxF%{@p5Z7*;|Yy`b*v zU`yOCA5Qz5-L#)?`CNL+#}W0q>662$*D}sJPGGX?*Q#R}L&0SJd2`OtrU$ zV_D^2pEJem@H48Y_}tRPICs`P{v74OqHCmiYt2hg{MMW1ZVNfci~KRs66!MGN{?_l zf#%0A*t-><_n&z2ZvqTF@Ad63 zeoHicMu_;qekM>yL9-jba;q5}wPMVb-ug`kPzxW2T9a_ddH(e8Y2%CgrlY4r4p32MP@9ph$=G2y~jwycz{FJB(WeyGdO z9#~yya1O;M^)s%&Y1JhpIdi95^rD5xP+@*W>G<_QZR|(C`L1|4i+6nNH&sU*US|HQ z-xN%!SmXdE7`&Uv*mkZgJ2gmsn2a5#xx3JaLeGQWR~3|=;lq)PA6^5u6eRob9y?#n z>i1&Y41I~qCgA<}LtR_8D^V6NWrB>hF*UkNo)0a54+Poy4iF3r0yMfk`t1Enewn=9 zQJ$ZQs)2h-T6kVc{H8Y)`D3+)L>vOt%q;nq@#1zoEX_YBwa&Y63Cos0pBV(_=1yOO z$W4ChUhB|!H_%L*W}?SbN^~-l=I-UN-K2ZbKI>$T!ChL~B=d&-x9%?ijrx=vB<1-u ze||p%!fP=&z)wufq;qrGJhQHHUa8FNVVV5RZ_O&$5f)T>ON#*DTztJ~^)2?yPk26z zoy}Z`@2(zHn_FX%j`v3;6TpdT-m8pUg=!SV&Qg^Lsr98(Z+R&MY)!%`hYPHzI=#$d z1xUT5dv*9V$PC!X^lDvK`W{9FL>pM#^Y8ya$TgPw*!uQ~5Glm}RAT{%gha{@$j%>H zw5MT6d4KL@>&49UOlHv$%IpmR+{FpW?*OL%cA7uj#XHlLtFiq1_p^3@3n&ayPj1Ro zZx4uAB5IcB=(uL8>woyIB?puim<5G|6KJe~hQfCD--V5q^H^nO% z(Dr4oIA=(%d;=0hbdOK=4Wm}>iJAOlUUr$}UxeSf3O%1peffvKo$~5$+I3ag8}uGz z_^3e1l7X)Kk2EewHVxJR^29@vH+Pc~liNO+B!xuZIqz(B3XwQF7SX&0kwg}u}`l+bp1oR}TW3YetdUg0uu-9v?Usf9}a?~5jA^WKtoq=Xy~@^?UEWjiiy-%Sa)5oCS?K5#jB81ny{?C9%cM&YTD7 zXrWFnun%rZNy`jLhc3k#-^%EW(kK;lwhIOavoPazatPu3atcuwht9;#?I(`YwyjWP zT*w;C(PC>T&q!-Lf>@ZZ)!@lH&ey6c5L2~9U1f^!qL=!qcvCiT7T&M7Vsi21+XTBX z(ayQp(%F>n7Tg5ls<^+~q-~3x=KW@e4;=h0zr6*Xt`+kMhdndexD&d|TWf)ZJi~F?F-&ehHDinPXT2!!m!M zg^$ywrgxs8u)wnpCcqQEZs#6XE!YB$iA4tGE^NO0qvUupze`=Yddq3{>O4XG?0|is zFSWDMb?KoOH2aa$0d&- z`NeRjZcgtMXIgEI-s26`5UsVK*7##ACN-zWxgX=J@75rLAszklF_M;unSLV(vy@75 z55)_e?q9h`0Xvvrt1B>;?=ef-&!C-jqH2&?S8QP0Ie&kc91l4fE7@MRaE+DvN+Y7d zh3gk9>yNb!)ZppkJ8q>97-G z$=q4DTb1;e%Bd#F-}f9KNZ24N)qw;UyNo6V9?f3xe9HP!lZ`nVJy%IMX8LsOT6odC zc(LUvNmetq9KZ%%U%7v@ACK`b#55EaXk~7V?({ijxovOsIr$$@T*XbJ2Jp)T@cRJO z^NhJ0VV5OwGK>tMZ8EVtmKu^&b9m%y+z^=WPqUUCKH7wA40AW0l>UR-*_euvdm&C| z-8tT|V9R;0=+9`gm}-*yDBkgD-l^%@gHy(e+%xe0uJaLi z*O&>z!sdN1Spb3CM=;!x(cjExZ>1NzG zLgpYT44~%I7t2N`(bH;1-fjzHokCsz>Wc*w0AoUg|6s$9Vs2;dgnwvd%-zK#+hn14 zl;s@<05~CwhL*^jfJvvlOc%`F;Z7KU!0Wr8J)LOY*39G8XE)1^1z2P>9Ui;51K0}S z9+z#?Ep%(S8DG7);W1O0x1vhoWFlza!A_V#27nvH%ugbp#=B5UisNAKy6^x{3+C%r z5uqLG`J}v`&lm4E9!@E+TGFO7jfcLnM(I}LC+$a9s=~xDFM}45vV0sDZIX`j$ zPuzGtz9B!_I(u>_NIJd3U103Z2!T2MhQ;3N>NY#?Ae@PIrA!=Q$bduWMtT(iZR!+q zq0H0?Y7Ncw++6h@#+e>do%%(Dvl1GFxxpOqBP4$Ld%b;Vkipcn5HTqX0y3Me>&A1!eR1_8o3Y??zHC> zX$z|8CjfBtxYLu%oiBcm1PzWKL3pkry>Src+cUQ@9IsCJ6&xerJ?mwUVh!b1 zC;_cpVG#A1skw6GJ`C~;`Fz3*41|alxwpi1;;{%|0GNS({^dhThn!F}!m!FBl9+_V zUE1IOyvXQk1musoyW}G{mYO+Y^Y~E?!C&P5=Y?* zE=2}Rd;rM5eyqCS;Kmy{C^j%(^3d;Zix>wnS1oh}F<#R9-|rq+!X;#BnttaZ{ZCzK z28O6B?yC4-+xD-2?N%n*afg*ki+@t6Axh`ku?UC{&3pj3XTzv`s6iB9Ty9tP3ly)+wJyX;!%xn8~gr!xXOANU+dJzBEozF40)-IzY_O09ohoJ3Z zP1wtMLwmhP4RYtdV&hm%NqztX)9mxJz4%`CEx)Y!uR=pXJJ=1pNTRnE4 zLiU1)SwjBrb0_$a@>8*H?xX6IDW&}q8<@9%V3)8N?T*1K!r2GzO9+0Z0^egk~ecD;%05Ik)T z(Uh2+eSBdU8Mif7^raefK7CZA%j*zm9GHMu4njJVs_%THpc5}`*swc|-a->eTlgxu zGF~!CZQ_HQMIW-fNr=LnxkB)oFbi zS>%Mkrar$GS-EJgl3MAIe@AJn|IJ(NB3e4y#p_>)49bUp5aWZLqHcas=B@{)ftGr! zhzHlWOMa+qA_9%A-)M!ivI%>8(`oXJi=7u$J+y?>>~xXY9`5<4!QEoEog$0Sl4dE# zeZ){T4>G-MiJIMmg(Rnh{72BTX=q1$uukv<<-O~EMydtxqf-mrshTA#ZQ*a06gnqj zPQsC<2zr7hL{BII+AJWqjI+CP!@W+o0}3rJslkYd$cNfR@7?O+8)_11I)au!Los~i zo`^jX;smZ+<#$mlxbbVmE@GFMgV!8$p+K?yNZLvf2YeY?SoKsi&{J19R5|o5ACqh& zaZh{mEfJdZ)=_szQr1+JRON)^1_%7rp&j>ZN=@RH&c`Ao*GsT8+1EmWQ6bogmiOja z<(imAp%FlWk!WQQZ}rW*mTmr*PkI{Kfgu_{s0w{1Kj{KHM%0K?+O#m$ z%=2C;_xsWo?fzXFmrSHpui(aO5N;d2J6npmY`qVMSBEwF2JM$({I8D$# z+EtBDpFXG*Xnkq%zMl@31za^pK>NHhtNP)Sa*~7^4(=-du?g=`bJrWIS4mhr4X)IG z#rd3}-1HuNq0iOCnGW3z6P|2v$*sk+PX>3CZ)+eni~zdbuso|`JFj345YM0p6AzqO zv&QP9!4pDA+an$u!)Ckm5;7f|x`O&JDnF{jZ!gxwT?Iq}O~~)be}um3O$!X)gL;DJ zT_n*^SXjfyH8@{l*U(X~_%axI>zRQ6m$@&3^kU_@Afsv>d4R-vdPrgty%08e+ep$4 zImF$Vm&3A(ZL_CB;&b1IOn;K5b2w)*leS%d?~r{snU??9UiGCX=>?-c_(uDK-Qu-R&>Pa_VEnUcnL zTF4B9xb0qf>i^1MuEJB?47tP$+TwIFUBE$Yah$%aZEnAba_-WMVbr37eu*N8zfZNz zRmuFwMZL7DlJDmG;#a9K@rEG@uA23(m5}wW?kp{RP{3PleW^7WdZwV0H50A6v5gyG7XNei0!ON!D2vgM${rb#96$0k4hs93ab0rZuBX+-yL}S@+K1dR}dY_4erWoXu3l9l3uv9|1s#B!wY;(__;xADPEpuQtY9l7mmqDs3N1wH{KB<(1x4QUC}t9eVouBESBZEIe+E<~quxz#9ln~0aU z1+<#ls~)w(k{jER$}vKtayt*1WVNUK#oJJ0-ibB9oLSxVV1ssMz8bBk;wHH3JyY9> zdH%?<}?gi>aTQFZRxYZGs<}w=TL>_fb|X{t(30-d5=6Z8^4>{B>5pZ z5A`G2nA~Ek_-v)QRES0Ns-G=%iG8}o{mCKPNK;u-|F&#d46at?JFQ*irW26r#&Obm z`?XpuDvDbf{_tsb_Il{BE?EeB^77+eKep63u9nOu7z-sIP0l5%glyjVIvyQlsOPQQ1VpWtLsMuv#E zWE)HzJUp(yDttBbTSk<}Q%kCz>6Au~95QZh z8UuKXVW_WW$u#nK$_k=Tb~<_!ymg(#xui6~rP!msN05Z7ZYn8Xl&M}RZo6;z7Cp%J zHK*<~5$mUN7NO*W1gi1O_Uc+SE>&I#*qRVvWZvx}}aUG4he##tSz~bRZoR=+N9QT!nA=4i@fmE7JtMIX@Uns4ovxQ-=LI z*ajka?@(#1l)cpShxboo4Gdfs7+TNO$)N>ItmQQCeG!m?WxcHLO$Fv>V-LA*0UjVWFYH9Q&sG8g>ChRlF zl=Pa!?8|j?5r2+4;&FSVX|7c-1N-Q4f}+mtiLSEh=Pe`(5H~aTQWpLeX8uPR(T3rk zN79Q(o6lv<&S@IjpH5F-OQ0(yMwdQ09Xv2fVWJI9E4enQ^&;2EKoC6app#5D!6sfQ zzkkOff?nDp;=_~Q(XYRSAJ&M!6kW@-glwrEK{(hz8=(!_c=3r^lrSsK4ERV0cdM02 z{ybf4fW=c&U5(0iWRhXoPv^vZ)!MT8`l18q-q5jIX<{dMs!^3A6{2DNobe9$Q;f7l z{F!DRDU{4#k{=;&VP}ins=X*6f%%PFYxsirsRGWs@smGM5Mio&I5>)MDU(Sb9Wn|>^gmmUXw9tsnb`h+B3YCp>%9Tfpjdq2?TcZ5AKZP zjWIxKK{gTyLcuO1_HR*?sw$QAj+O5Hge(RtUuXUb6(RFGvtXmB(woki1{6ccV87jm zF+47=S@MF^8C#+0X8!dANUqd=OOrwvvW8C*#JVi;r208{?Ts8A z6|4kWq6>RiZGSF5wT!5jON>ge6$(9q8LC6K(z0qH92o~6-uKZnkbbtU=|D#Kjl(EX zt%I~`j`#ZpUyftY^gxN{!@g?^<0gexK;4xU4e*$a-*R z&8@ya60fvsdj6yOpt1$=xXZcOP1_TK9kdJ{)^e*6nxXmFHa+-s_wzb!qp0H1v;>cr;YYI;`%FdpDAmD^w+b;D(xW`~Q`B2y_3Sk* zezH$Lr%eZxraktW>a6@y20n6VXdbm!4gV0|@2^PJ+{1~iAHvz1tl_4^nL2$(RS^@5 z?|30$BN0*K^+bP-PFTj^RI?0x!N$QgSpQQ(q~5{cAWeQE?IWn5qLqy6j@`GBQb>`) z)Ak&-udX%oC*LM%OV7uDc`=D~?i?**8Y?U^i1(pzaAFkFp0iRwEF~J)>iiBlGes*O46?7m(8HQK7ZofhD{b036_YQ5-c=J!^G*Rr;7@9l9 z?fX|XUkc<+O@&Kqn*l0&oz?x7`y||_lF@Hc0TOQ&)zv$C){sbFszgfydx_JE&n`3PcGp<)x>%uXp3(V^5)yru>C|9OhK{-Gr z&3?IO_wkyjVu#G*dPi4RX+W;^B_1$jGeLH2Sc=1JbGtK#$&;~=#p7fxhZx+~`#Sbx z$8Ebu^bk>$gx0oht^$*B)^t$|V5gunsY+lg+hKL%D;Q%&i_m12 z0miMkgleiWKb@EkH`P5k9xa==;|b+As?;UpRlC5ve0lhxXNMMMa~FEU&Zv-p$%vN3 zg%wmgvveyC>{HrU{iw3GrE8@0-#LcAMRshcgGEL)vj@V_@rmeKjy51vN0zFOKJJH|l?VAg` z+srQ3wl}&>-e=sQ&tv$5={z{@+BDoqmSc#CQB%y zti=n83RCcd&5e4ma;N&#O(&#=Rc8&qUE779Pd0^TkOd=&xeRJmfO?3=90cNG7|HnK z;xg%1aOMd|HjQA99%si2Sbfc|mCp|ZL&zNJ-~{QPoc4WpBcwn;aNKIxW`Ozk(DxGMc@YY>>3ao6-B)pG`mF1?;v}X36Mg< zbVV@!&huT;^nw2o_}6U3{|Oi}FGw%4XZw95lQ@=Q@;`{5cCD=?IZN5$vcQtQHdZaYl?0ga_{9Iw3$NAPyI26TII$qNdSj} z7EkyMhM`d%lT0mt+p@>hN$2*iOx8=**=1)drJFjoz(Q^Osk}S|6k$i2?T3BIM8u*Y z*Y2CSbreE!FOEU9FeC(rgSCU+L;sRwPVjYN3$6&a#5ZsmkPoEEd&7^DAH@ac=cOGSH{0(EEzkU-{CWr`frFJWGP38Tq{il zCouCrE-TXt>HaY>#DB>|&&UQ+WVrFPE&Gy2C;hbclBJzQEFB5?43~}vixjyMdFo&* z32yxS9=`yjKC9+3Hz^g!*WHWwF^#h1z9swS`!jOdGIrP{xjU(4z?u=UJofa|`~Le4 zH)Y>!C|!o$J0Pp72=ocuE4rX4 zj?U~tOX4d_`s`8kPBLxx%KrxIztudk+0gDGS`lH4{L*@E7pkXV~NcC zhHCmUe_O?Ytl@6eNt*0r=i24XrzJB7krw?pC^3&x`Q#f)qKhOe#-9eE{Q)|nHyWC9>N?-Y z3oA2AQfWQcpqmOJ{&r3Ydh7GX2@Py5ckfy3f+r%48Hl%0o%!>6jc zbFt?{C}E3i#sIH^r3As%_!AQ~NR0ZJn6$<1RDw%D?)C{Ty}{DY=&iN%-2*oQ@4hA7 zlxI^w{Y01=b#eaYRGMLiF%7NqaT1;E^ADw!_rDTXlJk4qqL7t;P$d{4mN%v?mPhK? zsC&v1^B}zt+vHI&llH+vo@O7vX z)-_96(oc^}5@TTWWX{Up_YijIa0*0{{vjm2#rh5{ZSmRZ@SSnM$p^OSldq-0~|+I1<9)Fk7@OhUU^F}0BW zNqdn4GhDd!EK5w<6KRt>2{UxL@f2^*XnR^rr5A^+X9r+MsN!Idz)or%(sn6}1%*NX z{Gw5S1{QB_i5r*Y#651W-jTL=BlVOhJ8gC95lKJDuHqM0nxgR&)7OZ{7^;G5H;clM zZpmgojj`p7(e;Aap&BDqLv{Mu3%4c7x?M%j@w?)pSUrZfVU70()$4TV>_GuGY&&6{ z{o}QsL@ZPPDw_J6n?G;?7*@b#dx1`4#T@i-@ieUIsbg^bGTXjou{c(hNtvCJ5ZGx`O*hNE?c;g z;&nj4m|QE&e=pj!)P$J-N3HH$ZI}q z{)sZ@rA@;BEu&E)Ti8{;jK9ip76BZX(Lt&XE0d?TiE>Upk+$&Zq+e{vk{dB$WV6@0$d?f88wq`yW7+YDhOw2zu=@ zhvs*a5&+jfQOGO!Z)WR1VGA=ta@jXPDd=Tj|CaIYo)mfNmo}2i2KKiB11eRS`LE0U ztFZktZ0-Raxp%|(Zz19Tv+6Fob`}F657ldw}cB=KKripMYs6XO6fo6A$`F zq?LelKc%YazGXGKO(IJbVTV+^Cg)4X?idpFmk`^kpBs3#hc6#DnIA& z{t1}ZcPs2B_$etV*Z$>!*}VJ5zFz^%{`NlsNls`4AmsLv?sv`krD*<-u>XzF|6ulC zHRb24e@IbV&oP0q`rHYv#xZww1~^u#NRaJXZyZ0ZzlV-KFdq&zm)au zZZC2@OkX+rN32X*;Bw-<>_;u9tGk08?&rxV^0ShKl#-LV)=A~|vov|r`7PB8a$BNP z?o-PP>#2!!AG0*Jgu#jqCaO%k_l*L*%Qsh@=vusX;G}fN7`m1xw#^Q_2j)rBrX>nn z4X##Bk#6gf?#DlkIq1b?P0oLkqg5oh){;g0U(aHTT2k_;FRZG{9M3e{9u*z#j3kyb zjV7XjyB}@(?)x+!keu2&V;9N1F3UdyX(clpT2dCs(YayDwx6VjvrLow7^Mi5n`-jJ ze&dT0opS-6CsRxDl(C=8#;8#%0s}G%tej=_WrUDTqV1wg)HK9>*UCHB4~w7sWF|xgs(6eq8kBguvKF*kJZEukfPF4LZa9cWuo4jQpg%TScTM^u?S!v6?rL zjOS$HO}k{LzSN4#w?AEE#lJ+QH?9yk+E`ixMgB|c-A00(N9Ng|kQ*(YvqK@73NUP~ z`$p05Q9!|jvQ(V`$Pq@&q9JD6pMjhYb%eC-%+{Gs*i)~M1-dG9*eXEzdM6B%Kfi+> zS)+z>=ngf%;-jaGZfy!y&L;a@P*umxJQMNPm7HpRW>OxjIgNONSY{_O*n>HDtX#ON7t03O^gD4RgS@72Q~aNoLiA`mF8s z`)TVK&gj;=$rsHCf9w!onhZd>>0L0BBKUJ}?wVtoPxE~a8>W#3(P~BdzNcP%$+rDb zTj#F>1ki{b?@}DTu8>Y&IwHEbP0hZ}yjovntrdkX^vte?$o6Tf`Pgugl+L|F4G?@o zX+t9pK~+(pvSI5l?p~JSf?+dkh3k0N=rd&$>%W`nJtS))K-g=Q%~orW zZt3LDo}o#J*pof58*%7@BgsJ-8GPG;DTM=W(ZG&k)3~+NCDDVUvY0dILHg9Q#L0+e zAz1dAP2P5uC2RO$Q)i#6xnK5s{jYIzw3n{6QiM2Le{rV7i9}Xv^xS2X<})jWYN-?0{tSHmVk6jMQXi+aCX3LRp0R$jN0| zdKMN2cFat6DSvtXgvm5>dh3o%7$ucV=*RiSqwwNY zwZsC>-fw%CvE-KQXUurHw>YSLc=-f53iz}aMkYih*qNzH?Q658qXQR#WwU-;ysw%* z^-W@eA-pC<)-gmJTu0bfEpHG@NKKWQ9(=<*YPK)e>W_@4UO~jP*>^0GBYDq971Ke7 zd#e~^D~#TKVd~-5L|PPfWFh(jv7Go`wRbWpJacITvN6i5d#x%?OZ0h!j%a6DZ^`{u z?{FShb4!gj5F!0+z6yBhgmerf9!Jo)|+9$7m}HeO(7p;b(?V4)6EZ_PtcT&qEWqaO2e zZD6)flk%y)<-#`;Nc(zS-o`bLv7_>nXctKlSW~RrHMqlx4|e8JPmIDvz7 z2kfdpdbLvFp2y=%p`QofD^McH(bs;_M}o758->{OLyQkUO%w^-y~m!;Y^TNg?)+!w zdu|KnGuM1J>fuK)jrfZOkU@>S#m79+2hLI6>;#dOK~*WLy_lHzp%+j#Bc@_DlsV+J zv9EJ}{JD0j@Ce7_@Z}pLj)kaA@^Kk!c6$Z3}uj9uf3lg!p3ffp{ zHPBEKKgm#G;Dsxlvwh7={<{$wLOISSearZw z9FIy*UPF6op zn(#TFH}Mh&CfJv8fsb5FzS#ZvRYdgW^tj$*a~iq;y6@=cfpRUWCOE=bK#d(jtr8-!LP$|2c-D0sA-yP{D-pJL?@ zZKiqewgy*)4|ww)DnM)Rc1eH<(}UB2^362f)v&>^z4D!%VWU}SB_AK5An#~v%fT&_ z?bm3D3YE>gKt?i_bI}-a<5$5`UvfT{A0A6e7s}sIFjQqSW#o3Hj3D% zr)(frB-waAJF4c1mz*2F-V0d(XTbvu`+2Zo2+MvyCUbvfubcZeWh_G`GOx(N!OpRG zoo#N=z{kS;NNV@Q@__57)|4=D`?x|o+oM+7;JxD4Gsi{oU_0|%SPLpYS@QKiU^##{ zc{f63%{4R3d8ZkMZkj>X^~p+LE_zLnEixtc;$H+q(3A_wu0Y`q4dv@sV`neo#E$5p zRW)!t?D7qxK(I1KTj0Y&2|mWK__XQGvmLuS0eon_smN$+&DyK63Af4y5x&Ell=c#I zM(kBZh!(#(1bceB`+L>-w4wY!T9O$q7T-13lg1y&%2G7br1|!ejx7*eXct}4)Z5+h zap-pz_h$3%eO8x9c5;S3`Sk=DM7Dc2QYr#tkev89I9P0Cj?wErI=`y}K23qW9nn$k zl(bu%2=m!m?q7{6PuFNaP26SZx9KWzr?VH%-Z4L`KCb?t;3~biHf!6C_556-xiDJf zs4KS`qfIe6y?o7#=}ufF`mx}>vTX0Y9FNT??ONk$BXKo`(Uj@QR9J>7 zn5?D$eo{0XK3>S2={P^1^-$(uk&Z;vwY;)e1G7fqi&Y{ii}h9Z3lb>Amf7w-y;# zgD!SK6{6D0Vt>3r?`GQ%T|a7s`fV0pd&~~0Fj%Rsg!LgW&Vpg*sgo(or6HKS%1$q` zOE=2QM!y4hm2eusp*;6>tgMI5FJG$%syD|xs?m6Y_!}hn$w@N2YPMc=Fs76HK(SS2 z;{ZX((aO=tQ3)guLLLATVSspY*Yx#Dx@+^s(3MY-XguRu=&ifw6}6Q~Uf&{STZ+&g z7d(t_-A=ja>GRwws9bx+URV~o3Kh+p*WW^9>SO!#NU?i5PlTvGXANF37zm8O)>hjQ zO_I>2(keIqywS|A8uJY%w~{eyG@QO)&uQhbNgeBuTyS40WvgS#OP6lLuRZaDLMid1 zd6}bwMMw9k^=lqCzL^+^>DH_dr)vAB-Q>zD8P5r5;x2G++ZANr&~LH4>K>-m(+Dc7 zb7zsX?x%AQ4XEnjklQ*(>dF-b+K{A!3~XQu649SeKw70d${0!z7+4yG7BC57~I44dL3B#nX3d?aJT53y#-t9%vABZ0>7!BCvKk@* z-Z)mGd&D=?HRpNwH;9gSpSLcXQOrdW*+ z#Yq?fCETjvjSp3Q(ZiV8V`Z*sn0URB$+UKXxOA(-$f=FGXyL5Z>6%H3Jd~k-Ert}r zs_}98M$OxSe!ZL~VM4di!|kfW5cOZt9$33Pgc}#XyX|)lmVItJ#=Pmcyp0z~>Hoz&fO$cuzK%a6mu9O&AxQK4cUwJEz{2JOdhoC{4Bx`HwEtYWDH(}FQ#3fGO ztO`ufGi8r#_ezeXoN1O1R)SMUb$)K)!uAO+w-35FsU$40@W`tP8^1neY|{-eT~;%WcWR9d~H@<`LvsAvL2JG zq(t7%eHf*8ZdlSSfsYDdE?hM+b||#EgW&m8@#Jw*0pjPcV;&OE*+@iS?sh-ishTHO zH4qhG^dqN@1g##x+v2${3Fl9{b&95j*tb4Iou&?+8~TRZOHyZ@!X}u9e2+8f{hQUx zqg&)mT6&auZk_IQ%yr(Fupf;vaq+Hdb(ly?#x@$~!G=a1k5-I7X05EF9V0b{8_CWe z5v?R>`Gdeq^os!N#ea>0{Fh&uHG`w^8@v~hQ!>}eUYxw9r`K$3G1I7>v5`j7klYbG{yd1&^R?D^-Vc(~ z@R=Kl2s60FML7Mlqimc&RZRF4;_rJtb!yz+CF8ra-ZZ9|7r28IPeMrgm2PuyLwm+R zpH`0?lwYN0&UzqwzlK$LF&oruc-YOO#gTM?cdSYU_Ly4M%mLo zWR^skhPFMmojA`R~>diw67v@MT5Tc3nG@$^HDLT$59(PW|rubvjnD9Ix=8J*&|Cq2l;obvmOV8M`CqDug@XV9 literal 115068 zcmZ^}19YX!wl*AfoK8BnZQC7N9UCjQZQHgxcG9tJ+qRwj+2`Kx-m~|){~EQ%SgYQe zRkJXkn$MhXsGN)_95g005D*ZYxR{Uv5YV>`ARv$kNRY1@w{RY3ARt&l)1N=(#DD(8 zm$S15m|7SC0g1s?s5mPr4x@Ec5&OmjAcpuPOr?{zOI-T-De2J!5XZ>rLxQ2Wt2upJ z^b7ko8de$Bd~yOkq>fNiNA9#(TN3V9ON(kz@^%oolf~2e`F@a*`q*xLvG%ymb$c)d zWU>OiccZBZgDphP$zcw+(@{xO0Q2viT~(-0|p|D3Pgr{{YI#Vdei%Z;EWha+z7HtU`dDk2Y!@2+_t={>+FvO zAp{Gij{~w(YgPOq^alzAL%3aE)soyCR7#qdLuf#$_D8m~b|vDz(Z0N`0Wszw#oa1XF6Q(NF_&G(q6E7YPMSn0rKk*cDUb zk(om(zCi~lJ-AfY(^J|akWm#4!oLjG8MPpl-21+5*%R-S5NOiULQ3*;&-fS{OZ4U= zh}Z{S(k2lX;YT>^+zZd_K|$IhSkVrN=P=88RU%6Yl4~2~%MB=|(frsl5qCuMvt6!^RAvf1za%rPDy(>=jA0{?PScCX$C+XUwB$TNRt*@E8Yqeze5>QR zN3Bh$b56p?AURM%c-jX?oX4a~2&;$|-3Ln5hbC@;v~jTSot|hV_i~0qC?x zTOZE`#CZEpGlNxRba(p|J~yUpU-6cj3$JzrU3c>C={2PXopyM3AKlL4wbL8Mh9pp~ zz!fA@PwW+;V2CPV(=s8hY3VkyW8&n!_$U)kkvQ>!m@%@$ z*wLZLJ*(Oz3(~4M_#xaIrbua~JdP<36Zv}adg>Keb*Kuz#(1X`C(L&)oO(6~&P>s! zT=zV0ro=4iY-t^lO{q;;>w>0fylHm~*dX9u-|fK=BS`eX2*d&K0qp^(0TdC4-ORM0 zw9>S2^=9?7G%3qi^}rR}hFvq2xvY}=Qu9)c($|u~-{Et{rB!m9xf?lN;_W}%qf5u@ z^zvA+vK|=Ec~%Y^xE#?r7dZ7e z2{TDJR~==JxpxpXVXPokhP1}kyLraXrwUJ3jzo?Dr%A^jC9A(z^NaHji;~OP6sr_2 zQ?$9Fx72R)Tw2;t+WgzF+Te*IM8P5vqE$KdH$$k1Hcq1F;OEdzlTR)eJ1alCeXJ1x zF$UEmITKr11g0j|?9(LF+`0qAyX%QPeoDHhb10&9UwBZPOO{=OQ#B?E|C( zKL+e};RcE$TBTh`eMy%|&84E!9a8P2-=zzS%%x;gc=u1Q{hwOi?%x1j(W$X&1*tO@ z>1t(a*)5T-anFRe?+1_6DX7}0si@}EY1D4i(lmn_6E)>EZ{rf{Bcl@f4HGBj^BU$@ zr^u()<)`IbmUotU7GaiV78fhEmY^25mSq-I%hTs=XLe_-t6GhN=j-Pr4~Y-S*ftRG z5T(6<;70j+`Hg(T{NEz$4u%}y9H<-%?7ZmXSU5~>(?@@%CkIud#Ifqk#+DhfGuJZh zGM(7hZ5M2D58H-Xx}OWf+ry^{XZF*mR$0~@JIvcpVb~zpYF1I3G%hqBuP-*6*;>4B zuI}Vq&Tfxx>hGi;*Y;dzB^RkC%u^RRtkayUA3ZMfH!eN7J)k|mzr5|A9;NTL4mR1B zkE>oU;TYgOr@lhHH9j}I-oDSicRpIf)59QuDShLHt%pSZb^+^wXo!~ZWeP{G)r!@G zPc~Pp2-z_$1d+KyCPF5rCgv$JstQX3)vwzJ+iKe^+dMxme$)|R5}C$M$CeRVi(rUC z<^Rri8*q(E=c?pLL&-#FXL!+z@QffV(&Og!T6>z`y_Bg=!%vq(gQ=0PDKoh2U+&u< zd}oa1Ww6rU?;E_HxVGP!B(*^*A~_RxiAT?9CcKu3vy5XbrBGvXwX<`)SxuNqxJh^r zHy|C)bYPgL|COemJs>72vzof_`fEwWyz?aGjJL(`CX#|x5g^0CH*aHhF=sH#Tntr& zBT-yfTyQD=Ngjft1m(t*$H;1Q2l+GLG+LdQ@PU_I`Yj+9YB?j&9(N2bt<@5jmCitW)+78`^1zuoM77$Tz+P{nM+c3AhEnIw&30rb zh-5$#mD=^rv2T%uBm+@+c--2fbNPklx#Dhhp|Dbtjrv`alCoXyKuKBt!8X-qIq)*0 zQ{z+X^JZ+nf{I(&LQPzULc`VISJ&_L{gwUx{b=QA9j(@&&LW5L(}vlFtSXzbr9V+E zGA)&!o?|aBa7kFoR!`0HwbyP&Hv1paBI!w1Zmnwd&m&1)G_Q@A)-!nqxNu&y{}$!-HNv-YBRH)hMgKNqc+zy&(*Zr2Tdl!Fxy1bW*eMu zHb(h^K2$p=Z8sa=+un^JwtQ~7X0vC!|Jc&M=*|*U{e1Omd~u!CyV#ylFLzS5l+7%t zZ0Bw0I3H>zl7{Pm3--o%AFa#>h*Cu#<7n7sxOxmw^kH1@aeUsYXn;#%FCRJ8B8 zmVVSe?gus2SPoW!X>z!-+?}MF%P-PuXWO)&i_Ao7u#s(Abdvm;cjbC~2;U6zI&H1n zqCK}d=BaQ;d5pYJdAE9e{cwtYFn|}ttLK(-aXE~=ny%x6{&-rd*<*OM?MIp=o6n@p z6m9KkrR94+F&$)juIspn<7@a_yIXsBp7j3qF+ampTD%*$Bm6mPR0XSj?soq0ru&sM=&AW5e!Vzoayt9+;qd-(^c%GA2{nHX>f{)h zOACxoX4u5{WZ;wxU#{d45Ugf0!U#C!un!2_=45%CBUs>85a2jepgv|03xRoPJybV^ z!OK4WE-kHKD*Y0LnB$W?5Ohi3DALQ9R~-ycdQ2c*ikH_{*2hQh72k=ICSc54hzqv_ z7RTT>-1Toj>Q;nL(Kmd23#P^Ic{IvjqyoxDOwAq$2$kfoFR-`*@f8pdaF(f(s)MSu z6sLi;C9R&JwZ0Lpi>1w1Yak$Q7tXJrmPQVG_%4DhO&qB*gOHarPjgODdZD$DJR1gyRZ}G2xJcK3=4mO;0bk5Gsw9ZVl z)^^5p3>+LBbo7jLjEpp2EokgrtsL}RXsqms{;QM!*N>2qy@8#njf1JR75-oS>giiM zI`9w@{x#75eEv&MBNx;E9m&f6zo+#zLAt-5&@s@`)BR81U!vT9)pE+2x)@og3Yl6O zS=oP$!OOrx&&d5Rf&X~)zeE0;sM`OEvaHcT!d7)#`x|4u__<_WQ1e9EW&oUu=NQ5&#TQ}X2oB&QvtO$W{K0d_J_=F`cVDR!% z#aZ}Ap>7A*6HMz_+j4JIS-?EK1V^ri1cZ6${IDVTIfS)_s!)=uOY5oYsmrO!FQ3Pc zt2Cott{ZL3Y)qSgl8VZYUoT3|?Z0(Ce{)9UkxvnT5aRz|rJ4xRu!up?)V1hGQ5+)h z4}||HDIy?x=rRUovnfhTdAxhvWzy@?Y#SKi zX+?A8hW722El`tZSy7aR!~SzVUo~BGkU1X5CnqNt7X#A8vl(_V2`wGQX2(s4=9Irv z;{-{SySGv^GS+I1B&kT`H2;4!Khg&W2X}LG%P<>o1r!18w=n0XF@&S*&@pLqiq$tc zZR7^ya?0|Tru}0tXN7ZA zscRRE^BiITI>$B=7>B8qXK50}(_bs3J)m%yK$)Am?O`&tfPUC~3sok_A={L#Dp?)|_JpJ=u+ z6zFGH-8nWi1o=&vu2+JF8%*7EFRTljh~H5&GNW_3c#d!+6NDcE?tqpPmn)I#pT__W zkso&b+e)f%IO?{sJ8A+}O_YJCb^7X&A1PIt{qkKhXO3Cz%Mx>F*fD}{i|Ez>mi69* z^Bdl<4dwK(>Hm=(8J|;JA6ZU)fTUvzXj}&*X=3!TfbLC>EgJiCFFik!K zONPe_hIuhCA^v|Wx&x5($pmy3(kVLCzry*}yR7@gz#3ocTiOBT3CMF2){?2^%67uK zm59~BF)Q1)w6ebCfTFb>5yT+4B3)L|T5hw96k_3j?~)(P3_+JQdARQd;Y?=f`_lOR z)mcow{5))6zB0D5#DSPgiZTm(9bq1^ux=7!wFb|2ns!lIx=eA);d|LW!N>gbSyO9X zh7U|})AnzB<72G?Ub%nM_-m+~d`NmxgCye-IPL_vWPglcSHO&=iGX4re`OzQ`izt~ z*a0^%F7U;Jh^AZ$G47scQ#Uf^u+jG+cE%`Ge-F;zY{gfU(SNtJE(_wW@{#Y*o$IZ= z9~D(;;t4Z+nUQzD0AFVoNR~_;R6p_bdxjS<#2uEjg@@9D-y{@P`ygtou;;{*8w zmO&cCe>fP=eoxmhyxHeRs=})xF&-tPn}x7-AWFnq8^vrHZ78U7qkxXfR{(c)r6pJDLgR}4r z!%~p_KdqqxsU-IzWn1#P^=L&c4Tu&CZ8y!>#U(|x^{iI%y!j6$h`mo&8jHEh!Ff$F z=fq|3Tm>n-j8^J1(bm+j@>&BUT+Lg2iw{tE?rMFZZ~sIy{#`I!*S#MB!9hW5txmTH zs_qjrE~s!4to4f#>1Zq|C4E9)M^)^|=xuQRadQ4dR;B%SO>N~s<<6i1?elN6G}MFo`u4iIhUg~8HDbYDR9Q)|2OlBvGGm5l=!HuuVE>~IJ{ippr=Q?1LDbpDR4 zz8tbSrezq}A>(OZh_gnzwRu|t%TC?*$lggu?Og|^-wd~vx%!xHM)r3gpl1S0%dD=K z?Z~zCfeZU$rj?Drl}p6^_$fmNxGS73k9odfAkZ0+U{>w-%3+q2psz9rwD5tKcJqNL zI#zXHKCnN-jl2$L&bJ2lDKCH_u)9jX|6McSA0~l5tKcc2L6$%kOuRB2>f$ZS&9!OE zjqN3++3L2>UMS4o^DG{Ob*M9%9{G907@5id=~-#yiHsm96@VBiEmR3|VGNq=Dm^0Y-3tLI_Q1YuKR!{aj5L zD(i@lU^VsH;yg2fCs$WrG{!{t`fxYFsG$k=(C0v^&6=q7-&??0~sAo*_Rg z2ebx0IwfWb(;t#?vCQ}}a^HR44tjj;gD^u@Y&upL8z)w@4t8c`;Y=&hnc*{(%p@Q3 zcO>Bt2TL(=7@D3LtkWov)3CR_68ov zt3+>#52htm6SIln#4|I`#oS!&eLZ8;dRcRt<$m#JFe^NXRyx?!LM=x0(%<5Ddjj)wsW7SAK?P8uCMub&SeM)J%t-@z3HyeW^Qzpd+Onv5CD z6u!sR#OR&N4P%`eU-f_J+e`{w@iPn+^ms4~eM>E=GQK2=;(pmXFz=yXB%wd6R z>G|Bl{Kh~gcuNF~34}xe+)b-BG3dZMvmkM)@Gv2 z%dkV)npdA+r#&?le1jGtCDpl|+7tHtd8|r@ROqjiOXhLEuuWHiKF8W6J#}U=y=v7u ze#dxpLtc7ofBV$P#8gCPR-K&Mz7Ndk=ulagOwjlCdftTRh4T-GiEs+q_x^Md$miD{DkwlMp%%0ovdWi&VH7+Q51nFccNgI-FbESo>j-Ai-5{2)1ZdAu^ zJWwZk+_G`ksu7i0omR(X^XKl75u@vw@eYrb{yTzc615SzN1dsJ_TNqCzuW^|0Hl-} z2IKjUf8Aj8@}E2peSKgxU(69Jg3E}3aozqC6cx{9Xnj$0zQMJixU7tdd;mHv?i}`j zNvo!VRF^ER5_vR9-yf4(OWs16DdBrkfgjxS(}^EFvGO0)GnryH#^d4Z;a2aD-oWVt zEqHfz(%Jrrt|NI&I%2vz`{%b-Ty&l$XH(2ubk=tgxAG*YyvnPqwZ1yE`=!HrELp`k z9LcSkLEN$>x-r*N6tni9CqtQAkF1rjs4}HVv|1}Q&f(Cc|A1l*$cV^D`MHe^Z2-mv ze%~9DT6j`90FaY|pa<2PJL&N-tn+diI!Kc}c3?-1NvT$>n;*ALH~!FdjR&2UMgL*Y zuP96TF&falvED7fW2?)@%2H}Wj`Lll|AV}*H?EW($q7QqNPo4)h{9#cxrnz7BOj)7wz0>nnp5WC6&b}%(w`1_yyhQIBXWHH{g z6K?+wQk2)pu+IDD*$0I2?N``76Ry_+S+BY$frC{=?=e*=MrQB?{UIlmg{1@_@j8t? zgG4>qC&CQVQxeT0cWfh1sfmsp%s362pE^DIarwK6`?Z6mQ}RbkeY>zFc2P3N>qqQ- zAh5$vkK9sp?`SBptk&I78soI;&eF{E%70!9|3!0bLF)Kfe!?V_%jbE1_r9s?csV~4 z2#!3w(a4_@LOyt+5~3!hJQk(bps<9ENbBRit_@l8IPkg1iN55#_AO>+W%4)mjxcZtU8YK>i|494Lz{0E<;1zcT*fDZ)pl`kC0NEuv)sD1y}L$ zyv*;zv@*U6*a|2`uq6we_vn;UQlxe!Y?SAh3RwDyfVYp$BXzqVmjPUkaZhdiV&E-g zk@Jc#v1!jFEoqSl8M}MX+AoH!UygbLf0`KYkjSjUceG0rs&F?xlW3qu+&vV)^b59i z!e(F60`K)orCjLpBY_8AY|U%1dYbMNdUCsSwW)nxo;PjE;8+x+JT_RM?KtV?>U3FG zYTf_bs}kwz2tL@KuH!i;Xs3^)Mw}UlJk-3reEq z*&p3Jao;{BDz|{H&J>pxvEV^E@S`7XiqFH>fF0ThDOwj#Bx-gVuxJ^!B zyHjDjR0J?2R9B^b0N>8jH+$=$`#A9|=CRwRj%5qTo73zv+<;Z>1d)G-P4AmG!kl{`UyW7pvu0W};C`NI93X*4ez} zy`9(6d7OZI-ccn$H#$uv#~v3&>PMJVC3uK_c!-Wb5p6;hl}`)kwo27CA5I{}qcYR~ z21k`k3iE^OnHI7qi(Ea1%0cwF+x_y-LB(Y2wfmtSoWJ&=Y;XD`I{c;1T=WXKd#cMd zBEzyh`vz-DXPi^eUJ6T+lqqCgYQY-EPu-RKA2IT26$o23=}gYaCxm~;?CPJ8F{Z)T zXXA|b`H88&PHiciRa6$mKVL%eiWwh0uj5ln_hw5gb5iuj6)4AUfs2Wv&fUj3mdofy zbR6|uHgV$hsui7o(*>uIZh*6EMLcHeabe6cFMW5+FBcD7>J13g|3ekOa&=^h_g?%K zbpQT&Rm237%EAR_*eDbzWkHQ3;xX3hqfe%MekH+Y*r1pJwl9!~WDTV`Gb0ozm;$&~ z%*1#~qS0WrS~FoTnYd`Z)J}(ekRT!=BCYdyzOi(jAMM}!(eb$9ZhM+El1Pv#qewg& z=rNqxoM7Ly?o_9xl3}=OI$v^Z8?DJ~>L; z`zF(yj{9ZFN^4shiChn)!xP7Q66zcpK;NH3=C83bgvN$vuAL;@{YSRbdR(h%q$@l?(WWG3lz_Udy@lStuprd=SYLnK0zhQ0Z=4HWjHhAALGBpMEJp01adY4 zTn;J+U#q2_Q?@bGT_xr0M)+|)9^kC`_K31qs&#c08>=bstkmh{Jode%DmiU#hxQJ} zmn*m`s5_3%Q={MY&#sIe_Vc7D*Lrxn16`VQsV$y z7Du(+VST>Q6`$WN{n_b}3W6H%(Ef3AfSWO=q=K zZ?ejq{5&bEs>CU4^Irjg}&-?YrQBcpfcR^4J-M=q1sGf zDxP+E$_AG%SbsLsGX0por{j|Y+@dXXf{jA0yilZn9WWTcY;?HHrYlz^E zD2`)-srVbR+0*SaHwUXZi$RNA=4n+YZY$P|X#ee8F>iy_xmkjtlUAO*L{o1gctcn# zg1JeHB5BunOo%x_hk3mgBtY}Y@jsj;DzJgU!O)G>)zOGvoPdm35a{levMT(Qu&9u{V3~ZNqz#s`fi5wHvLz7G4T+$aZwCIQiOw4zzD=1o``)~di-Ya#Dbxc$ z=#vP*S;=d(gJk)0k3(mYKL}qg62r4!iP;kpSQ?&U?%>$g__voQ3*~g7et8(yEo)e} z-*5TDSBfyw${q9TLN8FEqM{`}#`okaB>?gIH*BfFPr_wr~$E``po}E12fE)b3bC$%a9zj%nt@>Ytg_AU|8elp9 z`1%GG8{$qg_4Se!}ZLRI;3-0x*!)~ zVYJsrkEaeV<|l2j{%ewpv$j|APp3GF0!4XRm9c{4_LJRF(V)wh4OgxAYZ0bvzoKT$ z8Z+>wf|yVi8ZhU6_4E=hm8G?8wVSR1A}`nLd9pV4S{4Cv&kzKp5Wfy7dTvaQ&y%ec zmTA8alAoa0o|@{GO8)S-dQ5_P=-q=psWK@%Fm(>;@+uLpW5&F7|Rs`5on z;Ru2Lf%2fBpe$qdQe!PiMO^)gT*(|x4HX7lvyi&8d%Mz4)4N%ABNV|nPAQ!oBGemXCK(y3$|+rA%)?kB72Jx=xg3A25b z2(P?(i@e|v5IV0bi+M$Vc z#V9O{BY%UUOkBmoX!p>1WX~R_71Hq6C4^RWVo{4ACHm;8=(izEoInlB*Ly_LexCLn z#H)5SkAeHC#8eY0Y09TlPXpjGa36{Ryaf~bNBW`#bn;x2OoMo(Kd6xy|0)E;7MezCw*7gQjlnX_L~2tvTQ`UI_bV~>$Gfy#%1I*?vt-BMGBB= z;jjo&mM*t)u@vTCKf09wY6rTv%$8JKu#`?h$Rl~~BjeTEy#1|=o6e+qI@Q;->E+HM z-h=0Mlv_8+ySg~KWG+%SRU2aIB*>;d-ZdNeM%Q&ylSnKFFYJTg4=I`FCRH8bJ z5#w~oUN-C8YCB)qe=wf@xy$zfT47?zpq0r4#%i&&_2}67x_(*oKr|Fx#bCPGdsLgk zbhWA!E5d(mG;Hm$i{$|X9KXkk^7H@YjJh0tm<*c%{yn4+`VaPdFp$!yA8u~Dp z6VKRVL|M}`e9EiJ7(Z(1h1oK8@!^}_SLLGV2R6C{K*b+k>Dq5nX_qQ+_NYmQOrm^cee z4U%SZ!3fE0Ec}ToHR0OQq~9M_%)L*6f|Rkr4;{@imHqi<%nbR2!J(JZ{^c><)|}r& zy#1-RDE24_Z|_>DAYb0`ch*4})JysSQn^ENy}Z{PHk)ScHk)puOvU4Y4>Xqjer8-x z=ga&~#`iq@*mlUXMOj5qZq;@z!iNn*txVjK-q$JQr}NZKBo_!i4w`zGcq~C}>`e*d#Q?!+b?5WMX zsgkkJz#JfsNv?Wg%+D&gX`eYB1sPoo5Lz@$71Pfc z!YHGMq*|LBQ2ojn|Ly+Z3+U380OS-cmN~CGAHnk1J-1!j5+IGF#c7aQ3v19vQpwTs z0-%-x{3nqkql_P27e+7Mm%l}8yZ-_Jw@xG9E_Ej;Q2==THTK|9q=&~JjJ>XA*ihX5 z^i-(LMDsO7E*P!nchfC`HC4hdMfkHEE<n?^x= zoyq`1emD$XAGg`Nx#*$(HKelzKfiPI1riN}Q$KEBEw*1c?lXBi{33^>9@%^w-)wv8 zr*oPf??paU%zjCDuGoR{RdXIu+B_C5zoJs9^cY~u0$#fGymSw+3D)$u?q`LSo{PE{ zZ5$o*dOZ&oh8;=%QWZ$(2pKW1tMe!?+~W6S7%v7{@U#acXKM$H@2oK+xshoNRJ2EH$be&YeCp#urAm(Ykim5 zN{UBLOBrQAX^=$uhD8ObYKKgx6vo0_6IX-)UZQ)WH}pchAyvWsFh^~cr1ZlGrb>C& z)0jS+;v71nAkdJh^ke(zk1_YDvn&V05Lz`T6gVR=DXhvG`rD(mUuzI(0%JQ5FxFIz zBxS?smft!W`~tvlpzl1WOV=-wJI0u>yS{s+NBd}-PImBM*egCY!hKSNQ)-&7c?+aC zIg~@L@w_x~kY=(_U96$GE$Sc<`8P)RQFtnzf0VWN7aMq5N!|n#Ail7_r*ad`b;BSYPHr9j4hs$0t#JIS$}KT8mZ+&P8`7(u z@2p&$>godnc!v~h#eq|cW>t){61rYk%M}dQ<4V9OPqgFs*KL_P0A!VJWW9;n?H0&ZzfBo1sBP-3(kC)*O1sY7_+Wgp|yw%IGO}qiR*lQ1V zoe(bc=kI6nB(f6j3}%im6A3K~xCKI-h9|Rq;ZH^zasD*xnWSI?Efmc1)3Zr#)v-dB)ZtErzrO%K zl#=unPPPl>@Kl&UYc|Id#P;!YtqXQw57|_yyPj?AI?>4Lbp-kZ_C6&nWC#PF?q^^g&@e&Z2+CJBq7c3Skon()iQVIPP66TBn5|g z+C1Y&Gm_v$`PgcVWZQQ-3GH-@1kW3q=)_G0(cZcfIdZkpq~VpcGCOf))V6ZXo?i4oIAzq%9aGy zlD+-qq->cj?cib7!5LCn#p5m*+Z2tDWhKEK{I~mg16t#bVHi*={&n2)%r@=-b@-18 zsuoxJ2)Aw+?XD4gz6PhK;YxifDfecbZcGk^!R(K_V(O&sT~UGdz@jrKSrrWul!(`b~vVQ!8gRUt0eZPNfnKA;mGL3(JG4aHU1?cub9benY)6`pvUgVb`*8l?Cnk7(YB>+wDw$m z#q!#UpnmCAFO&9B^eTc<5#=}zvQ_k6RHM zL-l@d**UAF3ct2q?yVr4VSpc0*M+OD(2sJZxNqaNWR@X00!OHfBl{gC@hVQtuYsgy zu3*`bS>RIREQzN2@iLneHH}h^H1BbFa&Ko+$pX=a``CG*GBm zxX=?Q6GRK?Z4U*oye2pfuf%akC&g-=Q%V7H z*nF?clSNSFC`YO19E60)N8mKr$q*wioVXg$T?V6^(qw0hJ-z3B#PR#tUqOxF`6xoq z=TRNNaULDOYmDcJVDsalDcxyc?=V-=+O!k7p5=scE9ba6b6w7=b`W*Dl4i>vC}ur6 z96kxFx09{ekX!n|`{U5tL+^Gr&09JYcKmxoq}wB?Kxj%5Yg$_8nC%MxLZn3~4j zPK#69PDfySKRLf5xc!$OF_j87^7?>-G81POq(eQ@{!>##_Q>dBnf|5y)%~UQkK(c* zXUJ4x!EUDRj+XC_E$@+F<5lyb87oE;Oh|(|c4Yd}kKEk;aEQU@B%}q#0PFZc6JYn$ z%FvzzlAqabmmHnmZ#CNu=J|Zw=oTa2c#iQm4ABCmhO3KIb&Jv|_kObCWsOyyP4K=> z4K4zDUNJP<{L$&GE#;#bgj%k?OcHn2oR%TN;6v`Z9Zu!kX0K(mDcJHg*_7M*#DWad zd3f(U!4#4@(h}4EOuFLV=09!|_ z5PI~@$4G~gu;+At&VsFLOqpGZ&GcJ#n$`F>BdL-Jt(<3XDwHjA4m9I@Tix+3bvKG@VmJlj8#F{RyWB{% z1{|eKNuC>1V?b{PiF`fLA9#oHfGttr@dKTssP#&{vEk_g1V5nA{MtT)5ogSBtX^hP zFuSrXyDM(O;rgj#BNI_Xp%ji>Q;(tW^`a%ZHR!S6Rn5h)aZEN)c#3vlBuPX=!mtjX z^}(z3H*N>(=526y*c#RzLMUXPok$Qk=&+VGKZt<8QJ2Lqvp@dgbl z(T1JnQtkatYwtzYN0!IbZ03Xzu#QxZA-F-}%t(BmcLbR(c7 z0h#^{nXrYM$p@bgBPt4&Y+&;}jtH-5lu@B+wUQ7c=*B8%04Nq1o6R@AFJwT835>C7 zcfLst?Ie}K4Oq=42)lAvJiL#ey73PWXKg!aI%U~!5f;H08#x@nz{ zo8;SrsRP!CNc21U4xymZkWxTM-~jZz|9(Ho1~F1du;KFsg_oe1?@B7HyQ`{pK=pQZ z_HO;_upjey!`eFn-&&{Hhlplb$b3^!n?{H)Y>!NwX-edGHKj>ILWi_2T59J8GZvl!1e*zCQLlXaxWm;9rvAuDpw5bKx#(S zQfM$7Gm4cE%n>%4p!`b1+o+u+8r>69Uudsd(oljgs6eqH!=?ucun2ISrfuB%rYJO% zD^fnfu#Nyi=e|4|5*6(4mpjOSLP^D*!mqAfIe#$u&TnjDW?sG2!|ao<2*9=CV<;Ei z861ku^|rrhW8Z$>L9~@Wom#kRQf1hm5inbAzwSIbn^M+*oaTE!mgS09xFFYX9Q{@A z{)SI_#!j7Nk3@0H72*jO83puhVBq1bZt{SYKZ06re9N{Wh3x!PHFc0I=4y6_)SzN` zH{y4{phvaC3hbY+bSC$!59uG3`X)&H%SJnm2DAB812!Wd=86$Hs24T zHg#SPv8-LTqS?w{XC2|!9NynMJ1;)4DmC2KS)4UL(={$SZ+)lXh=`)UqYaWmxX+lV zdD#xLd3!!;VJX%)g?^J(lhV79-*nmTCTZ1qE9msTmF#fT+Ei3%;zo!(@S`0wgBIW& zeXUZ(i5JC*k0d1{>=$gMPe1A3{Ovt87l&qB1_l`Iaaby5=c?bLwnpmOQ1K zMlV%f#BZNJ@RbD_xYpx0UBx5x;R%;aWqYot4$UbSSlsQQVJWizdP($S3xaQmlhi(S z)9$bImoAjgaPv%_k-TqALIc#np+5}^g_kn%yTykGp!mY z@R80wEIV8{(P6EfVZptSa7hC3Ws9R2(PLdVjCf+O#26HRggLh6%A%*k z$@eaEAamroaT$Mjywm)vE2SH34qk^uX4Gv2KrH6el|2d?A3+iJc0zA|ts*X`lUuEg zrb6}j@pXRDmn!9=n|AuVvWL$4 z1E`td?~pSHt^q&YbL`uBDE@lW=iA`%!CgSUZf2{iRXOSlK7g&>k`08iRjqp4O z>JDRvA@2b4+`jvo?cnkQ#G05zscucP*|J*t3kM>IYpePNVyu6w-+?Xx7KqO0%cAS} zP(4C2oJ5NuU8;ESVuaU7#gllX$G9a)CdHs5`G{hUl9iI+Xk#bd7s`*=uDbSZsR#@^h|kUG6S!&CoaT=csf7>GM<4G zqk4(rRJwVUfi-vD1P!L|8)=bo6mE@_qu@@8K#R>HD!O6}vc6_sx*&~A_COel%ycKh zguQ4}bZ}n{l~8uitS=4LkJy?-QO!Kb`qyCE2`^H)Qk%mZAzf*tFsvC{R<$8Tz@vFj ztJ}p-LCsPz=sJp`1hHDs1hviPvOO-@3y7vVtnm?=ZOADRZWw0hsT@P9xn{E}FduMx zHa_Q?qaCq9X~GU$}lAg5biiJfU=<*aPTboK}5)WRu~Rhhvbz2#-0x+NeyNoi0g#q(SL@AC-Bpll|rO z*IU9zh9C>p&VH+Zx&1j9MPoT>+*O^Me`3GzS?N8U>wur{`hbzkMA}l#mo6f9s71&P>&A#02V$|nn-msh+DsS z@iRL!1shuN$QPY3)|rKo+ReFQA_Lgm*jatL7lubl_^JJI)*+4-?c^8)qcV5xdf&4X z$$;Gt{U-vN!n7g-ZCn7-;$$)y(*E7oIg11!GTx z(FXb<(~eNw{WI&aVDnB1TWqVrygqv%!8iM2D(M;9EA!&vhza6x1IO)Fl?4QHBURpA z1<+=9`DjuYeR8?xu$qVloHUSuOy-P9u#}G#2;$W76+$kJ{eV%){zsk|))B`EI4LT(5dF*Q->(@PS*j8YM0`iZ#5yt~t8v#AC?30)T{~vqr9GvIYy#cprYa zHnwdywr#6XV>GsnhK+68w%atelke_1FV5-dZ@!uD&u`x8Gt-@U=DD%=Ubxm;`&y;S zxe(h(+XU}26iA(zf}Biyz!;KqJiZxc;SkfLzYTE+f#{#w@naZBO-5*XtH_Q!P~jDR zoqsj?tQvi&y@kxVx}nTz3HQ#ekwJrc@9~MR-NcnpkQd6?gbNH#A;B&LpWw?>^I&W1Rm9}WjzVTzp!HT1BCW-wBnt>z1NP;6xhX>ye8Ko+IYLSoMevaEk|gF~BV zQIX%L_P&#v?Tg@Y^TvyobjfOy5{4>6k3|^_#T?1tBEvfL#MkW@FKLTji>Pi zU!rJGtL5m^hm2Wp?Vq}3AA0As@FUP8z!$BrVc_KrHizG07-ed&IJ%xL)f*=wfISrW zby;`1vS>coe!vW7SnYg+;~T^ZhQNhxi4_uVgboiHsexU(z^pnmT;ADPoUR~w4+R{3-AhD>=AK#TV&!S=zi{iX1c3+K1`txzD3lsw-jop_e8d7n+U~7to z1~hI`1t7PZfPqBKFXQR9K7JT5NvBVP(L971tXI;SDDu)3gKKW3phi3jnFbY7WK4%Z z2-Y{jy6#b(FIdI${`?}vxjp>uF**}}D+tHu91-U0-n~@uAmZ_LJl~)IdXE6_uCj)$ zIpvy`d0S9@{{lWg_%0qRl%UdM*oWl3JNENR1pPb0D3=F(B$H#0LU&fjqwjC57Gv^JsX~vdW^^t~_ z#9X;Xyd3;ujBBz4X@KI72hl_d0#!*f7VyA8W0wg3S=w?JI}}rqvF`ZJZy)Pe+so6*^j$M(G`~d$e{=OGkY(~?R`^6GxX68i_X!@>>j;iX z1)Z#HV-pOWp|%}UQvRGJSdVBSlWzBuPvBJ=#;Q24NT9a(q5ky{ptL9L)-_MJ*yd943C0;d^rMl@jiXK>~{Fd?JYW z^f-RjWo?hgEgc16sBtxTT{QE0U!@~ZV}93IndeTDxiV_{D8Mrpx8*Kfsa=4Hn$);M zO6n(7?!s?lZabVV`!wbUp@U&q#m|BlMoXpES+PVB{E^f*))`v4g;}+Mh$R#&ly?P@ z3ND8`(b6Kdt9U`gIH#r^mLGQDW0PN7hLwuJFqShajmO7}gl!C3QWfY<+7EE>u<(y= zagv2~K^Pw`H~Fh*jL2jmy|#X-zgx~aT(h0UN>~g`=OdAwp^9O`ApT#lAJB= z`}KysLv*-DbyPYA)1~k!hOftu8G*&JV7M_*heSu8s9Mn`~G;=5;%LpmQ zDjve;tI!ayFe#n$n8b<(SVr6o+6?{?4^4ViYGHgabkqFGQ9{lH8vUB2@48mX;%*b9 zQ8v9Xv3r`W_+J_{Mt88kqy(+2Rab6d^gE~DEYIH<^zV>?D7lzFgPOb)XGY*htjqS( z+z}S&7Drd*Zxo(9I@Ef4@FkNr;_$^KC4I>*|G9a zqE5gse_kvkZ2x%<#gXmx!3Afhl{oVoqaSpPG$#egZ?rBiG)_D!Ec9M-%F8>YJ`$uK zkLIayzeV7vny^~_D>gdN zZ@FPK6x#0kSgDxe%Sz^5LQk4z{s4$y__(;0T<_3=Rifjd{R%+Rbb-4s2Y*6ftJ zltI}tJg@CVoWbON@#DxB>bY7hM9Kq~-IdyPJxJo-oFKKLiZM>!^?MM0)u%t_Jr88HYF8Fu=|o{SL% z!ZM1Zo5m#;a)G+>ii&o0J(YM3z&-$ZnF!FFXuTC#hl(Y5YV(}OG>%pDJ>=0L4RgD- zWds3JRLvSLkA#3-M%)^ndAHi(F7#9m{ezz8#e&+MnEY+c7p;5T>cmP8Y-*Ktz`1kT zDOV2L<&l)i?9Rhml3AN7Y_BrIxnbdaUQp_BUck8<2fVM&cm6;1k`l0N-BL}7nwDLFWs8W z6<-~Mc@njplb>h|vU)fe990W+!3YAAfdd1v1beB_O(o!5glaeDZd5R5-NS;9njYf3 zT$`D(M;1<{qYzl|4@yAtkV+$3u|D3akz7tySd7*jW{AzmGp(peEJ48mE4d;rNg*t? z!eoav(e`RQB63{fPhV6D#8C~;erC};Pf$XfGA9x1?c!tX!Q9m)ypz)vBc-@W)7@FO zHYqH93Z#9k8cWE&hVsJx#Iq5^$y~MQ{eAmgm!PB+EC(Gooa6> z9_Y7f7*duyPD(AQu+TV)EYJ8M&sRDg^kGs+{@QzusVwQU>zO&QaJpSmOI8&UmC1w` zZ@~j3?>aX!=8OHC6|E@IU6$D28ly@vmU2l+r;pn9BHtfaTMm9gF*y6Y91@s~z=+#;*XT zh@=U}?)2nprlb?ui+YqgF*u+q4;z*^tfz=qDO2y2$NRi>n^>qzScAH5X`k=uXZmk0fRD-M`VI_f%Wv}y{giSo zwg?3GgC)5rMs7|g=C!<^u3R6z!h%x9 z-%q!FOLKs?(|SOD3Vs)1tK(O7T?-mrNhP(=NM-eKGqu?Se_rf7()!SuJaYPO_wAI> z)nXTJ1*k9f5$s#9MCrgey8?6?^y*`O^FFN~w&N=n>!7yQ4?@A|IiA(w`E0@^`UQsN z@p7pt&ts9pCn@ezceT4QPnuhLo;gKoo^IA?GrPEt%JQ&41`qd-3?3h73eiNca&{5u zGLdC_qj$sh<}$vFQ!l<6AH{S$v=s%v;=qLl`&zw*U7;X&5WISfT3xV>?t0C`UF6|q zGaV-;YIUW2;HbXD#epj=cFb^|ADlsFAY5l|eIJR4njjl;*wp(MJNk%0-u;-nQ9=z7 z-u2bd+lHl21h0Jw8)A3BD7N3ol;HL3UNuYl5E|T>c4)0uzP$-i3+*(KNcy5eg;2*^ zheWD^1-;yCL>5vGZ55X;uLjPsXMzApw*5%8Vykh@7ef&hoBZbO2V(QiOB|R)TJO7I zniVVSp;U)vdOG*l6w`s{e&V^wiiqj-Px;@ssD@m{&wMmCM7?ujd>85E^-F<6pam+&A&ao_L39qaTgS}g#^Yos! z=?FRY!XAs)pYslD+%+(u!X+g2R0C3hGXWzRAMeZJo28;7x=?%#oD2&om5EWip*Kb@ zAXbLGrw{ZQ(g&ao>1pz7*ckKNI&%%>rz4o@uhVyBRQQ4{y>upR2+>v>9eF=^E@ye0N{i>ZiDQJwSpzxrTB|;E;LSZt9-~WbVDhgv0P~*b`>nTYG%~NWy)JBR4EEOpCO^>y8gJw`XuvK>-|%OaDP-d z`HxIIDYrk%t5QfNrhE&G8B?n~_G^`Ghg(zSwFI@2Z!Z=whqJ^6zil&1tiG9yiTC8@ zyu#>0)#-vIQ@EjhI9qksMn*>$;%8Yy0dIx9xN*JEuAwAJEL#;}0ZX;-Y;A|_asdu% z8M75M7kwBys@&pdw#e~a?eA-+V{crZ$Wc`?2q-~3pyhvz(s5aJI$o&*YR=KPEj@-! zyG4A$z+DwgPIM`uPdfOt1<#a2&Df1L%8+ssR9d#sph&~q+s@;^(t%gm>%YibYmC9@ z;fODDc*}37HrnTxV0(dcM;RLTb}HeBaQd)sWIJcx&lFnfiaKH9$t!%hSK2<9Y3Ub4 z^Q}lFsUjoi^PvAG3hOw6X@xxCDdu=gSwM;DSOE$pPU?gIa@)Unxje#wcdW$1t>*yeJ}xE6?(kkErtwF10auxEn?HX@n|~GtFp+%3cRxCz8XL&h98DB6x6WL`a6@+TM{W|s z2)@f;?04|leG92$*NevDo!cx(A_4i)t;!bL1baTYMirXL$NhM~1xPrR3x610kYOU- zKU`uXQ6rY-me#&}4suIVQwWp~J2AX- zxHUI>@39AUin8aO9)-_VP3}mtd1hFI^cj@ee>zr{CwbQ=jVJ_+h?ed)J1_oD+Hr|B zgLIi5uJwIn%hQH_rvzDKV7f+tXe8gkj(BW*7qA;0{xLlr#M7Gg{k)*ikQQ`|~f^!l=XY)xg`mn`)I?kZ-+6i%omVj2GNRlPQ-J@M|qpUKbV0n^9ORAGyWjm zK`W8MnPi>vR;OYaH%@>TsUFSCWO{PyZ9r$!!b_iqY-kx0Tp@8I*tn=nxB&503|w8+dS!-azh!p95XTt;X_RAeVVvD~KPa8p=oEOC|LzEac}|MI-pPxJ>? z2Amw0t@B&qePPc#uI86>FS>Yy&&is%8#L{> zc&a!%?vkm)aa7C_C^-aD@d?ZUBSwDun3kj9tNe8I*qRUdy0eoc-B<;U`JN`v5{xJ% z2Kud1`@Htv>j~83S~LFIf#lD!Jc8zdhoE!Zm;@ZNu(Xm#0otpojIZRyDNuZ<;?Mav$$F@b1U@OfVr1Oa{V3=> zmVY{?^Y&PW*;^|ygz19G5kD`1Z6Fw9`|xe(DC*Z6b|dIcpQO|ZyZDmV?EaW7S+pqd zSO!zH{8>Lv^<=(l#?8fVW-MbNbxJTuoViFKDP}2R<&>v8^0F9_=$=|&w`-}qMszOK zh?-VYOn76FfH(aY%!WVeQ3;4CD_ahp9obq zX}2CrIEr@9>m2eka=Rf760eHeyXvbJ8I2(gM4GdU#9;fqWRs~u1HQ!XPHu~R%aC69 z=`Hu)&e8wHmcaK8R3eao8SJjIavi4+!gZvUg7u)rikSY*aW0Oi%*z1n3I5w3M|<5; z+>)6U2OAlZcjnHT(A z3}}WAlv8EiJw~{+_#p|ds`lY>YOpdH4kFNAx)R!mK-zR1dA6s^>CuUf_^9x=9NKe6P9x_+h-A6gJ_;5=i8k9U-RjY4Bw8dGHyVh6ZOK z)SdZ}Y8h<8XZ+#aRI$c0RLw98NMMsy^Kv1Z-}IJUX3R%KdgpZ8Z(gwfLVaz6A` z^FW|_@1xE=rVjTHr%tRIlKEz~QZF1$>7JhP)AtWosCxQ+qPBjoHXJ4H6VJK579ECRT0|mtp30xFU@bFfl%#5Z^b6+ z${kh2O@NXa#Zg^(7u|amlMx#O&Y{?W%B>^>EZ&x!bqJ-&@ubM$FN&HGY3h`zTp-FB zq5CK>ujhLak?3O$+xn>@B)=%SUiEKcyuvb_CNiemvae?$!einjhj=2trIt84S3NFk zNO*>AFw|>QH5=7Awr5U&0fFc_!ZO^9cw2Z2z==vKJee}C5h6A=r@O8CchT>BE(r5F zn>e4t87t%5R7X5UWJYW2I&|LEnlqqvYiVC5dx_i*OCAd}Kn&UVE#;?NsDx{WJBuUI zxoKKp+>)hq*wC(LiuK#q%$&~0OT|@awsGa@M}#=;HNwsYrCJ1l!ZkzFJ5u7Nm8;hd z`BV1UpND941#Htl(+E9P8{}{*{87S?{3=~32-iEAP>~XmSMHk9=t9uJWX*hUuNj}~ zNl--!zHDXpOZP(%xI(}{*sD;>H-2ug=GaUWy_X4V%of)Vrwbe1{B7-*BqSy4~_g-y)d|*(^mH9W&_|H7btI_NdDvY%* zPRU4TY$ccX4~c_GzsPek_ef1{>t`pL)fEuO4D<@5S==y|!&KI4g{Y41QM&z|R>;Rq zM)z2Cbfn{OV9x+Wqq;F)@;-u#2p7A@dGa0xhCa#Zfeu;rNcHWiKal zN+#h{&w{Flf?D0)3&h!0A@jt&=D5$N1UmPY3Z!!wpm&P4rq5tLnFA_eQZgKlk)6 zpA@1F&}zYm1Vu*a*~aK(HzTY=169VUWsIv!4CsZ5iX@8X8DxoMftD1C=W=F=L|o4C z3Vd7obje+q-ri=O(>Cs7j#l?DZlwAGn8k`u&c;Mk=n~M_bq{d9QYtIp`A(9r*UFN{ zbu}+^ZD%ihby_)GFZ*samPo*)Z&O_$yxYjJB?7qRvg=RJG{!!z@1A6i7*U`>DbhH+ zDM>R9&XD-!jQ^iJksm~|1TPs--?e_c`T;Jv&4e_tY~rWdG^M+u-1Fg+F1o{u?KQ*! z!(q@Qn#h?9-Ce~?eEZdOwi5Z-gdP3EDp6?WB;f`Af4ndvUA7+I*4ZI{T{q=S8lVj zB3nY`IM$T>OTC}I&mxq2pZ1`UIU6w;dre7zq1AUoogRVLM}*|qzu9PtQnmShMA9=G zN{CM5q7GT%BNw#J_&(zohNy2Sx`m3+gC?>`g$;g)7vqL^8nr2v0`>E&&#x`7?~NAa zA5|D)H-{H)>C-5Bbj$_lV?~-{L@7`L7X>b5DeGFkZ)8m0cr4xtnv~(HCDDfERiWs4 zF;MLZIKL+uea>)wqg${zW^3GlX(us#b~hJq!` zbF`hO9_y1Osiq@)3x~W&q>h0G=*PhkUB4R@p-?7mbspcW=P^UX!K%CP?8@hrNv-2ETU2LRSDLmb)9|q5<4Dnp3f@! zl^6j0m_-XzPbRm=hZ#$}^tks`%)`cVJWnNcR!mCK zz>;;Hf_-Lf+pTDG#ZoL4)5DUtd+B}9t~t3YjNq+~!hy;uqbN1D-=3Tb*qq>tqNMN| zu?2q`-qWXBLi3hOrDZjekocMHEV{&S4$_mwbQ0}7)-T|6xce@vW4A~-U;e_FIbBC> znOYXN!D{Vqp6E=h6N#*S3JGI)XPE2^BysYS&8ZpBQ>tq6( zsX`Or1I`k?>%W?X!kOf3v_vs9FJ2T3WhBoC8rH*8!?+i-jBxZ&?(o3VFrDB?Sj#->)j8}EVr9MXg%sdwNFLAf zUW|iz3+Z=!{OU=@3@kQzy#Y)blw&zrK;`w($caNc=TViyR}FV=EVNKd-CV(6Rp6f* z5u1q4LXnXKDDA@0Jyh2pm9nHo*~=w(hUq?YbTu@-EzR}y;h7#LrnE^((SOh3a5DX3 zGDvp4%oZ2PQh2>Wa(L4F2o95~%863qDKSTXV+GfD8>#>vn`%YMnDAERb>DF}MFJ<- z!dL}dxS5Id`xVDNG!spSappt*+<0m6dyapYo1Z53i-4|jk@0g?FvSptTEAikUx3x8 zeubi^+-O-Y_7$^=$ggZ}dkljT)mN~g>4hg=m%Ys%_cpAD8Vd&Gwf)BwJhh{Gb&RGa zd@&h%8aT5ahJ?{F&SniTk%SANl;dH3WN#+jz)7LH)(cvSiEgVh1V3jhHD&| zwV+a8$?8!cN0w5>zfHNm&yFo5^NibBY-;S`2*N5A3X zaa)VyuTRLtx8*71Dkk-vuCIMo(QFwG(&2QrKN}!JZ&uhB;LxV*4Qfz>4Cv-45UuJ| zuY=q}kN-h;t=}U^X%XsHV>Z#+nj>YS?*vsk}X|XT{@aYg*yzT=5>z&ve-(1^-G=exJEPIBdJ)^ zbNx1X{yMMeW7DF4{pIGNrecyu&l)uQ2KOE`{qc+Os4Nb z9&=VE$ReM;UtD*Fw^1EOyg)4G%kzz{j9ov=c$8$1UA4vwBgFTsR{(3R~EM#t(w^FVeR!i*-cSUILa zlY}luZ|4JVTppwYBcO(m^w6auB1B=x-{!=7$s6yyQY+ywoigi(o~Jyik`$O*+(FSUp_Mp=@~bTLA{V+mKBk*IU2gB9zaC9cI8W#M)Z80a zVDCcp`T*LbU|f#o7ydnmRK5pA>d}Ic{+vRuk&(1yYK;oYE^Y8uQL|KxQjIA+>n#Ha zIQ-@t`AdfVy(@!!!KOwXIWleN0t1WkJU{J~00ODK?p z>O=G%oQa-cik4Za1y()u?Pqe$Qf}xvPhL*6(&wumqAa}j*DySj)4eXa#$rLt^_gjS z8k`!VhDLPlC-+7dBu}|g5N1~?`x5@U35w#tX(Xu-dH!e ztR_GN)x$?xuQJRrMjz>&$#Vp==2Gki53_VEz%7!pHarF?AZ4vg56I1RoAA!0aQ;nK8C7F#^_BMY+k1t%Q8kz6CRXOV~o}A0LQ48TPEz+-XE-P)Dd5uT5_8^`2#c6Ff2o?1d$50EMmbyVveqCI*85 zKk;9u2hm67o$keF8C#R^*^#fu_}E>C6m-RQO^*D>GJ-ENWetJq}!PXTQ6H%U6~qr}1c zihsAsy3n8;{pQAVUx@GBm*r>5%gZThGIK^)L6&D3&e~^IWk$DG?)nUyepra%$d1-9 zh7U{1|NRQUk>s?FIxQQ{mbhQN>pvI?fl8PWaS-{@kwIgn{DRm|)p3M!fN#~|SXVyv zE7hA1o^%$;Y}oP}U-g0Cy1fJ-)d3w5$^(xiFKX3rr#hijnUEt?KJF*2iLeSqs*xd* zv_#I?tUhID|FX@pghA4);cTas8CmdZ@0UHq%OOYH zKoo7q=A(m}E)Kq1m~*t({ObSlk)hb8;OR_mf|yKAN0}F>Q$Ph(0c_zHpsu845!bd` z^@xp(f*l_ncxkV=epMV3M6U}lcY~yFhPs!rr=raJBtB@jp;IhqssOi!23fFBq$EQK z&Is5iA{d(%mf!zbW%-xS3~AIv9o@F1|Hy(FN^$ZzgHmeIdrg3<+s7Z=r!e=L@oPyq zBI2wFxwy#R*GN|#GR}hCSy4=I`PFhwYzm_tgA2eY-2o7;FM569EWR940X6>gYWY>J z8PU6X6gn>$yYed-bRKF?P>S(-M~~L9yL1@U$2|e#yYt9PfQ>> zaFb&1HL0z|DP#!r?3Eg8A~iQR50-u)-)Q2y69yQ(vr@_npm^_51%|ZVx`XUu``GjN zH)Q0ctkb*1{8Ipmkmv$LR8$gq+X;$*DwX<_Y}JvtN58?@ot>TATQ=yF=jUfUd26Hh zAdDg%^(mYDwsnkq{k!5ax70l@6lJlO-QNoF1JTTG<3=JSgSH{9etjhXi`WMu)A#JsaWkASqTL`o2c_)qTTQmMKw&0p1Q z30)x*IgLO>kmq?1+lcbg*nJ$X$`x9Ff&NRPuav-{;ig0z^%`#-qLOb|&6y1@hzdlk zxj|<628C`rI$n|TG3q$(l7j%7M{V@9XNkYfHO->8BFbEtAzr$+bgMPV4at-Tb&K9E%B_?k8{KMu9^%nAK?Zg zg!YS9YMTBkko1t&FCGe_D0_mA62;gVx2$ko0vaIFkLtW0lsng{kv8U>mLNp^2NnCD zlVcqC>YAbMb;Z6B^6^Mq=XgHUEbL@VXz~=hd7;NXpY%2v^`EJ+zX-se!r%z`dWJrE zN>5{in27k3dd@jqWf!{G%%!z0YkZ;2=T#Z>s#iGd;oaTA?8-!Y%D{Km0IAd#oAv zw6TKejCtP_a{!EWBgAWr1xrHW|3i6M5p7s#r?sz47pxqRE`ON~%Z!R&L!jr~G&LFe znle2k~qMciCGx0AJ{{Nopn)pxCEe)EQVvYTS zBP}LCQx51K7TO*!BlG<1|0mu|lLyyZUl&_p8I;muHUKjhqL>brN-Zul{FN~Ak7wIM z^ASO%2_G22tWj2C^y1I=Ju~~g&Yxfh<&ePJHb|BM=Bxbi_Y*__;sllVLqhD|@BQn_ zXGw$RrI?|plKjI{{_}cQP-4sqnbE|*{_fA~{qNVmmxlk%+V7<9f4k*>yXCK1>i<;4TL07${dO*l(w zcBIMa;4Q#^S44R5;|I_PQ@zP?zoVmLM&6WI8C95d&D9|L13AW;Sz)?zsjA#t9%{lx zOXtxP=AS2*X&ZNtA7=q}Er-Xmetd@!`XF3;F%5P_%1>RtRou=5K8T~jl#4p9n*o`( zJ!IUWO(j&0G5N}PjBGG@(bL;XYpKC?3T<5m8dq)cZVswInv8rorZic2Ohnz z$3V49%$~NtO_-X}kYF=Uejzp>QB-`dKocX?_&p)jy|KVHdN^P6f*0GLAs!YB2a<49 zQ6iRU>WldPV|!t|(V~0%!{H70x1awW*JLxNmn^i`xl3y^u1Zb9@i?!6*N9{df}w3W zosy(2(`>YFfkOLTLPm%n6M3K?}voOofyeu zfUmnj{_Y(D9!HIIDhv6;t>4Zs-HiT~q3M9kOEB&0=%tZr_j<|z9=_H6#u(*BMaMG$ zYAs5tFC6D)n6}+tC>XN_pgu0wY6dnHo&MvgY4Dc?e`e47%TrV6UxEaGZnJ?CPbUzY$NV zQWZ(j5~p5k?sPB*_pUz;Dj!0D_DU}>5hzR4s?MiU87_;xz1HaUIwlS*dTUAAd=dUv}DY!#4i8ZIYmtN3yG z<9Xt-RbhglHO7crxBy3|9FhCh1bQABp|blp*TqQ5*S02+aO@0!W{Ww?+2Z)g&u*uk z5GLapMrp>~E)GhA?WiyL#Pq8?FxTZkf??p#r$z-4i#D zM!ogZ{lQlVIP4xm=?nt*5FVI^m@TyQ!?ji%;dtPN3myWvo)6bYi`6D#{;K?G1^%;Z z9UqEE*`=w~Yan5-Tz@iJ*ROpn>7UjX@i*HZ{%yavC>==R;kl6vZucAl-4gghB>pm6^aLXX+k;#{of%RpP&LAuggQXG3N^wC z*VwE-?-yo9D52*hA8H6i#5ewf9QzOd={)M^LfA2fta5xycX^9VUMWx5dAEA! zY`UNljEYVy<3FqY*TG#1pG2@REWe`^Ky6#emqI>Pu}elrK7k@P46xW9AP5M4tLTQe zSg6S6g_M)g3}s(9!{ha8;l=lOzWVCT4CrAx@|9|bZ)0R>w!XaZb(UM)V(!4SlV4?V z2nk#WUe^E=nO4hncNdAj^`QB*k;D?PHhPx)j5Nm#u(@0-c-bHP=pYnoOy#_z)=dDx z_nL0`sc8Mc;n)cF(;UXm4hC8PuI`^)d_xJ;2jkQez*Mb4U4B(AQ%|JP0`aT5%(G6^ z{%E#QB(Hc2ut~tPf`^&z1A=r1Xp{bM91QzYg?x$lG2WL)#xb^c+85kvb2>=B^=G$8 zq6E|#3w{hRM`hop`VheeaojyU@gx~xvD@dZzxcw?kjv$OK36g+Ab!790K2!Pn+6mJ zE~ewE)4Uy`lI{o8alqi{7xv+!OiDF0VgV5#@L7N^K`vJ$V8Cb~xm9j~?0Bh`^4jxp z&-GDw_?M}lZAV993d`wo&Y;^dhUVrd?BKwVrOe4=(Q_*rb>l!9Entif{ zb>nVsBu{8Y56T5VLHY4=QdE}fOenIQ=C|h5-iL3<$~0%x3D^@KvVGgnf@LbK4{lsM z=^P&RQf-=e`c}&{SXJ-})T)hr9s&UyLG7&`i9o6a7$}&?!bTw`l<%2bvQnppqBI8Y zS#&#IS%Q;&jTut=?YpA;D58$WQK=zMhP3yoUXPxC$_B1$wbC#di!O{aW^B(r44iKm zq9|xCt;5fqBxuTKR zK}30{X(ByxqDk>*2dTPzE_)-%0NHUdFUmo8<)TlZanvxb$7v^vIt|;0HK>}~;mOsX z)+zqI(h7&B+itw#1378{-i0Bs3!5SX0A_p;*#{sWQFU*LHwdN_%r zLi(_7+4^9nD7IaKDcP_7>G}6}5_|8nl0YhC35yc}V$gK0|A_RcPS>NCDVG(lad@@F zn7UFrfv?v3vGxFEQmb6Ek(?gMmq1>8;~NGPVfT6B0G2GF08vau<6tIqCcDEP=>6~? zg;T!@I3QN?j)DrM-~j&{*WBGAE^T<%XI{NnQt9`~rQdHv#e8Ph&bIsAeG$_MwmKy~ z_wx=*&e%RSu|F%Wd%rvZoJ1Jy-U&Q_H{=K?+v>17{a)xWk|12v-o;JU727w?=3Hvg z{&k{ezGT_I-_gjVQR70419|o%eJ#o9+WHxljkqQs6TP&OI*^MnKhpa_Ai$%FMN{7q6S$twi}L(c z&;MtVBZAO&sTW2!rGI5s%qXYj(DxdgvQ^y!)1U zGMOt1FwnB`KFq4<2rq!*$9 z<_w2g>F*CfW0#9@-e&qT&V)yj(-yMS_gL_tN`Twrkib5j!0!is1(ZtzBAhhOpK3f; zs^$~Qt7e=J`JcJ_>wsO7K-g;ZaG~;DMY<0e)6r`44Pcew!+7<&Kv}MOmdyR9^bI z%St(jKLScXsYreoATomvL);2EhHVG(T|CdX+u=B@@F7J6zxC6Ik;Ho8qz~rG1PyxE zbwP=IOa#0EU+;Nf@n(~WoXPs}F3X^cebLK1*L3^l<~HhaV}X|nWxB6nMm1_oS7EQL ze(lFp{^!C{@o(?=u0P6P! za8%D9{tw$*1S&7XNKD43HBDF=9qQab<*X^Yr7nq!j|hVt~$8;mTll4f(PK^qjahaovd)1^Cw| zP9ld!ZqK%#p6%{__L#HlMF@kXxipR7eZKvPzh=J3WqRH3f1k{)lt8f$Lk+agV?%*d z_?=4_#RXDi1dbV>?0(!xR#BVeQlwg$LuPukRI5-ZbLDCu(z#Jm-ef+W!Tp@YhrcD8 z$-_O-F`7hg@MwaW-O{}%;sc@b4q|_;l~X^u!?)^k2N2pqX`^HoVu7=*-pHk!ZMMQl z;IU4;U)*(f0$?xzZvG-4{`0>z`T6(e76bP|MII$cYy?>N4AN^IHv=&4!?%fyApNHR z3kuj7aPZ!9F5qwq$msX6Bziw?iRakt1udm)0We~U4A{lbJzhl4$afY8Ay}?=Tc2q~ zR!04!(+6Y8-i*c~!0{t)pVcH{4sEtrm0qL)?>jfm@d2re*jAspK6&7`cmOe-uQ=Fv z!j?bh4CwLzuAFYgR;FHi4jK*0AJ`u&Nely=)=63CU4h_EB$ZCNy^*gJ-3JVP(_z&x z*C4yZ{t)F?Tqe_~{!B?!TY$^$<>|=ATM&K=0q%^SiW$JvQIOV%g)Rm+^H=o80Mb$V zs=a!0a*yaU1Yp~ltDf+N{_R}J-%4N}vKW*GV=ys?^C6}rjo<^4LmatW*Bkxk$Gc!M zt_;mN!0VKVfA95Ti+@cH07Ys#_^p`LW|s#)Hb|&mtae@=OABBn3LoE7`$UJ6$!7!b zK0xRclo03qHfb1e;=u3Sx!zc412{*PN$aB2?1~L1MqH% zLZTEB7Nlkx9ZY1cF<-%REvA@$%k=ir*RcOE()m5Ckbs8OX6kJM;7{KQ6kHA*9YP)q>q`sh;h%I^t1d#gE(wDFXi2T zWaqB~A;PFqIO#9pie>7|72pddch3(e7*wi=m_N>hlmQzSMI_vEC8UY$8iRBu%LWj( zkRs6np51zt(J)_gS6a?$i<`p>I0@C5rfnx!CG3qCa8%XwjUEYvXBvRG?_{6B!3s5b zZ#VqXB2X)1Sg-+|Pxb(KxD`BookjAN?Oh9S{bFvuBP0TqL2o~jPZY?eb2$BT*TkXG%rKeFd+imp(qkF#*muC{?zY}H@(ylc}>&gO7^I?Gz z#ntY0E2?In&;5p4PCjXdrCHf9v12vS6zvZ-k@R#FZC%+drP z*!1AT59mWmttLm^4??p3cqW;i&bm8jl*&}vEo^jLrP-EKXspCvEu&E^Qyp7O#mvd zBQ^%s7ch4Y5HA5wG_6~y64T+(Xz`* zgWWsPh%nRdi3({yxvc%ESV|a-hF%#0b1+8+3SY=9N2Kh7q;AKOAdS4m#~tLC#zxD< z=|V^B4Hok69NXW;`P#DM;@?~VQ5n(ENF*6s#C)$5+HlvHEapl9ln=hm_8FbJDw&|O zNG=zgj0n&B@eJr)D*D#YuWFyq)&DZIda!apa36>v36BRqN&;}GQeXI5T#kk}H#K&& zD((-e$bZ0vR8ZoSK~g@F->w#-PY+aHR9SKlPIla1-NgTnqj;gjm|}TPQq8G<7xe!? z04c_#KwzDV{aeHHzhD1v*8Z|>{0+l#bw)}7POE6z3;uob7<&1eD zbw)QY7{!0PqJK*#`E*GV@Rwn|IRw~4$Xj-1&Idr1%QeL{xv8UlC~|&QA`y%6tuKRi z=JPSEHlsE`75hTw?Mu%H5`YARe{wre$ex`_5f5n2;`pm>`_C9v9t3$5bqeRJ@lST= ze|+*lUWXXVZ_<2$|F3rWO9@0k+$SN%fXlv;#ovBi#K)JDtk?ik?Dv~X2lEk<5KKxD zww{^S{r2nkh*74bXve-k{&K_r-sXcCpyL$8Y+6!&E3Hw`fC~NJYyYyM|36;s&>dqE z0CyIO$C9E^AZwad>h}%-QBwf4z6Csq{r!D4WlJDgZ~|afz@<0UkI4f-W+j7*LoRkK znd~#*M7G`kpxl-v1u_WS7bA>6fn$#F=cef0F-nk4+0TW_E6qR<>VoSL0Q;b4bdHA$ zNSR?_VH64lK&t36b-2gD?P0$~VTf=2VU2B+dY$Q@n~pH|{0 z1EABT499*R;P&selcM!tH{)scT+_F0@rZ<_sWO>gbwc1kxny4)>gA%Dh@fBZoB_Tz>RL=3m>SzP%r9|%$=dTgEXXbOM=Jr67Ufxd|t+X}VjO5m~$>p)@^ z2*r)K0&z6Pcwxp%5+Elyy4LDmME!0F0HkXCoBjCSmg{&K@+k$2(T#=oK$TFGAE>}b zah|+AjgODqsb2gACPk4smak3PxLJS)ta9>6pecpFKK^&dH`I#!+ zbIaADT57%B`rLRuP^MPB2gLE80btD~cIIuc{?%|WCX5xE$#g?Rwb}UySo{GH!c~_5 zXf$A6_X7J4w5tpU=gKs&z+-XVrRA+vxgHeU5=9Mza_OM)Kmr~*%vZk$D?^_;AV{%! zW)M&n%IkWni-MiMqIx%=14@KEpfoAfW9fziWdhB{2y!mpPwG@E}K__-s$-_ zx@3!X7xri%GbdnD%g?UmU|R!Y8Jxlmm*{u30BHdLTiP*f2_oJ%+vicdM zpi~Fr94?wmv3grVJ*>v26zfgI<%2KW@ zng=$4elm1TRse8U{DU;~cEVIm+Ebb;Ad1Xbkr0JE=*tc+yv=r_s(Q+!M@5+;Rm<3G z*tZRo(V6K;>EcWM`4)CGEgCQ6dO*OvovQ_W5G4%#2QtlCb3rB*7ooKftSDN61=WyWNvs7@bnr#Kw84W)hY-!&YUtAw35gPk(4=Jy!etj5C>ATSK2E?UyeQvhR3b|uh+{v_Hl z7<_sG_E%K90B)6%0|cqTINUg~LL4?|^ApN?|7tIz!bMJ*K*1r~552RSCyqTpTn0yT z#>DHBs?aryht67Cex;zoqOoF=iuyng%(p@pPH^AA^t0=&LLlsTsR#jQ8asDtoDr}5 zHb9@nMzh&WGf9FxJ_%K5`CCI{O9h+HJ_khcKoezt0c`hGB`thxH(YO<#f7! zp5E-V7$lgj!C_%iK-i%~SC60t<4K>MYg2#OF4pG;5RXv=TuApp)7kxh59X?Y;Rj~E zjzbwdtkcs}HKlt$(rBAc=XN(gcwsc?0ZCl;GP0(<6tg5RJCmLGK(b};1k@#L$Le$@ zEEh3YjZx2qC-9zewuykg$2z8@I zfgukI&u4*E))hm6FVQ4~uE01{^1I4bk(E{pi04}@LI@gV?-M)wex+b1Z4zam-eh$_ zzvRm>%r(E1{%9pSnk1JaEsdVEeE{aS_CX4eLK_tf1j8=r*7#hbGc= z`}j8%1Yv}gJyN)zO&2?` z_>RzcC>xPk?PwVu243Jvm($E=_bgzvB(LAotwp*6DYwr-?A?iV#ZxyM9<{yUJlV$X zLOm<${2*_)4z6^3^wM=pti9jt4*w5n-Do z+$j;t5(RCEBJm%k<}gM!o0Wsl1tc$w8;(S_tD*}delfP<%L8qIk^>JkwQmK6uF=Lh z*O^;Sm3#rVrDkj?$#wm+-|XQv!pN&6z84?8<&^u2XRk2fPlZBF#!?Y|7ey zbPm}J1x01VE_y1EDMY{qvt8Gj(+?QAb@a`!rsjgf1%df3ZL@u$GEhiXKl)z)g1v^C z$#zym(e|07r<){uq`_`+P`yC2+M}k9Q47@-p|OvZUcICG6VlSrcw`q)v+w{CfjrR^ z7+X`me_(+MFyr2G=1`y7MPxAk#Jgtx00f)eaHUMM!IU{|7U9W4<$BVht-;i0v7>RQ z&ldU}*-<5|jqW;7E@G@SAc7OmmrODu_N~_QRyr)^ z)}09a-Uah{vm-|Wwez7PlptE3me1#t%B~etO@Cyp9=Z zNV0|Df09;NV$}2*f`5&%F1MkGqsyX4~l{Vz{IP0fv7H-J)KH?)eZf915q-{_5;yz%^ z+PordYxDWXmBk6%ZKdL?v7N(2tR1%Qjl!4ZT-49-WJ*-W3jFFzhy_pZI?2pO)-rfE z^)vC*NjVY7Li!C2vxMF6rlYv_Z7X7?#CavHD}@o;>{w4I=PV6t94Zwtg8X6u`OHa_ z=c-AE^TQ8;Nl{dBgKa*{3@&EIHb)n3`vU$Uxl$4LOr!?yIJ>x&wOaQ{8#S@RF0qNR zbz5f(c&ws`TMep?0mK{rN~neXbS8S{?M&F>L_B^jTF=6o*8tEb8#V_xzK6<5OuNSq zl0~_-DGcF81)}E97M(SM^FHj})|`2rNqyOXGQ+S0wk(cqPIrBoOU3W=sPN4WRrOFcn`3#je*Xv;;k!mqxMk<}J`*HK$vg8F^NG$l?!V{9tdlH9`y% zrE_yT*a<_y=ujD|{dCj&qM&ku-`cGsi16z_>9V_{GTnTTAC`4SAJ$v%W26`?7`p;sf5>fkit(CqABG+RZvq8w zy#Z4FhXSrX7Tif)Sk4IDy$AP2<5b)~6h;qldRDSPF)h@ZPndm6MF3wH%sr`aMWMa* zUf7aukkf8HiDha)MG;k2J&w3KHIYoStro!j0~hY+QaWklM~f{iCIMtxHb}%*h)~`3 zQqkvNe09{hNZsE9Xn1ZOnxZ*Y6eq2l(5weFcEb~3boGS4u$lND*3Q^&?b$&5pFF)j z`HV4$Q%^4Ib>ou-RsYQICoY8L@+!6TrHE(O79&)m)9f~X7}lq<4`hxfQn?eI8@Q8n z9XoU;-E-SB1uxI4b%mmsyl$~yYQn>-5)!b^7fexuSpFi4yQ0vqu@XY%=2eNb(C+EZ zmDHunVx8A5_-Qu21<>8=^xsi6_dFCtu6md1UB+Q#$l zmsVG-guo^8)>&A%;CN@_BcJ@Oe2Nvb*R?Zrj(l^sEh+tDW|i$zr&+%+vSs_I@C{MZ zF3mS~!ii{~wE$7r@J#>BZaGkF0IWx4G6Uv}7Y=Q6gz6pQKy#^DHhoJYbB12~Pv~t%?wwzo5eQz9Nxyjo>dg?9_3*4Q4@n z=f(_}aJk0Hq+fGo0~N8nQIei#_<`VwTf!T*1(kp72JOo2$@Y{p2nG)B^{L-+CCUdn zbDDF4+>kp_g`?cM-kcj%7X-1uZhWpGivKJi`Z9bxuFe8K`6<%VrOvd27hy2F~bq{{xAt;%aRJRv0w%DBAB@m{EzC99ih z2B(#A8N$MH_vZ$2EeTQ!+e#-WO_+s$g;0snpnC7d%~XVr*Vaqq9RDEPqCkDn>`Mwuocy zUL2ShKodR@uQfLlQf0|yOr}fya=rxNW@q|w(UkAVxUuIuM1(c2CzXO6af*K)Mj%vO znJrpQD_IQmq87l?;h%7v@vW-{2&4W)NEpsyGO!J9L)-9F!xaJ~RC)TYIr?JWvjFN!cdu2w6&ta#!QBANMYx!Dbz{~q6Fe(wcD z9u;5lzcJKxi?6>LiN5!%Yc25pzSH>>lRP-9LUhCgbNc!lK60E^e*m8eGh?u0qE`K( z-RWAW@e<$bMr64m=kWd%$h_8D&Qw8#19xdo2_nYy4lF*hSQN2e)5!AEYq30@6tLnG zVd!v&BqS!PkCqvCgXA<)grJ1cjlDO7!(L3fYqv8E>54XDt4s9}pfwj%-qO%T^Se88*BOACJE zBq;(x;LatYFnrO@grX(~29)TM)lO9 zKfXUIy5B1`I;!CWsJF$FL))f7_o(i0z3ty3q9s9dg(XCiIgXeo7YS?c@E}mdaeSMz zjQ}Y|AQbb4q9@IV zRkYWEGFlR4V+IIOdU%5Pyp>B)C>02Ucrsr(!5QC}dMT^K2$Sq2A7r7R04mhSc%~n( zTpMI3^~e=u8v&68F8B-Yem@Qg<%DDnY7?3=jn%s`0;Wv-(FQ7e@-9t95VIS4HlYyhMVCK(;ik0)07~OnW`50%7j4! zI3(`ZH6ynTbKDdQU%rTx`G*Hr3YDkb-+{zLHTFFj#|<_t&iDX8Q+t?8Cybnrv4{;z zR%#rX0og5OuvKi@vI)n$#WnC~snajad#{NPAEQhSbHWp3UxDxZ84>Jt_|7?oCl z&3NOHii0NP@WW>wp2qXcfaXKGjASEd&&?2J{pUn^_s{Ig>ab&BdaWn~K;bCN- zxsc-#QDba!{}qUO3 zWW6__2lpvc^a^)%Z;0v@e2O`Yz z1U)$g%U+}Qdj2xEyKYgd89-rJ=Fbh>C* zLeMLI>d&!e{h)I9iNO2%C=)nGfzllsL>n&i?lG&qF(jhjnglfo52(5!oQjmE8_i7T z2P7p;0N4u!!0Og~T}B#k8txZgwtaKBNxH@gB4e(16)?f*ece>3D?%Fu@BDz`crR<1 z`Yb+N2iZP1(7vEgjf{;>r7a=dBZMo^>K%l(Y`dLIlckH;^q%<)mub(~v7v{56aM`)n=Bp2TDEuA=wFt`US2YrvAtQ7u55 zhlByE+W`5ZsTsW3+Wzz@t>mM>fMCvG-SoHcAxDfi>XK@i*oN6Ik)k_vdYr*zK#nomn12RAM==SyUbfO6Y8#$w!JU2L+&Bi|UFFsX0khy0aBZ%rF+wP!E=MOi ziry`KjFHWs6sMk=>iH2XdD>wj8iM?B|I4?LDg+?74-Rk6FrUpwucvyaE(x0t!s4#g z&)J1dV2p$TMy)HdV;GtfJnhD^bigM?T~4&@?TiOLCCMBRC6B2q^^ItM`~e>qX6+|L zU=ObzyA?pqw2awlD#s)plTmryX#`3beIl9pM;gy9dFRRa^kBj@OJ9F}mz}5=^XrHwo#sDa)q8H@5 z&Icdw`%c!_+s!}p=2bVbKV~-e7ZPtQJ|9MH!_cEy*y=vnIheeDRX8NJK+zwKi&m_C zZ_czE0Hns&DGk~HKr}s1t|Chg}@3kWiAhrE^1oyWsXh*Tt&Icz!ZmkbzgDq6N z9zufKS?Cs3r=ua5ui=RsKh}>(kTLMPBcx5vDQQPt0bIdW49&QIZz=Ae*@Kp>YbuTs zChC2KwEro2nGn~sU{>-%Qv_P3PwOyf&vCG?3iZ_|n1&Qyn!U1;5;e4|NM> zWCRHRelmVL+sm|xDsUDEd%fSZ|En67_SO8xXJ%>*0+!|jj+BO3#*mm$jQz1b@;yVp zb$PAuz2iE-m=$a|B6Js$i80-02FyR}WR|p;jy{tIio;aLHxBmZ+`k zw@OV6wHIOFSNE8k+Em_IX`^h@X6+|`_8JNx?_LII4b~zNuv7y*j*(2Sk zF|cO{Jw~ow!PgRB1BKtt+XAAgeYP;_Ub>9Smi6^jPtM`5;5xmEO<@UIk8wXCb-fuH z%s?n=$3szf>F2fzb%oTlz%un{oh!63ADD>Kb>*P{Bz40|l=}iGOILHy@_xG2Q+S{K zG3%ggpSE$Mnj4=2^ra)E!PP~TuS3T|K)44UF?;ekzVfaZ<34MdG?a6d?<_0Ue@6SV zzKyYMtKXfCikuL(&^-&U%4V4+Q6nUUR~9gq`Mr&7%|5yMw*Mb|Frb-kNi{{^<1zgJ z?+d_q?3y)PMqW|kulX^i;o&8;T#DFqWri^nu3!girptf-EkJQs<&3PWE#v{?=G4uxpHLW8%P!CY!B{dn8sWBO7R?T zN-jkw3kSF$>Xbu?e)TDI$oq4ochQC$2@2uvgjMuR-~^JR>MQ0?6;y+sYEWkQlOh^ zyYAh=ir{pXv2YVxA0oHiZLfjPtB8(XjEQi~)_r`a4-aMA8zU~3-OCV_(Bua|Fv`J0 zhS=9prDAJAhJ1{tQWYIDtnw=){zkG6+(2nADAyBPDDuKXHcQ0RES-{%IwUl&9>jG%1tx#8}2X z=lx{_8f+*=1^gyN(mvhR4W%Nj!>%^vCLpQ^W)A)@)zK1_TsTvJ1+1M}>C`M$X2nRW zOR{~{=1@@NXVuF1Jqd6`eA{Ji=by39c{Jf)?QI&ydM^YlgBZm*FlOtwVk6L`s-s8T z)(>v0#&?IK=Oi_p;y%o$2Jw~_QFe8%K{{AY3`#UV#3hW@@_&E=A*TqUpN&gnA(%B;Or(J!tmo zt9j&}o|f~8sJT(sWLEApK#Y>}cqvECH#}@CO>Y*|+QVV_Nj&*QpxMlPuJ}ENig$M5}IkZvz=L8U1<9$`$IG&4H06ZsJqnrfBzeX7LFMLt1NQ33ZYtYO4%ZkiQ# z9-1egzD2Zx$IuKa>ENyg?Uu@1rZ|rG=85zWZkLT!%F(y#!ZSyyE4`=wvUVzgPTkg& z0%=;ZCp?LWBD9>^{#0*ztzoZ6MBNv3psBfAHmdXCPD~UU8<nsjlcMv-a~QgV}t45T}LyAuk7PVCPEP8**E>QMw!Z zp6b~igTuw%xRrWTl*l2wH{r^CaKGq$qrWOL@N*^Q!fbJ2?4xUP-bprnzEeGNkgZpM z#C7{+PVE~3oRp|jvKd8nOW#R=cytdD(iS%KjJbWRs**K02Tt-Td9LDEX6rO8!y+H! zrDZy^23syw1m0&0Z@&{ux>gRAc=Ub&HEk-Gz<@9j5)k!}UA$_f{ zynIKuGW#$!1f!~ZK0Q;S&N!-@)Kq7l5E+^p@{y)3Kjr(-`J2}yZ$Yz>`nsVSk}(Mk zm9)`Lgl)T*9~J-67HY#p_%=J&p`LDsE~B)0sAA9GKp#I}ZiW+`p>r%5-#}6FGDdU7cxbM33$YrcrfMAdlVw(G*{uGBJkU^%zDmbh&EE8vrw(lxN#%}2|#N=4AWA# zw(|2&afx-pCpe^HOPU`?feintyN+XfoubHVhXrDJSX5MT?LjWwr>y^%5-p$R^`20{ zw(rNXPpJVG-qwFL)VTiTi3${#AX8}$F|q&gRmYO#HP=6~+s=4iTMcg>-xtpBpuXnL zTnLKtrSPT?`yXAKC7UxHe1br6>$kj9_7WYxVv$R* z5AJ_X19HK9)YrZasyq7Kx>!ri9uc=2CJbp=O0(Tk!Ee3go)FG30+|qZm4-0mDZ)ix zQyMONfnT#2)~gTyr=Jo^2xETyDF=JGr8rLhUs(cyP&iW?8V&}j|7wCk56q+iGRvDtnL51OCERu$l&14 z^r}qdV2YC@|N9y~g^9knms&U`()-VKbN+Sn|F1jfy*`Xo)#Qa*9J5*BlYE8%TvSXl zBxNzT>=ST3kwCx`aells(>mC8{eg>B7Vd%uC%?J4E?o3I%ywBUCTPwnpDWR}bKt+m ziipA)V*;mvhTzTrdeeV>D_Zq(^Zozxujw#BlCS@-dj~zEeP{|)$dp}WwNubPNP}fF zS70{C!`k<;6(Fb0tXC~n^OV4tW;&VPZ%({@@ z<=FUBHu+QSg-SXZ99CvJg|+r6kfyR6$lX70@ zHNRx7n+gi$8EtEM>=zpx*E+KV+*x$%kd=vmG~pg6K+BU%_Yp=0P?I?R0o)@%P>!$d+lHIDX%{yqdKA!(*cvn^^$~aC9z(i7q1Oug^?X8vWl%aHUOUI~ix%6jYXM~D zfFSFED=3Z55q+Ahz-l624p23H?`BnVIvdQ;Dzc9xV1$vQj^lS|l?2JPjt>YXivATm zY9f8PJSr84z}x#DJN#W~t~~F%@-O#i;sSg0=SiN+-pe#`?cz9e+V+xWu-EzPJZo8V zoSMgA4pUmix+;twYNUCwd~UjEyRcEp`5QBwycq za(1^JBg^S0265sgiP3?meV?bPbg_tjaL8B6MEEOBUe!{RNe^H6mO-^5m+&cuJ1UtSKQqCB3r0X62%*`B$K>|dzjzoNDhFahTn`f`c=9Xtk>o8=T}zYtCi zR|g|TAPXeKK_xZ4#rEF5M!C-wiEeeYT}Zb}^g5<7Q~aAU{g3~sSYD$qYx@0X zYmRGhfVs!KV{BA`tMp!_mB~T@Z9VqeaUjd&9&lK%N-Grrx0u}mjs7mc8v~L`^Q3>6 zWKoIvsV-SIzYVEgDGlLXsL5eKW8psSl%-nIhzN_^mW~vkH|&5MOMbmI2#p&@cKN58J^s3T%4a73JMd7i-sN+&5iLs~x3Rl}t z6w3J^Y0<8y0w6zv>E6@4THZ55kZYy8#q9fpRE{B*Y*zhKKoE*f*?;cvvH_yyPOXB@ zV3seE{`4m98jxLrW(Pkmg~2wr1WFpBZ1?v@e`QAs&H?%3~Q5< z|4|=^O1|uha6c^n{mG`Hg+NA!&+Z+vpz~gLxnUQ@M85Y@xm}6I^zsCZ)@()3()R;| zARm|8@Ix^5bdhk+0Z*0Vmp_a>2l>x1(9>e%1x7z$5+{D^3m(qk0aP@_fS#An9yb)Z zKV^tMD2R^bHQB!=a>Nk>?`|3fnfP}>ASy=LE=;4z5^M&J5Tf0_3C(+=y`a_QsK230 z`hIA}O?+VDyv3oF0#0LR@8co5R6bU&Gqtx-$bE0eT74`UZ3e`t0ODio#fH<`8T|~( z0gfk~_6}jaUL4Rm6a%Zvrol2C%=dc!7!Ah)N^*ADo~eicRM z!K(Ai=W78UK^hJWY06K!Ab-7TPYb^mv9Ed2S;e>e29InfQ;k1)a1@JG z{n`ZTo}@|@Q2%|h(E~dbbDq5)mt3m)F7Z;;8l%MQ@BknqMX+7CT)-Yx)NL{USy+0k;m~S7t9gBC>E+cm0LeMfW6`VK+W$fLB;T z4hdK6a}=d{y218ZZ+@RX5=Ns<;R*ONLAb?T;obBL@&(GE9xQVAX%FguQPL zV9j{Y`V`QLZ&e=+{@mKx@r8kuK&a~^ZD=&&qp<33USy$uM>)ew7D=p$hdxZP6QYdLwlh9XZPX5kT~{FO%F5?1Z_ z3SulNCcl|{K|nFEp?Ff@=c)kDHIOBtFH4P`L&WXy^vL`%$&rk3lxjlNNcQ!by(ku-|Y5`qnQ9l@qBWA`s)U2xy> zp{_rSj8%u38Yz?ms&8)|`#I$W{X19ut*r4P`ENQ~&iShmE2T>nhcnh&Xx7@zJ~5_6 zil0J6IUp(B>D;#qwW_$k${j6F8I=V}WFM-ox{;Kau)QJLK{*M>kNY(f)Zb`QitYtRCWWX--RDeBy?tgP=H8m zMrqq^AZQQBbc^;p-v-LU??6RF1Xm4s#Fd3_LpFh7;2OoBbcDVka72u&fVF>nHZmJJ zhILG5*`Oj3KQDQx5+&!#pGJlOzk?jrP%yLV#B+Q2%paG@8hl8`L`TczsP?&aRx^+C zp_;J|)J2z4&ft!=RsjPwRg;+W4s#&BLUrBAS;yj(;*;(lmX%`dDNLyi-6)x*?aAY5 z6V0?W(h{6LQuVdMvS!=FmGslTE7AviYQM^xLbUB)9iJ=JPW`{wS(?xs%%1K;vV<$Y z>V}25e9-TNT_>d~q%{dnne0Sqd+_QyDUT}NE-_Q!=I1zCQZ!A8;-_g|Lk;%XPO_xA zwx2CQ5zxQqP-Tym`w$au&X)2tnrdzw@tAanO`YD~N(g48-nyAga4ciLS6)#(v&Kdh zW+Zp=?v(0Fj*GGQ+xdm6*xjS}6E#w;UZ?}id|_$gScaRxku3j4gX~*rzq796UjIi* zd=6oOA}v%?JgrE<^EZJ&`qv9!sfKO!4l{-B%Zq{L4X&P{3CBXqdL$!Y$pIxQ11%vC z7-l801<=;NQsjZU#wKMrxS03rmF=PnP|>`F(Gfh2G}mKJ{*CWzQ?Mh&IAL=&w9Ciw zc&(AY^u=?2>vUNo?`M!BRcbv)<7K`j!wC>zm)Ax)9;HAtx}@dTUxiv!S>Y1mU#_V8 z)Gjq_ttQAjNW~2!{uU>};K3DtA}&=}cSMA6)BS?*w)T=r{Q(9kgWq2u=9dmq{4_=O zWwutC^-|6!aBu>z{?h5bgFhi3u>yYqOhoZeTnblXBO~`A(gbtlRE?uQT?d2Uhg_88 zmox4=4m+u2G+Q+g|0%4cc`P$d^Sk9lJ^i-q*JL?Y7`~?(N!EERP@CZA;}Lx`RMdux z&Gdfs6Yq-r8?)1uHbg!Sah=oiEB7lq`obx`1f$4F)k_LY(N0Wib7|?`qK|pDig4_V zn&$G}c4yzE5_!36w+1jiyy{|^zGDreMzIC43HqIhf6ToD}*IV{xKC5G|KkEj^}kwkPOK-0jle5Ec`Y`PW^ zT=P{)dZlc+%5Z9Xx9rL9ko8ge1L@LrQiH3-JkQAcVM*4WZYuQ_IV44&w?nK;j-Mhq z!-(xlS4N*t(J=JGBvUn;>v)6w=_h`{7({2rW4!lJES_%vqU(m@VN~!_InA0&%~z6) z2a!#Sz4^_Yjf;G+e51B&2g|#)<1m*QJ>E^MyPKe;Pqi%kuenKy0f!NS2C6Xc1i=Ad z&*8QhAv&Y&2l8jG&IVbl5e$GBo?U7Na~AZp7TYT3A3)b;rm#JS^4*fbka0s}$g9`6 zp^yAhejgaVeXJ9&pMo^&)r4IQA&+_U4hLNaNbHQ`ZNxDx2@If_kyjH7gi|r32%_8j zpDchz^IhD`-`x|X>R%C}_O7X(&a=BMytdOQDeAl7oisq*2!%`v0$Xm^3|}@@d7p59s+N7Qiz-929mT;<*5Puy8VHa=edDh)udYC zoH7zOzMHR^=Qj1Tw>=(rmhy09c0K-7%fWWh#hY+ZAYy@58A8-5*7kI_zda0o*RnfE znq<=;_@MIqpe92G|-86 z_pYSbtsgoI9(m~^m3Hl)<|<_a-t4BjdekX^z(<1K;q}jVgw^yW zqs7kP1e931L%!kt8DHIbts<06OLiAJH^XhX(%I}nqYU>0{oj5IqiqN4SZDLI3&fQzqn8FC9U5zJbYg zIn1qu9q`g%OLEH_nIw@0w;vK!i}*FRk9F#V?zT&f_i>m~s4V*V^56tl+RA~>fr$t8 zjJbRHr&`qe4dUFzbt%*u?*>I@{UU4;yS&M*uMF<8mGfc5fBh+K#V+t0iVr>eG^-)K z7+@x}(S{@tFgPH;p=Jlol_74u6mVee= z!{{2fJ_RwUJcn7gv6GPRON(YEF!6}m&`B;?Q6vG zX=KKkpBWa0d*YUenavtX_2eJpoF46Xa2E6#_&vM0RhiYKm^l|*06Rm6DZ^k|Aja5m zB`r@fgLaMgAZ58owY}kxtyk60{rR2WRbWG9c*<^xa^dB>%?QfnuIetvh>-mQ^9Z7% zkU&`~N&9VEDG?AM;V^IuzXQ$LA8(4xZN-ap?KkUZ4T>??Ya{=(w=^i*90=TF|5z}{(+~T>oBJ2Z%tmPlJ;3Q{0cj6mB`ucy*5z5YAxNMyWXm1Q&yQ3oQwja zKF`OEDb1@+!QP8s^f&0X{rdlcBff=w&^f@75hhMo!}qY?plH)AVi(SAGTs5{x*tWH zW!0(=#G(o@Lz;0Otxl=`xBxn?-Ll9&B$4Cp43-J?fXfNY<7SWKRehR3$3ZUUCsj#M;-WI8`83 zEA>refTkb%@B~(~9YP~AzU2xeJ=s3>j{opfA8}$Mbv#-X(I1kGHuUNgS!iK~_nRx+vEJ>_0(zN~Ezdsg}3KuFFx;5M!cqw{@!yas%e*TieltCU^~nULm|9suD^Z;qIg~$ZIsecCvbq5 zbPI?iW(6Fyr`bZ@+;t&N*+9fJn#jY+Yxo1rREZ=Pz`zzoDgwy?4Asv-d~WS6`kyxc z=TVVoBGRFi{BZf8GJO*MMJ0*Tk|Dz~Hy}Lc!0jc%#(fjOUg}XifrY-f%7MML4)7a5 z$Ovguf>mn{KPsuQMq2hRafbBDmu6Dl9WOSX-)zK^@;RFVVU`p$r!pJw4^Q{Nu@oY} zbI6=vNh9M1A}8@(kD#SdOmp1L4SUI8YeBeLV!w>oXJvi2fLT0?NSleSDK0fa53^FP zAQ{%f{J4vt8_nXN3g}S{gy(q*oF{2kh3m{485c;BMLX%_^slqN50AyascujtUI;p9 zTm>XOu$n2U>l3;W*C)31ylvjz;(ZD(I@6C1J{$dI){Vj2hZ$s`ax>eOV~4lzvVksl zR~zS&iZhj8Pyk&aLGT*QEFq~`L|4N`U{R!>GL@VF4e2~Zz#z*c!BFz96CfH2$M>Gd-e z;F2N`zPR$awy8B)$+cHJJR-b~)|F62=ezd2Qi$aTBRW4PJXFz&$UMeA2#LhB9p0_b zn^Vuhah2&`P5BkJI6lg4lUCC!C&0KB7xuVFYyex3l>|$?=9z}qCP%l7vGc-U^&w1n zM%U-^Vc(mqH-3%CPHCgrA~z;8aRPr0cTk?Xfg_^PF$4gT0iqb_zOA1tgQoEL*XOSk z??3A;Bk))xhHQ`z+XQ)ZZezt`f1i4?;9|SY=@9S`-@fuYhaGI|9pbkCAZjz?*rrDV zsXKPSPN0d^3$^uZ5kO7-iOmr1+PNlJz#OzCtT9xfVrnfySB3yaD1Q7Xj3?bTH&N5mfXM<+4vJpFIPu(>!pU1XM zM(mZIQ#02_cD`}cD<&>{rBxfPQjwAkc=a8r8wWT@ZT60%sh_DoAB@j<7Lpt(71@Q0 z7G6T9pk^mxd%ZW_Pj6F<*V(cKcQRcbobkou>wrY~e%p)HGazRgB=;Mro9q^z#%QGs zLWgl_!XF?_eO}stSnL1w7F2Eo_xC zc~5}^(~9dD+7clAJ2Nfj!1iKy9PX5d|8GI^^}Bt!bq!XkFeR%Gj8(MhYgRxoR3C=) z!E#p4T|lwLBv5^QxD?lkpIcS}qli`ElQ2`g|56Pk46JzWWZ@lcaY=4^0r`<%TinUD z!DOf{eSruA9f+{>zX?Bq4r#*z+HSzbvh!{|+g8ctvdnnQ2Y`9QHMCU7Ag&*zNrY z8sZEs+*wSK5gxy9p!Ft|d7^<|Z@ZY|5Ey0eA2wLeqmkgc`%-F=Em)u!Qk5{YX{wd2 z(Yo-!9zggPsv@YY{xfet#M#QFcBy7F(;;S_Whp*!PQ`SD@6Bxq}VP$$R5 zblrt+B2&bsZ`S9=@XRXcc7^{>^&m0uh~;?_l|JLuzJIuFc+5K=Pukr^f2tH;YELxk zR>+7Uly`dn*Y6ykB~6*A>ZX8eAYIS^Qn#4MHy^Z(Ayl;}R?Fa7Z{SI>&+!^YD;^IB zMwf%WPrtnaWSKF4ks=v-U4SVI=R2dZ$<=fGlDk0SATa)d11lQw7MNYptMdr39kxCU z>Lno;JE4U34M32rAaPMdyUz`vj53`JX`~{U^!~$gfU$JoOc4ZbNXYnTVR(!kDM*9p zKj#)$5>t#B=C-hipwr?}3C@B5;h6k51DIWF*24`P7(6j6(XvOCeIydfNKhcXLK_Hc z2hs>UZPCNn!(MV<*_kf@btOjkGUs=4`f!##M`}s?%ZI57;a}5Hj98Tz-+4U(;$NM4 zAmg!-$v>R%{yK}%Z9P$wo?#OBj^Axc$C%62^({&ug5PF56bOZ8*;6u?QdhKHWLh4!VRa zNX7aNERwlIZe?hDlzc2!Lu{6f84Cl)#vgF={kL2h+b1@|03o zpzWD*Q}sNr??aq|cbzYXbnX0H=ptQM3J>VYUKIezDzjo+r8ycx(n!_yKevBC&U|7oG2wq*j8;n;4N zI1iT_J`NZOu?m4uY1>3{=I{7$>_q31LjCal7z^p)AYhaTKt85Jp)ihkX87xrtVKYg zJz~{7XJMvu{8e}7pFwCdmdXdtvlvpMa0IB^G)0U6ZVS)SEblL;^TIxE@}f0(W9- zjbDqmKv*@W3*>ljZb~0?xIg`clYuYOMh#;m!>yvtNXgW$8i zP@dpu{J@;TTfToM%Nnt8+C*BeGw|)m$40gG>!i=&ujSHc^UcT7AN~H7iI;U3E-_0# zoP->2xKgv9exNcBC=i_1`&-@YKb`nPbFp#0OuktX#6X>Gm+F@>I8LQi87Xh~X2^Kv zZenF_Hg@_$M^E-sEQ+{8aZ;uHCNiF$m$G06(eT30oP@2!AhSAcg=@JJDg4qZk}(81)fXnTed- zzkDv@WnAuaG@NnPl2|-ASf=|aq5EcqVv(2#`BI>mRajs?fVY)L&TP9<2N3)a-PPbO zFYTJta@{S-aPn!dm&rF3%0!Iev+V7wox67iytDt{ESD%nMIjAR2NFUQm;sbqa|M`4 z6y7LG)0r$aj(K=_vj#YvKy{C?cm;%A4e)h|@ha3{f*4fIuy2tcP0pxVA&o`j{pnB) zQ2J{e!5|rPd$zRuLMTaVjk7IN?X_jTAsoySdRg;5m!U}${EW5{7ix38eV2`5HMugG z2p{1sfxi>kDFl?Zu^btqZ}iwxdFe!?OiU1`xQNpxRlhIN^VHi1%xr+7xx7hj=s zn`b17GM-ky5w};qiOwRt85ot((0Wg5tPVSi?7=i*j=HL=uq%^D<(}0&bcK8f%kA2D z4jjiW46Z*<*hpS5n4GrLV{buq6sm z_9oe4FU{`>?<~sk(B$`KY&tSuPBRkx*DFVTS+p41?C>+y>X&f_`4~l1)svIbJueHO z&)#USi&F)+J4MwCMypljEA%65Y1)$I^&bZ*H+BNjlkfLUj-w$mDbu8R zIz#fZeR0in4+EC8AGNg6E8UGTn}cPt!0MixdV1P*%N zP14uIC^c)oF(TOb7H})n8_<8aZVtDU%_;5q!37_Dn)#3b6+MAd=?gg~+YP`H`z z($gcv=%6u*F@i#Mey?6P)+mEIzPIx%w!mJ)j9Uz-g)cwp(wEx2NpE*+u=Nu$1`$`cf@n=OioiO@uQNdWt8{N1T(yI$~k<<+&+9R7;i><4$bMBECTKi>dFONK@n z-i3RdiBl=&*AODo49N$YPo(6+)&o*P&7?ybo^SVvroHgUNV~eEU2XZ@j60d1A*zyc z1vp_aNs5d#`u!YAiSQVc`Bx4mQz}=gf)NkNqF(bsj^o0RN$KR!g_g0|De9`!Y`-}F zLBEfH*$&&BE#G6pR-@Ja`b!`uOEYJ!4sjMWJ(xnQ_TV!RdnSJ7;hE-1#fvdn*}9y( zs_~cxr{k)Ia=WjI#>UDQ-tI-FRZQAu<;{<4f6q1_8Z^Q7#Wvf@NhAKVD~FJah2G7v zfNG9T$0aInG8meRPiZpC!YV^J$U zn4C}nXl9}5)HdTQsIWargWiOL@Mn*rHm#W>?@y~!!O{@^p*9eUia(qja$|n?hMO5_ zHfS|Y`eA71CoUOIePWl7TZNh8%~@_M(|%Oo7rUyv!s%EtE2ONU_~z|%s+cIaJOyKP zQo)W5GO-r}BOiMKb_ivJ6uV;RAJ0ij&J)3pE(rSpt^OWW=JOXPD&57>I(hoE4=1U;mV0Rn3ZX#AS zJAMl$c_acw;A`yftAVpe%`+;dhrOcahHZ0KA)+=x;>c@8js z9x_8rW7fEo@=!CYFx{@gBvMwmYBgH3%GF6sTOx1T{jQ06&CaQ9;gDM%vnyXAuLQK-rR1(YA>}pBZ-R<@K?LVFWZy>bixzSIuFHOo*kE-p5%- z{5&)b5_4ni#I8V8A)fU8Ok0J!wjc3BY~jx4(YV|=LpsqS5uS%Y>5}DLFj0NC1j8Vw zqZS(D^0*2BgO{m;B%G)9)9Q1>$%aph%l1roB(uz0^#GH@?=J4Mrr@|o*G1jI!@aZ| zQ8={@pQ5r2yW(!!yF5{7KIdCPX|i%#F$ z7ONY;jw$En`S={iVFbH9638@iQ5QneCn7UUAYhCDLukU1Ho+K6lL#%Q(f;(l%#D0e z5?L2VabqS#4@R6XjYt#;2NNC+Nd*^02@}~)Q&DwX@kD!8Vr>ZNY73-iD=PeCQ^*KF z4-Ftv6#|n&6Gl07=-e3p)ax+OPO;yx6V`S=?96Nq0J)F7$Tyq6?bGU!+R-@roF$L6 ziGXZ4d%Kx@0lhHj&7_3}b;kUu_&|z3Qltw_jAK^0l5RR@AIXtytxDsod#v>-vF8cA z6pdoK%r}bdjMWqS$YX?LWp8T2{6p1FNnbk%rvPDk8_p=vJOn195TcN)Z!+5bY(a@% zz0>)c_u;X=E%~nyFY#E^k@aA4LuVhcc4avN46-x!caEcaY~*e8vlU&=(tVq9Je}-^ z9$)dVJgdBF`O;dLy7|^OKLRbrmJ|$Ed$u3I)mgcGS9(Wnm77G=rUrp^yf`Ac)TbvR zQE6R)!QkD?-Nn+p&s){NwTYT5isNEnl z&k$WWIQjJR_t#I;HG8HH!~1j_d182%$sbCSd{0Yh(VR#qr&DljWy)_h?HjE-lJX_niNz2oPySj=x?v5#)Pa4%5mA>0!BWiHgpmXl@PQ8^pgWSm?sPO_vb!N646 zkba5XGyMa)@)9vr-phnAn)HWosJ>D9Hvb{uw+VVk0wlVOkr7D_&XV;MULs+bRGubA zqlY@9pNtuR>R^K z#Cd7cX(iK+aUu~LE}yZGWwSNxFXR!75r>S+b7OaDr;?14NIjmlqhtHF1KEye_-d$D zVKOG&ip+R0O+x^eCRiuGD-opw4NP)*Y?AXi-+fP(5`9hpU-$5~lpms}H1B1mEl*-G zauJK!094QG6~uS9OPjkf+2JSA@X`F95|xP39wm6I0aG+u43yHEcXz_+s5<551?&_s zD*CX_MmTQ#x@w~K`;jmR#yZ{Y<_2Lkp1;h3^>>-J8CoN3)8QjL+2DI|_OA5|fBCL-{4!oO zd-vKcG)*$#eWmX-Q?5$AAI!POL!Z1ID4Ax6QH_CUvqjOVL>y(Y**vli-mD0e zy8ZiKyEDcVklpOR9f4=GT76iz7a~Q2TrNm7Zrt3rfhW~_)3_@c|IlBP-QJwu<-TNU zAZakk08$5WTrUl!2wpp&IE#?FkmqDjt;yR__qIua#jx4i=Bp8q-enBtQ3jl^)fA&T#<*T=&=x4nH<0}7}gVPo#^}9UUNQKu8wB75Lgq@*w zsm?MHjzu_!DB67ql9bu|&gP}o0!CaIO~-yGLO~eEC=jd50z}WYmBshT|-??wH+iclCMzV0=>btfcrMoR|})9QC*9q{|V9_!cR%O*uDhoRzu z-RsX^tSqpq+rP&UF5JbGba2|2$~6b{#1R|`c%T|hUPeA`JtYUfX_~)&j2}kOC6zBV ztKSOFR2R{feYg9vWqPE}e?Htn`{1^LU-?48KlK(W&ptLZq9w3kkUJMGl~azFxAs~6 zD;~KX3)ISpV9v>Ru1X{$na%en7Wa7frT#8I3H~nUnqV1$)=!=d;5we-<5d!+dL1kj zaHVC-rTz`{V4ff4^PmTwWZq3af_zoq;*uY!D#b+$%NtG9uQu7TWs9O*FTHM?s1f1j zBQm~XDlTXZ&lsaLiFh=T=j*iS)$2DjrkdiQ$ZhoegynPmEi%4IrrSKo7R-<@JGmOS zYzWJ=fp>q@sX+tDF=PT`?rWMP*`%NKo0`#sHiEXA{bW^Jhe3VWKp0vK*&Vbw)4iSZ zPFt~}8s~|Zy<^EP3wcj?P27}hc*>-2!1(7(b&8J(6@%DmE^=k_$@NHcur#M9^Lp1O zAyTHamia-wN7KkkbC9E}KPm~@x8|F*lOPe*%JAunOJ1CSIY{7x1#iOSmFG=5YP`^~ z5mzobdWUdtel4U0ZCcm#^5a$G2=#IT?_=BW{N%(J#7*M!*-p2seSZ@+*y6ElyjctB zElIBVxUb_@gVtMNLY-v>T`+AAkK`l}v+A=Mm;0*+=K%vmJxDMX&;V7Yn(ydzunxp^n4;CBjc2QT3v1Tc6a;3lN%IP>H+NJ zh&BLhI{&NYg;JE2h=_=0sYro!qOMlD=VQ8gR{lbsy5jv51*q%YWj0JY)oE(3pV$N` z9(;I9e1|!oaj!&noVc{2p%r^Tt_2=;>RdT|4!vpvAl?1QCfUB^kLfwZZb&bzSEXwRlzoRy=0nv znHj-p^0)#KKQRSnv|o|_ukdz9NKO+?542Rt{}By$rLnevHfdW^EPO+LhgL-Ozj*^Qc!!@dKM_a*9@Dyw`gQ z{X=Px z71<>nT8-YllhCfbkajR5Nh+#zD^=o<-*J|RB{x9peDXBL^=@m~ki6^!K102CsUorp zgQAwLe}rIh=S^e4Ad166SgTt&Y?qJK&bRWgvrzA>SvSRK3)HIV;Q{B+(^{F+9B%70 zX=P#v5&x`T6tX;Jnoi?%AFDmQ+~XTZysZ7}bpbAl|IxYDPX#*3n-3d%jGPD5p?7Z> z_Jh9TabumlkPIKH5QsV?3~Ev>&Zo}$<we!>A68J_P1P?*(`v2w)96=_j@ zh5*bvImK2qaz zdFY-V7D@ZzdcGt(BPXgZwg_?GEFalGT4;jAs^=$anGbW(wQ0TUeSIPmo=AONf#92_ z4stIq*OQlT&K6s3@aNPJGE#GzdUqHu5D#RznI8i~!*heVKD9G_;x0*@^sL6j{er|NYGH;9G5`g63-I40HleX5T~ zowwyhXy_EUuxw|S?|$|?!qiPW$L4KiLlxBTW#6gNjjxa2t*>!V=uo_mp2Ul6e@RhW zSbVE|J2ej|Hv`%Q|D9V#II=FPi%^9C~ELRV>00)vLMtLcQkbxv)sW8c?f&$SHQ}qqOUr+js}^y_`DqfaCgG?7&8lmQDXCiQIV8;;5ej$etkg2l+3nQ=jS}Ko)`# zU_RGiIyNfWI$|neIIKCmu|M>@(bMPWbo28ZKEESc;vTHsdh}@g)>}M1^lYr7=r!UI ztKG(Y`(}_I#a+qqxv=^Btl=-*+es$r9kZp!4gFsZieUHhU;AWrQ;M?1hPhF6cv9yr zvDfMkP?3iGG=4(&o=uMRCMUDYAp~$Rji^QGu3b0BWES@Gt3ZOQB_sR{racd!qTfY9 z?Q(_D19#h-$`|fK?5+zGX{Nru0dinUlecd|q*2vAw2+@bp6F{RmE2#K9S!e~m*mtl zI+_3q|65CN(&4Yx#${GP`Qb0GK%z{^2yn1q{RY{HgMf1Lp%EzS+nO>I`Y~FhInq-X z+QPR8b|D9FW6vUhy#n-OGdeBGxbpUL52?>FW36NfUvA+L6BOnTmH0Eilboi?=#U%%;}cqSYa!3T$uD?0TG(g z&+Ywf$}i3EOs8hPjq&25{llg}AKrj@;^wyKJBrJdO%y>rwu_jp5Xbemm=8I5Ii1a_ zL8$%n=He5{!2S$7_u_0^*(z*)qR*SI`EZx)NR;v_O&@A~rB?im(Q6PneQTz+mb7~>G3F-$v_{P%^GTjkzt3HpmV z5I^6iyKgp#H*~5*vRdz{7^RdyvYv4LAE(iO+ri9QUg|SIs&MXnfo>gnsRZ0!KZG8q zhNbxqj)O7Dp_<3XWj?1qqpp5f@lW!U#QPr^Hl|PIWpZ64Gupr&U{6D>vzzvZc$?dce`ttV$Eb z5hn#7@QdL-&Zo7)sqt$#L$KT%Q^YVddVM9vln@Nocl0f#(it?l@zvCXhW0#MT_~uc&@k ze&ryg%k0YhlS|c2$pnds#aE<7V?-{OW#Jx4%okf^rjmy}4_5<{Ta?b+K5D#j6kk^) zM-?nkic{|LwP?&ZeU?q;*jLd0eyCVwnQs^0?J-F`8A0IO)Br)=o(o;8zhOo)gUOir zhiB1kETRDhQbzDhSjaCtR!D?3Jfd1bTF?K`*js~)u4?=oT2l{<6pGoxQ?@0NtA8_pq9gvdE= zP3?AZ{2~RKLY2${3n&Uq_7nhCZuyM)dN~ z)VgZvjHU7uxg*xZ3EMB5T_*d9$XV~F5yXO7t%zc$GqMF{*uDM6{2AP8v;ie!R%^#N zw$Vp(MAn8avBfgcA1}?|_Z+Q*AMMZiU#-pACkmb#(wL{6cRI6Z@rJ5$Y3~w+vqLjk>k1=7MZ)4|fvL!Zsh| z>~KZk*(vVL2K+=xz>VUJ33uAbX+GTPp9*(Yy&y{VKm zdYIq%ulG5!jPP9AYc@E_T%UspHj+m;J$aI8HD%~URlH--lQ5v=aer zv*kdA$%Fk0HCeHD>OL_1j%=h5160X~nGHcq1&O>P`7*pDZpDxrdCb}?D|tLpoOGT1 zh#xO??)+=4=TVvX>Ac#QU5M_8hLTl_WxsQ(SD=^o53ja0r`bMDv9IRb+i2_U)la$> z0$#8sUwkMN?i}5SiYcffW513JZFlEoe^(JOu7clA1JMi?Dw&kQHE+83R5Ylz z(!7(`OW;uin+{khd2V=m?=@cW}4)=C)c<*h2oX3Hj5@~NOQJEai(PEK#9;i6Q z7$2=UDY048{(*>%G2h0wg44HBdwA(HVkfB62_CPfI_N|UzIy8H=7P(tC$nC?iUVuy zc^9nwGp-YLH3#R#ZP+>k@rZk#Z&6?vPPnS{=err;PZva2*Y@Im{7yk?EQ^D5L}bly zXfMY(j-OFy1+$w)oujzjwpb{{l#-p027&mvN8L0A-^7@Hj`a@`l#y2qf3g{@O-2zN zSc@c}=82l;=SH* z!@`@s)M)uvXRFYek3#CWAz{&^*pl?*hHZr_elzE2_vAcSB{vb{>7f*StmO4VHhkkf zi3#Y5A(wJ@G(5QwCOx&*i*!9&#@9l4yF?5K_Gmpe+gPpu$oJL}mfraEPg?)VJ|YHB zv%RS-)L?!(USvZDYJC5TF%#dlQ5sW(_x&x`7?#a=9$>jq^(r zI?FiEIjU%NpetL+5#-=EPZzk?(`ei}Zq!FtC}ll_w(~c}j?XbeDlU{CU^pRyF{_wX642}fA$MbIg;#A0`No<^HkYVZw z?H>MUj~&dnNo$5am&qTKq1N>g^@pjST0KDxjc*N!uI-BKJVW&~#^;s@>Ue{pc3&#l z7eiCUW53zmwO^kRRG6ky!gz=fr#gt&%fOE0;E%QIDlXMM>?KQ9`k8ZgEr<2dej?+8 zHX)mO8*S2 z9LEM}L%?tj4M~NCEN!!wjs;t*7+$w4fGirHRedzqW z>~n^--@k(3kY%ih3wrlTq#!12Aq+lNe+d%bR!xp%49VPQ-K^^8ZFEdrh6URcGLX4v zY#eWswe&VWjON$n)gHYL z9R5vOt<={-q-*IX2UNk=x|_<5g^@sVmj`idO^)dL8&vJq(@X~L_p8>&S_Q# zq(9;%*?F2$CA?%V(!HyDa1@a->9-8=hEoTPKee) zq11puuw*sa2}(GGo}I5Ee7ed%eEqS1iGHARA3lsAJd;TWFKqNhVDQ`E-WbVV{vo3$ zUlf)xMh-;Sj?+aER*}_GN&HOC#pJ zN)TgR>f<_j>NfSP+v+OXi(notHBsjW1=4U<_xP%_u>gUYfWl%F;L+Alh^nLqLt>+5 z+Toq}9@xoJjzrr@1a{Hk^WtOn1MlaqK}6nYe~BkO z)OM(IBYc!GZ#a z6g*Id8WChep^eflZ$#QtjEP}jz=FD(V*=Lf;mn zpl@dqseN`)&)fEVy&WhPY6U2NNWIWyb^(x*CxCbzM$`u=W=OjY&Br5CH!=*++M|RK zfd!j(p2v*%AYy_A43+55$o}>57zGj%j%@_hVlos>tU!@~`*FnEe@XkA$)yiOXw!5G zgxDx7EszfT12aSic~I>|*%|&U@awu}pA2$#s^BD0iodafWqt+{!I*6vJim)=+vBy$mYl0zsnh&2ZH=A$D)Q=CZM>ho$nwy zBLU~hL?MWpS(T0JFjh`j-Oefvi+kF{e92Y&#I~KMU5R%yEtKM;Dsn%c$(@9|i`7EN zcNqzT)8(51T>E%FDjKW$59l2LJ>N>4WR0n{$+y4WsXz`f(0h&`Ek5ku1#W;6K*`9P ziN!u#e%ztresR-{EzT5_3594G`+-@Wt>l36q%9hUSR~?bc_&-+B)X;A71|rh+#@JE zlp=VKlJVNa+k6un6IM@57J?ar1QacUotZiDMiL%_1}5qeK}vzu?PSS#Fg8Dnu1`-4 zly2j%XWwB2`L^${*0UQz=T9fJR(z>rpINj3^-Gl!NyRD zqd(Qljx{}s65TC@#vo>chq>Y9@f`hoKk5${lAJsNXLG~FviSg8A;39!JohuPwyjrB zA{}5N5*#5C&N%^a@K*$z;!h%1K3B*8rk_QJxFQHVXg&(f7=A9)Kk$mqnQfmEq({R5 zUaU~qUke#DZqHc!!*{?T&`#LcZ_0o_9FFqPq z>>(d{pO1`IrgI>IY_CxmZR9O0ygUzZiwx5SWkg(r`f#$vSL{SdL>Cz;naF-IG55vk zSx1?%w5lEB8Ln*j2SQ(fYYyP!4DTNtWS@BYL{84tIOE-DztO>X91Mz-3=0bjaA{&1 zV~5JK=tn%kNC^Tm?g%QH)4F=O1z0x$pP+oiUg&8gp>2+Vk`l_~jf5AFsF+z7l+=A7MlzcAL*Yh0V$b3`4bW*mDGIN8y8?4Q zq3m=#JfMnN_L&rxY5xpI090JytP%1a1S82ZuZ&54fHpZ5@X(=Hpv@Z?Yr*;garlg; zzb%mPc)QqtD{Hj6(}rTGYA4aF=78xSKz3Y{=OOSyT@n`|;Ee~Sf&0v6^{gzFX?kcS z#SLeO%22>DFnMAtkcUS|Bu=4H=BXPUpkX2g3FM0sc~y8-XHeBqyFbPS zQHq50&qVqUtM>0J0g*p02uW3>(!c-rU;Y36YK@3!E*dvUg{Ne}O$mV*2rd-?OFLond0+9`a%{`&y`Q$!>|KoitJ z)ug|h)V~>YV2Hn=K=aMk@;)*B^KeBVfF>khrIa84@eOn#f(eZ`2}+v$Xa9e{3Wouj zh(M?*Qvc%{_%mU&52!f)jRo-kW%BfS@c@*b80i4_u%?xX@;}bAz@Sh8AZ7I0x66n% z@%)Pb)PKt9Z&#t1fkyc+6e18n$zxIm&N+rRQ!D{V3-FN;-LLvU@b^^! zX8|yIESL~IK><1uhJlJnFkp}{psw@cy#f7J1P3h z@;^?3Knnq2l2~xSF+u&)6rlVDtPEudK|bD+_o@`zN={YhOOsk{`XK`uh_v1o`L z+6ak;`YUVHQeE92MR_!LX)?a$GI+(f%bj%=kB8P?8%`upp!F}g5ncUJF`m)iq5m#| z3F1E?k!A+l%`)0rbX4Qq^gRo7(_j!8**GX(x=QEATg1x7(S7Ohv88l)W6TL^E@@Q8 zJ4S^{H>!~V!28H1_~E5<*8JinUpFR%%uPya3=m9<#}UZN%m4V=1DH=L6#&LX<);$* zhYuep=K$D2p-{M%Wc2?n+X9F{jkrOyK7|I`Rk2d7dMn6jwG4DN2)DB}DsyQ=?gJ)$ zV6;P+5IcUE^i^NJ&jvZ2tZP0`TvUdc6O1BwEnJdqi@V~p7We1nv4*T)WG}ggVdCas zZvrYhI+WxP%ix~-2 ziUlG7VcYihcD^o3mw*@}eXINVKu{RYLFU+)j7Qe?VBBX!2Nrc5M^WHBcO}UlCH(EP zB2prtZ$mKuEI)8oQY6_;-kJj0zb%VEXE;!>5rGkl#ZS(5z;RAEd2psuR0(bI7YE6b zcoAY&Qq=?w>-&oS+lU6}I6@ZOn#K)^R||N-M0P_97)o%eLfyo%?mo{guAH-Q;Lcgi z+KV~K$nlI&ia$3-mHl<2@nnvujaKK@UpJ;2S`eA0&}bImrEYC)0UX&NRNHZe(?yJ0 z0edJc0FXoALOPEplL`_<3?+|Xn#v)#Z2T%7svizG%Y)O!(?Z;^}C3A!AMZ3{*p zO@MH<2SfJ|0uL*S*g*R-ycoL!OJby4jRVVN&I1)ikxOL63C&#H-bD5@P!Xs&AX5aW zQvl+yPIl3@AE0svlIwm(ThO&#^!(wrv>kO;r?aZ;bH;Q^PbZ3>geflN!znWh$r#>O z&S!9;=XVO-q#VTq^Z>Bruv6}zu*KEFhYQ>Vn2p2irAO~G;(!r;7QLLp$Gv`nCUN$MOgyp|f%m-*%{ z_l5RW=lH?9(^yC#e-X1qhb)X zvjniCHIr=`5spHtzhGL-Zn4|0j-k);Bj(T>;Eq~?v(NRyB`ancloWR{UkIs|8A`j{ z9_$_c-R*Kc&TdSDhJgf0RxCtpr6`B#f<9mvMg6~3P3j*9=mRAx+TZLzUKJ6DV$gL0 zCy=znclKNjM`pYLttbXt63%4Y@V&LFmKlYNgD|~4nwRlzTbZy_fyvAl#WX1|l9Jg> zyCn*OTi6Bb#1v*r`@2-~s$DDXiKC%HxK`3EB>r?ZC)K|WDTrWpKmi-H13EMl`gczS z5(@MsO-3(lh)I9_#&iikmZ0rGaIQdSTj+;NIlC~;??IC*CXtlKtA|uWcMh+Mo8V&6 zW&&DZN4$>E9)LXx)OtGs$R@yk91)+3g=T>1PB9;2*F{j=g`(mAwKx6*5&mrb#K>AX z<8wCdOp0EFSx@-`9uY;C8KfQCC9tWph*0@@c6>i44L6^my;zh}_h5R46?`f)A3Yqh z&@-KY*DIu8L{__H>I6X-0L*84jOq&PrAT?dS>3UpHLrNu{equPATJV@csY0(EH0;^ zqAYx#G7{9}3Dn%3wAr$^J)Eje(s}I!*S8{NWtzX4Te|_l2`v<4g-z^Qz`*TmjLQ7o zz&Qv7@GJw{vkz%?ZLYtxQDyC7oLU%c&9Ojwr;K3=1afSy7!Sk#{CHI!PTnZSshifo zj1XXWzzCr`PvUsP5oUY^dxYCVg5kP)Qq;kGaUyJ7Ps;qmogSTxhG-TQ@_8(G!C z{D&Cd>vL{yR)^vJA1B~R0wd~ZdieK$n(iyOKtNG5T${+Z8(N8YVzbirPgqEWgEAnL zSqA0Ygd35L@hmIL%fwL9C-O$oxQQtX0plOz#eSht;0C~>@bB3zSb{7hRDJ{85XlwT z0szr^&=ip@NeocSEBA}{?K$@jug0a$^giG4yy)e4z)UworTzrY)cLw%L)~`L!)zgJ z(3@4!i&A-(iMYdfn>yd(ZV}_i{4LHK)Bf%D#!`NGOSFZpe4)bc`;vzLR`=rh%PGv> zC$nc?=(zERx75JXe8 z-A;MUj_sw0*+8}%3jIp0W@3D_*HhXZV0(;yg(gr$|99UN6LhPMMoRx%HPM-U$V-|F zX3JEqqI>~Z?@%#L7^Cql0(E`BGnk_u#z#goMlW`CLNWwa|H*v&A1mwOzv((=at7~{RM>1*_4n+|K>aw0WDkugkQ-PwqE;hx7|tiA|GQyDYnTPnT?5 zo-}g#!DgdH-{VvA0YAv zZLS8ZyQ-L0b5>2OGlpBYGaKG7l1#Y|kl}mTBtHrrVhOj#wEC&r#j9>hvGMVrwJAAK z3JbCvt%P0Lz&hFibqLxmrvBfZ$ij4TRA?M3c>C7Dt={gD{eORi2+m?{^YuC00YI2E?U!g*shQ-7OX);9ml2-O=8_WcV(x?~iqnWK}6Oc#BGGrk!e7~l(L=(3n& zR9hlY!Fa(WrUSW{GG^%fU!_wMmSq<|t5=v{A&@gNGBRa{FBqUmNRIOMj4mGES3THg zt88YZuu6SIJ~8FlW81L%C9ktNvY&i1^KC@*miqPSnUQyDQfZ>;U13srD>CHz7Z#e+Ry%TBX=*EIVT! zZYdMiGy%)_8-<;LLRVx>&SWe2-Kf2Ik?%e8t6&*~?6|`s>E1$1c6y5A_Wq+rCHzQo z61Wv;QJ!7ct_|$yj=Ih0??!A@Ae-fIy}(PuMN9~mpGUg!C2}u}bt3AsYOyEw1A3YK zW%l0U&C|mEwJ{#vZ$i4&%aOJ6Hc#fv!DKqVmy9soOvvl4{yEZJAO`elG>y?1i+>Fc zAu2QCTT#&#UOt(oEfN8b$8wFSl)RFX5+5DkZSgk)t)FX)dmZ24Bt7F`v)5-%Qrs8Y zOIrpaDHN;*>Xe$#cYGzQ9~q;TzG=(-y0V+j(nR+@T0m=rf-9;~-JBURRwsAP0@g~N z!Xpf;F-hXNDM{xJ7nEQ7CEn(FVlsE`(Q2FAE>knxG*~=Bo5UVtfa_q=F8AYX>q+S5 zdF6C+9b8u4R5`rcX3+Iy@XU%;)Z4bb8a|FHoj!TcWT|*80J7tWC?9ME(w#mwbvI%* zzi>Llvp3o>v@w@;EC^ecWeRtyr#Za;E1zDE<+&fRx-97U_D8!vf~Rq)>Gy9chAR%^ z*PU{>ZrBTm-BhV+ilG7r=F&|?$~7ACKdhUeek)C85xSN3fr8UXhC_6pJ7Fo3sMUBh z96L?POT6D-K2TQXmHD-7@PSh%>+18P@thvbUexZcpU` z2%q%9rsO#}h#3sRZF<^YGUdkO%V72{X(MKDdU~cg0gqoi9YYo@SPxe6lC@FM3?l{74AfnUdU3yp@;SL41N*7G%AM}Y=rnxTBNx(}(4TFps3_kB_y>7OQkVTum7 zY0&a<-F}Jb%;J=9fqeE#PW5>KzMXh9M2{9-Ce+07^Jih_d0I*2$`lQmCD%QJmxLhN@5#yoFlA-xfo$5o+YSbl?-Un1OgP>~*<(@9gQ(Fvj=+r4~NksoAH7N1VR1YpJMZt zoLPf2SPV)FYb)FAIOd6ut4i?NZ&pvI+q&I#HdZM3OuzH~$rK_;Kmo?)W%u_awqDe@ zb)q#oI&*Fwc?M?8@8*Gt)gmvFrmjuKdNoE|QSvpdP?VARy%6$GU2Nuw^SYfy1ozl| zR-coDx)W-|e1Xv_bq&FE(>WVH*Yu$CCnGn<5H;^~SUIJ^dI1249_^N@@?FCotH&9J zX6uZ_yP(h?8JCdbtn+Boo>K(iU_ggzzZ2@a4jTaw(KYtdCTV4UM9E z3229%1aGHt+iN`|$uwHkI+8k{eXSO3J@4UDqjMrnrCCbxD_eUxgW~C`i2I{3eri;N z_=Z>*VEL4KgMJpqi-oC=;elPXh8|)mb;xW*!ZRH4AXp)Nu8 zwnN-2-+)h})a<j5?*W`3Py-iVjh%8KN18#HnVC7O(a_Lz?>;Ir zhIJb_R2dl(p&U^6+2I=BGL2*L-A-qB3Xn_tum@9Fr9( zI$qv;@vbP|m*+_G0ty(ik_uQ&Lg$w6SK=31$o1g@$tT?iJ|Qg5(*{ReNCZe=ktlO^ zoPHZ;vj(GN&T*WPEI~Q z$Yl=jl{UV6rM1%-6!ADjXyI1z;XMw=-90p6jF(w?byQ)kgf`G4Te-gX&3iLVwx}YW zCb_KcN}kimc^j|GmxbOy^Y&I)E!}UOgc7!%`8;z|YOBE^>(4r@^_;GJ6J<3F9Kjio zZPHh1Z)b3QDNw29PC}_Ttj>{*G!-p2Of?&6U8GrCgv-ln}qA^R&9B0j#k3M+dkDKneae570Y-JZ)*ftqTNX5G>q z`7BvEEsZXYP`T5dyVDmUL`~*|j-=2l~zu0@r<~o*T zUDsk}vSdjXGc&VfF*7qWSqv64Gc#JuXvqSLEoNqBrqiBttu@!)=l+3v<3{v{5o1)v z=&tU{&dSPsUZw50(AYMO^t$d`&I<`XpRF6cH7WmD)~}as7B^!<9>sQ}%WRh!r_d|2 zMQ6=gcL_|>=)niA`{7SdbH-#Ne} zpj=kdF}u}G0vZ%n+hal{!9np4U}Y9ND7gBA^V@ssR#3XE8IN0K6V|Q``su z0p4slK|Stqqr29Wo=UCeXut)U6y<{`WSRnG5zIQ7p_Yh?t6SNyX9yjhyZHCac$Idk zPH1AaOI~85CB~Tq&$A3h^2Z8@=|@3fT`E086emq|HhPlgr9c+HgQxRo-S5`hbCljp z8IO3pP0L^g?W<9Y9kWnr%wJh&-kxsuo3%eB(Nv4lBFq+(Ox(dp1ya?2HX$lptx9fg zoPYls7L+0wi5BEC(p~ainyx*Y&s7pf#m{RL&?^43NAZ%$o|Gn;g?{?ocLc~mn0pbF zC7`~AO?g^{ioWwc)PkcI=5@B4n3U3ef1=S9JM6Lh)uDwqBzZJ2b=JC=F^WPo3N?n* zQNLs;TkM*)&hq(b?rZA1zu&I#)3Og|7uX4x#|!#8gJ9~Z28p|l`r}DbjcBf)Es%D| zX@53>DJ8rw^B`Fz3H%M($Aq|?%6WPLX;Qhdg^R5eE$1C#CoHiV_U^V(_?_|d9+_&I z`U@r&e^3Iic**EryOn$|r7O%nNw41w=@)ZbiKNHMVGaqCdLV8B=eAL)=}Mv6Ot`O& zdY-Rv&%tV+K147f8PwaXhG%6MR}+DzdEQk*gPH&-HbaN2*FmK09W66Li8XMQXV~wZ z!D?gC9vnEpY3)v;m9(-RW{|VK@vXRRHlD6t`b4gTKbaP}VJB)UH|MwYq{A8{j$1FC zX_~q2CUOw1-5|5+2EX2LY$Tajx*}AhD0!Q_IIQ@P?>m@1%B?jAnYUl&Tl?^oKs|oO z2M?X$Htd7urVnrx`UwCO@ro}ajucxd6$bjmW-k;0W|f^<2R!S?)l!d<<_Xwa?2YA^>T7k^1>^bBywF4Da2- zWWg7}!-WTP8zN1K-IV%jN#9U8r>K_xb2{m+*FTjRHsHe4a5RSRiGn<=3|kRW)*isH zGx#YLOSkT&TRpHrWRB4C4$wqyayBde`eDr#!FjT~6sYhDOzxm(!Z{n_FVTUBPAzA+ z%4Isk&e1?^^qpRzr+Jfj`mz9x%X|phD(zJjtavnT*vqM0;(}` z^W@OmU8!Hq;%NG1aV z1SJUVH}xoZmWj=fyGu>?$i9*%->$cnm%qKy0=10LZ9G~h2zZvzG`)5)ZB!^$DkVH6 zjO3sAN1wuLSoUZ8N+{cvNt5YQ89IElGRSSs!>4n~U$z&@NHaRzD<5l02iRnoW4!4q z!yJ3ntPDttyEqRfMwd#57fz7BCc|NmEsT^(uIYEM`erN!UZTY(BQ&zJr&WO$5#mU>l9$a76wg) zDlmLE)l|OCKRW-pl}W7L?rP(0Yd#c)My@y@4Ypkd!0s*=&2n~IXtY~mheabW6@nd} zw<73~1I~YR4Qxi{hXK3^M7ghK^r@=Mw6fe|y;FC-@6UPws~o5w9w2B;Lt>J?3&PSZ zRjI&?#D}OO>pAH8s{dd#xnWmdU;y-;$muBG$1Tns`eJ5~U zd)W$K4E>X8%pC;GJ;6sqR<_JkH&6b9#@!CDVym^}`YNS%nss#quTH*3biK8f za8!axUMW4@@%mQdk?LWg>MQqw5ldkdF>nzpwd2ocONfZjVg;b`i2%VN7L-ei#s-f~ zt>{fmRg~0E>f*B~srH;NZ&6vZr0&8X|L)fCH)u2;rPkwpl(lSJyaDDCG zuEc|-G+jxSjkMqm8fX?pP&K@5k-Hd+(XYF4!}EB%**mbz({wh;RnO2}-rISyoN8ZA z(0*9Nt>+ndm#;W)Z|M_XoL}r85&P4oCZz&2Eyh5`ct538`g+rs-~OHjYOGqlUk`mp zE_#$fxysoyUQn%|sM}>egX+JH=G3|8V>WKvwUo|{Rnw`J-;61hHG0Lo8Ah~8rLwa} zGvPpXcM;jV8{8RsgK>|(j;&vR9<+L#v#xH&|32RAyjc_0i*#mJqTjG`eM=i%xsMBx zsK5G?6D+9!L-zR%OJ%?qa!Yq!iclN6u&|Iq<-y7vKX8|Q3 zhWp}c%gYA0FzL*vm9h1=JNCv@HSz$y+#~(db>C#T@2W}n5VBM!-uC0YTmubY-}s4I z7b57pO5?@mOBKLAS3%*s`K5`q&d$C(UK_Y3W)QkCd_q}LxmWEd0vh< zp2torX?MJt=UC3V*7n)})V(ie_S0?l&%y=Lq57;#nDe!c`gH%v)WpU*Lr}T*veqq= z8{@aNg4!$nEk%Vh`;#`ZR8U?~WOOKuV2mkDc9!mZi$>;>Y8j+Ert}mAZ(JN` zn$xh3XSDF4Y>HkpL0h4LVRj z9)b;1I^WK!b>Gfuah);AN;jy~=1=gikL(CF|8YpcTSy^cGsaKE9u*f&oyF6C8cna0 zGXrJ%gBF;W2rXA}(|GTdpX@p&>mtU01?15%kkzHYCR2MumlJfw_{Sf7M4w9!cV$WF!(JWXz%RGT z`jdR*zL51BM!nx&PWEE~2aKU)zqRgV9{ALB#ViIUaE|SWULhtGvRMhE=;b`<+P97U ztchgnrQA_Qq`R2Rr+s6)AXTn-^y% zd<(XyP@eoHv#;F4Y`WxW)yfxEzR>O0Zoqq@q6IMU@m2eF%CwR!Fz!;mZ#{L%^mCYu z>F1Z*%q1MBfn>niwe?x;TRb8&-gM{K^lp6!fe^C1%wHh-F(&0ssIXq;BML{*AC76E z$eeA@eDZ3!icf^LypCZc;z#MP$_?y#_~UdRHchI=0`CJ6A$~0y*xO>CS3^k4NzXhX zb$%;sm6-d_m*%W75~NlZnde=L4dMlugYV$R_8?DUI+uV-uLUJthjCoz4{&l(d*557R8x>Z2FW_ z+LY{DuXCTXJq|_}g_RJEvSD?kMJk&UyJ+yWbh49j9L08-)$rj|lh<+Sgo7aqta|oq zikb@x_3e&?5?QTEC#e*Q;H3l&cItSmC02VgGAd*~RY^jUhYI1kG@8{QVgCZ9>LZOp z&R*PCEjpL1KYE)wx*@g(&=F zdz!qY&mo?TMN%neKsmmryAmoK*6yGnU1~*D+we-!un2jN8;BVxi3Sns3OaQ#02Qv% z>&BNut8`p|6D*K6(hn_71HdWH@SAq2IKxTx7{jj+J+)so#VzN~bSli!l?f$M@L%ATCxpVdfY`x1ul9aTFtMabu1zUZsU2PXCaVQ_ zTQFJn=Of&fkh)^u5%j5Retb=ZmKFA(M>?)XuwdSa!r)Sb-y|RR2|GD}Fp#y2eM& z&!H&hUyo*FP9XB977RqBuUS`WlL{z8Hh`JBa>|&{vmlScIE`RO*6vsag9RFkWse;Q z-7^aQPEbb)k%yfRK0p#0%woU`=#ByMuJ^4JCV>ZPGw{^DD~T5thCN8EU_fENi$Y{B;uq30g6qR5h_;GBrHkjJ* zEkj}U`w^Sia=X;jN!-J#vq1`B7;SZ|dxF_|k(FC5yYY$Su51-m3C;TBWI)Qpb@?1Z z0A1y}yFIXpcVxmZ$smS5*J*ILCHncxe7mNrf5@PMCTUB1A`13@dzKAo)`G zlB0WL7k0*gwSg@s$Vj6scZNC)c8+fu^S2}rNQRk89+ zE8QoD^5~Dh)EqJzFti9|*o;zEM;@UYF|8V$e1k+vh?2P1ei^Mqc=1YKI*9JZs{59= z0p3Sf6*Q!gHvb?<^f3y^NuyrNH0MPJDyZ{!78$cwCRS(Ae$)0_n^O*^+<6bqSu}Yw z5l}oblFzk>=6O%EFb6U|)(e6ZyoB+|=nSB+us%C!P18?z{2XV(*n+o?lgIbS%6RNW zc(%9Odpr!mgZe#CA(V5Hw-}cgAojuRGpMC>78k2ILOEfWi;MV%fQsN41v^E&S-Tw^ zL_y%turwk;Qg@gOWuDg1`Y4_HC(zvZ6S3XVfegyN%)YZhRX-B6Ez~Kb;tG%w)512E z%vw2QIb0#Z?NrFOJ{v$Y9EJ%QoP*0iQ1}D-u4M2!^0>L1Xq@P;&;1E?+@i;Jx<##Y z+~}x0Fo(6A^%sR?LDn2ffIMT{w5?zwn2cj0s?q3HLxsj~xVJ9^Vathf(NiEGjc}+5 zX!xH6qG&_(=jrTt26KmS`rQn}60FXNkS)M%yI2IdvDyzKpfbkK@;mdRW+-0b5KTt1 zg^>|*suzoTA6pNhv8T|rSL+~$3PZWPRQbN}p8}~(vpXiSTa?H3tnDGr(l^sYg_e$f zd~CN+Gy#d2+AR3T?jXwZ9hogTn5Vbp$==V=O?Kn&Va{krc${Js&p!N+8QaW9S&mmV z6b{|)52<=Fj`xz%5_?JY9^+Me8(eN<88%~eBol0tq0Wo<0X`%iq&UoM$_q^w9I_1x zvJSeo8F+tKjblnBJf;L%A`FG>YH-YIuR~hq zEG}MLIbozRf!F-SbDj`%iDf8u`$?}@WVS^yHH~p^nO=@cuPLw}Q3m8210zliS?-4D zc4s1nhJ^8A;WYU(+s||XN}iOiUF%xhF@kZ-Kd~;3^sB*hB65Sw9Fy)L`24_xSdo$R zluh!ScCXi?eH6ViAhW?}-!3B&O%TSwvWC|n?Ug%_`m)}J-vo04hgIAMZoT8R7uepc zT911?oqNr^SazDqoa8nwRl+y_lFjr2lB9xDrC3ZU^IJX6dx&MzVBuay{r1iKbY>Ro zU}aXj_c<$WcgA8q&1$2$NdiOpird=cx#7FT<6{Vck}AW>H+`{0AH>Umgz>^TqNmw) zXcHBd2*KKzAGMHX>^aZ10NFCa8w$%}U{|k(`ECu)9pTc1g*72L4HK6YEXjU1Nez`frB48m*qJ54 z8+XY3h>VMi6~sIz2VP8i772D@+H1{x7Ac&-mCco&E)gLk6+Uz}o)>maXTlnwl`squ zT;u4#BuvR^#c=JO;b4}j<&QA~Zz}@;FIT4WOlx#{&*fzNL5azq>Rw$!uf=b4Qhse?X(~g+*Etp3~^}QY3OPgl23ypkWDn*md4 z%!U-#?d;U>(ed^P=`5B}|ExU?B~vl?;8l!_PKqzZ##`}y!wIbUvT1=;2+84Hs_EG-wO zYsZpelE_KjiAn|+4i`IGPFm96$V=rO4;6cCmTUjJ-deb>>FL0tR8*WhwU3gUKd3G}-zW3NUC0f6unLcCJ=2 zNli%p$`rQJ$mvUR>P{r_qw6DRFoOQ(@CxC|$j%CHy&5f@ov;es9v8Q>h)A)E$NG~9;V3X7Sp`#W zS-o5*;^SQf z2K!Vb8)!euvKTY5u{x1yb;PP1eAvM(2;O7q+YAoL+sfcu9 zm>CW;)hnp2@Ji9tHKlb?N_A=@Nokes)nBg*T#ml&o=%$)b9FG)$8la3+?Pf<^xp15C^UkML{p|q$miq>DJSx<&RDj3dzc-&Y6X{ zTuD0aVIR)hPO95yK0ILW=85t)2Qn45nk{R$D2rAc6q@>s)ggbjm*~yXIlBBG^vu!9 zsK4l$KPoC;B(rT0g_Fj<=J4FM$|c#T0nxnERFNvOWlLg5JoB-SBL_javhknBqrE@W zHuF$7Q;$&nU+m19QZ?AP`AgLs=1$i-c_wYndpYf*;mTdmE?5;Qto2wjJ6ds82>1P$ z^TS0Hs7<{F{}@W)Ry9Qapn6jO>e0>xqApH;trA$z%65W`^VYEuzkp2Iy`kJ;kSyA} zy*JB-A(*?`_q5g?^lexi#z*O1xW?m{0`jDQp`M@QG?TvTx~b)5m{lA*t!%YeDhLju z%GiT5WQ-zb*xiyk8|A@vcvNm{F`n#i@iMLDoh!&S?UV5l?59eCWsutGx#u4|>~ht; zlk22DLB(N2_pC2B;zL$Q5{lG#s3HGexz+Dg_v0~S2xn2S?7Sb6s8B(eK)^B)-K@Po zK$wY7#LL}(3#(`>tB-%NUqm#~hmKM0l7`{wPy9|qGrRLl*Tf*`x;qmq8ePiLXSdfi z7<e&@kc0M^Qg>tn69#-ArQnx`!mIF=bR{z|vlC&-*QwP&kn%EG-PrsEL$@5ye;qVW?H6gd{nF=1oI%yAJ$2 zm_tE56E#urFc8j|N}k!^;7YSBszes2$tlfKPi>MbENsRIjZFH-MW8-cafg$;OeG^6aa@xVH3nH2^xU{1qUWY_=m{7W%+dvu zJuiTeCy_`HM%&k+vPhH&;~E)R`xgD7d*|9`Wss(PFr|0WZIpHhk=c-*Nt{Ud4`CDW z`?dLel#NxVgVv|N5u3!tCHP~T+l#UWZNZ>uUC@7Yr!v{1I8pqzgLm^+l-raoPAsvZb`p*Qim8C0+fW8)l#ukgUD z$l9l0KHo1Tp}~l9*4G?2v8$o>2Y&+f@m;vp=PN$|OFoFQ>f>eoDdoL|bvXUTOi+1; z&oi(pmKWF_Tj#eO%>1;M45p_i$Qv*i=6FLpDp9yX{7~jI-P@Ow-5RH^ ziK$}kc*7aZKxR3gj+d6nkhtl05|ZShtn=P$sI5V7r4IAsQbwss^JELz>64nr4EtLi z)yJA8WAT*5E5y_*MMM-3?T`&teX|+|g7?dU7(U5z;uYUwxtyx4dWu)K+zmyMr$Xd* zCv(}s*8w00WEQH(Ds1(6y#yd* z|5#`5!nvsP_auYVW%|U&p30M6J}a>pe48?}bhk%dAV*4s>01bFVn(XY43MqMBSoJ) z28n><1$bUsO>XsQkgxusJE0Vk{RkW5cx62d3}%drp5p|5t<-V6)z+$wqNi6!v(S@f z4UN!Za~;!)2}SsF#&4lRb*_W{t;cs$O@g=eK63>inBU%>a3SfRRmkK=bXCpJx)*({ zLPVs1Y9J`2~Gi7axP#0&6S=Wf$R>R{uqk!;y(F}s0scbqeRRPK^U z6l4kl!*Ff(*^11Su4_0Ro4NC9rC512d!iX9m)ar_u4(~jtBp_o*Ubv+IO z>?^AA^83F$S__dvy*_4W>dO>>*VHj0*eLG;`Bb9e#wEB%OAYDW8?JPfVaO5Fw>W9U52E&kpt!q0vE9GhUx=J-4IS%|kiC-f;L>_4(?(9vvxAwL`pM zA1X+MLPp3g$QDNQ(F=Z|XMTA|wX6*#oR3+}3Vh&5Pgn3aRi>VsdSKba?Ef#W`RfoZ8n4HhIY z2=4olNosP=8So(j>l=CMG~r=G7&Vgzy0qDA3IWl6A% zWNj+yv8?+XQJ(aAgw-L=K4=6-P*}aXfT#No9Dqv`Ln_B2f(guCRg5=-a{eWRKF%FN z#wxGNJ%PX0H4QO$D7xFKFCwx!Wi`Ex9 z`vkE~p}i$97kewx%90@%{>Qif5ImjO8wsvIulXK1{{9~XPq|wgTY%tsqOJDq*v(@G z_lMv~PLPQc89dH6?&oTN7Q$b!%V1DGEJxpdGTcg-nxy_wFhRqxi@@*eO{mo;15D?p zmc^Mb8lOF!KCAkyaf}Xx5JMF8`%jYxFG>8#q{;wS=exal8>^s-Sezu!$WsY75Yq>k z@9Jd~MO<%mWaRH}_uW`@+M%QoOF_cmUHEPi0#5P)t>GrPcR1j=J~hU98+P7SPcZ`R z(rHX43pEDiaCi}cpbApTO?EfiplwpW+r@vTLo6-`+&=Ie6uvjYxBNkk=Kyzcn_i}M!D64ng5CHbQHs0 zA&g7WvQ6h05x=rF^XhG9XX8MNw^{BGp}eaQjDa+R`sR^LrEo)Gm&}U~zoa1v@m+?s zPd57lCe5QbsaA!e#j9e9c>!`K4Sb6?c*C9jN2LANTy|A`$^aqg#Bv8xMqvCxqO?5auXqJPKs$g za4`G2{|=z!K@e_T->MG$jD|1#hcqNsrat*8Clu*clD0Amdq?y%>NorU0(Tzq>dgc@ zo(O$r2&Q`FJ1qGF?p$}CD26S%9uy(k78fg+#{saNkjzCB=zn87sk52kJw|jC0#Ta6D7Lq3)|E6~u+!nK)7F(t+m`7l&ieNAlxY+4QqWP+8%(SgL zbAG*t1xu7ns} zbp%u@2+x0xg@o6&&xI}xgAV!j;pU5WSWCcQCt}HOqMzM3Z$L%Au);@Yt+u%i_%-P> zcWjTqx6tc8yvVV1QR}@6!=LMXNpkfRY})_fcUFn)PK)FgMdNWpBA6Z^{DbertL>Qa zlX)%dx2UgY;yU&T>F&PUw8AMzCu(E55sFy4{C=D`-vJLx$-mm1<_LqN+5fS3tlpEu+5wJ>iOENhzz+H2tjkPR&T83{2S8$o zY4oKj6H;7r{_ZI;CSGLKIrPza(Zpb8NZ~8yM4IR%pj7*){x6QFt%4CIIaoGC!XK>K z&QL-(z#s%tEw)4t9G^BZh#%0w1k9fAPJdQtGz39_B%istUt?j~j7yHHS!qsfl~*3? zVA{5nV*2t0F}^r3sPza8m{kZ+kIg6CHL3evn^oVdp#hNvMnl-8=%M`?KvqUhok*E- zMmo?B1HhTS5y&?g*M_vQ+b6Wfrc8(pa-zzYXg(AX z0Y9%9GqmBFgJ@xxa7@=2{Oq_VCt4eeCzblhe8=q{t-bRv;K4V6+=E|n!~y2XadCnb zEgYXUl)wE}Mb4-Rq)=@T-@e}VXsSxXm2rs7M;;A;>eO9#Y(ihEFB*0yh59Pqs}SVY zrOi<$=z znF z?uco;Y(Bgl8gtz)FL&*5hD-?=85G;F+dGG`;-k9 zKWR^z4aP8x4(<;c=+|~sKp17zWnxYx@I@*ZhX@pcZq7UOI%)4O%*YNFz&aeG-w{4% zmyFgKK1-xtZ&&OGGgUOUN@v!Wb-ev0Mp`g!d;WEL(icXY=YFp>&p916!O`TBT7^gd z96m9%s zf)V9$K9En58ON@rXhogC<}o4qCU8TXTlHlA>o>_cNx{2y9Lj&g!(y?Z1wy4e4a(18 z+IFJl&i%M1Sq!`_4b+?G&Zx){Pah2nw@&DPYN(3O3dT7ZCs#`5_@N6`2q`MMMDl zuS327W@mRViqED&w8s8bNL#abU#^)Gge-M1@bX(cs8vGoq2H=BM;zAhr_M3Pa7{P| zi!}W7?3x}9jFP|-{SJbYkNr)P&IGa;=k4m*hk));l~Tz#fXJE39sdoEFo_$bIy)@C zwpm&T9qI>tA`4D3b|U*l$p{4m1^Ms8-xvn@^*;Y6eVwYk1*gdtr3ge5r$5Qr`-M+R zH+&T2N4M02ZmidxkD@I*iCiU-NKm}tn)mH5Cgv;)KYsRoXgY#dih>45gZ52988uuv zVmd_PIXQ}969uj}w-_asMQJ*^Wq|tM7fb+}gzWYbIhb_>Uj#GlIqnOjOE#3pr{lXC zfh)0hxfj&aj7^p@lRw^U^Yfl5n=y!|t~uRtn5mG2Du}2W91SeBM{kuxn6o@)x6{;k zHD)b}_~|CgIbw)Aq6HxFztm(6ia1H&D7*Z9eE_+3?18lS>Yov>c}5eUa2U5X{870$ z^A>lP(&?-*Sv-}*h4zp~G|RZ9P5z0S1-*sNs4YZ(D|1nV%!;kCdikx6mySA|gezPe z<*zpF&gjcxB|<5TU_DVL2fy08-JcFi*4_8B1Xj!#zESRE(`q&8XOivE8~GhK z{~7+mG(S~yN#=|6g-pZM`M!cG#t1)XWeAW}yNYif3^t}GwZy}tt@N2z^VUoSCcDKE z+`236Cg*Q9&z`vNGrLL}Y0qJzava2OiYd2$xTE4;X&k(SKo`$!pbOP!QIg}W9J({S z!&Uw*cl6D&%F)+8{6FoJqlu>cTDk@MQ^J^$T|3{Nj)5R7N*GWmhVOkd$?JI+smC|2 zE0P9`AF@I)Q)Pnp)e!k0rM9rE$w`vB@PkzUim~IMmot@;rqdS3+C;YssTJl z1a;spheIFuXo+rsnX@It;6+d{J|1MzuZ7{0SiGwjN0ARSzxN=hsWugVvcLq z*!qur8CW`8%-`oFUrX5Wke<_g0sp(7W~~^?&{1Q7+P=7%Pf9dVq6yGXQld04Gp*5- z4w~Nn@z=th$Jv4V#GNOoM$FJsWRCvZB%p8m*pH3q1PYmwTETC=&^g6K zd60n;)p(-(er}VK>o#v;D~H8ZvEAiV6Nkn0V^qZdau1eLFf*_ za3;3IJ;kWo_4PH1Ih*x@KAS_nV6I@<(9jTJh)h9K?)Jy-)ZSj9hL0lH^%B@3f6zb_ zb&(+EK4x!82PfbE*n}Z82DQ$@v>l0IAy$H1e#hAjg0cwU`}V&ck#mDey1{6dbYyew z>kKc0FNv*1gR~%3eLEE{N~!<;dT}UhXQl2nSzpcYY8FK6h~DoJD| z$j4%(64|T7FfnDFs=kA4>M!6T1dM;~EVxCnt09;??Z3?eU>y9@#Xd*``MUi^<_NZs zT+1?-Hy-EMf$yCzTvsA{`bb!&-FB+TV-Ah`lZEtLmW6`J?8O;!)1rJ>iOhrM7qyrH zB^8xn^wi#1Mw|tOSZ^pRG~?~jEHM;kFnE0qAl3HK<_qWF^HN1hpw_fJ**D5uXZ^Ef zpTQgfU_F@p{LY%O?U7MWgjji9`w|J$r7!mkeV83lBiLw&CLP|R7jNq%kOP9jyQk6Q z$rzvhYT{6ux?WTOjX__o!WNE)S1)W*urL_`9%nuU;7Dt0OI72+%MX)McS==sz%7|T zoMuO?ipg@oaN7S&(7z95J5Xr4w8~buz=(}0lP@V#z1%!=@{Io)8R%4j?|!F4ng6u^ zPshJLn1RNd@)Rfv{?|bM$JTx2U|FYA6Zb&YS zcsfja=}FkXPxb%*>rlV{_YOF6v@5p~%R|KH5u>t_5v7%%C zD!uGVI2ZAM-t&JwM>E=>jS1KuWad&sH+nA9!ap9=53MDTY#NC`iR2St0Q>A6(-p0A z%jz~iD zmcY;dAMM>pz4E^$3Z6aU@;o#Rzrg+}96@I6hxes^c?< z;)cu_6ND3@kkMA5uHb}#!CEu-(PG|0^BeW)Pvu{Nz`StKx^=$a!DED-1?$o zJclSX;N@Tb=jE@mlVI1}hTHCNJ&54xqCS)y9~hFU!fs{U9T(6JFS{Ugg)t~PfB=U; z4MWZtE^5vh;ZO3kZhU&+ejKKZ!k*62)`+|%n)8W&6RrN2K0ebgLbFoY& zL;VncA8IlUA4TJ-+q`3H$An8OT=I03JQT2>hxkE<`v9gw*YlLM;uoM0Rs!hTE-o(q zqC@{7I6y%U6om8y1mo^xY;74jXMWde#(^_uDamw{tWfe+J$=^y+kPZt&ZD3dUrGgG3HlZfAL5oZA= z45q`JDsK!K0$K3yhNjx)nNkHq;l?tsGDm2#rxft{0G-ceJd?A~A>j~M8RqbXv2)oh zH7wSfeIidcismfV>-M4Kg`Mqn8_*4#{g%x|2|61mu%~?sOpWbFrLm_>&Js^nu#&Uz zKkq_0(6;p_@MOV$=Ie#1EOys}nLIk_dDnJVx3!Dl+fQv*oz|E(R7A>Sfk9~6L4!A+ znG27R)lB6xjEIdwf(yh@T3y}D-G15&6@;Rrl$#21Sfa-<3ip=bK(MgcEV1~!xM}M| zQUuT`b^E-MP#U1Q3_!Dt+O^$GNhLI=8K;ytd6?>ksf>txj{u%Y0;j4c&31P4DB1rq zrUW3M0pv)ohfJ@_cY5xp!wMzQlF})8NXoeU_B>(?yC1u_pDp?`5W^;at8CM8*7Bs0H1ix`88o$8xtUmE0fOBdNoW9 z76TU`IBJ+lW*)kS`XQg{Z@V#s9Ltgb&pV6pUvnbr$9nPWSlI}&#@?ya8lqS4<>SAO4^Vdfreh*B5DA3qli&BE@ zG?bj2LaOecbGCM3k%0oTQDL@5Rxn*7iKfCXIw@hNBQzqvhI5FabVVVOthk$rlAx}r zN76Cd>kq3P7Q+;b|1O=u-uM5t;CvagY+C)WH}q9aCi{7}w|7gNgp`-|$>4ZEW?8#< zD9=UB)MaJmPHAV8osWC_4BPwdZ7)jBr=!?S$pu%m$lJ$aP0dP*O-)l8qC6Jx45|h`e+@?bjj$ZjhuU|L#bhG6Ubd#%mm zB!pN`!}YnaU-)yD_hvvn-^)wJ53L+M`J3mGy~+GW>GaIU$MvPmql|~a!u}Y!cU;lE z_Oay>dJUujbG-7W=<%DEex&Y?Z#kk?;m)FFH*u9&lbuTY|LsLgiR%AKBy`QP-a*yY z1cjkP>wYd}W=UkHo2aaifjWTXytL|HkR%tnMg=7s8_}FzS!%ShNJq@{I4Zm3F6&eg+6bZLf3>_h!LaIf zYV7hF``V57yondkmg(lEG+$-Rmhye=VY1xLE~nvO&S|*wr+C6NYqP;x9u$dDYkr$J zza6bEx6|Ph9ygK8eWi}VRD4~s%c8{yE5fV`1T8dI|nLt`KPUe~5bPsHoomi&qduqyz+{r8|dC zQ2}X`?w$c@hEC}QrD2Bd?(XjH?(S~5=kxvDyY7Fi#hE{5&2ye}p7-ASwMPlO$9M;v z3E-9f0P zg)v@@oD4x3-Xo34a9uigQkt|!*hjHoOLRVQti6;{j`bQ-#pG=`lrJMeP=S(@*3cFBn(9gyZ z7#U-m;Qe&)8Y|O!4P#0qRCQgYBEJ4C%yq88c_I8|h&9FoE)qo|6$TRNySzSPe-y12 z$hLH@e0kbFB9%@w5IG7opSzAKVLX4L%y{%>kzl&Q#k zKI`Xvv0WC<6(^n|>@KK+()dYsBt;~}ax|e>yWy>+?pqw1)3f}3%Erbr6Li)W36jOz z%i^JfqrDR@1V@oU=%M zYsP=Ni}%Q09Ia!A5W4s>UygE8Yp9m){Pff~+*7s0S|4ot`<>T+Q}krlWBt-xW;%d4 z)C7n+|4p7_3xSbuiN{p|p#P4{ngXFU0C{|D;|XH}S;pAxV!xp@0SBSlQ?CC@~uNr1C&nbB{g09O5Mx8z&*eVm=-$Dpx){S!Y~ zZEmTPG}B6Yus@y%nN(@8Zg%%^3};mHx0W5bC0{j_psM@6IBK9yP-Hvb$&D%{MfY+Y zy=lMKorb-ZK!x$Hf;tyrY5}wrrT2PozW`MQ=J%TAc#HuT(erbX4D4UMK!k>Y{EtPW zVgeDzYlyCdW3ACdwOP+dlkw`>>u`QLmlE6G9}zy{v2KKYr;r!E%VZAvZ~?UI{4ObK z-bto+h5KfOaX*268M}WjB}%Ng%tktr)L`xOwO~`t~?LOiK2^?%bi? z%|AR^=A~bfp;!7nft>}U%n~??K!GqpfHz(o>8wnO>5fkCKi^K3v1P{1qOHB|-kVbL z;4$l4H=?yL5qC03pCpwIeD&WTth$d`l8b;eB|#MTgFiY&JOe4J_MyiH$6qw zd3l0TcOJvcn^1KAKF6O40gml?v)Lmaw>ESUj599?9Xh+waq~q#>yo-^PM237kP7@_ zPutoNoWT5pvnrV+QR!w)$@%5XySjcbZKy(A%WYFpl}v>5M6bXZYbWImO+jPI{+E~! zdj4I)4~IUbxbMgy$)*tP! zviKBq8h6|M0i=wh&c!F<)#;hMM76-+T!wk)Z4V5nk`L2}gVLAZ8=Q?^GuiND{FC)<}eENdC?7-K)uB_1CIBT1e?Ud>;zmn=)B{ z*1Cxzt1%A{tO_J^yZF-qpV|XKV_psdUv?IWlCl>*tsy%R-%!J~iFlwP8&p9UI+hBs z_Oa~VH8z`&5XH}6X5)M6FVVW`edhC(K~7CHMpi1Z4~%rbQ75AJr;E(OtX6dHXGVXs zd{S?6;erJa+Xl`VOZ^&99@ns@HbF!s;<3zpFUP+tIwr>--%gDT0eGIOSAe#4SSGP& zB(p%R*jlOclR55RVg;YoP$QsPf|lRf6FQ}QiKh5Ij4{eyzhjJk;{4C~&011MAm=HU z7ai_v8;3vPV?q&TPB>lvJ+R=j1m#9tOQ|5cpzyvGX55@$d{`r?=jk1fj+Ba$e@<1f z!SxhTWPx0bp>T@#0-puBz&%#~(R4razTQ%%%6ptnuLo|@A;qenp zks75X1k!Ut^?ru6o~Aq%8-e0yAiO(a!NV3CXQFdQE$5rsE4Z9a`;)=%%LPVKSatgxv4hq>_m)M1 z`x;GHcQtMlojZidm1XIEBK&X0NU_0`jUpuEAlF$?Q>%($$@fAxS@U#w9DO)aCXqU` zciG;sCsF_HxqAuAw+d?&@*%W_pK+v9nQbE{=03VIZz;jQe)Wlyl60lo+2(UflzBsfgtRT-%VoQS4%B5lch?=$>){QpjR|t|Fh$<67Kao>(B1-M}+{6J}Z8$C7?| zx0CW;rf@ULd6;kK?`MZW;4Jkcl!ymoCLX9T8Dd)VO)hsh(l{T&gmMB&TDkktZ7R0 zn@`3e-va*DZ`Y8MlUr$aXRY6*V9d3lv36+$OizK9fUw&LpgI#dgLv+ul`C^tScL)~ zM}=6&L!sjzNEwZlaqhb6+P)MnO9Z@}28g6$(&&W%m-#z}Q*bt!HbuBcDk!08%m+?tOCHqre11JozqXPhiwx2Ibo^MKE|eg2@tiUZKk z&&#z_#>4f%^_mpQp~gtjs1YN^6K?I)t`=w8ppyiBbAC_C8{4)Cc?gZ^;o`-4bv}B2)XHJMs>R z+ZP9S@u8t%<=Z^)0sY#6_$7V_}J+t|@kn{(f)jeod~{s;00dj3JkC%vk%%T6xHhImXexSXbzqO`<1#RInlu6I*` zeIxfiS(NtF2~cnTe_>slao?gxK^gvj!zDH7|wRaBv;Im8M%oN>ep5 z4+~fnq;|X}CSoq8-@`Yj?&q+=8X_55brE5)x0q$8faF9%!e?tFV+JgG{t~0kKpW@O zcvC4QQ5kkzn;HUPHzyB|=DE(jZYr&oj43kBKR11-EVX|kF1UMW6f||*uar5H05I=A z=T?L*&k&LO-3)+I=yfBni0w&0qZ`i|@`W7X(+z=bGjLakg-EsBuOi!>wjskf?$Ny- z*WCu%GB`WbZGiKx$nC06HAN4Jez@$%GQd-%4Xk>3p{vOStCk_t3jGd$;GRzQs~@Eu%^h$+1U7?J;*5pqj}@m1mxtLU z-{TbG^3rTgvw1g7KhCZ+)?G%Cr@B>bF@-j$1BJ;VPnQ#3uaVd3{Wtql6+k#R@ulYB z(R=X~)o0tg>Vb4J(sU_x4jK++$_)wUVkIGjUD={mF2X9SvXq{I# z5oVP`=g6{hEEQu&7p`wHr3c5i>i}&S2J7iooe>w=T)t`-8S`Y^uHZ~nT}>| zBmOB9Wxy(sLDosGAr#>sZ)6oZFRq%STYN`9Q(<%kz* z&ffOG#^awfA6|LKUXP|5={!Z?YV7E(8+-E3kJR-6 zvVu3GKGh^^=xsc1u@fEKBCp{aXjULW>qJ{wk2Va(#NfMsVc}n&{nn44y`??YJb9LQ z`7j}{cpHK-d@}p4o|h~f_LFojq{{U0Z^lBt4j;c}dPLk!1$Dip7~f*+-GZTg0;MgH zEg@7)Ym#zC2dlW=EwQv7Et$2JP0&Nk@%RLup1p~w>EQyA+~c2qJpJX~Wk_cIFEIsL zU)6dmgioT68h1kX5aHtij8bWkQ<50v1$|Ko25tx9W#n!pRu;vQQ(wSdkVmFMm#_LK zh1T*LDP=E%4{t)lIZ2uw7PT^hPKS&-sU-*ro4YN|=T~TwkDq)Z_#4Ie=ni2I<9pgL zpRH!;i1MFJs4aIkYsJg#(-{9DK-pS~bp;;kI+g`3*j2qxM7o*%+$=3Uc{da^>@-Sw z7?ZPVoo#DuiYfx5R_|__%;a$0@?4xzYs@i)kfXOvCY?uiY}yvrG0fFk2ITm*>*n4R z507Zyg_+ifWi)BJpo+Z33EsD34mytNUc-1F&=ZxX!(Hr4lr}+TLm@&<}}!zAej8qe_=6) zAj?O5It4AQ`{0l&K=_N&JH`zc>_%)!aPac+-^14UQ`KFJ%c$~6c^B}tO_wXgehRuX z+gXt1#Day(o!WW;lB~lpLyI;zmPj<919`zCQ*?wCieu7lR;xn~mp! z&Ow%}Vlzya&x=cuqZm=}b7%YUui?iurwr?9+bq5$-hdR3dsDZ03tc*G#14@Js~{3={wzI-4BX!?09O+Cc7CYo_7{^ zH33pXiCqcR)*M}(Mjm~K+(=xi0ZkazVJH{48T#t4fo}}D6T?d!k z=~<$4keEB(Cl^+9k*Y`ixHY$#v$61~JSS37=1h=5Azf^^gwcv<#p9E%>>px*etwe) ze;VQ>i`xDUAei=s`R24NbI-w6dyDnuWStlnj_vl&+x8~j16!yKa<=+oDUoCYExw`8 z_B@qnhDZaL&CPIAfAeAOJLVmc>A{hH`;{{n`TI~x4~$obakx^*pjUidhH` zdrOb0FX!M%-RV1!TYQS6`2bX+*-&N-XSBU(``(PYul~gSNlb6K(YgDhqHiHaJN#oM=EuLlt669hjZ{b+-akW@=39dT_YbpM zRWmvrB>(w^ojPX|B*wCxQg*_YHbJc0xf$Ucdl)4AaHsX4NwLJWp# zo7vlVnWHpA66dxe-ATjH!KDFBu-*Mt@=BxoDZ(Ov7&CweOCZAEwR<^-u!^J)!epKB3><}p8iP4 zTXo^pupOX9icZ|~GDy#9d>UU`p4YbqliA2^WuIQn5jc5Gv(rvmbebeeOMMqCPLe%d%(iUwKaT(p1`bW0ES&crB^P+`lkcY;1Tk5|*r+qrud82b^%5ez2^UaHD=~Y`*^;=O^c; zK7av;ROg)U4`F%UHegrhy$<#sV#C)?pbrL}sYwKVvq2ZX4Av^3<^5Btf zl*-S`~YNrVSpJ(or(ZbOJWjjr;)pWYXB|p3CfCSHAxWgY*Fn(9!b7U&2E)~dG zWD;B)-4u11S)_V-YL9ODdNy`t@F~1|YPCu}zY9EiOzlsk0X@55%g2fo&>H&%XCX{~p!7Ti- zL1S%O3{!U7#^MHwkk=$p)Pd1YvxxRa?GvoGbJ(iOjT>Byd=qW4;Q0Az^u3`cbW9_5 z#`(EEc}z~II2+{o-;m=IHMF``rjGk)2OpA(r}>uN_XeWby$O1VaH>@4p^9=erCE`E zu{-ij$O>n^4%jeh)T>>TJNk8Ra{NdiMV=_2E$Iv@Y1koM@L*_KK7rrUH)B6Vm@H?w z<7AgvD+{}*^FZv!bE8UFnXUv3pJ-f%Xu5;PCs0jMMJRl zA9x>ciE?N0m|3>f5Yb~UbzpIQ(1Q31o~LF*I9}|F0(ag{Xu0iTg{K)g`E5M?ZB21p zvX67DsQnS&+PYjY8|hTbh94w5?ag;oIbq%``=PM26E~ZxB(U)Uuv;CTc3|o@I`>!Ver>QX+^G15+fa`T;uk}Rtl)N|YnGTcV-FRkEtNC32#V%YsIY_&x zJy{&f0x6DwgAeL))3CA@o<5qs$#OHS>+B7;U@Zlf!)ouv8^#;Y|d?^mNNH+Wy39*=~=%3 zWvg$3=~>E?@4$N7xi+|tTb;0PriM|o={UKHg_k9+)|IB&S0}71tp#sXlS~KZgXB)z`!rg{?lw~&oFX=Tm7*(jLoZ;T z5|Fv)2eeYOUdF};!+&IE%9y8j#h39}tf&dmy71OcoknmR8c=lHCS)~Z4jwwK`Lr+T zX+#aX@#( zD{#U>=?m)YpJ@|yXF~@FHc=!P4hfn{uS1Pb_>}L;)a=- zQ)ab>lsBpL8A55tHZ?u6Zz10(J7?9c8rqVWX;RSozBhhZ`m!A_;wgOLGf2qp@!+pu z&`9;Dk_j*Zw=G%#|2w;`)Sf=jv=>C_!};wNGn<}0XAh9Faz-}YfZRi)AnXgs@#xdL zSFbpZCBKU*sR8y|B2%4GdMhQO-~^o{>vj|rJ9%X-B! zj89GHQxxISyD*upit*5reFk_R(ru$fi;?TeP8+gQ*ci9RcE6vW*fMhuz2tY#?X(MO zRyhLUL+#sECQ#DiN7$`S+a!@IEsip+9Q05FDETWzfesm>!FVZGobJ=EZWcGPA=?cM z4#TES>MPuZI{l9H4rkFEk0K$>%e7r{kB%`AP+wPt^DD}E!%B5lK0#6TwIyZj(t;4F zoJRrAX9tHIUd*iFNbaKl_wA_>muik^nV_%{dnMX=rrbClRe>S;&@NOHC(G_*Ohxx~kTz=(S41A6P`!R>Ge~ec#v+*jcE!5NEWk zu9de%j@QU*bf(DXa3yZ6XdAaF3^Ke9jFbmF6=otY3|*1r zyuO7w5%z2eeI@}2y}$Tov&$P@_bV|vCw>OqnNDzpe*&iW?{9zg){{#$aNZ^i8R0pd z8#Hc5_nH0$@+NNQd->_gq$Ci7^E}O(V}O)2Yd@_HA+hPi$~EjY`mbF^j<^$>L$PmI z;6zY~Tg8!L+fEF7q>*duEP`z+PckRoqK*Sf!RPm%@=_=@X!Dl5+cegic`(z^l@%AVMXHc*cL~5H${- zS$vwfJ%)iDv|kd0u$?SX5>A|hDL8v-%qWMj!xAgLMmCVfq*;NLww4vq!UWxS#0!cO z&i(l`5y2bSm3jKydO)Q!R_+dpD|X!H*gT6Cs~j4*e{6guoh9CguZKPgqZzJP`y8Jx znk$;gl1c%Vh9YVQry(gZCGrboaTgSzV!`6(Nz-R0eRvjyR?B7O}YBo6I^ZQxkz*QFM+8}^ej z^!=&lo>Z2xD9TfB&^Q`n2<2DR8$MZnO*jP6R2PEUOk2E2wmK>$g0ks_V{x{HS$+NI_he(Phc0>c>Px6F<5oGDOuaNmASSi^ zBPddsv$bb-7_0Ut$cph)vY7L9fgPELJ7ndo($4aRcqYZulF2oeuf-bmD?yv|)|Rub z4N={FbhY>!D2CUZiEpHnucTQ{|H%O<-O?j0W&Y? zEe4&L<{#Hor69xf>6fR)!w8M-6E?yH!tcH_eHt|TluZ8uXd!Bt3x;t`0q?`Pd#N&Z z`8dY9wD^9o0rQr@Nj>z>`5Wio5Mr&jw_@j%FUjKm7g`(Y{Q+`zc19^7JN)h9I8W6a z8Ut4QA%I}oJ!oif$INK4`ql6C$1_=l9$t2Z{=xBiYr`2JqbaqRz>AiI3KrNH?#zoM z>Ssoakp?kkE-_3p$gKrCV6g;%R>Wht_utMN8>IU~+sq-03BejXccotw^{LA{GYKsB zO#bys4nBbN<0VT;@q7$aAF~Q5TxHPB0)kIz@QM*-sh=%AO2rNi1Y`hMBptOgUhP4|8=J8j#@ zLQ;#17;VEC?Eb&EpT@cvh9obmu|}-hZ_=H_9YfsrqQHY0BBQlv!~Fc+-?(ey*U4&r z4HA5P402R0Gbj+%wi?k8-}_G!)O}FYE%0wF%T*fKL4K)$lpFxZ_)HJe!R{CBdyn!p zWSx)NUAJ=0E<3hnVK+f(#1}hd%fkfmz69IU)5c!1+dq6Exy@8qg?dPZ*$Mv-5WE|z;^q_OUBKq@{sJi};axZl4P(Nv zUmx7qoN%cJ{ufwov_;C3lF#$)=OHsnYw|P{|A#5%Kt)chdX4h!VfA3(7W#v|-l+aJ zBAVd}!K--0ptAXLcjUGsS)RXkVaWsUrwh^SEm|CCR%T{pWAr?62ovafSzyEsfbC<2 z>h5-9-i-TTl129@5QgF+VMm~ZJqaRyh2@gH>|oWAhm44 zDhv_20ar#-Mn30beDOHb3A>erk)uwl6WNPC&vh=LV|VAi=yGLLH7?$4PD8vAZk*Ic zA|7f!88?$lO-4Cq1Fczmd%Erq&n_x=h+H{Hwoi@l^+zXaZG2`tR4k;?t*}CO-hf{4_OE>o!(QwQdY7}i1nFrabRv1LYk`>y2T~M%&+i4|fwgGl+lUKD z?Ts1cU6yL>oQn7^ZojsVx=e#F3ofL!mGAcE%hO^^W#sH1iz>S8C*6{m3t6}Gb;fAG z>hXO{$$&K&xAWAIuX=ER)z-av)nZP~5If9lpBTb%V+ql8%VpQ$J9OD!QZ5c(gX%w0 zn&VPvLdohArKkwG4OCf$_f z#|sq9rn7{z)V7c+)c_i>PH{=u!Z32Y9(9JGmF7M#e!uK%LBh3|>jEHw_l|)@MK;M=O~I zAJOVmV+tIy#i?6-O_iQt0ii+&KVHlXpG@RO=rXw;Rn3%mKABB}DDE-K!0!gly(>QW z5b&+0hp?>2Bc>R3?rVIXrDnLmRn@|hIEhadS-K`?{+|4_ z))}h-T8V0MU1zi%0b4c@M&VkEQmYk`Yp`*U#gwgaz4+?PO(FtBqBPaie#B=2JuO|V z>SN0uPxoa9n^E%OscQ_)n~~C=!~or$kwMqA86{1&hk-c{agV&Urg(QmJKOX*^nBDH zSu=8P(5&HJ%ZXF0XfPV=xh{4Pwoc&DWHnIcPe8Tz+Rh5EYkLe7aMQ_PoY*#( ze9bAUHQg2k%+Bv2Sk2roMw zjk!EXZE#Yy;%)~pVt~OM%oA^aTO8AFS66g}N*$O72@@}m#O~JH*cJ}-&94t3IoQW; zzAwjewh)Hu4czn=~O+3IE32_~cb}qpk|8`(unfM*TsY_TD2`PGuF=uMgfs zHHf-~HAyJIw)h<~nq4$B{;(O3DT6VBzO>M(V_(dvAgvw``2&(3sOKjVkRR#i!SLTT znRA_L3mVy^%(IrU(4@+}h4qNA(Y~4`Izj8~V6bf=xFCn1-U!ui78;DJ3SWC26x+{u zif|!Ey9zUM@D`jRwW7mlX}`N=6zNW8;C5X`Q6M_K{~V|XuMwTDai~6@H!BgkS%lK` z)R9(K$S9cVtUA~5Op|)t>KpX%GUI&TmiN4m>eIKe+{vUe_|5hdoJS_3{&iZkwyc1I zh}>T$9j*|U)t0ck$yrcIzN*ExkQb!XAJEHDji&JymCa_j^etXJnin|U7>vgYSz-Yh zV&KzGgWJ1+@&QQ*+{V9%DCrA%dBQR+=NyBaY$w=7a+T|%^))6N3I-pF_S+d!tupM` zAKL-M8P-ghBrUY!&fgP*beV>i7ZlmCeuxb<+dI z>j((Xku5**y!--F;x5gvR{v&fpanNYr%@T_;vUd{mOJ7+;!i0NR<-NF9*{;e8$-s)S$v_ zDubK;z5RYBI@|bR9sHH*H=@Fs!AJ<66)IkTedN#&zRtGt5={ry&nR#d07|ku8!LmU zA4BJ2u)qHWP4XBSF4U33=9PW^I7{M#-THUrKuSZms@X1~l!e+dLl2P1B+_ecm|Y%Zvi|k(xfHGBpS*9ogt?#< z&ljPeU|`Q~DN`s`cfJSQQ8nEOmz|-w=>fCwR-D_GQz_Htkqyr_+kwU=N|ZDI5pUsa zzV8WurGg&@zDLt&ZFzaJiki!oB`Rq7VB_PIdyN}6<;|Ja5j32gWJwQt51xJGMOhwK z#~o0A#Y0_*wrHfF)62t1JcqcG;wg-XzN2;t%rO$;>)rB~>>NhG;5ob+gP)1CyA^r-S^SWq&`S0|Hn$w43NSmkb;S)MS{vc=VtH>xOW@xcy>| zY>?d`Fz;WAd9?x#JnS?4go=B%*oBE;Qdsj&ZrW{v!;qFt*W>w2sWi1TFv5D$V!=F4 z2*Vk zL5HZxN3SmK50CoXepffh0EP{~Gg%yD+n?8Bcg0e?v55YYf9Uaz77j|G#Uk|6a4iAj z-A?7(mjXT+zwW5=cH2w@_+cm+XAT-h8Y7I~y;E!9R@IJ6jZVhyjATv$;mYx~nD*klHs zA!gD174+rZY50&EI*qViSlm9~L!{4EUxa{1lAj(gMnLv-V6R|~pszsvRej%Vnc-e2 zmZF;FTw$#_^9B~kEH|x(_YH3cl?S^nfJK$o6SEM21^qzmE3pLRTX&LjD%2dSuTgt+DM`97 ziYv_lB&+*8GoUmhou)k2HUlSh!n2mF>{kPxtHJ9`p>BpGeev?_dmXIrM1;5dkEcoA zDsBg0wiIsKZtRhm+xMR)#I??EPR?btmp#k_`pJf>oe1!_qKoDoz7jcy=*RRCW?Q-P zCvIzB)*ZOmMua5Wbj>`(y^QwosYkcYW1BahIKW%+qZa&m=^zj|W!!T%v zgxuh?!hHwP%4fp!2545knDgEw;j8SD?bugG;S_xYC_bJ=D`6*uT-RVX;wB zsEhO&*5pu&=ZBe|_;!bQ`04jj%G)1yFLl|+O}Fom`h!kz<7?anG#Tnkdl*CNe5c(v zigiiXB3~8rpM4OBbM1VIO=Yu0sK>x5&VSr9R;{#Y7Zd=U zu^Hf1GH#>3i6?QtNC;TNGYWy)y{qjIq=XqB#bn{ z7T#H&r{V0d!HlvyHE9nMf6W9?{5MnzffjvRP+rH4bVxin7XG8 zTAIv%=%t6P`}Bk$)L{1Dq*mmgaPB<4!5nN?eCzX$PL^=8i{2m0y~UY`HNowDHjzgd z_N<$B9}Uw+>vVpfaP`ZWgBrbm@8=bZdH)PT7sU&I*Ed|nyVa2+V{^Z%fR#r1L&WCK z>5Wc@^C|mOmcALPkid$I*+=E=5cluDrk#wGxoqIcEiA0 z>|VP6ywkFOUjW9_-({eVJxNr@z~OVQjGPs}4;ea&cULM-*QC4mZx)J;59Ji1E@ZIZ9S93Y^+mYOK0`uUnI`j%YfMG#DWt0u`4NGG|6B<)GK`3l> z(ES`ao>`vfNggl9zfudv>93>yowQui=?KZ_>udo*v$zr3@42~^E`x|N9xpSVjtMMZ z*qb-kN;^=OBpn-%@{OB`{y777!r4{->cNaS`-=l|+Y8VIN}i~oV!C|o^0yW3YOmGg z!e)*evL|z#GaRxVe5_$s4}uogh~G=Q|8(syl+A!dYYA!6P{s_&j$_b8>ekXWU`*Ci zSv%Fq8jsBIbRWdRm0>Tav}AY>@G4TKQDo(!wPqc%BS0bvocHfuO7%Q66?t_}M=aka z-&f4CH|(=+3cPS?sC(rF%4lql?DfVX{y;8*Rj`I44W$ zApc7{(x`CgNszQBxk#TnE{YvW0Dg2T5<|X}0n%R)fVEL-b0D5i8G`0&KYQ!7fdh&$ zVot2CN2^F^Jm9Ddc^vyZu4h{H5lz02V=IQ2oBhCpSnWPtFEOau5z95OKz_OcuCgjwa$Fs$8|>8vG~uS{70 z43#dIS`L3xj#3*o=I@_}+I&p_2nEQ2Jv`U_hU$CNE~;XDz+i}Ef~I&8r7$-vU+h2c z5d8;j|8Uc@!*~f=u>@tBp-dA&4n{;S_$ySwl?JG=UKX92T>`yr44!oMpQPgRh`tc4 zkB4w0VxHS-z4U|+n|`~ceU<6;z+p^D>@iKj`&P-=0F8auj{X-Rr$viEvjRG$=_m?l z{>zx(k-qUUMldIa0BwmAO|Fj98t1%^dLdYD!5DfVR#sNEfpWZe2s%p%TSF~h3cGmL zA1lGq_1O#yZV^GyhMiMZus*TpdTqh#XPk=I6`7r#zTJ8!&kI)mmUK9px*!p4w(nEC z0Y;eV-eX9ebb=SoUrrp}Uj!e$@~K)2qgGOuz0|#99K0jOnPQbp%cxMTc~aP*)MU!> zsdi&NCs=(gh9QSmEi3zvAAlgxQK5H~5j0=N9VL|^8Dx-s@j#?A_^%$F$mP7b-Y+CIH9|0BM4QBjLvN8#N*@H1%hLB*c8)3Q-q#gSR7pYA;2;5zULD-y)!ezgay!?DE{)l$)YQqP&>#zqAZf z>6Mb#q5GhntNK9W`##$`ZqdI*D6a@A*MN`8K#554mkI%~;j({^tD^MM4^FntvjyI{ zHl6A*&%Wfb3AA#0db@zMbc$=-twJTL7pL=22h;)V*rbU>eMdNQ19TKomhS|B`2QA9 ze<<|xZGLI_EYRU{3iuOsY`Spw;M1K-GMmr|5kBmZpG9w!uzg2CS|4K34o8^u+LgoYVW& zy*^HURt{FyXvjJOz$uL6V&*I|@T-=o3zSBtke}I}+F@Q>n8u&kA=l{XKQn#`7 z-cI~K?cHfm6ITESa6pQLOA_uV6-x*v3K}kff+jSefe($kdYIn!7ET+B7r&4 zk#qc7_p8qG^50Ll3nWsze!~>YnfaX3L5WpsRF>XlB+@~8w?6@kmCEKudcddUhM&cf z4XiRaQ8XjEh2!}*DeY0Xho9p5^*LMqaf+fN_ze|V0KfG~@6Gr)QcCG1la z;&?6tjt4?UJKTP`_V!fWYNxLcv{r9y@^IGye4;ca^qabkMpCBUBmHn8~S*dO0; zc}3Q11^(z%b>;2P{ZWAd_HnS`n1OHqq{sL`Kqu5}How_Szc)>Ec)53jQoftHF>(tM zI9fRs?hAb$W_`ce({Kod5Z>bDIrr^GBNU*L>i(5N4zWfD7(~aaoll5Q=u^0Zv_;9Y zDjcfhJ8&6lT%)s_c9)}QHcDW#fvB7`_R)IRvQglc;YaArX5;#~G4oS34eILZRDh1w zoOwt=5M8F?`?dmUk{ieP&6y7WaSO58ed2TNN99mLIvCvix!CcsPsL@FUm zOOpb;(RA0keew8~=%t93Bo#%3V`%6PgNz3z{_>Wovu6n$jG>`jCNBS;O;*>W9ml7j z()qb(QvV~Y0T_-}EZ)^Ay*^ZoiBBm4YK!a8?w9HiGwDy^vuYAGy-oB&v@~*b5$*W^ z?vwCBaZRuu+OsbIM(yxv{5ZI!(MzGA{D7R*xC!@uifO=pV-)it>X1gw-~)Fxy+ll0 z)Nr|j#z`(w_Y(MffBdIxDDO3)dIR*$B<>@Yg(gXO)nVy$+QOODipJTp>&JghlCe$b zG~q?scT|$+aW+&f1Tq#{mIxOt%QfXiUcx%%O*H8=XvUXURmrC1f)_#S4m-0?!vmBL zPTjBBsT86BvkLee8uc)C>FuyM8DJ!D)0kVk zfKspb+;stPQIIlPaw~l-EJBJqa8PPTcNXCyMoDche+s3Z&2*Zr`YW3g@hN)g7OvOZeEnxgRV<;``~ zC5vV!q>~9^pZ`m>!4Hw`8@W}YD-YiQgi?9E>=i&-s1uc0S&;jMs5Qchs#!Goy);@uX};s4p&cymNjIOYYmCFGgCYpT#$UZ~TFB@u82 zcn&{1Y%iD}(X#i1J9W9tI`5!EXvjM+Pu5+*)s~ln0&7ebh(_k;no**=e>;SKraSY2=hE3W$A^Id$vzZfS(@h}rH yzr>SE#Mw|BDI#KA94X>R5gY+=kp91dbaBVy7X2q5yI}qj;6t*d5bJHcV*dtdkp5`^ diff --git a/docs/img/duration.png b/docs/img/duration.png index 18d723c005a4860a403bae2088d0de70b2596074..b59fc092573a679010f5ff2bd66873a4a150ca8d 100644 GIT binary patch literal 250846 zcmb5WWmM$9vM!9f4uf0c?(Xg`gS)%CyE_aFgFAz}ySuwHxI^Q9+2@@1-ute7Km2#E zbSIspQe9Qa^Q$DaI#N+y0uc@e4g>@QQA$!&83Y7+9|Qzq3I^({=JtDz90&*kxuuAR zqLhdTK+(zG%+kgb1Vl3OXBxC>@+d~Y5rdSZq(z`&VaOK5$q%ns)jvV82@v0;5nzP| z<7lZ{FjPfrh|7ta8bRtxBRXk=It6G;DbI|0!^Sd51FEtgw>%#+pFbJgFE?5r%|5tT zKuU+|#Pab}5d(A#yEy4-6S#=TeoA@4ND6|Nf?x`oJCo)ItgJ|am_EHLc)H^Q>K@T~ z(yMb`Kg&yBwOW#bx`(MK~hR>nmExw&^2`?z~ssA3hKVs1t-v;2-^jokMGvECJ2ZSWM%Lm z)|H`~l;2&wnSAt{zeFgo^Eii~w~mNWX#h|UYRo@!4xYA?GDz^V`@geO>*EnnjBBbj zpIf3b7Yp;#$i~M7R)9X21<*97-Jo@`@;Z)ufJ=OXu!h0{$yKH2;RLJWH$gL$ciB(a z;t)X@3qr7tCx-!Q!ba6ns`Jv>jRA{?5W;+;=9)8OC{7)frKc zR2ztN`wo+oPgE}0MKc-~k5%WXH-a2R(1a?pXTsF~i3g7TcZM?*(JYk)dkX&cFTBRA z2wwWM0*rdhV6npraK!~+enW~z9KN>?3=;Y7X(+~P_*d~wqm@#C(Qc$mEd`CoB);+5 zsn5g;=xMjs_t~MA3OE=@DZnm-id&UFNPt-$0on|j2!MeACr{{0MJf)Y1z23! zo`-gS=gSK~8~~tQ2W%#od5G3^NWkRWx(x;mEeygJYtMU zm;_TKI153H1a=nzmHHA_lX{FxQQug3poe~gXH;BK;H$W5NFWeFq4h`RiCXI*KMr=@ zNIu<#Pr^R-@P+Tam??M&ACvNu>X5jg+R34RkMxhogv}SkjA3%2Y`?pz@2AUESLa1} z4E)Hm&kQ^R<=KFo`X2Dw+p3%Q@TyTng6&^^kibv}j!kBr{64BCAf9Vf@R$3qnA)E! z>%0gW-$E@R5E4v5OQdf-{F~`pC>?Cv~PdW5)J#Y z&X4w@Ugpen`Q7>+!;ggAa=&{v~_Oe>d8ycHYw^BrHKfgE3Toyh9)Rnvp%odcuv%kJ=qPUs$~aW)T15GCmkOuz0j*3 z-0S2g^f%D0aDTEHm=_4`Ak09qUxFi~vJ@#O)G+HX;V>mT>{|5~Sa-MP>4N-X4@A zGzT;XiUtq|ItSE@C~i<|b3}jaZ!a* z7g59hj7p(Saihln(N=M%Y8g-=&Q9T-Xgs0=lm$9D(Y2=gL~h6t~ONj|NpOep~FyIJY3ToCl%MDx>N?<=wIgl|O2himequ<)0N+RXDlaVvUN8!uJ8E zWQ%%ib-zQk5^%I_tF#)|3>$H+-cMazCGN5Ab?%E!c^0St2+RX4H7r6b9N41RnyfNs zJVb|&i|+v3;av%OG6G^(OQcp>R&}iPtmCZi)=kzAmX;Q+7G#!_Gi6KM`HO=+SQrQx zB;mE0wK?d*SCo@VTLqj2(h?VhuG)l)&9m|IIYr$+KJ)#$1M`G_1&$bG^}+`0 ziG<~K8>a4B^%sQkiMx>KlLXFM6&Q<8r)y1sm?70Cc_yuzSJ>m((XwqBZP@V|7+EG+ zD4WzR?`KmDF7&hZyEIy6SmvZBYNo8()O*#hADTELJ9j;6-Lf2B(b3W+(P3z^YVp=D z)S75wXf0`PHRfwRYaVDWYUnnSYk6y!Yundc)(F|^*gml@vZvW7X>(`?UO=r+&4$co z7lamAi*MHytU<1zuPm^fUPs^xsS*BbTtFo=q-`Ln3u=C#V+L+v! z-&k(nzMi>@xYsy!TuR=-KIWRPAG|F{%TVuH_N(shyy&R!boUa#wdP9YXmUFzAw(l= zn#RbX*DtpJePy;yFJLUtq#wI2;;-Ug^~wHO0=fjM3EB>(2gMB$1&$2ag$#uIjkbUj z0~rpP2GtyH5keZK8oGhZ20MvRiF^U?jF^GGgd-0V52KIm7i`+|x8_vCcW+~O65E|Q z2nUr=3H^p~j$9dU6)#jgHd+ham1B)p3u_Y#+(?6|+~{&pYv^$BgDIPn-_7U#iNZ0? zv3_rAV1BP;=qkoP4dqu2@2-6Qk9^izIw>|wWo6yD*ZrM+;C>aXG5JKg1CbbmSi)S2 zw6vV`X6l*%lgZ!Tk(OlYW4T@O4DDo4-IBqExnP)1Fp zRJzD{P4rW=r2G^4DX-~D>7N++EMEe#Pgz1(=%*UeTTKq<`)2yK&35ZK>(|V1r}Y?L zdou&Lv=51$=Z~5uS+dl5tG)eJ-tgfm;p;HpH1`!%_3qpZfqjQY!GE6zFBiZ6SWo?EQMJ*M(WKE`|ADQ!+Aw9a8ReQ& z{`6zH47~iYYFWSAdeNNF>Nu~B@OQXSA@><~< zOSk8nTC<$J_G%Nudgyw)+rR~<=8>Lyvx@%_(5pdDYrEcht}1S`w>{%wiuW{(`i?�=P*mWfK|qt+tEM>nz}-qz^Whh--Xrryb^^> zSWf5jis{uwTg7&5?|!mJ89DrH{K+h1o_M{7cAq!t z>!0)Ln(7QX(OpZPd^azXxXE@doBLZnZZtjQ?^53x!`-GIM0l{BLG37?KRYq= zwQ+!TkkoVr0l^^q*Bewyne-Y21T@D|Rl`L?PL|u)-j?3b#NNo1-ow`6t2YP;uLt*6 z(bm+(5a3~JW9Q86!AJaGJ-ENh|Egvn2K-kS7i&IZ4LL=Ch`p03fQ_D&o{^Xz4gdh~ zI+>VpD~pQ%5Bb+OK4J?O7YA+z26uONdUqCjdna=SCN3^621aHEW@fst9(2y0b}oh< zbau`p|2@e6H;$;Ov$2z!zm?Aq*Zge-{>vVCmZA!(Upd&n zjPsS5e|=E?cm7od{S04f6+{C85d@JE6;ky8JFM!S5>u!6@S})m8Q5zx(B;Rxl1ClG5UT^KW=!3eHkY#nsI?&BHjCH{pu+5 zvE#}++aBV4vwfP3mQ)bzpQq&%K@iAi>#<~963}mef0{%9f_mO*1V9^vf6j^2Ar@9K zp?gTMmFu4~`lbl>eSZ~6G63SACIB>yV!||N=>{om0?0p2iV5bp|5*5MfIvKepb$TF zu2b@VAikhbw(UO`BLKoF;L9+00b_DW|5%I!5J=Cae`>>j8DRwUOTEq@)ErX(!Ib$j z2-H8d#+Sff26>x?1A@}0wSa*+fl&QNyof)OgF42pl?KPhR>9j>{v*B;FdIHWyu8a| z*9rovT*xpT#S~T9H$rmU(heSlq*M6*a?O*miT_-=0b~gzK|Ti3QH9|lf~x92HmCD< z_bOqoI-_XbPeVQHebXS8Ehzu!u1g>1BPKOCz%`Zs6%bkWCONYI#}huVoaA3j{>0K0 zj)j?i!@!_dLg^p@(@lc^?T*r5zc~2gLk(Z2+k)T`0P`p@2W}b+AtR`fLMnQY7{bfs zRBS=Tg{gh~7rU&d8nQB?R}!nZ>*l$^4}Gl4zPtLIK;B@-FAJ11F%pld&NM#;Eg zj6`s@)6tL9SVr`-9_=kxGQ;yp`I^ZnHnPq={J+xD=#em~I@U`vKrk3!_OsdEo5+7K z3zs7V<6wY?D?5o>E`zcsoo-bFP|(tLFt?|4)%iXsAC_iMG3VFW+ox>Zlu{;ytha)c z2U321`?NH z?g%zFVD3goj`*)SMl_EPEjjDqf8pj}fK)%RM*dNg?y{Yc@r$P22`}mJE*Byjms-Q; zt3`u(LIKKkJ(UW=KAUD0(*E*tJW>})Jo^tIA^Mo(V=joqg%-54* zzZ6hCY$f&__9a;Td=Q=?l zTLRgW!rhPJAERcA8-$Zi!m>zXkY{AS_x=f2f05T;Y}jy0Ld2^lMun#r!KXG6=ixq~ zgw1&}bmF%pc$7URDwKR*IK1`ve&;ST*?^RZi=-z&S`787qF;qicp*f%G@x&`1iGx6 zuWxr4Bc*}BcMu($EC~!$k8gjIyOScE7vlNn1W|R1=0C?>r?4JZ)cyMzaKNv2FmSTZ zQ4-+91Yyxs>)tnl@1#ngC*O>mMxg;^yrQ1_@o755sD^=c6r*GrNxI~Z9d>)NWs+2! z9m^4j3CW5p^Fec_dRxlOEo%B)*r?G|^sq6J=<3yeNd)MXH~l}3IOP70ClgQe7{1icFuo^(bChS1R^9vl6zyjAF7K16}gM_+?Ov$@&Ay5|c}5 z_j5SFI>u$bD}-KW;SvDLn{D1Jvx1lNEHOS}bXRuz0bX-1db} zH6djbaQ)^vkxU9TwZ7X5fO@5r0DQIq(XW|~4s%>KAw9*6#GZ2(OiuXxD;D&d<%wT- z$(W`QQybt%zoBHx2os~Wek?a8-Q9p_evC^;Mj!-GbjrXLAgvt)J3`?6V8+peTU&zk zS=PCYuw&@I@|O=odTB2#>XrT*Bm!7`D8)eUy|zPkks9IXCnNJCBtMO*z&##lZm|QDBmywH3 z?()sALH2Z8A_#DK$>}Cc?ZY}Sb!hGe@z-&-nAH0e<{mdhyx>O#DgyCL4eOksTaYJC zDv#g7CM&Q{P`CiiBk$SrqfOYPha5Wl^e|;svYNZqe7UA zbLY^ubm-l-Xpn8j{nV$^>A<8dU5x|7BPh0a3NZo`kvI)c%{V~TBuxLNsQ(gy7bT_Rz8!caEP-;~M_E;)Kt z_8-d-=l84(`+Pq16xn?b6xCrVkvF(H9)SPewsQ64%*tgX7jRNf|2Ja zJ9gw;ACKimx=i~w=YQvrIM{oZ3%o&!b z8*!ny;1bLrlV@l1;Ya1`l7HJ3AcpL;tG{j*!pBx@+w;CfRdv8D2C0Z!G-4U6n%*7s zb?)H2_4KtZd-&&f|A&%2_OeKmVewT{byfJ8J-ObJgUUUA?yW;wb>;A>`*U~y$AWQv zIbYpTzw_FYk|rWsZ%?M7Us+9s_xWp)R^Z&JCzGRgEChR5S^{Ng77MmlOziEMxAWc^ zYheyLuEw{Lv%IZju&$ja*Y-}7_~=D@0?l(8Pp$RJbn3Ys3we#I3|f-FvY`|)>?X6| z4$$y3?b;0#p{;;9c-3lyH~*`x&bgP!cv4v54jMMZWE5q7z8}OSB4mf>u$s4C$XwUs zMx%u2fI(rMIc!_K*92BpTGtSD`)7C!3Q*sovrHMYLD$6iEdu(;-eXy17Xiw)Zifh_ zqnVNJEaG14G#M+;h)Qn0Xe26}tg9{c?8J^lOd0`5eBbfj?XfzNmeP)~l1=WBo}v4R z1#^nY6os@(pjO}Gw7-iL8AF4Oky*BsY!X|G1+n4#&D6OV^Wle{c*3^dd5h}XvCKB^ z&H@5ho(1yl%LY+vzpz{xIGY(~PG*dEhD;yPm}U6gb6-*2HNslp7N<{&pJ~)6E*)+< zS8|h0TAI4eb?tZZJn&0RM)7R>k^sDObCnSK!vF zN#>_&$tF7GT0xFknIWjc1iXc$e}_7(K7~AOfe4ZDL*Q|6dFmR#-L>f)AE73wU7~^4 zzyrAEqG<1C*dy}06*LuPQ{WIC&|hXbLR5^$Tz~O#3ex!CqDhV{r{wd=YZJo~YA>U% zmI=6#NyZGGDA;ASEUNXBnx2p-bf^Tk+4SwhuV2lRCR!3t-Bq#qWNfTfGcrFZr0tF# zrGMT6yNiUiGpcDCHiVKX8y^757=QyrN^YRgWV(qKJ`Mx<7DmH!cd9!$84jahiy&T4M}AM|UO1`6hOQ>S!BzQv znM_I9Pf{>>&{M2dw78g04%;RhWVOU6W$0=H&h7UYrWgm04YwRlp4%ssCFC&1mn-^< zLh)K+vu*CKP7L|<>M6bB&ldiWIj>$&pixG#GP^&&Bq-=1Ua{>KwZHyr`MQDNG^nxa zPaP_t*K4W9PiGh0Vddr(wuS>H_x;R_()RgDJpUm}_C3H5S-_;TI+4%Om2fi_7 z$kpgfrV-IM0S@~=wEPI+S+r4cy$J0Ejr&1pJGregu~0jm7OaPpr)bc8WU3zlrXQ-^ zhh)Y!TW7~tIu7hpD~GKtK>2yExkhpH$OmQuu0#jDFj_5g5jP{+jWng`>e%fO%NB8O z5%X8=$0>36&W*f@wOWbnU)_J5qBm1+?K1&ay-EP$>O1?J-2GrkMJwhC2|y`Se!ns{ z0~iLrCyBGv%#8@MKLFya|FzKR66ECDS>M-TSugX+2u%*05I^i$rig|2@oHVY3#_N# z6SovXUw}Sb>RcQr-lrQOIyMt#PIoW(XH4OMaIQp2#!rnG@%k6w1O5fWpB6&(io{n6 z*=`drT@AZZKu*}yB_Vv;IJ=8HfApnFrRH^>8geO`^a8>s*cr3SR1y#1kE3% z@2OIhB>>iDJT9qk4YgeGzq*whD-bLWF)m%zzS-$$!5iX)_NvH>WFBAlV{xID(!D%T zPA^vxPcM`wUW}H44w%EIj!9@g>ZME2G;!l_RB(&Ijip(0jJKDwo3#`Ao4u@)v*U3S zyUVA0Pgdn%ySpVgJ=1IHezcf|yU_oTHR|u5h;<&j1ZVDWma){RsEJwY5Yf|Z(-L>L zC2_spd7iK4IZ%jxXF0wJ@4e`XIrz2dc?$lLMzcKXjoW0}jCHgr3L&g(SqBW%`xc2w zPFH@-yx{)QM|-up6~0x9atVE`pqgOMc52R*hYX{xgmdukDOfaFizA^kTGHSkjm3^Bjo)MA>GMBY26yZGK#z$c&70)juntzb_} zD(l-|2PVsu2Xbr{b+m#I=~BK|WYh$Ta!&OR&1}m}S$aW>M*>1>P(I6VLmVBDa+@L8 z%Y;ly@*Zbk-1FrwVZWZF(jNHPA`iSDWV*4VC$K!HiJ$j7r)+)bo3Z0UHp!=g^jtjg zdvo5|=+z|sdN@;YpiO=(>{dOG&xQS< zz@k+d2&n(iSFg#D4UfF7>pg-GNX!>=1U? z`Sl@apR=4-ZH$eud7M**+grXRYTXrjH-k$uQf?LL2O#EDePM4?k2+$AQM|V#cWRgl z=%N=&VDGPy{Vw_}%#`QkFxOo7b`f4A)CF?#d)hR0wo261;yy{CPXK*?e!6H}`}0lh zaN{`_B2yYe9SaKBc&pnWWdd4}_>JD6$OTqjfbtL1@kne1GU|GJu4}a?5cH33T}$hX zpGLdX%VL=o_jtiIBlYL>=ZNNJRekR4c;34mXq|;)b4@tB8j?5tOg@<`XYA7Rtz<^o z&%6xiG3exyaf-)nI^;$hs|S|rbwl9u`7`k|P$sRd(G(0P-^=s$E@62F4`%q)W*tMB z)?Uw8y=Jf3nAv2N0Vb^~ZwcQ}o)y9y%ue$3>-bt%*{H_xE{l6;&GRtB@~D9(E3JEE>N-H8iWG4gQmkN&@6Ci}&ez_p~sbVw(}=ltcYsw;~2 zinReqn-aTTAE4kOToMEC0^U__dbG*dNB>IK(c03KPWM@3<*+4>dn5G%_d=KK*|aCp zukSH_rYD_zqznU(z1Hkw-44Mp;`shsRq6Y~4c&D5G&KG^evyDr>EPdUZK9~y-Nw~E zwFK`!fpgA_H7{P55f27Yt0n3!B#(I4vrVCU^M}RA4s8`5YBa^`tg1EW`|iUzshF5m ztTE`}dXJOW*Q@2imatIa1xMcs2TQ5!u4c+B_GE1b>Xy0ZxLHz7ckE<_Tx3TU!W0?S zA{z7-5{A{^E#9pu$u_I1&K8%>>FtK6OUhGzV*Xy5=4>=+xQO$RTFLnpTGDR$YVk>s z9eb_F-QY|kHE_2iAoN`g*|XC9g2x5BGrPg=Yg@W_GNxJWo^|P(^)n(a1zDRdiS%w0 z+>EX-nEt_Z4RvYWlub5;Kjql{V*dV?osN2h;beW#qknp(QNL^GeY%TNHIzs>-ifTp zg}{9vag#z``ju&ZCjNvat;ED?nLw{=t((5ex6*g3&pCM-Q)Qmw)sb)YeOo-|Lx&>edoLBlS@=!uC{#8PBt8QnA37=iY z&X!T78QgJvzE?un-M;ZI^eVz_#yTQ{la3MDuYbRJRGF(YhBbg}%@3k5>tro-iSEzK zIV|`#lOK8K#13cSU2Ik5{gyRLu7sMGGpGdVH%0>7K*vnFkLp*yXNI z2>|W#Iq2v@H2v!zfn^7izsj*og3*RP>W|*5MCze=$jgnvuyC)v1z;nc#O__uUmcT< z)oHssHb-q&axeC=7Y-Ua!!x5zUhIYqR0&6|G_OqM4Jzj2%Cm+!&6ei|4=$+>2E8k^ z-C2>5WC+>*kU0LJ{`GDb{BaLA)W2-pAr`$fed}7DabfB0V=_mp=|KxWY*;@O`+4iJ zY6L7{pPUG^8E#~OS$NyZ$%aBM;1vPx4rV3=>7=e#R08i4RoQ` zJ-tt%Hl-$+@c9j~2gWM0?1kTVrh7Xs&k;j#d%Ol3Gi%5eY@%%kc4vO8@ zfO|C%t$6h4ddR21)Y!A!n5KTAe>*;@o9r^reqAva8xL=?NnWfVEKradSXr^k7Q znv{g0{B9o{ zcC83^ZDr3RUXRJnk{iLiSM@hOp+5PJW;jql^%^gE1jaPii*C*-54H}aTD|Bf zV#XQI4tZB9eLxPZ80-7;Mcti6$A)H8i;XZu65Wx(fHN+}_$&+`nwhRsn7^H(2tk!5 zS^u{O);>MmANV4Gs~|5aVM6{h+A3>>^>XOwE#j8uEYa<{eO~HS`FOZw;?i+ZEVfBu zU8}Cw?4oF?MdDBtC6W~#=z7#fEyP22BEe$j`F2(KlKDv80oZjYjqAw zg}59M4+@?28>W^iRht2}B@jzzq`#&Iij>6HzuZu$hHob7PJ2+Fot-u6jl6Gu3AnGz zYkD9Bb-zsg-b!P^>N#6BVTaF=D1Nklk|p+aLrW|SVm@m%HX=ym-R$Rw>H7WULeKz;F)|?7#)PdAfuV zioa1ZzablmmuQ&7iBM~UF?tSW!8E7k$3b;DGoTe7?48;p0-CFo4T_Fq;g!dQ`a?2( z*h|WliMd7`ytifCRCu;OjvrsNA8g}m{qnRY;_bvCw3uZ|ZAH0@7Iq(FO|?7Jdl}w- z=i@d%DWVCU9bX(9MVqNnT|S0^j(RsYY`7{|p={df*aKkr&tVAjEh!;i??p&jl3?gp zAq5)}UATWk0WhWs^0 z&0AU4aEIjtObe<)%U#qU{uaTTbMZ<$bx6gIwic`|Pq*+^c(H=tqS@@(zFMxP@7zVG zJg9t@F+E0_#` z?O}nfRjC)YQe(dYXV7C=w{N^XP#M5dqj+cCFZ1)d_FJXMwH&EJ+d5IqvX%oDJW#7HW!;0m%f=i!^hEHj+z}IDV@NusAm5a3Dm#X?W~_YMp3x63ump!()Jf^fI?V$X05dt0Zt zHdkc+#t9 zA{6+DfhcE(g#GZEI%Rxh4Tu7!?Im#R$I z+DqoyBZUj&Xgc;&5XM_@p5OC#M9nwf2pe=_MW%X5h`F7jWDN3bH>7K=_M_v9J(K0z zx85<-6N&%a;`YN)3uDN&pW!}^NWo|~%=-h}yknKy4Pi*M$!myh&!0i;-40*kH(Fr?(x z&)o2xKybRhMLIV8vQk4_Pf(i^JQ!ipyQ;O?Mycu+kl!496RRb-p6|-*0HjR^?Z@zy zT2R!ND%Ec+ZCd9+0;flhzUOvMMJ-=$?-%B3I?D8?@pp#5T4R<?L4)m#Uw`W^s;1KCWIm^I8C<=MIGqKAJyosdAptav2*!{PJFr2VlRX4 z@_scVyiEHh-^7kP!W`L;`&!rYf55&rBFn67UJK0mYaZn$o)@IfcS?V$V()3%yAQ;tIT?5W4__^bbFpdi=`#4k%jerA8=7Wd?xAVr=+*l;XmDEN$2ZA!!i#@w_qN(x zhXaoN%*e3@j)Hg(y5)k5A|v632?DN%X_K~pEYb%Ww1>!Di$QjjGo z8luXQDuo|YTE@l4p2bP_Ji|Yp_t9t*hH23}>OAq7@f_btWI{1wU3HM>SUEZQWAMm1 z0*MhKs4dF6WHuH4*vbmLlCY*gN+$(9&pDeQgdjg}@JkK+u>8VknyopnPg4e3zB zKfv^_f(O=21GM~&wzK$W0CWof&+&AhC9R_M-2-O=<>QP5j=WHVm?aR^RMU8!@p+GtQ+8$@mS*CNlDD zlib@Dx1Jw25lLB}#JJjcH^)VSqD9O2%ROWE;#%7J9rSDsp0?k%F8S}i49Q*;^*Tv$ zN?OFnLc@joUCO?NEE$4#&#H-GUqghBXlu;n1WnBK-<9sh$6&qw-4fT`o2M$%NB!iD zMc#{1*lN@6KN;xkg`QNJre(8e=8@W~_r5VUW%#*2#x&Viqk8L}Hl4#Si~JZHWvQWW z6zdPl>aQ1?d|N|q74tagL|Tsjs5*efG$Zu0ZJ({#wv|x$`QcIPY-dkMTy)+KZlLRN zZ|z%V>h^VJwtB_1QuEr0&;R}vEW%rvm}nTmc%XuEFQ{+P_y#rYaWP&V*u90^u%gNw zAx1j>^2mxYqD7%qeKfO$yF;e=s@;zf=Hzd^dJJ4nzPOzvu&WmM(4%O&SmVEf$Y${u zD=dVI;8mJ2HfVy|0`WDcW#z#A!fj84xMMbhYNliiu!u%q+E75XrS_IsRI0?yTg9{< zlgHt#`HlyIoyF&4&%$%Fdg!&yzdO!Vzj`ua&m_MODd}m)lqRsdygWl_u5_n;4|Oi^ z0(H+|obNCn`iuWyE&BWYEb1Qn)iF*9Yq;vb4%xL`@J}gBIw+F>Ni6QrUmMSEG4K;a zXe4}lWhK9{A-Zqsm36H5oEE-yX7xUGX34r6M0=v7#xxU*NI;`wtdS$SBX8=eot z5Zk&{)W~{nX^FS70>o7X#51>&AyC1>y_NN%yktPwS%pt@+|z<_I`%ifS)nJA?lfPe z!F)F3aIXy(xYlVP`-6I)h6K5(Rl6@q_gh5oqRSh6Aan7j3n-#cP7qWMK?oiY<}njq zxihSC8VwiVJ>)}k5YA8Qxd{4w-sulc{Iq_wT0-}ur^M2Cd&)o_WXU!yVT%s$``Su@!$k^-iC!fa)(s~xF11?7KG3m71 zUfj=Ji!`G(?cP|Z^M?rx1laWJdOl+z3Cpf*&F|m>x(H0w^JW50)eUs&(%pUB`?PVN z@SY%QET5N$82DLw*ZP2o50DVbTCwO+sL3+lLTHC(XU6)Sju ztil4f?sL^5kPGW3Rz#&{pa~hhiu7Eu4V=fkgk#rr7`H`0KBFlHradA!f6JJ| z$6&QtU25K;{-*I|4Y}nWgo~~x=+xwj7lRC+hoP!A%2-Ma>uSzRn7{5>hRuFRJYJ{- zgJr_fK*&ha(Od>%1!lMi zXsB25dNC&p8rp9>Kv~5Lly9FFy^jsqe9)4jr~XM&Xy`odVaeC)cHni{n7 z!A$X~ejXNcHF92EZiHq-whzNj|En?u=Ve6CRApj;D`HH`DG!HW(pyKcJCi zwk_d{n1or7FHVsl3x}Dk_NsxW!26+{z0X}7fpBk0>xIRguj?bCwF+^*h*wlgv7JO5 z*jcMpU-#eag%?)v2u(IH<-IM0Z}2Ay4XS|-&uIBoAqDo?VEu_l24Y@3sfr=*`YsCY zk)gxS2n`1%W^`d`t_idvCV?9s^cpe84DaT2Y>!vS8OzC+KgZIWs-KgF`lPjF&=)2Z zj=O*UyeQ8M>Y#s28vgyFB6=4gB8q%v{R%HyYEg|!ZE&2Y*3raJ%iu6|aWXzic>OJZ z(AFokDd22Myq}T9u0TL@Q=(I0Rxe}h8qGu6KWN70Wsc2!o5&MSTw2RTdo8@o@po() z5JFG!s#0%tp}hWTeuU+c(E=3MUwomAYPd`9&i8KpbN$!U>x$j|wz^Z6#XA0aL9l1G z4zzK9_BCzIVDa+os{}*G2>Lkwi96qYmEsT#@Bycu;ibu!Q(rYlIIC-=w24Xbg55H1 zlTYCP5i7hy3>vdNdZGd($tP#0-&^8fRev+!9H+YxHLBk$b0d32vdYgfLU>#(ir_z? z+}h)M*j%Iq+x4H^d+h5apQsIcEicxwz6tg|muJ<@k25bHF@-NAbA|FH&`QE7F9$Dv ze64cEdyl_O9WTMMRt|5(j!1f5aN*?dqoQLz38M8|%d+F#d&k*ZR&N);XL0jd6VR<~ z=mMTl;Jm2oni8~doFb6K;z;Zf4kLO$irc`(CwTpjdPs%)%ut#dmm$H zzNYT>dQ;z3Upq#Vp1rG#0K6#U02kX!AC}?6odv|2*T?QcAJ^Td;lla7xv8jFke!nK45vG z#N1riTW+fz3B}wk?47P#TaY>ZQyXk5>pLdB)#NtyKJYD3&TtAX4-48C^+mO4!h>A?FWbl@?JOGh>k7AP#RzNUKvEk7g^+3j!F2Y+Tm z(`8OJV*WFDbJ?q{lOBD1ECia_d*riGm&LQv7CwtEd^cA8?pdPuz zaa>zEA^-)x?5*!L1`|Rp+9Z1XYnA?yrzpTy6FS4f9&LDo@2?D79WNXB@;293q>%=;vqHJnIc2n6J+FldGhu1%UYY4RI8J%c zR`X|NWrDs(h4SUmbTo=xMmu-4`690JT z=PP!*r;*)H8s|1Ggyi@d^6qj;W}u2UsH7gvflvs1wEz*b>hSoKw>#7bM)3 z^WILa}jTNoG8yNcW(Iu?nBp@Zc55NA_g9URf@t9*J!ZKhv3(rydyfRw#>Cv{3H zazYzXdy=kn*!fcX)-5eZ=MrmBKz#^T#oF4i80Et2(PBQOWMV^#?4g8tg44zQ-$Dv!^^-{%Otg3Rs<|7Iy7L)yr2xdgOQYcwg4G;JtNh{<+J#z-m{op8M1Vm%fva?pc%E7X7`+G*fw+u6NhNnrhhX zkC1SpM22X`=9#TAUU>fCJA&W&g0BttDDhWL5yNep!zgf7UpKVO@at#6e&n2yFJBxK0IPVcB<3=J=Pb*$=;k>#_d9yR*OE3dMkufK^n-tm=N zec0{TUNwjI{<>aVpI-lCcg}T_{O_TMa4@8L32LqV2b%l~?Jf zc{(6%!?|41reoP6*tktsxtB~)7byG7CKK}v4_Pa5M=Lh~e3Xx3ITFiAM=LsRejglT z5zie#^YF)`ub`#)No}I6=>0T3temO+5NNss0Y(rDrMi~L;$rR?r`W#XsO*my=G}!+OI3J(QC$3suBw3m zIs>7*9hojlo1(L z4<%cFhf^&uKeJ45OVnZ_I$^cJCJukdXCa5GjK8M84Ax_-*$?onRiN`}qq(stNPat0 z#>fdL%)OG+FH0vZt1PG%xcXuFM~XYtXZlUS1J{!Cprc_C(^64m$vzBn@jmD@`}&N? zI5`o-1uv1NLgk6*_%wB-=s9spBz;2fzv=rGWYV?E6q3@vNJ)t5Y0(4ktiL%<&wr~i zt|ldgB7xu&o~0o2rwPGG3kzKg3-c*D9<0%yh!%rZhcNOz`~9uMtF@5!3G&=gRud-2 zG_l!jJX%5I*ATm6TtCap!gF-1Og_K~Gfce_109|Kk6o=RJfUPTt;Vk8|4IVhmo3c;n3yF2INVIC$8e(=S` zj~7?-SK3(~?8J^5yCPvSpz(Wngy*#cL9{4#ydGV!PYl&OgPakXVYT0B`C~SURV9c^ zy;$O_0X_Qd_x?9&%~}SMiZ~SQ-C;RB#WIdssubCnT@YPXgE)B^6^WRu;w|09UG5AN zS>gLUc!+dOiWg1J*cEyDsC4TY9ESUp*{|#Q^L%&7XGmRMpVr40COAJ*8apnqHPD0h zw+q21&I7RIh6Qdp0)^|E9r%*m*7Z&mzWnj`KBw>9fu{5kH3%lZWK2oCO+B*4IEl%K zzdnrT@cGGLwrTqIxXe&P?2_Hb2p~JJsLvK$91Ai1p#s~CVNdk#kaI_Y6M;48k`Y=s z{$~&MjLmjBv!xLwR576YS+Dl#Y&XxVexMlT_Y*T0R&J~9f~FQF$i{&G__VJ?S0ZxZ zy87U7gtdya_p*b-j?_jLlcnfw{kM7z=51c!%iL`BzSQ)Kn&q75!C}_k1T0`K+a2wZ zFmOH(72~$WF2EdB)zkHN;Gp`OV6FgSK!6m4_$uynH4nWAn-HwUEG)<5BSl+7=zS!c zLC1K@)mtXwR5?$taZ!PlYv4SCYStzvWo5za&n`t*Q|5CElAQu6B*VJ17%{`QT&mYF zebAe%=`O=8&?luDyyex#P}VwYf6e#Q+3S@Dz~@skWa-sj>Eb2-*yNe zVcvLUH$g5}6HzN{Yi#LEyTf0_h5%h`D%6=?w+8qGi6a2dC-il=U(DV>jleeI5gQdQ zPxAjm-h-y(M~MU8YITTnsm@S&=z;-Wt8q-*=`Y*j9q9STBL#T^Xq!RogXC`9NRf^< z>g%S8d-rG}lxgL=8Lt66JCKd^$vle1XBPvNNU2EC**@}QK-|l6bSilk1&JF{KEo~J zPQzJZP?&Q0xL0dVNgW9n+YI@{LZGqXYbvvel0^Ir4W-R0iv9dA5lCq>6}xNjuX5sKVw98_} zt5K_%w+1bP=-XbTy>-sP9Z4IgZ@SkIn~mZb z-_scdON!vuH(SJ*r@3`(rKHo_ex-||9+ej?=L9|`El!!FIzS30gV>gG6 zKQ{1jh0@rzlY&xqFA!?CqY2Jxag8Ynkmyz~R*R5Qzv>r=R$_DnA)ufJjxh}Rs0L}I zPK!zNS!tV6A12pD1y?Ni8?#=-&)FucZp0<$t4J>ht|LP)vB9sgAHYZe9j@Q=Si?W~ ztr5k~_hGj1O1{9I$y`H&!h|TGdc+FI+H@;63$81gQpsxa(EE2p3ZN5bay@j}@2?vk zW~NKE$c&{~EfKoW$b=dJ+twKOp4|h_I^&A9BYYKaYOY3748dK57nrF2B!(p4Fi>UJgE>2$HXFQ|~9DJtZ!*nHy8sW2^= ztfx?)ODsw#-)jM!eq{@0zp7AE9Nf5!F^gUS*egKZ=Y632&`}sH7y{`a4!D&e*$YX9 zo}>YUA6=&|*jg*7#wXS7`7-l-jmxB7U}ZhMpX;2h`QcszK;;$v;e;r-c;I+_GuZEQ zB|NV^{#8NqsV$F>I^kPBjaE22ROmx);-J` zWi({;i$xMJlQ^~h^Yzn%OWAeD?~{Cn>0&jh76iAfP#mw|@`wS1c81h03#e(4-L8m^ zbvfh9BP9W!hY{#MM6xhMZGe)^PdK7*u1>nqUXjyn0IF6$(1b3~wrwJ`UEK%+Y_VSX|A|X z-^KYgSB~o=flOnlN69ePGKV=! znXEReEX=Nl5XotFP``bqL%UOlN@FR%TG2<<)rS3HoGNN{^*T~AsQSOK06MW1^09lr zy5Q*sgcV&DO$>USr4jVFVV%RwS9;z)J8_G7M`5N`Y<}LNv&<35k`v5P=l@z-)wbM4 zG^IRn)c?&}N>$Pm$M1O3Dl`0gx$}L4|$1&rtr$7a4gF>!0q9|tt9EyKgU6IW6Xwa%B{bS zc(8YZ>!=ZXUz{>k1YWRtKD-^^)WSVh`@T7*`{nhy`Zux)!%E=U zxtP{H`UUzzE;g3MMYp5p%wKsX0y{-`IQ|L;tK9XyJxBD-1OecE~ zyPRBP2A$W0U&-sBS}~ftS-;{WyP#P<^p2ZBMj`Ae;?^F$9PYdrLo*Bdh;8!bfNx0 zZVQ&%8>1-uHaq2Z>^*UHbQL;mx~GjB&Mlx?(Zq{xfo&iL`-mR5#oA+B!HtQcXQ$i6 z!_mjEPT~9-rm(VmYkEF;bslB-l2)SN9vl$i#Ps?j$t}FTW!65d+>*bFU_tCT?oS*A ztV<=-_MsP&m2B2>>WqN8&bM{#y84qZgJpBOn+*!1)fTWf4HQ((VpE%wqG*)nJYVe( zqiEnvIwXIOG}i$J$_H{`Pn_s%`PKX71qma7P`l`9z`KW2vX>CxOghxTjG$ z;dZQ<dOy}$Dhbp0Itsp^s_VO zW^q6J!T-JJSzl=`7Fi<8aC}Q!E30Vx&l*}#Y1g;!r)z6tPX1#R`#u6G!{gf%6?$CS z>F88{{(_tEK<@808Fv8ZG!TMdTV+1 zclwjkKG5~Ji4i$w>0z?Uu8j+l!3eEB!L#3|NfOG!Bk+dGd^W7N3wMY!X(rhT4y|Ya z;qI(MU8-ex*3iA3D-Dp>Yvp+m^=KnkiVAG&vQ;RM|sV{MQaKdm@;C&kkh zi>VI5;r^K%3`J9!lnU(Kc-2$!DKXv!txnvRf_;;xoiqP~GocYBsz&nFtOvm!)dD09 zXYW7*dEVayz>g{RZE{~Q-~#F4WK=`mz8bL|O3u*{Fp%~98mW#u?N`Rfp&=EE6BN#C zs0=jL-1x=D7DZI%x(qwj)6r2Bsx6seVxVVVf)--z#`SVC`4Lvfo3^y_Fk05b^Sekk z954y-2HFyb;C7%G-*9;iX^X5o-AUZ_ckUFjqsDaFMeI{%{1l!nl_q3u!;Wa7g#x=4>@-^i zMzK+v3{yqjl_UnLUk6o60;@VN50&L!U!6*zM*vWRG$eu1_C0SwU$L%ly59&5HkFjp zjqW^Z->bQ_uy)_s-^@}XV4<61#dzT}3=G{J*7O0cl8T8cutwLkw!wytw$v5F2|tCi zPVWeVCim%S`RT=@Q7hcA7InT&K2D{o@77D!kA0^a|B{@3eI1isQ9K1BJ{dzs?_8lW zH`*B3HdL=ul!`{hUN;1LY6#OyKuR?_2?JtTRdBzzw!IkNX;In80z@hE9itWH4tFr& zTgH#3X`WjtOKsb;sho%aKf{da($2&|^to?{U>bzA@uB1hl(Ix+-gBX>GHf+PM<#Px zw>5icIk>>{Q)ChFY2(k8P;5}h$J$=kKFaO(B%F*X==SU)Tds7=XVqM4de~0U>LHw4 zO%L@iQI<1rrh{JB7(on!LWYg0{6SAMtR6i-@YqcoJT6*EUi%3g&SK>#r8&j;PdR-H z(F_1Zq=4qp+pI(utND=;RnoUya9{E9v!|x@`j3$Ri8q$;Y#_*#L?fE_{Rd;i*eS2C zD6coGczEZ@Y9j7U(EH(%I&IHybwJe5$F4&v)-N*qLifom)H)x@?m`SFFo(L;f*FtHJfHs8w!T3cJhFWt(4*FK@b+JNtlmeqaght% z{2$^*)mV}a+q^dBACCPlMUTBtT%}Zf{6{|Qr@E_6M}>Y$ zm}s;@g6?jVV$C?m3$b1*p`Nea&F&#>jk82xdmj0bc#FfYh&L}UdsiBNct7;o?MW&# zbz6zIjsxCDOlVu$L91~#4$|YNEYPm_w<^dUEsJT2dweFysOda3L3Z}6j$1W?`4@Z^ z>5jt>*4HW`jo;TExRDge|AJXD z_)~ZX+{2W5I`g;*6kE_S`0RvjYJiW7gPs=`>tx2=f7rUeu_^z+vF;MYQpkDo>+LJK zUF5IQ(&(>n_Q{Uepi4r!LB4@x=k)VTkY6{qL|8CoM$(7gZQWoYHnuVe8b4bG(au%| zhql94@p9tb&oRq{1eJI;3&m7h?d-z9bof;+$5yYBN5%0)^)R{c&`g9)AOU_2W*Zew?Ai2p+X?3_HA|ifKZkiF5?yMC_%as|7 zX93>TDZdkTZM>?{#R?hAUpU@k1@JxspW`|s;$+VrmmQd0+C7|?y;J-SzXIG^%YDa# z$t1PK)#n;`x0C9x4wlRpa8V?SrSQVS;DS(r+5-c~849tSVdJqHT{2BC^9e}io4*9+!HoE)MC`^V6(;SqaJfrHBV2V|B+|5WwDs;*GNKI1r0`4Udko zY{?lnjJrjU;vu?tWJJHatbN)A!4h8O$hAVglN6osb|h9|5A^=8$t1YH;$7v3OjZ&_tJN@7<@z=qm@qD zO{UbdX-Vb&!;0r8>d@ZXB?%8gwe|>sP3q9fnQk)ZZesYATQfL+;X65`JM`3T(0M`QO9=%Rv+X1r`f*y;AF)>ai6xKC>_qnV;ZWGFbt1A71wnVMI0Z4> zxEQo!38&tb-0rr0m+_d3A6|GxjYgS z%q1)!Vu$#HGnP<2Vn<#*`%SPXPgGiCp-9z%O=B%;_p4ZK(x7=5$mJVhuEY`$h3vUK_k4^9siA%S67~<)6_`~Ct!-y{ikAAGf$Cr&nLRiy4Cz;8|#^^cj;eI zpW3jxQL_4P&-ghxnx@iP1TU6Y!BI*86x*F=TGr?3BN=Ip%ec;<7Q59Vx)4a>}N35nusx%2^M13OXQufRCgp-%36=hDW5#(uEb{{c90= zz`Hg$pc71Xz^DGu3+Og=IW~sxfu5UtO{A7IGx+;B3)M56{gyW~FY|)e z7O+#S=y&)B$&z8fden zQ1R>%eSfrgr7d4OLd2J4-rwAY1n+&eu0Ta22iz(d%NU13bSI<(~0(|M)51b{^#Ma6g@zVbE)Fb;tk6 z%B(&x7fJ|qX}_lGb|8p6@(sX*ER`~!Y7uljPnW?dj!UcmHYom9XCD%|-ahc+`gTeD z$9a4PbR5!wYw$2d9BRZznHOE+`rNXrdh^jbvJX)&4R!b{2 zsor#j6|2#QFjCVag;wABGi}f^D$UGu{3Gut*WPGP)?VhchjYT9}Hqqc;K$g&~aQ;=>hZcB_IoR&;|sx{l!8VIP^7eB$rnBt90=W*;#TUQPNm2Wx~z z+axk4Y+2&^NsBeT53gh+gW+q9A~As*OF6L7t%H%veTf#2+oDk0)bd?$8t=ep#1@w= z{`J7`bRz9-X2p2UqZ40o_mg;aR>Rpis^?P|1n8{rc5+!Sm&~O|g!@)KeWm^*__Hb; zZ0S?}qX2QLv$E{bM3RC~npN;wvx0PjbRp8Vl}&WfJlTqlab=HV_IBlvkjjcXA+60E z_WhnL@Gxpd`-WA%b4=T*)el~!506+{%+YVAqyRGzE1hB39IVZ|erJ}TWk10V1=N;_ zB|E|gyYl_|2OUU(N|B6Qq$1Ly1!?RHATWP@vo5k{>_lQ0=Zdl6{Kmbu%(&HnX7^Xt zeA%B8K8!$|=V}SN_Po~mKlqQvXP>Vh35aXfdm(!1gYg$~M@obZa1WjbZ$fHN4@zdR z$u~Y?bJ&YJ4`h|~>|J7UKHqNM1lr^XFvtfy*VCeQVg#ssgg>%A9b(DJa8cYgM3Sw8 zFV-3?k+;ZLp-=(}zL~JW8x!k31B*Eb=x@y%hBQWzMr@hch0fTMt|)0f;bGLNPNbnDhT9~YQC`6pCB9-wFiUsOGUFI_Nx-~ z>lKhK@{LI13eadigP(zJcUSu>US2`L!@Xn8Q~l_&^{KdREy2^4uasKoYC{sb3zZljN_El8pjje3gme z#Ww4K0D+H03(0o7WB%(}tp1rdWE87|MNbV-y{pe{FiFtm>NT#{#h!>zlfLTFCzW-c z`3hr#9k=t4Z!Ocd`!!i&o57QX;#2ZFK2(gKhn$>}gxTAu8K3Y677=y=1^?`uM7rp? znrU@@6#M7^bAR2Oup>6D-em*=u3VyxmR%}(Vx#v?7K?>k4kKl=cealE>_YZN&IWiX zA5{$(z6;&)-_G#Nn+@D%T>JQFj)}PxZR4U`GL}LCYrQMv)y}4+J_OcHb(q5sJh*}W zbZ-f>pwT`Y=^*{n11{6ZaAHp$1es}{UUXeZX*0Ed~W{P|^2WzKUJk*0b zvpEYA>2R^#Qgm|S&S*4hd6By`2`C;*Mh=iFpRvfzg)Oto8Q!A_X#`;~SPt=HkK4O% z;n`(*#oflJSga}r;lr+ssz^PoTcn5fiB*Vc+mFzy+6GX`KXq$PCe!>b{U9=PN#joa z62(2L;}gh%;qro&>9*?t&tR{8K2nlmDaiYJp3iz8uo88_T4mvLI{t;z&WjS|@ph0* z2t6kh7_Kk5@UKv@^ag%{sg|J|^7t7lHmxsvZ+XeH!Tl=5SLhtA8ak^@D1-Z2J4S7j zJyjH0pVhWH^8t?o6l0TVfUU#99(L3&P_ZIljG^JJ$u)2F9izLb%0y0>1e%sysB*mt zMHCX4US$AAF4=ye3>Rv)Oy>S%)3xZHp~A@GH~XpVyz>@zM6LLOHc zcXF?M+TqZy?y05oGk#x#&&tS&1L{|O$+~ZC5TO8FKl@AAhcj8&VwQw7Pe)soY4L-F1Cx+t_&vavf{O2+R1qw*yUu4B64EFplz}vGEcNfG~ zhW=>d%aWae1<&X9yA+HW_T2r@3jCB=l%$on@{bzvUyn(=KMZVvX-2-t@K?$MTR>t5 zdVP4Rg@o=gd>KT?K-lt(x;*(HT);ad@s;kdH)*JBPPk3Ws%84%h;7=S>gVD=dYQ)Yb>isoY#NhHPxZu0OEz- z_@94|l@U8#!}nl|))z4})@{E%iMh$8)f9||SQi}?g=b(m zAp6Aeb!$58=iDP(fL4ndwLn(7>5|}|?RVf7l{+nG*-S1SJ^bduMJ8wr*Eghe;+cZl z`%LM1eL^~$FQq0Z{;xp04~AcqA)}LnP(!Nm2`h*9hiniLy{mB}4@cj{hevd)chgBx zP~4kdWzKtXY1FTMhoWNha%e0l+ky}RTj>OZr?Yd)R9g`Y?=YVv?~>kR^{UY0Fkw%h z@E~Pqm@xzSMROIU9^A2ZPUPHqc7}S)ddEo~?X6uFh;Sn(z%|K6AL(}mw zltGz{MQy>c25~ar3wDH2-SoAqnwh|M&(?mR3%`I+mE~lSKzU>}^~Im;^7IbvZfVbW z91?v^$^K%T|CQ{fXh=uJSgrsbz28iBHZ(c~wZT;RvfPgCFa$BTpI{}yC`~?ctCd1r zL9z(HH^?bR0?+|6p7g=D;x7mxO_-5>Qy1k~zg7pV&}s}a)>X52le%S9P?sW*FiW23 z+xF7ojkdrJ9$Y|PJDv$Ioq<88n@4~LZAr2L$n`?Bdemhbu2 zmh0imjyQglkD21{E#)Vni2qlm`>$jLoG6hVKzbbRl@1N;mym_fZIH#u40eFUJfe;f z#JscX`M@X2geYdh>I=|N3rE|fG8BPR{$kH!vgq5o%*01hA%6c3A5OnYUxUcAb*cJW zJu#O-7i{X_)oUvT&CtnP4Z);_WB?^Au&7l```(rQ}gkDbee1Y!6e=Wp6 z{Rk^!4DyJq?~nlky>n#sKeO^Eh{hf0U*nf)WQ9M{z?OM~y`u*O?LN#8IUc9;o%?;; z)P^f&Q=PW#rvP~S#=S_0a}ow7@psREAU6^ycLC8GG(HK9izX{|+nL{h8yQYBE9`7h zZIfWWcm8SqLfTG3nUG73BL08Ky!h)e|36!JK4uf@ah0Rs8)o8l9cr3rQmOTWjwmp`>iE1p}&?`{B6r- zD#oCLh@f3&>P2ja7&-bD86anO3k{&t;@NLFi1oS1buaX$x-na1!-q*)8S%@lc@>P(; zCvcz_-b^T#jF7)%;{82}v7^86hx?@(br*p&)SDafQE!kertbVVq|)8BQJ16T0XmO| zw;h z$1{82y$@`yn#0u_Kf)oqtprzc;k~jUL%zs;Yu)&KyD+uA>?Zm-Nh)5`?HM+>5&+jF zfEwrxk9|Mt=rwBGg0{z^FR*vXM2eSrFSRVI)lY_neun6_SbezNzwtR zM_kw|54v@-X3{}1ASSBQ@)!wl zA~vb1B8W`@@b^omw|FR$2=UNY7jj8uEb`b>Pu7G8z);AZa~oL}BPpx;r#v;7w3__Fy~hU+^&GG4@^rkc7N6CwEtX(dxX zC$fxAla!C=UW);0XU%czlA> zBLL^28Z2`~7U9PyzS_dtj^y%snmR-8@Jr86S-nUGHR6JJw57(~%wN%eJEH158$i|x zfx$emZOig7fmN5Fqn@GTv>mwMl?|_`M{%QcttX^2g3%o7`-zBtwUYU2P7rnl>of7! zlGVRQ^k487{xI7!gCYj?^#e=Y68v9VKO1e}hFxBmYv#Pj_C3f`ogf=!S)ezRq#;TV z@E@Koopo2toN)3(ZGrRy$h1-d+a$V`{(=zpS2_PbTmA)5{;?udCO+=+KJFC@@%j2f zoe@OU+qJ60fCoL&(v`E6SN()xzdo-6_`fpP-?qeJ1rTEH=ETE5N^41@$;c_hVlJeV zM|Zs+9lusq9|Ro^oeQxu{s$1!6tEv^0?3DR{?D4%Q^K2u2(hTpf3nuAqmY$Zq<(Cw zo7MjTMtmm+rLf2wKE&Ss-^$Lv$_3$%dlAg|{;D7RZOi|E|GYqHlpRS>JW2my-OXS1asvv|b7mzB8f5U+S z_*eRad+klELxB>uWgR-Y*XcIl%l*qbE3+RU)M&Mo5pwIws&WIBHK^5pkrsTTv(bU? zF#9df`u>~J#=p+HKk0MIXgbeV%6;M z%KqS(*`J`Og~9&aqwpF=u_k~x{+)o`+&?S~mfd<0HHv^s5(sF2I__SoH8W~lAp(0G z9+piP^2-E!Y;i-XlK2Pxe!SQo4Kn_lx0V*}@d5w@J18Y7Nlr@(e~V1a?=FHz1b8u2 zQ33lx3ZNgR_F(*HJ${h#fc=56F`Nsz+tbrSM6sKoa&>!kea&PpH9!aE0lnj6e5L*G z$o=2u!(T%fZusL*XVj0L1JOjS*`ox2mkH-v9me&d|Ltf0?(u;tTR|5v`sITbKvmU6 zAbQvW^bU{cx{$K1#J`>O|F=_R$aw!YPHI48JmCI%LP+lAHIPE#_3`>(?JMGND~)^T zKUu_&?l#qh;d;yDw6v1>xD7$4OSR|zZ8cyJm$VpmH2#Vm}P>kih?i^ZvVMNiYs-5b_`a*QW75 zT)IzCrA(dv-nVB>RR1^r{$FqDEc*m;Wnl3D`44}e{RECUsJRjC_4Oa^qEAq5@XVy3 zgr7?PY}fr~1Z);{Hai_$ygh>`n3zzC6f>@zm~15jBIr;tFoL3q__o)*KoUXFNP)i0 z(7Qp3!{Gm9#q&CUe<+x>OOa)d$>3yUlZ6Td^6A_xhb~6Kf`|F^vHyVYCgHtbE(ZpO zho>vIec*%BJ39j)90rKoAnOef2VoNagsA+3$&(C7GW5mU@OuY3aFD2y=0jEj?{xUz z9(F`3Fd(i(_DqcaJ?H(83HMjoY(B^o)7DP;U!CA{T89;7`SKRKd>0lhPN$Fj2jnor z=NABfp0OM1P9UTHU}o4% zOpRavV3Ze(_B;?*)%q}`Us#q^nseK%t(R=B4W{sgFe1PxzM`&u;?_(LbxCuDQO-3p zOs5mQYhh?A<~U1@+$G@4j+?Eod%$(%vbleDubx}uYT%R1cNXv4(z6~FI1P;&$=_z4 zw;sJU_7WD@7;s$p)+!Xr96#*e1{RF+511-o61yoK?6ESha3{x>fk;@wYr$weRXA`r zpvATC=71#d+8k@Km7d7z1sR^;&PV_5m34CGxz>kXQYp*7H!%v4AZdGZf6jf*46~K; zd3fu)6sfVSTd*c!Pt6UE3WoxW{6w;U;uh8-kecY^ldyv)!+7jMlifZeB5h zGT%Xe?>WiUz<)52uw(%v__0Z!&cx8O)e|w$WBs^BuwYAvMI3@~2L>Z9p&8>4is+b@~_b3S^T3Tn7SGlo$T_NDdtqPZ zOpkV)S$prkUp<4b$kONQU1&ENjD^#I8Sbt_lo2PTPtV9ctPld}Kuq~P9aDxw<{KIL z!>Rtd3S)^W1AY#>s4bp~)~%9He3BoWdb9ISOQ57vw;@V6GEq4x4G!pjYfhm3(k@- z9Qv)+Uh>1<4r8uShMJMSj-7-uxKb+8`IY1N()oXHzRF^VLlrW_6nmy2Ee?1!<` zH1i4D=@h$8JCxs?-F_{Od|v^Ajc_&(AM#?(gt~}fXdC`miSihMZ;Orv@w&4^yMn^Z zFYW2Eu&AGCP|^(bkjyN$nJt*Vn+m$80Es=8 z3*K;K+x|AqC=Msv$`?M%C$_^v*9K(`>7t-9IRpPjG3qp29O&IBa&P&6pkM{p36p|7 zs?4x`EU{B~&wyD%X`ffCN zMBMAF6E(g7Mi07x$klh5SKsn{7GCGrxTkS3Iw{9dG7twT;D~yYgEU5E(7iaVb#xj7 zid%2?6Opa;4zgMj@pymjNc0i-Ri+}s`7rmn`AX4ZTa_XajN$FeEyog0SYL2akuZQ6k;y(DkUAkpFpGQYxjI_5Km_ ze)WI@WiY9bpTY^ihmt?*FoZDPJhLLr>=(>;+xGjrFA;hcVG#0g;qQhwDm=1|QF2&; zZ{2L@yRL~vBoc-$!tIpLy!M}|g}#Zq?N+P=_f!wz#vzukB%gQ>k94fTphyJ062y6N zV+WaTybiY;G`LirN9TwUe9Ma`Sn*;0O>o-Zug64j0gUhQZ^OZ@Ec|c9_3srEQB~4d z|F#9pv*ETQi4C&n5Yxeg*(-SpYAr_(*fb{-1wXP|Q$q~|f-g0u zYwTj1H+@1IGWd=po8>0opxktx&sn>DgIg8643!4(T`t;A+&PvOwDy#JNBzC2vxw%S z{()R!hJ!?p)pR}Qb;taQ!Ix`V=fvJvDGVnJ@S6LXgE6aiUNESAL(gdmEjrNO-{M-m zm1dUvnTqteE>z-x@kj}VdHlTC)C_|OcU~e9UpZpZ=>PzK{D~&=(y{5$GqbAc#)NB=ni>aw(cLy$lDw3K8S@01MkZC+oL-J&BK`X$z`Lj zMRnflK!Ynvc_Av6DWQ2gD}>FNku6*9>3VjYmH1r`@2-+33{#=yWxFTsa<%4c+?7fw zJkp4(W#5}CCRX#HRFji)o^9=WFuzH#H`fBJhYhn~${(K#>J%oR2t@pA9mz?rDmJ1a62F4hW^YfatDQ@QA@sM zig6b-aFJun=+Ca391oB9=~|BrkfpUIhRH~adjA6j= zqFH}&sOYmXiSuenX_>QgSxo?YCswRdJlfOREVg>Ka07!d%x5>Noywv{w5w(ZbYak3 zf58_2I92FG&Pm2lQ*}X^WI^cVY%R|=B@w0?myDFqL`L*_{j@`#!Yva=Jm_lW{R2uY;?TT2gN4X# zb#@A5iG5%*0P`I2vTm@(OwSKrvCX?RpSA;zoQvbvib3M-@|9uOhhV{q`nOIK$&*JB zx<@l#)rM>T#91HB+mD-;FXIzu&D#2?RRp3*iVmV*^u#c4fxeGf=5RQ5w@_ zfS85ZIidlUhNXSz4p!Y#6vpDYe4A&Ca!yi6gGxzp_v2ufl|CJjeR*tDTaxuV0Y69% z*5Cw~nI|&vIv>0h?l=;$N@K(XNdb^M zptqiP(uSVfu<#amBcof<{;U0l3YL01c=YW`iLye?orj~WOO11(%PgiBD3aGlC1035 zAk@ICt+_k<@25MF^o^czx@xKueav(Y60)|wG3gT*;3FKsD%#j=pE12$Qs`GzaYIw< zyN!BVwYvawR<32Nig1^0S96Z7Hb`alOFv3`L&wU@GyPksn#>Zl}oLD z=5?TnP6*#l~MEtU0;Oqf-9@uY{n?!N|L41}Yu z6&B=0Sqi_&xY2q~wEs3I&dVYU>E-F+tsyKH?^fMc&zcyL^!*%FB_oRMu-^h-7I=c}PUV!?DMQ5RHKkMwyqrLi z?TaEBqu9dfcn90Be1eH?f6|;S({zCe#E01>!d5xt%obXH#)xMc7u$z<{!$V$haRFE zdN)w7p_DRDG;rtUnj`%u)u2Nr**W{uc-DPOK>ErF%kFvFZR@2<2Ie>6H^Xr2xT+@M2M_G5U`GRz`K9!_t!ih&izJTN_034*P{=;Cy)- z2twzC+gg5|a5jj!Ikzb`h~b5|kMt#lA`z(V7=@k>HA(IpOYJ~YkNsf<61J9E&+vUW+`gB|$-sb@~9y&_y^60*dap?E~N@_Jfs#cLlh2YS$ zcuB3@AqhT>9#{FD1!lKrlDra_cOO9!&tMvy0q9(+{le|Ek0?8?6FQBy4(>BkDr*Yn z{jQBI_}=y1y{#;hf%jelKr035MPf=aM~<9fcOGe#Rtkimb*S0YJ>|3iBw;qz^>{e$ zmZvmOs7`%(w9MG)|FXv)gg9uZ_$udU4L7Ki;O|}e{0(vN-+3sD|A5aS3@*sPbJF%g ziSm;AeIO)@uf2boELAXPkrnox08z=Jq*fLicC?$W9p84Km~HrCAmK~L#M5Fs2@4Wt zz~txT;ZJa$8+xD{p1vSHN@O+%&MD+#d~V7(q-*kl=G%CLr5bZ^sD~B)Ic0qr-erPg zLlONu@faNa;XKT7V${Oe04F1e)tsKA1A8p{DTwO{=7HDz{&+(*3G{LUw8#p-B}LhB zgI}o^R5T(E-Y59C-8VJtG(oXW`@}fpd%4RYO1DB77(Z?tY_Cn!KQfU|akj(x4lY11 z^u&=4DBXCzu~mz;!yZ)2oloLi#M~&TWE?ViT4iRD}Jv=G6aqEs91Cu{|bw} za}!B=&~htT<}19D3yM!&Do7U#y~;-_H8$?0bQTYc-znIKIbV0xQm* zLiQe4wuxOT_6={c0{| zRQoOuj^6`eUkwWgUd|_%MK?h)iQ@!gk~PUG<@Gv zAaDI9g$@RaJzYYF1C=h%Gck_E25eOEmi!vMO#HswoxOZ|G@~VQnxrF07h=+ecq+^V zHfBP%Opl_RHV7mOIkB$unfQ)3Q=-*kn~^fe5F|GkpUh&8-tGFkTJxHZd*osv(`PdUw- z)2n%Em-%6ltbgvG1q_bvGmXiuBjJ*UoN3$XxK^_KHPd49~oi(5f3%{fQuG{LR@mdFg z9;I!DCN~f&$5EI6g$1AyMG#%Ey#2k9l+G!iuF5d%tPFd*q`<%+%hKYPqv8(*36y(+ zphu2UTgx^g;5Pxv+sO_)s1?8QC;h|Fvt-^E8iiwT(jVK7CecUV%h5?x|JM6n9QVO~ z9NRnNNy`3UkHXa-f{Int;Oo35hv?fj%TLMQI<X&l0;8X)aEh;SVXM`+0YtXCt)uK_Sx-aHqW3Us0cyh2`9FZ!fePyUbiAj z&r<-tkbXTlPDW6yY~8KFo3hD90~syy8X5EZSyR{KOqKDYW<%Y_rCMx1aaGAip4be1f@9hR^3Y%h##l99Z&{Dq zP6vxvsoH#JJxB7vnkdHkKfmh24f+)z3`%Xy=e#;p6z-<3eJBovAI?Y?Kht3EIg6L5 z0fskEhNqbONR3;5lAz#ZRc;d&RxX!$e3SgXR<3$_3K7Z1O2;VM1Ys{FxVjVUv+H3D zbZ5Dw?Ei%X<9#Q#8yT}&Pf0?JeGexR8kfAD3^-=PEYs|)*BqjDM7hErh_9) zZ!;gHsZPR3-=>P5lp}O)%>O8+<-54r>IV%YI=jW6PK8Da`a-*iP?1(=q78pK7Fs?| zaklRKN3zIapn2xUFC9WxSu~*#J0|Daq+IkCk3GZZ~54G`FroD zJK?`=Z-yV@9{;YxFllkA+L&C6Mo(AH3w^k6J*w{b6wo+Ep1^bP2{p}#OQ*VHbUM!? zsLt_O2t55Nf;{EQljuf`PtHsw)rG(;vZa;MzO!0H)NuUv1zF@Rf~5_?PBSE|VgmVT zv{vV8xP8RO2)C6N+uMFwiG?6K*u}e}6s}p6kG9GaNAsuV4 ze9~NVK01CrJ@j~jVe_hzDsrSfXpTQ5crk$UZFDyvCxWoR&ZCsT5`7r}I4 zoxULnlef>{{>G^o@WhAO2apuEyfU-JS5U|1nG}dw?fr-!%`_hZT$J=RptUei zgeS@`WCH!dy{Xes5nVGmvAdnOhFkeS>RTpzS&toong@0u%v0E+o~@`u)K-NVZRuN} zbKb1Bh?VR{v#GeSWKo6B+d~4*96sgQ8i~HkWsF`s5j%xa1gD8&ID&h=}zRhlfA zOLg*}db#)WH-AnBJWe<>?5W(PmBcwNJiYXJ)3*v$(i~sOd#Vr56ZNrBi^nm3@V~_K znz_uv`MuB(Ar)HTHO}j0@%q(X<+v;`uKHugxFZl5OQ5nBH;`)3$P zyACKkYJ@nuoi=SBrV z0PHR!B&)ar>q(jHeI1?6{W)G^TqjZX&o2^G{)le*8djV_i z4dR0ve<*O|NF(1EaZ$O_HZXC4VuKapmHQA9@tzJ;HnkM>a zbT0pORU%B-ZiC|O++N<2mo=X$xY|_Lj#Fu{mb?AaafnIu>N`r6nzB36wml_>8h_ne z>(OYZ45*xM#`~bqqHuvU*WOz0(;E^m=RJ&n5B=yN74b3A3^ql^Qo3u?V1ZMW5!yewZ8HEC`vL<}64-y<;2 zZl}4U$_I5}8%+64dqmxykAJ7WO~98j{?1PyhP?v~&KCb-bmt_ zX^EWVD=tDSZPV&C*az0NqK>KD5?_jWN%zM4HtpMI*H*!<)0+633rcL$_f3}89W2>h z2jT|Jn!ztR_z|pE1ZsSjPE)V*`Fl+_$VW37^UUM@`h-X251t4rieuU%F5V{k?E(wM3s16C}YhPT@hug0U5U#~$G*Z}MaAKq8&t^o?l_wm^jF>HYM35Y`lD z_dNA4EuRm>RLBq!AEY35 zA)XJ^`5Aah=i3AY;opM$+MF`wDu!HnjNP(Tfg1+Yx=0WDwOdorUskw2<&Mv<_1!=i|{&d%FC`_KX~$z;VN?M{b|ld z@<_vTHSt@+gH4Ne^rC_YLN$Uiq@L!SH#c@}gIO^V4!uo>>Fj)Z7W8OIV~BU|)vbx~Eo#9Y4H1i&<2MGt^;8{h&CixfO00g_ ze(uzB$0-ZDqjU-dGmvIq&CgS;ao_r|GzWRlHiU1RG5#u`W>-Jwv`l5?W}hi)GRTQJ z5+hZqnxFW7Zoh{unokWlm7;$@<~UyiWF-VTjpObY4egK(3x3voB!bgAZ_#)R(2C~| z~;)3C|}>>?fUIdJj$XoDD^TTk*65G1L5e$urD*l&6nfRryGjd?rz*SxSM4jt1yRoRlI$)=)7tC?_K{tb#rv`)*ZAP>Fr!aABZR= z@?I15`<_m9)h|*b`ALIhp6UQ^7)0_CMjYCPb@Bk{shj%g&tqww>9zxah3>a;PpOGj zFXWRZwEUpn_#t;6T^TzpbJU^Il_QkqYi-ATnKQ4jlj!D4B{(-j{XMx1jbLQ&D)@qk zDEs{4aVrIy|6I__SL2%y3Wl;u0WMv0u(I)`{Yv9;^(Pc#k&*4ZGd7+l?P)86H-lR+ zoq4cuBRaI0KF?Sk-s~q4a2cu^%-E8H$V5;5xd}WFQ^&nur;SzbV8%+x?rvXugudi} zCcRSmmJOA1o{FZdf32z+?>k=jIsV9jw$dfp?xmyUPtOpqXeVs5sUo^?(~7>EhDooS zPHUIxG@BJA&-EYZP&4sTzfq2qamonq41xJD03Y>dJGzHc!qlaQxBke?LlXsR7{36o z&uQR2?VE%S7g{#peguX$C23MeD&5d?5M1ZP% za>*d#4G$dlBDU{QlUzQyZ_X{b&CQy2QPatx-9Ti!H=Jen?uP7;2 z;DbLJx%FI4cAxh+d)7{NRIemWNl~QbQVE> zD7YS!09y`LRVak(i`R=Cf%(KkO8Kk7$eFIL2Fsxj;)GC6SNvq$gp3DeI&3?1Nri&%A-dYPNc#&!t~z?+Nd zg;xSduRi^@#e@KEo!Z9TT^eZRCaLlHLJLYbBCY7I;2&Mm+43r0?7VusV;Ml`59$A< zZ#Tt$FkU|oE03mZf0Q8*D4HYO^lQYEHWPJL(aqy1R&Y*owlb z0z0oMCU66~giCQ9#h2ht^Z+5|`BPSHfF=_h8!+d)W!^+jpF3@!(=BrD?lFpYHiI4L zHpqJ+#VL*^k6>)~k&MXBpnIKeD$PmNiJH5MPp8oL$|4TciL7~WKJABrR96r+URW5G zvD*o0KX*0;5l?J}Cst6cf_T|!rD2YqYE+dJ61Dx(dy>lfBD$qI4YOfy;{B(jy(Et= zz>`=Mp&oVhz|jZX9QaU>DL~+9gwELchVaDe!iybL5aIU2*31|8Q*5$qmz@EwoNbL< z(z%YFgMKTmeMIi;_{%DYvIT)`0DjmFHR(lXw%9lP!VeSG1pOS|O@?n7dWFz*a);{o z8+q;2g%JZLFI8S$1FDqG>oF!Wm4*-4(`QC>6Bh?oz0jVM!Q0utITJ1V^W0M%FxdSm z^>7~5VV!%#X(luWrTN`F)pu4R>|mV!ZFiYtnrU{8TUXX#eJr=Ty4^Xisil|@Cf_tER;mRcXccwmtP}8sYb>qfcKYqRn?Gufe?dnbj z(+670__d+ed2a+AC+rZ!6}c0?&Cb{xF5QBO`#PTj*alblkafziOFno1=Yw8#0oyH> zW>Z_!IYx6g0@*FrmE&eOrgG6Jy^s=VFNUK1+d|8KSm^6#ttby^aj=Hv_scW|QwKN% zo3q;lh9=?up;cpgK_l_3hA`>r1yuAlmSC;3%)1I_*!M-k*ErQ8LbdY@NMde!{3kjM zLt} z;e4F{i&j(!BICrz%=*z%CVdc^$FP@qyQF zA|QH_+{bB(BCcd(&h_>V0pH)oYo#_Pj`Pf+cQz|(T&`6f`dWq@k{7@+Gcyy|@s=Jx z7jL0%5F9>zW@C2E$Iku@5Ji9Bg8{& zOp1$E&9gImKYspJRbl8ibM08xUZhXu;Po;hEZ^x1Rwn;UAKBDJFlV%Ealy?WK|30f z?M$`l`MBP@U_Yx<>wN?#YxpVWh^U27tsxsj-oV-(#j?2=2U?5RNAUVr0!mtBPBksx z59pm_AzV4($k2L+<~PyqW@<)5e#hsv0>H!xe{8M(mBKN?xe_2h0qw4 zw@=bii?s-rRHh$eqlBJxRbt4R?pgO%BRBa5&z5JwhtngCH{d+r;7MU@bEeYFP@zT- zU??EZU2Gaq006Ufn7s*gg(yu@!OZ_2T7P=!mVpjcT?qVB;7?3nq@I#bOLqO?3j2DY zm#`Y%(yllh8ksV*_o2h_Bw{;vU}dAq-D1hH)qaFqp7LuXcxoGVpqUqVb^4V>lK2SK zz-OXfeDxmv2!6N#leBcNxn9k_-mFM|K6MRbuUD`mI9`}(khcjR=K(6o0E7JSFVv5A z2}b>z8@ud;&^KPKBNNv+kA?PfTexSy#-ZrYY!s+{7{`A%Y>#w9x?(FxhuTT03sYW+ z@>9M8m7Au~-pn;gAvL3?X$c3z$OJ>XR3O4vGh1YpBJAS%BbW&5{y#ow3AiHOqWYti zqGC`qB4v~cMQd=f!s5Ay{bk>(Y_u$?=WwSEJjYh)%sr9!wPjsIx zBN6}uTj~{(uYH|6vNHX11de^`FrXf-076|ppDvfm_Iwh3;$3f+gWaYmG*rr4(Fv%I z&iJODVRb=C!5Oh&rsgK~GTXJj2o~#grkz$^;|cZ|1s0FfF|VjN7^#u&opgm4EEW85QFvfP`m|?z5wo8gEp}Y@@Gb?8xnQ^{1qO2? zx6M!bt6eb;w4QuzUQcqn?*`W{XJf-z=;_XI{0#!3Et7q$ec$v=b(@Hj>@4PbEvxCX z^+uhxOO3tlU_sNP0tHOFa7Gn>=XC80wqosz!>*7u)s>@4y*@40@Nw;*vc)ZregI#; zt~g2AZuP*R{|7?lAh18gsBCR4aM09JUNR}CulTbX774K`!y>bC98?}(EMhar6#y}7 z8pV8Jka`^odmdh0XA)z4*+5VsjC}x5dx4%YX#sd^w;9=4(T<42+w-}>8;)0_te{kX zvGWvKjR@|1NxKcsR0=V1WU)Fg{=#b;FM_>e#mLjgrlg2*UuJ}#WL`FO9>bSQ9?|Qbd0 zA>Zx=4=)LlX>gU07JpYbcP@?{e13K9G!=43Z+{TXwWZB9E!M`eGR6o4osRb#uw=8r z`%Kni8qJnEd{V`Bm~1*jh;EhFBrWKxTO2BDFv^+sAU#Q6@ay=N)MN#ipX%C<_YO;+ zpUzf(+UMgB2qX;sJ!I04>aD)j6=1H<+Jb`Fy>opOIJr zF|MM_IsWG1JbLYj))&>OmhfbB$~d49T6*R-H#t+i-a|?{Kc0Hx*|B3ZZ1!Uh@5N80 zwMpaT*!kg5%lmPAzo{ht$MSYI77oTqY>(`r^R#Lv9Q2`@#fB3K)#=&?d{Gg+R*Ua0 zqKH*<0w#J+e*cuFX;?3tea1w4|Dk^e@XCJFAP{#pz_mfgjd!wxYP{)WnSH6Qu&AZc zSD4|Y_p4$Js;`V08r==b9mT~K&9+(yRI`{LyUmB%`$%rYiI+lceMoswHG|Gn!nCd3 zM%^_U=26UdAzE2HY%uZZdG9poTtdI z8azs^$CY!VUPpcTu@(y0LNcL^H5lYa2ZSpR#n!ZReY)|eqB3!Xn`v8x%7EeptF}!4 z?ztUM2qpkNZ$b)ONRYU0easEe{q1m@`~YsHr2Ls~?&#BE)!~b1q{4uZqu`vZBwazXd+*=kZ_1F{xZnt5mXkzU((smx=JH%jY<%3 zPy-UU7sbGxh`XXTA?%%|f^yoETC(kTDp3XEsuC+=d&fj(Co%d@W36cPM}s!fLlVV~ zorOuCr{@*+`y(-DZLLfkJT^`SzSNREAN4+NOs~M?tDBg4KiOlGU$!-J2H=*O!|3mC zPWD{ypRbb-WVMnt6OmgrBgpjV>K{HAPjW0V?RaFayJOEJ&MNRi@*~I7Hm83%e-)-9D!= z4?|?{d$D9GUq0MMz0Q=E{)piGhW?uDj5dMzF#nUz588o&>bU0%>^$h|08Ehzz4x?h zWv#8Pid7D?jIFfhL63JF-|3$I??@?fx|`o|haoq8)?OzodWxYY@z-4*s5Mv*ovXj#dHQKp$w^Qw*rI|25;^ zvk-4B`hBz;=03XKFOJ3-aJe^XlrMQ6LM4M=wZ8|UP5cxR&^z8Akz3=nrO~LlmFVs3 z)2=ZqeYf#WuR1S`7|k=mB(r{FC@y`AZF>!}P$~zdvF14@zk9c{P^!v$n)Dwa4cCg- z3p#bo5*X*wIwjb~DDZFiclW^mjq-NLYGFXZliBi-=fJ*mPq5AR9ePD~hjLD~BiH!& zzR^13U%Eqa#j`oVjtproe%O)ez|W5fsZbm=l0V;9UTS5cL!0DNnh2^%^^qzg=VXk3+;YLwOR{BDd+EE#e15H9WMy2OYJ_o#4Iid)(AacJXod_hw zF{F1r5sfk3Q7Ookv?F|e8BN?ubNas{UOPnphahYi+euO$-h6^uJ@5M5`|H)vw$^}_ zVxLHS++cYburIs%9~Qk~seU^Am+S(%kHqn5)vKqJa#8GS3xBdX3}ve|r?A+USa_y6 zu0L4J{k39Y@cExNl3V}t#+u4Xue+KxgT*h0zsAXGFgK4-JpF6)wyB$KdAL)Z+25u5 zf3Rjq00M!ozNF;j{NsvJV^bo04#qobcf<-;2Ne!i^{?0IYe=mTsX52Re zCcnR*r~lgR9@5rQv>OZk6q~=NT>o<;9FkEw({$ieSN(tVd8}HFm=p;$LM7kH{?DlJ z;VX>eK!uqb5ZZ z#>?2V+)(@PR{38BRH}}mzAqIw47C5fD*RtEm92VeA=|Z>#jc<7*P!)ZtNnkq1>7ZJ zCl{2G4fm#iW&yB=e#B6*Y$|#IM0s!xMDW_Fp;*sljPSoc<$vuKT3b;cX#U44e(3D?yh&9kW+6e{S&gM;aJ4T$MKh4+o~VASO0o=;25FyPyTYg_5*32 zEGtgX%{p-_JmY~JPtTH;^+|0vcSB`9FNG@~Xy z2a09*`8&$&@;#Uh;}7vt9!`o5Pe+k|-=)(0e=v%eJEthIu0Ka1o|)vhZ$Inq=rwesLs1Pj*C*;r10?3bH!|EXEw zYCKwObRm~HoN}{5eChX_k{`O!M}2RCOEd~vfAyZ5-v3l2RJW zvmbYU6-YiEiZXwD7Sk{kXw*RrD`WeY#(*&(z(V7X)6udO`3toYd<7HRNss0 zdENL5?ufCuaQGNm!jx19iEkCiVfhpqRY0@U486+4$GRhBV`Q^E4MrhYU37qF}2zFREeAS1)i{HSU#*N(v8KJ}`V zt&Uy)Fw#!u7SFLrep7^ay~L$=Rh&@^0-Mr)2z?+Q<-)_0F?|?VI5_V_aCHV~$T92*C#3bIuy58wd-IZXruQWuUaI6Qj%QJEDaoa`u(fCSB(6OjVW=lC# zFdPNnV4@Vn>?R+!G^@>hB#~Kmj-qocxOORke-tabEc9MT8fbO1x?R7;74N*Sc4?YD zceX}X;yKW|&eO;3uUTQu{;pP02|MA+v;x=M8nn@hv^Jt+GfFJhZ2Ts$5xg+1<2vt| z?(z${k{-VYr=uBIXIsa$0sFksAVairnwy%rp|LWmk9=2=!80A|W78G%j-RJANX5nH zIVewqxYQYX6K9*rHl(4Q*4>WdVE~@WmFukaIE3dO{qYy_uu;`qE>Rf|Ms{*;%nv`{ zy)2};|IJ69v1vJ?Ls?YEcu%(1{bXl96MkHJu37_Iz>XKCS`tGz9P=QS-9xwYHsa>p zTvb%-?*5pc0ezWoFl&Qm?*%k+^+IT(W&27iW_XhdYB;l7XP-I+h&*7|kf5&Xeb%=- zBxDnS#oeU4sq=WJ!))n!;DewEHoPBnAD1qJFl>nD_@2mw{)d zB2bbGS?6Zf}UFNGe6{PaW8YsVlYaE574b}9>K6IJdxa7>KqLA$zu zLH%6>!mrdyQjrNY@;1YxI>bD(xTS=)4esTJ{~rRn|Vv60~ZIyJn@ z%O|gWx{7*IGwT~nDiGay6+Nk4`Z${BNpq0cVemy_LtT|g!3VGgeVi`&JhWlz@LvG^ z@G<6p_UC&7Ki>VbRZXn`c8)R1X!Dd&8!|gHaV)C zT3YfNRt%2^@Y#aXBn+~_2x^1C5hqgjtg7e)E zk9Rt;lBHj4Own4m;Eh(G-N*zXc;0h`&T&OR!Eqm!M`-}MY>;UE(Drw}{^p#~-v>;H zORj?+-icfV-wJG)HBd5quaWC<#V%+9$^k6tusiO%MU-FVLE{ z*m$f0Cz>ccHHihI1tkxr`hFn|FE0(QP0OHa1ZzI~sm#VW{}{aO`2!Bi~#mYi!(Wt>s<`DT#F>S(IWB#ZS3E2gFVA zZYSwv68N3)LS=uxJRtWqOeP4MmN?Do2sLsQ_aeA^6=ZSr)s5?a#_Zh+PfGRB{CBbA zSaz|ALZ91A3fs8MEJ7o`$c%+gP)`e)-~8Np-#+?2q>5?XC9Pe~MF})bSaWeyY2;aU z`h07`#^kK_?!Nxl)uR}dl#TNc9uh{=c3NQai4rJ*5#EBcvqJo*4tUbzuxzUUG3#cL z;H%~*$c+ttvR;llw?XiVZ0fF@^myW~hnei6{Br%zBabD%UR@!+0FS$E-y>%Z%D=ly zf|@4OukY>;AG)`fUdK?A1%K^vd^quw@KiMiyJ)3ZGJP?;*^)o~0lH*69wLvVgXB~d)dlY!J7IFQ0 zZ-$lzv?!=!hC6J~mW4!*FbFBQl{gO0-MjYx@Q{bO0}5h#My$rtJT-Vf2myu*g%Uqd zR-`S&UcH5YOKqnxIp`shK&U_13dT1xN!amsLeS_ zr&5Oe-QyB|sH@P%vmV63e1RAA85yqbnt$X=Hk)Bl>cxHr32f+KAG~63s)p}&I?D0h zGk3(3LVfSyx=#^ZorFvz`+8|piUhJP&lhf}w~tvv)4fs*((g$`U0Ab;K{EiE2h;nn zQcvcAmQY0HAegQxu4&h2x`WO#GbqwPJfaKCKv82I7NAitYM;)I0X>A*tdBrLN%%|)Zq<;3&L-RYnteU3sCuW_R(yt@Bg2Q28Z9QJIs=(iCZE+WuquZlJ zYVP-SL-QpBEr*UHwJT?CUYVDML80+szz&+u|u znq{Oz@(K$Lx+Ib57w<0Zv=v8z^SAnU)UqISI@mhA-UwNqyLQe4xOQO}I0Z_zb$OEFe!d zda1(e7fC{p_B4EZ&P|+!T0$ezN*TEEvFhH?_BU8H>D!#La5>i zRwyWzq-4Xra9(4GfgPm>L~-$gB@21lRpK44W@U@DwQ4l4Mx%pENL_)=PZ_MAsD~ar zsU7(x-@VZS>UAECS5p6@b?-(@^V>+Ix0~9q={GfOR5D5iQw_^bd^!^fEf=P*+T? zBnf)+5Rxe#P#DJ6IOWW ztH1Z+WK51Nk%)sl0RUh*ZN$P6$iPbOSZr6~upA>2b#bR_x^}#jNlUjcKe}(hxJa;B z4$L?;eoCH?mG zihxbr(?S)NqDR1ODp19I`(m%BQEGAx7{;V42O%^$FF5L0Q=d_<^WC90@9+yg9mE%X9i!iND7Ig zGnoNawD>gL-Qi-@uNo7%4I2q7<-1R zxm>%!z%KRX#7E^`B&_lbg2`gX+Flfa@uc|+ge`gSg-MGLqZ#cxKPY(?8<&#xXQw50 zqN)NR^PHW?sX3qF``JLVW+}Jh!tIvHVdlF1d9zzAZR8ne*@*Q;FGkgOx6NT(aPx%m zujInIi#D36o~omG(XKmZT4-2dvxm2^+wr@o6UU7=GEqYv7*+fRrbft$>FFQsTH-7; z*OO|rC$njDyB~i8$f(-qv{ZSE_6r#l(hFiwRm+c3o3Sqo5<9+)d6m4cXYQ{wjPy8A zr``8(eP0}L>$HZ5gu8k5^)4KQSc}*i_6y;+oQIOGR8j>a9fjJbt z`=JVib5h;$bmIL|!Fjih3{7v3+^*6l;^Rs=?H`ky7X$9bA@mLh*Wt?IE^lc*D4T?m z!qP*&8|kd5b?y#zzY(kP-Vw(`H6$V*q-cG8FYnw?4e+J9i->ZPe$e;OPW~4yD1`UP zACTp{xO?{f#R8ER2lFqgQYA~0T;M~=x4b74rifTKt{0(sB09!`scHcdweFQRbU57C>mTzl9*=6fbzZCE=T< zoim(woM!jaybfKun&W_%G-8=P%E&^`Pc*=nS*!3iFq-5{r&!3iKlz-r&*E-R;MbSC zh`oZdSkPH)qmCPsvrdH_+J_LO3Io*Cn=;0Gvm5()>hhOW&q=iiUZv(dUFIR|?Wh z`c}s%6M&9LebzF+g}ghuLf!RO(R5;=>s#WLiQU0^(O@Ok=jI$ zjvJGe_Nw#XDw2{=(HJ_WN4DuekUA6G5pe2Snto+gW@t3yeiO)yHiDRec9q|lxER^< z>|O^D^VAH2%;3LgFYoB^vU5eKOusC0Ri^@L^cJ+&+kDpcl$D5>HuFvQ)P~~9&?a8i zj`mgaDwbt6v76iJ$@-x5WxzbJHJ zOMtK$ymZ|$ou$0?@RHP9rT%@w?Qed#9bm2#QL73*C+E7K)YF5S>$(iNy0&isGMI<$ z*0n2fTpxm?-!}a^ULjy%CYtx|y1el>H&a>_3H}zsf9O*kQyW_Eh<~tFeAB@e+7Kei z4522;Yg8NE9WJfdk#LX*rfq`gIhZynzx2IftCj-|av#+;l{cT-5=*E?2?}+F@L!q1rY}2Z# zQ|z%bhvGLUJ!9b1`t3;fGrz>`+g~ts}BU#cSH`pA;6-!2lys$VrR0AWv!6Z2JhT!JGQ<6;Ph zXom!WcqBqdZxj?JuWD=#3I6LYfNNI&c-Mm9_^Tk7@M61r7f$Htg&jG8u9X@?uCpQD>25b}ANXWYhR=ZFC3Tz0rU+rx&@i2F-*WMu z$ryRUr2;%Y0r=P4XRUVpDLakY5Sf-L$Pqx2BkD1Xl2*E&$geaGeC(LWZrf_8E0{hh z9GB0%d1&xu+0=e_&^u(F_E|yeEh#SRXG(MjOfm(xcd}UTEJ!J^2cFJz1n4Oaoa0TB zWmj}}ei#44`iCq_yC0t$<9lm3eqKx~U)V3FTud6Ayl`d1q({)sl5!1iy^>@vdPlJN z6vjbyb+tIW;&5fdbE9G7Ab?)9qNB0m%Xv@rbG)u(djaXxErg_`sFy<;GlcNK=q`j* zxlcq&T=(;XME&Z!IUbhbkawo&P;B#S(B@u}fizO*vCPfekMCzh2C#PG(D$^Vv&g?`yWsfSXA=fW7n8NjB7)PRp+9}bF4Ln3b zxs~E`96)KSpmMz7Oah_R%b$_zD~Vqo@5Vc>X^ryG(>am-LYj0*DYav^XP=*6WObf6 zK<-*$)NSB_-N^`-2T$-9={1^p&0By7+Ogq~*=lI<>d)Q8YGBf;xDpm)uK=g5G}S!M zCB=qjnvD6PXHCbn-b%VP3xtZ%Feaa+G1=}m6YsN7C@)$(nk!cllL=g$lecl!u|ouL zm7uGWb0ul+Fx`CA>a95Bq?`GE=td+U^H`G zT#0J_(TvEgLAzmyCLC7E1}8gHQjgB>VOw7dUtN8$FY$HaG6FfJ@F9Z4L;w*%*9H9!by#k#io zZe|5S0y(6)=(0E}E}5UPelI297H@0!L@7bDnLBOAh&tPKoUSmnW&&wtzjRAYt_;mw z_F=m>Ulr0j9la`A`sKV8sA0$6snn?h(T-oVc|%v>WOP#ZwnVcKR1v?^IV|HQ>$I1< zX;=K|Z2D;T;5RuT-7oIX}#jf!Cn$L@niszHbqX()n=oWi{lu3UzhmiOB{-#%7K#C{$wv@VXW8WX1Rn%=R;*DUf9ukA(r3wQzouN1l z%PjrVm6de@?=-Gt1i3k9eQ-xk&IQ`SJrbe&HVG+UJKUqj<6X@6z*JNewX-vPpKg{E zfq{+KX5e{cn{Q{4hKu+c&b|Ovs z)0gd5>)6w6NA%g0hx`DT5FLj;?9o4e-7oRyzl8yeB$CS;BbyuXLE>qE4Hm~Sczmd5 zqM!=9^q&rguFIZAms~H0MPo;{TwN$ZL@q+)eNUnabZ{WJa6nbPtRnb4BQwpNg=YKW zbw|*)Fy*$S+xLqJ7oOsEvH4V`6lwEMUS>iuKnuxVN({t z^ad)@(G>4CGs<UFK+S3iGVLri_bX-u5(i}#lmn&oT1b_R>S$ZAiv75E`t+7#ozYf{qmneiy_ME|Al zk^W7mZv^eK%*EbahHs;*uRtWDfBib&>|2xO_f_GK1m5t=incEI-+NIV%b9kVdL z0WQuQ`I_BU61SAg=yIT~Z5D!k>(O=s$}T1(?<{jU{U5|e#1D*1i2WMOOuRblFWj}A*2NQ>6M!laN92yO_r@pkA*qkAG-suEP-K2T ze*^&!MGe<>x(!%a04~V2JCRURbxWPGh`etXEm$~HGHP}&5uCGGwFlj-8nzWu@IXv# z?sdsfoJ?`f?Na56=s^{DR>}q0(*)ui$FIf3qeVkZW10aIylZOM2l^?F`@S7pG`F+x z-z**0Kczs3o``)bgH|2`863Ss{cqze)mGO+QZ@q@ZXozMjX>s2kFc})Qe_6gf;T8{ z+2@{Ns?)eWJXI&!B$kD@%qkSQhi+746!n3Sc>o6DIpAJ5IrP>pi&*C_!ro!HXEGgA z*aLHms}%cuoM?EM)B z;zADWSBL%~M_0?xKv$DcuU!jg%c2a=J160vH_8ICYi@E3MxD~D9gi1yR{h2CId5bF zC!PYnMhEp(cldU)t~ls&JpJu`P;k3l)h^Qo2rw!AwOKWhsg`R+G@Rn>>?5ki^a{&2srtG+pabbO7kwdev%2lr2T< z$OCY4pm+Su6XaBIg-PsKRJPz>tn*DjXL^2W;=WeEMl5<2>Uo02ymoIO> zX4RsSGV+p{(m}vO(&gHeFix2#5cmP+(@GF+jRks-E6;#Yg)-2{E>HUvS+L;IXzil$ zfxTw++v|{)XL(l*<;(TU+o~70n9@oaCxHuq0sf;7NP&oA;#Enpa56FWCHHy8=bNNp zVXei$oi#F7HO~{Xe#*PsZaTR1^j-Da=aQMvB|LcS20MnV5*3b`NBX|l#!$XOyJ?LZ zIZb!ye<*l3*Zd#sy=PQY>(($TAfO`RwxNK4peRB>=^X?Ck*YMQQR##ty@nzxO79&) zFVdv9gosM-H4sWfLJbgVLJ~r5_G$Z^{eI`1@5lXd?-=i&m>*IbNHXgBLhk(8$BJ)}Ev9_X=82XI&Gp&E z71&CXW^bYWU{L4$cb0{MWQ{uSfNl-I+Y=+#Nf#RzSbcQVWJIy>JEI?Ru^aczR)ftq z$U!LzjG*e_j)6C+mMYmrhppEfu65mouAX)&x)@M}zvg~sI1lP-l&R=t2{zmSdL`$8 z5^PXHgGN`bXSFX6E26kOewLM>?X>l|I5vFNdAw4DQz2NG@r5;bzRC~5D`K%3Bet4r z1tuR?9Ixg7Bo7n~K0nN8&k3{*${fO#3hXmlY#TJzcApJiaKoyMuKv8;a@dsmB+@I? z&ijs_M5Gj7nOehka`@-_FEj~k=eoV%2sSdeN=>9Hg-FUAMdkea2$4Dcv(L_ z2#N!pgR25DOQrr}Ux6bu{39W_pPVHQ<#e+U2w~@p^WcJ+SzYE7wP-f0f<_m7HRxg$ z;8V`%lxv3VfI?Oi>v(a|Q3QxVjre&o+{|Y~MSGlQxhk=t(=WC_{C$?6UNvt*1J@CT zm;7~>(HW;~S@KbS^Y}Rt-rVJPW~C`>WTH*Nf~>dIy=hXyB^p4tm|XYML)v3SW_3}s z6khwbL1eMo1=*`!5C1l+Grgv5XZ8f6LeGr$alg-JAHNxv@l}MJ`EmMARihQ&p_TZ^ zRXD*%@|v^ZbMrj-^}tWwpK@91esShEh9P!(6*wg8)(ZiOC{ktP@x;x8aX9?;sxztO zNoRs!<-O4bgQSt}9KQEOg4qItii6|$#@g`o#0wbK-gKh}i9u$0XooUqCloq)+7L$`^#QDOzSUFlBT zPNP-U2`V&xmDhL1tZ%$FJ?Nj=+hRgEB7H5YRgvE;%(7cm43tnwZ4J0viVFv~D6oTx zlwBqmKjqrF9xi?i@H)Dp&Vd=FO^54;t)Hly4WhQ{GZ0a`REtNJReAbBv?UpxANAx8 zooLn3t&Qt1`W`}`^dC#brb|i?kq#YBrb*WpeDi==x)cSK{z{fv-R=9axTE)q?ZvPc z@reTPoHnIeyp%XwBp@iv-)q^GH4BiMI+p7FqoqD15Q=d3mIaMIZBd9k-6tMGmL2S| z_9^;$n1m9UwzNWm?vgB7J=V`|iAPgAO9+h=jTsD%h-rz6$rsE&AX}ES8P+YyZP;~> z8pQ|?kJe9^KI6T3O8D9Y>8qtj(zQ6BDiB7sDl**h^Rk`u=Vfhw@WXC%;77T9lO&fq zlOzeD1hk~2YTpv&S^=?}`6vxD_!zaSIm>55o?8GV5fr+I-HY&a9VxB!{e=`@gQ{&h zUVA(OAHo?9^wxjP7-mS~wUD@C$t4|=>}7G;+n<8R&S(0qChic`VpLbMeAaV1-GJEH z+_?iwB*64TfXhKOzF-Itl)^e2{T^U;aPkwzD%`KelT%P!Z7EE_Jvz(UvcMpcZmux3 zdb-oL=8RKM57w~l>{)C1vs#yymTf7`s1iRjXRrA|h;4tjM?*tp3sw~Gld6eOhOO+j z)-V}AnDF>fg-Hzoy)u*f>7rgY$% zKGRHA(wpwuSF;~W;1>m|Mg_x&eo*|*;Om=;bp1>Uv3=3|L6p)v#0oXgo$%q^co&^H zYVOf7W_7H}<8_9?_js=q!Fb|)o9vxgJJauyyZ4pT6_$kC2DJ5E`<=%S!w@Wv9~-Ud zh|sZKQj^|+Qv?)&KH1qKJCcENoZk7;l&d$7QEeV_&MQ6zeyNsivH0qFpT0KHg%lvh zwJy=t@JPAt{xo*R&LO;kg_Vyq4{MbkKdxLR* zZEe^g%~l;TcS`O{H0LJ0^O4$*!ep?SewUy4pbBu$^xbKfkhd%k2EG^19b&}N-Bp@m zVYFNOvUv5GT(E5n*)g+`Awo{8rvE~bA3kNd$=)@I?>)jB=lX7ru_`puA3XBxH6Y`WEv$v!R1%mk3XXr28PTkk?#ppYHT?bY;w?iucHPut+&NMkvCUvj)14C$C_?Y(})ZyGLY-$hKqOHHEdjk_-b&ZP#1waK$m0c6!R|n}tt}ClPHo{Ks~_ z!V6n){SIcbAB)e6C*RgGMeJJCi{~}Tvr9;E4-dEtylAe}pd+QFC*KqkU(aUy8mNqF zn>FE@BUA$4J|ey$fcAn6exo4}GulZfDD(TPkcme)ht_@vu3(;FA#=g`pcx|tLUmhZ z<3656xLq(7w%?v)*GCi}0`IVb8e#O<6NsEE(bSb1SjtZ0ryE-PS%1kV>|1;YbX+;; zyH728HhB-Jj_v#0wIyEnjm^Z~8=&~Iid166=D5}n6>J)ID498|dHucsH{5Boz@2!d zS~--X8r%e=W~gV2?6%C9o`@EiwE>?D>$INSO;=c*PdQQ<5H)IZL9$Vh7~b_=6&gZO zx{G6AW$w=0ku18^u;XguqoB+7h@TY35=~0pipaA)F^n2r3X~hTn;nhi>;TnBzoEltOLw8+@6QQ4i8?e)LYkf z3xU&2b*ObzElm?14-r;CHL;5;i8}|ooT5N^>2Bs842M?vy1Fby zSw|ajvJ^XUa{XYj+MY5m?)Bn}L%I3Mkh)I)xC}L+qW*CG7~AyDXmzgM%4VHq)+W~b zu2t0zr#$}rjnNvgeZ~$u#t9$!ddSNS*iXN*din?5eq^gc9e_F9vAo$%yS({pT${2$ zKVT~N6LZQeXkd+-ssJG?L~$6L>DbJ$nd^wrMzJ9%3RH1Zv;G&_-{jS*gPqjTw z$lC_2!EA3W1t}?fjyh>5)>#EMt#NGnvN!LydfeOc7aheBGFb!pfUdD#@Mp $HXa zVx4p?-Pk<&4aTeq8y;={UME)fR?J0cz0woDP^aP^eEck>`q;=Hq&dh#a&#aSUyl2@ zDVj%?@(+%+>-^j+CQYDs9}Ixl1L5!8N6$Kl6l<#cskfPuAVmWGbIK>X`$I_sFYxE4 z?!I9$@jR@o3~-VK&69o?v)~^LK3RU)8NqUiWsSNeqyrb)e;Y|CK(AQ$=MN)#Co{lZ zD2UU{rrVtj7*`x3oewG9=V{f*W2bJfe@v!;g)gV73!RP~sYh^K*t}EB1oLHR0|cUc z+Y6KJHr;|d;STh{m8Catyd|kB-Uxj8DHEpCrq4eCS{(VZ?@j+>|4{wA>7?jd~ z4;Hk%A3Qq1n%W?5*+!5R#<9(k6>%je)0z$L$a0zTn}VWs9O-nXo8&>bD0LsbxtT_D%;Bj*2Q?E&UpTP%eN3+ot!a?l2W zTyFz$HoEH9%rzX{1~fyqrLkr{Ur;#Zea+RCd(OR2xoO3-(7ML5CHYEY-~%&K_&{pj zdNG6ub|)k)Crm=Kq5kHL9+wyEpE?oCZV&fPrOY`uM4rO0+@0bmCp8Ou0p(UFeU>(q z^HfB)s!i$k4lx;V%1{+w*(+yzX2~N1Lia7~-89n{%EMRk%+_p8@44{YM3pUT>~-37 zDw-1xY@DO3D>sEEuf3^uWRl9PQh>62=m#d^3nI?+u*fZmrHk*`1`BYER4_TKojlWd zBHc<&fyjTf-s|Il#9cOfA}VjB#?qLKyNWn@l+&gs@awU59Gs-?w^}eGvZLy3;Sx6W zY+P`BYPEz^Ib5J{u+ViiP7!N66R_kElnX6PI;=74!Hz2tum!ZscKx$UM9pC_Z8+9| z`%H@PMZqS_wPa4k>vHh*l5I-}PSFv9!`st`@4PG-od2R1le3T(0#z6ZH0QEcV}XxO z5)chy`sxHX?RCzpbAMY(YD?a;oRfxm~@u-$tIOYHNyRq}`LaoFrNV z2=GkL@0w*C@ZPSWWMBq=AXix$SP0)T!+-Eaum$wsrEL{t5q%i0AOZb)vjV?o`+(Z& zlv>wT!07_qjah#&N8m?0RFsg8A&dogi_%|^(bHs1Hgz#OcrOQYcTSHyU}f$>c}5te z#c)>Wo?q%Ij2B{{YVK8H=ErVx2vUdDlP`DdPKd)c$OPuSsA74^8$xfxg0;Tb)WNZ9 zcz+#>w2;-(T-BXJRS$(YmIjgZ8(Md2j2mQ=PuAI>kskMOc(z0GTymt%0x{4bamJ)| ziqtnKkv}uy?I=;dhKikUZ*qKp%ziFKmS&n~SbpHR!7MqT-Yhwss$HI7;uKU&?0uZ>>|O6}CuXX7%CORv7sO4SOMTS}V7SmT4|iLO~F)i!|K z#j@Mf8X?t4U-6L|&U-4Py&P;xi4rXwG6wbzBg}aj zi?!U&>0xz)6I$8cHG#Wk6_#0WoBSkT;4`{khlSf#u^O<^f(C|Gqs2z>AS-{+h*llz z)p2Eoqw_ZY;ZE?k&haw@-nlAacHiOExrnrxyzRHg8U>ck!|ajUDmWQo$Gy^~cp=FG zg`a?bh{CV8&KQtF0^ejW3?OgQi5vRLg;CQ5Fvv#_%{m$3Tr-8=9|gfxG*B&uohF(B4}b!j38o z_7TSwb+r6(_gjjIB5}=!u5eMdT7wlFbW_Ix^t-&1FcUE|1;x$F8TpD>fbgtp+tX}Q znkgSl1MsC&B|WO?0CCpY*V*CIv@Y!vP~ zYUkw$>t@s!h|6_mpS?bZpKxN7l^^OL?J&;7ka4GnKBXlz#+CC`4b8Kj8$2zM3-Le@uUz7k%f?yL zCDZQNygXdNlpe}^ZBL+@L{HKZYk-sj;C?(Ssm zg9J-ToesbS%dbJJqpt0Vo*WkzSNPEF5|FEYDL6kD;z!p!QjY`j%M=58p0+Wy+ok1+ zf1~Up9l4ECy;A`l?9_k}XCPKqowWd`uH-S&aWp?*vS5~?etK)M3XGMNfFe$N)C*dYvb zqagrL?Ys_k&#CE|N=iWljIc)PpbPiHftPIaWW4{7vk?19i#b=I8lxq?FU1hppFcUX zw>>qV_taxw_@6o;T~mw89{U{+C*Trx7||AgZG53oH#KYFyLcdHf)3W?tb0sQ+yy83O$+2 z+%wbe+Ub#I?RU=3H`R18&!xHQ9PL!izjbJOK>8|P2&jRjh_Tp`L!dCDhRf-2nc;rQ zXE1cuk3msj#mo1mPE1B0)yZlRK*gka<-`sAVKKkP+s*yKV(6x|@5gU|U1zJURayLp z9r4F*Kq)jSOUc-Hh4W|Cih94vO3=z$_L#$NV?N1|pJbcQfQFAmD8Y$sW)^t`wn9gH z5t`^&h+v|6D%fm}kKg-vV!zGdXR-6>@YXK(hZ@Bi`44Mi5K0xqDShAcFVgcPGskk( zghDT?UpjmJ@S3_QC!nNJEBS*oW*O>5aBGUghkZZ07jOu(f=)vNVvk=VF+nz4>ci`T zP2Tr4RTX}N??D$zTISb&yoigbMab*u9UhqDj~5M_@q@5x zN+3!;ORgr9dR3ZzA}t0SrKhYH((6uud9~?%V*A3d5e;=Nq7o_%^qc;pH!DbH;#@&h zVA=f0)c_#ZaV3#v?22;4dXgW@TeUiE= zo+;!L+%5CH?cl}iBLQ~Outz_$X=V4Ej*7%oJ1_Q&W*uy*J{SAo65`3% zn5>`c`oM<}aLBhI-h}!A67nIQuVWpLKbhsdGq`7(sX1=+-1pESDCFmk%R3&bs_(Dt z;c!HjfZaN9v?uMprurxsqDox-QDv3vb*iK(o4vg;4(Y0S{xRMqDZsIj-%2_Q8lO=d zWG)*25Sjn_q)0{VYo7S_T~KsDr878nGbLOc3@*IpW1S7!D}R}r`=fpNmu2sc_q~VQ zSVp)uf^NfELiOc~_8qVZ_~(;#k4LOCzlx;V%B@Xmg5pE<7sii{=|>NoA9e-vm( zwEkjn4cKTGZTLRAmpUoUcI-wfMGS5;FYY1G*>dzFm+F}tiA*Djkhu1Pz0o_w0O ze{JR=yFR&^!wrcVNm*B{Oc|$;D8~-|<`2wsNEWObh>*t1%@ig5#>}8mSO{VNt|J_S zw})eh(Z^i%kOM_69Gsn;KM`_FkCZu)8JDqKT|V>)?gUT;R@-}quMNZgTL|(US^eM^ zOQm_-EiS48>9u2Tf&-2JQqdpK9lNrxxp^K5%01Cxo=6372PaqrGo#_S0j$kWb?=}R ztZEQtFNU8V(+_9ygR7?Y2*89=`bBs`bpm3ttI};4nr3fd+v`k&#x@1RhMDV?=60ix zmy?%uFx>cKM^Z-9s+xR?1NY$u%r-!cYjvRc*N2^lU(i;~>FT~j-`Mn$YgJQ>{iH$? zJ?qofDKoclSgqq?;QB#yZ|zcXQ&fm>{cfoeNjAP;TxxArN=ufmVvkgSY^4vYLB2)NwmvaBE{^b6x(Kf-+>iKJbQ ztpNWfPsa>CF5tXqEqoIZaWjLazk2#NZ2OJmT7@dXb9bs<93JiDUR#1;>y}GsVL?+% zIsug-Z7OX{1U-RuNBO*F#_>v1$P~cJ__lDzv_*&!w?B^VhGi)hYV~1I>bK#;AoTUu zv7bUoofk50uDn&!oAK^^g-v&!$&WQ$ARQ-1UfSj`oKq7o+fXMjL58{|LB(|A^pDaR z6~XmJX^3lEu^B=u`Qb{oVt3;ZL!5YRo2dTfiLY)zTCz%}$HGs4NI>NqEAwHev4}&D z`qwa@oKJX}A7F-y7CjrJ6G>$A6004)zdC8f;%#;{xYD8RVeS06oYG{rc}&ww))4RQOq<1H*E>z4yY4eBH;)A~7fC5& z5W1^PES+uUW^5-se~rjfMHxqqQZ%*y$f!b&j8?FUf1`{})qp+kaA4h#E)ZzU?3931 zub1$z#SJ^r;sNZE39O_a{@m<;*|H6O(0e&0zuBGH6Xbv=#u8W8A#mY3vXr^ zFD?k~8#|mLsW?ktGOUvaEpj;Da50@xXJ`klm=@wm

    icF;W`oYizAyyp zkiPr5@yTJ(Wny;TmX;)aN4VOjI<7nBO6{|6s)s$cR8F$hmycNyYuBi)?UAE<2k+r; zID4$Yg#3DN2@H`jdwKzwl$c|R*1GFne7wKT&I4F?R+71Rbb^mxT3#28aVutNYQ)kV2?K2OHY!gWC86Xcg6Q)&h#=GuaasX2VqwVL84{a;SlJq zw7{~~uS<>XJag|>YkH7K5qsp~XvbNxndl?-CFs-H3KBmiIH|1~uFmOxm)XGFSI6Qp z9Rbk_*q;2^L89Ca_FO5m@qf}kA<~<^I-mhhz15hY^alKTNc6eJH`39U6N3^3bT{}U z`PggHM|GJZUt}AO5-nPAbInK2dzU*5idD$5viI<$+X97>E_&rmHPtoTJ+EiVL&2DWD{HrD+%^*$^I zBPtYU=kBsE=_nYg9ZE@yWM-Ey3yY@u^LIH5_zyzo?F3>p04;;a%B7M`Vwco{mW27%hEtvOFSv3S0c^J#0!a zX>O1h<+^L^?o~x%Zz4THAKS-NJ1mrP}^UJ|n?7us(e5sBmnF|%1X(D+qAV=a%Uyv8TJktO|hG z#5-8UyLO5t|F2YEndnE(X*(Qt=5fe|(n)5So@;J4;_)*E67q)*C z59c_$gZLE!jrIH}ys1*s(=x&dn!9mf@|w;dPCW*^J>6jqj|hX8GmGDy)pXwHe{7|yxVJg&?MiYvjYEo zq7A`uNeVX`?qfS6?sueQmT8HbQhrEo<}9 z)DMlGs$I*Bc8P=j0JUt_J`wOVuY45Mqp&_Sq;N6T3A48EssdY)z1X}mj-5GoWt3%R zi*%iZ5P8fug&`2Kor1Gv)&<~9bfOZ()=5b1T4X-vVpf-_KWQ_sxS|PDQCAgB3T(w` zLj9T&-|~JmTS4%NH)_P+47f)E#|*ysIWTNalAHulzV41iVH1(2E0*5%=GDpBmQ=sv z`P2T{1iq5KMhxkRXn^i=)C6ccWovn+|C`BeXzb^Ch@osy!Z^@>Rs#gKH1+Tc{`!6| z*E`z{WtRuEFkcpusLP~~g#9Fzbhaxx%w{v>iM|E+v`%yEY*7Dv%iUh}FowQons&0_ z=u9_oeZEq>RdS5a+e3@J`7PI2fyN3Pc?#(kW185euamx8oc443TPUokET`lGc)H@o z>NUqMX<0`6+=8A!kQ@}D!25t40g}8UA)q(mhLY=^rcQ>H4R^S|k=WLE#Ocx*!P9L* zc7~V3w|Y3suPTaVLduSwStAp2Xr_M|?ZYLwQ3h>k{o{7(-VYF0w=%_()rb|n1{-yV z>nk4P*R6lG*gbjroxyu=I(X}ZYsuFh{&ILG$fiVO?)z-gp*QKHyf+e+Y4MR0qq_D zRIZ>n84yC|k>Bd~YXED?W!;{RexW*`@Jcn+pkA?Z^(>h%+S~oPeFAQH7u1b3)N5nU8e<&hk+F=-a?9KwsjLOxG-gRQi8HFDO8tOY#QvXryH;8s+n5Z z-CU+1{slaoBrL|oayblbt@uXaZ_)6wn;p!hvV8@c{)gSA5+p7Py4nZr1FfyyH5jmr^=rU zl6t}@J)W(CcjaA=>I}Kyl@cwM7ZrTFvo^mWk-S&YL4Im2n`tC0+h9}7 zJk|=iI~0`UOXC;M8RnXd2y!NBvYcmoXSqXp?4N^fbQ&h2r5#8H34wd79bi>{uS zoHYLk`c4k$YzU596*?a*?6Z$g2p%{)rmR1tD0zZ+pU){5qgHG?T(c0$?xf6l!RO|C znb&8fS1CI%!5M_IKZ9Alu(YMEqG!o@`!d}-gSq>yMglhvI7(;;64WR=VJ!khev~s+ zUquuYc(9phh!RRjJTVK~wf$mr=QKJMkdZ%sX1b4Y8*u?o*_cp1d4uRwcy}(Fsp0)@ zHuBrF0~rIz)i1E!D^<|NRIUO@d!J#@~i-~E|wP6+Z1n5)(Fc2OR= z5{U?pg2vk-Zgo8AHd!r?$;MMwM|oVL9>#zKS;aZDb?w<}zot?uZH{eLe;kA9OUlny z3;S-#6QoFtHwx!3k_mG|{y9%mc7YX1-O!*6{G6vvR(W1o8ZgLE!1nZTqAJ|+?V4`m zu`vh2J+;S?`5Tqp?I9B;UF{09Y*t(o+d$+^fXCGGlN72`l51lYq(xHbQVF6+j%tqo zX6M@yAqnQcykNp>b&Tn|?RvvM8nkNS*$^33gSptISyBqdQ9JKFTmH1_FB*>{pnF9l znz{NuqSjokK6pWRa~Bft5$-Ugx4k;-Sb~nM$t3>*Lv7k^H2P21HjEvBSg$m*2|h7&EOuhK}`5i_IQ1T_te{!In5dcVvAe=@k_9^c{NAHkmIVNy06 zC;9wCu44wPstgo75Pf;QF!_#<%(ndve_K}R8pC_WEjk>lhd~d*J_Plj-K(GHAGb8& z`L2S$!n~XuQzo)1fDc?c00z0a_tN-f19`}nkuEeoQN1TtwRo^xX>5H=JQ3QAy{c&0 zG(Q)M9>V&B=iE(O&n96P{3-LFpRHjC?5RD|K2xNbOLHuT)2N!d^K~W{psp3BM~&uW zZ5R6m9|s?Ab?5owRp!~fg>0!DEB8|F$z!So99(eKK`VoZ727$1Xr}b9BImU$ZfVy6 zPL`pfPqDP4K{J#}wR!mL@L{!lHL{-=qfjvxH zOvTT76L0D!vWv_6e_8epL`a7wSMm!pohw|RWBIslY>88y=S}k(F?ys3bbHV708&!; zqsFKc2=abD%ZY60Euu#KpLV;NCTJ`-^IR->T!R6f_cyTPaV4_C*9LGpdI# zBDR=-K3xqg>$_4b($kx%MZ$=bc=#XkH2kif72Td0Zqtj1!MyQAFMj`~ZPws5>lS{O zhw*lBw^8LyC(d(|i66SfQf>W3jc|`v3mhu<8j``^yShIC^Za9D#`~t3W97!G=Vf4s zQ{a!tCncyw?cgXKpnKm=ZEA^J`>UySbwEK1yd%EVGW^$b&gIgeefv@tgRoe9FsXma z%_~*!7O2)P9PKD@&Y9uvV9W(^GZtYTv-p+O^^;O76_UO0=W|pw5?7FMGfvoV|8UK_ zF>6{%V-N2XznWU%#;0)j1Lyv_&S0h%ec8kwzR(IYA06WWHFWB_#6au&vZsQ~#e-v0 zK||wLNT!9ygQ(v9@)N_Z*BrveSw3&Y#y3y3WU{Wyv;eWATUriz`{swEthLZ653xkN z6xPXMH>?qfa0>6g;KmvLW%?5Qc{~lFpaja(CcfivmVRq)dYJGQHM_NBxd^g@Ob!x{kaB6|6ZlW*x}rG zVcs^zR+65?i6Q&&ONzhwe=CEh27Ct`$xUR20 zY&m*wn5lbm)GN}WD$stK6h(0Y#au3=7{zufa>?DZvo9hJb#nr4ut~|+$1q-S^IC$1 z3-@m>5y`Vwt?6z6g(}bnQKmmtGg}y*B!dd%WenbBlJ@Po;l6V)uDZEfEr<&jtvreG zIF!ym9ShqXz`nufsaUwk*;ujtEi;*FX{qOSF`zHMBFz|S2epBo@3 zx4+fMOq%EN(CtkIPV#foHfSlCHRuHC%Wht5skbppM^*1C zn|7HmuFXA140uHYKC7|Hv94T}SJ7;nYpcfJ)cKJ252*in^-(1xB#`6DRq@`GB5Z&C zWIz!{n)PYyR8b+Myj1xC@btxd&!~5Nk_&u6clt~ZA|(3pZ@=AB4?T_s_VsH@^nQaX z&wMVcZ?ab$|E?ot`QZ>}I*__ln|Add;opA*I)3*PCJ8AiwR`dwQ@3O)NL^lYJw9#X zMZ0&l_Ld3rKg0DeyJzF4-m2yzaW484ZSo!Jy0*|nNA>K*xaxZ*D}O{%{^5xHTal2` zI_+t+rsV5$r!SUvQkS6JF5Ni8yW9|N_JqlQe!#z=yfpszThX=!Bh=W833V-?nXzn; zM8$GMf=&A$0`bqQ|J~ZZ(l7M?6{=Obz|DfF-Nz#7UmRu#zHu7ot6kq^1nu3|^ZwVz z{WA?6>#5XYgAwk@(Vj7=|7w^BNj6rb*r^JCAn@wlqBZuy@C?uUYQG=IO#j1s{Wqrh zgI)iXeSj4xZa5q14dB|#Nl@)qRy^gIO!mlUzG_+<^?wvA{F$k8)tIPEI+_{#jWJNxBL>`}F8Gp@p2`;rtdtl`)}NwK9_-&Xs+gv}r26Mu$?`xgC~p1WvUo1(v# zSN!?WUSN9D46UG4KfAB_Kj*i*EI0O}$B+;pn{-=ppy zfIN9ZLzNFLCMNv|m7P8FyXJPO^;l5o+<&t-E6%~JE?fuW2{w_)k ze<$Py;OpLR{}E$<0p!wiD%zA?jEv6xGgJN@#((`S_cN6q#)sEl68doi z2)zHdfd1ox-XxVC<^D`x0{ zOC71SF@({-Q1}nr`*##7pSu{R+cVhr^wxi;jX?vIHlG)01-bs`L;p;G|39*Hs5?s! z7TF+hEAPeM*xIo`iA+4Sp`4H#+$K%z{!Fy{1|b34=ay|&{>mmCwMx`U))xqEHSI69 zX$UWvUUB?)3Y?;C(57;x^X#t*;s1WO((f8Z^%dJ|Z~wtl|CVlgv%l*GjesHIe~_nt z{~Ln{Ds7*>vc2_x!SVkK@ZXU1zZm{I^ZhSN{C|=q5_)J!cx>voZ^|=Gt`H76-w}fG z7;&mhUwr^xSa@4QM<<1Nh)ETbhWt12uf%+{v@j$v6uo@4^)XwB6EgP^^K$<5nRoQw zTgW@?k}gS=W=(zD^KIGOZD;#Z#nN6v&n|Ge{rJs&&aGPmGgVonk;r!k{C@cV{Koy` zt(MDh&cI%&o!c&YAZUZ%@3>O;yVwm2n+X7~&C8~Pog+tL6*s7T?!hm5fJKaTRMu@g zPAH&->D;;F2GC1^+gGf6r`gqdO!qZ1<3((oT+1pxpEKF-?)&%&bO!*K5l;2XV}{+4 zzN6iWUlW=S@Jc->PBzTfdl&)zbR-^k258+@-(r&xwUug#4!A1)?erv~%0eJh!r7e` z6U`3QW;^R!=zfSwG1e7M#dz8)yqFWujU^y_ZSp+*PWjfH(Sll8!qTl&|u67{vR zWTh0Wk_Iv*^fJ1LLG)4m8S>^IwBBeMjBPT?Jy22OQl@{GuwpVbd~RwWDF@+8dfv6Q z;aRj~@I|euUwqrUcVtVXvSPNVg8oxR&~T3Y-i#eYy5lG*>B`+)%O)2KyZhtk6sBI9 z4O5Qyhjj!9TeD5FkEqj!Wi;6&T{tiTQ7V6s>HoM2WxKEUkcO~HaL476zTM`(k&vFj zD}ZoeiN?ym%;2K0eLUtmi$>a1U`iiH<)uY%GaY%=)u$M)lsO(1gu z4$sXM_d|DO!qLUr;@8=)r_LFW3Weh~#X&aTaTW=4h3|_gItVuzw?{eFmF5CwSz?>~ z38R+XBz8Wj*q3lz=F4OrBffF11*V+_YpO zX!$u!StXQNe8}{Qhsv!65LE#lbl}X@WIYt)cqgV6olY;MvQtOf+l$P_xTHV?0b|G~ z{-&R^zf8kTLPG+(1{1COzGxkeB^|ak_K&IRFK?f9rEixtXc`5yMxQS;wghjvFuB&g zBtX4BX<<^DxP{I1p{^G-dXx0WZ6K;dvkwFoTs;$ z6RhK7PU35=5`sh~RFFcoZLm6}^)n{55@r^j34F-6Fl}~M_?OF9>bPt&Yj1<3MUa}N z)}lmIt+19-5Os#&SdEoBzh^u&=`Yt&UO(njl`gV=&D( z<`LGmE~&|Ro0#;yuU5uAnLcx?4NsdiB|3IA#sJzX#89!dB7iWjtal$VtEkRq`s3M? zR%B5coFCTJGf+dmy{hf}NkK?YwgYlcwRj-l3E#o)6hKwKMprvoA;>ngO2n$;v4)T7 zSfn7h6cXh-XgD}gK}+pqs}#yWRA+&!C^SraFWjI*6Eq5nz-K$q)EOE^Y(eAQ$8jW{ zM+9oM{9WTl-IR2)f8Wy39&H)ks|93xArZt0#3X$^95|yjYp53eH1X_ZOD1r;Cr;|d z!$oP+CbErCx;b?msRwh^DGA|>&oTUGwKBj4XLxo9jF&mG9<7pmZa}%7KB&+8tSuVr z_!cgWCxmXic_t?M^{oK_j#Wpdi2#N+wuV)r{55^^u@S?pVmUlX!hK6Lgdo?QE$|q z|7@cQf$+uuDbON%QLKtla5-sqD@n@esa<^mlm%q1P zg%SdPXa@I?lFup63QuK39wk5im1b0e6Op=-oqD#X8VI-4Q6N}5XQO_`(eBX`Wsg=e zT??IO8&9JnFNrezXDcg|4WLK0GO~(vH7LIcH{Z*!3|y#x6)LvdX?F9u(n6+1R|q3I z&#YAg7jRh_P$f!K;Es*nr&>okG|E`b%T z)^4YJYuZ&-JOKJ%Xc@LJt7bX8V>Nt@eRzoDGsX|>>R_67XMFoQPa^EvGMH;(^R#?^ z?d4{VProU-V!^K|S5Cjh@+@+K(ir6CTlWEAMp0V{3SlA-#(GaF3eh;??363 zAIhamSH$`S6L^)=y-* zBO4GskmK6~U7`QstfVj{H=DTzelrgZ2@KQgC((!)o3*SCatf*NHH&(g`uGepRh0A$ z#>xxN>z^yqBUKLvVH8?O^upZPc@8@z7B|M>_K~;<=APL;dhhR3lvi^&kuaW<(9o9g zG>6X+{)-rD<5;txa6OLYw7m+=_Rrd|P;A26&EgQ7H1EN0>3b7(!aXG&a)TepX1{nR}tAer+i0{&4$_>Ak_=H&>ndPDT`L1ccV!b?mw66+%?L;;@ z0f|YCIr#emQu{86H_!IiQB!>MNuSOhV6s$ZXq=?8VrePVc0@0CF8${NOLIEni~-eG z6lZC{D(N|9-csb@=-VO9w7hxZLt5fYYfx%sg%&x?z)rrZ55|?zjn5A^c#)ns&u{aw z$Brs1T+EFY*tsdd4eZ-{-4HkIuTRqAveVB6Jrflva<%nwD1cSz1`RsrS(-Bu_?e3P z_&Xco5GM3t%lzr-JOak8;PTnr8ipg@^iyJU*)xVB8O`R%IrHcGiHFgRX}ViNRxayX z^yyox+o+!+$LwN_WEYX$TWaHXo>8@2yA<8Np4(D)H2!GesV5 zSBbN9|3zay!%?i%!iW}+_bKIi!`#ppCd#J~>zTPHt8N?1!FO5VFe5}0n10v`OStqz zQ3%Evr`v_|HU=bpgT1ERD*{y8D=BE(f`F@+p#wXxdX|G49j-MVX{PR{!POo^L8#Y{bJyfceYe1N&UzUB2yk+Klvjtbb8pw=z0 zcnZ8sxwLoD?`0S5diHio-IEb0cwA4@wUk3|OBeNEGFSIy)|mm(_cTHz&_!4GV}58~L)znOI?Fw394 zw#WX>M$o~J`eJO0$>3pOmQy`fm)ZvlQ}eAP+=;7InC=4>A_lDrGaFxH=U~xt6RT`K z#Gn55Qs4}Rui~2hCOGtE9Amx()raFZ0i~PN>Kk zgC|N0f{LJ%7tB$uFh5qaoGL5pg&#o~))wKVM&mW`%MQ1&B~C01&%inF zHLS%}^UGJ6dzw;T2BSN{@_&12l3&oxgr>}HAKVbJS-74~?H!S9`}mFgf%5};s3^=7 zpeF!_1yF~L*A6oW?V!^64j|5xoNVjxPm{%Q$6byB}dU>D%#a_NNd zjuQ`4dnGiK9KX^^Sc&FOq-P;DTCzOcl^k{MQM`in*;x;)(r{1i}*4h;+ z|A0Es&4V9){`E+MhQQ1&(evqU&OK%}vA9+*%(Xhc8ui~T!{@m-&Kgyk-f>3O#}{im zU;8vYE_&_1i9W{^rT6q`W);gR<#Z{Rx6lR=nY|1ix~{0?c(RN-?SKq;2tie?nJ>?s zjLCPm7jVW&Fx~a9%!IY~sE)^i;Cnijx$UM_2`;-QsP^8yDKpEjreT{BJ(;QEz-^eW8HrT6AU;j) ze#uJw@q$IPJnN1~1?6UtL?}({;H1$;*-BzhT$5VyysVD%ak>n|1_!g~ih@2rwxs1j z?EtTyJt9HYB9qRqBVnoM+g|5-3W;3aa{p<$<^EMu7aIsDsFr&mX1LvkrC+TCUD;G= zPaSyiz-Ww&Jk4f>aH3uM5F7W?>2^WtfwswgSHEJHm#7z!MmuT6`;g6mv~%b8IqmZS z=z^xmbQ$#(MRB9;fLmjZs3`IM;GNKC@!OZFy&VvY_1{!nU8${$-&-*f$b+%|mUKrM zGK&7w4WQHtOW6Oz-dhIM`7L{cfk4m%ciXtT2X}Y(;O@aKxVuBJph1JX1b26L*NtqP z*?Fh#y=UGz`Jb7p`8ZY2ho?5wlD5_T>t6k6e|9~k0<$m4^Y&OwL3#V>JnO=$t@Gf=uq&{&phZhuZ+(PEKMr>|C4vj34 zx5A5M6ck9$2qJTGsXeswx9z37&?cG03%?lXaHZ<6LfT?f6fR)S6)NBzXj|a`XJrtG z6LfLRa1|#;W^4;qR2$8Y4eL%ht{oo|>Vw_!Ng@t_easc|>8ZxDQF7m}ITQQ2PnT*X z<|}j#8tuRibI0sY70OC`B3^gJyNeA*kCxY>WS>^ovjWGRen+3w@c#%enZc?`3wD?( zDoR5QQ+$Cu{)6oV99L-9Cy>T)k(}4vQs}+>fw3+!Rd4s7C~_zM3*!8%=?y0ZAL@<` zoty)GKZ7P@##een+Y=U530 zw}EqJBAobrmHd!JjQ7H4HC<5Kaw8!35U?tKJ~~TGOdMGSdGlVI|7Q%t^G`mg^CK0+ zf8sabn;_($d{9^PFwK8*3iBmlXZ8KT(43TvOd?o~=k%fG+k~OG&NX09h$qn0kM}gNdElo}Ax86Oby^xQ1-#5I}Z7MBbM|+@Muwa#mKAAAToAPBE53zP%_ClGpoV zI|kvn9EIHTIBtm$UAl=(|0W;*zr?owPoWQ_?QJrx#^;J1IT@arP}-LUiphRcHB{j) zs69ERNvIMIyHzU6YInHS(H_2RttmAlf%Xsgqs#-~PW3P9<6JvY_Zqn&@(+8d2E$U6 zS-ZNKib}si>_j8}f|`ed%ZXlp-Ip$;+I|V?&@NkW&0~Xwf|g~n+vxCt+6|;?<2L~d z;nj?qnBaQSIdFuL5!{4-;G0^R`dr4hwxJK*^-|AIG%Ei-?cp+J(K!E;m<)) zmhkJ(K3!+jhawzTV{@{;c|WV~aTi(=D+qS*ilt-yMKsvJqQPz(nXIS}$x6O?i2^w% z3g#615;B}JB6w-`l90eFo#cZgsNE5jM2~-OO$Ft^X?+8#cq`H7Lx%CvNJ{z{ynFxet^|DqiHUJn$ayu@qv-E@ETBr`!NXeBF{9>E zfqANHakVH<(&XasBTR5&sM-uzjVlt2Y{!{ZfRDEl#Q!4bDMp$-Tz2PB8d zM<4zbd>f78F49ul5}IZJ8Z0D4iT}?iN<#(^@N!p3|5d{l0|7>XEvUf-|8>pky9d^0 zNU7Fe`|SLDvvrp`EO9W^0JWh%hnDqQ#*JxhvfxIO3=^5CA=Stf!Cl`JVzGQ~+_x|t z4hx!{WDaIarp1e>pRTB^)J=VKeu;;G+Qik(RvE5~^r%g_QBz!$_E>zzQMp^+2;#L& zqo$pZ%3t?llA3xo@If-Wa6X!uV64%@L;0}Wr-sc|J6;V!q?dUEIWBgnTk;3YD#Z+JwVO59J+1w1md}!8r(17QAYR534ijv3uG8Dt{qjs+a!)e8 z234L^ebt+YS2xx9-rLHnC!)o1!JhsWTmL8>N~$fjG2%;WULVV#HRZx$ysZ~=(j-XW zH)&5EzQdK9)7zM~R^#;B+r#>bh1pw5YGBy1{Yk2`c3KWaSY%97zGP7CvFHYzE?qP( z(V49DTusAvSq+l~_pyG6$Blbqb55d5`}}LUk&W^H%>2bAbs!Cwdl{SKl}rzgpl&<>G%!1x1$`%t<&-VNpG*vtLvGFzsZDtcLiUvj?NRvw2ABD5Paw*@3BPPw zZsK36M{iF8gBPK?WCN;`w-OePO7^gW68@KeiO;OE1cSl1k^OxA8^{sVPENT%?&;=UM+|dUa(>yy4d?UQO`YnH=O6hog4%WAURvT#sP$>J z$Rn?}%qS=nTaL!uH67~Z7ujW(iia^DO&b5HL_aT0Zw^j3d)TgJaL{iXLX6NdCEo74 z+w@vL-hkm0;8ZWP2us~#{U#&fxz_RVd5zwF-6e~+Q>fq~ z=api?EaW{f6H_69htL2!jUNc4$?_GPy@^-osZ!^6OI>|SVJH}MD3wj~jV#-85Orv}4pmcBR_toTyp=eMi#>0;{8mU@d-hAru zp7Ym%@km_{$K7b4=jlgUzt-Z|Mf-)Qy`Y84(v4(3K-EBVN9<50$Q1#d&sC70QK83S zNbbfRoBzC}!jyD?z^z3!t5UFLzI@iW;6XdWvDD)}g?}Q++*n|BOv*r4w z;Do7WpP>PpgT+>%)T)8|U395v{SH@*gnE+z#O5dZp*D{MURSXYT#jg;4yvDsE#L9^ zpg5_Wp4s;}4zTzb@VHpBD#c4Hb%f+o%2#s_p!iY?-4Bp`&~T;sQjeZjMf^-wg3WU6 zhfkV__za%-&#!-jMOyt=SVfNU7g zRwB!_YWs;urzP}tpEss|Kg7d`Yh21e0x=LrPyc;sU+yx3s+6GbkxGwAFM@SF?FJ4@ zwLwc@66n^Z85ja*)Qz>3mnxgisV?tm#Objt@B-U>@m33BxKvU*zy+ZJAe1D7;KM4y zZJY6b{mwV4Q~#}krp+KkUj#1|Cz6|^Tc5BAL+(>qpWTAvS z!VK$;EbkRif&9w?AexB0>9x=?dfKdzkZxiIv>~rGzo=jET<|-qzk9x`U3atPG~(7l z9mlYLhOw*jTFF^_`FsxU$wnT$Oz>@0L)|bc*5Mkafxf$rmy+4}(85I52mNG&`*@Ss zZxd7#a5gafheCE9`?3NSbQx){?#WBvt^x#2am>JU~s6E;r%Z&bCgQZ zB?}**d(@nV`DhA@-5zwmi{rlgikBqAG0)qI7|JGbjiUK*EPI1u?aCnA6B(*H7xKmK z6>$~Wjj&apUeK^v*~wUJQr{SM+2y{x2A-Wg$8aY^SQT5iDiW}v8Gd@BONGPdIq1&I zZj1ORn$z9D4!x*K%%X5=QLb|xXRjo|dpSc$uNq=Fc-Wjq?80SGj{vQJWt9kHl$g_< zVJV_S`QsK{q>hmP0ZtZ=`<;uLSM|*i7CRo3$p>l7m|5V}>5}u1f{a?r0DeDgAkMvj1UCuc@+;+NStes3332`~zIL+S=LQ&L6*UUN9fe`- zW1PV*&+^7k3hm-M8VGj1%M#Z&xAV41;v--r-@vX1eRof{O8;9%sL9&-xK^+jNP9lT zp38rMXrjA@F9yyl+V?{hqvSl4n{yQ2<0b3;QTVO9Q$FsF*N}VT*p9nKeJd;y|0ACD z)Uxk%594O@sE(=8Z#cT9Gk&hvaR%=weBPB`@SSMB7fm>RV^Z2kGAkzfM1eID;@d-f z?M$3*h$(wV@n38{3@_gn*0{FwfBfBPuC;fP&f+1biEV41)%|53FV1mebo8MXXCknF zSR+2WRVt(*+InWBH`|e$JfPO3^RS z`F)IE4cc|lXeh7Qb3jPy!ody^6B^v zb`D?v{0mG&vvu*aE>`;sQQ+J#0q|Xoa?w^UK0(}9B;U{Cz29e^F1KrLHL(fFyhGN7{gbxZ`i^t24HS z481fCpLBPDJFGk(>JV`aqr9*KX`SpegD5ZAROi{SMLq)4A2~=lPmneCSSaGWlrz1~ z(;m(!yC=Onz7fDCq8Kdc8pA~U-A9St#~rh!BCiJLkbY2UG(vo>5S??W$l8E4)7}z` zRN=wfQmLh;vhRnfo`bEqQ=QXhp9VRo7mZyny8TMsz?G*(h31v(#y0DqEiyy*^+G}y z%|s4pQ<85JdJm-oz-Rh{($Z+ji6ubMCn4V|<#EXheHRv`?Y4q2 zaZuApVLKjqfKUtOZ-B26Km2*MUUZ=K?m86@L)$)l)k@z|C(hoi6!*cc({=$qJ2x@a zUYg6tt%-eDU~|=3?_^-Cbv!p&5tROupr~VoU1fmx+(oU4RCFNrXAlNDKhvaqm+ti% zEHe^mP;2^^o1W&T`T+>Kv4sJhm-}}nyJlYmcldd!2j(~gf%M7`+E5;6_iIQaTpsmN zXC&t*fz0Q=`|Nbm@^Lfu?tHA!(kkl4;4uWm?6qe2E;v`_AZ~^=gKb&uf^gLJc`Tlf z*f$BwwXd7_;krD9@2o)>#HK~W{UatnS98`QW>OqIzq$8dz0_ev0Oz+Y&Y0V3C0O6( zOKe2n^AwXfiw3B~DAQ9i(DQs@9`iMf^ zqg_b*+*(55lrHuAhE)WZc{CettJO$+k?Y>? z2gttCmTq0+KSM{Fc@Luy7UU7UfIRMhnZ$b!WJB4V_;TMiH`qZgU#t3N41d-F5ZH7U ziHer%KW3l7pEs>qrU!FXQNu z&QDCt@c@^O?-A81v%7qMQ#4EeM`VarNhljPLg4v; zMDi>mb5+%Y9mZ~jKIm>plEw1KHoeXSF^h88Bw;!c6}X1}DCcib)4>D6pu=UI>~C>e zfs`Fr*}k@DqX!?vckN--tfr!8E=c+-6x%KFH9~DS7if$=IDFNmb+96)+X>nSdbaIk zEt2u>X`nk~I8ig}JW2P#Nm^cgWWqbeS4!7Ts(7VY*vbR1{Au*Pj`p+Pyokx*@6ocz zm(`SKs>Qke`n`J)3h1P_E{M8LI8h82SG@nW^|4BJGL?s!lD)~E_KEfP_wsoM;qG;r zG{0WmXuK-U98>kOK)jmu)5eZ$7_0kd&FuFRDL%nCjOG28Bc_n@YaY6UZW!CVF_tm6 z7LoSx)PtN~22NL)-`>aX_;q~3qwt-0KdgP~ewXTAQG$Ht1SJ5-xsQjsB0q0&EVM3RnR~gd-$jUjt5$Ae&JLw-u()ufi%3Ir+xT!9X9Uc=96#DG?cXE%8}>c~2*++>b%{ z$#Jil(R!Rm^~Wc?z10ugoc#6}m)o-RS$t5f=}Xk;1!LX%4_5FC`#1{S>_`i$ZT=sY@4Yj_DnsW7@XYi(Mwo;7S^-4gole_?-KsnS+Jb(j zl5^mSsuEKFs98>0Xam=_-Sl$|kb+36bicq4j}rpN(Q-#wLXNF_GDV8D8r0$z=HmwV zNLog)%c5#*DNTS~KFhlhrU&2FSjl%=qWYaRwj9~Hr73bt=qZXIf{lrSqWTr%MSVT{ zctQX${B)06f-(18{WjU?7F%(UgdjBdv+NFhZdKXW+7t~ z0}+Zu)Gx=mX2P@nBP)bvw8JL=_PD*;`k%iiqmRHm);z*+O_+S$)k#%%_ z8_KowCWFZKJmoev%E;8Bm0M3d#g0S%XQL-9%sf3_B2coS&y=SPr!a|5Bx^8Bym7CH zzK8s>2w`KtJh+ncB^QpB9j4mhaYozSQw6;$@irhCu;JI{{#yapiv9gQ_e|olRw%QW zVWSbq9FJW0`9tXQD5F9-t>fh)M=XzT#*bDM?TtK#>+cM8NE>AcyW2G}$=hp|;za_t zTezdz%0;Fp^Wve9W4td^7dyf3+CfgSnW1oqo}OF90pPM`v8Y1Kapgui8{Cal0SyH!!R}4x1Zf~Ly{J7hiWyI^_XV4Dqad` zBc5Cdc3YG2DG?Q-YIDmS!*?#PiS3CY1nu8GLf503l8HsKuz->)4l@Mi3A@))@^@mi zXI{>DVvhsbW=!4XgrDtpS%iONkJd@> zT_w52#Xivv(z3=73B-WY55`M1rWsrtv$JGqQbtU6t6Z|#yf{))Qp;}k6phgUKPaR% zemEEhJpHwL$Ur)RnucSa)92l~z^v#4+;}Du=#3|Fjc1oW(dW}-X~&A3s4+aVk-L1BG@lvHC|3HXT)+m4U0Vc{(04G=|wYWxv>G7`8%RyTtqgZ!* zn%JxR{2Ibv@LE+35K@EKSy6fKcTCULHikT}FBV0}PpB2WexSC3&U(&k&OQm>^KCo6 z4R*b6(ImB$5a88-#Ny15PuI4&B-Cpq*^hu|;(74YT-UMY&dkm|4<>~3=oExv3h%d> zKlui0GQ;LCo6d)upJ?CLb?^TDe$Aw$WX$YyBiK!$sqd11A2=ex6qH`7$7uXob4Ic_ zjLC=pa8gqrcNLwCTEp-5-iVy;1(%+#eGNYoa3MdNY(fbA&>~Hjoe&96TDJIlwN31Y zKak|swrnIet401pUY3+bU=_hdlVtNGvs{62%@5WNg>y7|X&c&av{8_s*$!)(h<>&z zmLP=gN>RYC1hFcKfW?`ecQ^mN;yR6J)f694ZNGNQ!LhdMYQ zAxTym{_ORpV|drLQ71X$N#(#-iq3ua!OkBN98A}sq{c?`^qRFCdTg*nJMr-$B>}4X z&qXw%yHq00m)o{ZN~NzP<%vy9kr9VHArMq5uaaCc@$F*(RZ#aKAK+j*jbYY3Pv}pV z$tNOjzCMGE^nBZLtINs$%zgvvu6ZQj&ELJLv*pAz5?~Pdk#i&1ptag*zRa&*BoMH5 zw%1m2uF&3$u=AZlfzZR6gW^Tm>E)r!-Qx$DU7xs62gap4^*=Z$qR6l{fQv(N3sUt~ zq;zUl$64fEg@f2B-U+!_an>xk-br4L9Y;Z^`}ffr#-nLEXZrEOJNm-C0?k(?qCfn^ z=AA-=%5&(5PtMjsX1-QuTR_{3AJw2v(uaAdCqrk2AgmYdk;k5WZj_#EG-lag&H3Ov z=Mb)Wgo(Dh&k1O=MtzTFA*ACTVkA(K;4fkYw84|E8S#t-eCK~pGaAO#VYW3x#Zwfy zRoz2jboK0IqrY2pc;4d6MMns<6VHT8_p|cAS`c4pJhN zS#ol{`&I46881vLA?ikN(k}W@lV0gPex)ns?nk7DIEDczeBm5=tX)$w=6rX}V`KT} zpWb{j2uK%g>Mh(c-LOjbIwI@jCO)jt9$J{Y7;ubLA00WoN`DDb=nQ$2mAFi*4L-tL zBN^RPH5mPdJ$WP-Jo9&IbFM>q^L1mXzss?X0oY@{JRIcg^dJcYZI=tr8n~^w94^%sdn})r>%ak9LZ1cOZVo;+4f(F* z9amlKxCh!9X{1Lr?U)PntN?EhNCp+~zv3d;XMd}3XqVTUQoO~q%ej;Rm@>Qa@eG?n z=4Y4r`B4u(4TS&vtBw5zMUpD02<=hN;jn+3(%92oo!0>0>2)LYEG||q5mGM1i_!Wq zR(d91sfT{+D5q6n9vkyu7#t}pvKg0*U9(V|e@t2TYEO}U`AMS_1e3E`i2y|ET@j6k zz+wC{7Nuj$m;eg@!mcvUis=GNM2BtIy56BedxBJaNSu=BRPAzvt(B~1`w^AFvnVKX zK58EN;T|m{cO9YA^T{@f3A3!Kf~IE_CMRj=gL11{VOri#u_6a!jK?wKQs4D68!qCl z=aEnE_&ykowxunV{4Q3@nSGt-PVnU?X~vlQ{tceGTa zE_+8CTR30qo`zBWGTZ-{Qg)-KlIwW)fio5<^9pi5_AA}R=18o?N!7`Zaka(fE1y}h zmFFGuoog;4zIXX5sF(e0S|a+oELqp7KOx<<#%%?5$gDWCyMK#FTzN#hFAdxdeB>nC z-04&KRo;!f8#)hN&^FR2MPTCi^ty9RHrkoLxUbon$1s|7R`sY=HNvT_#l3Huk}qfD zfO=9dd?Ff`I`qkiJX^aFUF4PE`e3F7#I|eV|%%_AspsmIS|tskcmb1|`b3vH(Laq(qamba8% zBEi3cXtWDMtz{4?-TStzQDdoJYOtW3Exe9_`SrZkaCs(DX2N%|_k%0u$os?t_Kai@b`q?o`j8|FBnzvc5LaD;#Y*K(kY zEJJ$yLr@u@>eyj-5Jgp8eH%vL9-^`hDD5&>RUrrEF}uB|X$L455->Vh|GArsu7sL9xb%%V!n4O)!H^-wHv(QGtR2-!KHVyp`u%U4IFIQieYQ zG%*+1JE>`CNA>+68TLk^dvUX^JEHOmC7Q-8rZL3mxoA?JA<#c1vrJdMFCv@SAu9Sr zJV>h_=Fhwjx7iaQrC9^w-4=6aa6rvgqfv2p!yFyk={cwry*u+c#Tm|~CVL`1PFvr< zLMwdsqJz{OW01}0kS%y}=T+1F)&w#&8W>l>_IQ&`cldGz?FNK$LoOsJpYFUjSU2K* zGz#u5%kfWmASG~m)NGr|p}*cIM!MG-XAhnK@>pw%YSFcX_J*!5mk|uS?z(StNh8~O z(QCvlPiG@AO)xXkXFjiHvl$C5WAX_XMLQHwnosS7IS}|1>ZEJExGtehI~aLFgBdm3 zKB=ZBD@0g4;<&PCa5#6895mH<*L8k&(Z%sqYVdKo+Y7}wVz=Thj#KK`__W&XY{#U( zw1NdkO3u87@itGntdQ@+4S)36O7s=q5zeV@23=qPEHLp}FNKm>#^LoVezDiq`BdZQ zsG0nncypl|#I_m_%w+;+_I#u!_d%JOwMY}xUe_#>1a~PAR|DRKCL#P7+AhrY&H2Hz zXG*#f(xf$mbR*vv*&2sB9&ik4oN{OH%a6oQn_OM-tJ^jxHn8d#EuL3=?^i13O?!&_ zU}Jwt1In(c@^;En+T{(5(Xp*2<;fsVn*?di$zQ?91GvB@#VZo<4!8eg8jS< zYWMoFHlyX}?H*2ze*S;}nIPm}pGg`>A>`d;81o``hX|g$m5S6b7B{1QQdWnuEz7VE zSKFgr_#^Apx-%zsytSM1HeNSUTGsl!%tZQuw&@G`HK$^{kGr0&g9XXV?~vY(aZI#% zVXe!C|K^*tghm$XXYPF@!+ zbyn85Z!J^IUl4Ao2yD*cNh#h1AO5+(s!Naq`e-KvogyV3$$cc@^miNQS*3z;OS z1zc)ElAk@Qlz(1sK2Qs(s7S^av*0I{5b^B!HUyV*&L?pmfEI(SZNrhz`^u6`xd*8_h4rAEmP1m^WZ&uq^Q~Ie1o|T%2d(8p|BwG#|BkAXn|gQ2-B35 zOeO`@*Od#f6Mf392V>)^EsB^iwMHyPUL-Mg3srXD-~|evwIBj9lPqZ6Fi>qtssA8gtLE36sk0i zi%&Knx`_iJprOV81Ej*_p9>45-=CZOw%jk?%=>}QGmSA34-jg40I1b1>q>(BskI`x+Zl1n)4Gi*_v{+TWgfi( z%k0!}O0A>1dg}g|)*22K`?U?;R6jSEMWDXyU2RHuT3N#)F?P0MXgh|qVroTaP>sPc z_OQOd){+QeG|6L8QZ z$)dw7!E&UrG+e4}Ti9b_QVHW8eG>4Kr9?inNm|fC)pDeWxT6*$I_Vy+fm2({;slVa z@CdG|G9rd3fH-v_cYD(Dc5dH;A;n|y8C-ZGO07wu1V4*Oi;s5v2~Oh03;bDbsln&G zW$v=LyV(|Y&oIxCUI_M|hKrYFP%^=`aPO_Dy!=L(N$~-mZ1e4paTL#_uzMEC@K*vI z&=2$Kg8ft4?0}(?g`UobpVLA-&yDj!qxe%f=uLDb6}#uWePQrHgS^J^R|*eL|zu# zf{W@gPo=m!Qmwd!xHE=DtFp9+p5+4N54Egnnm26SC35K!rtzfCu`M z40)r2Yj9sTU3s=>00L#1^)ZoG*X04@Z|1d9p)M3Nb! zk$=HT!5At49}Y~C0rl%Itid=)U)7WeVQuc#D=4TK7};mz5IYk7G;6 z`Un1Vp@K;gQ?dWs44##+>yimB2pLQ@;#q8zFYp6CZ+P(mZExj}=?MYe0819ezXViZ z()l}(fk}d|GZr|8ZpQm<%McZo7)&*h5@wPvbg&?ClKj6HBqV~*K^MND*2DF7o*zGc z@D^+vf+xBo%Y}NV>Ue~Id+L0PBo+y&E2gI7frR~EEL{L6gts4Ig3b#P@l;x#~nUOaX$+^_Mjl8to0f9!&=ac&F79jgkL@ zC?ft*^et@EW~!={&bOBfY+iRJ@b#m9tPSER8nytaQ>Y|lbI+(!3(V3SCsO};`S+LSPILw^PAcy3Lq09PnTwXCZqj{+Qf8#$`dzjljsrh)rmH z_}kMzC_CAt5?Ci=f2HvIJhI7u=gRZgxjmc`@q!~XR5Njr1kk~;ad2#dZ;9O>2)u>c z`(9^;MMffwcupQpl6E<6baH_2u6bpKT^%nyo!R^8Ak6rl)i1Q1ozN(Upzh8EMeGd3 z=KH<9DlUi#-+PkxF3itc-L^a)m(^3fqtWFf1>L*luweXyRPX*#bPwCoaNoJPxss}? z*skSjX)@#vs1~=27~hx2tq9sm*bV7PU>sR4g_pxBQdZH`+=5zN6ap55Mc&C=M~157 z{wU6w=HcuQ)0D60t2ouw)rV`XjhH_rf=e|9z|yFlLR|F^HUr?#LA_CGZc6kbIX5m}vd`BLDApJwOn=Y~)-PnSWZ+|F%f*9A5wb9Dlb0|4Od^pH}}$ zjemas|A2Q693 zuNTWWe2gdfac${RCa7VbAK(I)O2~)T2V@D%GWGXT`#Allm!%80nbX$Jp z+##)1B4VA8PLGos5DJ;fkz3(eGan8MHT|5BL(!UW_^GDJXV}HRVr{c9(135@$qRlb857USIW8D-gsE^{_4rN zVS6Q=WRKg5AGVXWg{g@ANj!_ke>~JLwxq>sVYT&ZPw;)K!&PQubM{VrE{gk6VruSd z^UC?-yq-JF5%o>f(a||uEVTbO)Xg_#m5Ak{T88F8!9u&%1qGrG(lqP0O(!vOp-l>8$vPPkQD(qsfH|QT>OqP zzQV537`4^YcD}rzJaN3~T~J)VT5nkA`@u}t(YnumL9ch1T_*fG_CCJVpwVG(y>{?a z2Sv5P74vs9=jlV^mi`#?*QL)eN&~JQjCDG646E^hk&VxzUM;Y^I!Xz#KaZLx?dp3~ z(FHnDElKPAsj2GEyiX?%Rd3C6JPaQDJM%0j`2 z!RlDVI8?Fs#jXOu!;G};W(%Z6p%lY!1C6ge)-8iPT7d**Scb3V7@kBzBsGya0|9j? z5MFklSy!Bk@L?uiVy_KTFW$Ex2`Qh#Qu%j0WL|6#j7_leR$9Dm7;OCxS5Npa1~h>k z#f#w}iT};Qlf(%gW_*F#%CG6n;%xB5X?*BQdi6s&Op-tDkJo%iH}ME#mSCF*B)@&T}ta|RsS`JFk= z7Q%C_O2amogh$IflLw7 z3r6_SiGS}B)o8ec)FW?Fgdr^`yZgq_5rVUNMqoqYpz~!ap=t@U^rMM^HIj?AeWHxx zLZ0O!y3AutjZR>~Pd8GZ7vF+m_eV42eibpG&5l;o3a>nZLRE)B!rKoU(vnv_f<{8G zvw!HuYDhuT>-cDz2zz+a>S2-9?et~j_@v9Mf_8qDkS)T=2NU=6tJ(cf9k zzV@92U3|J&2tClcZd84GeT7$Wt;_hY@IcTPi<5(+)WCf8f8&iLzzDA7cKhP-){A4p zJrnsv*O=Q;3SC>pDARkTevp8G_d@t!`{hwzEg;Q*c%@ybZEX(YG18HHb4p~r$~!Ze z3j2rk<#8VG_S;h;Zl-zLz@3{>uJA$`^4(U0TC!|c*Ui`>(2`UfM+tqlAiAMhBEZqU z&IjQugAZiZUFAH&yd8GEbqu1q7hOQA`*2RqgGR2-W}k(1IB~TBBVi36S~Wj2QjTxZ z`!kL*hJi)vM5PSW!SK)GwkMsUHwJ$1+2%$duGv#s?-Y;DY@N1v1QIT9H#k|spJOw6 z_T1^1aaiPpR|2R9-=5cUj1L6dtoEpw^=4vzA9>B-Gc%iQCFVk!_gjfUg3q!x% znA=S4rl&3lK1zo1opMTUjRW?6H28W+MVaV3_KjZ}#3N^w0-XzlmZvmY@Ep7ev)l+#ZT(dGm?f6DKs%Y``(ce>aFI=?6mC$ z@JUH9LXD5l3haxVu@PRHt)`8;Gh-Gf5A$IUt!~K`e63jMJ}fF2@2`VKU|P4YQrDPL z3(nN=z^qs--%C$Rh^QO28I_W$3Et;xGiHZZFYkX;Za=SOEjIl9{VGfZBSj(Yo5zR5 zSwm{g8rM&#ZQYk|y8~(mmlHG5z6M{~CC>3t%RELhu^O$P@WfRwsD*@tP}FVIUk9BK zip&74Ud+?Ic;JQ?#*!(QgH|$Ql47U1Pt2!1kc~EFv>b-7&tjflC5pqr6Qn%OL3N?a zs+n}colBM_RM?@A)HL=Z#ZHXY(`6~$FGLGoBABFI*$I1^WO-xkk>qs`!+joU{L-09 zKg?AOf)n;4<8sP262rHZTDTbT@;~5hH_ZD*#3gaV4@69}dyzrCi_kc_b+wlni4o6w zhSGz-m}b1A>BoCmsi)U{URQ7=c1bS$;)YKfugZ)>%Y3KKf3(%(A43A@u7`i;_yh}-#07oO;DSmk1DOUeDJ2r zFzn2yhXR#}Dg^EhVs- zbJyD0NI!8b2laU+Qihc}9l@os6BBNv)&|CHiO}aA4JX%}Ccb{nq&>#}9k6cwi8;9H zP;LJu!ub<^SYY@&maPwL9Xo5Ac+!hD$F4Gj!(y3|J`=a~Z@322%hsMre&Amc zfZs@DUnUUlj)Xaax&p9^3}xoY*N=$Cjmk$3#OV6yEuGdVR4?YZq$X2wtG5v)50VCe zU{R)Xa@IA#1t>RR-?LIfjmeuckb{=C(?QQvX{IN4b6+~j>FHG9MC%L!3OWXutg{(5 zm-$LBzr&=-mZeg&6P&dgWK!TMh59mzxr*-7md*8xqW>y*VoQ!z?dqZR95~FGa zoumU9%_m(_B9wV)F>j?~Ln~(a_Vf_0o!2?%`5K9Sykt!texbMRtE!oHVk7UN*=$sP z9fr6D18yH|Y4fp}-q9rT3&XNi1gF5`^?7Ynl;hB>y0t`^bslm<(2fo0m%2?vkYwts zNS^DuFUD7{xYHl*avq9Six?*2V3jjran;OV~F8dz&3J zfm^>sIpPdOLEkZc%UlaCI7#=O@}`F=D(8Z%Sy$_WGi67*PEUoX7?~DI)v<~IIx!f7 z%=JH?jN3+wOYY|Gw5^EzKFJh(Vs6k(`7K#Xbz%1Ap07MCeHkyoDld^LU|De2$Vb-9 z{8Ku(LSyUXcC1N&z~1Kjm|Yy)hxGx6T05E?Q5-LcKB7c8aZhp6ZuO5_^K8^OOV}X< zREqB#stZTQ`*TU59~V=vKHx6Iq_WNprZQ`tMAwY=-W0HE^2uEE1e5HWJ@P5Rmn_!7 z=}ANlY&Ow&Fd+6yG0LQ;an$c#7b@rnJRV$_-R5?J9e+({5bjFh{`HWnplgJ{Qf$J- zQclhzw`FTJOG?^H|JEzgZ+@eSI9r3|M>ETl7ysKh`?kez+i0jl!Yj|Gq#T*wydGtR zs}9R-B<|=saok1|pXS~5b5?D5iA+$HOn0tXw8oX6Tah z#NsopElb0YMzT!0y|nf}v@Si?5q@)|^1$o(TcGw!DfIsSv^6^-}BY$o!VGKa_} zv#>;@v`Hglo^&RaV!a6?e@4F7{(?ri_{3Ii93|R&r{{F+T+w+rk|^wxpfcp6a8!~t zA?%7JQj!;fL-ScVEWDGe^`#1uMOwTQ@~`#^qur?9EiN%(7mo<9UxL&&9v2u@{ppv# z_sloiKj8(aTK=fJl@Qli5%Y8w4kN75=Vij{lhG=xXI-)4rGVUoH;GK$H_A-46Pr>+ zoY1zfGp=Wz>S?PwF6?Vr`a$3xa^!BG2Fh+y-kH@#d8|h$6HUUU`hi~e>+$O17!(`W zSnp1{PB#U(i;T|UU=aWAh(u~iE^nJvc|eLL)ITg5H{DClbvuPmYCnH<)rjX4E{t8Iu~_*zP&D8Pz86W9NR=4RubdbUS;zffPdBZ5V1WcFOLuy-A+Ry;)R&@ zI8T+a@*1;ce8&6M=o7MEQM_436f9#zKoG%f-|;GmZpQZ9N3|`;V^{On<2fwcx5|O5 z#HwWZ=3?#bTWB1jfir(=O~7YaN)*$)QyfXkZRv~VGOj-Eph@^a`P^fF(f2V~4-;7i zGk6@aM5tNk+;#?7hZ%;x8m5w(>rx0$I8u7w-qgo+cuP(kpT`}9PHoIR)HwWpio4=R zsVBE-+gRm!I0?s^(}1k}+~+TOBUJ(d@#dwzZz|GDXidtQl(nUA>6~v>H>G&otaf4% z`4AjeQy)c4N>U|+k==f0hAN%CK4h+1^W_lL-Jq3X`0D!y!~6U66Yj{9R;R?1urPDu3R#bYMJk!Q#n3kF_5LikOjy@tH2JdJ*a^iaDXw>A%D5v+sM&PKg0Gr!fd(U8Ej$H zFj0F{g>yMI_gV};`srtitky)M4R&kzSoz1HRMd-B*nvlpM@yBXOZ(GgL*G-s&XtYU zavW~ai!oV7{*&<$=RgY~*^9{qzisyC2U-#g1$0Ns0vXibR>*63mKkfz`)$>eICM3JQgVR*+lmoevI)qGl?}1gt z#373md|ptR{UE$!%V~Q=V@KdzwUivATry=`fp182Dl3T$JOmDNUXsJxTt3srQa&&I zp!E(aKTqZQ$H!Y5k%c*ik3;&aN0v@ylep&x8*hpP2KrYa?M~%e4<8YZD-8~rnA^I4 zZqtHozcbOFR#wdpTPH*9oRA>|^t7=HLue+GDGfn1paW+dr&%(8e=eswXy{5`RGgr(3vj>C~# zrGrVKeO2VdsQH2(?z~`0P}G5(nAPLNU|+{oPvd}KZL!{y_z1_=@ebSl;K=lnGvTO* zwU{dAVwi(HdEpt<-5YZo5FSzW(#Dp1=4*C#MJk6Xr=i9~1N2tKJyk8>S_QfQF-v(ZzudVHvq z!n_V%JZ2b9J2do7VUX6$VT+9_fbZ$S^E|HE5%tX;5}5NFFP)T;Ur<(m+aN-{JrhW( zvykQ01nYXPJJ(cd?L;fIb`fRVdK&N7gfeV<8**skj$}(n_9-uwv?vmrjOih!>*;M_ zl@jf8Q~B@pA-Rt96fc}Fb@@YSD?DdYsYO{i8|<-4hi%J6*iLR z~VinFGV(_~qAnv9*890eyMI|&j3kBOJ$Z9Fc6ev{%=|^jR>`KW!UEiO%^y}6& z5;8AQG`o53aa!vQjx7}0KDoI%t6L~;6Kq!ru#ntq~(R>IEtiWAXsP4gx8HwD2d zoeX77f5oQVGtr_`*+_BU0@`pb?;N&+C7Un#Y&Qtba*Jni{jmIrc6fZvC zu=MYtX^3SfTbI(bd3$4*Ey=i*prdPD1UUj*U;CLB*X%9Iq%lpc_cVglHJb@r`j(7h z5#>8in#Yr0I*`ncMXtN{Do>Y`=BFdEseLGkx9Vykyfya#b!>}1mVJWcpB@7u6ST#h z^UZ+n@2(o(9@2c3Ellm@akBH&!>izR1O8VSXzJLJUmvDPQwb z2~Vwz{92kkv01)2M~B`+|4ssZ)Sr!~X>2DgIBlt5w3M}EJel@ZZn6mVi(-~gp|i4F zMLMn89Il#|N}GFNJ8#e=cGc%&k;r8j;qavE$9-@NrKtGm%2G zM|&^S)4?&GXF7ZsX)%9?{61vyqZYH7MA4{hfzun`zIv&ly>4u|hC$(U=f+NvA+<}{ zAvLxwIBV5ySqb=l!68m$#ZuDhb%m_}=7Gqi6iF*OX`Qb5+)|dXhXLPDZFziJqi#&^27+ab+-7@{kCrkzuT_s#6S?qCQW{YjpZ%h%NQ% zJms>qLuAP7N0ml?eY_LrCBC(|wMQ~4qN=*j7aG+#$qt53-r8clc}JDldrQ&V_s(N} z9;O*zH*`l+=Smo(Oy9Y@0@OG7OzSgc2M?bPH7L=ybSRzVc6__-dorv!XN6)>hR0Dl z3K_pdS+D#$rrWA~t%PM;^QKCND#3v(e7ukFem2!I#Tm_7Et{F&sBFM69UUT>B zscM5uWo}^yzx`3XSn^$>>h$b{ zS5a3tGsO!E_WeZ7%Djs&;t?;3FYUfdj^XAaZFb-lCS>bjbtc*$ zvY(vN);tB5*UJ8{j56;0giHA1gnM9!%@9|)3D#%g<30LY$3 zIW9+ujv=luLyV`!HdWgp5M3qkA&W&}Fc!+}daWCM)Ft&Qs}-bAzDY4L^8N^!Z(p5j zA~m<2?2J*&7GOzsmNY)ZoQjHnR~{zvDws#CesN9h?Cqy~S5<0M{iOwBPD)x0xHSk$ z>06k-Ur6EuHg62P0@aj=EeONkm z=~SktmdlzTF#qu7HSPJ5(xCeQ_gA6KHYO^k$Ar|+y%80Sg<2UAaKeRq(rH+E?RW=5 zJ4bbAQ@QKoUMEQxyIJ2>&fm46Y*#jm%Hy_li^vI>>{|W4o}djmiw#lvLPVX_INRw^ zRXOXXn5Y=8xqs`{w~qU7<3=kw#t)*}y0N)z>Fvx-^VhelH=dlh&aMmOgmRc9#GqzM zzZxT3oj_6)xhUh4d3#UqP}_dv)5{J=ZPeQBc%qqEwp#N{eW(5$p@hy8lY_N}1D@@*LOt>mNF{Ge zxBKQmUTfHnjZmc!1ySn7@{@OQ#}`rS@Jr?lyH*=GcSD|bqgvC} zztDU#D+Y>u#pfe2hPc?1UUJ-X*b&2@_?GAoVeU#v4vGjGyA zhzvMh-db8`7nO1D52W*3fnOjbnjCd=&+r?sJYPfKs(aUH$r$o&;u-U3sCTGk;gq>| zamyGp60ON?*Sx;3+R8!Pr2f+(sc#+;#1^*@H1%koY1rZ}Pjv?#g-7>}r}tNX3T;0x zaMI(@d*n4aB3+1}HmXTuR1~+$R2?vD8n*mRH@Dm?A>u>R7ry zZjH|pQy7=p_n15J7Z4tf%Z;Fj$Nn#KTouNemPaDB?)2I;xGVK#tFj%Xhtv<>+brFZ z4@VrgAyBqA+}92+P#BsTYPT{IDR@DV+S&8MyVI}G6pxXDck3w1q_T6w(%$@Oy5Zze zQGosvyMBc@&itV4?mLrlwb^~IFD+QiHl2$Ezv)f0CQ2?oXBkTEP|)Xjf>nePa!2{i zzCBaPbG3Fso{y)#Q=2D+kM-i$6Bio^PDA+CH`rwY-!5IWJfcZSG!V%>#UQ!hJl4#& ze7T~#Z-2u5cHp4{S?+_Qj?c8Nq1bUFm4OJ1_qMd&#opLFiF~nWVy<&} zVRggX=TFb&4d}UiF`&TzSI-3l5pHdY*Bi(EBDPCrN+#j$sjBDF=4paphLfZrVNgd* zEz7PXm(CpAGGtLf_(x6VLH&Zeb z9u&T}3l9(#>oq-$2tpS-T$lyYs}=L><~?#py#hl#1Axc;6170m7h?> z^VNo3CG%ux3gQ}2+P##i91oz{aw@=@+wv9T7f;X2qbwn7eo9lTf9BqK>{jE9KgWIV zyMM#RxP7c<#!I%k+S(Xpp8nsbmx{T;(KFMYxQG8vLC8GaT_7xOJBP3?3J0mmGM5x9ya%)&#SiJ$rqQ-s1A^@R z?1q&w3j6dYUoLg;Ih1^zDjp~2XTOKPu~lufVVLr$HCA)DS{Z+&D1N$2B>iZ2QqO5+ zY%9eu?irr1pS?5B8JhRSy`=hKd!94a}WqXh~pXA$h2V37z*b-mAmsY(fAFrB++=a7z)?y4Y_d)L-+haLv@0#5@-2M zUcZ&M)nZ|4+222ewQKW$uJ^OSvm3Z`wLxO8`SN+4xvfv&>2Ie$Nm?;}7N0Gg&Lmep zz=9T;w@i7NhsK|g%e1n+dXN0X}FNY9~b zX@>4&f!QN@-{PBHzmMr4J^)7bO}iuR-)GM1+~KG}_BKG>S=sX}_eGL`!RbV!FLCTl z|2tnX4APCQaAVH#j0kY;4vhs09c&=%w}tVuiG|T0)(U7BMV}ycBAYlD`yr)jp$~Hm z%l3+dY#9<&#oZ3zPmkmk%18nz|H*u|L9wq&@|pO5^Gel!`BN7-+E)YVbU^q=@BbW< z_?OnN|MM7}kDa0i(b{6e+obUi$ifP;GBlJplm?s8KkvGHRya* zL5uo>eC{(PXvgf`_6o>EUdY7b;*u2`z|5A>-9?^ z@$7Jxbn?6Zi=F-2CEsUoR5rgvhxJwlP90%XYJ@mzk%45+Jo^e)i(dS0K zNg;SP4U5@LAoSFJ)QI{^mBy{rQ@eg4TdAR?u0pXbFK0;-rqY23^VLvVh3LCM_9);- z?gSt|(tlJJDTh20nnIvOzpk(U?lT`bbgN1QiwrMVBef*igRAY(KJuQWWj-l)uU`L* zJ-a?6&dW85u5BbvI`A;F>2ybLOha}nD#*A6Cqw*8nI(;P+k-5jv)b3VmncVxLW>&I zyT~YP|5SVaG0p#zh4?aqWIbM~xTXRwmMs#r{jvI*_|+N@N=Wlg$J(LYIW(9gX|J^$b|Jod)40I~|JR=dz zAz4`skxgU?4Ut(amy?hO%~OvuH;kKB3-0k`V^eQDdQPCvsXc9pG57GD>u6@~y_t^; zS%*(K@(H~Bc#qlXrk*yrJvsUx&FJ@ks1F48gf8(~rU2j7yQi=FZimvA;e%l(dsWUU z_KsCTNMVFap5;?abJ+`oxjQP_gZT-Js7$q|(Gi>V)!CMot*+gQJI+y9&@`XHi%hmq`8IDWfF(91(+6V{+Ex@PB0S1GcuNi_uC0rx!J?us}*Y_b3zgBimNgm@V{0=@uKHjlFPKg%mKmy3PSPb0i6KRhbKmHd*pC zzmHqO!P7-9k2j;zokzzROV$X9p)%S|sgg75Xc59==ORG12O2qeO|H$Wc= z`~@WjXds%uEM8rEm{j%bo{A%F`gSgAP?Lim^P18++7vlyoXCg_? z`NYcK*5C4b(5JlN4|lY~26GdprYiPmU;z1OywXh|d=DIkFtbE)nC}j-LuRSP;Si|9 zh!~j;gF|@*n8HW)6I?`hz|6!9E@J8koQdS(wuxdI2s;xCEqawm5xOuD_4a zepF5#`scR*uHJh>EQ#UdSBu64o9oq8MSkUe6P)Z7>2ECac&RGTY85l<2cs?wA#L0v z{}By_z8bIwaa%XS_(_xGqa`sSQP3;^W>5X!1%tdyQs?OtNP9fep?DZy^A!C&a@ZIv#Ws2eP`zuU8`< zQ`G4Pn+uEc!zCboxpW|Hm?QrY3$zK*YsCKB;9g$6zh&%rXFfbSo@dglJAdkyN|lG- zd-Nam<>4ec+!8y&Kyy{!XV|f}UwWLka1tdvgf#~u9ws16AcLI-roG$c`$)9z4(uV3 zq>e)zyvv;*=1-^nqy8o|_3y9VU7}TgQ^i=)KSR5GHiX@^>c<=9gF@zpDZrBSC;`ao zBPAo(OSl@Kw`1s1f4qlLUv#=!J`upGv$;lD@*jafxSZ5n!f+RuV4};XPyT*MYD z{G=}sciSgko(_H5BPwL+pg${z!8IR1IKjw5martPi89NFwWZ3`yAEs zNs8_>lxr52+vONpND`J5V5074$MNuIlyw(#yfkU#u)qML*TmmVKTmQ5vNL)>pGE;o z17t^Ydozr04{G;&4o_JAHVKZaEeUjC&WDeJb29?z=lOmeF8V4_`>b?o&d!K~-f0TJ zlRt*vJc158;vUoDm-)Qzg$KQLE_1mgTo+}X{;R#{9b_y9AlfMMyLxy?8L1MVN+>py>>14{D%N@NRYmg#Ortb-`BtN16n9TTmI}AQX5=ifg zB1b_Dg8Z8#mH{5nlTvele252W-9^JGc%|hf2q{Y4f%1Q)lzddschUrM`%L)NM%@aK zR9Vm9!P#lpNv-e9so0TDu9WYLgk-S6OtY?haI`>r{Z@8Wp29`66}1 zLI1REUFXvAdEmS{*ylRsLKxj7Whx!}JBGA3G1j+wSn4t`k zFXQT!L)tzqezn1wZ2I1Xa3Ydbzgj<^v$F5ZkRZMzH4@p%KvQIOVcTD$oTuBKIoP%U z^Y}_wd9HXmbO+en0cx~h3qap~%kcyjb8ZkJjJX9D|4vLkUA;?KBAgBHm?<*w)yPKe zR1m`TUd+`ydF2yt>)7t3V1_s0Fm{dqYPo4oOj!b8C~J6(K7nGu7C6nuixxdefGub| zWsD4b4z#CT%4tk^5AN(X?!CZHuWMlI0WaVMJoc3#Qb9_|o2slA1Gw&jk+zTc&hb<} z)DlYEnTF%AMf#*&0O=OhPi2}aKD6qcS1mp?UlOm{Y4X7xn|aj1$c94hy@=-wGkKDpY?_wF2B3!nmc9~$e8 zdso{jW-`X(f^H{>);iDy1gwzh<{Zs!s4Z-dFzWwh3kK-G7RKLJ_%ObG*qS}dVIJ-E z@>Cr+64G7HuwtrGb!dnFdfS2ls9lyJgRv)20GM$~G4mQ8Ds(@nS)D~>n82IJN?6zM z4Zu{M6zyjmSiqasM02|;uA`6*{1rNczkSp?*C(VXdBOXArS&!O2_ti=d41{yoA;{? zvT@)_AJz|bH{OT=?pE#+?W}VL@Rn9H^&z@rsKN1QM__z~nn;L zFand~uv#nH%!ML^(wnLpGU!}^6DK*G*L6hx6IbzV02Upb$uIBj24UK+64R-E$Hh&% zc+PX`h2ho8>_wkKTBNF&T}lE5V~taMp#vqWttRqASsRGkcmsAhiqiAI!`@f1>IRGj z0_spgXdLnEFH&vF54sh~uMUg+SwUQ)Rd@BI%RfVuQa@AWny(E=YjcOMINI%hL1Byp zP}~TKMNJTn1H`#0Bj36o97mW>4)ip~{Ws=?K;LmfEYXie z+Ld0)upV5lI`r9f#ELvY73vps4=!1#^uY*dWrXf2n_Drq6e^(7Dn%t7Oz57BB!|*A zfPoJjb}DMkV}L!ymFh^0B|!JYZ5<){*PgyXUvQ4HiH){bM&4~N&Y-{WrdyNtIAJQs zcIS+ts6ntrk3YDhd>ipu>Nj}& z`A-`>a<*GCU4nwV@jhABc5<$#N_Q%HGRI1emfR2J#xE~raZ(KYVfNCWY~E)ffFrjq z!W0qy{uH*GKcHx-bp==B;xn(ef089!&J4bOPsp_kL_`IE0N=JR)xMXH;dQR>p zC=nynyh-DnXXBM#Kpe^Cg$nBUf0ZRe7FUqFWxjg?{lx+X(;LkQdB?E!HhR9@9Rt_Da<V=}+?uzsa|&hAu!5XHL^NK|c}17lp?d}3{L z?M|lIy8y?_3~q0*6cVeimY$wCIdX0W3IFI(P62l-dDeK76s+Na9k^g)WT<&SWU2|B zN%gt^o+eAQrjDTn+szX?@i zQXKagCj_P#TC5l5@}k12YS^lV`KxUVB}e_4nFx^#bn z>QX@umN-JoB;0E^j5R*w!>@%yQS%2~|ZbKdDb|AM75H|u^QTJCfo$mr89`K74zYO=aUVnDV!!T(JUlfvakMhYy;w4f5MXeAe8+b@ zk%7VCXcRnx!vzLciTIB9x>0`x^hg?JT&=gaZE6?w;(I!(;x=YzOl0X0PY6Vl@4cA;csGT+-3_#+q!leUN8?xDN>_K%8dHc6excHiuZOeRF?u&VUta+(%yBrg<$#fY0q)mS2 zUfJ^ZaebRDy^EuTt*GSlm`o$~8pS0veYLVI;3cNRk8mC&QZ8-h;{?VKyFDxs z=uO*u>Y$7NwXO@)&kfFqsE@(oHHfB!G#J3*-y-esz@3A|pOnoI!r#0D79WMNMfT@X z(qKBU`0ClgPlu(hV_)@B@5YYzWoR>rT67TY2~`Z6R5rDop8F5DkA&I?XV zo1N?tBWpf5{1FZdtmSrVLo6aHRoxk0un!yNy;mMOn;u+M@9UD_;@;kTsC5JW1PACs ztbqU#mAVe!>=h~NwcAdKl@A78!kN=|@MUCwt`$5RWl<1l|)C z{^_mT9u3NfsH9*s@V;f3_xLk%dcCjQ!JdR zX(fx0t)rD$f|5k~)}F!Ah%S9I+vz-sjuzvco6)3anv;E{!eOz)2Dy;)8m68odcw_h zoLyI(dCo3e7*_Ks7pZ*wOZ}M<9@WFS@dZ1#WXb&Um+r?W+M&t@E-lOd$GdX*6yYM( z-5eFi3ow5nMQ|R7N8eZ++QpTd@4=amlI_5t%*LxK7w8-pL8^5Gt?dfQYre`SsPhC{B&SE2lv`jb(o^H@o`VcR1!Uo*wa z-0ZPp1Dmh}&OLER!S}#iC7$Bv1U9v3HargBa<}_m1=HXD_A?wCFn?P=T~NQ2{JwQ2 zc?mG!ANRgMrD!}w+^6SbEORHA;!nB{FkpeI;5M8q6a(yLV{qq8AAY2N@}y*LIo5{=wW7gmi~PCf6kmVr==2`S5O}v9PHXnsGrDPRIwoI+2Jmy;2tNcf$cbjY{S9nf+e@7X$itR zAq97C2-Tw?%97WyXOPmd3jctEUF<=)f9A<21ULZotd{mQ^UA?ZB6eqewpe4QI$LCFZ4ZMwY|P#IHwdg8}HP zF`Mz7#v4v{2h|m{S)2Ceg<6wz+{OFr;m7)iOqZWC0VJ0kwHtk{d@XWg8#hm$A%*=U z{&IyD1nn}UeBxJmfZ3@$DiusB12{27QT2xJApkb&?^??6bl4gfD$C#05oJm1@bj=} zONIX2s$Ay=^nlxxk&@C-C4Ev`fd9GzwI(MKt)$F8YQsn9a(Ic3aDy?{U__zcERo=I zKCwE!ciG0cm9m=-UG&lXbcx)zdBt;ox@r0_|Ms5*sWWlxY=mjL;3)gefeF__ zPH!ZL_BFEj`%2U(yAQLIGV1$L7S^Xt&0raXn{+vG0%NC$g*eg07~8wcd$VnTKlWi# zIvJ+Jw5f2;3FP;Y)v;4?7bK@2A~CqH`yp|^2!hVh_#xriUAeD7lL}(>akW^&2+Wa~ zCqvD}BiTKpPII4y|6(_?v-Zv!x&Ec5sPHHenUg_2%wEgXNH5bVg6QHH^g$WS5^?36 z;MV$Gjf)XNA}9Sa7~`kV2iKr0Ll!XcR4pX4#ML)0Z3_dKI0@y7-67SB$Ph#7sq-pk zEtNM~D~2&TvQAn;k{frxNVs>LK0W|UZsk1=cZrOLb#T1pC;1gXnm&`g-0yn{J7i(k z?4@>%TD-~y+b~+d8lgWxaMYfSGYC6c4p^KPL~d~xYI3%V((eco<)cxR2=#vU}2k0Z4>P~XmA)w5K^c4G4+(1)M5sgvS;dj?W^^#rl7b8$|Q}cQI##r?*B42%Pk4!R{{Zfgff< zP-uWYxQa(LP+Zu4kN9MPCjm<`j^eLsa!q41-XU0 z|MI75pdM#!!Jc(ojobR*0m(Jq9uA^M{xg~x(;|gn`jD^r`H^LF7ot6hWy0S&H{o2t z;Zhb!#`Y|W-sq3O?4|EAR2YW?hsdty*O#UO4&lY0(lG>FB?k$C@cAd;5ZE^Z=&m^( zg9orASc}#3*RO#?h-`%qTG{6Yik^F|;dC10v6%}7i7;k;I=j!c04(L>t>8^IS&$kD zLf9Q{0*a$c?2GzI6a2;dyY&xL#4p|*d>D9e*Ot!>6ns=?u%E9;*BWV3Z8{w>zDs7^ zRs~MU+KV+bXOx?y*IV|} zeh^?(CBX~#r^#Ie1RyNvDO0-+yyzmu+>2}o=8yXy6BU^Hjsf89#P(^q!7DT@P!9QH z|F(W&B49YQAFSz+cOP#32Oy&T@azl>k5l`Ib`cQFu%DxeE;gIL@PlB1;qx7s#vGu- zJM}5mK!>N9^8a%gF7fHr@Rnv^_&pgBS%+CYc9Fvvn)&ohhoikdUrT`HQvk3(42KQv zJ_MPwd=;K)6$l7(2cJvM06w&mHhkm`!s#xqwXpwh_8M#ojR#F^@(7M| zoBs)bzEF|w*n*OPOGNQUULfNOt#02rP;ekfd`PWSj`U`L{k@TkN zFHhVHsW?iYQvdnQMai1Eo@HYob>&zhXAmHLK~Z23@TaUYZAoAn?UtpH-?U^^5@BCk7} zvG(9Q09Kd{UEm84Lyqr5=h;hl;yirj!E!(GxBr0g3qj#!D%n9gn3o9Ee7tO-^sH5d zajUJ?IQvaoQFm>0{WPfSFc{d3;=?Wmv3yD*dj#qavGqycb+du@E8?JWgfRjsx5EkB z=YsrQU+(EL>TeCgz_FkJ?2Trqxd|}r3+jzME{J)%t9~W3PWcDFM7gJidO^4Q4rtmk zY?}P0#()bI&^*|uPz8(chepIiAd^!qfgr$3&Ynx=EN8j`AA&~2YZuHX-33lmvfNZf zj{x!Gdz*|bN&m6Py^Pt!|3ilz^?kPU}H+zRzCB*(BJ3kSF;>mBe?8Pp($L-~3G2A~Yrba}ZP0R(fDH+;C#Gu-u44_JHj#&=I$q_-1dieV051|2_;_q7ksntOk)K z0#LXq`v&`CI}`AQk0%(69ygI0-0^*TKFQF7DGo?m(@cml%k%g?#{t;EfKMgT(q4E6LJv{hpidX`umY8C2Mvpz;5#%{uAu%;%{z5ZD zgb7%En>ZDo=nbF)R+xA}y%zxT)9TwaRv!b9V0!A)@e!~Qaj&p%*D_9G5K>$bsaoal zqLwZA?3U*G+U+joN-fRo(G!B)rZCrd64Jfk?-DlK{K;Q^#zFnUf9|1TH)6{)s z3#y_6NHGmM(jF`-6bo4q}76JBZLQ zULd`>79^JUi{E7BeS$wC#S15o5ogQyn~TE(%|<{;#04{Lznrsr4PGfFgbC zGogdVaMDgxtlu_C4E@1eT4YuP2=!ad1hOqtMT(>NjlSR+YC5;qGoPd7SR*>k$WuZs zQwLoD$J@o!7moG^n5?5>$ifT!YWI$@Fh1a}lx{LgozQIPtT86!Pc}3UehITm$TE#s zU6A3;f5Dz=@0)Xz;X1Na3B?igz!qy}yA0-VK>W_9%w3QK%KRD7LHyQ#E(QS=#K_tx zCapj5+XSFz8C4CqWd&6j)G1jfsF!{aAd5%@*PcDx+iJuo2++3=m4b$A9zbFs{+ua+ zBES^60*gb{fxAM(b-pSn^jDJ5&dURRx@q^c{(+rB;7U`;<2oQ%iGe3!q)ywx`9TTSdpa2i<9MJ-8^_F)eCKcec zGB>eeiF=`zkb)SO!&*3?TpJ(zYJ3Ci`7k8!YbZkv8P0#P$akao z1)~wn{V4$r3{dYO=Bo;*g8>eEt8RuCz8$!g?7!z8vbdW24^BRUp6-_VLe)x%`w!Bf zUA3AP3+K^>L_oAU;hT8~-wvuWACa2$5cfePqwGpha6kpW`V|cBbZ|40@YEQfat3=h z_PPq-u2k^;jZ5J-3w1?W^WoKH;-7?v(z)(IZkYe-PsIW#{Hba%{H>d{q_4a{C5F0p z1i_OD#1S53D5b*BgL0ogbgE7mI=B${k>lNO+jco0PfzAT*M5z*z-dzAVGd~!M8-c&Qzhmuz3dJ%D3q1TkSVnh+Wp>EBwus)2fmLO? z(G&KiVuIe;<;RL%VR{s98lh4@zX}Gwui_n14&bKoJv(1BG+E=SL#3AjP?224oL~uP zD;_#SgW&Ilx5h(htH4YCnwQ@YKTva3zv{l2?ECVBnr=MPVq*Up`b$P?DK-3_asXI! zuzC*f0Y#Ab7};!13^4c>zn<@VKs&KgQfKZ%0~3ER9_gQT=UuUFkn7m~N3dz+;A5nI zqJ-b=3C8|86d2n-!rPh@j7{mO7xf-E6qaV{_c-X-&y(;d{)}yQHMS8L8%(NaF_5QyC6u|x0<_=m!gIEe+UTmwgO$EHqBl< zD-+=P+9{tu#Q_G^ZukSw?lIsm3ik}j(*fVM!gufv_>;3LLOsVT;*HDP+nG|cu(E?# z>3h^s)h{j%vb9#Q+cJKBturz1W`br5pmuC`h4I@8fJ@%LmH25GN?f9YZ(2PDw5#Qt z{_-CO=c>XuZgW~ZP*P=;xuzN^SAfef;-rJEg8 zHs;T6)NLDMfi&+95hocjXaytw_pM-ex`Yjxy>_2U_2|lH1Vk>c%G8;5Chp$2 z?E9pDX<2OM?BJ!uLDc8}fS~rPVMzqN8y{`z@U=d5<1E;n;04VK<~F5)<1Yav%(e_&sAU$S1s$DnPb#z+%K3GC`XD*;)(0mM``9%gB`W$ za%HIfVMlobFZZU%@*fP9<>7G}42-KC2U4)*KN+7)mJc!?Vri(bH=Hb1k?<3{o!?s_ zCjjW<4w$-*XX1K@;X%}D>;pK zRM_p_FLButvdyc|{E<->YuXZPnvxrlv2dIcBW3hCMeeE#-G5~BqeSjQ-g~xb+Ww$7SH{`Y_RxzhQRVa6rt3=rbys=qgevJSbkHl!whi-Ur5$RHn5#Hf z`856Uq-Ub}BHEsFu*eX3S6`0ytSm*<>%jW&-gEe*RH&4$OaTd8H6Y^Eq7Ja0g@B?* zNHlqagbg&7B=4#{OL`1|P*L3E9bY0ql@pyAq2ELo7G4^_)kSh|$;A&PmiXEURoeJ? z4tLX2F9Q3hH>U^Lu~j1GAFXGD>vz{)^m|9XEUaDK5d9elEc&lkvlPd*-i>0Yf!T z3P-9;hZ9pb{&bwpl-&x%k}OOT=}yZy=iQvd6V)u^FSZX_$jdb3DOYY^)9kgS5cm4j zEkvqMwPaVl`vua^0r4Z^D?3%(qIFvI_*!x%6oksFEIS-(J62-zfx|`=y+<);N!)pQ zyj+6Kb>pV$;Esw7f~$L3&a~cGb*4xq2+Vv8kDWT$_al%cJHsko(v%z3oOz{DZI$0R z49*t3IX@!-e-rRJ)L&-}2IX$-Zd1U$1>pn#qwj5~6e0cex6O$eGJrclrFhMqWI-W| z-YffDf*bNSl0|pLwc?YAr%R&a&}q0In?d~7g|of~WpVTyeJArSER)80Nn2a`8`&H@ zNAVfN?w6%EoW6)GoJUS4=8w|&WG)=*9h7TVgX)rx;q=F|(m^ zhutKDcCxo^vC;Qs3T`(0cr<>29f; z46-0Xk$hXNn_R6cCEnmKA@1oz&uJv71K>LE+rA0x6|~K->{SLxni7Sc^i>3qG}NfC zNqPX3e1Zf9^%8FJDjIigi4Brgg0WU0jJswBdZ{ z-g)ZRdD=6nl|W*i=fb3^;r^V ze5K?FhE?D3?u~_(jPLilvc-QbJQx3_N?_}< zx8aICp9%tQ1~lM4YUQ=bBR1dk$=8>k&K^%*b+d1v8QSO*6zfrY?_PQXd%j2P^BSA| zu#sbaP9Vp|&k9C9Y_R_-v=I)Lb@nLs!)?rE(CI6?S{C%6)raztd!8r&-Eb|oZ^7Y! zgI3wTegS{@7RDDfC?=4sE>lf<*X+A%vplZAJr^R7Jx%}GVL) zBH!o{WdVK4(_*x{R(Y;7y5eO%fC_)hD5hCt-<^ukwVEEKyN#DOJFAcLGR83vCsb;Q z>KMWE1Ax7&__qWKO9?4IaN)B+Geq|9vb1qIv@HEc6R-qVc?}LUFHVWcgBQX{P@~en zG_$68_fPtf@i0_s(4!C?ZW5Y!A%VmTND!KMfvPkpo5TK3WphfvhTlk;%2&M1{4(>J z{nqrWr$%qi7U-khDMLGRhr-S=7(fJs-21<&n$sc+)2gl*$brq_K36LyxnJ(iwphD>|-{v(S3#9Gh5r&}u(TUYK(aYa4B2FQWu;Fg?Hp z$OpER0O*-ZXt9YGTK(~3CkvDW2?DfPBnc7(XtC*!!4}d&6NF=FP++?E#J1^QSReE} zkU(aQ`7Iw0FpFSTBaxR z|B?2VVO4ic`!FD_0)li4NP~cMHwY3^(kLa8y6FxPX#|um5s(H!x+J6<0qNX?bZ=_E zYb$zP_w&C$z4vkOncv#8CeE2TXBHyUE6oOZ!o8AR{*|-m{E~*sFKJA@E9+#say@#-zxgdtx^<(vR4ovDF@E2l)iG_Iod&T6QvWRP7i0|>CE&2(5YDLUX2FzTKb<#KYtK`@+0_~ z&|g&McmAv81Vz+O<$4UOX-LigO97RO`YjZ%BccgypHaBwDLv+hERd05yr9BsL^(sC z>IVA@pePHi6f`YB?1+NLKb`+5rC&}fmRQu{yIDR&m z2Z`c8NQg2e(m8;5em3_=aUtsX72DWka3)Xl@ zzdJn$yjj^Lc=EbgonVR6r}JEx3raUkfwD%t3c38~^EeF2uLcujm)ChPI!l0s-oS{< zVBi4lr{wr+W{-gty$LA6ITT@ZDM_1?g_;`XOiL6pN=Ad{uGjEFV_t^lEsN{Hfl9hZDZ5wNR` zZ|~p!!-ZMW_*KNtxw0#hPNtO(DnhD8T_G>c`B+_+nQGaW*SU`Up?C^8O(y${DVZO} zFh>xTX0K$y2hNCl{%vni)xB>Nw6zHH0ygoldsh8rUVrX+nLaaF+CF!uRV4Rkxqm*i z+xR4Ysfz3P75G4!gy24-SLTQI|LFsFGyiYR1DZMjE}4MI7YfL(P1z3D?2AoA2Qz{>@i}1 zpK5U(zXnH-lVI3qG^ER{#H0s-k00R;Y!Skb;LKFg2Es4kEVR@I8icUAe>7+h@Iz-l z#Q)KtA+y-YJ!!W|ZJGL=-));7(GTEe>D1QF75h6tOYi@`1?S!+UEb1T2Lu&$)$?EM zl!$5_pqWCbc?sfcB&78+H2?p7=J)^Ndpv;e8Rt~nJTLVzJHY#2WS$bn-T(Huf-UAh z81nHR^56sPYKP~g9*+-j{uj&5Hq%tooL>ukd9?lhb-4PAA1Z;1+3i!(iPB9*eu>b? z`j!ut|BG+qWz769u;$y868M0MU*bw+O3MeG|K$Uv^qE2vT7cp4pFFu9(JVpK4T3l3 zm*f0w?yn>aK|CJ~Ob7tJ45EhcW%!`>MX%C>4BROh2g05H13#*N!H@A@@FTt!Ux&Nv zzqXCzJ(R#LFyM`yms)Ecp!=O7{AVPOd}k5xp9oLG05}bbh?s-uK{=atcl=BHfA^rS zw4b5REEZxGi5)r1mLit-ho=%4@1YDnK!>+-UTUd%fcn3DU<-U;m7Op(p3x~q+u^$> z>^gM}1wYgXL;z>ZR#Be~62f|kbTLGTS?p2&%nF=CZK=6gCkQdoh>%POLd^6yH}^jw z#`9N**}4ibig!qtsm9s2bI~5nu|BJj_TH!CI!=V*l}OVB!%W*dR?+?k`&DEiUH-o% z*Db+b+5uO;k5JV``9}5{6(l4vxT1`trW`Jwf)Vi02!}k004V_P#1#OLqHCD1U!iDWV&h|yZ^yI9;y(```R>}u;KByFC0;$i2)4+5l$+=B_sY9 z11q@X(YVnvcA!THhx(5m!G28+^<3#uv?F)d^cT4qRyh9QgmIP8*Dz?jyaf2?Dl}`b z;i|VU%>IXe{wUHcw$e0fYK%J0K=sm4pHMlKaG&WJj^8)T5!1{27G@NLf87un+nh@x z&Z}q_#XFSq6tS4B;ZV>|i;0oo>5&0_WQOJkng9F-%=^lKqtUYJK^^b!X&+Ni_bARR z_P?}uECQomEIm%=0EOy?8vtnnluJ2V7uhFTBX9R}^bKu%sl?Z;VN=F=ird1>H=&^J z7?WwjGcpdK9im0??++qKu+lJ25dk#VbJ@sEEQQE#&KFq}yx{0jHQRedgc)rHZ8t?Fst$u;x z_U{%3EM~ZgXX7yteOt2v9VX2{oP-snA<4{`aNQp$fd!0B6|ZtAo4G@W<$dbr&(yH> zf9G_vI$hS|;|IQj8mR7=BVLyW0(^~kq7dyKt4Nv)an?CU3f{^bc2V+@ z$cV#Sx1g$NuxC?H>F|XpDZ*SO{nkDsMGvy8{k?}VE!X^-WnAzP>km2Jj|`;sW0)i>z=JK0l;;=9F=QqK zs-Ww+#oB@T6>(?(=D5G10C4S+sARl8=xapwbWzwqsJWN~DI4-lf-lsJei8{X;>zoz z>KKp{j%$&O_ET9Jafq7E7+frRpKBjT37kUqiW?rzmndP(6i2!UVjTK4D`Cb0LKgGY z_y?~6H4~zSNqAi#jVk8s=sm>cEw;SM{hv=E#JO(?SnNV6ui{2JAJEJ%u(le0G}vRo z!si8V0b(7{XA$A*H(++2n3`|`)yki}vQNJoGFT9@q;!T$(5EM>ZwBlz|TW?)@ZMWbLRmOr>uLpk}X?zc_*0rEqdOwY` zl8&fN`56J}8qwGJO8fvg_ty{w-l_&o*yvCe@0Xs58b^;)lZn|b_(obnVJFs+; zHcb!MGh$EX9wg?_7zbSK z>;(!E2+z#(3UC598_Cyf1G+)y zsJTkNDp4d?czhgz$Gw^CCqte&dhfSjCtlJ*-wsir>|P(|_F14NWxo?uho~`x3rcOG zWZkK&1Z20mKITsH=XN=Q@|Pcr6JDD(q*yR3!Wyaxy_pqBr?KIWAYab!1YVoMPJ^gz z?q&s^gXCb3+(XW&7DP@k<$>%cf@QF?tiiKm(#)Gp0(W`z#5I+Kg!OmMp#6)sBXR`^ zN|_O;n{NA^50{{w{H1^U?Ve{5Zsovj)}iqArn$Ellt z#!u^}D#kKhT9m%UPY!dc-Gg5DvhB6LiZBBbz=m$NHl7TCRtZsO-)sfLpV~q7!~84m zRTmB)U1zedEpSgwH<*TjKJrk`+{WdIJ)Y+Tp1jkbVK{4GN`>cSXpGR{^80Cc=hV01 zv+4d`TT%G10o!(xSQxkQqjL|m#iF`&+~fd`xj9@^7ND4#t!>{h4QQIkwoq^`0l8HS zbt-Z;`V6?!H6g1RUgBxbqyFBMY`go_4YXc1y-I%${R|vBJVGLJQ6p;G?`^4Zp^{v} z-~Oaq_J{-F_DqI@6lKgDX((rlqIOdaEM~*Oiqh{fF#KFJyg-Y39`)c9Q_ zEqT2d0A?31umn=tdkwR_*($wM%M}G))}51HCyc21&}isNc~zug1KXhNCGq;4{Xn-b zc;mv_oV!az{>vXGJg3%f>I#_4Nfq^!i*-k4T~S@vB9$;gRNWYcm`N2(FC-nHfInQT9we zA22HU{#7o4-QBzj`N|{y-)?=+tVxVh`Jn^2Z;K=pw81b(F=<~UHOrE-_$-!< zjp3$Jr*s1Shlk?C#8$#fv_-C@{pRpM-=DFn4s;tKEfArK0nzz8gqW5p;0IE3OUEk z4@%qc-jN7r2wz4$rV+tfTxV8>+}G; zsSHqL$N8S^>c5SgRWx}VK;fp#Cp?M-%YRQV5n=y5(;U*j5z z96CB|aFef+fGBkY+CzJPh4!ozfJ|$y|0A1PvK`T(CqS`4^4Ra9CrQcX5&Ikra(M zkm(h*(z@#8q17$F5zLdc#9HNgn&UZZ^|+RdH1&I?e*Q{1ITo0XefjF(+r5^jJvz zp*l3N$AiTP7`i9io8m0j1z!+BTTEIIRZh_l?UP)T>jpLjOiNB4SKeUM>&y8k*7YfJ z98kUkSSpLv=wOrsL@MXDfEgM!PUPcb`))7$q-2$Sb2W%1v(JzdR7C^AVE?SXrs>5M zTA`r@lEwO}JEzN%%ax}u`Ookd}bybX@P>>MR&D|AL78 z?wX39Yxnv5|DVuIM>kYQ67Ldt-ObewL8s5q| z&gqawMJc;y5iGJAdnW2#+lAj*}1|kO7%~aDheO+c@rBig%kQDLBNCz$xOnhT z7w}hkeHNa_JL7dAm=_eN3AA_cuwFqql#4q(d~!6;@4K_%M=C62`u(yi3BF&UU+2mP zeB=TN@wwVe1>xIQv*#uL**X0;4Lke^wR&}pkSNGrR9$7-TgaZn!7a?YeX-&f77!iZCvF!C_v}#Ju@3ls+-*Cq zgM35@pnpz$BMBM^O5d8ewvqmH)vE2hxk z-G`OW7NsLskI3CCl_b3-)-@XwM$KYtt;6edF5AxGx9<)%%yEJB8yb`tLTM^~<$Y<6 z&xd~&04_tXo(wyQYH8+~+d77Es1QL!@-zIPS+)4Vm0Vm`9Bvo19kFrBX$A>s7tEnU z0ZoSwkz5d=hH%h;S_1Nl>SC=tE?$Wq*wgWSJ_CX|a@S;wTdSrewszYR8lltxlUiZe zYrLoP^`rH}S`}N{GXAS#>0?0S5W{{LlxirZS7{rSXa`Hxp5|6oovD2UwUBgQ`rdLn z%4>Rd!{lf7@^d{K=OK1?v&GHlK2N?-jjOTL`lwEvp@6t8?Zj9sYDE1N`gJj;Ow z6HDOPeVQ1f{<~{BWv~mq;%ai=B0zsj<#)e7?SN@`xB%ggo|}>La2y{uzCGJ=2F~XL z>%5XqYg|!S|I3AvWQf{Y!rtKbJ~CJ3p2069HPh{tKMJsb@=q*}&&DcqJtFZP zbvSB!MMaAj^*eHR8lkFZ`Xh7)8aq4)vV1$f&Yb_inF{i@-j(e~3)N8N*>G6NFrQTi&I^3vM~Iqoh^0;s$E8mK-WKI#m2d z`l*F=*R)g+z=?h!_-2`=nfLdp50`CQclM)hP0K>D;H;%|6Qd4UE9tA<*X`}!PBt)- zJrJ-6InO6vUg*AF(H3!e-za?uHhe-gd<}I2Q0**L*;cVozF=3$&+r=Cd5wzjAH{si zoeQyER!t)J@+%vy_(6Sz@`k|9u?C$2p^=#iRpGIMk%b_a=WRUO>lykvM-zFC#5JLt z?9)0OKLu~v>J@}TpTj0SG-vu}{-IV~u;W&+=Uq@TW-DQc^M^IcIfcO z$~l$imv|R}szWc~-ms>DgN6&fbczlsLqOt94nl#=aWjy0&`0}{X zW-EJ5UdI_j=c3+UY2VE9dq#qp@Oc{GHm)?a`Ab zRicz18LvLB_O5AQ*p}>U(C5?6ntEAdCc$FSAS1!0L59YsJW=O$qN=4`P-LRJwWabac8W8;kbh-C_~ zF;gTF^sElV9!2D4&!eOLeu{gQ@3$-O%w`a4b3WR?mhD>1;C{H1DWvbQ{fu2yx&f{~ zYR|JdGm>do6h%!#!tBFptrH~hfAL~9C2QwttdDZxruQf`^2>lDTMUYU;k(X~Z{kew z;Z?!iE&0Zf2^{%|IPsP#703wTE;Xw;`!{EtJ>liV{Eu}3#m?k$cDi~sdY3$D4-+t_ zWz#+V#-XaPATuRYZaPqA=E*Rlal3_i&5e5FJpdnB=}d0Ygvv>DK0{++NRez^-R-_F ze$xl;MIl_2%2|^89tNj%*U*`xbR-I_;bFJa^1@9$OFO~F@^{(uVW;k{k%Bn8M= zp83Al`U?#A6!wnU#!X+U4Rts8^6`rGQP&f&KnUiKU_8b3PT1=z%LxX}JetSaXZ!JI zM8ti6mgHPsZd9z6FN`sCza3rYGzu5N(Rc%4&vcBeW34=#U2;r$(_o-e^DY|{n4rmh z-zOQys`@4Sc4s?1JVKKr=(3&K>8*-+Iwz{thbrC+eC9;q!v-A+8wxrvDH>)Fq7*xa zI6##{fDxz5n)hdIOi<>xfYM&-_w=p?9{ff?V#lBH^Z!|&ViCk#5g8G( z3AU$r^G!4wcokZR4}8cb>|b5MozB}=4Hlo2$tMc88-hT_)^?7v)*Gz6^55gy4xvvM zUrWI?a$xlYoKW+!V|(qGD@aQZ`-5mV&?0Z~8qnnL=ggNl(hvhg3%Lc&Ltk@Di9Orm zt8RopEcXH4x0=PnRfV0@`4Yg{Wlm_AKp=`_9C){gcDn^`EkvEzE*PQau#Bigv@=uOFudZ~`+WqE zQNG#sv~Qr3MHJb)b0WNE`!$yPd@bal36e#es=kt0nP{x+=iR_OP+2&F^ zp&wJ9YY&FYWmt>2h$md&S*Ae|vYS)SIpV~P%Z>lFdS&Q>Gch0PVH9BxLbVV2y#TnG zAnEd@b57kumjw6`=2hK{G>k_2`A3QOE2?ycg3ed8ns*UHq9?OZqC-xFHk{mh2B5lg zTRBqR~pv<(_c;ce2{L-(RhKDn1>AX{f>RL}dz|2!Vs7NIS zO`yTBB@7b0G^$W{GUXG3vWVIQr{v`qnI6Br$$bQ8RoCmk6X~XJFA*pB*Ma3=(8(;G z{vh$%vo?5Q}74z+QXr$CAV1X6&!R za{N$q)5LFnTpkd_1D&D(8l8JGIg6h_MxZU&ekAXK_D>Rut=Bo0-I*33<<`h5YJZS3 zxGz^*T##sARB-ydA)VCrE?RV&c!HU!8m0`$8o|)CNv>~ockJ;b|M z$0^#>TE~#XAp37*W&`qXANBh}LOt)KpwC?sxX(+Bj?){TEk3y1c&1Zp<&4<$B7UYG zwT*H28s`cvxG~Sdx;VI?dqiF1FI3HgC-~YVK!OSyd9&|;HVG(j8R=Mvyt&<5DnhSg zctP!|evNERwWycF+CfNC*CfuN5XRx@XPJYI!9b|#e51zJ4Dqtp z0V~8;ZT?O7Qvbc+7Lbx=T(c*jhLLQjp8w3MN^LW8DI~W@Wk59?DGMRU7$gxPV14U)4?OZ$OVNl=qb49Om5?@=O0JfnDUZ;yx({z;w z1t)!~%0&Yj_YRAfd@eaNXMYX{Y*1vq1-NUma;c%#P-Sq6=sSD7K&m2=u${9nsbV%7 znV=8%&F;CEv>1liG13U#G&|9!&}>4C)dk%uzRMdNf2Me%+!0zE7&h{1QSA)96j0;t+pQ0GpQ>K8ckYH9v_#r7c+@^B&X@2+6ZD) zx6dGYbCz4~b8?jr*bTXiYhI>OquLOb08+!cbR!;#0@w>DAygM6!HqKLo{xyF6yZ2{ zAP(#`9=K#PiE}4h8)CUiQJ28$b#=1yS0us*MltHYNSg^C&Karybmk5=7vkNlI2KXd zQp8DdSzGdBKm$QYl7Hu3)~k%k=c)YOpgJEq_nDKm>a#9waLeNlPrnOxY{%BR(MBp=L2 zgt}jyVG-|AMuNtx&y1(>@=I;jDUP7DO z%Bd@n7lRo=6@cIOMO2FSpnpng_-w9mY3nT4%&w3WhtwQD$fww3>K*U>t|H>=9%#@>c#PqD^c=s$`Eukwi?bXcZWPR~1LDRW==6eyZbfys=2U2j^AWS<}2I4@Para*HcRc~m6AchoRII6#cc3ld`XF8X+zkB)FAo@-h< zVzird8F#|Z*&y?8;xq{UPTn1YHi9wkGIu0cN}&AdwjG#=6MnX3-wRRYpEiGvnQ{Ij z|F%IROW@$g?nAIyYeYYJ(OeOcG<{g3@|ZPfE7R)BY5!pM)Gveqax zPd~ptKC(byhs&?=x!4VL3iAVl3BefG6yqckp{g}zYqnGfU`X2)R%(YrEV?cq_yqbw z6+&q6!gji*$?q@c2)HOOu#V(cU0#ZNT}lmr`sMob{z1sn1zIF?{GjJN`gx!PE?}+V z-td-9mjb|@&|kO0Ef;fp^9QX{N_3slhm}^0m?EBfN4B3<&b2Y$AW&3fexJ{s~iauTYa08+nW$-`8n#p$Lw^GdMCwl;9}1xijd7 z*slB(%|Ip4&jv%XJr-a|C1cC!V0Ryn#}y2yP;p_jofbsvx@^kM49iiI0FIK50 zi4WOGjv3H&yjl#_M7lnwS~LnMtw$rI?+%s02Vn!6^J?5j-p--b4b?$9-xlZH|{^3CZDPivs`ZH7u zqQU5$$S3(aKZ%|9{=xxE4P}O8+y-s6E=nstn z7}OY5TY12?!vR^2P^2&tG|pmmJFyCrnCO?>YHxfG>Jsm;4_=CZ-qi!p0p~ug=lVdw ztBPWkChitOX!->Z+>^byGUV58IrzZItGC~eiZgytiQBrTRU_-Q7NPN{i-d#{ju?w$ zf{8_dxm(=WusRv;yfg`PSvT<->8&&InAgU63!?q36#VsR;ckM(pxIw1m-?a|46pO! zw4IjECyR`D3HQd>4k9b{mxtj8K}qAz0d*i2w`}@ zaz`~@N9%c~>_?K1;lMrFgkgJqHf&lGdBSmevI`SNy-994DgwHOmvx(lIX-{t4%ZN& z6b)eVU*sFGi57{s@)oPe54Qi<0Dp?x zlrih8^7U6Q#ogI!Wb4mWGlh}gZDDHG@TjffNd6;CJ}{nBNsDcHiXgF?v`&E$lK@}m z|E%XAqP}xfimNauLYOj%FYY3QiS_+t!ZBhq&N6j>6R>hRrTA;!xdJq$NbybH0HTy2 zi~&o5zQyG1I$PEGc2^UD@t4Q%fZpebjqEi{1GcDY7-Sa-M(&4-h^Sy0NCG}+-+nE~ zsq7HEl)PWG!xICrq)uC8m59co*}Xq>N&G80!5v=+_ZXS94`R&^8Xm>8pR#Yv*_8;El;iJQ9MOxh+ng~E2EdsBR% zUHtu_{Ihls`vv~)>{$X^hw5ZAu5b$2m9Zz0a^%G#!YBWimEATa_eE`{&*q&j@KqMl z-A(H>AdKR45K=W?oUm5|?w5v$)7=*W?esW}7J9TV9@Cl|^7S7GcF-43dx^&!*BVYo zDlvtrIUJ&Z4t+r=J4R=NMD~j6QA;}}Iby^Ju`);dx{NU!n<1gk`mwrrz+cLgxl`lX zp9L;U`iP_MUNwynN0m(Z%kPHa=E3%ei}-o5LvAl$x3E`&svbS9eZh>VhPa_eh!PVolfiRZYY@RvIvxx<5NqO zwv?p7-DkZ1;^_@%u+y##>_N)=iUQ=hm721L(_kU=l&aozlm*e}oZBnrt1_ja`jY^V z7=TQ8PR?o4-`{ik(@$@jX!Un*YcZU=J>1_exiH7OMV`A^vs(KW@*sA)X$p`krvUcA zC}~}POW@#sjZ**SdpB?X;1;pB1zud@D>VCp7XS#r_Ba(KQ^3h$OhW+-qveXrq~s!kTS$?UohvK8>c8<`lm$XJWh4LEJPU@C zk9EUg-5SS!){5HYh8A{_v6dU%##=0K-*=lkXvh%lV#ys#9CAZ9gPIg5BeiUClz@$m zNS>hkOrIo(_)GQmx_zY=(`i4rG=|9ItP1Y^rrs1aa2+XU0fNN4Cz-?V z5dp}D=7A28N1^!p;TNsRwF(rY;wZaRT>YR=5K=x`_%gDNIO8=PV;6kS78@{{IJpbW zwQ2hI8OtQiO`JgHsl)#q$1KBl_{9kDJ*l;(s^#d+(_oAX^}2^r+#l58w+F}zaVwf} zg&00{Fsj{29BfL0C+gvcvBrcj%Z1L9s3Xv zEl`l@G@a1qvK4qIbh*L%zK4FRdR$yFtSXeb#G>yxGg9z?AZ5i*28-!k|#aJ7*RSY10e&X zlS9NRM;lR-M{hWqdtrz6&G2s!zZq1&$8|^E0Ym~srAey0osqh&i`>+O# z($+*x0E*pIWiPv#xTwX9!;)%EpNO${(bCO&2kA?xCa~D#0C@WcfkURaY!00Vj0Vlz44kB9G_oa?a!=+(@x6S7qA&={uO1HpmumcM0*XUX|_x~d8_lR&gY0Z4pbT>WK0Pp6g-kN02_B< zOQ+0eFoPE>6C|BX@>Nbw+LHuH^d#eqFDxr%A~(OMg;J2VhZUo~p*DW^wU`@Gjd}N= zG@ig`kbOOkFj#D~b)GGFYov9)`MWOfn`wW{d+5_hb(DmevlpuPE4$@Cz>~Zz6+a`$ z4SMvQ?n<1G=z5)sX`%GYXFrHK>y~Ir-Xw@gCiH!>Cqlr*2nvSNV|)n_$TUeK*1>Uh~)7b;PPDZ zMkCYrGY>?R@Plw*?$1NGR_`_&P!rqov=xmr9liIke@RC@=7+Saf$?t5R2wzE#ax!C zR+ivi&P;5HePze0tdneHqwb!p=p{7&PImpXv1imMc)m!#L}vyn7oOR!`aEF$53iAV z4Mb8%kxnVXp6f7RBEg(n`U?bE+__tyMBdb;;-xO zO4i^%CJ0i3%WhD@B^L&68Wk7bLw~58I=2wi`+e`U#BA^L{x*K?-jm=QCzT!~9j$0^ zRz=-4O%-s~Wzm>hkCkd7Vuz%5}^@d74gRS`Rew&>9{k2;45A4DR$UkY4zLn^aATQ+HJGgIv3`Cj3 z^dS{DEt$YxrS3ghh$Z__YX%XiB8AE=8#g=48_#9)ANRk5$E0$?leL%RWt-RV4-1Gr zeiG|(HGd(zBXO>&pj;Czj0|H_rMLM|?J!dQ}R$_8i%($*B-(uWV8H zl;rPePY_T`K140*Sm8!H*ysB4#sLcidJ)HP4%2@7Omibqo?&7qGhb|Zj0KOzJ1_l? z7gGqp+MO}{mN;*Iz++J}rPm&un8|v6<`(;*7o+0^tmRA5VUBi_6ZjOx!^7)R+ie1y?i`t67|}+o!52XH2k%IQ#b|{y z<%bmB4{eUc%Pn5zv&#B%J$wGr2=1%oZnTkS|3D<)@xb-ap6#RsR$y;tR= zfnu>!l1d2wNG}Yf=k(~dY@7L_zHp?=UmthbN)6qmPc`iP6H?nzoF8%qb(E)|7(RT)*e>(c^??r+B{ou() zYe#jIPu`j1;L4d%zS+K+k{IKruyIZpLEcR9=bG`D7!l5JP42HV5~FWT~3?3IKsrUb5CdTgpt3chczHOYcB2xVnxCg$p-|KE4~*ZeH!Nzmo*@EA%58m~wK9aihZz2Q&{ozgMpMh-14o}l@&KroKv#ZwmP}`7E zOVQg^@uXly%J-xbjVD|)770=ng01gohej-Jv&HQ+FEf$k+BXDkDWpuxmDpZI`KzfN zh{gXK;}tdu{+`_(4dIO*cxivwK>Mb`rct_#l9r!a-k zT^@$@X_|^NG5MP&*-R*{s_IKtmEf797A?uPkV8qAJNGYqqISl+g^VT-JWoDZraTsP zzbkkd{13+PZ8`ynJF=f~OqID~B}91an}XbO$XVhy|BM)XyRi>-V)c}qm&x}V&L-~< zdi2Vk&+{TLv|GD5^g)0JaH2i$^uj;dB(e&7Q||OEV_UK`bkwzoVhbka%f%$l;Qux< zF@L$ZpsEbw;Y5BG#E{CzD9-q-lNtOEW|gq-VL+&fSX*u`0bl8ASd3f}Y8?3vNBSC- zHoDBhwhpEsQvrVs>k?FUw(6A~L6w_Ou%(-BnHWJApRNm1x=NwQO{)KWh2IOfiymisRU9+Z*J+(c{03D%M?ZEHScpnp$D^V z6s~e8qNmJmKic3->iq#Viz8v&PS=cSuRyS`FF&|_54yk-sTn$U0|dt89qjzUv)#xx z-(MLbllgQ^gk|tG^}{|qm*oe*5(xRjwZbH(qpQCT_}b-EYxvLOYIf2XY9$pEJ=FfO z=ig)oV*10^iM1X*x%>qk~xzlj7N%#T)C>sp=qX8r+n5>1E&wW5y#3IB;vDqoWYtK^%F zMepFZ<9oqglpc$Vp;=EQVt^0ec9pAZp?pzH8}lB0!|yjp4{l+{?>Bm+ zWwoZG0^I*g6j39d8g5R{ZBj1kOVN6k@NMuCdKF~X)nzvU1K@sKnWn%Y{7=0UMdjF9 zgS(G=f*TcFS9J|rcS^<1E$H=Xw5eZv_|n1inE1W?uauR>1)LQkFXkL@d#T+*<49FO z4wjV;`^ib~d3u(sF_@k9#_+PL?MsfA+eHRVdL6ecPiHLpv8kPn-!c&eB6{`_b{D$Ye6HW^Cirr zY!>xNc6BRZNR<1vopV7$rv~YiQweRLr;vx|KdO z%>*hy5&_okFOz&yf+Y6b^ntNITy3Bk;!-7x+I#9cRt!J1OoX9n!aFo?mSCc`uznqo*q@r5$9=>j@`;Bq(Qcc; zdup#$uO3s=5=Vz~+H<>55F*iYnL|%P`M~hY^syD!<o52t9Z*0U=mFa(IKU(jXEzCeIHLBE3$z;8;;w{f?@)}XQ~Rhcgihl8c6;jk zZq;;bNY!4xS_&0vX3J9?5q9Xoq*czsnqjY@dCk@*6~;BJzlYpO{w2i^|48u}5tJh( zrh@V(g8)PU5E~UtC#l|&Ldu?DC9&hvcO?FQvA%X_kjq+ zPlol-pj)ug@Vm5kphN|5QlFQ9PPij-`Y-1&(_E{`F9z3f^y!&JcDmA#c4NdLdIdOs zkNW5H=%g9kpQ0O$v9J4Zez<;11||5tx8eq+@hz^+cVI;4DnAAw)tq*x5eW$i4rlzB z){KNCfrJ=^->A2CB-E-nz9`Pa!{<j5b;-?u$cwBO zVJ`PwXO3;pG=93ZtBI_Ltiy(IpYuCJ;Gk+Y$)#%v7@cBPPz~{+;$a@6;zc`5(%0tA z=zdXDkm5Ld5Ayu*@Q}}yNz!DZ+^UE|ztT=iiVAz4 z(r-9NHM6Q}ycYe(BR$M##CaxlM(CU8SmPJ=l8*Bp5^;7`YoX!aRDBm2y|ux6;(7!Q z-y$I=o0K2FQ@vI%#nD@$NGC0Mt>PzW_W7*(OT>8!>X4%EcDm!xL!C*!FaqHW>M4}? zL*E&+(KoeEaL(U4e#*+%sF*}Yj!5H6-(kgn6XR>V;_xD`y_;L2DM7w=-zq^zIB2?_ zv!@jD`%<<{7@4yPx=zb>u)SJVGQIqI6IYGM{lDDg7M#9YsICH zQbh5@FpsXN^s+j18aS@IRuA=^ESZK#Y-}s?b3`A$cRVY6abDj4+EO8%2c;pY#VH%F z-!j72wprkm^q^%DiFN&yr(j+@CeTX!y_1zYbjd`O9_-2d^T&>Pk7gD{-r47ggjntQ z@DJe>^|g;by(aYz`8!~XH`3?13%Gcm^m$2N&UaC2&BR0$%SNP~2<)=pKMVCuORhd$ zvrpfi#AJ8Jx>wUo!F6-gH~0e){IbGa-#d;JarD;^A&EATuCU|Xc?PJn(#adVw2i(kkEYcHwQG7))Zy5b?r5G~g+h?=H zc~L67xszR_J4u-A$l?MJ{q3h;+#{pg)Nq^nHgDt_^g{)}GDU3nRp}pHehT=MK|*&9P{@_ZwA zLhu~jEXTFI-gLALx)k+G-cLrCcf)Rm8lv^+7B{hYjW>3kXugb~P9uwW$|@^2s0tFH1B5^hm!)ak~`PS1u&f$K}GijHa>m(*sB#n6dz=Z{Z$u^$k3RCBZG^HK-t zhy?2_Wq;OjdAg{vIPJlr%_+5pk~81-__9Z>z-Yj`_C`l#upplC5b3f#&w5$|s_7Y= zkv!Lo@q6;vlkk*f`#{gTqV19WH{@QtyOW;GGsc&p<9%asH7DAuH6$zCd0mg_hIR;f zAg!MqFe2V)%o)|fEv7J0>vY+^>MmV|#V0c-5dL2Ko$}rYIq;+B`-}o4C8`At?vB8bSAI2gYph?MPR<#F?ghJ>nLwC{+=0G zSK4IAsbzNgE}l$@yAd)`0cYE5I^(A`we$7Zoc$F34Eh$PmeXWm@@Pbau}{s{AOd^K zJZ}s?Q}Sd@vzrf%earVUv`s;V@!i8%Vin3fw^uM7#$PqQKpwamA6XQ@YZeWEw-<~~ zAuh&syG=9N_d%aQ;Zey(h>yI#v1OY*K(6Nmk6X5)`(HBBv zeJkykRVQ~=seHV4L-Dwq&I#B^E|1yud4}{M3XVE$HC4#wgF=0G+2TUSz_QvyKS6CG zz>4~mS`zyP^&h{Hjt114l7g4;as6qdn(K;byjTU7)^jxI^82S3Hdx8UR-^(hSaQx! zjJ~%1?9;LJL6*)I4P9P#ipRCYxQM}D;cU0wZ|C!yy-dQ2KU%mKt2EG%D=gUU>q+pT z$t#_Q1?lcPp}lVt#uBr&>d#A;=}u4Xjx#upS~P#A5Kl<&5%Yf_fn{lTA7wE;hU&DK z!=iS@H-521|AktqMlsT@IbkM=ZyOSXo6Vh%Ny9^Prt%ABGEKtWkeuxzakdt6`+&Q2 zlzX$HnC^x0pl5!9IMR~)p#--QX*+y@<4yqET?kRRUW@lShr{y1s4{uL#apC5QA*+^ zDoIphggz>-*1Pe&1$?%s^Zn4-nV`5Wa_4C+2j@nmrUq^QP3uIoNm{G_kFBqas(S0b zhQpx+lmb>}X-ZA{Z zhdb`rzrEL9Ypyxx+GovyKCH%LDuHri@ri#?K-LUiV)0J=4W@vNc^%U^JrBJL_ zwDZ)pEn1f}ml%_1bBAJkukrK`OGS;?+u#q~>-w--qUPW!Mc8s-M{+G&VPR`& z&H5ufxujDK*;B9Xm1ZH+o2wnYy1L=&K?N=@opoPI6ju89Hmc*5E(X8s^st|_CZVp7 z!EIXj^r@FavEZV{*`__gntcjhI8{=ZT@utCisdvCJ~66tMB&t>Yocan5;R&MkT3j< z2;I<6M$g%cMy&f!jmph@>yVVk`Ut?8dPev-A6Jc7AAd0{*m3@;gz4Gkn>x(m$*DY$ zFv}?6Q~h87TFeuK&Eq&2Q%+tG_vAQ4+63hhcC*)=ItRO4^M!Z!|NF{5G7!RAH-c3+ z8c7CkQ0QI?-9HAuZ0?vi;O#A_c3E>=s=nG2&p!T|BaLdxF-rngYeXN|<$j~n`AuX0 zY)DeU@*NsI4s4{MJk5SbHW4@y<_9TQVWD`M%ffA`6U2`@Ap^}9-tEi;tN&`CBKY&!QkCVt}kakz1v(qE0(5} zDT*cTOR3!Ck z^xCg01>=T%aY999EPt=Hu6wA}4{y##@eY)ZFz7Crk;0cwMMBv@zAivkCftQtM(@0u@CqGQeAN>W(j3>6u?FG5* zAGpbuz6ezrWi@Z!j+QuD8#{z@MQsf58c-ZVPkydPJ?eMt7pz)NrT53c!`~7@uJu2& z7PXlhR*Y>rI#7f)(T~y;Hc{Ne94e~f-&}=qv??u}JC)_(AT)kJXM2%hlzsW|d+(GNkv%k*ga|gX z{k$gnfwON4_=)l1-WCPSyw;ih{0*ky^>=!DWfa-G!hss!Pq0`!*qUWon4AIS(+lJ=Vd$!~dG$UnN4^Hyy**3+sEmjgvh2KI z?j&WfrismFA1C^;=z%uvxA@I;IH@2N4?S!KI=`>t?NI1!-jO`J(7;Yg(L3?BI&hXT z#w7bY)eY%Gik{ndM?=w}_n2O1Z@QtocO#ec2o;uB>n-e@&me2@iv9Fb` zAhhY|sgIF))}EB^-A-#b>UHxOMOqzV)bG(CwhZ9Tqij_S^ew4^-H=Y^9uoslw@_SvKnM#;xqTEvx}en@x1Z!cG9yl~@HEAv2pA_(l%PUcAuqR{o<)Ce!QM?WI@>(R)TI2q(1iLUUtG-p0f7+bn;kc z*s`g5mrNr%t#X6hE0O}VadQ>h?D)1A8BL%9O`zAzOlWY*ZbB@+PH>`MNV^+4Ul(?2 zxDD|0mgUN8$&GgzY{hrXvd2>pf9e$)u%4INo(b7L+^;LQn z!>wT#Tg41Cd(55Sxg_VYIoxCCdg!!;;CqPvSANyL&Z(E53_f{e3ak=oWa72jk#H=B zBDE@-zI1&99b=r%^`Z*>FZr{7Wy14DT}m%JW`3ej$O}9H|+qB;_(k3Ghz|7 zGQ+T#b^gTzBO-6=g%4>cI$yFYO}+2+ZqMQgJ(lIcbQK6vQE7t98cY0wog=2j2~$yP z%$3CzPKfRS^E$gV&a#SVsf^EW=p-fvP;$VvZl7wgmMslJ!T0ykKH55*m|j8*$3;5p zGI0 zAXNC%LT%y+qE;O^1)g*bvAu>3vFsCbHgmnle9CK}X&DH5;|^j?-JwFhtwVU`cKWZ@ zeq-zwJG*rl9!23TS;t=*bQWA8AC2XDLy(Bi-~3kO%KW$ox#6 zEeA~nI&a71vX#8zq@ng}eZ!m;3H#_!aWA+fi-4X6`)L>oHLUQYqo%)S84b~yHS;l! z;TJOgyE0__t-B!c6LEpdibd9B_;{&npY+2@w+yA5;W|h9=ENN#ncW*NJPFqDFARk= zOYF|*s9U95*@-eQkd{YsFzzLj}v;Yj8@@)>}})1lOhci2rH*@=jto-EgiPu1N}SY})- zI+=ht%srT%Q4`UMEsp;8%Kn0gis4#MH=eV7S=WQ`b-@OHV7%s)*m`ijApF5AUJ~-x{Dz zAJl`Ro@~NIs-}oKdRFN-eFciJgFfD-qo=*H!EK1j${k!a+d~T9Oo&M91wEJPY9>up zMpmchUF@;;LW9FQzLiVj$D!r!6=3-)||&-h{4+q5J6mIgK-H;ExWKq^h3I&3M$0oZq^ zB4*lmzJ9XI&u-sYv@_5IysJsZ5^IXZy`b;v;NA&~M49EiKBYa|1f2fLsiF9Zjwij=^mz zoetVuw9i7c7TC$xZ@Fl)oGsDq)h5sraEh#7Ko&xc$(9DEbZpO##=T;MtDsN$9jvfH zhfbflAccSa`it3e{@uf-6&|iCv0@i|r3A1}tedTy?-KX>@GJxExQGFZ`bDN44)~ae zgDR6pI}T^}-cFHE+1#V?MnFRcA|+=Q-Uw;}2qCXIH4XTQ-e}od5II-iw>*X`}09ri%wJdDMuN?d)bwcr=V=ry7%(ETH)w1kB7CgGMD`+$?jFlusA@%h z5!O?tp1Pg0X7hT}OQLY*=Cv0;0b zO|gua3gW&_JH2V4e=j9IbxWWq@KJ!k67F&G`yg}(54|rTCfL(ku*o;qk=LK`gg#K< zjb=U+&b3X*_!KXps)J2nE)^Rh;HCNrI=^)r+4PJb9c5><^bP2iHR5tWw@jn`BT2`j z$mLI|2L+WIvBu(m3o2rbK1tlXY`zzR)N~pq^`4HDd9=}xj8wjlwr8$Qk@+uQ!S=X9 zZ7!iT?(w;fFUpbo>MpOZRqN9F!Jf#}-vqG94g6PhV)b#UJ87Na$9TvrD{;hjvZ4H$ zk|{cgUV{8yXGUxtAIX2OPqR4Gi2ZK37gmjZGWesA-MU7OxV)ooxUzyrPST+K3J$aZ zd7i~c1c3koTi^AO&EM^#;bC-|3v=-T{73UXvs|E#^kC@AVtD zep@ZRk3jpyH*fc7O2{^6i9ygoeD9jl6C1FdbBJvhK?YV}XX65YHII>+_6-3VHT#TY z?-dz`v#&Nbx-yV_O#azc0aCclBZl)ji^Ygm=Vm>@S!WXx+}OH0hl&cG^QBRgE0LOn zh}6XRtC9M=TeJA&B0tE8ae$Ymj^v%9CsNg6Te?}|XJHBvnfJ3{w71fAast@R!uQ8y z)h+&ElAD*wF}pfNO$$PNFK_$4M4Ub+*W~nFuf+tFYV*O?(v##|h%rra!qi6YuvW-uxKk+mYo9@|2lR2Lk~EfF3@zhm=dEhgFAT6DMCm z38hG|)k>`Pon?&vJgVLYei$uYGh}5E7v@qNHIU{aRCe+MYO%*UqVvU6?fL=my#7&G z=&o!03WBFsgCL!kBuBJ|dNaV;q;L&R?YqQLhU6w8I>>8X!|ydC*^DhN39Hh;5qEC3 zAjy5`fCEGy{jdZy(#l74OI=oC4L}$o1O+Opu_gKYLzu`p4#-7sB~sTItu7}NRfc7F zI=7<AOUGuQAKoWNH|i?+*rWxD%Bp(x47 z$5ks7aje!n6V3TL{Zj%SR}&Nl<*x(X`48pQf3#=SaSAgQe;ZY5vGw{?C< zb%X>k$uE{L!Ehre)muD>bE#|nI-X(_EF0|T3gHb70H#Z*P8!%a71juuEX#*J@vP{A z*27fgV^*zzSCY~e3MQSrk3+@FNGC6Uqd0pmd4a?%8nWzkGR`3RKpscgst_$9n;7+f zJ9We#Fi)`NhzJ$Q0s#JRCzO20Ont1`uyLwTYA6IRSrM+c^l@4x;5233tx;&dQ@oQY)~sUP^s6s0R`+BK(W;5{%m0tV zM#6lV_^YI*T}@OEyO*ar3bF?yIFypCfWQwh08IRet6|Kc5ILc})81I@F)H ztT+(leJlAv3Evib$IYj!TpbP_V*m}+xb{)LAJkIC`Nc0ChRfzNozn|-&3c{(8>rgL zeXnL2DOE>JG{C)IjLghOYp2_x4W0FlM6;^5z2H@dcHbtLKeRPT-+jfJe6>4~MsOgQ z5V^(qYi-%l*3RQu+{{&H=aW^xRWP34D%d77J)KCaqF9Bi9Sz1vl$UD|KVG?PV5mE0 z{h<%fftN;&6k{G`Y9;3FSsc1PKai-{Y_~|LulE52zaCeoF)zCs$4@3CWlne4QL#rM zLMGLNi}Db}z(82eFjZ7@{wm1TPB&HPJ)~EXKB4|sPHB)u@DX8}d-qE799Sf9UgGWh zX+3R1aAzT9n4X`H!q3awpdu${!@xaO+zq zXYDz`-inU(ZRhxnBFXNTPtIxx$>;O`OQ9f_!pXtSSD9U0o&cRwuu4wLQmAqVMJ-Xw z$Yo>mX_LR0Er+s)N!6Rn*ID-Z8@ji8DvE5Gjf z5wo>zf)Do46XQ(2ew4^uwQZv>NxXpEIRcrp^#^*D)Me-gz?k0XTBA!xEV8Fa;rb+h z5$5xXIbQw!Q`)Y`{>)G*U3;7z_NzOIzx4{7HkI$*J2JGnn)DAQHytH&%t)qW=>NCk zmuwD5tLlfRJ$$|GUU#F<98a!(j`DeANwM8GA09`nF52jc-kcRNQ|pgVH}4TpM+pxY zX-H32885MBr%h?)^{tI7stc2 zBP8d|$lB`JG`clTt*>cCf*q+4g;a-8pawj6miM7=dbWOJTNyT zoS}i-S$&LzvYCap@b(p0N`oAPxb+Xvuflx;8wEdC0TvTcq2tmF3bx-JevvA_7sO&r zi6!e_*=c$ph)+u1pGW^4=XO~cM{4C;udWZ(=!ohB3;{~PJ{J>Im|b<~Z0^K=3Y0i~25Q4&QC{c$+%PA~YvOa}vS^(~XRJm&HujstAs^P8MV+ z?7qX={?FpyRvustC7|ms9!Q5II4)$NvfRW$@KIt$Yn(YNM$ZMdkoP_oYUi)BKhO`W zp;p`c_C^^jm5}i!HW^~6oE1a?y2m#{4b$X;ndF=sU3__FtP@bY>G<5LDoRn9T!XKa zDIX~QEnuP$tN8A57nmsKle>q2iGtyL`XFCN5XJCV_4%S={f0O8JM5|Gv7HG>HfO8i zv^=?TfF+=jK|7J}s~3n z1m_=2+IYkNO5Ts@<^lHxRQ{8D(>nqq;*7K>ZDaL=9kWMfTzn!&F+g$?A|;AaDA`EJ z^_|Z^r+SPwHs$yO5RmGs5<%RnA0W&Id=}7ssHDZ z3>ocBjJsf0pOV!*F~Kwa2|q3WXk(U(ea!M7at5XIgA9TkU2D8Tj&>|Oz25h+E;4DS zI9W#7KvTbKddu5%mLaS^Ge&! z{6Wgt`oOi|h?H1VF2Sw_5nrA)_G3#U!fUCOWX;_@5FjC>Baz24mnzn4N;~hVqrylcHV!q!ZpQyNLM=&J)8Yv;J!E{^~N-X!o|wKu;|n&)n3V|j<)l#n2e%`6!A5j z&1=-X|8bE3k@z4KYxOp|tGC{=;1T4cTuVRCOO@gSZ10^88PF4Iu}B1AQV{|Zqj+4_IV{B@~7aDgrl*}LD-XyCc0Y&yf)Eqfpc^%we%X_ld(90CpAfY8sLfFWx1T`V z_t{C&d69AAGnh2@1g3d4Zg&dGQ+IDiGdoREn?MYbn#>SNVWiG>WJy8$dgI$-Fg*^q z0s}AT2catqg@Kt@0GLF{K!B1`g@~EPw8BQU7oi#9)!Q2!Z#Q+(9owb$wjUMA4PfFs z7u1)s@Ou?}nr3vucpm8xrQksMd90~GNv%%HAiNsAtzAV*I+9MlM?14#C87R0pm8~~ zGH&T!hlg_hrctfBH=bfx{GiYXXd1Unoy~s4dKLwpxph5QAjM6B_{7qgq~eQ`{f@El z1InIips8Pl_Rz3jTN?A%mNrY(a~V|cU*8iS_c_yjCA^KiHhFutO8spvzZW4FnnWaF z5?4XE7XuTIh_i^0kjoFs`^C{x&9@1Aw22y2vi@Bp6@sYq*WG6S9Efe5tM*UdM4UNa zz7#ND}pINB%;8`S1qP?wUsw^*d~DA8~dla z^Q=+PbCSHcBXZj8T?bLOKf9w34W7maeMP*^c9g>Ice3)hOo8W717+2n`F1(X@6v@F zp3Ey$`iXQ$s!A)ds78f zT?x^j<%9n0I;GVJ0x0XQ}4d9Ji8CrfY(KmG02kEMPV}Q zS37uV7$PeB3=tdpOQ&;qfYz~q&oI5V#1p3&0A7Y%`nG))fOvyX^gbh_f!)6y=S}>$Z>7x{=%Wu zzj~$j|Mp7S5x&Kh8JNGBe4l}hl9sNQ0*5$ufgfu6feXsaVYh`2bLoj*6aSWPMt?PH zBHfp!YCE*N+Dw_UxQTJd&#rEEu>&f5kf8$3kke0I6Tm5Vz`oKU9p0r8T70`Gjb!~R zSEi^aRzH{&h0xMZ9}1*6U$|PTR~wl-;8y7?S*m+p;Lw)d8DzM!Rt%wE8w>e+V*%3q zKi#}sPL1Of5HzeCE$bqER4&yDUD4gGGEZZFYocc@a-N6%n?2f7$l3M22*G?>UN(U1 zVZB@S)A`HSKRO4kBb*OwIa`5I)Zom{>b-UU0W!rB9?VO5dfSLpU%uMUaix@2@(H1E(HQWnJP2Kh(fXz)5K(& z3`p@){N0)-XgAklT%5avgcjXEmi(+(AK}0c{Ii-rz9^U*zHuQc0nXq&ONI7qs&$qQ z?!Q{5>-fK@yyt(Yyu{VJO1z7h$Teq=NN*BXStXX}`aU@+VqABhz-WyaemJV@02vF2 z{^XcsU8B|gD8~ZF1%j%Rq_;5M>*cs`Ld=VbM~7IZnHqB}EhKDKDq30Q1Gv`y1Ijo*9#I0>)C>IZLQjZZvb*}s}>;5n?<74kGswtMt6H59o|kcaZmG^Q#CZ zM{jIq^x-_yk*ni(Kmp5q4Fk4C)FIddl=EPOrJow?-}8VqVpB0_g~1 zF`~0GssyK#_|N!&xbHlOlpjc`&Gh@Ge~;wum@OEgz@+>CRrqL&&ei)IpJk3bCjARAgI{k)pu2$IP$U%? zNj4(7BVo$WcQ@wp5>VIYhKoF6>b3XYo~^~Js_0IB1s<;MtuBHaK7LVov;xHTC9XK_sEZXc+S?MWl!`k z>EJvN+3uftXqxwTO-WlXwOrd~$ZnGiy)xrO|G`jap=tOv{I6S7ulOr5J^h`SUJsMa z+m%~}L-84}|9{B)*X6T6DJev#8m~Q3Q4aoz&>4>2X7pRy_J7wh%#gI=;NcQ6>L8RA zaNy3ocS?(lhmdk;<#~w&A(&y*E^kUDHsPbo9em4eL1^6>*5H>C!tW6bc`J z;+d&d&1&z2oBKQ?#?##g+k_maIqmP{#0Ua)78?7ylN833>x!@G{Y{URP+rheilo61^Fjiy38#A#L?Dgr||YbT)Rs+GC$)P9_y zB@Y84@2Q)pqgYomKA>?!5ph74@;eUN`QP6s_62bu_P>SSFHr!q^ygqRg@|tCHBlio zRp;Rs(Akm2jSaynk0}kEPO5Y9bkA{SlZnLfGL5r1s3*XEG*bz|e$3J|$J@6xt;1#F zlEpu<^7UQQ5YMis;L;o#CBrh8jg+qUfQ+UzEe}@dQ;b0GY1*OGLb}jg;-I)ZYFidJ-@tT^r#|AEk*X+1vR;>VI^$<54troR zT|EngnjyxrzJkSi*5^&^GBhR7;=4f(Eh+4pS-=aRu32UAHX}O?1q?oz#G2at9SE>r zRZ8`DmC9HN-O-@nm4et2^o6g(dOn^al^_r0E7qrcV%_kl`2wG9Wr;$zVszJZ-tt7M zS+P(Tt+M^Vzq(l8vr42CK%l~Mv**WGy179*@VkPw{sU!rnJURX>a$XXC%M(Dd??>u z-C2uySE(cWO`3r*hr8Dw>L*K6;=tc7Yo}eTOi`T_+el_wTr@~796QVU0grV5YX174 z?CLrnx+8?%v8ni{%`c-L<=i}D)QFBZQN0nK7CLkA#1xoEM5hjgKYz&Rk0DF~rU&3? zT4AC5&Sq(nTDd$FxE>8#KuC{n3u|}zq16;3NdqNA^aSBv&UM(Ygk1c0Q75X!07 zyqxMu^Q+FXD7aLre8CfU27XPjiZAxrcCLM|ii*;}+IO8?z4~{ky)9#GsbNnzYJ!hd zPI#(r9Yi7@ufm}V*c`ye=qo1SDFNf~CTIz4F6jD)Gf266Ea1fzIL1c_=Dt9&qg>)a z*G|lzNj3=oLbCLIf=|#$woe|BOYlx3-dT@*k!?Pt*=gw!8_>cJ1XBJmgq@8RofDXwYp-17gxhi5sQwBf7-SM)-N|{fh7~td zjUrvI#2Uifg@}d)k~d#v8QZ|`p{%5(eP`HD;?R|xYL}bSCW~WBi&)uLoRP>5Yrd&; z`Q7pGP4Hld(ieStr7xtNX=K5J*m3$HDQ-@SO!s+e-A1Uu|6YQ)?J~28j7VBZRn`mw z{DfGBA#2GSm zyu;Hz-#(B)AIt2Y;g4}lT2y>htvqaBZ}sWqlVkzMTyGqF8IXo@j0bX%L^`kq3}z*T z>A<&q^vwcKriB8NSvSY&n`Vtv1Np8G>&WVjdocryFQMPLfsw)$vy#z%@ZO5vpe4GhA5GOqehqc5e=LyYmA7PQ1_A$SPzftZGG${^-J3 zrCyjggL(mNyH&LS=6kQm_KARQvPfBbaoF%@>v-g2hKIhVaBEZ5X`jcmPMFe^ZnG15 zVpBB=-~AW=8Is}4exeILb>>N(-8lG6>L&h^oMeUmjUn^}jnRgFhxC(T78--6DV11D z5;vr2R6sY{L1=bB@DM^qCZgtV)N{-oI(x~6VI`w8MYQk=Eb(Q1^7hbs00wIQFm2hP z{O|qH!cIQO{V&_HEJxO}{eI&;+`9eQ^1Z1@$0So@m1<-F^ElDv)XJ@Dw`I){GSXVXkw&(wjCy0YflyEJv$ko^cA>;Z0?sg!3$!lvKJh*<3hBsC$* zRote{tj`(i0w>->7vAB`XKN)iIYbJ0-Ix04<*#5h;-CCB?JSoicU6_zL1gXh=0Lp8 zUCbK$1?DW`W?J7+M3Z6oVe~N)rjh5Fv>N4X8)F}yO?>xGR*XhRWV5G=VaRdD4;(P9 zK;S#DXd5-U_7%6C<`WU;s)SRV=T7uF(^YOBwgu9*cj}{5{WC>6Qa#)lw$_}mh__2$ zEK)e=jL+Xe+R)r$*u049&h=bwDc9on6Ib6^G*Um$;kk1>!$-=sD#049dZZXpHzX^C ze0*}^o~K!o)lGH$i^BNu5Eef{%vXJi@ukf=HB|8c2%yK3gNqu{97X0vJ6&{-Pi z_&{aIhF>{ixtlk}KFHI-5W4En;t~YKrY?P);1f+epdm~&c1%!oI_0<2c$k4}z^6#R z07R$k;xMy)lJw135W0s|Tu*;{=GZ4m7t1Uv?c? zWPjwn?b60Ns;w8!QyyD|e4KeReEQfV@{mLK&Fgl#9VOjzv~I2;f8=8eQvU;{s-6$& z-!g|cc~`GmLZ8jG=Vjs-F>7!edFiPNaj&i@35o;`s9CEr8=hpEdik ze@Lv+*K?2Z6~d{h?<2_sf6%$Wl?>p=T@9~f(?GHI7Z4+MA=Qbtj4Ur@fPdgF-#-U4 z_Kk=E-69d>0TChya~So%bEwWwiqYg_1f<9oDRM{?4zprmKO8EaKO`=*_VwE7Ot>%L zwoziWp9WVu9^wD1Pk_Uo0oYBV`(!+G==5IpfsoP{+q*6ATh6Zyy!&#%%c`{>x@Hug zw9TDq;yGwP@Zvb3%EN27(BW?9sIthZfTfNgWn@k6-PQJ7o;l;gQ`dIG%Vul>&f`)0 z7!osIH{-DMRLi_Y>>4Ra_H~lOQ!RsK~VV^Y8kkrx6*$cnITw9Nx+a_pU;1S&^b|GY5k-Y5I?JlZuTfM3`=? ze)J_?zQ==RAT$XAw8Ae7xd@3qTjZN5NENER@#9?zlpD?hvv;=X-|_zsbx!50lX z3#z_P*zRpb6-izx$9Ow+)f#m)jC=!klvOF&#aA}+nqmKE zYEu3XQg%URR%oawQHGVNwNh!OF5UD4kSME7JBXXa>+xo#fNAYLu=%$M40Ede&k$2Y?VH;7Ny`@1 zh+ZTw*uHRY;@`mp4^qr;-<#l4T$oO)CqGa*EMeYJ#=Q~+0E_vCV;%|8P)FX|?(y!4 zjAzW59&aa}x(jjY0pmU4Y}IjJ`FidY5}A7ZWT5%iR+H177_l^)4Bib-;Uox8dl|`O z=kK*+jt~T`bvnOs8|+ugV2(D@J&fiqui%$AS5e2`3I(juo3}>dv~<(u+wXPw$;N#> zw__~W0DU#;dq&f_`f_XZiKwyZnPa7PnD8LAXqDr_8256F=SquPDi-fM@RPY$iV_?H zc?xwdm$}UFyo2cIJcQ-(BK{`HDAmWf z(RGz})af%$+h)KL0d@Vx11{Q6{9gEtvHKwtj!7O<0nv~lfxHWXAB6jD+fyC_tDFz6 z*OQ6#Jnr(TM!z@ukQCtzlFU^xEyOsoKO2c#i@8M5hFq^-e-VP`zg5x4P3)Rz(|tf{ zne6~krmnr%T>WRw^s_VckrxeqV%UEbyaC<+aK)+(;NHtOK6sFw z8+p3-@OUEFnp9L;81#$CEicatKi#rn()D1CJjGvXoRuSkACi6$>pq?$%e*chJfj6s zSQq~vSC8Oe&}_EK!_g{OO8;-olFV|HhoMwWbC}EH;Q4#ge_Z)i^@u4@ba3rwe7?#_ zr&B^i$wUw;s>hI6p}@4cxeMD%9&5ym+WPaJLK&L75cfen0}6Nm^|#KAFrakXeSCh4 z^+CKt#aE>r^S{nWDGu>H9mTuaoM>|oq&~{~i`q#tp~Uzd#rdeIyiOU;Mvm`opc2IV zCN~`u~=|zz&SR*#n>C7X0?;oD}YwW=V>2&1IDxMR$^I-dvXH=G+mRTfBJ)PdvE#ZyxrJvc!u6YcnNcO85bF zljq5%0oq*uQHE*da=O6#;?U@c(-a@I^RR)KG|V>Z+ZJ8&2@X^6Yw1s`o&uk82!jyg zS4^ulZ))(Q&W{VQ_d+JG%GCuOX6c((rPHmzXOwmaOhv)&Lm>iQM(8C%c!EzhZWUEI z2JyHv|6NBDDaE7FBB=K2S+nmbaOwhgX2Q)ztlZ*1wo zgq0B#g)MW(W?rFG`-%iIN)P_3%X_Mb|6+r<{A-qZa@E8sAd8;#x z>EH|Hh(9D^-o&JU*O)}~HUTg$1F8yzFEwk`hsuc~Zzh?W9bG6i4Joy9yJRMou8$Z* z&)B=hO=%ioh$xi^8y6~O_z0vKVRw6lhg*onhVVL(5&=H${yY~Le(@KuW?$Q&XG^ff z9fqKL|FwMNDS|B>SgkT->peYEo$awg4vq+6U0NSW9dlZkqC&pd&x5J-y$1EIgIf0= zd|lpU2CzwQ-VpT}8lm?y#`R!Nd63V?**Miim@`|geoKs6dNUV2+x-l2pFrTbc%s%y7VloBaqy!`k5D?~c4QX_6e&6~CYM}W0c5^Chgf_JQ zdBcJZH^c>+=CfN(aRR6by;lWgYcDr%(WiMKZvOn?YDs_{F3FjcANha7V&HOz_-sOU zh;mKeIW|}h42lF)C#@&+xsZ2|*uc)^`hDBj7DEK z)^9vTnj`AyeP{b15_yn!tLKBySeSWc+8;Yo_&;9bqb~{qUIpS&-#9|wr@=%j2G{%D z7&T`X>X|9x9+Qn|?m(Bi9=)O5qyw%?V3Ea@rg4K9fUCZKV|1I*`;EyT>~au ze2%)8dcaC-!|xZLb%?^Oe%oaa!aBu-e7t>u-4-AFs7dN_Ga@6#es}O2dCKSdPt%^h z8TBkaae8xTYpK0$(Y=)qCXx(mJt5+^95qrN)81tq+)MDeI0kq8NT58tTHsPde?nvE z9Xjyb@pB!>g*Ybt>N#^P+|*TcIzurV!)T)Dv`1-+&|P-U9y9|cmZzUStj+&WK%iBDNRlY z&(ptye+Ua3D9h|LdJa4ob+*e6A12ONhvMU(t>$pttu^QNs?jG_>)1?d+bln|H9XGq z*lbX7!D%@^ln+qTV-dQr07I73GXhC|2bDbqCvWgSZ_cA_R`v^1xi8l-sP=r=n>E5i zrqj243OUaeL6Rg%h{*HhF;b3>qY!y?hikXN98S4S4XkFp6Blc5Pz=H&Devd6JT;Ld zJ&0mP7IaXdZ#!+e(y$p&h>SIxlX|3_Og~}&a>reMdFu73HKoz(MTy$iBiIvUHueslDyVN7S-3%@YMsc@poN>}CDrxhP5 z3cIOSZxd(}ly;Nv6n~j~WU&BUq?az#$?<v*B^P=o*-xs?!)+)}FJN$B(EcVIT+7BBjE5cj-de{CYji&Is%L*drZ=6}! zJufNW#~ZA9*D`{-{;Uj2CX+J`WLgrDPJrBDaHR)>3i-M-5R;?X4b)(zx0_f zk)XlD)8+M;eA7^H8f4<#;ShxT*M~gL-1sGU;NV(`*G?fPt~Hv`n^g+%gdC?-j7&~e z!6)n^?#Htcj2#dlKX;*MP?~_8o^5esMNgcAi(&UO_lA-? ztzr~8-jAY&PdD4#VZu~KVWJ+4{2cphhCJ&x{STb#iLz@SxQvxc%9yHJ+yVrRAj5x3 zaC&WTcOUT!q`4#8%qOkpCd+vl{e;I3taH76F;*1Vo~rf3j?f}aC?gcx#j>q5UjUoLhVRbjvWMeMgdz{ z0#`e*VW<*ie_Q`nu5W?oi3AEd4!y#@;cYZCX7kP*sJSg$IG$`X&Q(@AznC8jLMVK)x9aJ z(&hLPZkr;~dJoRoQzpo_&S!Oa1*3|^a*4M)!34}U44i%bfvtvj0%GyFJ7b`A)Z>}!tQh#`gxanArDU;Cj6##3};O5)%pjUA7)dxo^K9(j^9Pj zF;g^Ey^n_qFx)D_PgZaHeYlhT)^C)!7csU^vXdN+squIz7GSKAMQg!FRURS~OQ7spUr$Q*F+DbHUr>C9V zMA*ecbnT6cg6q(MJ~m?3W?L_yJ=E3pw+VuV4eCh(FVwEVcu$h|H8+i-L8Y8eQC`o# zn}fAJ&$L{BHPT()%!`?AsoLhAD)O=GRw?e-un^CB^5*A5#lAN^A6&2XSRepw>6E2t zT37^%2}+(;nvZ~?<~YgDI4|aNgcjGW479ijOdFib#DUVYPuiUUI93qsk$A@iw61hj z##zKC@!LC-(tuar>`41_M7?ZtV+c;f*vpXrb8t8m5TiGQYQKBUAvU#(yO)Gw?d=>- zRn+op*zxmzuYmSgU$MbooktrMqHBL#g1vj9@YNkXejrj$@KS+Lw&?${_1)2MuHD-q zB*aM&K|~GFTlDB@O4R7xr06wK!x(}fh+Zao?=9+J%uFJpMsHy-5xw^r3^U)8bKdhi z=dAVpXIaZyp8MYS-q&^Q_WUTiM*aMaECVX06}p|&x=)Y?DyrJ;v>$*=iV<|V?hFmk z!S|)K4J8r+l<0ki>&+SfE$40oeFerj9`pZ=CwodHNOk1;F9KZ%0;$)=fl&zXY8#V) z$@YQm#ghxPxe_w>I|~KOZUZN8{HcEQEik1r5)=P>0OnhdKL=oD0FnDn#&9UmWAh$G zYn^t9^mldo46qSpqX=@VdpDXYjV$pexqGb1JxWuIlw@JRa|%gq z(7vgr6dd>lF7!}}8|(aA@A<2$@{^QxHYdr}$l4Fi$OGP))J=fb$EJL%QoLOJ!y^FZ zd5w{s*IMOY+RA;>z}r6o@q})c!pVtq--0bF>a3X;`GgjC@#4O)pUt5RuLFidY9ovF zNJkrgYW^f{$7%?Py)slk$l+%gUC8rH_jTzilTln&!rL55N#Eu593u0FxSS-hJ8*+# z;^j(aTi+{jJfu~lj^+@}=X3Wisz0*oE7m!A;0Qb_0avywD;G)oF%OhzpqzUBup1S)vKXpLQ-cl@cA4y?9Q49s7bV6ORhWhMv7 zIOVY<2Pia3)F-c1b+_sLS2dc*8}ryTSb@6tzH4{a+W_XLhNtVTxsZdylsA$n;-)F6S^t#WiIF69d9Z!O6~==a@xGYh`$K9(q#0n5{O-rKP zd;1ps#_vg!!qeoTFW|rGpUMHAGVVjK-G7UXX@47+n$L5+FOMrXFZ>*6_5a{bTe|b} zMVQW?6|#x+MOnEJ$gJa{W1Y5=J{Nq)mRD(0$A9RVyuA6rm7+P%;LUQ;x2dA=jhet^ z9lXx&o>wkK!_s%DIln8X+un5t^6bN1GG9bmWS`3M=4$<}RYw`f$hcqQwBw%#w6JGV~Th>sIOQrD$(_h?wQeZ&n?8wsV55XSkk6N!!qoKCG2n);{<+LUQ%v(B`h3!RW zlNRgb&hBx^n|TlxNB{w#eOvazva(ykbov9ZL_@!5Do~&TVff z^-Q_}Ad^?C*24teo>lTe`mwp{W{x_;qaghBfsH$whHHWavij=oiRYK^+|Qt@l9>LWn2dmiDX zp1@|GOxFNC#JSx%vwMKAAPJBGoWlxtOkmgL(FNb{5%%@kxnK7@mlQk*uc#X&z|V1P zzx(gi%Odh!qBD^Mi=xqnzizopf7=UqfsvO$ zVZiTxo~D`Mt?!q!673a!aXfpZTx*~|=1#7~FUT7)PK-#*4{X(Ex_mDp`fKw4moJUE zsJUSrKewCI9-N%qH1L8%HMO;EpbkQ{ftG{Nl*A-4@IYy2E~37`n&3R+IG79Z2w$`^ zyFD}o6ar)g*9HhQ9PmEFp~#23r1_S#4z8F3JXiK4vDizlG7#oJu7IZG17O@mpqng3 zj5G@oD$YY8&q>059qXPOgyZRUta$u z*X{Vzcw&$SWmI^9~mZijN(|715|sfrHD#ysP%^Y2D0M~d*>ok{Lbe}CFP_k z4@WvfPEU4FY<2nXZoT^oJGpPUa{cebO9Jq-aQGYKW#xxqIsTVk1Bi&Jr0ECtf49^1 zaM-unW`BQ>RJb{6)l0|-26PCQo|^wu^L;KMI|ES7DXD19@k9OVFl~kFYY|=9X*+$V zfZ_#n9pwnv2(mXk$VDWvp^TlTWvHh6cHB+_io2-W7s@lQ?ZvWhd4**l+4OtA+>)SM zo{{O8D&{ZIJtJ+@Dscqn!EmpsS6Oo{xQCd8X9JWcFNlnSLxpq?8LGb-4okYv0laOS z?n>hur+L)RS@4!}4YL>OrOxc{UY@;;!uyGAUR4I>{r*u#zhg7*aqV|Tf?k=?2lm2U zPTfE-^J@{MP{zOORycx%ertJ1rc!lndMC~SMn66Qfe>AZaT&)BAZ$i`hEPOElp}H> zLT{-rZC%7!+H2%J?u`3us1NTE2lKPpQ?l<&ed653CFc!VC1qb&UNKkF>Q{g3K5Af> zW(WWtz$HxQ*j)tKb-&9+CfA#ENC{j(vdApMkx{2}q!y&GdB%1dCF~nhWCDvw%x_Y3UDO%ySk!^ zPnPSwIOmf__rRC;=VXdCOrOdX+)A*zIxYXjugo4(R}2xp+2Fp|k<(9kM=tQsLE6Ga z+IL=t$}SPZkCM3iGYs&a6<+B+Fr`{n1;xwd!ZJk$slZ+IriX`|yt|*j!;w>}BKCxX z3wX)mAuYDN;zX(6LtXY4Qp=_ybW?6N4DU~0AQt~QJ+NKo9?Bcm z>o?8Bm93ize0ikA0_&Zj%=_Xyxy!f%iQmH%x^%IX7;$vs*(iUFuL~^T%g;^9l;G^f zKQj?P>z{9>%n2^nZ{ZJ9h^E949A*X3IAH$Fe`(JY&u`kZ1dhIY`X$A*HIHRbmmU29 zbaf8jKl^4^<=Jn&tEhB}KPSNE{v2<(#zQS2Mf7%JUO0;!*!hKBVQSck&kNGwkqV3N}$Ne$U`yLrnT~&QVtO3R#MO9}6ms(8H827xP z>?E>u_uz^=GQXaTxZFtpmred8+V7l)7Pi*O4<;Esy`gNFJa#qTR;qpb(>r^F%AttU z)_`f~SLyEh=%F8~jm+1c`Ch(C6mod+D3>B)^?i)P8v8pe^=)FUcvKo7!lqsKiHmm; zb?9=Q$*vXa_^vMx7wK_h?pfcyKARZz@V@@j6!1WQo0*xiM8*DcA8zhkf!tpdg!IKA za*|+kkQMwu3(ZI9{bjO~`7c=`zLlU(k(~@ya!?zm1(*$b@zG0rg4R@-=NR(&$YEfmFcZO=?so>4yZT9Te1adgwfaAu)>4=ozl_QEm~*GOldQv zv@87P#YLl*6I(2WNg#B3{w@8KO1jT!8QxU-L?1ebWwx>HR!;~;L&k;kl34~m5bCZH z4{Okg)!NN-BwEiLsX7WHhHK3cW4sJM^_bs?0;#X;?mU@4Ra?SVnO1D!z0)l5quQx6 zbE8aO8==Z3|7=d9d3)c1bc$|Kw~T{0!*KHEEM>~`8wjG-$_6pkDd#{@KTtY!58F@Y zcq>s(`9@fAJQ^!q0x`=Koh9V+C@SScH1}jo-_)O%9KbIZE)emG2}^=Yb3wQWegEAR z<~EOwYZ+-ep8FL+x;*TVR)zkD8zP!LwAVkUG3O0CdORiVp9>8!4sYj(7b88)@nWi6 zn2z@%9I?NjK`aSKw`?n?MCOcBcw{UKMn~&BpE&Ui5@2ONukBt`bPcxq=6{Sz?#Ut8 zCd4k8xePbRNrY#Fhf@FF_99@{PPanvcgp<&Mr-I1npzW+MRp)#9cs?4MRxWT=VHe*{kiEiXX zl*qhxj8WQT+Vw z8qT)z@P6SMn1<#yjTp(P?vKks>p5r1_)-?u7=i$i{0m7R7rk|=m%rkU+h5qLoS$=`R?e(b;PHtv>wYf_n_sSC%N& zwj1@$hq;QX+4w6rtK2kUZoac6U)Wix7?bMVLl4>MQP6XD6i=A0$dff>gw3y;)cx-o`J9}bD(;dN2 zLg~k(yd2%47kC#y^B;PfNkR=lDtNeD2|{HIQ%1k#if`RkYsjJY{6z3v2^Ycxd91LU z0FI)|fvVf*)GObtJseK-M<~uwc-YA6BceWP{#~*$#E6H@Ai{od(jw4NrZ|~FH@hEO1;^Df0@&o972+-P8Ed7WXs0hXt5u%y z|Fi7~SbpQm8UNrx;Nw_t*W`jLE|Z@lJc@&o+^+TwR4Waes!i>6a{KMvZ(3F!b%K*W zm`m!$+_E<$xJ)A%gzrpG4mc|7l|)TFiDPdi-cYU4SWynrP}v`}*)Ivw;~}7f)<&!* zNSc~!>&B9L$^f= z%rfN&*to)jq@W6>EU$E7aDrD-Y!Zq<bKjee#7EXoA7{4i*9c!r#Hw=mi)jPly-Fz@s^KiB zdyF?e%xRB{usZc;BI@(7qxtDjdxPUsH+iI18obi81iiA4S;bF=j8#~R&%#_gVKg6g zF4%8-u>l)eUQjmtVr%#z_ez1RPU9^hNwJUKEzmqR6(##iV5iu;&f!D~q}d=ACO=v9 zr~oTp4HBL#*icHFIolBTBg$N_Qj-_$yjh(EOtAdNDld?=({*QZC@VA@oMJZM&Dp$Nf4bWp6i(5QonMt*2 zKBu9{6m-JgUUwK*q-dsj9}HY+g|UcZ;TdZ5{Yem)xSuYwUVLQEsi;@r323-dzt3W( zRJQtj2zEG)o@K!aBj1!q+5Z6*@O;Yj=VI#rZE2F!<^YGB@zXEnpgGy&P@Hb3$L3^Q zD}~MrqZIp%T0eBiNiy}y?p{m*d`?>a`PG;PV!psnUhZjF9nmY`IAY`kYprIt&lsf1 zL-;}!U2Mba_2DeSL#`hTD{2Uc(NbzP;6a_@QVkh+A8_0OT*hrjePauXnc)v2r@suJ z?^joM&HDIR)gvrH9a|^-{8iTMZBbQim&t2{$RBdplC3K9OLTKq81Mz&&iabt4clOE zktC{7gZ4A`ChT&3Wn3IrL@#v=U2BoVX1nxQzip$ror58~mk>ZT z71>%hz$A}Pb;FW|^3NGzJVm}=d153{x);6VLb!IC_feWF;6bq5nRou~ zWvCPATA>xNz#sF|Z1@!u3q`Oo00Ph&txO{L?XB`=MEsByVc^}a{P+UmruC=NC%o}- znrqdO3j|F2uM@*G!f}~*qgDmxI#hq~E2pzj=Hy{-5gy4CHWxA5>VD~=e35ua0y9+(5U9s#|sRJlkUJuVUkHL_6(Dl;pS^Myf*FA2WgImpDYmbI;x=U!Wnvz=Q};Lzx@6<~ zl6O?%bz6edOW6iWd49fjx<)Y;g-tm;P89&Hgg2aQB}FxWC~Q`8P`6o$%RxR7KY%j% zj2!FLS4Z?ZsRn-5=UmX<`myu3T4$N6yN3ag`GaEyQp(@Z@9yqihR(<8D}D8xtzAV= z)1T@Uly;rGcc&}tLrZ*HH_rnT#1R+V^8K>J1z%{i8|wSEjHF*SYB0$Q><_PVa7PQw zK_z>wU1ss>jRPO%%lUr1{hUuZ6vPlC)X00I6O|lGjTB_fA*NKkbUbnDr#SK&Wy+JQ zCRPp5*XSLeT0V zTuIEtFPSC+C8K8m;PC5L+0+d8@6o#zriaH` zV(`UYJ~caC{I<1Ujwq^jb+1>}wc#LN#ELI@WqxbZV8# zuMoLP$f5@&EPEIhdXN>g9_Y9ApGD_{alIZ9mbdUHYrFb?&#)< zv!ya~=^u{}AOb}ja+f9=A*sxAJQJVagNdV;dnRK9|F>q6lQ&-P0rExT3q92%kgt0U znMUfwM9#&K0E<1?A#f??QP6fzo!lnkciw4u=`Bd97cEZvJ(e|yfYO-%iZR*wRXBsj z;eSK^2xJE9nYnG7m@aT>%21BP(wMy?%Y%)EB(&Jx9nSBoTp&Slfirb>>ArlgK^ zV^`mkcfF#0om&R5pRmu1k7fDpcU2obhkSRh*xkl%+%&GIm9CpnJcE5OU?%w{_Lt8L zeoI!z-_01YG^QD`)Izty8Wo)9y#I5j61v6C0?m^c>|6d&en7EL5;Vmxc*WkZ_m79( z;wsN-&+RoDYQ8&ywBv%lDrPclOJd&&2pFgOOGl|ZNi}@BcDxztA*UVlN7h*ytN7j! zO#E9yFVEv=O3z#DP%0o99%G9;$A>2^C6VJ-lV$y2mZ~KVK9SQ4%(TpO4~=o+De7)? zO%=dNKiwkVQRS+kVi>pNdMYF21wUYT&sKw`xz_y$$rd|_CGHt{Rc{TebTENxc1BP=#!a$w4( zFJycVnvM-T#BbwA_N8uF5#ywK3B$5+aS+~+fKAkeX?FTxiyUprbsbX}RIP7#b4jes zF0f8Ww@4o%bej0e@yMUmd0kRVS}}R<+5&AnY-O=O-4LSSNHZ;!5 z`@tZGz?j-XN z)cpoaKU`a)*ki(*t8uvWFmPMZwlZEX>?`YG))d>$ZxPJ*E*1ZlKVZf&q)JLP8=CoY zY~*Lf!WicUWDEgD>WG81n%9(m6jl#mtios2X?2uR1KBsH@U-wLqx07_) zCSczu9X0xig-2I)_OGO4tBs#o-*W=HJ!loua5mqIdWv|Hho+k5Nyl%kl?{DcPfIDv z>)z0C^5A_xn5_x6EA-tuo1vnd9ber`mkW?C7foCBW-~2`y@kR&_GF;ApL?5B8x!gl zr1vDkJ^!5gFh|ZB;uIi*L2SdIU)F`EgRCSCwB^p)rz1+JPL?x;y~YG1cW%eu3@*m& zzK~ja*ngdG<~Z2v8qrNQ#DGe$YA$zpA2lDfb@mg!gaS9A@FXzQ*ujyVXd3>YiuePd z%3y=h&qCKfmBC$sy^qxUI8L|`E z+(jG4fzr&IiFOTP_u;2;DoQ@tXWT3Ig&rmBb`Y&^o$1tUuIDMjV zszI`oSF(z(nP`9^w*NYk{wyH6Bi33?-}B69@9Lq6_n8Z| z`vn8*yULyIw_(@sBK@=%PcbR1-4eR#3-y}5tv&^EA+CYLIRYgzjeaWUZCL7nm$CIf z$b-MpsT&Z@0=8sdmXTe71y&C4Gtd(w1@p%aK4~dnoztDt#x|m#_uh09Yq-ZidX;9^ zELx}z!C9U0!#dVs(!qyrUi*WI9&$T>G^hpTnA%dx*^HM(_Z>7?$ochKHtZ8xRbF5C zo0{$Y#;MBwd){U>Wuf?MSG&_5QTKjPgY{KwP8J4zQ}KH~IO^f=0JXd&&t-Azj~7}= zWs&i^NmA_-h97-n zmwIQgMH@M#e0Gio<0qC=Zl&M1=N#8|zvZ`R0bJrFRmq54dd+(csFXoHxSb z>8ZYipO`gCpmAq!h_0&T{(z?KA5hPWqiwIqghu1(ew1_^$*wq}O48PEnHekQzRc=9 z+r+JX?CMl*^>JJ{_a8S&<0++0Wx4o2EdN=OG>vXiCCXzv3>06Y`-iskd*Z9rE$XF4 zd+C$7G;^+Q`(DfFa-o4zmHA)KD73iZ!wC!d*HL2>9kL^7x+0=Gse00v#Pj*4h~qCyQExJ@)24Rq7|R?fUA*BtIP<9F>902AT~OU# zRMC51=d5XaSJ`$%TMcJBT0)uX?;EFaAo;C**(jC*?=8*& ztu|jhyRfDo(CDVr`wizMji&sH*GWIjbBcL==od;Ue6_)wVz|dsjW+$dY@ZYL%6EY} zOS(GmTQR62YLj6Y`QVC3N`T56*z5O@!M=m5^NSpR6L}zn*N2=*-uXt$gp?CNFiZD@ z&yg=3IF@Kb_Q}B7fB7`Z(7Cn~NXu-v3v(dSwMcng=WwxHj|;hR+JYM@c0srg+cz;RqgtqXMKmz%QTD5 z^UKc6_J+vKC(eo7_lZwZWlZ`Q0b4n#luh)-MIsAK8$<=2__W) zIo?V4Og13{)2&mb-a;97!~9ICkf{d&lN&&G0HLejXiSj*YK=(mLxn^dMgIyGk%sj~cwiuvG z=g9*??@!JB<8121k>&w<<^5PM*MO!g2ig@<2C-h4ox7#a6X%IA5cHpmLAi z(zX^)1dD<)G4qhodNpy2_pwfSGqQ%W8g#G8sEM65F*r|b4dZ2m#U1vnDbcTrLdpI0 zXoE_kj#qsD=>7pdAI;yPU1Y8^=Amw9N_kSRx0Xx;sK6aGhFnQxua@mbTnyz@$eJ%p zIevh{iheHF*A#x81l)rEME>^d3>MpxKyhy+83c-{^Ua6sXboGhyy5P5NxDP@mOJMa z($$ELzVNcSc{A&DVouQbu@C)NRx_bYi*efM6-?b%u{zIo>3rL^&B+Fid?y3W_Mu;p zm-}~^d3&FM!bT@v)LNEXF;2O}1=ky~;NGJ~uNSb>=dQk!!;(x#MX`EtYjIt6>ST4a z&RKQrbhm7Fqy%~kaHSdcsVM(%yATc_hjO2T zpGN~Qn0!g_e$0frw~Q`arQYJOHEN4HqkgvINaGzm+Qt3OLd>}JXgkzW5-h9TM!9Wp z&;V8Om9dQ}PO}gGzkpIp3j~xMm3_@gNjW5pGYZEFb%y3Z=QJA5p2X;2WGo(l3SHF} ze@w)9csXQKoakwrpaS78{tH&JWBA}U|IOsp?`m~hpPl!6gL~T!zTWb~z)DzzK?dwO zorEpwuB9|>gYk#)IWH1m!kK!0^?X}T5&Y{IPwUb7`<8M6bn8aBHSdoCt?D4(B`hwwtYsYL6xA7qt~QRqI>Ry*BC=_-5+0el-H78^svg)48SQ zjXW015)BmSn4{X40I}SU$4MT*I+;?GGe+_k0WXmFvOi^C4Un=4E*jB^+@BTs%;vPM zw8_KTtJrllBX&5qwBPK~9^dj7#chwFzDMB&LPGmZ@bOouTdkeZw_fk8G?_flv3(Ah zN7%`(Fc|~q=bLV~WvJ6X5E=8>hh;FP^->u*ZO70Nqo+> zr{j`IP-CDO3&}sC*e77#_Lco~Rcr6v9Y}zFR0(!9U~AIzFuMu!QB* zd>3<95x&zM7Z=x;CSkwa%nAP+4*pN2y`RfO%KA-jtxYzcsN}tIErzY~+yzhN>pqTn zT;>_3<KmG4{|PZ$`sLBE8|*<2q6kI1EY;R=sh5!Nv8s+?J+bGH}V2Msc05)qfH zEJ~XG@3C;`{RS7di4sXSsU9C14SDu0T#X=;>N3t%U+tPP^4Yo z=JnPAi~fK&Ug6YVEwx~{{O6zDSEXES5X|~E*qh&aW!*Lya1l|f*0gB(ZAQ9*PFS~_$#k5H9 zU)6{Qe8z(xmIA(H>b~jnNh7)MC|^-)+aZ11Du^}9?_=%#h45*#qn!V&n>F4rHPdkzv-hL zB(iyF*_oYrc#FNYVgJQw@yytop+*wb4gC`h&@TZ;GI&PGY*9RhBMiWy`trJ9oPK8Z zk?0`pziz3DJD&1E;NkjvFZXE%)~s8p+RW=&vYa*UvwRU!n&%0@xBZHO zJTEfd9QT~qkcBTMUfBqYIuVh}KO4+B#bOOTZkf*>rbw41M5$7uV?K5mmqmj5Lb*8;t$D#9{)5W8Lr zw*7g2u?}1C!0%Kxx>lK!z5ECA2~30yEuhQBnuvVFy9C22a@ynEp(fhLL7;+Vn6^7m z18_yCgbEe3OkWR1JZk*o=&FeDK|;BA4KNG;vS~=Hs`KIeWO&Q9}YZB zilVVez%D_={c4m7Qg9}m8&dgmWd70BGQ-F^pM1#V`sosxTnK@V!sbd;OO>ZFCc9Y} z6dS}Zvon`$fpcp)g8Diu#bIvXPUf_6v)boEn3STa;}|`>o%or zX@kEN?F2(1n7vSx+bNBzsQUjojNg8>L!0AbNSpfW#i#)Ll?9#diJsY|;~vC*2aGLA zP1+@XZrjcN%dgeafWu+>mN!06uOZiYlAx?=mlhWV0a!P*B97br8MuZvP_oD3-~we! zM)8=ckTwAIr4gJO2;)@lz1v#?XLveamITdzGr{g>41hWMe_Vyx7Tr3 zaQJ9R9Uo=7qXO100Qs(&8_;f*yz@76Ml-ywp-oR^3mU)Z9qqVy;Z&C^+1yv_b-pS6 zfc9hD!&bU+BS};d?dPVQ&f}|MUuHYJ@V@rHVn4*h8O6bNlB~5lWP;}=6mR9$dQA1! zeX2o$=ni%)UgzKy48DU3FnCY6d>=BuaD#2#A1KIpGIE|ZHnxkE86opNhu=$ihy*l_ zcgL#5*K0MMI}`XH%o3!P_{h&K9Pzz)Vfjb`!V>9`K)>|rZ_2`PA=s4C=3zXplL>j3 zuLk)^O~+*1o!EcqjhTZkHF90HK_~JgX>zC_ZilgDYMT zzAw-$+F$P(v{a?buLK#2MjwFb>b7-xDvd5SF}W^Nb$qG zQ|S~&ntrM;VJqcuZ_?F@7Nh+Rwx%q(pgvnv0Bq3Rz=(oLdI;Gk-99L62;7e4`9HuF z(Dl_x)2=Tq`-Q|1ZYDhkgZ<|Ve?bZXO zqHxNm0174H0IGceKt7zF9ex?8dXl;DIO=^mz}-swa|SyrD9lR>yKoN?(RWx6Jkie6 zcJ!!qk@UbRmB>@CV*xvQe|8bIIOCB;5PX(@gU#@Z;7Z?P$M(qYN-^+gRh%Q*c@%Ck zf`IR6zsr98A#xOVD=j_uL3W>fs%}w4j@n}x+Ub(08v4muq=u89rXWUd)nT{1pIzE4 zy;609Sx|HuJ*|d=L#aR5hvh+}2~F6uF6K-$+&Ts>2e;|3)947~euxs7vsj1NI=?{e z!QD?Fw%IAGfnAfFrzy|B?BhgVM85dL7l%#mwvKek@yy1m)&Jq%6l_FfSb6M0iW$3bqj$b&qiC?YWMG5`9H4kyWoZ6 zEeGb(>Vee=*~uUl!h@@6#VEc)TZ~~jmM2axyr6b!+QOl~rf*fS8OZw7x?2yLuwx^B zGe8ziji@-_`6^|M{sKyR7QfaO7TyC8P1NXiLn{d>Pjidcak@Y&=NQ0*^3h9!BByg? zGBG~Y@XFIDy4Bf$?{ZLtO2fnzEH}F3KoU$ltCpYfYJEjXCU|^DCSlxtP@$9E{-7|z zJe59G_CcdSp>xNfR`3k+V0gq@-M7n-vpOrjy35P^4$ZN%&TXcZoH3nQ(6WN9O6TGQKIPJ_%=brVb<*5a6jG~ zz?uAYKShb_qJ50`Tk6!NB?E*neU#Y7d?Y27XMKbD2>WSy#o!!MJRGa;BUxD+i`1Rb!d-UjkMJF@Rf0;!%7zj4h3>)sJzsM&4 zw>{R<+>h5yC=z~%95YJ{I;tV2%rt`w{ad zLKVCQSnOZF(tHc(zmcy4*eFcz)WqF^HR4MK^OaQK(MpqsbJm83T|ep_T{~L0V5`5L zYDQdnTd@TUdF3^{A=D!f^BH#JL{DJ?{Ce$lvwX0i0=@$^5=049NZAr@#e4#_EZ;Um z_#_d)KO%Q$vu5+kRb*)#D1%!%UroqBP@YH*`fbEp_na#C*x6yIWhKV=-MMH#)N6(6 zh${u60a_2(^lc!%Gfj28jiWbq$T7c-6Rt;=tl{8bDT+nGI*o$qtT+c1Spm#RiU8AX z%O(mit?HmykbcCK#1Hd z(iN)@i+c=n1=pgh1kqTQ=5pth%Jne^Z~DDWd9bn9V6xALbGWaGv17-lq7KHG254>i z*m0m67&1@=H;r=1>YFR-p77gr1$LJmOivc}!2EV-DK_#K-%eaFl@HIzj1RHPc*XzA zs{EMU3H7OE%KrOkAlQw4kTEMg=3&rEYSu5!eyzQtQwO)o?UamJ=cRD$SBpNlt&1NN zza1`pWc&bh=_89tEy!ROgC8M}e#iODjOPqPjt4)3%#j65;*NI0$RW*J>E{G64)^F> zjp2Sf=|9ouLa-$8jCB*dA5!l3OFKTqq#~89V>^8F5lSxOQ8;N}NJgONuVd&17c|n@ zCs0x|Z#n-^&|M4t6{_ga5+MQR7>y9tM7UEwU#{Gvc2$ZBa-EzuMA*hiz@|XlD_vS| z33T;Gj^KK-bve%r#D$ed94kN|TblXI?~u*d;krd|G_1f>Us<{pv;7V0sJBjFu|n78 zG4%P^@uOGhhFwu~U$qouUL!d3(a32}x45~~8uw%WQm1iIwXpV(sH3HpEpMfbUKSkB zPm`4<)REu0hon_t&tpGL*p>=7rZR_TI-?ZxeD90UZ&}|Wr)L#&UgR~`%HvFR3isgz zt2m(a%rK|5@9jR6!o5T$G4+67O-r&G?PQ!V7dQbVVXSymQHsehxHcMb_%h)bE7Ny& zAM4vaL2yqin#*ay5K^abjT=w;y=Y(T1qE%oC!8d0)}KUgx+h0>)@WfI&`pELdX)F} zaWB;cpKJeQ831j;`~gQ&$XUvlm-{2N1s;J8B6r}6BR`m?62JW622f5`o?n7}@>eSrx2Tuy&9S#1h`<`gK;I~F1# zbK^;GWC9TIR4Yc8p91dVQ9@y+>FRmh|P(7%{$%qkrKPa zhad44{Q)i5p}Kk{Ye}{)s9a!-(ccxH*V{HF;NesrFl;y{L{oZw2Vdw&L9mY7$7Cq$~9Ni}O20tn6>hPwU?sJAA)&+PMAlT^vO9h?WQIVZ7a7$X?#anT)|j3G!`Tn`hXbcRvKqa83OKjYUi z!|8@+Si~00)AZjF@TunSy3*BeleXtc5mwBnKAVZ!67O&o-@0smeiy9s>m*F8mUzqS zh6L1q*t+V-SOlUy+Pd9`vT~(<&a;~FX$L!~P@I^Knu1ljvPs}LmJ1G!wjA-B20Q4c zBAxn1x^uxb?70|2k$VE6sc?j8yfWb)JFqwi$ncP*_bn#AJz}?*sIq|eVTbuB8h#^T zV>+Gcv+|!WLt3uLVW+aW+n#N6O1mm8N11M8O_~tzxBDo}oatNx%I!q+(FfsMo}?|5 z_pxS^^|2E3YGFP>l`2gojy*2Kv^JAib$}k5HH*ds>eK3LH^`6}6(md9zL^CCdms(n zp-;amsE5GqN$!vkE&(Bxe0MymFdFH+|%;^(eYH28BL1 zN*n9OE>Q=}o`+wduSF;~W>KB?0awYdUOvuDH zzW1llJ7^hGa&+4R6F23Be{;IY>(be{m#2=TEh^gnwQbjh9lQMIXQ+g*Qz?0R>-{=q zMw#aH&@2$}Tb}-M`AW5U<3=};58iq4VFdxEggw#yKs2A2FUx_KQ zB2}iKC+jBSVwi2#xroJYXY2Oz#hmN%IR;BhB6y`iT>7QE9u)s49I2E4R-fgK z;)xL+u4gK1+T`u8Qtro7cZolK0>JChHF%?v;nc-caCf;2ehlA;KbOTe9RcvaxcmjR z_HsjgL|OUMntY%%Py5s4WkL2L_pak}Z0<67pyuo#X|NDoccX+%UwGbj8UW&RC?)iI zZU7mXhBpiMNG!uB=|odG^YQP!Eg~3(A^y*qLEr~7okQd5UUcG4BQ5yVgO867zgYwj zBvzGsRaZ!5p3jXWqb;#7)_Q={pEbPH2~t+Loljat=VbK`QOq^D>d(7)$_I`?idK!d z+~e2PwuWUrVS9y-)>v${lmBG6XUC4xV%PX0iu zWkJduvmH$+x&UHx{+AB%?XRlf2f$QfWuc4E>L*`_x_m?c#2Fc2awo z^DQauwd!6gU1-U>)2KH<<_vWBu~oCH&g0OVTR8AZK^2uVdo~-y{UCZpvHxTiP5*kf zBx>?-4@h{J&=TgdTGhASCiZ{XUbdOMv4sOw*1=}U)_6f4eFD&222=)dSZ%=MEw=6*)@}N`wDN(3uo}dJ^ZYoqGz*$V3H!Qm z!8SwtWJSrARj?)u z^b2WVVrwDT!OT2{Jok2#qL;U?kBkNRK-|aA`gq-rZU;HCazG_A*84;f%uTC)pvd>` zz1u*5=VpY*7_$v~*h*)d+_g&jVMs?y4DA-!^9Pt0Nvt-0s`eN(GAJ#@Thu}8aR9%>~k_eI$|n@k9Y(sK3irjSdZG2xDBsN z1If0HwoThSh(P;nQ3^_8!gTzIX@cyi?sN4P>~Ie&9=c_~CIV>#2R}D^xkRoMC1&|7 zH-T~G8$Rs?l5tnq92200^~lC4PRTCE?TeDW`#3TDeMoP$bPn(O$ImX3xe}X~7omH@ zYRi-FPXf=Lt$2AC?(P^aq9oF(|L4R%x08wU0(G633W|8r( zb0^BFL}GoQBWvGcRA2xf!4FJ{q&91^f@73((?0I7x21nWReq`z8mm|sFB$j!g-}`A z2pxk6_C7=g-K0VOIyL_`n}Bn~J+l7QqmDj;A0CC3>g=OH5!h8!dhVHo1=SGRk!zu&1- z_0_#~t4eX=s)mo)})230?OS;@1 z{2lM)iIXsfhWK#q+C0tt=eK=chFwoKQ+uK4Y3cnGzMbe<=0(>~vds5V$?LjYM6D^= z-G7idHHf1mpda1YIgaTTyIo_Iv-&=WFdAP1a#tUe3n)KH&d4B$>d| zIU`1rhnn`;#M==Ws~oSLu4~}fm5|8b2Sm1~euM}lP-d)Q_?-p?9?PP6Y!jGGD->p`2`a zPs4yCJXIZL0x{vOi0rUvtEXK z>a_^jYo!G{MM&0_LB+U`@mx3ZHv>05r1-Z#JnO}^U}TAq%HC6qUR=w@nv{9%jUcV; zYzG^Xx)-cbH%|Yb*fa1mAXG^RrHT|DLZqG8wEepfZNX}tQF*%X5ztQ zB0m8f8OHob@vhkMkb+{$9kBo>i?T$@9k~s}V)61GwL?x}B{P}%s(}2DXYKVik2S5m zIp`(^3SM#l9CYP3hXuqh*ZaADN~!tt5LNIdu+)!0ZcRUS!;B6{e#X>i*g69GkK5FY z%w>OR^2zHDq7`mJ6~NcvSJ8}F3veLNEKiC>K1Z|bX#;XQ26T1H;RNIv_S((|4!z~? z3>NY!mU&38-()e9A1spZLms;ezjFbAs`=0ECm%nqbIdx2>zkB_@3>N@GUpxVuFBW@ z`Gx!C(r(3Nn}eI?spuTpdi$N_J;A7`Flm*=p%6Bz;XULnW6}ZBkmSTU%R)h)@L)$0 z13JQ4v9};iG#+!jp;3az*1c)=qMmv2oqO6rFy+-Mk!JGUW6;#XTeUqYw z34mn=5a&`?Q_|LBo2jQM^^)_0kL2nd?8J`6g$MY+wXB|un@(xO= z-_uF$g}9lFsiXIFn4(z*coYE-XL|i>juG2QT-i$@k7Mmw&hblY(s1e35u~i0-p>BF z=046z5agMzVC`=Vy0Tml=ox$wg?5SDPJ9eo3@&UBT0ZId^ykjrQ{Dh+SwNB?6{)OoNH~n2N=p4bqr31+mcF)84>~?24%m8 z%oZWtjln4ef)j?YNTV1ER&yM1+uBW5@hMazaLnmcdg&@1NiDs3EDO2xd*X} zu*r=XUxo9m>Iq?K1|!iM;Y`uLXjU>4EvJ18)-IpKxpdht7CuoajJ4O#*eOoS1oU@v zv+hNsXoN?6p4Ya*l+s=^wn)CVCWe5>_C^JkwNmzWg3q#CQe&=7$L?UaoDdq3598$j zj`+m44*zr~bo+gfLTXAW{*T3jj63QZQw%M2@BP1S1D8bDLhdTpF8;<+)__?s(9k~PvD z*MKhz%exp#%_DfI*%JK3 z(wBvWpHbfDC$TJzsBeKLi>#m|-DI-+sfkAkW^R2isdKYqk}->WZZj8MD#!72-c?Q? zEgL;5d4@7-d`DCpDIlAjP|Byo~~g zrOe6q_^mUog-BIGW!;_S-5;5huAP0NHcej5PvCh&OlAUC%iWDRDskp5q>`Cf%FmNU z3-({B;2?nG(M1jAoG;jOD{SC3Qs{WBjAMvw8`JRfbo6?`GX=5KJO`rs*fs7V1da0w zM<0Sj)hz-~)7=XCj9l+@BkX%tEV2hkWC8nlB&qkkp8fC0T;V=}rEWw&K2FwpCiFSd z@gY)Xf5?VrOP6g@oSX$dU}i-EYJj-|RqU=u@tS;rS^Nmu(eA;L<6OBraoZhJe%@Z+ z9c}j>%?!M_ogWgUpq*u~I-<1w*fZA7UZu4W{EL6BKmht6QM~3D1hbcm09}vnIc9e?}&LR>-UL=T`oK|8mj);=H18mAcvdBk*r-t?p}k|*&Ujb z$hYsjMTO8z*C{^Ro122&tpl~L_1IaMk;q3CEwZg~J=SWhnP<#-KV585E}f6=5d`>hLyUBF zcjqmJfo|-lWVo7FLyM!$sPe#TWl__?=gjimZ*aI!VQ%@NbPiTA`}`p6&3k5S%fm%f z8_QW)xvWU4Eo(+wc#;XPd2eR7Yv)#?IeMC8GZGi;f+pN6+H5GZ;0-pz@^Pvo2eymG zJx?y)(zUD5+}YzTzWDRUjWPe`ihj|ZJ;BE$3p6`%2CuN>-#@?XK)^eBlb5r*L^R!9t!>!JdW- zu}kK9WlV0GMUR@Z&Y9s%SQ=+?U*Il`g<{xmE56aJVqG<%^wyf(i;H>SIVgXNe8Ubs zBQ75&y$uiA-yEiLMc_YwGL;uIb9Jtzl=)4tbE{G6=KpzA$McC|$CSF03xf1E3zrp*U!5+Em*u7FSjo zCkYL`TArQFV(bpa z^e0#@qh&8L;uTQhX>chV+>w%Q_d8}G;eFy?>O|$MiH|WbU4Gp_8bZ%R8#gnhHTP4x zfXV25f2y~&5RzDtNuOL0<{GIZ!#b(iMN{y)Ln$#ll zDwEh}q6;3<#MdisM_s&`0 z_>YV=l&~%;;H7~$rem!)F=~KxqEp`W!4j}WH3EbU{&}|mL#=-@kEJo(I3g!(kj}D; z3tjGVCpA91PyCsSG^t{QJ=b2JDIkB_;`H4SAd7>sGc)Wr1jvR_OI+8Q4%pAWohG3X zptW>M6-6h*07VyLZ(#@uI{%Py$f$fD;bR^~jX7mbE}Ws|IOx11>3(%`YNA}kbmZN% zZQNrw%Rzp(`z--KQW6GKB2b~(7fke$^+HtY2>@pO=HLeOf>#E`!{rYwlaKk@0CZ5R z=Gz&3-r!?<)_Nrp)Og)sU;nEXDfpP1U9@}$U2htBQ1%~}np;jk_cY~|EN75NyDxl? z^dv1{v)e>dHe3O5baH8a0w_ZZVX2aR@T+R28|?P|Ur9j0A@c`Mc>f0KfZ^=Y0VVaG z{x(mXomD(N^@8j4aHBeMuN$`EuX*bm*%~vEHN1rZ5^1sOuhy8EuO5sU9P`b9_TcQ2 zOmLj7O7iJt@mv9K3 zW$%-?^&jsC630Rx#xfsH@HkA1X+*49Z~-FirrEJY`v73^Y21_Pq=6mzdVu%fFH$4K zs#F+(nBmCnc0$Z-K3BSZKluRm`x(cr-uy1t*%FzJ&P#3^3lWX4j?;MLn$5(aMnLtL zY?7#v0MER&Y-e+Wch69z=sHq@wnO{^T?#kR0f(@2|iH^gc{?oEpDX%SUi% z$oMvo9<_i||n}0h&J#)qL|_18Lyr&$iCt5dszEeHkdg@`V}5`Bz(*KdKLY-SQO_ z?_H;5IWX2kxGu`Ny(_->LfXR;WwG3{WjmP6jJ$aYH$y%tj8$y)@PDE*HC$bg8v{1^ zIbCDGjSDwRuuf!XHHGrVXJ#B_7~vhrdlQ@_c3~V za?cjVUCG@c@4Zz)FSQ@we9&|yMb*?&;S|oBbuwy?<4NHug3(YzGE%dvcdq~v^{Ler zj*}PIeS}@F$aoU?mI1l;D%TzdU82B6GZg_%f69Ql`ZaVobfAPnhhW#O6pJqGf}Wej z17ng$+?m<(%U_qhjOs_+Rr2D#nEFL;jJ(7d0Rs%zve9z{+i3bat&W3;AuzEF<)(9> zG=SGWCsqKg<993xTwPv8Ded7)X9)Eo@qDtyfNRG|JI+?|&n+(T6E4^3(IB=XhawB} ztk&M&61KgA!^iL`4sJdNZ;0WOH<)X{uhEDx-n5s7?6COp^NkXKFhL{E+-uJRJgd>F z(`qQiU81-rC-lAkHe2BATA3pxuXulNV}ac|Ek_@>uO)04wHl6(p%EX&>usPZJVuXr zuD}2mEakIBi$@3`netylCxGBYj|C|*UnV`ijgqKq%O+Y*UV zHM0P{lN90Bdj5Ac1=-8Q-%s94b8{Y#kEd-kWqfVP%kc13)ZYeZj5;yJ8$*dSJcaa^ z4}e8|Cn`%!ssIr3V=pPmMZq!XCBf^(frIL1SI`bm1TTrKHl2w>_tp`5I96QKQ&9~e zjDKh3?jf92ZyWL#hzY80z@hCG0YMaiZiXlcaKOLLRT|9v>!dG(1BN=~3~B>?8q`ua zb=b<1MT2G!xHpSBH}BzNvV~MTnny#=21ix1g53tSj*5t11oK^aqJP^DoT%~x=S2EY zkk6%7KN}nY7K$IH9d&r2zm@Zu?)ARq1gmHSLax~U!GjQ@uu6_3VQThpX8D#+r31dg z+~V)kS?mCP+2({Ze*ztlw3CH8Z5JnS);~_BwoE?C!LdSm@4d?=R02y|klf2VixFUL zM3DyD{{SaZX7#^%rZ@EZVG{UIcgF<{_84T!Ye&DshV4HX`V^`B{RGyH*j7AG)cNqlO;ih zmk%{fVYzqaBrQOwi+{NhNsxz6gd9W%8Bh?#cnN$wr*n6S#JMiG{MBUQmn@)9WOISw zC{p~4fV!(aR%AA<>|mvn0`wvqLAu;2+R)ySqF{O%f$~d;y`h$@r%_$6M&e<==))Yi>?jBKQ2O3R(7CnS8gA z9ige~3YC0HA)n4Fe_jDm=N zcoZ8KDC)#89cw$(Wv`LJ_t9Alo556?&a#BXr`um|i9PzkE-dujx=R-ylOSTG7*kKv zOU6m^9C+zjc_;IGTqkJ(z)}3mOS?gbKt*h(@P;28f{7C7qUHuUZfNL`LNZ646yj(! zqmkjA_%sQgMu~DLt_-6cyXrd?5za(NI?P7tHCiKcV&|plTd!%ScAJ{gB)AQAi}PRE%Ky z?XBstA4c~Fxl2&Vmk<}JbSl|(qKeIcb<}F#vFDz|1<7{fq0I?bY_#72=?9HZn@d6t7iyRk z1l8h6IP%A{0sjS+y0SwCa=fvW`CkrKotXllcB7By2Ayu54yaFut!WC5tX*pU)sxRX zjwmc8(v5U2P{(M#w0vVIzs#whoMslL+0Njht!?ypmkk6tJcI8UWcvdnodlZdGtCO| zCSRMvG%^xvUhEVuWh!rzG&NB%MShbFtQ+4&mpE&YJeo_nMF|RB=}WhLgt-l-(*+5j zvRtC%mZ@N%o5l%I^##7*xhAGC^STtAbyL_Sn`?BCe+BlG;BcgQ;ap<`vjDAI&ea{f z%gy81jXFY%!I_NtPFZdQ^5gi<^%$Pe<|OI;-H)L@g(7lukq+Lx&#zGAmCmCap0sf7@^%~m`$bHg{InA|#9axU|iM$B|#;cHAE-v8E^;aOox73KprNEiKN;eoo zv_AprB+DhugCb#-}%fvWF%5)^qzGcY89YL_+9!kyZwevrMkmd z#^=1fUakC}LCniE)AnJ)+`obAD zG=Jl0W0B7<`RgKF&GeT4U79FN>HrVnF2QDvj$ko6j<`BIDN?Vvc13v(V9O;q>57?p zapMinh^^*yjQe}4Nga}~CPZS>5+HolCWoX*q&ePGq3z@ZH$%*KCa)Bt%m4Do>>vsc zTPby(2F6mz{wgi#ni4wp10Sw&2zOF=a|(J;!+&IFPoTuV?=m>rEXZbeBI#sMO!v)q z1+u7fC1+ieqivTLqf8Ha3pQ@LWzXxm6)u;wlO9wgF(G&x=jkyMQh^nW(vC|Uzsm=& zkI5sgK2V;-*)XSVfm?A_xV!NJ5L*F0Z=DLHZIt)lNAf|-^LMx;g#~(8DlSO;2nyEt z{4_fMb-E`YXeOYaN=`s796a^v}b=+L#Sxt1Xm-I8`i=tL&XibwXN-jaVUn25^c%|0p`{L z77c?nwT;#~pJSF{PjPvBYXHPuO#vxcR#F6HCNbLOx*r#yXqv%DuX`QHGabd*b5R=H zQLBbqJ(Pz7moUIHc8yk<-1tm4_2quHywGL#+;yXBoRr3s8!9SxpxaxX16r2VeqS9< z&+s$KHE<-#E=l}aaquow&?e2du?K9)&!}!kB{@b|T}Bz!z?{TQhZSZE3S06o@u|N8 zQIlk!@g*2k*n)araPbZV-b}UiVw6GrFW}LYXbF6X1}MFMaoBWx1XwQ5SR|oT|IVfX z!#tug&@v?SR*6}hPA*L26=_9Q=1XH zM!H)*kXZm!6`Nodf)u$Mk+;rID%b6~zLMoaF%}L?wY0pALPzy)%8tq3>IX;;W$9oQ zO$<~Ac@kW|bAD|Zj!^CJlBW8LbKrS=@psG#H2-%HYEk0n>m+&jIad%5d}7FPgOGko zZ0s1Kyx3J+5dBIqBDbq~_;h>LP(t)+4v^o8Vt&LQBMp4HleIo=^koo!5}cTEEM*0X z6C-J(Sq+7sz!_aR6rsGS1N2TKqGy72Y>ff|w~AiX(Zuk1=nXnUD%HZ+!)XLg^?r+K zkC+#Tsxp2n*{uA)*Yl!UOh90B`2?be?}vggm@nxq)fOnIuTr>vgT6hBlIFz+OGrta zIMb>xfe=bux&94z0=V)Qqwc45T>_3!Er=ycg%fB=OZq^T$YD1=&Edl=FOWEi1VrOh zH+_Rznixu(oG4Xb;A9MA#5k0J9QD&nwM{`l*8I`R#^4;VAZohPv~C8PkEjJ zY_l^V`Sc-6IhhR3BbX?Z8O~~PGUsf2R+Yjfer~X)tX3_H69C?!gJxC*8s^NI`RzD3 zbiC^HtXbgcL%__`|J+VMXeI-zxeF-kG;9tx9aK)8R)3-gWUplPn3gUc=qfa`6u40D zge8edpd20@7Q=kg_PDX1?2nE8-XZv-k2RzrP7PPrc6_ECb$>JH6J^iN;~>ZPF7Tn2|1N?yVZ z-uB?Yvw#73CnO$547H|(W@Z2v2RiUG!0!{_N4*0xQ}ctF@BVWoeUpDN-+hl%>Hiv4 z!h7y~@=q^-BegoRP)8Q($k`n^yQ8@2&y)WsZaRvajzaLG5d7#h_kZSVE|>P%jrK|$ zgWKFF0gdC%_GBaIlDD{T*JUDmf3tq2RD5bh{KuZ+C_BE{p> z74HWjQm5Oe93dnnr&Z13hXQ~t{v%NekbjUU^*apT5rW4(-O$GuT^qBE=h5h!t9#jJ zm-O76X51y8dH-%``;c{{KF7>Jty>WZeKdSw|^a7lOJC?7K2RqucnXy`(VdU(wKS$aGGBjoF))m zbycZx4umOMY12ks2G|jV!TsxG1p&a~c04?56ruNuq>YM29vfu!)hmmhMZh}aqkTA0 zyo6cW?2gE$Jxqch^(c}13}`K>%V%~lKz7;9mD$!0cl+5KM9BQSvBa;)sk$d!j5&`{I8Nh_?Xn(;8lV3^&rmVI0gZv zfVJrm=XJH|y(_}~X@IX#xgcrB#Q=S`Ktt=a`|9)ENLTq-gNM=Bklf{HE>hQ)j_(OJ z0M$lKCc#k>X&4I)!`Cw4$T6*@=H8HT3Fos>^MH&C5PBjhP!rGZk+;CPSd>8L@<$L* ze)_?AU|dvPluXfOdL*{7+iWKkG3p1?0nf~2if*RLNU?hbj>O7)EhY@$A^w+X0Of~x zlc%mASQZjw@U3rfwy9)XhdQ&JQ`8O9gvdTnLs?1yHb%2r=*F}HJkzq>@`i*mK;$wS zsc-zN4a#wRkO!#!l?QN1IxtDMUuGF#O?zxKS?1}!XXNNPsvKr1&^+4CI3Vn(4?gr4 zwG;`yYxKkeSAwW?d=R3UvN88`z{67^wbKE2sdGPlcN3vHxv;;8`9N^6Y}>P2WGHD$ zHp$boRstwDsvo=Fr-RG+nzijI<{GegjGxt$UPDmy?-r+kqV~JzsYMC6V|)?B4>rBz6KB0jbL zc2B*?<*F#3<5nA2`v$4kc{LvgOxH(?$KLWZ&CUobc{fZ~*wNj_HJ_%mn#MhE+a6RB znQ)Ug@=i}vQsfhFn>eAh={Bc{-Tm_wdNVLK{y#6x8ZI9efzVu23Gxf$Wov`S<-2(K z1Oj%~%o_vO#)FLzgXJY zeNKZ(Q1U`irv2=W4*da0LpI@}svp9S)eUWnaHP2qIFLMy6IcBh0-P7WY}0!aC|fwy z)_K^pUVel}jj7>Go)F**sLLRmCZ#WNve~)MM1&*yh_DXU+ptkrSp-L4(2&U1tLkI^ z*emh1UinFhGleZlX}th;zaE!r*$tKREn=o099B*m3MIfogQNUTAI@=%(veq1 z9H?wPLWZ)~x8$ckpiEQOe^C@l2PkeOmsC77SQP||*+n0;NSqs`7kTxeb6Ov$h`Qc$ zmr$tKZ6b@-?hroiQ`JuargInj413OkmcsOrToe2t+C!gp6|C;8GDk)qzHX}>vFMo( z(Z^=qsQYt#z6EA8{MLGf8>A$1^zW$={$rP`_`pt|r#D0hTqBxzuNx|KT3x~b;MX~m zoOdknD4oK5&$TlgZ@;p{&UH@_Z=Fdo-p>($|H;x#YKO82Zydo%oFWKiF^izC*Lq8B zMHKA-xgg^vt-Sh^7pG}Ya3o9PCBGO_)2jY{?-22lgiy>rb_ZUkD}A!NLn;BF%o z`{e-7E9TVV{^rqwy46qy>IsK<6D`G?cw4ZxC42`GM=nVMI$%qwtJ$Na!J$EGJEQ|K zK*j>s?gHV|UVePG?xMvTKPR9v6Y7s&i*hOm_Jg$zk&n_(YCE1212DaypV~!NU|pcK z9ZDQP*&l)2iUod@?h^1? zPGD_^vOPDnj{%wZ3IgHOlw6`r3k8P)vJc@rrD}cg$LfrswW<0zMSw&34zd^xWSnn@ zKssawtnHBAWF;R1GQqO|!U=eC{h|sDI20amqy~;rp9GHwmqKfMb!>(i9Lg6KFguIp zRj@YdcCfZX84O)XNMTOaA1Ta{!W=2gk%c+3Fh?%w$R+(9mMR>@Fh?=WQP6u7^d7yE z{ttd78I0Dg1e)#HZ4HFOXmV=p2*-MG;?<~dIfm)-ogv)#k3gzwlr}x#%y`_SwDkf1 z?(P<;nAc1Vp>=hTMdtM(f3~q*I(z>#9oWpPyVMw0=Ir|?>arfDvqzGH;-AoCQh}cW z0MhYm06Zc^SXbza&VtuFwrnY4kwDTyt!8kdG9 zgh{NK3k~R0(n;mYR`TsTe`f#u+@-aqXG(Qq3FfdDhv zooidG(yG-v{Q?18%ki9WtzG|>%?2hv>v2D3Ac4w`j=0IGH~0;Jrw64mY$ zT=qOk<^C1}kYM@C4<1(pKS-&mWhsLry>QPd>n8wYK5Ofrdzb(KnOI+^k`DkI`S3c* z(hvaUIaZ_eB>w<1$M6(6eOL|(eRBPUb!2I^&2VKj7@-|67Z%S;_jl^#@?{%^HuLjB z_nOfAk`=3!FeWp&ffEANiE8^tqd+amq8ETx zkL`kA>kL8=m{#&cuM&X3q7o;&#DIY8C}y4=ffn~}u*`(*c-ytIjSuytQxdF2h9iu;2z)oG|oMsF64 zF;E_18eL9Feo)GU0DPsY?BGDf}$wdKRFHC)1y(`megkw{# zmJ{TH69V30G`Z+VFE|=WR^88g2ljI~yat8b!#a}UEb>}5HTsZyPq<{(QcO=c2gn3? zhM&p(J3{;)sRD(nSH}V>7&KV_f(%b{_^`m$g1+rv7yND&v+c0e>snK1kV;aiJiVsv zg$iSV%}tiSU90aNCcS|65SGEz#J%8Gg^3sk2kd4S%ai18NG>l|;vQ{{wXBUbZ-l(H zuiLJAy~e#y%2zQaOSHt?Q;Z`lzU;(%)CC7sX381{8X%7oZDFH+WA)<#baa8~ioI`~R&U&m=(5B51o3|F(AiNUA=@{ z)z7tU(VI`w?-(Ky$eG#$=c&S7;_{K(<-+&IR5Q!bGM*SuiJkgRM8a~&n5(rZk-U^w zb~&mkA4X(Pe{cos<^ApT#{VZMI;US;_4v4cJY1ex-i*S*MzE3p-oDSmnrQjmMi_DY zS+VP(9~6}IscFEL8#|mv&q5Xr8n^_oVYWKdw}xa;=iYiM9mfVR=EvztVR8aWBuEu89Ucu#OXT>>=y|eH`WwD4 z(=Q8we~ElMQ*|EH>l%Q8L!Awd`O1OWye_Nkom2GgIZL#>MVS`R5Zkx`(u+O0f;1#h zMq=zMM;cBAJ8*(z%zkSEkc z=`O0=Wa9`5^#N$Z$qq?NQ9v%c1r59e$SS$-%$J)0U%bJ>rUnKqN}z$mXdm@CfNApG zrIN28W5n$X4!BRI2JDV|m=7JSSPftuw(17b#+djksZEjW_Bsd#DSH_THnU0xR$h8( zF4Mgxhs~IVYJUrXK-bBr-V|%RuyF8suT16d^dF=E z%kX1Vv>(92C9?n)o&&Wp_?>?M^lci6z{5>Dsv-cAWL?!iLP)j0?lUyF7qk+|D||3) zVK(B_v|WHMb(rkIt^91NxpybFTz+*Zye7uq%uh#dOHIOI<7&K##D#VH^}5#EsUNMRj_V!l}d3~ z9j_ZvF+F@a4)qH8{Z}GJV)I4;lfAI`MS*~opF3~YnxiT& z{>}~X)602q81K&WbVkT;Y&MJ)%FXOmio=QJJ>B+Sajf3SLj-;;GH4~q*3!e+L;=0j z`a%Cu46ilk12m8XWM35IRFMc||4={5_y?4v(WUNWoRq*ke1`22zBi{sRqZ4ohK)da3(8!Y z|BiH>&R(hMHmI&{${$;PC4YfGMS{{4d9FI!)v9l|d(XuasJTjrp_pFi2W2%3HBblC z#Vn^*Lr7Dhf%Cwoq_Lu&*n%pI>?_KSV4&Us8u*u@&_jyiQ0YFVbX#I`DwboaE&^ss zgk#2rBIB27kktDt(>gUy0`PfdNl)(G)df@yDWbdbqvzIe0#!}8w%N}m@Ud{F8oCBp zhZo%JvZ7p6)^dXR4T#|F0W^2<>-4n^$jU+YnIEhOB6vRqR<0+5YoI^_SUGT?|J5Ef zKiHUGeuF{kb=?D?3Mw@z zX?<5fN(~wSJzGd<89ZLe?1NvQ$Yr4kg}R7zew}Ijm;X5p6!o5NB`O}^-~^9}5GsNX ziIm<2)qx}jB@Zz%-3ed|8x+_RZ;OC6E4L_4oCQ4{FBvl@8i)WXLIZzv9!gw&u>ew( zRuQWZxZ6k?zL>>!J(RW(j~2hfn28!m)te`;2T}fh_X+yNa?Bby;u6Ii5{{EyWOOx+`)h2)brHBUthQ zB;hlxQR5vz3IPGJRU{B&nOW%4q3(|*AYW}3F?7UxJ|fB=5#^66D~>8Fjw&mTN^$<- z034O#9Mx_e)ovXX`yCbg9aV=ORfisx6aP2m#K^F!kT^6^IQ{mH!_;=&O6{(g^tyf7 z<~@nE35v1x44xGJ`* z4A(vkCM$KFaB71{@e^D7D@b5O- zA1~h;^IvJihBx)GVZ|nMO-OWP-7nHEa-pT@7)(~@U|FDY{i7&AG7J^le7Ql* zb++ye{fA)yXX(>*V*bO|g#eO*s{pbNaT7G&@71Y!%M{|q$OP6UaNsOPziowYU1uym z5Og26@H%L;Qv1$-He#)e%&M3#&Cz*)SO&G zjz6iqF;TeJa?s_2ewo`u?azqbvE%|M%d&+zG^{X~|0FnaPnQ9lvnP z1l%&?4u+&gW|T<*Zz z7Q0+d<7HaEUkqFFL%LN^omWIOyel%GOnZ1e+VclEESo5?h7VzYO!uPUG^yX$bw|QzvDfe?MClC=1`#4(oUGmd`q)If?OXfjzrg8GbqJhR zy4FPv5wx#;aq3Wqz-efJ1}ue55#`GRfz!YF%=He?z+o8lG9{#UcZWwJ%+6A!4l}(n zk*3RmImooOU~~S`Gt)}4cI1x!vr&0dd{pI@h8v7~_fG3(!&I+_b?RAp87~grWEr~s zr7erGt{}y+4W7J)azHVk^Olyo+enye$OP`i&De?`U>Pog zX~>3>V=6bBviYEU!P@rZ@V)3DfzllknI09eQY$q#!AUrkqFgd*zW`%cK!x{4S_|?l zifQ`^S-7k$3H#@u%WNXf2wM|@R!>jG zg7z=JvROb5a7ooqfj|!K1?*9sfVnL~yF_0|0`O`nXZDazz}yxG1LyHF$ZrqcCggcb zA!VNK+9!SQrsl@3A+d{BWd(x$e)+a|6yVCX;}VmI?g+YG-h4EY$Ga zEwO_Ye<9k3cCu3htt;%6ub1;4hxwtOUr7<$*K<2=Z=842vLg3n!hqHKZKI^K^O*cR zq4T$wEld0=g(3@S)_|NyGSLLxQNNkM|zBi)w1;1`@m& z>hI^Q*0&bo&20>qah1A0x0hFO!NTj#i0foNhnqHh-n%UPQU7usJp0Snucd+qWA>!> zBURFWtYiRaP~tuN@{Bw%l%t>O?R0|mCf-vo@T6R1Y=QLS;LedQpy1qE(GYuM_|uf=r2##&Cpsos5M z=XQwoQvss7EiqBI+s`MR16{o?`}SGBo2&*TPkB%}OG@adCAD;c%|&Cf2$3pL81T zEX>;q-@9sR{|V(c7c_TSSd`{|CRa`Cq*8q3jHFvGDu4A(kyf+Vp65f>*0$MwiYKsP zF+Ow<+Rl8|b$#z(3oX~vZ)o}_A<_ZmeZ(%3J|iN+9lsJb_LwrQURNa1m^wPI-WQR zBP=UOQTyW+sv9FCEyz>9WgH&^2jWdi3_0yCRugq^gnIsA`bRhZ`jY*XKLu4)XTX-9 zJO8(BVuofp!rvFS*SBjVh^sh@fl%iy7hd>q2S6=3cHgv8|+3QO2#S1fsRpb@Xs5Cu1Tf!6xfk zC3sRI)ta$crd@tsIHJ$1_Xa#^h(n`|PV{|Xkip*xK4#E%XkFt?z^>~-Ttm$PE(#tX zd)=dENevfiZIb%3;TgpnYSEGH&h1X9rYTllmB}7T_Zi6>%DY1u-rVDS)Mk*q>3tO& zp6U}Pm`sam@?2`0M$|;S%CFi$j5M7zP&6i9e3rXhztXOe++e-uDNSLrx!ArQ>(%wi zVOYFvJyvbTrmu8?E0A~dZT(YI7vs3gDB&*~rR9|gzLjo9`SxomCHBep_)nL%^1>tU zOh=iA>a?INEuwWAWxKd0Q)ln#FD2w)HR709F2E-F=dbV!NAzP#zgavqSO2u0s+RM< zEOB<=GMvIfHtDC0htT)6`4XgO;F&)rmh55-R0PvZFXt(-Ykz=3Mtr;~ZWfen@Zf_W z(!){byKGA6wk(ET%VQu1wH8%g`aGYRuUO3Ayyq5cMPSt^Fbgm8&2{h}ag1ifmy6nu zC{_i7bR*B}+EvgWX+rO*V-f3J$$%E&!n8;}m8W8Fc8KR5-NhD(VlnO0E}j>z8^=nZ zcAVSn1i!3_g;idPMcB)Q=T}dJI#!~wDz|yQZ;V*Fe=HrgHxF4&S)Tj+F>LDKjYz*Y4*2n(+mVbc(3Bc%^#Sj5*2&9hKrhXL{Ji*Q*;R;z>PHPSUi!Po`aj(z}cJglp@nmR`

    gX*`Z!T7^}drviA0h#iobs+$Qqa&nvK_5_BSnz6b*@2Yy{IYYa?Q< zMZfG|=f%r*v{obqE7yotA3dJIZYP`1@nS@-d(O8yhQ(OSV*@v4Uk2r zSg^tCF_t${?QWZA^5YmM5kq9ey2cl-!h`b(=DF-4YgIkYXuU5f?*iK&@ilph5u>a z^AT|ewO}I+eai0f)*6qQYI&(De=QOiaf6uEqWreRvz5{B(+V{#m(}s?;4f4Xcp3tH zL;*!cRBwEPuIr)HeCvEcXld)TDTSgGzeVcOaEi6>jCIPlVbM>$LnDvh7blK+<(H;? zO(|AZtXy70aCAO>|Ewwekvf*HeIUC1wDSScNF!?I`=|XwvHea$w$gM1mrR9P z#M`F(mV@37hq0l%C$h3rE#^G-_9bNd#GHLMWhi75es<~_Sjf^Yn8(k5T3?bIxM46+ zDL7^&9}?KN;C#z{S#H)mdj9$Ho<>DTWKK&7YOZy?!ag-&L`Ilr_$sma-y-(?1|IpO z_1Q6%i1rAk$x+9^j}Atf9-=?<-C!0Q?w(%K?L`J_`zp6%;I#&Lp@D6;SG0BUQ{8j52|GT6614WcpqsC_p2n8iEn*!DVgJiOdFYAnoQEP zp-f1w)!<_L!xBDJi+;4m&Ma1cOv_?=Pvq9fOf2=v3pp2*wtwJeYk+-|j^{{b?##P; z2OX&i4i0GJXAiKNy9@UQfHhK41_8;9Ny*31=Usc>vAcx=7{i*fN7c3UprC|OuKX)K zqNMPgT+9s0KF5zvLuyc*?@p?<9U7%+&(I%Uqj;dwc)BPNWpm2~ouAkf)|8^PD6(c$ zoQZO2;RCz{Yk9)yCRCBHmQyG0 zHof6DZw-RepHb6f^^VMM^3^DpWs|?azB&kBGZUCU={{q z#oi+x9jKBn{ZPBT>0c!aB3d*BxwOJ)4iNb8c~YjU;BT|?u65UU0}obu^Qae6R&bxS zJ@*&xb9u7Jm+{dp!G|d)Oo+C`h@C_$)5dtGT~cu=ZHt!I=3DOxW=y=Ed3bkTXxb*o zF(u{m{o=%t?l8gBw)w}rqKLGw!|~|h+f9#_!!sJ_V&~eZ`Ja-f+~_LTDdR=|KU{rx zR8!x!H6jXD6bns?O8KD_0U=69l-{I@5U@d{O9?F`K~YeVA_xc=g3^l=Lk~$5qzM73 zp@m2<0YVEQ2?<}k@4ol$`~Et~7-x*+oVE8_bIvvQ-dMa1CBPv98ykTdgt$%uap)^Z zC#~wk!@x1^Nq+5KW8|>`{K~dP7rGs~?YhLsRUH{X-En?>;D610*I*p`)Up1u69%xP z;GuT}_f7wujO|7(Sf;0v>iW>{bO|*YZ}?(rQ-EA#hC2?eOaTvLdg=+Qj55@!x9gR- zM}r8Q+($~NN&`{>F!OC$qYDinJLA{foo-c;W-U>zDf;E(RLo@7wce9fOm&ZXA4-oT z0s7=f8@bN+tq?6`MqI9OHTl-q-aGlAe_B!W-OCmE_tM-77v!7{2i&T)XkQI#gxfwN zla!!JP(Z6$)efFtr%Nw9vH%6ovo5}4tYEtAlhZrPa?2Ff*DzRA9rZ6@FwF zZ4A}#r)iaXN{}dN8M}=8@lc{7ax3aPkwZH$qGCGH-!YtnusG;)hqKcphqFhDW%T*% zn|qXzckSBCv$91~fbHMlbT7FC&x)=?HJVVZOOI{E3Ux-_JqCYjgs00F6NN}#F@>Xl zIv0Cn8WT3abDK{J3L!D=qkmKq65ARh;j%N^h+6a@s~vj05S)Mgj5hHgCX}=tEV2Fc z8+mi>(Z^8G7k|_sQ!?ew;Xdp15Q+k2a$iG%T5cGjG7drYqiGiY-P^@)iuF&n>ioxK zIW>~TDF(Io3{h2Q4Pd`lsHX0~!}|^-%!+^6y!2L<@b0&FY{{;OV&RIeSxQDSbF{0Q z(WC`)y`n41VDR?e(D}#k)t?_xZf9mRgh}zj7Ohb>5|C(nuD2c3nY8>|Tv;}?@W((2Dn!|(c+YR{Mx3jf%p^?-y#Xw)A zTAGxHW){4@kaawDS+4s!Hg%%Rb28hsTcEShG2Iqp*U|u_hp$l`0^pA!hvc|3!fu8eqtA)Nxi+Z;QE|pyL?+?10_S{k8ad@GUl9+J5X2=xO8t zgjjSqgtE2kNrN)9V0YZwl=5#ZK~fZ$${OQ3Bm%oYXs2#wg<0{Ur?IzdfviH0)(WOW zfQNPQ)mF4`BvFcY1!|QF%>-ANy-D3(cX=n0f4ZWrpl5qcgt)1ZF|84;1s&Fn(^?Ze z5s0{`sM@hr7NpjK%1|7i4kXhtTC&lR+C>{<^9K7TUVGGwjQ0&%qB>HF5$0=ORew=g zehuk=B2jSlmKj*zaf)iYJL$$%OvamXruzPcnvD4o(M`=))S-Sa&hK}<+&Sk6R>z7k zS%-#O#mZeOi7U+5C$$*G!u_OftmA-dJX8Ve6<{A%rPY@8{hzpo8HA&obm+X@$YP{3-~114*ABNqR6Wx=nI1xtl_hjCI>WIAOXJwW9epV87?}5QfHjHx;Z--;Brxmm7woexen(#6e$_j5un=D7If{ zHNIGpRP`~nT&*+=w^eTrK|O&WGs+F=P2HxBTUw&zERUkD5VbM2ll=e9?f>G`!M~ee zNS+%?+Sg@M_UowUTXp?sbs1&za2gb4!4@$qfWeS7@5w^l!8%vTN@<5JM3@tdM4Rr* zq+LyMdo}dN`QaVdu#A7_xnaNz*&ZskUDH$RmSRkUy(S8I077CL-e^BmTuIfP4T+_{ z!8~+;%{MD%#3Qq*{1(0jit3A72#T#@#Nc)Vc5zi!X(j{_a7x#I(}Y<#;ug7aCv5Qt z@7&hc_I}FZ=^%6HdvW?b#)ZV;pB9uX#8KE-&uhwLsFoUausixes5C@)B8D=w{@bz(MgHB(`?@RheNsq%kFKp*Ccrb0{u#( zchhKtY~@9R{jcHHX~GsX3`tA%ib4kI>clbxgOp9&vc7@xdke=0(B8xqYMaYFmM-}!wBqr^WbKg*aOi@s-VI)2XSTvAp$}0s zHtNbub^Wd6g!WA343V`B2P1~OE<-1CJ@9k^TN{-UJBfAyRgqOTBA&amI|c7AD%#4oOcR}h)2CA;L#_C?APeA&v?qCSPTBEPtusL6jQ6NnZ!8D8@V)o= z68B!`qFE!2gI=f;b||2Z<7Hs)XV3YFv2xYn@c+S%|0=5596eS0{AGAPXPfB#`Te8& zL|s;4m0pofn7Z1~J#z}PwzZcMi9E5rCkCx(Lt*{HxhrlC#CN7v>VR#JGshnS-!W>6 zogV_@8FF%MznEaCQUx#6ynO9s3K`NoI97g5?l60BOq`?3d;`K~sXq* z;R-cBX$-Smtv-TP1P2S9P>qCO+?tRI>4I|SCn;sUlKH6rPVGcI2 zcQe91B&o3SimI9eVR+VSZ(bFugq(A)pSwz|XW`N#B3W+o2EOJ$9|w=;B9~~@4;vGk zoCp|qU1){x@5Er_hgd)2Yucqx=LaH8lM$y4T;i-%IVc=@oGKoJ`esqrYrcdzI_Db0 zxWA^aBMN%kr5-Mk4s=Wb*7kN{I;lD_eZlF{!wXM@cXl-<>mp63uXbi1KI%%++`!v^ z`mEI?7^LudGKu-TB%tEeAHRq9j(*3C(w0;ej>D^3Mrjesoh>%o&@ij}>iao8-_jwm z%l!or{C(A*u#uBzW2qzs@xPmR>qfR+_!09RPp$aUfR#6!R{Tml+8ELA8$O%?^J*l& zH8W$}uO*Xf#l1$Xeg!u4&HEo`&-;*w!~V-46b=q&VNcpbAX}}NiULn3$>Q`E+~LDt z>Gv>{<&Ilz>M6FHWseiKb%4*3T(4%iX4$N`y0B{fQjojA_|MKyPF?mjXHuaBNFNPD zw)x}5{pwdsV4b!PLN~P}b9lT&Tb4+dkN~sqG`Caq;_e6gUmC7$sy6U57x2(A+}IR! zg!%|5W>#EP{#X{#0MqlRmiSO**Cc&cwXMG@EpvrYcQmo@%x|1#`rO3XbRUYy{1SGl zPb+;ZQ*i@(sq4GZOjAZpwriLCPW^?*cB#`W$D4Q+4ifJ)?`>naT*tc)hkr_){J7BP zL-NTZ(*|&BG8UKk{zv2ZD{}ol#aYR@==ji%mXs({@a*@()5j&nqtJ@+ehBent?U+x zrUNUbW1EH%+nSq~+WUNMy2iJ$aHMM)qoByZZ%Gsv_5Cjl+)~`dN4Pap8F`?9j zSxX0}VfKP=Wd`8XtGHMO?RY_dXlTw@lB}RQ!GqCzvysf8zCW4F zU&~$}GyA9Ws$C}+Y72WIH@)8*ky@eCmU!$L&+(T+znAsXT_WzD)LOI9V;T%9>vZ`C zZ@D73a%l}a$O8HPI=6{_nkz1yHZCID=KqLHn_i%!@w96FMze{!@D!4_4|JBsDAV+V zDIJayIkM5>`Uy{=#x1>1xLo1B?}w)Pp1IzRWzVHgJwM3b;b(!~(%CGnRjY05YcCkV z(*m{V!Cea*SHIcgg!W01*^|MK^NACCfb$6OPFi;*VLIp&6mD>WX&1Z(pn^SFh#su& zwTf3BayLrfRLEe?BGHFHa(WBW_EeO1Hx@@BX&$FD4ci5A2M%Dg-)GKusd!uAspF6A zik$k?MskZ4+Zu&dHM)pih9tmK6*n=J0|?)Yo|(=j!U10u2Dmhn9sff~mbk5>-M$~9 zI}mqx`_aMCF&{F_s;-UVH6i2h?&?C|O$Ua#lI9{rftep*ru9n$C7?X=i_!AF4|+%>M!wIwIWhVouIO;%-r^0}tQEIKW<|fs zM!-&lY8*5mmiCZvENTEcGGI;()7@!%N^$cwb|c?-z0qS0J3$e--YefbU!S==dfhQM zGTb6Pz~6jJ>kLMDSTmlkvv>s->iasfh=twPS^O56GbrrQj0G~MkAv0v0VD&U#}_hj zXofWic3^bZUd2q1>myJ8FR%IA88_e*aO8QzLC)_ z`09jhX7vyLE)6y({Ue{1w-Nz@taJW-8b&qbgL^GB_>zKr(p~Y zD9FY&Z|*nPqka4qnn=J4?f=zX#~n7qk1k8 zUJO+CTuTQL%TYaZWSg_b6mUqRt543?0dt6;OkNXO@L~suEUK~(R0sLP2ksojhpKh1 zEi_g%r<`D1z=eAk&cSMJ?(TiO8wXiHn#Ab&@41kq4eUS4n71r46X5BTxhg0V3=pSkLN`?40IP{ z;h|PDK1`R9;nCfy9HKm;0`==j)+DWZ>DEEkty$s6z!{xhY`U}-2hl=RuHu}M?T?MU zNH^j{B-`zkzo{N5uia_6?}1zLhK)*}f5qD=?0a*t+SNfH0(9tmJ+fU9gGt(3WIb9a z40#1iLC@A`S>^jO%DU#4xBMEY*3OCEaJtgel)v_;HW|+kr`rWQ^RstJYoGv@IKBPr zK5E;tT~MR8Z@+-DNZz$<*P_EWBep`fWa;Q7P343&Q5<9e^sOl+a(JcGsI$GJ-DgZ7 z7tn4#b5R@iHa(%NeNjvm`>Xw3fO}G|VEpoa<_#AY}!QlA7x zDA{x%^I2c;O3XaUlGlliv%|sP^5g2hTW+ z)cL#lgi>xD@x_T-Ju>!TxkV^YvLdI`3h+zdvC=H(J=eXo3(~C}f;Z&BGA~4yiTTr8 zuU-XeN?^WQ*TLT--PLjsdnXG1*$&ZRp@wF54cad9|1#gH7O|$N>qhj3h__tKfp?MV zGQ`W;BxaLSBSyKK3Cbsjo9;vVw=!Ji3rN6JR+)Ij+Ap&Pj$RAQbpaK3fozv_mNd+? zCmn(5QT}Pq36v#TosLk8uXHoyKk@N9z~64ic1sctEwnNDr|$bJ80lgnm)>13|DJ5DJF)YYzxRoq+ zQnrG&>U0D>Psqy&rtOdmuLhfzOu>*~@SDekfWZ zCT(3P3hWx1gkYgn`s>O3t^y-fZQ&UPrW#0b&W_*N`?TgR{`l=1KYB+@jeGal<0W4% zo70-ZwlnZdH_fjVAtW;8%Nk)%m+j{Vl?Y9|6Ev#8YjUxljjvs!+0=)9DgweJkOQK0 zN^MoGxEXCgkACazc7tE?ek;H2r}=!?r?Inci{z1#0)-_5B7ofi*XTlAQ`JPtIV?Yu z!-`*CE!|&jwQTp>@#GY$fsJjmB~d&F@YODfmsmOTW9)CKZ&9buvEIx>x5^HqGHC<& zRwb|b!ng;)qtn=9Jky~AcMka0uwV%(>5)UsadNk+LIPQU!NJ+d70AgC(o}bKHi#jA zXN&VHs8MIXgKM1>-+5Q<{b4^DYj>h=X7t$Ds=85d_qU=S`Qew>v;YWtq3Y09_x83t z=MbER$&af6PP^51O(KGL+{c3w=3kZ7BtqU4>quCDIZZMT&k2X%A{6`QV$+IHw4c7khwAdX^H;YeBK=$R*cCq|P*8_6hN} zH#X~0-s_vLl$#QzG$2t`%-`3beGfyc(%y6m%9{i+;tN*k)6o|UaEQ9j3OI^1AIVVi zSp*3rmPIGli#PyEg_PP-N3_;+^k(CF1MYkSW39t!2 zdVc@lDld6Zfd%oo?|0&}E{fsW_CR&B5%TbPq!Lizcj!a?rq_D4zd(%QUxabp)vx2! zj~m=^9=IkP%uMN8lJ*m%$Nqft%UwS*8?O`bNmDWK>DU)#h@i@>`s~bXZP$)Wh*t|s zRbncYxXhhQ)4ez76UsUI@vD4)zGUj!7BCAo?&6 zTC&jA)rYPykNLFbpWA^SGo1uL=4p>o=eAltoINQseAv|B*dSVl_L67|Y0fT&;*wTO8Y53I|)9FZw|6-bC7u79l$e zZVY(cVY4{u58p6r8_YJR#WB?bVx+y)S92RG?#yF_nVvgXk$nyzu~(RW#pT85!fpf= z54C0N)1i;|_)9UNagfps=ze4m;XXB+y1zk$}hd-(txtO3ddlrn)0i$or$%jH-~3uV}#ng zG)jLcT=iRm2&exf;AVne}tAJ76ZE4koCabJHDj{PA!5$I&-fH9S&Yk>i&!kHY z#{atm_ph~-L4Vg$g4C=1BhMJPrjL<@|9IZqu9s(|IsI|A*dC8RknIs?c3cfVWS&Se zqilr+I~Qi?Pu>L;&l$Nm0738K9)3u7BxL;QyPm#m218hEKg|K4x@Xvr8uKV~3zI$@ zP1=eC#Wu9adRn%b;{5W^UfnEqXTvDHJ)Hn>*mg?mOqbsB{}o7&=(SgH=SccnqJ!0r zDX!^$_FJ$DAu76X8yrzFPR&NNL_TQaPf@H&)mzi8>3>}Sy}puCRdE={J~OUL&!T;U z-qHCDrvQLF>uQ3Bj>kD(UmNP6h)}g3yVbeRxcNK3Pg}Ms*e{6V%A^#;9qyNhm+lgteJn5khs7hTKhq^Pe5ACLFF0rubc*VV`^JYB4+U>-mMQU*mPX|>#rQajFTZ#yAHLqEuEfQ{@no>wKr^x2sJzu)f{612R()?Kda ze5Z>CX>6EypX%MC@pJ$82EfnLIE|FL<9mLwwg&9DeMan*n+`k})u4E==xGulS`Lax z_aQw0Fj*IRHqpnmaG`D1FU_nMc_+oCc1gQ!MunqQ0nEyJI|cBaeCbXLo8hj881J$! z+NKV=^|i!8yJcKMU2BnJkShgL(Ho)5jKndY&UdK;Lqd{b&udL{x_YakTQEvD_WD6x z5yqqGt}T|RIl)q7L6@O6ebU(%e)v)fXd_HD_;~zR+LQ5D`>ur5N7$tj^ zDEW76sndXtmtnty7{izOBS>JN3d|~w1|*I31i*T4KK(S5yGk?r!|3;-TvB&>KOf|j zVMys>3pUie6ItsFPIvBly#&);Xp41k&4>+l7g6h<(<;6?6zNThY~n{}t5O|QH#nv~ z)*+XP5Zr}!7UVo<@R-wmYOP9-DCue)=A4b>cg=F;(DO0Ai6Ixa(?LMCI-%3%>Z59J zi+f1wmu-X&yLF$_AO6a&9k?(m@n40_TI4TOJbTxGoeK|H#yaPr8i%?ky<$Kuk>q4@ z1bp*p0zkDLuo@a$1zWILr4g-T-7Xj^hx$HNTePXE`{Ja8T62$K#Sf7^cQ~H68X48# z3j6*4-V1=z%9TVvNs}r^k%DC==I1t2JaMwU^@0OBO61tnaI3tP2gVPCSnZYJYq!AZ zr1V>Q{^#NMOLT=98gXvjP`Q({P&D(-W1KSET~^`3V(407)HS{^>des3-Y%Mvfg8!F z#g%emoM}!uAxiV@j)ZDUd(X6iMwPT<*Yh*adg;AZ-tcs}f;+=GH4(sMA5&i44}>mZ zJxCOiblc0l3ulAaQJH7@x+xhnjbrN<)pq7oiJ?gx=~Vw8A{3b#Mxy3dUx-9W?QO=e zpqwT~DC=&Dwjv8gV3zl`=Z8`z19fU7Ip<})xTI`)X`Qw|g<3bcK0IWdzLh$$dFKSJ zw>50vE?f6~(72@p|02Egrau*l^O)rvaE4_)Qz5iIA;k3sJ7{vm?LU)$8Frj9!pA(i~sTMYVP80FN*5W&=W@##|=i={tPGebmYl?CHHiYs!%qf*{L)}Zp{Ot zy6pHg=v!iUO7lCC+>@Qy|61KA#6c{8sHf*{ouIek7@#TmK0zr2^dS_KRNAMAVsG{@ zhoeu{epGvq|9yS>-M5dWsO{haOivjhV<&uvh4<);Bo?$a6U9fP2y;9$o4Wx8(8hMC ziww?5MOP`kqHAd@D(c+xyE_u47hVVXb2nDIx?*+llUAymj;-~M2D1>}qAt9HN7?BG%k4M4!@&yfoQ7hh%??hkbsVz0S~?Z;k*-00 zcGo;n>B~md1)bE4nEhw0kb0Q{tm0jCO8ezUBZV0>Pnpjm^u~7;Jy!-tIaBqTJc%!H zqUZ*R7JSG{3}c!-qvDz zykWOhr+b;IZ*f3TMQ$Gnv`2o$r7@tZL_va>h~ncxTxa5&Jml zhOgfhBdKRYbkd=M~hIiu@m4n`I3gFCx?^duV}FG5_|X zOkXu2oejW4&viz9W&grkW!QG;~=0(CloHU6xSAM@a==3^ipjHoF`5$ej$jPmo^ZHDeT8p#}vRyqxhtmg=cU8vO4yrBewzu-}6=n;778b5Cn+0$j6~7jyvh;*McauOoL~Us($1%kXzxQOg(CNY^*6GsS z6D)hD1ikciWI7UvN_p73XE$;=2CHaLrn`Zq;i2%Mj=C!%#R1~@pqq*Et-{N6HW`q` ztdvD*XOSZ;$CeCwwLN1ZT+h3DQ~(MXioi>*9wlVfF!RJt=e3%FmP+^?j#=4W$|J6L z@s+iPGb_mHBpWrKO%5;#Qy!tZu0(X5fhN!%q=VF^LN|X;v!l_O8Ox1lBVB~=q(zf7 zl?2^76|8>VgJ07r6kb8Q8xsE!=j;(;z3{Q-)Ngh!|8e^Lv(kwi=LinZu%PaS#_#!F zKK8*kv!T2o!d1g{-(h^oz|o{jb(pH9MfgZ&M6eSw$REj?b6xCzH|F{Ox+!dgISoY4 zsg5@HmA$MFvSHVXgey=2{%w$vE*1JW7g%TkMiD`hN9q>6SK*jURhMZ68B~pN*dwb? zU?j!2qL=ge7P~#yk~^P@)R0TDKf+UeEzcv&C0kc?=bM5E%f|C<4D9h!ez&@?iWhNAuN;515wJQW6H`9lD75y?D~`PgQw_A7 zI`AC9ajRFKxeun4ntg{4e_Z^F27zwpXSDHt`+LJB6N%89A}gBo1xJ5Kpq@v$u2gB7 zR<*N7Wjm;W_`3TkywczP3~=c-ylpPceH+=xlPdv7yL{_8>ux3aQMD;y};YME!U?gywtIn30o)7FraISd>jIl(G}U*)X*ifU=*sg ztsy>5+xD9jwx=wyYlfEgt2YKaIgsAId-|Wx`|l*w?jL`CTaTG|zE(VP(K!n=*Hw#0 zZM%=c0(QY!=YRN_{UC=JdGvW7@Bw<@;{rJDbjH|V;Kz?2kj1H$>J3sddx7`TeMQd% zR)E7@382yQGNC%FPaZ+l&C7aUb;dVbt5KW}|Bbw<+!y-FGhCawkD@7lzboe1_(w@R zxpF)T?fb_sIC{ZPlxZp17a+Lu`p3xW#~0g0^ht&v`QnV^`O*67C%^xRX%)XM3aI|^ zM@>lRcv!3>X=>`o_?H>=M6`ET@bNT&-{lLMdcBi}CaNw^!HhSflfzbN?%8EQCBdd8 z$)+~;8xq!tpiP9EM|O%VZf{=#E;Yi8lP2(PM{AyMD;jzaGF=W*|3b;ATgHVl8ckIcRuH4_^vu&Mo~aX!_Or$`}YXXwP|wbM&Y z06(IBxdGY{Zcq|;_GJ?J&uH$NX?YEqLf6rfUQZ#F)iDoxH5Wn*xt`>B~fUaUTuywZ5u_;$P1 z6>amDE8JClKYOPhL8+Ff7oB)CE)ftKVSa1iJWUI5e#EmLzo8$@*$AE2A*Lm^fkfkP zDPg<~Yd%#$u=3bTIehEU{iR|)U*R+{W!1TA54wV(CJacyq65ZrLpUf zduLh^pnqpLn?;bKAl^nAEW%T+7vJ`T82pCJvbc(V7JBVzGWEG}{L<@`>t35F<&}q5 z1h_G3utvM5@nwA5r?}s&mczX}sPm;3{YP8Z0SuRPK6NE&Z$+tld)8=j_R(Ft)pFbB z4=;sRWktt-X+d6&BDF59mZphr4h@O6&F4xtKctD-D87Pz))A0u1b!fU`%H{8u{K<^ zW>__t?qgS|3ozEN>_;)m3yoT%SpfR8P9MkRh z#wQd=G$})?N)rNz{SgQ6ql`;zEt;eW;g)yI-d)YTOu$cIbmyBtF1DqR4W+e+%7j8_ z2!dC0d%Ns^qto#EU(qS~pXgKxrTX=wOOu}53cv4PW9U2LY)?;W#2x0|FR*`a1);sq zBFVV7f8-hBO4lo}{v8l)tQD1fb7qjG? z1N2SA9xM<9?^um4x_bSkj&b1Iq3HevNOuvv{gO00i_9|$9vwB=Oz8kf>HH#3$anSz zI&ETg6@_}<@3ravD2P%NTv0dIBk3=B2Os_ABnQ6w)h~beX4+*B^Po=yhD^8mo4gAZ zW5NRu^QVPmtJaKnC`8WW5?E~$N81Zh>^f4OEX0UwHwqpQ(&X@m= zjy6Up^~fyXV)uu@y#7y8bwvgAEYWJv{)ryD)cw?p%i+F0+julnU#)S&n-yM(%K49lyU{3`elrK&sqyOku zaxJ`oRUSfs_#h?AO$WY?NNrY3HBZcBy620=JyQ)!tVG_HSddow|rk*pMU~g}XSRf+E*iyz;tBBH?Ox6kaGY;;i)XNPSe>3<3?4+K3 zKJ@3`N$+2m;J=wqjB7Au(zJsnHKv{AXr*ZA&V3~Ni2LH&d29f=G4ddK$Du!4eGT`% z;-WA7oS&oiCFDz(3Xf=KSV<=irDQh^sa=r^_wRtn2JqiP{v2No+n(KR5Zf5i2z?h- zg+6Jc&0V}LBp`O>Plc-$l1PPxv!H=uKW`$B9ooIzX?clgT26Xc&Km|3^PTT&o%p#r zZoULDIDd;Vd5Y%1DI}NdAKI#rikc0DVl3t)m5t?y6ll9CS>y;%VAuMUFCPo7CvT%6 z;b-^8A~pNRWz*S|-!wjm(aJz?5?zu~L9Bnc*KpnpAiz4_TiN%lsr{%Dv5{O^visPc zno?OOrEAQ&*Wl5wS97DMSnKZ?(ZpDk^xn+1rSfl~AJ$fPPQDcuOfbhC)m-^)IW_(e zmr&JrDPBl$G0;T9$5~({)F6-r3EkO}8r`a@%I6Ywz2RCv_3HlBXYtPt>Nk?#6Zt0m zW@ORMW@boFzQqfQ1NWtBHg9)=i(vkkw)hNdUW!S1-S?kwYf!K=TYxu>p_wl2x33_1 zG!~qyYb=j|>*X4-^KRr242R*9G zsn&i&_cxgoJpwT$&P<|0Ide}zlA8P%5qPc3d0IK|+*P9oZ@rE*CE6ljielPou)rg# zU(J|<Ix14^EnQwqGaRdW#t{u5q^Ch>4EPz3-znNQ zPVlz0ppzy<=o0BYEvCNiHpXh!*=Aqk7sYsAvd?OBh`x~-*GUY&L1k)Xc}LBauCl1mlG9`GIy~Q?dru^$?@znj(pz|zF_0Ol+Mf_% z7UfbH55D^2WEMHw&9M_aJYt+5n?yb|vf?Q{{Q2nQ`?ZquIzbw5(r*7ZHse6F=TfIQ z>GMcZ#Ff_^i1#YEcVXa6++B=G49#Owo0PfKyCE~XSaGX4d$tJ$+E**TY}h|p(o@I; zkEC19=h%I{xTw7e?$i`U6trF`DT7){tBuvNK41D=2Kv3|F>bxYn#WP~&z&45FPR8v*FN4oF)4a&Is{cqhyoKM67!vq)E&y&+8<+7Iuo^hCS zLFam$G=gTMuhqemT-slL&cUU+$Tmz$FrURsp^pr-C#_12QU48}|9$BI@^ErxPObed zooJEE_&ZB$twocUenPI2t^=%wr3ewFL&l@bS?!f^0}Jfs4Y}uV>U`ZT>X?|=hcBT! zD{^3o)Y{N1n{omCp{8@Roj&4BVI8@U&g6R*bhO6wM$P&M>p~VdyF%oJ_R|y$Cr%)_!-@pl|tQmPe1X+;>*UrHDBQPMOu?~KMcm!!(7D&N51UI6Y5TV zMIU{$Q{n7y9AtLYTw1S@b%l9IJ?QtlX2Lb(sVWhnOV*)T-O5~7*&fk{$HzFP%lhv@ zgFY{$herjCzo9-w{KLKUcit@BvKeR)JN^5nOZ(RzfJXknnPhTUod$U0cuOS$!JBli zP$VpQS$S4t62;o+ESrsJU(LVxFHZCCpwtsjbLLKT#J@md|ik*!FU=;QIpMkgx$E5JFqIm!lW7ws=aMgSq7p0sNe zcMI?=_iUs{nt?43az6|qi2Y!H`y9CK_v%ul^ z)iE7~XR#M|zTOW^ZXE3<+GHYmFqg?D(?&<2jWgLW3pc33!NU{6msceu7G#=l*6f?l zh`=MFXt)Dv+}+N4%UccecDuIC<&|!3aviZh#iW@(szi^#e7Gdvd=Ej<22cJmVYLuT zwhHBFuLx&6G${&TG z?&(z4CFowpOmc1rD$ek4Z=G=Q!j%uda#65fu;u8vT?mb!ApU-KqT%wSEs|eF_Z)RT z{9cy0%hS$q(+5WOS^V;^O-XPzV9EUYlTTr1KXv>GB0mZdGu>?Vk6N4?M$p>*)&f*l&363=~*EdiDRtBpcSV9;X@hL$93km_y4A{(U$CyEnjM|&-kwY|I% zd5sI@vk^2J5U>A#VkLQjQ|4O&md-g!&q5CEh<8?Np3jX)e!14!B%o$2^Woy%(ZEqe za+lP+*xg=l(=lfV?E^OUE7RFA|fxPz%nH%KQgk%}xAglTO$H|+{lFANW&SDzXZ**pR)C8@rOu@gtG z3x1EV(bnn=8K(ADwoksUoES-3nNq(JnSSLD!ig&v_}x+1->3Qv?;zxq`O)B=2MwHh z)LXBd%*@Wiq31KEil+Auuk?;TgN2!nw%oj~LxfCUd#RJZqyO&Me;ZzRMpH!|rIFnLcIE%SRkpuJJk|1=JT$X*~Ek)KJBpumgJ4o@@#>f6VMYl_- zh`kfFYkQw{MS6&%l_Zg~iQvqu&(ud&M`J$6ytLy_p0o7gur=vnyzs1sNhpIpt0_H- zBCJJ}j64b4HNW4DfAXo?2VoGkylL_q@MD;sT5(DIjm0B$t1&5M*Y9a<9_&g#O&@@lBwb;I}^7C8BeV zL`g9))}wGh*PU--bepSDs4j7-QCkb$lom~HUDIXAPu%5==n@8AhRgW2A>hC@c>s6P zc4!vwACK-sVKnnPG{3;6uL(S9uZMVfgOgyt9v*!5z@BEHdEgB!aW^(-d(1~5O2P!L z-F(`30bm_=a=Ci7JIDZ6+nuiR(c*w6&Y+c_`;c7VNY!X5dte8{3zX1KIH?! zZ{Ocr5Uw5G`g}fnc?$6};)9Q4%{t<_)6DGpjF??^#m&6{Md#s4VMI{zUI**Ya=yg)eb5qht~=E1qiytI13*Ex5cK9wNXn zoJR8Kgz)@4#o)W0a!A=`|CwsmVJ$hhE3;&OHHJIgQ5`WrJ@LVU^H;BM@PK5-%?q7W zPCZFpT#_LaGl)rx7iS~vw6%;wqzMB9A^0-l(^fXG!&h6=8^OO<=T}$Vn5152)1{w*EP$cK4v+q z4Uf%ob4xu&EQclUwhe1{FqAI(?hBUSW3Pk@&|OEun9sYY3Jb* zByg;4)6y@ z_mtZbQG3T2XN63_ej@*@@D_u6CnK;Zv|saio?LRxrBl&d?fi)l3I@;y^ydIf$(uX^ znCRrVub=${c`l6h;-xm)tlZ8%x8zKzOj)NJ9%FFoJO0{#>I{w0sq zlAKv^;ay>eoA?8P%&lh7O+tTX8h5%KYO9#Az`wptsPUvQD#`< zWH&Q2**9BHDp!^0F)p}+x8X4H4sg+nx)1`T8Z_a2B!t9{P+9U4x+Cetp-NWg;3QAM9A{B*(RfH zBl|r_9`-IZ(c5T~cBsL)W?zbp_u#TJh&r}#hUIWD3PTZpv!C5;wACrkGsY=uFB_|B zPF_NQ{E9qHR2yJhhR52Tcl<$P4?aTJtV)g-YfSEqlDlu2n&tkl?)dLPzIrj|WmBwW z9{b%&7_3ab#-%C1A}uNL)A;3saSL|>P6}*NaZYkAvR`A}isi?xuNYjeJoXMP)6u3B z>6v*vuFd8Mo$Kt)(9zM>iEPPF=&Omr1zA1Sf#pHO!{?S)c1_NU+{>@It=$n`(k1)c zpATZy&mPYr2@TP?$vbyDgN-XSMajc!8mkp4cC`GlV#Dm^Sq|-tIKx41kTX!crr@c? zwNPut2YKKQF&^Xflp;W$HtdlUE5LtGPph9~vgWpx$10Zo{(es}J}u2VnNa9|-9oUi z@YJ!?UD*wa$u1htwI&YX8B5(=h>6N6<^qZ56fWn|gT)S9MkMH%9|6l2!&V!k>~N|LKwa`>8tkI3pK;|8sitn|T7|N-F-b zEZ?EIl#8>v5zu$$Ia9m+87d91ipVH4WKv?sk8jUaTD-o|M4N)+2iG7bhWtB-u>6;i z^5PA_=}(nQFd88MImFsFI@8s0rY<(0;SS(N2-tVXkh`XEl5725L+8O)k^H1?e^|&r|QwqzLg> z!jhJV=H;OUWgJp>+)Kph{}2GZ(;Y6H$DxfCRAF6@#-51gVcyKhbo+GnA}P!x_P8*! z$z-af%kj3u$@eCZ=$B1MhjGP9bH!@x;Z`d9^UJ2hJLSOqMxi0%Y;fZM#WVu=Cd-?3 zn|vlguVhA6>m1mg$IpuM4744e(63;|so$a2jLN|ee!4t<=^F;;I6xu(3OuP}hn+UD zX~)z&0%M4R*?f@I`4bHXJpInJ|E$|Ezo2v?cN!V+toNBVSWgwVeJB8rvL#y;Ll^v{ zc8M6&}-ij(YuhC*l1$z-yF3WmSuQU1Wx{9y9} ztq=_g5+fXvB%zi#s)Bk=TAKa;_f8gtIKy`QL9pfgizMEPN$kr=o~grQ&yLZ6Nq_>P zEN|g_*YVfGR+Emhv4z^N&n~<7#$BEJV{}4gbyY!yJoPYe{*S4!X3Ld&x1hamP1_&+ z@3R4Da-0(r$;k>0UI|Ds2}x+VT^AN=#s% zrT0-#NI;uIsc2MV>WT~@8f>w1f8kl@rTnS?zc8%m>2dzo>W2##;cI{FJJTtFJF)(}-1G3x(+ zlznwn98Q*QkRZV&IBDE1KyY^_xI=JAaCdiY+$A^!cXw$#I0Oss*0{gU?EGeDciz6Y z^GBc4r%#_-Rkx~c)s^q7I0+X14)y=G`Ty2hwTTZ|_^WET@A+}(wuhhYWO13BOqKcf z?baKNe-1eRy=y#ykT8w)=S#%P8!J%GGK`2%wj`)Y8U}TMx9~nea_7ocAW0U5<>Tuy zdrj@R&w~so6w39n)tPfRv+{~JLqTz<&D%46UiZIE=syh7f71Zu6jy;nRy*Zo?}FTs zA!}a3<@ipD4CwBAyku^=e;-)8N4}WFT#ty|ySJE~vJ=R&rkMQcF?kqiAjRa%1v1u` z-3e`X&v_UJf!taj*^~F~E>~YN{9e}P5uB3C72ei6ZBM1Kd(bJ+X91NOIx?szqQjXia$;nQYZ~6Xs(17tl6w;W897>d!JYI-TWE z=#flA3=rRL?AS*(%%amg0FlqJE2B*EI8!N{t#3U7c(aAZTjqbH#y)F8-H;`#%N!QzHIvKVHE>JhEH3!}LAtXrZzK zXN~Sd8U51!lgjGhS^lK<1Zl;-f4SrS_s;#3Oh}v(@!pBF^neChn&hX-w!H>en$pHK z7$uN3sl9jeN5+3~O8y^$Af5X;^JmOz{og}AMC^|%X_mC^x4ko$7>I4O{$7x8=^63) zR7Rk*U!KFNc4Kq7v+iPZX6kWG_psA^ zp+`wP7DW+4RiwF&&H91*Y!isEPN*qZ<0VO-VOi1q_n&a)KhM-irMZ8a#r$)?{;kK{ zu|gAA;AnHyk@gz4ZI~+Bl#?Gx3q{d4gdEDuXJ?wsRZOc4&Vvn_$tPZ-$>Sb!aF5@A z>55NyQ_!b$(qzOgX>z>TJoVJkPrI>0W6YM^9((jO+4%u6mV?`-yr$#yb=y7NhtFw& zRii8T%%0a743A$MLG_`b4W$WE`NuSbWRn1_C9MU9x`6OE#eudJiTKQC*PBTr5WbUP zq;6KAnpFDg9=1xA(d-e*>`~2^4uTy{Uj3w`N%&tb>x33LL%$du`5(qcTG&i%Rv9*I*pd!U$3`qFuEU3or6F5v}=VV z`7m^ST+6Z2i$}o~t6{UB#urRguQcGC zB~I?j5Y$1qz<|h>PaZ|tq+p_1m;}# zRnC4qmQ8yRss>|}E_JErt5Ti4IAqBY3zDICY&|^s6KPT##+3@B8vO@ppceZ<`0S=lJ3 z*N4Wj(2*;vZ@@WxrTxSWRmIgUshmfdSrqO8f!)9|If8RNN!tGmaeYC5fNz;!5%SOc zg4+n8Q?j2ypOWvl%p$YW5O@8GO`KNE@=Iw<{W~+^7%Fv$uRy6pvrZ{{U$bsgGb!vP`Jyp)F%&X>kzul+*vTAu zUeV6)lSLV@ceVV|)EK6x=z*%)2&3}rYZ>A{_T_50kOCGs&Jo`9%S9B&U6M^{(xJKZ z2UU7}`(y+db3?@s4WlT(|GXUY0>bnKi1Kd|mn&++5DNsbz}#RF6gcBwg;_*_2Bu z#3}O&z01@NX0gabypxF@;9eB7t=@qd&+VP@MzGJrum;1Yc>EooufQ=Aw%irZ90J;* zdBLru$JF}rnjHaGyX>fJ%cIwc+6@Q0T~_WGnheE-Q8xRNUv9fVWKTCLFX{mdaCB{p zT`EK1WYC^LN*Z|;s2rE$x5D=G>up(Lh~4AxEM*!CRYJF-;sBzFyN-dx1R&XSXbcXH zvi5Yf)WQALOOznF-CgFzvXXLq67W(Ql_pZ2H!KC{=77A=S{zi-p+NTV}A7cfl)sK@JV8k@J06<`CWo;|Ketc+NsElYFa8 zTTXa}`%e!Iq;d%*zV#&X4tK?`bz_5d`N%;}RRKX>DvTVhPZgw$^b6K0aT;Z&ailmY zpgn_%WRhoqyaKZWwEzBNA;RPYW#7*E@lLNdy>0eG;26rm?;+nCPF7R;oQAWJlvnmo zUN$jz7*TbeL*txyNrqO*#usPGCqAWMjuZ+RLGRoLV;J7W?NxA`gV&Z<+lBRT)y|K( zEqV3Q*&L*{^`Ty-9OjxJFD#B$FN5U1Vp!*_819v0aYFfrQ$2H=E9yOTDNEGAUz|h+ zl!BSPofvMC!Qb=#Z1mvk$V|t}oEgMDyPv+zuhh3AUY)6%F?Y89xlI!rT)uPMy48yN z*C)Y#uRbvqu68H|r8c;`&XGni=atBgD+ZLQTO33wZ9ZIu)(W!w`YHp5(Y zx7p6u}&TJf&{|@8OJ$k|GFp&M+)^V7dM;$}*0 z%DxgVe1hu9+DvQxv83CCin2?GR6dcI4sA1gx-~Kl0nbJ~Yv_X?bt%fQ z&VaC1+^V?IoDPZ*&dOJ*kq||oabbzZr$gtdZ_B2D059L($bI%VauSb3sMa}bwn@W_ zg+nF)c33$i3j_;TCBY!EXA?k^$zc-fRFnfQ-y5*k5h(+QUhPPcV`(4 zwO5=D*K?BcsZy)@Esq=KroA!i!|-h0_~|A`aaLTRdUP8qh)2^4j(E^Eq-9cTj|=Mr z0R*EB;2T9Bno-5(fnS$k6)Ex$VLGa%eUdX=UhD%tEcfosXk!tsf(czYhqnNz(r0$5 zldq+IPV?(c%vn3s3#C8Md_7WBiG)@KBv<()CX;^jOYO&;gJy3IZR-RK>SWCXTWIJ? ziFD53wBD8)SgD+I(lifP0V@pxoKEI7go=MoNgW7vGd5!Te|fBsto|eva2=}16dOPB zNw7^e8&>j_$>mY;y1(OFwg1*IU7W$|MiZ+*qH1Vj_qveGZ!5~MX9I1cb$Ywimz;nS zLB9q9;IE#Vq?mVar<;SKx$>(q%!fU5j8(!3g&Od>eCg-(yKrnp6+N%bwYcp2hc5|zKp`d zkY|@n?X_)ux*qdzLS!B}O)}6mvJNV5-CC;pthJlBncjK289zjb)v;M9jvqzP<(|l~ zn)xHVEDN{2BV|?}B>MC-(BYIdjXU$4d;Bc7`?A59#0j<7u{p*wJOpG7d&~oSq)FN( zaGj~^ta*tMzdq zM=+_2kaOU1g081&OK`x8H=SYnVQW4VXtBh@eZ8w{$y1}0ooV})dEQXv#Bm_{r^Tlc z1Xih~42QC>1;;~8xri94fs(n=<_mfDwS3*nL*;c}W^NSEPloxG-Y>uYXxHb^jd|Uq zX3fZOj9g0;d~&hr9Y3+GsHH0U%cXI`g65V3Jq9_el6Ag#OEn6b=2C3vdK{?X3s~}> z3gYg5A z3IH$JY)87S=&#OF`hqSwDMlT~sgLcMAqQ&Yfk84|msJ!CZ)7{`qNLBMoxTrfT)iJ8NV`tep!&_>y7s}{!mLzxjaOw}1=Q(Q zbi7(ij-=cs-$U*3EyC{b(H79#KVh|g(u8|zqc%dg zHwim8>4lwApz>lh46qzDaQ)=t<-zR;;HG~sDipJozdzy+wlJ_MS)#dBERVOwW19h3 z0He1ew(Gcyx4T#JEau4fZjFscpqZn*PsRD|?GHc>wCNpc^I18FAi$A-dTC%@9Gj6WGYE^{ir^YKNj-MGwR3f z$d2+j#{G7$P?~_NNPShdV1OGCFDx3jf3KYZz@^@ZV;#?|Fu=&po|Sa3IFS;!A1X_v zv}UK?SgB1p%72M&nzU^4UJcGewCcOr4~9iR+yNHcaNJU4^fq9ws8Ul8arp5Z!wE5+ zC1OM{LB)Uwn75A}1B#;+HEDEnmd$$!PGlnF62r;$h+I$`F4|F39=yN>0tJ0d1kSlg zjuiCbM#yxssrW*D(7jQEQ$B{J>YD?gPx+NEY$~jnZ}(U}{TLkXmnSc{8G*`S=?_>* z1n}A5Y|F`St?AS>#7bR1kuCz&$f&#A6pajo{+A%S@ByA~o#$c;!buBf+!ppZ` zFPkcKC}*;UF|06CyS{WqWeGrOo0yGKRlZZce$Mz2z3pwawzgWh#adXROO^c~vikrd zb<8Fzw<@7qdo~qw2;lBQI$Ivlyef|`^S|T$j1wro{e*EhnVjcbyscVzn+ zB-#H~1GJgn1u0UX4D_4WfCLBR1gBV9BD>wwV$Zm2@)?$#WY)F_B5oNBv&MNuHk9x+ z#VRD%I__hvfa&Q-tZwJ%`1)5uG*ic&X?Gt4A0Ahguq@^8<~=b9l_ZggX_2a9j2H!l z+kVAW)Pm9k^R-HxzZLR*^(Ys188={PVAZ9P%b|(OUpDDcZ=0w+?ba@3xZK+<$~}cA z6dqbNuz`L5^eIlZ#4)adt?P2o%VxJIsupqG)$zi6;d6)XX;pRl_AwD)J#6bd8ZRqj z5&4PTK9>8g`H?!keT{#aVMmkiD_4xuNSZ>@Q>VLhbE8B?;tSnhfe!rGZfjxCwJ=>Y^jI`GO3fekl9B2NNVOp^WvnDzE}Z<{Lr^rx+MVk9@+83iy;%>aHuqu z=r2#Yf)44PP~s^&!)$vS;rGuWe$015;OLQ;H$BLoUgpQ8!##ueuu}l*YIGSx3M{7K zq}zr1>bq#THhMJK?GpN)y8_(J({(kS=$QMqd>9(>Hl5dp=*$DBzfRFPP@e8Ot6679 zkBmb`N1ZEAt^0I?=$_(=dLm3wj>i>!uAr={@o*?r13n(Zy|GexGu-)>3O$CuYxWB# z{?{io(dgGX-+M{)J=OQI`A(#u%MFM9G*gM0?uS^y?O@Hp^$?_I@s^<&pvGq#a#BTF zkAk*a*J!Y+P{ixXcXx1p+T-@`mN2#^PW3PIbW#TO^T(X?A<{S$X@*RM&8CC>;41Bb@cE6jhWgNm})Ako`f6qxoiY zbJO~g4+BRaGB-MjC~V{-Oii;vxZcXq2$AWB$QVkwi9yVn9+DJb$SoxGurw)8MBXPE z1r)Zi=0cyhev|w4bRc6lEOUi1!$OiEXcTvmh6N$$)(d@2oddmWrzvT4ha4&D@HEev zT39FwrC|Q0aC9^t+ejdpx6(II70^PmHCiLw{X=6clng;q_9ge}tdlm0vaitQm5$Z` zm5glw++g5wJ2FJV_PN4|}%+TfuS!$3d1@)k@*OpmRyk^-5v6?VN4 zPDm}!{m$nzJu*q*l8|p07Aic>AP(4eSNW$w>;(g~FB{{X0V}CxK$R{7hR3rZvm_Tp zcbj(^vGRsq-?~-%+m0kZnlt{z1z@sCPX5s+BxGV^b3} zx2HawvK^(JMYguWtC>ja%kX2hym@$rKjBKX(os1l$I{tF|M*RakG&i9!fxZ`)?~je(5Tm%1cG^t&nU%!v)?RbggZ zUf7lPD3ki-RQw|j{#UWYYNoQ1#60>>4vmIEiOgCnG2KwRO(YGG$fef8g_hmiJF%mp zv3wNbAJ@ZEd5s;*(%ubi80^mb3_iv~su?Y(dbYLxM>^5;w5h0wb(sqI>GP_> z{Y79PrA4gQMkj&0H5kUhV~8*9i&G!3@**=<;il`nRot*B^Y9OH{yDIc^o@{k9~0sC zT5kl^Mnb_K@xJGh^!zAq6bQ1PUVBPr+sRi6kB$m|0w)%U-h8@SA^QEKs~*QzW$bWW z+ZLT2%h@zyOlqRTF2t!j>wCrX>%ep|vPt=s2}u$(l(PCD`tU3+)nyCk+xcBWz;nY8 zm+>^pj=p$AMIv+cmM%E@#ys0w!sqGSdmiJN-uI3owOWuOr~Sg%wg&gvi0)wkz7TBR z7s^WT3kR9hpjC@N;A-|EKZ^>5G0Aw~s1V8wJ7AgfX(d2|W^!7XHF`(?wPrSNrTjH) zTH_0IC&taD_c<}?D{9z78dP3(X%_DlHpn9!6CBfBDaaxJaySHlW9z@xakxHYBaXjd zif#sdZyFAghgccDq-v&^Y<^fi2kVmMmJ{wav@$nZ_0VN^$bB^Usx3(RtW}6Br%IpF znEvj58z4^kp4EK9k+q{7L`aVUgzW?zj#l$R2s2CAx3v~)0Znf(>q6v8Qr;c2Rt*j< zOG_PTTS^}9XFGH;2q?k!o~kMbW&O~&>SHl-)h$-`O_4YbB|BVzMN9{%-o3xB{F;%` zrpD--fjswJ5~E}ZL3*w1yQZq!B>FmsqvVIy0k|qDC3&PZeyM}q8w!mYsoKMfn%d#= zm)5WEr3W^-SzRZb@OgU$E@PSDm73mTA~sMLjwMXn^0XQFu-FJYiH4DLjdElK6(P0x z7nZuvYGzS+XN;iWZEa$JIQKJN*|2{}Z^Z3EgCUvQ zaZd`LgEJk5P#aEUq__m^m#M3<+mS*dZg2sk_1PL1Bl3`6q!X5ae(jQ7T?pFy9mP*Y z?215~$dS*qRDv4%-}y#K8zwc1o0RC69^hwcetjwr@>ZpVhh`_L@~cO8J%^2zpI+hP z+|ONT2@;BT7Glg4F%6Fyn!IA;H!YTk z_Y`f&;QQQ{6qh0ug&Eb84^UP)aw- z96~e)Bm0&^!1lHQcksbDhUcy0?42zYRtfTY;PN>?7s9)j++f7@06w~Pep)pwNtu(= zx2#k9A+GBsK55MYwI|zD<>Us*4dPf8U++=t4^sQ@w|4snze9iHSvIvBryG-PLEWC@ zT=5F(A!3q8K~es~F($H=t95qqEaz|+p1B&l*6LIlVrrF8NZ4#wzi* zCV8)GcnW_Aj#va08nZg^ITdX!B8s@rTa=|WHU5@u`jXIHhejY+oV*$o=<%#&5Bmdg z+b5S`by4y5l#?j**$sOw<;Erl=sN7z7@a&Ygxm?-G(e>FkpTtbBnKm%C5%f{2-wKF zjwBswNu!TaIcWPF`WkZ*Sari+Amu+x$jE^bqI_XZ_zIR17;FnJ@-kocRk&|@gzKxU zMR9zKBqDzxaYwWf-}PaqhV{7-h%E$9d2XI zz8_<4SKJyTONNS@tdA;!LbiT7yv_@b3Qke&cfANsNjX z=)*z>2h_az6ch(~tEHoW=uY!1@ZBt>dhr(#qI9OLq;$ov1rPf^+(P3RYc+@&YP&HN zw>YjspqyW)vdf(Mzsk(!|_t+;om@j~P#a|68`m49bWWT9OPPsTqDYqOc#b|^H7Dvp}2SUk^yxL^oP=Nd8O$;aU zA<@KyT3us;MbEM*q=GambCELin|KjZ8sVpPQ8uAlMO_JMX)mLSsQjNWsKwcf;4(Pq<={dmb)#u#~TwDGyE_5I?1G&poi%Z(9erP!-}WF#Af8 z?gx$gtK=%LB^eQOOsb1Cq*axcd5f$m&J9shO?GqATqmlk#{erJo6QJrch-l;qwuew z5CogqA`1$Q9~IbW{|;wTSw&+`&?;ugMXW;eGGUF&u++^{-rV4VyToWJ#FsN)DESQ4abc*mCRw4=c z-?Bsfv|jqo<9;%fJe%U@@lY~;Rid+41Ov$VlHFP0TjLcL4X22VWy#k8v*WpjObH@Y%#=hAIUVV1N}!VEcB?DZC?wjE!r%26$@gCGsUl?@?K!F4 z>4y70#dhIQOp&C=>#kRn`%~pM5X+#>K`}>=!6P# z2*@MbgA*-odLqd}NJ(w2>jBTu@vwoqZ>lmee)tHu6tuFW&UNFsytOjSh(mk7w*0+! znYeU8FbWvjDzaE;V3&t^HtQ@918l^DN8GM^|5>tGZw`ZJ%<NqnyGRu87n+$3YH{!R$;C#Cks8HC_deFq zQ9XPqH{N$PCOQ@5PbhPyc#Cj9plPCef<`ilE!O^UCuE|($zRuCRNh@qO%KJ!uo$-c z_MrJ?oYf!s>+i4_?$ms{nTEQya#(YQ7?j%ghZiY1*6ZvNB$gdx9I)}JeX6<}R5GbH zC5u$l#b`qshwqnmtzF7IZo}m8*z_Yr;D8ewV&zx$6C1VGngz;h`R3r$_=sgS0@DSau+(tY58jsufEiUyuL&&O|$0{+7|kg z{d!&Y@+M3}?XRUQw36bcn@tRwLmadCu_{j%#ij`n))xd00z3T`Y!=GABZeNhsc-*L zr{ixWM&gQi?`M5eLhK?HzlYX@)E;AZGi=7OE?3%8EBlM{;=-!znRRplS? z5ZV&3eis+zAK_&qh!A3WcPgOUk?ifJrSI8mp8X$%#W*rfmrxJgk-+2iDCne0Zl+KJ zk1-Y5^xU9sN96PJjQj8PryhCoB*pfAnm7yNEY@wucW>R_rq;<5_Yn1re@T7wI|M-n z)a3}b?d9h#SO!X;t z#yrdG%@bZH`D-;EnowGFwTX|Xu^$ZnwNY!dXk*Ex%+w`VWAK$gLT2Zx+YGaX1nFlnTB4PB>Vfr| zB!)#89#5k(4p9rq(Nj=9khm0&!ZAjq_X2EGN)Ypr;YUH`cV)P?3l+ay`BYe8k`vv&b{_bVr#%dH@`wzws@`iLd&4?w z(xb>HK`{^%eEli4l=y~R2O7d=2dn$49p$6{et)k)XSC2n9O~i7rqmnqG-rQ~Io~HvnNlQia6!~~8vc^LB>qSwR)Z1lFcksGcv5%pL!v|1XfYZYR?9VG(Vs*w z_|Uoi<@q{`P*<1%6g$gQR7^Pr${iyTf9yF+Q#kZWl*Fyk&~GGs^V!V|NrO~5x84X%Z|_dEac@_H5lgkOdJfD7MONxYTc#I23RfCUB7pes&Q}A*9`#!lbC5azeV59^d{x z9oDQll{^*857*mnozRIkJYIaGeVe8ukhoC9&k!6Z>tMIDQWC^BSE~4J#n38?ZF?ep zlw>rt=Y{iR_AG8%SxWG1g)$c9Qra$ldx-fgkkC2~EtJt=upV;>N@c^TA0sZXWF1+3 zh-h@%O`bN);X#a~{WUe}gO`BJA};geX90K9`eUJ1xPVPYhaI-b$`)BNC{h4-9Skpb z1^ZJv+R=6)^=Sedd`?15_SI`A(m3l$Cwgw)LD7A$bz_*4;F+B+Rf>)Q773$mw6I={ z@s}k53zXEGFPOn4DM|Nz(O+K}Evn3?^k3|i*60WkkF7lU?CfKa8M3Jht0fIk+D>C& zU-qx1l^;evE+Jw1ur|s7fy{8F`bf&FL^O|lz!T3MwU${h<_NxT?n%t+;j|^NniTEistYLrDEMp(R=xOA zg?e4PvEj??>*^Bn)MV>=ryNOR#{POo27k)?ZYJB;)5o9Tm#zzkent?4in<91c%lhd zhGsYg_-U)env@3@xDFpqn$8A1LlL%}1gffhlDg{DO5li(YZed)aZ9*H$LbYNVKpHK8$5G!oU&Ud09UqT^KZ36g)kjch2sC!Sys z=sW2a{PYG-D5RVpaxt=zD$x8Lv&WL!mxeM}vrZtNdW$yHzCLlexyMZtG_g@lJ4-$W zNe3BK&Hirj_j_yi=a`?+KGqJ>S;SJB-^L|ej7GGIDi*-l=wby1?v zB2{FZ6(adoZbvOCcHrUClw0`|02Ay2f22JDPw1M~fE}_N(Q@uI7Hnx_Y=RGqEfZln zHE+%8Q++%epSEeT$OG*eDG)*PN`yj{keJyzR<(OVWgv$;5JD^viJc`QS+ zWK=;I=#&TeEbNbzk@}yBAgw?lR4gzwn6PZ0x_zbbBhH7EdCm_Bk!O9VcPfcFsDh2% zgO7FAX?EedUIuV6UUYo%_Igy#B*6e!+GB*ZMt@FF(mbwE5f535HNqD3@Tt1wE?nSj z7nf(2IGs^UKu8M^;1rokio=U?7w4R=69_V#-Twl1FF(_wh<7*<0mye2#5T zkEy4tBV&`NQD^0LyW)2KZ$Vxxt^w8Q?C;Ni8L!;^W`}~5&W;7Mrt`iZ>k%&tHR7@x9vldo??y|D<@THBQML6!_qKvy@^gM*9h`KlyRKLPF*A z+A}51AxHle$anMZ@b!te)_oV@{ya^2$6gIWm8T3!c&UAnlo>MTMe9L2D## z%or^^0P&>3VCoS`<1UUf=F^!#wL*6`j7>QeO4H4sCfBmmE@1>vZHsBOH;SDyTP5bV zsgEjMR_rI>sS{wj=OI5X$R~=FP%!1x5qu;L$G_9eY<@=wH#oABKWcix!Agt>5~EHt z1Oq6MKA%%n{g#*4Fp(S3)(VLJL_ELVf9t!lsdp}_By_HdLdS8_KwbHZF_IeDfA%Ru zxdp|Uu10y9|5^7+wp?R1@=ZYjoG`(MG=@bFu_83v__A7V37+o{4hf_nLIsAAy69nrr`X0mRk3O)Se~k-MF&qm;;<+9?fmnj88E`C5z3=bsT>^1J1O2)zpVpbbU`6TOxiQiFhKBS~n&f!fwOi;ZOx zbnh4H^uC7hjfA@f`JdfC2iF~c{>cGhUPos-oVlUSe2(T`xvJ(Sl6pW|L?`O{Hqs}# zX9o!mmwwMnlE(K>b+=$~5n}s3GmUZ8xRB~A@-20>gy4v@;F+eON3uTzFH9}>V_ySmiW9QoC{o!@7doV`0aJ@y6B8! zkrD6(E~nQTbuMon4%7KBh?oGZLpsphG1K$anbzlU$6^#)^xaE8;Y^fjjhg+3_a9?J z>~R&w+uG2w$#DhaHFA8F<^eEB1Y9z>HgPCb{6`5LK(n=rm8ghFs4{qyV$+~vjvERFywl@seQ|*J_T0>pHE#n*(Ws~w--nw|(W_t9CRsyqQr`MD9 ziu8xV_&xljj-WL#B5pR8*nvGWNnTB@_CbT%n2(z27h{*^M{x*}m;f1R?(mpUL{e4?{Sr);+zXfch<^EJ;(?-r@ltzQGM zem)FU%J2g@h>-Hztv^?P5!MIXH60SP@g9WqyXH*3L*)#bHd5M;lypkp0z3p6!D~)G z;CS~5I(#w-<%`1bd!=Uu2_^|g$J;)>6!j|$<0|;UX0`SL3pt*)Y6-a%5P$C%=WDbC z3Sk}VtB)XK{$8v3a%8WC)ONHn^j6FNwx_Z9R=Tf|bs4@7Mh5bhU_sf}nEuWH31K>m ztnw1K{@7(mRlX`C6x7}zT*%VQH`CkJx4oo?*^hWatcTlE76845HVGU>#Ilb!NSlM1 zXPE8jG&vl{xR39oD_%HOcZIw%Y-WP7YP}{@?~`tQ3XbwthfD_&Ykc?v#eW`U3X>#p zBSXR)M%81BjY0y}x*4fJbD^xhH148_jYWKiyL~&t0Gx8^st-jllj$y4%puqg!7({J=NSg+wliXXJ-gaZ!;X*xs7JIck^~I zRi~M?m~30cF(z*>-V&NJ?siO3zT;bWVNNHPg-+=E{WNc2jN$HaB5{mufSDyJ(D^k7 z@`fRfin>M^m6a~>NB`xizS!c6tfq0VJ^L75ct28&5x0Q*d9;^&i!eCWH2Qk<^OaK-o| z*s?}}Z5|DQ-o?|V6zrP$*6gzl)~5MERk%!@*2JV;w_Zr956c3Yi{TCumxw>4;IC~t znWi0IjvcKQ?!ery)#X__c`V^8_*MK}G1Ja37kehtQL*{AjOcy?79ryY0W?b4Oscu< z$d;E{N?awm#17rb{%C{3AA{etph8QyBUy8ceBy!q+W z3IlZ`dvhVYzqJ8BC>-gYRgXm_EGJgLeEA0X`|g-sEw;AIWJ@a6ZZVrq>CZw-Gdl;$;_vZ-r3rJrFt;h7)^nOHT>KLbhon{L~tcsi1=(Sf`&=twu;+EUnqh|b* zY^=<4R)SR)>-Kc1=Y^N2l9Ou8uwq<@pDMLo2M?S_EY7FnSl{Ri1t)$5fXbjo>8qhF9bzi%p*OsHYJ^ zfJoz#DCueGgx$msiB9Qep9D*9r+`)qhaFiBl@W2}bgJqQp<;n(QVoz7$QsHcglx&b zAi7(Ut1U0@Oh3K%w-V~c0H1sf8V-rDz<=+rFAQPR>VR%DOK1DT5@?>TOrM8ff(I=? zTIINc{EdL|@7x*P)_*XV9c#0D1;{r`rB{4$;}81Gc{n84{xM-&pcekmTCY)7aHOwR zArSFN9sGjuVtFj3t!-Cs3fK$52cY#}S*nUJnjI6LVRLP)535594Y!7r*?4vN?Lv}u zBMT8sV4-nQ<&r{kuCi-oX@O0nDoyxY0kdjbxa7r zmYl}5UM0znbvDjmnnOc>4h|#5{`QUxYoyQ=f@g>E$3f18o{LQ*jxH=5NjadtgfSaLf_8YrwV|2JToBv^lF0kO1&M}bKQ6l94rN@){Ct4i%+FX1bCdYJdc zhZ9%R_(MsGlm9ao@?U_CLR5c9Q>Ypvl4I#r zgL9%?z-}^)6-!0+k(gz0uo~yT(eK>0KYSr=81q+-IR2+(zPuSWo)?(+ln$=Z$soZo z{xQbYgXXJ%y&Et}$jsm4;eUw5{%08PU%1)gKZ1)f7M>j7{7*>&cr(l-A%dwL%*iRi zNPjwxLcEK+JFlP3RAJKBZ0GLo-c#aH@mEUiUv%-WSj{V%KhRN{m2@p8Qec?ssSGi1 zf|;cKpEg`~&@L6{8M7=QI`Pu`aGEKHhpYZ?XUkot zAggbefMSeUi?b+BZn2tWc+h#rX-~|O+n1mIud>en#iH+s{$uM_q0@cm$$$zXTyypD zhyWa_v1k+zf(yd*I>GQ6!#&hQF{){{)c!pC;mG zKEn3%sq5=PMc2zw-iL8jHWk=fV*eP>ZIp@Ul4ggtCgCPix zv96I_t|)jAZcJ}Yp9}lmUW$>)mNrLC?TLTShT4+Y9iQ6_6Ta>THYsG697R8UQWPhl z$B@FkU;g{Rv~H{IhMNaM!@JDMhuWN-&Uhrexb{9In@ReVbqa(*Uc~-iuVW`p=%YOW zI``KXUq}MLoVg-tsZLF%vkhwD$`Qu@Mru>_YwA}X;$X#Xz;1Swj(KkZJ*qvj><|9i zn5X@!|3Xy%KTXx&XY(f(!c$6M+xFJ$^QYlQG+#Y8SqjCM3qfBpG=Y-FZBJ;gaq>0@ zc|LZr_7Bw=OJ*;!E|b;1UgvF`Gk~um-{vY$tE>HHAGd3EUrhD6;l`}|zh`JV?|fgx z*UBx?k1Jo|Zvn4cemYGyYw_O}Y=k~{#vh^lab@9^U>FRyCE1#hF#NBz; ze1382@lRan|DWl}MSxHnyguK)Ha(5{@-n^-J-(6{yk5K}_;`yw?D-qFK_TA1A_hHO zA92EFwqM0{I$(8iKK{n_*Vg{+^NjNXK6h1lxW#HoTQI0Jko<&Rl$#st^?1VAk4d|~ ztnolLeN8AOWg~qQoe}R!ulNpWBcwU7D-Z#0Gi^<@rMyk!yruKZiJ7^C)S~{9)+@BLQ!SyB`J_RSA?*{V&YXjbM=sU<^DHM?u(|!+Q0oZ@baPtw$!j+!qbFHUgT#Br(Z}Zw2mYk`PBRmsyIkl$gyXS!Xh?3wSR# zrr=LRVg>A2y!+PDkujob#;#FdC}bmaTJuRbFS&56e2XUKo70>ya(!Pri^dq3FB`mr zz0t!mSg&gq z!5v=~1;=q5i7eg?Ic>asntBCJpd@EN{p=cB+77;mU}cd{j((17#?3`0QsHpj4y=W! z3Xw=I(8Slwo+7 z`g+$kKZM5x<8h3d@Vs_~XCz!R4Eqk_m>(3-yEFIf6KqiWv+TEs8_)Q$&vKKD{Xu!W zY8j03@qA`>eb5)pc;};qk$E6N)4%y%xQ09t@jA||2jTXDL=!jBP>;U-5db#~!ai;4 z&$Jj1{=|x&wy#?^mfaj)y38Y(4{i}58p@{N*D) z+l}sgzwqF@i1(%}2*mPoJkpJ4#$%aX8=1N=EBLd|nQ?$?3qDgMzIo=Ei0Gz#Y&+t` zXR;al=GxZJ^ee_OWy1yh*iZSqHDiu`NSuAxJdzm1m50v(l$UM9p-a42P96*cFVogY z&b~Xny7Qg~>&R=I3~tPWt=5Y7H{+W2wvEacx%rhPVj2qmI7k-$p=|9oWElMQMLF=_^bfwz?DLNS@t=RVx>FH3|SDCg@t0;yk-*oV_zo@3}M{8Lb3f#TYK&;%m5KEoK3%AI_&d| z#OES0$8a%t(3#1=)s%(fgX!ht#fl(i-p315K4y)BcmW=XU>?Ffo=k|-7ru+lvjTq6 zgNfjt_^?ki+%*)IL(jA$+*n+em48f|4_Nns+|$k*0g*GaIHyWTMX+ zgyA*Qq<^p;ZagzhG@CaMwB=`B_;2cLo+ZG8M?VZMKGq6RP?H}DX`W}eX+*kN0; z?;!jQxD{gV($76Z;Afuk@MHcraL+zx#w^pvHSyzN-ctxeoNey*n%*q4$p9C8X0U<` zXT~Gk@;LKnAXjnRn($1BuVC{JXZSNcme(984}-WdX)6JTeCut);Fp1d>GPV0mme`> zKj#?D1Gv|aCoy0HwW8zEkGQ#d!P#VG=7lH_%f$Dq!GJ|$K>i_pW??%H&hVo*-+fGj zorWhK<{me!S|79lE5*+OjyqRhGiF#udLx`0*XG$}lZlQwgA+4Dnx;E=b#Qsn-yF=7 z9SR52bJxh$9xM;>;ytd3HP4LF2X&?w!}`HJEaA?>WJ3@TkmcuL>cjdmKTcWU&Oeh5 z)55ihhaaZPeutnoU-lmye9!P;A7))xM$;dSE7GDr<0B&L&oO%wo!OPu^l`>VWYe9z zyx7l~GV9g9zb%IuCvfK|;`ambw~4b37)Km0NW?tX&}OZzHWAYX42!hwxItd*SJnyj zM10m2Va$Ap9?0CdoARdAg$+8d7OMT}v7k&HQ5U)MiO-$5vT4>C{-%0(;~ zi|N?9B7H9|F*WIX`SIC>PY2s4aZRjP7UoYJ?HKX$Wn5mG07x5mOrMG8!Ozs+^?Q?s zEdw(&_#!C3&+&+k#xwnnVVN#|_>SYZET0=DU1E#!nfW*?#)DqOleefh>thn|{2A2- zvi+28#`z%+^R0uxjKlaW7Z3A%u%*DAoo{ih@L5jGSY{smQ9t%U^Ikz2`1xSw_3(FN zl+RPf<8zbW1nifrC$TcsqPrKqFTaJ+K^TW^&2Mk!MWm4go>M>>51qbvNwICrGYI!A zFXQoG>#{y(9>>1nQ*PM&#}@;OKh*#1bw zjgt+J3)YWmGynK+5aYog{zp6>49kOkp6^}LFK}kvnWy=lKp@U3c^^F7_@FbfVH&J6 zn!w=6_HyMTzAPIL#v@*4zKXgs9{VazUfaTt^}>DhJz%4nbXi$q%C5p^C(FgO34eb? zG;M+Oi8uXNSEj{h8j~R|wj6$)=)yGEf=q{f%+v`##K42g z>pAw58&eT+4xeD@=E6lj20FfZ;7(i^5{?b7|AGOZ$kSc7_&Pem%kqB}dixn&_w8F6 zBu?y&{$8~Grq}48kB=T_50Bzw@^-)X6t|AxxrR)4p4ZR~6?kT|ODTA{a~@t5ZG$_H ztAA=u#V+5sCrh>nlG6|Wx$P#m*xUW{^W(!yi!pjZiX6_J4vp|vc?u^yWcd^*I?t*p+0pujk%e=XwU~; zPWSrb!y0S~NJ%?W2nH`Mo$}gFnhTO&Z}H$6Y>0l>=1hwm|Nhp%cz7-;t}ihBvyDPW zvlYFuFi z4sLsi@8&-Q zIk~2`5Q7zSM*26GNtfHBg*+0qDssTdSCL-K%-1~VMgIU^x~0F<$0t->aw}az`fW+J z!S%BX=*>BvqXpkB`(m;FttaXSM;BQ{DdYGPPZZFe|HXmgAb6MFDh>4!7~>gT+>l_$*+AZ zP3FVbJ&q5e<-H9(!8O=nrlW%ceRAP%>p4NAT)^gA@{;Mb-2E1!mwepw4KqarbK;t9 z!$4AxCy#UyhY|3l>i;L>Ujkl{$KR)g@ ze0V2Aw%35^doqqcYZ>pU&fIg>Hph3^O96Z9-dtt=b=M1peovl78a{Ox{$73Z_e}3D z{OBXU=lkB{{6oJoJ7sg?&F77S`$9hb^`u{2Zq_?JuX2#*K9~KE|H5xAyn)^HRkz1Fe3(C5{`q_{`m=Lqy2seFodApE>#11g%8g8Uc>^ zM8dt(dGak@@pOHgL4dOnSNq6O{;FmX}BVKNp z@z>Axr4bzc7~_v}q`e$nnD_C`XXMS3V|ut17fn9B{5j+`5HMJJx8_0Ado?t);e*cP z{^_K)<$^JBP~WiR?eF2)UNv~*;ro*()sdq|%Bz38)@B`yH!&;*74rbuf(-WLZKCi! z02R;C&K?_}7oKSoqiq`u8s{)pA%b(_2h^=*|9z-?haAC+Su5!?(9z4^JJ| zeSZGfV&VPafByU9l>7DTuZP#yl#v3BSKfbokC#THLpf83m)4ID|M>g@YcJWqKRmxY zW~f%O!M(=w5=+9vV!W@z4bEmjWK|CJBj=eX8gB7!-YY;Y;|ppv6mnOR{|7gAY%7*{}Zn{qQcJC5hkp1YZ<)FA~X+4t#Bo zlk@P;xA`*x@rZ`QDr$rKF+Gbv!b*t#Nr=Wsz~X?PYLmo-sYSXEV@xx%2jAh(wbLQ~ z&<%9g=tj%S_9cA1@gYX|9S;Pn;O+IjZ{gJkgTlQqAb$ zYx)k?Yq7-t!GQ-OFYWNQ`!?mZ_s8XSX#Blgse#M$@iU?F+KQjr=hKhz3Zcf@)Zm$t z%E5bSar4@J=^-N9)Zpg=rLTlHYwSkVE`D;`@K?H$J2vSD`qYP2+W?MWv=P6GEcC0G zL!X=&o6S!3u$Mne-CIEG4-P81)8fb&v^4Tb(u<*e0HZMmd)(!Rx1HbPBlPb80$+S* zc`ZEV5-$dGyL1*zhV^V?OC{nGFJyPwC^;nZ(U|$k2V{%ujV-2qOxD<@wb^`Xt#5u3 zo*fE@chk%`M7ME*j||ntvk6(O5n~Iwh?C#PD_=Xj_)&*R8Y~*2j<<=>M1Sb*}t189X(G%%`r7Y0fp`0((h+~XkcW}Pq7AD-kVu{k?j zclgng3!ba|8OIkr09LBT#dN&+0$*M7uslU>0NUtNtjUu*;HsV9-g60lMGBvCfH~hL zPtSP$-W@!e7L9C%4eZ!3lYey9SVx6y?Bz0TlonRh>5-3YkschsfZmILi)7{Y5B2$K z@Z%vsv1H9+j6z;G_`f#4cq5ZPHdXP$Umuxl)~fDhc#kKwb+>#K|3;irEoAUTGCK9C zgm3ea?w-i6!fr1q2uV;)ThoE@ilOMORJT!lN*T{pgYf5s|kR^4`js(nm&^K9q`fr(Fsz@KAvstO(4X8;A68r-auU?Ucg}moUxwmo5eo7_Z(@|@N}m`V;trvY$S(0CHoF8EntWPq{Ugl`S|IuR= z40IE7*0*T=ir0Pgp}S|ki;b1qAdu(Adq2d7cltmFOa*UfY2Tq)z@{NA000HpNkl~fv^Sa61hvr^} z4^KF$7??SWU-^#?Mq+mNe%RmM(mv5TR-E{E`(W%Y2Fbc{blyLDW$Wo5zWVR{s1DwJ zHW+_o4M%P4fnLLQ&F8?tU88o&7q^@18#7zfK@R`KIKSH3MSu9njRu;03Cj65`06f< znMdUuX&kIvgcg3mz+=yM$0O5g8xL@(RKx^+;gGMGM~^&2{^hZd(v`K~@_PVZqepj} z!-naM8~+%%ih14PyMOrAHU1))2&0RRLSFn2LSHb(;D@fi^Z)919@*X#<{Gz7(-$%* z=nyR3Sqm+uo5#erzHjADKCEn>rJxNwH~NsL#2kMb?=?>mBa*W-eW38=#)(fd9#)o9=8xff7T-a?#;5P9@d+25n*+uBTY<+HKZkEGz_U3^oMwt+`l!3F*)4Hm&Z2Zb06_Wo`Ro`@?GtX*}CrZ z9w2sJBH8dix}Lpmf?&vo#|*H!G6KJ?7Z@k~ZVefqvsF40oRhICneTYsM+X+g;r`t| zdP2n;9<~llAFrJU|L?y#snN_H9IAqSa|wNj>6g~7`sF2g1nr$&MPFUtOU|*hI)0)N zsvU}F`0*OfJC__^yVh615uY6zy33vps~_m1cJL>kUj6BbKd~bpamHyMjShG*S@loR zrtB78@_vlL%^%U?uI3hVsSgf*C%8Q~ea`lHNycu=2Y+zafRAs%zQtsymh1RAVDav5 zZwC*(ZjaI348j9v!oca3x$FzYtbVdhH-40ZZaVU_bByqApWMCf@#oyqysL;y@_n|p zGajuV6Q|#3I-dv*xR~aTV$U;t#=yiEwpQzb9+#(smD}YWGLu2Pb6+@}nRHZmWC`1B zRR>&fxU4_HId9ooy>Y^T4)A`~!{o>w;c^x_e(RH)$zh+tA9oH_Cp%bM)jj^#UYi)@ zCr{ncCXW}Ip%@%JAu^iv^L@ESKYj3tZc~hPJ3i@$KKaM^A3*Kl+FVkfChr-XFDYce z`<+7oh!=eFkI5c<Hjy4H;K|E6FD~199qXn(uAyB?BHXSJ~PsoovlnyLA$Oz2ocV5$*Tr(T_8pbG{v}@YOWC@G;Nk zH}7VwwKub-13FdMTh(P9?mtuVQNB%nY~3*8P@a&tHV>zl{l`>xNe>?aW)pJh$0J>N zv47_a{R-dUL-;|1AF`b>m`jcGCA87SpfTs?*!z}`Ed32=aT*+cjNMs$b@nEYHoTm5 zB$*E`qm20*)(Q@Yb);zm>nRAd^&sOjTRJVp_!7d!uj3JNtyjnb(_`uZ`a!I9_8M9Zkz`ICCzaEUNJ1Yt0z#53Ewx{6zzw zC-?1J?5o^*gOB`|J1jTD%Z79SI&Z$#;wEvLKqCa3rXD^JtO*J2c1Oog_-%O){ISA_8bd}_f~O`5^FPAbVC zoWJOc#&T1TgXtOmy%avC}C&ky+;yVi}#F|rS(dbAg? zE>Zrk_u)UngV7(^m>%#wuSmF300000NkvXX Hu0mjf_+ms& literal 254497 zcma&M1yr3omp@DkE$;3v#a)WE#odcL9Gru@yBwgnLveR^cXxMpcl&eSnQz{i_1&3U zR#@vfNp_N*{I+E81j|Z`z{B9cfPsO*i-`)#gMmRqfq_AkLw$U&VZP0LuYmhuA|N0u zCLlm0YXbzBm>YtDiNcpCJ1Qy+V78W#dH?c53G#{=Po`=SKlAZX)TQ$y`z5Of1%d9W z>hNjKC*;#mNNGs@;UP@F8d60SmBUkEJ|;sIGlVMpKCexMOcZVlzai)B zJcT?fh-A+4x=VQkR3Yley8VJ=0Kes}lArM#gPQKw9t@yB>or|Us}yBNLTOlbD2j?? zYeE5Y$3kKB+8Cw^iH~R~0V%W`p;IJBu^YTzDHA05BcT$AiQ)HK<0In69LaZM)=ptC zPKCs1G}fS!PcVLpw@&5NjMUahlr*`0h>v|$h7G9sH{Q?dwq%?6#2WPUP!ivGCcTV| z#JjSPgl+xL=;O$8i9mLnH$s!!&``EWmh}B%zgcBHO3@^KP-z)v$@VHF(IKxT7C`0c z39>`blz|b)dUk41$+}ArURe4^b&GaqRS1Ga@o)kdAL*IIXeDr_1w^;~J7Ne2TUFAE z)Ye2P<>*JlemmLvIijJF!-erGCGeo zAL|Gzv2c}SCJ4_7)cQ*VC+anu0o!-+o1&&D!=$S*p0=$hB`=6TJR&=A>LdUQiDafB zpZeP)g9hs%YSunlek^>?c@NCnWKxI@gE2O- z{2|!_iz=hpo@<sLeY^N_s7BBoV;1`Q8VFUK#7;Q1#pGO8b-Xu-mH zKCibvbgDKBwoQwB>Bswv3r%?8i}4z_3*zR|V|hWVc+rIneBx=x{74Mf`<<5T2eqV- zYWz0>d0z0{_8(N<%qq}kx}@g*C*`;ve!jAhA{%5W-l105rCs>d>|fWhuiJ6=yBI;< z6-@L1uSC6{3E;av&bqHESQx%N`UNUqAy)I<@o&tv(uUG}bd&hDCI=oD=cj$aLqet2*&%ffzdCw~j8T-59h&)4Lfq=Pn$c zu~#L)GQOWfv2;eBll%x$AqndzL+j?rjL9b|400C)PdX|RJ;!_*#kl=vx!i`_e zVfQsvq%OlX(~~7OO)6bVTX;=!O$L}bJQu>0`TaMQ3d4i3&P&MFQnTQ&|X>#P~xIA`9)w=G_YK*KQ@ zT`9m9)gjd;m9XNaVr6sor6l}*X?@Jf_^P6Xy!pF@Gfs_BV`(&z?!>Wz?P;4*t2DDh zGc}XXCekysv&G{)+0J5B>JpkmYBYipzu9DpJ*csAEo z`qtqEL<0QMFCY9px}HXCVhrS*unO3~=$Z;xA+;BI3`Yy6#do&a1`5|lw#YS4m}i{` z(+RiqQuHGC+HAr1=7E}|oG8307AVXl!;|e2ZKPhLa&pZiWfJ&yjxK!f8=h~T0Ui;F zk*e8=ljg~)MXKoyVa`zxBv&uHceDu@S{R8KX0%DPF0@i~ed?nX#TCyZ;wytg;(9fs zhs87MX1GUaN5JBvVs49Ti%jzn3sdvc#Yzis^DB!Y^Rk7BljdWaWA-J@+P;&O6Y|^G z+ju-HNJPkjE&K2ufYl6Mz{)Uc+oY0og@tnz>B$_3*W&2jMmLpgWNREmnj5_sGjk}A} zwR(;Q&&%^`S*PQx{mbfW$-Cuk=P8Lfno+aFIWAz5WBHxiS=Q>AJC7TTJHq4h&e4AI zR#RV{ZSjc8#XP<~;X~pR^mFY)&C}J()JxkP5RnlU2}1D`4_q}A+NV=EHxvWRn0Haw zcP>{bC%LyeUqVX%)j%AUA!saUY+`JdAgv-l-&_8)vAdzV!M4GRe2QE}ibHA=IT2Yz z0u;s;fzB$-a_MyrPv$P=Nw5JeKcgw$7x>lZcXKF<~p@DvI>a zdKS8O`d*kL`Is#AcDnm6MlWnP$0)2&bIFgzoT9N(>Par7qb#DB3%;nbINR9RUoORr z$6UtTis@5~q}nmfFe)XfrT238T!wvNR{%(J@y}S9o=)pe zvF1VN;*00yx*(d6-SkUM&&scE^aA}5B^ip=t?7(5OiSlB>1 z2RHMsQF>ZhHSPpT3| zyk8I)UUk{I)H$0>!WUBfZ3%|qlbS3D*ctS-rrbgwBz7HXgH+00iEVX9JGC*n=%}@? zU91N;1IT+NFle2x?YrmL$Wu^+21bBxZ3~Zd4<*-2vpJ;_9JDVQ)YL7qyNW;MZmkoo z7W~gr+SK1P-!6xDN@#d~nyZRwe^GbVS86X@*;(A_*@^fWp{>~z(3WdAa#S-ln^tC3 zG~XEBAl*>v?mqnZ2p@<0)AGJvuJXdg&}!#ZN;o;r(xpkY`e87xo$jd?2S~T{xZ!$m zS63@!UQ|+9S~xGgrd9oN5x$LY;gYv8TA7;eFyK&g7QVKkaiXf((q}vtg3~Ol zI#uI%xjMuj@T$@_X1!MX()?lwx$bq@K9xS{*=Wu9s53=eCh+7@`{+ESd%7{HR_yT8 zLMAo4w1uyx^`yU^R0_ToKF|~UWvDa@AVL#yfUj4LixwnAAY&1af4 zYqvL@tuU8F$zGQh&gXtAd7r3YgU=JkHQVw>s7x(9WN zawe5NRivr2iJt%E(4>#$p{n&XiofP<`Fi>GVa)T>>&zreLEe`CrqJ7vVHup(iOb1t zV*y9$-D`3)hdWSW4ZC^iW%cM~0r3lA+V#kN|9$;y^h#d9*hKo-?cUAZ&?gx0L)ve@ zF~)`=oEl)gQbR@&;y)y$cyq@Oe#EV(Bmu$y40(klsE?P!--r9q^zlO!23R-iNAvG9 zFuE8n@_lFB-`X`b18MZ~<$oOly&{4C$RLtloQrf$=zfegCvDw9_SWvM{%_{p!R^@~<9W-@pH= zW*{N@R~I`oUJ?~4St0?TjUf>mJu5vU2_Fm*5fP7#0pP2=pzwc^zyIPTF}Aa_`pUrI z=;%oA$U+aaF=Ali;^JapWM*JyrhD%}XX|Wfr|U##X-oPagZzKv2pZbz+n89{nE)+` z{u)eV>DmnT3g&=U)W>zovgr`QJ!Y|4quo%*goPN&j2Zf0FVr{Kdk5W9dKk z>tD6+8^#C2!|-35&j+)UJRJZA_6uti<~_Znc1G04wN6;$;kF!}wt{jFa}pelukAfU(& zq+OcSXWo?Wq{4ez>Y9L*D^iYoUy=qJ@>Z*+&lG1)Jknjppf@=k^et9%`8Gly1^7gu zk&!<9b1_c)h6KUZ%a_L|r@t`Vcf2pGby_JnA~N`xGI2B^_wNCW)4V)7jMHLig}y-u z>U>5%_v{}1%~w=JLiF*!7X``oCx78bSRc14>;au?gffQZ@%}ms=v(tx@f<~J^iZK& zH**!)#Q#q7zkZj0f%w&G)1V0hgQU)cX$bQGX|``J4<>|QxDfF8P?ity#Is*KC4so7OljDOcOW{mpla<1MQF#Hde z{LQN)(xR3Vf07RxVTh6<_aeH1dAk z(l6tW#u6oeva3Pgct1o}m_ig+*Rlb=o;NB>X>PMfWSzLWx^gw@boek8MSK4l6Q8~F z(p*AJo0^(b-(VMpZ^`U3WNS&`gT9m{9r7B=&IjLILz*r!x*@Npc%wG_JUTzVXHxQ2$5L6#e}-_JNrlt`I0}H19%AuSd4W z@(kbYDrtR2ZYKqwd&oM)LOR4;Bg^J5OYl!cn4EEmq!$TsN`W&xtryL8Nq^vHfgVa|M66sL%r>i9Uz7kOUm>F=4Y=wvic&Vm1@^1kBZ@?n@m{kan!pG}V43Qj#x)58; zGFjM;7={w-OI~1cCW||c#)d1p5(O}0Q)d+4phkiqo}iCy$wLw=E;sCb&r}ayXBQK( z?Y3q4>7;aMqWe_YLjO@^wZoB*$K0|N$~jDDjeX@T01-IfcWAq9fo`crjXKSd>Nw|o zz|cj8WrFbH$vBP&h&V$Hr~AJ{hf?n-eLpSGFms$s33tA#n+G)cF9F-Enojmc)6=1u zyRCI24g+>_?+9;bP(MeM;5yAI#B%SAj$m%;i5f+zFcs+i$%aIWud8;>2)kniGO{}x z{TRcf^H`k?b*SHdbGHfQJRPnt9k=%@WwWF6&I97(cT($W)*4&SL?~8k{S{-w$S_Hqi~~DZnVd$sX6kUuh+S)ce`s}eI4e)+-`y2>Ae6d`P#a7`QP!%)#tZx zN9XHV_xi`Wov&uB0#C=4!u5TYdgZbt{{+>&0y;Z`u?g?KG3gx=lA)`OXgQmL0?O6& zHh@ifHO&&wW%&a5wssb~wB#!f%|yGGpul8lYb}B_R`6=~$=V33>P#;^cn=UIXoLhz z4c4B*VSwA>$~YSM%-irgk9?UrJyn^W$7TlD_FS3;d^~6n%U}3RwZ#iN9yb^(!^0Fr z9Y=@C9an4-mv0*^Sf^!<3rwxV`~wsI?(RiL%~Amp6{NXxHgrRAt6t8}8QJmi@!{zx z8KZY=*9_~No7&bhl1qn5PKNeuxG>2tRoE;+;}sg2aa=!%hWsTH5+I@@`+LMhRULb8 z{4ABYxE3)WVg?1}Bq)ZBSo}LW1=^lwQk+gY9 z1dNvx1yQ91>CumVa*Ykp=I0<(%34S&Q;7Q$oNtjZGBSF(o0-w5C9U;W*s&;uk?IvE z9lJj{`OoyzZdO%38)(47Mi2*Z#Un-$?y3z34#==u zao5)KUWx%AFAQ|YQ=uC-^GWG}6DXwzj&^fVHY#8z|EZGg$i~r@Y(=5hjP%{XKWFg-pH}hlxVhoVrxLhWMX-CDd;^c%3RPGZkVoqyj<{ zwPV^Y`*j{zgy)C$J(oI22#vqjfAwi_K@A>n8DMk?W)zC+1#@C>hTj@Q|E`dafruuK zK$e*$Xhc9`#w^3j-_``I4x&4H9t}GZc&Q8EWa4P? zQSB;GTv2Qm9TueoboddiL6ER3cXD&J$#R4Pb4*BoXcFDxPsA_!U#d@W`7f7)N;KnS zfVg8;GI@$kZ+N@W0Kcrh$>f-7>t>cpZ`B%MwGM9TC7<-Sp_a+L2LNVv636 zGHltbuW5PdXKAfE_lqc{76C_}8;jDn9@h>J%<}5~MlN<&=Wq=#_#z<3AEmTjJKC6G zilxdGIe_8};=+QB<=paN$K2kkFHtlJX3>n`ahehnS$zn#4T2C*dXhL>-1_X>x9dQZ zgiI4BnJhYh#6%m}6QxV`-Ci(6sjI&*(!gz2T>p;%BdRks!~XNY)S-$CJiUmwWq8M? z>?wUrn*cz_`xNSD9Lu6RDnwjG9`N0n6XBMx{>5FJZq-Wep)vh)#Hu#bavYtmD|F;= zO&~og`B7K}2#W;!t2(T_Di#tfhGM29J#MMXW9!(fne)tGc%A$Dz4&b!9e35yrZ~H5 z6>U%#XCYA$c_+?yqYqcK6iVZSe$`Gb@)+H*;=gnkdz>Ws+}+)a7XWdIl7Oa@THKcD z!k_*gLZs{4xJ=)Swfn`1*CPn=7G6S8&Um~d6LMn%8EtMa%x zY__W2da&7=o~q{jm{?-lq1h#>Q|ezCJi3zriR?_1Zb%3@+}+=rvahAyeR@&mwwVTb z_Hsly`r}j_f?mR2j*>7X(!YdURIw`}M+=D)3o7)j;lDH9Z-0m?FRjt6SSP!uccg2Kf|t zUwdR0sz$wT6ukk&!k-ils<%6wgb)u@mZ6z$Uo3&pcMaE1Zx7mM%a@6LSzHTV_m#&1 zVrYrtOoGJv5oE#xf;1AH1NW*^5@jSLy@d*V;jNS!#}|EsGT?0&y|^NI)00s_F6Ptb zPZK9oB3*|+>3CaHQ$H!GR@BzAn~KjB|3NrFlF9p`06(mlB`?KBJj-o5%dlLdtlF>j8V3t&m)O4`Yp?R&TmdECA1*t?zy$4l0$0a-fv{XFT^YiZrj6OGcG zABMEUk7Gt9^9bF}J46WGWAF~wfNd}1`I90dGVwz@M`dju&B*$#u}hXsC2ei#6hY|R z>t5hVTpbfnSy}(A6v`(cK0`u&Sz25S0c>#C(5^_9s^r+l2gPxfU=B*d=twmgsDP9ot z$$EEkt&hJAu!^effnPe+_X`(%PmhUcOiFpHbaViZ0TO`n!RYQ)c5>b(msDKjKrV7C*i{Aflrf=yS|)Z{u4 zMIMSlXG$R+QK_tHA1gMe4s5y6ay^+D#(FscdXg$4cf<~jsfj#g->12rHjME-A!T#& zHb^;@VfoOM)@N>f%ZBX9t5|kDYw6R#v_gFnvsKo1XG6TBt6N5ibwM>V+jU8pV8448 z&6m5iu}{Z=1C%OEk*{bf&yv=c(fSuAUrAyZ2MTnCI(N!vduh zh@GSJtM=bSHWR);n5KZ^UQ~FqSZAx9vsoE89t%_#a8|v^*~YaN@mmM`iJ4;-Dk-5E ze5w$q07>LfB`<7hOKVThJ*_j04%m~yg2>-bGcOWK|G3CTCDS%CRf4suK1We?De^?M z)#c2Lp>=}|7)BQdI}r8}+IzmnNE6uBwEfxfiq~3QM8?r*)4yPpmGf(Q+uO?-hL=C8 zQM{%@N1|DU288`dMIV8~$jqG8enudHO+WFrz8BE4`T2`6uBnKKNFP;(2`*f;XGv^Q zN~FvFN#9c4$ZV@D)o+O~f`LF8LK*rAKXw?>iq5O2rnQ$*bFAeRhouoc7?DcP$Ai4K zyOuMi93vMiG1C5uYy2gk$5k>%>qS_uaNbQoP#O}5=78ch1 z%bDlcPRF7LgZqI5$7X4IbE{NWQg8uzZ4Dsdkl(@HzU7?>5&N|1isS${_NA*GgvRSh z35!HYqI0zYGwr9vu-_FfH;3rc%P3SMalf!M_mQ2GL+ z@O|OXgs^fNK@ZRYnL3=qXf$%3%i)Rkfw>@7f z$nf42 zFL%r&Db4`bQo}g+(RL&wUYoF|Pit?lb!{&egowOisd6=cySpq&?VKdNxdYD3Ys3;h zyseFIMiILq4Yf7}Q5}wwv8D&KwvuA!IaM>s34np%L>L_-6mkn$#yO1DA*Tj}q)(yk=nB?|W ztU&v@4~&PF?d2s2PlnDGNxM(JqS(r1w@hHv-!4Er@k+??6J2 z`#5#WiUs5cr`<#&k9`5cMZHkE!?9nXA@yB$uTmTy&`D+b}|$$Lhgc9 zC4aSF`+d(5#qT`AZJYFUfiGEFdynMryu|VY2tXQRz>jK6<9iXrrq{Sz9md3Nq=I7V z5b`HMEz#_XdNd;M1|Qo6#$!TSzPECHf`-TEJYXyRQZKcETp|`DYG+}FKEyAfXQE$9 zcRk_9hQPOPhA=77<3xX`J&nAv1yr-+N^zpDN{zaUUnUvA#YYTC#swW*(TOCJ- ztNKICLMo@NPp56KH)R%lyy2Ur7Wv?st|v7Ztpcb{&MFLnmGvI24Q4(|>FSbbJ+w@w}pvO^g>8A55(S?Q}l#w~J7ileF&RB2A#cLRDSLgy{Nub-xM zor&3UQvxhuYVUa2VenjFKIt}t11yW62`*k*F+@H0{2uHibGd*m`1er%b{|~s04k(Uv8{v zenk%K=rt?sEB8#*S-hD(6A7@`IxYTP^bfgNf||AK#FmN5t(UP?uZ3x~ z`mBcdTdH?`{I8L_PUiHgm40>bqAb9}YLs3KdX4+UF*m?q!f?| zPSImO-VX!1_vktvI|k7xLwVVxs@6G_G#yGic7&aFXaVG5)vm(*`;5uqFzsqASjyh@ z6z9P+HD|qzWOVym_|)vWI*)3Xi(9+6jnDUWi)ZfLXp^oT|cJ z2s%wO*+7)N!>8GKS@f^JFiGVWWwp%A`^kji?Ke)F&H!Fe#zhB8VJ~pJriyerX4fow zXlYKH!S5~-B!xywy&mP@>Z7azt}^Uae~Yij@V9b3yw7axEHMwcDgG6ekUH+XJnS#( zAA|a!%L?Ph(jq)!zPz@BBO+P*G^POhQ4@Zv^kVMv{5DS&)%I1bG$m4RjL9j8&Sx`S z|60H$&L^erRa`dmQP^D_gXTD5cKTbY2HZ7_t6oOFDcR5n)=7>5%>qJXrFIC^pei$RgR<;&eM*xS1@h58R$>fn`1$xfjdaJNR*OyCLCMVKu1^H ziHO-+sPMbdRdmoJ|<(s(NlHJ5}`J z@=eji$tn&J3sq{7cpJB_T3t5}c4pbBBc1|QUSt}N&Kzann?@bxda6ULHRxHyNMT}& z1SRSY4^I!VqM?g|GN|Td=`t)YOY4PDN_qSr{~-ObepE6UK9BomkJaG-##hq;U3^P&_lzwWTtuFF<__8koij2K^BA19`Y|XE_sTo8+{tAnLJk!VO&gKZC;MKYT z51hB3(=;2j^mA|gtfqUyQC4Q#&;I%7ybEzpj9H$aBk&oXG0O93EMD^c-C#vWX8;6C z=;i&+!#^oyR7<_?1-;+hs)PddXztYh%F2wm3W>3iMcfAv-2IHg@BJJ*JsMRKLjt7|#2 zX#TM=v`E|;bd+W9-2OVn{}MfdA78t+JV|uP0FUH6jMfbifWi!AAxc~`nxjfb*rgY|ZpeORo zC9k@pMMFNmMW`tADklOo#nL%y)5lLR6ISX=2io_uS=^_)kB?i)@eS$E6Pg}vxPuLJ zjhcLWop3a2WH(BK#0++rFp&J7AF@KP{CR8(*_T}@@d5SaE%1}!>3ql$SgVr(H@^-z zv}gC$U8)d%`EtMRXGQQ`bfY?ptkSk%pJiMzbB}&M7Fks27GGG6CyyL8`rKdrD6~EM z_B{5+IJd^V_AnaZeymR$)WfXbdaV6=iMh#G{8?k*9^wS4%zUm}i>NDmMJAij{d&T# zDaGfLv>f;qMFvA|#AtE#p1uLjX#=~Y_D24vui0Z0W}3sw!89qK;#BP5CFnn|`pBqv zhRLlX;d&^6rbNsp>z}3h9t=`$%pu|yFq89Tw zw~1FT@5W(NBi_Nz*W!}F^c2zK9Q-QwsdB644qXi%bf9XKs#F%D8{ii#fWgrM3>_Y! zJFyP4JMAY1_~!1Ed4{%a1pD>-QR$5{Rv9rP=1yjuGr7V%X zXtqbsDl&y9WuEo*bm|V|@#pJluxLL{#@ELdr(LHBEW0$ntPfr4@H`z?;R+vZUcU`0 zx9Z6^*5%cX(2Z~uxGVhTCc(JG4;wX5YzQ`&os|6cn0n@xw{arbKhdz4b|lg*{t&_c zyn$$@$b>NQ_%312eq%UFAc5wQcH>EnM*B-=(rk)A50AH}wZ6$ZLud@?D~af+X!@Fm zZ1^6ky`f~1SB@|{G=>QAUGh@7p8PQN-NK^MO&rm+wsw((>nlz5tlGM~(|6H?J@QQ( z%L0Aswp08&6#IUAl(`9$7y|_{Z8ip>yE-YoF%S$nZmV_bk#FgJ{LMT7qXPvHtHydG zRgF|R6Duqw>218-&Hx}|VmR12jwR2r?FU2$V+?2mwV^-i_1JX9*6^-qzuqgq)lYSF zj0|n^mspvJ)2CbG;|J?*riW-K2213W%Oecth-cK-=iEPHp2bX85$_i0Ce?rNgTW>E zh>675%q!k2&3)1x!-v-cD=`1P%Q3E`_{4p|j>05vd7aK)-Me`AmwPuXh^B3WIQ?Kv zJrBJ}>xFoYUOK1R_w~0gg5D%QQ23(~sHQ;WBfL@R&em506d_Zctrd&aLCW;29>?H( zVG6>9WDjAVTx+c@b00WMDel2EE7q%rs9XY?{fI{wfFMHZ^%50tAmWv=?!z$4*f%)Z z?216&xki=Ba6ERE4fY)V`i*^(ZtxZu~b3#v4L)*Fe^Pkf6MZxWtSkrUmVk%3yGscBm%G(G@)eyOO=4$T5^nXQHH@mHVW`u^u&|Xvpj~#eifijKr`- zELzj6yE5LZf`y;D+qV}eI9sCr8bXr&0B_@J`->h-&vkIZ@-$QxVJK!3_7GbpD{z~w zr7KiyuYT#Dsl^*!7rW8E2&Q$!GSqe=;5W%@&jG|wJ>YgfXWKK=t!a>p zfoE=3@JKIWjQI;!b~^;DuB9YD27k3XA77Q%6!~cTIobLLT)UNSAhPcKoj+}WTHdz6 zVaSi5JIoWT(6)DC+FAXx1h?@%<2U(QBtw&%1O;wW97uqi=sxFb-lhvLXq?VNue0-4 z1xeq5zf->xKR0f*gZ=UN6q{!(E(yi&#$hbarg-EL@z~pqLHDtx2M~wn<}6nk0(~RR zDZT@h7T&3r8EcTZ1N9;eHRC-Wv9SkpRPf?&AL0LSQ!zHXVM zS~9O;Zn29(mJxi028E2e0_>H52eb|$GlFl&xQo-%Q|DPNb(ZgeIN7`zkl8>wLKN84 zi>gQCSoYdYspG|Lvvb^q;u)3Gg>Cf43#vw6pqCSBr zk-21(ej{EI@mWMQSv*;?$;U_&b<3)8y@;br;DT|o;KSCDOFWs&W*}rPMBZzpA3-y~ zF&bAj8%U--f+DtBaX7YRi(P5KYvahOTfvKHr!~y+D32dfH5W%9#?Xe6Fh+7q`u| zoi9SXF*~rK6}1oS55~z1HN;&R2?W;N$hVQ3g2xY5rlnNL1c_P3MQ+4uzgXqGZf>Uc z&Ur=8IpUdmCVGPy-a0W`+jsoPFA22a9a@h$iY{)pH@+>}Ua`{VwY03z(h*uhsH#ch z&?271w>|IdrPv}k(u>HoM!x5^EB1CiSM8Z{z20v%s6X^8bxf_ltGa4T(tGFWb+6d;4Vr^dJnt@!nV1T<5k;RcvBjjEOrjAQWCg5 zhvS~TZH;)gUES_@w!U1nEjK=wE)hI!l|-y~9960V8{cYi-yY||hn|#gZg^L4;n5l| zu~|#k-Yy~M_^;Ow87EQ_#4ALxRJbe&i;MRcb~oc+rywI*&u4F6I@{f{b%YgrOuma$?bmV=PanjiFA~YlkAH_)$Jj0Q+Dyh}|R$r6s)eBOsIc)#c|3RgoR` z2M+0g%Em7*j-R-~WfiRWRr%CDj`GnKAn|8$W%RSkLE?(K=)!SSr&{ zlJ_bc*gO0bnYSHbbAYmKT8Nd1b^ip*oEs8pax03)CyZf|4vE%7hlC2gYE%B`9EVc8 zn@kiy~s7P-7AOjx+0Dz`jGerv5Ud>Yvk zUw4yhll1&C4H=271G>LtE6k5_t8(I_^)q3x(y=|R=u_j1(;<@z7gP8-8($g0^Fd`} zY;s-!T&><+kc>rXSU6DN6?Ve3wgtPYtROLhRtF%Z4!?Zt;nonjwIn8-7SnjE`-+RNFWt!pVHXQKVyc+cl(y!%Hc71A(m z<%f?B^1}xH5|Jw8tYFmHzlC(pktt(tusN8ZWwLTtup@z#lf$YB1- zbNWS3gxr61_XJpm2g-WV<85NFg9Etqp+xh>VzANVn&`5Jt5>ZT^7@8MbD^0_gs6FT zVieY8^2sw?m&=DY`+rVBt`VZz3rFB?Steyv=y}+|A>2VVmTBh4`O>5W!m%EcvaDoK z5!quA4jGJHMTaK(?V$cJ=tRFk{kVU{tMoBAKV)#ABY}5}RXqo^hmOCFu{8h;chWPp zu0i$eRSXoom&@}Hl`Hu|4q~{nTC@;l#ZpE92zt?J)c!e~!p*wjG;G?3ueY}j zG3p=}b7_g;XE~ZqM%|z3GScpnRjM}q?^vHv&<1D5l8Lh8yXAu!v`e+)O&+_?4YoeJ zM;4pPH!|aZHEgiH$Ir?|yRmAUSEeka7I}!O3Bb`p2XUKO>9sR1#1~^9J!tU5{N1~L zZ!r=bPY`g>1Fv(QEL9*ZkYoiW?b|L|^GRgQ+_TAfhU=;O`o?Oxp`}DAOY(@3y3N&+ zZLmbhCMSJ9#FqB*!0e)^jTgNwI%hNmYG|;pDfth3%aVIkq?O{8`epb~( zlIxHS>-mOr^I}amcxmOE9ByN7iMPRtJh-hGG3EWMj+iQc9WEi4-Svb>y65WypsQN0 zt<=)ZAi;`*|9e2|A z(A8P>_Y5vY8Fuq}NXG?3%ou&OVt&{m>=k`JSVbl;EP4W3E#XTrF)M}VVT>r0HGK7iaHsN3{N{7ld` z(nGyN1Dj_2m`uC&EbZTuioovGAItS(&`btd65U=lu%sTS&>=Z(x14qzpD@b_O;~ix za`%$EV%uAG?L~Y%+E2kDorMz>p;=qK>hnMZX2GgHL%#zO!Vn@j89=HCt~7dA;BzsR z0&KuU9sP&nsaw9P+}(h+gH=vVUl}06*3p1(C+Zc%B$r3=Nxo-B-`J{#LWr|5JcA=E zB4jJX$(E?H|J!6X0dM-$_TNPzy$gufU&wG%-(M*AuD!9c7`oc3FJJW^PK~ymbp!?u z%BhRY)hOD11QH`W_IB$P$~6C6ZX4pn2o7l*V&@Cjgrc+PiMj3k|KJ z$Ab!AV5cWQD_Iz7dCT6+w5(&@qE>I}&?`R{Q$7Gy(OEuF-)-a6J!99EGC-pxPaV{z zo;j%O@7DHKR~ZzQ=VeW@)K#f)x~{BGN^g#kDD(*H`j~xQTdwc@906DMe8udVWZM|< z=%y9Yw~#a{ljp9Q>^*|;FdT_;1K7SdTNF?1Vp{nrg|o)gDtZ=wEpqW~gOzoT?r`n{ zBwjo56{!Ntl*|RPV>z^bZs0~RV3ye|-ttab-&Wbie_bh$^Ok*&px4bnGUDrDev*$yH57E5iMzfm-+c$->-*L0 zJR{5jO`CtoFkZ7)QL+0$HML3yXk_F{x_)5G@l{=zIa&k}9-n4jamL)PCBU~=kl*R*zlj18AM8RMaazka$k zyWXq24n-<(UY<<#f+w*4T=w?J(8&~RAU-!1qGt}`R;WY96z!8Vkg8i{98s2WPajCU zQT_<#Jnl@v*FV3f;^TJB7ywOyF?#@i*l+>AbWWA%;QvcBo_Who`m3gAwQe}Cim7R> z%i)RpOO=P485C zC?dP&2;%}^>D5gw%;1}5dCxD?OXjbB#?v+G^}>%92zxP&WrkjLo!YB6;!E4uI*5Mk zY}_O3-)(YwO1n9xmWaA9-#Bd4GuVI~s;j?!e#{b^PJqoCRGux>H*`?TK?Uvav!4IT zZJBQ^oIq_JmJaeNAez?JKUTkf*hRQ~IM8y&l~A#Or<=-g`>M&KIlJdN6kyMMq`atf zVu54=^-$?}?#zvgyExoGpfit2v&YHtNCwo!Gs2;X4HLbQa& zghF-P2EjC*LQ zT^l#KzJBUdlD&R9avi@%d86{S`3y>M`BUO9Uq`J3mcc&1W#Y&CWL!1}G#TvU0X>uR z>Eg9vxn{-XO{0c*AKQ%_?7M0USW0%! zsTb2EYz^qER$v>Ni}54lAxDYIQ=M2IzQ6HIbq(0pzHh~Mc0cf=+jm$?#2sq*G}+Z3 zY|NmJF;IfFI?vg!-oH|;=Zslyjve63iHl>|;ozp45)~kk$nSF`=dk~|V6xG9D3L$F z%o8#d=HRgEsE>Q;4Xc~@o4(#>&z>Kx!Qo^Q z;CV`La41YSlPGs9Tbnwyct|PblPH- zi5uG5)(%&fc@QqehP-~Dcp6)^i!Ao?=`pI)(!Ak+n?rW;!@vC(o#jS7%1s*mDh-cxsycq$?82@o_wYUbo>S zqz;03o)95&mvOETp0!sm(boN?-RpS$i4OniX>1y)Kre2y6iM!2L55BNL|Zt|q#UM7 zGt-szmpQKtG3>pV48MF+$Lg`lopjhxy`@Pn?;NrTnZkX91|(CwGFxK+Cf9)PB3*`j zfK>duD#f=TVy=&V*h|J?4rXl5k~4CVe5{-`iPIm+QmL-3Uu|VZ;2i5PdoiWRqPv&sa41v$mMXp`$T<<|7;=EM&OeES*k zw%tAyj2Y&UvBT5rym+>euEN+~r@!bo2I#RKV!*sLQ(dCJnVD?%d3@kGbwiA`d8N(= z%k^vhNTTgt2b|G6vGN4jPOlhm%zS05#hJy0>krLUZ({VEPO8PLp<5u!ba`66Y>MS6 z%Q=*9Frw(R=E^#g)i!GS=g!lu-j3M%(yVKc+4LxnD|9PQ!5ai{4{N+~kmB6;Q6edl zCo9X>_Br|flT>!P!Y!PH$NVK;{uJ(H;j18}kS#u2iUBAI43WoK-TOhEvA&%%BBF~C< zJ1x#L=_01lkRR>R^D@qw78H#t44zgdDcz~0+N!&rjYo$>1P6GrSURdv`>`m=yHeR4 z`Z?Rd=w&}`yj^3cF-D2M8FFl186S+t;kX817AYN(_NQnERr{Nx5cfb2i&fMw=RWL> zUu)Jh&ROk;%0uQL7L5FDdNrhNJwI%~=2wV&Ega2Ej&%uraz9B})Q5V&7KjQy+~cX1 z>%&GssUlx~S9CqHLwvEo-f}gli7rMEKX2lbcco(C#6t8A0%}ZLPg%-mqoQ;B@f>a0 zQrH!!Iu!OEO z*>DBBeX*}(8&Q93RGh_q=Vqemu6gn%##JKNl^^|NF}!1V8Z(8!z^mA9GZvkv>)i2m zOJZm*rQ9LU~ZyDh5_bZ;?TLov%=K5@GW%8b6vXSN8 z(5sC^=21s=&QX&n=Kii4A3F91SBmVDk@klm=kmrAciMgDO2)9miQl&B(lhWFJw{Zj_=gx?apvrEgUD=6U@B6dfRhm^PVGf_J{$Au(%H zcch<1@lk>33@z(h13pLeP_2=*vs-_#C{lDfC8bx0h4mHR?9Z3Em7T@0h0s3K(UiuN zdd6zwoHQk;Mb>7b`(b7p?`_h3m(?98v3;Ri4`ISK;$v1+Eh96Txa|@xv|_t_(*|>Bg#j z_}cYmcCic6Ac-_L*G-YelV~8sWT!zIa(1~h;La(mYYj@|cE%i!l_Hxx78-VF6M({} zeW)wJ-v=4+3@E-(cLlmAdoG{PTGfaPKIR}nZiNpinB4v2NQ(wP-a$zkKCae7<+N`LJSWBNwYl@hkJ6slgp|`;NRmc+ESfhsIHjF;qTfibSS~sl+>l$i^Fet| z##RQMN$1--)=u`6kPfe_t8cd|d{jmADD~TVcuf+n5N!`-(HgUF_xBr5ByDQfP1<(u zo)tgeD96EquzMNNY#ejb9|BX8IcQ)SJ;&cHlixY2iI_^4d`rtMNQiTG$lDqr#bpvb z6-s#M&HiNIQmN${{ccI7m2O}b66V8t^TvV@+R>R6Ijki@iN&EZ?p9%4!ne2d4ZAO& zeej%?j}x~uf8~Ph3d+yc9SqsTNP9s-Z=%>E%vA2)R^SnOT<))-un3^D{3?n_h1GXVZhxu3}=y$z#%KXtm;n_j|qL@I7?8s;>!2R=TSVZ}tlpB6vtg{NjUFl^!dxS3jg_)KB27VpQ0D=+AD^Pt@8iciB# z-FSRolCe>547eOagZD4HK5ZRK7v-V~y?ADQF4H@^aL!gNNbvacgszQPy-DB8*#!po zD=`=E3Hp6*3}(J97G_04F$P}4xDuB&nw_d)yQ&H*glNa+lO(8^DKCxOzyUKQTTfx~ z>S~0mm8z0F{}9TZO)@s+FC9vnP6j~;sy!n@1R4=_aW0hwW0Dw2dXW|DTSv`+YZDF9 z{Y%3qfqYOVSTY6azr|!mPSNp0>wZmZxB57%RxXrrubCtQzLkJje}a2|J4J5}6bTxO zvSCJPC^gf|`r(<51EH<|Ayqlc$!y(aZsj;`_Cg<)Ybucl~N5y}|D z$ZIHCcrytQVbYCPza*g%r5y15R4|S@tcNT~yM#j?4V9$xiuY|4$+@TWb8K9}XN$;B z9yX!=2(2H>*i66!jAR#%qnRFWM6w zzIg9IeZ8+}T%a?*45t^dlcrEI7pwF+dPuShmeMB?h@zu9hg_5H#q~3h~D@5aKQHSJQn>-ED@Z-kBqxckbsYscq1ao+r%|6FN0%ZYnqqdOC8=YDGhJ#L}l>wrn0mgb;7)@Ubr4FaCPw+;v z5IKH43iw)-2G#1*d6e(IEtjG@RQYDVHX|FRrZ-{XG#jU;L4mh423AkrEC5}Fi?6o; zlH6i~ySsLj>;5Q2%chr1c!<@4D2E9j&kX8X(uOVxltI$^dAaEk>E0t*n+!xum+CDT zDv~R2_nM`X&^2#LCBpm6et(hk_K6ORY;BkbQnGtS=PtIiVNc@^Px#Bt7FR4$2KQ@I zc47~H(_sWH<`T?r*!8QnIEwuw^*Z;3!>1kF!;xwOh9zrRNX`;D>x#=H2d9UEh$u01 z%UD2XA5j?6>ph27J@qT>6T+v0{M61H_z6NGPlGaALs68~3 znrNWzcsY3hnTNhL8r0A0f&8CI-{YVX$p*;_0Cbsxlj-xo+p> zeC)=HKzISlR$D-gVM)vVb*B4xek^MMRCHn1*PVgYmGw7Qs}V=p;}-~yOC~u!OgX6NdXiXjYUudrr2>>BGr$dpJ*2J~JTx^FvkB-y^rk{RlsedP54 zVhkoZ;<5_)mASrIUvR%0>HXNiTWzQyk0cfLx;zQuDwzrg_Vy;DH|upbJJ@1c`G%Rd zQ6?F!GHn&Ltd#v{A?%4a2AO6HlOlIp)uk7DGo^)A@G+bh=*65j z4(m(e-O_Vrw0Cu(+zsE8MVO(syW2Jnlb?FjwQi_|Fj(T6Ney?6)d-{~=i?V)yP6s+ z_tos|?by2PAebEr?Kr=rv3;8|GCXurxTN}vIfkCrP!k>4x{9bm3L8+Uo>*c24&BR|Mnmu=XiJ?@ocV#O*O>9V4`p6IdX%n}_ z=y(Xf*^yOVqIlsK8$~<^CHEc+*SwvXUIDTN2&oR{XW@hunJhm?guP)Qp9Z8@Siwto zPO*Kc%I`gUQ5-h9iNJ;Rg+0c)Ay8i>Vi*Z!vmLi@wc_aIo0x#CX}%X$wHqO3U0zH+ zi(L!mIQzXF7N!hoz@JjR&e+N>g0-^pt7%-LYm9g8L{~ym<(C}v>F}mrN9PH?j(7dI zk>uEwVirMS4aaXkHg#yj?#dS1_}!w!V{+GmD)3GE^i_e1vYf9R0yzRN-*M=7Vjt*Io{n?(V*?BHqoS z5_&x*!<1u5@P2V=+s?TivmZ)EgP*WZFuK7+)q?HtP|AzVkE2m<)JxrD3bwc8*O<^s z7&ppQFsp0(t?V5cY@LX^P=5M4PrL0aVAcf}X?Udz%Bv`#iV1fZ8__DRE6$zzIa}Vl zms~zV@cw$eiT&x2-JRK)`Z${Pfbizl=PdR4)omvGIEJeQm+!4lbaSCqXAGH(q(%EMYqFOKisU;oT8WPZDl}meV~?SrSvNKk=JyG=r6+L zzqX8Lb9>|kVM{{7AtcG1XKdyy|P4X!a;qZFlm8 z+tvbE&e`CN98wiY^btXst+9s4trBtiksgyalYOmux@(=y=8NXus;2KvtF)69G&Zw7 zCg>w`v1bgZg=$M~cUDG)wJj95iFU0|^CRejrCtf|aB6x?aeO?ToMa3yXO-omvlNx; zq7`eHVsytW*cwZjr))YliliU^<~G2UX(s)v!2hpL#IKEkDlDHYF&oZNNM4h-9J~3> z`!;ca>*-V5`o1h1vS&0^dhOhhZZ@Bfy# z+-=2uT8EBmTD>qf41$YD;5>E1>eUE=TZG>yF1pc_+%DR z#&wm<9dl(q=As_Ke>p1fU`X#2r#dV^P8aQnm@SYaXX2b=Q6>+mrIW>fAnD5sA2vZ4 zENeY+d#sq)H6|EpnQXLZnvq8De(tjQ&)wC}PCJjJvt^%A8|$_6$mjLxMe$Gdt)s}2 zUJ||H-^7xghI5<`+kbvwFSWSU7TBxzHHf02MGMW*mxrskQc?2fj!8=GafG*Qn5K+J zCH61O^#77+{imJafl`Y+Ul(mS(WZ|I!EDK;r{;k=W~;io+9w({L8QXVi*rJ)arekP zw0G5$L*+3o3@?9KzJIs8A1x{dk&^(=D}NdQ7h0FeUd^2|ye>3bh-#X|P=ixJWZom@ zMOmJi1npl$C5F>phDRMe8tp+skPa=DZzZIV5{AwYER>(_ya?y)#b# z1#f)w{T8lbp}KVsqVH>GUyKE zC(m*2_$t)E=mcysy5$q;B5MRedUjD$>h|*cV^Uz;?GKz$8#}4B*IF$)A@sufp+G%MxhDw@ zLP93qh+Xdsgy0wYZ39VW2aUGVDbF?RgoVt6B556}HCtz;Gw#uU)q?U5iHhZiki%hO z^V=B7H8r#@d=jYeQ)B{8FNi7IVU>ap1`8D@+EbeYW`V`(%#XoDid5Q4c0ikGeDOEk#>PxG+N`)AzPz zLKXSISLk|5{^_!KG3tz)p6mk}_I=G);yzQw`aDe4{N?@`-Wu@DO{alU1`7w*lY&Ru z9P%(213?>(Px<3yieU9bzA5j%D5@1tFzby)iQdf%OfmH|=vp?%+gSe6+4-&Oc)DOh z<0x3GW#yiNKwDA(r(E#Qh)(XIvhx_Dg_U|qeR?z=d+0%VsdOlh)X5^D(y)cXBAV{J zSu?nrm~cU%S4`3xeV-A8Y%no8bG#_V?Vxz`bKXWH8wJr|S+vxc2gRxj!hj7v9XhCp zs&380%)~m^Y`hMY?sPyZOE^0Ow_p|MAa~mNqam=$34@vuX)@iVmw$8#nLo9%pm*gX z>-8uNRoNUhAwKjIVTjsD7wEpl11lKY}6EIus7WZj% zYuL|8JYVdX(TkTWi*Dxng~#|0cBIxyQpOIHFt}pDLU;y}iFE@v zOY>C?o%2}%JkX|{-~OBPX^8R@rb@3CF^K%DdYXD^#UST z=jqK;#E^3*oiP+c>Nx%AF431c#D1iryLstqin@bk`dxxQ`8s={g2&JL!yi>}^g#K& z#pqfye-$zK7Ki)eoK;BOWwF-#z}m8~>r|INu^$o?rCY}{A9Z?__)`kivQMX)Yick* zU!80?SM-Xd3LAgVnsBc2|e5gry7~ z*^WmPwH871Jv_VH$ML<%HJdt7b)poUSV!bAS`+)2Z}zAOiVAJvB;!!{qM0=aap;&x z)jq0D4L$%Ga z2e)vUx@i5-S?XKSmurzaKl5Z~aGX{(`JgI*-{LZf4tO=4Cij$PW}6=_1w*!6cn;HT zPM68^80#5!m}7Yt2WFxZIznG`^Juja18o5#itDEQJ&=HF=7mBbj;x?Jx>B$HOV{Fy zpX#n{)9xH^{4Dvna9KSig#N@Vum!(k9-Z7|TS4jyo!ML)+eJ8z!XhXw74 zMOK@V*2~h#bo%AS&_~^~sFX-7KAHAY6F2v(hX(}JXFDf+bZeAX4D{XBe!WNYIl16{Ryt-g`^sm?I3oueL__KouJlZ zFsOAiU5Q!ff4EWj8Y*SkABa)Z)Mm#H$&U99nmPlNR{I|JyLS$#V8!^fR0VpCmW4(gUR9GJc+IOM6q8)(JAs^7^Lpi(I^>L|> z?A49YN$J%d_^D$iMuRTNMmGKjfwk=c! zU{%QBV9^_WrcAM#K4li4l%|ftIyPrZ7?S_jxadJGU87E&ekn7dMz*h%6z#UDn>!_t zKa>8G$9M_j6(N(i{b!udTY~S}rtEN$&wkZn`L|&0VNtR$fqYm2E3x#(e{d#%A>)A% zx2pV~gt(`tryrPql^yvn|Ce8z8n#fgDSo}aq2V(jO#8S0XlDvN{=h6Vrmy18e_)y( zBgcM((7YCfSLUjSBS&#dp+c=ktr7LrFE#W(5ebXujpr+1buS8|gl@JH)qggDzC|6m(Fjr`qUm&Cc6(%*b>b~a#euFh7a zo7Un_h^hX=Q*Bq!+{}fuy}vJ5#!Fqn3H#E5+mKxlwcbU5j*d=VK2P4I=|0rvpQrra zr|@v}<*>2RQT=P9)aXIMW(7f58ShzDcM0B{*6^k=a&UZkJ25iCR6Juq`ztDp|Kj;g zkkr=JPF1F@uBq8LaU!7c5jIqdI19mGGE|V(AdeuD(94qb3(5S`Ln8Xb?=3B$_C;|* zH$qf7NM+>La~F-#IwQG0bMu_vG_p zVk2M?(YK1g{yBN5g z$oaGmG+~|bifyJd7D!A!o!p98&JJ(_L(5&JrXxasBqe}OzQaGwKaNzf#j1G(*_$c< zLp5dfJ?f}>T|H4a>tf&^vgWHpA+?u}4@P5{QTheZd^CTi6F9^g+1*pgch*$HcXsuc zIw22o*v&~la-LrC2R&qn;bDDO_PRn+qUj$I;S@gv!cWao@pA-rRnoDjaUxJjst1AR#%Cw7K`Ty!Y4?~!#66`_czW?AgV5|Yw!D0dP z=Gd6R-*?ypmLhbb$)%o)X~E%v5#9`YOA7PYT6LTtih@Q$>i?k^_Xr}C{t4eDD4$OJ zzf1qSa5Hr zj3Lf3cLTYg?KS`L$bB%EErg~htOVV&DNJ5W1w(Igwo#io-Fr4D|A0yrtaA|cus()i z*>)QhP#W^fwPB5C*_YVWUq%0w-B1GM8w0L{XL#I0S;YV?#C^zA+_=S=;@=nZ;}u2d zy%}-R!k1uHnn0ED!4G-yWz%1R|BUv%kn~&N+<4$%?E5Z?z^sCZN7317{}0I?pi=o( zf{h9zr|>|y+j>gGNaWrjnetsm>K|7r`%|mePyDhY0Iek!#s4E_`t^@sQUa2syg>e+ z*tmC4+02h&^}ajVjj{h-N`8rYc)*en>X^4l=M(?QhquHZtnCZ@vpBT1-6II3PmiDoAN}Q}2S52I#^@d+-!h(Gy!)3AK8EQCdeo*oCq0V%_y6#>&-~9< zydy?^hb6XJ(LVLJulnx~K7v{ne56BbNx22{-z)q3#|gt=2?NQFr*+K#>kC4AnW&B)k0ulaomn^{(E&GHA|Lex1#DT*kPQmYb z@?Vee4-r9P2Tx?pCnEIE?fa`3{m(z~69v{9Jk$x6 zpDwUSwaRobp0!MZL#bF@L`0-B0FyzvRNI`Be{}nBsr`Gl#2dTKP_;ThHW5XmO`Rl<)T%&I2{#9b2!k%;0T5 zLyHUG+{`dtR*dGbJD)ak1l*;XO&_N#x?d%(0G_0h+GV(l)N=WJd6AXjFcP633$Z~q zh&SsAR+{d2XU5L8yXE~K&v#X_#p8@|WeNWE+Nwgx1jyFeZUmoWS>6TVvLj+K{@5-k zv0QAvqeyWncfPQ!ksOHDY4`CqZE(3d28>W%Gq|5M%9mR`Af?{yRbt6A;qRXeb016; zsVz3UwttB2WTcM&mA?PmZLft&$LN})AJq*bJZ*z#Nc^~+r`HuIBF0_4R}4lHBI28} z&$opV@u};XW4!V|+Zzxvoq4&!jen4PgzxSr{e0JEM!_MaQYKW#&a{x_-fasB-?Ru#|gi!vnTO@+q zyQE9=UnJ*|M=UI<7o(xz&jsfLPC!qJ(yN;sXvYf8_H0o-e=3qnu5D7p{s7dtGSqLT z*TR@D=*_#F41$4JXR4I{A{xN>1-u*jV2>8#258wD+PvZWooT_^ne1u}2jhXzk&OGK zxLunxIPBT1_Jo-Zw!#4_HF>DcR*eZ|Ak6R#jio?-z_|@1bKWF#I)ig!x0tB}T)ec6 ziR2dINO0)X`VHy-rBeT?bCED1tWmr6TYlan;=kUVZs?M(J#nvUhl zR2U7PE57e1aKE(%qeL1U5BB#($B%(XXQ~RJuQHLt7V&z+g~^|fkEmyAP#vcYTe3@W z-<{%GHI|7+QBUZ?0XkBP@7X~|u+Bj8bEH!C3$)qD5;%X}o&dI3cfE)wkT)GLf0LtM z2pTAdMkL_EL;v#i^HZ81X|t<i5C zy%-UXgOH*Ku%phnwV{Dp-65Ki#+TyW(hb+=dtlcOp6rj8YJvNOL8l6}e}?UTx#a8n z;$x3dWdd?rc)ui`{p{1wY0 z7~%{3qp0M_zg_JD3bN2p4AVzxyKw1PqUileZ!W-PL`kF0z8~gT39MQ8NHgX^8c3z3 zn%gu*^!Km&@3rU>BC+ex=IhZ&{^-QiqCF^(F3L;IA$NSFU8e1(iUbR2x#;Y}(tF?yX!mfoU`BzWF z9BMgQkJhKXzh%2>QnTUie0ILt21l;+D`3JLnnyPd`^M^W`Q9<6@w;dY{d_79q|%d! z(1}cuh#onoBDI=*FjUC?IZ;2@klbPjM{o8A8fhRh-+_!R6=dTXMvzNvPZT8#C&MJz zAjo440D^ZK>g*2QtgwEdo=4IJfcn-5h1d;*)rOu0b-T=uXhG<-CzimcEG z@`^p06?c4qWZ5d1A}PZJEC@{&vsSaq!fQ(v%xxb8+(wpF)I4iA;nkSvSqmSPYO6*1 zA)FRGN@ztrj13erQ8X%9(j*R}sWA?u-)%=LStvcuXXjYyXt0RW{rQ`$7)=`*%9m-Q zpG54e9qXc*^tyz_kEg3Tv}!Epcc&{J8n}xY&p{#L`S!RNginb=?jX>^(EIuFd82@Y z4zrd@G|xi6g0 ztBP|Vb#m8t?-bPHhX`n)dY9T%@0;v~E$8kc-cEXJX5 zX@corQ=Hc09?S@4QcXEMEb=$jUDo*Wc=h*Rp+_<2H8ca>l57m8Feqwf3#hWpVan7#TcR%Lv~f?94|zL zBNE9|=p19rrhzuX)N)yxF{s4r_d2UvB`oC;+LVEf2OZiW7!0{ z-x&byaw|`ECnd6RPbg6c*R^qe60K(AmGX4{2Z zQVBJ4Hew+!IFDC*^&FuwRuRvD>br4x(DZ_*s+GoJeKry&3f~MVd5mW&O&)rdAQPq} z?RXL3dc_>)?PFYec8gckLdE7PhPlih`4IR;>fLR@x%}{WYL`$OlU}U$D$%>Q9`k-3 zWNSD!= z;l@x>P?>z5tPP_LjSFB*J;t!vD;7W$w|Q%`XQK+YCcmnPSZREXQokC%aubm=0xSqb zLXVsfc8`m@tS+e*Rk@aSL~PLLEGd^jy1&zNqWLvo&u;GYAur_)sQl2~kwW>v?} z`c)0YGPOrHLFcLTQLR<+t4&NjFbwCadPAV-7=P^vA!DGgORXDnK|UvZ&gcZq2|6e| zM$W}4Y=q>hLb&fa+PGZDNuN?LWUHsz!CpCdFUr4WNxGJPA?9I`l}5dr=@cz&!= zyD~|$$;s9Ty>KEe6)`8tLI15L68mxK9B9G<$j51A(!Cs&vuT%1fp#pOH=C|9Q_@bR z@IzXF@s&tmJIi|cGO5S6YgSELCB5zPXa&+y1Pi{B08=l&FdZlA?doW{f1KXdm=|&r zl%kt}4mCfAHZ&U^m&*sQC<#_4Y-&&4Rf$y^j0$SQ;NuQdDgHM-J-#ufgBOeL+=dDR zcJz~Ffa+TCg;%p4uzY|lJNXEEFnfOy+`rLzvafHXVUz|R1_Dq~^;&pB|IXV3-U;jy z6FECFtP+V};Gu+wB*kb`TZ2B;QUva#MQFy^({0+1% zVcsLbY1C?eh0x-&5H%@KEQ&W9hQQc*HyYG->2I>EQEx}3RGc_80(@@}T40$@^c{?s z8}z@n->yX;+Xe#bus4wor2%4w2#UmILF#Y1vAKRf?=M=`23Aju%I|8u86m5q(D#R< zwVc+X7@Ku98XPQuvjl9@8Z=oR>o-hM4U^FDuy_^|-G;%~wgcc{b>Uihy^7|biA&js zSkr@VS1IFMBRB`7qu1(>W~ocpQdC=^84c_9tVTAjq z=k^!>;5{*tFpQ3PD&&cYh+SA0Asx|JTxkD@QHtyOeiVE>FV)ArKHp4vsbWU*J)`Zh zJp9*ohl=M>V0a8>thV@aDDm+XwNcQ68KN51{-GWTu?=nkjaqe~V~I;OX|C|)5st;x z*QW%Lrpxv}zS9eAMD*$IT*3iKJ^&UghTGlEOsR*T6})zQ^8AH)(1IuL;n~6Bq9dA> zw|qaHYGo)Q#kkPxME+Xf5XO;H`yRg!$Blcr>hxKe8gd$VCI|I9^hzZPR{0DqWsOcJ z*{@?iC`|CB6o&}lT3rd=2CKDxo=jFe1fhFaq6rd~YUm}06yHtT7Qhp6I(wvXz z$5l3#W+BTp;K|DOUlh-)MymG#MUk=?=%|!UFDB}=S2YQdL#QFd^^QwHoHeVEGCyVw>kHI(p|wfq&drVPb`@m$2F%dU8LS{ZJ{M0RQ>JPEn2Mu}tY# z;3muTMMu6x7^W<@Np=Rhdj5r-hluaaNQSQxI`cRVi}4}wDQir=cUh202sfxZU#kcP z<6%?om`=gjQE9Q4P#gLX$*35qBu9E=j9zA0T{(zsN0_A!8?)J;TU3f?LE3eiD0~-D z%75iZ>VdE8^*b!3L!6Mviz!Gm4H0q!$*t(^7_swgwZ;4b8dWifYl>a_BB{(K3a$6@ zRDrFGHpN#6-Om0L%5)12XESQj*8d^w8I=ou)+e|~SyG5CDQd&qbrgBPMCRSvBwN(z za7IA=MYqN6&UCVPuV|OXac5#r>*sY0Sm`1#r^ZWIt~)bkB?wV7q9iF?uJz8fz$F%g zh&5x!(5R992pptk#n4Tx4~FycCN-Pyz?_%r*$^25u_)@b{g)U?YYCS1RE&?ZGmW7! z?Qs=LG&v&;^9FQe1(ixP@$)@1yIW~L@5_|_wn*d<9A}#tARH#g07^#G$U6f z{Ly5gv92V3ZY&a6)OIArF{aYLRu~>-0U|@L?<-{_e3&B^@YTcBGnG9xGy3}Y)~jwo z4C^1URTW#VUIo#Kr8W3K6c6|XY?GkTPu>IS;xHo|xf(FvMH9`X~or!NEgfsp_tPp!4RsEJ0Q+%6Gt0_8*YlUvv$4nMvTmkJ*LhOEVFJ`vvT0j~OwGJiBGVwE0=KvDzx(&k|-Q^sb~eM9YB z!OlWCUWOW*M%n~+tLX07+1Ww%McSA*llv9M1Nd(o?7W#e7k9=BVt9!5=ok}I32EG; zO4?f`{5-)}3tyVRyyK}dJrc|=%eu8_#{Afivt@d|yvWyGgpM?1%g5M|>`~suAp9wo z&P21sCJBUIhXqbX3R+?7pRny)M2xL(ZQ1a z_mSUd)Vp+eJzz%>5?j=rfWq6|AARixCg!Mvf#?=d;nWX%*$ij192Skbtvu?lt{~0{ z^(6A-EOCoXsYn%&g|R@(M75$WsSj~p%z?n&Sx=ifHKjv<1_>^^adU0~06kRG!&OgJ zGp8w~(-P{%P2=8%La+*GamPp6j9dja52Dj3*5+txS|baNL`vH&IBnFmO#xUXa-0?= zZG=5JmyPV5WGv%X3mecICh);nK~7(cKs;$MU!kCW9w#84a$4D|Elu((2#1%O1d6W}pu-}_Pz=P=5d;9^Q|6xubk zwTgy>(cx9D&6y^Qv-Nov1MhFACIXiDX0YPS*_%n%751ZfrAEWa?|l1PpSgJGecMA4 z(HGO21p{Ty_UF~8CR`jGPr$?+InCFT!M}AYLM5S4c(EdGCQrA%XT+!RQ%#nMaQ5Jm zHyWoQIj9CWqB7iFu7tpCVdQRljWVKLGvqT-xEUmU0lK?>B@$H=?~GfknaP&p)pLR3 zrZ{qs7jE^`=lk=xevHwkng!AiJ6t9-NDEmU7UOfhxcz=f|j{3dhh0%*oXjFqR!oo6?U|NP~Y99L> zt^}Cl80ACxoLVWvrU1@1rzcV%jmG8u*v?U;G%WE49Fa}2YCK`S^(|Az};$4jwLdBn($9nvIs1{lIs#o)g5 zPhK+PA$n`lnz7bzy=pm62AE~@G$!BKqJfxkBiWhwm7Fymy2fu)5D$BACiw%Rek`C> z*i0;%Rz|e6H3BpC18r%xnaj_+i{aNhv`YRFMpAjQS?UzbeoiRAqzQM#l%A$qlcV*x$v$U-?aoI^@a9^fyv{vz5}`O;~;#jYJ+kc*l2Fv#@7YvN`@ zT-qWP`>Jp$n1Q;=t#e7^y;~bj(XtgE-y4`TdSdnD`a|d2C$IK8=O$*Q@1Se~-dAF0 zJa>AU2xH(kc(hF5_8MplxGev?wEZfe;orDC-}U~jRZe<1c)1)&ghXW^QS-_Uz&Coq z@m^c|t`ZmBZ)A_Zm`=2OZN>omwViS9CpV8X$RH5&&NsU{|0srHTJMdZ2qu{>)%iS| z0L43rwUBkvYHo3AV@`K;G-d0Su><&JR18OhsGY_TZ|d)g(&R zyBw2B&dQa*YqA@b>2|h>1B|mw3xo!-a1Jp;mw-Ot3ItxY*EL0azlF<@gdnFLfkaHi+6!GvIXG>>Kp4!6K1llZh8=k15X zhs*)4Uq>d02p6RvZs1VN@QP@{YZ5TBFS|r1SHgrXoOHg)KMafEIq3)hPPTycauJFH z!|94=J0@-HB>ETz?Mo6logF}k=Mn)o7wPh$mn=rO`FL+U&m6$M58f^3&y$p&&*spL z$2up<^tyY7owX^jehV}qqi`g*#g2JD;LgRk(b(63iEdpkN1uT6quq}V%jT=~2ZZ>& z5~bc`vBvP5gSURLRFt)QXtSNva-YX9nyoVfUA0m!C}C<8By=A8 z>fwTD+l=z*tOtjgO3>uN;R7aD!dMagVYsGN-%@#AQ&q>;sMR!rYqaQgGgxI7ihTZ~ z+MdpXMKB1b%3`(}Mvq9eP@~Q!OG)6nK?yt+Z`26B>?%fGgUBP>aV_hIb0JBqG##f_ zLXXJZqMk66kDmeF9OehkpMdjUrNzw^fQ0i$KQCH-M!hk_SqZD3*Y)_FTB(I9Z^#ZT z6K4FTTDKFd7fe5$3cQ-;uv|ZV(iE8l)#Zb^x(c)BR$<%{D_ueJui-E*z`-`0gP57- zF4inYpjfh``8uB^t+h!f5C!__Gc2TR-mU}LZ$0k@4S%$Hgo(cgL*6_-DgZclS3R<9 z06{UYsg2TzT#e3!r!Efy8jJnb@Y1dA0Nl4inonD`9+m+f8g~F+1DIZ!3*24`oBf&? zA6y}UP{tU?1-a}`&%(>BFg)R8Ps=LVEKzTN(mZ_+4;`7T;UrM507y9$Qm?b#>;s<5 zz`6PF_$U~^js3i!fU0imQZc zrcP~51b$0LTMYizNCvyA zXrwh$R{|>Q!L+ogA}d-Dtt8te0k_X9UwbdU9{>^#^Y)V}Vyn<5R((ZecfA)EV2CMA zdPbrZ@d0&dHi}tq7S;QhxsPaS4DVx@fDg+9k*1agE=cna=h0p?&fy}eW6@O%%?cTo ze(BBPy7l;Xj_eojkX8BlVhYq$9zQt%Fz(@sJdGcxWrwhaO&C;`-E*AJSwLDI0P9)l zBVH{qGK`V_Q~kgTpwg69ITf8QI}v&b8c}*u-XynYjw15UGoj>Z~U&$%vB$u}#^F|myv6%X(tT-y( zY!{4q4duT?;Q#&7ng}Gic0qva-~VaB?Pjkpil#wBB9VhhT0Fd5p-`Er#*GwX9zF+|HA0MI6J_hoWZwW_NP)@7 z!wl{qIYe)q9LdV^!X_}b*Q0w}!FtxB2HVmDec94{EH`XO>4(6B*8x(Bmw*n*YO*+K zl`_ZzSXdN}2f8Tfb|ihJ$``Drk#BkC$+r*vD3TAyF{Sd1({2xwU5-UqW$f#3VxK_UppnttBjZ1+u) zRu8(}U3<$q5+6u^#^!jR8kD(VR%^ebI7o)nTw<7?FzOb0LGyn9STXW>N{O$rN~gC^ zS7=xNIC}sLJK2#YU^X$?D8x$e57?ADL!$vz3;Q1aCGwRZW_tTR)CR}-_Ne;-x_+|~ z$PD4A^BcDwU>0HT(mxpX6p4<&Ln<&0zMn2P@W_uM{SbX|x`11g7B1Wi zyB?*a88(G9#i*Jws@uCLSyz>A5nFr~RYLR4gf{;(HL)(Gg7{s}S20O#lVLb=g08$>C&zBw`&1VRVHPsa__P5%T7AwKNZb^gOj3O0`!FG zWoIP?@~?W)XhtV*fjUq~if&x6)vVShtOqLHdd5htOmMkz5f3v2LOGh7K3oWrP? zFHf~rj>Dwu_0|L;re{z;@-+?P=dg3?r%}V+WHVOGKBlT*n8`~wzhU{bd?G%Ev3aBibIOKySoHQaEiM-6bhwyad+1g4Ni+Y zMT67g1ef5DH{JW3-+9*F&tCfjocGTpbIy!8CL^%izV-d$LA3!F*55S4Vc0=_rd z+fJ9}okS6HD~RJRx5w$4THxVfMr*Q@7rkXF(xWL*I1*@aNxW3#s5@+xiz!m{K}k`D z>Pv7oDj<{*F4Jd!Se+9#)R-%yQZg?2VZ%sfKpEJWN*El3_D@t5<+J}3b4YTgKf(UW z3j4#mq9RWeaTS^h{$?ChMR%{%rGXMvLzaTK z#um!ud}EgR$7xJQi;}&{tCDtq{-==p5BU5g=A)4@mN zL&WfDzM+D}^Hks%b>QvOnm!xkeP{(o*mt&Q;|9OLtU zL<~k`KB`5e9YnFOR7(6x)KrmIP#8k#-RI&@oCN+jVBb`cK{R2h>gK1L=ft4 z=;?xdMKXnK4nO0#9*z*BnLKx^)I7b;9_`jLCioxBgg)8^p=+A)AGqc#!xItmx+lQ* z>MGCH<=D_#xZOU{%;gfk{UcMU= z%XDSh1^<&fe*Y+hJu>*uuR%Rj$9wzk;oOumkAt32f3sbE=aBG3kp30=zk6dEfj<=l zqLNWqXm9vTUMLy1?S^Ct;)IEXJZVyC7eGC-DTZ`s;>DL!tZE;^Tkow(sFGlzE}-RV>1{%ll-$ zSPOL;ZN`TrEz@nK=Dl|FmC}Xb0h>iLGIn#lFrwSx!uRO6U8T=U{2o^0=~S&vRuF^ZJKI-q(pI8(mG#{T+sn z>DoOtGlIS29rJ%A=v+`VG|6w~6A9zSpa;28|EyG)a7|BQ{VKTqJ3)mQYP^2DTX9Kt z2Uw1g>3=h@^hDcX#SF;KnSi-?=`U;&itoyK*+VG&WHRXc>@xbwgZGcEtXt|`LCY9B z0%&!QPNkf#dAmRbbblA|gQnekw#;z1l2_2;@5NiBvS=@uq6T$FfW7aF#~*ZyzeNy0 z%dMDkDZj=&*{Yha?q&u-&pK3#JN$r=CPgBqaC>4zu}vB{wl2&9_c(xor4%sZoOS~ zaoIqWrEx!(6KgL((zN62wY&WwE_!JZEzkb)UnV(f5X7HAWG=?0NoQxB2GEHMg8|Y2RlQI$|s6hx3xP%=lz>s z3--Q_0iMA^gkugG)oyWS-q~n?;U>5BYSloabWYhqiC;mV0+%NnEP^Z1Y>z2o_0k0` zJ)+?^W84aOV>mYHE;>Fa*Jb`)1EXct;uB!ZZA->r5bKylLDxG;%h*hC{J3#g?aR8A`<4fnjs8s1gk=lL_K4kZ(CLL z`guDpy8cRFSDYC~HdJUde0a3@F0|OcY}Ug0YLkdgg%;zqs3RND4sqg1@2>d9+6q&dDZ2g@yqxk9BNMj-$7{0JqMf z*v6h7QxP=OLRro=oqQ@gb8ch_A>nlVBvN-CP5T)uw0?X#fXzH|zgxqoI$evgd|bPO zqlhdZMEFDOX~Y=0{ikp1YFzt}K2{w!jL_XWJcMn&-i+xoSlbR1`fVq&wQOD(ct#AKc{o=FL}WE=2XCs*)m6if~!q>oibpfntlv+dBS&ei_)RL z*D(a0)4kdCG53k>UhKr?^X8RIDq&vCM?Q-(yxXCT)&TlzJFaj#bw#KdG<#GP>Ce=F zCVF)3bYgOj=pbAo@*vZWvyp|vkJ086Eo4Vg`c^|#Hn@&jWjmH;DYScBfz=P!c~jUP z=FHP>*!=-A1+}8&Cxj#-{?QLFIlrZZjo10wrZ8(2Nln9k4(RQX-`S<}FE;}43O!Zy z>*bMngDHDO73sW{sT<3_q{BmyM$;*WWe)7p)9o>W13A-mUNPd6y z+c7f5PA&kD)A{5bCCvNmq)K*Ax(blQN{lGr?xsTx{JG;WymyqZpLvPPYEl6+7qlC^ zoSr=XEzS$;gUn__2&*EaHMoMtD6LB~Vtqvuo*2(n^u{~xJdM-0u`Z+MK*HxB?D}4o z)g!jmVR7ZCq-r)00l@X%9oizgcE~(^3onZ3rYEabd*8{Lw6G6c#D^?0hU+ZYQ;DMM zXIs}Zcnp;C?gqwJT2q0Ef42zHp(+t&6?{m8RWxUJa+wW1g z?{?n&EjIh|Rm_K3T*k}GsKMLT>D^5oiO`|Ih4l)6nucjZ-QNBzZG zAY$zyh&=XSqhfEs=t~7h5t)=V>a{#X(KZb11%%e*(vmyO94n4Q z)wz>nD^t&s7@4fB%ijNZ2tyYrDGDQ8fCqW{BEvC)?HA!udj{wR%tIy?Msn#5E{oUW z(ZoKhWua6ao{eOE+G?dy$N)siLbgW9hlK3W_AI%oxJ>HdPF6d$hf>XAs6_Jn%Ukye zoopD9&eYx3b^PQ`3-x~KmV zmIWz^Q>urs-;>U?2C4JA>ve$keLV*DJ75j9_|RMeX3>_as{5=0I+q-{Flsl9d>8o) z1*DaZ`Dzd@k$mUZgwt)7-%{AFt)pzi(p5~vCQ6RnaF}mW?ssRtyG+sp z!{`Shg)@w<`*Z!RKCaBVKV?T0kl zuWS4izPjONroCTq^oM&4#gF=TWK{)=@ZGGs)M~Nq$v{{YHDv4g#&n9Cj#cMWHU?~g z?EI^uF3!~cglY7ReL{hwG4$x;nax!E+KnP@4qldb{m=$r>e8#!n0E8ZQadag=9S3# z!_oNAcbYnkvXHFw!>K9a=~>Xf<-W z=Wp7XOJ2j85fbjH;rlfPD;Qwj!x~PG39h;QPZ!3f|EtVj zf6@P*6@3Zg=0=E-@VEo*nmf>9lj3BqGcj`-7s%SMJ*~w@+jQPZALqR!(MMifMUm=m z^lRXk@?as~c>CK(P)eXdvkVoH(z^Uv<;_P}_9Da$5UJlnH&;KmE02EqnV&wAhmBUX zVzu3$q4{b|x>WWhMLrlU4wYTx8eyy|>zS9&Fc=&cEH3DoPZK5?l_N*#Mft6g!zbIE z4q`;{?YJP$g$w+6Ytmwlk3=32N{)^PHc5#QyMM}7BF%5Q>_Ch_c&@MD4V4f;Yu(bb zG4S|o*GF7=MrPcAm7;^qB%y%pFT~$6zn~tPuW!0ixtNMD6kk%7nGRv8(=j^QW$k0# zcifHnqn~Yzn5=ASEtR1nU`Fa-)}NhykR|{sw4PnO9FkXq`o*yY2&n@br{abt^Bfq@ zJ}Dztel~vw)X(unfa;P7%@!g?Rz8I#_6>Gm+uka<)7SuHAGZVTjm#VD5iC`DTBq$l zY|^b7}HMMB*VYyj*knQTs2HFDNTu*lapj z!W|jpwp4TcIj-04zkbo)(g@UqH6o198fr5l&Pz87l?5+0N8;Kw!qGqhwXG6(D1XmA z-T9j0g4rpxpwP!VvhmPLe(SL6UA&8%6U4+dSfT-gyS36%r*M|lCx^<-!zKk<<>HZJ z)H4d|!qv!6%5sZ;6IN#7;a;xOqFz*J`qb4@ka4lvAD!23RnQI2gxZV{vtiQTV!c+Q z5EaT+?`gAQ3|uSed;xR*{1af}{E{osP)YeNU|5h93M;K$cv!9I#A@41?2^SlP25pu z$Lk+ceH~(JG?q$|D69FH;VW-RHc5-WQcjgE?rP)h$11;Dq6Q{>)^8TW*(n`4wpcy^ zXBh5Dz08g*-u~gp^3hgNveuD9ptx=X!HC&=jL{O9I7kb?U0;7o= zcq`157h%BPLC|kcqgzGDTBobFG7{6LnZUaAJl7L7Hek?-=gC;G%f^%QCx(qCDIBbD zqEZM|FUD4!)W&U>F}XQ>$}f1&)#M@lz-M63HP5F^!lXli5ofZ?UGTi)3)I3zOl8{< zh2UwB?Qe%3eSuq|64Kk(q?60U4`y8=gYHj8i%g1Wrb-}z+gFdC391MZLgR0f2DN!A zX{s4&Yld2?_Kl}wMF=b^=gQF&EecDcFqv>q*e}Tyzo$4X7425!v}{I%pjC?`xq5cT zv=e{&xOV~3eINKlRrx3n8!Gf+sGPNrR`$8l?W!s#EE@!IV~8paG>~&7cbAw9OVNLZ zVs6R${I_RFpiTwe<4UVg1C#fM&{UOr_ww2H!o0(UKs6&fw-xu$eZkaIMhg~c;gGZ4 zJ$E=owueMoS?;)CXg`#ldGzPzLQ)?;LwND7rsrlDv;p&OvQ+Y}4!mq+Q8lxxcDgi5TYS759XiW%P9Cu640`baO;^~bD%HP}9C9y%Lf-V@Hg*{}MA~SBcrcL( zeB~=j=J#Cf*X=J7QtY(1QAIfLFtg;ghjl+9gyrLV+p~kXm#g>Yc{Pg#R{j@pk;K;l z?y|sn5A&Aii&%ae}zLnYh{1(!DV8$^QSw5mSG!Oyg#>pZHa_7{AQ z53x#il)<)I;z!pV{*rrInt$~VZTv_6W!O(a^%-Q25@OpYj;D~-&Wu#lF z<`9XZtTU3udlw1>Us$E=FI04eUwc}tyBGqM-qd@`tk<3Z#oe;@F#kGA=>OT#U|S_0 zsadUIHt}g^)E2Dp=rk)vaI8|;f=FxY^$4D_AdJ<1|8%+Sd~+48ffs6$Yd&0k zj|z$B3;clxNt#36qfByyQ<#Vz^Jv7IS+nNtz0TV;fMK8wBBNz`8dQ>fpx~7MIb9tU z-Mh1k9?#bo10y^ffrD)}x%72!``kuzU~R!t=BtEDZIP7kou#|&nU>~56-kgp;Vg#^U$c>IY_P$&`L3eabA)P+-rVYtN!o{+KPS@w|#m5ds_&!m%0Ao(1!bKNO^qj9j!E5fHsy zD*|LF>ijhKn+KS=+3(%x*tHOJ=Gb=6r&Xhg(oHK&DBM2Of4{6LYL*MU*^f0(hvoVT zz;S-Y&__SL1SF))fML*%ldKaFZC>}Xy#DFi0{l*Ps?FOxqECIr#sevW4oafqpvyTJ zr{A@LJ3WS3sA)f4ek{pymxI3E@{wnjfzFSxDZVKwt7-zWHNL3jTg_ej?il~66nl2r zX%cCZj@Vr$;ZEtNu8(5V>r_Bx_sJRL&{T=1a@6$<0qHdtB}Cl~sDVrxN9tNg5!YRw zI9k9(laQYB1$FCjuEXS9;uyA#hw5K^Z?8>dioo)aoqGY_thiUPcNuqzmEJR^H#@N( zHn7RiRTW-@1jz3xcEc^1&UdOH7JT+K1{XesgEiv zpE+t_DNX&c2bTi0dR1N{_T5}dExw%>_<*lqMNZJw zH||kY%5WmsIT^tYn9c8oxcw8df!^0uE9oGpEz|-Sxtknq1hpw9c0i_@ifxAHvr)EI#_*<2a6{OFXWRj#V%j_ph_nVoD~}ZnVrGgir=pM+{Z4yG&x`96zS{&$ z4Gy;+RDMg&za{T-n26kwr!Nu;7HKcYx-D)4gac}G81CSWL zA1Y#s99w@I*x!8`_S(;CZ}cO;h+@w_&0=zG)aSB`tO~94!q2P{1qae^?qxbK)WMp} z;BZW#{9E=s_SM@H+-v*RtPG`04SVkS@3J9_G!l_0uRN>(s$Cohi?Sv{p?4B~ty~Y= zpT`*WXdpUi0RcOtIg%yTMUhfR@BAk!{5P=&e^^WJmh&Q?PwUYx%d<`Kjh3C0ak<*} zT`rL|7j{4T3A1qZ+`$o=?w)GwY&1hSC=c6eF8cs$npREWMv$wyb)T0P6<=UhLoSJ) z(P_*1mc3+0sj2X0(CIHF4+C23iGGiy&Gv z+Y0S?(&|H9=B+Ig;&h{h8+RKZJF^?piy(FC^NV_5uZp9VcG^nL(Zv2nI^)Yq5Szw@ z#iF-Xl0CptyaHqZPhrQuZoNv@wkMS~zc14jGG{;ENgs=ov3RO==n#hq=8#W$#u#>X z^VjZ-;+d;KCmX^SXcD?lNlU`um%xLVEMm~}EoO1E*WHS?1@~pHh&du`3`E+P=tNp$ zI|Ql_n?5GXy|0wQAAw+vk-Hq9o3)Y2It5Dat_7L23h-w)@r`{nmC9EiIv%Wa49>OP zA)L0IzGi!}EDlx>K>O8Ed!sV_#VQYX#IUNgvYRGhfp$R$DRe~p`THIxZ}bbC-tMkBcrq6Btm0Zm|T3f~mcSYm8&bX8QsP`hbL%wA%?P!S=Ky8tIPpwr zBM`qdyazvLMGVut9bTDW8rD;WHFFvJmQ@t@(o$;Uab^wMia!X0;&`+6`n>ZjT{S(B z-1Lf{;-*b5;Qp`Ig|@ZFi17!vu>$#p)$llRGV5$^8yxTRQsgEbI|MpC1G##Ia@{UA z)eB>pvxaO1w5MQyxh|>pvvA`rnWWxmd^sqCxloetcHOhh#n%{NgA^wl;-M^SG0eIn(MRfLH^&EUxSi$ixi zOuPI9+YVDb%Nw_rUbnZqpW|Jt{1;`yW`TK-7<=1fz6yE%gIc5Z^Fapgo!SII&OF2B z0rdM1OPPXdziW@h*l$`Txq$6-mAHq)x}0YDYggHLRS;{+oUbzV#N(xkh+XS4^xO{Y zMaWMV=T>-h&BZQp_hZh}Fv;={rGKX`rKBk{2^6)+ifU@nGOXqYSbC&ZG6EPnd`4?R z+XhHvz?uL$R>S!vxM4&*yQxfs;%FVxs{H&}v1IRRf^7_`TmAK78v>N5HPT(m6h@?b zsg0-nN?2MGm!$#qapW8>vnT>OD!@Hk z`IoLjNut$Xj*58ZtE~^7UOYG2YnVto-5UFyEux+zmoPl|!HJUpNRH(Q%d30JvuYtq zs)es;Ib8q*hAB>61_%Kk*?f4RR96!E&O3}ym4jJm_o)rZS-;_82WRP5j)(@UD4Tpr z9T}~djoGnZ!{C@C3+7uG{~qqE{-}9C{EOBeHj+B6G&3#h5uq?U(UXsiP2YoAn&ZKIeaSonnZ`?W^oDABL!oq zu_*9%)z9Tq1mo$q4Z~)hn_xcg3@LD9t!4;T(D6Nh_&2g6+eCkUL$SoWtiw?AEAQk8 zUOz{EyC3k(Lw1E@Fblw6n5Q>3HCpXdjNwr}k2oHw(H;T@?hlmD*W0kyF^jHetY#S# z>>$(u1XR?gy9Kf*ivHI@ZnYaAX5ntGKTKWmX@|bEC*(pk0@#t}$(xtlN zor!%>_D!I4w@Pza1B0_va!(ajV!C8-H|Z!N7IEx!QeK?l@Uxu^PgsKtBiQgO$B9=% z*-roD!HB{9H&c!I!Zy`c3Nhy8I-%zXB^KG6%d{hlj5vmlwf604Zgbi@O*<$#1j*8M zYM{L@HAo&gOPix`FJCep$z5EjN!>0L%O|W`{{+J2axrht^r0m%DQ6c2 zix$fvM+(iQ z)tWckU(DXVThJQ_+?)+4=;m15a;rwAZf?Y*R_pD6B4+lN_bb2UO^BWL*nvu?u^hg=>Ud}5 zfCKX~81yn{-Py?r1V?C%-pgr(ntR>iWL+QL1VTlHPK#&Cg*UDSOQ3)BR>9ObFIYR<3s zUy7Y}Uj#UL+Us7-y#&an?qz)RZrcGL8d%Ppyq4eFM|OP6t9g~x0)no@H5Cs&lP<&! z1naFPIPPmmc4bU9hJR|%luPa<+{W|fcdlB>D#lyob;>Q=XD{8B{MIQp(hJKfv2c(x#T@?`%PNB)) z1&>7vZM$XcZS12UGN$RoqCwA-llwp9w8s7bpbKENTWF=;VOvnUm1iky<<249?}IE( zAb_m$yYE^ELpG>96g$mMR_*-p>T554T_fzr=(G07A?O?Or#$>I=FJ=o;)$Mrxd_L- zZ}(egfyC1*w*V2?S@3I^Uppp{$0mQ=idPX|Wstn2!Pp$9-a}JOLZl`MC!Xhw%o)XW zpED2&({J`Zx2^A1l*Wc)asb7#q6KeMC;M~A8qP)E)n*h^7J@ykCj`@iR5lGLR02>e zLqf-Z4kvR)gbqJk)BultKb1V;PGwHLWe@ zf)Fo*`woh>^=I>`!KB_23~41oEvw*$Mdj|ZbRD^e{j1xY4<(jiBQLniI$_a^$!?H@ zSPQCicc%8C+xwsU!r1KxbD@x5MSU#4Zhp=OveB;pdTYkB=!e2oGdvG?HyRWvrVc2s z)sH|~E2Wv4ljjo!j{Y~}Zb|dV8N~I)_)EKx?$Y9dlFMiVn+3`i^@in5sXYC5@7SZ^ zD5zvXVc{zCmhf!29!fH#BQly1yEUJa+|~8}?gh~BrX#0gp-;1+PjI~Sz?W#^5CixP zep*gVzp!GI2lqW7X*}SDeU|8HRaKlxBxt>B2CBJf`nUBRe~*IyegoMlSmD?BSINlt za3#v15caHR%sqfqU#;!#%Ns*2dumAKd)Ad9P(%Z$S5!0`zPt$l5GMsNN*nTz#JWz^ zt>FjU-C_eZxuluXt!P#5eb*Uop8Z;mvYE4>0VQ)(TX&9tNLvv?Y7KQoN`J*`YvDt=sahQ#(wQIe@_S1OBj zL0Yr`T;3yw`k}dQKnlOzyJ~RZQO;^xaPTMjF=+BS@GC?knj`eVS6x!rW{74}^F#0V z3iWQ^wT!d~heByU5>Phh`Hd=jzZ+AE<5E>>j$@4OuQ7?ea)F~_)lIUA;FNGi_-xd? z>9U)KW{4(#uuIae%{dU8BCDw`BN=vI+Dr&DG|D36IipBw-6HiKYZZPE>YRFR*86g+ z3{?BEDpyv#W$R}xy296BQiKmyYGIN;XEvI*%hF<%VTV5dA#z{WRK2dkxhNmzS$%R$ zFty1Ih!hh(or#JbvLHC>DfJmp47hb4yAj=T{Avaz*WZ#9dc}^}B=>;p*rhhZ16Le* z3mcgXO{fF4%$&2R%;#TkG#(ye)JvVRzpGlPNAQ;t6%@8A(8g11pv$S6-KeD09k zFr;Zi z92-!nPg@wD$?#~fVTmtFlSJ|6>AM}GcDAKliOCdt+SYcK)p|v!?f@%}ES5#byS>OD z77~(eeRb7kxeS}USYrdi$z42wtoQmcN*^Mjof<^rOC*ZH%i`j=g&VNR1{9vLhe|l> zoNt8$O5>r?aq(l5v}Kto5aX)p@v1d?GdlrZ#u)mj^G=S30uc`Gg@ zu@(Q2yPT=waxgiy?UW^?OHvss36XCZE`C2>8H$C2s{&1x|AlLFmo{8IBWCcsgWYH@ zUxMnn*^nLNOXcq)Von!krYz(VgZBrUli^l}BJ`9@G{h>u^6=`-$`c*1LdVD0yRsyg zmL45K0FY0$2%B+moJPHnl(85o|I~BQyRM+w*Q{>Ghe3})?E3hk*f`4K3{^DZ^KH%m zgsD$d!(|FKpA2?8zZK0Y7t*!3Q|LkL^hOt6-nnrdlsXzl0Qr8+Hu?vqvr6}T0YdKi zjEG_ji=Sb2AX2XNLxu@=LbQFA$O zcu2}wt3nOEfy_ep-H2S*K1cUdlttAX@wJ-t?kBKHGSl%%Dy^ZtMfZijQM6Gt^8Be% zP)s}PPpK8}wP~%{>iSo73tQT8crweMWph3ZN`C#x@N4DA&To&4sZOuowvGbJrYq|j zjZe<$g6=vGiZL*jd{$TMAl{ds8TZ!b_Gbv}?nBs#lq6?hdOSyh@a8H{hf1hk>5XA) z_V$DbTY*2H7nbaQ3w(O8`!=YKuV|wwD7dDin5WLqAIl7dWR-Yo1}Q)Ii$CVB@#U;k zqX~4_G)NIB%9-W$s8+8xCeI-M9`){W!ky|`1rLx)e>!A{%=o^$n#)HY)d|S#Ks^A(E;=Lw##xEyjZRhmvN3ac)YwC5q5Xq1{Z&dt@7~ zMKeO3%52Y?Myz#VrzQ{BaY?xpKwO37T z`vm{7vgHzYlKm8-Xj_gxG>y06=5o}pIkG3{0%%52S%LMC^zx3Pbk9t6uw(Vz_OR%v zAm(WjqaGjR{Hp2aS9++sAxt)$CeUAIPv®oKjia8b5nR!nis^mjPVk7Z0RCIGEoN49e>UO!K;?t}7EhgWb63@} z^z6kUBZiz|Hf6i9fnc9jW-!k!KY4NZGHnuI0o~#i7P>Fr-?xboTX;c}%vOAzIr~j% zs=T*eqN<*>$~IZI-XqMjM&sq9nPFVwPGd8Tpe2kQx3=Ia#g_vag1sfOyDT(pO_mZb z1wi{hC+qNqs3H3o>!3v2?P!W^+dKvDSllMFo~XnoOM?1Wmu9N>QrvdF`c>;FDzG@y zLW#kT5q&dq3kJ_H1f%{PXuV0RG0ib!t=^!kB}#puj@_2~21r-tj0LrFXt~*eLN09%eVzB^ynFSRxo(%iXMu%MQ{mx9 zg)V7S*KU|S_pH>cj;PJ}^m;utwRauj3BKH+ag&9f$8~@yeN_r>hZr2?>MH$Ird)Q} zH=8&|_Q~MAPMlAfpR8W)YMtG&Ctoxu-dL1;3C6Zgj8L6vGZ}Jg0OaO3$1)`|%0 z5hhYP=g~~oDn4WQbu-=uDDW$~^y88?WVK--xVRk`+(*Q@-zqspx|w=+$_Gc7WJXYs zh=>I*mUL8zin>*kJKv~0z>mFI&+WEmYw+87@*JOfXojA*_O{99**)JO^%jP?w-|Ml zHyfHoM5{JTh9(lshxpD^7~iO``&V=?mKX}w1!rOp!BN|LO5f;}9|axqy1LtTP{6mC z^%!CU-bkYqmhj)d$e90Lc1?Nk#PlvL?&5LSuDDA|CP?Y&3$u=N(CRB>aS0^A#}00( zT36nCzSR`aHt0QkA9RoC!Bc&mUL*;yjPGeHWhxQ)RV#m8p2$@CgyiN$32h@cI9}Md z9+VD`x2o^vp^l?Y55Rzjt){G}0ETj+P6yeB+3jtUb}sj*#Jz47^(>?tz;#+Z&03Zz zsdxc7o$o$x_EJ3PljX4$Yf`hOfp3(j&l?ynepyas@4aoap+;AbK8eVTpCnHm0_h#| z=kNB+e*rp#wR}gZ^9><>L)dL)xr!ZJx?_zN1b@0RAKq*@>d|`#vYdw<&@&8c{(uwYk@% z`_&vTrxG3c)J3R?D`E)AY+!412I+jjzf?_XXk=-fCKLYQlcgtH0%moPB~hCY#CBCX zJ6hzeZ;X>ea0H#I4_1p+o>I-I6@BF{*P13-P_HZ1Japtuhw>Wf5ur!-73;NK4ojfW zo^D{G`Cm5rjPj7G)SGz8HcZvbT0@zMnaMi;8F0&N zJcVvO4(#c%)8#Xtc%g?;rBJuXZ`Z3SJNMCUw5hgAf<8IG{)&h}Q<(kzDAi&zNlT(x z8G*o%tXXoCp+bwCAGX8?vakyCEZ>d2N~qZW@`r_{w6HEj65%aM`13H4T0}hD+WEev z6iAzsTVz#1(0-M1G1Pmv*D7d;=+njdYP{2_(y({BP?4ME)2RP2M3|Jo*(~FaC^JC2 zXAIoH!g3q?_DEj4xy{~uol4Z_(8mCpmuXwVx@yz8o%*|Ik2%@o3CBcG;%}gQ;^;^B zvd*6c%^gI2LIM6w!x?ws)S^S@LVnJ3OTMi$nET4&xcfTyESN*3F__!Aj$5Lm`g?s5 zfxEZO#QY$)w9WQN zy!{>X@WnE^pGjGLn^j%$QxC7RRvbS^SJAU&I~;T+g1n`LHgpv1@QL+ECMi-Oe1f~u z*bQ_yiHfvF^b2pxDGMgp%9EL_o@RI{CkLRt_|kk`VJOfmI%o$!Y=tM<6j!F&VXr6k zsCSsw?(`TemVH(!nzwzZa^K+5nkTp%2g_E>fIe~JS9OYB#ul%%n+##6Ez$6z#1h-v zfu8Kp&IN^Wuf6TWaWqWCPoqIre5=y*FHp<&mk`TH7a*DeKFfOvU#!p} zZuCrH1h`o+u#anbUEenMktt*W&U2cdj<7KQ6!n)@paEp%I>ufJHHQtHCJ!%Y9ir-l z1jj26%1wP^Xnvi){mL6Op0B)laeFs*5vxD_6Lq%0MEJ{kaDi|Y9eHJgqHpW&@&-e} z&6K3bGg@h3qp7l%U6Y@MNrne{GMnphAd8E>lT}oz%L|9Ftdpz|Ig)ZHAyL9_sv~=X z0MgiOas$T1{1eZKlH+b86=*CUnc^l2f?-Fm0v3Pkf+q3jKN@%#f!4Kg11%5XACe3L z2g_p79GvSC0zbW2`guKP8PYMrFq4A?Nyp6T*t&uCqgKDqis9rD{u#OSroU_g!Oi|c zJyxKn4u0J$(bJ?wB?Tf<`<-8@CEkW-7_-0k>qL`xN2M>RPpfhW7JjZpyf8w3v22v` z3Dr;~8vIU#t=w+*llDWWG@!DYC7zHxHBOY&)F;HZU-9pu`Z122R4hA^5XS}lD6!tk zs*Na{Lcnro8W@pP)TDu-ikzP-lBt|zI?z)dvc6IW#(ZeOxGpk6$Q-jFSRH_j@t@hF zqyU8|TGBD_*1FDlGk;5WawIW(;mKc#(oDG=oYEXoBS6{AhQd-V=>wY-F1T6NX=qYG z*N8;TRXx5n=WoX84AaJQDc|l2BKl9wEN?Qu0BM?TA z)*ooXcU>zRH_y9EN{kL_RKeC0+V_PQ45nq+>Xa|i-vkdxKY!xdt4~Sbk~N=@HgX+? zw_>-IEX*lyanv4gwjsZQsP?QvbaZ--XAj;SAzP`Q6I%|)Fn6fWZO?0%-$6tU-}Sv& z7>Z zJ6R2J^y24NhR1|1GN~`TIt#8(V_rmS4L7xCL<3wr*B+Va@ptI0ZP|hJS;!X?9HMEd zLc3YsKw5M>8dYtUk)?HB(aV?pu2c~AF5|cM3f?p|73Ny^Ms1!qs@Z{?q0xeL4Tq|y zg65{fSjcf(YIlphY6(A=+dIve@RCUL<;YNPv*k{5lR7({yZaL4!x{2wy7E02_g>rj zy0Rd1apJ-`OFh?Y;GU*ViSN|O!WSJyKY|iSQv>myGc)KxLAx$cUl3!yHaaDnpIzjp z-EK2^U_ELRQ9I=J>wD;zn0{N1VT=gcyfNV#ZnR_PVb;SnJo?#8zK&~0$7Y&f{N3wd z=Y;{yYxt2tb3qlg3oHOxUDVEsd-NiXjKm~k4)5{Z8gq&WG~jk9#WRb`#H}#02o~5z zhIszbW5ko> zUPUNaBgPD(tKuTVGee`L)|CN$$t>X{?SN*`!8< zzXx?kiHOHCf6Yhl9Nz*CLmlMBtxm^LWbB8|d|BL$!<=eVCyQ`vc&&HioV;3=qfp(` z>d*MlR$$3&rV+CP)pZVA_l0*Ozj-OSO7eIHWoz$G*reTc>oW­7#J@#GUyMQ;p7w zacUO}PC9QAhjo|ehE6~cvHAUH?X@?l^ZOU#clQTHduyuBw_omhg)b}V9Oinm=GWTq zlNBGle8X9fM2@0g|K221eXST&b7ke!vdC9_V!XsFw&oz$DOqP$isr0XdYR|E&ezdf zEBtmzAP;hD%1+I$C%7mmdN!I9cso^BD+UTF_E}|#YPCZi*+!aNbf{_LRJ;wapBQSQ zCChN00B>iv{+d^Q+YhhvZzCzZA-A0^x?$*XQK~W2ykIEPqNtd7IC0F(i3W6el_-%8 zb=6JOcn$q>hpc{zoHNOJ2haUN=`WPeh-P+@_~XlxWe(`eWhuKdbDRSSJfTc$= zs^cacP^-Fw?owGfXC^%JMd^+L+45eHrdq?#&S5x?i1cW2cm1s}>6*1i0m&un`fNeK zuhFZZ_~ec#-}9OU!@vWxM6VN7ePM|~$2P~@${?T{XMtUe8RO}W@^q0JKnL5MQ&9c*wj;Fk6sqt z3o&JfK2r7E$zspUf6!pvNpQr9a z5_EbW_xZ=RC)Ikdjt9(`)y)3B6q2~&?c9udG={VLu=?;xuM^QLTN~TfvkGB$Zf@y+ zSHmJvnMd${*!#<{s!9yg@is$PB^c@pTNG$vl-6t5tTkTlp_KlMH0fS4T|f>UZ!YrE2a zaTZn7>b`vfA=01Y5$E(OYY3tN2i)+sjcV?FL1c{mdxDZy;!m78(2N#o+M0* zr>$y=GuQZ#p%mc78@lFHP~Mv6H&Mtvt}SAE@d9&v>-@~vSKV%)!2LK5Ri{r@(!@Ul zuS4-P@Hk#zfBs^t##b_~OV~}&rEssd`l#6X!eZZC&FVSia7x77WMIK~Z~s6-QWB>% zz=s%`aqrIN*M(e3#`7}mC~lI{8`E13TbotI90YQ#hih3K-izwol4Em$1l$FsouX7e zhjq^JcR%zclu1puovVD3KtE5rrL}8u;Wvof+xIm{N%Us z9$BAH3s;-ZHJJ{4{jkWF`|QYv>XRPk?JvtavyGvpXL(f9P_>N@g2@FFrw0PAhQ_J` zocK?^F||D04fS%zE5(=XJlJzt93gnOJLRrM=sUm0KzJ2Re=3Yq(zACu!+hg>68|RYHabS#+w$l z%tMdMY@G+G^i$>XS3eaTyp|0(S|{nCbxV*aexLSD%!!^6a)|S0Vu4Yj%Wvo6n`G|V z85w^N(ahsJwfidA>hS_dh4EgGPnfraH+UH(h?SQbU47#r7kf?CI%?8*ifv!o8quE9 zY@i#W9|m(bhveGXM4MgYEEz`0oE-AD3%Jd0V_x!J@@>CZJGOSBDu?FU zt6OcK#h0`5-weH(*Y#*4<)xRv2bm7MS~dFPqLrD9+-k?9OSfmeH^RZQ#m;?}$&?0o z)xyWBbZ&#ktp^f}aRa-m3a0}^@r6Ddo^4bbmt5U&bq<~Mtlk|-u3Zc%KRG|QnE(&Y zpwB2KpPemAPU3flt(?_D%AVJqgMJZ`)7lCAR@&XNmsNYN3+VcdPhRX98clyV4CSmL zadvsGf7LvKqK0dCF+%)5X6(DXy`z!h**weEj&nWqYI4GNw>s$Tb-br#p`_6NjEYz7 zZ5d9vp|k3koe91?PUpSlE%#(%MLf9~;wIvW_U)zSDWbuT{HkMSZW}l_LK;Hk<|)RF zIop1AUqb4GBt18`0Lw`0NL^W;&vH+0RM%i}MaYn9#k)E}dns8o@4g zlhIf0YZ_m3Sb|o+Dx<+@eRMvd9@t2squf~QG}$Qa9sO-v z{O8t_9)W6iPNYjDJbFD5-j@3>F&Kjm+_pZZpAO`hFS$BXV`KUd4ws1>Y^^xyKZfpb zt-EYC5*rZA>t{jZJHB4?mNIrBI3N8I+L)nk*?L$!12q{| z_C7h!TYXt;KP=kl>@^%&@8B>N5zPrPtcO5w2fr+6xa}*Vggql#r7$aZdhXeK50}tg zhieiu&u1=en!$*!<=8(yn|PJWcU1Tl6+iOk?1*W2ysa8tJZ^X4bhVzmhTX~7?bS|$ z^>fE$u9}OMUIE+sV&eGU9A=Lcb#gkq9}ve9Lybw}bgd`>TPF49-TP>PEg&^L z@0?xDHA`~u?$|&#Ru=Mx$tXd5J}!ec77*g87e6o`jKg6bVcC5g)yv$V27 zOD01iz&M=-ovOAu)xwucb(?j9nk#Q5dK@;=I&|_ZMPNWlPE&F_YlxQ23-}qM;#&?6 z8}~{tINe^m&tb5jlziRPdavPpWiyv(n1!dZa^{WM7V8}G#zb`|p82aS9vR-tOl}_o z)*FRWubjIPej(6bGQU|}nfC67!}aPnkI&b4t0Q`}I;}iMnwm~8_EB1WCC>*h_S9UQ z_)bU-EYG~dHC1u40x{$HdDzpTu5nWq@?nyaHwV1~Ivnqb*)9lAj?Khrk!uN&&okMP zDtNVGd-KpO_)&c8a5argou<&4Zj+AZzcRY;ma>$+F`nN()G;O8K2oylE*K=aM9V|2 zs+f5&RUIz0Jf2EubaN0%=K}IlKED1)Ca+}h*u?H@i$kvV>3jVp5koIt@_4j+6bBW9 zcN(401!T}_w%_0HSuU$4Ae@NWwHpV4Oi!&3OWiw{?yF_`0>bZH-pSnbYilh%>wk0m z*e6u3_C&ad?5ZLm_whgw+pv!{jd@atb(nl~k9%3#HSu^onkeK+dIik3 z3hva5bB;+D6=hph1DF&3V=eH(@7N&jU{iHsy_;jq9z%DOH=cc09UE%rrVqbhiK<%*UG_ zZc^A=L<{%}!{3R(IgJ#@Y}fE!9d`c57@K|tiRE0h=HSb)QalcgziH<0r>V0*UVdkt zulv&(|3ObVpI3ECDQzKccm6?@e_V4zR8Y%A$>k>H-|6;`D{iG@{_aumJd);&9R4-9 zVXx|j@(&5c{zmsT+sTN_7O|L9Vzr4DU$*XOG-R~Yw7`*39#eeAYeJIkf2t!jQ(?hYq!3})$C+z#lB$c`@}#a*BvQ=N z?B`P=|8so`1LEjPW&$RUhu06;@yH|Qe?GiN73x*Lxic)SQ{<~5Idw5Rfu&+G6T-ZYeoaP2-J`AKF0JN~&#aj=Sj-S}`h0pxcKYGu-zhts3Uv-9}>1=H!bad++k zs0bc`*^~6L`r^@YHD|pnNMuyECh@n>9#{?9>;1we4?W`^x2#_GxH9F1OnJEJfz8b4m zR02Dn523wni>=)l-VQ^37)|O{^95UK%=f8@`Ro<%{mcxxa3HO=?P=Y=Q2j3o?UH@N z`(v}eF!;|Sr858#T8@%Ty8H76f33SH3}8Zn+y67a{Utpj=i^p0fRU3%`62v<13Fe$WLU?U#T3$(6q$^v~~p`tUDk;Bnz!(7;9c z|2IJ+_?V`C7-FhQ(_V}~gOUPIl~OE%X?CX-m*7F{B?N*_E0O@ItgH+=&dNieOOfA^ zlj@p0Gr9STUHD5Yq_e^tv1sNq3izrWHwZXlABG5@z-1e$lp)L!W7jaLr?wBX%eH4LqFzZiH4GX!jm{F?R&O z032sxCuc{w{Wqfi#1XkDm(uu&dU<@jWRU-D4)zvXDY|BpTg)5rHyJ>YWTSWkj z5cf5{$HZ{)a z8S%`LAF1~f(OqBWW9-)%Yg+a-yvH!FSV{$>7p?LQy5fHP=qGy*j8{U><)`{Ijh{1N zQ>+0nEc?K)c&Hzq>@>(RtBzIJ8Y2(!pA2`}+1iFW+-XOEzRbr7NDrVIPD?2$BrZ&4 zx?BweW@U*s2TGQ(4jkh9nAgFynQ8#^Fh)&cuRPUK24cni?m)NVU0Fi}IXbB&q`5fe zf${OP6h>DXS+~iPi9w!gCf}~TeEBjxCkMQ?B7k(*v~)ZyOk1tWVGA(&3je?_`OCiu z;9+!!`k-#;%Ue_q(-F++CKC74=*iQ0?H(|h%^5JjGRV7B=;km}=ez}6Ou#a|)GKvE zHLfQ$dpYKMQ6M3_qt)B!B>DJ7pgNe%u6(vWUg^vsg&<~^%0bd2!u1vQT^lZ~K2(sdZT$*02Tl}&+smI4@o zZT%4qyOkWqjI&s#WCEoA z_JNNmn0ccOM%S8}no2tkC>!+m6SZmi_T_?UP6p!5AV+T*UGB=%^yC6PYA|=zKGvp3B%{VYv7mrCOYE(QHJIu?JO=>WJ3OpR zjahj74CiDwO(1sKpjt8=QCjF2rZTcqQc?syf5loU$espM`3j+6VA4uKR+ET~_kCg^ z?1y>99O`CZW^2awYgC5AufqHf+)fYWlHkWjq#sZ`M%LHYjff0hyw6duWLJ<~U0fWh zv^V!`zX!!0*6rukm_a}ST)~8>OS+eCj3Q!L)wRAk(>f@E`09fsh_75aDy89Py{xE6 znZhb*IN)cW1^F){L$|p$p2E*oeT@YH#Cq!3EBJB95#}x2)`cnp_*o`Wdl1EyM_XNj zUnH271)1dd9kV{TVUx!g2)^dbTF~I<_WvcyfAP!yCCa}<0qX2u@%hVR_|paZ|1nY6 zZrp$U<6Qu74f4NA_gA0w)xSjfmnbm#{QoPTC6owvWqDbp(ryg|5>_1MR}eHOFg^2l z_YgvIG=KZX2?%lbz|4F8tpf<2v z?j9K#aSlgRQq06m&#zPR{LFt)-dPRip*99H8}+46x67HCn+H;>xTHkMfw&2z0t{t6 z{)>|PL)#@RU#6vF!ru8vAA|cj-}PjF4{vvw&)`Rhze4tZ3h`klK`!P$-wNblXOD9` zUaCEvW7JJ)kV#z#Q#FfZ)_qDtFI*+=?0k?K&ck6gLcqs0btv5=Y4*}AmuO343BmcU zBg4)&Et@9MJ_bB;5F#hyAHunCvlbUWxSO{<-6A{bSZ_(p zW%rC44StmFdcAK?S(PYsZ*Pz9N+For4T8>{qWCitMMcFxb+VG9owSh=RIdOZp@UD# z8&n_svUi=~LUDJ)e zrKY0_H;YOvWet|bm5Lxqh?6dusB(^4*$(N_0bXw6QLHrA$H&J%J|58B@HWAIt}4vD zqWa#KA>gZ^S_oXMNzEf2UH22(A9{1q{BoTOY&woUeJl9j?5f2~ra8F{tcpZ2XA%j=_v)NirYY zq51acG|714J?V0*xfU?NIVmgaehUUDGx9oG8F9US>&s~5=+M1BP313cHaS+7M^@1% zJ$PufW0_$tE?&~jX-fN#;>SPfapQRcy8;ej-|u9joK7dg?z_TZ@bFp@L3?|<=5bH> z^2$n4wdTplAhFBlXX-#ameiu6A}Q(@dmkD07u?Sm(qgQh$5U|~hl+-+M_weUx2wNZ zj&^ez=#Xzexw*^Y)T1%D+iEF`P3*4c?Akjz$!#}3cS7b;10VUw??|N_@pPLAt;O6u z3Wnsxx_YcgYmH#R{Q2L5d6wz7SM6;DYLBx?T=O`|!}e&6CW>7hpR}&?%&m!w_dc!4 z;{G9@zoIx@&Eq;K(PG;Ci1z*DvB4IjBO~0VBgJ2d3Hn3=9?j$~C~Tb9UhI9qsW^Cj z;I;|bxj8I$39e50&4d4PCKuDd!mHMNtA(_4h;!@yV>^?R^nvzLHyx-=ARZ5S=oR>( zFhBD0ak{K6=H-2cQ%4%WqXZTqW89mbQ$d(WTs<=-b*q0@EYIcdMSdWum(ivvkFw?`& zO@PuSowT1=;#3vO^I8jRT;KhVtOEm(yxmI(r1lo5&Q1*q_Kq5aO}xQsij8IqNG_lV zT%$P?M6ea?Y=>-8Jccg4&idg!?^88vmrt`KpV5=3R|xkg^+^4StsmbI$_Q$$et05W z^|069`f0V^@m+>q_1LQmVyW`;=N(-# z&oeY!PC^}LW7|rDq%vJ}12#m42TGWd`%_%M40n~fyO)hBE6J9pvv`sBi|ql+joYDz z7(|%@XPy-^RO?5}?)?z^7iY#X99btsMHl-U9QK6i>5+S3-GgoTs>IcUsBqU7R0gel zq2s$IL*4R=VhNGkO9DOs;{*`q(cqN1|ybFl1}&g zAr~Fph1kIw`?cBNB7WQR*^JcK(FIKU_`^Pyx2Ab^L~F#A#DB(^KY#e%_VCmVs7v;wDO&+b%is+^!7{bq2!#XS8>%6!w;GB_PZK_E#IKdCuTcTOe zqaWQ+CN<2)WHp6H^xu*GkgKV3aiSE>!$#STHvA3M8vTvJ1t)U<92Guq*-$rW1j5pP z7ql*X*(oz>o#=7i?75bf3^mo9yhtwSl`wtAhKlXjqx1!V!0dvQ;|h&cNGWRUc#390 zwRYcUs4B#s&OxW6Ustw0$~HC#8uSi6;S#)nxS)Tqnlvd}{?yk9F>e#ll?hKSK>h^Z z5a0@eHi_S2k_uKlE%#D1=l$NJwR3bVnVGfS%rL!=xvqH+=py(E!h;v!-+yyek_+|yrY5a$a zO~u7yyc~O#J5-WHy#+HsN@UuBx|dN#5LS=^`6D<|{8F;uS93V~MmvX}M@~`Ix)JXL zJkgs69cD42gw)i?7cF!+FSOxDvy&N`Y$a9prov~=D?!i}8rs4 zgw0@|js4*2O70PO4L;q6q3zFaM08KW^6*OC*5<;A{bJ~qiYs$WH8+Yf;CEfkDkPj+ z$DgQ~p=c^x60m(#KbOW4&s{TBR3mDrrPGDtFS7>spf6*vf9QQMI}|FTIEI z+F*}LeB@*`bDe&)NZKZM*kK+e&tcWTxuRlUv0-u#jZId@we{LKd2Dj6OE5?JX)07 zS?u9M3}?_Pl&yJ*K502O#>&E@!AoJ4X5Fm*NESc0g1WzWvglj;!XSH19J#_`dijM% ztYIE};*%}_wJ>*%RA%*D^f~_M#KhTQ#r`xg4wZku$*@ML*3#HYQ_2-dKG~uDWnePaC|gVrEF-feV&99uineOUT6LF)A`if${q~_ z>%@U|Xtnn9Vx2adwX1`nQS%+EX8gCfvP|z6yRMyc6+u&_g?ffE`^_C5D37GT7oM-c z|53UucF@qh*DJ{#!3uG(Rkch5u*YOfA36$7ON}4mca(88D$@FxT;Q1qccEEDK=B9j zHJpYfV94ctj@rA|vX|c72S`u-xLUq50UYcc`{g`x_BG0`X!8;{9M_A$pz8q zl>WAPqc*EMooY4@W%1*v5V;PdFhdl|x-l5X!fjh>-=lPfN5!sYD{a)Z?|)>?z=!*6 zv?l2!@qaLS0_ZkcLue;&uI0Dnf_Z`k8x9Wr$C-f5bZ)ELqS3uqfiIN?q@c*Sj_TwkU1Ql=l`*7L83Goui2)B1QWhD{sZ^ z0Vz;47x5NHNN}TvoGj!fH92>--=H!vxV-3uC{ErIW9&Vqq`CAo$M7ZX z_ca@8jZbQ_8Vn2-9|6Y$jo=_v9Dx)`Am4;4T0Qu-j607dtI{LrMCIR@8y3BgdIhLa z$KI?~-(cvxKt9WwJObK20kpkGbsQucts0G<8!*IUm_2n$v&WK}T2KP?44=8FShVqg zZLZ4rM@06=5xCe{eagg#BD&0()5!%<^rMmv`vz&TBOjjfTnGn2@4z?SA5dXUtg-Wt zDgFC_1?4nXz}Eo!gyU4jU}Wq&+#xB5=+F=Bu>2+_$pgb|0E_kcOZTYXXvZhCP>x4Jg4)gl!U;#Q=??@O1WS3C;%a81^& zbV?vLMp5liK|xS{&eF;w!6+qDRc$D$uTLyV2^|C42Q^9#j53F>M&*5N(eSw$F@gxpavJ+`tcG8Gkj3po$D*DB+XKdmeLvjHS8e`yT$f_N$z5&?*$aNyN163*JZunXwlo3Gh$a<*rDj**W{J?ugr5&is zpXZP`mgk<7?Zjj^8H+}QUMza1Addnx0(Q`Nr1x75=(P*IugHJ4+Z!lweu&Cgm2aO% z+WwmgpM7WqEBkWK&Ti)D;yIJx}b9@@R*-NF-OJrWUX{?aPVM4 zRZ-EtiO9YfLaZi~Zy!yY?&zX^Vjn*ddu4lxYlbRd{ju(3>!{t?)MSdj^Ss)gWHiI` z=#A=(y(;TyL_tZyqs#aK`n4N>v|?%&F!S|frnBqVf=)AzQnPUtIr{{MN!J6R<)Qe~ z=J=311je8^Go>9UHTpxAd<@83lSW;H^NO>4vv@iRs%?bIro4`HJje5M-j$kJ+|o7gPtzz)v!y zlfq6|d49aYSUwwM-N8j=^@aaGJ)!7%-YP;8L!hs*m)08%4Q<19!`Xg)?C2|5;FFmY z(eRtJOm?mGrEL++Nlo%rg?QTey;Oa9!*-;m>xo#Vu2-N&-^10?v-sNZf>*QdN1n{oaNvlCd6cf<7RoP8!WN#^TGhg?RV#bm^p<<$hDiBEd3p zbZF;p*kAaQO=a&UebV!szS3XQf^Q(AYbf=RHLQj2 zI{0g3RBCq74vkN1Pm7~587kvIj!x-E(6 zOIPfmHk@#iP=5wg!fx!`@LJi_ z2NK!O(?lBwwzAjmT~ym_y){1JeGWg7n_%NHTx1YFamWP8j;I@wbU*F|O?^#Yb>BCx z@sleq>XeA9D4)46iq1=A0#b3o9kGt* z*Th{Rt1r1BFsQ9XRk`)H^>WZV?j4#~&$Lq2o^UW28RA7^ah|N^ASffgN5Z-_<#4*y zWDeTDSIc!a+zu)Y+N0#Bhf*bC@wM|oOY{oK`4kENPGqN8G&?)(MWvuFyZ?+rK+9>2w~2H~q$)H>zRU@)GN7RrgI4_f1G?wP{3I0R$#- z)8rrBA$Dsc+HpD4H{&@g+>S;3Ge*&K7C8>GfV(3M^kd(4GF4a%tC8?X%tf6C1)i)w z{4_?w>E#vD#N~QGRi>=%IuaG`$|1P zTERS~cie@^XP;mcJM~h$CYaf5D~!-1^c0M|0)>_30#RC#&8znYSUNao@?SpvG<3YT zg5H`zyGxz|5+?zMG#XNtyA`%91cS-alg^X-atiQcEIH4*t0>TEJgw@;doFu(jPB({ zMJ#se*6nP)X_6b`sb%{8nevkt=LhbaZ%F7=A=za$_Kx7mh5HKG-p9{~q95oz-?-wK z{ywy&h7JcPAV9@%}N5Dj|~PQaos=GlAw?IQ(}C7t;5-89)#x~#VT z$=laXmNNGP&hCDRdZgGcsaT@UVJjAoWE)Jy+7aCk@({xlEb-2}$<`9;llksZFb7T` ze}`{_I~B1;6aSW;dh*0c+I=Qat)Mtthum#%kSTSH?T5q=fCY#?p{Y`PJl< z^r6c*YMUJHI~=gWt5I0}qggMZ9wmGSa{rRG@%XyI1;yU6NHz@B-dy{2N7m4Ir_}ua z0r!w1K~vtYte6vym>Qw5p^*|}UAD%6fB+>w<}do7t=f0BUY|jSQWWQyysH%W`2Jg`~RFpR_sz(C>tUzX<(<+M*bAJ6y*@y+C zO-)VDavLI|XnqL~uKxJusRAewyf_m+T!uLcd*7P(h*S@3Qrd~%0vB&zz+`nVk42Hs z@#$2GNmiHi&gLieyp1jRkF6$%!4jx8Isp=;3|uAUoQA7$R46{?Y?C(P?F|Ec1*O#$ z| zE@R0eNcBs-15t<`Cyj84r~&by!?EYFO|zFmaf;Q(qaV3u$;oXSV{K3w$~$k7{)@^pRjrswDsA2*#S(H+>=Gh_j& zK}6Y;BUojmQ_aspGB=jHqTpPRS6>>1@0EasT4>rce0>#5&_Oi`6sBMpJCBVfk(h(AfPi|6t^`5a zm5>cYxk*pef5)l#UOKpM)6aYOCGK}_b)Hnc-KK?z-T$KvX1!}>sDm#m3aBH$t!wFT2T^XpiPB`XYvrXx_wR4`v5 zMyu1$b=_NiMfXy*myo=+t9vb&wk%3792`qK76-zbYdw~@A_5JSHB^_`BV4g z+AZ8@>LZIve{(_}w{mHvF}I^Vqry1SU{pS2IhVc0iuI)sff~!fucL^>OLtxoPEVs4a6Dda)=ybmRMv z3tqOpz}HQB;CS~koTGg|Gig4c9nuQk+iH5+zGTDcxdzApuAXD@Yb91n4BAYTY-3fs zAyvDB8*3*PeryIwdm?-YSs?*vH-qf=zLld*^y3b19tr25cBq#b>b=BILndNzw5!H( z1J)gGhZ85R;i|`TGzeiL_!SWEtsnPmbGfLTs7%oh7(+H~q^(I;Z7C^aNZjN6A)-%?;oD&*9PXcS7hyZ96|yOpiw zRRdvofM_w&Apo}5^n|8oisF(%vfTXx?z#1}=)(7b`Y2~e)Y-XNV(O5iWZ&5pp#H8q z_Y*O~`^*OSS?BcB-Q4#&o2>011mD+pREB5&6SK#3gcP%Df+D5dn0uW|d5XYsf5cq` zEwnHz)*66SQLFa$qFgJ*wphsjZUQ-LL-xUz-4`mpj>woEh2` zyce3X*@B85oKwO}+4GqE{P{kuZL zx31Z09!YK*Xw?t6h9xjGnYK~bEQ^TXzjhuw?& z1)myB9|`%gt(S0A=$ZeYRuS7RuX+-l`aWD1>l&8c@MG;_Pahn7l$#_(&cl&jaa?cV zDtzz{2W%iQ!(K&6*I$PEWG;zp?`A;Gt%(!>g+4!i)M)Cy#uf>k$TBI8Lo9i`sJC!; zR_m1|k^aVao3?KmdnML!cWBR37F6z^lORM+KH&FnZ=fpnE3`7RA(icqD?1%SZ(4%8 zu{FppvDF{`A3F0DOlNxI`Kr&QQ7Hgkx0S{(vFruL-s}c9GB~k#H}TVMGJNaz8`AHX zq)ga?ZMbX80mIFi#z&a+kTMQQ{A!QQ4^oy*40$pSoUCz15E#(kkYrxfMIqa~z_D}5Uo_^jLWof>>sFzivA?1( zl$S#8RFvZyRtgm&({a@sQ1KO0kI2|NM;e)_Dz|T1Ny@}d)P7-71(gQgo<785oQMGN zWAm}AA^KUV%ekg(*f^zUV(}U7UE)09w0CGLK1g{W0Mx_VfQFa@L#w+JEpl92W;`D}ZgI^Ne0u#fH^+$fAg?23RcMctY185IMMIf@Zh@v7``m%h>b zF#Y$D4J*;w>@-B|xq3)1sMTi9&0s^8AWyrV{w$2DV+F-X zg>Jjc`tDNqUZID67tCy(ZT@fl^lB2^>1pcBqXQ%?$K{L)WJfbmn8!s8?(~ii6%VBx z9ZL*V;wKaaM38xv2cs8O(%BHwfkJA_)A9L4J~T?l#qw`aQq1i=^b7#`1{@vt;NR6N z4zNO%^e2CFq)3r#qG_3PY&^^Dv99c3kq@UOlIVtV`|S1Cuz=UmN-np%3ROnnz2M4% zFeKs$>!xr*Ca(xvkR-F&j`W-GYHBN+C}+i1=v&uz)LcR=ukT9NarAvv3_|9SUcTxl za{amoW*05i3T=>H;KJS|HhOGuvV%zLf}0&CYhX8jL%I?w=kic&`%(E!T=GYdH8{6-(< zm{IF}p$~dr6qU5|^syIlx~wsdQwEnRIvyz$kiYSeL$E6??>}fS<;o*TH!S%2 zxbyAdiyYlYjk_Kb(W?92q7}~X>{hq$*3BO=#0bIn(mZfsDuN8%>0X-h$Fw-$KavgE z^l2?0*i|d)2)(yjyW_aF8dqV+x6m+keo6lh?Q?8IKIAulzzFk@mn<3{&i7^YaIjU? zfHUvNssHfg+uGJ!uEJp;-Mw_<(;qy2dOQ&!&4COMj(}psPmz>`M%y;>>3-qH>y&b5 zLE-@AXHGU}AD-%&r{WdonhxJ>xtBN%v3F{!3cG?(k*U9`!=1c#_fA;~FPAt5qd8bGjKO2*#)G#=q`Hh)t|3Q8FUbW6adxnSm=L$DZQltg38e!Ze^>Bc=%h9_)5a zfH1vuG3aG4X*VhGX@@#N|Hr)oF4UkrqRmfyAQWn`7l5rgHh`aWPzquypP>koc>XSD zp#CkD?4eGF>FB`3`xpS#^We-K*nA>hOqJQ}@F4Vd- zW=~Q}D8nvueI&D>tW3P=MyeYL`?cP&O70pa;fdpU2%llt`jn@SxlQYHM38*-DogOR ztLuZYDd(WK))1)i1`zW(9d1tTEOh#A3G8+~25(&{8fEq^VyxQy0&3cXoo?&a@v24o z{<>&$w@_K_(Rr(#$pcC%#H>Xf3lDVjV{1(uwvQZk`f&}SX^m%!C4E}WRU(V$Q!zJA z5)s8P4IzSG4r(vXu14iwwc!H~YJ#^GIe~TpMa6K}S&lJ_$J*M7D|An0;xA4YNZbw+ z&qAK3+zll zAI&xg>&(*7&eZ z-Yu9Y<#JFt8Ov%IL=9fAbJ8tLQlh!H-OA`op-1fWO-|}E2;<&4k{P0_I-}UfvmDtp zfe4^UFSS{8-DD&c5?KjTh7Z2}51e3n=hsEx+B-TrT3Qx(f7m-XXf-Uu4j>UQE4jJx zumlt*tEza11!vXUIy$hu#Kpylh=>Z~9xYyeUL?S30*|NEpfl(rJvdE@?50R%x7sf9c#Gc84fOul-h-Ef+MvhotA@NMa64s#7# z+me^Eva;4Y?HoLDXeN%}byOZ}Ovg^G1TuY1?fH#p28|#dj=N6s?BvJ00Ngiv?wP)k z>}^wJvfZ9S-50(7UBgX=7rp4sIwrvBzaKg)x7!>9g|9R{pOL2jj*o8bU4^AfOgZE5 zJa5qV@$D=3>FI5=arE$LF+7;lWC`CX3E~MTiNXHh>>fG|JKe(S zAOS?lqTflrpx^dnV`8d1*S@`5wVop;k47a+cn({Jh;67<YL+Rp5&> zPK|EhGnP7xcuQzJLIi1@hosz>bagKBl^h(TS(Ny$0o zaABK;+vm`v>CHLC3NcDM{9vY^H8B#Mg`x0SO02=!1**lMjn|*<{w-vWMX?m>9x8nU z0}eK}%Mbk+-rTbQ2?$f)l70&|?~)X#Qc%h+P2wluS$mA^ys^bun^7Hyq|wmS@|pDj zi#a&SRd`KV>ZxSfPY__H>~hU5nk!#!gC5I`evOV@>&)OmZB2DMyD6T^4&{wt_<`bE z%t+nkI%1oxLb`9>bfCTyqm*snjaR}?f0j<2Hflb-NM3f^?58keB$qpg9dd(W%w)GbTvq@?+R2# z)+>qb9ep4aPvSyCLJ_=I-f839x`qX!L>b*)<~JFWj-TUsS0D*$yrSiH?|CWM z2~qj!6fIhjJnVCsQ1gyO66U(jxVW!n@=M-{%{0^1C8DD)d9QsDqPaHV+Ipzod zMgJY%wXmD<${5fvXAmpd^@dIqC!_sf?JGg*%l8s#;=`snXFA z2>khx**y0V+oA%X4o%o8wSVTo;aw(&uZ35zExP`o%Mf;wOK4uU8?0RYPy@ z4;@_9y3i?}JRT_J=HVIN&***`=d_&R>4|)z+^yuO~vcVpLsW&<_g60-(30 zY|XRf8qTdHjQElMe%+em*vRNi2ba)}L20x;*4?wi%Z~E8GUxYYHx#m_iBELG7-p93 zy?I4{5^>V7%TEql_+9I#vT7ecGbwwSC$eHTc!S&I6+M4u!0K<`0+kJ{_FpY8pCnu| zT(VSN`V<~=nPR%L0aJWp`jI>L9R6QMI1fhLdE6DNsMN0khv3Z(Ep-u`_w6uYFLI#h zq2h*Afp^@2?i_|D{FW(7i`@F-vGb2(m|3l?Iw#DUD2C@db}LyM%~>~>=GWU)xk{L} z5>Z(6D1J&ucnsa{8nnla-kldc_JA-5S!aFGY3|m%UcWT@#$Z1`vVv7IGB>u+$q6?g z`0k$()<^1F+8)do-UI{$ov}ROVPX2LpbQgDb^Uy=UOo}iVJG?FtaOO|X(iUu(bD_v zuGZI6!kuUX>aMMlYTHhqn>tfNCyE~!wu_VTL^=**wf}|Y@1O383VMF7kYHEsC`o5f zlZ&8_giaAPksY*VOYJ2~26Y$u4Ifn$u6~WZN%p76Q$8o9MAF50+v?gx_ANw;Ssmjd zMIKKjO{X}K@|lHc#q=&gAqdK7## z+wevsrjJ*4DYE_cZTn!X&f2No%z@i|^$|JcJj_X~w0@U8(P5Pwb8YtFYd`bD z;@08{6J(v=n9Gg7Ajgzfc`S0hz}fx0lkh>RL1IWe z{!V9@ym1Rjb#sQY?w+BtuGzroNJT@Ut0+rO0aeK)Y3H%@EiT;lVXpT@srv1$gYB`? z!su$Re<-de)gTR(?%wc`?_rv!4?3d=eLL*fwhP)jEP0~d!GduzhPgZY#8u1}Gv3ZcIu^MW#$Ulg8(U1o<(Klz>eZ!a*eq7XAqI5hk zsk_YM>X_b1r=$P=>>m0o7Na?B^!sEQbuL*`F@1k0>egIqrjTjV|?VPU;VrA|lFqQrLJ+%oZiBjrz5 z{gx|8xP01(d^+lf6eJ~W!mbbe`$Q;v#!lKNp6H-77puo3*?9QMB0sAG7v$YNBGZ1h zEdz$xl9(NRx#|{7-TCCtr)qn}Rer;lD+*1_bGMVk#|6#|x8LHU<7mXGFwY3bZ+4NA zrmZGEnE7}GNMuXb%v@$+*8S%B&aV{(tPW3pk0g!#(>8s+1p@RN^2aywVKll4S#qvD ziFPRJ=jYrK(bC;=4eQ%&s)HB54X(xLW0u(j#SK zDnGSZvM{-uudmsrxu3+D9hmKX>+g)%e&G8EJl6Wf)|h#iH2rdcMy`fu?RaJQ73bY!R>TUpG$0Lt^#fkbxJY{HE3$?>soCu zYz!KfT>qsWL%NHEmPb_Nq7*)?lxO*#oeNF>(`#*s`deO2lj(*gEO;kgPq>uvG$WE? zoeka^7hTJ^O%S>lX9Vfjcvd|YF=D>Sdu40;rGa+j@jE;U`_7%QprRj6j9@ZIfBWuI z;ohG?2X!kCz2S<%?UT}!k>+ANa~wJO4*9%V7WH2&@YTpy;af}}DGLl#lv@gwJw|9v zVlYg~dTLV!hI(|Zm#G-(wa2q(gXmIZ&l-q*l0%42I($f#^@F{6oK!?8G3e*rUF@h8 zzlmp~+y9}qJRc|H26jGh68_onc6~)vWV5iE>X3QlXyseouAs%uSgG_!bI<-)k zQ@2lMNbhU;B}jRQb!#QEVuCxyaXxurPn*7+;<28rwOK5YLZ$YaA(MFyo71MU;>W6N zPBD&R?Co)s4&ZFZ_V%x8U!Rr})yU8}_bWUJcQE@aO-Nb@0(zl=9iP2!{=K-Gex{d- zQdAADMbFdZk7JN5zrK9Hdvq$L8{Ky*+Z`+3EK7X$+RRGBp}Ln!_}M>2!29O!1vVLb z=*}|u%e~pvN$p>=rR*eVWEmrGPn*_1B?y!~OQdX_*VTPNi5Wh|!l*Ze={_-UQ=z^I z9LdjNrGcvZgN_!^s|NZ3 z+vkmj+g&GLw4!ScDr?_!XpWOBpN7;H9M&9);05vp0UvXMrRUB+BwF)0C zZZ`Ga80UA+^MAkaNn!8no@-{!T5IOEBhXH$uCe^VhN@+Oj|FzB)Vkx`6k`blY^c&F zTM4vn-`@fGwrz3tN5uucJ;|P2Hnw;#les$5AmUqGhBwduKqcSWCvnU};MI_73+4A! zGv;We7;rSC)wi7QM_u&Yi*vu~LacJi>99{?dDQ>VUM(Ci{N%}->gX%vOwH^r7KCzN zZ-C39>Zwzuzpkt#d_cqNa=k5D-yr*(Bs*f25~hFBa6b3iDJ;T{+z_#-&P%Erci}C` zZ>CiM3Q)y@zxj8_mcnrW62-0J8-PM~-&UzQth{?LbNcKieuDG0nqIZ2)~%Soa9EH3 zznK*RsvQN?DFq$P&%vP9+uj}Ip9qFlcQ#!Gndh_4!Y}fyzADyv=T%3#SC@6~WCZJO z3b#CP06AyS50l8-q0LSy%$#TzW+no;_fmbHd0^ZY6FCk^g|d25uDWV5%`T|+Fr*F8pUth?>k zcjtw<-dvuH+!l@pd0TGXDgtadM1NBmVbPP2KC$+RVG(Yh&(L{KA>Fj`H|N81o zn(L1}w@x;9s+2yEdT|478+}&dk}JdZ?SDA#WuXhcAy|B9e(SL<641`Ovwll!6(08sLAH-tIXFy7&dq1Z zHG^p;yd8}SuIa4oTOiT$A8KEV{#R{VRua&YYPeoKj6tOk=}Vf9QqRy(IThFR5}G{#$KK=O#{Yvxe?Nc1Wloa}6=DOG@|xqOjt?{L2G>{=->;|N z>U>)D&`~b4G7wTgD4oY%oUhBCGC1L?&InK&pqCAV;E)!wz}|;#-DT;LRn=0<@URHu z3bW}i7XwWR^I=)#D|{Q5anqigG4KXSuh> z&ct!F{IbhRo3N!nC2%y%7qX~tO!uU{JeyR@)D87NJ*bF^kN?8Mf|T11t$vd0^hu0H zABXNd98C?ODRk*XuAZJMtDD++~KBrvj|)!1uTrnkz1CqM*;0p>)qYzN|kK)0EARjs0(0 z@WR4GB4U)fxHYdJPeSh%K$ti&EuA-5E`m&UI(O9twwK#U%FnH*#AuxB=XnV+673RA!A;>0=O#ZBRbtUJMVRKsKk9IapY*$lj znzdtBu0-11nrC>*ZWD0#f0ufZ;7 ze5CK?;Z>3qb{8}2{;MM|*Hq~FID%nRBfRz6;Jecb2Tcp6UTMV(jPiN1X}+5eJ)3s# zKe3V(`5jDN>!|{l?iohm2^W~qC`koNbDxkXOiMJu^hY}aLCMYZ_Yzx3)Gn7C;wMLBS zIi5OoO;7K=AYhM?dc3>mDpp#NS7!P1wg3DQVzo~kq?dP>ckry{h9ep*)a|OM!dgFP zG4|5V$cOgY_Nm*bfR{HXxR?Z9%U>imou#(n_;9+T8k>HYyxzXWPIiQj~sD- z(rM)H#$j*GK}zlaCk^ExA0wgcDdwCj7NzHQ>n1)R$5qYY%h{gwM^pUzD89{n_rQd&^YbCbg;5 z_6-^inWzGq@?KhpvQ-zsr7ArL$|*HLSP9V>IpXlbE96FF>qZ#>Sb8tDfbsI21%Q0MDWy{6}~2 z3Pb@Sw?0NQ%6e-wf=v3nOk7e=3XKxAg9-#_1A*SFO_{LMBzi75lKAB`Vay%GE1;3% zza}p)fhVt94C%dZs^^%2ElPHF0rc<7A~lt~OP8U7PY6lwno#XFXvq=VrRh%mH1yw%K5}qO9o6r`&b+JI zUCKM1&xv$&JCE*_E9H6M< z+nyyNg!!5+3D`ecQoM9Pv(FuI6HjFG<2d13Y?0Fgd4;ihS>caq#}gXpOYto^&=DYy z(%263*!vP`QVW`N?SpvEGe0djP2ctJoxa!6jq?6atLGaipz^i-k}aVlh&Xs)#xEkF zLQDZSJW9TpWgOY7Br`K5*teNiq<#e4V+f?wsqyS9f)Atloy9Gr1g-xQ|oeJ@OSkhg@1NiPs zmV|{v0dl@Hdis&U^o&b8?&D~ggM{-H{DE&avz_>g1+~WdGxri1_;Ij*4!gw>IE7Hr za1#;{f&G_v08HZ4I5D4%SJM9y3#Ym!`ty8s@}c^w4?ZvyFed$B4t_Je;$CHNfq{tH z>Lww8O-UNkw>?Eb=RkH}psM=kxG|P3_sPwR0{srPbgfSZdkk8xEBF1G&_JCf%l_MI z@jSWzJlpc~P>}e)HapM1%~dFpaMe?-<2f%KzDgc=ULW3L^8fx}$X|b015e#JM3hc7 zmAnL73q1=N-ly}%JVU-3S0uwzMf~k4HNKhs1373&*i~AaS#xjfQa7zl*9wP^O@Cot z%vY!X&v!vm0HW>PM)uh;isOb@>Dtf|UFL3aE7!~K zJ?)*HuzefA0kA45UQOf}!&syq|a! z)b;{MOZ-5(S)^wfpK|!Yr=02mngkKJBpxRe!m|GUt?lJNrB-nVgE$-_=hqA;oNM<= zr+2R4CC1!(bq|$7rxVmM{#Mmrmw?a{as>kgb6;u-X<;OxC3Vr<3Fti;UD9RL3tab4*hnngM$S83HDRy zIX((jH4NtqnD)c^3jX`*!NdsEd~I)U0G#Q8pZqq!Dlg{(U+e~Ued5QjA~>VxnLOR2 zeSwEi67?I62s-vF`u!1wbVPwdyU$}{z(q_r%Gv+~TX@II=5QK`cz<`yfY2MMuCM<+ zUqXB7y5H^lXi;N{VREp+2A>8xc3LHnB; z^MXI^9(JLOI@?kIjb%!DDcuxDc@mgGz`K586{y zIL_4tTyq020$i~B-yw$6YABsA2W9Q+HT(De{Z>eUK$sDq6 zeI6eU*f|iUi)8HvwS^q}1Hb-xTp8-XY1{H^8*F_LxQn>oUo;0=mfzegujbkUClK0; zH+zS-Iv}OOjU+vQ@EzbTuBr+o%*lU4-RxW_-pR zr39AFq<}3%=evr*M{jvve;6nwfqg$&x17>0p7q({?uYUsmE`oaZRcLI0f6igD{xN- zkRk4$7!JS#ZawI%C@~+$YxB^{@XD@l{gsaB5%a!*JD*w#U?C z;ycs*B&5gif7gEH7yCpYpuZ|1CMKO!tA zU52fvS0h^QDA*}FeH7qhsU1whhT?&TqT(@NLP&GDZ2UlQIP&U}djTXGYh0qHYX69m z>-_pJYBmOHg78459`OTw06~#9uX)VWRt^=uY7%}#xKjxTw0k;d?yffo6sHEf|A6Iy zcV6JoTT}RXLTfRL5!jZz5hxh?cOcNB03hhy*$K|IUxonK>_;h!T;7{XHGcA;j71nP1o7JvcQ3Oj4rrQmWv9 z*+X7M@z6Qwueco`1cYwn^*EM|EL6LHi{X`hHMR)Kc4>NYL!K;ZL~?Csr*zK1Fpu+< zxAnzK?^N>p63PFSVABHC5#;`FkC7 zC*x|8TokWkWEcfU)9dERt%5W9$vWv`Ddlfk9%H_bxw~8pfAjcXS1(`E3n9~epmY?r ziT-@n*$GA*$4+x+{mFG|{l!h(s#VZIX))o=Ku$lx-^vF(94w44jt2<`3-_F033G&` zU<3X63K~E_oU0%zL`(M(65Cf$Zyv*omsV7cr2xqS8K%t=gOCSfP{tqhJ82;C0hgwA zOO+w24CzV7u-C5EHQiq!%g>%E@O(9B|6o0YnC=h~3iQ5o9$@)GJ~m3i`!U#m7jj=8 z&#ln*emH%@jg0wvjGf+c3fCXzm_gY_tH2N$?D*C<*mRQT0jv_#`Si#=@-Dr^XmSrP%yH=U|>}4PjQD*)g^bnZ_eqd-KdO@2$wFP(*PDgk2Kfd!w zGTCM;kJS=H=1@Fye_A#ai5j~PJqxksLw-%xGCuiW+w`$8om$EuJMna=kXk`0Ukcm3P?7sm}d>b+n)9fD+eFfYg@E0FgVbQb+zEQOE?D zp!L{G|5(Ih+iK-3TCo}Dc01VPB;FekyY=knY=vOfqm+zQA_?&Nc*vOGhVqI2hmL zXDc8}7vd1;UqKFZra%uTcRJ6`1m%s7bGt{>cT`OYEz)X1T&y!tz^&|IC9U_2@gG7T zfL6q+%cEh=ZS|ro2WDJrmIXF$r=(|DDKYAsm_>9rW@>E`?TZ{S)~iBEl~ZOok*rqR zolau}H6sTzfdzWsr-gcKmMl?Wq#Pydo!x&n2Dh0V{`o8rQ2W3^TuTdldgb-0tJ8Fg zUi;?c(HLBEE9>@YwpahL%#XIK(~&LZUsPg?HiS*?&M2Zge||-e8T3J-JY3p!5xM-H zqtZSx#??;{?mbzTxGEcUOsae@)1hiyb&af&--n+E??J$X2XJ5e5aL?(h+jPinrvS} z9)sbm%VI5{+C&M2Cc6K{fwI8If+YLEFGu}pm%!ED!lCf9?4azrf~#w+G=5k!gC0xe z!}iq&sVO&d{&=O7lWXv*>dx9xI{v>Zp+JkAfKZ28^I3{pY=Um1u4)l}-+jEluxZhT zfbNvKKkq*AWMaxh^Qj+jyL=UaaMDPshXy81Q|-~@$=VgAWYpxGHK&KH4r7o5kiC$6 zy8zu{vp}v*cYz|AyK3syt5>kc?d{k5tmr!xXSz`7M!$g?gBS4O$&u34MMFGIOikUr zj0N?LKT!;y@E;W{IxT^Q4%^2kE30WAB){sC>HizcV&%5Xl=3he#eCHc#6pK-Py9w1w1PP5ZFkuLG?r$5R%%wk>@fbKKrN>la zQ%av~w#&P=PL-!v9cJYrKdtF(JQpd*?#j0+<)S_G;^x(}u`?d}hv6Xt7#>6xx%a`c1t$=Q9&iE@DZf15Og(CSaM>?>b3!)B4Z}L z_Fo7Q3B{`p*g~}q&$_rMUPKhwJPQOPzz$a(uEDoiVlsB^l*#tX`7lB9!2*CMr}XHH zTM5^rPVwFX^ePI}H0wciakBj*f^;nIAHXdj0DztszLC;8!h|d!#_uzy``jP6 z?YTj(>G5ux9d?;eQg;8cB){v|(F;1>(_jUY(&__#xRA>Axb~rR#(OWE#ztfB*+7J7 zHauoq?Q~>m8Z>plKaeNO{LW{!3PvbR0H#$}H$w$wJiqz6V0zQCKaSs)ARKQZu_c|C2M98wG}Z_>Y@FUyNK+cEIvuQ= z?g^(b4ceT;&sE#KH=Ml)v6O(5D?bgu=2-HCo+}`D51t5Z2QlVpf~_(byMBltMQZX7 z+fur0nGk=<;x!czB+87lBuHD>IMkD5?)Fy3E&OJ$-$)@m-c0T5`lk*L$)u@XS8{7O z;O+5E&DdX$e$oy8Qu?;PPgOQTD{04;*8kr?s9ZqX-)eb}|Lg&F2&#kmyWY{XKrUTF+}okz>F&D3bF zWZby`YnM&50f5UbqO&R3Li!gLQ9*seA{%XWuAuq^dTO?F${G3ehmX^0fr#YEVc61@ znpdCSg~WZ;`Fz`rrQ!>`+lXg#FKFTzv0rrix&HL8ugmyU{yLi>5}F<%zlVe(0Kqe( zUA~v<0CS|2gKe&cOn{mGc&LQP{CP+Vkt5~(uU`7;M}Rqmo15q!!DSZ9cy5ndhbwG| z=5IR_j7)}L=q4?zNbo`32mncr^27|jPuso4B}5|hI`QuU!ggD3_z$7H_Ec}^_~^o{ zykt8&p(@=UXBM}r46sVBe=gQNr0w8r-dGXE|4aM_@CUJp>RDBEU_D zfgdau06vn#v4L^Fl$ND$335fAyLstU8FdQvg=Kv!;yiifMt zB>WOyG4+!!3Tz2T;FeMjW4}r@$4DiO-7*{)n9?)u8Y7w_Mp6Y5Ch?vV<7^qKWoT$V z>S|LN3okz}qnNH4_&|H2o?juD*TwCooHS3~<3#b{>sd!g)&E5jGUc%KHxRH`E-~~1 zF#pfNo>R&Dp#`x>gwIuk#TGk_)t0_k9Zv4kEX{ZcQ4L8f*W-33 zzqabEZz`#Gs!E5NOyH0W1W2rHu%?r zBw=U>o<5uFKmn!5#RXLAuOlvSlrC3sa?U7ciW1M0d27WtHCz42#sPAG_t6popB&=} zTN+Szj=axum-FjCQTj_wS$8K<#mGFe^ggZdRR&} zKIF6(oQl2F-3NAiL_XQ(4>_sGUs7PF$c&*aaXJt~_ckKKGzlcU>?{X1xiK?qDu*Dv zdSz>?;g%~?1U?<5AR);tD(VM;U_!EUUO!I%#Dw%hK>{%rk1k(-xR_Y$G#~pWc0!P+ z&T)}R-FJAxMMYjBwRdvPS0umnw<0}(Po#$@MWxHll19{na%n+@7H%W9SGd-JmWsO1 zo|oQRYwTeyINp97XOXk0cGN(dms}|@vBLVJxh8|zg&hHrCOx8l=ymFCPUg!KaOzq- z6;X+?^Jvh=s^?Y;-$Pp3VF0sS$76dE;8}3X{Mv;fDMwE;9?1zJa1q!7d$r{8_+?=KqO!8` zX%I^IhFbfk*ZT!L5Bnw!Bm^>&Dw+^aKdA8L1y$r8zu27i?GLk3EXCPi8IG_}rhheY zk#E0Qu>ELipx9`K1qHPpTyry>q4!Y-^P`{CpW0r|uo18Z*ilH2rTp%$WL~{Ad%S4o!tz2JgEQ#8@3JS~}GXywz_tMrHqEN5*5;45vDl zJo>Du-8@+?u;u#2el2SCI9BnL))tR-k8`cza%A=n>SMm$+meGb{zDoXiG~Zm@`;YM zbQ<1&=V~3VxaXmorH6gZvTOE%3Dd&+;y3#ZGR1F0?<&!V9aa}#o?SX`AyybAT_c=* z6t56nh~RFQ@tq^`^o;hr9+JNsGIK1NL&qo8Fa|ur2F5kW-~*7^}aRc9#Vc z&dPsg4q&%1b`%c;>?WW+dC16sRt{Lez?G4E>I>%KTz_;0%`ux(p-KlKsKSKQMt-L< zh9V|s0CrBE*Du{mXMIFF2NnoBzV}9?9FG;(ALjvQqDW-N7-^(PcLFm{BMYBe6n4Jg~YQYrH1lb6stR9eG4xpO@7-oD<13985tD z?kAT@iiz#OK3nD`cE(3Wf+(|4+(}!QTQJ7@F%@+GENyYY?Y4;YgwsB2p+=&LbHl2h z&S@@mY_QSJe{J_Qd;Z6IOuF93Yg3dMmp(D>f4;mI65P2R7JXBVx}H?ORXnR44f0;f z$GF>_4SkzjUrX6{4T?5bEx;_Q$zl871nKH1$P$p5Wdhtx=@xwaLaJ)T^ur4OE?mC& z@R6@hpYn1#>MVn17fZ?tcuEQyRw^?KrWOHKw>wzaQL>^DeD|OWn|?QKiPQANr3t{~ znZ*g%3KGEhmI~7 z?`dS(y!$by1o(4A%3YdhbyKb`D{1NR_(D4BwbGaUxH32Lhd4#oZG0Es?D|chSJBwM zzUSR;;Ph%J6HK1@sPTzXMqX0qYqrAoU)s~J9}J%%eGcyH+y-tA;@eBzG@(bL6I?+> z?M&LaVu4XAk=NIjgGpl?5eF=)b`5rZyVlYfw+rJb*fs&rAU{(Nv*)2bhH6*stTy7W zX0(~ZF+9qyK5BuHl;SilWWheck=2)_Poh?${iKc5y~+K)}Uhz)ulRhno_O z;?Tg}xB?jZFg4sg2ifclrtCW>I+2NRP@`Y|s;6FP+c=VWjBi~A-W;T6%$0mdywAs0 z7Z%ruB93tQP7h$qw_e}J+}_w7WEUfti@I^LZk>9sUZ1YS1j(JT3)^^f8ttunacmK4 zf7TJrFQEN&G*SEFNu+yE9Fet+*-ZR}J!h`rp-_T90?onh=k$f3Mv}Yr;Kzf9uu(tf zE}5&goK8bI>%a(zl1yLk^u>h&iX*JY1oo&kJmY6G2VT9fvEIpX{@-h42SwZTIErde5 zY{cnaoKjYKmhr8l)@>TuAUfkp!o4#?DNikMc_fZbR{Q+AfRRd z3zO6o@&aQ58Ki++qD5Hd#dzz6Qz$d{aBZJ6!nr=lQ<<@CvX`-MA$^o@6!;8M8ZtNC zjo;z6{zNOwrt+;qe)C-tz#NS~_lFUR7N~*lY-iIP;(u&F8X6dIJME!xQ={7irwAn+ zJZ(&#otx{(FLgUGiH(gdG0S^L=m=?%#std5lkm^azgORC7+!hVUaS`FB-lL~i z8?0l@e+S`}hla!-pyg`iaC(Beb8mh*%#`;r#6(FNFhRdfu6Hymd16FGpEHUozMnru z88}5T6VOTvW@5?l5wL~fsV}{C-O5Y$s2{Wf_@9CEtbWohp28AZ8NN#0QGXW}pQ(=7 zj$Ft4AarJx%avVW1V~15%U#jZttq@P9F#p_KSOh+d>o36>OJzx9Y1#Vfe{DE(Q3>p zc9WvE=4xAUURGBKZ5W|c0Tt-Zm(b8pZ|A+bxPZO1a6e17+3bJQ{1{B%q@t-w3x4wm zt~f#T!`G2*F;f!;+3u(HqFK0)B2*>BBQr6+icI{;qwCyRpz0J1@54(4{xwGtzyxtZ zE_#RBd}{qskpH?o{>aI3h6orcBb0>lWA<#c5?hy_c%`Q4KOPrZ+O)OVT3|gTAE`z1 zUdIJn5+nJHhw4mGaYO4KJa;1=23#j=D8PnLJ&E0jdo&!;?Z`4e*>{}KG&B4VELD)?{P4JGryYd`-Me*^>f#d++Jo)#PN*X;eL6ios=q!^ znh94V`{~OsI&Gzf;;_SA{rK@i`)(C;O@odcj7;6>4jxONb-*TJz=&NMMrNcuaoGjE;IFq&7oA zPf4NST*SEf^SU^Q%Qss$3>V-X-#o#)AnnHWYm`wO34Z$mJwIsA7xLp*gAymhp7Jn4 z{sPx8JKL>67OrbJz$U6yaAKM1b>l@to4;4VHb-oYI0F37+7BV$%q~?HUU*AJhNxl&bPI*|^ zylR$sg97;z;jmpI`UU08uIM50kwh5*<2?xyIiy{) zo5g9#L72~sR_&=khieyh^#DMr-#lO1%L~OtzdpS)dje&%TpJ&grdRKc2 zdx4<bm5GND5L#^yf=X!wlDd5p_n3~QR9n1 zqJuEWQsZmOO;5xDHP$PSn{VIlQ}$GmQTB>76wYg7<=*V#(~@@D>y#|PjuRqrL$0#U zki?hFdCnUh_64!(@Spk+&@vxbsTw2!78G`Br+x?H4jFUh2R9pOcVJFr@= z_*3aig7UkvysxH}E<78Orknc#W~0x&31#KNYaTd50hXy!A~kt2CW^}W*1KO6YLk(N zVUs%EY{UbAhIZvB#!BvGe^hid^vjntafSRg0UJ7xcSDan`n@sTu6iL{vrz4$Aa;|G z68;_ML_rPfZ7H8zwc<)z5^k3MUipE3#TRSoj7O-nv;I?2S_}#CZQ@KfWv#Wtmf6~O zWU48gyV|uYJdT}yz_K#rGWts!No!``W6Vgq> zvbEetmOnOnGk-CyoX}ahZa<5BCprDRtaQrV28RW6B83Ipgl2wAa!;UrajcN zcYc~r)fi>a`pmUBO-gP$AV#Exv`b6n#2o=)lTD$Voy z)6%H!ShC+)+wKooVD2c4VAIDz+ZjLH)9ohS?I51_S={8pVS<}G53rZAcrWJ-KHYIX z$ltvr6~)E$JWB@qE#*hF&203&>#l9=SC8QW%P%@#1wM@swJfo|awa9kc!}smM2PeT zGTz0svX~@&!*HX3BA2Y@^+FpxLY%Z>aQIysJN_6uj@pBr{JXAW6PYk=8a$ivZy`W^ z4Y-$$VmGt8Q=y34#5rorTtDp9KH4?jX>^x~AK9YON^sXR0&j0|`Utec`&xW5z5AgP zJKYhCDam~)-Iw*=CsLC;bjB(PY7TiA;1Oz+6tZ-zUnF25wR8&s=78E&3HGq zDiTpB-~yzZizUO@nb6xlg6-n2WJ=>k+8#=#mI{b@3Ma8%zJlfjng zhC510ILW+mI|XRzU=L$S%p97$$qe8$7Y+=Ws}J%+-a#ObF9EALFVw|c&jQZrSC2}t zSYX>IgPSwBWmuBwcc6h`Lk?gBm*#;Hg%b-^;?&Wo&4opccv*G%oVsh(et&4szG7P8 z?mnWXy!={Tg69h9yJ1K8IUpzJ&d%RWuqeh`TId@aL?vrGiEit?VS|M6Rr&No8yo1i zg-lJIdaxucWbPg;2dj#u`MCqLb$7Ju)Y;>+#EIM!=3je#gGm98?A_S; znBUfWF3nkYoQx-I5R~*`S0ErD1!#DEc^x8rDF1mKL_?@WU(7e=@a~9wn9O||y;(<# zL3pj^emWt)=OEz{YJ+{o zXYm_VIP%s{W@Zo?(BxWd`yY}?Md)=kd(I^Gqvdjp8nvD6fQ<{$i?gfQ?{Cfaauz&% zwmNkbgBra~x{nKoFS`@tB@$DeF&?q(HY9iG>BQ4h`SmYsb1MMSttH`)d)WcU4U>rR zx7|B@<^w4kU@Mq*S+re`O0J$45|iBjbi@fOL@J%gNaESz3lnlF@!Z0m8k1lm`C%#j zOs!O%i`@Nsq*R4*Z6ujok82(c0t%9tkng$uMNpHw|9o34;QY?pA^pc~ zBS*$CD^qv*A;)A@IIJ9)gr%cg;a@tswBAImG|PDvMX5gPdXKkpOnpaC%q*rV+gie) z#rGvW$*DF>i-zlNRGeuTEQEs^DvaiF`(`Xbo{aS-L1@i_(EbIE7Q*B(N(4iKW4@(8 z{^3wx;FK|nH51DX7h`LY(~HyR6W7o*&Nb)V^dC!$)l3EwZ0&m|o+Wdz7UvUH7cRKR zYbhH4tXMj}4O?CMM%_i1n0!%HMfaGY-}ZJlQBlZe6B-@=={AuE_234L19JS4IW_0Pacc#AlPa_LW36?V-#>fli^F7 zF%@7@3kNlk518xXxzZtzxs{1e@C%?^ep{m=-=ZOvrUo^3&66vi>sAqIv9Qg;bV%H< z+PR<1eb97?p1rzLW>UyW(oBijY1{Q3=|t_j@K2&i4u8{t_B5yS-g?z35FKaWJm;sj z>68oSR3Kcx3ui-s3Ar~fF@46$@$pi2!6JTcHDqIY7?A(&@1MIoqse~Dtfy)=_nOtn z{boOrGJ6iQXkYq+4YqyVcf)~|tKg($zb5df!PaM6m&g|K_NfheoO8tG;^3GNxjP)a zWcjqFvi93bE9tXUB(E>%0W%TP-$l;S&T=Lf-%U{fo9mVs=)WrJlrTL3&2La5bB>Y? z;?&k`jt{qnKeBZ@U8Vv<10Y%7gWo;#LV`)To~jNGJFshzD`g^EIpW^|C+e^LT$C?1 zkE`O=qV8k1QM{rTkM&1G3g83GLL*m_iE=++16Wr5llzJNMs)3GUWhBSyKrmkzt?!D zmQh{Y_$59vYuKo(Qq&vRJvqO~J!~XvYe$iMKY!2}^$mp@=VeqB`fj_BX3+cM@y#r% z=oqd3mMwB@{kwHfY%q_vr`GYH(U@>!G6KW=VwPBLL#1m-$^|fw;h3#L{pOf&XBZKJ zK0bKHrO8d{^c01?o)m1gG~KYCDIi_3F)!+sVh$>dt0pkOt!?LGFcF{q7sp_|B%w2S;-|a7 zIl(4m+IyX73Hc+nqrV|_=TGmdk_+kXZ&9_Izk|zzeMVqy5Gu#EKabc_g)e4^Fa&JAkcsGm> zjF@dFBOE-QLg&(Is#{CsxVrevJ^DKFa-9zaaQjsNj#^81t`G#TJt(0lj1vnLv%-m8 zgXiUYYSx~>7Rsh}#Wh#s{ecA|W-mycw$>AWM|Ed2gVJb*fYjX}X4Ec^v1b`yIReC19@BC*S_vIBF2Ax^{dz7WhqOCb5l@UcG%}ODQt{zu4SzB z*w?RAj*l!(NMz?W?B5xF=Jn-iN&*!}Z5{Bj0&n41lS6-u!!s|(HD;_r9Imgp zb=|T}(3jMhQ{>X=q5iS=VN^n{YBM3@o79Xh8CynIks0R>v_JJ831|sf7E>x+z>I3J zOc@!~BfrZL#NwpgrcjN6em$lW(i>bS0~B?-z;zV$sDnJUP%2U~x&213ZM`vN=vfW-7kxX2avhxhO8m!lr4{!RVO0(vCMN}*RfLs zqp%WrO%Zq;2Dy0G-gc(Gkf&3SxU^tWz%sx8Oqw2&$)2+Q7DExmq(otO)JFduR~xfX zk}o#WGYBN$EhWA8k$Z#kjrdy0WFj+b16(Uz4Pnh5ji;@Yi(lKW_O))lYBq@DdOw5F zd+rr>)bP>i0CEZ?TnPVqOa{&s^?SQ7y{0{B&CZPkH~fq!;o;l2J_3Bmo(yp^4yZP& zwop;x$%)JUOBp@^us+XVJfFd|lgO6D=jqGo1Ph1e<0!gkjt|UVtU@hxVAV4nv0%hP zlSF-`)1t;DSQf@zt@MB2`P{p$fk*(>h)cV{l*cKf1(OWY-u#1(t-KieZM8hc=dWoB zH$!H$SXI~Eg+*Qa;m*__P)Ph=t)Oq^VP>E}2|R<^^m%rWncW@KAoU}#CQOdYvZvI& z&n7SnnRukGD7?A_^Io!y9}?g}B3OLo%l<-S0%68ruM zv4iTXkh|eK)e<>t#2JG``eA3;)z6%W^Sp^UTceJAQiNXa8N+}d#a4{#2ATJQ7TY43c$mn;qj_2;SRH_bFz=vn|>H@ z_PSm?pK7|jZ7zJXzpS+#_m(GpQF*M~TRo5o#3h|2+}?Z~3NE{h5KLAdlU(R*3i;5>orcB0_NWwN=v18IDW*_n#)Gf_`A z$$Gby3aVNt5bBug(6HttDsU4os$pJwdlGs#Kp2Bw;>TN7vNO7X-#|qcVAtL_P3sM6QYNLje}e`PI7p54bAf{nnVO zX+?RGLgpQ1MeF9d;rKV3_2T;y7OAivg(pilgQzoFDzCGeVHU?-WEkj8A(k(g35%?AOn%UDbidGX{~&f!KExMedy%#@@IrA5>MW#t(7yf z8A45dHIH-^7!t-t7*PsD@y1?iM*{{tgjyG=S;*ckE~m|@AGzYI0uj2W*?v)M^~~yzb1bWFePVGgQXytQ$$_ z=&Jasn8j_-J0rTbjJlZN6U*xHm zywY()lA*hlUv8dN_lY9by%yBIVi$uOW(NIIIp5rwYr;4^RuPC zw-r(ot-==XQI)UlSE#9j8#$YHc;EY-gAyOc?)~EESjjU~r1W@BAH4V91Oj_^gV)hs z(dr^&THNjau#6EVhgH4ls$P>Kwp_C`)`E>7WdjFi#x|NoX09iac1j|(s;q~KAi#JE zETX~~;uu$+XmoAYemUZ56!=n^@EQHFivr_}t+c_T`ja=j^tZ=uyA68ng0!@Bw{&-R zNH@~mpmc+Dr?h~;|3ZA8-{bqud^7hj7c&lC=bU}^UTf`j&Mo=Aef6y$IB^sOuyc(D z!@Y5cqc|aslPxc4=Vd5A)n0}~R0n}s;-<8)4`BC9Pc%dm#$HO>(Zz8UCtM|}XgI71 zh(I@&8C!h3d7n7s);}u?298%5tuhiPEqmts!%ywfV;G9Dr-+{zgxU*WtSi09Kr|Ia z8WaxIPuH2-+Q@3R+b7t}i%j;RlnksT`Cr`+oET2|I8@d5*y2SC;G|mud&gp47q@x! z;7coQp%S}Xnlp&4A#vQ^a#@_R*$@myvz%6jZ%XzxM3>R384?TO-d_^B+@x)`O1Fd& zEZr!fvd+$Yb2!l@`{9tdNN9wd#OEVQ5j>vrV~|koE@*ZGD;U*NU95V!vHjcF3t(y! zkh4Bhy4t^)<~z_L6hLXMkZdjz`yg*QRT+}~`L%W%n(lS==5kk93+czLljXuz`xrio zg+fD@9}ipd%iqd2NDEu8K(o_ALp25#Qh^61twSh#r9PB;CX_mJinGR*B=7ko?uXHa&M51{I!8ET{W$A1NI%9}VNA!}rE7Kw z9ZXuN>O9Rco8Guq{XSdi{iT;6l1|#O*LX8 zo)Y8kfUC~82iaTnv{yJqrEWvwyg0DNEuYR?NRqlC;*+3B)9GKV;O#M;Z?2v1hbCUW zc2hPej1i)zJ=UndSc|X1%_4-{mh*NbP1S9#G;AQNri&CgZg7+S{@E#{?xwOm>47o| z+cBrVcM(Lub6Uu;m{54dZkn4RMDc`mH%Y;5JH6Ioz7J^#eAXzXjXklPp}8enO!4b@lIb&wIV%EbeDlOxL zdjrGRa34D6o%(W_&f}=EqoTQPCk3&3&5?ia z)7P7QD}xap5EVNMI|MkgdJp7>elSsycFye$)+`U9J1(3U#1d|OP0jnu*15d_3IJv1 ziLEZkudool8Cfjlsnt6uu6Rwqw3c1sm_hX+UGcWZ*w1^4VTokKw?m&8OwWR4$>=M} zQ=!U1Zg`kEW8lj}(nc@CkI-0zohw;%VXhwEnU;U9SSY)%T#OM~o zTx9_RLk8z&bfS;fQCy(CRr}(=uWEFKm+f-ARr+-$r&SF*=ld>_rkCI1u)~6bl2Yif z4N%KanropqVWl?^*BwQ(xsmF5c*RWe0}7zg7o&c$Z5vReB=|^>nB$$syuq=~WuIlH zHIX67b&pMF(urwvDl#++GI`-9AuX5FoT}#TOrYfl0gO27yaQ)&(L_o+Hab!MAVh%g z%Vr>XUqcW<ij||zv4$1SvA#AgHNdqtaR8vhc>8wnyv$2C*VwH! zs`*E}b-VVWOLS;g!IX6SPn6BfZB{`X$QV*kl?-&7+Zg*qZ)-?f2z2HzknAv=6-9;2 zDF0}B-DMJu#6|k-4DVKMiGx#wBCt|HVqjhB?6ie1z=7f}3hJ2Ppm>;xL`^GPCweF; zrrxe6v%R|3csGTSo2Do@^IiOnpJ4PLu3b7;XZGYo*rjOh@~;30MH}$NveHro6{8lr z!nbR91k%D;s~NWQZMR569i_h@6&Ky|r0!Q98$0f(_El`?_TKH(Hy3Ge6D3LXaQl1{ z1v3yg#P1^S^#|Zi??G|pT|5Pw7nBr_m}Hj(c`k_rLJ4&gEER@yBE_bp7^%ix;pf?} z!91wre7RAUg-VPJiXv$;CZbEop;p2AW;gp?{q-rTiTQT#FX~``6eWFfQog$)VD&=` zR!_?7IxcdBuH`U>F6!72CG^z1wOU55yl@j$%F88##}s9zkGgqoryj#_n@Oo0IUpU_ zn=WGB4;F+LK&9GG#5TI)I)HMQwfePh-+8j41a-olHvTqbt25!(+`DL#dhE$TPl)eK z^=3$f+dLuRakFY(2AMzV6>smuWdJ`vy#HHh<2T6aFc8TkV0ys!oUp(>r58goAIS7R zvE%txeJh_0Qvs{931X5Tw?b_t0toP8{JFLPHR&w5R1dLj8yfIZE=Th+rc17Xx?Z0_ zqqdT$7F>kpe$H$szt7M^t7FfAcEh|B+%3z~vFjDzjq9MowM3bZ#qg3%v&hk-wfqYV z_<>SR(m58zu9(=VKc{NRj2hkGxS`IDOmZf^^}EtT^GFUU^}}`(>84 z?Ity3nG+nc+d-DD{T#&y+LV9VDM$YB6_Z?J;8$=sU#@fgOEy1;v}P)n@C{>mHH)AD zbeWLi>HINy-^AStopHCE1_F`RG~bdr)rv+3vkB4~P3Lbp+l^~=$#*3A^jE|2kWx}s zzVTwte%}a8b*?~nj7$g(G>M16MMN`KVyjnf6>1~ogs4FMUE{+q zia|b;oMRrC6TxzG>Ko)xo)A&&3aL&Se`_&Vo)~@|=mFH-$rlqVu zY>PZt_L&rGuk^|v2vRiKFxw2gkm>s!T|+PZXq2Z z#_Jijm@;>cR&hLjMmJQgehTl@c-n{P0aMQ?Wocf~9&MUP08>rwIOWG{r;)zP7&Qq| z3EIA=`b;hdREG6ZfH*#Zz4t7r%t-S`1T_iSjmZ-YBEP=yF`6&A7Nl(zO_3tXnIJN} z3qd=IRB73T=sO673){fL*o1aivW5?i_~b|&g#fNeMf#F8WUg;mZuF|tka$tG0(DO5 z8)t^JkCIq;o2{JJR>-nR>mR-M;BUM*TGr{cv-s z*DV58+gCCRXRgFqjtvc^Ch^4dbv>~1rm6%w@`vHcC+x&>*Wy3es_`yL3 znfDACq;C@2A8@ZxrFbx1_q(_F67A>|p?yR}|3R5<7I@Db(#EOMKlbrm^=VAV_t~r_ zFMmfxmL5Yv;v;}Q<`dBH{@zN!^8TXwjbYBb;>PtIPbp8LsW0CAf>%+r>OWbRg$pF^ zKt*;W&ixs}FwB7?@kZ?GliyyebM*&r-@P04^9Cr*1YE7ArpGgt64|VQ+Vca5Og(-4kmgGzfk~#kFeU&XIBA$J#5V_JGlDD!d}Rb-i+6@ zyo_0MbFAo5pb(zdX@N`XNp&f?Qh zU~?z(0$8p9uum*4Xir=C_eY83<9UZ$eg#2<+tyDqw*-MI3M)2tFrPF^kXG^NwVT{U zgELBCl4LZ$m>dIMFAe=lefSKDdgLvd1a(LHv( zPr3dsT~!@4et{)9)dQp@&(uonxNB=?8RREf1g19JS?5EY8&8Hoa2`J1&lx+)gVN6^ zQaq}6SZ?dL{f5&^dI&II{X3?K+TXcuLyjt1s=@a~cvG z-)S3%n+p=F!GN%b2Hy-f>q=se;uve12>Y%)+2Gy04x$M zwVQ6EFY4lZ3hGNPwDPC0@Zmv*d1+@}7rx-P^m=)=Fxv#r3b+mRk60Feaj5vCE^@8Y zZy0i`g&gD=JEPk&G7OR0kUJO0_G5{Mj>(-=H0t4ptB1~dS03oSATSCo+t|Le$?FbV zR;1DIhU4f*hkX*lboq?=CE%j6ip+B8JTVmiS9^J>9-3&p4{#X`p)S=79D`K#QOy^S z^c!(&ZWp{94RaKZXi*=Cr}(3%2S1CXYOgXks3GO7s89m*b~uaM>^K9ccCS!8)o6V_ zx7V!|1xCGHaxZiJI)0(-{#?V0-;qDNm0wZ}b7x2Tvf#3Ywufc;HaujO>J_ju+d^Pn zbK1LWnt##;aHTxB^z2m6VCuuEDebTny!00aI(1D*cH+tb*fHD3K`Sg8!=*7{dNfUm zg;xIDVHa2n*XGBZhmhh)~;Dw(`=S%LGLytB7;y>6V_osMpr z*F5t_$1HcbLSj=rh3X2wK@Gh0YS}@ALAq5rVw|nTBW3oSRjoMH5Bc`(#!9j+>_n0w z7h&(^qvOohF1ILYU?AKggZrOj4=pi=_7k}-m{E`im;n_N2RdTy-%SKOV4{F8=y>ET zI^-;-Q=BE)b+G`bZ;jm0-c5oe)k@o7p=zqO(HP}>8B_{wZXgtsPD%11CbM4Q7sdMA zD%Vj>A5I>d5Ko!~@fL9_ZiZupyw0E$>_JG5r~DSB0!jVxVvY0Vb!Pzn`kVT?tZESE zu$=2J?At3lqe%%K%uwWgy9}__!MhBB*I43Jg==VP34_-h2asv3c7*tpU=Z-)z|415 z^$$G#y?;KTexNUd1|r|*xp>a*N@A`VF?cAUtbA3nGs`W}Dvb^#myUs?am1IjjVMR45k=5scY_|8MXa6uvYFw_m;ZG6P?P!%00%PDLH`m{qc2T`-W_q^!0im^*_ENt&EL2Gc=R6AU08M#6J-zZ*>vZl*NV$=&Rl5!45$i4Cz3BI9rK z`c&r5$uAeTIQno!_ppVG`$w;#nbwa1tpf<1q?WahFu=k=0Fgoi>ZbT>98tVJ^l|+k zSi^)08r>T^X!D{JrTfBVs$3=auJZe&vM(#9aJFXS@2T!ecbd13xMUIp*OFyTLj?|I z69g)Gk(7d1*aTKiUXO3!)_v3DF(P4Cqx~{lQgz(@=f!P9JI=8z%Y=2xzX6R9Hz=}p zk}Yg3tRWuRth+7Mqt5Q))lyw>sh0;Hhd#IcDQL;KJi3G}h8qHuB<}j8yJOIRG6sap z5>C$V#>#I%{j_5xQb3MF2JDCnw^A+xLRb~aV*W}xENB@DSbQaBbq9&)RvKW@Une*? zu4YsQ;}7WXS5q*It}7VTHG5|0B|5j@M0}iXe(hQy-&MK!uox zzX2i0vjg+V0PI;$H@R6YzI5>V1@c|1*mK&M=6rb~3A={oB4GmqasIRgepQK%{vo}8G zNw4ox>zVGDkP4O88xf)p>$XYmF5KQl9FT=9w>4zwP4O6IR3^-RnkgHI{G~_FxNzrQ z-S1%Yapu_c%A!%Ox=bg$Fqj@0{V9PEBf%)NECH>E<&N|s>iis_OL;*Vq;bIlrk_pu4JlUd^6}fi!L#kxr%=O2-E0KbELYKrdL{2YqhSWWIi{q-e$+6 zl}rQ`lH#n_F==omp8l#AYN1V}u2ux)lAiXt6x*hNxJk83!V_%|h0W`__(*aJ@XtLD z@2A~nw%HM}Gb)c}LJND^Le3#f1-|FFi#XUdJaE?z?c(#W)Q7t4TS5QRD zUdFhQAzSQz>D8uWN7ht$jBB7#G-F>eHsEHL9HDq|CvSAmJ_I$UVfV zS5g?wIiqr9Ha3M>b9tkf^=>pTP;p8mlnfrtAA+Thv_c4IU_6n-5tOV* zKNC4k=Y_drXpW;3lqBh?I0Iq4`m)<0VFtyFsn^*`K4C2LAEAy3Uz4 zmJYsGW9JhkM{-sMu1X7&=;A`#U&L{M=y>-8LPN=K0m|j~+W@_C zOdc)6&#lRs9z}(Ng)C_V?$4^A$T&j{uc{UhHza~TOkI)mQ|rz86`8^-k30Fd16(s+ z)uSeMZWUKQ?@3j>GJB2~Z74{DYh0pwY9egZfIYp z$;Wu)d^(n3?FqRJoHrPB4n;zOEtkI-Y+;?qR(dH#Wb&DB@XWjo+H3vhx|Wp-dlhL6 zY|#IqR(}YV3sscx8TPRL1rnFVPP3l}S>GwR(OCZsl@@WQ3@OTup_Gj*&pSL;%-uwE zmUKvXgb3l!C8S>t`fz=B*g5&v&k^M%=~d0%I2b7DAz|?wCpz-t$B&;NsVu}KTsH_$ zmai$OG?rGCyNe5ZR&3RUI|B-gODG?G2!LSS@l$1_zCc6$UIw3D;Z^6vgSrTf*DPDE zgFE$@FekAMXRWhlOqttC)ijqAOrAZpjR8;Y3BwrgMmzm8BHfTMG-f3o-FuTr%C_@h z!n9WaKEeyGCf9MLbkV0r9-xIl?yKH+?7;rdN`oUe2oqxg;q7X&vN;t`;?^IPQ!j5? zv+r@mS2JW7tbZ50W6mEbVns)PIWt{c90nyDA1pqe%;dM-uDft3<2rW(?le^x%IGb81Hcg(M6@XZv(Pr6 znhf*Xg#mT&AHV@TNFqBQG;N5~r#3hBXs1~=_Cpqkgu=%rLZ{CLQcPloG^@WS$PATE z?f{)Tg%~o2W-I5Z1{V*n7Y{^e7jQ6Gd*Ty^jvJ^FueLU#a1T zHmRAYa-c!nUp)^haOl-Zj`1Z8;h>%9K!>xr8QH2yrO(Mdzdr_0mCtWbl4^T&R3CIq zQ~>uhnegJ?89IPCjZ-(O1ogUBmF`y&%GK1(TJ}(zWBJzQXT*nP?8VM!GiHm8Dh zTob*5ItWU(N>ke)$G2R$2D^)dm3-qNk4tN*FnE)c5+9xJnEJzWSVXNr?oMuwsfZtp z(FYnxm0j$brT^%uxZgYq+30Vuq%O_bnY=?q((7-?+i^cat zIL=kmBa6z)z606Nvmyo(Z)(%=#7Y7mY};=La8n=%6{9MAzYqI4qiRgBJB0V`c3 zDqIOQK*X2g45k*x`YcP@&X7BNL9YK~IhBtIzIdMPmUEhSMd^}_v`PW1YK!=Ua=rrl zfj#e~x@dm|$e)Cb;Q=K>8`$;hq(V>B{Qo`!ybz(Pga>z*^Ia>1Qa?QBGUwE822vpL zn9htZB66aF(W@rcDc>|xGf4#tGU`j96G0&sKQGui3US?P{_rcS65H0zb1nE!`4G12 zIKPR4@DhA~m`eQP2AZTP#?r7NG&m!;6xBBG&xk?RfwDZ-Ub!icTa?LdT%Fbq5&aGSVNd_mr;xUS{QBS51@p;+g&8)$D^US>b(gf9BvLXI*2TkM zWcXEZLqL)Pg~Q{ot%h%7pgGYnOncm6jM{IF5@b+xe+Dd(4Cb+f&uwyU%WoQ|G`{l% zvj>rJ_792PAT1?LgdQqF$+FkaZRggUU+imMqMYT?a%A*6v(jn(p|=1@$P0bhvVeQ!3X+9B(6(CHOhM=vV1OYcse=3R%GNb*2 zIl-Cjw|BbvbF-y8Ga?LRrTWVBg)*PO^3l#)F65GvGvzF#wCL}5g*h1LQ1?Dr3P&!9 zm-ESAy3XMieAM;Zf56bxp?Fdds-pm;OW#7n4EskSc@Fw*5fF$l*VFXK^!ltt3TKOu zd9p=$x{pV3crtYF+t}-MOQOrO>Q1}Q@K`L2h;h%zloj+x2{;BxhgILFUfQ6siUM8f zCfAKKYj^KOydgl+BX;UO^>Ud88mKeOQ1v2KfG5ImGI+eM3cvS?^I&89>&Cn{;QaIP zCk4tu_JD!*UrU^%3(<9k&G6<(r(CR|<)-x|`~u&W50$_nP`Vgyv_n@(`|^FB;|ml3 zn|9#TZa?2iP^--}-%jI+&;;}_KG#wdzq!pawL`2Ij>F;scP$BfSpQ&TtwkP~ji_b* zrta{PYuj!d6~bK6!lTRm%jqG0$v2$l%b{DuR&Xp3g7d3D1CeuvdIP#xlYWX> z{`s~1DM6XKK>s>o!B1NlD1gm0KJWPAIHYcdcT@z4t`}GgRJI;}id1KSmIxI_9G?0- zWX0s1s_I4Vzm|dJLkU-p(APSM19Wib(xFQ(ghSmlApRLf?jkgv&CE|b9#FuPAkcMA zMCwaszs{$ABN5!|=TxRa(T(;aUkt6ym}A=`i|kNbETs@F|BlXDRis^2rh~W^8w&N@ zgz}*$0Tp-q)Tl0Z!%Du2ET8GEi^BJF&GgJR%Sl~~(eB50yPkS81E}sUEXce3AH^It zk)SKWxinI>cK~KSwTyc4!@Y8P#IEOaV2Q_py|eMQO&S)!d7k9J_5K7sSKup{v=Yp5W#KHmWs#0ucKuJ0_};8yj&-9eDH=5D8*s#pxHN$8 zJ~wlSzdvo#4T0o8PL?xv+aoG#JbA*M#)xo6%i`NffsX{z*Wp7G4+9c|Q9jA=5 z<^*z9nQ;gXkCeisgrnOdu*27nxdSYNtxLsYff>$TaM|Y#=~-!8fHnGef^ms5`K(2n zX|}8k)Z?PyTt$LjYTC;TN(TI1=R!amXp{i@8H~#ua|nNdRZ}uhs@|elOVvc=IP)e* z3}-+!o(@$bK`K>aH%K_kZ9=?+4d`Zu^7RyLW0_V3tW$7yPDdg^_diEDkFfXa6F}#- z11_GBO9zO(#K%;*{*`temo;?jFT!+OQ=X>}Q&T89YTRFGA*#q(SZzN}-!{GRB(6)9 z)9Tl;v=ia|(~{sUfqCKuK5o4U}@00)rN?x*5` z%^mP$8u|C*Q%VVXWmroJI>m5-SJKV}*r5c8aNcXiSo?U>&-^rcHtF3TxI+@vb;$z> z6By5SU;fTn#BMNSlLJ0AGdgzO<`c)z4#RMW^^~gfKytZHKIkur|{X zX(|8Mpa2O;UKM?IJHd1;sewu+dbLV{$E3+&Wbs`GNxaIxX%QO>ayuHxi!Pr+%bzp< zA81SL1REtJOoFKu*iu!{K~6&EU)rW#o*=EyA?%?I5vGNGf5>vT!AU7#Ar`Of?V_#v z4`Q8s-Y$Vr=eIx;(KpjOFH6Gna#3QIA%mz{y1a=9tVF{I9q>7xEqiIaV)nm?6O-bckqZE7A8a z_<&Jdp)0mh1jybvnu>9RvGbY18c zFQ;)S?L!Sx{-I`1-ODY6-^s@QRw>zq*|wBNN`H#P5{hrmY^PU_T)Zk_3s7RGqsf43 z0wR9%)K-bS3(|PbF@Jt`{6fu&QzMV83wMPQqK|>9)#e~uqn7MS*7Xy~(KBD|@^Dedxa@&lC*8Kry;q#I@A!TqHCJk8sABprv z(AzeP@Cgd$(8i4N-h@*SP9#@HGT(Ba7In|d5ly#mHLveu0ms$5 zWIig&$F<(-J(4e41)yk*c%p@P+W=Y}Ywk37@FwxxN1L48J{1*asg%X+g}$@VFJD=p zDQ}1YJ-NO?n4j;&T*0zJZS<5S{<#HIodB#xX)6aa!e0o*)(;FS$CT(a1`o%Y&WZj2 zURmFkEZ2)%H<~!E6kX&rj#n(VH2y~baKz+rC2vQiU@K~7TOxt%a`*_IPLI)xnFc4U z%BP1prZdS)-B)thLy+ouL@x6GfIN0!IDOjAM>nb))<{n-<0h;fr$+^2-b%yV?`gpqt1cNP=@Pi?(i#h{YQJ6AmOlX5iqtOuEmAqHd_6$s1du?~KV%ly_{F zw{PaJaepNWSM))b^=6-xLFCDD9r0vf$mM!v{9n-H4N^rT)(b`%Z2Igz6ZD=VI7!Z} zW13YmTTEYBy2vA2Q-4ZA{<^6mqQ?o#;CAJUi0EM7TvgJCz(bN;2bRDVR_pOkt>i@z zYOsJyuakkE#`VKLRkaExk75hkK~C|m&&zg>ec>D40E41BQ4>*czq9&;Cv}>{SIXJ> znbM;BDLL7}bd$y@3vQs%0RTiCJ{$A&Y9oJW`42^ky(+{}2SCZ$x+VB@|0i(&XbDG< z66APQw~|z3_rZw- zJbqjrE_@(7AP(b-;yvV64l(%G!p8hhdU_P##vh*D4PX$alpkA$FEs;W7mJi=a}aWA zEaV$cOD0puyvFVhmzE1aFH~SB1xMCZj#P;{YbmI(D0kZtP6ZDKr^+aL9U!d zQZaIXZI$uWNQ_+3%aC-5HAv3s!kB%;aNym}B7unh)%-aL<*zKci3+aS2%NzCR=!$G zhwm|d1RuW8J=ltQC3-+9mc7!yl?j{R&vs~lY@%#=IWq5@YC$cs z446xojDzF4qwgQu-=NoJyoKuC6K!My$PzA1od3HX>gc)6km-WRqa*`TmKo+D3dhYR zHdfXv55!`C6Bv5H&08q~xMG0niA3yK=`;Sl=5HVes=gk4xdQ}Esq_jiGBuLATJG)d6gFnTfICqb0}Gbd!(quG~QpAP39xNV@ud7 zd=ey(Aa2qO#*Q3c8tX*YOVMMx@z=bjCtJ({YW{!H8Hjih29O^p8N(DeENY0Z{sjSF z0n16c#CW9X3q-&NK)QE-=W;j0s|zR(gFdD}|He`7Fbtu*5YCm(ZBb;!>VRD}_*iH5 zN6))+mUR51n!7FOjO%ig%xY{jVl0-scd6#DTU=CvImKf0N(PO7>>`>>+3N!?8oq?$ z*kcqd;J>AZfViO#tE#W4tgOt=_7TW~fHiP!&Cv3D=Fb1he4s0|ERG5a(drBuz|d5L z6H7>MCC&aXCLMSIv57B6bpN}+fjjbUip>aYEjG(#3MD82vd}G3vv4aX_s317D-QFa zHWW?_HevT0>ZB>X&?SGxJJ zZ=XM`?=~_5fyOkxvMlY2?K0lWYFk>#YJc6tZj538txsNs_MPzzezTQ2eJ+FY9UL!p z1<_M_+T`*PB?%J%?mk+qw;nyg3!eA+cP^hauXPe;3q*X2y6`#D86jijq>G9BR78bd z%+Wh=HmE~Xh|CSJV4_RMu;ux@ zmSXL3^>UCIaG19->7Rf3H5!ZuLvaW&w!F4_W9LL*;7aE4;bKQT1bRTjO!aZ8N`eHQ zp@PobRH~WVa4!Wnd;E?s9vq1;N4z+#*p%7hOa6OP6s_cPA#U~cEy8zBpF7Rbc4IG? z52~Rzy76RF%^_3r-9yN$fU}4R^R85v#tOkS>2m4MhtNz-7|J{vXkm?Yv%j=A~8X({z)34Mv?6fc0W0F!Z-5NiV zd_WplurT^AjPZrVCjo6g=Ts`TS(8aK!Vnx}>!o#sM zg&!~{^dfC1l7q0ta%Bp`-}d&N7YB}0cg0|T7i{Fl#)82GqQH>hseRGTFy0wn)Eqv%k6z+TP&|uMu!QP?K@(&k8s0t_NF1(X4A(BkHHI9H|naF(b{gBj}e!;c6%TukT1Q|kG@9sBq5CY?`Vxd9aI z>9RvpKE*F|Rl7h?$kgts+;w~g8DFL1+4--y?QGS8Q@G9AJ5v~AH1XXj zS#;PHM5x_My+#v#O)+J|+lxG51h_HSp%6e@5ICw@+3=}Kf%@XI!pp;*KP~c<>0zBI zC6%qJUr*CU=$OUmmgxH|Jt0B@&o9-yVWTS>qFcCjP^% z<`(PQn}`4rA-)1uv08KOw$(wa>l^U;UfI~q_Em~LDqjQO6?o9pwY#v_T(9O$BJp{I zpy2W)z%h0Zn7SvNw)^AX8pk< zO;kS(YpDsNN+Xk{09l`08kBGd#q&Lbzb_yF74a~2E%}wRz-NLKeriV##2LOE!^g@mj-s2EW!R5v=N z-CmLh6SS_iaPNj63Xx!NB#c;`feMV%D zL5t5PBpR@d&3r*KGM8f~*GSn07PpxkMs$x~9`@8+g;+fpVRhuyW55B24uA)EZPLW_ z0G=%1!?s$5#lvd`Y*?USSC2c@@wLMH#bNV27%92Ui0Nk@@Z0(Bf+%!0Ey08?d|>5t?zwC zVk$10VuDo90mLKt{%v^iU>@=Ow)*E(bLQS#Km&-h)XFKRS0BiqI(s=2go@c`_NK+3 zFVvr5@+vk~pd7eYX-Y(&Jh+a27WvVi12q>Qv-GqO@HeyWZGt7!^}kB+U%lj;N#QVm z{dhBqg^5|s0k8GJiKhj)j*3%(&8$VouPiipObG8k+U4)9k4*Z}X8n-QhgDG>W20Sy z)@^2e-snwpn<3jHe#A4KJ4*!fKWPRWH3T37CyYm9LJp52m)XU%TdJRc^Iv0&6R{|KQ-o^mU6gnqGP=pO zp>YE8aDh}gt3-Y>`wIh~4v442wBA3ug{i2*M6tDK5@~zSlp1{>Jjdnz)s-mKL~4bg z{&xbsI~X87_seI#QY8;ENVG=)$3Qdm|4qYyDGTNH04Fo-K*yEQjTb~mW29?T2##I3_2|F~eqb5top z@Wbj|=l2EQ!q0#w*#$UpIR~*a#sg=dST-B7uTomI6JZz*a^62BtGTz!f5=gnVmZG?&GRD1H>;6UKf|=qSarsi0ckt3s zxU1+en=hZN6z)WiYjY?z2ExW+)G#vCz1jLz@+7a~1|4XNswEDYm`ex0?qWmzZ(d~u zdFNGlJ9QlsWd3OE^RTj(;HTyA_qD2!(2G~2G6zYZqlWfCs9{JOp&w+JcJ#FF#G^p# zdK1b$kr2R3N9=Ma5_zYIFk5Kb`bh|iG3bQ`rDWf9_W*W7kkMHLIrj`7=eg)-2CI!r z+?CvhCN=DsZZNrf_2c#kL7T_a>8cmOytltBttfHsk#f zt&M272vN+g(8PL{2u*D=%Q5v5^nOPk=ldDR&s}d`yoIv$OM_$!*2*saY^)?&B%{Om zmO7eds1YmYAIa|Dh&vbYj$@9@{kF58Y%4GGlVIb_2vb7^lU7esw*Aa;`2*@GFm@XI zXRQxj!g~|&2_2umAS~f4l@s+~ft!!fcfmD)b(Fsa>{Rm2zAY&5(=n|F_Lsf3I``VI z0_i^7}v|zT3$z!j*E6jbz|&rSXU$%k`ulzNucSCQEG$;{)Xya zpM~=jO@9qN|9+G-Og|AOGllbNe#ePCN1eq1zh~bL>g^8$ z^lFfxu*AEr0|hg<$D`QMUvVm5<0fg76nEAN!$`P*c|mG|d*O7!)`#t15KKdD!p>Hu z-2F4tAAY}*4EOJTK9H9{kiOu}!?zj3u6+jWW>`gm@p>#4>UEqp+?`i)<3E)@bm8ys zBIs_}hZ>VJ4t6(_%ezx|dhRx>|G0Gegj=E6mw{2ygx00FbSIJkoJiv{;mO~x9=w(3(YK&+t)T70c!(VaoU zNE8=K0W@1Q^!KbEm+`Px?mL}CN^tY!J+!qod+>;=wPC5T z*kxD>+ z3Q8P{E9nBJi6k>RwW*cEUATv7aWsSv)<1jsGVib6PQ|(aH+ld1mbJ2delcGV&~GKD zknbUdTFF<4oQ7ZxhweJ98q7``EyhsbD?wgff{-X3LfZZU&H>dXLh@aogc15in@oAo z2Vb~nG%15d?0;xjUt&2$hpZo67c5dO6rR^boa@!4i2VIw3@IXMkIK!m)K`k~VHlI_ zK5W_>H6UF51-T!~hvj`{O~O)sk8v+VER}nCiPzrf|;p?l{T4NYKE0AzJ1$v zMz=e#rZTQ-+vxbS+k2Rss`Y?tYTdK-_Upk>`@?J{O`Qtrg4#eHzFm{=PI(YzL1EKr zu6*msxqD*u_(*w)HU0s=jix$DEEc-7%(vs}=EInjJmX=+L24#M+FHSARdnUvUox`A zc2)*rY#S~~iWXw|L60g4tsy%uVwV=b#2TCANT;h9p0wuweg-@t@&co=#UAtoA3urb z;i{H5X17t56K>=E2rYLE8f_&|fz8zzR^H#@J1N$vZz^zc@Yz!}+w>V(+tIy|C6zPS zWV~8K^2}?>CZz=>Zn29x*(XH?3)SbiYBy^iIlX*YQ3N5_=+S}s1G@LiTH&lMoiMHn z&^U??dtfMv#ts#^q_pzO8$$PvPVh7qViKZ~4#CHLYT1BpVHISAY&)cS0>cQJK@^E@XAFC~PH?yF+v{ooKx4y<#h#v1q$~0DczkrjMnJ_0g?wlTLr|zLR z*H{Js?pS9Nn+9S?YF_=seo=kw`x+{I0vkb@Z^43%4`$jt;Rkil3=C9OMoMBr*wS#b z{G0_R#(`cJX{{UNPGM2XWx8#iqnXE%m$G4|U#SgWB$mtAU)wP4#ADz(Ob2}HQ^E6q zLPW!yuBBk+BwVeN+*Bw^p@?@pfxg^7ll3<%)e!AC zA#g}paZHMgAruP;A56F`=0c*G`ALzY?g(u`s$0>sVxE}ht}^b#Z&nj?yRh$IuKLDa z*QI-+5Yh8=<-w>)QJMuKH^YdQ##djjd4tXCt{=iO>1{+2S;U^fYZEs^>08x_#rsO5 z;t$m&COOUh*6GOe>o*DCE>Km6_YbB^X0b}ObF`F<-%7q+gnpGye^vm<#MOb`9H2(8 zmJLs)WRV!oGJWj4i9;5oguH05Jta%B<&OU32#cQ%(I{8jQ1Vq{DAwzJ*r04T-Taw7Sx!Nqho#>AlH*>1rU_V8`SGfx1w#q zr8w0HLhs7xC|L3}Dv?bU!ULQz_q}zcX>%)pA9^G)x)N!lq1}>^07PSY6d9b~q6rx5 zw`psU7sgvb=4Q!>@%kSh%EqC9SM849vV(3L-0SFS7>bI}Uawtt?p9_?T&!nLVY#C! zKFVqm)e=?Z$ky;fKV7wi{zV3JN)HfiNF!Dv%GrRsyRd3-qCIjR{vcIN3{MBr7CU*t z{eZW=HworC3HeH&-8LnBX89nWEHzgbl=cvf=Raky8$wSFHX^UZxqeCBkq+ruI~QoZgO?8!rXE66kKioi-3f+1?ni2~E_D8}uO+4#Z3kF>pp1 zUqCF^2F_hs#ZdC6cm`osb4o+!D5%MOZ*Mk~+hB_0u2(Fg)}>OlC~ZFc zvptNxG0zZ%e40L#$!W_jbh4^7osxU_)Dp0nPJhY@g}PD1Q~9+yK3kTUGF^mTbkwrf zq{L|TeRu6NruQx+DAgq{h7QO>;SNM)?;n?$U6vRP<#s8 z**pnc`dFD{vUrvLlTq$jjaEyYm(~~w<_rGB&R|r(X+q4nxGktmDxLPH^+lB>|N6Xd z9K7zrN3`mtq1R@qbr62*>5tNn^F6Q^^z3Ae{F46HD34D%&7?xOh>q;lobA%k zXm4;K=_mJSk7d>rGrRu}ofE;e@MYgo`1HOMi7%*L@vWM&@9T<@hR zE6RUdl`~j~Lvxz~Bio~L=91A1Vh}K8sRcPz(|;)bx(c|gzLn(~=1{g=#XjpTb;*{g zi1Xsyq$L5EZxTp13fN_4AJu=#YK|vO{prAX7Cgg7w@T=dSm-@-tt9J}%P9Rg(vtWz zvs0`qU6nPjWbmt)EdzI?JGYVIK>B5Izod!xz%58i*`| z4APi#&%RSkdvYppJif0^c%Z0;|1D!^(DpzyK?t3rUjJ<_q=3XGSEY5$PyL{sxzQ`ZWyP0cU*3N~b_zs|0A8Xk9C z2j(wKA4XOdeX0w4-Q>C_R<>v(mc^G1HL$j(YkPftKC@%-N^1aaO`hSkqZs2y!yK-g z?%N-q)yJ}S=06L?*%L>{is>P<0jRcfqQuqhYUAoWtb=qpnt7{5AP; zm_SC=ibGD(;CC`%>+!83pfqC)ZVV;3#bDTZ%jKf=>g$&?{|5@JSAk1_nOAl1MS9sC$Db=#e3}r~F zjZneD9G?I(IBJIH76MWz+jY!SWS3eS7ZTkCKTI!*f&`Lzwso)(Gtpyy?4;p*0MzOY z;|y|vK)5tbZ0s+qm#D0{5n=%B)~FBPvimJuq**qV!sck`BgK4EAEi@dQUKmhG3zx%=HXK-HP*Lx+8l$ zxMlz+yb)bs)}pt1O*mFp@eM3+_Qr8m!NW1KQ4mZvOIEoo6$i7^cP1cG$8?fRCWtlZ@^`&|M&wKwa35Qdd zWIp(4w&U|3@2YG7CcB@Y;3>suj0aPw*8BPb?S%$l)Y>uo>(Ak$C{T=|L8tJd{5NDY zyi)A!3IT$i%Z3yxnghxrQcU{}&n2&$$_0AjvZ zhFf|Ew-^HkC?BEO3;D0T(V2uvbOO}1dWQ-^)UjSdA zDZ#voY@nOf^Rv9Piy=!5i+uhkv>aa>$GUbHGcfzf+^-t<@&Y#vwu#%LSl@%(R~web zjN@E`>ms7PIAL{=zucUCg1%plTt}re4f8E(a*$8&G?Vbd%=~@o>o3ji&TGlUnWY&L zT6+TAJuRs4hjV@($5R$g^va9V)iJ|G2owLn{nS*AhAOKlAwMY_V>D}IRsch}> z(dGuDY~U+b%cIY;AdfMNIXf!uH?JYGE$XET6Zw6Zit!Iiw2N;d^PM$=YI_-+I^gL< zk-mK6Z(4vxFDrI}#n_gftb+N{)a}U|2GyUC7rr6Lfvo})~@Yd9IR?0+delTJr|0GU@ zi6^(fdy83JAHiiHj#QsOk23b${N*=$K`dah8q)Gl^PpTi*2JmV>y1jN!+IBbEyuao z2XODT)(_v z^^8M71IYGWBKdwrtRLncyP-YM7R$XiV;0J|S-J|^u&FU}2l~am6)fV?*%Yvlv+Di5 zcx(ORleDNTU%Z!w-ds~KZ_u3K)Hc++ebPTS?~m%pNf9MhY2h;>FGZ1_rxd$whU4**@JXNv zne&R2H82=j$AAY9d7C$K_cl*j=VNv^7}SN4PJ`m%+GX{p<5Nq9^wOf zE??ZIA~l6hIFpS2AMoSdQM#1ZT%jExOOg{5%oN1!=~LK&&^h%`{w#1s_|I6M{P-|Q zRbU@qOLr88F>y$P zh9d)Zczu;0lTOGo*zUco>RhSs@*uX?<@{vZ|4#dRarh=T{3N=zqs`u^pe5fbXB4b) z)u$62(x5>0NfJ{owv7NCVfWN!yk-f%8J+Ic_Cq~1DVwyjuF7+XV{V*FgIQ9Nax&p7 zt1ui6g^Gj)E2By>=poeZFPd#uQ|4m(yp-=Xw^Nw9En#Xky7KE4o^&B%_rEV!o;b3_ zMBZl8=vn1daHtpmK?zp!*%3W}Lc=guhdwgzV~+QmnYuu3Uulyg3eKa<2k7v6;A7qr z6vd9HWQYGh8MQZaTIpq6$K?#+>5XSuUUx6#Nih@Du->WHAZ_?$3g}c`Os@ zpt?+xhXvbzS+tfA27Pjm2Ur*;Y@y(-I=-JU6nkE@aC7fuBP<_|K*UKwMbHG7^g&bYg{*KkbZYUJcpedB^rrS)KKnM{_xP7^?JSwA*szvgYf(G7yNbF&)bikUxlKaa*gAaa{4b_M0+Wn%p==iE z>|e1FU8I^Ry=J49zA24@f-4;~n!aeHvV-8;XkwHW_+-3~1i*YObZpKz>(k6omKt|m zP*)Qy%Aj~L%xCOmmLZpmPo6SMZvfztw%^z&TLtm60XY*uqAsySx?>@QJb*wNlSFkJ zYFPNfmRw4AT~#=jI2@Higb2nite+(zTrHlPB9o$mv-?SxOcOJf4C);K4>_b64l}dx z8=uz5-!pkS$P!gaT^VPq0j`|9Le0!M^Otp5auNa3juv;z<2UvD-&tymNoM~sYrC>b<{+x#TE-);*(!?qQ`oHm1Hmk zYmgAZ7mr>yK}pF?P%(00A6_MpR1pl5_=sS6_5$FqAxF!6wu87Z5vfoFGv8S27%56i z%h)euQvq!)a9+>-gtA!yp34-NkKkBSh+X$q|$#7IJM z#gteE!V@@)S+|e0XQj=j_%ovGg>k`b$C_PS4v0{*E%CNFJ2&kV%SI{ZfTvYkG&U&; zc4L~S@aj&XJhkj*Wr(G$7xY+x-)Ux?bBs*f4N}y+4Cm1$Y#ru7E;@Yv zc`aJu7kE*Xaz3c~i3Fv_ih;84G7gs@I#oyt);c5!afxD0_=UmzGK)=P01z-r{!9ml;uwfO1J=R* zHZLkNv*s`QRgxQF>{LrQ?!MF|6nIGp1Cu;p<#_{7nszCb(P3QEsgVqMyF}%ePfMsd z}NjXqzG;M*?Ej50g{=6mAf_arD>tSs|m%xDG8Fo~e+VrxDZ z8`d<;uRTj!=VJ=uj}$xJIL$=XethPLv2?vi=>^N*ngQ`5hU2OZe$Qm)8#8jjhA_(ue~bw1Kf)Str?B zJ8^$ka#jpZl5W%OFP{5ktMsY^Q`cs;eIhfXoQ69%MmjVN_Aa3@G&J>EZ|BXd@vb~r zg??D_i4h!5(3apk$vHISUR!@UZSGtRcAB?v_JfU0WxDP!oNLY+_&QdBIo?IG;hkE|HOn6;>~Y|Jy{>-oE1t!=e+#n9B%h;d-W=RYX%#pS*xrT5X^6-w zVobM)HS7djAEQ-5{xJwWKi%W&RU?8Zq-29Qe%8c}r=^%!xFI%e7QWk;=2Wk)T*ghq z)Aq?*(+(5PKwkTGE@agz_SJwQQ)T*Zg=uGFiUWakkh%hKrd>vdHCwSmlu|jhl{eN> z>s_1swdE<>n^7UXiv}t0gBxN*s=RM)f-A5f`+nnsf{St zz;dmIZDq@b(SUzXg3}uUJaPC;R!;Di&AC1Jk|5s_&5;AocY}UKX_=PU(!==ZOZl2T z>siGZRso~42$-X;x5svu8O_^bQ;@r)916vH!COyZ?Hk9sr+!x*{C7PW&Fy2XHT}*= zOFXP^j5I#JCVEj{=WpvGqhzk(#kFFN9*h3qE_n>wol9Z6_6YEjr%9v#3$u^4}lc*Kb?p!*En#N;tpdLi2`ZTD=PihV!!9?Bo@G z%Ns3{#kq2^JL2iNjZUX-`SNjDc9r~Od1M#ZcjgMGqF%;S`xG|CW|!~%OvG~uSLG^kwkt_2ffPg-`9|QHAdMo40&Sb#MVvQ=L`w-hbYlh}DZFU_ zi4_-Q?n9@GhDV9FcGOIgRZCYs1CNAK;+Q~K6_XI8MO-W-Au)ljQ2Ojl{zp@OWVP4z zkptbI?YDR9`=?~WVz~Y3fjne27S<|OBYUICT~YKn%v+VYOl1mgQ3jHY>|cUS2tLLBcdmW-1N2_WeU&W#=S@4nCSa8|pGb zc=}+LF*9rMkc+OAgvwOz3x{ifeGXj-AzhLI#8HyOGl85#TU1cIILS4WHVTsA?A*Cv zEy+y2-bu=&D8RZ-YI59SJTg}JwTPRFx54&P1jfN#QybljBt(l)l-LzQIC8yXx7E%5 z#EY9GYt4AD`45LrJ-U*~4Qgf~d^yNNWy! zZ3k$%-7?z-=_Gr-`p0l)8pjXUIJ4D$W4q3~wCQkCtNJgU1*2B4*WU-t5n5u_+z5ld zbOdr7vBM;e^56|lIijd!+M$W*H6rKe-QL+9|CQstC|OZ4+yhtHhjt0iG%OTY9U$gl z^VMT$4fE`+EOW#R%nS?pIO7jbJCp@@mK4XzJ>*CbyMn-`H`(O#c?JNTOH`0q}ENCa-54`#i7;s@6um!ij-jAQ!D_BX>(7s^J#K^F?_ ztuSk4<9Fb9?|G=?+Qo(?u}`m1VYc}*T7@etOz)?^hMcg$$by8c9888R#?~x9_sT72 zeaA~nh_l-6AElUB$~AIZ;TZMmNyCfzb$VMD&uz&&JZa@qRCFmcF-x~_E4gi80Em{D z@@byXG?JTt+9;FoY{9%yTESzC`akDl08oHDFg+^myHVhV!(v%+M zWeQ{uJfbXBEzlC@b;Kter>`RW{M^b0Qy$Ki;q}N;+CyX-!Q&^2 zGeR-6eo3so#6_2dYHcyEcI0q!)${VMbfprOPR__!OY~TC@+IbBMBK@o!>9 zK92_a4lUa;8MC)w@{@(d`t zeQO#}6dHW0;O8A&gr0y(0yApb03VYuYB>ig4!FNNKtsOi&Nc!Zs_i7zw!lm_mA@TK z9Y{TI%9`7D?E*~*(Vt1Q94V<)_$A;tt=bnzK_q0KX3~+23lcIZvDFs4X?AcRMfPGX)8-byeJsJ;8R*?eCd1V`q)kN$d+F!`8?&58t7k1HQK9AP+QwVTK5c`PPK?w8_wAQ75cpF{WR}b>(^d(CzOL z?5+V4qWW~{?X*KrNkYXn8o8Y|L)B(s8J1shq7ZFa>k+@$M7lL~(wM&fJ{5O(OlIZ{ z{{4D!ZnRV|e7X_EL1ZBtVSNWPf4YfSIrygNCI%3;c}z6Btwo!+B{z@w!jVS1mC(_g z+l$VLNt!b}QJsz`r8woGX!XZ$url>W04W?5^*NVJ$)bqx7gzN;wArKh&AW&q*xx{V z8)Z8EBQ~Z|^+JWJjSQ!2yU3BM2+J4Lf|Ua&`_m3X?X#2+AdHGFiVXCcRk<8zlRhLTanOpCW3;V;GB-E&7>`k%dArw6@_^8?m0 zV?c&@ATYv~e{BV2&0DHsRVvJOu?{~K^EkSSP#*zLcuj5iM5l?z)5RD#h-?J&?Yq^- zL$lx?M=@*;Br7VvWt!Q;pX`Rev}~9IQAIy*lWN>|I6BE%e?=Bzo17D_-=Q1@w^gkr zw>KLNCq}vToeOLtU3L>av`+b4z$A8kx)i#1ett%eHQW0>^GC{IOxpT7yo}5MKAdG z*wrCct20t;>GZ9CRuwQmdNoZ-Ov z8$8w72XQuh66HIx;2smoB->9&l##A2ptctnsgnEJ6(dqaOcj3c#L&vmwr zG#<7^j&)xV$_sE~mVc`HQ&1hbl<^uJvucPMB9~hnrV-_d*}MKcD=kzhR?pAS7!`?f zM^EAiIo7L)V8T`T5k8V|p2Sp@Q0_Deff^rmBawbk%*&VJ0^L?gH3{pabT1?^3M~xV zh`0~t2b7>30K#OTpIm)!h~{at4$lD^h3NZM+7@y_)jKDRR(*hcfrjk92H!6U+9P~7 zRD~?3VdJqq2`J?ct-|c$`9MNZqcJT=EkV%$5@h)AU0v6W>E~IT2H)x9_e2p^wC9u& z_J@Ez>NeEac|%1yqWfMjRER=R*!BpZjSl+Yzzk}i3q1e~UvL=NVnG&ba;182f>uP3 z<>oeU?+U!B@cOrBQVAPde?TGmd8ER{64zp z*WNNZZvi?FwRhWAFq8h0IIH;|sSR6~f+CN;zD6cB5kJgJw+(hQ0z`+q*s+ zeW0pb)An(~&mEO7$4=Ms=#4bheRdZa>3m-$FcVhuIx3NlH^^L<>*rwqFOX~JAEyuQ zWFUR1&5HP{p5K00lOes#?xwK!ni$yptaaNO7(QTjgqMVIoi z_(ymM?->9GdlSGkXncl=4-!h=!pi_ELQ>2&9~NIy?Rn**EcMX4-pA~*!XRM(P!g=4dGqff0DJ#FiJ`~=lQp>r*&EJN~o|mV&}{qg>>5kg`9)%lQ5DbDa<(3c z65rLF-%8`PRQ6jBE^IV}Ixyap)ji;eFK^;Vt&z$F*JdXk3eQn7AH1iHw#!Wgl>^e0 zsr+O-rTVdg^RXJBsk}5P)4Ybdwu2vJ+pJ+-R|iB+Aqr+ z9LE68V0Y!80ZL*{J}5@_eW&%w36>b3{{tn;m9^dfvH!jEvdeTkm*!Rufif(z5eaT! z0zf_7A4FtB4*Y6A*98I71n@~3#`kbd17vNFswkGY`g)O^2f6@ZIC7jG3kNc~Orh4Y z5+{?9(-YnpSR>wKiiK^f+D8w-)LDJt0i!#(jQ>7K=%133h!u zOhi~*o-#}RNePL7BN#5764@?Pmy2<;sl0Ig_uf;q3pZ6+Hc2%@L?4zy;=mXRe$k~f zHPQvw^vcA<4x7as(Nd12kN<9k8G=ZvaJ1q?7Dqhw62Lz%1(Vl+U`x%F~5^^3am zUxgNAJkj$Y@a^iQqFgC?xoFacuhCNIbNTv1u1-GJx})=suD{{0Lp9=XLGAr0)p#8a z`+p%!8Do}FBZxsVeM zg1(pCM$U53toqU~iMqAv-A?0a*z^Owa3%p_e7HTT#6XFk*v#5f!HL9f0!h^zug9?0 zp?>W_;oNdbtEp+QsI2rf>0%xw1DuHPer=)0$FxKDtd!;(iaXa&lsmd1TdzgwD2a`O z_`mU?Sr6qRuS9Zn`z>eY$O=cpVEHNqdr**F(1S6Dqq2PHQhmS>08#LN%4N}0GMn5!A`n+^lote05 z<-QzL1p>+E$k8@^f7OCB?!plaCuZqoakT6C3NuP0H;nwLD(p2;NJ(&??}-rvpUZ)n z1pidt%4sAV6N6I|ve8f`oAIPf6h##hnVR~#6*nf6@xZ2X}-=hnP#R-%vX6SX8H+R=hg4IOZpV(7`N<=`JWX2-pOtV93 zw3=h33THJ?;wG=}Vdgxz4ut?A&k&Zw)FNwQBkX799Qt&16P_sJz$SVs ztUXXqXct(TbIVP7P_4n3NZ-YIL~TLck?t8Of1^u%UJ?gmX8*p&k}%kF964Gq&h)?B zaQ0Om?0dW!d9sx3Hi~Yl7K{4iKgmA^haw1@*yZ(wPAHI-3fU7Ui zjumd%6&mVBUU;p!qvSypq;4bHs?lT%G23*yi7qhA@!r&+YkR-b8-j z0PGx?ErladV$Jg=h)1~Pa7G&K{xHh6a`bUAO(#feV#5zxK@a2qf#Ca@#w7s2Ir(|w7U#LA3}E^k4*nBEP)T6;2# z5xpo5yvLm`&_ zNVpg+V)cG1#$*hT5R5aUJ^$eGocNIvQ34#o0zKy$00p05oI-mR!#njIVDXSsLZ!C? zPU^jMM9*0E9ym52jR7tI^@1!#FQJPAlu!Mmk8h<8r?qRs&@_U*Po1S!TE#4Cv_-;4R664n{yO3&IS+ zeI?=kTeyF})2!*NLpIv`o>2vrveG7@nButdtQnz_ePg?sX^hz`(?ueRM3X1)AQ7Vk z7lq2P#gAqjaD}Qkce!e`u*xI;iKHDwf=m?X==QaD$vc$f>JcuQq7FooprNBtHe1r= za^xT*jX82|+n)|AV=Zox$*0G>R>wqGd)qW{y1zcvl;f$*kiD)S&mn8?aSjW57)jJE zY9P4^%(0044iPpRLPD#hpj}+vR%Yuq4S+8_!N9|5nuj5_x+$&gVib;};i$QL*twhZ zRNs@(@Gw>~AaocXIBV!)ZPY?$B_zFIoCkT5)N{B9UhlPa&Fqv_%Atu&gAA9O>q zGbQ3XR4ebI#A2(v>*vhKRfc3b+{R3qIfHKp0PyF7<3y>`ciN8s&{U&`sr`|d{jdh4 zB8@RaxZ!M+kU-}jzxnDsx!nTYw_Xo6?L!Qs&mhe7O_-<~bKk()zYxh_RIdP@IgdVlq(ekhoeqf_)mHoWZVzN~#emM( zC;2%3m2KpGe?v6_J-=MFU$j$n-cqJxE8%4J6BiC(qo+Fw%lCYK2F@$)AA0NHS>_MDb|Qa<+Uy2#k#UmP!k5j7i|dOF}Q^ zh2p~os~=zYeu|YxXhEK{?yxn9TmyJc`eDT^0;nhV;x!uXIOFotkC`vELfqMjw&7&8 z1AJSU_vm?1^PUhi{wbeR(neZafLaid^ec+aLN1@XQA`BFWZv{_z)5yECI|m}=r~Pt zL*j6vMll=klGmA77KV>5{skt$G0-=7u`7vd3Gsz1Md# zoYSuTfb^x?{Y3ReEX$+{8!LBdA$=sq7;`C*-r+NTYnzIjskq#z-}U^De*F@=v=Ir3 z76l!vc$hYYvvI$M=PJ6LvC)7SErmt49BFpd?Bae45jas;F{>iTuZj~m6z_;PfM7bG z!>W!ctA`SWJE`$YKvjThEB16=qmn(MV91+V49k=UA@DZj+YuVqlY8?IsfPtmN6{LQ zyE7&{f4j@GQp*G_2YH7Ro9=vv?(!o}J3(1Chz6AJ4yM<(@8a~mEYDH;pq5yb(eO7x zvXO>cL^D%Ni2zzgx9s0i6X^)kEID%6s=!HP>{Xb)9UMO=;_*G*m8}C=w;EHz0=c7r zejrK?f`Pf8Jr$Qih`O2bS9z2AglWHQYcjhK4#zI5UewBn?_LVW!Lc-DA*s^%cx4_5v}# zWNB^kW{g2U1Bpr)PkiL}+>~))h^!Oe?&?{YAe5bN2IAKG{jHf`v{wO;AZ9_#tMa$2 z{S2?>$MABVKas#wr|5M^aY(WbLo}U4zcg(^;1)KH5T;t^*Sqv)U|foT;YfyZe-=EG zj{EIabEPT9SdCBg?w86FPl9Prt3#ABBbK&D7bf_%NHVIYF%0CNUsDM)88~+c!3LzM zCRp@Y*69lW{7TJ*wgx-|+X^`quv_~Xf7~HGgL)_jog?P2MwJgs=@Ci^WwGCWXxh!S z9`i!1ipbteKwyCGmjg!&ibkyO5w{|O36`8<1jEcsFzweKpIn5N9`-*o@w)B>$B(pS z?`SHXyO7t5bz8b6=0N9fS(ZgXG_+V@AV4^Ed z&sKldp>7k#rHDe8Q^ed^?zL5m$8l#a;f(k%{zjKEuvOECCrAU*>Y3pqQx<1&P-6` zv=keca`Dk5icn-r)XkMc>TJ6VJj}O0-=p~@onZv0Iv(&WU+EyNo#sr>IZwTpc0>_n z6^rsP#lJO(LR}|%&u4nQ<9r0|M+0YyTzTQ7g^pg2B;zXQO z-jouJrSI|^QM!Co-m)vC7VSb)PA2^#eUHc!Q+NB!xY?|J50hdR{}$lyOc>G$)=^k6K89}B z85k_jtrt9}=<<@&@ZQhm{prs3Muu!OJvYhzqV%t^CbagtI(AWFXOKG9I?;pYOk^kh z7HY_BMv_y7$gfRi&qQxLx<=0iL{FW~4OUc@R(^S(N&z2%C>E_e1=aJY3NaW)Alz?4 z@-1on(@)6E02wp|@K+or4taQ3FNuFqBw1V$$AsyI>R;Q|*zA$m3xQ>yru;7-%P}fjY&orC&{H;(5KHeN-_6?xCBDH23)I>kc-6;KVi|m; zEMTl&zKD=Y#ZC7LH{P*!YZ7srwPv~Uc^rMW{{9*j(KqxM7UmnN1#T@S%v^wtKOBqY z`_{xXpo|c#RlvqL8F}+fl#(#PXqcDx!zFqC=deRmy$O0Qidx!(CxNP8DYoW3K&1p+sxBDriUs;Ku5VvJ@lb0j)F8hmt*rm9npduC`$rQ|63p!wP zPnxqYo&g0%_Ic6wk$`B^0~o{%(PbB`%)--#MBAl&j}6by>GFO3>+r|-##v}dmYm=C zMPj$`iar##7Pl*LgujL6`?FuNIQonI!{awwStVH*>FKAD$EYE&S%Au^!9OgLk(pcF z0|;f)sRpUpVrUy(T>M@qN(wjKZjiGYEtZGfx8VfiQfCoy!0M6vx59~oB(2UYUslpHf*EvJ4LeJcW>MZo^gh-Taigo& z5uwY+g$DhmpJ&pqN~#F9)K{=YSPu$3t?v(TAyLJQ8VNlHaQufwRD~LDlJ)jsMzp2F zAHj*C(`nq5vtD@hzK0udjT(RZIb72_GPJoW_%Yt&?vMI;Y}sQ%=5brtUjKd<-ds{l zwl;i|IWX{dQgr)m=VQtBz~0GQw+iYZiSER(yTk1`k^YB>+m!XL`Q-z|;FX|R32ixg zeX7N;hk8oEnY*7rE)Gw{ET?`Rz}q1)w>&k)00TWaFfCLj(nZ1nrSYNbk0X8|&pd2# z4W=IY=WUjF-tQg1yMEtVNIWDlO}Ia5OmK`Z74$tSiEkjIG!nV6uLCYq~MKb1FW% zSTKF2{Scb->nQbcr_QH9grMuPt zz|OCcf-q${qy=Sw&RH}s5ipsQ&c~aytWQ3u{^u^k=bvTWpgQ<-FS?a8SnIOFz9>dm483(al&h+ghrVAlAdEWr$^?!d zhn}(Cq_5|jpy;m%na>%r+Rl1`B1Us$qYzDPk;=8xc}#g<(sgd=`51s7BK|-}CLdIA zh<0c3%#PV;A85Ov;C6n(OL@vdO~<(Wr=jhh-(!70jhOh9no<=#i{WzzYiDkH>I2OMT{JM*HI`wsYAeQ` z+h@zqnQ(Rkyy_ob9RG=9NMY*s?FCw43+IV~Sy#!+R+^fw+CL8qLCgN4tFN}dM?j>a zY}a~nOviL589K(UlRN8F`WcC^i>l4_%Lgr}5Vsg7@q85y^=g9JC&Fw^6`9B~4$};{ zN&WZ3gwv+}Dye`7QbrRN(TGs1(qZr>RsI^U*=zrfr)5#t(yohG$keTm!jR!M$QxBf z;Q6y$oe9=xJS;68>T3NN?SsLuub<(eqc}z%;GI#{a(Wrw##+35C=ozKb}4hbyQuQ@ zxl~=h&cpUzFMZ=)#l@q&346v`?Ykc*)WGc3tQZ}1Fb~O{m-@QJ_lijbWNnmX87OoU z(E95^aE2g=enb2|lv~xj%Osi35U6gJl&edEgoRo)nmyR!19=izmxpUSlNZ)QwrHn!8s;Z61QJr# zbv-APivI3%_!$Z+{@>dr(hYM|H6poN*@PDfgpId5C}ALQIc+1#-?u$T;2<-nv}t7Trgd0x5rFnR`%@c^vs$3PeXtF7=<ZtdSi6;qF;_ zwe;F&!Se`STr-=V*ho$|%A7+Kvj_i>3o@7aJ=hsrQ;uWBWnV#NjM+p6wNgvt2u)^c zqgy-eH|l73LVCyVM@#z-$)hTw_r*!(ZVc@-O$0i((&KeP{X?2&cvKSc7c7GtEY1<9ZM`UPv3?7NUu#h}lOsxn+&FR2kD^bVABW zLvT*Rm#WBe+xP9_f_p_`2orT`y>F^_CI2*L!Nz{mV!?Llyu7s|+2U{If8jnHN$gfP zA`<9J@~qzyt7B; zx8%1x`<1JTpK}GDY&@~Mrx?@K~Cvun$jljY#f?O*rnKA}_ejMhp^OnNk zgekHZ4MD+(dCT>Oy@sefd+NJgPXl;d5%l<_ zX^UM*f@TVnpWO8!>K*HKM&QzvYlu5`MpQ$KSNv!C80iN7=qG1ea1{)!qit7wcf|0K{*rT@o$&{lS9U zQG4#5Rb2V!4}lJZ$@Dsyy%>+(Ko8vr2%IkBTHQf^`7z-+khTcK3jIL?kBseD0hmfZ zWg-1h$o!m65~!qDl2{NJ^355jB!+_Vmk3R*NU07&d@tp#cUPULUR6TT5iMT5>-?p( zjfXy)9&8PYlw?9Uv6!-3%DX7me2ip7&(vgLdG%G!an`P5 z2Z8dU7>>TGKOP5eRn&npHjuWqA05#XKW+A}6bUExrKU|r0b|8jwi>(ZEo6#wB~^-f zI0ZU^knE6T-pgEqJLl)XIl}el;e6Mb?h}e7mQTVHVV$I|oa#@Kjc4U>dFC4^IA$*Z z$a-HG@GT3E{*vh*ugcNywOakfy&uqn!ryHT?Zy}wHUDv7NbB2%t@z^XA2|QF)^P0t z_DB8pI;y`4yIC+Jj{fMLVO$!kVf-z($6;{My*J zo8%UH-w5p?$|7B=aBO5MjYB&hZz@3_96jTZ(O*-+LzmfWM>ySZGxD?QZOEFW%OjxP zxH|XpXq70MH2hSM@`&4MJ`q|}XMhAM5ldgJJR*y3R2S9&0Dw79ine@Ti@n;zX#4Vc z*TeX+^lOryOzj$F6leWS7ELYH4NUoLU&XzK8qL_Jdc5N>%40Bcz}G}C%kTQpn{Sog z?qbumn(1~o5~wG&H*YY23xGov%we{7S2^7#2Qf9Cl5Csi(UoLwSmjMb3$q5Xli7r) zC}kuI?Tr|^4$Exe3(gv_i=~}FfZxB7RTNV!3e_)cmO+iImWh@U8vI0I0YtmK@iX?M zE`rj+`_oDq+?*Uc8K(!uc!W?jVuSoU>HYIIQ+lIuFd*_ zB+@SO-B>c4TO9a(4N;b&s5xrU<`$2R+NpXl4$l@@x!o8W0J+M zQrUDN1)i1JS9N*Ie{Vc@b@b zw7=-&QG(W^Rhb9B>(weu4ZWU4?wD>dzQQgq%OZa@7-mYILBJXR-BItcaW>b_&P*H$ zBp~fPBh2LQ4Gz6V?31hePK}1*Y{hD5FM*Sw(`-U64E~6EY_Bc;;dLVuJxvNMq;RV5 zLtOANF@Qi^2LJhbeSm518FhJPZhd!s$7Cd<(lY>AMglw4<5)h9<7F0OlzbFSEu8&lm~T65Oe(BC8`v)|@nWC>WaQSd!!=4S3cUm{B+7Y*h#cMvIak%>yu z#mro6DMw`@=>TNw1K*3)wpj;5YGOM?RLv26VIj9v4;s0;lSpwSEt}RSR!zdcWxfe9 zeTxku9tu67QsI_2mp?=(qg7NEh&H1m^M9EX+l#7#xKU;qv0v#rJ};U?4n1>tDIBeQ zZVDE|HX2Y6@lh~biu1u+yeeVlM(j=P*)mu75yI=gEMsM5@;GY{S4d}T4{>s_v*Q4NEit!)cpOLpzw^(emo&V8y3co7DmhuC}y>v0n4ti{5x zxrSv3ACZ;bztk?Xu`pz$saWh`>l}yr3)YvDA#sTd3RV+*8@+Z4oTK^cxR~M+t2EI} z5PX(&L`YWO*FHo!LC7}3GH_{WeOlMvA=98c9oz1X(ne>-}0K(Pn75cO8hiH2ideVK!qw&5EftM zo#QF^AHJB{KGUa^xe@5%Tsa_;i>w4}TTIWHeV<5}Oou3mHt0e_HEeji-1-QzIfU{Q zOQ|xIMx%^#;r6H=w|n~D4*`XI2TLPDgj$UxyamicgSIE!1&q$51_>4~LdY}?5pPQj z#LM#D9y2<>AGGH)wj(8LTeZJrdnbs+buF2>fT&}!TbuS08z~{PAp}{&dMUNTae84b znlXT|<{$lYUyM?L$4WM1mgoFSWC&Sd69Fv`9G{&e+yeP_|8KMvCIK^N>!;~cg3}FR z&A=Zc%xtX8f&+T9|BVnL0TIIDb(NtCuf(0$P4iT8CCoCEcxy$(%w28?uGw9Y9`0Uv zvq&0gJQO0;m?`#E4cm?KQEJj2ln5k;*Pg zOi9XC)=ox^Y_F&T9A%ZYmp$2|JGCl(JXgtMxWh^K1F%Ej5^fRA}iu%dimRc#qs8zRN<62P_wHE>+&Xu z-ghn|7@(65KmB+7~<$QiIqfXyJl^06vl`wB&2T zgSvwFjihHiJzcfAdYSp3k}z0Mp&#_4XMrj*Or9c97x^cc_oJp+i6m1D8i}lNYE~0wMY~+fe#+C2mx~y%f+U4bTaD z>>b#e*RbcF&H0RvE$J|5$OZgWVWUq@!{dw(D97~}fFu+}&A1$w;F!UqZdmbl*>?pY zmxAI{zGWy@bnW@{^Vsvel{3oGGMA<&P>aMk(SN7%qOmuE$jNB>%Z?pO zCU^kQLjdw7{i@ELS9&;9?XGXrWsa@AX(OD5DU0d^HcQk&U?L>9Xfglj9o*7hx-#-; z)585Jsl=)W+Rcc>q!!BTeCTL)h-*u~Py|>#pA{~Kq4cNiIf9x4^#_RbK@%cHk_UW6x0cGUwyMB#$xW9%V5thU z^4UT$*|7c!on!$%xG3U@sYk|xAN^U}vFym+u8s8iUmh5JpXBW%ZGK4@3<2BuN8*e+CpLN!Dbr9w`YCZ?k80=;VRDnZ`f!|Z zpl0`FXrX<4PNyC#-@{@Bb{2%^j~as7u>LNlq*Gj#f+(37(6%~16wGR3{QY;k;S61a zRUj}Lx2z|=w)pt3A~6S%3!k1Q9+6n$D~U~uQ0R3jQb-$32-N1_z>`TaEl8@TC8eHY?pEB~6-pX2mDW^9jY5(nRwSFWp*3Hi*E;*lY% z&1V;1GKOp;R-2{dbr?Jy+7u_|p@hLNoKTv0r6ub0ZVyVWo0XTEEPS}UBHG?P$D@h2 zrw8i|lR&PsTPM?-54`8FJ-%BU;wVEBdAZZ;OJ`iQr_~k5qQjA63=6l=vm8mkOziLG zamJz4pi0hfbma`Y-d7&!4%0A%(*!ymHjF>`bZh3|bxz~^34Ljoz%VG4zr;EP|MC5W zRSIJFKlSH_wL0a3A*~Hyu|Oi}pPv=Y4>5svD60y^f_IaLMHN_RrSvmP)1Z@AP(NYN z#G2(2q!g)o`rLV6v~KQ(43);=l{mBpf4sSKx2eLlhdO6k_Ty9dv4w2^?PoCngHgtj zxve~^yuOW7^gu8~$~VMv-MICu)Gz!w|3zPyPHgi?Mh#g+2NDexk;X|(=7V5;i!4Lb zg0H2|_J|Q}z4C344`2V-|2-w(e6mX1no#5Tc09+wm5yC2unjN z^&-5ZI`c{@0lb*#KsWBllq&PA-T}gJw zF;VO}-&1k2E*=$VJYYvd6+FxEx9*~{D?+H!?t1xn1zn#O-1-!SO8jE7nvw`bk_JU3 zaLR_0AM*e{x&wNS682y<&L57VM{$xrqW^<2O`P%-mn-C>P)|Zqcy%=yD@eC}XD_#QA4?wagXPESCfBZ9G9}G&YvPzXe z8S!^V$2HD>eMtf)yE$Hx)PZIcDATCJ8Q*}^!pIfbs#9*RaNcj5-Ar*we#453PV zjNNat(8>a6x+jcMcDfNpZ>EZkB$sHHU6^PJf)jdK-ig9@^du^%tDjJ9Gc4{8R7%>A zBJ9rNr2#bIWXC`u1kx{di24`~gnf0UP{CoSToV6;*;0^ssp@NC9Q(2~%B9~&7cRz= z9+Ds5fm2V*Tjq3saWp?s?3t*QhA>dU$%TTpxl%GjRG&IDMVo{`Pqt|0hYFIks#pHV zN~evUz!~!;4ASw+kP;QaNw|qp49xtA0SHFs*<2=A$w0e>xt)T#1!<3>bn!r1G}5PX z^Pp~65|SBHy7db^awd$0{y*ektw&AH0wlGoRi7U%@(aFbTujOq<)}~N&L6W*I~cJY z_(t@fgWm7)!@b|2DV)t8-C@BdR!;^%S0jybGyPmhv0tR;rx7MVqS2%N{`jz2MBV9- zK6%X%p999UCz7yC%`<)?*j$hZAMcr4YLC*^0DQvz?-$q`F>e%i*v$EtJBduet4KEuExPAcI%(1@T3 zUcWZ;djFgpeHn@oSKiU*Zg!c>hZJddYnoc|OG>Jiv)Tj!-VsWSrqE&Lj>b&;yXJV~ z`+@Acac6{LvOA6QfghW{7&m7;sz8GK;iZ|M&Y>Az-mBpc8?f;>oMAnP<_?{Va8OvTWMO1f z1GG*r@t!2xJMM8p7N3Rv@X@Ep(|5S-{%PASHXEkYYKWA6`ObIqa8 zAixVsHc5nJkg(xp)_B`F-5GU%XDd?m*L}vo@Wt_3^>=ZEJJeDre_B{XK*z?v1w`QD z-%55vg9PLZ0Ak+zu^LOBoqA^X@lr<=0nB1B9yVPx)MHk5?h8GP2O5LpeiRG7yK6>o z8oR4x&rgZ)MFv#06zp)3=;PZiJ-QZ(illZ2T+nWMiwt!{vbTW8_qiusbydc_uCiyBo;r|t<~a~?E2B6ONr zb9;mzj4nP-^Va>*BCNFRL8M9`K2j#<{vg$2u>dk1EH2y^L@iC5$r%%ac&hDFwkH^^ z^L>@qpeXb=?@2;eSgzC@R35$_)b5uPiU2Yv3EYb-Cb_c0$Boo%(wpe^nrmb{;2=Z* zs7%zNgQujQgLG;pJ&zeX!m73Q`S(WqcS||5L11&79DFysl`kRA!dlj1Y>7?$&lqM@ zqdcR?RLp?S%Sz%GZdnRZL3!R?V(n+wUcDN4#FVoRT*6f!YT7~Ji2Y)G@NbwMb=Npr z#qmyGha@Gwi}`#-9BL=$V7pV_pN`Ghbo zRC2h~Az*LRDAp7V5$pECA%LEomH$nmh2@6Z5$gIMFi}5@fh=qMKf5VJF9aQtca$pL zs=vS~Xf?etG%&}NU!S;+?Smq91~2g#3poK-2C)5s#8;hG=G9efqKPeTabdFT2a67v z(1I-tcNE`@QnnqEzlT8XV?@SZ{7@9F3CQ0Ow-wpOtjEA`>YWLJ5BMbDiRIKCtIFLRq1=S=(+Iom~8aYMRKS1DZ@DFabu{*7n|1 z)`k5@G~z^=R``BlP%ZFbs)rH$w9qVV@g#jRErNy=dwresflF?2VluusU>DLA1lvh^~v6&y(e zMt}>3f39A+j{cYmoqN^Uw}`bh%h&V8E-J?$e^pWEFLMXp?@hAl@WE{wVT+d^jmc|` zlPUYi zTz<(v2CBL<%;Do4*`+L?qSS=i!)k!LaJdi+H7h1j!%s-N-T7!ttnPI`>?rtCZ?DJE z(H_*Uu=CyEaxhF2PV&bxQDQiE)A4b`F1ZVwSi5IE1X5AH4IXWa%4{X$odhl|*tS3z zM`zhT+^KT6cE<brTcu6#37zO146n_9a`Zd8t40=&QTgf6(^1>&Sq{vpP!31)l z5~F3lCI19qVzWpzUHF93JM)PE13Gu=C6hEg7AtJ;ut1utqzP%wZDobuxs?I6d9xg+ z4ID2#ZM`yMTOdIghOQ&&ccl>4;~Wt~xF8=~NcA;_YQ42Y^o9Ag9KF5V<3^Y!0b

    {% endfor %}
    rwtCsjapZm7!^y($ry&fC*F&2Wz zsDh*vO@T=|u%P!~c3Oal2O$1Lg?F8;1Xc$}KcD1`i!Vs1f$^o6x_yw5WzUZ^yt3DB zkOSbSh0MR(3K8UJxp6f?-tt2fUg3P*dhv^Mv8we?_wyLVNn~$_L1X4wBO%9vh2^j%o!W+vEFq*U4*m#Rv^vj^e0EH zVlomYP``m5rKNmCZJgheqZGl#Eom$uYnsZ5k;H;8=^+XKRKqj3So5(cMSBsZxPL#P%D_|FuuiR zo8&dwFMRn$OJ;HKp#J*ELk8flDU8}`&`G5xx0%y@O=s&(!+4`r{;3q|D@uTAFhG*c z!rfKl`QF*uS1A%qo&vp*5Xl8vGO&^ImkWzE{BSkwC-9qhIUny-_0qq>Y6F3HbUW<* zQmKhQhygZc0eZ&DuvFsy-61t5QmV)x30yZw)A}!zKS-)~$7N+a+c#GnGF+m2B?idAa+P zK_+L!q4j7bpxex$jw3SG$asqXm(qBSqgCU;1p5`yoD?8(A)UZt?|3&=-6 zlNU;bXNTB$tw=SEv|B`{tc8dB&l0>oks7tU_8ZV(V9N=Dzq}-{DF9FwE2W+6|bM6;311(Cu{-L)a={!1GoNgD?R$qcDuZY5nhCq^Y zNFx2%lNQy;E><$Fy9WMn&a1VO=lNOWeSxW?@DLpsj8`nKXM2adqZt!r#MrMp*3BgR zv+2=R1${ovQ|pYp(kN zX06f!sD4fhp!a2!cX%~qnARuB*a^G6tUr}7KeCaac|@Q^CWa@yY7{P{jpBPrlRu8q z8FSPH;J02R9Bq(_zb6dA)p|6RFrBnV3ht3AUQjorkYK0n?jRiVFLvqwJ-mPScn}64 z*FJq%ku>69#WNS_ArgPnQl7niFCJefNTY~kSo?_wFcs~N$yRAF=?Oh=gXrve1rf{O z*cU;oyU2@QB?)BEVD*dmX}lk#a13pHPs=z_)P*DMnqm%EX^Q1a^Je#Uq4l$;2QN`- z5FZ!}?%_jRVsKF5umkk!$rQC;z?Zfj8+fs<(p&0|sNaISOe-(;H{j8Kln6U|YK0E3 z_hG66&yqf_+dg+5kvC3kTv}eAwP|wTM?RSG!Ff@CwH{A=WQz3NwOC3ZL74|5FMpPh z$f^HozO<$)jEV|Kd@mfe^0+b!A)}=G3ts>S-j8%*`GGQFC!Z&$k|5Ma&5ajpP@Hu#6(PZ8AvR$ly4ce)0@E@Ew4BYm@hfo=IU`ED zq_ayo*I5(WKDZPwv()ov%%D{m;Y%!vl&xd_YVgUtc-%tzs~$p9^f3RjlTT>wGdd0p z1#Js%z4;DXlntwi)s1Ni9z^7hFRk9xx)=Fg%5jx2nk?!%@1vmfg6<@R7%_OC4(CeI zVqC_t29Fizxq_5Lnr|#y@BgNwe}6O~&si9s+SdY5M!a5}Kh|@- z2*{h2VwT!0Jx>$iom!KT#$a+l>uklcREV$7oYe3@dwO2+BXaRXs;je85DobtP*<$1K zjUu(E1ngmm#pY92>mBWXcHV*-zbo;>NP$;*AwxW83%bl2*ezdNW{R5mSTsLlq$i7| zu8AdZ+-ksL32IqcycpYX;=b&2Q6|Nlfe@@esKdL3%zlk+l{(CYb zLiip|Aj8`r>|KrI=eIVO7UJox3K*X{>WSpyBAYpEzhGf-Hw+QQxKANJubT{w0ISF5 zXL7bJgcA`O0MU)!QN0LsTmm%eo7k#%p+YNYEjq!1pY8^*%uO7+ZY!Z`f_b=M=p&nx zOY$^pZkXSL%A|3|=9&2;!Vy87@|gfI^1&HMI`gFes~m{F_2eWV?7VF*#?(VDkA3XN zE37IEB6z5148G!VeRT_V01RWRy|H;~t0!NFNR&SM-(@#h_cokW5gR&;axrG1=wBrb zb*ygA-J9o!otKiWo?27-)ujwd-{k3J$j_7x$hTN8PfItC9RPu{e{r**w;=)J8wp4TB02fCP z8%#=sLl!iKFH8vJDTiRWYwa`!E^fePx!rodalRxSIL7vp=Jq>z?l5UIiy$;W;?#s= zn_0JRRPclw{68-Mhie50DUiJdLctVU*}w0pU)W~=H#`t=)c8M@{y)`*#2ghjq~s%D za(#@j<|jv6&J9|O@tPE9Dky;SJ#x`E?|KyUxY?TZbW)A58-mCkShbpnBt8sHk|=P4 zBr*@bBkEH-!=?XEgZ*93Ry~WN7U1iPOK-j8zIhn6KuOD|^)!A!^0{acI?+r=m8`++ z#m%OCc1H!`ZvKt0-%m3Puk?P){ud+!Kyw4Pi~zwE(g2cP&!G6u2phXAzsI5KW3KF8 zqjJflFQq?x=BAu-v(9tk;&^3g#32kX`g1l8RtS{wy7QRI*zK)b@H*=oSLA6MCW{$bPj|kZKV3hPnO* z879m}d!#qAE5NykU-0Q%9wDywO(&s=SNLsF!R{}+u$qg2Sf8JFa@c}`d z*8f)f7-h&Y!m8C)GAVzw3()z)&^5cM>l#NYLMm(x>hQg+q`wnQyz=O=|85Z1aSwUu zmDE`^NpDHV7xKjf;|u!jv#ida*jqM%%mNSo1u~1Lv@Xaz+DyfT0))M1h>6x<4SIZ+ zv6UQ@EglRLqB~7|C4S({D6kcg2Nw=EwOjJUB0fXp3&$@%JwrL#!aVsYs-7227t0KI zHxj^Q_Q(s=j3@BGMh9fSv#TQ(?S!J12UB~3N8^Ngl~ho6xKhuk3+f+sz5nC&}vnzEo2*Akj-`(ov#-q zdU3SpU;uR({vSUYD>)P??*C6i;PSBD3=PG{a6fEpP?G0-?PO%Oq2=$@NQ1hsP`~@) z+jB!OzUMa8(e_3-d_bk0Rslna5rnC`-9dsVmSKZY#FhvmjOabwq-?Lc#3z zJClz^p08-m04G-)>21tmdOU$h_kT2+mZFgXu^acxF&zD@w^wV=t~S#4E{~x)*#xPw z&`m;~^uMrIRxF+$^DTwiqkG_WH9*KXVV|~ZX_^sCD9{YZtV~CZi-G;|CVP3|Y8bzW zSF<5rv8+<5d0+7g8A%0ACy=#W%9CNC^>UpHYdESJp6%~6I@IX3@rOb3Mt&F+_8~lg zD%@r5gWWp?9l}6^|dI|fOQOyE1C0#5fB&xN= zO*tM&?<}-xUBaW#Vgi0+ip=OM2nc7KxxasIW}FNeUSB#&Ge&_Ac=~#80GyOGcAokY8XNofhyDG45CnNvvIWgW72rbk-3dh9WLL z#*$j`|D7}_00`jK+Ar<%xj5ECz!exwN6iJ1)-_374soDvC@ZPY_xq40+lN_*0?)2R zXS<<`r#0N4!at$F2fw88!ivg%knk<+*d7dHQi^eQ zzU#=Ibpge?O*am^p5Go1mo+vr@MG?aUJe$)KPkK0{95(?#ya+6(J$jfZx8=s@7rV z#Y%shE4_KPH?B~(vH~%YEb}AvmuK&uNqL1R$J2-`u?m1=17-wkxHUSXqsQN9xt$)? z#1imNkMFZfoF_}>yDdT*Lj$xJvU8`O{d$mPOy<41F01>vf0P)R<1;8@&>+=}H@$LC zHj`)I7?(J5>?O+$&dt>Po$2t8bePN{aOB$aq zif7hwL}=<~McJPkZ}^-gH8s8ZAz++L4PH_C)~i#v9z;W?xCw}a|3;SOV6c#+sM%;p z58Vc^!pg=!FA@=0l8+5@xQKPi+F@Y&7z|RR&)1(th2hW=<+?T4e#y^uCCQ%1%nTl;HolmY2s4HK*ElV~A!@pW zB3rB4mku6~;RPpWnk@vaiPjt0&Bdcb7WYTo;gF?gJC=9O~6s;q}wdsXI7xep)feeuhv z-`flA*^}{klnyi0^P`;7CRM%}o80}@;u!NZNw*>A*{$t!M2s?Iji&&?TCg+^zAd9z zh@)7&O&<$yKjzbf!Y~gcGL5~DYET#kW{1MIwLI?|I_r?MN?!zHs=leL*&7oDV7Ovl zK6Nr!I3fD}H__t%a?N)DIBwyIU2=SrmHN2hyA*oDA0&GdWRO;S#utpt=B~-vVUZ(t zP?rzM%HkuaZ08Z`oTOdYf1Yy@yOB$byCg=x0Kwz}=tjrCqHl-I!*=kXGJfcdkH|R; z_GLc&A^n24s?&!f>sh^Tz9n#AHP=+$OB`MCuZDy?cXSBYfrr_`hTrjH!7;2PrIYN^sYQCj}O0d@j|_54q#LdHi7^! zo6NXxt1tH_z0t$8m#9b<#7^16J*FV#gC;k(_=-hMUjK?76@&F13H)phv@SYs^b{{8 z&sZ|%YRaCkxclp{1}%wIdfy^u?+|1_I4}h(tl9qJrY&K-lt6^f3R;X*&4UziiXVoz$AeXT%iZvwMx-Eg&zx5qb&!tMeUl{v&jhmCP!vWaF8nDC{VZWpDE%0o-K%e z5Q5AF+D~wezvea!vx7^Z8ZRiZ0zgJttOQSh3%xxh6;F)%Ae~=?zvGfw$Vi7w(gTS4 zu+jUm(8{w8$ic}^-I{Ly%$6qo#pN3mGwPp({oGi5(H31}!;~3wxPOfeQ_68M=@1Mge z6$t5Hk%k$jUeURDxX)k_!+%zNK%Id4yP%`4|49%e0lSvb4yMO-!s=twBE(Y~1KHPa zwtPxh@&~EbT(z1!l`n0)KP+yZMI%!+?kKE}FCHc(0~PD8ZQE+pq+tKmHBtta^soN~ z-By2}-=nj+yDYN)xs>IkhN7#1I<1V$kLb%v3U+E5fW2#{zeF`c%^-?e8^9MAqhCDP zdLGsD=&CBK9}L?#{VTDg-BBOGv`Co|*L@5F(U0i0*XlviFA$!Br9sBUF^zNo1qW`A)k<>82al?PhdaX7J*A;+at=DUImK}tWA@}_WdIg`AJFAXsXXTEq;NKLX_6` zoYWeA{iaQQSXZ(NrAux>y(#CRptUpM0>4`#kZ_pFYM8c2Xdq^G^(La?RhoMimG;4} z_Y_{`3&>^WTvSfaQM=W3B~;aF-O1kST&m&;3+F$c7PtUVNhTj0MDzAf4+Dc` zYPV}c`!yhN)aeiThZ)r<0v~^#7TCkx_aHZ)2PM&Hf${|XyJ5o14XjNp=&e73POBP6 zyWsTn);94vEv@7HTnlpJ`}sXjR5O_qGHMM((SoO!b*^w=9LAY?p<(NsJYvEU_4@v6 zr1JxHnG4^qGSS{|z`5HSk0+!&+z<%&>rg!|GXHnRU-0Yx!}s}QPr`cZpeNQU5(~X zYS1Sh+hJ}EayPFnfdCrq>^3K(fHS5{(;5Xfk^IY>)(YS12%F$Z`x(RRyqSUnUuk^f zP@n`D!QrZBCY2`g#rJBPl^7LK|FCUiaP03&rC%s|!p=Q}_8pNcJ7QTt{Fs$c?^P>L z@9=Re6rSu{X>^&hokD1AnFx1B5U3tQ4{N{*JaFLFP*+!n`LJM5pEgMt-Cy>j!>9VM zv&Rbo(`s>S8C{Q31ZP~Az4pcT-^4{cWU=YC_`VAP9@ztCb=Lo|o`BEOqW?JdYc!V8 z$ghEgZ3+$kISlFj{_bURuB|wMX8RX=Lh{<@^>K|=cGHBZSPpWg6tY0wUYb#1FltVe zUIO~5(js&L+Kvn?l@sU{i<%x*f-iPY<*GxeFV3IuCAq$m@MW~@Z>NtS2qVH8QmGY> zz}bx0FA@)aA?lL4ZrP9#u=H_Jx(O>}!436KzRt`NTYj3N&H-`f5oOk-YSEBieSdg2$Z7lGb zN|H<&|JUm}T-(QVx8a|}`2Sjq6cX6G#h;4ODItPzBSJ}P3eNHqHMk}|7Qee zGh6B{`rd2%C0+0bXP8>s1{j5ta??)z19NtS^EZ38p`)tvZ<&Vu^^W7Fx10^V!G%l5^9WzqMJQr1{CEdjkuYb{wVQD;UR1Y?q;=-(AmLY&ne-bfTSkhEEO(9UM99o~)lJu3 z5Y?GTB85yk73=i$vE#O~uq3ccFbgQ{_|8Td3$A(G{SO309cG1Op4LayT%VN{SYKYo zLs}?NERG@;@H#F`4%;4FSzUF$+&GWEZgRUf>(C-zhXRRy{{I?EMv}uuzYne4lkj_C zQq>|c)aH1|CMtxs=;iaQ1I|NfzeRxmwbL;5j7J037t&CIz06Nzz)#X#?7Zu|LT6P2 zs^k3qSNE#4(50neO-F}g8R$%vL<`9LS>jU- z;#`A0f*#v1Z>=`%4>BIxH;rR|q<3;lMu6v8IW*KKO?CzD$8>XAj2Pfi>@5X;kr*Q# zlIzMb{!Nl3mEe1ix47l>?4J_LFZu-R?aGT|>}Cl4zBEMzCj#s^o>ALxk}XX>7R#qr z))NdJmoK89GblpvD<3*a*B^sYKGP;nAFg2_d3eoq{)0y=kAU*Nxp^9$Dcs`xd4l7L zwL3*BQvd+yEGoG_nrxWeGLtsjG(&Dq$wL?AbW!QKYOvVgQH6 zM#m-&&hx!#zu0#XJg#9+GE84(3F}oOOhFJ?E_3;nLp^gra3DpkMxTmqm7moCYJhF` zPl6^a6}xdG8N2tv-tiWzI2bw89_&|gAau^zdK=DDIfY~r?{z=M=KF&CLy&Uh(273Nb&T-j~Rt+FJC|99b`~j|e;#$R&CyQPdhR(Esy6>xR{V z6lVaSszRkOZS_rIzMB$Dl&ARE*#O8$G13~7$kKBAaE_L zo>DWQBVAz;^3H(qv8z@3kvt$*1rz*%2!h!j>h2;tqn%%-C(33-Ql8Ny@QrZW(Aiul zSoLc0JcL`$vmY6N7UCBJ$m3Y9VN_-on;fIailbTzu;Bn&~~^Ks@~ z$yEVp?YP=o(P#xjYuX?W|jE~=*^0b8>T6ko@PoA7oX9sx?U4|$HDo^V)+&^z(A_rVaO`$SjYnTuzCDtn?XZ(BaA-CpNjAcKpdqsYeRFo@q@M2Qh@rqYX>N zmvu|9is2GdLI7+=kW<|lmi@7_tNj47DSgTE^}9ki(8dl{xBFYjH%xdxT!?Vs*5J3o+~8q7 ziSV0q*{@itpFbkFVg__6s>Ov!;yxFcFh3f777S0<>ZE>BLAE&;RP3$jb+-yp$n zv__Pbo%u5{>Msl#J_W;f!OSqQAa~ld<{=pL9tj-->CiAqcM=^%qH_natL2I4B>P)C;6%7zg) z{p{Gzb$&BjA*GeeLa2yO5?0@$!44SlSW@U8{Tv$61D3Q-?pcoqkL?2Zk-YC;(Ph)q zau<8LKCUvlBj=ko%CSA^Y%a0?4tBd`IbG#6L&z(&tux3S2?0dsfG$youwn+69^b*J zpfSKX(SoHrV8-}R7qRaVid`FpS!5Gs_=UDk^4p#tj}nmrdc5lMoqCEt`AC$s|6J z2m>HAa4-PX&UZBNZ=O9pdJO{I|J0BFgr|F05X@eme3utFU(Kh|!Av97FRovl=gw>w7vj}TcdHY_XzGhVdyb+fKS zD4C${Fadv6+$7bdq1@iCqsg7iNhW*sZwM%xkZ{U_n*q`+s-1b9X=IB{%^Hp9RCfW6 zdsd!vc4UG2cU!V)O~XN{LdwnrT#Ge}Ric7X#ByAV<%FaVk)iGk2&;1bm#X{klwEdj znUf!gpl^nToFp!iLQTHyccZxLik}iRBlR8U+HkR*a( zrLA*uA+)Wct577f9@z0Z3xCTJ_6NHRYnN?({Xu3ODUqbA`e6mJiWQzjx8l)++Q~%#6Ci9yljzi#%@^nr;-~t1{K0XS;Lc$= zqW;qa{3nQlny9GPj>}<_{eBE{Y0GLWrXNX*I5uX{VW571$p(@;D!cQcMIeabT0WW{ zzvJ!#Uqb$Eb0*EH0XPhiC2U2^@zSb*RzHd=79J~Fp(h`$t}8dJnddcNx$*_QO~kU1 zrrhZ%ALRL6q(*5usia|`BuU*epOnw>?R)Mb%nU;5>49zJPcA9EPI*cSQpd*v{D={v ztfwG)5-}Z;UXISE0z$m1tfVF+bhHpHPvv zNHjUg36p(a4aHYbdf974fX7k0kIDv7fM=c%=*ZGs} z6C@mNco?GyR_Ww1&_o{n3FfDD>Q%+@#RA1I=eA-)mX&YFw-r{_OG9|PBdfK{Zmd7; zet$2MN@|a~5an&7CfpW*2UMsXzc`#ckDRd^oVr79_*`R0n@_4(;vl>%(u zS-d>i+ocaUd(+mH`epQGRF+M)Vtdg0dU#nm|H#$)RzTwm@Rcxt1*H4<25(2$A>J>` z+3L2&9_*%MTQB(Mms7(?-lg)3$NM!(27|9mRou@^JGChy#d`YScpNugnsDh={e-1V z-l|Dz%%`KX`ZWMJstTW3A%nDGV%R#MGkORsw%)C;ffz{KD4%vy&p?Vq50fgm5~@j5 zLXK?DG5U4g49a3nb8-Um=NM^`b8-ABr^=isMe#Uo!xhxV(`7-**bYK6!f<(trcqVa z5?wMBHK#;>Bx-ZxXGm@%I9xJ=7Hl#ZcRyT&PhTcP8X1`oBY2?H60{lFRY?D^4kR=Z zc<7dT)(OTOq0s;;COI4FxYVLdWqo}kPb4L)geuAX+#F5Vjc8cTtE_mM7@d=oBg|99 z!2?Y^o2SEvZsAj0bYE~fu313F)-z{L6?xOr_V&d-3}WZ@Hu9vKYTj4j=wM#33~^YW zuwS0K>|)E`j~o&;PsdnOwxn`Vc%$LxH3Htc1Hm~0kY@?+sh%Ko2XqV_FZ4+1vg9RV zP{T74_ZM&Mkv#ZKAr0U%20`@>-AC}pj~@W_$hAGDYTVb8vMP3qB|)kE&esc#b{E?{ z)Abmewsjrd0~^k?^mKu@t3JtO1~dvC)`xuJ_oE=p590rBS_HwstA@(Uw9lPb^XYt> zRW%x++b)EBwnRP9Tu!NEPw@eZ8udgvQ#x*L>zAF2t0f4{zXI}(6$`>ILe9*=+0g9) zyxVjCA6M_dUTM^9i-s$?6b_=N})UqxDN z*e;x;P#%ETPj`}c`o+nAVa&KnF^>)Ovk#t#HB7Cfl3Nns z+he9jmUc6?z1ivh`Zy7Sq3(U(zS(gaB>?w-?DGeA=sD}n7451|HSpnKh!$i$5$@>w z0etuW7xiOXfUuOg`C5M*d&NNw zU()|Ii5-$I7-9}lOl=%72gsI3q95ht6V$yfJl&bn_JE-FrLaJYp@=hieqWVEoF!D$ zA374upT=?_Ym{QvEgV&dGqMd)xHW2zfrE`vNdC-YeaHch9scPvZT21$3va4fRi=D< zyU?`8X}@Rue$hac&GjKq+k`X8*-NiN3lLP?8W{6O?eq6J*T@yG?qr=JliIdQh)qxtPL%i_Q)HJQRMn$e$M|qw-5p_ z8HK2;wR)5Hrz=02TkYPLVJ1q&EN)>hZ^2!cS%K2Uv%GBJ=vv>}iV9rAz&76h6;;VF zK&PUkm)J719s&gNs_C@I?B&T5|K3(lvn*od^Dgau-Q*W%i1_S7MekL*eY45k#=v7^ zy)Oq%WF>C$qay3cr&c2-PYRpWRtS|BFf|JKqi3euOut7C6;kB>?r!1-4jvG_H#k)# zKm>SUHsE3>)tFuWEfBB#ZDCH0r(D(RtCa&@&75(g=Xm(7kSbn&3Z32>luD8ltlQ4F z(bYHLI8DDzwIV0xA16b0hj`O8u-S^CTw?tf3u%%;IAX{g90VFouzs!7@#?nSm$Nhm zUW@j*blp8)J8ofXca~sO8FF%U8FCP@s+8l-ZKTs{^`izD_lXlCDM|%{%*FY+WJaDJ zXRIj0L*Lhwq;}a0w8I6vmGi#m*9xxvx}9~1dsyF$FcM>7wdaloVhwU87Po_;@aash zK^!p6%h!|K*wotb)?FXZT|wU$O~+zrM2tI({{=09@CF|&4G%69entrb?Bw~9%%;~o z1(V71>L`sbz`MYG_ktxO0E~R9Q$o=9dTtHSZx79E8YH9HM2-Uqaa)J>n?6Am-wRsO zq_0eb&{aW22@w30Q7PTVbBo3G}%0aUDXW`^Bg@jY_@zFRHJ=aPAuZ>?yThNBjw=e#Kp@#9Ek`PWqdC=Af-8Z zaFIArLB+Xb)EGx!7j?ol0Eu*2qq~z_7eQXdDzC{r1P^3ZOX@bgg_IEO*FI^l?{-jV z@=Chluf2DL<+k^A&?~1siT^W@3oEV@&l}r~uV-w(x#O8nIYx2ZdZRg$MaWls_ALK+ zFBx0ufVr1g@gGo6|3q?-QSJ}Ie>rpyvy!9aAisL$+%&=7y2U;8Wdd;!F06E6kd@L{ zCQ&z`K=%Rzq;Kfr#D!&f=hu848MTjkHe!pb%Us`iZ_WCPU?n_!f1;NI=NXcecc0g3 zzpp|owpR5#9(!F$qz@4VO<8qYvSlHUkK6`-YAD-?i}oEQhZPqaTeVKY*Z|aF<&w-x zKLe)8k6)|TR0I|4q(Ho=e2nL~xC7C>{hYE$)wVNpB<{3eE`MrG`hJs1nV3VdBoj#z zsrS;OrhD3ZYnE6>BLmKK8x~wMYSIo!i9=SgJ^RG1Ua1uf76K*uT-+E!0bMOE3ubJfCWsvblm}bvw_&dWF*Xt?D3o{aB%RKsS}nz>NI0RY}z3HOeL_?pf>AhY#s~c@d3TJV;C?u^{A58cBeH zp{0)9xLax^Fo2r{BpB zfXq9jF>$wKbEN2j4tfBJy(;yCkP?+|TCypyR}B<&O)6v)+LjJ2@sWG8jXzzR zlM~}O=7c*9+O^fT4o5nKxZy6H;)^8>yw9703Jk2i%&VVaPrUx>0He0^Yy7lHd-m!L zeFZ*PBYCLKXc~MgZTwQCzK~y^#L)z`o<;4d6%YJ~bMK|w9&XNL|IemyIFu$OETO?k zz*r6_D&#=M3ZLx{H#uAbX%goIG+^k_h#c&xF8xf{mJNr4BiXv1H;qo2pgz2eWNI6+ zt0g-LT&LcG?Nd!p!HC3;$ATZ;`PCX)UHy0?huyS;DWeyco7ST`i zvbniG%(o=2mDnhp%w&u0ddT4NyI3&rUBsdo*?f`)VM*8b5I)fk7zQA|ucR=ieirer zeGCHc(FEUUSQv!$sqtzjJecQ`uw5bexE=m><8l6;gKBXg^dV;8vJl3dmfG=nUgNtd zW!Y>F}sizS$*0p+%* zh~2hhF0{=dfsKK+z-nzSC~Mv>c#AQL4$DTEF>>XP`77G;B&;PY!GA)!DyqLJ`N&K* z-}3}rEpt}P`T2P+5cf$NPNp#I5u+e#&|;{IyJC?}{dVS=vqF^mDnywY*8jO591sAu z1g7}Egf2f6LrI{VGWD{zz)cGBP3W6ejU)<(u-sdygB|7QLRdZ%Gf;#5lilIxPHaie z7l{PhC}zovIVhetu~PHo#ud(OYzqk%HLddUNof@<3R7#~8`Bc9?X|P?O2`^1!pbiK zP&U5*^G!wZ0JU^h_v_IqE?PGw`vD<)c>{*MiaU% zn}DFXvacvH4l^vWakR=p0}ajyf|NwqXt>XW0nrQ#rg4d3xAn}y!VN;35d7bmoVYlLiw?JlH?B$LBZ9wbFy~

    )IR6E)85M<#9vK`A znT@QQ`D>z|O#C}E?6Tv%>#_h!S0tM|M2~UF4t*hzMQOuX%xti-LTpTtxKxhu+is|0 zjjp2Omj`G@jb)K)#U@Vl-=VxXz*7N|urJ5be%yHKbr9p!sONu&NAwb3SjhVr!v67T|Zh0k|;5TLDe4cG?A!(sMb=*E(%jKn zUPQEQQ8m3D+$LdT_ha+zkV!R*T=kVi(Z2WhJN<@y^7^)OV`mMs;hO0gY1_ZWEiS(E zUl4r0E2XEcq^tv#-~d9Bm=@wsG!yUC=N7j56tzxtdwm(m&;U9FbxPPvLBU=i(jW;4 zFHz^JLzxg}8-YK0klFW_DYa7^C{8g$x?L=V;$9rP4lB2#5cz1JTkNgwt_pe}oACrE z@-Bf)+Bet|qH46ywMtjg1TNSEXFc382F++?ulLRE2c?}M=gaG}-lll{5)|8KQ$T4L z72HqdpAO5`rOmslIJ&?L+y8@!nOo)b?8nW^n6!?vkyw4a_v`+;5d}`<|A4i5=dC=or!Gm<_XBiy8A|5CWmLf#oojx2Ixjq{`+N_`ipDP*A$b@}l9BY;q$7`HNQ!B>>iZ_3pq@4x}b0zBM4Hi<;| z{Gy!DUd>m9Kq@0qQyNa2h_&0iH%^3X1?-RL7keUOzcf5#cl3Kcp7|Sn`H&jA505Aa;V}Yg($)OaN(Dp0F zsv*vy*@JvDurVNSG~R>un4c&G)?t9Dz>penjTRbtjfErpA5IAX1(lCN02^zA1FEzC zU1&uI(il9IQnqlaMJ&zzU(Q*tQk*?Q_-g*2w!~ng{!CM*3f5(O-Vv@l_``K;f^C+K zUgnp^X*YHiQLr@8wa@@rd1^eQ{UN|27@+I(@|B+YCkzBeAVF4J+Z!m9xPwqTsvWI= z&SS%B8W<0oUw9-l7&NG(z{YYLCV_FI6$GTBc%)wqp{)pGfiCQo>W?R}|9oj%WRSuF zfeWrUEm>u{#-^Y96Bt7tFiHM9-o2&m6k+W;u9Y>?jr|uTHKa*EA9}jG1o?aVSv`D@ zVS}CqWl9KO(U`5!hy;9obw*pEFCx8rXAYFEjlT|+4t+87zJLQAoT%aqlCJ3dOxy;& znQ#o>ce9IckWaA>xMXe?Y^EQWI@gHfqF6@OqVOjFpDc(?3n6+K?cdY5IZvaZgjYeb z?;}MHhD=LV>NQpTX;2`Hn~LPcb9T*lfJzKY>IfVh+^QG2rzRMrZrv1FI~P@ zIQ3iJGPr==!87)=Be}FBO8LV6Z;BqfX1v^ZXm{F93zx?xXCeYRG)~^o!o5~sPL*(y zM0s+a*jmMAOZW3jxnCghX&F%Z_zYcKTsT0xx4W2Frp)f~4N#>KuL!;U0h57f&-nw{ zZ#Qh!&TaiXjApP_hMEym7FHDslz*{}cy7DF|1w3bA}2Ft)!_&&^~s5hCIEzgl^D8m z_N3d5QMf_Z6XC?o`~2_)wvv`)&p+l0mIUrjjj{b7=o_ zMtI9*qu8$i!{&BF4(b91{BVP0HHOS2bXjay_t9SNF8SchRD^&28Wdj?cL6pI%YW{% zHmM)Oh&EX8@c)wtAxJ>nObD9k#y3i|odltYy^yIOhzCBTn5x90{R#gxtm;dKg3E&l zfV-Jc1)Skny6i9j{r?!?w*2|a2&)EM?pFbf^j&}SjdJ31k*tzSH4d#4Jv%?os-Ti} zkdRG4@UU!VK{vFG105&;VqnkVd~Uc1BYD$JKOr&5O60>;xNb&cx4VAp7eM=(Kmfu( zmTfcWrWtn8=SPhFvMGIkWS{WDt4@}x-j562_Y>vsM(TPR9)9~*hDK{W+1OKpdQO9f z_B-j`zA)gp2#D%49CzK#uvq3OEH95boFXoFw|yGMn25H+!g zseCVn)bCa?Vh-vaU~VwSMj!*^#VW?@Kg&tws72H7WQCVBi1%F?OJ}a0W2-U3gzccC zmaz*eM^-qn$Cvr4N~O@G>rvTIX$*N-*`U<_hdmW%#H;t)&D<^ ze~OF>A2U>+kBvk9pQA7q20=4n@e8T^e_qHf*2Jb#u`#(?hhH81heJ-IdUKHJ`Q zWWNNIqCihs6s`})dWjzT&M#NkFipM@iAd1lxQ76tBnXgsuBU`eLK?&@9;thJY^bnU zpn74q$rnYF0)7PJBhL9C%h6|rV~_bULiSxSXBYf|%#hg=L3Q|1_ zpvo=N`$;z;0yB5P`x2aP?1oefd=mlC|G|Tt(1ylQ2(sFM{~w9U-@-;DP>MyLA;1RP z)cVT*t8J*{FXGGD;xmvQLt(VUlDq2sx!|0;I$>YZ_vA#AASMz1_@|?_3#fMkPAAb8fW-F-Loo~o89tRte!$kb$TcGy z)P_tMG%7leo}N>^p~3abHAuQ879T@Q=c9Kkr!PQD9#kz zSP@t}L>9|290j}=DjQfXZin?HCLsMz_4TR(9sn9rV59i1(TUMBcxR-H1s1L$jj`-Z z;X=*vl~`ax2dSYQV&$Am%odY3_fXr+lwStbuNeJNtyBjGK7Kh_2Sk6B~qbv0fZ$+ogEnF=)xyVdziG8 zM63u#*f1pgx<=?S#s(%0)YOcrENz*^|p5D`odAzQfT6T&*FE-ZlkH-rUuojBJff&|rnA zg3Jz0CtpHY`D;xy3$HG~ZiiU(J zmu`1BbEpP{Fys*RVED@AIhL3pM;!KeYB;gmM&O`+AG9t#GHzN9B~QU-k!D8Uyo;|d zr?4!fxxPjV@1aIHtGMaBSUD03!=$5+{dZe~PmseRo6T&-HnZtBabk>busP3UuF(b+ zK(}apl)Cx#tYnX#gG;_0x2hGG-?tBg_rl z-`D~y%1kSDs6hM|@pnjz=n^=RfNPB4SStoflYVL8b4q9PS}7VGk}Np^F3KD~TP0B{ z;NK_i3zGugTe83ChcG3uV0ks#H&viFPrU91z|JAYu8`#L5tG%Ya%afBU;oT6V$mSS zBO!0_MdDxR8{h>K|1@p!sjL-Oc-{>6NUt~b^||xkEjhl#6*O|oE*R<0%ACdp)<(nw zO(DiG5&lM&H$UIUlci5^(QAnGo<&br7KBUKR4)cZaitLl-k>mN(?9ULg3N!_*o7Q! zGhDl}gRYZ>;dz>ij>{E}eQfF8XNKz1E|e^2CM2Upg~zE=hS$eb z!4e0%99Q=O?N1Z|`%B!rVlQC9rba$UpgEbOp;|nC!4p_ymxkBSNUh$a-VV+Nb>z`4{nJcC;N_U_ff>PjsIo>sO;Q4DHAXf%#d5OU6 zV8}M01&1bHLicSNY8~p{Q?toxHb%(^*8~HP5C9R=^>|(Hz9R*O(K`JNy&=#~7AkGc zcg*!ND7ni50--{K|5l4FCcP@Y+l(T zd_PYbo=meUH1dzJhKq^|RI`J^pd*(fcabz|SwTT~iM-AujRq?Tk9}EStETd+QM}3X z1~&Xa%$i{DwY@n{yz%}iO#%-wd{AXjD!rhU4x*sGG0=r*8xmN?F{dTX^g;8l?ky~e zk{8VjsE~lus=1|w^~bwd`ZWgbcUlp7t&GA`*TUsyPw^flkzhzxIo9z< z(~H8O3%x%F(y=$P_abxA&s09YRF{t(fb}ms`uh|6h1dJ%g^GICz9bU6C!vqH?!*#x{u=MLmCT%AMY-}Zgh7BiQwQ2AxB4aYrLJUaJ1Y&+{^3UP zb#RK-#WQ=CH~ioDNdScEISd-w;2{nz@K~1zcmx0{n^gkaW}!Ajr`lBzqO?7!;;KG0`(Xt5kJmz>?ZzSci)%gQmzH} z+EuZB%OPX(JZ~4yOrxN7v(1_CieGK?Ptz!akZ7h=Fl z0F~8uON%*ZKa(pSr&~=T0ZBgU&eW;M@vx$7J-25y+GhWgtMUs!f@pBeEoH?nVXt0m)Gff5(3(-=SUr= zx9g@~cDG^tPgCCBbyU z+^NaZqYz7n|ARdy{KBHM@!`)M!Q8gp7kBnu@g-!=;5%XO*Wi%19}GAy{0F=FlXoSg zk5IIVH@(2SKPp+Lw8XO2U^q(gJJ;9MpQrEAsHl6d!2bDimj)F|J4a1wg|YyY^Z#gs z{!1djs6YOrMnnkFvsI}^En%w=beF02xo5yRO35;3WXDk3IGNXYFX*A|N_gni*qDP< zTpI{-MEn{MAYvXxcs$BP@l{4bkfZ=$djaM^f=OM(4>9$!|1A>@R{^#vFDAQ5xg{E! zc{CcO|5zmeK7Z~t0D)#h`&!zMGd`tP53@UyY@>wKT-2~7&_G+%#86ngUc#LmnTS~S ztK(9p=vVceIsN8`vnW$Ym@;`UNS9kbZN@rxcp9Yk$p`tBmW&{oZcl_+$rr-XCd(~M z45zyW9-j&y7J51dEd#JTwjT_VE(cs!zs`x01*IoHWd|9b3MFAMV-4C zG5q)frxy;-7C71MAoT@U5LvuuV2#&br2a=1OyGylLX;pbK$CDz_UBWFxA24P4ss!s zmb2+UnEq{I!zhz8uvc_)j@OvQxwKlEUJRIb+#W)A7Ki^3FMaOV9CdTh?ZYs;`%s69 zBVt6{)P)`)SE6o5c3&WhK%zFS6C#t8n*(k%!>YYg!Z+AU==0a2u|Tst6I)TL;k6kf z7a-=5W%sGMRE(IFm;LU=jSSH@?N=OeUkkWRo0yRX%+^A)+0%c8aetDE0}j57p47`P(b>hY$IWA;VeqAyIbzTg5v&BB{(s=1Yx zw->se5nTT`)1b*M3txdWBRHeADaiYh$~A9k!QbGz(hxdNqKFKQ%`Z^0_xOskd9u1a zO@AkIXcmL<+AetT-v4ys=y@2zv)a5i%K;%>Xe-820+aH!D={cD$M4mx;uy-I-NbnkEDQbejlgPD!3qu{Pem&%H!=i zc($CA@4k5B!gJYw21@7l!({q`xuWnlTHDUY#}a*`R`sIGsC!&Drmj=GI&nDk>E!q- z&wQao((_^ND`5+!jUD%~H=G-eKsLUZZDY7AZpw3q_?A#qaKk|OC6R1>vb$nPbtegh zdZ=#xGP)yw6)FEZp?Cg`F?bmGcBgah$EB)WqYNiKs;LvMpno3s+at)DH1l&XW(j`O zq5ixpcTgpP?QC(2G5P#&Rhh!#`e#@!l zddy+hQ3o8KCkc2rAccMf+`i(^AT&!4>3G%RzAKUk;*7Fi1_!j$Vq49%=#b<@MQd8c z_oo9JfPixP9Zzs3%KmoGkzdb46-lLRHEjy2DRvzG2qm8B@vQu0dJ1CZjLgAyPQO>g ze!HT+ubS|vQ96y^kR$Z!?9uav9%gHYIAzj9K4j{!iDeV)QNY9~?A}{=zJeZw1`09e z+m=Y0;OjEh4=K62V8#Y97iBnZmKSKhkqeB`S2uULSSh6L(mOU62;d@#WXOPpej<6) zr?7UG4?nXhdtGSbRr`(}?k$SVF+>8YkO?|H9z|cAyPE%shUH0n-qK>>KRZxuMK}}Jnjd7uz;`6r_4PR=ZDDd@z(rv zlV1V;Fl%93*O-gcUo!k$-2=JbFTz+cZkKFMP_Rj|XCP6B?Z%;-DYMI`we0k-iPbY3 zlyhA=`L?egx_p+0)YKl*FNqL_az`blwO@hT6^>*MuOCs*-UO|Y=ecugD&}hZ$BE_% zbg**g#ScO!G64t^Z{&T-4Qtla^-tE=gMN>&L%i`-3I$6t1wN)|WGFuu+pase^e7+6 z{AZ8RE}b9NI>Iv658$5Wy$-9+f+)*)rx{QAjo8L*RM899DtUHQ2oi8{`+CE0J#fd$N)q{rDDX zt_xQ6DgOM*RcoeBO;zi6W%Z{?4#n}7y-=j@2rDR3lQhKg9?`G{^fUxIR}BF zM$$FOmE@o+y@im5BbwQdV!J5iY?D8;s_+pj#hG@KhAWac!w!`}9*Z~U3oMfG75!Sk z2*E>pvvu+0x)@$zJ0DBQn8SkCzG$+#@)3;sHC1g4Sw_S${yBv?IO@4`@a{q`U_7d4`K?TR1Pby^$^TVCVgON4LKLHjp%Cg7qeXz zSi5`?0kU@NqaI5t6Ycn(nF<8JrCRum_Xi@5zL}Va@*M|wJ7n2Ut2t52#wH#qyLZGJ zZX?U4$2vkcSMl7q6k{ORMaKnpj!D>d_;qhR8tU3}?|2mgiqGRR) zyqw}0xTYzQen10(e|Ucv5$XO_I!EySIjb*qG#}N z+}0hCLYR;j4`_f}SsCiSO|E;T6^PH9YC6*RV;;>qv1ES_NMG2|u#>EQ~r|DUg(=8=WIE6`hCA4QA2kW<51CMj(co%OT&^@@hlvB zGfRH-%wzrco*U0}OW2kCqjs;tFj^r8=JZQv>bR0`bCQUb_(W|XwM7r_hmv9kt4miI zY(yH?xpGE;Qv!7uN~q#Da7U9d3|jvChS93hWc=&&QQP;JO~knwQV#Gxx*gAtEq0QA zw-7jv=ly}~{Pa=!z5CDyv&7N(JR_@-Wak1sFtE)bxTRf!c^`8S9?5>#q0=EADbKm2 zKi28bT1CGV{$}RB$(Ifxd|C$sq(25t2qRW~NOc`I)h_ODZ;iBzZ&Tlf2hD74W>dH) z#-sdhk;U>Fk>B?_K*Wx#HyX<`V;*dP6*rVOM}e;9H{$kWZ69^A2;CK`NTh40&0eKW zH$H;WgBa)Erg%~-w1tuof$~*PU5(u5n&Yf$5R3tVOUB+pyVNS23zkJm zKCS^p#B?OMv3GxzG&&K33&(G8Nmfob zy6eE>wGF{<_`*akq_>#&WSR#%ipyW$*Z5PjKTD9a5@s^9e8X|Oy93U1W8{o zNsTo!7w--YiK~BFpx5Q8yFL_+_`X>0EOyanoIX$v=AWd=$P?lP^dQrIeG=+A4%p?g z*hWhn*&(6(Us08|TU&V@A|HZsi3=OR$euTH3!qJTtfj&bP}w4VS2u5wNrra=)P$&R z#`EZ-{UDiE%P4B>K~CXx1lYb^Rxp853Dt}=%%Ouk)wcE@zt>8}AUx5(3FZZS zUZB`qKZ=X&7IoM=5VV;$taQDQR+F zuCwE-xEbJ;D$*02s_19}oN&}FVlA0{-!;|Lvz%8LebM~Bw`p>+5paOW8Z}BI&qX(B zcW;c26)l48Q_+#lG4gEft%;-~uU#emO)xw+bN)d=XLTL5R!y8Sm|fX8AyWw!ZiD*u zB@O}1l*53WsN>5ot)(&M@W)J~bOO!(gNB^NOyc%1dElff5@=Xq{q&2+(68OX7sje! z(g=ihAeRn#LFaF^k=TL9wV-brwB}nG)=z5*ZZl<#)7aC=5BMb@15hDc;W*c%=#J(|Hg7JdSPSw;yVM@9Ke!@T??XB$bt< z5H>$kmT17GnGk;~5}$GOr7N+F2=*|C?luZJCUy*=Rk`W-h(9Ap>E5mMDw5}pDZYiq zggY83I{$SRtQ5UR6PmeUH5xCLvb`@$^XhUGNf{}XWDLcfl35N@cL_l; za#}oEJ7mr01=lpoOmjEXxUgzMv^Ini-8r=2L~8dz zC0m*#7V^mf&ER&o`b8RyiKg%}-HKtR%W^3Km&QIq{XK7RNiTxv{rEK1il|;sV)o{3 zCh438nkDOBJvQ1AsNk1-@J@DU+pdv-Ts!hvYmu;tVEamrTLU`^M9@goEKLoFf47Dd zEl*w4u+T&n(ccJ-NZPdJ;lT`WzGec~n!-N+G4&B&5QLh&S$F?^#bEy3msklRGM32+ zb4#{lw9o7huqW9OXPUBrwhNJa3JV|e%b@Rw-a1=jpcmGd0y`5~i#A0GnU2yA%jQYn zF>^Lc=`lKvsGD<2X=Id#pad!>qtK%+HCzC4vJ(F9RrbhusK;yiKs|&%?lnIbV zT^VuwCLEds;SweuaVyOqb?^qt_cFTNiycn9$1@xT^3kS$;|WAk=+1Zd<-F zw*@rH%Z6jo<-16{Ar3(8%l%W~Jbz2Osv={ExhqU!c^SNdu>KPrJVE$>NB*c>$iX?NfMB}qF$Q)u@d8=03FOTi*C|nCg zbQr1aTCXS_N(osOF3GG@F8D8mZ@TnKydD%LVa+oQ<;wLvTowsJ9m?T*S&o-_ zG;lKsGjFQ=*IVq5S$nn_WZFED(>x460xlRnj(I5;D^)OR&SerNSXzIlhy3DIaGlcrZ2+C+i|oNTQ*j5MC<}5>$IbXk zj}AA7ziQ#d>LfEhv}ICguz)#(b^w~H zz@I;PhN$q>fko2d&8MZwN^&N2<2?!+F#wVW*IB31q^nrDgE+17QpABWKrb9?@bOc7 zwLTjz_@#Z~4>b>SU3edd1h zMA)>_<3?J`pd~X$&PX*u2aVqHul6;2zFW{7Is1Mepfl^#mB#1XGf;8jS=Ht1H;h7t zjVxt`z1Qt0DEQ(Nn@xo9p&z>SLX5x5HWRe?O~J19TUp>9Q%l|sRAF&Dkxa{1LHmA)ZUv>5$Kq~z=;-Lav z3HRkRMh6&d4pBJcdTt@1+4cQGj=x3A06btneLgfb*~C!;i;1KZ$MHV3{9SeEp$+0_ z;+Xl=Kg(8CnrNqb_9t>?#5j`)YkG~oou89#7v)Q!xZEd4CB-eGM3>+nS8R_Tpu>E> z>X8+utz6t%p`zMJe7D42aSqojh}{i1&tftZ6#l;JONQ&xKle5rE6xqHs?uqgJ@0B@ zXFdCoKZNplkXG<*9V+LAi)YqlP8s;NvEd6GPo(cp8SEuKRZb2G$L8PlAuJwVx^Ce4 zlRtPc0F73(zyWZvFlaKeLAO*4(Gy2BxKTvuS42eb@1E45($K{V_no~a#|my2;ayr6n|0NGm0zS5G&!drq2Jvy{cwD|EWB;BzSEyM~xyQy`%( z{oJFpG{rd$rYwc(pAWw^a1SIA%Vi#0-cor=JOK~%334?XY61CncDd)Cot9}!Ynlr& zY!c>s)c>j>Tk5MmZUI}X3I>a0feTqBm*?l_cXmYJkx*~_*$t+h!Y@~A-?Qsrj=VIS z;zC=aB8WNVQYb=%3WBM7O?tnGqb-Uk3~d|CE1HaRilp}~cK2iAv$K;k@ggm@sV(Al z4}gU?ko}2_>#6j81w1)FRXyq~E-8Io)XgIl5KR80j%;4T$t^~L-vM^%y-XLyf?V^U zEFqbBW+=O)Rxq(d%1ee-iU!ik#s1a_O4B#)@k&dIrfnjbp^11-7 zL?-39-mq@i3BmOtY=N5>qL3wLVJre%?1yloragKJCkRCffBxR5538@v?EL$<#LEjg z|IWkkxFMW$6qN5MB#T zHrcv*Qr zDz-k0n1H9GI2!&N1u>QVXD}oe=N`P~Ne`l?SsFMVBNe0B=<^cAY-haDe4F{s;O<_kgSGq=vuE zOo-CyV*|&$8-=6`G+V2lV|mDXZ=1;U_W32axZiHT99?(lf5Shw|uAnnGD{vnz!#7u59BVpdfN_VFT z7h1;*Ua-A$yyD18=S)CIHwU-NMlr^OL`P#;2~$AeUKW zmS9F$)w1kqw(wQDD4Fxx$cie^KwW0vD6nKBNm-k3ZCxSYL9wP7i7#1)y>UC4fwyCm zn}iw-Ppk}xViFs6ROE)}12w7aj5i$66(A2q-lr9x8Fesn=F+V;C6JutzA=U}n$43|Og@Ahx zNzxCvqVuA3ao)M^>rKS}t#2BR9diGZ7z%YLXS*&Cs~{loCL=BMsBzEVWQ(}nm^C!u=jLsxJ<_+E-)A3FQp-YQN(iMoq;*H>2CX@@A_M=oEr8hxMsC)} zWK^>@^BO&}MBkoJ!$k)Q_jA|FCB_ZjSt4FnjWOPQ0wXx1>8^?OZh}2CgTy&- z+mwY1ZQ8ndW9?dc%+!K-Fi>)g@iD9<1BsyW{Y7ob_HUKd;q|vLM5u(ani}Jpoilr~ z&`ncJ!XVYc(o}qW_G7}>GsL^3IuoH+aSmqS z&$ZCqt}E59>trtEqB2e^r;kZ0!;WiS$NWR6l|r&6CC5iHdXR8g#m_;UeC4Wo0y4O% zuOXa*4K9Xf%jechzc&1l*j~YO?Nz2TYEuUYFePlwyrmvK`R zIEd!5x0l9LxyOgqDOCNVa(@S!vhmMBs*Eyuu%9Z{pP`S3(e3#hBmEdZHYs=iw4S(G zmqyHoZ^6bBz5V>Yn!$MO+&$@xET^u74lSz#Yf`T+yqC7$=QJyr!j?_f>CzA+860Z% zgFb~D;!U0SQ}Ig2=luMQgF1@}W_pMKc6hB|7`)kR0)RY1B zs3;&aR$J{SUiaRUn9@`=`mSVJlwDR(GM9G2ujh|#x@v{nOJKQ?Qz^FdV-SB{n$E1r zWT+q?O}-5T;g+#+bn&6CEL?xl>CoORD=387$d>H>)KIwcmt3vD_i0)>&F2joz&wV5 z(B9#@wR1)z?PbhObw{TawA8*lb$r1oWKXQ$j!?BfOkU)m=3be4%QrmJ3$up@(}$SMx#%K09ZF*-xXg`M8oG ze`z}$Ob}tyHHR|=#F%WV+z4q9Q@a(^dN)p+@wc@-z6w^CH^MwnPg0Lm?o4SbP9Kc= zRDO|>l~HbhrG+iyaX~AwI4_v_|5Gvm{{__>UuJ)o0kuQYAA0ngJWNw68|<@5OB+rt zn{9O~S{7dG$+d6P@E=!acLxn0dti26PY1*MTb*5Q6!e5f<>X+p;zMjz_mVTwUX7n( z1cDNxznQE5_scm)0xlUqnS4QeJib?UAB?$Rrnhv7J@_*J8yqGPCkOlO|I(bZTaOjr zNAaouPF;tuM-pYM2<8vxyKdBbiM{U`L0|PBj0Q2`$G^~V( z{6Iw|PE>rMOb}*?`_0xHriDlq$h&H*b;i)c`t6&JcpSrLQ2vGvcghXJyy_V`| z-8=7Mp;Nb1JX>N`$pY~|kNxfhzc?VWaiZRYx`z4X z*FNj(eLIX)`<@Z73S>M%izV6SuxER_Gkc`eMq8Wv9?R4ASJJkg;LoRB01=La|B%p2 znrPQHASkWN_%(5Kv}k+k6egwIx8$JS`331ig{{C6gyVT8!gLMTHsbyMricnN2yd;8(ZD#_XoBJjjVix#22UrfmcaAEZJrEDk&`Fp64?8z zCPugF^=%-yweWL4hY-ZOUqadOTZ|Hlw`bRDw(+(Oj(rkpkU`i3X(t64?@X#+#s3&x zFS4o=Cg@AY1t$1@6(oFB0=MY{Aua!GYjwXx-nM;V`rK$d5m;iyr-5TrMLu1P&vHeY zJ@&+nhy`Q=7_{M6?oI9672)9DPx~=eLw_PrmWHYWEA=wr;W+Ke&Yq{!S%)ouEqWPU zkn=!auW~&ndKn~bbbW%Ahv{HDFHB{JO3@FQ-2lUIB!!`gQOUY+OuQOa_rB0&9!o#i zN>|+yD){k>n){2WF{vvFmZ&7Wf6P*+#vgR>CYAilKykfFI80%2P?qsw59~HHWXn zh+!I9AvmNd2#6;g!4l^T<8LONzuuvrJg^{bl*UMFZTHO~UA8c%#>|BBy_Mco`z2^J zb;H!4;^pd1)|;W396!Y#BIZMvrvo^2a{OWtYm1v-gC5s~LIVIrO zEjxN@c;6wNOzF8eo!EK|dJW}$(bi1g!xzP_FGPn__=;bcfT`|O@1}G_P(#3ksC@O* z!U5!Jbu#5Twsw9*5}3RJpY_kty9c0Lj5#9-%cNf@mI^2fZR%uh-*_G`!>7_P@6w~b z=X{&D2qSASJHd>{HU4M6N$^-Mzic~SJ`j)&_K8kmx?r`ec%)rbjNZ}QDhhB)pZ;ng zVRSeeLt-+u&M66+fofUOM(>S>0X#{e0SQ9|{#yZ6@#BVX-Qw+j9@WOVYb1XVbg$o0 z>nF7NB(W9ugQ2lqpYI-a;%GD51nS%VC>5qT>Rg`BRb`s^$ou@)Wj-V#1LH(6FW`}^ zjBf!7z)#66svV#Bx;`d*B_3-9HZzkjkbnN3B#bhIxF~zU)sqss)%J5dMoj)4i-sNl z3cVYGf+%ku$-q}QGJENJXvw|W*GZhh>I7mjI$FeFx^C^{b{MdAMcWxBo`(`la7P1A z=&Bb%nRtj?9u^@sk$Mr}d#7ZLL*XlJ!3>3V$z|oJRGi5l#NyCSmxjL`8 z!rE!7FJrSy=QPad^7U?*lg!*ocCcWV|<2Sb^$fqmy|uR-VSiQ7Gxzq2zSC9*r2S}!Yke805SZnI-L zpqHN%{7R38Yc!b#r! z!xihMd+$UYQwr~bF>)ZubYZbY{i1;W@VV zW-`x-2nmt2FNT@VnURP$=wEyXBm07%@BkJPr;!Zd@L2O0JYZ|{_res#(I`GBCl)u_ z{jCqf@KOf(VE?+FiqS5UT%RhBT;&agFO+Fww`zUyqD0aVq6twnC@v-gbH-yqLEo4x zrir1u{jg=2$_-7im>5;ewE;A8mnQu&O_@$>aJ%*;oxKyC64#5o_;rZSlk34G#oMsC z^b)gb^0#&U4+M{s5j@rFp49O*7f zypT@dz>cw3Km5VA_p{a9QjEzJCXw$;aWU8T$CSO2@TA0m{#xtrlkN`KA-D|iDma07 zpYX-y#~Y*3LohCWMMN>!p@_OSboY(>yGb+;;%A9M0e5l^!Q+~|h~@>&36xIR`jH&t zmhVX!x@s5&g$K;<+(y#zJ%bRH?ph6Ha@wC6SM<@6)r7NiQqzwsEM|kxJ)3!&>Ai|y z%Ig|$#_NV9a!KNDdrnBxH%B`xx?P_*PGebVk5fOH9%5`7CC689GnLWNr3_9w)^*P3 z<0ARyvAhVcjCIID{zDNk;Ci?rZ*}rj1IKlzga0t1?r1JM%t$kJLoHMG`u6srEXaaP zpeJ?7wtjklzDC1Mh;}g5c`GA~gPt&XgZnAr0J(gBnNn=;_aVX`#qdG&XnJk7X&7q* z+|v*7r+&3YieG_LYd$$WY)%^*&}!5u=GLq}&)~nPrBZ?-Z~dkBHX>51%s&F84F9GC z*(2JoIL?~gqfl+Pc%yt14nx4D4gs_#qk{F>Tn!?)`d$8Hm7AAAIf3d6KA#oF@;Oxc;W-UFe%jg0;?j#n04dM6ThzS7Rx(rtbi?kVgO zNCs1w7jdE~;cu>_Nn=o5%s%8Z$>aLDFG5!QB)|LEZbs1c>B3J;PSJKQ}1N$*L7XWfjY zHkStkf7wg6*HyAcT%MeAHk*@WLllA@&_{z2WlBq2(Lq;izs3O>Vx_v@#dFg}Cm8D{ z8B21Ah{QA|*e%nKH?Tcm^rZv1kcir6yRmUVo z{?fwKggt@ydtsQdaytzAYmv_?Gcgk6Qj}lT;5V|lV97M;C)uAf6q64|zydzhfbXvI zL)63`jEA=0b9&3mDoxo9Q>4AaEXr7WTD(G&ZD#5ucip#qqHFPkw3sgb0B`F1K9(cV zwE zTP$QSx8y(sX*p~E>ur)Rmw?|X?eO9kCL!un4$ntQ@NPh`n)0>YzOv9BAPy)*&k(jT z1mnas*E>XzaH9azjai-xl0h_24Zh`b@ZcKH(QsY|O+AMUq>iM~dHL}@c7>3iJd8a$!_7 z%fHfwSCipd{6jPZ0UOwXf$mm|=10*ykbLELK+e^AVIqe*Jh-nH=K&HSaTXCpw731t z)%=19YRYi{aieH)Gz`EnhJ^!G5@6OBl@-cN8l2P7`yk&cz~Ge+M)$VnRDAX(S35eQ zzxb)?m5_NV35#kSOQ8Ik<~Wt-X7!iu3?nvL6~QHjhaJd?C&>+NK~aG}_IiR5hGdD< z<^{G89z#NMYEX>>jY>_U2?k3dcg$m8ea)Bb6OH>C3_MbEGa$VqW_X+N+Rp;M_8#S> z+8geVg2ilbk!_UwT?i@E1S{}cZ*0NS(KM=??9Cd)K$Juf?F+aYJ{Ksb>g@`=wL167 zbUk(U^iUM3dmYk?W-^5+2y=+n(Nq z>&OsV<%-a)0YG^Iu-P&xjr*eeQ@kHW#l_idV6RU2rtxx4T-w}TMYui-tgL9{zVnBf z@Ft?S@Bcka1h+1@0Qc6Ch{f1!w(!Z6_xi?j;*zN@s&m&{(0?`@d@soIP!lhH=GMAr zevq)Pxf^7Y4pJ)riJvfD*ytqXxDoFvz#X>ziO$a^rM=)0s~pegzw!22PFr70v@3HH z#{|{{Gc#w2D;i?Ie4Jb@vj50FL(U~Jf3sFxjxGfIMo|45M1)<8VDb%%D~(jYn77gv z`p7cz^@7WuTZ6rLO!qZERGdz1(f}hV7_vaSKOUFBl@^MsIz_I0`FI0KgHF7qM*Yew z4YtuDVC(Abo|138pJj0#duxb@8C1YMI_zhu$;N zvr{uyGddf~I^3#|XJF9K61X7YID!2mknkhlxI8gmeD=8F55qrTheNvjnQO8KLu>X1 zp#Al55QdJJh=fFQ;Ag+ZHJxrO)HW=iGd$nFd%L?g7Z(?|7aMt2Z(IaE#FtfURl1*C zWL#Cgb_#QV7^^Cn!96k3i?KHF2q(l6KKcZx1I=EBC&J!$u!0eyq46Q~FssT0XRG!E zirpUGdcSsj@wYKhiqrBgCIb(2)t!&8eSa`oLb^3+T{P){9j&Y&@(ma%aWItx))k8m z$M14~B8iIfPsXkvzK&{_uv|7$fW!%ZQIk)Ojf`Bi8_)u88Z4``yGWozm!a{*Fzp*U zO?1w{l-dRx9#~bHMc~G4m_Rsjv?6@3on^7d6M zC93L#Um2C^I&C`*kKjbwhNVAE7Nci=SmpxF9$jQH%L%+p z-)3#T)C?T4>3&`vyYAL(Z9r>*uxNusB#tl&pci(YXuClWATUB78X*3g&S;4<6n*U& z>xVLg;A7@m!C3PgAJOr1d%Ke?dRbLJierWuSXb=4(}!&xM4%x~mke-s_kF;qHL5=E zco7|y!D&oD^9KUDC5Z<;fm)xdw>v}hmU)D0i5x!XMY=1N9yQ<_odpPIa*{Z1w(J$= z%Rg(3c2ksYCu$64m0B(m#$-Dy2Y$ZRq_j$^st#3c(m{LrhtK+kfB#1Q5yTh<9Zb3( zN}RV9ppU8~#TSopYPe4$e1m(?8vC_N$RIYZ#i75q<}g+UL&H zm30)sL{#1U@b?F9;oc}MHN zMccd5>I!8P8VoL*-2y1M;w(3IE68jc!EtWPjOz>HXtFmjzX-LT8 zqOowUd{uQ3NNi8nv42q^jlh(^W0N{Kua!0Nj-o89^Yv*Hf`R`FXtKHm zheojt-6)2VWTX9vkB*RWokjp?A`hIYxo3qT4KvheN_vJACYS{OV=4Nhu%1Z{GGR5B`oqVyU znJh0Jh|tJ%HYZS;(PasBI!p@(ljh#A;gvt3eP}Z?n_%_DIq^jwadPuvr(|1ihb-<% zTw%M=yQ?fEBs+yPE^*i*-Fp0rLfrv+D(_`Bp5%6N^y{PteG-Le5nr50-1sn+&!3nM z0)~5d3KdcuDh9e@tfahT zLFsph`rMvGZPGP8g;I1@ShPpYvte_xXz<<-%G23e+ZlbSGs|&GZ1=-RpSXt6nI6Au)+oc21NFSPVI#}OXN+h&Z=Ki zS}tvRzDo{pQZ3uEWmGe#~eXJome`UAKBDhxy;fpAzz%K@p=JkeYh zHK>22lk6*iL}+hr*iO*EwMtJk-VjRS8tl`_-xYXE@I$Sh5E`$}=J$rM0?hIDHa1Qc zd4~zLxIdD=wn{(WC9r?J-4YvEn2NmBLbLiV8$q$;jX|7L^O5S?DbEQdzdn9FK=+hj zC3>+M_PaAPJ*XNHU%@pEMhw{N9`txpZL1row{G6M;H?`5vSP=d?xSn1z*K{g820jg zwk+UEaP!~S%e{K}jLqwrXJ{2iWT!=d9Z0+}0O;l-+3c#uWgzw(+UdQ}3~qhgaW z?RU$!6UZ$cz|mV7FeU0J_H{A~4iBk=6vWAzW*N{@K{92_k}Q^ImWvU)Ge(wb{K;&- zbqSDT2_{@c0WCK0L78}yK+d1#n!O#|k$NZh!OMv0^P_d6R9sf@zCGE*wyn6*)|xgK&N1=@{)e`ow{AgVA#7=*c}4|*%{ zJbiIFLBj!r34t>e60&5gAtOKKv~m_=VKPQnSPt9Sgz@)PlA9maUaT(T{bE-YVY z2VA)V!t~*zG*_x85qBL6A?H-dEKxMh2CR2L*Y_)cjHdE}O+>U+&&aR!q;9j`r(-${ zLkxTkh$6Y>mt82V@c(52c$<|uDdz==I5M@rFtcK_3{)D0u*CMEGGPO&iTG>^ySb!heyH}aH?ZV6pJ05x9(_fay+j{RZy_K;**~fm-z)5QM6h>iT`)<{6zko%*Ts7> zVG+ZEB27VP+Nt~WEq9E_V*fEQPm9Oc8IM4z56Lbk-r2uwS7OJIHP&RuQ?2L|V^oq7 zqH0EA(FED|b33xQ*^(5jB65OUaMc@n^Y{$8nMjtEI3*ESZIXj#-JSllW7~;%N68vB z$)}Q+`@BWTzMqe&n(nAf*>zo4Y!*7NQkn423)=K8+}EwbQ^!xWDn>G>)yxHoq-lFEBUuJf-y znB=V%!@-n73H3IV>Pz{}L>7lq7ms4kB&PpjETxByTo(|xY%o)Ovte1D;T{xI*<|jN zqoq*}f>M9a?3P)c-^iSr)%6wUW!$c_8kMa>Q z(4f81z>0qNq{3tZOrnKVl{6t!Q#-u8X|Rwul(sk2bcN&8f-JHUI2x!FPrWIzSG-0w zaSECHO>zAU5`VClt_qVc6`2S>!&b2mm)URcX{!8SP(3P*xG z^^@bEKRbNSp1exA0-Dfu=d8;DwEZXb;O!;J3aqaFQb6JOR#S4hTI$%Oi1kiF^i4Pf ztfS11iliayf$jjyA=6!ujnwnkp`V2)dq79%cDm)2X{cPKLVfei#$-3wTFwb+$2m$R z+)^G}^syy*Obt;QKj@nU@Iiz~&o=|QR^{t=?)v=*?ZiY7ZKWLVIB&)eQ-Ey15bk;B z-MlmnS@Ew+4}{>l2|l}2y94Z3e&hC)uBe(X(3Ie^tcd0^%m7YZD<@nUT4h~X5K0&Z zE;=g*VtSYlRc=5%1Ld+khQ_T(6cPvMC&IfkWYaajmBPXV(oIi<{H+++>IyuCy#WVQ ztA10w)K>8qb+7hp@Ja%G{h{Xl6q-SkrfD0ZNY!JpPjt9C+)3MjhI3|Xe$zw}_mwQe zB*aJwg+QlEyHp>@f_&1#l&}rBhFvhLCjUcmRj|*^68c1R>Y8L)VCbo$$PiEm?~z_ww%AeG|lXbbfa*H=Wcrn{WFfB;uJ>%+|eVfXJ{3uxRq@aG1O z7iD^fH9_gpha?Y$2R=beiKx2()Moh6EMV$7_(5!u$&e?da~JkiVCk2y%%PiP;=WWeK`s7x>22E6h-)b)b)$if1E7oXU;NYs$5$YnHxJ#?eKl6<1h4o=3a<;0}uG~JZ`D#DEi@ihPv7mr7+S|N8LZUgW+7yX^j6tUD zTNH~j=Wtyv7bw#sO8$_uhrqR!>9(u`BuphS>D25wGXSnqq9Tegw0ov2VsPDmlGSuJ z>krwAUB!+57&uVjp}2lw;#|tSU33|3vT^Xe_AWb5Qd`?YE1yIXz)`C<<{kGC_zF;P zNs)6>qsllcmivs0&kc)D=rES*n4r`f+><-Kkl3GN&MzE8^y0Or-yb89A>ga-8)0t2%37 zE~*0c#!IiHi~?JN+|sAK58oH`xdE`9@o?&t6Pc810**iEd-9#pvT9Su!ST zOj&6W5%ov4dtUtWh>H`dVGOk0nC+>UhYoqu2tYy#{v#p(1=v$M538tgl_ zU3b6cactIB=HfQf%WW*tS-4E}UO!#AxSiVow7pV`fD~8Lcha3{n0{!aCk@cVyt-%O$r zq%GrC&{M+DYkle(yv*%}$v8j=n+?W|?wQBB*dNb^pP}58c_EHo8kLZ&Q&*EIeN`!b0Pkb+5Zv

    w?om_3B~@c!9joNr0|ne547_5McM>bY^xvP@aOZg>Z+sn8Tx}RNk!5z zQ4hYySI6f~1qh%2buaA(rM9xN^W%o6->(Xa$GvYv)Z;rlrYH3PtQbT_H~$kyk;^!< z!T+s`KclDTnRr*apts2+qQSVdZ z_u0;-s_voLM&Du{K50S~McY7^GsC@Y}EgyQL&)0<`L;iz7+NH5>uCv*4q??8 z@nB;h?IL@|gD#5Mw?%r?Ou@W*Iu2hbWKINFu537JF*VN3>{7Zo(pq_!Bv4GLAfF~T zNXEY@DmFPT!ER91`QxhncrhkF-{I^p24@2brc&9;LOg|+^$vjzpC4Fftx}dEUgK%O zr=c!!_!BKi?=>joOHb2(TT)+=g!RcZYJTBsG924+sWp=oNN#UKKZjf-olN7ZSF-{d zEjweAk_qfA0MnA4k$o{YZ@QPxUjlk4@ye4R01`*v47Nd!1AgaX+IaSt zsD@WhDy5w|;0e)c+^TdTB%tw^GBVJm5R_D*k)LZ}yPh#r&gJvfI%I!V6meX&X0zh6 zV6AuwVtD;d?y%^0p3OQ+TXDm^<3-kkv?-qo?c?Lnf+QL(#{TWuJOZl~t#;71L$~EL zSsFQk4OUb2N+V1h9J+e_k1KPPqpyJ-cWZO? zb-racG088ef~V5%h{w5g%Y}{}qCnwz;#XICF=s0mJ7~O5PPMq=Q`aTGNAG(+_*pLZ z)zyCiGtT01myZRtS(P7o=&Np4PzN@nw1E8vT%?H{=>G|WI8q=#Mc;L+9-hz%JxIKo*xOry66rPh8(IYMaWCT`j%TaUwK@;C(J{ z>80exS254&$$H$|A~2`R?ao)Mz;_XO9Oe>;_!SDlJ-r1QvsN8^IF+-tl{At_4|00h z8l3LWA{vKIYVGs2i$n77e>ei@9gqzx1DY4zyWfdbM(l$Ar1B?gm|4`=;KG!So$fej zG;>kSnb{>^gr93UXYliUNXru2nqF}+w?YCA>)V%VwQ8rc$4<;yZFeK~HJ!0tl*6gU zr>jVGa5Agf7Y_=K8nhYpfr~Tn?_u<$BSUR7sqeGhoRpBrpta$9Xawvw!JEzIZjH9q z0KG(6{U3*DdT8}*CDQ|FTt7PC|8C4A91N)5?IiuQFN^{c_5$Ag1c?+N6^0@xD(I|n_Tt>SquOxw8KqbRQ^DFF}m38uFV87WQ~L392(q) zUv{Z&cYLKxEpR12j-0>t%orsEGNzfQM*ilY%WG-bT*Cy3Tl0UNiTr$H1&-bw3Y35% zE%|Mrrqd35S4XGi;Fy`UUX}fy-0z!80HDZp)v2+grBt6Z$QS6=xKjCAzEmpb=`xB5 z+7wz~lBjoJC(_Rw0!EzskijF4+ZJkX8ElE-qCg=Bm z>880~+c9{x#>~C$eD<5qzYVj|sX-+dy0?_vuUUnL&kPJX`0tY6hkvo8*E>d$>J=J@ zCQ!Na?_t#wd{Qr(shGe982^ca1H6+rLlB4w0vsAhDvf>pzX`iXX3I?V*MjpAfWbyZ zaUl6Q>uszD6hdBi=`!e`CW-`*J67tL$92nkqLnB+TnG zYBS+SL{Oycq-SfvprkGU(EVjA$y3sP$i}1_(7ahs#yAN;PQ%74|Bb=)Lgd#w)USz{ zF?I1vyS#sb>n! ze19cD*j!UDIdXQiQkGVuHBQ2kWA2O{+U1UNmhve(k}@lI)NNh?W@6u}8Rz}@L6-6} zgn*D0C}WQ*`CJ9d=+!i68Rh2&a0a{uOSOL`<8O1|?osxLP2N^8cZvC<5QqV_gn^rKTc-Lx;Y~ z_2518!GcNgC+XeRWB!Z4+7 z@mU0v{Ot>-fpdN1!>{M@R84Z@oDlOu9C28}daYX5UAyQ(L5FmqF$U5Sn;(SO&$`2A zj%3`v5gV_)%Ii!HO0HtbyDLIGo_MHUl81$I)`(83T{PKQGENP^RPxf-Mp>FB_HjDD z6oreJC~;Sj4tg8qd2H-dZHO<8omXdV$+&)HYVy3}(44>~(Lcp7& zvxr`NW7OYtc8zEAmQseD1xYap^b&P=eYWi1`90(J3_Qjag(Q9eK@7V&?brN$Z6;Q zIeHH60i7g;Vh2dtHY;s3H8o`mbPX)l&{w_`-xHSaEMjz7IISvHT?RJiFy*1PrPboU zcVfXc#$WSNoYp#-Y5p{t1GavO4{;~VM@n0uFz@bn1wGKax?3SpAyYQ^0+Mcj?SGFZ z?+_`>5RIBd)O2g+fJ=fVzyM+xK*N{8;^cPrwR$z@t^pL}y1KnO56OUa4YSUEaP`M& zIC}N`K`CqMZciz`zZ7c;jrg=N%n3+_7+Sz{o`}5Pq^98Ja9Dq^EHyj!`|8e-pTcx&1)(Ifm>v})@FdMz3*_lI80l_sixwc2H$Ln`2=>@;>D zmRo`Uk%IkUME$nVVrQ=s$?SUdm$@Tm_;(Zz;xm4I;BY}NH$)pl)ZWl)tcyCHYYiV=+BW*3@|A&sNiEW4JvdK_%$ z;W$|uvj#}oSR8br*(lhtcR|suKIo@ye-BT6Bhp;1%v|Y+-8Y|!%gAutzq0>kdlWAn z);$Qw|73KE*pb3@ELbQDrGV?<&zfF@kWi|u+x@Tx%9e++eEox6XwLoCo7Te2#p#`cBW-a) zdcKcUZf(7In-Zj&Xx&lsMc6OnuoYk^w*5Ra74u02tb%N~&d9{4ou0R-d(58~{!zFU z^T?)QE3_;UP)GanZgy30p47F7h;aFMAQ9lrZ>T4svbanYP}9V%H#5wo!s~Zu1;X(Q zC43YMzz!majz0yp3|MyAGclIr_ZGjG+HId(F z8vIpCuxqt$_fDAy*O4o{6nj8MCBqb%#upWEl0)(@qL@Xj){^3IEg-HFHEEi=PhJAq z_{9dU#+`t~-#A}fcE1$PxX(!NrjWM#0Z4Iiz9Sq5TpP}@(}q_ybduC7qn%EfF5t`H zWg~A2K@B!}nwYC&{gJJBq)%b;KhE(lmc6ndeahgiEHhchh#fK!xz3E9JaJRhFwvm# zM=lX)?Y#>fUv^Tmcm1TYC#d1tXCc^NH(Cma7wBi!u6Eu>2Oat9LO^@l>;r5?8jI!| z&9=*aGlQ*?4b8E8dZ{WDJqzs;pwFnn9lAq)@bdR#jHDCGd$eWUTRup0O&UKuKCZg~ zQu?n6Art1)$ZjLH- ztdx++Ig*n8J(ZLX|K73r(Hc&q*+?>HW@v=5paj4Wep#?p;y3g99+4rU^d}6GruMzI zIk`l2f(Ib~U9Kwp!`%@QEDwD;`^vnB6vJO@(gW}^jEjj2Ppof>rWKep;Mv(zocXh0 z@c@Cao;{)zHDEZ01$bM>M=Jk8z9r8h1<+UoQV`;7Fv=fRE`O`}Do$J(LG?iZgh2%7 z`pi1kfcO5YF0h;N45{WF^Q1S5$ql#9W_Zf=w2zB?Y>uW=`|J8~A+tJeM2~EJcMfXX zZs3sqHItWR8Y2?pbSX@=0;?|*d3b}?Vh|@v{amdIqF2YnNQs5t(voE8q(aw;^}^1W zLX>b|(pP?9D1<@x?Cv}5^5uTZGlFj%6b;-#*q#s) z1+_;}Hy<_%lIG9=F?0L#!&`kbP3gf&PxZ3?P({}5i=j&L`{b|32p98lGN2Z0h8Xb1 z2760mYmH&?v57hvd0gnvS-af!_Qoj)Og`U^j(kqq={B9IFPfF*W$)+4-1R2&L)K-S zQ4e2qoVP$tP7(g!%C6N#rhxBzz-|Y{FSh6X!V=Nf#!lYy6HW)kwtdEX8Dc?D)@N8{ z+fbP86}sqC4HaObsGspWT)FOk!e@$W)cxl`-}~1tG3b=dp|S$o55v3gb?kEZyD^0_dds>zjs_E&;8I2*J(KdlwgV0^nX9YzCFKQ9F$z&V%1dscf#n!~vU|zgE^m2Y!y`6^TN| z%Du#f%a9R_$!T; z;sHbS>x!P|%ikwbCA+<_!{;rHd71>!j7$kE6$P6vpSI{2F$4~^Xh`Rs=5MFi<*T`-@xcT*XdswM~kWo z)l;gaIV4kqLD;b4V5OX3ILLKAgU&lkolUBZ%^Ifp@*3fTmx|j!TYpNpOM7DSPDJC> zpUNV$R4+R+T(vc|)f{)O78E7+K!gN%xb^B^Ix8+Rab+Wi9OGcU&IF~))pWO!VoT#7 zMBw*y?H^BMT(_M^op28>vto$EMXpEt!E+ZBm{1%)zeqe~wi-=sdGS}nz8cV8oz>h` z>6d@(Cgaq(%-#{QxFFZmS<{#~UE0bme~QFwKmxsgzHprQB0J6()jzfW*y$4o6W>9q z{SihXPC@0pbQ?v^K|Rbxe?glzgB^PK7l^=ALD10J@*ZRRtujw2H0?wkx+$2VABBKB z%@tHQcE2d@SV5KILTaEbmJPzkZ!NtE1e&)Zg~h1HR^0GQiV!#{7Cb<}m10-zg%9`D zYO7TwTPcinDi?7wHoL%xhN+NV??IVf;#H#S^>LKr-P8BXoz1YIP{ZmI`=U(ARF)2o zTaKO#ad#;rZL^sbpQlYfi6?jC7fLm+LN<3mTKf)<^v+KcAsE-7W8$x<81OV+V~^28 z{YzN_Bk2oP#J6w1^9&-(c#p?PJhYdYze ztK1q1bVuTxD&?3T!x+Rp{h19E%zPHdKic<>49~i5j+m}(C)M5UHw%@2lil4`(UcH% zFckASoF!0ywJ|X!uo#ShnJ=;hdOn=3jsKz|9W2w}@-;{KPFG1jVRbpjP5S)<^zxcW zVvj)mUMwI2VkRy;3g(LYHZ)4zn4wPDESU))g$ok|^OP{-*tK5D87ggR%9dS!;Qb4n zgW#v+515jgJh^Ed;+J9(-6Y^6^Ck;TsJUP__%2|Bq6x3s;ualDui$p0BA3mDR$#&L zknU{0NZ8?w*LNT+m_k|DMqmU^o{CpG;ijp?zIi% zOVW_XF2LlbryNpEJAI#dhLjN8Va)3YHX2QL4`T(eU@t7Y=>N6>5y#1_`$?}>6Au}e zTf9rod&yB50Nm{)uk9f#kDsoG3HaB^e;H>baX2+De_Mke)Xr{|vv-{LIO{o65If|E zSRA)lFK$D)JC{W)Z@Y{E)Pv!aF?o@JYy4;F%dkj*i>pBn<73WR?ZAucD8q+#T(pluBvHRnLZDkQ}v>dU2j!q}tPK#IgN#69z$E@qT%>_t(FjamDh7|!WC-aGpBgsAqj2r^ws_Ie*6`D60l#F>a-VG3 z2)6D3J4v@T>@^STEzdu{-|^~qjU;D@ZEX$`dL&la5m5Cg?~}~4v<+suWF`3u40XNP z`S_G-nn)3^>2X4GSwqn(Y8Ft~V?+B1YX zaQ^Onj=H&2e!0xLwf;zKj>!P=VWw|efycKF1JBk#3+#fEzJBN_Inrkf4Hie}vNO3G zZZb4mO_uV`U5oMusXO_@ggQAU97niQM$}}>h73fBIKXMnT3rl;w^mkIz<<1B zDk*}BA~8D@(4OUPBZI4s;Vn6>ANRLu)>R-9?Lq|1e)vym)qB|sgT&bV$Pz9(<#rmc zhx^egJSkj+=d9UyM^k2Y1K7a_^m`jRsC?^5M^Y)f!!xxe9m*r?C!_yw_BVre zSLj*6S>7*>wZ43_0^jh&j`X4j6e2X&drYE$Tr^lQs@WkVu$bb7KFnhz=z+Vn{PFxo z_Bp&;|6k+zvR}@IhgTN>gjST1s{8+`q<3Mrt59|sG`Q*l_IT` zBx;eicuCy*0A^e94{6EsZGB%T5=`zgB%$!V^nVFfHC(O*u%rs$U zW}Gl{CKG06PM*zs@9ygRvC_TLN>)qmuI?_E?6zH1eNL$I@x*Z$g>F4^6DdSVp%`;! z??Kcn%pYlzvZkY)h*bKE5Laj*5Q+Ahs{ohm-0=KO^;<%Dx<`^i@KyOQ*x)e~Jug=Z z`GLq*Zc-T>XQ&1{O4L^E(cS%z%f-gEvB5bHCafqug4bmqifMv$A>A(8J{$kd8mRg) z)&i_t&IDf=C0x$cD6J|oKDNwBX_UrekyiuD(YecC_h{lh_?o4t@i5jSkf(vqn;GqY zReF{}GE1ceyA&}a*J#<=s}Agqi%i7BocH?E392GL4M`44!oKoVrU+9i6v4j^^ zY_L!m5p5&>UFQ%aKHQb&O>~fvq>=~yPin67>#d=}2~rxbsA()RIOL)Oub!nF8&W#X z;sTcaG`|mQ6^JuQ4L>pg5(~>NHB!R$9Eu1(f#5?*0PN$YFd--Q?=y?#-JK6w*k1BQ zxYWyKQ^~f?dJL<{;jI@VIHRUMYgcUsue1#+Ie6<*-11E967<|d)$oZl*@psQwo34$ za4}YjY3JwV>Ly?9GV}`i%O#r>AS`_5GU)!X^#m7rxi@lhAPFAIY}}dO$e6COzSjh2 zTRrwWb4o?}$9lb;rZet2gT&@__X{jb%Pi-3Dko>VfWzYx2b_&4!Tx3j zAC$+VZDZ0WU}Osl+y16$;YONJ#n!MAr`?{SK?VO%GG&X~gh=0A)V1aHSeAnV z3h|zdetF(tm}X)ss#NB%67j$5uiwFjK(Na3t88n68X}2h6Hoh#*5|>t3g^3|8Dv^? zP%r(z0jNoWKKh?O5D!H+>2nLuq;q9y+{&$Aj@vVy=%qRTg#YX#iQ@^&&643G)q2E{ z2xqP|KCU7snv;B?4&P9T>r~ARV}ab}gEN1OF00jkb4kT=CJNi$_LS5#|K49JvBK-a z?C{2TP(?NIvZD-PBSbgWV#$do3A5V9>EM%wl%d?&xhIcuq5T${*AxkB8JSW(gECUub&BhEb>D*$ZSRtO4x8CP-L(^DNQDy6R3xxP8zA zG1oY|tdVl*=LEY%Pl`AueT}i4-%`WW;qvW?OpS?RQx8x{L-K*HL~FxLB$n&-OxVc7 zifHlAC2^R$M3K=t;IPvjKXfZ_WK{l?2*q1-%5~PDz(Tgj+l+gp*bj+;S|k-52D;xC z{A71-GY$2QbDJar&JdgDz^e{`MRHD6u31GT!9bCUdHx36=eR$7@&UF4_Lv$Dh$$#{ z1Z|8>h1LGSTzBpO9gz7m*s}xXP>s_GXAHKycCYm+kN@0Otcg$)^Pkjxp^%a9x2! zTVdKT$M3K;+?N2`1@t^sKK-%lbNOT-A7*FiURTm5G%=5}T4>Z$g@X2wY&J%@w!l&f z=!rUW3VhZ;!-qY+7~qnR83=jV( zL;=Q@nv-`|)n}>rX|-_YB%NH*0e>GzWpLG}06*LG)jFTZj-7JrHBL-e+jNA$*{Mh4 zPtDu|N%)kEdiE-ng#$gSv2ltkk3R(;uB~Nc49vu;J>Ep?9{Zy zIU5I)gU~xk*2RoFPmUp^6(gsDK{_IezMZY*TYB7m-LsG8=K8`$iQ5l9RLe}lHTd2- zFB9^&s;DpfRAeU7Km0vO)@xxoESA=;DtbxhiV@N9VK|1of}JfhiND=_``737g|BaV zQjwJE>T-}lqTH%|O?JATfa*b!B|xF!g;a?gTOj7-d2lY*QjMV z?7EPt+9<_Rc4tFHIF%75TUMN!JMSsX0_D+#?$InHL$e@^d4;3nI`o-_+Yu!>#!xC% zz_#(&Rbo{pBcU{5c*Rzdz89Amp0F??Izq;PUs);4uT3$Sgbc>|#Dg;a_d_VC#m)x^ z9aYXO$2grP`X4`m5Tx;+8g3@k&OyNk5vdV06h6~WLaQA^fP-B(g(vBG@M#3FJ9ATi#utvpZ_^#MD^8>eH zJ4AeYiB>3*>@>+wbWQM;F6icIX6o)2>$tn@USTWuPAT}7>SvSlx0TRH-*vZC@&pHo?m92n8Z1^FWWc|N zpJ$7Z=+-HMRowFSz~|}umKp|RPxLzwi;xp zZU@wmCYz5~#~_{%@15 zkQikliX>bOAF+$QV0yb4!eE19d&~1L!;snLe~S{K$(eI!%q7#d!$*?v`0~*P<}B_= zd@ES^*>iA6aZq}M-ok!@q^5=)D?Yj!))o3!bFF_Y0U(!uAj{9e(+9>kc=e+vWTP?N zqC#A*ucwhA4S^{102nWcOPno|qeXHYpYqfYJwrY&3S>L8V8yVML^#PtmMvp^ov-Hz z^>MKX*jV4}epYOFxxSamW#izg2w23XNLHc`^+e7EHuUDh>Siqp6u78YSJ|}ZQw4yg zi464Ei)V|+cZvmY#4x0wSE-NG-RiejyCpL%H%q&;Xt^5TyUl z%|U+LSZ=^f!w^tRbnYg(wPzwVA|N&kvKo~1{z@T%ic%U%R5#zeC5lR-Ix4{Cl7rlKJG!sUG7eeFP_;Ir48!d_gtC2wa8ke+ClL3KXK5jxBEv1|YrY`PUyU)g6 z3RF0hbf{gxw_Wpk+o#Nvh%1-J=&sb0oH%njc{(1u#h6;n!OtY0e=?ZcS#05VJsQ5a z$;cgqN(Psct7c?E=w&xJXzL#oFG3B&Bkdg=WJV$Y4^ImllO5%`l$XjVi)Y?(17Eb7 zg>t>tOKMXdoOjg`{1kn23T*H=Gq~YMpw&N@Y;^VB{3-91wUFi@EnpLa%RL3fm(Ceh zODyI4p-fC~1`-rF{_11Irnf(rzBYA;*rr zek~MmZ1kU>{+;!22}vy3D7%_F#xB^jrD(uxMJC9UNS4g0I*i8=B&pR0Pig>T-F3i0>qgz0upbeIatE@9_h>#C_n7;xpwyUE#ANcnaE;(dxb|P&}tn zlR8=c6YYSiEtD}yhs1RTMovWGC(!wQQfYk;~QmUD~yu+6$bpv;hT;R*A zfFcWm8S1f+Zn}WyG||u{O)KsZcyoy|7rMu)<}C^J`sx7BAnmhf3H1zb^rcARk;(bh zFscx&-uq#f1+$8N;YAB-orP0{$Xks$ndDfy`As6F|v49#Kumvt}`fPPTdt7qr@6`j5C%%a-U3B zZY>2wmy<8OvP*HnC^#eD^`Hn0JlyZ=SCJqre_>vE=NOo=y4)7`iXjpJBQPz;M zOrRkbU`cfEqZ0R6EAA8B+|R?)hT!P*Sesk(Nt4Ot|A|mhVsf%JTnb`iW*LV=ut;!0 z^Le6eaJr#~ZH~iI_mIT4yYcLOmW#<|s_Uxqk>&kzRFd^kv6&Zvnr!r&>^ah*nEt&y zCYmZIPHu0cI~qM*%M~*@6~6np*kaVggX|J7{8?Sh9?+<`n$^-7CK$q{fZMx%aL+H7 z8mR-pq>>7%=BVDMzw0D-!D#yEmR#F}Lggw&qOanq)RAj$yc#*{bMhA?wqU;fy#FUB ze~fOYs)OVe7^68FmtWFxRWfdC}DbF1yA0`AWRL9Z-Fd~VOl+nMvy@osK3kdK%WRI5h`teFa8k#XqA zwVe=uOu0U5Mg<*Hpj9*EeDkokrYD+TF(8E9P z<|-8DFms~D4WwR^M@f_Hr3k2peULu%P8K+NCWVWI3<(6s#N&-^RtFnJ#(r<}797La zp_j1my}b6nQS$Tqvoe%W^FD&8I?|o?>nLzxV$)1eQ5d7Yyq#sWu(_VY^ot?LLHO{= z3s)J|82E33bvM|ZRiU}+EifUD0ISiDwEzmrD7(Qv94LwgdP(b$i<=rBl{d+Od3}-y zc>HrrPQ%##=x%A>{9eD{X}P^ydw5NFGTz4=;tY#%XQIH@yQmWrU1Kuuvz%c=Vb|d| zvoE|s1G$igbDcqc5z9`+p+hvS6>^6J=liv%8*}2|h(&~%u|x`E$Bh+I3f&?={O$Ic zDash=`S@!ecAz50iOufxY^KZ_EWz)r`P?yAse55aL)}e7NGS~q;y|Bqxbe+~48m!I zHLBde`J=OC1Ezi%v-?g6dEug@P0x5dqQ%*i29jpyX9+{jp$JHbP@VA+o`2D36FW9hbVLs&G=tKBwIy& zB^oW=EHLYQTb<*}%`#=V#hru(X)^zp(jfz6rKF#ejM-ytb4+yMGv+LPf z=S&Axn4r-enlab#TspW*Rr%1y;>EMiyRm+Q`2hgO597qaW! zo(K9@_#P+&)I&fEu%bN}9ny0uVUT|r2SLftjUq;}^Z55P`dL&qyHM-#PRS7ianl>X z5f4w@FowCp#n4?`=;1~#4}{rrt`wj1Dq>bAfw5Ht)e!WE=EReScTV$`lKThq5Kx9S zTYx<@#*W9fLHmi=Fz_HEhj7yNkPH|UG9nI|TK^Q>VXF2YoFr+3l3<2fLKj7t*>Lgu zi^^KyF&(8zcEWuV)AU~6Nm7h@sdGxsK?+gJZv-DdfUIVOaL1an*2X0RMA9E9aAq3M z^Ge*ar=7M^TPQ_&$@^lx*#|?M-SEZwcR1(0k@i3e1Sn3uY7_UQ6(;XB*K}f9&{mDo z&;%^+$uWkK)(awSQrroCS%nFI?T_c|TsOw)h|xur+M7)rBxCjI^;Yu^wVs{xL`cc6 zbQrVBh+7bLte*ri^orWiq+cN?Dt5$76C4!nDNo$7UgoB}GVp~EAnLy?7d%+pL=K^A z8bp&0iuJk~5zz&$q*_3d#DOt2#URCO1ikL-O@mDDOLv*Jyv_(YbHqk}^es;!nvE-k zm_hgi9a>tW9vSSPq!;fbRlRCh%c{tku@D8j0TaU#2zK1dl%J$A*At6`0@1MIOyx&| zP;jv@>Tk)a$*Yu0wB~~H#@L(>QxC;g*pQ@g>KSa;mvp+>Vn?{cfaVgb{qBrOlUyb#$y0KMyp2>h*|jX-!M7c1b%&X+%)?}K=9{H*#wqCVMMV+se5L&u(-+KFO!d7h-*Wm zaPE0?)~@KCHh5!NZPLfd)>r_3 zJ2DBt)k4RRwu^h=N3GGl4P1M32oR)(@hLs7mCr++E4Pk>^*Zde_(|u%=OGM0j-H4f-`}dTVe9l8;k8dCru-)(WEQ5@&+7 zo3okNuvdNTg&y>^t0`ciTu_JuK7i2#W-tik6`#y{fzAWEjN!`UL6WGo_? zcDyl_qeJxSNw%GS0yJ16dV=FR(=lO}hjWeh?jvgeGr*PUsn`s|9jZF2|6S$!lMmopP^lqzv1%(D%=FBw(n`jQI>2W3T&>_ zxwsTk0Z8afebUG1iYf1rc|pF9nHb;SGZSiNNSaH?@%5wRIQp3pn@6B7y)VTX>n9B- zK-TL2U;%&%V+j1_?8>Q$46_bjSLZg+9lx9dVe4mqYMRd3QTTBOzxbJIVw|Qs(Oh=8!Qq({5WI8dTb}~C+#O>N5yYy2DEBl3E4cw- z>PK{m23{!=<3v>J5dX@{4OxjtBX&H!Pu@WiZ|C-!VT!lJ4$qJtF7+D^HYc3l|IPqc3=UX*6kKga7afxTLR z!+*shN)`_Le0-nL0MUEP)+fQ;6j3Fd#pK+Zuw#ou0#W9z{*|?o&xBureuWnnr{pd2 zHERmEJ9410?t`_(eB#FdqJxL8y4{Z?Guu&$3^|m5MUY;L2+J9}@QFvT+av8JUw}=L z%z6wt7duw(9Fx;g&%AT^bmkq~k2-Ca%sQ*gHow3urwZ>`ZU8L0DGZFVw0E*hTv*+? zbghz;lU)BtqH6GWvR=o0J5=lANWQ>);LJ@ZqQoo#2op_F@R*Sdrjem*o6jaf4+-RSUTm6UmzRbu5c~HX`5zoOVq(G=jjF`MAcGznm3uHioCa5!lqkg?PLAk z`~;gZ_s=g6Z-U%z{`0CdAXdrZx`;KcpZjrX?pJ4LwIZ7rV!$@4dfw0If%Xl29H)r` zlhiWsff7>~h?M5lR^FyzqUj2yfc}_p`FhE1P!qT<{8_4cyO6O~-e2OoS^zV^qeMK} zj4T$Ypi!Cd?o`0vJW?{eG-qMRi#es%;0KB@5CEOOcSQicxoGs3JG~>tL^?^8ITrAe z!a0aSa*RHhaEMGAGS)d(u7f~GPytj3TjzlFL!uP?~|((4RSi~3#+shUV(Rk7^n=D2nfG{#vJ zC_&l_QRmX>dV4`Bc|xOwnsX9bh0t%fZY5OQA9_oy38kJiV^*_1tgKD%X(lEnuDPi- z>zM2&$MH)swHY-F<^AFx2NAC)XXE;mrF!7hYKB~@_LUcZq`z~*CRqkNTnxD~)dq;~ z5OUo&#QgWKMGzONv1usUcAS_vk^z3}Y5>JN6OH!9KqfJ?5~&aWK0-lCbhlc`UQf@v z8W^J(ut`tqB^u-b8pO_8uTj_7DLlGxHd=;{K`Fi7NZ2x$dnr&aR$gIIqod2umN67dTrZcFt}2Ax^Q496pL1exBro$GIu_IEWDfb^tlEC@$a15cTh#a0o3b! zV1o0Awwzi(G^a9#0}Pjqs$z;}SD`}2VYlaV1nlGf*@ zI-siG7&8s%fuu>JpGy~8yxFiuW_Lr#Y-O5l6v`vmtufDf{OwxZYwFqv1DQ+Ipi3X` zd1aa4GhU{_*%eCzH{D!mP7V%6G`#+z_}Q;rjA{P$s7rUmOg*DnxCmCxOq~^^3QbyX zNt*rmC3=W5Wi_f!1^c@24=M>)IhIy~aW> z4syNzWHSmk(mPWNP5S0|tUN9l=$X7UgwhBDCK;K5>5%cFRFOb8*&YDa=bX2_^jg)7DGgz)QG3p9E*+})Y>{rB2ItM?;xb{@h01%M zn1j5I?S~(?t1B++1TQm69*Qr7HN*ss^@)*DC8n{tNGa-z$Quvri!;oqm+C#?F}Fy z_mdX&=6|-(Ps7dV@{jyL+Bg&CU3WE=B@1qqZ3DO6Z$W6Tl?-=xT79XVgwWN{$`QDK z^dZ%KG z^~8BzDKy>4s-3`gDmU@ZT**Ji4m+btblb4h20e0Gi{pi>ysXn^oK=32lce z#**lPRy9bMw!@P~wLIe18CI&9!y3jOv<^k)P=KJDA|LGq$|PfAc|4Crxw*+9Li zFu*i#gWZ49DPT0s<9XISSACJ{l(_TM@}aZ##TtTyU#lzRJ2IlEgDl{J`7~MDKJej~ zJPVT_)y|L{S;LmYl4^qu!5j(iHfi|Jr-{QDC|GE4Q?779%fd#Dp$Ux45T6uHiy-=p z?xPS?&qq_aWkuJA4u!Bokh_wRu4Tc2v(h2aw7r~2k`NXu?@}>lObL%tH-*QQzR0Rx zwM}$efAkg8k^2%%#a!G0;-pol!y4)|#f$`+jY>59E{x}j#BU{MDPMq({!Z)aUTNFu zA}9d9#DV`l5Y+4VWLp7&V2o@(SM8xN{=Q`$-G=2n%@ct-#<0-Ep}IF`4`}o4Z$&6% zV55U^d1>Nez<{Ya20i73r>7`4YMF=#Ng3dw=E2Wz{7&@IDcXR^y1nR8Rl>Xpd1f@q z3i2V7Ro#hblL~ql4~p_M`^~;Jd|v5KTK8|AMPOu zkUA*NNTW1?>~h#l1<~;gtuWZv%Z3k6auc{G<$g%M4Fv@vETgp%^^y z=0eQ*T1U+&Po3Es-SzlXr`U|qx+PuV2oV>QJnkj1d`|6BAm=P46t9_X(65})KQs}B zMQpdA&+5%c>%FjI1b+fX>CJCHgTcWs3YY8FSh1U`vSk8l{)DgOKhfd;vYtX> zwj|pWTl_>@R0&3?b0AF@jKd@(Ep9L3(gmAPOZYVfb6jrbgDNx`krWI7rzCJktuUi`ai}RHL(d_wX_!*zCP8-m9hApZn>Rws2J@kvHo$N5 zBFgENU^jT4-pq#FLOZY?kY@QourKDI2&;WA>Xi&C+C|@EM*(h&rAvl87_u}GSaCK> zHVl@x)AYxj%cW`GU&83%Y+rX(Q1U{joDt-OSm(kGi z&!kzoUpyp%$>Vea$<(YmDtfyU5v2Qgt|%AH2;pV@NFZsU{%w5P<TUfs-Xa5xa+10 zDDCVQ^-zO?Um&N}8;sNa65kEUCQ_CF#S-Vn^JDu%A2H01%W$`;e|(F16M|4cf<$lk zW$pH5UF!zDx3p~s4LmQC|AuyJu)Zy@35eoOHzZx}-FAsN;XC7fI$`jL)Ik#Y~XlsQ`^_IT&{^3MrQSBgtQr;&# zExxtr@`&Qz{l2&t|IhCfPv6G&pA(4f(#aWmL*R;`k*73(6GK{Ym>*H$pHF=%lsu0v zMTV067+bWM$r~E1y`Zj1{-zC839OG37wT(G&0d;o?U1*r`=}b52P#z` z&X4}bg}HS=bQKs(1DWrEn(kfW!fFq;DDH55ODF;~_Q&mo&e6KT{t{p;NbJu0*B|p( z?=7oupAkG7JxOEBS8Tax+sR=kU2S2J2X59FcW_K+*8b~_%#Vl@%?FX{y@)QWejWku zcbgc*T;w$e!H6G1Cm88no>>ZF#U4>yPuK_J)l~(QiL+#QwDY1X!Pu~PyHbN3!>AGt z3VmhN;)?5#2)9EJh+@f~HS}jmg6D_Dv?bQ0&*4Wl^ZCCi9e{)~Q|Vs4DO#+Y%vt4s zNZwYC7Qve1f(gi%Kx~L6qJ#N%vF3&Il?=|Pq8RQ@ShQcyY$TM`hB}-wWIUb+v3MWl zj65q0cPMH|%KD_poD{Tn@+Dm@nMgk)m9g>`)lEPtX*Y;0f`w#?G6MUEIOD9aWP~{b#cyTFiIqp{(U^*WSZ+j)yp6Yxn$a?; z$yiBwMabv8!0pv|TIz^s3%&lGGB<=%Y`FHl_8yHVOdmlAGluin$L_mo-;d+D?ry>( zfE>p1Haq}uCZ^DbA!{H{J=djfGt;!(A6J82ys}sj`szd2&E|JUIL@%&tq%u_Ye!jU z=9JoMK;7`s)sC5Jelsz^Q3~%(g{YfBR!gYtK1Ea@*t;-U=%aBdjisR#XgpjonUFF$h zLW=*ADMvt)N;1cH+wscxu8{wTIDy{l@^PGP)q=U;YD9ik5nA&jD)*M#|MF!94!>x+ zwF69c6I?{>B`t7tXJ(k@FJ|w^tMkSKt-72W2d4sCr~D5WLN?<^H`>~V{* zBb{7T{9zX0oM7+wRsuEZYJ9&%*|Hhjb9KfNC3hjSf2dI7GkziL}sF$5b4KXxF2 z21JfP5qTX>OVZPP#Q6aiO@H?LTE>1OJWHf{m|#?`nKtQwAYo4A2{gSUW4gFJeNdNs zSorSC6vg|-a2Y#h^f^{1lD23Tw~b4a1Ui-l;qT^sALlO;CEBV-8FZxddugI^aPa65 znB+uk7NHgkcfLt3d%Yvf`dSRR*jJ!%Uh+Q`ePFru#x;#j>);yR3j67~Y9BvC4@SY_Pdw%kFrK(Qmsx^@5d@l<&g^|1(aWPL|#f(jY>hhIEg1{v(tOE}) z=QSzlv%uXVr$quCpiKF}rtb&)fJlVeFEc-Ie`9Yf>3{wC)W!SpjRf#RBSAw?XbJ40 zZEgo$`Vz~URa|T>AR2*K{qG&AOwK@F1HN^|aPToLUnGoiMj5$=4eZ%(xj3p~2wsxX zUk3e2{2FH~tX5g>fkGKEA?u@wIJ;T{< z=%{MMqSN`-Z}IXOwG6@AcESfV|I6J=JO2VOHMZ{0_NM^T9Pj2#7W8~$pKnluKjZip zhrs=k_kJ_ZQ04=3_*uTbdO=z)u*xXh35WUJo)W?l?TV_bG zxku9dz9S+wo99jf2N%M3H*eC60DS&wzqk2a-)4Zt|621J21m$Y{P58<&0*4~ju`oo zf%G;{0aEF|0oEEX%Hx_Q@%H$$TW`wJ2dn+e(Cad;;Tzm)63&pewtY|E`||Lx|1~Nc z&uNUdt~*HTA_P^QF*sA;vU|S=woewDtW0K%p!I+}x-uY_SW%K32K*}KL%PHQ8cjJ; zIMAEjK)Qzo45mW#SATAzLB#`IpfHURmBZ|jOoe-xAaN=cT06@fZ7K~CRWVj5hz7BH z(yzb&7Qal>z10Z(w+Ku}GTbG_zn*T_b`aGE0|rw*OiK!B(D>G^hNR7`8YT$*xm(&G z@()wZ!HC<7dMOqTRM6x3ijIAYpvh<)yod=RE{i!Fe~9M+@P3BLRAYDcg_ygP*N2N5~R^!bbG6#8od6k-ST7=nKfph zaMX8Wy2#YK)kywzXRR_lxl9+-zuNwXykV~f<=^d1b)weV5Gx3fj`4F=?kZO7{wrM& z@CAp@=gVU=0>H63t!aPY+Uxmxdk}GD4F1u5GpNKUYv%v9n13_O$nN*?a$*%JVDGn+ z8P-s<#5g<(`eAmu&wusHz&ER38WCTM3b2qb-0tOTc&bpxu-9s?__nCt-NWznKdZiE zPF6~U)F5MXa@C>t4ux&T659WHpPP&_HB_uAPIvRt0Q;ho z>+&#zZ-VqN`MsVZzyq#%j0ADHJ8t%~5tjYTF2pyfD1Vb~w*r@a{49k#Y1#HRpuTRr z*D82AUA316{FaDK|5314#TwWahEcApPq6h9b5FH1frc*Up=>a85xlTRtN6>)D&s7l zo@#L>x?H>be#%=VY9W71H7j>E`l0OqRlV+G{*Lnyc54$kcr_n}UF3p}j3-k%M{9Nz zXGQ!N*?3(cWkJYG@_PSR3S=oAUJohvDv^QizHKuSDPwCF*=l0%*hV^yUrYc6vFB&C zk?d3b%-pcAe~_?7MQH~_4n#l8T%L5Z8x}lJ7E?a^D!}O$ZZC-<=)%!{N|`Q81pU2V z>n}-qJT9qN$mcJ_9BK62-VaWfv>VYsEmN$6jq>@CC~^1gjw2V^T%Q}?4-u*T7GJa#m9^LcMWsjO+G*x;E1OqM~uTI>R-XC!& zkWc?{a|i?WzfYvh6Lt-<2#_XQ&?0dP(C}U2gKc3kYj}zO-&=S@Fy%3O6?Yyi(Hs9j z%jWh74yjT(v?V8gP=isxXY!D*0S03-+5`>3FG0adXV^bC-iP~y15x)?>uV`)Ch9@d zq<{76#flA5qEIcBP@{1w0fC=egG|mg%XWUyqCw{n*me+mml?$jp-99N=+RfOkFZ7x){IJs4v&4)&=(t(MV%8&0j8E$lw03uw!|E52$ibsEfSP|vkkxN$Lv}Or zc6@1_2TV!+^PK9o+qOJ~MMAH6u&btmlv`O4oDmD-L<^F@7LASDV9ISdXjw*(vXF~) z#-2q+2$gMCc2{d3#aZv`p4FzgqNQ;Aw_HC^+{Y*g1cHX!hWY3>Y7iu>-gW3j9KA1A z+x~v!EVYzQQ)H}8=tqSj7FS3FoYr4w`E)+MTlQ(oh_)V{Sind6pL6{2(m>2maM)cI zQC~4QZ5F>L@6PVQ%(b5a!qUT$4WABK^Vg1DOZWq{R(R=3FqCQ4361jonD3drOHO8W zCoh)En5gevp{5!;&+@q~4{`+!ciN$;A7|nADab@iiG1S%#BY6T$7H5&=wYt`TlesA zv|vP<72~b6jru>avaL!p4Z}PyA}RACcKAMShK`!njHAWky^fA!io$K*dcJ}smSl1z zz)m#48v)=j#J<*8Fd7yzo3kfld|p z_%!H#rHOZ@gJ4yrwJzbrRHP6r++LD;~Kv^}O0 z%n+y6IewFZAXOi!c}DaDpg#O*)!jo$bhwwIr1Z{^Gi-mMi_>+v&-5}8AO{#e8YlQ0 zP!znCF{JyQdm30KqEp8Iq3y$z0}5)%OEW(41ycn)^fF(X#+hfk_WnKod;K|je)Vr% zq$;xHtNO01t+}JDdp*A4f^r>yV~@3-RaRZ~zF*Ok;f^8jJ;eM9N z=G6*(LL%TX-S?BTCwx6L2HB~&sREjzZ3rWu>e_PHUs@n7GMPP}o=aB+=nQ>)cM7DM z;^v{Fd_{q^r;g+_2>8!}{p0i4rys-hah{`2!A39!o)TB>dq3K-DfqcY1FW381$w@W z(${2E-ks|*zL zd3ny4;|%=G$LYbNn6-XmYHv;KL0Z>Jcq->VNjtt!p)bqzQYB9Q-BkOPJ0 zueTnYt7=-kE*eJjyv{0C^}TJcd!7lHO~)V!eGIX6?Ry@7$KtMLGgoy-1VH`Aa)5f$ z!t;BKy)6RY^B~a<$hm3_L&v6Jn*Gm#^SlZ&67YX2f%@VF9M@Py<(zbh*}wC=uR196 zeZHwo6R*4PXEK;f93C2#Roz4d!Tw{xK{+`LW10l`5Pq?l{sQ`s*7z3a$q=;Xmu|qs z%xiI^pI^?xBPxcZMl zen!g&{byAKIZ!>Whx+7?3>7I+&%v#}$>ApJdY(4EJ=l}TDfd$>vpLQ)nMoDYDy82m zS=gfQ*`CQ*RN5o-v`u zq8t(U(nRYAH%%GfKHmRh=!}qC0G7VpkPx)=!Y>Bsaki?icLI&IiF=BYr7D@TYAU+> zF;c;2&t(2I9O^>=*gQKN4qKS~9zc-#0w{{-f9tvSMO1Rz_I*~<&}(5~9%oD&&YYS4 z-r=ABvG2wEuu z{f|3A>?_-*aXd}OF4IINdT{m2)chks1mTxl?$6qee>@Oyc%1d@F+P>z1?67_R+xG& zO1AVXnkKRM?wbv%!V*cS@|?Tll=-G^J6+qKSKnPgH#^FQMzUap=J0luu&ljIK35tw z42CUN=u~0x#OrC~IL|wd=J47d;7#_Md6G^f-1YlY`b(h1=Wd*mC9lr5QWLpx00P&h zWm(34j0T`M~W5BYhj z-+yCz|CW`Dh}0JjS0eRteR2Ul?(KVDj}*VYsV%(#pU?JYP!ylgo^FX)hXqprWr<0XfHlah$=jfJ$wwi}Ua4pBI*mWB}v4L-ReYnWSl%M}EBiGbI>l zv_EfoYw3h}uhkm~yv@gJSNPCD!w*<1xmO^%z4d)w+D{=cHCyDc*==%Wm8steKRQiN zmm%qxZ)aE9e7gp<{9X@rqzXZ=txr`@OeuoC=Y$nYpPB6HJaY%JQ1p=?Cp_PW`R$J* z4u+lw<-EIOpc0$a0&Oz(uX=z)b<$u( zdj!h)+wsKACl|ozqxAfK-KWsGkYBjz#ppx)DKz5`c;BYC)x1=UY_&suYOun&CQTY_ROH(`F*H{3X0ZEO#uXcTzh4xDzcF zZuodcSQoADd5Dfy5s4?|pB1pOUx#)nOnRebksE46r;Os)C>Ro`Nz1i>RE_S%cW0;c zX#~IiJ^D}8nbP{R))4s~kZ4tO@VKS?y+k3kWt&gRb4DVz-&Wcz>m^s;1!e^Ryq~t$ zpd6G*VTRL==j&}$E!tIBBuw-}>QOzsD2T~ef7mk&KPC;ng4ANlBE2UIJ*@pZ->jh2 zVauX@{2r?e=CPQ@2HDN`>Fmq2$H6HlOyfRlXwG4B=i?ecVCoN&WbE2^!=mXsscO*n zSuYAz)w&fR-~RkB_TI9ou4q{s4FnItJxFkOcY?b+K^E>1+$FdKcW2=a!2$$#cY?cH zaA)OC&Re%?e>?jJoNuWVP-D*7V|4c;JvJb2reV#kL2);6){NG$sv1-4q5-cVxzDQy z+PjL+H%})uz?Wa*&^JXnIkK*(!m;bf4_yX!^pgC*gl2|V&sX2uc^Gqih~dAdKbrFA zLfvzEzwVNK3Qtf=ntn!yzlIRL^lY;Ys2rkxSoqxUWwBhP*(BEn;iy|~t-;cuyk7d6 z+#^3_RgxWwblG}qt2+iX<`Myd4^PV^kUDE@{PW+}it@NOc2ghsk(SU%koJqlQhwIj z1S3`b9H9dZf$d{k4x^AZc3@_151N4(22LXjnBrg5@@YnG$48vgX03P2HU$%^u?KR{ zE}Bv>(Ea~)BB92BMU9m!q~F|UEUj_67Uh2oVtVNe8sC#sKF<%R$b{wW8ExUJwd+aI8l|zU0*9PV?g) z?`duCFX`Sv{UErH+o!ZRzKiP9z=1v=2^uqtE6_v0njvB9&&w8`B;r48UUSofGp}R> zCmZkS(ucCfYR1@;PRS-;4GKdpx57xMN{Y1mTSL-zqY)f~nf=EwT7~Zzb!l)YwtGQD z{HL`++}gyUUiUe0nIKuZQW4NNTgKT=}$R3-{i+g0dJ^4u)bxJO`Hu@1nXPWhOJ0~P8W@4QR}7U zIm>a^K46iveh9eU|#tLgdT^LjPG~=x@>V29d>F`NHG92KUL43_a<29%fOL6H9hQy z0>6~~S^E?Wy)g245CbBQovIx*oE}Ycl;HDey|@bh?amL)mSs()uN;}|pICdQt!MJs z0z(E&+CTJShQ5+jrkbun`Fb^RK#r9Bx5=b>YT>>#rNn6*olnu3#}}{zh^o_mK4Ig? z0R30AJ)2TF+*kzOt;jP=ZBG?s0)3nVuqaPY14}gvRRh1UN{OC*OaBCV(QM~}u8kF# z)KsIoEe>KzrGPq&A{LdRVgYC0lgLEyWJ2)WzSh+Hc3!KQHGeP^KMg7JI0^gLq418F zW*2&z9&l#l`MyIbcad_|K;lwwtBg@1eir;ND~mfuo&@2EMB)=oOp6+f*j1;xaXi+b z71%zxyRdW3%aX@Y{s5Go8o6h%SyaFCVGmK&E99(ewrwJA2Sx!cU)v6#+f4ttIzDyDtwE(@&`N; zFZX4!aX=Qc^Dx=P|00Nl63FMk)H=p%IOf-)J1{zk&P ztnj&0_KU@fq!>=CF{#Ox1iyF!q#?)}e!GydHbV5dnM*>_W+A)ESG^h)JBl$2xnR#g zsTiAU*^>Oeu;&qS-#Ix6l>AYY3R`D6+^degz{Y(EVfN{MKRRs>o&F?dx2aWcglE%? zjMI5j!k#s%0IMNG@|{&LoanVMK6a1nE!8a^yXNa$e^J6_^#kUevd+|f z`{*92mzs=Nv>Y*{X}a(~x%|Pt81l=0uKHM{h9#PmB??Q9BO1u!%J3PAt?Q+PMaGD< z)5bDp7*x@@R(uUn5H1T@_GGOEA0W<1cbS}4Ce&f7tE_vzx)@;P;0EN1RrLY{_^4xP)p$p^z9WeBOt7$KB;@aK}~ zXp@uLu4Eo4+3CFHK==JyCop{`8MPdy_%tijbQYD40;enfVmtP;S|SODaWn{3=lf9avMKyjcgzz-XUGuSF)wkIuC`%~JDidfPhi`%~bySCC>Juy+J z$BgZoe-eLG4@q~1Rt!pq(k#;@ABs1LzCE1=$aRNQcMoaj+6keC$n=A=<_*OM8q*Wu zMw3dJ(8^n*`{?#)&sk(Rg@O4i8;%qP@ofS+?(~;qtYAT*`wfT`Ua$0JSYYe5~IL-$`X6$*)7T;txv;y(3;RN*pWvJ0dH! zlXDvTd#n_!E{DoYX#3}30J0LM2fExcr7LeKrPfl%3sItos}a^tV*+aU&`VoYH%u%} z?4oo$|9m5w3!pEmUAC@R%D0x6tbfJAKWw{BZQ&(Qc)m*Q{Jo!H^AqL%uprhvZul?; zm;F61hGd``!mbwWS5r$pWY&mp8_eh_9dy)Ks!y0Rhe+Imwgx_z{Zw?tv(kcNMgwKP zJ7B_sJexQCp53(f;sYsPR&V4m&8fdm3z{g=X^_NRKqe3vZb|j}(%zX-5(LJlAJMg` zP$8A&7Gai_2)jgDevpnS4%Y9#VG^*?l&SZKdS>G#+z^!eChafVy*Mn5^s)OeV|n@5 z^B_CbK=#PV8bOn921Pt=He^2w8OMRhd5qO19~f}M5iHGt!+?}!k>fdntt3_p9QRaB zUQa<2Y40k-OM=kdeyr|q*|^A}YdonT4Y)}q@X)!hA#e_wT|19nA7l&qfPrRCTv-q= zIwgGGJGz|bB&4-J8Yy}T6}UEuIM+=Lw-p|{AVbDP*@$aW(wT}>627A2uzf`csr;AV z{c0lSDMDBDEbl>9P*9m%ZWKzrq|rX|QCGnuLu$hh%HRDT7dnV&kT)9iLhH<@R0KiC z4f8jP$DKDt%$%5Vo`03ujll>zCcTaL1EP~t@@>8#b+p%g93*|<0_;CP$k;SKJzgHhv>LxvVr@mmkeb{0LpT{lQj4GK-;Yw_&o0y(ym9>o4(C&r3t zjmAB6n67*p_DEM@vlusET^99@?1>Kvbug2poTNgioysw2Y#-V)U}zkQ2p$$EnXh*Z zBI4o=)bkf-47zgECi7CXO&cY)fvwKm6SA{fEHs#Y;NM%EC0{8Ap(UyBKtlUVP-WOS6 z!D+csOX;s4-s1SPJ&SzUqxLo(ki=cBGKRT`BD~^POjjAMrwI&w5(_S7Od`+Zvh^R> z&vcjw>^j6ys{8Q`O*HeE>_}-E`poeGELID2AhD;iQqrgQ+O^A!jO8niEJdW+ZE~c- zkWhU~7of{Qsi&1@chU}~!LZ@yS(wj^ehJR-C@0m%oZ@?t?SQZ-jlY84z^k}O*UQSL zZmxk#Wi10w%61z8-}53OV>$ojFO5k&m{NN;BC;~oN75WHHU#=@hK0tUH9CDubJr=% zr!K;ZF4wf=jBp?x)~u(BE}!xD(|$61aH$3q4)M`$m?kigh6JvEgyxC8)jfiwV|Gpm zKlEYbKG64S50%f_n|re&D$~ca2%1o3R2WrA;dxyUvN;2I6llg0f%1270mulJCjK4C z%TYDF^!qlQ1k@$bAV3Fpam5jv4se6-`b z#%GWIcCPCbhoNpz^k4d#a3 z26$df!E-joCGZYCQ`!?iwkDnMHQO}jR-U;d`qqao{IZ35gq2K_7HUM1S$Uh`9x0V_7tD8$iKh^0~LS5E9O zPTN#i2R%K?{jyEXL=;wWDU*wIL2TQ!hUOrfWI>%Xmihg5iQdq!Q{euC&6s5!SOOk@ z>lmOY*dV0Uj6Ga?$z5hmnLIF2!hfvo9K?}jXY|+%gui=esiq=ny9IQ&KaK0eXve({ za`Fo_dwWWpnj_v}!+3vWwZ|03hDpt1u}Vu^*h@9^TQW;U510{O;!&odcBGHRyZu(3 z?Qmxtiy+7@aMFK7dIB5g&yRpLPm3UQkclZ5VZ@LAJJ%&<7LFi^Q9dDsndQ|5Sqv^kHQm4ZQHaQtb` z45rpS$MLom{*jO={>ISvA1~u$W%@0x%ZgH30nqZ1_SMQpMuYu|D%9xfxZMuXgs$a= zq{heY@-miqb#(?>F&4K^D_)=N3wLa>$+px{2W*fB55HU{ri82s1H6+2mjA3U=i5f4 zHk6+u5eJ8jv7t0^)@!Y&B&j_5Tt*q1O3+4%ztikX;qj$-Jt`zfO3<7r74ZKaLtd;9 zfB`=Zw1%q1L@~iSmiD;M0NgFs%AqZ<(AJ*cn-G^3YLsnZaEZ0FxtCHa3Z_z2d_!|W zzzV1gz1np2x-llPf(w<#Td?|Nf14Qz^~u`$W3af^JxtDXNmz|xw1ylBEmmgK`^M2W z%MF?%A7S-x$+m|+1OKv>_L|UqH%{CH-E`yr;Yi8u7*BJY zWuU(b%PXSbCx9^xnq!;H4YpXH!kN5_nE4q{s)_t~gB93EM`=dHL6nN2^;_28OO_I; zXw-#tJf(f6@6OEp`xO4b3&UTwQK|enXK0fn?uxW=Nzm<}HE>&RDvNvibqcX-ng|$D z%M>PjnvuS}S3^1CQk7nVQeRV&=F^;6=Qhx)FLiGUs>~CViTuwR>Qt(-H?slK5G5$-xR!FNPH#xq?vJ z)FdPDu^SNBty<2uO?4cWxJBg~BIp}4>jGu$hgOBUq7x-SD9d_VjCXrhV0aB?K#V2G zHh+I_Z>^on0PtIttb25{5N$~uw?J)OyLQWzTrbXfV=iFxpc_qgoYTz(8JB)o3%Y1y zwW?`*lYEQ!YM*7b*!5FA`>p%w(3s-Mkv#%%=nAESJcR8Lqk2=?>tC-(!7Gkv%p-JNX%s59!BUli7yVoz$Jc8qCzqUeZX@gyJiRk!)SB%dS{dG*tt$Cl zVV=BJ0xkb^WWDZ|-W}Nq z&sHSZq7hxnf6lu-=cYGl^8cR&@DI?an}li{H|L9m=>m3APsdg9PfRUUxDhWavU$qM z-|bBa+G(VaO&F94V~?r3q0;7rhIMp)56r5qF2DADHI0y^^EM)A>S|n3Ax09 z(WsNatuKX3_+_3HM(8Y z9PCuaUB-2sGOt6Tu&sLG`Y<=(Ttb>wa99;1`lN;m?e+u!lBp$MuL;l3EWaTXlllm5 z&fiZG0;AA$0HB`t;$7nW8}+4lSr>gVo9Y!@Kg@UxB(eEFiC!3NH%;k$t$q1Woa{dB zOF$U0eY^cK=|wPO{83FV;rQgZqQ-n^pq!2AuO-i@0&=Xt)esVW<)+HF;f8dRGDaQB zO-K=sDV|MqJn;py-{pr7QeT|JQP0$FmQy=!ia@=gAvuKG-DJMNt!y6Kh}RZX@Xh{p zc)1GL42Y~sWEReXp{#yC@p8|;fGB9~HzYi!C2(13yCR!_pt%DG!a8>nuS3#fh4GI? zakhqdAi4fiBF=1}t}mo4V`18_k}rC7xZ zFi0A-5@wr{Pj6J%?+`QSc_jE3S``_8RhnB{JU&LZYGz}WHF+g++k^{KvsbE2+wah6 zS?XWpY5f4b=;3yCqjT(x+dXSu1W@xAuCip(DT9u)c&&07qbTZ4vQUX~5UO%2r zfgItYoS3=JwI=ev{m=`7?-~j+PP?>)!hOBAsG}e86`WBcT?3hZZ9=0E?w||v3e(9~ zw2-FVP@8Ih<8BlO7?Dp%>4OECzCe|#Lgc&8o=Xa+skKt4QK@Kx-)_;u`+E6@6UG@! z!z~YStf;aR@>FSSZbtXy__|Dn*la@0@v{&~yQ|OfH#hRcq3lW^^xp$qcy7)7+F$lVS?g&Q z$)iFS8>((N5tuuy40ry_KcsFChiExO*`x` zdxb!e67eYN+z`aRn@J2;)$_-xQne1rZ@q93@FH*>EGU^)I#Plur1zWocZr%}4@Z

    RYi%6M zUEnJeZj>+otLB0r8B?j))r}x5#i~0;vJG_UPfwV4fuB&iS%T~U`pf)%Z_hWNQa0wd z7#V02x$)@boB$%!5K1?xHVzVpGm3_fxn#+wPMkrP(F8DD{7HjS2ImO5lJ+0B$Um=L z)lZdX^`i3Vh3kkr4T8}OaeI_nm!|79s=3aGKwwgBlq@>o9-n?8cnDeoghi9dAr{a7 z3;^TeF9@qQ-9=Z(_o1})%+D|EsXum;x%lo5Kb%~uP+)BKw%rNbzv>gnbca0#+w)K%y{9FuvPxXPz!)dxJrIB7&j6VOB5dA_06;|^je8?^bgY=82}qzCrGQ{A zXyN=Rq)uRH;$NhD-sOjIURP<}guO4WT)w)#p}8cF6<|9p+vx@AG?c9ycu|^V?Cgl( z0=9Iqs23K&|M^YP+o5Piq|Gz@KwojGIwyEi{o}(=_eZBmlO!gcL{ymTucKr!Q>t=c zdF*d8Ik_)37L3~tR-^$HDQ1HeH8kXR>i?Rg^DL89GYK~uv31lPK|^D1EXVQ(=an*P9~54>Dmw1UkOQp z>~5A16@bN9yjlp+OnaoV6PF1aOW)2oGT@BsjWNRtz{FZAJ-$VJ#redYPnTtjGcNiR zz-NX4fH(6lENI;;HV)$RN^b)un+@|!I{_!uE*UMM#8W*UV3XlC@p|DA zo+cEsjSWlS{fcYr^w$qDP}___H*y(_|9n%#a~1O=tjP%`ySMriv7;34&%6L~8%u(v z-wxZCOz^(icsbe{V&iy&ozDzQrRN2qS+$!qH1%&rK@>er4HS&xTfR?dirOQ zAFV08&Oo-BD|x&P2E~kr#PrH(36ceD&!DXnr+ZC6J+-d;llO6Z2=~sRt~Gr>G=|w1 zJ0{k?8dH==lshz8WR9+KyA$XZrE&HCqWVAd)@pzow|}x86CU&B<`us7HLWQx zR%=j(_mRDSntQ#Gt`m!^U$k7fHOmMn!CFqm6*6HW(fS`QOypkYeY7WpkpbX#~D>mwo3 zDCh`?Q{sB?4EyZH^KMG2JiL8(3BKPxk=eV$0aZIUsNawly49powLhK$c`)e= z&`-*I!7iP|M7ZxIKvvA_QO8-@eHNgGDwX@JioI~^JS6WNMRHP{58s{|a8q)hVuuQZ z@Cyq6DVl!L;(I5W;cCh}CYL`r`jjxK5OpwKt2n~P?$*LQI=bSU5-{wnps^Lgf}q(v zY=Sky0&EBcy546%4#?PbGE_l$(NlzA+iiZpFu3~d+|eahN~%zqY)v*`PZX#o5uTNJ zk%RA|v;VBFPgkYKuL^$YLP-Q&3q1kIBn5d)4w|@?8zt0{k`|7hS+E~a8MNg8InO%C z39U031nzN2>(ZNcpx# zN?1zPQBpu*$xDbG_1{Pp`w@{#fB9f2{qLevK$R-{_Lph#up|DPzdZjY$GpWz@cD1F zn?-1VViGGlQvAPIN8T}@3a#)$lo9&h6t{pRH&7J;SO$`$1lfO~H&u3pW{cKn52ncd zZyt~qSOz_80U7%Ls{DmQFXe;6A1F5vP55tKbRHJ4jO6bs@&BtP8TbM9Y(S~2!Av?# z{ogzw60i)}?o#RhRbmYMK)IqwXkB?~+%D#S@qjcTz%q)A)G7X(udP81#L>K*^i{-v zAs9{)1(u=8LWl9+eC={fAdZeHDIWiu2mHTq{*N#h`M+@fk1+qgaQ@GI{C}tOf25`V zJDvYCE&bo={2yrvC^nGkC~+hbWQ+j!1mG{(na)G~$JRSk?+UF}ET7I{g>iZFcfBPAGsgVJ*VV z>cye+_3ll{qqGNznjUt;{EzUz1fmMRTmVFQj@F@zf$O{m`=o&L(+QFe5p=4)+Y+Eg zu(-+sbk(B__42N!i4;ViKf_9g`2ef~`(p9HPVzi(=Q3eLjWEzn>N&JG!sf7mdhFZI zQXatHi_r39TK7`)@ZOss;PE=`0y4isnK&$aBsHW!>$~tn)Hmw@0C6ID5(@b|kU%cr zg?^}B>3xbgZq}u`LJzF|eDQKU39PR7#zOn_*LEcR1Ym>ECv6JG3Y3QiUX?#!4LJmi zGLu_sx2<9XcCJd;l52tvBoZV^uG%ZAX7w3Ocz+s>SrZ0S}UXNrn-n6^Z znz-UWz|{d-{fP?zm=^LT3gTd`*6e`d*tT{Uy(B}VID|OT1%Q|K1KvDF4lRXw`Um`h z9`!zfLY2tpHTR9zeeTX-YgFS*f5;V1!WjyasS#t|JmnKDM8V?-=>0cC%crFdQe3weuZy;8Jxw?C<(_^7_)Ad?kmU5OkViyG8~i&Hu)&d8v4HDY{08CS;yZ0QEh< zxy-8l20kax;nGgSP7Yvx8kgapKVoxIT>_Bl{^89m-8ab1u4ONU$*~0`CHc)n3Mfs- z08Iqy-NURmN=e+z>~UJx5gDzi16dbU8bD3~49B_0%O&-?=K+z>ypo?&{Jf@~*}R5U zx=l5&x0+%_Toob`-XF4&nq8aTdrcDCcYgCl*lG^rR%X|?`*Z(< z_%1sRs0D9dc1zHmEdPP(QL}8~N#z(=UGP_vvGu-3bVY_f3=HoG1`6kyha?H1!y^DZ~zEuk^^NpQZZ+tC8~+&Wc%h6=;C5Vu0Nb7TezDHP zTLJExUA2sg*-%{&8`uwlk!Xhnp>_AvN^OTZ%{g9ns@lkDgAad!=VszK*|bZ|wOY)B z0KL5)g(Cn&J6$8iJNAlW%|m}+=%ULRsvq>9Dd-8XFkWoYw!?@xAARF$+G6Z;Jol}< zSX~EU>@ndb^vs|Ed*&;!he8C$Li%0-QO4EAc>zfJbPAC6H+hskwZD7b4 z$?!BQG!eg0oe8hCp{QEUFS8jzT*c&po5?635_>+PhPIlI$>Kw=|O4KItAdIgx3g!v=>*|*+i zH!h@sUI=`@WvVE0FVbo(bpO;%{gL0^ws(jOg;QFmf{IPXYo5-RXj!CMrkJQj%YY&I zws()w52Hd^D~%6U0qPcGboYCA^t-ldf%2**);q;DkDVXU$O@r=g~Rj*=Rz8A4zWvk zy<83gd?g=OXgz^1peoP>6x}WLTix&^rZ(t1!)B@aO%XO@>i|2jV&C>7$~rtUePPKW zH~WK%&~2F#DWYV?n?I@p9TbACc_Aoot8C<2@{0V*-p@4&ymnNT7U?nnHw6X7Agdan zVx`D|5kVdu1^z>8CFl{b9>$bT3OzH(Aq*d_I#0F(2E)3Gl7#`B$Zn!j=FS&3p;N{B z;Flvn%+5Os0W4#|#D97cFMn_?(*&bB=6HFWPna>|lFD$bp6o+85r%)!#o`8`L>3SiYM0@8vf`AmpDq~BBCEW2&fxZ0;KmzAn z)woa&tHGy{+Tx2+X-u8-@alD0&@N#1X-Vi=z7)6E_1{G%CD)Awa0(UuYR^DQ{LIo= zk-j4JgP>nkK?px-NeF6!^s8-Mf6qn4xfeD%!S)8Oib{)0lvh@a(PJ_y{+#-TJuJ-_T^AB+Y%Lj%~ob2?LcT2EX~$dEBhK2hJEk zI-@%@Q}$GaugS1sVZU%nREg@V@MFg3!phM@%Xv#tiZluwWI4a7E!D--6T)(QHcx{C zNOKkjLTreJKER(&t)sI;XsBnHS`WdEZIlH2QCFfxGBLrT$uTe@*W{N|^O^Roh?HNX z9jltxy;SC{w`L_i?)=rcAOC9*qEVTHtUXli^PIXkASCFL?ghGfef>m=cPo%9Q;1<3 zE;UZ0Gm_*LVC3j`GcC1*qNsT^Z8_UW1K%1(Gnqq z__*B%jr9HW^O%Lub^87lw#<5Qt{QQ>`HUfR<)_C&cj~ooTuP%qf2f>)^!`)Y?w%&o zYOphUhoZ}*Yf9%>j&+PP_qqOq6fbyPT|7qt4z^)}X3S}n(PCVOwEr`)HT$uKS6ZdX z&U=bmz`u&gapSTUFYJ%JZM@5)+A)>X;Uxl!dzIs<669p8 zf*hxwY-_vWke?7H)vQ0Aum#3ovVq-FSdw)8R;F2O@>!6JElG5kBCO-T#O!UBz>9>? z{5l*u%p$vEZf&6>qge2~e?{%^e@u4RFO0G~?R1cTT%{s+?kNmFMyFH7YT?n% z0QO=DT>V|vdguTzL~+n$O3A`_0bAb>X>}dsv-BDS{vQI?i_Ew078vu14%}7%$9uwL z7OH2W0#}_17gCp(;in)q?u1uMQSu@zy2Z`JXd;6DpnwPEVC%&MR+|uSb17=E~Vnlwg_s-!eYF$M>EA^|>Fp#s?SQMSv zpTPVfPys*zRms2>rc_k7BB{aKKj8!JzXQgLU_?x}6+3Q!baG6Y zIeTb+V077$5Joi5{i?Ilz#=4)A1S{$FaC#&p8pyVF>1qWwUvKlFYi~t-b&bmCggMv z7J{I{}8 z8_UW8O8RI6rr#GPvtl6L&^_Y*2+7Og2N%brLBp4AT-&Dn1EP+8m6SB?Rd}oER=~Hu z;_Em?e$z~baFg7!!%Yag6ooAPW=07aEK8Q-#H#VNXE4B|@}Z=JBq|93)-_gIQuf~9 z@)&5ArW+$4arJ1(ED;ur*`P?juj9M)C%UIDKmr4jjKCSQ$+@WVarcu5zVcuc7m(h= zM&EHHfR4(t&h;#P3HBhcw#8bc7p%!6)7V{mY0a()lDPm*oMq#6pjxfH*9NPLzh%5{ znAG0dYhSa$>mVZHk*^4@+W20LBDSF2eBtyeJ3_`=7Ca$ZozL3R#0YQB<9_QihERHx z=PSuo)!`TJO51|ddcSL+LaFqnM@mVnL6gH1i+MM9FeV&&Jxw)++xZ~z(z{9-shgQf zeB-$tBvt3li4`!5)lAfQKLj=Vj41K%a5^NHWkdH@OxX9glw@l)df)lacR@0ELpKR0 z`$s_x-$;5`QN$hFpNk5^*Aauu5tbt5xb71&J}V9pV0>73^H&_u;(CbSytF9QktI{Q zVC>7a|1zQbuWA}363y5URV7Bmy~iOE5+uR&%TS#bNI_>YgmWK=k5k~Lbq zVcB)k%hIgilC84E4RTaBzVHG36b$v$1)eY(YA2HMQSAoT1H0f5~5upA&a~n*f3{(SL_k(!7XA zMe@jxYWTMZ9(OYIMY7b|cpBz^IFRH1sYv0^DZL$1r38Tkqq!AS-=eM@YqTRZqSc9C zC*fi2SzDI>=+yS={+6%63#{X16#}irbv`8KzEJ#(ybRA#SZo1B#DzH==6w^~Pcvbw zGX8DDKUoVE+^*4T&*cTGN!u_R8551vsLU6Dxo9C>1!XbVJgpuGrCMQ=n<`f0tGBuS znAdCAV}f)3-k7+5mqg6I#*dnQgdj1FyqV(&r&6pD&BEK`jr@^lRdV2vQaBB9tdn%n zk(FSjNdWF%>OC+*#=lhoQi=`lM&%+jODEZo{$Uu9Y~5SRhQy7bN{ZdfmuXu^UTjJ2 zdhS+qQ=pAwU?3JCpiXA&0fT1ve@3EHd5!9z=M0|%?(mw=i~|x+PgE8y>WmX8FW!LY znzH!nI@fwwJWaLvPZWtrWK#T5fj^Jjx0EB&U*?KNtQSR}7ArHsHkn9!M#H6tCk7X+-e-(kiDtiLTj{U##-U zRJT8zdif&z?GQ*)7KB>F9~;R8o&uWXbgSyxk{7LKd)pi#q{ zP#?Qdy0Kj>)e#6G*rI2w$VIW#0F2vWoIbfEFuM{%lW=H3GX~56u-X}6k;nT?NLhsS z%~?Y{%`5hoL+jodRt4TqeeYL2y95^N=*TsZEl^*?Ow+^C4W7ll;>*#v&{7r4?W-ztNEk_utS7 z2B$Awgg|9LO0SFN|7S|iA#$LQ1zy|zW5ki^2Ux}E6t{kUP(vd!5q?ovy*DA=5SwY# zkFyoL_X6gcuB@;xCBm2&E)bLl6nvvi*Sx1$!@qCo|JgJp%jj z>}R7A#TyR;++$&S^1f0rVPywKHE`Uv-W~)hDP-GX)xB_>UfGHm6%1>MaSZv}>hufT z*}z;PdDgH`-vsd1h#)M_fhbD*CE9B{Wk%o&PwSin+EWsQotk>Munq*6v_JckucW9P zrZIQL9s#=WfP$R%+Cpamu(aAp7A_8l2d_ylMD5YC)>|XapB#R7A4{WZo^19z`Xz1!qEe(S zH8?Y#GA8X-x1EEP1+HkgY8OpuAb{4sE5LK!9s7dUIVPoDm1$l&n(Gy2)b-C=93Joe z4?&0r@m-%skC7!xZtZjJCi*gqdyfDZY}6eX@x^K8d2*i;9SYe1^-B^d+tVBd1K%;4pA=!HLqX{F* znrFP6Xy~0oK$u1rG7kud2LtbsDwUQIVT;!5%2<8Bo3O{cGBqRfZ7cEY+ zF2ZqbMQuc4n+q-BjtLoiJ^JOPMOSw=G?_6-$oIfi2US~3ZUJv0i;ihQuOUvO8n%Wi zaHH}eA(z8@G{n1HC&@&=g%OG8Bh&L|;MZ*J+_SIikn`C+R864xJg3mwIf!XzJA{}C}h)(JFz+P{7 z3{{%J^AUU(Zj1GfKc!}iTrqkUL`_iy=UgWy)n1;%60zZcc_9WR%KiKhh@a!#_e%bt zOtbhLI7|4CB_b)39y#|flP_HXC^1#L$RbjI-v zT0{ofvf+zI=o#%dZFZ4ZNYxhfo78^cPqZ{o1IcDwM^%b5AkR8($rEBC9fsP(yN~GM zE=u&EBXucV8TXJA`-6r;&!rD^6wgBitBtmB61`YGq(kuu<;=QS*e~ifi#I#2v;PMF zRTa@zSapl4qW2+aokl{+Mzlm(tcwl*f-nLnCXmfoox#)TMIS;j?yr2*$)d~4%M1f36b^d zr>6%lDpA!N!c-W^4k~0`h6)ZbTC?$^yhCi~UpxYebQ<-h*JS(fLeBn4=h8LdJj&?a z@9VvBcvo-)v6EVqVQ2advJ9Hr)Oj8nbS%!EZOZhNolxVd*XK|b)UR@sonsvJ(&knA z#Hug|6lUw=Xy3Q_r5_FXsL!!4JBM~XC{NikM!Mk^?N(SG`_Oi*WOBt%z(IFKA|*OR zEDq80GA02wyk%z+Q61dy;vRQISl;P_LwxB4M~_?k0<+3hPcWR2aLp(x%thYgKu|BYJ@IXj>@>#yx7P?iwLi< z61kz02%N)JvY0g5h!Tl^nU08&;NJ?%qL>X?3koA$eEaj<`DnSM{9w8{lZi_F{5VDJ zM9pOl6cjXwe)}$4hZM?>!I*>5LWfUY6i1QhNHXIC)`;v?tg1$Kxu|-w4msAHQ+*v? ze!0TFaod$hyxJ7v?%_(wnL)k?GI{}=@zXRU{Jmrqs}FZpSTx}`P2|EhjZQN+$`az7 zdUmyaWHCY(Cm0u7+^n``$@vnwB{`7I1Kd|`@&r*ejMt}Zf6yH(@%1FPKn7l%S z7h)oT7WZ{!(0;RlqGJR^ppg7Ep?d~~J*4w)$2=9OFhhe5Pg$}0VTiff<^ttiSnhvL z@SQxYqF7ixoJ4SHC%OCWtKSMJJo@RKmP(Ob&c9DQf>l~>wZo}&g+yR$QE2Q>aw(XJ zd@dQ5tC_JetG0Upvp|wHN(QlIN^{?+cI8-++!fXoRx`a=h5bsI*DhM6#UW$oJ(%KY zW4HAUFk(Kpi7=V0UHYBW1%eMm>qlObk{yWdCl_h?IoUM35w)g ztG;AiXV>CsHI74+R&om^se5~srHVM$7B}zgL#4@(oe3!N5|3c z{_c$==#VIPi{1VvcC&7t10!_>R$d~rB=kS#n`ZCpX8)6mro zDPM$BN#Lsp5JUr@cIb|!sje|U&F}eO`<6mSyPjnO)E^&-L6PL`WMCiG@C7<-@)hNH zRSZgbX`v#>myMYdw^XR{T~X)?JgbBbU_@J3y?V&p%1&Rr_HbAY#}#uZ((KkWKuN#FSCI5>Iq3a3_)>^MxE z^l_Miz=i=FTGJvr?C`gzaLGju=TLtt?dRnvNyvDLw>jToV`!L4J;HL(G`C6Un*RbH zK{0V;{5o|Yg;s;^B(E*BXk!g_@t{wncE2tY?$fO{tV6zfA(zNU2fr zha%+|C2U~v0;-xLI>C~ydMq`eV`;Acm#~k~9)_k=iPF95D7F~zPIP+w1lDkIj+bdU zIO61WzbzZ^m>XT1oS16YoxJ{kq`hTOmGAq-OLym{kxuCb>5dIZcXyX`H%Nn&bSWSu z-QC?K-Cas^9=`wcdw*um+%vww8QAgMbzN(H7WAX+g4ak?8Y1~Gab$3sDSuU8b5C}W zeaR|R^(BLZ$_2B_U=OvAT)fj+{bHP0$NhH9d2g_lF80}{J^pk!6Ee5~zWB6Cs^_5Z z>ei#ojoD8J0*5Rv{A9W%Hk!Hzol7ILVR$+SrbRdTpZmj!URmkcep>`4l6sZhq)_f9 zU4LX;A%vr}kfOZ3BMQHT==$dc`45GpEMro*#Fu;s!-GxR9W-F!&?!o@tP(???>5>$qc#&@Qc0yk57zq;Tr5VY zv!#qcqj8*df^0q1b&CIRA5z3PT*Yybo{pzHR&iObjeq@~k?TlJu9wUI9t(l`RE$#N zXA)6QzSMm7WZop?!zKM4_aK2>#{6gc6D$MWg#V z#bkjn0s1WMWh3GFS^*h>`{<$K<88*^=a!I;6vsvi7k@lOs2vkGN5Mp=SF>KvHK!$3 zvt9^dIXnXav#sUiK9pAi%k0D?dZfrMJ1%7TO5>JF_`Jx(wuLd%FHm*+LPCfeLEm1H zJ{;v8L|CR$cp_`}N3!a0r{+q?8C95A>G%r*NJbeh;eM=KytgIffS-IbwNJEbrJxmZ zFGh2FPo-V@_{_4%bW^0t&R8D~M{xz&cIET}cOUy*y)5GSgs{0*?jDaPCZPADH$$Mr zm_U)soFB#eE$dzC}8pa=HV^509UTD@srTXj+vf+_`XbnXI=D{%iX9ok;f5}b)K_MQ1O85Z|QcYV6Mz z;9}C(nle?WJDKJmjDGDKRrrNFS>WDU+<%TYfo5KaD~y?1SoRUn-)xwwsI}`R3LmT| zSmK~??4t6l1)2wgFY+TuuS?CM`J)NmqRqNtdQ(QtN&?@k8YBIx{XR!dgS& z57*fX{AEAxW1f4@Qq|V=L`5%pk||la)SL2n_2=mFJ>GVqs>cMn405iwB^|R9a~@yo zNV74yk!&tybReVU*jA8!6Od*=H)cW% zKpb-DEMc6fAeafW*4uZsz%l$i=f#bI##EA@;a%wcl9a)G4;o|F}vph8fDVo)keiiEKSar5Qd&lzw-PUdiHEerQ(qGKP641 ztVEuSIgP80IaF2nlGtV3Jd1fS8EUDtZ26D9P8=Oem-4aUqT|UMf)jrIGWP*(ktdi< zmxPex+HlM(`tji^X9>C~u3zc7x{BtYKSdfX!tM{CC-{qG-d)*$KR1A3fiX%}jBSJC zgWLlf_}BYH&9>hv8GU02kY{9p9@5$5zggkdNr;Ew@gRgJg*J9iPovw)2VVvGK%iu= z)nj*Loqtrryb^6Dm`v{aZD+aL0#=c}<+JP{_1{4~@rVy1WLcXZp_Y9|ife(n^Z$@~ zsYq*;H#x`E?pXC@mj!YFFabcF@a;uwE2!6hM~Aj9fwELsOtZN(d3X)rqD%^D_Ys-^ zX6z2$sA;AuiQqHl8yh4r@*O!VBXdD!G(8HsX`F;gVWao%!721Xj&)aAU(q+v{mw< zydEQCB25{#o;c3I45@mnbcv@msuedwq0<{mW4eHRQMo4<{DJI09+w&^^{hE!=LM2$ z`s#g4#Wx=fQAEEI^5yZBgAzFxLB+QP0wp!ZxR@rgZa!qgJrU#Bg)+1A6aR3h&#JA; zGTxPyK5@JxbL;l*zg0uHaG0*1&IEm_F0^0mzmHgHeZ&Jy^*Nx@;CKIm$TzI}q4Swy z0y>7jVs)qnP76hTE-{mQpaC|I3ZUTsdB%HKWD6SpW*}N8JfQ z<65sA(_7-swGa0@@jSkag2To$eE(ws^d#=h6Vrm?W{$?^2oDZ5uLd(sIyKXB0fK1E z#gnmZVphr0m;Q0G31Y9IR{-dm*qlOif~mrl!iOSaH}=gdK4C`B^ea~r4He3cL%bm8 zx9ey_d@zJUX(;W=J5=Vi08G{z&qooI5oIcxCL~q_bbN`t^;o-N>n6{`*o-!fyPNO{ zOjvFA)p`?9*YhW^)47fuAogiZs2t!?unXp%nQ2lgefj8i3SvH98{G|g1}ebkE$IKF z6u_Y2u~_9hcvpYLHjJxa^oe*@`0?w9C7t4Frk(VS#YABkvwF=-k=vnI8_<+}#epw-;K+4T=>>VO3HkR)(X*o* z3A^oWEu%V*oZyAueKgh_5&YrXqraK&(n3Vk-0Bs>@R4oW14quY1!n=_$^VDxnU)MT zla5mYCG&<(x?dV~{v~HhcDsU~*6J)>iubpM9KI%YGE~A=^OAtdgU#K9`il}y8%d&# z6*Z)XDFvT5g?&ub>8NxnR?ej(fWaH>edDCF+iL3e17h@1a>;=k$(FtzB{H;EBq2iucG)j?}s zMiZ&VqGp4S(JF={vcB9Q{UiQ_80}3D`&pA=cIAapgiI6d6g&CxcGC` zFr!W}ZCd^R&9Fvn(A*h;mDuZ98kyGwiR1HArJ%r|9tSGZKMcv_WLz*1=)gLtz6@6K z?_KGhCa_a5E)O~gdthH}=qbF+<1S%(pahZ75UK@8O!jV!f5AvXRWGvb26a_SrHjMz zeGP5ROGgzIi<_Zvr&)Q1qSX+uYTnNr%et?Lw(~FGA)XLDC>@=3@(v5}Zt9Gl%NQ&Y zPI>rvvfNBNaL_?;^)8YFRM$L(dU0c7YQJJE=RIyezCch2-pze(Ty+re>Fa=uTi=0^{R^McdNkqz3)i3DS{czKh2#!{*&DNCt^LhK0 zwW|-bFKmCm3)@GEjN5INwwi#`yCyv`#@$#*!9X1%98YRo$=nH;u5D zy5R#-lmA@Lu9(D3ekBp05mvwk|L|IZGc%REyhb!ClK7!Tz1cq=^#c&@p(O5rQ@1Th zp_T}t8jp0whi9hUWU&}2Y)EIg66u`nT;|TyPWMC7RL``SQ_|fg4zdZLkutOz7ds|! zgp(IL7Qcp&Wo$OwxXjLHS&3(SW_XZEayrfqL%vJHT+-!)sj%C`?xm44?UVhfrabpd zN?V`SjBMU*$g%i zc;%nD<9%p@z9k@ifzBM|gF?a!pj}=8IC_-~N;vbBRgtEEA~=?);riubBZCF~Z(Sm^ z#RPitbxQbHt14a&dIH1HhWX#_^>?w!8H>GilUIuBX6#;N!%DQ=fcS#z??UA%{zdE- ztwXE@jR|sh0%acm1KpD&M_jpfd0p_NU4dy=wmgb~ zL#u2+JFD>Xrn<+4@F>)Su}cokf92ck-D3dhjthA>qKa8Vk>Cv|JwnB>&`BCbSvs?m zm8=6Y%q=iUzbn{k>)7QS@AW)Sd$<{DCrMmX=CFBLBMSGVT$h-dx{wV&5J& z_(q~B_ggj=<~$bBa|J`T-5$IXq?LKna>K({sPcs zHRZjXm7fWAIiGzCFts0@$lLR9Ro_LlE#JGOP468hWm}$m;qErvQ&sT)-0aU*DNgvt zFq?xW5z-zw*OUFV?2R7H#^z0a)<}m~m6P`w;nsnn!>^A3jAEiikZG&?%qoCt?{EMMiN zVdT?Gdrs_LMO@%Q?@&zpLI`4aeXrQHJsbq@Xg2WsH?3pDoAY1`)jXF$$)x~(6cVyp zck07m0$D2KD$>{20~)e2_Ko8~LRRQevM3tDfuUN-{=di@L0dn%HWoF+1Dz2xv6nyZ zPq2zyr!3R)U;R;$nwv5*4KfzP&vg(AuOSY?g!75N`NCG+Rx977Sw}<0Te8=8Ds`9b43stlDLA;PI(eSsrfmBkb4i*tSKR$bRC-tat0m1kX_J z({pluQlz0a^l1zgE~IjL9&4Jb4g&G_F^|zLG8C~^m(}xmd65&5c>d- zNH=Q%g(G&!=K>M0c@dy>g5Ox^A@<>(XzeKG6dnF-L7;XBT&~9=vUQPYCPF2D@!ldjM zPJyA7&DE@eGU9!3QY5$W6n5I{RPE&+Hx7Nf^6`<=q2sVIoat8FVw!Q?-nA1JQ8^14S?M)fuSImnOclHTJ_E}@# z;f1euRZcY6wr{OVoI;@NaZ6`vrkXqof?mm_cO$;2VIXpo($F2%rYDTmRp+9#qGctF zl|}_*;9Go2nIsBlV_ob8+o^D8g}j8!ZUKSK9O$i_f~>m6O*}mT+-?b&C@xz$5C%2r z;wPEnJM^y#!KNEOxq4aG?npG_zI`Z`9BQr5v*;KuHp#Zb4s(;J&S1;1q7gwiLK~{q znqQOXZ}VtEA$uDd!fYpfl@JSm$&YV7N?^keq-OH%8*_AvM{~h~phTVWwTnl``?SA^ z!Q{K{@|}fsbu;}5ZNsD$B3x>fG(!0Hz79g9Y#vOSUts5t)E$nw;7X|_^!)%Ap45Uv zEvoufKOp_tJvpR!w8iJ6i|M!Ym@O6y8-(uk^qa0(d!i+>)yOR9+>wVWI}0A-}lp4vd=ijw-k?- z5zej(SGf6!p<+@SRatlBpcTR`@IByJdE~ke%|SWo@!Y-=o@FvKdgK zZ>fu=oYVEz5Ja7yzZ;|+Of;bT<(`NZv90YBQLAEDBQB&6=?^uoeM=Ay>9WR}?-LRv zh!WCKQ!-K?)23^IT%7aAdx4Tbbx+>z@dT}IfriJU%2@X~u+nt4ikRqmYZ@vfVrKyU zZK2X6616zxk9+XF29j$|0EoLABhD|pXT=3X`CP0u6J>^zwDYI0Ks z^eBwfeCHtqvJ>5wiVjP~m(MOW8cQwd!YnhHE9dZpcESvO8**+=>nZx2V!cufV=NKO zdL~q1!TFBF!!j*Y(h;!}-rFZbQRXqPFjZ6Sm!wkLCmG$b!p=!2Z?+c-o5Hz2-`?tC z2gX><2+WC|nN68wi?PBa8WM_=FeHmq6Qu}E8O<4Vt;FTh?4r~f5HcuyP(oB{6==*@ z74L=lyvL6^8lU_F7B2qzbfU*Q3;%mJV&?d~Tj*TR+rv2;AM~W-vM^YU^7u3v$NL>K z+&jxo-xn?j#sTG=%^v{^L@j$h8io|3|K3iJy4EGKwd%rM)W>@bo-52LNC zZRe}?hX4zb!8;-Ep3ik(nT2u$r z_akrvi#ZyTYbw(w7GnkByAlGGdcIjIJ?*nHk_&E9$nvQPeds8RJ-CXDJ^Nhh6352_>PlEJ{GF`5ko%a~#u{mMsxTX25@Rcn1gVO)Qa0xJw3sRr zOGEu-HF)d!n5zn?QqvSrOFh0(Vg0C+dONfs$+)O@wPbFRG&rkLp}6cAkXp@bEEgMi zoeQZ(?&am#o#6>+KiA|ID_amc$nG~jd=BSob{USrIg&(Cc!+|2A0gq;!)yx|`opk# z#2FOck%eOE&&vCrKch9rr$2pl>oM?b4^N%+rKLX0XF^vTK30Juc#vgKL-bu`Wd2bu zkZWhCVu0^Zx!@P9fHgS z=skk1b}N<{D>?d02K0+X&=HS!%Nog#mcF3((cKr6e?PF)bu0|k67`*LJCZVbXuqFb zt=)RI1oqqOUeAr^{Ch;e(+^P^aKA_A44hj-4FaT<-i%cvds^NeV$Ei4AJ!)1W0Y(3 z>v{c}28Q9-1C>qcB}P=TjbM#XhRmDzhzV!n`p37<#8I++wPdeiK2Q_fKshSYBWGSTVkp z=Kql4Jl|HUhJ-iyeF$f@s!HWAM9A|?=pw2i>@>8<9ql#?wq)bb`g<>;__hczx|H`t zCWO?)Gf4cFJkCM%1xQuBKt62QH=^h+RevA8JbdJp7>_RF7(s8|>bRI`WDp6d?`3zv z`DTfo!Truc3`JndhAtY0l(Gw&9pK0O1jGx;{Q@IKUJN%+=BN>JwtW?bFVC~0jKTW( zf1hh#(7CG-1`9YuA@T5a2~^9~pc!-ifv8y%doc`CnP757od5Iu74abtvO?!H_T(^3 z25-WD1H*}2zZk??bm3o+uk@WiwU_bP2he?slJSH%_Yy6J*#qfZ_X84a-xy717bn+X zCMD_@Ri=~WT(cJH6~@ljCY23H@WZ6f8%M8hZ#2`Dw}V(m_-u9zV-FAZJy+6A-bLiV zJ-0`63^1*kEmUX=M!e^Dzh8_Q?^gsueB0^Q@z5?!MgSC_2OtFlvP@qP%7E(!VRIu8 zcjwf+4cJ-LVZYE_mF6}h`uNlT`&>7cqqY(|=aEXt8Ek^^(c_E|8gWwOK!orax*Z<- zrI{F%H#&=4 z`i^8vi{!B3v48;w51(Xe=X7goBl+L=`Tm{$e;lyzGwRW}C~~>~_nY2-+yA%m|Lbq= zSKa4)YU~PHuCrV%G!*3FIyB6-2?AH*?Gau9%YI2nY!DLww*xIkKr{(_RNFn zDGJ0O9b%aRBdfbFHumptFdh)(+Q)-(e<~8o{P%z1nfdb@KkEN))q4LImRQCy7MHbT z&(5O(AOLW&^aWc3_h(^X)&o7xj`Og zX%xr+&iCEdIz$oEhYwDmCQj$JUje+Ob`SuCd(yz?OdTrw6E&J!2K3kq;Q7Jpya!w! zxj>|fCxBFx?^rGPXqBibi-W_)y~nfP1y}|fHiMDlNQGM1!quv?h?{^u$PxQ*CI-L7 z)<5P^4p^*#$2MV_vP)=rap$_KD$z^+uRv{wqMK~BfTwK>RaW$P^%V)hRx%K#4DMy} z-CP7Xb7@a7A!{9dR#)-#-KOtpXvGx;tFX*4;g>=uviWWR-f2%L?AYa2FOaXuVKY}o zai5a=|4yi>Tw)p80sbYH>(H+twt|p(6yLe`lt5<(Z2=(n5M9p{x>GTv64qK;Xvy1c zv+)2pBV_S9Q#iP1)~>mX0**2-tY+v=XL|=xBCk{l(X1y6y8I^~qRbhvr9K0>S^Pj3 zjE$mt?E)yPh(|#}*H?Njyo3M$08T|t2<%;e!J0{ID(SZ@Wk7AL+qauy*cbF(^apG@ z-9VxyE(EJFtzzcyVVbNl0Y`7~51WbsFLs3xM^)kO;hBIQ{3_ivj%|%e^?tx+zEEGB zeQs`B7b+aOkC@p1ehbFTviUtO*Z25yu8v6*b#1Py(*LGLfD1X_K+7}>h`Lo}Jg`zc znx2__!J?~y5;F(*Y^H?Jj5vw8?b7GJ72J$G=5w?)co^>WP1mMl@0*TMNc{m3PeWrn`@M~2qX$6!T?cB!?}>^3 zNCDR%H8S(>giLGnYGy!H6q+1Q%%}R$uIOYt1w0&&Ku^c@M?YXA>jze}j*mjW{{i_K zMJn$P@+9w0EMS1H{1^_C%d0#QbnxnRck}kRlYKzu?L2`(Zph>ET)kx~&D_!N+ndJ3 zvh=4)h~OCJ(}`mDV(07Knfs>H#y zS1o$|yHeRp@t=MGKDHMi<&@?&7cfZ!9E%99UEoJTJd^JP55GUZI{$bq{w6BDyoiM- z4W0`dWU4vU+4-jJ^YtpLqckhUl_JzL?r63Yz;mSXU;^~Zvh#bKC*?r5=to zcCB;(v>c^DY${sJ{K_;vpxc)n{SO^pOTD=(xuy{?bQy0M(_J$Sq5EG#7h-(<2Yi~+ zRz=AhFSF)RN&mOsVWFX@4=%Z~j6F{9@eT;uI>ldYWS=~&Z3275pxjr2nfQ4=^8PR19(f@(o*1ex@Ouy+G z9gyG=eDWprJGLrtA1{LR((>rIJvI!f4XKrpWw~XpN~$JF5ufIXuKD-pU&j|<`oA~z z%)YK$6@Gn>pmkdOEs9mt1Vlo340i8W$0$ttdo1iFgEKvPanRq!b~w%-&$ynyjKX%V zoyI&dH_srw+8)eQXkIb*A;e*DWQ*kbaOR?)>xN2$xe(vBTz&*)s81Ql@NC?S*=l)A<|9cK#Eg+Cvf?IQ6)JH<{ zKhfRz>HgiwctUs?oSQ4(VEzfns7)9VH+g&jVt3o(9n&Y%uSVDil}fcGANhU$+D&x2;tQ|s-02f` z0KQw#{a&XIq#Qc=0@$D;)qn*>2vI1ZoW(%L)-&$CS+nTgW8g(CHbof2B3Tns8UtsY zP!fRQ01~Mh>41a~^No1VV}UW<*~fEtka&1!qX3yys<3w5`(>Y0i&GK9$*vRuXah?E zt1Dph4Z+2Lpl&w*kgCW<36Hc0Y_@>dftMUNoWo_4w8n?~7w8wDboLKSu(CA*842pd zU^TzDsrjXNB}F2y=AiVU36CeBvP_SD#!Qw_!cXbsUv@7 zpc|zT6DC1mZ}uTv46vAYBp8*jMr)~lh_BMra}Mhdn1Y!gLxwgC(@TYt)q=yqs(DX| zz9G5rjQDCF$AWDK$-PTui)XUYDCDnL18HH7>84xtn;sSlPSOPr@(<|f+h05bK#AO25&41Fxg59K- z#K7h(yVr84^waII$%-imx`Em|M)Ys{HUfz~c-2YupX61&l!^~aB;EMM5d=GCU-C@M zE$y@k6eDlxQL&3@o z5K|I?`+-Jilus9HbW5{WXnDh3c(@2Sp<|cJBQZ0p>=bs(tfU%jN$Mp?CvRu4bb$3| zM8ZhcZ-;1O=n=}S+UKs=DD&t5N19`Bf3vf;y0*|#Fd=FtjN(0U!@5ysdE`tO7Lw8M z#P4hBk(_ZkUQqP(HZxpz-pH@h$|2bgrK)ov3+!NoJBa$wn&@iw9SBoo}NHR8cn!25e}hVM4@jv@wZrTx%E5J(y33x1Ax*?>gl6*wrk$Is3z5l`z}Hoa#@ zJ+{07fkgnh;JIiMuyl{Vi-gZSZL!Sb0l`~RAi$5-Wbe2 zm(T_1l+x~p?-#xiPgQkB$50JC0vFFKIGgOUOm6TbZV3?qqq=`wwxeiAg1oQ#@yP&f z9jVBXI2xZ>>siArcpyE1nV^dHg5r=dDsS{KQKt^#dpZ#cVDVszPv(rikN3Z&Y%T8l z)D1s?p7swSDL2vYJM+>}_BX@KyF;O;G|zqsh=>7R{*qi&tiE(IAQ*HxIJ51HB=?)9b@zfKp~OLPgpeQ7Kq@CJ3TSl#G(P2uccPm?D{CI;86HEB65= z7djI%X)WE^H{7qq6r27-!Y{NOrNV-xNlEh>UL*$t4Bc`N$~wwwV_AFMoH2fbUJx_d z?sO1FGd1BOgI$Yj50d)6BgbhSHq+2W_)Le;On75xxbP6Ybog@e>65p z>>by%h(^MuuV*FIcNGX$pIPo7%o%b;RU7pFm1MclBEZIk+Q&BRP{)=o*YSVBgjwpn zucfvw8)vYCy9g^S9a-FlPX#`NuX{z(N_5ake9GucE7by&MbJMER}T#j3WEF}{u*^l zK+mAVvDhIpi+L9k={mdlH>hdZg(;qy4`9|~YemE2C5sF$e1JRNv-zSMW;)`5y?>kx ze!7wTrm@h}7j61Q=wV5&8Y>XwClaF{C~eB?KMEbb_K;d`24^Y-MKxrZJM_W89{Q`y zM}LV%oNms#r!47)t>};-tRMWLw6&;x?xHk5VSi%!!zSva0BaFC#_7r>f89>>Q#m5cpWJZh7Z)0=Z0=rudbCp3)SxR>qBjcjCVYT3R zStMpd4nt=!e25&5_r1n$Q4v`t1mP*bg(7BpIVI0C25@~>>oBZ3WdyEP{?8IOA&}oQ zO6{-42yTv~Im`%z5jU<3%r*>8-9Lp_Wxp*~87$S>_yWfzn><>}>5}aKRgIKfjs7TU zPC4H5q2le}hMc~w`GTDmO>|ep^iZa^n+@15cU(gnV~$e~Vu`Mk##oz)$TnuwRWxqm*3Apr_zsnKq|=UG6{ufc z$G%%s1J)aH6{H?f$!(ibQ$^QG#~D1P=&9-osWaJ!_jcwOkmbOxmYuu=6po;QKNx(b zJO(HJyGGjtW8b?!aJp47nARCsqaHsP=PtQp@EmkU&#cvsD~T9RkEWKm2WiHF%c}px z>Dl6Sv5mQESAnd$qxLYxljhNq2?l`o10e^Oj(aiR`Sv(i%x1tg-cQn*WiXcRwp^o zIqbt2K4&yJ#}kLZG%qm<=#Kr;NnP$c`Cw;>Y&u^%eJE2!y3r}^XA8R1zh0V51B`Is zz+JlxPj#N4g@`@=6Wo}-od{(I%V6hq8FACD|5OvLDiW)wLE?=*MQ#7IO9_uAd9UYWSJcHV1G8a?e36%e$0t+NO3UCp1cWG5ggtPOvvws00 z*0hCTu6AiOSjD(ipt9$M@TrJbob^e;SBY3nlpH6A(_b5U_G8wUEGS36_sQ(Mju|qV zp+eIba;dYb`;CYYAmZQ=y+Xyo-0^}AgOKKT$lV<@msh(Pnz3y0tx7SGMC-g~p6i^{B7|h)Ej~i%wpGu|p~6l2 zqB7)eeB`4)ilDiCFSxYcVuvbr=}jp@>@=h3WIl1ETlupBF@c6xSgED~{#x%KP0|l` zoGXzR-IB+`{iQX`KmargiCWr#80_cd)Isb2lgDzBJfJ-eMv zceYH)5hZI(%^>PRZ<0awC`e4X8rjIui1htJ1K>F(K^QUlSlm|LG=nkY74{=rvtaX8 z;jvycjO;acXmwA+>e`H+(Ar>h1sk4~55|VkP)`6S=2$#_v>bq{By5SJ3(Yx@ifovG zV;Vx8>m-CEOEu}J?}K(ZvHoQk4fltOVLvyMVRnT_2(S6ibRrU}GFK}}Sw>IMZ7;HJ z9t#pg1G$Q?9-C~cb(S>5ldQd7{0Q#cO)8>yaENPgAU0$27u(!2~bTnIs>4|FB0*)aqv zct=$fRC){%l2m?ohoZo~(I$M`v81OJ^$V42Q*i46!v*NqNl`LFC$p~LlR>R2hRbGiy7pFAdra|&5d@~LUKmbZ*(IZ;x^o5zJ%S! z>A`f8x0Vb%93e-+=9f)tq?RYxK`KtG=^g+gn1r01B4w zf*x|kirkIyWi&XfF?A2oNWvl3B3O6j{MgLxgBqJ?=Fcupvy~M!d&wp5+B$0qyi64D z61|;1?hjasDaKly)Dgjvyc0Z5=(sybVbXG8_{!SPqDue5?9EtU+^#uP|4raAVKyYL zC^c6X52OQ^4J`*Hj%oe5BgqF1cMd*l^LiQcd1&ZOq|Xv$Vp^1GiKm(^=}|Ozdq}$b zygb_sYY97qN#zJ0XvUwjt;GWKE;(hKgi3zq@#1@?UoSwYOUKMzyF&`T`k`{I%J@O} z#me7LPbqNLP?|=_2s6MTIP}?h@#mZUqy&Y+R4gffNTw#TVc!0!nI$cSKk}g^SCROz zJFdGU^(8OS?|v&n-daUG8jGzxfjOoic&5x3Pub?+p1v+8Bt6nZR~0r721@$dLY~9X zH5`N%XR~ELdA%oF9WR0;&DT9GTIa^p70)pp7Vi=4@lbodiA~!R7p%}E{}R$XxD9|I z)o-iM)6^ZHODL;Mg4vVS&&|A{@ zch_}$SN{NA?%eFK$?Wtj!l$^; zHzuXvt}xHl}L!lj8YvQ$sI!^w(>BFPPl%BPIS2)S-V9~KB4Tx-F0=%<9( zTl8QNj=mjLc6jt-s8%V3lFIsO7~qdAPAx zyv=N&TW-mvR3b332S0iXnv$3PA`lM`md&g=J;Er&&PWD_{1 z#ba!JRpH|$bWYNyA#8ws$#;2x>m_BGMCOXSM3|k9x)gyuFIecl$W7hMoZrbIuqgM zHDie(yhOkYW@ws)V`p94#HMgbj?(D(cyDS<4wX# zRN@g6806R@hoB9^=t5%3iX=?oH==h#nV@VQD?vUuV`d7sT#jv;Xe-x6@FaUL*Z}R< z(?}~6Bp&palX4`a$Bork>r2^?2v~vt@oFKKcMT+_lRrUnu{nXSTe zRj%2hCmnI^VF*LXnZ79MgC&=T%o6)v#%o109qtS`(##}f8~LDtLgCGZU5=$H@G_Y` zWL{M+48jC!9lt?Y{SR`p+2F~a4&L58iKPKs0hA&aW1_kRpPrFfdz^J@T(0C~NHK0v ztEkI}`X$5Pk}#kk+cCNG$*C*MY;_Gm{{qUB^HCLw?88K01BWEo%AHOtT_Ghn( z%Ky#$p;(Kj-Du7tO_g&Tk0ph=z-U~n`}O-gp0R4j%Rg}hHA&m{(n_jz65~|`B{(yA zp+IXpUppPh-jz^TxI&B|1a|&>Zq{&A@m&ezpF6a+FqqcI5@KUStS&J{WB)t(f7-flGIj~KnrKqT+xrD;|93H~PfMftMk zk=ZjRPwJs{LRV;_E>Z}lg^8d?nZ}wLJqX8bCcWla9il7=DsM;bQORgIlq@us;$mb3 ziV+_DBM}@?kOAi8z)cU&V;smyN1`<}pDt*HVUk!N8KUBxDR5G2BQ#W5E^>>8ZOtXq z8)lOOA?Sm2>A4Y?k~9T@vFF&m8m>NgE>+Hb>wtR*omFraw8yq~Up~Jd+@e;OZ_rq% zA7(y0;{0A`EEmiSr=C@?D?!E2MB{Gq}nBxk%uyBXtiP~=s21aRJXAv^L!Rd{+sg$D%B8>fi4IY*y9x{`{2I2SHO&~TA zu|c$@e6nL;n8L6IBc1KMa0t}`X5z~b$Q+<|H=qX|nZ9JY291Y8#{Y<;2F0LHUqL3J zA?o6=qJ?Txjg%h``---6N?)-d2uW%}?59TOsXa^@Ag-nxT&B!YUbQqLr6I5Cti(Ab z4I!;s*tQQIy-!EdN^s*x+U@~zGXYbZO2aVB9hedjI9M^XG|25i0y*#@1GhoXt_%V6 zTe*QXuqM{UvnTo_e}k!EgxDhc-(O>zcZ4ycJgRq~#be;2v0w}a?4E9p5?uyHS5`^w zhx=*U-*>(@k;%OqdX{Q6QW%RTdm#sA3N2;$zS`cQ*n%S;Hi3KP$gpBH9HygL_m8RJ zv_T^NF8WU1{$;9yg@U!2JJkke7^)aOh`;pym6HV{`<@mJWwe%wx=O%?EExIuxv7M- z%iyb4xfTvfxNApew%@gL7q{)U-ASw^O|Iy=-|+uhf6E6CQd z#7tzrzRHWc4->@aBSkjgxzcR1o`~do|3g^8Pp1%Wd6ytd5 zFe(sc4o-4D<{r&dV;v@qL$A?34WyFpCY8)hO3UL(_$QtqksV4u5*^_=tW3MFrGs9QK@ z-PM`;uoYvef=EN7y!xZ9kgs2$Sv$xV5s4rX?V>_mBkoo z@#_o{CS)X*hY#{m&D<&7yrRqV<|{!YzzNzNVBiL%e{ycx2IRnp+hhthP^&eZxFK!V z{ovdY^?Kse4)%nRi_A23aZKMggTF|!>wO}a4esuaJL1jNmBnW%91Z%mLiN8j-S89F zqqA|8;PNFhmdzAbyRTSotW?s~b|pURQ@MmO*9H2eo%3xLAq(J>wz1(~109s17e`WA z-%JP`;Qkmd#xjUc)|YTL15eQ?XfRbTKWxTCk3`#8nsAxa2%+il*EJC)qn=X=VKa${ z6TXv-%AvmR22*YE-g2{#;h%ju-YxO#DXkvX_tKzfg&AKv9#? zL2msK_eUS)pfR->BLs2T_D?+RZWCoT2!QuZ)WZdiTujtBN!&HQoXgG zW09_V-}wVh{Chzv$&2AF?pqacV~4M93lWvOIsPkp#z7vutJFDkbk1DE|I^c1ctzDl zYkU~GI|QV=yOA71X(Xf@q@)|9J0+wW0Ribj(7_o>1f*jK=@bxdG3Yr^=ki>}6(rfa@vsz~L@xv`#D}2EStRin)h= z54p+N+=q`qKFwx0$xO5isg{{}BJFBD`H)DM1nNCJJv!dg?VB!`*a)KC}fCYLE3<;sT1*OxrPd1rnzNuj;2585Ejz1kvIapRtcmS_TJ&6=K}k`$*`HoxM2@|?^0gmlPy^P;kbiQszg{&kcB zN|LU-?M(--3yk9^vdN8y9m0}Ou3%g%CQ{%uom(fov3bhw6ZkuMPCR)OFgDwWf3EHQ zOJt%P%iHhg0@Pyy><^yl&s_bFTMFulPzbiE-^Fj*(T3IR&NsPY;swtw;QoQRiGA}- zeG6A_7LLWQLhzSBWo|X6fgq8MeO9rh$7BuCV71RVwf?|6#RFb`)vI%a@RXDw`^c$} zA{KL1JXsu0hqxO>3bXSwBM>5B|1%EUviD0A4XEm?EI1b39404K{Gps4<#5us=!P_= zo4U>W&oAc_fhP@f{?26{uv1nF(%Bku1HE`!Zi%Xw%9{AMi1Ou#)LsL#ztCn2n$-ORM_Qn z_Te)q8l`>aoKiI>!QYYd{m=b$lJ`T#EoVXJ?&NKXv51UR9y<5X8U~3)>&6HNAI{IG z;dv5E)!)syI~Jy?0imEPe%0`AAmtF{T!ZDaRv~jEKS2pe!EJQn*M&DD3_1peb2#$C zy6_iqt``ztAv5ViqM|a*rMXT$auAXNr2|i&I(ilj_|TP(HQYX+9}xQh?OVg{(O4X9 zwSf^TpDvhS=+Jdkn%mWdr|(kUKknc!T_6*^m`mJ1#`k0BQo;84l15{K#>%neavFB| zGqutW){%+?I)A2!Rx1rst<;|1KnOte%y(4UpxrXOfLEq>>{S^oh0+rw0~HKQxPXNU zw7VNwvRf}5bvI4`T&l; zKW&{=PGT&0Unk-A`bHQxC7O50apdXWd9LF`Pe|()>2Q1*MU^=2Y+Ydy`%Ve8^JK)^ z{Z74mvVhe$Pt<5-582y#eInVgi#k2&Iy^UC+A$?h<2$F^(gZ#MZuoYBr=DNSmZ`z4pJ&l0-V6%Jg14*2T!6uoJNS z@5d(!6mfWK*N$q~j?SBGwvfDdY{UIt7=mzxn!(WM4^aNIl%mWu-b1O6I` z*neTM{2vBD7%uVYO(N1vQdP;33hqCp|irh`1{T!3} zD`V_baBiN5_Kh4Lcu4lNxL_~PAn$_ebFEpKqYQi0c_NJ4+5eBIve(+U<-arw17)Rv zI%r;lAu8ylvCO@}L3RTNIGpYzZ{kH?*Sx{vx_!MgZV2khokG-+jVsaoh6L?w-(VO( z>20JRJNh%JniHE!Wziq)<~uGEb5Lttyr-_0ML%qS=^ zTMK+EbF_Bg*~FnrMzc3PG$7;4kTcKLFSp!|T^&Ayi*dRMw=_veB;j@YY!+3d^~;NB zlorn#S>4kQ?Jmkh-|pQS%_nJk*O;iXP}YOTPd;y|3TUv(31uHuOPb`>r+>CimAx>= z+fQCwVz@8H1^(LFrHV#h_shOE=mpTz;phO^Yp$@-RuTtz0T%e_i=v z9!J8%IwV~;=*p1U~fd=8W3>(&;+e}O-KmG>Hg{ z&A|ke&3KFtXzl@1P*qE2Qe&2mC9fAK^xh+aGGm`g($_+L`*;?gj%TfZ@`zACjcYy= z7Gi`%I4lF)PIlEDc`BNSr)o(Wr^^0MsU01=Yo+$=uwLz#ru{?2O1vGIae1k74*@5% zK~4{Cucy4Cs&!};4Ai}`;hN&VDjOdxzjZqQpKfmf=|d6> z@f>jhYy8?*3Ro;VGn*M(>}Dx4B|9$G_3Ox1S}J?-2_=%|2ykZhD@nnGu2xX$wKvPx zQ@k%BkYtZ*a9Mg3|AK<<90rYRdYwoxo~$;1+f_~gF!?tQ7|}W%kru)bqQoe{6jm%j zV%}6YWzn52+fx#F1w--M)VPlP`$(Dj z%{8h}l8<6mGsDD4(c}3Gf0|YB%kW_OjnPTUl$3TnN&m!SAnY(g#DYCbW@^aOYG)1~ zY6OA91J6;cw{-ifHyowTaCxiMh$(fx081i5lvyir30I=5?*g`&^%+3_HqpIdJSfK3 z+TVz-d;)7NLe1T>=Xk#@HRUeM0VE$&#ZFShNQ`xhx1I7-#CcZflh;C>kioNj>?utb zab4UG7XkwRWN&iu*vscgl2Cqhs&_=tnszm=uMJ*%3Dbkha#*G-xDdO!Nmxv!8FF^g z4_-TwZO~KDa9%E~rkv=$tUJr7h9Q^u;iPGxS@+N5Y?c?WqJik``i$+r@Va_%oenUpvX#KPN%(rZ zQ1N{4zv)P^x79e_F-_{eDvqANHfVWkAlYf3HPMh#K^BmA82q*8wMP$H zQ`W-dw7|qr=Afo$rV@lDP0522@%I^WaHL3i^d9NQsG%J8a26q5fArE`PqTy<=$(Gy zi@lyB@{E@@y3&g;VJ{KaAU=F}Ooa>i#8Xe9KpNFRSjai?P@%;RFTV5l@r14%(;RP% zrG>suj@v}%;oxc4>v82~GbO+Ry0{BSb-}RL{jLxI%n}Y;hWReAXL4?1K8t2Rdz=3S zl}IV4A+Ug(+&NyM;@StrY^yU^OIml9H2Ie3C!INt%C)AJPVlk5;H%9NuOZIQ-N*ia zDFoSb>M?&*nRqmbI2H*6p+!iq>wniA^( z$d)mdIf+~yiuCQWkUuwME1kF6HCz@$0%LGN?%-@3cbwilHHmPg$oFllAPbQ{Zba7YsnD92)udNlImVuo$(U^2 zcaz?%zA46=zbYt_T7<|L5zBE$g@OaY{bqZ;n}5X&W1CfqFj#7+34x+ESN3ZNk$Q?U zDAJ}^*tcasIGO(nI$=NCSCNuseoT7#4*%rMVHJ+L)7Hk0i5S-_vj0t}5}D)n7*A0{KTOJyXJ zhtBn;ew;wk$-Bp~GbRGUe&Hsg9%v0^;+c|4SmmE0x8RH>bRmmH6y(%F46l^;ok
    }r<_EVJK{e}eK76Jrmv)Lk(Z4PMIF$N z+B`EB>sG+=V1@oc{pvke^x$_}1R#bMdgowf71xV??f0OUVTp5?@~rKNfFNt>VGkprhR`;Rc3FF^VL*~}9c~-BwIduEZCj(fhHHiBdgMD} z)E4ZWAOh)O+R)Pmmr#89FbpEq|D(DjF`cPygS1>0v8V(? zrWwDC-CWG`FK0G}cvs@>%s-|K`Xl=?ttn6j_>>tU3x>P^|92-f@6rg>=M8%Ke#Zz* z?-}Dt7*!FTcCK@o1ZHt*p&N!98LI+VoN8u|mRyr!C48EV{`}IjF;WG2dhoCxWbo{4k&$%L2cltj0c1jiI>iJSjPPP?R$0ABila!G z+Vq{GK*Pj8y=FZa70Q9_e{h{^rEx!R7G?Yh$b5z$C!6U@iP<~ZRdC8D^|yQ>j~RJ$ z-r7jg8=KZIezAFGn@C4P(~hL5SUFCNs?L^M^cEWMff^ulzbaEZ6F}{2<_?|Wtt%5M z9k3S$yPsNq@dt$`-V2{*H9c3_FNsU8ed0%H*$t+Yo; z6Vi$GnFSy)tJG$*f6DDKJ|-2@;3m!BD~5w}ELY52@;gS`H%C!g>Y#M823$B&26jI4 zEgV%tgPxc+*^#^1)Clq1?0w zK@3BsBJ}!7c2GEKG79g9o1w~$2-z?uZDZ;yqmmnTgk;m$VMpJ*+~dw!+|H(6e?|-8 zVbkvM+UaIkL-r3DCS!I}G`>A~?2T46VZ;*iFy+^upL(dGa+@SZsIeBy1v9ygK^NEg zVCA*dk<7ogy#0@kye+IW!kHah4xPlCXUfQ&Gw|ltUoMqPf_w+L7twCKZq7kTB&QqIr&kwOUVQ(adUO!Z_G?PU^1u8x>DC z=<^Lv@JE^lKKZj8G9s;<@Zj4A9o- z6w@O9-C(vK)5${2>H6eU;<u_fRSGGDS&WhqKL4jibi!-GK^Qv+ zg6Euo;6r=cVwEj9yHO)@w*@}+lB^KF)F0CLueQYr@H*?rEce5SdsXRLZU?cy08wHK zv;?ghBzG*C+6}?|aL&^b!f^ZZtytPo*R@kD71R$Xe|jAJH&0JatlX4^n>2W;3)y=c zdr>F}4Y^Z(MNH}H>4m;JlM7o<5^^J%iMS=wu(e`Fn5TlbH}MtpZeHb;GD>Vl67lvP zz+7yov3~5paUL2DR0v>()q#B$K4hHYXglS{30!10QxQ6DCwa}}W1-X3qeOU4Rq`Fcxh!|JQrEa6HG7e@CwiW>~2fb^r0bHss-YN7%&Gm`No) zCD*+t=85>ac`$4<`i4{`xb&Z=Da7}x-~I{Z!U)L z?hPvQR&~=&f2DgU5WR6EndT`pvD?S_zk=4GL_d;#&)B!sOr_r5GA_UF1#W@3X++lQ ze&X;`{8!K-b%%(%kJB*DeaEfQ(Sb3?aer824bf^T%}XLr-eCw6W33BZ=}inf*zG;)P!LDlZtG@DcX z>D?<4hs3`L$+|0nvXuZWDZ}k*ltOhTZ%(Ksaa5_Esm#Jfi*N*vHpg#t9Ov2ei;WUn zXQ#@pi3uYF`)TsK+8=z&liyTnJ)>FhsIUG7=@QW05bys4-19hf8QS85pgA_rkX{{+ z@b8W1iWXTltgfB7n9@ZXttFj zO1XFt*yiIV`H&|S&p|}jQ)wNgSYKfWEyHBnbRn8@|Mk^3>8Phy$LY^1-*{vy@A7?Q zlm;@?wgYqwr^>I*i1;B2OQj4n!bzVXgdAO@^2ZQt_k1X(mGQ4ybn`O1Jbpzz?vj&% z(C5=Hd^t={%0$K!9C52sn4C=Xpw2UNAeu*ndL6!_`)MZ;tvI6}CC;J?FkBxZSccY-Vy!O&O%uBZ#(Xk+i3Z5I5l9Zj zWx{J8n~{7HB`&w`D^?xi-nSH(+ZaUjHq^*CfkCEN4sJGX&K`sp5|MGTNgpA!iaX@SIDd_U`wM0^!D@tz7s=~)94<}*F?L{QkrY;Tq zZx}2$ARYKx7;h(@)wFyn1}ST`sVzv1SQAjOaWdMEObte2*9&5Fn%+U#j}36`FRMI6p~Xx30) z#ascds~>6%lg-NB`t$z}1Oht%EZ>bSeN_nrbE_nTAjvX2Z;+g20VjAQb3#odG$Ipx zGLYUNSAqmuvd5q0JpLLl${yeNoj$(M=Tw-uc>dz|?@@N)+?=`ofm(ADV%cE8fYOS3 zJC~pg(f6y@?@#4=5avV^fd7+@^6Mx4ZN&~M5lpY^c8s`XXZNBI@y5T<7A9qt-P^6z zul6dBbRn$oa;`g4O}xf!~gppJx&_A=o; w`<)S$%%K?ZibwEUyf3HR|6)*99^R{us8qyq1G(6UP7v^@L$s7XDOyGT4{?(2EC2ui diff --git a/docs/img/gantt.png b/docs/img/gantt.png index c462adbdf4bdf93fd8cc4cbca7e83244bc5a801a..d74e1e5c3e5dd78ee193b5259de7cfdf8bda7038 100644 GIT binary patch literal 121795 zcmb5Ub9g3CvoIXnwz08oYlDrQ&Bk_e-?6c=ZQIty+1TdBykq0r=bZDt@42q$`{Vb_ z)icx6-PP5Fsi~>12o)t6WCQ{PFfcG=Iax_nFfiDCFffQfIGDdJ$A;g(gMlH@SV>5z z$Vo_$sW>~BTiKd{fyqWBr^2cy{>Je?Vv&=TwG2=x3f_V`(eR8`FAt24gTjzUf)^W# zVWe-vQJ1WvsHA9V25Tq_?`90_7GW&=a`vMybS#s~zb5B#3;3Az{K?{RxzYY;{=vrq zRyN!qRY0VU?5}U!!^_MV$45?`EawR)D+*Z#hAV2}LRH|uvLXv+_Vlg{^dKf{c*GV+ zug!h^tSoZ@VxWM@GVZ(91ScXslYzAn%1}+g`p+QVm(1Tv#YM2n=h20Ni*qMBl>`cz zX4UM-wkwM3qwRBoC6(E>@M3~t>*!BFDAC*%HefXb#WA3X+XtMF?>4r_iAa!SX9^%U zRA8G{-htjsKL#vb!j-uNT!OLNN2Tbr$NCxfj?LHGqk2&WA<I#HsYk&VZ???Rb7Lh!JvKdLON zIT+!N1(#4jUMbqc@H-|JuK{Q{iu#kJ1zlm!lx^UX2onF#3~vasc?tvm6yoh4MD1A# zqV#EHIIXCm633OB6<0EgYbs2Nu)Y18KnX0Mv6P_kAC))l_Aer=_M<(TNtgnrvCUV` z{idFnqZbK*PrLO#&yMwUH$wqmM7YJ!32V}aNN_8|!CS%O$>5+MY2x}*P)Y(A$t*#3 z=OMjVLizs4gJhUj{+n^;0Lg|f8Mxf*JhV6H2v{o0J&1#DK^3rjBk87}rNpCgk2s?; zra`oEE@Dul0lmdH-}>WM6MmVLp}%nozz%0XW>$e{iPhY+Wst~WFb5*?C2jOkABVcH zWuNZCCgC4{35D&wm@5OskEsNy^eJ7@?G>@HBK*R$;0r`?qu5-(bl!m)2bl7-v;@%} z13vN{vI5S)1va3ku>4>9+70p_UbTxU@%<_f;#e9W@u_VR-+yb0Nay`1{KNl8O0yhf zlON9DQ=}_KDI{`;erNFBc13($NZo=b;Gf+WV{%l7Q-#i2Z=#LqjxIp47vFrX-NPMf z5t%rWi2M%cn-MF-Sz2X{5#||25;iU2Lzc@!Y4lJJrHDC+g~=@K@^3`a-tQW$cl9;Hx zO*DVN3>6{|OyjBe7($}WL#Hn)%b)NH=7S6>3X+WsyvU!fZpyn&Y3MtD8ZwSQh6;?P zKUW@tEreblN*&=jY;+zp5o~E6^bB}SfPiDQYY=W8Y8NCVTwoj;fw+t-I^`fyy;K$& z>L4z>7)3lU6||fLJ32XqlsP40JbMvQxp)hi+bC$%c%Ro9V@ZpbQZD`G)i@uCq5g+KYe$Ve%Z`HRM?1hLeQ!F$5YR8e ztHdYlH}I`6Kk6B{7bv|z+yJRG(NU_ev`J|6aO-ekaHTt~^NIi&Dhzx{g#v-M2i0($ zL7liLCqhu*XZ@Rk`yNiwsOKnco>;U%Jk&blou4llm;|ziNas#81U)%=tAg= z>EV-qCebIk(-UiSRNbjt`BzDE(|X05jOyQfy>WJCYESo$*ia}n7N_Ba>c2p=?#p(cr_a+Pr}keyOhBy~h}WUtA0OZi9? z&N5nlG^y(f+u>KoxCgrDK8S_X{HX0$-L05VE7!DAX|K9bO|G)8A;{yGYF23$zxO|- zUNq!t$OzGmBha&}(QRHcZYH#TKXrAJxyQTLzb`%&Sez~wnJ2T-whXp(xnZ|5Rn2cQCaI+H*hv`j&pk0wAegYSy{GQQd>>VR4nlqEDrs`!$HEK z46Dzo&vnYamRlDU5X~&@mh_5!FuQoVELz1N24#^lEQ z#&YNO)y!S^z4ocoQsNH&G2e9K&~0IArdH3gZ*6b)MOS0Dho=ak4POdRi~BhxDJE&l zG)^wFQHeta$b6ew#6+aUD0*AMPtC99ll!w2dJ35$<^n+! zbQp9hOlz2BFjc5}$ObAG{3KE}>II?;awhf?ff8ISoDsfnklC+4b*I`sdmAH@_#W(m z1n8vSu&-I?XjF+-i9)2KBXzOec-91U@iy@we`vE+{yqH|a7cepazKJ(4c8b#TZ*rg$yTjq%0Q3qb^tu6{(Xii6W%I^_S;lW zt%pX?)F??;&M#j0%N0`@^AoF()k^^WDMv5|^HfuMyXoP4|4je3`EDa`=7hb|PdQmgcCnv?WAdnWR$3eIm;r{->Z`9E}@&vqK3%r3?^Qylcg z^vU&|e2TZnh!0K=LJndzrK+v9C-n_FUM@1qZKgEZ)NOSYbQlcQHSl#-o2G0xf4b#X zK4~mhKvq81EF1NjzP}=GAgA)YIBYcj{;sxKqMQkx>0l4PKBr>xO|3q;t@-;|;#$!f zM=$VAvsKYSZ?%PGJ!HMpeei-;=g3g2Rn70{#m0ifmp+bPVYDQ ztK@kt9W569$etyj(DlnCVWPe3=KhwqJHydZZT%Az>4W*wVH9D6+++A6{W$s1%IvWIV)6>=9rxk6>bVQL3?n8L63%%} zdOJD!bvIo!oorOq%jp;WOGL3F*JQ4xU&1t5E_LEGgn=XSzPx zkdLa;j!6BV4;ow^u$EWII9^hJS%CB^Tqd6>z4v&`4*Un0n>&e>1jTPhch#qYenQ0E zft?*fNHEz+FmH5J6ck%LqBm87A4cW+!I; z{y4xp%Idg)f#Fd9>jalmrMdzG1JAKi*LKxb{K{|QV8?81>hQyi8DQu5cQhE7Ab|gG z)y~Y-m<(WNYwyAj5Tf|65&VDa|7vETAp5T&t~Npx+KMV<5)RI0WL(Ug%&Zi`2xMer zg3hMq{Hl`D{{#N_Nr=MI)zy)og~h|egV}?F*}>U@g^iDokA;<;g`J)0?+7LrpuMXx zfXUv4^1mnfKl4bMxtKUxIl5Xo*pvNhUgIAQZmvQU6#qi>e~!SoTQjK0Q@`~+FxUF?PHs+#Mu1!@{;+4h^|;& z-Bw+t-M7W?UHILr)@fkrY)Dd4l)tE0ucJtyD3l;g`18o%sLFVHcft`(y5EA%WootC z&#Sum(Quj&-0R`o4IU9b4(uP7UmE1B_8Z#e>g5SgDsjE^g6-%SWZ?g_utKpz1A)&> zo}-Pb{}ED(E8l#_)-XZn=Ss<>O!9w>kPI9xk8Ijn6oTp>mv&MAKyPNz0~$GhDA|8n z0tL_hIS;|$_aVR$ZrZ_;GpYUqN)T?UApbcTRB?zd zWp6H`O1s=Y)quac@>d}I|5zG-32qnNQDzv0ZPWjQ05=Le28Y(N=@j_5lS1)I{#&`j$D9paC?L^@a9=GTf7Pu4LshI6f-uGYKjG0|6?K zX+b^zT2TKD2Om&YQ_+ztH87%wzU76jHJpriYyQd%U$ZJZ*U^$Z6LLlk?8CCv8uAOR zd6b^o{^w%XlKvNxIl>f$#d`&6Mn?StGJp(J2NfzOh|F@kBC_(Wn!n9=R`5~`?Gn`j zYRnTp#lP;0T+|qC#Pg4R*ZMhU?wYF*4)p*d9>;bmt-gxB#WMxnGn7M7ajiB{!Wxvze!~%4w_Z{S zqlfCp2WdG-yYFAW#}P#`(7)O&rTTCYn9wf;w0502_pfATf6q*|mXjXQ8%oZi!9ZsO zYStzuJ)`Xl-A)cahy4edUV{;!kG?-y-$&WKdXqlqHAxhKh?3NWcDSf#DD_PCHa~Hd zzPc;ayJ+KU+%Rfrc^IHFmN?I`?;Wb3Wa23v^wLn|ge!-HlIo8XeiM*xV-ed=*#etf zO>uI)DTVfISf>~fNY;xG_$G*OvpTRx^p9VV00#27$Us4LS3oF{2!i|5Vnrfp zIAqBM97W&DErF%xq^s(66fK~+g{zf#FP-;OW_vc_HcR~9$7_1*JNYatF%ikN0ue}m znW&_CLhcZ7o%oNXyF-sTsv%|=yZG}a`QeQ|b{d}=dv2OC)D{Ws`kI60)A2tD1uI3qIktK5=~Tl6bezkF zsAb;g)5u(uUaz*(qku5)F@4Wbf%aR0e{7R}EmFpv1$L2CU{5HJtT93QeTZ38Wv4iX z-^^acpxy@8q6XIraC$HjL9Gf&%#ro82&1x*Btnp{mG;-a;|U(w3_;13cQMPsIsYzG zjoyEyZLa@OnHlnr zc}V5~_eTDAot|J&E?VF7uy~!cbJCu>7lgdN_6G%0(UVDte`S6KlS~f#68mG!wK9O0>R5uC3GU1i79vKl7W`u59<(h0GrdF(?wSe*i8sel zpdzb&G5CNlq%g@u9bH8shw(d_9`dKB=@<>5yx`CXMg`QQraPp?nO$m9GHCSa9=R%o zseYReMyQXwU#_bd__UujTY4>vmNeu-9dg!Nyo0SHFn0F9TX;6Z+Yv%BlKiXG+T?}j zc}Mj$^94^Kanhj>Y^7Z1>6!b&+lZYspA=4Qq%6LREeAu!u)W*PU*E+rrgye=9_fwI zDV~4Sff6tGH1T%RhQbHYOyfFvm_gXNY>&AW26PiXvylzx-}jj%_U@g(ammrGxl zWQ`o0oIC;i@v%+}Bs=IcEjPR+k-+~-2v7W;Ld5Sb-9owV9rg18PM|i>dt5uXE^OH= z0sUZ6ZHrn)ReXQEbSvp3R*2Ra0vGZ5C>jiOgE2Hu=6w_r_Es^O!0r;L7dd^{aeD1e zucqH~x;be}|Akd_E4x#6OQOxo=wPyC(oA%KKLKU;qut8Ibfsgm2d5K2kkZ>LPWNn+ z8l&0)76q>olf}p2@yRRK1DZ!Bf0Jq<5+JYcC9yiZo_3QJjpkYa3}7vEkFyQ8p(uxi zN&W-gI1*#37|7q#F@b0F=-qm3`%^>1PblYzlH8g`1l+mf$2PqBSeHF z`O|hQ{M;dE8b{rOQ?I~MZ%|*TzJ01`KSbX&^ldw$!{>h@O~^rEh`2=wP(LfwRfihr z6q=zXH~J(ihxcOpW?^b1M!Pj@(6Lw%YPMlUEcj^S_fS18{|cdG8AAB|Hgd@^3vnJucXHyq?onu_`YDV7<>c`?yQg~nPVUYv;zAMUFP)k3 zr(L?ON34v~GlDBn42nKQnqX9pRe8jDB&MxWnh}YLlV8jodReZJVqaMyg~m+5gf5jT zOMoTvktf>?33R%KFVq0%w5^{YZb7YZT=QP-) zu!AKUfS$eVgA${o^`=qXrbN*{gs&VNT|n)~A?Ga9|3w|j4eeDZmWX(yuR)gWPO+@? z!6Uzbmt_C)`*p0pvp}-H&B*}2(mH=s4;2MMl+VPY1FHV1F1oCECLfTBIf(r6;)FF!z&1brvE*g_~4KJA(#eeVhX=CJ4+LAg#)pLBU;^;z@g1HzDp z(i%NxojzUg5Eq<)#J9^%R>VQ0a={C5qXhZ9G)rDKm)ji<@1?IO{n`bW0*tVVzm_%c zvbACrq@-+zk5wupf&tOjr_M+9ECy3Kl|8j*Kz*fa3qOAQ!VFm@+av=esb(;tEp>0* zE&j{GJ-aDqzU7(rocGH|-r?wblV0P>h3$e@6OUV#p*cq_Ap1GU3 zBD;(`&m?^-YeNrH?iZlckaQ8A^R(p=FVkS^`J`L1TB2{e|!Hs z40BS<=Z37wuEn(MyA5XvbrAYN3YE^ ziPih@YXTDaR&q*=92{M4JWYIjK8DL-Ta|Bd*!ibfJXSbD+#ho0lPD**3yHaJ^$X~R z-3QYq*^+n3HI2;a*xOQDetWQT2+|Vri|={IAN}ORSyI%W=0Blob!e=x$y!eDZ25~U zU3S@hr3IR{g6eJK@C3%>!AV>{jA#1v06%axTf=tx-%PMU*3xu53?@;O$k z3loCbNsN`VmK4VC*N5jJ?1wpq(sA4FcWLULpNQLpI}2YS^2<g?1djK_?115YLT`cZ=Sw?EPYWXX(LZ0M`yQ{x`v z#rO7*^*H?eT74tCaulgz82HOFUCj1fyVrehB`lC(+;kjYt)6Q4ocwNFM6|Ly-Di?5&F5S zEf+zt9>W&Zk{r=fGl2&`zGc>N6hV=<&FSQuASGAj-{Nz%#s-$rm$!WG+~592A|s=O zk!Y1Gr9_V}b$B%|ZRSLO?z^2qcWlT;v7m0ce35}wA`Zg$wXyjP8Q^!7MJ)|^C468x zk-11q;wfMD20h-jlD8yfV!L&A3=(wYp4vEEoxd?}e3@%D*NAvvC*jKhD1|cW(@VI2 zXWU5r7Fio@G;U>C*qz__-tA>k7-D#;Vs5w16g#EoTRv1X{=BRd_qg)~51rv%Z-e&P z7(6Kj;tU=xDg)mEuIJ-$0dDP>SWh@@2p~p=b_-S*c9?O>)pHvusA=Ss8F7pxeuE5GK4gi1Y&~yZ>0lh;l9JvF)o8UOfBrE^ z1GG5^RbccvS^ZdwxQ9g@w^KXRZ+_s|Ye7*J{)Da^2k|(hP8CNg=mXN)os?FajeL(c zR0*(ICs~F7`8>=~ui!73*!z0XFLE(wkp0+6G@gF# z&Cx}>B!1)JzqPu$8F-UydMXfZ6fTeHozQ!b*U!OE6|RX3{St!5n{QeOcaEsxpXQaU zcg!4ZV^OrKne=b+ycw?`V7|gEl+!Rc)m{j=^~6&1M*Q19#?G{Cp0R5i(gnW5pcUwL zxvA&EFYQajX)@BRy{k7vxn4ut%B0*$)Ln}B(jGs35)T*Pw^|a>RHJ=au;`X` zKQA?Yu-$vfz31v{G!WSm44+z`%W&|hTm93SbPxb4IJ%|g>}_3g;PBkdys!9eaBl-- zG53wtX%x%(gwBxSMhCmVi?YrN+D+mypjud_%pfC5e<_c8(j{1Y&N8rXy%YF0ot zh8qK)44K#yZ{{FKhI|VvyZpx780Ah~AyJv?&HhnNmg~eVVfeYsb8~C=DEI3(mQ#o^ znUTPD2%*&B00G^#pypWIP84JUPEgFo=nGnXH=_09>Bwa=f3ZB?c&@Pxw|wu~;#UhG z6p1wU$2}=u;kp}+x+h=TG?OR}!!FLlO#Udjav_7}T(Gc}0acMBuoy?(4==FCk*OH;hqo_!2RYnCO7X~_ZDYw{>51E)m8}vPF$BdzA z-^mIiH($cJ6p*iAI!UOaeh0lF@EaW2%+9QCk*~#)xQAWOTu~F4@9{IQ?M9z$flol} zcPF2j%}1zV1jl^A(v(3aFS#pI(Y^5$4^R#bS=$-1X$Ti6p1oG=t<*Dr0lSDY3eq~f zGSCs2dg^x?c2;t*Jo+ZHZBTC*YA+rZt%^l5>(FXR)+%bb*nkr>E#%kFHPldIIbD8F zLG0m@T|!(?Sm?^)`u@=z*d;tD$Y)W$TPZ_`7+Ir9{Zm#3;LYG?_V_0X$nM!&YJVdB zurUjxI7^t?Tr>}#_5n#!MUec3o|4BtJ{7lij6Tk06xYt9qbK*gp%U|`KHD?oX1sfm zUjE}s#ic9Af;oxq_4`2ys-12V$1-TG<(}8>s75oy6Sn(CzFUX$(nHOB_nOY{V_9IWhKZc?w$S$_alzvs zMkOT1LI#=xk|z=ozRSV}A`Wu}fp33$45E16qOxE288$x8BPF=-DDJ-)$&c;3=%Io5 zeZ|*e(?fl~=C%t`D|A0x-DrHeIbGOX+~VY9diSJd2(QLIm_GVp`n=35o%^shGEvcZ zUSgJYmedqDTx-0wD|JLYn6x-Ft~+h@&M3i1MEO-1+pT(@2M@*ueXBHduybMj_t>tV zAq1c_Yqha3X`IwT@*cF))C4p9j=EGv#kDI|V>bM-dBfXf45BkqHFVfq_`3d+?H8)1 z9j8g>qgJiyaDm5GN9?X~`n5 zdd}xXT}wqV_&x6SS46`j9paSmeQhV`yp%1!wDmHa9$cBJ=W1=?frW}{T^+EaQ)(8H zQZw{dExoO}&>V->!zVkz^`Hs9-rD@tAmYSDHt;Cqr%qBKO=5?~%-m()0yd=jx2>Gv zt3BSVFCzWV-xH8(v<9NAvXuk+V4_6me+b%GdbBzXiA7QOhI!&(V=x9e3c2YU)jHMA zs+n$D1vfr^JI9MUrBvSOP+oVrDk$zs<%_9T5~y)U*-JARPl^+$&_n~(+i%Y3^D=gC zH`NSkRa>@nV!tdS;-MOaUE?r$`s6%hC6b&4C7S;ZEL`P9$>gee&y^X&PQ_j4?JZSj(t|cW zoTtV^Rw491#RI?cTo$~Q#V1487KB0Yu~qT^>eZB^zR|1Hfr}c4G`PO7JK)L~``2T^ z#h?N&oj*u@X+9`7R-jV6wJ-B{!wI)RYzaRXW46HlF`qyM_FM#cOB8?i1qtQs+H&ua zUhyYZ(laE8hj?DcRrP@USFPVEd2ozlSOdWN9P1f?UByatU=qxYzfhHbVhXF6Mw8n| zc)OUQxPKp0J)w%~%_C3Zup~>M^~J{5CTK?JwLx{w`zid6giUP8I9C`NxF4D(yklr( zR9Ml@$BQx$Y?ssIS$&5uJQWPhgiAY~}o7F!x5( z>Rxz9#PxeB!TGtsbifEkmUo3(ysPYyZE14qR>Y;+=5eT#BMLP7vY1TznmhGCbC*3?U`XDQ4L%oh3UtKZ7 zb*Ylk1Kz=5OM&hFTRn%$l;V z!H>`xDziR!a`AibR`28qp+qXHwfug^B45dHP8Kccp8~7p%jkl8YRzo19(?+}NoR_3B7Ws>d_?YjRT&_}K5H)f0s)yVbGj6V0DYUO zB02B3Nuv-$HEMStKPoLrw(5-|Ey!bBPZT}FVZ0<|@w6Pubr7Jd!NFv#On$zIZD0bo zZ{TQ>k9CIv zGu#atdFP9j-)a#D7WrpS7!dh|aNMmsVrwgoB$}!ne0NSRDV7w*{ zY{J=L0t&k+@GdYSBq=BoHB^cNyc`|POA$uDZs?^d0ITw^H$FX&(ZEKux9m_So!FPq zj{~fInF3T_9W`G-GRP6tHq2Gy68p+xyqSp;_F9baTldXQ`Lfudrq{Ox(~ESW&Z}DFS^l!m zr+^9uT6K=ybau*s{r>7Ya^TOiOZLoia|DuDMUQN{Gm+-+9ARbhRNUY*20Vs&rK}Wb zWAeF1O)gB;E3(UUhc`|IUqz)vpN#pae~-rRC@cK(M5z3@hK4$AC887b6DQ~&dp6k! z-NXaE*7&$sg&r4{CL~7J*%MKVUPhbW@7sr`5;V7ALj5s73m@^2f%A-t<&R_$^A#*e zUXpB@3qczPWww8DxtJj=8$C_RfofGqSO9g?oUKO{!2e;Q~(JFSoL^e2^&lqalzQe9@nI|S#aAq2BX zC9VmZzRz8$v+icGkI&M%V=lzIY6;609Y@kWm94eYeqbS@Ma3Xyid;nxd*aI0Bcu z?MQn-!cen9^p+M!*KkHJ5I!CIiBPZ)N^nj2E&1qP0ac;Pxa?S5;_NItV1;p2zd0W2 z{7F%t;qj+!u!jd)@RJJCMg`BtY9of@xj)n(jXca??K#vl$l=V?S?-uCH_V&5x)+Zb zGk&@|bL7T!wMJ;pbo7cbD7FXcdAidh2|W5N9o*`^iUmo5PS4T0B!7K-YTCf^t3g;y zAlc`YRAASQ_Um1tY6}_EFQT!EO|RmQ(rtN87JWQlOwmu6IlR$AaR9_!Y zFjYuVf5}z1^R}edM?a3~vqe3E--Oq8lU>+#Jyv(Q`G>FfFC)1qwX}l^eoeNfHNV%< zxO%U&v!PACd$D!x-3Bf5ZjgCo@iRXJFX8c0Ak=3;41wTF^3dq299Pol_m+6XTo%%Dp=y0ywaNXra+SB6E{k1W$t2PYJ zD~hzmp2Fa}+;LNqSGBv@$#&h8)KN_+UG0?Nm-w9zb|`ibyX^qZv(Xh{#LPVMjt|H{ zNi^$H*t~*BesOE(k8?RcdelP`-jZ9!nGaU~nDR>da&YmI_rA>oX;R67d}egadV`uY z1?`QW=N3Jip>f7Nk!|?Lv;b%7gPDRE9~yQM%!EGKr{(PEerWx!byb`#H1HMd2XsyC z8dstwV0888C~k(ItjpGV!K=6Ni$1>BPFeZ_!ikV@7n5PZcYsOcMeoi+oR!LKpnd`j8BkZP>ifq zJp`I8}WD1(WyVDbAAm}CtA zFJ(5G{E|e_ZB2jRtAoCN5P-}7WC>jwku}^Y$)FC2DH?Xhg|$X?FTayMIoU!5S80Gu z;x@ZO7YQdYWy0S$vWLpLtkzWDsJX0VeV#~lt<*-qB?Y~5 zseM44ujMGqkJ8Tr;il#DMmX0}Ki2HMWA~at`))|h2r9@}o2y1SHS)B%<}6#jM8mbr z8xE4mBDZAxq}kKr!TZ@0gbp79o+xr5x7>*z*?Td#1$$4D0OQ-Y6gLiORw#XgFQ7Qr zzv;HJeRhWiBWO7ME6gKYKuzQ-tB6ilxc~Mr@tC7K*NeNat$@(^vHst4cHV&SWsbZk zd*=M1<-7Z+##zfL%cq)&uUEq5dp?^qlPS;L4k6N5bcwAODSe|%4v`4O^W;g^t`mLX zmb_*la!E%fW)BTI3&Gyoe*8S};S(Re)ac8=yZp3Pte5G{#Os)fKoY1=I-gE?Y`A*{ zRO0v^k;&%Hn`b^Wl1u)rB9jB+;jh!qC`Mw(b{Z9FwnG9}8<1h~W7QqgdjRBbYM@FD z>iP69rA?6^mn>T&0v?rVU+|s1EI1zSQKCpgfBBPjvUk|%6=^b&aj=%$wn@w&hlDnq z=d4FlW_bF8!XVQ`mRbJ>9TYjcsnRl!A@$t9S9wyaw!O6Gu0j8JKP)x8+n`J|a=2gMl3#QDMP#4c$aP7#9A(H5RU1c^Grj^*QL6AT-5^pz{iRsgiAal6K zHF^d1C}9J!cJJIKZp4Zn^!+^#D$C-f9BJ>E_~WfH3H( zBcRudKN9l8Pp|cqI>YZ_+a6(@%xdy66wN~e+p#Ig1JzPX zrjw0*O?duXa$|ZyFtj44a70{5toK9lvM4n5s_-aIx-=GT_vmL;1fS=!1!lZ(xDtFv zno!o*gTO=%Ds+fdqwN>n!(|XTxQGMRvM+#f;8j{Z%XwE#Vw|RL?Z~xYpRZd%^`|Mz zuneW^)q@2|qByW{a`OAkApQ-$v+P|+>e5xuX0)gy$sgY!)ZQe;m>KvUD4>9%@-)sQ zR}HeR5Q!G)o8s8;esT#K!Tm6Y&$<#HILpK4%H^NUS2OX?NQouo5Wot4&UFi8h{^|D z*V9c%$ZwPAQ+YR6+ZOLo!7a$r>BRCYjB_Lw1tukyryjho#9Otce6-o6keiCa&(fM= zU<>SouMT?cfzKaH8$&lY6VLx+u9d6jQ#Mv7;R;RSDqn%hHJ3TXKSw(+^zaJNE+&?K z$y+~fZbmQ8hc{>LV^1OnvbVWw0LLg7JOZ(Ga6kY!A)2llHUB=hMSYybXfst<$_i_x z*ys^udEbTUo3;;K?momO!m7ra#~%)GeDF6JHb6^n8y6Ri>u1ISNTh(gnm=oFfWi-i zphQbSB#mg7Qr(qwM^)dI4n2~gAQ`ANBdA~^17_$Y=e<@?3L48c?;OPBgLXN*j{Y`DS6=Iyjx)1h4xb7B)@NaO0gsP0Ke`dH zxb>s46RPu)Hyt)(E?p?aS*+U5JFXy>U{iy3W+22VJ?VAajU3>matr=O`f0O`5Gm@a zf4Y-)RA*Le+tdFN4$O2g77Vw6mmxm;8IH7R-7z1{I3VFFA}G1ef9+L!-Jlkc{tbJ} zs6$?Ot<69@f0y}Uc>thkV`M*1Dmmb6ySmw$X^Buv4lUtbUXY$&$GMRh`fWL^5YFX7 z|7J$n#gya{(PYO!zce^>qC;i*+>Ixc8SU*62=$qH;d%PpwcHfrTB}JnfBO7@V7BiD z=Qs8cQSM^8M(!cN3~$nT;^8`7_1Y4>;$FVqz@q=C#l)igt78EN8POeQ8m9S5)L}^c zpd05x#&7u{7_kwxw`I$61=LblD(}~|!q?i08?XBC2fU7?>*e0F!pvPjQfBptZKdwD z8(;Vxeg9stIzHYwob^Br?OxQ#viS{W8gMaQ8PvOl+O(q19xg?t@cNhp_cs<{Q?Pu` zcv(m3}31T8N>JVBL-ag+weA#mlsCQd_01{+eXi3VeR)WMyARCjfV2%Y$ z0@_!fys~qKcfwo1*N4CA!jUb#)I$Y!mDm9h>2yi@ZySZZhIZ0rasx0~+w$MHyVZ_1 z^>KqOB>pY&!EqJJ&_;DR7|IWbULd|Ub8>`XCiZEliZm5{3cX?1|MfNpKg?sL7R~Cx zVSL%_a)eLXn5Z+QU1;F~qPUR46lU;kg3}%^vhg4r2f3Jsh}X4CPU@MCkXym~o_=m( zFYkW$RPko_G}_%MH`l#tgC9a?+Q(qUli#Lk%4z*^?A?)xXu3Ur%J;N2yO)=Rg(eSUM(;Tk$AB*m z2*byujA~yEPiDqg2?agMzlzaI_|@clI`h!!&H(2B=GCCYPfr;ctM0rs=APn-_+WKJ ziM-KMR@Zx82t1mM#4d{motzZ+tpLwcS$=UtVK8-qGSZlN?Q)LPm6B3yLB{} zg!HtnvZZZc>|fMh_O+So3H4;=m6dv7Dno((I;tJtjBi*hJ%F4Sm8FAXE^0!O<9wE^ z%F!8QE-?cUOs9crO%}6fhkK})`^{c`X%*@dMt}F?T*88j+o;fjD)FCiA`zF)C5=(o$p=0p1_FRRwQ_)0qt&l9ccp z^+xOq7~*@Im7jvatcfBzw?z_nh)9Ag0lV;U0oAVq&YIQzb1g_!dIrLmi;GC3q#+ub z&wdm}U7pK@=RWp%Nq2B%xlG#hkfoDc^Y^jX+s%-zfd9FNi+=n{z!MLB{xFUO9iLe**JmR*ZrP2i^}Sof5RR>O-dqHb+Qej} z+uP5-j}-Ii(vGXiLF(ob8S~e0Potarfr%%r6pVcQNz%Dt791)3mfrd~T{z4tHna?b zcDh)2;>rB+clSKANv()zwZ5J6i;n&Sd4kv(X=%nh5hP+YKdU1zMEuacniwan*ZeJs z|B~Y{si-79LcBIIq3t*2Ccm0@Y&RCxTj{Hr(Q9@P9zSe2vIFI#BI(AHn|~MbpvIBn zy8WK#OVd$qA{oescmd%WeBtO8t$_eF_e;df4x}{8LM=I@t&)y%nUhWR12NRq=WXy4 zJ}SD5%ZY6$x`fuGP^7T9OL1(dqhTum-N{hKL37IFTi-60_*msi$+E1(_PWaG31H_8 zO8HgR^;9!XIppd6@xaFPOYlo;yh|ynGlmu6>;_vd*>n^R7t#|vj|uk9S8Z}_acdo4 z+Cex$s)r!AgP)YN4RBbg45O7p>o%-J&)|lax-b$@Km?ouPviy9JHFuhS(EvLoZz89 zoOrjECt+bU$m>N%nV6J~SIBQ8!r}X_2=qIP@B4aq4pQdm`+gv(ThZ!zj658F&!JuJ zU#X)NM6|6xal8ANMPzXvx|M$M$@bJ@w1EPy51&W&RFm?n`~Z2x?qvwCx5X$Ux_B|Z z{dp?Z<_;8<M8b5;BU|(snY_KoKbD#dD=EJQB#(`9gl~D$ zO4D=d%%#(Nr8l_4!T zB?MY;BDe9Q(!D?)jF`p?#9_KlX_@ozWa zoD$u>?o~7G$gi7Ats6M|K8rkFNIYw$B7tv0e=}>&>@tr4*8CrzYy0@@e@5k+77^n^ zUGz^-bDm-0Z`K5$Wz$zEoE^Fi6Y5yRP%Z>0!$%otm5Rxroy+j^G~$_%x6aE?YL}1}N@EXD1IG>h!djmy@UDpRMkb z2L~-ot}R4?Jcjii!ra^0eh6ZND=}#IvvBnVBLx(nc&?mYIbIuZ!Ss?XlE=@*SN-}Q zqnmSRyJVb2wV`g`5EIJ@TSu!$MM#ZmlD>NGD9!(_61dA;kp8>N-~BYd``jT%h-#v} zSuQoXvUqmT?xco<+P$Q_lk4p5y99yX`N#_OOyBl%SoCm zGhu0t8n%kg>$RsvLC-IQsc;3a+(1YklcSq$`xSbJU!BV1mH@`(5VtHAhK?h2wm$9S zUGwc9gQk3tCg5kpme729pe%@x$9D5aJxVxNJb?kCu#fVZKL}CiRjj;4c9=wZ=sNHW zJw0lNrp%>AB@(`Zw_UPb_?t?e3i&F8!zu3f@A>OoAgeGM@*y|_v zFhJG14_^MxDfSpkV_moK@yK7xcwT>eG}&)7ec-$nQQf&_Fi6R`jshrMFDQpj*;vFL zW+1;=T$`**R(O2n*O1yLC3d~CS8MU*1$jKI_O;M<`n>`<3<$lH{(sD38xn-vQL)CB zBd}7Y3rl+c2NU80Y04|P{+ZVyL@+iB+@{IZbfoFnWl%a{3n=fB?WY~ue&f&ThwPP@ zqm^qiVziGAMH8zCCu=5tVQ&NziRT?ri?GKd+ihypX#?i$NSruByxgfj!LUpRBCKGR z-ztJ5O3u!FmU|E@O^gI2^@w0io_}1eEH^+;@ptB{ZL0QEMT}O=S3!Q?TRCRf$}Mqd zI;tr5JbJwN(N+^QP#hEcVxazAB%O>@6bTeqHfb-@iUfrF_JAX1G^Xm@7OInSd%uVA zI(sGvYlZgyH<5C^J<~W_S9TslicQ+riP$N|2-&}JJ<2d!alw>0b72%-mp~#`g%*>k zr=~eQK%}MTnPC)l_@e}XPqX5=XCpH8|FQShL2-S{+Hisd*AUz#NN{&|65KU};6Z}> z;1b*u++Bma1sL322Zsp^3=A$G=bn4sx9T3r@BaB!y;U%^XV2by*6M!xS-sZk?lIFs z%KH*2NLWmn=FAZFK{9UoPLNecDQ-byRx{r7;F zKQ=>{pHh5H4ZpNEau7~jTaa_EhWvy)j?PcCn1Rv+)vuYfnv0Z{!#y$5CVLP$ilJEdsmx0H07 zW?UXeD}@TToANJgQ$D3m9qUn<5S-EhpPJ<8JrPU~8=`_8efAF+_u8(qcVpLSFurn4 z+KK)>)R{+k9}>LpLsWNQfbF@i-z@1Sc175ZO=EtR<*SZonl(Zld@V{yoFu8~AG^H* z%)$RK3Z-Dyrvls@athi&;BSNc?q)#NS&4;Lt9$rPI6oihJjlEBo61&AAd(DbCSh$~ zCj1Whoa2RKGgx_r6~ojLcjAg<#TmaL1*XQc$qE^;lr~4Wgs26b>C1{HTYJapztDEf z-28gT>=n(8Fl+g|6qi(2K(1p z*T%H!auTzR^22feJ9Ct(SX8KvQ#6lQ!d|g@IdDbxi54K^Q5ot{a)%x`FNOWOVXaA= zJDgEONP%jH(hF&dJhZzzpQ=J2%%vSMB}b1UnA;|p_~0I^y%BIgKv4@`(`cz^kFj+y zv8eF32dsoSuM~SOPzbFxidnYJq^A9D+%K5js z7E^Y!Kt5DDDa49?JXsNvXRGB`)6F%)en!P?o0)M4eGBDuKyd9U-h!-3@W0sTf+A-K zuw=V^p~@`N;st{mbO!7ZQ-WdzngA_Nf9e#>Y#7qe|)^JK|$)i{u%S!S6-=mP~xoTF7 z%}Gwc6V=VWSCiilfpfvTtyD^^;*140kYUO3`x%DO(AN_%eJ7K+!5>J}crGok`{TJf zN!vmBbPk$SC_UXKA1R$e^WK*8Dck7dN}mcRkh{NcY$T={w4tt-M-Cp>d^4a`AEw%Z z*1~us<{m)z>qiaaP!Ge6iVnL>qx;uPIn;W-mh z`T6EjKsW=5+abeTUs-I=ZtXc2gt=2le3c)dKS>?32&%1^hno-};Px$Tt!%7&MJlZp+b6#vU z^Pv!?X`sy)wO-#mMky{-^(Yw{+G>wjBKcSZjEQ6?$713EA(qQHkJz5wh>3$}FW}*C#@P&#>%ZVZqsQXGZXT4J9yc zHTXz*A%_Zm4m_tMCNjfwV(8cb<`(QJ*>F4BLp&YBc8C+X?m^`C@)8>xZIiep;MY}nk*cPQ$3^)Bw*$lB zfgJb#BvpYp4u(JoFVu0{Yx?DY!0x$>5_7M?($5#6Vu$R6)<&((hxsmIx$@qok)ew! zN-A78*}ii(BoAIG=xFgo?k-%iUzeY3YS+r^9M*$UGh)KA*(m~jOlvOR^giV>neUpP zHo>*One4b@{V(={9IpFHxvkW}#;kp|MHA5g4j16h6CCsso!sPIMH@5E!Pzl=zS<(S zJc$18E=YyAQQU}1aYf=YK0f}~=U3Y0ls9L-zi!8%!%!GO_aUFIxef+ayj_52Tnj>c z9IISUFv&k^0Ze6`WH<1GZ?9eR&$}|*rtz&cZ7A15+9vuUMI`6xHW4@Q$2RLp4=JIb z5Oql5*XxUY!H%mDuS7CMB(nAuO-FyUT!c1r4#jhO6}PALY`)uuGf|4ixBrv{3j85{ zdgIKAk~NWxO2CbFBBS~7&6ZAg?f^c|TN5WszM(LV`goE?;7E+{S*{0?(QXvd&)h3*RBlsykimY0wFFOPNB-wP>kiOI>frvx9Xr&PcMjGxylT@AgbN*Ve8-I{}5fztmlc3}V8I zQHqsER*oGwLLp46SY;Z=!&m!`q1UL4+y*@gy*jg!-l7D)lI0#XE;yd0K|*=CF*DIU z*D*y=7p#wlQV>^8z+D@K>-cG6HjgxL6{%B~%KIHP!Y^+bWOijNM}{9ieVc@NNVesR zRTSB$qdF&BeNco+nK2@R)4#gR) zq_tkLQ;UxZ#qmswE1NMmY)p&aYO?OnQ*AQ0o-k)5b)dz#`w+}&d;_u`FdUP)_IOW$ zK}}+TK6KI=PhCZ@_d9Ujqi-DT@;c;85j`EN)EBNHRu?qW%AJEl=RPD|EwR{*wq|0Moz_Mpxlwd?U!c>zT-MI zURzTwI9~bwoXr}Tm(8Uz@fgHs7as)~P<$obK#JDbMqs+8)T?9|sjpd(pDWa1eS915 z>G+-@VrWPm(#Q50Jj5j$;`*);{8Owqy#!Jcam%ArJ913}1jO9;z*Fv65w$o~t_rx> zq}VM{j8Ghgsa~99G|x*JBevNAQL+h=FBpWMx{Ny&vm6$tx!2uH9x0oWSgD?>rIcvk zIz*2xJ&DPbJ5L>#UQK`$ArA}ox2=!$e7a@uBagnrj-MTavzpeGS5B$(?5+VV|N18)nM%71~-kbO-=$Mg%)N~*DRsu`S{AW80gC*5N@(I2F{%fkh#cdB+&CoSA65iCEsQT%qLYF?8~LguVCZY@q4do6mGng15?<`+A~MkLraP z!YEQW#g=7nbmXYVdPk6{OgK3WEj%jTglz?YG2ZbP7q*JzLjC$iM@Zon(I4?@(2jE-fI~o>VB0Fht%H?l8-;Aav>A~P`X96Q zN^kNJJR8#P>+Lk0zYxQ990L_jRJr-i^rB-xb5;(F31Q_kOCzU!p_ z*bQAd0;n!>>8bd zwMr{m97Oo7Q_zExYa5jyt<}d+GvS;8o$@3`d<*)V6;YPDM-R6Hk+n7lpuM+t#kfM; zg3&5)VJItq{w|(f-AgYz=}TK*T_$Z-JjXyu_4gKS73zC!En$A!j`<{bvQ83~uwQ0Y zf%_HLfgDh^E9KC3{hy-#ZiVxm2C<|(RP$U?wz&>QjyilZRRh~}3S$S))90UP8?H3x zU8`;hIfrDzY7*q+cZs7)-IPIt(}gZMN<~)?J9K(cqdJt>MIt|w;-i}P3;U^c9{BV7 zrNMaazjBV)zY}`Pxu27MbiVU!qIa&XyK$y5N&li8k@(N$MtpF(g%0FBg7_E;V;HGv zHl@#Xs*%Ip8WKwE31un}J(g8KZoQy|WIcY-F$>M0ZVV|RPbkt+{=3ZL*=Jr=B(^tWiZ!q!{t zZk4$cBLqN)fmbH}`;V!|>86IXGABL!Agc0Xp@C+1Nnx$xtc-76cQdu}Eb~^zgO2zC z9{^)U5IJko*K`|e>r3q`( z%_t(DcjWFJ(bWeNE%!~ne9!qp&2dsbKvPD+y4*52CNzO)$=n0;s}^v{C)SEei97tY z`yzdYP%$wN(VSr#$!c~!+41+05KibzoOpuy0hm?P0cc}wsgF5vrq)>ZaM_P!xvG>~ zPT!7Y;(a>JHWn!i>|YUAeY{5%e&)>~3+PB)eX}ECY2u=SE^MDh`OSyf$we-g2CYZe zvn*KJQeHYj7&k@Ca9T>k+!&@qOZDoL6>6^uLctaVZ%$tOp1`EC1D)^&g^XMt2I^D# zO-fCs#oMFzc)BErWRght{a6Y1GE!Hh*#Jm|x^d{*+K8rRgO$zc5Z_M}e6DqwkYl3> z(HPky`zXkm9>WutEr@~|_=FL7GlD(bKq}~|`0e2#DAGwB4;3@Xj!l%&ld7nk@SgIO zNtfR4CA$IlCb{)1kO&6+-U=o~d>JxYhCgP*g_vmKQO$i?YzCC-0C@o-L^;Q*a}yn7xlXdG*arK)q}Rj#`?aMYittT%q|X1H`^QCV44QHq-T#BFheSJE$x?y-?}@l;9J}#VRQ{^ z5Zeqm2f6(GI*;-WoqyjO&@|$W231)oXI0s}wf@oWmx!LlxgvbM7UZ`*ckPB7782)s$Nn3(&vyYz60kJ=Hd=KKRJ#$xtdu4KPh{JtOYnp~iXB&Wl+_Ksc5 z=U(llG|P3?mFez**%{a-VYl&o@$2C!2%8bL+S%aKB-Z0|3~wo%lD1u>L;fSPh~r{0 z<#RfrZj%W-+e@?7?d5z?TfG`c+-fY+6VgYIbJSZTo~|E z;6?iCBMez#2I*%#-c75Hj&(!{E-~@p(?>&2yjPU1Ml%+E!BhlMe$*O?=Ia_*BtdyH z{9%tXKLP%oIMu~m*~vV693RO5WkEN(Sk52d>*&sWEZVqTDQ*TE08?%^%azAmkbM?e z|2W@P7P@dzwlGVEa~mHE~_8 zud_ck-TpQeqFN1W`ScE20DE*R@VqG{#bMgmaF@fEYOy4qO=@$*Xn63cwQ6ba_(k?| z*Sa8zXm2>{KVyUc9_{jgC20G&Xweu4n}}ZK{h5Ubi2tXg@*U|Mg&tj^Z}r83Cqkcx*r4MRXmzZZVQ3Lk z@<2~6GP4V#bBN>HdsIkk51~dn%nHCrL}Hd#R+g_xOz>&MC+1UfsqkCxG1Y$ls{CZl!j=+G}WKWC2JZZ?~_)5sE86Q5utZ{CJm1+l#}f zG-az%5&Yc>&d*rNwf&CJ+WPz>=|i1s*RXUH&2?M0YR%6Kw*H9)Kkm%ZXfI(h%yxL! zFBGnaU5@^CI)!;tUrxT?a}E4}0GpmP?IpcxVw5*AKzxO& zpK`KH11}qu`V0FDMJT2n$_ri=hlvb&DehrJ z^-N0x;lqJ2cl(IVgC5S0dy)c&g@@*CCp=m4;6NYtcJpt?<&+$rw(f0w_LOo#zgYYuS zGU*@!*>V$RA43JMVVJY<7pdv!lXz zjH-}t0lpGbtwmVj<$bCDo7pox(6VX-t!wsyc5VEX)25*S2CimeOIRtNOe-zIf^JeRPZyQ`mS` zQ2aD{V$g@w8G>xLLdsuxRu5vKHz7F2!Z@$$UdtitY_K1y7%%D% z+zg)hWgY_#c)T3u6EwUz(N<{zekwwq=%Gv-znXXg+doZu?@>u_PI8lY3bS!=U6RHP z;^@gob}cB(U@z_yuN8fJNOSA9*SIU|@;C`#>;B$L9CIjI)cp}!ipXAI_Rq!mu4iyP zUSg^Wo|MaMfIA{%9PFooiPAvvuXvMHK!8LA$%wnPCX->a`rs}{~kVZjSuy~*TU zzk#!x99WOzdDSCNo{$G9eqX06r?;C`4=Ge`I5Ul(iINQ}`NJ#V#P-F>9@<4B7q^=1 zdt}G3WcP5ltxh>BSY(W>AB)kpbndN@j7sshT$T!%dch=wk?qNNa`0j3G!dq;j zmr-Nny)=krzlD{aeZEX;pIAXJa%O5=VEjbnZNtSTPK)=EVV$xYK=b~Ilo44ob~10_ z_F$>*tx?VzMd$+%|YS-}Jq!(pc? zg^C2T>I6)7c)dEz-QD+ix=5~GhC7WaxyF+9Sivff2Z*21$xPup^T~oA7RjX#ck-4QVJvMQM^#Tuo@&La?^GOOwf0}yo{HP$Gsx!$QlouAObcj8-ieX2et1?k~A|Dp>uxRqz8WPhG zw|>)}-;j|1H131{#T-n$kT-G|-G)5Apg($8j|g-(dJ&3v$sgL^lv3ou@DnR)y$f(K zElb>EyC#bLJ$n1Ogp;qhZ5UE3LV-N;K;K#u_FEDfpQa9Oq%-Uv-@>}5*?$s+#0jxk z>mH`p@KZ?oW*Xy4pM{h>C+}g-ZJpW}nzd0>mLCn3W5LL9)Q3?aVpkEV)7X=hIQ~Y) zVqN}=>%{m)mx71fc4BDQPT+t;w2NZK>_a7q(ocHJY4dkiI(Bin=^qQ+5|d+b*57r0 zQJ87`(BW1uvR^8Sk)K2V5wBpB8+DKwIpX!{CoPB8n4C>WcNdu9X%Ut|vl}}{R zPNp--x{+2SL_KYpo+?ep%HYIHj4K-4!0Kab1Cyw*EnPzW#Y=}+u*EI6>B7fjuu;8( z&K}oSqsIxk>~lZx+v!5XA!gXE@NFsENB^@<8jPT`*-(zuy(6a-40bQ|s!PP>jmw?Y zf#~~(tNKI$^;%3Li5-Cu_Tiwu+B>97Ub`z%;bP{Y!2S&|H}-@CK!OSGE}@iUYX3#Y zlZ-@7lbtw5vg~G-_AfI{P`SgX(~EnFe~#Ild0n$@V;#P{$8x=#=#H9G$)0tI<)JGy zIA7;o-2g79zbA~^&+1zYV#B&sS>Z{<#NN#`TN7;L@EasYE_(Odg7v1kSzxSeOD_IfHo3!OwUc0F^I{l$ZZ3%4|HZx|%S=+xdDl z38q#?l8>j+qNQ#v!K>2Kh9-~4w*Ri-4`fJqg~LtBqB^hXp@ErtP4t@MfH-c?9<&S> zoq#!IbgnR}PxF&B&xep%n>>qE{Rlsn0qf1i1+wnC88VK4&uI>a2JMHwda!U^<+Il+ z2I6@{Zy#)+?H*~rXv;SDO1{(QB2NCy2OTqj)cokO=jt8FvB0PiN?mL}eol{GpbEQ} z4<^54aw9UD@R|ox`#xvffqd5l_?skjOD-O<>xxdik2SCq^@2YQl~#ljcyy?Ac3Q>1 ziQS;ba6F&RBipy@8vquVtZj{-vk;H0Wa`7G8+>{28++BVFBhz0Iu!3OimoZ?XV~$T z!XMd-(X@kFI_e&ji;Fo?A9Q-L64(P4lbSlOLMZ?B8ChSueH?45jCwIsf*S+kGqwJn zxYyZO=HYb>xgokOH%rQ{*Us$aVBn;8de?pvx$|-0*FA|7<{0lFoh_ZbG;!^C7!1TYBwz#cIx^R!lsL-?SsUpikC1;y{p*ffwXhE$V-wD zEQ$^W;dt}PzsZ?26Sh9Sy^jdDYYNim0YtUzPOJeAu)tWXq%XUSFkkW%O{exS=$?;D zw1)lu)=?!0V5!)uoQk}HJ~q-rV2@EW^BH~Sux1+>$y`I@9#U58?%>vkACxX z(F?|AZ~XK6m%S*QThwCql|Ep?cmpukqAatL?Mzqv)x$)A77+JZd#O*B^TyH+J-li5z+=1Q3p!^7Uqg;r*2*;5BaapN8O18rXxSQU zEc~$2_>DvjI+s1Bux!*f-g`Stq26R?TbR`A!Ri;xFJ_OuT2%#*Pe@*gF@umLYSBTn z=^nkqD;V#)o-nO7s_LzQZfSmHivl?NlLRd^K5s0^y&!Ry*q=Z3t9dc zh|f`_Qur*1?iOPN7a7og5#Iutke|;n958tN%udNo!@K-_oPSP8`X6WR^-%Ca&iXz| zn^s_kl{?))iQ(Z*m_=n=_5f77yV1H%Dsf>*!mE$84(d@!Rj#o+GQg?HS(n$tONc#d z$vBjUlaq6$YawiHXaJVGVIM|T-3WOm1h)tDzydr%R0(n_wPG5dc2p=(AodM*9nqGj zRIobR+}xN5S+9>O`LCf2O3poA6y+*}%FgP~2v+jQYiNz`iTD`VCbO?^KS73OI1x;b z^qvbqKMEtvCaCw_S%rULVd`FCAhVWi=gTRp zNmP3}I2*OG#m2?Is|+|;UBH=|OmlqL2GN5k@SeI~d(u(nQoKzrV!)FA7nMBzxwgmO z&$OseULoGJHx#I9c>w7Pd!$5>f2_TZXjCY^Y#;tC;FSt=E z>gA`?A-Fp0xYWfs<&T4hr;aK?NW%DY{8u2=P4H&eUaU&cew$af?TJ@#A73`(_4|1M z_O6PI{AR6a0=_EZyS6LP5|!8=@%n!+*erVubd?b0HDU1pZZP2}oW?DvLQlhWSd2ur zC-zYi5?A(z&VuQT=|d4ImytgH$jKw2%ZF7bsFl?Nv5*y>R!rmBGcj_F-aK$V>sXl& znZR>y$yMuNwA%-kwBU1o#F!1I_UKZBKz@t@xo^!vQ55TRW6-Pdz_pGKM8#OLQc9|hcf_QQ}a$WFds9bd|y;Sg`bx5Ym2h>>CP{TT;5+l-b+Jr{r;(PJYZ_~@ z=dRXw9r%Fl54m&hBkNKWx?WX*Y!GnOoXJ6R{i>I@@{Z9Clf8T|{L#siE@I zofuBPyJ?5mKM7+JMc(LWYPqwKQrl8hvGwv|Dta}``RYtu2VUzSJZUB^FuMG116 zziEX2v#-Z{c#q13gq_U3&HfnOGXcikR9{x?x#P>x$13^VuHPNbh=2L1FL8k};;g7p z)nRxzTIPb1f?_(wGO&x=;O3JH%x_`wT;k&M6$NqXKPO!LyC0Zn#z%N|6Rt!_z(Xe@ z4^Dt`O03-M{|$#!S;7G_C;=iGo3;PEod0hj+^zLiT;h=les}MGSk`}fD~_TXXDRaU zdh##M2N+U{ONbKWI3@l&Sp4_-)2F32_-lqTspBWC00ik0Tx^U*|3zsfY-X#VZ-J@) z;w6aocDXt^mHvp!ftMYdclb59?K)P~f%so=Yy*sYro~k0BwyN!YpREQ`!=V_k@NFS z#7n6FT7*YSH+MHncQ?ND&+rQOAH-bUEiK10&NG+jhPd`g5wOKyy!@{hIM1rTAN=1_ z|MKWa!e6|TT$K^`uU!5An}-H`q!hb1umt@>8vnBE|BzL{FJ$J?!{X7=_dF}DuHdC_ z-%v9}ef2FTvS5#59APCAD>Fu=>G<25cvj3Pfzdd4K(Q3;j>>oK-<~y0$0{)6FNL( zvy1Y+g;4vyfJ3JI&o7&JHIkn1;@mG2OuGC!*y1+209|}>md#%DL=!lFLFy&O-2z`a zcv9P8Ph$5VtWNRu?X!%AJ`XRTPV5l=pT7NXZCixXri7sHR%Sup=~P`HLgiw~VXvT< z1HsTngxJ6G0BM4nKo4&PMMdbV^ZC||$IGTp@yF}4ZOcjw*AC2q3)t&kxpgl79&0i7OtkE;|o`fd-XG{D0r~XR*;_EWKsDDLd zN37zMChk&-e?=GFbmEi{YE#6&A~IJ3%DCz_D!RX-i%ne0IQRG(xW6E>fL63;_^!p_ z{uN#PKluJ#Q~wXX|7CA~6~46kn-_;Ox98bU6?u8ozL!gu5v*BI?2=@08=EptE-u#< zyQb3Tk-LG3W?mTqTI=G0^(W-wQh!8DB#))1GFL-}i@-maElqr@l9h4rNrp1pQ62S< z^k3Yq#MV(Tmd+=nb4D8Qik{u&Ky}SyI~t{uAizgbfbowGLNB7kgs!3HFq*Ex@l{!v^fj)%l56mLo+#Q4^TB1>`98$f<1BaoQqgAebyENgKGAW1y zq)$rh{gX*j8h*U|y#y-vh!zjR1CER+S#==qydG9C{S*ITKXViXYcDvINt|*+bpjnG z`G?^$ApChCz|qrPdht9ck!&jDD*o{+j5d8AhsY_sVIZ zs~qCT`trpi-|8*;s)&;aS9OY&v}!C8j~a?Z>XLo6_>4XtdHwtyhsyUQ^3KD$VU4dc z_JlC5boe1e-=tX#lcY;w*RN9FgmCaO!M79ZP1SUx2PTd-Q-BX4GTWa#ha6M1j0y41 z{qNqeHDj_4@j~x;tH-jm;2j><<5F*m+bSQq^4YAyx(22bRsJs<#bccg^B31=U$Tp8 zuawob2hnOhLW;H=0M2J?cgYZUJs0u#dspu!Yff(jR2AMH=*a8HHh(rTuiNC0`aaytpM1 z>$=1=`exq24olsRX2*Z4>6HZ)Rwm6~YKXwP6Zm@p*)IdyeOXS!|K!X>OXh$OpepI& zqGxO4Gt;!OO;>Jwugg?yx&!VV*Pc|xmcIJzf*zReRfkVz#z?#j2xf+Ygf6-hFmS5M zmmxx1^qxoHHl-dd`vNtz*68bqx3J@#<@vTem)Mi^E~eii#omhsJnbBT;-agu3x5LU zWDcSYG4P7$%y;+VsEC(<_AY2kQ7cQh3TuOLa++tnAL(Q)z{^?ap~uE|+l|ZAyb|hgJ5Mj>i@&QbnV3I(E+2u-V0IbgezQ<3RH)W-T?4 zY~|_>`uW|#-4!N>c>~7m*`&9O%@oTBdaeDNeK3{DcTokiyYIi&nX`<*^)5kOQCT!D z5TLP*?R+-$_w;=sQvPKeM8Kya^m0g3Xd7h2^iJ#6I-XawRlRdMsxJwo$aUV6z693N ztdYJn^%1&$Tp81S4~@nDIN-7{Q%7-m0^v>|Qhv1N%4n}EoL_fc;;S^EFt^+&=x}ht zdLnb$tEO5>JzkhbN-c|~iNo^TIQsBlC?f zrj_h`WM2%SHF;#c%d|dJaaaT(od4x#l&Qb z#=7&1;l8&Sejd>%EnUAGko+dO+d~!t5-RSgX6QnijVTf7NLG(F745+ zu3RE8->}A*olGfF8%jvC8l??oHOKIseYN-xdMt|B$7%c{H?3lERbMAS0J*Lg18p4E zdOdbY)N{UrZ^~Go?Qg%+xd{`KRNq4G54TI)dd1}~4gr|PPKj5faunny#GKYe0$s} z!yQPh#pL0BZLbnWqOyjEzVo5_=I(v9r|l50$b{VObCC-8Ggz_f6Kz{+Gm7hiNMSOP zO4`>wp^%GEv3A>XW!DL{)-LenkdzRtJ0IL(Ui717q8M>cwAz~z;MDzKapJlWP$9_b zxya}4MgMwl1BcLSo%oYq^iMxd<+X|rV>PhsG;*jCP4J}D+cgJCdlS#xj)QWn9q`X; zg27;@)X7199uG;q7>j}XXC>Qtg4^|P&<)W(xsrY%uvW9!hrY2+i0+;2(tJ~EtZbB3 zF05#^UbHWH?NjPJf$U)bMPY_C5;PK@MO{`lG)+rWGb7sRy*AAsypb1h4frJrloYPv zZnoD@jrhIc7Z6CBlE@Wfp7yG>3 z6xZ%v?L};~Su|?v=P!Dvei<7jDRo=7hvL9_%2m58oYA1`0R-t)e1CsEIFbkov;1=t%K*< z=o$TucZ(d!Pthvzbun%~4}WrX>!-*#&&`hW`8#`M)h40$D|gqS1J%qtFQ0sLx}U9r zX09-|O38y#%<;R+*Jm$9N(40f2TTVHus)NfAMm@vddq-H`bQ72hV5+ClOybRa$z}~ zbDx38!~(-{4CGeo>)~dMj765C2{nZ%-a{JJ44l-w(7v`iOcm>p3NStT=E>h@s^!Pa zHFQAM%-Y4En6IL@c1Q8c7!Y*i(qT48sp&>p?QdQJl~|V z){XHaIdq(TBQ4SQnvl!HD-vH2b||D@XPx>p41IylqS;GBfo!!w!9mbowPs#liI1z` zk=9$@*V(LZrpU5m4rkpuSJ4*vUakY&A$0Skk?Nbd%YCtrg01-eM%1T$aE(&7-IDOD&9{P&@hX z!F_#BLZwtWLoj!ktLHc&!JteF+<3GoV5DPNRIYW(lI3fPFs#|`oh@3olAchsF7mr= z=y&q|Q|+img>RNPOLE$jfze9Vlh<6sEAuSsnX2u8Picy6a2Q)C|Bkl12^1+G#n&&` z0YLQ3lLf}LE~)cWd)21LjfJbQ&{u0Lp?i!Ye7x;M4V~}3jn~Vh?7SI^RrmmutYo}W zzZ|rbeFEQRS;^_#5UKsjk$uJVi3@6>?2J^Q$Ag63cpWXvpDPamd0@HArZDx+{S~+q$&HxLIVkYhA~Zjnf-q*iYx-`J^k`#tvi>Yn$^zlB;zHbL_>$l| z3HomX9NdQLx{S~VVNk$5fF)&pEpbu&F}aJt`XVsQwV+@p7Ekvx5?GF_j$W+;QGopdti zd#;m)dDI81vwkQW`tYp2vYp!DHkx-XA8sO~O2$Ivt7*@3@K&nC=GP z2Sq!Jlf4|)zKCco&WyfvfY{ngBmbZfb)mPMxSw>d&?&U|?0*vw|FUBPsSz=H&L!|5`+x%SN(KaH^N)@ zULOMiT5>jDv!&I+ zd5B@$p5^MueD|x0S)`!y9r+xuH~-~-`M#n@9uiLSWGX2q3q?0d9p^bhVP@{H=Eh;2 z;y%euQT9>;HlrW(Y)JGAl*!ktF|gR?SnuS02dwtOjbq5qiUx0nI5X|1v^$R$b6Z^# zd)_-I1`t6-Yc!8ebJZKbvr{XwLUB(D9~tg9J1ufNqaTI{&0SGP#F$iUo}SCPwyu|{ zHN~wzjJv(UJoMo7neG@q*%Na6*g(|lx1Sc)uuj}3I-s8FYlgS-ab3}a@@d3G!J+-f zcXH_SR)tn7btV{58wtB8Y55|W* zD-d_3U$0$F733--oa>_(4>j=^M7P1Qwyp%rqR;QfuRB%VNv~x2Y85_XQi@N3lTTIN za$I_?J7eXY>s{qS@X0{=TSjq8fT)VCM#t2qPa1EHSt@~bum2h9x>7>nF5*a%0_8@x zGTxlTzjV$6q)}#>!UF@t8Cu zEH9Oyq0HLi`<;tJ^z7Ef(Bj}X{qXZ>`=dnP2#5Y5m4*wz_VlHy&8)ajK;7i!H0UGAlK7>tIGdP*AUEv_$Gh)t!gqNw&CXxj zkcZIeo-nwQe0EZ9c!gj+n0CvrMUi|H1g|=^7K-I4#x{d`$CkHI+K4E}Nl*gc?C6={ zQO1d7TSkjN`W$5^nmxDfwFPG2zn

    $UQHggEG);a1C;LUP0#_VcL`cP&kCU{dsU) z1w5=%l&NHks<~E*KbnBE&t5_2FbpU-;wY@(ZVp-+ubPoQs9ddqMAj08Y~)X(zoqR0 zG)wl4-D3b#>X~hgbA$ckjv{(Qd)Inuu>i#o6276=ZerpvK6u@^Mz8t(^c!%dJA$Tr zd&ww|Uw%b_?%gGss>0A$t?wNg>dWOyvZ0)9jj_7go}RiiW7;eRvLx`WhPmlY+$*xo z+Pmg-P!G(5>@3`5vg*@?!sz8e?xCiO!{1rh)SZ?IuEJopsTD98I0qv7H$E2j^Dd5$;rlPMFA#`bC{6VA zN_lDI_$r=955bLlv8?B9!qz!OW#uY2@|)#XQ0?Z2<-L8ptfAtKQw&%qx6{*Q2iu(>EebcQp^w zcOLPFO90y(OF;7FiHAi0U|GBKqyb5WfSD`?;rIM^d280#>S=c}EtXCAqZwuUS1lXo zl|$1N{hq_e$LK{P4d%Bm#!zBj9{N?+_xaFg8UqVo~al#-+@*(E1g=zFTdMC^EHtEX6aeZoQVXo-#U9 z$iyAv2!y(4iwpF2tDgB?ibe|h1e8ch!{z37T`>OWOyGV{cEkV2-FwD0xomx)TSQT+ihy+3*pVgz(m@224$`ZD)Bq7e z4-ip`^d?e5l}_lrC4fkg-g^>iBE1s=B;4$C?%D4?zw@5`-p}{L^XVz`%&eI;Yu5U& z{~C7I5wX(G$j2UgT3{7nB5%*XwPD0DXTJkuYplLMl-_>A?~y8sjX9<`yEj{BmAws? z>cx)n^Yx`A7>!Mc?RG8@BWIx@veEwUO7jnC2q=}Bi}~7e$JjcD)50@01uRq>bYC|6 z+*SHJ><8e#93Zna{s@jPSz|#DV|V3aDBj8aiup+R9R6Cy9$ij>ce@eHa zPHkF4vTc_)kuFMUJHDT;T7Zkro)$|K4h(J4UPsNjZxy5gBzqXfAJmSO*5eFv+jnHA zOEs=%LX=_)yoq9_j%$;to@DD!{Z3ENp;Ro>x*5Xd~2T_HlNvS zn%;xG9|P zu=7!ozxfGh(F`%?_iT()%r-b^3eJme zsX9)X)rZmrJ56$eHx%I~M4?c*sxl$agoT4|dgkc@gYW}C`|_h`c<8S#cdw*FoI2Nq z{wprD@l@2El0No(Q(HyXR}R#A70fbw=sGqs-P8GH356(^1g8PRtn@heh z`{Mj@E4clwr0s7hjv=F>+J-;e(h`Nc`#!t1ME;DZwMz3>cyVNDp`;(8a=abkogi;- z#MmxDbElfiFoBx>RheMmyZ2EjQJ&w_BELvL>+%8^@z1OE86Jx1RINW0BNl`?gO&!* z$U!_xLD1Ve10GI?JQ)Y$Kl3S$pF`+29cOLKB)3%i;p{j0zi%30So&$iHAzXPC^uZZ zYiiR|ofYTVWq9}97t&vfLnkq?)|>>d{-ICe?n?S!SA-M_lpph-3p?tGEYUnih`xK5 ztF(8W5Ghf9pj!>dRRF6V9>;PPrB@F;WQ%LP^DUEp)qVozv+jt%cPLlxPD~Ok>;#T{ zk79=Hv6V;>YnuPE328|Nl5j(SzOQ_>b1Co}&y~qOQP52eh@sPlbJAPoh~9MYlV5bt z7W$Y%L<(O3Tbkr8z?(y1RIAHr3PjVx*!mC^I=b?Ne;&I+#9t=k;2uo~p@#Yk^I*-I z32Kv_Or!Z`pbnJR13V~Vx!Mp@kW0(r4VH!MGS&H%nkzjIais47fAQ*g7);%hE}<5| z1kD~KNIUmQ12=xA&@-gcG=bZ8%!TCND0D$Z2~&2Z6W`}=542i(8}3X#85{FU$th?Q zi4SZPY7>gA?Tohx860E{VnlNC1srNKMibUP{#zsS+e_gs1xf{_uj_~d&M zuL8KUu5H`kg^SM%)tCN8B-SORWVGRuGupU?fUjOLOaB-UQus7zaFS0d7Uv`^COXfgvo)T^Y%FD+%p8DZngfT|*<((`Ev4T_jlkq{ zV#yrOf3I}b==brO`srq5Yr2VF>g(&Tt$vpe>&Bacs=ssX3{Y-A^?NphN$mld(%U+Q zYQO1lVf#5sf2D7{!w35^%IuTm7bc*Pm~9ZU?c0+9vy1}y6k-jsAJDp$A|)KuZ2jyS z?DG>4sI6*Y;8L6{o$Y7$@}XDD%aSfedm4OqK$?clxi@Wtf>YWp$!qv_t|FOEgR_|< zm;i=5WeC<={dVqsFAkW^IQNT2J}aw<6B{3yPWe=d;9S zeG3pMRXp{}sQzU4FVho~(7qmKSfw~7X7f z6}-8Ixvq?kjy|*L{Sq{_bxyv|;D3ZmL{CjT(g#1*^ZS))OWad2`+96+=I>fnAE%5) ze)FUw3JKA;ECjavK7b8ap^m03GOQzq9#2S3Ub~`aTXr(e>#;22$$56ITPyzQRaDs%NE@T?m!5K|ISgRje=smB zhvv=Jh@Wsxhp)21B(*ShFFaN}Mz!X5uLce;Cu{Zzcdbb~%Z>X2s1nz9S2R44*I0V6 zpRoPwCL4zmMXFZGnpmW3e=8eHh!Q22-4g-94ujATvpC3_c$gk?uIT z*$#W~2EF(Xc2TcHTe@|Y!b(DJ%R_X3x-^OL)Zd+UZin)52el_=bv*;io-Ce^txWNg ztg*>Aq@Ty8E}$y&?#o*CgU1(^t`+4SC|b>>B_C{;jjcSOR0DUs>y1!rkd(O_OB2np zz~AvQKTu2TC6^Ped%{OR=T-~GY)kl!`R(LK0x$RzPI2`t&zHk!SY~(x zEnd8W`usO=*m z%;9Y~)|`^D$DmxZ8<@G4@u@vE01Fnb^?fV06aKLPQf4CNo10*hU%zn0Ha(@~JlV&E z?@?(URsV(cm2L!|zrrN?tmCRss9!F}zHPtn)cFo)OO-a5HffGeFs5|kjP+CEwmon{ zIoNmGV|m$dQylXNHgD;(!huIfhu5ucq>si{Vm~R0XEcV)luP7Q(GL>hCz_mL4H_}! zhF-skPm-^``To(ZnR3aiZR>-(*XnTfoWfnH+~|VC zI*Y5Eec`pv1=m+dc{keH*~iTOm*Iz>SX1n5nnhf8b(QIMhXuzO! z)Ym@|5am9-qx_lUR(vxJ!&B-eGnHLe+_|YbO=C6p^O{!PLU)cnhWnvIcAQcnM^F6h z&&Dld9zs0P!%>b$k1L(^Uko-gfA#zRTX3{trlqgT5Y6#CZB4D|YG90yYAtn8<+i6H zTz}w@a*TAfZ;BnVP_q?_YMNq=dQ+P9eLlUAafpr|7*OQn zJd1y59zWFIWE6QMO91gfmxx$lH?~6+cNjVUg zT*pTaKhyskV2fN7IpY(XGuq=bie zJ^#rXKm2rKZ~c^Lp_io=wLLRZuh7}5TS0($Z0wtDlYO6hw#%{qbL;7sIW#o`o8f%N z$OEVk24r^7-`wGO&uct(o|Pf(walofZO`d^kHTa(Ymai7X4a)zZyl!)7$^NxQ$pke zeSC;FE%{fXmAd0h)w_Bnb$9Zye1rG7mQBdOj2y@NC8)QU&9c|6T7ZVNAr+Ms{AT9< zZRfFg#f=WKi(xH3sSo|Bm8N}4wi1sQ!RwT>Ipnyq-C^cq@m0G6lXIW+adr@hB;Fk{fu#_=e5 zT2vYKg5*3NS-)-@bfr7&{q3|wX*F@e*<<0p{>$h*n3thj_Z+cc+q03(3|!#-(F{D7 zD%0eft-QkyWZH6Uwq6O^9r{$IfCbg}U6V?M(%3?wG-+?@&e(L+h6viW{I~^V!b~+I zba8X6pN6)KcXr(OXznLL0@ zo0Jld5M_i#vaPW3Dg?c|+xn5C-r@*udp7Hg1lr<6nM)j%TT3|lMp$@ei_{QW4>vNA zu%FQ!$8q6rK2fg3ILuWG6d$TpNb0pmgJKs<&uimanLbO zqI`u3Avf3ki{M5n=073wp$Zb^%5*ETvv9p~PWM4IU=sc59dqok7hv!43(euwuS=9I z=T{m+^l@b|Qs-VLLaO0jfU^VqY#oV{0VmPa90I)K&CXIBOc84|rp6^Jp~KZWc<5(w z<6$Rd3042_d?4)@(aUtvvd5Ejb{eg~wSP*syT2Pb*(@;*+Lz*tHzzAO<~i zoszA&%&BAN=Gx}>z-e`HOV%@sra=ofY$<{u(jQ4KC^`D-x45}BqEe1zlIQq8AGUs)P{o2tC&@_N*0cvDpTSYtjWzRA9&GsTZQ2zP@7l}$< zDwBhD7bI`Zqi~&Qh=&46ee~0C$^@OWLZ@ISFE{UZ`iix*(0Q%^hT>vhnlsm9dWG2i z4P9*=KUB11aEOzVV|FSOjTUEM3jpz}!niYlbV8{bjCp=|q#^v|OnT6AhHxdytfieg zjI$Ktm$DWpDUnOzvbj9BBTDiRF_6tp)NuCIt%bDXX3G1mD~M%Tqc@4oDA1Q1=&{P# z-SsalQ5rWI=UK?7JTD2&I;lU3`+Tr1t=p(T`1*oQ1^W`R@h&@4KO>IANI>Bq_RBH|fE1b;Iq#B+hrfUSp%QeP%7rGrD3>deh&Z ze4qYmWD+QLXFRQv1v;X{aMSU2(|SWR=g_cMKDFDGmV3=f_9x?s*-#2|#mrXUf1KAw zkvuXTEG{iTV4jEio20(cFSa4y5#7|m>?K};!*#Uuz~7j8UsCB#Nrl19a^E+#^_#Gx zhEUp8WZ)ibWcm|OQ?W%f^yZzW`ERa`(#7byS-x9yn3w6eSV!n$e*|u84DAUVsVkk+ zYY=dH0_yHE8p{ymd1%NVH_ISO(FnI-YcGR4sB`v3=tDX7BR)3UsEuJ5atLBHA9YARk_l$BgwX^3uxD)4lq?|S;WBaHQ{jB}{FVm6icF)c9oEDQh zljW?BHXtxbNasgNz8f>rU0TyinG4^Zl;ddz)i}c8regrwgTQ zD1Wl4oMy8!xha`f(pUH0{6oH>e^?pV;V41Eear2dX6v7ozsOdX8|kmjp+7a*^E>-&lethwomsma=Vb>> zLq0G&2zWZcj@Kvpe~&=?LlR*O&vAh*O1;%?pZSFe8Da$A_z(me$Gfyi$OSg1sT>je_>bzc<_>MNpYR@$UwXe#}k{gOk!wPJ&V<#sU{%{t5VpVZ_QHHy_qP6lVk zk1+0f$*7-WiE})&?w4*pX7SJ?n@cM^Fh8!djmapwJX!1T^_DPA+Y*B?PsZh;Iy{R9 z+E;m_>hNXz(Dv0zHS3^*H0Ka;i#r}DFOc;Bjdjh0D;rjXlju_+ z3T>V=6)TgQGmuMRUbK4ObB(FX?cnLo`0dV(`}UCsmouadu24}t>SZY8J7BVJW=?Bw zu33Yv34`D~jz3xk3D7f+-WuQew|s7X{%QeMq(Zl!4?A~8&RLay`q-LbhdfYKwJmS6 zUtFyzw1Z$9uRA;z_#2Qkmn9j`PHLeGGqa<e_n<(SU)i_(NWl~#@(SHSy5VM z=O&>-6+oXGkQ=(4o_%pvl){B-)?Jfawp;h=P^0rbXj({-LN+Z{Dc$Vx`XlxR8?{uO zsf^Jz)h+qN$r&a{?H^y(a^z`d?e*^Bmxwg(w43{0&E~YK07*;uu9Cq^jxEYDU6ocnsX%l z5rU(45&(Y0Yh2L2%rq?kk1@hvf^3LT`eI3aZ#o-T{lQh?v_?r=0U=C>LVvYw>1fq_ z8FzOXGU7+=%mivS1Gis4aE*$dUCce%9C^Qiot?YEgI(C8WD=vPZG4%z3MKJC5{q5W zI2xo=E5?)=vo2yH+?@MZ^UgBpkjL4zfC)5bLnwr!1uNB2;}blAspy=a7WChJDK60t zjhUi85a(PYq$P#MjK1~EbixF4+F^p}EanvkNIvxIt)($78*DG}n?D({pv#KaSEMDA z?8l|E-`MmUnrFog)=0mYmhswxk@_?|J5>Ow*~uAZ*`@qDL-zLy?>#Fsj|n}jXl#;; z5a`E6XxTAlm*FvS*P30#(hWu>xxTKdKIC1^CcPUzJ7DaRVP;pC{wrOR2=)>g7hh4o zk)FmjIo8rUpS3_+1&=zoDDbcojyXG83)EAb+dtAM64ukynF^@im`z45`!_@OyY%74 zPF@#>r2q#^80UKO17d=h`Lj7TDmHt8+|D6;_B*`>AbfY?25SAHX?V6or9aTvn(~Bn2zV z5kx8BV0}TEe1pV(BfDyV@Vdcn%R|0r`rk^$-=iST98U8 zt=3rt*Z{UN4TBYCT~plB!d=a>fdM%rsc${J{<0wL-HHY0h;}$ zu=m}+Tm@Gf%lg)(4E6b2HWxTnISjoTg5OO^8H48}FNh|e+d#YOlOiDgJ zVaF~O`9+{K|A~RcE`EK4hgY;waq}RY6<_C0iWn)$Cf9m@#RsS)c!(#Z{9Nh*JA2dokX0?mNJmFd8e5P0nNP27Nc__>KDJkcJa>C#bikj1} zeTUny)-i*m78ZS_={{k*!&A7dE(wJ29d!-iBznRsfpDpfSFb>o7*#K@-pok+Cw}Yh zcgekS$C>QrkmEk;Z7q-Cet}j=_W=9~Elcv;5K}LCedPHQ9lDci9Y}~)gAw?9cV>*K zRe$Xs`94ye%OZhefASJi!MnR8WJld@y3C3`B>Fm)!j@&*C!0p_GBnmDIc3AQZX#hG znIqX?=Qp(JQ{8d7$NrLGOZMB%t$A)2zuF_`L?NF_{m6shFO#X$-TN*#vz2SgOAIff z#2MsE*m)3Gr)u_2D99lrfH9!_We`DOF0J8UdnUd>8}xz)e=XFjlrm&{&=W68g{yN!!m2mR-aE;iAzC>cOvr;T2HO<{SP0&)Csx*v1wzfwZ+ zciFgEE@W7tSe${B^cyc+%L_A;`pFREEw)_>cKKE>E0Zh{=!^$t(FAvTCQkBNglI&V=ceNu6mJB@A+#22XIr6CqX^d zONDn3{F;qBStMDAY;*4bHTS;aS?DKoL~@G;nP!A*EEiuvUNJfJZ0?%Z9uIEgvr4^4 zUb6CnWbX1lb%E1MW#)&+2v@08%W|>uJ4LtX-$lEUr;LzQcq_EOnltL&i=oN-10gmf z{xQHY{8*p69r7^J8z#Jzxzci2997D>k4%qt1{@nL+`AV-zua z!;wAZmS*$P5%iJd8w+hyvlARqv%3)rZ*9sJd)zR`z#E$>__ifS%;zJx=WGLa+%%Kg z(1tvatn#d51R56Zjw~Ys{fe2B7KL{{UybXloWj`SF)g3PPQEYzM}7%`mL7II?lUWY zHRThCSHmCpHj^Q`-9Q~fA9h&-UWkPe9v@=3hQ7{Vx}tjW7tstyi<|+G(;ir}zgc>P zQEhm-6~LQ~)a`@?IpR2lWQ~I+4cS)iyIL2d68mVG7OuoSxN8~FwJFjuu|vgNhQKHw zIOm$x1-5xECPg3Pb9{%ux@ zlA~CrBJbk$PZ#w&G7~r78E+`tZ^4A&2S!O?Pr%}45hW|Ttd@%7@wI^raYdthcor#F z?YD5;>-A?0D=}Xm{KLq&>M|%kn=aY#LcZH{MyS3oBkyByBXM^ax$HSvOtxBc4rT0b zQ65sxRh#&1dJ_I_J%4ap#y2pZ$PRzcAE4Fqu^hh1$<$Oa_Rw9K|7c*Ecz_xVRh;tUR+XaU|R0SqH=bmQ_{|B$&tw=arFKKz-9pJzs)2Z z3h&N|uUeJ}nD-)#=MF477xnoF2=K-gTD<_j1>gkgrHHm+!gc^!#^lWl3TErH%%$tNf5VQzfb@Df#>n=&E zj&_w5JK~dhi-4X_>Hh#AXKm3kB&zdp{1bN}k1)6Ied@^7(!8BYo5&b7XTq`zE9>9T zCCGgzKaFfrNN>u`j1gaDlGRB*S!M?DPk`j$A=Q+_zF~kq*l?~lDo=X2+h?mG&^4qLqr1K*q zBl9_Xc8Eho>oe5_YXtVjZX8p6s9#5mV3D3PIm;Waoiad_iBtZSY0~w|>7D7a@v~|6 z@G14-bj%V_LK@|JWB<3GGfkrjQNx7O5 zj(yC908-LOx@o8muEJ+rL&{yhUir=S+OvE)gzIOWz8|E8@nAAx^lwgjS&LM6NqE*&$1D4iC=xd+ zq4mt;c*#@8Eqv_}`Zr(b88)u|1-=Dx5mL4uU&hZq+qceq(fl##u#H}GZRq0Y!zqc^ zowY~od`yYv=I6UwWgg2{?b{IxV2}ZWz1Hu|&nY4>%w!^r4~?bemzl1Fjtpfqt58vR zl)8#<{%vB-GsZs#!A?!no%ai5CTM?|k*IPnFAp_E!_cu#gGQK1mL?N?uDChGNM z-jd1nnPR%Pc>U@Xe9n9nN_qd!oMDnyvo|HtgrJ2cRz?{Z7_RU1qGG|_a|K}>d;i}^ zLc(rWkl}lWWJ@#UWa1d|p8U;F#Dy7vE%AATl& zX=uc{39gBRpurR|KRls+Ru~`+xe_NeFn7R#Ay}`>R&T`bpV> zU+iw$CSJHyZ2`h*z=XgHScb*+{B5Wb^si6;V++W-N&1G^w**DKzkYw={F)zd%}pA9 zP5|L4?+eMllsdHGt0_OG{|83hiGZI2b$bVHraQmr~9 zpgt?I-Y4z()6Zsu{xxy_gqlaaB7w1QravhT|LZNRGS@1Ftb{BDrm&XLe&$f zU?aE|MRhN>+{=pgPc8Om-zKiz=rOqbWA0yl{O5Eaja$prKbrH&nf&)VB;(;;V4Ro* ztQm3~`;S~vXBC5(ap zJlcL9DRf-S`>fi_D=!sG*K+(-AIeI0K|}(lp}W&TF81U&@U$ckB&XXMhmHIMtu~@&^FN&@xt=V9v z@%T?j66mN6Ei0M7DeFRszk=eow8E9!yF!T9tiii?GTX#5(fLsnv_4{k&vN)~NA_rk zQW!*t(w5T{hid2O2l!$ zk`X)`sIgJVq(nWg(0uGTXn~G2jk^PwN{+Ke%xv`u)h(2E{l05a-ddUo{mpAbjAH$( z20N>2ZL8Li-`4vuT!=3qsM)Yj@zGC~VZ07u?NdzuHE(B<4R0yZh3oB`@Y^=MCw*R_ ztH)4%#ZDR^=Le~y}7G*{Km$Sb|vZ|5>_ zxvBmv9`%SYf96N1FlkZfAdFBy(lfQ=Y)mu_}m~IQdSh^`ENrNSJdCYmt|5 zD0B}9E-m)O_?qYOI}1D$W8v#`W55jEZlK$B02()~R4*YK1U91lHmTVIJVEeRB z!^}mg@w7X=ev07LMvvLKwMx1M-nhGNc;5O{K7o(CK&9n`2{8Hsa~+W18WGSLJ?p1U z>%6wTjhNR1H9PhxVjnCVWx5~qKMOmyf^=<7Nr}1z(mxnq`f4)1sg9=6wF4iGrubje-+ZdKJ>au!!Ix%9{!FnR2a7oame+D>7EpLr1}=> z5L!pg>e+k@yx8D!!01*p0|VJ`jQAV-&lFAPUK{U_Pi_Gke;m35^qb*6>!!>oZX`&?4i$;fTlrS8va^YKuJ@QTjh8`cl6HfgH zwfrTQRl}ssd3gC&_w;T8&PNO717oFi9s()$FYvwS!+Qrn-yd4U!%KW-h&My}DLb97 zF5G97MbQxj#szCJUbL^;YWEw7HI6yk51X|pAcCezooAst5|HEb?p3t;SxFPe#yQ~^ z--zee$>gCOG1gw})Tz-{J;qK1#J*EeX1!B;5&pIAu+`IHyqhvp-P{P|5q0GACNZ3s ze|V>xZWs)CnwcmtEdu=|M;y3F%>f{sh`Sa8RjxyGEXRG_#WRnO=Y$LT$Nl}A)|S3h zY(3Vn$Cjx#S^muIK)pKeGdHkXU!qr>#oW0$EujLbCjM$xoJ}}U@Kr4-mm^*lHk&vC zh^~PSB-g~s8%zNq^CcFoaLqAuU!)|TYRkau*D9l#ZfAgn)wqD~#Fvm0vp~F=2I7%K ziFq2vF1H-^Vy+LmF^6fW$H7rSwR;?VPR(N&Y}#Xx_BiJ1;qFec#JSoXeV7v7t(y7N zFNk?S?vg2eW0Nlc#cl?3#{q z7fI#vp>I+T(ZA8A(z05BF+Rcu=MFpQFZ-eB;&O{wnXCmrtEevfs9t-435tEoOg*sj zMa~h^r}lLIsRqafEW#uN&OFpN>Q!^730zZtP}=JL^PD3dOdCK~)Pc@kav1ZO&&O** zigUaDF3Nntt+Gd78u#w))5*Q|@o{t^0{QGHXhzr0&+mxTLo(v=KE032RSMLUGuxf> z29#m$5zNe+#|XH`QC{sC`zQ`MLmwK zC${a;A5cVlGXe%h0WKqb{hUvhj8Mo14e5wdiv&MXN$e!Fjco&R;F$8@`7tm2a3yOy zS_Hf9P(-Y7PuPA2ff#xA-H7UY>eby1PmVp@b3Tom3D`97g`O&3bL&HonYAKI6K$b3 z^@j=OsO;-{uYH}*y~bg6komCYfGAK$K}7h`cjXJ}$Wbjb-uV8jbILT)r&Je3USjL2 z0vx4fcXrOCCH?ivA=Sr_g|H2-Mb<<_{vzyRks$SwuzK-{=}XmG#f#nY-dpe1iJWhS_oPR+5&L%CSI7yi6!7`-yYS`h5T3u0B_0{J1G6)W<#4Yio1; z5%9|X1pfon@a`&v4)68+R0eYjHLczda5Ku>pwq-`Xn^SF0L!fmxYj_Ai%tJB237oE zq@RW#VB?hVi?>sz6@z6zz_U+j$+RAOsUJjR!0uVbSIP)4&88~fZQ&xL8n2dSioh&N zGH-?ka9^Zrl2QZV9#(aRA{RHKj*r7}Np@r_Z!oqGvFcd`v@Q8mwtTAnmw3Lqr4xxO zFU`{3UbF`6a;OJPwQ8fUONbx&))t>=UEWSDaK4Evo$Dp-KO6e6=eW97&f=MKgDKO3 z*91pY2N6jFc_urin+muCEE)>wK%@^tIyNDkfa8N0v+l0_=bMd8 zwG-#GKqU34U!)+4<!jnOXl-Oggf}vYP^AM&fF1YP@T>(1;h$ z=ZAz+PS$2i1CICm9EILzxnm-Wsg}2z9t=RS>?oA4+0n_e1Zc|a*u*mC*zS>&$8=w1 zdfD(2@R)`q9;bxQ0%)W?qB?>Xum>t3fXsOr2yb(nCNcT3pGBjo@68@zMj;6NRd*{z?P?dHA$q+g79x zOL17BF6p5hcA>GtjnP=)9rO^qeV&aFZS`ijut{GVaBvY7gs_P_kyS_d^HH7S#?9%k z0_(rO9pia#RTjPdWcWDeXOsUhUsELK_+^dbgp2wE8p1vhQg6{k@V2uxAb2j&#MO(z z-)N*?L*4ksh)FB=RisyYcsROsd7^J#o88bj+2Qu)hHym{A~S6~GoaqwD_P^&rvaKv ztiR9SsK9DM#*ZdBV`lSVOM?hsFN!H`^&yp#wxa&?J-;m?-9SYm!>MC)#IuLe>SbX! z{@Cn@(T&-5G5Yz*Tx?G*n2R&y>$Qe)+#U1>+( zbfQlmSYomYjvxU2ivhYzC|T{|HoIa)UUq_V;HZU+BN2HKPJI4je%OdZ?US+pnyN() zKy)PVyuw`CU2it8RwgvG|GazXb{E13zkT7S)w(^?4vu`2xji*>vD=#3dcd=@K@^J_ z{|=-ZuVp$@;sh@TKRB#Kck-7{CQf*qLa(oss~3&0cgl_?rxWALILZ@bn@Oqg7vsO0 zx2J?_W#z`DH+W;~1C-77wgZ54_;RRujF{=9JZCCL!^VK$m|Z`3kDpInpjQZkBM0AZ zkU}l;{EG`9mav&w-A_}_-Ab>UJ}yhgKF;YGkYz+PA8&11Lq99B8z?Jy+mHZo(J5Y} zGhRgNd75U*D z`OVvvzf<(voft%C>>Vpj>Z%DTThx?a><$&{eOQY#Z8;M`;`PHyE{y_rW=$Y$jzqI4 zaOF_Uq3QND#DpH4R&$JA#LHXU7u`faksk?DPH9ifF&d zC+PvF4n+x#q9xp07t6Bl81#ji-FC~E$pD7_{zHdO%C%8dJ^W&*b}`2@DuRFB3uRTQ zS>mj$yIt5$2(&%@=>S^K@fc*jE7_bc_I38?^kPPme)nY(!lI!4G)6A*UKtcmg>n5xG9XNU$^r86jxYx%WcS(iKUwErG6WVf z^}we66n-2*-FWrqOF=opyC^;X_e$#JF-nQc=2G_CW-iJ&fLiR_sl|P!T_k?h*|EzAfPrngt8AWHgQFhsO z_Jq6!xTeAS?exrG`CQDCnG@o0IW!Ewt2)#6%9o6n@yUzgo@Hlw*pK2qm^p>3`Bd!CcFKj=uk`=QEGNPZE*y@i%{}pT&;P-Wlqo1AHZzcNXUa7#G{^89*5)(=ER!6q%GOTW7Y#I@M`+1_heS7ti{!?6?s6^dW{g|ru)0bmPHQjCAYk)zm_ zN*E(v#KxUf9iz%OcL4;GUWA)e9~V%oSR%mg}m15+xjr@{2nk@zHw>d1%B_FF7DlD#X&T& z_zL6a1w`S1Q4l5f^M_%?$lW0|6!)6flIN|7A;E2nMGu+;O-pNUc_9?M>DK7+VdN7K zLK{Y#`U(;*+aZD;UVFTYp7_?2P0q3HT>Q{;o?V;T$KT?dYQ=wG)$N^34l4W1>#e#{ zUH>O5E#b{`wXSh#$x2C+sAi9m;*^#GKt#beqotxAX~tg3+m%mS$3exfx}$l(rHxD1 zrw>#Tl?yA!XeQtS=4w`*5kK1)+4p7>+2>o6&rmo&IrjRV6HnbPv6P4!eP>;eW=hUx ztaz_>O3te}<5~^Mt-AUOe14gt_z^yyv4^SU;!ovEtyC`J7kiY7SgWPVkwXWkW7B03={dY&g%sd+$M%1?_N_ z=h;JvCsTPUf=Bekqwi2d{}svp%G+1n`81Az4%b&&V_l1qMED)JOLRCOkAHy63PsOh zxGN_}aXX3!V+T)e?g#7NmJF)Lzy-4P$Mg1{zu#cE)H@YCh+7pm-W%+G#Q>Z8X=Mp1 zchDCTE4KG zlK7D6Q{{EN(SjPpEs40*_2oGHDtr9bo*1j<$g{0M1O!?P6Ct(J-!7R z6mPpAQrx|_IK9+a=sM?ISogxO?nh_Jciicbw)5Ei@n7>ZuOLp?LBYxucH3$Q{SsKQ z43&BxCGPduI&y4xt+r!iKxDIcXXenX`JPmPAE?K)$P9XS>NM4qb@BzGQ0DGp90uZ0 zGL>tFqCm1t#_yr>g7jWCD&^og|3{z4wmqW%>>JqPxCm~>ErrzR)loIPGDf~O?d01= z(+{kV)A}2IfNt$>I~;gE>E7an+}J>q_urWubA9mmYr2DTG`9z|=qE9<;Y+JH{vqRl zN#ITH$*!(iaph;off&Hn@pJdir3%kqa_-Np5RwVLA`Y7E0wM2_h%$XL+Ieq>wxtLH zv~t;0wrT*sREN6vV~jB?e$ZXdn;K*nJ~#h??seEB+}9spK#haK2j!ol#9pQ0$JjU4 z_ONC_kC;rTM3%aC{anRbv~N+^>v9@LO$Rrxlr;3Q@NQQfT2-ew=w3i{pLnxgU`1|r zDe}i&%w2hOt66x)?(9&S%G9rKu1U>w&d%~}HReuArUTf;Q1Vp5{ak)&SO94g#POc# zsrd^&HP=w%o(o10Q%UqnJe0EN`xBJ#SQB?PKU}WVJK_(Nfa^y1#q7$2VIM>m;IK0Y_9iy=6Jm!7q zRAP0cLRGA{dLV<#<*&^LR#78cPBAww=~b<#5TnBc%;>Wg!gcHU zmozz)GlR}Yf_Sc+dH2w>QX`HH&zswun>Szm3Ymi_$buKUcP~u~J{a{XH=3S|N#g2E z{WRU#H`v4IynCLg5?HYWR9O%@K_(i)Eijg4rH%btC`1+0pmWyC-~ds73#o(n*re?v zp(GGSwt8;7BRM@IEYmB%(_Z>M|M`7;HFk6ft?1@J=TqtE-m_yBZ6)LU?yhUy(OUk$ z=jpSh_8c@kvEQVg@m|j5$WfqLk>0@@N+UWgJ-=U%6n`rtAc6D2*1kObDxG|Hqi=g_ z0+D2`6>lEK?}%_PXD;?%PpbNLaE~um7C+jGd>&p4nM}uVFKg|dDcekLbiFnVaG;tw zFw6t1u*i7cGT%hKBriZOcDY^%mW)dtGk;nYd+p?=Y3$KJjKH59Rr4y|ag~nNSi2{D z$}{aNcy)_4*=aK3T%r2}gU?52lMH>kqbRb& z4b)}0F?6}0+^0}G_J_Dd1shM*>uIY2Rw8fA}+W+h-{xr!YL#SEa@0n?G?p*{mD1@%xc(W8uVKN^`-xZxwnjq zE7|ry0|5d7f?I+H2<{fFA$TCTyM++kf;+)I!QDf!#$B3VO|Zt@-8*RGyk^cl_sqoZx5p{kFE?>=|++ zuLvYPrRqo3E+2HAxy$=?sSeNTLRA7*fb;9gbfgM~7CuT1L{7DSEX(5Ks!JaMq-QHz zS`DVI66$IiToliCKTB1KuGx*j=3D7ktLRJG+)+_%L0Z`=7t~QW+`bK_gc9{9;=DvvN?Swdczt?9)R7jDAl6*s3Uu9t_^kiX z>NVCzTVH@|_6B2eR5&pEeC2S7+yFq-n8TgW3JS_;WDS2nl1spJt<%L|F*e!C1fDe zQ-WRv-Dp~VWH$jYX+O9-B7}g!sn5=n5*}^DwjK%7gzB1`R%d}I)-P437@^dOf?`Q3 z5dv8%CFF-m=Fa!7DGk(MuV!zL1NO?VN{RUdrs#CBnv=$F&&Wz5W;rG>lTHh$V%bLS zNKc<(L~aNu3i)BEl8c7V3*C;4qSait?0J}CP%ON7v4z|Ngq<8l)hFb1rLqFZsW-G2 zSr%OmYp=#y+HgP6{cOYnnlKFj9mzZ=MmMI5g=_<-<0H1F%D2NZ2j3g0t$%~cjUc** z_!>L$(}`4X6A#!X`86n=a8-JxGbYCL#YLu4021k)+MY`veH|!_FUYIhagdKGxq2N? zDGsSr^SSNH`#DWmTVbR5jps>1oPc6pd&m9Q?SycM6L8q39tjJ(L|*yju+0wLtwZ`L zSaR+e68+O?IT=w9m!i>RA~RQ+)s1U%NS0lqYZjY@|IOpiPnPGaQw6oC=jYNgqC7WQ zIFWpFSiRpMt?)6vaRX|BmvH+80ORVrdWvB%pU+*MN&09hahZ*ZEzP0rt?GF0)g?9* zwxpMoo7)%A9{$Vn}PQ*v@x|EgC41T`%LDChq*JTZrW;R`6xKT|2Hdv?|DStFI z*fP5gRcJM;1{|Bg1eI5Q+OQFGUy6CKov)U*)0_OI zt0%m41Kg%F(3GmhuZ^UhNxsqtQf1e-7Vk9eICOjJg)J%X-Wh1b8FC zbmA)L`f*B0yS4r@pfBDKaY-so&mPr{lr@eD$1Q|+)$t!OsWJ$vz*EVkoW+!L-dXn) z=e_ABS}IH!IFNT=tW#FcOw~8vtbNTfuWz@QC`L8}UV}U|z_PCqpt$2JHpv3M85~%| z(^7hqh2iG0oyQrb!d4X^Dn!sq2pP`C_Nm@pnnZYlxpR780tsIQ0Xa_hP7f_A+e3D(bc(xeoI##$O#BS-DyO*<1@$>yWKM9tr z99nd(wSFmG%^Qc7=m>OBLRy94y!#w^^Ex%JA<}H8hqGg;Sd`apD352cRy7l>;^nV& z21LafbUjoex!kR;EBX1zoo;;nI$RwaQsJ&Jbtd%kS^1NTXC@Addl(K>g@n7bV_IR4 z?>*_BwP?@}xkuxueqent#No`6SAkNUS;9Gv1dWv|2Jh7yXaMOm*3y6IvX zy;=~dOHiN$s+lSN;Zuk=D%HSS9Jx)bavZr2JK0tSzQ{?W7ql{!H2V0ovy75xm+LC5 z%$0{^G^NUfUOF?KjJ{K{>GwXt^IO=qV|wbJ@L$i{?vY{I6+%28&^MRZr@j>FSa(Us zFTSyW<2;=2?vzd41UMh4tdL zz-8lX+Qv~Je2ZrPC)u$TC~(r`V01yD9$M|%)%Ql&Znv@N`9i#|r2|(n;SfsQM<$?5 zvzA4{p@vrc%&M80T2&okZFTr;hCQOYT@a*L_r+2)sTw2UG3{)Y4RQmq_5H%5eP8o1 zj=JLW7M6~KelPTT;oHhk4n2+cWm_y&uF%0+-jz=d48Y8sfV2!i)8-knik-GKbQm)R zc9j5gQA=Kn(w=G9&DiJ_T6Nf2wRkhQTeuPgAX?q!Wf2CsVexF`A|3l|IV$)_4nj!Z zG@@46VaL4FQS9K!ssf7%AMP~90TMUy0WPoCMqVF9pEMkhNtcomvEbB?=U(7(yLgcB zgFPZyWhtTR0nZtLiCI&?M6UNfl@fz&e&DW%(Tv%Mujekgd!`^=p(6)S;2 zn2blh4m4&8FTq-Lt=ZTFB7=ZwuBBDcm7OZNr|YUDi^?Z;2OUs=0oi%o4n;g&66AeG zcH_Y3@Y&NF$85KE9)tDZ{MNgz9Ar*-67)1OBXOUgiGK?-4fElYKg??5%1ltJ16|mmHG#+QB=ztcVl0m$~bnLl^yVytwAw)FxQ#CZC5>>Z_zcIAv~x}&0X#pIuB8b zt&k!Vcp3$@m|Acs5va;&D1*k)$Rf+Wrs0sIjiSze>ZWq7LvBN)2u{1 z5IWv?B~*28wZvl}(jp?j{T(Xk9c$lnl|2=)7CqV2H)=*OcHmX%0%T=9o4DK!6+9Bg zr9r8ybMb3nQca!iE8B|)n#g}#fHb7i{hELAEg;B(jO?ltXO$PGxaxRq@pGgFQe|(h z3u*{D-Iv~d7C0Au?}+Ogcb8|#h zz*6&*2n}cljq*(;@^P*NcgJX-Z|R`+_7so@H;pIGhqj|~*Abeq5n{!>uSur$As+_w zn!D9Te=mkmeNx|(Z-TlWAt*IxGlo8>aeF##V{h3wSG7#kUAz=#l92aoatc>`*ApHaKGf5-1f$*JTz2o-oTeLpG^%_SR}2Dh}Ter zXeKj1JTv5(VCC$o0o{t)Q<{1Gb{DrYdMFfs!`tZ{Oc2Z6JeHVI&HrFgeeYMiTO0BC z6*Jn2>)UO@y*r94&Fg(pWd+YcC54lF_=1HxpB{avmZtlm9a))?{y5ABBFw1}Dr#Uu z-R(J;@w03MXvWml-FI^o*dSfRfM-0_(r~e-B!x3_vNVzxii73QM|*pez~oTXT=Y(V zEu2Z|ATQ?q3=Ik$xexIKh3CL0!}7pmtI6a5Pn=~icam#eV9D#djilEpUvYEwuM7wP zQy6ZG*Z9-qD}dG(CLQ&9fM_w8%5EwNmrx#qnOQN(KXcZotnsb3Wa;Ceeev@Ps+H|< z`l-`syxc}1$v^dh%pTQB!t{>cLA+S+5sK|( zM**tV8O?kpql?O>jb6Z*W>J_c;=vgr;BJDs&>X=2mjLaqb_Jjy({j zWq=}-10E(pMf_yRaj;%Bm1)>f8g3dR7B!xF`ju&e$0SNVUCj@xD6Xg&)Xu)T#phkD zKk?-bCvji4SJIqq5+8sK-x}^8%|2-pqpyh&A-ktTynJ5B@Fif0$?ND`Kwt-?S zIu$s&U}hP0-J)M2L0Jd>Rl0KLo!#&TK5B&Z$(o_02G~vMnyR#xdhJ?k2ix-F=#A?; zKQv|)onhVk3`E;^I?(UP5&covx3olTQ!gKLtugU{b;Y-}T}FekhN})*ElO@gOrK^Q za#7FRVod8q7n{EW$1Kj_J3ZWk=h!;cwRG_lg8MxRP&G|%XKH_K+xEZ4$pswZIN>XDjOZI?t84mw67<&Xc`Bb6#k5K2D!PAIKCF3V{P%+aEyJ9Ytkm zvrZvoEk?nLjecpB5Onhfr}usYzGKC{lkm5amDrlSJPOPEg5$io=zCYeHYtDVOI_xY-Cb`uRHL08%%xQh5L;|&}|wJx9(S99t|6$ zqfdQ~y#tkjh6=-un3d`!qIhfIkea7H{*RJM^7T^MnI#5{zfQdYi`mZR1zohOQe z*SIi`$r6<==et{dNOyL;;ad|rkg9eFV+*g@y;T&ezS6;7`BDG&$ZHO(L}mY7EI{a( z1Zufy&tCE|ohPj4=<>u&A1I*Uxr(L+f&v`uNuG2z8FZ&&&CU;Woup`dI?i`0&l#{v zvS6C__}chva5#yl?2KQPLx_t~FP6!n9eAlvJ|A)_#75~b?2qTjwN01#=hMY`yJu*!#QdsM?$H2Eyj@b;BXc&t7PlTaM}>@D&Tk*?mERu_D$gxkce)vEOms}VK{ z4L{~SN10533c+YK_e8Wy`FG@$Jk0_qL)6 zKOEBO4!1?Fe6kQ)ZQo()O|QSB4t~ESh55_694LdEK@IJ_J7=&~; z;mp3U>>7`a$%H|)fk1`1ALDwSC+e{|5@#aJWw1)zRIk{bQ7|ipmM*J930f)B9Cj{e z2-U276T%~+-}4=5{|wKZ2fU!Cq`(4vIvBC`INp`%ZO%+NuLMg@Ypf9c`g+{paUgPL`@a|~Z)@*=k zSI^o(BG`UHK{+~(#Xj<+uHAEgCx+bL>zAM)ySD_?K>akwuJXegYw3kTX_=#5M_h23 zGJB~B1BihSemtf(1sx3l`ir;hmm1{6L27VXOgNOjocbPGB7bbh72WU5<*DztRDJR##e zlctf+W32tXzH_J_x9Es{U>ZJ(K;jGsL3S}R%6TxK&Y1+VRa&>joi?9706SOd}~ zf1sVZVXt1+a_BGXXqvZQ#C0afI;Mp8Ec7g>8xQQTr7qq=n7x8_(I( zo$mnS0IK&Q9ImWBjhoM_bDLcsm+9~)EHd?ot_nYT=~jNebe}7Af5(|1WZh7#$+{PrWe%lV&5eX2r6Dm2Ezeq0BYJM%jWiJ1>>c%Xb2}JPSXb`Ww7xuS)Q_@~8&hKEl46}gKu*{t!qZ{{D_6-x|Tzi|~+U`tKnH?Xr*{mGB@u0cVYX)y~ zmu9YC<8{oH=^Uox{WkS=?N`+l7Oa~~js(4PgzW7l-fZA(8fo@?aK8Jhfwxwg_{H+A z&BfamZH(q=vr6ZW=$RhQpN-{>Bc;o@9yK72VNcG!_g)(S!TUo=pL}f%7d)1BUGpN%UdoVB1*oZ_?SwBU_xsCN zMk9jTJHYnNZ;d=arXRooHdPf~7{b?j;SzG&SZLyv&XUhn>gd@7O_aTyM3Pozd;6p& zews@diw6a{>mVIG;b!8|px#>gd3D{-S4tS+O?drS>1vAk!`G@UdeR{RMMfD$;7 zc!xJnHIv8J-~+`Y&s6h~cRZ90V+^le2Z}0em4Hp1gaqXA*+TNtgzNDnTcdEpEfU^6 zawb8*eCC}E-ZnzUO)#im;M(-q`@RR9%Z-4?SirJ~)c=`b@Y?E1d@Y#s<=g5W^!^!X zSJSEeS7$}S16shfL>Pp%@Z=n&sm!(TW_`yuuRgU)8}U-X*Bj9nAdQkR54B%oi6i4X zW}ywfiC5xS7q;D|Eh#pl@>Q9=TyaggU@+W-a30nWAkT~EvLA)f?}TT@LgjreG$5N6*HGR!s zc}Ab`mY05l(!Xdnmu@W1GGdrpWXyb~h8hscH|JgBRl)?3W$a3Kb#j;DLUOmcfry8w z8(L|vc!1{eV=v1Q%`n|PfyI% zB}K-9MDDk!eb=5_(_?qQAPA*rOb;(5uFvBkZIz@MQUuUAc?I*t9mym912P$S=>BzPITPUZx2mWe<6= zY2mQ+Hru(TSiuczwz2l0;u?4o%ffv8v97RO>t*6fU*C{6%>;$q z?tF>&HGe>5;-gx5ILcq?&1R zzbv$b`Pkl3V$I37x>vdde9Y^>Zy<`Aj^R#I^6girgzEIl#WhT~mNrbgH*17@USXKd zyT23CnxA1s*9_+=sJc$2i;(L4TKf%b+`^;CEL=<5O4!>>_?Z+!?O0Ydi?LR$i0&Cb z{L2p&bB&g~u#zB@V6jQQ0k#KEB=OR0k(di=?dl_C;h4juT30T0w_;$ODe19@vF}x9 zOK_zzt!W)r*C%APb3v-~*yjbCZrnk=$9PJ+*5|oaRk%5rQ}Y#|G8vJsA%zM}(RGg| z+=E+9TYgZ7-$#s_-LBUfu8HV^FfG5}wg4E>GbC;N_a#&E^RKI0D>ig~3DG#H6BRcU z2QeKYPcCqw=)iPfmqrMMbgd9qXzzZlRJxJOk|5=gQG$@Xjk{pt+A0=Tr7}c&V2oh( zJ~At2nHz0w#hFad*R#pEVAHmX<>%XAYjK*@Aocx8WY`BJ$(AEc8!ZC$IRC$|Hd$z zBZYC%4$kzxZ-C`<@;<2stj3hMtag?@vR3Pv=Bc&Bw_Hcz4m9_*E!fZnpU133_2)ew z;2Y2C0uR45W478|(rFNj7=$Wjw!!zfWjw%YFk6?nX-+0QhJt?4123klR3_Yeo9`i@ ztWe)%9)^*7xqh?O;(XlpEpox=Kplrrf*(z@@wK055Z#?KngHwHvIyMorWV;H=8bfV z6xL0afaOQ0eZt8fmL0_5N-!6-bn{KZHksK7*45Z;Ev)iznM#i}oWSlJibfD! z{CX~Usyku=!O6_jF(f*9xa4yjd^a06%N2qC?uNQtBFJl>tsKt!2A#IOgxtN}E~j5N z1d}a$4S}zcPxue%l_a%IoAS@NUhVNbD*QUqx<$~N#pDzOa^54x*IB+_Hvk-OOu2z^ z7hUbVUL|7fSPdw4)G5Jy2bDxhfas?u-?WoX`S>q^*aaa|Ks88% zlRSLR%k7|E=j|#IZ_CN6reX)~Hrl7ay%n6^EMtmmsCX5*`@zKa2V$cOHoi}~?ukhM z3D7825D9w33|KyWhtAGwq<@osuWYETR1}kEA?y>yw0MUkBA0rGTcXR>LH;_G$j5=& zZ(O<5%^OXnEXMF`tl>gnwXOLPMT0)5nBw$F3?bsS=~>$BdS~xAr}~q^w}9GMmdZXv zlb`)hrZPt~UtX`ZU+niee*!;8PJU&Usw+&|a3-s5f|bk-BxdbD2Wou>VwcUz#l`M)Y=SvjpAhiQW=|ETrFpdriWd}^7%Udh zV3oiL#h~YUr_VIJ!wElnaPr3+m~QFk`_nAvO_OH(7Yw(2fP4DPy4BBu8idj8>MJ}~ zdh$C?tAp*+mSNJtFFaGPgKnXYcHh1h=rcW^iO#7_Hx^;UFl>Cfe&A_3X`inZ*hS%b z74Ck{t=&%csMKff6yqhGs>zyt3iF+H?b?9664h>D)_Un(~t~lPk(v4Sy^q-mhC$N*BHw;S}~Q))lTxpi4H@!i=8^3Fuo_z zgc?QKI(BF`sABY^OL%19=;8G+->4~?{;X>XFXP!yTmdK zVyjQaQzS2!WRQ;wXVZ-oJW4s702_DnLVp) zr<(b!Y6HV(%(`JbTcq|;w;o4B`Kk{0_gba$PgR*&OkSKymrWyT@b-p*#s6kfTx4acyv4kswk_WR5%AoLfjnWD1nY)x zx#(mfup)Eev|2@iHBRso;YwNE2Us-UaTLjiw(_;av>EX7SdO@o??ReXse>~xi~cE6 zl)B?0_N40|divwtrrj-X-w8Kq8weoMiHJD}_Yg?@HQeJklku)jQ)s9RJH|=i$q5-g zT{wL3b(aT`KvNIjpf-sK(m>%t&ZvzNRee^x_BdruP_X`ney_HvWDBzDWkI z39IyE^y#-v0-LqHoWxQdh5~>vSP8^r8oNW}`sQ;3HU*c!<1Yp(vqHePpE#RxP3%5LiCg7_ZVrOhoab!q)7$zM4YP z*;S&NZZjcIPb5OyWseZ8tRvlAjek`D_{Zn(b*3S1;g!TZ76EIDB2jUlQFi(}ZzfLi z*hO>;RiE>M%{x72p#UTyaG|kRr_MCpQiBKCk-Bm|1x@&A6V}YhkjzzVfwh?$l?MK* zFWb{2KtD?h&rQOgz^k3j|0#7WjCb26?6$y!y`-T(>V3{~>w+i8OPhpujODC#QXW_4 z6eNTxaza4`p(STqA?GL>Z6UyG2tKSFw_Xcr4$s`1!FC$Y{sMORx~Iw>WJ>~spR4O8 zIM@3@o}>5*ONOVEwwvH^d*yp$2YzpiD#xWyFRtdH!j;bvTC%VjGfNZ{V6FnK<(-Hq zpq~3gh;G)3y0(sI2^nc|CA`b{q>{Lj$@iLwKYVG>&ICuPx#B zuJjr@ZaboH`)fZ&rf~Qvh)*(JoScvrr?v>|1`Fl<$0Zu3Vvhb*=99%I``Qc+h+JYJ zn%E^_TZyG$GX7b^K*l?6p-CXw;l8Q1C^3&fAb}l=^sa#*XNrT%dod>=A2Y=+qRL4l znqa4_go%FR%7Jn8qD|<&^sA6z_>x~ygIyZJ6kptkcbz(;#U`?4ZbGbkSVhO_C4=^# zQd*3tTK$8OMJKfdJqDDIO?aZV zD#9oYj}9|lKD}{1@j4Iuq<(|ufe?bqH1OG@x})s<8BLpV}C8+1vN04 zsP$F2)FJHsuL9Z7yLZ)4?|}_qJ14Jz#dPszV&HQEhK;aCS8Zoslu0(OuG03(J2IF+T_D~FY`TsE1_uY9 zhz0?y+5Ih%ZKC|p+*7kCNPF)|bFQ}&bpFSd*V4fo@o@G)p)xGf!;%t?SIcQHpOt4$ zVav7gF7Zj-9QT`w$EX9hniyO1;$ajUtv zXB$X$jwjQwHvJ|(~x9Rr7!d%N{E*%xiWe4VQb|1Qc1kD+|G^cg~9RU(GI$H zjLOMmE(oS56G}Zn10!0$qUZ)LJZv@!(=T>kV;XAv@R+U}W}}Rm)F{3<XEF7aca;A66xEi~hCy*k07+#4p_LisQxoY>ha6XcJ00TjOVy=XA$d)ViCMl5K?%BeDBr zR+YxS@Kc=T-^NQ9VAOsu>Dm8RdOJzyi*OSQkWr2`M;~LWTy8xisYcXiZHc1YodxmQ z*`WbIpV@X-=T;OW2kF=b{M;T_>uwVcF1w$L^etZ`ziUdg;b=-g9X6$bPswTl_aGg= z4rRal5Ia3>=Jf)a-O0hT1QKX%rZktp#wpa8mfnO0tv9%)@0#|;*gAu;Q77C8Y!Av1 zTJ3kV<;8BjEVmTQ+Sp9hGb7dAC#qCo_`n!(qFEH7>Aano=mcyJSkVR|K&U+!KiT{{7*4Rwj*xu z+zA0;*-(3UI^sT&=Ik}i(CK$Vb7Qq)GY&)kH2qg)xuR|F*pu9-;9q>L@8Il|!~FGMDxDDx&^?@HZC!bt#Dr z*9ofVe}9@jwmjJzUWst`B`vEz+uQ$?zrRC*Kl08s&V~x%FF(^0Ai{U#&}Nm~!}PPVBGBRAPiP z#70nBkja32To9d%{5N_0>k|JByiPA49hv`yXSh4T#b9#&hqS#-%ijKp)Bj;Vn-d5A z2{(kgfK>5Tllyuqs^MWE|*!^3DHzggyXXC&@>vkiS=H_y#DlW;6srQuasX zF4iO>Q3UAtTMiWeqwlFn!DXx2)7;1V<*ybMPw&5H4v3kWyfX~j{5c?-)PAqeL_W+z{p0d^C7qDVIdx{@;56{C{NcwZE_T z4Z~qNOg3z*A44v)U6nd`B@rMj2Qc0IN2e4;#DS{>AZGMV;cseoMr&y{#v_L>%r9L! z@tz36?q-30oyNs_M()?I`Zcd1vGn=hZR!taQ-2A!IQ6DEYo>qt`LE*qAJY*{dC>WQ zQZ?m&6#0+Z{m)JRVu^O_-mHvJOJrnZx)PPn8I9kjMZl$(N34lulW!VG`)8H?gDYkC zJ(zCaq6csK@6&Lj!A&;xQwrbjiz0y*F201;GwavCPh;`G8ut#T-EWJ)pHU9(pNyV+ zGxonvGfD$z&0^GS@$bI%XO#bISv?W%j_T)=)hoYGQ}7nf+Gu`%%RivQzkTX&3MyuX zBTC7BnR(~$)5ylbSu1#(VG@}Uq0J*p-GZ1oKi9jeVv*v9vf#mw3tDTfiGQEQ{ed;1ooegf_et3YKbG~hnqmKan&SsQb{{pL`3GVDANosp z@Z*H@$tvUDr`dS$h2W@LY#hVvC%dL)m{|H)MU59z6Yen7!BKy||JOV8{boHb6Aq1FGlI{y_8 zi%}j>u*drY)9*{XFB#678Af{d-vXU~4C?-MQw{S01*5J9H2yI-`A5ol4mfN5#IGZ$ ze;Z-GKYu{M=dhGAn%}0OH+?`Qj$c*uhZjd{)Ae=~J~v0;D*9+MI9C@m!`W{hbKA_~ zPn2lF+>ztH_=zBr!fqWHOVkoBwapEgN({Oj|0VA56M(S$@%sV<%`%-@*)c-iz`uh< zu>ibAOxo~V_aP}MDHL2>T&3y{^64(K773+6*3Zz?-XVf+j?s@#P8^oo5xDIZNso?> z2&$<{oyT#O8=%m{uCA^NIk)hlc*^CjFuaSqYv@RW! ztF?M2bWGpfULTDqijY88!lQKxbgo(1ljm#_-6q1lfBLmtQX<-gQ+~?tsZAeB^qTarOGq! z%NCpY`T3`tUD<{N_L$Rsu6Azf*j}76|1FsJLy`@8Xnq4~y*Sk)2pk+7f_XAWziUn& z!Sx4eH7cT6rp+cqPe^E~N+0a&;|x6MQ^|aRmi`nW`uc8%!Hl@J$h?BNPMX3SU)bZc zy>Tnk%B`C|f>W*<|FV#y+Gt zzMdc`Uu%D{;JTFKeSO~D7eVyJ?^U~lVI2LHovkhK%XenvnR3K{5 z8WDm)qQa2re9m+49mVhf6WN@2e+WsHY0rV+X)&h`c+V%lY+YS&q8`BaU#RFGE-eb* z4c&Gop759eI*B}`#gfQ0)xhW@#IyDJ@)ri0#A5`rpHBPJ9M-cnMJlK}x&9XFz{gV)O<>% zztJpCP$CKU@}=^RIFuLV4xb9%8%6-Hf6PAMl&CvSXd867N|Iv)(LV_N1Op|WM(=Vdf2(4~1{$AM6GwD$tEHg(k;?WP^s5~ax zYn@LESd+mIF4GqwTGOIKk*c6za3XE|EqQ6J;Y; zp@bYN;Uc|0ye=eWK}5Rei3Y~+Ta7r2XuWvQ3q2d+$T48T$r28%C3F>f|E((-c!!w+ zt&Co=zx$XT^rn;?4fPLxg__0RV#6^g%cKSs`)2BOlS);&QjRTVCTN2x6yi2eT^X@&;E0pZJV_;C$fyvHxPcE9*x4@ZRH z_f`wWBRvS0LF)fs!i^6Q1yESTM0TH&iH=QAXE%+P&Eea4o^{9GwGP*O7=|c&zjW%Z zCcfG#NUmAN#E^-sIKI5>E!W1wF(AOfVoj6|j0k(1t%thZ-PwV3yhpegb8V;-@=ch| zsK>iiEc!(n&EPbhYrh*e)A7-^noed`v~;jR$U>*N(fRkG!mp|+`e~WGL|8)Xc*i=3 zP4XSbJCDUUS*o8V7`=fTd}kyfLO^rRjz!F}OA0%W&2u3`u{Q|2b-U42)FFAAbHRuG zeP&tSWqK9WN)NUZD)7Jdln0Ec5c}^?g$gx39KLrjM=OW3Q|Kt70%lLZ28>Fj(M6Qd z%&X#dkb{^7v)O8hQ*W-$0I6VWZBV)6y|kgl33b8b2Y7Am<&xu(`ojDSgfQ4*AzzMi zvm%K;AWILqJ*js-f4Y{={Ut`aaQfHH0r8k8H!s~nkeO4tBn|ZnL!w?wf3?P12NTo* zD4$AsMvi-X=3=S0O%A>lJ3_j7a1PgkQ8NPv{SmxZrx`VmVlZSr8R~3a?^~Hgrz1Z^ zDWYn)|8n%COt{kc`H?q;vBf`oA4CWAkZgl5yO8za!|$-re9j$XB$6_R&=X6!b=s1 zoD*j=Y|yi*?gm@f=>mxUHcx{&OXhjh_CAz|59f_V{_oY`Gw#K zXyYwzIU7W`TLSF(%UEcpW%rXR)pg;Yk^CS@1pyI*4{L-qM(3(JYFtO*0VOq z40hEl3+W)j8$-=dd1BO#9QG!GvBYsd0DY9hv;&`XM+9f>8hHw=KvW>;am3yNm$nB6 zPuj42Pt;DPL+4vna>UDKNlMTwcf>>pIqq5F=H-rEF3eeVB>O(kZS8J&5hb^#I(nq@>kz?mk#BkcUhi?mFzS3!^q89Xh#1;*D!r9-BH)bm#_MS*O zUBL7bWBz1+tHI~kiFlsZO#zQr+&xLY*#eQ1wFzC-4yr5$kw(xMO!lJ_u?zeOk(Z#E zK+{!(VBPgRZVWvUPfK5N*Iha3)ML(Y>BDDz3IDX&Nont z=e&WRci)(b=Q8Cgs95Q>=1YE=rPdcB zcg&3wHhvm5c(=Neo90yB(%-a$*R!T;s78~*MSUDm{afPWQc{9SDa=M%Z1uTIr_|;l z0xo%%{5#2uPyuGC$3jg;R04eXn4Ydd5v7Ijm{Ygh02^yz{;LkNqM-mX-lp~NOw(PD z(EuJ>c2=UAc{zB2@1D$4pW)ON%%RFrX_yQ6ben+OeZ*%xWSf2enQggeyhjsPJvC1FmiD|*+oZcM*&w~hix7Odk zL20H$&%8+|HGMa+7M%jj$?uoGIv?beD(-(y`L;*Kl-@b&t@8_RNha89X2Rodq6T^( z2)Q4{SUvSvZ#F_|-PcNpa9yKidR$8MLW;=qvya2Y1h_8*Olc@h(<85AkM#6`qldlB zqAFW;K`}Eu)6KelNgRMWgy$apNlH{ZC55X^BA|%(IBozAp|_T5Y^T?`6#_E++Q)50 zFSzfDJxX8^k^N88W$6Y#(%bYyWWk#Yva!!2P$mT>o*~w~Tr&HbWs$t$ZrZ6H?A`sGpCV@% zzgdYYV0R`Kz8!K_UnVvxh`ZBL-Q-T=g(hw(URAp?Z*VwbG#pZbxlhQ9GR984D}twyw$v3+3iMA~{@ zl|~w-=qpI>Hg&wQFYA`1wPrXsQ&|VmYsD6ax{oG^9lSCgO(!8%3T(cVm!tSG{gd4I zgQIVxb<<-u?*{62F`uzCT(J9YNT%a@1(b%_^Zlc$QHjv|_U91jPm>+qxr$FGs*sxl z^byY*OF7&RELjmwm|-b&a{SCVKi>5?&f7z(gKIrY#^>x|JQyG2#AGa@1ncHq6-_SGbF)6cZq zm^v-2)6qxNuik`>m)3z^>Z5%JD zoMDg!7KAtc$!=jdrK>fkqvS{LINF*+fva-jIYG3>9V?F874s;8>`oi4Qzbj%;bIf3JQ2$09_ZI4bJ@rXe^s4yM&8@D8{tII`j%m~jU5$-L zGbD3gzGOtzMi}-qYdo=hg?PD>phqd<*rBf_eD;?5{8GP~MlF~hyRQ|Eu-E+CLQD>p zS13yEcFlO%HVwxT^w67Zk{g@oM6O)Y+9OpfQiKwvo|*%f?f;?ft)rsw+O=URDWyv~ zq+6t08W9kX?(XiG0i;`ykVXZR?q=vN0qGbSx;tlpchqy%dXA63_j%X(>-*N?j{)}V zxbE2Zz2n*|^p{89bHyoJ+jz3utg4@lZZ4jaGAWl19o*n)nj{4QlFG9g-;G3uDoo_6Y*q6)DWmCwz#kxi@mijh$r0WaF`yL$ynHV`(@~94{trb3iQ;6jx6O#JZl0^qR+K}Y5a+nr-a*dnPM!bL(ubQaVN@gF(R0W2t1!v! zPE@0=NiBL(sKq%Ie7Q^}ef`f>E{Sa%F60xddgB^$z05;E*1Oa&2-d1HUrLWP) zfFD>XJ5(zRH1yPraJA2sUTo}Wah)2x5q6v8__(z0PUcViP(E|B#N*}hnyK5I9(zim zWox8WesHrm>>G}=$L@j=^0U$-sKTdj&f#sowvw-P(AN0dm{JT6qqdP@zcFp+cY@zMO^iZz>zI!4~wi# z_rk>M-;Cwh@)qA&HRqIp7FSRQWa4eNZP4s=4Z>=mRG{byF&DL@) zhue&eQtuewv}rl|dX>}Jbx4~9@mA1PpUO*=2U8R;Gv~$rHG6p@SBoSGC8n($hvnmD zg~i=od7sm+l+F4~+3jd<)6OBaDXg@nEt;o0ikRTaPA%P-h0mGGvdykCuOVrlBP92) zgk0M&yWe)8b-k4=-YqnpH-_5UXVVAJIg}qYz4SiEF)KqG97uMJ`>@{qQczK_!%1*3 zw0J;{zZ<4UhW2sVTI^|nQ?g6d*O-0UMSfJX zv)hdBX60A!bKEMNHN=@dRFQsi%6d-zn7#nI_TjJy-^v>!ZLihi$Z1K;s1 zcQ~A6A3-@-ooK!uUIvLwJQ}D6Up;aCCZOudN-rr$D-GS5qK9TW7g7l?I zFTi(RT>7hu=-YXV0MOFdS#>$*M1XcYf{@ec<_vHlum`^DK?0owIx$M0wk`GD={ zzM)JyRqC9K0Erj8-E^Qh(dcKc%>5%Nc4o#cs2-wHXlt0=kI%7~`|OQ)o-&4IN32mM z)a@s`pda@i+U*~_hY`mjI%+|7$$D$9Tsbo4{jkmC9r{CAa1*XT3mIJ(Z)Fm zkiHZnVKlPu(7Rb!bw2MhBC!#B_dHHF)0OWO>DeStPjRBRitVhb$X=wE82cFZC<}xK zzNi!vL;P=^N&}uTU|zq_^W7!)=T;06u3C3Wk6t#DX-IsIFtjPe9a&2~-`r3VKLV>h z+6!boP9!l(N4(3~U%dk+k(_^>w>2yoq)Onv`SNcgn5so@hRkT7iiu+M~e8p#pVLi98`&?g%$4QZy{z(b=s z?j`Jdn^<^d)}n)apHvq^1YVGB7eR@Zqk)Wzae}r#80SZZA#yu^M4*TUg=qdYr;`w5 zMg`vsX4jKAQ1eYoXTUeMVV>`T`kCGLW42}Co@<90_wNckgm^oJcW>U)2?^>ng#1Wv zU8Qe7v8Kk{E=Uc&xhifD_86MN=)Q674iWrVNImUe?bt5)^0-4qu*rx}Ct)!rhj9<$&tyc!6YK0JGUV^`rZF3}b-FYypA^l^>Ds}g29 z>)VfC(@$}1SokNW9XbZJEbJ`5mc`4@=a;q_Aca3uEUmDO;BsANU~7laX1?S#K`!#F zoxE=yXCZYtemp7ps8M_wXFX`vV+=C5U?J_mMS__Uxmkiw4vFhQ8BJHTp~ZK z^TBqqQ>m6p6?J4RTmk^k44;j@d^iA!HNL35pe-e8x62Rvb~)(7Xl+poQ*!a+biBh| zAjleXJmAgYI|1d?h zP7Z*AR^D=g0pAdb|G@KBlg|-{jx8sW7BrQ3$mlFPw}K}Mgiwj@(K#*A2csf zwmr7Q8w%O|Ij*j)_6#p+_=fhi^nkBj8^Nls=$gIU>H-1U;>F6#OKXA|9Mk8PiVg~# zx_sJvGkb+_SM(mT{62$19gY@>%p)3Y#DsG)iVc=l^8lNB;mJd z`6J|rttIOfU$==JCADlB=!HvSL-NQH_j#Ta8@~6d{`%BBhaj|NmnxIN#5Gf}d!tAF z=xB&6^QmpQj@((4N?#Pa_?%mMj0JY2>mX5ge=`8&y;{kCyIc)^n?er^lrS`HA4`v5 zd|WPgk^Ir)+js=}fzq3*O7B7MmXa3K#yU#_zX5%-Q19m~K8FtgKRwg=<706r)&m%q zmPpd#(`yMif5E9@cpK?9B6njUeUpukUwuq0(U1bAGM&I^E6E zElf|v*VN?e60aPrq}Nf8Lnx+;kDSuBBlZN{>Z{E!=zMa-?ygoL2LXPn9RW9I9Y-r& zu|P?2U*!@Db(_Mfkm*U z*vydE0m5_&rVon+`me8fOG!Yc4q!ct?Iy=+z$DVi%&k^YUJ> z|K>c(P@x|ikDD~w_c0tqI7>|wF58rBmo#C9p=s>~ET&{h_-OFVk9w*O<#K{HEjZoW zr9))Fy($2E%}UhDg}j zL)0*hPoRI0?LF@YonZvr5U>ea=Qs-?_VTh3@-1qm_sw&cT8@e1ILHftsS}UQ?7as} z!=jOsdlQM&guQM+=q`7FC(z1?AM$xuMuu?+DUELrt0Wm;*Lfedv2?1pjJb7oE!_o` zDCfTXcpav7d@{eJY014c3uHS0ReIsD7CygUcQ;_=p6&@Ko9Q%I&o|huYXVehx7B+z z?$03lypF=rYV+%90-UsjBQ9;)nFL#SL}K{eFbG?oS3R?Y%|a zZCA3g%15!^%4~6%l^7(<4-l*e<%OLD7FkW~EcC25#q@fOvb)exYj@V2!?3OIY!a?F@J^JEyg{B@TH~xHR20X{mD$7}3<@}20N)WAG`Wu#O5HtXlyGL8 z9xH|F9|8D%rVI4O(z8_>Hc2)JJ8t0-yOIIYzD55iKN!QzBo_Tfryhu1t?nBzQu$3r ziRbf$iVWRnvu!=r?oZM7Bjz03OPMcWUcHYYbFv#qRkb_3XkQhMwV?;(tORlE77T{X zCH#qbfMP?E%=D1foVKXvxS6cGP`{iE?0mN^rhMi(U*P8`Jg_f%tm$AQ1*c$KvBSG# zuLKx}V)t^bjr_t7riWYCot}}*;@j+z#)of6@mJtaUK(QlsD%WIrKHbkZmIRf-RHEr z1(iaCE^l+{7Ho3Mn33nr=I8gnDp(|?yXjJ*zj1k^J?VJ zFmvD zz$MYdytc(PPg#>wI*+@eq}`SKcf*1+AN@)OtZzRs*Nj>7pty4D z^wCrve8(~=H?nRIv(k%UGTRteP#Sg5q?~BhcxyoWzE2dwX3W*)r?M%gAcuZ3o^n6> zw1k}?LT~YcV=~#P=}jUK*v_n;aG9vAdJ%Z&d!MpJoy65@9c9+B zg-2`8fx9h06hst>Ol0CoE+#gf6#fcRF}J_yC^gVj54Nr5W!BG7_k@t!vw1k-5++C9|o-!;_VyIQ^)*AyCu=_TBI zgkM_-Ee7vAL%ELVd9Xq=;&nTtt6FuYU{gpXbGl7fN7#f_EI5oGZMx0}=18qOnn9p5 z^bnBVkWA%A!StD|4w91X+vieRPL(m|JXkQXyIjpD`6Qn3Mmr@m-$y-9dpi!(Gp|&t z_^n&du0sCm`BTHiF8`(Cz}%|iWxvC`($dG_SQNoGWq_-daQG@-Z)YgxaX(%B4>Iso zSOXt%DkplJH)1f9$iN7b=&+ZEwO|0!30PefK4?G+(9)~)Iqj$@^~*dC@seCF^Aa*K zD5RTzaO=`Cvr=p~8XQ-Z#aLOyG&I~ff>|{(R1KkAI7z2FoEh|^|C+k*Z`f(G>#=#a zDC6h+qXWSX+Xr`Us1DLzw5|E>GgT=bruKH&2uL)-R|oma*)4<@TIm2fd(}&$c~zU? zEUj_fzF6vA>58{QH6NpS&8E^T0@XLmKG+EOBK9%ma9f~n&a`l8j5KFZe5rmLz4?pPdw^coee9`aZk@6mTzEF{}`tJVIeN+u`QHrePz?<_~?^hh;u=mT8OU<({!4& z)zjE=?s+89hr{bv-)b5TV$a#m(nd++v0VMFtK;-xH1XP){R(~7>s)P|O@>Xy;bosZ z128^Tw_?@@w}1Y)okok~-jd+$5;v~ALEpC-11^d4cJU3Z&NE-0UGOdMNh$& zr@Z&xT$@84!VdX0S2$GiW2qv7{r&l1|8{^$7AeStaE(Gz2rpFIYN`2A8)1-?PRypq z0HA%)qE!gP(xg#aj_}n;QmYq`1v=7Zqt~B-V6FwR%Mt*>X^`umhoY}U_do!X-&qT^ z>4JtRmzd(BrRJF?uL!f)lNNPySlU}AY)r&!-Ym}JJ+J#l$8qQt@ql4D&7-j_LZWUO zr(dIPNL5paO#Sr4dB2b~uZ3^dc@(+H%)};I@oj(Mpk0ZX0K-u87-biq%X$$5z~a>T zJ4NX;{oZh)9*f%x*|q+s#$8260|cUy5!y+9c(K_n(Rj^ov%VO#QlyL%ySkoszW@AiTTr6hq* z-QQ7#sjGK=g0xtaw1nyNWVXriUZ#gVS zGnp5nW4h9~&G0M*vIBpB6lP|;^;l;06lTvwJut7o9441d0`S&m()Ie~<>DLH37I~; zj_{w$3CI))PnbWD&7GUx^H~>QUiSLV8xUfia}{^tXsn1^u*pKa*z7z-gzNDo;#dw8 zBcsI{&jlfBbHp#0MT&axuqpnNZe7|t$vExJv35Sz#q1<#r)Zx<-RW0t;<}z!ivZ2b zJYRc)W5eOqO5fC)BOi+|#Vj)Iq&NM8*zQ!|2Xw-ahxhFPrU>>S(HnueY`WT|MC{$9 ztODc{E=1*`Kd!i@j<I4M^^mXcz^iwH#WcSD-Q0w7V|=u2&A8yQD{f~e_E9k};1%q}{em`cA)*%7 zW6o^Ru;wn*5!_>X;7ds{LAtPiI~XpGPhu&#?EZutCHIh4bGTeOmA)VRMbB6}+b&v` zVws5hQS^w2J}$zY;uckfaff{$i<-`2bo3RYibo{l&1zMuaJR*+qKnR53P$6ug9M9}dl?j&N2!UnQYD{*mktZve5ZCXHdiHwnb`{#UTA7!K zPPnBF2CA}4;t1BxmrtLO#r9p!7#BQieYa`}W!5;beNKczGHp@?wg^><+ikR^OE@kk zkTYWDXG;XkWzcchkUdJR8Qq*e>8>>271ZPO>tku=pTX^yTKR0~9szM$d(E&v`7Ybx zT@E8u{Qwi3bE4L|*S2^gIgEm3Ia-5aeBx?@dA)VZNO4g~q(-tGAz=((46D4fd{6D9 z(~s>X^bWth>7XY)&{I?foV_#9rNO@TCdi(XvTR~IR6L8boO`iL4#TgBB9y+p@Kr*p zgJj&$G^H%Q)n*}?uz6*2iCpkJszGCkk>*TXRiRN|T;sv*OLLQ{WX+a2!cOrK)*f+f zzkWINaL<-9{N{bO#DL29S*U3SO=iGsZE_O_u&QA-ta>t<&FM2&x`o55k5IHi6}0#T zGoyW`kjjmsD3h8s8yj?q3t{tzSkXgz*FVDTCttJ zH2P)Mhd9M_YTi{T^*GQV^*y1#Oz>eiQIlE3gb>zLy&a~ymMgciTZP3-^lYDv)G9>P z-shaMY#K@QIaan>(Qle~4HK|l7UI6y^Z-H7PDE3Rp(G!iITR@C-WzoCK=8!ny00Hz zQl86xq5K}jd%sR%;YIrdqLX+baXC&-~)LM{bJOr@M)gf#Oqg*J%T%t4M)u zB6vd3Sf`({9Eklf-ZuaDNMgk^x;|0w8sh?bgYv6p^T7r~RVw6gUD{pz;pZGWBg zdIwb7sNQH`=m{%D^7OYj4d~?VO!?~GT(v;J&2!Ci!`RNw7ycVOImelizvK9x7k3Vo z3FTL~T<76eZ_FBbW&*`v2aAX6Gc7{jiF{hG?k>Fsu3KrV#KZS^c)g7?*j^a;5IDhp zP}03WEvLF~w8io?K(M`^(OJwX-#~c?R861~5nfjK^SmK|nyVxj_UZA6HunfFy+#le z+H^cQGIWIRQu~N4HQ@C<;U;^uqQm$XG2EmVBGnofBt)x&!RzPVCIRmpd<2V|5KvPD zpd%jjBB2&4d>?{gT|RY8Y8C$Cq|;+{T17?#9YtyvJVYL7PWx$gPx0@{RW2;Dotm_> zYbna9v>&Ly*dg{BPlj;J_t!A@ZyHo1pQC%dedZmNPWP4OqRRl8%_q+qFL2>V)$56xsN56sU~#(SI-&w^pe@olIPA zn9S(5Xx(EalTt$CG|RS7KZ&Y$Ea;_EkTd@c8FewA<99CXJ< z&2eX>AAK3WlCP zQK(oN*=^_TGotpzlJhpW<(WS?Hq~pZ=EmjDnVyeq;T>Dwu2X4WA={91Kcu+OsI-1T zA)>!QmdN6(z3AE&K)Ehnf&X8sDS=qx9KP29B_?S0&J zzOqhu>Y_?|zR$^P7F2t+GvAY1S#y&6j`#G5H}P66aZq=_oQ8f9`)l1YeJRi7`U-PL zfWX6DgQof0w0A=?oVP(u{7YH*G88-^~-WM5)O7SmzC37xDd7PTt%bcan{my1q%{n~Esbxf<}lz{@RrQI{b1Lu%RYN z!4?z=`-sGDf%f)EU--)Ect;#Ier))eH)j#0;^WDj zZJDxy!Uz%%2Gd1iK}vNnB1NjvgigMGBP%u@B_Ky%NHesRy{f^YBoT~{d2;okp4*9K zn-#DUepv~49l!p%OG>b7Qhuzt+*-$^wSs9G35D#5J#YP5kenqId2V;$jeyC|O6DQRKwz@GCvxV&E9nG= zKOEpm!VzT=Z-U!1A|*ZFgVHd55Tc%vk)IOih*D*ABctjC7A`M&Ff+=Zw=~`c=Yf~= zf6YIZt`)Wi;%NEGW7e%rZL0n@LbZYzpCmP!4{}b29j#56t=Zy#;RRyzTVP!&l+cH@ z1=_d}%@oR{jVC!J>gIaW@6@y7)E{$W`Eky?*b~&j-{?H=1YwAtD37J8f}Q09L*XZ1&N-`1A%^wqde45I_P{Ci2W|j7vmi zwde>ce(yW6xEfM-d1hOAePnU0)kawkzlu6^K|j|8aC%9L>CusNEcT~G6_IgAp1z11$s<^Hg+rb%jZ|44my;idUK|nBTq3YrHi?Vgzh?`!ekL5G zV=*ypBpT0ekoCeg7^bl(a350$Qt}gR4~exFyR8_~9U3suoeyAaI^v1Hl~k#&aggKa zJ3AWZ&}_4%QPVFcUtz}DZSFurlNwMJLFk|;1oT$!WZ{lpe4}3WHb2fGZEn~dLw>?D zG74faV{q%pZIbIt)<{U@%hg%L_-gT1h>A{q#`l|l2}(VNVuiOdh$Yx1#2k1MR%C#) z$sJaoj(S|(@Io)1-jYmnm9ErHcU(q?TI1 z<{{BANk!*?Dd-c#hp|L`r$ek#-YH4%Jgt|qT$&Ap=>cNmC;)Dwh5DA5pr4uA+V2HW zFfB(A*GEzxij~!`YP=+77-8ygj|lx5B}arN!ikBj|A|s`E!!+!h`A-Hfi>If*lf-4sOnLCqQ%N1eQRHrDQ$`x&je zasE7u{lf~ei@Ki`bCl4uV~((J5nuZKHPN(_Zv5Q!^w=Vx60%s0f(mwD;xVE0JiH=O zb^=UPT%{jbBPo}sg^i|k&BE5u<;#p0dYp#X1s-e?(_~pI30j~Cvp*bFJiRI*($lQ!Ww5FTo3Urs$`=JB%H>T$vZ-NO;ck@cq z#O7M~XMIR_>!|o4Y=p&iwPk+pnFlkuN-xSZgKL!J#Q|FnEJ^19I3u;Y&eA2zwW^@XlZzb zZBye``-qatR(PQ?=rmLLC~=RbdB*ux@8*i8gcl)KPrBL`cZ2}kq-C4HX_c4#(%K%6 z*~FfPvLww+`>u8f5@=8*AOTx3ZOc@7J%HN$y19&$0VK4iuK9ua^Q7DM^YnRN?mA-* z7R7=hk`+M;x|&nYapW5O!8R-3lW)hO-YjUyb>0wO|nt(rl~Oz*mRHo%1e=X{V{KESb-D z^@NRFWR@Vs(Bqaz`(&(u#1Zf5Ak%TWWBj^)#xSnYX-)1(2)$TO*3tr z(0-dIrH5Yj3-#1ugp@ju)w{x@BYVbxta}~PSN`o)3~^CP98ZVevIPt1H$DnGZu6*~ zWUPr^hdrsx4ByY{xmD6Q()Kx;NcG5B4zlnI(d(|Rsxof*{AtOnZ&U1Yp31sd%VMtC zatvX(o(S4-nUKTk@H&5rsBy9(YZHKrvuFWwxvO9s6`+Z~5{P4S0eLBt^ku~17f_Xk z*Z+VIJ~SG|Vixoi({D#va>he1AwEVA7+%UJxs61pJ0qQD$lBR(CZt_+>Z37>xfU#7 zP>t0m%;t5Z-XtCP3|a@_^_b-o2X{Z^_-;4jz<sRP##*$lC$i{1c7d5BTcjg139cYFEf=9zn15a-W%~KtKt4 zD^e0yy+3zM!Lrx(qq;kKqP5m|A9V%s^ui^;)9c4cydj5k#tRQF0|479=V?Ns!7VW} zzToyu)Ve1!@O3=KxC0pRT9fqIfm`g=vYB|Dsy4J!>JXn(c@KC+t&u(S$o#qoiAWF zvW3rAk3=?tTbQtvdy<-=NQ@CK)8HAa5cGH+TGh<@i&h9CuL@ z$TqE8S{&NLgR^Jd^(H@Ls8U|~7URtWg>u=6U9Pf+s@ftHTaHrU$&2Z?jNCxQlw%tR zka99fzJ|UfiZekgqSJ-nrLik>cqU~tv05Pf(tR{4HTW$gUxGrglUsb2hJ57G(W}jo z#-ULA?ui|RJ|^n-k71A}NytMMc23|U1wi?>b#zCCsJFU&-{Yq%$Th;bDr4tsCjOFu zFHcfEYgBmUCw46S?0{h@ACnC0J`>+NcUo8|fb{o$V19|fkMqjM>JBv>%J660Kpi3S za>5iJ%4R@pCt|mNt$k1&^9gVA-H)a{i&g|%ig+RT%IN1TP-JoIG|Eu1EUa#%#i`*Q zDkCAUCup%}0+?DdE2!zABpg4Q;%#3_MRv`1%6z(XypB~2e$BBP|55tJteNxO;#cQM z#>QTX`3VEeCGL`Lx<0DBgUPD=PM6{m!3psCs^zAIORAHhF7s+2N}{7`XQ^i>D_e>) zMXF4iJC}_l<;>WRyvN}q*oZz%9OU6x%+VAY@$v3r=YEZ!pze>09@n3rFvuMc_Ea~& za&sn{lnEV*m#34R%gV&sU`xITT$Kv1xO7=C?hPO4l{z?l^RY7b(bhi7C-lC6Dr@sn zHqzHd*q4YpZTUP6U`7RGALi;AS6^ju29-mPUN<4-P^En`zoM-!CVT%+% zQN|}WISEvpY4m-HpJeebqK$~I9i|-HUy|`&W$)2Bk>eMdth6?I@19#$#_-3_&zRqP z*H<|isGNre`xkYSWdC`$4<^%Y6x`A_8`HG-D4=Q)_YI<&uiuG4jC7^8mXj37qqMq_ zvN^6MEOdlfKP0`BFrv`HD<=`%YVN+K<=~6mZx$69oq84R=NqUU zGfi|foStUikh*vjzwxG%z#i#aJ@W?yuO8&-B;BB2c~VkaDw5TXKH5^i2%R+& z@O&`FXMhD?HaZ}+FV4*Iuav~tem*nvLw|9K`f!vD$^7(88+CEf%_3Ce{cr|^^abi}saDt-My$DHQ-3I(pQUdUE*5V1l(nOcY|Z%)$p7n2E% zJvRog5ZQ4DKart!;;7Rsm3g-=*pD`|NWEH~!iCjivEJYGJ)&+|+wp3o;o+rt&xHiu z%GEd8tj-4ZspCBM{^-IGq`E@nOZ*EyS#_`OB5(xxWvI~W^M22+=(nv6Q}Uz0?azky zY?74G=6VGf(lDXQTwl(@!n4)mGkvRvm!j1eJfDf%&I)~%vXmtagSL0Y!Z+8v){3=i zK0B_Ydvn^}2XG!>-jah21A3ix%8`9M&Z5p?tEQr08xZfEN@f#|aLz6NV(LR~Lm7F= zWt7yyKVwNQ40QMcR46vPVM@4Hm55={jxSnAMtzLJiiEL9Z%=lQB7x+YRUUP)uS}q{ zz$}wg2esqlFA<0&;$i?cmc_u57bFc&x@QNH96E%A%p%LZU#S#`(I_aCX-t3iH+*z# zpx~!0C$QXNks?mL9 zzZD?Ycj(Zzal@-2b)h8z4Z%Z!u=0>DveW&qcF(fEWdW(F1I$`T&0~eTpU4~Q`}R_J zwy|u-sceF*Na5st5^)-b2pX26Ii>GI6CbleR z51c&x@S_hd2z2zV0V#|VbnauRc;wjwKVz75DeZKC8DCjdX%VeCIIw11Wj4EDEe z@ux^nS_c6{hRM-Mkk8g=P<%8a+;NPX53@cRAtN6OUTDcr^8!WV*Ba437D8{iQZL3Y zk-0X?k*>;Bq=66+9*H=*I-d6v*YI^g_?5~H{G*q^qQ370YdmeEl_7uXRRTIPQ@>TG zdd*<_z{w-uG&aRvyl+Y2@X{B!;OXGXrOUVlPQ>6#awzN5-SoC>R5z)gE1+~T?VLZ4 zOTc$czoxgl{~2EdBCF9Dqn+HRg>&brZndd3zB9R%3rt$S^J5ssMVJ2F0^PvTtub1~ z(xvjq|E>iI%xewnYY{fap@6%*=M#s5=ETx!U$iJTp7UX<&1 zMd(U@xsVR0mRnQLTR*MHWV^gZGQE(OK>MYrC0N_k?Vey^nKm@EWw*WZgc7?uU&`Uf zKNT*90O6OPdL;??Y>PPg?-#t~;fxReAA{Avz@Pzo(Ch-=8i^xFG*P+a1YTn<1E3BB z1O(JLXz;W2++EoW`2V)Ih9}R7fN`?DF8Hz1!`=M{tI;(^Zyd!C)i&MD(=-9}ZpK^& zY&fMrs5oo|n~9ArhAJT^ArKWVWl@Q$;Fyp=E|X5=;T&;sI?!&3-SRG})UP|r48IHt zKV7A!ck|yHg+H0z1p|L0qof}h27@I8|4*r9t#T?E;a`NOXq};GIN(Fc`*|1yI zd{h}9!1h+L_aodyaju}?`?yOmkgCHN$LOk)_*tPb+d#xmqo;t%qoEl==V!R=rb}Y3 z%BstJ%IzNg+J*R_^wS3M*6wd++Dwg&AHeAqlzVjt>~^m>1GZm=k=#eB_yz;*9v#4W4TsCAI%>k@Vb0qoO;iXc`mvxi6}+$ zQ=LTVscY)$ApC9Dv6mTPIBGh5sqf#<2Atr0Jn1Pdu>*NO8b_8H~vrpZ#>gH`zL!Z(sXE;Ez-3kskpQ!z#hcoxFHL&c8y2X)X9F1>`L7!p+r-YxtX zkElQVh{zLdA^2G(SLY%aA}tJB@Tc+;O={hcjFi9#Ip|) zRq$@)s6m>m-fICB8#IACH`e04F;C1Q4iy8nG>>;wBF}b&u>aksjLQWE!?ye^hyR@! zw)PxWBSpWLd&cq-g2_O&`vHTk|qSzqv zd#3VGp8&CWA9#&=1G}f=h721ye@YErj;SB`K1Xcm-xj~&&g{tx__>7-uHR_I)U%%s z{o+;m*NG^QrvKiIvNz;K-thJMAmhJrdi-nIKR(kb1rnXm!5*ZE{LfMS>j?k&EMbY4 zOP%v!EII$DCj7tskeI-It=ts?m(Ty}H>iV#pTc#6Cx`XrzrGFohXVcaD_18R#rPv6 zV>tQOpBs(;^p_6`5|}6VR1Y59-yb1D{r{Via&HM}zi5?Bbuo6}(D*2-e~<9^nV!US ztyf|qQW!gRDaIMw-FSqu-?OEz*T{GYN|-j+D(pyKr3-^0 zL{*=EQ1FJ};{jC}IYCk8ONFPz^Gs#$1Dj*skaIC`Ea)(3Uw$xp8s5e{`e@+8%-)O4 z>pSo4!&6_-oh$sjdiG&`N=ga{9UBoD4ToAH@BzX~F1j)`!o!~U!IAi@EB=@C0Py)2v zImq}A_54?Nf>PiDt~al@d;T9p_gznd+Q`Xd9smETVL!)eU$n6c~cvkuNNq+@BRT% zay5h?7|p&w_-}}?(gP*#a?c}HY5qFGpj$XlhhErUa(-3!JQCiXf{+@|->ov&5)N1T zYnF&#g=@;%Kgq>I_<^VVt24@Xy038NL6b!Hd)W6nuLjl7{$Xgdobdhu28vUEH~6mo z7J9krbT`uWWZdAb>}?w^_$Cz`m2(f<lG)a8mv z-Vc8-Tt|r0@AT3f>374%!tjPXUrhg95hPfD)>S!E`@631+Hh+!Q|-b1t2Ife|Lo;Q zGTz`{4eyl)N&tYjQ_6og0aSGfY9JT1=k>43f{^GE#zuR;e^2U{5&O8R^yoY%@`6{6 zF~JB4LjMmEl1D`JDMa}F&x)@S08f#)Opu0t&VcmrNPb}}{xJjqbT65IN!^CgrT%xA zsa4@>B~17mw=5LbIeTdOQC97`OT5|uR#mJ!?v3TrddCWoV zAb*)om4E0)1ub=gQRPw0FW|l_83SPc<{Y(s{XbqTwg?l~XM-X}soF@9FwH8ju2lsI z#`@N->9}$5E3b0Ji5MoukmgACVjG0tkVF}JpY?ts2=gl36Bf^rEw~ty3!+sy(hK95HWD!X z#q#&Wa}DF3&&wV!N9V^c>vBj^w<)cl|ARcV;aGC2d%yT4Z(Bw#aUbr`-B6MiepyBU ziIrcI*?}4c_%#2h^?8^m%1Hq05D^R1ysU3E^3EE_M_33}l7#*!WQGCgP)?GV6R4O{ zd=9tI1?m!}jT$sXwmQb;xLxb$3Y@azioN_G+mN?gwmI>v4j&ql;#QU?^;|ZR%U6}b z+YKM8?o@I(dx{Va$oN{+z}<_cXE`M?P32-6gK9yrun;1=ziV$a+;!@~kiY%%onPYH zVdik!9^#3ui?QfW3zJtDMMnXv5}*o|&s?_vjLDaMymK>k2rw3gP+% z#$tS5R!1bk>?N@mQ^-vpDk!c|L$+@?S>R?PD=ke=G03+u>v~SWW6hnsnDiV}DG7hz zTL)`8DfL!PZ+Kb@3>D9SYI2`+YdO-}~TCmUp#vkkKUfx=nq zjjz)A`vX>^a)~>`CL@@hjkEtYHx>U)napOt^`k5uLv0)$&;t%?J871+OB1qCu6oEZAkOd;fC_7jQV0`6)+ z;i|65-PJNdKlm_S7xjek6TugizIE^C{CJG>IXzbk<$bl=5i08P_q}{R{z3Uy^ufZo zbeGw^*eYufs>hkp8EzlE&mgB;hP>^D)JEY+^6&mG2InpKlpGGi@lj?dM!e(i!+%= z9;J$_lXZyK7F;TSpLzq^tKe?`Lnr<(6u*%K>2svA*v7}j4N}bR8aRo$&gri$V0w7I zFBppVHz%&z29HqgkNcEVw>W&hovgdA@z|`~TKrLga*`pa`wvq`DH}YPTwk8?{Mm-J zNziqJD+@kPmT2Yp@cG=feT(p=&RJ?0#A)|w0yw4r;ooum77ULOi1yrH{SK*We+FW@ z8b1F}_z}FY<(7=YZ{3H&I{@CqQWgEXwUW!j4JBxg2vz2{j8fsDjq`cxC;GpKHofp^ zlWetw=r>~D;Vl&$7{=25y>Kc#@NIjVx%38%yryOZ+YCsB ztKg?*^4~MFKpH%(@sGti)C1%7&qSzk|GhUA@rZqQu5xSldsc|o0eeVqeuW%CQk+L{ zy3anNo0mZ5yCdxT3I+#ex4f+OM4F2>sK+*z_&g4R|9rT;PGTfamrttT);fL z)w;KULEB(Mh}ZjH?InN5Z1(W+fE5elcHwDq_nW`xS{Dp(p{4w(b0awGbXz6qA9DE(*3r)#DmA2%t3of znL@rCrtTbKRAP-LlPD)-Ro{Vm5}>zFR`68}jx}J534t$lR=gA9A3<{6GcQ zTf`S2-ZaT}e0_Z^kl0A@Hq~$SAnWVN@uS{ZX)S%yh9-aiRK^YXH(jl>1MrCBL6lyp zGX;_d@!5UOdF>P;RVoat8*BFd<)UOWj?0Ao_oaDP8;1eaNR1C2X%n2N>x%v=g^Hi~ z#zBky@2t=B>HjWDtUMbhV@}9X3W7;g4*RK;_f>}0+;!GMjvvK|MZK%0;0`!-*Kb|G zS0Sd7)`I%$+YEJQxj(whUXJ1^_lG}4rrb9|@B?t;&%8XV!(ili=zo@9bz&0&L!w(7 zVnAIBm$(ab59#{57Wl#k;EVtAw~1$?Y%oIAsM```>w$*kOWD2kTA_zzs3)NTw4vLV z>S=~TAp@YgbsX`*AlU4p>7y(*`-%j_%T*UEXIhkt+a3yxyu0I|ax;$Pac(#IIFJt` z!$!)-XU1YD)h;{z+X5-F<6tAhhPN9VaBh8RYH{-r7?v_o#{aaX|8@ovGVt{5-BHpn z`}nS!OVEGW9i6{&(z@z}*I&rI>+~p}d4F>23R0)y3q*K(~GpFGbw`|R@BpS{m>RFD_ukVjfp3pE9xJS}mV9egJfpYWwK z2Uyt8cuvL;R7(-F5lhiLH4m#)05AN z^Ssyya6sT8LFk4v|HDIoFD#t+yyLHWRqv#MQ(Jueg+(KkwiRy7Q@-FhE|Q=XxIy21 z?am)?@xi1YfT3?W5e&g5osaZavJQ!K5<_P1fi`h+e>srn6ClOm41)Rs{ zZ{L9?6vK+W0@gY*^Pp=}>fET$_TGsb8-m?&RZlNdW10shXKJ{4paY)DqUHA|=0mC~ zp-3oQJtV{0!NH-Zw6xU28C}cXBBNz!`*WUz^9&f~iL%gbYf!ntAJ?8CxY*r&t2)|` zZcd77$DPJnye>(TG>27h`FWg5+#Fr#=1p#vZy$Cn*IIE2^ea<)Z;@flj+9tQUK{L! z^}i9;zxp%Zw`;e;F~9(QXf@c;s@7j|lWp0WAKWMnsuCFMtPUc0_hgArd#d|I~QbrtN4T9_Qz^%w+WYjT?{UIQwPG6uYlZog!5k<=?<=fY|U&b!bs7G-iDn1;Ef(J`falskNo~d;mH5`^|nkY zLO?VEsiQk96ZlM}go(87eeF}^)>@>Mjg8IyOM6oML4hs;KoFVKGx|TuX-8o^K=dCn z77p2dMrkR?r`@Z!3?sK}6`@{ZANN&k1yhVRe2QMW=j^aBKi8Ps#&&*aMJjGlz0+O6^?w!=zrr!K0-k)OUDJ5wPq| z$-1d0TKGWT>?HZR1H3$V3%i$esK?;Y&s0|m;D5{!-CH}8iUA-hA5Gg$?Z?}R{4Blr zf*S`DcSa$^;5cUV0C%npwfia8;LEVGj;^sU(==vGGwUxPtaTHOpP9x$B2YXh80AaK z&<3R3F9%+%mygDnsQ+LR43XB40iE(Cm&oi)j={fGbY?Szfb75hGs~a=4}(oa)NMap z1$|%&XKlH$mV1nr(= z3vEBFmYB2Nr7*JfQO#5Aw*I#KVwnB8{g#WVQKL_?YuC{bFWea4bNyMzC z-nJN;iAI)JJ{993&w)2Cqj(;Y^Eg^$bfC6BB@kDDePJ+$c*YbzzS>QW=FqLQc=FcQ z%5CDYb9t5mP82YQ!Y$f?e%Q9+;5Oy>d7{@V$PhxS5ig_rQF+aV6TYd+JSC{h54m}s zJcgckwm+4RdXz7uD7!AC+Ue1-+<|spanw;X_d8;~NVfUg_YC|I0q(2g)EIAwj8Fgs z&V#mfh(D6e$v&}l#tKFWo0aW8-d+)|bIKpejL-4`6&)X2jzp1f+{2Fm(L(^}Dm>O< zTUYB3{|p;uL{0a#vxm~IGNTDu7wYu7?{UH{ej}!Fw?uY`G<|G+w{QCFG?U=%J6Pz* z%Oi7qd~5Q!+b4M>3@2t2JuNLA&&D!U#;MiuX{3!2#KcC7(~3)?o$<>mY>om1UXp@@ zU4QiDIp&jmFTiL&=uu4Ag7iH}FyxYci{W}^F%7b*`{vIQ~vRqOALL)Y`wyjG} z)7L(>Jtt-Oq@8GK)*IK@^)G}dr1=k4VqKE=py%;8N1BeKv?KIvc&(Uyyy|clywOw; zeP|k^sA-JammUI0)x=6yDfrhxeftoYz_6>f>>cyHdGs5>Sm(t;6`>d(on9ef(Nj#o%qo4ckNP07_kBxX@@`a8Ik#P6TV1VbL(0!|DcjUg8hDG zY%$bN8M=h_n;PzFc*H4!T0iv^n%_OrwU^W}iZW6AiN97^1pd0LMDOUfbd1=A{vk}# zLEXUxvQ{dLP|ey>&18GWQO^B;!#3Kqt+!6l$7K?WVdK%~oHyQ;OgoUsh^tWCW#(x^ zUaLq1;f|Z%_#?r|>mae1?T(JHaUbrH-c5z4Cb-6{AmwQKxjz*b0A@%^ruq(kF=d`| zsVYS($;+#IKEt|9Cggok6HU}e8)6WSgk6Zph0D^T!LSlFUpLYw?e#*y|Z` zalcI^mO#rwf2IsW3}tZ2xw@?kkM{r1xRU~s^M+nFE^x1}MX-OPh`@aes_)M{=IwiZ zWCp>~V0QX8*WUk^!sRc4fQ&`DYwr#8n_n)A{TZwBMh4)(v5kcKn>Vyx#|wgX`CPhTIm?R8Bn&SPk%!+3W~dbt+d3C5&unZ!B#s*MKwW zWAc8)_$Gk!#4^|A!FR^Bi!m<#Ttw(!3dv$K_ z=Zt!P#X8tczq)d}*gez?ww6Nksz$rO4R%=ru+phFfaJA$q7uF^v;R@l{gRqMpcib9 zGqj-|c3`S_uq7)9yiXxKLSNv$1}McbPZ_GVaFlmUlN%ir44usEuPPT0nh}kCt%c_^ zI1PjZ_>!k@%`1q!>iMTrkD$$0HAwVYS-z*}AGLeU?M=JDR9w$~z-bm!dt)h9MSU-) zTw4X=mWLFil^z6cX!|yJ0Qt$*f@OI>>=}{tgU1-)1Kwq`GB<|tJ*N;T@i)|WyNZ|P zbbgo)SqoxW54-E5&yQd*G4Yksb;Ztvd^K^Rwx{#5pQFdc%^`f?3sYmBv`A$5_>xs_ ztUX0?htf2Qc)O5XnaJ#2Ez1E>eipJZV5WE?CtM8?i7YR*KP@s3F&MJ0mhq9$71z(M zw3bYPoQPPo1XQQw`c{v+m7|h*_&7m3gO$9eTVddsuBTx?Tp7mu-dbMw4)nw=#O{rW z{p)^;wcQ*}27%#CamU^}+xNwBGK;XcIQ0`^fh$Y(2J6v*jx+hOKp_cS750R68DoL! zk-t5^@)@5}x$1Vl8XEQ@x%%Xc=x9jyW!YrhBS3kHu`LNg-W)lm!*^9@h|;SG9;wE zuAAI+&iQOGA!;|3X19lGc^v`HD^re-3VE?M&N4CB+czO@j$FO(_qo5wy-ZJ}#d@-} zH`&Yo(_Kx)lqV?#(cQ?Km5(=6#`$}SO`Rz*g%fojswxwQR{ZcmvqEE5FvZa;o0pO_ zhs!e@)XtVgl&Q5~*4N$1?0BiqG z5rO+)pkQ>xBEe~ibOh!MgD_*`1i(~ ziZX;fj750068jKwC@1&4@ZWRIy7kECyD6piRdoD;->w8Fwq>b}hwX3HRo!v}#x1mu zlA*FWa2dZz#z2z8Jh4LXPhHC1C!n=1XR5=L8%M*o_D>m}R|KUwl@b6|e}*6bh2Me% zopR$La1aFRRZ<;%CTi`oKA{ z5HciR0(|hBf`0T5*eb9ZOmDs|_Yor;4+MY;>x<04QHh7ANclywz7tou=l$5T(Eguf zn5V){xhCNJapO9ZtvZ5}L&@(pNH$nWCBH2Wf8u)CrheM+0Xy~SfU>klwct$4;CXz? z3ZwQMwZsvCLP_qfyZL~~ONwD7;gMv0nqGe4g05^VoUeVC-(BYQ-ukk_1HKNNEpaNA zum@*g>-Xf9Ydr+D1`)h*5O@YlPu{Lva3+=hi4@@XQ*Qtb(;6yD_%ZY81qcKxW3rcl zK`USI~`S>x;;omKzBgkq!&uLV8~ky3S>FORvoFkNLUj4c9d zi!!K1ZA=20$G1IS8H|^8Up8~gP@$(y`%BwV_XYZ$PiLhxPmbsK7SB9@gd6IhcyTl(@b+`6Ca*!8(a|+A>I^K>E>^j;q7DGpAj>PoGikRP`gGPsqyL_DmgjyLzPCi>4~>6ig78Azr<;Y zqmhE5&321j6F4GfQv2R3;dE?iK0u@y_xGSPJg~P+SSEk#zc2%n@8U08NNaLf=U}TYwN1oCuSZmjnp#2 za}dGRkvw7U*F>D2N1BcQGOSQO@W6X=S`aj?vG>62BHt8W@@;M-#toSom((DFeV8A| z9ljtRJ+8_UxS=W>%fJ&=jQG`uFuC&0MQ_LMUMMh@04phm?x|?(XIskOqcvY`na!Q8i|&r)kt#5WlkEU% ze^dm}0EgEQ`LVB)6L2L?;Uy;9o(J%HK=F}bQcuAkPCfNJ`c3u0I2Z%-5CrHwIWGST zdS@`xiSs1*&z++LrkT)%th@sMeJi`a2Oi0n7`#;hz@_9NJIYxoa52XJ>mq z0}cW|!KgKB-C=Y>s2l?jJMp!I)Q^Rlhfjb}p86oXGqVl`le?+he;K$;Q2g4nbDwRQ zZcip~6x_yh=Dr=nr2_PO3?QR37uXqCBIH-VthYp}PGC1NTOREO{xbnofZO3%`1ts^ zNJYuBZ+F?5<912^0pafAPLs$Xh6MJTF!<&HwZO~#TNPUr>{vHzctiyAl;`jF1$G|! zb){3|fv4;ksToq;{)!Xu>tE`%Z*+DCP^<&kx8|VqT%Nv7Iw1f8qs9ezs|^qBG}C^! zFdy@Pj2GREt+XBjW5U(3g?=jY3m ze%H|uxCbZ+yXMgUjIt!Mzn3_`kvoPEZy2Vv74%NaVYt# zfa_t5^!B^H5)e}QUnYFDlFvP0&i`t_2+RbEBV!(Cdt`fV|3bWsVSM@o?#N<+kUjlv z6=;mU3Ni1MoowX{;|9i$Z4duPIQ1I98lVZC5nspsYl1dVUpModOWR-F2>{ZfO%`ky z2N?zu7{r*lm;c0*?U`>+XCV$`KB%d_?e>d&&|a%nS|Q`_yO*SeZREWEA7yxRg;583 zX}_Zij9;hjWh}+V)`abAXn(vpQqjd1nTyWeG}W=fCm#bvy|lEFG)JcA2t>F*{A2No z0yy;5&-YQv{#R_*%wnoiZzg za}Iv2>ar2+Y>KGpHN)h6AL4dXiF(aJ)qO=L!~WwC4??-6wN%J31nobF;)6x7$PQ zBYWmsz1B|$PvUW*}20#%E?|N-lMIP(%L-=MQ z@10gvJ?%?=@S*XcpRzbl);^NHK;w)c?X<)Z@d_`9){F&6V>~Zh)b&IwXQ*}Ms}VmX zFY#QODv=O>v}~rCKO_$@z~-l*&4-As3-Esee7!q}TR2FV*cK+Bcw@aE-RLEFC_3OG z><2!9$H&y#vW$ebz(rkbCeILT%qI&g5 zBN7)m<9vO=Ne+gF?^a3IiJ-*de;*MQ(oB5MSK8d$hjy0Am|sm1YcvhM>esHOe5Q*p zB=Fyf)9p&-gm;wjV^pta7T`#Z3ukx(*sP&63q=|}nKU0ViW`^L4hH z)$~j5NAOmjbf%(>)<417S~xN*b+G)`Bcr-QW|fAIS!)---{!Iu3qS3GfF@96%3sUv z#I<(5#t@1%aXbU$%JF$wP{ds^NDBMNdcI1cC>Os3n)W`u0U*ilRR<*+LF(E@* zU+moCF)JP2H)g!u=`M;}afhjpdZJe!=cM%(Ys5|Fy+ScV;Op$EMbs@66~#v%DX8&FTUI@5Y{oy%NL zv-73L{}LOi2$=KlTIm5j4RqXX)lTu7feV9}d;Bp6!%IDq-K##nKA0ueBl$UFs!HBZ ztSraJIH6cMc#WR>Zq+?--Qxr`csFQmZ?IVY3OBO37%;sgi)B`sni}#ooVCA1~y-jd|KgGT>&D)8yK(-IXftbVGD{{qW5%ny=ijdgVOf+VWZ91q5OA?3+QhrE zcW$49&Zch(RZ~v1s5zb)vQXlPZ$Bn3?jk)AhG?p<3KZ(S;3B5QwAJ6L1v+7L){j2kHlz|D$YW5Zvvik{33Y@(en^GXPx-?V zS9s2q{BlrRF=Dj~c@$FTDz(p1yLnm%(ahwehsBINUs1rH8vbVbe}P#X_c7-BHn9aj z(4iy*O}7cUKz5@w&J z@)>^u$Suv)8l7gZ$=kJ6IpNPcn}N#@)%_6nZbhFKH$OPnJmg<2G3mTKlA&L-+|Ymy znI#C1K8MY(fC!hf`_*cH+6ym1t!$LfT_4eZ?Y~iJ2dsc;`MlIOny+3SETsI=1KYqM z{#R0!Stw2blwWo&cMipZa!Yw7wrRLRZKkNxlHXhi5+}@HRGE6IEm@ewMZ3wlm}lzq zhIOK`QIQn4a30gexb~*YIrb&bh85(~O03e#vXutSirz(Z_Ba>zT6mQYuJ^>T_sFnK zz_yAkPX^0N=ber@{j|_v>Rk9PtwGVO&g~^pi>kaXxVa1i;~zdc z_#Y6Z0$vdhFjG%=!dLjJa)${ZMIQdaom9Rwvwn`11bQPBnM%LzG5mBN{7NkC0pB-d z*umH**$L|t-u~+2qg?`1T-pVUoMKzY0{4I-1|*ed^3Be_8K%Dw1g7`3Wi)}&m=a!> zqxIz%pn3rkt$>qKw698fY{{Q45H-DQ#kqQ2SNID0Se({># zV##ur29emz*X%^^chVPG{nAdngNZ>#23hl;+P`g#^)Er*#~148^7N}@4joy;B|1&; zZ3G3}L?^7%Xp@B-F{s<7JO|N&B>bE;#i>^qv3No#upZ9-P`L1TCPwsR{Te*v-hj zB&nx)#;o6Oj4TJY`_HbTue$u@gO^;OCI6O~E2yo6B+6F)YT|B8i z+fV&?o{68t6CK(YKRx{7(e&cFxiZ`i-F0pZrBZE9U>V zpCKarv>o@Vu0&u+mqsQ>1E%PtYuM`Ky}NI#oBfu{?!O9f6knGayEDq0DDnca5cc6- zzhyT(8V63U?Ko*VNZekz>#bj`9cSSPk z-ci=h_|tz4Z+XzOEN%~#=vBSO-_w4Gm+UO=u%!y(9_R=MRe-L?6z-O;NJQv|jBz6~ zP7A|W6#Yb9LKtD;}e(JmHtCx+>8W4mchoI)<2kspa0IyciCa*ECVt8{@U}dQ@ zUoG_e4+HVQ&sl3fb_ZeV?xI}tHf1Pw_UTSPlW6lDiuOoyn54SDD9rEoV>^zz*v!$U zZEfSKNq@k_(3cby>3zNeOWGO9p^;-=%NsB^&seA;I_%C3Un4y9lN0A|mgY7`nVAu9 zxbq_-$k}XTe^*wnDGWIl?a=90NA0Y9eff%LQh*xcXL%37%GSohkK zT&ULO%}rqJFmx~+`0sf$zL^K%aQ=FNZJ-J=yIR+Z5#sLn2CDN1DDA}(GC$RP7TPK8 zxbR~EIq>e`I;}nJogi(t57Hq(UOT+8W=5Q@ZM^9m-!Gilvo8F&Il`B#$Nwc4NVma3Ul zGGq{Ww1veT#_A8sQZYAhzM$p%;lm3)H97|Ho7jVC!cTR$L=>sNZ+1WSoE#s`HcAOZ z%IqbrqB#!q)Gbxy*f|*I&Vy=l|L1S0?R$gYX{*M+as1;cBu{Ckk`XLF{jnD$AYNNJ zw8ku2*=h!@ObxWM)7?;OJ!$gkP`;}ep{G#IKtJEArjFgziG4OhwD|Znd^CRT^HB2b zr!YCKxUIJy=xIY8V}zD2flo}c4(lq{^iOA#onXW@N0OV_3v;EO0CIJ|LY`ZY3as&+ z#Lr^b{sldNoprO{S0#fDbdB8;jyQvF;1P5cxyL&d7zv%X(mVbMw_R}%y0ahnhHem@g zq~WXeg0_{`E)c$(+EW~e*9=!mcv8Y+6S{`x2u=HF$KDdBHa=6R(~v(tCL5cqz8Z`* z=EsuzA}4375}mylZiO#r+E7YLoF!aEm(p!;aXpgcVI_mz;LRy6lX^)tv1j&pB@Izc z8gvK;8|phGo&6vDj@#r9UM>M+HMB9I`)MjxmpQii2~1VY|2F?E3;72A_}^Fnp^l}; z#{yP6Tc&KA)%M_jbBiB(qG)rcL{jzlPKiXKx<``2mvL&bk>95l0>2KGQm0#TW^B4s z$L5Foli3Yh(vDG%_a+UB(t?;v92Ok8>ZOZ z?^`=&)oQ~rTB7^hm=A2j3Dp|s;Ig~y3s6N7)oI6T2zXdMmvLB+f9tlr#~0$DAEa-X zP_HkoUf-z%!>o4pKg@Q3(c#qfx70ZR7bVp9?&Ng+><^a# z=OXrtm(}3V%xHTlvU_H(ojc9{aLS)#+^3b+yK<*zI`87MgWQuk!PZp1$I}m)XSBx0 z>BPJEq9E6#0y(BEvBT)7N*s6Qn7mQ*T!QPH^~pEfaJ$YWpHR$FEoV`g3X-*LZfEc! z{2ymo^u~u>01osnKQ%h#1)8@{t^Ml0xT;%oUC?PKl__9RZ!+VFcE)7O`EVg|3n zIOc8}lrWKi=AS*2+a#6MDqkAd;oPx(#dJbRP3KgZLZPZXXt(~mSbke}DhKvUuJ4JF zi4}QtyDvKgu538pd$3X3sNNfW=kuAI&MNfPJNS>gcf&VTrs4MwBi2H47~a@B*%40@|Lyqo3ovX`z&Q{=?9ZDKLho-l_mZ9Y=PXNzZ{Eg z&0G`mu5b--W0JP7D3Gg}*R-f1+cxxrY)B|KXnXxYCYJ`e>1{Q*TTnkaVPMv^OuOTz zebC+{nM3=WUd<&bX!XV)vJWL^1`cTH>5tc*RRn23Z}uVEP&@6Q)WkeaOwyV8kNc#X zrLq`T#DHaykN!W)@_&})tF`>#CcZftmDFhlTZNhC1m*Le`Mj^b2@WU|najpJU%&YI zhmS76!NtzI9Qr8|{`QDKRgheMwX>hN?JvW5CcQNKP ziTWf0^NRMNb>hKrKGeLwdIsbrlQ>Pc$XYII`F1;3GVu}4xYScmI9y1NIuySoJ6Kfk z>jB#{Ps8F{i7!$_Qslb($g0a8o{j2adcti@Q;XTM-H@ct+_vIFu%&c9@&RJ&l%8Vy z@%2ijYTDz84@E0dqH!r*P4BburT(I%?vr&_=(p;;l^^hd7ys+sQH~jmX`3$?yHh2k zV=H)b*iKtCMbYnQmD}6wq~EA+yJo!>HIt0pcNuw+ZfGib8`R2{^36xQ+>~e^b{Z>& zca4s0L{G?}kuFghGgfLz-jkY@Q}zW_t5OQJxTyz|lr@g-E$x8yLKwL@V%<3oleFM@)zws(edJ z;z`aG>$2+7+RN$Haf1-}fEkq*8&IfsSQr6Axn*L-%(*sUt=867w!-wongo|ulONWx z4h3?@TmIQhLd1vHpi8nl4JXP9B)W&Ob>oDzI;gAV=*sJ!kMFEs7A)yGO?TY7-#CXz z+(5c@7{hz(XYleD55gJubAmr%~}=x>iP)G z-od#Mb3o^em5c2uo;$v7DYKy|GW|(b1#(=~eZJ!_t8l+Rv6p}BsC_NG7DZC_&vosy z&XrG5l0xDZrU-t)uI}rtmwhljyv;q>dFf4Bw|ZiQFQR00J?5FBz-JOr;kb$t!5w{l zqsp(~st}P-PqQswY#xc0-I|-fl~XK; zYZIKmDqIne=0Yy%_MCD{@1o&PNU*vF3vh>Li7AKnWG6?qA_c^64)N!oE9Yvw--4W)Y&dcZp3tNevK%@;N4#5?Vq z1Lb_4-W{zwZ34#=YyDbp6vh_b8n%eG&B{nW9@rMp9Q=o=g=?{d^W1FJlkTcDKd{4k zmmPwkgaz^$t0s8PvUUH?@|n3ItL5r_yWw@XB%|wu`x5im_E^qmH!vk#81pU?KEd_g zqLd)r(KiZKaUA=$2G{#ZaloII3pw*-YNIg5dEAhh4)*WtgF6d6T5hg zpY~au+n=m$mhvKuW&Ob0YswECmm90>(Bj1D+sk}mFHQ!CIgTNXnIUNQ-1ouoxv>I1 zJ`0;L8N$7Zu@P&c!FZwaG~`~Vc>uQtA;s8m%AWbIPSz0*2KT2uV+p$d=|#T$6Qnew z^;89yk_uff4=EotzAT7a2AxcHp1z*BhJ8=YSe4;!EiE{Dz~k&qnw@qw)TtqD2GJPb zhdpI07Wtw}IlMg0N%g^$g&$?pwDA`hPL}2rGbmLlR~eo0S~JU#JA&D z@l1Uvop!KN1vj{OGlPBE1~wC-lTSvjI*w_l8Mlu&515Vc4Jke?1fa2}1&*Du4=6C8 ztd@A}ra5=CW(@kp^5H#=HgAk;lwe*7*6DcP^g`TTX=!|%WdX04@tGu&X^qBdIKjg7 z)%W!^i^&SMoTJaN9o}ZVDE{eMP!LVShW34@~!Y`I6>`v zJ6buc19K$B>Q-AJAp*f-94p~XWTy9AwTRKB)eJlBYb8;Ta|F3FfYI>pmaUW1l2{&D zQs6a2(?Y8vw;J4W=bH=)ccsJ7Yl9by#W3I9Pev0-KDL+sDp^^%<8&<4#I0d z`1Cs$x$q-Mb^&|>8EL0|-0-~xH$cwU1n6V(2$<7x)U~>~FmC+&=UoQJ)0*gcbSdjZ z6gj4=FU2%+;;q7H_DeiLt%9jX%G!3iy!7r+kR`jT^Be5^XZ ze^}jxoL3UTIV_4 zewr1qo@P{yK&pJ|J`zY}IglnSkLG*An+`b=!*?LM+r#fpl~YPM_DQQ)kD++6jY)Nl zYkRw)bm7uWKCNhZml)!&XH2<&Jsa7uR}90v%9WL(&bbnqh!*4RXY4vL@vB12ezk`W<&-IQJpI;c^X*x}Ez>}GGgnfhn8AmpxNN>ihexFu zVm?#3U44>OgoS7~T+&&sUA2^il9j5&P1%nWTg9_mcm0EeMRJ1bDVQWPO{Ay911Qzv zo?v0)DBJD)LsPjlIg}P;=~IF(5_c#Q`x1|VnrWgASfi!p$kTRuoM)dQYn`*}E^xi4 zVA9*%Umy0%H=R0MAXoLSH#@q7SVyrp(}2#-nsc-iZjScw`J!;?td}{^`(M`dp#r1> z5R(Zde1eZA3+3!g7Wox5FeVC*YC|eMA%#B`!+Tcaj>JT-`%rnGB z9f`^j9ALDfH)q%t-0^`5&4f8eeq{E_K3 zkynRHA12yUX+UrCZRpTAwO9}BhTH2K=HBNuEi@4!C~bid^a3;9v$vmKPLWvlAh-&b zJrBr?Y6k{6-6?r`>(c$Jn{#%ZEVvxynHY+CiC$T91ZBeg^{ABfAsYdU{;7nLY6Lns zcR6FF17e!*fB_OE4PHz^o)0+7h3}Gc_M98YS;ZH~*=fyhRkLG~q)?{iR0Zh$M-6lF zrp!=9Mbdg-42qVo%mGjX8D>vbZo%%G58y4+KJ@)-1l-wMnl~0Wx}%o1%KZY~E;r9l zNMe-$Ki$WwnVAYpJ=!ZKR=9b^owLA#GdSrCcniiojH~x~RV<(ISWOK{wf2HmpX&U9 zMuW#^&a&lm>vH}^w-n+k%#0E|mzIx~$!SMN6bg$s@xq&Kt`=Q(=5;^PVm&h%W~Rpc zP9aXCP}8ciM`l`on#I5_&T!Rr{av#G{e|lLii~dMI*i&R6V&en=5{>-$E`#6Q(AGI z7E8(PY{a%VsBkt$DUar_Uva-H9UNw}Y&6bddW_hpl28C!q$YJ@%Oyi$i$$U`J<&?c zy_*RoHc4A>9Tq&6-FNzr zv5(i?@u%$5*{%Ly7K&jPJ%3&6g6;iOrUhL(4?N-Zc}Pefl+PKmRjF9Z+A z4Af4~mpKJlrcJw?kuXnI4=c}jp`TY%f>$3-qSUuXC`~_^UMKjJ%E4t;Hj%x!(!_#3 zas~_gt74d|U!C_Yix4to;D~FSult2OjyXg36m*f*3Ts=5VdT~3^LHH_I;$oWvy7n1 z@N}Ik^H$lUyJkvtVmNKFri@70UoS$#&VsT=1uT{RY8wN<81gTaSz&sYD9 zfYO9K3?mxRHOh|XTI^hhv@mnjx#6TvLaf%6r9IN9;hv*^kuitUPaWrLYF#XGmTNOs z=}D>J347n^x|9++aA8*Q^$D|5gBQv(Q5-j)22APNGwrk-yk|8ssXXC|E-5RUk1aff z^@w(=Z*`Dsv`)ws$LVa`*9-7Ih0|vLBXSbId+Xs6B}R}d|Fvf(GwNOmhsAY) zI|1yCAu#K=rAZOe9amuwlppjPPIL?g;Tx1Eu2+9jC~}~(n+{X#&+o>o)(-fp-w(d^=+p9xLukqTI*GW8X+-hh3jDU zx$D9$VifS2d3L*IChsEVK<84q6+ETqF7aLb0Hj2DEWlM-q$uhy`Dd|bWQ)oMyVLfU zJQOI`!-iG8Z(NUC)cu1UWxhG!k&? znoZe`#d0*uLp(MdI-io0PBq(>7!9AyG@N8J3X;weh=d=$q?;`;m(Qy~Vyo#V)3WRM zx3q;@R)r`Ly~+{jr2}>Ow&%_KL^7?R>dZ?0Px&+wbv0?G%A69vH)L1(5^gqMw3vpv zZ4HyXKH}UY@~QK4J(p{xWq8Uoaw)9y@asOfe1*2-Q$W@Z8od=0Ee_5@(>{g!OQ;d+d9`LW9 zC>U9hyfRK1;v_kwmf2A#gyMC=Kgl!C_fk83HPc2*>}hT~zlHw}PP;f74f#UdtFpgN z58LF(N;61PmkAdYNzt-%!{B7?hZ`OWR`e+xaHhY1l0Q7Bdx?X~=z5x8CRzwFanF0T zdU#I&s)Da(MtcW>U9NU(I)ru&Y3`59vfRWK zT6u!=rh9yCZ2p?jKAim9%bB9Y{^i;f&MLtQ$1sCvS7m9tF~J4S=nvZhbx^SHsy9Tlw|+oySS7;YsfWErKj_w*8h&Q zIS>=tjcm`GFiOpO9hR&LVJ!p&<;lT45zn5Y}^N*i+@eqVBK^`D#$%*zh%Crndo zLrHg6cX@7}@>g>RIJ@GS5~-`^k*TfdIYDZDFYZgST_0N)a^E+{eO+QfQSH*^DPC!4 zeWbzDt=(rvMYjpHz%Gqc#cq-E0Kd%#+(UI4I=g&J|MKgS<qx)LUyA85M#NLz-}DMjvxX@c%~564Qy zrga>O+@)X<<9%MNvid;EN>nI(@YoP`@PvD*ecSpF+Z#A5sgaG^T_t&CC5W88Uq+_% z=J4jNltRr^Mcz%bcDwu4ho96FOLy@>RIT31i2jq`Lg}(%WsF1n3TF~cyjY!sTa0vEWF*@U*2IeU1WB8{Pu_f zCV-A6(XxHp*2xN?QMr3E^Dy9nr^F`|+O?${U$0RUt#4b`cb>jtCo%cZ)db7qM9{z} zPlsFPvz%PBUnXccbgHyCZ(KC5sePf!iV@AfEjT0bIK@fiET)X!P3~W5()-2K18Jho znjDLflbKCg>s+Z^Yqwn(p^cVzV(^Ogd5zP~YkQvRNf zu$BIYy){KkpZ|HWNTcbz^8I_`HYYtH*3w>CxT34MAc(`VE1O#Cu~uO@KiQ~ z2|B;#K7CZfW@=ft+jq*%RW~K?a%$4EJZuY`q7}!X8x)#w|5U9LFLo#?^Jq#$hOW$H z|4R{gN7iLk?@8~$y@r2`T~AXwEZQHZ{u;}yC7{BqA)vx3Fz19S63Ao`b=Q=z;EO5G zmM*ULmMtTM&*ZY5a7~PMBrOh;N#g2E(oCsW z|H7cxFv9^sLVPYEbMtMI+8YKFiJfC4UY(#j1K!%+`TeVkuVEHt4^ow9qu2o)N=Mtx zY?trqIhiHJv=d}hEk~0@XFi*E0wGhJI;giF}n$RU#HKJ==QIes&%b`XQyOzAF<)>wA5P4pYI{|3|N%i zhKtm&H1zxIt)-wzI2TW1i9i>ZE`fT=`#fe*>0*Eg#c#f~(w~|knK3y%fadP*_C9aG ziGJ3_UB#9~aRSh>MBLX$JVB<4OXk99_v=;E{-xKoTMz6JN+)%*dJ^&kqbj9Qf+-sd9<|Riqw2WD8Yb!x z*Sf>qxWhUU!|M)^rk}sj6-$D<(yS-i5h4xe&|Q6*8FiHIy4eJWLYBYz3N`QCchE+0 z^(}g*P}kr?y~rSvY-z_?HE^oYM%5_SJP)iVBDvZ{0{+BLk-Z=@+ia%Cs3N)XVP9|4 zLAz*!e-F0k(I{SuFBh2ygAQf<%&Fc<(jK;Ak7-~-s zkA>t_p?MrSF=>OU=G7-@G7NCK=zP(iFIIOnj>nRg!XzcL&|3LxTU3s&T4zIdZ%K)} zjXvDn)T3#Ma%>7Iu*Yx!m;R|e;-4(X%m{ANSTO~PhrAQk{jPT+dY*PWQKBI2?jOT9 z?Ci7dJBT*uCB;KH=M$K;+Z^1@7Zl$p42kBiYa63_$VsIuJ~Oz2PEtq9@G!WlSAOGs zQ%a=^d3u1f*6h5}VI}UlZY$rB?Wx*}lbr5fXf+y2q9}6Al$8B(aG#lJMxF0R>8w9( zI)&ZDIzC5P*ts&tl)=K&I`E1Jx>t)o(j9(Zzb9$Md9fj7&BXSKC}P9eMA}unZ<5e7 zHc^v1X~Ydn0U`HczP|rh+PPHztyx@u%1$+2>bTH>3?+!$VD$+6FFyy)4=#rY6UmD* z@KA72L8FyxSy^hru$JJc%#4m?CXt6i+iWZvqHqf07wgF94scj`_(BWmKB2Pq^+^X9cQL-**oM$SR zQssB&3$saIwu3xb%zg+}rlgNyW`=yAV&(w?DMJ=+Cu|gZI9EJ%o1ZP2#!na0Yvcn- zMzd8hOb)I+Lm!(TWC@5sFn(-y=nllgX=i*>;{b8Ne%LL***+30fR ze(h-U(z`}0@nm_YQeFuglOd|!-|kbTnoU&u`wpE<9IMw%HkI=86GD?$*f4+LJIS9F z58Jp_J90MmtPHI7{$Km{Yu3TuyYp`qt38Xizt98f3HX`b2DW&E9shYBJ$T{u#x-YG z-ka&?O@8+JHxop<%%XgnG zDSl^OW+Tded*A%+D!#`jK z0@mAp-ub!nrXJlGH-B64%`>roKW%&TdCsvjTvv@VOI|g8>-*7bvFp~HrP9wIuS|b# z)%0eMm~`FSs~^+!!*mU%nJw2o|8h~yi>+6iw^<(LyUzV-?(>(o=7h>~Y2P_uacj=I z_tPGJ|7ln8`|6BaUxVD&$QtXF$#*-SFK(O9URpCloJB6c!SNsC!#&Vr(juH2vs9;D z%6J*EUC(UY`@khq-+r@mYz>NB)$0eyK zb42gb61!NNU$Nbp#cn50Kl$Hlmj0>OWTGJ7-t)j~mAkT+e%0Q&O{xH#mz3tROq~k6 z|1|8SkPh+;EMzGSipUcs;Pq|Z%Z?y+U?3)n5mV5NOp%yNYL zYm+xE62*GbdjJ1b{^Mie|Mb@F+igB?#)gDe_&F;`Sso?97YV-5IPw4E{{Na~e-76lE3f^WG zjbV=e4wt{L|NYy(bN!#M_Q!YY|F5*GR)C*(g3=mA_*TK7k?CyHpZxRxKK%Pv|2J0V z?@#?l)w6l}=O2jSgq%i?q!`WLMnV~4PyT8tf%?khX|9^h@_r2}kioB2AkQ+9U9FMLWXa`Ws z@9OXG>PwZJMbOU>Lsu?v!hb>Jg#FXGKK*RfPa@=JgPFjF{^XhBkb5YxdlKX}hi5=_ xGt~orAm;+4MYXV2@Ths%W0HjMOLhGBpK;-z{gZ#)V(w%B0#8>zmvv4FO#s$kD_#Hq literal 213845 zcma&M1yEegwl<6g2?Td{f)j#!aEB1wU1kP%cXxLZ2=4A0+}+*X2G^f+@BQz0tInx< zv#Y0SPw!r>tDj!|tQD##FM*7Rj|c$)fh;8{`W*rSffNGb10_7vdyY5e@AnT#-z-E# z6s1H&NEPj@O)Y>X5D=2cWophplm~IzD=2+q0?4V+oaF^{C*fP22jQ*8p6Y1 zxobGV&ijSI4u_S8H69%y4rroQ)zCW4R~Lr+RWo4&3!nD_x3c(K-d^@HQt#TV&sOeM zdBJ-l5avsWJ6GD;A4wHd3qzve_Y|b%w%NSkkQP{=BuQbqp44tVMEYhUOtl>U|;<~BRiqQkTQX<5MI!uMI()}N8VI& zb)80=6Ga8Gz3$N*TdR`};NE^lHAdd{RWJPY6Pu1PW*^bCNcS~cPPZI=S4L$-aX5~a ze0x$EXBVVAc5RMWgDOBeoPru&`KenX??(@Gqe@PM?6;I^YaHx=pW09;oAczt=3l$T zA-I%N<1xR6l))kf{J3?gtYx9IMWvz7??-v;uQ6%DD7^7~-ms_KDkRfpVuF_u;+yg@ zGn4+4iz;p(c+QkWnNJ$wuyrFgwSxd}k7~s ztP;(2Ng5@lvB;k;_5seAm{drS{K|!rzg-eIx5Vr93F~xYO}C0#S)zsz^7e^(gx~|9 zND>N7I~@i-SOA=LINb;i5a1wbNk8^%a19_uN-Et9D5gX7UeFB0$aP&b>Z?@XhM~S$ zq!8gA!>1Onkk&ro0cK$$a_>bi$ti1DEGd)B8Wl!(ZNnT zAX0k61Xer;M|TvdUk`U>qA^7W=pM6^dK!Ajwis;VxD*Jfj(ij{NRL9X zw{QdD*n>-|$>y>%@T~L+TS|p!`NB zE2fbwMEso}dav^vt?w6g1b_hrFz~dJz$?IC5l&*0GQ&6A2Cw`NQ7s402Htfi!NDJv z2;VAJCR3kOL)8@MJwI22S9M%$|6Zdab)KZaK^7_wIIW08gWUyjAc1P+O8`-qyy1Qo z#|g(R#BsN-A&Cu?*=BPit5sxlSK9?CAHG6Q!G?y5fNlhHSMv7pC7lPeZg_SN^Va;O z(=*|k1H@Nsk#cM~+!G$=O0`n83`Y5$OuFMC1wsDt-RuuBfyWJuK=1p|r`w0n zhb0cTotYMtR+JX5*{GS8CJTzy3|u0v+qP7j$tt`l0u*T#Jr(vBhtHT5RVc3iTKnlG z)h5ywT{K#2@W_ghx;PRzax>yNGCk5VqK1!;zlXnukHP*M|Kry=0O15z!KKhPzf7>n zIHG}^>1T~$jTZID(&bV$AjfhV(SW=WPIYok$zs96-Qqcy_L#XmrbJKbc+t*`UAavb zpctTO@xelVmSL`Rg73GhRE?I5&afuEsLW4xg_7^bA#-wNx}fs<8mCgHtlKZA{7d_F zypFg$b36t->C-RP$4#W>lkCP5x3zv(Ra|?3! z^OH+Cl`FoVr|9xVZ)kvXT$)<3TK!uIT2Uw>Bw!*@qt$r~*F)$j){dfQP-bwBlaJ2l zJIdd>e5_GTV~i??evWNqky)5qb4}U=ZDRkK4qByflz5E7jA9^iwb`*2Z;WkIYMlh; zo{BSyxAjr?q4n8qBli_Vw8*(o`%*7b17xGp9a8P&UgYxf0kR4y0=vhT{`XDKH_xVC z(W$W-zf-4x=^7;(*-eqIaS!C+m%TfN6l`7WRBQl48iN~y9Am%MSXF7&^QiRd(6F>& z-Plp-tQLUa81vY=^thB4bPdV@hJh@BXG_%}XdoC=0<2h^JZ(L(JK98In|qE+0C$vth|O*Cpm}n_vG_H^!)VrynB3* zzTMK_U|%|_ez`zoMDmdOgz#MdQ1=9WnSSZGvqoY0hzj!qmJg{G9uxKq$phUOC*gf7 z9Jp30SCZe`Tr8tz$25^e{t`78HMcMaq{yp(U+AlR+T7dJ*ks@2M>|8Sp}?oGh@Ffr zA-5LClR(HV&UNc^jY{V&=T5`QL~UbzG>GtwAkR186YyHOpWQx}uS_FNSH$^PrBqd7 zbl$tzv)ljjC02md%5b-*|8nfoerufC1|y&9M9L)|H=~jKQa%n8_oe8w2Aiv$o#WMV z!bHMV!mX4M^=PI8>nw{(nr3#Nq>TJ>>fDpcf*PRXDCI<;$@nVrGl#ONJh$Mijpf;l z(e&2>gnT0Dg1my?=TdL9Ay_{U+}LuyaF|@fiv%18jEnl_u;(yUWjAU+Q%*D2@YS#l zGXoq|lPQJ1lN}}nNpec-8B3U2k9wtJq}R|F2y#+6duM6tx~-w5Mlndt@vRy^jvRvQ z;9Nsn`PXT@L0-}1npHIr^eNvJ^$Np3jWbGX?amLuiM@&?ik+KMj&prgcFzZz4l_Jy z$e8he{>1!hz`?D>)nXB~m=S1CJc69o0wU&MHqxE;2!D{-b7lxpuXHD~HyG>I!{KJ6 z)4O)F9oh<_>XX4{aJ_cynP;cUKo=Vvwf5*(d}MqmyI!8lE0^J9c+sY#Yg63&p{jIi zn`*Ndc%IRr^``T7HL_bq&!-C1kkb3CRPO0H^7x3HM4)PQ->6i5>1JZH`zj}%o@C|LqEY)Wl+?-iRF7}Xxcs>3 zet6eVF9s|rt1d5IkYCrWeYuRLv7usK$pne8;_RCgY=zN&qyq0`oHJ|2eO zDy}hI=X|v`EEx2v-Z5^wUjNeiVgk3}bJaPWJ>}hO%kro{O;#cDLZI&E;aIi*?Z zqzY2V{9WEAP}hDs&`2SN+>RXVjrTHKo@**WAALxqWt-vZF?jik(M#h-up7BbQzI{6 znKtLJKl598K8=Q>AuCG2^Gx zbz@%Ew&Pm#T79=0R9^+^uYl3!c4falN(Cs*GwEjAw4I7iMQU-7Kfs z-ZmF;mfyXmw{m(~%dF$IF2AfDzbvABCds-Uy&t%5e2rf%2pXTvKEK_+xf_N>^gUt_ z`iVV00^`#3(I+!(>{BviN`^0Q@(>h3BMo^3vTE2X9C2f^643z?WD68z95zJHS16$H zETRFn+xPzS9-&ShonU&y!tXJMM>$ZqGQLsN=Z{Z%c+f2P5CWecpPo4G?!1?L$Br5x z@xgFsZVBv;!Oz62un?M7_s_&Ny1RF^Wdk6?@s(<|;rM^>MKtMod zTl`RWP?wYCF|r0R85mm|nlQP5Y~D*lK=8TnyuX4>91KWZKtL;d9v6P{|0==r{{F9Q zW^&U1D&hd(Cs&tKBo(o?Ga+SX`pU#YE`Uf%O3G(vY|8UpRQ!L$-~aKGn>#qz@Gvtw zJ3BKuvoTrQnK83+b8|DZd}03bh4H-vqrIz@gMka9l|99OSMvYYBWhxAWM^UHU}0@V z`mcHo46Pj<_{qut)zJSw|J_d$7mNSbl9m1c4C{S>%>Vkr%*w>V{J(X-i}L*|mq*dU z#RRA>Y5_8_vVU(w;0qfY2j71Q{M)Dh*W~{Z)%bs+tn8d${;%l&`Sd@ce9ZqE!v7i4 zf6v!{<-Sjt03sjr|4zODqQaB14+Mk|gp{c84;RRjOavbymDGBL*|r92W+~?a40wQ~ zDEZJ9ZHNykVmCz2&p$o`Tb&1;&$&Now;y?*!=UI$zxc5+g@i>RLi^FClk!$0z*l1+ zph}WTIZqwy6u%anB@>3ZTU(R0<*i?7a^KB3C%MV1tnRrV^U{QDFCaG|1B4D^7myZ1 zK85+fhx}8LrGOGfMRKpFsKTp{t73rN#_sb`>sNO>3tI@RT<5uH^#@stN??EYid|5c!K z0KsK*aM(h{WRNqx3#s^|e-q#clP(A+Q)xz@JltzorAS-3f6V0|BlkD;$#OzTnm1{e z|N7Lf@^Vkt*K`C} zrRCAYh-BJ)Ri0OO9xVZkI7s0PR-885i3-8|bJF)jW04D+VE)ZM#b6v{ZgMmWLuaWX z*Fonu%47EQ?Uzt=D(;kiamhdH=KG43IztVp%t*1|)`*Q|aL4Pj)D~aYuW1pg z1OmtxN|O)c;DH`;<46`9OX2^d>HnoJDd<8nxnAU5F(v9V3}ptf_4y@V`oYKa%A$|u zG5R&_OFJ=6pZ>D0dsO&8Ea{`<>1bG{N3$-{<{}M=4#U=mV`5680zQ4gv6^UqwFqB* zWvTtOO>(dh5*U>0Dyw=#PxV6eW$rrfP(U#bkFp8ae^rew>BcNQv(7_Cs2nAfmP+=N zUS4_VUrmZA+Qm0~qKiXKS9(u#!l_v_;2igQuk8dejV0eq`CqJ4Cnb%(C0=HRjS^$o zGOWYET_{YA`civM4Mt0o{eFH~;$Mz{qSE2;`;w)#wV4hZ)E}UNpX_q9mckUWibQ2z z^g~vjdr|FQ^u`B10>zi^&@#@$Qw1hHu;Q~HC@vzh`7a=8IzvbWhM+hLw#v5`vmIfx zhT3n`G&CcIxKQaYb~_{*F1IfNPW_QA8uz9rKDU6zUjZ;2RJ?R&F-4_ZFzW>O-$;aX zHlyK+>$9;cy8m6TyrGV%eIgx;AW-WkzhjF12CXkHN{y{F+4ncYDVMwsG`)Vi3~cu; zTQ-#~%PML3XXIwtg=f%dP7r8lXn+$&4f=IDNY=%nRFDhL5u6YI5_=L$vU&4lvMQ)K z7wnn(7Ts9NAodsNAugkzLs|Q4Sp{Z!!b6u^`UL-Ii9fTAL>un}BawN50t_`@r|VLAhEO>H;gS5zJUw-9gu=Nn+3lb-!_+>_LR__)eO7wSuW*=}1lL zpmAY9^_)Tv2l~mpQTDLXNJN%G#Tma-s(Y!ec>a8>$V+W4u@)ZhSh2=GH=0lDJNUWM zB=~GDrP^!nkcRo~tRX#@D;uFkcx6Ky`kN{t>p6L~Zk%=tjlVu?QV9i|7$aSB5?oVW zQ{U}eDr1f5_#!Y|u71TXp0OU7&d-Zs976wx_~z5_{F?WVT3KAamgfDGyxpher7LJ# zvfpe{CuuY6f1zK#tW+$uaqLPG!DtJ8QSdZ;*00ALnGLg8Z|C9mTQ1^#bwTA+4w1hA z@Pw=6KN$Z|Mo=o&K&eSl(-9O=5;YY(t=h*92MK?+6HAUv2Wy?T4xhmhU+=Bnd90?_ z14cN}`}Zi8=j2c*2%}h<5?`QcS`{1H$|!h+jJ46f;lN=zGCtH(+gYt1seS&N-jcyn zYEJGf$EC0=AEamN#b%6+$=h+PxolURf$IccCePblu1eRk>b3K!LhyA~a2vHZLPSOM zMaH(`X)O;dX+Z#Y5ljv19jjAk=H#k~a3$?1B55t}jWFa$>jnD?5kar5^7BO!`!U#5 z8&H$dv)1I5w%S`~zbsJ+e1WuvDzf% zea}(cp{kp>+eubN=I7s7r1uyhS80s*St(3E5qYizzk((5_V#|Ni~bnq-V2LJ*Iq&I zVWZEUBh5nVJJDJ++p3q1OF`>NH{H7X+Ls=1IWpmB>>s5)F^OuGF+N6z%xC|?ZboFk zK-BSSi~d-TeP%xHM@E-4r04Pr$B|KsJc&``iBahRy~sM!oO6lCg!!NCJaQI0D3Jkm z3>RAaqoRVxJ%n3w(9`Ry*AycoSMH8jqeCT?I^xLSP<^dvMr2j2{qdOu>&5dnIIr`q*U^1s%rLQhgHq8j zH0Gq;cw1zKk)XmoZ2?Jv5wcX2cm)m{EzZVMu$1Y?OOU}t7J}w>Ar;f81-zP5i~E@1 z)2(v!vF0Uumzh!%pE^nL707PDx_#m$dr?EuEuf1k)@BfxXp7#{XHd%y`e(2vg_@4* z?D+`PH|vunA6aXBB8Ge2@J3udQ>0CV^OGfmN3@l_P_KT*|7BD0@v|LC8}WI5h9?r7HHeOIC-?{U5re2)BC8)1^b?Wdg1 zQ(uivL6X_e3o%>5UsY5sqDWM+qE(UP*j5K~4nO`G2S!5%h4tKL?aVLzx*;mAG*!|Fi+SBJiuq`Y3QW| zF{b|HkAOqm-ECV;MOk;>FZlFujGTFgZ+g}GCz0QCnRc&M`2#}}D_C6?adwdwWt5`ItgRLYd#=3hAv3;ii|G|}&8 zkAS?XnQTINSZz`W(0AKhf&;;jUi@8)zYtu4Mt=c>E$lC=v+UMPOc}MVCXxm0_M5er*N{KZ9{vWv|ahv+2!W2#fi>Sijv{ zu$k58R#C-^m#+@lejmJHuHXSWDyta@C`SzqHUornnv`^Ws*?|Z3kd`(5^5FCYtJ0D z%?vt|)PI8^956|YTH?K@r>DX`ewa&Hgy=w{>Aj0O`FwZ|EmB5}eFm%Zl$Z9mS647x z^pRn?qy(WB7itZqmZF1j7Zn*GZ~&Q$Q&9@ja2|&&pb2(SR854=s6^6sFb)`Er1Nq) zLTl+Dtc7{|`skP5aj}^fo-L{awgn!9720t!9P^l})R%B)3!99vNYerbj{s7w85Y0+ zU}7Ys%n#1qs=qO{Bd8$bO{td!(87SZX7OZxqYxT|T6zlO)(Q{vXTx4=_#fER^&+|I zJK0f1`!vnXb)UEW3JHUIO3`|P(6L)u9#@MO*n>jXf`l=f%Bz=~sa)&aDHTf<(!rXlRg{H@s%tbS^dgBi{v`F;L_)r9lKgjn|1mdg*T$F@p z1m;xzrPZ}Q$1bC%MmABw4cYd=$}tV}+bUPBpqgoB zyIw`be`$7x*nn~|ERT@p~ILcV-2 z5XQFUVEk7{4;>V)LU$zAXwVXnsV8)4x<;F+;mte4)iFiKql*&(kKUv^BmA4GFOvUv zZ^DySHE{pf=M^W5!z4Ypa1s*u6>($(d$g)i_FyIqs^hAv5qv`eHy!9WEnL)e?1Wrd zCITk!!^!E_yjl;9;Y||3@(fRxT)f3uTkiDIL+l;{!o=8pbl)h_MxO_Yc)A0WFB+}l zwja0bxCdA=jrka|RZ3JH9UW6msN)xbHKRdHb#}W$XJ=M5T`A-u%LUQsL5;?P=(m4v zhmxVL-c9z@h6KW;JjbD_gZy->Bapam`d^%0@7Ho%>Hf4vqj%iGD6gP2-g4<+&U6Bw zaHDwL*c0`BXG$jPV>k-mW^71%U|wfrSYcAXPa1I-&}`nWE|+Q7^*Y>@v+weMk+dlH ztq#@QOIb%tWym%mfg52NY)?2avp2kZUet5BAC!1~3h9|N7y)*%yY?x);z$Y$$DPgx47A! zw~HVUh%%^Bf#mf3{B20B{W}A8S=0OFVs4D*m_f7azGeO8%qW^)+;DFs^=sy+sls*X)nJW6wxTIcimp=9Mtfa1YaPJLbMW zDM_!KQUY_|DDozg+Us>q_S=OxvmnXn$cLU5lX>}1gP5^o(gg0m*`=w|2-<`L!quFn zNG>B8VuR$6ixCdDh$cJ?lNLmk0;&=H@oEqCcVnUV`ds(01&1Q4bF@JO6v>O?{It( z=h18@_iN^-^JnU2Q6d1+hqwJCEi2azU%09M)Nb2#&pWYpX|w!k1%Zbhy|+g-yF%}d zmsQL?_SiMy;B|gQqE%G z*;Hk=Wu-lP`#g>#ykEzXY@H?*#tyBF2iZ*n9mMiOWR2NP!(}Pi7`RI)@fkG?BiZcv z%Ql~Bh@Ylt*6-G{`QPT+-X@Vnr68f^X?+os3IR-^`X9GO61^kd?t(PVy$Ouz;+9!F zQb?{)yzil3#`nSRtRn}$o({Ei;UD|DSaIy#aq#G)@+yBBvryWGjV%+3;Pj!B%RZmQprD|0prHua;rD+TbEnsb2jF>K`@ciqDgX4P*9P)OTdd6e0kgLQZuv? zKTG1d`dKS@Obb}m3x7;9s6Mkafr*0XMVZ3q)UfJAFkNq{m2Jn3BKXV_t;n>oL`LzP z{2Ub(70J2gQYrx&vQ7UqKbs>Ke%A5kRStQky@qB`zU|0${J!}g>X)sy@9r|sJ!s-- zDnyX|bqG}1<{2BfWwOir)8GKM#rscHh0Tk2dIwz>Hx=IeoGp_7kv0Kdz!Yad$L|1$ z;7pp$A`taxh6O~o!`NC#zSwWFu7(1rWDQ3ZJLt%%=t~pX473@<7e_L@1&nmn_T!WF zQUFre`ZDj!DNlj6u%KpKC(2C0n`~}vD^k^3OtpNu5|(M*d{Te+Yy_+_k#xqCgzqWm zKIUmz#cF8cC5cMcEA2Y&(>De;gUhteM-cr=q>CbKf~(`mH;7_i@$w z^|#l%bMLpO*hO&{CXrh92Y^rtp%W<|xeTy=(KOw<{UIgGQ9)sx+ zfb}8IOp{@6*t^<=`?Vt5_n=9X{%akv4%4D}jtXzo!TOByHy#Q|s{WvmpU(lHmeYD% z*&DFR+kd)q)j&vEdBZ(V!b@FE7_#M? zqA-C2`(lu<7}e|_#7XSF6aiYREgr}6gYabx<&^n0hPsVD8&M1d)ln*ieh1Jd>L_0n zc%p=-575%k*Hg(_WnpR!vUm2t@-b^lo|m)#=et#8HU#%WN0W#q?#5v!hCJxOnRk5z z=s2A#?{+i4{H*(W&)c?RopbxP-IN24bn>{_<&^33B)AnKg~qY=Mo00L>GW}vz53KY zZ8`HwgQNr)?W^|V{7x}0>?_yqr*-qW(E`$+n&BsbPv;%pt2rZ{0f2>1WC1QDB-YdE zEio)OTEx-u?vIBBB(6JXgtD>;*S8HFZB~KDoblq>u6uY(z3GFYsLpkR)agHozGx_e zMEog%34XE&Fo9TQ)<|{e;9Y|D>%u^sm^*m~SQq_M2Z7Jx)NB2+@Ea>Ipu>2Fuf#CNz zNRxe%eL5WwA+P+OZjSm;2(rk&$V5O#y|*{M(`JG+|21Pfamt3U#-tOBRPxw`EkmD# zJ@o?yf}qv5zpFu7!mZx6a5{w4a+`{gCl4C8ql`e@+0ie~?cmIg=UwI`@N9+GZ2l7X10_?(vmG`~ z4Pic*=lSB#w1kwilatrubauTg?p}4RdNv8`cFtBDkp2N?&yMGwh>g zU3;Q&F70BLG_?y3aQnSr=pQx8Ody z9TKDp(%h#XZTO?KUvz&|n&;AZ)!YCfARG})H6CmK2j*Nq4nxQe-?HY;kH$$aDsGU7 z+0|~NsgUuA3Rs5jnlxY;THQw(6{?$b0QAu2wt=_EBh~Md&_&Dcm`8){sb=~1g`$s4 zBRVqWm+@*{&y*eFVr3p%`>_t(O3QM4`H!d?4Ni(a{nfAKVe0$KdqgXSCJUZrsD=EkcVoPoU%zw;;dgrIj1bX z!;wUa{_G+ooOcx-p-m^G*C5_-X)y+464u-Z?&tXrIYU9Y2U2hM(QoFav`#%RKNe#K zdxNCAE;f58J3npHr6ZH}!>dCqPMG!6w!SZfb_vVoR!?3_`$-v!N}X*TOO!0#u=Y*VI`bSM(I3&kr20O1xv5xXuDQ}y&Qi*Ma!XX%*PRKb=*hRp^T!KJr| z%g#jb!EfJZHEt~q5L!vsqQ!WN@9v%Obw223dOu#I-QVbX$Iy@bE{d*&?Y%`?3meYoWD#`48MQ`kg6W#szJt15q zfMex{67Ck_@xm=gcsb(3d?q|FR3m3hi8gkQj%23pM5xx~c>d+0hfO+C?gz2+O-sL? zLxaFs(rpXgXnEox`F7(E!>d7vWJPYxMq7Y8V7O8?%lp+8L(&{W&DzL_0ztRkYoB-h za6j^$KO?|l(oS(G-a?c;AH3EA33%NCD%$1<)S=e1H6A_I?&Xt-I6!J;%vtp`;CD6C zYIUg+ls0^~rx(m~BQL=DV1z&2Xiht^5{5#^)~>@^9)z=H=twb9YLIBUl(jL!oF#wsiI(fm%_}eoI zg`vrdSC=0w$>T(D$YXIC=xFc0hJtlL!zN@n{SdC?46}UUuI=1Y7FtWiiRfgSeF71% z{Vyys&gcAa+zE|h+&$j)$W3%LEo3>KoV!9{YUCe?j9@A&X1h7{a$+Vp%!SSikU-(y ziCL^MGxE=}9i&SBE#_GgFd^<>UPHF;{dVWgsBSs0<7{#;SS4_Qy4MCr?6adt?Eu_+`u=^Jx(nKJ&%nUW>uYvZ zN9-T$M0Q9Pr>Tz~UXU98krmGj$5Tj=8R*=EkD+mmeG?ZI+i5PCn4Gpx$t@ zXI@6W>9*daYpXeX?33S>Jgv+|x3p)ZPoG_J}w!5{(gz@sH-BKu`rBF45Z^ebaJgDIZq+#eSmMio-9hd^@xyNI7?nC3+#kA!=Az$`mY zkVJ}NZ}TOL3i~{N|Dc74+KXyKa0W9t(@t+$$y(O0vLpLlWB2UuOe6s3^tJf7)8KKz z4pp5$5n3Pmna;e{u~VlRPi>K%&IagA{WTcUl8^q-P=-=GO{^z=*?g(FutEWr?sf1ak z>myU6PmKAx0U(nLB@6{Np244n@c0fr@x&`oIUOQMliN_gyGQ zYhwXIFZca|_0?HfZ$n6fDo6Oi=9vf?4@9Nk=ec^c&?2Zu^R#HPZ#IkV4rr7`mc_Qx zDZ0AtI$ln#rGB3lXf635+0M1vc6)+M4+{;66Gs}f^U4kS80}wf=IxPxpX+T<*j&xa zpVBTUC*zfsp|}a4uwqX?YhvO3o4?h!ibDTwQ*iM%p3^Uqi_LI8OLxulq2?h;TST4K zOgjVFPqz9Sezz8STi;u3yJV^BbYu09W{NC-fYO)G#tTzx0svJ&#jyf)45fFZCpUbE z&hSmLnqzJDO_TlEvT4P}nAl}@a$}-c@;FPpkzq<}2VqkoAx^kK6J|o#u5b5qP`CSf zwcK|Fxe*N>xSR01El}lU9(llhE-g7capMJ7Sags@euxEF!gEx0?)EVE(8E%E1>bOx zc7`2_eM-!7IzZ#%i1VS=P_3ik@^mYs`KY+Wp;5rI?rv&mpaA*%U>J=r~aHe}E$hLovi40g&Hw@ud zvLd`8Fu9~$QgffqcEP6}qG%qYgoNRvMBJC?M%##2sDOUvhZa6Im) z+R7(YQh`Q6hEfbEWl#UtwFczC=wz;lL0v3l|sj1f8$EK(yTn0<<9rN_GhPPtBl+BaN=P29X-^$6U(`q`o zmYPqhhXr0QqqW8n94IK4aP|#1KJ4_FdC7c<#R&Pz|9FsF!lk}wQ7xfRw;V%*)e(`w zp7G9Gth2#f7%*TnsK@O6A){0nSqKr_PDIP8YTCxdN&hqbsY)UFD)pU^g8lf!+PI+1 z{owT#VW}SSTbG#5HmZVMk08$mpvLWJZUk0JefIeQoT?iaT&Xhbf5(6Bio)4aL8jxx z^BWMVlF6>Oei6i(Qu@h>cPQK7XHE47Y@#Tw!*^b)##y5ALx?yP*X)UFvmHtrzZ#k~ zGg%&ZVc!6t21`zB-8m7HHLmhpaGLn})4wyb+ftpc>7Lc6@1YVS6x^@wYq!)u;KNb- za26u|72`wPFsE^K6=0wdse3uCiG2RRG9d0U6hT?!Q#fHk!T@_Ka~3kbmR9|dd1l*6 z)qz}nH4?#EHTsrT)yj&V*R`Yd{ViOF&HRX`pA1O^?-S9Gj(u*p3{&|JOjaC|lQ1d= zGBteo=9b*oHRVssCg3Lf!d0fsu7RIy51je7NgB^EVk22T92g6Y_^9O*k%}qLO2FcB z^mS<-66?wN_FMM1At#;m+82F`AMss7nhFcsKduV=x5tgK^oH)912E+8uL}f0zZ@I0 zZFtd3x|As4FcuD={m7xx6Jc`^>^$-`?VY#A#5@0pfH>CZkwre97K5CncYa)w74hxL%~xbG3CSJ@{2=&bp;Hr?+B1vJf|s|Wf}i5Tt?L8anI;y*yd!%Ozdy2f5~{U~Mh~_1;l_&8GWmtgj@w6nrw+4$V@)PKBI4g4 zGt|}lG+D?GfuLCPh$bIn>7j)bCPy}KIzr1f%53SYHkax)p9Jp7iu%=eNe6AQ)VsD+ zeQc1Xjo_qWGjxj!1M3SAhu}tJ@70ATUBU1izFGuR?4G*51U`kk(S~#Nt1t8Ac+`z1 zYo^DJJa|k9!!4?XVF;T8%3}Xu1wI-EPnaVBYG9v%AerN`ih_F6geMFIzSv=00xyhI z;YWh?`wz(&Ndp1TU)7OM3TE%~Lt}4O_>hM!tgJX>hJ(}f;5daWpgd zz3oUVa$=3nX^1y0xz8H68?}mJO8g4C-iZ}6X7!&7tUd2%2?yd3&cDYuzMmu;Ixz8c zv5hD}3&qbkMiK?ZrnP}+Vn6cdHK#Pv@>@zV2CX z!#d;qE_XF|t2Zb7&Wz2+QT^xbFBRhgm)dt%<>kpdxmmt*9v{k1qB|ZI%M|#V$Yp=P z3Uszy;(CdsJL%eN{&7?u3PQk?4$Pl{YKD(f#CLi1b7yYS6*~yO;0=!$t!@Tjrxi~^C z$dI0Qszfznl=!B%o4pW&Xl6cw6Wa|JBdxD@4_r1g?)_>%qk%p9bfngiw&wkEtS5oU zPOuJFOFlKD&pq&mi{w%pgQb60N2Mk)GtAw)-En;2s~Av`Q%P@RAw>vRt#sGsFy~LK zpr(7s!zWz+iW+jKv|u7=%Y5!{)f>_AI0ZH;*0wNDXAg+S)cNJjrXELGm_LiJn%snU zK0oAYH`8P~#1#RI4L8o22U42F^A{-`7qnnKOqf(>0AX64wA`&Q`Zu}3Xnba33-O_F z>u$sLYimjyQUl1xfW8QFj~nh4Z3mxAlX3r8kMnB@o~xs@MKEop)kUI2Nu9~lp*@vN zd!c#0(WML@MpQ?LggJ-5n@;%~a(isw(7)2h&%EhKv7#@6GL7VPaooN^4K(*`xF|PE?N{9x!0B7~yoBy4B+2EgDv#Z|4@|`T+0$ydQ~!Q0 zkh2KD+@pY{8s+-n>wC_3wgmZ0wiYSY(@tT4)hy zCBMyv?G_5sim};e%0))z^_@l42RFS~V;bsypt3^h*RqgHcb)zg#gt~)t+^*T<4 zy%!yV#cFVxeY1W7xUnkRHFy2l@?^`n^c8os*L~=b8hzJBgsd$(8cu3nR=~O)CPpl+ zIX-W_epu%C*`MSIXFi=nZ&^}cL^HlBwvb-*dhbw>Na+P8S-WRjyM>x;oprsWm9u5u z_Fi)2jS-j#JY4DV9?nme?R;2+>1 zgKUuKsJLk^q#D`G!u-DX_9iDO};EA^r~zRIb5sD}|(Tn&p%Ifl0` z*TJasc*q9(?Sp#v+#C#fJ|v!xa6_)DlwI(Tz0~WT0fk`*)4e-Y?6QO^4w7puHSI3m zrwI5>%zyzKc(c64Cmv7ld@Fe@-)lVEkmETdcu5m@$%65OkGPNv-&PufBmmjO%q6MR z1GU&iF{%2S<14=c(6<&$oTTfE1@2!{xV}0N65JhPM(TjK4Q(1FCZbQj)e#bZ4u}%| z)BeSuWC$iM^s-9!mM^U_MM_=Z94`ez{3U?$(L*7zle##^&W^=2y40|w@v=1sdc_&0 zaREB=4#{Q_1ErrWCPd`ux&uDHk)?fv*|LFb(n6a%1ueF(1%76Go5CV6>%JwCV1l@} zT`fZ(Ip8HhF8>c4#C7X%tnybG87D>hs2PD> zO}c$3AUtlckKo{bRNBOE`f)jYG&3`u$;}{sU#)$2((via{Mz17_^JY7`^E6Bz+SO( z`zdxCX895R&+FoW8x)vvhA=iV#JG<+#(11-W;BnGXx2XuK2(Nj1J>$Hx*xO>Z2U@6Jq{47V(&B?g_2@qTUq zzRF{vf;(Gxx()fZ zhWoigA97efB0>0*0U9O@gr&GnF1qD5rib=7m2m$XWAr`udXc-)Ik&Eb=k7fTopkB(=EAIBr>qOwR>!f8h8!0V=heD}amL`P-su0*d2gaz*2f4z_=~JWxN#-go zrhnMx)%T+5PEC4QSd|q=}2BpTwU<1ax?SgJwts#!^g^+M^a+7oUgwR(o zleO(a2f;8KS0j*`j}LunBhTu&nh7 zb}r8-6$=db5;p;&a+ovrH|1}r=nI~t6yvngwly6Lw@pi*EW z`_NTFp-gl`V5#YuIq|34{tw*mj>}hx_@!%)^4$u;f4lR(NBsk!*2Gwf(BFe=?$Hj; z8c#R?1OVDYFOMiO$r5t?*=b9#t! z?r7X6dE#(o&00J~L}nUmuB73z12msYzY=&g_3K6d5c=(vh!fNaJY zoteuW#@kL*ovC~QY-Ww4W-VnLwxCGXQ-%eu6Z6l`wWG91+Lg`Dn@C&h@0Y3 zmQif6M}*Bd8kc;1sM9|cr=M`Yz3VOu=y~$rI6@-iESL96b{^pmnW&Kt;h&iWbs^ok60K` z+xbdtTmKt_Ok%}z1%HJLiO>K9Pve9^HhGC#yX5lI^A(&;!1RTms%a=sQ`2j=ZN)La)SOs^Q zXjKynU3Q!0jX;g*Bwgl48gAN$im9i&d{6#A)ao^nF|e3uB^^U+4F}-S%`@Qbg>nN{+)CutM`M8_NY&V)BR1pKWK>T6z6|!Wr`n$HDk_i1TfN~; z&mo?|epGW4b5!4;g^*;XWpJv#POm)3*6h$@%x#Jo3%Pb>3N?Y-N_cvWNOsOghbsQl1k#=jkh2 zuipJ~ebq;F)fr0%NyrpVqXRBWj^jiG;=Yw?Er$OedvE=a)wg{MD*^(F(jncAbV*8= zNOwpIJao4pA>Ca{cXx|)cZYO$2?)H4bMJZY_j8U{{R8fA@YG&=?X~6{bIiHM1f>6( zP9~O;2bv^KcwT~=sz((2qlT8Miv^ci6CdtQ$1Ea`QuHq4YpT+ydcks!KL@&8qv|! z6n+ea4M*SjE*rBNKD{rwD_CA~*5+GB2=N<06?674oX*tgE%naS>|AowX0kA5{-U1a zJ1ffJT|D2v%w-buJYFi#(K||&*o7ZmsFK#q>ND206A(iY^3k|{22{=EsJ3uLdQk(Y zr)jtAwE`ze=Q{R-jj!%DeK?=j$_<=tXttQ{l8K&T)4yB~etNvWBwV!QG+CDUM0Fy# zeOwEXz(JomwL^JL7nLI1#P!hMUAMg_v@a?Cy)()82b2wIO>7E?MtK>PhlgkJM+ZB90As1N;|-QAIW@}xUx*9Z3-Zcu zyz@$RHQLBSkVA%qKx`Q@rUxbuas05DiYybMqpRM+yeHG#7?XB+Awt;rm^_wpLH!Lj z%shF5WGdCA))!J+zycyMY1s>z+mr2y| z@g|3uPwVJT?(POfQ;4*tR?1{QO7x{~7k*}g;Fa9@tRmlJWu0l`l_3U<52GRX(^>{c zlP_Mg$&xv^2FTe?Ua|o$ZNv^#o0FSq1^$Z_f!G#UC0|ly!~FT-aVjU^8`3|8X6a_V zQpwZ!>FFE%1?@EiN84#r+*KMiI%=Vfo`W z$0yzuW?Si%oc6XQKfi`d%uKEaWoTNIe+hZ1l80qYa9MsX>I5~bjk(GoQ`vvSu}gW4 z#C{96H2C!PXNl~0GZaUq%2rK#P=Te}@`~(y0Up+wS6TL$B+d0#Ui}Hwx%PgfS@Sx~ z0{j6+o9rZ4y^^NQ`%MPr&{DnhHz9IU@Z_x4lBs0=vtMIHA*>E2UJT@^!0 zO!BKczm2Am*-(6c#lXbRA$RlB-O_H(*5>x$#!b2lE zm68^<_IEd3B%x1-9~rWph;H;JZj6#15+*WOa;ZvQaegB`_ki5@+pKQwWa1hJJ>{ba8;~+0vp*Ky;aPQ_O_mO z)W|P(5kry$HQBM^d@GL@@$kUyz$K{Vo=i-kPT>=+#g6|u)c0LRy|R6j(=p)wGR}5% zZtuUnVU~hZAmJ9ppM;CY`p9*IbkdF-y~p$DLn4`I_)DIM>g;W!ee>h4qmkE2q|ZyGUStQnOWSf7iAztw28Vvo!= z7*`C36zs&2k`3Cg*U{QA4{FAp&{ba<21C^_UBPm{!^}a{l3nOgMLiGteGEZMAsBWGpj3v^*JPJmn5zT(K^5d^^HcvwP!a@^R-?=;9)uPND3+_RDTmWuth(UG@?h z&(RCi?W-EO&|!)=<59`P2c)S2`fv~0gEh{MA97-9IWFlGlRW)u&xAYE(ou~-#a@3y zo^>)j-gG?=bV9t;`|h?cWewY$glsB*cm*AYdi>P84h$r=4=k;&k65g^&r5Fkar1rV z_AC4WqB2oro>U$wvR_s7B%op$26S1a>%GJBa5FTBGaWR*X=RZ7{=!-%H>&Syih*Q5 zZZT;wCllAZoe}}I)o4TGW-O)|zDK&0hSW6kOA(m_E&|RARfEAE7EPM@o-~(ra?GTN zO%sk>Y0rEcB~wW@~7mtNet1(faaA0dX-n)4XdNi^PVuK1=C#|oov3@55GOfnaUy6{w zew5v52Io6aQEUW);l654Q0upx=1AsI z;waQ6ThFxh4F*%=Lv}EB z;aJ2&uZZMlOKxl;>6OK457p;tpsgZgZ2HIo-9I>Yj<>!qyGocI4RG?#50FVMZp*oJ zTKK7|Y}^7;Q$C@!r)-t*aOD_TnzPVta~^j$tCszv-p{KIEZrRA$6Js-SI2vc)Pr=6 zPz{+`h*J3~`EjruUCi8v)gg={IqpmNpmh8N?vi#YPH#jPC@TzljrSeOns(?nOR_C= zeFYf}K02Wq`)+OM+IKACmZpxuZCS`gF3WS$Kk;usjpQ1o3_~K6Ehwfh$8{fWI*u;3 ze1)nPT;T)?buy#l>d>ieUc-GBVHPJzH!dXcQnM{Scv?*n$$s2x!0F0!*zzdeGP+m= z1+hyRRS(o^w z{Du_1C!JLS_u1UG##B__L7V_n;PPw6QM77{@i~@i&Ylfe30!{_L^Hzc5j}OSo}`E( z1%JtJ^?1wQi@Z~1BM^gwe|!ME0n2`OV#MIih4os5V8yor5krz3x^<5b4-|BIZOSBe zh7wa396W4=A+bF7ao>7WiV%Y@biqO{gdOie#Yy|5UgrIIf@PwHyG_MPnN7^PQ&8La zqxp8Hxd3yMz0cpT!*Ju1dOQP^K9;&XB2LmH#RN*q4cpH~gAFow!lD@?9KH|~A(96*(QaVjh)^UTnKuZ9@;{e^DQ z-r#yk(fW^uNjw7t6Z8XuXP*r9%B1Uzdzb>hgIYKo*N_Q%k}h3~`}*zvAoQ7$zzWs% z`@=7FH29gO3pyh@OBjSx?G^8e-mUNCbG^#IRKNE7IXt7R*sh%Nbd#PJWtN10-ldQ6fTRLdF)_L`Z9Z=t|J0nFYdL!s#E zE;g=?n5qmQr`9FfP!h{q@i4m<7Gl6Ufc$Wzm)cGyo^kmM(*;gGAeEv5gF%gum5=3C zxyIaz$qe`LE3RYL;+-GFm7KR>C`e)_0*H7)@E+8bF2@u?S4$sCBpO)Ls4}}Y9ER@D z)OZ@$zV6UAy@BQ=Re7?tPGo^nz$dZ%i6o??S>@FAjh?L9FA80=m_56ROBi8mUc?p< zI?}4z2X-Y}mo_`4cZ8VDFq!pd$|=y7;w!5lq(K-}m@cyRu>&=7OUpLZ$m~g{iGqEi z{fQr;;bX!S6@QO|$!ngSzvQXbaUdRMuzF{Z`X1sgDO4$>ng26%?qz}2&280iu4$Bn z%ckQi{LXv}olnpfg4bS~K<>%pQL`^4ueqP)8Oq4gO;^pw`|8w{QK9E2@^DuJxS+N_ zN7tAy7Vq_2A?RZ*S*FVbSf6YxZ2z!l+UVv(h~KRGYHf%e9dB|oQTy0^_>!?Ox!d^< zH(d)*)W~y2K`BFn|DbC`RJ(THXGiTjm&I!>r2wxyX*^ZMFpMJx6ukEDWnpotlEJsG z$C0TGpq9au#Jfi5Imh=0FUb`ZIWn7{EoE)I7S3DRHjv{T75*vqB^lRtNT3bp((_U4 z`@LXw=K;Z=?KQsB1)R?(%u&^Z^dze{5!rqSIY)GSeS_h)gLmNQR~W7&4FADSaw++4 zB(|wd!x99t7K+H`kt#)TjIWfcZ0QY@aJY@4$q`n-0~Z9jmf zh`+ALSSY*SK2hcn~WIj|mNi zxs-E`HAXnWu-0HHFR1+26IgE-Y->Rw7uCLK@Rw6QkER(rF~Z6jn{ShSrU zQN{%`O zQ*CZiOVw<6!2{lQ$h&Z!MqJK@?Xw zEoJ-!H9pv2?8WiE@8fyR(7KM6=#%}p-Z`9e2ie-P)&#mrkx~iO$jK>r52T;Hy}frf z>p`Rs=#ANym?L=JiQIz-I`1b&*z$Et7ue#e?>*L;Q;)t_NHIDbetnGEbH1Z}mvF19 znDg@Vp=4gAf2(CT<07)C>9q*K8&K-81OFP%kmHNU2rA@#IJt`DCur#;aCE4NmJ`x^ zHD0`#nE7~?z~kppU%8M4_|7u{&bCiK%J@mtXw13`w2hMOV%YgRzP(~=z0KZ<_cnfG zLfY6$>(<@l#$p7w8VfTTrdx4*U43b$T-ykL!1n1{Q_o21lS>@?8D?yF&zaIg&W6d> zmmG(~OK}(D9pIQL^|=j$2VMd$e!z%r*m!Z=d~hm1Cyl#UpE zpE6!NdoH6p_5=OIRK0SmTc2ThC-<>`M0IvTVGuj$3kAgW zu(kYM5U#FpS23OBK+bOd<|UU?$`lD1-hv71sE{Jnnj=(7^WaedZ2HEDsC&P>li7_T zCwk;|__5jLU?nUfC1m|I-NIP$Gw6b6^`4y0QzHO^+b;0ey$xI?WUMlClo#GjzO(T%$nG;0ij1HaKLV~)n;c=+bbQvehRgo96dwx%yen-IcO$MXrHDl6!Cn6{$fhu+!NW%g<+15t@C z)r(8v7CdF&Pm5GNo~0QIvEPbH5hkdjgvNZbauw4dkoW5)&x@w<+24AlG3qz3KInjF zAylRQN>ks!u#XV6Kx^6kqIgm3*in>?V+)UE5XFN|Lz2{5wLX*;388S9) zYI<=d#kZJDO$Z#Xo;b>mxHF$n@lR|-Q$7g62rK2UsP}&){7W99iYcUYSCcX73PM&` z@%>X+!cya>4T^OPO>0+ao`hEL zLvpdNbz`4%jnABeHJ*(r(>mDu(F}?(gb&5VVny<@Y%8gPo>j$t#`l!#Lf_k9^h`N3 z-+6hEF_~{yHDjkhU9gVrS8VP-Wu4e`pt=wuEpbG2aI%jGm4a*XJJvZC<>*CaTTa9J z;QfnyJ4D1@SV5$gurO7_>EL=~v#?@%h6|)qu3Ir=@W+Qn(L3GZt%)epOV1w5))JRT zqk7UNN`6u{^s*(rhBFYfR$n&foSOeBdoglD-xt@1G3Op|J)3Feg0;G0t#=vJoO0{4 z)5P(Xc8G>Ltb^niX^?+B3m{R*Km<+xrA{Q4DLchE?*LmvY__SnIn`EPK9Y~i9zKCg zXm5I;)gpBeHeQPUQzZI`7om_wKII3ghuusj$Rny~QE4?&HR@QHJ!{2gmky6B2I`7V zvR;4VkNGA=A^tMF-z&)Uy=`D@{YGr9Yv#(ByhNU5B1ujA{u*Ds5xt${fekZeol&v< zg~v4Os1lk{#3j1%)kLhhdKsj6GJnljsH-1J(1SK1c5G%mYs-zCuV-HPnJP}8<<)BF%yC!Xj7;hxYq`Qc!jl^TW5Q1X0<}S08*0bTG@qIEJubBqF<0DFOG-qp z>+9XG)vm)?SiZKEdN_0^!VY{{_|h zKezn--=1g>N8gjrx?2mCD6U(%7Uesxozyb)t*Q{wA-Ahg&|PMiVI|UtT77K?RhSb<=!$YY|9s>ch;M2b;G%6*l2&1YU)fRKY-GT@r`E@tKfxYDBsG92_{i;z32+L z?`(z-tGW!JCdCS7Kag-eOU%GUcI(Ec3&q!O?Yqg}Z}u2|blpE&omomIf$>oucUCR# zO8FPT#D9=49VsnQK{_}tw8bW#(^NDMA<-TYm18O&gQ;jP_2IOX!*l6(Sj8V<esf(4^xhLE5y3C%yH`%=z^H5OgZg#O&067Idl81ejsZ;bliC3XMxO56mr zZ#=Oz_cT&Sj8Mv#lK3o)$K7eBedyTrx@hPtHJe^qm0l&$*m;rBpaurua}B6?TSl_+06}-A^XY% zv0@9cv1(g3eGj?ZJBL{a`~RdhT3E;!{GqEPpmD5K_=8l_ipfe%1*q)1Dd3!$ztT?+ zj5^pcCrE4N<1>uTrZMoE@1(mATdm_LRm3hUF2&eX6h1AV3t)>VVX$gCf)Q)>O)`J3fz%}!tSW1LDhQ89}yosSUOZeQ+;fqwgD$o}+!R#QoX zuuDz5;SxW}VikD^AYLNrh4#0+0zc`Hs1>HwO{bVh>q)NX1$~6*j0(q;uR-YRDc!F& zcS4%3hE3yXBCcS!yvlx&4*aJB0o#d9|6W~*oFsVAy}!rQA4%kas@Rw+C*1W)%`3Q# z_WXfqo`r^V&VXeG!a6$SYS>xStd6^?l*(}fn`Ud*LC;}&i;x$sQk;=}ysB^314zS# z3hC%-goQh~qxyul_!N?)q*PTG>EIm$KT3slLEdTf#4t_wEju0!PvSaFe~h&W;;8z1 z_4!vRq5rZ|lW%madu;tx3~+9k+q?cHt<;yot-Of2*VP zQv1!lj?$-BQjG8aB_t=E_-&G_mUt@erZi{{v&NGH|I_jU;i%l^D~Ie`Ax%l$N!Vbu zbfk@h1=QS_u_#2|@v@Cj`=2v&A%%ek(60<#`+Q|LCB*a|mZ=^(;@ZN{j$oj9eXcqG zmSExf!^CB*QsNU1dWEy>Q>eoQZIu0GT3OS@R9|3u*?@86?yDY`gHO!>l!{^W`(4|DdcqLm!&0L4s zk0DnJuG)fB61SMK%b><1lztP@s}qW)sszc`?Om6N#Z5G{F^X$fwG3mt5fkzTaqJUo zsTWSi!Ku7AGt6*5a0qx!g#T>*vv559!v=jVnD1roobxN(rxq7w+~ws*e*~c-yLs%; zYE+RbBdFlh8Pa#uOx+dky}EGuMmtv7w>i#Q+<{EjRS|YF!+!T2htINQ;(D-mK)Y7g zz-SSHrK;zJsZ}4BMWaFF7}E&k>Dxi#m2z6f5u(yY;P-`kl8S4_0Pb~2h+?we!z!2;Ly%W+ArYp(ihrHj?; zCr;!(eKq>wRT6=_nqdZBd|LNYP#BnoU3h>cR6Vw{mQsjxT+~FWgCE+I7Pv zDH8ib+Eo5RtcVaQ(ywJTNGDOKOkQ#&&Xzp3S2`t-pXYC+l;gIcTXWr^w*O-Kz~F-@ z9>POb#MRCIM!8MO>E0_@;o1cJtvEDlT#}l`1Gw2u?SCuoz7ZAeT-~j9Cj0ZwBrf@i zs#2;8voW=>fDbKzL}RFk9W*$p3=gT4 z4rzXti}5$UU#JCcEKfO-h$8-BL~xM`aR4#OE>I`$hM1b_HP&&j2;W(PE(zs}|xZ zpfogM`{G^LPi$sGBv)lcIqkpuJzBWWMRpDj^33vdOiRR2IEl}7Nm7gThWFui`X**n z{g9@R#^Gx8ip1W0C@TMgG;%gW-19dZ6UlQ@>;nVCgeD1WP;een9xW_w_$0Z4Y(7^p zD>7vD3T7$~h0+_NQ|Yfq8U%+_?^|a$qPFKA^aAf)WiiSWrF>@5v~otPoPi(&=UpSy z--JL&p2H~o3cX5f>du8l6>Lkx5IbnLf%2k0ku1DCn0J0ACx7LV2%##Ww*&NJRB7^5 zpG7-_ zH17-bJz3NRlfA#bKa(?Zh-((v)U}WqB5IsG-d7Rj1~n!0Vx?$4&t#?1zu8JSSPw@> zICArZ#wWWiC%{baC>18H+qgEH+WaK?RUN~B1qVVpDo|Q{Co#O)Y)fY3Y>YWABGn^T zs$4v!JW4x%H3YVYFAJ>_woXYocfNoDEQ@7ShEGXFpenKdM=1Zxf60?t#PrL-7rOkP zzAVYCCnKVh7Bv1FS9>O@GZJ93HHw?u5<~x&GWH;Kw8X+~lw9^_H~{=v0Wa;3`){LP(iiGdH>;@Tyb|HZIdxKP7dss=U{-u}hD z(BVP}Yhl9^)!d-{8y7kNO?r;qdc! z-Pkcnq!j*M(6Vol+Mw4R@#Acu%Ck3;-O2k@8Hub7g#;cVqr)Hzmq`qO61C<#b2{PC zapqCx#AzdU8YN|AMP)wCc^>&Y=S55?qz@0E2_HO!S^taaoMa0>o=I{t58A(aBN^v+hn z`%f3|3Wi3SvR+Z#NFw_0k^OV>QOLn8F>q18{?o-*MZm<@9_85#{MY0A&+)>e^6N-q z;i3QW;?^W!;^!R4IgS3$TKLZ&`(Ib{^ais;78DZj$BWm7gNZk9*iK^p;|gg7vxFid zD*VTb|G(w=|CZ-(Z39@It1>m@=x)$#kf{=@m5#HW$%E-~3Xfa;5}zlS^s;F}wbp9^ z=rrpzedK>!m(R$MBy@nh<}~;M%BycsFbJe%WYKi$RgOo?{*_2s6P}MAXGz7ESuFBa zTV5WonU3Wwt!GOP?9bO{ODD4Fwtf2g@i7fvuCy^E`IUqDWRYgMVLwxpV8*ks=fB=w z|L4^(E$Cyhgn;RZ4-X%uxL<9>wWnc*b=gj+8rf}+<;tduIGtpboAvQA3I)I$tX; z?Dl3VZ>}$zfmX>(wZ+|0CxOLc6VIO642CZvj^Sui8#vzr1mnKVjp= zp0r3)PoK3*?u_S80%6ixkGoqIs263mK*xe z5;Vqni_}&oF}!T3j5j?F;5ZB#ht1b+ueZYOZsWU>3P{Bcp*lZWy3zPjjVUyPzvm79P#9j}UiW~Nap z_yLpwb(MauM#{`ongZeCrZ}=kPe3;vg%qMk4%2<$#Z-^m%itHR)Y?t13k^;VyHlkm z<9Sr_S*}3&ZrFg+7~=IdO$dG#4gQad1R_Z&4K@4dIk)9;H^sqR?Wl`0P^XL)PGC|l z)}os&0P9mam3LmfMcU(ZSd`UdtmeDdlQ~UB8?chI;|vt|F(47kA-q?c3_flKw)7({ ztHZ;w7VI*w_9}49YrYuvC$M#rg29bfnW^igpYN6rzDed9S8jnwxVzk5ZUD)aJ92VH z*edZqZ(>;FE}QQyo^jI8@wrt(aOZUofHLrCmh`-<_5hImCx4SPRqt?6X*zjjKP^2S z(>otklH&JbP4s%vOV^N3-$yhm+*X>@ewxx0Ve5bg1Ax|;CrLn0G zRHEzy8IWUxnf@>aGbfZkI!D#lgwl*t=lkA*H7R@(G7#86@y_?d+N?4VH zvtE!D4ZA|QObO2pOCV?N3C(G7#$e3p7nD(iPOEab63X{1w`fZxJJtP4-J%hF>odv} znhgDj*6|;`eHpwb3p~Us`QuLlE!$z?GT1-`NrbsFgI?l2yOYe0?uvna@JLU_m9_WQ zEoLgVz=YYKq_1uwxm}+JD|`p%7IWmY-()m;UIJ6-`eVuOk;4r**=wDLT@xtv$EC9? zcz^3C9Ej5{K=RoXXgVjVD~mv^(;R_3Yh68*4QU)A0q@*AnQ1LByKKfIZPBBoU93QR z&NZhTXxh60tF})RbpnpN<>j_$4l@t(UhfjP|5&PCbU#6t52}VoCAZy~=#NdsV>8I3 zyhz!ZERL;~@ee^LSN_cJIg%+E3x*9iu&wcLd$ZNW>ec4od=Vw$*a9(MYBjk!;R%aM z6{kDS4%cp?NKTBy_skkCwfbbnVNl7nsdq@5elAii?~oK^OW4l-;7y=rULGG)?sQ^k zrF3U7L{+j~&JjzaT%?dIhuy|G`X>01!0+%JlOMnDCdP36LY3L9R--deR+cnpbkb0W zWASI-r)M|ni(^E|biIr~c)| z3-rzRqPX~N^-z2s4NqzG@uecodP9P6L*RE1QYido19UP@@-*wur|G2)IKL2K?;1a| zJC+;SUGVmc*VB|aD9|e*z94qam0c88k;&Wo#6SqWyQ`ZHH!giNR)<`kXi+pq=9T>} zTmf+}k?{Kyw~S3O*vQV-7HW7SiR#Et=pvdY_^+?}FE>)}mi!3nUVpBRlg{5@l6aC{ zG)*nh0F;fgldOAAw?-S|&1V#S6*}TzO+IKkZ)FZ=GFg1&!s|8!W%Qaqs7G@2$1z2Q zj`1l!jx5uf4%Bd^BcJ_P3C*7X2*j-s~*(m|K_!P{n;w7HwnFA(W zF#E))zr&1Si17NsFp2zP&VR-gzBu5 zX;<&hMFiFcPbC3```)ZZ_*~t9+O>Xv_QIxlyGc!}@c?MjjNZ1x&IrYLsb!6o)-bv* zVreCbdx?k7m;$4p`c@Qn!J$lq>8#>7ec?;wXz>`|Fbm~VX`jH+%usy+bQNW_g1Pv* zbjp&Fm-6TY&YML3Ax%G}X3;i4iF2_`Dz87DEXmSx;L8bv<9XV;vPPGSCg5B(P^H$m z!7+;a&KKV;qu*dtHdAfRlZ0}ksF0PW$Uph*6bwU`b$Ce|DHdxrf+iug*%S{U*XMiY zP3%SheR*1ykQzGjuvfhwZpGa0W+6_*w=nfI_D5am2!1k0=1IrEC&{E&7Lc!8HYFKY zXBd(xi7L#Esxr+fOodeT&3cN%^~YhDk2IqyPY<*H59%^aNfk|IQKw~pise21AN%dJ zm%w{a6cojG81&emjVt%24LcM_L{syC?j221<+;}mEgv54)5C9gdeuoZU2W%!i;J71 z{~qm4$MDCfmUKgep{6btJq`p(NU*kyhNaukkb2K1DW1fI;JtEIr%|uk6{ko(1-{go zIU1kk{ZQqk7S0WlbvK960Is&MnM7&6W_?wn2lT3#OvEi(yS+5U0_ZzQt(dp@q90%c zfGffW)Q$$#s?FW{7&~iV6n4N4w>>cnfr`6F4P||id{UfYLCc88;EY9eP`>NAVig`c za^|rQNw`3LBsby}0s(d+@S%~9+gog8KNSFJ^=7b)78=Wl zA{j^?b|)H--YZ&vQGl63#>%K<Z_<3_Uj?@jb z46!g5hVikcd7(~OeMph&N{rpoU(JcoWH7qmJs8qiPa~R5*4+Srk7qWan;4roubnhm z=1`Y(vq=Zcd#%IzN;&pz;X$O1a|SxL_}4Frn!5^ zS`aQL(_4I{>a`a6DhDmQep|uiilQ>wSC3pWZzU;C_9FBt<%PAgrpLyswfNELhtX>l zr7$(W{FqCDW)u7Ugs3BTF?mX(t+q0T37>Lah((@(lND*R3Eba7(AwRf%3?eMFx}kG zI8K9}D1TKEn2d$)VFRdC{&zCYGPaxyQn6{VU;*)qi|9u1$_bJv#xk6=K6wefAAPmf zuY4VzBnSWsIKU0N(As2MZ9WhDE-x4<@UrYHK6-*^|sLL#7Mb>JUa!`Qoo0VUjEt#VV^!SJ>PXDm>rhuJSu}OJJ-CO z%Whj}gcn^^8e!*nt(OEn%o`iaXP5_p^LNnRb613rMpH;cxAgrmv9KS)Tt1R26R(W} z9yoKe)*I_^l8N2bvkl^g^SuaCk}vx`G{vWDopVv*Nv0GV6A1v|t;K5ZuXpx^dykeG zNNO|2_fzS{3Q1#5zC6TJUOkS`j{<7F1=t5K$l2aq$??~TuMo776XHPceHn@ z3%HRnTPmM(R%o1#+A;5rdtVkErMYs_tCSGBlCrU0gOF5sxLZH(tQ}MPM@W&h=1|;l zYep&0&FN^YmUC|LRhD!jh`W)E6|NDot!yR=R6QRqtq$J@io-aQV)Z*k_4*?lG+9rw zIZQl5)Pd>A+d-jp(FSoull^X%f_l4gc_Qh*ZOcC?&j0-?XCS{l+SIBxJ&I5+m3bcp zIxpxxL9(S4jKfY3Odt+!J_yPqq!7?3ksJ#^{3EVq{YX@N=?U#YZ_pUFHlqXc5KN}) z5!Ls2b&f{siDK=SPuQv}6V2Dky%crZANulBR*uE<`PySUAHQ#(pZ6DuPQj6)jJCKH3v1u_$o)u$y$OvE7w!RvM-b^c$JoCI4;w z%pXD$h6huHL8k_79c4WH{p-hh&5=iHl_qQJiMT#zaznY`NzN(riwKW+6XQuHMCKRe z`tXZ7HO1J>t%D%V)hpnQfC3qs4|i-i2z>wwy0|(Aia-8QO7GF6RL^P)&o+kzkG~`m z#F6ZYKEs+HGMAgzM@G+)zM2^pr&3CNIa*x{I)7b^^h&(UaCi&<=Dv#nO$j14laTCU zD_l&|ZCfQ3(Fth9VwtZ2z&_FQLfzx3t&h3+s`*NE)vA%{)Ce6Iav#RX96@iE20=2p z^3~r`=4EuAyObruavG%m_mQ12)k!YpvLAHls0`mc`qWs+Di`OC{KIPL{HK@p-6|4R zdBl5X=mS)VHtT&>_0*CDlnQQZYq?By!JZ3L z3u1*PH`M`3WP3b6_Vv)^@jU1S;38#pJ?Hv+c)|JU-f1$4CUV)CXk?I@*ZKmBJ;6EL z{=9a5G^q(E&ZO-Vvb7>+&Qsx$*Q+B8MZ89Lt#wE%ouLlbROv0YW5{ktQ+-SNeP~+W zm=&Sup0QHf@6Tb2=mcV4(4Zhx@R=~vL4?jR#0tF0el6n&k?_BB`&|O}92F{w>MV2wf-M2X zqJyW>dlP}Ljq3x+INeU^wAN_v&4Az7FU@f3s{cHPQ&&&=Q!K_31|8dl>Z!Wa8dU`E z20rDhRp_2bM!Lr3ZKv4$=hM$^Z>;&=^^VfTA*&i9_OaTP^9V3KB zGz4RXIyJ0Jh!KnomiT_RJj_&vgJ@hbBD*due_eiv#` z2;+Zl{RD&dZC3?9w5R7Zlw@5zAS`ad?`}yrm$Br5P{#C>T=wmJ2$k|!uDsD$j_m%D zR0s`yVKeoL`-{cFk+xt**DKLe_>j`ykxu-2BK z)a2NBFv?H78aaN35-vWmIO&U^pNO>rmI^O}4FLF{o!7IuXlbS==+sdmJb(-ho=pJ7 zJIu}@t%-%2^%Ke>P;vmfy2bAs0DuetpdP1z=E ztX5iJ$K%myl%TIlpVAV=u2T&yt@gz;e}k?#LHR^#=6iIb(%@)i zAs;^SB5A)!UWVrksP&4kkV!6<*qc+3taYuw9NK*;cL{xL@ZtyX%mo^?OosD-m2m^; zVt)^#oEYY}Zf)9+9~43%Vlaze^d}GZ<>|^ zV-J9t)uU$dmCLK0VvE`}4DHLBe^EKgmwhj}!I`!2=h!Tg{_TDMeLm%?HsGzVW5f&JaVcR!J8L27QM%);B)k1|I^b;a9OR3ff+O7BY9) zV*PHX7egga-qQ_i-#O5n50z`#A3^s3%x$XUGQ6^AwSn0Gp8@`r$tdHzXU+AAPN(!j zVXS4R$^jy}w(q9-9>=H$7=tLi7655@hgf>(AU!g13`7ivvw;7%9}%axEN~@#V0*A| z)mou_4=P=bL!eTvQ0<`4xc zlitr6dPs}u$=O>M)ouWGA(V3+Yf&7=T*9{j9M>iLczM1p2l%Y9SYtNUlmnng-lFLl z{{n)?Weevq{|JZZ_k4j5G?Thcq{PvE%ZMV?I04J8%-Aah`ZYkiaUl&!v(MWCvKV?l zAnKlEvl-y*oPT9JlEGgbdaO)UPsY9(8}&sL-)R*@bq+2&Z2Rl3p!XhcZZ}f7TvNr-o0;GM~`;{pKtG-Xz z*+3Zwyj00dp;SQ5aV*^IOL zrxzWYIcLh(g9(1rdEx=2=En1tFSbTI_wIlwx^}!@*6?87EbZ?hZy!+@yUjDb5aJ~P zoBblEuxXmhFTig!-R1U_D6c6cEx*HA0r!3 z2Q`|ntnH>aBlmrQG9Cr<`r?2flYjjng1L`>&j>`=@7`DfL*!Yf4C3LvTFSJ<9dCIT zAK4DYDwR`jT?ZHh&LZ{U%{`$-^;&BKWwBAKNBxwK5<&c$N?*50Jk2o^%Eo$}DGU=M z(`VChsCeXYT0i9atI4oBuz;j`)MZYehNLpdSg_?7c5~&y6l4F zqDwp!ktxsSN$=xzkm{Io)eae010lw_fr-3#Bb%AI=V_1eVw0PW{2=#=t<5hpxhpX8sHH9mR0Dgf*v6) zF+S(M@&|swv^#>V%!(5zpOVtne)$5ii?2-rZCLF`N%eWpK-%hxsXwb#$mhsmVh*lYwJfMG;xl`nQH#%_wH4wsOK(T3W9D`@81-iMhQMkT-k)j(a-g)c> za!Djkc~OjSe`G;PQXj@i6+k(3_lfa7dgKt_2Ns;*>vZ+cVB7F!dyUMlq7Ttg9V6Ic zYF>xO4v1|Q0al6s0{ zc#5R&N>K!3lUgox0pe-C(}2#~Hpw*Ej*@?4g%cG;DLdD%9FioW_O0^jY-a}_o|fE= zYpC|U&O4^x@Ujn}fCEX;EfH7E1@7T{w-|DGces}$%>J2ygnQUYo_Wfz2R69X+7`9ErynZp0G}nvTHPo)~E3#J%%DPc=#B?Y3Z6xPvN{dy+ zSwX|pc_6!Eq1`lnPuIG+O{mTRni(|POjp;66+}J*DPO8oawCbJ^CQs05Dv|q1qDs5 z&x|3MQ|>WY+bv-Bn(Q-V??V#bkH@Urvzh&Puk6Jv5i5GOhrS552cXYWV~3|dq;1YJ z$6~{iWG`SjxTn77T>T~}-M~_}^J;Q6Gl8(&o9p8IJCMxIjd{5kbs@~v@bkDA9oif) zKIZR?EslRu;Yc`O{@z&que2pVqG9wD=YRm1%#r;B_!?|fhMIbIAcS~B1P9-OF&XW(6;yJDLKQp0a<-hBJ zfZVuLX`DfZJAkoncRx4y<|DS7iH*CDKA&%o1wEjeOtx0#blBIZZm>o}lb-CFx0|!- z^p_zNQDqX(jtZlai3otFDCc%F9894rZ*DoxveJyr$RNGoH||45jNh+?N=!&*HVSF* zmEa!{y@~Qnzsh}5NNH=)a3u2Gij^Al^;P;tSHfRs#AZ4nHA!O`_8f~r2T*Dt#daU4 zlbYsQ+~>mHUIE%fWZRdpe)%FtHeEk{TS@jK$h3GDH#kZJ2kK+~02E(dKu=BKA1q%A z^s$f?wvkH#_9cj#`pmN|UNc|!C$c-tMt!RTRE)@k5VE>3B;gtjUU-2eFLrb1Q3~^gPPG?`Ro>q+l%HqVT3ZW z^dx01C*d?0G z3ysfp7|IsuXm2oISi8kzFmgy-q#Z=iC8#a+jXe``q`j6*4PJ|fhks6!Y3+=t zZx&n5G?RbWcTd#ABEFbPazvGTXi&d?B8X){!q}Ug>?>xH)XDu87CmfM5q=6_q&Ed_ z!E)`8$mdepXMlC!czYQ$wZ-LkIx)VN{>jZVA$66@-#)y;%z`B2_XgCH-|z_pom$~- zK`{scwPJWY<@aC%UI)U6IF4wwZyv{wk+zhAq9F4E*3pok??f!#64m%VCvcxJ&K7=& z0s5Jx@GQEXNb9wd!IL6DTRbUZ7C%_9l?pVw*q>jkX4WEMRwZ@^$xWhCv9s{f@4#{4 zTMYV04maF4EPh(4S>7lS=PT59dvhI_VMZ;9vhABOC&r7DZY$m-cYV6G65_cvnjPq& zBdi(KA{MaFx*Zxc-oVMm1Beblcu!0Jwr;?4SBfyIB?g00kSj*&9cXgnak@XRJ~Ixi z=*!yGh&UxyPy+%S#K$LZm~TP#k=@Bk|NqC{TSmpTHSMB1Bt#&ABv^0=7Th5?B*AIi zt#NB4xI;)n2=3Ncf;2%JZyW+ao5sD7;562_H{9Oud(IhezWa^+-5>Yo;a86}7IUpt zvue(&dY-BoZxZt|1b31`*x6V`nIbDmrIyj8SCTdV4aAa_Ev*=f{B|BDwf(~489q(V z6va5OI3*ULBjYq2hm7&k*Q#K91@}2jmXcS`U|o!slS!ipnhmkkO%m&&zmMfra%jP( zyNv03u#Swg$;sc5wHpCy?@3}JPf5MqkDNmj)-nzH#g1dPJh%W>Sp{`%)snh<^5MtjKJ znS6GU8p=9+3&6fifb{fd>HE)LU^#VvM+s4{rLoD- zF9g)Zb`IntubuuO7lLI<{bu?QslfeY^l7g?4%Rj9`*wFn`NB$lMc^Ov6(2sN{)Ll~ zdf)z`6-NjbLbu(yV@Lf@IibHoqF6pe_D0zbJfnLb*N%ntlCXbR7Stzl)amUNEP zzInURr~kySqGJmYe?UO>)$l{sKc)HqS2)m4k5m6u{MSd~e;V@N><p#B# zzej!j?yr#Dr}&!x^YgzdVoAu%FY*58x&QqX%kF*4q41AXlz(dpGxpB3udn`lir#-- z4S9>z3xY~I+<%IV{+0d@NyFZ0@iO;+BF$lMCU_=oCiZ^?`ajI*{}t%}{~CJO!y)sh zCe!IXOKS)?`axd8JP4Emb*(!!ovszTOC74#>3qT2W#}b{F?dchK03V2S_D#@qYzUK0KtW z;C=kRlMVkiC~X|?TbJb@>7aJkdqkV>RW3ICYs5dQ#c6%|)5hXN?tT54B9bYB_n#|N zl~_n@H;1$de2SKRkT3HygP)h$HJv zw}0B(&^Xtyq#`2!Up5eOdF;ZpGCj)opHFpr9FkKA?dJ_F$g^jtaGH|j@}7OFN%^PH z+BF_JKF0ZDxZgj*kEUek&z}Bv+sFnX(*N9R$g!&>=co2jjz1ptJY}st^8C<^E8ge}D69K5QKab^26t^~eAB6aW0OUOH^S3HiMgn?Ho|e?<2mgU6^~ z@04HnTr2&Li~s#kzOUHokpC;v|1ToFs9v()oo;a%?B}dbaan-fN+a;PV#~6%^UYcv zr_~yhkPkTwo66)GEEcMf|5XoFMYY!`b|^x9j%yBz$H&CXCRiFHz=2*w!mJd_~4h)_OxXL<>!o zP6u!Lga?0zD_=!}7n|i!1ABNqpQOJ$ORG1-4EMq7(6ip!*?F^*oZj7)I(TI5iSNHG zL6)dg%}|%rx+eUERW?WE+_(C7iB{pZtrEHA=7JYqO8K6`qdT2M@&rRUiCe;I9uKS}}wG>Ok1D>%xOCkuMt}n|}ZV zGv4Chq$`?DMj%AznstoOw?2n(y*lvgV2su$2&?UB1j3Ct+}GO16h9_e=3eWSd)M0> zKyCEx{!~BEw#@f5htrV&8!NRdo9z{IE&vh<20S9;?}p6DeFH@nr)H}&U>ilV8|@TJ zwuN%5^)Y;E@80!yMhnZp*|)VhOkoxW3@iDs@&B0i-w&DNOjL-7y#d+ud7t*qj+$Gp z2;Xscnm)|OzY{tUKT6jkvuz9bKjUl3|1GP1>7r6Mv}b8u9d3UTNgCyk$Fa23 zV&9V;+rvSe8C-)UVFXlat6XzuJuF35!KV^Z1RUpDn^X>7XGeJXtUeO9I{6ARt**)5 zYJFQcwZ31(&O882)h_)FD4c}n2*)dA$D1;3HSCSgl0@XFw;3bcL@WjNW}9-z-uXQ# z^Kj`grshPyN|Hiznrv+hrS`rDoS8N}0s9A_E_VUNOj%P~c3QtTP$k_1hI!s8t(A=X zn(t$uyQo^I(X<#fFf7Yvy}!V?awSAo`$PqKf;3L9g(kLiIWz|HkoLZd#Q?FMLDM6w z^TWGUEw=J#2MO(R(z-FM8ll=cwq!4a^TKv#3s2sDV&ND;8WZ9jjS%*&Y^SvBdnpP4(TaWii?dOZ*so>*ld2VM?EeXTcps;1(&-Rl^ zBaZvY)DEeGG-LT)c zwO+X%&|uFPe%t;ycE3l{qx13eX@Sa<$yoOReTg>D+Twy0niA2-X=uWCdHdqM8+VGX z0QOMZgzkjrJHhAv)~(IHj1wMq`XdfmV8A+Vyj&f*-n=Qb2^2pTnmzp&(3}>Q@$b3x6Y^#iK**k+u z^B(0z8fNtTXn`r>uOA4C!+8a|0<$Kl?Dw>3sGx@we-_(T#Ys=%J67vDUbv5Zi4V)T zC(H8E2zDizeLYgPyP75C*H?eqPbKdj@2b;R-?64l>Y!7&gLa?XddhHPNH6wXp^l7! zK=pwgz!}-NCIEF<*Tg63_0|?Lxyg2&9<8=UG%3!`HM+YPM`zQtz0Dw$anxw` z9@2I`o=+;-<@dWy`X2V4a_X+HG&Hd}h&#V_Q!IIsmUt}n=2M=|9y?9j`Lv1uDTdEI z8mS<+N@XO08fQ)(KXj8f11*00$Xk*?6f?(Aaf0z(Aj`InPY@rAX*v6CzaTX6tzhl^bdD01Y7LGjw^(~*q2!t%h`}->efk$ zS^4FFkkeGc^e{`>1;1QOOFkRl#@0k|Zcu;mewS0pZM6x)#9wpm6#~vk!ui6@q|CoG zs<)v6AjnW*zvYE$+%+`YrjcFH{-S2$eApyVR+=~wz6#F}1}@xX!XtYRPbA9zU^5{m zT%K=TxK#3E)gG|oZpXkNQc`Yjc(_+h(H|O5;-}}ba572g?nt}0DZF$QCYt({KSI$M zGgQ+N{>0@%X(C{{_3mS|N|U-PtwQJdNr@4=&-cL7;zyjFXpE;@62PQ2NI2&ks}IPy zL|(T=iG2GT>z?~pb$YCCFa@>GyU-u=JqcLrrZ#kzvO46e7B#&;nc92||JEcnT^%{B z`NfuO7(^#qtQho+DpN*CehoK%>DYbkPAP#2PYrPIcr9rWV#mTYZw{ZzVf%0MP~)x&OiJi$&h_-4=yp<#^1^*=4((tH$k znD!_Te*Jb+u=qvyX?-JUp%6fqcI;RgzJy6iaVr~c=|FLoRijQ!<8CDL5|=Uofs>guq;kG-kYHMG06$eOc8`r znLvJcyLc(*s6(5Bo;qY3&Zk5`{(9p~NLISwaQtRB;jRFN9eYY$EcO($1nf*v@ei6- zT4Nm_+~L8@GdcLq1NSeS9f~oVExzy+?v?pSxPGqf^g2XXhx4K?1de z5AHoG%>7xEW@Apwx!zh$DeJ)VV2wqnd0Ut5;L5UaRozz$?NCia9XqU`M(=Tgv90M?D zmT8s9OC4D-wWZ19Nr$Ah&YE0JqVXrGPFqyDCQr`#c6SFv2^&^gzq_~ev|WnXz=;*) zaSQ?H=c!@5E_!Xv<+6qZ^vT-5Gypz@&!;_<`t@%@=|tAZBQ><$?QdJH+F^i5o^84A z>;25CITlW<|59L^FBXDDcNi&HlMD?9<+s2u&)$KTqGt~GK5f69PcZV{H$J#t!d#Wg3S<Djoxh+ z)hZh)s_c8C3o!Uv%9NVlWkQ!^Ti7q?EYTsI-eG9!Y+As?EeD45+8xsxC%y6SgM@-$ z4xC>hH~v66}nJNe3H4hCQ1^~Bp%38z*ep2Ncz^~qbP|IO9AxJ92eXGbEPP{hhyUQPvk7 zCD}&6=(&pf5SP)Khm*$1D3L?Ha;nwB@llPA;qcP}!ItfllRn~MofI-)PNU8C>9twM z2U8aK_j0#pLL(z-d{(fITEcgu?7YL?J5a7mYy=nZ#o0?SN8G%4nS^Dc7}$`Xozu5i zl;U*A`D*+P5r5+V{rH59ppvq!CPQ&KRFiq}#amdhi4E3Q_bW!;P$@pjg3jGs4V+s! zq;$JqFLECNG8)-$y|=?dfE1oz{hI`@?Przdy_Igy&y@7cstKiAq}Z8BMsTZQ>)UB& z4|!?R>il}cK5?qy{1NE&I^TPPh_-oGxE8|@-9~cqR(SNWYzOsZ%^(1c;dTA#Cs^LF zavN>1fBAKWm8e1Ov$?xYNmT{wJ<5cTo013VzpMGBn(u~s zdFBNomNgk$yE>l=g&ZVOXxjyN6uo-TxOu)MXJ%CETewGU-_rPvx2Je^s^J{M=|PO3 zYYwnxPwz=*h{=KNt+%cu>%M+kc2rB_=@>j{Z*cPS2&}I&v<~VBLJqcoYexHl8AYEb zsg+&6lE35)S#oOfQ$%i=D!1}<8o?4$*h>8O%3M8j7H>=&6`NEgAs?sY<1FY+u1C4y znwkAwD)^pere!t(uMkf^s4!L56jtW&}h+p+CD`M4-SLEBxBx_c5VXF>WXSqPZA zUi5S9?+|o7?ek}(*pS1k2l=f5-bWdVa2g3asV&Oq&P!<`ZlWhpcS_*d>FeZimGaN~ z)nL7u`oN*|7Wu94VzT~n@qV-VKDF6n(Z%}UL)qp5{S$>dZ>_s%%Sec5OK64 z73&_T-iD3kg|)<+TRddpr%(DpX28q-7lAG#sd2H+5(ix_wlNt^gF?CQeuLiBO1_Y2 z8zOWMfO5{?BnU8WfvEa%OT23nVTgO9dwR^=C9ra_n=jto7joy(S=$aj$FB0$+YSP^ z&fdBkp8!;`CjyRe(_vmaO;`eN)|#r1NlCl&;KY@Is#p97p=2kbt`5%Lhcm&U#nGmk zFEgL)i`bk}SXxo(%WBRHtYsKZvgNfHLe)zGc-gy9NWF-O`S0jXNT^Zccn*)$NmHWR z@r$Xq`gta2tto`2zHUw~XM-TmW9*w?HX@~>SI&m~>tJ4<#U*6Jo43sV+4}W1B`M+4 z=-vMK3gM-2{-pCIXfk1dsBAdHPPN4fLBrk|hGx7hpX8;K8jxv1dFXRea3U^`2_5GC zqqov7jX#M}zzJ@5DC{r5ct3t7#uV|m&9W7gRXMY%0_zmNs{|5C?~V?&+!_y$m&Ae( zT_tz`$#H}aw9m#$oWh6E*J@JHfxCv{^oU))-)-=B&j?_n{YXPI6YI; zH0YGhA-#Sa%Y1O6b=?NMMqL$9Px;!D{%+U(R|;gkGoRofEGvm(a-}YfRr%P3xDc#b&RVi3L=- z{7>{?5d*9@7qd{w=t<6$_WinO1)&O5TqknSR*^@`lk&Yr$;j@#h*gG!zjrkq}@P0D%)@GK=Fk=w>r z$LllhuNv%jbzSiTWPS`MeuDIUJ5@)0Vm%lyQBUj$EC*RKdSO7SN-3=&u(p~lc2nAI zlLgm`N`|f5Mp16UoRg(A+&u%-9siZ471P$-`mRCtT6J#fbDWZP3UvdrK3toPK}CX=Ku+CFnJHmq;hCxK zhNN)6({UN?#+FTdR)kCHCA3iKLNgQ_71?=9NO^R#Sdz^EGKT3!{Fonc3idOFO#!eL zS(lkbR0RR_>~%RR>#Ez-Y*yq@84nQ`-Ss}r@iB~O$SBX#Ga1v&Zl^EY@Y)u&1~xUG zIH6gfYNOJ8oXQ4-9?wb(WzcXi@bHk^(CwKbn$>`DnTSEnjOAsP3PImg_44T%%d|^J z-BjbfZtg||Z#cf8o z!KxJ|Z{IOr3^MEhwz)mZbFdSH=}aAKU1MWAsYT3dOW|)qf(QApX6g>=@sNfKpVtG# zgjJ&t%Vu-{KYh*&Z{sG`UX(d%RB!*DQ}3Vr>5Gh$h@Ts6P~2K-7E(0Cb0B_*(VRe= zqS?$44DiGZ0*LIl+|^yvL4dctAu1YV$U=1^lfn*m4=*o&Gg#W6WRWc44b*AAN)5R1MrYtUsY>vP=Q#n42wQyt+a+S- zll%y(y~IjQEp85EDDy+BY!rf-C@?^`u_3v=@U$JEWS+veZPp zN}F-u(#;o1S-!i_$e{x6b))nd~cxzrLaRbK7B5~6?>5@=e>zdHKM%y z*ZOr0m0XHTt_v;6*yeL>Q2ZHsBX0*JU)5;m8bub{(t6jpVYkMr%<3($ee+PQlV8?M zd&IC`wjF(&NZb{)L4PaCv%J=4{#1(W3h{J4LPS`e@4E9Y1k2J}&$r<~1|W>u?0J>C zvn-8aQOM{`JwRlRGdh)nGHqz4M`r@@Lz%-L{KZO9$W$o_8i`er&6}!~NwsEX znj#*-SB@nYFfY1^g8fA@>0O=CG|rEvop&C6Cv_WGc~kSWCx|?CQDG(rx6ZMDdxVX1 zhIsg6!`?2aPr3Nb^_FsNZIt#(#1*?#NaU2$Sles?i5nz>M$kbX-6Z*-GaQ$OoyEps z6@D8c{yc1_dAzhl>q{1-=>}bHD!svPLijCLMQDS z4x=mA@xv)peut3XEIpn6FMQ?@M_D>>C_v6z`+L3sCaGN@Gw8V0&l4n;n(B%fAAm>d z)zn<3HwOpY>J^6%Ro^ZNjfA_=6@QH$|80xpG3J^=3hXr@L&1h1gRYrSFlGuk8~ zXuf5SG$XOZFxX=WMCSPrzpjCG2YcP4rgwtdkvj1r1HP!%iVfwL3=ex_-j9G+|FQvh z>;)f%^UVmz$A^EoU*=knCQg*)=AEu?%k~Xho$RQkN$7wu_V!=5l_(Y1EqLV&D5hTb znotREJ)L*#QsCEF9aq+PzMJ3Brvh(diXi5h7%M4fQ~ z912=gH3?cNIteJh7>C6=gKR(9qn>Xnp`!C9Yo5z$=GdP;ywWQn+QZl-%z4U5K?}CM zdlxWk>_$ok;ii0*Ew1x?3vWL+IAG)u!VW2NhhkH-E_s^v&GzDRV2i31Cw?2zbYKpz3{?M!d=LW~H9MT|Y1?OOayAuAVm_k@A1ZS` z`ju@D`AKyQ{ha%EO4Ob@<(qX(0ceS4{G4*GRZOS93O9muK+5Zsg7sCO+-Ycn(J7Ezk4vGb&l3?zz14*70zt{D)@&x~xf}@L@e|TPq=C2M0 zzX~{-6^fq>l0zv*f~|MhI$m{+)+@KAOKs0A6oVYf&_8EraVfDYrAWl2cgBSuh9{A` z#|K{?RlICZ$-?L`bMFyPsI^8-zzRXZgc_hw&Yr&9nZ3X9Sh;pxaP6S%I;-9+D{D*h z`Y-p*YH*S==)9HBJbtv zX4P|h_4BN70DWs{t_=PxrdyzuX5#$KXPb3o>Z_38lkfW?UV96~KIhlW-Ep!E``h8l zfwilzM~XoAp~87cS*-Sn37K1$gCtiWQV zTJ*i!jMoj46&~eLXvw;q{HkSxswaeyhTt@vpf$cFFu)$&wTduho^?jeIS1)Lb z+0=Qu(u?g8*Pxv1E=?uDQBFfVV_`Q@_b=EP_oY9CC^fUNrS ze$l!orf$LOW#ZeVZalv;;}c)XHXWB}x)RhV6Okigi7jyrfGv_1b+~EzZw4dFgN|T& zf{1d7Fk&3{*S@iH#A-fo7WH))+-#Uw+`QP-MMfKI7EU)%>{qk`i>_*As$I`L z*)K5x4$XbkCA{(R8enqIx^fp9rHjAW4<)jgD?UqyU5oO^0khx;jHT0~6BTSd*v zeK(4h)Adrxyor#(+O3(*o;QUr!tjEB(17TDRl;&^h~VAGD&mos9~Mb;4%#(j@z=Iz zbBh@lIMC2>sKzboB}_Gpxkzh{3njyr!C$hO5oD;N^0pDfz);h=RAZxKrJf7VXUV^r z8yn%!9||HOrWx2tuz9aW3YcjMbaxjhgl;b*QkfQ~<<%00?S@SJNxGc2a-T%3EOU~_ z#rRPg3WIjHQd4OSfu7X+jjEm`f2mz;*D?Cvt)o5p<@v(S6URl$DLFIyX$kTQFuXmK z{S^Wy85?m1zq?2X6>>&8akP8wEO>9+GAnTLrj0I;D(TdOPbk29a z4HuKq05P9Nj>`N6kUshE(x!f8CKs9N4B7kEsJ+pxPJm`p^L3t4ef6k6?oSQ!dV_Hb zDN}@>1xX25Yc`KkUmCvJ=wt@M!r$=kEo4ySu`}_9r-_LNp)nrzna(Q)F^QWZXV6;v-D+G$?7wBQH zYcM1S%XK@Zwm*&J+4qi!>(!pU46psX_qrv-MI&#X zg!Mbr%Vw(TM9f1r*BAr~70mDgb2mP>oA=J_hX-l6DkjIVI0V)zU_Gs=&YzSh#4hKb zl}*xKba7oc8Bgur00;M0ES8K^U+B&6>u!BR{qJ4?p=y%Mg5a2<+^%BXqx!SFK8;z8 zoqL#NJm7Spemy6gc16C8OP$v@e0(+1=6JkyN-E!|YyH=R(-;xJgFbPoyLT95nPJz@SMMUjy39IVcbUy z$|cGnV^4KG$F8fov%O9GY*|)!b-v*%x!g${99ZInnPh>rDfX`{qRkOmie`yHmIKV9 zg5^KjHLy^5e)Or();DXTsZO%$roy;tH=3J(Ej3ojHvm{$cD>5ai&7^bX?CPssX%y+ zZRw5EDTM)Ii!F^M0m&qyPbuWv@F=W^PZSqZl=eBA;0b#+^3ZPEeAGy93;YFUua!9a z+<+*idedn#W5IVAsh^+ik1wwXO8#^fVx|-k%bdEls3Vlxm`9@XN(q0-dnQf+Z-P`G zKKq1Gsz_X@NEUCET}NPo$;PaUf#V3xl;L-hW43V?Nz(7F29x`^-(DDo0|jeI*7?~N zB{Ys$b(K&_@Y7x_Lu)g<3;cgi@57>6NX^S<4Fkg!U5oT8t=z`djHZ}tAcfc@Engl? zWQ54vl*o(f^6|O~{JC>Us`6!Y|{opUvL={pMxzy!Y=9 z3o_vnV=rsdosMdqrEhqM_Oe@X0*|*l;wDbm7Mm?&b+3Ok-;af+yaNQX^0C#=)Vkc> z92(}uqbuNSIVlkqSBv-=Tlnt_@?RdlAU~(>L)Pz|J$}-WTx{25= z>SO)rns4vk!3d_x68QbYir+Z2c8rtss4Cs17jGJ}4Z^_S86_^_X>YgLE3;chuhzES z0#9nG=ocQ?7!!u^Y$%ieK}w)lB-T zJUeNpUXXuNoXSZWc!QvTp(qp+5IJXY1ci!JBHV}6+pV!I5AYoK^mSwb2d`9Ogd5YK zwe1K6NSrJ|`HgC};o*R!qCI|Q&=W^-O&exH7gojR)8pi*p2p;aCk>j$NWt4G7H&Jq}#?7;v@*XUbdvAJ%$~0Gpk(EGlq5^9y<54jJ6Gda@o9GAWTXmoIlP_ zH5>AI=Z1@%QOlt$@u&8j1nT`T83w?hmHzf2~%$NMZ;_aL!1io)=HSs(Rc&&K5t zOxRt>ni<6T$rGrsKX$j!=3;pi=&3R2cgd3D{W`g!(%l5pSEVHc#?l5UrGm4lf4D7< z`|k_jDjs^7>M+7ll6*I+ESz)S9@vX=5u~+PMAmjxRt7gy?-#T%7q^=1K|Cl*$`vOQ zh9nw(jgOsgR>|6OAwAp|51f;EL>jLc*Ex}Kj<;ke%d2$p-A!_FUcoO4PuA0*4P`_1__w~;W%pwd6G8Wk&9}< z*HlVPlHV@JW}cydW$ssOp;F`qY^=Z2_VXM*pqXYQX~U z#r3h2VLiTev-Qh@U3UE;r!Ma#6aV~#ei*M{Bq-T9TilhCYJ{xVhc@kzI90Akw#lE= zIjkdbf|K#8mtQB+&{|-utb(a$ie1=cjTAb>1W1@H(zck(tX1s7V zm=;*&M&2Fo`E*j;v)D%@bhx<=sMR`K&ek75)~-~A^s%i%ZK>UT z?IMS8LDw{N0ETX|oY7<@l93k*XKFSCE{LzJX+6T&?r6>lIuSd>o}z9#j9K;i3)gZ| zx{y--=-jbd+2p9#YaX^~W0W;R0yFP|CaYV#TN+IK&4)CcFRgS7Q1&EadHTU}Il4QY zO;VXwrKDCtm+c~94g7%{mn&<#`)O|!<;u1Bv7>Likk)PY`Ym^H%SsyN`GR@DoE%`TPUU5}^}tVkCk8F<%#cYk;*fXIPbpD-)J)_`QCl zJGv|-2_+wWR)BI+f=`uJ6n>t6snQ?!0iwgok>d(M&i{PeZV< zknra@A5|S)IX1p_%cc{@oCvIlRv}$^-$}XzyQyDd_C7Ku}($!CvHKYzU9v6SocdF-!@ zT8ru@4OSE*afw3`iO!m3NzAO{3YS5AzCOybSliI@*=?HX*4LG}S6jqxacp@fV7JMj zf21!vGdC$8eK87-{UDcwW1K#KPzPN19&BK1lwH6qeWf@YC&uIX`jx(6PEXM`CQyxQ1Jg ztx9?=+Z(ui^H{%VB$S1boy%Mj>4gRebt#bNp5=^93v1ZnRQ=QoqZFc{tG`^e}_<(aguRA0#ZaQOA%O{h5*?N*n7RLFZZOQeS#){S&I6F9HBw zx_J#6zLx&G#kz;k7e%+;-cHlo1NVEhS&MKEsTp%Tw6}Rgcg|g7B%V-}&9MiGJg{Jrb>E})ub=uWrqgnYVSHl# zw~)GZ*Nxrl#8et&OCYBNGO(($#mbY7^ekn{yMtY=*vZYRz%e^+0p|d++p?ou15w>8 zXW+2)>DP01*Nk{yv}SBCE>q@FdH;RRO`QTe-HGEt?FE*NUIzELXiQzPCK(pKM=5(U z5)urMn3HJAwV!nvaU&}QrT$B^?|34-`=zxuSFun7p{p;Hj?gyLB|$ZW zS+I#It)UtAOjRW$vIF4FjE&a>G6Riu zt|&D#dL~~v5wPt7%-f?a->xYDRLCA+`{`zerZ1PyI2bhcw#8}i#ugd6EzN4LsYu>u zx;ve2hnVdpi)OaV?^j6CvZINKj_6XUku>_BmibeRvS?*sji&wS@b6->du#~+4bXd-fB<45HYT(Bl9w7>PR^C z^QP0CL_!HPea$u|L$$({*V7bS!S6l1&0w=_D%o~2(u(Y#s%!^DPaiVjrwcxb(DpXZ zaG+>+8yE99bOQ+$M)(n-AbWns6Q4dB99k{cnOxoEv|N2it!6uVz(xIB=|{-$=g$kW z`IOQ!xcCutjhgS?v(}Lq%iKnOd)dK;6H(64VN?r|g5CFf=eND_ z-aYC5Ed0X4XM+Qn!JeV>!Sti-vrM$Ft($L-?_g6Wj9cZ$Yh@Xl>k!FYG1O6;Qu)%d9cWYHbU|LdFhC#J$?Fg_6qU4SlzMQ{pKDVqcB@JiL407gg-X z&f3)=0Bmuricj3SD{sQ%^}UtowcE0_`EH#gpXVcU0D;LuzYA;Yt)E4Y-eB{Mu-<&m zzJp{!TifaP>vp>@O@!axs2-}dTCTYT|H|CTpgHV0Yqd`g5V2(%I2_X5rIosC7rg1+ z#{xpbpxSaCgC~TyC#Oe}RWp=FZ!=5F0!Kr=Qv`F4Na96(oiSogn-BQD$`9ae`8x@k zKXn0t+d8W{y!fIox2@drV#PKl=bjwY0&N+O9A|igFJoaF1f|{*myQ;e7g|mQOk@Mm zu>QSAHjMYRg5=`cLTUUjF9HtM%dXdsGJ+Rh&fLy0@#|X~t(RS_gI7w#%G~=nW9Kh6 z`3vIjh_f{9#XfL48HGbvHw>;Y`w=y9TDjsvCcX!YLFRA{!GIe;thUU|9je?;@4m%c zgS>i=!$f3hCK{HGelPh!YTCUsSY$8Z(8c4X=Dcfh4<0=0*x3I({a|}9J#%qVTN!G* z#UDE%Lm03~aRC<>TMsxt4>n0@yI8Co)V%0gVkMi`xuI7@tdUaJc2QWuxpsrh>u%Q6 zJ^Sw8!MXSFDV_9(zwY2b=sW8q&sR+%7HOM{yl$^q4l0}48h%-7 zF6ge!dR@*98s37qVed(f`wJQq_$F-E8awVBGlmpA&sWr{efBlVwOS|j2aY4BDOc&X z+D6;J^byhQY|#27prI};VKtNbX2}!Ux*e{D*>njvW9o8Bmz7HfU!Zr5#f#@bUyve3 zgXcTzS6M!?p5MjgSsxV73zr~wZzf4%&rhVXTxv-iU`KiLoKyN$@0i^@zx9`umL=|H zDl4A#VTP1^ZR(7_^d1e}`T=kBjHI6Z;(%%^xy?xG`G6L8y_;?}IxxD92giFy6x)BP zqUQ|C@G8~{`zMx7@-}YT&RwP5bP5(}{4LvY0<%bvE)|Xm?fpZ<&+P|qDKduA(-TF= zY!fw(Hz;XYfnO9|m3NBE-Cn8y=MQeVQYKn5QX85*$K@Amj=Z`0ya+BswV;F4vwSOo zk`6E7PxU{sQP! zYXzh}Uzku=iKJSdi{&NILBc8Q2;sMdMbmpPGc46uE~ zOa%KJ9{IbpXl~P{Uw`~wW&oE`k(#w#A*A(p6}d>GV8P<&sdjT3bx290|ldgZpL7Qf2XRP`nuFVZNUxB5_O4?p1L!Yxr`nCc3$&(ZA1pIA0JVykE z|F)Zd8IYMS{`HD&M)f5R^>k3Jn?g?Dwk3T9>f{jEQ5bAZ6?HiKaf@;#?dZ^If^!GVlAtiTA`O_19^==Zd1GSDzXn^Qm4v zWY|zeGi9bfE@Y5wHof2a^IW#=_(pK0;!qRc#=&Mi{+3T_iHI`Da>H|bFGzcKfZIu` ze#xzEE_m605(weABfOX6k}$r};5V(ht({s~9J(^99mElQKyReS5^O=Dv*C`ShY=1S zuzXYH2bxO^8Lhk;?ZsKf1cpwiDbi8Y(J+Z@LPURZUU7b(Tx6kl)qABd(3BcwI|{RY z5Sg;pJ6eDM0&fJ6_5nvzE{%0NFY{6ij%mM-F-x538sR|afo0Gk<#OAs`ZGD5Y(V;2 zP4hY*epcunHG*nu);fsA!4{fsM%$$9O59qi{7Mdt5w;?}jsA_+aWAn}0uv486CLa*$W>MHFAK)|3XsRtSbf!`kuq*Xc#4aK1 zxeT=RV{uZ9vE23ODTr~T?vjH=%H;q>p5w`xl&v=u{i%3iD>|Ci^u2s|`I@Lt5JVlr z{j=A?lyiRO(r3fp2{oH?!v^ngbqch)&jYf*^?u+yajq6n_5ZNXo zQPQ2#4Hh+o#DFlQl)wxq-GU$r(lsLolrckN(N&P}WVQUD#!5Gs?o0TbsVt#_`-*sz zZ`;xQbCC@Hox=dNn@Xaak5n`YQ4+efllbMIX6<&xqmvxvVx$Wa9cPOx%_5~OFC+C- zBdX$vAYZP+OY zXNbtbB9?Y2Mv?J7VIL?4vMkwaAB->6!ZIorm42-y6l;CEdwfaFlqHS8r`hj&% z@yj&%cH>CxSuzQG&vi(f9>)Md(#+vw0o1jbZrz+~Gk2UK%#a(}c&+KN78iW)8kM8m z6kQR}w`|v)zIQB{Om(H{HkTjOLvp9bu3n>b#6JIdE_(FS!WcQ%T!Uhhwzh`pi%-&w z9x?SZwXa%1I3T44$921B2u zpc9!bvn^?~T@%n_p_KUjAVnwlw)t}FE}pcd?u$XB-{w;DdAuHmZoNuNt0`(e+%(c= z4Q$WT@=NBmr}9;8$w40+Z^O*L6bpTtYjjoOrtayA?e}Ztxa+6y}i_H37pSFB3^t|T}h6uzLogu@Tud* z3kUvj$U zPiU^I_P_-}JJH78s^`R4+mp+_?7d=qEt>PhBA*cfwajn#gf7J;;ANdO6l>J$Xx__N z9e%`dmHsWyE1gz+iB>U*^QrCAO)QJwkH;x+tJ87zp`Al%4NakVwsI#Tn zs%?%;Qq>qK+gTOGKQeqY{d(0r)176vRBIuKv!-(FU9Z&@nR_fDsA<{G?s=`i*sHWC zzpMba$10(mb)yiztBc3Mt5(Rrtn25A+Y!XS%qv$FMtsx<{wkV4Mg0_9UX#24-m_Em zz-#q1xB}eFg}dr?$Mr56x!e$Ux4v^Pj}IvYs18dJ8b?Tl2|@oOa@>~*b|i4x+qq;- zf>k8&F{XE&KiyvN;r-S5GQ5)Ejve9;uy^ zx#q;3SsR%jlwcZ2A)uX8O5q>)Qy%{MFwb%JY{_a2XEi<7cZ_m+m>I8w@#bGcxy$+F z8+O0*aVpZrw961h|GXIqxLHE9+&OA4Nzc)~1w&$u zwil>ftGwkiv3X=a1ThNNbI8P{VNxXbAlJ%eC#!bMVS7wT(Ru&Hb?bBE8hQ$k5xc%q z0x@~>@YnSxNx_Yd>{r9hb{^ftS`A=x} zb4q@G_<#A_c-{m#*j;ilL6*CV(sj1G=aQp-7lLfGfUu@_*Tpw??&nm259jYo zdLSO$;QM8Zk{mFPkH4tSO>8wsI4D+Zze&44T;ZlmtN{bFik_)ahxXRsG4a^_Uvuao zIfsLjXV;-UkNf-0|28}i$?G^}*s$yJp&Uei?d_iu_}d5DK#w!9xwaPxzbT5J^YMQ^ z*m{C%{&$~$PRq|3```ThPk#QUB>uO*eyGO(fA5RBxFfq6&^Ts!jeX`ff%vI4{`0{{ zUVrk%^_vBT#nx0L@NA~^yMK};@QW{$7=_i~xNxq&tNLG-=!ZFYtSosR_WD7_zl)M?;Z|7{=t`)aDXS@M5*2-RmFU^>ViX8elw!;c|zEy{q{faRzwW|^7D}y|F#gAgK_8n?fYTyrWWTQ(0_BEG2k74HppO= z5Ws8~d#1Ad+eK&j=l-#t{~5)P3Gn2{#2g&1Q;GPOi#*Q6+WdX;_7Eqe;O&>&J1_Xu2f zhD<@Pvq*K8U6uHI!|K{kKXoNbck++Gt9dyLsef(4U(cz#gya9}%n0bTAPzdlKAu(C zcz%&X5cCV3Yu&i?MHure#$D#%wnES<>6T^&x`|O><>xyW6{5J_IAIg%xIEsc?YpLF z6!PM_uHXcbP1`N$M~Y6FjETv-pdYPl7=sw-W=yiJ(nI+O-rh;PcB(#4QcT)C2d8&n zM+Tug>wsS75@a*V*pu{UF~&H8gC9E4)$M@BW~MID#*t>BO_3LT8^`QN5G{@n4vqfE zHssw4(#O3 z&X^;KkR<=yWhyRS4El=5Ii#Mu(5(_|N>{GLU8uTdYUe?I>hML*swbd}?K>;bXEOJ# z#Zc;;A)LgTWJ*;@%&v*TfiBqrh(LmAf|Khlf_C3Urqk@gmPcKj7?%&jT_%ep`l86p zIp3U!GqJCm6+TwoPs+q15|EQq7oXZ|o(sV&cE#Pgu8(dy%rnS#F*;_(!hr7H0CzW& zaf)YaXgP`dt2-gZHf{p;+by}9r#%ZVRWb;B(Ar5X#<0;}_AvtVgKo3Qp{jf(8g3e8 zc=?8$Cy_8q;`*o;`^1?UNK!swx{%YxbU0&u_*_++F1?_@+XS1ra?HU3ZKJpT0djJG z+p0f>C`rk4pdYe4*#x>j@}TE+t$4b4Vd<`4&h|H3GEz_1y0BQN_}bnV>=akr-WIyJ zNa<@b@=h&Qs6b{Cv(Ee&&|lgs_{@lHsK!##-(Y}Ww#ri+%UQH8Qa&6{1d=!-T<}>d z(bVG(Mm6JudU$#9Lm%}T%*DnieU(-{(A++J4)rEOoh9h~T*{8g_czV&8&t4ofH<8t zMO)0Dr_Pp`x0QcMklyYF*>N$p2jXk&?$gBjWS->~ADEL@CwuBXpE`1`B}}P%9@(Fk zmJ1a%QJ6S*)skU^?vEf&;T!?)U5oL`pH_W(({D7-h`M~L7xGyLgO;YQUq|(ADfn@! zWCSV#r>A-JrW~S)<}~4zv;uy>cB3dM)>({mSD4b_w0lk-PwZ(=yS5GojkkRWvzo`m zk-vVUrK_0Ro)P2pIVhqBZq7AiD<;lXjK{oIZu|T7Q=KOL{);$y?aqeMz{@Md`Zy@; z;=c7yF!V2{gMty<@u}+DyXbLw%nv^>7rA!ovz^gin!s!q6V9a@Wga(1>QUxvUUJltv|X{oJ^2mT_$4T3F@8Z!)htwv{h2%#+NUl2rKtPBn<>Om zFT}*LeF=u`&=#h9x`GPBoP~EsoAtaqniZ&Z0=>Xudo^9xKY^xt+n=t%vScZ&fXkc| zhDBr;izpeATDD6haUp{ zyH+Ur0(9F`+OXbtDV5y|QxcCJXQ+=621>+fifkfEd{;xFZx!@_1pD(GKk*m&?xpxO*{gwJV(jXqvVIv3mCn1F}6O$pjl#g)=$6n!Lu zuvIP>C@6-0AT^sB6v;5UHj{(a?ckj|fJwQr$vuAmx{hrFzQ8ED1^J-9S6pi0eA$Z0 zlwGg(Jl?4iRdFY2;=L?l&4&6rm=psH(V4fd<8sZWfkLy6pb=#4cD9ucXl=(F4Z4wO zn#LDzc8CnD4rY!xfL>v@n#ebSsTm;P>MGW;A2J?6lp^pn4$;k~x`N!dKYb}e<$y@Nz2y@^RtkQQ%T;+mb}Tt3_b zzSSrpvBmevmbFJa6{RH;?g#RgI;tdHO70tBdMEn>b`gc<^EC&nv0oH)_VDyd?6xZU z3*yW=>bAat9@JK>b^WWiu|;ExDQ-r3jX!C?2TQy>-4bTuLX?lb1rR+=gu&3w@DP6f z5i|N33&fK|o?5-VR*f9sXjFV)_;@_B5ES?JC8hiK*onj04`QozTA52J$Wu2571v!) zcb=TZ9r@?#mZg!wvQE>+{6x}wYbVER0RrP?Y6+A0Ba6e!-N~)?Ttvy@oq^_i6(->dj^khqyIa*2BkfwAi(tcv4 zlD3nLx5$&YCvRxWZbwupy!H96g>=XYCJtG$MY?N`f`HqiYGTDqB+Q&Fanc?poom6q z?oan1vcB<*sm{zWa^arMbllBKvFw&)BK&7TM{O^4U)kLIO8? z&tIT2V%e>?1G*+cIIDSIfVQg7w$(W(k?=He+Pj(#9QYl41KRZ0LXpNzDtlmmcIkjC~y8{};0(@AQG11kUTERz?h9rPQLsRv~I6zN4ot>3Up@9jAD~Cg^ z%f!L>fS@$!TWc;h49;_?ytZSVMd<#`D;8eRVgs6>^&}qiCd*8=3rv7UdMA4bD;HS$ z6QGKqW02z>m+{Sn@Hu%ICJr*Ql}x=O^MWNUZN8$hWTbfbIh|)m+tJ|}bA2$L(Z{<< z=Mvh7(c8MveRj8Ix4JuEBU%S&6=H1R)v~EpqU7lI6d4nzP8CzI4kC++;~oAfE?(Pl zmrS|^P#YO${I|D2ldGJ4OL-@ncwc$sg}v{&f=9O=N>e@*i5C^wm-@{L{M_)K9akRn z)%Blc0Y!6i2<^P`+Lp>Igh6Q)HF=1)csJdGZ`*3w5hF;Mda!M$CT!TmJ~_ z+CwU7;F{8h#cpha9&Hs%IVcpJ9Rf7$T7ddQUXI|HVc9o$Nugd5BQrv3k+uGU)0J9b zM%YE=D^aUlajyhWh{Pi@Sve7~iRy3l7wNs&m*h=wo2--0N=(=d=s@e(&U%ctWbxVo zoz2*b#BD{jb%18Z#G3MTvktWfI_cj+f?QLC_b;$!1slcRB@m*sGJZ^g0<7`%zQsu{J_A+iSdXXBSp-?oHdBOLfMlY0?hZ8$6D-xA57M(OpkM^8BCRzO$N zux)CdSP~uX2b-P>R@fFPcIVcnktEi4x_D0FRB!2BoN0(9JUBV*Jjnv;!}v?;x_9@) zet)S+NHc}Q=&i2(s)*bV-b1YepOUr=&Lq!Ir*{dQ>R{QvlgxzN<6m$qmmf8KA>NZ# zU}>N;WvJzM^1d)B1KMA!PHf(B0-%$9T%|@dZJO%l9?r_`@sImj7`P4b+kNBW{C-c%0F)l$zU}Ja6S)_6eyn~#FI%&ROB{j5z(iT2S zZoPrJ6sn=&@XEhBOXcAUSEAP}%J03uW_;Sss4k$<8&Z^09io4eg_eER+^P+&TnTeS z>b}4}DN9jx`!?%m(8)S}>(rW*3FBS%lEJq*AzQT{zo}$LY3yo1o@-5|Fsf^5$*{f2 zNf*~EaoOSx9W?Ksls_|Nmr=YAFPAl_L`!Cm`*S9t=&(uT)!PsypP3{fD~RQzj*&+1 zdR$_LaAVO;W)$ra+#y9FSb7-NH=rgSx<@{9k=iqV_!DH)SH_=R?;<$(yZ-xhL#U({ zyBf>k7iMOl!(sUCt`3EDT9GiemFWU&%etKz&SyfLRuxw^>+}C} z7Qn*0#YPic%b5zBdgX;~yaqd-iPVf2+T@+IoP`}Q#J*N~a6KlA+`Vkch?S~Xst4uO z+I?#Lky_^IDiYV*ZLZevmhEjxL)jrM^!*R3vqqX#;+TEwKvzvr19yo&JoF%+{zhc8 zYRko}k<=Dc_efF6s5Pn&deP2$xbWTonhOi^MqQ8!?skE-Ek~XrPfAZF*h3pYHo$a z>Lb+26|DQtplzo^HT;MTA)bTT4TMcETo^aYB-xL5U?nO@Njw4cA9ePvroX3ie}+*Z zrU*3cg_fM(yk%AR!nbROY?w`C(+1Clr93twnzMGZo6nHY4tZ*ZY5xX^Ql5M1|T zv(()0xzQ)%bd|KNEF-R};&@jh!QA&kxT6Y3yN0ZHaV6tX>UOXx%~|7>rlR2EffCgc z6j8`f2cmN$&Y0RJv0RW1^iH>n=t(T>6tBFJQ_#v{x3;raFqXVKBy;uc{N`9=y*;6@ z$52*%Fs^GP0Z^-;3Ffi2iAh05Va~M_qrB760X(B5M4YwvYJb0#aWnL&-<0Py=4_y?Z;YCg zJCt^zEtsNcAI>1!`|V42kuEq8oASSh-`VLZe&0OWW}pHbF>`Mb^m%h)=^tMfIc<34Xvq<*{z%))(hOs znyZ}G=b!BqCa!>Oftx9M{Cx^wayrG`hJ7!?jfgUl=f<0HgU?868gd2Dm8exp&SZ)2 zdsL3?2SQi7vZC6LD{hrxD;yTrp6iPwT)k}pTWwpZn83#?3xj^+Hs}>Kk8QVKeK@3t zI;-@8t|+3n)0Nn=a-{?}d#2yNkE7fa){oLbJ85jOPHouPS_TCfhF+)3ijRC!-x*lW z_1&Q~Z<~S|?Sgk3me=S2iu3x#5yjh2V=rs8q~f1wT~(2LJ4pSKT*XDHKsi~ey+U{9 z35dGji7RP@<)-7MHQU)$F1uq6bHi>NM~R^IJX#kT9%+wuKC*>A$jh(CgAm;iT>nUn z?p3Y9NBnKXBo>i@9E!4&&J~k`jjWi!2wLF;sN0#Z^l1dLFX6Y!z=R$I^=ZQMKhea^ zYq*;S!%K#9pZoKcr?6(_-MQ+GY6MW@HVEGstRkFb@399}X@gjHQ`KvDUh6_we$1<3 zEsJa@1{{D3kH0Q4@BZ#L{F(|+gUe{|S#ORgVaO;6tA>b^)~VLG_E-mioSW6i&h0h#-Ac5A=zBthml>uctE0^` zCw29`VqP`}De_%Lr+x_;W4>@C{^ zyC-q7DPeQABFDXwvu7D#O~@p*=VFvc^U^Ojv}}BW@v@0@P*=Yxf#6(oFT3`5?zxK- zD@Dhr^+jW5*Ds#fa!jlrn(eR8p!)Wv1L$u?UR($9zAX)tog0%GFLNB&E|6ZjrGXh` zjcn)W^A`=zI|2y3moLg8457-x$OPqAQMo^$4luTJ1cFpe8KhYwH@3hTo)URT1b%r; zO>|uQ8;Qr;PHhwi365WVS!i@W6X{d}G^Y>{fI+A?bR#Q-0IuTWo*OsrW&PRJUVQ5f zVTA2C56EJ;t-0u}umsvE?_`~E+Gt_aXi}gg(r5JqC|s?WUg)0I`*C(M+}be!It_J% zhH6sV=T+!9T;V(w#g$CD>Nx~U@2x`P4XP|4>2j5xo^^qn9+$x&f$-V)2F>#;{V*$q zufvKWD{Ab%{Z;L=YvUl`NlS2EM8hR`8Pqkct^k-2|Ng~mLHGHkCUP73uz9UTy7<(r z_9zc+%*FHD2NNeX;aNr9a`f{m8gE)qA9ujHk**vATPGXHTHbzt`4%5pY%McU{F(&? zE0&cwfG3!hl(d5!OvH2vZ^V77O?85!_%+wGUZ**ciRjo)KHr=^{=PZ;O6j%25c`WG zV)IGpZOEJW`KpHYE%E67VLOG_4>3~dKgW`=g$XjzA{>OBliX{L(5@#@V~57X2W(31 zSQ~YbI#-LM?@bap2kEzKRA>^-hf4RY-;XV!o0)@LRbnXSVC!|rN@dCEv!~C5hD5D} zYB1Yk9NuSyx#UGa8gUzz0ZwC_tIejkHA73e*L)kB!WL-S-66G=>KFT24>TUKzaNyF ze{VSv?G%lgK`UOZ=BGh|Utd-5<~BwW4?aW`_sQ*{<4dAOCQjnqiZ9PEw%S`h8Yo-C zgJg)N^eBpC)3vKrrmS~&hF1Z6%1Sx?LLR;6?AzBI8Coc_Zve5X^&SlkQ4NjbsNU*M z`I6EB3}_M1|7sD~RbBmE*skFf6eGgGsz+1F)wclQ9Yw>g087w#d2`>3Aw%Q-^Ji#( znMz?n=}v9*XhHRQBN=kE*j`!-KN}*zt?ISWEMRbkQ1n~f*Wih>M(jhNReV*>x+~#y z`1ifeXS&AM-npk20*^Yv4TurtEUkK_A*PG4c#-rIM#yCuaM19^_Ae1i;`I zd0{l%PDIJlL5gizJ9nNK_ViH<5J6D)CwHsn#L^2pgqNo4-6HYZGHi#cF(-+L_ggh; zLLm)k&*K7$^l|xq?ar`;9>A*GgSg!fzc)%MCpw5*EKll3&~YnYdnsSU45_0hP_9?B zTQQYwe71*tCTqo0M7N^jw%wN=ytI4(LRqwd2B@4aaK=pI?bL@XK44wfwyNG<)OtK) zylM`dg(J43mg4RAv$LkSZ>N)d19TiGUqGjb85QBF#l5mS^6Ji23-4kJ;`1KVr-G== zZdd{s)WRHnGrFl(-kDcW+iw7tJxn~=OI&2oEpC##02u-DJ zyI48I@^h5$I6w?)cdOToJ$#{arC4h~fuOx8*Rh$N@E~Y`6>2aukwJ}aoBL+?g@iwB1sqHhNS;yT%~>EwsNw^1dJXTe z!*Gk;!mTHE0gQ)Xp#W9nxt<=2{cX{HdDHkSy{y>6X7+GB9#3CJL;S5HW!RhHKzZ)T z43}Q$OkX3~XJ(^EB4uBLa}P5dh`&K_3Tq)NEt?Pl^EC|U%HV|5^Yp~MHdAw+6>|#) znJ||QHTTzO)}rn1v=SCi)j`_1akF`MV~R3W|9!8e~Bt_SSqF52XtB`?|@ zl1CDVZG9%na~-H0-$vWe#k-ZX6yaVB?YBG!GYIX-T-K^VF&F|08{EHbvAY# zHgdXSdqA%?$FO*ffA38HN?$8#^aeL&wb6x}_Jalri*s`q_Z*LFPj*j;oKHS7im!VT zx>7|DTPkGur@sJ~@y@zz||86+jBD2N^2DcIa#yb-4Sn2)!J|i zJBiuqHzJ9&dFR)IthTl)aW$k<%E{NH{}mwpp2Fk`$0!u_LNHN@^2yp^X#Q(W~} zv*eRg0FMtOCYz3f2*tkFc6+&7E~DCJa^;C+o!9={AUDCeGy6h$C&#M zYv7LwmeD+ec}&mH5B#BtA_mvCJb$_RzZrxvbe30VGZmd{uBnp=%!$Y+tVzFisw(>2 z19GEcV%6uRJ?i{1?+uvHO}m}Cr!Kv}en~&``g+V6&kzQxM*eP-QO8s7$Q1_e$bTv_c*zQwv+CoSsT?CA_3jI*q5CGyaQdRIT z!HZ9|g`T=sFX=D!Me}iBU}{~xWM~1E6FYq{Q5=Aoc#ld_|K7z{IRWzv0*ypY8p))Wm>&n7A9LfsICaYJ0{$TXmDa%99+FKLUgg*=_s-w=ZAE^GuCO{tgBzwv zHGwoZ&DmQe)X(j2ocic3KQyDmB=M;DtYR#Gw#F<<5a-fkXp-wU6jCzrpbc<+m^u5T z;QlK!NIil(Z{}5HsZV_(4(|m(ghABo&7z{vp=)CZE45@&DOM|XEnuApb%S^}RrGAh zWilPqH#Db0^tzvT#FbO?;~tX9=$E?oP~X#k4^UU(g1)Wi$h1kvFmIMAVV4e{C}+c%ebIB z(2rXQlH$vI&8FIW-*?Nu8{Coetv23n&Npm&H{7En5ih99FQs-9<`d^>+It)4zGQtU z^5(CF`p3KKM|S|EDH60JAJQR>^FLifVeJZmbhZyK%Oxtn_b87YMkn7uLKpdo z`nuNRHvsJ367WW|jkY%((NR;JS%TASG@!)HMHvQq$Lx<}VuJ$1nFxO|Sr__uQS`h> zUFQW?`q_~D-MTaQr*HMUR9rL|@TacR0R49T_CZ_oCWh)uAl8mpMAVQ~tro?lTt@m1 zwR$)%-`KO0Bixd0jdDg5u`h?8;lGUbxxr1 zED|MBhV*O6Y7^@$?a2yAp0;PsYt&^opC3>Kmclkb5|=w-)kNe?@yEf9f>OZVGHZND zWA4Neq%z3Ila_qy!rrhvO9F2YI~S8P6gUeNBC*Q-%FML_Fi0~ z7=?tL1{D8B<&|^*1y^=+RSSHlPU;V_t>Xgkh)-PAaQd$Pv5jogfF*V4h}HeAHuIl! zBfKz=Gb3CdMA-iPsQRv+7TRQaIpc*mQ#DM|cubqBY-g*!_eLQVugy=_>?tm_4c+>$Z*4cK zw#Jv6^_LDe6$nptm;Ty}!oVZR_F}lJ@w+|x$37L-1J`WM$^7p=zgGBnjrD(be)KyC z7GLN5Fae$%*9SLJ*gIW6@e~tL_3NSNc;$FWOhyHU`xKc1yKm2u{IyR!R`M0lEt9Qo%zj^^Ad{At{ z_Yj45UCV~=j{G69ZU7ythv4+4Kx!v1YXvZL#aaZ*WtU)X=_3C{r# zH^gE8%LHN$Ujila=o(*e3<06U;Wyx@YyuB9=%$#n7e4<(kfmJ(DA_0>e8?CX@3!;o zLgWO%#|QMDxsAtI)$H+)+4cY$Wo^k^xyBH1_mK5~#ke(Onh0=2NFwm=fO9>wY78hK z>Dihkv89p1J?|7ML46h_aMl3Z#=;XYj?B|j8-pk%0ECp9326T^EdFEy99DfL+T(Ht zENLd=a+Em$xCcxyZZXX#AFz1d0~f2p9@$@N!&oB8N1}$g3TR3I{zbdTLj68umN&X9 zb`HqYbc(CspYon^{s9^IT#c3t5eH}S=a@t+B7nIB+|^O@Hi@GJrKUiN*kIUORcZ34 zAHTll)Ex(L0kY<3#cBco<CMv<4qKGw25X zq1l(2p(G`dF@Tx11U%He4HTz_3seVilwNnQgx4!ug=ChjH8nu=#m4(BwlIZ0GbG+ceJPhZI=XmGD{dd|2}ieD>O6F zyB~e4nQ+zPZM_q|n^1$W+fciLknc0a1G*!lc_lS}U?J*UI?Hj!SWm@mce}Q zmy=!GuZ7OUeKi8C{Sfs93uaM3tAN-dKy8Ai!lLC;xF%&X&DeZX)!9It3GTrqG-#yWg|*PuB;ezPwY6k1;9^XN*G5 z4+HF6@BTo)woNVV?!6i=Pq)TC6yZ>j7LtF`OiGSggKkA;HQ{k5y^<09B%SdrPxqd zH&gq@8H?Obw{ly8(W|Qkh`tsR|5kU-sT`yO-M)EIwl#uC=NN2iu{Q+(O|}55n{KX7 zCcCna<{w?s(W(S^ccIavVK##_buAYl1ceZ-5%bJ!z^p}f!@ru!uC&@eM`UJ~XkE*6 z-CsyM0Yuwe0J){s(<8N2>K$9)kQOkTA#)!ah$z}xgHshr%>>-Cm!s^(0mas;poPIH z+Hn%3D?$Q5R)CUWVCl0W*(doRw30GzIHNHxcj7y7egn%rUJ-_(6uTpQIkR1@YiB!5uGhw5L3W3dItb-tGpy6J;|@Di}Ov6sd`kQvF0CM6%vlTE8XAyF*eGIrVp*(CXE`UU-~Rhu0d?n68q*n)h0M2nHAX3Y^4UMr_VOR662f~(-RM;8Et}!G9=h7jOIC; z)#jTNPvK!h`{0%jp@n}x1mfrJ6Jr2KoFTEfpcrc~W)^14ELdK1Z#yAf-{1%j%>?3H zxw6cR=zy%Uhh&Nn*)SZ{aYS|TsPW1?C>j8TW|^sMVXq?XsL+K2-JZV0j3z6sn1iA5 zk)OzNK?*k}zxoeOal&PQrlEJq_!SxQ$DrWJ9l#buNzNdT ziU!UDHi4~qUs4zn8^E6bktjV{yELojAu>cARv58Tq7@6Urd=Q)Afe6|l3`6*X5@>e z-O!g61MQEdwt(GpmD55aGD*$Osw9I4XxsYhHQ!!ZsqJV|aZ7u9NXz(%uM{nwDbSea>KD&|MPrLj%{g^oZlAc4-U!w4%B^_|- zgrUNR4V1@R1|@LD53Cnij`s&@b0}mU$-HJOkdd>NE&>$qqMhusaKfbGdwHS7C0$Zv zGfy}HLPe}3svH>mZhSq^!ANARBYve29=l#ldVQCR#vVQ5-$<(zSZghqCzrbAeXb@=i5z7)doQ)_>~VIjbuTH0gg zugkeU=r|RCY7GVn(Fss6;PUx4_6p#vfrOJ488$f7wvx^#d;`4PIv{pC`QD%(w+Flm zhWsr4jovaVLs{1BF+e(&6{!TIFtowBhUR#MY9QGvK+Q#&xecH-L*%r#xpD3LS_7En zCjvoMHKiG z>$cuFM3*~wPgvhA%eo)k`k@@;^5jtL1R1gdRrmkqx&2(8YIc3!=J5o*Tv`tw$yI|J(NQNmD9#wVd$KjuLfUG zQG`o`F;ZO%a%rhcn(}CTw+6Z2mtRl~lu&<8f0XeJVurSg5GPi*K|p~iP!5h+heTFj9QWSXxwUh5q`LWszoMHiqHLe$-~f$<`ifIQa`T_P0%J_TB2KVF29 zsJ-^Fd-*N&pHB2&o;j#;IuEmIOIF@0lU!A@Z@bk5FYZ&f$gQ?70B0H^EwYQ%4Wk&9 zOH-cE^Q+$j-oXelgFI6UVo32Db7%3{JQbnN^e}K9^$Z>i^(aWwIlq^)N$w)$_d4tX zb)!C+A(=#!RD}+E=R-N}3ZtrsmG0*Gu4IYb^3H6uHO-5d3q2|UsvP-Hi|-q#rQxD7 zey{+G(E%=I#`&HTbZQA*J;GcLXl{3hTa&Z|vI33&K%sG#0z>JPPxS;td8DXWm~S6U zPn}hg0%z2<18}2BapPQq3kQAq(nEENEKnjTQBr$)D;;YtapK9+6IBRvIemYMoE*(zOn#`sJ_5#xY2``f$llIdLD%93exxp|8^ zNJY?OKRM%bnR4t$^>M`VGc5}?I9%=z=kNGSg0qD#3oiJKx?!Mepuk!Mk7fah<)}^s z=6@BY?h^vfW?AgdKL@;Gl$@}R9c9_I%9xQi^tl`cB@EkzY}4ps4u-3L+FY3DH9+p- ziCD3eek4x#C9Qt0s;NMcjC0fc`akR%q%b3eM}X+&L0_(WL=8PIC**cXUkjUm`SJb| zqu{_C1vo-9bc7$!#!80OqEDB0oVFAI2+Cs&$Jaf;ab2}ZL?If`@e)wsO0oRxAtj@v z2!Q=AZ(5q(?RsM$7&8Cm_+TxON|XB=5toJOz`!HxgOyS}Smpau(>dsPvKTxd>sr6K z=ZDM9hz8O%a(+RD&VBWmiF!mo<;3OFT?UUj{R?g4Q2}OuV37X!m1Hi+!)@>5kx~9R zgse*fQ6@f1TH_y{gPs;=9e}nBm=1pbmut>L(Wg2I@%>w%z68RtYVn3f>C=@ZT5EE> z2iJJDrr$ahVfzbW&_x@MZ4!R*f4S!N)hL?>u$~_osv^24ABuO2)RgE?|G~T{z{KMP z7@hC^`Mo}W0A8oCqqS|o8PD;nQDq$)=WfJRm7Yx+Iz3zF#X;qT_$`@&C&;x6$~8tqT6XpyOKw03GM-%*j8s*z>Pp z;8g%wO$sv){=;Sg>yv~8rN-_c=_gD>CarGwaaDr`-o+CUgXiuGJhe ziCsA{A3WVz0^_Jar&KM{6q`h#UjQ_+4=A`2;E00^FErnZKxCNwg|w19|QzmG_Hat4)1Wt#&z;ii>qpDH+nU$Erx?h0SRHOt>HP2ie#vp|RO!!Y*WBB@| zGmj^hx5adUj|MoLoSnc<#aiT)>vFm`05VVEoi6~tss%2#`imjQ?~^Zc0nBt=*U_N& zy)tlfn?aT8HIR%PzA#h+7?ioS$NNP>AR`nAYfjmtbo-U!fp=U79Kbuba^H;2 zbVEsLl{xIws+Ut}`Q!l~XrRBPixPyuMbVxqndth94a$ETZ3`*)WFM0B-9!5gp)%vIhGKf+`p8V1KuGyZ-d- zepeA6fiYXv{b0*EBa|B2H(YRSHB@QT9l0aiGuNUGOY2-=V3gmZn!cy?$d^9D%|cqE zZnOVe0+(_tcY;vB6nQi#DCQ*Au@fXKT=h;D^ZOGvMFTtqPo@3|5T4TpiEEu^6D?>?RO*nY%X+y{5oL87dw~!a3)E#pmQJ+IRqYDc%HfziabJC=mCc3 zkUJo#6$6N}uO!$~4><9@=Z%So*)!cTEh}wnt}q1uDC-(2HoarQ?qBQ@Z{5lait5II zq6nr&Zpi6_$XRK(Td~!qr09k|Xc41xt&4cw>*?YAn~%b^#Kd|l~w zf|0)G>0?3zuG|EQ5?4Ax*ogzdFqLs{AXGl`?wpX5B5p^-ycUKcI_}^e%&68H6$Fqx z#FqV5xvf5$vk+JupXH-Te={Nyg<2fD66wyMulm-)ETt;;CPQSTq`pt2c^?P52-)X> z7AuP7T|Jr#445L7bAU<$qDkeq7WBj%FIvPh@Pp|T@`_YdYFep@5c?;~V-6l_(BZlD z>%=T9&o4Z}P{yRa@XGJ3GVZ&9<;>*wQ(4LWW>WX@X#EV;$**ok-TEr>js)+Zofi-35(NcQWse&6IYQgTrf-$mguI4}iSB zvkmY{k$buXSQg$!Ta33;`~vwe>rdJSh#Q$PH%_q%d=0yBYzgY9!gI0#gh|!3B^7iN z-+eT&R@Q@yyPL0KrkDF&hY)^uF;Id0sTMjFwr?I3IBm{$1-uXh%I&Qih^$o#t|8>8 zX?1AVS-5qk>Peq*txvTazpR6<+diGxDn0v*OWCrSTbafeD(|aD>T4Hs)6}dph*G}d z1wPi52a;z``OkBMDDOGKJ^MOrGW89u5Um@@tw;AT@$LyStGpcn=P?KY)&=Pmo)r-N z^D-#J*fx@w2J>8Q!6W5m!oP}7g!Cd5kD|Cf#hOku!ruLk!M8OCs%uxyI;Vgi z+e_CMr0?cCL&Dp81@G8VpzGLc+zz;VgQ@Snck;7T!&w3Dx+qqkRm$VSS@(hiCaKaA z#E8nL`(KdR5JNd5nQ7B$-H_nRx}bUynMX#qHv{bAwR}o$#<(XKO2~HRLe(+wmnZ{U zcaXFhObnA%iH)*;2k6alU+-|dDPVUoh(MGiqw6jj@8-v-H7oleH~9Herc zCZ|Dca0!D%!TiE;IO`Iw3ZbVC+MH7>_~g74)&MHUIr5?+mFJ*Sc=VBX6l z|J_qLtcfs!9BENL4b&T=!fsM(T_S4v8P1C)|HAD(ugU9WK)n`*na_2^PpyJOd}@I) ze_W_*k(Dn+-geBsLdYOefH-A6eyouvH&~SH<7=<+{8iqRQ8GhV2xv3Q)@lv}73=eX zF_gykGh>o?t00JEfA3caZ;a~Z3*@>(o)*Vy$04rs+QM+I4un9Pcd`WQU1g}=>Xg>P zpAO~fNt`!$U!+NoOLMSBEt%Dq4HJ+uIUXvzwfae9P!Sv~RhDu0CBf6+_rAR>&zTB} zxZ0!yui#jMZF)KCDeh&R@UPH4s%7_bFi|NT%y(So-@N>4-)7zxb*soJfaI1udDgmB z#O_0-L%kG-46;dg!BMUdYaWAog-9kI$*Le2a`3DkB zw^cd;@DZk)$?=hs%|maJLy=>E$H^=Z@kuEs+$a6WI2(cy z#G8HS$|NB~8&AVcBJL_UV_|fTu9f9avv_$hId2+Gz@U$7I6J**N1QBqR38U}T$`dvqd$pq?O={hWt=q1K3PGl zr4`h(q(6=8wE-w}>w@akKu+KS zwntYTViK9f-ZM7_<*|;f#${sBsvoHjXO14a9L~L@6*7>`*l$&I11Lfj@XecFs?33d z&Nef$;feJKZi!HdjHOquZ|9bBYu#I!)26M)A3Da>n`XwjPD1=7Q14!NVG{=01`u(S zJi`<^>9f40AHiclSof+7a08nn6Y{6N0!IUa2|eV^SSD$qx$o#&K+!yogC(S5+^1Gz z8nE)U0pjBf(5`NevUOR#*KoBuLZ4nV{^@TU05z%XXdn<_i+yUsyts!QR$csOh0@3X zHV3Xc9n9jseEm%?C_%;gDB%cQfim4$vQJXrzzP~)0?%u?sJA}88w@2hSy1`V>dUFh z*f$-&xF4)IfXu{4$XH__1w>z6M?bilf7shbr0qx)9J&V1CgS5UQtwGqN~2+^wIP4A zn$h~@O~vu%wjh@2a##zyV>=m^EAPWVt?Ji-yg6(hY+K^W|rk>AcaT)v7t95kTBA*@4H9+*{~j}IjW(I)q0wsFw+NH zQjyZn%=19PsgQ=V&3*(xyY46d4|{JNm1W-ki;jp&s0dP0l1fXrfI%Z2BHazrEg&G> zQj*f$4T>P$At{KoASj_o3Fo>e-ZSs)!Qc7gthLu(YyUB8%^Dqep8NjBb$#kOoVD}4 z`2rN{N$I4Qo`s&5bp@_5XThfQ%t>}K7vkgK2#vRlj5c7J7>G7s9Y~xoDdj*|Nur|u zP{TM)u4f()98}>W0Ne}qf{6cm-c+@Er3vUnllkeKMV1-n*IU~ zhM=G_`i{aAAJZQqgJ|z>$figi6A=Fv6^o~4S_3R!<^whtR5J|B=TGrYLe#D5XsiV@ zgw&lB)CdW-vun_j9)GauAPv4=)SZGVVo+dXw%8eY#CT>7?$obWGTXYN$jay_ai_OH z5UVSR7;pv1P0DDDVtiKMa=Y+$Z)<0npoQwH;S7i{S_e3f= ze3tSAuhlaVHC5z;D_mGXjSVxWpF;QFFj0Q)mubGTg|~kR&jK>8^e*1lyOSBoPWyfy zQBzi+O znS@3PbLLErY--Y_gL%h&PYDg@D=lP#lVT6SE%?6LlK3aIvWD4rj)d1c7?$RB)+OTE zbsTnjS7Uz?8QzkoDP_6fTLf8mmudpXsW9TNM&JL{-~th&2OGKn2IWDNMh zD;uK)sxtmWP0PLX2NjAmx8@q4yWW?a>b}X}545fFOv8NnjG2WsR1dxe7Pln_8kV2T z<2-QyG#PV)*t(Gwrvr>sC%Sid?=F#y27w>{nq`y@TWM}i2xh^ddM}m{^w(iaoe2`J zWsf7*z9fbX9AcfnN>4h8`MhB#RqJS+U}2|VSxnGmFm3LwKN7jKnyt*W{PTk1QYo3z zzt+02s!e)PR7yuZ9B2tCOXvEGP0^wP;oaZTT7K^Foj@#IDT__Yp+tAQkxYr#}A^$v%hwNHq?Yqf@&9<41)To?{`7b;3?6KpR)#&z4a+OB_z zr*#^Kvw!jVq*Uh(StvQE+Y7$(5NnS#Tb!6!0|GMi5HueZ=FevlFCC}Ka}Xj8PzgBs=52# zloS{t&X#x!5hD?%OVl*y+H=H%N_8B#^Oi(dDB zD&UHPi{~8luIZRq;P~6>h>Z#8m@+k5sYxg>Xqp(?qx*0m1GTMpl7;0}a)B>a3l3EX z#Yn63&Z|u|;O_XGN!1BIe5$>4E5A}P|E)=D@zHvj*?6($ zk$@4@9m=#;BJBk`hZmwfa<^{DE}MTX-l(`>$z3bFp7*}N>1$HLx4PE@wLdzRNDAMM z+gu3wJR|6|$rm!haVyWt;YGISf{WRB)>fg&v9w0AdvyEbRtM9I1#&Ya8p&%0M7y=0 zhf9zBzJ6K5-8IdvD3-d{MPR3`nnRd%@O7d7U6ad}lf6SwP5wzrOeLQcw%XHKS&J&w z1>qyN+o?niS(Md8Zp-%M6uE43i99xxhaZ>h`=8KAWk5Tllr!q=^t51igZR79mxT!S zduug(j_dIrizt@cB3!GPXSe1+5)4*E+{Ve zIi6PK;?iovuiVybNH>yE8wHWbPxxo435T(GCDeR9MAiQ%b#T2$odcy-JrIfUoa6{I z>3BfvusVFMeT^$2v$aSuM`lt=a^lNW)n~xtlyq{$ZX|h->LgolU}dY=pL@bmMby#% zuU{9bHkN2~-l%AOR*sweQaK{yP**6Sb+6GLC`rX;(TLcJGKx0cd(SSK(#BZX34OP7 z@?>e_C)jJo6$-e0YUSQ7^Tw-aJ+>EpK-=48*3jm-e_)6>!AmG!w3=K-?39~47t8zH z+QN+w_fGlqKOdmCBsPmq!Sk4Jyr}jn8)q0M%y70F;kmF}M|E|w z)b^Jzaw|CdBJQ->lZF7={Kd@6xxfCnT~LU{G}a3y{`!e`ox3<8DYNGF>jxgg2oJog z;By7tXY=Y$&c`(+sgbms8gb{BO6Hb}WiAO{vY&D=C3m(i`0G`3P{49|F{pQ# z`|J0$!>n`eeIqg7(?4FmGTOJg;(^+(g?yeliob1!ZZ)0$F1v1*gMQ7wb`ou<7oJ6y zjXmz){@VY#wY_gpLIZSF?USu*FIdqK%d^nQk=@R6UmVoWNB}96aOO!^Eaqc{hhLjK zU;5!sK?0!}XBSc*gKsXG&tVOUd4VaoOtv6MtkP`M?2XqMOh%pUnF!33CcaC xd4 z`r7~83WIyFvx#P3aBN%)C1ib?M0YLX?glh?Ro%tIRo@2@B@WP1BZX=efX0{aN%SYM zuON{@fC+CyU6bvF{AD-jknus5as2bJqZdB?4*Y49W2&GBj}}(=0@IaTTZiZJrZhM;2FEzITK$p`VEC*kuWCZGy=8(dcZ-R27ff`xh0?xgz*#)S;vs z>ycQ3dpbaBWjft<{l=mrCHH->Q#;wUHm~ON3yBIYB(_I|-Kq5a% z>)CEki`3E~%qu_3{Z~K-{^ooNNNII4QcJF$dg{bC!LhJ2K0ahlbLE__l21|Xd{9+= zDJ6diNQXhwaXB2xLr_^f8#{C_l$bjV@S+Euu_=uNr6ERNluo^3mi@iXyg!n&T+`p) zW^r^{Og6Bbs+7P7j_y(k$--xdKuDUAiwwQuxxYkk*>)7*@Ou}uo_>c0s?vNsjLTDk zbC`UT(>WEvy&90*U69?d8~3||vi|kSvD*|v%~;fT&RqBSCVzRmO0`4_z(4zAIY2kR z0Q1^CldzwiXw_}9?}5~;P~8UF;Dh|?0oP<^eKK*v0J!`+q(9ZYQ;=c}#bp91If@4{spBy6k(*besvy4k@yn%+w z0WDahjgA_`Kk2FgH#xkde9TzqdyKYn#8{!2RT`DGh&JO)M^4Uxl)W6=0)o!FTGgjM z?e4p9%reI2CEbwNWpUqoJdBMJ-*c-;;8oW83-jW@9|f0)I0DFTk_B1;G@IoC>l}%6 zh9%B!DVp}c=eGq&z9=r!2{?w#i!tL=k7^J?J8evT2U%@E^S&Fv+NoghG>}K6QZw^= zh>RLUsiuhVZYDq)5g?Xd(BZQM0*Vx}lA)oH6*Ga<6(%G!Qp^he8gQv}9--&&kphz$ z*^#M8B^D+C;V}tXdLUHSCNXe zvP&SBBi)LQGzTRx>Q;W=3vTO_%lIa8+HGp0!SQS@ zMD9GC789l7<7mABe)AteV1Ad4yQZhaQ=z_{9D2bFz97tzv)NWaRg%UixM$&pz%OWo z_K}uHY9`1u(}GQX^9RV3?3mbqR=+``$9kEVP?S=+xAdEY#-x4dg1`_!MeTZRJksP= z9*|kwwta>;$LltbtGQ2tODs|eSBb7t#aPU~zAc$&_l@p|d*(AVs=>y(xwMB2M&1t} zP!0z)t3YxtkhF0a)k*yA?(>r)dA>>0M3>>PjOt=TC!t>h3^Lwne) zg5RTL6ixkskkbq|`i4ElMp|?1Giuo_La?UXxPwM>-EsTEBBy;y3)`D)_0@g2f z5i6Pp01}*Ju_?T(G8zg;+RFo-zLClNPGJRLdy%CSN!;Kbf2bci?=N}#-ZS#h`nQ0w zB$JY(qbV}S*GCc{IU&<8`lw}jmvAM&J-olR51cN+!F_?Qii!YiH5)FGc+&_<;t)HK zibM{Jcw$VMX`}%d!(w-Ubna&_Tn@pc6YR;Zwu!h+ks-gR{glHX|IjdYw>dZ|>KEvc z#v6?4^V&}_l71`62}-}t#ZIj|kM5GC;wp3Z*8G*%+uR?z+k1NF8P6!Jsnnc5HLdxTpHn7xee2Y&H%yNLf(J>ofqfa;|5>b2x=TPFzRM{T-yL@Bx>y0Wuswz(1 zYi}_KOF0WY+dgW>DDdLydw~iB^wXTB{>DSK9#@@h>`YK1P%sOAmKC zo7Ee(#R7U6Pn=08ep|S@;T-(1+PIN3H01ZIk0<;29sSCTv(oAvTrX-*xM3;OB~u=^pyx%E>`gAO zr>pg%u<}URwPK5v#!(6Jjs=H`?q}W>3T|?G*0njW^ z5v?2EY=PRAmsB#@KVZRnU#5hfYfP-lKM^X(50O( zjO_M{P+=hc2a;H&_8g9HJEPxRmHX%S`A?BcL%^cLz?*mWnEp+Bcw=p_MW9z`Iz!kM z&PdTxzXg!fwD1>LHExOOQE-t_6POCKElX^3!*M5Bez3ctZiq)<`k+9i=+nJNJ5at5 zI9H5AZHSDY+2UCba)Dw*tXe7mKp;CZy`kCXfsH@V|a&1FL&z>Y2Bejtw z2dm!j3ne=SjdFi=6aTDg)5uVd7iRlD&pr8&9*A~7w*4(mrCeeAcqrFYKndLJ{0?bvYk7za^-+LFiWTjsbVy(uWL?AO@ccGn}R#Ij2`KGi{* z`Y?H=RE)YQW_MC3egy&kWY~Bcw$qktU9E3!)I|8Md*1}5j8rtju+GhYK2K~B(N~;) zw#1Q`&)$T@6AJ2KWe0QI3SnNVt1soNL_f9JxFamhr#IU}I`0Qe2A9P;jZ$o?H)n;j zWz80U+}iM23IB?HT94%2PNnl^%y}$**yygaC?Afji0Kd%SK|Ei?}sz`^<5N+jJ1f6 zGu}lD>@J=_0a2Z&5_W~cEAMF#OSbVqiVY?o=<1&JA_3uz--6d!+{}yTErBRwHXF{A zSk!Gq53ywqV?korE0RAChPgy-34pPw)va+S#FCKeQO^xe2g15d8UfB*Pl7WeEa@R) zo$`P^p{RN)=zMrPJ2IgGbY`rUg2>|N&9__W_Y-4YwqV6cx{r* zFHq^at?y*J!9%L*FauM>00o4X|0F)UzjFdfvf593^w9l-k8RSuf~PL&OOzk+BQLdl zpth=S!C;VU!}C_Gv6@NG3{3LTq?{DFf)Z8;{fAmGt|o5o)$U!{4SQWa1%0ZSr|Z5C z&Vk;~m3e<<8cW2%;#H^W{F^5~4_IX}zK+3sj7|%ygu^^rcia11?bPcp9|ti}+8xM! z5xppS8TBsdjb3GhZ33>nv#OWt21!ktQ83Tr!L&PX{jrebIJ+;2Csms>(bBsu9V<@= zlQ2qo1mVzk+?Wzc6y5@QTSrLrb#~5MrgPDgFZ0b+xeN601B&>Kd*f}uTFU19NN~#I z;G2Zo-R)zc>YGdC@15IaQYuF*Omk$9rRyCxB#VeUuD>{(8-hls5mW-Pi_Ie*mjBu4 z|9o7}gQ53cDpy$ImUU!9V(9cfz&Hd6S1+Z>*d`(Bz;iR50=8NQF~vp2*oV^Dv^4LT zR&MKVo#$*HAe)8ydvcT||9LIfFelOP!+c*h&w@_P42r$JFllEysEpeudB|oI)?4rF zrXnmXH<(dAv4H9n&3gMLN%Vl=-FgR&o>}CA3Ssgvi~#lA;zu8>QSlp^4WsQ zT-#q$>TO2ck3f9p2vb>J%1VxdKF7LRS6X%JZf!i1*!&rEq5;VLt&}xdQ%l39TAWRm zGSeVWTIjg9@=#FpJ=BjC(bP*b5WQD^6|q}CyF`pHR^Dvz8mlQ3(FCqu>mtt>E~Cau zNsc<5n=ZeSD@F4_O&~s62{=a!Cco4{_Y(9-5>j%PCJtXQoML(HfWb|KmM z!qeN2Dc|VV3|I&xv?p+ytE>-pPS5u_t}5jb`4I$cTvsE#D@oMIRnlD;CKjODJ*vxu z206o28B1p>-hJQvi2nGfC))JKmnlKgvLQm$3HIG_ht(PW2hTDSP6Q4$#*L>JZC2C% zuCk)P2Y`%-WwrLq+vStl)Pm2eTL-w0HP;jys(;uzm@%jb{nU&*uql48sgYb?H#(sF z2iz~zbnUAG!I@(}*A`#&$Vnxq+qcHdW?viV-lNwm_~Ixv@Kec&i*x|LtnCh0ue9+> zS)|YfJpsYTZ4HwcA==!aMN$*Hnv%A zer!I+$C0cU8vPk@se<7OqXgyqV+$0u+Hhk$S~2gk3n5G$ru@eLez@k!L+>j}$8v*Z1c?uuNUf<&e^chJO9-0hjQW?=k-sD!ZKH`JLS<3Q7y4K}KI-!@ z&v={Ke^AsIYIsNz1*;Xba(doG+NLV=hrCIrRdhfOke3ScN)Pj%3OfJs^amdz@K}!| z57xgw!@scm@+(ki1RL5$o%#}g|ASBjyDuyB-TCKj{@>sjEhap=@`pEW=A8!H{a(ra z@A`1we%*SpG79jbb6LEBp(`%ExovaGLh#O}0(f&o%{*1cU zK2YO*`zXa&=6gncIJbBUx}UyyR(%*E^$04P{Lk!PJA|MH0X|?ICU)&X!3jw3;c_2L zw;2+JJld02UDj;lMQR zpI72SrpO+5Va1tGIyxeOg-mOB~jnSO_#O2oLr^P!&#WC`i@!_ISD`cgr;DC z2Gv_;h*Pkbs)Md{gbZ*mJpW#^z0El)(U6BG6eqI%`)9t!pR%@uC(-r>S`90dWGbUj z5iA+HpXWCb;Q`HU)x4A?z_*ZW7Qj?j5cHb3l+%1{9fC~|R8udQNhRNHDg55rzUT44 ziEe|86@l?Ml+6$Td<$SSva67lFrJ&*)6`9M%CGlAc_0SI|7u~C>?7ARk}n_x{OmDe zZ^$bTDpBJ92{1>aB9bp6;bfS2=Pu{vZL(YGL?71M=)$B^1(PR?K)tRGrMBYA8WFV} zz~~G3FtdlP266cSI`Xx`FD zecaTn3XWPjjWT9|7HB@v&!NHdM5I!RvV0ccJ6dJ9bbcD}fPSG9_o-x^TMiSP}YHR8rk#QLiA=IV$144LUaK{ZRSq@J7PjXYJ zcki>&&DKME`@wnPQdB;eR&GOaIRBBj9xB+|K5M`3JtM@z>nKUZlg}U9sivIaWUs_gRjIVeyV%HLa zs1!f#HBpln2RY~Uwd{vPGR;@fLMh|e+yYOp{Nu_iy-oiK)InSD*j{(>v*;TS((jlm$+%pNQAagGNGU2@lh*f5oeoG#CO%#p8^pjo zrv=7-D zIM)=TE6L(5Jm1J-*3>UE4M6{;T+WMSb*OlGTDc{s3)PA8#M1+w@ASfjcf;DG>d*!t z0pY^yZ0ct&gDE^V^9X>Kcz*v9@y*>&Lp~l|IM`omD+&#+u69MiGXZ*9j3zI}E{szDIvIBT1;`zx%6I|mW5q>y|> zk|VkS4IWJm@oiW8uOD71${w`(58n=YuO}<6kUN-`$!HnQtJ{M5A-nwn1T~m5Q@{Yi zDQusY_aBg{mffU&&lTnET>(Z)ysviPV-e+QR|aV@IrluX(##Avx-LFJ#u)?ZZ?iee zo^mK8ez-g1q@8`+eQy&1<1dk+FDk(FMTtW=RdQm(&byf!?FeCD@OEAW(wMSEB!z>k zruSO&=^!#x=jYUEin0fWe?xiaY@@1&R~?*r=L#ou%pBnem^rl-%T~>>sHtZ|EwnA6 ziZ^2NzIr3@W(rs(+M;0VE(16R|z{;L{Luw@87~x}kB=g5Uz&hZ#Ye}p@hV>ef$&9Z1x)s5B1{$m-Y@^=z zNZSYPUnk0_0D#A45chs`@M1)~q)>ko-ob6I1Zj^6{pgD^>T=DZPirAme+#ABe(c$4 z)sk=)0P4zX-3J=qsfA<5>K#7I3hw*TvzpcAI?lHna9MPTzY%>cQh!gl?R?aI!N%+Y zrIBt4*ZAKkv@iDax7&VcnOQ)i7=(HUyF{1j1>7NtRv?!HF@A{C%OUTy0N0XF5~Nzy zxVJ8gKF6~>X7f+#4M7dNuwrQ`z!p~#XlTJB!PoX7ezgS@7q+wFXGWUqIT5hk2`4`GS{!M5AS z2hMqdw{n_JcYV;`gUg?7Y$(Ts#XF#n)?4p_2xY8u*Cd4Kh$rB7UAl&&4%ZKt*hf}O z+6;$1w<+!R4T<7G;o_5PZTHasdjCUMJLcMaKEmrcYWRi`yu{{HXm$icfq}J>KUbpN zvc`9?&n=S#mMF{cH?4F=Qh~qih5>iKg^R{oFBM|YpnlPt8WO!=A=bqe)u_xY6e^`U z`M28pAHAy+^sctrcUERMQHU){&zvD1)r2F7p#+sGP}fu3QPLn-q?*z7`xGalWFj1f4;^ zB8f&R<4JI3;H#^X@sS+CL)8#4$0Nlw17SlViG^XL1XO;Kd0MOsE8pfWnQeV6t25lS zSQqG;8BrfAH^!_<5HNK77U(pioKT#F?TmH(x>bj0WN!PDm%vdtl>X`(z*L&p+2g@@ z*YukUo(<8DWYB=pXg97f?#hIW+q9Xt2e?Sq3K<-;vw72E>t6<3*yf4<5tW6JGep5` z#wos=XcG-CtD%s^W_@_9%t%78v?bZIiGmS{r=~#V;h@E zz*{}b!dQoUcy0Tj1}ztA`vMTkKP|Blt2Pz@TtBZWh!zQEIs!6`rwY)6w9Q~E z>3DmVc3GD>aE}WCAdJFIqcRPqiq$vNAj{@BpV=oc0LEh=O=5RpNDD57p7$MFnHgvv zR*M`gY`R1vSKq#lGrc8wqqo<(cJ4gRgDH1f)R3fdK;?P$50p>q2GgQYrzy*$R7Q+V zSM3&N~9@qCXvq4d6yuw)a zncj5~tyjA+4aCopPb!hSxGglEy5tH!#0H@S1E_CK)NHN*?b^g=)Gzb!FD>&FlV+6aN6=e+QWiD`6V>dS!P>| zXurg-NUYQ13(N;ll^fU1fNm!-Y*9N=X6Oym>+HBnn;*!e;`jcJ(l;*Kj#MU4cj&v5 z@mAU~e>+`F(1wT!Sfy5dI5K}LVGe%5bhipH_I9O!Z0Bplmg#X8ovo43Pl|aSyfgc7 z5%27>c3U0-X#;^JJ*uB~u>jODL#Z>2bq4i zbn+<0#6u>`}>XI53*Yh`fTMM3=aUWidpzeh;_?7%Oa{bpf^_Ilj zO!W!Um^|{7VAyUEo#$;zgE%Kf}e>fQ~}){zB6-rOJXKB;Ug7A-%K6f8V@D*Vau2##^l z1`e4*Q}k(=iG$3Un=>||62qv&__0-pg28eAOwTycTl!WK`qdLvr3Lxq9|^V0J61d= zqK_SxQ#Tqz*6VwNKODUt8%>yPaUh;`$V*iJsj;;7F5J%Y>wNt?l|sS82EOlknn$lR z;^vFinv8u4!vHEe!JPT%#GzxpN%I&5m9*2}D!8VO zL4~_EJIzlo=OkHz=VNf~RV?|GehBMVWww%#G{l`VZdxuT4G& z7d*B0YRdl8wBUdIyfk@$ojBK)Y;6Dar~C)_4}E~Z#YF)-r;8_%%h%)`0vWlwB>e*K zhF>JC4=2941;f1iUn}h2PeYMv~8qFKH}U(KUK5$`=^(rg@yv1l~fYG1FcG$)A(+i zW8b%mFHAnVivRw#a7|bME+z{3KzynW;`aytJOjQyZ}AJeQ%yF%zeMDOO}NE+UB^QTv3rw2H$%v3<5cOPplR!ewmzgS$tM0q)N1b4x zJaN!;{UH;*=vl*MKE~)#^>2li+$74eom}MS#FJa5B3ct@4QkC}XasF^|HH=9^7bS8 z`YtKqkLS00g$bHxX)k-?j0E$&=a1+-DzBipgPZ(3`M&{10O7 z{aZS3xK2V}`TY7lokOn`F*wI$bX%}{qD&u|jR|oXbpU<}q$ zs_GmL5C^3EOjI9vWj0o*wzUZo4rIWo>5q-~HLJFZw|V#RBga5c!2%)z1i$=xbaZcBjs z$Xq#C()4?d5x}5~Gr2MB!+;?;^OIKT0X(_5VZYi8*4H9LpHTuqn-+Wzr_OUDg{JBw z(=52<1w|3r*N6wG<0}B6Du0~yDlg;oMGE`j;>3%O=_d!BjW{7Qq%Jm(61gmKnz=#P z4X|S@e^%0J07sBzut~w*RpQ+H=e_} zA^bFk`DLT^a&LmWArG1^wva!#GelnM8`NJyeev86t_n3q#lV{qgKVH4r@Zhq30adt z!?hidB!Bg*Dlv!2m2(emRO>I1@;%>$2y_>S|AtREPk?%sgt_vo=i{j^9c7&W zneZJ@>&URdedq_n%+aGMbs7La26o|t{V43iQHT&(D;th1fhO#^qKvMuy>B~Y&m!&@ zd*96kfT%+==`VjkAZm#b{VOScVju77;j~cHZ=vv}Je4zmz%D-c-4$~xuDjp>HwMF< z3#560Sa{jK!9b8quVuJ48#Y?3fXwFEQQ2khPCUj~o6ODBatwWmy$ zIW$1uAk>s=xT@9w!qRv^FI8dJe_15)h>oo(X+eM=&%v}hsV_lY=H#fvPnB9FDJk%~ zA7vj)ir)|fwUa-&$T4iK4@e|p?L%U{zc~CUmA|k3hg64XG3M?!NN4skZxJ$JFg}@{$I`iwsI8ZEf#V z<%#etyS;`3?w_j+l%VCO&#Cz<3&1*kje#^r*niV2pByFMzEqfs{!Ks?r6$#=K2hYx z3`mLqZ6>FaA~BG=LRU*?#*f7_LnT#0xbL0462s=2P^FwB^>tZKD?uSOu zwd*eC^3_9h4W*|3G481RCsBW(c5Js7yH>+E;?H`b>!+`;0!R>6@mv)99INMRoE8z~ z>pnFWlWcsO-21K63om(6B+p7X$pCUCK28{PA)MDP$v;bn;S%1sZPJC_Y;*gMbYx&X zc*owvYC24Pg{MDBv0J3{2js1C4s)X9{Gm}lGRM50XCpG-tUog%*F{1465C@h3y^F- z-JXkG!)Pcjj`eMKh#+9$Z1@!_Y7%_D#A2J+4L&4*Pkj90r%$qFkGSN6^~CxWw(yT_ zbdLn)qiSPQ#U#gh`Z3WK#voXtbNTsV;Z`aZcG~Kr-wsqysYr4KkZaodL9i<&TMW!r z7>emb4}55HMc_Wpy+a6Epk^lzr)UpgBi*&2_JJg&Rv@7cph>;r@Vn@JUiDqyamCef zglSnnCh(smkL+fd`U9MKthnZz+f7Li2s_6)GZG5zz`97Y!JEvB;#}{kvZ%!MHxYhC z{=aNm31rsfS$UY8KJ5<3AU zJmX!qZ7!T!%c6mJGXZ|Nn2aC%3nLG0DL5ZfTSksG0;Y0bJ4E8y{l!>4iuyKxF;Oqt z0FLKt%&z6e%P0H^heGIXwCO<^iB|Qde7E)6FIKBBGN{{wfr%w-gqJuiyI+|rNG-CK zB`Z-mEZQ@oybVI0%FV#t`cj;kPfnp4%su6`H%im_Kguy zwhsgv()qU9*PL`_aqZjX-2(mSPtXviW(6LdBFw^`>q#t z^rPh#dp$DZql|zy#&Qz#^m9@v3euEahz^p#*mO*nJyqUCWE z$6x@_Eg}P)p`3otlTCl2O#9F~wCs8)7o~=;(x|iwAWT~hn*4A|M5`vr%a#xi@`W*K z15-BNdyO(73kEu-J8vM%XwcsWmEK0;peGdOFIS3>^K`0rcN&>^UExlD;ErR-7E@d5 z_TvjOFM|HgLKJN#SOnFpNmZ5iRCMoK`3b8zRunwa*Vf=a5tD$O>>RWgG=i6B_cqu;OQlZ7%kB)}Op>36Lrs2v=?L3%4%NolJp}vE0}uyEu3qs8^?#k*1M&hCgb09gE8~ z7lM>rl;?cbj}o2`hsd6p1j;^G=#FVUi=jT|7~%4kDs2f(#M{91C?ttIn78`Lt@A8^)kS%U8lzOJi?JAbI?^yeMtbL(7&5YZHrg!p%CBq}r`Gaz4SGwuQ z(kq!=xA7~HCa0!+E6@^)P6Yu^#iKfT6)xU80Um^gFH^nrtbv*8_{TA&d`MR`2SR$@ zR*%c5rL)n7iJ2mcSit#li1hh(EAIlBQ~~8mLeNG&aowg3eQK>dSKZ4s<}0?ewy#yV z84VxOY4Wd^zRle&D8E_Y30}EPve)3oY3+Q2V~oTbDen#lJ#-e?J{;c=KgPGysu{UbZAH@sC2(6N{(AYP95S zM0{w2x!>Z|z7YId?vq_cX9#0N5u@TLL5XrkMD82%b29Q}VsAt{?;EsWPJEJB6b-?R zlS{%~B{J073gWTNn|fWtIP*#*7{BOIRr=$gmgc0z5RGWDjGJ!M*~cPW@kiH;ld$El z;2Di(nX?kbXPZo5tA`%xQ9n}W! zoG}L)rhXKR6z`o>E9+!kZI5QV912Fu*7yHnRm7_UxIsVU zgLukz*C(f+H$mj0K-UP27wsBHSSL{Dal{`%|1aZnf0Mgu|I($q$;hAxP#Xl=`=k1e zcW%Wfay}vSO`~LFs5u;vI0htSO%$ z#(CZte+U`Ns<}wY4Z*aXh8do_Sso%`I^6#P=lotSdY4~)9Nby1UalY$&vea2P08R9 z*gb<^osl5Twa_2u|I&(G{kZo7+c|1KhRX7|`}~^mc0mUe3YR`)i=)=tAXs6SG7*n z&J4=cLjgquBS(*_EhG{jKQ$j|)+1t-iP^toE!O}Eea>@gA<@H}T8~8@;9WrXM{KS< ziNl;+mDI6PBRY~W-vR9wo>YzaLg}iZMvr#=7Pn?hn3zv1v*h5XQb_(a7*ObI|J+?J z1_DH6>`m7DvXL9cbw^H3hnmxVC5ytWb39RD9XuC1eeIrBg5)jg^CIK|KN2OMnzMdH zZWJ|p)2em$VJ#dU&sNQ8ZRUy_zI1Cv=6TMEvNGH?=#hz;v_Zxe>Aro|7HRQV`C|?C zK>Yc@xDRD~tIh(p4p=GmtSEwIUsn5EAk$jk@vUrjvuuFVICk4mDg-cT4?di)95iLi zi_|*o@VrGmwpx2S!zziFoBkqG;ri%`*%8X(wbUmEbUV~d6sX1u=uzvVi!CzYyv8d_3~gp07QJZ>CW{Bs(6aen;tCv7=)HHB9n5J@NyswW zTm3NT*K)<3Xc_e*hza6&Q6!eb|kNE5Q^*&q6bQ?_hD=LM_eHFcs_!Ij>g>JI7I2LcPDa6uRyc6*{pbknvBSh28Z%mo5)_Uct?9%NMD(7WekMykY zPxoXh`-7NQ+>?@Jmk%YfvFY>fkhuViv3cLiqI%LO=9`INj{ruiW5y0oKwKPD>(!*R z=;Bn@Anl48Q+|m7 zQB=(l&vpH>T>p;UxyWjqykJ4-!l+p&eTLoWOL0SiL|J((xRd?KTup9IcdEZjd;`6K zlJyVzAH-WtoCqz5Vze#ZvTW-~vQlyy-yoL>Kv|@mBfMm#=Yf2-3^{A#)c^`_s=+`k0-`YUCO>a`mcxZ_b+t#+#5}nx(65B11h*vKCBv6Gn9e=;YC#MuQ!d^T-)>yv|LDkD&5a#4ZcUY|DjvNwt zk0o9!Vo0`NJRl0gux6hCH^c#z1vkycMnnJ5&9Ag8FE)LlFRAnv2H_~ z*8Ak;r!6;=nfmY0AATq|>{);C3+y%?HlZI=Zn&b)Z-MWieU!-glRFCh;LCudB0XTS z1IJ8GANsAjMI1*)YD-FKBr-dlop%Ia12H57NeHYK3p(CSFr`AQo~!BopR5(nF)(3v zVo%fGg@$bSUX>Bw(PfhkfBXsh&n5tPs5FE7A+Wz4qJ0xxZ8rgzX@IfHBPJ(@O(#;9 z30RehgoxmQ9Rfd=c=-b`6^#bW)y*M3+Ca?Gm3lN z6X_zswV7VRbAhbyV9N9T`q=FI8E;U_XDm+V2TR_dEb%)S7Ymc3(psmJgkpB34lhxw z58aRmA}HF5F)#$_BL^}>I_W9((`)6ai24^BgCm_fERmV*$J;7QCMA*5pOE<#L61Gb z!p3QUNc;2wS#H4QY}>P;@DQIVUnMdQ{E1LF9G8OX6;!~rWsGuHM&37$FS8xg1~`yL z^d2CL1$@Yq>U#DRl77`P%?&g&#mZ#$y|+1y#DlJ_P9c%O`t|Jwwbu9v%?VgKHgVLk zMZEgUsQS~vJ*(1}wC4iLYJod5Lqn#-e0%MDHHAKDhy-D#VI`1qICRu>t7y4LFF_8=N8XdXU_oe<6hZ zQI%Pn^fRpMFw3q4B2;=nCV}3i;)pucm&h%&9Ix+;zw837_)zLk;tZ^eP!e9YDSuK& z!e%VXr}xV1-`(t2sQ&>-(C`=VpJm98SHU8aNAD<(f~i#5m0?sNy++~RbfYB>TK!0L zQHJ`Dm&V+eM6s7y?*iacL7H!VRdTBlM9-fgzE4jrH3Lgq0B!_jqp)~Q^}i`)e!nE; zTo7lv$6NqKSpH>B$_gV3}?W}?XKte_AKla^^P#yZ%rf(`M9wi=@scXvaZO{L@-JgmC z5;;!Do~=KsgFE)5DO%*`z!7Uf1EW#@Bpl%kS?2CEyFGN|3k+|wgX%%y&h+FrR@+GA z8RUzd(YuJI-;X+9>v?JgKraD4Psv9QQnP5!P+n8Gf>Mo0LGl@cB66juJSnfZLhRiK z_nil?6?`Fg=xv#hK@6U}$P8V{RX2H90LdgbO0TgQh<_d{%8**rt%ML`kuW@Od^e@# zOPgQM!XU3yB>ZmgZg!hd3^*YkBu`?pPJyib4j*A?Qd+RPFXOPGsI9!i{mB-ZU|N?echz+44tQGD(O6 zb(XVYEw?!L(}1_W$A?yGuj5!3Lj0|ZUPJJV$H$?Jh}lEKozFsZg{uR{yGByDELvrh z=_XPVD%a|aaP7Y4m-o>~%7oHR4KZLZgUy{YQOrJcx@=B{LW2!y)?49;U z5A}V%?Dkh$42paQ?b*^?Z*$=Hy#*#VNe_DCAoPP8!jTL(wyF3jNQ=jF7fB-xzB z9+DdSH>L9i_!ll?ld`?fvD-|nbrE>}C}am9>UxOVu*Bh6jzZ0U#PiR#y+mAao6UoH zErFM74mb%6YI69JgV!~+D2X0R-jQmGK`AvoL&d!=f4Q{r{&r8ct{*kDq+`w_Xx2h{fxc(A^z%lYp}aytFLrDlqJK9>r3Pban0$~ zw%`?~CYc8p32rUMdr5M_mx#BGoh3v-_P7fSncG@taTjUvkL=_N|_gq6< zmadSvk(U$}_Ux3O$URe*L6cJ-G0_V_qrK?zambG!HL#W|y<7$91(M6*ZTHaaNcwgd-zui4W#t~N z(Ef%<)*MvoEmm~NFWJ}z%+Bg4R-_~yTQhhKl(<mA)@tmz{q zFt=hWAeLn`iuY&n><%R?w!&u2OFb`On{he6;Cv6Rfx|`u=uzAiZ1y&9zmx8O#gJiVB|`|a}^A&amnkpGCY)Met`aL0dq{wf*Cc?i8D>v-nIh_+qq zX3`fv{su8!^4X;X(vvxF*GHbL-2IqSwG_4rim^CCDaPvH`v! zJ>fn~ss$7S*N(!YWg^nY-k%e7UBV7toM)OrY90slw});}(K>~%tpo2WV+gg4noPn{ z%MntwyZhj{V=kV>7zEhFTo#u*({@*)K83%i5NqXfkYZ^oeez&r3nVEKAYwKnYSIrK z-n~(@?NbNhh{oJgk(2c*jd+cmB0%?>)+Zb-DYD<7sJo=RHqsqvQ-%a1ta~IhKTfFJ zq;@x6ri>j>eNqRZjJeMX${`~0G70|%SYf^Ef|99^=7OdiR=RIBPL}es{u_Dn30pu$ z7!GXX@S&7%1WMq_3*Hmq7$zk+M0>n)OY>0Vnwsrc^N{-3eIWMPA*T+7e4%(!u z^~V%s*q0gvq1-cmCf<&|bAi!rn-f)FQo!FJQ77#Msd(ny+;!PV>=k6LWj||kvQBOd z0&+^MKB4K$J8tGQ79m){$L2Xm;`wM*L12gwUf3Xi-zs<m!$_tge^Ac!MJF!l$ z#xg>$pyyvUeczWIogIuOWFavmalFmj2prkEQ)ec)Icjo4*SY;m$!ggSBW2W%>%FZs z;&yTjO$H^eJ@r;ADl8mn-j(A22RVqJ`03;G>KQVhrQaW~_PK6+qQcs!XTjdVzPCEc zh-DMuQk0#yfEN+6NWZASeCy%vDi94RoeQlw4nDc|yu+xqSyf(lWGc<)E!t7nzZ&^n zIqmMeMv2e<*jR#2mAdp@wKS^EY5K3=-@rcNH&kjnSvq{$ZzQju_7Ye5T6FNGp~~E# zh}F$*Omp3nK#24T;I}gU*lTDwhZ_%V49o}B56+GG0XI3%BMq{cj_GFr#s}(xA)Yql z^mT6&xo)aAH{ph3l|3j~xmh5)e;jNhSa?YkC#|=Yn0F3CB;`q^etd*C+oi0@p~-z0 zBAq&)g&%sWA13(k?zs?k)|a|#KOSs9KCLksXzq)kGKiIS%fubB$D>w0Xdo$C7&pIT z#dC;%OLmTT?zh*=wPj9&1Zio9YnDJ^(Q%mVb|E*YI%jb|OIOh0WOIJMTt;|R?zE6f zn{#Mu^8999;TAWB>29<`X;O5dua)tCV2uCTmzBxvHuX;6>Y7dstpSMjx&S+$8MZs2lK=Q;jY6jS*Y;Hx?FLCjQ zXpl{+S+>WFw6b`9V(^T9t1EK7_f708_6gWu`Wj+hlrnbu#6IQc-piy7-(O@}sqG&a z@4sP(f4gFaMnGJqWk^r7e*F0#(<6}wZf~W5_k@f3#*1A`@Ge&8(^yRwExjOgNbI*NDBjJ>xvL~VLnKT`bXgE)p9aQpS1zm!{~x0o;) zot-t=f2Uvb6EMGhY5VZY3!FS9yN`|hFVHTsJVC=bFJG+l`)XhNMt7sx-|xF09b$2S zbDu!**Oo)+D+q{sXt{X#*NY`F95V8FGXmeMC#JU`J})5sKq%PYKB{jLz zX_cLd1v0n=LRA3+4;%7Kt7om7wyTPR$e0fd&ZmLKY`ePP`1eo#?~9j62bOq70O?E5 zONjA23&l~aJ1xA0=ZA#YG>Ld2_s>wQ8~{3wq!heAQ-jSY{jClFUmw{*%NEU@ytR|l zGjfoH>|m~R7I!zUMO9AYD>*59^Y#d0+5>>@_Ud?%Uv?}=UmC&?4Fw_=0Di_>>%5ey z)ymSiS;K$7%)gaFe&T*MN35R4^gpQ(+J2yXMW8_FP)vi18Q2@4e`I7g73v3|oCW+* zr;p67YpsS&p!o%$@wz*^cMHNihgKl(7(Cb_D#xCmzfC0=R zY6F&J0TUv9-vfATYRVjvtpH^HO_3XD@!(-&@y2zsKQ#qf5>kTTl*d;3XIM*zD>zxxCZz_hn*|pD#LJs?&@E)SD?iDUri?*@hQjf4~Q`pNU&bb9}IAU$g zNj@PRUIRhjoXi8NFP_M|m})k?L_G*I6i-Gi9NdJKUx2rHk$KVqnELfDH-6yk?=Cze z2P)A0tsfVv=B$$U;r1@@b*M2M)tRClX`HlmTDc*u`{xLaqsJE{Ak;|x?H7%8)gXH| zSSYxDh>#p{h*cXzh+{A#dAZxpJ`w@#vLXaBxqh1XGt>gCo$`R!9R>+rrcl?*2?yCD zJ@`nq%OZ63-E<<%s+Xxs^J7>R*X&bY-L5dpzvgpD}#q3ks# zwQT4BE!9To(eKAUe2YH=A>UO)V;sIFg4-Mzo7e-~PPHMMAqTX?)GwH}LbbX;K63qw z5eQC{F9Y^@g}CF@S{DcNzNONeucLtuj)57LoCS??4FrPe2EQ+ZTa_{gT6j@hXQ`$;n~~3rc01Ao)tI(5T|er{8{& zm=Isl+Ajv|YMc$3vq1nSfhoYNPx1Z1B+Q$}ra;txD3!|+^R%w)DD+0EDlO$St%>r7g8lI z=%uLr5`u))W?WBB&hzlX(^M)ZZbStN=voJa!(6e~2D#aq=7X83^cUQ2h5!f5>1KN- zk~HcP0_G0En>+*BcC-zs!Mi*1)w?!(_ym>b zHTQ4Jf3G+Z#vNjT>8vf@GwaPnnw1&IEQr+SO4`M0F5{y(k62lCIq`eI6YOjTni)i_ zBp~fKEJL5tX5hQffyydoBHnAWf=*TxmTlMFk78$uRW7cm+u_P!5u7!1p}tD}d$6sH z_7TiYk{ld850I2g(@SmYr@nU(-wP!Xh;$emyaG6`gPlZIYz#Lv^?69^#>N5>B&!_^ zGYjm(2$ZpWEpBW-&g`S5=nq06CAZIW0)k43+(81P9hO-r-kz{)52vZ?-BvK_FC}cG z9c|OcKD^xiNWJp<{0Fvz7260)b&m_nxP6Yvbs@itxxB;JB(wi5OkkNHQ(5-HULxz* z0<(~C2M#?gt#wFMsIaJvB;gKtqfj{Y(s^?1g13#lx7p(m#CqXSMiO>j4Z3TjWZv7u z9%1S=?O4Yrmd@&_i1^^UJhQ#TqACp^7H0WZN+Ge1i;N>w7?gJ?Uj_3AnuQI8RHlIq zn!}(NW+ZR1O2X5}zkwzz%wJsuYipRU zH^_vpp`Fp-RBTUDya9wu=S&1AfmFmdRJ9&}{y07Q*^G%DfHzXXrT-py)l?g(+6Puu z>0lK$C;RJZjTT@(!}pZrL0!`bLNYbHx<9_oDsZJ^aRcOWJMROmqlDnSe4e%6Uy62m z+SHd&3rO2|0UjsW&@qXdA-vIY=E2hKC91R0+?v|v*Hs78!uC#-pQocC^6HjA9T_>Q z@^{s`Dmi!d#Zfvsq}O$-WMmQcH_wC?HD?J7tEB>?AGO`(ti(~|rzFj`3X-SQ9hP`Q zEIi#6VMW6h^s+wxNuzRk8)I%;YB*NkM%=?de``ze<=a@-y)mX3lBsg2LlOF#+oBcD zq8?-sxGa1EZB`wqNAPwSmtY6`rUjx{fh1$j4?_$~t#)bHSQT;6<@%qvn40jJQO$e_ z0_`D9XYN6-7982uIxtPWo868KeWumy)(NJGQ-O)BFDT*gy~yy832fL(ySZoybHY$> z?o7Yff+{04`5rveG930h*eCO`V<-VUCmby2ynE|ntL}3c$iLd^YL-@3oog@f_)d`b z_HZSNnj)c>j1r^!9N8TcYleo1>N}iB+GOrEHZM<`!5=F)hG{@CcDYHE%`$8h4&sF! zm}=6CnihUP+LqCkwRld2?FIBO{?_et;CdpgVUpxtayaRGc(?F(rp>}uha75L1Kn8a z_B0iRTF8y0W{?jZi;7FzG_LIcX_v^4x=YWOI3!zYK+(9I>T%H!(KxIHELu+gemSqM z46J&Ozlt8eR!fC2g;~F?WPyPaNMDF(p7>eC{!^ zJ}obzx%voySi?)`>dDXi6KJ8#Z+2ePYX{Q~qUV)V2Ne3fw0IwSS`@t zp4JwdcUp3FBW!9oFqc2m+@92P?#~D$OC-+AeNH2W?vZ@f7*v#Z?pJxz6TVWdKCSCe zQ%oh~w1P2RbaA%65!a332UhIhMB)Q7U}~Sz7LrQ>N9;zob(@B};G(DrAal>r)H@FR z?EYi{uI;JFs@&dDM_IDj+Iz=Pi*2fz^$uaYA8DLc<@*&DTy`g*q;pCANei4=12uu= zvA6usY5dSE8Q0WBVP;c9e#@kIX`*~S%&vfU>c7cbO%T3chWNq>@oOtKWur_O=abh8 z5@#+ej?Z8Ut%;p4%o2fQWV=CN8w6(uqDMpsC>9eWXd?iw9ZEVgc7(zKYu40)tEMYF zQ-@g+3A2Rk@F>!Hm&a=w({6J)GqOORcw+1Rl;L-Bd7+x z`KngN&A05bL-YLpcUBCHYg?%~aImWp&e2@RWUI<${NWi`Hdu!%2m^jbKGGYvdvW6y ze{Nz&F^+fx6HndESOk;zDAE%g|Lsa_-~FJrI!5>R+VXJBo{O3`^1m|wqA&P)Rp&GE zH19;f=iOZ%QjPG7Il!?J9aTPR>cVj3{;8xkcbENJMk$AGMts=G6Zw1^KNI3PjS)e8 zIWBCY+GgCxH+M$qg*;m$fq4{Hvd17!rT5-mcTuLP5H-lz%kuhM&~Sn`wI{XCPV4+S z=#gViJt1P36aqWt!g%Xc|*4 zaQG#v4ViWqw)8qQ1<`IVl`di6gpq6(7L}S5Nv00Nyr?IMcgdmW%j1nm)te$sQS7aX z78TIB!6@~Lh#h$kyC1#W*8M5tK*vQ$v7Ujr-3tz$Bz801IF_lR-wr3q&7c&C$F>~s z*5+UMjU(>b11GBQZ3fL*n4_+QH#RAyiR7iY>8z=APrDcv=0bq%8qO5-O9TP~ zP6r=Ej|IFWI&K}|DTXpNAkg-Cw8H!z*~23H1E}JJ2z3P|;=eb7$^TR8u&0k98~8(i zQs*?%ErJ+N>5WxcDWo!YMlhgf>(f8hVtht@MQ-4Z*>%H&Qwf@a8cizF5$kEGSm(7V zd`A~3hS!ENZo4S1%)l5#X)Tcs?-mQYzMN_92MOgKxvh~wMnRXgsR(r^VDcy(hA2IA zz*Up941Vjxeo5Yc>O{r|tgi_ulQv{?w^LXvp6@H9Y z<^3Wfd&i$qy=ce4RlEz;VvsVI*j@@9hbt3RA4+ZiuI!MO$Pcr~49af&9M2OsA%PR7H;;#giT&yGd>bXqS}+w*S| zA}KmScrxeVPsZ$uwz<+}~{gu%wtPT29kZNyDd`e>j`}LwPn7dN$Uo z!`XfHKT*v8m@{xUK#h|X<#qc%-|lbs)^8c|W+#%DVc_PPI$mhS(;c+!Uf>LyB= zBR;X-`*~})EfFHa!FF=vFY$e`tV=B8AKkbA_@u|Y)~hbp|8~p%eK{r;K=atY{Yv0O z^yBAG`$<5`!EC1E_tJ|DW$~BPU9AnTp8oW9)o+>hg!?Hr4UHJHP{GL#Y(GDfdIj7e zT2cMU6PL>$EHF3t=oL|d>A(Kuv;d)MY>m3wpKwusey8S0YS#bRmnIf+_7a}@*1jH+ z*$npmu(wbg;-{sNkT7@f${Rw}V0SzUeeZo#4Qrup)+=TXzkBxFjOWRqTBAtUT1CrI- zJ1%E0fTI7_)zO^)c;NrzkK9eLG+q;Xo&U#2>#r5GJx&Hk1GU9S_2Hr&-t(#F&FRJ)#t{3Z8?(C-CL8` zoqk%^Usb5?q;FTO9eI2Xs2w)WM$VOq($uaDWt?%uLP61$`|8U~9p5e>@oApRgtGkz zf(FAHfj18da#p`84K;awMbu&ERsceRSY)f{@|kCy9G+WdOyfv2Q0oA<2GQEvghQJX z=j(8%6>u$FC4{=`>t_hb{R zbTJrI@5$}Por+OHG7o&GS8v1!ZpPRJoeFX&drniQ-+@mNa4n=Z0t=%(0Nn$@A-wq;0Up3!aD^bploHq9 z1jhu?NCAZ5@;y2Lda48DFCrHFqg}wj>)1uc4$35w@-fkMc;uzBR_$MQq4yI8BWIOY zPgt4zQJQ9S*0S7%5~fq&vlX>B!t$P}3xU-aZ;Z@A$QjeK@LN(eiJ(?<`x zHcLF>e9DjV>jT+cP8B$1CzIAV&{%i*daqX!;8)y016b|H0{L+l1~lvM;0YqRjUc*+ zzl%L~A_P}cNHD@2;y$$B7`$DXbn@}xjqLaKKmn}$*{IsEn+Zu;aLQfN-U@}bg#N8e zM{yngH?4x(JK%Dbt?~&2d6Xew#EClm20${Q-?pxHffajKQLhuJ&*3M-yBZo1%}5}z zJo^jDG%ELE>nezMigwA{l^;P5PDhc8zR2y2a7ghILf#KSmDJnI`M;E+X?n*T7%Yx@#RKQMhG$U)B)G%->5PK^lh z-6s&;^nh+pI2mFuNob~XxQI;=J7Vqz_b;HJ4(wdo#3wPpllE$Riq&SZ2$X#f%33*Z zf>mhvR%fc2rRMP!{eM_ox+1fMz|N63TP5~?li6I#e$_i3&+RIBTLX^m($l1QENK68 zKxA!ZEq&m4@M^=D^+}HZKT@cel#J#d1!=Gr&KsWBc)Qy3BD2*uN^?h-d*l?+;Ii?#!DDR&y)~pWQ=v_~6vv*a6iK3(F%vJ?U{I{K84vc&LyUkrC+B?$!?)RT zZ0}I3bFwzBRje?zUZx6%8HoH0SP17E$yv%-M{}YrNpYxmhLJY9th65N^%Nt1GyGxl z-1F?V+2td$%$so-Og0&+dpafVefHUeCMx_IpXBRI9l3*#O(Hk>&fSBZ@&T%%)lMIV z7_AYd5scWf68Vz1W`TRBx*8kn3AT8kahQ-RhOy#Z^Lsi?3Mn8QkoRX`mIZhIY1@Tc z+0?`Ix`c#kOUDk7hw2$o>Ug1k5*+pWHNei@Ec!wR#?82=WRUH9CAtpDXo)B-{|#HD z!Epn4qFEx(c7RTchaY)glKKeY=w{niLH+Jd=DpI}jcIb?MwniI3A`1ZeIJ=8iTa)2 zn}C6svz-$YKd0MUU0P?83SKohp z1PyklTL8LOHYsB)bzcYts-M90Yj=v<%u2dHW^lx(ZMh_u_ci)x5*~uAbKulsyV!@uRYALbdQl*S!x`t>cEC9>S7+lUSVfuT~oBs z1ou}ROVx`+{&D=y!Q~!Bgog#c2fLtnF%iaXbcirS9|Q^aCp^3%7-by20Tz5@r-?i7P-T@cUb@FQnemaRnLK z3KkuOj_>Zix;w*Yv3%%x=NZW&9;4HOt!LcphrhTFbx>&G_==$fg_-g*F@p+89pp#E zUR|TSm}C85XVF-XRHIa~Mm zA#WyU@Q4WtJqqh8jxlzqqkGSnk9{4H5V}mx+mPLZ@@HIM@Ycc7z#ePA_YETArD(!d zXamd*6@!Xukb?QYD=zrEmdxu=NlJYBz3Q`KYz=59%c982JHa4k7etCTM8$D=B^Kep zWnJ^-2b6}B_tE)^;D7|-+Y*|ugu1CZ7)Vg3p1r}GE)P$oA;G2Gl=0fXzQcY~wFd)t z6{|eF^tyq;GwPhV@?2xv6d6V`0pEs$6Wl0)=Ry2!NQV3omG4`%Ka)RP4%DaLwTnh9+!eaE1SGmn$=X)Y&Zbk92z zGLIvC3Figw^GMFfBka!=2{gokTRhF}C}D|(ttVQFp=*XG)zf~tEWq$l^ASV|*WxCr ze80Vmb^7v>0*L?228F5EYQjR-x2A?8YM-!r;Adi5{2m+cq$e4&sWp-KP<$ur_Al{tv9{5iQP{&y)RWshF*YjiVm`e#bCR03 zZBrjlW!j_|bcDQBDH6v!vST902hM$r1I&((RU7QVzj5sSa8%?SpW1RtnrA$ZI0zO# zwEcbtli%q0_{V!`LE;#j&>K)~uD;KF$oY02Z`3MzDZ0Y!$3K73&;@em!jd<&;q&*h zhnS1L$Pw}#QyO>U55mC-BdLCCG5EUc-`1up$;RcTzrkavcc<=pj@@+SH1C%QK^T`H zQ#wS;6L$B9L)jW_y;}qpLy&2j*!~xwxm%`s()nFXIvQW$xmWMiJ_+?hjp%u~1ep~p zo;S2!H|=+%+hT+jQ}09&N=>D4HCwRH8o!@yTE91zuVRNqc+YX+{I$kN3#oylmUz_^ zxARLnJ#2*QAjMpg`T>fEq1<9tx5&lsBwr3p{Yd5xk9{D@Imw}Uy=ImT7)g3RVDRu& z3H^%X)4SJ46w_2=qXd5eks}TR7d@UX>4`(6Lgw}dz_sYcq4>Kp<@<~?1&IKjRi5a0 zd;fuc*C8cgp+=D=mi{$Afa?n;c)~kWba1E7C22Tml*BCxstbwFFx~h;R^hQp(5pl} zw!&k&QfTufeqfm>r!O0qJ2x@K^E2D@^QIz}7xQV&W>6+aNF%0c5(?9oddG7)w~#cb zA!dWMIMReGTno(3)Tof!Tyr7Zdr?utezssld;4LhzI4C>woi*)fNYZ8x6Mv+c1}#u zVUV1qVw#16q|5(#fRJhjMthvvB3r8J-&?tV0l9zCXn58hPxFRjm%2QIm?Wu2xj42J zigD4VXH4EDhW?9BT3AY(H#>+Sa_eQqAqDwPPPP^+?`4`10vL)UsdkV(X(C}Wi63Du zvLW>o>L8ht2!Cn|CTXB+dnZZq0Af>AOC>4s3of5W#u=C$0rfAd-*QZfdb66-<)lYE zlXTka;~E4Z2{X!Y>fKpWGjS5jfk^`|`!TGoit@?Pgq5=MPirnsB$q zL>FytF_|QJ@ZghH)XS~Vs?`(je17A#I#y85_fB8bJM&C;C$O<@ClTcWx9%e}j(I7c z6_e((rwlmv@%MB;VXU3zhd?LEcjo#JDsE%dl7KY5DFao4JE>#Kx`Rp<^*w^|Pv@-$ zQot$o&=czFxJ80r`LvLS}OqS^gT7>z)0!4`I zX|teO)8BmPS6scVeTVCk*VaKL`2xsid;1XAF>*EZab#_=gg&X|-qNT5=Ut~Ij}YC& z+aK z(K-^rlo!z){xpC?pjE`j8ZBad_$v%cJej>u}hU(W6+vBL<#+< zT-B$s^W(4LcPy&SYkeZEr{R5{eaj)*L4RJ}M#8j(I*k8Ov#_Zdc5&67+0FDSZow)r z$Anb1DNxOha$EftVtBL_cQa8-C%K1}3sdr9jJ||hvdt_D*A?QGnTAiLLXh$Z_e#o_ z<^qprkbVQn2xuanT@r}&7B&ctl9W4C^WX2)VtJ(n+^dmID^S-2USU#;(Y>EOWZs#b z5PR&LYJ&(b0^|U;%L&hW4)|YdebKEw~rlr37 zI{Wg$Ssd^B61d7U!%zhiiGC5(KHyI5lBFElRz%iieIbV%mT2YFPE??xkLEh9@-dL| z~R`-cxf7GtEd4D zeHp&s;pusHULsOGB-eOvX@i{fHU4Gt2@|b9Mk@c>dj9xb`W8AJTfEkICex9t@={6Z z8Z*-YpSAf*iL5JDEfZE_A;Tz-*Ap-oGBkS?t@>?soO!+VZnmXD-mVfeFHdNAZmvkv z1aobMp@OjRiw!i?#t^KR$Nkd^Q`#s%7QcIIs?3(5h<)h)-8 z8j=Qq|FA74zIb>nnBQipzI$&(wyErb+gDOMg^djEAB=BcR*G<=Hq8-J9&7g(^TI{K zy_XuG@D@DcDw?Tu3~y$ewkwl9uN0$_jzxB#&{B6_8Aw|p1BXApvmY{fod{|}aw?D* zM1Pm4Yyb^cnW(WtUgw(nl;4dXktJ&d*t1QTB{17rTZ{9~(%R7po@Ef|AuE5Aj(l2T zm!omaf#)9!@>1tU-!joAF~Q1-JqTcO*lIs3{VMW-xtY^Z{IGJ(OjRnLv))O_## zXaS$TzrjamYvs2ABy)L{;SaC4)DPzP8TE1&E9J3-nv=JEP<0#O#QZqQNNqtFxK$sG zRk?PXH8OAEB_`7kXS@9~L}_ zb|?M7pB)YplBzM1n*ZrUH^QHb&`KH3{&P#Nzy0#%{E*>JQ5F5_{S*7BpZ`Fl4sUwP zlk|Tv`Wb=o8q<1E1KF?N^xSoD7g9Dj|Ld7fKDS>Ell`qygbikeiD2OOFh^a&{%dRb ztK1()tYohQ2mbn#ApGTvHHl_7kr5Dg`cHiX!JCHK`@aZkc0(MA6jx6scmKSm7#LM) zb}LO*{{n%8_l(wuTe+5vnSat&`SbJqk%oNWqJLsT^Yg#M9Om->>6ccfPRl;<=S?eP zK*wXz8Un8h2}jE>3HbRSh9%$f!nw9l&+Qkeyny>akY|Kt1V-v z{0ZG~#NJzwmV@;U{qgrM@pqmDe>8%w*UBE1U~0LY$hrPw$8NdE>wD&pmBszEWh3Zy zpaRhZ#ciSdzqOOf6gM1O+6uqDY0YS9|Ley?0cN*-hy`5S#jLsLUB+gcaTf7@1`arK)`gBi36JCpK$&q5EaD^l z>KE_%{+I!{tSAOYl!XSWqzY2s^Etp+rsY36mGm{g2eC?JxtG?yB8!`|EoDYo*6P9+Bqs z;NvlYHkQIj7vexZd782_N3}dUdkG0j z1QM+ah&(urm~jeUpV$N*$hAD|PkV$mj?tGYoRl?IMC!Fx-pf)cKPqDdI*S|xt%Es_ z5q^)&@_I(-Lk`Pxum7}x`tR!&6F{D`f)%an)Ej_Um&HPObx>wiZY4uuMJcH93M&x7 zJmN+4`}2wwd9R4R$4;*^ZU5-q&<-ZlZ;ohAa)FdaZT z)jl-v5+qk}iOj{-HX=C|i08W7HHU3zE!(1{he3PMN3tusGzWzru`p0|>TbT^)Fq;^ zm*>Co4eUBaft(FYVnQh}3W#TJG9>~9wGl}N!1LtfkmtI5yB5hpQ!4LWf^oQ~;ikS!C+yUQR5<;zr(a=Q5U^7+LQO#fG z>Yb6P(}di4GLBC=K#Wj3pZkhFC}#oV4kg7)6ob;Du>Fc1_q>#N4#i+f-dCvNoY_B* zl<4xAOzM5m`H=AG%gN!&%fQE&m`h9l`dsDVY7Qb~cxoP{z$>tl9hFJ`Y6Kz&F=D&C zkby!)Ex(B4d>h=apu{quscgYg5AuI32I3w;d#7Rm@T+{ToBlakFlpJ2yAx=geHsvK zA?S0cGC_KfAcTEzXArU7{1e99-3P6?!Edk5IyTJ&VetS%-#lAth3&CrUb2&HLocYJ zS}|A-z&OZA;Ig=qfr?~HtRe)`&ck;T;=* zI{<^f*sw7{4NVRRkO5`Kg)6y&RzvA7i{Qh8ShS?R80iwBljMh|bDq?0x+vB6$TOAw zWv~mS|3+>zXnfc-n4Q*P+SFeybg*3y0qd$C5Zi=QMi&&9f7Sq!MB{~hXVS;q_O5ak}a1Jp?Qb`4(n%+{c*)eiM&#dT%TFQfS+ zYV)s$I5LDb!geycH1}u+OvW&#R2%RtgCYRx6rNF6GpCnyoC1(NNtM}&nv)B!b$8aT zQbAoCX@p7S-C7rLNuWUJhK;;86(G3UROGAWE2r@KeU# z1*X-nU8r2elpcpb#y~lDv#Or}d})sVFhzkNJl@@O;55MymI1yp{d#GMFMMEdz4ozd z`mgDM!!&+I4F7nSh&JZ9MT8&LjH7+SJ^3K1o4{?Z3Zdg;+A13Cx;aYm|Kgs}vWwZ{5MGwbA_3tV zaalyOM9K941@113kqKW%4WGLE-t>!~U<93Qd%3KG$QJtCSjp;IinC_Az4@Y9^Q;c> zS5Sq`s|ZH`9=wlDFp+J`jdP*Kna7YD(8zWS7r0C+ST*AUg9>>LrelovRC9$pxg51~ zh7R~r(%i3OpM%YT{F_y7jZ79Y1-(n+HIG;w+6>KxGWhoV)z)GetHHyKw(UO|w*Ml9L~zKk}r76(%F`wZQFbw#xc; z^Kdky8mtWsS%hi0(gml?15nlVrwGLb7lGHvSBSJ)mQ~G}BDf(cEAV0vF!&TQM#HQy zrk#8ik{H~;+N_+7C&HJzYb$=>;Td2{<*S5rfEbAvBwABukxr#HKX7mnv}JmTZFU_t zQ6-GW_NP?4qTEC1CRy|VHi5S@jzcG#Lkl?rF<585PFf91j*A@E$7BS zJw0Wu{Gs#5*By9PA*ExYD!*<6w*8O}prqoXc6&J!c)b*{Ut-1NEIG%9Mm=P8%_WFn zs@;x}Ecx2!R`V+m>YUy6uMGOqJZO(Kn!@cs=ea0&N6#)nI%ouVva+kMZ&fgdDYN1I zxUFib*?;Yl_fvu|$RtjubRq&HvjPiiMJFi7w8@tBoTcj~XZ>39$fegKuk4;gV?ryR zycU5?AI@g{820Aa65NCfn^VoA-XM^>I@149ZC?Gy_b*DXw8u!Xy%Ck$O}d&1c`yCQ zhIOEGBj!76;0M5M(MMo2>@jD}SuZ*BkrSpG+7r_6DcmW4nB}uzzPHt+Slt^5;I>Jz ztf@SaZsiw@1^rgM)QWf^{uJD@?vXCVr*)}t{t1@fqfx<=83E5sj47+Ast7yRm64o% zO1yc7b~7L+=4OU)bUiLCEJUP{fK`B(;B-J%tZK2Q$8{;h0+4?IFRqv^DW8K*Lv!3f z!{f$|KFdvmJnAHf8$%4=Sl_^@hJ8lW?8!0s*R}A|D#mEp0wNzpuS6!~W{2CX^27tC z)s^0~>>VL{YcP+U3n7;9AY5{YK1HRx{&*BKDxlh1 zcO{Hq@J2sIc?*oYyrjoIzR8lh>J$M6!HMff9-`W7BvTOZrCFjg&Z+rEN$u736(U+a zTqllG0?M0r17e(_^cXc}8p1o+8rNXfa%Cp4wvp7SixRPa+$2~7T6$7>KQ*I8N^Zkz zCGz06Q(&TlDy&1|mNsDzwl4QvhM}-l9f{jBVEEwd^)l}G70VdGJUi5uULi5MJ&nWk z+`Z)m-0IMkvmOEN1>dAgz81Tp3!PXAbK@h-!maFNOX0K9sSR%HiEJ>tN0}Ch&4*X% zQqX22VF~M){#6sT5{1^{Rvzj0F04+GxdP;tuT@j zo9Z$mgGJ~RJAc!r1|wW`@QFjXiK=SHoC&d{CQ*bI6DG5HVYXp(@xPpI#&RU=CSYQ$ zg_a5x3}(AMrM&qp*qyy%bbHtLqLZ!Bz=lw5>^|(I9S`T`fk`!wx1HSq#?Qoe#{~!1 z&`qG=H3BPA$`l)qobABBe6PO`<0e$Cwo+g>FwlL`xAgJ?0lt+GBrm`J` zlN5(-v(eoiehUh7jR#XsDd61m85#A0m{Tdifr7dMlC~>%~l>{=oHl$kZ_wc#dx$#3`O@ z<(;`gPGuW2E*IWZ4D#t~tlu2f_*LET&M=2Erix}VQ%nsfUCSPl7-@u&t?`bg9e#HN zaQOVTnQ6Iml#s|b?CbTkPKo*Nx)5R1u)NkB8FhqAhyxmu?AEgw!BZ+X>@RE8=^QES z0|DZi>%9QW?!c)4POoIAA`fycn9bW4wy9=leP$Y=dwV-psBXh>@7@QDJ98SNR!tPF zvD)*w(bbAE55tudY8Uw5@g(5a2dkk+JqZ5JcxYGhmBpLkWH^`m5^N$CoAJquRCs?m ztq(*(&n04W%W<|xKe_hmUOoQ*-E02^ zM5xD$eLF5~YQ! zlio5Fb>sz^d-dlMGAo13zt+Wn9H9W!Ui|-g>HmX1 z{(sHpXVK^masHFc&vMZZDDPl+ZlW?b?X`Rs3f-;1gSY7D57-pmQI7GSO;bd1hipF{ zG>_PtbaGBpWEBhfST$&l^(xLqH%(DS9`)85O9cP;yn2?X{k@;hn=%;!+b?`%`1P^~ z2?78&q}$&u;pfkxSZ_R--!P7P!n32_oc6aTa!4M1zv|0ZP#D?JMY$&a@v@7s_IgA9 zSIc-R09GabcguK#7FMdSR59NVVL12_$O(woE)aYhfSKV9&;>9|owb({5UTd&q5aXp zUi1BLI$+)1!@myAF5>km$67rGek2Hmnlr5F#_eW65)nIjP_QG4S=!he#|8v)&rgQK z(=(8~Y}uNi&>uk3PJ@E<5*~~6S8d~f>xn8YJ7B&Dyw}gbOV_eYaqR&ccxSlAYOCQ) z5CmSqDvPicfk?%&C@X|R-Tl&I0o-v?Zy2M&wD5~zQ=a876g5UFz>8;P+LLVf0__3O ziVjc%fd_v8paNLx)kJ*O2Q{cMP%cgrFS#!feDUoxTHGBG3orq?+*_!oBp=oSOD2}A zj1`D7UnfEK5ih)`VQV%JGIYAET0#^H2Fms z*e$Xq39^B8jzVBow_@?iwHOf`4!PQjyuU4u|9MIn%J^aKr?I8BeAWVK<(ci^Y~s?V zm$RDZN&`=IaGsUU(Sc$RDXO?`$YhKSmS(!E!Nj zM4~y)DJ6(=?+`D3fjTtCO?Hu4g9i+MsK2P|Lisw%x!j*5mEUozhV}t~fiL%#i4`5q zpgJ=2J>Iwt?YKV2kh0ENs3-}4!@Wc^m&y5Tv(hq4 z*nXx8SnD{nZz~Y#zXM>=<7`a&T{;ivZG)7Hk@j`l(ho@SZ2X(?@z_^hbTGci)g5X< zh|h?+(mS5u!W;n1zk~asU7B4IXB+#A-AA`Pu=L$-_fnxr(8ZG0dnBFj{5l`>5TWy( zngTdtO(*_TQrK1*L0~0fuTc7g{V?1ktQawuH#a#z1C2V7s17=kQ`cdve?YdY0bFhL z8=;n)&{X?%XkYV~<3J(auHX~@z5vKexghnfZyZh71n$E=ATmQ8^$@{XFEXvWaYqZH z5|%r>E~*Xca3G!rb>TUutHFi!2KncYCmUF(!W<#-RTOQ+KEi;%gG9^%`B=b0p3V_O z7sGV%^$+I70A_sj{pGPRX%qYhk8V9Xf5P*K^*97gdBa?%;3{gZud*|%AEivyGymxV zeR3IK`S?5CWJpLmjEVYwyq|vuJC&?|Oe(PQ&BU$WZ}RAORcYVVF(qN&(tE1nIkbI# z6>vF+@j?wf)jR3CZ6or*+)uX=@ZBM0Ju0^27H2O)q1QJT(gRTJ<&QUC@)zaKF9fzJ zZMT8$91M0;h35gZeE^8%*E8?(u=hyfHm)I&e!vx^VEb^P;x~`05RAwN-7SL4y)G^9 zAc8hz@$eC-iVX$3h`*!{EJwH9o8v|)4I)TC_05zyR%pMZMFbt@>bPe>HDFRTAAb{{ zM0ygioZy@Nc6_rZ9v}U8je?eizO`E}NAszKb=(Jc{6d+UOa)9VcAW)g<5Mc=D5MR& zd;9i{+hP@f1m|G|b7wX@O=1H}BilHg!bFyxy$@;h8dRo6X2HRlI@Yuu*&_^@xdftY z*;iF$dG!}q%6|tF_VCcgJUdhFBz6iMCs;7Pjr`eES<)_ zAX!jT;v=9*)0_;9>E;7wWCq`S;Y6um{~Ne7!ZDy%sW>`0nF{ia^mj^LcA<2#HUAL`g3Pn zd{em)M|L4ROS!vEC;kK2uS@yv+zYJ?bpz47WYP>>#l;yjnzkDSO367cU*Pb}Q`mTI zHRehRuoJrY9Ue29=n2Qi5kaaSU(-cx2LZ$CQ8GWaZ#~Zy2V7$^>g_MTi_T``@(Nu1 z)Vvg{`!t{{q3Z)xd~ty&@r91Ql`rU2et;HD{;s<`%ug3mdSQP2p9EjvU=nkGc_m9sNv zUGL0RjBtEtqVm3lb~B_0^^3`!EH(feo;MY+FVXe9YGJADNE2^phV+ljx3)0E44QR(Zg{wQH<{|V>w1x6__asP zR9WFqiy20qayNsoYgG3z>SrdvBAeYff$$ztS`ZFBrwBfo%g`;XoEzj@Oaoh`#Z~k) zGBi>nWmO4c4f>bc4a0u>$+aY?uzvsW^qG-qRiXgX2^WpN7w19 zvVu$psEknLV%v-=>Dxi(_Y|?7R8c*W@yP*q zq?Nvj+%+MYq_2Y=rv|y4%vUj{ zxg#rAbTqLD$;MVRd;xl49!zwoCna5AL-*hy`obQ{XihHcC{LwYC}2urk<@$x^TdR5 z<@8p>$&2i2bn-)AM z&n^i9;8L|m8O11qd37I5y+eZqQ%HB9 zq*7*a*W3Jiy_y-rQTlwN%H@0X19o5ItF)a~!)?Q}a}7zSuk)cc_clM=&wYO9rHZ7a zFzrj6qm_5KT=W;RA{-L;kvj8$8qZKbE=jjdqd=+p%7i#~Zh$P{P)cE%fe9k`e*d~v$Ii6bNZJ}emXo&(qer6&HtCa?W)ehrKDmthu`z_Np>88@ zrZ<~HBA@c~&f_^40p@A-IF4|BD0KY}*|Y|Y$S3jhrj6Ed z*kxxJj?PP&sK`fa0^`nzGE-?5R`k=@2rJyej8#u_X{KNjHF~cyY<{BN-OIa;)I$Z# zvtB}4jOK0fE=GiZmKRh$Q?CjLF$CJ>l5!U6VejLEgG`xjnUQyf!#j;XhuEym15`{n z5<)fdi&8*yp`Ql1SamvFFF1Yr%3t<%x)J4jI&X(FU1DZ7Cs~PoVx=@gOqTCCbR9pJ zi}vQ~Y0919VBV=a)HRQ_mDvqmR2S(yeu?9c?{I@s>aM`?pQ^c@0_6?kV+GTu*A8Kb zYh7=SprH~>dyoaWjeM`4Re`%}r!pPw$?Okp&iruMZ3YY~IVm_^oe|^J9&IG`TB&Sp zeza1#^Y#RORDD?zSiTZsCi-g~9A#@s-<^0pj<8rYD)||q2gm+ia=`ieJ8eg2uHY<1 znIA3g)nrcb_Rn#W4#lktp!jHjx&2hpU2|Xi3T=&MR@c`c6A+fWK`Q5jXyDD3HH)!g zZx}zwdRrcA!~LT3^Fy0<&UdYLLAkA%PxfH|qx4&5ilVBDD=EMU_fsI?%(fjUBqF3jaxt;JsLg#jK2kb;$&1*}<2SKs_4%Phs?Lg$9sD+>!b z2D{M5*AF>Ng%n+}sbGqyUD*##jQ||MH zFjzABkhqXhpx{(Ho~uR`LDWD{>cTUccSp=aJv@0YFY)bXNa?JRo8zXJewcXe;X0%T zwTFJy&LG4@`D!zUYi7&%lM4DGBY3t_ERGN11OziC%n%F9_gXHyfow}bOf-8Yd}1;q z&H(eMbauPw`GjT64SV841 z;jZ8-ZRqS>Uuy<#ToHpPk!XFLuL7h-=c=Q-RAy=D5PFChLnNw0n_!Sw#HU?}jm0^hGTvAGTfUJ(MK{_6s zyVGu3%)EAE-q-4VG)WotJ*5asDOr{wU^$=g2EquFBFdEc${~YcmH0N2sM%D4L=$#W zm(gE&^uWMbj?3&T_bL4I;=*|2piT~>UDZChcx|t-{0FlX645q&O6orJl9eoO(5t+L z{pf=m{N@Nea14jMjd?f$_G$<>J>KGVZIaZ`A)2OQV^sVjB7q-E492AqjWg8}r=0aN zHJgeo{6ujzv@L007tW?weBvX%=V4Ggl>&s0(U&C8|9JRD&f@SMLfTDg`e($iM}ftz z*gi0HGSYGp0%xt5-CMunD?K4TLNyd~&Jr&Bqpe_AfeMCb`)o1o%eud-V2F7q@B6P6 z3@bNUZGDqSb~TNF*5}QaE?5P@d~kCwnG&IeA;qfX)ih4T7)U`>W6UavG4>KA%iLevo;wLP zO%j7Q+AK*Nn8B=G85G#V80Aotg)gyn?#vC5;f^J@PTm$`#|uwZV@Ko)-(_knHH^Pc zm+;=2q$TxN9>bKDT)KVF6Htz2_fUJ@0&h#3ULeDy-`PHw$bGljw_{iKABdg6qn>fU zx7YMO|D0$rI7UV4aE?S%Q}f(OPLx1Cx#5bS`3jc-!RvEWA(XP=%}g;Fg`Kz$AK;` z1`%%B33?q25vuI75;5zBa^Z@w2n9jb+Qbhp0$oqZE@*+-e%Y2@vL%f3g2O(cIL27( zLw`fSk*!h$3C4-Y&=A^g{=%(Gqx3&E1;uzwX15{IihD$%1G=F?_gVf~838IW`HE9y zB96o5%sTUT4TrS&i*wz>x1tTWzw3CKIXJWyhe5I@CnK-nR@Bi&ehHYzQKi9d)4^6h z^JwJ-5$TP-VF#1I< z))jEatPdePO43=_$xV?NP4$3ITz_dBEk7NIcGOuqz3 zyu_MIg!J%{zK!~chEKtFzi6|rU`qB6u%9bbPmV_|gS|U#|H1OgvqpW8R&-Yg(jLhr z@x3a1bdLD6)~%~?*9f$rz`{(wUw5M*pVvb%Tav{1#g;h4eVw1PleZm&k~UkY>H9FB zAV*=~skn=2`-FzP1jh#$=8EbCYNB|gWM&{oulD)=^-Ri=%BUltKBQ@e56FK%{f3b9b!;Kq16y(L8%|L+a=XjRBypo7L>Q zMf(>u5E>jXLyz<)gw-EenzX#Ge~MLsY=}ujMH9e-6p59+Z`jA*{Eb{0S0Ci=Q^m#} z^5cbioF>Nd59C+1YJTWUe?lOKi-Lu*JNw|k^Z(X$$*n3)^^}9iDeIaNY?WKfqcQ-T z+H@ty+nmtIt$=o>Pt`ZX&SO$$O3NMbSJ8LpJd3P}tD>=8yctc?@Fiv-->Ico-On(Z zJo1aS)L_$qi}7-WwBPxZBmNiu#p*Q0p-1@nd%LmDy1y<+JQN}8DeG$EvZ+%-g_@FD zK{HE52lwlKg9(p5mmd!^YQ3s7&<R7P&?%jE9=66N0vSfqXG5cc0#ufP2( ztdkgq>@RZN&<-8HUY2|pATcRfw=w?nGx1|5|C zR5}6{X%RYq^f%%cJWiJZe{I*j4vCrSkCL7!zE#*;;o&qO{f!jO1%0{*5r`yd{dm72 zB;ZFYBY|OY`hzvUArjzs5+k0D#ca?2!t=Hx_oc438*N4HuP*~14Y(rK%Kws#rVTyS zoO1&n|AqCGL(;HB3uw^#w0_<0I3%cxy`4fNt7z*Q{`dlM@L(x>d8GXp z?+TKO7NTzd_`f_c7vPD}=>F>tNY9Wi!h$`6m7iB}+jx2D{RQf)*VBMI6tF{OOX3fd z{ryg9K88CzK>sH4>aP!n?jN>y;I9eR3A#1@`Z%~+PuqCT$WTt#idbV(X1&ciLAyHo zX}PEQ)mHblKi};?jYdzMfoH_2F8_V=I`pzwyq%9k_v`=MZ~{3_!Tv!~>HtPcT$M>U zghAZq4`8NBmvU2to=1zJ?{D5i>_-{;EpP#(zhi=n#u?xzD1xwYo9}F+E$wv$MJ4bQ zS70Us_R07_!5jTsH-m6Y)L&LEkl4l@hbhQnTrEYYOX??*g#}5%2r&RTR2FX&CfbY8 z-n~;hctQbKkft|;nPBk#Nbi30A(itXn!hd`e{#R4U2nR)3>;EVp&)|`8G1m>KEYZ1 z%KWr?5BuNm>t8Dh4zVG;d_$SrdD5#mNKXW0K_YDvqx7EOR|et`8nXA`Lh1uF1q6kt z7^#WeHaN4_bIVxNlHJ$F{h)ZY(*&e^PIjdbl~k_4u*QTWUR(F1cMU>d6HlJiSbRgZ zEk9Crz;AudDBA1=7+`2b-Jf0&Jpu>uj!N=G8DHZPm!7iH^HKZPIzMrI{cHYbF!cGbXL z#C?)ZMr-jJ`5?)6{7E-Usk|llx z9uO%Q2XBKSmq`B0Y9uWeN<<))&1<`u90ez8*`rj}YAPWI10gOS;EYOY<|(<6$RtpK zF=o65O}3Zd_2mc6WXA3U!#6~AS}``xQw)J;&I4#X(yd5;K?4p)w0bV?*mFRzefL-|ox} zfo(-KGhq*yO!gLV+>rcgeBk(@?Urzz6g3J|IL#8J7LgW3j`7&}D|-X)@sV)zDzSXO3=>WN;u$! zjOkd;RyMu6AAGqe35gXLgQnw%1|WQqR|?EG!@CuUGZ5?{+z0+K#K3}k7V%Kvlqx=g z9Bco=v35s0MG9(66NgO26C6W9SwqKpg+eZ5Um8(3CpX_hWWkeqgGePp2}7yWv2H^? z87UkAU|L_`X?b{S_zV&+(f9>^?u|<-ZydqAfiQZRxJ231ToFZk)gVE{rglCiL+N1$ z5v7!Au~G4c_RXck@-Z*ypywJ%eUS7ezx5yvN{?BOW|mHdpR`kDCHc^McSO=!ncbu( zU7|?z@+kUZP-0k?DchAr)G-fL?yeFk-4&LjBwnIyd7$w?LhEV5^;kjzyyz1II&xQK z`91xzF`OJbhMS^e5yDi^U?Zd)&WvJvO7up7^>Vn>bM6kA-lf%|#k}R4bPP{i!*vD~&cki+tR-TdFXt#zy{PjuMjNaJ0f+e&v) zV9)B+y#RXnYUm`#=X4@I0vUwvU77{iqKr}E=k?O4adK&g1;uFU99KA|(@iLsk>00g zWsEZqRWb2L+&WPNgsmHrBHYlx{>-W)ftp8`V>};=@ssFE>4%~CX!CMJ_N$_IaE^%Y zt~*)u5I8^L-X!=)hmiiXiJ+BA&+B7h4mFzo0&AEO0X;VH2Mj9NA^8F0Zo;^qr!>&c z9|A@&I6ivq8BwQorVj9<&J4OdSTVimf_#TQ^HOSt#V?nkxS|PjN|+1^48mixZw^1t zUmuTvR_*r5)}F-&bm92yBfv;UiL&y{FzG9mzS@Fdd-+}}IWu1J%WohyzATL~n!e{? z@IC&++h77z5MbM1`_)w>r~c5i=3qTtm{8J$Kw(8;iK-g%9itOw10be9-m9SaE2y5^ zmzWlqJw^1ba2;jr zNigwKx~U#S_2gLKqcuhUbECis1b@{NU4n> z-;t^^uyZ|pylTii6xQ}s8{S6%M~X?j1er&4EK)et=pgF|+q{x~1$JDhR%I249Nc1D zt04$b+^6zU1bd#v6-Chia5TVXEr+Mbrh`Fy!!X;?=m4HLd)u>~Md;3Xno{tHEHBgw zb6|21Oyb@kF6#}lU51)02>|IpKj91D>io(vr4!3VLsL=rA!W!KsJ^lsrrTOyrJS3qhEUKix22-p!~Gu>M``()1|v{kWm`fLkgW70>YV}Pjg z%#+iviTivX(61Q*ZwZOM5wBoT?+MRf)dT(=ha5_!Q2?x4^xpOe&x9df$62q{F>oU7 zBs*@QomHQ1og8B*|219UU~Hzt8wTxa*C4{w>OJ^HgY2AY<`S~ShK(5t&N6abXL&v!{ce}xSG=Ql2WxUl=Biep<38Sg!@m-MbR@W`KmuFB zXB+qf7z1-Mipcy?!ox4!`qVCU32htv&%ynl-|uN0u%4Vb1Mb}6DtFBsAb}Cw)IBiq zyoRD$hq{gC*;`0kfzTx!0uyo}2uuM$uX!-8Sc_L)#vs*_N*)hDxwbe|ypuGPS6D}jzr z?;z!T-AZrf*TNl#1*31QmkXdI%~@mWQa5BXU~UZCD=qi1i+fFg;~KB7SSn06)N%T4 zJKhB!tW>aoiTSWZK-0222n#HG^aa#Ls9&we-b!`rOqt#B1qzseIaApnsjnG5jl;|W zCm8cH%kGrXZy}y*!utys#|Le7#K8vOI?X+BqNkD2PJvtNPj++ZNzoH z0UkN1_2~BCk1G@xT(yR|;BMjA4=LU5?~A<_AoyIGvk|G*83T7mPiic{B7x29Ja#B1 zmT?#in?(-1#cmGfjC13jP$~Lww5RTOKF{LsC;F$3$V&8SB9BwkFAyq9V)=o%@rk>) zah%7*qY4NWSPbhy`Zlokl5}kh)*4tX&m7LsT+}Z@Sqy7$CRs!jptyw4pQfJ%A^1h( zOHNFCL$jub86*Do(dmgtB#v_4L7&lH2{ujNz^0Ii90|08!XN)v%~}LRd0o}5itc6p z^;$p00vW6Qu>o$hC!}9HqB4R`x(|N(GwmAPZR)=ZxCidjk>4NjrI!FxjQ=_!%8v$g z`eVTfq=eIT5r*z--G44X|KU$;#(~Vrx9xv@&-{ktu_npa09j#j5G!DoDW_2i!CEuHWMgCAFM1)R_Ud?DNr9Qc17^?|jZz39sr+VA`8 zKfp(Od(~ZA?7v(}D;P1gO^g@NP7uFtye3j2kQqDj4BfHn*GnwTjx>30E{h^p@ITzQ zzmS|v_-p_70Y;U$&Dqp{#ofZCpSG2<%yqlI6FJi^v8MEIAKE|`b_`O2kE6eTQz#SR z6_~m1WP^6Z`E?CTk(WR8)Z*&zAH#5!x3^0Li@z7US3w%I&*AQZzL8546vbSqO4_}d zV|FxPHeTliY~x2ll?!D^UIEMuM;=`ltxiJe)CMTv8=!R^3AcC>nvVW1JUxUra$use zAU0Y8c61^ZRon5pT@Zliv!2Lj#WyZB!@&C&Ol&e=5`?^_gb|k@%4*44`az z?L&`nuGK@fcyA}IA^p`VbnhNa-rQbX&S zs7?nq3GQ>$4NZz|7$_0BE`)j-fh~aJmeYxUyWju3TKR?WdG(qcOtp_YEZ>0hs^2W=xbQmJ zKNUn7j>9EH63q7>-43gCn(&88OE$7;KnA=Zu+6S{J_I47;qn{-NTGSP5@aV=J!Tpa z%@~})X9SH^Kz$EQZ`FW(eKU#>fi!jBbd#zcM5zrUD+9?*$hxWd(2<_49H!Lt1QH2@ z3QORi5rIqZb&?fY18j~9d54w^Q{{n20d{lRfcUiaW{f1-#spFn_yfPlvBQlMyTbP|qQGxj%vuBR|S~$dCB(V1+3!agl zmpe5G`8(wY{qGs(#p41cAQ^R~^E5$;r6HInTq#Qz!&0wbnQxtT1PkQ;+wxn}`ends zl{3U|H{-_);+8McmHNW5=V9oC;aK{rP!C^@^}D9G(N?IdPNr9Dy;2 zt!3#%y6_-M5+pb#aJ+G<+rUZ$;l1aWOG}Or_(yg-Ob_1=!V%3Muo~rs4hgfD@cto4 z=gji`Gg8zoj6tuu#lNX8zn#9hd;{h!DABQM^jWg}Hs2*u1JGX{|I%IGukd&>*GzRn z^=CggfnPPgiPJasAhA&7Y8z6AP#@7YxW;eB?jsp0fIqqf3x_DUQ7gN5&>`IqE8&pR zZ3FZq2)2{dQO`X|c_pHwHUSx>6mhi_bp!YkNp)v;X(HVs#N@r(a6=_`>2@o|{ zs^yR_b$N2?RBYa*k#G0K;oNmPRJHnIbOEMbnvL zWJu1rNlg9%0@Za(H=eSwh)q2pYJIt*TpGQ})a(V5vvbVxT)aophZgyK(kV7haU4z0BJrCTS*!g8M0!_UHbH-wLw7M^ z*YIX&Qo1lODGukGy)1+D+Xgam1ugf7gt2gniTglx6VSzyrbW~Gb7Av`eA^^a+Pg$a zv%T1Re}XW_K|D^uGO-+Gm3e?f4J=V!Lh0?FFK>8~bT0tyqN%J39y>zAkJB0p5TO*D zp9PJOh`T=!w+AWP{96CRgfAGT;)qxGVo%!13L0f%g|_o(^G0|@46Gc2eQ`h@ zPo5XZ9Yu&~Rc1%aqqnVZPw_uGOTp(b#M8}R2MEi$usMm4$1JJ+NIMrxhnFbV(fVU4 z&1iY+=9~|BVqq&hd1~97avn%RYwFZzYu&XboqSgdi?&qOW|r`i)@7K?dWhA)c&vk& zV_leK5HV^DH4w%)KqbBwdJlTYi7cI^~u@E*yzv-pPk5(0J(9w)+M^>p%%4 zzAiO6Fc~5uGTiyjCl!FGyK1I{CSj3c6QgLH-(oySmU-tHy6?cCa-uNc=h@<{(p`SHP29pFF+C z5BhM^?3>hX&PYiC7v^D3tS(kH92wwzJL)BMmGYRN37m*(-R&G$e7 z#;;G4-mup_B`1QHf|IKl?BB*TM8_-i=_ho&oDRvqr@TNm_wIqJ!?q&6xFe*HwfB+e zsR30Xx=ZUc?hRauZQIXrq(DWy`a*loR#g>{DpjbH4{;iXUdPa4`G%W1LG>J!8y=1kOCf8p7H{^w@Suo#B`*v^b$#^AIUofm zs!)b~=H0en4Ri2Ph=g8|?xjnZ^ng+@o6+q)l>z1~!PI{@H9>Dr+4ucp!$nrArTEb{ z-WF;%fFtBBFJG0tajk|~ZEoCfAeOzqjopJ|6hGS1n|}<}lQjO&e*99{JKAa_>hD}w z@YQ;=Su8kin2?#3D*45pz{|-<=?(%}LWQ!En0&8Zg2cljUmfA79nTy#Q2GxSPCMkwC`#`rh!3Y>V*-`AS;T zwy8Tc*I`UjY(QEq`ssW$?VNgr?Z1~_nx`gvrFu5__94XH>x8k!s3@P%5L*=(CmaO+ zFjW#g!mP;$Qy_CioFE^8Nsycz!85sGX$EEAQ{=WEOveS9BcihfvyF&1vU?GQ{?esKDh$?qw}yT*yQ!CJmKK#n?ED9A8Llkc@>9iUV@Nt z7*7H%g22Wc*raU7*PXKnQ9xx&(*TEW0$iJYIE)!xNb|pe20eZpw35{ z3&3O6!Ni9$en@@pn?vb4te3N%o|w-UNwK6;oA2B9V0<@tIm;<`eFOC>#IaMv)a)J6 z?H6^M?c$*b)XK8`p>v7D&+1sC9Ym9;uOuiLDDuV0H~n1B%s`g5r>OAZ`|q(6>O(CY zA8;=)na*^1J}1+%HL{Yi(6uA4tP8TjzP5e#>c$DgeyO8o;J~0f<8WX)?9-9{>@k~~ zMYZZDUP^@ctz*4t@+i@f-T8^O58j;vsvGu91$tfa6$f{oqd|z+G>;Ix=7@4&?vxTE z5U|+t(pYZeUJG)H-(XU}rIi$~k{*M~_DmBb6Fg`UF_=y^S**m|?$=U4>a_>R{h(-I zhTG&;lgf)X`TSJT)nk0XYrbl9S!uof76V&z&J&tq3GTWCFWv-(|Dlw?92>I7 z9I6%p83X#fNOt%4xI+0V9F)pP5+fQi8g`#=> z2cp>|arOi}nrCHa-+Eo9F-iWum=$TgR=AbKhBTmL?}aDKN&}SbTlu!zZ<+X3w3JY z$L{k9x{{fbP!iQsgQQ@In>(7YNk1ecGm6dl#p2yzg1d<8yFg}yBdWzE25X5hwjRz_ zcP6fi5@CY|B)oaJ!6jHr>nrIfmL#MqH9DlSp0F_NkZqTIEF-J<1Tx-i%tSOU{*1Q_ zYGmu~b92al1!<91rZg=T5#pPQ*X2K--_RB)N|OsKlglyQZb_KGXW8oJ3`~7YO)diI z6K3!4Pj|vJG6mJTZN=l*0pS1ZV3iFoNFxQod)iX!8guf zBk7QKmpXpC^iv*>kMoObW=fOOI;lvPze8^cYkbL_Yh)W&=s(9OZX60-h5}*vmkz__ zQHJHWhWMi0Ltcj2^GFV>G)d%o7eG+xmQrIpLu|--8B)0e3Svu+SqV(-xYcmy(9@{X z5~Zw6GzqtKassWKY;@H4I2Mz%D5rplmKmn@aFxBv+lAEp$S`R1#GW2_EN3ZdnUP(% zq<@lTn5m-Uw0XrT%=0?VO5Ms996j2qDPLc>KN%VagOwfrg@(uO7HZV0-b+$GA=|5a z4JG2wgB6Bs1W$tQW-G&EX|$U1hqsE_nOCBMTt@YW#)-={1Kq~O>Z`)~-`R^*^qTA| z>Wg$zMjlFac(6lMpJAastkm14m*hBHlo_D!H?rRmKozF&<-Q*v9G?+Qb$UxK_nrPyrOE)vec)6UDi2j))<(dJ+ zdf(tjX`uJLT+eigppNI__>;u$DE*w)kl63`1buWOY+`B42STm0JJ$eQ;6LbWCF^^X z0c=K_q(B6xG0U`)q?OZF7&^33x&VQbm)k$Rt7{IY`T_S``vS=`f;En;PR`~skjxOI zt3ewd5Wl+$&8u})TmdPUi$R=wA0#>khY6N}b}G<wgv6jPw@ihZ+#A=-wEwTCW(f zFVezqVYKc{BEX+MYbg1t@yeR8h^W z5}6QR?|?^f(O>1?gx3EAr!=1;x&YT};@4;b!TcSq8hZQDM&?BmLpjtZC=Gkhn0-C%fP4?&CdkGAK* z3nS>2Z+{ALKOGq5mrv{bm~lQCN~gIQ+WPZbe)ep|DaCQL4F@0dF-dRsSKQu2d+Pe* zKjRRw_G_{-y%C9Lw^EAo$DtJ4ros52RT%gtjA z1W~U68Eo&IabjdP5MKSCvR$1!Z*xuJ{rLy36r&g&&`xilB8IiUaADp!{PHA-G5tYO z1CcMUL5x$uYn}X!P^EuEDm4TvLEhjvwnv1~@ki=1=-%3=k4`QNv*tg$HvHna9KjBJ zF{oT;sl5)!Gx%6L`WOC1?>pSjW&Gmd{Sd6HXCjL$75YV7Uv%|jqgDS32*WqVs|0n)n*B*i6wFd;Araj4XsK*TvFpw~$*tC(jM10c`7?$%9E{u()yHMQ60 zWX1x}F$X$X5KL!qLD&z_4P)Re?{pKH?*u7sO_4(eKs6g6Mv>+C1gG_oTD64&LJRGU z?_lxVgL4F^|HOnI%o3#T!-3%R&?48H+}FZK$d3RQnGyj3Zy_jD=Do&XvU(>gZDJgIS4D;($FJ|6#MDyYB4+pvfDyO=4 zIbeB41i}Sfpg01HX2ZG%fQ=Jy7f`#d_~fhbH6bbc+pE3F_xLRvrx=R)d&FaJmo^Rd ze*kBjGk}dQPNE-Vx_QbI6;rz5H;fP|8^djU5aAN<$6f-3z7A5B4v4C$YZL#%s8>x( zE7G)QBnqvJI%?eSsBtHX`RfLSZMrE9>rZ`UQ!_`4lJhrR$&;1|gAxa5SZ5 zXf=v!?z}VE6E)>Qw*Bis(G-|vxe)88L6@wourIUK6P_EF-QNqtGlUhvo-qQ1+l=L5 zgKoMNmF%9JbE}dGKIw_>1h;51Kn~K*Mq&x%&WD9?M^xM@r{QBNXB&yVm<}#_c4Ne5 zrc^WS`{V29nc%g$%f})M|H=aRhQ~UyEq0vH6qX@AW<}S-`$C{6s0Ax*~$n}GEbo| z;iX;QqbJrriamw8Pf-t_7QLrUO#gzO<~ogZyc`cTNYr@+%K9OeXo9WffiIvfjQBW> zAbJ2x2M2+KrWVllP~Z#T)nx0+H_YS?(G9a$u7=)lZCA>OGe-zv;4q}QM8h}k3y-b|=v&^(KR;s;63((JpDLDAMo;l(v>Yj-du#N5d_U+Asb zns2~YJ9iC8b4&u7mj0e+*C#7tr0LJdWQ6X#I<8=aR)G?@#{$qlkkpR^&W|f?1NP5j zj?+3M&Vv$sgJAy%9VR?EVDtz2@BZK)=)ZJTqs$bosFIa4qJ&c(Tnzm2I0r*>rV{E$ z++wz6Z#s-BVv?IukYBXs+;@3c%_JFiTDv12CFsZ+-Oipd%{c>>h%;W_DRJ5IMgeH( zyFU)XLt#Yl1kitaGol5@-a3L*kemcTzjgTfn?Vo|5VPD=q4*-k`MURf$~gWdkV%hx z6t;OR-+j(L9q7Q&nBpe#%}9=s^+coCy^5XoEVl$Y!45agoQDQmEprsXiT^eR{535u zKLa7mcV{ii&M1V1$j1c`N~h{V82KYLx&nSiB4kZJ9Rftnhv(D#nW~X^!`GtE-x-6{ zw)ngv&`kk}_hmL<#ZA$Qgv)DUR9$Q}ZXrRgCT@AXb$#>46`27fv>hr_P+`?iR{^&g z-e&Kr1tr6&Y6cC4uJ)>ehZa{kJU?f1BaL=9N}5s~Vu*PX(owar{0ZSS7|>+0JW6zZ zCD0F~-s3KjzQJW0T~UtO9J3ABUu>9kOQR=aOv@Xr1kdYFUMBwHGC=;U8Z8;$k9=8= zpovLZ2@vH@GvxLHyPMjkIYev&d%DWVJI5&BoYb4}^7q(X%PEod_I>rxzdYKcCXE`5 z-epR3`fb+gqD$GXw{KqfG0u@@d9Q!j<;;OTB_lzT1mqbDSnr>}#_ROb)E#jl90I5J zdZCEq1B2cRRdJm)Igj3)2NE{AtoI1e!j_Z)Buvl1u{%mCX9g)gXjbB9dqr7e=4m!u z{Th^g--VCz7jJsT@i!wy9Ja}k%=VZ99 zH-RR>(Yc>iU+M|*esUv4jF72Lww!}Qp{g|kn;y>DwZn%QcW9|ZXxDmX<~%!jnG;t! z+Z+>m=_D=*ol-Lmy%QCnD}GAVq(Wi-HWE!#ET8EGgb?BifUp+HL<%B2a_Fo5lTI$!Y*(lj8RM`R4K3Iq4!nr?oFO6$f|bgUWBXNCX<$ zyZ~V%+MSfQ6#N5b6sUZ*xd!PS)126d8%5M~>%POeQ|fg)Tnp@-p)?nEk;d1s*f|S= zcTSM!@%myj>f7~7U~ohinkf&GZFHAv>OMu_OI;QUVKTFrbs&ZK?TP-o_xF$MT@Si^ zKGE+b`pps781gTv_HaJU?VK)R$hiZe=;=8_UP@OMSw&IUl!T{7_OWHpD9efERvNAlSqYoQ~8W5ES^dBqIn<1L5TE;UF8o{9D| zKWwow^hP^o|K%NkuN-Xf2fR=0{tMo}a#z52$GPg>>^@M+q~Gl=>7!ea9wyM*3w$>O zYPlbVAkFaJ9}pb&)f<#F!ylqhoax=8la4@;fP%A$Il{u{Eo9@}<*0_q%NCKC6Utdb zav<0KGj^+Ig%<*H2S5{(AAaISYZN`=3}cTQ2Cn)jB;;l4V{Hr}(2of3BFcS`MY(kD zLK2Fbsua>*v-M$caT(U=2F^9>yCbPCaaZkG z=Cy;K$|ZQc1V(b|Qs#L?agGr^N@0_V0Pt>a^&fQ=&Fr+h^h(7?gz##XZ$nsIcxR-A zB-HRI&c-2-epsG#;APT#hn#*!y#y<0{ijbQ5&xv0!9y%@tZjTIwHpi4fadE~x|XzpRCR1)(CY9il@;l%IMcg1!Xc%Rtc} znD~_PP)|HpN3@*frz-)1Y5nf7NyK-486M9eyK`d z4Y)YoE^$jCpg$=^YQr}Q}cP7f8XP}f=%>mQ#00tY1QO16r@&}}*; zOaB2JKppw_7zikM&1Iny>`V>w=p!{ec6A1>to=}pZEcsx0<)W+o>TY{y5#KVW-4g@ zl=ex70kd(9hTDL^7ToJWB8Tk(wGC z@9*Kk!CV-NRjiU$O17h^pC_Gv-EqFQpYKlm`GsZO;G$2IuemGbjl;2PLJ`HsU-Q0Q!!^<1#{t`R+Jz%1;()_)AUuhRR6;0JAW1{ z%KXCURp+(7uBOg5L;9z9%EavH#~jN{-4zPVGi`m3T+2Y|6gw`L0_Fs-uWBCFUL(Y= z!R@7>A>a(?5GXBN&Y}9Y)w6XbRVAJ@v+zn6#SQu_hR9j41auExJ~^GBOqLndGAI7| zPiF$Lz8Pp`Lq?y~{~PEa?GIqXzN86P@>>}EP5i#=Dd#c>?16OAtIJ^P&HupUGg|%= zCXd;yviq|jt*Ek~I*8w1`igPm6YtmPP@;b@0I$8|rYNaScPH^O;znF9LsU)xz~=%p z3rbI>%PP!{1=pxGcl#kyBQZ2S@0^YkD8LBJkvjC}pRx(PZ6>eq|= zv=Fd8J(7z50;*?0w8G8)jF!$+9P%e%;+Y!!m&$`>U5-en=3MFKat zm%EkRz`*Z>1mFAoFo8< z?zS6um}oUMKLU*tXvy*r5je~%29uNla9BgK^+9<4-WbaPGgENU8q(iaL8!h9s;O?M z5gk~_PzBNx?3h15DqktWK-5rcR05 zpwtYMHb{N~L~`KZ4<#6w2g4a7?Cz~L=EMsn`aSV+$AS@K^Ln3#l$DJfcZUt)=>+#e z#lmmIQwRfqRhkQth4htHppc&dCncrZmflzngCY`Zk*}MzZVShx4o{P6`fKTdR@h4} zG&rS%6}g@uRnop4UsaxxL#TKVQSYF-TewohGnzJZi->+ z2m~{dfbp%!R6}eDQWPY&SXRfq4YM^Q^hYQ^myen9gQO0Ky2_n*$O;!~$8r$P0Q@nI z4e^G~i45%20RBINzm0);CMuui&Sc=ePxdswBSUUIoN9+y;(X%*UI1bw{Vr`7{aIiN zLX`$W1+(bpAya+_@+At=uEWuYEdXT89@BERaDs!hKklqqsWC{%{JEmsA_Hv8497xE z=wO~7U?Fj@HV#=2v!r}T<~u_Shw8q<*UAhqA^ZgUE0RWU*n0YUl>#;rBuOJx7#{FcS--x=z;*q!;uNN0mADDa6eOYg?>~4O0B_1n zb=k0W3V^wBhML|`)+F4f@6Q7~z-0h#GCqL@HCqSiBQwtBvXwbFhSc1S{mArH1F}K?)_!`28MCE`6)pQaN8TDB9RU= zG!Ej-@6Wp4L$1bQr06nR%-KRWZG5oca59DobEp%(K=PQko{U6^ccqHNmSEGkYfilJ z>)TbvKYwQuBP6ig*hS^+cQ{Wr0MNLrT>LFh>L8e!&709f40NCr94@_>WJa2H(ys73 zd4^w+$$wRgoUCzf5*s4K({GzXQ0uBMVXvp1u7|F)xrfa{IY`eSK#-6nap;m>x_~61 zTr;y5f`-EG(y`PwexBm_-Gc1#P1@32xT#j2!`8!?wMhS7uFCc&e75`j$S53JxmQ8{w|TCMM+E9?8l01n1gqa1Lvv3iE#M z)<`6=jQ^F3N|>;+c5CdF%l`4c0b#mS3w!p*p%eU4TFPD1w0K=R!$c=tOW}JX8H~f{ z$%PCb5jKQV3fe!hN@z3TSWk+Jc7O4<`8S8pkH>3r)N#G}a9!4kwFrm_jCpq8pt-Tl zQkaI4Jr%_^p=8QlG9`7OgHPOa>4AU?V*%mCdm*2fg=2AR^H~`wAt;|P(^j6{kcn2y zj`*O3H{O29MO>oY53!3Y4r=$}&q)reK2*-SSzN=iXGw|#$DUTpzPXj~Ras&|gj~$? z!VOjTCd**U{0-%_(qQJaZ}ByzAoT3T4_AnwZfm4ig=$wLyV_>spr-nbL~Iv*@gV-d zs3v-2F^H$jtNz{E`DT-oPH9c_d zs9;vSKC|S4RrbwUFUy2$qpz5knoxQE(>H6|cQqN}LQIbrQR`njRW86cW6$zE z%|+LV|GsHko2)X+W)Ikge;TusDE$>EdmZ;T{MVfgarhKf)G5NRB1N|U=CzX>UGr4nPq&GUjk_jHCZ&jm?RME$l zn&5QN+8n8ia;DA^2Nqa`>C#H??Zs!03FXfDM43sz@qDhkE|ZYj5itx*Kz5CNBm;x_ zw<`=d>F+CwVHsIm1!yW4w?%ad8mFO??0Stx!6MVU(@jb)@wh*L*ohQ=#D*!Z8xQi% zpvH#>95?7)A=>C?&{;dBc!N)QnR(9splwEKY!7%$g$A-^FKSqDzatluk1@xi61uQ@ zIapo@DBAUrx}c!^7KU5O^h5OaW2b~hYxecufjbC z3YFjn7gxre>0kp>j?wZ1l*w~G;FivZFQ)Anty2y7vEMz%e z2wYr}9b%iq@c>B_gab>Tgi^B^on# zhjZd7dCZP=2`ibjPVB^qm0tGeI10cfIn1adjJ!!qR<2_uFugP6-AEXewTfO# z4~osRRFX@5ftRrSx_`mz1*HvnK%tq6-^aVxFTpP>b+dcUXIy4yv!9SZ8Frw~cr28P zy>XVrmLSRH5tden{#V(d^tf3;L3IM8+c@izJjpQmTHDO$$1df{0ZKbVpZFZ#8-WkOXtTy1xi zeZK_@LfH%RV<@*c9_W>qJ1STYQam}F6+?3|xWL&2f7R?iYakSe20a=K>}>+$_R- zIuJ+ywEp!<`}kW|2P_`#LA13C*+JrGMV`D%LI*SP`b?oN=`Yx`in;TyX}u}GWI;^E z_fFZO6hP~yE9~5zUe72yqC<$iJg#*QWma8Su-yKVxv#4&#Ux#~#^vcU`@+Ka)C0EK zLiti+KkEtI#B}XD$tgHQef1Bi8>x2kiJ!mZ-M!Ww^G@;B7!+k{e7dJ+cJu0CQ!oB; zk<%_YX?=HZ*Mx(q_C&fXutZ4uQq*gQzaPi`K@|Ps3zq#cA!W6L zav5{c;>D-lQt=K>jZuKwiAeEC@*eZQ470{ulWTmk&Cj-gROL^bmx{*qVp_p6(lW`; zW{Iu1IfHOgdfL{nxo$I!y>B2sE>y+z8JxWqM;e9V>~$UEhD*Fu{_?f*X=8}0UU`F~ zz84Y%e>uzl{U5*<-G%|5gvsR@R%jGmJ)Qk@V{pMp!5F>v3dFlDbfa2t{>zUKWex-)q(eN^3Whh`94HMr*#rbZC*#%Fsa)q1$ZQvcfOO3sK>?&JlOpvkvK zJe6Vn;a1ht&)pUK70OQv%#is&6>U>LN~&JjDH6-UW^Y02=!wPI9jH{UxTmvR=mUnO`8=nPHD6o(!}xs&QMx$0?U1r1$us^ufwdP|6|z_&t3S~| zo>uP%$hTuPX^&s!I^sQ0s5d*bHp-FaA-@*=@`R<_Z8)>uMo#T;F0gy+jr7OVz2~CT zSs*^B0}6e3k}I$1GS<~z4d%R}ebZ_P8{rcgnOuiKQ(?xE(CN_((@!Ps^!VEwq#&t2Dbk1_tW;(U7o?LyChJ3z#9h7((KJXeCd zMxm)2(aXCqmGmXM>wN0-Z`QgtK|KFHrT_2oJwO`BhGK|}1t7OXrs<+3G3Kk4(AiSZ zFS8&9;4LhD~20P2=-!!{3R6DT~p$w#3IP@zVym%xk$ldV~NW%{8$%jm&(*-eb!sIV03jq4*!6eisfWDO`TY_@yX#M-g-8gf>Pu7XctQ=nK8o(qsS4hoUK)eMsbVj%kT~kh=>VyBF7_5Pt0@ zHRZ`%<{2xQzMu1zhU4c_KWYn}&+6t>DmpB03?|8u4OpCzmk zi}o(CG%}I9wS3Gj@@Kw=b}i7wI%=PUN0D+5U4h^@EK!=XiTW;=*!Ds{J&Jjw5|faA zo&_<00`PnRP{9otW?VeysY8IR7A}z@-l0r+`wq>Waa14q)q%^TT+7hegeDr}2}+WE zag?- zzKF(XOEVpj#O}?wl(eGWEKS_$wb<7%&((cGm^tz_Kh;sL%eh&Nt|3kSAGDv?$neec z#(1hDj5(sjBpRf6s$k5M7t6@1+Tzho8FXJHB}Oe;7f6sU8q_gYb1%8Ak8J`!D9nXd z34h~9;v1PJhQbNsKDnwlJ=R-I+vJZ&~H#JGmw8;}7$S(mW05{h` z^MqIA5=KD3AxuohZ^;G~)pG)2=eDnDeUe>&?t`K!$mSz)DlseDtQqevbyTLl{%eneC$!WE| zM6Dq{fezJ6d1<6Sz$W0c>YCSa_WHMbejqUe<)4VGruuuuoq* zKCv1!lqcKn4pEU8_zi}QMV*eVjOh8(@qeV;@yN;(W}juntPMuz(*`D}+C)q`cgr6S z)&4spF+a~FM2T}9$G*d@9~431MpxT@ES~bUZo_Jp$>qy359Ol$6;DO3?#K%bzd&wE zknyWx;kYK1sVxgMtf$`HL4VS~7gIoKa{{zsOUfD z@SJIE!l!V6Nu?Pv$WKVEXO+C*#If#o!dnsg#kGm&Hyt@w%n6w#L}AKv@1DD&l9!^nX9rcuP2S|-No*4R;|4Jd^Wv>)U`KMBxM zM@#jVoFo!HK-=&|Y z)g#zrpWKe*Lg79lv^Z|O7j9Yup@~;T@Qm;Z-og}f&=njq#%906(Yym3o-cFBxsf;Q zwbg=+%2(RTw}oka!x=ZD{TFOLl(U}o1_JcsG^;Ul%j%wQzmp1_dyF2!h;>i+;L3YI?(bBW;fUm4OrjFGt$c6Yj}NlKg|Ov*4nhB=7Q0m(A{!%}0bZ*|t~aA- z4&RGpJ0E*iqLKfF3#fd1gUNI_trsi*Z|%6h)GdGq4&{}f+>yMNNt(VIiY3Z@cv0|P zmNwe+cTdZ-aVk#_ypAQ%E}^kn$UT+Z_}4EADFF*04XDvOZDjxFnEZ#}-raid?yhxe zrTym@+%#~&cgs)rxcu`7tE50-l~-SelKZz4{`-%h2mpV!i=&SJ46(|o1NWTk(?^~E zfa3qVAws@&BpNVl?3Rhre}IA*vhLVmGOMNxyCCOz=Qk1TRsDy;cccsc_EG=RY^Wfl zNx~t|Q)LZ*Rg?aeh|HCH6A|*M%>4zPa{do^dP3p7cY{vy`JUg&ZPP2nVzh+}QUv{> zk6#9uV6T~?_^l*oc&u(yEndeRZYV1WWsvPZ%gI33{+Rzo($3!T!J`b*1@Ii z_+KsmQxF}Rm&e=x#S|?ROws=Dq5ao#%u2zqEL}|9`tmR9*!n!0u;aJewzGw?yoXDS zehE|GmHzAR|9AJ2L7qOmTl#s5#A_TAX-2O|aG zg5Z>XP5*ui$hhANRfSl7ZbsP9uyex;tpw7e7t&l&IHyqA;Lw3C)Gih5 zikgF0IiH+Qc?yv;3MpJgFf#xgSL@NU1^&~w9|d{@)C>X>s3qxeV#I$00oGdBMxU_# zSjzO9{Q+}}SG5zi;9$b)nv0+kJwS09Je3 zvw8=6A91UiKZcMaDiZ+oS|6A8zY2W+FL&eb{bNcnT+cw|c0i6%+_Tf?o3yXKYX-zv z)bodNc0P=7-yA)Tn5d~MSkXSZ*A&xLP9jRZRAvP;W{7oNn7()Jy?7WA3;X8F!4dMy zU8jzCR9?c@luGavrZ$5v#?W)-DIn4=0F#=U^t-uW+6-k#eJQjtuDN2?5Y;$}H~LDf z=F7d`Ttff;ius=%*Z(Ih&)(OS7CLCm^q$b@47&ke6jV4n!~D!YkQv4?Tpdte-Eepk zk#b&ZEwA8Lx7`hs-=rFAk3K55Rh@cC2pEiD8}WJB``QP2cS3&(*>)ih!dZat25_WW zj~!xr0iq-P_uBf-P;R8bO%&b#dd|=UkY4$Z?lbM+ptcKMsY@OAO0@X^A7BNcfPSLG zt^)8FUtZfgfFnDBraRXZjvKsHEFP%Qb}DvcfV5*75V<*8bLNXASTx5lFhIB8)@n4I z1%WrBasfft(-Er|Pl;wDEWjB&c zFrDESNB()1+z!xyv@a%GkxTu8_Fo84x{|tXW6$(4G%RKnby1iauyUfpmig>pxS48^ zpcAUMC`zawmYCqI)$+C~s93D?%vXRPb*CJ9ggf>K--FD&pSp{$J_m7O;k}q3^foP> zTi69-Y1H;2AIB7R&aA3FV2B3-=7L4T;Xig}7J`Mcg+hhwK_7MqY9L};1?$?$v>+z# zFemH#HTL7N-wBL{+(u3hM_#?kZk#IODyQ_J$?rraWa#2W%SaeSfgN1%Sx30y$RMg_ zhTD=x`6x@74r;Tdn9AI{2V^vQVBJRhLR_p0j0TaZ1mc5(h_zlotG)rzqB0JMR0Vj> z2~mh`H(d72-NY&5YGCrO!kCLtn&NORYNHMv@6*ZAz{wAu%KfB=#IR*R*c|m2c|lpX zF&z#+s?0yuXX4gA5F=LtMl8-mYifwSSvaM;c<1PACnJF#3T<1XVX4jTm1tX`Z*J*L z$B1me?HeK(gJG{_)my&ya{i}oEH$|(oM`AJ^Mn_K2w}Wj4--HBrwq$L*-+{eifw2? zV|yA1(e^x1(yNdoG5aetLCCD_8MeHYv$TZDirE3#L6rV#ki-x0w@2N6@(r@a(dpA$ zJAu;v5S&q(DFoNJsC^zG_I6ZnM`pl~QY>159%H}{b@zFhAShB=yGIDgFw)I&>Pt^8 zC2pW`-s^Jc*M0!e)5f?fq?&vh9Fj^>*w`UhO~(b3l#((l6X0b$^zM;2$K#Q#qo3R; z0pQuIH%J*)0@$zC()lzyX}vHfGL(rnI{@q++RqR#L7HPX&w>`>N=B-QlHy^_))^rN zxJ^~%GN0bNU0@a|X1ZG`S_S_kn(j-Dr#!sU;^B~r2>!7-DanQSZmKm)d6 zPNo}-TGT4xFYl{96AQXX0RzE>T)Q;wpYPNfh+=^>lI_Ywhvir>8kP3^q&tsXDvCVa zNd}w52z!FFRKXGR`74lX*|X{`hO2z_9FnD%Y>)~vXmFe9DTt3a>l(5icyH~lrk_Rt zZlIr8r;#B88>G1$u#+n;;_PVB0}0o+bPlvE0ffaAu`cG&gQP2P_Hx zjq2gCfgb{H%gqbH!rJ1|>V-D9bQHAM&*&naE*iNp$b>nJROF)?+tIT z(DEwYn`J_mWMoXl_|iS5f=OM9`T!-D z%gLkgFzo~Eq!-OmAHD=wz1bHM&3W5}y=R=t+#6neFvK+yTKNH2G5-t4q%1!U;fB}5 zK_Z?Zs9IDNyVE+WQa0W@{70qoe_o*eOjm(TRzV-XybSN1`C@|T3T>*U?_C#b!N+v8 z$jloi_c58mz*aii9>@UGLjKahu1?5MrznU&U`x;CsDtbo$~?O?6i1-VnGsLCf$})e zoPog`P7fxG0Np*W^yBA#>V4i#54|w3sJ-k!Gzz4q8BpMV@Mle z)TBBrr6vi;{6w5*Ma$bp5U>_m+_nGOW=za^O;M$z4B0l#(I$h`TB7oJw zcPjb|0;(9ISHb*S;d-AW;=(>5s6m6uajf0|o z#OfX8o_vidCUgB}d?}IOu$7XN&o6{MJn@4M!qW{+x|^WWR5Gd{)=Vd7qSEaBJS&lb zerqhC^)%l^eAzP9@ zKc;Nup_`_kKy`7uUc;y&4plVNS84vraM6S2joT17;c_T5j+Y!sM-KP#&U4#n+&3j0 zbn1~`+LOX1Td?y1^OX{F`gP{57uln69}<7%1w_c2&}tesYpG{`JC%!AND38VXGSlC zPfN$B0F4W)EPa-Q2xp$&yAqwWTzeX5KSm!al_)OJ?9UM#_E-wh3ck=g7 zt(CEo;geW9@iD$UC z#~$-pi%!3NgIlt)r0u|Yo8_e2X5ei@AgE)<^+hPrLA}6%yxvp(Iwv_+(CEsx{hWZX z%K~xab^f35){N@n@P0A7ar7nleVro8794{&>&JyHL^MW*{E1$f%wc&$Ny32z6MDrR zk8zRDr`o0cIrSBu=Z(QHzfJ%{(4d7}1i!yLlIpvRd=TbrUUd@{8RSuk^K^dA)5pF> zXzo-z+Okz#s;yiJGmpkqW^cy{r1XLJ{(^mc$6FOM4ZEcs$LN~k2lV4t`+S)Ao}x)J zH4N1kU1#4&PKdC#Mq@F;56m@!(XLYu%5H-N75ei-tZ+Mu zgM#T<2hLmP!NWp^$7vidfxUTm^Fg5s2qf1FJg|GayCvtXK($adE*q6E z|4RQda41|Pd0VE`)}alN!7h!#>>uLp<8|0 za}Hbl#vvj=tgrc*8A?0^+X%spZ@_K}B7bPT-U!COW-K2LvB}!BU&bxX1IQ?w{)6X@ zp3__IN6c1%WxZbWz3WWxt~A!LeteQQ*{^jgVVF!?|nrU4j2#A7_#VD)9>R zS)f1M*s%pT+VRO*3ZEZm*~ra4*MmrI8Ay{QUBO;P_tHB0vON_?;9UAXGS1e+Yqm3r zf+A=vlKYznypxNkfX=Fn#(o9w5jMF{-BDP+xhd?22gzD5wMqpZd{Y>p5qM%n@Voph z1{uO`>K&LJ^2v2v3FkyzjDv*|WlK5X8n4Z8cs4F~9$7o>6I@^p@1c<5uUgHEq3 zO-YSxXbtfN6ftz}Af!Lq>EeCqX@PPfXFA_3dDUylA0Plqj-ac2&c&@G+B%N2d=`sv zosryYGh>~RR=te=_)wlLF6R+UqLWhikoth|BDht`Db2hKHkg}wyF0N4k|i372(D@x zgKFme*I!v-4ALdPo_!dqQKQsiD>hIg;!@34_fQPSq?e1(2G#DTcoOWFW-kl&hyI{vp-o7((nlzen21LPgP5xX({NY_dgB)-~TVm z7UFH}eiY3|*y&RrM})-~sD?b3^BKBWcz2fDZ2W=IrWOk(N~TfEz+H0`-#jqtWl5!+ zZ$h*{zoR{D)nIL4-!eH8j1I!Ro?z}XmpCDOE7*-Fs4zJ!Dt<-6MoB<-`1}LuGd!?? z6r8u&6lU#GBF()!wGLHgJ6Xw4#@Nl3kj`Iy1LGtm&zK3zIBtt*$Q z#Cx|<^=bZu`fvBmHoJ48Sp`d1m`Sa^-u0`Yn0uiy@ou2t3Vv0QuFsQJ#viz8~gs@!|yTux7>w~JnYUmsM>o^!gr@;{~J zsXBg5?(0BLevHLZt>Qa3o5P$eBOs|O#ZBzjfz4DC^e~HvMfU?4-aSl8+tsw1t3{%L z0r3Ioc?zMkUoK|Wi3z;8F#067ZI;KKRC{y9{d3#T!0@0S_~_ZMv}s*JXcG!o$k+BX zM%*8=@yW;lY!pR#n+t*4b-?2!^X7P7*aLy8`|8CkED%BH4 z?ZiJLy6-`%Tj%>*kZs4+F0Cppz0uWINWpIrozy)r*Gjq%@boGeg_7b zRaj^v34`Hk8EKsuL4}@uL~jYNvz$<4D5x?dxXO`g3vatLl&;_=s|WxfWpt?Rk^_{` zd7bg}VPB(040c2Gqz#_hwL!196!q&Lt0e6!ntVUU_H5G5es*7x7eh-)p#OZo4ZG@E z#D*>fQ?~wbsIptssSiaBc=>U!BqKC6PfDUZB|Os2GLGIx)Ap17F42nvbP7rA)*rHt zyuZ2`P9{v}r{y5o7l_VpI+nS7^bBKDP^ZvD(nC#GBTfNQ?wkilJHV6)liWRkWzfD} z9z;t)=9=G_aN74bUk0t(Xf8b~=BL7XS&}ntc`D(&GMjWfwTx>kV-?hRx=hZu>L^wt zptn^iukxh;dRaTiBid^Q;a3=I`?TynerJmC(lT4seE&!jn^cy}&&II9ZKZWKTByw? zitN-v4s!$njp7vI#{9MirQ>WiQS4trI>NGKCcqT{G3zcMe(X{K+c+0WTkvG(6W=G> zw3l4@hI3P@`gvmCUy2Djq0^3P-Qc@_bDA+h2&Q37-`xQdJh&J&{PHuY$z?zNMcG?zYAO7Ni?T_2~^=6z7kshJ#q znp*2wC^8g%0iL{c`_Y(5w)^ZMZitxj@3+-^VR~|4~ui zcr>5;x->G{=ECtGWNVrhavV7|=%`o)=;KUj8cwpO-Ne7XeFk}b@#%RZ1)nX!@@SDm zM3=3I@n^mFJmja{BvNJg?9QrPxqm)l&71IiV#F=8{QX4!{U32@oz_O#{tXN-7<@FIX^Ei@0Y{{3kFeykDnaH~B>4gZ&mhzSo) zu`&KzjDJ3bPM90p3E92MJ#Ccv=l-i;z{9V}9`YsNpHE>9Bv{-THTTs1+{n}ZQ;6oT z^YSZsrkKqd@vC#|zTxafr^@0@n)X5b(=_art%ZhoviH`~yqi`dinMA?(ew$bJmt z4Uic406+%GG*6*QC(oZ)7aIZ%PsidH#U$eGS;dKBp)z zx;L&H5HP(Zs4-Fc=nC(iO_C;qgaN&ze z>u2zDg@11kYZSY$k2xbALtl=;usq@SSob9VIlK87uJJxqwr-9PXeF@hXiyL>uIQ0= z<(~rH{vdG{bhF^?)&R&OxGf-hOkuKe)svA|Mor+<=lrlVPG`88$#qh5u&+=x=NS$K zN})n!hANCX_L~jFWC+1D;GhmV--7hhyBFl)8KyLzdye?#4M9y9bquh=J4h%>U?!s? zc?b~EyW>N<(HJD+2MP{eub<|<;lX#{FO&(qpF*H5#DPd)K_A^pJQCb<+870FlL@eZ)T(^+n`( z-)y_i?%~&%tgqpl(WaHIFm7-due4~9)R~CsqouVxfZ@8?Zv8b@MYIXXK61mvA$EN) z{p>bG5+0i80RMgpkLGSEUvsqnNMisxeM_^*E5fh#k0Z z+ph`1L++0LT_HQ<^|(>$0x6$~c^#)b&|SUromK4{ zoJK;~eXMVoCo$-$I?2B84{e6WxUrTuqhVk#`+J#JdVblrEQKx%bgU0*6?dIWYk0p! zCJsQnv~)OnA&40-)28QaIOa%1=#So(&@8? zLJof^?6F|s^OuR6U+wC&!+>cKCMEOqJL^g=@d%HcqGcEb){pOioaKYK%ODe<-A@gh zAYS?cKaQ92J>|_tG!Xv>yE^E8wF)Xm=Z9M)n>`@^y~9uH{{i|A1KctKqZnTI2;$pR zyK!glJ2s$yLcF&?pQwL3y)WenC|ziN%6gC8tTVL=?WzotdAOW#=5acp<^2E!U6;N@ z_(yPd%WNP^EO6{@V|!PczvNt+AISkg*L~eR#>*QBLK5-fCT_&z4d_`qd3Nr@K+&OL zB_CQZ^$AXk-gt1&c}_k=&V!lH=66(G5C_w<_ztDD*Z#U?VH@FA#{o;!Bry1`u~2DDWf{eMh`YB$4FZB9N=hAzn8^`MMs=UT%2J z$E*}B;wq~`+=1*TwJLx+bNJR~txLCJA9jy7ufj#3>0+{0|8Y4w_2={xNEM^uHq!_f zlGM3O*4Ztlo(fOd&f{}ev|-~nReq>H-e#oaR85aFrzd+c@u8k+2yM6X3e9m}^llSu z8;I`V*DyIpEo5fpOGn;4;*D@);Wp06id(0AeJU@b_^$Dy)EjKy5IUP+w}=k`6fH$t zUsu{R_f_A(+;<+mI?64Ahj#0+n`28Sm~G0EX5U&h*zXZZ_!Xw~ zaCcpHDNX!N$H1~PYs7(j#s>@7jYL59R_`X3*+J1WQz=-FYt`{|i~{Dz0+h43qvYXk z5ed-SV{THCjq{9OAsK=<+trl`0EUx>xNRNxOGSscM>#Jmo6_UwckBLy#0!a@4PyQZ ztz0_^?#mo#WX9A6bC$`yWBm4G` z?m2}3%1uYyW%R;q9)UzeVOMvl2QEF2N`p??TEGFr!s@_{Gnj98%&|A63XelNUBAFdEYbb9u}&r zzPts+>+k|uy5{3c>jxD?$P{>1w!Yd(le|P=xBT3-Ly>3P9IVi?C|WPeZ6c9wX;pP% zp%QBE+n=#xha7b8+IA}j{`ROf}YeiE*Kc3x*ne+mn zxJc;baI@O$4l?}!TS}sZu)7-j81}G;dv1~-wuXp%*3}&(U*vMEa6e;PmM|Wj!>q)v ziciAfEBB)!3wzurDxK2LTBI}bVyLAn#ahKwRk?2VX79#0aP$S=WiZvVOtu)(+w69^ z2y9svW+=4KDFK6}+)+~y-jfMU7M(+$grse@bCv_S*icgXpJSo~tigw94tz3Wu4@G-B}AbJN(-6=tSoq1b~8;jdux$s(Y zx%(PA!!zk1A3FpWwx_?8@KU|AH}n-(SjFZE?VYDam+6jkB*>1>5uPZ{gPs%7s-eT-)pnRK3)ZM&MLJ(9}vg|ecGB3rJ=4A9)yTfC*~V( zRRO_U6=Eh@yUYT)?-9+r_I&gm1|ptto?K^OJW)+f8R#m!g3PtFR@8cmy9J~Pc!QK?@_~#{W3gT{)dADjv9=W7&36L$ zH5yb>Z#C@Wy4rQ1NB5HZG#042K5(4+EdH}jqrov}nsA~{PV}vzcLeD9vIDiu^#*I4 z{c{Ag)*~15zhm7dOx6;bZ;$q=C-y;?nPNVBpR^f_Il5aj#^pb`8&)pmJFxD7vqWdt zUs&+^3?Y&!2Hjyh|6*ElsK)O(wjJyL%fLl_n z2+B&gKj4ADZx_}q)Z?(MgK} zA~JK$Xzz{&>$wd|Rz*I!8}nAxuZ-#t` zPC_!~g^cL4vedef$#M=qd2Js(ljX{}fL>{Uck?#)6gNr;pC-%Z2aHYoYneSYRt4C` z%Z7Ng8ik^wE{>3D#0gq9y?BzfN%3SvZ{jPr%@m%dUCD5g2a)~!UgzhhqJzGn2Urh- zqZkPnC7(TM$D4{BAmsY?;l;TF-aUB?vaL5@Y!c%}RK1yVT1RmX`D@{wL zJco+MzCU+h5R;jAFGcmPlva#`@|ioe1m2$CHFj|)iwa*F`w5F!POEjUTuN|g$Jbru zFOBLdjwNuC{Lt#Xvrqj}aJ4XJR%%t?c=q`j#^Rji=*B0ejI((FK5~uHn49#QON!Ar zt;(6ml9~(i^!hzhU;2<2QywQ#aPz$!i@YPvYMRwPup2^LhXqe&sjp>wM)HNO7|#na z#XSCLz_5=-q~B1srFQ?J&wS`ng~3gg%G)Q3-IekEZwuSr!05ZLhi=Pz?wg7QF}Qp% z@+VbuK2m;RX7iz1lxOJI3`C^`vtvpPzS|y>KTy8lB&D$9fkA($dg_BLfKJM2EU#S> z*^M(bK0VC%Dg>_I^uAZRgRSC_&d#LFj>$m?TDp3yq_aYU921AUH&b1U_hRq&2o1*3 zxb^8K(eHk}8L2K8N2kmP-+0dGtR@o@4DWI|5)_bL@&KVtjsgoEs{t*Go9xh?FU?_H z`(ZPx_76Z|kO2vK{oU13neysIop0*2-$xyXj1ylH-V$jfzRXrOS7TeD*nxkG!DSnY zHiZ^*QrC1Icp%lCI^0%NlF33E@~!x|8Cc7Tjk*Okw$v6&^DJwaRFh;#NHz|>pJ%(5 zLmIoi20C9wVRK&F4Hi;qLd)IufQ@Z~oOTZoMcK*$n;=J7mL?w`Bm)K3Z#YGm<& z=GXLoL49A@OCeLo;p|zq=<=plqa4qdSW*x8_fDx#2y{7Y=&2*er^W0Mt zjJ`ucH>J(1IwQ1+`r;-;JAN90FoMu2qGHlK-qfvx%-zDTQ_p@~WPaVeUM?2@5%KV~ z8?@|eItK1t5hy+~LKY1fxO0RmsOGE=#OM-9XaZZ}VsPsvcPOKDTsTLh+b zuLH}${rHf2*X)H1UBHF#J!%L_S*lH2lt&te#hO<4KR}fVfPxe`V z?0Pisea&==GMCTxqmO5H@%UXX} z`jGA76+7D=hfZt}fT`LwxVg>z&?yl!Q|6)I^JmWq&Jh?^5=!#Sbw-4J)Dopc*Bha?%bAbKZWy7{DwrP+ zTTl2MFMr;c$4oxghI%LZGs6i9DuX*>5^9&tY{4=l&};pwjHIQ|M@~~{JzJ?WEoGQDk~W&kCfx}4y5EpNtlXjFD5Klb&^GQg>FAXOEaJeK zwKh96EX-pW+Bt8z-xj+zJtU&9{HQUae#B1k7UTt;dW9|B{G~9*j3>ZZ7Hth?M^V8~ zDW)%AlWxPb0z9Sr5}wf7GuMup5>!;QL`~4A-ET#(`-4FsaaPTUZL6;}kvZ0xG}eV> zC%%q7fSpaY`@l|j*z0WR2M1Mh$K zGKh2%_EBh)nv{|e^kXpNb$}w2YFs>=sSVgn(-m5jCzUGXD=Xm8B^`uLNpr^@SC<)j zP~>(UwS2fVXkdF6|LSww3VJmB%gLslZDEzfYENi^1*{-&YvF0U(lZkQjdImRPW}6% z)jlOVB#xiTZq#2%{5p;Vl+AvV3E!V>e`^mp9q9Hdh{Wu6EMg0^ALOF;#rOf3BwaHf zAWXi-+U|`#=Tcs!#(;ClEXNV4HONGLeXcVt8QJ#Dp3T{fIdS~qx%GQNuepBe=86k{ zdM=NOiYf^Ogot1sW3^EzOEyw+aGM3nz1z=lk(_-UFW2dJ$5c`Wp!od>G!%~?D6*p% zgHM7j{t5R@K_|5dk?Ag^G8JorBK+q$uIJi1628Ah*UQ_oe+F&Ak-NV9Y`@ila1RwHI&|64WWO(M-=(QlJ&P#hb)8 zUa!Y{Aa?lk zb?IiBIL47dLK$0Y)X7fb+bLKV?_dVnPEPLWVqti*tHgP3+!ABb0wv#~gF$gA)*Avr zHyM6;S%Yx3d*pFtW4G9rRYF0Ex;9DdZhMdh1<}!iz#POfYCv?8|H{I4CnF!iIuxq5 zFJMmlka#Q`Jf3`fh41SbV@CAwlZQWu3m&SxE3V*Us-U2JB;;>%SxB4_rSP`?#c6YS z6-6{`&6ZB5_v{@_J$69XoQ7^v9UcEyQPa$lFQ&5^=WH_K83m5J^CadpH0bjV5L$_pRpsa zY2M3wXxJA7l^Xl?L>&yU7yPjQ!IxJbtVGO%5o!L>=14#O0k&lm*7XVKZGv0)XazNP z6SwrAUw80WLSi>sZ*)Z3h^BX$K40(gYPKfgM0G-#t)hP7!>ozt-^DdjDqUgDk5~G9 zm~3MOmxw6P0#E*3pu`}vpBAIKU(vg-lPu++&<``O`#repL zb+~v?5N_J>^;K>Px+gbDsOz>rj16A`>it*V&4w%CY2moRnXQ+6 zhbEt|PRR0a=%7a#Kc=E|EFHc6^l{K3@lPIwU~F55OwKc-C1KjZzC+&il^bhZUMjCpPyApn+fW)W{EOFY&dVq|3>7kQu(^rz^tUy~ z>HM5gNbDOo9utxzsNTFOSw&1{RGqfORM56gIqI3q5GSmWp^BFz+|Dy_{*&pD*j&X7 zC8Kq7LrevStq&Gvq}J}-a^iGBn`8QfR;`|Qs+7MrT4L*h&3e}`nRxTK&Bc#4--o@D zJ=y}uA5Kk~jkK#`2p;g+4yb;f7~N=6LMh!gQy-gaWj)78FmnY5x8zqjrj*p3hXAbH zNKvcH2SM8F6~=F+@lyzfa$Ol`dPUf6{CXmEGiPG0EFSQ*bL#sPjNi)N*rv%9<|pge zQv2kDo__5?tZPTg6+)ahKu-{fB)%PB!9M&Tp_;pNL{|l!ByeWj8n~s3=gdTl$o%-R zh5iEe?%c%+pE5JQYt(xDAG6Avx7cr>)vhndJ8l`?4*lpkOe|^}XO1gbj;UY0{^hOU zEnKB!js@Twn)~$(n=OI)mgkgn*8&&A7VfyA25O<2{-U`tKX%FpbI!EkM z&=gHHIKMG#yyp$bjcD3t|B*>R6jb8=zJKJ)=@Obn<_`qtYJnG@MDd1RDXsfgkM=;s zH8vKoLrxcbuvp`8Tl46mFVMWXP}LQ!Cg5~Px}+Q7`W(J6O)v%T6~|xatPOY`q_OTJ zc8p)R#UCDTwcgqT5vN9@?Q;0OKELH@@R@%no^|M1vIpM;BS@_d--e>SS9$zXtXJ*8 z?%CcqmeXfT62^<6+otrVjwAp21=m%8L@hZ6zvQ3RaQ}ruTL5m%G-u8B&mZ?*0wC>V zh3eO*Mq2!Z87tZ1Jb+`R#5UUPZo@b8hd?kudV3g9xPW4SWs3*^>#hr6a|(^|67#uQjzP<1%d2z> zp3n**Zp9D_KxkstJ6+GB@(zc{BrNJ-vhx<#I!vatQP`F zOLQ!g^K_p27&*twyTa;V74geomL*8?Ze8(iHX-?Bl`9J&x z50OWAE7K6`%v6w}<~< zm@xk531PES`O@nq0x3P9bE~qS0|s|ZH&vKM{&2S8LlA*gSt&DBq{u$o15+mhAg7F^ zm4bKa_+#uPi1@3RpF`xEZNY%4XTb*j_E96z?A!1tf`A3+`sO zPCW_|B#%7`P}s3w`5f=O~tN-JG*tVa3ZRn zs)&3?AdX(y7t}7Ep9u;|php^aqg0~&L#`lZ|DKawQUE3$X|08`+xD+(964iwrt*B4 z1V`)>q?ai;fhw2+Yx5E4nbQoX=;c7(E&La!;@1)@&=qblMO$C$_&c4#s3!?M+IgCj)~84cCeIy6PYu^afI~K zU$auntKc(hE^6o7`8qItB_Ze06rbS1?nkZ{6SFo@1Mf0pM)K7qw+VjZH6#&R?2wnr4$;c$Y`!0Yk z$MWV1%QIJ?Z=S!6^NK`pHkjJ<1ojEC@5XaXc;pID6@r>3{{(at!8CKw`^gAT!6+`A zeYS8+wz^ZVXqmHy((55?p$B!~EC*vimW@~Lb5IyHBIcq7&T^OCVP+3vWO|Aoch-0N zrY>2_5hEK>9tCIt2Dxs z{QB;13X?LB&65xX!sSg6LE`_qhx60-b`wv-S>n083ah&Xti$Y9AuaB@7)0~d;*rP3v78J-op+C3u$1txm<@Me&3<_@mYkrva6CjiI}C+x?v0&Bo|Dc zN{uo;kI%sDp5p(^@W$P|;Lxn#<$y_>>36?DS}b0p-oTAU)B9jZ+P7}0z;D%m_L`4) z3SW#y5g^@=@^*n`{1nGz9&k){(MKGUL8%Rn$u4*5m3h=(3@sPGEDIH>4|}eq8dHXA(35^_5248JzgPhq= zr7;{jF*;fu`=!gwC#sBf4;3kr;n2Fbx|yqLo#lqYij;r1ce%y?ej&j?sEk&1T{`)X zWpYxxTQ6|>@M#0nQprn^w639=cpFvG*RWhA8WJ-8+cCKl$UxwjoM`Ovt?A@=Y@q4( zBe+opfNA#VOEAb6>G(Xz7QSTKC>F8A5<%mxxpv;Q3xbf6Y-DG(RbJj(TRGi(FbUza zrlQo5sM(5VIS~lNOsx3uEb-ZMn(Cl%mnI-ghZ&cRq_l2qj6GNif-kp?1X}Jj zM7NJ6T4_izsrvK}F748G=#I?KRM9IgTpGQLBdqZgmE*igsWb&=*Wj@sx1lN_!R#v| z(V|&5im(g{)?vw3o22U10W}9SO-56iDDr@-p>f$QM0)m{1AfPxwUlyaaWpN0D17-s zu+}tN#pijZ;Ku?DW)sKvHyD&Xaw>+3U+riCnFr{H!ggDa#trrs1 zWWo7Lo(_f*5+i0YJbBR}BN4ZLZbIl%xC?9VPSO=euuWcMurfY~l5227omU@tYJCSK zflmTYU?#z$ThbK}S}TR8l>x^iP<9=3K))ce2QcN7V3ZF_2XTOW3-NKVzz&4A1UaTQn&eoGt%R`$n zqvxA>s%-&Glz0fIqA7A|_p@mk?=?<@Y6sU*9nMUHkyKTNp_tPFMaUg0elv$oh7YnI zB}?>VI-6J82?w7!IR53F?2RI)O~k~-kB=`Ke2mUu5E%jQcTi-0l(<9aE>FO@17Ty6 zdZm+tvZe2h=bzhftnW2In`GUo z^i%veqAB*=ttHii6M@3blb>=zk*fOLa?b~YUnVx{M;H*?-~-q2zl@V#$fvMBf)#pU z4`l|UpRu-mbqu}L!1E&wF=Z7&&*jY$Rzf8O=XA0II7bB|&M+(G;>NwMlr-)nCzW6{ z>>)GpQj7gG!Y66nZZ=4k6Ur68H^2FDiOi2>u8fHG~LF3BQ?l0+hV|a za+UUSY&7$%ROk;i(iGvm{0+nd4RO=ux-of-$6(Y`Mr z>B*g^IZtjuJ=n|ha*gM1I-fI@)u7*|cmT%Risfn0KFbi9iOQ?d)vVj+2|fFx_U*Yy;7+~ zZ8ahyR#xr6#*s;(e^+PCqtQP~u0>ZbHj_|+Bz&av_g>S91#1#Ug5?64wCF_wn)VtG zrA_Nb$yY!IVjk^%owOWZegw|0lkg_C6aIXGp%mLXi;>vA%k(II;36{8GkJ{yH&fNcFBx2gw+J)wZ7|!im z`u0SyH(?Di*Y9v(swp_C3%-@Fw+~i!gkFeCaF7L0(Uj>6)H;O|vpkF4MB#K!>_!A} z>|rqoOj$?s=5hZ>4s{zn&2~!~91CJ=8Zxw%ykon8E6F;`&&tkt^?Ld3R*ln`YG=_Rd7pl%s`GV=C`Rkpro)m9q zqOe~U%2o&Oj?7snlN_i*wTYw45$lLsd?~+W0)f)WmYN=I`BbV_C$Cl&TS`3rNOmR^ z(lKz?d@99eJv&LP${+u+P6h!pfs(`V$;ldYsqM&*3twu^<=EnEPdA}93EKi!i7?`_XlFKs;rMgxFPIHXR>s`IOc(|We)GRg%=PnT8)Qn;|Ktt5X6=wNFpd7*2?Op8%Sg>yha4As;de~3Vt(p8B zDgpaddk{chp9P$G{4&SHr){UuH{EFJtlE{g!&jc`LPNRccf3VGrErY&gP~W{eG`Ne z$JmNDi8TRPw(eKeikgB(X#CEzbfjs+b*@sNqR0sXA1oC3;osfl86d7Yf#Rh-2^>+L zqAP8(-d559iS$9@U#7|CGh%!szEeVApvVPCA>$*Rm z>$+viGo&w4Fs_L3~ z%I@0zCzi$92`!&8t9^IOU9JpTOCM$7Y~qgRvA_K`35SD({Wv17IA#BgWFN4Q+b|HJ zG=C#f;mLT1T1Df4^g>}rb;o5%$ybQvm6;>XHTslatB_UC09(8`s}E{@_5-hfi>P^} zWl(exXyO;K1;)T=7xbe;$K18&1pRSBWTI(%uCsCWGxCmq+~oOS!(y=M)zYsfdwVzG zxPS3G_5hLFggL%j+uc7?c;3AH+EH}q!hk~rEQkEQ7-hRbXZ?qp@;4-t=Z?BZByKH| z-@W%LHZ|n66Fh(SXrpqN-^l=x7V%8szHY!y8}QKx=k=i4~BV7?t<6ySoKLqK1Bqn5tijBacpSMxG8x3 z%L~BswjSxUcxuQX?EKjt_j&HSC69G?k}IjDHQH_ul>C?t)E(?pUDVFX8YdG}o+F*u z!!}zq6T7=m!xcJWIvS~fbim%J1Qs`_w(XYL!|aMM-*Ti7XXP%$g1u9(I>PClBHO_E zl*uKveu5n{vb*VAS|u}#ZU#AZH=2@zmDDwbyE*^}yH>mFx` zoFlCEAG-EZPYT^6zq=;5h?hjSVnZaoRtGlAw$e2-*p}=%F4`!CtmI}@>^h!WIuQqA z2Axw16^3JZJw2!LBdwnOvRVFxzH({h;ggG+{<2xd)c0fcZ~(LX(!JLYZ0B}!zmxG8 zW;KRva-4*%L}y6rMOO(S`%Y;3uHYz!-+_#XPJqhD@@Tl>)f%3Fj$qrE@_rCb&{iv6 z=*c~{K8#wewh3&dU(CbeibAgzA#?qeO(-ui+0*{6U% zY!66Ic4Jje^6FQ_1(*vO#^0*ejn|p0^RVD`oBr2;*-dLD@H5XlO)Kn=wXS>bvm6)7 zY>5J$c^?`tKF`TjjWM_{y>Jf|PcRZwJtFjEWU}-d>Tw8x-}CB`#D_PHi8IweK~_Q9)ES6rP`gneC`zKWQ}7AgUSm=R{GHq8|4qLUj;;$gQzD&HZSLkZ z5lcm0zPnI4-VmvUJ0wjfp&9R%lPMlz{!s@Wwo%x}_{6smcy-qW z?7u*_h;2-0XdJNC31)PX05WvxE29V{Fp0Pp!VmZiR+dH{XldSlBv}Kr8N>OTq8%FG z3j%Yi0ETY1sKDxUhdd%JgVsOipVp$gA@?%u{(&n8W5F%OO61*DMtqlp2yVFJ?Qe_~ zq*r)h^KyJ>9O30~l>Wz#^2b&K$X-Uh-*oNuL(k|xe{T;!l+Ozv53i(ty}^JUf|zPH z)&2hEw58z#|E%{rRNDUwMv*y#fC%@=zPcU`?jWD;HKIHYbj!PU82Iw*D-5XvG?%s5 z;XifArTFu&H$W>~nxB2;zi7;LgN?HnZ~l6*R&NUGz4e`ujh_}~htiVw0ZM!G{^gn9 zBWYQPJM+NR;buE*@{E0~cjuly?qUYp<&#+teu2aP;Xf2eX6!yCCtM_6O{a*z{oj9F*EjqiFdwL1`e&H*-+ook3~tQ-eY8JT&@HSt{6fy6 z?{j|qCE`Zr4DlxquY^lvN^iW2-c4O&s_*K~if_35kH_dUk{ByBB&qO^pZsNO*Dlyb z968?C|9JWP#|&Pr)qqGu>mNJG-(KC}Ig)I${qp8o?%$Q*(FqM)mo5KYFs?YZZNm8P zTlxQDf|e&BNi{o|tGr770Km=&e42HLLkfbbeL&K32X84Q`a<3^!)UO!fQ)tU1KV4c3jQGm$hg70mw*V8+PuM&ME0hfbp(e<_b z2Lw~aUjq2h>QWcy9a7PW;5WzW&uMxnKC)qK#yezMRiFb)@lJA0K8=>;>yk^rd3~T0 z$01Aw{=MGE_?|qQh08M71z@ZAiVLXlPmE^RbhjYBQN}o4(7(Itys+KRcrfoUlAq*Q ze}3@LD_(*)E}M5|+4x%wZo+Y{l_e5oEtn=_2DW&BboV3^1EgKR87mL!7c<~BBau0v z`S7ltuy+PhK+mScc_e8UAVoyooulcFI0OQ!h|@lfafjkO?16k2*Cw3%LV(!pU}J`_HN){IARZ$1MI;tetZWyu5{OwcOS30fs?jC zF0&X2ejms;eS*P4-VS_wiPwL~w|Nf@9u!iN<7B& z3ru59K|?NySQUxbYu4YS71FKw6$Q&OQ{Q`TNc>~ga*8Yr+8Vp7Q`|^ml2&_Vz96Ff zEbyI0jDu%u67~_`CyI>AlQrcj=+Y@`dMrX5ZvxtzK_1>al5H}2jtv4>RdTw;{R|M$ z2i;Dwcc>+Oi2+iP|fB855bTi$m}rlHY6;#I-Nsv{3y zCvGib2>=rALGhblcK1m*93(>t3r=8|_R#dpJiyS;VI~(rK$qsTJ}H@(=rRr%KJYZi zIPQzxK0#ou^~gXyT^#t)r;HJ!^w+;!DyP@4gqqSp_~N3;xum$|0s9|~hZZc_qF4u} zI=Z5Dm^eM{Tiz?cBZ&8o>t(^c#j00S@EDW=JlA%B)B%oMRExAK>glJ=ODMCz(`S+P zc466rU6`>mz}Sn&d$ zCGpcYJ9ou$VqeXTj8lPEafY^9#zpbWrWve1Vb2TJ*~3pEC!`*x$*Z1UAIjUOs3Zg; zhDE)+rP?pj;?asVgD?5*EVy#Lbs(m-z~h?g#B9UFzReE6z-*Rk;Qkt;o~3Ksb^tbm zpbDqaS*!g6U)+6Y!{(_rhmrj}Rp(ypT(;e#+rctPJb7WAeb4he~aa)`?edphrx_FVl_5J>EjW4lb0rHVdEgJ3DaC&$zbV+ow9fz!kH8GJgdPr$&}VKmxSSiQC=auqhKLgDKK?tq7$d#q=@1>mX2 z<(fj7MAVBn5FZZK7){J7BZw;Hu&+gr7G-zO&`WDhK{A%RG)Xa3(VWBs%drcl5xp#|xRh$K@&Uz?&U0~R?TdD$b?B!v6#N@lG0Kay z{#6QkpuV{KJvuj(116`~cN8z3x&{G+3f-4JQq-$v3P8*uMIqr+ZASw6m-jaL-nn~s zDm8Fa;xz!KKp5eD=TpRqSQ% znT}d~M=#A5=DP~Iv`}{0Fte}v&g9BNrc4Q_!Q!MLIUq;Qm23%T- z$uTr#BFfyL5yW~U+#!h<`0p4Y6auOx{4otH94p3P z4Uc?^3Hdm^@72&vE?ie;MGSCBG+w#D$)14Q;*Cq;;^LJM8bXa8(6OK-P^E0Y=hgSV zz%%A-v__s}=G!F00*V~!7>=rxyTVv4;De9ouP2ORf=TE}Ej+234q|USgMll5<5fG^ zhyaXAPrFEd%oW&=XH4mRpORvUd)DGLcIa|mRWN4DGFJ< z4W}jyzt2ms9fBdCO|2|;U6htlT_Rz9e`2>a&1~?q!`8XYXH^}8 zJD%}I;pv}uNnsZN(UYy_u3Cf4Gr32ptlU{~U;?XoOk?Rd>F$tzZQH|wd}Q}q=2~Hw zU4*f-3RiBbM)mM3$F~K8dX+!mWV<)^>dtc728CZX6$#!^QsXf|3tFHSQ&jsI_4MQf z!idCubYFGT;6ZNc1lam(D*9bkKVGf~CuFeh=L&E*0Z)A^`*MilSexsr&TptAuUJcR za6+i3Kpl;WS6&=%=lfxVRf#F)hqMs05L0{eUT+++45~+@)&8_s&Pyo+a`+DDGqgYN zHrXe%K!KH>L#o4QK7C{ivS7lWKiNC9>rodB&U`Qy*6cPyRnWnELIO0I+&0k=sKSSP z$7nu*JVIKBgAkOB-^g9LZ?LVgdPXML?W?_e17(!vT>0~#(J`aml!6hB;1nUDnnb*< z&EM69*KR%0p7oMn&_yG%hjmeVLrEcP4n`*Yyj_VwqJuO{VBv!w&_o<4sS%Gjp; zA^=VtRe#D256Q~JB9Er;(0_z`LKu%xsskmp&Ww*?YFZVW;pAMq30Ks=mSUMF=#d`MF^|Y5KZgw_zoDu?gSi9TP1TD`nLV zf}{aEmmW(wWPJI==zo~P{4p;n2JXIqsG9Lsl!)_aZ2zsmo+m&MPD^}FY>3ZvK@#J- zd<9pQFl==T`l0IU(R-%g*ljXSrT@h0m8oRtDu5t}Vp%18D{+swSm5&WoM~n8(-4!R zbDEUDjfK#hy%Xo*2|6Ur_09pA&0A5&;bPm6jCbuvPtt4GxtV{4u$dR=a)h9Yl3+T& z%c+@@@h*=+4|*btk}G=OnR=J+l>h_v7#n24|~S#V#L1|#cu{X zSNlEC&d!89WePkY{EctzNpC5}&;xwkhNSz>ISesxo)6!!+isn4_G2=ijUM7M9R-fo zpfql8z&JMbekO3XPS4*AT{(1W;gonw>EdWX=cDh1$;9mWvLV-i358nI#gf4eJ}4Q`CCd ziEzrW_9G@xky_)ONC`S#0BaUpsUt7j1dnADQYngpsH*lD5HFv9mtXE!`QA>yUGa+d z%i8SQ?p50Q-IJ6|O2PWfM%I6n|+emu4Er5-@r{3GARr^L1vUD{gyDH~Ar1y{qb9X1Qo@%tesz2v(luJN(i& z+&Vw*^b?oTb9Y9s*)~$n=NeeigCX^~ZR(KYvo+93r^uBZENTTIXF&}V=&%RE_df4G ziH@ZqBg%{&=;a^cl464DuIFWa%(Hj~y-;~AsSWbWejVUsP3q4!$h<$p_u>t55G;97 zji-6fMscA}CUt~rm=|)6MrisvPIM=+%^4?EE@1ZGo5yP`FTuEI?W2;m3Rb;c1|=5gmD}Sw zQZ6bL($mVU#a!NHbvupG*|c?RZ_5S1;(L2ia;Q;pUh$2lZ~Msx;u&9($H4Mth(wDr$eld{-R(?+vwZJN(9%BrR$)~|<-F@$@^UG4@) zL-8|bF0>u$Dk<~x`2LkXg`b=sV{!RfpNkNZ;?osJ607!0qVkk2t^uQ@%;afwMQ{dJ zC?$=nXTUtMG8T))eTCqq)rbC%f2u%h=`{ zA_X$s)b9NbX*+yZxRSSDg_kAm1=Q0SPU{CMvz&|Lr0_RzVFf@*ExOE@PV0`{^n9A&kQ(U7JS7N;5%7JRkv`A>&2Taj4`3NZ$ zkhR52(7z&d$L@P|q3hm6^_3u4<`T3O=CQdxg-3~;CTr+D6y6f26&(beu5^#picjka zo-bGWkUo=U-Y(xWEw^hhd89j<_N<%P+P92tpjFz)d#cc&PL&y|7+m9%NDa%}DUK>xg-Aj-X@ z65P5BFM{oBIepvN$qRH&sr1I~EGwGY83pF>kI5pg;^Z^HSGB)3B^jp;*Kdqsxq0Ec-!+NPI!ArC48rmPfDd%hp)Xxl@$gXTok;!n((lv^FT?n|t<`mis?IxzRd+tZ=7y%Kjdu zB1cBRl}}>les5LFt%}?gGs=^{N6@s(U~t{pfQmmnlMnf>1>D$9RP!bDsc>5oJ%b&q zFQ{(g=N@J#ooI3Ndd0o#uDO|q-$l4&G90I#-F{4}y`lqe!L3%HmVMt&lxL=4XIfRJ z@;?QJBJ+z9Pk3#8iF%?#ry0L4ZAlM&FHye@`o`aXggZ#i=l{Lh|4fGdpI^79&g&l- z{&o{~4eO>FrM72MCWz%52^og%lzo~-|FME9;<3J1nkvb-ADX1}>pmN`0ERtUzTs2* zk5~8?MEC0lOWYqwl9o;xbx*-8_ngOU2YBHASf20#8y*4Mu_f8dK<#-ay}yPR(|>}qV$fVv%CfZed!L?3wfz7DX(D_(@L?5md;6?^ z^Uk|lYs8VKO==w0W;+36dhB{|=r;`mY@M%c;oJs#`svk!6Le)`O4mu7!Lfe^`XDac zUTI5tqO(}FV9(@`n^xH1{$%`nA3dU!i{iZi5sAja$6fG!VX*hn9sKKi>6-`+-?09* z>2Ej-v8w>z;mQLOn2aLkDi}=ZS@u5R3Jfz}xf#t!!kVk+IEb50M2L)*8YXy&rfj4g zT9=Y*Ge^MgMo2HH|D+YcpVC`)!=&_~54eJ|oJ;J7s4fqT&9&SVpsj6LFarleD-1F%zAm);Yv!Qc> zZgB(TUpnotH^D#UQ>MRVWyCOUlb#DabIUkWPNk1$9|aFOqiIDP5}WuY)?qRp2$m%d zqrmx@eKLH1u%bQh%xG!~g@kkp)k~n-#1grc5#qf_2&UQ*CY}C#lH?uMOZ`uuqC4Z- zzI_~+vtW#qc9zYwVLu}_Tq$FAx9U=n;=SN2_~GRU8^*@I>TNe~!MiHEzSulRbjSrz z2<`@|*4#ZR1-{%TchN@7C+Y!LxH@XL?x8Y>^0)cf^eNu>3@JX$(%KUe3&X?IeCQ4| zE+L7S>Av=hA=r3Z3q7sWlG0~5jez=U%XRoOqr(ZO>oT4yD+)%A!vz=_pn86U3O;#) zolivYgD+adktrZ7$3mCa^pSe{JH~PMMCaI#_)wbp}63Od8SDayRkrb|(5hUpb z=B$P_xij}_0RX-sl03#VeaIu+pLu5c#*pku29T)~=COZ-83~S~T}eIQXeBtg7;2;C z+8QjjE9um1T$M$2EeCsFD>=SO1BZx6*03Z>VB(x&AIO_BZhVyURuwmy{jTx>9>a5T zRpSc~#?%79L_RpLc~BUX83{wWuR$bgy9WHwEXQ=WOA~gtYzWdMTBB|t11>_7uWzUV{Bc~qJ@cT z&g`c!3(yaL8jn3H^pM&v35yVPm<|GB1y>=UGY3+*AleB3&Z`vK3lcYqr2MGcLa$b~ zLvu&0KCH5p)b$fKnr(z_x=|S(y8@EfSP%a6A!hR!0&ZLHzVDuK7i72clOv{oA#h zgd1hZFM5(Suqv@ESdLv<2H6l{q1SU#?IfwtDM$5USzq~|!!1EUjJfYz6$vEFs=`lt z82S=#EFYg0%&xJf8-5D=ru2R1rdk5rs*-W$#@35+4d`15pz|KO6?jTnt^rVqMrFt1XcW8S`LLRX zk_W61rr|@nJN!ozo&f+N*AQ-Xs(tp}stq$!91qFL!{aF>3At8<3!p^~$BrwS@6Zgi z37G$psMUbh!WzzaB5YqX@q#jsqVb*(xGdT-DiZuy{}$i4WW-<)ccb=5t@GGvOhLfj zxBLTjCuNoZELsxFFVo}i{eYek;T=Z^z@FBT9)sG|+gke=A+CaZdDmv@t&Yv^d~MO1 zKe+I%1R*o`?c2xSzA$(lqwtfIm17+zxi&H2swbN< z-od*dpp;y6+fBb&i@Sz;ER;f~DP9OlfmD@?ZR?qC5KYs>@fsw;dQVl;KgaSZmFcRc|%KM!<0n_h*_XtHhEAIg>H2{K8G z5aW!u%*FS$6_rQVvK~pMC7DF{kmWQ&KIf5y=uDQK*a2?1UqLj6HdC1Fp(RtzA}grNDa2W456NiH@8%AcOFRZMU_N_&2=dFRChMTwJU7L9Yq*1 z;X-#z5s$bJgD-1I&-a(G?-5Aj*dvB)WZGdTE@pFl^jcm4IAd-rPMAkWKF~eXiABwZ z)kQ`U9E{ry1E>WqaqweoSq%GpZ^Tb;v4lw`-+Pfb^NL0K7Glgjp(3)dxZI6d)yWF0 zB=N1rLE&@m?%X9f7;Tf*(oy-M;bV9BI=cxa7vHq#y!0os9H9%SThhBTL^IOwzin_y zZ{uWAV(;q)j=x1)+^e_~A@o2#(ZN*pRKR4ZMmdAw$;0N|CwM!8vrTP z6dUwZDl!y4&K@@`>fGL~`>DE+or}7x#z#zvb`4mTj#;E#sU8vK)I@do2tP>BC(Dy5x4-6xslht0QcM1W;oZVGN_uqpIXJKzTEc4#}HN%#l`fr~p zX0Prn=(v!%;RJboM-$yzPPFIm`WpD7QHXS(5vF0b<2{-PCSV!ZL81hUC(8cfE%JZ#&$gu6()Imy`TT8NIg(yIX}b1=^MFA*$5uH!H8z!6ct*P$RxaAK zadzopTi`=+4!LwBQyR~5Qc&imp5q+l@S*tF$1&Zb#E;O8bU5?-9Ns?C=Zj~td%kQr zmktj4N~9t7HEX4ZwRbRpgIfH$1`erQE86r0%IEWH{(98ONL=w(NjRZR!2uApp}2%% z@(clM!9+%1v&=R82b)RMH=)*_iK5q_vCw&X`WW}hG}JBp%R|;7c4*6{5x(#3(?zjV zkCL)FaxI#mr}tpF*i%wAK9)Gk{xxoa)t;Z@l0lVml->GcZKn5^sGiHM6*28a_D8)I znycur9XGJfm*Ayg*}*XSVMxlfKZ8lt$>8C<8h*AH0*}c~v`3WoKk1i=7gc@Y7*$VY z`uz)3mM(9Wk7B;WF&&Zi{4xo;<*Kh5qaFA}p^0Oh<>=JZX76P|e19h6NEp4WmfzAy z^Jp<&KYjS)^$;wJN{_1boMzPtwmtR2=UyUNXbVe9LH~)|YdKny?@qIw?ET^c&Bii! z^hOg7e;S?zx3D+Q3XeJqhKWdW^~g^9_;*TEbf9=i;#S;`C!K2KSt5?fyC&nF{h26> zp*va*1?b|uaKWANCd+f1!(s^2y#}(1lTU@Pu zZ^jvve|AnSs|MW2Yz}MO+DP;DFT=bEFYI@F#RR!y!k3DYl`1?0%zX^TQBP<)+9i@H=qIPyy;TOWTA;hmyx;)D%+{ATf5VrQKp$$R;$!- z#Macn_wSxPnpem{X1hafXR%E=O=e(1RKHXm(Xp=(*@i8)Jj7Y8wh*wd$H3H_`aY^u{*bmr)t6KWS4GR#hIt4 z5`0@YCB3STVJz^&Y=QDoG}LL-In_pOMA0DSd^vn-+% zXg99n)Ivobj+Ymu*~IxDa~RuWoZAWwDi?5+K+k5X$+xO6rZa5 z86s$IhL;!CEr)$v=$MJ9DLm?KT{HxVqL59PIOT52w?7ywMQ}Px#d*dC7!R!#$eVY10 zz!rj20fi%Ci~W1z!g~-GGdDetQFj0z4XgJQJBj16qA;Ww>VfXTZ->^-r*BQ?bo}xZ zuq3y8N20!?;$`~R-cnyr!bY1_bZizYfnhvhI#dnuNUvZUMpQosJR$mT)tKgvKr4Ty zB4`AOyinB8W@j z0H#oG6Q56B4d0iBLdevuo&zJ*A;<4ceE^aM<6t+O#J1;WkJwC-U#SG3ahABHDu(X| z`o%bk+*jc;_i7+&S{;2_J)EE$am9dYFBUEFysD0$KH!}DwB$Zch0ZSd1UFisKes@n zt9c?_`!n>uB`v@V+v;kIAY*9nBTQMDSf)mq%b+>;7{NlJ~$iYLdC`r4N}7H2=SxtZ=ZSSB|nGiDCm@7 z(2vIV9tJEAj6LsufZU^au+Q-|jK}gAgEmdI0bSiKk%9f-^Ty&xglRV~%8h+{PQx;% zNlA)sV=MH;+s>)iauR-28^=R462pk|wVC5*ePu{xXL}%~i}*N-KBgY%GANKJ@%=TQ0Vq8Dmmw- zjkH*HhT#;U+cbISJ)!~S^=0v~q*PbKF5?AE@MpdZJLAQ4)T+|I#ol&^CNV1t8{_z9Jdkipc{a3 zTpSdyr$P7u=mRfuq(Wwxu$VoZ*`xz&*0+oBF`48nf6sxo{xG2f9Knv@K7&bl>K1 zOb$+bl~>*)Ok9}sAzX}q3C4rTD|61D$GK?|uBeN3n_qzO;3?r5-N8!U zfYrrQF6de5FALlGUqM0G@0TpIu{4&zT-aodvBk(^PA-o{l^656b8XiwU}T2LH!Fpo zQcvFTVUr-v9;PP6R#d1mGU-DtEpoYhWvh<*Dxo&5|0LLht?yzqTD9_hOx;N3RKkFk zZBEaR2#%d{nam3kC=w1u&9oHtKHeeIXJLff=CAaD^qZUqEa!2ktd;17N3KrWcQOW_ z!4Y1<^Nf1A?cHax>TD}-s;pkm!Ozr^8H+ENKaHBaie~dFPR_mJLfQX}Hqev*^fJdj zM*M8@*#i$MX|+H#nmVe@r5a$s!?P({h*|C(CgjHfl^I;kmY;c5Peq@`b}@ths0zS^;|dnV)G0!m6# zxb!H_dYM0>ZZb>oy4KsNZx?5t&Rg`P^|o0qZh`)UfTIaRLS$K0{=hk@$W%TTp1Etz z96lX{0))?Pvi0ywdpp~n*26j?<3o|U+swI`8yN*`6p&lTj&MmT8HMIdf9#M3?}Ir67ZwQr{6RD zpkw#lIB12-Mu`7C5MbpvqAtZy3e3J)KYjXa+9Lh=!Ap_OBvX22rXf85izePe=QGXB zj#t{q*X3NK-AM>T1$h>`1z7w4S$63)x4yohTe@F(;V))^zlJ%%8^%DM;V2#S?;I~f zJJ)pkMwXun6+b7~{}=}2^mm_GcSeKY`_4~#vT5+7L1^yu1nW%~YpHx*&O7|lABu24 zQnMA>{^Y@GeTl%|{KOfcVqLYy0GmBWvasg9pqn^<>NmNbM-xORF9lp-5uKU8x-|Q- z`Ot!J={8JzCk-S%I$2pO(R*|R9sCH={b+8tGF8Wgiu%gfmmx7#JwBRp$+)uIMQqm9 z>Y~HNn;e_1r%OPAaL4zdIN!H1yh6Qz4HFN|2mO~wI`;Rz20frzP(;_|hBfL`KVx?d zV8uQ-Cr)3*tmKBiaCly4F&NniaPaxVY%qy904j$VD#4G$l301r@H_`HXj82~ek!!j zAq#o~&sd{RKd`0&mEHt)-p^8xg*fX&S!)9+1JKkma|tfBU@ zLH`_LI;s77A?*#U5&N5Z8$=sX|9q`KKQ%T9^!u{ZD;NL#)BijKwY&+xHubY=_@(Yq z>}XX4ETy1M*qP?N^-#lJlA;K?IRtN{G=H!@jC zgKASpj)(-RT$I%VN#t--!%L0dzJI|7czZ@6miu31m@nWxasR6f z(~+_y_hXO5*}uG{4zG~QWJRGCp@ycL!&sqw1HQ_r(LYjKgH@5a5yy+k-zUe)oI}2P z{gao60qlRQs)A&A&UtZ0-G0AhhIu}mM_x@|s8qu1V}Fi>7oc#RTqXc!QF3%GI~+DHsDiiRH5Grl zH}+6BqCTwUVJ_xCvjTPB`1XxvhPbv$7S+|j6{1NUsTz^ zA6I3sLKq8wK`8zgwJ30c>-Gpe@qjy#H+b@?+E#u%KofJIC3*tcWeYEJ`5bHMTx%}* z`CC+4cY;Kv$+jo?%ezLeyN-VDbRYBHf^*p0D)DNmM^eo69I>O>NsK#NvjABNSFOgxOznA z`Cz{E0ldq=^plz!8%}L^5W6|Q_p_Zb@JkBQC(&e6Po3+)CyKND31FlrygOCGV_m^Q z50C0JhVKf#eGUHgc?k_#uru62tZ5fhSd!ify})?!1c5zSIaoU)E~#^fM=!K35bqlz zYcHY}NMKMiBWgqEQX#vp6OZ!R1;Uk#=~E|)K_pbw&!Fn&C$Zb{Gs0r%1bKAzy6guZ zEHQ&nu3;Rl2lR|cJBrKhO2c=DUdM6%t#gO`cE`z1&1tw!knowXCZrTn1Lwam?~|&V z1erszm)7>h)+GXQW}HeE7S@pD+)o^r>#qZR_)_^~igY0u4IyW84dw^g5bDOz$2bR{K?JVK*laamba}QHNtSxvK!Bp07^O2z+ekrn zll=zrgN^cwWVk&6sQ4CGPC7+AWd<%r!dUglh1(tsLZ=Z)eIJ5SX?N-J!MRZc%h0{A zK4BdrfXEE#r!F7-9LwKZzS6QgFKE-Z`#=TZ*ZI86kywMKsrd&~9?sD3l(DSSh&t(c zQrZe2c1U3SNj7CmKJF|Ea3d+F{aO2dY76(YSIj$naj!BYyzJLWIiY%40a8FoF+t-% zWB}?D^#Pc7$!CL6X;g=Z$V;+dQBZBgWW0k;_B}xBaCM}-6}URo#C5xU%|1zYU9J+I zEv0wvxCmj$jMFbDgVbVeKMfV?arzS$-MzU!Y~w5~f@jD>+zqT}pV2bo9{)cd! zEo%^dgr^fsq)CTGheNHd_uxg=^DxCU_2RL7pu9| zyVKll;=m+LKr7PirZ@mY0r<7nb3V(|7WiTiI86FMj-Gl zpHl!NkCelr_f@Vc6X^jLCCHY_@KYQn{P^i&=Yj0A2R&MWZG^Q~AJ(*dno*Z7@F2F{ zf6)&EALV>?FPJnTfybW?K+wrP6EleG$c;zk%x&AEBoK7`3&BpE+~D$bOcIxrZar`g z7a~bh+9T~)Ti-fg>=O#BxIcr25nq^hTvsI}6y%~C!h3c^b`F&LC*G`jSyzSnv6#2# zyw@5k%Da#c30}T7yEvWRacV#b_CFGU`0Wbwp2zsk(-Dl%rX8@NuNeQw%@NrqOGp4@ zDkbUD%e`%To&0<$^-)=p7h2tyBAH-9CU!Rx(>*TUxzFVk_;Vy5G0h-%5Q0%pTb^w- zs;}4?zNL8;O~e*S^HnkCJ5ax0q&{{B&_i)iP*Hq=NzEM>p>r_DlUPFV96M8e)H+VH zsO#kSvg((tmkW=cAE#zq)D!1M9XKK+g_%~2e${5gV(2{VpoN5WsoUwcpOEBaOpm$G1cykTm@BQCJWW2+-Y2CV zMl8!J&Y~P?IdmNUmR8VDGF_SQfskeUKJ?(Tdo|?}que``{S!h;QXW(-!9%xbXQg^S zWC*^6ebt6|dA*X0%vKe`<@`Da^jKEBGc z{SE^@+arZ9Ak~R(G%44$5Pf*A>mnJH20uq@63?(YelU}|!CuOIF8W~)NDNw4VtcqM zC1MTBFSQ#y294i2v)R*ff|^n4wfydxX3-eVQ2Cqy>U=e~u2tXTI0n?zJ@kcCdxGW- zB6IyRyeiq_mQ+51E0BI=MLmPweL!RCxjzHU6vlH_3uG)zc2zkQnNjvLq=H9crn0Fm zO4Rr5$|-c8K>~Ptd@J;x$!1&=1^w$*5e$?gqO@NDPq&-a0ZYfHsIi8QsfQT!@acKu ze49AcC6;B?T}16|fvT}`?4`#^$pu@-702xe>0K7Lk2d8qz&n?dDoyk7X1`Nh3s|U& z^4JzFTMfuEzTF`@PcomUdZHY{T)cV-?>vWC=qr{=noQZM zg;@OC1g6D!A|DFcGTOs7X#`SoQ;qLySu|4k@FYDa9_ut6FI31Z#@x?#n}mm0?*3Ca z(}$SvGtcnJpMz7th9f(zTQB*1!+H*4t`c3d3}T=(fr}M>n!e9Tn%{1R6uop>a#Qw*;IZfEx<^%Ev;i2-9sBNYYPcrQy7kTAN&crJlNH*H(hiC+up9u za`J9bW8LISo0T38yj_OR;hbYxupzsGZVAD}9Zt7#YJkdJlBy_EdckHnGh#un~7uW&K)WH!YpFC$I>;WIEs z3#As#H+sr@2Aq0+ zuBX|cltw*44r?cwD$i_lqFwVjxZ=GBDHsy!=75fQAGrq}`nhY`n+=r*35iH5s7rN> zl&uKg8lUNVc<{|;edX5~ce~=~l-xOL1wZyiKPI^nI2$gIUoo-<{zTMZjRerg`ZiB; zRMj~ZuTj=AL-cH#xFlg524BBMq9TKb=JFdRHLLZ}4 z41!g^*id;I5<|~*1Oec|&g^LN`xaT-Zjn7L-)t=5p1WTf5GhZgThXan{1`)3|= zxYChy$eAW_DN!Oh+5?|iTp~}c`+6G=*VVFl@av$qusaLcGSgLQwiuxv2>c?V7bmt& zE<}nEgazgGJ+moP>?orsa}JoqUA&R%$3^o(C+esv+Tn}=Z%`Zx0j7fJawfd?PxUk=&#$JEUcbSSt#9E6d{!wquNL!@nt&rVBg+wg zvU02aW2a*eixfD3%s2xvFxA}t4fs|OfA(Vq)U0)|2+Gz8fyakB$S9{ElLCY4RWBB4 zD97YKUGIn_0hsh$cHkG7!Xr8LY;~3gy4@7r*TCUd$07%v{p!gds>$^0MxY*HfI{+F z%ynZD1`^dyBExnL3%^e1O73rj)ey&6MG=M4$_Gy^aP>MHQaZ$+-?FWhG9CCW`+ca; za+;}|A1D?K$}KcF{OEQD$L~|habk5JJ$6E+^l>rt`D+dijLxKB}>!<^6D-gte>#Kh4JoNPYqr2HSS}?8H-szEhOP=JZ_y zKEHlQc6QLijotmC+H^)^&VtBdXs%bzPhVio=my%Za9v0*n1{BA^YS(XDI$v(1b)F{ zYrOMN)rQ>+Og(dfe$^LLc%4fTRvLTMRb6_c${g zrfXZPn$L-SP{}|Hyt>JWeSw~-S1_1KsAmbFR<6nB;90J^f_Q*9ea%t~=5} zi4t?MXV&Q@2JL3Qw-3CPW6xRcJ$!%Wg|Sy!ublvT~o$l`}5AlQ>re_?}xGnc>}N+3FM&=R@hOpY^wNp`#8RBOTJ$(k=};e zpD&%?8eUTbEE(bE?!e>zvE#a(g<+vg%dE3}QU7P^qcMk6ti>>G|{Y)M$-tJaCzmnJYL+n;+6r+y3& z*Kvie#AH7$Jyu$9`-sxwr;<}7%6ZdMl<5MnPw&$xp`!fVmMZMH#cwHW5^Sadu1?FA zc$Iu3M5;3+GgNBcT^04I$G7g)uz#g^E5LA@3yN*OK7oTD)b zjixnqtW7uZ)oUo3XiVvn4m;kZSs}ArQsYO>!`B{ea&!wH`TqszbjFR;oMR+0?v&1> z-4$=k4p7wcPI~(VEzl!m13ZQfcZr`Y%#dnvGyVjJg>{gQXN*pi>v~zgS5{%&v#7@% zFIAev^D8>k>tFSU zJ+DY62{_bKvDOT38KX641;z4s!NXaXd3>%w+a!58e2n;`SGwzeo=!mn=2QcFc$~#^ zS}|X_B1j$a=!~Y2Ko0~8PTJ;7rWf5b5vm zk@nN*)t}9uMSm7 zfK8^?^942)F*yVwC6ohyZe%O7qPaRm`aCF&Kx5zp+6cleP_<8{E%-ga|zEy`*o4m)kA*%uYQ9(6Rv)e zKxF%&XY8-WnH!N4&RQ@jJ^0rj{>S1(+ZN{23bo9HN{3ybUxUelXE5-L_PxMFW(k#_UfUJ;;$b>n-=(Z5DmUU zz=uWh46&smZ_R$=!=Q+zU=S{k-d6C5{{5SWAr0>g|B4D+m~K-9Eb*(`YYG42zax#O zcF3Q~Y~X+#?dS5%Lr<1Ja0CLtiAsZGJ|_NtH2w&2u>8MQ`=3Wn=Ku9|D=|OTwfwi6 zQ1JP<6G5qdWdy;>IdA237K+BP#(#hMxb30Cr}*|+To;mBbfAf+CQXag<<-?`Frh5Sig53I={!&3u%SqDinA>m^^j5wlXPt+{Jmc z3_y@xFt-^2F&H=p8@8SKrIb8uw6+fq+{ytzKaV!s@$p7Vbp4AH2VNbBlc9uFTjparTg5tq+EtiISFHto}PLR;VtN|C|{act@c$f>tDy2E@{>a9%@ug zHMRWo;T_COq;DH%0qH~HSptQZO~W6ke%4u<;=tDya^T5EQsoNjs?XqK-E8DWcTjSq z1L`4T}W$>Qlg@N z(l-EfF1_qj&gkZ(Gm)AE{KD&jLv94El* z6SrAmiDt$5SS$KGqrI4-Z#|1qy3ohJy#WYl^~IoMc*hedP-&4?^Stl`B!RS&W5QnY zrg3f`yt?~@X*D80`9S=rNs4b3+d3Z2n<%;uW9Dnst`r6b(3;-~f(gJ}RNjg066qX# z!Oh31N#!=?5CWZsg@uw}kRqmC$O3R*ANvfEi7rIj5DK2zrle0*tsg{cYQu>wpVBHY>j>3j;MGggU_pktohQDrmza`>&p<`hsAAE_ zl^<**wUU>G|JG-vRIpIzxCoe-l}O1=d>z)4hmwP&n~d9$NLiE*K*;xot~wFb*S zP2Wbv_wczJAOAO<)$ZZHIRQv>Wb#PlK& zuG@i|f19Yv$}W~pwYO_VM)HljigQmzWH$V_jv~{cr0}g2bur7a-2T#EW*iPh>|BCE z(`P25ll)TxvwlAe$=h3%>ODp;1A-qP+%0x_-Sd9CT7{?^jdK7<*@guRoAgGTn0rsu zL5vfV$Lg_Vn5(BlGeXNo-Nq6MHoLYQ(LpF?lE%zd&lzWXzXo)?A;Fm?xY484yhbs& zw+m6sfSQapBe?C%2Y71grC<<1B1(I;BnH9#A?0t3x*7V>+5e|rIn5GInF|W+{PSZp zn!E9X(8!AU3w?j;J42DeoQzvmrX~NZf!AI_WOe{U_0a`_3^gnnzN}O-fx`I+OA$8P zJCJYFC-ETtXZ7Vg-gJH3T>g%oRh?$mucAZAA(b$VUEC^l0$tfP5s%HDkCgwS@*pm^ zZ>_#G>2P&{5Nkt|FCAyOU}48Q$Gy?Q`95n~wIK5v=L;0xEReS2EMIDXzR_o-IFA3# z8Z`{_s-r%@P{8X>-S@;TQg!ODP}%;NjL8$#3XII2_}RK9B1OsnU`j&2e0$44ZM4=o z0+^@UH*@+1Tntreuo~d8+mk;7vC<{?NRov1Ih^3Bn{5Xu{0+MxPZO0pkCQh~INVLD z-)*=Y3Ssw08=EQ_TfgCCJj{C5XBa7J9%}v1!Wu!!1lOoJ*7*{#pNXG}3YUr&a5u*uD&nUz621uI}wRbAIh17%m zn57bbL3u!rNLU-=XilVHu>|&{DpYQ#r#l09b!&}mnz>cJcxbJ!6RqF6Re0=Vr$?*J z_p+)>qS!sdqRZ?F9Qds*-`dlUmbDpd8BWE?61TZR;(UP2SPJR&dR37M0~+gqXysw$ z(V)jV5=xgF?~?r+N&?<)-Bh?PIgFnl1l zIU=(Z8d4RnO;y&Qm`h5LsTP||)|Xe?MY>6ySLs_eQz3CX@SJ2cl{o_cl+44Pwn<4m zTEDV~ru9fwe2JJwo!1fLX@kK0;B%aOa%Q?F)$Qk1S?)Xx?PNwvmzx(U@>q4^n)i`3Y5qUh)mSO9<%Nn) zo!Q_j_gr5|&bVCrTiA&{gK{jDdqskH*qWX4YS%;Ul~NHuz$;_SY4YCaUkSw5xe|zH zizz&r8X#0x{(^D9bZkQ|R=A^tpA4ROoqIxL0#aR`>T&e+BLp@L!5?|8sNOHv>( zeI?)EkV8i}8G>k$75oZ4XlM*b9~NiGoH=97Z7iqcqDZfbsq5C|8%r^(n%Q)_E&A{KebYs0q1*f79vwfZazTW)^)QEnmQB zeWp(_V&j41R(=5?kw@XB7$fn^3rv)+;!suQ5>s{@+c5JODFnY?OzSq)1HUp&m8k}x zg;O1CSC79JtEdfmfic2_!IGeGnbS1$dhdxmbsu(}^?~m>g&!`Gy5{1!OVGccGg{W| z;qlKr<=z;AWt3L8XD`$Q`xLEB=l@k=q~xP z?;JBi6GuTm!6?%R`D}Z5h5^QZ&;niYbNMq9Y>z_13UkhlRju3;w&w^19$!P!R2XCp z4mubr4Y`RYPs})5@1t5Um&-<~(@h*pb-S-_xpr~6jA=pGlxVO|0UC9(1#MT`j{Y&6 zk(AdRnjgl;uq~guc|v;@hK2HGcO!Q=yqg?EtPa}MjJ|mCjkc>2rL;S{N+GP%V>IZR zNY}j>{rcTnBidF*=9?0Mf2BW7({CDGZn8g|Wp(%aB#+@IX@WI9*LJOR_ShMn*>$Ql z?bJuNZDmz0nbgs_*}2i$POZt2$Jtsz?gCLabIla#+^uO==H5p4Sn{4zcM|*j`s`8RiIQ&C#Y$sh%ejL%* zjSQzK!JKBA90L6027gD5)ZS(mhZkQy2;j7Mk9snFPX?rW~jV{dH)1WsKM&g;^#N_|2zQ2^Kf7xLgqWD}C%wPY@ zcfb0{Og;t6T_3sCwlll3=MP)DCh<(#*L3gSm*THS@IHCVikSBjo_-7e&j8}Al$!3# z3>LlXS}vG9$5V%cbX;0L;Gd>`V3(Rl6fk zUIzWc5}2e(mIoFv{nL_L2;9&arLuP6w`au*z)JCU1e&t+W{UgqrhFDRw{BC`$ED1o z0Tf!b{}-PWt2Z^W<4=GS!IkPk}#zzW@GKE|WL zOveG3&qkwcAB?(#V)a+dh6Ot?1?g1nyvq(61x~=9%zQDllXNFgiw2e;H<_o;n(nj6 zvAho!%Fe}tn`v!1KmkC^X|U^OUJZN)^gQi2H8GTha#1=5#Mle#bk5*^LxZb*evNQ^ z7g(qm%Kj7=57&v$ZY@0k~6g-x&a*N)uvA?6EP&X%ktdZ2d93_rC;^r zXShH~nu&P`sr#subXT?#A!VvPvS`w;YD(9)Z@IBbC`kDe(`!*T38DU^F#VD4KUnK{ zUq1W!71CmzNKep>%RyaQJ${$gt^~>M-RNuU2)KlWagG}RYakZjS2cK$shEK17X#LK z?;DKic13B)^{e5~2$ADA1|Mb`w)@uia1>mWz20P~LC<5wI=5iPXIFZ4;2i?hb#Qv# zl5iR#rmiq@mT+<;Q}6G-hZiz;>cbd+x7%eiCU18py|xThVrFSb&&N%c=3$!%55kKw zIe_<8<~+2AcZjLXF&@&&T^aYVj!>FuE}=ns1t(fUvJw<(ISNoJsPx;?CbXG@4^jY? zceVR%5r*HLmLvIBhzT^HARK64>l?}Ju+wNE z_IhPSdk#P%Ey~%KuksXl_7qBckAr)g!Pcn)Zrijtl%w_Lq}> z;sq88k2Z)PYu*B)D0Lm&Sv(6$Lv@7kF1ocSSiS~FQ22l++WB^(pHIeE<1#@#?$1!= z#$ZVNVVXJh?+_0C#JDMPl`Adzr9$aM)Zw3?${J|hd&YAt0}Kh^!4xVG89M;Ek`3{j zoj~~^YT}LeOIupohJs>j3U6-sLNDp!xX6{J z&3ea(#bu=bL_|Jqsb(Ib^p-HLplQD{1Lvb;0(X0=UNkuY3Q`}a+iwF*{t*T5^=_MJ zeL-a!zmV}?v$j9o(7MEa(!}Vgw^VXPG3BO3*%i5hA7^VDTVw3EsS0|LhTH%Bu5NC{ z711Jt{yh^_iC4Y^$;}s}MrF(`O1Q@?Z028~#yj+O>t5K}v-0Sck%;~SPRa`PIp<`j zr#$@ra%XBBr0h1G&8*vSXyQ&^NpaTOB6Wi=&vC;T2V)k=-QnuVMuRmN#w0Bt7CffK zJ8LYQ#Ow()JeDDFgNhpvxP82Kz>r+Nef)Qq_2=g}kO3>}LmBrPn07Me-^q0smbq^i Y)zCL|tI1$0>7qq?KN#%D*0l8dADfZv)c^nh diff --git a/docs/img/graph.png b/docs/img/graph.png index cbc58e6d8f146ee365df9095d04d606227b00c53..ea2b7054e22e98f47eb9a0a0eaed8698c671c7c7 100644 GIT binary patch literal 152205 zcmb5Ub9g3AmpB|_GO=x26Wg{uv5h;)9oxvWx}6Rs#P0sjr>8wdypyp*JgnCrf$4ND6|NfM5!mJCo)4EiX%gm_C0f0NnA3YM;<~(yFuH zzRF9S0VoI{l63nnRY8exFGL{CxDsTO(0YjK$1#q8aYux(6w~O!Q?6K^J~#+1LJ6sgzfy#$9C#k;sitperNE&*OsB1 zl;2;yn|$_}zlJIN;BgK@Zy6S&)+9nc_+$Q=eek@MkU@%{)raD&^T zrAU~cRyH=qzXJ57%#XGy^#-+*jn{Gb6I=oX!Ws$-BuABjhZC%p-vrf2-eo^-lS2e) zG~l~+ECtN17HmWfl?E@p-LL?6ASG-#p;@CQf9iOVYWi<1rhRA9>>-@T72RP4N%j74 zH#C@pJmPY}PTG-}Sgcxr{xD($K_jxvo(Xf`7alnF?lfmGyjcn@_9WchE}Z6!2wvKh z0*pq~K#{}p?XnAz`3)H=N$B4GZGZ?Gz(|bOXjk!FvxQ24$!@q)JqeY^B(~w&sn^64 zb@(D7;CZJ8_~KAQeLLV!CGbNC8Mi8JfB>^R473R}o(Kj4oFc9_1)<2Fj>zK5_B^-? zjW5>^zMly7+HWJy%tN%cT>>WiCI{&qG8~$WbPw#HgI5vc!9ct|q8NWT?g?X9!X%I~ z&RGaz*uSgrR;f3BC86861o@qf2YN6aJfq@@5?{qtQ{p=@6l!01uBf#x;?qFKjpXxv z=mhLjH(%)9tC@m_@G%)LnJ%ddvYi|{TDWgmCTyM{W)!mvRonelT_1gph6XRvlmBO~ zeWw2zD9<|NB%0q_Pm5meU=(fOyW&{9W!{G4;|b>)bF} zV1c#}DWAY0^1a?i^ELiWK6xV+kKgZ}7~`Wq7!}A&HO88#Zpb_&d+`l7nw>vF%p((r z65&5!e9~k2*orHxP(nRJ2|}ktfJE6Gqy~>Q5OSy!XmBhqFXJH1I(NV5h=+XG=0QkpL5{CFPh<`HOuNARNb&JXn+0?G zOcB9yK-8WKkHJKm-L-n6GW~F`p+1QqqQIGnKnwh+|4e!}%MTcG|ANQxLs5iM_hZlb zW&@_vgYXCU96CA&5)Zn#2XY#;%Ad!f(j^cx2eBO-93~(R2}fAM1(~!TuSP5r39%m& zR){2?lMGTygaw(HM9hp7E}o?TuT;2^$aVN?*l3^A3E@Q4jutT-3odg*33q z5lJ*jZZ!C6trhpGmVOoDKPbK8jfZt_WpACF=v&gf!`I~qa!OR_sQHs1q%llE3lnNX zog$o4Z^gB=Rzz1+0PVGcfKI2vTmFN(QN$zxW%iOE^F*g)Wl613t-n{LJH&t@`7?AD zpY^KR{I=MYF>V2F*^ffORX?kHm3PXH7$ZH9M~h+8?7>@Jw%6& zi|&cILp$U2Wdy{o7Rju%t!mlo*v8o0tsAW$EiEltEXXYWzL8oJd0g$j_USW@CN5{VTbKQ>D@TX;hg|bQQ z(tZ~8zKIVe2s|~ zhW4V)W<#FVi`IeGf~H;rg|?TbxsLsx%RfT4y0*_h7Jj7KDCuzM_+LP+P0j?(WaS6v zTZ?b~$zO$BMqgf7^WQTLIX1HeM zrXevJb*Oapq>OcfYaWp0oOnlzse_p{CZ3v6<+4PzBvxfxWw5@!(QoIq?zuj(KDWNq zwsk#yANHVm>bRJ=jeX2DRX1>#pPHf3x#Uyb)p5~Y*WvCdfNRZ_!qMn(4TRQ4IP84J) zWGYlss6`N2h-&aUB0KEF_e#VIIA{0_^hF$bm{=GCY@a~W?%hA9n!vsFp$TkvmH-@N zLM8MYrdbMQycN7)@#sixbXSg5UTv%mEbyP2%;i5X2ebzd2R@mzIQiYYAD$^4V;t-D zCj001iU+Tvd{dEr{o&n_&r{1|tD%=-w^UZvn|<5g-oM?if;FZXPjes^V-$;?5|nZ3e%j^pOb zO$!8i5>kfMrQ8LaP3cWs?7Uo$0u)%iSw>m2OkYi^M$dQNZfKu70*Hi?x7E}=7KY$wyoF5Nq4vG#4Fsxwef+>r!axKsK}Z#|iQ zTsns&&T~hN6Rf{Ac&oj9mf!K=sNiZb-?jD?RrT-PjBa}me+KTp48;1tz7=+g7K-XR z8OJ~6$wl4_1PolrWTsZ=+%_cXcy^BGR^*-EDNoMcb#Z@cznpE?MwwoWZKT-i3h5H- zIsyx~M)40$4}uS3)x|2UG$(ZRT3;{HORXo>npJJIWwdDZ*3_`IR_Z5hHX>ZJ%b(Sj z%D~H?s+J78Og`S=*Wpt+UhUWGMhsO}iWD*+GwkhP*Jc%sl~ihyn;Q(z5?2dWS-Sx5 z>P>R?IxCHgYr$)6Zv7XWT1WaCO)9=ex1RO-+FNzjvsE!0J#86}%NHxPsWnYDgv*p0 zPw!9lTh&*w%Uf>Ud)--qhtNxed4x2!f;eVB?%8i~+tQWOxe;S|))&t=ypIvz1R9&L znp6pAxNG034lHhFdW?gIUUkGOJBCApfAU}WSVt66)yA#fFEz{U$vi0n+#1= zq=*58y53k{*OR1^`Ixcfu>$Ux0=ayp9#;4DmuYi#DOuB5F-NG`?p;V7SjY4)G9T3s zjFtKX?`vOIkwDbfR)cT@_)h&-@u!K$CI#SXQD%2Hxm%5H9?pA{U!h@|08b|rOnepkrjsvWN zq?R)X2nPATZcr&@vTG0!&@4+;O&3i$S#D!{TLvQ&`=6!^9<~mDp+P`+J-GiCZB1Q_ zh&*gOb?pWzdEbi4nB{5D-BSDN!L+576`9kbY_lua8^QMMhYL$CsCvW(7&gN;S-i zE&69=8ebX$SL}A0B}?tv~+?n8>;b9a1K+~)9J2<9PyOi5G9hnaO=UyJ(I zR~=a#@B0nzbHTw7l79a52_hin5M_fawn+US2+JXw zvi?7?4dLVm0{MdHH!7F#&j|y81n7hRQ~mwbTpT#W=bwOf1JBlix0eGDWxIrb{LT15 zh(V`~m^+sEXXq_9=0?Jo1%O(B<0$$8nGAo_3J>#d?`; zmw`+AkCjdYDnldz0r=WCk%IdcP*PRa8Y|W}q=UTaoP1wjj(IroI7j}+6I|+KRy+W+ zDzzUnu|^O53%hj_mCHBrAB$HP-yaWoU&c<8A$f^0IWA5y+;1EIFq@$z2LLVjrmm-Z zV877w5P9f`=;z?eIbpCUlk*%bD(Njs5I%)UZXS`8M1N3^|!d#hufbV;_7_As|RGpVEBB|6Mv>(h>lSOr`z&2{- zh7`Cqi_D^`i1vb!FP*w3un`8ddv&gGe*M$Q@mHMP?Ej*yo$H530f!d9h-YOQ>5Ttp z0&Z`+=U6!vp`UL$Ha{s(`&*Oe1fwk43f#mxp)}y9Rl2UEOe*Uy0*&e_I?H9PU2u^B zM)Hs2v*Z}XvYtGPrEVOManEf0gM09-p zOZY{`l+aYU1gx+^pqjhl|N1yf#W_qSceV=2N)3Lwq=Ts(Fvm!Kr?v`b~sr0 z+C$5)mRFXNbancaM7jg%eVycP%bp{uCDoA+?FI%)DBL-MNw~u()K!(d|DwWh`EEt! zr1yWY5!Xa<&H=n`CVgW|%Z=Ju=>pmj1;x^$NbLvF#onzDrBtz3IC=V!A=N`RTmgR` zB2m>8F{J`R-vMFw&?@f*!Y1ixRLIm<8dvlg-;#zN=h)eK-S)NByILb;q7dnHrDt33 z6WBbC;Tny99-PVdg8vm_qKW6Qj^R7mKb~vNgAh)taZAPxdEUW%T_Y;a;?gbiaFRPd zN=b-rk++j~v%GiK$gMumqR&JYXW=-F1d|uA*SS}UOmgzj(8EV++#}=46H%dU8==;? zcFdiI%u;9TDzn1EdRaF(lm@^Xl^u6PjSt=HIE8c>L;$6`59L3n%?-jyH*QI!ApmeA z(sppNA|TZe#ESAwi63S2hWPUO;q%%x^e`qEoT#l>>YM2E04C9(ae{#8n+azlVd#-} zG?f5#JdfTk*sQ5ZIMl6iArBcBljb{e4F7}y@B$e@fLmO-?J55L8YbQZ=K7TQYW8G< zYoB zBL|aQZek-+?z4d8^d!oA8*RPwBKPjFx6i5z0^bAPrD1NJeOsvJ&k!{KFAD@5qUb#YX(Z7Xq>}yV19EcCREWlQi3E+QxEvrfIj5+JR#{g;K~ZT4 z0wX|x+Rx)L`nG6{%VQ!tusi!Sn?|QM|3E;^YoyUij;zE>LuwK~`uTA=XM*0IpLo#v z(}Q%F6z%i2A%WMICg$5*lycNg9JeS z>{|*Y%S`oQ4V8)%;pu0jlAPWCBw*PF!g^hkORafHCYS>$d-6AvMTVwE-GVY>m1hWy zj@07p<1vZPcOJ3~MF9^<3aR;0LbngjSHty`1}I$Nqr#s6((Yofuw81TPUk)mp9M3i zbmq;%0du8l&vtwn^kjQaSN4o=*H4LvhYSgb$d{WsfMPk76N9dl0^w#~%Tqk;R=c{U z0fyfp2ZS^l!k?=xfl1?t#$ZYL=2Ig;0fi(&FLyB*-+1~V1?h5oiC=(sXH*nLV|eyW zCdj4DWGv(LpyECel7BFwlsgC(@<@<2FFsFD?k^M)@R~4DUBwqMkAvFUq?T~h{zR2m1RnWJFm^t9D*3Kd z2Hx9iLNEEW@^Uz~e%EeXtxCBt&nIDK`5J+is|RT_RJJNxf9a!2Tdr<`5M1BzIS)$u zviS3RV(J{GKX-yxAFdh>M1?M+Wd4T6^ z3CxQ!N} zbv!Im6xYG;KaVQqHjno@@t=HK-0~RBhOTw+2aukKlrQ8e8%zX4hwGH5rBWUI4mDD4>OFD%Slsxw5Z$Vy7j8? zgzr})y5TPQBi*m5@|dTZAT=UlIcg?XH5BF+Yj5Fz!O);JlTQO^uyog)usinYvM$aZ zf4)rBb{=5`>e)W>T;J}xE-|#0c_4i{-IDcjdFNAwEU;0{5;ezcIu9UtYREmxHJ;Ey z|6RTJyjdluC(%^88g2#7opLfVV#k57_gh<_4Ao-Zc8d8&Tjt%IbA~7U&T19vZ;6Qg z5&H0wv*^&6;T|^Cq?oUDY2%%vN$CaM&s*g;b>&J)p^tNzn6slB%q*IV{?m97VPB|GTB%|R; zxASq-#od^Msm=C-b&8B^)QTTNUn1vjY!}D5%DLQ|P_ZMASEaa&EVt4M zR+NVemNbahLsF_c8gS4Fe!E(5f`S>I@#|ZzhURVV#gRfQkq+23((X&QQ}EjC0ADri z5yky7YJ!T%Y{TOC{Qja=F>6~ONCXIhLT zkwKF!9lZ=jVr-nS@pMQfc9*Yam)Pbg3CfTf{=98#t&@#*dl*?B17D;#U3g#L^>wnf z>GdS|(ti5wI<)T*oyHzan4GfoVkGKv0j zRo?OApsRhTdF;J)ZAUq05^kpz5ed{pdwG7pXpmIxn|=#15aYd=y9d8xDhq!0fF;0@>}p|UF|IXeY-V=pDzjBq(@Va*S#4O&)<}!N2CDdR zi(mSY?tRJqCSLFVtT zpUGnIyP^BZ5eQT03jVP_uqam``{^AA_CC%fyHPLp*>cE{+T%xnIaoqdL^*bLp>5c5 z9ObO{2blV9TN)$RXE~nwcRCyTT?>6!DStDbDYFF7()@ zexONUBw4@SbS9DS)N52O$`m~{<$3facl~7veM#7Pd8B>geAUQ*KQxtdqURR}JnCOL zT&2Rpqq<@*v&}T0i#8QI1V;$!?!@1ttv1iaKEWFQ!=o+5zzK?wS=Vm?lGX&WB{-h3 z!h_){{pTHa?7dJwT4r#WcWGv;W!omRX2RKg|2Kteq0GZ5%83!L6Uk@$cl~BnsH<*G z5A_i6O*e=9qTJ}mfXT;LyWde5o-KX9T7XTw%=Rw_7;V%?Tbx3h(fNX!`X@{8LL<;{ zIZ0%iv4V;Z5}}1G`dB!^o7cK14(~bRxF9{R)B2@JYYN;F#m3lJX@+TR`^gIaMA>1k zlf?CW_ckj^WzD=_bJ|ic-F^lvDYG%0XkTvl=-B+w*%I6SzTye_#MAk66Vi0Rd`mvS zTZ~((>U!*$#@NSRu^vkY0v(5HoPwD7Q#It62`x8ewb)bXBHgnhZL*MieKM%rS%#Q^ zcC&6)Hp`m#l}csa_qMjT14zI|inhuL)PrtaZ7Z`*N5>Z%U6O%Mn62wd2#?p88^HT; zSB8E=kz?=Df_fpcn+5g&E$9$~ltQ{)-)0i*uv(Rx^dqA+$m@4<7;fsKaG1D}L+Tt7 z~mDL?_?qT?5S+t82Nr7mF3h&B{CZ?Mw>aAEvfp(zWDuR*9VuwKj(+?*_BP z_BN+L#&VOksOqKu*TB$&$r0x&t+zBEb1&+QtGtDS*o-Eb;sh~rZnpk)PRgrQMgdnV zV)zJ*g!qicv2q#pkrIZ=!XE^@zMrPQhO_XctGl&YH}c zu#PrFAp+)9Ob5XE$nh9uwbge`OQz}}wKnVc5Ic;7))A+(ONh3d_8Fao36ZMu*k<3x z@GHq7pPtS7DD@s5t9BLbp37*48C%*SMMD7`Hg(V?j05hmuVTAZ^P_8ae%Yzy+cG?* zA*nsg8*2u2jSee;x3%50SWG7dW<^R-?zStGx!(^TGK#Wc!iIf*2Tg>wej~Kru^64}q2NbEPw1DJp zID40H-WP;pZC#_WxuTls`aVN&?UceD%f`;a{@kVfy#`nyOS#yj1Qs*08SnFbJ4$@w zZg^~6r{F|!`^_|W0KBLHPGxtVS1Zi~u%PW}TH(d1DTTI~k=A1Efn8PWa3rW7jOo_4 z#?<+ATC}_$HEoz>P9AZ$o5=iffP8CFf4^q}bF>|AY05wgA5A=2^(Z75WVBMhaVx-{ zbL&Aa+&Gqb5Yp?8=_wiHLXrn0x{IJaQK?r?n`VU)ePOr^^IqO4#EP+gK)7LZx`)ed z)~iu;4;Pf_^vwqEIs(rgh4J-oPIDexJrlf`RY&S~aNqNZ@HmV3rpK&#?%gc1ML1rY zPQ|awIA+rAiTUmaVQ$N`^9W(!zNhfRUnT#1fg@H0mgzJF`tlw^3$A|eVL?VZfD?xt z^AFz|scj-md3|{=km+=(i*h3`6Dd#iCiyHS%64Or+kAa;XL*~19=K9XCrVyg6h$f3Yi&8Tg#&7N8o;}#px97vsv#K<7uDSSPe|e_HK6S zu;B)1gH0ntil9{?G8cNaBqXfifAdD6n}=f1Pn24x4v(>4?a*VLg+87*kn=D_SW9l% zi=xM}r8~M0DZ)@Ctk7f>`C-dy@~w~bwd3%i06R#X-kSM&eGhYDcK&idwzuY;+;^y0i!^ zvN+bZmMTVN`k>_1WT#Ts6t`I?sixOY)*@PpV_dy-0TQm5y2|NxyPfv$CqkDcrF!S~ z9C|=cYEmUW;rBjn*y`{&e^lA(#Ll^0TB((|mH^Whv+wYjsF6NK6TI4F^uw;Qt2c?h zdf-WrT;IjkiV_ItK5J~ED>$9rQG`B|U)c-PZhBel-Uwct@Ly>U3XjUf;XN(`x9z-E~-sjD?G=VIJ}@;};`wZavBgD}CWt3f2}} z`Ks94BG+#2PDV}lgDDd0=k-)Swn6r~w913Yd z>HD_FlUG0n2Kqkc(}_*{PVv<`<<|D6xdC2TH&UtB3W_pKEbZ=a6)NRT0t@=bFMj71 znWD4RiLteZ>PTw?>+GiMb(2^2HvBisR};e3M7_@+ulI$&m){JuvX^N0qq-QwSR&vR5RDNi%mTFnG8accNMiyA@#W=EZ8P-g$06a-(u7EK1TS0u3$wTx$MOFg(U*ODZ;H9N3>u^4@GdP&Knkd2NmMBB6W=}&Vo z{;M3jI1qKPv+f93B~k~?Ls4!FhJ}0WD&Secy?Mzh5c^@m%@aimZ>28r6WLN|jO%YW zR_JeO^h5$I8THUiS;)Oe{IO(~L=1dRl8>UvF}AhOKgLXDvAltv*{w%8zmsoHTE=0c zTvr_%?v@oW@%lEB$HI>^o9ql3@KX4#baj5xrgE&Uefu~=j|#2s^N7T#%sx6(WnG&) zyNM5HHSMXP!wa3j;rN`|GmRKE(B@!=u7$TXCt7{OsJ|IYCSV4bEZVOfxY&u#DB|y9 zP%Rn6ErD`}p~p=YI&Tmtx44i8r70u}vI(-fR9+zIU!|K9P=k|wkCYI~ox@kA|7!Y<76gD}o*fhHp(aF0oL`7peEN&3pjSWa`A1(0tt!HU?|OQt#5 zZQS;Gv=#9~RY(24OW`z`&dw)BuT9A_D$~MTDvcMpc1PiZ9PNCUSbrWn$hGeFeM<9e z5O{Mw&dbwo`2BF|9Gg~*sNz^IAKs2L6mNIZ3%*RoKInxLV&_s9 zb?+l86zrkB-FTdB#T7~%p1_nU=wL7`JtxJ?R{W;`Ceb31iM8xIc1xR^D>c*^t!bH& zey0KA4ZAO>?)M^e51W(2S4&91QS>Mv2!6(`GxVrwbN;DEyJCR~hIw_+p^EuvYWVkd z`BBwU2W)o0Ji^V)%Y=uI$bE*dghj%vVH;Aer;lc*^kJ4<9NLAV5vMW-on(3nHM!XD z^LZo^d;aPR&Db-irJx9#IWHF$VuN{=k7#n*LSxlm$gvbxJaoUmiAuFeADAx2qq2w% zye3i>VA_o~M4+>uSy&ua-{alx;;GJy$p3bmLB6df&?5tbA|Ra9Cu&p3O0YDalpzS( zq9(+q=ql?BBAEI%vMAE&?cgUWGOv>YVgw7t&V2l)y&v7(sgQp4fxEzr%$c6!w*fGwf09dM*J9C;4Xpn z-TLG7)**TndoiXM`SG;u6blSsJ}*P^dYxuIQ?>CF*ZSv}KFVuN>YhGwDnH@kJ`9OE zs#VrcX-;;&@UZPlsjNhmS{DhK9ZZhrS8xgb*}YlO6N3(tXNfC6mV+7x`~Q&w(~;VL_RITs~3K3RU0X(W_c(0WcO#=cb>ZDgQCzZHHP(e}1g zs4BFt?&5Dm^&*`A<5kK3CVn{t%;SWDOi-RRnVHmMIk*m&>Dzr#RyaP>4uvLGGia9O z2Q4>o&n}J|`_}t_K1{2kkQEO_EHs-hj*hB)O>~)N|HY!>BO@CRl;j)u_cMX(A#y{$ zV70I3Z(t8?F=$y`MDRuf-bxI8c2Q3s(^aKxLiaOs;o;(%95E<@Pi~3aPK5n3(c0_C zU^fgf{d=v%5PX9YzXvji1Q2GV%Sp9Qgu@OHCpD95o~cBqE2C>*(4l#v1BbKaN^#Js z^BpU=F+SP*pM8ngOEDIRQywnN2I~$lS1I!wFA@~iI5_S$g*`mk`KR-Y>70p9!Gk|}eHN~>l-5zyWydo8wPYLsmF07Jrbp9Szio)!57 z+e66lv*L7}y7*ANyn;g}AKT}(q#IjNYHzLCYkb8TqU~J-xG&pbX@3?%%Fnp%@+iJU#ralFV9annw1Z@NonFZu0B$=ym%^psc z3~N>svE~mHBWpNusF7MkgE(PS0nuf1-{W*{!T??xa^&AS+QR9X5m8D0BIcuqqwxLq zL&#DFEgB}I?QP&h{iIKUHz7Rk^6xQ(NL}qW2~an*v_hXlJUr4{uMKj^QzD%S>IU$M zu){~23$}n-nqb(USC~5ch>mW1-T-5J=%srTP#sOGi&|Q36 zgr%TFf_dlMKEk8#N50rc58`!Dw{AALtoD!<_BikEVJ|*)pErelDsaWVjTVNd`+Q;P z1}&A7V9b9yur_aAx~=K#;&|k<6EV-!{(drz0@0xwX0==w3D-qgA@U0LwfRk#38_Z@ z`;{e3AJlg8t2Z4HrbVQ@6SU1ce48cu;^y{CJfXN6Tg+X*OU{= zu1nL5%D!UlBUEb2m>#OXg+io}UUAE&1WCpF^KUOXO{)osPO~o|OWmS^mE&c49#aS> zlCP#Ae3S#^BCV;OkmIjfpo*nNt7s9pl2F-K6y)vHA-t>|7H-wxw-NQYCPX6j?bjk( zRLk(t%*rtD+?Pcmj1!25-=ea#!N8&v5 z^^gH4Hbm;zxTw2HDy9HHn~+>fwZ0rzv|ACu?ZORHb?%U_J-z@uZ6D?|hXuZipme0( z4Xl|v#Req7;zO%_h=Vp|f~rv{Ipb|iYH<8WS>MpUPta6)h=b&S5!HcKk?kgp0^@uM z(kDvTx|Xrmrje-EkyHBJNwO5H$vZQa8dpE#5ld6wQyG&kHuxU(g}@uB(}3}xC_5O( zr|lEgA_UfUMgl`a{o;0|1RDaH$ao>O)P^5febK)O{Xmca=HtH@HwJH-FMf;7s-`TgxAdp_ z;hqYLSHG`2)I5yG8Mw_QrBd%5aWr8*W4@ZItAm(fs5^t+^7g-b1*)8Qk&dkRee({t zp-hw3V;K$Q!+{>z^EaJUsOBT@`tE|H-%_R4+{Og=Q<@4*OoC+3R%S$p@WR4E5}yc! zA(rx->iA=+Xgw&pYwndSq zd-`34g8yQ4#e&O^5Kq@f8JD~Fr{`f(??9o}1+FPfyaRxT8U#wXd{4$xtXN9Lsr;Tz zqIp%J!f?&Fd%;pnNPh@!-CCj)bTjPdONbEJOQM-e*}?~&!#R#*+uHF1TX2LnPZ>8M zUQ(^q5-3j>nf7L5i5iZ;bTu|dzZ@>q{Oe92Ebs|Azw5Qrd*v6y+pcNikixg5U?MpJ z);md436x0&w_*t=*e(Umd)f+(N>QA?m(mlwIVRaihEU`vb=vPb{XKkr?htWMNYxIu zmMj)`m+TH^gH_rA^&^TF(QT&6J8p6b8inbFBuq&mfxLwnt(q~A8DCP6oB;a_Gfamv z{|R%a$-u-cw@V!N@vvcU>a%gV%i)z@a1}bl(rzz0Q0n4sWw8RG`IZ@;s%!2hrJP$a z@GKdGu=&}Wh?2-NyZ#+Y_7{Tzk-ayehb8>k+yo2+rXjk@RbsViBB@INhtQ_{++3ZQ zbF@9T=eh@P6~Bk5w8TNmAQH~QAgk?~JQpnEzDH^fO~r5xbDgX z9YT=4?$6f`^s%(7t4Te&>WN?Wom&;Y-Y3`d+_ocpdeKbWI4!LPZbzq=N_lbK(^W0Z z-)1IlCT37;-#hIR;J}yqmh2Lid^Zf`y$mNtZ!>jDu3%_uW$~GU2vGHxsJnCY*;B>T zjML^f!^+afVHW+QqbPY%J4NY(l;z0Fk8^F8Iw{&nV=KlHX zSs;R^KXz05Scj$OdI+BAl~#-(4!-!B9-6tar?($?2O1^<-A^4^%yx<`jS zu2xlj|Ms)v8`=-uX!L~2oa6=jjhIVkQej5R=JVBSutn(9!0l-~VNy>Ttw%#f*bvMH zhT`1cE`*YE2|Q5 z@aWn`5ghSeolbSrddBD8UCzCk>FwIO8W;X;!)zYyBove9=<$(6c~qYlWYJ%vYq40{ ztVzrz31}Ax-?p{3GlwapKQ)JgjU57`!jG+lcpt1jhU_tDZJ>|-{@{}{P?R7C9=@lk06C(`0uZ{hLsX!7P=7rJ&9``4K3w>3`j_#cgLAyJp| zk|^{y!;h=?;^s`h_Y$!UTQE?A^#TrwH*Q5zEaXCYKV8xAQtw`QlvOLHWl*2QyQ6*qYU$} z`DGpVGGp5`>Jlng>0&fUtsy+Z3U)l#KD273 zmlAs!!Tlj)vmia&f06Sraw>N_avJUCSe@-wvCa*qH3igL_T;v%pR8GX8WnNiZ1KC< zPZ1d!h85MVKiG=48vU(C``2u;R&?qO`nTmFL2Y*VEmP0nR>Ey-Gs{)Wl4c`qIpDzYDW zr|}D|x%0f3W>6v9k6{n4D?-w`LjP^@}d3n~Qb?yf*Kd{qr{?5%-lVTvA z!s`|2juY#$fbeM#n^{^*?e?9b7y`h9&u;z*MWR0Dx)6YeG!KA!cpI* zoBp_e zdW78L3?LVqu^0D;0lv;C96e5k{CWIL;y#Z%|)AEq*=8k(V*12?|Zh}~9)b5#s8GbfN=54wWcIq5|X$-nrbAGl+H{w(oc3QYO)#>^6VI_+~yE9yxTqKCw9hBqX8|0Yfuf zf;JwAE%u&!gVo^smBdxt_N#8V9CO8dUw=*Zl&2zXf2bp$fnVH$!!ktp5;P%`XQ3VY ziM{h^t4Q?PI-h_b$X5nW{}g8!XS$3zYZO+i#iiCgayl&#E$D7l3@*HmfKQ7nRtz$9 z4u-n=XU1ZDNM}=SoXSo25^~n#&hdODQMJ1Lmu9DU$hO%Vfz#!^xIBq1DItl(jQMaz zz8E~XPfK||^0urdv8J$=_=MHfpH0}}kXTPQ*YBQF-0oRUU$1cz%lloR{>HgEv|d;b zxY-)d*3Dk7g!t1}R2$i&AVj2II|7;q-SMGy^?M*aZorB2I^t*GJ*75{Nb@_kl7?{2 zx(QYcFEs7!S`7s0`B$%3s}vjWWbvN1kJTZGhZgj(vM9!I^ihCV*ypVMA&T?Y=N~P- z6ZM*kmPdAjSu9N{>(Z#!7wnIw+FD?;O5DZbE(P&H4r?5@^0V`a0GB$nQ z0e#?zMQu4_Ye`r(X!)aT6xBnhqe+`wrFybcfpn_5H|J$!I(n@DthZOMitod`aO~IQ z-Ll=omd337{VM)kM+YEl9&?i5n(og)(bCMDghKl;`UL)^Gv5P^!79vcWAq9p&#o-uuN_0wpB$+-rPNF5h_2Ba;YBj@H#X=k zHvaVmlQ~l&GRnrTuHEw6P8;M(6lDobJ`-r#p(pLs}75`%S3?ls(1`N6;qp?8Ar?ui*6qd zfrZ--|7^^~_E^a1Hz5%Qhd*W`H)0vT3)jVf(aB;h5yBSROqr2fYRlY6@`6YfB&6_enVzIMb)FhmR{fB zFxj56j>aYKuEi;Gt0gfZM!XWxDs~sp2Pc7)r^z4~rJ#V))mlil{rp49zc{%D!NDyD z;aY~ELjot6+O){K&ZC8w8xOy_%{!oddaBV$?fo;+QTaXF;Ma98_gci27VNFof#FkQ zmR+@TFbm74|2V;|ul<^;zV)20`53Xs9y-7`?&b0w6&%@^M|j?P8J-UV{YAof+jB}r z0oz|Oco+kQuM188*BIg7eQZ*^ppbj`{s}>P*`mMackV=A-Zp6{EH^t|vXo7i)5xXa z?>8J?fA=iz+!B8f4E||ViA-lMUmBA!ro<)CUYgp@cJc-sf??M^F+sdYH;*hf_mj$Y z^s;BxL}X9qot075LBCkrTq0Bkit)IO$?iXY0w$oS^=b>w18yXxi`rV9X*s6cj{-&2 zF}$)K9KbGW;44lSFE>e1Wp=_>v9(Biz3X#WOcd4IKA zRc3NIf+3oaN0IA1rK?SVCpv2>C_idenCBdMe?-#+9CylXihmLu-EP@0)7ba5DU4Bh z&@G7%pEvyd1N4z5^o)i5GNikAsI%wyPRI#ErZy>HA8h zd&iNM-RdG+fL~n@XT{;8|DGPqk}8@w>*De5)1srVrpcp(Xy_bMfh7XtvALncEFt2S z#Lj~b5uIqGh~Ie-5V%t7Umxh+RGE>DM4czJgXJ9N6&GrC*@({qoKs&38Ag0D{%Hc& zUTWa)A8aQhQvPLr=)Ff?L0iH+k}rq*>mHJ)#?EU3dH>ShuK4vl61w6m6q;`CDvvQM zbeMVcrOn4Z0}W*m#ckPx%fIzUJj67^(oW$K17iFYK34vEBo))6>EO5H$0p!#qmshB zMIP*GHSka4ob8v%8`Rom=9+WozK8YMV|u&Z=}4BbI%C_Tx9ys0$en6XC~-VTq+{yz z<}5E9f8aeqdY<50y(?1el~dSI>&6h$w<>vA{cpR9nTM7v8q(WX1a6N;YA+z?mHXpLPa|cU z?;C(;A)1f!e=&=?prAo0_Gvt-n->JMkjO2XV*wwb2gEQ-uURc1fJHdLCoE2{c-c)L zlX2f8{ye*Q4Sc}+X%ygfpc8fN29JlJ0K9jr(T<9qNxo8SC{)4Nh6)ky`26}a^P+9P zre&;9W#@esb#E>$^E#*S9TN&#-*d$!ELMWnG2odE>Fn|5QjmN|z@ znVG3fMpy;+1UJZ!aa_Q{n{$Zal4{4@o68T~rJfz-AqgM@&7bUu#8~1Gcky|%)O?+E z-UuqM$Q(vfnw5E@qJ#F2=5e3;tRo`$JjQpH)8(4%-LOf%3;!(fFHc0`b@nYFdjb0b z^R7%4{F5j74i+0cbqu@f4bri-y%0~cZp~6gb-=qI=aSqblkA_R=qBq?uEin*Q7sHt zCV64DBSWkV2;#ygIo+ycqUz$3QgLfUeD`p1hq_FkNx0k9#9ZjeZwSd4M$u%KeqDI~ zAA4^V7e~{*izY~L3qgVmmV^X%*Wm7M!7aFZ1`iHFg1c*i!_1(;eQ+np;4-)$-o5w# z+`Q-8$#;J)n2YAu-BoL?dTLoc&6n_oifa5NF);!Sr4~v`j3N~$qs#<8bj>P*!D(|w zM!y-$xnW2#{-xC!K29X4PJUx&=FNUq@h^v4z*i+yaa8Z%JSFi5O;pAm5<*R8jtzPV z=Foc+Lk_;CPTW1csknBH!8cwH)t_Qb2R>dg1cB<)jeO^WbH-%(ut&GmV9Z-gU%&Jj zpc>Zesxx>x&Jtfg|CmMS(Akn%bZYr|=>s{0qST0;pf$>V)O≷VGH&=yvozmWBbG zjE5U@^qOiVtAS?dCu(tk_F2c{!k1 zu1})xMr+OMPV!&?uACW}8wDmm{3&=_coZc8XLD|uJ$`>6rI;v|M)JIn1%PTO1|dH9 zT>obM-2}2ER*&uLYxSZ-QJd%59d5zBg|$}X2f^AWheKX$w1Al-1skww?A$W00Ltot z@BL?Te8$J1#+n3T9u#f@Pj{RW<&TG@#dt{6!_f=PpwlfA2f9?G4Hru)zzp{eKAig1 zSIhpoie7mb=a0i#8Qxe$9jN83^9$-5B3rIOzDXcqD=4(7aXN4-6lZXV8rQiupZw7a zctJxHNzmt(8CEouWR814v`*FgL2@=s48}3)GQ8XPE%tBI*c(4&>6kuDMQXT^K)FM4ydT`7u&s-hT9;e@SX)$(b6 zse9@G3D3%#U+9BGxOIS#?4h zsGMWh?4X8#T|8Tmf2P~!wIEua4tsd7qTI|rQl(yzP=}rCXN*$hpv?UNSW~fIR0@F8 zcic9B;Ra5$v?I0eeW}I3=#YDOFc?KJcHnU9Xn)Hwy4S1NcrZG5cvkqX{J`r8@xGs( zx|h)U#LS_?O9q`h>JoQxryD!Jm@$j5x;N1tN=R2u1E}gV7H%z;EOuhv3%A%g_@x>wdKFQ47E!=d_6+sY+jnXp|zg+o_KV`n>iZ0e^X7+5BVq< zi5Ox?FKQUO*-%0%h)4ME_B47L$BQW6<=kf13f1w7uo=VGwSJAD{TTa_^%v9EI|}lo zFYNjVVc$EgqD0$3ET>JCFwYoroC)O%sP`|DA8zy$E$0#>GMQRP>v%MHZa-MIY~QqT z(a;VO_4OK%TgwR!&Q`W=JfnV9e!9$LMR(9M(pT4a%DFbHRs8~UP7Qa{^`{Ie_p1>; z6S6g}lm9RbBgR?5;wiIG3 zV-nuWL5QHrXz9cQ^szseGJIx@$rRV%GANX@uvqe5;=aCDdzwld3i$`=m;*smFA2$= z3u7A=F37&sLC;;(VbiLlcyBb6Y=jjv>p?z&Z90u*lR;vGclTi%MtcS~g)aQuP>*aR zFP=@1+iTdzN?L_{77RrJZrgq@QGn%Ptn!$-BB8{{729!WY~msw>&sxEa6rHYro>U% zGEe^pG~5ms8Jjbwxq0U%vg#OpOi^6EZ&Xz@j^J#H-~od^1(nOfNCqm%t(d%=*p&}& z;PFZt#k01zFQbdJ4L8tHe@W9!_d)`bsZh|IqYLy-iz4;<&kp1~c`hC1S)g_M{zD5b zV!D2hwL0DDHTK0j2sF5c4}jQXINuV}dL-r1S)9=q&^ty5Nl|)J0S~elJ-ltK0(e*Z zZ7@g=Tx*odrL`qEH;O7tF=RwHMfK_zwGl|yd+Nrs=5DsNeDWiz$Nds2;JjRbF^g$# zFFJcwF6>g;dV|**Iz&vrjb%j!Pa5hN);cY=8uwizVHAm6zB_npYTw>PD_!^<0WQ24 z%(;K4HK^I&I-ffiGNfB23=E&F*#H9G(E-s4gM8@%S$HjVq|F8_~NMEx}w}*9&$8Qf8Q{wRPf#6a>Qv` z(+s)L@`AH_n!&R5>tY=DEs~CCZw5?9!?U_N@$eN^+UBhMXsAaLyb)dubIoFNBW zkMxGJ_f))?%;*knfr-@@_lyo`R>>_i{h9M%c;lf<%}mrOpL15xX@@@VUAW^>)^x0v zZZ9bn%gFAWj4ajAMeO#%f~d$)ET1ysR|-6Z+?t5vZ7m}!T_e@;STlh->%I?l@x3I| z@fR#3AJS=6;nbicMvPBa-_V2NHQz?23CUjKW75m(adi93+C=m}?jNMJP3_fsO2^5! zwU)C+Kp(Us4-7N?P640|SCQZ^*SH%Pf%T2J53!nbjMcz4g4N0|hjSitLc^ywCmDxx z+-CKHK49Kx2YpNDlcP#Qr344H|HAvRz z9K*f6sH~47NzMs4D`C|u4Asm=0eBhyK^pFxW0Z4S*%+yExM(|U2Rpf`k?hs6MvZ)r z^wzX?agv2369!E;0-#wMGXpZiW4m!o#ld)NY8=D6b3>A7C8OP)E~AZ+7m5z{0nsHW;kZkUI<3wes(y)^qr%AVAMen4CTH_{ zHslU>=)71$xlFnUy@umyg@}*xXm3FO!v&Bdck9kQ9=IY7xG~mqpPL#genLb7TT<%z z%A#Ltu}9d%qYEMT^Aplt`KAK8y^owX)=M8*x0ZB!H$RWF-plY`${I%*S-nv>wa0|L zhVfsLx|Q{b)t7(mzr+dy@FL@p?qOLiYukKe&`9~Q8SY8qaT5iasY+$_)mrlxsif0t z6AokV=9bQ#d#U(xIgfDuw0Q$x_ZQ7ujQ;Wi13WCZ4110q4-WGJ%acNE!I}Q6gFXLA z-YzNsY21#(F0cDp>t&&tJSOj^{L$Un9$)ZoNswZO)T3Wc3#LzY+QD@sH1NPy9LPeJ?{(7@ER5R^ zjF}j5V)SJ4X1}|Th^zi$wHsioFXhbJ(?8hZ#Wo>WRkf2tkq%+$+JRf}Fgx_d)dqs} zhNA>FnOTeXWX-gx(6HRSdQgy%j4?S0OniSL1;3Rs>yVcD;H{d>brY5mGmo+M5&=2b zcYPRuy7s^T((;%gvOWbAwF>0tZ2RyRM?x2GaTe<&Cj5zMtZ}D)GR__v2D>q1GdnTb zsKiHv9-bA|?}foSS#!H--^pewK@uv9N4yE9D`%?)A21#AglK5DXXJnVgM*K6;V743 zarx%m1NF|M-#?@_mtmU#qlE8Z=j;}wbw7`UA=xhfNQ{=7YK_^8t!js ztu=psziYc-j6yw|o}7aVz^ymXu#)U*8cL7X9`}nJc9ndBI0>cu_yq`>rPQzvpc$`bU5@|LNBJ%PiNYyzTw0zECzc%A9a!b zI`(2;aRa?y+Thc)M9B_LI!&WfbXjOII^4L)kfL1IaLfLw6YBFKFuj^2JscD+u)FJ2 ztyPTWU~ln+ko#C25&BMPgdZatxQY8+Bl>Bw&-QIF1-Zy$zg4jWr(!X$C3v0o$r8Jr zrLv8JebeqrRbP7%hEgkzs?%4+h!B6D!F^HW=~*K0p+}G1fjG0E>8L2<{`K;0)fl`g+UPEQC(Ticu8HSz!ol6En6g^#9w}ZU8pSx4bGI8#=%1YaN+gCod0kb1XH-+ME+eym1hU9gOBh$!_u@$V2k*aA1g0h0uo;6Q( zdkXsd44a{rc{5rhx}qt`CKkpn*?w$`4W>Ph#my8z_A>`|k$;GE0ko9StQ7D$+D$Y%j zTMEg3DksvkRbKiAkwoA5%6!rMHJ>#@bjoK-EBQ0O7-szcizt;SO!P%#p`1)W@aMgmL1IN;hy_022h_K!xU5Eg z2A1ODNjMEFZF-~Com0m0o!G^L`;_dH%dJ-f?E*H)Z+XCN{lJ$e`d`H!N(=-~6!PZ| z$KG!zStZPPGOQkU@t%)Y@xI2ENs8ke_T*7YTaQSHO8vp=wFzZ9keA=+#g+5}bT0Bo z{)4a^Za2iiaWfNy*eSOYOg2){)MDn+)gw%zhWTxW^GAzjMMpmzg|3iHYjpvy`85c; zo1xvt``&yRUISaBmOU?wfUt*|gSa-bb=ak8#&O)#1-a=WKUG$lnv(mknfo2>0zzAk zYEz!b_M+^qs>@uCsE9a9oNP7)PsM5ibUQTCfx zrtct$j{9=aQtHC`buO`y9#xSe+JQF@H6L8_ZIYsr=_NjgpM>qoSwP z`J;R^_;qSej0V45`1YFcB6Nu%Uy+S@q-`?1^G7S`gPZh1&%`V1`g#zHTTB$c<502b z&ySFXbSbKyqhG}L!-me9aUR)zYl14rP1eEh;a>6i{0QVaxIF{k-o9s3=#1D*i1W=& zAjx1jErT#Qso;uz!+q$)S>~TdDSKb-gdE-rbR+K8*X5i=x{+=M8`=cuo2iOyE#asyjN0@z;VgQSkN(yO zYl6s#TMO6}Ndu@DWMSC#8^UdIWi@wcu^I3*eA9afqAK!fopHc(=Hp^o`O9ucwTA|7 zZGw$QwnL)SAk-JlW2KTg&^<)vs@A>ffxdpWmsFn(e?=dsYho3c&#B^Wf78!0#huTP z2nj<*6|Oq3LeO);2(a~S_<|U-Fk5(P(^xvf31sJpAT@NE2B(|WPA(X zO1j;hN!DJJpY&Ev-)L`^Gb3@%AwJ>{@5G4nfKlnk+Q{d_?hg<>+I395J7C=@ zK259X{Ne(Ed9;FxpAxj0WpK=0rNM&pbhFhBovn@||1(mD?gmb7m;x?NYfQRv@XX$a zK3gW2`(pF6T{k$~EWEAc3q`z3D`37g`Kd~kS&a~`dko(HHJ}1p8Tw_&&L5i{cae>m z-0Au0=Uepq)i!U}c-b1WsfH5Xz+?c=zD8VI0^jt3R+n76RWScgJWj%W{x>*l9goJu z%b=JC(P&TCY&z$Al;6!4ZwGHX@d-+mxXAgaJ^c9fRnLG7Z)G3{a}v6gWx=5r52R6Y zq3zGa8*S%@Mp-rj$CW&t^M-s5SxuGeP+?;ZHq$s`jV6A4r#d>O0KM3YTDtGCPV6%r zEPB70&FbV_4rm^mE|}+;RzUd%J@=sN@c!3STY_M(TJOH|fAClTsXhjCrJw!=XjF@% zXmk$(v3RRWwnBYP?n$PT@($00NG=%N_v7*7;@N@o(+`vqEt1xhfHG4zm`oahzbZc|V82&Wsg?@pVGFubG|9^vvLt!tveRi=!4Gv0D1IxbTN8?g4QP zJ8arAPab*R|BcC+rt*A>9-}zIrWw@If%v0?b;SU#1$7wxx%TB*le@hUxLT`S%m`VV zJU!~(`+GrqhdOvMyLYzU2_6)*(xz24s%=lz(?>yu9L5}F?!$D6Qve>Sx@r+_CHHwK zO93_dM*D_Lxz6YLV?9R?z1PDtDkj1kRRHjq^>K4a8{((JpVvjp9QI>K_)VC(| zd>s{XNK0J702@3xZOE4IJ$x{Rj(P7s<69^q9do=O?~zaPvs?$;b?()@X6b$(R8r<$ ziT&snedk74WgCS;GntPm(j5#8U#?hZLisT~9i9e!xZksqcKIv7hBt2@)bQ zqnyV<{6kl-j4y5j&Ts)I(k+gUTS$;0Dp}S<;Y_yHKz7!+~W z;)+ci>uCgP>&e6nBY!^LRA+?DR%}L3D{LOP4-b3XTK*Esp#L~w%Zi7v7_@z(C+66Q zAcmFiOaICzXj~MwKk(AiW(B9b%8{Yr48k7`+t+tgmLqZ46bt@z6dE`AjzXZKuZTLT zulMQT6z^|46=TCwKA6;%8K+InuL?v_|0hz!pO_~4ESN}j+i-80UAV4&5;olYe7hG> z!d)VLO$#-<>UVK+_Mz2VEcD**N5xL3(;TQ5pkeVloo8zgxjK>NuevYiS+K|R_LswO zp%foIg@VVYoDpVRpUr!`>U=(z$K5!3tbDpQzp`HtrR1YvmeTmv)Y$ijvH2NrFx6HI z`F(u>ec+v_E(fd;-RDB!Oxve3U(02D?OI;K1+{Cf#o2*8m6Y#&_+$0F(hf#B&_$ji_vTX; z)iitOnjL4l2wG;xp+X2Nn(N#2|C$PceGv%h z`>5;tlt>f%lx^#pPIqBD|GZVMYrWCy8VtB)DI(UkBj;D@%mL9xm5xl zk)5V$Lci)u20lo$42{x9byrBxt}}^_D!EhOT6~2$Po4?n@K>v0ugfI9Vef$(+0=)s zfVryDI)U0q}PxZa`C+x&NqpwtE7kXF~di|1VJ{ki& zf@hJUfO=tXS|fyaw{)xkoNEH>w&XTxUvi~iSFWdsoLZD6a4MewZRHn&TZzSh;5k8%cEqk~HgJ zgvxI)Vk!o<_LP95*<_o{Ce9PxE3olfOPc|{GY-*Z&xpX{jK zNt=H~Wex(JtR=Kc2a6}gzHB?zPe@F7xAWn-GoM*8L)m?WNV$i>*S^s&%vTgP)_hF! zsKM3RUx3&BHQ2jXiG7*bV&Gt_BOGHj39qFTHK$&O0N0#{8XH;B;+>2nf}8KFlym0; zwJ!-X1~Jti%;)v5KXv@9AWSL73v$?ByvJBSowNKKUH9 zZM2qOJ#-=ugsBvxo3q8(btK!dd0rs&2Z@eJ>2ZG3MP0KSdS@@P-sdCJ1iwpot=fCEH5bR z{qlg=Xig}cEpEj#*}NW$@4wXB!di)IVgg-qN| z$u508r#YtT+^Q8x7>~^9(8`8{*i{=2p14ECNZv3DfZKBTKnOK2f?XgkbmO7H6ypOb zJ&#`ExUc86O>Zf;>oENaHa7&YDc8@Bx?pl@LGOS?bgJ8x_8RI+nkHw6ECcSaejS-U z6WPbSq4RlZ=fpy**BzYDhrwQNDW&G|4vG1~G*T!kYIk0<3>Mq)Z3u=ZUxzyVhzqMb<55#s)x;>+G|5(<$3lW?bmd_1hQq+}#_k z#!Nq)kNAsw31QewUd)02nXZPCp1 zDYm}Vaa@fPh}!#=%zRduBOWhrr%-(xt7#WUmHw%~Jv~Vj^@u>PrOA3>tHr%e5LPOL zlb>Vx=~clH2YRnO>euxX6Ai$96eFJl7WLw!h@K$uBA0Mt)Py|R*%+EMi(*asx}!GJ z>#2^TSWSCC3i$X2$zcK703pWthS!?}p{FPgTk# zCz|C?+D_mPl5BtPJRmWub2I{%;72_*> zG2<657IQ@8au4f8dsEJ3_uOk!@|305gvV|Hu^-i^rP~y{z6{!}e7K@Bf)N)P%M z9x;JP8_^>MJsN!V)-zRs-Zx+dlMEfn_8&iVA=`4hQoXTlBDz}@8JQrvBT21Rs<`0AeQSx04(D;(b~CoTD;OJ>eMzXJjjqYe=q0-)CiFs^ zO!Sou5Op`@H?!>km?kW8xJMkb<)*Mnx2ZuSzd+1&#`h)rOgKZ>dkO@Mgx;dBiB?;L zo4LQ!0nZ0pv8e7?hyg6#KMSnByJuQ9B>8FFVHmD6xBKYJk%5Nf{EsCBMc;GYO~H;q_cOE%(N-Aut)50 z3F&+}W-Cf>4R04|R7rBRf6|w)AsfPK9hh~@eyQQe3NC``ta`m*W8-Wm+$l|`&ZnVp z!(Pqyt(STM`n95mkFVF%w12+M!9gU5qtJ3#M4=2cRMck1!4q>y4(?<>pcJ@o_;h>| zt9%BcZ>M9$3rr@cJ{7H_!*RF7Q|Mwhs-yjObDW{2^7y6#g=pPaKGM7yz+3P4gYs-~ z>*%_&SHD)I&MS^TY<+n(9(Hs2`P62*l=$_#QLSo9db~N)!V0?>6tkT!ZgSw?vSe7WKRQD9ijo^^ghvFp1Ig)UZyzl zkk<}py_o1-w+{cZ@tS0~zk9~}=l$d%6lQ-VIPP_Kr}!^EK*w9W8q;nFXy(TI;^v`z z#Eqoh!JTXU)tsGDCI??9l~PjDI<7(HN05u4L6yY#M@d@fz4c_vikk1z%F*1#FleOy zw};@!52lIxDPR#EQdl?TdZ!X3mHy7a=dcl>JdWZf3*=Wsjp)tW@*^y%cj z!fx20UBK4G61*EEN6yaNxID}vjDqY{vVLrc!=L;vBw<%Nw+QPkIiKpkKYl)Sv~|qLa8VSZhRQpOZDx^@kcs-Nvq872+!KK1I|4(wv`_z*VCuFKNgi-s>$f>&`Vqf+t@%#t-tMRzSnylPP+ZorvH9+d^lH9|lgv z8^!hM8&k;&Lr2f?i~5NMwMdz7o=5MdSyM3SQUyGiPrpsqQ1UU5R9dPk#z+FLLX1~U zUq;}*8pKru$MaW!mfGvW4wI+NWz#}G7w;5DWa2ww`^E8h>N#!YUSe7wEfyDRAKB?; zNFuy(Tt}N#Znv3a_Z(;Q3fS@$9sOXmw4#@mLLUuzyQJ;vjRAC7kIj<$Bc<`N)LUiZ znD2;qAwTHIss=Xt?JzMxKd#8eQy-%fQyNYBc|`$$W5?TXi)bpUKYQ*INU5ZPzNsn9 zZn>f!?pXULpkoxNl#j%Qd9>DIFFbj zcYpihX-JVfDd+0sq!hwJ5r^T#)*BfGQA5C|7b!oYvGyl<`SCLFV5vx=3r6G~2ATEQ zjlz`t?)_Ai2MS|9uP>N979ye6i$p)>8VGsNteybiTkLB6yJr7yGC<&-G$jnl3;#Qf z0erLatpuG%!r0X-G>MT*k@X)>+?9Q!!4)$#=O4xnp=yX5(z9>ojP5vPB13Dkz65mO zrXZX!zsx#kkGu86`Z!$<^^~^w!wOuL3A>(cF-Ma{m zblTEit0!46(NqABc_!k7pI_=kWE@|8kfH18QiHGwY$|==*!a%uRNeREm-MqXJZd4X zlK6c*z{;bU{|W#3mx3EpQj!UP@tMwl!L&(PjJAR6kro^rKSC* zRKY{W`_P2i61V&TW_|3a1Vj-aoZ`96RbI7S3)_lS0>WCo+iefMgS)u1Wv=8W18_H$ z!oEJ9EGPWdLZffJSX-tN{x9UXnug!!Br4>PMRFY1Cf50Km8v>-V-2rbx5c~1sE>fM zxMB1gzLT7QA|{4N-DMZs^AK)X-SU+;mEtn-*|-T0ea7*)nOM071ox_atVWXDs6<{< zb8Z^<@?E~?i8H>ZyC1TVR37AWt|<1NAKlF#BC}@xJ3Rc)bl=WjL}D=D<4*-~$}r#e zNZ0y(RIzo338+dJr&K^^n_d%MFJA!Jd@XLzeT`#F8cb0TR0AW#z*b?TYcqz!tWEg$URRm{(-!Z6^=zY;0!KKF z@soA#lC|8O0utAsZGr56&j()>^^f3Ga5qjho5Y+&CrXj%r!8gvyBr3Jrvv3I z)h`feNyN&J((1#6G>xhLM-G3cod>Rn{evx7v_~mSUd~!6iv;{~?ICYuFb3bE-E#Gv zAmYCr4nN^BV|dby#!|gwTt!F|1a*PueQB=!xq;w1#;Ot3otlbaWL4g;2>4CuAXN)k z-%$aF$53Me;V`g@A_k4&-(3d(@rt7z>Jxla&+{UkNI$7AIhJ#I-&(3sW)bou!N$4E z6f$Z8x-}!`UCBL{Q-(jcI`F|Adq5&v9hLTmd!0lo;#bu8))z@}kSsYqWpO&fh+T z*T*ZmO#V}dqYMC=ww3-pl6!0-RdKbS2&We!3g312u?WyAgd3eKHzWM@Q=g%kNl-rd zOz$`S{a?OR*H%^g{g(G$UbA=g*7e+fX7=>?nb{>dNy>jgq>YE=HoBiK_h`j~Ej8a@%>WAam{{|UJ@N*~wqJhpX|Iv#6?EvKZ? zFIjx^Uzp0%hcP(+EyVwDQ}YE&U4aUh+#IzuP{BAv>c7nmWz zxL29{u^xwD_S83s^2h9cQ>o23u&h&vN9daspApe#z*Ocg&xe&JCsR)A3Gv}HPPIrv#>v%8k7Zt$yDKLRb@eNR>#{PsOd-ECk%!LK z7gl(G9$dQ)ZcXP^C>Cj%O1>mS1Y!*>vRMW-lYkx{ZY?X8Po^K~>FLdp;nQ8aZ$}aT zVxZ5FOfa;8pz|7p>%_!YMmSziM1Vngd3o#?%O{$YO2ibT-|GJ;vL~s|B3CFcqi-C5 zH%Q3@JddL|9YE&+q2-M(CnSF{3g8vee%G{~OTXTugR-isa}WCO17iWfjdetG`af&; zITFKj`nbk0xTeze3nTyQ!v2-ftI$$mzZ+NS2xTGh4vfEfMjpDhsPFN3jd7PU&&ul6 z$$GBH^86P{p}ubRKkMZg!U+aa(Df&W>ld4wo0fsC_&q3ZJthBOmj6g5XA7Z2d3nCJ zqyHPNA5g&9cbIDKuM}4-!jg!C3?ISrjQtC%euhv_L>X5NCikKKi$#E|@kK<44vYSe z3+rhHCE#a3--Cl${J+t22?7#a#abKq7b6xqKt|oLb?D9d8}kzMcXio zP#XF0Ux@#Ob|}l9CZ|rK`6tl$XFke4&k?TY7k<$Gjk%~iO)h_;2H|gnHjjbZH?tfg zi1#O$dwN(nod_jGQYgXS2yN2viwNYxegEsi8AK}MU!CZ|puNy=go$v4kPeV!|>~{5HbN2^?7inGl0I8Ik@J1)-{%m2!K|F&-K@*oW$oOJ-@u-WxA zuLt-2I6(uq*{5bjrJBr`8@V*qEhKU`hpE!x*o+TJX$UHyC=8QUH+)E%Kk1XNj9SMI zsaq(ragJ>$6Q~nor_kN8GCg-d+bHKY`J-DTO$|IH=}Z4|;f_D18!PrME?}Dh1>QE) zYH*-hk)J98k<9hhn({Z=^E?p@l?N$-p_#_Mjw|@{Pn=?vUlS5%yjS-5lepg)WCs{l zxgS)Fqtm$^i+Rsdl6!OY274oH8-n6PPu!~om@ht{keL(YyZY#`XTs<7De};$X?ne->>eT( zkXoGe(E3V$(>q#s%y^rb6Jp6f(*Qv(Y;`?a+>CK^_vkGnStb^JIv}(z_$RY`Q!k=` zcnuIcQdexmwsBICrF>tdzWLSkW*dyasB4`q2MUsmp(qeo$h(|T+9;9M{9yYBxS*~KNRTKeoWmUz7lSKw-^A~_!v-SyNdcTA;~)oAj|Os;GU zyB2hXO#+K5l5_b0zHQ3U{AQ_C)0MyP1UDLsV`oYL?|yK{{w6Q4x3bHLsuJUL)8?+{ z7#xiBusX~`RPpKaAZL>e@^)3HUtjZ4*DO0b{aESantE6GI_#@hqQzDackN7h&8OVJ zu#=RSFx+v#6CkE#SnFK9f?PXoLmmb^oikhM*b5&;E8mtaZp2)+&g2EoAZjLNv3&cv%qePQb{z^OT(JbTmB| ze$S2YeQjxY?9N3nbgOYe6Tsn8Ax!e!bZO|oKl`Z(h{8O@W(n+K-X_t(=WK9Me=*7h zbJOnjs^oB)W`6qPhkDxDp7sGsZ-=lj8axdOkE)@Iw3(cz4)*sR8qA_^=?6H0%Mn2o96usbnHY#)YMAwCUDU>sz0_9|&Cre04&0WC(*yduB%PaXk z=2p^NZ^gxu^ex!G3Mc8aEr8n}5z?J@=OWH)&1;kM)ngz}(OwX2KS-d^AY30SVB`*5 z8O|*PRROftJ!W$`< zr5-s8pn6|}YmY7aJANF2w%;9VoD;$Z9a@Ba6LZO)5sVWazZ7D>Y4v9_t|w2@U}PZwFt-i#{XTuT6`&e~5dQ{xtl49%Etb`h^z_V>D2bMM+0 zyh|K6br0kQr|sq@1~?b)%k%CUd*gC#?v_dHR#J+nELJWn{PI8V87+#5g>g-jsM+hr zQVrOH#B3X+c7POxWC+*qSs4Bl`vNy!mB>(1xP`mhE}u=rsvnP0+i{BxDg`*k?O@pn z($$-eKkZ%#lk+gC%S-Xgu=%P@X1Y9XZ;pPk)&`kOpU|zZD`4lMWS#ukTfkge7Pow* z@6v`Kf|p*k>&(-3R3V)7tr*bZv>g=YD3oqIrDm{7jw^$2|*&teWYTuRLSu2yNrr^}aIqpB9fJ+T@$KM)yNLvjXRzd}Vg@0S<@Z z2nx@@@O~34atgA6```va=eFrtz9)9jLS*+3)8^p(p|wLYwq;l6QH-S5PTk?$VeXF_ zyH}a-*XsFa8*p@vC=C6<)gbJxt93;1Vm_7*g%%|RG0Dp794U3a<98+i&lS-5LpuKC z8W;2e@Q`~#FQ?EhJVS8&Ylq6_WgiY1?=tty)dzgcwyO=<`dxN$gI4oW zH7k41a@e(O;gY)#<*O9z&tBa__=idxnPTJBckdZF%)oy3Zdy3!;Q%=8kp9W#AVp^S z>#^Zu7;mORz@6IWeEpr@c|BH&QG-J!K)bGg-9!m!y7tRUyoIqc;jNGy=I^QExaxI$E zwp{?$6i7nOU}8(Q-dg8=dR&gbtg}%_T_XR}YmG7+&#zU3EqpP2;veeE0 z6;0AUkz)1gHWnV{bpns)WwuxY5f}=sF(DDn%JT#^xXl02C5NaYoS?vq!mfiXw$HvO z?m!sRTCFPi&YIOcup=;ANO1<#k^??_vfKEjx`QwBup9P^G63WVz$Vu|OuWZD=ie9` z6!TSIFQajWv5a)QMHVpJOBlelHy7IlpN#JP_5kCznf^thzB|+p6Ubs!m5>F&Ef_i zN#<;^&(Vs-;t&?PC^tLV{oBNj%Je#Ggyn{ z@}dKaw1{J7b7y8ZbZ^3zEHA~cMuHja!lSLe%?rL%c5}e@CWS@P5!kO(Tt>w9G=rT6 zniRp+O0h0JT&=iBbm&?oN8hD76=X6iu&!xd-4(wD{q}{FjYTX%yx(N>+IFs!aub8C!Y-4=KkiR^)3Ah z@p|(^exiq-=KEvpBy%4;%#Gr-tjh=N6QJLUA~X|9Tc&p0gSa5{kdovba4s*{3@%Sa z6l$t&#-lRvzv|1}7d)kz(LRzL?tA7UMuW&R^MnMJI{i07JBVKNQEk^Vk5bm;RVt@j zMCHa^FJw%;mXfrrWZNJE@@=vyMsQ9)_YJvwuUn&YW}JJ?>bu^yUh)AcjNWy7D_MHC zqZz3kClCJWi4m;Popz^7Jk@#q;>G2eCM|ITcxuCj!_=DYBJFt1b<&gX)>wXR^40s3 z<|I2XB5!J!-`TKfIu)v38fn`kH<3YxZkKiR#~-EsZZr@z9R~5I#aPj$L?Kr zD=EU>%WWVk6rVg1-w~?JZtdg7Y4PK6_ffVpI>hNtxgFSMaZHr-=ayrniJbf@c3XFm zD>08Jpc)&WCRCSRDFxbb^3RhmCTUD;O6Mp~;ENM>*@~K9@oFS7bpEQVzu!bT!k$aeBh11hmC4uQ z)5+|)_I6n3MtJ^l<7g^J^qd=ak~i*BheN9oi0JAjkn5|H3per}#k5&Fx2hry#U9fz zEY}|tf)-gm(Jydu_-(wy8pe7xu>)1%W`SLhEP0>0#kZau2EL+<6V5I^{|Fvm^-X&> z_k$jtv*Cz1wl` zJQ3#apmd}xDBqcKZ*}^Cdeua1!`^{k43^DFgIMfm1B)$Ft;ile6{~?L=lv>+R!zpV z(mUH$4=VrWTHva`Blrjnh>vF zxdFbH$8-oRGzUx>gX=f32$Di!=-BoXbkN`hogbYC`Sb97dcOUUUqKSm7VobqzUOX$ z^|n?j?z$Lt+Xi`2{KO3A?k)WI1#hfgP}r~J82pM!fBqX3PpL0zI3@Lzu{rgcM%@@K ze&FSOnh>w?7Ue%NcqZw)#{c00_;(Z>C-9WG>33l&k@*ZPz}l#M+IB=9e&-6yKtz_$ z76V*%mv+bVrPK}R;~sm{EPa8#Ze7zpFW~p`loWz97vz^;c6K1pIjCS^<%_N5k`kSV z-i7nU^otYbwB3QIL+( zHN>PdAnlUT{nG^8+Nyl3aO)TQPC@>*wM0%*w?zFGL+e%{s3J|{yYs~%BZnO-UxTu) zg_L=kbKayrC(Z`Fx@z4@0Hi{)^lMrj*8v-hKEFQS;6iq0r(Vm^;L}#zWxcLMNNasS zz}J}68xCt#zw}Iv(sQra_az^H^%Pz~O_CB{2drG%n^a}W7pB_M4xi3{9j}UbNpR?| zb&Cj_;26@!UmE6IyH5=-kqul*)+lQx*3AmW=PAw-w$0rAW*j?FeTz*%a*wtAQwp!agM*k0iez@FIv5M&(mn;IX0DI%9 z{yAkb8@_?QQJH%>{J4&`5nE}Po0ynr1B8u@_EV0A3GweM*WGYoT2EpbNLpwTip*o z0~icQwO(1DgLu;}eXnW>n-$7JNBW?Ni-(7MAjgoK7_T>VUNP5@zCC6}a(?~Txk$PH zeiYw@j84Wqm~DnA<_DxeqEX^*V969lDmg4 z#_M^vCbqo0tF26&Lud7nBQn&>&y@*G%`LSP%xKHv$`+wh0sX8M{s%teL=Ji-YnKcd z0D&!+=?!FEfjl42^q(G_?(IBMEE#2CGh0$f2NI-~HX9XvER44B^`Z(#RE__Kz4r`j zvgz7J6)7SjB4VK{iV7;7&_O{^I?`)Ykls54k{c0FIwHM;(yR0?pnyQAfdGMo8cKiw z0YVazY@YZ1-f|p!-}kfk|NSGsl9?;ltXXSj)_JaT^xO4&BS)id7e%nRbPP^^VaD;R zX*@Jnc)`Cu`x-K7G4FFHejZ8~`FOx4=8vh}@aLyJ9)Nvd)|^uCVf#z&m+uYkBl6;` zVA`zG-HI6DgzB-z@Fbs6F_72oqlX#oZz|P%;%`@~p=Q0c-yvMM@3O92364kAs@S&O z?5#Uq3P)R3Nv?~j8j$?V03nA;OaMifEV-wBvD`~3C&2Gc#|P|Nyq5~dnpU|C>u&le zmg$2J>1Z`}JU#nwgv1#!y$o%zh7xuDG5wT>m33FH8dss8l-#nz+OqrhPe##)-@P!{ zN2)g|&hXIU;=FHGqit9OAtk5E38{o@|m;U$-#dbdOE}f^4yr& zJ+7hsoA6Q+`;(Wl#}IP5-`_Bs9CB6Kp2oz=HyKR9Wkc^d8;_@p}oG*(-we+>7b1XsR;O{Gn{U#_O~|v|92&)K|YPA8DPE zx&Ffpe3)%=ILM&xO7p07WY`E2yXk99S;E-qoD5iFT-#V+v75V3H+j>PDcPTtJ6qgt z$ts)7YE&UK*#_1#l@;&ent}#Qlw!JY&F`^QV}E%rAxR9oxlu9v(ef>1X+~}zNd_O8;co_CjF)>%4 zzMJ_ESWHhY6q*TeUg?W%K%{qEl~$*@K!14*--++|$g^`wchKLuk*p3W>I8PpjOQf< zIb+O2Fipbuo_>w@zC!RGS3z*u5Zz4>8SRU4j3jO<*k%%jVX9k_t*!GU$)>) zGn;R6@4W%vn9jm*tu@CNN4UbU)5y@p(b~XDWDu811LJCRGTgcOF8!(wcLg2Ou-aa*EaItM8JR zk37WZ+enGu8B&!h6hj+R({p~}({g^Yin^=5yshvHDykBu%Q>xZ{rHBfMdhoiI}5QZ z^x|KX^ZGj!N)D%>w+C*zivgtOIrVp7S&rr+t~Iy83%rP%H{D=|qIFyf9yg$7w@{~5ES&%uGp!>EIMPL&Wg>>DRL=^ z3lL|!xqo(qdDrD#&{ws@Ef^)0!70uKsClSam%3UC=~td}LpPufwQ_FvEJ$&DG*V4z zk$K~JfqxIQ!2jZ^!(0iMw{!QaN0re;+Y|>EVE;uf zv%<2L87DDrs)>v_rijT;jh!G+HvVJtI(D@_;$AuCv0*) z%Pvkp((w_-#^>p2W$$}*%*0v0D!*!zG%(;KN8Jxk`N}_=b7vdW7t>pZ(5!w)1rtuJH5wkA2cGn8`~`xaP7?E{fF{WN$%>6!`GcZ zkSy8e5sk2q{jq_m9F|VdjX-2)ha~=V;|dW?$XTeZJgf~)3W8b zE5d&*gO96Wn%4xB_AMRwDIu^{x>WXIm zn|PM+GSr}0XEXe!wcgve`G1_Z|2(@J|J>{%Op9-d_7k_9AM8ofd_i9iW>GbqgA%b|%Ra(n5jUgersI?0k5yi8E6 z1?{<>v{byzY&=7@U$Lb-a>Hcn>`PL?0sOc!mtr-RG+WGhp4$y(Klf_NzZ>z4CFL3zDFZM`y1VOgk!1&aNpPp%EBwz0VY|1###$z_ zQJ4F)=D_|qX1>H2dktZCf3-dH?VtW*qt#DG=|$_u+a9AkdKN;4ti&{c(WKrjjGGC} zc}lBVEbWL2A6122n6pAiOOh$)w8afPYlVu$f=mrM$1T?){k95#J7d$R zc>O*Gh*7AllPbKC!mUG5q06MI1B7Rnp8jmP_XjjkTt8^&#w%{wyCbq99*}^q(k(gP5TI`xQOcDPv@XJ*H9a(uh6pyXrvtp;iE3(Vty?wOl|Y>ZV=nn9aBw) zg5st=zd{+;CH*FDS~>e-*;Z(J5t?gM=`#MJrW5v?7Agp;N*5Jss9keZgBR3!#Xi70 zRb&s;-`w=j&R+5(Fj&-6vU``9gR3*jjLPkd4SM1uj0A-gZirTNG>CP;^?bYJo6XX# zM`H@WGGraG&2bt8$+&if;U4;-{LJ?X4NX)IOBNhu%}>W;kK zt8pDyup%(iZ+Gf=p>elrgjog4uOexGE?si=3^8w~AxssmliKwj8Nac56M??sn|7tb zGdz+N(V}8m{TAEHAX=@pc)QZyx5B1n`o(pQ(M*p$-qGptrn99ZtiKc!2v|963yNYG z@>49fW0t}t#=LgJUq%S}w-s2SWTQiaU*xwu?N_x<&A+qa+R)JDW5wB+DQuY=={N*s z7CiPtX#TL~WDq}WBU@kTQelhNznDep1M{9fZru`!AD7~KVf6L7BM!aZ5ICu`GvA5u zG>EfV`Dp*%vRLFZNg`Jl z;Yn=zPV<@&$Z~P(^ha}kn$XiD?cvj2_iFeQwRWy)^q{3{@{`}Ax#Mj}DSWW!;Tr9Y z2L^mtQ9!%*MM9HQh5q>djCNGRv>7te;-CTJp?bG^IMe%mW*kyKE1lz~=*;P^NzN3q zaH;M5h#DU@LwHWVe8$}Xsy9u@Q31M(_x?cH@TMr}jg`wAO6gJ}mJ8myP|RZ9k@ zPvdsZ8lP3O{89OWQ4RZ_TQVAkrNoqLu2_%Gir2pdS3$r&@#9wta)zWoSXKA_t{e8v zDU6waBRtd)ztieTccAa4$CzsJPHp7YHBm1$kp%xZdRF9vP7@XAI;53w+O}n(&B=tr zN!X3x_PKvGCSy*2dm^380T_>}kaq+Vkjlr(^X>mYI1Dvs%pxgMiUks|CRTh@Zngo` z+Qn6EcY;2(=zgdj#df>P@kOr0P9XVF2(GrWmNQp1^t9jIWh^sSwP>eO7H^QdIN!5o z*SvCs{lk;n@0JChL$|)&IA_Ifx;C}a8EEd$JfMT0o3%*?&5-I?2jX|+)Q+EZ8hEQY z8rm^sAC~B^7TC!8JvLBEpFy$jrEFW4+(}A#;l>kmGla60&7|%;A?6={qUGGYpR4+%BlicL2}g|tv`n?cDC)^g+xOYR=w4< z6J)HWl;YA^Z_S~;jkj2pp&I9&TtlE$v+KPSARBgMH_$Oi|nduy$yOufN>j1_>8~ zdf*B8%c&vLAI#|v))$n1klR)MoyS!LHbLAr3=fM1{iBKFkq0p+`i!vOJ- z-2f32(y>O>9;KZ#6}~_yV`*4wv5JaW$t*{(&~I$1Q6fSby~7%LKFt@Pi;bM72zG1- zz-5%=pyy70$goYo(H1^yQqk^Ftx3ikX|Jk{fj4r;Bl^=RJFt!ONvm;Ft_ToajKHD~ zic3T#+I}_T9I~L9qBQ zN|wjlM(gIK@bzUQ8brmfY<$`ifOpo-o7kyFd$cn$W3Q5{GvqnEu+6GU84!?AcWYqi z=ynvCO|wQ;E;S?5jE4!b^*U0q;&f9Xq&z#9A$*Q=9pPE043RrcO<&-DbHe%8 z3X@diagoEDYJ3}Y`s;fSbscpg0KS>Gcv3aC^DYO{VELaE}=S}5z93R70B%e_t0 z`93De0IXcyUT>hzZH3}&9E4Sd{GgHJnMye|xh~oo+-tlQ;>_FdS-)PM##^%MwbHG< zow1)d-n6$E)aQq$PBV9{T4WYf5COaemIy$FN$m))NGDr}RyBIlhmRb&s;Xzb73esV4va6P6YTkcn~SADQ^ zj#x!&ki4ZrkL!8IVVl7DdJ#}yDA7LPPI~Af6Tk`G19e(x{VZekKpVUIXmID6YwNlA zC^f6EKcyaQwWppT%KSynUB0dv>Yvl;-*iuG=B(!hOj+x!^rDURJ-N?{{d!&VFO_=YvRvhB30=R z-Uo#&2+110eNKs(o|}{nX)Tq-bADTVc{)9@#q)w7F6pJC`t{}|R2HfA!)nECLDMYG zODpXky7ZZ((qvY?hqqj)9~N~wxy&Z8PTyl|qFn3{8XVsEBGw|D06R2fAM>$5m-szO-aV?y@yjhoFc^bIq09x2$TXR_U3aMLN-H67s@ zGhg&-Ey0?ziJ`#SHtWSaxMoS%@`a?b*&&m_`C$Eg?Z94NEG`kD*zA-{q^0p+ zU+?xG5!cR4HFx!sAC6@AbpGI4+~1TBCka5^VV8K>k=vwHQ#_sh0WQxOZm`}ZZ^*>jT5 zHhjx68Edz;5WI~fP+4o_TL8nLXoo$Hro%oOPzK*Kiz{T2Qut#}1c`!Z-w$$>(TS@G z2|+c61bw!nI|hN$R`DGUEWDL>#gF#N`lrM-{wL7*OS?`9O4?`e#ChG+T@E{cUf0XlX4f@ioTqxQehOpDRs^lzU)H!AW#VYfYh zmfO@Csm1c_N)>02E9+jbNIlqYzbQ0&7zb7?;4aNLU9?;~9+rUDCW@W~A*_^wXh_Q4 zr}=uq?*46-3Kd~%ZHj=zLJ3+x`H}a&04}fvUJEJ{NISkPzEj)Trwgt{1@r8V^Rvc^ z3VHFsrrQN0l)WC|`rqT{|cjKxny$-nlNHrF!ack#5U#@`HQk zZ6=B1qQ?zVz+3F>_vbnSYXzV`@J4`0&IV<|^nwgpf5CSbG0^oLG!P_C?2e%mAI2MG zR1CEYYw2`7p)x{)hQT(8T_5$CjMF(kY&*8K-3)O$!@)Yri_NpXn)^$j0;j9&$Cc*r ziCulCH~$85_*aqDwL2a;XMoaH$Kl{SZ@GILI?m5 z(r7}&cEOd+o0oQGiC;TgJjx3%_ai2YkI=BTsX}#@{-`|!{RMEIO}WX#-?SKJ4vCA^ zdjUqr-Dlv`$lG|yePO4$qB{Dc-CM}~t-sQNVEfh6gGa6~;ie{~uZ-@S^nY?Ga`)Je z0AQ;2-u7#}5a`|9Gi4xQXRQkKRlnbk%oxqSZH-hE`3r|zY%qbnFJ-Hjg7~anyqxpa z7j}(CXc!8hL!&vp!{U!hD6)8Wdoo;7Og+FTxcWIipJ`^>Xl+NrV{yr!%Rgb~?1%nk zxelK8e_*$UwlZe*D=X^lAI1>_;pNHW?JaR!l$YK>aGl$egE_fU8p8F4mZHocpudr- zHcJ;SsxkQLET|)p$jmXO;MMji`!3ROGxm&c#nx|$b!}sY#hR$ceW%XKS=1keWz6f1 z>6!RpswP25L(gz(-u@Z%xO<1!L3g~bh2xCB!ZB`0%G`D|mxGw{eZz5xA!f^^n(IJ{ z4$jbr;;Toh;;X*ln=erCsc6M!cdk*p84R;#Wa%&3bbi-ocJ=2TBlX6dw85`XGlCom31qv`p~fi0{6$S+0XnmjSrVa|Lr*50oS@Rr=V2f|~gtV?64 zju+agh5?7>adnrgr57_Mu-n`*_YUlyL?YerU6*r6JUY^v?t(J!p;$&I4ns za!NgM^9o%Z4RsA~c7kQp9>}}zJrbHLVdb~~tsh~O z(q?T94-6$}(H#)q(4CI5qa$)VM>xTE0iVfF572qbvEtBJPGr#tHmYt3E3dZQT6g0g z+|7yKgK~i|2Wu@N-IGobrg}tAW6Pf9Er`5;W;W?%)|XGo2w5fWH+PA)K|PK@{1N5V zgaoX*zx#v`NR!{EVB{I_xE47>$WavSo#+qxCgENBGTWJ3t)XE)u_iV)+#;s$#&w=#DFM$|)Ql1abrOMdmt!}QSOuX% zs=AUL>n3!OWh}0@1N46w*hFgmpi7@V$Tx5BZFN_~{EJ|q>18shz^Uiay|a=^x}r<_e- zs88&`xXTvvExAnBu)Q?%;mPnYm?^pnl3^L_T6VSurBy@E#dA*`Z1L&_$KXnW5# zD+tjoRPTxQfL>xw&Uqiv3(SML>ZZLNOnt%VSlBcxD9yUAa0`6{JMU0sBmTIfxTEWK zx(u7d`(+Vdz$D{G-XbdlqEkwx0kQr*=P=jmvFitm)U-tAh>hAY(H{-JhU49GtK^&C zTjdT#HKrdwj-j+j>=VcSOiFmF=6=eQ`_S)8e=sYa;tYd`2d_JV)d^a>?)J(CB-QyT@nLRbQG|4$EBbhtv7OS1>308b;}Jl1{d6d=67;j*gp7lRw4- zE34hBc9zL@C!XS^KhARHJuo3{$I6Zmuc!i_Def=KYNPNsg-pY5l%h|Hu1segcSklr zksZH(IL-cwNku9`mDWQ%$Sr|>3 zq3a>LymnVe%V%UE?#Wsk3Z_`V)vxjU)Af#m3f15D@1tdp5t-$xrH++Tb6H`y=LY>r ztQ5?_Q=CE@Jt0$3WO(i$z`OCJ_i}3K77#L1GDNhC+24MoeaRH zm3A9z@B9r)@h|B0p4MUy%j#%WVg=7ZS4KjO%qm-v?z4;c4Eg-{a>rEef2+BaoJ{9d zUt$!f4HMI7HY;C#`ZIfQaBvG&mhyxRw4ynz+^Hk^-u|Bm{(LfYonoPZd!4S6X&dru%31D=l@||Mrog)gP6gUm%DVJ!oYgG%+$FP~1BjQ|@DWGlY7yJLi#jH>0`lsj#j=1Dk(muI4*mxO!GXIC<&3|NY&e5pxErp&UctZ*}cS&7~cRfbvknL39)Dgxc z2e;y-{O|sW@=wiwb)b9m$@nA!xhH#L=&E!%O%wjKF^5A5UBMI9^KWN$XaL{`7xd^zV}t_g3%=wJn9TY+=X*c-m|5f>@H1MqFb`^}}5EE|pP-`Y~@7HQa|eO@vnLAv%;o#sv`3XL-t^x1#=f;5We zNGhSL)SyV<5L1Y_`84IIiQ<=PU2wLYm|(7>kb^V5PH_)C6ln-gen(0iukFm?*J|~K zB5f^8zsms1UTCb0iCObqt^S$MwpLalTj?j@;&c_O;}WU-h0%q&UDMg*RA^tRqjPJM z)Rc)qBt?en)6y&Ae^^sa#8Gq8(2T9#aV5h#wb%j|=sVgkz{Td4+LP#mq$y+ArN|5^ zC(|827mr;!4I!N^-sED~bXK82sJG7^Z$C74Wgq@~ZgeiUKb&90a}qdJ^6UcnxX2wS z9ozcT{PkNfJfq3G$y!>kqDfQ3$PLNB>pDDot5DXea&tjWXsW%F8tY%*Gk^62kAPgA z|G3WdGOs(+y=)vpDXy^RmcHs{mMJAWSr}Q?7dMs8OA%?$GWlPw0DGN>q(4Y7w z`)p)V9cGxC3ojQ3SGvFdR=&`sWw4)E*s%JhiTKo`>H?t);%PVwvF6=p3Lf*YTz7YP z#!I=#cDN{`7|N^I^$K&$d+0hp9@D5NBBAd3)QJN%;u5ZWjnNdcaa!z~*|1dj&ZM$J znewL#k~dPC`jB)WRdw`O`RAh=?1rcn>Cl@sx{?#6W;+3TL=z4;(%`e%|88ninvz2@)mEcxZH(jnPnRndQ6`!x(dICPZCq9nz? z*l;HB6mG-|H!>_4-=1|k8B=*}WHfvui%0ai5pG?xG$x3ZEh_ncMo^S)OPddM?EK_t zqlDGGc89qVaa2XwtCN#JgF1p^5#;@6@2#dr@IeC*vW}eMd^N8BJ&m3l6%Fm!QAr4YJo{M?^>fzaN2MW0aRc$1dT+?Masfx%{s*;pYxEH{u zm;F=jzB+707jsw2;!V(W%n(uSml%>S;W3SIe7=p{Ezk9pn^tnAAw$tnsD=@7yw$`i zzny(R>4$Ut!7y(nF7ojHSAUuV{|+6?(d`KmJO_iDql z0KsP+IKfU>;>xu4Ic-=40S>EpRH*gK_nN%aT<~6k7v1dfwJ z8TA$G+1<9pqQDik)()EpH9NZvPP@StHD*o+Y@!(OT-QKwTzu}O$olTlfNUY<^MCHV zDQa~Ip44{QYoyYbi7&Kn9)4t$T^Krz-5(UXFYXi4+k|}G+s|tfW@5b$CqxMuw!Dx2 zIs&8n<9V(0zeS+*sI+I+b00VFSOobKBjSjZPxq`QPVc6k^aH<^S(R&sbe-y0AII=4 z?EP@gbYLpd&8h-{e8#?Ob6I4`SGmefm&1E*atV!0APUMG0a;dSW;dmb3nx#PG)1H;p5azR*gJdunoFFC6lgLr<=G53hyoxefN|MGUo zn|lCw1amU$jqVS!v122L=!Fjt+$-8Pa8J|gXMwB0|L$)?uUJv8fcyDZ|W0VZUy%#K4gxYs}Hh)hfLL9mX zVBX~MS51ez*Ky*kxO1OP9m%Zqj;mdgFNBrKU;10Fx5s)!1K-xl z?RVTKvsbOgrAG`5)<>-un}hAu=&Et}!s^w$y84M=FeL49F3Ny(xB+p8zxL*KudT8{ zHBeh`NHVv_mwg#v2WPhbP#gSo6ljnfrDOd~DQ>eIPkpB-WLf7c%easmXpPMnPbmXk z)^abl_a+%OnGGMEF;S->?Z=-fy`s;-aDkhc&>X-<%3~@ND#}!f(E!#sP_%u)O+lv? z7I2m9XtI*Tq)`ZHEhxm%O^dWNW#YK@6T6AW?oO?mpboW%uDNq(NVt@ArQr}*!Rp)^ zrODhJ!Lirg$Y28>(++9q=u6c2E7vNAZ@if+$T=KBpduFU5Vxa~p6gvUEZo)5BHBx^ zGK$P)lUia&mrnpf|eAttUh~vYq8p=D7yagi;GC8NUzSQQuyt^i9vY@NI5SS&sgCNGsw5%`U z69jk6+ssjvmq9kaM0}muaSR^=%o8OL62z%$*j`iD(NsU2ME>&t8Y_(4aZXDuRzp*# zSG1@U+2mX-XosA;IBFoKq-(ZjR60L@4v_b~e(7bhn;3uC!dzWjRY$vDmkJTWqEH&! zgf7$7?qVn)b8atOP5B>X!7RGaAoc-W3GY7Plg%AZQNwJ-%5t!bRW1)%tZFB9G{lrY zxMTT3W~Nfb*G%BV_L2jE0`M7+`R$pUIujeTiPb$dglBXmZ~rHv;mjP>^@8^)XVv98 zbpi#hO-^mbclgEVoTAl$7I>G1dq*2KaYm)=OyK-P+5}-um0`e>A$-Lo+*;uv6&Wg|98i}uoqkXf zX79c@D;dqRf(*lF$|BH5y0FSd$UUFw6&eT-{&Tb4QoSEF8K1H}Q5S2D8QQ;aw^O`a z&T0(=oPb-`SEZZAUw0km&QsiMAQir-_)>0l(*^OuQI;^-nGZeF&rq{clC4B7$m^|C zOLio8_Qacp1}8_AIkDjQv5M`)W;-<{w@?IN+eF+`oF;km zbMS)a1d)fNKaaBVrEZ6mevdvGIaE9&nD~;xhpE@(yC&Y@2;A9c&ba8WUD${Q)#qk} zSCLh%;I+14n3&iWdhA7i`pZ>&0gV(_-d|lf9}cIK>@ z_0~lj7@&3YGbKCBeYD4WkHfjBCzsjajOTR8ni>aj9(_)?WuN#g*U5?6hH)D^K}urZ zcG>UbLPsT%(RLs5(M2CPBHBl=WpAvU_r>2zLN$p~5K_A#nQ~ANm!1a?YWGAa`#KwB z=CYR8lZDHVI>MI5Bk4^cNg@DVMQ0M2Qb+)7_cM|Kk58~RP%z~ck!_`9adF*xJR0pE zfUz4}xB0j~eNm4kO+RZp-su8i@yH$Z2j?s9J=t5sfvj<>idG`OLJ2Y63~&R-J(5)7 ztiY39YEA@j-Ui05Rhu5!Noz)7aBG7VF!>%1(7V-h@+O;eBz=Vwi3|cB&Dh9>z^%HC z;?@`K|J2Ux`x&;CH!MafI3Kpv8U4!fMa#97{J5diuSNjO=nEtNmlIfp2C)tR#Ck*g zHQJtEsKQi4Voy9R@1CiYw&kAk?mU*&+sQgmq8?o!hL}*f8rO2MlU?LZh=hM;*Mq=Z zEc4_kFMjeUwdf>{G_4trvC!?fZNh!6X`GpeU|aDCsiX>dlSDiBI{J28sn zZv9$OHr_FVj*4Sm_$lf%wW@igcq{NT<7#eJIb>mFM*0sbCz|JgYLBoWj&_kd`mXu= z439zZn6-}F&Ft4;`P0bPl2TnFPasw{+7A!6D#|RMT#wVro1aame88 zvawB7@t^ZoYo=Es{<4A)>N;J?`z7yNb&%r*pvLPCxWwxVb5sE<*3&{vONEz$ym7_P z(&71g^-d^tyP3fxqFuQrOi4kz>hsl+eqY1=oLwT;%6$44?JNIuzEb+#eC5vZ@3J2K z{=SZHOa~6wfKF5-)d(pUd00^`&qg_|YpGqav_D|SW;X8lH0%uIzz?@cYg>N3vG)0q zubIy!y_r0dwKeJhJ6lg&5*Kz8`O15;JrqQq?EMkJ(WE3M>{{Br1KmqtWtl8o)kJ9f z7U7X34B({G9-NxJ({{UHp6c&jXCf|1GG`9^_{oM2MN9pZ@{VFkEIgr_nxCzG?5Ol^ zgR5&Y(|DV&R`NF(M%@qS4*b(OGoPahi{P6~oGAEsQRUGJ7Qg4vZ)Wb(ZkRu6xMC-J z&_$;K+4)59r*aZ;KXqQekY;u|Zm9Nz3M@)gARv0->Mkg6$;WjIzoEVez=X}NukE^d z!-6#$fu3$OaoXp@VO{su2ZZq$VNBCz;}_-lq0plKwJCcHFJf_hbf3(zhMG%@)%cP; zJ-@Gs-^Lv!P~$d_W-w7w#DT+~RyOENMc3)yBnPJ;{WyuP2c`;sR`H9YX6Oj`Clq6u z&c7cbd~S*yDH?#abSxfZm*ca9v;y1!9PRl`ZTh%t9kh*je4 zM@vOhoJWp}{xX>LLo0rIc-dU8TcGhK_O!PNTT%_Iq;B!Q5gqhh^iFlfX$uktC*c%k{}fOY)Ua? zVpDE+Z>nH?rq-$1r__hAR(85k_)wB5^97~T>$SQ@R0AQFI>mjT!s$j|k8z+>y{^l} z^`K@YJ8C6Nh5Ng@_07BA{dZa_w2nXFFf=&VS^Q|e8l0a3b-IRAeP`x9h=+J|YCu`0 zcGa61TH@NPF=rCmJ6^8Wulzb9v^j}Pam)O zb_1t)(bQE^FASeC{X^|d3gcwdvCC9fr{yid4vSH14hOG;7GZZu63gTQJmWxX!y0Ux z3#P)D#6qSn#P98+f;PJjD-}%6SiES zOQTJ;zO`hw7R6g={YA8NLrMEbA23Zm5|4lLM^w@|utoT*lGkF$nWV$_V=Fn#Yg}}v zEkzo?6iKWYzPBFx^fjs4f8sk3cLh77G+U0>tWcr>Ox#axGU;2Z#Re$7qbS3s@eFsx zzv4bxaQ9AQ;qCp_aH5Hp_m_-!ngA-$%T8iiy_O|-YDGhC1Kb=my4_%ZJiM&{ z#^Bx_&ymbV*GTf17IH3?s-*u8)Y;}jFEC`6cOpMrX(>JiD0QLio-Enh0K1%aXzLx{ zA};@ti>asxHY|UzCO%j=c<*^juM%ewa(T3VT_3{oEB7`MT^J?FQC&c*6QY{XBDj;6 zytCd4{EMn{V29RcE7V%WEld$e{&FOStxo*WH^q- zd|tU*UuMa~oT(A)cBgh#fysUVWX5d<0=!*cqIqk;#yl#^85`x;I#w3#b#hgM5Aw%C zC|_&eDgRVjluFP_&>m6Bp1jVa>cP8p;QpMBl}oiip>WS%0AEut7s#te&?=-vjX2-g zzdjYYezR~CmY(pVxafU`YfF%}#Fyji)rcQg*j6C$KhLh*->U%{ti9=9cuPotA1zz) z0d1A?#rU7QcmjmTk%!p~{x%`zDKS9L5)RZk*X@w+$)1}Acx-p)<0?7{c5zAQ8YUtOXCwo%~RmbWfxoaH(FDm$7HhhzCcx2;h zZ*fbKA)!{x0)1|RY?K`24AYti&4&LK7bNUrX~0nNOS1R0I;LfLP)JxMHE8NR7rVb% zcx*$Xr7EUh_#1F*36olX^;q%tdJePLNVs5cw1V(2NCR2rjfleNwL8GQuy>LBBkgO; zPA?QV;#Hcg?^8+Dch^Y=JeAf&(V*z*9ozAmE`uv2CUU(}EMG-2%B_c-ck_nXl7@_& zVP6A5-jLO;q>k!|K;Kb}-~egsadRME(=Deyl%;mkuf}d7d}kkyL(nlTx?VA(b}E3up&3RzRH8n@8nwLIpvXK;gOn-MsrE%Jf?jj z^_Z5fUR*fDYy5t66fhTg$h}K_m*iQ~(##+b_4tz;HN3O@2Y#vUW#Tnq^8V2=rMX11 zeM<@dcDLuKS!J3x_e!(sV`-yj4dUhD)|dcO>^jeUB&;_Qs@NHExF5R{(hfmN=vFvf zb_|#SG$rikjRoo?ejTK$%RFZHPkC5qYrnF3N;uhR#l*<}Zg6EfMN!YcdiBt>{Z=u` zk5k9uiJa*V7RQSy46NJHqG|b1Cp%`34s?8dSra+xa{T?-w9vs}T=KgHyZG0AB_DMo znP1$rXks$cEgGb(v6`urx4g!rylMrdscM^Fxy0W>4*NZ~Ptd=+mm?KK{yY{1p$vwR z?kTx9PNhOEJML^l)-DI*%Rc(9}9^fDS2bC=k?Q_*%PiTV!LUj8Da>)KO!+TI~XFW6Y*6+)kw?tJ0IVj%t zQH1yX5g$(kg4KiB=CnjI_Tj%$l%jJ9;-}^(D|1vXwE&kcyp-%8SLfKS!A=@WgH#OfGbT7i_BB+{TX&AMcF#j(NEyrJb~o{ zEJYhl`Go4CH%j^b$KB0fWCConD9uKXMr zixZtcdK#m>{a10-rMrh-ws%|I=67DN>ktx@uobP+K9;Uhe@gB@Cqzh=-SMIcY%_jI zi}2P9Z9mR+bgGk|U5jt7OVZ@aXL2JmQ+C;rk@#M;%+hhX-8(r&>krp~q{c6)Ec_Xb zzq*JBI_G1c4W{nenZgZ{qeAYaj(Z92(<&ns*4+P&a80gcEn**q99h1`@ zl7@~kvDPN<-h4rRz9OkBl6%=$@ksM=BcX?=Cs(MKi;uhYNuXi%qkGLzMlQQ+gseYa zaK2LwDV%;`_|}A#ow3&|P>0DJ#{1cF_Jc^_$dcYv9)#Uizk9)yj(w(Hv38M(QfQkM zPJhFX_uhg7$w`}d=P+oUpWosM2G)1XN#vxg*hldaJgT3wm~RE9*y!v;kr&Mk1BWd2 z%)G1gaAb11tzz<0)BrX^&>5k=(SS}6rTWpA33PsANbCHz5RyDnzB{zq9d+0#D>Lxc zPiF7i6Y31afWfvb*ZOwXP-++H@qKwzhj^8sD!+wNW}AwoOo-P@x9Djwsy2wuT6AuNxqaqGIlT7xz=G6Y{pRk z^}_0R5v7S8;H?|zD}I>tO$9}=hnZavrKEwb+O!flPvJ`&iR4iU!UQ?ery=M8H%KI zk}h-n^z{5{5-|O(7f8m`yEd=}{nK9P*YgOdaOl3){o%Iy;^vYBQIn?&5nP^;4l3RK zZy-u5_pSInDeC|0?oGXSG@^GBOulauBX5|J#>gf2(3lQu1XxQQ9?9EZ9T*s{cyO?~ z?rG!ADzW~_@AOG`{R&fHeeA{62cMLY``+iLLk=&!Mj$uX>TiBB?mqu*kYe!exq-yB z9sjmP>U-7!lmp}Hi!d^Hy4>8AZc+fVuCj^)I_%5Jf08>XoFzcd#P zZkTJ4eg+86K&U5($$-4dkL-0vfT&rbOs6u~`$%ael$C9d*2kQ3;nH!~1+z`1mx1@< zp}suObq<9?iCwox`v~c{R@X0SJ*g^}z1E7(;t07eb`(!_736WZP?tCWd{(1qt+c%) zo6OM!CXXiag5$#5@Zrhtzf0nleh%q%=`joV`vl#wM_p%)E2u;tw`AWn-@wrWdVc{r z#i1aRM35*T!*`Ewckf9}?seuUIcDZdW?9j^&EtI$0bDumfj5?4)V*)MNCRUpx&3|= z{;I;d=3>VS^oEp0HFS!Ui(d!2zjV0~``5^F;cKgv45^>tMmSMt-IP&C=_41Y)e}38 zCQa+hb@+vtnijLQqL!?$JQ5MV54luivi za=9TzKGRu%56h8Bw5mX-iBTm9LE`jWdd6SvplTkAyfYC>}bJ;kEAUf;_yq9;-q7sF)y>~v&& zG7WSe5YBLa_StKvpM)e2U12!jDC?V+QP(G+8NxRCVwx4STh*EUgME#kJZW*~4ADqt z<2k>llAL8CBoc6zI~H2!XO$&`LmYv2$#5q1ihBoMoNCt5_u?J^ppaG(`a{JW%ooR( zb|M?uk@x3H^SaJtTccO`a+w##^spQ!Kg*kj@d0crGb`|E%Q%F^ z91i>I9X#^aLY{=BF&3e*JDxp*p9ssbi8rc7qqAbOp9w_*ms>RDv=q)fw0PFERIkyB zX~~lZGV=}86CQ-aNkgk&%ifXCG*1@lJD=Yc^WX9LIynIxv+C$9)BTwLgpgb8ocy9(eLeOK^R$n2JP)3Jf zQi%~}aXJ=%2cvC*zQxe=P_ehKnP%sF)GIGDP2As>5$lT|_hqG1+a?m=-3FhQ4*q(3Vpg)^W=2ILglVcv=oTk`-uKrpTk*{>Bd8%kyS{HlWXo ziyUd(icW5VH%G!DXkUpoHmHGYWJ}sCAM3u6avAW&2?_v*W$7{GtIiDzX-)*NGM46^ z;y4~v$L8I4;%1q(j^Ze>4xxk?H&_A#Uu7pa)31GZgjY}Cb%;OzMrsCDaK38U(}u4t zPoQ3xyK4{6;wRm)JKU&%wV}ay2BOI=sc3VJJ8~OY;G$6bng%Fb4EK-5LyB!_5!jmC z_nimAunb6VX1~7iPC_nqW0^&N1zSm^KWC=Iw~PGvmQBgv6h*WYkmvbfk`L%qLyl^ zCl|G7dpknEKq5U2?Usz%7rHdF3#opdG*5!JL>Ti;U|W7rnk$AZ{pm!xhh#C0F({xi z;b*n9Q`&iQ)adKb4o##T7l%;z76miBC+KN1|0~jj^T8n!9KpS1p()qrF!dW9?GT1R zx|#%BWqcEuq*`>voV~|R$yMO)Pm_?huliHN@{Q`kPohYZ1a)ap*@UAqOf| z^MyE}RIT*s&3WH+b>vEgwkpT&X`@*KHq`4+BKpkb&_Of1J5?RFr9=~S>})!jY9}&Wj1#Y-tL}&4Hc@hxkO=eiL&=#2YEGByytttCBPiRx9v_!W>oCV5V3Sb-|hfzP7#3{R7emT@u6e>Ay? z@BTCs1-)sr5W!%&Io0C=Kz!8WA?U9L_$C0T@ZF zxJ@#I+VU3N=dgp9%hhoi(`#u?Txah5qT@kM)15}PWp?1Qezm+x7TGdrH_##Uu8E6z z?ig;?ooSkv>T~qxr^{-mBU|^o({H2iV%Rz|$d_ONbLv}F6Y0I0r$)U4%fMhZ7lB|D?BaJZNYI#z>o$0@77{4(vVF0>H68!;?OYwdVoWHg%O^ znG{mb3iL^4olHk(NOzr5I|be@xk%SCW#n4*x2MLjOT;2O_EVgz94kWy-~+9GW6KEK z{^BLUwX{Spn^o)>SkR}rYq*KA#XAri43i~_%q+Hc#W=*QEj{Xj2`5vB+SHdIp*GVJ zB@DwyGSrLXRdpG+;GW#z-WIt^9SK-A`(=wt?}Ap)c7IE`JpT3&ve?-mO9lk#o?S`XokerP6@Z&14yH8EcJI(omLn11T)~)>seO#6B^D&}7AA zKw5OIp=qhg1V8rkPY>lywFcS1wt2Cf=g;ZqgE4pKvqAL|PXTeUg8Zsa1o=c*Lxl9* zRH+GCqa)bq(FczerYkzqtgz>U1oa5pqJn(OE;O$LaVFVvN7&+)%W8|U2z@kCE5$H` zwIdn)u|8PeTy1V3E-if|qKc2o({7VIflo6qS{)I#^7ga?in?I~UiqoR#OOH`xH)$} zVS^IXn|G2rWp)&140s{HCmp#3v)&g6jq9T>-c#VCYqY^j-c#|zYCi2_^us4rI(Dn~ z_sTHw?mTO={)a#66i0UA7P26^8Rj*otIMJ&zY*3@=Ar5F3%-=Sd@)hG-zfLUVvp!? zm}6|Dx+L#&2$Gx=H#{MFkM|SqCW{!mH|a~0!Oi#Qt(qYLHdZ;gXrXKLlc`t%0$a?z zO}~fv`zc0F%2Pv{0Y*7(Pkw8gwL!FAx9lSd2wYhR=!J{{6TZPJIKLigOxgOG_B>0O z^(>T?&`b3U1|W<%0`^Zmu3M_{t(?9IobK~jK1Ut!e*^t$1I-QiaG-V9Kd@*9{)lL7 z;pRAUV$ya7;MXC4ZMaOxM<%n1@f9y)FK?Us$&)UXCJ%wU*9Gzbt_zUHYl!3u@JQ`dJ1Jh<=cwbHZb7jPJW z6GyI98TXv#*`Crurb{-qxV6GL1@~Wh0_xD{mzg*D(vx17o$g$%XsEmC?eyR zr_ryM?j*MRmSVSrhwI|9PBUWQrzJ`|PA)Ta8Ub*QS*s4>&RjVM+#OkagvF%DeM)yw zd28nxHT74?`~G&yUrIbX{opI<-j9M6JXwQ&bG#(Q<~#qfNg!q(SvjPF1O|Y#?yJ z?zm=R2EQ~?L^8J1DxS8>ywue3+M=zdFB9`&beA-h7^SFY2(A`N*aQ$c-6=KOx1#sc zVqBeVT1$~IYTPk;8i{f#!CoRkDD-(H;(%k!>nDXtuXQg=h?(njZo~D+7vy(|Tq$Xd zEjjXa6VE)_&bJF;8kE5ixM4A0zsF=gc%4#dzIRYuCIcZkdtP8JzVO1f0am8<7FV&< zgRF-jI|^zL-ZZv=eOwigfHmO)Fl&K=FV~WiIqNr*NL+in$*Tk|2`NsANc436ZY}PC zgxnX&^cKfiWih>07s_bKv`08W?5(B?A5cHkbBMyoCKtO{6LDQd%t#yZq1h7Bf`U$hDmY}k?i+=gfs zx0Xw;)$v9p2ny-;hzcid@LtX}V(D=)GrQVWgKZmTwE|~AW7QHswGc8}!ybpez6SDm zX!G&Y4|d#yx&6o84;L5iQ(^!T0Nv#0lg@>BCuZsAw@1XWa;wSw8lVp#={d@+Q63z{ zM(58t@{Kw|h}$z;IuD2*p7#5BkTj@Psks_1_B1QA1USBHN{wx0k&tD)r4?XAD+*a; z)g7TQjSVY`DQq?4K#bIz)6-(d))VhkBhI;(XwO-ryPL#Y{y7HLX*ZX>ftwa*xJ}7! zr((Yztl|-m6Jc$VNbjSY)AFCI79F8R`dT3FX8hv1#IyJ2-I*FHcnk~F_%FBa+>Q8W zo|lBiL7LfE4_wcxpsAaWSyoBVh^omGi`m@Bg!!`MZ(kOH{jZs_^qwwVGxZK5s;+iR zr@tLuOAxkg%;GWcQ+6@@Dg=J7^|T*LF@CG5Nq0*-TY5xoTL0{F_I6mC$~v(7IlR(V zudRfu?GG;AwOnHq)2@FsGSPeW?D-XO5kx?#7|;FbHm_dsAbq~O{_YZxc~*S1QDS9L zp>M;GMCBqBKM3(A4KCU7q33_R2=*{qD^mt)#+tP{?*fS?sq%Jl1p)^iu1Qo}l4gp6 z{E|9Qn0Hg3KG5$n^p4rT5o<7fI0Bz&v28ZZoIp}+R-|}BG?SxWgfA}{O4OS$x<14U zZ4w=gZQkxW<`>7WN12D{kh!wa#lxxR7jRc0yB&Jf{+=H`3%l_;OLBEyRjnVZ!Ingu z0;4!vgM6q5JAVoAZ`s;qUJ=|8WbR)NJasIGMd$5xQx!C#U4HFQ4y1RM{1(CR%r4-WizWAveGaL>jCKC(dzqaqHrv zrr!A57qeq310QL6lPc5$y0jy)rf)i`-{H&$T=R}$EjwA{Eq#qxCHMH&G^ZlNw}A#{BI`9kN^=E6wyOO8jwD8cqW#v*9V{hB>eAm* z|BZFIN+e`+SR%yJjvMY0{vpK1)d}_MWoi-wz6<-BOT#9Ig0}i2f1fC$=MUuPSNPXxEU_w0rWkB}kXw#o4HR_WP_?)37vWmWu0=mVtlD$(Y zhrRSB=H7LgdM4I_LPfmGf;7I-_%d0}AM#*gI?cleIdHn>{Q>x!p5+o6#1} z69K%lWlvHn?)|!MxbTk_`*zf8&LQJ~maotO+r-_rUE8^1IW3b}jN6rq-+$Pyy#C@c!lseH#+9F{odazcl0thQm#a9j7{eIM|%ybvcr~lk>J9-etrb-Y# za)g&G3bA*7txpa9THMEm69KquTp(z-2`2Y%tf9;Cbug4(4rRB#NN84LGWK*G)J5G} zE5>m3?hy!u+xEA1r|9)(+9wlrLAguK*(e-DP5~&zV@hMhFGNRm$32I{B1<$&l)Qy= z&|qQxqw!x~6xVjbb~veD+(5OK;U-OhA1uWx)RR90E4eQt%IgN(tT;)>Mvm=BU+hnb zmt)E9lB$~09pG;kcTE=t}X7wQi+pFpU4%(!U7%4FC)M3R9f~fgN5ByShH#)nB zG*nnQBYk5UhH9?19rTyA0r%ihJeHh(rU$V$n*-E1i*C&48vS6eAZwmkJx(k|ickw% zN(Lo^okX#y%G*eQZTZoJ_St1ji>VCv4a|CTQ&JY{`rRAOXvpz=J;&o~n z^q98^U{M*CgC4pIr(^zI$s=kZ=!5qu2u2FqBkc@+HoD}bTT~gQOEflzgg-mo(vy;# zwG3pQ=wUaIrZt9lIh#`Ov$-7mjD&b{NB)G%VXXq$LSUv8qc^X=ku!A2+I-m-Nx7du zW6a}y=AoN<+Fqt`xD96dMuYx?1~TAvAa}vKlq#gc87Vt-hfqmS#vtUSoM2tFf%EtvK$-GSL>lbON}6!2NvWtHz6^dK#_! z2XDm2e$Q&i8m56KUpUIHoA;~oY2~d=-$SZ^zRaRt-&ZSx6Z?zA+2(ovJC!jkJ~4UH zth*vsAU90}u?@n4+-PdZT3OTR@Np|p@M`xZG(R6kpYM(S$!0ZUTlWF*aZu^9K(9zA zu>{5I0IY!KAaewS3H42?0Uk*N*`{3AlhuVbHqEBEjA^+ zs)OnO?qSx$>O(xPMJw-)8yq*5i2}j*N&HjezZlVjM=YC`EL)x%dU@H;}DC;RTw76zWCmEnwt}#sy$5-e!MOm%~vF8t8XdviAYK1Jc;u@6x>~V$-Wq& z2ouR@dmBIcz4PXDo)(0Lo*N1;;x$oVOrX;8xK{5Nl~CxdJUqQeER1cQunUO+w%_oi zOo(ol(ZOBbJ}t_GS;)vgo^5!~B5@_#%9A0sp3ms{#QK(tMFfmavzNah@W`e4@W(s# z`rOIL+}NFX=nnRfJ=JOl)b_kU=Qy^&T8+kt0y5)G<%gB|oX-j)L)$R23E&{+q<-p`oDHj=Z#tppj zIG`9(K;L!a`klGl$xHR~o|G(dE*_XNvu1`w_Mn5<7gOsAs@h~Ocj0)hIVms0sgXFG zHm!qum!1B*Z*Tgi{ano{$IqJ(cz(^F#sOTDZ}r2k?d^^#SB_Pk*0Rrp5&3cj*dMbc z@0F_08s|r$cdgMdb}XuLWbP+zF*)zUn%s1?k z&v?J7wsk>j43)^v`|gf&O@@+nZf&T2GsceTxm&h%QDjKigaR)N3-{-s_XGpc;Hqn$ z((N~=dUiOMK?IYr4nk?=MP&O&QzaE@fn}KYp?`cyTs36W=OcsA(RdzWVJo}NA^-DKSkbs@8r9}m@L!wMUiMHK-ZO&ER(Wp~;la-7=el|31|I@d`BH}Y1OOIkT zZg~cT&Q`jsiQpDBmBwNA*V(8Ax@_E^&2f!usvzdzG!SxOb4R}akxre&wwXPNdk{h3 zZ4FEL6{^EY9t+O1_v`fwUUPDiv5^wumZ}yIH)%HR1Nxj1PH%B~T73ce>B0Og)}V9o zl6U4AO~x}&U`-o2mP)nGZzXf~Tpu)qGt04C^rD!bza{`sx#rQne)NXAtS#x=_2f!e zX)11u7?bLQKl2PO?!klXX|!ssZd>b&Xk+BSpm7h^-9ir|?tz{<;PVgyN2?~CA!m9< zJ}vrRphn-oC9^|0btf5(OU)^7ILMsziZ@{?#jOm3lULv}!g&dcRlPtZhb9K=J*C

    AKYYA_X4~ne*7~LO7a_tF{G}Nh{xTdw`@n>lFNF{SX*~#{M`?Bz-w}f zR@o=Y^cknIX3>5eZ+S`EbZ1VaF5?8sg5XX=#5M}~*;bi)qfP1NGGbH8-94LEmNH8M z6R&KzbB!gfwZ~4)q!Z4c2N=CuY0tYTEW}APNgL*qIwXq+F6%UCKP+Bv4x3~wc0-6C z7(?6n0-jLf0Q6q9JE!d~Hl!OB(w+v1C5%1+&D|W5tY-iP{*4=>`-1e)nFk_SNG0Bn zVOw<}2X0`#o2D7li+PLINY#EQ-;Mx^A0=mTTf=>;?5%h45-UUc~4A_bhv>W4zTh^kG9Qby6?0Sc&frb~udqW+s8X4H{3S z$8+#}F#~o^3$W<{vkuXaOr+n#)5$0yaduKhI;*W_IXGFZGBAO*dZWe{q_m4ZCyec> zaX&b;+0j(-lLNOzd|iV#*;-3-Kl!t3zDbbDjXhC=sSHi6Iy~*ch4@a zwA5NX#_2u`V{kB#OnlX}P71(EPk96ZPD2zgtCcRVg(G?47784C4;03;-YJd7t2zul zWK3>9iy_L-TQ^Ou2O*VHJWJb~X1zjL)yZ>dT)57(5itNj48TaK;ZSH(%D~whpgW+yiH*XBZ{;T0X%~d>S1% zY0`b_*-JnqS_&$3VAs!DR)-I9Ty<>QY@9nOAPZCAe}M9CY;7RG-}x~TYyW%mq;(~8 zSpjQr@rk%X->JV^pAEc>{Bdnd%bQiqWplwFA4u7I!>dBK<(di_T;9}Bk0|slBS&A_ z1gL^{wW+$RUf)wJtqxPC%shR8Lzci05STv`aO#_z}am>ZBh`?>T&t1-75hl^1unvN zuWetCrP1PTH^AA|vU>NzkW;QBN{KTd6JqqsWyy%ep zyk$m~@p|-u+m=cUmq^^qmF`uPRebeAg1XF>Ie&=&rS-z-7wKph{xU`qOX{6N1DAIq zZ37oOUV4&Il1&9y9?!ydjhj5#$XY2kufn&u(!Fil;`{*#MQKXAgFH$jeXJ8g9A$?x zk=_O_R}KoVOT)>9y0oC6w@tzU1voB{?IJ*7wZt|88~|=Iivti^HJ*BgHC;eEXsL|e zMbIyAg5Mif7{Y zT)eVKRab|&GM2(SD6(+@i*m*2slfGmQ&oEQ>f&j=NbOg)H$loH2`8_Q{#&;;E9pHb zTd}Ge!Ppuz4#auE4PrH5pi$Q@K|E5K$o(NX{IH9~aBaowXh^Qk^ffnt!D8=dzizT&^o1@vIO+4DqEeus6 zK$=>x$c0o&nnA+)@2(GzRbRqNr48ZR45eSj(64V1!Ac+5WR-amBT$YR7^YEL-6ye; zrd4}WN^6G~EVtSo8s%Dy)ws`xnY1dzlP?h6XRZp!Ko>8NpCx?p^qhN%a(edjqTM~h zsK+y~tK^0P@x%E9Tl|)tUFyXhBcV-dzJ9Y^PtQkJ*{Sy5588-NJr^U}4m?5UwKZov zF5Zgwo`6Ss4&ut-#e82lqNe0k-I z@jNX?bEh{_D=w6E(~lBBJG6NN?qP@3AMkA-_9xn1))E0h=;LSfnnGqBTlwj>i>E6A z@~ih13gjuSr(93`wi#Z{k!|i?O8sN4<%)-F9VWt!kEIx#s|I#hisjtM{dlV73Tu(R z*DyOIE!M)b%`7SGn3nHv6Ran4H_AT={&T%s2eCU8QXMV>YA<*FU_*HPW~1*y@wR+_ zDU%R;mHDm7s%KpV^(OM!j@RxQ+7_iaq>*cPTt#vzTM-$7ayGA5ZdwKe{O8GIckpv&T;3jF8%3s)6Lnd3``@}7 zOAfZrliDwRcK!*a#B|0-Mjecc&-6r(?f0d=rq4F$rQSv#bT{5JtG12aio*)T#S3}` zC(NaON=iTcj`W1`Rx1DrZ<_Y&z_rjQV+u3b664Q9qdejpj`HvMg;w#VGR@V77^nU0 zH8jHjnvH?kQoB7P=MlvTTXz#I_xvF<+aots^N}HMp74p_y$v zw2?USSy67~riT0`265l{h~x%dgl!6mop6nrqK&Fv1oUgqBfVk9?MubH1aY zX6(gRW(G6O<%)n&L+d-(#H=@~ny@<>4=eE74(_2`wG8`dF0dEzqdM5+nw@pAM+;n# zS-Fr~jUE@M^7Bl4BFUFl)WKHHNBB{|<&~Sa(X%`^*SMDev~a?smov*SRr1qv`d}p1 zt)*LA@E5=5`WT?vi-r-B?QkjdX{ic56#k&8J2cRfF-b(yT7~Z`GQ-V#1&WQ^jgG0C zNfYA}HD3<_2^PA@c^0->I~Hl&`H>lpCE@PTy&-Jxz_Bl0ow{f$&sUqWUu2a#V#;eb z%+k|$4eH)sseNSbY2KINGBqTof+1EGy57To-Oo&~f5#(=kPUko&Q7ghu6G+4m)iX^ z6#Ez9`^o&^$M`rC{ed*;9CItf71jM0sWp!ip2^nU>7Aw=foY@Y)_(D=$hGPEm%_ld z#`hgEsZEQPkh3!yLt8yl%&k@}h&4pma0T4#9-|%>!~%8_Da!i4cVmsWy}g5-ZH^d) zAN#UG8F&VT%iebWN5VVD;Ps6QhgKNRoToodT@WhO@R2&RomWT#Hwq*nR$v_`rypvL z7B4Lb4af0YGUaK5Zy&gIpzY?=`r zdg%8Z-Z6W6#F1OS@@iPDCi}Ep4}^z`fWy%HZQ^}dA?G*Vk-5GuHIB@aoLc|XMr146 zT96t=@sg*Uhna0U)Oua?J+zyt+|QlmOq)GlwRXdM4vT8EiuoG-VuA>VDz9*M{f5xk+TF3`P1yGMQ@C7P=Zahc zG?>sLO=wBI>C=c&zetxb#Oz{Coiu!As;$*gN-tb23;tHKhBxUY$4E0e z|6q04dm@fK(&ZH-Lx8*XAcd{7+^C6+8ZHXufXrJGYb^o!BS>W>ll zt?`}B@B&I;-C=F!!?CZ0!cOV)Bq#(`S<6qP?)TNS*;T5JH;Se=*Z0sQXa37Oq(wdb zz1vYn-&Mq=E_hL-?>E@73MVCR0~yVU@_M~z>Qbufl!LYsCW}IODE4ffOC+OfukeTL zly=2oG*iMj%9^|PqfdW1>3n3^R>=KM(L7ZS&mhng#rM$f$uQsvcqr03D7L3n6>Cnz zIpESWs;ERAT`q=>P#_n0B5(6l(vT^V{2W)gd#yp+taeOvQ=Gdw(Z|gN%s@uAA80|RQOOIF^<3SmJ zPEWTy@cMPfDK%U0`>sXjJDnLmjdi2WWiQj;E&p4e1gw5lzlW)p*%jk`TA_u(;`b2Q zV=TY?qWuqDfI??N3{*K9Hu^}<0fSf9e(a?!LwpIUGdS!V$vfNYwdxsm6@w=Ke|6`- zTznRW#_W$LwRHK8BqHpUknkH)idH7?{tmMrC!H|;(?9+{&kwq%;7Zl7L;kJz^dlktV^%X8w%>sL9LbpS!JG6SjQuC9{&!D4YyOcA zdYe3z!9On=wTGttTOSl}XMwPtFI&G|jYQDLlWrRQ>hCF>e{*mDg@gHTA@9kBx$_%I zO87icE6&h;{yQ_uIl9FEQ48Sj&;Qf42p^=4?AG9f4qF6y5@KZf0E+*Pw*N}`XBlWT zoPLgmBdDjCXA864M7wA~dFUI9IiW?C2>9nPz_0&F6#nJ8pgKliz2Hkg;%jFzek=8n zJrf^#`v+~Ft=MF-DA@nh*?)@DzqIiNQ3TE>5NW$@u?fZ7(@{eSGdz*A2Emi39U#VXgOL;Lso zBsh%z8Ox_R>tc1c)b;)(cfR_yFTuYnR7}9<3L|+V)8b)cFlKvyjLzZTi>BAzvTQcchLV=*dV$Cf#27Mqw=VJoPR1%AT)G-x)-->3MU#I zmmc^S{jb|ED=dBJ15j7@W52Z@j=e@n0mP1|TBsN-AGxt^MiK zrwsd@L7ks0?-5-o+r8FUioPJKMXfp2)_gG&lJ@&cfdmnhwlAt#X%h^b%qESNjU+NS zf$e8)uR3w+bTiemJSWxP!_uRo|7f@bqUAzmZjQpa8#;;4mrs$uexXlHL$Kwu>^F$p zURhZg85bAu_5)sq`AkuPQsH+xY+pA@pLb&Gb!%S2Iyywigxr3VGD#%-{H>j_QE)WG z(8qTzb`c>VSJ=P(GN&om2coGiwju-YPpr1J5uZS7hCf!9vtWx7@61-=AEMw9^hF-MJgy6 zK`ZnEZ!j1}u zE+^NwV3HojGRo=u=s~eQAO?YINzf1UekuwIg|`aauDG;_B7Q&OBtAEgLgr;(2gSwP zKV@Lk%oHiJR9{S+Y2*^h`S5DOBZ|O7vE5VCyXA5vCnlO9?8WBsGNOM*gt~oPJt*vA zI{bd7tF9Fx@WR>XbQIuxvWTm~JV-V&DeY@j%6jm{`VZCx5<(NmW!IH)s)*UFDCMMx zDiUYR&L6%hnY{lFze@X)WOYR)M%Qa^G&P`IigvZW>-?`7)L;mtyLDDev$i7=xFT*d z&{GsXS%?MTUyDf3r%h*&xfULlH-*nRg5^@}2YQWarfTb03EFqtU0Z$QpX8HZs|9ru zCl!<53Ot>|ELNK-CP{obRs8Yh5qK7x5*=*-%OC|gE#={6qRHj#-9=edshf?(dx<~B z5i9Xc-_(AocKFPMCI=R^$;32#0nC}CZ?o}8QovGgaQA`^_Rg)WohtU5{kTclfON>% z4kkl7OG^Pu5Bhxe!~}$hvmms2YtA;BW}S7eAU`Tf8>!DGhPS^;)=c*i%cQai3yYaEL)-0f24SBti0{T{7)yF@j~D~;3<(p#DeY2Dvb1x z(aj*kSa-#1sTr!^c0bA^+o4N(cvDlRvbcq%%%f9>UrJFvSm>wI{KqDfXdo0ch&|q( z(9bnY{9@eog%xj?6C5Ca_wL>KN!{8vPwz`s=G5yZ0ctNYAH|R#c(5~ttVX@T`Nndk zjV5LWpdZ|^d2c%=Ib*STFcYT4(RAC+QVrcKiHerBJx*%S($C!)owkK4OGU#e49Fms z+DDDD{esoqBErDwdR9hsP`&yZ%G^O6aXS_~b2TQ=Mn;OP5eFe|kqN03_EM=8F*xwg ztT6@#itP>(EFfrSkw8cAg}4iP?fUZQ2%tCc9k!;-bciu1K9dBnL-``oHN1$w*Y0Em zG}uvE_Bb2=d|MUB#POG2>%elR#o>U*59~{jM*>MDWl1z1-n5jhRv7eAlSisDob8LX zZd)ZVLoiJw{LLOwl)*rPct$nCJ{Jbtwp0ky7M~HKEh?{3aT;6%DReFuLNDrDV>Th! z`7;lAT>50tssn*G>B4MQF@FO7AL0JY2odg~5YK%_E)|e*H!1}cDn0Orj;bOF;F85hkGt$D{~C_~)(-38oAj z#rzqzNI^S9+fxNCd(Uv#!SEkxEIpxuFQtSw8$n~~?Hur})I5?3DU)ttqg~{D#2r|B zT4+5DylhpyU(csYT;jNE5&m>S^Hai-IO-o!(*M8I2|q!vo;%ucZ@t&bYz$#)dcs;v z;OGH{Fjv{V`VNWO#Zdx9DDW?a@)O39X#LSKjF7ky;WMuAvPesd31#Bp4}W~q7#yhZ z{2ft#$3}u;VHW+sUl@UL{DS4h)RByCsbAjy#d8LzQNCcQ@UbYt1+;O8yad$#yM!ee zz|+0x!bDJJ3MBru~TjYP;15f3aSSA2kl- zHC|qW>2MD(MK>O1dCU<=!gBUj%KE;SyE)M@XRtPoo4{A87?eem463TQ^e?=BEGEVq za7wIyGxbOjCo^dj`3?fUk=aI9<7r#du@RfjiA)rqU`OL4(<3xSn)Js5GxsNo(w-~q z!NavG4D*S9#kE8TefII?sxEF3@VVJ`??6RbQA~l%P8Lf3Ql`jjsQ#nfhe#W%cE29PuZRo2!STob(Fz zHdUYMoNho%I~Y6y|4l&`Oe4o76&%$hPjD z`@fDOQ3R0sTvDLAqsGYeL;(#ICx-U|ai#1usp^aZ`-Z`@yarid6R73vVO6{YDM?&w0irYJ-RPQqPcWyB$2^fO}kJQ+f&8F6pXzn3X9 zlF{`6RCuf`)jyw=<`-fO!4784qd6bC+9FK_|??|nRJPEXzVW~n0jiWYn&t2^*pMSa{JmXcj$FY!F%vy>cCV9xT=7rQ=P!Y{>xMYQZWe2Mh6DpuKS zsYxagWf~l>a1pp}VwxaC1vvw$jkLq2(?7M;Cy5{5L*7~WAk!&c6h4_HnjEqxDlaeSfwK~PjRfZlu_{%?Hi|u)xI+1BGy4H$*)x#!sLPd!?DAi*t z$qT1b$_x{!{3Zu?JHNEu52TuP=LedgV$Kjf)Kz%oxF$(}UjZfg@Bic~&dZK9aF`&@ z zBB}*e{R`_{LdGAX0&opZJN9J0ZRCO;;YJsNX#)G!ns4CIarZ#{SsTL+!AZnQb6=rd+#O}1n!YM%+vN#D5U$+@}3>kbI;fa$Cgk0 z1Kals@a05FY=&h-Nd&}TFGsl?VBcxwDeQB3T?~FtGH^Pqr2Z&i!onSz6pHw)AsQe# z%RZd+g^>0xQH9zj;i-~2ZZ`zBS`yVFBubo>$_e*0?e|BvBlsif881N$$5R)=vgj5R zJ{Pv3G#Z2F>+@cBf==44Q=by8i|cCq2@)l!oMrHL>s0H4^_?4!zib|%n~x`vv)4KC z(^mRx_pCJ4jx%Jl*xl*{RkgrS1*gkfnp~cD+fs;h*^tQWXEY!2Q3(EV==J0S$gl1t z#L{R~ZGIpzp84K}S#Y*Sq$T^hp3S(NlaKbcqX8#=f5hd9s=+vGd&K?XY4qA<)Me*3 zWJ+(t{H|YYFcxA~6-WSOGpLE(kZjbmq$)|)0T`;HChXumFw0V;;}P93;lgNKBCnyL zAqmfwYCrn>8uOWNLi$2K-taK8z~Ck(-iyi>&6X;o+@Pb&^DyCV8=2^?wZ4C%*)X56 z3RelTrdQOQJdJQm6kkcz%Oky@qMQ~!{DRgmhm6@@?D*wN)kU#kfn=#h0sRI>$@?8r z9w)*8?bkP6g|sSA*sB5V%H}**wE5PGcPCwpzb0sE4zt{fMdD`*yQ8d#hUgEKtPS6G z5R@d?vl!c`Y&y+h&EYBni1u)@hVt%KDi2am-{o2Jgr)@}0?}c-2NnWAHc5UbrlBC6 zv~a^YZceLGOG|+&?sG2iZUD&8=hTbDw4ZLc?a{aygV(^-O^Dbc+I@<4>wMnh@n`p> z)+Gl$iT79NK~;$UTDHsK@)ITdiYR?%p)I*{tgLPMnM^fM2>L-QTf*D?iE>nXop7T$ zze?s_=G|a9Ox|enx(#jk{Y}`N<28olk2cF?nFMm>S=Gg^u(UH^gOV>=at(*8tjm!hljhgl+ zi8TYq+*0-zlUE^cOk=wpe_=c+xtFScI9JsMk@AY z{9@#zZu90ytY_D)mp)GVeMo<5$6@-k-Zlk}x@RqCRiD}ANW zu0gf+9EewVQ*bvH78t%pwS5+oTXKPwlGXXIPoeLX2# zUS=w`?Nfr(DyZulL#*d~A5=LrG8L0rQuKOaw!EGrZ)UnG3aL9+o$9zE)9>rp+<(Iu z52SA&BYk^tCqKRM@}e zanknZ--rUwbHOn3=}Jo7dY*F@r(M|6^E~G)VLG-X9UsdoeXr?w=I$s(WNxBvJk@j4 zFa+{NuHVIEe70JbtMSu~Ovd1BnX;mmeQuVA%v9MA_{BHRO^|tRf_Oz(3&7DLAU1q;!so}RzU^Ljr(_7V!27NRugg}Y?Dn<(R5 zm(%`?CpY-_9n@HQ9o{g;@pDH_#lxjw9VlnleB3ngozS^%J_;{SGY_|Idu5EQd5@sC zrXu{7qa)^%tQ|6kGTo&9649k(GQ1MM?0S^>$_;_1N6E&vI6;RygT+%O!`7o zt%p|a>LGLMF4$%B56%AsSGRV*=lYq_`AdO(>2PJNBtQZsBM_;%NA~s)u;4S>{(R*s zq8~3>@;2FgyZ)o_$HoD(vyv4c5>Fr7iC5;WHugI&9)2Uv4O@pJj3YV;=s6Pw^3hp8Vdbj00{M+EjKuA)?uOl$Z}!%=F>K6KDDj2!>V;V= z9z{E2t}DKPAH5xae{Z)lh}_*J+W5mdY3;{6tebm3?wPsX8Ln<$%n`rKp{&hsUG^x( zOm+z~llx*s!fw3!;clCTe5MG=Mq;v|z2;1+*Th+3UN#9U@9v4eyRv~K5oFEnXDskD z)-A-~d!BJVj3i8)GXwp68O__Z5Isk!FiB#DZ*HMx_#!a~rmdYO0`Uz-etO^b5U%Uo z1ZkTVfs@HVY~WxF7BFo8#=}pVD0>An7vv_!t01YdzLBC z&N+l#sXrkrZ3AAL{hm*NA4DLZzYF7rPlOtN6wI_)D4(E=>;PXIQKRz zdbp>0q{z!_dZO-#6%P-`Y{e|D1MZtL2Fdjcu;I1ocwwa~b#qLfHw~?-1W(R;I%$|P z&xb3=*C`et4X_ICfbzG}>4*?MZC5I?veK~lwb|J0J7^KM@FO_ZGXFO8Am|sij~IU!)=hl|s>i!) zmbl6=J3_hhhtUH0jFk^!@ZUc~T{-Bq%zn^Ge7+kNI_ipv`DMdUC3PiwUjKJdxE+BJ zconbO`zPrmAAhhD{&mk}BTYY?zM`G$^T}y#ddh*Qx`6CGI<8*hXW-F6HsA7j8CZD# zXgsst8L)2ON3ih0-iE+k_i@R$K7yecuj22ERf+3Lf48jA{w{UhTwH(8Oh?-4rpv8Q zMwD@RNJzD@Vba&Pqab#d3ket2fGX$i(kq02<9nBLDkV355Fj}9A!_nb&1Xcx>_ z`h;x0ss%}Y1NzXQZsVyd~pK~Z)$yGcFi9r0*Za%3*_^+$(&~r_BED#PLGiATsHp4iI6{{ zar%lJPJh7pp$X0VJjdJout|YoSILZAOp)xKGXamSOh@ngp2n!;cw4s{CtCDBd7cY; z&u3u!2fWWYJ-3UX$#|c0mMEQIX{+zTnDdq>eb4E5=U!NmO7eZwji=V!)QeVmsP%h0 zY<%J(+`d@mUn6E<*=@Ettw7uOL_hiM7bJ-bq`Be(DKBF+?jQVsYmAkEU2er&PmDx@ zv1crE#0O>0_@T@!UGva9(bn#H{bXaVHzQ%TF>X8SCRyg-vyq6c7};}-F+X?5VGkdP z{Q!G>_Ro5}i=04S1kiVj06IMmqn~~meSGHW@^>wkIpu2+75v-F;h!gn*821tTvpw( zu<4Cic;-E0lLG@qWB<=${U&SMDlGmy4ITSlhmPTx(bHqSM}hoe;qh8}kbM$+0ixh@ zwDbXbT^&h*{MXOjfd8Je%b+}J%6!~>MQiI|PImgYc<#QN@Rsp&AH2Etu`;z@MS*-2 zG=64*1V|u&KxF0~S$OxQMaWRG*ZBkbU)=+ZYnW#0kEL(NiuXUmAtjiv(h8$)>}mDO z0huYUFYb_4R?)y8BO3TdxVvsc*V>tQ zb@EJ8z*KhxmM#-5b@fP)D`WMe82rF{#xF^|L}1lD2^s5N$ISOseCDMEVE%NHaLrvq zMTjDc^kpw#u_1U;LYi5~H1d~;4?k2};YsM#w>eO+7bf1;1NmDQqR$vp7@sTx&7oau zW8JHhVV*Zg!qZFV8fDbNs?rz8>r>exVt1WLKlJ4c-0`Fcw6#!Ngy#>7@LVlbZ zS)MN%)WZ!8>gG30#k6Y@b%NM7OSITz<1()EU6?{67VL@}3^D*5c1MKO-Ba?y~1#){vXTlwF*pH@}edyr+|&Dw`PN zUz4oL>H0--x?X2m_>K|2QexaD>GV|P%E>`iL4kaTLwvl_n)jYg>f(nn{xxXG02X{9LU98`pg>-m zW)IM`)f%=wml;TXTQ>ftS~&aOI#K$_t(ZR5>~GBqSeu@{ zkiBk|elkJ&b`kEIoiTbnBL1#>CrW^9EF9SfBIlMX8MjUxih7yrFnQ*C>I-Y5GtZSL zdGf%b6&lJK9lv>h0)?NVw-KwqaV_7 z={btNH-DFT%J>oJR7DOJ%3%@pPSm|d1P%8qK|^`BHC+PHv`tGzT2_{PAp*tt`e@vw zfi<@HL;1q}lqKV-j9vBXBT>TX%i3sfd;{i9zXn-i#p9=}V-jTzae>nc4T3e#{2Z~0 zBnQ7_JdBeAB#Ft1!L`eM2MmuD05v4tR+4V&;&Cz-+6D45t{#$j;^R?M4~)LdE&8K*O!{`q< zKlCB?KA$S?{7?(Wg3X7#&jrKCr_2#^qZj#~7R=t|eZ(TgFPF!_pV1@iu7JytsZXqFM`KvYL3mq%{V_xUm zWIv;46X}+6#0{Q0?Bq(tnir^XzcG>%8%WSB9Fwrd+FFnGzJvEo(=5~ z+0MH}edZ(z#B*~*WAQpFnJW11D|~nCELyj{3QfFWl#v&ns;vZcpQ+Xe^6hU}h1Feq zBlf}zaiP>&@tI=e9axHMmW@XH+y9KL&PlkiI2*%%-vw*r{7b{W!NaXC!{NhA@vFt7 z18#u?`7$FI zy$sujwlRW~>{*VaQ6jFyO%%&<*Ba|uh}Hs=9$ePn&G?UDL64qTAy$Njz~3kbJ>NiG|4^djuv4!{cBFIm62=ZD;4#It#48)mM_R*Lm z7Wz(Is8lQv?w=@kwqZ5b8ygLma6m6{JvZ3R^_&RggGC^hu>)U&oEnMo8FYGsWxUn& zKH#(*mu2FVH)`2r@UcjillOed*NK%Qouige9^Gw{OpfTFV=&UtlNEHiwEI0+?oAs8 zolb9LS+3)|O?nlhIyi5ZJfz(>(QR3Ril?Kj29My=%xn}0s3@@r`)I)es1+4-85{+y z5xapKL8sWRmi;(^7vveIo?UTbsk7X$)NzJU&!fhovx_zCtWV(nd=w;lkkW@B@3gvi zO-7Q*KpSK-kTv?4zeOe~J(b~#R}1_Sb=N5}c<5L|GBItr`iDl?C-CSzK{WH7*x(%wQ7a``Jey60}nhk&JpW+BPzM4O=~=tP#`^$YQ5JaU(){1eDG=G z*>2`i|Bb<8^Je14=0-!QM$s{Q+!XATNq=rzWIQIxV7@rSaSgOffaqx8xJTGt(_WP} z-_>Rv{Aj6o1~MlJx#EnLWpZZAGC8hrIU8r<>6N=(LDXK$_DF1!hi!Y0TWwOL?Fq-z zt*sU=BgK2n>&knK9;9~)0;LDBbn--G?3YH)u|>8YhWV3wxcXYM=I+c?Z(i-Ybwu8l zojC4{(-BX_cY}W(uzzS#4!S>8@%~|1g1i=X1NH^u4hGq8mj0OhPW93x`u+|!os@=K z%*mp`yf=8I1ADomKz+`*qt>LK@`ETOl=~QOw(MGOwAy!<&r{~zROdW_9_;9 zIt{m&lVStbuiN{LTY)R}r9AEF?|=V$muszj%T_0_u zdsH$}G^qD?*L%8CK-Xgv;^q%o;`J!U^7&ck9kLZj~ zA{aMS-b%5cOI1e_9@daxp#t07k``e%!utZ z2E%(N;qm|)wk^o}^`uiOmizY`%ikr+dv2<{=jwM^=lf?)`fNPCY^PhUB4gswVE6y)8=@(T%&K$s8+N#Da3#Uf%gn?#1HQKE&CG z7HE4#DfZ+ZGeWnxWf1O;2R>N89{*J926}8UU3%e;c9%+rP7Cr9AR+>8e)i>1qnJdb zh{qLQ+jk$V#Gn7^ox5A^e^6ZCpOiAL_fbaS)b=jieN$^A%9CGa;I(ZB5Y^%u#L9bU zu88q!GvhFJ`R6<*p!-al6nfmFTN1PKdeUm`Gc@(>?sUvVqW9CH;N17{M6JwjVpz%)zXUVh1KO#uteKt}Q@SUOEk#f}vfn zM_i0rKfVMdCH0K8)LbvhW3g&#BI@bqrHFa^7xl#WjUv@}tzB=Gu0$;L#T8tRwnTHz zHWEwEoDc$a-==V*kD!m@|HsyY-qLBoIL?b5S3&WA8|7 z{;J0RI(x{sMf2q7f|0%R%-*(UWdYa1H!Qy7o)6Wg#zb6UR^sZ)BO-)9VhG`g>4s^k$&!bo*f?jlxLTLN z>GtA6esYou=zo&|?s4J*_K;YI&v-4({RP$@996F2)39@*a*^k9Veg9c2xi~+h|Eq8 zLi7zfz1cXuQY`c{4k2;mW4L_xI;`BOU%DgFK6w~e#iFOvuQ z8j}Z9KI3+ziW_)&qK=)9$Gb^L>v&^j*Ncofb%@*sTRm{oMA5&Hk#x=|NVV1wBw&9zotY(Lp7k<1iVDATN`)W~otH znK&?H$_RA5zAa+Z`tj0^l+=^9Oq~jH3&nHaUsA!g2MuLe3Df1McTAD$S)Miu;_b8A zw_pbk_w3n&ef#!dy}Weo+_@9qfB(Im_cm?X;MQAjMM_EvZn)tF`?#+ie}jk4u=aiD zJm|IGNq%c{%-@>sPxufWbX@vg*WbPRJTgJiSjNK3>w{OqZU?IFUUOOKBC;jAC3nLm z$V2+3ZQ{{IHnDVHB9`uIa@Kt5BWyFYYI(`-)f;h5jv`~3UJ54>^VX)Kq3)kpxsOTg zg>G>T@$tHq5ZzohGn{DmG7rF7Z(jCWwogVk8FbIyxDslW9H*BYmrHLs8lQ;XH^t)6 z&MnxoTN*lRGY9Em{}!Bs_9}=L0Bj+{OOP+ZQkf_yOVr9H98!eC(QK^TvRfTMq`hR& zLFuE6H~J_lkBUQV!$a7za*OnRtS#h%yyTz#LxCG!{X^ZS^Ssy8U7*he>rWs3R1gu}_>sQ5Ap7wO*%9Y|NZmXLp-@bi&^zGXhx8Hs{+O~B~fcKZz zfbD4+ZKG$L!M?xSOF<>SBumtplRtO7&_l)&ezZjPDBBB^**ACZz2(Y8*k$T^X1-0RQ09b0}{mB-XgzCH|gs=KdY{-c0E_@cFZ$z{at>;i0Wx3Hm5a(?jdtrD=ZzT9p=5>3l`WMQRPLxNCMY-u1CzFmw8NlRzK_X^Y(~Lw131}<}W(FoAe7>V(*rf*r_Mf>2_GR^;R0uG9cf(TO1C3ybfE8 zPDvyld-Zi(E*|Cy`PRg~=IHfcZ(NE3WGr5b11jE#`e+f`S~QuH`0nci;$B{DtuW|8 z@krzp)Q#inPZX}{elHqlufnHC%!Bot#~@a$hYo$6i{sY#X4_uB!%a=Twp`Ym;UelQ zOZ{dj{4N*yC(J_Kc;D}2sb52sGXq(Fzx()Yd?42NCr(%s3+gvQqlm`1>E63h_~*sg zZush#+(frR?4|Yaecso0+AX@=hr8P~!AbET@Y;J?W3VEQXolb3e=n|d@1nY;%`fK< z0o`Y+9fEwqDxB&!(6o%#0{cVXh_$*{z~{|D+KJv+-N6*(2Z|uCEacq;`42C|sothw zUT}CRF8b30c=@4!O6TjJ9fH%kPFh%w= z*?8uQ3-Oei5FpzZdmhF$>w&+IeIBumfE_zzB0=XTaJ70C$b^B1C4ad0?|A-734cOF z{`q$qYfu{(`}q?{koRekDB}B;#gF5u70Ly?OVBCI+l4n*EXSIRPYgHp&Lmwzkht+0 zHON}q*A=3vxGWwwW+b}F8uCuj-W9@Q&ucAU^L$CJSZRu@cJH-<)%g-^o8A{s$fCb` zuv@uf-nHdTEMKt(pX{^D*zBYn7V>(Kz%lr(U%u_&K6mum#6C}Y9Slt)=oPK3nYIzB zPp>pwRJIqlZIi`qn|_gPDYLrg$gFPXrq7v{=$`AMYO&UrATO~T+$iqiS30A!_a`k` z>t^w8pK$KzmtOr6?CYEibWd9{2Uuqc2FTVbi zC%ugm@U*Nm>v!`rfx!r0j&vd=9krgVx8=Y0KuVFgyWJ=5ZV`D2IwV}~+hR{=Sm(ju zHxv6N`&;8q*Rg9tL&ejXW4JHWSx}#z10v3IHDNz+Bd7=B+z0J;Egy>1m-k5#dn0rE zIP4gy)-QL3(c+u^ywor}ox)ZdZTKW(P505rOdslH zxu7{{xB2?J>$$CCa{tufI*IG>IelX7lbTDp1o>qS%Xf)M$D!+{Bu^}-2AP&qc0#g~ zAnyOijFII})KV@%ZdLW%_m{k{_CRJ~Sr(S&Rq-7Y#gcL#cJ4i7MUyCAPf}f8Py9t* z#mCDtczF2mVMB~xym&D_{q$257Z;;(o=e`5U%FdF@o)z2*|3 zlBd0f;R#V@+6AA=GA}Pm-^28p(=dBJ9(;D2oQuJvS7*!1td&K_z`vRVW(#t1I$12+ zhUjmp0JYijX;T92SXi5Scg$ucuyu`67TDDNCPcwxiFJmpK~~ z7wt5zGW!cKt!(Ai_v}CDzL70t=r+~ruN3K}fg(F%*6R}xYi0(ed6{@%;v(S?Era$8 z5u2k$fEek^%yzYrr6NcuSGT9-LcdmjR-_^N@bx^f%gO$sJQtt#y?lVKd)*(cVE?eJ z{*XQwurHW`p!1m^-cNFK_Ti*>bh;#=gJFdlY&tV<$~J`Z;NBZN)6xA({o8fk!1|o| zo9iEG3i83F<0BqFg1rBAohy;Pxa-c@XS%MA>yRMgn~p1RId94~1^IyWJKX&p%~|5g z?3brK-M)Q0UVH5|y!F;wGSGNJjD({xWXKQ<95@gO3BlzrV0)_DH6LxGcR7Q6e;3b1 zraSshSe7ML%JsCndOaaD57Wg1UWUUflQS*d_mg#;kTv}_Y#ph*D!KaYI;;-oYOD2o zfMYB{2M*%1>e_18JDA;$`N@yHESyZHUm}y~tyUd&n;9t{_FgwV>}?!(pRAcKljYaS zq#qrhc;dTHmR}2K8Lz`zp9gcYuymn#QafZGh!pXA%&7jjxs86yH;?OfN_~rI-x1BN z43>&p`hugfWPmaRkS((PNW40^yZkm?WjEV)!kpn9tg!NX7jJX2;*DE>!Q>&9@oVLtvqFaBM|qwBD*u&z4)}Wp==bVS z3G(ZQ-;C$>08hU05$=t3ypI*;VRP2!qJ~&Eiz<}3I;j<7q7<3zP-7K)xEDcgjdD#U5V>%uJB95B7T`z z#7D*8{^56^ftw&7jay{Wd%TF>kaQMHI!aP+CD!!G%9`HTWCFobnLtn~F~|HC!;{Y1 zf_zk~E*PHN&I)%D@2+?aUyIh>O04aZ;~QJYJwMLi-KGDBuS><{{qKh0N~P5~(r(_i zr!o1@yh=O+tbYxA3@i8+GV$PUG7{;Vp+>FHm%I_ zxbk_ta;b2w9gFMim8 z=l^t$!Si#mW7jV1*ph*aPn1AqZoS3gerTzhR047nJdY5WCrY`%&qk3$3^^_9zP60N32mC>nZl4>>)urum)L9#fW{SM;{NfA z%n;G-F;XnlPw(A=Wn0YhsuSi*_vsL%Hxt<+U@MD!wJCRf6XfMxz0w`b7I(SNW*Emk z2k?!9>W>x@f+$zy77^s#*0Z%^vBhk^11UkLBZ3lUyCMJpKmbWZK~#L28bk;@XgJ)L zyUd`|p&+l^(VJVo4E*(#fj_mR#!Kk7YdU6Zbxjb`!k%kD&$`Ux`C|+6UKaAEQ15xY z*Fd31eC~Zbu~4_Hzm*{0Y{gJ{W{RBHG#O)u$WuoitOvD){<-fjS^w>UdOoRV_ufkqF>Pk5Z4F-}LgF|P64eV; zya=9Yw&3Y6jjBCfp20(PuNKKVBFd_7?b@}5D1YC5_u;O)?n1|o9nq*!Bh;x=r_2od z8$5Jgr0+X_YslM6N$C4s|NfkP3@bt%m%i8acdy@-j>FL>apuY2e%0M;BFKwn_$K4c zTP(g)1|z9sys<8PL#c;SwOom0Drc!ViB5ZXaptq&U=DI*R{7-#H{+5cmIoiN_R>S_ z5?@B91)sRZpEVAb+w&_TaXN1qCMwG}6~~;8m~2@3DepR-URw%eQ2s&35;qkWH#wG<205%qSll(J^I|VbF$%W$0r9pX+aACPDml$_y0U#|FFFM(TevE zwHWTJ@0hD5X9m+3OoDSZrWGD_Xc(2YT+typED&=T;o^f{luRjZwO~q znSS}odDA`-8(w38w->*>SQwET%6Y9AhsayUPcwGTb#xN)}x=IW&(uiR-R{oDnezS z1OA=?dT=>Zg8bVe$UiND{G+dbgp{jXZ@FJCy#s^iDS5|zYxBtKSUJ2UR;K(0f5-)< ziI9J{Q^=R#U#|ae41R7q+CKU^mZn_g9zi+3QIIbuFa{?95;!LWLfiK^gF|2aQ#>nu zjqi>e!70}eqFI(^Z87ZLn{d{0(T@79(7k&HM5qPiWs8upgS-zniC8WBEpEYxyJC@- zCj;>fE=6oh@jiO8P}W*MhOhU2hCTaLAQ9w8xC`=E{B{^_x6I15D|pnu0>g&f4p++{ zQje??pS$-v>DI)fU5oE6K|V^{#19u&^JaF{->){Wz$J6ZQUw0j zMBsN_UnpfT?+0 z9PtE_E{nC-X;(*!4==^=!=1eZ`IjF$B^L0e%X*g}pNA)&{uN$O7m6j&up2syh+g0j zB6SQ43Q)Ej~2Pxjvi;=1%T zczM8f+!@;eu@_#bPl?5RHU`eT&amir9rWG{EXcbH8+Hu#jB`_vKQ_yJ+0_<}I-Guy zH7zz{utx;=J*6$sqeo|#Riv`GpEDj$ZQLizp)SM1&N+s#)70T!PREC1dyxMAel)x8 zHgt|{Y#+9FapM$t!7BOt2954pE8}C3DWu+-=C9lew>TsK^T_%?; z$SaZFOv~jZz?If9Y)gM1rOmEGx6ZMy!vXd}*$5WpWg$&om}SzovFY{RbQbiBtoPc5 zF1M`4K=#7n#*44M&KOi1=i9PcuRX7<-yVA@4L4eD=A0H-Y8|)Xk?X|8;}%zxdav6w ztQ`i^^`?)tBklEC?Lhz$!#|oYYsYTWF_@o`F^Iq95cbOKcs0n`A1gzWo^6ru!_9y8 zd|7{a+i{;09E?oK%e>)=KZtT^3wiH#E?qDFz9v$XFC9IAY+CHys3!>;k?6q$J)yzJ zXhJ#4<2eBAJ$PYOTTS^JxDnKrF?#Zk>se)c@o*GYi*LEBM;h+oJ%v5}9Myty$OL&= z+QYOMa%2YDdI)%CLzdyPevp{RCqMrVrY5>if-2mz4O{n>;<~Qg(85|n-b*-iMy>bl zXJ>+}i@nHR7u!7TT=TPHF6%W(p`mdvzaFE$fP^J6GPpLzRV*cX9Hx_-{7J1GQXm^4y5}_4nf}RJ9z&7yzf81ackis zYy^4p9vdfp4$s-}EyWG&a2Z7JAo{dvq3T)+MI-Y3)cq4NtrIcb>7_@vP3E9@>_JPo z_l&^pdo&(@JP9wnut`kiM4W%!vri+|a5Epkg573+L1|w)pPnMA&X@gL(1Y|wKECOjyXxeX9)449q7 z@#Cg!7%3Lq<`|kf7AKx4CRmedLz%wkv!s0G)E4pq>sLLLs$eT`dwN6$R6qLYBc!LN zP^g$@dnku?Pho~ z((nAT{E`UvL$7!B*H!Afto7y1zMm{tG*Xr;5({}x4}#upwQkQHV+nqW+e~+{TDK$4bc%8o`hOoE<@-J_EjWme)0bkwrrl<;))F}K zfHar3kjehj(t4wbOzby){c5s*K^~SqI~RM^0crPNm3Cjf=IXK=V*{2a4%d=8x>WM1 zC!OndShw|#@dq8+8K*gc-MydU>zS*st?tkAJ_A&KEAt%i_YBbQ)u9pO3$cFqZww3h zNB;Cz48Otkn=Z`z44)q=F|aEyRzGZh2$MI!64~n^?brSAj`%eA(_dG@ahVamByT-# z?)MzfeI`D7_6~PJ#Rw7n9}4opvtmdRAc1p7pep+wXG#i@FD}}%WpbxpDqgv1zvj?J z5xG6pde1VO?&#}_my-{(KEc=d$1QjEj&NQU^6rAR7SjzY_&ct2TAjqpI5?!$27nJm*~ebg0V@qfGZ zF5__tzn<3wbf2kq2=WO_@J;W5XmTbG&%bbu2(|IkpKq|VZLAzCz#IR_L%(awIRB-b zze=5VSlHjZ3jh3dXPJh&14|Ca;>iwjzThvoXnr5@hx;<}`#p@y$9>h!d)K$FOSH_!>x?7#v1d3dRDRCGlg8XtR z<~R}6Kaog=JikiTsE;+GgE%Z%J+dccIb#urHi$UX!aDB!D3q7w?xu_7PE+vCoE|VG zq!R2}o+|F5O(04v=_5Aqrh9u4)YDz-DCcc?LGrRuB6W!RdXOM)G~N-n(>l&{c`;Y6 z>*9vIgoR@|+nw`WBCMsF8O8Rgi73ezcm7*ss&3;p=+Vb98Nr!#*$-7C=q6IPlo!7`l)o$@!*SIX>j2H7{d3pbMpS!zYd68}y zcN3Cgk6JU<-CyjDxby&lxXGI*Zt`qNrSu4L_U^;cg42j>(h?my#3EY;*yrj2_Dk|Y zlHSqR#kx;M+kLs;)0ThEfd^;#Wue=k%XQSUvhx*}PDf2_NY!q9oe2enuq4?gwrJ^+24rAg@ZL*RRwi)Z%U7F7Y&yDxOBf zHD>1YZq}rwe0lEmHJ*E+7vz=U`!iTQRi{;g9Sc%1-dK)g2;SK~2930Pd>J5pe#3mR z`nKkiipV&!d#X%+f*jwD&LzIB^K}{BKb_C%iP$iwkF}1yJf~MbF3(Q&^mmDsdZ1Ht za*xaP&2Km@h@z`ZK$vcNJ&`3$b}g5*URG)80aO3=rU!J5L0M^|hDsZ?Pp+IOZDb$2 zjZ|J(`WhYu%)CY!OQP(_mnllO^+2MpNn{d7sGj>S67GA!vnQn4mS!-j?>vonTL zaXWgyayu&CROSr#Jz(k#UzPh689Y?^`SQyzjkRxYx#gD73J4OnzVG}_Mv=advEI9V zpSi62rmV_D59_|j-?bWjQ|B7^^u6vkZrvxb1$nujW%-@zzSra~!oi$u?EH8eKKe8p z`z>)yM#M4ZrB_ifTTSK@L0%@x4fZigt-KmU$&7eCkZ-$-HK?dR?){Irf^_kcv&=Z# zzFV>^sbCGrt0OWf-$4F1yrcuVw%Lc@$vq6|J1mjDLqp>V{ihU}Y4>B%UOfR#iwpX@ zE40p+{aet3^v0lle|#ifXBH_1AhJD@yGd@OU!%9k1`Wirupv4mCn9h5Ls%xWB@-tt zk#^h_zY1G=&?=J_0(1l+`V8iIp!0{N@utdu%6Cvxp8*HQTBbI0y->i<@fLIdC9%cnQ+irmpzF0 zlkBAv@a&dDBFL)|vjBp;nlN+R`**GKwp9AV8OO~&Z+X&JC3D|`FK5GzrpJ&#+xg1+<%d^~5A0e~@yUdOH5dPiKUIBIjoc|Hj8s>~IA4)}Wp==bW- z2=c&ZOH=UByj-A*i06OwFP@itxe_-I{)3kwzvQEpXlK3ds`9E$3-ZnS{x3F9y2(6P z!M_ybD+rF4kpKx)DS=RoNfqqfjI|%`7en;lV#w{jgzpk$tz3rO6JpffDiPfqe1<=( zFNi>k7}0ku3wc%0h=?fpe-;f|-+)_NAHm{P+btnJ0@2M|;L>I-(V~eMLT>+B`I{F3 z+hHNEzgxsyf_x%|-*J_7)b*j39E^ylhYQYrXJkdG2J!;?M3>YVWNd?uj#OtnLhk6D6K!-tzL;m_nD`-fs&ec9E< z_u?OxqO%C@*~S|0s>;Q3{mg}8xo%d!66EX8--y3YUX2d=n5f~UW~ADGT~t17!ixK zHM|)^r_u8lCdzgC)7gXZqAX$>(`$yTwc5sY!q{^k5OzC4wW9MPQ&F(MOdhoF90RWYN%YAIhjMvcHk+A|M8idfC{MnRTL?8rKT zr2eVskzoFAj4YP%;%uCE`%|9uW@7w;-Qome4AxFga*HKf*06teOBojOPt;YJ(#z7I^G8gqhlorV= z>$2WTUp<%9vupqRuIJ>C`R`!N^=j&oYpD+vJg!qR;+h>pMuzo#?NmqKX)aK+fMkS1ngtF%M4(IZYqM!BdZ9*q` zF3%BHrFsC>dm@FKAV08;+oX!730)yg8TLtCnN68;S--(>S+Cvy+v|t>$a^SWetbl0 z916BQIG8EVwne*rdicn(CL;Ou%Hc2ZDmY#d0iiP4@HcqqjMMgPArlH0$-UvViO67D z#5WbIybXp`-j3-#Fh0ZVlypi?M&tcEu+`B?(f6>v_q~2sTDXvBK&H%M6vu7$gvaW+ z*A60c=@y)fOG59PV{HVb1vywa;c1BT7{gmlN6S^auYN#g+I@y7-c7{16m7ZYHBj!+ z|A@2(&CwArt(_o(ymLE(tc_bxbV*0_=y=*3TvuzjFIk8i-5RBXm@Vte@03f;HVJs$ zvUK-sXLZ|U=U?|VtU>#O@>`iAllIz=nTN?q?tP$woGheg7olU19zePo&X=ftFw5#^ zc$M8*wuqhZf-A`#A-=-pbiO)c2IH{!l^CeZkZAvI~Lw&2G28E@iOkG|#Ev zdxM#f_9PXs&v~+v?vUiUm9nnPd;7xJ)>Du# zyV3%j_1CA?gUg{3`d@LcPuIeS@W(su!Lno8LOxb3>&><2JuT~1 zM0$OBx~vaS$vjO0Bv4}sgre_ZyJwGRf%`_>Eqw&pQ&~T@>0BEy|&|(4$8&6Yb!4P0&`YhgD3YV;?HCMf;*e&vo2BI6yyhqX!acb zIdB}V)y z4%=J74;#^2EatcQxK^zrzi~%z^m$i=%K)2iL0(yK4#jwJ>E?TDDx%D77&C5+%WZs# zSaHsH9CNoiu4&p8&rH1y>EoX=W{o!;y%6a`+ah<%0*rrZg-c{I5|Q=dpMM5J6JzZp z^R_L=Q%}5X2^{9(rk#f1g{dRZ)|$O;2lm)06rp@-s-)`>;p39WqEFOT%-Wc54~nbV z?<2(`%@GWpMn5Cr((#00rRfsTcdec(mYaH#fF1;wCGPXKVb)Vmi0fuYCPYko=7p*D zI`{V4_71C0M`(Mu@{OQ=ffmhpDo3{o<`I&f7Qv6JSX zx1Sbq8{*#mCE34RoiED6%m*hzjNV13o`uJrn8^sgnGa%- z3EG|zP?>ae`SqvUfJlIQL0=P2pMe`em$0j?YCQ8_D9@hmmNl$1Oy0Hy81%#nM{s#I zPD#f5Gv*>kynbZK{cO9ex24G~^jf9nlTyaje&cHC{0_SOd7`(d4mH z5Vds%Hs*AaWlQGBvL*iOO*8ByR#9^dtEfV3Suhn(t#F@shPbYH{+WrEHN3<(PQ0i{ zkg~W`{<*J9>G>Iad!Vfw<)Mp_Gv=VXHQC3xhJ2zti&Cv;k!MQh>U-D;2vYE4I~K?Q z{faW`*mYmo_kjoi3sMKca7JOlbM;Yjpzn2CkoWjat0i*`PwahVP@GM>Wi~9D=(AcL*A6aCi5?ZGZs=3(g>evy=Bb-=3|r-#Po9t^K)GPf<`z zKhu3{ci-LD^=6(wFB=s)=Ls18c*#P#n5?Z+4mKe=aC^8l3c9F6sa7r@&;l*qT_`!% zq_qGohJA=_VpdvjQHvNejv|b4LO?>PmQON%DWXD-96JSTwg#P+k;Q42{MdQ~dm_v5 z690Q1hi?v}y{fDdM#6a95y2J9E_W56$?HVYw+D6-KqVaCh+&AytK{c3!IEe}u%718T_i;HPTW zb5JJ2;}5Bk7pN<&o~p(k)nDHR+-76VIwT9=#dW-T5vpnu(3iNzLv<-#teFi%K5+>+ zaZYBY+*RxGq`Jo2Dx-UNkB6mWahNE;37PV%y17Ra$;xdO9VKabsIr0AP>6HGtb6n~ z!!L5d;OM=W@Qk}`wc$#RdggbGOEJ+d`qNB+Y+jiGUc=26U2GbULl%*HA zmLnTQR7{gB`EiwH#oWvM@m3lvQ zCP0N@6RxQF((8J;!LX#ozv}%LAOq5P;>XR+H>7@(GBQd8*TVobDyE(3HAe9%KeR` zMGs%ljyv)SF}|5nX{+vsO4KIn`5BK~-h^3t)le@ApT52(GF9s2kY z@rwJj^(Ri90>+J0kUJbNFjoW>o}(Oh?qbjUG7`Huh>yoBrGeRvoA1vxtSQ*9`7jO| zx5<)^$L$Y;y+DK?`)xp9l_mroK$Ptc5duBW!SrUonJ8G2e5F9yd+KrFb~`UI2{zWE zYH+CmChP}eWv;vC2wj1E&%0u2VgrEH?iF85seM%3Fq`I-ici_g^)9n{Lrir$2C2}S zX-@Q+jg#XY=8#~Fjr`{(PN<}1YrG?3x< zTljWpfXbMta0TyPgu^u(%p6eiT~Xh9n8`^peO7GWyKcj?_qh#L8zcsGI>PB-gDd=n zsX&4#=jHS7pyej%VrSg*em-?cUikZEr`EEnemFX_8#cQ{l`s3Ppvc}&*Yt*hRJ8;H zEcCWt>syI6#>smZR4zIUQJZyry-R zgP_Jo>}~eo4?c2=cN&jx*&kM5Nt~|bhDY6a92A$Rvtlgua*a{({=*1|lAkH7+J1b1 z0R#+@aa5F&p2#D7WjvfT%wg%DdzLKZ8WJ1M@V~^kozySeZP)mQqs}t-3tR+lk}Z$b z!+8}h@Lcj&9XaMzo0_BizlnDp@B1b0gh#riReiLdnl_0W_H1tLH|p^CoO5e>9g5R4BzG+LWc(l^X$4t`-lMkh- z@qEJ-tzxWVz5S7k8+-3cID+-#xEIc)#4u|8O01>*tC&9}FHtAu68)w|-p6_Er(=GJ z6gW}5f|5_g!weK=9zxDgorDDg|AbiNtQZZ&TeT(lmQWO~ZI005MOMO6Q8*0CIp)jh zZ^Tif6rRMK`aJPDyOLV9{ey>9V|=CZ)w%ahzCr>qw9NTq*uS108_R>Ozs+PpD1FU1 zVd15=Wk;}PJ}08A>(mRhj3Lo0!UIZ!R|i)|Q|j{;U>kwTGrrdnSx5OF891LQJ*aJM~-H9rIV$n~fE-H}B7(#`pk%lvabG(b&26 zCz_MXnz3l-UuhBO(1u9>ABd0idL}B^`l5{O#7+7Xz9>HCU|5uV_hTJG zk?ng1zX3WYMf1TYR;*eW_-bV`BRDLqH1<{lk44^96E z-=APmmE>o8YG3}{$@@t$AYo}ViB_sS{!SCN8ATG>Gdss8qwy*wL|@RSH8J!1)r@LF zYBmiMJB{ec@<%-i(aIu(k_u8!33qONYiX0Ud4m2=MitKijUSEX;W={yHcAE^*%V)E zkvIty82WH9L~cO%Ge7^UAr#PPb4dCF(JEDJp-SB4zRU4;NmE~zh3|kFe zRiUoOL|ug#<<#vemWVj%+b5Ub6m6;1AkNCvzWCJ4<<(*XD$6ctLBeZEtjGfCZ}w-y zv8L0fn09uB5 zVnUF?R=Q1~WCyCF^fA8b`O@h4!fzI!cSnTV7N%E+NLQc5=tbjsNd&(Ha9GZ)4D6Q~ zMcL;>6_nW8E_)Etn@@j#9S%U%n||c_+R!=u^j0yVXGBG`KUFpPTqF5>*?NLA$LXkN zNw=#1{dUEy7PV+t2t+C;l28i}jKw2|pjFA6YP^lSG&Y4O)$#)q9vL zxJ$>c4&gzDa|niygCFSQiHQ%8lfFRGQVkrj{KCZ)HdN&6bNmH4M^zmb-02c<=v)d# zOo~Cba+p7VUKu`k#oA7=dGXF!Xw$bIyIz0uOE||_fn^1}%5l2liudNyL!z!Y_=c`P zfUOUGJKiD(<|Lc;Z477&|GJFLZ*ZX0lSbp~fVE>|VnADnw-_w}^+I?sI1sZF;$O`}RB>Ke4 zI(hcF9u@0G>f$kjdlCTTcI29PLKW_8gpUe8(e_O;g{9QG7a}2>uTX*ehkHVX> z0zKHT%q~X(wpwwj4uMayiBRm%6|N@GxmNfM_k|x3^lFSI;fzmSA(sBBebP-boa37Z zsRnD;g@U;7>MT06K39-+C{%|rqy~?}WLQb#!`ErtZF>5Q+&n6EHWP;9ymyi61)M;J zo(Orp4&>YTsR?x#r%>>kXeDZ0H*A z{IW*-0ConeHB$gHk)a$cyZPg41$nP2xB^4-SjU@Vw^vqdrgJ8vJ9RSz2v1&02;E5} zXcvcHh5?n006K-eBHReVC^1-LCP5j9<_BgdnNA8DtHhjy}(Lku~es*GOF^%IlC5eD&!=8< zJB+nFfNJ=G&?{C@b})mW0O!56JG_|YlT5GIbP|sgV%f>KxdeElVyT-n>t@ljO_~cX zli?mZaEjcGq>foK=1do8UAzkWCFAace8Aaz(9?y&GK2z$R{jHQkS(QkA+AbHoSU0F ziJM2?NA6f_p`s3??Q2beM!BOAmAT=aC29FUkg){}P{ing=bFU~%82_yQ`xe>^{wIe ziu64&fm{7M%}tg#;XYGmS%|z-G#GP9C&*p_s_%}DhquWqHKIPZCM}cVFhIx!-)RaD z@NSxY%LtX0?!Z4SU7s$sLM)@ni28vzVS}KxgqpqZ5jfLx-c0S?A3C~jdiok05ZAWQ zQzn71BfsJcWc~ND{BrH(iHLt{6BP-6 zC64|z3CJ}NpUGrVJ0~)NrDwVuhepPtuKrMv_)EUMpqO-0#BO`@FE#SY^ZZs7f`tdP zT{`}`AM#CNFD|@X=g^>Y0SgX43AY}8ZlJ-B9auATbtnk*SWYwGGQcmn{G_2~sQQX6 zFD=n#9UA@7eGbK02B8D>gE{>kyEs+ZD&qADPKR%=`6S)xUZwfONC$>`=`BSExYjvpXsQz(gwKJofuB{~W5c zgrf&t1NEZs^zC-tYLI6EPKu=7hJpMhrf>8OQ`8s_{-gV|)`%V|69%|)EBspPt+ z@`aby@mCUOsRL0mh|lwsm?E8a&heIxjhozVsn9{P|HG5@a~@bm4~=+Gp9=D=UH z9z=|A1Ocd@ZF>HJ6VKGN&H_H9c?oIvs~ zwM7KVJ*LUK;a1sW(QZ@<;b%SRrWu+~*Ro zz(HbosO;VqZhHeY#sK^i#=He?k`-J~{~2=w%2Yg=57|=xG`*1pRx>zoJB@z?{zq{B z&ne~FI_li!5)kN9Fa|KEr&Jm9|BY$9=j z00Azrga`{>n^%8iXY-20qT|gUC!{iH!lXjZft;cbdvUhy2R8qqqVwOU>$<9I8A4G_ zv2*<>kE!8-@Q-x^ynt&`90f-o`;N$_^xY>m;s?Z);p;K#o&*8Xjq_k8YZf!&B5F1> z(fFr%t3MHI-U^O1AB=k1OfA*`pC4jM3~mtpfa(7u9{gE5$iZ)5y2j&Va)I`{qDX#1 z5&sdSVMn0QvdR|p%Yy}g_eRgF5Y1xMjZyASSwlk#I(mAIV%033an?UXg`)eGWWUYo~e=M*b4lTf?ciy-5`A*v&eS6de%z#=F0>n8#sfA|*tmzCjNyq<@5@U$XhIkEPS=X7t= zE5zi&nLe!6+C1!f4z-E3^%I8ID2>xh)iKnZaBmi+l z?&H(r!*xHmuYV+W2Z`&!Jm*3^rZRjO_Nez4Git(~7c3DG5tDF+3+DtD&0qC@1xo8O zv(wXM8-N^q*w+0HSup)m7@5o;`a3B*rmm}3-ZpSOUC*o7jp)HGZ+0D;#&7xc_Tr28 z`M6O1ypplO;M2ZNCtu$+23ON}Wa0YXDE&2YQdrDV(j6f>G_`)lxVbcb#vk}G^e7P% z=Z-obFS!kUF6Wv~doY$10WnG+(;3u}pU*wpe`&ov{^mioIbO>AFP(}DxHs8>1ih~} zLc2{qgvX7DNU*XF+b6K&Y3Uk{cyV%+`ltP6|JKZKgIX@@azI|xzLV=Epk=^S1wP{( zDswx$zxS@QUBZth7pf#WKF~@-$G~vho1``Twqzc{th`-mj0RnS3-m3GA*y;DPD=Vm ziML&F7E9>k(kUj|hx9Hu+d|Y!JadmSW7(fSf3i_B)5E_J9^s~^-{0>_K6U%QK$(Qo zPrdpwT>y=TgX-h{(YG5p9LzYAfY-Rb{pr%=$-MmRder2xM*Csj1AycP;3fP6&p&H{ zV5`>K`fW)&)a_w5WF>fpU&ZB`B#L!u56eq+@R=x}04A=F2F$KL`q*x4u(Z=V0s*5bs_j6^N*4NC88vIJ7GsAcD^?1!zN zIkV7zHe^sIKx3IYIo;@G;o_)F+pj-wYr96ie0@U{hwdfrx^t&z)7`sz z!6$fUZ^~S$;77pWLq-+wUtY-$Nl)UJXqZzaTLBCsB8b08>`%c5&(YN00+2K3{9N0L z99;KjdHdWLzp|FOUz&tp>@JyI*^= zwc*FYoj|+tnB>2`Ax`Ru4sNCJs{6l@iTwu(s(=3Z9F0Y~qrtW=48SQc?DTIPkUC=c z({`R6PMrgfHGk_-A4g;v(s4%Ofzt&2CD#A*Pyc1b?*ZOND%CW$Lt zeYxN8;rbLkNGklpe)IR95(~JX%bF2V#s02p;7<2{B5db(Dtp{Lm zb-@$T8zM@`zvQ{CLOspOXhit8-UTuH%SV$1zcQnJxh0b>Di04yf75m0GkiGl?+uy0 zVHhvs-GA1_q(>C0ykav{}#QTy38M&$|D-I{rg@OBm}pq zjsG6^Ar|AGd4M)WE&TY!-xM|O8r-YkN%6nKefa;C^Z%9ef9E;5N=P@C9(0#3*y)@_Kmf0m2&0mu1o^utj%=oYJ)$ALDyV#S#3!EY|(4(FDUA<#uc#QNX zWxAp@5h82JR69S)jrTfDvv^!JCNqtxIl!a%gAAT^qfmeBT#1A?wVj%p3XAu;hT{kYFXLF0X%=Je z&sShv{(8(&0|6zUFfX6rRIPjxK&_=R3Y3D z*7Wqu;zKOO;q4t2!`e6Y$|u)BzR26NVrtVtg`C`n=%cu=S+kmPILmzW+r!Rpq~SgL;uFi0{mC)*RQ}b7Ebr(bk;04!b)Z3@_)a z&132JYFK(|60~#Q1LcC{+->Uh7P9P5jTQuQNyg~@^E@MkMV}i4KeSe+f;QW`jDxZ4 z*%m@X)Y=J-Rxz=6Vh7;w_}D3h=>)VbCt8UgT5jMM$Tb75d$t#kMS5~$c)lj8pq+OJ zGWM}c8`~)zK6)%x<#JB>^uOFUbNzNApQVzq0xb#hFv-B*q?e7;$-po5>!{P4)ZXgY z#27SljZn>-sD{{Yf9PogsGyxG{rN<<-(&EsT^#X#$7b1HwMgOnQ04+`N)L_yc zLvc;)&{j<)g97UfemP&Zf6xEOhe}A4)pt;~*0+9#63KqIV^7;Ddd)fiP8^G53FlH* z&ED2}Gvy}%^VtCYuLIh^skSsPqlG>w5AeUpLK`Yl%|)>p)Rv#2(Y_Ar*@Cbjw-A}c z)wGZo2<)*VLw^Ln5n-If42jb{Kg?-~q{zDX`X| z%|GWD%>>yM?1_Q*FjkYf)Cjbh{_dpW*KO$)eJkO03Fs5Q<$mMB9Blc@)$bmMuH$-O zXeEt&vz>{GW#QuGVdH)ST%LjZ08k5aA@+HNNL0MYMJh$E;S4ZG63n70`Rf3V@)g8< zue8G^B&2h(Bow9lUQ@Fje)NO(7CtHBnPBIAGz^h^Vo=;E_g;a)nQ#&$WH!r<~*8w^k$1pHPE=C4qdT)}! z>2R?#{xJo1dfJ=6N_XErJJY4{pboAskg9G1{hl(FNH@XAD`?zl`u+GwviyYIXQVx2 z!9{2WY+2xOwjz5hjik@Xe{@5lBEKXLLanCXfgVx4ww-r9XWDL^FxjK_yB2D&^~A}i zb^+M*>j3CAD$eI9fI>*8x@* zOsYf|{=DtmKbqv6nT8)@x>f#p8%9Th2qCxab)01=}LCW`cPvlkHeY;WO+EF0n*vp_$`vzyOR*oj;2Frp1oBCB58vzI!x-@ zw*j@qLW_+FV0;Tj8`L>bL2Tyrw(l1jO&DAsIy26WOK|5d(`t3;60n$5zXA@dS*Z!& zoL>LgYOZ7s@n?24--wRLuR86N8Iz6sBuTdEO{bRI=EN-rX>iAHcUBwhQ72IGPE_8@F0|djtm6JzEZGF&#FQ3iCgy zPNTX_vgsw)>{bwp9js4Iv`-z|5Df;xnUb5(op1X*v)tHIEaltY=&U|Zie+e)OulK7 zyiXE^+LYM?PRl_N+vmEz5UuQ)DZOdWj+WinW*u&9wY|(BURRLTH^mWdF z1O@`~vSi}9LH+K$$`0q7_G|JE=umbPbJO+$4MT6ktu4>6Z%r)V*TteOD&WX3HjQTY zN2;bb!t=#UAN#C7c3%PFrU!`1@Y)%^wITB5_d%LeLBia>hNB21Y1O^kNM4Fm!t^>m zr8Rx^z>ko7W+FMHS&+n6raYVQ>YkA|Ajz39d&*uEm>iKZP9U8zBhPt%Km4 z*s^1HY_ypL+J|Z``^y@K)mJMbF1*Ly0vPW?-IdqWU4M}%=}QPf+Ppl(dWl}XH4=sQ z&iL2C#!JS$t+{bsfj$Kz4E|=*5u*h_+q%(0_6@}CKeZgVMU`+o_X}__hg7gG+_85` z#Dhx>08kCns z2JvRwn1FsFn8xHs3fFI7*IsZM$#TOd*L{xV8-}42SHIv^8k0v)Zt2@0hOr^i$g!gr z8aJZ-9t+QQ!)b85A4eXVT%E+dJ-_lMVn05B=Rb{6#%>y+#O?|n_3RlWa-KWdW}UO# zs(@{C?1WBn+|3?+HlfCd9(Xzmgt1*#90Q{#deT2~dQ;d#hHd~n^5qa6?9AAJoA$$l zBg$&s4xREDFlE!*32^ud7X4R934RonJ0drLIMUiI5>9OZpxs_ruGA0muCAo!6&e-6 z5o;90$WJ=BJ9!FVIy*R3oqt=dydRr{zZn>aGU*+A0u`SW!$@)Un2kYsTf@xi-Lav! zB4X`-U=^hluQk& zK7R##eEYP+xbl^>%3&3C<@OW1m^0{`SRoMpjac-4{=eRHQ4W6B0r#xXR`BFMUZ%Ri zytgWj%u_C}kBOQ}&l7Z^<2vx0bvm0HjdzehO>Wu=xrt@b7;_lI`WYwv*8R+PB`+&9 z`BgtejHJu9>V7l(c^i_+VXX!7;)2Z$u%+Ivzonr0M)&dih0v#fQTUk`@^4Xn=@Spt z@^5S(8fWWy$TKcVGFtB~1y6b=-_`aJ_SJiu(M*oR={Uc^SBBKx<;%+;53AL^16 zaTWNAH%=wS4JIE6-J%7X*+zT0NWoA`Si5lSB1j#Cx=9k>MXd-mtv*T`~~e z-ls-cR1TC{NFh|Z4|`)KSyaE{G34QJQ*_w$PF=TzkIQl496e*SY!71P0wp3jf1y!=f0Zo2t8CwO)e+PIPv<0sKn!= z3HJ~QL=V?IBQAcj0|#7KHWxra95l6e?z^wLP;u6ZE8RAlE1*R)l(Ajq=jDV$oR0IN z@vedgrs6^^h+!rdN4>{5oI}lT-hALa_8yePy#MTopti#JmufW z=L_)8-HkejePF_b6in3K)z+1JE?}VJ1UpQqgpdAD^- zp8evEfX8k_#D!=M)~ok$R!{&aKfL=?8=sBm1$xxwab$RJ&w6Ik;fK}Tcw#^JE=Z-DhY|2t&g>^BX_4PhXQm2Dov`GSxZ5qV6gn~00 zd- z?VlCcDid3bAJ=g??onh{B)_*Z`10E)rMO;4D_h;UTlk4x3g$9#U;JnhG<+y~Re2QK zoS^Wj%Z{c>`Z`0&g)*5)o{-6eEf4gYVb=4O9E^#HZ=kOp@^J{3{m< znj7BA_?C(BH{d)AqbEq>Y`q1;FvH3d94)p$^4R*sc!6+BoGsntzbGw~27Q;5CJS!X z?zZ)6Lj7@;J7&s_y8P{Ctvo*7YXAH@H-ZG{@^X7 z@bVL_psC&z7mfJ~!4P5|Gtq7t8o`<-n;)14mlM|LLvyBYF&yN-R0yo25^58X`|N)K zScEtVOi%Zy>+8cY>h(>D-sj7hW(o1-rkPeQh87m)R@C7WXNT2&A}p4-_HQYsB+~oM zQok4~OJi-z{w;Vb&v#YXC;EVu5J-7=;}Z_@5-Qx>pgZ_w%RmPrfovqJ3@LFtT`CCV z)D)pdT3#G=j#Ro~K#fX7?*x8SYOs)>4Slg5LpxFC6Te1s)<Nk47w5j$u-|5YhUXD#9TXz&AHNX-N_K_>gd<5T@ zlr33?*1sw+jo%gNYC6d^>*p1|<@GFCKx|JYb{&wQC`a>cJ~dDSTLO%jNCn}f`6&_>Z+ug}13%6Wb> z1DM;j@r`VU*bhDk#R~Fo<>ci)=B5ewgb6_Ct1|)~*ukJs_9CteJ4J^&7Ygq2)UbXp z{4-S@+K=d~$=CLVPE}}6ElYX0~KJpu63=<9zZ2CNY_0UXekyo zKV$7F6Gz(Eym#q{Fbm>NL8Gc}Au69II?< zyHTqJ@@EpPAEPe^FPdWf#yB2MI^^*lF&W7O3rimnfDmnT{8WvzMG=k~MWZb;A9ez*GeVp<@ zrfx0g&4CZ&y+Znfa_QoPuNBi81YdeBBh33H!W$`6)aSG`h(X_RasR^~>)yCu@e4A} zi%s?Hg!$$<qn#n|R5!-I=A^XiG$?OK$}-4lzUW zy%lUAo!KdudTWRb)%@unW3g4v(qoD)-@!4FHE?23P$czkbQ|GHlTzc1^sS1D)Z249 z@K1cZxumyqTJ41A=&1Az*ugfMJT+ycdZlU_`ZS@mLociPabw&#i((Gnc+8{QYljK+ zji_r@H3oqT3G0%iC^PiJkTeze1oh-?p{4u0{DeiEuW)mW!U|Af98ZfIftCHw^m{d zkEPXpvc^5;n^f%(p6`;@R`iNQd(2ypy_iUdGg~yQ!ZaZAiz&=D{(0 zY8TofUKRc{Y4lj&z;qCg%X~#I8?qwWVer)8K#``?3-_>%r6&)mb>iMrnsT=L4 zzskBDIajsC(8i4&V=<5VYzJLw2tvupG>obu4&I>hL7}|#O!(sUrT0OxrAf`s|isgb<4$LrdW6P@f!1qtcZj7#@gPf<7T zxw29Bl|s%VFa1+o{ym$E*d~4t-W{M9E=*15XPG6Ihfnwo$Z)>0^Vb~i&%Iu1M z?^@FHUQwCHyQca*V+yOPJ|EXK)@74R%w<2SD73LAn1}Mt4c6^F|K;6jFS#(+!Ksi0 z99pkZ5O++|Y#QO=vAFs9I$rBD8+2IParpXNNd4o<(=FX$Y&BJ}1d42&+nm?}DYOI8 z!z-i>=HVsp+2juqZden=%OK#C&L-z(Bba(jHyXXhb4r{369N(BHpRhbFjJ@Xoq$1wa>4RTPA@N*z;P#hD7A5&|01OPd$1AqL-fX& zPVOQK?uot$)7JYvwz8^MO<-opM8j9mJM_)P^7VP$YJ@kT+*xxosv4_&|K{7;^E!6p z^r(*hHKc?M*k&Sn&%w5d0Kk zuEnlC>5#ja)o{@dE{S@x#jHTS=?u%xD)~(k8MLHLT$53JawcjXZD9CadD)pcRj;t6 zsF_6Bc2*b{%h79UZB35vq0L{jv4XU?!1Wi0&ROaylvW0>!f7 z?6WworZB(k;iPuNr%_yMNwqWnoJ~u+rjv5$BP#UP|44TTi=ejIgU<8~Pw?-op%FV- zC)(W+U7z3yUu-@>b$1w;aVX zdH00q!Jgf*`<@}m436WnUG&3jdqsUZT}c2iGIvswX$=nTd;ypbo-1hJHnrkOa!~dep1ngs-^|lG(bFn#KH7_Yt^-Z(x#dy_1=S`;{!6&kWVfK|&l;dB9 zX60II6VHhwA2FaVn_mqr@h5F0q{CKM8$97$Lwp}-`;Auxg@*_Y@N8Q4g{cP$Rh3Wi_pGu#5HhWIFoU@js z&*iP`3XcaJwtX80#^WmCmP&>5(=k)qoiT>-?Qd%%n{3w&971p~nT1lP?3q#I(mtuZ z^kWj@dY%I~hRg!scb13~pG-QB6aILax{C}P15`7 zDtkS=>6}fR*xW?Y#O{@P`_f(M7qgRB5}b7~leo~es*f`P(J1Lpm_amtpLIx)PMG8C zn$hH7S0tC4;(S}3R=ga`)B-g-l1I*unTR*VtV1VIIE7MxtBLfZu+4y4m~nW@<8%aG zXIR$?_QZ?%L>)))ZEh*8nwR^0*h%X{j|jkK%q%jH28i{7G^!pBCwpsN9e=cD_7!Wu zis%53z;Uoin_kA)yqej_a$tE;ELZj(6b~ztRnG2#zTjR9rmlD@4QOjvTF4+9dj^t_ zjce?VlMMdf=Li3bGdmz=BL1#SMn|_wZg)ccx%ne5^A-72Ldn)*KTNSGS2fRtHu$}{ zSe+a5HM?j$DY4zopAD^ z0z}P)^#*OXcQe?V*PC(#y@Q9u`FgX!k^gP!_@}lqb53xmVC;>Vz?8$Nr?Yd|0bGYk zHxBi`A2A#4NKCOjwynFWZMcjdw&1)*Bz9;0CBqRxc)b;@i}Q6PaOBx~kb`u#z~P)H zWWRPPE-P}7;?=@)uY2_azwss9nH`-tn4cdSKH1CsE}K>Lj7+7Xap_tdUPG+Br+~Ij z*8##t(dM&!rn#LTrW20yu8jaSvb?AI*@#*qHwcyFK^McjxJK~cc{&#&!`re(Z5-5OxWyDZ5PRLbNxPEjSS12vNOLianrT2~87Bn*zwucK9J3eaVhbf_56#1Dt&XLPz|4Zml$(ZV-e{NJbxr zOZ}IwcE=4vj)R#rTFI(_duP8}!m|ZE&G;Nb5Wg;t4}ZUTR#ruA!%3BS5U647SKnkuc(8a=`qi~3e}KW-6J%!eRCM6>%m8+Yw*_L;gFH@AC8XFp z0c-1qfuC=JJ5y$yOf1iAB7N!=xBt;q?tr!;)+hA;WMcl8<9+2t6Goes&+%-j+I=9d zo3w6nVXtWPAX#dyX1=u;;_o$DA}VIdo5Q7I9r6nA5no*u#(RZsR4Rbr7qD2nvrPsn0UajI-Xt-Z}OXikD>Rw9x37v`jNJ~8L6v>yxW*E_yksV2#zxaxKRmm#~txHyxJYle+Wf0%Fj{rBX zLhyx3(M0xhW0f3Va$sA%S|L-f$Js#G)%hl&EK2g0&tICbycUw4A-!%n;B zl&aQZ>MjjVhg5Wa9!Y9L@&+5p7gB5;x-TBec+IDXSJE`2m?2gNJ|#qjutEryx^Mv-LEG+!1O>Av>XFk)wMLqwMo#z5 zWhVIGJiN~xkkR$dA1xiXE&|{7t-Z!lu+HvE{J~j1wKd1|8TdY+j(7{4T0^2mg!9Qof#B1L!571P z>zctkye-!U7uR1nqnzR7I< z6@||c+RBeUUGO@+`Bn_a4EA)R`R{wk2#6@_S_T51`9ZpRs zl1^V655XT3Nt+dXnRG$FszDTPPWbziZCXnfmd%%&wz)Ax?W>8AH=ZwQ=Tr<_Ednde z1^2Y|O&l^;!#visw^r+#i|0HEZNBVAeVT{XRMeF_1RR{zU2*B92FCT@kQgLD78~vQ z&8j&XNtei-3_tl<@^hZ1Xk88GYVG;Nf)xv{n1=?JtYw~8umg9*11e3CHs)&hfT ziSy>VcUECLfzB_tU}WShV*E=6O7G*1N&8fA%_n=Znm4*S@;=(K+k7ZXovHENL0rg~ ztop{f-}AnDQET}l`8)Ddk6DYNORmuB3$-}OQ8B>ARmdzzgBXTj{ewBrnb~e!&$>H( z0lx4t*8}YL8{A2xH+y{7Bn<{x^VEZC%E3dPXY=l?BNN$ya5N-hFmd>6U-vEG9#jK8 zz}bn_t{r8uW-aT|cEY@)QRQGo`#wwbUEB;wMEwmK$rpE_IIKj|i0 z%1-1lY!Kifq;dI@BYtDJAAd_|GTXO?l!7ut-Hg6>5%!yJy=kXh?oAl+R_DsY4QfSp zBvD;CcY#rx)_iwO5Vi2AXR+s@HAIbq?9NWFlOXcGyim~^LL?NX>nI5l;qix?C!*|g zD?Jllkh{W#1ygZ-g3lEUoR|LDph$9|Y?O<^oL`N*ixa{UNd1IZ7+LhiaivKRkMRJ|x5=oKXc+y%DBS(7Imvz9 zkrS6Z<^CMO_O4$^*%Ds=fuD`uS+YB8hgV}>2%+2q+L){F!3WWx3cqA0g02XU8N)P~ z7gnCvEA3s1{t<7^0)&P9t+|)Hez#gXcA=*uom|C*9jz60#f|IX3Pl=I3s)E0b>*Gl zx^l_8>-Uq|mxzyF;+G578-9b0E(!fKF)#8`lK^Ui3`~Is(JvGp^ydNG9#m<1dhzpe$S%k;4HPj1`V@90Kso?OYBaz0kE@sh@ucJO{EnJ4sQRb;z2y>p@ zp=3V}SLf=r=vv*^h2+bHMUJglh+AC}plJ|3&#i9$;kwdpJ!JAO^8Vgyj_DZ{<>%X2 z7v}3s(w$|}?4>)Aga9eEV>st3A-&|CgrQAC-kl(s$BjaK z1|-IH>gr`YAtdH<{;4@L`TVjng!sV;gKF11_=_4qh(0yiiR+0isb0N_R*E{D<#sM< z$o)gVqrmc0c7TsTwp97DZMUw0*>deR>*G$0YY_gSUs4Pl&IT?8qlI)VFK^rH{_~-c zi>UfRB&ghfy4ChqOyGI;XGyGWWKARCjD=qmvia495~`!M$-`NkksLyDbx}d<$iks4 zt*$^n?ffo!?Q^N}x?<_o6$c>?zc^PdE~}=(=EDdA;cN}E+nL~JITxredY^Lr2*+68 zY`jqkhpzu}18x=;<>z@co!tcvIpO-n3#`-HA>YtVyBx%K*t=js}p}08lY?Nfky^o@WG%(&8Kf$uR?$hqXi6 zj1Frf+mlnRW&sTt%gF*QoH7y~?{UAq&(-q&G*g2|jyy|L{$|w_d;Tz%k(m6sCMK8W z(nMEyg2ip^+0hEp4V;4KJVGBYqZ%3Gr_7YLL6_%&e%=sE<6^b8$m7NArS=@3u^M4F0#(xs{(y|)OcAYBkpDM4vcBfSTe5|Ab}K!5-Vy@b$0 zLP+k*+WVYw*52EAzcKC^_n+?{MvU(}Gv_m({(GJ|JAWX|pZAO-1-$O-;?vufQnT-#jGDuk zH7ebHdZX1X%C(D_3I%#Lf=py4CCMNvaLWpnt@`3(1Zt1A6!16&c!g0jHVIL z>LOmX4S8~Vq}l36IjkF{SYe0{we7V`iYgOUypv*Pz5Phfh}GeRslf!;i%u#B48aWU*m%bfQj!QIj0fO&o5B5w?tRgco9Xtsy}I*JVuHF2*!V6nO@|vfZ#c&Wr=AyYv$;X?_AI+~??=fl zFMeFCiNZTI`0wb*4~$?gc()&0{^W=?Q{0~q@sPjDu$92w6WGL@kJ~?2&~wuN@yAI) zA&+9g==qNWMv5xDPaahgTnDR{Gb&p~WH8>wpO?e#s>43W%s(`-WUP|WlqDRixGUbV zQom`cd(ElaaL?%6Y{lB#AG&6^2hl1o(HVjYXWI&XY@K+mcl}Hg(`~w+r)j> zlSqm2l|Jo!GYuSbj;85an!1!KQL$OE>!)w&Flsz`lJ8Fz zb7Iz@Kz*9W;a}((2cykb?)f=$-&lU?E|vroDz{<0x2+QJ&Ap!s`iJj@+fSUZ?%{2_ zfPHzRyHXbKw{yK@tJmqW+=J0s%jQQSKMsnI{wU=OEPT`W`Kz_H-AKd$p|x+Ea}|HE zy}AlW4QkJV2}bBSUHfF2DO2FQ9ctGlho3UhB>!{6n(v736{fiiU&@Q5e0|&bV#-Ya zn(2ssUC_{p3BMKo*ul_q&Ibpe#^ieLG{vOBf%9j}saws@D` zdl|-Sh2L_Np3#^2neXpy#t!8llB8HnfJN9cAgx)2@kti_2xO(AB_r%vBTOEKFd|-o1=z5-zCxmp{Iyj zoM%fmk3dSVr*>aBzX_5TKMs@M8Pk5}GZ~OlhpF#fVj0Zk|2A=H#!2EqnwYF?@*^4) z4=6DLi@%YvDx?ed?OK+67ziH(&cSq7dOfupxKA!Y`T8(qlNt?!v7VHH?~0rMz(@^_ z!&vfR3iLy~CGX5*6hpKQq*xd>maI%{*ZP2O{eM*XIIfBt4!F`qAC+ zGse!p4r%R21no|{>~GqSgjQ6$_V_|5JOeCzCnTyL$&(ePQ+6U)C``$0$$pQ4e&_Z% zea^YKgE~8X?x3y9=J=Hp+@Bj=e==-NPQo|R^oAdNnU3I)o;TxlZU7|G@YdfWsrff$+&D`U-vw3edvJPs(j&R+TfiJ)B$I>UMK8Bn*LylgrV)TLLuGx7waHM|#bJZOk`di_1S)?Le4 z4C?^@-rd(YsB2-HUX1s456o8O#H$MyXa5B>{h!g@4eL{YF^vmWDq6>>sUP(MzZbL%QP%r zGu_Fk?@Jpv-ZffY8|Ma2q^nf*g$31W&DXEOj9=XB{|@F6C#eLW$yZmU+cz%R%122x z{*ZnJZ6+)FV5ef22J;1kgxZ6XTCD1i2T1E&zrsbK+3;CxPbK;Tm&f@5mk7DFFv(44 z2Cm~C3tYK!s^t|^3-FTrd^cyjrxijnDq{AN-zABKEqNgQwe03O$oF%C#4Kb0V@5*{ zWN$!WKRAX6+}((#wIpGKS&9$uck%&bHe0pkf&&qMX+V?Ewh8woKeqq}(NPY7a{3Qn zTx};*ujscNJ(v;@mIQ?+O}z2lfJSU+@8|FN`Eup;?eV(AZu6}~qnwsPsfw8TOEvf# z<`?V{TrK{D&{TJ9PGH23pQJc+RTFe3G?nj4=@T7S^-+$zL}XmtA0C`89twIAj>m=B z?P=HSVh_fzTs?7(GyP(?rNQwdJUkq{rJU&@1pD+l{DA$H)hfiPz304L1=X_Kul!?k z7yf)j!N$j&fRN$XJ1=;jMeqLH_Z*$cK0(^DiV4lAUkcbhZZ$8}@!T-xSp+BYPWvtw zyJ~d{IZ0~unk`2=2Qxlq=8b^X9dmQ%81#|$fPowsm#*@0gWTfv4}0Rf5` z6>z_p)*r`BRHH7j*m{PYK@YoMFhAOx#t%Kd0n~tO^p-ksD|?*)E`+$J+Ud?q|M|7_ zdZDQM&M^AQS~Ze5I;5mR{E1?IglOoIZ{^_0$&!TU8mK+{Q0u{M%cIC#qk$5V-AN7U z$cdm*Vqo2Iy2GivrLe|?6CJRc0#8sB^zIl(h^7&vSVT8g8~V89{)5zZQWa`+G!nRF9r<|nT#E3!tM)$>#t?z3TlYw%&daK# z4fV#ZGpCm~7LHllzbIOHdkI=R-t4lw^VBws*euv?^|^S7X`zm?WXMwv8`%j2+>EN ztdB>I>`U?-2dqT4Z~MvY_NE%WCiH{HR;tcm;fnJ3yXHzuUSnCLCcqjv)W^5J$9+5; z_ju0XxO^Lsn6)KFKID{I`vzCc1*fq<<1MvggpV9X zoJmx5*T2ri+>7R7ht>9}TXa+b`hT5^^`$eXu7_OGG;06z*#F6k>>1BbY_h8y8~8uj zf#;_FP~CvOW-iUTG)q&*f0pKr4tcLG_Kc0q z`n1c3S(+4`B~|lp_e7Z5S_N|;bIcNT)-zGJZK6eO7;@+zKYwyEmpNw_yq*y#iDwp= zcm%ZeI84=~PB<#``S)&t97CW8>`vW+vXhua{dV(=`tv`I&>TB)ieK$lAKu=#W(p0` zRx#52)f4~q@&FBu|IyIjjdv~Vxf<;B9V6}E4R>s_{dZ%XdVK6iXplfwVdC$eJb{V* z%{ou<|2#oML(o(+68znhr#iTfQl%L{k*w3lj%~(1FT72ix~KU06XcFlqyHgw)dWtQ zz;GQgO8)JyVnRvHlmpkoe>|whc>3{Ro?g&qr?5EYiSqHkFYFTE5sIoWVf$suUpp8Y z6uN`3QU%w1LkhBB5WuzL$K`I*{`MCOXj3FCOb5(lZ}U@tKIgZUZ$t!s$xQ!bmDtN) zaP3?XYA~sn!+O8rgOYn{{3wxzNp;p=^XOkRvSkElWcJ|TH#7bzLVIkpq#UVo?eEn1 zx6!Xv1B_bs5;&ucI&=Ns3I2*e|CfUQS@<8%&}_~3M2nWzUgQnh`1YXk@!bml@!d5f zT&|hWBaff(v@I&B7V&SO-4qlyx*(wBpu17 zvxa%P+BFCHss~R_p1<}v=NWrP>RUs5>Jsu;g+|(;JMt*AJU`DZ4V2ka5-jp8?V-)8~M_iuIK|rms&KmCmDdoic<bKJCfnWBG2BbZpqZ_x;gwo7 zn58+x@6^QYX<(Gsz=;8xPxFFw2ssYzlIFEn#w(u~CTgAONPTfv@VDeCI0a(saIGG5 z5SK`>`eJ=uF+sAee20cGvL!d8P5)0F_n$|(b0+jYKLtV|2`d7Jdd<=h2wMwqQ>PN6 zM0^J$M(66p{TsiMrhh2u5dr=L$xo$0u&t^TLf_(8h2uAz z(bz-VSQ#5KiBL9T>ZI|s;UiyYkUFgebxsR47Wz3FN-BcF&dZ^KJV|qs;McT_=P+If zbI+c=&D3V@{LFx-+iOUzDFyQ-RnvF4xA>HIqJO0rNqyFg1`> zb|#XB5D<1*pX#hbHP1edQ6v8lg~ljh8>J_g4|b7~nsMBm1%k^?;Co#|e1WTFI4Jl) z)b}<&B>OYvl^TrfQ2K8M@NbN)W=sE8F#)`kn`+aMaN$j%ul=K-TDz2&ww%rh2>=o&b;462`)hN&Yd_Ul zVoPeK0_xB^DqjFqk<&u18Zi~Uhh#c%Zp5{hc`XgRGB$KOIIsPZA}&Ahl_xal=n;$G zkmU@++p7r%!&QK5)2782d^Xi8#D5hybwJSh$wA?n4n&{MlmD!*|C=(G%G53&B{Uzb zxw-?#C+3^U`)^_ZmQsIhW)|d4Lzpdj`{#d4)PMXCs(I>?JwIi+2-*%T@4VICclH5k z%nEWsm5hro^TW+ zznhpg?QNcpX23~w@0Nm#A=-g+JKJ6hXVy;OhC+uL77Q1FoJ5QKVpjO|wPy>B%D&^GT^|#{&lW_eX)g#Q4RLOg2KY&fpz<+7?#TaC8 zNIU4RRwG^Nt}>0C>X5tHSM?h`@zXlA{Uioq;XApvm>oaxOt(orBwD;W5>P`|&xxC! z1+Fmnxo={U-+jQyV0I~DssS~947ns!*^*nnPxquenZ)7_YWxx9NB|@@0j`eP_H2>!Q?Fn z4p2j_X0AWIVxuI;H@guCE_q++{YLN}2ZY|E+ti~R^G4m} zx2lQG{vG9R0-@Y&mfVyd`a!+d6Ze=X28*6X7|eg1)v1ln-2fIs$n}zu|hRyP(&!Hh0fCzsAcTr;7^p)1m9{Q6NbhX6gA~azE5)V08b?nd6 z;|dL`z}{u~j~e&C**4TcB!L05(EXm(ib6!7JB1<6BNZ6K3N#+Dw0^5)@J~9x!3S!v zvy(Qjf-avOzD|`j0KWrBBSYE_S-cit?OD=RUp(M{w*q1FaB&#n{wd0kp+PP(gfN#1 zzYxKlXjw{TwA#C&9bm|Bbw)G!_bX2+p#X;q59%PA0aruHq5GZR#y7g`!otEawMr+1 z1c1W2wgaNZY35RAAK-APe;y{zEgPPoAvj2%z4rFssQI_$%n2qi%o$408GZ{|ZXWVk zey|H%tL|hc*EQeZvwyE*#_+;l-(N*J{HKOwU^f|C@0-f$UM$B}&z{RT+XOunkp6HeQLD92RsN z(RXZXYp`gncfrD_E5QeXfcr0AJ;}=`yNNt?S?neM9QYcR!XARR1U0 zbjO%f5zy};_s(lCXR$#8H5aEub%}q3da#}9SlH=yMG>EIZ%s@2Yh8p*d<>ijo)%xa z0-Ju3p~?DAehbENm7qks<2#k*sjKAj!S@II=TWL_&PoS%!@o&-V?9)AZSqN-z@+Rr zD34n~`$RhHlloZUd1ao6DX`Z!COFtOh`apY0xf*?H&SGNIHF2L+o<>_;^6YcoBVj9 z#r9N=qYFxNkRg4il@guiIFZt@i$HcLQ5I@geZiMs3wc^wY+yuaR_oFR&yQeW>`L;mX!e|PBA6o90abd%x! z0q$rG+FZmZs4#4NIU0}Z}elSag3R-o&7&^t+1eg7|q+WNXmH~S*OpkrtFn*~c` zTC=nmkTV(aFOU3<7@_T_PIWlmr`?Q|YCLb8uLgS*QYu7^^{jUQ z;gTlP5Qddbt<*QD!6ew=RA=;90`LNK4d^jlt{n`RE~r6eB-IJc0{p@$8XANP&nvji zfJwVO9!!1wlYoS28u6nzT8bWdGhn#p7oJkpl*Z8KY9O>^f>qPeveE`d%*s9Y7^*Xx zzj(L>Zx6Si_on7{sup4LrUQwuEI86T4>~M|4+ZUgRN{G9!F!ZTl~!zc^0UW^O7$>&b)|Yn@rq= zjF~aJDseg{9>gkEG+jiHOyPoVv?r05Jt;B!pfsHupI)op|q%AWe2sv3Oz9w7PP zgiJok07lTE>{C^_u&y{NH=jlDHgeifntkoAXwpFd(Gw3U|Q>VnKRgO6LTGoTdTK*u08zZZ zz{9UxRe8z}{J=RysbZ5Ul=LI^?cXHjzkQg0e#i;}81+5uvL3$`PYrfh<`qnle?3$A zQYoLChjQi+O7uNXh;J%=fAGfM_O3LobekjmCv&oUU|+u!rQn^mg^O$#OER}iT}>$e#)E;idAq-FJqGjG46n={V{ z93L@FmRj!s-i-8aE(<2Gb?-7O`b)Pn?mf`L zdu)?6l?5Xptq$VqaKtD1Fg5ykoSAY&HDn$`c8YB$t;f{5{8*K*HxXmRBsrnXFQ+gir;wRZV9AESR;ix^%gh$jOSVRa@JP=UqRVs) z;#%OkuPsyc;vDt6rc^!ad!f#YFtJrDW3fPqC%$Jo_$HsLh4kb)X+k=KZGbB(;xknO z+g%#gQpm1jP0rNUmK6jzGw)%Sy!pXi<&`Aw_qlV1HJ`!!5T@Q#rW9u$nQy!0YOpJd zmgk_EV8zWt$1IWr7$!EY3 z^CzvUI5)?9n;hx1fGm}ZQFHXJioU){@JMoYwwajre7Pg&ndm1AjjQ%B3xs^J)f=4= zmfIfcerMAeAAs-50eWP~%d*O+5Gt$_% z22(pzpF@L?WA3^UKkPb*Gr0Es7MqH&M<#5B*o-qqJid{k4Zh;?TmDLeuJW?Gwbr+X zEDKc&^V)ck%u!371AE{gO5vY-l4L`O8pvz~I8hW?OPEhXA3~ugc@(dSh>BL^2T_kG z{yF7#W=v5&>P&`6gjRZhVRv&-b9dkvd;1!EjYFMJ|9n3Hu$Jdy_AO+kT$D;&*DY{^ zNqj7;Z%BUpz3Cj$669gA0-FGy&%nuGUqX_^%qYVj>{kRt(8bXKt=~~r>@~tVefgl8 zCRxej=Q3<1V^)+_-*wkO3F#s88=nJ$?oW`P_K^3URG`1Ov^A@OyoU>qZ_4=UmI95y zQnu}_eB_M?`Sn`#fXAAzdZ_FU}{n3evXOOFwFe;Vd`RiJUpqpK(1McQ#$50K^4 z@0IdkBDUMLN71+o3#~Zr}ptWL-kH)NH7jWyKA|CzW=MSzbvrRk#-snE5cQ`ccAaa3+w+uU#%I`V_psi)+FLOR6}50(<4Cnx+Y*PwK^&N5~Uw^(eJ}e8h=)^52Pt`obSe+TGKff0PxFCi- z6ht$b4an$ODDRHix^MSu!4I>V1Nryn&xA+uIBjO${yK&K~9 zd3kk%!EWn3+oIx@cA@u8I+wMb8#9FcCJMg?IA5=T z=k6Q6b<2inktYqZC^9+-{?o;@C_c&_90#Ov;J&b}j8P!B`h!KCdc^bC=PBp{2UU3m1J_&40){Id z$oQ7`Lk5|s`p=KwA5Nzkh;&?^L*vvyxf?SPeC_tB7U!Uio3{_iT(*!In8Fu#j;g)B zvwJuUyUI2qVs=j(zM#08ReBU{f~u9Oam`+$JttleRRS(kUvKBCnUr^cTve(;jdJ-; zdYepm4eH@~TtlQ``;%DN-L)aA(4871E3gZS+r!|-^MO+^XQ6Eq*)=u;9s=^*$Yjro zyXAvDk5x~^SMzM&ap!!$?MN-peZ4Y3`NVK2c-VOX*G+}&weYnIr8y6rt?_1|sxD2R z11tz-m4i8;8=E3)UNxTQs(NPS)~!ECZ+<`;<-o0eo?tRq^|m27V-*= znc-hA8>C`f0=%1x(0e>7O~(AbpI*zh$8xc=3h72B=__82Q{Lk2{PuV-3B4^w0Kfxyz)DGt;)j{c|0%lv=CWQp3W#8d{y1g*0A1$`De}H% z11<51w~$3ijhNf^%=IeVIUiam7%jlo-e(LSQ?0d9{m>u1$?n(-_|je>F<OZ-C9)FQqiXy8U@OvPG*ZIf`EqhZ*sAzhq z!Cv|qX1Tnt$DLeH+3v^39<5f&l5#anthWkS%H@!R__@%yuC|al=S`*`y{?1F>t7z281*QuSfj zV=|1EaTYkgJ`%We;Y!GMHeZ_$@>z}P_)H$v?S2~%&?$&`L+3-?SD^b~HcGTK62Q|% zczR8Yjb*ol2ZE@I-FYB;;ToCgL_c)FtxpBt$|N^&ijFVH?If3MaoibZs#@shc_3&g z`IxF=o5ZLK0jV5V5+p3@bDEn>V4V$LnW!1FVxJ?Sq^DRKrpE;|Kvlk_IZ$!}V5>IR zWY>a>)jDcAv{(%wFAaD8iMKbVwN#03@IO*%?(tBg(Fee~!QOkno0l~R1qxE+N}I_6 zzz=0k<_}YQ%~%`wq1(<97w>>W0UTQ407L));qkz^mA(ZAYDDeP2g>1A>vhA^D!!5k zTj)dFAzqa{8aS8csr6Ke3iE_s1SqP(fB8n%TbhC^Aw!?ei*Z}5O(tc-| zI&13!uiFot{?Jc67P3=h)piJAhaYE(GSpYd-8tfCLv7aL=D@7KD~+O)YrX=-JY0v+ z`}ShLfQng2)6R$}_P7%dm6;!*(Ewb?=)8LgYUmS=)4{^ z7x`yBpfpMvC@R6642Llw(1c(DKkroh#>E1-c=E8aOSC2B=^3g757W^KO8FBq! z(9=5>Oig~9n9ee{neSmwnH4Yz55NaIjB#U6OhbVJE5@<|O{5k|oT!<ZCoEqYN3jSoH+j{uyMlVCV^n(gA~? zG}dmkO-OMaGZy;NMl@@!2?!DGJs7VO6T^}5wW&jGffpCC<^63`FCRNe~SFSZPQ zWW{r?WuH0T8(C~N@G8exMV_*>gn=9&VMabgGC`8D9$6 zU(?$FCM4?9WRLiV=~1a37g$%_CMU{X7={zKPjwQEQ$@Ql~9*h;8EB&SaG+&0&ahk zQi>H#U{Em##&r8Vof?{m9eGlrudC!^>NWO7XMMELSsB0XSw@-MXEzTR-KA?Y+9%1& zng@o5p?yvSWqy4jw;n4E*#qNn!h8!+>7y&Hdt+0oUDOc`z?I@UI@#_WJ=D-v0l}`3 zCTl-q{l_d&dY3l(h56b_Yt~skDKe72h#88s@*%U-9T<(t+izZ`#l~VQoBiu^+77UNma-Z~xaKDDx%L*D>2NHNkWv`OQt>5P3f6tmh4S?573Bn*FX z?{e~O_XS+4_1*|?C9(E3If1>^mBaQ#D&nMAlr-0@0|rPRYz4 ztYl0G_VM7&y^x>HsA5cND*!_~5-9}QJ}pWEaY3_1KW*nVoKE#EvZjBVMzTpxMFdOI zTyJlZTjMc|`;dsVN{)?%j@3?=`x-!&nozVCeU=5>w$)K_5vYJaeLG9&CeH7>n(f_w zF`>;x#U7OaRt2)ONk<`C2RRMYcpROcE~RVjLDgRP%jYLQY`RpK2$dj z9u8~MVTX^%IcF3>gEboVke8)B0M$EWV!bCVFiT8z`C%dZ6nfWv*+ic@2=;jLFxsG~ zUfBgq7Y71#i-HXOosvYsifoHu}PWomo5Flc3uyCs&O@ij&Lb6HNNEL7tju%e1Q4LvtM8J`A?deDu^-GKVDzewclUb&!pq{#gyX(T#)_(8U4UjK%0sRC`viWO8KX*E z%sz}I59{>|GNB^Xb?#~A$Z_FdT#cavW;c53R)gg0D4(?feMX1g$d@{@v6fAWnvW<8 zh{Nxys1qOq^=oXKt-(una~dsrbpQR>}!1|S1AsVF4y zWX!R5s4E+P9ntnxlrIvewR1~9r+sOnHlzzCbv2+>T~a?LL&^9ivYJqIzSuA5EXZ@_ zot4i*AJP<1X(XQR^kmXOj)X;LZE>k7N5%Y~XjP+13!70vN`ax))#&;*Ulxp7rx%ZV`9?&h8LeQ=pM2U zz)3A#?PxkPI};}ja#L0_)4Dm*oAmI@gkl`l41yHjCY z7B*S9H;9(JX{3SlYTX~D4WC^$F`*>xR5EK&xsY{Z0doN{*# zT8p_XMI^s2v+K^{Am1$faaW2`Rr1L&9(~ax_)Te0_69BccgJyelK{icg-u)S#PDRmFz%_O<&1sGwA1OPqV~Oy^?A z4T>LnUdxF`&*4Wir2DNwTxn2H5c!=X_d=D+8)S)S0yM6GBB#v!S!UNTH3mj`xwpH{ z<7uL`GqE?9kTjbfVP)nc)+BR@H{jX*)lAYUp`C(Eg$oZkvmDa#w|DdN;~glS=ZAHa zgUi%FPK#9PFS8_o{nO079~#s%fFca3%qG&KZLNnR_r?=Cr8dSdyCd&*(cn5@>PiiE zzCNmk8c+ePRu<+-iF3*zP;Z_4ZI4PVW#YU*nNGv%RrLF?{XdV8pD49|57dGKXPCT9 z`a7a&sTnd>ju{?IN+>TE%Hj5#Uz88XkJ=mTX_;O2`fAR}(&X>7YDtN?I+>+_&*W)} z&^2GfCiETOEYJ+DAIZlJgh<+dy=bhEV34`y)LAhHK5pry6Eg`0?(-3&;~Rr@vvuqm zjer^;mw$TRG54Jcy<6sXd-G@vZYexls4sHRSK?f=hznn(`boVVBq-cw`|eSjiU-_D z?dhyF`833BmSc&0{^?-D?n0uIIe6jiwo0$!R?g$*ClL>>;1AyUqxZJswc7rouc6HMAjiJC3S&{FuU*;OBkhE?hdXx;B+@wW=_7Jm zFFcdb!Aa<&i4Vs+bq}hI+=I_JFdKaDt`KTGC2()aj%dgeZxP_Z4RW}>T`%^&2W+}a zem|%YVh^c#t21H}WPBRlt(4bsZs@Fz_(KR#?FVT;fid8dQ+QaG)qaVcxSrI=5^O`> z|5@?TnBP_**`jzd z621lIzp_AnOc{P8Q_gz@CVX6UImSVY3L;Y>%e9fJsIykwS1uL^5O5t% zo$*>8kBHxM)|B$#KF}`STkRWhohJ$VNt~OV@q+iBrSGbh6V-jR$@X9q_vO|HUjDmS z!ONzt`9A#3E1xWEBifW&fr@J`F6S@TiAMAVYd~+mM3cqMk z%Q%*an4`@c%c|N_197EGay=VUe0pQuJ~ZgjVg?gf06_jF0p!046sXL~0FRLuqIVZ= z%ktZ9o9WNJXjXbLrMq6nzt6Ay7#Y-gmB?z?wTPe?j*b`=TzLH=RSZC=FYaV+edtpx z(>I@5-&}RMi{cB~xUNic>ls>3q}%Kq*o;ohovVPDFPg=x!RFy?{vd6)2@ri;DxdIJ z!_ZDf0DQkO5;M7EaZjf*f64&Ts1-uQ@wp-Osp6@Hu{-Rrs)y?Cq~jG zi2I!w974MD=Y>+0`IVb&myg%n3-B^4rcgBQW8R5S106b ze_fXIu0lm0@0c37`%katTH{4a8S&jwpF5l3uaXk^uEpp|4qM$BWAJ%Ow@C-}baR&R zE!x{?VHq@O+Wa9}S7i7x2s_o3Z2$o+eT5n3>(_ZwOVXI;9r`&?psxUKkd`6GD0h+T z4*T(s1!?TZ6Q5P=7mW3G>h4b5oXWAQj+><+j3Xo&Eev;jv>k3Ex;c9P7$Q_TcVZO{ zpwrfm%(iG2*vt+0@2{!Bf;IO2wua=K*v)1_$ca0F>kW?PM(#$XLF%c+P#!e|b7-oH z_s8Ix>ahd6s&Kn>?hHidm|_Z_aGe99=}n_f@BmLutE4MTf){$gOK9ZW^hQ9Q^_5E{ ziJRbi%C!v9=)89dIYJ3gmHCQXW0HkDxrnz}xt;~K?rH4UQ7!$@5=1D|Ze!YSpOJ%| z%T8ny(8pHkHq3{-53qgQxJf4TOR;t!EIIits);WjH_wH3&Tp!QJ#5ac%5_C|B4+=~sqF4xS%5`pFdqyO1u)U-9dX?#GAe3|*5H*#xU| z&JJ3)WwR_nkHt^^nI@8eHQs4>j6!3)wtDM5ysnD+Y2SC`LRMcKH~Q0X9_bz1LQL*U z+$lQWfjR`NNjNxMk*kCiIZ>;(dzp*GOz2}Atf(kY)o2~#S5T5D-xp-SB#o^`2o~`u z11C@ZkY6c3YMZ*&(7wyF(2fvEvL83OkPuJX78k#oxHIP1g*#VwSDsuE8q_SBB)D&| zYFZRmY!+|#nyB?DKv$1nRgoUkY@x*u2Fe584SD)C;)cFFuKA<}BAnK(cNU=Ctm3`c z1c2v#?LfhTz)r>oJxVe(NRYi3IJ<@DZ86J1CF>!57!oO#^dBWCJxxJay1a66c)e+V z!Ntb`yLrS>c1W`AB*Eob#Ra=F4`8Zd9#E8i;Qh9t!2Fv=a2}iK^(yuLSxx^(W~my3 zZ59@Wg=eKPTUe8-oWdtGgvLJ_=gC$3*SN$|g%Yf;$h2i zq@+OKcUt1yHsxgOJhmG=fTAFFvRjVGeh8u4iq=@!32@s`D!a3M0{Kf^jP7VDf`XtiytF25v4`DH8th?3RHl*pI!hH#dzD& zZ4M8*zSd7}5lH;$6(VuXf}@77#`dIM)2uQ+6FWY$R{re{hX-_(XS3?)!1zZeAJtDN zdscMyAaOoyVyQEpZuizy507_~Yv4mq<65K9kr4ohoocX@HdOIvCn$9`bV$b^?uSYvLw7OF zxhF6cHMhl|Q91guF++*F5j;K*wy`Kxvz+Nbpm(rrcif4cn~G%e-?2Ot zP4G1ze!kAtsa+&nM zLTQpt5NqBnQf76|2T>}(#f=b7=(9|Mt${j6m#ZG>8nhetoY|Gh7{-1AO2{4t3z$`Z zA?K)sC*r?45>mRi_!G>`EaAToq;8AFMWqw@qIlBy*~K#0yzzAxTc+N67~m& z%KBHopbiD8rH(Qw@apwWu^nfA)k_8F!Z5}13;2K3#&kJ!AXXh6KRNmW`TEANE=n8LTI`=lE-o#!&+mfQS%^K{GDJEHwJsuY2A=l4v z_!`ilZ{R=B6=q6{>l2r=-o3>KIa(;mj;kn(X@b(XJNLCAn$?mvb&BQRaQ*1KR__Q0 zNB0GzU4=AoJvwm_+`iXXp<=WB;!XeV)OD%tUX0s6iapDhf z&iDL~%Ddfr5BlHrIsII4mf-8iXOMWOc;OYVm;SeShJZEcYnm2;`m3;qWWU?SG_|%~ z+-{meU;8HPr{CM!4*An*_$=C61ZF6T=sPP3_HrgMttBxg2Jq=EPv&M#>1v1)o))Wt zfJ1GH2}3c2p>6{3Vbx>zu>UKDS4B}J`mi(gBsQZjib~Xqdg@{z{r(Ck>TB;$GQg%Dn_n)>9|+ zP12Y!6*{B&)`sY&PZyUU<1bg9b}Vo^t*U`2)dhFRQ}FfJ)cMA@V0oV@{{{Kw&)EzB zSOTL-E3VM?Le&Ot1^*!~{421S0RYoDCDL5H3K67H(xgLoXCg#s%VQb5rh8waZX0hnG8R<Gx_hv68};x`x#Qit{)`jy)9oBX!FT-S?ydYPxfMEhxEE(aNglSgReS)9wkz@T zdGd*C#{A7)P~X?h62uh`Ulgomx2&_CCq-#pt?Tq2et^Yxm#zYhrUaR9+*yHEg(&+@ zc$NXs)miSEUwU9~|AH1&fov_;=l@FC9(HQa7O=ysB+bp-vJdLlxe$EvvLg?jG63+z zwDfB{gz&`tU{!{er50`+*m@8#1=Owl62`eRb!bN@K>pM<>mMBtpYLP=wOc)K7hWBZ zMPft$bt|I7gx}Adx?rvnt!bj_6o&wW^yu4BgF`e2;93Cd`2|-zS0fyzse=W8)b@1| z;18(((~ZNnOelCGfP^ifq&0Nrt+bAz@0e7yH2tDeeaEVM>->Qq)l^}4x zWcme4{8J3rg!eAM0SqWVwvJRQ$m6>o)e4Rh=v{Rq2ZbOh4V_t;5%4}b%M1VsAs%xf z4NA@?3QP2;?0tbUNAs=Um3IR94*arx=gvPWwE()fI0N18hrKKehaJZ8q=OEQEz-LQ zJ&9sv)!tF>$3U~p3aj6;YhUYWU!LT^ji!cG9z1uD0TtZlpF}6^_MqCOSa_0FN)Bqq zPDBY3Hz1Ghr;9#V(|ow ziDrjwkNlQ#HgBo#i^hnJK8@O*> zU&t3qD*RD3v2tuvyv9|9^FKS?mCQhJZK>+&J!y;sV7$3FU6tiGi*?%Z6R_AkHUOlX z!F8Fqc1~_9DTN4Yu7j=~;%jh_m$NcyzNJn_5_vGou{&&d2sWJ+y@I#evPm@)$v$lF z67*vw3 zZ{tP0U*<$A?n;JgQFd>h4A|HUb2d^DYb6}L1+ zp<#OHU5M>$El-@NyfF$mL?O7%G{iqNs0(Xr@QV^kai9RnstBw_3b}dagq7jS%pzT8^CFs0X*3+d~za2j4@IykSZN3IRwgKT^3ZsL3*uT&1fx6&rxjP)!WIHLl zGUDI+ZA5$nwdV>Tr$x9KFvy&Fc4IvPE_s|}b9n9pe*zTxpi5FR!IcoaO8=h=ezNR= z!jP3kR5qWoVhDZ&yDX3XxqPbDpOS|DT1uC)J%VWRv&i{wz(>QxHkvyqflE^d4^9tr z-CF}1@GIp^r+`yiQV{u>zvv4y1-uZlCozz;jdPzFWN%M2NDb1bkKi)PLTXOVBIrnp z_fWC8|EIn0jA{bw+I6s?qF@6=ss&MLA|Snts0ausNRgHq5D^d~QbI`)9T}tr3m_du ziqeE2okXQ~q=gcQhGu9XAtVG+?!oWgA9sC&?~iY-`{SDpGFY&QN`r@TF>7sIY2iU&tlDswk z;^w`|ac{uw$!*A|ps=^C{HOs7tVOV;MK{s!CCc*EK zq3s_jfujvdDU<_pEBdFcFcW61{Vj=B825hJIH(xs$LW851b;sGQS<9J?yq~%xIg+s z-y)6}|DQ(Md#PX**rbp@?BzqlaZQhMl1=5)^pAIlD^rGrJ9Hn^d1XF|#$&*s0Z4t#T)z(aK9 zPCA!B%ABu%4N1>OuESK8%;cK)zkozoGv@8P^TlyMZHO8(wwc_W8UV4dDgguLnODMg z9X#1k={w;Hlw8cZxHSsJBlp|6#c z9on$JaGtDp4{I1DILR*;#(un&l!S^!FZ2zEg5CwRY#7@NwD2Rghrt!1T2DLO|8FlI?GsSb}{a2wMi)DAR>Ezj2sE zbd*x{DG>$xeThyFOz4YaHR^srjEk9oFj;x?Uzq^nzyoE6!^K#ti$A+f@N0=rzzs#y zt}O=6&4Q!TSoQ^lf}$%$`4L@^`DA(PySX-Y-7&u6D>3bF zj0x*&4MV<|NdG#3ZieG$8b7DlG!|$~%Pz#`4c*Nj!dlmSna$6C!8jDo7-+?GIxo2> zm{{195!JMjC2sxI??1$9GxRlpnor~tlQDsV#djR{OOE;oOX>)Z8`wAxu&Yd}8r8>B ze2%0ok2t55vFZ$B(at-fBPEfVDvNp(`PZN6*3&-rfBn#LG>knns<~P8nEV)6lotK@ z`Et!L9H_ttz1E;_LB*>-ZwS;~I}%hwGS>uNo`~yKqKu<%5JG7rO?t$2aya|MO%d#J zq4UHK@8edZ#5Tw=?!qk*W@31wApUA>^WP^_dtNZ#!%WBXvGvT$%~XM|r#zVVcMM^v zH(IkY)b}fsz~2??9Q(@N{`V8AQwCDvs^?TEvtaJ?OKmS-J%h{+l~9{1AA9jJW`|p! zuf$Vt@k*!_{DP)@3-4DKp45{XGR#|X$LjQMw+Z+*6gA=mutkR4fv1TLc(gRmY0UR> zZ;pJzN3wCKP1DEXex>uLPI5T!hu$qj|43B8K$fnWeB~h1IvS*JuzObl+>2#wX3fi> z22P`x+dCp6v$Ro(r;c}efrPM^zSn2HdGltwZoKiSCx#|`tXGmAMb=`lDUCel!%D=L zANKG?!PZi0yYhaB&daVvk9WB;(kjPP~xMPtgU@)e06p5&Fw5@_*%j$>oLZS z@o{3Jb+YYv*Wn}LC!x;_rXqi*!q3N9+6<$v5$@B%_6y`RcoGYD)hnl+J8m(oF3k2w zC#RUm7<|tRg1R1{NoVY4g*J%c_e7L#ysYB;>%*BR`AR^ zFZBhM79kxQ%L#!Vj)Ejfc(~kH&U!SD7DMXX%GXp`F?!O#kRwXJwC{+jiIskaY`Rk6 z*IYSANO5Jn!8KJn=Dy9KM87TS3QdFBdl;k;EDG-qDX8P7e0ChDHPRPNQ1-Tj`}x*KAVW0Wg6 zeBIf&j1;Ymytu8N@1kojsOr_>I69H%h2#4^Up!vgzJLz)bdh+*wm-_bNxwiJWq-P5%Ld)Y6U)c-EDbPgh6s-) zk5R?AZqwDxDlt2)L5ZsFx_N#|(kIZ7*K5|_oun$aA)AF*qkG=A%&am$x}jYaenLTC zQ(UCjRE7xJ885QG)xy#}jOe#SLJqV*A^tMB`D*vykqrAY^o+E>xWVzV20MTBd)O-o z5WM+mCYnh0q<`Yu+Z3VU3aG#NxHM!=c==MmO+ZkDd?ifQ(f6m-z$^@4z|jR!7VspE z#J!R25Ap?0gt5b%C(7tvzs38Ki~nvM52arLeOZ*#z06`~)b*G)d2sXVKWrwdn*16# z53#`HlN!olH!-&xH_Bb-ifZmT1rED|?jT3CV*Z|ylLt@v0EFviHRpY8_UJO-QsVP1 zwd!>IWw_hP)Q%{uXG(XB%^I6XxRc5h~~?Cf&eJgg~imCM|5 zsoO%`>t?&Vfl`aSQ$vm+cIS!zY^5xLjV>zciwyROz_9?l|0!aiP0rGJ)DFvQZw*bh zvO3lW69EY&_0Yp;@<&E3h7)#f&!lR)&9X=Ha(C^%Zhe~J??K4VlcFS@P!Z8Kj|_G3 zS2*p(q181Qs0sPNEHRq+hJZyEja4@h9OWlK$xDUf>g5N#+}y?&LgUgfy)}m%cRkX0 zh164?c|s>Rh#bSO5I)wMcZW990dqeMYNy)oea%adx!EqYqTuL^NI9}K6Xfw##$_r!#7=bIR=_5&$Po9-GB;sk!xn-55(Vp) z<*s)lGX%E^lyH<81ix;nk%OIG4w>FtSrN2(%8PqAlmyjtVfjMM?StB5lW{Im?# zcx@%?Z9=*2k-D(0VeB0b;hSmNR#$;*m?9K6y`u2jdLy%dI(x5h0jN88-t@~O+-_Zq zXyR4t21==C4=p&yChf~0W6tDQq}lyB^vaMsj?ZcCNmyBucP~T5ePLQfxUji?i4Ce4 z!Kdp&(uWT6Fbn6FK4Jz1lm!){RIRrl&E8ph=dbOrzuaiTKYtjitD>kVMs!*i^K4z? zRhXC!LZvGbM{b-VN2ppv3Dt3LI5}ey*%5~ZyCYut{t>NL9Q<_6SwB$$78yh{$Vp_p zgzloJq$kDSZpj!Ue4x)%SU==tn?8h8)pmhJYkR^sE4~c*3oP1r&x^~9ZDAW(L(bJC zSKMm)S^KfFn6v{e&244#YfG<2EMy?B;*M8KX_v@L%1V)qIA;kn7ncJla~{h-=936z z^cZ72kTT>wTIbsB!-%eC_6L|>l2rJ3b`i~Gd<|Ny5Na0pCfZYqZ3`WQfxey>eGXr| zidz|SRp3=lJdxq1u6IAJNg#fV)p1%h4ZEO;;^uatcdZ<( zE=>+>#|E~3^!@%=^zy_=Qy>neg5on~bM{kX5V-x0b*Ox$1$bZ>T+Tm>+&Xl-{kpiq zokWUy!=o44`fiBF+XE-V7YXIDWw=?XFt~8I(PpsXR#3sauxCUFwNkH130o$08q^JU zRVcZiUJ^p^juuG-xX+ilk7QOw8FXvtOUn`c9_W^@l_(XNQ1z~7S~_Q}4kC$>^*rv9 zk~b2)d$Nzl^mYl->eXFefzOhRIBPjKSLdp-&9JjwKBJX_0+SkD?PHtmOAXi;$Cws` z{9;u7J%fS)n_S9UL!RPV^YLcA-YK&o6uSxCj49Szw+OTu(=+_GNQE^wS8#$7%gQ{q z)0{74_Sk|n$yUZ7g zb~a*A95d6au;Pf+lk2qNQIbt9js#l4LmMi*ivp%wSpwbYcWBGP(+}{4a}#;!7_{X2 zo@f_JCp;2r?wEQ@ekl>hm9^z^HK|#BfHnA9u zC?Cz8cJN5mrIU}l-G-xJ23R!3tb(rUH%qrVVx zs_U8(K4_@x??$1^oJ{zLC(1HaL*=%fYQELD!P_M2UPx`tgT;rR)Xk8GXVl;ycaR29 zAs?q}RV}vB(q2~v#~+O&);3u~|2giKL4jlKePzI^k#i#^t=pzQEF|*22QeB(-23Dq z+ASCVSfmB}(h0lbt9P#liZUpR^W8@)Bo7GKDHa@>yA-|q$>TF&`a~jjR)Iu5m=TMV z{$c00u#moUl34U@ni&e#u8|Y6iPRwrBu$A;q2tTTq!ylT>A5xD>a*^K!oLajTCljf zPi@5F$hA3x>+gbUaKlOo@L-dvT8toL^}7@mK2n2a2h(4LAFiFsq`kxKI<#Hiul>hr zW3t?tBL;fx+>)8|p|)K*6Ckb@i%hW-12lxU?n?MS7q-&mKwuQZN^4(?o}g%BY1`n) zp3lA7&-4ef>ZS+ugL_21Q#!dE~ZW4~bOV68qKKYQO1q(RmaywG`hP@5j$NXkSKq zz51o2{){BiqXt3h@`c%UNp2F4{Khx70e>C=%EZjjYujmK(5p1U8wEbrT!ND*56&~N zdu2JaV5TzlDGGcOn%vptr_znXuncSRs&9>iXku(_88DN7-hiF^er%98y{kJXYlx3Q zNe}(O+^5~kU5*Qf40Nd2wyMYMU8x~6Wa>mNCC{Ht>UNTV6!9KR z5jF!@WP0kreaoiqZ-L?XfnXJ<=|s0;+p##*TsO2au&+G-Pwt3ESClnVKYKd)1$?J& zrR(+Rx?s4u$BeX!0pmx7ai_Z6CVuUqArlC~B{p3KA1l>cz{irzS=?95n_AiZP;XNH zkl_1mrJxgFAWelA{fb5Jc%OXSVl>p1O+<9ZR5J$$oM5G_-A=qHanhIYD3TD0ZIu0@ zzFbOVukraCzd8uW#z7q4I0*Y1AWRITmwz^Diy=H3q(eC+d;mVarkQu39cnLW_e08`@Gu#GlG`MU7l5D6s8sT0yX zO+nqIPnydes%}iNn&v1d*8Vr(^-oFM*mX;+Mf82zN=Jd_4AyVsrMo$>`|sdGnjXb| zBw{$vQL3h>dy_NFYzCRXnb9e*obCr6{4{yHqG%|5Lc9ddqHXBWM}@&(9Sl&NlIZ8TwmZfjA1X(@)&w(<)WlmU6G*Y`h2g~5)+DQo0@cvOG9UHWOLRGplZ z1MM1{Y>7I965de1q)t5t`k;t-Z^4iv!weUMYqP#iQ>G$+&p22`$=-a7HmMug4 zdE*4YjWhVGYv*L5qN50r)deGMh>JcJ)IN36BK5E|`y0QvHYgAOe!o;O40a}t2Smoj zHe|VVfgVPqsOfG;#ad9|qXJtbkTCG6c)KP*iDP1IYp!l~wEJY1EH>;$mop@bC_r z{6x+AUUZ{zS8U2jL1=}ZQu$6br6gf~bmd74pI4H`1I&A@H=SRTD?yG-s^)q$v!POD zCf$xH#~hto&c&gCijx~qTUYVuO0};G2>?ym3YTM!*bLu(y~Gj4EF`6G_wYbCNVpHG!9jRzGY2VUoj$mXixWY?CCL~HgsE{fAhiJq{N%=odliO<0Km0 zd$8asb)vZ$=&#m}hn(9xnUc`>bu=cTY8`Swxl_N^?9s$ZVEHI=_>fS&1hbzYN?9ZxIaSObiFu=!PzKJS1+3%SUkk^6( z0hH(xx8`#3>SCv=b3mws=wp1N{+}PkGqmKr zC20q(M8RUz>tcrlBDmIn;x!+u?cg&$mYcN<3ZEA=snO?_+xvXC8Fbxr9??60_yr#; zWl*%GIk}Qq9)71_RJJK*INQq0L&_g$%?#Oo-x%URMWKM^#Yjgur9=Nz)MoF9jG>Gp zxL&O_7(P}`PLq92xl#yI=|p5qiGS;R`%zeP9tAI-e54N+P?xo8v!aor$=XdvY0Lb(vjG$+5xb`69+22PvjCwXl^rh^y`Ab5Y~K=d zm3t}t*+e@#C;fYo!e$xr3BvEvKkKXzTQVzkYqUuH;v$VNRTMFS;-Zj)(qigbu3%m- zOf3$zp^Tm2c<-sKS;8TYJsDuOL)Yo(`Expf1HmlI>lLf*%qmNEG%E>E# zE?6_jEvUnE?E!GW_lzLYQ_^!f@JlgZJfpBj!>ABe4}ss@lJo$O2su&#lCBijY2;&4 z86+msaV7>We>v^2fE8;+v^&a?zauhDD8oe;`R%V+vD)m1<)!_MHnqngK^+ny4p$Ah zQ?KrK%_qyv_2-Mbwhc{Z*UY425fYrNh||_`jc-+xJ#rJF`g60x*Ks7;tNVW%v8<>R z41hse-5e6QE_G)=%6GaJbD|^2QA%iEosTIy@#7A<-k=&s86q5N!z-wLxq7l=C-IsL z#6B4bG<1II-hk5(0}K!xRB2VGF$#HBAnS-RTfDNKmf_;GKeN1P7xfcO;7zaqy*!NF zTSbhnL#NoTzwi(2@7Hk~d8U}ZyTCrkZI6z>)#$oXx@2PN+1m4|_x$Wup4`pon@t|o zaZzE)tISu9%;+BLe2H=ox#$@Vf18P%?8$ZgT>RebN)gVa=*b>^pY3Ore71l1gI7lT zG(R2BuRZJFOj?W8P>)kCAbbEXQV?U?O~}y~43`Pt%o@sm|5DyhGn|WxwMIfhJeiuD9zi9=A-A$c-J>PjCuwb-Hi& zeLO+-b4o%-q|arwzo~9rwI^anvXrFixb#Xo_6y=TNv_?aL4V+T$ak8yzPaZ~9!Z!y z^%xOo;U#>|vAA5e@DwG=oI(um%e&9b3hmtd+v z%d_TV(Mg}bdI~L9y<2<+Nj+p7BXxkTn_Gzl2@P7_(3= z59&y=?mY}YWy^*KUOf{V@K*G=RZr?56zcTA4Swt5>1gS+rnRUV;|Vv+#C-d-Fly8a zN%h}~DhmmWguw>5>L7Te{D?p3PuzNt{>6++#fkY%BzsAt5P=!Z&)o5vesuXTZj zrh(>*h=NFYi(~@3v_05U{x8gKWyQP6-mm*(kpP_GmCT_U77>ByFXxu0;_9y1{<&-I z%SSNC1M{HngESi`(jR`Z54rU1bE7-C^DjzX?fNgSo!!Q@tLYM&{JVi3KCDs{?BMTC zUMLU!vtsvw_Rp?f_*d_id$Ge)!}QY^_4F1>(r!0cdfST_4;XuPFZjFdUWQy*f2P6z z-P@IBThp9PrnLacnbfECW_nIZMM9Q% zrQj{2>y^B(tYf>)*0G6m%EFlaD4XuX26M4Dn)Ro&wQaA=iP0#miBM8puF%u9wY9_N zZe0PhFiWbbIE@PL7skPkl*stALIh>32pG-}zV7-a<62mbHC?ndR?!tNg+-^(HJ|mq*5oPFg@~01f)}LwStgV=h3ex4nDNpvfth&7R-K@lM=f0MB zfc&Wu(m*3kd_qEot%C#h+9fUf`5+mHV3%Wh^8>NI-U4!#mFvy)7$V$7P~Y*(AQ@*b zJh9^?3W9Y>J}NyWHa6kLRX*AHu$qPlC}9kXd3w^zwsB9+ua7&KS}W8Bn@YehJ3P=u z%yy@w-BwA2?74%@I*uwAoD9Vk!*mU(tXQ^x)*?CDP`}q@%B)Xnsg7TbLUtL-s`Kr9 zZ~w|x0l3eVww-f3W38hE@{e9k(mXcH50$RVXI?1oz-GiH_~J4T*lNH^b&JTCVn}}1 zk%EK9J+zb}l!Z=j`US^!pZAEx_HOv7@D?vgyjV-_b3D0P|4ks)Te5|y4?=!ABjetT zMdId$yKSf1M(}NU0mR)?*-h<24qEmE0Qf#rd!bio>%w~3{=;5|SgMBb`%N*-<2B(o z&v$fE$Gj<|*z(?Kp}xUM?Y{ zv^LFwIG$~>Yts5)vaPO^YB5fwsOD~93S!c+sV*Qc)y1WfPS@0Y!cfn6oke<%($a}N zf-$2zAW(F%2X`BKPE&ikV|_4oa8{M(O86xe_p7{Q*Aau$1_Fv^q(q8*;y~{>O`~ws zS-W_I?={?`DmoS?b8XeV%KT%uPgQ|nMQ-hi(HZFDqNyFQB`5$%UlV>YQSchjyGuN! z4S4mv!U8sT{XbNG&h#@On5T$H_DZ=OM#L&q$qIGk+h`LHHQ(AZ}O5OBAEl2Yn~VF zbaQQ<6vGIkQ0pqFxO7Y68OMyd%S9tHaS+OuwO}Q@HKqxN+jGtTlsM?8F z0+|MPHfCr>;P0k~@8?6vkJ{ZR_3fiO=A{lsy4lpEhGLxEoSrcfGhQ+hDXY9`Xe|P!O8aD?lf#TT=Ro}+kI)3J^gY_qfU39N6zr@ z4D1$#nNZr5{PEOD^n6PxS)403%PkR%r;^F$| z$FLC~pKy8$TG>TP*$H`vu2iC?SSKfV80o5TT3<-EoJt)LhLw(<~dG|3-fDSX@JC%YqhYQGCZNC$7g-swwJ0 zIG%!Y#E8D$yUe61szrArmRx0TRqJ~(HDo$2V!*Od*ud)!*OQeUkz8`*`J(uKd6BDT zn7I_Fj`!dM`BDV7Q%;f^g+DB)XiCwXMMP7HvT|K*e`!10$JuyPZB$2&1jmxC4gR@>+S^jmt zWR42?t-s}ICuPz5vyGSuZ@IDpr=GHq>+reg;>K*IjW{~e6L$4!mUbBoUNxzZ(MR(e zwqjPt9!VtZhe-K`gv9R7n4V(IEuPs+_mIDG)lctD_{s=R7i+0lFurY28G z+oZL}=Yxo&xbJBQBp4IsZ3eWTLNx!mjP~a@82SLhk6S0=pu9glo$AhYrb2l&w1Mhz zM`N7Qs|DpaWdRtSl4D;-nkrlSvZn93{Vyz{@667*n%reu?Jj^qIymY>M;u`qbLF@(4Y9R%EO3J@z%t^#b`!f&%}kX@5CR(8YU zfKn>1veyqTE<1_;%f3>()7L`v6|7(&`rl=4ePE5Sw;)2)&4i)O+Nzqpl;wa7&O{=YHl11&Sb^p&(tp^Xoo;6-^p%8_40`&OXy zPjM%VF9WusBX5Hx_+x;AI%rhP3(6*O9qDLD`cgXp{w4QhsOY8#uOylJqNV!wJ+6?r zp#?L{lsWrg@H2I2mRIzL!Yb*}P$A_W-~)c~mThy`2-A)!<5nKb!&kb5Gko-il#!gN zW&rhbhua3JBqgo9zoK7!Jj4lc<3$FVDVj#>rfqk~ibJc4*$;}a=fJD-} zj!{&=NULHMKisE2c5s4=X-(Tc2?4a!zhh`zUsWETEkg|q@ z3nTRZ2u#X!^|h#UYbwI%~=X|w)Q;}&{62-`ZX%btBn{ix28GkHsRK+zcmXV8ae=DVHaN(Dd%&rn<$bSwm!c2X8I6j<&ZEc4+c)(r z4-GwlOBnrHOTraL6m#v`kzW@GB&*AOTSJASFKlrb!qns&w7Wn=Aj#|5L2L23HPvUP znAgH6oCzlB)zM~p&!G4DUwrx?xJUsJyjd9TOgN@HSGFH9E*3WNx0JLsBjrE<-?q&(v+MlU=-Yjz1vAu z_>3qXr;iu-&F~>l+fWVboustdVD^yh|K?sg@%KL?kTbPPs37K zApDr|C|LOg*8LMFKDW^@9#MGaSCBhKZTwRlsLazhx@+?b)M~>9 z4UvFNXNh_1$!?x*Z>aT#?E(Hm%r!r9QxE!I|DOiN$aZ(z&z;_U<9>d^tqmoweU3K& zX%*5S`am{0SZ@9n{rm(OBk+6SL+^j7GzwP125-=^>R^cJ<_yZub%=p0qS|09{2x{^ zX$;h>GD-%m|I?vu=!qRIfB!4D2u?%B0GJdz`eXWm|0a<9fB6f;|B30v9=t=^vPIC< zsU(@^7^!_!yf8)M?2-^NX wj{E-gdjER8f4$znUhn^f*UOaWWATKpH_>Yk7Q)_c0Y4^}EG`yZ_~X(40&k2BcmMzZ literal 333294 zcmaI81yr2NvNlYBAi*WLJHg#0XmAT|!CeO!oDkgIH4r?wySohTZiBn)&p!9wbN2V& zn>}m2YxQ(@b$8WU^>lT0&4eh*OCo*1`v3s}fh6@sOc??K)*Au>1``hQttPTpPzwSA zQN%)2R8dM)ltj_d&eX!%1Onm7Sk{G4y8)Y-0?n__yXh&n zt#&8Nw<~;?yCV?hiyyWxbaW6%6x0fXqu_QGq~*5QykQaNnIXTBKzBj9DeCATKAE`z zpfqzGXJRg{vrJ!Ce=?wB!@aBEp3Wfsgt|-m#0D{lHuNz7qQ zndAqvB4)xO*p={i)C^oLPLsUK~Q6!Kam+=U38<79#LROnZT8Qp4X>A zBZ&kcZ78|9O`*+-p<1)O>{1`vX^;%y+|Z&LBW?L=6#mS{`os_o{$N_9_mU;2SBAbL zqdKBE97{vCHKBsNW1}*5W&WWWRgh#j88xf|xl1zdYY$X|YEHPUNOF}OHkN<34kXIP z9NA}c_AUtsZk3ccO!nZ?_aFSf-ndrOFn@ADrKZd8M|tS4Hfh8ty!L%s2T*JllIk!r z!pR8pPkuKulkUz%l>h{sGA2^wlY~2MUW-p|!@~hkZ5aoove^~A%P?g`X!J~S75h|D z8PL{Jir@+i#W2rW)TWVHqP6ygT8VKiBHI<<55&ZzM2rwnDg5%wHJ)ozqV5|(tzL}jW>E`s-=UKxy z)pDFr)K`mSLYzaml;TygnP39f@V014yx5mtOt^lF!e4d2Fim=x6X*lJPz!_;Nk{z( znmP`|K_#1MEc_(=z@)=|fRVe0S(t#-d)7;Q%u*V06YY{QE8mPXcosA?{O#A&uP{u} z4}k(vK`z@iBy@)HEVxcc3(>P*YWZ+wA4s3S@C}k9b;dY}#z4osqu~VhAe%%fmlDk0 zP$r4G5m)7Yy?n*mA^}x`8&HY|a6&(fZYxr~?C;1#VTuiWddN!bsqZ1(q<4trRv@4} z_^yynau9;Gi4y?J5mZtY$MIC_swx3cM4TkWFTg8cEQk`d>#qEODoUHZ3sL$iay}wb z^Fm_WMXQiv4mwcfJw)j{5z0#5vuSw zUP1F696dxNKg@dDeV1m7So^f}ccY{LY4M5ggi-?L9b)+Tj5y!nHNG>14!-B_z=kA6 z?E6elA@WI9Tr)|SNLc`Cw?l-+msJDa%7END;J5yAs4IC`LUZWxn-o$`GW=c+2?eGMHop}jse6@&Y zf5J|ALvSVM5$6rWVV9mEu|1^OMpFZeZA4UO>lq0@o85 zu&f@Y&AD@zC)`yTh@YR&;Mlri&d5Z9HOL|cC@_2YbK(n0ii5qxpi&P@#P}xUTB!Fa zlD6X_&AlXIrG7<^P=jNJ2O_p@s}ilL%41Omh%VS7H)0bG2T~o08RX;&I;em@7_^#V`qd(XL4ksWS zy$_~OuMfWOqXg_$MrvScQEHf0gH~#)tWAtoz#>uYmZkc1X5n>_RgrelV_|=B*tA(u zx#HT-)ogF6R?*g|qR|?I2NsN!g^_@f>k+S!sgdRpbv!)0UA$F1431xT2<)>~1moNV z=fYb8GC?Mw4+e6kw3@@3%^DHEmwu~Tb1tP44#*o}S0z=KEELS&E}U}fjG4=0O7^6H zinga6%j`3)imkLPU@YWk=x2YA^Z#;_s@9g#9oC`~lgZ{#C{aEPo|P-rvni{qcKPj+ zdBb`vun4Z@bH?GFdOn6c{}o&pTY)lh`*sOx%Ad zTq<75EyxAuC;jG9DOWyC*5iv>*SySeZEX42;@5)Tf+NfeEdu% zA*NNSWx_i5Sb{;KwU4R~t(9RaOd#c z@2>Ib`pMKgDkVnqSIVSynr4Y+R%3))>^<4#^X@HuGL{}z3YHapD!n_s97DhMSmp1^ zr%~yZp45!l0`z!r6z2GuFmD6Qri=N3z>I2rT6@L=)Gatq>&;uEsDPwX{K zfciBerSPjQM;2JJGL|zOGhDb=9A+Jez%2ufo%eZRtzqMNlRK$&OB~D2ZC0&^2wbpS zl}lLl+9x`<=O=3oT#Y^#XIF}@N0)mSHCM8?%iC^KGIMleRw;8lcB#OMThG(n)l)Bi z&ktV64^KOXdudzE{q=y~qZ;S)ghs^oDUa|^b@#Q8m(Nqr?YDL)%m}E^U*Gd1*1%!D zKSA_FH^z>CGlf033Y7}7JNvUG)U4=6(ukj8=3?d+=2pq_8p`v16^|Rc8=4y&8vr?7kYO3pIKuBS!@k=diu}D&HAA0>r9{$8 z&hoDqKaA|#IKsMxvq7kIjW*Z_-CL_Hm&#-%$T_An1Mui6{ zz+kLPADf5alm3;v!_a0RWuFWdz1x*@&m0G3I=cAasGVo~!UMy7>DAI~UYQIR{j<)e zPpyi(U)7Xu98&BT0#4K0wO@5#FGhAs>G;*GHKp`vwcU(VJBn9!7I%7gqST`Fb(;g* z^PNTyYo}&2%k4|%n<5+K8_T@BMjjrJ67kh+?;4b<&fQJycV6Tq(h_amn>B0hhY~v& z9_#Sz7?vJ3Joazv>%^@~N~_9>=jGS*YM##{w+YRB&K`cc7i^4GWn{Swy40RVuC3@C zYwEW4n}b5}S|l{5YJnH4!$N^C8tov5wYuk)XA{`I`o)OJXB^Bg>X!RV!S z$KQ!qp{|w}v`n3K0#E-^nM|2i|CL^@DsMoC9 zDVt{8_-=2))F6a<1MvQBTn z*SEv(KlmQd3uj}2Mxb3A5x!@Hjv*(#OHTLYOB#a2Z=fa%M^X!YfhB54QXx{=aka6w1@O8G zko~I%?_2q=Y9=z0e|2%P5+Ku%QzQ|!b2K60U}R@xCKLQXLPEmtXl%->EGF^aYn>*p7XO*a7VzI}y)nr2R|^vhBQw*#$9^N_ z|Erc)(ZbclT0_jj#>5uzHisY!4+}T{zX<+U(|@M?8>!~sNI5t-|4#b1rvE18XZnkU zzp?aR^7>cpo4^D=@H72e_<|o=YdoAGAcP^L#6Ev@eRtFf`<+7FxmoKo_T7=$J*e)2V2$a>C;T!1Z?q`#xKH<=+WPyDM;SI z|L-DJq_E9KaTV47(e!TuVJYGLg{^-aC0W=A<&aIUp9r$1;SV!Wkocwr2xo#l%2zyz z7SF9iNBT@pj}tg#{yeZSYH~ymKYkj|d3xHF&d2zYeIAaY%0G_(4xSzzQgxy+b+!9s z^s~FX7Ql|onaoC-hw9&~{BJRQV?o_hO;nM$GeqIzFVO%=DMpITsk=uz-c)d5CH?RI z|Fc@(RxuLzMk(_X%iyPql(Fn#@4)irTX*@^5NNAJwGxgun_yw-Of6?)U;iiJYFI;~ zCOtkpyy1Z6+@W(@)%c)>I>zPUy>%Wqa$QGygrcv%zusnvv;Bsi^8b2)|IJ!h5QNrc zv1$nn3{31*I=aW?;K~=!#)QsHh9nF*$$wH#A>VOV2UZCuD!F76v2{g1@J|ll z$AySF06T`2-ls21{tLn0&<$k=PoMrlG%~>u7fh|a$I0{fCFjx`KsZ@Q6K*FXYciI(Ygs+eOp~(A-rBDcFII+s;y+bb*)I+=-ElW5Sef-M-s-bw$uWt=tuEMAIF)bE)>9 zEK>m*ORX;RhQrU7UXKmuFM%!!Zbyq^)xpZN*104`$jR!p0-lm|yBzH51bv|cJ8PijnuQ}fk|G^$Kw4irCF@mhi z=PpM)O&3Gk*_tjJ$#WGJ(}ikQ9>!TpQpG2do1cVK@NMJ%2%r@Ta7V9tAldg{p}a=Y z`!F9hj0PtyYsot8k6JYY&VVlqyEfE1>N*Y`MJcTvPMC^hjWAmi~|SlpWU5B@K#bRGXVg3sBW!YQ5_MwO-nY9vf!sS#68^ zBwmxBW7*_rkhmrd>RPedGgEbN@R*Mts{vG2fi`RFXQ?xEC>lQN8+c7~z4%>^@ zhlg&2`o>2SwUx`4FS$%}R`)wgF8q&+vu;*%$l=%+9K0$Id`$F6R{tPQ5Zs6=UFL%K z*yQEvk^-;$bf7ko%k+f)oliM;1&!KFHAk(r2?PFblV>*x$@B3t;A(Dg5L+d~W(SV4>nnquPf(}tXV0uSuB)ebL!Cm0 z(p7%Wm#z|O?+O=PBLcgFNkGd0UlZW|n9A<(e#VaWPbw%3$%WvXlDFMx%g`F=F&vG1 zPpkD_Us6J%g8y#a&*K=I6I}E9u&%CmjshEgqOqi|pHdH7ZO9CIxIjEL`DNy>Czy8< zz8MbFT4KW=CAqY4+gF9JrGXW9oTF-NK5pd3xgg}4_9%2V$i#cTisC`BdxQY_Q054@ ztIu^b)9&lb^0~gF745!O1~MLKmU2W3*?|WK|$bf zikNXF4``5js0ZOuwcvg~le;$-LG=RC#I?YCZpQ4ii>(H072Nz8Td<&fdEk4SIm)s; zF9z9g0!{(H`}_JTbQ&FZ5N{BwLUtotj*n3qx*f1Hu2Dk3gn>3eb5QFqG?$VbyFhB+$D#&hsgwgMSGo_ zhs#Ox>V;oiR`X8Zp@W8Py;k*+p8ulj)XdUrwF879SJ_nFtrPBJE)8MULioWE*&m@n z7?Kb&OvydpXlEE=>3C`7);r4km73btqaei8ioaZ8`(8|G-Cifqt#2iH4^;bGk7>u$ zP&i4NeG))TbmaiGpcF9)?=G%~h?bk1SzfwRQhj5e+hFKn^r-#<+Km*TN8{IWaYpn zb$32sOJ+jmZT)4}bIhL>cN5%^o(HzP0=H&L?Ag@YIg}U?pRX!l_6s>Ji2>NlBTVmI z10gavE?WkhXpD57;=z%yY<%%%*1B_VZ-;s*Pm~>!U2sCQHXQroMN`+IOASob!@D`% z>tXemodsfVmHr6cuyuT05C10J;jk$E>Ybj-pAKD~H8Cy2b<$?wwC&xNElAzS&+3~0 z;Hc1(LOMCW!`>8+SGJrJG2PC6iVeOV7=SZ{CHAyWI$wQ}zH}9~TXpJAdcD!?1JmiH}C?}b;l<I_u}$cM69ne?6$K;(gtezhT3mr$as0s&UzuSYWtDnDc!+Sf}F7m&{tI| z&Z$9{V2-$WY%N2_XKLaN=IAv?s^uj)n^ak)j_4YH8;VA`U((ZXNP5UoZb_uma;0?;5Zz4bCYp=9NgvRqCvj2 z;1FP&DB?sbj8SCNqI#C!@ljJy5tEKe`&ry;G!HxTdb2<4wh?7xG~D(%ET)t?d^f3F zH-tW-9e}Ij7uzyb5nj+)kSnO7yYHmeByt(z-SiKh6(&6WZwdMI?M4PP&COk{skyec zHp!Pm{5CZ!otqP`4b$b{b8p*5L^57&3y<11&}V)8a*g-=StWJt*~xq1{g#`HqK}O^ zI&PEa=(kqgJ};_tHdgkR)#u=eS^}TlmYK3!4$HK%*U^&K{KoyJ1O<-?fOf5n3UrZZ3o@<%}_dx(>nDc%xDyWt8w!H7mJK4&(>ZB+Zbz8 z{j|hCDO$%SwPJBPLj9YM-nNj?@3z8TBmzSx`b zxf}~II*lgwNP6hboZ~;7X2^20IefrJJ{-+mR;k0_^fSkW8-PNZ&A@(NWt%v3 z;VbU4qpj!t643rKGrSg%ts}iifJQS7H+m8a=| z^l~1h?|r{xL7@M%iB0S{K*|>Xd;)BL>?VF;KGMIbN+PuHcvmf?3_MxBU%hQhl28FO z%PHOqy=)0hen=#KISVRp+{HtOf`Te{?WMzu=tF!?_bRW6{}VD}ee_)!3Iwpy7?FW2 z8W%LjwI77riJ#i_+?PvMTP%8uM=?rAM?Eh?Rgb|fUY=Wob+))`P0ohk&-jZ9-WFRN zYpV|Ru-9@}5n62#x2SbP^=O-T^r{jN;QbZw??U=ek@p zE}2O;2g4FAHr!%7YV+AX?Vyy?RkcM1u`b&92rM;_;THluv+wl+4@u0~&{WeMH*8LuEVd>VG)kOYOgqu=*7h}`Glx6yG}E|nyq|+^??;)4+-{eq@}dLn zs#11|b&{DHw`1Ql-L_%F`?h%v5%N3$ThDdDBuUBt;S`Ydm<_2O21>W(Oj~_1v3OYAxQWHSUT>vcp9Jae*X4v#8YqXL>L`Q#-Nl z<5Yx%gy_FM%=Hrjml3~q$Eq0~nN=L&<lIC)7(}4~-@3$RJ7^z{@>Y-@fxbL5D@M@{fV*?Q-qy8>*E?&{qY@&YsX@(6eE>Z$UrFLj>QK|0%1?tdLtc5GbgvZZ>dLHtH*fYeXSp5R65qs=L8_5hs$^3r8jD6) z8++UYoSKHpSfv~^W=%bE(2WMRu%dwsdx5F4?Xw6XKh-i|$!>|?o)1hyCvthI9R zZ=yxH-gOhp3&Nb=wCkneI?iJvxNi<6(3#nnubODQ;O8zUu<4@2Jsa|Hx?KBA+?aWK z66^p??VP@{)u8il8e!%yVb>(sc^zsSbg!?z+|;AA-|Zijx4%4~c%S(}dlw-yT2R=D zL|1!Wm*{&fZxIDS2{3sSHx+`Npx*bp;`0@Dviz zsl2BD$n-ZW_4QKt*3@1DP7=yO?YOkE=#(LaAjf^ZSv)S#Ul3Xxipd0;E@;G{VHvmR zhGTnPWE^|`R?9N4tp`8XdyJK+uRPpn15Gj9gr~R9ydrVNrZv6chC0Jw z)whK_?^MD*8rKXBjGF|5UG~Xab7FM8ZEhx}F0*u$f8&VM2wFemxm2i3*}BA=;~V&j z-QD#I5us$!!7_iW%7DlLuZ5%jF7muJI5qE6C5NF{pi*s!^|9Y&- zGk(f6Y~_`HVGw#;08P%VLdOx?c`PjzH?g@BXj$$#7=WCQFSB0x_3B%DM|~k8<2=$t zdHIvYzqOg!an(;BDomW2t;UpU*S9?%+ktCR7e-1Sk$mOSI8~W4|Jp6Vx8ws+cFYSN z)JB$u?Y`|vF2N~HV}w7Lm0vi3>1Us-c2?N zUL9XjbJj=ZUjM$z=k^2fXnBxdq;b8OeaFj`-T(9AdE|Yp&gXT140(w6d-Lz^gx5F2 zm|iwed~jIU?;eO>&xx7b24#@>&pK8GwzIo{ZK*!9nr0gn>BzAMRQX(jXTB(6qT5C{ z(Lwwa@I}jcTMI$+gfMGQyIfq>^Bvr3R`aiu>D)fAw?0uM3A@f;)*YUvvR>jd0`B#n zXY?tbe@qxyrLru-vhNneANxfzj8SpxoVyq&=Xg*wh>!`gg8Rk&WCJhw*6xu}FzCl# zmP!IBJ5Zb5ZATjsKHksg50?3g+5Yzp9f@K5_k)5^+(E3 zhO$)-H#qfYUvj~#$i1GSyhfKDKTe%AGyUBq)c6FQSbP-@(`*Btr)C`X zRjD}=N~bKAN~A%fZgm40csHcphOUG?A4ncEPFzU*goQEKO9 zHGLoOHs>26p2vT3sa;oAz`&`o!d5HQ`p83&HS)p9=K3&)vV&jx6cH)T9-6`6g^J)C zj|tJ!I_&AKJ99R&&`8Px1iB7A)(EgmhG%+H6P_M~AwsQwC9@z)du!;Y|gcK77ICKquzL{8u+ z@2gxXV40T0b7f}LOQFrsHKK_FYgZoJKF`5#EeSlZNzE9@bC(T=0$*&#_`XE{`P0P| zo-`?{KN2u%`&X%Ptf_(uQVT}8mDN0bi1Q+XL@AJIJ{0@uyq}oZ^eyn-o0p`wSgYUh zSqk1yQs6*|v`A6a*k#6TMpd11B5QoZ0or9j@uRd=?BhgL%FCmnTUTeue5K;eL{Qlxw-%1%7@?^-4jdWKXxyY) zEah}i54xlQ;%2tngl)8&ML^|Z@p=`qEng}B?#+whYoSVMFJj&PFycoT&NR{}EZ*|+ zRa67LH6u^!)8uprn1jPU-N}YBD}ReL>Z>;YRrF#gSa))^46iWG&sWbNN{C5S&a4sf z(&sX5jky6;G~07BE)(<8C7e78dQ_NvjLFr}YbzEX^f5p{wNfnA*T|sfeyFR^VV)UP z2X-@FHjzN4VY6=m&!DzKL0bx7EheNQ$WtSuFkoO1Jat~tu4lI}`YCQDpXIxc9qmO* zyU*%nW>Th{!7fotYvPA5THiQoGbCTB?bCwnr=eaif!t)=16!EtaEg~m8>d}mxv-MIH6Y(xa$`+qY_INV zsUvp2x`SMrs?gsc{uI`Jhj8#S@C~-dTuH@8;mpjljjMXp#NJWdO_QOCM+>v+1CyT~>G|3BMl0Nc^&qINaA%TL>^VR%=KQ+I>^TYLg_7#hp zTEvl(>gG zAHPo=CZ+yVU~N!{lP2Ks23SBbPMH?QqH zPfhxl1(&K?xgTSXe7s}5j(=F2>AS4kbB-s?9Ef4jA6ju(aZ^S_37x40*;6=pi0Tbs zsgzCGFq;O84Z@tWzh79-#1JqXavzYD`IP3^&=`Rn?YDkCT_AYfgQ3s^qN9()ZTre? z?C==z>|_-xR-{o-MNxd;17CD7>`MH4*#1I?Mw)pZZq{;S!bHx@vXQB>@K@mG4qXB% z_$_I*pK3X>=?NB%w}i{}7gO_5m1Xmh2=}Wpl)_xjx_UihsnaHgalQ_~+W*Jkto>}s ziIH+#h~KDS&~3vZ(=-$rnagaCr2#5llTy^AL6z$SgxZZBVEHz6 zV7^GCwn(<@zTd1WTb~SN0Y$A2>oVZ=!B{ej)c0*))Wk(HKw}<8+5Qb-CyTXW#MC*l;4ht+Xona=C_GaKxn0QX zhq?B*$@(B9ziv)e(1xO(P<;Zk7rkKF=OL}6l(Lv5_E8e;lVrVQ^8rSl9@}5}3Y4c+ zbgM&DqV9XYCL(M#uRWj8W^sR3^BC4%MUPV*%v?;=PSu~h%M)IZj0AJxQF0YJqW5Nh zP5FNHeCqQWU%+Nt{Z~M1Dp;)O>KY^KPAutUgi6b&8-KN9TI8?tE&{GxYDt~EAfQZ_ z=PC}kW00~K3jOZ)6dl<>QKFXMzA#Jo3oCZo%VeQoNLOC$o~rI!#*4#H`rFZ;0BYtE z`|ist?I$gEyu*eA?-M{)n4qgyrU5<4uG)VhNxpgF*$)M$nFUF)b8KyccYOLS7LQ;? zVvU74{WSBLYDnr;cifOh&qETJrzjfP&3unqd3Lkabx|UBmbTH_gzj!wf(lc1e7bIj zXrDMQI?y@r1mDU=m+NWQ$NNyh_POKfNG3E(FTo~r1m-~V9!7{Dn=f+WSdE28(~1M2 zXUi>E`O{~U1=#4U31Vztp`E&GBYofZeZwTvu@gP?%af-b%x$}=1?<=i`Mv!@@F5u57`^*fuq9Y zaFoIX@1`r_Pj~!yMawkVu5K@c8kjY{NN}c^rZC5I3tr+GAR*{cn{D_SR4kI}p7U)n z^*|Gy$BC7J?TSE<-zH)QiE9~xr6RhI2w6&w%88qYzCt9q0|I;h6W50-u}(`tak6SF zLc3B5Hmo7XD8l_S07QF_ruiI%YLZ%w>OYAInwy3q$PS{ur0$?sc`HjbmwT)kWs{ zOvQ+-`#8*wFCW&E3x}^)Dg}wp$Z8NGm`ovpY?mXxWggBO(t>YY8N@@%NAsf&A53UL zg#*_HGI~dADU?c%z@r54M-dsw=Y$DcoV>@M^R$dKze^XC`ejN`6rCH~f76|kS|=}X zJ&oPECq4m4&vgqAWJl=b!Qp<#bH4QuGTt`@FZO!6`ThK2&>){wUbO7%IhCrg*INj} z1a7@s!+N=)Qrb^W7A9VpPjFZac2_-|Ez&`%yfvb9MmeVYA(%Xp~-bu${ONC3HQe}IHAA79D#yAoM(BKK!%T*#HsZ*bg}N(NQ?c&D_N zv_Ig`Z|CO~ET&XJ@%r7jpR`U{v&RZk(&}sYf^DG#f}bq2r2dLe^j*@`;GaA*v;M1V#?37f0BaY>%svCGw*#=1UjH9u4rUhKO$r~I_r7zk%+ zR#_nzi3uF1$Ct`d{i}J-r$oM7QYl|#{_<_Q?-J`s6F-LY)92wDNP3V!0XG4hKvC`UqI9#F_`a&1=-Ay_&RdY#;2Lqz+SVF*E5dNC#i z;|Q)2FL4M*^)clJVgL1GxvTgCUaH-L^p7xf1)1nVbYv)G-MSy_b&e4tpa?ns&s`}W zE#j$m!Do@bV3Ns2#A|TJIQoLq!x)?3idYV!4pns+c>ID#zL&{?c&g#J!~z@$ySYav zNomOp^XqWp;RZVPM`+mE2Wr_zc?2#~K;gqAH%^Ed5AiDUq;k#klG{MF!DG3U*1C9#8b&A=q)w@0}M{a|V z?_&z0M@)w%9Bm=|1Yf9aLnO8lk&psRw!0%>-;))5NDSjVo!c4pPj9p~u?-1 zntO^JoVZv93no=3uuHyIhtku=Q;B-w=|T1I%CyBC`CHzcMkw?qXg5GB$wvZYvHvja zorNamj8BOFdJ!<{*_#U0I5*$q{L~av!qjrnFVwJ3ZT9-o0w}**XjFgMOsDgH$sP7_ z9NPhB0tR`^bkA~$pAIh8bSu545*g;AYJlP8xP`f7K985gkE`O?s~U8wx{$WA>qE=* zffCy4rF>7*9#8jcLf2bh#Fc`Z61vdiIFqx;oCFSPd%aIPyHt*sVl0Acvf(C1x%byx z>?k-7VM5JQ)x^pNXx~zN5e>yYgn!o*TPj=xNTzO!pUA+oD6Qb_v&*XP(#ApG%D;gR zrEYCVdhyPP#dL(_q3YIjT!Uzs9CwqW(Cmz%#wJgijv&jfE&s*D*koMAUc&OaxS%Cr z{gj3>De%G}P11BAe_ohX3Ir!m*{|4U`7p($%FnlwNr+qDAPvMq2R57}ep30A-PFYLqB7*m zKutmT7Vi6N_z1`jpOE>EsMvXWNl~S4@|ci!BzSyg2l8*bna~ zUtGv%-gXj;9c+(h>ZbCt#t zX!M$7L2+To%x%AoU>FAtf{Ueq$R3RECy?55vc=3Rv-1GjCNutRN5B@-J9M?Tp?W zw8eb%^Be6vKAgR=ZeUkVmduE#*?!5isD@$DQ`Z5X<%5AvPs^EF_w)_C zIB2*NjfU3FTZR=v11V(CO`l4W6UrczMXnaqKKnMUP<+n-HOA|dWyXxtCz-?Z0hghD zIeSa)cRu%_RLZrd(trX&g%)Lkgp~41rseb?f+%W|SSMHTXA+lCAiKj< zWIMwr{Ey1aEpYckHhNdMq=8~N6Ei2V15t!2qk8_7^+Oiz(Pf+2hH&F5%7TgJovwzl z-}xbNH)X6Nq)A3ZOvxm?c?G-knnHkr8Qv!Wh5#9&s4F^8{BobC=+dPHI+7 z!1MMuDzy8C&OmnA=Z4CHC$W!a8pSz|j?AW`91J<)fYuzSbVHqc(SyxK)E;`zLik|c zBwpvhVYZn-tie0Rp!Kin;f`prP2(Vp^6@2MgEx;UaGV{m%7z0W=)du*nk0n(3*=n8PI#kMg< zCTD57mK4Sjfj(f7lUL}4r(zx^sUM^99H$IIWO!CVhNxS!zS=FXTiQk74eExNRHj+0=;bDhx6?e2`PuP^8l zkC!VI)dnO?)r)03A5G!PGc&Rk^vSmmk40N$wCKyw&))dKWAp2UgJPt8QUUB47tM>5Ld(HjnfNcpA(mK^Y~uFD`RWw0Y}apb*Ty6xLIbRK zU2+^IA>BAR==Rf**Tc^+h;e8?D^JmpQuA|l(B&W z8f4b*X)KL)Nem|>nx{K3`r%xPgfx^n#Ua+|(p-T%l0pv?6&o9dt1UfU>Vfc|gWl!r z=kZ~+3MjCwT@Q822wQR7EzqczC$xI(n}S}K}gvvzeP#2NOw^zemL5XzSFvX=u_v+ za}WbY^71gbatiHGcj;sgfW^LH_;r0mJXM#c!xJ&2B8KM{FAHLhV6E-z?C9)Xz?)>N z{b?zl5#_PGRe{1~Bd@%?t+|H|VzK|g*4i_LsF80JCD#HGaqT4Y1*|Kek1PoZ)>6K4 z(0T9S=ncmE823(i3pN<=X%B6R2b};$x5v<_p@(vx$5BHvCYWqS^%JM_=$0YI{qK?+ z+Uk;BTA!&nhuXk+J>;#l0Y5S{8R%jji{D08sjLbtZB5c>r8H-b5&(yFN#-_r;MZTo zatG%RI+S)=|H*FuR)q28mJlH}x%z5Ef1bSsH0xkw|B}bmc9PrPc($1_dJ~#pWjj%b zGQ*cS=H|B6c}&cNZN1Y1s8F@5r4Dsz<(!yzT8=7fe60r(xyx0$dvH#G9^tANBcD?0 zX{dG-ZN!#TXOT&W!54OOHg0PlnrW?gi83h^yVL>jxK!UGG3~SAA7rqG^y9f zBJ?t^bBV4;H^oOOE7e}ms&|>`xofVEAZq@QwePYw#>Tv{8`QkU6M=Qh-%z>{t-7OA z`jd;epK&|f(wie=s1NW^NoyjXw8ueCogt+3$#%cQ#yL;PtYJw3_+mkK>$;p6#Qmeq z^DV$#O&gDAxl3x&M%LheMG(rSrQohgKI1s|c11CuEUP{U8h@>s3sw|*eSO#TXV~I5!Og+OvkFr<~Z)PA!>EPgfY>N_h>DLMQ1elRxQ7zt{wz~PIPK@ax9ugV;MQlIZ9L9JGq1c%y!Nq zzrYzlXvc4|w;7YY#7dEsrTi+2qXmpi0*jwK)UWk>%dK=Q*`U+JuZu^Coh2JF+hO@R?hgp!Td5WA2-s~447yAV3W zK)~CdTA&R5O8*iawK>AvNkJs{1*UcRE+E5^F2%t=x-jv>mZdz$O0)A&dP9ZN48ir) zV!^}hUUl-PFe7!7;LA|=& zzx_syo&j@ry>{oaoR#cpV!EL;;i^WTR4a6aREQ&9~Do>QVA9Nhz zb>nRL1Vf2$thk)m9fgQOZhgiaT9CPb>1~YfB2M_(vNH*_m8wzv@{Y~4s4nHPg1esT zyM<2v5w-E;GXQ$PwA{=`5siyLo#}}(MDGW&SC5c5p(s_^r#s23o!G z*sa0wxIH4vL52F~Tw>&XUB?zLBjm$H;saLg0ur5e;D&j^3#MU>=j#YYJ|rxCdE9}G zk^0o1VUU+ym3Z^+7H{|dO5)r86Fq~dY?ig9a%iJ<35!zHATCm^FH-DBpkV?3dIy?f z|6y@)L-k~Hg73G5&-RYy*8Szt=UzO`O)xpDt$wn0_8Tl;qBt=)ZQZUZt)@Cuax97D ztA3S*#e=B3NurHD7&Y-7#wbR|{Y4D(5tSyopKc@y0~SATxOnfYN_Yh+nvM;5 zW5`(|-;We#WcB)ITLg;Xrf%f9ZhSEhQz05(0?~jsC<5=Cn9tWcDp*#*js0{v`m?w9 z>PCxJQ-djXU}>t7#M~ zs2PHM&DPi)k)^~XYq)@ltEx(zg!mnLeU`0Sc<&cDbA(h&l8wGH}dIb_Z*zjN#qi8HKgj zr(ie{CE}+NNt_P(6#E`joV0SkXR{Hx7xz9HD4e|KNW|{Z*hfmQ!?>^a_^jvCVi&<0 zyf^cqvK$YMo;$-{JW@Ce|^rRoY|$6sW~-E4e( z!;k9_w6K2Ejv@{r<^POtr*qTbfJ<(l%h?DgDP6pTcXimYqNKzzRs-!{v>a?$V~-jO8-?{Na(5 zPk1}kol_G@HSu}V>mpi-C?CD=jm0~ADUc6-eP3P8Yf-0U39!&q@4JJ$_yn&d2Pm^gyqQJAs3 zL5q;u>j*uE$U+O6&LYhqw9lEQ@SWO&x%MVKDd+pw z<7G~1WNt`^mc`2cX)i_gF|n+*>vzMiqrDdSlq+3z4Ony7-i?)6GL5G~bb?aI#TcHh z?f!2@-^(qdOWP`c^m9j_lURMg$BYcrbbulJnWRQqktUVfN@0p&DT;g8!jgzCOS)*W z(8#Gy_X~y~A7Eb6d#+%l<`oRW=svW<^Py_m{tWi@D7k3DR-ozi<5XZ(6X)_@+9@nW zb}zv$&#!f1g_jNDfF4h}rP}` zM40{_!G^YJ{t=E|F|wqqAWA#3S4*wrp%|0|ub0;I7}v{`x!~*f%u;UKk98apM0$DK$t!2BH?6i)3*{5IDiRNC(IhGRZLw zqkzqc(ygoYHc$!Vm=t~2=U#Bas85|lpNQlM7*L!!cO8FCl(!0_(GTYnejcQ@w{5r_G6})`mQF`LQ!^WTPkXNM{HaG3N z0pN<3J4nl3Tbt)Rz?SAaIL`RR(RIOh3li3)f<-g=g{uaT4#F>jS=b%JML6}pnD@Z+ z>hVjIi8tQUnu0;xwbj($toMi-m~HXOU_{-16J@?g+J2R0Pn;vdI~SmYFRi2GFM|{6 zg)|5BTSI}~J9=77E%J(V@C|=53O911AC#WVc017gv^rSE`IPhJ=Xyx2(Of$-SET|t zXZL92Pj_x)C96%j*Zr<8isph*w-zEaO!m}!tu_PDhS{Phbj>UL0c$Eq?IBhNL>$wjbPkWZ* zasyd1yOtcjTpc62EayjN;wlhVY)KY zz-j#^NqfCqMiM}n>3v)4z3M=*fspYSZ<2!3-Re*s?`@B!m*O9e1u}N{#-Fqj7(PZQ zColBn2!hG!;dz!xD0_w5tlx`4CNp%x4wor|f58H7ofetGx42T+hzc{M%rU6%VfO7w zWlVB2>=I4k0=mSv?w)devVYYU5^1rm`Y|bx*}&kEsR>n01u;k-DMf8?VDC+k{?%4G zW&lz({7qM)g(-%kYoqPGbr{c&SMSv;S0SJSCE=UxI$EVj*mIoJ3QMD`rM)(0$Y3!+ zTy+AD9hu> zizKc=O-7MB)t$mWRPp?UnnTIY#Y$@ljPFIF^vwY{E62kp-#znv6h=Lgo66aHy$hO^qY9@ zbuOj1kTN~83?~C(rvp^i+9`LO*Xv9Uj3Ak_oPzLz_@tlR-|Bk0*F&QI5o6uEn=sl&{wmZIAq8CT1BxB9$VxRI~86K||SgV~y$;k32zI&otKcm4=9CNlhHm#Vt&1^85kmpTj5S z;$Vb>0@PAe=Wwyk^&iy}L*B!iU79x2lMYJ0a_OlcDH}E9?RYcNuNG^}Q8ELaYl4T4 zAkzKR$LrJ<0Dob~A02=9omCL}=XeTNZ@I%tASHUjyB{obNuv}Vg@HLtM%|J9Uhzb0 zRtb)Abg#T-Y>~0$2J4dSaQuiRmc=MS*GvaE_HaI}>GD# zj4|V&8nMZDS+BL``XXbuAei|jM%5i3g*yO6lj{EkMSp`KI)axh4DxNsS`9O`@{;J_ zIujWysgs?7-`%yIJ1J64cD?R7(t9j$^VGZFN7lAqyD1v@sD6EtqcZI3t`|`|%RNxv zE~z=G8(!`5;jNtmI+~DgNi-D8a>6Zqh4lju_a^(aUP?7wg-L3{{6>U2csy_opEy~X zh&%u_McQO5BYD%?zCGtrtEs>t;h>TEHBsfXbCS#x**l=@=e@M(ZbUn?Zk^T9e9c z1hz35jN>|X*(Vy6w2|~Db$`_QS$1bM`xnUx3l|q5_-(L9ir~Bu)&&nqi#P5(yifdK z3h0GAe>g74gZ%W$3N=KoKPMpW^KFupot2*e-J+Wc2 z!&mOexp|b)A^0FtkCzK3YIZCu7eCgmZbxU^JClA<^_S`P!Vz6{q+i-5i*O2wfg<*4 zxG8V9uaubOgIBMRob)Sg6uZepKpItGq{+a2t4rSZgK9a7+N9B;eb}%yEBbx;g6g5( zXA}dm+o9LWl;^6mZG0iyQASFdKlPImS-Ppa1TjO$wX3sLI4)Ku(Ep*-I+c;Pl9C zS0!*FN?>kaK7hR6#+k1BY<;2Yy2M^na;v_;RYpW%->AUwcCF&w8Ux?^Qj>>diVO!T^=EWGn$ko^30du0_6hj_I;fj%P0&D;+a=%1nj`ms37#BM~K&}wz7)E7ND(Mx8dnI0= z+#VHUYF@qlm1J6crB~jI=|Hs67V%QeuHl`Cp~GfYeI@I7(#R9P^^?pBnugQM!45h+ z{HQJa)((1Vt@s8DMax@tuND4vuTEBTe@~OqD9v6_1yvRvw{-UrN}*YQlc|<@Alohc zu9VpyuL{rYjqU;2N$(msLS(yl%_MKd&`#VT1LMtw^U9QR$FyVqQ0T>336c)cOq6JG zK8oiY@)zMU*OmMm)H=mq{y21Jxo0VAGRC5^nI_kCT}2XdZcil+*x?YMXcsw26YZ(P z3t!1|!i>AMMJ91pv$iO@FIZLk-}Aj+U(8WjL@9cWT< zc9RsRd>!}Ksb4iAn5`Wsy5O1J>sT!9^P)%huB#(|;RDaw=yr3nk6K+*&O&^o46%$U zvw5BSH#NUc2V44E_SNiVC$!?%*ipQsVamb8r=l{#LN zULTnVN1U+0Q?5zser{hUOJ!sDaZJh@yg>$qY8(;$Kb^b3&bhYQQP`Zi^BzLqq<;_D zPm0rOvD!*;9t@a_`c7ee^0k#Vjtd8!oCWzd^(eU4X0!E=^ztBXHV6d1E*Ir;P2LHW*B%GDb`?` z+dqrV`vr&-tS4B6j~OmT%Sjha1$h)9)^e;_if8QExuc=Eklaq*?OrR$2pt>Gx~%f( zLQ`V8s%cn3c23ozC1C;U-?#5S1>QCU)0Abgtm4Om4-5MHGrN^&K;Xh$WsPolmC3{F zR*&<94a3UMh1E7HIoJaOb!4$zcGM!PFX#8H$0Yy@F&;e* zBp+}3e?f@wG4Gw7@99?sGcy#)CMwJ2+47ZXH2d>UlIU8@Dk*v6jiz@}oU*}gkr(9& zEwfE|6P!kG#%E-n734Z4`X*ER4FV`dxSqN2X%Ij3U+o|xV?uw>{OA#mP$TH~=B>$? z$z0sACk2rhqvV>&0{c{n{6s?W#mfMl;}cRk50%HzkD}$33kf}%d4mS7;BmAx*JPrH z0Z(#nbPtJxj(eQVt-^{!KEB(5(~8|eh}k)6+9_dD)cr%0eq9X8+v497yWZywv8fM1 z$Gw1L2khx|rgj87?T5e4=KbeWJv4MRf%*7?9iZ)gg2Yz<&6Sq0c*Qz&;bM*Z4bN_miI?G(BVJ9BRZYzpMc2uO%R)+@=0}HJi*iRv7Gm&<(lmS6pw~s4g(x#&pB)j(MyKBI+~$mnh(f+QXWd)y$JtsU zG4ac9&CZ-^E&r>zKUuzyrw|d#;13OYmD0^nCXddTUV5Er)xz=HpB-9%Iy(&L9=BmU z2Q>EeNj@a}91soZ!J+4{TkE`3^9^GzMQog(F(Bk~d9*L(ybZ!soz#3Ymx9og^ag1| zv%3)mpx$?zq%JVFBVH|QP`)^bI&~G!WvUYDR&&9uu$PMGmV67+r;52dMmV*CrqI4iY z)#%szFL+9PTM*6J>5FCyqCs5IiLIXl>D-n1r@gnQ9z^sv!A{=b;YAM5TT}1r=uIuV z`9pJuv{RDbo_!q~&r7+{+CkYP*Ok*>!eecWXyWX1j%|?8`*LHrX7*6b?)PTI^_>jU zdhgB1QZBlu>2&kgBklDm2DORa4zUI_9!}LZRXlvRxSeof)R^T=A*`rT<4;^xY6XT zX387=a~&S_9v^uxj1F;c_~9)JYt6WK8^ezBgr#atp+v=tKsFu=074aEa(|C=<4D;i zRwtkP4bPBbaZ^PX@%)-v7@Ptjo5eZ)_e35dG%(W zXx(sc_LF?zcjeOY++K9nR3o<6W`ps&bedoA^g>)7Ad~C{~`02G-1yN*WFFZB-=K1 ztW;5?ucX!vqnF_4jHnERK_YoiD+Q0O**lAWDf_D&2SY@0SLKPyw z(=W7T;)!>3O1cV7Y^sMEj1;} zyg{S2FcRxEG>^>judFw?v2U?bZEg^BZ~IL}7Sftc8}w?P`&C3f>j2<5V*je>*4R_Y zZ&Q(^YdbFXH3`>bSCGED4tK%S&BX1yS~d1DE39h#ur0-lWUCN!=EwK;t-}C++?sys zck4QH3?7C75Oa>We zs#}lIf1u-D-W}vQ-%K^YYaSdu|LyTVrvv^o7zjWG#{(*ItL%X_+R+t{K15YxYGELd zUX9oN4c%(fF=i8%>gu5`+s#}hyqPETed8K^X#}f!RhQYh?+o`ti@W^Xa@l8z5x?%Z z^|sp=kh;EK#m*ql!w~+fTG>sn+S1a}&s8hDJ|dd}{&bp6mLKuDS{gE0Nx3SA60bxW zGDSA)bTY-|4bKDJ?2GE1?5<=l?=M>#HY*{xTK{6ytWb$;mKl)t_V%CIey%@RImy&E z8rXy9cjMJ?KPn&b;L!&iJ256qs>7g0sR} z#}})O{p4zo&b2Ge!-iEW@#5dAIMbdbygsm@gFm~%xb_wvPq!Z3#=eB%OieFZ#6Dv7 zXT*^iip7L&TwB|ZZO(`k*Ho&I7M!V33MRl1o~$yfi|tbx?K(}wUZQC1n4C{7{QR8Sg^ z;a^WwAkap;Wi+F^+-;9N4O?JUQ|+PnfY&0%)+PQGCtgV3>$G|bFm%42mSBAGpd z&sm=n3-&lmJXrr5x+t~es&n!LjQxhW2hBE{CrRb}!v(pC)r`ZcaHywE=7Pk>fyUf_ zSxcKrjH!Bi%GXCrOGLbqPdXriVkvf`9{NDeEE?TJ{IwH>Zt=RR5^mPhb5GNt;(BHl z#B(~(UaX?93G3s&=Ne+ogzKO#A0sa(r&E1h5NutyHPRoeQ@>vjkxV-4Y&FZsz{V)> zGsI10iE`dFg0&Q<W5Pcdw#*Q<o_iT4hPfy783%kX?5Qu-gVb2oE-m8CU z;UN6{*-`;Lyj77h?W}Q3eCXl%7pXbGY|!#ziT;+W`>P`U&yosA8t(&J+X^OwZ&-a1 z!G#kP|Mcm9<;kOHYEzR)1Alfo2T#Wm?|L#N(y&^-aqj{CvS5J`>oswIa z>~XXFAK>v{pZxUnf)Yw82(q^nbq(%MQgI87Y&tLCe*WshTOY*Jq-75<1Heee4-I(R z^|eP2Su6i>^8Fv*K!ytAHonB$8xIK1DhF8%j#$+i@B(tq8-Uh~c{L~Eg!_mna~a3?VVnF#Lyg_t-QwcnXlbe?TAYU$v{e6cD}qB>L*H{# zP*C{$`#*_ew=gp~(4KDXA2g+8D&po%?q->K+1W*wW@-Q13Mx{5TMIsd=nz45H}?sf zAJ8!UnqW<)FFP?A>yqh>oz&L z<_hCA>&uZ*4mktOfxxU0>|3vs^#oyaF)H&#@SZnxbTu*<8xD$Q=7la=2ZUwz#t`Xc zR~-%eNiW&mbI?2?3GburYfK)GRPfG8;j>l?*c~U-6?ZoxoUrqO#-O#c6DhCKHX&qy z-BTFZ+UcF))-|iezlgb_*Wn-#v9ohY`?(w_CbeI3PMmJ76=j+1BrATSU}igK!%uWO z?SEGA@u?J()RVgApoWmSVKs;wYYb-AGIzS=n7v(7yR3;g`Sm9Wy{YSTD_-3?H^RaknrNl;m~echXu~i7;lbg=h zj0wtTy^Y@V(@QP_zP)(S*--`UaspP%vR5>ox@ z=}lDTdb$x25rJpjXlrLzF-)a^f{a|E({$4cjq;=fPPixYd`U3X@~ThCt@-57htlSo zo$K_i-CaBa0-YAGdtlbchHATo6sD-{g6n3rMzPMSFEZEGR%X~izixOEAjEh&$p2V1 zDt8O8NaWobxD-+Wu2}$qH*ov{P-oOxXV8ZK`%LUVvhA|VS8Ebgr+N2)9N5|8;DUOq z`sX`OHBkYf2C$}(ln@#!Sg^bdT=uZRZ=s;e{mFvXLRkPw83xT%qyY>TUH5b{!3i}P z;B`Nk2Seu9-T6IYIPED?M;8GtD+156VHhU2N}`q&mz6;61_#rzykS9j^6B&LBK67v z?n54HM>efu4c3`ddR;!eUr{gM22wpQrV;5IQLRmR-7oe4HRd0$F<1KY&Pa%$o_&NT zz@WeNKzvITe*OS_aE|xm1D39bEg)&W=ziE_Ia_1YB%|3DPAqIOg{4;Yb@=Z3)H6bT zTBP|Azyv}$H{xI%Om^-8Mywp)^;)cs8+C^i*N1*39W>OJ&pvv-Kp{vC_F9rh?gAR7 z)Ajb75v1Z2*=v9>9x?$BM^>JoFWis45Ob`q0W2+>-=twY=&E$=$AzWMZh%glLDBw! zBIwo6Z_hr~=%A>HdGQRdjGDJ|5^pZ{Z!Z_UP0&`R$_><4=(ECopy92iE6B+NJG2q8 z=y(9)kI6t14x^_F6PZhr9|(_~jwk*i==sn8DG-DqLA*U3?zUe&N#L^N)4KkWG()7m z>t57F-(eF0gTgPj*Ws|y^|ZnHFy$@6tmY)3EX8Zkg=7tCd-hHMHz^$a3P0(4|NRMs zJ{a?ZKkS(gJVBhmX$-yY1BAZCFyZ+Aa`A8|UAQgy4V`)g3_sp-^;p5@T@dqt9$IIG z5u_q?eyHYD5*nf%vVA|VSU8IuAq>;^JJxy*pOzpy~y zar9e_5u@I0oMz880ELjxa)!kkiv8{Ye5g{YbKl22=-_OGd)kh|Z)VcmH^|R<;a%)%MSSVD(jUG*{$0H;6O4RkW|sd2|dqLbv=JX zI~1K^OOYOS6@m=p_jd*`X_Q~FqvGt1NwPB#DSGR{P;A3J&<>Q;((r&3EKp7F3cEmlg(0Uc~QS$*1!$X*3 zbvyg26$NvknDuYpo5l2Wy>e-VQ?K|R?@lLmy_*uT+D)_+wVbAn4oX_Fv)0mu0~)|( zWpujD9=|ALwbF6&8vwUHtW8rXw*)wqWfj4`dFMr01s`x5sXV}|I&8ij(8fj69a-%P z;l5t`)n0k{t0}H_mDXs!!SPB?Ftj__`Qh$bI)+xg!C{+3G>EKFauyKG20QKFobMvU zF$KT-_$MJ`Oemzcpi>jsa~YL+Nf7=x8Bm%IV2JsY_lejVm%T5r!R_3P{>|QvDWyP0 zs)kiL=etVNQ3{SxFN&8hV%Xa2W#XASgRurF1~?~JKa)MTi-KR0g!nKt)D#zU)ozil>k+{(0PKwr$4cmo; z!$ii=3~aUB6gRV_5*boc%~G8vt?hUa%T?AdIjQ&8cAEpq9NS_Zz?{SJQs9NjHf;#N zN(4{J*M3Ber3Q4mFfyT-O$MqWngsw8MO){z=V)%$_DOuTjy>!yPi8m)EK9BFXjhiF z`Nk69kAD>%Z+(0z?UdMsCy$-%wlkLRLiPwa7f)~>1+rR6-OB5Uq#_``>V5ab3*5yH zU2XM&M(8j@2;qOYTAc+Xdqp>8`>0>O9Qb0Lh45?rY*b$Sx3jaWLD@kntNFX4#t8iP zB9DN_aVN!m_l&#m5rt6{a`VS(_=p_kSwMJ<%`_OtNcv~wN1f=wVoOW#!(QcJms7OM z8nZLKBI|cnr|ttfc4Oe`2)(Y-@#Om)C$(cSZ-?B@cZi$n8#qUp@9jFdHNKakPhy<; zzrj=EK2EY9K(H`WA&F3je}n6==y_?R(F3?%wXYF`J4eN}3&28-%T!Lm{2TxY_-)Ow zk&A~ACQfSE_rqdoeJaxv1b<-oCV9v_$g?`@cs{O*r|x<%SMSGIdO@zo9J9s+cv5eG z&g`RZL5%;&7wYBZMxu2W^31_Z?aRIfLhn0UXof~|Q2)^?QsZt3xK3tHDdO~9<~Hi# z<}w!68ng+|Pyrx*EjVI-H9Y)`3zjf~dRY^6n^g}W>+J$eWo94}=vB{%)6GHgvoq0( z>1=5%40XV%gAC_k2tAGjJI2u%+d5q)KE7<9Rr4>mT4}{@2#W%_<{t)B?*X?1x>C5H_9KdqI(n$cu-HOK2Df{mz%h0eV1hC1Aa^WjCqBB zc$F12&Rm9y8bPgX=HgO!HX;q3Eo{B=+ZW!QT^2hP#c^kh@yn`xJ7z_Bflis8kL<=f zeS;jCc>VT;7l1mIXA|SGzv=A9+BtO95;Xx5uC$*zSTQ|u3ShWxH0jnL9EeJcBGOr} z-s|6435I~@vFc#*>_weB{_`D&J~!N2^@_qOpYyD|$O`(DNy^6_`;zvx%w90rD(~f- zFaBZhV7V+u3o!VL7}4Fo){l9)sB&P1;hOGUs09#o26#?oCM{Cqo+#?Tj(g3ct#AbW z!!*hKDTne=Z_~_cdM@ne3ln`VN+DLPwbE8$vmYjUGy8dCu8G2UccR$m!71Xp(ThQ& z>g%#SwRwdBJZ_;OUYNz@ZfVPmto4iSX>h%DdwfMup{ts!iKsK1qwQBAL!(xGJiJIE zeP3H|w7MQc2`Ae1<4Uy1)>=gOO+QByr7>X?-nZ7Io|Zj-Hh^?HqU^w8r}OR_2pGR# zFHC}7W76f(6a zYv@T6zfRK@ZFX(?U`MG$O{2P-a4l4uPfA78*n-rkL)SO-&vE%7K$>VZl;)H8O)J&w zCLhn*z$VJ$YB|85Kg1br&erd0;tEhy3=u3Dq?n9Zg%iTHz*v3T=Kqu&10SfOVlh)i zEv=&83(*rxy z4jwn@GloaPUOMOm;RfI>xZI7b*iA;KD5z4(CAo|#h`VCkwtY}7L2@_)-32Ye>8>=E za~k%b-YLLbz^XPn&)JFto}$UUoLj6yMB`j+<_HL@A#~fULMB*z_(P1AxVlR%rebxW<20jkbyAzx6T28Sj3d@S3W z+JR~S8c1MWfgGNsU?uXqw`~Kz4^& zS3zFz0X#QSLB9bcRw(!!>C<(>zjbB{xf6{UEZP-}*#z_ozw*DXu*TBo+Sq@xqTyP; z&o+<)YNmd#+Wu^>BdHsWcG-ETy;OyRR{ZBnu21)$7ps?h%!>=|y z6uPf-Ai!cnS!uq(n{*KXIp(_$FIT+MYMpD}bv{?JW%2op+0~&1PXgQZSO~8UOSKW3 z77PaD=~Zeh)Y%DG-$3T*o3Gkx5zEmLZZm@EPC88*0&%L%HTO3X0G<~hqp{g#g6ms1 zU121x%Oq2L+$XJj%~VCQB$YRy5BQ=;5;nYrs)}>6>zaOy-7WBT0`L)7(x) zKR{8ePs64JW8vZcmXu;+FqOZoWTi8hw6Ou%tO+s%Ys?Z+j*Ai5x%b*(%f|#7|AR>a z`!WnIOTc(#Vlw$s8NRchd z=+Pq`CfFrfDO8Qp&iZVBFS3umnv?ERt(fmfzW_F%H9O{W*gy|7j9tLl9FR!}OvGiv z(Xj_8cIn&tsQ>KnWEikiKkJKqA*?($qUM<1)zH994%6`dC^~7E4KR+KU%1;)%0OX& zTTY*wWG4lb2~8=AHn-nAtjS&U^brnXCV;uBTf2wpK`R4oj>I7ZA()nFW`~6la?2QV zSlhE&jV|TxTjhJ9TjsFTF`ekz+pa@tf-bS4B^rm)$o!J%g7=4^qn;4@u=2^br7hjF3v4?~p zI9!6lu)Nr0*#vcJ2O~^R(waT&{y!ztC*zE zrE7&xjtYceza(Wv>=EIqy{a{0FpNWoxU%)48shlvwx4~G;h6sub4WJ&3K2=xk9u1y zc>TSpaw`ZUnS4gu!Rf6#^;72sc_L1#(%-pgt%3}c&j$R#!>Np)xnyJo{1j?@+xIR_{4&rU$pGsj{D`Uj@w2 zfdGsnp!O9`|8){5jM7H{VkFfXcm!ChetlbO5JtcNkW)DB3ccx2g&rW8)0W71OqMOocd_0r;qvu<4pzLUMk% zDqsT3?E&3p&^vgF94Z+lK4wLy<&6pol9I!$_R-g6=wvyn%@@8vx5Fa<` zSh{|J0U(|cs$q>(I&2Cs1^w^g*@s79DQ#@K9e`EKIGkpEm+imtr{7loB&kDusel)A}ktZ+aM=rw3*4?g?UJhbD2%gCB;U^TI>(P+Pm^< z&dgRfyrkD({Ouz<`5U>5$Z({cDM%+Z0Q9y^dgz&g3DrkS3Rs=ZKnzMaA&aT9^xb1U zz~X^mC#iQ+B)(GG{t@O>Ch0x+xC03Q6k}!sTF-nU@PNayRi;WyC2s?1)ynkHHQShc ztxP43tPEX{tfD3X;7HoH3b(Ezl^zF?^4$lOKCJj(iFIbZy*n1#px8Bm@w!<6sI4t# zVK@T76!`=eYMWRR(Pt1VJ60cM+d7}s4IpfApc>s?>}zl1By-(*D0u+TL9DJ(*j5e~*JB$L~w5?+m zZ>$XU3>@pDOc72q!W^3T1jhYy6}veHb@q%~{d9|L*#ReAt{op_uFb@j8AWbmsjKxpgpHZ%$ks>GYiKSq$4 z^ZmB&ww?%5&j^bZ_aC9ha=9WfHm0Ne@Df#Dt~U-4bnBKRPmvdR(Z>Xra>-9g$N z7-$64b$viVR0q!JAlerl+s=rJ>13e-eblUR&^CPphEq*li&ecf&hDm6acP@GU(-;r zWl>iD7P`7#iEI3e)DD0;Z<)GgL@-6qoG(`-N!3-VhQ1DGbfyXVdQ{7#@+F_LosTOH zNIK()-7s0sRKdN$z2d^G3lw-7P(qA)dtpa~!oA0!n>D@W@i5u)fLZr}A@WzTQm))^ zn7bbFGiLe{O@bQgR>w`O1^oDMtF5H(n{P6h!gn8g1=LH_s4D;#?FoRMa0fw{(?Bhy zB+olO&;6y%Pk<2D<*{{Q+>s7s>;|Bd%|~sBR_62dAw2&7tp(6FA9uOnK1kWtU~0!i zF+_%Qj23`hj<4gM7(jx*)f(#q1A*VhsZ^gNZU>+wPAax|fW_MAv_FGJDpq&Q!laL2 zrNd4U-gUIx4uF%`!v=8e!~nc^yfc<`ktBiU=us;q$6zYXFn=n3magxBDNM2BBD!wds|zN?{d*z^gkUJ4S8|4Oiruq}iVSUg8&+FD-;i7+wNNR{&YhwRYV*1Z7e68Ad61btcA7XjTi2_4|k^ncA@wd$6Q% z^=#@cF9?78py%#pkvAqCH9dJ>a^syhEKMl1JHTGv8qS1wI=<+pFyW)I)Vm=Q%Wwu` zp{+yX(6cMhw9U-Rs+4Het38O>%n)qxz{B1FAR#baCm8D7sH6_4+!=48U~46xz2K)k zpy(!M#|!A;_#?l^4VE}uXeu_>ON0}L#}1#7#mm!e;X_ZK2Vo1uaRZPL6b31L!L>mG?+LqKx%?ie z@~d5NgM9WY|8?%_!g1Vz0UfMA&&h>d2f^cXKG5pxYht7TeI?*XTFL=o3;ST}g9DGL zDqK&E*%z>cbyHk7aCFHMQ7>i$!3dFGUcY#vIAiJC81kmw4-tSzy0GdoEq6xBd!(zt-OB?C9WZl)M=Z37nb&v$pK zux2e64`8fCrNKue4zn?;qWWXS5MI{Z`TCgRv@0O_e*#P=f0UGFs3k_>NSkzH2o(WC zTrv?2xc7n%Uq$5Il{TJ80dCn7mPFs{{(KS`w$|c#Yy9;cd+N*rnv3`JNH#e@NhfVkvC; zg3$m}+J`NVd_Y1yGDc-uV>$bdTs_Bh=!bR7{RMzpEy_yX9!~}1!C#E9lJ>8@f`uot z-{`X)#qd2f3XK2<|A1e{_=&cw`qqQY>&|Y;9aqW9tJ1i?O__(Gi8^Uj5dijTGibX6 zpzW;z1%RpPN|RMzSkq|2{SK-WC%d#`r+c7r1UCUux1D#K#If>0h$NT>QmIhYCCLW> zB=iL40|E^p&j952s2--X4UW6mLM{N7ZomrNb5Mr5OUIP>yI;r$m}weL&&xBMZ!jIn ze|OYwNF1V>@I0J|=mRaBGaGRBvUmsRgT9gskJkRGvv9s;p&;|v0^q~tF4c@|M?fL3$GW&d5=l>5Xy{^9rwYk@vXYT~4Ln zO8t!etoU`vpEjkWSx02f5H;KV@<4qPMES1K$aeQYrx0 z|8CWm7z79fT1#}I&$X4;UPI*@ss1!&;G=_bH2F%WRmbqjnF7m-vEn=TM1fn>^~J1p zx7|iC@n+>9UvN@vbb+PL9EJgNXqW-suh%c{DByAVy2sh>j5|h>so_bDkm4&3l7Yd^ zz+A0ID96n@4ZKJWY8D%^$mlPT;Tt~=+sctrertE*F4udwbA}?-UOAX_`Egi7JDO|^@1*!x;j*UOm}oUESY8a1Y$m$aD9hSlyK{MzYj!AHS~|~a;%%LDQvKWbdFa# zp`ArRB2wJjS-+q#=ryXSVlWU!Rxy@Jzz6z9Ev=2iq{rNhUx$d^9syX7*kCj?HL>ql z5B+k3P0A1e6d&zzNImt3b|y;+Q)mt8Z#XUn_)3w9w=rg;9dX0gc;t&NiZiA{n6cM>0-S;@NtYB?pi@0(uLmGkM<{(g zf(U0*2eLBm3^o^CQw-9Pa3w=uF{qw}kN&`*N2OGfoTeY?^SK_>Z(eMV!hd%Df9$TN8;^BA|kpjTH?*pbuY<(ct22}gZ;`+=?gJ03km9?r^lBk+f2p;Ch`GU(&Q=NFiM z$%`)QJ>gCwQAi$^_n|I>_A|c{?LeLfbAbpF5w3Z3pzI^>XsXwq@D@F-&7DzA7?AVE z(-x-;vTonILe9u!rtLcUJ;w+6ZTLXFM~b$UH`zGFJvVtv)o=)2naQwF+-$iXOJt+`@emv(63iaX1ds)?e%pfa!^$A zs?)GVxcQW;9A8w$?8uM$!r$-L@mJSg*P9~Ali}Eq0f=aSV!EL6MJ6piyE)s%87+JN z4~GfTYjj$gJyDc6uc&`=a{e{WAVc#ul052Tq4+&b0jmPzTD*6Ws zCItNpenM}oi{uhJ@}jcoe5KLiN2C#>(~+W{Pv!sH-<(s7EBMRN_+ubqDe6UoP$lAj zaPVr)m9b z{Cx<*@+jX~qyN!Ve{t*5BuI)AW*&9w|0tsQ>&5_XT2x!XObjIVt&% ztKlyaHxvc^y`S2%teStK_WwQp<#ob2#s4+f|25cu+Y0|*R~SXMG?mrs&=f`cWZB5c^vVE26+$VHc%>3OP}X|Mjx6+{DkvtNc{Ju z;I8;-?Qfsa5p(8KF3=(*v~rdcgGay!4jXu+MEXloxU+ z9!duk|6>*ikpg0n*bNQ)^-m`G7XnJ#BM>E9e?Ye4pZK1Cp_8~Y0$l&H`RD%)H-8c# zV6((ie1`0Q!@B?bV*if~|9@<;&@insQC|JphxktGss(ZP#iGtfs8NC8r1h*40b2sM zygnkiIR5&VcFW0+dPu;AXp+$wSb1g5pTS;lgZzW;40`p+&oD{c+SE7_0|!jjXt9u{fYil3K+ND~6D~`1iKH zq(;UGyuAA!rK}npoZVr+KNU;8ya3Y8zrVvly7>6lTO_HOf@@UgJ0veR)an@|m~$## z{wHk{ddQTpKXn$Er*h*a17^;)0(p&42TJC(n9im_)SPYU32{Rym_BrBg7_kd0&ah3 zr?=ma%;f}Y+?8;$n@p9=^}RT7Z(=M}L#c`Hy1qF;RX1#j<-hNG9`{RY=h`|Idx%?I zb#cIean~7HdCsY^rFL&r)(T)rI z)s@&={AD_Z-`_QPOUtx6{uJy|j$`uTmmN!1VDMhB74;u5Bm0JQ&3AHk?uHFD(66fM z!Ouo#Lj?55cjTdO@za7t{4~yux%FC)#vGR9)T>f_ z!=A|!2)Bu1_ZBQ|N1x8+2)`*b{tq^H1>-JkCLy?w}z2t$A3 zb}jF8*8(Xabw31e96yvavEF=rQ%EgZd}Fa&PxFdb?&VJJO6}sOGQ&xSmRX5rx$U2I zZ^OSzW8+^t9-5oP_7+#g*yity{8D|rK#hKXv%Bgn{B*C(&BjI8|FsLrz#HaSMEpXb zZLc7uM1WPzbfB^m0Fw$@&#I_9`(mA{*;a3#`}g*grU{*?_p>&7TNS+9HMePTwXZYa z@yq10Zzpw_Sc-usR@kjOtK#^nZEY<;*pKU-VN=)C_|L=yIpVPb1E#-uvQa33R`SSR8~dyomzP%k(JZ#B z$76g+TgZ;*(>sNSQya`?z%D&&_hPKEIQ#Vj*%;JHIE4YUuu`wu%nmv63pT64b zOwB|d^-jp~uHfUO2<+Q(BJ|b{TB<6BUAGIW`W@`-7VThM@Kdz>X*oH&B0o7j9@yCM zE)iLWTI*^p3{Q`jX0G5%8a*%H+DxT!CYjYYhT`T$!{KqMr@7~JKvHJoHqkfb2D&F} zVUj z54k?etV2pTGz90Hu&iy^yd(4mUKO2T7R8SSOsIp4p#xV6};xoCrwmX28E zidsYf9!`-;z5bs9FSvbU%7Z1HdDY#W33dkZ0S3kAqDS^KWf_GK6#ZtFEt3pcCJ`Ae zSTraKbSl5u3ctPu=}UTV@91>oSzI=hXtmhYC)xU1-rqJ_^}JefT=I&<6Z zn~DfA2?xCcYQNoM+AXLuA4t8R= z^RKksF%-SsHkZSs8od)}F+iR=yR8VGVGI7XcTDP38u5N&*YlT|kwin4!DUtFQzfwS zn3zjfb?Qt~^jw2&JA}))l~YM68C8x3am$0+WP0RS5$`f|!q%wXZAfO)z)H*dk%cti z88yf|dx>SbIEjNc7N3Q3al;FkOj+x1G%-7-6{N6{f=UzIjpB6fHGlUoHNuZkZb+lT)kspdCnHl+#vbikw2`t5OXMU2|ZyFZXs;5E#X@&uVkLZ^ro z=MB#hyTr7mX!3VQ(B}&KbSeq|SF?fqRyXwnbJ12+S7RxJMT$N_rg}UCXa#%(eIEo~ zV|HfmnQ;BGnmL@!wu>x+T*?Eupl$`sLacYjOG}MGS80?go6y(>2L)^&=8iwVvi_{a zTkWJXL6`8Ax&8*Zc-PLF3-E*ENR)?PfjUCN{=?}zSflprhk3jIRKZ4%} z=Vb)=k;`U8?Oz4+g+tIMUL2D!OuUw$2`JaJNiImX-pLHoQoZ&V%Y-$hr)C0Wj0MGL6U~9>f5C1HzoW3+?H6J`~ zL@L91{9WV?OLET3uaKS-l8-|467=>bL52m6AreB!6uD?BqAS>C?``(pcZ6XNd;f;X z;tWw6cU(r`uYDoCyn87I6cGn4ju4MGx++cJTsT^L+SEMp`Kxvw+dOsn6tJGmq@_+A zf0)JwRWIFf*XqP)fZixsXo|m*#Er2s%)h_U?QVPe`s|ogH(;JCkN34(_dcg?iaRIG z2#w?6aw;24K@%a|7mlyGCFZgl>p8!KDw8vfn#}J;3jiL`B{;~#MXVw}HCd4=UuFaB z+XT~CvY*10AuhL>;OSFB@&=0^g0>r)78?COvupFHcu(fk8oc4!iRI@vzF0q_Gwg6a z2eXCxNi$&948!rN=>=ZU`HP-ePxv7a7w0s&@@sjo`ZgZy$>ZB>PBigwFd)8&@*ur_ zlfE_$5OjUEI;KqQJXYzmJ?~xRFcFYQ@h7p}@~}F#Vg~F(UEl@C#X2DgghLh^9K>bg zMaw_0sBm;Gi%jvf9{$_bjUL!v1 z&Di=G-gJ&?m5&{deN9!e7ydLq2ge~3(hL%~MO9^ltT1D9X9Z}#?&UdhTKeU)Z0%$^ z2+z|%BzTtSpe}B2i=j(wV9>v{nUxSf=a~@q1v_Dx4k*J&ipLz96a47u)w++EjtqlQ zowY*|YIndB#Wcg_N>*Qd;CkaYPcl+svFpQI@@uk|Qiqi`T8UsGYdCz2jWoKpfGiC& zC0E@^?*&lxd2rvTK=;Ee_0B2x#`=evEj(S2*#NTz5ve+vA=wJ#W7!9~h20J@HS2jc z4y|$;u1QhYkdwi93iwr~D-B@Nw~xogZPzZg%0G}Jo*}C{3#ICjhxx=Shd&mpN)G2M zf)WJsgdW6I40L{}WoEQM_NFp=tw>7^!kw%9Bkb4^NbzozFr!@WB5AjwiVBfW9&)-F z!BAFJ$vKr!DgwswP(6N!h-P8{7`R7eRhNupgw$8_B*TDsnVV z>Z!+|PVk_5kKYNyH?akzShqtW#C=I4?+({9%iv&9RN3-!Up#E43n`lm&Y9vdJ9cBy zBBs)tK2MQr85lKkzY0-AF~6W}^>%Q=lCgY52Gee+q-s;kn8 zn0Js&8~pquQcnP|pqmjS^2VvO=w_1Piwvp}&+$@?#r)@4gII-eiM#1Xz^Ael_~}~> zPKPuZ)y?vq`)=ZO?@sPU?FA?YlxNRimZM+EIb*$`x%1LuXa8hNjy$arBJa%-2{l_& zQE0QmYk&&bidIgH8D|YIBZi|+_l3hmdavS*tjM8)OE^bUjn=t+!)s&9o9Rl^IP?70# z^2THRCclFvPO+iB!1|AqpIMY(EAehkQnhN&a$Q-{(mrw;{j@LXfdz7F$1b-%ssTu1 zqc@yb=d{Aw9x849Vea(1MG*CMjq}g+4p~)(5R+A>X@Q~k)fzH9Zw+9EGN3z)Q&RoA zA?um8(_-BbP74kx}00| zGwJF_2t9po-^dYxA*O@0zl>nNzfcc@u#-3V8F9Ch6t9252{LNk_?1chUxTo;r2B9>OP^SNd;g+L8xl}*n!IR?5WZtN&Hgv< zRH9CkB%i>)V(OM$_SpByv*gE%9+NklY-0>N5*PdP;F!G5xE_s9P%6=tS}ejWlIiqsKm;gQ+p z#d+1MO%{4u0F0}nmeJ7faqzw>4%&DX^TWkQr^OZtg16JX9v-Vopg8E$I%FspRgJvm00PPGl8sU3$kkwNeg=za#}vIY&#U|XilrIWO|lHAOTdFoXVy>+ za}gM#Zs&3NdKTS1E}s)|<5T~n@5XbcXp$Ya)`L25`qf?PIg1_w!uxxoGOlxe@A|`jWgE#lKMgn~ zTVlCo-`!6vXJ+31-d*15bHNZ7^I*z(xP(<6wjZrvWS8|BsDCe4yjYD7vKUQ$e_VxN zO0uCx`)RaEb|Y6oZCCHC6elj9?jo+`Q=>aQRmJ+^y3kNCbZw*SGgM+5Tzrg|OD?Sx z<{Na z6SL3$^7QCZlbPUbFfMYnxl0Z6J2&r}91JL*jc^-?S~*)x6xZ^>=nm`FG8wB@!%@-5 z5a2VlSV3Hb?#BD_Zpynh@C`w_)JyJOqwL7#%0P7rNCOHoj6FG>@al=(&baP|IV%c9xUN0Ws(+521sn*)@8be|N@zy?(DtzrYe#^xORtwH|M zVeAD@|1^WT`Lah(spWd)6<7HwZPYGlWAd`(OKW%D2iY%umh~DP`q*tA&yl&-ic}Ll z(f*X{n?ah+$2zv!t73f)$+8#Qim&vg!kx8QXMymLpzmAhk`acECX&l zpWpm`YPr17rD=TehG@;axyV359EZlbxgL_?tuSuWL^nos*7e*lk9-b-`bu`N zwppdiZ`=$&@pk1v3?(RGhDGb=Q~ePf`hKL@Sj9L6CR9K2{dcBnWsP|_|BmOzQM>gb zoqVY^cM`R-7@jo2wRyZI!0iYSJnsY_$-zaBQOUqExwetKNmh5`D|E&FKJ){dMP6kwxG+qkPx*8AETA{(PM-_N9bHxo~Xlv}8*6RO$85I(21<|-S!9kcF- zZc3Homcq{0{YT-wQocSgPX%j43)5CS_8!qOi*h9X;XDTdt$9` z8z1Fw&SCPi1z%3w98s=iGe>cU2(*S!lmHl9^LDW!rw#QD9LmGWyfM}-r1*FEfvC!! z`cg`814p*EnP&dYU~OiyT%wIk%0Fffweo$cZQ-{zSqM#IPurO#z@#Y?qlVPcmDx$S zcCUlyV%h_eKx*L?bd`NV>B1poQ}wP9+@>CAl4@=K6^*g>D*W6z>y3UB2LQhw32u$o z2A>y}JmU|{4uhSHGH^18&7tc)^a5O}W5xtTJN@|b*JpYoONY%|AuopOt6~*>3@uOk ztT>8c{H|`aG$C2<4;?x(YnD~lqPzEr6iDxDcW8GUyVfT6#{;Y;9peU7nFXp_5EqS; z#M6|VF*yZtoLbGEecgw$l|RbSlIYNtn8tNKi%<-EMUUJl+%7ztDP=juS7QpZ*=X+K zuNUv{BRQsEYKJ$kb5294tCMRS?ELT-PL$Id>s$C=UE7}1Vd+oyt031|OuN-td_<+& zvqOt#9uUBx@|XOYhz<${F;1h_Moyw1+X3)d5;%~Tb)9`}J||*`NN0<48!xpS7gliZ zeH@|6xl-Wd!DtfGTltF`zRRAzZ;m$LeQDlFo=5)1-MyvV#q>6)Q7l4lQF8>8ka!K2oDLzXy}rLStE65nNE;G>+9j}N?uwb%GXd2 znZQ;U7qb-Dq=@dSN>o<>toF$|t_P9fXktB6!4f`t3J?(e;4KuG-Qh7^$o3KHzo9HBU>fT#DS>ON(kT0;u}vbh#O^h4EYp2xQls;3w3vdiHl%HW_LHe+ z+jF11!R4;@7JhGK=a8#Qkr+`4Cb8pn;&zeYwglYdu2=lGW3<)Xf6Mek378XPv?mhG zZ^@~xF#^7yvRy7SRvhU#U%uT1Zk1pf=v!qre_U15Gb{DKzb9(S6pVX)(u~A)S30IJGM~(^qNIp#UPG*->~kfkTvPq~TaKHE#iD${ zBB|h)$slNqDiA?pf!}R-h_IfjqJPU?wlXom|1}eo6R}%ggUww9q&N#!JH5OZafTG! zFa4;#+(y~VSR%nU{fNB#oJHHZ7ICnsbnU+V{N9=!sMw*~1QdXF`e-!mT<{wADmq9Q zmAs=Br@CG?Pq>P-V}H7Uz0(?;HW}LzTYkSz%TM7H8vp=e%%j8qygrUGH1+arcQR$S z%&}OA5pwa}UGh=p0ZQsT*RAnTY?Q3!9B_ZXdCrY?K#~&^TXrZ&vb$i0;&iw0T@!4B zN+i%A|Dq=B*>fyRo#M(PsSYkM1s?pX?LrmATtoslp-B34!_ zsGIq70yKBPfYSK#mrTwt`M+|KyJH&_XYrndS;js?RIJg6pAt$n;4~2iF$#VW7l|nk z5o5d$GRN0z;J6*N@r2PEN_)O<*7pnng1(EN24*fq-+Xsc5vf-57e%39b@w)pA$l!N zp~JNjJzz$G%PUC?ePldn;8)nsjMo1l8d0p`2rx{KY5c~b5c}zS?D$CL{V<_Z;*APM zHXs{kz8$2KdYMLnXN{q=2I=<*$y&X?AH-vrrU-IU@OAy1y1I(?(WDm&m0u zV^~UHtFa*F4lP%Uh=2XEcrk-!Arnnp=>s^HW1POqJW7^|EaURt>QDMSKdSf5-9q}b zO4&(|D@5^?Y6d$d&NJe+6aFvo$lfsEh&aeUiU4kPJ94qfrIERH3kkb%24rxrq7>?DPY-#9JU6f+D*@!c8*XI>e7SMC zM)0O`)RP0KUnf8;fYgY1PEy%IAL@a+qrEEY47a8(U$9=MG}Z>6hZB>TsavRG0{k=H z@hkpHpe$5mV?X>~Ij^oqc^YZErAYMaFED>Sa&~e#fMPUR;o+eBxgor0>+CQLN?8F@ zxtzZWAYNze+8-~{RUwJFAg$s6%kWqB(K>Cau8>51sBx9I#@m#6jn@6e&gS}EPm}&s zouA@n{*=fZkLUXR5~K%yHop`vjb$-bi4U|1_~fUxGsQ#3LX@7J7ge7pJW0&$6vpf3 zb2(hdhoaRH7vP+d$kx%^%1)%In|bs2+P8V2YxrY#GyL#3skS#$63Lmn;505X##YZn zk&bs!!oezuLu`2KA@8>d7Ks&5$ZU=(gZ!%eX74M3adyYyu4haD7VN_fNSA(0P1JKz zcdDtM`T53TJg@l}>3w{Smhpyq1i{}UdR+g!#X3OG?9&y>ym(&HkG-2=BzWb+`j� zG|_|kn_kOMWv3p)v$GaLoC_k&LXX7uUWUDhR4?!|_xnj~NAt=>_SU(#s6Dfeg-QN7 zzmlHK6N_+f_)qPR@79&YMQ-~@DLEM}QeIDFs*Xcyu}T`bur4cLjvIPgbHJ`IPs_D& zTvzrTZYwB4J7tQ>V=u-FP$n9nuive?Jb`q6d_r}ClE+s8JZ&|z#Lm*s(oh5EXvhO9XAjJ30ym8Ph&Id)?<_|`9*;RqS8L2!4u(ya`H^` z%;ji%3q|=x@%`WoV*YUZwQnd!6i0NG>UK8-cb-o<)Jk93i=nAeL90Lv*prE7!JQDC zH`(XWeYo$X+uyF8yw}KjubRoK)V2)`ka-R``%v>32sUGiU}lPMMuFKukqNypq&;E7 zF>n2jEt!&&yu%7$PecP(3ad{!$0m)Dk(rGS2G&#DyjrK!d(+kEwU4zG>=yI6yGzzK z>D*RaE0*A&g4;uNnBsa=BCUFc7CuC*%B3f^BXutUpTiv*)pQf!vuA6IW92Z%DMOEs zszipoWe{m*_jiv#m>)6Z!_G1gFAmI?lp&(#L0&nfg>!y0Cu4X zc2s8(x8_QNc8-^tk@GBw|8?95F3e-Tv_(N%E-)$iNP8DZp1qp~^2w~e?6JnlbxG8& z&_bru2!+hC_P*|uKh0~ta&}q*MQ^rxi%ERx=e5A0+%H^QIeG2u!2$#mW@tMubO4ig zx}HBh`g=F+$k7svr@9qAWBXlU|k(@{xm?3*XUhT=&$x{ z{CbtFdnXFNfYSpyEK%OPZV>7|LjVs@FfNyLq+1XzUGv7=u2i5Ew>~x{L|4ijY;R!a zj1v;R4iX>?x^-B{-<<3Uv`a0^t>ZNtPEY@U{^R(*RB6d-~e4W zq@y-HI$W;qgm_Pt6c_se)}ZFr1U^!6H$W#eh8AMSHALy%O>Doii#iDEX0wHQEls)k zKhs8t5#k2&j?kZf9OVE3nY#9?$Q$U`3nlVF zy}!=w?-x<(82UZ={t$aC&84B8h7(<+w6~W>=C+qK&U3-XYUB!vQc^ZmgC`nc>A|aO zFMVHoQBw=7cvGLGW}d62Yzog$Fa4r|)X!4V96nz&{zcedguT{jg=Ys@&9_7?qBwfqb8{LDL@SNnjzzyi zoC&<_P=FT?W#lLM`G>Pind3}ji*}eU3CF_`{}rg!{Ka9$`PBJyr>ZL5y>@t+FpOsx zwJIIPAD5!iVDa!A`TAmGmM)as+E6&s^5z%(Qp4;&qOCz^WY_4n#TA-i1-zaTZ9~GF zi0_-juyVv=Q8)83I8Ia*QJ$M|9Psw4L6K)FzfG1Gd;;(AI7Scx}60|=} zm+UQI0KeZY>0gW6gLgoNBy%Zlz2$kooeK`8_p6y6kV+bZk;7_G5 zILmwJe+H3*IRZ#%^xwn~o0puJjc9y?ont-bGW5#}=r9PG0KA4%H%L4L9&PckWv*w#Cqxji@$B3a4YbRK>O z>qdK1TiT9MLj{E^L;%ipIpcL-Yz7;LDJ5e{q6=R;QHxs4j&1|tQdFS26HBis=GMw` z?)G}YyTBn-i|_V$1jBh%K0mZ%13qniQ!NTxo(ZOi?5ay^g*!0|BlqMyXS!KhE!S5( z_u~>!@muUa%fcXa5IR!1MJAl&CWZ8{C^)_1=G{v>jp=pmZS-{txwGs~3MZ+oGzOoC z)%8o^w-z@S7W!a4I{Ef7FApj)Vw(I0$@d*DA5C7xGULbSPQ3Y%B}~sP7o+Vr&u%@F zeD!LT{ezH^IW+CC(M&~Px#Q|iviB}|r5_8!*y_l-Pdb^6QSuGVk%~c`7kj#Hk(zLf zB7R)HsM8_4^O6irZaA9LY(S%%IS(HA{P!%-o9_ehzg(TL3Z4H~+^(tNC% zkJ0{u={KI_{dM_<0eXID=b4VvHn-J}h#BZvhoCu_jYGxt)^VS2e7x-f;oVe5-b5CC z=rreDMxwWX?~}H3hup5E%ONxc=83#e@7`_ZJFgLTM+7h5KE;n;r|xr(X43u~AW{0h z8;hC}YBckCyFfUCXa5K|U+3*a5;zn9^wM~1()K`7z-39$nm)$p(6}_#hkQ-cS?*4m z;XdhUXBc{{dEgXQimI-ZOknYFf>S$BC|mw+53W7fYg4IlKZdS7ZEq*E=CRc%ci>7t z*oZr38o_iqBRE22MeKC#st@+1(wBuTL^Us-?D>1zOxtG17JWXCp$VG5Oa76e$tQj? zCwO!%ALzXm;O{m?Jok8eJui2^X?<}X&Uy590a!tDz>~xpj0Sw= zjuvjEJX@RWvKs7Nc_hi*qBR%BaCt5TpQLWXA)u?>!#kTRt7h!dU1|Gdr!2WZ4GV-_ z7ZW?SvWuIp2Hl`)$@|m;w*7fl;FoAK`z@W_qNju~1)AshoFqfwi z5vT1+T_%`d;fyiDmsQF9yf47*i+AM1C_qpvxAW2Ire9&U1}bzSK1gwvaOf%ssY4lz zoaczT9rEeQPBj?a#c!8SDsqUqv~U!$f}^6|>uDvHl44aKcpz*-d}K~of`K`A7SCTX zi-bh>p}d1YT!eZ&S{5s`oix2-)w&~Gq=N&pa#sO|ws-JUP3oH7&{;Uq0Xtkh#qM^- zX1}aYOnE|fs+RbCHl`FN9L&;?JZc9{)sH-D68HA@iO8wKKtpwmVa zxdlH9a!2#!&*oAG%16*XQ!a-^y@)VRg%zJ;-bC?6%q+yG=DT2Zd8S@UJZ3hz z9bV*^HRq}$r*H)|3E+9a`qyb>6)pmmGBNt=jqYd35-{*5CE{F=$EILa`~@MEYcddr z?`rD8?74VDjve(@ZZfmh1*bW=rWEPb1O%F6iaOZrX$Yx+-cX|09R2*e7l4EH<+5Z* ziZF#v9%%h3=o|>2>~12fH6syMQ%c`#w^S+Ah$F+1jBp}BQ6hIlzjY;Y^iJz~rIbN5 zFjJ=6>L|`buES|&n2zH2adWvg8Q@s?5&JiN5k;W5HE*ot8@1_uMz6zWu}o`sz3qX; zK_s-lWRW{wiCq$$sTe2?oYQ4YJn6A9!mStfe7=)@*(M}9iRWnex6~w`gVs(K*aEgVPbKXq<6j%?W=t;b z=yq*?mslbC#-LBLT$x2@?m~W8v;L%Brf;Y)u!embgOZjHqSdog+#VbXmg`4I)>ma4 z)8bHT()9LVHik6;U83MaeY+@?f39b~HV9j$WLo1UacYS(xE3Cu{U{XJEgXlEMTUFN3^mMpF_QH3H904cRDX)>O1 zMs22lVri0l^HFkT)(SY(8yvv z@$?DNwTeP|f%)xv%%;b7B=2iF%I4*V;*^&3UHn2=xq4m=H4&c0ucKV58CE~ec{91- zVr+7sj%#`QD<~N^|KRMPC;P}2m)e*a9{9wprCZO<>MY!}YmhgBl}C};QO@Y?Xu(Ip zvYsEYKoyS0k}U)j$RSJfKWrm{2I<*}v?b{JS4?AJ`8*Hz*MGRA+!%ReB6 z$i5wsjn+%an`fkc9t29`_~^aN+uBVVYQ4n|vvJvR!3=c%Gs^5w2`d8e0M_NYDLPisE{V?_LtpJ!67jx@WV>8g!FB;JnLJ6u8wmD~W zEEkZ*w=?$e8<@+}p9ON^g4RS5ZH2Kw|MHE|Opg9;u@`FimZn?52mAch+nZ}^LRGo< z6CGr{pU@k%PYNgERE;^Rv1O~*F3_J_UJ+hSgQBUaL{dv#iDC??bT(ATW7tE;w!dxs zNl0gqG}jZ4&v|soj@HRm zFMncZuR%tQzn%yC@(=Oq?E%3u0+x`LC|?6P90{?NCz_qDz;P$nTm0S?%f7Dr6dxKL z4!yyYFu9Y5JT-XQkf>I^57-#chEdRcVQX;I53up~h7&=5DS?N-48B62CfHC-xHw6O zQm)$5PG1r=Lh$UZE}?Z|g!&W37R2Gi*uC52UZ&-$^46$LTg_^*(_V*H6TV(QS)G>K zJmpoSO3fk)Y-(9p3gL30u$T{DyA;gF-Q`VsAm0(o_n8gFUY6TQ0&F%XsxHr$2}xf5 zFJf0knl!-Q<}P*6c$sERwwL0M!0rxJR0MzD=&j8l(q6;BH2p7vd|Ky5a$XLRJQ6;H z8fRLjRgKK1DD$z$hxU1ym-JW5^Nna%WtCf%C+CMvb*7l)7~^GEdy7@(y3~m0BN!js z+gW*Ygn*;wp==HqK38ryd%_&OH`aqM3!xmo+@tGZ0abB=h&fLPSJ5fiWTtiVg)KW# z-5+|pQxt4%U}4G7;yMtOv8_YvB?`q(VR@89JQ z`Ve=3BvjdRrD!gnGIrX@!S=gdRWQ#ohWiet9{b&_yzz36GY7A!_$XP;3Zxma(AiO@ zOoSPgOSyiUeL1T^HT0QVaNOVwyq(75JBcdb3J7R0Hiv#A5JK?@Sz-+n@5pQW&PXkZ z(YyY3;&U%fVQ>okC444<6huS>yIXM-Yg?&T%xcCM33BP(MKP8B+UVi^?(#3{FT%BK z%QgF747%A`MGju=of=6F-+zkdr8Uv76`Yeb|2%5Fzmo7`Z|muhvd?eaycYP=s!+|p zy6eh#NkmApBlydvMSjSe8z8e&R;8T$((Y8rx)VjeXX*`@`T1qoAJ-XEVxxD(UH56$ zCwvOxRqsG)Ars|BB^|3io!f&OG1s@kV7@9T?dT`Ki3jxEn0+P*>KqHc>1*nYD+GVM zq;tep`0`cDgYA61fU=!b(BpbQJ>NdA?pN|Cd(%GF7{R5P$APuJTH;1?R-t4@mmR@~ z1!jS&EzUCp&r6$w+Z!#SobBt)$UZ7+oreE2o}~0-tP96xQ9BN+{x3wu% zm@%oH5~x?>eEdyUmWfktzKDxK?JU%+*kB_z6z{7*gAmZ`aKrp6t;9v>b+sFtmpqg$ zVZYMvLI*$DIwn&xxj^QYn3Z|U!bD}e>Ppx`FXGG58TPtr*r4a9`4*;#MRG93ERaT< zgf!i~_i8ijW_dS=iid48N}9at4O4;36xk+|nvgR+G-d)j$7~tXhu$I_N0MbzQr)JQ zOHoYjD1MLiIz$Lq=}H6#dLN$p4ONpmg&eG4F=A2(7P3$MT?kV0b{HK5o$dDR_Yzt%b1~==)iMjrt15I z_evdGf0mfKyIhq=$#R7xf;1)zj;w2tnHzpNT3@=%8Nr9Ez`uSI#FdezNqVg_m+d>U zp+%DKi1q|DAre@N{>60n*WdSwpninkc!#8-uj(B4WvE!vNG-go?-4x%*WFh9X;jc6 z)r^CDJ16@#ox361hpl~W^|;Vx_OTxNlSa>Y_MWbGUskt2TL?YP$~eWHo2u zszyj2Qt{zFA)U)X7P>eh3|It3Yn3^N%f>38E7h93%}g6D4Qn$?rpku}ui%IkT3KZ? zrE`0sgGY+!v}6wS3#nm?qgjBA_iuuNd1;jQe7%KQu{>;;P$onS%YAC>RpcIh(_0&^7Xxvh?%McWcw2(c zihYg9b^a_jwJEB6#;3kpKcp{pN9K{T_yZV$^H{Kzro$+EK9PFew;&?t~ro_&3M~)b`{l(?3FnEt}ZBx;>tx>pfz{!XVK50*1 zYc#FAe9WjVwQwQKsg$qiGo+v!KV47DVd{+G*7p2IGImv)f|AtfgyZ6Nl+*VyTB=j`Wm);DOgQi9IoO5dP$Q9H%4@3IXev?Aqx%M3eo3ghoZDG zgutdG_A6HUDG8ZZy!aWq6g_v6dq)N`@_V8psq>q-Nb_F1zMr+8C#@*!s2dw;u=I*jm>?Jdo<0f;DXg@&Dz@62H~W3oo2OqM?&VeD#@;@ zMvL`a&M@=Bt+xGOqdWIpQaj_xNm5d#!^sBwX?sRQi+t%hKC=(MZ~xdNq0v(?S+%Jb z`f&3C!kUY?V0X%ivWNx0^Epm%+S}^u&-hnaLXP_{#~GQvG#6n&W!yY7(gj*A4=x7g zl+TjWwyA&L8bL4b5v?sJY>hc3$s04&f!G%#yx5ZMWX)#oPO~Yqa}-{)L*CK$ zRZpZgX!|Z^@w_c3IfFVlvF&D8Rb3eV!@>OJ&cuL!tv8_DrO(omTF_lEuV9W2$lcZb zREH<{35Bc!ZpMR1T*}k!;?F2V1--hq3Y`ccqjzAJpf3TkXR&=l|WO6BbuuzdN7%q=7Oj{PaiE;bwPk@U@wnBBIz4B=dlvOx^kWILJQDr;{9Agw|dL z$=_bq3}xPIy@Qr`z!AR<|m#i5S^A+J&M|Bu$ua zx;mZnN;-hAZ@9+1H?FL8Ha2L8$AfZSV^eSkQ52YSclh4@lwUL?7|~CcWTH9Z9YTUm z-Mm{TUTox9$;}@ca>g^~fimDWX<|HjBth7(lF-rXPtc=f-I7iPl8kKD7j|B?+66Dh z%zKs%(2xT4I*S+iv_yGqdoS*+QXPEybRX8a=|f18bQY34^;{eIhD@VTDw5KJ;UxIs>2;qIjS zoJ;6$)t~Fp$#uBwg|4m_okb4_(S|w9qRSFH5Ps&MB(wFSLH$zZmFevkM(#r#T4x1!+M(YT zJnct@WsmpG!8nIpS83xOxUFmC%<5kUEZ-SGs0_Yi&`|Jz!=8RW&NtkhrxgUW@Y}Ja za-5HQ9<2mj*lJzW8pQF*vRbDM8l2`rk7F%T864AAcCj58Vp@(D3Z{*cle1934oI(p z&X;$vXcp6oW25|rFC=LOzjp6U6F1bwDaK{{>QI5Onmzi&4!p|- zL5VG|;dGitKn+XgWR6~Il4r<~l%-$q@*9~b=(NlW7i+>jiB}3-u10(q6oSJ~`{P_N z`rNmYYN@@`oZTxU-^R1>!_Qt=#g26eq+y&99+t>$IWY_Vbg|Z6L_yIt@jE$EaA&7c zvGX5)2S^(21;NCHiY4+2s%!oadv6{Nb^FGFMk*qTN4HGKh7WLoa=i3W9FG>f9~gg?)!7! zpKw9bwQ$!cJ#OEca|cU4)8(5} zswaiM3U^#PNqwwxp5{tVubPjKl}+GtxraWWwXAL#?Qg}*MjTUTX?US@8Ip?hWVttW ztxzJOFKU9(c9qqPHAq=nYxXn%3EW4+{bS=e(yAl&es z>~nYBmlSTv=iF&))Q=cLA6T0^i0l^F_SV)QIp}DbZS}PsSIa+GCO`|PndlB{{wl%NqW00yJL{!uOE|LOdqv*8x zDkW<;FIb+=UBdc+omL4~Jhi@`cVgj<)D3g857ol2dTb*s8q5c(&3sHxVLoKcRGdCd z*36fz7_`gxs&_&*V$@BR(3O!#t|LCueg5kQ2dQsiErMEa$vmp1_`(5%lV$atyUl0H zm{=4-iany^6XGhXV?5|sGm0^j?{UQwo?9TM9-#@fiynImx!1<+p63_-4Efw}&B|Vo zV4sv$$yB;*iUZ8L%MNhIpQx2I-X5!{U{!(1?*F7N zghZB3n-ADtnihj`Uvf|?6>|Zppqme}?N+HR&-c4pr9Tr}(G{I`oautFY1#0KIhHIv z6iIm)eGcaKP~yRsIQc6*1PDdJ5Is1Rfa-eed|Y7)-5~dqb?c4!oYLw7OWxIwlcg+B z^a{1A&cjU%Rz>6|b0JwB26?e?kK;#g#rp;~wZ{}b769y;-xZ*|y;)ax*S6=$lJUb8zhDt4x*j4Cd9F{oME<@1!=5Nh+xBwnO}KV@(AMg7BUNR=tF_gY z(VjCgj699yb9GVJ6lpE#lvnD2gMSZe?V|g)ilK*46n^kF^eh%8-*egRGtXt)rwQMB z>K~9IK4-tBR}}=6+vN|%eVAw|^(?2@N-T}UQh%5{)_B)c_J(`l`@6YW>mpy5n-XA3Q3x`XSZkkJz@Ckb2YHNg*Y6Oi=fFU*m9cHTUiJ!Ew)YOG@zV{ca&*No|%Y{=O7PfE1VPJE87u@}kWgYVCkZ+n3jgmgQU0lH4Pd%L0YCJ99A~w%!_|)dkIO-QtJ=jmGMlW=b5syn?9k~qOkikYV$GbKv0tBE?E zyCBw)XwXY`>(==TA0Dy4tnaiTr(sv2V0bBf^3I2+^a+A_=5Z8UQ)`JkuH(Lg=^&I& zG6+rbz~k{HCI0m&I)ab4BpLb8+0HqN?b(%?28E4Ok-g=|L;X#1bo0H0j`au2Py5l6 zcPmf11ur=#xEhdCP@ZL1d~}SAoWeo9)fin#;R*urSQp_^Dh<}nsV*(~mCU;)9e(}4 za+GJCG_C7uPV3SB`srWQP@d%_&vaBJ|MxGe1lj%FnC^HSt?qw6WM}vHq4~QzN7@4l zDE5$N;@~$fsQ%H_U%vx-m+9E?TOa?f^uGpuXd;_h?+FMgY`K0qMt+7Y%d7U}~EsO@}A{qo`ej)1iU*#N=G zZS?AIbN^@Q|J#O@(OGpS5x>vXpLOk=eu3N_{Q1A8e*Hs*0ayU6gV3+D^>;581lavo zHRq21`?r_0fsOIsfd8KM|5o^yz5U;o{zXpwzj%hrZqDk|%fAIUQ5&!+H-0_;e=ixe zs&%^esB?~(Kcx13-r0XY@IPx?Gl9gwq}_DBe=oo;>PpsxL}}+8`}gmSO%*p9tHOML zzX$%j!+;+T5UQvy{4ITcnLjdeP3jlOq^<8!f1kHs2a+qq?vIpuL;m-Ger>|q8K`>j z%FMan>in}*YZY)g3~}H2_nIjMfvPPh+GPKIOvyEm7W^ijp8nUGkA>y{Rci)3{J+q4 zhAy6k=%n2y`?a@!kF2Z;sM?us?f+s-6To%k%)tM5>EEUe4pc3&@|E>B{ot=sT4sRk zcD;Gx*E{U*vaN$a)j5#HYroa`XGcPvftC1gk^k8(|1I+08|SY%`Ty=oE_*-d&a*TP zQ0)YKykKM=v^kRU79bVoDg0roX-!T62k7Mt078UUlB31|PaQ~%ObFS-y*~#xf)@Bq z17ry>lM9pF4C>VSCccy z87@N(DLRcT{p@|zgUyS9-wG60=>@y=p&{9Z6qJ#IG0n9HB%A`wu?=8<**;$}YU<+J zOvGMGa`=cO2PR>}#4$ZF+R)MubI7*dqzRK@mH)D2l6%;#&Rekvia}Nd$43Bps=WOS zvER3!ZO{H9a*BwqZ3IY^0Pfy?nWg!Ii*T!8Vs<&v>Bc^D9`NC6MUQX;a1{+LPR_2! z$%_zfm;ZB+$jG0Z;1=^2-b*Juxd)_7*SQ@RzJ6H{qfbuJ4rS-i_2YV2KOd(l-ZY@X zjNhI57Thb@Xx^n6+J$usS~5fV`aw~CX8W!lN-wX9K2~gAHxW4CHqtI7(HFSYVc^6e zh1XSr8A}FrujA&xdm$hjohV!{A|`~8vpr1 zFkSe+ylY%wl4rwD( z;0_=8?((Kb2iozPOP&I40P#WmBxB8In~157FU-^3Z;oEi@qD5lI&pmINO~FEww))A z9~BOu4Y{WC@-{3kdD2tu3(m>4d4FA_ zJlr*wns>XSbJqXNY?_FIqv#Z6{)g`YU#N`C3hpj9(kEI-A2!74=`U zWMJZ8brc4)U$EjS(B4#n5>h=GPtv)$FNBJqBGpDdV|`f#5oEnRLw`iwuvQBVk3be` z5;;s~Q`r@*$*2?78tzwpv@k!(r8d^-$rSAxQS&mDD%6ucZ_lrXOwL-##FKniRyDuM z|50T0QSCysvGu76S}*P^-8oHqV^Q+vpF*qMgCGgkF3rp_n=as3DUoB0g?tqfj=!~4 zQnw)RgLukNu*);|0j}g!uLpjbN3@Yp`_o!Z8;x3w)Q4I|tM~~GIM0w3cftwmlhy>< zF+s$zcicfbU4s_ukuF^F1vIs{Yd1D|{?t88FY-=j@DOKzwHUQ31Fu>h&srCWKaZ;D zZRQg`>Q(%kEZ;1xv7sTrPi+jm5)7|?oU}B8h$=TUT5BI%;|{kIdT3hq&7ca z@XwE|$tXM18C#*l6i13+&8?sDjeG$}4M`JY%|1ig(r1j9YHESZE~`GZlT%o!mkbPqsN-Yrlnr?o*MqC@sSFzAqW##)dVo{o)DfNcIF zqg5clvy&(skC$XB5AM@pTtB3x18$-oa+Bb!S5lXA3m=XGX*Gglqi}!*fB{A@S@|8f z*_v^r-o}w)x5TpiS=4bXaFt>5T}vQ#DS!g8OY8eqX>rP(DCh%R=G^albhZr)O}Mms z`1dYeaoA6&v7~NXEGP`#ot({W+HP74GBS0}kqPMx=X3R{Y8tnQ2ahjg02eSzEtg~6 z^om>Uk9KCd>uMACoP8&8U5Hf4FdO6@XYK-zZax&tgH|#=;wL4#rh~mT1%|W*BZsJ} zc&H3tn18=*wEu9{s5z`JSTjmaA?^C+uzt(liu)nYE*QU3H3Xe>t)1@7c~xe!(;u?k zFw6QNPAi3&IX!n}yOXKHBp7PavcFjh8`+AD4xFcSEIw}_l^%W`DI1geOr#9YQtzx= zLbvjEE(&1s+R>^KKBWhJUVN~d)I#F23T4S~MEW`8`#@#s0>2!GCAd}gNfCQyb}p*A z0G@mP_sr)h!Z)*92%7 zC%Aj0x#AC}WMc&jpFFS$c5SGH3YpkR~#5w0@@FiiDf+<~jV<;N@c zDc}lD1IgwRG&h6R0<0XGSG`M@m@Wko(q>6NI}~Qh4TBm0Rj-ARzQ%I)#TeHTfQ$rq zkREQ0JC1?vZ4T)P`&W4Rg=7Fa^_HEHQXfGC#e5FXBTAE*gqoT10UFx)Yod~*PE~pF zoU@Jv%&dOyQUY>3%zJ(A!Gw2o!QTO;&}o#@MkHXo$6v*}xT>nIm?)P%>?v>Lp6XpV z`vOonJyLJID)WcxhmhvjDs6;qHszvU-FC3xD!16D+1o%V6vLzdPA(R7-Ni(Lv3y^O z)MRl+{|Ds~zALa^v9<)#FiL*2ODEV95{vX`XR;!FW32pks5}CkW_0>DJH)+*>V8Vw zZIFO-BhM&EP$eO^8yz8{GB4BJ-5I9a`K!CDXY4)&tkyX$N%p>p6QA7Me$MQs<{Ayz z`W!WJ@*u?B&wgLP^2bXa2woi%iE_4i*F-v62I1YCXPEcuDPb^*TPp{O{o2~`+b)TV~ z%z&ToD&pDUUTz{`L=M{6_CE;*rVQy%_^Udrdk*27dzYj3WiPrlr!T6g&ZNPb2iOjN zcI>V;xT~%rOzjo8qlJk^m2toVnY_k_U&!-q+mMacR>d|F`U~q|&AXB#+H3p1ExXhB zrE~$_A-<4g?=W{GnHB7WB}8a!-dDr1*dcK-cWAl++%oGWwKsdPH@mx1Yt^0tDA_JM zLQOUujjQkp9Sn58|7o0VIHvJ zsgD`tLkA_Knf}RjXOo|cg{A#Yba>%}sFALQF*EljY`uNGZQ|+)2Pdc-v&eLee(=3C zHn?vyAswx7Ktcy%L0pmh7?F?^;b5Yl34hFPzRUO6U{{fc@2JK5Rp3Nm%NeK*34ocb z&amIBj!$Fp?Bu{91KiSwAhENVTy_0>6FPz47#g+*0M>Guz|7dexbUY<_ZRqd{4|jD z@RHX6pP83wYzp2BUgAg>w4l^Op0PWHySF*m@%=}1NFWxcrlyu2p0EXo|7(y6m&H4y zi`#2=YAt)ZUe?({WOGIvgR6SO^WD7Wo5qW}5cee@iajff`Rc^}8H}*6y2JKYx{$Xt zpW~WRR`ZwybM90p!llfbzUL~XYRVEH&&$Q=o(G;b#Zep{`#^j1lSXJyR&xAS@->bF zY{M)m&cUI*s?R$g@ojy&YrGWqU9v61S|57PxP=wdn=+Dip{Ey1RhLyjAi1jS@<}!V zqC7p~F1S&XxyLF#P{NLTQUe6@pAA@R(!aynnR7mE*T_>^Wt^I-o3PjY^OAje#kPy9 zS$_MMn~TXjeaGFu^TT$XJ#io2_Q}lO-vZ7jirB)^r=yzIZzLj6v+3SE0j6K1WF`je zG~yucz$-|^yT{PedCS#3z^v}mF0fL>65l45{n|Ih#nFcSzj+N5^{qD+>P$2`yw>}~ zQ8VSpAH2w|DwMs09-gsj;TCHTPHM=jxUY93xfva_8b7PXr2Q>Sfpqs<+6JZOrxY=k z)I!e5zJ-f;;X97OYXP%8{C;U}V@y=pbE?4>_qn;n&27~mGE4<-WU*P;zLKm+Bq2%B z2S1}XuI-ML1}E0yDM{bM6y&Nkh6XeBUdc}5p=RCBi zUiV60pU-#0vN^k^n*ouZa02-w(s8&L$KgQrXgxxPEyvlpC&o8c$9aiQF4Q^dB@H2t znb>oMFEM4XMSo)9&bC8KYHycH$oD5~2`|9A!s*N1*XO`)wG;OWBWq%tQbsW)&jRo2 zuhf`M=vYtTa3r{ZO5K-%ftX15_#Pp!m3?q?XI)HuxVdZh7Z9r^7D$UjZDs85#%(CT zXZLvsrhtkK^hoW&rTr{0YAOK3-+$-%9CXZpdn7@=u$6s&GI=Yks9Vr?F~2!|n3fHX0JAigQ&Q^pIuf`qo2< z@=(hgi|k~$$V1P+1ec$)m%*Tz?^KrC2ctvgZtE5FNlg~v$n{fGa7=JaJnPsL7^X+` z8spd2Y|y~1dUXpkY1HQ=?e}||b#>mOPhY6(XOk1lDMsaO@M(TxD~yHV&Wq=~iYk+? z3}1e|@lJ4BG@^CcU7}+}2T)lnHV+lXKAaghpJtJXd^cB&9(usk^t8$M0|%#F3d`v_ z*kxqak2-Xa+m4{44A}Nf=QZOV=^VcJ1{sZ2gd0}<^}6mnOo)H%zKka>FyT0NfDq%! zRIUU(hfdkFfCH5Hjnf~A&Jv!_RNHxlm5MD(0_pgg)ysO}9z{E*XVm&(Zdz>D7o+hr zLEa2YMV0+oYZp-k|n+7{jK|t0);e_6YO#rDSk=Em*C>A6W3uvH;torv2b7*fG+R0!!z# z{}wmfSfRCF`&G!@PZQseP6UT~xkF?HPzlSQ5Mt{Vog<^G_7kZfR>;!Pr{inN$eE?O z24*8ZF|nnznT=fWbe?6&p^@N)y_W?&3QHzv|Ick#;MUu+vqUIyJjKc$?`@`9o~Mx5 zpucR|siie++3tK3Klyn|W5V)%)J~hci8>LJQ8WC8W|5z1_6wjk_Dqt3`}z4me#%#* zy}pI>jLLpsxGKRAR_~V5scLV=)wKB2OT(QoS?x96Zz{X`*0!TQKV)!)W;OPs4YFp~ z12L3e7l0Z|%ZhjLIzK{Kxl@b0={dsDp|o)C#SrMsa$TJ`tbjRvaYl>C_Otbr_n(KAgS*DEyf``b!uiHyEB~);|kYO*kHMUCm72pI0Qd0 z^}~WPQohQ1syUwhoDh?uo}+IqOfA`i2T#krbC8HOm+nTG2k=S@XUE)^Fp8^>;764r z?fdt8S)MYz$&@oGaf8s&1a1~uHGRBjW(WObGqX%*7IxGB%W*d2W?%b{G=6tfAl<%Y z`{(lQeYEByQ&$($?pLVqBdTSQN>X=J>+_hkXFo?aB5Ik!n*1)K>mVV!-J9X{4@wp> zt!gbl&4uz%2|LauU!DulF)zGG8!Ck09sA??Kt@_imQYQ?qlO~n6T6?Wv5Qy-$E(N5 z&itfKmOZ&n@pwGHE1@Q?B^pD2=RUx_yUkKK&G@h)&C{WOb~N&CBAK*aa7pR>yJ&AE z6M-kb7^mg8$XGK2Pp$^Ps$m1IBd=mb>;4le(MUp?l&)GZb)wX~kv=}!Fn|+df+6my;txjQWrXtB*HX`)l)jX zqhzPP(vH`KXNm`FX5!A7PUvE92BqmvGlb`QGj(1?pD(srM8w90!9AX8v?oLl9+$CQ zw)<+Xbx#HDAx!7p%amn~r3yG1A>yQde5j*N3l~9pp(^(cc$j(+7rCCMF8;ikukyY- z0sbaWa|6M%CaF|7%rWjIjKw70slV^rf1o{|%3ITLnaS*0lQO|;ba#3!m{=9HK4mVB zdc1MPwLgEzTk(wyD&T5TA5?awispq!uUN{_w-e_IY>R1j5(=L*ELUi5+*_S8=WdWM?q|!w_7X+Y zH)@EH&8z&F5KT{MMK}23zSC21+d=hZ^M~E}NwMpDeMY6`kVQ)YO{`r5phd!6j`^h- znFz{qW$j3m77dTY-*)_rnBx2ays|3#C60P4L;bFb+H4>YVp-}mPDuKJ`;-|i+g@8I$+AE`!d=;99d6Y)s)p4H^!`~f1y_X=GOVE4%!iJ7e@54=U`4cWq zaeNg}Sk(=|;+2()x<;(xy85V!&y+&0a4D)Gsvw#EV*vBGhiV(_0^d8gsL()G&;24) zhun0z-D^C;?0Fe_;k*4bX8{*q$*mmU5E%mefza&sa;n1vb4Fly_p!hzq!>%ayF<#gA4@ZkJ&ke|T3rhBOS#+R(P>seN9u3PpZAso?nBGeh6UpDLfw z0onWHvN}bOdU2fgI{R#dg9sjpNmD;+U^KaSNzDIBE}Zp7uV|r{NaYYz*1K1W7j6_B zz1gcIdlC7pmbf_s6TobJU)MF&mAKby)Q~PU)>qp1fr(3gm6kmfU_}P6qIgtB{}*TXl2a1#ZEb^EUC{B>?gOJw$eE6WXy zo_>-69AaLuG2f(=p!9ba7o~Zr?VwwxCx7VZELH^4e@9%b9}|MnUx$}`;^XAl1xyIW zSjWUjW@Pi{J;PoS>4y3}tL8O71Q6@c-ASXKLAV8F^#p0qmI=b48{x=GW49}*7#tT( zUf{MW-CN8EB;+~PD}z~e@^%zjd_u*L&J)OCq$x0Vd~q@xjd`SgqI*msWW2Jphy1wP zlg?+~&7a`qZHMEuf4OS5PN+m8hq(B&n6i89I@~KG*>-?OgiN1WZ8u3#V-%cjb;CsZ zxYLG`^Gx)VdrDnu#$+4eNy`T=Jj_V;Ug8z4e1k6XiDFCoR~s$@_S{M8A7b`hx-ROc zJfV|+>vg=>c(BXmd0liflQ?a^;mNA2OfD)`#dDu?=>qiLkoAm3n~wVnxpp2tR0{$? zk=`$oO>@BGP7u=itt$>VXmw7tcf9DiUPQ#YP*2b$rYlX@@M%{%?<-uX3t%>T&ZmJ4 z45AoWy>{=u>Z*Y0B}*bN&xm-c>lT#h+d( z9Nxx;thiXERY~yFlJH%1B^x6`xO(>CaB@9&CZ?}|$wR-Zj0eArz@3@zs^tTR%!wPS ztLRwbD8xD{IEEbkKBDj~CFPEnyegXY3`6;C>v!??xLYu(1qWGEmtNVa z(HcIR{c$me(X8Hpo2%FI;Xs7tor=hL+PWpVXk0h6i|z}_Sexk z)eh=o6C}H?O#4XgpkNE9u{go9YMWat?^zvYRi1kbR4%=1kTv~^wt2H*^v(3EQ5(i~ zW&iiNYb!CX`E<-$H=6+4cI3I(JYEiKLv$V`Mp>7>wYO`gYcm^Z-Nz}7;G&QY7zG?3Us%5SlvPPbnh!xCQ%Oj zNX7U1nz=^~xBkl#(|jiOI=3eQY11t`%nwLZoALKkA2hroR@9 zv0UUUmItwe^i2VA8)!9A;DhE*`9;*`pN_l&8C%Y>ji-`<)duez1N2|tg+tYS*VRVj zSuggzv-e}{<(k|cJ@I_(=MK=s8^N^QFQ*hfS2pD_{3Yo?yyT^Q+|Mf1YJf0cn(+@Ot4pP)8}}_tvERdpRiAOz5&uMRNtqCJnh z{(^xzP624xyBAWeH0b<4J<-@eK&zQfgEVO=V+ob~>HRy2Qo@9IOwODc=)U%`*h`sn zNgZo1A$71N7(nl)T8D0fVUtIZFuEyn*2A(36z1xTv(mCHdC%u&Z6YQMba?enG zIr%|lfZVJ5(oX^s#Cwd}see+WHTP;akqvD=?tWySyP?|;-|Sqg`(^={m4ddXr5Oq0 zHYz6UO>YS2KVYdiY*p-Io~}Om%0cI+R|nzHhy*hz%9 z?EWz!SfXFE;&X%ikgK}}l4L=Cl#VSYo@XdLT%)ssUY#@N;GN5yFF(bKp6Fwgvt5(W z>^Y+qj_g4UbIh)iT5il>vtvEx8v;gg%DQy>%Rg!gri4s&JyVd9yf{S(MFCi$T21JLqXp`)coN1VlBzWIFwxQFGKz z5+z^Ptqw>XYlJq=)kh;ZpW7cwF&cQ%NC?zoL70>BRzuYdij;26mGszUu1;ZJ^D;Se zZof(TSdla|vI96I9>9uaAPi=u&!VMbujK0A^Zer!rS8{ZWRz(q>}LDcY~Ki|tLAF< z4rpkqPVy1_ig|OQ@7_yx;v2Yo%C;p^kTnx_O7S`V`rgJ6gJ_~6f7|P^MscfFxh7HD z)2Mlxq>P#B&8$q1A0HW$$g&7EQ+LWA>6C^0Y8Had`dnfD93;$d%Y^ev5OFv0)HqO) zbzJjrl^*L;e`}i|rO@XE%_h!m0(O{^esq`UiP74Wg^~w*8~Hjx_io|>$zT&AxcMF3 z^iH;+JJ;~AmePAn7aN6V5tkm^$gKN(N!yVzDZuySFj&cx?xz=2zMCU9t2QtTrf^&3 z!#>7bL%UmjD(iH{Xvm#o~MIF%SPb}B;_PqLNw8E7#ab)WlCIB@!}21^*eg2vU% zJC5Y0xtXRWv01DEZX^bEoyuxHsg|X621pADrm%Z|e@BC1u&{Q%pGe$ZPikq*I;CIf zyy7l`;It_>uJGA=Pv-t19)`G7w-wXe2WK5`ZiBauWr?kH3?5HnmlxC+)d3($oX(H-km4ej21V&2Of*egM;9VJ)2CO4$ zDPUcM&xCW|-3Hlqt{xTGDFavQ8SF9MkbGFsRUY`d=wQ8KzT@*OKA8BFiP?yo+xx$O z0)K)ER4y93%}bstGWK7v3!Mrw|JLtSF38G5U|t3kMt_?m(qoN~atq&S$)V#fFRy1+ z0!XI3tSAk-cN_KTm|2TDTbcB;`e+KaQ*43XA5DRK?!W!|+&()(dF0%J(=-(i;x?8O z%I{&(4W zS9=whpv?yOSL{=0uEmf&I5pqjm+24H0iKrBDT}@NZdHj*U>@c^=_=)y-5IYud;W>9eM^e=&lxFkcClo)X*DWpWav$^ zU_3lG!m9&?Uq_s4lXVbYu4PhyY@gY1nIiz+l3=#J6%tFkx9B#>qc=axDrg#gFMwNu8$!0cjGBbw%aNMKF^?*^ z`NE~vhXe+CEdy2(Gk#1LIzs)oHxRS#4l732weluPW6ohwVb*?nRyAoGZ?aq-_=bj) z7Wd}Zc0Prr?wyIB4?ftP6)ChUTVC}}0;9Gpgw7emi03Ucc$2RrDe;difU{1M0A(oe z1Xi>YMoDnTa-QI2wI#R^Q(a2B5F4{vG)9b9{A@JL*!%=o1)$DZ`qOFZNuF;ofOGQ! z(gMy|?&iNGo!VPyc@4b!cJ5w63O^=O&Bra@>?DT}L>)CTI8o(+$U~Lhk*~9SJcle* z?yb}2IKB&)awwi^jhz>W^~#)dXdB}AVNmLCK=er<>h|?9d8c&9ZPH|>E?Fv-iO<(>1%T~g%LhrG-EOtWu!S|>DPZ`qN-Iq?m%-klXC3jq*}VRLxe zodJM(m_BFeLV>>mI@p>M5(EM@a6cJyw9)W=xxr$YWp!ea1h{#}n1;Zmaa&l;gaH!a z#e2JLzZyG>`#-}IKEXP(9H_=w!L~_YCL|ZD!xD;V!5P3V^5MS9z)~47>+=%SDfkhW z(u1foIZ7K&vYATQM8ePzDUmLq7Ev&jIQ<>XW+`V9q{di*Iz z2JbHeqz@WbO@bx4@M_tK$GD_v9XI3FIEDIbQwQ^fVznLMvQdO5a?>5bKik-7XEb)m zyjAlcs>Vp#PhY&_#}nU3tY7sLyteI)2^yp2InA_*K(yC>F3JJ?opC zp;<p~YJ1`Tk;Ydmmr_g?`z0hgwvXQYGBYDHkO<-R= zp~86QGu-8PwIT6x?wD(b=sCcx+GAL;1Gp0UmFOTjb!)w!F1S3INnGmPN zv9dZvm<;gvy|DX%M_yNaH88;<9y$t`H}!p7U|r{O2|QUviNY(J^mkOj;CS47MWRx? zZlY(Web;jJROcYNIP0l>vk+!}mSTyC`YHon&2sgjF}A+4MR99?Ytr57ATROs{8cB2 zRbOx-yzZm9HhNnC_Y%QW@c=4PmvvR$!_B7iKm>!f7Zza++8Efp)I(0xfZM<*9I|80 z8%22Hr=nsa*uYKMU0tQ1N@S5IvodYJy&SD_OfBG1JpMpt5y?s=E-3g7TnBj_VJn=rm#sX8R|UD;u0vPp#EX zOMsdhdr(xC1h0INz^~MPX;CrZ!Kp5{9M+n)uL%9*X+qJea*l%;d)4Qq6coNP$o>CURQ2gkpoG zBG*c(+gZpcu7a({S?G8r{1~s8en~awC5@*O=jORU{3licL+)O49G~bpGs`Y@28*1h zk&j~I)gdos3M5EWQUbZ(X(Ho|R5>6zWvo3~2d;#MamdDPA%!t+rPMjmZ((1H&7-?=`s`-$&zgE4 ziAaAKra!4QT)?f>JNp8O_oI#8cuXnIhQA|;CtB1#2 zsb+sh`0_Iv+(@G=zr`EbW}BhXmG2?f;dRC56DP3DsVzFy+H( z;()Aa6RyjYE85t>ez3ZDd{7y!zPHGmA^EYBvW)5F8~G<5isdFbw50>)f)L&zF5+Ff z0ArX#Ze<|ylFx^8s>;oCS?}D3=pd}s^|B{(TfN?QgDz+`tee$3s6U}b45F0WnygZC zbZpjNc)2gv0(88Bko?8(@O20$~OBFD>y^DQDB<-7V~Lku-u@%c&*?(ZFq1U%)&(wS_aXvs;2V3 zYBl5eT-|6E!Ax-9%hQ@KOfP@C6rR_+yg+AZL}dYO?9<++-dp1LzPcU4x!ltwdi=EH zw!6h^+pCfy5#E7AvKI^IL@+P!uYZWrSpvj8xEvRi;!vNeBn)wcNtO#5+Ac|yd6k#K z?t#6p7VuRzYtK#sewA~ZG%=~t2i7AWv1nzMIQg-i`%Rmp;Ixri0_^ITXxw!a(obIb zNi?o2E}7z^*gQ zt9o#dkM$U}?N>s-tccD{ml`gmrfBEYsrw)ALwRb_Fbujrx0i4@zVML;;B-q6djPQj zP9Gy2?OzQ1dG=| zue6>js)ThuX!46QtA{u!Zpjt9T}>^H9b)LrGLM#9@AP^Ab^L79v}h2H1RcA1wewJU zs{SIESj#=^jpfwpHD8q*@~4WbR8pPPAP`?>C0hRyKHQrMK$u7KTwfdu4Zqk4`?{Xo z5YGMSXFj!^*Ru_`;GwjcgpvxA-EU@=)yW$v+E>ya*7u7Iz!m+Ol7zb$cHPb5x;g%d zJF|`hAl&hd+tulZ$T)0+q6)?SJdc9kgflALe=(oY{J{;RL~?Lxcy{fiTSL0f1Yes- z@Ak<^*S+p!=6KTD=T(^te?iF)QRddo00tY^gS<`m$MSsAXJ>U^3+O4tp&a_3KC(oE z-V|sZSvYQO>pFFpvaP={6L3D!uDx{4$s~2VBS)Nqk_K2_x~(671G_yAapsel%Ub_9 zt6FJ*h8>R?#QuddPd>z%2c9aBKT;1&1X!cppw}n=D$__GBa1Gaej|QFM~%z}W>?D> zaqlk^28fqE2zkT&))9+_C=&oh@A`~3{p%Fb2(kNKN$q}oq?WV4Ms{1%Tr{){v9l^)9B@PBPMJ_T~C?)&xl-0|*1h`-$q6jgc5jbZ3q zCr9dW$uO7GKwM4BprM&-^?We^j#c263W81&PT%C+;Y-o^KQ}m?1%QliIQ2uer##12 zM77`?iq=IZ*6q-&ZD^7G=~ec-_%3_y|BV^= z1I}0GB;HVv4g|E}4=$x|gf#ZKy2bMr%+d|mpEd=As-Hm2fGI((d{9U_vG>VGjXN^# z0&-Codg>$`hlk~!;INUYKeXtLz*$jqCFT#k``^IcQiQ) z?5gOS0ay{}%N!7JfS5IfeRIpApzO_66wli56^1Z5!WtMVis!umr1=a01sCI38R_*n z-}$qj$?@8a#V_4S6g3o|sP2k1et*w0^Tf!+I^($$5S*a##quWZL`I!R*)$6L&l$$1 zwa%-c#~JXnDahOS(F99f@dcufQvqElo;o1_Hxwk&?EKg1?$U}RNif179C<+-SQE}EkT+^UPmRlA`rtvCTqg* zS7c2rg~mR@03@bIHdn4!fxUC*o7|NEO`(6lk&)4WpbgwgOg$>4{&z5}wJ6}s@%{zB z?9zY2m?B>RKfdpi`10R>Yz~OlxeC0|cGQRdzwte4Z8?$q zX{oNL3UF?Ql)AqBUkktdaq(&EVXTae?1sPFIep!K?J)ay5wd~Gxqqtur3O>r=B%YD zxPD}*jturJME2y7e~(!0U)z%nzl`L;O-8m}7_<73*0Sf| zZ*LsXOntw0z=zQEa`E?+Wm-DG3mT%}!odCs*{a(jyZ9)B<-xlV#ww%iIBaQa4YOMV zZf`sAdpjMn#o!L;w+Ef|W%3BPpo#eWB9KU;J1&AzP(ktH3o z=G_ob+_XN?R_fH{asK-DOBlLkku*$-3A?i1+n@krNqgP@>$F>cr78mA6P<^@hySn! z9>#xI6-AlJ!}h;@swI6FqDogNC-AJob^(46Zdcoq6H(V!UKpi+FDTsi8*xTMT!*Tq z(w7!LA2q`sw($L6;uR9&@QLHrqt+?vz~znT9C}Y&-7G_;@e|xEj_5x5!*$$R#U$lp zA3zND>sUoK0TZan;~KyOxI;x+6lv-H*+1!C(X`*c_u}Fr$;h3sD6#TW;Y!PG<~)nv zhVwsbcWdL>x9e<1xmUI=4`v9NQv6~K#ojgfK-gExD z(>r^5q0SS_+b8PGB!7FjD=Tg+iynDKA9*B)!U0RW&=dY&z@`6mPZ=Hc+c^I=(mRAh#v|xmVJbA$yxSw1=NH1DP(ii_mdg zn({z!_hG^-h{8%TG1|PMSb1Z>f3@SK&T_Z&xf$R$;V|(!MX>w}wFC+qb z7C`VGn)d2kaX28n^%Ri!M0z$5_8z1B6_xH|ir(Sm4FOU~1On#WZBmsj_7ID77|jl} zgIkdtGbaFGJ-1H(erV$TXTL2EkVr>BFd6PdNk&d>Z2b(dliDX5+&f-czgV#G$kMfX zo*294{gmr)&$w0cp!HdW=FR}8BPK~EpYL}L>IWMM(sR{Bps3YKk|}{ zlpD@)dqKNQu4e=Tk;t8s6I{MZZ)jqCo0Rbb9QWpFvwc%y@ zcQO|%?6zA>#Coy|Fj*5Ny)DZXJr!>TpVoh~+Fwu7ngx=?xCn;;8B=m-@8(A&h95hO z0?x5XsRV36UQ3p~)80$iic!FX8GCuu+vP+>puF~p#rLdf{p0}tWCKV*fIO2ZwDWte z>0}llH{>7bZ$gP&-+O?jfauau#+IKSIgxE+g5YH!8AtL5zqT{dT36};ym1ogaK-n4 z>=rR8Kol)hd={|pS6MmEz|aui2%#EdU=zguA9P&>RF>)175GGukZzC$De3NR1O!1! zy1Tney1PLd5s*~r?ohg0LP{F^pKt2U%)S3wEEj_g@B2M*VxN8XLA3>+`Mx=*E>Aq$7T4VnRNr z(gAHl!n*W-;zeq!1PJI|jYaX$--S821O!qpL8eg1j0R?%PS&jKLD5(alM~cNke+{HMk?HbN=LA2?tLve62`5s5CvirJov#iF zmoIxP^={@sm-ac>oO>V%lze+3r+2NQGF~ijV2-FXDRBX`FIoWQ^xOM3hATj$vF{~z zm0!xz?F0>{{XiV;Y@;@m#KTM!h==rR0G>SVA;A5JsVGNAs{8Qq4S~dr0wi;aPsbx+ z=$xUta5cQv_j)o%@W}rO|9*8R>wV8dT%|#zf205uHx@B1uj>LHH8>*HgD>{rz z;C_a7ePq?z7zti9jY6eBelYQmJ7RoiIERWPfEx8BXq3uy&;{hPA0K*2R)^_wF1u(j z@N;~wchdml%_nYv{U)1coC$U@;~o(0Vs%bIt2S-9sp;o^RJkqcTmO!Ik;K~k z2$2khEFSt_(Z`~(+1u4@Z4~ZuX1ru4j+zECJWv8!(FD-in>hNY6rE<}l%nI(Pbc#l zQ|qbIm`H!LFKGD9vg<(HHOYA8uyMHGXhlvORBOb#ceXK{328^PNCB0SMb~8k9`k{w z$*jY(r1xG_j?LwXY7^UBSpky?T=oYmunGN#wWETi2T*||sIY_Phtm3cDL&ozA$56fnvb!)>P&*}dnIT(sCSN(@69jrelOM0vBA!P> zdA+xwhsNx((BltfAn(?}Ae6J=EhKhva$0Cv9s=nZ#lQmg5 z41KC!_PL=P3AbT86sJpRfobevyQoUy)arbetpTK-w(DsbF|i3y=;8*W22_PJ)A ze%MS?V?^Ax3lRx^YQO2OvX4Ke)W7dofT# zig>4*fcBH2kmqzf#>c5S0jePc7!k{5IqrActc&jf{}r_zCu$fBp3tkx@ZYQ)EY`&o^U5;#%@b zp8?hE`0PG&>5y*e>WnZR)ZG*9$D{8$burGT3N_!1?t;U~=Y9t@ScwVQE3JAmQCz0>s^m%J!nM8?vE>{=(ok!DC|3(D) z&8afrfk-D{Hi+D+1Mt-t!T^6>d35IW0Xze*eRGHDgzS7|syP+Vj^l!E`_)=jUgd@Tz) z899!Hew_;8d-6{1RxH-e;lJbIvARC}#@}G-4^|MFOHAHKJB}iTpOqkd@Km_f>evqs zwQmR0z01Huz|?IMWBnbMldeERYPDdwQCB?s!Tm+6uO@Y+PX}gpteQ%c742O#h%Qd+ z=<%JB?iK;MgBzJCLjB-+qJg)D2s-R+K=hsx9lzF0_uNXxnQVx@yyMuJ%phU;p`FD75_)? z=xrwEUjd0L{P_l#DQoZ3XZ!_tzkrI{HV}-QPfw@!DNP9g} zeZjT`zk@r5N?tGnoes)WgEp7r*RQ1-M$4EUIJGOPXCQFk+jv-~NIi5(Oo*9?;8|>7 zOP07QG4;QqJspI_Z>JzGSN)1Cn)Tp1@agRv)&-yfB)EH+<;Sh%HB~@@{n@GUkmp2_ z1S2&c;|Czhu4Lx&M=1zqA7^oa8)Vd;b2Vf-p~N#-h%J*}Ym{E!4o|obsD8s1i33i* zw7%olAxq!+J>n;YvB%ivPiE#uj5ds~+H3NOZD1MQ!nii|m*2n-ch58|Rl0b`HXzl^ zFLc$fq+kajM($C$?=;agtG;pc#va{vaADaJKlm6pL<{ zyr%5o)Dsz4wkRavz7#n5M|6Kg}C5c6a7Oln@H@t&|}<(4C4ihgPhnE zP14b0m4xojYJ{J0Mo=*-oMbcrpP?{fj%ep;!{Y6%1Zb;1d*|B5cK*X^HAX$NC9_U< zv>z4nbw=upO+v_Jws+)<1xqbz1=G?#aG%&pVS0=(P%+d>?9LJ8N28~;SZ1`hoQ4({ zwg~J^e#oeizpxCYaN)kXO~7sK&`Y)<{FhB`vqCC-qRGkQWx(I)L&C;A(_0KdIK%G4 z`?70$QwT)BpfV}T!6sXQ;tj-z=J3nXzcvF=d1>Jc3S+G4lCgf83f*WDzv9J~d7VYl zPUUf^@wp%!U&FAinHVXhArQ%`YQnZ3IVCKowuWUQ z=1QAY{(Vo5fvRH6wvVbMtRvmgp=+z>US)$%4_h`%@kb4d_1&rr5uuaU7FAsTNr?XZ z4UUv6EP;*?+geDE>I!h9Bb|pQmyJMEX?1-T_9OM5S<+uQ5J=WOXPb)6aw0Gw`$HuY zdr3U^fzI8h1&6BFjFCEJ*#CZpe;Njwz-VHH10n;nKmW_4L{&w1kcxb2n=ASoA>hyV zVI#}kTaz!L1H=BCIZ3#fuxW>bjwkA)w@+N-kJ4X`vhz9*k5Gv5APc@lcyC((f% zxzuPggj#eZUXF?9{&M{Si2LAM90YRt*vSh17&v8WJv_mm3o_U)ZjFV}xuIo0dFSdY%@+^JhWsOUs#toI z(O)O0sztpEyDX<=26TTXyg(z}I`V3>{&*@e93XU@&(H+25VDj8KsZ=uvfD~C^#R?_ zp2L>9Y?CNWYuEJkLKZhjzkF8J_pCpj>bNW&qwu^nW()2iYXJh7XXET^!06>6sqzClh)?X}njz&wY z^_hAfXclXG4eVQur~^UML#=}^zff;RL(@y({uQ00fogOo;Hb=^%~-zwd=5f8kUH6O za-RFnUw+l|alU;T^Q+bMzU9rsZ`$U=y*B4F3hukp{C{czOwT#B*T3le*Ye^$4$JPf zD=uaAx5wmO@q=@LhTvRD$kFrJs(Sy*b^Y2(U=Jim!S(*`Q%&L@OmBC&8fx2mckZwq z+!FdR@?YsHDf2o=C-7 zd1Uu3R*geE5TS-1D6CxoO3w10*y%qP<98Dw0^q5K3=tR_ZLoU3e(h&HO2zv6&80mk zoboLxh8iVepr2p7gt7<(tNTHIKJD9lU6wourhzE4E}RIftV)|W4xB7}IL1R|RyPIO z+lbj;mG5m_l@y#)vr%mY_=ihXZwl2o7K@kWK8|kmZw{u6_Vr1xvRIh?K5zKH|Fowd z-Q}?<`5KEr$>bkJ$i<{ly#h#PvI|UQrvLq~fB%V$1eji>lEvh#4si$(UjE|E=lng9 z&AD71{U1}V!04&pzxLbpk&}P_Q zDIg`LWGq3ti&&xi*`qg_nDKR;^mUxsga7Zn!o^og#J@*glf#mu*}?*Xl4y>gw?p06 zC{He?`45r4Jacd9l=NY76B}~}`qds2YtaVxmgN0?oLAT%voF7+$DU<+Y`>(yfN6zD zM&jA`Q>A1A9;Xjdka@#K3RTRAyVGHElYVkS6G9NFoprZmPT8+S$PSuW;-9m~{{E7P zGGM_(vvHhdAr9GXJwpepp-)Hh-i>8xMO!xgc<-$ip>C~~r-u45-(u3gEnDbZUV}$} z<87Q=!}F{ullZA^C+=N>0)#)_?yqGMevH=VxidzV)7h87^*p)7<>18pIU5-GB~Cr| z+4Dj%i#O@Ho?s}iruB=D!#7+)ODo4qQJ&i`Fv~c-5%%xBUCWg#O>EN_n_h355(p*; z!c9v{{5hCT{I73FvmuJSyOimt@9{h@6b6w!PREter4?WSilv!Er{*5Xf!0nQ#w%xisgdAcv($b+7hyqQaG7*TX(R$AhrjRrmFY~i zJ~6LTUz74oe!$z>ILDyLYLlt$NCW&JyR=$&MFLbkF`T~+)nDKE7ebHlu3{Q51ME14 z8^GOc06u1Bd8|y5Hx`;CFW&atU(%T1iX$oOzg7?|rLYYq*z0*y;e!ZaFX&0TjuEfZ*;L7%KPweDTPF?T=nl+a!I+`uCpxxw#u8VD@lRuP$SR@B$)KC)H-L*?A#DE%5()h`AJa$Xu2K z3b!?)d?t6?3&Mv;IPzMs1dadeQ$s)a0)n(hG@%6tC2W(7U(rLUXd?HoYOgj;Xr}7_ zzyDm4KZMZSr+FD*>FdfOf0)G>#MYnYhZ2kPQ?=OXql)7L=^?dI*j4&;ytGt0~=@VjJ ztdkPH(;VOXJUUlJAf0wF&t*J-MRbbGpb=YUIMw){6Z@Z|y9hp_Vgp;%@jD9t4~D(< zw#$OgB$vQ<0JEg3sLEdz1ay8RTqfl5mqSsA<_;q~+ZtS)HVc;^oFL}5BQ>H2FY&X@ zAMObWKOT6zBiMxQx7hQaJsbGgeooBSTbBdV1TNkZu=)fX7ET9qU!P=7t>>cuFRStv zYE=wqUUs7U3&ZYBzyAfy!1u4^xNf8{=l1qJ6rQ8m0^e}OH@vP+n(n-Q0XniCD_27d z^OrWQ2Ahv62w&V!w|>Q}q|(7ixIx9cw?u^Qr<^`6S=flJZ71M0QO zGBk_lhlfpaX?OQ`H~keFAIQHuu66_ulje%{xJ}5=j9c@`1}olyH_uOqKC*DW2VgSG z-~;Vs{9Al0c&gqZeAL7~c7|^-d&QIZfih3)t!52K7O<{>GEe5cM%Ed=w&JtzG&wby zT&^ufHG8R=nm3ue&UA?I(d(y?_-tlv%$c2s)Y} z5>`1Cm6-PoNc6Y?sf=U8;d%n69smX6yw|0Cz4n5F>Q$+VabGMMxSVbuExwL1h3EOp zdRlC$m%V&;rtn?6$zc;v?bY+e5oPn1l9_Y}j?dzxNZf%n;D%=N@iNblA7j(8+bnRi z#%1Jqe1A5j%NoPI7FH0=`yH@mk!J-M^uU;nfQLvEn3LxQo3(x)_=06;wo#?^f@&5NU*m&B@uOI+``Jz_kV{16wGy|`35j@+pRgnxvb zR+j*SUGYoS0Yt{NBFM_)7tUEDkc{!Pe2EC&s>#bFOyr<7ZUWU-eP~}vj^`l{Avdu` ztqBS?Y^|_AtQ}O*K!2&#*S9U68^B65O4*Sx9IvXh(*y(e07QAem0{ZlnO7s;jMqt=pL`Tz;)rLVbl7_VTWFzj7`1E>4~0Px+(Vm&o9Em>;lsXW$m*TX zwhEC6*uw%Uiuwa_jE+I>ooM;sbM~quJP(I9e+H-Znyo)ll1Q4Ri}1%e?EmGG!WjJv z479Q=fXkAmg_X_$3uTS!j3|R8F&cLo%ZeUWS;;oae_G5h!1Br zOP_c=n0P4;JGsB+Z^XL;K7Gd00n=vT2QPr!3}N^a9$+Y>-ILPhxGuS;en2HLC^!5; ziO2nxF`nt%$?E3(z=hT*_!6s@$)H@la%(iF)oQjzRgovyCI+7k-Ws<%vr;C3x)CoS zJc8xt3pP;~W~O3i8uWNN)TwBHKL7J5ZMo2XtEm$7F^1t?X6tlRH1Uj;LA(Xv{Ugg= z4(1A3gpDP?NcBqjNsgJWiYq20k#B>cIU?vi4}=bZsQv>2X0MAyS3R(0vH3#VSr$C1 zXm3{d-P*Dz3?qR2VW+Vzcd&ivV9=x5vnMJli{kYl}Gn$+0}#`W<8&GWSoQH7Gv z9O3zG9W_xuipzqeJT)w4cts%*<|~TIDqsrJfwfvwXmx*k^(-J!3H5#jdAQ2g$}E6L zP=@U>fdH!`Td}9$Y+o)y4xX?dkqe`U$-91i!rGz7!6*MK1dBrier%(*;7hphOUMyL zPdIiSbJ^>F#|4xlX>6vzBO%y9B1o%X6_;!8WY6=Qt8 z0R0?PxW4)jDQyFEreTEH=uFUw6t!4yJ)du{C1{Z=Ve%%0mMD4mUYzWsPBER!Zk{pu z#wz^*MAZeijy=CAL9q6dbf@#9GBk1* zz>-Wf?mfj-NZ6`od6wXB6ilC68EXQ=`OZ0B##RB3Dym9$2f5bCk&PqGtM0YwKB|D% z_hXwa0x^0iu?AI{kF-;faVk^cDoUiN0eS@r84lK!5U;iD$ev;IwdS%Hd7^FOemWr! z$m{)X?Ob^@-@V}DbN&Jr~XBozk3HF4H|K<&OMRabpZacuCT(Uwj94S zY5Ovg;sVz8lQE3TaLkYlqH5Bx&K&kK-h=q9WBA*C{Q}R(0ElnxSwex;?bqCTRkkay z_;n2vlF^8ck+uU`^JeQF014M$!IEu1RXi<9HLEf%H<}kPJ!?m-2b}J z(ncV1!o1oqu_Z^thA~nQB@$Qb{)HpSL4gIwiVx$i`R#e3B^~+`TJ~+r=ZpAoUC&{i zR=&)&xZ1A$0_E7G=bz6oSi-X4$LGef!js?V_JHCeV#iad@vYbyd6zEw*$)?rq>xZx((v*y+p~C= za%nh|wa}3PF)F~vT4Icr(Z5<+9@)Tml6=uF8y_+yFO2MDYPOOPJ%(?`zU{)J*}hG` zogVSdWJ8BZdgQt3SQcrHMgNE4A?b`Xc zm=V5~OfWQ@cH7dQop2j>JQD)Hu`ZgO=G_>o&ePq$8siG`$eGxlf*e-@hUK-Ud70I8 zE+t0{_NW^nUs7rn!!`>1lPk(7GesI6tZMtjDES+4@&+f9^VNXyOw^?Rw!B9&!mfIP zBK`7^_{axLqtbN@3%KPzm|F%Q#RW!`V#wyqc`a3j60PGW#&f*Q2%U{rlYMq}CbiUd zUyUemnBxMJa}c{CUqlFpkA-^#v+p;uc{+s$|3IJ>X|o(4Cl`aidR8d#m|LIs`?pFt zHE@*yNJf}$ucTz!Pxe`Gus+X3qtiq4bW$*1EEV=PgCf_C?$b8LDLAy_#ne7s%HuxCctytQJ zu@i#TRf-dg-ns?0o7f3q80I5~^;}|YNCellLmQcC3O4F->K4)QCx7`M>02-K$HZ_{ zFJ8~8WFWqJFY{IVfmI>nYT@{Mg^sNRc`Q>av@u5B#})wI#V3z;3}a{7_ImVs&c&Xy z7~A3K@*S2(-LrApm9VYCe*I3Z%pc^YDMZ@c?kin18MSic<+d0)AO1U#loXz*t%5-F zP}<0@(y;!SF())JL?mI_rPPV1=}34=NQmlDd$Hv)zVHFDpe|C& zLbJ0aQM7qhSZBq3=sfpJBRKTai$EbSJH<^eSYyEwvk?@>M%F2A+TS^$=)!MXx@4q;~>x zw$%(P^dkJZ?HpglVA?6jT^lyR6(8QKqc$-W36GvUyiEd64@Ji<@5r#NDL>T1LNRJs zTsVda;y-H8U&s_rLl4Sk|0Sd{*5N60W&RVZmd-7dWx?o zQCa|&w#l9lYv5zS3oTfgxtHb%W$5o@?7|>~h$OK3ewo8aX!|uE_koXfkzsn-SBK@yhIcSe%c&0 zxUU1hfHqH_EXeq$x?t6?W|FgoBTY@^7z?Ld*E1R#($$sxQ0}}(Uj7twQ1%Cw{ZQr_ zf+-%M3=P|LP)jGy3U{vJrx#Et$ZK9<>m+x5h(?eZusa{k2dWdhDq!f5zz-K*ttq`)Mj5Zp2=|XO=ktVnb@vJj! z@iO`O6Ytk?nZ!Rq0)HpcH|6dR5m3>eXzPxm7pRJ$OKZbI>`14?&h9c;jUNxD|Jzkk1mX0}#gCZi!0Cg&IEv5CrgS3IJ=-Zo=;1gM1yWBO zH2R_+U}@D`eW0=U217|=ppg)`Gh~uRCWM%hsWuyXKC8ZyIPh&ED&oO6T0KS0mu~{z zrk?F{vcUy8_p*i{x`x`HBA4JVTL&2bx?GQGd^?|KH~#Qti4HVmZJ`ihX?zzGGt@bG8S^RF+&6MUZ$~?vG%|Z`QM2*sUU#v zdP8epqx)Z?3pWYlRqC|3Br2>CM4}El1So&Q>|ARk!*n3{iWjQa@##bpM1WR%G+n3Z z9F}2CB4b90FT~x02pv0y>#d{%%?3g$m;>ZrpzV?e1ROI2kV_U~-42iURV<1N<0`>g z$U#IgR_MbRH}Y2GZ{LAkN?_M79e;PVQJ(!u(hZ?kHs+IK8zH=#(mHt*^04HY*TgMK z4f>Z5B0*I?Z8v7#&3^$f8@y25#pm%940;#~OaLtA1lNFUC^EhZ$e}?>B%!G^6tQj+ zk=~D^KHP;)o$8<=VMHVVLxOzi&WYwNRwe>bNqz91p_+`!GbWDqsq_T3SjLl|BUyZ! zW^uv7{o;PqXJ`iOFr~8yyfMg8c-DkJCgWn4>$voZ|K(f4`~W(JkXsrCfLWO2GBGo= zsS;&&9ZE%5R&m(xBjd=j#?VNSG#HVTzSf<7id$-(X_$tvT|gQWE;I)jV`r*OUtdif zLiLUi(W_+kk}+3t>N}QX@xSrT4I=P?d3x13B7~XHQn~F767x8rIZDP#b zznyM>B8pi2w-n6OEt3QvipO(jUlb|lhfjPl8z(pLsL*bTpPB3%E6(ND30T^7GZjZy+ZQvwR`)`{Rr#h1*c- zF{orAuG!cM;{~%Is{iPHbMCzN%^PGJ3?CZZjtv|?GQF;&Wl4`Ml&L0Dp|5gcmi*^_ z3KL*}#5RORM@&%@Hi25945VCLorVvEh!zY=_@2+zm)OttpkGKmlpytM8$w}&a{#YK zj`j$)WT@EKrrmm#Dhf}bf`+JQ`n}%h_xFYF&LLiqowe?WQyJwTb55u4D+GxY{{+8c zJ0K?*E~hCd!}{@fpyl9C`bN2um} zGx<(|Vlu12W>GeF2PD;97IgrM7Wh?nYHFSN)n2r>y%N5+?Df|K(D9q46=U=HV)XOk z=w5a`8oM33G&w9cU#(i@>#vKZw*~0ASKT7oAgnto<9$8lj(#Zw3v7e;PF|>J$nLFB0fUN8v(xVM`=|G2nnW>_guUB?LXBR( z{O^j8l5JkTQ%(~#CNGsR7AI9AA#XslfLm;Gh#PRM%AueHYf~!z=5%WqSVw)eP5|3s z3#98L6LEv_TqoP3dd*JeFU={<14N1@VmWaBu{&+aH7>_99ru*n9Rt=%Bt=DLB?jG<#^UMoA{%{ORF+fMss=6U*lIlN_=|7oJnL2 z;R%`%3Xx#=3zYM@jBRMg8>MldsT6|T)@;nGNH)nppwme|&;IS7ATRWK0C*R`uzU_2 zw6KUuD}Z2UJ73ReW3YPeG*!~sYzGj!4#u5jpV6nD$u@f5mvxiL(d2V|`mfv52GBvO zj|p2cCLU354XQAZ7*71_A-2&jHr|?P?+Hj&$g_W}`zp8#L`*ER5yEdVHClb{{9xdq z;wN^!O;)q4h1&o-VqFOMGZ*n+Ab+!mlP~@QyVZscEP&lLI5Ll8Bt=*r~dcOc^%S@!$uBl)5xA=n89s!Ng z1Hh?WJESkqfaymi;(pQMVjbX+3D~i;WhM4}tNJu}`0pK@Y=(v1#IS zJ>>VgJhGeLTW%HN+6I;Zr6G6!k5?kqN8DffOi1MclH+o=hSwi0J{af+4=JFIBYJ+D zV)@$v@b4l6^#l8nU=MQGCM7VIX#CL=5~zYh=l6lL-N`1lA5*2O8bwA7e=KId#2a9W zq{rylxRPo4!HhRLwX&B0f`U$mYl6>T`seO?K|%H>rXR^+`$eFW4k^bR+XEinxw`(k zzXM2$Cy)!f#f>Za1C^uX^M}?FZho}(lBJIk-P*7gRmC@%=XDqf)qZdN8+0d(jM$~_ zAxIY@psEIrzCAUw=lQT8*rW_dNpt3fvOk?9{trc@kYwRBkuQB(&ALm6D}0mP|~a&tN= zBm`>iP3@@ViIIPVeuT2(vF_tGLNI3G(@)Z%4@a+JUcht0Ig+lPqVf*26miA^}B>nRfWU|`|_Sl^OL$89zE zzV`^!X{W1nT0o`jKB7VC0IUr-Hv@^!RDlHs-0*jd_$Z*~{%@i8EoSU=t*?3wmOq3X3Y7yK+G&4g+&AYC z*eIH8H}VAz#JlO|a(}#(wDoUg1v*Fi7r74xZ@-nl`T+(j${mndI{k4FeydRJN|M&= zj)DT#!S%SCYEauj!*cXZ(l0LGP*h^sk^xTZxi0rx&-)oR5M24?F7W2Vu59ax9f%QYk)EkZ7v$fxZ7c?quE-<$q+Q2h|NAqMmU^eZ87 z>WJvl1hAmmQt=`$z<2QqK+|2y<))+A_9UxEKtVF=bmvR!c{lWF@etXq=U2L?7iBn= z>G1`VxnNP^BZPN{pi?^yHAM~Vo%X&(X%Lx>{oI>Ys`4*sD3$dy{YCLAa^v{d0;8j< zbEFIxQ+7d);{eYAU0u1JiXN7X>-m)aZwuw$huF8r8V?X2xu zIFKp8mZJk$^s+7{d{)X*=sR-}oSJjVT%KC6dT?|SpN4$nJ3}hMb1S@0W=H!9wMay& zZ@zu3mBnSqipF9$0hv)3bz6E6bPOr0u`CEV#&YS^D^h3f)PAU>wZ-ct*AY#tjNW6O z6D?=k-%O%pE6yC28ve06Xj<=0Wk#Fx#Yy`S_Gdn<=K(S^sC{{#-+&rPHjg8;bT1fY_%2CVO|y9;2& zQl9I5v03CL!8lz=2_r(B*K%f{LgaR=PTtfyZ!Pql?J!n#NR{C&5kY$7eA7tCwn4Ke zFuS?HB*XjIS?hO$y$Qu2uT(2w-umJCD;G!-0~u56`TC!%4#X5A@JPrw&s4gSQr2Z0 z-FueZrT55qmQo3%31N%$=sue-zjJP=Ou+P>%XCC}35ic?d#8{AzUrNJC&0PqwER|~ zQY72Yi}90h7UAr!*|ydFj{T9`%xV2maq#y2jkfN_oZU@G&_N+Dh^4?8hF)rVfv0g> z>>OU`(!8+HaXJ#XIvcg~nAGx64dL|)zR3n-zU1TC5_yW8yTmkvOR`wyeY>&LqWP3- zmgFet5#1p72ako$#0C?w4FFmbib8k}@*>}pewwD4ryN;=kgy*^Z;o9RbI53>rOR>O zM~m@Bk2-RW%2Z{SH~hJ2_`og2*{eQ7?<0l<*4r2q#UEpzU~^r8x?yMN6IMSB8koeo_frcgzAOp#ap5TfJfoZJZc~m54LuqWm68|vb zDj^_~OXNu+kba8^t)rx~o3EM1BH=J_Uu`7NzpC2*`%5+{=8A`7aDhV!>Qx}S$_IsI zP>E`4TB36Pn_~d>FrOw@V|1MDB?@`$I>hJx3K{^Zz1Lk-l56C5-JR%;DYE*@m0*Ml zgK}Mq^PXg1MI41xjrtK7Wvl`cysq7B5cd52vx6vc+Oe_A=s9eY@Cu+uTh7&$=7fM~ zcn|!?4d_-lGR*Yj^fj?rb%~r#0RHaOPOf|59uS{yK-f~6+y(9H0p=>iD&SQa_~8L4 zq6YJSAGrk(Sjj+ELfP(Oi)-yS9F;+kMgcobkOJNEOG>-z-D07mHaKWyo*w>UchVH> zxFFJiJY-F>E$uYutASC`f`YhSU<+pqJ6kgdKqzM3fD}m3)V|#qBuu}5hFvJr#v|J` zinkxZ3z$w^UAMd(#V3TEpm&Jr)G#fTlbFxNYcuIp$9F=4LlDTc+O2j}%XTlM-eBTH z*4S7rG{z+rfLIi*2AkvwG7<_{v;4STDKyc%B)cy!e z&MWN|0dyhYV7UB2p{>^S$5M)(4^PadE+{KkkeD_{&@#dvldWW*UfQzMs2pAwyl>aZ zaP=5UWvQ4Z=GW?6Y~N-WY4Q0J2k<|GhNa#kA0^W(AR3qVQ>sdg+KqPga4VbI$>d#H zbXtQ%tfZn;OrP)Rd%M##$ieOVK#mZ7b+YMrb9E0w?p7lk8%7Mh6>x`NKeG`9`6!>u zzD5K~2U%G@7CeaPoi89TDe~Xjd%O}Br51NVc0FDW=ogEuyM^;hivq6CYL{Dmh0?I; z)y2JC6g7o|BaaIc6!YiR7Yv9zsUVIvU9R~BRtG8zTORy&lbjt290Ycq7@Cl z2|n{9L)M;8-3tf{#Foon?)GPEfvli(aVYY_B1{JC!$9Zjf}lUlQ6V9)w!1Dp97`J% z!gWy*OwD~tt@e=?-zw6c4ieR9))@6eYwn;aj5Nh`z6(Tj(E{fhpFHBsG_?>vegH*p zu@L0d!}PIa`2{F(A!2_7N?b5GpMnN}U>;P|W-4v4hZe|EyHsrkm)+J;qHEYO(-J!# za3s7+>KC{1`CeN5!n2=$4aDqV8Yzg5*^Dl#yJ{=mv+K>kb3&xp~dTy%^T`CRRvC}^<_T_cD0Fwf?rSZ;T)CKS@K$aS$h_RP2 z%_DTV0>_Z4@;Uy>;i^&y zFab87FV`Y&ubsY6?ShsGx%1VMUpyjy_}w6xNK?Kn%`}$;x8Ul^MBgI6PV?D~W`jF! z=;i9WvXwM!lS7^u1!qO>RY>^(s0e@P9bz*Xdh@uw&i`r#P+o_pI&|oaXtJJ(1M8Fa zi}|^Mi73j|={8M|gmBB$8%NG9Ms_I35>hca8btF3RZN%GR)v@RFiCzd!FeMWjN&w zJ2b!yCV0)r!i6Xq8AR{`ZGRyT{ku_{AB>e?cGv;{flm*#&01g<5oE9zXBYKtMv3BZxyWC4r97HK zPB;VPk;zZ)r+^2r2!hy`BT-KY9zFMVyy>NY@z4MZRt);%W=(U9kLgE9Ta4CjI-jjS z-uU{mry~$jcMllM$&@K3wk`*WOA-5gcQVjuI(@c^q4L ztf~yz`0K}QN*Iq>PyImxuBmgFhTS@S$n(T}OK}J$HF?Fo?NdZQ9A+f7LOPmglqOLv z($owvD@}62`}v-aj{(b!^_RX)|EaBslRQo8+eXV3O+H7OEoFHLD76)E{y-q`HuR#M zAhz*RSxpOne%m#(uD1t+Fjt`qq&((B{@fl}H(a?|ZPQR@PqM?vCZv`Vn z--aS&KSv?tB>HHbC?$sAwZOJs)nS!=^rhtM(0UV>FpY#ums(JbTm72z;7!a_Ka)hG zXP&vZz9kk9-p_A~k3W#23B;xLZzP9%A|CqMJ*7Tj<$7S`fE46s>5<=(TlB%mn5I^I zMetTK=`T~zHea_f-9cBVS`eH;dvc$wy#V^!V;8Z_O%TALT+Jd^@UhNQ-jF7 z#DM9#4GOLy>#B^vRHO@0Bz+?xS?Ss=mX!?Z9GA*_f$|kjM6={ACuTI{od){6B0X_sDpgSZ9=`$t>YJ{+) zaJFGrq8zkq1t_seNT!~)&coF+C_ zkJ+9iAE4#bg31!eJk38%hdR9wak3FepR>)R5kE=;=8^HUtaWT*zlnXlgT24Y2wwts z+2n{)EvMzRX49jGS;=O^tP*Qsj?|sg%ZS$Lmk@^L_e01R7K=<|lgI%>n~oobJXF>! z07A&W%a6!E&W)7j9{SOi`-PCN@mT`o>MSigVXGwVxPQzoRfL)lzAexEqzR+3px#T& zNTRx!XK>r9V|LWxl5Rmt6))yNCraT6PSAk z_x;|-X#Bey7YO6b-aS=uypU85G8>SNhrjZ@Jl|?b946`E<0L`MP+)2J@aXf9b;;YP zy|>E@8J@Z4RTSlXIkfPNa{%C$;EoS;;G2y06hBaZRio-CZn)CT4%8 z;YVh6L+u&G$Hd?reL~x+a2QP{ptUWgNZ{TzvXPud$)>N3e@TYQqw+^e7AE`}N~2RR z)(TV*gq0rkffyGGQ}0j{Z!M6HXB-8u+JuG+a$&f^6ht>=bUrA4Gd2Gz`Ez{lX_S)5 z>E-kaY9Jgg#|jdg^&I=wp8R2@XdRtHqq#ors2)4_;I`s};K7N7PK}Pca3+%$MYVn2 zdClr<|8)L{raSutE`^eY=vo#Qy6u|h#nE2i+JpVMyl&FGD`b53buI- zU3cuc!a}j~>0a{?G-t_{N#vPCCgv+9qQly+Bt@2_e-*>B$-@`q-oRKRY&&TmA={tj@J|i;nULXrVxHpV^!Q{rvc56XDn*X? zD#PpTQHmUaC#M5#@L>#o3oU^e`An>mMM z_J}<|tZ=urh{Ytu=0y{BR~zeG;OvJLZ4iS80iQ$ zf!^&+E{gHDF9?H0;Q-h8&(krT?)qUTIGIa%|L+5AKA&>XE1O3JMg zji=KLj$4Mj`jJIPDt6$&NT{QvX!-}EqSAb*FkaRe6_M{{%iGKH!&C=O-~@TT%adJ7 ztOV1Me4SL=%H%)eFf4|rw9R+su~cP~BYPn!9IS}`Wt5Sae>E?GZ2Qe&*#fcS_M_3` z)czyCv{${U#K-F$mPY)GuDHst|BN(ZLFUQMzf0gSf!5{?7m6hma8jaNK;p5Jmr00s zfeo$NJ4vokk5M+jkj<|}7lEXO?8!qThn5eOa-!H$6{)e!HH0}!6BrO_w;#FM2{{Q*b9Y+$k`v-#aQdY z5}=re(7~`5^wRKV^&Y|T{`eqAW3RB5NS0+NX@z35Jw&2BihVHb9tw!i@UUCgV(IUWYelM8V-YQeZ94;j{fFg(ToRILtV2Ys_60GCEni zd@vF-GEG5BW#^a~9~!vt(MOT$kS~YxLLm=0#T5@I*xB6?ESV1z%zXX@XTOWAv3f8m z(w}$@Gdy%l+AdLvAq&&JPt&w2q6y&trx$JE@FEicfxcDoY*7_v^4MH`GV0Ka}%hw)jGs z);()6;nHXEMzWVOO4=-028&>SQSC}8G&7JzsQ})Gw1TAA@X#V~+b#2%s4sKNI!d9> z-8GA9{wc|+q^tFx4LQPP7zi(ARGc$GdQM3Jvov+OPA}rcyWn8&*`LS4nz7w2^X(-W zynRKwa3WbXX&#-sJ)S@!AU7pQ*#mM@h~26_p`aJ{1xd*$1XH|Z6P_>+QvktyX0+sB zD-&%ZQ9^u6{q1~eESv}&mAn)v10lZmzl_KoweAE-v&<#J()*3*XhORD%is;aAlWRl z&BGJ{G4(KfSR5|Hg;=x`SOu4kAVf&q0|YMHFil9!8^0j@)veeT?lp%Mbl|ex5Xeej zpJ6DH!(=@|A?T7YHn6ulu67t3ukPi@@O*Wf-emkROTyY^x^qG!lSBKHJK=QJkHS@9 zD@>w`#<;C7iPJver5(@gkQ32hY+?I8HWkgB;D+M)cNpX(lPU=MlHMM_Ms4wGLXQuI z@Ob63QcTVO=HU<%!J}n!jeJ#7rP^a1kr_-3tW``z!i>&G(E>Px7a_8Tj5}_IQ{$9W zZA$R1s&48he8DF>jiuDPc2ftjkTI7u9T25ET(4r;d#>=Nkzsx^>T&emdNag@W~T`k zJ2v{sw}(Bv3Fmpyu|Vf$S0+ul>DP8gl0@|{!)`yK%t*p2ySz;Z#0pWV;rCqtK;AZd zsB*g&x}T8{&a&(?#By0*=qcUYxB00yz<>Col}j(7n>eyzS%`u<8%ps=EaDeMv5|H> z)aMJ#H1@hkNVFsl(bJzjU9X1-5gXs0#%90YT(}N?yi@sS-H#L(V8SC3mlm%fHDrFW zVOA+o%Gvz1g+sfw91(4PH=*MNQtK{vAk&k4u~2~UUM8B5t4bk*^HW+FJ{d>qg=2ZA zz!O_bj5*j%T%L&zCQt?;a5WSq!<>pSHhsZkZ`FQ!4O&jclg`Z^hNsC0t)9m^jdOhn zqH*2Qlo<_*Z1axP)eomf^WdtAaOy{Nmv=;HJfpcZK}JD%mYIu>=lIL;^W&;m(g*x> z%t920l8t)>LM8B-B8R06`Rrors8NrVvkDez{wl@YmB8jJ)U_1OuJ3TY$n9laY z@bF_M!O*k*SrkPG-k9%PhhdhMsB zXg_h97GOqzD{m-z^b-(z~Z;zk>w zT~J6aHXDK}6?Mkr@D54CVRU^`*981@98K8Es3xQGERu`Ko+foqYEr$gCam>kH0}O! zOiyX?k7@6l;XSU;oo7A9gnZy0G|nF-H{VpXS%)}vFf}9Q6gI84|HdJ!C8b;lS|OGv zYI~qRr2}WquYLq*WGs3H?7g2M9=xuqZu@i7>9dYcjaMU;-R-y}l?T4tz_?~q48XHn zw?OGqtb=}hMBB88k8lNyh_C#f+u1nC*EtVm*3hc{MI&ppNau@hd0oho&*|trH=Or~ zx2!{XswMP4RI!Ueb16L;1D1=|Sq)&?mQsC1$ zlnSlZA7XvMV;puxe~e^)Uw0+1RMyX7k)2A8n?H+g&DrEpwFHm$21bhr>5`DrBk0)v7>Tw@ya9s?VWAvTBEXPbU^cBk@>ZmC`^9(kTvQ6?>QI(YLHpNbtiPJ?&6X zftZF;+QJTmA3R>eYc4Vk!q%Oq483{88RL((&rx1OCTFabr%Ivfm(#|#S_LFKpVE4(N6Ta(DoGD~w^r|q3Sv@4 zhl{PvB+YbhXXlt|(Dm}HG@w!*JJYvC^F4&vw9}tnj-=4!B;(sWQu(R9@vZUbei*Xs3Ef<}WpdHo%tOdFJTX9bGvuVbyrjY|9QOy-mSs-4I{i)s^? z#Wuy+&QmlbN| z>h%Y#u^g7)&>RDE`)TvKw!~=dm&S(b-Ks^{jRP$s7#?AXH-}YNEy(6cqCb1+>*&dm zw=M4a>(F*oKiQTlSAn}TvDd@}nLnx5CpIvL1SqhG0dUmxXbqyT zMz!9E{^rhz{2}I39+;fFUX+0URF?pqHa=VRN2Y~%sj%M#nx<^GCnW>1V&p?|T+fUv zp|_UIs|?rY2Yany41W+H+Q|G|hI%kc6%~a85J+~$>sPskF$(s+RT|zhk?ux7K)O>&2@$?CKF|BU z-&!vJa4iSszRx*(pS^$AwP)S8$UF^d83M>h87jUaR2d_d&e(Avd_)$`p%=v~Bb!sG z>Es=fCxx{3_eli2RS52-t zx~*dP_B$5?6psLd2sx=qX44mJMCsW*`rd$}?*-7dD>-(9*0f7s51r6BGQbvEg=5n} z7bVg*KpR$6|i7Ja#mc{&!m%F*?8@)by)$B!}^&W_@&@5fv zyWrf6IfkWC5+9Wp+Bh z2dcWpgJm)Ql931qKhVB6*uO4Or;Dxbv88P`1ML72n_JR#EmDeNa5=f=#Q64cxJY02 z6X{H)OSi;Jw5d_m#&c5&al^hFqEB)S%fEM)0|+o$znoqm$klpn=+0&dlHnj z=UY9wm`U06o0Iex^G~3%t+G+{gffEV$Gool(?B8(FJjJPCrZR%8$v>my<}x&M|3@H zJFP{&>~wE2+GoZ(%0QNe|=F4*M-Z+M5^@(nd25ynii4+hNO%606k9nm_&ls!KTxs< zd9>|fBNpG|X$x-4SuW{hHXsZgtILk$8Gp|g>v~iuy0gv*zZ`pRKL@m5=57!dg4Ej| zGsOhX9BqCnIGHrXiI)xZ6&Z8hTJ}R2Z8nyXT)NU~^4c=nHrYWm10UbM?_#1e!E(Uq z5V@~*O63uwd6mQ9o&~=m(u#soZ@6E~v=mlYTt^rdjR#p8UV!kBDiEHFr+^mgdF6?7 z*y#rm+m1FGX^*iX6JtAkjqZ@Gcg4-gC}d@FwiIX24U*a9%Xg}&17{ti9; zJ3yu3noYv!n<(i4NJ!Wu&=$!Kb*$Af%+^UgFpKX?B6?Acwh7R%mfO=|<`>0r&ScAt z?i=V7ONO!c&J}nwq?`f#tzEr1VJ)KV)hLP@2IC*U@aBP!@scR8{06>GE-V!;%|GmuKYG)py;ckzO~6z^Z9+&7&o^Y)-z^s_(m=p`gz$$=O@1Yyl7L0PFQCz2 z4zf9G;`t|$#_tPJ@fhXN(KuKKkHJf02Eji zVIXH{2ZunEkA%uM0v(Qeh)dXU=k~W-U2WIVO$c> zReSp12k^`jo(bRKTKk8q7;f}MgCpi^x<&Th;sjl}_q{m*Bxg6zd&*vw80Z2riJP% z{QB<-qK@-I!gZjz7%P!YGDQljUu)aF25=QPr>B8SgDeM!7JE-iDQJ3NC23;)@5jQP zyl*-wPD7q@-X0<1ww(C#_BXU|22{NPPbDh))}cXC)x+fva2$cZ>a5ST6cG5S$=a~! zbXe8@mkjKQ=m0IC8~})cBb#3fuax}XPY*)@QIE5;SV5Z739@MFh3rVbAiFb`MPs=N zteD3^N1oHTC+z>FJYnLuD6pIKXt$T;+MQ<`aJ{w1gmMOV33%+8LSIWX7^nj>J=Vj$ z{K@b=Q<63VI*BYYv9uV8ia6kgI6Hu6KsyGHjJ{Jgkl^t###-VP?`E8#Gj?HB5haRf zl~HQ#jChPKbQGAB5PaaE>^x%%{KW$3>x=8MGu%P4eDEO*=sUxy?AxP$=aX`aICVDj za0VdSb4%bQ)#nP}NswuO0SIWPFI~1_7rxplzW_wAQ#R)}pDFbC?lHiGUpLaep^xig_Uk+II+il(8N_BdQa1AVDaWB*u(m~d_6N`n3 zEee)`w58BKRbmlFIY&4`@3&97ptfHCXUxo@)!I{l$fGOX0hQ@I#KnhdvsNEXHZ{h9 zy@)&p2};O7?k+ir9N?BLag~H3=-z|osBQv2`yKH5o6E7d_nYszxCeo3wp<72{$Q?t z$msxlkSDNu34ndiaIWQ5SpQ|!wf3NI7v7J9K?~cV;5aJ%Nwz#Kihxa*OV$>!#B{j zUAmA{@4#aPy<pQX$l`9WX8;gwF{f{p)JM%DBA~ETYP(#H ztXT%;D_X#{vrr0O@nacRnJEDKvG3DPs&Mpq2UYvzGv>HV$>U0F$}G#=ZWYhRb&w;Z zs`f}Q@o|0=okW$-z0X)sB$hs0FqdNut@S2Abi1ylK0z3sqv{(@TEchzUhk?~AD!P` zP-OCm+ROUaG2A*Re!+7aiIc$&?N&g@=U#kV|15=7w~|qPvoe+ELHswPAH2YTj+DN( z#iSpDk$HXB#t2@~H$()w&L`uLMhvPDWq z6bEF5Xj|#o_+qZPW{r>^5d6IQ#~qAyxZw~ql{c!Rm*kT11CnSfe_t+f@r@kv)DS?< z^>bEu%6lu5NIyOT)?;x`kx+4;ILK`G{ZRyzn6TTPX za3cIBmMWNoBqJ9tbzmw*M;jgZMmH%-DPX<+)!teF%aAYkBBP$mq>Oo}+%Mo$ANuti zWY0PQeWnT+R0|0~P({LYa48>-Y8vxIGy(k^Kqi7q3u*S!W=%0Y+hEVb^|#7y*Ugj@ zNP!>m97O>#HVOQ9X6HXZW9v9JswBY0@Ds}kHz-X=nFGo?+jBeJ`e_4bGH9S~!6Dh+ zNCJxsSFVqt!l!F>Xa@YNM=+#a1jI+V-A}pC=jjwv)FKS(Y`@Q@{OP%apY6b}JuoyD zU#~zG9ZlOTR9(*-h)&Sib8>%ysh^+lJK{z_Y)G;Q_@NYGsPj>YNTG>OxNc1fgUkaE z;uM~nSYH&${2ab8dkElZE)i{m2(t_lL3i{Y++BRHe~;-jUT*^{{xfC0q-v4Jz>w#c zp<=-z@2OVw*IAJGOng)(v3Q9B4=(>NvS33N1=q7WT?ERyfa#4{b;8W^SRZrVMmbNy zM$ERVU38Zn3{4@lHzJ^n3+l}r0bTugcKTJW1y7@&E)P(Sjbx5G@28gMbZyK`fOJ4T zdVq;xO zJnW|Szh_@(S)fCuziVwZ3J)CF6W^AZc?)cD1d^Y1T+wSB0Wgx>_sS@E37F>OcNnf9-#um}BF7Cht1{wBkY{@J7s6nu9WO1bPb60`XrZH#z7jn2K zKo~8ze5@VYP7D!kQ*U%`L%7G8II_9lsHd{&2Zp|G^Kw6J!|w+@de{3hi%0u zXXPyf6O`SkzomCdJ;?ppA3{5$d2rj7mhz`NOpo!HJ_eL5m(B?X)H960$@6&ZkYjUl zRmm75ZvH;ut}_%!QA?B`AY?GNkEZe7;(^NYk@Kg?aPbH%@j|H@L#aDLUK>IT$HIee z<0W#ImQRAyeSb9VkbA5K;L;E!uC2C)3t1?YD$FUsPGQvXj995B5xw$6s+#AbFT^4x z{a=M|P20R)K ziDfON3sqB`G~$$X+G-+*V7Osf5fDC+w3IE7ekp_Xk$-`Xtt0cMe7Ub9G3$XS|Gqja zDoZ}Y^3YY2AE39P{Q&%zA=`e7@pM$4=)?CLe<3azwalu{v?5|K?Sfd~*#|w>fiLIC z1RJ1W&%rIJojZ8iJ3v6d>*Id1WVsg=g~m1+*WI6>4&)U)_gf(UJsXIIQEX12V(D{G zKEMk4z;$J|!5$Sj9^&=A-R~YCg(%QDeiD;BCbbD};m(1}A_3e2e;NY~)KOw@i!_z_ zRix~N;_;}A9l=Z?u7O`HYP7obnMt5HIpiz4JBOnx+{w62`{+_a0G`LfI6=br*8m_a z;sF;P`LH(&Vpw2Q3xab&Wrk7pJ9`LU=$#_gdiure#OhxeU?Exn@H>n$`}#$hk&7B# z8r;A>K}{b8Rs`Wt+Z(`5@brTQrBqXXK!%*fL2AYGz zWNTlG#=mzAp^IUSpjF3R^AFFDtX9f6$p$wMq)Zui+2P1?s^lS7$g>n60Paps!&qgs zk>GkpLn`!Tr$@L70Fo-=Appehb@TiRX*aBW5yjfasm{Vji4V%@#x-cy@;)dG6e0vW zW@Sk7-%d}m*PcrKDOAbaA~V5$VYl3)u>`KizWiKpNV`NBUM6m+cl*TAqAxMY z40~QiuMZ2a6N^D3nAc6GX$a~VtR8o z^@-A;O4_u(5^WQ9NO+x=Htfb-uF9>~MU5jk?_x#$4XC!r_tNqAuPvM=+ZowYus|flfU67MxTibspVF(J1^Cq1O9xe1Kb& zzn$G=Cr_iZhkUQ6iyVv1MwEnBw@WISn6K~(?`_t)Y&0Uy;-Z|$)LQjLvBKX=B&9LP zH)&*x0^#;U)K*OtI>^(#FGhqFAg68a;az>*N15I$vn{UwyKN81KpNZg_Hv^r)lJz( zU6n^(Mp9QZoyb0?J+i&jBekLCj&_oRLY6yKgU7D+5F_yJN};2{I+Uoa2^1}1u^wJd zDp)mPX#Ter2F1QiNy?H~l(!EP*aBGiCw&F`%sO~|t{fKapXXeE{Q|~z+x!s>EQ6_K z@Agf1@4xFQr^J2QK*;h!s?wy3Ke&A5+!|OO*t~0dz?B&mP8YVqos4sk4n1vLSd|!; z8E_v47woZUv=x2+rxq9dp)h=xgk_a%;q=ba!svr~pcU2|M6fk$A>r^b4oAZ6d`QS= z(b}bhll^SeLlB2LD2!sUV4jcl6~~t~`?d@G=QXT$^5W<}^t%V1?FlyoJiG4UMY*(( zwvcHQ!)XjUriY&YxXG&h&7zGan`4tj43H8~%18s3DpKddQZ=M%h7O;YaK&TXen$BT zw3K!Sx3~1NF{J6Id5_-a$NfhURmgzQsxf*(glS3cL~}xk*9+?fisNW$8HZTH+4LyQ zHE6hbzn5kBqNiXuG)&>$dj{s#H3|jO2h-fb+*s4Fj8Rwa_|V)h*9_u~DA`XaS1U?d z=*I^QwvMfK;=|kg5-6QG5wZYxFQoKbDWc9Mm{7ImwDF;xCgXM1QO$D3PW%aqBAd!q5~r(Y%PNNG%!-HRLy{pRd|j!DgE^S51l}4d z%Lw;`3rNwIYu|H}5r(T|8PbNhf>2!Uv`m?I=Ak6?C?#ixK^6C^K5vH*#{OQmX|Dyq zhlsMue2}p~zxr8js!}3DttEIZ4Mb2`%FiPBXBuhtoFg8>yZ;ET$dS}iJUydJG@p@d z^RZA<{1Ds&?m*-<=vrAAIz@8Z@`x%{YNeg!@6A2ucH`iw7AVnW`z>5k=wCcXjeRZ! zx7t&MT=hqji+F+Sosxi!NLiYLcB2E8eGE=86@4Au8#*Z5(SQ+@!16CBl0AWNJXm>^ z`-C)bkV>@h>S@8jLzbWvOOcm;)bt%$56AJs`eM$bl9~#7V>JCL1gFm&(9mO}i?B%R zcKZq^N#Q6{;D>r@ie|2{RxXP%?mc`0Ll^aFTd`(ZEZXhS_XN3Ixhd=0dO%2T{MLaY zn1d|0x%u}7aQnnfaU57!<@)@$LgPZ?zaN0?0un#C%?NG4 z4CRP2y8=e5*Lce^zgVAeu3QpXq^@HMeoqURql<+KGKpOp3tixXOy(kR@T^=dd=uu_ zogcEU^k4}`%&hBtttd)}^%By^fD@hfJ>Sl=-h73iVNM}qL0j$|v|bp>k(k^-Dcdt= z%8^Kx5d&t>MzR{XIhkq_q@Y`4uDni<^E7ZIPP6Iv2JQ$G7I;hwi#&~A=ZV2Y(G#2c z7Vk?t@`0v0!w^;4xCeXlhy^*sq;}KY+vl|3rRxz#>L2~B3je>(A~s~?v8OuOQ%Hv9 zC4iTXv9+`{&+PQIM5m2rXiWb57;tGiNaj7BoFx&b%JyKb+r#GghB)&r0f9-@Iec&R z@_Bt4e=o|hLghEO+}^z<*quuPlmAmA3#`Y#xBqHW8po5;UFhTNhLj<=NddjF1>kbJ^Ta@t4MGKlJ0ya%*C-4zU56!gWK>Bz_M=W|;BWNUfk~aZ z5C40k+*INN??4v>33-sezZY_6%Xj{4&}RG{*~cUN_Xz%XtE>fT z?ZH0AvVR0wKzzcET=Xi!XY~Z#ep5X zCb+zh@cwb6hU}=c6b-(cRzN7F(lZ<}veg;M_1WvH5j>L1xRW=h0>z~JOum}rBX4<8 zkAZ~66}W!<)1!o^a=H-VVqndX8lM-Qd;%`c(i#F|(?I+~$YChXolN_39u&LZg0Az) zx2$mA4ra&lLD@#$VEd2p(zmM;5bI|{XOvBBc~8851Gm{qB#L;Vu$=DKuaoLZ{^QTw z6AJK%D+euR>f?KqV>eS^6H$YT(de|LkP=(j8~K)+<|B#0)Q_3yxA^Dwedwyd+gy9N z%P&5&(lp$yk@NdUCzrVZL}XCe=P;FeQV0ftV0b>EbcX8--4ZlK5c*D5TtlJ;Vs2s! zG;V*d);5Alx}8^nX>VkTo}%7!mAZf3dU|pm*uQQAPE@x;#~WQ!+F0)XD{5Ur(u_uO zi(a>n*Ez_?d+|^vWh?cQe!#PCG1EVM=5K-Q!5UbjYKw8)b-#>kdIq;*QF@!cvg|a> z3Z7+MPTieTRRaJ8|0z)~g3@lcveX-_0Y?ZZ)Rw$%_thE&ziI&}lEr={)PvaqhNVEo`t~iAfnsD4C8j z`RpBFK4gqv&0ir)x;`68=@^`5v(#Ky(;CtuY12sItaY91dz+CZBao@`$^xb?pnxn{ z|4ipbo`KPIh_Prce+Gd^4b#4TijUa!Rh80h&t7RB7yXMy=N%PY=Gi8TT*SqO@7kcP zAY<{sT5d^-T*$3l_g8O1pvoU!Qc=ez=7$JHeFFQmkT4zJ0l_^)Co%NrtZhzOKg_3p zAPDjx(-J;?`S{@{MHVf@C)r7Jnwvkw1d6}ox~nC2FR;|AP%Ng*@mqAhts#Et)9j32tN2+$kq^X_q%XY&OYOFP@oq7n$2VOSDYtslk{x zqBcKvdUDELUmUQg^A_X-VKty*(4EaeoCK0h9|xUfF0=UEKffQz=YG zS5cSD;QsI1yGR@oK82^CXr#-q$EtafPmRsXUg6YWGdBIBx==2fxR+RA@$@-rkUug1 zYv@;CXB*X2RZxtr5J-xeK#8L;kXt@0sRAGa51CXU=P`Eoy>fCp+klw0jj>kWKVHqD zqLCJS3f$~39-*aQT6wmlsPme#e7YM)V0oT@cBOLt`zq4~YT6wKtBQC2#2m*jGa6oPAy{9O}i4rpZK-h4~W$^wQ0j_a^l z9lM7-!3V(@{6MZ{BV1o4h*c8*oF5A)M735h)ao`6#*NRg zJ5ULCu?5uxJ5#g754ZoNN2}YO}MQ@%~Aa=-6WC^5!6qh;LQ{7v))A zaqLZ=x-D+3UDVFI%oU9gO1Lc0iiMaPtL%tXvVU2A;s@wsO_b}5h|tg=u2|s8+a(7v z7uqLDa`aZEs?h@UqSGF_N0dhz>lA~1gz#;32dqv|ci!-Q|DogL+UfV8U^KNEws9a~ z|EeS3onb9KsaU`58_`wki?7JA#WWKRA+=vhV?w{Ll^rlIX7K5bY@ z;ETV)>6z>=$-LuxGlgWw&%-qbbgnqYpmNdsb$%1m=R*_IVe?Z(D-6#nM>Au8Ranfp z@VLJiEG!_V0^P^!v;z@y5txrG4kBNEb=pj^>qSsFgZIrQ1m05F(y?~tkRd-j=LV~ zVFNY@-Z9o*VoUpZ+}0O$!SxsFIHrUbkx4)46;}V-wG`KaciWhs4iN;*;dE>@0ZFUQ zOr(+NM_bcgajEYVzcOPIJ{^Wb5krcHeWtj6y%sm0wb~eJL!D88@Dwar7nH=^9D&pQV zg30;KY0!366H7J>`lax4(yDV^#8f!&4elo*KMT^blEma6RN;D$HGR6nm%?E%j^enU zxS*p}r0Hoype}nad_^g?V zj?->TF%E-AV@G^_nYf}=g(hzL>2&0~RP4Oj77ibCl^slzpbhXwopAzcU z%HM^l)i}xJ(G(i$BmAR|;-F9#Ytb`1{!9ovlj7|du&?rrk1VPY_T5HsJI5;xnWp40 z?tyMU_jr&ALnamW}d{cB0s_=Rdroq8eAiSQY^2P z#d?lBHtiCEjA&zOY(91|u*Gfb3W-3U%#2nNZ8GQY*`w*4u_M41Fz=Wf+!^mwh&Ok! z2_3fRyQ@da(TbC;gM$U)DmbgQcwde@sRvZCe-;GKkj6534(&=Op_bWDli--3es=JV zA};ei-RRO)yFEV(S_X`U_W*I(kfy3SP6zuYSdgX)QTN_Gh)sZ%vuE)#neEg0BwOcI zGOuX%I5oPF2@e#Gpq~+< z*Ukd%&BZ_Vvq`e@d5}L^4h5Z=AmPyS5k((mbG{Q^HwwCA2r;Q3aXS8oRHWq+k5AZ< z>D(x#t^|%#mbOy$sijWw!I?g`qowyL`NknCd%eh>_)P8h=#oraIE| z_uj(JV;|l;CDY_)7D!OH^Av1+w3ttXA(4sF98JK5iTDc0*Bva$e%c=}unj)TZ*(nq zB`DH=l;QE!gXk;rt)GNdH4nBhZ*ca5B$Gq(r|Ei<7`FSggkkaSha%Wist5#9TCPsb z0!@|^)AF}=+01cJfsR#K#;^nx%E#WB{v=PWgd`m*>2@A2rOSMu6yo;+&C&ksX+%UTg@ z+0Ir>l~Cs;L#(Uk{JUprKW+T{M%0db8?Ocf~7dw~YC3 zmef$XQ?<^!i_d;*vJtQlgQ&iUG(`*OtP;OTrnrb;$YMC2+)+ zd`yr3F%CcGAZ7sOxLKjd&+lTw34zuf2c%I*xrRK#2iWRQ*~xHL(`r!IW}|z8q72Cp zx~Z_C20J|mtJere`G&^+Bhi%uG^k#WbpsWVFp1FPCvXPN8(-&PP|o5!3dVpvrppbF zYoQ8%t~nrh(y--%GySe13abUB!dUg;En3`cG@*)v3wGTNP}4km7~vq^AE%~lNx$%j zgil{MMB**Imuw~7OT~dm^y%OQ4=`op4aQPRKa;=AMA7qgdKJ%yVx7s*l?V?{EZBrxYae6!&HzB!iKwNJ7^s1Igat)xJN;-U7 z!=GMzksK65Jz$hyxEy(p-BW%XO31@2%B=ug|FvCEKn>%9wCe5kq(Y0&!Ck)$z)0fe zS}oJvZ7?QXtw+n((jMZ~E`ZS?K_=}T#@-^hPRk7?-Cu4KI&qc<2RkDA*uE*2WF}Ab zR3-GYlDH`ei)hzK4T|Q-Te6ClC%w^V6$+HvqFY&dS+LVnzfhn~4`!maV6Giuf$(XCl$Nxium@MTYAHrqBwgv0po5*I3(w0ff6`W8Kg1l_e4S4z@nwDZ?}0HhSr>j;+js+J{P7?$SA1d603g zaudC4DvCEqEK1GDmKNtWt!+E68;Q5O3%|yPa5#QV^QJWiT(P4xLlEMMIn|; z*{Tjv2!)9&pwXRYOh_o~QU=F}iwjN(J^LvQ{#J*eXnO62oio8CcV)2Ea9>=#bzHvp zy}Yjl7lEBjFbc<-A?J5Gm-pyMhDAfV#ccV-fsEggpKV$60wh435(Y(r1dLI0I6mbo z`>4H;IL7pecr6YE{G%|WKrYr7bm)f##7(J#tym}*PJ7SZ)KK1<3n1>dnmR6wgbL6O zuiuvJg{bm1{#-?F@aD+T5r__~{Vb+2=k{Xr#i_f^1KYp|<-RifR)gb0Mr~BokrD_m zuXVgDL-|nhXwPZ*5FY-soAhhqF_kLr&Y=$tdP6;5II~BQs)Oqb&v??5@{iIowsn3c zr0dYp;S5&ubOw@|M!r&KG24Ep7@-+bMQcoVFpZU}>vz`n)%Z(L?bZf0d#K&ix=7j_SKF`HRS(W>Bc)t~|2}T<+<92X zv090L%3gJX(Z!#$;yYAIU9!LK}ft=g?rfk4JmTwDt9&F`a$A(#OV6R{lc~4W5CTbu^ES)8)F)tlJ?{ zdoFW-I;@dJGpL$eA>(!LgY#-5$1Iz>wuZkE?SXZ+%L;Wcpry~Y^akvTHIE=o$x-C~ z%yi8vl0nGQTGrdZCW0Cz<}TZ~v6)5QBYG37j=A{|R)gt4V|d1n?sEER`L(}M-rR&yzbOGG>ILe_iB)dGfmbomxb?z=oxPe_u37e^%`U3396FVSzwW!;6>} z$0X2yBYO+SG zuc#yW;Pn{t3kKYKX1+ujS8TdRUEu$b8voCi5Ql$>H6}Rqh{XdwXAd_8P1=2>Sb)Er_jA!$<8O(P#64w#(BX=?LV98={4KT9J@ICFodJe zbFm|9o{PUyHq6QBY`-;06ZcQ6mK6(tb+#YS0h|+Utzq77V8-Cygn)lzH8Pn?+{*xu z)(f?d6|U8I{CkCo&9g|C6Y#o2%TFi7_6kjs53e2})F0dM0HeeG0IPHr8M*(S6GPD8 z?&zVbo_X=yN6_4;hq}x<@nq#Qa@Aip6BPODp6IA)cBZn88EOf^<9(s`#_jFkUV)Zbqc-0IIT2d zP)jA?eMO$X3x8$c5ECz>hD|${8O-m#tf4$^X#W%KzL0=$z^`a_)q!Hj)4crla;dDM zndwmz@mlK$+-!D#0%P9Cybuw7Hd+!wM##z1H(q9YPE63R0gK5~uT9#87e2tLN$mIh zIDf2Nn%1Nm_)#wC0VXj5+hE?^mVMEwUC}%ISG!gH(D=lS8hA%xx}Ds_NgCDo!nNZRMP=!!y zjOBbszOPDL(Q2V^z;<5RB%h=}wH6yZpe&iyI4KMPP_dIvVkM9i(6o$mUUclgz~8_A zkZMr7B%Bm)>MO*%u;mBKc<_PvIwjl zpd>$Bx1pRUg+^WbD3!T=YB)dmEgt;#_tY%?C2+%pPdrT91)WLE_chb%ijK6luf>FV zmZ!2NxL}qzNWf;A-8cFjX?X_=97Ar60D8)365HD6XWhmnz+*Y~vw`ZDLmhd5k*8V5 z?<>AD)unQnH%nDcTN{GGLZ{E>@15#*M7@5T`T2+*;eORhiK+<@KcG533?1Kp zu~O1rB)YNdh;(zl7u2VqlyY<%)qC~W3wS36AH4#dkezf3^0CN|X~q%c-?(tWV?=;D zSor|+DB_VM(jD3`7Dihjrm_B1rRZY2H(_S;ug@Z$=snvUoA8Q zOwuu)W53CIKdq^y^FeA5f?%|ZDRfwg!QkEg<@dL5U%ZaU6@NHxXIS7*!^98+QsfL* zJrP;k7OOj1gUl(WPqU8N$Z8{gfy$uh#k2O;KxXu!+;P6d8 zd14(Fc)Okh#1WnFCJ3Ok@N-HtohM}oK=;UJW+bxj-ubpGmF0G;{fnOcmx~zfwJ;fT zgF>fKM_!mEI+ARw;Q9b^Opg$|&~SADFZm2{mQ|?1YOd`bEN7yN-1;d(XT(a?k-yYA zC(M!@DY&4L(dZqX){`Ra*zpjl!8#Rz^z5OXwH+W=6FZ-fDtj(o$#0(x-sdmByB)vM zc+7M+sSe3ua0d0y&EHtIT!`F|{IiiJ6re~KiwZBfZl;Ey9AAZiqvvRUgpdFzIN5P4 zbw{Y;ZNe#IC+3ZUNn)`kH2j9 zxj3?4x*H?-+4{S|eK=gkmvVv_YM6QLZeU$eS5H_eV|`wn<=qLzD!TyV$iIL+Q}4wB zL<-;}u@^$bev66Zg%p6J!fym-Vj@05donL*F_amlqs;DEU-UE-lfrJs3e+gd%l+df z;mS>gc$7~;-Vyvzd&=H1 z(|3C|9_up*e|PZJkRta0IhtHj$`73;&} z*wKO`FdTy}M0wDk)RXQ=tR`h#^_3{sRDigoA@IRY)pyeY5tnb?G?Bi?^t^HMV5yb7 zS4HNB=Az4;5hI@W0i{~U-A-(CVvPS4flr;guODFjhJ~4oPzz1gD8Z-rs*y6 zqO|}bDPf2Rp%G+HZ{p}*u)jbB57Qu6Ty4B<*Z5s&8%h9ZwinXg{|4@c{F>1}x4|VE zolLN{=B>GE+l-d3YyFD>WOy(Jn=#Bh1X}sfbRt)Rbfuu1{|(q^4s_EB1Ohixh0xn= zzIFi+WfYkjMtQ$iVzPRco}|ND{$`Ro(i&{iHm`Nfk~N7x8`REiLc>%%d$8ofc=vuA zJLbQsd%M{t*GM2m$@?iJPdkyOaw?M<{fAImszZE0hTe=CJ6HkWtxy2-kvv_>32!V8 zlX;t((TEGU9OUlbtz2bSix@P=>MV*^Bo$@-u`twY61VizzZU@l9|p|n%% z;S`5KzG{=(&(^asxEUqLpKJvJ2J9`6D!$xUIobiE@7c{S?VjHq(r+&IzKszu-z>jH z(mHHdm-;Ep4TAd8XC^E760Vnl=e)nBXI1!B3MrAHnD>oR*6Nt6&KB{{O|6O&F_}>_93j?{;PY?-}$AlV=cp_r{ zzE_acQ$Zv|_|{}llD$MA#nEP|`t^+@fv@4vEY}#n$w`S1hH99k32?_}&zUzO)*YSK z3pJ+tC%As6<^XJu4}$#zVBy+CK+jGJu-7-=^RdVH#H1{Z{e(2?(;!pM;fQeAup?4fr35BuM;xY>F2$@%$s^Z%B0BF zWQr>JN`mEYVh#Q)hqj5=p5gD2RObE(5-{BxEF}B^@OH$}u_`?ek~CB#HZTY|C|`eK zg{G|lCgH`gdwl9=+nY@1v3UUU56^f4ZC6FxuZ7v-ArxufBG}b8kWs)vjOJ&4f0FRg zzn|pX-}bK@S_p12xI0hHvVOU{v36v97)8gpa?(w42TK(MUfu9>EK!-4NL4{4zDRv2 z)2g(Dg;@pY(SYeZWqG+qsd}Eq)96H2zUC8Au&a_(uJvY0l(7;biZz^dR7*Nhm-A%+ zwPBv~*3C|PnzFymHg-daYxErSF|CriVu;MsB)0CXp3#|R)}8gi851Rett4!f$;=Vj zL?z>HWA#{!7s{35^mQu!T7rtmd%;mTV>f%BYp_wV2kxHkyYGK|Z+FvAbDw58_(J0r zzKQ;39bVWVSb1arv23Bkf(=@M0~Pq6SV;UtV?&n}I_HeYHUI%xs5WZsC48Q+l_QwK zJ^Y^|Ts-$FvhDD^zqTdKvx|)pjEfl!Mc$h`E&7v88{k~W` zeM|wTfRNMggGN?~jGx&MBmbt5i_W6`O`|@}4s)U@Q^KF*bj^fu+Hn7rv$y*9BI+^i zzLQ)iiaT^$3_pZQvP?~}6|dQFu3re7CB&zrLX&r=n}5x$9+@AKU-&f*wNG43 zxA!5HP4xSSK>PilpQ%9L?)fsuy#=|!+FV4c;ol^eR|4Eac$+3BNBgLbK?<2l8%O~} z20<8t#nt3MmDdmMB=*{`by)mClctJTa2Az}lhX{dtA8P)uldF;A^q+x!{@ujqX(lmEAm@sFkMn`3%>zU# z+#IU~Qj)#7$0mvMWHe!aWbnZ^+hy^IANDhLvF%m5hW~dXz{|iUx zFs_4<&(^|QniHxURZ9|rnL_?N_kA?#gMyIB4O#b^>?h>^{Ejf_G!ocV))12y*w9;c z2pk*@iZ`a7>*;f7@7A6RY03|L5&Kd8;p-0s`Yz8eWF(*dbNJ?xLc;y))jdFkfYP#l z!HT(6NnmXN6YNzhm^K)r{XAc52tFxHDchb?Kvnd*@EM+t`5$aa%Ou)VFt)q|2p(Gn z^izz)9EWZpLUZgj8oUEN?25x+c6bx#z+q}MtA>r0n2h^5L2MM)CyF%I*JprV->c9D z=W6$X`|#_7|5g}GV9@<0PJTJ&qreUju3kB5EW=|LMz;}{C;s9`jSvQ4DZXsiax}}f zDhn=8;O$yMA!$$h!9Ac)$thk2f8Menns~~F8yi~70s%FMKVV>qRpKn`4-deAmM#!ED~6iTF4+OVDjLz=(b+7Y zD3qz2y-=EJwc*;|uV46e)2@YFx%jJ3oF56M4f?@lilGg-1qjWU7?sp~s|fiMQ3?*n zTxIwZP-b&+sgZzIu7GQ`J}NPn7R)e?Q(wZja~wE8=oOa*j-ZNxQAS4OHxtx&re3`z zRXBaw=bQ2kZJyr=R(vLzaWUwu!#W^h`i4hM-$V^d5UsQ=Uf_WRf?yqhVQ)@u`0~}8 z4;U+JJa4#}P>^2XAv_jXjCYh6Zj;)lx$nx9ba1QcBetIY{7qP)#!v*rna3I8_EdSt z{130uufn(1v~`Z#KslseOJI!CqkDA?`O~4cO;zvgD{EG<=i}&V^ce%N%Gfx%J8;xw zy`~%-lg*bxDo=&Jgx4(Go}?oMOPzb_16khVVhei|W_8#6Y+HKxWciZ~eAm+Hn#abo zwTI6^ifrRLG#d?GZaw+5v&Cn90*-KMcoMk{ztL^;iMzC?Kr7X;2HrhamG}TbySv>H zV;^9-dh^g11jRMOTFUk=Rjnyj!=#hj{_)S=U;mG$vy7^;`?@|LAOa#Gjew+dcY}0G z3)0=)(%m7Q0@B?r-QC^YEg;~#&;5VikB-4`z`4$~_gZuP=3LF8j@qU{fsZn41;R<& z%Hba`?(Q3Oo^C|{(ebV{Zt=~{p%`%T17qr)^h{p-;_GNxM_H!@Blwu<<3uh3GJDZW zNTcF(;~b@F!qG>u5+$lzH$!;cd?eu{?J+~HWfiR&!}W+{peln`A})bFQqNp@~nPi+P%l$BMH#OO18VqGDJ1o@{{l zqb-7m{;r3Uhcu|EmM>XM6qka{&_eLZtd8nk=_NewOS@05ggM8in9THxxu{F+7cIta zMz;Q|RB=~*+#t9QiVlMoNn!o}la|dx1P|pU-r|RV@AI9IkA|$I4SwPNt9QJW%qV`R zTEJkJV02}&C|&^wGyRZ#7LRZ4PXhgvYi`lxe>m-XNeeRcm$c^t6&lkZ`qfB1Y5n(u zjNtlatCCp!$P;5LMPAgx*D%IGzaK_zw83th{4dgUb;^T+)wiY`N1&BLhBhAi{j|#f zT=~lHl#lTIj5D0(n$DV!cjj|zmh|T%mWR~ne|+kDeVj2~>W0U613Y@qyA0k!S$fqM zpAX3PYQbWJ6i_%=tZ6AP7{RW1!HgjK57I~{(HzdgfhX_f5H-F%lNe0o8CWIpXjLb`;#4uM=%rhz`YGr92WkFX$m0Ga2-`e!IsS{v*hD zzu^zyYwy8&NbLwR8}KT)feXd(kH|Z|+D>@HM6s{A8r&`*AvN6e`TzVy!9!TNjLYv1e%K{V zV2>vUUO_#hnztTENq8O;k&^uBHKVA85R=})E|R$&h`lJs{!ZIuq>cMcwp~`s&r>|Y zDy-t$+#iYmiaiBQ*mfQHa8hv9nEUxC&M*(tDk;nYr>p6TafC`t)<3f!m5at{D&wU1 zEjTFU)kD+|Qtk$iM@bdN=MgUa4x=poE@zQym4CU%xlYqb+3`C!o@4(I>cOl(BjyGP zX3HTYOEQPoAg!TbvaE5}q0hp@ltn`Ivqhkyacgh@W$GG@R7_XCI{hnoLM1fIkRghAULdDxJ)ODy#*ly)pc#M;9&pv>-2)Qu*EpBHVEc+r0c6c@eKnyz&fj<^ z)fL%D1^MirRP}s;=wU1LUVU}TFE-QCdJ2HH4S%D((Y`ahBjw{WaS&NJ}W9t}z;0l6vFf2P-unU(y6g zJ?IXn;Slmiwz+jRq2HD+?$ktDdjKX@gWD13Ohe|aKZtqcs7c8-(3V&M3)%p#WdxRhA%^k# zaNWd(6`;6)be6Dx-?>#1j+#H^|F z1-8tv!3)}&ksP|0?B2CJtHFAOH9`>^OhW3J3V+MO+hO!-7r#+davM5+`U3~>D^rMA zYX7!re=t5d1@2wx%&`f|E-4ctW;4~4I-!v3P)k9 zt_K`nV`(=^&op@=K`}xT7k>@hix{wb)8+&9q-14Q-2{?~Hnvg}j~P_-jG#NvrM2Tr zNy#g|X;%}ET0c(r^4~Vo4@77E*AXcD1Eps+`#rO-U_XNT7y5dH%2ccFABv)ZSE{$N zipB4o^_sjuY0uG6q+ntLw%RkOEVkWY1&WMwe`K`HKYtT~&F}V8?pC}G(jYSmZ~!Cn z0xRvZvS#;VZn0qYE}9@8X*OD?*>$7UJ(y|I81(s>8mxBuzWMol{(!geO5emB=N>80 zFv5g|AL%nMHp|FT^}gF~OL~;VSVDH1L5{R_dtRKN?09HN>vcy)@QbIZti`Fd5hw#v z9;DgdUMfnnF)!>B1kJLrEbq89Hj*2b%TV+kzW-}PTT|Pc464eMU52>#8KZp~`QlSu z@8&TQ16@-#_+jL&a(*-g8)r#J8q%gm--P5C^~XDIWRuqQy4iiHGt@Mk=#i|N=`2MW zNu-oXj-jS{k2|CR#zO;>ro|t5 z(&SB3qocshtWM!Vlxm!t0W0`J605|Q)g<6YpdTut7Z_)d+xQOIJ5dk(Yk}2PNA4bR z>jlXn1Ba;bCQhkaRW20%3VrpI=;?Sxjlb1BP%5U8f|uVS5T6~6lygmU9gb_K#lMY3 zru)d43$Lpx*-qL!Pme*9vVq@2vn1Q)uOR6=fDE}9LDMC~mn1@Q^~zTc6oQ7AT$+^Q zQdc7hqOJqX*Kc?Rt}G@Wl*p%5E8o7B&I}v*zdjb8v_mC_?#@%HdTlGRsrG9OOJ23KX01|q6}M{R;t>&q zjKeH#&OUe?&k_qkRD+mFrfT&b1Uu}>X^Ln9iwnuE1V8hBq3!TG6t`jfa%t^`AA7+) zFKW)y+IH-k>M*b2GG?Jj;0{NO*taRKv^5g3Xs_&$4!7(lVpY`7mW*I_(X}U4S6O4f zo!Q6568+b26Z#^sB#V;*#RMCTahwW;?9+gMXS|+Jpj4T?nmoBh@0@aXDk5fI(f>U; zF%sRx5O-teo(599=2%&AO~`|TLURBMy2#dWi3~lwnBSZjM=$*OY2PR|dQ`=+g09Hn zwhu-(@ZPncGO5hyVfq*v)-7d1$1b_mLn&u?PSENQuZ3bx%Y04p&hjs0D)jxuVL$rc z{-3$dQQHj~#b3e?Rd9O`o%_gvL*!B7@IOO$FkL4iD8dnw=W!K>gW!ipp0b{0v6kTn z!#(O+cbN1JBV6`uQ{EYs(l9wU79If|M9eO0%rh|kE7|cL(D6tALT7}n=f%g8yH)o6z z`>Vgn7TmsnOY-1ePf-@_svWKZ|K^hc+r?qTOHkz##I#pMuS9i@byJI{8>O=sp36zH z1(LZ+{u*b{Ru>f%JaJbG8Pm(E(C0Z}Mc~+4HPN$s6F1&IgW>z2!@*Ztw^J~<)LPo@ z*vqXM2HNU>LT@jEjMf;O*PF>u%EZA8jmshct}GEi9}jME-FNKUG4?U%RqnE6rO4wu z;p(c`d@{+?hfi23+&w;QJ3I{XK7T%KA#Fcy`-U)T;n(qOaHjKcm5`KSw5W(5T`U=> zC<$o{7obxD?N`L?}zoH zmITGjYAaI9%?8h*2%V)BBroAke7zQeMB~+4lhs@*<0r-DKpGz{Hou}_QQfk z$K)v6IUbL^WgAz#h1BmMbdGg%=aN)MK9OFvy9iiP@MwEItM;n%uh};A5Cev1V-l$6 zr3U?QeVWJZ#9v^HFpZft-&m+(;Cvc_(Fpwt_{L>bWm&Cr&~MS_1As zL3?>+J_D_$tABsJloP|j1xBoZ0?p2I7lnB-7BK3K!d@{2hyutd+Cb`)i0&1Pi$+=s=`e>a)ol7d-q=vXinTF?%?H})L|IUmy~w9<{g(G) zoP1p&aQ*oCAIs$D>+e}!yP;|?^4T>-MHKFjWd)S($Ny;VrbD$opa1&wt#TXRx8H1^ z3-C|=T`i`-TxjGp13Mb)j^HCgbyHw@&HGFX)+wZSy2;39( zTqVWGO<Kj7c={MvD3mJzE+tLz>;~eF#G~R01g>Hd#%>)LOiO#rHy$_ZgVp~6? z*)u9C)!&+I#s`0jX9%?_Fdry}e4hC=+6V2VI+BTFhSllj=$B{iR zgekyB2qs$DnQ0G40oHA=e8DW}1V+;&!)fUIo)a?PX3aOFcZ<+fY`XGlzdS=AgGcZ( z+#tZPA6+S!<(AViXt?GOPLkohh5eke{JgfhS$WoaOwmQdcqjyWua0wh6k?&>6a@EJ zhQGE1b=L$fdqbSrR(t66{qa_8)NSy7)W&NXpN*l&VepJmoqaYHP$*JZp(8M`SUR3| zRi9djFzd6I?jb-+Ss18L%G`}|bs|MJ0L_CDn48@O#EG=Vgy$2MtALaIL0NzzN<7suv|9Nx>OtV4|;KSz1$uzf+XC_D@5S^T!#;H%E*FRSzCN z)VX%n4*?-VFVD|(;Lqp_sn_Ud-vN`oX&`rg&K<<@r=CWQ5DKpk+-8G(}$N2 z=UWr{d#)~`07)6l|H73eu3TozzK&!)Kb6_;X)HTc%m>tfQ`}NXt8XP_{v{T@xj-X^ zK|sLWnXUwI-CvSgL6;n)HWc6GwN^UO6r{B#xAN6mB^i&y$0S77{<|i(T146Zp40&h z3=&-}u+Z_%1o|nGfrkzL-t5ffiyZMJi%_WeeSL=UR-LlCxv!Y1dRMK36luz+16kaE zPijOp$M$RG@%*Vs9AtCWTwb%joM3^5h+keA;_X7sm|%R(u#?luSSjtkAqsS%=UEbkQBBHs8JH%NA^U6VJ`g zkBIM5t}#krggb8OM@5$ZyX>wIkO_J30oLQdPx-%z!VeK3zuOOhvseKWrm*i<;@4uq z8_;4FZS+3?XSQxvIteZId)gPbUQD$UaMojbieY1${UZ3VA4P}6NJl*YQQSZSjCm7# z+9Z;HKL%HT=mp_v$>N`@l_aKQh~Wmty!J3G7oI`;!|=RnUQ$+iVWReV;t0&uKfU3k z846aYOzCplUWafyKgKzal|O48dWyiA^{Qx3=-lqKWQS;D-~&e}_1bT5?-K{CE##K6 zZ^ld9$=oPnaa01+D;g~aq~?1wTk-^pSvtvskpq$3W68E{ zh$pTPTt9|z59s{k+E1Up%7{c^{|G}f+Vlm(e0Gnz*Ih7e9cuDfFq;jPQQQGCOD&ts z0Ky`W@Bcr+v9^Wa30BXBG=T}={(~1kAUO18R6?*lfqhlY?CJ^3FTjTLaGHu%`}P+s z2_)re;LYR*>waf%$PI$efi%-|R$4P={-Xx00-?ykxE}8C?E6QEuny*TT2Nv8Lu`&L?C&>HZ zm~sg%;mYN|(?d%X`1I9^#jz&C3K*C>;i#UEV~iA=>5X9 z6Nu^>hol!m&Fj2eVe`MP&B5jdqvCUFOmX4~YXn#Pg(UNTVMfW$#~wEv;FW^hBI2mY zd*Yh`4dOmFbz*@Mbo0M@u{2M>Urz194hnSer5->HVf4p+9}BLmNC-jL`A$TLX@)YY znwaHpXqo|eWbv>)HQ5;7A1y;uIA3G_-pyCmNzb(9@wlbGBOE4^rUtMBP5@}hF&Sg- z(P3;`H>=76_NGqEL2v-@g&!_}jqd9Q3tN#GT7Awwx1Rm}_X3eHn(!c8$lQ=8Q?4r0 z01%=HRc|0JTdtUSt!&2Oge+WIliLj|z!Ru21D~JC2)z&4RN?~okE%el=VcF{=|ncz zV&DLfoFE-3J>AKk0XtPF2qZ9fo9&O%jXpDawy3V}@G zgACK9+PZ~)gE#RFE8OKq&wxeOsx%S7GZ-qD+Z*Br$n#$I=say<|9(ETL2|zYau(Gt z)CsyYZRcQJroD7rumwbP*dud`CIoc2QVas9Zg4vx2t{x*m{yXzVGv%9qYt5I61bNh8Uc>>5 zm8$pgo7Y+H{!-cYHnQ(%z!r<%MSTW zJYfcl8j@yJuhEXz zDq|GK4Zt1!JS`acAjww&0xcwKrMLLMK5qngA|n}c zIWr88sOY)r9Z9h|99t-?KO{}ty6Tah^IJekkIby^hzXXanaS-20ErK51ux_5fxZL7 z-k2VjoMf=$EL9*?i*hf)G=1*o^W~1Ga`+|a$?n_j0hF{3(gR|(l z&3V4Tu}$biiR>~~Ns-BFXGBcQ8l%CTT#-Miut;(hljQq5CZ;Tr2#Pes{I;LkEX|T6 zZyvniFw6AfCXb!^>L?W@C6P~q@9jYaqm8hph+?6uY18J!a4K2|nd{Q|`zV&!)CTV7 zMsW`nR5t3ghcbU34wGjyiDIyS{G?WqDK8K#p*kc@qVSz>l&Wd*Dec?eqdM^w?;c3Y zoQ@ZyDjU^t7@c8#tG}p`m;p+4sp28yUa~^JRh7Xnv5>OoGzgSEr_Q~(zu82f`iG*N zeBrPL=0mhJ0ndKT^}tFrQ7SI|CDM(_j2GJ-W>GC>omb%U=K=;TWgqT*(3ed`==0EJ zqxj<(`U1MI3U;lh;}!q@Jy26oW?kVT=WKpywpl;NUSqP`d($}oqo@~VpOQ)-zUN&& zL6XmO#1A196E?_FsrT$WmZOF{czn+9lZ^){(syQI3Vd+C3EN~C6`D+f#G;&n=HXuLhb!)H8M(hNP`(g zv~p)`3K!u_K2tuMj%73ATrMROExD271j_bUO7m+6j9@4w5%b7}9*yi&j-TfH`RKVM z=bsPfccoOv8pMO!VH7ih81rZD2=D1%TZnjrYD=)(&u#-5lBdmi^uM0h<^}Z-nBk=gs6$9KlG%06|lar z+jYDM!(r0@tT4WN5&8tq8VNAfBybTXo)Cqly`ZRP(s_yYe%b~Bi&^7?1#d&RmPhx0 zNQ14}fhP<@8K*^_X9bsiBZZaO?1; zvj>N2kUOn7q6kd}Kx_nks4O99a{Mh$ow9zchGV6)a+TR*H`jp9A!IXuPlEn4x>UKq zb957Tk=Q`s%>fK0*_=S_u(o6%Ajc;GzL>dRqpn?mhiR5aa5SL_ffCe9^=kJ?fxMGFp|E4@*g{VlTW2)YtMM}F{k+{Us~c?&;2U==e@ov-&I|Fyf7=& zN~=PYv?v5Sy-ZgJ|fB{C5kn@KK%MXh1A3&nHB zm036QuuS;DAy45XBGI#-bupO$?a#RL{BvEHk7nd7-&uD}+$kj2M@t)D9ycB%j8)N6 zY~{(-i=$RdcM@~!TX*X;pU$b+a#q*EJCx()obJp`DEH~eJ@hz^04i){J-hgzJGex` z0;-pJ|n`KM?r~?KT+?Lhz=v>1(}bS7@I}8HyI5?m(?i2y?H7Ua?qzn zlDi#{&$TKgFGCkF3g`qW!set2n^V3OZ$3ifKIMcyICnIpCNk2jfor!hEnD9AI==i}wuxIKo^q*NFwU&Q7kZxAZG70~;*mPTi|C z&(pejioMh%kY}Z z?Gtk!R>RmI4td(fE;3`Od}FkciP5QrzNgdR1%3TeFJw-RxpH0Rx-pKAR{43BLdKo) z*L`tqpHQU*u5GKxn4{Z3(ILAjBQAS^vy(`Rneu@IicdUPKLeeyll|1iECy)`ohn$b zy%%}(!6M&kvHSFB$lbrH>CVrd#bV&c@@jZNqJ%fktUC`psysLVSbe3v;OueK8TeL2Apq2U~!cRU~|H6C*L|~_|{iS}> zyWlG5ImwK6I6u7w9r7QS@c5Rtpl!jAU(gGB&2b{xYTF+DP<<5{g*+7@R-3jZPo@XP zz^-`r-N=EZ@ zIMa9;EtUzWiL*1%3NOj(Gk!+2H(`)}BDW>YX_?vCwUk3z?Z2YMPu z=S6Nul2ytz%4MQ98?Y=JJ;zr?1Uk~vJ>qZ{qbo9smaqtP5<9HW#xF7x77)}7E;AqT zD)OouQ;a|M9>b_f*HmmZg;C(JEjycY2ir-@R1HSsA{>j#4l`X=Uc_)k z7T$ZGD!(uW3mH-d6B_$S9ag--yp2OL z?gG6{#Y3p#W+c8n|teofYgM9-}$?2VIA16hP=Xl`G5q_IATqs`O`wmsVB>BdA1X^<7?oJvmv zuZ-&Fx!DJfU*D6a<@I6no<|M8Zctmi_MbMT{O8fC+aArf0 zHzRl0&KL!&>#ck~XD7SNHP0Fa)J?vRnq4k^gc=+An#@5^-CR@}sh?Qbux!mELbc6( z%FH>?MQSjp!K;44(KE%Ci}xz8rwrfe9(~Y|JDbPUg|`u5s?n_E=T|CDR<#vbPdjv$ z&z(Hs@p{M9^}KFT>ztC(U&s!wDRvAag@v+~3Ha zwRq2pZ$&El89k?Fp+chUCdK;RvH0;U#C%;iK*KckpWzNwn$b$x*2Ib7Z=46j7-?oAMO zcT381@pMfWwGsehoEBE-4;DRz;s7v2!!TAR4a0KM)KZPAqSwLYM=X@|#|T0(OG+k4 zMV-(Po9^rO9Jo7Q&uU<@NTL4IWjONDJq+`EOXrj^gL_UWT7S+brNK`h6nak+PbehQ zx_g}K>&AkqPIZ+MtFD256@{^{fkJ9yCJ;$B*f_iT0BFMkBeju6#hkzEo$tRMW4&ZJ z;Rs?K$H4RcWoY{6EA36xkgM|Uc*0qx+5AF}Sf8%bWqr!;XVIu<+0wD|88#`7{CDJ| zr3b>0_69;2Q>+Z;wL-~xKr>JUU>kJIOO>9Bm)z5I z;Yq2fVr4pzUseO0;Y+I^_7u71d1WSw-g#vZ>H#J~a)$vlI+{XhW#}h;JZQQvB${R} z`Dp~JKCLl~-|QCFV_vYZZ8iybs6oOubdb}XOM@*(t_58X)UhajXq@INj>>_QQq>xH zmG&CTuT=rF3gZzfJou#a>k8*m6<{t83~I^_8K=%ylMNq=&wJNfM2kJi;?0@0y9$?2 zu8chGC=|4!vD=})u)7P@I8*!7j6=Udagk$2k+{|@GnAvAk$(%zimv6gv`(S-VO6cq zuNiBaD&=N`^|EF~u{y1^p5@`mS}(7W)kaxoocoQz;K*T*_1~<&Yoh z0&vZ$EDWeJmN*G4ZhK2zUNR1tq^sq#`O2U!x8P|mwGH$ubyc`F{&^Q*A8D0jdRvZ3 zt&Cp)<6HhQCf!Dth-&wY* ziO;k$vy}mZWZ6_pY@3elSMy$ijyYmBY%9f65+}U3FypYaYv#kds2~Pa*e_st=caF%}jdse38K|Za1HzA%sC)3pHSVc9O3cZS466eIOCx-th+>X0NodlZ)onpCW zMX+-oSi{NsJ+sF~@ObibG^y^-0afKS1P`rvh%Xp66~hY4awSahWFZ$+x8*G?U7IGx!;Dt-)m|2Ad7H1}PQdbBo&sUV+HrqEE#N0F%?MbO!| zi4-zWmDgIzEG4Y|M|9@FnWz3fCUa+qSg{n>?DQ84XN-dl2!@h<%Y#Aa^&O>Dfga5# z)x(zFVltH2^mIXSX880$x!&(R!=31wr)v>xe{>f4Rs2r)DEAQS&P&Hgu{)hOHinnD z;(0yV`>0t`&(oh%x1WTs1)Jk!(eukB-iE1%ek<++4GDwJT-0ldk|10O?F1wT@sTP( z&=!SdDZH;|Y*qheK>Z5+((EwEeXl=z$4fq?SAP)ml@Y5fQ5jDC;#5yUfa6r;sd-p# z_V!otpU$7M+5b*4E=ij7RW|$EsZttPy|4N7K6_i1(`nI}LzBC;=L&(rIERR@EDcOI;*<#f=xnYk-2aax(y`|aC?ccyQn*P}! zu}tKGrzbTz8Y#(N6W)bXf9YV!YCF!X_Y1i@_D1hcHKIzmR-YF$s$Dc@gJyMFin@75 zYKLJR+|^t6J!(pC)L9b1nPD!V@6DDQ3?Vb2VY zrnDK)CZD_+=rb+Mc5U+xEAl(k<<(!-#WMZ17Af}p-}Fma5NeqX=o%WE7+;6F(B}?; z0Nc&8;eLVg)F%aT5JFP?*Rw~~Wv*yu=ECLlIytqrgb-||!e>xafRIE;@FVN>+wtMQ zzWc*_bf2kB>Ugd7o@pOk=Xis9KkPWj5U{hBd(4J}pnb~0#Sd|nvfEd-cKrGBdEqBj z3J&Q@DVC;u*XXYfyD<_(zIje5*u=~}S*F&p_KG>Z<{LEOy)n^}xK~&W zt@57|)s@SXFmc@hV3iwMHzqeNII!d$45Mz`#{ci>x@wS9^~a|;#nB^dCFs$im1Id; zhVCU5eE4`h%lP={6fQ28zVdK-pD}7b>`5t_vOE{DptV}n3XB9D+c&6Ds?QJKI#KZP zN%goYHsB)}_B^5X=65h{#mT<$2UYxPZYVB719 z!F_E7hmi8A00*l_AFpbJsh@yzg{&iFmYv}T&7CX?VqDWFW31nY<@|fe2E0EB!u8(p z)zHRXriFHY12Qe&fUArad-+^VdWam_^^E_Xfn@MKjVdWqK|*i!FRFy)R{;dIZ8Gy0 zX#-N!n;aNIS{i~JZMKdCO5=jb%(!JE!t7`_pF9X2!9XokX5UiQt7=R(fyMLwLWYQe z;D+9uaQV3Sm#6cnspCUXduYb^PNxy_p?yhDQAF=pxQ3VQ#-GJLlIoi3jvXS6y7H)b zp0GJ{65iFAPX?q`U96p?E$N`^oeN)LCO= zT05SGiYnU0Mv7{YTtk?}*O)=!EQ=^limqnCL7BEhNvI*!pq;ed_SYz9TB(VL-yHMO z5M0|qro0)EB*$^+7I-%&;viIiToN0%ia0ECJJJCZJil|eN%xKCasaL)9)x zLV>ZWs%J4FS3L*wI8%ke=zEbqcVYr3h+fx*ru*|yHzNrTed`Tw0B6loaGYR9WzXkN zx+TYDl6*(|h)`*yXEajfgty`uz{v~cG2kI6h5ruhfhZcT1BMSl4N3btCL0<{}fZ`@M!Q3sra@= z9V&OIg9R>LaKM=V)BAp%0=#?lMX$f7mt!(MmvL{#w}#T#`-`Eh+w$D+k{Bt+S66TmqnWwhMc^r|WN|{~Xd2{_< zcj*nZ%1-;=tY_EwV}c_#LTIlkT3C1Ty{wkL5;QuTu9bnkaTd(?vA2(9;&E zA)T%PZ$~@uYY+Dlju?`itXn1r8+pvy?XpYc3}18KNVwNinRxuMIlq&bo;1ZEsoc0h zV*jc7^Gk(b2P@}jfLjVH$DTLQTdS5@DpZOk<&;B$S8dvpNqJ<$)@f!7F{wL$Na{pp zwF98il(bPw#VB(UU%l0iTZ9{aUy~FKN9H%_mn&SLV85u*S4_S=#-y|B?pFZA!P2i! z6_qYFIq}e8hx=x1YgQ@Z^a^3epv5jPtu6tbyq7UzBtHI>jsdssicdoB7p^YWiPw^my6oo}JjEpP}Q{vwRshHkE~p}Lz< zS-)z^txDDZFwZ9dj?O<*Q2|SlLLJudlblvNRx%EgQV>3Z1WbI+M6qd}P!^mjeojhsy#f}4!9=b`?YDzk+f?2)ocRn_yw`I~>{TYnlF#lNwsjXx$l zv|;^<;)_-hWov#{WXwQ``rs>gi*P}xYc{I4&bl?2c??Xzv(6_VH~-ubfM-E z(L)OTkI8e~^;@wCzF#7>%UuWyae<<{+OK*eKGZLDT9mz9*@s*21-lgeN{qXMWum9= zgI+&+s5S||wcs9J!)R&+_Q#9Nr}(U=od}jO3A_FDx>v~WhP#7C5fhi20%vQ>)K<20 z*xa~u5!;T6OY|e@@Nlu2vW+oDhaKw2X-V_1Q$xblY4ho-ULe-^%eHQLzfSpf36AMj zD^6E(EDbb_{qJ8CewMpE*dFMeyLA@bYrKVVE>YetC{HaG&L2j&2sUuuo<6j1%tErX zZq~ATZ|Mdrvg6CA;EvH|``g)F4QsiCMG}mk}@btddtHQhPv{JU)5WYB-SVs{n+|7@O8 z8Q-5JtA*zS`)_EKq4*4Y9fDQ;tVzxr5tYXIVuN_i<%o_o_A6Nb3Xm0nv0>d>6|?u1 zFAOZQG3=2R;AP@G96wj8>uJqZN@2Z)QKA*EWo>uoTK{x+Z%M8>(+>2@2atN9;=r7- zSCNunM^YS-)EL%BYjT=X31BF9>sV2n=@>r5K~mfPKNMZ{gz+VRP~yw z@`;RjR8)ORZ&yR?+DG>7?e_pLC9(6`39evjoXp36yNz3M)3TxJnme48;ghwIbiW&I z(TeF{5D<9}G+;|vs?s(I`{_yKtBmq8XDPp)J0ZA2KbkK*O&+tp3xud~YY0y^J^SXV zk}Z9-w`!yWV6-DmVhdPxB;#I2%H!3K!313jA)&H~@#q9ODa9YWpDy7Z-~`}$*@v0` zQV&#|%1;jV?&}GvEt^|>S;iu5=-?k`8YS?cYn$&#fuGA8E6l_G*s=C7Wx|kgaT{Qe zD6lsl{?d$XMp%4j4*NN~IyCuOihyC7O0{+SNHwnv{5iLqXYt=W zTP-MQ##fdxI%DExCOsdLxsoL6exW+-nq#3vdG7T!q$fXlZZzsw?Syaz{@nI{MU_Zd zP1UFB4p@lsaH9m}axRQxOA?(jM<&y)O1qZM?bqG=_6bCG|Iow*9B=6GSC~!jgmw*^ zvjkQz4&ncNc-nA&=lr|4@MjRh%KxMnw}0e+1Z)pqS*TmN6mQAEubwkl$HlrPJF(hV`X*p2LEq-oa^5mZF z&K-C|nNHl1Juhuyf${*Eh(~!pUS1-*ZQHi;8whycy(!NrbWj-~= z^nC_3bcv*uQ*qw-&i8UW``K$E0q?(>4Zkl2kxF%rXB3o|0}yU9n$LkF(I@Mz@wsfF zcHs;PLbAO>>dwYb%-3{zaigEWc=33n(?4wpWr;Q_NQ;QLerdD$Gf~=WU+bXc5*6_n z{U_g<;(G4c?b(mnIc1i;FOo+y`n%>JBSlk#c#NIU*F?ax{xWh;^Zi{1HA--+4k)gG z@v{;nyycr_xupPy$~UnN+1@@riIxAK1yJ=RDO|i0BEG7CQ{Q@)2LHL34@N!IGyUzP z6A&mf-f<L;i=THgqtz#6rC{{j#o zZ}4*ajn<0>=f;ELN#O%HRn>x(B;a5$WDR=~MO%IQAuD6yw8y)z!K~NrX65H#O$Vpb zRv>V`uPVv&B^aRoB)kd}{a-MOU^X6XGB3}4zc(OoU`INo9J~($*SFJR!G_*p;mCKt z*8jcynRO&H1hqxB6m^;xBRY~q2Cn$H8$%Kg!9nP~lqU>i7BAfNU0 zNav-2aH~%aibD-%0R}UXKY#)H68+ZN(`N`*t6&NdM29JwXM_Y5!b76QyD) z4_Q$RqQh`zSSY0jnVXeVW);+7S@=1U1+KSsy3K*r1UzUa*w0lhU`)HPQ0}-WMZ)#x zc)FsOd!eP7a5VXdFME6~hKU>Nu zxkx7T<0vFRQ=@UHip!QYA3z!pwB7*#Dh(acAS@*pP#uk=OE<&eRwKFEF&=6ko% zQfoGGY-Lmf#7}NN04~}^^3~lGKfS;&eMhaFN17?89@t#v^-V=E9Eaj z^5#uIdD0$ae_H_b)Q@r&o{r7e?$bM2+v`#rMFAx<#>2{ zo-?ClUG>!X{0Mu^&HtEP5Ny`odGTcWL%%DsOi$b~TV1&m5lfljI0XXzOhL5QfiO9o zDF5?~QtUX~&rKEo$QttxaLgd>4pcna;OGvgZ$1Ea*b@!5oLlhWAE3b0&(LT?VDYl- zOqy*BFoirv(^Z0SuQwo6rqgcCZGX|B4Umn$vKH6*?iRYSPC&jD*)WyF3iwF-vH=li zCFGUl4e5Z8&%jpc-kTThm}e>ZC|+A|q9C|`-$Fg2Sb zgwEkdK-u6xvh*P9xOF?AoF}yITvVYawOnT_J*FszoAy$r$v9daOlaPKds+{Hz_tTA zn8045r_Bau14JSM3I%+Jx$+eCh!x)W0tOX=+!?4Q2Ao)nFzEmZ8_vkp4ND+qf7I{k zWFK-+gPjE0x`ng_X>VRlfOr2IpngZje84&V-3Nyqqk=erLh*JAj0ZEspe#{1~QPtfniWL?eha;JbPUcSnFk#_mz z9@v=u=3h{j>#FtupS#=A0kaPhx5hhCY$8#lAz#>Sgmi#}%G3oIo!Q9>Lq6#PJV*oZ zw&4>df+LzQj9kht^IcQ2KL1H&_uZ#LsAvNjudOb^PEua;Eshi2I~)78B4|Am#rG?j7sG0 z(h_$HKrkTnF+|K%T0y#UPEgUH4&?)FAjo;@1*BuwQBC6pDBWs+k9vpHA?8vGRolGi zSJ-Y1K^2JaC8RSy$O)%UHJT%xWc)XFtH6I$`VmII?_TC20bt4xCmX3co|mH9NS66z zeGb-~b0r^N6jffKqQ+>{l2shT*J8>-j!8aL5Nkl72Y+lAnKPCs?Bib#4kF59bPKqb z)S4`+&24T#;VN%rxEW$xH>({;h_R(S|C{bK3L;I?{cmQgV&shWok2M6&@#q%(;}5{G|AG6Y z%E=hKo~z(J1TA^~2cJlO_IEII(gB^8gx`fBQH0>72VJ+uQ|_&?xZtOE^<22~mb9E8 zUu~30eDmp4-up4WNh)wGq_z^g#ww_VFEh9EI{zxXpDltvI2X1{L`xvr-fR^x{3l%? zkybug4ma4I=UB9%`3A3lZ!BG6k*OWo8?Johy6sacXtVjsBY!MSBEvGtiX>kH40i@J zL&fhY-6>&|z_n#FmG%L?EHqx<{Q0xy@*We1R)NqEZ<&w$+n zKaMreKwx6w{;WIuj}ZC#G%Z~(2$$0?)9dljZC2f?Zg%VL7_|-6)E;(aipcwksXqbK z#}@G=ct;R)=@o=WehbBwdeDnd$*?El090$K?~S0Pzc*r07;zb-kPUCfL>^V%YH0!G z@1C?1ot0Qm{SwMqIi`&a6~2eO`&AzBcH}dSiqcpGLcIj$j_4u1I|iIb{gJC<{UT~* z^PZ^7?Xa5nag} zsU8IeoC35q_yC%=J(>dFZfgN!U!{70ga;PXZ%N8gca6hyOu!&J6TOG$K`y^u6KK{k z8MQ#2G+j-b>+OYfB;e$J)VZd}>8cmq@NFP+z3Vnv{f6}9=&Q-m%C@jy--hAdy6tMh zz>UvS8t6IkR&HK7W?eBVHf8-(a6fWklGVbAssF(}h zZYzE7W(W<-KkB%Xi* zkr=M~PW#{fOx-eMIER_xPw2Cv{U^q(c{#tQ1Noks$O}2pKExSB?gt+e#~SD1u(c}F z=XoDwXcgGz`FOn)|zw75$fGf zuzI4G+rw0qQ0L>%YCzFZ`7h=6 zXyGF+zujsRyqaU29|yw$N=o8SASj6L`uy^+0MC}EBHaGhitZZ)C4Jku@Xahf#yfE9 ziENw!yGEdJGdS?a=HJ@|zCkDKD>k*D7HG{4^W0Mnt2Bo)vth0hAS$Im@e^ba>!OMK zg&0oXuKJ?L1J({qSv2_;kVcD2=8z0_)08F(DA~izyf>9KK6~!O180gVVN#b4xRduM zC#x`Da1TS_Yzn(4Y$>a;8)Fk80W5NyO8d^1ukUEhS6Sd znzoVqW7n@5c3mOrA4oJ-%a}H7m}kfHB^~R8u;t7)?#q|0NOzgHL!+fkzhYQ?*nVl~ z;MMd2jRlRUPOK>Qm^BvY)1+Z|q2`%K%Ij6Nr2f;zAwayA5p;X^h4qji(xqPm>q2kw z}-=43pY${Rzp)#~&w8Pu3c(wc$|B0)2*+}@-$i^E~l*$R?@mjyf z{a;qrpWRB!8&Jw}YGGxDCqUCvWZpU~oiWB7S`N&ZKtqVlGegTFBg;HQD(ntXPV3^) zR`;T4eF+R0GyfpM1H}vGKh2v=hJiJ|c})xZh;u(S18QHZmI9hRCG=zWRr0>ni{co5 z?O48Qs$&jMM$@PC{UyrW{J8Ko!7OT+yQpTMlPN_(%mzn^g(bL+?G8q8(~sUjdzi;2 zl=%aGjX3~~Cca1mw1=lh`3yYL`%#}|a-NxTA9?ecGAB{aV>(d9dShPvZ=nW0K>Y`y zs7acuCqI5i#A3{9cmP2P%~&fqKTOQh9L2?#Ktj@qq5^SNZM}hTmKG!ETzK zFq(=*LtCk?!AW9u;!1pJHUkN5QTvLE>ADh43{#gzZ@1dbwGGPL@-VBbQbRb+L0%xJ zxl69zSf~=u{cjDzkTvv#qfoQKLjUBw7&Ho}OHi%098ZuXAN3r2z8es*s~IA{PxE;I zjj=)bC?l{6EN1oI6T{FFCHw!?oBypw_4})rT6WjJ-s3{8eyw;Uu)Z3#h8~4@dCMwDnXkwn`WTw ze-Oie>u>=^`RUE6kP{TRu{;5sA)la)MaJqoS;0WMav6?c_F{-H=yveow~7(_A$l12 zm^42|kcpxZMaAE@!)iZ8;-6LCd|gCtVdzMPzbksR`+?XB)h~Wmmr4wq?_el)Id5cY zEJ4TsSXBQxZ8mK|#fM~IR1kw5w%`!1O!wo}&tTtwH|ahXLn;DdwL_bT1$YJY=xw|R zqP0`vK!BYK+Ey)(OknKd)*J*7S4iFbzbjyZzOlgv16v<5kUSqZR{6@SkNNimlS7Yn z;#KYH(=akNfVcG6_&MtO(zFyURu-u1QvI8kY=MGvo9bB#G#Klk#J_a!{P!E85>qbV zNk(pO=Gm{;)H<&9_>OE<(?@U3;}Sd(Nqt*f_Znwv00MXYAG!ewbVC-Pr2<$aD>S)_ zN6Qb$Xo8|gfFdh-XB$U=!8@~6+(|A>_T_dn0FhFb--qXe)O1(#0? zkb>F+GxCJRR&|Bod~h3nhiFp(HDh0~#u^$gD?0;t7mZPeJGVN51W;8BGs7xS-4TGV zMpr;1ftKGo700{)Vs8Q$P>)!IK9w3iWw)2+$Kq=rPqHY>^Bu_-3<)3#C?C`(;QYPs zniAXto}!`%q?aU4Qc^o2ItLYmIkBVS1WkFJa)&*+{p#b6^M&Oamm4rd^pU?l05-5g zFyew4hv5Mx|L+Zgf!;ydxb_!1jg}y|2ZVG$^wtMvnD@W~OIN<>-#rj53M79dbQ&c7 zU0I@wc!h~pT|?~Lar~ZTv?+K$Ib4#pFqU?rX?SlERgU)8_@mfd8QH+~{&X z7~`!(2fU)MXy|kDNNxjx0$yz@>X>-52C#0s$>tD`O`77ETDOA)9v#JibBH0P|LAey z{R==IqMt@7Z16fj1fy8tC_uQ%dmu;wHA7Jt!-Z1{nq)zyuA4rPC6OtJnf>=={}%f8 zA`wSfP&2$>;|>xNY952q!O;!COJHot<>99Z5GULjU!No38U1%A^SQ2KEIR}CO&R&9 zfj(lMLwq&N?TWoXcMeZo%(zB|0^~{~LAoHMo7@E1lIBoEU)iVq1aFbg$I)ELg&SQ7 zhl;Y9;Arb6g-4rlX4LB-sNxu86fx$hCtR8#Y5<~^uz!yV7Idxj6F#2@v(0Pf=6#We zh*-!J)W+KY)mTndUB|Y5*r?nQ1AF&)}O*7)7KY} zpaOViN9zAz6nY{DuuZBN(euj@OmiifpofNPI2y=A!uNqhm4iXQ&&YL{3;bR-ZUDf{ z7eAM2&$2XT%SP~+-j&5gnyS@Iz_~Q$Lv;a+xyFqrJ_&sHz?9a#b;+*nuei>pZi+nm z$3GCW+YCEv8NChx8Vv$~crBAmR(=*}4IB7L_<6-Dw*k^Qi$Qo3;q>Z7%WyEF<%Uxc zaPwGe_W*818zZ;A19Qth>C|=;F0DA%2589FY%d1(_k_0?I^WqiQEUI$Z5?iJwN?3 z&IFC;GHDj|qB{Eb-`(-Kmi&aXD)(q93prL@vcO2*gK10qj(~2~#3ivPG zAHTf9SP53k8TZGM=+PeDz9TDToDupEav{Ki9a`S795xg`tYYS`t!;j3$Z{#B-116I zvpUt-s352fC%~`>6PzAv`P>J^n3j{t8cnand+2kxjXhEv!aLhB{pu=74l+CUn*FaS zch*%+OuPhZ0&`U7RYazj;symQ$PB44RDO;EKByYaz{)Pzw=xa=`pj#ROQ{s-#Y9om zd9@O3N0RDX>G`!jJzuz$iyuETd$j{E^5cdTyCEetFJcpWgPwQX!>S$>oT*>!OWD_f z6l8cCJxac^wd~9Zewh4T^*d~jiJ4RE&dFOG>AgVL9?; zxW=_sb;IysLcA86sWq81fLTWYFHMG_-K8A(c}%=K-kz&n+*c5(`-ELL(ky1LM5<1T zv%;|QoC-W0DDdNE_0r_V9iZ_p=|a)U%7Ff~U-l9B;YwiUAS~0H zH#CED)%^OZKLG!$<~X>Om$sX|o@|Bi84X?$tot1)s0cZm{q+RZc?rlqCJ^q&rWBBR{yDx59Jg=3o~JG?GfL4eV{Ek)p?6Q*o$LT- zaErRdb8Y4mt&)I6BB&Qi;$}pX+i;pHCMQXpx&gW7^KZq3iJ1Lh zc%|1~Hwm{#g|5XMKq2$eajpHO7EvWGZ{y(QAC;V$F{io$^$6XPd$8as#_n*TsulQN?5St`|I70Y5BGJaGsmd3bk ztl&DuTMzLkR?*XO8ED}f=KFx2(lBfIQybRrkt(j52m3lgb7Ie({Un=nKkGa1h8>H7 zQJ`zw_&nZ)?T;=H(_@h}m{B+87;?%X0L_1D6QA;VVv-Tk(nAsmT%p-ZPRijxf&Xdu zP|91Sc=Vta?@O93W)C{X?HFr+9;+BREx!(Z@Fz=hV~9Qj$kG+~Q#n_(x%8VrNj|!L zLcqq}jJ!H6&?u`15J&j}y1gdcuDmZ0fR7h_5KCiCcBd6H9Vlt>nylr)NA zG%0c}ZClYmF$spCB-wpIIGk?mJv5dQ%6LlJ%S~gaTnYuY^D`ihEWr>cU4VJ5vs!k8 zD#q*a3Q<&>l_9|{yy>*e7(t|_cQQW+jLzAtW+{ze=^mtFjI=&uakBUdtMOce5g56r z=o=J5<~A97U{E=(JS0mH{KL-epJlXtRB=JdFdkNSq{EbonftWX*nPp$UqjY3<=44b z8gF~9CdzIudZ_dwtk`8`2fuwlJJop#d;xK*5d_fTKg==KVcTUEaJI~9#)J{y#V=C_ zDJ3FvV{x!XmQqDmoAs4IfT&S85A%Y;pD}A6^?5Qx>&P3TeJg_fS(X6sjLvZ)w*9Ur z%To{_!^7FAC>X952YoWr7u!k__1$kMJa%;Df}^#?*~dE@-J_}(bw)FF z7Ov7l@E&KLw`LWt)CeB(js9ur8O>kkQMQ&OFOUHKvU(*=CW=*%rmo6|KB_9(qNCcNGH6XduH%J zS{!LkZSM`;XH}SnK;P7rF6c>^2mo(giMxM?JsqV^*Y4Drsmz&9;koJ5b8?2WwUvgL zHv6<#wDq&f={YgL7};C~jg$YQz5DFVldBe88A%ou62|0jD3)O}I8EBC{PUP0ClX{x zVuIOlN2*+rW)UQ;C{k!6IMR|_`yGrIEHpX#E@|7psd-f&5G2laFviNkW;zuAG>T$& zCRsNglq8Xq+H?c+yMvysutZILs3 z)c79@8k0N;zLIa{lK8kyQ^s&4MLmd8lgMT+K%K*19R+JTqY%@SN(zjgUMq;#b0Q@8 znv5O|+j59UZ+ObRP#$08!`~MeurvYaDfz^CN)kJwRs|_)NH(;+_Pzv9jwKZujycf~ zCnuo3QVrf>vKxj?E$JoPd-zKVU8_#rsloTO_b1VElzR2ZEa8GF#?Do^swl&}*sFhd zMo<1Ka>Fs1s_lqhbCkAm`YF|f2=kYxRE3rzu~dCfSj-%8ZMQI_vV! zDy-#HHMxT(Qmi-?Q>fpMUg5nMQv^=U7i0NT=xQhl2jk%|Z!|7H!Ag-=u(IZraz`{+ zzlcSG7Wp=&5+?F6@2#~c%imWE-H-`O8Ozf-{?VuN^rCD#Qr9$uwH8>bBWwLkn-Dz? zdvT|7q(hbGr;%L9+`h!?qR%7kN@<`M6(p%p$$zEwpt$RLL-bG3qdB{QZY*a?rlWv& zV~uz%K)>%`$9OKX;m+u0FiSETy{%?=Durx)rN!V6k}i1*f6d@uXqagQLgvuAeko0u zyiH8`7KdIR=5KF&-1gP6{}f0(vUzfl`#05mvzH#iEJ)E)@sb9u3qAWs!zkac(NpO9 zvjHbHIIE40@@e&&r|0D^*|r^26pDpD7`GJ$g+6@dDT^Mxy=%jw4QMIp)QJXbcAatF z(}@~?os8zTTTAih8CUZOblg?fs&&}dW)=E52a>w?8;8jwjPd73FD@3z+}^#BKR_PI zERkBqfI1Q+Llm-7{u(HKTvRnzCXrH+6|Vw}h9&^VXhRswQSHB=E(PKZG4uL45-)SP zdN1ie77%|Gt8j2c*Z0*1TjL5$-Ca@$yc`Zy5Zr!$`CYs83;w!g_x>NBX^zU1bJ_jV z%hS`y4;FInna)j-Vq{dgz0ZX|*vL~XB11piRu&8#za;zabBF{#IcsKyKDaa(&FH-m zl{DmKj+ts3*G<7skXtJ1g0D9u80(cyfQcL|j$G2=6BfUnH8EKj}|>rlqjOhU|Uh z++tL56(lPYN)wXs@16V>H#6aOW_=QiqBU_%ZI9)W+0w{;tOpNBN76EtlAeZnL4#2k zAG4`lgEqa9W)&SmmhDh8HFB}7Af0n4C3(yhSPT!-;3v5adRvJlUK-UJ*D&*2Zfmev zG;L_P3}TO*AAhJJvRt-#ITFCJ!b5x3u(f|9uo17JNOOR`lqx?j(lMC%S3NBksVsVH z$~K}C^x-B)${g*O%^cYVTlUQdj4-+o8MB$srVL+VGM~n#+XuBYlFw{L%z;K+I+4GO z({@mj!A9Sl#u+D&cp49l+KsKWWrK7b{iwaRR8wz$28n3iGeO@$xn@1#ZDQp50Nr7- z%DS53q{56rsgYacM$BXHH3(fy%)Hyg)lgK=Ov8{<`KswKH?jHN)-~y&Kf(*4$%U~XfJL&B3B}w=GI@PN1lYeZc!9vOuu5EQ|N2B9nn94cL>CXav+x@T9 ze_z)+W1MK!M<=ERJl^Jdy77qwf3RH$t1BOiS5-};I3Y~5D-j8fgNlxpkGL4?8aXDC zZ>t(*-J^|fX6?ZG*_DKIuu(OIi<*)*cEw=TbG}wrZK;?V_n^bY^adE-gBX5|KjH(!}tZs zowMe&k;|N%%%EA*L&a7V_t1rh0jZl$q4I3ye?5dWXO*W%0^5zWB4{t)JOS$QHHJ6G zl!Cm;xz`U|4*GK8gSX19ZB=a#jKp<#JEj9shos@&4VyMM^GHcHPNBnV-J}hNohIZ7 zZor|3I3V}WcGa3^L9KJS$iG8jS7fp$Jv17Fj}m75#a!61x@J<9CY^Fj1iV!NR{^*v zo!cnshcK3H$zswKyM3*sUBkFzZOxZ)vZ0SN`&` z;n}l*FiXDV_gfs7<4J6O7HWHYI?IJe2WaKDqcb zto;5NBCi>@pw@>Q-rvYbUvka=sajRoLOQc-_y$9HW_DBC;QU=I`Zo5&L{Nt+ULK zljRrCnb+*D)u_Ij@Te?G0Ao89YjT(8kFxXkFyHSpH3I$bnlSi)nWeQdn{_EDH0DBo z><`Kp@JOf1Z^Y))$`XwHmh#~h-oqhJdi`veU!jgeU4r(L7w%Fny)Mbh;CZ)(gbO#z z1>Wg}dqk^XuYY6pglnEN4D9$z#@jx8*qi-RVQ{i9Mh`-KA8^=4isbgzTrMJpOl>k9 zQ+97?b>Cn84Kn+ztM7tsBCA!60vV2LHK&**eV&5)ORb3ho!=&~RMNzUHMb!~`wtm% z_xyxsrr>;)KND1g#w>;_zijd^q!6!-u$Cr|&F{xP?j{dK=^)ZYavMWK zPCzy%X$x75uZfI|Y3gREBznbyr+8*|Fa$ngC$hH|XGW@ql@-^JBp`uge%DKVC7?7h zsLW=RtZH;QJb~@N$jS71E33+~WQk#`7`l2%VMg*CMV_XrP_KV!&ObJ$2xEKbr8h8{ z>B6T+g>Ev8khomt7AN07N4QIiR5IwKkI{>0N{!ds;qZ_kxzLmIat<5>laz4#{Gx!l)h4R6Z-Vz=ezphVRSN8V)JHvY?MPN{j$#H8sXenq%vb1SmWi7- zvOkBm6Ei;%Wo%yCa4yaM+u3Qn#cR)jo3YV;$2f>sQ4x9$%?0Q3RJ)K&5gB1iH5mJn z%ncKi*@3WXVex^1kA+N8`eucbQ-|qWsMq}7{wvRZhBuKCR0IJ&7$Mn3fhHh7|4L4aXxUnx^Sh_(%S=B)yQ3ZtI?cRNx~i)sVogC6F# zY8V!1@*&Z`-p229?Wy!9CW$v_Y|Dv;EhK~x^u|wiHiv0z-wFQkW?l-tq|TtOw=@P< z05PK&v3w}6m)~qNfx=5HO6c5ykgQ%ptnG&60haQY;I^#_Gbn1;q!u~T@_p34jAa%x zUbAh3kr02&LNUkSSd)uFhR*7Mx36ib**Lc+`)!$*Wj(Ln^e1~xS`^W+s81m*`3Oq! zl{cWqgynT=dc0;i9f?p1&#tN|I+MhW$fq4a{C-7}@x6-T<%|QpVJY{&H&M%d z(Bby#`pFIBh?t0Ih^8yUNC$p`93&gUnJoh_E9Fqde?ciMqDHQA|YybN(HP)yS zyGbxR^%LVP*?svn6wkU@T^_?j-9EzN*2D%=f*7#j|-ooHbcF z%qPuMcFhWycFp$D8^n@FAEi+2mdWpkV_?;HW6iYgdG9ZcRA3o%CF*>@CT1Z9C(=55Xfcuc4HU+=0zV$0tv`uR0Cf z%Z!n=b7beEhK9b)D-lx$x5rFE@Ale zrQlD9W4$=r8-cE?X-DP^tcgv2LOt4CCVi3x8KKCIDNF>?L_+k=| zmxiBdckz^l1X1a`NKq=TaTAJuU7jeMU?Lgr9BgrXXp-QENjO^M)M2YG{;?z~A^MK#D?%2%ib+9{88b_t1)z;6i;OiEYekTMl82>xmt>N%PA*+a8j-wM zmg665e9E)i^mhQiWjZ^!@*;m?l@p9WMK`u3!E_8hdC7PowuR7ww;p6Un%2 zDrT*f!h=(~!Oo~fC`}-bh9){I{gYrQY2cIXNw3onF)Hk10J>}PGTK!!410a;pm*y~ zUyj-;ne2i+e|!hXrr_CV^?z4js|%!$*qyQP`L{ zmLf#mc211;<8Re*`viU?F}Mh?D}A6E`pRQt^j~g2Bn_FQ%|Yv2*yMNS6-mB_X@P}0 zOVs$rCfhhm5D(#Cwjx!f(0tvAaP<81neHoB1v@-63-E}vIvg3j$K=-aF3bU+z z{Hj8fKpvF+-p}^{JHgHEfTU%ne#6N4WM%63cSF+Xg(iuc zq(~M-F!1^dOf2|GGDpkX`YmIz*Ig@ z%I*J=M*B6Xi|t%5(|k;O+XPm~9bkbr_?(8Po0VZJL5ZC(GamFP?1WPyHF}IYITmtO zmlj)z@MQ8mv1OxQabXD1E|T%oIXN6+x}}6(FOkH3j89u{i8jdgLI10#_Iq9l1-^gF zfqfPV2cK3F>u!fEPlMoyj%n-nss%?#u@h5+mU}d&@*l3v3`436w0lP%>CqA)VdT3Bt~6KPR~Jr?EghoWv!u%W+tjua*bB~~Gm+!)&%$^}vUOT^ZgR{&hg1@rGV3H;R|e1zP6W9W$g8#p zMykR(#R`?4B-t+g)+l;MsNQDdzWr+(%9vrakMW;%8DXJes9~9+-=>XcSi_wae_%Bu z0}_&CsBA>}`W9dN_qNebKa?)<)J@#R?rX;XW+&{`o@Y>M_tGyXhSDp{w_Oby=f+T% z5ftYW(VF|!9aB8aJ*ap+EnTdPB_udsxcVW8g>SJ7){Dn8I|n6bX)H0v!*CZfs@8~+ z(>P1{n}*g^cDCGAXz7G0NDKe5qz!pHQV)k0!~CpF>xr&VE^l|b%5bw$fnjVdmD_YU ztfJZNJlPwn_4Jj2+Fo_r1{&sA4C!stI~)`_2r1--1gRrqC0uj+BMi?BG!jTP#EOTj6i3|g|ncEo>@TeeEh+sGaLz`a=XFJUw z=1%qFJ-+=0d4XYagRL5+J}2sdbc9YSsS3ya&R;$<7DkAi6dr`&SQu>~R-kZ`h-&50 z5!W#X4f!v-D(=dL=el4cIG6binO2RCApRWkFn>;@+rAXBt^fARy|j}|?60P6++Yum zkd4SR#L(7eNiSiEo|S$Xi+pRRxo@L2BAbMoCAWF$Fd;4(1(r*cX60v~)~=lJw)qJ4 z;U<16rTC_<>g91XhISbtr>@#>;0YdwIpV|>D4igmQiMV!_?ch9Eu+sp9(5<|-Zj)6 zfk2YhJ9kz^-q_eq#Nn#!H{0qrm8wXzODB+X>royD2{e&M zCVnwH^1V)cWVXBYw}roK7Vj&OL@GZ`Q&WwYGZ6&W*N9n^|*-WhQ>^tV8iI)uua;k^E-NzW)DA-95&K524}@%Px6ht->dMw zhvtWwG~*P;1w7L{#hjrSZ7+DT!w<@AVy^w_I{EW>9sJCgBu+UO(SxVa&Yr`I%CCsQ zV~kzSk)9mL`Oy%5Mm^;huJBiH2yIFU-jDaw2$u>WrW8$vU|9C6=tdFVMbfAz&O_`8 zKwTELUCe1yn)bJO&3ivrZDSynG3Qe)n9u!%=(b$EstTi}d23o&m`D*7rm1kxIhQEG zTaGk13QX{UK`_av0pEJM+dli<(RYe*?!Vy#pX+jEO>C`Tyl(c3eb3}Fr6^;TjTFU_ zx|gYurnO-}{Kz5+M+<+dmSmA^l?6--*(lMj9Y_(6s5ye?7SDxK{DSU~#rD_I z8&GRdlA;UoZD@}SMrl_SGpXJfphMFTsyh~>X>K)GQK4qA60)(M zWhPR7Qj#}~J&fPAo?Uuwk^OQ1eU2gkQJr=|24Gm&6QzxLkg%LP4OulFSc4{)dNjny z65Oi)kxu^SqeS%EjDAoZ)j1$DaTj7OV6?F^rpiVC<<@`qT8)6hN-Si^qGQ3p@?}o|*tY-1M@BJy4cs0L-gfOz)TC4E4HN_adfGBA2bAouf z@0AUr$5t2^`XS;Sphju}x-LWBmFeuu=~UWP-9q+4V(3`)=c23A->K#w>C2HP;}gTu z`~)e8i3;e~*?8CD95ZI5)PfhmL){mLyj&dqglcc0$AY??)*q&qHuv|8s9Kj!ZQAS2 zM=y%X|UrC-jBg^J06v zh|=T@bRQg}PX@cKHm_6Pps4i1Cxh_N!s?@=NF0c)5*HaR822~a7hg4BFx7+;Atq=n z55LY|>a1U~EKG+vcm~!aI!H$)*XKUY3|) zEEA(2jvlYa*Z(x&yES`alKWX-YgzA_k^onG0vK_5g3ycFo%0lD{8?4i_ir`xW9jT%?E^!6e!pdeGh)JSdLzwfX{q|} zFSZ!Ny!Hl65h$ezu?%;=&%U^WsHRrn=g%{d|Gnt@50Liz2ngCZTuues zsT5BG@z4_A2rmJll#K4ntgZ*#dl+g7gHW;r;BtEelCu8|hl`Gb_tC-yLd(74S^V&% zg-CJc)bKBKIL0$YraCXs3Y*cO3pf=Rts7NOs;Uc}9>y4xi}wQR`}&FVD6~ahY1oQJ zQeszPY-TbKMRs*^QOKQa0jk7s+CrE#P$gzJi=vRRv)ZaK?gLF-xSktEx(F#!Ef8Nz z4`qu;CvqX~OpRGVeA1KqSHHeIwcS}-u6Ov76=1Q^Dh^8=(GM12Xq$>D=PHXRizQ1x zl5#U7wN2&8zF9O{1cM*D<(PyySgB}z)s(qZ+V0TdlGB^mZkhK4EH0xRTQ|N2hp(Sq zTlupEFNwQ0(9ZD4C&t{(af`&BI)A?;WUe}G-+l6I6u9v-#+AHOtj|VXbPnYEsB>RM zZ_y(~k#2&!3TQEtu&%Co@X>DyWs@Pt0VMMPKUzf*LJj8z9MPqnd_SY^Iqj#%?#_>| zdmv&(KrT8rcl`sK<;@bc4}8gOj0`;&@F;?C_j?_s&8aVf1s?t6Ips2;eyu}4 zjS2>ca{)I#$1)!lNO-S}p)!G=$~?$)bUBAORRc|!QqIH>;20dA0rnYfo%_X3UK4dx z4=|8UqZ$M1Te%)U{i6_NJ3`3^rujx|jH2#C2#uT*yg-DYAumiD#s9FM8?<{Pj8lzU zR@u1bC58BJunu9$rU4>SEfD<*SP*aAXaGU!?f28(?GzbF3EQojF?u_|4vN#7gGi3K zCtm21VHM zkRU`nxxi+;fIL+-4ZV1@$(-MAD<^>6dk+}2A>a7-(wyh@y=`|_ZY+ldtAVQK8N_TX zo5ac~KXc!=K9H0uESV0+H5TyR9eyYDc@RpsK(}Irg6?T`ti{c9MqM2yw}8I!JjO6~ zby4Vix~lt#`(=F}h#xj*I)PFup_#2iG(lh~AYlkKk^3D~Hrmw}y{*k;86#u%?`iT@ zGKhU71BiRHiwizy>tVl_G--jzLz$NqCjmMfwmGPgi z33c;ztvVs0JOwP>0_|rvMOCzqFx>vH{xQ~i$3Fak3#|cH?xY}uLMX^lDpJCn+l5dI z&t%!VxGA2Vn-S@F3k1Kk=YV_VpDz|KO(Xj;x18<_}Fp6EsiTAW1lpKNanCI5~|qE>23*;(n_M zyQqU%rJNJsEi`P^h*e@*|Yeh`|!w410^5kw&^9sp5KmBw@ z0OCJwa~h>T%44x(H%WDVd0HMt#l*pbvR=FC;=;+n7cXq+V50Rn$WOE?WmI0-p9)dh z=w6n#lfn4~{r2BSxA)?;tD3c=nkL<1+nDX1`^=wP?tkRQBPU(;lg_dv^5DaCKS~bb zImdd{>K+)h=6Dzrclm@xYAa4ex-JN>2xfhP8=a2sQEKo*(zM79#2V%Q+lwk|3VUWY z(KC=n7pBWwt1fQ0$qk5m>YskcbYw7L{+Eo#sN>*VP2JGQxK#H@HH zbBH`#sy0zLa_{^h%iA$5+Z(NdZb{9V4UjDo;e*EktGF$i9fZ8D9CP*|C#@<}+?J)1 zJQDT23vYt17*KXD_J*Q{dz$A{kCIbh;vMcZ!K0S{A?h0uVO!GDh=CIN$*S>*ozPBx z^@EX|zk0qr3(xcAw$5YLoVN9nXn5_*SqnAHk{DN zAKzf@`y1w;ar22S;Y>m-6s>GK8mw>$CX$%8fZDsHeDPx1Mxaq+Vlk=PFWrG*{qKcM z<^;x6>d5zbc#Y2~&(q_^Zu8UaZ)Ur+lZm4(&5qj3<6a4xTGP|tt?mgKfmTC_UO+XK{ADIA{}QCG(6KzH$1^ozO`fZS zV|~M9IU}nzbwy7js*Hv4VG@kN+-<|zVX1O6oE^oj_e8_szx46fQOJ?45Y;sa`eg?7 z{VWKB5($*{|692qB2`893)hTsi^Ju-ibwNDyDV?~+`2s{X`>5MW{w-=m_4s4XE0!3 z(Oc2;rit~UEc>}(b!pE0PiGJ_4H~edi@I+`?eJdA`$gjA9N5HW+CV$JALPq{H*c>g zw`_qNogP(@AtoaS6BH$wr3a_Ei&hg7iX~F&h8{5rdBj*}=h!7=LAW5vpi`4)oaLdUanb}M@LS7nLf)#F-rO{O0 zN^h>}>NMZ3IWdE?9_KU2(I(9-H>Q5kwR1U7GJ@4UWrXpqXz(!6fOX zV8Qt#pl8769J9Q;sw0d2hm6D)K+`XK$qA9nfCpRT45K*FL+x607DUc&hG$>dUS`Qrc*)U~d&x2;j&KBM*;C~;a#hgvH;{!RNP z{MaA+9r*QWmS%Wg$~r?;#CLjaW`D>U&N9Rb6B-0^H>J<8oi(oHi)koY5PPezEWwWJs+^7DovURyCaBZ*j|5 zl8g=_zC(ltPutI$C&|bzC1?l{<@(^EBX^V)vdB5qv|*(_Ec1u@$>nnnID`zu)D-_# zuw^on{`($zlMG*N6Tw2^{KO1}IIIc|< z)WD3xp1Z z{!n9qW(q^rL?KM1{tBAwPpJt{wSQ$cfOQEs>uj!S8=E}&X_QvF zqSr+{vX2=dhl3@_V8U=D3pLl2{@NzScr>~>k0t?$3NccULJYgUJ%(x;q=8%2%un70 zTp)(MG6RqDO}l%x8<8R`bhnNgly)u#6X{Ma*V6f>k7qQC>+C_{je*{X2jZ^DC+mi$ z6-u*;edS>={0nL6W58`P5F<-@lhp4}Kk5XpFctuqJ&SBSLphgR?pQN3ek+JdMvtsD z=B(*A_cct!VH|rv#`kl8?`FzJm~6RZaC9nT%8@<48N;69zv#+;_ zKo%qxBvHN0NBKM<_U?j@fouH#o&ca>UKs<%G35Qfq$uL6YGvfPJP^c07E1b~X6WF~ z|C-jXSd{Bnoq6Pof8P|-DLErOOb+T%#NDiks#+=Ag4dwSHc?cWl74Ry8NA+hHU^Yh zsIeBZi(Rg*%h`})NHw@C5qXrF zHhC4T4TxjWcYoS|a%H3PV+Q9@FN2q=7stvpqRdEB?nbfTtTq+{WzScpBeM@7mqCrR zZl)|ZW9Ib9NfU}byo;!qmv9Wp4ZJ+%LxnR{OkJAjfSwW3InCo|>7vEf_%1d6U-!p?HEF4bm+! zq>Q-I+?FkAX~Hkx$us)gIjnyEn=fez2SWh#(!Hi4E4;uKWu{GaWyb7|G{j}-#2UX7 z^NpS?)Sd5XQ_*0(55&#L$Od;t2$niL61df3*#Kbb zx|7`xG5I=A$sG(JizA{a=%45g-_I3Qho3#JC7Gh`&p%a=Ks3MP2L=_bR zE=4EwurM3fz*i~~80NBGN7AJ?)V~;(043Not)D>n)o9&QMY}Ndkp0(GcOl(aX=ZnC z0`?7twsU`>Tt!NN4-FXZWwv`+?4z|;t+2g^J$&m|tWp?8=nYXT{*=y^3McyJ-p|6R zo!34!DwtJ`1au75ry7`+Iy8sP;j@ueC1T&)1qcs-rAA>AG$j(S&4D5Ujxzr}`YbCL zd4VSvx40p?*D6|5cCnOyzf`symO#|km!-6`7u!QUuB-!eB5$1^du%1c z4yQb^AZ|fDv2bsHd#bqvnXJ5hxk&8H$_|vQZl8E}v(!D3X0h_a{>x=*x|02%4G~O# zB(8Z_HIVpmM2%-#p%O6yhZ1o5hwH7v*oKk=T~jbb*W|8XJ(-i0Vg1U;ITS3U`P94) zlIRKTXD2Rbqa%8W=R9h-5*gK(%r49ick=BHHO-;x8zM8~?wLLj*tIHc0_`Ci>2n*%t6uI2H|FQy5{c1G6p<^F6q+tc;_hjQHvP z2O*Zf<0&B&a!mO=4ATbC%HL3*$ACeR$qz>vp%q5U)tGI**z-ZZCWsiipD0DEuAZ!7 zj}_x>Waw_$uXkQ8U)mZsY#! zP^z&ZZ9sr}TI&zDU#%cPH``skhp^Ley!U3UR*POu`D+)FMZsqm(c?}Kj0Vs(mUTWq z%t;mfQZ5pgSc`^Z{QPnYuoZivpeESHs@@_zkzPYq7Rn^!$G=nX^M>KV`G-hI?CS5G z$Dc}RAifXOND$);7zun%2Lp5w5B$ex`d9H4XpH z;;mQ8)`t(dfw<`HNXLWwXFhv8q4AKd(<)nCR1^g9l}eb|1HZF8h*?CHNbwDnPBE?C zM_l!hpL)rMo63*_{D&atY7t&(!pQwJCd2i~QsYhvKBC=gzb`L`3;pr*()~R4URwnd zZI`lVo?d6H28Hw`Y$cZTFuY=)D1P$pOsVMKZ^U0N@N_ogV3u1_!!+Qw;(MD%WS`O> zU!@+-yLlSxe|gzaQV~D-I`M2(f&R}2_<<^mwvV4Sjxk__66 zBPqBv-D>-QjK%B0n8*xNy^FwkTQ&GFP#7BO6f;{(jQeEt$wT&~Q$Iy|@y(q)19 z*S6eKO1Er1^}#Aa5=Fg&%F}$1u|dq`Li!Qt99$5uN7o>LjG5uswewqhxm-;1<70zs z-AN><_o!qi^AP<6P_|BJbUvu*=l!M)nr`>HKK$L<*tz~N-{>&&jDhwu^LNNAja^8x zB+#id6xnc;AgeXPL1Q2UMq5Ppi?h7;Qx_+~vtu3lop~PrE_JMSDd#gSckw{pz1N$8 z1=7a)7BdKeFpJ=O#^L>J^H%})Mf2ayzQq$_zN~+CzUzNjYi`lO&S!?Pd$^QKeKEX5 z{AcH}L#|;K>WBh2`w`k&w>~Jof#TM~sSj4Yj1c%L28nNs22b#ASBTu`8cxK{^A>+X zOa^0komuehI-Yw~Oon|&(v-^t;5x|^NeSJ69w&H%7&9`&kdmhvL z`e3T?pDR3K%WJ64Hf%9U%!itBblHVgd3ZB+sP5#zb`ky@ZPCbxP%Ez-u-p`TF z>esT-F3-@yd2lu_!>u)Ioto>_8JF{0Pi-AEW}b$yLvK2cteD!^faKy9+LcjAm1P<_ zX>zSwpLTr_&G8e%Kw7jUPg1A~?G1pu~$d|p=|aKVa(D-RZ69R>A@R61l5cr@InL*1m!U2uQzC5eOM!#;CD-mIjAttrTkU`#5ZaF&IqBw49eZV8 zpkP)jGS+!I)bDcP*>zPo7ay`MaCn+XwE&MJc69oDobO|g%L^S4L>&Ux2Wz;lEt=t(bpXunw!;pyreY!;pX`NW9lsfqKwwHVZlMVq)Q3u1__arPU$W| zy1PqCT2eZsySuwfI;Fd$LE>Ap_c`bNe)&(Gnde#SzOOtv0*=eepY0F5pJg1!)A;~~ zX}A3Tc@nMUQrlrvMV{B&8R9p}^(U2)Zf;eYu~u1pOJqS%KM&#ZcAct_`+3DRjmH9f09jky1(3i ziO;HtVjd_=`(hA(?rRXOtdM(BhFP2?&9Y7kMD5?4IgwYB!t?mQp9*YDP{7tnvl zfx;F5YhlMh0_G>id9#$ln(4&G0L0-3siQ6|*H~OG;N?XlnoWBeO}@gdHsRuCev{GM zds5pDO9ZV6?pxI!f<}h`No5iye>n+CF4j*gpaL04=`7lPgS=M15b32mkDZ|v3p7UTd-N>Fb&fCV?)K}+rL`Zkn5 zmfB7rM}qn62JXCf3=mmhCvpdwddGinJnb7VK!@?Fa~3ckJ+&J@mJdnC$KucZzmEuL z0&3Vk7Eb1J0OTJt%oQvww6eL032nTcjMfq3>$U);ft)$m19rLbZN!RwlW+(P?Y;H4 zz|OnLx#$;@&TU7I6$^N_7$V##hyzB@?^JPWz27>SUmTRle78Ul;srEs1yEELk+`Dt z2UvbvpLPTBbk#vN!J$$&=obM+zmeb>CFy5*_x=?uh|wg}vv)o~1;J+Yf`(J?CD#UI z675xA-x<$m582;IuQpb7a|&`Y*X8u)4G+Bt)Be97i&WkN!ct)ZzH5+e`|rkvOc*kK zks}!kG7z(LTippZ{}O9@d|gVo9oBoY2ON>Mb>sG8?dLKpH@jRyh={Pgtzs(k2=_fm zJL7^|`}+W*@YuD*^W7!XH8!$YCxygz*Xj75!LkeDOrHLNoSHIEIc0`-im-l3!W<;c zt(AqpPbBL&_nq4HN9X634vBRa^rglX_om$dB&)cfL+8yLI8Q(!uyYrS(OQR}wkY-P zo1^SQp^N*;FQC_!otltEU$8RQ?U9Bc`8^M;s(*=X&c0iAJ>?~|hY#8mp#3^Q!txUY zBw{Gcj?RDiTbtp!f%uX(q6I+dz({}NX%QhV9zkf7Z_Zy?>LB;a=cf73LZ{2Lf_QB_ z_3!`NS;0%SX9=+ehz=_qh>lGhD=R?Tu68&6bKiRE6=g>sOr%r3?(*#Q0KVdRaC|ql zT^Cf^LmJj>2iig8K)zyEtE+LITpABR{z3ju;9<+#$31c07i*cLgbf$zAV8At=byL_ z0Q{I!diHv6bofi*%J)UF$rH<=Q0xRSXG3@pL8<#k#CZ%YJp|hq{blP=uOXl^0wzGj zs_gy^$WvVU=F$NJzvaYt*70(b_?*@%jXkr@#Rl*yHN91(ywPv|yIcPEcrc;p*iw}j ziq!bn{Odk$+^q{5*QlEG{P^d2g;8}}e|>8!087sG;cvgsng82qYN;$0Nk|JfnriDl z9ra{_tqcLL1T^0iOKb|-SgrPAz%)Hu8p-TLLe_y5dG)be z%ktx1nHI*?E7&yX(47`xJ5Y`cxt(CK1yzLbKG)S5XCL)eZDI9VGO+ozHhB_^LPKrIQ;JiA)SCjIP2j7#3;yxs_5-rX7zjmr+eS= zE7;^OFuO!LFT90tt~*ePI7irC5azBaK3D>UaAZ#n4_hFC2r^E$$z_Axro3VJA++7; zuMUe*>83vj0jlE$B-S(VnBcfhjmd|`3TFBv>~*`fpd2*%vxxl_%(TW#%(dn~?uGRu zB=>daB>QsCP(T`M(k<29-{$piCjt+2j>lzyfPCWC`0}4xCe5^JI!-gdc~Iav*1)3# z0e%J_+tyK=K^X^U^PuOujOQ%LqHnAp^$aPC|A89WfIDLam2-2U4eDopq}NMVQwg7E z3Tt8%p?Mv#2HLn)4@Ef=NxT8joh?*hg8W>F&ggv`XoBW(F`!@h7yOcbOR-X)Q~4k6;*b9^0; zlwuk7b>~$X_+~T_VZ{!hCFkHYRF&Kr8{S-CF-qh2;z`HD;Wzu86XTTSAyWCXbZh2Y z4@i|L{1mkg2Und@K&(41lT#g6C&z*ZOHA1-6MFl;JV#mUca9H8hCh*}AITo_#rO#@ zOF_~@&qWnHtQNh;1GSY^2jf;*ovYO2PgO+hFONZXCl81iy{w-pv;$YvFF%G~6zyGj zj3}b9n>A{;s5Sar-t7}{2{0gWZ0Nw+BCf~*aukk@KB^3hKFseMh9fp{{}prCZYtqe z7Ap~L+Ks+QDT?tKaHVd5!ckS@1$w=05W1(o(G#+L(K7>nGE;5na1SPei55i&1pjER zJmZ$V?u-4lv7Ett4S)wslsL+kK`v(6x#|e)mkbby#6U9gm#(xy<)RV^#)h&KFT@)h zN(1)Z;6*yRg%F(16ssi+&`q?a07GA3;FodwGim3-H4(X+Xc_*0+UHmtvD+qrpj*a6 zh*p46?l!FB%2FGgJ#MSl;HnkF@Z2Q&Lptl(JvNli=Rt)CDf(vMy@95j7I&$HiDQ&7 zP3zOCH*5aYUWn-tdOkv z`!G{x_zc1@u8cJdW^o+4w+nLS_&|)d7Y*VF&vRX}YO1wBsQe%U6Btg+=rMW;G&$j} zj#}4Dx^?!%icM~-f7D)ET7n_OqVDL}G&;y5kYr!%u(c5xqj;iEO6P|tLdp9ZY_|8P zgn*$>j>y^UkJ2-Y;7HC>nEb&b)bUOl{)A{WOE#?l+Z;N&ftc;$UBYh!zYefr{NE{w zZ+=?k3_>9=ScK|HWH}e!6^2zCX33|!w1K;Yxnb=a0H^Gr0RNPjbDJgt8kS`$g7f^@ zqH=~?v{~cJ*zdFbUO5+lwnMh!0eQ_s@oZxChM1d{BYtrDgYma|{;r$oAriQ8p?02C z$BYRheIgM2zTsC)DD(T>WF!gKV0w;QPjvA~JL&fe zXUM#q$t;PZ8zEqu!wjJbf7j{$%5Qn~j&|`~85yma7wpe>=Q*+ypk(#^pu`X5ch5Ia zjSkl=c^3)P+64v;FSycZmUklLv7WnWE`$s8>F!Cco5<|<6A-s*|&^?3&6;whiQX>r#Of#ZgP`a zGjzk*v|X~xu#llV>2MH>yheA&i=9ok1Ok~$U&`DGWfA3P!3ZAMJ4x|l@z6Z7Pk3qv zSWS>4k9156!*G#tFfvoQ6h$sRPMFXUV{Iuw(A0rLxek z&;IB?NFKqRg7YcXE*IsK62R94Xa^QH1X?x3DE6~AKOE`42l-(FE<^G;plfWg`L!Ql ztb#lLft1m_XCQ27FpW(RYR>uX0Qm7V+b5LnbfDMm4mKoVQi~)+ZrGw(M z&mkAklZp@miZ0Epjo#1DxdPgn+r1W=hB8+DrdRGr9?> zMjIIe=;qN$?6Xmr`=y|X*2o!mmKUGI4h{&!uOrBlif;cc%X!jzqwDb+dgq22B35ol z@wvVxD6zvfWrSQEeYngi)Qn|x`VlfIw$c?#;s{k9CN8 zhowr0#>XVys8s(zDnM+zk(#vlvxw}#9+`91)$lcMFd;NL4E&IF67)F&dlR$!t7fj1 zMbWRJTJvSTELz6Nb)MFLJd`%_Z5NfCJF;`Ru2(HmJJ(>vs-`U1P?M}rJ@)nuA>o~c zZB?>lypsEb>F7MZ?=*J9*;c{N)Kt#)DCyxLC*1{STS>`qA{V9*kIG+21Rv>C9uUto z`FVxG){~H%_S+eBu>&LKt&G0?RKBeEO62S+ErSo>`+Gf?G3<;*@?Q1MUd__L%s zIkx58mCeKYme0s|=$Pkl>kL}2jGqUa>^#k#?5Ydfl)%rf{~Z5QrKT`GGQT49*4o+O zjsFIG9)0I}sGn7{co{u?MAHETraSQ~-UzAhKRhbtCc?NH%tG;LT$xA7v;xgQ!Xu}W z?uAwb(FK~<2ysFk{ge5B-WABC|A&r14F#hos{5vS-ZfC*c%JRUb6~0Yd@D&Cs~jQb zGdXqJ7FJ{Pd>Tz_cXK&$5qQ`MMsK?(H5x^_zTJMa-=gkv@%i%Yl$zLkt8H2)m5Vqx ztm%{%}NEloQc9FNo{*klwEKI(_>@&4#DlHFLv+#BHm zZrh$3QL%hDlsNizlmN`mA6pROpEW>@R}r6$l9$;aM$3(sTTx@awTQyp0uWcSMC~q; z-IOG=s*OV5Ga=5vOESZ9mt+ zkt7Ed4^4YwQD0CllDEn3lnN)#XE$XE#l@ps`_mZjo~(dWv8YHGOna+|Orh5<3r2WH z)l}@l7jmq7Vo0t9mO>5S^88_O{^O{eYx>~R*;wZH>^ls>_h(0z0s(wnybt!oIu^h6!W#bFc*QBTJj{<0lr zKxc^_`x7{gQ~WDNmy5pOcrL=BN}ro851Rh+vU>~QNrH-shJV|7SG=#Bi>*qA(X*9CJuVU@TLM>AC0>f2!)axf115c~or*6<|ENEcwcs z{`!oj;+?Xtrz36?Hj6~TzkQ6!IaxIXQsM>Y1uH0YA7gH7{oRyv)8J~SgDvIVg^sVX zs1^Qn@^fN_A=1b%X{iFMUyZH%vpM^*W9|D7LzLApgKQ}W)mOsJq_Awk`L9kif;O-| z)UxJM=#q@tjfO9(?|%7eym-xvoSEs4RV#(|E%6E|2=6%(F{W$T9~s#V<8m)&1SP_c zcQvD+r6xzEQ!m_>N^dG=rjf3OFN1w@TVv@_@6M(z>ni{!-7B!Ce>vl;7!Bx-lR3E{t+~4_c>Kdvnstj{v@23<*OO7)sgAdgl zi5qIP=wYk09<_s#tw@MEvkp7)cDhvpg}-(g5P#|xWH!YtAf0~|P!iaoj>O;3zHv@J zt*uIx+e^iL^>yN{)TFc)3?ESUrS9S(aC~s;jlXHS3*Bf@MYx@9z??aTR5MM-Z>tcEWK%S!a$s8;O4ZnHS&zl1$9J=gqU zd=bAK_B5Um@Xsn5zpTWd`W@)){wKSNP6XgFNPgyZmj5D`ZVm+WN>wZ_BUvy}Z5{gC z*`DIWKYwxa8vbmZ#Z)$hs~;Kfl|oVgU&D4E)sDbWJ?<<<=q$a|bNf`ax_?}piuI}u zPmy4Z^0iVb&6VSrcOzp6isY6^bABpA`n&;)ZtMWs+->4Qbco(vquy!`+QLvE+-rfE z@3K3c6mFFrAt{`sLEK?$NZqM^LVp32n==2FD%iN)O~GoPA)hX$i(x;;tZ~Fl&9x1i zX>fTa(k6r<(rQU&*Sk(M`Q5j^95s!Up<|5x-5n$2UCONPNF#zOtS#w{z{1fc7Ei8ns;@W77xD(tRoq@6nBWS2^XUqJ!x$zNuk4UOgb)(Kc)*pAxe-_i zR+0$)jA2~OdE>uJ5T@>q-|DAEL6IljCbgZlHLlR~m@<>F~1_Z7Y9 zmrfZ(b%m_k$@1Ctq*w>B&T<5s>1!j0+RmixMJrB|tC1CWqk+m#YcxAj+Mi|zMl{}} zV8%pmnwhv1@jc_YjD%{xQo`iYd3jf&@0PZlzvYnU;Oyp1P)#AHI`c5NcrYx^{%>Pr zL{aKfFdy@PB^5elNUB-p?cHaS^pIHr08$~_=Vj~%bxo0con+Xe;AEVEx(B+DS+7tE zG;FJXet|C)9sC%)k#yxQIT(!(C2SHtx1>M(_H1NcKV+Xo-Y>D(82#Yj<^N^|RG>%U zFqb6VARFbuSJ(Ipfy*JjFa(2HbM7+rZk4KVTGdBU6g&oH8$mxuf|@6(FQg#CvFZsG zy>P{Hof;P0=nD6?#js{%jInwB>1O3AMv<|}6#Ct<^}kJe?a|JEfrkKV06m^L@ADU| z@Yy0^QmJ^M4!j;R|IQpZSMQndG?ex5(BZhEU}=6&&kf|y2Ha}s_W@|5e!ghSB>-RD zX-Jt26(o%kBy0cCj=MV=`a*zc86Gse3EtG7dU^o{Zc)c$Ef&5EGzI#F{|I5&371VV zr0O1Iu5Xu6oMwqkgkhL{@zE@$hzvnmusOv|-0Cm^glwT2yD(Z4mUi2#2Ji{4!%fV( z>N-;M%=5dn$W&tNC=`TliI6wD1B^Ypv{6?yk7aS2E}5TBioo@Zg)fuN(BXQqPJh$+ z4h8UAhIi;H6lg0}p9oB5ZIixI%Xn3iLD(i0r9_Kj(+58%#mc__JcX}&kd@w;Ay$ix zwWlF8+EbT^vi2QODDvNMTa1*Xap#|yiNRPznY$SYg3?W62x24}R-*L#PrR;v9bemy zgV@k}^0i2XU@{Yxno_=Xy(;_&ca2FQR&fvDvnEoxKM_iSR{3yjn|uxh!y&n%n2Bi7Ow?czP-7mJ3QcleI0(yKG; zvu0%e#SVGMd3|El?nPw|p?ecRbx-gI?T^$A0@YC78xF+5WeCnNgPi1JmUldjS8{m_ zrKA^k@JcV=gN(Jw!1_h-(10h)k%o$5}g4oU+k;zoFpAL^B;F! zG))@i!^^AcZ;G#-yVdH@zIVl_QLtiw(5RF2(oQu6mQd(IpGGl(^-?h_G^CMb4yf?H z7Jr8H;AkHv*WNC;>hHoI05YM75yIK*UY(-X;5IVBTFiWIBvH>JKqZq>u^^jLdnm~s zY(m`sZ2Sw+HZEu_?{8i-VOU@%MYeB-`$1tG85t>&F}ubU8`-o{?Y51GT>2Ff58oxs zDS?&WAcDe^)UB7wrWX>vmVsFq+!%X%Nxq%R1M>yauhy;lTw7gw@}64^1wkErwJpr3v-Nt7rf1J0+LFYu{5Z`D4|G`2?X)NDm$LV$ktxE%Fe47^UMnu zH)!8&)WO2ma%A@6K4gT;2WQCDGU?BI>- zLT+U0SEObQkdnaPey&^pn&Z-=n5p4lp0Q6g=EJ!<^-i-dY9XciY2MY&gS+s}hT%e@ z10yosO0n4VDo9oNaumaAzRfYf**qeo)Vgo;eV8wf)vA>i_x?Q-Y{G77PfFu^YphQf zf|(yMBy{JH2V#p-aM8Gmg31(-$8NWzpMzeuU1a>Qj#!MDom$cn=Hs$!R;Y)YJ^(&| zUhkxK{^(vA=96cUft~7Rl613?BjDv@E{LB3Ci>ntV)Vh>ml)75zp65k2 zw1Ktu=gwkt++D!)v?FEN(kgZbM+>k0e_KNQWE91>1>qz)!jebiGMJ;!*p zPof#A+=_l0qz+`KUo2G74LQndq|?c<1+Fjv|G9_9isu4%IU+2v^#jP0u2y?t65Lp= z8oj(fE8+?tcuRO*pnu2vjldCO5SO;hqw;yY(`gAn8L=!U zYM`ofXKE8D`0~EHs%F~$RJba$+qbTAIeWs@MhQQ)b_~P~9<-c@Eo!=69YxuKd;mn#a4HlVIY>u(*&%OhD2jGi8wh;sC4zd?;>!Ixt8yHYz z!VT)HcXYH@)0fKgk56Ayy}HsS(~d6Xh2@l51)SCY!r$Cb_`7?h>Ob`>{~>oxl_9o6 z%amYxA$8u^1xA};mJB~$a(u6m*?BQ(8P;(FXW}L85qp9x2I?xR#*g~flv*2+A4H6+ zN6>}I;~q!;eKl21{B#N+s{$myEeGN62*@l~1jSOz`eg8MZ%T^16bfcAF`gE`>;g66 zCX^0o9s;<^rZs%8S|Zm0NUubNUpXEMfQ^V01eT1FukYRIKRtuLjs_Yh4-0}U{)c1- ze>ft*%nktqCYszG2rxGAeSiwJBJ*Waf(&rW$C+H29B-wrW3AX;FU-1%F(n%H4IJT; zm}aAyVf?3(ml6|$N1eKRA?`%fF_@?4#fLGDP6AMeI6xLWo#mdTC`>v5>MlV5FbNP! zrMZL(H-nUE%en>Q^gCO-x1~_GqS!fs$YI_eFC80CL-u=b$M>tFvfH3&B>san+$L15WB7xBfw}pp3aZa}-McK=`G*jv zZQ?)0Frk7~?sZkYTS>^+$ohkSQHseQh6!7|o5KKELhew^qP! zgz2|>ad{%LPuA3nk@)o2k@4ha)SGekq{)KGJIMs)dKisb`%4R8MCeN@U_nBlC4sDuhi^U;)9=eq)BkQThg}rHY)U)7dz%8AD1Fx9_0`jXw+}2BFCkZ}_ z0)2mIVHwC&SJm^n(yaOTwFaF3u9f4FliA`PWf&7BSeeLfS5W@} zL;s3l%<}^7pIKxxRw)7m%zy>E2cWA0(D*D^5ZvS1y)HdHd_4C32uXS-dA;ZclV9%I zWdcy+rDWmuu3p=dvW0!b6UhCo9NYYsgt!j$Kp{$;!$t}Xz=l);RHnp5byUOJo^xDB z&Fja^E8bHXHiS(h6C1?<*i-iF*xMDOubng?QcA?>$>I#fsPvt zIkaM<&~j2>fefJe0JKF=X;}p4qAAe0ps#-Lw=0M7_%OE;m}?#c#t?{K^_vfesr_R~ z9LUsLqiB{9r$uZJTACPuFo4$#DY}VLfFE> z?8=}@$Nal2^_4J)>Cb~n5*z`AgL$F4pztqp&~r7Qi4M1S?-U5)W&ox71KqSBRVXrM-xiY!H*;LYTz5jeTf=l zY3R4~17MgV0UebpaS`M|DeT+p&*)M~`z}3LFw+dG$Obe5BT<)ffPyVmbZbiYIJEjQ z&1E$SP?n-$QfdU(O+6hV%0>#k9@j@nzgfrdwLa`7TQK>sEA+nBb_d<817G-ln%~{e zwSyiw4Bsm$ziVpDW)hq75-e!+fUyzvwdc6^Ef<^{WQS7#prbE!;;E=0* z{xdaZx_V@X<%v%2d0K|<5ws5uCX1nd+WVm_3oE)XR1@#Vm{m*=73&xq+of#(HAzFp zB&NFO6+*E`PiL6T z77e5&xmu~-j>Io2a28MYI>wQa7ayXK-qW>ZZ1Ve2@{Xn9;vZ7W0D6D zj6YI9dH8763@z7W5JiHmSam}s>Tq-XM{{R{pfN`;m}%VCvDNDsZwR}CGu zlG<5Kjn+g=j!<+5@M}=1@-%Nj1eyWq5{91snz?Z+1`Ie6S~P|(Xp+4y31sYixNgD| zyK%9t4w^!IU|`U=zS^G>S2q1=Ig-k4w*j0nr>Ul!Czi=!>?}0{UD1CI#i`X~tQCVb3e% zCm=%hK=C%KoHO}c<#lZ{q@`r+$(B92^!na~54NF&#kmaU2zNYu(+$ZT=rr4}N^SOa z+=NI#jP%D&O_j+F`&Z1Y^8(jJ<(~3$OAi*_DZI+W>I67HM&2TO+$F0`D#G5?q8siO zox1Kg_}4CbQr(>z!C6YnRZlM)(k4eu^73qesvfaFzOTSP@7Ob7@Fp z=@QJtGh5%+>0;9srO?b6zr8Sd^fauKFw9Q!6QqYiMfA(t*R|1iOQ+s9A5q{wE<+O& zpxN|xg`WZp{TffYq$r%#G>mY{-o5s-P{XS>dSGk10}>f5*H8~sYr%2u&2Z4?i!z>4 z#7x^MpVCQ^ZMM^ws;Qou!DbZtVCfC~_Ex`GCLL*Z(zZLPQubbl@r%J|{WjtJWHmcA zc$=(KO~uVc+4qqmFuG+79+fbT+c9B)EptpXX909{O474yajG`2eM7D?{^X0pzCJGS z+>(X!MjUENLU=nWL@c}pI!84?O(j%0)l!}L7S_7J^lYt@oVuaNyyXHsniizoZs%W5 z=Jf+_=5XcTqn@Uow6%X}iWkD9_7lXTI0LLlvNI2Z-M$Y#H?&dMeT+#p+o`Bp{kQoc z<@4>70KfZncB4S@lFq7a5~`g~sY7#i?p4r>qG9~ko5qgjM8SRrX-eZS9)Q@!QqUFX zO1KNTc!e{l1by{{`oVcKxB*q&4HDfu>K-P(Wo`RnGUcWcutg=-A=uSL28uP9lW&>C zr!<_vP z_gt82eK;T9Gqfvscdrc9=@kBsd3^-8X>tec<)Z7wO`K%4Fw&)+=t%ntm%piT}E zALGI*Ph)$BPBfBm%&_?(IV6IaQX$a@-@e?>@u`>DyAp&(RKxFnoIyAOc62E+qeM$0 zSSBF207A|iXhEv3IEFB;D{y7_dme+7XDDoegJ6r4^So~P7uY~WJLgwf?Y9QNu>VPq zq{RWpRV*-%x2CU*E}}Z^kj=YW@Bu+L_@gFaD)Vq`(K!t+$&b zcsQ@2lDxfw3?axKF{biQlqlb28AwdEBiU{zyECY$2MVbK{+5Q&j-?)wasTLIi;L^PNK|0a z(@*`j@W@E+aLzhfq3J|yml?lg-Urr*X7GNJ1ax* zEr3yg0n_Jr%O91xr}*DI5^AyzZ*mQuYUwrQ(ybrmu|+4D>W|kza}mILnwd)T2B9X< zS110V=`4s0_a5X!&8pUE<|6+dLp5C8$XAUKmZK-tKbD6ashXPAuwt0PK?m!==H(Mk z15vjEeV@*kVszn{ecTS2>ph*c0n#+nbs{EDY58(#$z$?@>+>SsH zhoXEvuN=aQ0XH}fWz8yFYo@ccR4NBVFJNHE)#P5*7c6BGJ=?VWp``JrG#mjsv*BCg z4KqxoZj5LkaeV^{cglAerAUXL%GiI;jOeBaIVQX4R> zt2claANsUOLKJDTK;3+b%)v$^-?h32?GYNQEpw8<>;Hu8P~L(Y!vFXqq^~SZGmFr^ zNK7zMsBHYf=%1x0WxDyyuckKA;1+iqrj@3)gMvyY3>J!NxX@k{wB+g@IV*0ia8DLz za~!f34a~KZ54}H3XJ3gm!}=wB0yBI<(+Um;uFbGi5F6fZN$5r|^mWip`RqOfd~@ z*Zisj7g98)sP5yiXURnX%R^apE=K^-5ef7E_LsXVy;8Qese%1!qG-m3C2d?fx(DYbrX2pF2QcV$zCO&Cfj3<7C3Q8B?axu8Kn&F z$P@Ri|IB``9%^wsb6Gri~#r?DUAV4vvs zG6`8goWl+O2gG7cnc-%yo{bDlQ3%)|X`_?k_9mTbK}B}UpMos{M|wNc?%Y*Hjyk1e z3>Dc6qg&hx0teZH(vrw#@!5V+6gkdUeCc{+Pu2zVUQmR4nmAe>9H@d5E!Y(8aNy`M6`j!R6q3D1rve7~{DASh!x3IDEINm`EMV(iwAGj&)MyXju*1pbCvA>i^zyMWJ<_yFEFQ9dyliwn8olTb-5Wau2rm6*P%sC_l0Gopk z{|2MZTwXf=WA+kZUfnWs+_iXLix3qBM85lN<<{BvHF#pRX+GI&qe74{XoNA;606l= zPv#Zmd7rV!hOH^b5wvnmK1V>tbG9P_ktKHDN$miE6E$<)?eeTs-h!LK3e_kMDC@|Z zcF>JAs8k#>S<~zZGJ<*4NKdQ53yu}=0oP>TY%yf%>}q-22Ju@ZgHvyL08VV@OQ10fr06ODT9-e8eq-F;Px_^GC86{7(VWp1Od z3yM~;k-cDHR_Eo%9~%GG2VEq;H&diAAa@)B`Y3Mm?pK_@p0XT(Z#{drKp|TM4qGU= zxMT64eLF$R{?|Zvmd0l*%HSF>?o7;Cw^Lc`zh}fSC4qF!IUFt@nwmY5>tDBlDRFd~ zrrDbaBN>V!QImG4(MDa*u2!Hw45U=Uoh~D?1P_>FV7yI)ufuN2Og>kj!q}66xUzJs ze&6cc6+2yRB^sV{c!TeueYAyzM45rV3v7ktwgBR?-KBgVIhS9D6HSa;QX5!w+(uXcCJC${v=jK%7tbEnzu9mfZz;hh!Ya z&(WneQw>83U-zBd2zt)Q5D}~|DdTRU{+yE7)?_kAUS{%-!pCAF9g#3X({4m~%&kH9 zH^)mZesH+Z^eGD(b`cIfSZzHdhRMs(@K>*3Wut)2(DYu582U)U`Ds179}1WAU0yD6 z)MXX?+zEVBE2du?sJ2vb8U)gt+GoEVYNoGW0Ys+%CQ`PLn0dCgEK9qXoW-E$JBT~t zx#9&2*>50UOTl9{>+wQON+A<99Y4q)drJ z2t=gq0|<43s)Inc^AEKS%{$dap!^C#e`_7ExpD+475<7vUIBK@2qG@)DOoNnP*WZY z>!%96+EkmCCT9gRQ!omp^N?}7f;e@aZJ{a)rohRm-Q)*i$$a*aqc!^xS{|N;ol104idvt? zZ@60RrqKBx^&?g-h--#~N;}BvLPWJq(P7`*uHT@yANMZRwLAUO^0t@u2<24QO&9}` zb2p#?gsk6zee#d~VsKMwz>DXxIc*12ysgsdJyQ!|@{Zjg34d)Ivb3L|d(x8YADtidkXbeQ*aZNa)8F&D(dSNKiLG9A~K~(z2`+?avJi0C(h{ zchVuCaIH$ba-WC@JUah7%O+Sa9)x)|YSf)y@8Mct)K%Z$(u8Bn;TO7{zCx84fxi0 z6h(m+_G?mK62mKoJPrGXzo6`_F5mHSr}o-#sT10++D;c_EF1d5MjvFf^fUY1Nn7`S zTow5n3;MT5B8L$XOMN?lrhgIgQB8AQPL$}S9~(4dhrl-Z`uBGZF=z&tVfx$Z4I9;E zvktkJZS0xr^CNk!T%xF(r3L_A6BppRFKp-IxG8aHb`jM$;DOOe*R~1>@zVK|&!cL+ z$0lqx_cgLh!IcZxa6sSsS6sFJ`CxOu4j+tL0D6M8Eefh*eU=QZ7R-#q)W3QF_(f>M zHITzWeF@xqYPxQLhr2wgjfyjgVdoyUzmC&RE4QOMd#;IkaGg)%K%}4L%$|YHlJz0rP zti@!ZT}%YUsAiz9=XG2}r~$gkj|JOIACl+AUFugmUdZqLaQpG3!4Pz%HUnRz!=N1@ z!t_U2VC{%Wd%7w!+QUP5V20&pE8~3FP)UfWNLx(h=Yqc~Q2@NeDu{k^2NlF?v(NRt ze|<X#2_-q=0$x1hF=sizmvar2@@;v5knzO~(4fO7%V#v?#A77`f| zO2LhsDK0h(;BZ}evx$so|Fpu8d7Ge-$3k6HgpGka&G<7j7CjH z)W#s4Pnz=qIO-xT3*HimvtF z9Ev;Wm4A0)_EEFOv|RLtpyBgJr1#3R_2nKO#Wpsz^K@ayUc;-Fy0~sO(V&Fv;TgB& zw;}YaJsfG3ikef~_ak$^zs`^1SuihRsilZyA$?^W}0dK?{Wv9XF6*Mw( z$*R6KzQnA~B*79iCxXc$pS+_x=!ZPGB2*fjPLxxHU3baWKAPew><*$2LUT)jzvbk@ESI$*P94gR8vGqY*csq(SAyZc!m zShm6l2_b)GyT!Wdlr&`x1l zH?3?1J@Ov~Ce+0xRYNv_@Bk>vDG;{ndf#nh@kldN2B_SFN>!xxR7IqxvWhMwad@zHu;LB0z>He6MaF0UH9`#j%49JIDE+(GB3uB7z02xP->*^m=zvPz?eE zmR(d3yA2EjSXU#6M9)NTy)H3tEFNq|YV-|_)=FMnjXX5wNE7E=c)Oh5Ojw|EsAeZC zYC$gD4%4|@Lpo(=o4|Zv3kr}Q*i>3S^i6qyUEM7_1trN;4i!@$3wTA!36udIc*KEC z^b365nZwX0ayTq&Am3*n2j5`@kjZ4f01)!TzE5Z*dOL z6_@$*!=Wj83u!DM@}Ldv5q_Y7 zX9&84n{q>V2f3DQUz${*=DX1lgU|+bzOeTL#um zb#N>_0(z=n+~DF^+LRePx}6b%|Kkz7aHt23jGp9t&`dDnLWjpst%4xZMt&sKay~^E zHoFwqb-sdV)B0cOc&47+F!bp(8S0*G>p?_F6m5W6R<~emBZTIjLo+7=hVH;OHx?1K zpr-#HV?mt&w6Gm@B|yHd7jw;g6Y!TXuwq95?!{;_JeV$&4M#k1F{$%yuT+f%BJMg86pe z!UcRjBDodMi&40r8zv?cXJBZs*Y3DLoVt=L4SW3O^L*o@rW-i2{9Wx={DMvLi|S^) zsKDmInA(P+YI7|${XC2 zbJ{7@=A*k3!zBnU7WX~f{L1RW1Oc$ZH9IA5qV~ydA>L^^OzC;P74CV^q$CWV~X5`I+*}r1RFfC zCP)fjP4yv`w;-+cr{YTpQ7Wg^BH%zD^93FYGbJ|+*W!SM>hasH{6ph9SS~5oPw>UV=o;aTCq2;4oq%ii;SWDfhmj zkD${CAdyKYAC75^8yTp`a)aR=}3 z-ARcSs&J)X=4;5UyWAhg>1rn-!%t6KbhPsmZu9(FFdX-3AfzG4T0)6V;1BVI*<8;Y zHK);GmDmEEd6RM=V)eJanRpcc>ZXkxCkzbziRwuvimX`}YE5un3V>~J?wJn>wOE|g zX1&WRvB2Nc`g@8pgMzY=kY?yqm=f5@e}a2A9G`Uq+L#O;WEIjgf#(Z>1#-^V`6F$f zZ}Er~FSF|bvpmKo9l|7?{t!*up-@<8+=t9Aq3eA$!&L_Io$eZJG zO=!Ut=@)VcoNIB40g88hTy2xpCgYdYjO<^-8&k`MsPZT^#YGDdWRGCeSq;VR2VeLA zJEp|$Q=W5fM$wo>dh!|Yup1|ZeSRUav;ogiTb$TWEihc7|JOj~4h4?T_}LY57a{(a~_+_NJU#cq;bBDHr-KL+n?z{|ezotwE#_b2Q z@R2%50%jePSpCu+6jEbYH`twxTF0%eK>G@nP$XF-LEa3DiApoVn`c(Tgk<=2h`4IN zct9M}%aauA4VkKzox!o?EAW_po@R9axwB`VK$u<+?#b%R^`h$|)?zLkZD2?&nNd<~ zt+Kl?P=)>np;;1|$#y4_U4Shu(!rwkHY3dDQJK>AmZ~zAs+eeG{Jfp4Z%Al$%nkVY z-dF=9<74}MpmU(`1UIN=Y30^1iivxZ`B+Dc8oNnymU_2LNfDl|>lXv`k4G#wl<0xi z2?5FrG5u4Y1sIzU(ie{hRI#OYj;ch&_`pP7PUPN$DekC|tDrzSQHY?A1lF`x=m3$X zDUE;*X?g{d!KsI;SC4&1A;54%2-j4M`>*&nbz3v`clVcm;2<7xjrrN=EcE3k9Wwe7 z(vk`t8wkeP!OnZh_LzOQn_Eb-X~x9V05Ha#*UJg<>>n#mNi>f_M7z9feaaF zU3*tFPt!!jo2wC+%jZHi@0z!M0N`~5;gln;NelddqV~jF!)2}GR-G)f>yf}90CvH4 zg9UW~dL&mBF1b2z$`VF+fD8blYAI^@j(Swi`4_{3nTh2%%f&t5^!^6`F472L9t!-2 z+C@sLB@-17eD2TseZwzk+FC#v1sVQ zlrbQSXnv=!FA_~;ONN6H3vj5%L{fK^mY@jEwl^M6kKog;iX{063#W$9ejCw0InjStJK^FD=A5916E_MzZ!V5v%RLi-4=1)QHL!kKW$1+ztc zNyWKCAG7}2b~*>PN|ExUnIrCH7n*(fjt^Io(Y_Ymu+ehB-_3Sss6;%%oQ2ID^*Jp8&mGTLcDV3>AK zf2X#hqv1t&ZkIxrbj+G>)3!Z5mE&0I2MNC462+>eXI+AuDKs<%lo-M>BhY~aq};r7 zWnO8pqo~%E`Q-)qJn5@S*aoQ8e8F1t8TIjw)5Oc6czW6w%qV8*%zFIxcI_dBJJU?;0?AiD3k0fCkOa1ml75nC|i*aQDF^&E;BlxL_LW#XhL^w&+ z=~ap#g(RrBrY^3w-_3Y(;g;{#CqCfg*}r8-{V~{KhnTN7u?dGG1L7O75iHie&G*uE zzZsZ0F;C8M1w4q8(4K)&^Gd3ZynI;m(RBI zIP}mBLkdWDOE*YJN_R^)NOyNjOG*kzw{(e;;z)ytfP$!~dw6#DefJZe^apd#IrsI? z>uft8Cu0szp6M2YIDGtmA!MY}`#!JI!0+w=Zzf8Y*EtieG~2_AO_Y8b>2fx0hqLE;*;D^emhRP=P~`kQZf@c>LW}q6> zk;;<_`KB*1rm3T2n6uM4AtF`W%Hk`q-$H#}cvUJj$=NZk4t>vy zGVi8*%<8Lyh~m{J;dQN>=FzVwA*M}^o{ia15-bj>S@KCFC#`xBlr~=Z+Uadgz1p5d z^^DbQ+zPiz2b*xMhLH);v=jMufnW$N~AGTgF6$`+PHB^`TSc+%*O#mu(#yb+)94rd894oCppjEf%{b`?t1R z_1`qPly-;68oV-PAcw8&rAS%csSf0hGLvXoFaz45fZ88g^P?82o@e-j@KJBVH^rB>6nejj5JlhVo`kTBGb{N6?aBG%AnEL~)zwjYawIy}Gwj0kM#vW#WSn z;wDW;5c3(vZ6ezk#TMhmp>$hp-lmFaaz?v!)h0%ru9O!$s=G0od&Ko9lbKUv--tv0 zudTJJsQTpCSUaBF0yY!mPI;hFyz98AUjBH9c}{u6V-*d-uN0B|v4c0v&dYkOdl_GB zn9*11-ft~5*06=zN|^@@i-@(oObw(6RNGe|YiZ>*VcM%SBEqWE3_N^+$Ul%sCBaHv zp0wJnz|W8OW#*=ZTXXWtEF7z&wm8_?;Q^G8QsGA4i<_&q4C^8!;sLZVyk7g>ImvCK zz#~+i$v@GFjlyhl7%bujG4wUUOlWqxxPgVYnQT4fW)a_yYr8iv2J4!_&L>g^Nr*TR zOV88Cyb^)dr1N)Y$UT%*FV8}g`r$7oU+=Q|5)iRo%!Br9)>{0PaxWu(P*&4fN^?DL zmqX(sxiMvvleGGTM;L=L)4`EmTT#NI`a%+*56ix=eX~T$NU`c$mEx3+l3QFq-AGDS zyLXuMl!t@(M!DInH&=DmCQYiQ)<}47=Q!Cyz&<_g7Yv(DFos?*lB=b9nr^>4q6OWB zl%xjRnJZU>G=xtdLHcRY#%hvtW3**-=%smoVXv0XVc&fLvQErV|1S3rR2nxkwxQd7 zfmP(3M&Uj}FW);W)MnY_yX)OwK_GgE!h#rL8M@eRB?gvjyP+hFB4_u{4_q-8U+ria znK-T&5@K_=vQ*`g?#?TCM-@SMjs2k9(EHq`-MrGxB4i8!o+E>$PXyhAt16HvSr}q; z|N21ZSax%aSr$nML-gtD(3F+FyW|~AQ%Z|gpvvos3`Zh}p{KNCzN4a)opq_?@b>S3 zt{ksPF2-hnPBIgbu|=cjG2lSMlx1oi1}$V;9|Fqq^AL+(0$eXU^htV7v`%Db*j$A!ZKpo&xb zaP5&5?%8&KFBLgGC62Se_aums7O4=n{;KZy$N<`JMkX#np(NEc*J$X@CXudP?B{33 zQ{hQ*F%SJtDO#}|<_gEXhXQ#_EzEO09{QUrs$oOJWW&Gm6zYVv8k~~AikXIZXun|} zMBE;=lc_6)ttT9z`eM@a6uJ`aeAOdd!{m)WtP;D%$y^Z6eroaPN&Hl5O_?rfb`7hu z7&&Q4Oe2WSk|nH2&(VpCDG_9|gNCEEzP%Vsm{R0KtITD9Lm@KA-b-0;exg?dC2%b* zY;{W6AqhCb^QCGQtLuUeD}#tZo2_a^aXaMs{trh#jw^KPt8GV8&ju^iLBB5>97?7g z)(le?{r*>O-`pEaT2AVNtrvM)VfXxPY}t~Dh+9Tg&YVl|^tiU&*iZ@*Mmp&xSSDS> z1q7bS-1p9wl>~eOiTV~RSnEo2#pw^ob5#E<<(J8tCy2RNPo&sA7B^YFZdnwa$d4rx zdpUWwGnrst9;&D1Z1?p?9{$x|Zuh^xTC|2R;Wv(=p}{y}5&`FybX1IO87$hmLuv3b zvhh{kRC$pu-hwQi_oZ!vbOsKaFNFyQ1e^U_;rJ!momZ zeC8=Oi|$)te!0kKmz4<*6gW!^3L6~=zAXVpU8Q34hrrGXtj^0`>3wm;k_dZj9-SHJ z68}Y`mH$6bwo52(`NF8fI+OkZG@gmrOF?l+&(|vA;v_{$N^jX{tww@apg~>fpfbjJ z!EW~)zuajrgO6G#$#F#h4@^N%emH_Um{(Gqx$l)yXKE}CUsM=7ev}J&b-L-}MOH$V z`t#f2H>gJBy+}JQR4CYoBJO*#m&l)BN1n+JuOft4d$oCzu8s?CAkA=~s)}G49%vI@ z=ziEPJj;wn*i?0WR5@W>wJN}J@HptYf+kDnhqcHWC|_|@6qJ4bNY4C!{kCiBP2|l$XkfQBR(S#(|gg%~@u` zFdvg;X7B_C>6Bqj>7>Nk*BA1BHq+Vcd_Vl#pr)kXTp36BJky#n>g?pubnt8H&%=Mo zWRAGh_crQV=5(gR#8+xodEa&u-5%i>5z*LO)-zc~7Gk%)K&-95a>=V>Qo+v6xQi)$ zD$$&rd^3a``YoB%7(tnetAL1CK+BZFP?R|cVIpuE9QpjFQhVj z8-tRxMPa}4$M{ovwe|GZu%k{&_Q(7kNM|!pb|LjBpEqKpHcWPsUv*S5JOO)wGO_hp ze4vn4W5=vL4pc!N;%#Q>dk>!m&Ie%Iv{Sz93zzqeR$4J+lwetntk?TxToa%b(6zfy zBbj1NF~_s@ZFPSD0Zo_V0mw2rdaEz)-i4M_xIW)m632Rlg`f}T!`S%?xE67=dXw{N zv`ZCz$J%JIl_mD@$LfmDJI8*cVrI-c_)~o}z5VQ)$}E?P))pZ-A&(!43*ZecR1U!9 z|9s}*FQ2nl;SOU0?nBC8$!)Ln7}NhTbI>~)@dNXAJDXr>IeOQ3_qWYx-c=`pxwr}q zvK{=S({;*_G~y{sd!@KtM52gQyi+w=DO9}*M;18_FT)!bE_0+-)^$zxxesT?f*h&z z={eD0XT~TIWxO3>_Wg2U&tQcm+F0vwFx4`f-Wl*)?%SsXjoUra2L|Uj-M*g$3K z*-MJ07ez_L3Mom@nNSW|#x9&l6mv$fOe9lA2|vIx2w`~TRNyikR!J#fkGh%lYu)Y- zZnzF zmKkeZBr_b@HQ6P>9F=2*4ULeMsXZJrRijjS3_IVol%8HO$V1sZ8K39}K9O9mCpF)g zh!p~C$Ob@#@1&lVmq(1{*-UZH>sMKqx=IKHY&jB*k020Z?`wPgsPH9%1{vw?dGJoQ-rTz4)il?6n$EDe6rCyF&v4AoJ}Vb zjM|K`kLjPAowc<{wMA<1iFuWJH8vc$P9`tD?T*_Txw?_*gvVhaceH zv{|o{}b98UA*xpQ;*vXN6BNDB{r@EGpT)YV|y8OahrbYo^Z?m*qX-->Us- zv^@qU&>Ve|$1vx2wxpbvnK4KDqp^iER9rU_1oLTTZHB3TQNJu^+xJ)mC7@`2Mqw@1 zr{^1+oo-R@9Y~8)jN>=8o=cbfAj}d-vD=lNw)II)+FaPS_Z1;>*c`|&#v_1=O1E>9 zmR?M&U__uHkt^P4+!eSCE-;N|$^fZdwV_)f^*OvkR_?h-3>>1RIm?5&Vp4o6k;F!~ z93n8UrL)4;9?m>@FmK%F2Hw&_ng%Lekun^J@Ec3LK#rhHX;?%DBia<4B@xSGaoUcQ z6fPO^);ZfyTp5D?)yy=u;7AU7N!Vy>a2bcyufB(%TUwsQwK)M6eRI;@0Tb$YEU zAhWH$zNe0oH6O`&X6pPa#~c>%n330OPes9AcfImE3%m*k0t?LHV2h-V?iZS21v1|-i+XS? zvB?M0xCCveN@Pc*#~_ucsk{ozVx!k()+yy^j0xU}KEQhz1yydlg&dV_Le-q*O2 zZ7-LaD4dN3nJhW}q7u_m1!o6JCNKH+QC#LV8kB%sX#vypz0h?SZkhyyShZ@yP_%Z&8PIB)GVi$2?{-|1tRo@luw zpiv?T-6B2Icc7t3tEc#lguq{mzKu@ZGNd($T~?>UPgQtgqm*gM$^^^4#EcmC;{{+R0tq+f7(+?Vn34de9-x`?ob zUrSCUGbhZwpFwpYRvw#*!#hH!?W6Qy)>?5-c-DvB4jy0YB`jZNMAmrfa z4rpb-QBz2izJCMSD(-nxJ2`2&&@ zDvDY1Op=mLl=mFW!eONwg4Rw^twxv2rb77dr0d@Xd>42Q+yYsjN2s+gd;GfWXnNK? zVl%NwygPCeD5m~Qyt4OYFrl0xK!=7L8>!j;mQ&fT8THV#OO8-RHM1e`@71lmf(M~m z`bdajeNkw=d-;tm^mC3xFUcmC%daqd$2Qo{7AD#=oW8GEJ};nAE=?4QYl$PiuVt%L z8~D=b$+^zYcfp1qg}-)6I_w=k>3_QVS@Gejavt|w4V`EZ+IHaHuZ&tZ*Vhi}+XCNM z7m%fxW3r5q+P38CxtvyWs#VXKGBRi$GRkam-QFOIE-Npr+mQ%*C~x`sd~{S)HnSj2 z+h2fgnTBZ1;TJ+jNhatY=|uxZQPTRqzc*wYjgcx*7~5ve^Pfze66+I6*E4_m|5ZQ25-X|UTEsVb!T|p2t>T4d|xCLo~uQ3@#=Kg#L+#u$LaAQX-O~F=ZV2(@?0&B zxFcEESd}Q^<`R{s_5x!&&}uhEwci0wljh1u08^JBbUutpDe0WlZWCK=OSZ*^4#=R9 zKHf8p@xLA)Ct@;FK+`wP*P~*tD#|?Dw@Ur$+*0R24q?88&`m4(QOLFX94{9J#Y(&{ z=GmU(^KE({UT0^6H6{{QM>sP(Nyn9RTywf`V$EEfC-`1XiApUsD);rbgvNR5dta%d zRTajI{L^i}9~l$@g`2`$I@(Af>qDESqtDjDlDvHTcw4w_EDW;8+x7G#_lma)cNea} z|6^`FqR|9+_exV%1ghGx>EdSu=BZe%H`;V2*>CA}W{3PI+GI2eM6Gu-!*4Kq|3z~3 zu2(4FbOgPzPc^%JKK|ryFAHmSm4_oCTY9_9w}wK0wzmv*eFDY&)F~OA9mABJsiEsk zO39YWTu&lka^xS#u?|8MXJJ4m$Q2*6M>mb3F=VN$QT@Q*1BGl z(9EN%Kh3t&v8S}^y25)fiO3;+8txOT8ysRKqJ4zr9?N8RZn`DUnZdKmJ=s%PL0!%# zO8tlh$MRCHVXQ~2U%z@E&f_fv*i}V_e+1m!`d35kQ~yM8<7qDlIj<~|db}U9L+=;6 z#;v%~@WD~|_p#p)%OEe6* zsC_P=DyGqGAKe12w#=$1FYBUzDkQ%fitF9?=?UKSm62V^~-S1)THw$mENfM2y z(GRsQZ@Kvgykm>S%Ud36+!=4+t#a9Nar-4C62SccD8Sojx1~$*Js2N56MQTV%_Q#X z^J?0Y*KKw)xdF49g@Wt@+tsr;G^_EJe(j9o1y{5# z6!!z!b|Q_QzNu&L9$m!FcoNPP%}oO*Ixa3i$M+6&r)AodhIH>HtPyu@O8i2pe2|Hr z>M74+@z6Ez@i`ujx<{MqZ-lGctC$2fIV_Yud^Rx3p{oIaSuqnDo)mmGWv|O?9ei@LSKIfC7bP(7yHBya<}S~O8tjqO z)ZA*}?K&Ti;GvMt=#qkuyfd`;Fc2Dipwc+4QNSOg7*wsAZ?4DFbHr$#w8-2cSP9npAcIWEmG+O#Woz{4(A;K9SCO~#Gi=h$My(@( zNsiUP#6BMw7{`+E`(lzQ4J3o*ww}{6vW~+baqx3t{c^b&pae>AKGi#|0p-ATY?nWa-r5f%8o+0=P`?(pvsK7WJ8v1kW21q_aO#EeHax`pc_V)J6WN#byXYwST zW+&dLJt~!44-3mUVoZZ>wb}(s zN?k4q8B8|{O<@zHJ|YHploN=rm zK!#0yo*%GdWNeE~q|6wX8wX(Ko5wrz*4AiR&f0M*&Dm3Kf~+MJO8xcy*X^#ozDGfNcnd?FRxuy z4sAUv$KvJI93W88v+NW8e!hWVkW{MV^At0Mf)| zyp(n9-S-@yM)Vw5nCs44BpX!8IUXN@Itwe+OJLr3)a<7Eg?9PdQnIo`7{5SWzsjx_ z0t^8|M^BUtUyr?NGSm|H?V7y{Z0I)_e5#B>83BP^d&9Tu59gC23I8+T1DJlPt*1)z z9NgbY+OOCxq%r9tn3PjH+&2AyOdmmChpUZHk%9t;A6<#KL-H3cl4ghCkUiH$1RGGA zKEX9syh|km<0&u$#}pEyjHaT?hIuA_qKqRPCeR^Y`qVU7me`_c-n_m9FVk88xyBPo zZR)vWFd}%FT7s^euXlXEZc7Cs&$ou+&cL!2P;qiEb2X`h6E3P?34W*NVZYcn?Pzkr zJ4XgZXnFB>t<@%-BS_JdlBG2XW*Wo0Xyl=z`eL8GgR}a^O@pi0*zrVkelW~7rW0l%Dy*(JME9Uad2Vhhycx(nWF&PQ0lii~ z-|$f@P;jFKa+=PDy9+&&D~wvLIlG7x(^D*NzxM%?dHjnvOInTk;5Yvc-tI@_B4)H) zv?B~BX_;^==vG`~WVCGyzlQA4AUqxw`>eUdh>if7XXZzlx{x81fkKt+Ri;TZmI-`i z@aX;gz9~t66&hCv!ZP{30fmXDBjYqsDQ`~G?(?bp&!aC{H7b%)UbNOVeSAs_ASHum z=f7^L6nm9bcA2u}@f;5F+N}Yf;A`mpI*B!yfg|8{a;mPmiuPwFYhZNz8FJCN`z(40 z4*)MjGXh?kysGb=Q*()wK|{#&YA91FaLc5`mw5F0e^nA>M?=?`L9BiG0X$<;fMo}A zsY()L*=d7NW)5@MClz>4hzbrsioXZcnH1l1OHoXmF1P$!Ch&*?{xgpS9Ah+3s32`TXI-5z+sCJ>$Y|oS0mxF8nf=x;xn6H?$Bx z+kfEBO@=lfmPq*5e?n!XQ$5ritaPw>C`jMXbhN_$!mB9Li(?J`7?6E|6P2SH-d!JqnxJ`G6fjp3&Yq*w~8}a{k0r=!u!v;jp$K!t} zOAdYt&;4@sMeGn z`U*$>c?dx8-#@7;0&IjPs?ggPIAmiWah6zxHG( zufcV_?UW^DH+1?FVv&JH&~Xui+_Y@w8)Ga~2tweK$_z~#0^DBU z>5v4S`vV4EgrnDj!k4JpsAuJ)8Qu8Ac%A?Ac7!a;*2#LvAL2XnngADAY5uxQt(&W3# z{h3g4!W7>=-EBJXH|8hc(Mjfy0WhVwjT_l4qNQiNNRgs%?HHa05Zszo7wAsz#*-(9thUg zberKxN-PUfy{}MA0?%zXzTW7spb4W{@I8oo0=HQY5;wj>JSbLyN$?l@eaHL5auxxf zFtSB_Bs4HKU%4pPfXCKCX!&m%7d+xy0X7ihA$ap_Uh{quUU349T<;RuX(_4=cg0=!I_5pbxg9O~cS zum8X-AVGD%{McR;r$kG4(8?t@G~je7Dr)u{2>rS=5ql#eQ`-dhDY!1_$F1qlV9o8mrW4?0Ri0e*pTGD@iZ!-#2H#I8rz%5 zaECkUqdiXvSDg_WKco~qSF2qfxs9~w@p1AI*uh`Z`)RjNwx9~ zG}f(yD!>8(2j_dwk8<{sktjo0z%Z)!ee*{PY&vs^($H(;?f5bBQI}N~ifh-kmI!)w zGPNi^8p;@C`eT?;xz37!ICb}A-9Ubm)AOjRYr6W048h@c(ktLmh$YH9akWXZeL|gF|Yc19C(<%dZ zZaC2-mdkG>F8;dah;s=j7!co%k)q*;>?EX!ybuwcXlX(Jy+jE<@^jJPqe>z|6l@j}EwY>fWDii6# z{+^489V#^BwB1I&!MSX{e)>wCmSD-v8(7K#NGQ84ef;wch~*YK3F8o`eHXJ^U0GWK zM(`jI@4fI!33|b2u?)DPGH}X`&!Tx5(XKR#_Z^v21%}xEpS`!tuJYWBs=c6 z&&H83UGn4nc6#nDK!zXh>-U_mYxj^cB10iIcYm!Kh@Gx5(0Vw19SIV69P6|!Ytm^X z(L5K!OD`$0gxTK7r=sCoZ$0j7kE6LqIexd zBJfJJj}(@1tIZY`^^e&Vpm#+A3&X+EY(EEUzqV3dJv-|D*F5oieS5*rzSH&fzh5`r zozMLpHff*esj~G8Z%%CF1J}?gUL)Tm)-|Y z(?7#R{4D#_$^k#RsnXRVhyEdCJ{FXW?FAuus8Nw z;=Ig8bx7VxOao4(v$wxpSf1FH5r zg|nCIJq8N~QZqe&InoQbycIeL)wL{^k;+lY&!J(P=PfZ9_ErRRxd?^7J-Qdh>PF;} z&6GlC*RXcdYcXkxY6I;%<7cg*!GvAh5C~+>e~vSF$4eTMJH8$WG*Ap2vOSVwP9opi zkjqggj^kUVjJ%6b)|zi1A&KfUXj3;34=nSIZyp9fTV5M~dmxeY!9@E08~$~a`tJm$ zaL(W0V*)mr`~W?6mk1fQcKbOKV23La_tkx0=R+b1TQm9v5IDwDQ+`HSWF1^45@oDv zWRfj{!how+_sE|W`~g6G<9AABd&HoC-$2;m=afW|%YotJS1l9OLL1pmdZAud07}HP zJ=LjD4Elnl^tIuUvD9udCB`F?=n5&3->-)$vOP(J+TJ;OS7=@lzw0XP0c|U9O>XJd zHZdI>D2#5OC6RF@62%ZkqrJc?$1)p|W93<;0l3^e{N4lkJKwU*2<}8;el`IddY!s( zs{txT&Q=i|MIi$C^?q z%~YqdI{KQ3GGs>2X=)58he1;oeeKW_kP!#D@@8T8t$ebrs<&uJUWfT&lUL5IYE6MxswiG_(qIOsxa!zrTI@Q|hbfAsb=IzV@DhTQ0y&JX_DM^JZp% z!=b=f(Y}Srpom65?(?St?=Oh41c0`O@5b4-<=5ZebQ8lo?Xj}+XFR*oR}$~&xvP_R z34BuavlIa-79g?+0*?WpZeZ=LX5{(e9`if2!xZ9f%tvmA`yaRh&r!0xVqv%4qxJ=1 z5GFMo{Fyg88W8B4lx$wL{SMyXYuD($O3K$jQ!^m9%t{GuB|rfx1W6Rnr}rXZcFH`W zDpl^sJ~Qa+ue@4^|ZpRpkD|@D34BetwgE zzxCO_uzbg5M;T&@M9RW%evyWm-I2K1B`GL8u0W=q?_25szXbnI+ss;3tToxrvLV>T zE1i!$X3U*4CsWM$@&fZ`Czue-hiY2mJ~6pUoNd?SHS*$`PO!n(m=L*N2(H;C*Ad{# zPTn=@h}b0#1@T@abkdp2ZjC4E1~Y7N>?p}#+)8@v36D%c@idg&`W_Q+YGwPD&wSaF)0B01@mqoRyfU zY@veBXoJUyzA_C}hAUG(sN5FO=OlskC7=zuIKQU)b`wE7hxgQ+wN(uqK^R0<2KwcnQsMt3s zmfYX&nG-LCm^*!pK@YdvvbhM0DLn|^iT=3vB{d3$5r3gVrEF3ifnoOhL+RHxGbh(& zMk&DkgRv^nC?2puO0894e|`a(s+@-8{^o*JuvKClKCK!gD7oz*k7{Nezx3yf}N ze>pkdz3bcn)lg+#$VN{m9>bT#*6QUe;PNxS-Fn*lMffqT$CcYkEuju00BLW~Tvlpj z6@wzs?HH_pQn1*la$kP&>(*=HRMHPb0Y+@TYC9EcU=qe24H}(!i?inn26Uzr<`+QZ z6;vFwdMNUz}eBA`@yrjgl?_#d$d?QYD_BLW6|(I zz}#PhK zYAp2!s3=d|c)K|1e=7-~%QMS+we&G(zpvg_}X6R7hA#Q=XE0A^G2~ z?I0Wn-I+^1JGt}W0zaT#6#j70XsYhwedGp1qCcjUqJJ~;rW#0Ppc|aETpVT)P zsfUXx{)CN?D1QKc#^INj8G68A`G&*MeZNx!LQ|m6>Yx)Geo!M65&?*8_vl&FKpni` zFW7BQXbLrS)MzqjqI%p-^gdkTAW^S0aVfxoDtbbHg9Opo^0UzMedQvSy3BWDHXBFH zZc76-&es8pNi?jl9^gs;DvZFJl*e8>ImPKgh2ueP)Y8M z00b789Iz=j*YYD-D*m;_EhjuQ|-tG@mBk@sS zv27mlF2k`8-xyhX3s{02r}Le%hV;-!<&oKJ_K*biGuu5`64k^N%&!G3kLWxqMO}8W zV)`%*miU3O{utwg?2#L%VZq*Req$sdpEymNXHMiqwF|!BgY$XT_w=Rn-ErPIIgXeL z!hHh1R@H7WrLtYH3f?4|Gu#dwJJ#Qt^h^*w(w2V|Vj5<2dFOWpAAZz36zhm=eZhLm z3XjG4uOYTKYPr$EuNOPe9l_4^!Arq67}Osx5zIy9pba>Hin!Y zx$-jMP)ckwTCwQSWytYkoqTFpH7@~g7TjW5l#1nm)f_P z9-bVa%ET@2pqS=2&X1YTkCoEo^bdKq5F>Bb!Y?uZUiPi;l%$9YR|)$rVMVmW`qN1U zwA6WC5#q2X=RVMEGHQh?I(do4c#Aala8$+=RYI(t#i*!itZUM4Oh-AH+9Dvee#mcl z)i0+Pd9a2Y9)(XshGu6qQmD5&rj&Jb*SemS0r*8EBBsjoFnAzD?~lBxk9$FO78dY8URv&0 zF1Dfvtq!|A;jbznAB_l5YUMZ)+c7Mx>94yP^l(ooSA8R|E`p4!)kv4e0!Ja(I|R?| z_Cz@w^oagBGTwY5z(7yUhPmryyME{qlAfPbTO0v`RoT+gRye==Vi{z=yr|sWb16oY z74l?m8N*dVKUuKnOb1wf&>Zj{rMwrYm=KJV6jTCvCE!nf>Z!j?g!n*$6|H%Fx{h8{ zN9nz5{9DkLucQsn*z+TaJ-h=wF11C-zcG^b+W@#+fp2`LY%X2X&C9p!Qs+(Vg`xBP z8Yo|A^qF!nmDa-=6KRgw4>6{=71+{k#u}P z(w^JyT88$X(P23MeXa9(>{yaTv)tAb<(@5;5WB-$>f)9PQr0Z`nt-e6Xo;E;LC)2*AUspSLD8 z@7I>=kdYYy@SD~w6>LR7Df$AQ;6WXByw{q;hK{~_&oS#*216>J>i*H-q`xIPev2cw zfG$+0!WORq$4tw&uhrvbJcTNSkSZB7-W^XLA=zlmsq=}(cR))4U+;P3lY9DGxrCwa z?;F<}joZ_gh9q#M3!lO)Ahy$>YeIIp@gs^0UsTOOzN?IAdcA;b$9v#?(wQ8GKF595@v8oZ%AmD43-8fWvL65ROI~#Q0_}B7gLR^U_#OiaH{*(!LHmo*%g8`v=oM0T+>mnHCnN}hhcsUe-?h7$BWxyjWekb;l z+2dF-mg1+4p~*KxrvK|RTw2)nf|Bk8<0!gt#4=-uAqE0={f?J0oa}E5m_A^Ix_akW z2`^XaH$<+L>Tnb^=rRFH5Uo7)N9t~w%)gSMlSL6BdGfrij9+X zL*uyuLL`Zn2HVt|MaCfl#AhSdJ~$Wj&gAV2gU9u{Z3>fVI~6ojBsBtIhk5Q)OOf59 zU)e&imeuBKN7##r>Sr-NeqIRdNFk@N662yD-E4^VHXx~H{IPBPC;O%iS&5?pmEJ9)kDq^GAEo(ou>KZaY;BP4cRB3Vvnzgz@?E{Wjcj{*RsTc_Co zm79xN(B42ohl?qA;xD>iyM;~Q$7=l*1L#`Sgx3{(M1A7ts_DalOe+Q$^lP$0v!>GS zat$(_AC^5wgiHN-bVT0fU`0zw*qniw>Bqs2(a;4hGA2k%Em6dvb<~d_hX- z#DAknIBFIDyF2sOS>8{!Vk)T7R2rB#$590Z0fkgi9IKoCl*C)n?}7>~85RcctVcLC zA#Rx*#6QX9-a9hHKH5)Y}<3tX079NSnsA@KzD0a%CRUvI|ydfCmW_5aJa(6BZWx~=_`FTiD|w7 z_p7_ObQupr?lnb9g^(+Y0DsNnV@sp>6xYFo#c{1&7l2N-H1ioJ49%PaVUS!z`l|Y# z{jQKZ?AVYNwo5vOxm?NpFI@Wf-6mvk0AgrEQ+Ql^biRA`-6 z<#%Lw9-$03+M!-Bj*bcthm=Qcj*E;O0$71P?cIQNqr>yyGci+zpAGNxted#^Wzl-F zCR4Pw@Ro7NC51+)>RK7Nh+1(G*uJ2)0PA`n>`%~2|4;KO%m+N*bsG%049XYdBD#M> zN?FVIAW&B~pbSHvkj)HJWiV1k8~Bb6@)T3f045TS4bU&7)?khl_cNTRjx1}@#|hb&It&k<)^(qMxoa-ff63GC0!At+j4!YuI=0)Ng&++#n-bf35jy zMY(gYtsr@-o%@k=L1Uy%xpTT+j~&^(uSE9J{JkL4UyW^v1%uTeqqtcUV1xl+X~O!) zZ>!0C5AuA@u=h@r@neL^GjokD1!q$3Ip8|*AbUzw-_XHL6Fc@XZ~4Me({OQ0V9Y1siYXp0c;e#z_Vt;T z)PRp_&eJ}J754Y6uN{)fOWr^;V}GqzJV}qf!>V_FFDYp+VH8|O%Hh~Den8-fm*5pH zV^tof@s9YF#*An{#gA#K;33aP8|Uf5S|*#z9=LE`AqPriEVhAS!Dg#(Ms2de{itI9 zA0Pa|o{+aUJ27Lc5|gX^5v}D^3-0+as`UE1`!59zo0^y0o>b24@at(83&r!m{WQ0=xTxp<@6j{79ea)&_SSyORxELLSW^Uuc zF;^paJ?T=-zG2m05KkUWZV+jfZah26HQsB9zO)Ns=0b|!Z4BI|n8)}sg7sO66M(8w zFQ_2E_b}hE*7pMFC&Xt(O&0)&RU~8UYtEmpw==YBZ!$q!>Tf{dAacTb)NnbQqwb$> zez|OpoZZ1>hJ5dOCYD0b1G=BT8tX@teRI=JIO$avNmAZYVd={_)N6?;lKQ4sPSoYVSmILr*@?AF1s_mU3`u`*;Yy>MNvo#XF5MC<^e;^gT6bSgSd3b=iQAG*s-&D`bZxmU)htK&>-96EV|F2AbPFU_=#NN zAwWoAjd2#AqLl#6F4y!QG50OeQihBlbX}Uiv*J#Yu55w{k|#Hn^dCPXO)jfXH-w`% z_Ba>o57k4fAf_Lv>zSXC{{5A?#QEy6%P;E;?N^VZZuJ3I>6qzkeAS@lm8-EljVtQ! z%U%NGN$86V5o!yGRWP^CEm>iLi?2C0vAg}eslcT!%lP=%;#uiqS5cd{?L7|Kv)U4w zfvHkJ@x;BW>-lA+s1Xo&AsE3)j{%_u18yfoSqp<6!pZ{NrS1MT$O_s>YEBmy2iwkn z0*6TM6g$UPFrCkDUpxYytXN#iOiKw}vpcP&kEUi;WXr|qt_rnp^BnW#!4GqRN2jX7 z(CGX7F>f+!RQ*<$fkuM~%ia}GAiZbaqk)m5M~xG9PGu#79N-&)j?R7BoxKe_S}d=% ze~Z5&AUUwpZkej`?CPCiH`gc{Xnx5Rgv%3h8?}VP7c&0H*~;%PlI$%lSF3+sZ6^4k zptweVBV8fGlr6i0t?n=Z;<(R%BXRPDj2OfDyE?u5Mo*``2^p>e5-Z-})Qx#X@)LYj zjEqE{r}T(CZ1#9R7GGl*qSk^q7N}L4-i69(kOv%s_HF_`mq-;6z`N#8{Zd4Fks!8e zG#DR0;Kw)6BJ`Wal&;TAe7(z)tF24=_L(Xg>l#NXl>y;BZ;t<>>MX;e3frzt3^4T2 z-H3E|cXxwycPk;n(A`KkNC?s`-2ze~B}%6#B?^f8-SfP0yx(8-5N7t?_r7ASbBzSa z6K7(ULNT#Bt6cLlsfmYb3Q?F@ZKkW3K9YU*%@Jmk#p7BK#dKtND)wq_iMWPUHqt!X z1brcil9-NdXqXMl2A|8mkA2>HNpMQ;cU1~yEPLr-DDvHm5AnzPYfnPr*gGN14q%w{ zU%gD7z&Qo#8=HLOFEA6wAZJp1({%b&7odDI6>9Ow7Y3_bwRjKtCj!)}Zbr%Dm|l7- z+G|GdoYnzEYJugK+kSt|bemAlQ?hEGH3i#+wzJgPsZSH8g!mTHk*^2*4GRX5u;)%Q z>ojE5JXowPvFDa}(59H*-!N*aow7HDFZCkT)J~Bc@qDZ8wcxxz;Bhu8Fc8;@tTs$# zSEi7zcrk39zTfo6HW=-0$@b-YfoPJlDUD>^%rMA$fZN^e2+l}P8|5X??C^yu4)xTT z9;4tEF!D?OBGf^rC5D7L6=%9nJGuP0G~Hwr|1(!CaVOMrVyKPi&6Md+D|K_hSqbxi zwWFR`pYs*aSFltS2W zA1fA%`Hj!vUM;f8%$ec0sbO5*U{VEnSJ`d%3te=5KOfmxEo0ZJ)Aq&~?gql3*eiPs zhA9=N)!D8(l$SUFe-j$aZ_fU=MCXAgLdY>FeYd|m z^v=S&AS$*RA^R7dH`Bx#3@^6=~W@UFvziL?0LUrFsJUfG{9eL?> z;kuIt6}BW!jAB^_dGY3Kt?dtwkDHj+9=YUc3;?9k_IXd_NpvVNL_Sh>I1;T5hi#Ny zM6rzkwk(`CL+Nj!DM01Ky#$8)WS_FDA;@$L@Nl9HH&KeaDW_E}B|PYAAt2B6d}$Rt z_Jyq75SoXFr&&|-@$f=hLML4jQC@>`G{mR(=T z9fi$scghqNztMs!Y@M5+N&fK_{qO9y#~wl@bi~!UQ|#;*zBonIWZV|2Dw-v>{V&5= zS4~KUoNk#8O6vR)Nx85xQt(i-iF&PmpfJ5-TYL1-aH(P&l7;TyHW9bJLR_2$;K1~`gRR3wjrB1(ANwo{Nxb{0Vm-bT_OaRzs~SlM z-3yga6uJk;---9;O#S)V3p}s`*aa?~gGXGrP+69@SU`kTZ4%<4>=2B`;Qy_Z0Ztv9 z;2E-5Qz12wO(D5%sJ22MY*#C|OLf}0qi;ttPYN;eDRpLxSN1V`V_kRb_S~#ovgizV?Nl_y zBS`YF7^M*e>Nw^5tsrqj*2$^5n#MI+@cAi&xbOPJE4`DTW?Q;kXHzb`A5XwbOaq0M z$Q3U(S?9-C0ue(Wr)T=?7(uOJoj&zo2&n6L__4dXdBC@S7K`*KB3QJ=kLG1@gZP?6 zv-={xm(0r&5pP-nK4M9ilqV?Mh1HDbQnm~cF&)$~ZCQgECc3_VgrAy`wfBt)8~tD_ zoXibtxO`hyBGa40H+)a@gs=y9IpSVGnsEXp6QNZ8pBHQl3bF-u@hpkyz4FPsc&IIF zdJI2ExptysMX}Wk|LnOE^~SRJ+^xapJ2)2z2;UI~k)g#@R z#cFBlTwbakg*Wfk(vJBH!@|OB1w(4SHdTGSY-Eu>!!{Nfck*M-P(ec`;j%3(cg%zY zIzje%*urbCoT|JQY_-Q!$TRiTr1H9-wrxI;*F!h!r>wXCCdcz`@-_Lf0_Kicfl6We zwy1SmfE>w(FA|}z%t~bm2zJog;Y{aJ`Q7Fflw2PUv6Sm8uC;Pak3KPj+j?24$*7*R zfc;`)lXj{to7-}umF+vOqG6C}i!1tN$cnJfhfYqmd{EoM5K5YiJXP&(*w9xq|r zXH|#j&pk~a9IV`}`7+ubZW$ia1RbR4-)WFcnIU2MeVM!_jLSIVkoVDb4xresvGwfu9uKhyh98MvM!BU`yIk{F>>S~aV90O3wDJR?NejC@27 zpR~!DU~|DGj}WpE84{H>&VV{lf_w5BZoIRAd6hCSQT;5Ljv;b=6c1ao(6Tomg4<#K z?Z%x--!SMNn!^XdCak-a(!EYllHjLdtuC?^dEkjzn-jykjW1Pj}=Hn>rrUeeTS6$+;~33#?%MX>UIYlghqA7Qfq zoV^^useLoKnfB3M>|e_J{myO_|6Dc%9j8@d;0Gl8*?$Dat_bNweTGaqC`WSAcpTBw zUJP8Re!qC}c)!kEiI-b%L0W?O0>>seIgX@)yLO^YP-)!dv)TZ7+2+A+IlSuFI5t%J z)5(D=5*tNY2~N*^wip%F*+uscdN?*=Y3?!b8#s*f&o)dnFF3Uyon13t$VYw)>bbA5 zOfu)W<6s`o`dHHU=GfDb=bl3}nHpIc>G3au^R&D{e`)S{iuVFo2W(?DzK`YHQ04lO z@FfSzzpb~$b+K4I-L2>D2#r8mMjvEJ#JY?bwyUDwc8A>dB7(@&z01M$9!Q@)KKYwZ z+Iko_($s`OUVr)DW1T?rL4&=eLB_x9)IR=2h~nCF+O#B6dO~fXb`kmS@hknj1EVQx z(-9X#nnv27LdviD*!twpbUrG@eq5__tZtEY5cjiC-}^YhPFW{hM>$9I`|Vkphb|Z& zlv&M%eARvu$XICbEb=E)QvqLGd#_$kwkS@9u?YFx8Hjc(zjWIBIa;TBl59Zu(UqvM zPQVq5f$XVp>L*QG{cnmL$!8S5@gYAu^};M0XdQ#`$(u}aVvLaONnRv8wd8&z!JtV^ zhKm<(2}e!5zM)*@E$koQ5S5zdmoBYA}>o|UF~hKNjfcWvALi&zD-{y zDAM{6Y@@+K)sbAG%^B7jRN`@(pAY*&-SFwx`Id}m**|~go5orZ;36qD3HaXin(p7< zV0~#cQslnRH7kX$gPhS5;xd}+bI7Q;G#=VO6Uofol5-28C)%PA4#GO8hHJ1XTwtSg z4=}!lS8|~5eZ_i{FjL4+n~Cy1rCD!&j>m$!s?Cj>29k_OHMl)f zmJbt^>V*hxq3(RnQ!W9|jU&VT?rl1)pXLnot)D| zwTRg`FhCJd%?uVU^ctXlnj>UngM~t|J*La|}P~>CW73%&M@S{~DRAQ1$irh^egFKnkpIA*7tCLZx(;oUm6REv5KLgliV6^N zeAT%g z^Nz5&{?tWOamk+N8(`{xtqm~6?r>R*Bu{oRPW1K|@;V-Pi5XN2l0I(L?(8+ga|6PwQ#Z^ z-UYxTrAYbk{q4D7VO5G=<;&H!x1ZJA|L@O@CI#Pv!t)cy?QkY|FA;K5#@%lT4t1>e zxI{Oaa?GNDpF#5~c4l4l5VX%x@Hp-+j`x@-CkjY9U%+&|&Q#~5ZA3&0hzC9cKkao& zk6kSn3Rw4%Ub=w@PV`Uue)^yc+Ef`*m8!gPD1uaMnFIYqbi(IzBlG`ntRD)-`qTsN z+)SPFL*<(S1yqn%>x2L3F5|)OLYiJO#%T2W5dk^IdI}iJ(MSd@{8@RWBuC!?Z7}Zu znZjcW8lGL!-`}__R>a%Q7m0nCw1<7rVH`sLmzr5D0;E$T%P% zu`C>3U-oeMAf-N11Em&!HdT|8wC<|7tt;H)xda^y-~v-{S6P6&5rlJFhQN?_X950~ zOdE7GjnoXCT92p%U67)jrZvB)jDbh~`^uY7;6BUxj{}LBDguab^CfzKE9KpHR-13D z4QAJ1p$KLx*lXP*!bM_QxYx%`4~2P!X+D^t*mwF^G6=R1biE3)%T+X^Pba z(l5j>*E2t70LTL~W&{Y82>8A!rj~;7)=ZusOuVju6b$z#FRG-t;9Fqj8-48zpJWBf zG0D^+AXu#|tvF*r7!BKp6<48!tNwr^!TZ4Wq?%eZM0SrJ^Esp>^p_gea(`dr3qG;0 zun6(NKTnB%?hVS)>6C~De|n(Z|D^oxxWzE@4FEYFS?Y#(kfO9Zu&r%@un{*O2dWG(zl2~D2tuQLU7~*tUG{|DT}{l|)TnIn_x^Ri zG5Kcobp1M)WZIzC=gY`2tv6#O>La5UvLBIcUaA`Cn=qw|JoyqM!_`n)|96U-NyUlE z?3<#{3wH7ZqjeLw!?00RZh&QWE7Q9L5dZL6>$!H#l@sUYp&Z?E`Y|)!wM81Hz{l zV3htGtlQS3O53%oY~_ZuAn6;xvlds#72C>HbrkHzW!5_OLlo-~Ie2A(;7%|WI-&9( z*OjC121hJn(B>8jGMq3C*!D~pNAH04OkPRqNJ`**WC9xi5{myW25gqzIYShelp#LP z8ZF9Nfze*mLM>O^iha*#qO1*_OWO1DHTA!}UzV=x*eSB&tAF~r|IQ5`7LzP5ya%Z$ z=wI}J=mdM%vJW1PGax0D6t%V$@Q}-A@RK%Yv@JY@JqB!t{D1H_5#d#G2UP2b%^O5&%lMiEE z_W@Ht=S8!ervmUXUxa)IV(`Mk|CAF=3~=bi!`*eEpzE&C=z}}HyLJ=?{?DdAJ1s>v zogm>v5AdKz1RN;p=(iog2gzVGB9+gu0FfLUm<5_E*U!Fho;d#QcLlnlPunk`k&(d< zlEEKTTi7L`Fth0X*NK66^R)`1`m~2l1NZExfUddLX!%w?9*$!MyATET`R5NPz+tFB zZ+wbi0ng+_SW!AoXD4)^vGD+8!w_(7GZpnax5eV z5`+I1ry|7Hh>X8tcfFT@poJzY|0;i*>2&q?`Jnf-=7CDJv1T(O931wYQ0vj$l1V6Z zJ1ElY5KIFpte@7A;36+;Eb!(Fxf{2mWR^aIC?}Igh+_>ieDF#jN9=(qBM^?!uT6u- zbiT{rz-f|JihNh1JFhJ3vE*@Lb57Iys3D5*29kCKlx6GLxdxH4(=b`jN`QKrVxB7$ zX$rnRWcz9&uKoxGT=G-kmQE`JqNw_(UsvkR%)y}tL4J&oOF$m65s^Z+y%4GEah2nR z_Mdy7R*$?=e`&Y)vD@~8A-xbCpdq|}&GUV}?nFC;0E|`;!6-?~u|#FN912yyQH>8y z2)=<+eU3UPSeCPxG+F?ao`xIGQa9+f#uAadFwFfcZH6ORPAMqr_G_Ma)3UBfhP`r&71CFJF87-vnSeV{;9=rH&sR=qL=`w zCn4H|%}WhD1U7EFMIlbUB$G!|zPq=Y&BId<#7AYXX0T-H2f$IR=DWI7OmCAg1UTAY z_u~|-&TlD)u$)Z2Cqn-Ca;1u(3nm+0TFeS|R;5pH)g=&x8!3wD5n+UEN9=0bPq?v| z2P*-XRk;-v2X`ay!I4oNOyZ=B{e-!o&8&VP%;FEg5CC|@f83wbux;ejWc}&htW?B< z$O3-PR7`F*Y9C1Y^Jm&p*Ya0D2ERMyigEy6+1mEaa+Q1_vFANN=?`(AgFxR;c|5Li z^fypWv)1V18DMAEd9hl4+5*fgp$7^=4(Ir!Izzx27qq3_q6>pEpiHPnbeh*?tIJksWYuFTI zpG(y=sGO)G=4FTtpnpS^!v%qj(E~Vfe5AFhOPQju2&=2zKE6KI6!U%@{3xL`s-DeT zE4oYFif5^n-X;wxtQ#CaX6%<~mFQS&N$^v1J8#^$>8-?y`vD=r%GqHLw_98JYe|!flx~V?%XaYG@(}?E9WO+;BKn{&Z3S*auO+ z{rFqdkn7WUqevuZ*-a7COs>vcv0@Y_AsHkyacC>+P1Q4|6u@AX+rGWro1-bcjA5uP zdnXs0VFlnSpy;Zs39_PYFaNjZ8@r!x-59e*f%Bg-s>G4e2i;zDfLsYgK*t}uY>0iw zo{4hU2f*yeTm4*a%Sb_13h+weg;Mn1 z%gbnLT#A87f;{OL_aI(kY$>B#^7(^$kKLCR`^hzjcWxE`bA`lFYI?s^2m((PCGvg= zLbwU6JJUO?7c5qQ76Qs0FLrCNTqn+vdgbvP$ji5b#X%K?Q2sipKV#14Hgl2r3+5(1 z4l)4S0`9iNnL%-%j}CY$1%vwrhu|h>ia34rC}AbC8AqTG@ozv<`2a5p$d-wy&--NEqt&-766B zE}5nULQ`2-JPC4`^zEDBI9UQx%@j@PIZf<@)8RBjP?6euO4=Ma&*lv$vdqoNe}|k+ zqx*@1j}!v5jlpF0)6-tfvZ;FLj*3XR&Ix=jpjwH9uL9fDy1R+unVJ9!;4uMDVy-_^ zSqVW)3a2$de1h6Ap;ZM9)0SV$ovc$}OU2K$4h6?tgx?T!q9HX{nu4xD zo_NU-ctgJuBNl!GB^hh7QPJ*`S#oR=_AxS;cwsM2;%TkEoj`?na_*vUI!m0vu!PO2 znJp!+4U&e|Fjes{6+s{qSuvOFxIcceFeWcY&jr1ZfI62SFQcGgfih{t0o~f+XG@F! zRyJKpnrN)TK3^L!e3}I*b1~frp^OcUea-JGQT;`4ou-kT77K?;sVFD|19BR|1){v(vq19<(O+~icqyO!pbBf zUbHOWkAu?D67acF1>cE)#bLjaep?F?H0iq0&c+`VWVrqNOp9rpJ&*ua?i_fT4XV3U z#NyMOZ$CL}(LH0lrJP~mO>@&Scvp_`l(Se0*NpA@H!(JGopf6Izy@m_5#4#XkS zs9-?yoc_v}N$gWC`k?Ot%B&#ny;o>2W^-(z2Ey1(WT>{U@ebhBZ-yO?{`24h!FB6% zzK(f!^f|ccrvVC^lVf4v> z#EwUOB|BP0Nzn1-0pe6e_|sHyuv5BL0$L5CbOQ2XB^52H>mcX5oKmDP{f;_G8Q;=n zy3s7z1|&Kklvjj2%#xJ=+CzhU=m&&ZyJ$7wHlE#5v{^VOTQ2|8N8K{NWNXXD0{*W$ z0JT3_w&#`#p+v3~UwZ{O9@Kan<>fA3yn_i8P!fiHT+1{jBq-nywKNZrL=kLs+0RnS3-R#$~$-4Y_J*uX# zog*;M5gwpbap&T?gS4Lk-+t9^Q}g7OlOH^vQ;Nix0=bUagA|yY-Xl{W^SwtI0v#a< zAMS1QJTy~QbC072uPIoqvL|9yU|>E(F;kTt5%Y8g{E7N+(OyskcchBQ1%soaU1}wb11vsrCtty}abSY2eleCEnuss02wcY_oLH z`Qmc?y5_{g^WLDlfI6q?NkJ$IO8codFHL zcQ(7)3EU(9HAh1Ps@U)(@wL-03kdXr+JuVA`5byCu;pd%r2CH3vsvm0S8QS2_QD$okK8In`Wl;?>h60_O$nOa!nxfJn~Ro!X(CSdkYjmN|~ zS;kFbo3l<_v8gKLmi?9lZ=211g9S^Mw`;AA*!3#l3&;L1sbd6i_HyCe}#Khj=Ico`H-)OA! z6uH&&G_(U zSWdxV`(5hexse!b-_eLDr+nmZ#NUxEIqNV7=7OY}tOo}`>!J3IwLPWHEW*ZOUh0>H z3qg9sf3_>ftqpjs=nY|=Yp+L0Nk5ndRSDz@IHI30$?GGCn&D|@X_!lnxSveZr+080 zfj0%rH1|~QfK&sQ3ZWoRJ7W$PWh(``5HggfW7S0qLZn&UpnbSg|nKxi|>)n$@#5q5F?r zLH`cUUiYP{1l2~gy@J?h#M1})@119xOkIANpQS8V8>BB{j)>TU z3^Do~<~uV6!s^wPLA|-S8DCWrf!T;*O(^O@CujU`pAYmqps(f^Wd0ZG5K5^vUY9S` zb1UTisS}rih+9gXa?BfN{_7@aHsGtFlspl?-p1M;>gq`mJlgl#p{HdVxR8`>?7N!Y zS2_*@KZ!Q%*483o0fq2%ef&wx>Su&w-2KR|a48$KbX7RWCGU8fp;n7b1*+jd5ebyR z;F`U`-Nv9{NXhauy$7UyroqlkX9*-nw(MN)DtVvNk#qNEs3 zn#Vu2nXu|_(^!qeT#I;ympM50)S<{lu>7RGyWKvLpdtjaOJA%N%RfZ>E-A#gQdWMy z^OIK;e8M`@gkQ3ZU@4K>X3Bp--=>D$LcrTsC6#(u1j{4|0e;wtX=Z4%8^_vwzOy|d ze9=Am{O>!3m|;1Vyo+Q&)n{Pe22)g2WCSPy)JzosxOFk!*rzDx!+~qCqPp8@#$EtJ zG6KNzT)~4G6;LiB)R zo%_0qK-&*d6B(zjM9U5(cLUMx>6qXrgIm3Sj=rsjY3IBGI*KaY68U#-QW4L9 zDf0b`Oj*>h*S)_NzW2*~d-`GbY~3$s)~wTEeW~l^p<3%{n-S5M5_sH|&e6Xg`&Ex+ z#IG`P&f?#G_%-;di!tcvpfW?A-%Vbi)J2DD2_gidK zN_!JF>-c4zFli+GNV?-!okt_=^Xfy>d-Nue+qXA9H8!Pp>>WIxrO)&E&JAUEd%y&7 zXF^9Atj=hWA9a_s*VQLgVOv2@2940NZP{x{hj^#;Mix%0bQf# z_1qAwFw?bJtWXLKPT2M|Fx5uGi2&uVCm5=h<9gFJ*Rzdc@}ls6gITx-bmoK5W-$rRJK#LSSyqwqDq&);3MGU zvC)o+3?cmpL{w9|(XPoCb3VbJ!Z-0!j5=(LKA5e_IGw$N|EYSr*p>XO|5B7vai`#8 z0ce9K^j58nOfB;OY|3N8NEokr$vO%*p(B6G5zkyEUa1nTizoi1K)Xn^1+0$<*IAf< zCEy4QL3u^&W2-StYjqh=!j99}eGzjqkbw&IyjA3WU<@t7w2XJo$-yx&P}V{@Zx=N?W{0K)KqUWL58i{|32q-pB@y=ObTq@A zr!i92Irnz355Tb3!Cd_UJ|&|St#%|UNXp|~RK8dMPjj04HLw5oZxxDzLMv0lFqjyj zHWdgs&$(2(b##*``b06}vWOIgem6T(<~TI|8@Cmd6f$Sv@_Hy z%-o%P;4QJTgkH2)R>GxoCrDJ%52PMoh$CkoO;(6t=5|q5X#5Tko%SGi<|guz1KsE& zQ{iiq&5xK@=>1;TbFtmmfG4&Ka3Jds#20gU=07W1eKvi{>;)#PX?z;P!0eA;QuFPj zcfvLxy(fU7&%f1I+xK^<^PMostfkMpBaQTnXG;>T&8q!v7l&2N*$4~QITN7VJ`0V7 z8~a9SIT5+SexVpti^#P82?c$krc){?YCs?}B8AckZ|^+=7}3CqAmEl7{|&{sKf4`6 zq@hP7pq)D0>Z1dKnwb|Wg2@ZUn@Kaw%GJmbxBDgahIND~A&5xLgpd}$gK0q=mUot| zF>Oz}yg+Nt7)p`+4IHhh%s=L39&=4w5ym2rDPUqjMe!BvirF>)zPI0)VI`H1wI2il5~k;~9YYnCkvex7Hi0?vTx=d-bYm@zRqft8dJK zBb<@C|H+%+9vY=fvbiT~Gs=mczDz^%cu$MRpfn&c8qAG$zpJu?qZ^Jv0Pk6_n$Mfe zMt`hsuy~3>@b=^MA1fl7nPo$Vf6!UD*!Y^FNaSbFfas`?h-jD{6f>c)zz)m6W$# zN=4+4`?AbDdw2q0^hL}Au}a?5Wj)s6w(1kTUj>8it5gIddD*LJyHQiz4zkS%-|;FSTVEULf1F@QYI9SJ zlQeNXpbe%jl{&D+SnUJaT2axTt^_tI6uF(ogpsn%1VaZD7|@BS(310p&4=NE%|&I5 z8%|sF$_S>rNO&M(ek8Ff&v{9TGM1Lev_x+|)DaA3Ie@>{NsV9M3aN?A?~tTw*1Qc& z1u)9%5_AkHkgP>blt(=z7c~S(R4?x@3H>mPM87ORBU0{vf(@k7AKr|5*??ith>K^C zRyERZGeOeZWB|aF-LQ6>sc_s7R5m`5NXysYZm-ukoXO=85HJW|##~vHxj$z5!`F|= z=|Dhp?m;e6Z8~cXis>KR&oK4$u`VUR7+A{ zOjK7-Y=?2hCmO$78GyY)Tj$t`V)(+4m?9%~h}yRUNRc*bH_S47{YXJQL+f62BA z|8pbef4D78unVDaj*EZHA%(RJ8WB08J%{H-UG|`_`K-H+{ zDW3(9Xg$bN7C#E7x1Bqdw*bNkK*UkEpSS&f3NAn61Vrd1ZxsF3XcW@Aib<0dU}XKW z2)ygTFJ)>vnQk)f&3HBU)E(7z$aS*B_h;Jk4Z~{m3 zLK*b&jR(g5mg;dETMytMX6(^FmY7c!aGI6HDSiW@@Fh}7w}@J5Lu-_ zraikpE1Pgry#|677VV<7as><#AtMz)wL~aHfduQQRSzRJn0_h5Iy`1M&sl=E!5pt_Jip{| zXatFU#IQvwG=dpMUq`}QN=3=2f8&Ac z2vpX%VkzCHEu2cwb0ZtpQ+u+inx_5Nk+>9`ws{su!|pLbjC57JKAzw|+DF<~vjbHw zKnQTVE%srwz9#-d#8~~Av-&p}lpoEP>Zl6QNZ%|Q^f0p1^!Ypdif5xh8LqJU^3!dD z6lGWzUZ?0Z<@Y&v&O0DBu+hgZMuq!Dl~+PL7-bFo{6Eup+}wd>sh5p`kFvdRRonup}e{6!N5Z#PY$mOE46p%lf za!Im?eWR>Y;Jg5Chk0y`JuV)UPPwUTpa`Slt=Uh7#U~B{ipM4}V&vEdBXl;4R zJMaB?sNwPKYG2TcX%%inPfV2kJ^hV?k0fPS7Z-$Fp=X()Y zaNYuAvQea*Ov!ZnT0`wo`%tZRv@9tMe0H>Sq3ARM`U{Fv0nF3HH1g3uLCxzBCR_}(tmcDcENPJ;=V z)y=XIC_fw9i%B8$zsx>D&r5T@M6P0z9f4Ed?6;W90zLPPDn<`inPf?Orlv!jF{ZnA z_`g4a{`0`1LdAXrlE#UwvFal`kiF%_>UQ2_no@=T{q0*@Jn{Oh?y7zJVF(r9;>FF! zn&M~Xjc?Ja;Hw6$J~pAJS-hw=cdI$JF9t96E3 zz>3qfxeSPhBz0;0XZgZ@Y5aT~f#m;mc1i6<8uNHr5$Zq(j>z>%3#NNaWK#36pX<)Y z_RO||u9IqV+HP-aJ@xkKO_FPmYPZMOqGy=4r>r81Qh53lxFuNi~Nzk<>+ue z%$2dsO#b<%?<9d>^fG2R2=~@;lnxlLWT*Q1g>d690Cn%YWir`gSxy*a-{V0xQ?{PB z&T)Rf0V+Wte8ZCZRm+q~oA`)e-y!W~te_i5Zp>^}F(^-Gr+wB1_^yvD(>lGkq(Kdp z@g5O;5QHnTEO7M|U;385eg<(H(&BwZNjRo%6VmQ>TC z^YkSn3Pf5#%HK^Q}4n6e|p^3&sY= zbe3-sb{eZNi{xygYbV>0elk9G`3DSqT8HUq_b84y?2m~Oha+WNWXcZs)z<+1g?8El zB;_EEwmHc8d7Plbm+{l|zqYd7ELw4?8X4YLDtz>(@zn{VD)s3JSifJ*jnA7TpEEu4 ztJ^DZg&vIvzwv9Z31JyQ%Yypxc?2zc=b$>D0`Vt)*|2GK)@``i>`XK%c}R5N#D>)1 zd4hLa>LDr0bNsXSZ_0(dLI`9>X5^&&j|6``nv_NGO>V3STU|GkrV^Dsgs?l~wQVv; zuVX;P5V-72K3TRwf+COP=p0sniwAZcuYq<#gMo&DO`c5>%1bg#Z5#CPvPOPr6%;u? zZ%(o_7v*E2Yq3L-v4^AYUSTNq5B|CO^sgv?bX_(s zmDR9*2?zjGOYXr2Q`mKBjD25J05>u;?zL#C!rBf;mjF)k*p9Pq{F&C=m z`hj8Wn`U=|WFle0)DBgl5^{t53imN52;etFF7t=bVVs}?CTQPr#v|EeRH0?1K4Y4 zT&$y*mhC{*Pb6^%woWeKUJUKXmz4dw)m=9zFAI%EMm~I*f3er7dfRqM*55%R@eCHT z?Nf>&S%E`kUUW=<4maT0KI zRM`PCCWg;83}QFDb^-Ee6`hzNyLXt89z$?s(CGK?rAmOt4>%qqY4r4T<688M#qZ~j zzXLpZgHkF5%~VaXfD~N+u|Ad0TkvtCg<`W}mXxq!*Ql#76Ec~Ka(MoNISBM)!s+4m za8!@#!re}c+x}Dr*q9@#l=48puj=G?oF0lB`DW$+UMRzpfj_+Ikw*& z*eNa_F@wR?0%C}ntgeM@tuP{~gsI){?wj8~z@mT_3V(Iiq-_58_qX`hE=tS7AGG^? z%nhP`dXrw!KPwU=yPe|TsPxxi8UG5BQ;9Ik*yJJyzx3SLoxB=zd*#PK4~dLGx(Q z^D9jPX-_l>EpB zfS|HuC8-an$v9a*n>D;bu!CvlP&_Er(R3|Ys`&I|)A8U!EccnmhVjC;JZ*BAL`V-p zR5-&7(!>S0{_%rOxqg(b8s2NN$&WvYQI5W4Wt*{KsC)MaIblM!eVok2Wtkj$%A9z5 z|FtRZ1q;?o!>5u)mFRO(LezGtf1GB@7cLW8b(wlxCX_kvmYN8{z`7|B33_aZ? z_`xyUnNAGhAJvVjdev?BE4MBl@Q(QfY`sLsu19nY{(&axl^DZE3$EsNJ;&U|AH0!) zvpE#L_#vqu88Vik6qW0^_#;w@vS9<(eNzN)X8k1te89X9bLbodsa9e7+jG*y7c}xN zsaP=e(oJHXP9kwXt=2dFH$1~&-3#i=Sgh<(v};S=PZs+uwK8F719H6Ec_6|Te|RGx zsq^{1NYXhQy{Raw#1@OTB4~c{FyW}J_TSf(b*l9`lvCxuVS&ew5YkP-kBfus`EW3R zHRuu6O#}7u1|bk_~hw_ z)jgdj4ueZNL&%7b782y}sFgwT@3a8}u<|HM{l5S4?$(1Ii9vNi-7o93J%*7%nP=tR z3(Cf5r6e^elhHRrPz#-moAhh`P0;In8xJ!<$iP9Q36h-ZUNAXNtIynHXRl}u^w_%t&Hpu@^%+IZ8q z;+jh>Bn-SAxVtUxd=iwOW^t@Ej^IRD0CKWHR#GXsL$ho@2 ztsz0HV#w*cDT%g|A zg*Y^`=!@0S{;kLw<6Ge_drYakiUORppH9%oaxMT;)y~{*?aAohYsbtNOnC6USe^Yx zq$Kyf8deR}ZHgfK}Qyd0S^R8<6)Wx5q31Zj$6YVqY0Qo3eS+r2&)_)q^T zCLFz1Mv)WZMn6=YWytPG$cth>kdVLSZTL#0BN^d?Q99LWj7c`wK~tmPAHlF7CoJ!0 zwHTcQqbz75|AeDe{-~8eXn#GK@Pz4L%C6n*X?Z}ScBjNe1d?I1UL3$f&IA3O9_O$S2d^Z_^JQZ&^nf!AZ4hWoV5|m($zH|hoBSvsa6e2< zWWg6hU$>FYdx90{w45G94kgF)Q-KCyA=?$@FF8d{nvNqPl!Bau=M78vjV2s$kxMXx zbN2dq+?m|y!8O;HlU?>_Kd$p+h0$B!W$3cb?bl+}I!~pVE(<`fzR2_1r`I~W@UPka zy)t&n1KXFf%wVG^)mPNgNpLh);Sq-Nmv=)<3hwgaRNwQ)uO!uUG0<~(bt*>Qx5bd~ zpJZ^P1?fDWawaq$KIXI6X=G5uG!;VciYH}06NqM_#AcOl>xCW1%VHvrOd)`8Lq4;@diV|88~J-y1T1!S+QAVS#)&q zHs%%C!u#WvvK`bITmid&xS8krcKukf8))4D__u^DZKJ&BC7*aMT^r@tM&(U1J%7gB zLhF8ebW8uw4R_2Kp-{4s`}o5ohuO+|RboQh@gK3z#0Y2OwV0mL7%`7|g{FSm zWQ0i_)kj?536GB03mX{dQEm86AbnS5#%Jay7@_3pd-gctG!iQY4x7S5k<5{`z1S&_ zIT4w1bZF`}krnr^JY>;iNP8qX*D`b7A%v}=Z~VPtx46x}Ku|r8VEZmEjwlML3E#yz zqi%!~7e|1T;a72C8$kiK7@nUs6cDNO=EKAcS_MQ1;8x77Y0-yjM#sD0+g6+Aw7|0CUnw5=lXQYfqj^}GQg^a<0?RTpM47iUehC^9 ztjqet2Mza(xbR-;cuU$5&Rl?k0psQsUKmBZO^#eF?x?){1A`;BG*ABzLj4})Z4_?8 z2`(}o)D_~RM&zF3=Y$IW_nI75iMm>RnUfYj>z}&j6eR!XB!>Z zF}ipRtYA`5B<3H#6oVVFw~b}{o=uG{zbTQ~KNxbvp{(0%Ry$l4K)BAZS}4TL!ce9e zDHr(*grB0hdsJC=p8^_*yb!s=z#=maa~~@|m9rt;2~o-C0&k$8CoJ}_Y`&%JO%W^^ zq2`diZze=xFP#qcRjnltrsCV;oaQR($%hvvq*ar&56~d@Vl81<(e}{l=PAE<{4cQj zTt{Ng^+&N9e~3VmMz?f+K!Uiyn$ZzjsZmSb5#ryWG^I^po`3pH)k38~13) zhl9a(JE$|WxH%L;EHCmSBMqFmsWEv_^34*)i%7e)yo@)y%(BrJXNwa*;R%H62{q+m z@9*g5qyF)un9?p~9EyU?v^9VZ@+yDzsOAB5zQ5{J_$7dZuagx7J*d7WVo0Zf;O!)A z9;o*H(9fz7$-ObsLhLI>4uW?1i1i7zFK5T<=fJX`yq+r}g84RaT4CHR^ThAg(SyYw zZ!t17&otKYP%33rc+9rkd)v3n+lFHMz1}F?YmY}RCfy)5M0H~X(9@9<2<^gxNi=)2 zI7JkF>~A-AlDthVy2fP04mQK5*f3NS=DfZ)2{K`n*z4UQAiP)sA6}|PdiSvao(FJGX+)C>4%gPLsl9+k-3vU~O93@_-%}V+j zsfc@RR9~&)kYN{@N=;tPydHG;nweB%wOQ}h^tS%9CrOBNuJVd>i4QZf#k-MpxTMLk(lw0Wa8S8%oIseBw2 z$3{B4Z}}7P8)=n(MW1hfj=ARTG5BNrJQK>Q{4>xvL=$Vqr_F8cDSb@rJ1om>F!uqw zQI7k1f8qu`YIYRItT?zi!s2k(xiR+>nZBh(bSS6#4ARIsjR6yBI#%n-Ih6PlXW@e3R7$s z#uLXEGoe9Kd+$?L#e~aVCj4Mvi7%t|k_rt{i?W+be*{H?JyP#J0D(^hg_#03&5O^oejN;K?Pkfz!a7Gr>h$;;hD0)n zW^>F(AB)U*@Gb~ROrGboRr&t;M;a+Dc33Xy>)_9vd-i;K*#)W#H@~QgV=#97)6_nU^;bj3-MV8WX+}o09oPAOC+2Y?Nd$HaRYQWN6|KDV zK2Wsw@oFD%!N6(@;>UgtR}GVvZ|~hi9Z}+{x|~y;^dbuVQQM`mRDFXW=9LR#q|o8+ zgD#IzfCHL9!e|oVm9Fg7C8R7_vN9O;`@Wb+VV0lrnrfbLT=BzzPkp;vgkzvE1Xxaz-ClgIT5A-M~>cTCHeB+e4aprUt3GjmS&lU+9nNdE>yT z=Si)`H?1lLMgf4wt2Ku;@63{=eQi%#dqm%R8lf+K zsriNkk6l=zw4=8!fn0HJJIzf_x5k=RryDmSOOi^NY@R52x1{PRQA-4)^7E7`EShig zST>D#AzEz>@I%wdxd<&o{*eBPWpEaj$9qG{J3J=Yz9zTBI?&6Lh3`a0s)NZC@$I6a zjvO8v1V#psz?1$26p2=eu)k-z_XVw;6>n*Ng`i8~(%_$Q=|#Efns}k?BQ$5cmC2ZC z|9df~*;@RQ;m2rgixqdHR=_n%L-Cw0G@p4qd?k}D|IizxZyM75;l)q~+(59M+gPya z#wPGG0>{HS$hDB0z9md)d7ta0W}6A;Sz-6XBiM?3G8_|N(w04l6XeVIwKzm zLn`U)T)5}EyGc95B9OuV`Fw^oKm$nx(a!6S@mr*LTwgr$mnuFr_4J>LEHr2=~U zY-POBQR|~63!jsqz21xEKpJvuuCQPYAveRNClz0{ivPIRPdNYFm|w%Hhu^d*P6a!~ zaJ4LlH7JH_jVN|FNhex~=ZZg&u7IvNz-q-VaEoJuh*pE&WfkqlLyI+2(o*tCxk0zg zFaySPgdvXz9~FAJpR1-Ocj>=tkj0iYH0)GxHNf;p2}%7s7qk#0yyv8}0!tz7s(Y|V zV@&^RgimBTc7^F0!=>H42YmeAFp}Ffk9;XOX=f2kdqOS}C z1mo=8!U=c+Ln&|@VAfyhp-6!;L}&Yc zuxuITBBVn}|1!dI%Mo_)DvQ;}`%QJycdPum@asz|x&Ipal7CqereZ9reWnBN-EJQN+4@tb$H6NI%z8A)y;bc5je~#`42`oN4z!;CQs#?Qe zv1<&Xcs*TTku*|XLmtJb5N2qOxV;v784-L5hi1L2=X&A^)+JfQVOlxTYl5uw}IvlFc zM!Tbf3qHkM`owyTnVAYk@gFGGc^xCsN`vP3ZpZAuMTj-SvGIiPG=uCDInQ|TidDLw zZKkGG_~dK+hColoUi_x<$W9i;kd-r#C5b1sq=gP^UTct#pF7K%O4z4%a_6UR@VaIb z$f+AGl85FWyv{vwMy5sc3oKCZm^Gz0^-<@qUDffm)1rTUf3c~#V_Tr!UAWxZwGzT{ znD`~=Kaf42SFnA*4`C67=={7Et2v9c(3o-V^V2{TzvQ=0S!bRTNk6bHU`{x7rD%Tr zh_fbV+~M80+vKSW6$v@bKAAH4yyK;X+W6V0u^F4{xrwSRO1%anX?p#U80-1Xd;R8G zM{a_D9kRL4x6dZzH$LG<33HAKM2m}!UHjsSUGc1|QgRBTCFZE8@ovN*!?CZ?mL`b= zbi@8PnNwt=TGaNev*f5vS|pUB@vMxODwV%9o6{lwsPqR`T+97eD? zPrVo_NoUbd{D^Cd&*Z#H6y;$$&jFe15WkFE#GjpGtqzK*bNEi2T4~Y8eJn%40-3*b zVvvWoeLnPhT4zMfxJk3&NveEb2Qwtks%Ip)x{`p`XgMK$X35gDT%BWI9yFfNO{7(0l7`n=FCNKNxh=v`6RsU;p?Q6`b$M1yq z(ymp8ex6AlSaU?WJ$$FxSuNyYgu2?Nm7B&R*qv>?v6?x(z%ua5tF0LSR+S~vJ~W&f z&mx~AW9l@`p^d-5OgA=4=spu}xdJRvx0%!4rV8aSX@x?ho+mel82#HYYO4p|=I|Ph zr?+HD8UaaQ3cV=Rg(MCcGp9G<#}fh!T)uo12XTM7wg+~< zI^4ticv}F;)xeM+`dL_tU^z{&ZAWQeMxYu)Ygj){7ju0^w)?6IIs0HJJ|o*zgvNdB zc+bm{65OMW#@l|6ybuJIqFm+tR2goKD!mJ>8+{3@9+PIoRYH-~5mz`{E;aDp#>gvH z2t6JsyA}JmG4XB_xzMo0O%ec3e$KwU^hji1&|5TbfatNDfg5|>CBsm!b5%@f4ZA4E ztmb>{x|Y_3C_8qE>Gl)^~gLbl&^^26Dr zQgjD#+iIR+cz=RFvigK$1=)hju8>lEPc5rze+C+01v~fkM2`te@;n=4Gxw9;>t-xB zx{2{INn)_gN{zfm-W`EAbwxYaa5uYx!QyYSVO+geW~Q5oi`vujm}em_5?1b2ZCsH# z8vI^z#gQLN@7k3u)`jN=*MZP>nPn*Ht784+zercfsmr@#HxH@;=tZdSDGqkfnTaH3 z(|08Xtt7`L27m(My|yuR-sq043&RT`wRL;Z0mZ>2M7;c|#i)As9mE-GpnvR#`w zH3P^{>ZGvbjn#p z&y`c4pw@FUJby%o?-hFmR?6vJAzpvDAjw5FOnAwZ)&h6ka`ONoiEXVpyb)do{ju5| z{T8$a4Srx#*1t8IPRD3D`N_vRBOWW>3TF=DvuZnNOT^;aeqLXSL3RvU4vJN zMj#s{qxn1C<(nOi^B3!MImE^F4b(Ejydvnam@6e!2c07T>(%08zGMD68e);FoF(dt`%{FLvDGsFsQ6h*PrWje_=poFi;TXltluSWzOK0NKZ;NXFKAM-^L zTvLkk*VdqJul_?@z7WwIGc3y2P2tQSUhEyJ!jC3cZ1cMk_Y2a^%yc8MMEW;6P1jFA zU0kv@|KgTp1^LiQ@cp>^!cFtvZ6!xr-G9FG1Okhf{%)-3`s(%ev+&Wr+{8`v9+{y; zsdrm;9EY>zV;dk4YKoC5rzWJ!yoE6{{GCa@Ud!#L-nsTlMcu2u!2v3EVL~)_7VrIH zInkN46W!v)bm#~c7w0$X<(XTzA_e-)BphG9{L#UVeb1hu7UM&4 z8j}%*uM9A8$F+*30anCmKVTK|z4#h=J$1S#{vu_+_2+`!H`60G-l)v$MftF#lJoX{ zY#YxtUhHR8vhQP4Y_qi9l9-ZtG4nn{`{f-sc_zAl(S3jxZ6vNHJf+ z8fRbkAS`_^(L$-O5KXhBQO(~-g>XS*Up@jt^kI%pL18M3(;3601PWoQ|Nbx%SXoLP z@`yH3)=!CZuip^!66xD189xIZ&7sHrC^~#(GP;=T#fgVG#9Kn=@EHbp8be0PWNW4$ASB`H^Z z4xhNbtzGOR_XM3&ZH3(?B)`W})7{_JT7))5MK#z634@U*Px-GNB^8QnR}IyRjp-!H z(Rjb@_>0_8A2NrOaM{5AhhJk-=Uc+?o@KDQJU4lfv%dorxUe>t$uja&QR!q;KW5`c zVMf>BVW}<%d<|(EM;~72aXn<~t>w93x@1Y%K+s;xk8- zSi@0;8=;YeY!{J4^uIN!<3D*a@n}qhu=!ceRQB!~MTKJ{(Y`Ru{cK*NQ2%@yOqHkn z$=qO6hQdLz0rh>UCH0|){fEpo+%IF=GHkVJLif`mv+jO?_yj8DRLdPm-+T=&PMXR3 zFgJST5|Pp4wKJvrlm+ccm2@TdU|AfL_+Ic=H;2ZBTCL^P2miw{+km*)aK1Z1!cM3l&uiguDQYZ!MfmU8?d|gV$(qCI`#P^UZojI?Z~r$Lr1OtSMtWBv zG0{)XByaCsCij|IrN@iA#{Io}fRPkBYUL^kE}&P7D1|Ydl7r=tO{!D!MCY;ag_J&u z>2xB>BcPOKpJVq&U8P}M2%{q$<`DH(+AT3>5N-#4A4HZ|GpbyhGMMWZ*<#SeTc5hw zQyqh=l4!n0L)ruph7|7 z+7I4Jge}=u9!)|iaO}Y`xO&A^_E|X%^;0xs8MZhkQAtmu4(Q@MPJa9~Bpncwa@Z~? z&4Y*u?F-xHtI-IDS9;841R;|N({(Eicwb*!E6o4lF*eQ=?OK>p0Fg-*kp>|hI&+SU zYA*_v;G(&U5qAHi;_)$Qc+w)obrN76&^0MIyf5{V|G987+jO0BHDUvzQ)csK%FD0Y zMlnFe=u|j?OO|l^k^kRGZpi}kS1KP?Wtk|>ubgZ@{IS^(=Kr(4YHVCaS^krlCIKl8 zkK3fKq+d@y-V=9lTl=MQO%Amn@T^s{zkV;WoTB92L(9tu%4X0W{%X$a6y657pY^jT z5sWLn%;J`l?1+f;~C*5BOGbC#++)c`5GNO%gWsFM$*?V?OBzS{RyUCw@z_A zUo~2Vt%M4LCc-xohcM?>3T7Fpt>FTN0wf2JptRP6p4~(JIHttkpx<9JOC3?dLzW&e zoetsUk%#7eZpbb9JX}O zK$3iIVjkqoi|psK#vi8TYsEg*IW{ z+bem@LSkBvkhZSb^4qC?<(smNT`{zuW~cNO*n(@S2#$cf!_um^J`5(R|1MZMM`aGjIbVMy@|?_Z%c`EFYK zyJLCNCa2rzc#+!dQV2FiiF3i&QGIu)pq6;LYko^wuT85yyN^-H5#)bQQ?)30IHp-|7E|WWX=Od!QNq2a&Y>R}~w2@vpp3 zwZKZ-uY%})pfR>Sf1wqqT)3J{AmaD#t*_bal^>Jf{`Ub6FUX97Pq(vKw9nMpuyvil zDpUX$u;i$vv?TXxpt(bEWA2l|=WB*Y5ZnBPPY&d$AFnp9+_6pV5>YNzR=X9GF8Jef z)9(vhAq|IfZST5dCFM&TSl^@?0!WQ>%*xUn%oHk6Jh$cd_5D-#Bn+`|c>nRiH=G8j zJcoi9Kc163_kXHHbr~uStbOJl6XMK2_JL zm?`y@UUzXYK99MB&XV={iOX6B$?Qq5<()<8dwp@9qtfNp11Su9h(m@A@;%8czYVhZ zVi!-9Lid3_T`h48jr_Zu=MV}uzQj5)50UB)=BsM{FeD!R#L56;`FwH$2|g))DECiD z2y~iV0G)d$PSq+B-_Qke5Sw%-WmGU;-AT&SY&t9S{J|A=3i;x2ZskJo%oKrhplB4z zT}50=3F_A~puH9NZDy-PGz|!vzN$7zB>eM!{`CWXEB_vHf_crcqbA`0ffi-2D_1{O zk1nIxO^wpzvr-2uUaO*SUY@^NAM>H~C<)cw-AJ-z!q{q7Owttuj(_{-T=r^A)gWf( zJmoA~`ezL{BPEGhoIfNHRJTV_!3|)_HfUXg-htws2tsVS*OSyU;mgs0xIWGf2H)~# z^}C>#Pf7t|r`rlx-9>d2!jMHqzbacF;J5=p-HWd>X(F$Wfg{<)gEON zUYFjWv6ch_HK&LC_knr0iVdE6r!%8QJBi)+11TRW4tX+wc6o|Tx%5+7hF-eOeiidh z5w1A}Hn<-W?Q|tbL(J7yLI12IpTXqQ;v5zE3WG(;xU=Ol-ZE>`_1aInEzE-btI%gK zbJ4Y}*0M8*=HtPD7~;!|RDR8u%GrHpXQUxj7`%gFfz|*iF-gG2jjDm^**S4&@?Q@a zf;UzP`~nY}oDaYl#pUIJ-V^NF5^pL-Sg8d+^nerG9T)uQ0swW`srsO?Jb?>{maq<3 zhDDKI>@#RyVz4ObQ?u-eri3898{gOd`5>7IRrX7pc7vjkF5kebPIjxJ#BsKoS!%59 z%Q$@FVH_K(7jCbsu34u7$gYVAUM0Lac37q&&+cj?6tfgb2a-Tt>lPfiC5(y#;H+S% z;@79DW%gb1JM!<(TX+m`D(Rr4(3ZNIQ~dArb#ZG0Uo_k~EONY}nke53_(sf;6wJqS z_Y(bPd(gMM*OS%RYQ|fAwwXk-z(57O;ykO$NWgY^`zr(kctH}`$yDG<_K1=BZ1wE` zyCyPdj#()XD)RNsixwXnCa(=8`&S!1dk>PgK;W-GRK~qZOSJE|7q5IxVO`KMf$NXv zfx-0)#&V-VVtgWmz)zLge*KEuxC0gfr~XbZ7XvTu)PGa}DnCfC!^K6}7!f_UZi7k& zVgO^kcY+4TXHPyxUGkMrFjS9v6c74iO*y+CWY-F=o`FT}D0no?s7+zv*IMI;+)I!q z{J05WmlKS!6?P}p#y{tz;ERVe=FdD+IARhQkMLv0z;n&Xh{rNFyPj9SxQG8`n-XDH z#RDUH52SBUOiqsIdQE=-23$RWHTFqtqb`}0PN4Zm)1og$e-bk~r^1>=&R5v+AJ{-- zOCJMauYI@g;uJ@BX@0?xW#d*IpK7HLRFR?62mI)ZNF4^{8;vOQt;o!$#~fXVoafORDx z9`~dR-a6AKatRPe`z_#%bFYXkLz^lFskTtfWgY)jz470lP?sjb*KOb;6Ax_y?VrWe zV4y$))B0>$uMHfN!@>)2oqE1<`OuemXmL&*(;5sH9 ziHHA_8#F&e$=mqwX9&98;J~B^>7*yaJK}hZwJuW>lk^xC)0nzK_TYbhiv*$y&1YDq z17zvq*P0&JsEpk9fvqMF*JtsJzEwZv9N19_m2&6%-tBi!I{E+S5$*DMZ^hRjz8=}i z@wf`($QoS`td3_YZ;Z)UuUI#`|s~V!%>MLtM3men53|3H+gIyV%jg!>|Gdn z*-at)JxQ5DM1A0{I+K{fW!vA;>G9ObI&Te!auss1aq?6^by#oDIFXzu!qSh8o_a?GpCCRKC5 zR_koyZHHYgpGg%8pX5&~I2vn^;v5L66mvlbK0c|UrtAmm3}$Eezjx&%>5qK}QaEcW z*O9AckH9rGl06^f*$|*afAnsUshZj_Bw*U0Y8m)ZJ7Onp7Q<`o|cK* zIh4~)4HrVQd_Omo1Zbn0#yo9n`iVG~DUBeM(I1=T44%OKVmEK<4 z_2l>!j^#t`#A*tMzah%!xF^rBP?-nz8p8It=NTf*4hV^<1>Zf)&H#78&hm{G4;!U? zexnL3)}Shf@B9_S(nCKViYth{+c@_&{js-K0~~}|jt_F2szs)*I130|;ouz>ko0K? zC`y3{Io+Roup)c>{I!kmI#~QsR%(lF7+#m}b>!{J(C@=uWE9PuLuUlm_dPDOGzt9; zdDFroQ}MSN?02E8Q~Kb%E?Bj+1ux`=ut(~a5r60kE-}^0aw|>UPn_>TN!pKFg2Q8Y z2D&MZ1rsd>-3n!j2TD|S$tz zT4umUpLmBG%NcK+OP#6V+Mz|V**Ow>MVnuvkw!kohg0f;y9%)=O6*GZ??S1B`us-t zkc?K-^)O@Z!6bh$#rfH%q|zuHGwuRBcW8qh>QCQm3q^26KG@)8Afes6&_ts~E0JM* zSzhfHK z`U*++M5*ma&Y|fh&VuWN_#M|(n#~;3d*++?_OQQB`nofPo~b?V5sG$DN~au)-=|#r znnXgcIVCKz2DXPW3+cQtmML_KJD(Hxel5m+Ey_V!CD;!#AL6YXzUt*!GlXnGL=~gK>x^Ak&Hd6o@opdj| zuCbCFZuHX#xMza$UdEZ;f|w&7OC_(*BTu4O30sBR!-a7olDmP#vk$IFB)3JDm0lfN z62r*Q6J}ifgO#LyuCqn^JRN35bGx$Z878<`hWm(kFup)}g7Ex4S+oLMxTx!2?p!y- zV>*Gg$FzISG9h;)x*59T?$D&nj2U4W&iUtP&}D>mma-e-S=|7=x9Og#EqXiBhbew* zjWg36QFL8OdV2h)FF5Jwg!@)4S9zRke0ZfQ(qRR=vkpx*QAWe>DvJ7;jt8^1-@Q6V zdja(YuJ)@>l%1-+vv%}gsC}z=2S(i(36Q4PV7atHG@Q#h6ne7bSCPCO>vb==m79Mw zJ~dyjQBCr=`|dvsN&pFZaH68&MU^U&v|q=5;Idq=*k4aY)`zbZmGu;TP&I`8Kh7u9 zWrBR-q(p26MLrEL*?x49?&RLD^D884aa@EKA6Z^8^B^FN8;wkNON^#M6z@jS-=cTy z5sao*9GOqf82>1I>I=DfdWVI#LeoI4+8+T%+CwVRqzZ_gc_CZdhXI#d)b$fF&8g0Fkj1K*HF#EnZ6K2K%)iRvuKif z(XqX>j?Z9@`FaA z<@r(#;+JJf!aC8LFblEajpW#{XU6o2tqbG|^>~3YmNnBVYHcxifv^CaG1Gczz<>4e z9qBzDIoXmOZNk`idqOf%>GPORBIj+HX0q_^#|k7kyJ!_3qrQ^4lFC~Zw?ETVNW<#| zODXpnD9s5J43s%f@bL4eQ-r%GZt=BplqkC*MN*Aq^lxpmcKAF_Cx4{gb}vOh^8!Mr ztyT;_zQpT6MWBaDF#7Uk)_zJ(V>Bp$@wT$tn7!|y1cHU?GgUsQS>X+cOYTnSKdICM zRW$x5mi~7vE2Q;<88TpU`I?R6(>>a~lHucD`7C$dimsyvpi2dBX|QWX+6>6ujmKD6 zJ=8D8+Q$BX@z*RAJb$bjiu4Anf2jUQ0S(Z}Pyu)M}#;@~KVvxlaC6=sd zgxb)>Ot0oaOKV1xQy!B+_a*Z{v4c&W%Lt=7{kcmXYz2 zno>m0Xye?Mk=B!&A+eam(C z7-8}_FaWUzZ9wEp=H+(pe~J%?o_^Ve);!C#VX8paRnC}N_iLG5v3+($!)HE~sVpU5 z3)xpr4O#lv%aYKR^cE)*{UJK}?lK?Ii|fQ3)^jc#0#}M=lF5tC`_6>UN86Xy?<%p?S08^3zz%nO+_tt_B!7@KYZP~@WI52@8b9w z%rRror6p@|JAUIu9xbr-@4%&RMXSguAZ`zCzvTjHBAV!#nykU-pc1wC z_iP&JJzpHzV4BFFwCr65S3g)+JH`*?UUGQ&L1!byCgl5kkFn`}DCb?l9WeU!$m&BA z@=5ljpFI(s0sd{wZ@H?7jHk38H7f|UD9ik z;}hd-SggqIkoV85B@;jhyfo!yzud44j~OjcXg_zIHmMPAUmjK6uBZIkSQ6UkB-zb3{Dn~H#us-x zFVus48&J(Q_25VIC|-+FeJ5NTQ>5U|$VDkO_;cK?1R_lGmA4&9{Tq?b?LDKFYE2h@s&xs?Bpu@;&J6V zOsg!yme|8TELQeWOxouYjDaPRpfmAuy>%AsOrpnqMB{zB_qg#}{qQy)OmG0Xy z;-@txI}GmgY~4Yj@V8lm#h7BJ`xmWxEm%t<{gBo)d`3L#vXq~KU@}j>GgMo~hYG=C z#X1%Myw-R9vi*|E$OV<4 zTnXw=dY#)B2UIAA>_@>$uw9nz1k?-~D10s_=4`@&<&RvGtR)@kSb1)-Ts~6#e{AEh zG6Wmwb^Cvk(A~n0BrD0Tt|zsI`yJPFvW zG$;C=*D=9LoD2f`y)g}r0I8tIlQ;Nw2@oL3OY^lq9Gilf$G#3@6J#Hc8=3{#+y;5e zSvPGZAw~dq_gGLlwL-}*&c7E@vKNdIestuvH_%G_4Wk>O$=9EPXqWy5aq zoP03{gQ*g}eNhAt0p<30HIkJfi=$BqdoBY?``YsDYq@}*JYz>Q9GDlY#6SZ(sypV! zDZtBa)d;96JMuIZ?|Hzf%4NeZU@WI!p!-=6kP(!7HNgMiV%1nSBBmj^Wpuxoucs3})2_nGaHWm_?>h_Um7U;2#(06~22%@Xh{vrY99j{gkYOk4XjAS#iLL4{pXh zBipnL`m;9lSx-6YED;NAPYKs$oy`>vbF2Bkz`aYgui#=)$4mbVEzln%S z?m%(o0~7*u%zps&z1d*@k@8{Jy*{85YtH(?C=&6`#*YuA^bznae|dki)w%h?QlSTc zg{tz46O-N@XkYN_^dpKH7NDBe{gmgIC88Y-0F_$xwbWOjwWY#`k|G zGU*fna_$6X%FhUil1O}>&C*zhll!3P5%%)kne%x{ zBXpec+wK3BE$E#tdc+s3omV&&HX#7@@jtd5a8pg@-hsiw=PP0?7e_crzvH(-`|N&m z&*k;$t03!#>M!#DKH&&z{@mL~-+^_*)M#r&Vf(x1%H8aS5A^$6S&@K(7C*Ju5<+CRMNZ*?|*<(eC%os-Vy7CPwc`?opZ7I?G-0Z zA25YaG94^-`a^fWeX9^c<5neU@rJ_bznsWifcN9DfgX3er*J*6H0vR_ek~c9Fb83T z0sRV{pK$kfkPw;{NCop=TmaMOx|ruS3?UFjzYE}jlCap&nxhn&b=^!`*^3hulEjXk z6ZIJLw!w2D)!1zge+r&3iSmb8Lxaz`0Gh;073mR_gUIZkFKj|=p;gogpd!=i8=mMNoDrKm~Uj_q+AS@TFp;o3X3EmgqcTcS&6KJ}F&VY}~4V)y;YN-p4#* zKZ!#s!+O9}60aWj>mobB@|^HI%XeyZ8{4}m;d2nD5f{FYCUhr%JRM-JPZ_sdG%Jeu z?C|k39KbL11z1Cx(Q7NKu+;$D|7)&FS21ks_>idPFn_@t)wvr7#N4@1ED19`@Qwb$ zmNvN=4p7x}m_&H+OG---6Q#CafH(yKe3Px|)vrQD1`Xo$EUN$LJ1e&T%8hH@eeZd2 z5Gn=yl-Qu249rm4e3l*xsMi|O{Kw8t=i$qkV$Fo zTxG-n*V7!SyK}^k^1;(K38Tz(z}yNF_m>7wABz&mS`zmOw*7*sDwF$qzRHJ>XW32A zRz3rdx8nQ83MIIWmTn97%44_O(Hm8!A;nJquHW4wCowQmw#5X;_%|(jFF2H6Hm&Kf zpbA>#xJYxrbcvH0E~Wbm4VcHA7$?7eXLv*oO8vh8^vu=sG^V(WxD#-YZgErj;o*jF z657q+rsAIN+d0bM66XMKr-w03ld*Vy5~i6S{+Yb;w{uY|fGslI+CMGsrTON5>WzJQ zyR`{!jM`)M`)=TFo-|daBj=#OY>Nv3Q^EL4eJnQbWGkWox0B_5hg755W{2aIoFVuNCN>s|f3S z-!DZ`S(y)9w{VL9bguY8SN9fsqTHvqEsn6!BbJZ)^L^+yE2QIuDat(A&?2&29Ln&;C@JG3v$X>qQdj6fPT%V277X-Qn|CCg{D{-tZSXWK>duNTb zSt^B$aTsfVi5V5i^^U++vtxHi3@++C=C&Sy&NoqdGYd?~vP^iUryv%*h{Z^H7?rfA z5_-|vJ83EJucywSu=kVF?{vRsykA{nq%txKnb@_7JA-#b2!^NwVrz* zrmBSMA6t@tfBkOW1!s#yz!bmU;XznRFYtmw7q>(|w;Fr>HY+jJC)Gz69yzwd5=@CT z9+jjRB79vv?uylu#aFIA3WMOXd4XTFBY^ukFYu8=pb*qF2omcy0R^(bw-2%Ldcz~@sl{u|v zKvOQ~@SfvgDnO>E@QCa3PrpYsFykSi6;=Cg{JK~psg&cbC3t$`WBbYl<&M9UJcaPZ?=9Xa`FAf$a@kClxkKO)#`fCh;P+XxTgz6F^o$kT$o07Hn(%nJ(u(H`S zZMdNhX760DxK0OhP}atZsaLWuhrvzTk46X(A8!Z!bK1^rbdhtK*Y;TA%E#Nf{Lb+?D^e~3 z3;Pn^ew|RCKGKt%HVE@^0RBghr0<|C!&31AB#l zU(~TIS!Jd5@Q#hR*}`Qt&o$VPYB$_Mkh8#^gXSTi z%h~p~rhk(n+^pZRJamq%Q>QwTc3bU$`fx-*4l@ZN0yfuwK9IIACfZXG-_t~G`lwsC z!_jq636{c(nz@pfdO9fRvS}-o!BZr?T2{7&3RzMOw)_lK7ml;=fx#9WWFpuGt31`m zGy_0XFfJy?u|tM`)V}5T@A!lV|Hg@D&OA9t&%(jvivU`(M7U;4XYG6NV`#7%GDR-L z)tEDx^L{g0uflUG>x9t#=Kb~V|DXe53);MwD8ET6_UQgP+!Zkmm(81NUcNyE@7jN3 z_oTs72ebnB$F4Sm-;3KS9_T*d-e}T!L9#KbP+85=6VQJ_6>j?!PuR|=-jfqUjWv&=pS#jh_2E9C=_ z;3&@RDKs#bt}u^Kfft`NUVkyo2u%O}SFk=Q2?2EVrm{aasUkAJ#17(?&yckYH}ZGf zGO1O5Yn&9X;1kxG{_9>K4MV(P0!-B2M z^8R8Z6Y5OvD)T@e%n-=7?`as)H@bX*(zMwPGq!8C*6({>(fb~3gIm6MutotxyB(BA z$>7{X_+EBJ`T(vwt^3_RcDjV5A>Ubp*Lg${hqZbiS6TAyQ_dIX-*|CTl}h5_4PYj; zB;dt}4e(3d>m%bhF%YrcTOWV}*x=8)DE;dja9ZoN8ZgKu){Y#iz4_*6&t#yG`QqH! zmKD_N#)DASJUa}%VwnSkRuz-~C$0f??k}h_MN!p`S zrC>7dYtY|Gs7f%wm%h36APb48Xa<@Dp$Sh+(|>@{DkawK+(-SWC)O$&f;)bH_BIl# z8~AbTt7T@`VoL%Uj^@7UYcJkZG4JFrke>AlLil=Jrscv>!qZXtfhcAmm$D||EO*u& z(hXbU@U(}irvG~06Z!BHoM-kLx$AyFJ`?;2E<);V_rd!Ywr(ek_;(EiTff(VDp}@< zMN;D)_@P6u{$$2Z~;MwKW>sO_n zEr#x3&_*!b5*-W=zFbl%gwIB+_ura? z^!aPI8XeQmipHcr=fZj>Ri5ianaEY;Z6a>>-X+!PIa1DoOF`i9I(W7#isO0du%okW zlPFVDA3}T2wBU@Qv|g}15JCce93`Xj02;j~_ePlb+aT)>AQmoz&garWe@`L3$h36< zjS!?fg4G4g+qqiR+L`B`<-oTvkMr{IcAMe(sI&^e-g>7K@GBgAo>YV!O+d8EK7guBZ65eRTU=4?lB9=)L1V=ATQyAzJ(LR%jEhFj|XHiHKTaVNPy>$jVymWUG` z_;a$cc*_hwGQEgJsP`TXU4Aw9Bix5emq|+;@H*1}yLTGA_6`YC#~Ered( z$e1;vn)K^jfGQ?aXXf9@x4kGFaP0NoplI=DfMNH&j|+`Xm_$<(DwfQRXSiC-8oa~v z9FY*&U&Z;m?VT3s%q=PZ$!c`J7y`+D{-VTp&{GJ3PxI?OCED8@RyA>_736WyqQPc zj@`PrFIJTzbrJd7xZ2$B@cX>`)|VtAO4h(|F?%rIDDrWx)GO+O2|=CdjgQV=9Va07 zO7rz__$?$v!<&ArvBcjh}Zv=kRr8YaKMP&9I|+aa#iZ>tBFBQAXSon)>gofLUR)d|l+6O@J}% z-#bolzvQoUN<^S)FsCh0#bV8~vgv~Q>E1v=udD=YvJHOTj7!2rhywWh!2wE(^o$X~ zHtP-q-j#9^`1#8_&eeRxfl0W~v{#GR!x^aR(6SM{7r!Ih%SII?_w?n50(w33|~5lHQ~6QvFajjz^st1OPB)f|C6XUma8d&rxXfG2FLh3)&5b@r>d(o85|} zML}p+v5wQtRMOXzk;EBI;S5=a8D&;EkrK{x9m=LKCA_ho`ZpK)o|JO_(xaUaqWixR zG;!{H2Q#;dF5~umBB;WG>VARSWd98h!SIcC*dgKT%UU`|8yofZ?C=(qV++&m>2NF+ zs<5W>aV2yC1?C%?Vu_RCmA_$9Qz0JLxBI!&()L`6q>&$K4#NK7ycrvt|6afa+zaS+ zNJ$P<#Umi&P`P3yzsCr-99Lq>zq9S2KM#9pZ*Yic!M(tFoQrg6d5=uDJ0B@`OAm}& zBaCyk?l|Conp}8K}nkLOgU+s2yUG>tjB?7Pb zW1DcUdmlrKP4E-<>^|1QKHZU;X{*06Njt!yl5AGW-?=<~^wU|3E+Fl@pny@N1rHpD_U?eMQLb+ z2^W^=!5=G+u#aH3UVr=65eL%*;I^p$mI$ZFa z9~5dOpop8-S!=$Rudt#+TlyD*2*Zq?CmrMa3X|{`hJNj*sjH2gSJuI${sSPSt1Wb7 ziC0g)Um=z!S~D6X#o6|ldiMo%i_i%SfYI_-9$k$gQ0gE@v;`y`1N`J2 zz1Q-rKqU{jy{fi3iViqe;ZNkL9frCBNZ)SI{6|DegkyzqfJNLd9-+2aEKpL3N&1rl z;IEC?)~@xmX7ZIg1JhfB@9HC0!aJN^zUdR|29{E8(cj6r7ZYi##vJhh$8Y!zg8Ar| z>baA7dVx^>9GGIDFpC8F?y5NQNE&k5UNW0JO>s!7-kN>+ukWI()G-h#H5l>VcR-62 zmAwttksKd1Pa_3d<8mBq;hEz|+NVu2e8^9%H9`LZi}QhG#aNyI1aAz)>%eBe z0VephQ{;L+FhqPH=?53Kg&yY(eGhUv0}9++#d}z?nT_qZfEmkJBflsi2ST2tKlNH& zZ6a=~Skbm~8d-`vb=#FUUP40(_KyF}?7rmqDR)#2#!xP$ce(l3q5o-fU3rV!$Ybyj z)*qU`Rkwl|;2pKbCZ6KY+)LyAI#&Q5?u834chF;ac)lwCEEJml&$*rncdF9!{`-*s z7X{#m!K(E3kE5 zBK##!?a;O=8usw=yG@>I(Y^`x`-5}=)Di#zizdCM|9RffH7b`W(&@~R zuURD;)7TfzlTt6ZEHuEY)uwhVhASlmFB4ri3O=8491+aohD$a9i(slwH$MQ?OH-BF zteOh5%7Wh$dwTF`ZCQMp24SVPS|cngv=V{!Mpu};VAL^(A!SmE$?1zr+pe07^HjwI zkhZ}P?=_9A!d^c+{EB6Gs>77yPu{NV56;{$32YQqHB_F?-S>jK<{{`%0yYok|7ABi z&uA9pdO_F9ZRZ!lPBJC+RBbIKK9+XJCWlf~nMKMC`-~t}_GI26;wFDf*EQ*lG#qdT zl?FwIuy;WpXdcynU>B3LEPzl;Yay@p+puWvGJ0yX$|5rD#kvE{2M%Qi1-o}zc|hiV z8!Wow;p`3Stpb0mI;Qw+%>xCS=qAD@BexoKaQznU+-6z6EvE^bx>niSZn=VNcgh4HE9@#l}jn##adoVcfraTFo*_m&2Tx#(O zK_@ag|8f(&l5t9r`7iD*1*2Yt+b&sA2nS__)7;q4{zps4o3B(@?v1{@i6ngfVZDbV zokst*r%4V-i98SdBMAU$sevV8Ps7R2<#`rD;hh{V4DE>z3z85n@Ha%vLG$eal~4C8 z$lw!8!z&<%gbT-Jgps1{rpyO`J}ow|k=la|s!u0Raf#&59y1kc;#d!7!M0IUL7= z4udtzLo2vhGqX;tG2MB+;=^nK-m=*FBA;bU2pjzQ_x7@2v0!wDa#5?FCu@`zZb%2vc(>M~@Pbs`zW$a#ZnlXsIX>N~OqhJYkRVATrO=vQstg6$RE z=S0r6BRY0JBqn@#qk2<+6EpM!-sp5Hed9GxIsgS)BA~v7)D5D`6!0v5H_o@%Xz+y| zYMfeJ>Z?$`P!EPi(W#Gs?Q;UENMM?hFMIT{7$-<(+O?JyfURW$ha-^8YdG}mD# zsc3^5>OupKQDowBOSeWw3^Jt_f<%_Fmp^vT8Gxb%c4)yt zYJ#t5N%E5~@kA7{jt`5aiF-SPn3`+x`WQ|h$CFBD%jlA``BTg{{W%2c8j>4g_Z2dz zYkyky6c)w{u7Kpn0ze>8EufG->Hv zHD1>nNo(DzI7o4eZSRdO0maAR4Q5q?KV^p8fWi8w zz!r&e0xVdxL)=H;oh#4f7=1@O0Xf%1|KZpa-n^FZjX?dLBCS%Tn1hn(^kVF<|4K%7agxl z1|x5_-sYns*-<~}CuM9+e7fr3(C8#!>P;jsVyjk*R?}A_ye)kE9=Zs2=kunSzge=MOZ!t?7(2L~!_! z(5Y-DWx~k38;}c)x;i4i24xZ~emK&-SJj`R$#`7Gf#e z`e6=`BEJDN(#pXYR^2n=nPUKh5^gw)Fb1MozycDDn8G(|43L3(SId$l%s>C;|DlYK zUi;Enbh1BN9{>`7pIi*_1Z+CkSLr*>sY!Ys7T{NLC+tf4uLPSD@(@_8)pQiFoo|pW z^l9jP4Qez=Y|VOGGJc<{@1{;qoip{|kP)JaD+7qPb`I2k0Jw%F3O6z(hPIjPnu~0C z0aEMRWiGjA$b(&cKr=l+qdNqNOGTcaj^N<^hal5vKt_qNkVtWigIoZZD#-W^0#;5a z@bVmCgQM)_RApj`g(cSv%w_Z`6654z7iK#qL|qBB@Z-0wel9C8H$xLg|_$@)ojSDG;sim}VxBG|2^ z?=z|vJ-LrJ&t17+^Fp@&0vJ6V?f^BU@D|L_h4XxWh=$LBglj;7yhBViT$xi3+bK_n z275UoM9{oJ(>wa!oqD#>u^^WtYyl`d?vst9rRDQj!&W9b(GbmlFXjN1$`vyRRX#)8qjItKyM0nik{h zphm64T%XPlTbS*kkCcxr#q+nRPbk~R?sNG~x*&j3>#`Fyl(NV{7eeMj_$;n9E2{rO z#WyE!t(h>;;!ok@9$$kKmKnKr?7pbKW_*@o@7If)Ofz4+K)}8OAps-}vP*qdY(&+@ zXh9rCezE&+f*;-{eAla=jza z(vqB+5fa+^ek}Bxif5?9lk{z2R_qqv)wW^|;wbi)m2pm~Dx|0UzkCGBBc$~?ux!lv z#eMU?``hMe)gskiIn$41(_%OiS+0ic11A@F8oFS2uQmQ&27u@MPi(M1HCJF8zD|Eg zu^I%eUz-&Ln;{-OWw8ZYW5lQ=Az?tRf~LXAbD-P_<1To)(knDRPPH=a)Xo4Ga@iT{ z8Y)cnCQP~!&zLCIz#PPKNpTMG&f5fBw{&snb8#2hRPNr15H3^XKY&iM+YnBeh%+;H zbBPp}{=D_bcB0GM?AMDh@ByNzEO3H3ta$CbdT>1>j#8Jn( zHYZ>WL=T!w=QjSJFNnku@ofmnc)`J$?icq#(i7C@4A-Ecmr|G|rdQ*;4ScuZrUeCk zjVNV6jRAx}?tM%|YbZ!Hjg4av>H6OdiALDJngLvXF0lgR-O;h}0fT}q0o6MTTiQn% z?Ame}0*R4~&@z^9lxT;|#-{+MdR*)ib(51Sgv;8+s5a5x68r{YZ|C@_E13@mL z$Y4n!#)1z-mZh<9i?iWUXQa<{$|MxhHr82K1mQRV!Wd^4)iPOpvALQm6Oj@zAO&(8 z0vOf&DaSzvD%Te&N$!h!XJzg`u*CQ@9)(#(10IvS{!Bc&gI=r}lG(whtH@^33*PgM z0|Y_695F&}!geN{!Nu({WoAD0<3s3K*r$$3(FxJlY`P8hV0KEUQjX(b0N)+sXv^13 zVDmHtXag|mD46m=g{P0Ox>@?^ED5dLJKyl6joaAyhmxnW#~5)e^9?bhV&1q2hKUE$ z@cuda{+5(&E^dL?cGeGXfs@paS8A?Dw)fF9 z>l8{B$Ix8sIjdAI#FrZNEz*`wpwA&w5Uy65=+F%18bmLW0QW zH?)nOh=gO13-Q>>t{RyY{gRWL#D*aGdgud%Cvn=tH8RG_T=gtq4G||#8jVqr0hNe7 z3KemWJ0`_Tp`ABg#r0>lM}-}^Zg-3>M}Baf>=a$kjnmk{A1D> zgDpa819;P8GT-t*S=y)6-nE)`Mu>mdkFNMbYc z!=?=GrCYrIMY0~4eenOy0MaD1zwpxsZcbkejH97D7VzXwkvwey)Bm^M( zwsB#H)Ka#h`%pfoUA#Dv4jv(~@-#X|CY(JD3zPa2kRJ@O%`zs3e??L3$ZH>?zWk{J zH(?9n6V*7g4z(`UsYddN3Q#O7nU>&l1C^CH>RcKJwXDEo>|OAwDROJhtc5_}wJA-N z4y_!iSoPasI2qUzgEXf(i8d6-q4MNq$=^;Ky-XyGV=E>Dsza%Yfh!5JGB%Y{(8Z|x zy?&m&_Zf?v1FQ@y`(o+oTM2x=a24DgSzXMyWsnJ}968b{?U5OJylUFzOo#6`z8YVH zaRF;uXr`K^@Z`Z&EAoHu1mqo|0h=Pg;=NoDJF_8mk@Cgj1G^y8vmX)pP`fD}i2n7& zpvA;OY*q9z53tXUo6X3_<@rt}iu}mkxX(H)pzD^=wH6u3I{b7+=(uWP&atBKrq{*6h*1-98x|lm13=JQ(Oco!EW7*6T>9 zZ;71t*xgJm=TTIWbc>`fk>Fm}&)?BKZXh7$P%n2aH5UdKDI(yHkr;$TRo2!;o~=H( z8K%}txcC8BUQkS0j}59=AoAP*%8DU*5iPPDcvBfage@&@U1AZjUpqr7ia+&o_f)`> zyS)=0pPUNk$0;3{Bev%JoO~V$RwstqMRtAHo{T-@m^+J!d`$nkt9>ko%+J+6PI4E! z+Wp%y76pSvdrj$;OvGaC4L8Zz0M`_lX_g(YwHR6za$aSb3P$uKRk+eJj;PZU&eQ(k zvWdz-*kZ*Dgg*5@1w_qs=y<7x)&1CK1Iq|}3SkeYl8H~M9fP3L&x!TA&pa#{`{0Mx z+*NdP@XUtIjJ5sLgFy97E~jn9GFjL!rNuU$yYB#D*nZ#17iY(_8j-a#Sovps$hQ24 zp{&&pnf2+c^4G2mRJd#X@*lRJjJ6N`#j-Rjmb-)-;7Rz8@flcu=1JLyi@nXBSY#iV zJNT>rvFw$`koQUBOolLMRN7t6EL;z!I99A11#>rLk1Pqnsb?imD=-HAM24xCjoXw9 zn*W@7;D2B$ugDZ*E7UdJ8^v*B3XC{0|_O&JZ+p%k)l22 z_sr}C@M=aJi4C>gPiN9hU(2Uk`1DJ1R6~qdR13#ngcuM5+ONv~*S35d2KuhaE(y1J ztW+m8S+3T#(Uprr%YeJ(M)|ltIh`-ht{@=4uc(g)&glkG49}sY3xM~_-(GwgVkciE zD>wz2>+zNGoR62$MTY3vjibTbS?e8L$3fGleh4sP`v7|!7F-yeKMxv-jGZ32DoDx` z`n`E8l2O=i{W!TT&uwk-h;;>~lG9~Ccy710P5Ar{?>DUy7UR%QJLOP$BfdMrDqsGmrfO!M;!~rFj z{ViHZ-0uh2$#VimFDJgEBN68Fk!y_~R6Cy3>)cM`epZcW1H&6P+4DB9Bqn(YPW(hD zM`#d~JDw@)&iEf!qdwCIoXStSFx-5;-r0B7O^Yv_WJe$-Hv$y*Ic_8e2)LLm zRNHZu20*{a)7_9?X$nkPIpK8R#VW){Laxv8mYCdqd@VwWWv&Bn(OM+ep~OPe0Gj+u zjMBWW;kb||e+VzrWG#-3SRC<+;fLf{gvxhMim($HNXW?(Fxdq(OCkwc zJhDA?7?lIFQ+r%P`=7pg@;auf}D6 zH13e)&k@0xP0pz)82}cB#E!39P#Og0j^=!#`}KoqB-3=GX9qa`(pbvbIOnQ^pBE0x!#bK`6Zm2?HNSd{auP}3 zZYpOUsv`QWMcmyO`-qGoKH1k8$FzW(988hav{A>nna8^>ZPxyBBd`%r-K?380WXqh z(~SQoQ(G(g61<=Q!mskVE?z2gGo!Sg9bR9ECU~1qZ$x-`_nCsZ6(Wrpi=qW>^xOwS zwzY{ai22|~kD0bW{{2jd6(TsPsTkIOi|GnA}HCtf1e#52emwv<5P3(+wv~byXvn^`EBEopI1$&Lluff zrxF%RaOK2!f3q%`U}8A#8}H9vFOmArgVf_2(>DvG=Wg(Ve11imCN!I#0OsGDCekTW zJC7K-st9Mkd92vkGG9~|Con&z3w!i%T&kDwT|Ow?NvJaagwMB_)P9&W3u+X+Y;ifP zN@8hR61_cW?YVE6oB1{nv{OrP^z8cBj zym0T_z22<27o##IYUrJ2V?Mw=_55Nrg@vaR{J`!pMcYd-py5{3R;%fpy9SWjjR`74 zF`OQ3-?yp1{7vzZBShOATvHbeK@hc6xPjaPLcb{ldZsGj& zSOG#JTHLyT>)MUapJv(V9=~WAYdm9HbC+20e5)occO*CzYJKN6>JoABLc<39&|8># zsy2Ftw#<#U?naW!?ViTg{GP`Jyz)b4G#_r`Cz*1Y5_ZbTE(r_vmoskB{z=waNs42U z$BmQZah$=yH>HflIzl~-uG{QgcV(lWzid8pe7ie;iDVF$5IXE1KlzQR=*43siX4C_ zVmI2iD$gA^Xy{+`2G35UGFEz*DJ-TS`)OJVp<7zfSw4iBVlC6*`swy9LwSTvgtBmp zts!(YNM7Q>xhJr)M2wVIYTC16CBzQk{v#p<$K z9FSo|o2$@$^EEnBEzicyqKK#0F`(b^e{U7cPX?=EvA-x^bhlA;b5X_#KN0)YATwj( zR?hO7UAAh1)E3~e=FWWaqEfm={%NZj?&ZFT;T5Uit5R?;oA;9R|Dz%ej$IIf5Y6#% zxGzaikGM4D2v|mQJ};-ibu;FUpzVHt?2@qa@;*?>3PqMDSuDk%`7(jCUCG<^ zuuM)`xHfjvpYeAt)ovXM(&Ci=>pAUl-30A8Y^iCh(do3WuO{knkY1XDzK{a5PGlW; zJXx+-74tIRBbx8SMa@o>KSe6{C!VY`^VbMPJo5`g&A1OP{aT$tO~YE(IqTm}m0`NS z{j{1(@L_p%+gsV9lCJ~Ncg)~u>k;{ZlRgprAZcqk!zzFcJSwulHa;>Ft7Kw?69vpj zK&Schc|^B(!iOHszXfYinc=UZ0Z&lbu1$rHaAJdr21y+dHal+V{S@k# zOVcWj(;y51hX@P74M%doqwQf*-KY#uU}PyT5GOViHxwq$vCYN8dLHvCWV-w0Esco^ z)4n95T?)C7Zt77=Tq0B0dh0UG(ynfST3ti`iZ|q|z zh!-tU{JZL&oI-_C@HPwF-{Vio9^RA-6^!#EI2>p*si_99DMgD@#4U^A;+2VBn#0`V zvOwYa))Pb3U$ciB4=K`K<@|jgL{TNk3or^sX2`!0kyjm_mOqr3rU1v!CZY}e7aAFh zZ1h3%WrDvNx9ZdG&1P-kYC z5w5#|i`8L{{(pfDvCID_P$cUMc5H+sn{}mW+Mr;8cUe^*6IcOjoAOfMZ6(O<2>A)0 zLA*3QBaQm?x3HEM*ef6Po9a)PsE80Ih)8d{n!g)Kn%-b6-~eh`a z&n)^zcC|j%@*A#2w>3GQN!7n&9^Ayx7xeeVOaYfIVka}LWO%T-g1iHr}(bGVDB49?MLDLPg57MkfNR97W z5c!Q3b+Lk{6fFjVYIKu{p*;b0#d`c(?fyC-hpfx8BTc#94E52jDFxHPBy}g`cxS$D z&&;A~WVoy0h0MSDTDPlIoP>R~-mf6pJ3nwI-$XzGR)=)vX2C@gOu z`hf{Wmf!L|?Md*+dr4hWQa--t8ZL`TOCI{t(9~`YL+rSD;CD zlX+ECz|^jvLjn%|!2xYF$7B@ZTE6 zKl9*|%t^+(W8gic{4)ibDKQ@S0aieXw$I58jlHE1N^G+0_mO|O`1PF&NsXE-oIc!~ z#Ku{oJy|468c`E{V5$5_)P{28Ap&X=x7Y#+#AK?Irh4!Pmj>|cc&7~l0yBehfE)S7 zFobOCtHl_*b9A6m@>H9pf=*-sC;7 z5%pEnOsebf6gkID4K{Hr8s?&-2|H68$iz^r`XV9kPT)?0DvlD1A;x!@DU2J{^oS9N zpe#GypQ(tY+0@OLCcUcz`HG;9giejh_d5R1+^_DLu68~RwI2b2rZS$O>jV6p+kG~h zC(UCr%xY{D@qQw;2+-dXe0f-0iuYeJ;{fGCha?vCKfLfi&8PC&Ql(yNVe`bhME;U2 zEv2jyUyQ;T`CLPJN17E;Qq!zX8YOK7PG02taNRYol>1zK_gQek%Y7v$Vo>>wV9unC zzIoHpxx98t+emj2jN<}zFeEzqG*bOW9%{ODWG|&`Y!ZZV~g9;6j=) zT}q8$;>8$HMd+&~Vm*NuiMGLbFo%IkuDM8?iWK@#itd0jDKW0H>UzXbQj zKJ>F*HZA=vGxfsd{~fvw9=8?J;e3_6vz(Z^m-m+7k~=VYqK5}#QY`ZPwN!lrTFB3$ zhx4jT)yMk4VfFwDLv(VX29RoY#NmO;kIT2f_G3dw%8@X{RqRLLU_WK~V&DGw)zxx9 zJ9?nI5f4T96Dot$#@$q&GkJ-Zt$L_Naz}ST$wE%zifI=Ud`BJqdQ(e*j~z*4>mt^8b`rp1G|>z4#twwYLa*4u*?)b>EeW z|6*gwm`3w&5`q9Yw=_|~0T9IZ2Yz4v-EB!QnyqA(R#@mgg2Ghv@Zk8;PUuYWyQ6QQoCwv?H)M6IrJ6l_Y-@7A2gA;F1P=og&tCxwaK4efC z628)_$UFgeYiM4V39j`(B*ow}IN_)pP-w?M+7r~lx2)nt`>#yq2CB78k|2|90F)$` z!F3wC#Xobn0|pJ%(!xGeA5dM#4nd=LJpjKKYjFhHL#4`ZLhE5j#WYbF^q?ONoL$^y zN=lu1-SaMUHjo&YKv<_n(O=GQXyHO)Sklmy@(8Fvhd^6)VKix50>puuSFo!kjl`MjfyeeYl`-y!Qc(18!mcnbg$dI5SQXfQD_0NeZD>T z5Y54=f8s;Y56~kHLK~1VDv-kx5pLp+#6E^ryW8Ow&kmwHa6RE$_C+mS-G601LYZB| z{0XyCUff_H`8fy;-UDn-tF>rp$bL^G3;>D9d_&f?m!hMuBG~ob9zeQAU=vN7l@Vl4 zx_bR<|DDT>1ii3Tm-;(rwU7^@1E42%VC_SmKrxxixqmY zw$0&ik7#%Rnb*UPE0L@Qrb@6p9mebjmrxT?pK|A9$35gTc27u0oH4BgtQ<8T&jUSsH({23HiUQUtvw$ zyRu#|C()jMLO%$w7ijq1fXRZozX-#U&A{*6?2KZ%zNWfGP3gA_Lt`0fya?FQQ+Yp0?~ptsQH1@4#xatO53W` zB0nXcNo|4E+L`7{{Gr$3$4Qq{;E`wn7{KIvS)EbOBiJOhK~$=A(|;!qWHJUQ0&EAN zRSS~%0jy}%OP4g@C@lZ39Zi9=m$s#pWU-^4tw|Q;0!&UWDBJ{j`~r$d0hli}&#a$b z5Ws+>te!5Ru;$SFvWdjH;`KTSi!;@gT=L762WqOz0G6I@)y-NK;|Y@lEd)MjZG~K? zbu(sr`SSmhb3oYZ`^Iq25^sS9hN~u0HwuGbc7Ul`i!Ykk6x@alE((>plmBD-X)yQz z#8pphUWh>rnPl4caOXOPuv&JmG#y#%pYMB9`aXYUXvOHU3UU1qXS&aPm4yG_GU+i4 zRNQR4C+Xjx%@}l$eTiED)zqtokrT z?>3@0I~4gLJb^+8KM3HhLR3JdxJUwI?$RP)sN+h#TpX>hm*RS${Fe0`qc zL}hg7Gah@Tw-XiM^F@Tr6=))AaQs*{gC#deA}dHt+qN!D56B<=lMO~@VQcdhVh+pr z1HI^Q8X${Kl4=}1p2J;2z%5AQz* ztb=LYm5K_~kB+(P0PhuquWb&7V%S8{Wz20WO6m#R$Jmd7=j2B5rq3;&VVjj|G4 zGRgG#dPu_8QTJPgXvW;P3L%hBXDN^>Gb5CTRb-!c0Tk8MTK@2zrrnS8lIZr8x~$S* z?w?K9V8=~7h5~vN=xXQY@P)l4gw|ht2-XA_c3sC-O6*6LIHzUVrK!5w#p8DH=t$Ux zy>gzyFkUYMg7CO|5lbzb7QB@J%Fd) zLx2}lQu#Q&v;no)86xs^to8F%C)(#yFON2nHCpU9#wmpU4ulL(|HldwZu9>IPD z0|fiSr7O4fZ=^5f(97~$?)DFjU^e`0-!ZUBl`E`p!l*G3OG-A1LO+Ap8`p7c>&_eG zKUFeaPB_GnZ|Q?XaQPI5`QvY_PsbM$@0*i?^80F(fm+%#FqW*x#xg;UcZZQ9w;J1q z<_}bLY`=knalZm8nbiZFZ!Rfh=BXbMj=pMah4*Nl*ui%tpRt78j z9E6f9p$x4lB&I_-@(A|%Uw_|1$Vh#G#8skh8Tsx^+Zq%f0kGAOc68?iLcXed2Ve+| zV5yQ?{Dwllcyhw(wnY(jnQO?5Ip?w1BBu1BVn=b39?^W!YtU zk~#*nm=?q90b-llrTFF=^rc9=g^06?B`b}e*C^faNvaI?l4XLooD3*_2TDxx4gKgI`2P)?WwM3#L=2N9Z$=cpU(7nR0DW&z5$u3yR8EAA9zDPM>Si>?420Iy#82g+HnjFj>X(CpYRu}{ zpI^`qU7*3Z@PvdpK z+-WPc`~>F7Eg;kzB02sZQaOBQ_*dzSxJYO43Tdn-Ub^1?i(Rb|b}QYf%m8eb25qSlKWB`zpLLrrJFqnx1Lj4)qP zO8m#JM>xHHoWdh=t*HaAcCvM47+6M?o|=fGj#mD&?o*IEyt%`2OXBr}aagb{41mOY^{$nO(mIPjA=(6KYw3z$@%Fn%n=^w+$#%qh2opz*c1Q z5xD6SKUaL%)e`fc)3g8iaFSP5FOzG>q9B9*;~a>J`g)v0D6qf&ZJ@~wvg%yoarRL` zr;q+nUTE_uJ3--06jTfI#T|NX2@cvaoIwpNC_I#BgK9{n<);vlQVei&{%meHqZk79 z0X6mtzJ`uX?aMh))9{G_tGRME%8_Tu2L52=u|s}}nKqd0|F@1sTk1>xMZ<%|YcNu0 z77$a!{vB!>@0U1Tsib1c*|-Cch4X~?nRbK;9w=lVo{FJ z`@*~jWbA;RFl;}QrFY!|GTLukz0PHI^In$a4Xfe6N@fR2oW_kxP>u+~B;`bNE{O(% z|Gm6O`FG#uhkU!R$pJz-@N{@&x>I{77{A7_fxa9M%yHm(H;}w)NC`XQLZw%+x`yk+ zgsL1vP3-u3+E6Z}&-^x_W`e69!1Xf>CK7u4Thkarx7xtUxt;=*IMSD;FCR|aC7AZP zf|w;o=*1Ov(Dy2YCm3B!y_Pb?#Ekb0DkbJ1LVT{w{H#D2JNBnBn7CFt$h(;O5`#HA z^YHr~+qZXojT2LfUupra#Ff)*4hz1&zieG9@C0#zRb&|$vEBh-BA#CIm399VFp$O$ zBFHpjs$;f2+$2bTnO5Gvc<1UyAoBA^fN$Hwc9o~{_D0@5CkbDA1Sn&*BS!hQvjTuD z_z39!Q;JItfZq}ttR-$H#(g^ai$@3jlfOmHbcK1DHXM4e(!kmZh}Y6OBgSjMmz@At zGK({BAF`hrMGZIgI{0 z@X0l|JIHN=A%FES?`H2@B`EF0ziy(y7Kzo~0A3E~EsGLgv6AB+=mOx{V86*Tdys2n zgK$IFeyFUoPzd#dGa?{f?&CrGN%ZSwK%xDbo-!_@^a9f8CfNs@pl?D1S1#GgflYr7Rv8{EUQMzwTfRFny%0(_OSQ;PwJZYH)&-QP=b(<5yHP#e zpx+NJEwoXZ-bC~;FjQz7vo8VHK5dTmX>^GDLWr-(Fg8``Et}OD1J*V!h!Qnb#b#cb?IfGtEZ!s{Y^uX&$@aCHa;;C@>5W->s8R zD)L&6i~Yd3_0DmS$4qg)7o&n+^^6&oSqZ8{>(*4PsQ*I45e6*(=TWPKMiBsE>19Op z77vd5>wuT|x9AJGJJ8y|`D?vv_kvwqym|?gk`3mOn_^G+#izL~3MEj>W`YZ+J>YPQ zxV$@%Yi+Hq=YiS2CsxT@*mO^jg!KXjQ#bAW-l5kpGTJRdi3z^a>4UHik*uExdw2Zj zx$=2Xeet}1)j|qIxv-CU-XMP_n-_HsZ==nn1LFnZFlE-m;lD$!Eh-G^3SnW#pi+wI z7=mt6Os2q|cDW@9XpQVNf(p%9z34_XHHvB{(Nv`GxQPbzVSYJ((u5%U_X~I#r^P-vC zP>pgV0CaP=X5N1hED`$95bqJjZ3 zM${lf_pn0^LGqmlyuVnyere0iF4TZg+65J!!Y1y5K4lpY!rx&4PG*T5I7LYrHbLEh zQkJu-=>-mk4ImFOp4Arn?*Wncul?NmtBON#yxrz{{z3me$Sowf8D} zYH26j0v7lMBLz0>-5!~Ya1&|aH6S8iVU7_$*#Pt5jftvu5{Qd_t8a{#zx3MdcyC!u zQj3ax)Xh1#I^KM{(Zm(`LTqIQM1OD-vbqohymQu9tD?I^rgu<}3vlZB-YJ1bn?2Bu zhUFyeLEc;MM8}NHRw3|pimAiO$aMZL_QCS6rNsDBN9~@|kLqF|k_+8H+p-J4INSD` zX>(zNsl;+Ir@5Vv1fd%Gpdd$`ZLD;p2n!9oB|atAgd>_rG`$A_(jqF<&;>u4!{?RL z6F`a*1+K$(PvcJC{>!(D9rpTR_kz3+?DqTACTIN`ch=k42h^YaJku@Kv+ZQ-c=*EW zPptz#Hwx+Kfs=BdxEF%Y5ZHl{(f2b$DH)gb&z*i52XNzbn3?1R__Ml-Mq%}wKcJ$;Qy>eO|r z!L4p%PGYrR2|5C+T?JZqg>*5gRww^X1JUil0|Uyi=P&TSOWe*npm(r7~`I9Th_A&B_oF zuoR2cY%n%%=R`nbioo+ihim)#?$o1vXQ*vkHm3#{jyS08A48<11-dYJ^H{1jg5&^c z7tO6`B;=S`-Cd&YkS$@*mfM|}FWfbfn;vNbWmG$qxPT1**7jFDHRWRf1@dNdzo_x(fY=+r7-mJsA3% z;@!O*vDjuE(572l^L7gWnWB-N5#7J2%q5h46;!Lq&yCSp+;s(l41K_8N{(d0UDdx} z8W#`U5rBWG6i&9|ImU;e%OVsl_crz^-zrq4Wc~yn<=uy!TJuZLc|#%oa!Ff`s&E8k zYJ?Tg8m{$(Z~U!LAo@1=q?!zV2_}5J0;8De5T-p`19;eLd}xj(68G$l@qw&IpQ48` zepSAEq@tiy^IZ}38sIhD$leJxb*-R}Rl(5!vLa&&am+XZ%mwY(BNF^Vq_y&7ZFCi5 zJwv4lK%kHaYRX%rN2G0CC8EYfcpZY}?tfG7G7A2Uo}J3_$M{TIBCeGYZ5J)iFNM!8 z#QJp`2k46Rs&0nW+3AQNHWLUpkdcHwP$&JpDRTq$oGd?0meK%_ua+G?!mVFq&oN1+ zf1zcUBjCzZPeh+M)(HG8w|v+{1DBeNDGZ1DaFc%teL2!8v0iD{Wz1)2J4k^7Rx50?f1S zC1=9?By4BnMNv*KrFVYEje`0w^#gqMMX$>}=AcfQ7b){1k=S!U-x{-wBaCO(Ur_~k3!`%;|B9~j)7Ukv1jo{v-TdTP zovYo+Pg~6Z1Jtm|Kv3DkRjh4vTFE$vIc7xK?U%PQPLVrLD6tG=ELa#YKZ+K@IutT5 zd)=W<&f5!5;8t`jltHB5_4IyRs=X|0U2qPwNCLe`yUzrj_y8>ViR5BcME0K06qi_s znYtwNT6r{_(D4#w?=$zH#pyivS1`2ALO;fB?nHaVOKfTqNSEuE!3E<5%BHyFIU&B``+Q9bB<}@NFfn{^>e?KTWT8Kp&6*H zdkus!?sV&;dADx~JnZD~=GtFJW1;N=CCo2vTDzN?G47-l0LI7dVH}u+F%% z?jQ0&&k{HD;@?S9D`f=W-@J`C0d+O1>PLn}H5F7UOIh5Ht+V9rNkCmb0)MdOQQIgWoZ;k=GGCIvq-q`=0_Ph8^SkfUR zI4(WF+uW1rWD?wD{5#vs#>&*!<+tU>P7&W2bm4`W4*IVxdMAB4p62|icci*jtZywH z#z%FD;D9Qk*BA60^K$%*UA*;)j|77vZNgY#;(!qz2|KIbo-Qb`4+UlEk1&W%bYIB* zT`D4?E+C&3Wz%~jxLAuYPp1?p#}!tO&Ez~cG*Ix_JS!moSoFb@^F%ul2^j^FQoA%f zq4G?KFR4&`rXFXqsUP`PjGt9t0q zPN&mAoN=wfaO+@$mg~&~FGQj>a}jSnM=z!Si&=A;pK6i#neLlJek{o{j&lROC9%fnj)m1z@A6!PY{Y|cxl-^!yH}|80JJrce8L4oFB~b#2 za{%H^OsGBp*@=$T0vn%H_zIq-p!y6lOcTeKfyu(pVW1esBXD!4ynR}xyS#&$N#A@? zVinTrxREtU`Pgvc)TVq9f5H)CTOv^;s|lNffLR?6g(xxIMRC9d=Xa#wMb(Ab+3ZPL zk0})=-xrTL`mx3jp0w8-RPUrfKX3oTpOe}?XI}NOX9=bzpI@jYf6T2c4XG1_mzY&= z$|~=*t3)xyId{AucMX(egc7V8;wXD1Lgl{>Kg_S}H6X>x8b6K3U5NTVD*UuD*mM7E z6^uy@pc}QAD;`eN3Ro|d<rY&70qLcY(PQ2zs?1toV z@ob}4QO~+$HhuS*DJa6}@w-IXx=UnMw#3cO*>f&`o(I?ZkwWkD}LB}ED%J7X?jzxjgs!NsBNaN(e+gPc^ zh4{Y8#eBB;SMEkTiYDk!`EQ?}^Zhwt`y~+l({{_$^G5vG-KHyY>W|am z#ELbs&uR^3M9m0<=ORVLTj?H_Wk}HzS?RwQ6SWz^Qli`qtQ_7&eos7IAp2k3yQK0} zuXE|%L#a+b;Hz=px}jAm`;9OqAt(B0)eWN=_>hzKqxBOvD$y68D^s*ER6K!=(!pon zk5nVx4!r*;cu&4`fz(XgUeu~qHFa*5hmQci!wJpW0v|u@@Un2ilS@TJssAkr~ zL$JuZlbIq9)LVfCOl?7B9mA^vXc+p|m0g~HLf0e8yF5ZYNVxXM zU0P0YG@GfnIEM??Y_Gqf{X&Rk3%s($u75|crX&0Oy`i!8x85LYJ?BZahqWWz{%IA8 zcx>qkKF&nFZfP6!K`6$b`3pc%=Cho%UgF1{svzY(jj_SVVy6dD{y;>uZ%pE2~BRAyFUwFPfn|Nlr=fE~j-R-DyzWIQdXYySUtG?@M8&u=x!E=PL%z0tWnNc#S z3o_e)nDe_32Y2P7naxnGo5q)Ot$ifYmQaw$h=={J6qj|`oA)1>@#!^}mm`fgMALGP zPcGg8f~?E2?KE6WnQ4&H)y%u(O8loM5e@lsY7+ecn%)MqOh5vpe)Y+DMlk5*r^GW~ z%dM8R%guYl%=l%w2O)!cTsp6|Z$k5EG(GPaJCS63P(LlL7nDr0zDVd()KAEcGju)B zGP0tyqmYSMMgp149lU2?`dMk%O_rmZ@=~L$;z{%>pSXYLxJ8GauYc^y4(@jhk6cl$ zb8uAWb2~CH0@L!426#HHNc?Ok?Z3mNH6PV7f0v>GoTE-FEx*we{2#I&-;6 z$c2rYN$cX9c4p=p#$#;)gTC}UjTo{+nlx#-)+r{cPX||ndM!4vT#fgALb~QL8ee-) ztnoCmMIzpcd-75o)@Cj)u-g7eN?Rj1WPZzQ@wPmz^SKArm^d(mFC&Hc&3r6Lq>zKv3v|C(E&$+(Tme(GQ-6s>gwyG-*|_>|lLM3u8-B@h1hajJ_v;){2qtc6dwjPA1WzMa2t6Ye<5S(N}Irf`;oG!wDdSbZ|yKXXwfT z*MrQ7%jXvRojAKl(#oMwx`JE!>+JJ_o9WbJ%}hG?5I_$L6C46@P3D&F;%t$yd$JtLhaR#N#_b^CX*IEscKFKty) z++Ya@QDD^|GtEI({J2)gg-UV>J<Mg^n;JU7DK?J0vySux)k?sZ&1VOq>knZm8 zPANfX>6naZce|vaYE}u*~%J4 zi6GC(VzOq?wxQPg43}oBL4D(lmzJE>g=X)yqdWeVsCTu_wIJ`mV@EIGRrd5SN|<$h z!mk{Ce_`w(*|+Vfkwy<{V;hM=H*M^X3#~`3C9O`11q;elgXZ&jRTA@C<<&u9`pZ0M z*~!w|Gh9VaV0PIsbHSBHxmAIh!Z<$7Yt0LtnN@4oe#KZ}`SU^zp;@~rrofwyqM-E` z8tS(R|E`mhPgl&~4L$_XHt_5>G)PszA|me|a*Oz34jw#*sI(toH0tzWkL!T%)bnFt zRX1van8Fqq>F*ULDAn~OjeyQDgGP5fZx5BuRyye7Is$JnqEIz}c=(f{_wTaW1``YT zS2nCh8RhtTEJ0q#

    qG+*kz#WwI-X1WZQ$`e+}PDB&9T>XLFgX}{Bhnhu=96SOc2 z`U?s=#Y{xxK~vG<$Z4fo7e;Y*a)Dl&@!Z?4jCVMjU5QNobIGpmgc)3tgaTKN8n&yH zrACWQ{;Q9P;n&lp!micCBaUH%zO^{}2&2HpZb?yIiT%$8lqAfwU^ykfDO0FvR1A?*~WXr2@ZJUZJV74n$*4 z=pO$#E|ngiJOg#4RRl}7lF**34Qprf)%ydigZbGMjruX3w;n5^F~YYN`sZVK{)^W2 zQSOY8^@?*;u_HsDX#P9Z^5F-T!{1;thv_>k$UcCceH+C_=VQm*2h&<_=x$QzMktp z*odZ!>q9QdyqWqgUdT#tG-v;Ns5enN6s94DLTP(7n(-W+Q8me3DNOo2~+{lP9H_;lrCH=zsM@q zh-6@LqD3z|^YPsvBkoI|h%- z|D&@%p@oW$NA+?vyn&FuU{K;d)3tLW8Cm5q(Y3l#?WEf8WDCp1M{9df@bX^cCXD4ry3h>%BDc^r(e1UCVe+n} zBiq4g9!cZDTZ`R}R8VA4zdIq$x5%G@@rTfb9+BGYX$$Fb&q8%r&*w*=Jc8K z1E8$O|L_HmaFp%B0k0AnjA5lpUqc;?e(|`q<{it##t95?Ac5yEGC|qtviGTVVb8Xfo;p?%CHCM@iPeRr&}K7h^J@h*Q{3(sWs7!9@uER#4!`DTJid zS(Ln4Ur`y?#xGK$q9~kJs>eu`idW7Ad|;TVqgiOASv31S2l~QrHBl+@l7u@VhN;vAHV%Z&FMb6r>7xbOPz_aYDb|})vQ06 zGR?SzHZK9@{?Uwy^>_TY;~G4IL+!Bow;qO{uDg*V{b zq=aMGD4l87AvrhDdjqt>gkth0%A41I-^A7c#2hI){^aik+%^sGpkI72{|dtfQTd7P zl9PUkJn~6R>2qBNXAG-nUUTUJ;1adn4F>BXvTb&|_I)^oE~Q&}O$Iv2j;+n-W}#t( z?=BKR6_#ng^p=L$X9Jt_?;N$l*2S~=G7!) z`ZXA>E`cwdKxjutAsyiJ4tqcBEFPo-r6$#X5m8QYRb9EjSq8ahN~408BkdP_OY?)2 zlLg6705i`l{R3DuU1!pg+ikW*JL=?P%y0*9u4NZMZ&0;t0o;X<-+2$>B?EzA@?_jv zhLPpJ8{-RQbkOIc?lC)WY&)KA{Qs8d0f4M}ko(?DP3k#)T;Fxx4Cqb{>2FbIBf$QL zs@gt$S>e+Z7N$nM&)l_`_U*2TxiJp?rN$lHmS{gz6kD4QgANp#Tf#bfTWKX9$nBpN z5=)E(bnKR@SDm@{mRsdfA!h*Ca^IcLfmM`^1@M%pc0*1Jz$E|vFa^o2MjTGHcmM9cEP}0Up7p1bhoH!M587cuH*wMlR+!3P zcJ8!1k%K^z*P|wCEb@ZKC&ng9X)&%I+)cj;nG(L+rLl8=4t9a9oT?c{4F@zx_o+bU zOvO_}Jx2}8{c$mc`#6|@WDqdwaT_9Yp6JvUF2{F(zY$o`<(>qakpBc?e83uK7*4NL&6r2kO-8pQ=O69}&^XCCAx^89%`h zFo;foY-q+x^Smx`HlQ?O=P+ zNqQ4<6jDNZc&GQL69R5a++8>3@cJvrUgPd_of+4w-vGZiIq}*^=dnZh@e>RsUa}(nC4li&J?em-eh}Udg7cQol=wcEM^k(Q zSOVYM1=R?GpS1{|m+Ylr`4pOzJo-L>sd(s@r%?_Hfkcn%Oo{P9l>e^;{3hnB{EE5G zVv5*eOs_`GQc-_WuZrc0musKaI`O}f`>%u{5Y1;3qWNs{h&p!|Lnhw$^~MM7nx;~%tOX2J4Qd`gex?G@JYDupU>8aQUo{q;05%~F)vqy+~CJJiJu6?;mH*x zzGIImO$(L`h-qSE%y4~zQ-lKGEC2&*8t6If!5?z-)H}hHf0|>S)Z(A?ZSntHdH~Bz zBW3SG+%s2km0r`x`vC!YS9kslDjSkTg@5df5ymsZ!9i4QVXqG6|Fo?feZArb#Wy5VWgu8zZkn4(j(C40GNI**?)ul&ER=628FO4u=>d>{OYOtzT~-B(S)_H zgZYF5b&eoeRn=o9S}K;fim9$}zFNNe7-J7HtvXV?>}1=Vx4ZwUTgMxeTFj(PE>;|s3zy|)WP&0#=3h`So&T6Q803FHJ+ksS2OYmtb zpBGF$Q)gI{yAJ@=kkq{h$xw6u-TMFanK8a^2>rV<^cd17W;WZNKG{-EC>SH`vB12j z$`H*stS_fKERWKHnBp#RLZjymJzKEkN>Ly*vfe3r=} zqm~U##EeB0C{vPEI67T?3p{3mHgf!rRQ!BQ;cOv#LllGzWs9fmUWRVjcIGel%kkVi zofm@+&s@=1S=+}AHwgjgFB5{bAkxxxem_MqkKjV-$At>v2i?sR;mZtb_u6gpnibE( z!2y~N4OWy2AsrU2HHATNg!ZBc1tU^Qds_(FLgvK%z{VCG)M+qdKe|Iqt4dDX;66nD zDL4L3&fMCuT1Z@eH_SnZ`_2IvDkj4KCT7!4peC_3R{gvTa48^SN)iEd`B^Q zsWG?>PdJ<=MLqcV>(X=s;4MvuTLFA`_%31LH&W03V1H5Wx9p{ieSNQyNb(@{)+% zG*ltL+chM|a3>~mj5|PT7Z6>FG|6`mvGgHwa9cQ1&R!eS%n`A7di%l^;1t6PJis9| zPN@4pnK$%<=9HC#B7u&c6-a#Zu_KveDITVqSuR>m%`;Lyec2HMWXPY2nOq??7OI6g zjNmMzKAmM=Nt$jGs5)!*=5H7Uf$^{>B^JI(Xw<4)Q@8@c|9_vv($W#*Hsr%fUoaRQ z8fDP_X;QaC4bgG?0wZ(C0l9PTlF%9`5bBkiD z3vH`H^^QKBSKBwGwmb0?Y8F$2RJD?^klMhGL9K=%=@$pe;Z?t8WX+$-OgKl==Kz&O zKQ#a-iCsMoIa<;r-%V^SPKIZ#{%l84q(UadyCIs(1rKk`qM}yjZjnzD!Ida=V;3cR z-Q)<^OydzZ;E=u7{LukxiPiC}q zvXkKNgg|sEqg9S$+7ko}5Nvjr=Mmm>gMp!oANrAqr>@AAbWKYnBcw0c`)~g0 z1}yY4Dz`FJ0d_H^;S6U0wL|LNf#u^>y2Fqgu;S;YiG%To2*&FN_Ndvi0KB9!|R@aI&owAKufPQ05}hVR;QC6r5+>{))j>N^}^DLyfG|{~a4@@b*beSoRWFmpvb%iw+u{non22a9zoZGD0=b&eB}H z;)f#aGJwPTF~5iTfv}6o{CrEUs}#X{NGP$~h3dbQph?^BWSvl5tbD~1p2saRKCk`q z;U9BQqok(@e;K9C$4ORdnv#H`BJ-wusaXsrF6$5+YvLqV9Vj9vgT z@w(i0N#3kSO7;j+6v8Ck04LT-TvD^5RGr>-=`K^_9Jv!!b)%>Lpneryhe-$TQ4$0r$A+GDg8yL@?k&Bbw-djC@1oVA!+_QJPDvNa3)U_3JP1~wtf&7D0HYhc)|_9n2rFnAAlK8z1`CN|Ncq|mm` zJ!S|Li#SkmYxuao!*@et42L-m_s;;gNnHbBKtII9mo7fT!`J@=SC^sp^-N}YxE?^2 zhQpO~QY+OI&cKEpNX(d&6<5ufvd?`&8RoRpYnR}1@Dr3``@yYvtqtK+djW2~ylRf( z!bypHmu2^P8wq3PdgoZH^25u~b$yQD1DO1t%%W&5eh29hljR)WTlTFpv_7aN6W7+~}*n=#8{qFN*HBEzadc(u3dkc)Pq*=h#4h1!otY=vU z%uR0L+d-}f=w0AS@?0+iyxl#TA0#l)bLkr=X*voxvTh&)-kWA@V_6w;(wYU(6)(|F zfo97!8^^^GqjD2a3%~E59(P5)t7~f5Il-vK=KynQdubN1R|Q?y_*ZU6eSxGH{_6v+ z+v+rY$A*9rKpA9+Tr^Y{|9e;K`9g9%T6(MW1pzw*tVd26zT&G0uxR)%6uOWqAbIxIo%>Klon#GMXhLW}p zE)}xO+R9&kc4O^shXG9L6+AK5TlNr$djhy()LrRa(V>-FA9_IhBk~~1U0|#UJ9^x|0)9_N#+{(l2GF*8)bG+^b_x?%$(wA;$`$O{gDPu|G{1)Ti z{Psl}Y?ZVKh}Z?7(YWw+L$mL^?O>oo?o!S>U=mh4ti3P7A)kC&Upxk{A2(IU0C~g_ z$i#Ruk>$XbD)s?BJfn|(Z_@JwonfK5C_XZNl(x)tAaA!CMV0Oq&8y*+Imc&kdt7Jy?g8}dbzg|0Ou8iL|orA3#JnjPF%ak&1d^sp;9 z;9ib}!_pGUB(STc$H!IBxbGT_q@>uH(a^U(&JW*R zZe*P<9<+nz9i!E0G9pxU*=n*)H1N~*5PiF(YV8MxEf$fh+B{EWyjz&lOj}=B>QvJ; zh1aL6_s?8DwL2L;9#hEbgHdF$>B@BwQ&B1F2UvlC0y@veO$4fZ)Q&FnuU-}}_~XBI z>$V{uRuTpYdD^*e%ST#uf>O(h*o+*xL6 zX`NSXn|0wVha0xsg;u3ARXg(s)BQ@`XH{?%dmQiWytik^JD$D%1-ZWa)4Yeg!zR2x zA^Y9^IUHem3p^9gV20RsCY70(L3txI&o%fH+ra)es7sZWjExh4cypTnio0wgB;=zJ zAq)6Z8zhd01a`};mG<6cnvy`Ta(0N@zz4^zrq^BC30bn!$i%ok4ff!jCb-lmbRkWy z?q7yV8wwD(vhkM9$>*woWX2A|$6U~^fUq<{2x1dQas)4-AwK zL{Rf{hU_R~LCJ-gB2vm^q5tJS&|fF&To2~7SU6pm+E^PZtzsy0s!BY&Gw>WmVvXro z-l`GicPJu0Q@Ack@Z1kep^8$I|^`f(g*Ui82Blx6Pl+7kVXkT$5Co?9N zSqgIu6#s^+_dXTJ#ruSZZ?io+{`@c##sL#dE(MFpU{t_lr^*>*I{| z$H__i*Pb6_kSemA3)iPQ>|NgO( zKdb}vqw5I~da@fEBod_+Z4an{e|xJnQ>Nop>ID4Z*sHP8q~YU zo0ucUGNlH`nt++M3o`$&-0)U>$TeEVIAu8Tdrw@3;-l2}FL}`?;E-c!j?8k7WM%}J}TK)zn8S+H*72h+o$ia=9P@WDVI2fRAKlfNDJ zd9I)ShZPO0KIJ{_)B=+)vIt2qgt-D&{Ji-Nn*+y+*QMfQEhMwOdbBG^CmBwHk4a0% z_)W*Lli9;$JTN4q=P*mZ7AgXX&@{8`YaSN6pFJRP_tK7O}AuOQ3d2=9puS+Sj5BrZw#)209E#8 zNM)SRuUC>;vk7rXgE*)g1f0_c^$8>bZKMuhSUCkc?$jGY z+YdO9rFap3k)$A-#{vcBK+JczzT?pTs1&ZdlmL>u2b7ytt|($IBJku(N>qzD`CpNG ziKg40=zI#3waF@Y_k`Vmr3j`@{-TtBjzSsCvhB*tis?n1X3>b=0&}rwF7S3Q)9N|U z!7%c6w}dzU=4(k*5Jg23@! z=&bWKBv}yg9+Dt2C_&!70%W*J|DF7>|2hB=-iP!N|G&%^XuofgqNOHZiiUAX@h^Z0 zW{!HABSz{@gb@i4DBlB%>RvzqlcQQ=j6vi0`D~t<1f)P&!_{$7!1$xfGqXJzY0}39 ziM(*72Dgp)i_BBeF=W6AqF?|re+kxjCPUFEnZesbl4PF!>&E3wXizP9U=+BjKoY_6 zXU#dt=l732aS%L@49YF-Rie$KN0oFGro- zxS>v9yDeMSzqGd-vU840KWK29LBsT8z|Bt(_9m1f8g+$?hhWhMsHZ@nHwJg2E!*|cuecW$ zVV=LK2C+q3p5P(FNcO>hy_!eu$KC3oE=W@CLs2iKtz}fG7|Ib9c!!Bet$93Cii9Nh`# zuZ#o=Sy~?)da*WG{+1Ok90wpRH0Q}2BRnr##_f*VP-S+ zL^F76IFrF{-33%ZvZ5|1T9JbYY|dIwo0HBV3brn;< z+jIQH)-v9dzN{86CgY$Ea$b|q*de&EB@WS;zv!tun2&CZm$0p)Sst#X^jZ@fz$vAB8y&Ca`q}C0nsLG$9kRK3)f?1;i;h}z@WG52xhysaJZL+`_n}O`z>y>ZTCuXF2p8bBZ3oH+ zdrNph!@QQcQ$_s?=(8ndcDT?&-ug`z$YkW~Pzuk!^#3Acy`7KZS-A!G!FZr#o+0SS z5bWNLwQg{LjrhqL0Mm=)0D~aY)kCY6josyd2zeM?iBB^XP`B@S;hu&S*RIa}GfN^U@T{ zE-C6;<7~cRL$!eik~-9~9!yZFZC|@Z7bY_2Jfj3opU{lIJ3RKOa8NAJ>K0ItR4o{2 z5AdR9!o4Sih2BPGHN~Q$XkJfDVE7vSYZSB*hFXQXG*RY~Whe*Dy0^hA;3}}7g@hHb z)BFrBM>2bn@IBlHX#s(qLO_X*HTx+N88ZdH1VT91-uc`Hv>N|G*Lh)nkGZN)(%Q3an9`|E*>lNT@L8qbe3^YC~%erYFVOc4W_dzU8`%uM{|6~1#xvZ7K( zVo33S`#9e7a^i6bnd^!}k@lPi(C5Ra^`mPBIwp4_cbA?4*V+cw`x(%3Lz21*%J(~C zfoUL3P`OtJ;)7zZe(X4S`6uF5wJ6+v?9hrd)$}}N+5%Z6>mY5v>SxRPgM-hCG=ol`(S?u3C-$2&B zzF3UK^rhFf*40AR!XZ|SLj9a5CI@T)Ar=fGb1Rj?SAkwtobTy|LA}`eBbaXv=A|_D zgdtb}L?^?+rkQTr3jg&T;yTY^E;YX-iR45R53aY*!^L^X$NrS|#tnPsca#G##Sj7C z|9dBm(|CF2vW`%@OL9+dkkI+K!_z_Md5LIapiwVAMUYRqXd(LRLHIFPBimkl^zF&W z^-3RW6Z1MGC=Gna``~$&+U6p~kwo5kiQb9J{$aRBAlpIe+}$*DVvN@!5A)-m@c#bQ z>5XXoeV9tW(`dl~3lFu5o*mMfwJ=c3N(DJ_UqepZP5wYEBFIt7-{CD)SCFSs*zvi$ z(+S4NSB}-u%2j`K!cNDQ!=&&L46u`;i^qm1ktiPIOO%D$^)kzYk?_!&#h+ZT@|R)B z*OH|*&K0b4$5fIce0djW_3Wq|URe+5YLG%*5ynRsun}YD9G3ZKcWdZXe!4&>Y3V_i zKQq#L0$Sq6Z@E5{Z^&X^;A8Za{nJ$srQxS2E?zMb=Ls?51+QGxbU~fo!_L>9$PP85 z81fp{)j{*m;6O@!zXf7#(cTTzfHDz>k6|JB?(M%Y>YA_QSnfzeA;A^mZ`i|(n267S zA5|)5VWc(x1yqmk2bHod0ID;MWkx7%v4BSrW#zAXswRr4K!SK>steS=$&ubvuM^kaXbPs3h(J z(!U0dw=NF*TQPRbJyKglr?Vy*2Y?y(ON+mX9>!p>q?w3q4uhKqSQLob#hjv-{o}n6 zoZ^RfJwmW>JnAU?H;Bgv??4ypcr~2xltHPo-VhhQacG3n=FHI0MO_ z-+1wDeptq83l4bpgwOCkUtSFGj$wAghTfs@CUZ_6HMRj~N&_sUItg%U_(KJcn`z%! zR>INhj?{24sbX$Oat?-i?_SWdd0omayfX3f{FY_|eSh#1{wCct z<2|oDt)$ur)JmB@>m%?1V~_g6dcsJosf`-qpB}N9oj48Am1wBZN}C)1ZIBmED%es9 zbXS(7OtB%JbvI?hu;?B~Sk2>m2z2m&tUQLd@Lg1#(aol2nKE8y!FCLha;0TFjYGeL z0tw8wV1u2&=RUS>>30lMR&M?RG@st1_0UD?OzJ})1!z$JxMf;=8wIex+>aH7yeC1SzYXk;F-@@QuN z-dJ=9ek=QrK%48~!53SPllCyeCB?Eej^_Bb`cu0l>e+A)_NxjHc(CNPT}Cw4yS5`) z{sz8la#Wh^;!FA9t4X8+L+xVoeAVlmP1{;>5VHV?!$Jn|_n?W?)? zsbut-tnH{b5+CgTc$xpWpu|h#w3y!buE2;lUPQ}4*P>3_H&B?+nuUZA;_0W1_GeN! zgNfq>q};|YXcR??qHmg)X#L5RtP0Fl+P7nlUr=sGG&kNx?u}5Y>29}m!+cD${jFAG z{@al=D)cRmmpUBYQdsdl#}!QuBhEJI@ht;3YIgvS4LNT{72|>c^v0PdHI1wt_Bz+u zw2n0z>myZo?|NP#j9cX9CgpHb`XT3ROp{}t&ZHKutOWly*`{G6xk6!~>-n&O?bOtu zPy!>4Ha^2b2S%IJ!=IfXVMmy2?x6XS*WaVvwIZ4`Jx%*vYK^Xv?c2g#GZ`BGX|#Aq z{^5N*>puRP6q+v1m$L^1&(&)wr-&DZh9HTP%N2(WI-GDUn|!Z4R6Lkl82FOpxs@Ti z=Zkdttm}XA%Og@|Bde5@5!uJvU0`2`W-N(1+76$B((R$^c;UO?li*%%XS8XwA)UQk3Odfy9-DjZ2`%aHPb+SPD!`8%C&mv z1-rhl$<2z`0E9UtbS^l-=Q^qck3oE28=?~6cla;BTsY`Zo!s(K%@q4J(lXhsJ zl0Aa$zzqWS0~||=H1kH%{eCTi>o`nl)+%h}sGTS5@Vh;8%EkCXso#i94X7{;CWEW~pc<(fF-$aPy#yT( zwtK$2s4?YdA@;0Sf883;hJlmTI-=2-ga1VuH==C$3Iwl`X;yBPbKaJ zgW~d4d~w&!?t)DMwX4+dYlU5I#btlROJSY^aMyukrIm~H?m&{)a~jxyQEYPi_qKPJ4&-=Ly$$O&rZl^rO0}7fTcEHKsO_IPA+LG48{Yj%NxQBEL2(zWOyjO z8Qg<3!`N=03D~sKS|vp$kIsWQ8yE*)Pgy6+#Fl-}Z!elh0&-}6jW|pzzT|{7jMJxo zd}Opv0ePC{)McH zMrBQ2D>{qqKWQxse0>Gurl<@-k6G<+m*#Ti3+KnG*`C2+9a-hhH;Fa!)b^RE{MbA? zj_W9r5)UnVCa8iNUJ9-ba0OO|0lp%#>jU*OtcFwH_uhjGf)fT@5CqVlX!qa3sEh|s zak*hd&?vi&&JgHN>d5OzZwgKO#t|>pAbE59F<^#F;+tC$*NS5cURZ4U(96WGh54Tw z6R0oHT7v%E=)^AAgBA%o-+u$1^%T|}AxdvQE^p-@r#N0g*50egDzTt?NL(*KwWHXk zQMr3mqT;mFDCA;?`-m;%w(5fSerpl@#E&rHwEq!0FpVQdbF9xdCMbCT0?LM}-je+8 zPyUPa@+;qv8S_&{FlWlve1Ru_kGD>CrIkF>kHm1#@XtSNac~MUwuTYUSR@KJWRZiJ zZn@xcv7hk;B6G#Ibxi1lKW}&wivO}FV*VNV%b9TcmSX7&66Q6jH-nxa$xK~U!`)4| zs6Tg}NPy6F)K3Ma*hR(nJMvQXb!N?;Tblh+kc+A~-DbiE3!K-s7rte9a$d6yW5Jv| z|J(v8O-f3$EKD$8v%Hl*J;(7z_?heRFPLR{Y2~d;7J7PZLSy0tBb!lBvW)B^`P%-L zG8ZhXVw>pfGh~pvQ>yZ4!WK51nU0}2Q~!SY&2$wUJ2C6idIs_E(K*g>437S8UI;At zAlG>I{lDXo3Z5|KqPEzKSz>o)jw#my= z^x8m}R9+z>R#&**V9nSr>ilfiVnB7%yt;Ekeripq739m4g(HN3tKS0RTS^aQ^{>)N_IFxFmaEZ-oBTG{{?Cav{r}T3s3hWX=)W`J)d&ivHOjPxi zhNkgCW{I6Z45jx?77*LhR1*B{>#dGINrn+h5C!}RUb1rco(&0E$nO=qlJu{9W{sq) z92s24QU2d@ff-B`jK#<)Lp#eUVv$q3cR12w|9X94ie#Z1s`8#fzu|LT@;k0sEcqn< zeCB>jVz1OuusmRhHA!c~^t`68Nt*gK913TnWa8zSqTIzd#NI!loXR74T9F7nus>Ml z-ZU9mM|dM4h6w+-$6On#YRRL%!5*}gVG5+PX4|xnb%viB?I8HURFY8Zrn=HYO9V~o zPj-Pr=7=LW_+Tt5?s8C9zEp8n)3#C;|*?rFD#CTSzw*4f2-d9gn};- zg=j~hx=E>^teZH!1DYSKm$nSgda;Z<49apkwfcwLy|y?`^EM8`QF)7Ga|Oai5olcL zn3S8Q`U0k&`}OBm_?mA1jj|KJO}@{oBm(SXIaO(_G0;>$dnH*GWOh9y=W0FnJUqY7 zu47<~%491k6W>pDlgWB!e>M0zP$KcpPmpcgDIw8gT)$NM=* zoN@DR;yBJUEs%a2OdMLX2_nob4)tInJtDI#PqLB7!VoHTC7!VO-ZzI*OTWHqq*0;Oq^o&d4;oBZcWTkp6#q%?_ zgn~B?8Nl2VL+>ipWL`;ASW%i)+-4ma>%ybjD&IEA&Kh+^U1EkA)PR;wnQ1@#2@>1q z#Mrj@BFtRw;s$SfUWMfiTxRu1@MR3goAa9fIQr@JWlYp?XFFT&AqcwYBx-3zH)QxF zj>0Q>qEf{vv4Iczgf=FKo2L$O3EJe;8Icnr)DiF-D9n`ngi2D#F?rYBMKQ0p_ zYxEuIJ57B_+Qra!=s$d$S?brVU*F~Iy`kj&Nd;nO1e#4wFpSXVvp&Juq(~hD#BuOj z^^V*YLke5dfsN)0E5@IlkMw`>;!)3t_HJT*7cfJ=<4nCkcclvORAujQ^P?33IOQJt zNOHJ)hgh;xj*H}SopT}}MYx3!q#}vB3u%L2uKA(GPs7Oc2@q1%Yr{MOX0_U$_4+;@ z;jH5IezallSSw73sPJpEKtj1I(EOu<&4KfJZ&UcLue|haU{DP%olWBDJI_d z3U^b1p2(jk?GLu?)i)lLMcA5K3Q>k!u1Vj*^($iWtv6}1f0h8>0^eUcx^bH5cudi5 zl*U?xZ^XaTnP5xsFopNHh92~t1|yG}R2bhdDGE3DsN6IcT58n>6FJb^|C^2Z^bIfI zpRAl(U{u39=>psN*qW9v^Ivn2^XyavWfk1mtvE&yxp12+Rc9;^&__9rk?adz)Gm9n%N zTUzjxH{e3~KvRFzTeA4h6{3YLG?)`6liA_j@$8-f11+QU_EnoNyXCJMvJc^gYo$7K z-)))J14i035^b2rP5kvurwT)=!b~}GOOKBXwcx?lbC2z=3vNaE^yH${il+Mn#DY63rj2o_p$8#$8$ItRXhf zwTF%oiWY@lFn)t9ivcqUA;-Le3>Pv|r0S+G;aa)@GFHUfUd zs>e3<=c1)(ljWsZ$tKIe>y8M8d~?WQ8~!uppfZGs|N8b`CL4i2G6_3Dj8y4Uxq;?Q zbJ+&M@m6(IZ}4IzRR4%c&V0c647M*C&Ewl`J!wVnCroY#K1Tqc!2UX_BpDhqgMLBmG} zuK&_4L4pP(6`;DcaiyX<0gNvOM8VZl17b@F+^=UsFwwOxsZ2w`EaJ+J9&r!n&eCwc9@aHp$zeTbyl zNKKui{_~ES*a=ALViEA~$ZIJ0vG^D&;e>_0_*Ur@Sy%~as!D!ZnGIT^%`u5jHqKYU zuo*>1#exvRv^|3xKc!H$&`y!U1=)E{YwUMAN`2=rPmn8?2PJ%2xcxJNnrBoGm=j2Y zzrYk_NHrZD;8ung4-HWEVc;>^h6#>bCgLJR_>A%`7da0dxIg{~gW1tN9fS>o^Y{=U z%H*f>!EmjlN2N?q9_a^Q9%!`0|AOIWyExpXfzb9SMH);vKrz=Z$_Zu)=zUe%CkJyD3XM0|zjDK3df$o%}*vs+NOx)jP?nPl}(>RB6BG#`)2XVtsb zuB9Wj{X`1_aS~Gni?l95O^=Y~vXG+`65K6>-Tgs0e3fT-B(0d3nIXA1SgOT&m(s{gdQTp6DTm7eOvA9!INQOq*7s z4^NgRA)Y2nuU<`yS0InaKy~-Ls`szM=U4rB%{brJ)B1nH<7Fv-g+bw3Y3_YhXlMDjGpDbtO$E?oZl)FRROzRwg{@*DmWcUkl7nsc&nV z$L_cr|BMzcN7o;ojSk9nHMW`JAjQNlxnMzO(yR$Y#Z*_cqMO4&f#VA1{kz~dmyr&) zCzJ<#g2Im4s6Q(y6bfyuTRCWUQpfna1?m?gtD!l*P**-Z{Ns7qg2$7_(UGFcuq&ED z%1xyUk=icORKXELe0>1ZI)mYVf9Pv@VAF%3$Kb74DtjvkKIUf9AK4E8`qLBpmI7!PU^C56c=BV$;P?N7cs(6!Uzntn$9M7DvukBE%?bQ#1W zqxSlM&mu+|my2hkc_L%Rh#@1;`W><0T=;Puwny5Da5D8qtGXDgn;sP3AQkO?n)*jTXp zaiC$X%$Juix@L;`1~j6OoVk29_!B`&3a=CDt1^!5PgndH_DWKJWWE|zIh2nt&x13r zUZs%3A1U-8-L7W$t_2LWQl?DyNI z)OossujKUg(bi}izS(upw)gx5-y={A>_aT+UQ)c!H~%Jr!2unJhDM9}T#{LDRj7z! zMDrQ0(D)T}%}A>XO+n@()fT~;QM`E?W!S6K`!#(k~9%?8FlvY9Jq^TYCH+2&+R8$zRNbn*FfBVVKiHIs{`%Wx)j{OGx zB2x1E8zmSHzb)obmn!}8`mwk9o@w?|?B2NMlo6$FZ2M=u3 zQxv2g)B7#Qpg5BHwEjpg-DUT_$JziN0s4ll^Y{FpBKbvgBSu7Me_s())Ps4FaaCz! z9Oyv7{^IFY=Y6&J#-EFlzrqrXC8l(CB4E+?P=808rlQP{<-om4#zX-BjdTqOIxv+C zpLnsUnghTiDaCGFL>)Gd=>b;sie;S5W2R)N5<yfC-_Ohk1MkuJ2w%S|S~Ib}&sD;szl3ztyf)yE zoqk`+r#gBLrBn@y(9g z{2mvkx4gPr*(FL@_;XFIZph$Oxa`XEyk>rhiuDLewHVmLV$g!DQ6Nm;5+&O&d+Qhy zM$OJ2xv5^Pun&gSsR)kpsP;6_l9Fsrhcgu^0?QcR!SUNE1P_0GlxSJEtESDkA&2X)Jhknat&zO@>W_=mOY5HgupWFH&SWr3J=#_o3 zLKLK!9aF{Mcuo`*mbKgMw6#mYeZ9d{Z|0apm-35{B=tz!Eaw05^wnWeZ{OE|0}LSy z-7s`W4#DGzr{J@Yze+w>cO;(Sj`}JRcpDPkLv1 zb(qg5=XT6k+26keH0?rXWA}d1kicKbp-_}zwO*CJMqki)A|^;YMvzdycyfV?cM4F) z&+Tc}i@?%NPuJ=ewAk$?GSUCi>Q(WlMdbnY%{?QxVg?-4Z)bVZ+t=p4FRDRV#tD9Z zD??BW1z)l%y)^JGRsHiiTax|_;FC6l*Wb9dbd=HE89V9Yv)#ksKg(m@y>fD1s3b!O zJd}nrS!|__vQloSG|0H$JSpEl;b=Q|`u)*#)t1Xm+ff8Y-UW_erP0pp-x40#3yv45 z;zl1F52v&up9RbKKtyA!UbTeQa(bhF#V8T%NNiLZP4LDAnkd4E)f{&?S7w}+p6U*b*WCcKCiRuANsL`s?VR{>j2j8RL zi^4V^MCPIqunk+tBf*81X<+WjYRr4vwa%@9m3@QaK=|w0JH;|cOIZq6Y`%eHI+Nah zD=`U+(x<`$-ka|mY{nZ5G2SYjC>6+sDSTf0_f$u69v0~L<5Nk@Z+%{H>ttZiFc2vV z5|i9y;kyU3MF+MDm=K-Og{o)Sqx2*j+`}1Auf0vv2m(4MVBC9>_P+f!Shs~jlJ>E% zzp?9;3L!?A)yuAA7?6Sbk7qzR^6-D&Dw?-!4&`PnX#ieeu9bqme~b3wkF{b zXcH<<=Q4-+<%zmlDtt){W#jUi&U>W5s8G#KCK-yRE`&Z?eBUl)2gsv}2{@TWr}>TEKR18^kDrIQH=zO)IJ5Z26cvj=T8H8;j3W?8P{Dr?YQuEg5 z&ySCHZ~BNr{Rk40DSisB?br87Y)dlSGd<35nM~pqp~)|ZvN`=-J`*lInEJ!Lgd}yBU)-FMW8mwj2nj_QD*3lUA z^T~Vb_|Nb^_*;~>1wE48Af{;`g>cL?e)!>`W0;737Ij?aFIrRYTnW>osRw+9^&g)* zRCJYhy955}*O;K_jf#v$sTHxUDpBW9xQ=Oer}cp+z0aW*?1TkF?$PVB`lvYqcp#}` z(@PqJ-BfH`LdenX1l^$$JlD`ll)wGtd5@G%fkC|`iU^!Q|ETX-dWVx@;C!m2(g4$> zgbM8gio`-Q}4oTbsYE{$vHso*+H^{;QcoM6A{@30Pyn|gOVtmvLT_>y7+525&FEj-=80^xF4n` z+U8K~eqU>e2-)ZkkVp}d5}TL-rY;}84#te@w-X6*x}pVnep^&|>4)u-4qnw4IQ)}j zw{NN->;aLCM{tUh`r!o4Bexys>yHD>na+hTr?L=acj~<>NEtX)%b+z+<5#3#mO!0tIsB;DdCKK_Bn(%+9c? zrlKsrJ3POI%L#2wtIu|T2U7O*9jjV;9ahvBQkNX^RROK}zM%Ec3k4(qA2Mff@VL$- z9jIQALgAt=fOMuf3um*8>VrZzv*?Iz?1t|7-ej3Zb}3(ygDjjLqHrKa|Eu^t z3Jc4jHOIYNs4VzPTPasK1|U;pU|QBj5*i!kFA$k0b<@RpmCO%6l>pm>=48+K6}@CV zI0kAbE0B(4gL4V}Q#zxCFA(!h-`<-ocm?DwIff~rj@lspiNrlMb9&Q!Vqe9}Lxw?_$ zLA4)m+bsv9XFnuggNK?pfjW(er)ER+wCEmKW2u}ZH>P!;9=-kzM-nAfvekv1=OwOA z;Jx!Z;v}@OmGKIR1VWpC!pjFG6VIV_?TVKPyGl1s1xvE?u^R+qG@SsLR^V3jTm^#l zc1PbvKy0{&A>j=+h<8mkRl#(9T*rO@y;7Vlv)oA#V4la0i+L(Al3RrRe)7w)m(1~B8ua)IA_Z3Vg`0n= zHXgx&yU^m6^+HNgNM}4+Qr_bja$)IckX16MOf#=X&L`hVGaO}cXbFafRvfMxkBkDh z8HWkW=idrMu`mw!imo#dc2Cr1M&3zFJAA(Qw)8E{TGNa%GW;2MbBAI6ArPseF-w&M zOO_Wzo1&2%SZv?tGJ^0h;u&SedV&8;5?*gTeZM#4LR~@bQhfFE+3q+=E8PM_&!J=c zGM@0y+t%9=zxN8fISO#+#m=lJ)?Ck9oBFHJY+X^BDdx{D&R+n_QQT!yhRr!rLCs5) z{ayFnHB&Eqmi?9{BUTxtEJqYJH{K%K&iik(f<1uYDoftS%d#~IOpl{OE2X`En4Bdu z4}&!iPeUuWZ{5{)1Hs)e&kV=#;-Tf1l|AQoF9|W z;O{YhY8<@|a~m~(M-w`6Zdo{z9J+i37HcGK&=Me@<_F|P+k9j?0fzH#!2`8LDXuu6 zveE!I)sdvma z5e|E8%6k~03q_5+N&+aeX(7V7P#>zPxg@n>xiH-8m$Vzj!>8NBAR5!F$UbqE0#w** zTsIzQrjq&H?Z2I>iM&T0S8Uok-~b>&%P@25kk*^R$1x_!J6ZP>%S(~EwC%vF3Z&tc zVil5et7o0=i?N0g(vp&If8XrUsA}L5ncp)X`@k9Vo44{&?eD_t7~(#us;xxocnZWr zeGb&3A~5=ZnuUdXz5PLj#}JVjjUDuA?ZfVP24CgL6nDsxRK&{_-b#O4jxY5~dY_mBLlcHP1Wz z2DhwFQqVYl@s>WxeK}`dKfPi-&>i=C#e)iAt6r*}rDlQe)ryXj2*SkFbCi)l zC1FhDpo$5i9RN(SfjxG?k<7M-b;Y$OilTC1RC^3iUh}BX6JQxmTW= zua_6SjN_}^?G<;J7@@yR7dpPIzOixU{VN+nNc=Uhehy4(6P0rgyToBGKzl~@O*sml zue1s1#OC)1|1ks!h8K3Cn!P5It7?t_W027~q9S2HJNmXZy83`Mq;7i3Wpo=#!nkT*k%!88Ty(1lXUK8Mw_yi#72eesq(U*MF~H>(@m)4?04Wj$%?6^-#I+6zlBVHL zBwR%z{0Q3HXvVlXAEHnaweU+5;2TtMWWO6RMgO&3`&QK`%7ld+J#TQHPRBcM*-$E> zBUq8)^4sI1_`w(ol9?W@4*>eufQMi2qVm&8eiLNY^lE9R*BoCHKrbAWoWI9Xh^2F7 z0(WTks9=Tn#JzvS8dLC@`U9>vY{h=1!3JQ8To<8&0GH)PfMXOo#rFokI#V#~`&_PD z4Ez03S8CJIpE5oSZCNViMz(PHD79&)^MOyhnD_Daq3&15!@#q9#O03mC(5`cp*=t% zBOm>q+pwW^gx&QqxVp(`=-poQZ*N=)5~RW6bR1eg+i6PSl-|xCehG<3R}>lYH938 zFTOp5BuX-e#bjx{TCNL{f7u!7!PZ4{MlCz&GMRa1Le~&Pv?~r4Vw9EN+s&D=R~S z9=B)*zHkiL1M+O)7rQi&$eFLHL%}1N?)@;UJb{8Y<)pxbnwa}jI z>);?y&K3?R--In zSH)};PP>)+YH&q=J6FxSCcyZWM|;Mxv898O<$4Shc5!Uw>dz-dbZQ7w7J-appbRXD zymQKfNG@V`0(85+fI0{?8@@EX(9wNg+gXh2LJ8x+R-^#*+pvv>^7`=Z?mn1%Y7tKCelXS+r>)?OZZ1SzvfMKsUN8B^o zXFmY@kMQwQqC2uf+bgVsc29q?sroT58IDdbNsGaIfJ&UfBJaLgR1Cr{x;Q0)E|pL z4DQBS;@0S+FPWh}cmV%)U33!feQ$rT=s7aKbwbuXb-uU`=`_^QUVUEk8}n}P&)B(L z9{NXua^HGee_P@e~MgIv7 zG=teM)qDoa6G_r!WBi=};PhjA-O_PIsJ11B|0RQ^h)0Do`FpOtKetFOOe7^d9}^$V4alG-^v zCm*IF3t$F`4d%mS5;C6i(?!VJtu4_R0H{eWV}CTNyie8)RBi)-lZ6qZcr;pdLkNd4 znGuzw5YU@#hP!gZ#yud@yal3I(Yc~-igG(r>B5q{7*fu@mV@J!MjY7%;}i$k9({y0E4 zm_Y+fb+(g8kyooMi`a+HQ9%SftOs-t&VhI}<4V2Jc9M2z2MG3}HCnUk=IA86VyT6X zz>W7@wmakY;Pl(c=#O&%1FtGCMn*<69oKyz-jnED6SsXJ3+nVe zmjb4){%`iC-%X2+fWg|@YA&TWeGoue8K6nyps&Eb%L@LuV3}ipG-)NgR6e=MhmY%6 z%&ydJyFSoeDkIETo{4|M)1or;IDU09S7OO++!7U>Z=!oZP0T7c{U)Fle3YIJDC|rf zYaT>!ak0>eEX4jq);TCv>pibQPpR)dQSBt&7D24gt5s3~2x3tRNMt~aEWAiUBB`6+ z#E|o7rV{u)xRsj}z4``nGMO6>R`V4QK@qP6fJ~Y1eGvvWGkylqURmD-pry6A9_hQQ zr-hn{)fO}%OQ2bM{6p(lBD@C#t7m0T=!V-<;!%OV7W}H?C3c6E)~VvWUVVYvl#B%^ z=~Eu3=F!jMaM`J-?+(csuDD5($dGxw&k0-JSwv*6ikJ3j2$-5;cltc`g#+iJrIDhwKZ9 zGTUI+m7pc7Id-HehJ0k@~n3Mw%K-Q_xJSIA5=?YpqM&=^dOD20nkSHtiq_)FrRJCt)dI9Ch_AB&_lGwSq}qi_8>oHLrV z?vQD)1p;trM$sVqDaWnPHA_mhW&s9`)h%UwtQC;&*Jm7zty&+n& zH=aQ#wPdfk@T{v}RKA)5%ye#|)kSfBbc1d4PTg0w#9y{}JlNk5yJfRK&C^_6;gHL( zzOT@EVmSK`K;h7(@XaAoEB3%=o)H@F-7bz6X7BZW2aS#t146ExF-k^7dcq9e=YA*) zFxYQeOzTQaw34@K)kAxe*)8A?1o;`X=86{Mh+r}gsgM$>N0A%B*|k@&LOlFD;E^E7 z!1J(!tNXD+Um-u!Ce^`vFb|L5#aJ0BfAXsBWKLx63@ofT=@ADNppb=3a+7Z&q z@|zspUUa?x8G}>?@sn$FP{4wAT4;SP_^H62>3$Ke^Ci$`&ZSqoYL<_f7VUu7@CS5w ze@8#4;lEbAttX%p_XouQc>QJ{91Y(aHLIp_2)w!RVU@nG7IVvY%r$gxHJJbXW>T7K zaF||hKGuuuN3Xqn?9Pndg0YHAvr9Kv+f6J-s}xTwTLWph1bpaVC$`tG`or(L8e_U* zd*CwPfIW`AuC)gB!CQeMP1HwExg0G4$iGZ@mT5^kk~u?1c7XXuGlf#nS{Za*^AGQU z5u=Pnzo0ZI;q#RyM3c?}M~(eMZUL$gCgwoza&W%Fl=M6$Jr?O*<;qLN+bDe4_fbLb zoKI4e?^@1^;>o+WTYt7zKq~U%5ivm=^u>T2c5f4KSjI^#A_6~SP>6H#hA9Pp#O#K# z?1)LK0_Fh>qRs@S(<=GJNRZ(a6a%wZ&poGn8Z@I!)&hi=*ZA}*b!T=}HSKV;wpggu zD89q&!6LuD_A|aJX!=9og7EQ0pX#VVR);mNWe~+LCz${)FiT3ZQIYZ0bv865E*hBl zbW>i`P+P06`RQV%_APKj{QKu;Enmhr7KK0=M7$6CR*%gHs>UUQWPX~jd7i!Q?J`4f z=?~Up7OQ0tEK+I<=l#f(hJUEyKF)3Yb2+WaBDNN#IQ#Qc780o4}##An=4Hx`eY23ow ziBK2#^+4eB%x8pJZS@z(6JI zAIY|6z2)QSg5JVH^WNY+6@#=t;dS}>1v1xLDasM8^i%Ye3gS7B7rrdi81B_?24ya) z$~}j*_2cN%-hvI0WR>w*ZkG-i3x-U{3fi~1z$!BEg6C9rHEQ$nRWv3alMIa3!I{sG zSL%QL>K;CB{Cul@&lw3tAI2v~VtHlvi15oQxKFGmV;q=V`q$>Ck;@;SrCFHE5+0HN z`@%mNLj921I)q^?Ezn*)i6ZG8;okGQF7=vJSA!?*MFg}0tg@5ZHD2GEKL>Cpau^%V z?9Z$JjG&O zP!W~VnJ{v8zm`#P;$Y|O8*8Z$O3Y-`<8^0UaV`}mL@@?Fm}k=(x;Ba`<*)T@V$AKa5Tj={X?;G z_jUO#2pfeBp^!|bK9Sp#OTleHU#7WDgPNZzE7)b7&!3LVT$C3IJHhRGn@Q1FS!?n@5~7 zR_(OWLutw9NkEQ*1#B$4vHo)6=*bt==hzqNeV}K|De-H>Wj>jh@Wn#SbBW*vJ$F%u zcDBD7nk+agiZh9#Z0V0&zEpb|*r3C!AMIPDg`M1ey;sJ!^3MD@X?N-R1uF(^2KmNH z##4PPR_G-$dQ*tSFO&mIhf;);pjm>~AL217t#-(7!K|Rf&mG=;bHuh1FSQ5!L!Qlq zE6X8_nPrmo?e*L#@!HcX@Q&HD{($ZnWjSH*l)wWozX6hd7rmcrU<9v`;FQt5Pz z-P7LZL1LhB!>9gWj?z|Gy~MnIb{E?IaGDa7-s4x29Ji>s!7hG8j1WTiqF5zGQ?lQ9 zFGAsPu|fxXknFlpNz_B}56b=Tv;b|EFv1sp&LO!qPi--RA5G_nk$B;G+->VNE|WdD~JI;g&f76lHm`UR6gtHJ=iC-D)MHW_9VxF#Hvobq4*w5!b4XiEOr zUX+-CC{6+IAYr8Nf$(qhafjFHij)+_Y9;l67 z7GgLt6YcZv_xCmT-w6Mvq;SG)5>QmQ2}Y$XHltd3zn_e3rW!C5d;ga^CYaI2M_Gb& zo$LDdj2|1PR6r3NeY)~E50o5c)B`@mF~(GkdV~cU?qRj|8go3`!{>iLp`JkFS6Wj`4S*-yX!JV99?LrdCi0u84FacCLt0 z#X85HS|&QKCM`)SgEzxF_W0)~<}%G>-vaO{p0Bj`3=AhMHU^1=oQR_FUev^Q%!V2l zs~6{F+BD$xdFb?cbtF%jV_R@>SqeQiunz?!F{;N!@AtfOxqSH_5`qMk z!<6;r6S72JW@kjoA<|0Xonhr!ZF@ubG;`Xw-dH^0uJu<>V|T*u(`;s%lt`5hVK|Sd zO)^JKnden5Wg-x!m-=5HYVikxDd-EkrgUVo(O!aJ%v&$z7fS~+O{}3f{fIL@T|RCE z-=eY!Vp7lDJ25nF7fAsjDMmEkILa zRrgILoHshrH!QK=jZW7{vfyGdXg-1P_C2K(q^LgdF&z-E7qr-ib0iDxBp{KJvPgMT zoxUieMSlDs4jN{Rs^7Jhg3YQ~CN|{K#FttTigu)zdwqP^+cL!i#E@lGMnN^5<)(pB zN>r0xw#Nywyd3_cNjTs0>p;};-%1?yoBj&1vb~`q8s;a37~)LHfnU5erVKh_6r~%b zn(1}JG`n7Ij8_M?3Asy_q8kPL!;BbRwpnV~9%A6F`yzkqxA^b zs$>6@i|QdGq#lmVva2hqFH~g> z8HS8kx828_z>P+)1j`UL@c)gOQ?Z?P;wxd!)~OcgNPXh&|K;l}=Ue4B|HYB7t3XzP6vgqJDkGpq+7-7{JSPdU`jy^s?aN|p)1*1nQxVq5R)+ITq%C${cDK&BKzzHZ7zI46zKHg^OT}5h4GI$mge!hphVy^jV z8g1%oF|gxo@h$X3K;}E>Io&-nq)LKszJ`eY=>OeF{?1|8XXVfC{uHOO-`8pi*WZEC@mrVWZ#kKk z!aU(mhAUggxnn^@9>?fu%{6fpve`6MtaYI$E~aDXBKPtORkex=8vg-v8nsQ`OR1$GSVMI-4>PxRKoMuiyFQPkbQ?rkKfQx zH3MP@LVIV+o^syql>%?4d1bGwBu1#A5$v6u2SN=V&&wo${}eJMnQw2mkh91#pW$N> zo5>(JOf&h6&4Yr@jzxIhEjZjiefb3Lm*GIq`(mtl0)cLOrOD|^sFxyYe^CLyzBd`2 z?bIIS=0yQ-E+~%5XT+e!u@&KhE-+>yl#Z zin<$Mz6!!y|KA2vt{W+lj=}cJZHx*=S#l8ybE6A)Vh0kIadyAbak+0a3ABW%9Ay&K zKDT?~Sd^OG{*qA-A|X|YQwV>+-id^|-X4fHM?6vJ`%&7Z|x;f;AXR8uSyz>S+un$1a$0nR!X=_^n)P zpZP0#n#+!f=*F<+*y4`?9|gz=F9I~oA{UB-!Rv=&%rb38EKk@Eb6>oF9Ehf#$(;5E z!YW#`B3*{3Hzk=CvhZ#3y$)?ty+gp5puMOqz2?wNsye#ZAYt>cz{94V$nMZgW?sEr zfg*l*0&OfpiFBnULLz)9cvaJP=YS~sJ}#LcK^8`=RRo1URVq`uYr-gbHLEp4Z(CjT zm~V9jQnLnvsYxmHlujIfCirD#vF!q^RW?8*{_}lSNj7tfsfa*2ZF~iWK$Zs#+O6v2s-kT&#A(syU!L!g#@Q@PocM>$o>Gv(3JGEP#Mm7|NFQS$ zm30$jZ|Af1hYGJlm?QLLsaFL5t}qh2eq8u#c;c-8(6V4J+gk_OD-)?TMd$i^EqH2A za`2F|k|ti5zlg565tnLPJ62#AbQOa6E>c-eoCK+H#>(*qGv*Va))M+%--oJudFZ=G zJpYK*C%*{0>d8J(E|3Pi&3OI!5r%siTBrsVA#C#@as^&!DiRll!|$&AN@u;)bife} zqg<=1>n;#bRMa{VjR8;I3i8?Dj#Xz;F<+HyrR$7~|ILS=(H*^0>WL1qC;lr=JWfPp z?%880D}u&^!Ab^V02~Vf^dDF=#iL(f@QDVfgDhYaP0NOK-(i*LIGgK6IcZ|Ek7mh_+*6SBtuMk=7x~FO9jt5& zq1w!!Y!Nin*(Id?y`(0*wM`*(t{y0B+b$`pr}M%g@m|ERlJHnulnwOV7ReyH&I)MB zNQ4vU2lb!4ABcj{5C}Ws<9(Gcu{oS2IFj2s_LL;}gKpuY-jEScHkioMd zrSV~YR!JfmhKC3@Pp8-mnON6Hi6e^6R1|L$pehgkhC26Kp1;_CDxanq^2?84i0lfPK#x3)=h zr;{$$a8NoXIzms*C;0?FuDZ;4#pbT?xho4J>ifHwNg;i3nP2x(1We~Gx@6R`h`6{@ z4R&3p2=$HLvI)oL+B?rOk8?Cl8S=qyk7as1o9M5*rKXWC_6yeyj<5_J5J_&MG{v{Xdl`G*{fx}T(Tu^Y0$p~URq0}DJAXgl z6^A9LbDMVpo+>mzd-?(0cLF{VyD|-hhuMQwZy7k=C1QdzG#HL{6)oyA+9&p5?~^KMrrksY7E}HD zg1ESetki!WeN1P<e?#Q0Hce_`no@pNTNu0Oz>xK%i{a!;xq*@b{pP3_ZfY z%IY@)BV7Q~rrM9?C9xlJ#9>*lRm@gdA*v6Lc|DfhnUC2yLQd?1H_+cNYgEsfuQUv~lylXRI@SF2!#+CyD_!wSC6XrF<)qv#2>P*YNMG9$8v_Wm!dD!h**5mKB18(peP=0FMhqEMMA&BY`5#r^6zQjgMj)$L$O8xdHlvBzm^j9Dau;VmI`Sc`|@a7zLmBRh47FA(wdoo1h3~QlOO@)&})CFfIeZDs@DVKrahe%-_P7-2IAY?B-^It4) zWQO(Aydrs|GH)3Bo!{1x5<`H3o0#h5dp&_w>Q|FDQ8H?+FYIle{3~MW>%_h|%6F2F zn19oO^Bv-XRa8z@A8%ZCxIOq;kG0v)vT-hM+tSh zwq0k)%oS@$$*2HOYNdT<*1Z#|_e{B8f|MXK&#I(jaxIJan)V+K+9NW@8ZlWqgWCu` zH8`t!)@ANFBxrdKJ!eiv0)ZBN&#JtGyWQ0Do$zs4e$%%FipqJEmMK|lB)oz73{lVE zF@_eHlYw;KeZ-!q-nxIN(qdqHi=*wB;}W+RIiC=T77+r7N9t}Fk-{ba174?axYqP( zKKrXXLUcBUPR5?}w+lhsmMIIb44$P$f4_VAt>)JmraB+&LEuNlhAggPGv)=#`8t6F z!mdzO8UIN#8ct7@$XR`g14{BcBT1AzdAn+Ue|c zrl}olYg7Jx+EQaD$l28YL)hWa)(|J$Wvu;U+6~9!4e&Ux&LBAm^xy#~S0xxkR`q0T zy1)FB#Z*F8bm{53p?-0AMzZrvGZv0dz6uwXUFf>aNZQKc#V!6~y&iBip&9g-*IUfU z?iATJ&3S=^L&)Ao{fX9>#6!9>nJ|~%JyXGQ8ico}$ll1Il3**S{R`m9(qP~@wgM$! zJ+X7zz$X8_17{^Ldc~II%pv?d7)>3QFzJzW#w9b*yMb_LPKE&*cTCV!xV;*F;ngDG z!l0OhZyY4zf0hfkEVfBcS#X(s@V{!@Pxw-502tK5*h}SXK(|Mw>#Er_*=4}crGR0X zxNTmc-C(T}oYGX63G9>e4w%w(_RWua8J`hm`R=Oo#m~yUt1>I!*lQ8lz2kN+&_I;3 zJ{UGiUnerK`2|*(-;=%sO%M4GiGH@GPoRy9k0!W#V}a6bUN!lI$@VDA z-ocdP8Ens_QkSy|f=>4uV*J5zOCb-}nQ*Q`bK0WkbcTodLcLFAz!`QF^^(+o6UCnK zGGzJbtJ~UEELdif^m(5#t4ch_k6zpMOo>zTT=@T3Zv$LPn4~hnSKu>FDCTJQuesqN3n09S$^97S zSBz2T>Qm)>?`@PR?l%4p3hH=4E>W{V^>c4~emS}^}3?bj6n8sJLX$~aVq z<3PrWD#72vPk_3=;YlOQ?<|-5Zy09t&#?u083sXdES4BCpEv&RSJP`0YyU?m9d*7f zeC6Eki{NX#PWRd-(*T{net!0+5_h(%+0V5*%=v|VEVv)?nVdO#4X>>k(=_Co^Z$~&n5KgUUp-%JbDj{|S20KXH= z0gQA9Q6t<|+#t}$XCi;-UW$2y4ww>(B1?Kj=T!)Gg{V;2#>$WV@%l2*+pde&qNDTO zgm$_QH^ui!+*974pVB-r>GZmA@REie7Y%E8v2Nwk1sb)ZzSe!fo)YHRk%PH#VjQax z0>fSw5f%ikkUb2$2vCT`Gp?yijCqWj;LIZT@E2-;#IbGRs%R>7GQ5&`7sy5~%lb!M z6;%IFx6IFUjN$G8BI9wz0iK2_Gs(0FejhT9<{`oPmB0&)O0ht+USZnD-ucXjD5?e| z2*M;I?l7 zyU}la5K_#KjQ_AYP7PX|KCidA108~+x00pVf9L-Qqwu0yNxmio4iBc0m8q}c=-k9o z%K+AunIsuNW$}1!rQPYX%^}!{>JRKh<#yv~ZVopY-`(%sj=tj3A2S<$UK8`sf=I^9 z?ILJl?XDx&2=5z@kSZ_Ni zTuw>fqvF$aJyv@Wr{y|89VgJIB9#DfY|{!-O|}j+%fk$04&kM?SnP98_LL_r;vUx* z%ELJ>VyUS)FGZX;CboWoSuUJP4igA&=4#D8YK{_9yzOWmUZqc_SNhX?`pIV2@$pYl zpW$Uws}OOG6G4@%5lNC?HKTH$mSX82fsfJ-n1~No6e`7Hdrr3c8Gvd9phvOwS`T&* z)VdyxBH}1U9+&1V z)tV9hR3AvFcL4Nr5Se=Z`!DLn1Bwu&?aOP6Awp@Dzr}7ZZD=}FROgK6K5V4;Jg63$yr32Eccbey zPQck$%5&SvY(C>|*X<#ojz-&?6@@^^hhwQil4DQxn^^td%sBf3&N!V10z4o3Y_-3?zlQ`U=EFe=!G(`;9KHc~Xz47W_;WGqxFM*Y z=;BzCN5O4y0u~^;PRuXY0$w&aGZ-nlL&g3t*}sW$vNH1d#|$+LBO0! z-};MRqMCMgpgbCV1(YDEyw8#MM$qL)02>W(PjJrP0%aJ6)8oJCGyIXqI3#TOjLHcr z&F&;T`UFc~njMxtd*AVl7WWel{ znL3kT4E#tE9B13M)c^0%FEPoOd-W81u`rj&xy`!$+uSm&-tC4hkATDi&XGeC0uPi~17aWCAR5gd z1uZG;PuV!U9Lp@8-21xrsgE3l1SseIK6ojKFyaP$+U~|3B zgCxBFhIh%>D4uw-PK#0k{VYM2nZVv-;wnlWr-CDp1?hZSz@>?(gzv^Pf*7Y*&O@>8 z11uq*=7`P$<{tPLs5(-J|NH;+F`K{N3XMfer*8Cj8Ckxv8+t`bk@=T zc>#bUlZ-=Ywb`HKj7(jF#cj@S;2&?VA9S7r+Sug7B*5D}n6DsFc|W>IR~eit5rukc zO+3YUz<+qVL@i^nT&oBWdWPbQ75+PZ1WH1L#ady7hk>m_NU!QWLO%DKW_N%oqMJs@ zKHt;x_T~!62N2Ck=qj>1n@tHO;-hs`g zNbXj{kpT~1&X1I)F&6v+Eogc76yDh9YEc1DSgrRU*)y5k<)ttK^8e!X18Dx=2|g_l z^fkmgtw)JP9j1O{R`V^yLczc5Hj^(a`_zQ$fm##-Dys4*WrPj`vupb@0xS^t`0wwR zyFC({+tj?PjqQRwU@|j8jc&6KIMK-um8HI)N;}k}dgOAC#_d2|gBdgwN0T!7bPYu| z;HLTxW|BrW!+t`D>IqT*c9nz7z0ZZ-F^Pg7h0hI}=lCjL;)yor6O{Vx{|u zOZ;4S#Cfp??msuWntP4j$WDkphuxYqztkI#i|NgW!t+8lv2@k`i)E;U*1#ykg%Li% zmy{LM&%xLt?6#wfNPA!De%&&M@SZlg{`@JI-7BxlWs$4#w$9yqssqo=tW~Q(arfNE z0F7sBx=-bZD(EL5AZDa~K21P|;glVKfS$yxM3AyL@rR!00-vxOyjSP$tMIpTC2HQ_ zjgoAnvZ|4%#H$GZ-}~gbI9||M+Y(6b2$n&^$hkyqY7pAU>HxIHJVb8-n3`XKG|QL^ zR%b1s)$Y9VfoLjOIHO@0@ud)Vs|cLT|mcx@+AOO%A zfkGMDYO`Q`djv=h_t7U;jG1a5@#MTBWBagr1?(ZfqjLbd2~5LS6z#nQ7U;33DfyLV zKt@6R?r|YTJnlbWR1hB~@I$n_=y&I8pJ>FjX28Yvvl~L;FCPrng2=GKE7^8knR;%v zKuB6O`2%)B?nBbFbC};iaVg{V!svAnJUns6!wC#ZYLA44FLp*#n!<#%E-KP_j74t5 z{C;t~bv-g_qcG=6o%WEuhAFJs*JLF>6*JX&31F8`=T;IbbEeI-8(>S#j00nR55)kmGI4V5O zUEUv#V0hE|a)ImjJG!}FbzrPn$0g^_+YFRYM z@1I|$Uvz?athoLvSL78n2_wO*L?F%S)L%?Y0MA z90m9aJUs;NqJb}1>M=*=%W=5}MA?-6rh&Tpg(+-4m_V?)el}HK>21CH{u955^O6&= znch5LIM46*@rsIR-;boqE5Jg0lF1+RoM~rp;#W;e?Q;jMX<#~Ig{(A%iu4K2Nmz@2 zAp_u?brZ=K3Rw56#hPVT%qZzu;tte+<|2o8bT)O19R~wsf>6mdF3B+k?8>nFS75CX z?oZl%yI7SC@H7(N#LvaQLuPOrCRebpRCP&k=q?$uP(~!OKcy$&@CVS$i|{)GSRw~* z=QI+qsqS0%gkZOH4DU20%p`_NvhNpVXrZWyK9ET59m#bNG%V8lm$=GK;0aF(Tat1(CUzom$EchbgP_=x)r8a! z%Q=hImSvgap_G$qDCT)Y zssqU`w4Yss>rp=fY_LWafoJ=jU(Zab%ZmATF|-TuA^7*vrOXG;NE!;#e0~Ho!dm*W zdeNrEq8y8h?;ROjq+4@W(?b0fDeXoQ86issHipL>kU-sXyDR?xfmU4fH@ISzPm}Nc zKc?O~D$2HNA0`GEdWMkh?(Q185hSHUrCTKg>F#a>L8JvlO1hD5kPZPA2|)xz{GG%7 zyx;o%WG!Ww>pElaV;`~9sQWDT9#5<;3Mzgvr-?~oB4pv(;G12swpA|YOa0TKIB5dy zuA>u4D+2zlFh-?dafV@Y>E(7mA#8>0_wXt7Fs=U6LYWtrC7Wn|_zn(ko*JoLB#fap zF0Jd}@&ugv;ROO`#O?E?y^zsPAmj&Jf}Sm7hwY-W_!3q*>1k}?Awf0=P_>tQL&lue zO}gNoW@%bg!h*Ce+z!>awyKwaCIL*e07f_2Ex02Z_*GePRrg;J! zRiRLww^&4FkF>)4M~xxbp%?L2ykA{A4OpUFlU(Si^p8>Zcth0g2QFjJP8I>oJeFC1Vx`@D+Cd-4nl z!wZQ=>GFRh1$e0&6+C85;cd<~S6kjupdrz;eitG7*NGEohw)tdepB5@cK0XZ>QFos zRS49pr|$-H9;-+tmA!0IRL;rpzr0qR=4Z({kq;tjjcO>1Id3@hQh|#7wwk`IqUD*6w|5zJx&7wS4dP|cM8nJ~>H9q#N? zsFkbkw@_mA=!dT)yjhD@7n;Ls&yP1=b!~$t$-B-IR1Zzhv+xEPPNx~QB2zw*-|;o0 zu~0iXGoUENiQx|@wLxfGV5uK}=8E2HaziQZIBj`)PRQKP1a|A6D)h@*W0EY zDc)#m#Rj`^p{X^&15rlZo%v`5325#kqC~eYl0jVRA-bpTH2Ih^lspc||Eh zA7pqqkfn2mM4yQ-eE27+@LI2v^zUzz$4cNBv?fLaRg&{^1B=8|_C83xr@OWURceG~ zey36`2I>Y=@QgqXiiFNJ=$O`ki%LA1&Ev*B`XHYlpACVY=aHE3>Ej5@QlTL3!&!t( zRVp5ZAOwT+HOD-T`WN6tgwzvh8Np`}sEY0h^w>}3rmR>AupONw4OLrc)n|UaD5`BThL53cNJyf?ZK;&%W;kKQF<0x>h9lhva3dL&Wj= z-5LMAzHJ1RVa<17apl=aH(&+~5g;Lt5GamMB9Zd>bFmggr{7Ge`Y-~J0dtAHPA1sB z5WGx1#Z&a%P{wS1v|-!shiN8-H}j|nc(E3}rmjA!h@Yl5!G4bqQasTP454Gm_-c{l zd=XF)k$|Mg@!=0l=;Kzr{f#5*2{h_4cMsrJWwU|5YXn~D9?FCc$i$Y+5iq5YA1X3^Z`yp2%HRl(c<3!xV zRG`H-CKl4v`VDkm#8SKu%v0_28^89K08TJh{e*9oPeWr`7ksea%q_0@uHJ7!McqY9 zeyM9jhImKSCtD_k4lB zmq3z>fRczIWJx#%=a)*Of|ln4`8AQ{kL!tTI@+yT?_No>Jc@%;A+eVb2z^mQR*rpq z-$(LGZJN%eRc#_j=Aqfo`6R0=yE^Ze+1|2iSK4`E0uq;(f0CTD$~&HU9}?A4zh!h} z5Vnfh*F4m0KK?NN?qS@_n#6If=jfLofqp=Ca^?fAddT&Ps8dyJL)j*usV#inS>0gu z&{cP1iP#^EHbEB{Sep3(N9#AmG?g37%>fdQ8x+j2_y9&^}V51JM`7#ASNqw>#oMu!8_38)B+1)qVtqr z1#zGwQfVzpm!|Pil|)h}K;ff?heq@(pL}&~*}10mgJ?ikvE4`_Q)Pn3+o+MvjM!rK z7v9T+lmW?jI|)JYrPPm9RscqNsqMGc3j%MIH~F8|_+b46+UYwtvbS5iK;zz7Stm(@K#n!)d6S;aE&wBG-4_7?lMD)n8--u)d~r2Gd|!Gc@M zEuKt%ld&Wm4_Z8IoF_MwsD8yWqJ9-y65UIm0eDYNlyCF{h+>8(Zxdmpk_^_s571VK z`<`q9EUg7mO`RN*u$dtj^zwG?y**)!j9OuR41aODc}9M z+`hOl@T@1=hyHDll!b?FPy53&+QR^TOS2hSSx9UmSM1M`1-7QW(M&#B&5@bh`3cZC zS^9r{^N7MD+Vdrl$f{W@u0^&SCv1Z~L_~H5ph1TtUbCk7RAA+z!sR(X5JSusJvyi^ zSV>$E2z)tOKXrtQ@1&BXU{id2%87h)!GX|ZnU7#RgcRC+w%hB=;+aw5_lL-n0cr5< z%aS*RDMMeXXMkK+t>-V02qS4Jmdl>u!u$Q?pLHKGCE<$|7m~U`@ViCmWc*1UlMCBe zpT*3o(jgtpCeqjl7yW(JxwJ*ReEajC=(>;FFM^x-Fyxa51oa)0HP0_@N6d~}zq+x1 z1#G;y?uGz?7mTxIH-h-$=uS$*iwm_C)FRt}@54C&roP(_u?|sHQjW@5a{v1VFrv!D zKcid*<=v*Hd=2slPk`cp`%Gz2j_i8!8s}8ovdX9?7t9BMp8On)OkWdkD;U#w$FQj- z%KuM43zi-cWK$i9Q~wH>)fP)R%Op$R0ALrG7^;54!tws{N!gbJyB?jr>*UfpWJhZ2 z`@!)o;E}?j0qTQ46nL3s23kS3bi+xh9r6YPVA4n9Q!)0Ttn7{08wFazn0vhAIR^7(@%cg|wysL@oCGeY94U5Q^~wbnYLTkJ`)4TLp`ME7_NzgwdV@ zkZ1h!SV8UQAEaBpNT#PGv%`Kx2(iDk=XO7U z>XWD?ygoNN0%+6}xNiVW1AHZ7MHxkY!`wo@1~i$#iR7;I?l8HFB|E#J#K$^<&%Vv6Bv1x|S@VrxuR*W;zj0d)98rlH z!-@xQfP^@oZX6g52)nH!-Py_{w8bapI|E-l@nL=mbtqp~z-JQ0r_Xjj0RINQADo7jBxg8>@8x1uC`y70qU0V=78cm;kq_GwOY0<>GslEr zV5fk}f{4iNT%w#!mW!E%p6_+uGIp>-cIp3PZr$!VfDkXyNie)DVv28-%YFDF?M6?vuoN0MOW)#iFQb)ir;jCm~U)p zo3(*$P6DAA_fI9^h~v*6s=OkUxvOk{1KxxO_oOcnR1*^9CI|HPa2iz)HyBlIF5)hn zL6b0L?)_L~7;U-xAP+oLVX>)F91WT8f%LcOWRz3a3eWK8pg`c*<-%*(ZnB&Y zTD7P4VCVq07`l~~wR}$~M$B57(uT&zNcE`fNU6xnBjBVti`i7$YF46xukh3pT;6r%w#b#FE~}zgmv&X&}qX*N~bQe8W7at1|h*^Mo)o0!Lfxg+bxr8pFHFdCB_%UyYiP@I`r7z%7cDlqYDekdux zzQ4e0QQS!CE)ll z2!2Z|U<=Wi(H(FoxgC2abcX+!_y<8M6*^yav*?2wsecwlq1uncT%<}c?V&y|`>`3K z;+oy-fD??PU(=V}>NKvmhHfbfuT3=SZ#b@ge4qG`dH-$xdS-*G?Ri#wDRW#BkihzFH15Oyvv)OgjQ??U3vB=ZfC(i1)Hf##w##pU3~?6 z&M`O_8}cI>>Q+NdVXH$~(L8KvM?3;wGCVKJp$3gsMxE!3(CcA&nt{@idmxkf-TC*$ z`yUiOpsx|7zx$;Q75UTIYW>Q0vDV@yt`M_=11`GNnD1Xr5T8=fn^-I5=6_wbqn(&E zEZrMIYq74JxA7TIQf~N*PV9@j6R#g>U@oA@hL3C_=AlJG(YEMcC;F-l&1Hfw^X)7& ztmlA*i=rdbp#6I5f)ZOwPoy&3-uf#kb{udBWw<@a$I84=vd;2nEPqgBsVyvAZ(3>{ z8rIbKN{N!mt5P$IH|891{ojaW5lSC7CXgGBm-1={H0oY0P5_#MP?|CRh|r!ZPhJeR zjl@tPG_C)pohSMjXDCTlF5SE@Z4ZbQZpuCJJn9B{MdTXlM!@tftXE}}QrN)p%aFJv6P8uj$~OY!oAl1Hh| zwzX7GR+FnNYI{HMPM&fHAt&_El2_uMp_81%YNXokete#kWBwe@335)mnxkZIX@K`L zvINWn=F7|D_f-SH!U}1D(ThmHdHY*og2Y)2?xL*U_JMtja%pHu**bO4zxF3zhfA1M zVb03Ej((ciq;2bkv?N?Imb=?44Uch0?fTgtlD|J-p&XsvQJ^Qv$pT5>D{ku)? zA%s?LlFX2NYz8TyzDVQW0+)S{F=MRqq8c-kbWOmNus|d3r}6U8qT71w9<&E&l{e2^ zB)q+F-Ts19bw<&9Gs#5RN5CkTsnfz!1m zzXGPv!8{GL%q+E*W&Vp;AIol@bnpRFte7!hft_)VtoeP@)z7a@E^HTknC{(uBjB_^ zpBxL#igC)SV#lrlb6+Iz&>qp<*7yC}*t^*(g?GSL&VSVVpWn)Vs$N1TG&?8#A!CE-GD>7iKMMP0Qi)=yFk#k=jW$p-ihSi z?OErly4WsYYRQ`;C$@(Ts>Y{O?ZA}{S?ax12!=G>SkezW|0t@Z@pFx6CtYYXC!Xfs z8mVhW&7Fpu^+*#vt9rz1@OVxX*mvr@ISbZa$SW4*tGEI$8H3wGLQ{#TJ^1BxhRP6y zD{r7Gpc>;d8BH1QeI}lL)?#6xDxT0kEzsMRQlEH{f)Ovd*DU=J@M_>530`7ePKOVe z&U-9FGqok3jM`B4%BFpJ@HKumORA)>?~UhohWIh=_+7yXw>=4ijw^Mlmpz?u zp=}3F%*^lXuax3RRc!-VceShU@nxzX7yy5?a2fQ@262&i4~_f-iRJuXJbokvhkDmU zcgHO@d@~&n{t^4eb8(0=Z|EP8w!I3vZI!h_rxxg$i1kl&03s=Fhc$toJ+>J&nZ6h* zZ490{El+rTXlKLdI|Cxaf{Q{bp82 zKb@6mb(^Ed)wdOND`IP0QyUe+B8JHg+F^J$6Tko<3Zg>=d&t^bAnrqC3KR1Hx!)2u zB(THCpWwQ92B6hE5$8RR<2zv`^30+I+_hg0!_OcPDDTuZiM8VgVzczwRDPT7U)FYt z0|(Pe-%)H$g_JJ{KS>C@JtCSwtO_ibPri~Pv>1KzVL_ux-47U{$A2-!X6uy58lpgb zHb)?;dS+DcR!Bv0DwFog_{;lFU}1C~ARxqc1T41hGsLpY4Ea5%A4@o}^jt*DFXR zG6?GU66i?lG?2s*+RBLPAGr_skYK0yo(7^}X{Y}NaTOABJ!IT;);?~J>t|JVO{dZr zkFd<8@4%97Ghhi46B*oGZtu=;Nb3I3nZ5|`>)1?K^?==Ay!s{~a`rNnp~CT}X~j*S zgN_)3DL~AO=42wq3vqqNYDd7PU}|qIMk458w5SJ!kY3r#P4vnhT*cjD)t(QW_D8s+ zxW|b0lVDWz_%bAVr)Ks;In#!H6VMEjzi!h7b!1a?o7>^5U^ihnE4}rKYK{Arf>X&l z^Z~H>Y8|XfFifl+IGPqJ@MOlNwT|%~J{0e^1V~b}Bgp;%e0yWXZduhezfp1u4*2TW z7W7M`cI0mcyBMeP@V+`l4Y!{i9!2rBBPI&KWTDd~fGJPZ?61di>F(Yp)fgO4{CQEr z<^hFEFyYSyBM4M)U+;BiD0NK@P3yRL6!q2F)?D!lbzBb}0V6%ha6P&X1+n$0l>?qi z`+PVbzELW*#+RxLcb(2BVEm@QEJbJ6S!OPBuMnDAQ0t`*phd85f#JYOpk}S;OOhK& z1ck8@q{)3lC)YN#=iz3f`^HPqj&X+)^-BE%=J`sPc+}#_K{?M;;kg+E&b85sbrqqe z?*yzGMmn?q_pkwHyST6x7$M&?uo+^Q#O}XGEs!#*B@V3@ zW>ju@U3oRFO6Qp+34LmvOuY;tp#zN|2#K=q?dRD}-{Lz6J+Ky5v~lM+T>B{2I-quM zXizqS^M>Nvf&;1~=6$3Gro}Mjo=q%Mq=+nt0AM?Z$FSBn#6~Xs!L}M3;a0*XR{*$( z1BRXs20{A{kG4nlmUkT9bUJYp-m%tC9OxHdu)#mhcuWyb-q(Ky$Xoo6#Gm#L1V`Yw z+{`I^0%Q3P3axkW9>96%!AOV=7I6Fw$!h59tHfg%W@h zY!6na>{7D>5vHT~2yjkSbhj1VGe8*zY%X*m4CMiiXBnd558E1c>E_U zn~5bta-oDy{St&x=vttCJzeewJ~GECiH)l;=5oY0)#r zSf{NAE8Bek=qG$Od3d|iiC&Tuj0t;(G(j0%|sf!zz+9bo~TaA9TUG zxb)oZ@JPDmp8y{6I|>ox-rVq4ebm^5l8``I#Y^XE0Za2YOtmS-=&1SO4SMIs@V&Es@vfA-qXV6 zQGFcy_vNqc7B>0c)E%*vt>_u>@J9?B6deD@)|T&<1}M1;EmuX3!e_1)yY0iJgFS<1 z070`3D6&%gLW&n0|JP$XJ;q@nG;s?gN<}K|4YcRf{LvlxZT$cS*T=Mp+Dai^c5Mv4 z#(G<&cBL)y>VdWSMKfM5Vjg0eZ&5h z;m_c}KId3%Vc_3bWTpmi`J_PgLg6;4>*j=b*-r%S{T6yV@kcI6;U*ePI8c~?3pG_& z#mT+wvez~8H#nRT)iefB%}dtIjlaNRPFznTUnELa;%^L$G3H~6Kx2D}IDW&NZHn*r znr16Pg`dzr>k5AK4H2c$2Z<6RGdW z$#x$jkR*3XTT08qJ-|+ooL(Rg_f!c*Vt~QK`@D`~$veS;DsMA9lxxUZ)kx5!Ib@SL z?Vr33@My4#R=XuHIGpb66kkMAhU><|NInL3_)ZI2`Si=5RbzXx5poL4s2ai!tiR)0 zF#Y=Lc-0~fW14&ar94*t?z#6eA=uHeZnOVJzI*c1Elny<_x+dNOxEvgt{)8l7;CQy zh|H4a{IqP))6Sm9!o{C(?!7|BoKRmsB|ib6c$z`j2ztDb&4hWkIFo{!aB5#+_9211 zXjugFqQFC|N~HZe=vqLm5!6jQ?0t~Gymy$jBqX=kGo^TwQiUATC_#6&A4<-zRrM+K z^;bRm=Xe`pz`6KoOZow%RMauWk=z;*LU8zgWl~F_!W-74(y0^VEU^CVH9XuVz{Q#k0+}%ea7&Q^&YCiP7I5@(C;@{mKxP; z1m_byXba^IiD@CTAxDQtp^wSECi6wfAclKbzS_!@WhZs0Yk7$bQTl4%9rf)gf-^CNukm3QPIKlyr_>skAe$%45GG)@`THhpeD`dA-7?DwC z@J+ws{&Wz@xaTthyAO|em;GJ7gAlUo>LYaMn%j5G#Vg=hOB5Z}YY2=yx#6w7^rb>v zxL#vLoO^Ytr1gSh_YCr3&MbquVIJsO;@?muONO zy&t75ljx(Rfq^m;c01-jfV0)*D<$xRxT!7PE#yl~)Xb>(0pR!Ya+By6H7#8$6)u`q z9u#Lmzp8Zo3DLb+e^EtAE;$9DqV9vA;q1uhb3uebR)Z&w*77fw>peKWQwGxlo?^7K z%5JY8#Dp1^xn815*rXm^?X!-`_JSFaO@43i`=hrPzXnXEer!qifU(Sv?*sQUmlYH_ zqO&MTdbD9v)N`BRSks2pq7S7(p4g9TCZ??@!Z?~0L8ToSb{Ln&k2yn8V9QuQlQ&jJ2{I_J!RE}L8V|?IJ6(vlpHEy>Efb^QQS82*diO3u zgOvc+cQXiTLjH?8doTw4ID~DTKd#5GnzH(Pn<_e$2|q?dE8nB_tpv6lE*UR3T9BtK zJhg~1nvi1r`U(uv{v~w?g?uc&{^!f?SpSz@GrWX;1NTi@sPJ`R`|bq27{!lbm%-q?P?2p9;68PK!GEt_{&)X>1^_!mD0tBx8%kt}3d&+|+ZQ-0`p|K&_ z&wE!A>J}sCF!=>796?2~#AO_zfB>K6!$@Cc@3ZccULh=MEkz^n!#J$;oAaUz!M#i2!>u*qQ$fCDRz%7Uwf4a1++8> zWp%R&f4|2)X3Yt{Su@wh68Zgs_P%eyFc_Lb_XjAw>yRJ=!S)i$t|=^E!UJ#U6ARX? z3`wnT+C-rY)uOm~P$V=ma1fK?K{ktJ*(2)r_LH>L6oOP4C$M4o4}yL|AW|{5^GGEW~*1J>D-5%aut1x#Ix2;|4Bif#}9+3h7^S`b+!QqpusM;K9zU29*XQ1|g_f z@MxuWj6V+idhsQ(4d4Y&RUkQ*__n<9J4L7=pfgm7VYQ%kK*&~9_OFmVd@TJk6ar?;_JwoiV?p&-Jt ziQR@@KGyK9CS!F#-(pEcg1%}N-ori;RBA znCVxMYfl6=ZT&DcxwHy_JL!{Ho52q<5_PEg|KMLXnJD}va}s5Jpqq_TZhfEFAa~#J zpE3v_!<>};Q(eCQ?={h%_kh6&BpJy$yP%wNcrRAHN#ye6xxMHio^FDo%XmTGV)@;> z@C-*tihf->xeYVPVE`?>{d;eJC8&qa~Zy~ zS?zcxhgokt7b>Qp;|%jnO|foI;Shv8aUaJ2x`*u>tl)??_q#!bE8)bN)pgvA&fX zyd$Gry@3tIF}J&>j*ML2A#|$*BoXy6q6)#75RQqACFpkLRd*+q6cibo2m;pt5&c8j zLqEoZZZP`T8_V$qe1^K2q!h2xJn#E|-*sG25=-xq{^UI<%&SeoL1lnxL#EtbOZ~P*Tl#oE<*T1x&>zJOY2qbVv;j$C%+SQ23qK%Mx;zQ-AEX4W`gE4LSByLDn{bQ} z5W@mwZiEaizvUMNbK=%xUa=5wjZ!abG|}$hk5I#_@if^CW9|+QLnyyqkAB2@V136| zz-;{kC?mRNPbTvX-uAxWOa7b1>SKoIkf_DxoDDk3h<9dd-9LLSUHCPP3s((0HU!Z z@M585`K6&hYK$rJq|mDQ#D!dB9yDkz6_-GM19Wf4Z7EWA-R)gS<|xuJf8`8}P(D)> zfLlWPax+YPzQ2vm>}NP-HYU^Ef5ih^gvOhTsF5kH^OdmI`0z2|%U4A^-sg3(d+eW^ zlvY@k^TYvRan)}hNP?lF@4<9F*)1k~2#_PcgYQEgrZ8gzjQe0yt3|;1UKWJ104<*l z;~IBVJce;T)Jk#y71r=GHPKSQ&mO+dBrtwdLy9X5Y+QB-GENO z_@thp%X+%gA45ExC4gR`SAzLzf|uf1dQ?omP%YEahN#O@9lKb~-rY+$l1I$Fe{u9I zUbW3_qpgpPV5o(7%ejz4d@mnRNf0WnfQ_+LCGH|(IBdn>@~qz&(7G2~ z6~nEp%jV3uuaQwX!0l3%)^8VB#H(}U&HV4QPk=`I>o5g#9z6i*Ofx{hqE0sUCI|v2 zzW~DC6Pf5z^iL@8a#>76M(t?1e1JI(6#ak-$H=>Z1P$Jghl1bS=E|o5HHYU*0)Ow= z{sst*DxjA|$H!;UUIGb)fTMP4s1WDZ%$9O}4kaNL5Z(C=)8 zzP`g+wqdlv{4y3&w{rp(m;F{Nm}O2J7W=axSB@GUL`jBiFmR6D zHn>}nP^N!^TiVtpHJN|R6wkvT?VqwW5YmE0`_r{~4HNb62G^;7q%rl_E8?9qY}=>c z(8&DljVWT=5A8?Y`L!-yy#cJcRR@5uI+x)4;y<5#AQR=E2G-6q)Y|lL%G~*Aa3@=U z6kH{*y)kS$Z3w!PJ%LgO5pw;6M|NMH?!Z5RbIspjz>w1rV)8X##MuaZ$g(ZVVoODD z#vooxRyWh>QoS8A_eI#~&Gy}i#sDJ+J2~WZ+K?fc5skdu7C%Z_0VwiMG&}(<>`k^I zSvVpY%FM#TW7U~IUIR`QyoPe(_g&{Rn&OF?csUB(1=GqLR-Nsr*I%&0c#g3fzM^+1fvy~oV z8y#G}PagWNY=;U}bLd`@VT-=q7wOKSl65IOR9QLxpusoy(^z9R*}-9*iB#1#xhzv- zdQYu|IQ0`>10VjHRNuj{zy+Tc3^fNhxmi=vrn^w%HUT%rpRpoo^s0<@zGSWO6eg@$ zKJPIw>dfSRC@#W5Kpox23;_$F4kiwiD$e1sDWh0MM^zL}dnHkm(s$;qUZ0)y@t?zd z0V{s_V`L;ATQ~tA4St*0H&ZkwH^AAQ}Cb3p;7bPL{^Z<68 z4kmj{3q3~8+tf8syk`Lk1|6hm3iJ$c1jQlXm;%C&vqB>8r?B379$ta$9hUep({xn4 z8i}$JK%wksM*D!@8c%?)m~povArat}6m33u!f-+dvBu{Xl4+uQP4c`4gY1SaN+%qV z3kQMei=dAyekI*HA-qT#hSPbu9TyH)8QQQ0gRkXM$F1lsA;OS6_s(SvJ!W=@?)aT( z2I#&FDj=ir+8qNLpiVjNXDEq5Xj%DgMNG*YQf=yWrzC?A57}&q$6fqyVYAmSb?UmX z@KvGW_fYSRZ65lPN;XAJ9Fx1Dj}dS_D)1|m>gvOu?E}=KTC1+Y->e@KnavCNn+?%H zv(UW-&ZOs;_z!;PCw2oBE7~&9guekAJ-6isj8WCMzfJze-G?m2$E3hNqV1R_yMbE( z2%Qy%_+{ODyE-Ca0YLHuZi~O}Gk*Jk3U;ty=KjTTGpD>o8*;fa7|9E;chR$t{8Sn= zkujpG;ib!3C<1|(uYzmNKP*~miVi#Uspc_N11wmTY`csBG!QUsi0|!*j!&OahFTHU z*Z7j3clzmtlwdJLfrC@C<1D6}3|-KT0C6jd#548(mRc`zckqW+RIWyMgZK-91uDTu z(6Av6b8tK@bNx`E7v2N+g6<(u`+>w$6m=gUp;YbciEa!=JOYe=;!QJWpqC?mBkH^` zM$EeKI(co=ZbnG&)BqkV7tw|2WuMRg1Vcfkh1bo1xlTwNt8HQn%hG%MB=z0271$Jc zHq%lr2x)xuKr$}IYq;oadIEeo#glB8BsEnx@<+)W_02YrT^%Mw2b zy)Y+W{R|bMWn&0-!GH+I+u!`myZrJUlm@ltt@+{3?1A#``yh=GnKUAeYlnlM!ZxjP zU+X~vpIOr(2xkEVbj?bM3D2m_>Q5)g9k*NJEn~eTN$;p=`bYD3xYez zx~e!h61a3`uOe9Wr&Q}xBP%1bBa21Pg}+XX-+al_Z?ZbU4+8RMYcuU|4|Om4P<)Z= zx%?byrCnSGoqx1*t_;bXXwrRG^I~qR;`8~n35zb(7^nC~P&k=FH(ntemiFfJ*P*@bM{xg;r13@TtJh)uu% zjSP1- zbDD>u{)1wRGQu~(T^=vD%u5=lWb3PQE-nRSsb8HtJXO0P9$O>$aw?F^}5kf1t zoHoX{x{a)E!%%K{7>XXAlp9tp1o4XBJ_6PKuo9!m)@5PY zjJ)iV<{nP?kdl*h{ zsO4Vfi|?f!YOzg>lCb7>bCG&FwK#ffK#V9Esfb+|nZ1PJS~cSp@zunK$EtmvvS^2~ z-4~%&=XOalgYTBEXcC_iHoLA2a=s{(_5<}9yiha6P3IRDVzaVOCI$(omp^7#Ssqmh`NTcmKKZQ`lN2Xvx+Gvg!e!O#YEJr9aC8Bil)-y#0cLKPQ zY2D;Z;@MP*1oCjfc8R9>&wVx#$GufC1hA~2lA2SWR{4pID`jub!< zVtqu+~UYJ4O?K1 zXdz|a*4Y(tVt@&vL4N~gV%}e{dCF1&X}nF}fb2a=KsUd@$E`>z1x$U6zEEW5)qrIq z&>1u=6ZlNoROn~ifnebXU2q8y;P8??E_`-dy_YlT$`JJBkSk-d%3RPwoebYnQCr3O z*YbSagxUG3-z1H?Pw79pzbmg?_^k8ek>~C5^utC6z0T96s-xXn{|kJnbWq1_xvmNc z9BqpeDD+Lg!w{QW0;B_E3`eHtY66!e4Kn!4dw;@JE%u`!p0EdGAKW>#=dLrc|}y_St1fXzNTkkTP_s zGlZ@~zfx=KJXrN06INMd><;@2TxQw<0})`CBi~w%2a6yon3-u;x2`h5P?YvL1Qy1R4gzs~O@sayM@+vgmc{e646cd50p|EM*dVWGjM-ppit z@?ss$?Q?D@$Qq{!OWe`pHEoP0ECe%TK)47y6~r?qSOD~R5G#K=W7G9F0QmU43Ps`DPq2Qsjsnt| zQiDxO!o&xA#uptR*D16+dIZx9A3b!1t4~Q$T~P&?dn7a{Nuy_t84}f~MebTZNPYJ& zh&HuGfBO+X$H?-e1>-=Dc$INX6w2Ntm?+wsAAEZ)mEwzjP6Pil$^ZO`JWT}tOE+*z z69qz+^dtI(nj>X|$3#muP(QH$cW%?ue|s2&`4VQ5YxA|^;gRdhrmM@RL>uQ}Nh_(6 zg+Ii(*XkCNI@Wg$h?^5+bcX({zc|jRi8!^@z5y~`FnR4Me>23f2X4-<>m7dpwORjL zP}j@98d;sIHp~T=Tbvlh)0pK#svtz#;UvUt3bbj_4Xd=ruHZy&wbv zmI3J$Xy1vBLxHkgbI!)e{O$nRX973G)o*r88R~4u&7wHWU$QG7-P<|^94xgZ@j^W9 zhh*}PwrYnyKL1zsv9GP{s5BbItQSZN^&D>6=4npO-0N}-U z(BNpu43NiW)!wZ5ZodZ|Fo2uhUm&Y4aLJ}9b~9ENXMlxNn%yF?y*3T5WJMF=D9XqQ z(X3%bNsdSswiJ@mdWIH(15pfyvXuR%pLtF)NxJKg=(`-z&^^*#G2(5U&w4C?PDqJh zfUyLZguQzOKQfmDQl@nu?|vYV37uUFNr3AI^_M!eN?sQ3vXcBF{q4wK z7qNMiS}(7)!kHMneh=4+_*_=^?c%4=tjIJ!;||)pIbQ$eb~?KAV%B8$h>~a$c|v}C z{-ZGP;^86H$#3%3CWH4#*aiWGsL;31y%)WQp<-qnfiP#(J!ud1Tz2>Ip*$0Qiib3f zfI4)}z=QX~R!5SHaF^SjIUjx~SRLsOtOEKx6-yTZAepM($ZwAm#1>hpV#wd_vY z%yJ8WBWU_Z*Ez~}uxJLfQ{_C$dsLL4Z5y>?B@`Z+Wku&9aNvTv{U?jb6RbLnp51IT zqn-^tB7Hu@3^ezsp1u6(0YJ>^x)oNrEG=6gL0N8^S1)Yo68l7g;Vmf6PdiJ9i}q(M&GL-vhhhQ+-%K_0dc z7xD2Se)7Z})NmRP%3&nH^+7`6l^$3*YeD{-B&U20bE(;f|v9AO6$u z-(ygqF9cXO^`y`JKr+zf$-`S@BtSekmF+ojz;l}EFV0LZj>0=2do%wC(_sEOrXie`OQiYj*O3CvKSCK1V8Y%ey!ZlZjwehEUR*8QHc zab(~cHO-L5NT4|sn&*7ckm9AYTc3A$PS3#+gYi;H)Z=)yNFM#A>a>f6>m5E7jv#?e zrODvXS79UjCBy-=TS_rT2c!=F4m)s1YQxXl81;!%cb8l{04f7|=Qu4~_`8hzG>;Tt z0?8gAqO&6G9E#h4Ffse>2gg}fL=&X55txiHD#tRPpoj?H{n~0+ur6!qw>?yKb8&~m zGVHn;U+&^6W##|!N@djEGFcl>|ICwJQ&et$-{Zt0zG$2&Nm|f( zfji$t=*@Y1CGwyJ?|~T;?JKbzIb{U$NBR)BwT~-MzF7iO7oZff8*lX3Qh1Bs1gcwy z@J)(AN1FE=bkdbbMsLO5&=*PG%0j=r90PCGN0?XSTEi=q^5AK}RH4>i&09RSz~E*T zsBD4J;@+2mMCwGBgil#~bk61R7+hcZIxiU$G2hg-5r+U%O6_cZ)O8BB6`)-hSOP3t zhq=2Y5{_IVy&Japx?qVw0Neq*Gmr==`bOIFtBbOd(j@b?O~tPd$Q5)v*`i050Hpfh2mfng@UIKRJ-2~5 z)r0YTehPlS%+D<J?UW)d>k#DgthYbb;s7fUd+Jr!$P|q>CMfVnhNQ^lZhGCtSYv z;X3r=>NX-#WA+?zfIEX>~k0V<~xV|F2Pus@~2rBQAF?=9B~#hPj&+ZJpbzZuVg2d zjm;_^uoN-}oJ-OvcFF9PAPZRJ{qfiGmsb11>$s&IJSr)GrcRzi5=V!u(KgKS!2~uW zT%P`sToiU#Fy?ju#1gS?t04jJD3Mybt3{MSM^#IbM zNkL@uhwW10L|U0xJK|~%Zx|cfDBwg^XWt~E4^@pL# zbbT1J0@+`HE-DaAjKMk=`>+*g3!J$Q7IO4=ZmBM=K(eOBtJ~ELeu@L~uZ12_tE=xdVI&2s`h=`oxlCBs(_-ZGIX!cuDE@~PYQL}-1*o5@Ylp| zg_3vU5*V5da_1~DJadSs%LLwx0NP>w+xN2>5rPI#5>~evWF29YwePbIr%ux3q^hgb%d#sCgt{3PLUjxPXP>*MJrrBZcPm6K>CizmZ}rQ@WY4o4W!@py#+ zYYsOx507pPRq%QP&2HZEU2k+i5rNOYFW?j8CZHXfiRj!wXlajH zw6<50%tbb_{~Ou=HKt0)`&-YK#=s94JJJT6$ScZ})S!;WhcFuro(y0SP$NCEV&8at zXyFSZ|9?zNv&vX< z5z`^jCVmFFANK8v)qsNZOV5jo%4jWyoT{0N{$XQ3xk>RdjzhwK7W@I2C9jH23+{^5 zQDYW?20Lu>pj0kf*YpHb_@*>17{k`cH@y&w;iNa`#?bKaeL_$ zc9wej(e1xk<-Ht1956iR@zQ>Om-M=037tNyJ710=;crow2!B87Vq(h}e4$I1iPwHO zSaf4}SnD%EZRY11&qpskL253x27nTtZw-}67D@_ZK#SoLin&ql9uuHYtZ{-9S4-OP z?C9*y!{g-6|SOOwGkpOq?gF z?!>=*Sxf;yD?1v+g`F6QsdU@K zody zlUl2U8zxs+?XGH6GA+oYyABvb-PK;yvtuvMv6ctzTT_v@M9&IeY81W3@c}aE`2lZMPoiRbbxlzyLY*%>6T+W*O_bSDv6 zc>o5(8hs&#)^vJiV!rm>45q2!kyWsnb`_fI;DjzSvhiV$x%q^;n{#x{jBiR`Q(7kx z#9R)r5E|R!4P8*Yr+-voi#oO;p(E>eG4~v0GDTsL;rN(!vM+lzvdvxzSOgm|#5Whl z!@1V|&)8;_@4nvP;!T*Sm?W75+t!!6X2tiMim3gyXVZ4NUXso&qaTbCWlyP#IdyMTZ9NBp$c*5N#5wWZU~oLPDooO;+>e~x!~HG@Ac|K)E&ZN{&_ zK|?ohl3_KSRd!G@izV8RoN%zX9OgWAyboT+URT{or;d76{`C6=sec!90)H{RwX-c3 zV6AyiKUMMq-Y*!Yo@oT?@6;+JT_%P(b;*&N3PhxBcP}pXbnTI+Q@H%xx)n|?bxC+k zZG1oV+W=8iY?73Qbh^cN>D0~^!n3W3)an}$+@3?hl+`wihts@5dW*!IamYuP>YxG`Ek#b3tiV@llTdrxxL}H9kU;^<7`nE#P@#v zUi7c8he&9vVQLW`;HzP`5=$$6QkhmQDbx9R^cp6E$s;nYinB0K4D(d#@>kapkrk_- zqowb$^AW?vdQ3F-`zHC??arjfScPtBun`ArC@NKVc3GeP@#Jw$C&ZH0%sTq6^Ka}; zguZOkJ54v%Ed2D>VfnE>49o|uN`}?D{HKiUpX{C|?`Ybo9&$InSFpeA{l&Onp@QPi zk7*kpxcsTp7BQlN>r1l;O($ZKA#$O6D!#v$uDHR{fgp^*R*Ii?Scxsm>Fq_QmFc=# zT^rC>^;WTUY%5lO9U8Vckzs)Lq>dKdbU@kpmsP*NPsSrr=VXBP|Qlx04JuO^se_VyA-rl{TfL_o{Ay350m*(b}E}vTF!N~HUxwA=RO%4u|F_eYu7^SXMLf0`v}})@{m1A`r@aO zcj6U3is9Fo&`XP3thijPIHIFT-3OlK4L4+~yx6I5;dqu1nUthVn`!v7 zYkT^fSE!%94p43&688-NORrBkHYMqr*0ab9{a)E;l?$ccBUd?g7}kO~&(M;_Gl_Hq zyjizLdFNZY@5^`e!Y4!|2X>h*D;GYhxsdnUE9X4xhpS;q1-sY|5101L8Cl2HI5{@< z>amHr>h~nG47M4pU-x8`BHn&q3sgA2@M%OxtVyE$EB5e#y<8u+ z+l$mh)X5j$*5!;+Z(>wm!KvW-^H*Xohos9`4|ngz9B@lWaFxd1LKs)g{q!0xwVD*= z416f&BHr~b(N=e3E3bPuDT+mFpr1wWo?GJr(MpBxd_Vsa^Pag^9pnaos(pD@nG-Tz0g|h8%F~`cV zMw&0vTar1tpSEs&YkX)$%6ZFp+~*(w9ctqOPYZ|2DoSWb4py-yvC-Y#ukU(yhoo6^ zH~b9w{RgV5Uf@ex7ZV&omS7>BYD71i{$Y4Zy;FfWtT4O3YG#|UVd(5>o@VpyPM7pw zb!twJ8^8XGHGws$)}vcbCd?FngGz zSs`oB)}PEDSw>sT%fs>uj+o=Qf-gw1xU-~nj>SJQMs_SHUWzZe_KSY9qUc6DFDC4y z_1@raW>yzN>{udgt8wS;*l}V)KRzP9l8ZadDwMj_lTkg9e%TdiS$?Fx@{X+!Ws4NE z-$ve+?y);D^oEe_Ox8`A^n0Aw>`$%!8#VmndwqMJB(AA`r7bqyBAb?%kcYARwnmzv z*p7t8A_~*37Rj0!vla zb&<7yAv46CJN=tV+zIK;kC*s5sZX6dxD(7l@TNa4#d@9egiANBvrd2C@p?J-0wF5d zk7v(g;Hvsng5vj){O#5kElc;A=aqCkuiBscVv$Gtm>1dQs4UiF{k6{hK6zw^e|Cjv zI>(H7MVDK6T-5I5JGr=T*KEGEx>B~h+m{e+l9;bo@AroHQ!WgeuvH-a9Y(Lw)$wI$l1Sn%tE8 z%MS<6JrFiQliCZWIkR5-1K?qhSI~YeMb1|$Jd!lhZgx1{7?|YuDO{LKg-kfeu`%L!ncj7MbR~gdE&3#0bR;b%Ukpx6( zuECdTmC+SEx+k~j^;oA#GBEP*?DK82Dy$jQtIQP~4El#A$)kGfoHF9Y$~oM!-VU2o zJQmAU?e8cja8p>;!3xvpSQ31f-u`rB%JsJ1hd-FF1_o)NbOtjaTRBdz87_^GPKz)d zJ}AmREm=G`Y_dTf8}N4U70a59IX3XN7HbkN$KrwOa8PT@EZ;(C(XR&rLH=4vME>4f z>Pbe@kBjPxPGs_v7c|FbC{grZ&~vUP-@+} zd4`l>mBjqw(id}Y&6`2OQ7v}WmzwA|mBs1jcs(A_L|p1Z5hTHQth(7{sO^Z*M2VTI zO`OXp;s(*25Uqf*%Gm^$id-g7r#=4sl!f(Po+7b~_z_91xBEG&7a66q-0GJ4W0-7I zv>ncOOY08Sc-ecoEZJ7NF+a`y+8V?9*5x|YwMn-_ifMBG{fLgv$eP>sP1-zE&tm&# z0t37L#f}_I&)Ot4d1qK)2-aHubr#b_(TV103dWEZRJ@q#D!QQ!zt=T-NLmWz8llxp zOdO)yKdpWK5}(?)GbI-v2p&v5_1-5Mi~e%vu0dyt#b3ZIRAmW^7QEHN{sAQIX#xK4_lLblg+_32e`nhYzSc(M0x) zvY+x*n}_y3XI8rRD-=+4Fb9P>oG{fht6$%)ocX=Rx-(cpupNf5^?MZXgP%$JKpXdw z_liN%`sqg=^h$XjS)cQ88cO_{tXXKgXI-80+kHaL;k(1(g#RXy<*U~_-z_E{&)pA@ z_?*e{#Npv@ZpVG7V*+tF5n_sqQlEBK9uY5F2g)Jl!dt|>_*U9ApBK*6 z;wA3Iv8Mwa4}RF)iq$TD|JL%!moA-E;9Aj^s%=D<69%Pmi8b0|N2?v&_^wky_qIlc<^j@GBizh%9l*Ry|R=E_6#DPb(R zBR+Sw%U&<$Hmmqm?aA6}oeI2rdhb5{VJ&zA%I-qUz5J(RYotJW*HIbFsywNJ)_LoR zi9UPqQkt)piOH~G01#{3-pJ3Ip zwompNt~BQ69~MNN2TD0Y5X$A0ZCbxOvhq#Z2IWpxhb8=-gkX?PO6%@05n)HYAAFx0 zSnwv%HV6@a$0IMKg~MAAbdgJz6hSLn0wQ^dJ>N-+#aw#gI{Uop7xI?L`yGC8jzUh2<$n-i3$ zMmcFBF3f?uJ&3s)06y6HIMl7q{vyDK?cDX|-sQvH)sT!O7|FC7S&j<$LPZWcO-U(W zMqxO`6xPEErf6VJaPQZF!>jtFp!x4cjMOr&Q=w098&sY&k5H+Vh}$$$3@v^;TjKmL z!NE5LCT`UiP;*f*N=CA{6zAY~C<&U9nTn<q_~BJ6oSrOMnWyi!-N=gjd`)KM z4)qsf&vB7Z1+6+feyCWrwBQx-KHKr(W|NC*s;M`J^$FzeA%1`bBO9>oW7$cC9Xa^5 z16_noGXh7;*P+>Z(+e%$?k>fWNI~5jyz_jWh96&Rj=IX1>F00j?)jxrDqcW!z@8;U znqLDH!h`P8>~}uI$sAdovs(ltDC%GjcxD^#LVol;c{a;vO!+D$V)*IYod^FSKHN_Y z;vg7ff$ik|v$@OV1kXPL^{97NPJX}q&$9$bj9n(=hF6`@s`Fv;=56~4If1t&sUPc? zt_VE8C1*&fZC}oq<2doN>T{I5M%-&T%vCswmrtWc*MKVw?PQ!x8@Lpz=tfMR6^g8S zu_n;qtPEUbkaLhc&-rwsNkUGgm6w});R7E~UCl1t&;(09vkLQVU}EJP**@0!KQq>sZe$REUD z>!T+#xTG$0w_T0@9e)4nB4m?y_M z9tPj*a$%5X?eYd{EU2>Vghp2md{d2$3W$$e_z(xcVUR2~6n1-2E4$c)|X} z8_q^N+iBL3P1sdWV*2hGtNALpnB>L;-_!QFS`lTFDApIg56d1VguzwXgwS+MG)2Yx zt>mc0v-{y##B)FtOQm5jQ@v*Hjkl)XO?^og5^EIg?$8vyF;yR<2oJpI^eBG$)^`9j z9ciy3Kjg=pnyS4PbJiEgro*>_W{8L}jy`_68}P%@H}06a{zG+QRetqVB}AZxtF`fvx#q{Vtw7bgWKV1xu3j>}DTgpr(o^UHixU`3A7h<2!K z{t)f2qkFu^qeNMG``>+D(Uoehp|XF5q3AHlU^j zr;Udv@(xD`EsU*>1O)5`Rp1OXYj`NyA2yHfS#<j~(q@f(~Z%QuYE@2i`mF;Z$k@hc;8N>%brW`O0qF^E}(GCx}5#pECht^r@9To4% z_6?uQZ%VNqyeMo^S_jp9DJq^ulE3?-=06%YPmN#A}@WM!5Td&H@5KTp9s%6O4KjPk_(S1wES8|C7$O+So>P5#|oAH{(6oU^;sRknevDX zNkx7A(h>338^Jb`XwHrz7RBnDo!ji64}sc7`?y>o&o!ziP3*(1cb6T~r7gQO0&Yo@ z-G^;G5tGlnvQ+6Sf!KFRxQ~AS_gPwRp@Nz+D_G^wFJ2)NDx#SKquM^(hFD#RC8>1} zdR5}ezGs+w$7yO(*@j~qx~`SwKkD&ozkLrFL=e@FlPMfGIVo$vD8_(a@X$-kZr+3b z98&p5f^#WG1@wxynJ1HW?e2>;+!Cf3WRPvz>3GR&|C;8Z{xL}Aoxnv8rpsxnI5@JQ zl^f%BE&tyg*T!0{QGN zsJpvKk>uoYlP>ckcDG{6*!y0UoL2;6-XICyeiP>P)^|JU#CiR>Du`qU?IQ$%Mj=Xy ztRq{GmG5Fl0mw>n7d(@m;6Bk)-Gc!lQec5fdc~j___krcw8#st9t9^IAQk?JB#!C;mJrRytcB?JV*>rY31`_|FOLoBVr`cH)wv3sldG! zbl{9C9SW9_{C~*(P|t5=v@^{NJAMuC^%zFhYVN+7U7DXnS0U=CR@2kAwiT*gP?yH} zEM=#g;bbMmww@w@XeO+mG6y@CLSNlr%>7&V6|#U>%cy#UI$HM#9tq9Goy zQNveJ!MenMsnD!G>NY8l zwr&&9eAA-Uzvh?sP#aHne@LQ^Y=$>MM@}+ByKAYf z5gp2D<+^H&4{q$}BTfU3dxXCZnrX5%tT^`-=j&~sT!^O^3x22SbZw z-nVI~u)`}2TIBSi>~SdoHT|je8qq?w?~GWicv2=iT%xFDqkCHjZtnAs5|3+= zrNUFa{i-p&c$;0)d&8V8@pXz|M3F?31nX~W(%c5aBrP;S>6!g2jrNii3pDJtK#yFr z%yXygP=QH*@XteXnd!iIQgyr7u8on{47pOSpz?ffiEW9q{xTtfi(-DG`x`E4l}U3!(uMfIEY-BlpomBR|^73l$r{N0 z_3O}m|4x-mW0E}&LD#iQFP84>=-SV59bV;#5hjonw`O*&h}E;7x<-!3zQ*bZd(|RX zq_qY{b%|shtK{6;O9lO`i5M$iFz31`Fi4Z=f$db()@I7d3i|;u@8ev9{o+N zSVoi<`^frTiKk_J%gsvCOVP##7%;Ssl}gjPw=*C!LHmOuoh9tN(l?sfXY+x6RRWqo zJwjlt?R3})o{BdK6xZ*c3=Qle8($~B1^LcSA_fJ^ZdtI!7z2l~>N36BnLZ-yM6eRHae}Z9kXjCp1&_Y1hr2%6-mB5tj_RQ@)Ba6%61q zy?10~lK=OV%Mv5`>|N|QvVEg=)P6&ZTF71nx2=!U1qdaZAU2X#D5&oawwOzFjd?%` z@%jNjb#BF}zo2O#7P6A*P)i!0CZ;eL(N+wVsS`L6|6H+zrI5f8OUpWFF4KUkxWR>c zc-(s!LuZ$N45_z8B@4rU$yxWs<&Sj1V|D%&4=$Jv{b2fY?Vwc#`$X#v;KIxTmvn@I zr{Aaa+C!s$g<^^cRm5WoA1eD>ub_<0IoKv4qy7m?4nOuLcS~Hp@i$Rw8>6p&%PP5T z`XQPLohYMu-~}ZyU+NF<7aCDiT?a%BLoH@0_#o?#_Ly=y^5!&;wN#s4&U+E)DncjI zHF61z)KgDgj_%Nd(f-wNJ2vy>Kh6jcQn23W>Qp?Ru4`~*Ce92|qPsn4(=De)Yg-<_ zYXR#@w;Lfu=eVSWa|nF;OCSGp8V)hq9_`Aeef;QOn$Uu>f<8Xy%38WE&~wb8H_=e~ z9rMsujolS9e?M{%`^SoDoedn>Re?>BfG{~7GC!@q#V zFa`l0VYAWCgs1gMheaG`i?_UJcgXB@O8|QrJ)sgFDe;wYNa=f)+3V>u%%|t8CO`-D z_lV-33QVE@eJ{jeANb1N1~MS5U0!}w`n$uU7w=0?$))Ak_Vf73A6GD;#6dtMm`PzC zW)2O}`Wc?Uc@i9`Zo_&BXd0}4^*?s^VZ0#x-Yxy-SxvwAAKs(N?w?Pl@d1k((F){e z3?JgJk7lE_ci}52dZ4-rpHxVQJR02=?pUMX8FHae3OJIDj}`4_koV#v8UbpqzRt41 zAG!aZlT{dvpm3l37{O$&n0+T_{|?r*4?2TWW7i(K#K@E_d!5-vgWE`yJn|| z2rGN$zssd!j!l!j!8v^mt`gQ+l=c@WYWJ-JtAwD+HP0~MPErS1l(I@`?4~VjR{iE2 z7MIB&Vec|lmV#G){6QgbN|?}h13=(MMCz(Q>P&3|h* z415xRCAkAC;D2A61MM*yZ+QOs&1Ih(;H5yfo&%-1EipOy=JgNMdCESfFM;_U{l!G_ z&f2H)daF(2hUAajoo|D2ZBLF_4my%F{e1o1{Ysht#WEv{zwrYTh*%{10&7%33J;q- zVs3?{BoAm^c&y(!3$o&lAWSv=Y1n`o`_@ok=cjaQA<>A#=M7r3OAENc^8aLRdQhF7 zp?0n3MjNSG!~4j}yaeU(g}#Vs2_~u$10DVlyoo_19}4Upp&LF{HjFpl0KPYSF2GM6 zl&BX#Az~Kx?ThqoH&$LfJewfmljSphtS+_ZGAF4pj~_KyvV?fqr;%5=%QmfSg|b!Q zdFJqzBq)#%hh3TlsWdd?AM5>I-dyg3{)CI1IXS-6FcUMc?*Yr>5|n@7HzxYW!1A;u zXvx=x^FqFG7yT)lLSYLE_S;5VW+jKh34FCPT5bmPMH9q+n74Y|uei3g)%H`t6U>!=1ITF!Um7j|CwuUlO_c$4+S zlXE8V?wUwOf#Z8B*Gw8c!^0jW1II#qC=|1-!~Fd$e&qbZnWx4^g7ru-K)wEHO|D$*X%Z2}AbB@{7P z!#3Yk|CX=J!GR?X5!0Ak>X@U`CHbzxe?dnX;mT@dudUf}zPyO?W&7Im>DgaS7XapwejKS75N)?cJ?irL5gj_O{(Ks_D5vPjs{~L2wflh(?c}W!Qdv3`@&SscaV;Av@AS)8|!u4 zt4)6`$TYn_C6PDg{FRl|t0eD-AY@tfiDwS}QPFj!o3jA;N&! zUClvh&#S_E8GH)ZoiX?wXW4E12k*Dnh$MGmZ%ZsEvGfR-VVyurF8o-p0;i&Cvq7KJ zn)TFIJ$muBltAN`-fV9jS_1+qELqChD>MH0ks1_8D^GB6yVT+*w;q-_$S)OoryXoO z6u=%rkGGlsme#5j#^Z3-|NG@sa@ISa^58M~US59S)DOU?*u%3?DSN{CG=Bq^2M3_% zmwnz!Ey=1NxX#5M9{q@IxRbC#lw)2P>440S#6}9EB3bGfL;F0Tk{cPXo=gX)uCL2l z`{0{#?_l|a>?bPjb6bQI)SI_;Y6qNIQe1%vjU5Kecx8Pz)za;(-5gX#eU+O~B5>Pm zo#kQ|IE9Zo9yF*PackbYlh+=mfP#7pyln}lS_z2ARFeA))eRg(jl18-sfUyKITkKS zFvL9odN)@@iV;!kf4DR5k0g>r4Ht!Br#ytlcu_~x<0IZ4-2I5{&0n*!Z)6iVydC4? z3Vbfl6zljDwq5YMnza>)UVw6Dt#I3wbvdt8?(Z)jNsFIbgf&gkqDtAfpdc;qH3P0N z;$Z9MIM~SG-Y_Gh*YCq2qf|p&$$jL%$H3UVVMsoi7Bh7EXwRCQoZKH4v|t?DHQJvQ z16r-7l;Blw?md(tbp-$8WIyXwk1Mrvv=p zdAzL*ibN|iOX^Bq32zi^0u4Gxu1$VY#31nTb|X@|=+0-$6C(9g?h%-UR@3I0^o%DS z6KSdjye5y%J>mdH?O3tiC_6QfT0N7`<#XC{%8DQKKB)jH|4aa5TeY z#ju|E+xolX=`W*rHAc-iuo0N~1D0`mby2m(7srX4LFMGxf$kk1Hcwm_244r41!r#3 zZ(yEtLOYmM?k1Y4T;X#gS18>=)#Sk;m=#7Cm}>qhRn*w|&>;Ef<9hgP2#AnAZeUo% z{2RS@9a4q-%iy#|D|nMV9e#%;tP0OHD3*E8=Um;+1tuWBR7b}5o+K{HFR3*;=dk>j z|Hge39#Tcff>VrbEFzo(>{}KjhL{=cL=gxT#90khV@2vlHkVr+FLzGbf99!LzJ`ptqM(yzm1(1?jl|Mvp z!B!cvGTKWD`kB(3keLQR17Se;-^hN0z54pQQ#nt42sNC7csIeCh%UGw3A(~My~j{^ zdS5io(dNSC%N9zs#IyO#BOBO?Mwv}FWluXet2PK$HgXoZoclKt<^awW$8|Nb3y^h96uIxA)o7GgaRvJh8OnuNO5DyKTC>p|29 zFp~u|kJ^9C#LR|z^XfIN-3s3atvS#p&$;ZETzX#XAE`nc#;LnyG{a;2O8;;F*2s*U zZpZ&f;9vb4gnXh=V9w>}cz2@&ojD1n)!{1UFOTRnZ1-UWe3;jYNzAx(tr;?dohf|H zb?Rv;+^s;DzwSMd=GaBy#uI!fC|6_e!a&=CJhcL8RJWcD0&2K7$!0S9SYHa{@gNV5 zz_6QOv+-z~Pe5yd_$7EKVjTOUolX$?Hl3zDM-d){Hv(>#suUe37zm11VwOCGHst3w z$PFCm%aa!EF1LJE2|jlJCb?D^>-qjAyggZ43WhD({B*S`VyeH%dwAi5`aA&axCB3N^Gcd;gxtG9OR9d#C4Z7b6~~kD z0fy`B`R;$JJ$kGD{Ptn3v@%&}#?ZS*2RmZZK=@Z6a5T6h9Z7xCCmC|hw1nZmZXK%a zDxwZQi@kQ%RN8Y(Cnip{_fwy)M4l_ULqf__Qwqbv(2+6KvN(Z|D`<~$QM)d(29<@A zqFWUqVQUautw0K^T>#DJ4nZ4b0=#Z#IMC8q%;cUd|w^;jIr=y^VLEKbbckxx{<+iVsh&TSsz9!HcR z=8C#{yT(TtsL95EDTq2b%LpUyY7mDy-l5SZ@ZstxIoy5??Y?zeCzx5!X2aW^upSo&5SWY9dKJY=wym*Z}&;|S+b$EBOS-1|fo zVBb_`w^SbVYz<;dRp*p(jS&q<5fRw`fIS;a6T}!}%^Jg^c z!ORj&t8lF35no;Ibbj$e7)cFmhFT#5j@gI-ZuNcc-iK&XPJ$RYb#>3C#c09m6pCi# zd+tfqCZBEF93Q{S)X;G(TkgV~O4W}*dhUzK+|l2vr9%pZH!zvHjwZ{Zv}C_@ce5TM zv1Z?IFv#l#e}g5hh@=kqo9JJsep{=Osq)DIKuLUGRONSCUc#o94L@;qM4X7KRT6ki zc?i2P{zJSqkPw~?5f*xwAW?Qi5V#IKn!h=$e;I-rtQF4EtGeVZV9biI#G(-QPXH7t zHV5k?2cN06Jz;Ym*&5N%?a7jpV3s^=+_u69>P426b7CA(#yVk3h@nN_I;cQMn1ZX? zfSux7EcmB#)JC9vj=uI;catgqly;x>gZ-X4&3rQ-4C1vLLo*2o!phVkc(NX2lyat# zeO#%9LXHQ19LVg}sRpUu0m zFb=c8LKr8;yuR8509rM)b{@^PouS%kF0p?PrJ2y!TKp~TH8fs~@sG2MA>>-SY zw>W#)R3JK=mp8oafGcWPut?i06UK6XN;}$rWOpXtwhwxbj*5paDkefsg5q>(*tWxyd&&pqU&3i0&GY zI%4c|T&kP3e-xYQ6;|3U;zscT@r|c3#!O}J*idi>)Mn9ow0oKy?rlU|_ntSsWiP$) z+q}+RZ-ok&Ry!>fo_{YARK@C`5ADb`=~QD!Vek$!{o5!f`u!&mu1^DJVVn~B$z0sK*A`;_^VTyFyu3_P%oan!y?((s@i^rYd9#D@7GxDs0rX4nuClMBaKe*VYOWl!I?HlptocO#nr_1Sx*sdtLGgx-pcx0h|4W}ftBvtx^X~Dy2l{NL_N;FnutDhF~ zQ{h+gkVXN3Yrh@)kv8a-RF0xzOXMXeq(9kb0b46|1^a&V%_$igL(vJ1;AI!^&@+E3 z!Intrs-F^q5E1fT+bU}d#6&G?BR)ulNnOi>AZ9V*dbljQG(jNLykOtvQ7s^*xc@bEo9HkiGHWh zCzg&l?$W-BTfrHqpJG$?qi1A9WfVlE$)0&_!j=(R}bKT`t?#+Kq5FVS$&*6%A z;!KB1Qp>&f`&Z;TEx%ERJpoLSVAw2AM=iZ0mZXJd;$_@&$L`d1;?S z`;_IT1Z|7=i!+JLXOJjrnRkdEyIkErAyOc6J{0@qICZKtTs6{fy#WeIgGs-1enCU) zkr`)hI6<`iIh}_96=ltop%8mA&-g^2?ddkkHliarGrJqD8T>1ze{Qq3rUp{Kcg`e19-+zq|1*=ERill@gL@Pl-Spd3}yRoe4wXM(U-6kgg9Dr``7NNz}2 zzUC-iB{I^4@>M-7yBk;r%+xg-izUTj6Z@gi1?^2aSgsl%)?2Mb2S>zKdMBSu0Vx!e5WnIVOKYjy7$rP z(+Zy!a`VbDTc%i(M!wLzE(1oMsdbEcHozBD%Uz;3yJUD}ek&O~lq z2(##F0N&ptfmhc8Mvt@)!fu9M;c2^VXh20A6)K^y#-2UN8&O=h5}ow24vNo^ZSR<5 zOFAjTG3%SGE}o-^5B<^nYwurJwB|oP`WHC-@!&q2C%AdFg&=Z>pq*qJ?UFxoyah!d zZQItaN=i%!XV>>EytGF+xaqO$WPc{(gLkO9hEdV4LCu@5&$5k(s`NIILT?c zlPCP1Mp>Te5z??0i{U|$>jm!h!ESRo?s!9v$LM!dR?_Yfwa=N64L5N7cL@W$3O%kK z5C^1l755!A&VPmUhx-z9h50dBT`V*5bV%S&~>^o={9MhL{(n(t{xn`zd1Ql5+6L`s@G0H@IbsI*_%`X zR@EL8?t71=d-J!Ax}hnNeEih}$Cjfb%FidO9hga9Kw)4Y5heCCG9E?c3xwd)zxKIS zoO&ERi)VAvfRR-DTz(%>Ek3u9^UY2Hk=Rp3D z@D*eEO78&F zktAm2nf2nwao79z#IvYwJWn3W{mk-1twcgBxe2%j<;JD=q1u7t=~;%^diq;l4ySQ1y)!Y?ypeY$ zWpD#1SD@w1N1>>YmeLJ*^dCci3p=mJ`hWIt4R;Iq5*{9Me+9E@PZgDhAT|SsyGVC) zCt2D4qGnF*;6xO0cT#zNnd=PV+BqiF5wcFe4t4Jt9(tZSp&?5ps->kilXdeXPCU6g zv1i3S?esfH7F z-3~UKYsEiE*w^uTv*O3+7KwvC4NLkqf-LbAWfXtU27VR`_1aTPv%6~Su^7>poy=9h zgi;-Y`qB}-lA9GQP}Gk<8uX}`WCD3AKFw4Cljxao5^0<^8Syb`Sx4}A>|%3l~k;}Ukc zed8}vFaiO;0*OfbQJskrt72V-Do*UM2bL$dM^&3AKjzN0n8DHo>b$(?ox@=UC(-Kc z83ifvg3_@Y6*?aAi5nh~i6fulazm{bvn~eF@M*?B;7+8l9N<6mW}x)!V*w*IiJvhP z1~*1YzY0yW$l3-wpl7U%8bdT1zpPMwX!FR~mvfez98cqVEIS@D!6|VAH7)i%W{ief z>>0gx8b)Tc1nqHdel%KzJ5yFhCLZBsVkUMW(5>sPz$<0@*p2rdV_RJE<}{+Q#{+>} zoZHmTd8&=^s8-F5M#&K-V<4R_ZunE8rjY(urg%_5q{;Ra8KvgHlA`=%;u)n#QSv2i zB`x{;v7bE+iU9ZkyCp4apq0ST!V=~ry3}YdNBu-9dbU^U5*12b!d?WrttnBbuiwUjy7N>A!L@gZ`)(g zBVxXKHRA_rUM;k><1DZ3<;4UnipGi32y;p51}EA`mK)1!d2{MHS;zCMq&Ly$lR4zv z?^L|fWA*IKbzLlcj8?v#7<_5jF*wRI^!KvB837Jk!dglhGcZKdXJ9{u;30ev8A!4! z`rN#a`a9_Zai9D80qX)c%JK=Sn>BFJhSZN)yL3n?lWkAs1?nE}#g+DUjYKD=dj*#! zJJV7TAX!%xhd8ni$ZLP1{O?uVptf_YVIh>)`FJvQ{E6K{DjR>S#LeG|@7}%<%n5DM z!?xLSM3|9qoU+0?G1UHtiI?Ls@v`?%T6uU-RZU+>?J50?(Nc=CJPi?Gf+=2NgvVx7 z+PDIvnCEanB?O3|SkAmzP|iqa6uPnGpL9+gv3*{dS|Ku~M?5=tx{O*N3D z^R$P*b>3-lYmnx9jA-3em*xf6W>mVE&qpfC39I<EY%1-sg+N&h)mWt!!m6tQgC0o`XI+b4eC)oJ2QNeJC;yLA7I2OYa z1QeXT)wd1+EVi18d2hfT>>cq*@`hZzpbxNx>NlMn9C-$LYTzxy(x&`o5#u*!vuSui zGln3H91GR#C168CUFr_$^#jaYY*;yRNzycRyc9%xUf?+nBB)5Wd*8;)uJ~;euJe^R ztHhq`n>Pa?CKJA{8ITm?SaxB7ve*la$dJIVRpnTWcQwYPdW=Q& z98F&MQ4=11XnjxS&7%qmtDW*O zFLe|1j+uq!T)F7x>Jso8rM)QTpzu42o|*2@GHFX`)`p@a6rQ)y*8r z!=61-_O-A0!hZ$F|JXp8P#dI)5pc!ei4CDQK)aITI`5-El2`FpHll(UGwk5T0DdiT z7bmtsD~`mPCo|*7$5~riK)|p?1JB+!kTd6D^-PFXYcu_s9!FE`$o$G#3Ub$Y?}{=d zL1S2D7L>+S?SJU~Va5$AwWCnq==`HW!}sTcUr6C2ODSfi|3lJqF#+z9|5w*nKt;9v z;mR-|DBaTCA>APzB8_x+mvo49Hz>{Dx8$7*F0{@X*514cW&R~-VYI0;5kHqcI^AOn zFi)1KtRqmRQww>4+GnCLdxv~huwdp8JE|C&MM8muKC{HP)Bh|IAaUj@BH^=NE)WQ# zE#UPzvxJ2H^(EStl_p`vXuei!x< zXM_qinpts4(bsPrBgFJ4PgR45d#MOd-6bjOP(I}d6HYK#Y{&YZd@p#w#;={!p-^s! z(u|Do$*p#$AHx?UI4>pZ4~Sy8S%16aY!airAZMI*f2g`!C)W#oh9NEgjmt!3B|fC` zJ^?a4v8hue8?6}NW(?)!QJj=_#|losXq1G(`9)b#@@Has*}FUfv!H=hb)*Frq@!#X@SY~81orrI& z_rK%X+cYMcPZQ=xvR{D#kI0lGix1J+=xbPGHO#7-IzPiqdcX{b-R=v0pd=0D zvEXS({Wy@s+nOfX4MSN;L@ag#rfAn$0trHg&iA65H_Q0lyXSkCK8L%j?=82i;-4e? ziZoMZnr72n{T8xL?KDLc`urB9juWXGd3R%6C^(0tMC4BbN(u7MTWlyXWY^_mhXVLr zDH5W3JArr%GNDjygEzM+OBr?dT@mdLC?iqlGUqXvpDQEqa1RI5K)A%jhzdj&p83`N$eI7{hb0JM58TmIUtGd#d`NgEf{Lo`#|Lqj$th&YTcpgmP&Af?Sm0 zPOasAdy%)TKQv&EK8{9@P+a!4AHiwn>oq?VqJEeY$v;GoB*&KK)|WuRo)|maMP{v6 zTen`rAmt}paqQpw`Kj=)ZUeX{U{LCHu11*rgHVY}sxctC16l*7XgGQ^CCXXh5&M9r zEaLC=0FnY6EQl|_80bL23^7Lt>D}V7#QS{qE{=R}2dElNUeDPE0?SGx7vJ4)B8~b+ zrtk5=3#14JaXd_a9s|siUPY368;955hA9hCUw?WbqjhHZq8r59Xi|2p?n&ey`1pW! z(OcM|#3VP{VcvT%gzUY7^~skAqP%5WgM<8SIH}v5%?15i>rj1a6Hc4^tc_&QY<}-K z<43IY)1EJMU;e4nFT>fq4!l&a2qA%3CKDo-L#ZHNJZrJyum#rZ(-E8)lB3P-c7}Q) z`c69ol!xN7H2(-oN*i*)_3GtZD1I@~nj%!_y1L^czOI7#M%?L+OcIXcMA|2s{Fg^Q9 zSdHJ0h9x{jRRv8V;-+ViV2I6_)@L2S7}+N9S@VAI24y$H%;szPH+a^|&3zg|Oa*i5 zan7n@gd?1j%P+%D)1dkIa_ZGOkEzGk=zVn4rZQ?-n@j!N&%b^Q65TH}QMJULlJ_+~ zx8;UP0ya%bC^AW~{AkEtymeK%!mRxU+3Zk{vA=3nh3T zJKr0JhSu#4p(KQd?RjxMQ2p>yk=$E;h`g@_CY}vkyNlLvj}=5jh6Hic)g#UNww!kQ zNxziISQIAzJhpHTqwlcQ7~W0Ze1g8X$w1?*)P4Po{OG9kg7Ip{_$Ct!YALc*Y8I&r z_Rle}7}Yu!Xz_hD-J*3yYsBYz*um1yC}FPiv8U!BqZ$lMF@CY8=ba zab$PM`DaPs{ktUPguCIx;v_5^0Z1Twz_QP#A>b7ieK1GZ^}8QbH9l+?GN8FTqJ&Cd zqK*lbA-cN8YWv{9sRg^+SC#mbJr6MbI)f!Ka!@U({H zG2;?xRZMFiBOl$9VM%e$UH=MKA7fuGPjl)Pi$q*XIX7$e(HegMtowz0oK(9dLvrH7 zWM>~^ZhjfErsj8KgiJeCH)6SiUrPp+gwE2$JU)@D$V?-VD#xIu_|(Eo zXxy%%k`IVP?mO~rdF^;GBIrL;TNl~8KlkW)6;4MMTS*8%;nJt#0#u!`qsP9|u;kvl z;1bLic`-lii_?9NVywPnJ8lggpRHV)EdBdNEoEKUe_PBWLw7kW{m%Drq&Gn^cQl z#;P$38)D7@Np!42GR5I?mNW4;JVr$o6FKbR7-WjulbwrXq?~4<>_yUhTkBS^DRoq= zj1Ic>qEKzem>D81(KhbU+d5x=Qk%{2@^C^6a9OPs@fFheAk=cmV zh?Oh0KOhc9V_G~ay1_$Py*56Kulrg^n9;3F?Z?xD!VR*V+dD|)0HqeiY!uEt!(fN- zFMHG6Gyt983WmvzMm`|AgZo6FfFW*u9{%YqM~H`daKRvY;MQKhWrQfkxLcIK#6kzZ z>jmfqG1o?g$_2j`{lV|Wc(q1xqw~%tsXF;kJx5#D5}yy-?)Txhcjgg>r`M-S_!r{kFaRgkMTG-v*gC};U(5O>bS(_Et9%{%u9EUEvi^lr5ePw{#*Por% z(;<=Fpo{6x)TD2Mx%nx{1t8TxUwX$>xJ3m1+4I~xVX5Tjv*rnEEJRSkM}lxTCQV(@ zOxN{EfOvWFLDG~AOA%3W!7#ED2L8Nw_vp8@l#V`8mqy0%!1__%_j! zR_jP*eH0K^alid^8hq~;vbamO&%Jyel8)X!I5>i6b`SmJoyEhyp6#&=sQ-^K!qobbfH4HhLHw>1Lu7*% zdA#0c%qVSx{z?md2bYWreJ0Hq9HskH7F>)?mWRVsG|M}V&Z-qP zi*@N5$Clk?wYhc?bRS0$A4i{D`jubBZBk_rns>jaPKjj7u!|plkf2vN(TFh6m+n6@(Vm;$$B#FTB!aw zWtv`yJF+kH2Q%GXJ>!gaJV$G>P`5C{qK&wgr}U~bbXt|RkQICEOR#=XL zRl}U~j)+Htn58zMbSPc3_(z&lPw7ic2mB!>70GdnL4O`6#U@59s6i`{a{AsJWerXp zz!T}GX&@R(au(IrX?ZF#lTQ;{q^`Tg;bn>06MY#FHH4`qc8DW&*Xo;fDdor3-Spf~=;$)X{x{NW!^9A@cAUsHrY6{R>&crUB8S1XZ?JBy zxjN`xvv&|TtzDgPRZ--yVy2Nf9gClJ`uYgdnYyAuW2~IzuZ3dnp^xw-xjK9N~87>){9$hZEMpGEs zFyws9=Ij$A(5ShMz+a8l%nx)1Vsvzi1{JG4225X>7=9j#8Pau&d7vntlDd^C-AT(R zd}55$1~Y!9NLA268+OC1Z;&-O*}>WN!8T<>p8$Cv$H7OzL0$>9+GUnZNJdvy$H?J4 zu!Zom;t;l!)*XTc@*{ta^4jdm>=&%oGqm``5egIup%hK0RHOShkt6pB?=|lC>)`aa z5Cw>odl3F(G7>-taY}RPK?Vs>HUX1qg@iN1UE#Z2U=YbXUNC%MVuo8G2!2JzyrEEw z{6=s7b2dSkjG0DW44Sh0x4DlOT8YyiNc?1{BB9YHdKP2JggL2(MH8=I5j<;{4Sgyb zWA16cyJ41^O+NH96Gu6hJ<72|S`47DN@mr60 z+x`AT!d_hpi90d*#;`Xpq-aX$GuBnF>3BC=Hj25(+d{OqW%0A<*uZPU! z>MH^R2!ENVVn-f}lsb9Q3(ul1f=T0R=HIg)#k4uFuDx-O&z>V<%mSH*oiA4NsPM?E zm`Z}CZ~!iIzTAiMUkAw2yp9VwpKY&JgmX5M}1jihzy~N<*GgkGG}a% z+7`T!$GdOawH`PP4m8LZTUBgmDmY9y3Tn}ALyBgrCd}#7xue}i7^6L7u9wQ6vvA0c z2gMY>EgD$uPh^=X?olr|PHW$Xwhuvv7&&+jIBR?6jEn&;hpEK*yiVGNJxG7LN(M+V@3GU!N51|Hs*c5nf2!m0^Auq@E9au4cr(IYzwJHepf(vK zH*S4)4_S@`x{(MVPkZ~~K?KpthzlWi1lE=if-n@0Ad;g9@R;G)Rvb}cZ|TEy2DQwU zD$OT3f6NOpSD-p!Yh5M~7)oolq_6(9ew>=Dp#3DVE zO?w;XSo0paa|%17!+qUIOeC{y+08Ee?-K}>&jn!mD@f7uURtad9(ob7s&Bk$7k;8{|7+ar@% zlJE}mI4_6J^9`Eim*OF9?>RDf1QF`gk`f~oak6MdfoLL_9Zf2TL0+O(YPvW1j@I}$ z9-m<9V-6@X3qsY6u~t(s1+`b7{+tuC+I5chL{0s zuDv$>tEVVDcM&K|Db|m4*<(~--0VR>8P2~opsMnUki#lM#`sD5?~Ue*S703P_Ku<% zKssL&wZjfuADhb0kl?hbW^{^td9w!hH#Kjep9R)pLokGFfGYa1uj$k80DrRs%PcT-MvA3$yN~e=R0d!Ws84PKMC%X{<1X zO3~)$vvmg)9xQ~e;aov?E9cN9#*}uzZhxi?P;<)wrDLCO+w@@QuG8h6piP>O6lW)= zK^@AjpFU6@_ONg~Zns;#3hKyP`$^bzWoC}fUWgh@Pc7kF?1&#WJYQh^LA8@MZ2jSf zXiy>;q|dqpk8GjzWJ~EpEHe{8Tif9N7|@i={|xni>mTpUGCXWuPFhE&Bn;#U4ug-H zbO&#Kri5bCl=D(yFYkyH^ZesFfH6=3`}=_V2`1H8DU`kI(yjK!6A1Mf5mSb(mRD4* z7P*6g_EUAjC8zXM!5^(r!twJzHX|ZvB_I=C&%*!4s zt5N$)G{aEtO2a@`ouiUb+}Fhe7Sf7S%?J!g4ZmK}Zx5h9_aEcxG-5eq_wU0QwI zE-ANE1ahW)1XNPD0V>d$2s%(0Jq}X|I-dk8JQthdH)jDL7R+*Nv?WF9()oQqpe{pdu7pp947m)hZg3{pPuH5MMao|B*k zmfh!~SpJS}WySS=)t)B3D3M2~LM?J3zfwO&`R|S=6NVV73$kL8KlA?969#*AY?B7V z8vP=)^LFjMIpYICHw)|xGJ+P}kw^FWa4~Od8n=rPqX+SMLU2hNJcj{#q-Q|5i}2J3 zXq4~S>XR8CRM&yR8&Ej$0?=kqwG&@?jwn=>#r-F68+rUZa}htRm68#{6O~BS1k+P{ z6BUwOwqO@I?+*~@TKUg$^RK#SI7=s9kl4hPGcvJ;Ysjv?ZJ&K(K>nk3JXbvTXx~BM zwf;M0zR`7i<0n4k*ZLJOV+3}AC6?5ol$HIqH4h7G)Q_++4v86{b`ExV32ifYn^ zlLr zcza4+u*i9Wg(+7tJ1v^(sSdau+RcxD+Ig7ii=b0WA)81dNimYW_5Ob@ zFx=i!i*m5kPGJULGt>Y?kyG6uJs^J(6>P?ot)DUS>$tf($OEE)K=h{SqoLPQ1E!EN z;h!&ugx)Drqn7sc>+~3E$6IE_b{i_kD4Z5`zl!{lv}Ag$%;F&c%Z z-mA&6B}DTaYaXRFjyM`UzmjG3?~Rns9s%Sbn@&wW&LoS1q5WlKflG7`J14XLig^I` zPTXV}_1EDFX)&vecB1lz9GTsJ@*Z&TNQrXdXMV@YtKZ%9l#mY!3^^8EkMviM-#`9M zP3Adk3e<^O;a8&l#|KdmXn`1IHzaWKZ|}vLCNVCNp;x(y&saleTR!3Wz;REJrub6Z z(=wGawU?Pe!i$T~Iwo;xTGiMzsr><6nQGLE;;a^Pgj%TZw``5SF8pmi+VKcjU1wG0 zEUGa==r6z-1?a!>0QqFVxUN0LsvEg$A9nj|tI^zm0jda>pk>Pp!*kh-{d%C>{?~f8 zBJXVTeRq^|QU8}c1b<-<)j~M_*JE`Y2x3ULRhHwg#-&3Z*{W)+%pisX^aeWx!aR`Vw9c%juI>z)fZ;N*SzV?3|6PZ9)Pei(%bizp>ywG^T zjh8@jt-QE~1M}SkWA3znpoy@%V1PIXXwJ*xdkzM#f|w*W8j7S{8czZCH~0>Ol)#_So@my3+9OUun? z-`zaH1To+tZ6#~^KA}f^o45Sm7uALfg4#T8!Ot`KAQmciF*lmb6}F@U>i!o2*OEaF zUo&GZfZa|IIYJTq4cZ+@9beePXk=sa?_+|Qd0;7Yz%cexUCXpL@n>)~9$30Qpsbom zD`PWi_XguPa%9g^)0M8QbA7@yYCE-c3SiGp0bX@Lp;s&H*GTzK02a@TbQK!TcvbDe z`y_BAlezMXfs%)3I$qm^BFigpe58Juf4AX_=Su0I_y3~~;EC1QH>ci8mO0=*Qsz_r zOxzNf@x9oP`y&`3Yl*!a4Z5yR!fQJ|0o(zvE&$5@3lkk(5P(8~jliG&jfvB;FaV_O zU?^Hpq@rLNO@&Me!*St%^|RZ@>z_vrKiFIJDWh3v_Z|9&wi89wfhQGle8F%JT9V@_ zbUY>nMAg>0LEs5)=pj)0dz?o%e7z+n-g-Qtq++X_T?ps}E}a8^W`Ql)4Co=iu=;lr zpuPZrBlrz#!1QmFe2O^D55YJeaSoh3g8~#^2e^XQC)Gi8JTU@~YDEvb5t&N*oajK^ zwi=Gm#2=Uh%$v*bGofoN))+bfRGP|{qDKQXQ8n3-gLNO?cB*0P!$tfu?QQ!2lCR_u zJOI{wX6`-ex9kpr$#TA+Yk#$x7QK`#5FCYcUBWo-RWy^*m;M0Q5{ZH;)RCRH^%%OB z!^YQKNd91iMGjH4mtJMY<9p2&DSmR6_}QS!L=3Mb0l*JvXr-Vz+7(%2i{nikYjHDs z4IKhm6agm^}ffL0L4RC3Ew8EJ94W9o|H`PD=2XMvDG^)6#sh0uS#1rk1rE zsU>@uZ%_wErjI@vq&q~e{(#mAOE>GekvkhghGYOpNdh;8zhZla{I;jQr|{_<=qJ9; zA+l`%`CHRB+YuP;F1X6pAh_xP0jKznD3#PI!>afklC-uGvox%vre$CICtv;F*b6;l zT$ULjjgj6@bz@6>Df9Ju<*vBAC7*pqe!(F?{t2!pN}=r71@61&aT`yx7aZktbVTYL z0mN3@eYmZ4U5$D?awwZXZph5=uQ=)7*AYnPiZ)p%C&P11vE@vM^BKNw$+gkYi0Y|9 zfffb&LceIfDK_dy1nbbF`*U^uL}@d$q5TT^66&9Mtq;XdW4Q&dFp|!IBp{ zH75Ko1nqR?;{d6&Ynvx(Ww!F%W-1%20E3NbGt#QbOF~|M5c<7w%Ayy`evhu>wfGv$ zK`Mq@pe;5beYFv92V6?cSEayj{qyYp`v9QNoYoakf;XrR!#Mm+0StPCJ8$~cAtgar zh;*B@IE?Ozw!^)nDwhu)v+BWP4upMn2k+^St5r3S$VbwGICax;DzlI5PbHCMbcWxI zuy4FG7jiCpoJKAlD3Ud$MAlGkr!ok5Ij`;7lveFo#M7=#m_Iv% zgBYJ>7RUA`EGAF2-?@X_1m^+(-o{Pa$=g!aZUVMFe(xJsPdO5iFtO2>C$ay&vHv?R zrK)itRvhVvx3z2-A&YK;XUpIrMdvLi$FSO+tu-Xfsp#|;P4U<(f2ugp+~ewU%@&c? zZ19Bx+1c{R`!s_KU|X($MyY!=;?dL2BYaMN0qF(a+Kuwe-q*|NlGtp00R)Y&N$uo3 z?wf0{&z}IQ8nh$OEO1RnrS(eMYL3!wgUYYk-h|#~-Nq2)aI% zDrfd^BATefsQ!uU{+~`Ve;d>poX+*LQ9;L267?%^gO}EKXWlK0Olt?53B_1aIUT`+ zJOKjB7u*~p8B(?RqiJYMzb1emH)@aJ9u?7m+7Tg%HCnHL<9a~;6FtBh<;A*!9NfI8 zm8RT6pdu9-ha;tX1}AgTjEp3JD_^GX=?@TQWpLh@sqspqyyCS_e@?W@8k11)!WB$x z*8}iNUN=+M#nJ&vC$TGkhPRY(ra5nJBXW-3bBySEMrt!5jal1OM-~sWHqWTuo%Q7G z|Bo%Z4fyltP!63NV21T0c5|EV!284`Q+0D*J?-8)2srS2A^#TiXgv=LDa8Sq=w22G zSYmUoo(S*pwAOUvfnG!~8AK*?dH~R5wQ$E+n^AND#(>tV2F21(?Jwb0wbe7~gsQ2F zVsBw1P||1i^x&m=yjp!UlscJ9d#&-zDdF%H{+^B4WP&pmyU0LhOs`m2qaz3ZMPa$K=AM+v&vtFO(!^iA z73{It|NUs~p}uqA{?h3YMF)Gj4tIArqs5^9mG6oIq@Aq+&^+(#lQprmyeP$BM$i?^ zeP3|1KBH=eA;lh_j-kq=(NXnH`EuYGjO2M$_oNHM7{V(V%G%tE82@whU?T>XA*PaSj|qj2Wo4m%l@b1DY;OP46Yf-` zl3!ri5I`EIK$GlBXdbs(*XotepQ_yd`_4lRPB_l{DOWL8DPb`17PG4RjSEf{PeW`O z{(FUB*{$$G=~KoxPf*ZIAR5JwYx9;tCztE_W?})R1cc-Isvkk%JDT8seLvp`oZ<|+ zCGoxBa5_K=SpxiuCAx6CCGqU&#*M?P?(aI=Ox45`kRMsUYRNtC$#Xg)y?SS4fpYhR1;B)jOc>0 z{=ePLKgWph22!wlHP-~z7D6x(xd;O6K`lVL*mq1;+*q-1F>r4WH8aKe)#9j>!f6gp zzVN}xQU^gt>z2lv*caRW?ElPzKJV0>egUS>LJ2%c-d69ct u=bdE5|L6AeePRMa{4eq|_-mj2E^oHWehoK}CB@$X{}g1NNLNXky!byBlrzHs diff --git a/docs/img/tree.png b/docs/img/tree.png index f3796b057249b7e0ee1b63a52270a623d1682fb6..06750aa4cb2025ede172d899215fe825ec57f297 100644 GIT binary patch literal 128024 zcmeFYWmIHMlQ4)&U#NaNnPySuwXURg|q$6%+Abv_MH8m zv%j}a-nvqUsr?QW_W)3xzI? z2ro1k{gti-TSc^ntemW=5v;Bxtn+JNr{LETnzJ9hA!F$j{#99^O%G7U^Cy$r<$5c~ z^n-^DtYoN8ERRqH$zR8?i<|LlEDtGVl9VT$3Wfds`Qtp7B^L($xwSZp}6bPjC@xG-me zLs1~VaYog)WUH*O4(dKTSYnBF6E_AJmZr`h2sx_z{5s6KpjdiTVcUT7@tykCSV0lu z%yeF)x-u-|^82ed8;mY?K8KMHNC|W(D;Qj`92G`hZiqSoV+=z% z=l$4CE)mqRKt!t;D!5xs_+Pa&>U<2gBZ9ynYWQ#x(gp`%dK9!}yOY zIwSIuY6Ib}m~ioVq~+hb=trYtaO*tuMo@kcH=)Vw8ME|%5<=qbPILc6GEJt(n?$(V zMbMZLAxxc;hf|LnEV5s|U3LbT-B4hVh3@U&28v*M7>e;3?kc=#w9*JN+m3XpC1UUz z$249$_8EI(j9kPAKJC={JlogO-VO%P2yzOc5mcoP662JIfj5K40pOq@sbc$*k&6Pp z0?e;$&VP1e^5^;^4FE8%{WoGw-9_sU^l6fR9|ejDRz6-gW3nO#j#3R^8mkSB*k)JiqdTSf)BiJW8vC_fa)L@thy|yTDyB zwbCoA+%S5d0xcnOe!)Yud)@bzYoeQc$|hW1|IFTKqoW$^3N+?gBMl5!G+wg3xW*fe zF3u3Mh=kz;q<1*qv>1N&;tEUjP|r}}&?ylgKsFb-{$nkaEXD*T0_*ehAFvkfyWd|) zhke=SMtjjN^XfYFYa2TZM?lWO4j<`^)mUItN+o~Zij4aCkFPPnj<32-zU%j!bVK-) z6A_lSe9P@OL4n8tQ++Bth7f6S)9j7R@F%#2`2avgLb3qB3;bzoCcRta1`U9}k+A*I z6=2l-IdZ;OLumIR*ASe;M&&>g!WQ>JPlHzl@Y+{82jS$PbU;GF1;(P{3rjeokq;2o zie;dp4B)^Ek;QRSKud|RqLGq`nUW*Ku@(@P3O4~x3t)H}K6+Kgwyi7bxvOoB*-k-$p3DQzxR*!L7lC!WC~f&&j$=P@v<9%H;9B zJt~E14rmS(4j>J54ygU0zCo+a79~50x0Vtpz(r3_l&2%DB)=ddBG;vINf4l+rN^V? zq5Vl$NC%(vE0HeIm5xZYt>RwA!oNbCliDlJXhi4s`>mrRLu;yc__{1{PKokYT7g6; zX>1el!uYyS$6t;qx8jY5KN?iD1Z?msqg?}Cvmb?iR{f~%Q`#x}qg<+Hq0m}!tCUn>S%sei6l+vy z6n^kOrCiYCs7w2)6^pNJQ>E3oYS>6%`F`r`BJqIxpz~08%DXUCDmVwQ&@c}+x99l9 z(PWu1?JhceTyzfrhIYm3$q0&FEmBx&S=O=FvyZd8Sv6TbT3DF3np0X#OqVSJ^A-ks zaIq1w$wO;1YO@_OZ=}|~@qSA$?iBTkc{I5I0rQ>TT-pl)D(nbtzjAE;ShwXf z_+b%mu4G)dw4X&gIN#6S@7!pSZjqfDrbOxv#JvPQ^8$L5K1fiuNgQJYIU-~wiC zawd2tEB|M{mH1Xo{wnk`*7Cv{e~sC)#464T*J?MR5hs<)pD`A*A#WFZml1~&mzKSf z9q!56k=m7;#)K%0;nLNUGPVhxd5+9P4IRIJs2R=FGv*VE|l9Z zX&CePkCd zC-B@@1M$&F6tQlYXQ`A3R|tQKM@491xp1xWY2j|*LjKTTDgSXfs5Nvr_`#CJE#T_? z@I>tp?NGlrIWV_ZJaiT5mxB7chHpnMPc@IdmO+ZcLP<$?_H}=I|8~C$-iYc?sy(R~ zlUVF*qO`QE^hWZkAdB&CTKGnS4t3J^Hh~q#+sM(QqGQ?7R7c~oQZx{f117 z>=l-Cd{;jpy^oJ433XU~@?GHBllj;>$%}E-*!j-u4gFJRAV4T- zTUE_{v8V#H!m_ef59<`~^ms^jP;@|yZ3$QZle!pBE}f-P*_fUV&E>%3pmOvKLn5qM z3U$<2O1X=Q&saZEOUf@!faaQ^gz<@)-{K_z@02Z=jd8Lewbl4=u5Y?;%XFunyMEP_ zU`mhqwKv0`NBfY>Y3`_Lf-O^>uiDFZ`HcvH2B8k;O>;O9Y%npmZ!#)OV;+sj2-snw)vi;A_Dj3&MAnkt^=O2ee} z#xIxb@+Z}$GRSgJ)slX<@%t;%I#LSPi`{ztsDbiIk$gIIx}7cj+N^?+qH=9gOQXSA z!fL@PTeruXTC=R3_DU1e+RwFi*MSRe%_BYaW@W#lTh9hPt*v^i*{bM`-uCpz<%^ZN zl-g!%l4a@*&>N^>tNJQtdCRqDuO}<$5O#?qkA%+V8@?&$J;yCUdzw-j5G96peerz5 z`xxa_u&EihS%qWvaK)EycYb9q zw^%*{p4;iXVrnJXN$&_b8^pck-g_b3FSZmB`r-V3r_iqL)lUhMktNC^n#eTbjnR8- z_kNSUPMTBKRAJ;NU(L@*x&Sar9WCMUxRLjz5LTH)1vJuf1x+%^Zv^5 zS@*HD$=F;)juJ?sH8YU`{sY9>il0nFRskSl=V$`pU}R@xCKLDq z008(LjZJ|{qT>IB{`VI@nYpvGJ&=jX&CQL`jg8UH(Ts_OhlhuWnU#r^mEkW0gOi7? zv!OeKtrPiw68XRBh?+PVIa=5|TiDqG{zcdDhnKcCaY-Qs^Z**g8# zu>KB^>0doeER4)d|4sWZD&N0afeIGxCe|9F7B(ifPJej_uyC_-^Zf_H|D)@FSpF|m z&Hq8=WdAqJ|JCzfn0!qC8o~b>(SJ_Yf3*IcFM%(7O#eOi0$-9ySC0RRgU~`$Ughr{ z;$O1)D>>1>Uo`)D|62!_bhQupdH@FY4NOW@NW~reJQLPHMfLGh9-kH!E*G@vb=4`J zHmjm?uA!i@)O31k_i%Mtc2;GAwlw%lGJy`D>zT4y03u^Zgql72=W*} zQe|bU?w;}jb-hl4vYa6D^73?OX=%~9_Bjkm!O;Q#qy(G^ZIe6)6)F`xx41!~FiEA7Ub%<=3*e2XzmQ~55nNV6yTk=;LIzyS4;#w$ywQUU;1R8*`= zOvDT)xLvhdk!w0jT&dG4v(ck9G~9`Y4b&ISk#Q8dLkN5H5W^W=PWh727EsR6_C*b;YUZ2Lsw*ojdYc~fC`q-e zj84`+T4<600ti3k$do#h@ee!b(bxhgiw^fR=jB$pFMwT3l^RLgu)y3W*lsEK0qPH~ z&q$2IBTyPt+{NfL0t*vb-`>w201uY#cjP5YK&T}rOEDIhoz&7ou)_k8P$XiRj`l2Y zi?nJ0GcL?vw9z+#f*}NOl4{2}DfoYu@_)PuOQPE$!PXoSaA-Bpr>(PY{Se{E$cY`O zR14ZlVxxe!xj(TdE2RA03s#O6c_TvRApr_sFqG%^mC|G;jXbUUVe0avxWZA8Q~cS( z;sZ{s%#a4iIbo8@=}KAn#ZaGBYh*_ug&c{XM#V5G>O+Qk{?TLEKS0^$0N}CE^rEA< zm{LO3WmEc4Vefh#SeOJva`|A4^iXwhZsINrO3;naZ+RFyXdJ=Y4C#FJ`_`A|!Q91) zq*bC=Z(Z^QZmlkb;hsGy<)xbO@l@IZ3pQs6czYK~jnlm2JRm6dkD}r`+)aljtFtJV z{?UdDj($yMzXv6CzWghs(XBG27+XGZw@PIc1Hw*fkM<-U6^rslSERvm=DPAXLpPcA z*{vuRCaFT7$Ss-ZSUxSd5)Z5Fm!#-KH>-yPo1b)suZ&Sczld5>7OF3IvB`rXl9C!O zeJGKL(*iGkYt*(StP7!1a7nT9Q!p_5#{Z*b??P%X0xiY=gI-!q;5ve@(V#=+duK3a zG+v%@iA17vR_0;qdf0TO7eF}VoA&qlno4`y@81ARlVY>`I_9y6)tjn^Nj4UIgckzq zYu90MV$#xD%>wb**q4uy^W3$k5vgDPT7i~_fuY8;!k&`A*1LbELUiZ@2pA<#!L@s9 z=4;ReU)6DQB&mjNk}OdA?q<({Dwu%Xg`zS<(_vTDk{UU8h{dA;|+B;FWFRw&7bSm(3E zEW&tHiY4aIf?d;pq)!9Rq_&IT;yoe2Q4w_m0+g=*ICz*|x+yVTCI-VoOJ(`9!?Uzf zK%%N*&$+Tc;R`>TG(_&}eD%-y>!7-=w5u zm66c&;=z^RZgkQQ-J}RRsP+K%MKDEU(d#p})%ZRp>Jx|Pv*n=_M^WP5>ekbT$?Vyv zS*k|@qiF@1zEGBwG043P@+E^Fm&}_Vx-;Z+*eAD+84J zTR$z<1V^yiZ<<^9SjRvG2TS6wlk~%M)1#qu(OAI7NZr0wQ#$u;9W0k3)De)X=c_b` zE~{cuXdg+ysk`%2;VLTeQ|daB(TaBA-ZuU5y%mdNYjGbDw1ARKNul`-vBSi~N3mp+ zQqZcrp=FG{q#4sJvx0tWvPEpFEEE8KyV+-Q3*OiHLTA@(M;e5`kfZ{a!ZfJ`$mg^H zH7*~_)x3#d$&6>WyCkzhlNRM=w4hRhd<12?&Xyoat;Jk3faEsa_-2PU^2ii9KN^26 z6>V>}T^^tBw*ArPMcjM)=XFk;-oC z*nt?B(X#9%NzqF%ccr!hnkuPK2-FCN1j?NvnW0Ws_$`hlioEu*->;z~Be|ck^C~4n zDTEMlo7rW|u>I{U)0Cw8QE~grLBC7b>bea5TH1^6OLU3GTgIu}Z5N;K39`AK&G>o8 zbEqB^adcRXR|)Z&EIV}>)~1RE5yp3^N!T~@Tzd=VRa)=X;8|@91mOc)8ejJYR4$aS^=>+GK43qc$ZT*;nkrqsAaKU$Tk*PF$C;Ra4K(^Pw1l&Y5@ zXv1e4bnw3Zd@I6X8oliw-XN}O>t3rY0HmIkCg5fDtv(%nabZId?u-E2VjfWYs2t?` zUL))CTciPw3nd`xB0)*{QZASwI}YLU%X^C;Xe-jBQ6eN-O5bw7S|wZjIecyYd1X zID;N-<}dK+y)c^Yv);V-QLQ3>FLpuk!tr9g2uI9Se(b<~m6Gm4xX$!%M*Kb$#7lIl z{ph=%iOpi}aeZjt(4p@9qz76PN~1_wl=6lB%Vbg}Tt3<-JDI+U?^R{1-&xl~_9ZV| z6UK&{1}CS2zV?hj7jeojy5-2)_VNvmf$^fkaoxTyXCAMr7vH=i2adt)_Y<$ORsVXU z{PQ|`%h&lm*`eZt%BxmOef#9{){x1Ej_xy$%zz7#m1iKQ&yliMBZ%kZV*%NV<4Uuw zTB9*{C++C2jq7da%;)??dr4Wlc2#)6b#G#{3M0a4{w&{BF^cuXM!9C3N3gTG6oh(t z@iOyV;NH~{A7<70uw3oPj)Wt<|Pm!3OM%qirOZ8~WUl3l1$O z-C3l)1Zq;!Vqvntjds@drC3RoKafm<xiS~PueJ-T57}+ly`Lfp3d>TvIM_N zr{vP<`_-x*wE93(p!SBiBW*4=wkc|gF;A7eb%p+|tZo&m&8=a$_;$Ih_%RaSOE1Wb zG&uk!sK3wGO>(nF0pnDCa6Ljw%6+kwf-oDMPhc80Jccz#2i~L81%u6O z^-=eB_H;rZxsSpb#uFzx{s`E>iIn4%i@8o+jzSnu?f3hy;X93M>JY=T)T-KrQl%=E zRjMx%ub|!tm7kRn0GE!EnGD5OvP29dtCq_quZ?prDMfWoDg-Y~m_pEi=0l#4sjZW{lr)L_bMHLF%W(c;$HwA|A9;=UTg#ATa;jq9x-0F(}S zYtT9D!FY>6o+yw_jlU%omPA>rV|xcRr@M`$Yr6l?FYa%zRXYl{FJ--9xGYyyj7r?{!4zIu(4GorOq9PluSQm! zZa2~%SRIc((&_y-oRzo^!Kzf`Ew{yNyg|s0_kUw~sX%olKHWV6C+DoroGBFs2B{`1 zwoC#@QH_7|{kB2h!64*nWo~X=Q<{_tB}qr2(GDO)i&bBd#|R?SWquXi)IrmB=~*Z! zx2aTX6FM)lRo7?9;4J}~_rVZk3e#W@PUTH9IngH%=Ma69(R!reN?O|_7N zHnXd=zpVYUTiYSf7S9S9CX}& z%OIEZyvF=B!UUg;z$D1rAqBr_C_#Cc5jhw5Sdo-F=SGeEGLO@9h2NPrbwGhdP`{RO_l+_hn!0#vQ zp@&p-$hUBDZu;8e2W+H+N?xFzD`~fOr`}~Y@U`ybA=kJZkos%r>YcLOm>QoxTMSfs zDj=ZJNa891i?^N^r+MFBSbor&DwulpZlxAC;|6N01C*}K&QP`q@h??~lm4#U-1JMs zWT4p^YUunS_F*w>>*2k5J1+6Dz52a1KWd&XIu4)4jI^PVm`ir|!PE(x*{{{p18=^M zLFcqWlyBsoF1Qq@syHLap=qup+!3g7b-b3(r{0{#H3W~)OpXdt5;~QtT$6Peq^Ks!3DP|nM^?XP;lwxG*k?ZD}JuX)OnkFk$5JMxfAE` zD+<+3JdqqUxT03&vM(Hs6J1b1Pdz0Y73<&q*J7uWtZHd*7nOJ-UN%Be(2u{$y3QZa zyC_j_2lv&B^A~(5pwdI8emHgBIPJqi85@RR>i1_Yv31dmFthngjqv%}EA?*m&AFyU)Z#$!{^>G$ectRG{;8^D)_-l|nU zNti#Hmezrt5m>QqZB2Kr>zv%t_fLkz=X>f+=`Z!&MU=lrXgZq9xzkh?VIz|F>fDQA zub5UjlA4mn^qbmNlR}uZ89rZ3x@%>ge2|#d;>`j*9Zos=0WWr>#9U1JYOZ7^+J988 z?GqJ#zo?@eQ+m{)u7vcR7y_*$KN1br2@ozXXb1CebzSYA`nl){s(p@BWF=zEhJ3>U z^#)hXTvCq!l3nl{l-a|ZdyVEr**%ORI90KDYp!D`+baD=^-Di@9sJH7`-t@7PGhgr zJkeiJPuXmqs6J@|!Q1Oej?e0eUw{oeH3^@}7yH*`?0U7$u^$O+P*SQk(~P1}q0*mJ zAnDOSF|;y{1tXr>8SsI# zyrL(gd$u4i7d-XmGeVsP@QmepjNi~BuvT2-O>9RLa7fBDqP618RG=~q@mo(}L%HFw z%-nGNA>oNloNaj?obJjJhZ0 zBDd<McxH5yB#gCYdNJgrY4KA+)$)JId7{`c4=y@p#;#{;m5Uy#A9|v z`iwp$_)4Zo1j*v5O2c4s*UCQ$G7L5w4zHh%w#Xh=X5CmK*6X_T)74qiTH6xt9zBXE zvq*R^k$3%Uo+%c57!ZJ+I}`TqIN}NLT3RnR*Y?Kpt4d(ZzxZvlcJA81c*5u7o12CB z6PKqzkqCtFXpDF|cUI{V{qx`aTVFei>}R)?zOng-w@vU>+u^jLW2T{sWr>H&9qrNovIahM`+XlOLtd-5 zDfUN(SB6YDB8|GzrsWc1H63^zXdPM+t%Rh=bj$hLgqz z_O|0yK&$C@;&{5ms!CNXvBy*jK$DG5iHht6W3$G1J=(|ADZrgtKEy_NYoc3tpwZ{Z zaBXSVNbnNxDFSq8$Z#fu$Y;|y5Fpi7IWTp`bJC+&Uj-t0oh*IaHnZO_d)AorN}a2@ z(BSP8TefLSVW4$WFQ?gT;@?;jMpXE$(c}*=RU19`q&|OV>i=klhp58t6qs75wrDB_ z?UBzNj>acwa{o%@u>Hs$9#4rCY_u{&HpV%fRG0@5mF%h0bs`)kJa?ZbJ9+GgU4+K^ z6<%zFJ#*>Dsz5?VHpU_sY6op9RqCh)Ysw}Ws7M;M&3VFCOH8nZqqd<34Cd!39HM-Q z)s&x4kq+gOnnDMU4{=wv6p$LcWmX*>j0M1RXVVFCm5_^wSH@ySiBxXJ@d6mXu8=x{ zwXUm#NlV3o?I$4oaS>L!>)|dM&W{REx6o>c{p$NdI;UU(T?q#6co!?NBYp0=K+I|p zf~eh0Ij7aXeJar6K|XhkoCTB-_{lYSR+&DQp1Jn>fcV3;gKwdumc3MC@@?@;p4TIX zrP1pR#;n`9%1W@$3nm2`y~kq?Y=O~)<)PsYA1abIx6Qlw372Obe1xiiU!t}9p&`v> zt|KwAS5$}}rIv$@Cfl)qg+@{0Gmpof+6ljJS|;Ve-9a3KA8v4PSt|wc(ArCXjjdW% z_KDR-2h(O$x*H_!7o)=t0Tv~~5EGxUhGEF#-~BD`a(h~{D7IU}0G$C!N*PK%XHFr7 zj&T_U_D#EEijt!Vq6AYj_P1zd5@^Fpybe>Bec9KWA6MV<{8Wx9m8mTo+_II|Atj{deGxaE6nBH?wR?uBNjlr+9S+uhl)nVFqV>p zF6T-Ib;Pt?Pkd&mdy=IPXDD}7MFr*WN!Qi&qtZG27xT~^UTCz;lCa`zttDmZ=b1%n z=?}^uZJ?h$s?Emm_0b_zZ#q+?f|EjGNCUsGa5E(~uQrJCQyfQO1sUSr%_hgr?{Fwd z`UF&dMaE_?bVXxNeX&>+sZ%nK6&0D+NCN8z!eC;GuPp)nh|gamp9M-ReD(q=*FPfm zA{PQ+EzcM7Li7<8+&lDSOEpUT;rEzdQ*`TgMJv`dy9%s-!oECmBmQVXia0nf-zzG9 zJrZx+-&tLlmGH|H!A-Pk8$yL^-)W=BL&|moe^|XkwD3oZ9=x{Z4QeuZ7Sl22<(L1B z+28oRCnwsYB{}V?&TR!t24D!lVnfniN5sq> zzv?GVr7xJ1EtkzZ5A4Ok0zEPOywXX1&?fu?ufLw}q$#pQ$VsoeTWz%u>lcx^9I^QH zx|3D<0FXB6$yfYLt@v9bXK8VnfDd4qD z)>+RQEhSbwg(tl+SA*K%~ zzXD=qPr^|vYG8RHn1md^Z>n%DihnH#rZb<6`9wIYArX_ z>zTw41UiY<;jO*OQ2*;eubqGThj~wtk!3T%6mHvd!s>P6 zuewuPSFlKq@9A3pu)n962d`9ja=79@In%q=JE8jUZ8nrWK{(=Kuw&FBC3o|(rc&oF zx&m?rk&CzR0Zroxkpn(l5SBA-+W@6aUlzdD)NHAv&e-D`oM___$iAcg)gPEqt5=-z z1$a~C%wGw@Yiz{GM2r_;r(|X13wM9Mz79@2dO1*;4Jq~g;%z;2Y{6&W z9WFlRuh~f#6E($8dhkUEY_zCi%})Uz@l%}4kY@Gn4MS;!ya+ku_e-h7cLv106Q(I7 z>8XU)t9c9B&+5;j?4>gXf^lt@iwV6&19suP&&WTfFLi2a+6?h@Ew)uZkPmpnMsOzagV#Fs=v-+!vh)$#PQUKgcDi7Sw_VYpo;q%f29 zt1B8_ZnabQci$q?YK_0qUqAUi(S?0CUTF(3lye&M00jZOsx~C{=yN-MYY;gzcS;Jk)Tsgw* z>P&^R9ca`w!K9?nZ(#J6$dB(#l(Z5LlUJP1cf!hQh+{o+5em>vjHR#;WOf);M~cWb zeBO77q9_5bos;M6K0^L_DFMMa1!q)2&O>>D>?bf>Da=H z-y5m+Q^y@=K3nKJs&dq&ePMA?BfJ{mWUs(|;x3XM6;pCPaau&o8^-$PtHk1zGIjeL zDDsqneE~Pznc3EiLj-q^7oAj3^dM=%A-^Nm=6(=9>Ls<|by!n&(01z^uJ|v`fEJhg zbq^<`xz1X9q#w>;-(n_FZCC(3ijRf`I(W{KOh4?yqhH#a9eF&6L46)@vuPj&C!ys$ z9UNiNbN^mgAlW>2=TK1bR00p`<3pjMNVj>PGm)WXv)dO)jRq0ByTMhL;i|QLWQ^VA ztW39ZKOnT!ny*K;GDX#J894v+`xk5(y%}pvI~*BI!<9;}Y*kaJorO z*_1h}hs(FY+l5{X^k3kFAM@0@SFT?_7BOB4=gq|1Jt;_y!H)0q_Q$oE$TDSSUV^ThFjoOWUq@1n+ZGp*k0Ta4zRjIHXslMTt z;=reX($kN7eKo}~e_bxik%{=5;A)P@%OKsU*!%@9*YM78+l3Wst}N`Sc(;Oyq*EzWq zs$^y8e&hAR#hTls6&gil=L$|x%BflsHU2!OYl}qem>K*fz5YRW3#hEW^%!>9b;}9v zy(_tMWlr41oN6N0Rbl)tjG6mjx;lkKiR?TIe*xX*Cqq6-hjv;b0|PbM>T~TC)d;ri zuQEjjCwmVDfZSLz1O0!b;9%k|?1pr>_9Nr?#-I}FDm81^oUGm>!?z_;8CtpzqJ6^; zcHn@l@^$KU0YdeV&ngutbQ!F0-5}7u4Pq!2D*(vc3wu)$`zVqTOU`=R6Mv5MHY)lE#{=5@U3+>+~ zIUFe-Ay^^Fzn1`ASs)gq0Bkwmo)A_J?F%0$C08vxJ)6v{dTprX@M2OPB>a*b6+H^^ zg0X#2CpH_^Cwn3s1e@BuKMhlX*SE2iY@}JvKg|jl=t(h=LvbVoD4cxi9QaEELME|o zO0u##B~ANe3g|nm8@M|Gul>K-J-^vG$pPh&d}IZmwHAfIW>`sH?eq)tod|3KF>$EybZ|Ayj1QTQXTXKAkc+Ng!YoYhDYdErDD{@&f3A9o}QucAcvF zy<;%raT>T_941k69;nE1VtWI`39-e{o3!v>6elMWw)`E3pBY(Fz*5zmB|r~t@2A%D zuvCYn_l>{k)41zsU+@t;E)!7Nx+p%>z=5KTwY9rV1cV68`-O0{9d(ZY1(I}*E)#1P zLAr|Kr#EsX<6#1lp_TZdF&m#bzSw8fd36W(K%SC18#EC9T$&Yd|5a*cWD(3c9iO>u zdIgVU#%!<~Q+N18TOW;whx8 zLb(CW6oY^YJpFN>$|h=~Jzqa0hLFx&)0U55PLDpVS6Xt*Ga$_|y?PC;CRq>jUg3M@ z>YS~O_DiH+Y%vJux2thCDy)|rUK>%&|(b6`Yz-=CbM3PqqGmfm}O<_M9dWTQnsPJJI)BWBWtte zO)mV$y7%BzR#Ya>$qORUu*9Ha)WuNsVm*1H(pB)yn^3;MM&j`cr8hK+2b~K5LO-e@ z(5&Z7Ro}kx?aP#;&A63+w*X4RvchSY7%4sMV=`Qa#tFyF8Gh(%Cz@VbV<(3-i^8#8 zWx7tQzzXNq`Of3Zo*jQJsLJL}EBp;p)Ck*ZgueQJPX`f%|9gm% zwcqQtS3ZskJsN`CiOVDd^T@KpyMRzJJl|TRPZTBani5v=xU`Ty+kELw%?A0b-pY@8 z0bz10IzkwhdSXFf;8k<5p%zuZ1V!u9t)}QWlzf4%oKNS7o6Hj7-Qk=5_bUIp&lfi$ ztp;o@Iw|8cW!d6qTHam;-J-(a>Ic48Oq57D;-iIFKby?~9nGTX#AEM~>8(;~vp+7Y zA@dqmK6Lk?xs}cbG|9s zXSjRq$z_<^FTM&n7uSXvFTu^AarG^}KV=LKvGjY`y{UCQ=A82$eM-kRxC-u3meCs8 z2IXZ%%R0z57*k3vj$=@L2CK!%%<*|hv$MlAepRYN#7%U|S$ z;h3N2*yYJYV|sdI&~b?)P-EK!znWD(Xsjj|pDn~1`m_-Q?Mp>+ms|_nitR#J@jUI$ zDUFc>eA*+co=T9~kX9^R#>h9j!ybJ2GnLO@ z5$f}D2U|BoFE*R>&pDu_Vl8j@qZsdQpWhu>eD`qkeV96uxqMey1TG1f$4`yphyU$J zWQ}$<@@!ogKbej+bbEC5@iq03o%)CIcU`!SG=UeF zFZP3|yWbvD3kYJ4RcN$7KHQp+j-$Z91*hX|G5_Z|l+@K( zPJSKn(0j!*XL_Xg58Dg4fSGY5%P~N@@>oK{%23@WHpI3*A1O&uPgrH`L9@ufMOAC@ zw77ZI06Z%48L4{l9niDhkvIbX@3~1q*)Jr|DE9kuB&JV@841L_0L9N!&W}wcSo!g4euo=RMo)yf>V1&m>J5i0$8v!hrn@dnhpn2 zzr(DSV^J)~oFTy-8Eqewn9xM_h%o<iLCa!Adw4!-mga{92>Nav5rvUwn(u0 zrzPsc_<`_2Kunt}X=nI4@fA1M^Pcj&I7s5a0daOZZ*7)S1@=gz`i1(MwTS#%=bAti z-!^gt1|YxbGJXsW!@2OKfXCZz*pcHw^lC$@lfBzD#{>K#DsHi9`TBOO4Mp!~V| zQffXn@OEmvyh5G3I7YoH6Qs_~-JXn@9C(P*&w(<}`OeNybRXdr8=ZPPZXLFWSB|p( z?BbWzlrNn=P(A$lw%!UGJ@P$fVzQNK{Ld=MIiFLe0}AFfB8BSO4JG33S8zM&4lM%S zAznDpGtWJ0_G_<)pl)p<-&Rv=?Wk*@fi{4BV41(tM&t^q{gYl--mdGB`{Mnv8=Jz1 zD4Rb-`(lnn==Q^ra)08Hp^l%U(XETcCpwbf0o#RxyiTu)eq%e81RFVJEQx;0Tu4c~ z9E>)-yM#}nJ7$3RSCUaEBOKZ6XsVNxWB|4$q6!^rK~Fjh4~I}721Eg>xGn5qJIw(d zX57<6gz;=maiRPG&xtCqx}dMz(5zIvu0%;9Rll`f=TDM4#O`DmLXytiefS}?C^S~^ zal+puSzRgg>6{9v%vDNSTnoE-MYEvO7F2rTPKmT+CaWUvR_{hWAU0BMv3F#L{Zy07 z7F_7mfOM&!w`Jeii8Z}|DW#-oQqatjXIrOj()jp1EUTB~N23UWN23pYCd2S{l;Ycl z;bHroydkcG8@XtCo8eh)$p{ZxNIN8xfC&^j)(f<&4VQ%foacW!!#$cDelh#azv`it*=aSS%&V% zjlM9SiF-14$s+|p!!wfVgCfn3CdA;USS34eDsK+hy6RE0u&7fTLv@Q*HP%zUsiA~^ zTXxH22V;qGKB!a5to#R=+M_&M7b0|>{W+P?)w@1K?1~Ovk*urk*EN5Hedk=U#-uOM z2Mg3M@Tm&&?gFfkmgT@JxzOd+gRepFZBdRN_ZH66(&=&hou~86L^MZrdZF=*WExYb zSI;j5DR6mvj2;L#=NGkSw_5;~d+qDXX;yGzs|Qq8M{+(*FE5X}E~uCl>TMpY-;tS& zIDllWD^z;P<=07zoCkrW3g2>Ai()CKKMu#)_?9{$xK_L<}f+B^SBxrP;74$ zyfmHnY=m>UO77Hjk*It9#-PzsbMDWzF^>A%MH^r1UjGi8fQdy{Z}471o1JsV3-n_{ z4xepnZgX93$q5Xo^NB*lICq)Hpcgy>KK6VT;FgX$W@>zJTF_BNAcR@XcJP?CQtLxdE#r?ndf`MI^)u4 zmXR7$_ho(3JzwjXecZ2DT*)0Ps7h^fYonLIDL_YK$ngTF4TiYg<^k%Ib9)RayTW@P zv@vnh63WGL(rE%e_Iro!(^;#I?7!?)P8^mIuTI#_@FsKf4J?prs5=RnzU_K+t#GNY zez9fPTaGx~70~1g<>+qmk;vMo?zK6)d9JbcXh8noaYnI;Gg`rNg-EnpwA!%gX?zS~#`m4U*KYg({!Mhcv!XF~3*F5= zelGjR{>Q=zEZlY6e4LHvJ=^E+HRH_{3X`IMIg7=ee5sEStIcdAp|Z7wd7X$?H#Qxv z-d^zwAzvIU+^L~4iNBF#E>T)%Ue1Ei{4lzD#K-&Zl<8K#En2O8HB2}89)L{WRug`- zz*cN1am~R^vg!Apu@Gj(;deC>aSxXx@2X0Z;vbyDzTkR&^1>OGJu_O7Ma8WSWPn1N z-}_tfysRIA{}+328C1!#vcChf7#oV+ALb+BNndHV22SZ7}FZg$+Eg`Yz9uaRJB}Q#yL~_>Yn_=w z_SQ6KPH`Ur^bf<%FB0nN%x&L$=_z{FHk*Hm=M|8Br<(dKR%`UVyiUS^$|e?e?>_8@ zC<*;((@=P+aE7-E@flDP)P*qSAy6++PlRG#c-PiHdnsRkYBnA@0l6IKE*v;zb`d5} zW~HU&zfc~x+t0V7x%smO$$NyvcJ2-{LsX=nRbIB6>FlIGLVXTcS)h8ciq=Hd>MDEExV4^7Hu@;p>pn^6S(SZZyDNw8Z9x#3#|3{GDg4A z{$A$2?QI?y5@6#vRL8++HRqoEKE!j8Ms11l$`!HdquU~S*ZDx&ChstM1s0Nr)q|HP z1T`*%q$$-)<)kjng}qu*tnrp>Y`Kfbrz6sP>c!7?(8vd$HSnqfd{4I_f+f<@_1j-M zzAYIRy0tX+_)&_0mk1($_kv=OhIj`LUbymw=^`fBC--A5@a1er{Ypm z$kSeCo(nJ6zolzuK;s7IG^qOS{2F7}-)*&%!-+Y7$K=`ff|Y;y@oA~TuI>eM-9Xad z0F>KkbBV3V!MwRJ+dW1!1+v)@xDVFPYs3g`W{(NTU}4+}xb9_HYcsn&3`fuVZWty3;g&_)*=F~F)QDd+PQx#T#e8q-(L3{)Ya zJ`42qMj}=h<3>d!R<_})M53BF^=dY6y%GEYxzJ)!z4gt%JgwSA z2#Pr|dNEO+Cv{6{V(xTpJbm_h6-S;D-9f(8eP*k-Y_`~AZQmgz6F7dFpPziH{1FL$ zfSXYB#2Kz(+p7z4WbdcE$SGj|Ge6MPk0yMZqCYJh(X)9%AFROg^QfKt*<+G|D&>{ zu7&ow)+KJ;IBe9z?quqUTv$1!%Chs(yPuE!oi#T53Nf_rNvA|4TE#kW1zD0!skw(` ztj>y+uk92u8DkDts2+~aWhUaWP;1ZTx+Ad@1v=aojE^t=sO7)qpDLDbx zULK$wQ(3{xeV=~!heQuQr>?n2u6fHo-$=#u0Z!N|H>GoKrF)?!2eVGoG;P`d&?tUT zLF#YP(7O@Th_*xG!>NT;fn68KSfObGMarBwTpVN|#Co(6k-TJFD-j7w^_vz1pxNXa zK?3MBM6)MCWxpZlOsaw&u%H--;J0-5)B-za-I^b0Vn6C!82JqLgL|j_Z>uX~R6>U% z7guPxR!I!0k~_R05E}((QjAg(*P=zZ?!{MZJJ`Y>P%uuJwe|3NO4oPLbyNyhj6c#w z=>&07Nc*j~`?=1jN$7mbZS)pD$Z6H}%Ud z`|8a)_-`@ha;!VrjmN-%Cy0vmc0^O8%{VS+*DA-m=gVD7L1UU{eBBqb+L!#cI|&wo zRDg~q0JR_)^NZYewl-ppwtZ+f5j}*30_JgR!9;Ai=|n4AjL`csq;R4?sX?jFRKxeW za%JB3YX+?573QyviNdRLDQs4*C%~_G1eB3mYew*IoEu?Mhr}ebJ!nqfEc7{OQnVBu zJ5Pl(=2*$Xu!y|4Q5ZY@Upza)t}~hzPg#8xgc>NONBrfxC?t89ZJZ+h7yLH!@M5U% zF1J9ZBgLsGC;v}|D07xjD6_GHDYHqkSZVS^x_P@|O{iLldYeL&;MWb}qLq{2YjPAl zyem0GCQ6D*(r;|K_b=t%JW@eHLusuZKLrGq-z=)fOKYupeUmaKE|9e01ZtqvpJDW$ z&E_1gC?B*yreQ+Yt&{!>w$2aNt%X!qYVT7WFZ`aR8Q<2A{)pOK{Ee`kI!!h<2HOr~ z2OL#-E}6kA8eJR0dyhR#8js4)|n57?~JR+g`)fyj31|FEdPKw3io z2^-tW0D1#+dzcLzWvYzUZRC6}Ehaxj7H#{|!_z9(wJkzO6k%}LV;gQluO4YP^r#2* z%e446$K5`s*TuC-Jib6A(z-JxR<2!AE&41H$J>sCTlYmhFLcv`exU2>dNW_YdZ2WG(xZY|a{#6NyoDs#-Di!1n;&DU45gNY_txiCoq$EuxouN)Aa-rs+4SkqI?tNkGG;7#)CVSN@)&bg8#cLgV` zmy+&#E-MJVq<#A)bnaa9l)m?j(l;e3fk}pR-xbvbw-RepDB8iz2z@B$lvQ-!rIdc& z=5yn!YD`iQ#Hsq=%!BWlzTzm#gTv_&JI3zN!?EINt3;|Px;nMK>Ji-y<~9{OmQkR= z*tL9KXGQT^^22pSdke4bD}t=~`sW23Nw#`I%mlA0T-UK}^>(#Zjm}qOQYsSn1KpIl zq;CkVdi`xc(rH7nS)`8WagCHSBgfi_J;~?xYX7`>O=g2wQ?UGtW(X8)1?3*0wTxH+ z0ir6Re8C9m#H>)#@6;$?zmiQ1jTHFAJ1RgDt!Bj~=XETK(g%*hUg#`ui08d>{X8SU zMigOszP>%-b6Y`>cy@`)6Hp#%&})T6p07h|1FtXYNefeWkhP_3^*;N?{^iF*pK$T} z<$lfe$>+?T9CTdgrE`Xqw)Y3Z8_NQNq)V}w7r4h>i0+tjv%*k8?-PSBfF?1&po_1`&1CN{Vl~h4TYYZ@5xi=b|yw^x)Hk%u>I0(EPkdEIZyd& z*>)i6LA`<#$WG*J39{7X^Oki!0OQ+2R)Yuj2)TEVP?~ra(jZ6> z0!inNOZ<(%&lUzPl;eV0??*he?mzI>pmhj#D7T0p$=!ANBEv)!;nk23X?}>XR6n-f z%&QmRR}pkQJm}?)mA&#M;tg8I!1!uEz8`?Z1(D5vQdA&JrvUm zrC?E_;QA~_#*`Mx6o|(tjE(q+RiX&!X0?{K4pza#?0=eL2PQtM9l6q-b=S+<=q>&+e@6~32Qtqwtu-f zbaR{bg3N7ob%W72!eOHFb1I6T$*Ir!tULGM+cAGg`Ni1jV=p(V z&7Klz^km4dUvlF*6omn+xnQMPHlF@ft$UkZmHYINxhWZbzVB35ORVoE{_H$$?&!mG zV8|tI_;JT^F`!rfJtN5X>f_(Fyw#pC?@ z_R3oaEY329IjX{NG7ad^1{tYNA8xLjjI%XXqk0~yd5(UpeU8QR#w+KV{i6F(R)@xNbUhQ&0Z$TsGb}Yj7M8!iZUwyk-bpYvYx-<47-0W(C_m3|Mz}=PBmK0a) z+b}#R+~2f#8^(K-71_;0mu`?V;(W+JJ;VttC!5C9x##I#tg>fQD8(bvnU~T_6x@Rb zvf~CF|H#MhCh!q7il4RXCB7T-byxA2H!XnLK~z^ByZEH6i=WOFKQ!LnXkh!vSG;|b z8(pv&^!`22q0Ft;ZnU%xX0zUm2TECS4@sj(Ua2vMXZw`p)e7Qi($9}+u2~inA^h+y zQn3L8?D6@Xo-Mvyx<2PiXyW53Px}}nC&g1xe^U5R)Y{rjB?%XEJcf8~;ohSLtkFnp zyf=P4I%EAhHw?oSz$%WTeN|mxS}1qXk0PSUCXAxwF*cVD5vKuZ5hk9Dlu?FQ@^@Sn z=vEv~3U#(EYbckznfagmSjFeV!?L<@Y&$^>q8y9Lr5g>V$q@ZsG>rnOQhqC4d-( zSfd*!;x6dYYdU^_E)K48t2Wac1z>(CK2RefEsn?xi;a+zc#c7i>Z$Mo)blBXBcpu( z1o62`Mc_(K$@|CW3%$9vpTJ*GcY_pft+=x?(F(i-MDRM@0hgl$3JF|1^OViky?XYUHcAmcbdt+x+egjXe<*Km}M(dbZf_Ookx-C$nkghvmK zX!><~=gMCD=i&V$5`DIc%*4ag#^GPs*D_)k-gUT@gHADev#Hida_-W}mVt%1U&SqG zg6jh@}km_Gxdh#`q`}`_N)Om#)4T{Gev*E z6;nGiFXebn@}AW|r%AWblxz5O-8R>ReMqF*-rihubmJ~}Z-ZN*5;N=Af?gR<(5MuX zR%0K|9sO;tVcpz2(oY6Q99quJ10D>G3D{t++tKe;{u-7(ggbqoYL61_$Du+jxYr7K zKEW)10NmYc$6Sy&i57_K^;I)0>G&D)!$z@VeYwnp4&hd0U_prY8xxBY(V^&_WnlM` zV#OWv?#hs9$WO+1ibu>V-&)=%6(uq-wazP)btxG~S_ev&U-cH0CEoWpoPBYX-7YH)1 z9ccp(u0n|m?vEBT*@iOrcRGS!VY4HR=($Zoh&8j9!J>3R#U@4xk+Xm0Zn&|=p@X$X zV~eY)pa&S#m#w<;L->h!LNy@b+H=`GW!WOK8wKk7wn{z8re^N6Lu}Z9w?f{0ra2$e zCMB62#V&@grEWUY6eV#rqwhdWX;q$ZH^UT1tXni9!@~ ze?mEn<0ca#%MrGLik07`Bl3}Hj^5>8`udCLTEH?8%iMrOXmu(TRi9>We~q}-CA6*P zGmvQ1fVIG55LR@LdFIx8;QmuzEdt(UdY*-+DLR|8JF78oTw6n><}h{j?+XR@k!3?gW}jwh-v@QX?IsN4 zds?`u001}!Kh{Sgpo|0RmJ4Q!$Z9HNjr?HOv+5F)>+9>xI_gqCgPbmwuGLSFQ(q1t zf+7$dF^nbQ)s$+ViD^f3u!;jH7D6 zbnf4rSwIMxNxx0Rn+y{& ztK&VKMcbVkb32wXpBY(@2$njo){N&4Z!>I;(@*Q3NyL|*&X2im_RJf2l<0n)j8r#; zW92an?vhUJDYqTT;|QXKt-lx2cS7|SBS!Nzf5ZrPmba>Hd!C9wT_14k&Zf}%x%FX`m-j-3@^Io>1S9?pVGj@cE1WI z{@Olwo~CbW(9K;@u2$+jTeh~)x`lKboanlKH4Yti*^D?dgn)>E*YNifrv7uuE*HrK zBSc$ix>R__pc`iWzSWGl6p~7smyNb8$=AcKV|lkcbL?;m`O3$OirIU$+dyTm{N>2B z?QK(hEpf5D?^8L?e17*dmbSmlXYTQsISy|;T>8y7c4?;gA-JoKA`24!zUw z;Oq0XyO>*7hr^+Nx36o--`4B;+!r1tR#ZUktuulm8xbI+o;~1Mq|YewLX|uwjMxj?HWSX38q;ljx&Y4S0`*EB z&cYNcW{yzbt5iy8rk90p)1^pI#(TJu_@1XC*{yL;87St~an*Vk@FEs6U8?lGf|1%R zeCBc%o{YYwNg(FL!qjit#ZEfbPkLq?!s)5qq92O4(IQg0fz6#7D+0g#BgD*M$YslppfqO+>O+@Pr}qm-luHew~#SgsIUFhBXP7r82TvCRSREW&#tR5{p5L%GXdA_idCzJrQpcWI@IHnvp zH4QmM8=8^8_p*vNDILLmuG#GQR86x~EhYL=Cw$cnbz`=YsEA2KZT+~tA9=O78&OrU zzMkH)dXZphd{NhIl8_AcETNVbLh}sxo%p`*b25qdet25F%dI!A?S_@pbuZpw1-}Ni z>36Am4bim1h-0_UdDrULI$x`}4=-9&GiaeE!IrYT$I|fx-V{Q<<*0Wnc?D-sdEA># zC5U78Q;e!&X>YFljfOdKhGo(SfexqVP8-YbR^W(Q6MpAXes(pgS5Mg`IQ&=44_#+- z8WDQRAq?X>EFGQYm?JQqYe|>BfGXGgnk&}T2kUk2z49$E9cx8$uM`_0M~M16qk~%~H+yF`0HkD2@-RYgpt(%(|`+~du0iElFJnrNna>BfzJ^pi7^16i9AbNnVBhMWfhL> zddGRlT+SyIWAviGO4s07LI<56ZdzjLzTwkk6CNQE$}cSNR_`wn@Kay%0d2gv786c8Wu~sf~%$IY%+t(kRrPr*}U}ooseg-`+kT@EMA& z@>cY?pFr5XDpxWgKsL4h9qH*S6{G$i&ss)${~|;RTj+2)60IZ5n(X98qPE8W<%eT< zDNS{RoZwX-nT+7e_vb*5l}zsx>T9UtHjUeok`C}DaFxw5`*8{4F2JX&cfwq1tc1{Z zK}~MxdzG$sUuz;A;>Z?$iWGThoa{|W-Lsuy>@uohespz19cbad;Ij( zJ*Pxi7$cszeng^!@iS5;774u`nyeobS_}#WE@cPJpj1vxe%Aq7%=X77>UCrs)Re*w z)#QqAL?aSeZ&nc>t5%+qR5S%N`pm2-^*M4dn7l&3mx&lOc%_o-v4)yX=JQ_53!odK z#rL2j{p5RInS08cdJ|Hcl-D6P^LDV`Z#AP>8IWwYp=-^3zpcl1iJGA~NC$~-GSsR^ z58OZC1M^U~v?Su}e4@J+Hh6A?FpA#tsDljwC_K{5!q-|upyPYX_jptzGS7XR#oX5` z9nPBP8kIsZ<86=IsB1ANynZ;1vAAk5Ozfd!Vna&Le*FUcJd6_ z13{{uIA}cJ;b1uaYQ~o*ImkWOuQ5Pazzx)30Ji#9)~YI#tzFKNBlqW zEv~+Sm&uG}dbf2l*L7A-pi$nu{22Gpa9L66E`!CRpN;c`Ax>5bb*d;Ft*2F(JpgUt zi=4jF3hR7+rdgR7K4zcvAY5;)FcNw#-``%9b6H?&J!Ru}kAAA!3hn|S17C3G!jthn zzP+m_tY4}RDW}HCw4iHUU%9WJeD7zD1l$Uwe))29nb5#zd+!Z2nA(eCSJR{;AM5n< zC!G~=qZ3V&j?a;}%{L4Gf)a$alwzceJE0sJ`y{tX)ZD)6;ZQe=CLQP+Fu_a^cj2k5 zARS*{0^mT+Wnw+GXA+*)Z8pW&HecwwoArRs$)_zH)@LD~*hbz?amU)yo5LOT!c+6_ zJoyfp*KC7nd~~NRn%6qb$;bUm&0V?N3}Xe;Lv_IFqd1|?+{PR6O~1jj#l8J92XD2` zFZr&!hUIFIa9_7UV1$|CjELsfo z-`~fpqK{@<`z^4EHpwj3^FiX#Ao~4@bl^`o=nL+>0Up+UX+n{MFxU0z`Refs$8QcZ zb}Ll$K`yVU8GSSom6YY4M3i?|CNz$*{yeD-su?{c&7udR1;E}i4N8iRlf%i^pT&qERJ7{UM(`O%4dn)v_6;%x?s0OsBB>=S6AH-WvF-o z@t0=DlA%5>GWKf5x{g97IVN1bb6u(iokx_le3&Gr*mHp`>K-x#vT_`2F{0I_Wqm=N!M}*q^gd_cZqzC5Jt{`-n&b|BCLiNdep@4FKIP&oX@AS)XS%$3^N& zysgk{$S8a*y3Gf_kP>06n1%X;^)Nw(qiAC%cc*G0f_H8M(C-_L8mNAuNzpjxvt z6CQC8&5hqcA@Ujr;d2eGx7phenj3EkWZv%zVRrTFx`s}1%|k|FL!`Ldg&ksjPF%rU$`s>R!&ds=um+s4f7%ZIkFM0^Ok&sjdDsjpE+|Hb5zpU? zQ%oX|e5b!kkE7MJIgO(p3L#PhDgZ=%5uKtN81=pMo=75+=gU>EMQmFM=XU00U|)-K zw!CPM6eIF_yl8~eBz?nHEANk8y zNfQrr_V~XwmoG$r=!*g>?vZMPs2LzGi11^#3c%rG7oWz-2{sY^Ge;*(4GnB9$H-2+ zDll=80#h!k-{J_q_*m$dvH~LxZe^g)PwL`_LXjW|Z52M}6xu={naWRPzZ)kevjm=+ zyI*~jf7dPFP@6s$p}|V3;P&IHKL#nffBW?B9o z0RJlh``@I}ewIXkutv`z5J8Fd2r;%PMqHXmb5%UI3Upi7ptK@m<99s*E_M0;`-f_h5PNtn`WxB@K; zBo+&77yiKF*(m^^9TSFrfJ){?64l8Q1#-auR&^FqDbhccVllA5_@e&?T(-Sv|;!k3Az2o{)-+wpB}*9BN!%OA=yUS zI$_Eiw{a!aPRMd+fy9ID1It?>^e0A#|A>v2{{`K=sK>(xJg*i$r2y#c_&@U%{YRAo z_K}-_{zemq-6X_+uB7-+vNuh9ymvvzG;0;_@4*j01PDwVHuXc~-T{SBXMt_1HwHu6 z&cE3-riSG^Nhgvh-|-#bQ+G=KMf(5vGe*GI-v|QMo8)iL%Jk>qw#g*NeA4`( z{-MIkYuYA`zp0ZADnBOyMoP+>$0jd+XLW9gT~&zvUXf;1x`Mh+GXnJ_<*qe9=jh zXtgQr?d{`YVgM2n5;?^AqkD|+71tfj|0K%8`^g%jC)!>*-^E9OD=ZHc3Sk8`m4yN> zRR@xf6zwaH4u1fx+%=N94&sVZ27Vs6emR&Z4^ql6S9AkeBusAx9~!4DA>t>f zNDrv-Ck4TjNF?38PhU%4du&WyGXhJ!Sn4>Mn3&+c9`i79>Giw=62Qp&f}&-8`b@zO z@-K%n!dV)%Ch~6^IU=yaA|lx;rK)BhJT4B7j<+K%)m)j!jcnGEd4Q=hn(Ih&i z7%O6OP}hL}(A_^>1fD1>FcujZ*=)V7`J>fR4Vh-6P2~NCk1nD3*-GQ&5~rd$k%oo_ zG+B%?fh3g#?|&EWpL+Ypp*l2$U*45G=B@(weqfMJZue4x0aqp8l!f|2gc+ z^leyCHKvx>d$(=74CamD_I_z%3Y`s(wIVurlD{hOKpmmVnOQ+;uyOUlXA$~E&5NhdRSj=u)z=Y{yA{_BbV&4$pj_J(Q!u<`s`+^F<(8s``M1(8- zjd}Ki{Wy0uOiJ-@ocsU!{a=&&f1e)QK!;U*LMzQXdZTLiyzz~Yzoq>(M4@H**87sj z^vzeWP_zU}%|vU_4LARbb^d!)@<-DHV3T5D7XA(U2F1Jpk7Ii&(UO_Vr%Zd^$n~on zC;R$uyy(C6^?>}*@^rbrw78^1(A>N@v;Ps}Z+J4lU)N_WaR{6KzjGBoz#r#A5A^>H ziT+=||I0Z3Z_WBvcxJDH9Bl61YX$>lPvMp}owS_i;w+HX}~wlO=Q4s|?0hu3{VOTRu+fDWn> zARI!dWBr9QnU#RjwOxQxQc|WIo!NI*2>7X%Yi=!>X2Kt4eDt>BQD6F4&;~DwNC5ly zdc(%XmT_MAP*@eeiHV8${W_|6gXKa+L5Zc-5APlZQX$H*o_`NG|M=Ie1{97o^(4E! zz_m;tY`8@HN20fv44C8oUnx23qa9enwnLSLvJHyAtM8ZBK-xTN45twT8!48O0GwI4~aS5wvrAOSV zQ8o1V0sVis+a9-qxW!vRwtRyOuUd(>2xh6vENcV_+f6KhK`c1@ zc>lYE!76>rjt0`q8G(gic@!`SFeu%>xOMxLlG(9<4cpf;PPHoKIWF9H@?zrh+VThK z?8H3QuG`s^U%}~2s{8x06>4MFoUH=p;T@6;2F7}B@??v+YZ0-fST!nmu9Bs1PRmXt z$nH8bvK_;dZplZVMsls*zfdpBd${2Yv8I16dH;?bFm@;rTE-ouf=$=0W>Z3o&}XRn z+sG-btgmAadl5}s9LG1H(eoM&OrjQg|38M^{~Je2iT@Cs0Y&@^2bhFo6^28E8#F1o z2iUL-iugRiT(hOVI$IGvQy0F3w*f-%pz8KY*xkXZPS3yyV;Y5J(9edA?YaG0=Ijmr zG2o?so!Uq}VecxYDzGtz8DcU@LQv~IjR{4Z_7YVbqC>{jZ&(iC=tkY~+5no)Ma5Fo zSBz@h^z}FAiO20{D;BbN#f35DAsDln=2|8=S}tc88k-jAJFm)ZvTGQq7TNXo1sR5z zS^JC6|FhwsHMxW8ql#C=C3+X|m%h<5C=7p$VCl;iiB!)RAx355&K$O|!wz_3RDCk-#z5C1RSjC>vtQBC zi*7=PBDxPAw^&7{bHhq7 zNP6_ZoalBA&Gb|*3=Lz=X?z96s1$HiKb4fK+c zlN*j?#bDtWE6LC;819x{Je>FBzAvurJ z4aRysE;gD%^0K+~lI( zLA$}@mKg}g=MoYAb8aDa)2!;MVyS-#hLTv;`Y6Nu9tT zB$EmdH?~fSW}h24$qcb=;;P}&wW~(lq9=#dW0yZkYmZ%@*nFgyqiGH6pAX?~uUfUh z;}}p7WSv6-GU9kp_X_W}9+QmV#Md6;pXQ@7g4R%XOnAr*Ei5c(Z&x$w!CPmu=uAj6 zQr8!nKgcz|7+wU<3T4N7V#H=-VHMHDD6?uEv%)7x6R=73Xj;E$d?#6{y(lS+D?qMgI9*lZr*bUGO&XC!FInF8NQj# zU|)U^k)tP^X2zbiXr80w0pef0p(oqt>Kjq$8c(TDXAm0-scvuHMcXM^gv~q!1~zJ-tW!qs-f&su*)I-J*l0NEA(D= zH82Fd_R3KG_Jwl1>1RrUe?`2B_P7xkWZVa{InA6^RQ&&Zr~g3K4scLJQV$8hBa<9J z!<;Xmr$>lgVV%I$$tLK^VM(x`$TJX*L6d4k?%Z%NPv#YTt(1fohv1x$dgxdsTz^#Fd$aEe$!l4nM zJi?LC2t#M>90VlE1WqW4zJd*xi%&&7x|zJYP%p_-Xw@BRdbMaWWOOq*K1tb8YB%KI zJFV`IT#3BgZr24b$A|mlooF8-EBp z02bOUf9T&}H^Yh-E-~=aNH2bNVS~i{LA;K=4*hY^fd1P&PhQ!v?dQYb`Nn_^bRJvA zm>YRJ|GzRv?LW*h-}2YtM1?>ki|T+3_45%DYuR$(SxSXU`&t;zf?fN3h-sx%(SF$C z%{AK=6*N=)Y(et82!|Gr&5}{CT|HTTU`8tG&1xZS=8vc`hVLy5r0q1!w=|fME4l_p zHMcx@7Cjoyr~{_?DtZGsw*yPg9_7WItZgh9CMB(%7;ys$slO~C;$#l=9Mt|1*8KY% zhuJEqA7os`D_#ce&X<9~tXF3X$ddKfkF9g>D1wZOsu$&qh}@>RR(|P-x=KeLD^_bu zL1^%`9XZN#9HDw&P>I7Mr}qq@gQ<e6*g#UR2OOsFz)1cF${!j!)f4lz!2Y# zqs6d2@~W3d>cb&rAY5M>CSl@jLI{;|spPm*NYYcxI9AD0UQiyRYW9`0NnGoJIk`FAmA8l@Mf+22kUUGakv)O4t#Dbp4kgalXg z2#0^hbBwS+6h?G5eH+ldnL93>vVjI0)~lvT`cTa<%I?@vEd{p|3g*1 zVI!$Ji#3e&mcH$Yvud+FuW+S>2w(jE&mtEQjJW6lygYLo+kuGwYpM~%sfCH5vIha| z*npqWEimkQTy{teQb$h;;s-M`Tv-jATyUBA4F-6-%}Bvw!742#;CL<+owIcbI(BCc zsRw3G2S@=3)>=AD;HwJ%8a6bsf`cNE<_B$sPve&yD||QfVAY>%m}F4)??AKu)p)8E zfxUNzqk_?pxg34ZODS|d3yE|NO)ICbG9V>>6+v{!7qe=SWdHTq_0H;f_RZ7NsSKw5 zBkaLXuW%es%Cb4Twl%zRoIyU%_>A4_@GjO&VYz~fNZRk=FUb-k{o}M$pwMyY<(1#c z_9n_1GD)$<8Sr&SS2h77y!s6O>&ozCFM|g;cT60(Z9>D&W81`#a%*hD)!j$XdzN9o z_N?@*GtKI9tF=STt~PTaNo&Zim?ZbI^{$0 z;&j)6U{{Fjd0AsO;v(F<+D3C_d)h$7-Mso%_7#d9hK<;`$Kkv}SWZm#Zjm0g0B`YU zqW|7HH;R(#zMiMtm$-E>%-tV5p9Tu01!GO?!OJGJLo)O;$=R!nJz+3Bs3sh{9pN|> zTtOi55S&jF&9Ltl{4^J+UaaT_5{+T2AJPL7aie6hg0XAK4eYSqiu!AM(j@*hyNy;P z0@&n7xkR~>xeNBf*efZ=6qCHq@k(C{*Q6$`l|A*qr?Y ztwva!z+OWJR{2{BtUo)Rfro6}z4z#wYf||_zEtV3uy0|176%B8`bAgNg zniF=zn0BD2wOaLAVs%KGl9le>??_s*Mk2Jgw<36A9B%fd) z!Pbk1?hcsD00v+-F6xo~X^r;VIY8K0{Bu#w_mwf*hcrhVkDB{!D}0CT(0T2P&6wvN z>X@tMT+iGL(5P)1aqPxLEy*6M-3r{Gix$Fj{<}Uj>e*GW%nnva!&au48-gZ4AzskW zSVoX5*U;zK(E4V}Up#FHZJHe8Op~ACA;^V@b#BF z!(0ufn;vz@`TEK7==zU}y?ReP`s9NCP0p$-gn!d7Pm!OoU;;aA@$M|qFNJY5k8HOk ziPfzI1)l?m5x30+)FR_dq8Y6A;;OQUKaNUg<1VF%StYj|BzaA?7eBFT($y^f(F|YO zKsnk(AYoWwV5C@Pv$E&HrCDcvoU)HV%BLnc&mrfgWy7gPH(0Esa&WR9MHMsa^!Zpr z4I_#$%yfXqcjD|!do5MW_!0YLr3PFpp+jRwY`b~bYYT;HutR0Cca5p6rRm-Du!7am zUDvcdfq|1L(dweLZ9$ojwqPS6WivIBGN!D!SYUxXL$EKBEob~T81;pc5RWXvQ(v9( zECz%*G8+B7BRc5#4RKKSl_!Ozl!w3E6>wO_Z+Sz&3BCpjO?DbIxT; z=dV@1&N$8$scm~Il3ZY^`bvz90su>*d;Z2Bb!v@uxw-Z<_fR77kps7i3#q?LHvnj~ zA}2L{0hccCL){a@w(L0-!&ZBgUqsZ~j*)+YW04R1=GKfn?iKMEQ%PS}Zzr1XUeZ1F$ywYf8xSDO) zEP_A0#9?=|p`37}p>?#u57szn@xFQ7ogo=_h6*~Jgsk2S`Mc8Nv*$_p=;D+D^UfsRVJ2*Yn>B&) z`qwcnp6u6OZ`x=GKJuYVD#s5?D672sD704V0KYU+hkEWYKO=C?0T)MvsH2Fhyok`z zgeJF3(1}dGXjshhTw!6CzO1^p`UZCHM+pSp*X0~w9>g!iyMJ&2v}WLYzzM!K+2Xm#EDQvbj z8pS3;1l|2?8~ouIG}Idl6U>ltu?Gyy@N0A2a;;U2+Flr$r;p(M#uVOG?si#+Bit&w*&8qWt2i2r$+_2+=1uE>5KJTDx)L?{t&Wn6FW1 zKOw7HirT@|fD84tC&%dBBggbxM4|F&p!vIT$d43`ytJGg!6nGKYIp;b77~Gx zO9jWDHUkTs!Fgb0{e#+rQcGWjiAn-alJF{<5Q1N)RaYYb65PfdjizGGn z@PcI)o$8W0V$Y>+_lcEb|3}C*_u;kB<&ELJvN_JfVug z0LZW-9YUd7e=qaK;XBxHd13+u2O4(ky_Yb19Ula%*h^td%crO^XKsc5LUGUC7S*WI z1+fW@o%*8EvNvvpbq9PlNUMGk3hEl*h$Sj&z*z?85sY~kW`#Q*AMJk5@73=T2S@iQ zYLt0AnCa+cgQlT#2oTT7XCFaVdAlgic8e2sU0+a6Bo?oEvn!hs+^}`2Sau}r!2W&E z6-Q1Q=wXmAY#cXxM}5G)BAAh^3W?ivUX+-a^LBxdW;h$oGJtB}rg?{< zi%j4_K-Wh-FVTJ5MctTC|5e%Zd%L~YNyBg{?1~jr@5@4BzLzmCUiX?Z_iPHzoPGG- zpE0icHMN-<&o;23S(JQ1@Gat{q4|TK7Fm-CHy4t9iu!8ZTmJ!U79N&%k7c8)Ijbz% zaGhocDy^=)dTPKDdPa;dx6J!GGf;ucmXwmtiLxA_!??7{at?>zk^;yrn+Wx7t?4z{ zuSSntlPw7){pajGR*;kjQ!*eevWs(xblPpky90qMy?orLmeA3Q?bl;fdK%<&e3uDM zHrdJy{s%-nxusW4l;%rjIa^<(>4zzM)3_&PwT%s>5AS-(?MTwlGh7p`$zBBE8*flW${hHD_5Q!GQ^4PY3eqUWuOn_~csqNE)wQBqs0_ci+ zW8%hJ^3<^FcS)$LS~6wqX3ka=7S`9o@J6J{j>ck0?<5ATIti(_3@E5S5Q|{Mm#Q*fGFL7q)2lg5q=^-)M)&4@)()X35 zrOU*p57W3@%6bxZ?A=8Cr_#WVudhq)5ygA28g=;R&m=Ld8FwdabAZL(6`A>eDMN-e z!oUyJhzeI4kV9*Q^El)6>9}%CtTaU<^GkYid;yehDwF`zX{B=-KRGFP!N9H!;)v(eqw;FccVJrAOx>0b%gVi#sc{HB1 zDP2`gPeC3}Vx&=on>Hyk!3CQ+FzjRYPW&eVLGa|DyUJ3)46xmSRlSdplHa_BEeV|^ z-h-0ep2P}pOLXMZEE3qy7kyly9ME|>=U8rfE65r64)-FtFKMsbkYAyuk-?HI`fcI; za>}B;JBPw@<`O;(9X0PcFCM zX`EBqy79s*n&@+ab)Z_N=Kco5>R0)#0OZI4>Vlo61qNWm9#fNIF6zr1O52AFxjAb!h=z2kF}cgsttYf5twC+ zv6&NnFe7Agf8nj6Fj4QcoZTAoAyuD++Vzbau+UcwnMZoj%Tj@|yehPUwh%XoEr9~; z5kEhi!uEsGss~+(oUBB5Mnv}?p}=7;#Fy0Vd$HL4DA*PAFAgSUG@?<^y=A7bFITkv z@A8k70wMa&odn%?{p_Fh%aV)g>*^*u{k?ghl`zAY>3%y~+gxwPJaWQsLxPpc|dMdrvJ* zv15~xcuae@VAPvLpu=y3VPTJjhl`7+>Q?icOKd8|5);;B$`TA~3;Rxf25wf}aj|({ z4SXDSBjeF`T)&n%v?$?*f5BBK4Y-#79EM4oAflgHBiA3%nf67!)(%ztG0S<#?Sy?M z^WhV8Pa_#gQu5M_kK@fYf@_tc7kJjy{plZ1l9j2d;;g}*84QczL4A9A)fEOr!`EB% zYy9DSxY3339A=&x_|`=nBeyba(l?o1*?zC#xh6;t!;*e{!6j=De*StP`I$GJ0=JZT zZm$(V*S4>AweI3s)OxxIaaBW^C?l^kJ4vMOQP}ImZawEf?-Z$&SqMjdV+W+d{a0WD1 z)aM0LJ(?DF-G<$}0#|1Ol2V$vu0^GpSZ3K!#wWZcb+VbV!bCZmrrp9;24=mIGlRfc z6l1x>Y14;f^+kZHE##V0(~BLZ4i=I{2<%A3!Qx!7Xj<&O>)C)NM+NVaf}(cy%eOe} zPCDO0P6C^kFF4$(#sUpU6alw9@;WK+e*S% zgQ7zUm3#B2AT5lNZ7d`kLU3#vel}?s3>RTd{DUDNKeQMaBb+)+J`@I!X(A-aG_OL0M zX<$QuXWHtIDJ*x4km_@?dd1uE5jut2@wO*dM`a4R^iTpz#y!ucZ4;H-X4>%fc)|B= zk-Qo@q=gv(3{TyiOS_n9-dZrV+d@<=tvN|DDI`7SO{)alxagtYbx@PtIjc~y5E1$G z(^bx{(xkTti~fA%64X#x;4j_<6hB;PZIi6GH7p+wf2wM6nOx{>YG4}63c(ydm$Cfb z@s+fy~+!tNvy~#I+MFc5;{-(fp=_t=MJNfY4V)|i^h~UduwsxY(Cq}VE)L+pSamI*Lp>!XXT=n z)(X?Os+e?M48?RjTnzS5WVa; z5p}mXm2>LMnmlaznLc!9I+l23@f*)PD<8j`)s6Q|<#70kL@deu0hF}2_%W4rig6rP%Q6KCW3$M2r3|CJs_^5FE+`e!_!(tTNG{Q zPkQIi%Oz_&!A{%E9sQQUadTdw)LNiH?Xw_C2HRG5jOJi){VSM)61f_YRs1EAK^n~g zwoyawVd8H(jfXLO4M*;$v5#yLdap32n(?o~yz|QAtj#?$YwU(HSwISG^@m#1bgT%n z8G0ywH}Zn7)QrBFu5?J*nsyA#^y;Xq5OX_6M5mRY(ehU1C<7zhHEAokyh6%5OnHr} z0zSww%U|P9Itbp$+m2;1R<-ixyuzJU9^|vd0a&fBN*ph6xuWp&JrC(moAGPw9laB? zlkgV_Kja22>D#Tw1B-sAHdS=4YRodPSCbXY38v)<8qfNT)H9wnw{$7Is^bVq-$f?4 zSWJ4GqukZKbi5!CgwkM~ldepyD=}d?Fsa@SQ}S1YNsOHMLC6KjdrjKZ)bzJszJpZz zL^&U9hz+Vz5}PK)e?g1Q9TZC$*(-a#m&8R_yoHvB6?2R~6Q56WT}?D*hp>VZZ+FzPo#Uw6V}0eMna& zy#IBA7Ss)$(3-i6GY;w?f0z)M6^Y-Q-@p8g{mbsA8Db&2`m7PagXI}5f6C&nV*{d| zX_VOjIFK<_TmeqZhiAteJQ0tG8o|=}p37RWux(Kp0349KZUzm$SNGK@6w=`2 zD@d2r;tvKo^NNbvlx1tfzHISO<+IW7kme_63?fENb0`b#XBPJt9awdE>Er4tE2ZVy zRvlEdi)oL=7`Dq`C9-jFcNm(KKM*~-!=Kix<;@b+^1J6s&mA!={Gcaym!VS7Q!=#6 z`1CVyizPYHC7>g0Yc&yJDR;D7I=|X|ZH<$&lODG06!3nG%C;@~R8W(6ZH79&QZbN) z@^iW+X23P8PfWP3Cq}s^+bY+`0rxpN;PYSyk6~hZ&!Pjy9hcku3^fd`3H-^y2`$MGw>K(Fv>DuHh{coZGXYS0)VX9dIac4_2p!mRK%Bhq?D`1 zK(#W%0*SW8u5@FjZ>fb$ZI+(SVrgs}qdgYWtw5Icxgmv~fjsTlSA`81-;*Pjv_$K* zon5@_4<>i(6k1vJ^DX6JWmKjS+;Itg+A?Py+G(6|A|_^=+GaZr3JB7zP-hAy+b)<{ z8AGbBNEZt22P=2e;j%Lw4K9B zfU~oW)lVkEXf(%QRcY->*eRH_9h*2E2dBvS+vqIJ&2C*s!9?c7(#hwS2Wk8j` zg!whuv@$q+8aHV{!F5IJ%TeVH$JK(aBISVyQgK~W^Y8O%kJAdl=VGU~ysVJuox4EC zS2SQPitc5Ub>*$bP$^sT)`4=*LcP?6%;appNWUTj2<}$R;SD^ zhOe@1+tb0yjgB_KcXrTL@?4_{F0HzNQ#*g@dCz4#2+P?^0Zb!P6MTETcMsA-(H2Ae zInM=F`i#6Gv#Y1H=-=bMc^F|;C>Pxoo6{4noLyHIf-T9t`PaoSLX2}C%S3IipFXZ! z&YB%HFBz?UWytZO&@-&tIHzwmJz-8<0ky*stmO;}+OqnLTq^(pYg@MyUKy5`I;<@)HLx@7TbSrnY#G0v7k&neuIs9K<|!$WX9n{`pU zth2^`Rd=^z+Ug7e$#1kZDs4)8{}ia_79l;Ne>#w*9-ClxwEBeUb;>0E{`tM6xTDU( zlbSam-3j6!7vk!e*iJ{uiRAOOp5J7wRKOgku^aPapxs*h3+?AMip$n@PcJ#`BE~j= zLg(pydF1JZ(%{cQER+UXBI}7B0c8dXXogx$HyQU!IJbJdWD#on`)$lh19RR6#Z&I;NTtC6-IMJXc0BxR%*y6Am>!PdCkEUP;WYQ%F*ObbFm6BzFOAUFCe=;a(P9= zIuOC*l?Txs*TOfx>Q;PS%9!DT`{FP%FCzle$Xz<9`p3;FqXd$I|{yrU$9kZ5o{YqBZ z9F?>^5~FF?Ap38}jB*80IAnW{MAB`23s(-aoHhROJ3@jA37otwYudAYr6-yg-#KuB z!5o|X`Hpv`(XBf8mg5zTN>DnqY;u{?_Vu zNtcQT)(XOuDsza5oLlAGFyZAWsEbdEvWy}T`^yJqSw9IXeZcH-m__Ag+T`+jJ+W_r~AyEZI9bYmZBIQb@MF*1f3loU(xxj$WIyfJRW z=444j#Ob+Oaw~1q4`jkh!>@oQKQ^Rkuic=QC9(BxTEZ=>{$T;zNQID<1`J7i6Yc#Q zpy>L_qMdGpvBGjx=OxFn*z*z6F5v>MXTo<^jQ_z$2IlSVByRN}-M#9h0ky`?d+gcS z%$+=MNPI^R00=86KFE3Rq`6>9+v&2t7&7Stdei<|q4@)nh|v?rlUp@XciqOte7T^Y zTFevIm)pz<`g%d{mp>OS*R_z7iM?UYwY*1fw6+UB9umA2QxjIa%Tx7O#US zZg+H2Km_V|{lvHsKLXb`6H&x~=Z(%?)}r<`u0uLfz^HvYBaT z#^<#p6Bfxf%xgZ%X=MLr?VEEzC(Kr)QYc+02fNLjW9EttZ)Cw`jg?V4LVN`EBZA*> zg0@h+QYst&;2@++&}fg7CaT_&MRM$QHX&;yx*EpSG+8dstSH#KX9bbrPvD-uh%1%; z0oOv&I}eTYT|AEm>MLbn6r`=)h@f&rTq}f}^Fkk`y%T<5JT0b60n}F%zCDf}opJgk zRqBZ@AH_*A2lH{!d@Bw;e@2_q-;RQr1SmYSC(C2Bl8JFo9xYg^Hc&IB8=8Hx_9GR| zv4xXH8P=eoL3I2E2w8~rw3Vz;BNgv#A$Ct9PsA~a*|W>4rVl&e7Kt~Rw0zSGR+~EG zwyN-pCS5Z2$E;XJzpm$Gkv)uBxY=ng)X8-FNEsuP>Pm&&Gub@g@_jq6e&?)qA!NuEZ5#l-klR2DaN(hQQaH- z@}QWy4E9@JhPUCfXMdhYfg1{YKaE&Ekue3Q?+QoJVf^VbqBsuc7>~ouBwzno_{Jg_ z#_+Tp=Xfgkm$=83w*w2DWW&j4*@%_B1E1?sk7YGL(82pQuT%pAiP=vGo!OJryqWTy!`hQT1L3TGnXmpUz4u~6@&CEfS`+#(D z*-7D|uC;B_0Z{R9zXLb7o6PX8zxZL6KO3a}t<6xfKo@a~)5?YPf_lYj2};yTgBE-h zPe<1G)AAhx_I%G<`oxse_oV~oc)RP3rqiF*jP{-x@vxlQi#{SGh(r|UwX50o^#wiC zd_#+J-2~SYf@);en~-JKof@gOhE&dON{qn@Hj8EaZ@RiOoU-qw5k<#WL%4q|c~f6+ zNkR5s@zp64T?LkR~njh0L+J~1Hg2sBV;{r6>(WJEl_D~cKc>wz+`_cBU!ET)W=yRnzf#xBaMI_WpLGNjHWjnM;_W&1q&EnJ zmuAN?w@##&r%MPa=a%Ryr8PjCTcA0-=AZSnFeF7cjy6n+!I-vDm;OphJHOMX%Ne!x zc*}s{ddZH~#bMs3rR+PpGvozjneXOh)%~*Ql&gw=*nN;4Pp{>8x{-wYagJN+vxNsF z^fZlS^!V{9KYnSY$dnX%lBg#uYc`vpp$**Vek?1$t(nrWIq?_V2>9GpoX55a4RQ$=GNnEEn zUEG}C+c<&r+8(Ok6Da%65(q<&1m&HixD4M4^*Tb{Hxx=Ubq4P9GuLn==X2DBe`95Jrpia`8 zsD50C$Olc{5)a-Y`gicptZ}jn+;80wl06zFd(S}n72o=X=%m5cEsBDtjz%ueY8Ig( z!R+lVl?d?b1l(WJG9-E=S_O5}WZQz?<4-yBh8ARl4`w3uue>d{RMYr0qs0wga9ReP zFXe!TC1UmpBoE0zbp~_k%6C<7R8QU@hVN|~s1|D~7QcI0MR-V(VTKCJn zej`;b316GpD|h}vwP?Q7?=^WjVHl9e74c-!G1paxqho9^; zB4g~`*S{nIGxIPUf&0t$rxeYAXFT?3NWy05-Au3Tq-B#?1GTJPX1W?}(I*p%T^MQN zDW!W2`!9hv^W%p|LHU^#m)iz0u1HolXnfn2-JkE78>@~CXbY!rynOF(Qpy&fjW3u$LW6J7WkU+lx3lV z$PQj^##NPXH$ci9GF>G1VCsF};nw5C+zW}8kGM3B9r|7LJKOyE`zJkxnh9v$lEd#r ztIw{?$Msm_>bTqh$4!}RLUMkzl_SXFK;J99n!a(e%|Jy9))XW3(_+{}o zw7w^59vz2ml%g=jXA3K)w+v_0rBK}ch5o_o2X_%|+lWptpr2%X%?DG6e*xyD_Berp zadhrk?q@qr4v2KQ$Z`kUaYO<5?f^||y|xQgRpwI@0m1r@==Lk^8|lQpRiQwW2SJ-zekkAcAx8y$OB%1jSg!tM{jfTSmE!0+(G3JiTrd|3 zw_>-ixKm_Y3G-pLAV%JHMYPx5kPLHlbOaFRjQhQU(iBAScd%z*_#zNBIY<>EM2)5M zUUr#sT8gzHp9Z_mZ)WvaHy$m(W8I#f80JC3qS|kxD zK;V>n!3F1|;5mJT9`z9DdQTINUL0On=L#kTH)<%7^5wryn?bZK1(8k6Rhek>$YOhYf^KU}|E)W=b0Hayi2Q6AC>&akZPPK`(Uw0mM|U#*mn z@GCv}H&vC&vHop)Sf2o9-dd5eb9yh?Ojnv#ZRU3d^IE`EbiZ929}Dn;mpdt>qYa~i zt$gnGQCN|vP|#k+w#&Q$n3!`N&|%q;oz#E$lEx+mi)z9m+o(X*&h`UtCxG)wKpGS; z?B8zXC=fg&Sh*M^N6$K7qB(MVQUYyS3NQirC{1J190ZO`Bvlg@25D{R{|^rKhaPM> zEjdbG9vUH>04O5Q!NL16Uz`6|qf?B#b$4%mcK}HO^h=0ZAl>l3edQpG0nt4lV!mQC`=F8;%$mkA6esKjx8U^W5{nUZ;^y&XZF5PF#d3#15Fh>`o_dYy zvJcO@_X&dLvdM&VDp`?awj#xj4K@UDKFvkRri0_7wW$;)3~9oFh{-Y65!rT(p21Ca zI0PoPuqXANiY$u}VU}r(17|`#jzUE4$U7VnZ-VsX2%VB#sZ;#<&0c*qyw>CK%gjSl zIc0*B*^KJ9e1>~f?M3NJDzM4%7QG&QhI(u}3L~`UDPhGs36@8rmWK;zyBzO{Nx2($)@pplm2}qFV+Vfwrlo$L zAM9W?XQ)_CZn_b7ecz)h+h<(0T8}A*hQ%LL=1aZ>ipL8zZ9As2*>@I@^RLNH>DCh& zcE0230Li`AO=Wk>1O3zFfUdx%Hp8Nh(NW@kJoAq7Ia-NJ0~W(^wvnqqRt1W#R1tq9 z8Cy}IPJPT{>o3g)xokNHhmhC`S3^HcbyP1a%i-2N(drJ2+B{OUdBa5rgmiE>ixu{U z;TQ`^BMHf;iDJP9!ghI`Ww{dI`jhY;Q|IVXNc^DOd-yZ4k3-FU`+yFi@&Axw#rTtA z1sJxcmzP^jS8W``AL(Z4%SBj#8}V^B#p{x*c&Tu+H%V_a+`Zw9y7~~#7Qne?r^`V90@{Nb|sxZRv7X*TE2970HGuCwI8P;1;1jt00ap}JYrCADPfgFd^h*}Qdam`hq3aLJ%x z&^GKlyUxZ2pHVgI>-6sB?Vhy+u);M6$#3P6W$#fq9$Y&sV&v2*hLz;{#!-MxG~Qhk zDbn-icDXUsU@a&%hB1D&al!q!buz|Hf|`H|a#5^dyHSXFmG`AV6jy>V>2F?2^IroV z$CTG+w~LEsIaJzL+#n z#O`|10#dNjUs@afltl7BQN>p#hoT=uXG2~~+_=DR;evba z3u9J~G-;wXjR%qQAlIi`FEVuJeYfWPV1O7>9O&?bobb~nGDF?`0UdRq&6d@^c^TX& z>G*^+bIp{_{P$*{+QaBFhLnK#B8t)JcbzaFJkR=^{x2`W!h$B9*M49o94N(v9NduGUuXwb7)XqOShah3mEXO_*+c~u%qNFIP*itdRytvHc{{e|lYgWnc z{R1N&L_O{Zc?&O}h-uC9CS(Ag?)6|$WvJT)9I@EU;9agKLu^HuLhb>@vMOQ%3 zPH|zM%NZCk`?KO}rCNvj0A2Q}x$xYl8%_dJH*TCkaV|<$HmScG4F*;I5F`wmit=)7 zu>;#oKpPAELHJY_8#DsqTLw86Qu*x4?BaxdoGKg{a9v2eJSCmluTKxK=&89hMN z!;$AoMn7Lyi>8B^APAP>4ViCyM7KTKqXVn_?c>BlTySh^#MKtx)5fp3@1wkdOzf%7 z07S5fX<4B(#$|>6Wb(90qmO&CF0w!!kK+Ycg^}MtH?d!r6~_JTFW175+m=3JxrmXU zW+DvOnV;bG2TDv_zk639lDcuGj+t~&X6B2dAdrQVe&;f6Z7z&&*0>@Jv@E$JcC+kk z<^6x#rl9ut2Q15D_(uusii%RSNsn?8L+2wwkqLENdtjFx4LwOOo3%G!E8!UA6;lf- zDXA8%lnD?vg{(ge9PCo8aEnIc=$GOaG0(0v$7+_9(m{7QX-WB#eYF1{tH2I zNHv)oViD6Ir6DnNcxbox^265Bs6{kg9;{6J$pOkeb@?*fz!it}KZyIUyZi&@o9hIJ zK_bVbsRk#;ErrY{<_ zcYix^{~M|}1cGKTC0_SDXNmgn_xhJU|6Ra83`XMSxF(c~sj-A$g}=9oe>)o@LkI0q z_)gBD_4mb;|Hg>)za@tFBO^%~bm4J`n4X@AaUjw${9ovn(%?@GfZYD1y!8r%_=g#! zW&X$Y|1h`z+gL-N1qHvr0Zh*Sp(wFjy^8XM0#6PazVEJ?5OP>AMl(mBD zusBy{91&RN-){Z)haobwf39R<73J}7M!CUafi>8vD(p$bgxHDU^V>xSDq=WoLYxS= zxN5y9;H4;PUwW;=2FqD}*X>5*5MFQVw@T9gG~$$%h&&+{24jo0)~4EQ$ZbNAQE6M(-jzE@7;Bh24ELVr{eaIQ!7YMMVuUf70 z;{sZP!xY?&=16Z>b|vM1l*<4SiY)b|8jME=!uR{l8AF*@o-nHCaM_+Vyn%-R_wN6w zp=&~@hEiS)yV|e~##c5op`j?kA|;sm>X1kmy_Vzq*7r-{KUk)C>Z?CFeqBTR*fF;E zj8GY7B>U~=w4pKtB36U0iRFyi2ebV@{JmZN42LY7# zsu;N#M1^6HB~IkG9eHYj`+&B)y7UX_#yj; zJNMK7poQEZxc|!l^k>z0QrKR?M^XQy2tN~&_l;0Rd=CA^^q&;5@JA6sMs4-G)8SD9 z7tSvz%foeM_@!}BR6Atfa2d=M-_aB&cKi3m%YUvchxE{Cy8O?+Z%!KU;80_qE>x2O z6j^5gQuv`WQPE2B|D=g#=(GO($q)sKppU@5Bw_v*vd!z&)7t)Tv+Mso#8!j8Ce@CG z=>`k5uigAXt!#KMPyVm|h7gT{Y~+N+;CgpjbKIrqh89-TEq*5}s zMg2zJ^KM$EWQZQZQ3}(!-$%}IgUX)nRZr5k23x}sl7NhV3gfbaSiTdN^JG0TxmUc; z-0kYbX%kXD!YiQThj}xsurhUmwBljNtWKQq>iw@}T3hzdYfxmr3r)cyG<{k}o>XE{2@J+5Xg-x$Rf#Yn*LC7OLI7w`-fsyx z4caGC$Oe>T2qQVa9J@a{^phC42nb&aO!*)Hps{mt2i%Xjlqp@OCgvRH0H{~`YCA_X zio$u-R`*&_ec=5Nqi>M@g-bD3NS^ZxVz8j^(!Eop7VG)Wo$OS(uRDdJJ0D`N@w%BN zfdVtzoZiQs-Ig2K)J8xLA|UnCMd-pQbH${5?8l^s$hzTfMcOr#yS&Ph$_d#SoYXdO zwGEqC_Ve+lhxG9eytN{(zq=a=iO7Dn(>vUC6rvMc=mmNtb#qRi?nej)`cdi?=e%sA zIV4d?Jv~{+l}4SlqN4IVT&eTC+7udlNV-l%xJ%c7s4W zX#qF178Hcg4Jn0jf1r8yq8i`=_P226Y^cg$&|>?_;H&rrV7xikWJ9>& zTQm<<*JBFE=WyQm=!$s2J4Bx*gBxhwnk z*7td;)oOvpNthhU9T2|a^$*~oK;ZThpCJwnQR(goO`!MJc&3( zf0I&3$vEFlp?6)5aeOTY>9dPI9f}*wVAuKdg;w#?p(+2D?6zRQ+7NF+9;2Xy8(KsMDN&z9%kDdUDmQ_o0bOb_FNot_IY)dJA}eY!$=~*Q#Vyu(|+P9wWFIIh6m}4BkCQMC_Fozwf_8Jdl~W z^JxQ;?=Ri>-8cI32Gu=Dsxy=!B#hir$P}+MjJ(P2Rh>rp4WxsuAZ+e#eTht6efTg_ zm-#+qfEv5eO;TL5Q`1jLAo)%HJn^IV2l3qymIi&lBl*C~z`cAso%7KXm2>}te)@$$ zMX@d-m1`vwFJ3kZE4bmOukk*Q_c1nk3ssF5ovHgCecs+Qqd4>d`& z2}8t&In%G+cU?vzRO#~!PP7f08c2{ib=R_{%koH03AS^8GN1*XOB2({1(pI^An6k^ z2J4&B4X-(1#zeiJJ{1;wI~!zkA*MWB!?$_B7J}!$Oo&SUI-Ik)`H2k3>f9mJI9Y`M zRR>;`3A)k^{D+g%(hC>E7q5|-q!mxG*cE=E$h+gZx=B}hG5Eqr+^lfmV*WMk`>^tq zIK>|3>438+gruAEDsz_k(63|i)<`&ML2xo(qQijq#(DYkLl>MVvCyl|Wn`RqE%H7^ z`^(xBHJWGdD+ctAMLUq)c6Te{%m+=g^^hILZ$Zu1^lPPL#GVC#_w$)m5oq=`UEcQI^x7W>R^J zaa((-scrvYaEC7@{X(-!EdX26`1YcuDsr^Ey9u)1i0NRsFx_` zu8{|<$@c9)G@rE|LLaN>_Rinz2zzBeP?&2f-hx%l^^*7Mh8p-~GrPeUa|~S`_!S27gVkeCbM>8Tb#{1G zeOR@hS&_Ag<`kIU+>;*2Uc^O!Vt4P#9K5C09%w4-ba>-+H7~qBG(MgirBJMsh*cb( z^P5s?V%%D~6@n+ePtVBzc*ORc7xCWc54xRg9bOB_j#>*a>vP6<;Uk-BBIkOe^!}6$ z(ya{cpqlu$vvB=V$9bouQP7RNzDOJO74IVSnqqhlVA!Usb;4+g6^t#`)*9NM;Bqri zbTwQgYKKLG0W5FJe`uA+j`Zjv*m@~rUuxv(ApSj4C#iG2OXSmL&^kl=)6Q;*;Lcl4 zzMPVGM`8gOY_KBK{8%p0&Eo_H`C8O!pK_2m&wII`d?lq-IU-bSmviGV%bFgIFGr1qCkZhr;NlgxXS6672cbNOX z+&L;dn^`DTGggQ7Hib`!a1tkJQf;`?_uv}D3xSS6G(Q67V7j(9ofa%{`i1SSWS$Yi zC2Oms)#xXAY3@aIBWDBUL;yeW6n*z{L0u{_?TkoHMw(|3$Q3((*1}etoB*W- z0B>ztzwWv@Z96qx1OL9~r){~MXsbLu8(?09s`1NSBB~hbolwm)aMfFixBMJg6|igv^o6lksQ1ww4ln~Rrs&;SD1JKI?WGP!C1MJ%3+GQ7rHGOqzhF&g=eRp z&O6h=eZ9dXVeJ}>tc&AuGg*(r&%9V-l-*pKL1Q#=r?FdOCA!V zD-W{DhR9=Fb-5s;1z{CDr2``@E-dNET)G@zsfNIktthFRgv~S<7i-wPJk#!@pnp z6pMLMLR5$^Ao?pVAV9zj+s~f#vMK(gbZ$b>T<~Xy7g+)Y&(R=+q4oQ7;ffrVnIW{? z&_Vv;(LmH21fteV3Xa^8ei7d#vOY_$7jC_OV!wd@%KS7{@vPHV$KGyVI3Nls2u7Z3K_Vm4aR9+Yv5&U2YU2D;^?aQT^GaqEx z%*5W&v%Yg0%qdDi5a{sk(+9c0Pb|XdKCu=@6C3(`xD?!THjTP{XS_yYZXtjc9Cvo_ z9UC{HO|jPae8Xt4`tWj;h3Z2vcRyf^fohtFGR%a{0?R|f@Xh3-IHHs|ACA=fhR+22 z4Xk!(P6U;RmCSaC71X~sBT&BmLR5W)zk~G~^9KS3EDiS85E|_JU~8*!hg-TmY9T-BI6(+rUFo&!yc{3}_qfufv^bKkE(T+Maag^W9!lpsH@A4HCaf zLnn;yxJq;Vy8Q^jn}HKUAx`Ul%;?6-eWN26S|^n<@M32nVu^cu3Hmfjecn!Wg@vQ25 zXVzPOib6ZL(;vzV%KH!&%f9S!%MIHYUVH8u889BJ@s5F~!N4B zYA}2@>vTSV($ZI!qyJ(OcD!anzh^@mes6R7i_hnl#J4&^_y7zaFk8USf8&viztHS8V8G z#Dem|0#>)<@8^eKMlQtGR5RH5UFv#nlc-ig+|>%GD*PTBj3a-kiinEahl|`bxI~Vu z+N^6oE7XV2eTi9Ue%od67|;UqV~fTRG*IeYTyf4oF1C0d|73YpSP1H{&zS%;WCg>3Oa?Zh-TYnR1zmKFIPG#RqF#L44=_IY^_m5jUxCm<5>h;?!R{u42bb3BZnqyYwxvj;@E`4iNE2rO#UmtV%k#ziL z4f6!=aO-{4j*eN8=lqD=Pxj=t`DIiE5xlRTE%@=B1$>@Gdv6f3_}QYTYu_$0QnnOWlH@PU=gk+dEF0R^;EZFt{YzR#MOd8o%UlG0!~Ypi#5jS{_x_k}V4Q`eF+_pn+Qe}$Idi@e>1*R1$OaHb`wKc(?YK&%2w zX{9l?v1B$XF?M{XJKFRuLB9O(y&MBWf4er3#JC9S+Ytw6jj{+oH!<3lWzu(s_%8r# zDt*tpwh~lohczNCbooFijch!68EeYh5mmbD=5t;0u3W}%!Iv8+U3HZ=dzqiPkd|5;_#Sn zR%7%wKt}I{!|s-&vbmg+AjdVc&YF{jW$$F?Zt!6(modxU7QJWPn=iCgb7}VMZ{+v9 zXZkH-d~96rvJEtcR4QCQY_UG6mMDDe3sHE;Du~Kog1Hb$eCr5P_RKWmSpmE=IQ6_V zxXoX+xBK+9g)aLXV?bfP3{)-EzNA_BCYlvuSP_N&vwNh-=+I(LQ7=p_)H(<2BAfYu z$LJscH7I!187}d_EBOoA^KC-f9gJvZm#RTbMG?Xgxo?d1nXS1Qj^nb{E>!7-pEN(Ojn&2KBf_s4A4gn7C&cP)(2Zw_McXx;29^56kyK92G zyS{U0?wy*M@6P@1`}5VSI<>3M-n)9QUh?a;x>t9%Qdu(|jSVG&f3jMFkmy|=V-U^~ zLmA-{T5cWJ+rYzd@k!K0ZXDcl`@lQK8@W!GuZYlTR0n;v2Yzsi4fn5w%>p&=wvO(I zw5|!FmV_5y#PC0(Wj7 zJbWATkP`xaJ_3^xn!zC_Il8M9ZQwT?xD>1?7kd^Oc&F!vvqX_!msWpGBfuO@^46_r zU!?{dy6(Jwc_5C8^CPq0`8JvEyk0{wd4bB*N+Ic9s)12ZrW+|~wfKH7(sE7bGrdIM z!Ze+adqw|hs!|Wnk`r6dt%JK`7fFr1O7lLL-AcU4uOKF|yv1JYla$?61(qqP#v~u+ zCLz7=&|#6^#^N65A7@?Vk)S&I&>~?dA}0zb zW=c z_)c42h7-D^ni!NVPBDa>Uy?zVbY0c(^ZZjTMSgc6x>DyBckggJ-s}oFh;ocG3}r1T z^dRqv_r7yY|Iv6hu(;u@QGNNSsLD-jno!-lL4(JrtickGlrh!8G&R{Q`H2Zds7lh3 z{`%zrw|mH@VJgXadwsw))2~6xR>*EVJu=i{&dnh2mXE!UUD$$xckq+NIq~VBO(|z& z8<|l-MNCqxQhF@|oWay?N%#GvhVk&$^H-kZ_kmt_s!O3qfg7hltF2;*u>P?gs{{Oq zCDlIQ0ci`%wrAL~D>Hj51)~HTB8p?`9#8R@9| zITUL=!A%l@noMe9yZ41omERQIOORoGadh5&l2{DO9Zs-MuI}wynT`p^z;+C&b#HeT z6f2;0v6Zc2-w~m~3>6%Q!EjiuSYJ@X&>uV+3Au`I;zqYd+;ZHY` zM@&$pKHU|yFcL@k<6u-n8ZEJqxjWgJ3bx$XG8}X#M$u&-M2+iamN+*6w_-Wd_hp$b zyBm#S?k0a=foq6SJgZgPUe(M~bn^8T)tBe=u)|XsB4=Ll`rzHu@P@wffIXylAG_5G zR%7Si$?0O`POT-Hk~7?&NCDZgL=;`A=fUb8mP7cep9h*$HWZwEa>2^BxT|AAFcO3U zeFP)CMJp$^Hu$=MrTm}=j@#M{474@^wX4o~@2t7EdVr$ZcmQQTF3(kBJM)S51BB@weFkLeN&#=`iYnp(@HZH#SB z65~K8|M3L-MFFMMnU2$=wt&ob4<|jO-r+?|`GN>*c_a(xEraNCapR9-Ot@i{fVpt} zS;8O3!I)^J`vVsZ)LF-fZs)y{Kyf%6@}80J&8_bX4(eT`%anqCfk$>kcle(=djU%g z7>0RXBYdq>dClGD(|jO`T8H@owOR!_N_=vNM&wm&HjaqJ_Vhr8t?vYgJTG?g|n#$ICWxn6ZC zGFR2^z}>L331&&=y(f#KZy;We(@TMFHB~xX$>FVso>ZeL|!299JIp~sVWT8#z z-aG|&E@ybCJZ%$v{&$QuJR=TC>4Qw@v2yu6HvR+AX}el{h!Q;~?Gs zt-9e$f^)g_J<<6lu#*q-k&u3@$=H~15ESlvLHzsaesM8^%I{+Kwc|_5Wev>G=Cx^4 zy6+4cZ?7EHa@;H@Lgu!W&wjK$!7HNU4N=!3i<)k)8E|l};2WGN+*lXZ3uzBZ&?C=$ z!JpfuCxe|?Ob0X1-jX;NuDb%Ey09j}|Y4IM??%EC5T$Hbxg#2jrU zw%$lDWLU*dEHl{zHzY1le-jGtrAD0#H6O%4Hl=67S3Vx4XZYxXx86%+{7}IQEuqqh zOJ>s*-rh`J+G)KQA1u2;`HRXp7<*FLkAzDY8nVks3m4{N7zh7K2!1(Rk1!;UG-a5c zJWT~9ct-k+$!g`{3-_7Q!il1#+Yz=@7_U#wuOnZPNQN(y zuIu~s5*Wy~jVrZ_4{!-y+XCHrCF=IYG~hWD0FNRJLux<3(r|>I_mczSKHoj!u?3|u z^K#qs)b}(3$`6P5A3BR^Y!VwnJ;pKBYI{speTD~WN!r#^)zF}Y~ZHT?Uz2O$wmv1M&V6@UHm8K5s6&1OoS zPBl}-{HMM|^G=}ja=KyLN?VhDR#;5DY#ViQ$iB4N>}v_ObcK2qLqo*G(k)Uz!rl|X zDe`5nT@}wSsTTgWhlyf?Bk*diSBMLF5`QV{tOPH=^Wh+5vD4eSd6HSow{PFmj@dQ2 zXx&yj3do#%u-=YMJNU_MIXz4}XEw*E9yT(#&fTx9WMI+iEF?^*5Hn17C>#)_Sx)0g*&W1myb;@QESekNoQtAf10o< zJg@4I68ne@Pvu7fVX`W|vg$tX6?5p_%uPu%(hm9Dj|}nMeX+A z#n%@-NkLW=WOv6dm3tb6p>KwlGK_(87V`2zHzpW&lu6XSS#_J#3fkR@87F8c`#9G`ogPdX+Sb-m8nX? zz8Sfl3cLiSPv*Z*nudPrW)Rp-xH^hKf^oF$o7k7`IXFvzAc-Q7U!_Emx9&@M%opZoSMLO+f5vswJpW8iaM(kmW9|>go;>B zD?Ra7mYpN`xaiG_kSpJYh4FCrl_dG}>x_==b5O+!W(2=3Ju*+aSz}bJ^aeTdrj;xK zU}Yi;pG6Gh^Tcz@;5IUE?(k-w%oIpWAwm)H!$OPzYUOLyU~>CAoUuh~cXzjStWBFP zKd)5lL`sR47%=`(%_NsUX+?z;=1+dh-Z4+*#7D`mIg=_j`ZM{So|;#5B)Oa@2x z`Ikw3bUSy?TqKv7Y3fTV!dwOEaaGC%3$%#3*Gop@8_Cpf?^hbO5eywW+V!0ya6seB zYoMk528)#zhj*hG2;aY+tF?-k;xc_a*Qz`<)26-o#$4NimTbtKT&~lOrB-pEsj){; zB3>UA?6cCaRVRLyFGX;rE|bzobn_Cz2(W&(rBrm1po- zp{oIAY*+}#9eqN$mhi9MIU)|+b8?o_=%k+$cfr(!NHkj~smp5#hc#B;v>zWbszy0@ zElWwae`WwQaaWFLdpIVFN6`Zb!*e3=d)@>`7EuA6Q`3+)R6q|N^_GXK1w~;_))lb> zf>C!YgDM)*G|s$G(1s+u&-TiEW!>I-XWrDgtyr5~K~sf8wW z^|0pdz?JJXkSIm^#1#BWp1$t6jp}+hGjQl8C0&>}xWU=6e?8;G6Y&EeKZxzzbS{*w zYNG!%59MqH4359z6wx*>HZA<=v_k~ETa~7%cVN$0LzZ(^bObxd9#mCQx<+0MtdjP(QNh=17mI`qv}8yc(PX) ztCP@$sq4&nHNj_~aaNpD2GlvtZB?!@wB4#R@)x&iOq|@2B>rkp-MdklU*oDv#8{!R z!XndcL3z}ox9e=kmGBbU+b8aJrk{c@uRbH8W7-yN* zC|V?Tvn{FZLv|&?Dc+d?VHAo6?*Yl_0Wk}+)o`Wvs|5-7U=PNfU0 zL%`l6aKkYTpB>nR9_7m>miZ;@9LN~;a+ zS);M@vU{qBfcW9bkC;U+q2$j$2d7j^1tGphcnqP!Y}8DdHwmqYQ6_(Wsw&WWtBs9q zCXB|cEYxZialR45|K%RX6LJh)`sCF?S)`3M?ql<)=W6)o~M-y5`GJ}RNht_+lW7$5;&%AQ*>L`jlIwV}G)cPz&)*H}`H#E8$ z=n$NI(d`U2&MmrNhCBw@n-dPzeJNEWhN-Wq#P@lfK69R}X6a7cd>Elsna$8`br3J{ zp}2Sy6s$pRMi8yYku_{bdHiW*C|lCX66@Co{qFsWx*#pS57e(nelF!k}RhWqnydb-}&(Uq7QwiL@ zXAg-t+EtS9*+gq$ezsnwHOO7s2VFE#xR5WD@}~mC=M^x z5{*e(4&0{X_pBdh(*0@^O8D6Dh!p&~k=I25e=P`Vg9rE6d(Qp2ron23)?iy5jTK+Q z&Dm1I=5D8s88|{KbQX^?OESi7y*eh4^EC4`p#iJ2^F%%y>9wqQxxsN`K`=U* zRa`gV0zsf}ql!&#JuNS|Qo%qeSR#A<*EZm=>-V{mc+>gE4m2S}SOvsNF=JVCC*-Eu zFCpEqNT|Bh>tDM$m<>`J_dBhbJ3f$kd>N~Uz+j#4^5|M|tR%!stcyTCj+n*$VQp)k zX02g6*b%dKBO)cDJ*g%?R}fOvjcRtELjizovYR}CdIA;M2VLsEcxWiQI>O09M%#_u z1eUO4CX>l;F1QckZ)C^XcBkgiw?erk#@sxP(w*BDE!Qrt`|Vp}IXntaR-MPjSYRXu z#59)qSGKilB;#(QO@nBf9Xy5BZ$uAr_xoNWw6oAh#t{nRRPswc>DTW#FqU|+tG?6_ z2S#8^-f*8@8sq#bM@R)vy5M{=1&+a-kiTnX*cluXNa~p5pGu$<>7iX`GsgOrQZ+Y% zLuY}8Qo~trz~@EPzRa-*_Fr95a&Fkqd6jqzN1oBpFr-^>F^!9PQF5Z@g}<;Yn%V3a zx#W{u5$2?&G^tVkBkbwQjX_)eI84giMzsd)xw-yg@bs*=hA*o7e(T$XBM&HfQk&XZsJ^UlphDdz0%_G76ru4aQA20hY@q=m=V? zK)4%;#|>$EU@f6YcWQu#z+Tm!TTi_uLX4dx))Aqj`YO*s6*+8enW-Z+l`>)PXi8X z#h(Y_>12K>LJ!XeAo8J8&1d#AHhk`m5QPeJn$37UlgNVbYv2@fO*#D+Vdy<+`{s zXl%4&i&iZ5#aD`LafOh+_ZjZHqL_3Wt#v~j0>R0qyY#+T&|J)xxv5Sxi`=@Gw5vxQ@`(v=oqI~dh1bgO(OG2KF;lsJ$-9;`#U2fSYqkl8hYfdbHKn>VME6f*IK=tH3 z2J2VE$Mdht@T8dH4-DtiDb-uISIf5@pNyR??=cFEmztIyU)2Na-1ZsBve|6AiVcD3 z8+m0pNA^=!Z7TdgXZ*BcPb%gl79*)Q$Pq1K&^jl%Q>$c9LK(-l-Pq($=^@qZYC^ z<3zMC2VVK4R$(jsOJ0w*0L~3yWX?yP04kN zRoz58AMwUgJ?;ADr6WySZ2vM>TwNNE0ItKh;{Iog$+^YD27x{z^?cD+>7%z;be)`) z9}vG7?mu*wo2IkpuC4){5yLdB-c7p+qywJrp#{BwTe8JFtvgv|3|Fci$3pws^6*~->%^}o89IP^n8Z1%^1x^f0H=>7P{kZFhz(9+Pd=Q#r~22xQm5;F*=ZPz-pY4 z^3rKH=&?5771sDk_Bl@dx?y+O?dJo2gaQx|ko>7_gG@w0dD6Gfb`vl^hHL~%3N~`+7A5C1gvIsZO!fDw7~UH+Oedo9b0DGP z4Ctus{9v8YssHT4a__5s^y{$SKtZ%abZq3{zJJTu{HSh zeQp|EaQb6J0deu|WZG?6JYy4lfyWJ@Cm}nczf%5tm>}0ng*3(wxe>TUhLS`F%edmE z!>B?ZVG9H<5Dkdj-%*83UmYJf+$#HpPABRz$cwQ=bo01tTV(L< z_l$@SWBUcN7zYD>LvY!x???OPoh?@#QP0Y6i(S#-#5EMl@63^5$JUBj;csh26}pfP z6?rdJy~rhLBO$|hLklad>ywo4k#t9!CpMDznRG+?IY zYu@QfG$)%B@B0JBo=Tw|JmFpZz?dE^E0tUtj*77WRVwH-05jd7R+PJ}m|SE6C*|=T z9cU*vUytzE*ow6Ax@+P5wb8q1TK4gd08;>=3f$gHy()j)6X8{0CHTUPekCSY?q<^s zRwMVCC`i_{aXD(XDAF4S{IT2i&ZyA=={nscy`j841cQ7=5H1IK5{jmKAJAOLOU+1| z4qnl6uATPT1Wiw5zm2gFjl!fuFL9T39gdy8dgR1KdJrY^QGbr9(hX^&s!uue&vSBeQ^wc{kr0Z8uGE%>nv^=qf@PAL9QoUX(v*7W386Z z1D3$Kz{yo*iXxI5_s%%30RIXU>&j3Sic#I`An%_134E!ibVC$aMXvOS3CGcHihc0* zurEM^(n`~Vcn@6b2d@71E@rJ?FlepSbg&jDWNW|PxBQ20LdB0M3-&!J)bo>Ry2YWa zfPC-0P<>0PS;rvlKoyRw?qgSbP+84R9DZL>cz35*sp};s6KHDPxKea!4fmJWvSW}F zLsF}dfgw{Zy)%W$axSdtHx=K-ps_{oq52(v$dcjuVA}qRVqDmV6$;;XW9iqVmmk{; zDC!RQXml)Y^&T4M7JLJ@nsD{eLcWL`I7O1uKC$y`I_-9Jl~9=IWi)tVGT(D&8gEQk z-NyQ@xcqQS=&}xMkR$imiQn;eNoW|lk`d*M+ULqX8VYB43B$=|4OF!} zqmI5)Q?KHrFR7jHqE36|c<96X8%>rmEjAmhr8&>=(=nu~OOX_ENZ$q2)bRi!D@b>t ztJ(_`2)kcMgsSplf4t8&tuJwe8a&jSJX`e9{B*40%T-b>U%vpe8>g)Ht{twT?H=YB zo1<(4{c9!5H}={aJqL4YGDh75v_zIg>rZyf3feZ@Iglg&7!MlHvB_7qtX@*{)t~1@ zdW(r~hy`;Lwp(~)TrTwCV?I1vxmiJLhM30pRKI+s+XhE|x|pc^f=kCt*4Os+`sE`7 zRy9X!Bx~m5bnGdL^Ya8MlA8NHto2jpL-w`=JSg!#!7kAaI(4dgyh?&^!Lql|A!Tdz ze5_SoyUr=jp%D@$cQ;&0w+3vyFjfQ}9!Y_EFewPj(z>kc@{6nEd0LEET5*zOlVZ4z^P(KBTG% z*hesZPSA{^KX_=&i%2hC3qO~wSqF22AbbCyv~7AM-ZPL(p_7Y1{65F>@XicKx;gtrm&qZRi`^UN0E=xKCHZ7b&d#eT~ycZ&YLcbtO#^)f_)*t(%0h6 zn(wS@i+Da@q!_aoz(bMdCvK&zuw zIbsqYVl~vBlb8!GKk30^5=Hkx~M;8}7i z(gWiWR=-m%d3-~BubRQ89EA1a&|g{1>KA5`T`vOHTUu56-9cMeZ5x;6yGsTlecf@j z{1T}N@mIP`_{>&W*m^H=-{q^^?talZonSGxFtld3+Oq>}vY8Ms?M1wsdUiMxF#0}> z#|zF~=^O6VwNOH1TPHiL=18jA_RHkLW6Zf4r-7Y$MHkp6;DYL%{#%78hyB&{%$LyU zc58}R?Ea4F@}qQaelHyUr&Bs@I_83A3Cpg#Rss|hS)es`)h)Zv!RHSkRbE?V^dWI9 zIYkYAx6cD4TaKa8sEK9bv!$=iVL_Evhb0v%6IL=uy=zVbbWmm^Cla_}AFDW&PeNFa zI%B3~`pfN*@3m(;RZL5q3ZAVMNRuR;QT*Ms3wYDWUjM`E^FDq1xs1dW;NDW7^o+*M zQ5uRLlk#&We97vjtN;Oy4-XU*CKeW(WGUHTDPQ}}!~tQJbPL8b)T`<~!(@cHtz@WY zK>@T-dup%E2;vNraNgX7EqH!aT~)&~dZUj(Ydc*r6|KQdLV@J{l-H%}Sk@;LEGRVX&{)am?X?D5-!h%*VCNBd}V(YMVkmVMK!TkvtPn&0^;0(+WxdNhec$7Hx*jKWbdwj#nTvm zR9u`jcROwCJDV0QmH@-cA+-`)h^cpGLH6&(Y4fHtaW~AhaNGivHq_RhF<0H@P_x}*% zskf=08CjWYsFdNmRl#ei;W7w|#-SD3&^QZFs(YcWA=%sS2S84mOfPM4apg$}0nZ}l ztR|CV58u@F3o}*bD{MR$42fTK7(KKpCbSa)l zOFqZ8qGON=t)IP*UG7U&tLV$5I4(DH{uz>Jpk)or>C#DZwt4${R z&)g)Sc$1BbxOor^jqb~ygHdC2?+gDqyui{^o`qu(AOk3{-n2d{cBEn-sr%6k|6Q2%H4 zd>!P;fjtk(Gb`^SEqw|?y6H=d-Ib-A&T@^QvV~S~;ro-or z`?;ZN0!fJHwsNumVB)1Cbc%qwzBok=^xap<_Vks>yBn2AWg^Aje2R}DvAX`zG~(G| z*Iv}tL@y+w+lPK4bEPwN6XuSv^6}NtXGB)Y^uQCKg>~5Di{#f~EG9C?ZbQj5_)v4j zS-W5GG3ElbUgHuMXuT!Y`&8B>Wy=1I^Q{{^E#DoeC^bi@p3)05fisknM+-8GZnL}4 zbAn7=L!u($QSOx@RA)Ll`xK3`s)%PGoVwVPar+S{qD7y#Hs!tQ8dY$ryH$HTOZBBE zIU$!3Rl9JHHD1oy-Jkfs9)F=^u8;)lTB- zU4!*JTdwle0?4Mm8En(mJfa46jdLqu{pH{x)(P~$_Vlgn+JBZS` z{#kYC7)oAOV<%eipycqhXP&vgv9~kE4frzV<$S0?vc{z7>~ahQ)Or^L18R8z09oDf%LMD(cse(>&@#6+P|jLHA2GWZO($(Zy$3mCw5^uAH;2&9Rhv zvbuD}PlbM_E3@>1*Nl4`TG#1YGJrO^&zBzciEh2=9!)c5A6ZsYX@87C!{w#oR=H0X_RVFzt6V3hyJD-rA+Kr{G>K>KK1P9U|x)NSq$Q= z?6Luki0iK7NK~81qcmffpZse1CUDFw5W{0_*KDw8-8b{(^ zd0U(~!_FqrHBMlhi!GbzUQD88faPK!A?a(s^IfO^xf4HAp;6XSrRWvLR?@aI6j=N^ zM$WS!I}4vk^$ibf>QvH%R>-YmW1u`uZ!N};>hKvFlf!S3Tc4a(2ys#C`EoD}zO&Pf zp}%u4x3)r1*mCO*#%Kfwht;Ug;<3oY7_J_0oA6U$J!mGO?WM!tCYW0M1~fzv(Am?+;o-ShD z`DRhaEzD;5#y_Ax zLsu;bEzpi33FB_k$({n`R1qy^ap@e`D|7^sIMItO3xk%-W%i$bF;4AXEa|&bHD)^Y z2}MP`dt%oac$W_>z8P+9!`sV@U(v4O_5-9$00Luj7h3amy$hMTnKXe0~uw4ZnA9fu@a^!adhfV$`0>v&y5S zFnX!v422mpvlz{Ye~l13Y7>@+@2q(x39@R8P^X3Z6ab(qe_5Cz^ck&ng0(M%a>6*v zR>>1qOO;bKf2}cx)$}|$ls!$@c26x@RSUc=hC1M#aeu4yldHjboW2aK-8^uesF1kg zrQNENS#9J&yLu6fo@l9dtj{s8I6^lUH7IIyO7nO^U{z_pS+MxbmV0#K_EK`SV$RX~ z$Zs0(D@6MkS8A8#JUsRXpngD){7J^R)M2%D*5LDtCI2a_C(i>$SJ}qa)xpn;Hnm?g zAi}49FFrXSU`IaJl=!JME^4UZVkyWauMGF;D%{;rHNi&uOvg}Kr5kFDfiPTyuzq|Q zHMHXXMr34$_X#Rq3iyc zB{|^1E765O^8-=frRO#ewT|COaG)bGB+Uzri3+MV5kt3C*N8{C3bD|1YvEZr!)hnO z5%x7~E<-trAPz^TS1pN#NN3<=XjST7Ff#NCM5xz zYmhMbIiA8j{baZ<4zI;lPB^V#ZIHl0hZ$IBu3S*2=`Yz82$jm7T+*F)RcSJkRZ4O; zCC6#C#4Zx|iG8*tFo&t-tD(pJ?n>zG%v^EM$n1XHBGTDaM?{2U@eUWZ+FosmUcLLq zzGt_y5ebMG7f^aa+Ps20%7=MKNU#nH6~MQq_}x)le>?AE^w>Vehh6;<4lbXEkuGG7 zt*XwGyzNB=ksZtV$Z~J`j@$z$AeS)~W$ASuQFPMv)lPuuDDPTPamQ-K)d?c~6SUP$ z8Vs3rGCbIje6+LsD{Pj7cDat94qdf268H)PB_tQ@gHl|}F|a!vLRZTl z^=_t^{xSV*U}jhYP8SPbTLd^`BU#PDRyfWOXUp>c*iMU1T#ZmyoBM^TwV_% z;$j&*E~a`3RzVl2x{Mv`l!xYHb8OlZLVGj98KOdzW?nx1!bBWE$Wbf$4I3x?3cZ@y z1IZefXNH1BCxq#R3pIU=-Jh=&yehZ0QK|Og?d7-m=E@vYLR(4JBU2i3<5J9k{-sJp z5p46y4ru|+W%;={^)ka+sokWFiX1H!l^n&LfkI_mt`f?U4PFKn*jP(}^~&errN%dM z^vn8MG)X+&2K)Y2w$*ivz_L)uzNZFQx2zBCT_HP8W!0poq@z2zrt zxhmrk!W_}%x?y!szr(l>#eeqFnT35j3F3TCMRMs4nR42Vz!&Dl2Z?g-0CB(QYi45B z#-gL?TAmuhTQypI7v;{5C)sPHqS*L(RLv)}KRv}F4<_?KJDohaI*eB4y#e?E-MlG&azXbnt}({yx-TA+xrd+EcVg zpq5{&o&>e*?sZ=C^|ztGO#D9Oa$=5uO`4FT&+E&Mg@%stbkCS}7zw<*){6RIa5m#O z);%7&^ewL&>PFUJ;{KO`!^&1`fXKn8?x|L8Nj~#DCoTCQ+ML+tJt1kZDNMjl!@;0S zi@mYM*HKp7?D?d=|`qBbJtT#TWalUR*{VR_~P`sy6N?>&ftvt zDdr+kgk^A^aSyFy6)wKw`Ro}@=w77a(QHiSi!kV{;@F>HtThYPz0F16Vwv$+Xe9xh zU2CsRh0joZcNcWGw3f9lxgK$B^_HnK=UqeB*?EFZo=-Jk!2U+oiJqfwg1a_Q?D>zR z8Z-ELAM$N>t^g|->i3Sk-z3aJtn2At|FIWK}+Uex|QjF-iF!uNWGtg;P;iBCztaFBB zoy6Ot_6?(YzW6Eci>`_WuVh#sQ$n*JVC@W_qwl-PRoLkMAyA4LTx|nxWXSNnQ4<>C}bB?F~g@(OUE)w`R+y*({r9{Y9KO(Teg-Nt~P5E`x?`V z?_kI}J8eJs(RZCfkcJxs;G9yKOAz`|U0>F@$XEHrIk`bUu}Mp1N#5}~!#D`6yNvo- zRC_{cC&PyI!LW4gcC3pV`g zeh`I1g0j_`Mv(8M*#6WdLCe&WU5+f?)eHwbnO`oN^=4N_y6p_tRYVD|khALC$3Axf zmo^Ds%ImVYCR(}u;+7;@K$MM|;iOgP`!h<+(HnCZPw|z<1uxqQbsAmmf;yC+AjWqG zho!CkLy4xc`{#a(S6ke1C#IpfpR#DrE6ijs%~V6E zG#nDZ$Ae1420*zXe(+Q(d1o}|E;lH4CW2wAP6#m8UOqKbl+k(Bgh(p@NtI2zKC0)m z+7Kh)xzQ$Q-u*!EaAflu_>x}OER?ZfxhL+|e2la;4;jjizi!Bm3pe}cFt)7O8!WM? zQg}55t?v0_Y}!r<(Q&RcfDnt1Y^o>?)Eh|RQi2d#GR*G_9d6-+RHlXy$jz&}Da^+) zmB4izh5(}ZptsetDV1w@X;jwiwLShe&WyjC%|BzLe_aJ}(Z9nCus%;;&QJK{^n2cW z87D!GPYC=Pu^=SGLSUXV|jqe^m)BA zb&gVE&HH6NaYzzN-0yk4ko=5)`%Q>-V3R}6N{%4J;;?wQIf<;ZFZ{T*(t2+%y=3?P zh+ErV7CkFPX#w&fjXGYryk%-YhNN2Rj^uB(V1){Oq!>k|P%14Fu520iEeaV^$1%b@yQWPeI&l%Ojrq~%1)$)=wnEWnLUq@aB#D#Sv|h=CySAB_BW zrv6q&&4#81i0%68R}N;pf*GhXbt~S41dGazle}`jPb8E4DY4$?BoLnP7P5>5kqZj3 zd|;AN#tiW>Qr4!#j`>UGkO=?AlW~MDgLpm&e~rg3l?S4IHjC`?Tr>Wgx_{FE$fGSG z2v1z)jWf_BAcfC535C>{Faw1R()8qi%^bhqc#@41&_fG`@OLOllggwHxqrftb}z>;A;2R?IoAbfCwjn3b;+E0B!Al$JWxk|MH<&nqGPn zqoLA6s|ED%y`PC%3hTY|EaVgH@(jxa<1atV>t;TI~~N){k6JRO;)Z=FK>kI;@K~_3^MNPrzep5 zXKK}d3hv*VsUQp?CQh3gyZR8`y~T--=s}Nv_VBUtki}RQE2Kj-IQ;C(TE=chjCS^D zDAgYeWdFT}e=k%0CZJfM^uE8gtKU`FTgeady3(fb#izfP`|tYj(+FZ!t}KNYN&o)M z|5>&T`R(-(?JpUWf2ogupiCMfL>uK`9f|%LP5w|uS+w6?k46}o`g@Iyg_z6cK-Q?? zpM&=wRQwJo9W)TZY|_|eyQcil2L8KLZCM~5l+mNb4pHdcCrAWonighv`uF$*LFA~f z3;+Y`Si^bMuRaQUSf`&{CI8k}1>gL?mN?`~8D|CgeG%KlLe~QA0;00%1%Hi+|7xcE zDzv3gmB~m--#Y|^O0SpabAHMTs{dsm|9PKN6w(LqiZXtZ`> z8en<IUP z{rAnR4;ZKKVg`lJJDx+_+ju$t5(EFm4sQ%V2L&JlRQml??OhBNDnH8q4I}=k3Rvl1 z<)W0aW8i4-xh@|6o0k7Uqb#bo9S*tI-Bu!G6)!L1A9g6{iczRsgdRb}r@0!a|NMXI z8dcP*_A6NuOQMk9I--9Lp8ZujDQ3)vt-;q>iI9D`LDl%%4m>|MkO)S|My{* zfAQc?3*NgQAiYFWIgXUn{{_MSNw0z^1Ui+0jM#r9E&Mf4zJ)@-ZnsH?p5bp%{}~I? z8!I?As;mDku?8_h&?x7?t+M9-WagjTK~?$9&zRXV(EnS=OVR|fp0M(Rp2@!jUjJWb zd?v{q9Wuc@u?kg3=ih4ZtGAySUni1iT&U~olfn%2x2n?rJstf_2!WZ>$Hm4zT87~t z-nh65&gk?X|CWM82^vA%eTiW!JH(}T@*ipWSfmCOqyNhU{>NX)XCcbvJfN=k_mqq1 zO&y7AFL5dnkESj3!pkGSo8(x`-{s7w-UGvTpY2Ko8<#j0WxH5j1jwGuC0g@uPBG5iAMD|Q+{ zl)&@xOnkabtA5@Tn5SZJp1}t`y%-a~e<0lXC3k;p-XRJ(;w#*~0s?~6vKg0Bop1g; zkm!&g)kBDx#PyVR^ihc7rJ2jtMs~YB)wf)#LkbQKmM5$z#)u;H-ttLFj)AO9iaX7E ze)sNOR7OSwHiPDSU)avkAfBkNp|1`+aKhkIRmNO(babM}nu<}7iFm)jCnDi}5)8*< zlh2ckhvx}B4b3?oNJ}F)T&zXdNwY^D<=jB56YcEl=lgSHBia8gbhwvNzkFLbI(B>O z(uF6YK}~%Cco!pjv5t+-84VqGn^j64zoo3Inm{T`Djo>= zeL1Zpmn{;A-0;%}0kP!E4YP_zR`=jZ`^zdSJ)IBg9=`xvh2H>!Rvifq%>WVkmRy(d z|Ik_Q5Ge>C_q5i{f7gdfVYf0@2hwnuVUp9G_G}l4624{t&<;6Je11IbLiw^#Wjvh@ zhd`k58!?`IcJ}t+-BgfZ^5L+0qSj*Wt0@pkRvz1LrdWklyEM=fdm>)3u~2n69)=p& zFFsdmX!hr@`O^i$5ndf1w3e49$AhS}G(-w9S_BDAak0}Lg9%xQHW2MBf%3_^=hHn5??ExL{H*VwqrGTN#I%~ImPGM|vK5Bx z|5D@vWQd_)-D;LZm~G+7_CO9@6^3aW=dCTI%%abUAmH-3I+%kC!c_1upnsGg?Qj!t zIHVGZQqIHwsP}b8Lk$Wej-=6Sic;HQLNq&5FfRY`ddUW~>bisLLk=e?x+iK>`R5JJ z!u%~HdZSr$_>DWCspV9-*E}Qvq1P@W20HG2HuS(r+rucCVp_~cm&rbvuUDSC@MIEE z%8nh|BdKz@5J#y%)i;kC5a8L)LhsnymUG4bpVs1y zK?2w&y9!2gal)4uU}XF%eYJlz(V~QNc`)(ziVn^)Vizk}N=Nu55L@qX_-2BYdAM_D zhpgSs*soDdop+v@5Sx*b(mPg|1XIZFsx zK!C;x?gV#t4+II`H0}^ASYyE&OM<&=(-7RTcfH*?buP4B(e+Es5= zy{l@8569Dk9@CRL(a+qXCaA2P$Oz?9F`@R4=_XsqBN7~Pi9UZHeI(AGy?q(7j@~J3 z_A4BeciuhJjZbG_SXF2+=yo@QA8wYGHTO+`bR~R~@7G%mF|j7yH2Xcet^4=Q**pj^ zG%+a3BQZxx^oN+oO_9P1+h1e9bwBlAoG)&u`mUUDyhm@WwZggS)O|eCS^d(6h(xL7 z>B>~-3U!&{gjMB_qP9b?T6tfI0T|cm;6YN;oZ!q|_v(qgNHR+QJz`U=7gjV8c z4DmdO37_^)WfMRiR1@!6iWbr&mn4h7WSN9kuC!#Nn82fV{=c&)(Tf!zCjF{=m)9a1hll$`#yU?$-z> zr77Wjx1iz*u9;pp2x8t^c#7L8Oera3bcx2U!!ECGU(xtOHZD(Z;%H2;InFBpSOkn) zTdJbPO{94dFD2UZ(Y^8gRf_4RTaAeetC&#(8kVvB{=7A;p4rz(?H9aR|IJNJDp0Eg zotgrM8>~GW)HfDywnQ4e(S9w*9U10ge0~tQ3CG?Qe;>+`$`06+n$5Qyhsw2`R8%cF zPh{b|I~5~f&}I_<)UAFe-CAT4xPZPb;*Df`;f-}9PPw?i`^zz*I!Zm|BJcw`-a!B` z)&iSkjoF94tsY818L1>0@7Rw~C(lv#w#dkM1-1V^;QG9c^kg(>L^dTt&EXt!7vJP} zJpHyjYk?-G^M>RV;P#Vq!rv$H5J?DT4It&krfCfW&i=!KkODhyF7#7jg8J)&O>Beh zvxM~Y$gB!ubMqj|GVQA1+sF%Kq1LlH?u7K@um?K#nKC1a`n_3MRpd|il!ZklW{+GT z{R=(o36VGM@Eu;oE+-?eD0t}1wFtb1lHsFm80KUz?f5R4G5t;sP6im0Iv{{fQiM`c)aWpr zrRm~i>XRB7n~-Rfpcfb+=X7SAj+;(tphXm z$S~u%3y|1-TYB~KqItov2$rJzsgQp-x~eV~B48ND<>7C@F?C5}Y+zARnfB=?en8B6 zLgL9#9)pPv{q^0~Chk6$+CnTawd!M;Yx!05)F4)tJafqh{SgjvHe8B{jcnSZTD=+J z_aq|fkw@J9qZPN=c3b@tBm-qjS6k(&Qh_wN{G22vgPI96B$JMn@$pF z@Gvplv{&iye87$==OEoWZ$S6)spE?`Ps{FRggt&HNvr%uDmW67T~%KSlNBSu=2_;I zuXy8Ry{4mY2~G=xUx{xiB!|{GDa(vLU#tWhAEU^`oJG>~i@yF2=Wo53=Z_Ctzr|;W zgNBa!_a=06n$>Ja=Jd%E6PzU^w;%McLOjY=Abd%#n@-tbBb`&t3?>7sHcfe7M!L9n z(nfS*G7p3Lt3B-}E|eF&0_?u*&=iZ@Q1V9=QJM;`bd!0#n?Kg?~hNXhgkf!W!?%}y%adJ5ZE&ls>9OwWNes@$ggHvz}n zUvjd|RKO@g|=jL7vV>4#a-1ogi{9_b)-qu#S$SKse=7GH8YCuGLk6POmK zB>*RzW!B_*yVq()_|G1+Y8$GGmm%%HQP5ZZqMIPufCf%W1F>?_G#b3&;>*$hJ(Xl}3rT7Jp z^tt&dIOjUQskdLgp*h+UdBJ8E89j|EK+DOY$&N;opc5tahRA7!!|Ot$96iUk3D>A4 zD2vQ#9Bo=^w<1Ex3$fTP$)Vo%sV!x=-|p!TbCK~;VtsCL!a8coxGgPT!WqOl0Ykn` z7?$nB zln}Vsnr|dzgV^5%xELV-K8z!+BU-?I9oUoD1d&FLb`$dqXnYaW?tz12b2@RNbNOpp z8qc0JK3eKOszLvzX!OPZkFcS^sqdr=W;EQ|)KIOafu<;T8m?`Uv%xRnrtXzhLnbq3 z-i-v$xT*lQv!tJW<*httY+KJ44W#m>Qj#EN+8!ELF9azM7%LVg<_T%Z^A+QSr?M{U z#vHobdftGff#g%|Xj-j&c*Sw)p;ntJJePdWui+M)K1dJU88S+8hJ~H&X3x2`hk6ES z$|uqjJrlOYlv~;h#?)6^Eh>#ZZEa?V-j^jp>}Jrt<*?G=^HwtGK`or3g;EeqM+!e4 zTPN(wEJ(rqdkhf8Qj-chw)7i&Zk$QxOCl(>-GEUzL8CbH3>WTuYk!jd8vT>BFA?(b ztuDXZ&!?Fp`bcVyJPYAC->EKeWW0dN)-_pVuTps6EE2#W;^;yB!R54G?9%majnoeX>f^G4LMLyH)Yx6Rq&LMHLQw&ADQyx<&oQy}tc$J9DEEprh zX~0z}#@7W)Xv&2jH52>78@HR^K0)W1GE9dsh`6S~19NLdTx+|xzdI?oQ8K&K4+{}3 zqE!9ggSx-kO7b-?nQpw*tDux7c641sQQXh8wyH6STBIm#37th89`e)3X=vXtJo7Mc zWSsUs#2id1yEsv{#?a^Ttj7gwV$-w_*=bO0%j{QHFYuLH55v_UMzirc*6IkURh}E> ztlW`jZBQoxL+^+|%sMVUdCI)0dQAGDG!d^dttwD4@kQfj#vX^$Zvt;F!Z$~3cGKch zqBV6m5g7<-`rfYru@~N+A-c;Mdx@{z_f;8)D|+Z_jg;g-N4&{Z3P9c&FEW-3^MF#Q z^dc{tnfnb?QDmI&1OL{?l&(&Vj10uifpJu!3(qHA77=PE{>JCdO|m$TwoqACGjXfU zR?5Ysf=O-l*6M=Sy=Cw@y+w83+`EH>jk1!&E^^m}JiRpjLq7XKxn3)_Sfo7Jcuwny z6aK|TmhpgBX6=34iiZ&{Fb2!wJ*on)OUOywXXZfHO3xP_xfsB7g$;itU5#F3+zUg` zTGW2vGWsdNiX9Y;qdxm}(%87ya7D|J+*7&osaa(U8kAD%4hc%lVd)i?wrhWinTV%irgGsJ9wV3oAT7*`?_PbQ6q1K7$?Qx zGg80i-em?E3vCC!bgr3XdYnCZ7$&t&yJ9N?3Eu#?2!^bFY%kuMZLha0-Ze44pEet^ zK4T;LW`E;BZ9hG2hHHF8xutM=XDS@ElPkF?!<4MqfSo=zNx4~py+r3ljBnJ=_AJPYs}&6swEpRY=9RIp=eGhgT_3ok zRJUCpedX(miJRceW4=H5afc>W~npNelc! zGsgNrEP_{r|M)4vhiSo>-i$lP?g+Z9OM>W*Oc9bGx@{Okh*33n%B?rusup3mK%`DN z%DA>{zTTxupC%G)-b1x=&P@#+_o8*mrR%xdJYC@imebRmntjHa*P^Qs>JhoOT3=J7 z%Q1$FlWp^?^?(f$q3?Fi@tk`zVg=rZ!mqb7n%ou*b@(skDlRZVx9yS@Iy@uK*tDVK z4eM+7=9oi#EpW}Q!x`lj!>^V<+ScWXZR!x~2#TS5#>e=jYn$>^sSHR6dm?8 za9M)7N>1w3*Htw2in&!%%HB>8@*^N%ees7==X+A*8HVii{VyS0l;>rj&Cu_=I z6)7uC(MeSy#O_>>)(IK3CRQY*R=dE8w+f9|8{l{ARxl;2vUTQJ=9gDbxVxPAxdvpv zIM>h?t&H2qSqv#G!O1Yj2w4wVvAj1A4wF`Qs;+pcNucPptT%-lSefM&LS2!V%i?I< zJNVxiuSQ_EH#Z&dwfsgDH?N6(6#;n*-5Szde zX7ZZaw~+Py`8vc*S@PC<*F01(tC8DDn2_r_V9|H4vODFnv?PbS0UvSR6blQBr=B4? z;#gQRQbYHIwsv=(o9pu&yccx_0~fusCzr9LBne7*VgaW|Y0Z4-pJ_5f%L#|k*@IGb zR4(B>=3YncJFql?9TMJGr-J89T#krM-Cf+9V*h@O#L5{=$Of2^mjvm&SxD>(Slm24 z5;!lta5UeZC!1HeMCPGO(^#d-jcWLB`U@L6MlBFT81%bT{&^k0^(FL>mp&8=KS z1|S<$6c1Bzki_f+hcH}5ee`NXY^EZoE#j+gQB1YL6yKqG9Dt}-BJXtza^hVk zX{65HGFKYtC{)gQ4p^ zE~dlti<)zN+0&+7UdsVn1T0z4_ilfcN}Wn;?6H5%$Vi$lJawoGFS&ueb3u$ z1fJtR8Se$~n@R1Ksnb9?0OW5N#Wm_&X!4Ts9(__|9`)isvV#C*a-o;6i zPZ&}4Z)}!Iwkf-W)KW!%3vB5xdUw+(YJ*pIFAinVqfyUXMc}LcJT_Rh0Sr3MO&P4@_bP8X(?I;LI!k%z`iRrB$L|WS7)9RkQI9vc6AW|E{!2 zdpH^R^cDmm)b{l_nfUV4d+@y7%1T$k;1cGwYNYVQ>CdmRl-#{Iym8%2 z8(I3)>ln1?JX$9G(;$+x7^NRAfPr&GssGSa!U_9(Ay1+W0n4|!)R(LC4^Doklh{E4 z0m>MT%JtjU%_$ewZoZi;ybXqYJ?<)OS~*-%9`Tpg%~u`j&R2yRYt`=tzhRr-?}{jh zEnU=<{!+}e1OGqnGv`g)hgkhUk>>anZsh;EA4lrwd-a30xHMDdL zaYzADLsScyb^k5~=x3J}g<-hIIk~ElBa|hyF;1eSkmEP19t&6SbG^LAPmCV6X-|oswUIPmO~jyCGQ<&u zCImPCz=<^LjA5W+rFzZDFE@P`G%{~>wa=89rIf;$=|76oZ+VqrOO}N6J&O_7ixd3 zUtiPROaS4hciL71gnsUp{jJTX;&-_-Y%MKmW&OSNwra)+%~g+TIqhD6<799EJ#asR zLjUEbCrB8yfBGop1WrkCQ}#*M*%Zw^@T`03xtOw|x$U}(z2-45*LB4<_jJjR85$gK zZVW6tQ=-PjC*B6XREn*4Jslzw^eJSe!^kQlPE|M^X%Q&*fUPo7KM#i!F(V8i7_8HWjY_+$Q4$etRMLqO}Ci9^` zg~UztJlkWYxwE7O>#AB6p&av8(4yS@zY(O);r7H zu`=@4GV9I!?{QE;+TTI$rCOlk9fQof#b*8kp*JG|-_G&u4O`8^tkrc> zCj{WZPbk|gcYy_;qbyGB+sGTg&l$-~Jwnxn1iDRIG-mtNSbRQ+`R2&{PQf678h-F{;qv8G=m%l7_Eaeux+assDdZX+=Kmgt5y zU!a(evVvWoS(ejow?tucJ6#QaKcz~XUkR}jt}uDiW5_DsD1f!L>j+X){jdH6m=_qQ&=XZOZJNDUDjywVHgL;XV zIQ_-^H3@9zhrO>6|@rTn0N!o;$|4U)A7 zS$QkVMVBEiPHv(*F4}Ai!*4>%nncmf#jihZfn#&n#*JhTDBJP{8K5}xHH?fs?_W*D8{9ovI%$#47@6%GhmhP1bV=TrJLAcT z2b*6n2&!%6F0zb>B;;Rqp1p>gIbEs%Xm^at3*u*uW)J)%eXE}@!6^e(s<(cKisc?z z#BYSfl@*|4wjK7`eoMUitdoh@Dg0t-_M)VKit%^7Hz!N@oH2 z1GoFP%92LA`bSaS;6nYE>C9D+z8#Ac@cP8tD z`R8WL=^~Z-@Mf{^&CUebDYDgzj(H*Ud9QWz>aRUo-w|{AMfQq!gnH9RX4gl+KpGBm zva(hdoewO~;#jQg%v*Y{z8F4DN7@@5v(T;s^#&VG7Iat(~EQ?6xgR%p2KR;v!BqGym|2o50r{WopkcEpEYwg z9n(KOu~XKJw!Y@|I&e)l51&70e=2MB$G2#-<!@wCK9kiv|FOsLV%uoG}Q_ypp1M@vj>MU!eO|zN-uL0aHc;sa%qu}_RKxfY3 z%xszC`!HV)?TugELOxdzEnQ2CLD4Im)VhuM!ArNwp8+V(XN+jaz9#cIu+f^0mfPI$ z0%NY4LNXVQ=d@{`jY+6=E`_TS7djYsKxZf1zd!?cHUQP)IeZHu_D)N(R*WrG%1~X} zBbltb1OkV-Wo+he?aXYe^*OgvUPy9oip&B^prXxRFAVHy=iO!4h85~3{8xQT1M^>B z=f}zjz7B$<4H~T?%c);C8R1nlIPeY1Wv6GP* z7B)0!wO`g(^=Er6?X>vGVKNlSd5WKV%)2P9!%uSu%kgpJ=Q|Ycw&eQRwh*9hi91T= zN*`7BA)x@^<-h}OrU$BgjUpdk2Akho+6VuIjbNOk1s9@#I?}47E}Q2KBP^r&s{Iv_ z74$z?YmIP>t=w^E6z+a&E~PY5)Di4>j5j}3Q!n^7n3urqVi3nWtr?l!UZXRsdw?@u z;ud4`YMYZ>O=&pUj`Y`FrMR~u*5{|D&P8L5mbgT1P-XU!=P1l z+)Ntc*nmlUYPz+mao?kc7OS<3eU zyGRMlm>5h>=rLX;;HvT09~^zk!O1;POrEJru614`;K<5m^J`Q*MWNO%`nt|iI=)$K zKgY-@WZ=Zg4#Q=rcbC&raV~#0e6k>+2A=MFNX&J^L?Z^AG~o#9dhm)cSK-a;wYg;! zLV?v#VvrWJt;YzibS)&u-Gdz)C6|2wIV+2V8TsNKGm9sX)@ zzDq=_*bHau3;jy%OM~{)c6VpwEdeTh9?LLIXqDJ9K3F4la3Xtaf$G=YFR_Q$kw%NY zlEiM>$o&nk;W={3v@B|m9V&yJd&<@&hpC1_@#gML+vk^~PW@7kmoagKCTt9o8?iq4 zebKkNB_&3)FAds?k|A-~#-Nw_RHE&FCfonXQrzFsVg?tM{!T9-7V&rk5db*Ji@%Ca zVF$T*rcGK_uFfA<*o*JZTaLZg?YMpC3 zsKHNTJC%=tFBDu{YOwgaLt}`@#*<9rxUeFJsrFpZOIcB$(f448%iLzivcz>l13SH$ zN7hCCy?SERPNI zJ(*;kZ;<^>X}cZ0e7PHv99Z0jD{erYVZ|X5QK;p3IA&r=UpLnTjXZRc4K^Q4uB+|X z{aze(a|-B3Rxa8L8$hi+{6gJ<*8V9hes=+I9bo(?`X{NO+p1OD?o zNY`szRxTYLv9%uFH+0{&O~btA&~bx}*|J|MpIBA1tz3m->k0ZOvN1DiWFLt#TV7n= z{@!Y`Rn77hvOaMYDQFF?rv$vYRb4)2tA%Te)a&zwwf43Yj@z~)KLRGW2@ysQv^9;WV8zVpbyjjd-mE9L0Q%SVe zJJztXkQ=vPJ;R^n=r=P7=gUmFv$7r)HkNddru5r_`pj;_V|R|L^DH~k(`{VOiW^ZA zW@hYlTNKKVc?ny6{i8+d4LjNE%r^(qR-Ot59M;~sY0z$n?F7aW1LUK_jp}&MvA|i` z;(CV@wEK$|gwi`--PGXKu10snnTO;*tF#MO$BVvxD5zVHr=|~%E@6C2RjfF!TGvJS zsX@E=z|eh0KJkA1YC8Zn>H21F@xIcp$Xds}m2}ajOCa|ktV|vjTj%D=JKM&N`}5MZ zxMDhLB7wj!?*6BhZ~}#g&dVL#=QD?8?jz+nIxIc|8lVhDeVO=X1>KRb@MFUkyUZ3N z4dIda1_y-<@Gkzx8_~;`||qD@0VI(SZsgn0b-L zm5mO99h4c#W)7jsRr!*^!QwExH;YE$4WxdURjE1Phf_C`n6|nxw#@|=-&p?H-^X_{ z+Z&-`fG!Uqh*aS6;Mc}Faj!>+(-qN!XwKE=a<%5)EN^nnUMpC7#u=4V4O;lQxRsTa zK!UCgSsOR(j&K?H&K=(}#RvJNPoY_$$FqV?^j4O<;h|aI_&gF;y3s(z3y}S%Jn@As zF{~Nv^SI1~xBku#{ak}5wV3U2e8MF^s4NP|LCG8KDatd6Cx80b#zqnq2JT~9{U)I{ zyK+*DaAGa75r;J~LplrU$q#N+`Sct-!YZJ;gf)(qa3U6cUAmgu)>|oyTGL1A%DlxQ z;v{k$-&Bdp{iRyzGJDn+{v1|A2D-K0YC{3OUMnIq4;o38iR77*LMoO6Nfs7r+@3n# z7ki|IU;WLdeCPEx)Tzr(F{>7T(&rweM|B}wD3H~qR4`#mtA~3 zO}Cr}b6a!=zOr`d*?I&vYTkJ`Q1$S?;LTJ*uAf$GaI|{W<>8qssF#w=ky=xuT%pS% zL#tc9<)5767Kz3B&{&XMPHa(I$MnKmm}+97Z`GpKb!kqyT<3H7)fcBmG2lKygd8)7 zVeDZqup+MKCUDGE8qZ;w?ymm%Ld!ZSCx{W7>H3;B(yLtmtVsYJ6sM!!)_Rg_ak{#4 z_TkfK3*Vco4e|C{ZJ+x+l|J)I-z=MQYQCh!4zQw8>7^r&4pWdr z4e^>*ZAN#rGcL=qKL$c=X8Gfok%Ra2h|7I#kk4ChOG#qQwsJHB8Vfo|XPqEhJW8Wh z`#ihzKKOnB+WAgpK$O?krQwy?4b;sutzW`dBctcr->$Cdb+t2i3 zMZCYn%iHishB@ZG)HbvsLBTyC66%$>8^^AK*^Y^d{T-!)1iTDk=s8q0>U@6@nB!&NBngos#?bFc}C-``iE3 zG9;b^PZ0oec@53So+0&530lDY24Za8ELs0p9upDhV zK%CoW(qc^wp_5goFYrbpx*+vIrEfF_|Hw zfH);+)7?*vV6?LVk0q%R6CIj#vkXARLZ&;(zp6z|_LBs+$B5>l=thtGqPhJii3x-d z6P1_BDb^uJkLS+;w~g}a87QF0DRbk$(Zw#tGeGf3b5u*oVp?2g4DK@k1R;-!em{@T z23HTcMo4mxT9F2c65I(H``CZ-P5!(Mn ztF6sUFB1cd57;+@4XbFfhxD>6D>5+a9qEB=K@f0DAm<*|!d>>X{o=~mp~-}nj`F~Tgs zAMuyqryGN(C-FfB;W%V;G`C|BzRoH;TseSu_x*t7ro-}5AE08Tm!P=w;vquyNg$eQ5>T`Vbbj@`Kk}w z!B(PLv{LD|_HWMpJ* zrq^0-ib_gK6Jukvh>Bt(kL_nN@H7gD{jX4af)s$jYHe!np4}{1P(VOwBe}UbJtt>r zFg~qQik1&WPfu^1E9fTren%w;q!O^J4oXf=CZ{S%6*WTWq(fapQ!_c~^+fAc;GtWt z+osa}kd@fv;WDH&iJus>nM*eQyX|TzkwU<+Z}}1K6@)(ddG_`=LYzH4J>3*Z*DZ@F zhWGJBb3S@5FC_f0eqa#^fn@zp3zMEUlJQ)u*+Q1e}ec zB!(vR-?e>Z{GY4-#~|}cFzgq;%ZWYSj1mGalL#fn|I1){8yKu@959_e^98H_`d|*}zb|QGMA*qb2R3m++lHbmw1)8?Tc(qsoyUB}MgQjx{be}*|5S+l z!26&lPo6wJp3w>c|9f|!eHgB{re@C34UMhzrjZ6~5}>Sw_ThOU7Tc?MEU`ugckM6J zpD=ixMu0k3SjzOTuD=EQpp=oD8sjcJC1@*HNe8_qm9C;if;}(G=*l6`qfkm>4Nc$q zsHsh-wM{ezlf$xr+a1-ZjL3}$`i#oCwr}}JjzE{JDo2g{a;t;vDn-q&k5^K;3?YSX zTO*Ma|0HSJF+wE$Mn_G=qPXy9fJ+qSy?X7mH!03#Q(hvgN^VRxQtp~-1sd17ZDA9%I-X^sl78%q=4LnMnv`BuIWED znpgkian_FGm)ReFG1#Ot{NhyzDZt1pI~<2b?7mj`r}yDvGxft0t-o>mf40L?w&Q!4 z*Is-o7c)>1#w0OJ@E6Sdsp)Oe;7Ufu+sh0-6i}P&P{u!84loR^92F_3s9;BsCI_S` z%0E5qFHHq65zcu+9Uv!(Di{oY@o9YjfKwpk5Byt%n(sOW+Q$7`P5j3X5=VbP8q8() zH&p-sf!+Hhi3jc$dvTz_6-$Z8az(2gqSPR0nx9trs z{F6*=5H_5&bgx1EPs03zkQo}!{F7kh5%juH zAfEgWlFsu&NZM~Uwf-Lj1DrqzhNbPMROBBd-NuH9JT2`N|6(nF(%K&elSG1G>^s9M z8~-5bZNuQFQ~*3O?)TaTE@6vKSDm=QOV678Z))3bciEbk51GjX7JRv<{0@2Uq$qZq z7pj^5y=a7(Uw=gIXRB(Rz@``0He8C8xI$^aKULt$+F#NWYUFijb2EB)fc0BTJ-|Xw z|7PUypW4B341P)mkU`^hT+a~xIg02I5kM5yKgpX2*)1603gO&4MyNNM4 ze-ycYvg0~zAm~=z&};ZL>^|Ft+~YxDf6KRO#KTzU?M=7S`S_^HZX+qD?ZQ<@RfSWK z@BsY_i>|)DS0vQu37=XlOYJ^W*#wC{o`(i{UR#lS9=wKK!`NeA2udzDnS^RjEc+W( z-PT>)p07XSGCr-liMI}sE|x5#ERO-Va;_mj@|JXPzJe(3EY~UC<$I)>3E>XzS&Q` zeK&HIbaU5{TK%1`sFQa#fs-Vo+}u) zr-e$yU83WA13b4qD570Zh<*v$x)t=GTM4hqZ7&(0nR&EdI6wBY*)xfH=TTDNV6Cc| zG4SUQfpJdp)az7B{Ls|JUQl@n@v>&KBf$y}3_$xbj@iX#bPZSYx*j7~H~)BKckAyw z!lQWq@`P?{KP4f!Z9OrMcBUq3LL;xx55%d`vF z!KrUVt2zuI0q~dlPis*1%@+;3anqBv-F?72AZ{I=5WNaz>(#Ies&dbJ@CkgN1y)p5 zYU=rhgLhB`v}KXxX4@+sz?tQ@?UHZhY}&Y z6p7k9=%5?rcJO+sHs3}2XWyGXJlIr?x(+PlEaWpM_@yPHd@YZvew0|uHTf%&EF(== zi41dhoJiOBW>7h314t5kupsx4A&LrVbce)I>P#IW^^2maYYIIOQ|a(F#Cj)5Ayx5X zUmg=G@y);wMXiI^<3iun-sw~wAg1EjmkEB{;FzdQ#V`F`46aXq9}12`0V#izi(GKB zld5vS`7Kdv@v<9|8Ii_ziJJJ3{lP(W#~WoAYg(G42XP>`3NzSm@9&m-lmf_W)_ZVe+n`Q4Y8$c)6? zsvSWy=mUbD55Z3}`KOEfLIEi$k=mnVqN300$X;;LHZ=NAqUv`{ zxGQDl%fa_*XQ4ScO0AB2AA^1pFV4UssV{TPxKKexJ_q(6v&wp=+CGXMl8lNU+lT`5 z`h_BlVUVQ;4ywpXc1yBfd%Nommm$biN!gFgA7}+(GJ>1~F^(Q5124c%uZHXi0pYvU zqvKweP?OHqTYaMS0s}FIBf|54SKRkcD>Y1 z{1f`Y=1G3NQIng7FhkkZjn55Rae^dGx_NNPEQvOcpREDqsYv~oapeiUpJz5~@7{hh zJNHuh>Peg?O@mk(GYS$R79ianixZ^ki`A2Ve&;=Y>TA5Xmnh*1J=5p&TeyN|7xNi~ z!OkoF27}5r*r=muCgxK65a*`3{R-b29m##=v;33az5X;WPFK0)#7>ykU(ARMa^|l- zOuZ*s!yM>T@WP@)=0XT^mDl>k$un|EUsTYAAVX|Crc}Ygkk>wdLhP%t3z-^@IN}p-;mjsnPyiI(~u<4iBMHD!G-0 zPGGI_j?UEp^NHLNT*d%G)hGWF1c;tHGdub50$ZKsxkOIC474Ub7h5m1VDyE2tOes` zpAE!PhLK=}km!Bk?aXeqE}JedPqmKrLK-y=^oOgn`n>KH9$(P#(~wr69yjqC>1-)t z3+m(-eakT!q!VO7L8A-)yb^f2R&SymS!|*AI{lk{gS*lVHpD9+y@UG7$rmgyKWppy z)xdD>_O9AiomEWpfqd@8d+rL&#ARRJ>#p38Zo zm*SzpMUC^Rr+j_b!H6ov4K$b%J08siu~B+}atj-4f85#&nQbGmfcA1hiycr2AASs@Y3}Q^8E~ONSp!;`V&CSQ?NGRzpYcA5{*36lK0rl(lXKLk zUWweVUPZFq&ao0pEq3L1cwMg7=C9Cn?+Ck5nZN1JYPj$yVADtH_obS<{t~4}4yvy+ zT=AUIP|u&f0w*|-J@N-OoJJgI&NoX^)u_@6;o&Y#okGj~RM0?}P|rPyS~*!`9kA?x zvh~D*p|9_K#8UHa-+_^^4aj{z%}Z=HSxB@|wa9MEn_l_BQ>{X-dCg@OQ{q6-lFiX0^TF$7R(ORjRLq>fg#Aa8c!_yA{l?kWhA<5+}=`{Ot82vc8En z`@Yu}JAhNwd_bk$OO4L}b)iw;+agIQ(0Dj7%s1pub}Y7=!;Gu>&9!uZImQ?D- zgSBjSzwYxEUe2y*RJ0m;W0Xf7SO+^nehm(6<9SHG;q1RS;6FAF<_ z*me}2wA)RvrO43qRv1#%63mjWQS~wzAIpTvQr@%@w5Tg{sazYnc>G!mGwx}>ZJb5! zj~BmJou1a@7jji@+&xbexZJ*kGVch_&oB8dD9)4LBFl!g=nZq&8*fu=**^Rvc!Tt_ zL6p?AWQ0Q+zk|Aj^vk2KyAwwcv&cvEY&DCX3=p^D{N9D_Pelho7@Tq&tFpjnW3Zv1 z0R`l@xj%uIX(y=FOlP}sg|a8XRkv-VapAiXm(HpVdC0#ie#j?1nf6;DtQ^!Y>m+&B zVQSXH7z5a)2q49r)-rCn;&b;P$9>X_Z?I)0|B_1;9%NQQYrdm?;U176@bPDMJVYle z&z?SRH!j^SNi3j=1R)7XY;EQCLbYuU`=s*5<2+?TI<& zbcYO8uL(%tC@W0g-g{?K^VGdPe41w3Ry#s2P~w#n@O@3?8<>C23`gf~eZU1dwwGI* zGXp*E7dnh$t$9zEV0}lE$iohcG>5_IrG~}hO#-F56Erw)@Wg&Squ$&04UgEy1R;{F z2CB?m+eudhok@wvhRg~9TUFf*WL1K`4)1aFr5S$vWz%TwIwLqdtPgfuJJYPSg-m3s zl-t8@n=TwM87is3nPnF#^AN~zwWmSzeAs@ir+vXn&r~?tuO?5+gVzsVjo1R;`x)|E zd5D{O#J^=DuQM`&`Mm}zHZ8{{3v6uU?06b^e+XyIF)HKpQ?XN98T3@@&s;zm|LK{V z)bS2qnWlgz*+TCKY2Uz3mZC)0bX}4S!L=9bN1ja9WsM4PrH#Oji}{rmRQ3h1kE)3yz8PW2%0|Cn4d~( zY8`$^X!h=n2@@pG_2k`Oty&W^yjl2syGK}evFTU7fj?e44H4elbw)>{Z04V+3pRgU zUn1Pz;~7r_K>J*7`<&XWZWS#fTX$(eD}|~P>&zinG=FvD@Wm4U+y(i^m>@)?mPP&} zQvZBXDW|8e_7Qp$<1xT;IdGpu#pb5xWnEhJ<<77!2&zNnFb!t1oU)Z!T+_)P(bTY2 zaY>+>ov3&1l-W>B|5p6j@pk(~wzlLlG4t~Pm8)*lQ2GW$VHR|^yihKFz4;0;*m9Ef zx8FJnJU=tu&J8py5j@-U!*0a1Sq=A{I3cYdmDJ7fa;9(_F=Q8jY=!JS0A0R5Eb*N9 z5;h|-?pAJ~gSrQ@#pELJAypoaB_LPTGl5pR7}vEU?2XN#nQqB;HNRJA`Dr>7ol+oF-LU;%qkd8KtxZL< z5gvDAgN)@Hzx+EE{fMfT-nAKwW(@OixADoq-BzygHaLRD_k8MnWMp2w&n%R?=)M0? zemFdHW}kT>lKEoc!=AWco@@$9{W)i~IIU;0$gA1X$_D@J&6L>L2K1th_!m(rJ7Wnp zLaOWVw^*#!DlHcKO2ho|ei|yJs`#IIGff-jeBGqoy(ji@x$G`@#rr%+0~}%W2W{vu zJ2;0CZS&TF0N<`Yy~mo@fqn-?ql&m#5yUuWI;0u9t^T{s-iieGO!YoCiRo6D)c$vG z^a52Cp-22E(ScE#g4)`=l0^oGSLyi!vitZAs-hFjl^Q#n(ld^d^iI9&cB?0~&PQD* zm*|7P&gCnwIOz^aYLQnP)HnvN-+K)i(llC)v9&Pmirsw<=O+r;{7W6Sjs3Us(xm|{ zS=Mn*A;^$|Z$A4+O@i8n@Bp9b0;Q6o_*(3yHCdP?7gVs-`I&ocScvBm?Vym>5r$Vq z!_KxAhg$b~te)uVr%kZkh(6ZY!w6hHkHkl*8%r7hIjP zn<aj?7uB5@YCd~abA7s||NS-)-uip5W6868?`|hZwwyj@9 zii(I>s8SRZr78$02%$)qqV%rx7CHnHfrE%B5PA`iCcR6IbQO?Z0|Wx0NGJ3XYLd6T z_Z-FZo%`-zZ@hoLu?J%$o9w;HoNKl<=Wn?%GvA<3xi`gKcuJAjC8Tfaw|$N>dwy8* zj0h!aX(vWs(-eHy?aFOmH~j>WXCK_X4G@hBTN-KAdbf6$E*oT%X>}AZ(~|ZIZ`vmX z%m#{S?6OdDjE<}=Hx<{APb3>s?(;h!{veho9gbPdO9+ZrPQG`%@Ca6RUSE8|b;@kP z!)^K;*80v@^w4>D@$M-_(2MOJuF-gXaqy^*V|7vrZmYo|t!`(q>`eYe9}s4{4$=QIm8wc7I3Ps|IuxnZ z0+(!QwiSZM?oN|M^2ZR?5=6a&>YZhrI9%)&j#flcP}tTlO1vhOUkyH~7=)eXFvFM3 zQ*k5amY^i(gXt7rL*+?*-&J3LtyEUD=YR@k({hyz9aAePTda*p*rG-;h=@`14!PbgB*8a1qABa`HKo#*da=#p_{wQMUb zMuuatcHX89wa(yhcD+WZ*-9;61N`B-RpFnWLi+DpXv~jvILoUimAY!|R?)n?>!9bMmmZlI+QOemmMvy)SxP74J%%j1 zX*2azHX%YxkKOeBoGqRv*-VWhvV!2|owhJ|5YfW8W6If~jK|UrRik!w?=#SlL+1me ziw~Lms2}aiz9xI}{1sqTR+fIe@k0EY?kLygZ=@_OD~v%MD@TUr3@=goMBGdNAvKTn zjSDanY$aWp_sIUhW@Aj=!)K@NsgjlU)X;mup;Br2&FT-m7f@4 zn_1uI!=>l;VPugGv|Xf!Moz1}RDxR$!Znyg;Tj`rNd1eAEAx||ga5f!Dn^XL8o9Xo8Rf>=jU*WplM1nCeg3KeHmKnw9>HiYtC;G82?pBz34k$ggNiPg*` z835J;T&jJ#d)b>(R}iABRjRxW+TYUeeyyV|jD;DGF7!zX^f?P%ULDlG;G4>3s_A)) zqZ}(*DNcKnP;>_Uq_?Ox;l}+?I2B3Tn;zS>n(wAxr)7gJfFQ(<slfRHQsQnOJ$AXi4 zG1A1p`%#I`I@xB+;_7&}hfxPfP0_5)+Ch@ADpL77e*LcCcGl#Up&DX?VzYcMHK6#* z(Eu(P?J_Z>LKWfeRzYV>hNK*h)!0YoYsEiFg-fk5VsGd?lSwEfAzVckLg`7d9TSxk z(4hdkmEDDnmpzMl+wHeyaSD{Ew$&y|4kH61WzC$F+8ALJnYl3vU6+?_lH%MH^q1`K zYG8r9GPgbrxic%RoO4ywQ;tP5ov;-?M^V{WK=IAew|e+lO3-rQLf5p(-0o1#yHu-Q z-t1s&nFyw^!@utvZI}Kb<~8cd&LI;3&O>psvY?z3l~$YU2rrb!t#ilbD*ZB&YMhaM zvo&n3l%hQ0_E%iI;$jqQ4M{(rWN{?9x+`@1IuA*AC*s3 zab;ON*6dc{yA0Q;WQaGO!rDpnIdj8ilj&pxLq@D~*gXmn9@2_ECE2IiwCACwN#>#F zPSi;X%-!=QJcxg0S>Tx-_u=asw(xTSyez6s57O8k;(~?v*8X>v)HTP6|O>wJsV&+i{rO-y!-8^4#ZqRuMPk5)1wzWT9{|9tL|F?z2 z`5@l#f{G`G6kpk&7>(4qR&a{ae8W9sTO4Tyk20kBT@!9zejOnCaa_f3L;#-S%g28& z53mv$7)R!%BR~0NK#ZDWB?Xcv%$bZ5t-k2aRLSTNC_TFGIPPy`kVj78?7_@gK=QP8;KIv6=jpo{zUoBotFVo6~Z;oY!*B( zL6tjA9Eb~WAgc{FaF@-kGkA>8M0wPI- zjWW^x6TuG2TTrG9g;MBYc|B57EkygDiSzZ23h%%2Ha>F6_VT@){1GdUez;jeP@>#9 z{a`-ECPXLs>EZGz8oHf|RbgZ6#Vr4KuAaHff0&B6BkHo=`!tcX&E;GhTg3>qbQGdA zGS9P$7pT8xO1^&YbjM2;a-s2@ZY0{HiSWmpy~ffTO`};zhrCf?aY6m|!HH9Csk_gP zXd-#V7ty%r;J!rmfy$S5V&x7xe$M8O-*Q0jE^VKyp5(h|&y*xQ&|m*4@NZkzl8$G$ zfrK-^fp4>-qRxdw0cxL6O92e7S~syA#JNY}rt{dWVXvf@z4tPt=G{OGPDp&!Z8DR} zzMwDrxlrQ!IF+PNc5<2S<_pBj8#4kda4SYv+Zq069U+yyf?B!YJs8;H5oqXKvVCp|T|Z z@(#-@H@w=1Cf_@Ip4AsytwmH>ETPexbmc_?GHQWqmwd(sy)w%E8269CXgQD-(#IS3JolJK&v~FrN!Cy6%SZy z`(8cK#03V4v{cnquF%qzRwbVk#!KdisKIhwKbn2Rs?1Qhg0sU;L+vV1Ot(*S7|4mp z#YlXYT1bevl+vF@kyuxSXk;G+LS%d|&Qbi47GA31XYNKF*vemU*Xro`#Fxt-X~=d~ zg#)1dA^hdWY_0-Fk^%_#6O1$pEaj4=5;_6-S+@!^nSyrgq6BWL_I+%hr8SlnHTlS2 z^ziU|sgGLKzP`1%*0+$H!KXFrV5iTv_~zaz$x^Mb^498RN{+*p=t8>XFu#X~W}nx( zvgtJ{<#3_aqrzqskyD`FZ|KvCVv{Rxt#u0FNin_`wqes%z-jb5=WNeRT{pF|GL}tNIWxE|0kW+dNMQyp7w>^-6-M(L!tX9{GdQc3>~cTKgu0=Za|#4BEw| z--SdyMKP>}u>k2(VEA0E_R^CPEy{6NS?-?KBGig%^T4W!2&nJ~?Uh50WcnxT3&}Is z%)5PZOK-Oy8DgI@1;gusg5%J!dX)q&kwbS^!3O3Mnuk7Y3|<#S58gj^8s`+Se*SxzsLLHgCH0JScNG#|4J84>2;` zg{{S2YB$=CUQKzQhNK7vAj1d?fia`w;X?flIMtM6!;kB;X z8ZVV4GwnuR!xK#1LAuM-f4qGkz;9gF=)Tql*j+vd=!^Uvd- ziMKO8r!C!nCOb~Byr8kL<(tpzx>Jof92G({j3Uy(8 zkRtSJ*0;xm(lsi-QFF8yGgl86dpp~YI@0x= zx0H0BMHow~_0;LWddTQ9smP2trRh$dA1oo{Omuw3N5pFwKNEMniQeG=>xcAWS6_b2 z*Vf5PZGu*6RfE1>Cuw`;W?{}>-5XtKV|+Tn=9C1!_uj(u3)`oI#4Yj@$2~$p7IuIu zA@10a`(%PKm#HDqlZjj_c=r$)rk1-JXI1fRReFsVjD~u)55*c^AXF!~9SsiZ*U##H zvP$#)81nEBx!nv(!X5CLQ5a|VIp|FJ3d19&Ak=x!zh^uF**NPj%UL1LU6qE3*u9Yg zGyaF)zsAS92I*CCS3F<2?ubS*GCg+#l9tvfR83-wk^HeXP$!4(?rST$D=#Z!XS3-X zG^XCa9dDb=N;q3+TrbDVZ?N~NOKy%-Fi6i@-Mqs<_+qt%&4>n*W_XlMmE@s?frEwJ z{tQnc&q%6l6nMnLA=yl%zrwnucb|S=%FOBFSdt0s`C=Uxx1aaKD-z8b2I z&#tvoI7l})jHuK5j2jlZiP^R`4tM*UmC8${CWTEwZd*MHPPoE%&wc(JWx<>|Ij1C^ zVdO}!Y>YmElFRpbQoxH?G@2Jfb>U;lNB*16?b;BD`-YbD6=SuA2~(JxHnZ(C=+fC# z@zM_Mm1;Rb|B*@Q9V%2|?vUk(LE-n<*BN6LH@JA-EyG!CT~_ShR+Ocdn9tfm51upE zTF=ifGX_n8X%A+G8ftqpT1XTPU|%blz0TivOD-Mfa}{!}QneZpVip+|Vm5eiE-|S% z7}im%-W(&2!A(6K4a8V*`hPiHwv}z?7+<#SSjCO7S-I{D(h~Do-O=en1-v8hDkHPA z7bm|=x|pYBPaka_L8llaeOlf;d+{#G0HaYb+sx|@)V)i$2=H4QB3jXJgh-eeVAlS% zk^)d>AF?HxxR3glakshO(HDSJXQkC@L~%3Gh2yr<)fDr?iJBYRE`=LkpCg()c@1LI z=u8TzD4LB&!yq9QTLTz_@!9%f+qO;Ksc9$eRkuu5iJagP^5lSm>%Ly#bReGI)S`tb zMzMlY<$3N?>6f{Vt}DIGB1?L3ozi+21DJGcjFcbQ(Ss3jj(s@Nmb)T#g%Ts_vdY2Y zg8aL!ALCD6e4%5QwC>rL_p$EiBT8Pu=V&VMU~k$MvpZi_qZN}&6Ws zZ)Th!bK$(%HH5bLa+%uW-F;V0gZA^I0zP+y>&ID5Ssp}Jr`K71-?_=iotVrf<;Y7E zOb_IfCr^-KTg*}fi|kM@*$A~)*pTYKZVP;Q^RUr;Ps`gg=Dg8BOC;WqEWF1KVV)Q7ko^83J_%`-hxoT9I+~W6m>IyTJlChMVycXw3 z6k+`mjs{b1nS(4^8wtFX(+BDM0`~M(DE6<|c;Ut(JH7n=Su9u_3G8KH2WC{(7_Rbw z#r*B^Bzgv?ARmFfsDq7Z&<~7;&a9-xB*a*(+xF=o4Y!~)C~_mm-WJl>>T0N zTXZ(s&`sQ@hYhWc8a=E0_eA?%Js?k~vZeUi;@Q4|3`!47MR|F?yL+PBH{&uovPFcpj!?qw*;{eqj6-(9|PV)z%dn5L?m$#Okda^bN2FktgcyT^Td z>H^Hn?G2N!;{5DQlbSikhDWOj8A`pl4?{B*F?--aM=P5tcX{5#Nk5(4F}HC?)mv{u2Zvr>fT3Wt_M={khYB^#<7)V6$rm7^JK}-qTFXZJ;{FbRDzoN# zG>@N-VgBf*!Wnud0{hSb6=6vD{%*lfB*O8H^Tc3D%c-_WSLv0eQBK?Q3!yNRvd_-) zp4GMpqoXcH-ho5vf+HHC|s?ZlM|ILkSXbX4+Qgm2w+C?ZPtP^VUH zq20Lvvjf@nq$&QGC3}q){r|NOz2rpOK zcTK5TWcPSvq_7_)jx8@AmIXWH9117tifU0TGe%~*3dmgU^UH|QZ=u#L4*T%NnAKA2 zX{r9|S`e;dr5N{eN0ZyQ;pPX<1ej0l&aF{v@pb`yiO~S-MoT+fbKO};^~XCU8E%C) zYgUCl^2O_0pmkRe+k8&Xrz@pr&5g}QM!1-+dEFfoYpiR?`U)K>v{|$ex$T=NcNJr* z0nwebMsa%fmM-S7kQYJT?VRSSOopwB1HsKQk!b6kdnmJ8s$u1mg*Y)Erszru;(LO* zTquySmjh{f$^7XM+he-}q2Su?4GO&UB6R2Bi1n5o>$z(4eQwnr#$pE7SnH|xZM1W* z9agq8wobLRCcC5^P;X_MuU7^Yn#BZsimFQvCPWuTtZ)ryUoSqT=lZ#Fkn(|WdEIPS z-y?B66Ky&c^Kz_Z6r46OIl}B~`K~C!z zQ4D^#n{RD@M~`h}q`#t;DBb0(#lox87h5-_4D9bE>?P-$6_Z8Mu&$(Zz^|rW-1vT? z7Fz-5Ad5T|&b=tR8YQ{8&F_=e4w)GIS2~~S%`tc^x_%RcTk2lJ`-~(B|ztA@@!t&rPNn469 zRC*1_NlWJ_W?2d>_%dIyTa#O&=-2>NpEw-f-3F3McdZQ^dur-Rcbc%wi-1Y9bjD`r ze}uqa{E$Oph|tid*lw$HuPikBO9+=%h7Ako8j%jw3`ix-N%u`h-~7xBy*HT?Lm2Yw z1+J%P>|f1SdTYrO6JE=MUyIk+>o2P$=e)zOR@A#-Ay-&a$(Eud`$w)4MjQI98jdnW-0kFL z+So#_;tL&b?^5{A(U`aoYB36!zZu)%bcXGtAZ0wN~v0^DFZs48@DHP6-Da2jObKwu4oeq=5RavCiRV5%DbIljKD_1PXdT-1{mG)nkzF|qIfW22RYCN zk_E+Le2ZJ}^cPMCaToQ*=ose~*B>aE`0QuGxMUrC=r!z-A*!X6PfC{RtIr*L z{Q=v}{(E}Z@$7u^BFPl;$j~;G3Ftz0;LeU+@nF{kLpD?WUFr0qhVe@UhuI4=xo%v200NQ zNP(PTN|EE(qzPsN*u}{w1iJ$1%o^3bPEr#HyMk&PeRjw49R>Io=Y}${wbMX?{@*iN zepYH_RDVXwg8E+>to4RZH3ph^Tl+I`QIIhWP*_?|wmiyqKjy_0m;t61qSR2x{y&sRU|uh`+2AWAnp zFjy;`W@49LIAAK;MMJS{{Vnjtr&}+7LrZ zx#FFf13fwtRW^D*tWQmf`It4fO?|DZXPFwDGjH3NaJ)X`_iBQL_n-eb&mpFxH4gt< zEbo7O^#4)h{w;idn**p9k!DKF`@4zqlS=<4UoQ$Mi}2>{p8tnca{QIy z(m%>4%1d&&{!7E&ydNz(Xe}%(Y+Bc+8u?AC-4y4$ldxEUb+6;^`&&2u_4SSd8U3{f z3L-p%|4^}geD+d7K)E3hyJfAS}>pQ=sffch@Y;r0QV-^Tq* z1pfKSkK+sg+i)g1?7xhd?FuPS(Ds~LL;k-EB=U#g;qa88{V(nQ?*#q_LjUgs{S7pSA?)7qTJjsYw_fo47xjPooUq?uzK-4YLLZ=c!HrK@ z|1K{g`PH)#K{`1CWePof^l?Yu)k%*3L?P$|J4e*l?!dnrE_e3GRO|0)jBuaVleJpJ zZ#i0j?fqh6W&0a^7?{l1bBYYkEEY=>3AXtMY^ZAXUZ>?7@LJaZL{RMqmlgib#qK5D z--H!;&I!Mn6BwteQ1p@!UgXsZegMp;eX&GOIR8w88ge64i$g&tvw~ z=Sctt;_A`pJH}&E*xN(KsI}959^<=vR?9$I(&tk8kim0K(%w|Ugd4t?N8 z%c2*DdDj?{(8P}dbrh>9ax2MRg&vnR^W8cn;PtMB$zHn!EXkLd4PH-lGan}#ee@4Y z%cbn;?PGVz!zImP9O`hwS_a`y>b9Q4*|X^-nZiC?pCnz{euTCLpEaAo;>$*bu9fra z)Vbw5jOygf;mfxB3sJVRJJX_rN9K>L8H~flj!z+AC7$M>SbUwSf9&qImESix8|x;r z>)b)edn-uPU@e}dLpZP84C{(E+NpB*2uJNiOKFQ%INJe zhe3GzKw}uF_~rX2w)b)G<;M5lu|TslWvF@kvSdb}H3w_%s84BWs63umV?~`&H8HVu zx@8{PSUB0^_J<3Jlvj=^Z8kvvDQXh z<_0DhyfmZ-J=(yQ+eCq=W=6BSv3$~PlED+FIb)wCqqT%h|oHTc{JxQnF#s&OIu#)bq%Nu&5w024z}4xKp!t zu8ZCtjif%N<5P=)Sv=H#-*XI`fvuU0E68!Myn^6h8DXFu&go997~x8^IGks4 zaN817V)90mrudq$K6FYShi=0X94W7?jODgB2gg`_nQe4n(HSjvWZus?n@??b()Avv zS=46yane=+7F0^qmfx{^8KshD)|X=hTah(ZyLL~lUuhaOr!$JLXw>=oG}wRC+M6@- zUCj)1C-{@dK4kVu0vM<_k%S6*8=Rp%EdP+WswaM7`D0qx!zx%-X)xWZ-5#?ZOr98P z{?Mo}<0c;ymDh5nSG8*Zle`VAmMrFw_1y*i$lq|F!j%Ai&n>UWxiRZVTR%?yu|D&GbTzXL*--EFCdX)~b?nPH<`Cjw<@nQw)Z_iXU5F7ZgqcR^H{| zR?TBpo*#kT>8T*Hv|{x|*|+5~J4HS|WVc3_e)vd^Trl73Jk={T+(Hi>{c7Wx)#}s7 zmTQFa2IRD_+$qf?+RH0N+LByR8`kUq>L{vLig$+zQ2SjuOqFNU-Okd8ZpT;hqGE9g z2$sR&eS@7^mA->NMk7{+hM&RqoW9+M(I5H$a#$tL0#FHAlXBVZ#|XC_E2%Q52v4}f zXKOoHogfNtFRx;%OQhsrMMjO67f?Z0ha#W_qCGbqfs@sGYKynLQkKxm&3(_WeLg zr4Y)N-_Q_J8vv*R+%E5<{bLH1{sal!T?^{a4A!@c3*s)6D1PZoP{_%Pvs|jY5~!b? z50ke^H_>+AVq!$8XUfZuXB8ItjPxuFOn?d=BXk=T>)dnnRjTzxEa$Di-v7{gvX}JM z_Y<`D_q)#*dYitrwJ)DO)kC}J_ul!govX_(zP<8ESO0XL#9_w8l?1rEkS%?3?vryt zAvSGM$EGVI0pR^Su=7&zv3XTHdv9p5FizW=U>5sG|6R!ze!k+F^c}|jzfTMiCko>`|W|NcT7>Ax~(IOLbK{LF9&AvPZq$)NICMCK|o8@(5%! zqD=pgu8Af7imTB>0A&bsDW?SQylfhSu3({*MnUaA+KHc6b1H0kofmg+v+_!U#UYky zxfru$Wpk|2rwV}c6d)|DbzHz@liAvzZ7B(M9MEIpoxQ1Dn4hOUo1HRRhm97$AwP}o zL=C3~A@c4`B`;vR2Q{^97G9kR`Lwi1)pT${fvwk?nY5^Dd_^tIe$DQ_US`k|7rxRt z$Kkzza+GOkr2W9E7s_^{{$WfCrueF*rYc*f@-e!xT>t_UxYBY&monomAS0!j>V%p9BXHc)cRrdeFV7BCG#0+0M5ma zh3-uS@j&AiC>zq_n~L@5(ZF34*>UOMb(t~mDD&v4peUgdBy#fBFR9S+HUrp4l6$Z`@fK&3=pDU8G;$K=>N~*RSa}pY2 zMt<*u*rp_x^hHr0rgL&9BNuGKeyEsPr;^swCac#0!FmQa(Wxec6umsQ_c)?bqEfpj z&Nqu)gJA06NCp+^pCuDfXIBo%qm2{$G zKCN%~dmfz;V5lu$WTHG476UK`IU${zI9EzHg8$3d5p67&8+O4aFS$Zvexemw5+s-q zEtodz-eQKzR_++5InSXXAChpr*OwGM>f1xB$X+PpSL^L1)gfVj=$WuGoe3lbu(_aT z2^vC5FMqA?6C|eqc6K=d<_QY=byre=RCTr9WF#2mc!_7Ib|>ds_e9?O1gMz8{?_{=%*9|3+Vkl14yCm(F-iTd8brcFA*{iukaLZ3 zzC`Pxhz63SMryEYbQa`)^Ozz7)a9Q*T^z^m=3Z0ZjfF=i6kkt_thi&Uq@Am(mH{8i z`T!X``oj&|Dy9WSexenR%J%hKzFXsyaoNyHKqxim1_@q0mHp6h3ht3K96@k5jRD8x z;JBXJBJa_CvPq~qHpPGy`zJ1D@G%4g!18C@Hbg6kXI*Dn(WMtcmn>BoxR08ViH;t9 zd_APO0PNRl?R%v=baMChza(RnkG;E>NFlU5Pr^Lviz$L`>IUS2#M8f}i7eo1YleLf z_hU0XkFC~o9Do+}E3dfqn=Zd-odZ>5cQi<=ou5DvC~m$|eqx4{3N<=1&`R-@F|_3C z{K`yv5OdpPc@-Y-zzi0s3^Ko03YUZ&R%|fkY^Cg>vN=2?xZ08jyD&Fp262y60 zHEE(jay*k=tU*5(K$VfW@Qk=}%3R_njNA`P@;e}BzhzF59beUNXsvrgz1S<7jIIiZ zsf`Q<)^c|_C$(DNV7b-{A_E3F?~$f?Y>*kR0U=;(Z)JN7%S|%u%=?1jLYct4jJubj z5r1GG;plm|rBkU%f#*-ehy7;73Ij_Ntl;@u$Brq`pTC7F0fjICD#Xj%M58}wIe zJ$p|?B-^q%SLo-z0TfKYuNsl>eR1yCy?^4I+*?5WF~}fE^w*poIa~k0I91YUxlvk!`-9rA75b#{ZsWr-jNpCq0n~p4oZdS#i~7sz*t` zX66^G9s5MzN}@=`VYL3qye-DX&-}ya`c(hHj)_fk5dWn*IO^Ch7fK}o)K2#|iJv!( zE`JWgH4+5xgt1cLShtpzY*pd~GJ|0a1!m5+2qIr&3EQz5xea76NcIPW^M$yj&A6i! z^R*Qs9a1=VhgI>Nz?D%4?fVTlwKyof`l@UhIHY{V_n zpRbc_Arp2O%V}>{O3Y2aUK>uSB_JqRVS~K62H6Xshv>3^(A0-Lk`H}8U2wq;6>@QN zml0v8S78oUiK0G7)alL?KiL8hVxk=3(USct9nev*ju%;H+lEZbqifRDY+=x{eQ}G`M|F>J8Lk(b&b;sLaCvfl zhxmJRGUeVl%t#spOkl&1)RL(TW2jq+cC?l}Ph8=Lq*1L;pVzL8db$J_bs2wekg&_{ zZYcR|s_gKKbMR81;d6SW=5IyvIE}qtkFw?fLqW<=ix154@`SOj1matmDZXP4wl*5c za<=As5Zzy>Hi8_P9Uc&x#Rx63)4xe&g&TG}#c6(L#rn3h&TW0K+}+-iGRA5`8sv0q;q zNYb=ky)+IKqSn*}j)cWw{W*E>!fOF%e6H0-)V!yQ3!NM2@=Z4_uiPCtBpSn6hDA0q*o{KJk+9W#|Ons1rbRgWp z)P)`54g;%)#e?4k7WW|_?2Js45_Fs*qYHh3s>?cUYRLNA+y32SzaT|w#o zGcG@^-^kmOa-4}>cy{+t3eXVKm; zR*jXtIUoIrgQY!0zrz9Z(Wbd|kK68YJl+K_0YZ-_0_{p<+57am$?C$eD-jjM6IWTMo+Ax8jq93|1MFXoC zf)QVn1IRtKH?>AtLMSZZ^!?s&VQX*k>m#~{0Qgr4ope&fA%Mml)V}p=;Q%(`T|Wsb zj-K}n9|{WcYx^REgqXRXVIgGc9NCYn3K|2`i*Jm?OZLN)x3gTm7r;UwSlsSixEB8+ zdQ^KM9@2xIQcSV6M1w`68#KB`>Gq0K#=xim!-D?Zo_scXl9B2^o`jv@NWcFpxXGdS zYwqdJsoVb!8vZjP_4J(H@qA119-!#~}ZmoUL z?$4AIC?3et*eT}U?@qU=hGAs@+gfk!qVyHD3>$RT!#bEp!_5)jSVn)zQZ$L8@98XY zM5A>3jVH6Ws2Wt+7|0rbx?^l_NgS#NBiQ8+Zdg0I8K`$N)@)rOe)$Gxs&K%F5>Iw; zvM|qcD++nca}rJ+S%a2AS3$MuQm}M(yv%D5*uy&7d|-q9Ob2MIQxLj073d$i@yi4N zWNbBCZ?>Pg6TX!d-^zDoW125oxV*JXWYm3iMS8cen0;hKvfs$0v}D^mvD^J+CR{7Q zs}TOCyvH*qM?qu(aS&LMI8$_UzZaXmy(}Jo_mra0&_a^8mThf!b)b)N?!L0{TL2Gu zH1@_{n$Bxr#{-1tw`bES(Dp+Ex&gl z``zTH5e5urA>&8dPE3r>Eso;F68q`4!h@|Yd{ZWje;@J@piMhJH&Tf}kjE{dp_lrE{ux^w#;%UuFIZY&G zt=(f6Dn*l5zCVdZxaNc{MC4VAZp)9F=*G)cXcQ4Gi&6V)UPSz^JMdTb(8tF=75|aw zbfoa&go@kIQ7>)XgN^Ui&Bnx5U$w+($BVHkcb-d&kiV0+|W8$F1xn0 znQ-L_ad#Y17^j2EbRGGa#QDodxh9}I#GnxmmQv#>u zQDG?}qZ>PG%Pn{cv7r^vXcQpm-*_FVeBCSwYIFVToj<|~z0Q6T)Exdc21YcXa$~qi z4N<{u9XHLY?7JV=4{3nuF&Q676OSC{-FLKfUdyyOv5g$PKFTTa=U)9j%DA*Nun@~E zYMB1IMIx#KW}HAPvO2Jjr~ICbUHaZqM*o-c`9qS9bgnBg+uI|_X>E@1>d+7W3B=+L znzvmhOrX`0@WU(B*dpX2GJ6FGgv7i4SYwTZY38c{z5oSI5I`37hUAx|__-W81(xUH zF75knGQ!4mW+5pJBR;DotdI!|*RoX%{>2PSx$BX8hka#=gyDG1@2J;qDycmHw{}u7 z6{$bzi!r9JvI>iiAMu&;&(+6>5nIIQ+;&sWX*K6Uy$|+fnXbjp1w<0J=SY|qKdH&~ z#Myq$c!&{tu6fA4S_rxKeqdE525Pl&HfEMa0Y@u{{TXu<97=Xf zYwJ}A&2}$*b|W3&zjHIw@(vCT9RtWOeqCgF@Gqx`A}tM-Ekx^UzojTjT9HOxRH!By zhhnvGDXG_nkcKYotU5&@M)qPR8AN+z#V5&r45481JuSX?-hB<+ke#J$Vnb|6$tReW z3CJH&SooGG3$luHXtDOcN*YAhe|FbW=`Wn)#fW;;%gtKkuYPksIJ8Y!xL`@||X^Z^W(tIzeq5>CrTAG)q zv})D)?yfLQK5MLP@64r}<-tvxdsPuG<*c|AKj_gdZf*@(*#Jnv^d72e+PB8!LhWzq zT^~pzfyDYuCsgziKXLY~H*O?+GFP}^t^SakKg2ElHi$641zjT^(gEMnGHM0~h&AW& zNA?3d9HRxVIFGq&KVu)q@1^#TMY1T63rVWtTQs27bk4Joha{%G@)$x_A9dS^XcE;xB&ZsF)W%T5{gDX+ z-(`g|)5N-JAnFgk6c29K@VmFL`WQ{%>+Bkfe|I_mX}Aa4&pwk!&J%VZ!b+Q{>H|BS zrTVr8)=X^;;+p98R$28y9=FhL?2~HC1+6E=TfdL#9|(OQ_v3w`^TeY--=PL!qyEmd z@l?n2tc*827UoTIvIrAR1BFj`9{$6l$GexRMn>+&#$d<~c*sjxR`6PPtEHj#_G_X5 z_k0VfDp+3bV3c_1()CNJAAkB-FM%Z1a|@E2am3BFgG9CNB`oo-$`+^P5pe}Wt!{{V zzP7>YTX8@!TuS4?ld8g=t5|!>=eq4h$}u{;BGaoV#=eYG91OY1b4n9gA z%8s*VV%NIqSI1qu>mZ=r)%d%hBv-i?a|&4wm+tac9veiUK-meKGlgUh6_C{EiKgv2=JB({laui zy!_+5p60y+Ks;gD2UDtYP~yh;jmVnqqU|bniA*8)e69J-7zbXx`)d2abfilIgUzGQ zGg{a&aU=Ow$uJGp&Kj0myN>T2)dnsZJl)10jNpn&OC`kKZO3i>GTxs-Pjhf`s_jYF z)YQnrj&2^D*f?6prxD+c<69mz9F`yVSWc#VE~n=39qP-~SYhF9yYqayhP?6HB-4xi z6&A7R+Hqul;A6I`VbUn=(^a0%k@!P?5Jc7>#Z&oq?v%WQ1j0Aj10+7da#znl%v)Xw zs&A|5&Ziu9rTDa^>CXoQ>U4d7uxYZl}U(8U>B7flc|zH2XD#GSb0r>=odDQ zHt@kSl#hsl<;0b>wXj!e&d$yr#7j@uxsJ=N&!_!tSQU(+`34hW-cq+iIlQm$j?=LA zHlznqT7o@ zf*pbtvN21dS~pwXrMx@)mpvo73~aZZ4^hv1v!0xAK>CvOC1H))@0Oo6hj^{YOVpL+U&V8?}jQ2cr~Y&oqeV#%EaAm_u>hib99Fe(Ow+-O{_XgwT5yuW}}Z8t<>L zzuUafj4Kg$Ktw+e1+{CDdens3ZDnE5I;hD}E#1N*v?)p3i3?1B_WsT#(j&jK?3dYV zN3W1Zl9rN|5(xg19#D!>(nuN{QS%u7{p8IvO^?|2@u+e2mbD`CEHf%Avw+AODep6p zp!eyHSJxMMlqb&Xh@0!_Sk=;6jhV%{Lv}-Q%v9c1b0HoTrACUw3!yxq(d7sERBfuv z0tTvEDGi}zwfZeMT58f@2QBce-fuF5r@3yv>B-*rCeB~z8M@r2_kD2aDZ(RB6qZo4 zt*hGFgna*ea@MA=C_6tR;At1m+XVnGx7hnDyep{inf2BM1vyV7_i2tZz=g+C_;L7| zzY@PS_?OxKLuGtE4UEO>;6=7i|6JeGvQYE!TUZ71i4_Pz>O(iAX_11He^Isj;gNg4hWCO( ztaftm_;;S9haVNOl>@mS`!ms{nNVk$(xJpAmzF8ZnAz++o86l+%MR-M5$9Ko64T0r zm>#p)$R^m8H$d~xQf|fEm1AKzdSDwAsr6lmode5ojbZugU%LZM^?!iW*}_BCjdd<4 zTm-IzLjXgWr0q0JJ?}X4|3UNSEnuy=8@cm3U>=C*$x`ph@x898X=_vTC^rzI>s$SYXZ($h0ZE#l{~e=?n^(Z1W0z65?ErBq~2(sCe>+ZqyHzzr%`t)r$c z-+JfHjI6lP`07nD0j#)I6e7MLA~4} z4U0{(NISRO!@d$!XnEL$BY#N&U4QVOQu>`92|?0NQsC>Mu@svka4oKZey|$*vQ+MG z$5hC%S(3DgqGYSY>huqj-7GK<#Fo2~I%y&~A-t&emFA^|am$yFh#d%rndZBv+HTmo z&svvUQHWhbP1W8rYH(mSi+!rmRa}0x&?o}@wS(jVBqrX;<+?@@K<~A)4S#a;v$->S zr^o=>uX&RJjLzYg(LiJi2ZC@PDKbTWIzoW{qW61uhC?kuh{K>$KeB^1a|u5}sCp_h zF+JOHY1Ty+@p>lj$|7`72yk&H6k3Fn?r2mNt(&|){1j_dq+Kb_)h}XUH##JjgpLO} z`AKP1u0}k4%%;Sr6%X{mRKYzrLayJh9^5&B4ljOwCO3Kv4rev_N)_GlQi~yvJBa>jD5}i zy;U^z^!fDf?{{4v??2}4x@P7*_kExHobx)bbDu+jShPh;7oEF0v1oO_tSV3v70014 zv|Lq+Uk8mXj~I%+nk(K~TzcE3;|ZkYEd6X{aH5r#C9NpYzF4tI&T)#rEc(ZB~ z!tY!bj}(~Zp#Qiwy!Nm#akV=wM7cY3BR3Lcf=UZWqB~!oy4~( zs22*I&y!b%#U}}SeB7tg^zthaxb z<>6k!z$ez*Pq|esYJr!dQS2HBfe0-beE6Q!C7{LDBcs?Z?ZG*DX_bnMnrwb+aN|PL z%yrqqv=(3gSKMwtz-w6|K{`da&uc^e&8a(DOiJsrGiAefUdk&weE9ghymP=*Q!nSZ z1y17`7d$6YplzqFVG4^b?@TQ%trBQ6ar-1{j9AlmjCAFsnR01X_Ftv{F^c?e4;ab7 zOQnrWa;2}>8eW{7nAVfov)(M>)Q_q-M`(#gpr z&ko~?QXIT-!$)#sMp_W1KlreOTIVY^GEE_SYs6+SD->^hyuEnhL z3m2M*iERx_zg=eZUT;dnPA_xT<_Cdzof-etSN%bRHe~w*f-D`d+FU#=kj3L8oG<+OEkpn2%@h^> zok?hlLmA(C6z#u<@^!AiOp$3i*;{kAwY&Kho%{p04UcYDMI%7CbT1-_}*BI9G4Ay(|dE%nLFUN^p|AJ1PA z{EDxuh3q{=6^tCI$;KmcYqBK9&SZ=uJste)K1Rvl$2-#8nqQ?lEZ^pQ#;XQ)ptyIB zd$>h?QfaBQSoD^%3I$W-k>80yDMa-kDr_A|GcB^6D(w`@e6YVsVlcKu7_48G$13g= zf6XEGSRZlt#>(k?=T>LW@y~YBs4r2gu3z4FZTIlTk4qwuOmZD`0-WKcUwRkn;Jvhc zAE6LAT-1LLOn}VXhWq%od(z?Jn6R)*_I=@Oc=U4Jjz%C0;-y0U%xvL6taNbPGoSFJ zVUVdhFgCK30YGV&BmCEo+K-B;S|G!ZaQZIQ4@?wd2ecgABKy(Y%S8isi|Vl*bBih~ zu96FtWiSCO{dt&nel~2BkuGAfpAoUmniI~pApagO<$Xpob{p?u>ml5FUdAmG8&fB2 zZsE4?F_8IFDXOWl0&g=hQA%uljA9Ay!$dE84%?1F%Vu?}@=A+$dWr zf{U*jT141rVFQGqw&psmSRbHkCL0_}rp}sEL{Wsc1Sj?N$rhq(Cmf|cp(Z;Ey~E;| zv5HIj1D`+Lu&`LpC|-L<8&STT`3M!w<1nptvq(ofBq6aTOWeBbsXhx@--e-_L|_+~ zcogX1NI_l5vE*ZW+S@6gmwoJque5Yq^XzchH$2|0%r6o)f&ggB-#j4F^%jTNe3a^;e%rdtYBGQ;wK^&c19} zCFc^get~{DICcfk8+cyJR=Gkgc8*)39StA39oXCTG|qyzy`!7XDiFnFFQtU;wP$U6 z8WvBjVuR=tIzT_I50jT4cWS{u?KR^O6S^o+DL8)2n}ap$RHRLS$b~D;xre49&ejd% zHezv0ed59<>G}bVLFO%j2SpOg!=Cz??wtz1!?^7{#6@ zUc)PGFII@_7<7aE@>e3K6kktNOosPm((_q2x~lS+Jk?bihwJ#dnT^ulbyJEY)${piFC%xst~w_qu@$V{8A^ z#q4Taf6S-zSz=ZxH*mU{0GPOuot!QtrfmA|&4I+aD>sM_rJopcZoSpU?TY)=b`W&hD%i|Fvw`fR|JGo$mXEL;PpF0;OH>J@$Q?tPoTZNRj z5#Jx}O!rRE2v|w%34aYg!t*S{GbfLfNC{)pqmSPTaI?o4r;2O$+lG3o*?O!i7we>R zG^%sgZ_tEho)erE<_yXdLYjLU6v5zLuwr~*Pc1AsUuUL-h(<9i0+M&FKUfo9uW_t@ zKXL3YaDs%4bBZ*HeacKIucd>lg|j7T=iw88ES$~hYgq_1vcr9dDk+NobvC}{IC>^L zICFVtI>vBNY{n%M7WyJ3<G+xeoE`Zu4#4kcxz7Uz) z09$G;ZR!fwd}0YPchS$%!i!kn#GGQD8Skn-Ksv`Xki68{3({j$l0{sk9%rb2FyLQo zVyb6VJf5D-eLCHa;g#gZ$+i8Mr2lD192Fyyh=a2h(dHMbPxay$Hx2j3hA$E3udM@) zgzGG~yUwr&q-q7dA>UAQAA47>dcK!Of>HLRvbK@dyp6kW-sV0(Xd5LR{9<)988b13 zP1C+xJ6EI=;dB!KjHR?16~F8;ZVw&ZpzJ3q0WUmfdn$8Bq_amFIxi#5pO918P5;rF zK5~@P7&PSmteU6iIFO@)3*>ppnY5aDNGpnmGBU4I^osLDCGAVlidYDlR2)?WaeiW( zP#Q+f>NrP}yWiN|pK5OBawNo#82Xu+ju^IkEwSd#A3aINS?#_W^4%=32^bIwvMRFN zH14>smII)Lg9U&11%m);R`==$#LVKfIgT=(acprGt%9M|nOFA<2Gt84P40*|24DxgRwB!{N72>#EXkm0 zMO22PmrK#g%(S-0nkk!2+f~oHF6JlC3it*@AGFGxwk!0l>nf5exiy(vNv42~a29S? zEw+7~6u1g0=?tBv7iq+?znDM}@f8I!G5(kaQ=3#|A zm%Ag%RtmCg(zd8^R_az!QpR?M3U>@-^01j4-Si#(S;m$lgKlEF-U%>|v}r_Nv6`Qb zt-LTx$nf}Zoe|owDXTO6d~Gmvoiz&uy76<$BbU~`xD z!+w&~0}3kcxAJk2ILS>6%P@$X2!SiFVux$jEX4g}7S=#UFyRYhTjp?h1vw`vHt?_!hz^S6z|y?+l&vcc>EevihL{E(VuDFU ztVdI>ZeO7}@1VPw8=dzED3ivMD9bX%9kRC|{MYc}eN(0M>ihvq+0hip#+wDL&S{B@ z0??@zNE<9RuQF4;+thZ}aYn1Z_?TG6qfT9ubG+*QD=;GZDgtZp_s-EaVwf4^2!A$P z3p=woeC?9?yI7IT%ltICMQ#6~!wyl?&$$UsCb(j0)pd z*Re`n!cC%=as%P43cQOR+umW)a3Pblw6-*D(U4H#^>O9ES~J!Qe2w(7lwwFM2f7wk z(H8ITPc?d#$M3dTPj4f1qo5$yk0_37=BBFRzy4Q_HU}|IU#QB>^M!NX#+O<==sc zuCMFl4M{gDsf9bQCMBO%V-(?hKl+Hm(_YQ0f|6xQSFLHhz2Je=ANPObtM^?&GEghR zVwSDWPrBxR9J~r*W%F-5&_S2-v~_iBGP0nywvV3CX~Fn;y&CHA+HOTUdRbnlnaXN4oMw_b@Fas2Ls@ZAqO!lzpYW6QIX-rn~9bl`Ey z)|24_gi(ecLCzQCA;3&*31Z|p|Jf*nu`w2*J`y^kCP!HHA?$ zynrmo?BJorKDcC{cngPC+ftUF3=i#z;(x%Z;gm!dc!Ek>Jd72L92(WsG(hYjrFv`d zrblq=e0G2LHb{fw3Ecrh|LDtZ41RA2AK1g zQ)i*EldFFC-^DP%FkgW_o$|3UzH6o4?a~pRIiDm@Q`0Lemv)U~dulSkfd;ef4MM>n zpi$R6`>ee?=gpgP8}Iz$+l0dYFN6%-{lZFr3QoR{DJGk0RyaqS?gX7O842jdD%W^g z>d@%?y)iTqrmhYy@#3&au-0$BjOrz^B>?}sh;=oTqGS-NHLZVq!jEfcIyL^)Ncwg` zOUCLY&jN1zgA(!KWm^@LPe(}R*h-_ezJ4jQJS?6w@$?$ezAxCW&L$@2@rE9U!}0YD zi?)(J9P^t!x>dV0BXedQm&!~ik)L-TY~#H&Tli}EzIqZKTM?DKA>_H-;Nm!W*8{8I zeLmA^K^s5YQO{-B9dB+ne8f{6tI4QZL)2^ly1HuB%emZ4Gl1n6X{?DcXwNGb@ApeK z)gmS%PH|_e+#2477MQV7ouLXS+6f=eSCAl_jXCm$51{Gf@YeeJjJbsWW`#^~Qb@;q z*3q?ZOpM!!?mjQGbxDPpVV?I!SqEKenuOJgPa3Th($6^RO(%-AkGVJ{@9jwlEz!1& z;%Y?)Q>*3>OI>FWB#}Mg6Dp0hFQ(m|^v)lDFq+J@_F! z#m8x5qJL*;byZ<$et2l;fzLL1QQ&xz&JU*ssfjKK1>LauN87<6;mN$Cf-j?+`G}LC zUDAlJZv}%v_tUuPUeD!`;6#!LmSPHA zRgZ_)H~wZp8`cTb9(Da3=dB>hWrveaO*gL9DedT3DVm5#sOf3b;L%^b%y%Tce*Z)m zRR%aKQBLjFE)^6$fjk^5^M)mzy@;qd!ee{^s58XmZImQd=oyI0py%s)2oi|#BP-?>Nojf-dArvBex(PV}%4jLE*+^9p=lN?q zN#ZiQH`#GAUYU&n)2?DnAS)>WmHvl0^|O?9w6&wynVA{9dc*iJFBO!whv=)%pAE$Cd9WPDkTfw+fhaF>0W_Fu^=g%&EyLl4ZeME@BLsBYw`SJ!Qy zfdV=tZh?cj=e0;RG3qBKlKOn$MPVs>t5G*|JDa!!1zov`o@@Ht&&WBS1++?k&d=ZD zUddGT{{%Fo62ANNE&-KvlmomJq?C^CJNmmCzLmV;Jw!-M45Q~_ve9^NS-Nan&AY;< zD1aHS3-A9}kX8QjrQDNeYG(&z0@KlQBxj3JCu@v;fI*R|TvGgA%DccmX#U(FY}a`wOV&d&k3XB9nZ#-mmrzB(iRfXjzBe zoWhlvm9NaGGUNuB*CUBW9qKggR@E#>1BSb@M^uGLNKlr7wQ-fLGDqyvBaQ7s@#h5j zqMczbC6W3M7#zDz8~Z0Sb+Bo9NPvjWyUK`v>Up~ICy`7aVm_j_(C4vmN!v-}L|c*H zR=ic*u|2Z>Z%6|;i+4;Bc2pDzIBm{oCh)R*GjL*(03=btOy7a*ufUL)n52#l>1Y=d zvMiy1GW8ouoi5Ey?fqrmJp8jq3N97f73(?h>nW^?&~UDsl)G~92QVm}*UaEmc$@{v z^Qxd6OP*MF#h=J;SdKKY5re&E?^W%Fv5jR50lK%>PIKiv{tN#Y zoJzT|3{6G6QSB~yBV&>AOG7VSd~5R&07;vRD@tILIqn}3WeYqiH5PUpQR}XA!+HB#OO@;E7(Bl2y#eux_1EnV_N<0rv=+#fIDHFzVXN z%ZO)2TOvA*<(F5t!*^hP|eY!bJ8*h>A zx%+X9w@IL85He}mlxCOlZ>ypPFW_e3NjzaVG@fMFOlNTvgOG)|_|Tpvw~hvls9a7@ zRP(k?qa3kDt-Ta|D=TvvZAp)uUpM~Kz{|(MVVzZ*&!g<7yt0(l?Z&JCd(d`;_znyH zN@M+?;)R{==cjyc88XDH2_&4C*w@Ym41u;&$DHVzc@mZUN^Gc-@adYu5%vbWN-N9#3hYQ?9FTVJ!=vS0=MDxUp zn(wgg0`YE!K^9juh-M0boef@gZyXQ161brtOUw))x?$jr6`CpB;RAVhd>BW`F->J# z6UDT#nCMg~^=5r5tJ0_@pJ;Jc4vkqW=4kQGnbyp=#9Q4PLKg=HDZ~%ZJ-G2XMZL_L z2($kDm?EXvCC9yq`IGn!+2;co0bO~%ylpn9r%9X`k5)L*=53$P+jCpw?Q-|{A`fjT ze4t38B6OzT(T0P9#mUeI|kcG6=YkSb70?fUrw@myocF%%7}t znp>{m4F@K*;qYYB~W+>VQk7v;YN$5KD*J?}Mw^ei-W&>DaALn;8_Tzs2K z(+6Z!iv1&uTMIe;9T5Y!A_RmC0wtZAq0-`hzjN^spkiMC$c6>B!1IMzdr!Ti@cPFvJ;k|LX5%bs5B2xh7bc9m_IZ=yS&V zb^H!RQE*M1WBQ?L)xV#KOao4in9^!(WXbZkN%#J^I@_`}s~C}V;pX?e?_bOhScNZS z1ZV&AZ;aCQ$u1??Pb~ySnPBDjq(K`7Di@#})vG}azjri)S{>KZ&%*VLx2aq_HQ&U* zU@O=wnd#^L3qL&``J5lZ=CKI%V-iHA89t93T3%|O7x(F-mp#bJ%?GW+Oke>oca}eg z|J6f6GUVSA0LG9BCY|q<@OGMEmbTOvMoiS^t$`>A^Obv(Bex75BMPUs;Tu~M7szd8 zWUj4@1bg?CCIqijJdthz!F%1 za7}>0yb{?fmeBgT^X~dr;kq^YpT=h^#B0yz%d3cp^alw$qrSIPt^7=k3J$K-7nIGK znQ>@ksh|pc_sZ>i66mY)ah>l6%mLx13B*Dnxp<9)zts8ivz%MN#A0}y&KF$8BqR`b zcGmEYm5DBG%D@E^*&Xjm%}-qTXMM?}SsTPfc+O2}+fVB-ob}L?!>G~irpy_F1hLlE zAwA}T{`b6ry%`0EO%RBp6vd{*nG2L|Z^j5yU z9JaDQoT@o-3W($#^948J9Ij&XLCLL!TK9?kz%x9;$tL#M184bMy>`q-Cj7OF7S&wc zYPyk0bgKQ^SOdCi$fOJv3l(pqIW$+LutsH6$}pLBOkUXCmACd(^gg3jPvGkI?Vf8# zQY+I@qn@u5=UwL5oO?X==g=C+thi5PLxg!>Ufw|fwLV(z?(1`yn=E4sy;A8iE}HaB zwfKx!<0SOJBhW&0psiLK;T@$gr@PT#(cD(6@~Kz5NJMl%Lr-s9pnck{_Xv{>gEC{x zw#Tm-v{r28_5|aawXtkuVxxbMQRWqiASML6di|ftV_j)$T4m1i&##M2+L4!>6WOyV zNX;*1XIdfgdI+g`0WL)vc@BAQOX0>6D={k3_14sWN)CvyIPdBStmsHfz}f|oy+XpARFG#tTJJmsuvj23+T7hK?WAF> zu1`_5zjdY{$4Qlflmd+-ev?G5D;M3yb2krfl%*!WyXSuznfEm|nxwo#hPgvKEyTf@ zQ7G5+gSfIg?68^F2EkUbZOsxt^|c%4&_pFzqsztC{_BQgG+-` zV`QUwbzl-O4Sw!Hgq-rnu3a5{a*Wz)O-nASeDml(>um#I7Y!IQYf8s3)0sphR_shN z*$?z*aK#>uDe6qLLbXF_xpiZ$J0mmIbnWXp!^Bh7jKq^qkHu)NE7Q=1>1V9`b49Et z(AI1QA^gH9d#0?8g^crbs*Oq3szguWY4_+!RgVna8L_m{3Eokyw1A5;@#kjMb@KAw zmSTe-d3D67Ws7cGk9nKXY#CctJ7|+uG5!!p!UIWj*FDtbJ@|%^3HxOf1lS;Km)ZW3 zIYdY8b6VA#2ucK95Bx9=auqqIc3u!4wV6_-pr1SaNGU6$W zz#AA@ylFy2xPQ8~4aw7Rl{>3xTwgibjeUP~A6T6G8af6wo4bOgGfVG4LLm{5?tvW| zMM^yD?eE!5e^>JDj!&+o`d|FlAYFASD__vkf8)HW5?=@Min+Lgj=g3?Z($OU0-m)4 zN&x)wO9y9m_O9Z!Rud1MBcG7UxI>%z=q^AiPlAlZ1xPZ ztX@TOp!d@HiNdS&zOw^X6T_z$3%{m@i)vt8`c&_hAnG4N+VIPDgD+ED8au+Wo=o;f zuv!l@)>zBS!A_+}U2zrwgCHSAjj^bAt$Q0bE@erCPRrau*ASzMsSlz&#HmIBJGAcj z$9|5f8twyP@}0NKI^DU(d95p96Mum z1cN+l=zbC2R^&BsLVhPCvvd2bVeiU#TUsp<2NTY+u2c2GXu5EEn`)A6$_-|zpo@!{ zkrRf!9oAimOqWXEK)WXku}Oz7SlH2Gudp#}>g4Xl8_dYRa8|-$9WNIS5%KL_K*B$h z9wA0muempMgdn*|losFS9P4l{v9;SEo9K=WXFAfgr3A6T|&G;ai(3*M5rt9%iAVktI zzsmp7)Zr-?RvA-y9fv~&{O9;}aphMJw7;c1!fDZ!dQJeZEfU_d@%(j&d$w#-`gPY% z`uaMW#ImqTxtIIU&;QiNox2y2T_-fEAM=tG>|WzC*N0gzg7XuzA3XxUhr%nfKzW{0 z;igl~L#(GG!g_M_BPHvan2bn;%SZYS6jo*cILnS@GYy3k|CF#2+8-}c@iA)~#{p~0p>>62cYYfK>9wXIef z1Si6gN}%d?6mf?l<=T4=a&c$QeoBt4r>LV+A83k-h2=7Yh1w3C#%AH$Dfs>Q-yZhV zIJ9p_8Q8(Vmt>`9KuJ;}(l~DowmT+oi6mttWtL11?8LDTIX1t9_}rQ1YVkiq=;={` zvNH_#Zc2z`{pp{fce-x$QNFqfaOalO)Zwgxu(m^N74uNP@@@6}X$0Vf0+?S5I;VH_ z&Y!x(wdXJiK#4!!-JagME4}?$y&!`v=yXsS4xH+VHSXEhMc|Qts{`NpktF%IZA+wY z1|2S$+GbqYXtz2xJz=nQzZbnaiQ(GVbX?ope}uo3z)!6Vp#$@D1)Ty|D?I_5t(oz; zC*eyba7)VyXXo{4wzl;~il4mhpZ-No0NW)dH;WPu&8VkyG>LBSuIzc-6H>jQ@tai0KZrDeZI?$Xe6LpY(+}ADmkfXbhJL@_vKbGrfD(X*BqIq_4#9+c z7i-9E;1teD6J|90`hB!OjcR^u{qGr+04BWw#UvlB#{c$x2|#rHi2?~~N&R{vso-EB zrAG!QB_wfXi^Oz-?3TkAnU literal 163147 zcmafb1z6MH_di&GNUMmz=n^UE934ssh#=h|-J?N3K!nj9Dk>sfqee(~$6$0#V52u; z4F352)%SUR|Bt`D?w-AO?>#rp``mNSIrqKAywp@7x%>Do9v&WvnyR7>9^UOLJiMDH zgm*rg{jBLm3 z&&$UhIp4kdrL1~`h%G|)Wo5e3?U{Gcgs@RP3b-NO8*WQG?Wg8u2}*KrsEI{hF0$eu zpDpJ3F6IcD1vf0^N?u6{F6GeRr3}q7Yv1{KL-sqrKHY=$ft<92URE4Fx$NyVJW_rU z4wZE(8k#$ol~C94#fo`ITKBnK>y_cF1}GgVo+|6QwF@Z+)uB6{+hY~h;oG63w@}KS z9!fg9VI{opZ!nRmiR*PMKH%aQm^Ij>qS_#sy^VMKZZ?nR?a)m!)S#5+laC5(@(>D- z+w#3^3DV+$A~j>XvUv%anYZ7(`8gzY)+APR^-$37DSLLVME9YwwbC!M1eSjD5iBrj zEhK&LUY|X~oeG`P)n!hHVYm4w6V>wH)WZT5Go;vVR6SnTy-_x3*3T2D=7x+8A>imC zi>EP{4eqhQM#)d(blzNrLOS&T46$Y{630g-P9ZB%5kTN z*k{%|MsN8xZEmTd-Y57X)|J4J<`iP;R4LyQo;Z3+u9=U`7Hnx$n5R1Oc^r)CB8(l{cx0|svTjb7Re zJ_#`lzds9?dK2rlf6s%xEG}Sm7xXC5zvFFG({fn>Dx<%knet}zWv96%)XrZ@o%hHb&$s;0OQb>G>{LR81 z{lGG+o-t1qkMg}D6Ok%q(g#8cqAI*M@%Nd!RK=^Pij;$uG?@{J#j4yM3|or+iK4xB z_Z7H)Nc@b=`jY&TdXj)d$!avy{z+EcW&!tTtv!ulBtX`2?C7`mG4EVhiy{GkD3`^( zLA>g2V}9V%8!tL__yr(@+?~%WpCHMn@et=C(_v_fd^cvzp3Hh-z&$ouO2FJ*|{KZqdJc@GGv3S)Ju8 z@dM=j8xJTSMBE*?|NH@lJ4d_z+rF{IOKqhw#W4*5`NU!|E%)42m##Zqv>z5egj?%X z(aeU^DQL3rv85-HCGsUPBrZMS;-Y!d@I*t8l9NaKiLS&KFQp>gEh?orOSOZ7!B483 zj$8p;2+k8O*P>THxu&wSRENHmeS59T_cH$_aQt^ojL^fl7e?yEpQgrL#!R`mpJa1l zIY)Gc^?CJVz6UBaXenuTz3S46EUi-gCcp9Zc~gJr6O*ae!t%np-f`Q(Q^h}}-?9qp z*5V15&E2gvc}qRn`1{d9^sWkGG=0K%f^WQODrVbu`zcxo&5RD;=Gn$Qe*2j8vHN3B z;U_|xj$Zuy@43Redd*9$=xk~PWWW>j2*G$!7j(VCnoAt^lep=GQF7s%AASZ$Wt(<9 z-(QEmZk)bTDpA@qR}cAyd4&?6gV*rAJ&Dt2*ptZ1Tym0hxB$`k70WjE40FUyYn<>m6bYB@7RFxSzTj ze15I@`bKf->ryI0Dq{{pj#nD?b>9uJ28cH^2ELls>i4qioh%nR^^-qxuxZZmZykOu z7l{|45ovM;%>A5w596(Nt_ZTSbLgoou4uB-swJy2b24+in)}rshNu`?$zxd%n0NnX zRj@|QVaHK9!Bl8!XN6!zJV*FYd#9_W3-jt56P2};#hPd}1bcXT40ua!W9ph}0~)cx zCwYCg$$rVcTDyn*$L0aZ!%hFagWz4OmE=v&HbiQ9 zTsiC{VvhhQ+8P;*xDdEvypq34!>bL2g&u}-guM?d3KI^)$;HbZg%97RdZ!RBdaJJ5 z>kZv&;bLH?xV43wuidM zS3O(Ed8KBDXpZPz#JQr5rSWQBXV@10gs8?|-ila(xiVb6I@8n6H20e{x7GC4Y=8A^ zHJFVeX*C(cK~l*A(?f19g>IVZ^nA(d#MB?MlZmLb zX09ePx+jat;p>BgMI(AqQ3%_ZhII}#yF5EnrNtg`chC4CyRvcxnpkjVUjdb7U26Ca|Te_Zz)3 z$1&|FNRkPXQq$wBu&S6|WF0%g$3_d{dW_@gC0sn9hqrKw7CbtV<&f5Rvl;FDmrMT7 z)~&6=13F@s*e@X`DZh$7Ogt`}$z{ra<8FKEbU7R5`NcB`p*!L$JnRw(vY3mM)X>c~ z>TUdG$T{cyaqTi9l?FXmNTqNKMJg0r&!Tmu}bav zrsb9Q3TJW9vqD-JKaqQxJET>RFN0k5YQ7LejC|gGF!E#MdHxQte7AB(e2NwpaMX27 zy}+l$M}WP1se>y%Og$@J3kkSdmWmS#CS#cp?s?=E5AxRI?Q|&Do^mm(s^2U)MAfG8x(Y(522nc_Z;#8 zryUAIeQv~GI|)Q?s&72;@E)@LdE=|;u1^X=!3c78a`BV^Ni+YehQ#&tpKLy6#(!1ua+GE^(0IwH;Ob$+D9kIw z%g-!xmywZC%EQ`LLPt^gKgqAZNi*AfdAUjO@%j4t^7;z$x_a3035bh}^YK69d-jay zx(1J@pNp3Th{wf~Q7@5FC9QOPHz+)oNZh@ zuZJchz%Tqv>RA#U0{2Qr|h|s^2{#((%k^UKrgqDYc%{80;FhfQ_itm5r z{bzkCzCWD$H_rXXZTVO3^?i}KE5-NU_gLnx6N9%Y9-b_onxfn*5I(Y*D3o3=^(w@4 zZ_7!8k}z6Uh1^|EMM;+2qV2(?T#wBwH#uBU2z?y|Jw1Y8?Zy|u5Ah#>Gx#UkiWKBF zRIg;)NR(3adyb?k($aLEQU&G))aAN$S#_I0Sd^!1|Fe@85`+ zE1ao*aA+cqcH%@|*y`9P8Yjdb^g2{5f~vW7#VN?~|2c&@$I6z`BHnqIn)MxW0`+F0 z!Fs*%{ZBu-d%Ul#6riP{sa#oEN$EL+GR4LIg-h2(csINQ0-C(NyuLzfYlY>AC?sz= zd2K$*|6nLwqZ&Q2es(%Know#~=kn91#`u<^?B5bQh3pU03iB{;AD`Dsj+>F8^k-k6 z3$rIBJtMc;lzGf5>;VAyv8hIEyA^AZ;c{yLIF%;x}S6Jso zCuvxdX>zi0F+~MUbWCn6tn(RD<?uB z7g6dc)*Ksz<{?4Q^0zogavLtD_vLV7>?`1p}UDa9p= zUCw>KtlrC4(pA-dQ)55A20l|fASWZGMBn@adV;E(f5Q+Q-(X0H4UF9HPE*0C3;fLc ztHP6Ixlt=s<@e~m>Fi`<P!!V}EOEVn@Tjt?_FOAR>&PQ$qVqO&;?ql2-Vu&0iGb zpQ6!wVOwvj(SoCU7hA7U2XeS6#VHA)?|JD2J>?tJX=AH zWEspOh9Kn5&I=TeSxew|)XzmBqLk)~<^AY|YWH4@G1NS(R`0J`Y*Qrc!U}w*0yu-Q z#P|uG!wT}AN(*VF^#giyOwenoJ1A~$QFic>Kt9JM8A*uyL%Vz5&cps7{^l(HKOk(# zkwC25HfudmTGu1{2Otj?4VyV8fyGXzAfpd6g1kp=i)_pLWCO!q{!Jx@Ifu20pc034 z6A1mDEd)Z1HBHkJb@MHpoStrcOomJ*r|oxM;QA*@(@pF`qSwK4asiFOmuff{^0LBgOSGVAV zh}yG_P3Nth)+w$t708PIJqb5%Wk_tC3hANW5rGx&(pCeXnmRcvtch$Ny_^@F@t-Qw z{2;-#{gv|{2K!&Ssp1<_mW9XvrwvIYl4q)D_V+>ngGCdvB`O!kaOE;!Ja4iram0MW zkiVypPDWpLR&`LHQb!rpp-aqM0oGDym(Gq6LTM|y&c2m}e~p^$wl&iz#42x&u?(^`gY4w<9Lyl+EcX zLI7RC0ZMWGiT+8S`xP>IP=elG=ftvLS^hGSQ2Gpwx2Kgoxju6aBF?T!jX>O-J`x%d zkdET77<*V(2fNUjZG>*{>KDR;+Q5&QXlXK-Vx}*9(96%oDDqX>XZ_l`cFgKCaCHLP zaqws@u%EahO3qAXe9Q$bvHmjPz-Y;qKxR$6$n?>ikiY;C`2TN z0d_ORMG2SKAoIYm5qMWm{+O_fB=%VTYU9QHw*a*!vfagIUAtHmzl3F-3FtF!rr7kY ze&9*Slf?y^`q|6i_WX|@<7csZG^)%##CIEARX%K>;OWpFTwkZtOu>2T@U+TS7Z2cEyA02-zs!i0hSuA0Zgn8Z5M>AHgyfE3oBIjq9*`F zEF=+3?om5JHD{-JW#+)L>Z*?!@Xkf$Wz+R9XMtA<>(5R`O{*7?StCLHyubso9kZa2 z!z^B`3Zq@pXi-ot<{)4i?%*u>)nIXuG0#`0ctJtR&qkO0wvpOD%h$D?`{{Wb+k8X* zxC*t&VDPb=Ry9lOu`{g?8Oq|s9&kv;O4>9kjl986I?Ux4pX^zDRrQ*>kfJQ+b8e?| ze$?{N@tR5R;sOU$^{{^z<3}81S(*Ok_aO+(*W?)#TMFD$$DYCCtKj9>+S<`&q41>Y zH;Xkv$oOC^xbn7)DG@R+%Y96WZE4xPo2&N|>t9XBmuwY!gtJc0T~>bc82>|j%sgLE z&wGZGTeSP-!Y*EQn{xWGDv#zZ8j@oBG+;I~CtCbXxIlt>#|xtHEupRtG9TWs@D8*RB+L@nU%3Y92ks!8Iz1E7!uu!O#Pd$U=HU9O)?Z!8bpdGW78;Yt!J7s5WAoi zBaz^J3&?QSCO&mKfiXVdaD#BJfcMVY=b_LMPo*bigfO)}%tp$Z6&PgG1jvKdP}CgE z?_w-k(lZg>`YxPCg>}sx{a7TJ0aR;p24pz#aTtT=5enGFAQDB|J7et=V zQ{`v2$GoJGo4?YJa6IYN+?FJU4L`^Bcgf-8dcyx|IbPe25KDsJ1A-dguAQx`_|E}E zO`ykqW@ct@T?=!4vK*SSYXJp~OO45m;&Lu9W#n+v30RDXiN5Av>ugJx;cEmm#8r(gyQ{+^@nye0syK(+RC>Gu(N zf_}YN94HP?<1T(q1N$Xq%ymENSRBf$Pm`tum6*mHpqo+IKqw zF02%PE^`~srgU%O zx3z_@JFTxmQe&qarS}dSe-9Pq>e)VBO_ZO{(#c965rN&DIB#~xZvmKGcbbF`Z6wbUmbTp6O-D?3~bvNF50 z7mXy2$QtSO#N5#Bzo^wV5e4(6quzK9`jyOusRidNx9xN%glO}`>jp>k9qRm4N89a;Q)URH8I$&LFL%mU&x04pkTKE9W%IKsECT$+1ga&2 zeo38N?(VxW#=103m4UE8*k17d)VeWB2}KsnI^BMR#g+Y+D_9^R^)|~ML z6w`}N+s^FOw;4|R4IYS;he~L$&9}`G-f{#Vp!#b^^6g>iU#-ZQ;JT|QhNP#p5cajn zr&r!P)3e)C#oXv-xeO_!1zEAC))Z;w7#52y-tb)tIw64cz_Wh_yNjo~HlZMscPzc! z{RAgUheMZSzevG*e8vH-uIVN|YL#BLb1mCLPOZ_|^}?PPKl6WF<5hcPOt@GtpdZlx zzx?`FNLk7=+tb3&3C+Uv)>|h{^=BnyfiPa9#=$b&DAp+^K(nqceLRYKVAn;u1>WJj za&=ZhvWGL^M&T44gSvN|k3UqGTm^8!NUkQg z!{`sV{V3kieG%glmTGj0o)ZnecI($jm5aCN_drpo`n7nqTX{d52Y8zXgVDuidOMc`trxL8 zCXHT(@I;1`otg4FzTkt{EtW4+%7xo`RBNdtsTL4E)vZr@lZoiXAJ@l;5*a?0ScZWM(G{s#t%N{AdZ)|iA#suz!?Uiv8VLAeP(W7NP@p(nC(r_hOj-qy~ZCzyvCCAXVU)COUDW{$rt{UTs3>6<%7}$i7(W)r_v5b3&#KveMq6T1Z=;F7 z%822~JZfkcp3odId9Qgy^d-05RQHbb90J^UasrrHvIjRE(lxo~bIfKv2mpn;=ywMB zJqPGZ>Sbl9Kd{)V?m6pU*9x{FxHu!j@BLNFFpL?W>eMgXv9dIYx{>Kxf#!XZjuTBy{)DWo#xdY2i^d!1Bbbf$8@1prYtBA9H2D}Z`9pkV zuqR5QTGCjbIRbBj;Fi-ilR1kH)x(Gz6>}9oteZs_tr5KVhIQ5r} z)kqEEl)`7pAZsjWO9d#1z3hT?5a%U<0<1JV?V4wdJ>x_-hx}BN1gL+rkdTqV(KBYQ z*xjbR6x5g|E5c%6KCvg zn=q~8x>&vzm>Aby&4i{;8$Q7gVEw{k67E7Pu z>>x44r!h<4#ms&%6kWr47MtxH8B)3vYl{%RUnQXPwe*>*gLg9w*b5pRgShSK&-#PO zp6xQ291oOzh(^A9M*nH(!KMIpa*v5BS6%H~)lAT&&)!~jb};CDZIgn8yw6>pBw1D( z38K-h?6{VD^pBsTE(uB-JM{KykBpr9a&eHQZ+5c}M%yUk^MVPfs27ggp3*l0M2t{O~l2KV0Rh{WCE7L2OUp#&Ks6cEETgqJB`r^LS!4 zm`9UEh~B-ELXYplHn_*^>Ri8RJ?X_yjjj5%<14aD#h`rC>_AiXy^i zbzCqfyd}MpH+i(IBy^sIMq3*r~`nTBZI$d2G}?@w^!7KC!iY3J}YpL z%z0j!5%H`J==3_~C(6g2UXaS%*!rv*j2=w)C$&6DU~lvJ5Q`cB1@ezv&AaE%(oh?0 ziPoLh6mOP&*sJnN$?k5m=uvk z+v;}WU-FX(Z{eTAyZ~>PYhw)p$JIDygHFyWClr~L1M|p-Lawk&i0bWlmUsF7+eKuQ zl@tx+>IaL1RMghMixskD0+5W^8W{;bl%US5A1mZqvoHJ7VSuu?6*r1ZD=}%BmO&S3 z(TJ$1-P*hxrdRx0`uF_Fj@i|m$W-lvnczf73aY=338HkC#>Br6G`6(lYKEPg8CO|6 z5}knzPCJ(VcbpggER3McurAT;jgLT4S66t`>+-t2g+-SR)RB{y^`IfeeYf&F@`nl3 z&t*(Bb!4h4LpaB+B2w-|IY;F(*alpC@#QibL(?m$1{U9>RNXex@@N3+id)bmT~p_F zNXC}y@(|n{Gr&SxS4FLF{ z1JqHE6s(GDMs^BGXhmQ{qD#g+p>6L+G3y+_*VMz z-}d;Nn2oKk>>-Yxd81Y{B62N$Ap%ek;mw1|LK$Hm_;?6RL~5FXO~2eU7m^x#MDII4 zhMD&qx&QFXqR2x(^3^YkL-ctWw4#SUz9#LfJ9tr%!{xVfHWbk)-te8u9zb?`bj673 ze$)<6-MnS%`BeGO6}gO)r7PSLCf*my(%f%I_ORy;)9nv2!ha1HyIYc)RE`X%SE`&I>81igGhAHx@c_$a~h@nn$q znQ*yZ+=gi1F;)Kt+1}%F)aN&iMV;TGY#r;swdO|$kG_s5TIL)*ChGxlespFZ>)~xe zRvkZ7P0{1MVU&c2_$4;d>Eo8SR(ca^UN_w-%#E-LHj1c$8-@%(8W$q&1fm?xvdhBO z=3+eQR)trH#Z^?6*8Q^y!v5HI?71U#dLrj0|1BeA(qYb;y~-CRb?&1rL9w&v@h;pX z_2j4D8M6NywTvojOnGmHu zU-R+!Pf@NN1hZFP5?hfp?@Qp-*@%9#Zi9n7ND+G3E!raBK+DF%Bz zthz!bszFE#4!Kz4;Q2gE-Fn(}{~7Z`=$3|S)q)S4hn-_XN~l*l=xTTGa@h2cl8jrI zg(>A?CIqMNtyWPo4AWGJ79JvT7C_a`m@31v^DnNSRx+>@$0C}RaAlQR+w|NsrUA~i zW{05!yAgs!;AyatE256(S%Z;)==q3lpaj>kN$iE4f#fG7Lx|V<2H)`j#=+PdY)lqyF*UYm%~Wf^z4Q8NDxU&@kFY- zYX)jR@a&5prh@*VUgS(sJ&QS_64=nD=u`j@Mb2la+%2d7zEk@mXU>qTACdR0W`tmW zSZwX%PY|`Yg26|B0pMW1_~ox^ah19TK80O8Vw$NQLedIJZaH?z+2Se+7`g4 zHDiJxW0X?LPON6we>_YD;5rCp>Q{LXi5V^kJ z!Fh#e*vpU5!>vwU|rV2L!|866@cweg(rs9x2Z+W_RI=UzM@3;#NT3rr1K8Sh@}JvzdXIR5w8-4GoD9B^feD||%jZBl>?73@AOslC zkqhMT9BG(dTR>jiTDZD(<&Snt^262*j~10nwPI&DhrSzT+~pt`|^--b8 zJujhiW%RRS5;*3@DI4dL1`*$SA1y~k$C-~4mCGTI@-wt8zxR;IO#qJkZKxxn-gi6q z$!Jw3#SYLzDw(YdCE^>$Un#X=)NVN|B1OtQv3UWo^d*cox9`du)QfXm_uBh_q2LuJ z(%qwj+W_K##Ml`pw?_U#0cuTYnUh6Ho~k^h)bf#ljctwR(wD}a?givA-Ir&drM`eE*uU1*i3%sqsC94O*V81u#!-*<@H#h@(!B#<6vkjc_b`c<|rsCJ(kErHoMPT&VAa*>xC1tpXu3#@{ zFJOX}ncCP*4o&BNJf^}??$Xb^ydPR~x}0RgI;o@x%i7tOADt8sQi!a@4Mq3>^Bmym<(__lCuZQZO+_s=%NY{V>iWArRy<- zssyaN*&n~3pvv37h3yHs=((@VSgtaWc^T*gLUagz1>pwQ?VWhoCL%A(cc{ zG(w98j5H!fJ~vxR9FHm_r~A6gz@vCJ3RYMbHCN}K8+CP}^&+AraK{rIK`GDr583ap zd_Fv)BYRD_q12BJlF6@Q6YDv=V;-G5#(suRK)3&CE%Iz=U$5aj$2_^vUy7^aSy?eC zJF)X7Ig$YGA8_-zK2icE*MoYp-W^Ts0woywgpoAJR-)@bG(jM>yS^)@ecAq-ZU zmp)#zl;C%LFc;ssAw|HsQY@WqAm2-I(rLQ-nbc=~p>gJtEs{*_l5vp}rNifE+4Nd^ zeTTaty4gKAD(KkCtfC1M_>ma4BOMqWlCVp5lCiL}RtiwIexpxW%Dtm}i4hWc%M)BF zx@pQ0;hG4ehoPt5GkNo=N=fkxQAemY4O-poX%XF0x-IBNWeMy`y5%x+UXj&9N^lS^3fl3;|L>P(0QQNn|acDg;OKXe;Wj! zILv{+YRx7f%E!(legfY4Ej;SEiN3b?{TnSdS5z05Y;}QiBc71-m21!a1=-l4&6(KF zhSg`7TKkkbTCx_({7kfi2_dED*Y2hy?=%d)+Lzg+b6bg|{!WmqokoXM#O5Ffwcq6y zrGp-+i+!<*7iTB(nq;}9BgMEGhR{}_Pd&Fi)*5bdmnk&qcN_k&u?|ZVV>rMAJ8yh` z0%0pRn&*uty2aTpEPd^~lw*Lw9n>xRk;iw`D>c=R%oC30B$v*He`ZKOu;X!Ux$+>P zoFO?@0njCo`>q*rLUVf*^wSVH{9#s%b4{Y{{mcE)yw4eyz^)6|pdBND^G`X!KK)#U zUoWYHDhvn-?m;%3T?35J5~6%mn)sCql^>VVTf-h5u^{k;TBYMNVJ;g(C)I{H_ETk< z2O|~Y#@UI7q7(M5i5cpC!{M;kbSnG%-!+}62U^dDT-o_tx*flXfmbWq!(|}pa)_nO zlknwoC?=@7K6>DlkM@8~h&9JHJjM`l_^)6?m}b_1c>3>XSY3&sE3m#`VzbWIV4#Bd z7q4*L8K$n$1Ephsj>l@_-m+5st*5L8rt6qhnGUybt&NsW%ePmrTeQZL7qPc?D#i7*3b)Ikf zIjrPNM3_s9%cs5gn1{=m?kGHRz)bW)7p@~Xmey@ffj`<9iyy!0QrByX7Zf?Fgs-D>0wu%m!waAdm$T6m z!ZXIYwQq7n2~oB?z0m#+0b_6lA~l z&*s~6WTz_o!Q?JL=K%Cp>{n?1!6&ivyQT}#Fjl~ECuG{ObulnrxXdCTARu|-M!S7U z@@4N{yN`0vM)gjJLzGR(_tie{hpsf;H-EtplWS`YXC2Y7^ynpH+BFCIcor$I6_QlK zS>>RzxC;G5G1C3TO?*a7wx?ahGYe@k=?Wf+tA2pilxuCEkB|ICCs|X;^`l_?$-(4^ zhv;womI&D1((SW7b>K!`+b#Rso4K*J>8r8+Z zLV`O@eYRs4U0Y2-5wn@QTyqE}MZ%v8aRd<8(bLYm28ph*ai>z9gJVqGE7}rb<~^-m zA#m@rDz9qF_n#rY3Hgc5Np z?^0>^>x;5}HkZ8BEoiXQEJejCTQ(EE6Q{cM6w1S*)X?j2q*v~*cgT2mEG*WA_tQ1U zC-zhb37N8eKpyCw*oxvJ;NmulpFei zDs>dU>TrrlkeT^j-h>f>xTiEsCFfN&49oI)I-m6yXK9Q4feP& z0D=0=n+T})FDX6#7j}rr1S?&d1 z2dv7cvxf=k2yqE@>7MXu1Od0=VhFY_B(^Ulic`wryOx{Fa$IJPZson8_a4X&)o9^p zXd;M~u!)MuJu<2$g6p$0M)bzLYuQOZG>^P&4~*F1kls>rXceGGfW<~E^u?B5MnN8Z zQO_~^kxo1|0_qk_!pn3e!?DOq?z<&mR^(wH$QQV-`t@6$L;@?suTSB|{j&Xzt&mS$+dIRZc8Pv7NL&psU{7t&bt^+0 z46(X6+jER@Ph5O~EdhD{Ght4jD4RL>2GWIF@5z!}<#rGsfPP!#79N-)jfxxm=u+YGCgp1l=0~r`;C!I|J+On%L91qS)%9ItZL`?ODD=Nn(mUZ zs4fqe>9tkS{&mw*ZQc>CC<30v48IT80aIMys~G)i7N(~aW%dLsf~L8OV#}NHIT0_r ztCoAA^G5O6ILnMAd*6UQ3ra3AjZ~Gh-lIb#E!3Zx>BfeoAXad%dVU#!j=`zXvYy|u zIohZ++y5f9^)(r(z;K=rblSlX{PHpp9LU<$6L(lCy-xUhkV7eWRE~g6OF}W`H@J41 z_RlW&PWYgZBK-w#!&k$Us7KM!UzgQ~gE#JoQHoF;fPL(YfYVgYJ;D9X{Ovk~iQ0M2 ztM7Ah;L{EWbqI&()KqG3*ey;oZy?fztX9dcH+E)~ z`<6yqA?p_+)7ep3va%b$xVg3k;^X3+ZDra;{hWN)w_639Vpf0oACnB*oOQUKs;G*0 zS6ac+bASiYTav68?$1szj;}DVx*+h@YK@zvAF?$g%9%M0$hAG;5?GUUMhL&&>%{vz z8{(dgxvUSyrnrU-7SJ6u%yt{_i9-bULd00s(FH;1YpG~m)-sKC+#foEYECMokP_TH zzd?3f(AstRHEssFDLlm=Itb3#?K+p@{G4l_pX1PCMHNHPUi0~AFC69G(wenU{)8ib zTU4;RV3pmaDj$uq%c-_Ya;*i-2Gby~x=~-?%BAI#R@~)H9WIUT-%1bN039r-%j3n! z&NyDTI{zPuY5gN!OB}xO-D{7TJUYAk!(Tt|9M*LHe08|HjLWL25RiO#F>ct`3gYxV`U^Zq=L0gLeRygo@rnMjLo$wBtV zVK%|$S>&t7BBTw`sE*Ol+S8Z8(^RO69-g|ZMMu#c53*fbNpb4!`H#x0t8t^u#x)P( z*r?s0GLOrHbfRdTU5p(|`Q}*pct_GJ0a9v?sRKJ3-Y`wmH!dQTsIxtyFoE)XKCnLJ zo8iWKu90*EJaq)I`NTB*EAKn8BhHaH7EyZ&&xYUl)RqKjkV!hN7)EJco$^CO!krRu zwl+rlEBU=`=NSw?&>%qzaiZK@Ut|-a%fKWNPqOKroL@I^sVEdrrnjQedy`D6u)w&6 z9>=q(GJIj6l2V#lJwo%lr-s3@{rV&;w>^dt&%IjfYWX@W&?MD$o?`WaK@|sCH$!Yr?$3PVnmk~@l zHR%YZ4XXKUuZ%ln+4f*C^f!L{1B%g z$;4XiVj3`#-h)Y4yFDPW8peaT2gq@6VY+t{O)N&)jT zq^!_A^r4`o5+VMa84Px2g%Z9KMIl^yU$)vK_MJWw1fA`f(?g|0M-`coR7@PIe&P@< zobS1)=ULs$-=4=SIo!D00kTWx`IQXJA4xV=;K}1-aJps4 zbj^8oJPW-_)y+7RY)-yilIZ;^#P%=5Olh#iGZ+0O$TaUIDs&-m0qwKK6ErFqG}mJ{lBPO$O{9mcD!ie&F6(TY{?d;%=C37FnR3cWFvxT;MD;%@}1e#owcr zs??r{g4Xo7*f#HOJ40aMImf#e8ww0|0a7JBGAQTc`e+jj%JCr1`Fk}LpMchU%8{X6 zHBmV|G_s+WyNu73W$CIb8{|Sq-W!XKo$w1VuGfhop0d>~z}Sax)acL4OzZ8OJb+tC zxF9XF}!-5P7~J5e4+WQyeIRy!uLoZ7^ZP%u(UI>ZuC6ZV$?Pa>2I5!)OFw(AEo%ASA^||Ix4{A4!c|zFfkm^7SwNC9VM@u{n+hpVTAVEX?*r zo|k%zjS$jO$cvOuvLJ`P2S>-8L8^k!8JyQ>XoAAeq@-dA>|xw4ZL?QhZVxM)6GgeZ zRXc;110Q2=8JXU9vZ4`(tOP7!O@;ILT}RE=C`_003xhDT2ss=#bw?!`EFR|CMQ7CM z;e?M1s}RZ2kB%8?gD}2gSsvwBHCB*B8bRU+CXY>!j3{@P>wkHvs3J4vk zbS7L`ao4D)J!in(7s&PhDh7Cj;7PHil)v_aQ88#l!~bLNt)rp}+l6lx6;v9O9y*i` z$)Q_H43I7<>1G&0q#FdJK>F;yEcF9 zHO#Di?|Wa@eZ{`+-*vi!b0M($Y>QqUhG1&DsA;)ym{~2SPa|3tB@!4Ai@)fGgk8Kv z$+MhteVz9a!(Ep1H0GIM2$q+Uu9ofR^nH} z4@fr9xC0b5=e0TkPk|$K{^WUMI_m!F;eUi1(GP5Smd#X38yu>#e7kbc1qSDj(R<*G z78_kVJM>}HQ6z4)bWlz$#yxzS%4lzQ1Z8na(!5s&RA!yxG9v6JP^HySqLVZNv~TIO z+||!R=P(La7Vd4ci(VtYQ&Sof6r=R~u_q0p31;skg{+BMm^-;OQEY78FR%Fnk|iO@ zw%23__HsgaT$Fqis|jhgpZW}{b+D!=mDCYFYUT8-7$v2MOW~@9Y^V9n)L%zH+k6)a zTwiu%M~YCmpTyi=F3hN{YqPt%81iDaWyHAYgiF~Tb&d6Y-~Gbpj92Z|m0TZS=U?|Y z=pp64{c&hB$al_YJZ!EZzq0x-t9Wrpq^sczH;V_^p5f!T^uEsZs`!l2KA1<`?_}_% zyrscs@EGF4Yw$b!t- ziBAL{Z@0=gBlfTdv%)@s?I$G35P&qHA?XXoKQR4$WH50jr6m^~u?42rxXQg_CQO5V z{p`TVS3im3F86&k$g~1IToKg#5{%~-WEvAs_o>o5TwUdn{mts&=HMD;ITQE?kgnw(KWK7N2{CWe&Jd8zUnc$Scu?T6gx%5!zuN<+K(r-Id!l6OarEnIMypD$vC6l z=a)LVQqo;|gU$Iv@vd?91vJHRoAu_Ad;HR|pHfZwAHo`u=c%!t=Hh#)^bi;S%lI^b z@=S176n0ai=t^TnH;xP&_3+hH-+JkZ7ewb+-lsRzA@Xoz!rI^8ab+g6<3Kx`dSa-L zzXa=qJ%c_f(U&@MCpoP>j5>@0b}a=JPY@1!S*KIqoo2i-n4C%9Q$5?{T1w zta}2c_#9@m40^Q|gVIKK5p(B{q12_Ul__!;wsZGE_jxx#yz;JoGoD4^2DU!-ik(=W zVZ>ejB${`c6m<=A_axL(XP!+wMgj*5y*a$54w3EBsv;|D2xkz(TMb6VZd+!Bu(fMakgQj@kP$OTzn#}4%l zo>O8(Ur_upAIZV1q4mNX#~t5QiRU?AYv$1o{)0l^58PteW93O9jgH1#6l2=LGuaM) zGsVf~eoM-0Af zdKjy(N&*F<_XN|LSdC5MHmzqRO~8-S|c;mF!t*jtN`|jCYNjKb;j#nr zeexv40X0A@H}LBsCQM|N#hh;yq5g_&P}d$um}JPgdfhp0anL}sJoX1aS%93qIVpb< z+HSu=FXyyWLQgczC(|@7?7mt3bjN{FBc# zjg1&Lf1f-j-p|l;8s{yM&e6HDo4uk&r({~V1nh>Yf>wBcLQQLs4z)^v;WP09Q!;;7 zACEhW`#STGXX>P;x#{7yvWAwQFZo$Y!9{@LCU1~EOhw+t$b^WLKfkB3WWZEH1 zg#|%g%u^<>8pO|&sEY#I1gJT$vvygZn>ntIih^c;|{NMY@`IW39+`tu8V>tBjd7Ct$AFiq0P^x?tjFhq<+z>Kk_# zM8u7h^A-t$8dH!h!#L@qf3|)~Xt6w?hE_rSnar6Rg!)owCeGzOzu(_ec;||7nU*Y9h2o+P8XdBD0A;D?xlcqv}f^N%*NwK^amTuME0_lT)oTq z#PKA|7=@_igS}eWNKX_h76R$`1o$39o#GU5Z4tS&;oBlthK-f}Gp(E%G|FmN!+@yB};P+;&dfgOhoRuy#l$kXbn z^_zLui9RqJyO9yZEp3zhg9RWDBHt}5dBD4}O!Q&bG0WlNmDU@Mvv%Jt5;1czq9ZL^H#x*c#IKYk9|djL7v)dGeg*1YqN0pL9f#g_5_`uY0|V_w)ypEN6r@tthxBm zbPe2#^zhpCgbVfFKK>ZGduE~H9Z%S`W5pZkQjA%k93_N{c+%# zxb44ks+kbmQNu;pVd|7pvShQU+SORKM$=RV`!+xJ=hdKuWS^gf2aBpoIDr(OW_^mBV zi)s|vDx$*At9&r5W@aMbNY9{dX=*zJEmw7K?y^dGff;XFr9Zo2Z7aagZ*NhGq8wK@ z4_u%p_5PB*JOesG`O2)gEh-hZ2W&;Dk?m8i^V;t5&zt&rSm8?@p{vl&8TIDeO3E;- z@Wwm$ra+q43YFdyLON7G_0*jP#6)&x;mUH|y(Ui4CsnQ;Bj8R>)CpLMHVrnqB1es3 zAmvXakyTw%2fm~9C6bImPJ|x*EM1-9SUvl!l^hA8SB~n4(Fm8ulk$F`gsinmA~x9b za|eq5UeZ4S|E}mYCpwre`0XYW(330H0aa30#nlO=G;RA!iCUWNziVE`+0b1(%cze- z0hd^I2Q40kWe3|vsq6MK{*`Lolv^^z9rBImR%y7}mBl7@lKmFa!1r>r&aRm0PgH!O zfbbwmuiU&dKhCe>YC7AB)Vx2+ct4k1gv!a_^EAoFAz~&Qwukd;A9>DFpVA$Wt}g6$ zSer1vQ%2H~zGKckJz34knaQK|3kjZ+Sz>gNLYmQj_4b!Fm0C`E{_N@us5MKoe=&b$ z0TAbJPhFLtI}{pyqWmb z#(wr*AxE~K&Bc#0!v8(AugljJR6R*rffYa?YfnwE&}55dZKT-~f4WdOt&`o-MLez> zCcXd;BHD6o=&2I91Eq*?R>PR`t|)5KueR~yXx^}|9Bh$Q{s$hzUgBm!(PzhU6VQ}a zOVgt9Szi`Rn4ApY&2km;iQFT=jU2K0 z+;|pVSPm;OaLU09-1*%nOB~SvBbB&xkwVF%+|YKT?UWW26yGMDR1{!$}$Hofs+q3v7}WnK(mZ-NSbO zdm|F1l}&Ll0<8ob5%9wKipW?zZ##<1ah4=k{)a;THX5C^mx|}~YDQt}q)6JC!KJ-} z^Gm|3&?C=a>hO~U=4%X|tDxrc=uC7(u(IJL-=kQAU|hLjj)B-~pn?SN-6PDJh0Ymn3`sCCp?>(4M1?m^LDl)m)NSabhtZ_x8 zJ-f>AF)^1w`4qjP#+?2x$z?fv%iUj^x96|YvP7#cJw$e~8*sTA@a>582Z?Yvc#CpM z4Jad?qIY*()sMm2vCKdEM35B_6z=+;^+Dm*w_gWKOv0BV z;c%6cU~<_vv9rAJ?+c2bAJ4Bir1+e{+@`|sCQLrYz&6L9=t(r2y}&-6-Xp*{)H@=C zf3|i}oR6h{Q4~Rqt!A1IWa(LDX0%^?jE4`&(K z<@(G_gN4V}l$*kDJ!dwx5I{m(#c{8$godw98ow=Wc#hvl<)G!)Rt2Kk(G*>#u=Y}i&c{8);48byir7V6TV zyLioJKK!D^3;HT{mC2Lm@XiU~L5}>8jyUU+=RCc8#z;E9J7QvGAU)IXcE~O=jwyds zy$)09!f7Y*&Zo$6hN;vz^y&=3Ra9J=e^;wXALpg5bTGfcHHE3u`H#FAz?JeI=P z+w8!5B~#%G;fh;hCBaGi^V9j?5`p_zm0o6+Ffbi$@O$y}44eoUiIs6;)DBVaSrBDs}prF@NsMDzAO z(V*CK)-8NPi_)VmGx~ul#~LS7h}7Mt>YXsY8nx}S!m%U5Zv+#76uBN0k-sSc^1Hs&MtQD zKP#yj^KS9uET;MCV_rQm7oJz!oMp~%AN!naRw{3F^N5cHZ}o*r#`Ecd7bT~#Rn{Q~ zfup3ucMg|<^;e!GXEGG!b#oc|=0(Cfbh1l$pQ6%cf4PzgWCa&K6#2~9^(-Yda`t$I zqHxUCjVF0oWJ$!{o@Q24^56xuN6pxrMBt|EwITQpP$ixyC;RlOZphDZE~i5|k@yz! zcH%ojArv|Rc4gt;(LZQ{`piXv!Xtl@Je7kgzyWpO#!|K&9JgL2m->EQ^rQE{^ir%45iUZr><#6RW}lE~G3~?SB_%?O3L}rhtC@Dt z=&FmO1!=ZKG?LN6+a7B`85yzgQ{F=I?ZfwJ^FkMo&5fQX+Ep78+mBOUEW@NPT^{@5 zqiOy2M^w39=X7S6fxTsf?1h)7zyp(MnstVa{hJdRIVmH@E6h)q=m}|KCROlg?Ww0) z6@fVM0qDWruiy^XlF~i9?=~5@qoQ0(cNjBqg)oOStB2k)i76Jp%grH=4HE}j6vfjY z{2$O!U~x{X4)g1v^o+6w-Gz{aQ9_)u>xGHj~`#hc|}K5<*;N-ied#z)!DT zRm+=dU#;oPE=l>o{q}m5MiZTa}NjnrXFa z+zsLV{_t1<;jIxCKOoL*Y|m8~hj}XuiXWF~ah^TjY`HAT0`jcu*Y%6hvc&A<*B0=) z$zbh>`NL}RngSm8ciD-)C3zfRYwi2H$OtslM0U59Gs^|au z(*F_9{@9S;w%dg6$LUa1oOH7l)Z0L7HOMHKq8s&e5+MbyJ25rMjul%;{dSx>#Vqxv zdQO>jC69SYOUBE3W^1F)B^L0nsOhc~vj1RX!v z#*w2}PV6^U5g9c(LojTPgPDYP##;yG7xLdlN>iM4_lcGVa+KY+$ChO0xdj(dJP0i4hI5@TY&VH)egpPFOTBRm}hQ1A^3?_a_UYskrckbl|9`Q=k z^6OBNOXAb$-PA~_>D4{;7=+dvT1ZoDE@Lr~owr>aLJ5AdDFyT&nKVo1UB0*|s-16a z621NA;LN7wRm=Rj6@vT2*#7>uwCm*nC};dDOw5TW?oCPIT7aPe$z@cSS$VcTdg)*t z2Ir5ZaP8PEv*`3spBg)tU3@tilZfq5Yy4UAFy$gaGS!~wCnN|;5=%6=o&c-M zSl9phN!jzo%qYbE_%zTA!6*8@vCN^(cXRs4OgJFLlIIok&$C}GK%sSmZuONN9lEUH zgVAf659jlpYnJoTP&XJZ9p~MBg|XSU{N6Z(yfLxfdb!~YKF~GNhBZ2?hGS)g2a=n*fZ*{;c(^SL;qT6_U61$hwI%04c^royd9<+C=9U#lnuQYlHO8J zZ7lKQeuTr)?XV6}TslSy@8Y0g649CpOGRGhz51-!xZ zh^>H5F(ZXfqLN!O^8Aa@md5tdBAfX6zGtJ+5z-^5(>X^yq!L$$J^NFCeqlL6aVa`%xo)l!C z=rEvIinuS&-$i_7sy9tJ{Tsb7R$`;LX8!zD>0a7VrPZMtb2FF4mBP0v<0K2QTtOV$ z5EBE%H{*nsx%A^3?I*5X77}|W;#X3wtj8QV^By7I5xR~h|G5j=-#BnieN1OL;R2Si zQ*Cj%6Cyl5X4gmLTQRVs_nLd#Kjv^NMYsniqwfogeI8$mfrHVMhv!_Lluc07wg(-8 zH}y#;-|gfe-iT4|Z3vp_bCtJx0`{UkGN5s;KdaJ+tPoO(19n_m1ld6^dFF+4YfIt0 zUFTYsbrj~KSLt(pAx%g1oM>r6Xi*qB#$mt01@kyDGh?wayVQMDNJ+?K-X^`ibGe^+ z8A}5ZdRkK3IC(0v>^bINiC$jb9k0E*JSzgJaJ8S2xLl7Dm|>lmxy)4NDzuksAqn%9 zC}_xjm9y8QwcQ~P-a~0!0IZUWPbb)xSPD8`abx$!|Bl_Wx0NWqSes4f*HHsjrA~Sh<+a)iT@MBs_ZNJn)s|b*)xYZAun;x z8P}draSHv-%P@W0MJAeqVR{Q5g}AzA@Q0gQPizugNk18-x|^j_PUq2CObRqMpf6Gq z?O_tJuI1Cx`yx_H$>$ca%-MnrD$ItuxK2V5vt%~m`0s`R62@f?^nA-Yb0_0yRnugCq7UpiZx}gTZV5Z2p5QGh)ozHH(UvpGt(}2C&4E1+%&Qqpx;fLb$Lx(vb=kY8ySbk;L z^op61O0*zf-3Tt*6A=69Dz8Ap)&YbUIpU9y0n7bdkUF>}V2H@qjnz$ny!_Xz=azHGu}dTd|Xmt-Sn) z+9PP~56?t^7x0VHhPZKNjeUJ*?zYA~&HhJITimNHou^Yrg>esjz9v;uPTgTN+O)-!68*A} zbI8sJ-n>Y7G&tDJbVgP{9pcN*E3)X}U<4$6oC_Wf;W9nk1WiA*E{{U_vg{Jsl~ygx zo*Fh@ZY;cUCGPLTZc8F`ISno{8Vlu zn@QBCmxei-$zQtPsx(c~=O=(p(RiS?YgLT|WK6p?gnEe9AG zc@SPSh$96Kz&w(`v})QpbV{Z_c=Yg2H;>g$qfMQk%g%GOU$K+Bi%)UXoUvUV)PmP! z%iL3|^DKR$lh}B=ronf9-FWyLc<^&%(Op>do_TIc||F(U(+N9BIUg@1>&-Nw} zHZ6x>kH0DcaUd=k{9Eu=K5K)s(TOOt$@uFp3TLg%Pl zy{%n>a~>Z4OHZ!9tGywwsQq$hey4iK{i=amZ|V(-n?~|UIMiyVx$&9!RI2Tb1fLk* zmN9m5!%v4E%VrACL=(V2@P2x(q!~FLeVOO6y7FK0I=o9!Sy~Tp5z{1f*f?t5{|#an zE~9B$L|TC<)MOGihD#;m^^i!sz8j3p z?mrg5mcy=JeMCy3E%-YK(gD=c*r}g@z&9qFaYGR}t zVNZwxQjCH8~zo;Z2?VO0-R{;HT$@$_?|>)LDB%qvoXi zGw7gg<(`*00qH-Zf3cSQ8%`bL@Y8NWS&x#Xhf?8DjrLl5-JT=VvKes_gr|)M**n|W z<2hiPSx+M*Q#HjK7b3J(4ShK^KXi!h(3$@%=#IF37xkOu3(@DtW5})3624X&fD_Kg3Ng08e^VB8pvG zJ?zPdt|di>@A&_Wg(Q`OFzuXso~xf7bV7fG6?j+0@ze{7Jy5Lu<+8058nPj99vlT*w;JZ zc-n&5;&8pq1PhX@ zR`1|ujG>rfvwPZYn^I0cDx^YD6V5u!KJPTZIec93G!dgzPweB@2 z-O;>v90#vpZCc4~i8<3{HRz(_EBPgAo=6P5wFs$SaJ#v+KcPBAQbol9BStNaf8q+# zP(3;!bwRYcLw3p{mf5vJH}i94QKihNH1o4RhBzIcc&^>KbTmuAoa(TY3#( z*o;W^OYqcK3acMu?hlGb1hwZE6!fQll!s<{MEx|Gl-B7kw?Dx6kSI_!%N^9bW>M50 z>Tv`2Lovz|Q|9s#Ob1WHeypHi`FB{CL9vj44L!s#hpv8gl*PQp@dk;+)m&{2ndZ`0 z=$7@>*`5=Ee!G5>r}mQR?2^94DZu!M-J~##+V-H>1v8!XRnoYcsNq_nFS=F1H&R)Jy8ip)V$N1DJ4G=b7m|Z z(BbfmtDZTaX+TVDY=IvjCX6FHbLSUPv8;ngU<21*N#3T=GMw|Sw1VQF%o6^yp>5iB zoXN$NmCRNShnaEW<|+VvUihs3$BqBXi}-_^t#hZpiy2z<%zD*Jaz6txKi^z@NOQsS z*WWzCf?-AdQc2y4Z$Z{NoMgzg`~r1pWw5h`a&@gN?;KAVowhCe&h3rW6DXFk^dgG{(ms+rox^dzeul+?H4NnxTJ%-16FzC zUnjm!1h~8Y&r#j~jrQ;)o~nH*2oOqKO5l!jgUoIY;50%0z8L)nHF6p)BlymvNnqU~ zjsv;uD(0cH*WBFv!XJa|oE#l}x>+(+vhr{GUC1y2=)Q%8oGz%3&-?9Y5#duoh_{up zafi;!uX)!X%F4=*M1+Mu?DrO*4yTT!FTOpb1&a!&Yp}*? zAYa;U9bXIgXW)4 zf09&^l(=Bn`tdl;Z{c&SoL+mase$@~7X4Ev71_5$5~O4_*0#4v_1#3(!s6+cGkVw| zufs$J|^ta_I1+` z4EOX20~|})j@52#FQn5K!lSc_I@nl2+x)fXfPyK4iTIsR%cCxACWaE3vf5{1+|;-v z=kEH6P&U1BUNmN@Z{zW|lfTa;f<7=;G)rrW#wqlfLHSc5olN>MF6I6bU7p}=j?#ZS zfTvpYE4nLv=9ysn9G;kO%EI^ENW{CCOKW`VIbT5jws;Z;IwZ=3Pn_55qpDUO%bUMf z_7r;OO@}(5?nzlS_iEZp=l%Oih9>E*LY_}81C4N2muX^vs9^BkgU)rNyRpu{ksqd~ zEb;f?T%4K|eN!1FoUfHb*6+f{xal{w|E_0c9>l3743cwuig%yio$kC#`YlR9i2JXj zj3-MicFwBlJkF<_?hL6a5IML#t{=AY@AQS(CW7SOG!J6pqqw8eogD>LMEslZiIst| zAySU{?{eUl@HTj>u2%Pl#^1m9@&;)|5r+`dU+?<=4eh^8;{Thqf1Z&4U&-3;Rizim zRENxt5_JE*BC%~`3{&J=COqL6GFt$L7^>W9SNE!nV&8lRUT!X{NKULk^5SVifzam& zrtn*xuB!tAu<1%wP0i`$t?eT$SC1-NVo!!JPzUPt7Z84A!du$uPH&wTY=2*2$*+r5 zJm(N%{;RS+jlQ|m&C^wxEAaPKDIc#pAxrKP)!#2^&cdMV$)9~c`LC%`1qMZ#*e&wA z1AlAA_7;zv8MuKc%=a(q|13WLdj9Z+>+8a&HcEeQ%s_dkJHaU0TG8@v-EyvzsF1`n zON7V?{_AD_`IAl+;0DaWoR1R!>S)MB0#E88c{<5oyNq!t0XIN_($h}+%R>kL^$d7Y znxBUM?%T-llYY^T4tbW=OL~lf(GR@4b+7;Y1UvAaRZTsHnzgXRY9PJ96FpNP8+5xs zr&!-oWSsPG&-BaEO#$%=>%mN0tD$V8vUWq4&ifj3^&T1*rw0RXfR4CjzLf)OIRTga z7G2T&0T&0lO9cQTQ%ECSD8kSroAdG*MI`7rD=O&xOJ=MkAfSCmJyZ}1)QJco92<&l z5!Gmc<&3Z2^#OBFtjVyQhNyM107{rQgPmUWe7FQOVdZS6g@}69|Keck#ky^#-T0S; zuK+OfTN_W+vp+wji4U!=lb_DCW=EutX5Ci0Xq=+e-ebjSOQBokVe6B{8MbrvReJs> zk#Z{BN z61D!@|YwwMH%%B9oLou;7Bx=3H?*IcQj ze-!i=qJh!bsAL`eud{a*g^U-P{RUiMbk@}fM+xlsb1f=qfznw|d%>=R_dQnqsb&S5 zS?Le;WGBb6cYogFO$GYyd}YwIzk9G%Ih5XXIMX)^Trq;F{+=sCufk0Q2HqPkLxxPr zoJGLX=y%_qoz8UWqKe3b)3P^dYl^S4jD1$45cJ%F_xh|os`--?%Us8M?f#m=ls=+D zMjQop_@AXkkg>1)@EeV>;gES^xxY4|Dxxj^Ti&~)T9KVmK_C%TpH`M>lAVzGCTI$x zfl-wz>5mJE=`=BixN>#7%+eeH#vzlS%u~x4AYT{a3=f?$ko7Rc< zj`jJ?*cPVUA!EzuopkzrIS!8NOyzwj2o~6CkRC?wU;EMWrPV*LDF1cdsr6Bbp8_+X zYx3A*WBd-&j}!yPx_9Irhg;JtK}1sBk4g)3{AcWeP64;*_Av;j7J?;(_Yd+R*cGd* zizA}joj@#T#>dAthh81bvg?YbvdD(4q;(LwWU7@F8#Q>2&T(e_e%!AW-*$ELHXk#M z2xmAoLw42A06`+{iTijKXZS4Y0XOM#sltB^>Yq=Nb7baYw?L9UH*vg2Lcnux#=np# zyr={?;?4J%0>cRD-ZY=@_3h^;v_v%9jpj#){d$YTH1@jEeD_z|Gz-xvX+jK+@0`cf zfDfXK;8&m^Rb8BVHNg5$eXc3iDdaFCBxGA@HPAn4;FJ~V+jz2)AxIdhlym|Vd%U?k zTewP>drTbsK=YoW%*}7CfoX!)>A<8j9KsP*cGmS?J0Q_Kh_30H33E+{Ocfv{hFgcC zz0iY=`=YHDWN$wIQP7ha;%k^2Ec-qbXy++Wnc3GahRVJW#PQ_T|IzJxj)AR&(m3(~ zq-wt|2d74iQswdogB7$2kP}oTuFG@@v%NEn_CK2Q8UTpWiPda1Il#u$FTlVwx4JmL zm%XdoENM`O#nM@fop6fGd!tPGe$>qS^ci|iTV8$>z~Y-neD5j9aMZWTdhmOhkdeA6 zAodABkTbYpIw?|Ib*J!>LPZu#)X%mq0`VB_P8 z5!r#s_#Yn)j%l14&BE6PUyHt{2Wa>V`n=C)Y7Z3NDT~EV>Dv{xE_rG0}V8RYd47?U?S}(~i z8nJ8&xVlWp*a6x9;k|4ldxUi2slRdZ z!e~I3C{>y}*hZiHhgQG4_DsE-LXm2Sm7SF|xr=+jB2LnrxkNddQhWD$;9T?L`+ph| z2hRWU(z=y&t3(aq2=`Yg zJmsm!x<Z8jgCRA$ z9t$|lg9QUF&t;lJLvd)W_OF?~?SDM^ih0#dcUJk*X&tb0y7CHK0V>DCQP*zA;^03@ z$7vE4bITboHIcTCopsx3Ix=4RRN8WRsz)%~ zT_tN_y))k=k9D7VzV?Kh{7+N$<=M^1F2v8bApls`IKTbIMz_Pq=S{#y5i38^P$GJe zSS=Ib7r?K$MOXzqFp<#&Nx<;C0l2W+v8~_lnTk;InaXTgi9!wmZUl*bg*ippL=~TLnz4J*5m;p*5*|-vxF^nuxa4 zY&`&2@?`ob2#)hZvFJ_;lW6w`NZ%Fl@6QW0ySU~sW2TEjL4pp!m*SGdqB}N2)^5^^ zji0>~PKA@wz<{krk8c-hWIV7Ll?KewSIc$5q~&ntSiUe?*Kpg}T9?y}Um~&Qv2ocn zD%cLHsk2XEAZpQS)M%Z@(p;%Y%TsG{SO-I8)tiaC@7=b*IKS6fnT+XW65%TgwT&N}(o(jdG; z8Qty*d?;e}Q85DppohaRz9__Yzu-{+Ob=NK{ig7YO&)`Jazz+bto^Zq7=lBDBq9G! z<*j=UQr!l{V6*_6&#b7=V>*@JqF<51h+JWvZJGq1QYKM4b9yx=z@PXlT@5$k&T}ov z$2<|Tc9}K-UId-2T^Qd!?wSsz6_hhUOnDJY zO{iE2IWOfTp((zpyL+PeZ_;}>`hc2P(E0CX9f6(|0o<{462dm%NFTQR)txdF{9$WC`?cb-jc-NifV8kDaK&>HP2mJAxr}Luq@k-3gT>!=AEIvXmGKOf zuyl#F)eV^d=Bz(i*Vq=B+2fxL&TBc_mI%=PdNLr4_&&bFBCTWMcfiu~XS@F~lI&Fh z_4K-Zgv^{%%f%=~%4rOi#~=gfE1f1_`DDoocwTD7zr9jMad2-oP@bKXB=aJ@Y?+K* z6F(pC=VW>G7w24ydk%b^BpSgsaQayNB`Rd}7mLj2W&xr%OrYqhyAxkKwIc7y2c7f(2LYPQu#xZEUTubHNT==(bgkE|@ZpLbAcTG0lE2*<9zyT{Yb zi4a|7r9uUlaU-u8maP>q`plddr^UE1>i3;d9WX5BsJZ(c9Wh zW=Fac#mCggXHRmJM_C6A+m#Kei~nr<&TmFOVxq~kA%H37Ayo#g-jT*LXbu1?>I?~+^e@^S}yqDXg3=3~-HwBR70Muu<< zxBt=h98S}GnL<*NavUgOW(O?+BwBWX&8?&7xT2Q10O+ zmV{T?O$ju7HEC&{2Ac5<08{_mite7V|H-n<9)m4a{7Nbq;%Z~emEY%wT^V4WvQxYh zn6GKbkJy05}9{tH4&F|uV-CFsqx ziUj=Y@SnsDv{|;V12|~3>t}P)e8hJfDb49Vg-kjZaw%*~73mIr1VmXP$iYe=+s?qg zT#+yQb-|CilHaC?Owrt}*o{^+Xi=3nOkI+WvX)GY>GlM^pieFM5N=_48;6y<>0t%B z3DZs_eRBv==2vcCR9yXNjF>7}rw{OG|8-6u0bSzVUIz(kR#L&pHB~ct)1qyf0bn$n z0M*EXsPQNPJIiU0cC0<^haCUYuj{jQZl?vGS^(FdrYJ6ktO2mZpAc`%?ga9-oD1ZU zQP|=OO^#(*+SsBLl+2mFX$Lix`OSWeG2{e7xu!ZG=du{gTHf^{trCPBWdyxON*R(|50@eD%IjB+1k9f3N^tZ@X&< zq<+o`fP=M<@UNx$$1HNJ&g3#Oj6v!FzmlN-Jmd}6ZeN|v2-8@lwN`+38pNB)Eb$UQ z#QJ?8W1UW2bNf;%qC zHS<6qWn!Z)!M~#wW>b*`U#!63qZHmTAlecySGblC?xKT-YhZYp>_=BYE2@|my&2ec z%cbM91m&FYnq~{8i>4e0GAbjRE{XuwVf>LnmoAxTg?01plGmdg%GfN9InnkuFc6$| zvxt@_jI^p!t#o+OFvX1muGmGe8pTuJZ3s+tc>~gL)btr>vL)c^RiTv6u25IL zM7fdPtJu6Or7ThC+NUscRwZP$Mt!VBM52qn-@ydIkZsx@-HZc-F?W zZQDGiZ>n*X#eh?#opF1VRY60E>e-D->!DcvtO?hpI~9-jtrA{6>@dm9qhoi?+;*s5 z9%PlQU$cP4#?0;mwDu@Eh7pIgC0S>GB`T7SE@0<=@IBxV6tyDFtpURZ8&p~ zWo7o%P9k%AoPhXCW{>f?)OYqbhuzF^>hFaLg%ol`Y)4r<4F<0VS#(hytpNcM=`?() z+S0^jlyM*Vs?ob#Eqg49W2shF^g=zNdzCt(@^@euuOFe7h!)Sj)$OA|r44rpO0P%P zcG;5Wvdbqi<_XwMe*ZE7;Z;Tk-;Nfw;2X~0c~|TEL-P-csPNI1nB$3Ug8Ev4R*s|a zockF3^%{osF`Z?GLv@-h5Oj}kFuYH?PH6&JP97E#^>Rs~2NF)7HY@d1UYLz?xTWj> zI+o>PDw85SMy~7(1j=%WB;2tUgsVEQ+_@7G z<__KeAViE$AJR<=TJ?8(hsG<*>^$zJ9Og;HQWopd+G;110j7>1GtK20;Z_dMZZ`>o z5s8)PTxFHI`* z>UA!n+3yU_BC0T~8zVr=NGIVLt)HbI%Zxj)$ie*Wz3ffFX&_U}oCUYfDD`(LxX@iW z09GLlmIk1^&*&0U-@8P9%~fh6W7@Se{1}gbr!e*PivwIb_ov7VtK zisU-o6y@Sh3dBcfe`A8v&t<-)u9Q;d@z+i3RgNsK-3V{oRd@2{FJ8M27cZ=I#vl|;~<4=v65befby+?FKsrwh&* zYg>p#>h<|td}ZTjxE$2gq97OY5GwKCaktL*O7_I(}N_iqXD z5fW3{G(%YuB5z*C(UuC_Keh?Cu-Jp#kVdxt@NkK z=5y^aCbDTfnvy*^-+TOKyqoyKAek!z4_k(E1RNna(v)h-p}<;xsmqpAbV$2P()~BV za+2lhtgl@L%sN;;*xg`q`CT0oEvX&7SYMnoxTL23vA3F+=expahk3|VHQCujgT=@4NubE*>*P5a$k{dJ`G zWYB{9H_DU#%fzGjo8AdM6gkMVa7#_7OmAeT-4`-Z(sL+WcJ=>`H!gk8@*F zA<1;hqJYU~){kSVrpdcHgxY+iV@@?*_8UJR%T-O6CPi$Hb~GOKv6ZRwtC8X{EC8un z`{pdqu1Zu8`J(UE(QcKj3aS_S*`19(OGV=5q$iEVuh&}og)nn(M&qqpGV%Z1C3+Xv&) zmXozhN@W`I4Z0)7k|u<%i^T#N83$*}^-;ubxSF4p6*$T2|6xqxO#KMkEN=r&Q8uj`g5ec~b zfFCj$d9>xDHTMky(?{;x6D(sf5d=Ouy_SvVE1%c1zo>+T6FH3MDDOhl+|1Bc%rxHd zh3H%W6J;5kXnvZN4QN4>rC}-}du9O0Y{=OiL?Mk&*LORn$fX?1A8!G47S`)F(lDX@ z`JYHFVukzNVdlgYz=ST+;u^%D2WCt>| zOrft8G;|7NJ#f9-9+~b(UXcZ<|=UBg$)DH?-~~?pXyV#DV0=V*S) za9)bh3+=g#Z*#|@HIGdof#9*v4~q`WAw$79)M-7TgQwoWGN%BLOC2sKF8Gf{CL#Sj zmPPAukEv>RZYs&S{!ABfFK)BOHZLE2ruM?gO^rB?P^PsFkIApl}nf;YcGjv__&x%C{HT&cXr za{it7|N9@m0$!+r$|16bK;a!T+P5!F;kEhgo65V%762^f)Y&`%hNSRyQ#dPa)@Nh7 z1J6U-Zdq0Qf?kUXRp;p{l_O^`>1CNq8`C8by6Q75`2hO*%1_g8Jy#|9t>dsMg(ra8 zaD+Y{>G1fQB#KV73lnIHc7yrLD#`|#oLgMk&Aj^}rL?@9yy9yF(CD&sd3itkRIT9= znfATEhfo>aqqUlqtSE6_X>+*C(Bw!|mVLpK>y(Dx@`RKg(pR4#@# z=A7Yf?P1tT`RPNNrmnI(p22bnBJBw4k$CkkG!qpH(CQ5~by< zU>e_-9n#QmkM4k+(ZvvI7h(lu(RV_4H3X~d%@^zBXBy)q$oi)8=toM!X!+mrWZA~- zD@SD?+P+_c;WKGSm4xe?DLKBopF;Y5AJ4so0)Q#eZmw_=ZL4nB0U# zZ~o=7@qfn45c*OjN*G8N=%TnEuctW35{(pr07#CGfn-xSs)M1=F?k7ha4KYB9>5Sx zE7~B9X1%dJ`~WieR@T6hUiN;XwYH_r>XWaw$id4!tEWgg9F0`5`?CsLOosF~{I%e$=Y?!Omu@bS_c+ZsF?tK6x)C@7ecPFPNsErj z@JsRy$P1lCSnPT!8qiUa3{4lQdPRHPxya%rHL4tu5M#WS@6 z%mO920!snSxd_|aw3>S)egQLE1aNS}r$*V5i=-gkiCzHC4H{EITRdmiEEk#x|9!4i zmvT>sxIr1kKO+@%0c00c{G%N}UrGfm;KF0MRHC2x0^)-!rk5q_p$4Jo-Wn+vdy@8& z)T>xpy-QyAKBq}a7du?_~j&NH6OKKhXTdFf-4j2aObnUv6e$k1j)WyLxCwUc9Fh=?lp) zbRK*z^;R`qTuh+vYlMCMRt_)9`L7~9L2bV-K>E1e-wyOrlQz=4hwi%x=3|}K@8PJH z+*8bDzL3d6#%8-O2lY~Q-8%8) z%oD+yPFWqtIe)MgkjS^mU>dM*FxqjzuxC=qDana2oUWuGnjw^)y5%dZB39>6JT zK&d;w3W)1`tJ(BFIzs8Zd!nDM14qA*INWDA6k*c2aQy+ZVezZNWtuC-2z0|!Mun2BsV@qLWfo{tZkCmp+MoBuC>!>W(#)0vzd`liDXKA~F?hkx5gvq1 zPK4D=Qe$*}IXdB|n+cAo3FSjH{Up-+$Y|o3(}n5sDHN?6axlx70>(cwv8Xt_2ri_H9b@5g1MbFw`U?A8H5 zg&#}$F>x_D?R@BDZjC&n-aQr+p|fUkVz{I)cj~fCvRSkoiDY26foeg{e@O}!uoqP&uolAS~miqfsjEx)Pk}iH=99H zvGfWDAp3?|S5ScCt+dbxglHKQF+QH7B&|YAjlYp`+8>}ngzg0;m{&a*hJ!!H7XKb6 z+z~~As3nC?a0hm2_|pESbfLV0Ob3L>+1Qe1i*+Qbxny;tOF4GK_JL(I_UoP zgceE<-FJ$YJl~oH-6I#ckMyXnME5Gk2Y}LoB>Atp&e5zJ23)z3tUA5e4bY{ru?GU< z$q2G?Uy|6}7==(hN z7n=C`;E8d$e|&Ol_vqj57ZqWa5K#tv{x8@RAZ~$2k@EG2#50a7+#@>B!9<=8Zd@Zq zI^HXw{vFI~g0iINok~ZwAZBa>zEtT{Sg7@~{&cD(Xd5zm7I9s>-wtm3@GxKAT&k4OOn})G(O>v_wbA1^-}g4P^&JQ!Uv&tGDfxyfY)-<0c zGA$3PJa(Pav#Zh^36X)zi+4SjibA=~M4M;0d#MOw;Bfxt!7!gb_D7~BJ2^5bCG68= zm)%H)B=^WPQ0q`x)vfHBNAsf<=$S@5$Ja;{^LE>PZw4X}1klUQ3-grdfFTlXj-h+- zOUp|4Mc*@nT`zz0 z>M5kHtYU~+z&YE;wJR+6?Tdr7pd4_WGj*2Uej<|))&4kh|5u)f=lr_A<)mE&y6WnH zIrJZFg94=a#F%qX9d>DH_oTkfoqV#L^EPPitY`V6Ksv#UbVG zR7&P&2|)|O8NI@)oB=?~9=ihMj-Q{2(XrLHMi2IX^CtND&MhH;8Xp%eS!->&0QWlQLXY||sw|JY!X&S(Uv zy#|J6@g)HHXq2GmXN1IEss(A9x6yFSdfJobbpUO_C`6pcM z?^f53dya1bTxW{sj}XYeP7yB{XaQa^gRl&qP)87C6_iM^~ z_A%vfeWVUR&7}DabOMrqCo4Cm4UouIg^T`g4E5g^0YdJ-F)dng%N4ud^e-TIiQFb1 zbV`jSlU^>b+QZ7fTLZg2$yN97%lRMwxa|Hev=?Sl{%PStUVKHR=ZIzCX81#H`}`FTrqVQuz}GAvDfRyOh5P&2vwj7Zj<8?P`;V>ie=kW-EbuiQWQ~86 zPvQoIF#xZ~&!fqOKbFw{{;>R>q5ics?w_?y8Sn0&S6O3~;6J@0xB=|I*HZugviZ<#oMs*`$4a zi|P^;FJZbT`5*pE#x^d7tC?zYMcy{Ww@7K!u;};uWwfF!tp7}P`8+t=0P5xY_{8vTr!u&obn2S08S>W*R zvVq`!X}=H@b^MX>`maP3(5H|gYtHU5NcfhUyjXL#QokWGzwFw{V!JVYaC!@KV1>b7 zv#UcbNhU)bH}Up{D@Q(1*Szm?Sfa~^@9u4d-;HK};W#p0o(<|fDgW~9zyavXy-PHz zv{KU)k*56X7|l+A44e}M0Adp#K-GT^^vN8!Ta+GWrvB?TUm!j55ikw7n7{l{J)Pl) z2kQauExcSlC(8M2 zm(P%YDj2(z%!gH3nfINJJ17o3bH@^lN~@00DX&u!Et(5L_ zCY+7G+kd!e3sc>{SIol5pek>~uJ!%QicGeY$A4>3?2M+gOx`Mhc;a}U3RS?Om zVZ`y(K|+F|SH2T<_&qBTSHZZgL~-EeKuO@(|CY|hcdng`jotFejE%2vV5j6*y}63< zWK>I%Gc`FSs;{eE(>gC#)MFZC%}+1BL(B+5Dc3p?s$C|j$*AgdVFQSKD?`*HI&KI; zRT9l;_y(qbG?BX=X)2N&l{+9f)gsENE#6z4>(K4gzg9zS$ASFwRfhY)g3~)Wj5d8_UEp7!-TMkAtEY5bsrREQc;BXMm*CFYv~t(gjhasr6!i&^w>V2&_+xMgq$Vd!T) zQDbpk_(}4lrvHm23Q7xAE#S7;tvvaoQf;|q*0gbBPd=#YV~`f>*7VRIrW5-(_~CZ% z^7w$l^RI%6yW%R_lkNOJD#|;Eg&5jz{Y3U3TyFAXrdd6oCz`p)1shH;&k9tjtC*wT zVQFc7ySpbcvbyjKKfT{oc|NGoq@K$^69%GT&ja08`w+fvT|?xg*$;n&9RkC}CR1Cg z0xccN4Gk)cx?MGb1v)27f>fczKA4A3a)%5G*CXK#=|r=_N{EeMFNKC2ThvpbZRwWV z+XR?BJSgY5D zXar&=KJcqRw+=IS(im1FpM01^Xz10&}r3($ib5!;f6 zRtXZ~;IMdK7VCKZ9Q6FuZv5#gJ5xSKYiUG^sU`Vs>@=l=3VVFvcV9IDyXcmz<9Uel zpnn=E0jO7T&v*-mqA08R_Sw1Kyl|B_^=up1Mgo~F<;Egw#u}8#^H&v^Rc)&k9d)A1 z_KUk@ri+#4p)E_C;08xDVT)a|j z*^tTr|IOH(oAP3T@41KMLr#?;SRK6QISMo=Rlp|k2SL**jQx1XT!b)NCC)af(%=Gj zdO_gz5{<0aGra;639U1$x3_06(`y(XBI&BFr8GE}d3FfbUf5RbZ|9uxP!5pEtfMDd z240GnFDl$k`|nKZRZR#sh!k^Ka`J0{Y4EXM^D}BzZ4`(Mg(WUrZzou6o*$?;w9Adv z+Fvx4)CSvEUY(>)<);EI?a{a5X{6V7b}bwBBK1Dc{JB+6PnL9U|-jtAfw_;TQ+o z#sHhK=t*$C>3xk5kjcose>|*IK)L&Qw3)69tE4nj{A*tO!-L!JY&US{CQ6+uQNwFm zL9fv@)6=~(6xXNjd7lai)-*Wn^gNKsXN{|BIQO*9!!e`j(n~GHg!^dnH_4|MkR0?k ztXvvi;Ejb>h-sfSndf=l#8N8RU2$l#3wRD?(v(IWfgQX-l@*Z#E*+zxQcoLv?vw|F z4NIa=Dgj=_lF2OF-9?nb2S0t*E-yB&hHrPZyjP}1K5QlRP7yC=J{q%F@`vyH&w8(a z#}ww`vviwi7>ZZZzS79&XE=lZjorv%SAL)}w5iADfj{{x?zlGv%G?seZn@rreKc>h$i>!z-&A(91~hgi z=Ny#g#{>r|wV?@w&B7`@CCK^XYiwcM(fLac?{dwV3Mf!(joiQ$TA8n|Yw2dp$FJ*O z;}PzQl;wjAYvtIaxRB}zMWlo`n}kJ=fAx03oJVYg*EtfbyO!@E z7!mXE+vt`u8y*~N>CFkzcP=@GTqG3DPZif?>KS~`b+JzPuX7Q*9`dgX9i^rLm;%B( zJ29eu#-Q5)&qwOJpoc%NYjfRlJXhd~MuKs)IK^?ZG}e$L^drbntF)@QbAzYcfy6wPna6 z_j+VNev9bM!s)oRc=Cf=>aLKZh>cm5lwyEkF)fN@)I1J#4k={wn!ELYnC9GBIeLg} zsWw0Ko<1xFw%GPrAqNTA{EZ80*jS7&^j8o^9KkH!;(*&$#@K32a=q)8M&^^79;q5C zWnup82o{K$YMH@7Iu*}+Q8Dj$0+?;Z9)Yx(`AoRR=IKt)LT*Pr%Z4~^kPf8;mRL(s zi3eNal>blF2cb_`O$jXT%;-{&zr4?{TRB?SA&*}yhort}N!BvC`;;7tHo7EcbpBQ& z)%Nf^rbw_`e|w$RPoxx*Y_1VRSMBhqRyLBN#ejg;c5okdgtK(wQg+UUOq_C6(_0_x z)|;Yz>dAILeLs)EVzm_OzXwN!ya;#%wf0s!e5=F$lREtFSI8I zbk)!YjB)y4}so@YJ%_fy%%^ew`lZLdBji}|E2I~K)?DW0OO|~iRwkI!ikCr=I!>DJeFiu)A#sgCBFL^65MvorXtPzimZ$1?YbF=x{T4Gf!=-_4 zBO|TB^Joz^x>2Kd=0lh=Zf&p97(P7PK<6p<<7pL^^v0yG& zewKyk3vU}!wu`ErMl@JH7CMx0{8CU6*GLH-MSu*Q_pA2)qy?H`-QQG}X9G}>Pn zN+T}B#@hR2ZsCQKdk59a$p>+#b_P^T`i#emv)P}zb9Sbb_qf1!Nk>XX2h{0Ch!dB6 zLG@2SrtjyT3^tPZUE{ie(oN^&d|jNb<#v?i*V<}(lKctth#^H=3|@M?hd9WA+M=lj zvRu@Z!|%*~1edjhR^JLbjILye_dhQ;ywSA~D^T5~{yZz^no4V(bjzV{MHJYaXzFIw zWYOikj_7Z4j3U-lyop;t@f$`;m`pb;SG7G}e!G~9ESYC32fz+Tq@7hg?~jH&EFj@6 z*N2v`6E^%->Tl;4Y=`E=WK+`@VD%Q1NxxZcgFdHU^79+VZf+iNcTqVbC5v`b)rdV< zX&z+mknyhK4eTC#loovL*S$av$V~1Ge!e|UMqClI(A-fm>xq~AuKEIw45tq@JH zbkO(HeOra`G?k7#H}uOsGrbLk<|mhTmaODlO~gY7;E~*)`R=WDQiH$H9V~?T3lh9Y zCrSxf&W;#w?rZxL9p&FA2|u6dl~BLQ;PhJgGSY^B3=`M}Z7Qr$z(5BfBTv2%%L5xD zd1}kr!Gf*zEVWT{X_~Gb5iJrF?-XRIXHWLPj{@GTSkDfdDKT-MA9c1% z*~XMQmtKy$)KtYEqMH>BFWo6-d90QoJ0y-rC&6+LCERDsiZaUj4))i^c;<6Egx4jm z?9}^u_e(a?;3yX_y@!ExEGXW5sZIG?iIB*@;ihOZ)!EG7l>Eo}66WV*g#<#F?Xj!+ z+DTN*hPqd$2=u`JjO}=7sYG4>hIiLOZg=0)cy^-u!P+z72Xr=fb;s3Qr)u(k-HTTN z7wy77%ZA)uXPnqOief(57aLgm-L0)E{5OOD9Mvi@XM^a;Aa~ZSFQS+uFkofgi&oaX zWqE$7gL=cW~zCap}4!XXqK zXv?q&RnI8_sPWB*xm%C9%=+43M=(p z!_}SMqVpDL@zo^dWPYgL8=K2WO7Jxq8yB8{1=i_{Az1yX0?v|b>Z;f5r@QapQA`v1 zMs8_JmwXyj=@-o=lWb53S=;@pRA#dUhMx!sPYQ&C=$N|_<{kUTbLWDuX6maO-Udge z+kSt3C;!V+-e`1~#;aybduf(N1Z^l|coB<;vdTc?xUi`P;&$b>r)WfSTUvl)Z;Sqm zo!YXnWw=jR!n@63UGz$&bt~Oo)HK?Z;%IFyWAw`gM|LVO1>iL&O5h57v&H3sF~YO- z<`xh^(O*FGM|rm;P3G{o-rG(R*c$i2ALaW6!C&Mkh~1 zz1{O7an!J(n26(0g~xdPpZF=b3GLWs`yL~L?Uox4Ll73@rwSB>#T5RWc9cI!uUB1km_kgH8jS3!q$ zl=D!1Lg25uIPdntPb?dWMQn*GjXg^0*z>fy>5k}QeRBx$78QT9eSpSa_7*wdll zlPKh3!ZRerN~JRFBhSmN+?0FW|2m++jtGKuZ>aiT?FsTZt-g}FH1dz2)}LA|PF%ky zIaB~cI#K{`73{3b>E6;tP$~$E2To~XQ8Uuigd;ris zCVF)-!Xo+qGOgIm{5$}VpDDqyJDx27Qzg*Zut1CB7*KyFZ;j`s0L*}iMu6@i|5=^Y zfb-&B)wdok^@Il?=GZ%?-T}@K>j3lbab(Gx^+*1zu|~z$_(b^%j2!_k?GXN}l!##%PL|S6BF-ol zhqn=I#G~n0Uc(PuPVG#a`66qszNC|Qt;-~l7koTGnpV3j#*livSMvQp*~?r_i~Yuj zyVNjl{%q)Nz=V|ciwCpO8yqE3g4lLa?U8nq?D2y|Zk|vd`qNev2mj#{5%-2SOE??L z)8673U5{Aflg?1k#Ukw^hYd^@wrvFU)TRhrW3y#%rp1`@-j2cf@>7kw$7Tv{ge9F2 zTXK-MCww6{mPh{QuDf}r*~B^Q&-y!ZgS9?D0KR5U?bE+};Mpr+b9q28xpq}d7p0%p1ZFDm zLyh(P${LL+czva%=Q)g69{Qpm)-JWKMsD|dbe-zT6-@u}pmTjKgfEmcV!BBnxz!

    IEFLzlC1ab+=Y39Ekz+{ln7oRuJion+zCw&cFhh|yR07RC6A-v>)-@hGLH^q>po z{D-^GT>W<W% zS=*>}K2hy#Gf9r!Z1wX9@;h%Oh`G&Z<|iRQaWDWFPJs98cZzGXWvc9fIm6h{-$lAr ztQy6XZqsd0*GZlA&8#Q^kgab6;CO)CeRJgYf@`sQ&PmaEHAXHOP?!$_vVjgGO^aK= z6-WSdYSkzVsDuXrOkIO<5D=-`-IH5elXNZC#vXztoY$59QqeO!G8n3(6aXh$&s19H zpQhN?=fgZVQn80_01|Z)DEAuRlvxS8?}L{d0VZam4+RYNHTcL5Py&4KqdE=jA}Q{# zn$%@uwtZ7yy7t>u|;7i;ZYHg~_wg$~v*!8lB3#TkAfC;{#Tsn4ll zXBbXa_BZ2nIZ0A`4H611r1tWQgQy$Oo6S7+yt8Vt)I4guhoNy~qh5!yEMH)gT3vQ` z98)+HSSOb*Vr8J}jooR6+jA0WuPYVs!db31tgOoJbq~fBhirDW=}VnuEwy!#!a(1y z-4rLaeGY*<2lP2TS^=6<1P)-QkI2}QpBz+Bt%cKlINib< zpSKyG@pHiatu}@B5;yNfDq7Zp)7TB&;101n_6PhC=xJaCi1^A?pDS8g+J5iZtlpqZ z^X>507~_m=NvizzH)W*AkxTN!TSvSn0R~)f`m5ik_4#j)Zc-;|gW6Zo6MpiAz}I!8 zD9M?GhlRpoz;wz(4o7LVD+I}n5)a3SkxRmRdaRB=F68G9PTu7ml`k*@~B zcVBY`5uBeF4?4pp(xklc6LjY_-?9?+Y<%rj?)Z?`HGG{PPLOg&(G>oO-*=%|bxx35 z<;2tu&Xzp4FQyaXZcseDP;2_D{)b}5CIu{`*|*iBqPR%unQ~0YGFTYsD#&~b1n-DQ zzU9+hNli67E#2bX2p_fg7QZ{7ac9q6E1VsB6;Ne2!5+_ICVm8)+~cxW_%t9?9-7m| zbK3daI$zo2(9Za@BK6Sm^Vq1Hb>g~)J%s^Kg6Kc=nopIAk}uwZB{11 zwq3_zmSc)EmaS&ym6N4tR7l_EEdVt}4N2UFMV5c3Ra`9_$Ip5x9~ZPZ=rG-RfPBNX z$g_j!hKy@;OjTzW2G8cP=I%toLHTqkJ~*X>_)JY78ZlH%TAYUEV0r@hKS{T;&KvlZ zScYc}vqlExs&P&*49K+c*HkJ+)~N*v*OQ|x&={CE#^pRWT|{{yppxa3@2J z9MaU%VbGo|HIX_7+_uOlCXDEb0F3@^lT<*7LkI#WN zm+IQyaOo{zobig|a;KzxURWoTZg!6;9%Kd>`S~weJ}ZzSUUsnuqJfRUW!b%Y+M@M6 z5?ZirBA!~L^R&=0pUi&n+A>#vwdkTRF=go1M!zT);8_NSWSfc-(CRJRZT6><5*t_V+Qd&9FQqHcuoIQC8dt^c+YNF&;^ z4P=2#00oow&rH9^=+cEOVxST~ujb`x+FkM?36Poa4@ook8ZR(H>%YK(#q3Wa?T14= zHoN>M3=YsY;?vA}LPmZcR$<3EcAd$k!)qKR0_z#ozIe~`(_3Za20TbbC{OkSLjvPX z2$R}kE+@awhizLmSEz_s7Q0tn?WvNY2WK!#Y|j{}K6s4HJg3pWO}a`}Omf#JRBo-m zWpdg$zGq*6xliWdML9Esu(3}}lWs}CUlNn#vOvyo;`xwwj;2hDj{mi##fYq9&RK-r zF4mkP>JEFLBG%#c20ATHvkA4ggAMMRpAN`hte9OF4KJE^jyw_SR63`0xcf-DT@+TM zLi5`&{3K|JudtmT_ojC9#cG2XPKd}N6xW66_;_Z19NJDAx8KRg5r|$pDNgmK$H({r zOl}svEI})nxA|_@U8F!0c-fhC!90-%gZxc45=9c}s&(k$Nu<&_vFTr)Ksis~o7#~HP>bAP=UY7l!j#s#{fvvGVf6|EjCe(k?c?82T(FsthN z^EM@Wm9N&p@DNSzbUd{}uU}mSJQ;IfEzmv-+gOmx*CP^KGjyD9ovs;w^xOCm$ahlyR$oDhykZOMZZ) zg4doapkjXBm7sVFy1tgk)C+yn1hnQgj_5$|xtdPKXxV%{AkoSY6)<$P$7gA}gT)6h zHYDHBw}1f>xTL;SpenDJa_+rB0FF9nYHg=GQ-VMl?@c5B^5rC*?=m?rzI2%%pu`Ax ziV;hggtdi5ksk+~C~iuu$8#cJfQ6aDP1$0p)qZW`z4_sIp-u&j{AHQM@c>`Ga;S%_ zcsmU6BsBd?kmGlKP7^Dpa&#}0O;+>-Oy-CIOQ(&*dy4$P;=PYR-0sh1dv9Rmhj*$7251R6Ow z`jYbb7ezl%i~Foy&6}?s!8n2^(is#9`cCpR&g}|h*&(;XW_9Zcm@P`?`Sv^3W(%bd zm~4To9Zt4-U~PhETHjLd`n;0C+RrG9>?2Uw4%&C;9P&^BV3m?^xx9C>_YX157u(*E%w$qnueMWV|S=WJ~GaA zQeAznhJ(HPYMq4eVyZ|(U+d+2D@rtRb{r|pOOFE59^;(mAkSFeo94=A3}PJf%Yd4+ zL?3%2pE`8+NXX&r>w0*qAp0p&r*JW`Zd$O9Q1^*4jfS-9!~Jn?c(Q1*qO)_qEj;j- z^eZ(8BEmN1if!N;=le<2Q0d7nFQUo2!Brh^q(qlVSWHy^q1F4F_s-_aQvFi}h~vYJ zakd-j1YZ$qOOLIQZ%(fK$nqq zv3Pe&oXgW4$0C>aMPBTaa2#Y7Xt2$ZP$RG?!diq#^$?< z2N!smHdki{)UN_?N5kooH(j7XR*t#-im0SNuY(8BWmma%`&KnWtlXi?L!_#8H>TZ? z2#t7C25c2ACcSpIo^MZQoH%eIm`!BqK8aZJQ`BCRR&w$tyQ)pu16%P1FlptPW{7yW znF#Doyoe>qt--xA-xf@DS>A<{lbpupIXj6Rn!X#o3cy;cR za0%{|XO3w;)($E4yUlnJ^s{&M6Qpf?6RuZ{&sly&lesDZ*>{Vy+RS>Gqk4W;L0%db zAfP*ya1^3ITm=nC>cH~sSi2}K9SX{T%n!Z$vT^-}*Yr+qg|=`x6&2*;P<=|c*c(6N zNIJX(UFez=TEW_YX!}OlDWola-(h`~ER;nhJO8mPQeoEV`;nRB{)GVVY{?fj%emr> z))zA-#B77DHBViiZtGw7DquL?&ENNU`aM@Wf{^A_VUUgw5ml$3FYzJ5cJ@Jm)Y_M?Lhe4om^ zyY8*@y}a~9c&-&&UMs3CW`N~}FL4t3WH`&u&?9$WHr{8;dYYmW_}IY@ z(6DEfR!s2X;OE8uzAkUIoMNoTiR%!gdT=cgt$uPT%admmN_F--IQ1kzW)2U*rf3(X zfJLH{Cwp&|)0!4m@P@8A4;0V!V1t75z8JI~R&|Xbf%C9NmB-^70{75Do+=U-MD5l4 zC%g zT^&0P5FT{BrGSbU5b517(=va6m`Ar%?Ckb8soOtUXM*dupoqw7kicIh~pnD zYL(5*6mb3WK1(2e=O;0%dER@Li$+MaDsw)axx-`rDuGbGTmmiNkeraSe2h(g0DGZR z3^pkIBfVmnkX5ath?D7zFY+NVM^Ghx@)D>#H0}ebb&wa{thcOBCliZeS@yFPLLKDA zkq$8+H3iB5CrGH!_3w_GUjerQvv|MyU-(Gh;R|h3w1HIh0MA3L=nV&ty{f5XfUw0v zQaQwNR?}hK%IHk*l=<}7*1x@tisbwkHApi+2THyI}z(~_E z>|Vm^C`t!^{}gQqVhLh2FtksSFc-t?APH0jOav#5QjbM1gzc;&=c8KPl4x^3jK^&} zRa+VY)T-!#sE;CDu>@3fyT13jC98V2BR-ACA@Q<^L||jtCzh-^Em#gD#>8^8L2dv$ z=6F`L{*cHV$xr2DBp+oA`mJI_NaMgqQakgo5Inj$u#H088A~n{AJkB{VAZ)a!>CUA zHL*b=X_%o!|GYN_dk-HzAKBTBdu4yu`s5^niv0k8K}4Qo*8TWQpr^xHX=hKpuKusOG$>x9|A%}$U<&m#Y57tlg-fVpV{pSRQ)q~ zJ>l4+uiw$MtIv}V5IxN`?C)}QS`b*a|6sN`lU6H*si&s@%wxpWwnFRmGSDvk-Nq1M zKeUl(P)GVSxQ75VQGqx4rZu$3acGET5-Es2;4{Cg@Is1S(ql_`$^%Yc&9qFT0qV$~ zp#n~+61p)LJ^jB%ZiUTu7|~dWZ$rMit366Oa*24FZJIGDvDaVLb;K4o1AeXtX*}zU z-5gS9+r`QDSfPM5md;9B;jawHDVc`B{dUAmwn~yAgZQDi-9stiKe8ZH8RE@oyR-M{iRm@2x+7`zIaE2iz>in_Jt zN{~xR$5y2EA&~>DYzFNhK7PFp(h#20H3ofgX{rjY0p**J`C$pJQDyh7`8Z}h3K&@G zjX;<9T%%(5i_f#*iij&6i?Hlaax>gu!E$;m1$k7SA+uGn;^s>mf^{hS7o8bo;!ojo zKLG`7V)Z0yzoS-#QQrfjNF*Vg#Sv+MLD(e~KiQ}{W7nl20ahPw(W1RoP-Va3vX8Qu#dR)Hg3us*#4u#+bL&=HL*?yF zlA>STZ|AGYiDo_U{NY$`ceZ%7vq|h;11xIhB(=xnYMrIdDe}PAOY^5t@A#Qq(|iJB z|C{QQz0Ygzu})0#@JMt{_^xqG9ofcm*?0{Ps4T70)IVEQ zVu~wjy#Ob`?Cmx0ie$|Z+hxS_1RXoRmVS^yZ|LqRhcBrCrB~C5%n#f8^lI&OiRzf2 zJwZ$K>kRUh&${u}Yu!On zv&SP~^2s!&?R=>l)CMm3(L8x{vXu{Fz*vcJ+6O@}%Lm=uy$c)DvWV*mBUbXK?95|# zes?#E7@(FA(7d6?@~75|pH67krj8BtL3{#VmVL;1%|bi@cQcfY)q9JiNrRf~&murx zg=);)ASfN|buKG`6veU?b>ez$@j_NGdU>_?OCUYqIibQ3;WQmZE;tk|xZamM(~|-j z(sNMD*P+-N%9o$0U(|l@oU{o8IMbtZfDvNABqTq=)NhH#edv`-tAR^ZOD}NnvLNEn zeJUMI0EvJqXn!mue>%P>FnjUxYw!+=9J_!w0_?nF zwuUgYH^|SD1@t*WKLovyEi8 z>z!~Wu%3C`e$?7_ShRICj{S7_zZ*P^TP`rJ@_aw7Xc}Y9?lxnkJP%YvNCV zUHao=njIDDE7WroSibfWI%&9mlyM5R4=-_nFYjSaIk{WC&muqdl|-H5%lP z8IpsFh?9znEO-fdS+PEi&9YWyA>Y!_9&+uJmQsEW)DMO+uhepaMeOReXhbjxpuKhW zDSOWvP3h=Gv~NYtwx`jl)~xLHenXd`DmE7s6)iZ>c>C-{#N^>Nt?@;V$D(ju$OoGC zICQZg-qol2aP%y-tE`*L3!Ai{SpPmO%-}kertYM|>k_t@3^fX18oyHko!Pc9r!w(wsLN??7i@mpuin?w4e-#ix zKtNg=1XQ|)?oufgkOqOFq`N`7Tj@qn>F#bA8tLxt?tRXEUw2&3^WS^zm;1$D%NNK} zhTqH?M|_X(hs%CePPJidD;J4;bkn%!m@8q#L0#5XuwW=YTRl#QWzAjvMK1aW$p(=x zQ|poJCmdE6QbYrWf)9rBZIS7a1AaV4(a$IjN_j}AdA_mq$>j2Rb1Aufp34AajmN(= zW!H+VdM3yrh~vTYh6ZeXa8z$1lS4(U1F_DIPuEj{Bag7d6l3e}(9F17wuv7jX_CX< z4-ttuzd$1n9e0X5XK0H@Sk@VcLr8eo0&)z<)^EV9`haOqx4*){?fE=rOxFE+()=c@ z^fr`z#-VGOyE9$ur(E(kIj(VPZ_;Oz+qQY^yZ$dN*F(hFc?7N?rY`fpi7EtB^?hGs zcG@0w(><10yn#b5E^%}JRPQ`lz!-$6Zc^f6k zse6(hag%0~WI7BxCY5}MtL0rj3DJl7%Q1U(2R3b;OT}`1cQ?OZ(}^*}HYw^o9 z>$UUTRn-mL<2S5#*d1}K1cHZev5ZqOpMCmOO8mtte*+A8EKpp%vMj*vr%p=&1Zkrv=g+N}q8ST&K(@v%UFs{|?)6-hB{v$$JkJ za#vTn))WCnMRE-;j9Fc3T@V@wVXY*=(+(7Wp4BY~%1oaa zDf;<3+9cTt(xTW82Swle_7C%M+gfA3N4tt1X6~N~LntUJoFG06Fb_l^?^NB#d=CyX;E!@Ox17-Em*s8PpXuKOeVn=Zc^u zSF|UG=R{ejTR(rlJ0>+-=*MaO`N;nSXpQwTI;c?2*;Na=Xg3r| z_kc&$V$7{UV)N(zG^@uPL=D1jxf~MosaB;!%U0?nDffN&S=W>uah!UNsb<38u z7^>}ObQ}aZbJ-Yv!t+s5QDx>a2O={vxw0zmmhe8|uLB_gUFZ0GE1CIY`@k|JdamNx z1UE4uays7CMIx#8RBLNS?q$`0;;?4&VuLZt@Upl1EJ;1UK>!Vzt?gAid1M_LJ_AYy zkCg#+rii$GI=bPDBdiafesvkf3h5gTO0Bois)arzcXJ9l9lv*2cp(%ty85BZ7-Rx{ zfa*iI@ztVwHO3Qhe)awc#{Aftiu+x0wrnEu(54iLl}|P3oueaOTzVL(Qj1svDb3?B z^PiN-dk6NH*9w)G9;(BaK@8#Z0WF`u`<7|+y8x4=kHZ`d98v|VJzPT1+1a`$p$ZkI zM=)P)i2iR=<5D8M4hshR5~aJt3~!e$Z)$~L$I9t(`MD1F@ckN**Qv4jiV4IQ%;kgq zX{tfdwNz%9#P#bn(F$NodPe?!$)_CFIhXkSWDqTy-_|&13fCBW+&kKN0dF7_V~ga@ z>f__;%ksnD62$rXos1>x&Wyw_z82qQD|dXK470;A_XEFLraakD7@K^bo^InbYH1sF{={8j4Ue3*Qa?nxGHApe@<@T7khdVx=ST2(c-c?u~``XsD@A`|-SRIY! zo?A%7HJ`|FFCwuU4W@leq~aMa*6XBbfm@)^GFU@iTh}W}qPk;w64SV~tc45RRaFCd zl>(wc34;t`{AvTGW--9wEI1mwS{^aDKhG%F#ASVYtuY`W@!h{!X!dg;`scTZ;*^ZB z$9HhA))6^AMxzbTK9V)E`Z9V|nLc{>RMv=KEs7lbhJ)M6%tn(+8OL7sGVH7hIYHwX zY$06#o`$5Wxrdb)<%lFP+)p#Z9`o?697F_IlFy%@C!8)9R;0VnrkTr%IXNp^&W*#T z@OF$Ib7Ka7yc2R+e`p{PS-!|7g@s5s_LU+C6(@+W*@q`hSDXd0gLAzNd|JH^@q5!U zGBV=uBqRova6c#_q5q0&+S$6fjY+!OK0B$mu^341a!4`ER#37#opu`;|Jm46*;sZu zlV^~o3-hWqx-^Fe!agCLmR(`d8(JN2^+u94_@nV>_fiy0>t#oPg z9ni|d+ToKwauI!XZEU1}DL7Ado* zfijzu#I^$Vg4|?gm;P{qRttV)CJ!p-8bS-x;L%Jge+KN%u|6|}5+MsG1M8$D#pup~ z{pQRHvuHiDSPPFjL31Gk_R4!s=Jt6_(1v&DRjicv|DHkT9vfm_(!c!p@v5un@osyqw2+iG^ZBj& z#%Y@Ak)wH}3xNyU6L%de(-Yg`O5buk{S<3+bDj)uzKyHcWnu=`1_5sKnrX&&;?w+B z`Mt8=D}TE0labvU_j3kaOjk}j^9bk*GVVx~&oHDDyByeSrEf~%lj^4^4 z+uCi>PBWMB(;nr~Al5jsykjE&6W&*h@KTY>Or7WI7J5SCvi*i)B{ZC(l(M%>f!}7`a&hIO~j!zksBJsp;YW`#)}q{yMl@W+MgE*hY~aHw#Cd^^!#h_%ujCP>T9kg z@2pfy&*E_fNLe?Jd)6NcB4#jn^jokU)I9YyEkSG{cM6t>kqjv^E7Y4>%6=ynD7s`C zP9`w(nA~+C|8-HT3NYh`Q9;L%iA(YyqZ4AeAn7$WHAMm6s<*TC`fNYVlS-pRo}+5v zpeMv}Vogkv$1u(?yg}A2MeY4I?YJB|3+MrBfb{hJ3z4WtUUH7O!c?XEzu3bWv*wwm3Os58jYWQgoYNrxw|CE-JJkwx-zKqQ+s8bZY zk6mUscJ&>eG8r#8GPEgiuh!z~w%>+XUl4UWyl+@m!eggGYt)JqAVc_uDm_EE&a8bf z*|#p_jbbyxHDl4hdpUNj%2nkciNhHBA|YSkSLdqW!zFDgjO@#q=ew#dRV>^nz(VJ4M*pJskm%IN8}IX}jp z`lG@V<0S+`CFFo~REya~lB;ZMOCTUD3Rfy4ispLae1mpqLK*)BYJC3F{YNTKu^s)S zyCCl_e}*h~xIFTVn6cUZbcq&gjJ7XOc|O-mYJW_BxQ@0SM&~h{a;H;ScYY9GW*f25 z>0IF84oh~tcEoo(NMyLtG}s=>sT#sD5Q|+dW!iQrJe3})4h9wjI(xi#Ur5aa6TLqY z-@d-eC^h+Z8>nJP?$`}EP~W;uXc#rF>aAp6Xic5f$(!Lx8%g!`?uQLU-}x@y*X{m# zx;KAaxL-8aO)z6MM|TwG!F|*{#n^)YfraCJ+=@$eR#U&ts2xe?swfvzJrf!0lO`}t zJ$^|bWZSB@j)rtMa3Ss=?mBt*5=IdfaH4a45zLty?N+zo+FsA;PZm4oUaW9cd`?Th zifi~j^F4R}Bq1XMwwqP4o~+6==ns^?rzPYm z^y{C6-}<2j0)nCyw0{k!c=G&Yb3{)kp!qz?Ge9b)^ilk*s+WBMu#$EFgsCc>R=-Pz zb0}YW)e{rqu5bo3{?cfVi_qv~=})`h=G;hZHE~UKJ9`V+1u{qyn^zs~KRzNh2Tg6r z1L(q%Km+hQJ#?)$C`oeInrI4SRIjk5{p08MWL#%N;blq1`KYpokXwg|86cb$l%a_m z)EYP2o$OY6pgs8hq+c~MtDfQJXty!FK;C^1*M@ZL+^ZDbSz3QYtDKyphNdJBD6;yIf4&e+ ziLc)L%nLc>`C~G^{Ah1Xk;b)^uouBGbvumrJD+oYk-F2pK6Ul6JKm+u?~VUpuiJP# z2CFt`xq4wSV|zi%ql-6=yL~0jB`2yLR79q4Kfgh~7uZF>D^rFF>yLbF;stBw8 zb9*?gcXBy>UF>p{tZ)@pgHG@LoswqV67F!^{3Ry5UFOZ0E*+SS7m0#5 zX~v|XGc&v;_Y0>V3n)ysmR}P4lNW2d-Ur`%JvT@yz1=mQzuck8le*CD8hHQ_rujr2 zE55|nESt!jdAIads#+;Awz6=Qcg?!;c(bzcVtEb6Abo@Of!3I@wsTMYymh7FWR2={ z{aS*_3}8Yo=Pr}FH9dU1Vz!kPt7VN+>&ifG)rM^jyhP0b2=^pq5~Us3{abJ6MPIKv zF0}=k0}|Z1nOqA0WIiy*=CL!?tWrSIs;en}C;V9x;8p9nVOP=ef|Ho0{;~*T&y*$; zePUih<>1P{#e$FP#j?daHeq=21g2%(O|u93m#44^2g}~eE6)R&HTK;mqM<1& zez`Gt*xH$!Xoo3ykp{FpvvGFlUfmc-#OT9?3qS?K!(@%s-A#I(^n_*yi7M@&JyV^>%1xdIP0^ zFv+s*Jq`mhr2W#ZL&|l(7SJ`-NH^h;cHy_P1gN0PM9GN)Q{Yg_#CZ2ZSrl+)o~c`c`5cJ8C6mvENho-%wvFZ+%WQ_U|)fdy6$;=!%13SvAZSyygmt zT-*ULluBpjVX4P0UuMM5iXDe8$i4HF zq4g1|!OCXx@myX(sNh(;U=`WLT=zM50WmjA{_Co-hM;GW35*qAJ#xM~-v_XH!7km( za|HjK?^a6c4J^#6t6SMZiByv5B#Ey5=P99cN2Oq-%;4nDYeaW^Vz8eSO@)*S91-~O z8r&KUT}6jIG7_`x5i>9Fdz6p%S##WNOMBP(uiRy*rfP;J#Jh>(eb0If@fv*(@760> zI-ysJ>CP4TB&l|)^DgXMi+MK@<}((9ziak+ta4Rvv>ta&e_ zChoRoVIGVZtM;NKu{%{lsPgdTnq&74P1n_oQD$ge1nHQ`bVYt;c~kIntMgx(VL=il zMH*w507TrK3`l%)XAg7;b-4N(r>3->H?!);%Ne78yMjfHRb<4#d zg((JOF^;U)z`XJlD5QZY^u+TuAjLhQf~-E#gsD!oO4kD)&$~6u@pFJPTxF_1`N(V# zofTdm*bdYNrei}wFv9hKDqp%-;DX}=s-_4sBXfD9~~w+@FKY3#(z zX7je4upAp(*H4q5Ba9n;bHCL7b7ACeC01}fMZl2jlKf?ku4}hkOey=@r;M7+uh)b1V66WIxwDD=zffNC^7(Q=8YwriOn0HaE z_B$#LJ+FSgP$tkI6LhU=EdFZrjsltbJ?%0WOn4bLOEQ)CVmqcpBl*r!Ha;TsbC33` z52H>+72rDd(cR&%QWdt{9B#i?O$5@K*1S8>aqE~ftaG5R+6Nv)q8_A&9EhU!Y+Ya0 z4I(7sL}apSCC1K54ocm=yhgZ0`K;|Y84$R4GCSVWJ)wtH`%u1)T$%54X^f2~#NPEF z;Wsvd3PD(q=@VS4!3fmPjO-lG_hl%7sg4U$5W8-E{hg8gUtU)%%FigIC%qWtV4@O^ z?Nl#_0-(VE-GzZSA(jP#iJ~vAl+p}EQx72*nv2Ke5?ug{4>N_WZ*JeHT-B#J{LI<` zw_`6`PSyiq6C96cXXr{8bN(0FFE^;;cT|LYI=|Rq!MfX?vX`2vvnrUr^?ORXwaXo~ zdiahO#^1{#AEOg*lQ>pcR-%*-n@SxF7QofWok}$;+~QZMW>`Du;8#4sKwcK-tv zq~=h^t%)fuld1)qte6iFd~Su)D##Ou3x_hLu0_<$NJj?eNu{G$z}e}mT_q22BiK#6 zzCSe>{Tc;l?YJGj@ttHV&u%(2ZkCoayMFdEvq#^mx8T4J6#b70*H(hC_eocQiP2Oe z$~SLev7;`g&O^uDB^25m`8gRyk~3HwG*M`sXqNjD!YN z<0DLvU*?Ao`c=j5w^I^WGCwySD%T$tg$5t_GEGE4{$m;@OoZ~LiwRMjlM+D{eAO?S ze>Zk{Q>#7W{Xr6kEXu{K?&DnK-u6EBFGFLbt>}p4F`<8u9X+dkK~kK+GlKs4e6VVB zuTQ?=81XZkC6PQ4#pm;xn(}Ge@+=dB&M*Jve#(Xw5s~~bAtykX2mYduCCVugIHiR} z?fZ_Dw&APpJ;OuP$@Zv^Jpb`-Gpa*ciJWyD$PfE}@aGdodj1If&(I}~EsFm;Lq7QT z(+;0SMEw6h?5{Qb|N1?PYdj=N`1kw2Meb=>4RAhlivSRrE z;muD=!H$T~9FPmBq5g-B?BE58?f%?svQ-CB0f0f7-(A`WIfEOJnCr`*mkwQ&{u~X6 z%u4|=TF%DjH^&3DU`C@hUCh4*M~*u^K+QLpp=Kka0B%^FUp{6X%vce34NjLCpiP(s z-PV2JEEipQ1klQWgKN$=o~O#LZ2-P^H#7K&mDOmFq7r}qE?K`B{UO#|EHUYT-Z-fSMfY18fcRu{>4t zorz+27%Pj<_~jCe)!73)Myv^##It}Sv+nGM0Q;rx6y`QpQUT7mIqzZQ|7;?-W1aBl zl#Ry<64ox)rKxWUhD(4~VmxZdcBNjtx}pu(e5#7rYr1}R$=IX%jYZlnNYrujUBi&^9 zM_?kGevk<$+iD0U=?EpQiyi_oR~7hZ8{bF%F+-`XcR4>}h<*gB&dd&A;_#CA^ zUCWj4TImVh8$We~OMvTc7e(0o3XUWBH$tzh&zHOw$~! zoL?t@+5m`R9SkJG{_vtqfuZC^7tm%`sH9UqN&s)~_zd_A<7ZZ4fzz}}XJ`?eglXtf zJ)jPb+9McG#54S=?lkb;{`*f{Eov9F%bwsp?>`w#p3h%=<;D$~2+)9%Q=9A2x0F1J zPX@S~LdL6CUTpC_AT%>gqDfLnL&pE4s*l&$=y9`5&N=A3O6zg8Ntf!Y;Uv9j0-r!& z{*w?8YIzHwscpN_Jk#YS%3O{I3rQW?K-fcqohTUDZ3?6wVCq^uk3%BF6=-Mb0{S{c zy@>Tl$`F5RaC#OS7pvH&{ma=sd?&0BB`M@mYyW#=Ev0H4PGMy*zN(Cy;~-_ib4 zYBW{$n;6VxdR(=P;U!T0@u#vsnkuEkb4+N>m|16(c2T{YWHoyIH#g~jUOKXNf1Mm@ zCdDQPGr@RB0P(6(w-kq*=fK%OFxmCF1SOV73>bI zX*%YEj}IwRL(GC1>hjE$Mi_1>Yp;Oc{F(eCIHWoj7(fbCg6Ob-!@I^ zaC86jx0drJO{XmSl_l7eom2p9!n_B!ovaFVWuK_eYq;8Sw>g@sz1qrMUS4qK->*Af zGUx%v06f%jeUO0tU+WXgGX%*xly{w@WWwvXI@O(d&VpsRK}i;AO%0*00~eeEp@|O= z#-GsVEnq&2+yZ?R8G!{b2hgehF^-EVV|%->aSz4>ruD2yVQgW_NbD0@@54o2baT_N zmq#>OydOnNM6q?0(xdm9P?ez=rsiJ~xdjtH&f4oa$OYmzG}pL`SUN=Xk=Bn8G7qzF z!9%d0#k&K&z6Q|2)M-8m;pUpiUJwh7FzpO+6MK0=r9tgzW8=wI_BMv47?5O~gmyn( z$@3jhRTYcGSUOT?eyeCHDXSfRM}MiLVhMK!D1P*~^ACXIGGp0bL~4e4K$hI_wHCo> zk>Mt03?)DfClJhyh;O@IzbmNJ(-r8H_Ol&+=6Bqjv0Y9M8apexI9>L6Lh8<-sr}lyA1^Yj6%ET@2O*A7RMVRL%mKf0g z=vv7qw|eN^%6jKHFoy>!u!xMPgG z`)>93$3I_uOpxtgV6E>;1}M?mxNiOV?QWIsi*F4~8p;Wd&0LD>@?7CtDZLLsJFd6e z9N9R}ly*ld=q$1)aF8nmcvfrbz-1ksI}7xKpNutu<3M;><5v6g78?>ugvy25J6F0C+hWA;rQ9a8*HP!Eosl4o!5N&ua* z+(U1SG$#d^C)qfMZobCy?2vyTvI|foA($s2?EMGcyY2K|M0reIf7-Xc%6#$JcGbD- z4$wnHMn6Dg&;G8T)&np@ZIo0vD4%FQE_coX4lxNUdp^e45CIGsKB`8#upEt z2Qe9A5PaHChsdW(-#pJ$yYum_txSdQC@McTQ@4?XFs+L_h;+2_8kd6Tqm+WwB$F!O zUip1%GD4RtIE}q`)o&xeE6Zg%;g_8FXID|SVvy^-i{Zhi=EPph>I+SSc0jI6cNZIW z4IZwCi||D^H-#k0BOpT?a3&HbPS=!sFAX#J6wGH{c+-*X)Hh|T4yaFUhu`X(4P1O{ zVRmv=`2M{0I~iL|{jA=i{d2U3-FSae6b@QY4Fn3`4r$!VkFh z>_%vpYiqfh%va~~aHYkXKC^so&i;^fCM8J+)W_qFN%D+?gBEAdf zNZC*0CDdeAX!dE|URS#_O-3}KEm9R6o4~#PpYnsP7GlMG*hl#^0rdwV68~%|bO>Fmm6n#4QKZGX`|^EJxtU64({O(?0FenSgSoU0 zqOlWZS(!43X#%Z%DdQ_#HK)WD{6h|OHqT=IMYST&$2{6I51I{YCRmzrJ@HbH%UDD> zs`GKRx@#_Daz|2K&BofC&0o?WVy7^;x*U0;n=ptu_D#y7mqd=R{W69XIIQhvV-=E? z?4dF^w%qz1!y6J480B9^dzcT^>H$KfrLW-6v0ptp*SmwIpZDo>+!(P_WOl<>DdFU+ zcz=~j{%HmNUK8QBC?u4utUbs4=9|TAaCAe#o}i*OlKAJRp7~Rd^Fqdl;5LqPK_Rgn z6q%bt)HdRe>h|Ll!Fz(DjS>%^cb^6cgBww%-V!Y=T@!I>9ZawPYG*BXG<_7vXHw+oimt5EYC5gLp!4B#|ePLGAjBK763OQ=uKy+{H z7S?DDg4XUeOPdfn zRNz)%m*QHKh-;72n%%zVo%n810nK;IDu2$h6@w>6#1Zz>$=ZKc(QfW#qD9kW5oscG zwgA1!NX~^SkL=$((JAOX>qH?KTBB~%4&%}aGmHbS8}3)P19{i)1+e3`Q9sj((n7cb z-~o#b`9maC@h@l$JQY)4=$QrmRC|DV_;Cz1;2inm_3=p%()9~on}T}!z@@37_=+x) zs?Q$|Y`*z^tj>H*HkWQfa88COo(yx zz>Y0Ei7)%)es!{`#ZlS24MwXvRM+07Pi*-p`1Iv@5(a$@dxYxEO8XevhVPg7?BX?b z4fPjlsEvEE^jkD>N2Fnvk!f@BAK#hF3JA6S6z1RIZkbe9Mw;D%YGUo6!3xMNtoc89 zmdmkKHSDU%F9b;b{+Abkd2TQnPSJnY){RyB zn{m5m2I>!IsDC;3oVj3$H(!R0NoDivzLjLs7T^ttOYzmQ&%|&1X(46c#G}u~(I^}r z5m4R(w3J$sv5CLa?dfInPO>{gT$oSbSS&gY0^0YzNsw;tcOX?#tT^l7k zY+<66{BLgz?5Qg#*L*#?&+_C1?~Qw+WldM3PD`1VFGFWh5r{>uC-^>3>rg?qyHViy z$+*+_BM5h@HtRZh`~WQj)iY(OrZ!1m)9D`pX4fttaWRFYs6j02sN#xkzG6ITX5)G@ zOx3#~Z_XS;aQY=vsQYeo>oi3@?1(axE(*>j|*2grAJ8KxzLG{1C zI^K*bYdhsq){boeDbTLs`!=T4x~x;qY9?uy_9N*%Y1rTPZiKIJbBL)4me72ofZO!M z>;J6BCs<7o1a$;28EMztktI-Bumip|`2!siFBl~mByMP_r^QEsRJML5)XW^JZB2SN z3$0LRc}c^Cd!))2XKAz=tx%Vf2)T99K;+LCBaVemioya`5-l34sdI@Nk2yrRaT6I~ z&j*gcXgl3^F4Yxie4>L>`mOpWt_(p$6}4|Yx!Koxw>hDkU<_mNDfhxxM`e(lCs(>I z8i=Xk)`Yzxze>{+VgvNbG6U-|)}rmN4-uEfADk8i1G%oeT=k0R?{!91q3Wcp`j3}1 z=fCy;ljOpS3Ezc3h}!=S&H1O*TAzUkuYA8Z4}05ZzTX z$^WjP4-&Or)q*{Qlo@T0+j0WiZtnJdXW;(xJ)s`N-mWn`y1aKIDESk~6h6}rBD&vo zH_%namhih?9H_KO7G_rR9i>_5sxIBp18P zOpu`y;{h7%E-M^5Pw=G8G>qVb`-m)8z9P6BEbP%r+yXy3efUXxEp}awHv^pl`R?+p z&Oj+x@SQMYcBm|lWe38GO`bjs+}h3NZE~SA(dM)`^cTilN!c>-OOCr!)dR|0FelhJ z{&p!m!Aci2`a4(%S1j$FNu=&`z?uH5Ea0hu0AdpB9m>81f2jUCT1h3g3o2Vecaa=2 z;&QMBdPv|M$jTYJRKCnv_ol>iB-`>JqHeE z9qm}(VgcFE5x7p&mU)s*j@7$3q>{qY0*o`5^;;-xp3@_H!lr_OZi+`|9$TZG;vw;_bPD{rZC@ZbY!DlY8Cj515Z1cTAgw9&`{?m zjR5gtD?DDLlULv@NjUCV_z@F2(B!8SIAOa2N@byDmR0?XuvuQGZkpJYna@ui8P~N} z#}q#=S&$6YABm=!2V&RP5<7Y8p?Yh)pD9#Q){3^7|4|39GWj2M0MwV%eDpk{f1!{( z5s(hF$pColyfAwL#JWpeRpRI_0DfDKAp*jPddKR1-t9IM)zA1gbhG9 z);$bngMMH7ehU`Fh3N`BkA8@lFW4{{@M-Ol+4ixl?2MRVtEdmU2Aez)ccjS6?ye4b z$D@_*r)bK)#Lyhil#TpvyfQH`J|1R==L=mU@Bw=@#z+wCmds6U6SyIF6 zBklf4?H6RquL%Pe@ggG>vGw(pr(!H5R^m0Woj8=my^@)qp;qAm%CvKT1Gn6jiGE8( zn82BeHvG82Q2yCJVh<&FR+(&%{u?jzd*m;Y@!%WZW!*@(cXfjEmjOtpRjx|xMQGRD z)+=2(IV&fO`IsQswvc!Mh`C3nm4x^wjL&ZhdzNL-D0>1519ljmkV(B*=mP)TO)*EX?S;UR4hjc%0z{HseKJBr`;n@>?ynB^?v& z4QP?;2-&<|R}DaZa%yHEV6k|&(MYNgbJnulJu;g#T2Z)C{o0P_=Lg3BMn3m`M}jhb zjM^z|`Rkr}&+qxB%o?g0eX8E29UrmFM?x&VhXp7X+pHV3Yn05 zU8S#%U}`cN!v%YxevhBXCnkG(5Gu)6)%?1A6LF%tr|?rwmht|ule|_IGWkM~Yjp8@ zjdxqm$Lr`{dj!>F-6XW&U3AwwVCKiX+_IDl#?W$EynLZf9vs1OP2RIaI)HYvAyd$`DctE!EfBD%lEcFMPORFZ&=Ij0&lCZ;&0d{VI`!sz=qzt`@-d% zcP6!ek%4+8kb+2Ei&b7+2wH$3w*Xwc4Nr6l$@)}%q^Au5IPvyDu%jva#yoI)G>wKs zhe;4os9sB|b{<|?_b!;t7VAk_D19A-+tVnQh`-(QZz!lgn`-C>;nx=OQ<6A_kL$3B z=>pO5y3$kQ5a($1b@su=z}9F_>rN{BB|70>?@ z{HCG9mkn~jm;Y)!{(Hau=Zmy8&_FBTsrUkDdVl9m{q3*dr@es2KFmn?cZK@@_{A?C z@Qav)<^Mf{|E~x6=l^<%2zqS#Uw_g5?KS?_%YO-jHwN``WB&70|Lfm?pU@)=8h=C- zIjH}-C;!*$!C%!gftrG{c5Nc(?*YjFd<;)`YfNFMHmaNCm)|(SbyRSHsGU zhFVg$sK!0)$m`*WUB#RQm^;6pE?*AihQw$dN{r8@u_}=7t+^75@xl!G`nx=f# zl^)(033_9jJ0_UBwGZO(O$Tz+Ce8+CzPmSFh1|fjGS^Kbod4*Jpe-J}S=0`?W-7+y z0xoBH0piznrzxO6_|7g0o%*n93H+ufW;5Ui{$5Q47w2Ig*Zj(CxLJVs^q;G4H`-Wj z*Fy=DjNudDG4N{wXrW)1C;BQ|H(tho2A3)QGb4H`$~kfvs7oe|IH*|5sJ8SE02ew9 z?7)uSvX%mZ2Th1P*FP+VFua0ZDP;c`(^-WJvtd4+AB<9LK5is9f^z$7?y(|_7&^J6 zxJpbo;a(z^Nz;1E=QR3QbsE;fT)l72yzZNU@!uJe4aP&&V8w07*VljH*M!{>B3Vc+ zUf6IyD}Qn)LG^tAKa75~30?I!KGc5<>eES%+B`ai7hh4kS7S1RY$l~^SSK@f4)E~H zUC%01^uH_3fG6K6wK5~MYgrBx7fFpbn~qi}sp?)xc9?5yWLt$5(5btCF3H~NOsJvq$t&B&XFHQXhtXg|muwp~Fc|KmIInS%Kc`JEgIY3KYrm_uSr=_QU!pIq z?dKtMdlF3@JeMJx2r*yjimXtv>?tP*T;)++?KUX{U@}1%uc^^Yl@*WbZMPsiLc}2D z{o}jNo0pn2Fs${-z1z*??xtLV~FJt8^ zKHy4o%o$me0%ue?gj50(H|fJ-n><;mF(O7S!a-`z*MDrRYpg)OH4b-|?I(;1^4`Zt%8_lGO$jdE2=l zRcGA$8ii=gbw!|<)44QEY+B^GJ(TIM-Qi~hSBM^W3tBQx_feQ9Bmth774fCPKYHH{ zf=2a!#S8z8$i78?j{qs9|fQfed1O%Uq8>sc5+wXWt zaEQKm0Mr21@SQjTopbWxWK+PNrJ8=sytT^6Zzu+=VrM`>Xop>w`uf>9u&(8Wea+7? zSDNlF>A*}3$RnSDcl?x_wTTvw!HTnoE(||zdmDzg7?nm2oPLaxJf)pf+(D;VP!E6J z)W`bf8!FPq>6rS!0Kv5ds8b(Y8)`$(8h~N;UE##lWe;%E)hQOf1E2quyZR)&`;=cc z{En%_>@XKHYpWpC*OEIy{8IHw>?hXMVRC_-57GYdKdCUY|Y;r2dR?4rS&iN~&k)erSD&S?`(pT+iZMQ7FL-5i~YZ2#@zPfJ4j z2GmdqU!?yEReFRUux^5a^YLVbcjiM>qQP(mC=x5OEVPNs^hr_p8{`&F(Ic==x9`Ah zb?x~eJ}Uklu|Z`-UMCx(2-{Z*@4?8~U`PAqP_8RCiIi$%r5UhER<3(K2aR!wS9QsA z1vWPm zwuA~`&^w?DA8JOe<=-qz>a)Z0ODvnS`4Wx2D4)ch%#?aZLt$X&YgdRoaqqWr94^%; zPrG*wpM^k^SUal^8mRuszADm_WcL=KAV%L@|D7>2n$UFk-%I=NN8nK;hfX=5`Te8N zjQtRYSlU+v@=~y`TiCUIo#5f5?&WIiITdLIy0OS72Q~F?61YO3_u(4zrG$wH211-Da0D-FY-vD_THT{jR-8jlv}&5)p0yJYCkR8g!pEuuwvW(xSt5Ho+y!iv$BLN*cOa zh&HkDHX}~59=U+CZ59>0-ZPI$+;v5V(^ymYdiRA&WR{Sk&~gZS9bE-kN=$xM zjyI^@{H!&V9{Og2Fuo7pes_u!k*_2xx|z4s${6ThTuG04Wzdet)~2Lgp0!l?+f`SK zvaBY&MV3dMxHEkh40C zZ_^p0ISN<|k?onc zoz64E88@Bu5m%Q>Tzb?;eoHsPPlVoKguv6nR6%C2D?DlW6;{NgOWDHi#zM(EV}6{T z<(rT8b7$Je(nrhZfW?Sd{RtsmrSIzS+HMthD!M@&x}zLWv+k46IjR^gemCWsw~o2{ zQLbM%T%lx08UslxbRB;Jnpt9@EV40tT%s~*>&`6AygQx1-g@_v+G|-VEDKGZC(cMt zl^E-rjaNGc7hV;9bTdy&iqAq_1Gp#L-4o3!y4C#==u%y+>o80DMiAs3aaR9P`BD=q zllxop@x==s_owAR{@VT&B41@TTqg%V;fgT!xe$14rwF*Hu`++5-iY!Id1IzpMQm0U z$?VPTmmGj`*ZGdW9Q#*ykDMfT74bFZxkNp@v^!wb)(|bn+YL8$f=9z)iRrV0mCx0Y zZnFDod6)?MSQf~2b%XI4>qf;lNJ(O(6XkJ=v1i1XKbJyJ}qSsGt%A?WC$(# z6C)9I$6#t}-Q&*)bS+e8SW;w8Kkyj2X`n7#Aa zrL!>dneJgt1rCH02Q@{+4E!Iyo}$P@;)LaC>z%$3Y(3WH>O2_mxEm=q9_c_ER+;gM zq}4tCgETpNhV)SNdZJb3$EXgmci7Q4#AZ7?P$j!?y3_Gx)9XNQya8RswSaZism%0` z7nh{lsefOx2q4)if1q0XSE}QghXgzqdf5khf6I-rQa#rt%)rlh?#&n#`u4LScF@Ep zBG2a@pJ9Uf0`V>I>y_CHMUt`MCg(zc;x_Q2QXiRM zwJ`2wc72)slS66xLc9MCNvYoBz)wabrSeyQos6gm{3Ran0ja@Mt`BXuppz+Ea1U9yx!3Z)=yj-mBp_{_2TGl3Qgv<3p)4Yw&=ura&G0x){jmuIAi!bnl!wN@JTc} zQ`Z9ku8=!Xod&a5@qHAY=xN4oGWPyEZVYM+MJcG^<6{KujD7_8oaLjbx>p(nY>vHf zzwA3dukt{eE+7aXFw}AYW|s}9-_|tEPECUzXuh+=Sp*lqeY%5ycVLQT3+it~akc3g z{wd&izdZiDup4wE7)$4!W6pvW4R1&PE^YXd0N;|dd1wFHl2pR>yWqA|WD90k%DRO9 z+zpZW!y-EL{ghV2U%$oa3Ma@d!1fYhf6W^Cn(DX=1aDTQC#mS`!AvzTwK~xum_P@0 z)EPZMO!8r}#m7mV$?mEN5pyNKB@~6NZMxE8A4c(x8ambIRTcytw=td2u=MuiMOiTZ zgq{ALJf-oY@bi3CynFDKee=>zMLu*9BKrs`}KM@fwyFoNEn6I7pPgGihI_X)au`&4hnj( zeN?-D!{r?CKYyqZ4CqB;yUt|McBl_bpSIR{s`@VkX-ZARe7^UBs-P5~q`2%IlC=QE&}aS#N-9fub$#FrD80x&9UjvbeVU5Ud4%vIaKZq_ z0wjlZj5_4aP5xt@AY3hsdh}$Q0feb8iHuG_3lMwN%+h@U7ktft(WcksI$j+5A)_G4 z)A+4n?K}HT*r?rrYLhukstPr4XPRwKdbQo%q(rEDIj37cPpg-YbPl~p^$p6JBnsD; zs)nk?ucUIIu5VrQ2DyCp1SR!N*DG9x1-DS;8VPG*MRjsHWgwXpI>_8L?D_U;F1#VG znXc8|RM5mF>!|eB`0w2XPT9|M33HxP|CV4`YAxiweTt4^h(J791+}9^ykCP>aDp~L z1Jc+N`ZDHm)cng{?yrme;V_rK;P>MaOA2f&`l007$H64L^zs*T`4)=Cm zsYdRnd{6#KVPIKwjH2Par$+~Aal9uhkk`b* zBM|M-A|UHar43%ubO#Z}vW#QfpZBgmhiPIpo=s~+*oADw`zp^peF1^~U(|gEP*hph ztpXw{NRq68fS?iu0m&fA1gMBg25CTYk|Y@nB#8==Rg@sP$r(hVM9CS+(BzzX`bSS%nw{s&X45hy$?j5 zNf#2=s*nwv>@Un!HOX$@8S=1`po5If6iUeoCR@PY^?j(KGTdbX#% zQ}RhMP+@n4d*x1!cM%OB;?{Kt;)JGOCxrH$W?;O`M^%?4yNL?1K>N+ctj0)rT#vS_ zJ0~h6tn4=iD%Y-KKJ_Ck#@MP1pGP(kw(b_bB_z87d%)CI%uFzdeVqpx)G2%-4*s5p zjQogR>MR{U-Wr77O-)jAjl`W28EzudKu_6~!$m*MaQ;SZ4+-f`!-1JwY|yA^Oh0o8 zZF5vEb0z+!!`ML83-V?BiMYcZ>}rDs66AcqH9`^6@YTZ?YlvdP1tTSKOQr6QSGo3^ zOPtg%lg26!kGL=n6&~+M)zn3@yZ6gO-wLk@nP(c6nR)JM(_CKT?R1%Z+h}8YC7mww zeTE>P6O^}hhcehSVdCvwdi`BgxI1)&U>T+FlmYdaYEgvn@Ynx0M{wTJ1s?r{pSqpE z3aq&Um=&}j;`q@1De2vfM(-;-06=kQa@CG z+5S!GhUQttYc5OKR2qkCG3Pa-7KBHWNOgzfo3Cn;DNo-?1~t~=rmxiGihFs_!VZga zZ_aJa^&;HMwB>V4jWXEPRlCR6t1;m^bv;ls7!xuZQ;DH?#`kBaU(bsG=p(sDv#@^- z%vrw!-7irmgR`%PoW1qRmWL{ayzMkLRm79Y2;UUH_+SK;wjK2|- zzfA$@c#*3+_MIn|xO>%zqv-eoHz)I_<$$!~&hwSHy7@Wm5E$^gh{P{exHy`R);fcd z_P)hFweA7gV|7G)*{e_Eif}KHr?zKx8kZDo`URI*-2fP`(^N$)XykG|v|&OY;VIuX2dRaUt8c}~Zn?0{_-E&UvXKADU39!jSaKzypBVEClqV(Bc3 z=6+ZfV5&YAo@4Eck@7j}e!6xm)9qiRm9t9xN2*@1)(*$M6n(No^d*Dm+&wKCG!*q5 zjK~`a)<``=eilt!6>{v0bchiX7)#z18*UOjGeMN`<@P^|3e=Zi_a@lE`MCXKw)4uq zIen~~S5}%G0UyO@NdfHNuFsbJHT9u&xEapv!w&d#!O3Q)?z;SWEOo#TpKiim1W`AH9ZzPBVvafgus!u1w*)THG z8wJWZT3SE+OUAoM`;0lr&>jIex7eCv(%WIP z{xQ20(A-87untQ!p??UCF!tu4`NWO>)K@UYvL%OH_O~@>XRbK>$?Z?bMHW6Yg%^U% z-_NMI`N$6R&RIOY11hRHC|P*zBcU3_QJ#hfep_HIn}ht=57U+#1}%9i*$ZdTb9DO~ zvhtmv(4pX;FLH7DkdV3&(`wGn23Za$JjNq;L8!184TC!UjDTib1q{s7T7CaHH2-`& z-SPttxSwN4-(}CN=QjQJl)?sxj@o|akDDb9=g0a9e$*i0{wA`?MZvHO;Qp{_>8_0} zY{MO2O5&}LYC9w7G(Un)`_)@<*DC-pzadlcTLKZO_m3wEel?5{UyTk+-Q6?v|M@^R z(;s2WYQCJfTO~gV0K1i0>28iHxB?W5EkmBEz=|V?TzsWtAM)!%u)KP)bj)RtWagSBnU0t7Qt#-H>yejOp^@!hmsO-(t zoy;!hN7HS!&?@1<_j&`M?kSSOX{ILu#x*TJM6uF<$g7;`I>wA!15keLaaMru%ere??I62sK;ePVX#IH zCAg*D{f!a-OTpWLvb2ksbk1%NwDq?oDi(`WA@(|uRoRusK%1uKVg%+fuL31fea0*j z&vvR=y`TuIm1nNGzlVhKknc}DwdnwSgIPZYb%E#M8LHl_FyBB%S;5an$%!| zGL&(Gl6G8ulhw4`4k5Vko!ZvNfQ{NV2`~GyUOdZ4RfC+-hnxzs?^R5YWO~^_=TWt{ud|OiW!7G< zvczf01qdM3etBH;El!a>86j)IS03C@Q8j0S>S>m=Z3O{;9>9-ZO~47kxgQKU zW=&*=v8?ovzJrmEQC^JhJU7P?R5tVdpv%4!XI>1{T19DgVf-wh#pqSOSDE#$es)@< zFI~o7-LuT;y?oX)BEYQo;-5^7A0S(T2*QK#&tChrrc0A?>~p_(xC{%WwLlyTkK z{NNxw70$;AXK#MiqtWQU&X$3bDT+;#W_XO$Izu$^JY`qQjLgCTLTTfq-dI^eev4tL zm{W~f!Ck&Xl_nD8n_>FV20qT$_%4Ng#p}k~*WM4R4pO|3;c91MZ4!R=o%CNZ%`fEd zvDBMs&3CH_nbY1s7u$<^Sa0>0!6GQhDFLO%a#-(GMp<(Pp>0Fb`)V+>{d$i>*VgTL9{on& zBxBgR7acE2l0WXvF&R-pUG3E4N;p&A;0_1P57M-QbxI91u1;Zt&f5)6Kt^)b*#Ebk ztgI-z^!n_rpQ-n_C>GvZgBGBBu0t|k2oS+nv#QQ5Xq36Sj8_O$t<`0KdS3Se;A0tn z33S>0hDj+QnW-P2-eimBGg0yLlp7bym;(GmU9pc~~)?`|5X;<{Wng zvIccBIwJ7ytGW|~SO@ozQm}_)G!$6BDM+8N1{5v^&BPw7mn~ zO!2Pd|8fa0T)ELk1lcw}9w{z<(n%<}(g;|-=V{V?(s?&UB_9Od6Iw_k;eOcvNz!;) zUMGJ8jIWNc?vifZiCoCd3ILie_&9T6_}O^<6RJ*>Bp6HUARLww^&Te4UMz=zwb|Zk ztkmvEu1llXgu+8al?c|hKDfWv`SG(NuBGF6n0_Cyz(+ezLEjVSn^&c_AjA_0f!u93 zZwJ6CM~6jk68$d1%Y4E_@r$_eW>2wV7D}A9AAfU3A{OXh{IU>Ykm5WSI8p&uuwz@o zt*7B%`Tp}^rDg`c(zAm}_CNp0wal9}b#nh`3nf^AK#-{_PmB%`dNet7NX1x+NF|9> zL3r`WHb55rNHLwAx8Tv#@W%CHQnsHpKEv|te5p}``MSNo$l8L^!je2Gx;qc1jQuS; zSl(B4iEAa__NDpm6@#jM4m9CwK5v^%dws$fG*oS@FN~e@AQ~)naj}I;gf5*yO&yBQO^hxp~H?8icKD z29S!kRY&HFGSfpsfq;{Z&Xcw5BPGMedik7F6yZFiDluI5KPZQppUl*6k|$SC2P%S3 zKd^D<;ch7z6fPpxf~+R3|FR$ToZ(VfVVpIys_E%Z`@LMPM!Rk@c82<`L6D{zxX>bE z+IPuJ>2?&zw;K3NVP|=?VTO1#`OTPp)*lA_{*q`bq9ld@>SLqnZ~;gyY}i$@7QT!b z^AAYy>Zcnj1JLAz`#4CJH{z}}(9;Y_xFS9n^2JX7x%CsghO`dXaj6YH@twnV=H?*a z@FFVN9LU1A8VdisJ4*PW|>t1$j~R>+66Jq-oM#&j6XNV&_Q~;Fk2hX8NQJ|{l0s~;;o5+W`Yb0DTS2XVp&L~7p$jort?AzjQ(gFeo45W87}*w8kRC%6{tp5=mK+<~+Adr;dj>aEzV4N8X;#mjBJSew40=<_{ zsEHo%w2<(9!RM5a%_D??=m6>W?h<6rrj>m_i==A0r#}vvb4;3_Z0--L-*I+lX z4e|CaLBm*xYU-ZN9gzsuBa>T^tO|w4Rgf{5j)b`=D0XQ2$Ac)Y!{qv+`nyD)pk?E` zM7|!8$U8wV6~7sb{dib)w%MR4IhsPvygaWFc!U&9Ec%qr216js{eiz- zvmFSexgdKNFg-qW{dmk}Y!k~pU|WrZDC<#annEHo2UVR!G-KO0yRIhCQl8>Jza*Vl zt!cRFxZMuk6mHKXTrB8LfxhS$_4oY)?3gV)6(%-KTs1*K?8DQmQGxY<~#FH&wB29_F~vtVT) z&`HC;mIj@Z_Se!AjiGoI3gvM`qy@J4u@eej{&0zUV!Hr z4qh@{bw9>Y)7ya*lbqpRB4GBV>4aGLw~MKvk@i3c7`9<($znjCyGp+J1;Rnu%7D|7 zj@{tx@_}KiJVa#mROpn>DO^5F{BH+;vhe!zx<tW=cGyOWRacYxr4X+t_xlba9(zVShtvN_@EN&) z{LedAS)%Sty-5R^Lkz3XGQPN`&IZTr>`14oDm@k{ z-Y~N`xVps8!4R*DHFFe9GKjr?&v|0EqF@Bl`33(t#)RXTN!LLe0C13siyLq5`oQ|z4)>$`g0h!>bD*4uhSvvQyH7QR4HvFdw< z8pv4Zx5T?~0cr9S-GJM2a+?>x31Ni(yI#4a3vCn?;|z7=-nq*Oe0(nkxgk6gByz@Ryi3WBpoBOE&io; zz?to5Wmqg?An{`bnr5q=lc?!i6KK3U@zSjQ<0~;*n-lg<_oOsh$=+sP$cgwHd1hte5qUZgK~^icQRSaP^Gg=OEBlF@GPB?%opnl(iMza6I}LtnBlD= z*q3&)tJY(hML#F>U|i=F*7A)pLA$wvzt$iHB#5P3Cv7|SBk+~EHuT2h_!3oV;LTYc zcS<}iM@L=~GBrqpia#663^ZVSi7a159gYQ38Q-)z0jaMYo>{v)1hSeI_WiMwqSL z!@_e8u%B*Kx7(lSG8U{Ee-IFN+ha|&kt*`smPADOrhJtRWZ(S=5+w@yiv7uqMZHo) z0sWyLOddlh$Qe;tbdM+BE(&-%78F0da^K5lNK#J-fKK*b!wk{%3;zSkjcZbPveooH zD79srBg)NPtZW&dNnXte+Qp1)mefWGy zs}?aIB*yzui4a2V6;diFp8F9LP17M&s#2?Wv_FE=i`-bAfu3y$8!R{M5@;=o-ju}- zX=j0Nld9hn%u)lR@5m~pE{9Ojo6Un&!ZxRFkz17*CSr(7!9~%c^aY*k{de>S82!35 zUzVTPc4GWE!Wnf&1Ihzk12`8DLxE+{L_99xai2@RFT9m4P~C^cuFXDY^KV?fZ^3MRm#Zfbl^!{L^Ij)Q z%!pd+tq|*-O|v`pCXvotQ_g`AYkia7n8gM|@z{zs_MTSa@jrW)Y%2TpU`*+cl+$fX z7Q)b7A(rXDADb-WX;&7Wuwr~YGTag$R9Ol_0acw;^maK*)uVLi@-);dMtSlVU8RcH zna{Y zJHtWJSk0<@}!5t zIMYj8qv6M6G&~GZR>Nr*90Z46D#|Zj=I1y46w%k$JW-V?azVzv#Gj11IcqVJ_mmeh z1Q*$`@|(!`lS`6vf997r&5m%{oyO)S!JpXxYJA+mdu?Kpqi++RP9#fFvM@$0 z;V4U^Bm$#k<1U1C2T~Xsb8^S43>*oe?tq9`*x8h-LHkV3>46}FYG*Vq_Or$32Nc#w z#ZFS;_*UB=ky@$YZwC*|uk&ebyBEO}Cwsrvri=f;ep%kxsI`E7fxd6l3>kB&9x~0`Z@QY#dAJ|TU*xpTG+Y8t4(LFVbrz4BDPaf^u zk1glciJ8xuF{CBtSivSuGj_}g{v_mn@qyTO`S2K&ioCY-T4ZP{m%rirf2Ok^4Rj{s zXVH>k7IG(cl5|wSo!EoQ+uA$Di{kHdna+)Ebi!jCWOv$;6&4_B&VvJ1&+w^+L6+iK zbhx%vu+|fF=uD=|d-^eFYM0v8nYVkZZVtuGmF}&kZH&`lF~%qghj}d2HuPPdA8DWU zneL=@D<~1&oU5c{_||$rIK}mOk@CE_2MsF#f zQUcJc)e+$YiXU_Eg|GOeNj)L`lO{(AzJOg%mXU5pnu?{6hA#!pUl|&jHY2SXy)rXe zX*uIxygzGP5`sqq_0Tt07b;GLIM|UeZ{-WpP;=YW+Nm3bW*haB4fBlswFy{wZF`4(s{41J>Js8D;5R zrsjZDg%Z|<-A_`+rruV;k9)EV1Q>Dx^|WfhW22Yy$xHec`Ye~s*p)udbGE6}7X5>1>*X0@>bS<+FG* zP@f)Pji58azuN{CkNt$a?B`~D?=rIu{395dv`&BWzsUUptdFjVH4}fDhq4Xk@Uf9V zHSQ21ZQ?r(%T)VHJtLIqm%d%F!OEZ&3bPA9eD&VRXzPCU>kLe5s_);e5H= z%_5&*ZGK7nQluWcv#o~q4Z%c~FD~WSRoD0WImAv7eV$B<*9}pw z(u~a+h`XPqOmBxgO7Q7a=I@GPz-4+8Q`V2m6i4vM0H9Pb5O8XTa!X8R=@qrq zJ6t3ctPJ0>7)Vg8CSP(Hwj}Nn>r&loj;WD(e>-EcI%d#}DeG=e2uK8vlLfx=;#6=5 zlr>UAb){?-I`cu|wJ#Fkn_Ir6j+-Wb&E@#H30Ci|kSFnAY+Fpn^B-mY;i>uF7) zFb84m*;v+;`$ZhoQfP*{XkW|;2R}N;1qsdgwh_*3M{28-qyfzRo$14+8`51hdI%yh zNFxt5`$$HgCo>zxYjE?Id+qjf)-?whD?HJ;tnVUhN>ZR_REoRqFwCcroqHES5&7SNvX?Z6t{nA?9z=7#&buZ)T!ml*Od}a6-QP9S6fL+Fx#ctscVVq}Qv<4aVt2ZukN!yB+GvqLDK_Hh&w`rx2!nVW zm2zib83s~l3gM*}p%1@e!jp#pixFD$j1%>6xM0rIZI=~;zjL~Fu43zxyDt-W^f>_6 z_ajTBJ|)wg3_=6&_fBXwt93^-+4NGSKJ~E|SIpOw4E)`y*e+|DtA<7feUsA@u%QL- zEM6+aNzvKS5h0#H|Ij~=RwR7;^gHc?=W8K|SVL!6hA$HN4=)dsHI@$NeH7-BYY@rBF#;)Y%3QX(_tv>2x}Korcv(!a!j<4-9@4DgeQ-*z{`EH^*CPYD1NIZA_Gt+{hMK3N;k#o)bE{{@UCmk&sh>V)}c$QBVf3l!AAZUN&|QWMEpV{! z1VZm57nifnk?`J7k(lC4j7;dMbf0U2C zlM`!ock9N$&^al88c9S`zAI;#stux{sa({;6Ks!khTqE=1(bDHF^&oRKX&F-Pry`?6ycupS2cad+56$;^{pag9x1hM(|JT9ZFypZS^v>Io-1Eb(>#UpWfX{W;5AS}4 z&QShn_B?W>#`H7Gz2`ql?l(oSkfmak`liW`l-iKA*})>4_~Sw14Y9R3k!v5PZ|v-~ zsymfp5!9TyB1v>pg2+qeg;=ke-7>YrJH1Gzf+zN2W3z-7OR2AYI2O4x(L2&Ef_hY; z#}1o?F=^-#pOzms!t(2s(h;Y>R@Ch2`jG!3P}qTeSLz=WNfey` zL{N0X@gqyi^XK}Et_QI2=v~V^iRc4F7)oR`unyu7iEDA;OixckORI6I1MvG5N!NW2 zd2H5(w*?6oj?u(>RV|z*9a^`lNyxtyjRUG=cR{K~@4v{9WH{_56w^A>4}+O1WL z>{KsdgHa)xL0w&4(n@m9U7Q+g$2rceviUUGkge5~{Cp}b3oI0>+QR(j5VT-XU)h{e zaJd#17;7*o0q4if@n%eFrPS)k1O3R5BXIoh=j|3rPR}S5Un)vV`+cm=9?CK9et^vi z9RSGt%?u2v>Uym>kwN@?^x=Zab?pD63gGh8^d z(SOoWXUVFJ`AH6(zM|*tMd(m!RN6G zsefsI&{^#J8^lAlJUo8&?q%XQXW6-lcvF$n|31dgl(j0o$DEgY$DV zzh5>N4i*mj`GGLiA0rI!d7L&cDmQlT@>opaV)-@l$Y=4N9dMf|zo{$#E#>gSn#KRpFPkKyXLNYv*i>}p=ehY}xjo{^ls$MP z&GXmik*{TxdZYEE=y00b?>Dt@F1}7NY5%BXgK@qo|M_aV^sg~OPFd{)+*FEE-ss<_ zcjYuuJzQNBmXG3}BNjjkvu5IR=VwWV?;{95rjF#*&j20Mr=v0M^shI+|4lv{Tweap z?BdLyhxp?oWoL||`5ifep0y^Xs`b~gwpI8L-QZBF>Kb!;`I`~N=uKmO?Zd*Tn= ze05Q2nw*xsCJenfXG(6R+wM6As+y50mk8TE)A==^e_Z6xfq#HieX6FT9z$bx!hY&Q z*ZZlNDBI|o{Y|l!8?%p6b=ZF0bgmwpqer>-LZ?VbY^<%%%*2!adFO~IIow8$qA&## z)SpdF{aki@I7kwcgaj|8NSI{~1Q_>Gij|;mqxuEWnIlS;X`<+Y6tV*_ZGQV zv9elaZpH);(p1O@t(O}ZlJ5Bcp>U8+*wBz9e_bOGlqX`DHYqjUWQ$HG{~BPaL(k3! zh}<9GXmv;*@YqyzvDz34R-s>2c6M2~${w)ARw}mfp8SfF&HJ6b_s-WF8C=C(aF(`t z&R$}s{2a}%FxtQ`-H|L{L!>(KRXo?P-DpLn^CDMjc8;u~(Zh!U_cWc9Ab^N+F1ERJ zG6ej{zd)^$wYa!=w_w~3cWPe?4BsjS(&FPM?Ba%1{>yM{iDMm_(ik&*P>8umlrnbN z@72@Us_ook6f$Y;+)T`TM!j{f7_zfvL(h0cH}uB@UXcQO3&t~N&TxagabDVae|zwY z(mnMv6OmSa#yweAFTV}2Ao=1#z8mRG z{A1ld;P#e;@}Zs;`o_@P$EI^0e|ykY6zNmM*TQ)-l&!U8*$ILYumwtZ7TZSy&9Y9C7&gfHcwPQS-0PqaV z%Fe#3t^M{!jHt*r;m7JO;d*2kwN!&<6^~|WUmLj?W&a4dYXu<-&0C#HN=j=3roN`D zVHAa)rTb61JGZ)3j!NZj-4o#b(yLj;#3ju5Tm@HZHv5sciUKrO3&FX{m+!X=FvrYx zVv3#yri0hXY?mWzw!g5`xq5HD_Jxkm->&ZWvK%W*PkvRTN~p85dip-*Ci{h;sqq-A zUOwVteY8!i*Q0xP?$iMLmIlZpUNv!kB;(vt2OJ8LZ+LTU69X9SMc`obepo zAv1_*Ik{%!g4|9yrGXM&y-lWg#eSkG#>_|sAe=EY@EP(7Co>I8Mb{}x77n%Ow;749 z=Qq_t1?EBZbfLjb2iORfYW(g#nQm`*A7(F#UL6T--l!)nfqI zxMZ`PHhY^BUphy^3@vo+6)uZ6H*cv?`<RW_d=pp--yTUpLA>?qt&t*AeGt%`Om>U~p4;;;Rji z0-%6aA^t&A#d^LbJ9$~q)4oZ z?M^8yJr~=N;4|%enATd!nQg$OfpJONWK`km&jLf$YQY`Q1!RhhAO5wW{e27dm3KRM z2+gBRBUCdNBcw#%s~%t80xI5aUxSpxQ3=dBG`%q?A?wXSOs3IL-~1Fp(PEU1Q!5@J z_C@gZi`ns={&{EpFJY4z9Zxm#EkB5SQ2tVj)e`PBH%_B-yT`}dyYG=@w{B?nP>!A? zKQFKSZgpWS>~gn_Zffuh?`D@|qv`<6HaO~#>Usn3IBD2k;xWU@e$FBeTi#eU0!zMB0;_35n`8CSIelYq@BqR)o)m) zhllJpA4o5ii@embHdV}P7nz)2zwENx>GBF-vIg8V1)o)I!S#;}a zeim&GZg2KHf9Yn!tq;m-Yejw{Q}2VNVie`FnK$p1ZYgbFUE!K5CW(qA{991)JGPhc z7cb>OXTBPRCE>*vJkf!P6@5aCTKM7&Rz|gu?MF;fQj-5;!iLo2q6+T>oEJRk>{b)^ zH>$fV>q z9iPtq5xV{RD-MVe*~~kC74oX~y*1p7zc1;(7d^a{BcqP=4(}a$miOuid*kmXY8(@tZ_`W$=dHh z`7m!)kM@*=2p@S~!6&%|U})wsra6c&a6|eFC$W6O3_;eCt!Qvy8W<)d=+t+t7F&;0EI53zRQ00y{6Mj+?q({`|HZBJFw%4`I~fnzXTi&(SEYR`Ql(QB zV=@TtF&euDxvmeG2P$^8%YZR<7TU4g?R$ynatKtmg3-PWzc(@%gYmrT{Csh$dOnOALmalLt4 zm3cC(_o#UEYCvNnechE!HHU3Xcq(#dEw|L8|D7X0`~n z`uo!O`&o&Vc{Au{Os1U4JZ?D=#jy9dBZCLKw>AHfd}oPMg+w9E)ZA6d`VLF#o-ALP zbOYBrM*N*^ZJ(n+7#BFJiAH&yAS1J%fS|QMEg>Pm-5o1!a5j&G!W6UB+T9T@b_6=T z(iB}+vp-J2ZPa=oi^H4Jc+URX_3N|e)|2-pI_c9{Ti%urBm1B8E=IF=5mdf8^TNPF z8fx1Wv;mK9h3oFZ8$%ugtB5j)o#kB#5cPu*B8Sj=`C+;)G)shB17)bdp633lCXDK8 z_7qkCn4kODzG!p-?fv6w6`hD@|0OuON8f}hjZ?C^eBC$CkZ5_r=N!vvZoT@&gN7dn zGG*QzahI9c{2D41-P3%UW?)Z>L0u##^XyB`>l8PFj=VavTN0*sd%icfNe_i7(IX_Z zZGB3$yu2*UlgOi-t1@_fq66Rf9-~@j7&JhM;1Z8Ma=17~PuIv?eZRSE9^>BD-rjEE za_LP|{4o*3kRse#L?@;D;Bd^0;pqMC41*fEo8X0$sgCb%0rI>ZvA57isg7A&Z^K6@T~TL;kHX7xui7k8ZR~(H)vNMjJ1tvqOe_ zbTPJ7>}Vr}u)Isv1}pN>5!rZ`6Oo*Dk$b+2_i*5&)&BDPXIqy)?Bl4pmDP}565QRO zoKF}NgLM3Z5AQD6=uo>|ToOzOIZDUgh zVn4Qm1997&txvGG1+9TM&wAIBqwV!sMYZQsyFtYYTr>my*E7-TdO4WuIdXim=*5+v zsMtT|MOn>>D?*H_RJ8U|sP>PPo_cy|sw`-8?c_8u2Nij#FLw)X7#ELE#Y=~9`m3k- zh-~p>#@`98K|fV|fc?yF9{Q1zsE?+XcDF&6cQ;LgJ!ZRa;@QU45wmzIdg_;^CMI&! zuh%r+Hl<`&@xGRm{1K$N%@9pC(aGpn2Qxc+YsnZ6-@=~P-D|V?AEdgtCu%ypJj#t{ zTe+)MaJTM%khyqF^~>3auQe7Xvnm(dTJty@7QX4TFYOG9uFA7Ti{2&gl%c>~5$ygA z=E&p$=F_IRaAlLCzkDlt@!L8}w)k}B&feb=?U^f^yH|HZBBc+w?cw2H3P=e~)1)I1 zHVPwlQ7yQr;>`5=Z3$Lf5YC*amuHf zqjPtr!wH?1STXPpohQyAqGMQ!(T(3$^Zg57lXDff$#G&7Cj4_{$A493aTwk2VQ2%n zA46|$VRTzBslIjHl95<&_c0+=DR7Y-f#KZoV_ zb^yVH`0=Cos`Q2LxtLy^$v8yID_K}7mzEMCGC)&eGOxXvEKq>=zVp8ZKnT!GnIoPv zZE%&ilUn8Lm%5J1)A7FNEJDAGM-dE&3jHHI`Ri9kS*`on-6Kw8(kh?&4U#$<{9aME zEHRmN&|Pyl&hamo`Qw6XOx?5zPztGQa8B{;N=hncr8B+JvY=HbbY3alh1&$EArnUyQv1O&kb&?cqCKUik1%Vs0;I z6(en0D2M9J8NFAua(kw4XOkg!;5^#VKG|R7zm2n7{SkZz17ZTv!j3L#i$d%H-0Qx= zr6b~{<6rAB$2f@cXV7gDGeaoqg$@AsqCEb~P7jp@Ww#=YR&F8Pt<2qO)>Y)Grf{C9 zH@K}t&B`CNXUyP0m~DSXW@W3gwYqM&C*x}T<5A*38ncU3-x?MrISQ`4PvuaN`RpQt z-2pJLao^-TSz_`DrwiNtL2>df%tMGF$I+zlohi(<*Kk-2K}gE+VxTr(e}S6Ynfe2V!W zJ_IKo2Ei~rEd?sY;fADDw3v|}?iY1^E>97&=$Z&K?__1G8Ty>Yw{E7Vk~$}|Qw_A# zj7o3cKWk2bno3YUggNIy_Pin(Xe|rSg(OK`wCn_H714WT`O<#&X4K*6m7`jRF%j zU`5Py;#zUp_#fEnOomQysPm%fr(X%(=VN^&E|-R4g@V*-cWzNS2JN`7ShFlzP?;uj zimr8|TqIpLV{JCx@16Q|IHnCNv1w`BL&4CkO+R>d?|q4PCd{;J^yCuqyoLcya;bj0 zDE^#hJT|#Op90z%XgQ^w+$<^TJL-+uPYoMBpi>~~6-mY|<0YEo&a^gldDHje%jzsp z)6ardJrJ}vwQnUezOq+Vw#?u3=(V@E&)*9d{#`5kdq58rl)E@+^*74u;k~ihR6M%x zyE|2Ed;29lx^s7DJd2>O$#InjDEhFj461hyg3lYrdaiRyd-TrPF&X=>vUsPYr0kw2 zL;Mbr#zJGI``e`-3JM-GH7^|h>&E^0it7rN@+}o9kA_s<=R7nu9ro&64i&vISKcSd zIxo{P5BGIG`o95ILkUtk4{wWZO)?r783lj(bU94a>G3D~m4dyj=xAD@P?{tOBZA8l z6V-ddQa`ZOt-1a}lbeMfKh_NZ(4*=!Ug<-zU0CFl5`%~qd`wrgVUUF4l-UdMyZV zSmL7}f2@xbXniRZrvJsw7rKR`-fiD**(Igb(ZoiYOd%MmgbsoP^IvHB$u{3&R=RkxTcC`Sl;Z9L)A2TidOy}-Na z2pMg@q1T!9q*}ZO1XKo@ud{`#L8&{qoy_crxU;Oe zw-jmtYvd?2tk(6hpJJ5t^As)AQ#5KVD+oDxwe2ETnw3EzbcOx!_H8*WCQ}sjRympz zZS*1p{~m7u$h?MwcKRSKSQxBNLpR?!LU>*u&1bWa{~d)rIpN2PZ4YF3J{=W|1j|RS z*{U`Dq%n~!Ca;JIP$s93NyCyS!QDLGgg&mPQz+oFUO3oo$^1AuJN;g&nt_5rcOc5` zVS}jc@{0@SW~HSwmN-MzZNdmlIn@j9golUsnfw=ssO&7UCs`crI7RANKV4@2tS8Sm z4W8|-QNNE8vNe@4P451xa{2S%#W!mDUqAD*s!#-jd4Pkox8W^S(;hc-s#t3;Fco44Br}sdnt~@w#5T$ai^98*2FznXC42?8vsbf{ zYyF*97+Vf;82pzRvxS3Z4p3LTH8aJ3kx=7>@jUYkzPIa_kF+(T)L9(`y`@|2X`@1e zoAP}pK)M{-Fx?^8|0j{*kMpNg$5JM(;W+2}ikzz6cjl4NJ;IgFa!ra}mnOIR2}9ZK zG`XCI?_6%A&KlJQ!=|W0y8RvPlLoX;GONwg$YrB~u8%|YU&n-q@(gnHyYtru-Th;D zZSm{w4H2F<8xTI5n;yJ15p8vC-PN?Mm6SaLZ9A`Fv2FSeTC?Z(6KCv#K1kZgz@JV` z12p#fgI6(+jf!W%ljLd%Tw0GPJJ02x`Q|@vhFi;-fNZy!E1g4awW&!yPUSTG-WPc* z{dltDt)}}@dHwr2v5(aI!vZD02qe8a(4n1&5-Fb!Vtx_|Muuf_V#d93_`?PVdf7?r z;TCqSKjO^MZ=wYxpZCpQ#oRwvp0Y`jnpANci?{KEeJx|N`j+uIcTodHUB2rBG`&r2 z(Jqa(JPfyaFEBtD5XAMmzduS?)ggLb0#PZR1DuVW2W>f4cTmLeqr(RwZ5^Fhbhu{MV7c*H>kp*uLvr#BP&bCm#d3e=cl>=5KteZ5 zHTpuV?vk$z?B7NY4vg?>77OxKMA>?8V3eyH@r@NGz6z)pKCF}|OtLN|J(QUp`HYV} zme?4WY(XHCmo_vs9F_+;dd(WAGFr;p+q=l>?ai!4J89{sCr+HO+cm9j@3B$sjA_j2 z*tuE;<~~E}!-m#I0m>1$Z;V?aJdwe?h;U*!0UT&5Yb`!O$Xtb6Mm*=VT zb0I!ydv`JaTZ%zz zmAmWo4@A2FNIeSfXrYSo0+qCdTm8EQml`Al*MUJAHLisJ@3?_m;Q0V0s_p{%VIH&D z%nsf$>7|s)7@9llvb4m?+2?JJHPmb5%A>x0y07W7mKEt4l429q^|)bkzsXd&XwyyIU-VkXqy0vD!b_}*Wq{G{iNZ~$ zJ<1q6|Dl$fn>h1d0N#AhiJbYKCC!VJz?DJf2G+aDo*F~$-PE2MMfXR^mpdP2uZ$4jDs zqZ^I>(pL$4hq;)r4m;GGw+0Khs8^EXwwFJP?r=XhPz|C!V=PJ5w<^9$6W8DEz6|X< zKoF$<>%q(4LfAZQ`jWtA!`7Az4yt1BQP#INVN>09YmsKJu{LJTL;PrSf8P=n6Z80_ zr4w^aZ$yu`)_<@_)HVjPuthN%BeCM%f#4>tvPMd}vn>c#x^~5sOj~;~q4(-iy*cfd zM{gy+93P%p#Dxg0S}M6C;^s(*cy^9CNg`IXl9s)v%6=_KW1)PRnk=Hv85+9)@!mhu!GZKG{QY-$tqEGb}r!M=>}qmiM{ZCkM=ev$0Nie z!`r_K1;iJ8lOMl0yIf`aq)crajmz1Qw7Rl~nk3wE5dGmcc0iAj36aXO0{x=!^RBfO z@gjn)HC|bMdAX|=w2WpNhs7pQsTI0z?d)w-li=G+T*?n|Hi*2;9NI_nK zw)m=XS?&iRQ*%X`N+p7D5_z7gYhrqwh+YpJpsE3z##?jsz>_WaH=CBSs-h3AVH1b!67&Xu&gO+a*#QKduIks(2}W zPz|w~;AzKJHnkl&$NiMVQ8Ag0ZHHX^<61YHW5cfh*z?-=XU?Zcvu;@(pHU5x`cgs% z_qgZxazULHyS9YT3TSRPJ~pQIL^=MAqD!xLNfJIvkd*M>7fwX5C7g>u1GMoucDWV~Pf!l1B=* zwpM8bRy0(P-on-gdI<#yhGPp8ON*K@JSg(Iy&m; z*V|Y^s~i~|B(d;;1&%`-2NpMuyFc9z1&0Jbm%ftv!+VszCl%PaBi#JREvJpi*9Mr? zy{C-qMR{Td@@8ZV4Kn(A8)q4HjO$~L^G0N0H7|||3Lf?-DE4+PE;W^e=vrhHiF1J0 zd7)#^cJH58tQQO9SA#fCy@BTF51DPf%@TJA7Qt4?G>*0IE_)6_RTZ{UMcKAAu5)Fe zgQk7_S&0+2deL4dx9W+r@3VgXo0C-`wI!uQhDo?}D5pF-neaaoLiLNd?Y!%O;kOC=YXT{y;Y}*A7=Miq2iyOaH93 zV{B?sz}DEfTB#D?md$&caN^Y8;cyFcUr>r-dvqann8E|Mk)?n|N{Zk^zX zxpU`+Pl+b>h(!RzD-*|Kz-dS`s64T;L)ty%`zz;rCGFjc#y zocbSC>0f`PGi2ADuqd31Y1kBu$c-Z~peSzhK4 zs{R86NiF@Tfq%bwYAeaNS_OvMIkvI;8uDCW<*=1CJH+d0w^vlqgU{NIB4~Pbb{s@2 z1gsu!;Z0UH?sv>R-iZaIk118O=WvJ$c7L_pkb};i6yO^f=q<9w);D3R0$}lR=MzP; zdg*FO36`+F0;Y0`L?nzD75)ssn&oSYN={V|tkxDDvU%-w9TMO=>?0p>PtFWff1)@^ z&XoS(3wG2$X*_xlTu*jsUvI0#e(0I<+1Q84RW}4!Yy(Rdpmr zpW7vqS1956=T!&e_cbE=kKQ=7S7kiF-f`WYJ8M@__*Z=s0+8c`%hov zQuVa@gx9i~dT|Htr-HKe{t;T#Dt5BUew=9OaQ9f7D+q z$~2;S)=IAH;7)Jf%7i1CE%C=~+_8$Xj{TReY+O9t#`d|MXIbc++;0vzZaB4TDta9y z1!%~egJ=;>L$-&lPGxY=2cp<)VbiW(u~q^(;0FuD*>E^LsFgC*rWDDJO@T5f4=Y3M zaI3wwaN(YTs=VBIN<)PiHmtAvG5|A)^9Mm|M)5xcto+^Ch9Z`-S^y7mP&4YD&3O$M z-*8Nx0{T%D@aAoN3?R1_31g4wJ=zf|3j79SH|xfZU%?!6HxQ7=JQ1i)<9kNzYq#-N zERj(h-1J2Lr3B{}cv$4*i*SlfF04kswn%KPSoalmq$~n>xMoTI~Ag9+YdwRR#|ZNioEP@h2JemLB9DwkB{yW>MB zvJ7Zu49^OgDYx>K)#-fo$=sa}$FSlJQWX|}H%$u|wu^9Kts3(6mGl8gvEG=L^d9J) za3Dtwlru9kkIY_>x$Zg{!IHg70`5vbm!5|$4^hqes#bsJW z!zsg6UZth-Z;>vicCDy5)!CucJ?s^$n|}^NG9T(g$(NXOr@9`qdl;D13;ueaNHDNd zbVQK@8CxYZia#{!oh5p2y&F>sdvN2*_kuPHzXJCFgbKMFSUoCwdZx|I+H}{Jm}GN_i@D@c zfCE*jH%I}A0uIxfz~A~lmHy&&7tsRB8|Re;AZuW^wPKL0H33qeqn-hxl!@Wv29oYGfVbd@!sahnvMqrhlpxhINbhL9yj=1Y zpxeG4PnGdN=XHM-^QMHDZl+Bt6TugHX!rtr*UsX3(gDK7CW*w>d^fnVf9>pbVyR9; zOg}S$`3t@lCFkwz9IjYV(VpFy@?@^CAjjAN+A{c#EPY!Sd%V{SK33N}B+g_UlNiM@ z^wJw*V4*kx7kJ=tis=~D{8n9q(mx$t@l+`V|L__LpurkrFyptwGe0SIx~pjGR!2Ha zfg!-4MnBRE#Q+Q1rzEGyV+86iR1R5lDPT|*V(SSunUWLJ{|U6B{27Rwo?_P(5?wjV4F*pVLsT`<_Nne>9q8s z0fE^9d^~X*XoPokbfWEk>d!oc>SD0~kU94LHAXI+?kgr4I7eoMt#|FrW$Tj%O87&n zIrk{z)!&F4p1K-r>DTo-?$>%VVZCj(Tu%Po&0zWkSh6F74YU%CPBNm@eb9+ zH}1-#MI+$wj;##rc*mvkLc_lem@QSEWn>)qO}9BWS%%AU0}?S#7T?WEWxYs!(nE*~ z`I#DGhw~}f@>@AZAgsxxP+u*E%66vo?ArptmA0!rYws@L8;_3Tq(=k<1iA@{kdUOT zAGHs6Rbso(f%}TJ5dhtfj3+tg5U*I4*I*w}IUl<0RfAt@u#7wDm}HcvtF;6lRabzl zYZ@>awub&{4Y6~9=n=a!)vRX+%P)95Yk!MB&9SX(WW?|0=7yaCZd2Pj@SP92`O+V> zAg+R_N6C3pY}f5xm(t~6lAkR!F95s00Xa%vINg--MHOs`jT!EFB{D!@*R|{LB|~;b z0lAfjx0iIxo+(A1O?*RscIFaAd?%aGZ}11ilfN5KEK1z{6ogHZR=!~5dV?MB5mHC8 z(U|5$uY`W{8^3Br;h(wtZGA6*i2BUT%({FZwvD;vQ0HfyV{R38>4pi{#uSF{v^T*o z^i|Kk#8tRt?s%L&p7!5^LVvfr?{QG2WK|v80nF=hJ6U`8B#NQ;O3TPiLz4{IJU8@W zMypjU@*x&hF=?5SgQOtv%_;I2Rg(>2gn1qyr{VEu8(MP9iW~!OD?lHBPA=qod&${5 zAIP&WnG!V_(7xa4O$$Tx9ID8Q0;hwv^PyX_iXoQV6|d!Xy@j_c6!3KDTvbayvJsy^ zS|_W>BZu}{#;x$R$&qm8T0tNw#1#@4Uy)ygKgd+-?q_DK%4)EBEQwX5NnB?@0byP0A_-|SSc&sc2CSf3cZ9dw|W`D{nrI64#aGEi_|j$4VphFY+lEJbnd z^ree3c*g}`p+%UL-<}VLUsm;nmv?LcKQZE>K?+xwMMRyH)UJrWnb_g_?ib3Rh{wM} zG=Ii2O-+f@LlEp)bXVQ#l#M6v6=N;w0vDM1^nerv}!o2hW$oAWX@pixH3o=Wc zlp0&<(l1fW11rGM%4yiR#)%|d3Lz7(SvK_Belc4HL^Clxg&M!E{onx?;16wzG#0ZD z(b!Q#tg5EnYc`TALHq5I==)A!cWb@@5`ikqo?P1X_$Q2@MN3oZnfc=J`pom_<<{Js zKO`;?xcP{i!cdt3Ww?@uzU#cH0Dj+kb z87F25kq}II+w|Su;i^o3zdc>5Ue)B7*YwA z_ECV5FU)R$|F3yrtd%kkT(+${lB9;pUEwA9SV5T_>Lj2{{O`Agb-WdM8X3-i__N^*T zhf4ZD1iQiq3?e-iEo^72B(bSEvO7thLSV86Ph#=TR-BkvV_gC*lQ_`5HbhvCwZsO! zW-AtYTHwlfPhFzirWHCBE)K;%y+yUX=rSPKCOvu$TEaRY!_d4_cA<$AJA{!G4owE* zUoD+JE23VXfm!Bd=CgfkP=CG#jCfHVnG{Nyb^VAE8o%m3#G5s!^wFGVx|a$8Z{C3Q zO$=*-*#cHZCeTl(RO4UyfQ_b7-Ad97_&i{JL9bT({{7L4&tF5S{qVmK;PqaoTucV3 z+Lx7OZoJ#4bZ-2SB_AIj?4;pFwg%FsIK5_bOo(XCY%~O>Nl1O_Q^3mVURWu)v9x+w zU_So-qs1AGZ7|)TBUnCq+Dm>92}0rix5+=u8Jo{e1qXSf%0C@^@#Ou3vjG_&4;E`& zGW2ZUaa*0%OnIsC(|~5sz$Bv$Y{&a43nMq3BO3`kF|FV%PC?F!(94;ypNXo*=vI6= zde*5+S;J+hUdg%z_w+xE2#i5Wh6fp1~qb5{!UvBf~EqfU3oUz$sFmXlGvD8s7dTsX+7sAzu zuYTnDtd0`Hr9!TxN>?R`k%2dkIy8p<^DJ30fjLIzF9==792s80U00gQ7A&OtnI-%j zBLN0@$$l>8NY<4glUzS9Jrf{_^m9bixqnr+XAmh(xO#<%4N7o0-*~PKPbGhz*o)rj zdYtD-o@}z*>3w6@{xRqL6$daR_ZZln5t;jf&<*~5b_`@tTr_FNvH4-j3^BIA>W1>$3Gds29k zy~d{>wztWF$x25FH(zuKXJlX!*kVmiLiC1_6TcVWCJ{df-5NM*iVMr-cw6iwAUy~7 zb*}IvnmyMcwkF79SpPm*{y8X#oT-t|84ml6`ItrQ%!_KJ27-FH#rNSF~c^VMZT zPDqUqpCJ=huQ;2y+J=i^wl@oI+j2OB0?7-3glXVFDVAU7hzdm+3W%m@(Dbb9{{!$+ z$+mv8055saBC@eKQ@^6p|0gKSo_kOm#T6NJ2!P4KyMxK#?EQWDj<@@tL%3k|A*C`k zjbDq-Rdf6z-6@kH=+tj5$HiSnBAjV&!4D9&(PuXh(v^L8vMU2|DxiCoU{WQG-ItNl zqu?g|kdlAI2@GbUf>@4u9%_8_=07_AKNJY{)3I2E?Yks9v7zjN7$#m}LLYcUa&j&) zn1&p&TUFJ+*Mx5slyED-d0Eiw``|Bu8r9=eA6&eof@J9=z7L$Bvr`$1Bfeb=@S<@g z(+epUa0OC|3il9!_1Po7ULfc+#(5}iJ6acrwz@ayoecp)V{trdrzPlRbkCks!8O;j znwszne0M4?vvj?z!ZhmS7}Vd*aptO8MmPUa!eAE_FZFQWIh6=0|1j*FM|$2NhlnNFF+OvJbsUMM^4BYc)kOj*1uXIwO|OCEo-*6$hkcn!bn{50kum>e z9c{(dkfo&+RUeb;=C)b!OhwO(GWJdzn;FWYH=o=(6DG;~g}z%+fjzq+)M1Z(b)R1s zcf_(E83B|I$fM=-s<2D_yrP+kCAZ<`e-t>ID@O;xii;Po3=%AdYhlZ^rZ+QNdizsE zxDhp|3Aj&p-o-t3ICkL=2ho693b|at_y}{|NIA|KsvhbHV{V$wy<~wd5Qjk4zZq;8 zAh+{GmJ5xEUhmr}E#x4u*VVAu&9Cpt=-v)d;8&F~77o8gkTWVoe`<(+uLtPxF$MM+8iLZh zeohH@SB7ur0#>6I(juBa!aLtC7ipjzd%9o_HcSCGSNfCPi2l{IY~tthL*y#3-a=za zNwj+rBP<7s{&*BuJ@>16;nMn3f$xKcEiL?!^zLQe^I8?Arga5)zVCkKdI{^I4FqT~NzmR_ z*TvdcK?#vArlh(4^F>^W($bbUHo#QOqYyc#VXu0PCcv5QT56r22Zj`f*7#Dijjx{j zOe;YTx+MlYkc#?rmHvadq!-}~?WaJ?WFaN%T0|_xbmx=lh)! zqCQ@a#$FA@7tXohN>kM!fLDT6Rb*q(UXJ#WF1BKMsQoYoPaJl~<(Wrhd2}v$B*L>= zYLIJgmWPk1U-WO6h<~oxFH+!c1f@WaeByxZ#wYHl)xjeXFV#sWy~8{5ouU(yg{rdh zRN{HBr7aiH-!vhe(t%&WCRi70Vl`1Aa@FM<5ml|GaLFLL((td-GfF1U?pd|QO*YE= z{QveDyg{GgJ>3d`@M*Pmdm9}s@CGJ|hLJhR6fEkyCa;iSNyyCs&BgR0t`T?@r84Ki z!phL6)7RWE#=3B{*~{#3rna!yL} zBcTV{=c9VEbiKS0u`G@V4`2by5pNtP9mrA%;789;liVj?c6}UxGl%1;KqbxMv7iH; z*Fej0rt6ziJ_aD<8KA+Y?sm#hGlQ>}Pl=lk_mFcSYf4JsnznU|C6wVOucfM$sZM}w z_L$NKRZH=Y7}12-DmSfY{sf7JFyBAvZSfKs=nWIsNEKt_k~e`Y{j0bB;G@EM5Dm zY?3+??@!U9PhjCKZAa#{-syvC{`zfdXIU0}_G~ZkITiIFccgLXS#i_!R}RaxN3*}8 z$5s4^p4B;bMu)tVW-CRd6=}XBlB<5(V06j({58-CdQNA5Z8Pd|r+&!gTkpKmmiqAA zUj2rD=^IT+%R zT)s@4AL_zD5W>a`N`b+^CRP3;Wu=ULrP71LxtbSkqp`q_tty)fpsR4pGfD5O$ntxH z!u&nM^v`Sa*QeJy?92=-9XykbRnV)!m7h<@gC>}7d8Ch#9QjrjA`c1!8lZYk26(ZlrHN#r;RlG_R->z@&aXT8MVOBuE(aReT*jQRqOAE z?{;Rbzy;44NL1UWf8g%LAnsq;)mVAk=*^2Et#4;K&*n7z*;{zQwz_*hSnV)ihE*dl8+SO87{Ty>AjZuf22sN($hGFbku{DH#le^b5KwND97_PPD zSoxBeE{D*N{lbpL1<=*H9gj~0m`Ta31kB~2r>EyA7|||{1`~zOE>0iwcJ2L*mMQSx zO!^@d)gErDW5RE(ba^ANin(r<-e?tp5(K-I(H|^ef(GCbU}|VpI#i&!7!904_}H96 z1YYNa$R59PhKSQ24g)29c^18bg9|cF$`2f}LEOUdn6mtOWMxL}9^9kK)re7{X)YFd zGS5e;ghuB~?W3u5ua{5I4Q`R1CKqF|_}16*k4FP1 zi>Lajlg;7oJ+&PmB6bvD(^$P>^Xc9kzf&|2*Gxq^xVYY~h>-C5L5s*Q{64F^U%Qxl z)&mhmNl*=%{>x|Hw@b3=a1ik=c=yAANX0G_axbB~&f4gtcg<+~dKr+^O7ypfB zU@%|>?6DR=Fp*b}-4`!#Bws-QP5(A^n%$WiCvb)8Eb|ORZ~==b120rZ_7jj`d8ZoK z!G+L$-_r0Op^k5>CkWSt^%OD7$OCeG3`~0eJi+p%nr?Fsp7x8cUs7w+><(B~n&xea zTF_|+1bhX!;V`t{Vg^yM#7)1^Df)tuLx;G#qTVM$U+?8s#(G9)N5{~9G$#TDG+=GE zKZXD8HQLs~#*urHRZ0+VOsCs~H@ty`&pj<0lc*a8RFMh%GDTa05;N>T_!o zE5o8sQ}6!4XjhPv6G;Wr>1@SS$OceoE>Q!z0o+{%y026`-lU@G1z(>R^KG0;QfJa~ zbapQ78+7@^Fl6tYPsK}JQ$V+)U)@=@yi0M~IIGMChGSm=u8 ziYN9Pdyk1%s9!h*)DB&POY{dN-Ccugl7jKg$0dC4N=0Le{8#6x+M`Pn1b@t?{UILs zYw%J;GD&N$r6=S!3+?EN|7O@-Sp{yKc4W4qvbit;Sk*Yd#g|{pfEbO(czs*KkLG9| z5Kuj6v5G_@VK!xP5?$TnkID6P@P8Qd>yKBbD^UwT} z4qHL(kM|)ymFt<7fu^2~G4XjNkoDfpXr5{7kWB0!ke0?}HtJ>R)AVgi6E=>YklE-P zGCSsisZ@#OkJX^R>B8&)W`z2^R&MΞq5do#Bt2oGaCD+}J zCu*P5KIVu(iDwx@?ZV1PNdt^bqd*F_j5YGyXz`i(2#^g~iRUa5`<-C3EEvs|jxZZy zwmxB70gKP~S+n+1E%`0E+bPre{;jz)R@9mo+TXoyh!8xGqXBI94n!CzL;6cHR0?vS zDU-%8EO!^x_xZIQuII>2Omx0$(?z!G6AO{jL(&yv&LuvF+q#?HbHx3!ESM~hb1CVp zO0Use+5R*?rnGrutQ|=y@n85PU%HbrHJF#}{HPVgj3Llh=ZA;E+t!`vaJMn+f@niW zLESWUtx+Zc2amsn_3;xY+EQ8fsCdm(4ubDS_R4AhXVHiS3tqH7!2>G+k8G~T5A;`j zv&89>?;+*QYF6(edw(&OAHz4V*qR|n&FvjpHt8pmq)VaOUKQ2bj*X!gZ3FC$r+`gH zuZ>v1Cvy+5%8yZVq&m;u5*Y_Mq$f)sj*9mgaf0yTDEDEI38(^08=bv-AfHc!-$a0{CELQ`Kt2 ziYR6xRc9`9RUKbp{MzHi{>5h}JVCIj8t}yuXfFc9{=xKc(z(ib8xI_sy|G$){_vKd z>*Z#u2u`z?2>w_mMAcJL7?SuDzVnQ{tCA>un)Wew1f^4R<&f)>DK%a`e&IUpB6_(M zm6U`J91>4I+tH1iOXxH zAgHP&;PdCt+fF!n`|c~Ck*JrEmBl?f2&~Q=ccQq|GTy&$f-oI@MfUm4^Vh2n?$b*h zfC1-+D#X`q57$bh+6Ji_*%*q-%PSf7U(rHJXi&o|*1*#27-0X?6L2ta)6^YcTMz?q zj80bDk$@cnX~W9GWc(ih<6oc5t{l0rIcsGney$L}vW6 zNT8e7`t)hQXD)r-=6@&94Rf$o_AlEwUW%M)Y_U4iln*MfC-%u?+8&D@hs+QmZ(i(V zI`?(wfh*m5(^3+`?Xm*ABU*8IuimejDd3Rkd~WLsl-ge<19J->F;2tc4Qk5CyIirbqL^0-fBun*MCsJNeU%uhk2_k`>uuF7Wg#JEs zAOFh3D*$PP6Z@Hqv?Si46|ZVEpAwKCnYMzaLIM9o)1IrVNVsfk6T5&zB2~RLcqO8c;vz7f)5U95UGsOLF*j!U{S8 zT+?EH`4d8zLO^f5i9zxb4xF>hhuu?~Sp0l17~*slTjRfkF@EtaECX3|;c_?r%VOc5 zZlRSCyX^tOHZ$RZ-uaOu;%`WnR|D*1$+JTyT}tenB%3kio|4T{-gFq!#$4Ll0{CDqDN`?T%OlZ| zO-JV*Kdtw@=|KIddra=V=*v~y2^4(6C1;UN6KSZX?>^bHCyVB>{_s%JWuDB)ZBz$V9p0U0KdVaS>bH#Wu|5cEfQ=x#1#n^ z^Io`yD2fQt$hNO2ONScp;EK2^_GRD?r%KiJ?x=K$K@6#={{GigY^w~B;RxLP88{Tcz~TZZPGmDxijRxrDA6kMEF z#y6&Oqd_*N$epEF##2X-wZM^ZAKcB}k)dPUC9foqxfdVfan@J<%Lrv}82&!+GQPFh z&WL<<$?Zm?3@VgF}HMa#`%XNNDUHDNJm z(wV*(CY&+B_d1y}(?LFC;pZqI)cu`L>9l0TUS=ZTv8mTRcWJ*+G=0H0q3~`}kHD*U zcI7eoDJHyvZ??*B;y}9^7;BX2A*Hcxy}g1bDShh$08kNefBazZy!+jM96Rz&sm8PN z6RFOOk$M03CQ{N-D`*_raI^3o7YFl~c1+gTy^>0m67^s(_0z*?%zd8JrDsuJ)!Dyr zGx9YcPlVo6Iy#ZsTm{}wzp#Rlo6+|Q|A@@Ekx7c0(C78>PNwo@-N><4OKh#md+=3 z0DP#%uu#l_W>eyWg(`7BA;zBQs{Zqw69f^ca2W-IlGYP`HpJCh3cCxZ3VO(PwVsPMk)anYs}Px>L*zEo)Df0bxQDj>2|P4O$0u~&q>1?zf_!B+u>tmR_vg#ixBD$7!xelgvYZGHaJ`F?f+ zOi}#wMB{k7--aRWy+hTq(iUT?V9YGNHR?)5O7%89$f8na_4#GK1|SW=(9qD8kF@lR zO+bR+06U`!mfP!H9!0PC zz&0EVH|1NkeOt=2E$N5!z03Z6WwP(7%ne6(Er#b%_CaeK;JbywX|yKQn(nOYC0?_9 z%{%ruo2ed8D0Lz^s8$$p@B9ki`6F?iDZ5QpTy)nLQTQ3P$gMMhnghVG$asc48+pyT z=rjAGw|;qh791WR)P(A@BZSAB!u3LA7Mf~BiY!GHSAnD3-@-9`^8gDAOV;eKqt0o* zJFC9&#yEI006MJzgoWLq63M2UBTPf*_XNzP&Nbm_PaF3zcb5+uy)h6ire6=XDpjt? z$SEiZtTycDr_=9=uR}G43Ye3g!rGag_5P(A1G6SR91dqFrZO|Iy|rQ87{=0_$4knZ zD!0;a@D#ih=38nu3=YOuKrDSeHTeF>MAyvLs{0RdmBE>fPcML<1J!krV#L6}F#G6? z(+L)7tHvpB5Cw^JQLjh_CWFO-o+Mxw!P{r-o(!y!H2@4JuMA){J#^!^il;(j@?r!6I_b!9b_J~2>oh^y$x#Fp%NDJbB zq+*qD-s}-)<^=o<)u6howg?NnoQGH15#nk9e3p`0?$xmd$avOZvkVd=1FSWpgILv+ z`u45tbPiQ1=s!pa*zrveooGwmPZy&W>G$nb;Hj>_wEMzw2In%id#sH=U(X+tA3{90 zcX0kmBczdc?Y09n^z#eEX#7~F8!tMq_4Y5$hNrJy-4A6J{P^~zpI>=OYuo1P;;hH& z&_r7z%@g^FwpEIBvCtfDxN3DmUHALaesb+Vd3rmgdf_PQcj(hg8f+qgi2vmd`n z<@KqB?r!Zx8BcA0$_m^5D7r_#SHS=IN3HPLNt>w{^FOYs6c}APe&gZ_+H88pU*)-!$b;})uG_A&3>ODQJv$X19D@MNTKJ*p z8-ER^fMYIwLF@1fjx4GkP_^^YAecxe+4lh#Mp%V~}E-HS=Ae z$+r3v9Xm*YK%E9>^R)dIFc7mF5zD8C@D}*~F7dm;fTI7m0ctXLInc7J%!+NS4Y0wiS@WGmLXTk!^c@S9a5bC*p}8 zIVgP-Xwu?`_s$WY(RtzN>nR8wg1~%vi(TF3AW?##I1`?W$#+0WC6z)Ix~xilkYm_3~Z{AVCMkN z_Fs~E7T3Gjt6+#zLu?wY*@M6{=rIUc@#xti+sa4FZ+?t7osN z0aO>@}b%i-Mwm)_IM(G{0a${Yy}3KAHr+omb)HWoxF zv2U07EiF2UFu+NYDDx6e3&22_$d6^`%K6umFG8ze6>l?az`8fQQziMzCWxAPY|?zD zz<>86*z0m&W_fv&R=AFQTykSTFz|i&na3}P*soG^_ZN!?m_Oq2JGZo+eRWi4T?Yaf#8jJQ=jruVH>mSxc{mq|E4F7dF_|K zDNW|3J7H#F9dsrSp8l1k!xM=J1q2Ab)RjS7vV_~}W~QkY0gMr0a!q|n-Zm&>l0Gbb z*p%ZT6Xc-~PEztJAK%R5M7meSnNt=~)WrwD{bm!2TP@`)@<+4tWEOVaE&Rsy2=N`bBu%F-IPchto5Z zb6ch0<~b2xtd>k@T0v~u&WozgsLF$;Ft=TScp5dS0Hj|kYgz^0xbOrx)`uALFT@!- zMFP?mgizz$3;6ZX^_Qpy%G8mXf&v~$0a46zB8}=AG~~PE9i6;2EZzk(sZh@#1ub!pWunv3VP;SKls~ZU`HP>Z~%wclJg3Mg~rJndWPoee& zto02~NI~4@7p1FukU(a~KCD^Yf#RQOf(OdIH{i|=C|eRziEEh^@fAiKEPq0Aibvtw zx4|f%)zV~_@%r_DQgHsY-PO9BX(|UJA-`U@&Nl>GNIpbKjUis4+6AZ2^B!w1Vk~u3 zy%mCtOwQMY3%h`n!;u)AFt}Li*j|ai`CjbA3&VCnye?S8e2TVyG=|5c3`-G83=}C+ zM~W;l$+XPnog&An0=~14VJ)q#O8!kb?f7i0A%g;O>zj;ca_EAUdFc{4E z8Mw$7@%IqW1`vG)M(*y!Ya~mHTfSZfvw1@6MX0i}heMt-0$vFv?P7f+pF?_vlxgNb zer9lUiriD+DS?LCl*FCA&{eKt+pVGbQ^9|suA77B>4i`K>$RZ18|POJNHI0B8qV6z z;>DE~N9g&FmG7r;@SKr~%bH!CyJ%o-q*P@0&5(w~JSc}h?S5d$yo*X}OG*S^w~kGI zeBPVh1`3jHzLb|p-RT-w?Twj247T>@u3vfrsLgOylRZnq08KD17XM4;3NoNE8zrd~K#_H3Qls{!~EWXUi z!9n_6tt&rcG_XQX`qFPC{=NM7OZZyhTgv`wo<&=_2A<;1eLEUI6r+xa$eWFuj5qQW zD~y|f7@%hUi1PU1)XkOVP52VX|KaxAzx~5H%kadU98iW)P0i@zZnKh7%ZiRu!BH;n zN+2OL4n3b)#DB>aq3h|bX39mm6P$Q@`bc#Cgo$Jx^yADXv~{D9c$$7c6dyPky%1l7 zfhv_eJA%r6zFlVl5=D$5*Y^N~w*(c~al?1KamvI%7*K5;9brLO_>_9fiiu^6 z&Q1l_Y~0x*OszN{5nGX!P|`rmxPJEq)xs#jqYS*a&iq}q?} zJ;q65k73D^O%MDOj=RiOzK9zw)R~Dpg(CYvuh%J%TajjPFKL85m3C9b{&-a;J|2t8?<(Meb&h?+1M=%FMoBs; zyazxHQLM73k9sr)FmEkze0{K;7Ctp?<;Aq8d&;Pvf+ih$v9_v}uV0#g$ONZ^kcZN- zN|lW6Xe`fOr}ztzXF8 z#H-}9S2rkV72q)MT=$gSYd2sL!v>PWPIQ%i2$68`IK^PqNF*vfDa4eh+^OGDWljhA z0Ct1z-S|cpy?bQZa?6g75j7j$KTcNBmhMjQ0-sxx+Hl12&8l_E@rt`?u)2Y+6(d2X zTgk(U@6Pn&mDn(`OaWDTInV{D)lBpB2ay72t>jb`CE03^fs2z(XZ(_mSrK2IUWOdT zhjn1)6pM>6uWoTF?r0f)Y%aux$Vrr3sceH%AovkmyHq%ut!c>Z{T=3rOe?6`MdPgrHgF|B#iV8AlG`ojHRo#B7= zAN=xxo`XXiYe|gc@*zYhdIg*v7~tq2QPRTK+0Q9$zX(4Rb~`!+1`1X8&AM$m=hQuK zL+ZiJKfn_f8#hweZX-hcVKf-#Vv=3l5@>f9c;1Xw-G&Z=vK|bjoH%a3EQVEI6L9o)1iXZb5HaXu>Gr$1g>Ta=#Jl12Gp( z#_mC&gcCFFu8R}c{;qEulxM&N(kXnyWOpr+!P9n;8l?Dg_-5>xh*I$)@j%oK7%cPq zOv z0gQf0ZA~4FFF&Z2|Lp}(SuLl0@7>PaVW6GaNb3mUYUNEC-$$bK`*L8QZ?(2OcO2QX zvpR9vCKar6gkX?!q$3xn}21v7un7iot`S)^V6Z$#-VU9;Jjo~tF=Yhy5#R;t@NlL zXIt_3qKWNsa?I_`nn1?agWVj={h-4lPLJb=h(#?I>JS5vjaO)kY&*_^5a7tKxwsP+ zhbF%Kr7#nL^Q9sbw&zsJyR$#j-``q0D_dD=bD0>IKlqx`xb$7Zb|qWJpMdP)NtcgtN&otN7GGhsu^_#j^EMCxPp?WKZ`-+Y}Fl_4FrLU zIr*~&aaTyoNsvc=!<*xhROdYEz`Bj#RRMumpe&G@pBF^-fJ(QX$KtnbTJ2B9hXkzx zUoFW+;Tn2>Ey)`8#B0ERF`)k?w zwN-6yAJa;0bJh?7A)2R~n%|Jsg!>%xtodyW zqzl|Bc%OO0QG;B4u!xeuysuB8ti26iK3Dc>1x-2ON7araLIf+C=q+ZF+S}U~s3}by zGK=a|vx@Z}5^cL4H57(GA}H%f0s6{_FD)gRUT&I`RT74?FD|Qad z^sNGK*VH;vj~TYZ8VSz2bufcrS`c2!MP;bk75^5WsM77u_T0{_Mt&keI&2RkmHg*y zmzAtk4j!dX5_8P9KS^w_gR@!Q&q;m7GSk+9PkH^Gt;wWnfF)P{cgkP4sRpp_q#TGq z!~#!fQs)q0hv%Of1&4&RzXfr0P2WESjK%0^Mg|8Tr4C?jb!K&MW$O$`-ejuczq;TQ z;pqfIdAOlcmyWl_@aj~woe902Oe+eh+#t?N+hTMsDoyl(;oWOpwmo2GH%y#uE{hkU z9Cj&EsAjV}>8pE!n7wvOU<%vzs_#x|xr%>3RSny$W83tTN|E!fSlnulqB=p2%L3fc z3c$&-HT(7cp(+0LDI(y2kF-lz)nNTSr`r2$UOT5seO$w;UZ}rcqA8{AkTmxn(C8`? zqZ27uuKLizIW#l1F*`)*E5aWaxYj)^aD5_-#bTB~lAU$oAdQWUWf?V4ouxtaP8<+M zh&h-oKN)}lXU!p?`H?FtW@RR$;J=G{1BWOeFH^@>lX||*0zFovJk-7?OKMEh2C?2{ z+3@C(NY@tdB=H23`$VHfkPFrf6ka~~CN(hd$wWu2_r^-|04S#mzAmsx_x+mxF$P5S zvkV^ry!#n^7fu%>osuqOTZ%vRp(fjbcZKEBK)N?WO3lgubQ}RhtU7uN;+-YAb;mu* zBNb|JFTt`Y?5KJ}AyRx0KXf_rW5g;y*!wcm-rt>Owj58wclkVt?i2V^)!n>6lBIfF zJNJKMg?dCHnBC=SON3`krCi);x7IjAnYV&$9vi54o^L+u(&;mer>js9@1UdxigE1efVUk#4(9#nr% zAg_wmO@gHKOr+osDYr5Z1wWcs+!5v42mN9?9JI&N-h4m`9M(6P%f0U3Bgdb#586nq zd)E=gxzgScw?QC=c1aMh?JI$8sdICyt0pe9-(+>>&KzCyb2r@Z0Jnr&#j#OBT%15+ zg{Ate+v^|{^}>7sLEnyHcEMW9Yx5Wu+k}>slq5g_$2IOY^8`k!c;K1>Qlti3eTKd% zPQ$3IG4W5(J1}YU=(_Ty)zWDgmvt>s*GqO_hlKOA5;Kfz)t1+VJZBGjTxK6o0bF- z`ESNvC0x8VpI*M$#{c2$%;TYa`}Y6+5~b3D zDA}?mTlRe^BnhFiQ+CGKjV&Yyxm|j14>zcD^^T2$94sAi{Vn@C&NTNN*;Xh6IrsAz zm0bRuFS%<&NL>?o!F*FFy(ZN$r$#iIEhgOiLk+ouCrh#DV zhN)osh7m#3tu)Uq@vQd6m-t2ibyZhL5?{TyP+d`01anK@9vhZBWLii{#Epv~F)oM# zQNw(dFVB(zsBka`Vk@fNf5z1xL#cF51)MF!qaB{0n8oqSXLR4a-8$PLgSQo5Eb8q! z)$>c`?ekbAKrTKudIPOc1^72b?-mvc$M15;ZUqsl{s2~pT5AHq=jUPXbnqu>a{S?a zA-PY5Fupdx897dxQP0UUPMUXucuGqz@flNQ%Zmy*+?(F1Ua3~|n3(+^Kt5x%O zVR=R{d%L^$?<1!K$d6m9^eQqHTa%qN{7ZAeDnkU|{$qRQhS^+g=kqNL_q!Q)!#Ryt zGmUp|?SYcFtFD0wPA#0%3A_R*H2PLHmiKzXK?i~JAw;R`r&dbrh*8_;p^LPiVXZ;4 zj(L+e6AMR8Jv43tGpb`;d~Ej7c#&8w;#C$gH+y2#;&i0~Zh6GMAl(0Sx!q9t=vdfN zD6MeIpt1i~;2hSBwnP;}qF>U9YXY%4v0fz6S9>{Wagc&>i5%U7vrz zC<(jm17Py4>b~1ry}n@NW|2kWQ{mNp@LeK9HfhTw8Pja5;yAfVlai)r3vmP|ql@Qs zMm1=P5`@b8PKDnw4{l4Cqi>Nvy@lT`rb*CRzGpbJ+XjXDbVns_4#;}TD6;wBjGLEU8sr@$gU6ZXQ{I{SbLCPmP9)b#r>{p)EJ6$ZyooQTNjCe}|v+nqRmyu!K0 z!zJs7A}pQT9`aR@2AsH{tCP)yv2WRWk4k$!X>N)Ah#hWY>z5`h*jTxxvA6^I!Pz5V z%E}B{V=A`iGl%c`ZfN_mErZsLp6iD~Pfu$3L(M1Q z*{tLHlJbTT7mFAcxWDW4pBI$xK%^@2R+ZkgEi+ej@L3e*i12!o_})#Qhm+!VRMq$K z2|o5Ro3B^8)!m@L-k_7bltB2`7Tf^YBw0;f&%kBK?|J`Fsw>u#1t*9>)ThF zpN@dXPV0e90~&8NY6!UpQN{WnJstOWbGAE0Q7{eqS#pkDTB^&kXS9^G$km4&nHkiP zM2lS>>pK|q&BO;J63d!?8yZ*cS5NDrl*N%0D#zn)OxWGPfd`~NKu}qwuY+QxSFe~Q zVS!>_uWnQQCU`i%PR3{323aUT8k}>4$i)^o41*gdoPa++o`k#C{8ffgitezmu&@)r zig}DI5cX_`akJ$t{1V84$cSg-wpq{@tTqqXu#wZ(S9H`7dM91L%IvCYM~x%sdze0C zNz;=PgK}%CqGcISf+IRyx3J&2DKEI^U zqe*Q;o{W;~AT2n0!s^{}ig1An^HTr26Vf7JCU{p_C(abA#tv_aDF}4Q zS-T7dy{)O;Y00`MkrRf61sSin+@7H9KI@g}St!Jlrf(Elh3osU3d`#Y<^cu3V?|ML z5dWU=4FYFs9!nyyN$4p0by&<+pGNcoC69wy zjtQ^uhSdG`LrYTFwHbIHnJq?>Js<1MhBPt zm5Q?>D8q|J-5VLa*blBHTK>a^N=^rk=?OCpI=?41W$m8_*)(TmtD**)8t?1Utr#tQ z?*4#Ex^v;fX`iC{g?@T+ImlH!$`Vxk-JWl8e`g9fgq2ZDup3t-4%|`6JqROll3?+& z^{8vTodbCpS8&lE+dqRrDRLF_!ZIVg=><+m!zI%c(d~on`A-LKL%j;~Ir6&FEG!o+ zH|<2lvdtCNDQ#)yqkHao>t1V*ltf0{BjO@it@@xu&BRvV&o%es+WXB(xl6JcHIk!DL z)Bftqx02`!T}bV>I&)b^P$*m%$)n7zgOd!n_T&2jvm#kV?1nojwm@aGq zA>ZzH>s<~^$iqXGT7NmUqXk?qz6`rPXtj0caI(s1`M6<7%S#2+RthMEYLQA-I}gX} zZCaQ{D-dj7V$s{kzK4hKd;TKwU6{vTcUgKKI^&qOKYs%?g2%07~loRTQ%wpK87=Jn42A) zH(R(K)~Gj*qxkK#{A2L?!yy26)RQhuh>9liz4;Qy4}~xnULQd{7Rh4kSGMu7hPvbZ zO?H9XHS^;RH$P?Gs#Ixi!CgMgb1X$MOM9*TeRSqAZru5WERIC^&?KBjaZw;VY^@z= zq>`J!-Ec>~Eum2Mi653&Nq^D$KYXn3AFDjpZk25@hhLXPREVS5DY=3x1(q&}t~Z(x;K zk5Rw%MJoK$YG9vmIcnhNT4$#m9?tfV47UiQ_LegJa?@kaBOI|rfy!rbW#4yirS{#= zDpUZ|STMeqZDcsmCpD5R!h2{Wnm2=JpKv_z0cteod3!_xC~acF#0SK={JG9uJ(o%nh*nsB=U8|2UD+r zq-R52^An^Xq zZw0Rp=Zxf>w1`roq_>_SfAvjWRAFMzudu|YvME%1`x-V@P;uBp$!ueO$$y}ROz0Pw z8Xj*)p#{M({Dx`&%dz0^BhRltc+lhVd|cJ(``enfRiOo`I8iRH)#@wRvC#Oaqh%AP zye3JLiw$nN=??5Q<#>U~0(d5C;r&k!?)}zWQy0D_%msgDpHV>o7mr!X8&0)Xt58ji zvHe;FoB2RI&&n}H$Qgkiivqe>Be8h=b(Wo?g_y#3rW(K+Vs}+bA7X(?PrFt` z6)4i4z63pkNvt7%yg(-+s6TUxv}I8YzRU3q#Q&OrItF34wY7b-x2>YJFMa6{t8NjI zIAhaWu}=e(S&)~W__9(hTE|}1TP&A?id}D}=E=oLvz)k}#2 zY&Se3f)+bRm*(j7a_U6DIny6gv_~^LI~%4Cz+<3QYune!;=mGFCiNpuGGU1vBmPH- zSM#4DMopWxU}*GE9IE`Ysp7keK!#kS#p+7114z5JojGS9kpXRNZLtGQeMsbeNrO{# zVoZ{5OD+ZVLoSYs3%tEL_~h|@NwM)0Y`J)oedic z02uu|0ahNb%hkJCwZK)&0qD%WC8G&oPmCQN+NJ*`e69|dhBp5xeR?#Uo&?Tie(!z* z=T}7~cykRG5%Sv$t{DkVFRVs&Frb&urb9JJ#!$p=@G{{@0*t}!0LrXIa2Emh!pjPV zn*?tyaFw?U1G82FXYa#CU{d-h;I?EyFu!COB9I1|YjXw!Hx0_uvOvSb|UOCH2{cb zI_9E5vA%a3zxl&u>-`yF+1Br$GCfO5Q$%$|e{h!JJX#)SFP^Mggkuuo(0c^Va(!S2 zdc@q^9CpS2Vp|kW4YmY0%fQv5O6r=I`^(BoaCTPq!op8x3xoOogWgf+;zEPzC?Cfqsdd@EQ;U1q+L4WP%#VMl?wlM7z`A~@| zjR@=5O#r1$Y2-TLnX-xHaTH~;C&s`{4W3o!O)bmd3JdXtBJj%SA<~%+swVh^u>TfV zF$6F4f>S3&JSPLKK;3)sWG6s#WsE}1~|cv}0=xg7xM=gB>Q)13N(OqP_8k1! zdIwyP^(A=z^^>pVJ_xcPKIF+!K<(zjaHH3s@i<0AW}lF@fan)lg(o;RT-aT8kGq!5 zrtl>D!-t#GlGfwp4<^9W#oy3k|M^9`Stm91S%6k3N2E`w3)SlT_n|G?VlEv$&2U@q zvZnt!-1>&QeVoM;NEsDW7LV#wAf7yA5YWU8_JD<*?{HBPfHh|=hqyFPnpvxM2LpeO z#GW=YcIj3$qAv7as@(gJi+0(zc$s@sXAAL1drfg~ zY|)d8IJ}y}(lBX3aq4sUE`Bq3sO0pgV0sIci^4V_cu0P~8fZa=3x ztvTi!9X_XS^$p)&y;tmb=LEhRJyz@y6F@`pTg3vLj0ywgj)KvmY|j z2Xr~yzTqrH8wn~~Z!rM3>04Ei$0_PETW0yeio;H=JOt5r*$yycBH2`;VkQUy1uz#_ z+So9gNsUmWs8s)!azX=*wI_eQumfC#l-Wns4n(kk-APkZM?YV_(0J5sP8VvQzBaJL zW??_Aki}m8+S%eW`+hWV4|Tc&6an0Hb=_Qp7jI47(Nt>h3zo#Ri;i%QISjm+E?sH3 zxH!iE1sn$5VL12AQ1;^|kp+yjVn&so#cnWAJdg~A=z^Z;BI^R@8MWRLpIy(n*5ud2 z}4qUgc+LG+Jj675cqvtcaXEZ4th}=FU{OJ{*3kKfo%Gu&d_{`$TmqcLAB{6`$c$}3zW`pRyx(rp7ZwahF@&e(*0U2d zw^+e1-o1CHiF`adI$Gu7dt-?PBgldl272w9pLU^yn7RY6wEax++CcufbLWh3YlLpr zkt%PexwPoGhqM()MVjKdHT&TT-Rbst%aEhv0ab;h+lP2>Mo+lBqS*Ep&!;*07e>xr zP0GDRo;qM?6soA91CILHE3gY%_nAa4Ng}6_ad6ifOT$5KLCeM||INYA8SG+^k5e6* zot$oBEAszABc%QCHESM#z+=Y%kpFhMEg0%v5X>S7m))UIkPlb|0RvQRDm1-}n$IW! zM?fb48C?Nl!U$~nM^H+k4A5IXC-0Vp&h=(wct@p1kNsi!`R6X$@8OlOH9y;ne<7DU zNj=xb5ZvT=W7Udw0PNY6KvG1LMfxC}+cy`?H%c0nO;ege=$X}oU&@da0u9!A)&QMw z{g7gYug0wRwWvoi39zU$yKHC5*uBjU5RA z2FWJYtDk)<+FT&pnj|h~8|7bVIP1~4-_~wmjz0Ov6i zlx4tKV2fH&Ayg2?2{Uh|?wH$xkR_XnX3}fZb6xw5WhV8Vv?l&Rv zhweweDEbR{;7$4K7}><`N3e&6{Y4RV z6DX{1r9=YvReK0RX2LiUeD=VrjOnU7kw ze!P5&V@N8-)u?~cBIW2&E8R1@*x|*WoZTk`&Tg!%aca55K<%N0fzrTGS2%<&t3{o9?s^YvxqM0NS=RkAwwscZ!qaL5+ zFmj_3;N{(O(_9MlfwAS_WcYym$<|H1+Sr z8r!kza9}Pr9n$K}6n2!z<7`uh|8h;mm#2UT{ECYDro7NY90n~};cUV2&2;>-EQ_R$ z6YJHdh6g?}fj@4BFlt!QK2JMl`7m1X395f-JcLE%O9yk9-7Ypp%d^hH#&U&g<OpdlMcsR^hWqR*I{5cLm$5(pMQY5%M}!&+ zODY@enva@1|EOgTkwTgWg}g!pOQG8!(u`RF(AMzTE#PM=KeXP{gDs544@(#*s#wn& z&~;0O%S(bw-cvpE@2#5;{2Fz>kXtSM#lvE9(X}A^$RX&2CT%}(&&(`DYfW-*TVam* zYM&A=6)q@NUog`qA=&~a)RfAn2k+5{pM&)oz9VC*4RqC%?!<2gbacAs&9CJG%JRvkfoV)WUJlR~rZP75(-)I1lY64J_*lS3$-U7!j1@ zN@``cw5`k}-IxSs*RE%Hyq$o2Tj-yv-JX@(%aE^-bv=wr$4pX* z5}*Jayytj-_A*@@2DfjR*}|gPKN~eX@Y6)PxF3IV{N=g&C2cKQ&Nopy*T1lxomtwe zmYhuvK^I9i(YwVMuF3y4Q2LtnSLzlHLWKWa%oEq}aa5ltkYh@dO=Q|TK5?RZ! zW`E}Qbgc8+dh-tQXKci7PbX&XH5PjU8QIanflbQj`O8f{6JPSgl*chc=%x-3^WeZR zIdoz0uTs)4(5w10NVzT0b$`|tf8`q}XKp_2^UdEbbb@AJ^hPzryo& zL>`nNpUcNfM>|&e0@;*v`4qvcpVE=f*yJ-^1_RI7T>YZDoAv_vyB7wf6Xj zTC5(Jo85_?em8TVm|;iO{v?J zZ2YawBsJ@iV;R)$8XDnl{(a7bqtNuU1?6a>fcdRxkL2zTE-o(ang+_rO;BIHou8Cl`N(6x8tTwNFR1`q zUfN@D6AQP?p4<6X1@K?ZkCV$cT5V@t|FAyB1FNR(V5tVd5*sT5jnk4aPfzgKo)Z8E zMh?m$b;bU7oBz=N$WCEaz`5~LTcV^kIL>o`bFFay{G#!^#w5|>3)kuTNQX3mv`&A^VzZOIiLJDtx&dbH6Qq?5h(>A+AK(88A zl&X|iSQvw9uH@Ku+N=E*f|b4IVvujTg$2FEptbJ~p?a>2P~BG2QRy&t9dvApwmX6% zd#kIDKwZizNLn_7y7Sq-6TYh8gcfaAKJm8q9fEYM_!NwlBHRs34qR50gySS!KRw#W zG?reXJNeHW=Z}AZ-+0pe2IyQ10+~OqAgYiv?Vpj9R`@3VpNV#(uYG-HZeB&+08gh#{8>+u&^097Z}S-ywR`(czBz8Qm80tSjsI%PO_Vm~sa+?AgvXCxHqN*WidSR#1 zI$x)4uo`5&&BBcxYDfyqnkmYFZDk5#jE^IGlX>w!>P7dluQDAVw`s)xhGqw4XES}{ zId`u7?}<*?jRoaRODOj4)6v0LA{0)d{Q>f&d3y{pu17=*joWm=gU&zDotD1__@#Ys zXu3U$jwsvoGOaC^jM&*&F9??mduEDG0u$ppwRQsN|I0Ec>sc2T5{hYa;DIJzbAXJ%( zTzSdQxb+tm1z5OaY-fSA9*8VJI^+IfZd~w%z)PY;QPsZHGz$H;P-&%uH5K5XwkHBA zSMH<6NbN?v)arc*y?0RVYu>L~u2rzU_zv0zGVOH5- zY6nkVCxhjNxp7tyr3ISaGp}aWHz^-QBi-W*&%9$Sl>xco@;o-^1CpZ2M(d2kC>*=q z3sx!a1^a9~Rd$RK~y z@(#en&vfB$99MEz_xiwpg?4t&e~y27FkN-nu6p^%%o3CdQ8{E-q`4r6@y5Rcw^kuS zzKWh836lPj1b@_){i`H6iy(l@qk-<$RK3$$?Q(%Cv~en2iGhEHAj5ZijvgjlUJ6UB zuO>-5Te4Jwtdl{GljfL07X_?cJ|>7{)_XbqkOM1geK6gG@VRIFhBP60{_J1Wz?Kt( zXs?q~IKUvGM(9iuph5a@+Z6^&LQ3GTYz{)N?Oi_1MVEv`(2BJ+A-iWPIB&eaVLTOG zy^}V;?Lhu^#A7Bg`2q%AwUKSV0m*h0P*^5&K55Bikq_~{vsWB`KnIqGpkqg1ka%*4Yv-H9v2?C_alM1znS22AKBhmNGmDM6qYl( zch7uUx7_6;C!^EWRCN69S{wMBOjQ| zY!H>=%E448ehmoyDy2)Y%N$8%a-4v~@^v z7hdNPr|=1LQv^%szLsFJ@PxQ_9BI@>M2_{hE$}rG2&{%k_j%#CDMKwS(3Gdb9D>_TwK+K7#x^WxwzZ^PbtBz3OJBHY^yMF|{En%pU=ZU*3F=B**dgg&tTtg~etQ+%eNh@4w@ zeCT=N<6i-EKSAul#<0(FWRsv6?OoV&n_+^USLx+gM{cs9VOrFlkaZ>WreO-=`^35a zqL#vYX=+=?Gpf5?;y)LWX3+3BK@2i}U{%vqnfa<_LrP>5g{f89MXuR$efinrA_Fi}Rk3~V+*U7#9JmMq|3(d^14QHG z@NQ`@*|CAwm)XLvTRYl#LlzuN`z@+-IBnpRSAyH_`_hTcN8RoItH?t4XRh36)6vZZ zo0(eN_5`#e zlXxC}lh9)B!2ff)+;8W&84W&{QeEpci`RhH9&218G~Tew34k4>r?`*DhTZ4zLWm{M zVUhnk1C@Jsn)w+xV^7)&%Vncbs@?9<8Hr*or!D%0%h{C5PmUis@VEpNsak@tx!!sa zqbs67jLv_m&w&ub`WjZ-JE(lPD}F)-4I91eXA7x0q`+xa>2v7I=}yV1#rQWgTT_ZA zWNsUcp9Hc`EWv>m_aCf%qC_6&n%>@-v*egjxl?j^vcH1&%>F`8J$JDx-M#T$0R|g~ zP7jWQmslV{tjZDIx2AQnmuuklB|W`V@#!dPPKS9jj0H_0oi0Vxiw;Fz@Dw*u2^nxa zRJAam0oJs%k$1eSR{CB6hz#)SBD>=95CZu=pZV*eaMC?Ih4Z>^UyrW7g&O3^$ z8!If&e7;FgoitOL80I>v@`+CD78Az*;epRS@X@a`=tj2jB;Ml&8+MpP8LmW_@kSKrhO-L)IT~A zp^VG@fSZ*1pFY3EqUp&f9-oo_E~~xzspzK2V_d(W!tvc)cAe9bV!lG%pf><$gJ$*n zlJ|eu`mm!?<+%U_YEknfCl>SZus#4Vm(XGY=pz3Mt$JMOsDbmh`}RuvOJA7S$9?HK zW?wCYJz9GD$7rTRMdjtAg0x!I@^HZk6bf;TuU}?%wlAfJeFMBxo#ht=` z06=l~1`a-l_`RoULdfCnn4IdXlqd0F0;7PjA@-Q?O5mJ}`x7Ya;Fw1Dj`wWJ0F{C>m6nH`6J3bb}51U$izH^-F z)Kq(!n}y#Gqh~{^kilgTLVa`Y+0@O#c+NC!Kt@mi@oC2Z+0cKn^7Y%KUvo+8W2;_Fhf>8)7)$t7%iH09#04J6FB6rt& zHeCcyuOY5}cUe2p<54cjxF4Dwn=5N%FY0TWf5D{@vE#n`)6my47O^FI?DrGQ@()Ab zu(_GGSd9^eRu8|R-U(}{8KQOt;4WYzM-RPbPPw)F68qfh%Nx>eO(yg56At7a<;=5b_ag)xcnC&+U zK)pHgz}p=t@p~i%!Mkr>HCXB6Zf;IQC`b?wVvP7A`;9j_kQ6MH&2>0ShfSALiLJkF zeP`WqD9b4Vg)0^d+HRr=Y3fxVOUB-|mEHo@lGgp5&i(by_85`_cYac5z|{$3c*HTx zrEqI_1x#{i&f1m-nfv7h<{T@;5|Frud ziZB4l8RO_V0!$$p^u4_iajv_eAd5a-K*;^--f_+*KsdpXiaAso7h?c)mGouM0Da4I z>Fsd9%7NeGWCXz#w^K-1FD zkh*sOehIMe8`tH0$29X)U@2%gwuI@a>`2%$(r?>@_b5T$OeB*=*?5I3S_ zcGtC>?Y7tF3LGbHh=b6yYMxuV*7D?fuX;3S-4=_T^|?|`nrGSh7MkM$VqBAizMF08?NxMwvgQW2)s3QC(8lq zuU&9X(k4urI=Gq8V+9&Z_>(3&>y90$csvi412?exhjq}_6Zg9H-sF709_`A0N`f@Bn!5YY31z@#h!I0Llhxs`G_!6Rsp z`O70Hb&yApO-*Kh$2lT4JMm(|&B}JFeoaP#Pq575-+7mkHVIZ$9V7ASoBipi;gPKl zx7*X@q>X%_--w|t0P@$JYeZ`fj`E(2grqv6vr#(T+4y4aO9_1(wbzQ_jn zW+CQ>CtxC&_~_zk#(?@A8=&2A!@P1&|#qbE&8)#GfUl1f;r2)V$B4}K9u5ZnKd z1q%{?$DzIT8ubNhi;IFtYl@{o^?LFNew(t>pt5fl_&?*l$-6LZHqOaMWo+2ae%5mw#I;u9|g zx2GJ1N;82Z$aUq)>#en!c&(oROK(Fte#73fq6Swt)z&1d=Ov^ci>+bnHU3B2#ier@ z$$K#A7skAy0EgV)J|3wDSD5K)`&!U#ByD@5GOiJjW+e#Q>< z0$4e$T`rtfXTNiQM?KddNG&;b23+5{vCE*JN!VP~Q&^nlKw}kP+WD+ZNqOXda;l!Z z0riHcRo}b1CJ|oXU@mrMpkpq&ttUVhk1d=k>Qz(0yZlobL@V+W^0|ca5X4F8<%+fP zgo%T<{k4Wn%S5R9hrZo42K!Ep_o%#bubMnf3ilqhx@(c3%fwi<41+Arci3DU{=SDM z`y~kO94-jzsY?zaQbSbTqyW~>xg7C~&wom#$FTDdCS}TFMtb@X`=r`sDbw`o?-S{& z#GA}DWnLGX?CkADPnY`c`{d(sLH*hKmTm!$ByIXGubrgL@&U%Ka;eT7LxkC6Q@EzC zE+^*6**#tsgzQRju6MhO-C!yIqDds|nzDBksEJ$RNIq}_?)Q)t48fGD2t0vh<%@ch zs~oU=Y`52kLJr?>$oQXkfK(gH&hdoM`5i(3ly%E8JnGJp6?tXI`~xt8={} z(;70B-s+N12RP9qGYR5aspaMPIq2ey3yZ~kec~cu2#fr!$J6czE=P2)G&+4)rLY17 ziy2g@&Syy}8Wh1gFyT2wI?*UxwEWH`(qMKbz+hFl%-^zV7#kES~yFxSU zd~n6_X9t=lyc|p(Vt2Ga_;F&+D{it@f~J?_wR|pP@R{4d4E;aRk{H0?Y^I|=w-pRU z?}2*8R?y6s21@GQ3SfR{dnagk-`uPtAAl%boW zv;4A#wzg$DzQe7G1;99ra*3xZz1a`0#bb&mDZfEhxMdx%Qh1LlppuY?yU*fSe{XR$ z(^7xRO>$82(m#~FO+6Kr7e^w}z|{JCuyT%G@vZez5Z$k56JC zT`e(=zjk{Mzf7|`aNody+rDxmO8Pe1A`Y@LUT?je;y$*39G3h>*gB2(t>F-flCz^>m4ADU%!rAs%Y(|oH zr!H+ZuLe;-IK#j^G{NS)_wnEC$T?5{#?}Uy62pGR@2K#TKcd19T*|Xn`XEBBi1hRu zONeC|KD>S|eqOsePU6kM1j#cu&o;Hn!B{7wCj!~gT1Mg}r~Egnf@|%X$-RlQceA}C zGDO8{uo7Eap$)M#g@hsn1&j816v9UQEU<)Jv#msP9vzzS@Pzdb(q+e@2p}2opys=p zIHzioAJZ1YKk>A|A}&DY=eFSWktJn``F-N)PTLpi7D<2a#Ww`8QkwaEZmVny{gs0K zZ4KO7jM!S71$xzOBI2FK*4rfvrB(S;iUk07H#q8`WsqLRKkSDXV`?{`SRqE~;k~Y> z>4$EAX}xH6*IH+9BS&LeS?G-Sum>}j_6u%(*p4Pk+_OrHS54BKKWdW9nqOb@ETcob z`U#E{$Ms3^9w^F`ab8WXwJzJ59g^LUM1(d;dWFRNJ)?SoXXak$l8-~wrtN-(k%}XYQ%A7<)%lHH{wqp=!o#INd$Cf>!m^vWq`}qZIo#p_X!u$k z4{o0L;jg5uQ-h5iJ81V0@>e&Q_yh|Ao;y8Ql7LH%s~P~;*CL}BZKQIF3Y(@5Q+~%P zXY`)qAldBz^DU=va141KC+BLDhhA14?tMmI%nq~ZlJuZbI&d4#^!W2A+4uU?qswpo zJu?h)!@Z@l9=yZfG3`6@PPldZNA6c@_^VHKi1)@lQF#z)UjA}O;_qaooRNhY2HoAF z-%myFIpyQ|#DoKnM*4NXD~1O}vT5p%*x>v(KL3A-a}_7eEKl2yBq_8848XF4FP*D@ z=zJam5ykMEFXc1zF+d%C*!zk5{}*X4Y%@*KX(L#9bMQS;^$nQ3%9oQD0A;`m^Zt3v zkolp>+*jbJLbrWIg~f-+TfUFnER>rpo`i8jSV#r0qaqbB9H?{{LUEE+I1#82tV zZhxmjNhW`i8kZi-kIDf^xf3#;{=W@H|Mr((m1E)~MtOXzWUag&Uf7fZk5;t; z0OsZ!5|*7@?41;Si_|N5qo#&h{XHzh^}kh(-!|fZoMFT-k18oTe;X5&cFM~s;;9L@ zek8Jaj|v|wMg8*pn6ynrb}g1BK>oG*@Zj#8s>v`hm-MNBgsyHbk2z zW==;@I|c`p@?p9tfEDfDIGI%<7HDo(&EZ1`MG<>yz-!ud_fJR!4yaDh`g`|(0VM)f z%RO^N!bP5X8W?i{F#we^9KZp*t}61G@sUGkHi%JhHJITHQ5eLRI@EyDyJw*f zQmK}SYy~!v09~*ren|cl6tSOa!t^k2-DT1D@89?JawFon=fSqKFV}mbtNm0#pit8% zTV{=G-N?z>Mvb#3LLosx-$Fz5haW4mJ5t14wV^P5dEJH1^y({>WXI;#$F!y|KXYBa zAbRXf?YqZW$A$FWdM~zCcWCVnmT$P+bG@gb+P~^0la#*XyzHz#gsW9jv&r9^?7$uU`(X1+gh4cYs+l!q(Hec}bnFL&i^%UH&mgB%B735WE ziFEbDhiGGXHWCxj0TieWrBLM&UPYr~4|8DOn6R(b<6heWkID27h>&K2$AM>95+m-p_TFd5?V@3#XMC>d*skoC|zSP7| zX+Cy6T(auIE%${ERH33pJH;Fjxvf_N*h@CP`w=s*O$QA+t2-#0o99V*)ewcx^~NVO zoceXBwKDeuHOPahg7R#IR>X}(SI(WMD7g07#SB`5{l)dU*IZ9My z@hd(zDFz9zp%FCj^gW%93q-;$b$`l}=Zd#`MSoFBo2$4#`vgN0zkF&@k+uHobjv~x z20tLF{P2-*UcMYyRsqi6c2!#zb3h7cbEW$cMQw8Q6Xz`7N2JtU@;O7Abh(=86(VIz zXfFeTjhGOlo$}2eI9@KsTxUSBXv}m*w&}Tgn=xa}(WC*Q{Loc=M7&J?a$L$W&@si- z;!E8i@ys-vr>ofX-Lt0Q(R(}NOQ({vNfyRdDMeY+2{LXN~EQhBH^DwoBM) z*<<%`Ho@U$SMm*MUcJw62tCcvO(Tb%gd?InOo+gMIRX?<-^9RG5!+NWG_8ETTPl=! z@&Ts+;3=yC5?`N1cM&!PNQR=?N{FU|(j z7YpC~^t_=b21gR67YUG*-6@RHo_A?!X$h^R+ns z<&;iu9{Td-jfb?9coOh_hBq-Aw1q-l7ZoQK#9U^> zW{c<04AP!4jLuA781b9S(Va5*Ho9)FSx}wu^j^?nIf$^Pyv_66u;B}@yxf)WsO#M| zjee(t-J}K-R(jNNDsD+No1RwcU{pZ&so||K*M{gsXc-``MDcW2PlU*&>vO$e^uqFc z%)Spu_1)hCp1T;i+(p+_f-YeS%9$hSB&m()b9a+Bq*UD@<2sE5@exPPAf;pEF?H^T zK&RQbsltl zb*IgATOM8o&?R*tvSr3Ns_KOtL@JQ=EE03RPHqv1f8h8X;%nUk-x#i5PveVqcN_68 zF>q>T;|x|kxJg{VkO{(7Ah=so+CQR-HxUgf@CE|ej?n9B;5Otso!9VI%A+7luehbHxITcrwjWfvfMik? zsDi)L#rNpr?KmM$&C=A(-SrIFNFd265HilOQ~MX6rm{FOAGqieY9G&`ipj_BmBiwX z?Pe>kqum}++0<&>M>@GpgwoGGI_o+)HRaJ*?zYSuOv5J z2!*%vPD-eZO~==#g;4w$X}qj&RbO9UjI%`tKx!cu^vayltP-FYcF=6MTJ~J^ z#WooBeBeWx0?ybUCPSd#T74>T-CdUxF`me>-0I;Xnr<=llREl1Go(x(T1CIJYy+uZ$p*f_)C8}kGK=91b;ek5CPd|A zWeG)&p@-!oMkSU&B5VuYUo~C%MkBMJ*uMBwcEWI}V+u;9TUN*~2de8bTb$&LLvhf) zUOUi_-6Zk1Ua(mc%SLsD(~O^9bLWYwimOAszJP)t@aX%uHwIDn??A2(%C1knGWQc$ zA-^aR6Ro@*I)@_{9UNT}aDJ60os&Wi^-VrE9G8mZH>o>@s`PTxS)f$y!Mr~=_VGMH z)ztnwhYX5&&UP+8=wJpc{8(61M;8Z5B+Om4(Gf5_z3#PrES1p@f|^<<_m4ExEtMp3 z^hDL}8xopT$S)Zh;80KY0>PI;^HZRj`^NKA^t3CbaOR+UL-Z@D z$0UnC8Q`542&J`J+cbZjtqL*xA3yx}1Kuo44#rUse7!{bUNnl&t8XNap`Iies_1f3 zcI`u!Czrb~8iC%|yiv%c==&XSg2$V4P*wpmv#MHeszdonZ?e==&`Zf=7KY{c+ zzzRFP+3BWm=Cm~MT$t_dDlW~uG2i@3-s{@T%uGCz4w!rL5R}_4 z)hd}4Q2nr{L|fDl#2N4KNzk2ASxKn&v&zL1T7XE&X(!p<5-_bCtD4HYPl_S^am(Ez z?A@M@UNm1w5B!AUyo68q<#X%V#SgNyHuVoT&9`UhN9@1ZjlK0^?OOU0|MnZsj} z!}5a%wYUs;5?JRpa9b0z;v!K?B^Z$?Lw!w6lh*^~)!^Az0OYN*$=51v0@-m!fZU2g zE*MumDg(=QbDR2P=%-ItoVNjHE)6`SI4E8?AG~aM`ic16>YG4$MoZ!tqu6}$eS=0| zkL)xRQ+2^RGah%h8R9##IMn&{+R)^Voc0STGRjfC||8R!!sL>#*SdjVb#;88OfZ*ZPbetJ@}9=u*l&B5#f0|%(BujWAha?l?aceKvZ+m-jRXb+EaD$ukHKeM{kRjo#_B;d~ zS69gXPMtM&kv)Hd{%QL?(?+hWbKQC?37n=N!jUW+enVc~|8}Cp*NV?QdyE-37qBwf zs{{`8px1IGYc0=lu%G`|q!d`8>Np7(pO2xW`$@^ueV&-ua*a_)q-Osa+eiUe){)n! zbE}x7qZlo#bH#+Ci{iQtvpgoq3S*-mls!X=zQ#ev5x(5kSF9H+4dxdgb85ySeMsDv66n(G5S&!|fomJ<*hCT7p`_1?7b<~00Z$=m& znVhJTX<1f>;p?g-LqPiqw;%OhXRD5Jd%glm5fLT?Y*Y(%H3!t#4fL$xyUo~@3B~Cn zdu1zavF#RI+uK4k|{7zoIN3|vB;h&uT)#IULS&{lYa8Sxl-Wr4=%%mwit$v*7! z9PizY2hT*5u@+@I2V5J3AIhP@^kzyr$hbwDjqgqv%{C&S92`)Sa=%pU&GLVQid|9M z@<`_igp;BQ*kTfpBFx(pjB$BLMyRBw-n{G=kpoveaI%ykVlC*M(XDf3)AOrL8_&~4 zJ+n`5YgMowF$O*N-;qRr zd9llh@PrnlNZJXkNyaokN8dbB?V@R>oe)03O7}WN&`3k zv4Z|}=ej_62*coTC~ams-++z+e+&_Kqig1Fogeqt0FROJ>M-d-m-@$x{t9@jPYQVU z)2wkxDfD`keFkXriPkIfl{dl|wmKseveamCX0-qLd4DWn<-`83mCcoMAKVT+{IW%C z(;%pa{do09jU97`op*9Z?qz}ARj3PT08p&iwvAMYR$%k~O*bd`&c4$>_(}hEnzMQ>t2U$CrADki&g7yvo8#)vx52{hpbZXXB`k3A<##v)ZLv z!FRRz=FWJiViQ24P&|8JG;#|^Cl$cf^_%RQ`>;&HA^jKTp0LOi?ljwlriurgUI3V4 zap*BP#$905emX%+H4_w&3u>f}NfevxLpk%2Ymr`yG=e$DoOwIvvA`@ZOWncotkfUZ z`%Mfr01y($&!il=Hb7x9{WR#M`L^lFqr~N)g#pVmzLAN$;`7vc7r3!K^Pt9KqwY6w;E+ zA{~q)%cwao{6wV~CqhpTo}qjkay*t>I8uB|LVaOK735t+e5AeP26iLq0`i(vT5Uv! zLNSUqSSNs$)&@K9y<9)1+TQTt)QdBqa8$!Y)rEX1n5IU-F+9M`=>teBu3bK@e&SUv zl-K#HV4LHYpU+y|`QLa`6d&c4OEsMxKWwrbyh__uCY7lE;iD|ReC1r&YOSsvwc)j2 zO%cI(A^IS8tuMTN4iQIW{x~L6ht`9A`pD4cFHG1e1v5LRGVRQc}R-wJAg z>l@^V_8RyhX=rG0LV0_s!;0J4nx1)TN#YuyUjFJrfFcO!*kvwlZEcA>`1qKI{0s@& zj&OF_MW0$jq_FCN>bs&f;Q%`eHcHCJb!G%^t<>}kc4V#roY4|LFTK9d->w(AsWkx^ zzE9VIl-DLeii4worPQS_fhXdM+9aD>O6>+YpB_4XbC>oiciVQ=hE|8{F{)99aV6s6Zu!Ej(WHb;E zj-0EUf1f)!Q1Flp+1#D$epbp8?&1v+7%WqH+>HN@PmMPI9iP${os$N^%}#BhLNyoq zjl2-0z6~#bh3F@PH9nP>bU%E2ox38c1B&b^uhE?pYlab7(TlzyfkFkOKD0_2<~FVY z`!l5x5R*ghdRE#iuaqj|V_p|@(qv|#WS%(J!G=CY;l6@>#p3XnvDZv-0Upfbbs!v@@$0=s1yWOLn5kKRNftLmdbR4xaI6e$AP(GMRb~epg z%kyW0Si~ltz!H(2!e+r}s0n;lF8Cp4B(cslkSDc-PpZrPyZ_-^OzSTFhE{YiY-2QU z^Ly>wmirh#>n9#SlDP^d<;~A%7!~+)c-0b5UA$kkBYH7kehC+L1q2ryddyOvHWV(91xDK4(g0GHwglkD;*N zsjNg%+3wd90&UaG4Ya#KKnzmTMzW}Q+ zi}7s{6}O>>Uyd&dUhdaLPe`IW@Vq2uVO#P%2zgENsWA2vRdEQ^dB%GgVyyraL@e8u zj#L=qBZhe2M<& zMgDzI}Qk&&nPe0wY^ay4m{@D)#JzC9f(2~vzEMb(aGH+RIz z&3ByU+@?~7z$`>;-!tb-uSfmg5+m58CCA774;)S;Zbvl&nCaI@*2v&wO2w%J4?Uk| zTLDY^I%b16CQblWUb#|__p{<0b;s&{pfbvh#G#~XtEz_VQQKB=f`%<()`i)If~;7ZXgN(k3D z%_EIID1yV(_y@w(0K$9ce3rnl!~!^}S;{>4xbKX(UA)_+k>j`}X~17A+65`~alkA~ z-PYE&<$mrbNU7Qk83TClZmtM>`Pc6x=TX!OfMhzbB8D@{K6)8})?vgo%l=ht$cuAl zDz-h=eg_TiYJJaA*U}4dPHkpP$$1}UPoT3`_4Tc>X$7avu}ZEqbwXuv69z@Ca^a;z z0gAR$VT}Up#kJ6nyi$K!@P52=bA*G>f<7dm+E1bbd%{*A6zv|)*4l~mpv_1Sf`ckOm_2!_q; z_R~Bs+0mg;SA=ebR%Su3Qx_pyK#}TeL!m4Dw>oO(piM6eUKZVdAYYBoB`?m{sAHk+ z37qq>B~fUeD$Pgx!2lu}lsCaBz0OByo9qMyPd-z|A5SeuC3RKpY;tt^PLxs~l~jiR$kww#53x%4$(6Gm6O$gAs`o`%Ac$HK$V@d_Uy*}?x zW(0WGkY37i4|o$sy4ly?!K>Oi4C$U9Pi200^c!5`sD<>L-)-hce^mpx`saL@oTy`qau~dJ3_3r!J-0C{# zJgFQcGVbL08+9e(I_JWFLN@d~00d(}3B@pu*Z0-&qSR^0LSgP9G^WN6uo-pryeK(r z#xChQqX7s0uCHGqh8I8l=+yp$rZmONen#K6=cE1dyuJrSFlld0hZ3~Q~s*!P*XpE6Kg<^&zF4r>J%M?#XdTZAG41mMS#EFgCcQ6RHy^B#XVS*?l6NNX~jzF&cLmX5e zD#m6X&$*)ACsf5D;%L6>mIX8Rlh#dBW03N_qC=U6AlFPgsTF~Zov}-HxNz~P^tX`t zLry2rrxpBAb~HTMC1F9N%|M>K4JAzYtJv*`4`F{!b^fgp&y0|}a3E2pzsK}mRBHdX zP(wYhI!xPo7zCOayn9E76zxB_u|79qvM0>*g0Uf|k5f8?w(78!9&_G(ysP>04Z`>x zPE1XU$WwN=w*daJpA*Ej%oN}#z<5LKD{bogludgXOJ<BMOLu|F=OzVREw%Geyms{n5z(T^+ne%n-V zE${+qvXQcTB2Qk0-dMs)@II=7n`x5?-s%axqNQs645TO+sUG*iCV| z4K5Rm_*JwM0oyny-A||XLg>GfpXi%rQEi0MrcztZ%0hA7N0~x5S|-zQ*B8sqJ0JeR zernY=32Zw8|G|F3yv#FT9+EYCu54q|jp9lx8n}7mJMam51(k%*+|+3sIL%))-fFsM zeq&!1m=QKUoBYw7_}@PjADy05BX#rK-_fRJ*;2DqmiEpIg~MKIr>Xbvyq7C6IfhmO9|pdiNlG5ckU@ron)%lwdV$FDVf_d4oVLokK$4C?3G0 z#P;V0KQu@{6cB?QE5q9-sf~QMEY1MKmAJY-Wl$N$v6GeqMckJk43<4ITm2+K8TkRg zNUH)0S)!~x-1)*M4~;kb8qzTZ`@wyZKl#MqANUfal}iBLPBcAf@`ty8hY+uRG7%(5 zCiwu7kGS?=&ij3mEBriE6ekb}A^Iuy`4Un<$t2U{Ov%dUev5~&NHEOB(EAi*pbn|@ zcIck%uGy!NTPVB9;klln6mb*G*G@4pB`AWqV-yrU3XFAlKvE-=-bETT%=D9dTMugY z^PoB%VJ_Q2*$?w2PPVqU$L&9GNOX(W+r`BteMz8X)S(~N(k16`DhxT%ktUzsp9S+; zpEdEV5^@oTATz+rNEpl|bRdmotrW+*1H$~|->6_AVcYX#XH1{W&VV886mXw8*!!w##qhYLr|k3nRpAYZHEM@k&shCnp{ zX(6wzcgT0u`uPEqq+4(L@sb{Zc+pyrn_qD=oRxM-g|t!+Okxt3`|1k90fNzh?G|SN zXiSJTdG`9#L)7Tx7q3jdJXa~TAEt_#P1A5z^Ne7*()N07JXEL&*v&5>`VX$u1v6TJ z%C+#pCn~4>Z{4~;))~9xv*eYo;!f_!+%$9>=9c!ti!L}Lc-q^62^es-aobg892o%t z@)9#ZtbmFGOnx4aw%MJfc3G259GswxV86_zlF*C9`3<-n9}Ctd7dVgLOSj9$Lwbr& zJzBbF@WL7!XLOXUbLq9rup!fHjQKLWboew%j{gH(%o(sFzJ$*y@MmNkN4%~9@smmC z`kJuKf|V`nF@4X<=?vx&Dt2zxt9rSG=fe)Oxy!XlE?k1_%34gqq=AY_lL;YJ?5H0- z1_}KLqkhJsgH_IRU75LW*H%|s2(Lkm!*n+}P;vu@>Bn}3*ol=a5Em$qCOG;JIu>$P zf^pc0Q~tYr0(2WU<~w?0V**f`)l(5T@@<0GE=@n60W^b%g(-l%2&O0>~?2UnV8g!XKMq6h?_D1s^Q?Znn(e6|~2OE|T-T3Ecz zIo99DlX$TI5t&G6?ytd%*run=%eD{JN3Yu z+BRxlQeAw`!3^5*|E-1r{{+LoNjuI;hq&*mG#e&5S{9b6Sm#~E*CG}n$Jk}AamQsb z^(4|0PWX73>FFg5z7>jguKx!k3gO##)Im*S;TK`kXFnue-Nfw-6;G*R&T9!EgXKy> zHyUASV&u$Row_OS<)a`GTip*lwd;(6AyN z)?vQUEL1x*=_&3oaw%ETB?_2cHQV|!@tV7L5zFON!uJOrT+elU5elIgeT zlp4zR?EHVOIj{~V+_SJoUFI?y=WN+AW&8)@koE#O6~5HP&3GO zaDf8AwY0Q?Q0K}4E08g$L4)GQEi~q^VjSGIRt@=G;4?w%knwK3O2kl{Q;gSQq+Dfa zk74zHWnYP^=##Ijjm(q4ltK0S^>@9hMn)-mkO&Wl4oHcV?*Ef|<*BnXG3&{qO2EKb z>hlFIUKCcb8%D--{0To@>Ym(46oa*(coO1-j0K@E&^2??EV?AOinP7p2#9xy-DIH6 zZvZ&0R1sX7b7T7UbmdjbB2~MS){O5Iq*Da0M9=zMGAhz^Wjn(2A^XLm0I|YVSyUkx z+3(7Ze;WasgG+{dT?6wHbFl7MPJdGqy-IP2eWkElT^M~lFQx}_PGuUb7)Bb`n#(7@ zS}W!SVYroQQHG2i`YH%*OwL)5M(_&kG6`JXz_Rri+qJ9OX~9pS4fN#{)rj$Wn@VTc z2pkyt^j2qEd9}&21wZ*%PBXR)Dd$-odcZ&3+9v<3)QIsJL{MAiNc(}q zbf<{HqJSN54uTn`DWgyF&yN)ZZc*Ir_WA!pC6U}9xL&%++pUn|sH*6g>nz=NvlEjaO2Z`D`C0VmNj!lnxM zVP;>)e))NlnNWqrd@$M3koeWtPR-kNr+As%6_g#%fv5f^bJP0|?$rsI0%}wCDFSb$ zkLf6@5=bVCm7^ukSmYt?iDo{Mf!h-IwE9a()Z8A_{=XYv-u)jqSlk|W^?VaafnNbo zU9>A$E=}?I>yqcl_A+OGZ}W1L*e<@tb#xWoM0UGw<9RKH3u)TPNmGnGf;zJj7%##- z;9H02=sRkM^Qkru_A3ecUy!dHB_k1&0JWNI9ChsEfP@aB64Ef8S?ZU;BY`yofKFJc zvZOHafue^UXdRVfKo2&uB81X;^=Eso1|*FJBL zTN}&VuoZ&6*YCq;!1sg(TbL$o?w)#E0O`w6YtkJ1z)_#SWTReI)C3`GVFB0#r1N%^ z!GwKObLU@n0~T?8M@CtPs`*eg8sY7UjesD37ZUQT$= zd^+opvZ?C8LOJ@O|!BXsH46+NIh z=7MhhVSrQmXLlfBhnMuUwEEXRE2%%->$1JspjI}&BJ*^y=DnTK=D>Ds{QdnDYDnng zKcGerhTSJ)K@p{;)ys0oXUy!&CpyTkX}$wogPEX52Df7 zg6d5Lh-5@}&t#8VcjPswZ+HvEf_)~!#)68hLIL{`7zAwky19OT+)(AgNf=`EHUFV( z`$7=@YsRI)-H~tDTr~}>9L`TO1ha3cjO|4l`j^~osd2Snv>q_!xP&^yw=^%4Z(RwN z{*xz!@n$esGGZLA&}r{EeJ zfB&ARo)b3!XcHddfQudG9PhKoqRW(PeznHsA*zJL% z#TV3N7l%9fHiOx@>Ui-;hCj(hKRLN*{aV=qWXMk)oE3jziAxYJ*oRZaE;N_sq1rP{ zlU*22{T0TAcUwdEClL=gdCOd-7S0d>Qw_2SKdj=QOyj3lxPA5 zsQ9&wO+c7>3mn%YV-tOn#Nkl@qKL=Y4_7pP|MXN-W6BWVK;za=%79V#eNB)sRi5%M z&`7aLQ2O4|px-FT=d8NdUDz>1yiNw=8C%4P3wEI3OI=AYoTXpK+ySB|v0byySF633 z8tM10l%#NW+fEh$U2ArtDLRiaWs!YFd$1vYr;@lA$=a~y>5~_8-QY#0=cd`Ak>+2N zC%=)N0~P@}5^D9pu#;W+eMAhzQ{svdE8hjiQuQ(t54CnqjDV#1t|BY-t)Uqi-kH}< z;ai!4yhdO4Kp?W*T3FHOv8b|Be!GDzq}`f8{6;PvJP3Nntd zh+pdXY+?T9ut(0L?t{)E@<{FIM#wNEyB0{Ect}v|a(Xe@Z6p$4Ft>ov!aSw*-QA29 zWBO~rl*nXZv1;vh^rX>;2mYnTWXCb!j3w6IC&S`X1A2ma_#PAm6~HhG`sD`y`(YE~Bbkl1ZnLVVkhLUFx6&=PYF`8GVe{(@QbRZApf=6O3n#{C z1NHt|_(5WXmCQPk=VPT>5Y;_h$iHm2Jn-32atZ48;*&>{LBDwZB)l6x@)C0`X z>6P{LtfhJlNCTuL=h~U4-)hb$J~k?~PqwCBRTZk{(icO-v`B(v(%F>h){=X8r*>(p z>8YtH$Of(vqVrYw>6qDUB7HabHA@h1QPPGnxmW5Qf7Zqj%7DTf6Z}&B}4c+dcr38mD1J-C*24 z3o|h@TLXsaXG@dpD=yk4--ZlAB!E;DqCcfGt$PYcl2#)8;?;>?s%7D8g|@wXa~)|< zbDYV|#w61e1nNxZO&3;hJu9{*DceWU2MNCH4#omKo@+3%z>K>P9x=~OoK?zlC=3Hj zj~&_MsrM!mqa%r#z-vYW1x+!Evp|058jK|Z2sUrGNfYizZ}yIKp=;y=d5|UBXgald z+ukBNM3zylZ?*KRbzrIEbW5xZ^&8lxmAW2bgCR+<8tgWVYQFki+69dbiqPRHiefNne`?klSXr+8slu9JFl)U=7?A!k{MO0fe5D z)raRoXXrB)sY`FXavK4#G8|QssOx23DerX~rj0@!@Bm=i6hMuf;LHYorK0B-c zWl+F0?`jA#^QsrrvN$o{3Q{}yJ~@<)J?9~z1Mhp;z`BeXJMG0#-->sN6TV!vNP>@S`}I&`oA*X-NL*o!0CMqj|X z#kH!N%kwvhb-sRoS3{$VD`5x)H$a}1)*fZNcpD(nNm!KjkS7P)SY+1tm^V;(;@gN| zwOYUbzEN(?=?$!O?(*ZRqbXCb_U->QayX>+V4`lHEfzTYcUFc#u`0#LVtjcG5HD8~ z3&aI$cO5%xnxPH(jE)UL;EHIU9ig(ELSeo7#uqG%Xh<2ovMXU@r{7U;?pa-ls^0*>d})u~VyE+#c}}&2dOpj6r=&Z z70Te#_@<`#ZXl|*!>A2njXC+~rSTw!(+cd8q5%7*GEh^py}jLmLEShjUQIr1WANBn zQDqWtl+;=r+FAS19oSV)0ix#R(h5)vNPsDj_^>$P38g|_t_g&eN3yyHnuZ?t!Q^ZLIF*RCN+*7Bu^!j$ zpY5na;o{(~qNKFm0W`ZoASR;XQ6r7tK%5T1F0NeCfx{6ksj{+mqm3y8AsbpgpokcO zrL=<4!&E>Iv$Uu*W2n z_Oa`qG1S(MDMa=&+t0@XYbvNV-58(0tY2UnaK-As9RmhyHRqL{-=UKD_UeRRi_C$A zx0*;#y~62ko<=M%ifRUf7pod$&i>CtcZNRpy)=od_1)DS)+b_%JyE%5+ zD|j3fsZ~pzFuVTzWb2q!$hW|uR_DgfQa4*ral1|$My$~rU8cVusl7^j`qFPcxf5aV z_R`*I{8_=8O;gR2ew)iCYf(xiB&f(mqo!gF?1h`xg)UXN;%wE&C{7snALKj&PCtX= z({0C9hMXdTYExZil^qSvs+yr64Y&AM7LjHPcpB~0#$|v(o)OClgmNFLQIgGb_E#{t z%pg4Z2!g5Q0K!wnEN;uf;FIu%^eQf{8DRVm+Pd)^vgEDyYFjIxwC_;RHG=`Obx8@n z0VTw?`-PqKox4@czs7!8a)q}Yl6!>jA=ojdd^dEm@e1{uJkq7qresuHyj)bBPJ z6rx5T0Whz#Ll-yZX7ybKu|>9Zn#E|{TFu<5k|^EWCtysw z$Zc^x<<;k8Ph+&cVs4$-#XOp^KDNXMeX!1hC^fC@`NST5<8|vLH!xJtGR;032TASa z=#ILR-T1Wq*gf6oJ4rCEaj9A@y?L|LQm0ZHb=*?u|Bi9?Z`j}en`F0VTJ=t!FDr8J zQa_n0l5k%@o5)8Gq8aoU+Y=wlM4F?L>I4P*MjHHYj>~=aY?>W05;5Q}ih~cZC|eRD z(|50_kLch$rOsXvO@l|H7lqyOiP`WIwuE@8e)5T4;{YsM|NK&pW_A;EXAYAAE^_v}gXW85;%_@B;;(6xPD797-)$vpihk9K@z-ZY>ol=VgmzFlaX{ z`|VV{nk7I8n_&yG(fsvzwMp3Rd407TFct;`Lj>>Nl%cA_4R5YLoy{s|A~8b*aEonvjCo07onCkCWF$rgjeJ7d$n1BTmJNKL^=I_UVN$352D

    nr{+BLPu6aTZY360Qe0e$lqnwN5rc5#%t;vlf`jomo*nONYQuB(L z=)v(PeM#;BnLRmW;WS{(O-jxv!5<0RcBWbZkwuL-g6 zeDaq+yzBmP1EF`~xJQkio)4=eAZV-+g}!6dk&)+zGdgD~@EiS4XTC)Pu10kGRmG!d z>N);}@CL8cb3r8Pa}=jzU}ENY|8WTRLhU{S)WPU({TzfWDUAk3yvWZ8k(Yw~Ft|8r z3ulM9{u24!{kjzAsh|Rw4B8ir)$#5?=I>~sw9bj&lM9X1@uv3{qYRy8mh3|)>aVQ4 zI=-L+esFyzcaQS%@kELiZKm_L`rf7UBOFB@(@9eq-mmUj9)Q}4#=Vl2_-q4s^sloK!wVS_}k z-zA)FQ-v~iT=4mm4ww^7X^Dp%+V=cN&Cp9G=OTn~d;ouL>Fr^c=4lqkszmcoi2TR> z{W-?;?*r}&!xbcx(M7+NiMFRDg*NKi1CMwpBU6v%1Od!?J0p9fC{y{ymi~*`P<9wg zcvdHqHu(RB&8SKHgl1`)RPg7o6953mp zichNqX48+I9Xfp;i6CG4{64F)cy$}dB=c(|flT5?=d;q3j2YWwd)_WVDlPNu=IEe@ ztoEUiD+el?HoNjq>Kt+txPQU1o|Q8uW=u6C5F0b@zw_2-Y59OyYOHeabiPJOfzkjC zH#g_hl$KRs+HaJfIDZ{Ra9?fLSA13~4e-sCsaV2krG+?z3TQUgV-vzsnC)XzX5i7T zxQJ)$4=t5!(TyqEM2mJZ%P+2fz^AUlf4sH7D7|&R;HK%UmT^<$`A=j^4qtParxxE;_AP&Rz44up`!5DHBt^hjNPvkGxg2*R})``gV*OgLQ}$J+d@G}q6tHjQsq1DPiS_fJ zN6AmZT?UaC64F*?GyE zgHXtNvB(>ck+91P0K^B;?WB{(l1AUj-_lA|OjWs?ylfbq*8i-1A5C}1! z%=6qYH{x|o(uh=v^wUF1wKkzRuF<6dXpz0$!W)x+ZxpSH}ZzXxwS{7-#v% zQ>C>Swpg6I#k@bMMW#4g&J`}WESNjvfvLi2!8ai#XQ74adcz({&V~w?g-Iamxl*4G z1SOqZg?@(`=0|fLyUnOBm)A4q5d->o7;vib#%>DJm)g zOy(>Efwf-rL>N%Loo)wdt-7$2(gOe_)MG81`ym;w)X`k%sH;Go3lMiw{^0aN@! zwG=&kw zH|IC%!d>k%iNNOTZ`?Ro3AIY$7h?;Yju|E%S+o7##wy(j7a$GPTTaTDa0OKb*hYC+ zKd^VYc;QHi9~2&{=I8~yyd1xO+`l76nL7Dc?lM&sw%`m09WT!t74+DbKwejkSH1cI z2mys%dB!H-<(|a{5Vt`Md5j+f-e2-Y8dbN*AR78YmaV_&q(Ex#*wRs73>Q)cl1v{F=L9n~<5Yl>k z7dI5PN2Hdjx)*|p0Ap{6t6AqhBVrz8Z4k}A`~7gbP?`B2q`6(NC?-lxE$CqYmW-F5&l7z3G+oBRp^%$mIjpNhTuZx%q6O_5(|lBqqojt zq|ko$#c@2_{pM&C>RL{2E+a0O2+|xAnHBSmYmJI~gIN8xnvJnMdw36Yya4ZR`M7K2 zgKWybT>pK(w|u%>FF5q$i#h8n%!*c>x?HkNQb@n*e(uwgVC?DNtKVjjSoZ%-2z-r$U`xU!C>OV zxvO`??FKklC7oY_9kfd-aF4NlkN|k@Q0n^Z$x>IGJ-kSxauQX!SBobOmWsxHD?a}+ zI!vNC`8gdtV=NpXoy}m*_!bO$O#(qxVue>vxu$pY=+P=OkRfhaSyrYC#>DzI_Fiy| z`cmCN@tX(qwh}H|d0Gt^4FmtaG@4mp)ahQ2dtRLa!kd!}@%jL$23I+xrsr-wc))(- z?d*~WwO1fMYrNOusO*N{%7dE6l(mu#zXP|e_nMyP%$J|`y0Y`@9CUaC>aAlbD>cP1 z_Db8ba!hY5Cwojp^tSY;x=kP~ykyWfq}R&cf72Gng7|~dHLY$t#zDw$fFy;q4kAeoD*SK75)8Guc~wr8>tJja7>pASW)w?)82^GZy+>+s1=oIDd z==;$-k+=}^@7_PmQH~jw=~$-ztbqNou2jG~3#xqGV+K6uX@!DrCB@3UjILTI5=P^) zQ+*{DZRY`VD?tnv3(>=Qw{C7p{w#Jicpvb4YcLD{(ui>9_AT2dGsd=fjQt*$cer|4 zpwyRoCYsa<#>KI=uCeAx>&ey`oGj}z{wM*)*mnr-H#D;r%}=TkO5rcae9rLi)xp#EqXt$g z3m7P&So;s^8-dFad&R!jVc>^h)lE2Y<%F8U$K2IM7$2zYe;w|9dfhE|&k3F2*K# azx>B`$x#z|(S6{LisEgB{NEou`F{Y4>Eq@A From 6a91a32a87017b725365a0e89c90e2c3a9f87636 Mon Sep 17 00:00:00 2001 From: r39132 Date: Mon, 10 Sep 2018 14:30:35 -0700 Subject: [PATCH 340/808] [AIRFLOW-3034]: Readme updates : Add Slack & Twitter, remove Gitter --- README.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 4c22be6d8cf05..e52d2a311ef3e 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ under the License. [![Documentation Status](https://readthedocs.org/projects/airflow/badge/?version=latest)](https://airflow.readthedocs.io/en/latest/?badge=latest) [![License](http://img.shields.io/:license-Apache%202-blue.svg)](http://www.apache.org/licenses/LICENSE-2.0.txt) [![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) -[![Join the chat at https://gitter.im/apache/incubator-airflow](https://badges.gitter.im/apache/incubator-airflow.svg)](https://gitter.im/apache/incubator-airflow?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) +[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheAirflow.svg?style=social&label=Follow)](https://twitter.com/ApacheAirflow) _NOTE: The transition from 1.8.0 (or before) to 1.8.1 (or after) requires uninstalling Apache Airflow before installing the new version. The package name was changed from `airflow` to `apache-airflow` as of version 1.8.1._ @@ -282,7 +282,7 @@ If you would like to become a maintainer, please review the Apache Airflow ## Links -* [Documentation](https://airflow.incubator.apache.org/) -* [Chat](https://gitter.im/apache/incubator-airflow) -* [Apache Airflow Incubation Status](http://incubator.apache.org/projects/airflow.html) -* [More](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Links) +- [Documentation](https://airflow.incubator.apache.org/) +- [Chat](https://apache-airflow-slack.herokuapp.com/) +- [Apache Airflow Incubation Status](http://incubator.apache.org/projects/airflow.html) +- [More](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Links) From 4a4648589776f3272dfdfe8a3a764968913b9777 Mon Sep 17 00:00:00 2001 From: Tao Feng Date: Sat, 5 Jan 2019 06:05:25 -0800 Subject: [PATCH 341/808] [AIRFLOW-3612] Remove incubation/incubator mention (#4419) --- .rat-excludes | 4 ++-- CHANGELOG.txt | 4 ++-- CONTRIBUTING.md | 8 ++++---- DISCLAIMER | 6 ------ INSTALL | 2 +- MANIFEST.in | 1 - README.md | 17 ++++++++++------- UPDATING.md | 8 ++++---- airflow/config_templates/default_airflow.cfg | 2 +- .../example_dags/example_twitter_README.md | 2 +- airflow/contrib/sensors/qubole_sensor.py | 4 ++-- airflow/example_dags/tutorial.py | 2 +- airflow/operators/slack_operator.py | 2 +- airflow/www/app.py | 4 ++-- airflow/www_rbac/app.py | 2 +- airflow/www_rbac/package.json | 2 +- dev/README.md | 4 ++-- dev/airflow-pr | 10 +++++----- docs/img/incubator.jpg | Bin 91227 -> 0 bytes docs/index.rst | 17 +---------------- docs/plugins.rst | 2 +- docs/project.rst | 6 +++--- docs/scheduler.rst | 2 +- docs/tutorial.rst | 4 ++-- scripts/ci/flake8_diff.sh | 4 ++-- scripts/ci/kubernetes/kube/configmaps.yaml | 4 ++-- setup.cfg | 2 +- setup.py | 6 +++--- tests/plugins/test_plugin.py | 2 +- tests/sensors/test_http_sensor.py | 6 +++--- 30 files changed, 60 insertions(+), 79 deletions(-) delete mode 100644 DISCLAIMER delete mode 100644 docs/img/incubator.jpg diff --git a/.rat-excludes b/.rat-excludes index 05f7b98000f89..916e9e268971d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -56,5 +56,5 @@ git_version flake8_diff.sh rat-results.txt -apache-airflow-.*\+incubating-source.tar.gz.* -apache-airflow-.*\+incubating-bin.tar.gz.* +apache-airflow-.*\+source.tar.gz.* +apache-airflow-.*\+bin.tar.gz.* diff --git a/CHANGELOG.txt b/CHANGELOG.txt index abb0563d71771..aa7b321a5cb5a 100644 --- a/CHANGELOG.txt +++ b/CHANGELOG.txt @@ -535,7 +535,7 @@ AIRFLOW 1.10.0, 2018-08-03 [AIRFLOW-2097] tz referenced before assignment [AIRFLOW-2152] Add Multiply to list of companies using Airflow [AIRFLOW-1551] Add operator to trigger Jenkins job -[AIRFLOW-2034] Fix mixup between %s and {} when using str.format Convention is to use .format for string formating oustide logging, else use lazy format See comment in related issue https://github.com/apache/incubator-airflow/pull/2823/files Identified problematic case using following command line .git/COMMIT_EDITMSG:`grep -r '%s'./* | grep '\.format('` +[AIRFLOW-2034] Fix mixup between %s and {} when using str.format Convention is to use .format for string formating oustide logging, else use lazy format See comment in related issue https://github.com/apache/airflow/pull/2823/files Identified problematic case using following command line .git/COMMIT_EDITMSG:`grep -r '%s'./* | grep '\.format('` [AIRFLOW-2102] Add custom_args to Sendgrid personalizations [AIRFLOW-1035][AIRFLOW-1053] import unicode_literals to parse Unicode in HQL [AIRFLOW-2127] Keep loggers during DB migrations @@ -1829,7 +1829,7 @@ AIRFLOW 1.7.1, 2016-05-19 ------------------------- - Fix : Don't treat premature tasks as could_not_run tasks -- AIRFLOW-92 Avoid unneeded upstream_failed session closes apache/incubator-airflow#1485 +- AIRFLOW-92 Avoid unneeded upstream_failed session closes apache/airflow#1485 - Add logic to lock DB and avoid race condition - Handle queued tasks from multiple jobs/executors - AIRFLOW-52 Warn about overwriting tasks in a DAG diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index d991eb51fa641..b7a1b1e0a361f 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -83,7 +83,7 @@ If you are proposing a feature: ## Documentation The latest API documentation is usually available -[here](https://airflow.incubator.apache.org/). To generate a local version, +[here](https://airflow.apache.org/). To generate a local version, you need to have set up an Airflow development environemnt (see below). Also install the `doc` extra. @@ -152,8 +152,8 @@ Before you submit a pull request from your forked repo, check that it meets these guidelines: 1. The pull request should include tests, either as doctests, unit tests, or -both. The airflow repo uses [Travis CI](https://travis-ci.org/apache/incubator-airflow) -to run the tests and [codecov](https://codecov.io/gh/apache/incubator-airflow) +both. The airflow repo uses [Travis CI](https://travis-ci.org/apache/airflow) +to run the tests and [codecov](https://codecov.io/gh/apache/airflow) to track coverage. You can set up both for free on your fork. It will help you making sure you do not break the build with your PR and that you help increase coverage. @@ -177,7 +177,7 @@ writing code that works in both Python 2 and 3, see the documentation at the Airflow requirement and should be used where possible). 8. As Airflow grows as a project, we try to enforce a more consistent style and try to follow the Python community guidelines. We track this -using [landscape.io](https://landscape.io/github/apache/incubator-airflow/), +using [landscape.io](https://landscape.io/github/apache/airflow/), which you can setup on your fork as well to check before you submit your PR. We currently enforce most [PEP8](https://www.python.org/dev/peps/pep-0008/) and a few other linting rules. It is usually a good idea to lint locally diff --git a/DISCLAIMER b/DISCLAIMER deleted file mode 100644 index 2758508789843..0000000000000 --- a/DISCLAIMER +++ /dev/null @@ -1,6 +0,0 @@ -Apache Airflow is an effort undergoing incubation at The Apache Software Foundation (ASF), -sponsored by the Apache Incubator. Incubation is required of all newly accepted projects -until a further review indicates that the infrastructure, communications, and decision -making process have stabilized in a manner consistent with other successful ASF projects. -While incubation status is not necessarily a reflection of the completeness or stability -of the code, it does indicate that the project has yet to be fully endorsed by the ASF. diff --git a/INSTALL b/INSTALL index b018839ab1a5c..00d458f450737 100644 --- a/INSTALL +++ b/INSTALL @@ -1,4 +1,4 @@ -# INSTALL / BUILD instructions for Apache Airflow (incubating) +# INSTALL / BUILD instructions for Apache Airflow # [required] fetch the tarball and untar the source # change into the directory that was untarred. diff --git a/MANIFEST.in b/MANIFEST.in index ec99c1f6b2b1e..2ad56c61dbfdc 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -18,7 +18,6 @@ include NOTICE include LICENSE -include DISCLAIMER include CHANGELOG.txt include README.md graft licenses/ diff --git a/README.md b/README.md index e52d2a311ef3e..633000c18e788 100644 --- a/README.md +++ b/README.md @@ -17,11 +17,11 @@ specific language governing permissions and limitations under the License. --> -# Apache Airflow (Incubating) +# Apache Airflow [![PyPI version](https://badge.fury.io/py/apache-airflow.svg)](https://badge.fury.io/py/apache-airflow) -[![Build Status](https://travis-ci.org/apache/incubator-airflow.svg?branch=master)](https://travis-ci.org/apache/incubator-airflow) -[![Coverage Status](https://img.shields.io/codecov/c/github/apache/incubator-airflow/master.svg)](https://codecov.io/github/apache/incubator-airflow?branch=master) +[![Build Status](https://travis-ci.org/apache/airflow.svg?branch=master)](https://travis-ci.org/apache/airflow) +[![Coverage Status](https://img.shields.io/codecov/c/github/apache/airflow/master.svg)](https://codecov.io/github/apache/airflow?branch=master) [![Documentation Status](https://readthedocs.org/projects/airflow/badge/?version=latest)](https://airflow.readthedocs.io/en/latest/?badge=latest) [![License](http://img.shields.io/:license-Apache%202-blue.svg)](http://www.apache.org/licenses/LICENSE-2.0.txt) [![PyPI - Python Version](https://img.shields.io/pypi/pyversions/apache-airflow.svg)](https://pypi.org/project/apache-airflow/) @@ -37,7 +37,7 @@ versionable, testable, and collaborative. Use Airflow to author workflows as directed acyclic graphs (DAGs) of tasks. The Airflow scheduler executes your tasks on an array of workers while following the specified dependencies. Rich command line utilities make performing complex surgeries on DAGs a snap. The rich user interface makes it easy to visualize pipelines running in production, monitor progress, and troubleshoot issues when needed. ## Getting started -Please visit the Airflow Platform documentation (latest **stable** release) for help with [installing Airflow](https://airflow.incubator.apache.org/installation.html), getting a [quick start](https://airflow.incubator.apache.org/start.html), or a more complete [tutorial](https://airflow.incubator.apache.org/tutorial.html). +Please visit the Airflow Platform documentation (latest **stable** release) for help with [installing Airflow](https://airflow.apache.org/installation.html), getting a [quick start](https://airflow.apache.org/start.html), or a more complete [tutorial](https://airflow.apache.org/tutorial.html). Documentation of GitHub master (latest development branch): [ReadTheDocs Documentation](https://airflow.readthedocs.io/en/latest/) @@ -85,6 +85,10 @@ unit of work and continuity. - **Code View**: Quick way to view source code of a DAG. ![](/docs/img/code.png) +## Contributing + +Want to help build Apache Airflow? Check out our [contributing documentation](https://github.com/apache/airflow/blob/master/CONTRIBUTING.md). + ## Who uses Apache Airflow? As the Apache Airflow community grows, we'd like to keep track of who is using @@ -273,7 +277,7 @@ Currently **officially** using Airflow: ## Who Maintains Apache Airflow? -Airflow is the work of the [community](https://github.com/apache/incubator-airflow/graphs/contributors), +Airflow is the work of the [community](https://github.com/apache/airflow/graphs/contributors), but the [core committers/maintainers](https://people.apache.org/committers-by-project.html#airflow) are responsible for reviewing and merging PRs as well as steering conversation around new feature requests. If you would like to become a maintainer, please review the Apache Airflow @@ -282,7 +286,6 @@ If you would like to become a maintainer, please review the Apache Airflow ## Links -- [Documentation](https://airflow.incubator.apache.org/) +- [Documentation](https://airflow.apache.org/) - [Chat](https://apache-airflow-slack.herokuapp.com/) -- [Apache Airflow Incubation Status](http://incubator.apache.org/projects/airflow.html) - [More](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Links) diff --git a/UPDATING.md b/UPDATING.md index 45ab7dc96942c..036538131280a 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -104,7 +104,7 @@ config file. If you want to use LDAP auth backend without TLS then you will habe to create a custom-auth backend based on -https://github.com/apache/incubator-airflow/blob/1.10.0/airflow/contrib/auth/backends/ldap_auth.py +https://github.com/apache/airflow/blob/1.10.0/airflow/contrib/auth/backends/ldap_auth.py ## Airflow 1.10 @@ -419,11 +419,11 @@ The `file_task_handler` logger has been made more flexible. The default format c #### I'm using S3Log or GCSLogs, what do I do!? -If you are logging to Google cloud storage, please see the [Google cloud platform documentation](https://airflow.incubator.apache.org/integration.html#gcp-google-cloud-platform) for logging instructions. +If you are logging to Google cloud storage, please see the [Google cloud platform documentation](https://airflow.apache.org/integration.html#gcp-google-cloud-platform) for logging instructions. If you are using S3, the instructions should be largely the same as the Google cloud platform instructions above. You will need a custom logging config. The `REMOTE_BASE_LOG_FOLDER` configuration key in your airflow config has been removed, therefore you will need to take the following steps: -- Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/incubator-airflow/blob/master/airflow/config_templates/airflow_local_settings.py). +- Copy the logging configuration from [`airflow/config_templates/airflow_logging_settings.py`](https://github.com/apache/airflow/blob/master/airflow/config_templates/airflow_local_settings.py). - Place it in a directory inside the Python import path `PYTHONPATH`. If you are using Python 2.7, ensuring that any `__init__.py` files exist so that it is importable. - Update the config by setting the path of `REMOTE_BASE_LOG_FOLDER` explicitly in the config. The `REMOTE_BASE_LOG_FOLDER` key is not used anymore. - Set the `logging_config_class` to the filename and dict. For example, if you place `custom_logging_config.py` on the base of your pythonpath, you will need to set `logging_config_class = custom_logging_config.LOGGING_CONFIG` in your config as Airflow 1.8. @@ -577,7 +577,7 @@ supported and will be removed entirely in Airflow 2.0 - Operators no longer accept arbitrary arguments Previously, `Operator.__init__()` accepted any arguments (either positional `*args` or keyword `**kwargs`) without - complaint. Now, invalid arguments will be rejected. (https://github.com/apache/incubator-airflow/pull/1285) + complaint. Now, invalid arguments will be rejected. (https://github.com/apache/airflow/pull/1285) - The config value secure_mode will default to True which will disable some insecure endpoints/features diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 384ff9df31f43..c8aa4061e7271 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -266,7 +266,7 @@ error_logfile = - expose_config = False # Set to true to turn on authentication: -# https://airflow.incubator.apache.org/security.html#web-authentication +# https://airflow.apache.org/security.html#web-authentication authenticate = False # Filter the list of dags by owner name (requires authentication to be enabled) diff --git a/airflow/contrib/example_dags/example_twitter_README.md b/airflow/contrib/example_dags/example_twitter_README.md index 0876bdb4d161a..7563b816656b6 100644 --- a/airflow/contrib/example_dags/example_twitter_README.md +++ b/airflow/contrib/example_dags/example_twitter_README.md @@ -50,6 +50,6 @@ CREATE TABLE toTwitter_A(id BIGINT, id_str STRING ``` When you review the code for the DAG, you will notice that these tasks are generated using for loop. These two for loops could be combined into one loop. However, in most cases, you will be running different analysis on your incoming incoming and outgoing tweets, and hence they are kept separated in this example. Final step is a running the broker script, brokerapi.py, which will run queries in Hive and store the summarized data to MySQL in our case. To connect to Hive, pyhs2 library is extremely useful and easy to use. To insert data into MySQL from Python, sqlalchemy is also a good one to use. -I hope you find this tutorial useful. If you have question feel free to ask me on [Twitter](https://twitter.com/EkhtiarSyed) or via the live Airflow chatroom room in [Gitter](https://gitter.im/apache/incubator-airflow).

    +I hope you find this tutorial useful. If you have question feel free to ask me on [Twitter](https://twitter.com/EkhtiarSyed) or via the live Airflow chatroom room in [Gitter](https://gitter.im/apache/airflow).

    -Ekhtiar Syed Last Update: 8-April-2016 diff --git a/airflow/contrib/sensors/qubole_sensor.py b/airflow/contrib/sensors/qubole_sensor.py index 7065ae9262595..895fc171bc704 100644 --- a/airflow/contrib/sensors/qubole_sensor.py +++ b/airflow/contrib/sensors/qubole_sensor.py @@ -82,7 +82,7 @@ class QuboleFileSensor(QuboleSensor): :param qubole_conn_id: Connection id which consists of qds auth_token :type qubole_conn_id: str :param data: a JSON object containing payload, whose presence needs to be checked - Check this `example `_ for sample payload structure. :type data: a JSON object @@ -105,7 +105,7 @@ class QubolePartitionSensor(QuboleSensor): :param qubole_conn_id: Connection id which consists of qds auth_token :type qubole_conn_id: str :param data: a JSON object containing payload, whose presence needs to be checked. - Check this `example `_ for sample payload structure. :type data: a JSON object diff --git a/airflow/example_dags/tutorial.py b/airflow/example_dags/tutorial.py index ccf2e6e2ee235..3ab107da83e50 100644 --- a/airflow/example_dags/tutorial.py +++ b/airflow/example_dags/tutorial.py @@ -20,7 +20,7 @@ """ ### Tutorial Documentation Documentation that goes along with the Airflow tutorial located -[here](https://airflow.incubator.apache.org/tutorial.html) +[here](https://airflow.apache.org/tutorial.html) """ from datetime import timedelta diff --git a/airflow/operators/slack_operator.py b/airflow/operators/slack_operator.py index 3382bc2788dd4..98a56947c0ee1 100644 --- a/airflow/operators/slack_operator.py +++ b/airflow/operators/slack_operator.py @@ -116,7 +116,7 @@ def __init__(self, 'Here is a cat video instead\n' 'https://www.youtube.com/watch?v=J---aiyznGQ', icon_url='https://raw.githubusercontent.com/apache/' - 'incubator-airflow/master/airflow/www/static/pin_100.jpg', + 'airflow/master/airflow/www/static/pin_100.jpg', attachments=None, *args, **kwargs): self.method = 'chat.postMessage' diff --git a/airflow/www/app.py b/airflow/www/app.py index 1140be9aa06e3..bc5ee8c816ad4 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -112,11 +112,11 @@ def create_app(config=None, testing=False): admin.add_link(base.MenuLink( category='Docs', name='Documentation', - url='https://airflow.incubator.apache.org/')) + url='https://airflow.apache.org/')) admin.add_link( base.MenuLink(category='Docs', name='Github', - url='https://github.com/apache/incubator-airflow')) + url='https://github.com/apache/airflow')) av(vs.VersionView(name='Version', category="About")) diff --git a/airflow/www_rbac/app.py b/airflow/www_rbac/app.py index 7a2b18418cd1c..ef1a7adad58ef 100644 --- a/airflow/www_rbac/app.py +++ b/airflow/www_rbac/app.py @@ -120,7 +120,7 @@ def init_views(appbuilder): category="Docs", category_icon="fa-cube") appbuilder.add_link("Github", - href='https://github.com/apache/incubator-airflow', + href='https://github.com/apache/airflow', category="Docs") appbuilder.add_link('Version', href='/version', diff --git a/airflow/www_rbac/package.json b/airflow/www_rbac/package.json index 5f0c4bc592d49..eb1053b941953 100644 --- a/airflow/www_rbac/package.json +++ b/airflow/www_rbac/package.json @@ -10,7 +10,7 @@ "license": "Apache-2.0", "repository": { "type": "git", - "url": "git+https://github.com/apache/incubator-airflow.git" + "url": "git+https://github.com/apache/airflow.git" }, "homepage": "http://airflow.apache.org/", "keywords": [ diff --git a/dev/README.md b/dev/README.md index fe3900e37795d..a7747526cf862 100755 --- a/dev/README.md +++ b/dev/README.md @@ -82,8 +82,8 @@ Users can configure this automatically by running `airflow-pr setup_git_remotes` ```bash $ git remote -v -github https://github.com/apache/incubator-airflow.git (fetch) -github https://github.com/apache/incubator-airflow.git (push) +github https://github.com/apache/airflow.git (fetch) +github https://github.com/apache/airflow.git (push) origin https://github.com//airflow (fetch) origin https://github.com//airflow (push) ``` diff --git a/dev/airflow-pr b/dev/airflow-pr index 80dc22ef38504..da996d1d6aa48 100755 --- a/dev/airflow-pr +++ b/dev/airflow-pr @@ -73,8 +73,8 @@ GITHUB_REMOTE_NAME = os.environ.get("GITHUB_REMOTE_NAME", "github") # scope. GITHUB_OAUTH_KEY = os.environ.get("GITHUB_OAUTH_KEY") -GITHUB_BASE = "https://github.com/apache/incubator-airflow/pull" -GITHUB_API_BASE = "https://api.github.com/repos/apache/incubator-airflow" +GITHUB_BASE = "https://github.com/apache/airflow/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/airflow" GITHUB_USER = 'asfgit' JIRA_BASE = "https://issues.apache.org/jira/browse" @@ -1013,8 +1013,8 @@ def setup_git_remotes(): GITHUB_REMOTE_NAME environment variable: git remote -v - github https://github.com/apache/incubator-airflow.git (fetch) - github https://github.com/apache/incubator-airflow.git (push) + github https://github.com/apache/airflow.git (fetch) + github https://github.com/apache/airflow.git (push) If these remotes already exist, the tool will display an error. """)) @@ -1022,7 +1022,7 @@ def setup_git_remotes(): error = False try: - run_cmd('git remote add github https://github.com/apache/incubator-airflow.git') + run_cmd('git remote add github https://github.com/apache/airflow.git') except: click.echo(click.style(reflow( '>>ERROR: Could not create github remote. If it already exists, ' diff --git a/docs/img/incubator.jpg b/docs/img/incubator.jpg deleted file mode 100644 index 6f34a85e81c7437adbc7209dd0b2280a1ea1568e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 91227 zcmeFZXH-+&x9}@P5JXTw5Tz3cy-P2G3WQ#yccchH=pY?Leb#T9wUz;5 zvRgc|ehN=cglDA5n{ZH!ssN9l6%~f)Hd_f33-^6gpX#yY=vtH}pT%7J`kNFNkpOa4 z-Axt4SPxwzQ;Ov)XxJCkE)MvXROSyAR#}93{xP$Em$oL@Bf-o@BV)A9>yQ?gKV8qi z%KZ~BA$H+Tg=KzU&!s1{{kVB3*dSmx%YMwPSx^6Z^r@$m#bNN_*3l+U;%ZCcq)lM| z#Qlhhw+PTr5w7;lb0(gMhlg!_R@@2iM4laGW`*Q;Um&F!Fh=|Ai?K~^=O2sp+usbQ z@HBb&1dS+_e9C#-Ed5zAn(@I>$+&7Vn(JArvu~O0KOvZe>R%nAdqUcq7U2w-@7#Op zUQ6SBsDDE4NBo`LCLJ|5D|EKmAbb?~VoU9Ne@dwK)w)vbi?+PJXM58{eN95L%WE7y zmg3eDU+7f{kk@3j{~(gF;OFHZa-L*mqkNUumFoGwj!smRq&c;*ya;{tgzX7eAij~?VOVaO&eaU~m%Jc`TV{?8 zl)99noceI%?nbLmpk?}80O1>;vJgt_gd~ZRb zd;8%#l^7wJS`WhWw_{qLH0|(YA+ChiY zctNP%YJqk~DrDTlgKF+p{b`?5XGG-mmcK~7+aofnM~nZ+lckNvCxrN=|$Vt{U-+W)Req=k;MO= z{*}SMcJQwo{ELErN%$`b|0UtSB>b0z|B~=u68=lVe@XZ+3I8SGza;#Zg#TYjXd50S zo^8h6cD%9ls!^ZlHog9{|9xVWh*IWeXX1t_Im>VE{dWm>rrYM_2mDm|?|0LmhdlTj z{<*$FsaIj-+yAgg1?@-^b2iyf!)G-^K+yjzX?1?TbA0qQkxEmEkKO*`RFVQ^3*6xU z%;07&QF`>}O_BK{Mf*GpS{9+JkevTH>O1|&cWV}7>|0;)(`*b$zh1;yB!k8}l(k5C z{$B&RJZMVFyw1)S0QCUgL&%AtiFY-c9v;{N@Sw3#DD(gG-v1eOl=HyPnE=b*1zbWz zLmi&g77SKeVZW!$l3Iqa;y0hpWF7coUSc^b##P~RRg zqF#E|n=x0&85@W&;3v=5VVC(Odek)gWv6G2rj4H0{Tm5SDuu%nTK<;&m%XvlMBx~R zSZm~E$-}%!x-{M=bvac3^=q+ypB!E}{iNp8+S*$05@C1gOT_VawqbeL$aZ<-;%bhR za^oqsYY>qRd-W{Jwz^ZooN%3WOal#o4oHlceojGO4}?(L(ngVTD*p{XYw5AEUMlTd zQ}OiitdalA0NXgPn`1kcDw;%^Om6w(Hk_~hs$1uGPn*YIR$H6|Zhv4cZMJb5=-6;* zHcKt7X&exMU6keMVfGw<&&^~X)AW0O`MjubIPBwhxjF|xSx3k|khRB_xIGZ5?Ve;s7cMgVhEUj&8`)w&~ z`UGhlvT$Wteck5hGtDXOMI z4uk>GTz-2b>{=Nj-Cd-vo>%x!?} z-^Id!{Q1SjJ?*0L1X7MU3Qe_Q7DP&_!lYGHDMp^D0~)y@Lo|4_zW(7c1UuJThi(!% zw)D2A3>70L1K7r|eHShm=8bc#o7}%9K=D02P)SttFA%Hz34RE?; z_WGMw*Cm@s%I|aJ>Kf?9Ojr7f=Xip_i{A?!6<9?oofAaBgqX|XcxjiHGb{`+dvm{j z)VCe(1?_FL59=@~n2lj$slArk(PzYpC9St6`;b>BH^RFy18;1gEn(R*x|1DEOc|>C z9~e@c%PQs0{__MjZ5z>+O;vwcj(LKgX>G5OnBVylK!z7+u?(5((PBgyOk!x7LD6we zGaj?^1wm74KHa;Hov@O!=9e?b+s$=!}4WTMsusF}0=U`x4`itS)U3V(Q2(O-Jz4jNeZmz4f z61E+<*q_;CHZ$&tm#&FC&bDIBCv#F(xns?zXBd5SIW;x4RMxk4(R-sSiggS#BT4y3dCv2CY&?AfJv%J@<(-WEkBCh_sv$pU)m+FuAbnyj?avvv z6`I`nokEoX3oeR~7OMNZ(!LYW5A9iV(7vkHwyd_#M;2o; zu@@8$vz-7aT*2MlUEg^|tMjm>nCJqQLyXA>xv7~scex5zdT!Joca-htlWmBMqu1+p z&^99D#!t6R+kBdLjtys$`BUB^^3SwK0`U|6`b0Ht77y{pBP9aD$Hq@9rzcA?Hba91 zie|k>($0>~&RWJ-Z2}FnFBhC~qvVe6pHwI)sVPI%{$rMRPJ>*O16#UBJgwX3pX>z0 z7zh5w(BlmPTV_Eg2VRo1ObMU9m#cKNOoCqg#^m2Not*n2cOgPj>ECQDPFU|lUJV*o zYpZ2(>{bt`VAaK)g1;mUUcsoOt|zxlATMmsa#+Sr0GS8kUE14bHF@D;w)X~N$5$K? z+d`KxLBaaPxP^si;d)tf72k)vjjit%icOvhqdFX*g^>R^DV;TIVKJGQCb{{FuC2kZ!Zx zdhQVMS|K+zU3m9|TI#Gp%`&8A>xuYZE7 zZC*dl3lJoX5@2+-NTmu)wc$Jg;K#6!%WHhJmSmp-peM)T{@iOug~g(oLPZGvr&>b( zN;wu#2XojLCf#(J-~R*YmL^?^C*t#^&baC&H%<$pZ$THrlT`D9Q z52f+ItujK7-%kNy>ypo)TI(bH6h4I~EweP#b<#FkiVe+XQ zJ^cQRHxU)-(g-0c%+^R1&7ULX+Inm_nqE8Op!5Xz%WQ}?Y(C_=tr3m7$k7f~V$lPx zt+_gDv4E`goNL#MkZBSJ1P3Y96DCMdp^iHY%r~TFdU1>?ZS>jHxD9ftH9b?0)lM_N zcC3EXT8luGZDRQ`e}`)H8VP?>O=Y5FZlXOoF@0;(WefDhqGro!TYdwQ@-|e8edYaj zB6Kzn0KIYljoWuP!;}4_W9>eyt;mBW;*;7_k42y|)OVD^)BG}C+{dboEpb;<$+a(O zt~$)&SK7`N1wQqx@at4F$i@{^uJfY{WS?>m``)9?KNi=y)_nh9Bo^=Y2EM=8V!GcJ z7J?hg#xBj7y&C1t+KMlk?91nu_SH9$%`_hWwvFC?R+KKWK8{Eq0zamdG%(nH4L&VF z6)^=GPS+p|Lf5+U&Z8x_2bp_sc&CG|GaTsssFJ%oxs?W#Z5)v*;cToE)NOH+)p|?1 zZOX8uRMPXd=!NF`$K!r z!n&n(8dD+FfN{ti@hvTHE~zZ&nwa+Kb^JcP1qR1Az_pAAA|C;CYR1Il}o4rJv|q7^?v9FwqfJU!f&k${hJo! zGq;Os4W?3!*J*XXC21Sw@tYWL7q`H26t*1w;})ko~?MQR_SEaKuo z9k3TO(RMknqnC4B-E_b_hc>#}G|UngzKZ!|0GbOAw{3IAdd}jRO2oKhOeEkbZ+1`{ ze5(+_@D#9h2Qe#?c@{i=8C=n-2v-jU9;|2W<7^a8zj zQ|dk;&Qw~MMoPZNc*eSr?!xV?I~=I<3io|wb8)Y0WMpcOSMkLs{O`WJtx1(IKk-6G z3kq6@z}P2#m=f(1z`#JklcUv{(AEr{KE$PjoPLD1VH8mfb=av>K36{0MK^auq5as0gKC92$z zJ%sLF&QnllS4fzf$`3veCp&UW(wJH{=*R*rHzS}|^j$Z%=V_&UPxDRz=-D=20&oj- zhI;W36vUCr802HB8?ev{G1})}c%FrLS7i#k`()GE%5QeZF)kh9A$nGCH6@!3lq@#f zfZQt3^w0qA=HXmNJ2Pwf<;=BN-?VC);ugy^`iDpX{+XZCCfz-2o32KM>0M1>I!q6t z7aI$3&U+J`46?&ySi*X+b9ft#q!K)ee&s{WHH zg4A6PXcvLb6b}Clxcua<4D0{LeFDF7u)InbsR!ob)DgZWHyI{b&+opC@CYwOF%7Uv z2Ki^IPthm7YLM*0IrUj*ohDzC)h`!F3WP5Wm+5T|j+Pmn zB&vU(PktNZo0WTTVkf%3t|A^y)jji!14<_t+s$WsJ3@pH4NYWex;v~lVi?k!B#@93 z&(eNH%&bdalB>aa2LbsxX;(oD1$xETZINg47(DaUU-qV7PU%Lhpg~szS)QgpstI>4 zI4`*R$?5^xh(!UGs(G)3L=KguSi?-(CIt2TKnA* zISQN=b)*9`j5Ci?*w#RHFC6MEy>VANohv0df^@ewqU$0?Ubi$O-e*`2NHQ{-^h`86 z^5cc#HoVNhXX8#vcWrbUO-fyLzq#I%(o4rdhZ6`k^{~e&e(yVKf4Ia1VqA%HU;gGR zdJ?xfQo`Tgk8&`vv6{a)HSv~zFnjE1CP+A;`TYIb+atv~K!5r0!AAE#KaPFAv!oUcgJ0AHInDE z#+2G0G5B!N_y=ZJPI<+!vvTQ(3JQBKe zlQg@Jiat43`Uhhd*nBLLX+CMYkW7(`yUZDujqx=8(K9{)H@BjFj9p8u}vsJb)JHsr#1S8r#e>sO^45f4I>$uz5@!36WVlL`c+B$?v>Ny7cN7Dsa}li z{-O@p8{(D*A0bDiaMFR}l?$l&Y=7LHFoZU2}e6w~g_{>* zTivev;j>FReGPn5)N=HNj5(I6JtYyxiEZdf*N)ted(RlV4}Z_F#QnN`6iXDlQZa>*L>0?sC*1pb*L<=>t&6TFFGh8GWoO&*!FRNV74h}@E1*cD# zT&+7{4R=(teDxe+{M*zm#EZ@}_+==COE%Um0>;DnRW!zrN`{tXAB}SWUXOf;t96}K zpEg%aeaKuvVIyA1#CT2nzAonAvc!53NtN%X3L2CZcR6;P&lCdcG5m5|4VpDA%6=Oo zC^acj>XYnJ0N~gBULp7UcdN7>&G5@h*!o&fA5s^5n0l`u$P_HrDi|PzMF`e5D)aYU z&ozGD#+Myfd@F6M(+?3C57J-bPZYIjZ3_$VKA8O0UL7#fa+6P9+*EhlBlrLIC8W!F zw)wxJCQC2%3|-r{`|ca2jGqqvp0HbvGOe)I^cV;S&$S?`5pCm`(@)mml{p1R*rvYD z-1L0Kch{_!%E?rSkSV`l1zAPJ^n8Dy6ny9{Yk-VilB#-gEpzg!wvT{#yVR4Sfc37Z zQ03<~h0Q?=9*PlYiKpfSb7y48Zf2xriVeY`x;1N#bo{>HYoATYY8I-7LAM#L-ebRv zer; z_uk(Wa&F>Sj>(Th?!z0`U~daO0kbN(z?N8hUZ?7u84EeegIKx0 zc5-cPmE%a$ASw5M1`+VwQdh+asiuuzWt*7&d?5RTr$)mxDQA<`I%!OZ8(XChE_}&G zAHXTSF{*QlIgZzpOfi3RM{KYZWuuzWDl@F}!4$cHaer|QHrQn@x(0C;9)q^f4LHk( zXZKMr69nJR3&^_fJ&I#gMypr#=Ii8u_q%jF(`t`uZ0;(IOv((u?^Nb{sw>Vq#PCwN?jk(#IpyTIJhhk5SC-s0Ltkm%?lpVYUaxS3Hhl%#of>#9%` zLR{DPgD3azoMj|H_lFg?XUt~)AA+|OkC*)-Bjior0eCDMbSiP@Kgtg+xGmG8e^)z! z!i6u(>Nos@-G{qkqku5{{F0timO}Db1rU!$STt(ZmT$eSYzvS_k9!j*6HYMGf!NNU zAs6za^Mm7)*vy>LPZS!1tGrP?NIzMSWwe`A8TrYNd2Jzzg+Il%NQtOg4yO`v-w0g3 z0fFt*C)7d&`LSsQk z6jC^`u?zJM^pVrk+ID~SSk($y&hP5!m1T5osVr2wcKjYK)f0j(nKv&R;}7nIwqFKr zPpYxfz~@NQQ3S7(?l(;e1^Xa=5JQYH8sM^Jev5JIWklQQF1~y3@ZnHf5@@J8we5ekeJ&PmqG!rl<7eGTOFshOw79Su4U%RRm(7HK3{-je7+=(DosKo zn4-c~M-n0|uwZV!|I~lXNl~LcoiC z{ZD{KSv>cD9?UKO9De(sep}}ndy=^SIK$_#uM8RJRMGo>>r`X(xkz~g`*Q-wLyXa< zsT1pGnE5yfI`O47d8@~&qV&s`FPHQF0z?!S+=0Pg_gP1;t&I^vLvC`YjrNgknYZoM zXDDyhs&Ix6DTsO}-}cuz>`tlVAM*J_QXig<$-??9N>h#*H@x4ckCU=aguA0t+OqSa z&kWC!uVuuHo_Bv&+YXh&k4e~+F*xaNT(}!^V@I#%wc~=CVtIT`{YpG_+gv?*%RdEG zLW_h>`(7eIhmrC@zn9}L%H>vG_p1|X(ye}r?7zJm${#EbGNs+Ao5PbeOOo4;EZ7B;D7YZLpuV4 zYa414!{7K3%6S5Dzi@0PXnf>{0IOGtb$2-_kwU6JWbNM&GcUTVSWVlSR<7?_z1gOx zP`iA-4HkFNy_4QVe44Gq{*t{5lm->((kRVF<_NPg&(6~Jp_Th%&#%y|R}<>jy)I_D zIP|ojw z;Ay>;?+XO&@J%9`Bjv1U(XjBAk&umY@-j1JT-aigM@Wi?>vT`Xe(6G5K**7yMAcOQ zOBp=O0OBToKUiX@TWYB;U(5qkAyrfoL^r^QnNg>E&9PADeNZ8u@-8#B9Q*;4%%14E{XGso-BNcIH z79&E9Ziz+d1pjtf{7DzI9YUPns(YP!m3|9&Z{UdaxnXh=i#D*P zjp{9o@GFx_BQQGiQ+7)T4mVw)c&J)5x1cE~u5k|u2A++tZecK!`D@k$tzoP};;Rm6 z7PckL12MRW^IA)wXS@2e|7DUkPk)iZAzvyANpEDaZ5VDw6AL~fu59SJaJ7-Y4zx^L zKA|au`YMaEMT8&hHP`h5r${iCp5YA%l)39+#oHA=fhWQ365ACO>tNe)^FXo1$I?U; zK}ViPo!&3Y2P8nnLT}}2=~C*RkFcE~WZ{P&%SYd0lz@6@PhzGK*Z9$J`C*syo&I}6 zsOF|`0~sDG^#M#do?9A&O+tTbSjlv+tWw=cI2z(ct9566#}gr9OZ$iBZpzup5w6Qv zLCoMx2AkO2gfQF*PLp$l#z%R)wvL}Mnaze~Z2D(~RI$LvrB`++D7`09p=g%+>5xnxg_%& z#c>mbyD*P;+>}51_u(BE)QqtpDYuO_|z)${^Ot@yYtZ6KCz(azU@uti;V_lt)Om9RS~dv*yTszAUVDxHlzNeAeS%ub$kM&THRY7?t5$9-oDq>&90AvVQOI`X3MrFdEg)Yn4Wc;wC6;2 zX|u@_gA*-Gbbo$f@g`t(?NOZ)>A%?!!uC(Dtr!S9@GF&o-hrV<0ES>9_hY@h&V+Z+ zU2kvA_@?&eC4!UGwDZrAW^^qG@S8Pf2Hwd$8yj)g59Emka~A$o2n;@182Pxd;3x(4 zHqKG%FlxQnwukmR%GXk`<-pJ9RHAI-Zjm?ixX`_!q&-=I8L&cAiu|>G;D|10FT!y3$%_%2B z)#EQ9e1CktZ1_qt;x1=0-iyttID}m2M)*iMw>;uyjyT|Qvk6{?2Uu2Y8dG+-GkSDq zD%l=5&&_uG$UEcAfO7-Ei#ox;SdR^V%I>lj|DkmtwP1@*8T{5QV8dOP_wTVbJMy?l zgC|tg`|jkiyPVZJa3^{3kXn`$Pc!m4)hJ9RpdmW4{GM)A1XnEHMUj5Y3@E(o`IQA- zp!c*&FO+<@9c#HJ@6hH>!|F~q7Z|>j87>s-;YRYn23Lm5O}tr+N8aiekCN_Z?C0(H z-J!KAJs<|66aMlb{QFfv2h2~bp6iKgO+7T6d#J}(LuS_p6T+`(g7BQ=W$6_Oq_xUn z*!WwQu`y;Nop`$#(v>#MwP60Bp8n!@5tsbho}14pB{@k#rLJpas?N&({AD^@L?+bs z+c$ouu;9XVSP-ElTc^n~zmR7J<7{&6C}#HSXB;~hW(0?wcaMvV4i^X`&a4YAG;z$$ zi!qDJ%5n=Eq2}*|g589!Wrxd8R(t47&K|H1h8E}?S$zg~bxj6UL{8w<$@Z*}CLdUM7e`L+nZSo-UvhETGO1K_) zaaR{WdVNq)G2>}H;7O_LSzp3Qq0_vxdk{|A;f=p+g^aghXVI-ihCT6_U*1Oroe4B> zm7LLKWGnrBKCh!EZg&TUPzRtw!DuJ%u|j=R;PWW>hPeI@C%YjzUVZW5fU0u(L8eBG zlX>W_JIbCG*IUC&kCj%w-4Giz>oMeb6UNJ078p+B*o*I_m`PQNIWKNLLS+!Kr6oT6 zNS`(;iE#?2+@#te*;jPOw~mU>Z5+uqz{5NmHyjFQqzQ$c(Tv#Yh1~%Ap9=1Dx4j#p zr)QlEl2+SB#jD(>W(Z2>6;+}WvAtI)Ue zWQpjb!f9OTuqkAAi4r$CGPfs>aln^l?|(cCO4$G8;T}Q$p8#7%d=-A7m^>S`#NT7* z>6zHhg_nH%$Y~zrCcLwuxTj^lVQr%6B9oG;u=DCvFM;fCc30!37(0(z7He`77SV5-nWj- zx2=z~BjcHxUf!#hMU|wJS-9T~q(pVKwp(Q~`XpAjs`!d3L51NnDf4Gdx~V#?gFS-$ zLL(+c6ffSaJ{?uRZ1ZDmo8=R}(o6r0RMXv^7Ga1(S@y2R&KJK;gS``CFJh{_jqueA z%g~71vP85Ubip^%dU{Tl0tqEQY=cDE#6@lmt{B}dOU%(z#Ch(;M?lC~38IaU=R4%K znXMwcMQ7$(GTh?3_D;J-dbLc?#v`9Lp6+^Xj$=*?ghzyN+ zD@y-bc5dGSip$Gtua$WlvU&_a6(K7yr!G=1onFcr-EoDu9klzqWlyDN<^{tobD2sC z&a8tR@e-mqJUNGRmdo_wbqMTG{7DZ9Vh}O>1Bp*Z7_V@SrgCzXq~SN*faBL;5fKp~ z{%d^^R*OxC{2v>aJ`fpD`o;4tizT2g-g^+-+iz7g80BrEmgk?Vq`XeXQg;hRckmt8 ze>B-F5FMHJKIL}aPA5<63g-J|IFR3o^wFTAw=oC4ZqJsgwrRj2YsKb<1CnkSeRD@- z&2CwYFK_)r!S5{69hXX!0-9w{NRU>nt zR^6_QGqb=Jg7zYtSwstscI_e%Vg{BE!|!<7%xO7<%&^kJdv6Ocn*)g`Dr*(XaBcZ# z&~vcTaA$yMm53g&8oo0=JwkAn|3mg(rkzCLus`-<*6zd7{fOG|EdA;PHdZsnxzSF- zkzyUnB}^|DKivKxrw%vLUJX`4ogT_9DPrtkt^EYZdG^f&qdP{g$&QMmiQ(pXzFT7j z25!&BE)nvxfBwh}u+&#@ANfc9{GQ&#eKm6n!3@ z0^U{MW$bf3P4f=E(@LL%2YHSkoxm(ZitnKyM6fk$;w%okY6+=1Xi^C@@8siS;Q=$a)ih!PL`)}Fq2)wWL`m%hZexs_fsJG@lc z`g{GJd)AE@H6x)FJ;1vul{U??7CRe=5^_dz%aEQpaz+sXG^IbVp;5$L@nXmKdd5y> za5q#QQ7^Lahddi0r?QyZJif)3`5Fw*C9lx?viNSTNO=G{H6GQL%f`ZzJKqw579B?_ z_+|pRFKbrk^`ue-q>+lVV=t1`@-SOg^{#9RKBvy<DkVRDKB%Zo4R#Xu(Rfzzt}~W@uz0r@8beN%G%!!`V_7u zV^ukBYm7Kw1@PW7d=@SjtICUQhaQ7z_roSSgPSh#G?4b(7hP!{-<}Ny=>N!PUt+K` zH_bj5BqDje=@g<78c_z_hlt%?zTOvH7|NKw$(T0FW~k}iXF9Gqw=6;KsEcl4l(TjG z+*g=0EC)B|U)u(lx^^Dx&~jKH1y;hi8`iFVg^SR4&!{U9O7nq`ADPQ79QB*-a~F8$ ztSB~Rj7MW^B-QiN6J@{y`ujh^#yZ>Uae<2C*B-R>0)id*^18WBse0_u0+e%<^y>fo z7eGehQuj@tzmk+VBJDUGw0o9`#F3!hq+{I+uV2z)HsqZ{xns2>@$UV*a@aBaey%Io z)M$8P2g%O$XukBQ&D)N|7!NIHZ~bK>idUu;R!pbCZKXlmm(HJH3aR+H@4x@DZCEvf zgg~=0%9)1&hSKAfpov_4%bJ{<_Bn0UdaD+ZVo}3O3)zixJq=MAzD3>K=EnuOUzPYi zjfacnh{EVhv)+0PH^20p(MdZF58w85G@>h8e_>PDLB1C5+a~ICIR7*b%h^b=2O!}$ z&CY~U6{hbo%Px=2yG@eH_G2uPB+D%}J7It|W@#d%+UBCLn*ZKrs$PLWD40|M8G=Fc zRzN_TUpkQBI|@e!0fZ5HzCbZuFI9cjK1A|~mBwlw3pzA3L}`>I0`v0`BHI*E7Uq z*)u6nj`yIN_^OaaNo840cgRzy88+NP$d`3eh zthmh6|2gu+MmVp2BxT1;n`@C}1%!?L3az+6yN?KF-{LNJopvfGPp@vD+z&SKf#OJN z7vQNeYdp3&Xz*WSFruD6CN5Z3yHGA?$XBuO@ep{PcQC(SQDW5+ZT4+h?`w9qFww2s z_Qa`Jl8k1C6}rYO%Cf5Sha{IPkTcE8>q zbm3cXG_9Kp%IvTP%a3SvNY*KTJi}7so~fing%z=fwTq%Kg79aU%>0{OF7=6)>WhS* z_R79h_a4=T7S4R4q1ioq&%M8P?l3kD_1eIC+)6|0J-wfC(3YIRbAKh|x(+w-xSHSz ztiO5cl7S6jAnMkqT^?R`(aUVW4x3b!a6_CJ2IoA6bs4`>y7PW^v_@pNQ?g$$cK@z? zAJvr&s8!wTst9CO#Tst9OK=)fiBhNKrPKFvN?d*|4b053l-C0s&MgdW`6T(iw%{#O z3xc=2-SEq?9IYgO5Ahx|(Ct9zO!3pmcv{%`7c+snB2v7MT7L5NzL6Cg6H@TVX_lp; zwOe6gs#85p2sDmhYj>1LO1o^N$THq3~AO0(uJj@m-cbXG+$+^LmbmH3y zZExh#(AkcxePxN^{1i!9%OA(oSlb}Z_NW~io!&Q>^;X^_*$T#D#;pD#dM2))d1cL{ z$7r~#)l9O4Gz?yYxMJ&0R}lkiA7r*Lxpv$Sl{0f)X0|4?tj#vQnH9a0u7va46gBEs z9Bj1Hu?{)QmD&8PC7*fbpW2BI=iIcC$f9CkKb3p>*>L9BP>kggt{>)-d$F7FDsG-*rAv1w4asBYq{I5V&wM_94aFQWI)E8lY5Pxv{Z{-EFU}Bz>~$bEt@WDD`@g zE2A?pa;*n5qBTqqWsAClQ}OBxsMSvEb@{{45VgA=()k^q?YiiaodFk%XvEgRhqd5A zc1&f7OT;L9b{W?*XqZEo`)iy9Pxka5#zo-D#GW2cw#Vxsm$@abeXR9iAdV>wr6l1j zKg*sP2CNRF8qyQ(2{v<7y}2Exn#JN6P&85nW3#)h?=(^fxO zAZ5WJ{x>y=T=noELyd*8ohw!==#{8azr-J7kBbNS zDn^P0&W1dM={A2y563p9iFsS2_0KI4l&4mdSj<@iE$mG3oMRZ(BM z=zDkG?0CN^4kC)ZHXXiS9F8}B(0?D~dz+8w87niSZsY*LA=*V{O84&kk`q&m{5cW~ z52PDdC*qo~l}V*5ODOPJNPG@CsSYSK2a=06aye^_z2GnkI@O$g^Ls}C<;;Y%uDSie zunsZ+f5DD;GRi-U7#rY#4_mn|rs|0~&(+fE9wLd3e)HFJAR3(;68{KxS>JE`XX5C? z1;Ng6XspR0>q0Gv(lStq_maYgXNy?^cPagO@7jx_o_A*BFdG>e0eOjbSK+CX=W`Qx zHaQFKt3*?!J5B8atXUY=I_8JYRSnqk-M0Elb!<|GecsLAK2Mcbu*$nyJZqmmx9U)O zWWBHu(J(I1h|NDKzdILVn)~kP0__VdU&vQcZkur}NgaYvi9h7rqJeNps|?)iXj{x# zUu;)6@PZV{!r~KkZ&*{9B_N9sP@6=`$lfvrDB!*`)jUd@aP-8OHtQhl6o)tBX~1Qv zt!siwvLR<&>c~zk>sDEvO8Y!jS-K!@XY>^q%2ki`kq~~*K(=U)_i!zc1^!x7oa_y< zGn)NAg=@2_9&FR)+30f@vuqD8x-i3T^`Ej;4Y?HNM$+VW&b6(~Si_eW7&e#GJw>ujSbPNx;y`8dE0T|D_J-b1vctmg- z@6TEfchi;xcai#g>CiugY}|*|aJE05BR$HpA<1ED`wa&u9I0eH)9iaGm-#r*lEydD zq08zwZ#gr?ixz>g)sX{$sm)2@rLmsV(0)## zIsRD4eqM)vyslJ}-=|n)`jP=kAG5SqDd$<#{`$KS+WpnBNYTNfPWnHRaao(L=nuDy zS$ho%QVrxZBzwKSEESs5_v^@Iu$^Fkb(2bxG9a2w+zX~dMK;jlPP@9qDOcGSKI-Yr zFKQ*XR-b~26g22c!b|KlbCn^H0FR*>tILF>pe2;TvF%FRJh2SIKS@E1 z6n4%tdneG9nBu=oQeAh=N$fRL^J{ys0F>^5YTl%~qCao)KygmJ7cctoV4~B}tg!2$ z+SUdSYx;;F)WPfvN!?$n&p{EDGTZTkL84f%jAPhJZm+rRXO7Yl-roFp zy1x?$;EXt`wBJ=OEq?cWr0)oAyTtp&DDY|94BwaQ15EE>N03lUBkQbh?+pA4E09Q+ zpPJ@DlXo=TNty5dA}GiSn`s~M`rMu0T_m8T2X@l~FNlsV9X%uKfst~+Y$_H33>@q$ zmv@c({7SS&PkOh?&z8d0_7*GyY2&mRi+N>b6=jhazP3uZh-#`Ao!&Zpo~vmFvQB|i zpXz7XPX;|rsLQakTa?^)BXjm(s3Kt32*Ux~XR?-HaZ$}@yPdE)rAMq(K4zxIVWNRI zO$rILk48PUJLFL4vY2kk_7p*h;$a69Xud-TMFF~PRRwXn9Q8LnG$|6Cj>^T&&qD^Z2z=tjEPcXMke%u^xoi3=6ALpgi9p2O=5{ zhQpU5vG!vvpnbr~&-O#vsYDbO~4DN4`oy6%R{@7rnw^KT&mX$cQ;1A!xiYrxhLPh0MbXMJaV zn3Tx&%%3&q?wG#jclU5Nd!H4Hz?P<){9kDwZuC=OfCALl zRRk@3bjwZqj_SKdtL#{*QPEL9=i!s8O;$wd*7A?D)6l0^uEv3ZWjM}C*C8YTX1S2sRG!BJWHkEbu2l?9C?AY?|F5pvpG0$&i!p$i-Ne{_8-!Zmk} ze=z66k-Q(nb2%VIv<%T!n(KlyT8@jwG;#Kd10J7zrOO;SQKbBJJb6a)f#ZX>2k3?V zt{s{XGl8XZT)vhG7wjW@wGS1bH+f#{d;Qm0X9!w@O(s6SRu{{`6x=vB;PoD}6f&Ta zt!u#0weRU@gF)rETH^ zBA*|7KSLth&k`6l?1n^Zn7?vjLesAK1iAQ>fXsO51U(X z#MW@QlB9*)1q5qNK#YGWC?>xK)xC`U@HxZWd{d1y=t|4PN5B{Fl;^qe4P&2XkU-wy z_q%9@{IqA~kak&m?*K)-QCivls_l3zKsm-U*v0nI2AM?&1*h<-`ga)EvuR1s7<5CY zXCI=r6PP_QQ7vu-v>$>YGso-ef6osNMhx84#}<9ei#*(gdDq#8fN#`iSHKo!*Mdi8 zdgxI{S(p8?el#vtvg?kol&_2SOg?O6{dr6P>*i~@=TAPY7x)pGcT8Mm%Z~ZBx#p&J z%pmF}@ODB-m3Ik-s9;w|W8}b5U5XVi$Y?Hy1!6n0xv+TFaZTs!de=Vjty;H|;lTIK zzDb7)XCt!@v0-GPSugZezx`5=suH3%ltKG{QP+JRN+v;2vi;?re(@uBw?Y3;ey5g7 zhU$Sl$X%;Nsn31??c2E0;l(r-b=#d?WDJo*anutx>+aNP2p%iiO=y}vVHBgW%5%Ee1<|iHhljCCj_f( z69e18KC%Av7%lwWL zBxGFYE?7-(jXcnhGcL`vXg-w_aGoca^*3u;rayOz{3$~>#rFK$Px+bUv8;7}l76zN zbzc-}EjSRk$8LI(hdkRP6%=(2NHLvt z95SLVDUUp+GAM_5sxbQ5b=qw4=|NgUBsxqJxok(ijf53hSJs!KKG!8nQfA3FlE7qI z5oD}yG)${DUT691sa&+p2iLoix|>yQD3Ay$H!7}Y#icyOvfyDbTZy>-Fi(;Huh;o- z(al}GvKe{wWU<=HvY$@0M<#uezYh}#-6F!vaPVY7Q?F)T{&x6k2(%}rug2E?nzr)B zjU3g*v)yhg9S^p)bHAzn?&Zsu57PDRVy`#c-rU^cBuwU*<7ibj^`#tVw8E+v)5+xI zP7{yJXQ%VoWVW2m(jhDrqq!i!**Z~wGc-33Z)-Q&adX&{1+D*MMnaP&U|D~RzdpD< zaHd%(uRU}iC(OWzFT2T;GOiL4JyPWaicBd++WLAe1bw*TQK8%L4lXw28CV<@n;hxhpGqj<7j*3a@NOtWsb ziNZ1Kqu|My`6#5e1!MMyZDHN)Gh?=;P5e!2Kg;&D``0d;&#`>wV|nKb((S&o?@UvO zZi5HrZ-0WBY}Vh}GG@8$`1~}>$CKqy7;nE5=E07FIQz$UKrligAi;yz@o9r4{>F{J zSr*&G_>CkiSsvT{NdqPfEQ=uI%_P>t^7(zREts+$eD9cFNMiX98`$sGk@?v-2FC1r zQ>e@M{CAf>oA0Lmj@TCh7Jdut6W@LN`{a9IIjwId?+>hlZDn5OVL9#kGwGnfrxQ3i zIicwlwI8FLM~4I-NgN9tTfGD!=|<8|SBI3ODao$i;}*Zoka^P6ArHaIy<&D3)nI2) z?c{8`l^KR98#lp34&2l@l!;WT1ROlOBRA+z07B87XNU}o&2$#0XrAyRH65uUpc6*^ zPYFFW80()pC2W4?1SW(IscLEZPIBz?@Wpf#Pa~+*meiDuli4-nl+&BCIQ16!Og>BS z!ZF3br>G6aw?0DO7n&~_|gl*?QUu`8)$b4BI^L19!^d(<+gC{_uw4a@7 z)K|&%N-wgLEZHrn%AW_VfJ1{)lE;`6##1q9D-Jj7+De_kQ;JYd@Hb zPfl`w^u?`vH*Zhpr_)rH<~CHRcFl3v-bst6uUAy|Hfznbn`95kc$=;1^w-(t`j2EO z>zQ)=a$O^tzg$w=13IA zc9Nb;dLaot{stH&3Y@Pey_3XM=y#JSm?K9a@TW=4OJO`&e^UUZphrRa$)w9k6cn!{ zQNX0Y%yKDEGyb##-#+W5;7tLT?fFU4-z2eJoO9TA#uQNb^{^eRmqIby$96JL8!Tk9 zd5`VmHI~mgl*InAtv^g67+_gk?I+>G->io*i9$2$z?6b8`_12NVYmG?GiF^_v0j$V zHnraee=`s3xtMf43CH%k;yvu%O~Rh<=kJpKUDDfUf3qJX%vkn&NqnCD|9%qN#p~=J zpCM3TA3kniStN#=Nq>>_pORPxUMz!sV!sJkFvp2~Wgpnr^8?=#Hn{P-;BUTH#%wG5 z{;)$ab(C=E0Yt z-PiSB{;R-83($vTI~Ve&{NAWOH;0Cnp^Q6{Yvf? zJKfw3D+TmrZk;<>9u;Gnb8!%)_)P#nV6)1F`+3Dv*YjS=>+|lUI347S+0VH-0m&eb zc#s*}-maxnJOA)IiBK&nhrGY-gQ4!~71%`_&Blb=@&z!6qNMwuifXKA>%B{)5kXsIy+4qX) zDW;|*Va3j5wj%amsoBwzhsx(hUg_1623Nh*&ZMnG(+4$O-|H!nOxoNY{#rD z8<%Eab#4{R)!I7I*?#ACxu0l>IBk&!x=mBOX@Zey);G>0A+w}aI@rv!=sW59e>c0% z-M(g>?t+VYvuPeSlP~;dSGS9w|KO+zXvv4j%gm30@MRu2p*nm>AP;0J`^##OCkS0m z%)g!MG#5t0jqAA``jy;WaFnBrxIEkNCTNvsqX>|*uT!YSg&>4!3dwC@xh+_;kIc)M#Ov*{ zGuaF*yWJ+XfnOl6k(lN+-e+B`hviUUKR@7hoW%aMLB$)Helv-E|G0r=x9e|B8Go9g zosaK;_5CD?8l>|B-x0qhzN6hF-siQ44efs7gf;umfCb+T)&vw-;K=K21Bvg4#Ao^4 z@_l^x_k(orvu7v4_pc`<#(59)FwKgvOVlp878(mx^{ z#Th!)Ne+ROpmQe7lRAn)uBWY=J@xZ9{XBBUOP7dguJz~19yvS+LI_y$I>Ac*%QeMh z5!9rlS4aYa<1C*?ccddk)A7Bzb64i!65~eBhUpTXT8ab-IaAX(ni>mgXlS~%zC4-c zTRGTi7R6b)3<9^b;?QQ7X~Z$DTx_f#63Kt!VaO&hu_f-r(k z9(nc{7I`;OOD;}wgW`2IvR)6LqwJDqZRv|c7ka|K%$qSw$qP;UuVg&z7MaGXmlyLD zMJWHyiVOK>o0_oIAe)z<)6l z?WoBUnV&T{e8~kkaXYL9%9me$ImgD$&gILa!G&v2jrMmo)0eE++1}aP$`zaDE?T)7 z!(o#jcMhElIZk-uVGf3b9l5pdG@V6KVbV0wN<+;InW;HNug}x%-LaYiJ}(j5+&_ii zpU6w=|JE5v4)ul@x?YoOPNrLBc$ewZg8V4rX#wZT>JE{{BOqtmPBgIrWkRM1J;I&BipvH-#Ze(3%DO%a*GF+l}C z`1Hwce#q%Cem^5vXL>ZQ;zv#=ath-7ohJdNZ$|1TQoyG$ka~y|{u5keniFWYB!NSw z6NF@Q(;O^WP!emR$44IT(JXI8X+6hbCqcqI{|!>8VB0z=MfdVrbNkA?l3ptL>^dV` z=A^M;U73#%fuN`fSTg2>Kn+V%hR^)GPd9w}n6q{Y|2(>$D;Iebdjc?)lg>5?P6&h6 z2XmehpQ&}95%_uL34t0*NI%hZuIHA=Y!BN_votKzPHtUT3p=TD6ap*i^cIq@b=QJC*wK~f~@>ylG@+hdSjX`@}FO|04ExU2&6et=k&CHuv$GcI5^nfxOVN}%F|DO>s!hAE(``6 zqg=h&>2^EAWPm;HW}eCq@~d1)SWNHTz5TP?)Ay8H4$@TXWO8~sTg+#97Ex{wk{@=O zY4Ph`-`baQSM$gHScKz{;(hMcwSFsyF8`maEXlhm7cgT6UZ*s$9+Clk%(uu>)te!o zA?v;0Z-fov&}|R9~qL6*|1vg(2c`Aw5xZ({ASL(^+_$-BB3gll; zY6BDAZ@*qX+j3JdWIa!vy~fFzfP|}rzlwpcmx4N1^7-D_H_ikP%b?jt&Uq9dNZ2su zHMaS8&&KR4!3c@KjQ0sr7_+Zw$sK5#X?Xmd&^|%;f!n<*{#f z&Jwi{pEfA@|Ji$!UdfIuLGKPezbQsCI3|-hRAps#3qlGiNRWWkri2vG%Ut)aYo-6d z|DaXZAJBq;HVr5nwGavrYEcbTH@d5sm93n}Ob$T?BVN4G$Lp@|JLcx$e2gF?NM z8-}1kS$-dZXSFCqy;Ba#1&zk~RB#B)tFFkjo)ws^Qr(4l0$~EHL=3w|kGr=R!_bkd zfnS}$*y$!tii?-j3|L1H6f9a?0FtJn;$6OnNLX=UqwVjNyT;HG$tFky&J!_HM(8T# zp*T-h!A$rAK2I~2aD^IFI}{+-dap|ZH75d#Tn#c&Q_4uBm=?qK<1kjJEZ7*v+^-^+ z#e6Omej-UIO(%mL@st=r#4MEdg*k9rouOo+K>;Mdiaz`e1li)2Wz2aMNV=1P@+Cyn zk_TFHhxq>yNcYUEZ5#zHxp$HnsHoP}}>q4weH#q^_}u%59vt{_YvcTb?-t^2YNaP+he z2;99ptoH`p8!&QrdsSz**BQO^{H+)Nq|@zPV?pe4-N-=~#Ss?4DpfMi=?N$dR1 zM<4v|^!V^lOIqB#F(02DsXu~oa4cDHhT?^XE)h}^8`Hi_@wAwvcc$rJOFfeDCJ2Yf zw9dDs?;)234UcHE&#`PEubW;y6MyZjM8!*ZMPe=zBpL1k)sg2cM= z*o!sIgj+6`>C=3+!|`Bv+X+j{GM(SfbU8i2*0m6LGz5P3v!Bg=@{^x@M2aIgA05p8 zI%ZGVZ#~4=fdIkOEH)`2O1zX^{So38a#e62VpjkC?&(=)aCCTMFh6{SD=}DysxC-5 zIA{S%+C(79o)HG{n113Hbt^_&@bg&^T}FPe)e7uZ5-Hxkhh8y&F>7Q5h$tPYs=9zN3E z3VsYpq3$47Zm>k}QohLj4o~ht;iZH{gDkO{9hxYv*o^wi>%hqcnm9I^6Wa?kma?~( zS18MT5Hegqb%q{}L>Pi;Ed3IH!jSrbvm$Q|1eSc0ibY5)!yD^a^j1&*=M-_BB7nLzMWb zTlYtRfE&j*-NnXwefmmWT<`2+spFvT4E8#`-a)TBf(YpxVAPLm4?CJ0jI`=$VK@t9 z4VzkUMb1y}pWQ#abBcAXDHJ}7B|0Ns(!{dS2g#R!i0S4slKyX=58`M!umAMU;8f{a(g{pzlDDpZ`=JX%tVgugPb0p!bJkeeafe0=c;=Vel~L``i!V|1g5&VfnV> za(?w)B%o|(j?2&O^li9>z{4Q0f+z_B9MS0-j>+Q6l`F@d`?2qGXGM94|Cm#AiISy6 z35B3PjfUcCbqFGRud`ZodW(6l+e|??QFspoK8sR06qA02Q%fR5OCn~_K+rT0CkxgomE4n}y>0W~0$miiD$IR(77|MF-_ylPg2~_P-VhLqjd4Bt zdbvC$3|S(7L46n$@D&-k{|L%<}Ap) z(4o<+v4#QC;;g7vm@bA(B2i<$A$268&Ly3G1Cs@H1la>V;L?ypx(L}henBpvd+MEx z$@y(;oo^*XXrUII`R9TT1s7bQl{5yIq{#NNAOMNDUfGuwB@ZaFQk3VYL1`gUkE!FZ zLO>SoV1%{9rzPnr@bW|#0);Cbi`7TFz2e|s&@dXlcnK=%f9My@pK>$h?+=RQ|M?FE zcg93V3J4Wz%Fgii?;PJgp8x_+BSP~8&HvI#(Hr?;LVR9q1}{GU+Ldm%zB(N3?(GbB zu5i_1-&2{mXGE-89wL7b&6=Yt;z`b8Hk+PLXVd8%Is{(f!uv;EP-50>P@*+n6jX(6 z_JuBSPl9vuFk)qsJ`pu3Px=c{v>E)^>ez(0(->xv%rl*-ty|a4%nc*NMmqQ7X1fCCNd^|61KP5r8K5M361~IaaE&WEK|NS-CqV&a^lPW6p#`bBH5n$i)d#g zn&8F>f75y*?ItR?oOco)emBDsHs;NK^Ua9MPT*Rfb-8hixIe5uqQ4Sn8*HOo4!w2T zkK8ufZeKDY>)!2N<(_AHi7W{Zy~LOOw>--*ujwU5-d;c5Gv$tC4vFcdT#|JN>~Z zu-A70l^I*Ex*wPwqaK$7dwd^DS~JSGMGA9CT*!5h3#68c{esXU6sCabzoda=mZ|Ks z*1&==BSwO)gnQBt!i^#$uM50njKrSgpTb{qMObqZzP3}0M>U4l>T=S5ZhoIZGCLX;bL6>ZKX?`SE*MZJ zfaIt5&_RKk!9MY7aJ5*x`4){$YcG%1eGv%cP4$c9z3h3)AN}Y@o#CS18&BrdUhm-Q zw_pGMpLgrd3+Qs}?e1SWKyl;%6Tc%CM2$Qkss-~_B4j!_x&7YTzre)rxvb;Ek3PCL zK0lr;oN16lYB!xa-6B=HMh$vXZr`P^$yQPew?{+?ym~MfR>DVhUJ>QzR>!+#{15Oi;1$Ud3BTwBjvx6syvQHR zWqgw@i1INKqUfe$xeuF{+x4k~A(H?1_&=1h4H8=~;_aWrfrOL&u@Cme&+T3#5-L)w z$dz&C`3_!DwwLkd&w3*SBh4AlckpknpXN7CBE_;z|7Um!Fw3!CVJPA7SG#O}Blt*XJ4`414O^ewff)KczbYq;vwz$&x4(<+vhTuGBG0)e z(QLo%t8Ko6mst5F{@>%@#ftcasE)h-Q*v=)nG|Xfa6~?3! zSSt$^n7&+=0@0@~H?#M>WU*x7sg6mn&WOj2!=lquqzXlJOgaZuehzbi#SUCT5-aYG1~UFyg6*snXJaF**Z{6jqf5h*h32%nSDb@Jc;`yJdcN3c z`n1`6=y0&;%&FVES=3RH;u6}}^QA4S`&)*jQ3LdSb|AS1^ZBw5%2i;ics!}fkH&Rp zf*gy&UQi_~xG6~pjxof?R{=?4tTSNT^qOW!TM^4@u0W{WkedV{J449tdu6Bm9|uT& zL9rfU6YbX^eJ6`ac{Xg8clU;id&qiB*^Ha}+;=j9`ohrI;`pdI`4MaCKRnn6Y(?vy z5CU#a-&pth>yvVPHe!Y@YosX$yQ7_(=yANz?e_M#Xk>?_YfyLUBxe*hG#s@drZqeI z_|BdC$M??=%vzkDoQ!9S$=m}Qr8^}?yu97X9h8VzyYrHmAV$^%i8^|TIC(&@Y={-} z=XB(ZRl_3Ij(49s;A!f`Y_Y85GM4KiJR4$;rS|etEW)!5{4pLqWa{PBxQfQUFtn2792G(60iRs{CTxb6B3i)22k zzAsZx&g+L1@s`6O!fs#v@Oex*030BCk#Pr!#EDL#;W}Od!l$sY|01uZb%)X0ho7&# zVF?=%V#|IJFF_%ZVjUvTK0nWy#&%e~W%+#AT-olQ;fwtv>Y>j53kPBEchIFlpTb}F zU;CQ9ec%2rY0{?m#!IjWa}mFP#QzE}ydT%uZ@*E7gK*d9dI+|zg}{?T04|C1*u#nJ z^14&L_S$Q`?|=XM`*23Mnxx+xA0MCo{onun$-i}8?s0nEFFpWYo%fsi+Wvg?5*L>^ z54h?=1igYF@pQfdqi@usU_o>LlVhOqnL6|+JlP+{9%!|aEKYg%Ld3B7rDD8JE{eno z$UPHH-Hcj9Uwsn{{C8MXMjb?;%5&?J ziZVdQDN{mZ7M5*2C6p+cfM0{pyUT`Rx+jUjWw8xV7V4fjCDUCf@Zlq39oCYw%0F*8m0izB(%O!wVLda5V!}e2$0)V7 z?u$UcS@w-|Z@f{IM@N5FFV3o+s$O>OeN^YL&=G7`n5>wIV^L$$ zYSEWtjm31n;=WRTv3$fGBhdOGGb%9CQndza5F^AyWZt*k!BeU=%@OZiD83eFwa)Zh zy$HUs_?#Vo|^G#vA^|s@WSEY4Yb~a`_`V zcRY0#$<~XYM&EeaHxWKJLZPDfw-Eg-XMW4o8@4XX_9-lVTBqLna+qM0Z9Sjm^7qJX zw+x8~k#XDSd&~4GS3*KWLc%n*Blj=&M}%Jl)~9{5-{#HVHb04asIxrVAmO9enaA?2 z)4H=aon=3c%W$wweuGTtSfBm#nd`P5>l6lZx$KXKx_NV-a{3G(;Ub|X93_-YFQN4` zIs4={wLh7dlQ1;ie%N-usZOH8zTdriEx(1CkD%Mh)cQXKBzTjg;yM3eF8)-u=ino5+x%u7Q-TtksuQB|F2RDXSc6YDb z8uUjuSnx)0NFpYz=?jcG%Z>mmE1O^S2-Y2*w)mp!VnJZ^!xmZu*$N^qq`~;JL_!rml zBF#Dx!8>a&vMR!U8^3+x$InkW%XY&QIn0RoD}+TPg}8CCSn?G4Go9)EIM$*6EN5On zvgMgaW!)mlwu$w9J)H=AhO<7)u^wT65C1-1C$gRUV*SGIRlMo_?IE7O z{Z852SHG)ByJ;-L_UeVlJ9z7{jHk&7d%w-wc=vc==y!KMI*twN90Pv4{~j;Q?aLAV z1n)bYFx3fD2{B<|8N$Upen->(2mF7=o4<7}1infLwBepjQC0f0IK@ zZYilyE~iHIA(n7UBtYc0^pu1KFSRusi((ZFQV1Ma%U0efKc~DU3(=6v9Csru!2(XK z8GSERM<;~Km!oWs@~I0!Z0uD*cYwBth|6se$>$%D+>XA!BoYLj8|EwT2!r^L!y;Ej zv^yp?$hl=Ho>oJ=jFDcl|Eia3JquM?N(5tppg~9~p4e)pD16))eGEY}<^uJ>dI?eA zWX!$qek54oh^;jhxOs3eTu8h;(x-A65HHjxL1JpZg&HY}dYMiFPZ#A|2mKPVC2;K^ zHWvhsn3x0x-%7rUWzG4WpzfuRVJd-qerFam`Ynmmtd6jJd^Cpjg5pUzDR^4!((u6bzm?3@Q20eTi;+^6gSv!lzWR24)#_4 zcY~wut({j#JENV~820=9-e7;kP3)u54g^WhG0*$}p=KsyI13rU!@FF2cnUc(9gok> z@1EQ{J3l?0sw#!(6?{@oC79akqB(t>I`L24H2<%CnD&h*2mqdu)RZLXchJ8x4#(m^ zxk;uHcB@(NZ&Abdw*a!rX zFOj-@qJE38-glWG5s4Ma|0Q0;oi#W_lZ^W;C&BUUwHE;wS=Wm^x?cN!JvKMQi0MQI zg`>#)XF2=eoVPr~=6Mk>0x2SDy!HF^eg4vk&mO@4ojt`g9EXt^8(wJK^y|yk!czW4vSDvRc~JOP~avMz5Dx^a$POn<*aXScu>CD86v{fDQ}e6?P|^;63|ZeM4XF6IFN%V zp(N2)hkDw}5`ltf6zdhofI+@=(Q%OPp;*_H!C$ecsOjKF2PMF6Di>r2q?tvd-eb9R zaa2qT(0U3H0||jR5@`|p8h7EWIQQkWAcKW!2zB$5Eqy=} zJBTr9T&ja=c)AZog#C!qet8a(1{6*O6c@jc$%e2Y9+9US0;7j$ROqOXzFGY~-H;|> zz{WTP0T?w&cx+jI2m_0vLQ58Lum&HR`f^%O&7fBGNep+ zBM2g`eNh<1hg=R+VMz;Yfkjt68vY55De&=hR=t!C1fp$Ld~C7Yr(DJN_=o5hx~>o< zd`Ymm)h^tRK5&INTP=sn`C^C)Jz?&Np@kqfwhQ=4Kp`}?LW+irv<(Lg>^UqK*j=Do zGfuV%uef!k`j|2%Xx6jOEPsuxR|OTrRT0HGToT81-p?)P8|5BngMXF)QnGz=XA%RT zo^=PqRiFC~uRwMj^m;pc-EL>t>2wF^j(FcltSk&7IlGGqX0^`GPR_BaHJx$c2@Mk141aQ1$|JPdeG!u@k~2GFwU5`vv#iFB5c&8M zKb(;?R=`NJt?Fvt%8p4V zaMP~GS}c|#w{j5dmq@G#XeJ~~$G#P1TfSa+=zW*V5uUC;xgOza8KyUlb=#f{lh{c4 z{Wvby<-#spY-e8UY`gIDx3G}dus-1+(P2C6v(9)o2!B6^_d918-6Cw>ckq_?IIt(# zAI;hJ+;_i+?LNb2L@tjbq5{VD&whwIW%!xaa?I~jn3+yE%8CH`3URF-~|^>~pCfrm+Uy;`BO-6FbB1UH1DU<#d|)vNP|LUADNuumkIqVM3`O zq(rBEFUWwfOY0^NiXLdLG7jF&p{UY~u(qTiE6A%twmH?kkdLCMmSv9kyF%b9;zJlE#C?l$6 zW*lS5{Kawvx;9vj>D;91f=YH)XAsBGaVt#t{3Bf{jZA7 zw_hk0zx}78`G5b9V!??|l=+6k@)>t|lYYj#x#?bc^@Se|_v=Z$GuR#WJ6HCuUB9sp z$Khaa?+UlUYT}mx=YfT2KXcJP&0;p5Psit6!#6B;;&PRF`L+{IuKJ#a9u@}qRt`B;jPNz<|(2BL$GV?e{ z9=iGy{|T2f`(a**B3#Nkz^5A5!CJ8@-v*^ z5<$C<&j`B+yvW{XIm_~t**w-K(&tWa(dD*?%zIkx25fx~YYWjEXFk(du8W-Qlo&Cu zE42u=zkP~?o6ht(t!d1|$xgKKHqVdnck!8!@Qd3|`z{e7ylh|W1UxM(w_id>gw{ID zV_!{YnoJ~Fh6D5S`bLsZIm?q+k{Gmn+xR?Q=eJAyUn<*r{H7T`w$V8AnAY}4tYp}H zk@k@NkLLUij=4K{zpt=1Y@h9e-^HiD^%8zM`|39^Zu^`1O^g!`5{tq{c;r3A_xjeg z5O^ve@D}6oe@14AMPN4S6=Te^9&$GKHltL@2XWP1_yqf~GiuKjr_0sOEG$8y(@-`Y z%y)sjt5^cU&1@<;ffGN#)GU~)h`)O}i&k^)t>jJ`I1+I_&O8vO_e5UgR>I}ML^F?w zxs%UlA~Lvg9^~4LBI12Y1Dyv^)^Yg?e+`R_D`;e?b8;t}S&CH!qD)>}YsydqhZ7>b zCDPth_erg|mE;k`%vuj6^jFA$v>0un6fkmos|75ZBdHZi^W<775*36BSdmRt)QNyx z6^SXV?^2H~kraU&1II_KLZV>7`Y#T&M=%vtJArv+~<#6yT^l*X@GOc_li4r*x zSn?%p10k14uDcL5Sl(4U&Nm_EWlh3&OgV^!1>7ZYP->6Tg;xkt0ZJisMk{Tv1eI_( zCro)Za@~i-mkP3ja{MRvMjnnF7zxf0?;5aF4j|kBT~jki25Hbhkd&)iMX$aAl%Phj zbg)_t%Kfri4&c7jy}i{7yWcIiPrE#aAUPckn$cdT7$4p*rU%a#$3K3fIJ)yT)xB0M z-*}@~ype{1&3h)DmqQHB zPq<2C0rz7vIls@nB51MC#5(dl?FK*{StVH4nS(&*YsIYN_;sQZh-uYzytlcKS z@AQgcpWfBfO>=dwbTS$ZLw;b~J;*12M!VN4D+Th6QOr6U*`0h$G~%PA1uAPy*KpiD zn-;C~lUt!uA`;3o59v>5{Hm&9(hiII>Mrta^P)GMqu5@qmD^M1r~9v;R3!a7c#&FV z@^9fY(oS&W1j+~af5N|q7a@EXFR}4i&ay<_MerpwzKj10{6*(T5`nZWI?MIbm~VS9 z@beSSg;pfew)+9~_T|s<=CfS;=us`qUq7c4akcNJ z7umNQ>({#8Qy?1N9!>2ugU#`eTU!SivQb^kNowh7zs;Qfv-;%$$}wVp{5 zzrV-Rep;{JP`cbUS+{LAuWiWfBly%h`|u84;@18-{v=B5yTpjFcm?lw@Ed;%Zyx*W z_ptwlb@o?h-}Cp&-@AU(yU%tH03@hq*q*w5PxvK*IdE48evuz3WY7QS`OyZl6%xa z6yO#txvT=MiZc{cBotKKPIq~TxO0P9p%S*3frxTkaR|1Wta5pCI_7s{(w%~MoG`yY z_&H4H4>Xe#(5+LRifBD?%sCKUOcy()6;=9}WsO_qBWx-rXX#!ofdFTr1H$auMVq{h zeTJ!F70s92>dX!I&}tG4OmdKV5S<`q#1?8Vm4t;s_jr%MQ|OGyNx&R2zR|dpgb8MI zErSLmPKAUINQ#8q3bCjuGU$f6SsH{;3jWq>25K%3~xwO_LL1YId{lry- znS3U1mzVro(!U1Qk)(q}Pgq4mxj%}e)U-@VlnhA=2MUx+(YcC^y4wsRet=bjUA!Es z5?z)_`EhX!Ar)d`imVf z+`0y73iUI)i9-AUVs^Azb*~luy{k}aeRYM(eH4$c!HaUZEI6F~POsPX;HZIAJei(% zZqJv9buEhf$bkLx>16iv^WEa4K`w9ar|k1@#{QJo<7n5j^v8|r8{>BFoK=hGukFA& z+wG#{24;T+9g-W}{_rZIP5C-@rW)rc5cu=??3`=Y7nt-tKRZ4=xqJ8aDfb}GJ0SV0 znWzXZ0phSX=%ir{h;SlKoOp?BF~r;Wn?izJMBi8)u^SOAM(2egCo!^3uxy7@{!jkR z@4YjKf>~KOmEh^rJ-`inVJJST_@}MgzG4bhz zs|+mASd+1Q!o;5SmymIn%Go=cERrn}?^B{c5L|SwVC*j;qIH|62%a1R5#-NuBEWtk zLF|5{&V$d3&NQ|~qD3(rk?q_T^FGR%&-QMMU|C20a*xVqnR*vay-2P^$2)k54f|^! z9R`jm(}>vHcH3yb4IAg0V|{wttj}&Ul!&lj)?vA^sq%bMXMG>zEzA1s|A%-HaGk`L z&UeP;c;g=B>}!UDVat3Pov=K@ORVaI`|t6_ON`yX-@^O7eCIfL(!Mw*gsg(BH3yXIcXB}X zIW@#ev5TTa0tj$y$jrgGCxivw{kjUVCe9Kmi;a^~jgyNZ;lthxf*g5W#KKfBizQS@ zRqbIy2b~M&HXIyuAh`GB9AX3_gx~W9V!}I5}PiSXe?VA;-=;75GEwDc;W@s3t*>@P-@dxt@}E%A`-8#A>xi_JMHeAhBfsI-HYw z2g%HM(kcq&G9+eX|Fl}BV3hDxFV*{4v<_yvKKdek%1FzWdLLwrUlIy(RU{r<*euPv zQ7}M2d=C*a#kNZHZ+g|c!0;4~O%NBbHV)WMfiFX1jy_Lkg-#E>&?;Gn&B&Uq#=Bj5 zD~X(<#$>?`>h)KUM(5VLB{Dbjmsa&+1j^P&5NyBb;8!)h2lz#g9nB{==lG*suHzx^EyHQ;`AN8oR8P+1 zYwOs4=65+S(=y?qpHbmD57KWCBQ8n{z=5JWypvw9SHAVuTk%~sML*pEzl~6FX^Wq-fJ;8!XMTe{^rhj~bGVR30$b~+_ix|kz8k-sUHGgT0n0bE!C1`mo4 zd%+{`8Ue8oe&VH~T12lWW{5G7+o_!i76rV_rBh@&5b8k8(hV;fp2FG2d@|f=uYl5G(xa*n}*}x;JKf&( z!Ems5<>31Dx;Grb?O-e-1Ic{FLbpob+{QjT{rK)jx2M?ng-0?UPtOq$RB;J#F^CfV zF6a`W;jQZ+`~l0PEn+wl#v{FO@_F}Za;Si<>Q1^I-9D4jXi!T7z7b54ZgjuaJG zHotRxdb%<)uh$Nxm{gQpUgDkcMp#Fu4x`19mZ{`nMUw=9wS-{M6uMNEAfp8-zrQ_k`w5Z=ejwJ>Zy za$j5zGO>{RYoG0>_5E)B6!Em~j0{_D?mNL~yZLPEd-%V_3l94N=*kxL6)dSbD%8qXCO8x)-vTBWKj#!7G3@>gdP| z-4PL62N_Q9UWD|Dz*_MhghuzlzRyZ=y6#v`scR#tb73f+j4woL9an7MtkhwlOCe+oNc0Lw$eRiWx`2NHMt|vIFL?}!WWULTEg>rCCP+9)KJY^jm zbgzVTgsOT`K1#L#wqdWF>O#cW^g{`C2G4FTiAWin6mM-%I*a{3k?p;dmCch zZuUx72rM)8flh0a31uxT;;h!XE+KZah+Lf}7!18|Vx6>;%`1O!M<86?O~dN?F6GG? z-r5TeFv>-(e3->iAhTY}kp+%nh?^EPIz2gP*g!ebqRLzki6;4Acc1ZgvdJ0nS zQ*uDlU_&aHc~_nTmSjLFjGBld9WXR8LPM$x|MOo0Sc*&qb&=SqtR{2P441*Q*p=01~d-wDz_&v**_9=3X;4IU@F@&r^qefL?Fth(l>qPvdpY zt<|4XfU_@=Wmd6gJ#|kMgPmj;;&F6ck8zG}dwyA}K6;Wk16k(Gi&~_n`G&QrCg_B9 zaTHe+?{Q?B=4uZVyEPYYg2&Dac3rnM#2t!?i~2vrF<`!?0is5_!h;@J%!^c}{cT>w zcUe>KMc&M5k>8!CHS?86>3osCafha?lXprSlW4p>2Lf=;oj-kFk<3clU}FsCAOdB-U9+;b*zB|3W7)2I;+`kd1turf+i^0pKZF zeV`uJY?V-}y3QVUXMu*nV=If%7`JCmCaUaY-LJr!_QDpeMLTTtmiX(~f_Th~1_Wt! zYM0+Mx&ymVrM=w^%dR`3)EC=92zhm>ss4vQjY+T-t{G<%N*0m?8_bM1kV$1T`MKf_ zCbY;@#@nYIQKyM06#VLp*#q{_$5En2gv%bP)_xt>~v+ z3F0>Q6eA<2Almku%{9Y?R?4Ak{fMNK`JpFd`%G(++%+D@o($IVXr%Fun^%JdJ*FJi zMbHLOJFich8qdkE>7(&=J)3Q=w%K~O=U{3&n>H!9XE_q{c}d8s3DxP7ld;k$6fuI? zW?Cnf%IFACVlvJ#-+bDCcXM!X@P+S0BH4urbPV&yF9;(!OAFXW3y;2H3n%J$%<31= zeU4Y~7{~x}U@BH(w}vcvkdx>(V(zeJ%0UCRXX@c|u(GU~`6nAAY-!dcL;sXXC?B}- zPYg=yK07_EerEXsyxrvtLmk>elNzJpFi-~=Nrw6Kk85%dl?2F^lpn_A$(_WKt=l(0 zg!U)wK7;swf?BRNT%x(jm=bu9FgZ}|TDbFct5uD&EixIJ5frgZo3^fpMjnUXhs$(_ zHh(K=*3Rc|qw0N!!koUT#@?nHt<&HFJGZ=R32WwS6vC17SKC?KI5&+S$<@u?t(E+; zDQjN#ldL(wng_m#gQn-H29GRY^T!-s6G7kn$H*2cxyO!mx^;jksR(ILox|5RUe^G> zb4Y*Rj-4}J%doOya`*O`|C137Go(_>lzVNeje^!X=iznje(mAZHpKk@b;J*N>b_)@}c3hL(}^fU9E&MBJQVcOAC@$0h_} zvHIm$+>8}u^nOv>*prAghAK>ftnU7)g4fS?YAtM0md`-x6PG$fm&H<~LV@Mr9$?_I zeaaW%Moh~BrN;wf!|E>LpzOHjv1Y0E2x~`*HA2ol^#LTI%3=m?t=x!5OjSKq8}k-Wpx{|2-44{*-P)~k`EAD% zzEF|HlR$hn!*&+Hy3JJC-qNbK8qHU>)4$X1-2HVcM?`TtTDC#}|0+@v4}yh`*aEL; zU3!0FKqWVnP}_lQYeY2?T!zi>SN3F-OV1KW^-DN}zUm-%p}3nHT$wr6DGW)hfmMQ> zfkH@Ef)L7DrXw_(IGG$XS6^#RFi87Pw>(aP?rl|s`}KD4s-g^G28I-Zvlsf#m&w&bWnt2jjFva?w)txcD zPBf3}lX|D)vPd&$`@L(}T0cUVnv8ib25U|(0^RZF*R}?14T%0MY^Uw~(d<^F6ts#t zSPqqV2vqy7F6~N*Yj*c>B*P5XLFyq;z>y-sppQUEwGUv-PGQ^%m+YUW)CMskpAg+f zAE!Rto!O0UN>JKQ@O$ag$hs&V;VMC;A%Y!!0C|zUWf(D(L_F|}e))@xT*Td0>kb?P z9JtGzd&FwUAY=$D%kMhkv|%s<5%HJ7>sV#ve+Q(}m+iz2Q8VP_f&oo(aLW825Y2uy z)E(?RdF+81Rr1t#C{ADmE!_m#8ZQhx)~D$2(mP>KFgXrHw2Gu`h|Y7efqXtq zbeU}KAF0gH=(;DgNTItG;SwEvPCppxgo0#=n)T9Cgt-xs;uv}UYG=B|Yru*&z_O)` zb3>gVJQn8#v;l1{3>lZ_Pdrw1Mk_rZ7IeNLDlXkFnWwJ`vsf;aij>7x4gPvW`)?rF zo>|axN36)YTaQD{R=e|a*RH`4&Sn%kKj_U2)fF7q$mi?w;%RzXPX4HVG?G`MXucke zEh$$r6AwODe~w27T2-eAcclBP#uRJbST?i=e?K;#hY(cKlJ#2ZyZmcOCr(x1Z_OOu)YO)bKqnv@ZyC#Zk*e;*yhD#UMb>UlCgJQZ(0Ilc zm5*GXnrOLf>fOPMFAd9PZIi7}t>wT6 zC9>^5^&o!J&g3*8i%z=^HC?s>z0g-zL+ zGIe_!REG3v)S;($8{xj!b-74UQNzH)+*u1_qNn@-%x};Ga+>4YdbbyaXEIKK#z1co z5Hna8T%xqya%v2msh{~PBFOq@qkP}x9(F;Iim5rkPPN*l1Z~Ahxk$jB@BE1JPV=sb za00cMTYEf?xHOz7cA9wr8crt%??45z_V*}WmZ)T{a9N4LT`~N)o+^E~Ha4;8z`C}F z--eQw?5~FMY++_IV0c0dDnU|oQVomB0tPew=3wQ6Yfa4q%QBP)aT6WCtcO%4{Yl`z zAO@E#CA<(c#ZTQp;#S;&Op^-C1>Qj~Ab+H6pqRD+w{+qt|0p72KwRu^8ka^;f_qTB z5iUPQhyv%jl?HRf;_z&t7DaM&3;_#uq@Xm$mqjvqhvFP=~e0w0R9!&tdWh=TK@ zc4&y4_`!t8;q&SRO3IeTccjRzht!w#5YMh|>{)JAKhifp-@Yz9SS(@U!S{-9t9Lc~ z)uu_1lHVqlNtRxWk+*(eJdP2fr! zrQC!Zejx-Em1aw!BomX=voL@v19$F4oDdS!up9O5Xrln1Z9`qKiQ+6WX$~FdFK3Rqz)HYI@ zI=%rc3UK~fj*js$pxV}1{+Z9$;I5yR^vn{YjW&e5ci7}-ZvE=fa^=4wMvFz>%xG6c zt`slCCUC4hcVINcj-fOBPk+fH+zH*Z7NpCp%g(VQK&*`UC`#z=v3(EK4J`{dm{6o$ zi7q4etLRsM2SsK8DygVrzw_*=7$U4FBfM6Rqu3)Gzho7L%p(6Pm zEzeD_)EnC+K%N%l6+{%6LFp<(5@|7e+Z%u;qs}IjT0kADudv*Z%Bwp=5|l*WCx<~N zEzJv57h-l5q1&hqF~&edW+Da;sk1viemsV1-`J^%qANtfg`bx28odOcWSU1Fm6`~t zP-il`S^)M4Y;1lAA4Rgv>4KwRoC6oZeB`x8s62fJbn8a6?i3nck~mnrr12Z5U=R2& z>EOOXD@!y_bQ5GGx7d-OSr|VzD*JqFtN?403W>R>1mfNCC_LjuF8pEK*W@b|L-Fy}WRLEGPQnfZ7r0oFBb&v5!9DYI=R!Cic0!-d9ru))#CZm93 zhx{HcS1U#_9ltfiAq#!T3elg;QY&?|{u*ZB9Xz?)h{0V(|tDWNYIIk%j6757TAp*N|B-DP&s+RZ^w7R_n z4Pn3AK&{44GC++3yHYSVC{pBwDjP z5`+O5n1NBE_A|Bs|M&9OFXaI*XRZkSy@s>~K8{(mD<5%-f@w5YsgsAs|33DY=+HzH zV@5g7arMc;l71BMFjemo5zAx1_76NZlLCbbd?}Pe?JvjgfDgZCB_ywF7@+WhQUv%w zXR~(%ku%Y!I-Pna7O%gJEYW|{6lQrTo=Tzd4LTNAIO{oU0Cu86Y<4D8A8zD9v2j~#eoV*VatF4AgfJX%b`+qVqieWpkF(y3-U%jvee-`3K6^Bt$^Roile~CiF|u; zaq+>oF7X$FY=iH?m}C1!Tfm(PxA^l+Iiw0og;}k|MA6=SLc!f$eZk#{ z72nY+ir(-MrZ6hDkkPYR6Oh6P5t-`Ik5H&nIWB&WLJaX8=HW>k$! zHv6R2Og!l{9Ve}ex~j??Eag|cwkdXyk%3Oxs(^46m#@+y-+KEDl0qvfpL+?9oOZ5kXClR2UftI_2Qd$)kG<`uMFoF6j=qL@1nbTP}|# z8{CFAzbrOfKmrQv3+ZhG0Mrzfb4j{K(aF9(71YWKS4=A@Q1N%+t?DDOf~f7j*93q$ zEu!lh-}?QCzY5ITVKs(eko;H0X6~zbILThR4+e}D-JrW9z@)*P||nKQyg#X zEKZ&@h>wLiiw=eEn~W}6rh=7W6BpYZu6aY{a|bXdI3|aF+|lfyojVffG8(jFSrwH8 z$BCiRTc5e)Z6&S6e-$gvVBwSI4GGYM9lX*g)DxT-2)cnm(B9Y)KLsIq zaCHB7Af2;W*Og!wnT|HDuw)*5_FU=Rt~eqN8n8nZG5rPl0Wxum4hON_=_jPiZL?lW4z{JSJ8n?M~ zQX*Immx!fi6m%!U0!2|M3TA)rC)nyvnL#faBpK0!DoCigrfd*mqpv5|tXPoPPmLg% zuZUPumk37EcFosCw64Q`zd#*<-2zhAGuxOwq4TM}duHyRD9(qO7k6-iJ&Ju}*lmob zAzZMLjxo2TRY1dzQ}_}A!_|io(tqEm>;Ibx-L#?Uw|saPXIrcvyS-Rb-ZHdo`MyBe z;i6gIkeNQt`VF1HF3er#>UEIAJLOvIm&CTfcmISdR(h5%T!v8=5E}ssPDOIwsHI|+ z70fGjDFVQSO-KB9iJRPzkyyavNnpZ3Cc|bR!el{$2>feQ zsH@N>eNm9U-C7Oj6%J*hrMX(6k?9P1YR!8_fyG9nz)I0Vy;%lDZy1ETFq&MsegWnS zV6Bl=Y@BdU;p@JI8*w{#Ersj$buQuw{*tz~ev8vyz7KS9lGF!?zCEj@iV=OIZB((C z`-)bmPB=lY4FVL>1|EGmaR|AZ zUTbnPi?&NT7CMy$f$vBTEh0xa$(<-{P|xCN0)<>Rh)W$GK34cV>ePab9;(2bi)@Aj z`<=iUO(YtH%Dcl1bN=p|27{AQD_?vkz6W9H0z#JUKSb0@2+tVkb$YHmVmqzRsnqOq z^7~rPg(P|~w-ea+)9lZ3I^x~|_O>B`9fY0V&xQuqPWzz)q71=kx9e|GD?P|8Ymq3~ zRniV(LC{f1xC`WJ?lkqd(%sqnga+f|U;~mPcg|0U8R4`6(t!!vvUb<2S01XB)AN2Q zi#m6KL`RF=`L~6fc6a!WoNOSO}$K^@s}?#D4H5}$#A}} zH6-JrnXNaxz0hDJ5rHDB0LyWN`z%x2VvA0oILovyU|8}-5i%*!$o>R;IqQoZ2P&2GOi(lahi-J=394FB+yMVMHJ#b@87RyE(&#bedtrayFZR6CN;zlcFA2ma6NN8aY?Df!{!dd`Tu zn+9DEHzd7C^CbVhp2P@hM+#^Ja(a>A0g`dQK+j%c5`981tSKqvpGMMu2mE}F72=++ zgMHBx8~7wVu}bR@WgiF@#x^~-oBELq$wcAMkmmK+s?ANNsE3$-`m>hvER#g~6hpth zui(fI8t5URc4@55gpg82!?eNQ%q6Z?RozuekN-u!%D8{@r^Qi};We^c`i;YY<5Eua zz76#d<8xo|8jPi@{;{Co8~@1m@U_Y8_0TmQlM`PJv?x6kYXe(Y?4X}-HqtQ#N- zTJ*n1>-BiZ1teznbpeKQygd)+K=F~Z z8za~_+R~K#XO6>yu8rzsV7?kv&CkuWsw(`1P$Sx8;u*qlTx0|>?@pRYqBev~{iO9m zEmFySfxlfs^jzSSmtxWGdg=?v_=HbV`ew15vA7js3&++#tP1^|iZR}f9hfGg$7d*2 zv@EDM!6?WI5&nLOfers3f}G%_0p(*Xv0g)H&HI?I6jHVt-27_qv*Fwj$HP=nJ}2jKmTCuAc4m~IY+1ua=m@n7 zf=nzKlR*2S#n&OamHvi$pfTN{Ls&GlNbpUNCF|xxzBAXcZ!q-A*T|bt18Y98^Dw1) zop!UUuGn~g7h3AZTZX3nNzWq1{!)4)>|*HZa?>bW1#n&Ys3@>nSw6LSY35tJmSO;E zGo`bmkQDjD*2vaGLL!t(BGh&IAW|?S-*(8W2f`2m2KS#mGF6yd@+Hun!bigE{o0Lq zXZt1*dgVQ_z86QLuFN>O@kB2KLll02at;TSvW1L4aMMo$R7bX#r#%rm<^JvGXBl2R zC4D(W`N%7}s3F+A0jxUZ(Ld%)yte4BXXJ{&q7M{qoHU$*xfJI*%aLQhtTg0NLh~*!YmOMg;2x&U36*ruI(gZkXbwTT+ddg zBv2Dn%HTE}R0^AGpitN;J@-)XeGeKQX0sV4AEx!~Ke!qObCwDq#4u=;Se}yC4M<3A z3hv!EHt@Scaf+52o{y4?Ej896KzyCk?w#Ba810 zeg^R1**$x79+!eCyPyP++|?j)P?15I3F%iTEW=cwDe|LBfu~@4WAxJ|t@47?WMz^5 zdAx)goIRl#YC&Eion%_v?A&DeYatuSke#ylOBy`vPYE{sV_B!r2GlGOVhC@)TqrXv z`R1?77E8=Kcd{I7oOCkBuBxVjG%ea_)BBb9rv6W#Z?@h$%<2GnNOQc2A-MRV=Kk2| zeUCMC6fqZ}dV_$WCBq$-hp1KmThu zhfW;*kQpsq{h^xG)l+5 z-Ws5wgRPEQ$~XpWilrPIkBWF)QJ7nSv_b_il^mkDGK>YsA_mG)*U0MgqRa&Ff?%0S zz5NC|5rlS-C9M&W0apSq#XsgfaP+GPMup-=O?#0O!dv-$9J3JQCyBnqL)z*3GoPJk z9qAk&S*|VT)YoPY*{JpXQiB9AM)dVO9NoV=6s7h$O$47keB2wgqi^bJ+Te;{5;(v} zFL7>>mD79huddgNpD-uO;K_9if!siaZlDnFU^RBT#Nb>=P&6}N!{9JQ zk7g**vbsgth&-I608;aFv_e^c91;m>h)x9FRJ2=I)N&HW-^0L=Tf&ZnD^(hp8sWCrHj)}r$s*y8}nR3UBIPveW(@EXso~og2@gu>Y!F@11-co-0{YD!&HYlFN|2xvlKqIFI1q-u>4h!S#XqV77h#cSPhs zfIK0{4>>!^Egy}jCe>hypKn?)cZh&GL-*If;K>lzT%Mml@QVg)tbKEdnS~5vq*|mp z3hB2v^Qf5~2~*Nz83%aKf1)NmN%n4_6)M?a4H71~^FndfGPw#8bc_>UYQZQGUj&Cw zUip6kea9k&Sm{sNkZZ9Q#((B884fnLO>#a$Pg0#$qTX?F} z*qim#0_!vh>9OkR$~{6VV5P3dUpsW2%=89_BDmI3e@+0T<0R65;LE_UDUyCt>EMGR zGnVh!%yz(09Yr=pqTulIFda%9C;hN^u(!DCBHy26*c4_W_cN(KeWWZW@S7LnY{{H| zt!4exnc;=a$U_b&jN;0_wNNf${b!i#mTuU)5ssFqj2;~~&mO{>rU1c)*iGVKhJJy7 zFPxp8i3)?>hMGNm;DQJ=?L!E?f)sMFyEg3%wB*bQ{Pe2by1IIl;B8l5xHL@96%Om< z7U1SS%#MSpcjeEL8QNnJZz|GO0gKwi>r-}=F+oS&=ff|6nynU%M1Iuka@_-(aD`s2+ti5@_sA?(!02W zt}_QYef7|M-5w!fgdd>a_X4kN@>GFLe(>>a_C1CsVpcEr^wL|>7pdF4v{rg{i5%_U zP||R1!J08OHT`|>8ih|cg0%(z)>(SHVHANVKOylSY6N{G?J7}!jm^gW`Jg4_rtkCZ zm4p`kgqxmGA)OKvT%xdA1<1RPL-IgoCEPD|x5h(}Db7|hibh)o8z@^_LA>puOz{A) zP8?CiCK-ngT=fR$>uQ^{4et;+PLuoI)fAtXltaKM9fiCIw=t|$6ldI8|%om5+0KVqV+c!Rto(y*nyc zQr&0nKZCqeJ#5U$I!4czHG6Ga|>W7(bF262iEa$4Wb@nH zs2M2B0t)fnp2g#={jY+pcOEF%aRCIM? z4au3G)d)Yi4IytAX~=M&#K2Cb4wd&k9u|=Tx~p`bixondi*ydwat7rHC?}3aU1VY- zXlGRx;e}L}{@KgqW8`YjJ$7T{hdtkA^LixEy_fH@l4-s8w!){;8hk`d3!PBT_rR#s4{R*ca<4FVt#|mvkxQ6!W#`TZbF25YM z-wG{NQklf8Ch2zmndnvU)I&&p1qhvnMPQfWBf?-dYEQGW+(zk*v#v7mF_qKV%IRG$ zQf-t&%ekc2AFeKJl&kT6Y$nMS|2%z?IF*>hW#SVRKimA^4Sl-Y{CkO%OSp6B>i3Rm-aS6VHi^ z{o1w^B?R?^Jl}PJ}B)l0W%Bf4xs%clW`0VNrX3-v1W` zKuMUmQri4nI&+-GssAf~id~`BUfy4)Le%5FHssc7gNO*!-P#f1-m{(3;U7Bw&ilv9 zizuV7mVEI-^ZZNH+7*7To5iOU8NLr5Hw@{ekLy$WGSNpW@s0ya&rqO;0;WzF%RJ6u ze$ZcRRW8UxQQ?mLaN*72PAVTSfGSh3s)B`7+t97vO)YRlymnIWIRn=GQ8ZaplhjD(v?M3vbYG>LAv$ zoGOKFRjZ<}vSE>(byRKRuv5;)51zG-u52qDd$c_`TW`GF8|%*}v93it%5^6<&VQZ1 zFJ{Rhu}}fbgB$k~*dj)sy*T&<(xPj}As-L|!2>kK4ebPfh{?pU12_zZs~80kSRrMnv>tw2_)<@k*pqJhS~xfgr&J0I2?meSE(ji8$_IbvS#~9w7%ZluFpe zY~V6r>j;Qc?X~oO;q7mF+Ch)D{koPTh_`X|_rD8KMyo{8l`?G)F1`ayjf_8E?NI=1 zDmr)81#%c|jt_!E9SOoQPayxt3*cq)Z07pSH!COj^gmP%x?sOQx=bXb`KMmb`BSYV z0jGTSo7^e4>!izV)i9twU?@`EVe&`@+Vh6hJ1>P(I+LtAYHL?-g)?t^MbprcTQWu) z{SD_F(DUr>a9jeEfn*cURleeEX}i34+84QnJM#g{qGX%rR_W?Vw|I^VKQne4k7s^g5aG7f(~5F1BJ zXY05IbDl(fM0w(491dl}yEW)QSojJV!uU&Qck+CmW@+L#HUEP`p{7de>xDg%VKbO; zG^1y6Dr1rC#spsW9e3tW_^&=b!6Z1+LP9%@T%v3NP_pV=bIO;9nRMx0CX^TCncd-Z z*E*lWOiDdRo_+Ge9m~x|&^M}VR+J@1G*aABx!P+^V?&?l@iq$kYM3@kgyx5R`eX=_ zrkAWePA9H$;7g0YSkvLqO6YdxzRf}L2TR97wQSKaqFIcgM)WLWhMWO4d;P)A&Qs}! zEm3<=F>&6E1H$j$a+CyYg1dKsyG4J&*@XF+W$T_yExas`{gM( zpjGHoXep}0&kP+@Cwl+Flz|!4(IolXw)WAB-rnCyw@pVX=s8PgYaj8gk`tO{hsxeZ zYtU$)G(zw6)+5cQ(KkmN`Z=npJK;^e1rFLh+UFQe)Mt->Jt*98Qe!#%#mJ8LqI)mt z6Y0FkzW(C6{!jK15`0g@pq@v-{!|uG%2sPw(};~T2H^4+obL=Ma(MTL{7RROE+xg=@ zXjFs+TnmnA*EgGWrIxASNXOl+;c_0 zBn>)H4E1xX|I6+pV;GIM`^3{tO!MQ=-PB@9)Nl6tA5PPPf5+?X$5l>Z|ACL|!ijc2 z<3G8T9o+#RUoPzP+(HFs1$^^MLLVk5Vo8R0zq!5Ptq_w0Z`HW%TmI0lzLSZQAcbrB)+&Oog zPfiSUG(wH;47C(>f|H$9%j(tvhaCAIVe(w|P`g&uMXj`^#qpWGy4dY+U)Pzm-hrmZ3v*l+FzXwV%iTg*jk=3kDU1u9K=OnS_PZ zes?}WDChecYNtU`BOB}H@k#Yew5QkSt4Ku>ASy~+KIRfp@!Fn6_hZM>;Zm^bPud$r z@5|P%Z{#!-6L`+Zi;l1Vm;CeA&W7+hK7sNj;J)9mjk5| z8@PNaT5u>T0W=^}QIB)j2#+)&jU5@@3&dpyuaug{N@aUELZ3Yi%UqJkIzCt+zT2EZ zl7#?Ywj{if4oUd>Ak|+G4NA*@e}7YgNF&5Zw_J(&wnNlG6Jd5HF6=%US#x-v?(w6c zKaISWp7E!y_>a;e#x!(piNqb$D~@77z_W{fp5IAxnUy%y_E>N=k}8CPF8_5D?v2T( z9s}7-=#?=;ewU&8{UE%>ioR5bze7Sz_(%b?`A_u(0dfS~ti8leQTHYM63co^%9J=t z(-!q@c7!1IijsSN^VVs8XY@d_c%KUC2L8-}y+}TJo^9OX?#=sH6W&e>?lGnWEW$0c zJNrI3Gz+Za1H8IeLvdaEPX0U?T9RCINR1dS++M0>Y~Cef>Q;lr9D~ z;EQwpmD<&$fnP|7zl;45w^tkW&3?g9FaB2LTjz2v3J9G_c1|Iu)P|a=lFFgtksRDz zfe9Z7Y13{PqPKc)(~svn-Vt`Vcf@wC@`q?X-QKVY2$hLp1-r<`id?UEI(w{e;IGk5 zrgIv=L{LE_8K(}TiQhE?Fa^!N=l(y)pBc&ZWI)ef3c%*`2@%)QYKPa>kdqT?v~P8b%ArTjDOFfCYr84@StIaqqa zfPP*{Hqs;8$u9?Ivr0;sLvQb5)v(3J_GFC(6t}6x}6s2iJ$^Q|E^wMygYmYK&s+1 z>}ojA@K6I0_U^hv_5CT4+4pHo$xG|2ZRz5eVRj*m2tmoX$nykme`Lz;gDL72q-^$g zCAO^85J9H`sNzt^PR^Pw^C>z>sid|Y^hqxki7Hh_fBD~$IpF;s2ob4Ex|9rPGHcIw zhRZ1gPRpK2nDN*FTIi_f#d0!EPNaxCeuj$v)d`EEqWA`Nu1yfbjJc@ho5^SXf0IM> zAD4nnIb^%%`u@c3|Pv+(5|J)J|*-dLw>4 zItae8J-e$4gHs#>?9$0VQW^2ZksYphv?smn^vZaAo5b@4;Ug+{V!@jVsR8oT;p81e z`~-nL7k(O~A|l)s3@PdBJ=Q%_s=0mIAEU;=qiC zi5lb(AYan_%?f&;=1(pPg!Q<)6Q{B+E*p(RW`%*v-p3cPJd)u6JD?#>=Z%Bldlq%# zc61?qoJqpWy`6TSCMJHovrjfn_)>QUsX%K>1?+QnzbHO*W=tZ2s)Ox&i)%M`EfIZwoo`4d7)!(x zJ$Ag5^woo~DQx_}6)f1w1PT4w&k>&=Mrkkhlac%s`SXT|L1{qm}@JFMCA#NL~WDD!*ww~-=xFQctYp#%)nJI7w3AMOE1&zQ{ch~eI# zSBUJJJK5<~Jo?r6NTtv`qyYxt$n8Zsx-Y$M;Av z8XMa3w3c_H<+UYeit(YhM0$d12-|m|lZSxgT6tU~J=1|MVu_7whQ(r86osOk|2yyh z9(XlU*ijI~S8h%5Ds`t`9NZ5yoRg+{&let)|+P5?@1_A05LGeB=W0nC^p!F z$-KC@kkyLgYa7JvOg{5iu!VEQgr$nHI0{{XdYCE*Pc)~S`i5)*t!@T7N7XihN9p;{ z*pv`d+&n;RxyO?i&P}w7KQiOf*4xX^J6{0R zR9qigjC8@z)#`iH`^%Gj>B;35Vqmmb&+BopILGH!tZRyY%K!1iAF7Fquol7fj&_TS z(FFn5<~v!2ar2TbCXO9! zUIMv3be1zYvQD0_ZC|L~Wp=FD&Bb!{kLJYEd3xkqJ<(VLr)5|nZ3f+qWt+%F_Kec8 z9d2N^<1edMC)?fdzZp95z+KTJ;cRn0h}5I8-Inu^qS&aaN3o|4)D|7|dT8iEZ=&!g zfcEL}+^biW`cf8eYRz={%#@p{$Ci_;mvu7MPv0(f_x+O`U3M~j%PG(0gKHEfx!`K_ zZp{5iH-hVVA=|043u0p~Qd_7k7}`6Z>*+vzG}e+k>M)!C?;3KTKA%c&OYxBBY%LT0 z;%d#FKbhx4W^ZfyIfW=GS9}$D9={(~-2c&!ZzaIT*TOyZNArTc+scYj@CvaZ`@qKT zciTqXYaQSyh~A~M4*B=84#PV`MkX1_pWwPMqzw>W^2rP!Njfv%KrPS_wT5nPD&XFv z4R6lhlY20GxUTK&FzJ5w4~S;2OK(TWHQQM#@7jG|sTV!($klJaQVfwllqA~Jjt`lyh%c?gT`*Qb9x+qYeol<> z?`uC(Rd(KTx>#)1C4pV=(-LKzx82ox>I4Fh1Tm8BdwVMb zZ+0rZ+Wz1}8Sjd9kH&k)%iwG%YiYEv?x~&V#fTX*Sn^W5oZS??a(SkV9i13H%JYmo zJlZ(&Yd0g279rFt1d4i5*QTX-x73?H6-eKCslqzLN!&I$lCPlONa_$`Q`t z9%@Ab9K z;j?kL{{I2nKqSACZ~lZF(}qdgI28 zPA2orS0Yk0b2#jPycYBrm)(-rgTBD-{cJerjF7gF<6>uoBT|^?J%ORt&;$;Y5O4T` z;t>7CbD~!Uibs6u8{r~U-v|q?BCkdJZ$eo8=|=#4Y^+sZ8)w}UqlpBD{o18my}!dL z74Ocd;g_D`6@nV8-PPvrMM$9ZN=Wu}4Y)2O_tLwoJa;VeYJAn4X&gcPS=X!&!VPRB zaBc~dm3{7t?q&shd-rYc-goW(?0h`9x4ZK|Hbky}e)H3uVBf!$^Xzd+4)S5dgBb5Q zLK5fWTHZa?hSP95FM>HIOy_b^T=qlHqg-($g5{Pd)H^8o>U+EL?U7)M$I-~$V-r!l zkM4sc{TYzmcpa%Od1hOA&dk}Cu51pmU%S@K_;#I4206MpA@M!Bg;pHQosvFYvxT{F zGGCWk*3Nsc$B&394U+Qjq4#+sgAbpo_OW)KC!NjO39Z7}aY%7vJ<^YL0xSI{D33ySO0A(rtVjYuqF_ZlohXy=evY|-YCG;q{y zU3!;#HU5^dfLi=nT=mOv(amKWH2QJaPtrGVc_pAfIhe+iaaPWBqHR3tOMMFHu2;7ja>~PN{dtfolhB*V8r_37vw?BE4fUg-s2GNQypn2$s%u40g6la3 zG#c&Au3vw6(DF$+FJ)Ry)@9tfE}3DSlW~@#QJNuyWB@H@M0k-h(c9JIXcRAcg+&8L z~S-#1;Q&kA7(9ZUa6Tg%1X+kM(WoIe9Xo zN=K{z&|Gp#bl$i3f(Pvn0Z-^I-E>|2h}U}UzI>1N_wi_~;ujzID}EDgGLQ$jKhzO< zFB=r!_+bne?b3_C>Q}UjUqt^6WlhF49<^IKCZuLp&l1_G$UkG`Xm+pIG54hHkR($L%~qoSQ^Pd(MB5`y`Fm2< zD&{n}-2h*2&%TD4sXL{f3`VX|3>;v0-5&31DXBjk)qP3*W=MbcD5zZ$8Cu-baS|YU z2pWW-8WJim{e*fddxgn0ucG3raDV;?SGzbNV_#r9=WS+_$Y=6ka(;dK_~VbyUVO3d zEP1X57&{CC2E56dv@JwusYh-fgIbiFKe2#82i%=25x)pPk3}7#89v;H?lEj&-7AAV zNyio+-Hc8l+Uy89pf7HL^7`YU5!4O7B;H{txs!dx@t{xeB)$5@wevmrE^V(6ANVQ$ z7}^$E*W&maUSFgkj&6EHpY#PNy1J|7{-O^L`oSoy#{tFh%y<`SPkgwekjs@pJ8uWI zqYZF3`<=Y-f)0IK@*_L?sgLv@+FT1^^>xKPW=VFgfiia!^a;R&H}%n7Y1Lq z`{dT%?zMfNXV2#lC$l8Br>Ej@?6$ePcSBNxueF;(+If-8DeY{Dxn(zFupJ~xJl`bo z7*`I2x5u{PIpo?n7G^^sA3-6wkw>s{Mnv2>k)3?OJq>sDWD0U{Jtx`Q2N6A-<#w9L z6Irpzo5x|!GFRp4tZNTWXBpC*m88wC<#Da}Tap6Zqg{zOLtRe?&0*~kN;%y#vA{6a^uGFmb;cN{a=&$ZwG%j`1PO( zoH0Q!mjxYf@p>m{Qs;>LS@6&Mc!F~Sx}FMtE69McxO}e<(aI2p7A6WVO-R4cr^|&5 zo{#zvt-lDq9z-)I>aY8dS{V;*OeDP1U=Aby8$rB25&TgQPaG51kO6_OGAicdCusnm zctkhD?t>nsiT`B~pLl=2NBJ%S(DXv^_k!qN7Z{qa2GNMNw}S8War*jPkYPm@;`D+M zwl0vz{}HS<$nb9UyT1ONpuQEy)3ZT*n$758>8~kNAL>i#j8}4UO;-OV=uDSuJlst_ z(29QgNMHYMkUkrS@H;{G>G{xEQO3Mv(r%ZwJ30Bx7TSZW){8tv}Ex z4k|=vIZpJ9KAAJgS2~)}O6So2N|4^&ZNTfdg3kr53H0te!AFDi44=~RXwEE04WD>~ z`r!58|JH~2kw&lR;RZGmSXly;@5;f~GFI{7FCPQV?A1I#VtgPA90J6cF`iF4b>;Q)q?tAAEFgVtA$yptzo%b z-f_?-+VP?f@bhdRT3Gc7Bm;Siw?%C)t;KU0aqlFKM`hm+0_7~ei{;&07tn=Py2KCz z>(q}%w9!pG=r45xX>^mJfuC;%p9?a+)&=#e>ol3JTkkUE@I`N*3F3cx33} zhk-~Y(iy_|)Mw@%_vO88pl|i3c@^Ldtx#X;KYdB3&7XK(XS}3XgX#marArS7^@H(I z*FQ|$+=4dk!o@aAo@9znWAK+jG(vp1rw@QS?gik1PMB+qA#>03!E)}UL48ntA;fR# znLa?je0VgL^vR-3lC*)11eTM)y&4e9QFlBSMK>t6KL6s@_QOv+w96@h(hhGOO!MKv z{34&{wPwEP=Oc&SB-KGHn5eLe`+Ibko6_B>xx!V&yAsK8GMMTMYPi<*JgiLEFmI`| z=Gb|rb4KvfwbRa!wDTf6-HwPm+3{$*BU+Kwl9aHW7eUT|cDo}<2^8IZv|XlP3q!wf zkVOe_LQ>qu_={e5meAYDx|~<`FNvA384Zj)6TFGtVzw;k0b@`(BC}Z4B9Q1>px^Ya zg;ZFJ>ywGs;$xxLraT-KSdK`ANty%NEH)HYs9p;;<5zwIMg2mHFk}eH4^C2t?#^m| zd!20UFsjIo+!<<51PicRlQWm?1r`B|VV1$E*6cZ4IV& z-yHETPh$C-Z4o;p-J@77%;QaM`bCEHZ1yBJcx=m!Im`!tc1v13FXGBuv)cQ7=b7fy zTc45*oW5K+9Nmp>1@Gp5uK_slVZ;F6F5>N`NmJ!X(`0*PaBk)$Vcoy(tyTep& zv5{mLZ;w*pXqIQ{4yO-3c)jtbohu0BV{tEoNVtweLd%r3ATe;)SzJ?Wz91OW==gpg zT3Ez?&|g~^p9%`mUyGx1ctX!2NO#{tUHRp+a4~imH5N7@{CSTSUnd zNVSC@3n^Z|9eh4Wj?(Z~MyhLzf0f0{Ng%zDT-2*gw2P}-|NJ_Lzt@8JW4yi+{3IwZ zonDpKlI620`*d(!K=+$LG~gHC>IV9cOvr;F!5ASo*Xx4%>5;ZdcI(!4(dE!nyc<*a z)IVl0{JUrBW*~~|SN%(W|2U{`<$pV9Y_8g;DRLs{2gYEHBf9nDApIiC-wmRbZ1v}j zpfc5;;-|@4I;g%zvwDEK8OnHMc;X3jA62HE#JKa{ef;es;>ZEAlg3ak)G>Y zvMb&bJrimkAb(@+xggn?Cx0ubZ?w%o*S>wIYa9Le$3NbF_K|DXzqoapMBl-n@%;AYnHB;^xf^_#BkT zQ(L>c*QPn<5yLm*sW}{($uca5en(4|7Fs*Zj!Mh!5i5DQE#|+@eVJ#(TPx5z9C1Q& zkc1Z!jMbg?TQSoR*bN;yj=@<;!G`ffa(=|Qk~Fwy%xB)o<5$g1X+^ejI=_8{ByY#9 zGbATWxQYVHYF|w%${s|V62XY0GMK03DZ0|+XgcoR#FdZPPsedUB0R7EwTt4YM}r8i zqJ2snln9`m3nB3q?N$r%{-2mU0(~+a=L3~_o-W^3i6@k=Sf<+NpTCj2bBe0^s}j-z z3y;a$1aC4oxvyg2={X&wyamX@Xz>_yTCSD@W%RIxunfRO70RuZK>KJ!gSdtKcZ1rr zz<#$!ax57Pd8=}{eI_UilqEy;;TQei4&ocHuk`q7!q?mL69gp>oylcrduAlJpZjheKO9zYpL{Ej1jnKUPZ1BfH zxyTrC*azft4vR}SNzxl7`%*FY(#>$XCi~D)_OYu1ruS0(E1{g972L_xpe4|I2a*Pc|6Mk0#{HeQmP}i!Uym|`h zQ`J*E(G#f8$WxhhLHTM!{nggG_1(#G!@ktcP=DzqJ@rd99_TN=^etoVX0UX(CcY#s z4JCU8)W->g#t1!rCdeVtW;rJ3m8H50@UC5e5A6bd|4tAMB@g+?0^oHWep}&l{xQyr z=3!O%qIc*W>iICwNpbR2Ryp!8&dA!lK_2?td;-C`@>j0P;$aBr1{O`ceG)_u)INIL zGk?&z>W4&E1~eGw+QDnhnH(&@#zV=J!}x7xOVO}6Ew~Pvo%-lJTr5&WE3EtTP$i&3 z)4g~tU+&eEerhFyGF4B7ig%A>wMnkxaah#cz4o`oj{Z#Imqi?-?(Jb+y4+^EA|Xd@ zI!Dg#NSr%oMzX7PGBg(=x9~AEvx=_@m9EFPDx>^qI1S`zkN1AXFB!FP+TJIn%m&UY z0o`lC@hh0c)uLJpjz!KwSY4m!g@g8Mu|)^x=UYJr1EbCYOG}2DlFOyRJXQ6o^O>MF zEU418-3^FD?l6#}_Q=&@&k=bp$f)qk`UgF7Og3=&C1BR8bxu* z?-#-0?4HXrq3Ty3-SO+b`suDXJ*G#DNB2*gg>&n|pxV%;`NCL5i?M|sawuNu_bU7* z`=tatZ(t*V)g(|8@oJ^+LGI7?_eXmlKe{*C-`)A*W?r0kL8 z0jRv@*Z4BL_Y&GAaorxh%41jE^H@pAzy6-bvbuRe{&uI85$nck9@jpMXDT-kJSiT(D$gzz$9l4s0 z$J>XAt05&CA)ONvmoDd7@|@Uq?&2mzlt`I`(>!lAZELc#Az7O_9iG){8FCpqTm9`% z65}gM@W!!eUI{$SV{4OfzC5x!x;Z->?Wc*)XS-VmZ$I|fs*`MMa`Wd)67Vb6kAlA& zEJx(;CT;<84>C6H1kiYkn_u2&vNoj&8Ru^Y|F0ke-ThUdjkpxBNdI>5$3Y9K#T&2M z&^8C*PVtwjkAZgZiVpld+s8jn_+Iee1^<2UqafJmcqfO;`j?I|4D}N|ga2*NEcSj- zpS~XaGkt=V%$L zjA6R#`WL~U1y|uW*)JuKuleG?3_cgcyE*fBg8IO?X1L!;*{Avjbk3Y??%?clKG60z zf*%IS{a*(41u)ORRY3jf`~9Fb!4AxigY-`O_XqFRoSCLJu#v#+C2(aN5>LVJ-W=zn zBIE7t7|z)#h!x#l+LWK(x2N$6Jo3~z1PtRLXNKPpT=!`etr@zq)pZrEuLux8HwUWL}-LE>6TrfYl9w&rzk`pxD)ajVW;qmn%jsOSb*bd2J)nD2n;f7RW zCxv(**CEt(@h3w|8{7~LdcJP9&y$&@>4b5UhE;9i=aK+zcqC}CVXQNH{PHKX=+LMW zv?{6Rhca+1aJ1gFwm%PA=qO9MF$7jyhF zR_&Ph1@@I-8JCG};0j1!xLSxm6Qs-ZmX0wJYnD56noPb$kiW(!Baz{${^x?O=`Fpf zdMixQ9fN8EU5s#aFp8xcTXfvc$Opm7o9LAWa@L>tdp5{PG*_BO%#rxKTjN$+WL%Cg zx~zxg6w$TPjha)H8Cn&&JQGUi%$fK!Kbbq}GtmCsnloF2n?D-~+)e_N_A0>W=FRbl z70_N$dyv&3Ke#h#vwP7FCA9O~rwy+@y}gqwbr^{S^CI&|-AQ;tT6hoN>$+6gQk6KRvz6l{wpjkC z3T?vGC7?eoph81~VZqq3=x|OfXbhdZvB;vKhLuItLTdq}9-s8r=y6TiIVtmzk@!hq zZn99T7eDGN{w%8H6x@qB3{Ax^zR-sbzo3gh9P|q_UibSj!3Hj=1lru#)23hbDbSB} z?$e;LM3?n*{Q`8B4y&ieC|zX;tIv-F#;P$Q9Sqy=Qtr;*qOax=MmjoO8+-5capM-R z-9nYf6@lWKbKGX z9mYF-yBEBs*$r$YaC-^di^cr*RNW>Ar@)( zu4*b8B!y@bi^sO+jz{LCQC@w#ogI?g%adaLD}Sq)k3@EF*XkbAI(x0lX^s@k^Ik11 z!m)JN7{lIVD^5gk(mjGDoz(FU+uAIYQz$mXI3sy(%!6M@pG~%ICBC1>!^TfP{ctte z%5DBEEdkwTk+onkl)f2!Hu!wduVEG*I9%{@c~`)k!YJ3b7JbGW+Jp=`k0A&-C1-)M z-ws-+@%MxNn!|-R3vLbQMDH2s2MfN%U%z}Oh*o{WkbOH?&Pk#hxa1Phhox6^@b`m$ zWu~(X^q&XK)*wHPEzb8XUHKY+>hMU^-wDza*X6(_dTGF1L5KfVkiLExq`yDyukkFs zo8%L@0$@QW^KS(4_FV9LJ-Wv^ycct1nmq>e#XMefi23ffgVH%7Z}upD8qixjaXidP zoGv@&c(x{(FX09@61cPyxKa+u{h8t&wYZkyHqB{?Yz5@g8yU2t?REkr@89iK62lmR zgl3$&6v=Uqmf&oNS+-`R{;Y^ymkx*J%F1SDNzWrq`>qtO%T67)@TErMU&ji2Q$KAKU z3GX)0eg+puLP(XL4N@q^82CI`D&7)K_@n%Hf(#1$G4h`eG9vJ0@x2%PrP*VkA2>;r z98J#$zaP{$zYKm9v~yx)Y~X50p!ABq{$7wF&LR0(@L%`WRsuSFQom+yZn{o47>Eo) z4vEnC;H=UW`7R5v1<=YN!4v1}tsp%{Gw16pgE+S@9gwlH`GcUj=-EDQe3HHU)z^vM zGe84pfxbX<d4hbVXzrx2@_Nw2^ zDA|q3UP+z>ihSqO$~@AgnM&R4{y>a74@7~R8*+iC;fW=4GG1?gs zoD%4f3u*Tf&}LqNGypwlLnot#fqt9OksRIPX>nyxa7vU{mtUSQ6X2y5dbFVvz2vYT zWb}w{;Oa@h7~?!J4D~VHEghuW98KZkJVqzz0wdFyr2llH^upMdexM(F7L?nO0mWHOo3KtfSCoDJ-@m<1;O z1a^HSaWC)Q$}?JaP}(bqyIW`f&3TY^_Cv1T(TI03DAPC4Qikb+EDKRiiCqte+~c9- z$W0uOo_PB!*<9&}@5|-Es?L!q1{cGYPhyANBU zmA{a742oC3PU4e6!9bAi*L23lqVbxf4P0UgFa&`eM`O%*Go{gGJ11t*#XdC-=`6$0 z7^TB>p!B|UJV}=h(1;&=Gi2?MNVBVnuiNcxCd;J(^y&kBrXTc4^=G1&5{7fT1=6*7 zknV93>7E}tghP-IZ{}KjDr4>`olA5B8wp%)3790W3>@t4<#k%q?Q6Tc`O4lnZ{+pZ zmZy>Ou-1KhnM4!Ke>))Uu`AP2;vI;%k;G%qW_8J3zCMo&Un2*Gy=dNDJKY|vDSh3H z`c&sZsJ1B0?BVR^jl3y?j=btN{?eBp)#7V7<~^D01T~adW?YYDb=OCKySHj(i=$h} zvJ2uINw;g#oCt64ZTI4SBt)FcnErcVpr5)9_j9@=@>B0LCvZ}-dtI;{r@Wpw&urzR zl1_W%k*$NQJIU!WtZCNX_UhlnyZiafx;kBY-LWOAyOm&^++GtbYyRe*<2~DlK0Uv8 zb7#E&S*kre9N#*4;e~%Wd+DWQ+`y_5poo6y`7v8e-ivSXh;|-M*E@xo1_g~LJ81c z2Ccs1AnVh=AN(lzX^<{+OcsAxp$ESkd@@KEgzBmFw$0VKzHETsGT4pVZv~A(e0~u8 zMUXLk*#RGbLH_NaKGP5SKzNy<=JpptBw4>3{4n@hklxU>Uj_ZxF$D7_{(cz5Gl%4< zAidqdMgmtt0yRNi31Wvt=EOyINOJI@{ksG6drfJBP!w973ha!S!E1+PZ+j=ZB6;gp zzeD16x0Uy{VeO1`p2kA_O?){CP`G0gM&f`jG$N&O&AIr9&F6Ut8p8`wzTN=U$@ zbM;{H_b41#{@c(yZLHpFHW669Nf9gjGQIuTk;>c%_aQ_PzRJy;2iwib$lPQ%!%mFN zh<4C@VDc#7aV?>Jk@juIly?TFmG9%~wOiB9<}^10UCGDcX#dTNu!ljkk}5oU0asT? zE0+9u^FYqU+?-4&+1lEhPWE^)FIK5dxS|BgP_p3E!ox_?)vH2vE`8H!et8pWgTYls zS1p7C4a?oLL0+TMVrtYf4o;&J?H1H(kMSTs#HV|!w4Lk?@KqXXapxG}uhM%(ZB7H) z&|7qiuL8w`x*@tDUi{iAt=dj>1Lu&yu>bU{zWOXkzScFUKT2olRmBrs8fr|bpMEe7 zYu!9eCzUZS<>iowS3QZY4MW|b)1_1RLK7YfR@L1qS0Q@wYbTbh$@U_6NHup9&{%Vb zv1jfnl)kQ9Y7QX_a~Tu|?ul!gPC)URM^@FlswY*@d`b{=aI$Y-95O0jRe+~KuxS`S8#6&8kg&gb>lYJp6%+QZTg5by#9B`BVDPw z&(*P57s5cdGm|qS-Q@~@o!&5pLwRD8i4U%K#q-^gAu|rG&#QM>z02hN<%|^)lS- zhA5-ma<=fc&f;qP42=hm*}NS@hxFo|92kzz1o23oWyGLs6_B6dqiy};QM0Fl>iZyA zBP7ucoKpg&v-*#Lq(A9x>5{&zYddZFrOj~(R|Pst51$Qk+V$t+n5(*L?0DpfUe&ls zbZx+6Ib2o@G`tt&VB^a?;Foc{L-Z9riLMOlKsz4jaXEd7URuycWMK}`H+DwE%PB0` zBxxled%9MkaZL7PqA%PNH*Xm4Q2X?N?95%OLiL?G%!7E**PjI64=#fI_%pw1lg^z- z-&4HJVfdnVLP+o6uY>Of-9r~S%ef+?#&b&^ z6lu;#GW%|c`mIP_(gRK3$XBwnU`v`b<( z?*2I;(wvbv9L|e$Cq`^}q=n|8;HO<>XC`m{GEe2LT6?>rgXtHCx2BW(ru#d)Cbtc& zCjpC3EmUQ|FB(V7)J!ETE*2HPc3J3t(uWq;9|b9`1>NE%&9(NI0eLOPRbCqmCCa-9 z(k#3j5xj7QTw7ROYjfFlleGcBDlWc;bG^im$+9{qhLNcW9_^RBh{qpNfXa<1vJ@rw?0X-C__RdqK-jsbf8knpXb z@nF21mxokwM?pW}-xx-F>43h+gZtG-i7o~7jO@$l&G%CCL8 zryumks!%#b=IXr>)E0fi<08oKUO6N}`BvpOMdpBaa~1yR86G)A<}mnn@Mh4MleaL@ z4QwQ^x&$u!1apidUp^mAvo~?ymk)`rz04eI;oe@DO%|aYCrRQ#hJ1gtq2W3j7UpwUx8O+Pe325s}|-@JJUYN z8Jhf?&f?zW>xz?uo$;d&Uz>jP>HC#kl^a-H0=n0NW5F|y;X+>~Tx*j9=U?*A3UDKd&{)>vyudcealcXpU_`@N6w`)WfU^3#Y_J3 ztOClZlY?v=2=#0Asr$pa>2K+>a&<4!m7vC48HRL=P(SClAUFEUZndPaA&ri!y^ZiYui5gelUCf^BZd4Okho5OaH09*K@R~79a6-79V|JVYg#ok@;2d z!#=dARl53haxJupE)8fd`pSSisjz(RTS&`@7-;B8MQ)o$AuXI7EQ{&8!FPiA#J9yA z;!V4+2Q9u9escJFPtj1HKOM!Bw!=>pbU#5ev`)R{``Ezj#%C9UfL`v+~xwWUMlT@o$Vm2D0(=X7HCm5s&xW$5TjcwC&{(2_c!lDAz!Aqc{>h^HA<+$NB(RzU&O2CFYwaAlfBDNtrr-U> z!T$dK;r8ye$?n!HC!=GV8-;DAWdQe6ZnpxIW0J>?B=q+tIUf>-Aql}25>S=#dQ-h6^R`x)xrYU!3cGPdmznC62fHcGtack3|Nl9T3MD$kZkV0t(?7$3xU z+|NGA&Heq+%|Cwi*A~;QmtTIltH{L$mX&~B)peBCqGO?Bcu2PZxwi^faNZ97dC)I@ z4ELY(A!CLnSpdHoWSrj!S{VD$l-SvTE_szfvjz1uI?=98?cnQ?U>SOet_5`MuY&(B z$cVw?Uj!KocyC&r&?1VqzYOA+Y_$*3O;_a=4i{{3Xfdw1L0kDO){G`}C|?VAq8nIK z0`vruv%Y;ds4uHu$e!L+KgvsQ;dg?Z6aD){P~R>KrOTXEKu5(PTHMzbsB|Hnb`P!s z_+SX*kM6u5{9l8w2N~(o7{~G%WAqnq@cH03dPI})=l&`*R5@b+sJHlQ33u?1{(L9+ ze+FL(lCLp@Z*lj@Ls~ft&gCT%pVd(-$T)pVn)p@FcTOk>5EV{&wT< zXbcWh#LZZ1gP~^GI|JrBBz?Y|dLLzXpu0LEi?Qz`A$elTw@AWE9GQc|I2k8?v&3Un zIVg+riafjHyG!>#{zzZj$?TlnTX)}sxU!+}BBw_E6v}M7~`Tfe9 zxhi?G^`rLeki@geS7UND-p%Jr;)tjj3|j9JHh*s`0gFzd9<(qqynt)>Rsp?d;bE+Q z7#!}9FrFPs+bc8hRNJ?*$n!9|ReAc(SPC zrTDSf%8OQuxrH5~nVc9t@`|gsWZ}B%s(W-?-ZGw~S6T55oL>Uus2|@BGE~7(w)Cv@ zPT$HaUo9Hd?@3x3=p)?#!Xdg>V^F%b3lm)#hWbPI@bGHzl|Jr1y#qxT-t3OJHm==A z!%!}{Tnf~oZ1JdWX?FwAi~hHQ_%+t_o$+Z*)iwTH%c~4}^?|vAF>cPP{!Vl)Am?&= z%pGVDnp4UlNwke`vO*JD8S`i`kDdp}91rR;591A=CxN_ZkVkl0zT7$!$P~}yYTW!H zs2}h|&ZSR6{MPk4V=Bcru#rF~f%6X5yC%nnAKsch_O<&apM9}Cd*Hrn*&NAl?n{CB z2^7zxWLevVIjYPfSI)wiDr&ZHHbgcLq~-Ov zN0}4Lu1W0s(P%I4$C`Sss)6|??n8D&G8MD}+H#8n5|^T@M0QhmUrc(t??Z?U`k~;TvgMNFEjjm|J($DzW#ip^12Vam?6>3fZLa1L&}oqRR_JEm znW9UBzMvr%AB!urzzoqnC~eig8`7#Qnv3p4kAb{2wH8!!lf35vN@`K%SaG&2t{ydD zB;b1pbQ5Y59pW{5)KmP6yC=kxP`--Seb+C2g?=qromT+rq8D@os#jm?)004c zP|l`M-=2q_s1;t(Hzeiay_HD2Ul6q8HIOWsv! zQJ?a#cuVvcl#Y{Wjf?Zhw<=eMv5OC54^PIgv8EkqHSTI`CaLjQJkybqiF@k+I@C|* zez+;eh-{?~0bSZAXQ49s4xOO%qPtq|Z`H-Qf)@YNc#_w>YDaw4KF2S5(QRz$1F|pu z8|qw=H?Wbw#Yw=Rxe73Q`Q@$YjX$_{a4Q!31Ealb*S7P0@a>rXIhNfX!E%tq@H=g& zF2Uun`0c8RX+Pg}IBlk$H(Aqzor?Jc!K^<-dM1FdahqwQ`vQA6xx@<@P4dVV=fcz@ zjOE{I@HkdW*H9=F_l7OBbm@r6gI43b=65#k@`i%(IPtbl_kX?8H~+F5(;nH%o4k~7 z*GG?TnY)f3c^}K4N4xU4me{T7c=Smc-_K6TXItaFnwXKZfz>3S4-2WMc!g?my$)Dl zETm;j)Pf__!CfbJVckpe(ts}W3|h}J5ae^eehIYr4*ARF(ol5LH;bv=5x;6OCKzsb zT7=?P2ra~9f`&>H77XQm+B_6=b)99L z<@Ox(!P`OlsikvYk}=y2$(yOTy>_cFkt6c$BZo&B^I`Pf zacRrbvD(e!p)9ngzVP3~ne!vv>wKF_(Hx9WnR6x0IY}1j`IDv-I(O2pQg0c(%uqbnM6(7Z) zP`=?ka#K&qSR1ZSYCp+eDJWjafXwk-dW6xEsra&>41c*#_$xYu^q4-J2k=M#(9NJ_7?zV@Ld?u;sFXyb@>&s!P2L<>C0LHY&{RIlXvh{GufCJM3g|ofQ;+Y` zyNm48RcP7d{SA}ANs4`GMer~FMK=ljPyb`SW3kJ6W(g~H;&&NrqrEb z+Ok4uk7{M;4`4%dgyoy>Pa zQfwZNLvS0bUdn7X?!v*&)BT6=Le=R&;F`>^FndSw3WxVriI+~^j&ApB7}l^{ zrJa-VqMhEsW%)n{Lx%BCG!^aWGH*g@MwxigUDxh0Ov;EfGA!)Ue^Y?W@mun*5dSqh zi7yN4VjMFz$xt{utxxr!VTQy4T?+#n%FDX_qqu^}}f(Z|V7BcpjRoFK(yr%jP?a zemv4S4sq$-Y56zl8wp&P1WX`T4(`9cH`|}sd}!YmwQCu>!M%AEwC0^?JU-9%#?x4=zXF` zarbJjtWj(TNq6mD#cPDSW)OPImUbCx7I(is)=)`w4(LHwjS=;%1KKSdmajDQ!nz*j zStpL}AuiTY_o{EnLE9y>#r$Voqnk70S7JufvcEPfXV=YRFHl_CI)EP_EN6jEf$B3n zRzJ2IYy2r0p;ujWW*M2GOa3w_$%w3q?nKvyq0S8J6W0d5h3Dm6-=O>P;8Q_65_TJm zPi;YM3~9(^1L^vu#yWb=1H*oJG!>C6l)ea+S3lJbjrjx84R}(yB%ca+)c1IT7ol6J z=1d?%ZNk&!d}alkn;QwNBLM^F$^eTRvH#?sJT$#Fo9yq5wr?iI&+#$WcXICAVDI0! zm0{Fko{?G-jJ(qzXCd-U##CQKsJ5RLx3d|rj`C-rH4sw9B{TcamVfNSmXZ?WwKy*w;?>Byv7P zI^V+ylAXxF7Sc3lM#lLbr_;X(IKVpEPMueSTCyZIVfS3JU%O^$@Q(Ke)N zeo}h}`6|rT{`8O|l*V9}H4RL+cSfJjb|-mUX12exb9ngr>uoZvqML9dfuRJVIc=e5 z#G9AOaBpLNm~n}C8)3=E5V#Tie$d;8JchtfImu|!W<3t^jl};!kIJjBoE2lsbsJ*| zLA6(DLilFz-v-|gGODBr85N6Qa7kx$RQ*-&V&2oP)vt`111^?#T`2FmP<^7$`dQG( zZ}GaPZfO8-ezEsFy+=UMM&2dUM3(|(^bKRrBPN=sz-A01;`X~BJ|^M!d&c&+&&ZJ=S)8vG#1q9`gZl8k$!tj2)HKuVZTUU21-YY zuanADd-NL5#t?&jo%T|!fxdYx__u@l&EryZ=FMQu3)-l@NVI`AjiHZ%#=iUJlk?D` zT@LKGgXRu=s?QG%i|xy+pCJ0_Dl~_XA--rBe`UUcujzqRwk6Mmd`j%2dqku46v&S0#I4leH)(ba~j2pG=d!bOlS zf8|ZIy(vpldyFfQ0UX_9>(`@Lb;hKt#4CWIV%Y|X;}Tb>cV3d}S*~36<&#S(*Puvu zEk2y(Dvx}i)zL?~vm}{|dk3Ut==1bh(lLS2_W0o7;8r60ySvlL`|oeQ&R(+GgeOXX zno&b&u|H`PC3$I}*0n=YzigWG+IU&0MPBz9xD1e|gTEboDrnf0F;QhJ%&lWEOlK6m z8pMk_&N3+RIv-|Ym1pVl zG~KpEa5~QCRg!r&xqf{zxpCvhW@2lC-~2g20(67nRECrVxHR;H>`NsX2@E>FQu;OC zXm}@R-qViywF?=k9uZ+YJQeh-re8VB2&wW(S{fM7+U!TjyseWekjKD+V4bmoU-WC2 z{Dkbb!)1;6WIYB&r$qg=ES>lwb11%Sxn!LKV~&xhthV*d6Ty=2&{)&Q5Mjke){Rn zgU27gF@5Q!mxhx-YkTu&BY|E5)Vv-|U{F}hEyRn%D@jWOh4F_y>*fPQT;U_J?c7Cea`97)Ditom&u3zXbaYK zpXhBtedyGm+F;oJ-QY7pb!*$BOzwfwPwnaNRr^*Q>j1fs2b%CGq#vadiLS(Vn4 zJ{?5+H-gHX2h@v4yqZglA3T0lz?h)tP#o&F+l>`+F$Ty&Tz+F6yK?Z+NAJ&WJn-n}`6NjU?pa;|y!}8zC$?{| z#%FOmG!?{$*X^S8KDRg!bU%M-a#P;~zEADF^IPY#26g|Mh@6u+h@;ry`JA6zW>4Vx9xg7w%Zlwey(#}=YG!V zl>B&~b1dEiuz5RpFfL&NX4eBUp~^JE-y*#ZKMW3|qnA9aKmeArMK`22O+>?Qdl~Ur zDztf0WhBJM{1lEqCDXg1j;8Pi$2`dgM=Zg1w?%557~^}Ks(t!jw~Ve0KzxtiX#YT_ z?zckLOa3p{dh)qzZQoGdeq(20wK+JFgP3~5Aq&N#(Hk})lC8eM z(@Zbx`T9}tPbWTHx!yvh>`eYfh-!`U8+!V{4)J0BdG0&E z-w*RrehKRSymVMOL`<8zT8G6IQ0fv0KUWlJ9PPgGk>7X?J)UKyFE{>tQhWfteUkug z3b88R%C}OTn!;Em?MnF3rTmx#jPqYtH7&T@%j9W{R$e)e~;z0Rf&@Yu7GxqOK^_tVaBJU5W;g8b8i8CBVupMghc?eMoPqk$p4)|#q zJ87*4^s-n3oEs(mE*318syX zUNRcq>Sox*gC;em%&S&a@Z_)b>A7i+*m!s-m*x6_o-pEND9{&gW7ne_UR@&CS4r3F zuc)hg9Dj}yU_=}^rar3&qJq>Rv;B`6e0o9d_!2nDG{!pw$SH1$(6-6nN+L0|(~I$( z>ivb&lBgtxQ+$Ygo|&NpmlViOk8_=Znibeq0U1!WiX^Sjc!Z()0C}bg`yQf0vk;L{ z0ncuiL&X<}Cz6mahPf)DrVihTedXeiyV9dr@w?1hxOA|CE-$P^DQp!mY8D^FG<1ZU zr*T&$LK<2v_zk?M{^*B0Wz$$hSu`{Nr~#4L9!q_YKiTqcJnMC)&p)()!G~nT=FJ&# zo7ES8P|yX+$4GK!X0)@>lL12rXWU0He$1478+$pJYW5zC!?xlADcw>0HPt*mL_OS6fN^;v^v3JCYyElSUeeaW=@ zORlM3A0F>wujO5xZVzIm_N%akpjU3|+IJ&FP2U<_Tm1zw%zbyt@yPpN-q+f5jxuuROA9hzVL;^0 z!U~ly-^2Fm9g%td70&QA&(4R1<3sn7Cb7hbK%y}r`4s}^jV9iy?mn6Gf3x4+XU{lonWFqx4|&g!_o3I$4n3BRVs- zFh*RyDt+(C^kNx?6lNGJ2N<(OVgF@!r=*+I9 z(j%N}iT=tW3lh9L#41xE?!9skYaJUG*o;5-vCiFb{=B&_2{KZ zk8JiQ9<7k~D{?ee1riZ&*velXhr@QhS$(8ApFflLD>L!M>avvRxgKQYV&&!;{G_^> zNE#+cx;`oNYXp1F%QDuT9cQmsQHF?!A_dqIdgMjXfAUpj(26pbdvLJ4pnP!W3u;{g zxA~8nJJa$mKQ77zzMqydB(LbS;zP6G9NJG>ry7Aou=|2PRp)+~cuja{5J4C=$zqvM z$&7ms^b-m?ISJRe@F5;ZA+J=u?4#G{i>Tbz_c{2&U@0cP%l$xQCB5CAm|z&$+%o&9 zQF&DWP2p<@MJLTna_2%9Q|#Y4)J(jsZ75jZk+9d1N47x_aW2@|`2EFQRum zza;yWPB;TUbw~HW@GLw~-S_UKBY`PdcfDtq>6ZCi84RT;0soU!*qp05GlNiB|ET<_AebSeKIdsKj!xWD|8}}kMj0y1SSEo z_j)oHxeEY;GkqpVNJIRBaK z_8}Y9j432fy2|#{+~N3N(5^%FzcT{95V>AN7N;6stA9Rt*9SH%=&_47TVkEVSy6W~ za>_D=3WbaFRnaw$I=E?yUDwdz1`i8D){)^!$-fO3ep`b*Kj)Jh=`D?Vr2<=xJ*eLG zRafy>P8Qq{?DSOsq*@@ecDtaf^Oyv%dwGsvq?Biw*KE9!Y~O8axCk5Ata~;O6I=^} zxvCfUkjdh>>FZ~^`$8o@7VvaPV4a^J;Ll3-HW<%wIJw&1?4?;Udno|^fxpBLC3^OO zuZfxf@@;{GyC!$$eQATvE7NuLF)xd#ytPr*t;82o%k^KgiKezH-cGN3r`P&up*}CZ zm+85BG2Z6n3a24%1=@rRPXKrosj1zOFKCCA4Z?J_S=tlcV{}7BOYW~5FDm<~#O(3= zaj@JZQ?R!+QREbX7r(N^Z>8-65io960SYR*I6{d+W8u=~ueFG#7aVaTQQVNP5iYx( z%u)ieM-u2ZHSqb5v9B@{b`;%&X>+OE*m&2iysO@f)@je`*vJ!!8Ee#t^!v<673Nbq zl30h>f@?Wg3cs@}yq_$OC2%>VnwNg_aem$+=$zetRb44C7xZ+*A2Tqw*+VItv4TUb z?4n;U@X;Om!UNyj#7Y^7ct4BbTZ__y604*X%$i$|t1?0}xR7z)XCKs~-Wb#T9ylgx zot>47nGJZ5e;U%|Zl3JmI4a2M^TOX-sk?wO@;B&MNmt%Q9{yCUfhM$wR(gZJ@5i;o zb)2ctf`nbw4yL_qqQbvpyP1#1Q?Zg&*^AsMe=udI>GaiV@;rH2>+S@&aXnD2I03T^ z__L7hFFHqQxS`R%5mtN&gNzO={+(w@DYuU>_(L2Kc$}2W9P9dL<5N zhBBVhLUT<2%3jU?&>XbUHb;g?9B@$7tEP-cR4NcANt93WZ$`X+-#8!xg)=j|sG0io zzl=*33gGl2oxAYew`hWP(u?QT+9+erpk!)x{qnnGkze^k!Gg>%32~{BuobZAguc;m z#-rzD<+6B2bGJReDnCX4XHjAAo%|5{>V5d2YT(i{wQA9}v+kbduaD^s)0E%XYdmk} zN)R~xYsp{6^c3|vOiWh#CdN^yys62=pYF8_!fInd{YGE>>@N>o6YrKi|nfR z96odh1_;7OJvQ^Q`-6v6DuA4SFR!TZzFyy|KrkZ}rrdE^vMgp3r~g1B682ofV|1o> z{}Vz3u18qz7(nk%>r=hfajUQB<=ZhUl*6${@HlZ$XN#SmT+YxFd9c@7BR;p;cK=dT zvstU)C%aVDk%R+H^=nB#PqrtDzbR>^Myq7+byAy|K2vRu9se&FodAZzwLeD3+7v1Mu65ok>qDBwtp(XQUs-{>obA^p7cMj5!_iBIlq*Kz2i; z&rs6mCED~^v(R=&QfD-g?EtBc_8OI&(`?Weh1bRrdOCPYR&R*+;sKpv0f4tP8vN1* z&-iqbqvqkWG?NK=3>3UL;U54-hbA0&8C4WDY9F4q!lw+pfx*oD4~@pyLL&2 zC|PiTO^Ic1(lb^91(03S2A5GHh@Cd4lIWEc5Xx!6pZ& ztxMUwO;Qghu%fEoSKVo%fIA}ES)7&WiNB%CL!Sr|Lgm=9)*@30_o&j=*D8pwdht|D zM#|#oSfN)G+xIUcxdcy4iPQe7@d-7pb$zd2cV2MI5MPQe2l~M6wY^R$xp|}Z1{A61 z)gIb0OsKc;#9GT2T|znD?nR8Rj9YSf&RcAuThQY>DaHq4~#QS9K$_FeDcYDwhl z@~R4)sxfc~sR1L-T2=!app-PPcb%JOu6&uH@dA#g>Dv`-c-Ctc??gT3Dlli%-O>Ls z*J$1g(GxHm0qHkj>uy4vbp}*mx*9`&@paI7wa23&Pq`G}1ZJ`S*0;!emyXT3 zbOzKP)zw1QV~CQP#e00Zc{(S|p4WC2L}=ckyN`fHL(Blgp5APwz>}-2)mT}DUmdF= zqoU9Jk1r!wNr*a`6LlY%ZK+PBahG%1iwM(Z}AyM~e&up5TKW$j-$! z-se!JlpC!)oan?(HC>EwIo<(qdSoOEdp(b(3vRa(@o0Y}tT5rgOvk^4Pk(36E?10Z znNsw&supBE48LK$a*dTi#|8ODDWb&cq8}jc&?f3JNKv2P1O)7_qUZ2t!U^I6u|nu| zHYes%<2ODhujUDwWKz31KfF8EZX-!(yfP^sBaL*tKsK%H`9V$fp&l~p%#1p8QAf;i z`HSFDx+90F(@ZvHKB!?EMOX;&)JH|Vo z7WqxS`L$-|h_2`{ESTTO#Cx}judlDeh;e_PML|Tozd?6KHK}*|zfqFp-1V0pD6LOj z-UpCnMZW8kU|Sy#4tThk`teo@D&&var|!gg&kkl^5v%r%6NT7^y#(N;z+xcL@C&r{ zI*14W+%+*53yZrogTfKxqz1$>px{WNK+BWvOg>aK0sW|Zpk~yV82So?u0hbqD$5Y} zNGz`^;r4So3;SbN&_&eCJSN~dfeV3!7dLw$z(OPv$5eT8ZTJIcG7)p%gPs?N@03gd z^Gz5ln)$w47~2s*NXTt2GRaJ$DaMb$(re3)3#05wJCbU^ZemQDGf)_ z!XDpWuc|luOxfsv9Ba5ut#7C%M5>$qE@s>QSUmL3b;Q`az0;iyHv7~a-=8UW88<_Q zbNY=JA&ng_R|5=Ll+!!QM0l?x4*`5|paed}(ua80?EJ+u+DFu~ zP9I4(4tHI1x;0;G>AS7Nb}brM7?N5$4Fijv6@KzJ$bZ^yVG7{eSdEbY*hS98X92@z z&Cyyc2bS%>5&VD2x{i!3XLPSKWeuO-F$t8Q-OAKasP(sC70_0!GX}6q3jPw?OkB+( zV-wIvgSu`jg^(gHra2LBe4FIOX|vpkA~RUBu>vL%=A?SFjZLuIcLLv@qj0>37lD zBAFYwo*y^4@j5z{QT)9K6KC6BiJh@5(tVU0J3+blyc&e9IjAFXkTpE|^`H_ip3b;f z==}bX*^p*;y+6O?cRPLqP*_7Oq-oG3Gdr)u>H(bcE0@)t4FWqQAbfXlfOexL?s3OP zYAjvu>#v0S%7M)E5Og9Vrc28u^F)1ao9%M2-Ia5aeEHqInKxg>mfCll)+5^)?|5TB z38Q_5SC$q!McXdqKiJCtm_O64*hk^dLX(A@(rxbPoqUCq0!!4d;?x~U#7G=Be-du| zs7BpR?=L&;t42Jm(tT6gS=wx_6U7%PG#UC^GZ}U0EY5y)w?LqFCyn>J_~0b-IMuke z0l<1ix;Sb6zGd%`U1n2HTvn8mNr>Ay+T>0m{9AjcSPF_}r%<|WHuEnK7P$!A^QxUQ z^KKE!@DtA-^S!$j@oa*N#r|+*xg)??y!B%8d`}UZp;i+xqtk{0DGUB562v7JI@ z+2n#2c2(C&`x(;CDsl`o#^=9q$(p@^HK}>~l);gAL-+UP&#UkEw8}~200VL;gNdhu zd%r4<^H_WK)Q_zdR&V6nu}`m#g0JeWz$RgkzcL8fB7??GzA)aDudDqw?>O@1mRKU4P(stx~>E?socC-0~{NRprXC&Eo{F3?6{c=Z++FgPw_& zo&N0pNkNt8CqNP;n@N$0?2LWZ1}|NlZz59n<_q*G5hGt(BE7@RG-rvmWv{`( z1^wl1{gs7?H>xp$F?|?*#p3Eogj4?)gs#~lojubwTd3kz6W6?Ywo!bu^pJV^P=k~- zV+_T$F;$FTul7r2(bdd`u%^1l`4%LX``gb*rq6z^y6?&Ta10N0R-)*yeMw)?TS;71 zcWBFI+&MD|U~NrlA4_K324)Ak{KUKf=HPUg86=f{RWVMZHYXKj$65XJM9$YE1ay0= zRz`F133B9aM~32A*S#OGs9c~>4wv1^N zAh;n6#PtnE3V$7%FNH>`pIeuYCeEJ>qP|4#_T#t)UbtdxV)1K?mA`)e9$7&nfR68c z;k$RBjgJvM5ld9$2T8>~&iZV%5k>Hfd_7BDwO|L~VxYo8)$#s?yZDUN6I+a2$`gYT z4#tA1R>`-dOs`0}Hh1tL34OnE&aX6IxkT1%2jefoDek2vM9B(@_1_W{ZBF;V`v_es zUP9os8gb-C--!@2tB|{ky5$?a4xbmiPeK@L|D-dTk#v`xWj2wc$c$yf4Uhz(oOzpJRm@XvN9s`zr2``zhRb zS=W9npI|6l31w_;1rsyzc z{>`5A#aJSWSW<9(lLnG^J3Pl6J2s9iN<)taT^7A>}_SC-+yED~dQ-j+CmP zNke{iPAf|3bmBCRX>3ZO%feaIqnkecrICH}uIVjc@@fI=|$3PmH%^A~bGt_jr5E6&FHdlG<@TrRrA%c^UM!_jcclkZOFc z4D&qB{_>vRA0yTe;S(90AMLA2hisP2$R?+`HTuKe)+6#p-#DyQ1JkIaAE{73eje~} zjL=|(JpQq1Kkcg$p!DR$$JjLo|3|+>?a&SM)1CC)O+0=rJbk6Eke?3YrKb9kPfSK9W^+zm&#e0mPfDVuBr)G(;6m+? zNm#!?#}~#ao%4Z;o3}w|yyv=p7gO-*1o>gzwZ)T-P->_Ivcsdb-s_Pc)wljG97GSk zdQKuUnMBZWZqmUKabK@YEub_kG6cQJB_G<-S+InrkEwCzKHf`Ln+a;pH)oR2Y9Guo z63=ENL}HXj=Ia1oOO)_EVY;@Sptz@hrTJRlt+FRU zaIxzVluy4*+gD#M;>5&t>It&?wf4^{vR*}`ki;&FvAki_-kY7=NPHS^unZLl@Rnsj z+ikc27P4uAE`%KOWOc?BUCgqLE8BB<5z=r3>HLMvUu-WxwqLCeCj_810&sfQkQG>~ zKMx4oabydOq~y|203&h`BknG&*FO$YjfOQZ)t^i*Ik+U)t3u_pd69(!FJS*q~ zk41dnUsj?zefU@P+sr6DbD`8kXm+wdcaX~R*X0sn=+zjzkC-u?B$vHUArCdzFS>_k zMkEUt)-;1#J%{AZg$2@YbjE)EZja*8NJtCzR{Z?{9IHNNmzDQ!i+}gjpjuX@@|ehlONP9ub+&AWu9Z_Dz6@PAn3ja3_&QhVeJ$Zr>N0 zgw-6oIs7zXBfJRIp;R?_&>FH{{zcI#yle zV;EA$L#U?<2H9w&$xu21GcYtR2_Gt+LxqcA#cq9;T6+`#FZHI97AqajcRZCBYQ?w9 z&~C#h{CP@q@P03Y8gx9_vAu4kXMR=AOoXi8)o`xp?~_+4y;~?s`9qa3$lWQzhP{ml z`22_2#T&0z!g>K6SuLHtXEDL9G@vV#d@SSnUKA^|m#5Mx&5H0m|1l7)1ydI415(Itw%IVNxKa$o zHhPggQ|%#T<89-0p2R7yGLsT6iLObv2-dA zHf8&s4)*LIGH!5((BTB5c|Vq8#8OOmM7$FH2P5U5`Tz=jZl5Gx^QsQzqUUIuQ!cZF z=T2Gg%JZ{2j(=^1RCQ=4k*8H9Bx+aZQ)@l_5w8>*r1XG@Wcq0A$Ok;03%DIl5dP%W zy32r&-}QxG=2`1F%b1@UX9`%Kxkcp9oKUq*$dF5OT_0kust3sf)mP*x5~|pdK~_28S2uf(onzz z#Go}v8sq$aOsYPsFhKOwwFEd^W(og8A(N(FoaVKadv!Jh5H~kLWiL~_r)!?Am4G;_ zq}B1ppQa@QGd|OXB_4s3w5wB|@34z(sY@paB6_qk{0(6SSlqAzw??!Lk zoshzrm%|*~NVNGDV@6%}R`Zn!bSJ&TH;C`-HqTc(|?n(Q*Sr6xpN=9BD^ z?6jS>rgcP?@_7**k4UaKi_)WY~tqQDR8}nt&yp?aTckl=Spq%ix}W=1g%Ms;|YKlG~+iJVVgIP$5ls zF72bAF*kDHCS5UEv#GzO#E4SRW>~1l3~FPpfl0`)?HhLL?YEQsbx~TOs%`v|Nb$(#b5UTCYSevIZP!+X%@lr#C~B zuX|)y&c?J}yU?~bAh2s~NEb2Y%juC=6kl`7QQKv$PUi=eS}LUmNR5!ufp@|;{% z`b~h?JHjO8XT`kn2RDoNEGfq9OMKkvG3ffEw3E7on}8%3?9wMQ=3v+5>72TF-Pw49 z5NJj~5_ywMXfW5D`P-Z;>(TeKVqJu9arPmwYD^M~2~}NKB;X+V*~C$t3GlkAaYmOM zphjvGuQLM{)KJoRd`Ye@gn`Du#8_M7UguA=BcWx7r#&-%zEWldkfj?~7465l4bdO|YK_JpHIlvBFWEy}o3up)OJ@YzE-w3vp(gh90Bsug|UM!b`Dvz=+0MO#80 zcKb4OMp}Ky)u5Hx@}|d0t9i37_kalvr>7$ezJwX~(PNOeb{Mc~H=L$x^DG;{`55OcTs*$&CTTcj@01EUQmaH&SOb1uc&APg5`iR%o}~4Qe{(|FSsVC5$tx z$IKm~EIP3@)q&+g=(WkOf|+yD?0OZwg)5sfjC#0~id+n_4|PF+h9bmdyXnGxtiEE- zLdSCd+Qf}X*IsJD-`y4;7pb{Gt`o3ibv<6s_jI;tN;F;q2O6%uC>tK$+HWhJPSlOv zRPpN5<;SUb7`uxYf#Av}%8mDsDYl1(N>k{#Bfmf`Xo;f@Xi)dP_GRk9w*+qil>n43 zdNxDh`COUiTtz~sPe+!{`Pqqcqklk+@q9vK+X6|B8s+`Dan#$Jgsd6Ii^tI# zgcpOjNSZe*_xE6tr`H?Vw`NOYlC=7oMz>ZOduD{V7yy%Me$!c*kDA@{)5q}=I2t5H zvdBBHwr__K?BGqA5f}#KXz&#I+N1EIdpX0gqzjLAdkJ&NPnBriN;}$%)@v%7d9v@^ z7h^~^m4(%BHjP8r1)xS8!}j0!bxb6BHTua_9F3O)mtqPXAk~>|#U}bq`q}#93DXe1 zrA)bPGhyU_vu^+_%eoK;dTzPguRO|wXW8ErY8B5WD>v0v4xcD>)Kvx;wDL{QutAl% zHNI{0ZX|NM4cGZB&m_OTAkgR13cY-8DZ{PmeDQ=^JNfk-WRo}es@a~`Tq!MH8>Ggw zZ(E#@>BbK@v&NL4{U{va;lzQRll_L0>za^765~l)2s@s^GQ}a54bZw_K@N{Oz)Ysr zdicryo1wV2Lf%2k;rOkj&vh`B)>wGw+E3R~mWOlOg1k!Fd98YZFM*xY;S&OIU- z2Hgq6p;sf{hQDDnw;+|7QD?1cNZLgNavsm>X?CC#FqUYWv9G>av#RN6VscgNIoX{u zlfaZ1i6D7%k<}BgGFtR6j@bqn@>+ z!Azx7exirTX9(6*CX3s7jO;98=FAIwoF(8<*_<@J?YL!lidPXhsX&4yy)N7A%GaVz z-=hwqe3_t9<(p>Gl;B<$i$!H;3O1}w`882+Ye-rZNs@H<7BBBPC>&ld1b-oS&MQk2 zbF=2M-b+D(_)=U0Dx8z`U99#kmtCgP6Y>(Ja|Wxb!;rR+qG}Dc#x{$UGF8bnRoqNR zRj_?)aey9$kPx9_@Rw*A$s7&f0~(5f9)39d5PgEc`CcP5_nShVg4 zxs3{{N>4#MOYnxA{U11Tpv-G4b9G`bj>|RYp%%7KH>VP`r|!TteduEpuLbGxvw&mR zqm}K^mZJvx27pr^h9(I}%}Q?xU8DJ=jP|xJ!UA15*Qnda_^57d(?^Tv46)RME3V`Y z;IVA0cW9K**$FAR;w(BSQ?1Y#`s!|2W#Ts^ZZAtaT@OHdrMd~(E$N4jOD%bh7JDnl zJURA`I@?LYm7X^9STg){Q+0bgEhcpci-=2V;a~H%y8}UvgwP+#!pf5lw?OUt>o(j9Pv7kNcM^{YAz1a@v!Wx96_>ay2ilT?^h& zl$0F;)fGWi;Hz)mEmpt_OOoV- zCF)F$yK1(tC0nhAo8C41ds|F>PBQ=)u2Jl-pc}_~%+vhCrmcIVB694Z?-n{M{lm=1 z1s&xT;AzOGC@)oex-FspJ?uwJ;`7NSoG<4m>6@@c0iLNf?|Jx{c@OZnW18borf&R> zG?N3RKJ|`1m*ALJ=(S*m1+C&9_g&u&!g!(LR@LjXlNLjdeebC`@0QZg2t&7iMQOJR z3iy)EbC{i?8=J>l?j<`YMpD7KJX;#KrdwC3ng`c6#B83qR=O+1GV^sQU7Cq0A(xH6 z%u+VvPb12geWQL{VE-R924CaHy_W{})CC-EV8(VjJr_Z5h{te9pok@V%K)3-wa;sJ z*_psb<9Qs4Z)5I3NdOPLl)N@wAr!-REHdM6B8zR}CpFaN8Lu?Zjoz2J7?q)2csv2T zO_VJXWH$#08V?fjiTB-R7l8sk?#N{vfkz<{g*&>e!%bTsoK8tg|R}oewR-(VOjB`Y(tJYx7t6%2$k8wvMDG>dskfj>eimejfm{rwzuCOOs zv9V_12kzuoSh_Dq3>~*v>h%?5u9Uqw-bi4<>_WE9`s6{1mpcGz&(({|%sK9*O$S9+ z5VGK~{B!}U%-+LJYu&?~J)(Q$_{KZ#LZi&~Vvw&x$<5KtUsq;9mPhp!AR8aqwCY{W z@@(wd|y)1TM@>7q-ocgVc;7W9B#bFV)q`jZHu7 zX@ziLIu0J^%v2~6U-FlgC!mbOPpo*f5In6%v}wzGepZ)s`M3`BWe)h$GvEbqiE_3A zJwp!n(!@5vaWxKKaVLh3e3mc_qH1l> zZpJt(I;_6UJfE4|XuLk%t{FK{CGXL(NXUsSU8*)LF&s7D8!t4+iq{Bq;h%GXRi%5?chRaBg8Di1$t#$YflOEVEr@aK+BKOb9gHpcx0 z;<-587|`c%wAU6i`~A6&T)vK3$&mmaYIh&7$^~{kqW1hu7*BM6>AuqQXA9Cv>sJ?6 z{zyuV?diQ1hofwx3_FmWi^g@6WO>GNKe8@{XZE9DxoR66!}eJGz~Sj0jpE9A+$w8X zfbUn4_H*E=Unb8p_c8=8>Q}}eL!3_z*t+2;qis|Q>h{Rz&XY@inw1a+uvf89Bm6Z! zkQV&a44D3gXLe;WpaS967+R^jZ$y%82(z43I{n8}_y z#I4Dihu)6kz4hbP{|sXF7m>-IK=%B5DscU-)@ljMl^zz{D2%L6%rTl%k9uRY#zaSu zgoK3p7i!V@UVEe&h8tp;S(SPf#|DMQsj^kG(>|q1eKd+W- z=S98l-gNSPSs_QvMf?t(K55`7`7F(6>+|98d4ac=vb3tLB-Bn1DOcTYT_{;DsSaPc zlHzY!Y3ekHQou~NQtf7nCNqv9)}XEBHua9VyE%okifq?`g}>ruSj)!SNBvl`&h@_N z@F?+9G3eGScJy}8>k}i3kz|^hN>C1=<(Z86FlaeG!vK6ky@OFOMy9$syOP?m6SduR zJ3cIB^ud4k`tK>sjtvalWZt|ugIq;m(d9YI?SLCS+y#(d41|ry5r9)f^x6rxxtm{o zHr^3VQ;Sh>yS{d*0IPd%M@i(I_m6qjo3b0I2c32XL*)3HekFk?|s=L zi;3}Gmj77{{IF;QSn=C7wmT-kSAf%VCkwgc2x;szfaGNQO2I8~zS3EYQ5xIzCtEy< ziW$h8DQ}^4HBP8!t%;vc1G8>{l3t!XbFWT{;+9hf#?2U60v1~cUcjjD$eiyIMj2@| z+M-H3?>$OJ@+XQaApc0uiAfv4pi+2~Y^m)vyGIR(X0~x9I%eOX>g{98m+5xTX;G^i z-W@||G_n8*#35nLVj#+L^TKzfqrF`JD?_(}NNFC#N~L;UV~!xR&8eFYyJ6QWU$Ij* z^=;0=bbh+yb&DhXuqvL0zWrGX!E{%Ta;L&Yj`EPgisCrUML~CsyJPRT+;j*sEzpf+K-RxS=MR>rA&jPPG%h5>x$C>!qS}eKUVoxsk?PdE7qs zdU?K1U$0^`m-%6n{tVAw`vjjN+H~!7LI4XRJ0AOWwY{ZFy!$K2g(!vyZt%{zb$m(htx@yPhl4);i z~QcSnFZKBc40M2Y=zbYe6c1a%@6dqzM4{nTux?5azV|U%|AwQ6$}TTG~TCf z`}ss+X%B7#rIIb5`=J@~aHD(h%HR$)(y#+lu+JGh=xvaqeRw<0`>7+3I`rODb2$6c zG`-7GLtRs)6;mzi20+xS`$a>WIWlmJn*`a4c+Hz+jY7CvU zwUgtkMKECV6$$m2hv6=d4yq4rgie+JcM0THx$oaq8vBtm+%!h!4S96Eh6jMO1f8!$ zs@}Pit81$vsJRE}4PcIfKR&J>EnKSZpS`W6r)_^$*DGxILAZ`o2@c)6?ke0C#%umZ z3YGit3YiCVIyd$7WEGb-HI+WEMC*IM7>={A@TYjt^bZeOxjENW{^$_&IqJyCxj7D# zt*o8q*B<{9|6ls?za&Gd052C@;7?6rjw{Q|Oi+CdPQ?rjS!qPINr!NSZ6!S9=b3}2 z@W)Dvl;(oAOj{kjPw(K|#%^qbW*sYc*bnKtTVd=+Zmn4vBp6|l#5@_uu; zis%+T%kR3WKpzgtmfxf#!76yOG;$p|26JUGdm3SQ?(p>Fha2V_*gP23*(RBJW=VrqHAO^n^&vPGpj~#%E0kb3*H29U(DqJ24Y8Z-O__tdtoH*?8~lQo zC^+{0rvrEIDpeBx7e z2aR`fa;G!9qpAOC3_9_<8+z@JI?U0*&L^D>dHjC{5cBjn#%A^CgSuUuctJ1yY^&;? zQ|l=dbLne$39Ufi=xk9wAHTr({%s`v?|-hg-%hdV661kETdS$mW4n4|P!{<_YXkC3`-q?z3R#oB#Vu{G5^8lX^yr zulV(`CN9_PJePZ4P3k5c{5qb|ZbL=2RLCxri)6#UVNwbknL zG}nLn{GT8E?pw*OUm^KdNdBRa{}qydXxV>kUjGWoze4h_s{9uP z|Dxbu6#R>Ve^Kx+3jRgGzbN?sLxI%%Z5)fsoup$E_s0l)w*OnoZ`I#azxwp;{{h|b B9r6GG diff --git a/docs/index.rst b/docs/index.rst index efd0a8b78db4c..b0054ad670bcf 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -18,25 +18,10 @@ .. image:: img/pin_large.png :width: 100 -.. image:: img/incubator.jpg - :width: 150 -Apache Airflow (incubating) Documentation +Apache Airflow Documentation ========================================= -.. important:: - - **Disclaimer**: Apache Airflow is an effort undergoing incubation at The - Apache Software Foundation (ASF), sponsored by the Apache Incubator. - Incubation is required of all newly accepted projects until a further - review indicates that the infrastructure, communications, and - decision making process have stabilized in a manner consistent with - other successful ASF projects. While incubation status is not - necessarily a reflection of the completeness or stability of - the code, it does indicate that the project has yet to be fully - endorsed by the ASF. - - Airflow is a platform to programmatically author, schedule and monitor workflows. diff --git a/docs/plugins.rst b/docs/plugins.rst index 61b49577c7eed..715211281b818 100644 --- a/docs/plugins.rst +++ b/docs/plugins.rst @@ -159,7 +159,7 @@ definitions in Airflow. ml = MenuLink( category='Test Plugin', name='Test Menu Link', - url='https://airflow.incubator.apache.org/') + url='https://airflow.apache.org/') # Creating a flask appbuilder BaseView class TestAppBuilderBaseView(AppBuilderBaseView): diff --git a/docs/project.rst b/docs/project.rst index 6e4074f0918ba..1f37ca2a2b42b 100644 --- a/docs/project.rst +++ b/docs/project.rst @@ -50,7 +50,7 @@ Committers For the full list of contributors, take a look at `Airflow's Github Contributor page: -`_ +`_ Resources & links @@ -58,8 +58,8 @@ Resources & links * `Airflow's official documentation `_ * Mailing list (send emails to - ``dev-subscribe@airflow.incubator.apache.org`` and/or - ``commits-subscribe@airflow.incubator.apache.org`` + ``dev-subscribe@airflow.apache.org`` and/or + ``commits-subscribe@airflow.apache.org`` to subscribe to each) * `Issues on Apache's Jira `_ * `Gitter (chat) Channel `_ diff --git a/docs/scheduler.rst b/docs/scheduler.rst index 377fdffac2837..e30cd05a7e1d2 100644 --- a/docs/scheduler.rst +++ b/docs/scheduler.rst @@ -114,7 +114,7 @@ interval series. """ Code that goes along with the Airflow tutorial located at: - https://github.com/apache/incubator-airflow/blob/master/airflow/example_dags/tutorial.py + https://github.com/apache/airflow/blob/master/airflow/example_dags/tutorial.py """ from airflow import DAG from airflow.operators.bash_operator import BashOperator diff --git a/docs/tutorial.rst b/docs/tutorial.rst index 69670d7b025cf..9d1c94f5b9b71 100644 --- a/docs/tutorial.rst +++ b/docs/tutorial.rst @@ -32,7 +32,7 @@ complicated, a line by line explanation follows below. """ Code that goes along with the Airflow tutorial located at: - https://github.com/apache/incubator-airflow/blob/master/airflow/example_dags/tutorial.py + https://github.com/apache/airflow/blob/master/airflow/example_dags/tutorial.py """ from airflow import DAG from airflow.operators.bash_operator import BashOperator @@ -308,7 +308,7 @@ something like this: """ Code that goes along with the Airflow tutorial located at: - https://github.com/apache/incubator-airflow/blob/master/airflow/example_dags/tutorial.py + https://github.com/apache/airflow/blob/master/airflow/example_dags/tutorial.py """ from airflow import DAG from airflow.operators.bash_operator import BashOperator diff --git a/scripts/ci/flake8_diff.sh b/scripts/ci/flake8_diff.sh index 376be9bc0fb5d..fcbbf8a52ffb3 100755 --- a/scripts/ci/flake8_diff.sh +++ b/scripts/ci/flake8_diff.sh @@ -30,7 +30,7 @@ # This script is used in Travis to check that PRs do not add obvious # flake8 violations. It relies on two things: # - find common ancestor between branch and -# apache/incubator-airflow remote +# apache/airflow remote # - run flake8 --diff on the diff between the branch and the common # ancestor # @@ -44,7 +44,7 @@ set -e # pipefail is necessary to propagate exit codes set -o pipefail -PROJECT=apache/incubator-airflow +PROJECT=apache/airflow PROJECT_URL=https://github.com/$PROJECT.git # Find the remote with the project name (upstream in most cases) diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index d10cbdc728a9f..c8e6b19076d1d 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -119,7 +119,7 @@ data: expose_config = False # Set to true to turn on authentication: - # https://airflow.incubator.apache.org/security.html#web-authentication + # https://airflow.apache.org/security.html#web-authentication authenticate = False # Filter the list of dags by owner name (requires authentication to be enabled) @@ -179,7 +179,7 @@ data: worker_dags_folder = /tmp/dags delete_worker_pods = True dags_in_image = False - git_repo = https://github.com/apache/incubator-airflow.git + git_repo = https://github.com/apache/airflow.git git_branch = master git_subpath = airflow/example_dags/ git_user = diff --git a/setup.cfg b/setup.cfg index 881fe0107d9b2..f29267e269727 100644 --- a/setup.cfg +++ b/setup.cfg @@ -20,7 +20,7 @@ name = Airflow summary = Airflow is a system to programmatically author, schedule and monitor data pipelines. description-file = README.md author = Apache Airflow PMC -author-email = dev@airflow.incubator.apache.org +author-email = dev@airflow.apache.org license = Apache License, Version 2.0 [files] diff --git a/setup.py b/setup.py index f01a360dce59e..7947c27482af8 100644 --- a/setup.py +++ b/setup.py @@ -405,10 +405,10 @@ def do_setup(): 'Topic :: System :: Monitoring', ], author='Apache Software Foundation', - author_email='dev@airflow.incubator.apache.org', - url='http://airflow.incubator.apache.org/', + author_email='dev@airflow.apache.org', + url='http://airflow.apache.org/', download_url=( - 'https://dist.apache.org/repos/dist/release/incubator/airflow/' + version), + 'https://dist.apache.org/repos/dist/release/airflow/' + version), cmdclass={ 'test': Tox, 'extra_clean': CleanCommand, diff --git a/tests/plugins/test_plugin.py b/tests/plugins/test_plugin.py index 0770da120bc4c..39135c13069a5 100644 --- a/tests/plugins/test_plugin.py +++ b/tests/plugins/test_plugin.py @@ -94,7 +94,7 @@ def test(self): ml = MenuLink( category='Test Plugin', name="Test Menu Link", - url="https://airflow.incubator.apache.org/") + url="https://airflow.apache.org/") # Defining the plugin class diff --git a/tests/sensors/test_http_sensor.py b/tests/sensors/test_http_sensor.py index 5e55aa56e9c49..ffea7cadfe828 100644 --- a/tests/sensors/test_http_sensor.py +++ b/tests/sensors/test_http_sensor.py @@ -140,7 +140,7 @@ class FakeSession(object): def __init__(self): self.response = requests.Response() self.response.status_code = 200 - self.response._content = 'apache/incubator-airflow'.encode('ascii', 'ignore') + self.response._content = 'apache/airflow'.encode('ascii', 'ignore') def send(self, request, **kwargs): return self.response @@ -178,7 +178,7 @@ def test_get_response_check(self): method='GET', endpoint='/search', data={"client": "ubuntu", "q": "airflow"}, - response_check=lambda response: ("apache/incubator-airflow" in response.text), + response_check=lambda response: ("apache/airflow" in response.text), headers={}, dag=self.dag) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) @@ -192,7 +192,7 @@ def test_sensor(self): request_params={"client": "ubuntu", "q": "airflow", 'date': '{{ds}}'}, headers={}, response_check=lambda response: ( - "apache/incubator-airflow/" + DEFAULT_DATE.strftime('%Y-%m-%d') + "apache/airflow/" + DEFAULT_DATE.strftime('%Y-%m-%d') in response.text), poke_interval=5, timeout=15, From 21263ea90375aa6ce02de4f6feb92bc02e9d35ca Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sun, 2 Dec 2018 11:08:26 +0000 Subject: [PATCH 342/808] [AIRFLOW-XXX] Add missing GCP operators to Docs (#4260) --- docs/integration.rst | 33 +++++++++++++++++++++------------ 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/docs/integration.rst b/docs/integration.rst index 284c372d47454..a04d71d825cb6 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -557,10 +557,11 @@ BigQuery Operators - :ref:`BigQueryValueCheckOperator` : Performs a simple value check using SQL code. - :ref:`BigQueryIntervalCheckOperator` : Checks that the values of metrics given as SQL expressions are within a certain tolerance of the ones from days_back before. - :ref:`BigQueryGetDataOperator` : Fetches the data from a BigQuery table and returns data in a python list +- :ref:`BigQueryCreateEmptyDatasetOperator` : Creates an empty BigQuery dataset. - :ref:`BigQueryCreateEmptyTableOperator` : Creates a new, empty table in the specified BigQuery dataset optionally with schema. - :ref:`BigQueryCreateExternalTableOperator` : Creates a new, external table in the dataset with the data in Google Cloud Storage. - :ref:`BigQueryDeleteDatasetOperator` : Deletes an existing BigQuery dataset. -- :ref:`BigQueryCreateEmptyDatasetOperator` : Creates an empty BigQuery dataset. +- :ref:`BigQueryTableDeleteOperator` : Deletes an existing BigQuery table. - :ref:`BigQueryOperator` : Executes BigQuery SQL queries in a specific BigQuery database. - :ref:`BigQueryToBigQueryOperator` : Copy a BigQuery table to another BigQuery table. - :ref:`BigQueryToCloudStorageOperator` : Transfers a BigQuery table to a Google Cloud Storage bucket @@ -608,13 +609,6 @@ BigQueryCreateExternalTableOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateExternalTableOperator -.. _BigQueryDeleteDatasetOperator: - -BigQueryDeleteDatasetOperator -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator - .. _BigQueryCreateEmptyDatasetOperator: BigQueryCreateEmptyDatasetOperator @@ -622,12 +616,12 @@ BigQueryCreateEmptyDatasetOperator .. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryCreateEmptyDatasetOperator -.. _BigQueryOperator: +.. _BigQueryDeleteDatasetOperator: -BigQueryOperator -^^^^^^^^^^^^^^^^ +BigQueryDeleteDatasetOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryOperator +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryDeleteDatasetOperator .. _BigQueryTableDeleteOperator: @@ -636,6 +630,13 @@ BigQueryTableDeleteOperator .. autoclass:: airflow.contrib.operators.bigquery_table_delete_operator.BigQueryTableDeleteOperator +.. _BigQueryOperator: + +BigQueryOperator +^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.bigquery_operator.BigQueryOperator + .. _BigQueryToBigQueryOperator: BigQueryToBigQueryOperator @@ -1242,6 +1243,7 @@ Storage Operators - :ref:`GoogleCloudStorageObjectCreateAclEntryOperator` : Creates a new ACL entry on the specified object. - :ref:`GoogleCloudStorageToBigQueryOperator` : Loads files from Google cloud storage into BigQuery. - :ref:`GoogleCloudStorageToGoogleCloudStorageOperator` : Copies objects from a bucket to another, with renaming if requested. +- :ref:`MySqlToGoogleCloudStorageOperator`: Copy data from any MySQL Database to Google cloud storage in JSON format. .. _FileToGoogleCloudStorageOperator: @@ -1299,6 +1301,13 @@ GoogleCloudStorageToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageOperator +.. _MySqlToGoogleCloudStorageOperator: + +MySqlToGoogleCloudStorageOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.mysql_to_gcs.MySqlToGoogleCloudStorageOperator + GoogleCloudStorageHook """""""""""""""""""""" From 2772a9d45b569e41e213664d063df17097290e5c Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Sun, 6 Jan 2019 14:35:31 -0500 Subject: [PATCH 343/808] [AIRFLOW-3531] Add gcs to gcs transfer operator. (#4331) --- airflow/contrib/hooks/gcp_transfer_hook.py | 6 +- .../operators/gcs_to_gcs_transfer_operator.py | 127 +++++++++++++++++ .../operators/s3_to_gcs_transfer_operator.py | 8 +- docs/code.rst | 1 + docs/integration.rst | 8 ++ .../test_gcs_to_gcs_transfer_operator.py | 131 ++++++++++++++++++ .../test_s3_to_gcs_transfer_operator.py | 2 + 7 files changed, 276 insertions(+), 7 deletions(-) create mode 100644 airflow/contrib/operators/gcs_to_gcs_transfer_operator.py create mode 100644 tests/contrib/operators/test_gcs_to_gcs_transfer_operator.py diff --git a/airflow/contrib/hooks/gcp_transfer_hook.py b/airflow/contrib/hooks/gcp_transfer_hook.py index 906dba786fb9c..6966ec3ae244e 100644 --- a/airflow/contrib/hooks/gcp_transfer_hook.py +++ b/airflow/contrib/hooks/gcp_transfer_hook.py @@ -26,7 +26,7 @@ from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook # Time to sleep between active checks of the operation results -TIME_TO_SLEEP_IN_SECONDS = 1 +TIME_TO_SLEEP_IN_SECONDS = 10 # noinspection PyAbstractClass @@ -56,10 +56,10 @@ def get_conn(self): http=http_authorized, cache_discovery=False) return self._conn - def create_transfer_job(self, project_id, description, schedule, transfer_spec): + def create_transfer_job(self, description, schedule, transfer_spec, project_id=None): transfer_job = { 'status': 'ENABLED', - 'projectId': project_id, + 'projectId': project_id or self.project_id, 'description': description, 'transferSpec': transfer_spec, 'schedule': schedule or self._schedule_once_now(), diff --git a/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py b/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py new file mode 100644 index 0000000000000..410d65821d205 --- /dev/null +++ b/airflow/contrib/operators/gcs_to_gcs_transfer_operator.py @@ -0,0 +1,127 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.models import BaseOperator +from airflow.contrib.hooks.gcp_transfer_hook import GCPTransferServiceHook +from airflow.utils.decorators import apply_defaults + + +class GoogleCloudStorageToGoogleCloudStorageTransferOperator(BaseOperator): + """ + Copies objects from a bucket to another using the GCP Storage Transfer + Service. + + :param source_bucket: The source Google cloud storage bucket where the + object is. (templated) + :type source_bucket: str + :param destination_bucket: The destination Google cloud storage bucket + where the object should be. (templated) + :type destination_bucket: str + :param project_id: The ID of the Google Cloud Platform Console project that + owns the job + :type project_id: str + :param gcp_conn_id: Optional connection ID to use when connecting to Google Cloud + Storage. + :type gcp_conn_id: str + :param delegate_to: The account to impersonate, if any. + For this to work, the service account making the request must have + domain-wide delegation enabled. + :type delegate_to: str + :param description: Optional transfer service job description + :type description: str + :param schedule: Optional transfer service schedule; see + https://cloud.google.com/storage-transfer/docs/reference/rest/v1/transferJobs. + If not set, run transfer job once as soon as the operator runs + :type schedule: dict + :param object_conditions: Optional transfer service object conditions; see + https://cloud.google.com/storage-transfer/docs/reference/rest/v1/TransferSpec#ObjectConditions + :type object_conditions: dict + :param transfer_options: Optional transfer service transfer options; see + https://cloud.google.com/storage-transfer/docs/reference/rest/v1/TransferSpec#TransferOptions + :type transfer_options: dict + :param wait: Wait for transfer to finish; defaults to `True` + :type wait: bool + + **Example**: + + .. code-block:: python + + gcs_to_gcs_transfer_op = GoogleCloudStorageToGoogleCloudStorageTransferOperator( + task_id='gcs_to_gcs_transfer_example', + source_bucket='my-source-bucket', + destination_bucket='my-destination-bucket', + project_id='my-gcp-project', + dag=my_dag) + """ + + template_fields = ('source_bucket', 'destination_bucket', 'description', 'object_conditions') + ui_color = '#e09411' + + @apply_defaults + def __init__(self, + source_bucket, + destination_bucket, + project_id=None, + gcp_conn_id='google_cloud_default', + delegate_to=None, + description=None, + schedule=None, + object_conditions=None, + transfer_options=None, + wait=True, + *args, + **kwargs): + + super(GoogleCloudStorageToGoogleCloudStorageTransferOperator, self).__init__( + *args, + **kwargs) + self.source_bucket = source_bucket + self.destination_bucket = destination_bucket + self.project_id = project_id + self.gcp_conn_id = gcp_conn_id + self.delegate_to = delegate_to + self.description = description + self.schedule = schedule + self.object_conditions = object_conditions or {} + self.transfer_options = transfer_options or {} + self.wait = wait + + def execute(self, context): + transfer_hook = GCPTransferServiceHook( + gcp_conn_id=self.gcp_conn_id, + delegate_to=self.delegate_to) + + job = transfer_hook.create_transfer_job( + project_id=self.project_id, + description=self.description, + schedule=self.schedule, + transfer_spec={ + 'gcsDataSource': { + 'bucketName': self.source_bucket, + }, + 'gcsDataSink': { + 'bucketName': self.destination_bucket, + }, + 'objectConditions': self.object_conditions, + 'transferOptions': self.transfer_options, + } + ) + + if self.wait: + transfer_hook.wait_for_transfer_job(job) diff --git a/airflow/contrib/operators/s3_to_gcs_transfer_operator.py b/airflow/contrib/operators/s3_to_gcs_transfer_operator.py index b0cf2ae515572..c46a9460e701c 100644 --- a/airflow/contrib/operators/s3_to_gcs_transfer_operator.py +++ b/airflow/contrib/operators/s3_to_gcs_transfer_operator.py @@ -33,7 +33,7 @@ class S3ToGoogleCloudStorageTransferOperator(BaseOperator): :param gcs_bucket: The destination Google Cloud Storage bucket where you want to store the files. (templated) :type gcs_bucket: str - :param project_id: The ID of the Google Cloud Platform Console project that + :param project_id: Optional ID of the Google Cloud Platform Console project that owns the job :type project_id: str :param aws_conn_id: The source S3 connection @@ -51,10 +51,10 @@ class S3ToGoogleCloudStorageTransferOperator(BaseOperator): https://cloud.google.com/storage-transfer/docs/reference/rest/v1/transferJobs. If not set, run transfer job once as soon as the operator runs :type schedule: dict - :param object_conditions: Transfer service object conditions; see + :param object_conditions: Optional transfer service object conditions; see https://cloud.google.com/storage-transfer/docs/reference/rest/v1/TransferSpec :type object_conditions: dict - :param transfer_options: Transfer service transfer options; see + :param transfer_options: Optional transfer service transfer options; see https://cloud.google.com/storage-transfer/docs/reference/rest/v1/TransferSpec :type transfer_options: dict :param wait: Wait for transfer to finish @@ -79,7 +79,7 @@ class S3ToGoogleCloudStorageTransferOperator(BaseOperator): def __init__(self, s3_bucket, gcs_bucket, - project_id, + project_id=None, aws_conn_id='aws_default', gcp_conn_id='google_cloud_default', delegate_to=None, diff --git a/docs/code.rst b/docs/code.rst index 6f82166b10ff9..1850bdb14b24f 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -179,6 +179,7 @@ Operators .. autoclass:: airflow.contrib.operators.gcs_operator.GoogleCloudStorageCreateBucketOperator .. autoclass:: airflow.contrib.operators.gcs_to_bq.GoogleCloudStorageToBigQueryOperator .. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageOperator +.. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageTransferOperator .. autoclass:: airflow.contrib.operators.gcs_to_s3.GoogleCloudStorageToS3Operator .. autoclass:: airflow.contrib.operators.hipchat_operator.HipChatAPIOperator .. autoclass:: airflow.contrib.operators.hipchat_operator.HipChatAPISendRoomNotificationOperator diff --git a/docs/integration.rst b/docs/integration.rst index a04d71d825cb6..c12e9ac57e62d 100644 --- a/docs/integration.rst +++ b/docs/integration.rst @@ -1243,6 +1243,7 @@ Storage Operators - :ref:`GoogleCloudStorageObjectCreateAclEntryOperator` : Creates a new ACL entry on the specified object. - :ref:`GoogleCloudStorageToBigQueryOperator` : Loads files from Google cloud storage into BigQuery. - :ref:`GoogleCloudStorageToGoogleCloudStorageOperator` : Copies objects from a bucket to another, with renaming if requested. +- :ref:`GoogleCloudStorageToGoogleCloudStorageTransferOperator` : Copies objects from a bucket to another using Google Transfer service. - :ref:`MySqlToGoogleCloudStorageOperator`: Copy data from any MySQL Database to Google cloud storage in JSON format. .. _FileToGoogleCloudStorageOperator: @@ -1301,6 +1302,13 @@ GoogleCloudStorageToGoogleCloudStorageOperator .. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageOperator +.. _GoogleCloudStorageToGoogleCloudStorageTransferOperator: + +GoogleCloudStorageToGoogleCloudStorageTransferOperator +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +.. autoclass:: airflow.contrib.operators.gcs_to_gcs.GoogleCloudStorageToGoogleCloudStorageTransferOperator + .. _MySqlToGoogleCloudStorageOperator: MySqlToGoogleCloudStorageOperator diff --git a/tests/contrib/operators/test_gcs_to_gcs_transfer_operator.py b/tests/contrib/operators/test_gcs_to_gcs_transfer_operator.py new file mode 100644 index 0000000000000..8c0cd4ebfcaa5 --- /dev/null +++ b/tests/contrib/operators/test_gcs_to_gcs_transfer_operator.py @@ -0,0 +1,131 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from airflow.contrib.operators.gcs_to_gcs_transfer_operator import \ + GoogleCloudStorageToGoogleCloudStorageTransferOperator + +try: + from unittest import mock +except ImportError: + try: + import mock + except ImportError: + mock = None + + +TASK_ID = 'test-gcs-gcs-transfer-operator' +SOURCE_BUCKET = 'test-source-bucket' +DESTINATION_BUCKET = 'test-destination-bucket' +PROJECT_ID = 'test-project' +DESCRIPTION = 'test-description' +SCHEDULE = { + 'scheduleStartDate': {'month': 10, 'day': 1, 'year': 2018}, + 'scheduleEndDate': {'month': 10, 'day': 31, 'year': 2018}, +} + + +class GoogleCloudStorageToGoogleCloudStorageTransferOperatorTest(unittest.TestCase): + def test_constructor(self): + """Test GoogleCloudStorageToGoogleCloudStorageTransferOperator instance is properly initialized.""" + + operator = GoogleCloudStorageToGoogleCloudStorageTransferOperator( + task_id=TASK_ID, + source_bucket=SOURCE_BUCKET, + destination_bucket=DESTINATION_BUCKET, + project_id=PROJECT_ID, + description=DESCRIPTION, + schedule=SCHEDULE, + ) + + self.assertEqual(operator.task_id, TASK_ID) + self.assertEqual(operator.source_bucket, SOURCE_BUCKET) + self.assertEqual(operator.destination_bucket, DESTINATION_BUCKET) + self.assertEqual(operator.project_id, PROJECT_ID) + self.assertEqual(operator.description, DESCRIPTION) + self.assertEqual(operator.schedule, SCHEDULE) + + @mock.patch('airflow.contrib.operators.gcs_to_gcs_transfer_operator.GCPTransferServiceHook') + def test_execute(self, mock_transfer_hook): + """Test the execute function when the run is successful.""" + + operator = GoogleCloudStorageToGoogleCloudStorageTransferOperator( + task_id=TASK_ID, + source_bucket=SOURCE_BUCKET, + destination_bucket=DESTINATION_BUCKET, + project_id=PROJECT_ID, + description=DESCRIPTION, + schedule=SCHEDULE, + ) + + operator.execute(None) + + mock_transfer_hook.return_value.create_transfer_job.assert_called_once_with( + project_id=PROJECT_ID, + description=DESCRIPTION, + schedule=SCHEDULE, + transfer_spec={ + 'gcsDataSource': { + 'bucketName': SOURCE_BUCKET, + }, + 'gcsDataSink': { + 'bucketName': DESTINATION_BUCKET, + }, + 'objectConditions': {}, + 'transferOptions': {} + } + ) + + mock_transfer_hook.return_value.wait_for_transfer_job.assert_called_once_with( + mock_transfer_hook.return_value.create_transfer_job.return_value + ) + + @mock.patch('airflow.contrib.operators.gcs_to_gcs_transfer_operator.GCPTransferServiceHook') + def test_execute_skip_wait(self, mock_transfer_hook): + """Test the execute function when the run is successful.""" + + operator = GoogleCloudStorageToGoogleCloudStorageTransferOperator( + task_id=TASK_ID, + source_bucket=SOURCE_BUCKET, + destination_bucket=DESTINATION_BUCKET, + project_id=PROJECT_ID, + description=DESCRIPTION, + wait=False, + ) + + operator.execute(None) + + mock_transfer_hook.return_value.create_transfer_job.assert_called_once_with( + project_id=PROJECT_ID, + description=DESCRIPTION, + schedule=None, + transfer_spec={ + 'gcsDataSource': { + 'bucketName': SOURCE_BUCKET, + }, + 'gcsDataSink': { + 'bucketName': DESTINATION_BUCKET, + }, + 'objectConditions': {}, + 'transferOptions': {} + } + ) + + assert not mock_transfer_hook.return_value.wait_for_transfer_job.called diff --git a/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py b/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py index 2bf51c0707577..0825364884a78 100644 --- a/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py +++ b/tests/contrib/operators/test_s3_to_gcs_transfer_operator.py @@ -59,6 +59,7 @@ def test_constructor(self): gcs_bucket=GCS_BUCKET, project_id=PROJECT_ID, description=DESCRIPTION, + schedule=SCHEDULE, ) self.assertEqual(operator.task_id, TASK_ID) @@ -66,6 +67,7 @@ def test_constructor(self): self.assertEqual(operator.gcs_bucket, GCS_BUCKET) self.assertEqual(operator.project_id, PROJECT_ID) self.assertEqual(operator.description, DESCRIPTION) + self.assertEqual(operator.schedule, SCHEDULE) @mock.patch('airflow.contrib.operators.s3_to_gcs_transfer_operator.GCPTransferServiceHook') @mock.patch('airflow.contrib.operators.s3_to_gcs_transfer_operator.S3Hook') From d36dd170b3b1ee2dbcbd1729f28034a5f8653e56 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sun, 6 Jan 2019 21:40:54 +0000 Subject: [PATCH 344/808] [AIRFLOW-3531] Fix test for GCS to GCS Transfer Hook (#4452) --- tests/contrib/hooks/test_gcp_transfer_hook.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/contrib/hooks/test_gcp_transfer_hook.py b/tests/contrib/hooks/test_gcp_transfer_hook.py index 1107775f9170c..e42550979d836 100644 --- a/tests/contrib/hooks/test_gcp_transfer_hook.py +++ b/tests/contrib/hooks/test_gcp_transfer_hook.py @@ -54,7 +54,10 @@ def test_create_transfer_job(self): 'gcsDataSink': {'bucketName': 'test-gcs-bucket'} } self.transfer_hook.create_transfer_job( - 'test-project', 'test-description', None, transfer_spec) + project_id='test-project', + description='test-description', + schedule=None, + transfer_spec=transfer_spec) mock_create.assert_called_once_with(body={ 'status': 'ENABLED', 'projectId': 'test-project', @@ -90,7 +93,10 @@ def test_create_transfer_job_custom_schedule(self): 'gcsDataSink': {'bucketName': 'test-gcs-bucket'} } self.transfer_hook.create_transfer_job( - 'test-project', 'test-description', schedule, transfer_spec) + project_id='test-project', + description='test-description', + schedule=schedule, + transfer_spec=transfer_spec) mock_create.assert_called_once_with(body={ 'status': 'ENABLED', 'projectId': 'test-project', From f71a86d66589f0a56953cdaffa45d889855e1d87 Mon Sep 17 00:00:00 2001 From: Dana Ma Date: Mon, 7 Jan 2019 08:51:01 +1100 Subject: [PATCH 345/808] [AIRFLOW-3610] Add region param for EMR jobflow creation (#4418) --- airflow/contrib/hooks/emr_hook.py | 5 +++-- airflow/contrib/operators/emr_create_job_flow_operator.py | 4 +++- tests/contrib/hooks/test_emr_hook.py | 2 +- tests/contrib/operators/test_emr_create_job_flow_operator.py | 2 ++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/airflow/contrib/hooks/emr_hook.py b/airflow/contrib/hooks/emr_hook.py index fcdf4ac84852e..7571d0988f697 100644 --- a/airflow/contrib/hooks/emr_hook.py +++ b/airflow/contrib/hooks/emr_hook.py @@ -27,12 +27,13 @@ class EmrHook(AwsHook): create_job_flow method. """ - def __init__(self, emr_conn_id=None, *args, **kwargs): + def __init__(self, emr_conn_id=None, region_name=None, *args, **kwargs): self.emr_conn_id = emr_conn_id + self.region_name = region_name super(EmrHook, self).__init__(*args, **kwargs) def get_conn(self): - self.conn = self.get_client_type('emr') + self.conn = self.get_client_type('emr', self.region_name) return self.conn def create_job_flow(self, job_flow_overrides): diff --git a/airflow/contrib/operators/emr_create_job_flow_operator.py b/airflow/contrib/operators/emr_create_job_flow_operator.py index 89be12f0657e4..bdf10b1ff654c 100644 --- a/airflow/contrib/operators/emr_create_job_flow_operator.py +++ b/airflow/contrib/operators/emr_create_job_flow_operator.py @@ -46,6 +46,7 @@ def __init__( aws_conn_id='s3_default', emr_conn_id='emr_default', job_flow_overrides=None, + region_name=None, *args, **kwargs): super(EmrCreateJobFlowOperator, self).__init__(*args, **kwargs) self.aws_conn_id = aws_conn_id @@ -53,9 +54,10 @@ def __init__( if job_flow_overrides is None: job_flow_overrides = {} self.job_flow_overrides = job_flow_overrides + self.region_name = region_name def execute(self, context): - emr = EmrHook(aws_conn_id=self.aws_conn_id, emr_conn_id=self.emr_conn_id) + emr = EmrHook(aws_conn_id=self.aws_conn_id, emr_conn_id=self.emr_conn_id, region_name=self.region_name) self.log.info( 'Creating JobFlow using aws-conn-id: %s, emr-conn-id: %s', diff --git a/tests/contrib/hooks/test_emr_hook.py b/tests/contrib/hooks/test_emr_hook.py index 814b4b317feae..9116b96ae3ed4 100644 --- a/tests/contrib/hooks/test_emr_hook.py +++ b/tests/contrib/hooks/test_emr_hook.py @@ -39,7 +39,7 @@ def setUp(self): @mock_emr def test_get_conn_returns_a_boto3_connection(self): - hook = EmrHook(aws_conn_id='aws_default') + hook = EmrHook(aws_conn_id='aws_default', region_name='ap-southeast-2') self.assertIsNotNone(hook.get_conn().list_clusters()) @mock_emr diff --git a/tests/contrib/operators/test_emr_create_job_flow_operator.py b/tests/contrib/operators/test_emr_create_job_flow_operator.py index 7dae30d43a809..18b07b23b4a6f 100644 --- a/tests/contrib/operators/test_emr_create_job_flow_operator.py +++ b/tests/contrib/operators/test_emr_create_job_flow_operator.py @@ -71,12 +71,14 @@ def setUp(self): aws_conn_id='aws_default', emr_conn_id='emr_default', job_flow_overrides=self._config, + region_name='ap-southeast-2', dag=DAG('test_dag_id', default_args=args) ) def test_init(self): self.assertEqual(self.operator.aws_conn_id, 'aws_default') self.assertEqual(self.operator.emr_conn_id, 'emr_default') + self.assertEqual(self.operator.region_name, 'ap-southeast-2') def test_render_template(self): ti = TaskInstance(self.operator, DEFAULT_DATE) From 42386f0e456f2409a68094634ae6766ec72ac970 Mon Sep 17 00:00:00 2001 From: Raja Gangopadhya Date: Sun, 6 Jan 2019 13:59:55 -0800 Subject: [PATCH 346/808] [AIRFLOW-2082] Resolve a bug in adding password_auth to api as auth method (#4343) --- airflow/contrib/auth/backends/password_auth.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/airflow/contrib/auth/backends/password_auth.py b/airflow/contrib/auth/backends/password_auth.py index dcdb1d1225d42..0bc04fcea8730 100644 --- a/airflow/contrib/auth/backends/password_auth.py +++ b/airflow/contrib/auth/backends/password_auth.py @@ -48,6 +48,9 @@ PY3 = version_info[0] == 3 +client_auth = None + + class AuthenticationError(Exception): pass From 1e0b95fa2c6573a81f8e457d09d32957967fa2ba Mon Sep 17 00:00:00 2001 From: Tao Feng Date: Sun, 6 Jan 2019 20:45:49 -0800 Subject: [PATCH 347/808] [AIRFLOW-XXX] Fix a flake8 error to unblock CI (#4453) --- airflow/contrib/operators/emr_create_job_flow_operator.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow/contrib/operators/emr_create_job_flow_operator.py b/airflow/contrib/operators/emr_create_job_flow_operator.py index bdf10b1ff654c..5a39713b8a931 100644 --- a/airflow/contrib/operators/emr_create_job_flow_operator.py +++ b/airflow/contrib/operators/emr_create_job_flow_operator.py @@ -57,7 +57,9 @@ def __init__( self.region_name = region_name def execute(self, context): - emr = EmrHook(aws_conn_id=self.aws_conn_id, emr_conn_id=self.emr_conn_id, region_name=self.region_name) + emr = EmrHook(aws_conn_id=self.aws_conn_id, + emr_conn_id=self.emr_conn_id, + region_name=self.region_name) self.log.info( 'Creating JobFlow using aws-conn-id: %s, emr-conn-id: %s', From 2a315c70831bebaf038ff5cf076fd526830c0430 Mon Sep 17 00:00:00 2001 From: Kevin Pullin Date: Mon, 7 Jan 2019 12:46:05 -0800 Subject: [PATCH 348/808] [AIRFLOW-3402] Support global k8s affinity and toleration configs (#4247) --- airflow/config_templates/default_airflow.cfg | 10 +++ .../example_kubernetes_executor.py | 32 ++++++- .../contrib/executors/kubernetes_executor.py | 23 ++++- airflow/contrib/kubernetes/pod.py | 4 + .../kubernetes/worker_configuration.py | 8 +- .../operators/kubernetes_pod_operator.py | 6 +- scripts/ci/kubernetes/kube/configmaps.yaml | 4 + .../executors/test_kubernetes_executor.py | 88 ++++++++++++++++++- 8 files changed, 162 insertions(+), 13 deletions(-) diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index c8aa4061e7271..a72604a536c01 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -630,6 +630,16 @@ gcp_service_account_keys = # It will raise an exception if called from a process not running in a kubernetes environment. in_cluster = True +# Affinity configuration as a single line formatted JSON object. +# See the affinity model for top-level key names (e.g. `nodeAffinity`, etc.): +# https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.12/#affinity-v1-core +affinity = + +# A list of toleration objects as a single line formatted JSON array +# See: +# https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.12/#toleration-v1-core +tolerations = + [kubernetes_node_selectors] # The Key-value pairs to be given to worker pods. # The worker pods will be scheduled to the nodes of the specified key-value pairs. diff --git a/airflow/contrib/example_dags/example_kubernetes_executor.py b/airflow/contrib/example_dags/example_kubernetes_executor.py index 1d9bb7304318b..d03e255ab3287 100644 --- a/airflow/contrib/example_dags/example_kubernetes_executor.py +++ b/airflow/contrib/example_dags/example_kubernetes_executor.py @@ -32,6 +32,31 @@ schedule_interval=None ) +affinity = { + 'podAntiAffinity': { + 'requiredDuringSchedulingIgnoredDuringExecution': [ + { + 'topologyKey': 'kubernetes.io/hostname', + 'labelSelector': { + 'matchExpressions': [ + { + 'key': 'app', + 'operator': 'In', + 'values': ['airflow'] + } + ] + } + } + ] + } +} + +tolerations = [{ + 'key': 'dedicated', + 'operator': 'Equal', + 'value': 'airflow' +}] + def print_stuff(): print("stuff!") @@ -59,11 +84,14 @@ def use_zip_binary(): executor_config={"KubernetesExecutor": {"image": "airflow/ci_zip:latest"}} ) -# Limit resources on this operator/task +# Limit resources on this operator/task with node affinity & tolerations three_task = PythonOperator( task_id="three_task", python_callable=print_stuff, dag=dag, executor_config={ - "KubernetesExecutor": {"request_memory": "128Mi", "limit_memory": "128Mi"}} + "KubernetesExecutor": {"request_memory": "128Mi", + "limit_memory": "128Mi", + "tolerations": tolerations, + "affinity": affinity}} ) start_task.set_downstream([one_task, two_task, three_task]) diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index dd9cd3ec53b15..e06a5f47e1ed0 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -16,6 +16,7 @@ # under the License. import base64 +import json import multiprocessing from queue import Queue from dateutil import parser @@ -40,7 +41,7 @@ class KubernetesExecutorConfig: def __init__(self, image=None, image_pull_policy=None, request_memory=None, request_cpu=None, limit_memory=None, limit_cpu=None, gcp_service_account_key=None, node_selectors=None, affinity=None, - annotations=None, volumes=None, volume_mounts=None): + annotations=None, volumes=None, volume_mounts=None, tolerations=None): self.image = image self.image_pull_policy = image_pull_policy self.request_memory = request_memory @@ -53,16 +54,18 @@ def __init__(self, image=None, image_pull_policy=None, request_memory=None, self.annotations = annotations self.volumes = volumes self.volume_mounts = volume_mounts + self.tolerations = tolerations def __repr__(self): return "{}(image={}, image_pull_policy={}, request_memory={}, request_cpu={}, " \ "limit_memory={}, limit_cpu={}, gcp_service_account_key={}, " \ "node_selectors={}, affinity={}, annotations={}, volumes={}, " \ - "volume_mounts={})" \ + "volume_mounts={}, tolerations={})" \ .format(KubernetesExecutorConfig.__name__, self.image, self.image_pull_policy, self.request_memory, self.request_cpu, self.limit_memory, self.limit_cpu, self.gcp_service_account_key, self.node_selectors, - self.affinity, self.annotations, self.volumes, self.volume_mounts) + self.affinity, self.annotations, self.volumes, self.volume_mounts, + self.tolerations) @staticmethod def from_dict(obj): @@ -88,6 +91,7 @@ def from_dict(obj): annotations=namespaced.get('annotations', {}), volumes=namespaced.get('volumes', []), volume_mounts=namespaced.get('volume_mounts', []), + tolerations=namespaced.get('tolerations', None), ) def as_dict(self): @@ -104,6 +108,7 @@ def as_dict(self): 'annotations': self.annotations, 'volumes': self.volumes, 'volume_mounts': self.volume_mounts, + 'tolerations': self.tolerations, } @@ -205,6 +210,18 @@ def __init__(self): # configmap self.airflow_configmap = conf.get(self.kubernetes_section, 'airflow_configmap') + affinity_json = conf.get(self.kubernetes_section, 'affinity') + if affinity_json: + self.kube_affinity = json.loads(affinity_json) + else: + self.kube_affinity = None + + tolerations_json = conf.get(self.kubernetes_section, 'tolerations') + if tolerations_json: + self.kube_tolerations = json.loads(tolerations_json) + else: + self.kube_tolerations = None + self._validate() def _validate(self): diff --git a/airflow/contrib/kubernetes/pod.py b/airflow/contrib/kubernetes/pod.py index bad5caa738e1b..6d2977592598a 100644 --- a/airflow/contrib/kubernetes/pod.py +++ b/airflow/contrib/kubernetes/pod.py @@ -60,6 +60,10 @@ class Pod: :type image_pull_secrets: str :param affinity: A dict containing a group of affinity scheduling rules :type affinity: dict + :param hostnetwork: If True enable host networking on the pod + :type hostnetwork: bool + :param tolerations: A list of kubernetes tolerations + :type tolerations: list """ def __init__( self, diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index d83be81c1f704..c2e7768baae8f 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -205,10 +205,13 @@ def make_pod(self, namespace, worker_uuid, pod_id, dag_id, task_id, execution_da limit_cpu=kube_executor_config.limit_cpu ) gcp_sa_key = kube_executor_config.gcp_service_account_key - annotations = kube_executor_config.annotations.copy() + annotations = dict(kube_executor_config.annotations) if gcp_sa_key: annotations['iam.cloud.google.com/service-account'] = gcp_sa_key + affinity = kube_executor_config.affinity or self.kube_config.kube_affinity + tolerations = kube_executor_config.tolerations or self.kube_config.kube_tolerations + return Pod( namespace=namespace, name=pod_id, @@ -234,5 +237,6 @@ def make_pod(self, namespace, worker_uuid, pod_id, dag_id, task_id, execution_da annotations=annotations, node_selectors=(kube_executor_config.node_selectors or self.kube_config.kube_node_selectors), - affinity=kube_executor_config.affinity + affinity=affinity, + tolerations=tolerations ) diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py index a29b61998d181..4494754f970f8 100644 --- a/airflow/contrib/operators/kubernetes_pod_operator.py +++ b/airflow/contrib/operators/kubernetes_pod_operator.py @@ -84,8 +84,10 @@ class KubernetesPodOperator(BaseOperator): /airflow/xcom/return.json in the container will also be pushed to an XCom when the container completes. :type xcom_push: bool - :param tolerations: Kubernetes tolerations - :type list of tolerations + :param hostnetwork: If True enable host networking on the pod + :type hostnetwork: bool + :param tolerations: A list of kubernetes tolerations + :type tolerations: list tolerations """ template_fields = ('cmds', 'arguments', 'env_vars', 'config_file') diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index c8e6b19076d1d..b5fa3e5f639ee 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -192,6 +192,10 @@ data: namespace = default gcp_service_account_keys = + # Example affinity and toleration definitions. + affinity = {"nodeAffinity":{"requiredDuringSchedulingIgnoredDuringExecution":{"nodeSelectorTerms":[{"matchExpressions":[{"key":"kubernetes.io/hostname","operator":"NotIn","values":["4e5e6a99-e28a-450b-bba9-e0124853de9b"]}]}]}}} + tolerations = [{ "key": "dedicated", "operator": "Equal", "value": "airflow", "effect": "NoSchedule" }, { "key": "prod", "operator": "Exists" }] + # For cloning DAGs from git repositories into volumes: https://github.com/kubernetes/git-sync git_sync_container_repository = gcr.io/google-containers/git-sync-amd64 git_sync_container_tag = v2.0.5 diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index f93a9d81e142d..1b5c4c015d963 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -14,6 +14,7 @@ # import unittest +import uuid import mock import re import string @@ -22,6 +23,7 @@ try: from airflow.contrib.executors.kubernetes_executor import AirflowKubernetesScheduler + from airflow.contrib.executors.kubernetes_executor import KubernetesExecutorConfig from airflow.contrib.kubernetes.worker_configuration import WorkerConfiguration except ImportError: AirflowKubernetesScheduler = None @@ -81,13 +83,42 @@ class TestKubernetesWorkerConfiguration(unittest.TestCase): Tests that if dags_volume_subpath/logs_volume_subpath configuration options are passed to worker pod config """ + + affinity_config = { + 'podAntiAffinity': { + 'requiredDuringSchedulingIgnoredDuringExecution': [ + { + 'topologyKey': 'kubernetes.io/hostname', + 'labelSelector': { + 'matchExpressions': [ + { + 'key': 'app', + 'operator': 'In', + 'values': ['airflow'] + } + ] + } + } + ] + } + } + + tolerations_config = [ + { + 'key': 'dedicated', + 'operator': 'Equal', + 'value': 'airflow' + }, + { + 'key': 'prod', + 'operator': 'Exists' + } + ] + def setUp(self): if AirflowKubernetesScheduler is None: self.skipTest("kubernetes python package is not installed") - self.pod = mock.patch( - 'airflow.contrib.kubernetes.worker_configuration.Pod' - ) self.resources = mock.patch( 'airflow.contrib.kubernetes.worker_configuration.Resources' ) @@ -95,7 +126,7 @@ def setUp(self): 'airflow.contrib.kubernetes.worker_configuration.Secret' ) - for patcher in [self.pod, self.resources, self.secret]: + for patcher in [self.resources, self.secret]: self.mock_foo = patcher.start() self.addCleanup(patcher.stop) @@ -152,6 +183,55 @@ def test_worker_environment_when_dags_folder_specified(self): self.assertEqual(dags_folder, env['AIRFLOW__CORE__DAGS_FOLDER']) + def test_make_pod_with_empty_executor_config(self): + self.kube_config.kube_affinity = self.affinity_config + self.kube_config.kube_tolerations = self.tolerations_config + + worker_config = WorkerConfiguration(self.kube_config) + kube_executor_config = KubernetesExecutorConfig(annotations=[], + volumes=[], + volume_mounts=[] + ) + + pod = worker_config.make_pod("default", str(uuid.uuid4()), "test_pod_id", "test_dag_id", + "test_task_id", str(datetime.utcnow()), "bash -c 'ls /'", + kube_executor_config) + + self.assertTrue(pod.affinity['podAntiAffinity'] is not None) + self.assertEqual('app', + pod.affinity['podAntiAffinity'] + ['requiredDuringSchedulingIgnoredDuringExecution'][0] + ['labelSelector'] + ['matchExpressions'][0] + ['key']) + + self.assertEqual(2, len(pod.tolerations)) + self.assertEqual('prod', pod.tolerations[1]['key']) + + def test_make_pod_with_executor_config(self): + worker_config = WorkerConfiguration(self.kube_config) + kube_executor_config = KubernetesExecutorConfig(affinity=self.affinity_config, + tolerations=self.tolerations_config, + annotations=[], + volumes=[], + volume_mounts=[] + ) + + pod = worker_config.make_pod("default", str(uuid.uuid4()), "test_pod_id", "test_dag_id", + "test_task_id", str(datetime.utcnow()), "bash -c 'ls /'", + kube_executor_config) + + self.assertTrue(pod.affinity['podAntiAffinity'] is not None) + self.assertEqual('app', + pod.affinity['podAntiAffinity'] + ['requiredDuringSchedulingIgnoredDuringExecution'][0] + ['labelSelector'] + ['matchExpressions'][0] + ['key']) + + self.assertEqual(2, len(pod.tolerations)) + self.assertEqual('prod', pod.tolerations[1]['key']) + def test_worker_pvc_dags(self): # Tests persistence volume config created when `dags_volume_claim` is set self.kube_config.dags_volume_claim = 'airflow-dags' From b0cbdbb00011a719178bf1f7d946e8cc02f2ae57 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 7 Jan 2019 21:05:50 +0000 Subject: [PATCH 349/808] Revert "[AIRFLOW-XXX] Switch to openjdk8 in Travis tests" This reverts commit 47ab0401d9a7c369151cc1800fee6adfe0efde53. --- .travis.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index fae1ecc5f04f3..e078d7c9ae884 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,7 +20,7 @@ sudo: true dist: trusty language: python jdk: - - openjdk8 + - oraclejdk8 services: - cassandra - mongodb @@ -93,7 +93,7 @@ before_install: - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys - ln -s ~/.ssh/authorized_keys ~/.ssh/authorized_keys2 - chmod 600 ~/.ssh/* - - jdk_switcher use openjdk8 + - jdk_switcher use oraclejdk8 install: - pip install --upgrade pip - pip install tox From 712ea9af64daeace77ecabc6a5a67308e59cd74b Mon Sep 17 00:00:00 2001 From: Gerardo Curiel Date: Wed, 22 Aug 2018 18:26:54 +1000 Subject: [PATCH 350/808] [AIRFLOW-2499] Dockerise CI pipeline (#3393) --- .travis.yml | 89 ++------- CONTRIBUTING.md | 180 +++++++++--------- README.md | 3 +- airflow/operators/python_operator.py | 2 - airflow/utils/db.py | 15 +- run_tox.sh | 21 -- run_unit_tests.sh | 64 +++---- scripts/ci/{ldap.sh => 1-setup-env.sh} | 27 ++- .../ci/{load_fixtures.sh => 2-setup-kdc.sh} | 30 +-- ...{travis_script.sh => 3-setup-databases.sh} | 25 +-- scripts/ci/{load_data.sh => 4-load-data.sh} | 11 +- scripts/ci/5-run-tests.sh | 6 + .../{check-license.sh => 6-check-license.sh} | 8 +- scripts/ci/airflow_travis.cfg | 6 +- scripts/ci/docker-compose.yml | 90 +++++++++ scripts/ci/{flake8_diff.sh => flake8-diff.sh} | 0 scripts/ci/{run_tests.sh => krb5/Dockerfile} | 51 ++--- .../ci/{ => krb5/krb-conf/client}/krb5.conf | 7 +- .../ci/{ => krb5/krb-conf/server}/kadm5.acl | 4 +- .../ci/{ => krb5/krb-conf/server}/kdc.conf | 4 +- .../ci/{setup_kdc.sh => krb5/start_kdc.sh} | 31 +-- scripts/ci/krb5/supervisord.conf | 43 +++++ scripts/ci/ldif/example.com.ldif | 24 --- .../ci/{my.cnf => mysql/conf.d/airflow.cnf} | 4 +- .../ldif/01-users.example.com.ldif} | 4 +- .../ldif/02-groups.example.com.ldif} | 6 +- .../ldif/03-manager.example.com.ldif} | 4 +- .../ci/openldap/ldif/04-rootdn.ldif | 13 +- scripts/ci/{ => openldap}/slapd.conf | 5 +- scripts/ci/run-ci.sh | 56 ++++++ scripts/ci/setup_env.sh | 166 ---------------- scripts/ci/test-environment.sh | 19 ++ tests/contrib/hooks/test_cassandra_hook.py | 2 +- tests/contrib/sensors/test_mongo_sensor.py | 2 +- tests/core.py | 4 +- tox.ini | 39 ++-- 36 files changed, 485 insertions(+), 580 deletions(-) delete mode 100755 run_tox.sh rename scripts/ci/{ldap.sh => 1-setup-env.sh} (67%) rename scripts/ci/{load_fixtures.sh => 2-setup-kdc.sh} (59%) rename scripts/ci/{travis_script.sh => 3-setup-databases.sh} (58%) rename scripts/ci/{load_data.sh => 4-load-data.sh} (81%) rename scripts/ci/{check-license.sh => 6-check-license.sh} (94%) create mode 100644 scripts/ci/docker-compose.yml rename scripts/ci/{flake8_diff.sh => flake8-diff.sh} (100%) rename scripts/ci/{run_tests.sh => krb5/Dockerfile} (51%) mode change 100755 => 100644 rename scripts/ci/{ => krb5/krb-conf/client}/krb5.conf (94%) rename scripts/ci/{ => krb5/krb-conf/server}/kadm5.acl (99%) rename scripts/ci/{ => krb5/krb-conf/server}/kdc.conf (99%) rename scripts/ci/{setup_kdc.sh => krb5/start_kdc.sh} (75%) create mode 100644 scripts/ci/krb5/supervisord.conf delete mode 100644 scripts/ci/ldif/example.com.ldif rename scripts/ci/{my.cnf => mysql/conf.d/airflow.cnf} (99%) rename scripts/ci/{ldif/users.example.com.ldif => openldap/ldif/01-users.example.com.ldif} (99%) rename scripts/ci/{ldif/groups.example.com.ldif => openldap/ldif/02-groups.example.com.ldif} (96%) rename scripts/ci/{ldif/manager.example.com.ldif => openldap/ldif/03-manager.example.com.ldif} (99%) rename init.sh => scripts/ci/openldap/ldif/04-rootdn.ldif (84%) rename scripts/ci/{ => openldap}/slapd.conf (99%) create mode 100755 scripts/ci/run-ci.sh delete mode 100755 scripts/ci/setup_env.sh create mode 100644 scripts/ci/test-environment.sh diff --git a/.travis.yml b/.travis.yml index e078d7c9ae884..9c7cfd02084db 100644 --- a/.travis.yml +++ b/.travis.yml @@ -19,95 +19,40 @@ sudo: true dist: trusty language: python -jdk: - - oraclejdk8 -services: - - cassandra - - mongodb - - mysql - - postgresql - - rabbitmq -addons: - apt: - packages: - - slapd - - ldap-utils - - openssh-server - - mysql-server-5.6 - - mysql-client-core-5.6 - - mysql-client-5.6 - - krb5-user - - krb5-kdc - - krb5-admin-server - - oracle-java8-installer - - python-selinux - postgresql: "9.2" -python: - - "2.7" - - "3.5" env: global: + - DOCKER_COMPOSE_VERSION=1.20.0 - SLUGIFY_USES_TEXT_UNIDECODE=yes - TRAVIS_CACHE=$HOME/.travis_cache/ - - KRB5_CONFIG=/etc/krb5.conf - - KRB5_KTNAME=/etc/airflow.keytab - # Travis on google cloud engine has a global /etc/boto.cfg that - # does not work with python 3 - - BOTO_CONFIG=/tmp/bogusvalue matrix: + - TOX_ENV=flake8 - TOX_ENV=py27-backend_mysql - TOX_ENV=py27-backend_sqlite - TOX_ENV=py27-backend_postgres - - TOX_ENV=py35-backend_mysql - - TOX_ENV=py35-backend_sqlite - - TOX_ENV=py35-backend_postgres - - TOX_ENV=flake8 + - TOX_ENV=py35-backend_mysql PYTHON_VERSION=3 + - TOX_ENV=py35-backend_sqlite PYTHON_VERSION=3 + - TOX_ENV=py35-backend_postgres PYTHON_VERSION=3 - TOX_ENV=py27-backend_postgres KUBERNETES_VERSION=v1.9.0 - - TOX_ENV=py35-backend_postgres KUBERNETES_VERSION=v1.10.0 -matrix: - exclude: - - python: "3.5" - env: TOX_ENV=py27-backend_mysql - - python: "3.5" - env: TOX_ENV=py27-backend_sqlite - - python: "3.5" - env: TOX_ENV=py27-backend_postgres - - python: "2.7" - env: TOX_ENV=py35-backend_mysql - - python: "2.7" - env: TOX_ENV=py35-backend_sqlite - - python: "2.7" - env: TOX_ENV=py35-backend_postgres - - python: "2.7" - env: TOX_ENV=flake8 - - python: "3.5" - env: TOX_ENV=py27-backend_postgres KUBERNETES_VERSION=v1.9.0 - - python: "2.7" - env: TOX_ENV=py35-backend_postgres KUBERNETES_VERSION=v1.10.0 + - TOX_ENV=py35-backend_postgres KUBERNETES_VERSION=v1.10.0 PYTHON_VERSION=3 cache: directories: - $HOME/.wheelhouse/ + - $HOME/.cache/pip - $HOME/.travis_cache/ before_install: - - yes | ssh-keygen -t rsa -C your_email@youremail.com -P '' -f ~/.ssh/id_rsa - - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys - - ln -s ~/.ssh/authorized_keys ~/.ssh/authorized_keys2 - - chmod 600 ~/.ssh/* - - jdk_switcher use oraclejdk8 + - sudo ls -lh $HOME/.cache/pip/ + - sudo rm -rf $HOME/.cache/pip/* $HOME/.wheelhouse/* + - sudo chown -R travis.travis $HOME/.cache/pip install: + # Use recent docker-compose version + - sudo rm /usr/local/bin/docker-compose + - curl -L https://github.com/docker/compose/releases/download/${DOCKER_COMPOSE_VERSION}/docker-compose-`uname -s`-`uname -m` > docker-compose + - chmod +x docker-compose + - sudo mv docker-compose /usr/local/bin - pip install --upgrade pip - - pip install tox - pip install codecov -before_script: - - cat "$TRAVIS_BUILD_DIR/scripts/ci/my.cnf" | sudo tee -a /etc/mysql/my.cnf - - mysql -e 'drop database if exists airflow; create database airflow' -u root - - sudo service mysql restart - - psql -c 'create database airflow;' -U postgres - - export PATH=${PATH}:/tmp/hive/bin - # Required for K8s v1.10.x. See - # https://github.com/kubernetes/kubernetes/issues/61058#issuecomment-372764783 - - sudo mount --make-shared / && sudo service docker restart script: - - ./scripts/ci/travis_script.sh + - docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh after_success: + - sudo chown -R travis.travis . - codecov diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index b7a1b1e0a361f..57178abb4b7cd 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -22,7 +22,6 @@ under the License. Contributions are welcome and are greatly appreciated! Every little bit helps, and credit will always be given. - # Table of Contents * [TOC](#table-of-contents) * [Types of Contributions](#types-of-contributions) @@ -34,11 +33,10 @@ little bit helps, and credit will always be given. * [Documentation](#documentation) * [Development and Testing](#development-and-testing) - [Setting up a development environment](#setting-up-a-development-environment) - - [Pull requests guidelines](#pull-request-guidelines) - - [Testing Locally](#testing-locally) + - [Running unit tests](#running-unit-tests) + * [Pull requests guidelines](#pull-request-guidelines) * [Changing the Metadata Database](#changing-the-metadata-database) - ## Types of Contributions ### Report Bugs @@ -98,55 +96,110 @@ extras to build the full API reference. ## Development and Testing -### Set up a development env using Docker +### Set up a development environment -Go to your Airflow directory and start a new docker container. You can choose between Python 2 or 3, whatever you prefer. +There are three ways to setup an Apache Airflow development environment. -``` -# Start docker in your Airflow directory -docker run -t -i -v `pwd`:/airflow/ -w /airflow/ -e SLUGIFY_USES_TEXT_UNIDECODE=yes python:2 bash +1. Using tools and libraries installed directly on your system. + + Install Python (2.7.x or 3.4.x), MySQL, and libxml by using system-level package + managers like yum, apt-get for Linux, or Homebrew for Mac OS at first. Refer to the [base CI Dockerfile](https://github.com/apache/incubator-airflow-ci/blob/master/Dockerfile.base) for + a comprehensive list of required packages. + + Then install python development requirements. It is usually best to work in a virtualenv: + + ```bash + cd $AIRFLOW_HOME + virtualenv env + source env/bin/activate + pip install -e .[devel] + ``` + +2. Using a Docker container + + Go to your Airflow directory and start a new docker container. You can choose between Python 2 or 3, whatever you prefer. + + ``` + # Start docker in your Airflow directory + docker run -t -i -v `pwd`:/airflow/ -w /airflow/ -e SLUGIFY_USES_TEXT_UNIDECODE=yes python:2 bash + + # Go to the Airflow directory + cd /airflow/ + + # Install Airflow with all the required dependencies, + # including the devel which will provide the development tools + pip install -e ".[hdfs,hive,druid,devel]" + + # Init the database + airflow initdb + + nosetests -v tests/hooks/test_druid_hook.py + + test_get_first_record (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok + test_get_records (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok + test_get_uri (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok + test_get_conn_url (tests.hooks.test_druid_hook.TestDruidHook) ... ok + test_submit_gone_wrong (tests.hooks.test_druid_hook.TestDruidHook) ... ok + test_submit_ok (tests.hooks.test_druid_hook.TestDruidHook) ... ok + test_submit_timeout (tests.hooks.test_druid_hook.TestDruidHook) ... ok + test_submit_unknown_response (tests.hooks.test_druid_hook.TestDruidHook) ... ok + + ---------------------------------------------------------------------- + Ran 8 tests in 3.036s + + OK + ``` -# Install Airflow with all the required dependencies, -# including the devel which will provide the development tools -pip install -e .[devel,druid,hdfs,hive] + The Airflow code is mounted inside of the Docker container, so if you change something using your favorite IDE, you can directly test is in the container. -# Init the database -airflow initdb +3. Using [Docker Compose](https://docs.docker.com/compose/) and Airflow's CI scripts. -nosetests -v tests/hooks/test_druid_hook.py + Start a docker container through Compose for development to avoid installing the packages directly on your system. The following will give you a shell inside a container, run all required service containers (MySQL, PostgresSQL, krb5 and so on) and install all the dependencies: - test_get_first_record (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok - test_get_records (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok - test_get_uri (tests.hooks.test_druid_hook.TestDruidDbApiHook) ... ok - test_get_conn_url (tests.hooks.test_druid_hook.TestDruidHook) ... ok - test_submit_gone_wrong (tests.hooks.test_druid_hook.TestDruidHook) ... ok - test_submit_ok (tests.hooks.test_druid_hook.TestDruidHook) ... ok - test_submit_timeout (tests.hooks.test_druid_hook.TestDruidHook) ... ok - test_submit_unknown_response (tests.hooks.test_druid_hook.TestDruidHook) ... ok + ```bash + docker-compose -f scripts/ci/docker-compose.yml run airflow-testing bash + # From the container + pip install -e .[devel] + # Run all the tests with python and mysql through tox + tox -e py35-backend_mysql + ``` - ---------------------------------------------------------------------- - Ran 8 tests in 3.036s +### Running unit tests + +To run tests locally, once your unit test environment is setup (directly on your +system or through our Docker setup) you should be able to simply run +``./run_unit_tests.sh`` at will. + +For example, in order to just execute the "core" unit tests, run the following: - OK ``` +./run_unit_tests.sh tests.core:CoreTest -s --logging-level=DEBUG +``` + +or a single test method: -The Airflow code is mounted inside of the Docker container, so if you change something using your favorite IDE, you can directly test is in the container. +``` +./run_unit_tests.sh tests.core:CoreTest.test_check_operators -s --logging-level=DEBUG +``` + +To run the whole test suite with Docker Compose, do: + +``` +# Install Docker Compose first, then this will run the tests +docker-compose -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh +``` -### Set up a development env using Virtualenv +Alternatively can also set up [Travis CI](https://travis-ci.org/) on your repo to automate this. +It is free for open source projects. -Please install python(2.7.x or 3.4.x), mysql, and libxml by using system-level package -managers like yum, apt-get for Linux, or homebrew for Mac OS at first. -It is usually best to work in a virtualenv and tox. Install development requirements: +For more information on how to run a subset of the tests, take a look at the +nosetests docs. - cd $AIRFLOW_HOME - virtualenv env - source env/bin/activate - pip install -e .[devel] - tox +See also the list of test classes and methods in `tests/core.py`. Feel free to customize based on the extras available in [setup.py](./setup.py) -### Pull Request Guidelines +## Pull Request Guidelines Before you submit a pull request from your forked repo, check that it meets these guidelines: @@ -187,59 +240,6 @@ using `flake8 airflow tests`. `git diff upstream/master -u -- "*.py" | flake8 -- commit messages and adhere to them. It makes the lives of those who come after you a lot easier. -### Testing locally - -#### TL;DR -Tests can then be run with (see also the [Running unit tests](#running-unit-tests) section below): - - ./run_unit_tests.sh - -Individual test files can be run with: - - nosetests [path to file] - -#### Running unit tests - -We *highly* recommend setting up [Travis CI](https://travis-ci.org/) on -your repo to automate this. It is free for open source projects. If for -some reason you cannot, you can use the steps below to run tests. - -Here are loose guidelines on how to get your environment to run the unit tests. -We do understand that no one out there can run the full test suite since -Airflow is meant to connect to virtually any external system and that you most -likely have only a subset of these in your environment. You should run the -CoreTests and tests related to things you touched in your PR. - -To set up a unit test environment, first take a look at `run_unit_tests.sh` and -understand that your ``AIRFLOW_CONFIG`` points to an alternate config file -while running the tests. You shouldn't have to alter this config file but -you may if need be. - -From that point, you can actually export these same environment variables in -your shell, start an Airflow webserver ``airflow webserver -d`` and go and -configure your connection. Default connections that are used in the tests -should already have been created, you just need to point them to the systems -where you want your tests to run. - -Once your unit test environment is setup, you should be able to simply run -``./run_unit_tests.sh`` at will. - -For example, in order to just execute the "core" unit tests, run the following: - -``` -./run_unit_tests.sh tests.core:CoreTest -s --logging-level=DEBUG -``` - -or a single test method: - -``` -./run_unit_tests.sh tests.core:CoreTest.test_check_operators -s --logging-level=DEBUG -``` - -For more information on how to run a subset of the tests, take a look at the -nosetests docs. - -See also the list of test classes and methods in `tests/core.py`. ### Changing the Metadata Database diff --git a/README.md b/README.md index 633000c18e788..fceb2a7d7bffb 100644 --- a/README.md +++ b/README.md @@ -89,7 +89,8 @@ unit of work and continuity. Want to help build Apache Airflow? Check out our [contributing documentation](https://github.com/apache/airflow/blob/master/CONTRIBUTING.md). -## Who uses Apache Airflow? + +## Who uses Airflow? As the Apache Airflow community grows, we'd like to keep track of who is using the platform. Please send a PR with your company name and @githubhandle diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py index a75f3a0207e41..e9007cfba20f2 100644 --- a/airflow/operators/python_operator.py +++ b/airflow/operators/python_operator.py @@ -176,10 +176,8 @@ class PythonVirtualenvOperator(PythonOperator): variable named virtualenv_string_args will be available (populated by string_args). In addition, one can pass stuff through op_args and op_kwargs, and one can use a return value. - Note that if your virtualenv runs in a different Python major version than Airflow, you cannot use return values, op_args, or op_kwargs. You can use string_args though. - :param python_callable: A python function with no references to outside variables, defined with def, which will be run in a virtualenv :type python_callable: function diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 74317a59c7a1f..aa770b27b6106 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -94,12 +94,12 @@ def initdb(rbac=False): merge_conn( models.Connection( conn_id='airflow_db', conn_type='mysql', - host='localhost', login='root', password='', + host='mysql', login='root', password='', schema='airflow')) merge_conn( models.Connection( conn_id='airflow_ci', conn_type='mysql', - host='localhost', login='root', extra="{\"local_infile\": true}", + host='mysql', login='root', extra="{\"local_infile\": true}", schema='airflow_ci')) merge_conn( models.Connection( @@ -141,18 +141,19 @@ def initdb(rbac=False): merge_conn( models.Connection( conn_id='mongo_default', conn_type='mongo', - host='localhost', port=27017)) + host='mongo', port=27017)) merge_conn( models.Connection( conn_id='mysql_default', conn_type='mysql', login='root', - host='localhost')) + host='mysql')) merge_conn( models.Connection( conn_id='postgres_default', conn_type='postgres', login='postgres', + password='airflow', schema='airflow', - host='localhost')) + host='postgres')) merge_conn( models.Connection( conn_id='sqlite_default', conn_type='sqlite', @@ -184,7 +185,7 @@ def initdb(rbac=False): merge_conn( models.Connection( conn_id='sftp_default', conn_type='sftp', - host='localhost', port=22, login='travis', + host='localhost', port=22, login='airflow', extra=''' {"key_file": "~/.ssh/id_rsa", "no_host_key_check": true} ''')) @@ -287,7 +288,7 @@ def initdb(rbac=False): merge_conn( models.Connection( conn_id='cassandra_default', conn_type='cassandra', - host='localhost', port=9042)) + host='cassandra', port=9042)) # Known event types KET = models.KnownEventType diff --git a/run_tox.sh b/run_tox.sh deleted file mode 100755 index b4f204d649a32..0000000000000 --- a/run_tox.sh +++ /dev/null @@ -1,21 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -set -o verbose - -python setup.py test --tox-args="-v -e $TOX_ENV" diff --git a/run_unit_tests.sh b/run_unit_tests.sh index 124abff44a844..2c4abbfaad605 100755 --- a/run_unit_tests.sh +++ b/run_unit_tests.sh @@ -21,7 +21,7 @@ set -x # environment -export AIRFLOW_HOME=${AIRFLOW_HOME:=~/airflow} +export AIRFLOW_HOME=${AIRFLOW_HOME:=~} export AIRFLOW__CORE__UNIT_TEST_MODE=True # add test/contrib to PYTHONPATH @@ -29,57 +29,41 @@ DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" export PYTHONPATH=$PYTHONPATH:${DIR}/tests/test_utils # any argument received is overriding the default nose execution arguments: - nose_args=$@ -#--with-doctest - # Generate the `airflow` executable if needed which airflow > /dev/null || python setup.py develop echo "Initializing the DB" +yes | airflow initdb yes | airflow resetdb -if [ "${TRAVIS}" ]; then - if [ -z "$nose_args" ]; then - nose_args="--with-coverage \ - --cover-erase \ - --cover-html \ - --cover-package=airflow \ - --cover-html-dir=airflow/www/static/coverage \ - --with-ignore-docstrings \ - --rednose \ - --with-timer \ - -v \ - --logging-level=DEBUG " - fi +if [ -z "$nose_args" ]; then + nose_args="--with-coverage \ + --cover-erase \ + --cover-html \ + --cover-package=airflow \ + --cover-html-dir=airflow/www/static/coverage \ + --with-ignore-docstrings \ + --rednose \ + --with-timer \ + -v \ + --logging-level=DEBUG " +fi - # For impersonation tests running on SQLite on Travis, make the database world readable so other - # users can update it - AIRFLOW_DB="/home/travis/airflow/airflow.db" - if [ -f "${AIRFLOW_DB}" ]; then - sudo chmod a+rw "${AIRFLOW_DB}" - fi +# For impersonation tests running on SQLite on Travis, make the database world readable so other +# users can update it +AIRFLOW_DB="$HOME/airflow.db" - # For impersonation tests on Travis, make airflow accessible to other users via the global PATH - # (which contains /usr/local/bin) - sudo ln -s "${VIRTUAL_ENV}/bin/airflow" /usr/local/bin/ -else - if [ -z "$nose_args" ]; then - nose_args="--with-coverage \ - --cover-erase \ - --cover-html \ - --cover-package=airflow \ - --cover-html-dir=airflow/www/static/coverage \ - --with-ignore-docstrings \ - --rednose \ - --with-timer \ - -s \ - -v \ - --logging-level=DEBUG " - fi +if [ -f "${AIRFLOW_DB}" ]; then + chmod a+rw "${AIRFLOW_DB}" + chmod g+rwx "${AIRFLOW_HOME}" fi +# For impersonation tests on Travis, make airflow accessible to other users via the global PATH +# (which contains /usr/local/bin) +sudo ln -sf "${VIRTUAL_ENV}/bin/airflow" /usr/local/bin/ + echo "Starting the unit tests with the following nose arguments: "$nose_args nosetests $nose_args diff --git a/scripts/ci/ldap.sh b/scripts/ci/1-setup-env.sh similarity index 67% rename from scripts/ci/ldap.sh rename to scripts/ci/1-setup-env.sh index d0e3043a9b685..0a976b35f027b 100755 --- a/scripts/ci/ldap.sh +++ b/scripts/ci/1-setup-env.sh @@ -8,9 +8,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -18,19 +18,16 @@ # specific language governing permissions and limitations # under the License. -set -o verbose - -DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) -LDAP_DB=/tmp/ldap_db - -echo "Creating database directory" - -rm -rf ${LDAP_DB} && mkdir ${LDAP_DB} && cp /usr/share/doc/slapd/examples/DB_CONFIG ${LDAP_DB} +set -exuo pipefail -echo "Launching OpenLDAP ..." +# Start MiniCluster +java -cp "/tmp/minicluster-1.1-SNAPSHOT/*" com.ing.minicluster.MiniCluster > /dev/null & -# Start slapd with non root privileges -slapd -h "ldap://127.0.0.1:3890/" -f ${DIR}/slapd.conf +# Set up ssh keys +echo 'yes' | ssh-keygen -t rsa -C your_email@youremail.com -P '' -f ~/.ssh/id_rsa +cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys +ln -s ~/.ssh/authorized_keys ~/.ssh/authorized_keys2 +chmod 600 ~/.ssh/* -# Wait for LDAP to start -sleep 1 +# SSH Service +sudo service ssh restart diff --git a/scripts/ci/load_fixtures.sh b/scripts/ci/2-setup-kdc.sh similarity index 59% rename from scripts/ci/load_fixtures.sh rename to scripts/ci/2-setup-kdc.sh index 55beb919dfda2..56824f5a5ee3f 100755 --- a/scripts/ci/load_fixtures.sh +++ b/scripts/ci/2-setup-kdc.sh @@ -8,9 +8,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -18,17 +18,21 @@ # specific language governing permissions and limitations # under the License. -set -o verbose +set -exuo pipefail + +DIRNAME=$(cd "$(dirname "$0")"; pwd) + +FQDN=`hostname` +ADMIN="admin" +PASS="airflow" +KRB5_KTNAME=/etc/airflow.keytab -DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) -FIXTURES_DIR="$DIR/ldif" -LOAD_ORDER=("example.com.ldif" "manager.example.com.ldif" "users.example.com.ldif" "groups.example.com.ldif") +cat /etc/hosts +echo "hostname: ${FQDN}" -load_fixture () { - ldapadd -x -H ldap://127.0.0.1:3890/ -D "cn=Manager,dc=example,dc=com" -w insecure -f $1 -} +sudo cp $DIRNAME/krb5/krb-conf/client/krb5.conf /etc/krb5.conf -for FILE in "${LOAD_ORDER[@]}" -do - load_fixture "${FIXTURES_DIR}/${FILE}" -done; +echo -e "${PASS}\n${PASS}" | sudo kadmin -p ${ADMIN}/admin -w ${PASS} -q "addprinc -randkey airflow/${FQDN}" +sudo kadmin -p ${ADMIN}/admin -w ${PASS} -q "ktadd -k ${KRB5_KTNAME} airflow" +sudo kadmin -p ${ADMIN}/admin -w ${PASS} -q "ktadd -k ${KRB5_KTNAME} airflow/${FQDN}" +sudo chmod 0644 ${KRB5_KTNAME} diff --git a/scripts/ci/travis_script.sh b/scripts/ci/3-setup-databases.sh similarity index 58% rename from scripts/ci/travis_script.sh rename to scripts/ci/3-setup-databases.sh index 52571cce14a35..2a5cb682e0119 100755 --- a/scripts/ci/travis_script.sh +++ b/scripts/ci/3-setup-databases.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash - # Licensed to the Apache Software Foundation (ASF) under one * # or more contributor license agreements. See the NOTICE file * # distributed with this work for additional information * @@ -17,24 +16,8 @@ # specific language governing permissions and limitations * # under the License. * -DIRNAME=$(cd "$(dirname "$0")"; pwd) -AIRFLOW_ROOT="$DIRNAME/../.." -cd $AIRFLOW_ROOT && pip --version && ls -l $HOME/.wheelhouse && tox --version +set -exuo pipefail + +MYSQL_HOST=mysql -if [ -z "$KUBERNETES_VERSION" ]; -then - tox -e $TOX_ENV -else - KUBERNETES_VERSION=${KUBERNETES_VERSION} $DIRNAME/kubernetes/setup_kubernetes.sh && \ - tox -e $TOX_ENV -- tests.contrib.minikube \ - --with-coverage \ - --cover-erase \ - --cover-html \ - --cover-package=airflow \ - --cover-html-dir=airflow/www/static/coverage \ - --with-ignore-docstrings \ - --rednose \ - --with-timer \ - -v \ - --logging-level=DEBUG -fi +mysql -h ${MYSQL_HOST} -u root -e 'drop database if exists airflow; create database airflow' diff --git a/scripts/ci/load_data.sh b/scripts/ci/4-load-data.sh similarity index 81% rename from scripts/ci/load_data.sh rename to scripts/ci/4-load-data.sh index 3422b07c20c13..7935482be075e 100755 --- a/scripts/ci/load_data.sh +++ b/scripts/ci/4-load-data.sh @@ -15,14 +15,15 @@ # KIND, either express or implied. See the License for the * # specific language governing permissions and limitations * # under the License. * -set -o verbose + +set -exuo pipefail DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) DATA_DIR="${DIR}/data" DATA_FILE="${DATA_DIR}/baby_names.csv" DATABASE=airflow_ci +HOST=mysql -mysqladmin -u root create ${DATABASE} -mysql -u root < ${DATA_DIR}/mysql_schema.sql -mysqlimport --local -u root --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 ${DATABASE} ${DATA_FILE} - +mysqladmin -h ${HOST} -u root create ${DATABASE} +mysql -h ${HOST} -u root < ${DATA_DIR}/mysql_schema.sql +mysqlimport --local -h ${HOST} -u root --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 ${DATABASE} ${DATA_FILE} diff --git a/scripts/ci/5-run-tests.sh b/scripts/ci/5-run-tests.sh index 8a74d824efa29..0a3add4762636 100755 --- a/scripts/ci/5-run-tests.sh +++ b/scripts/ci/5-run-tests.sh @@ -45,6 +45,12 @@ echo Backend: $AIRFLOW__CORE__SQL_ALCHEMY_CONN export AIRFLOW_HOME=${AIRFLOW_HOME:=~} export AIRFLOW__CORE__UNIT_TEST_MODE=True +# configuration test +export AIRFLOW__TESTSECTION__TESTKEY=testvalue + +# use Airflow 2.0-style imports +export AIRFLOW_USE_NEW_IMPORTS=1 + # any argument received is overriding the default nose execution arguments: nose_args=$@ diff --git a/scripts/ci/check-license.sh b/scripts/ci/6-check-license.sh similarity index 94% rename from scripts/ci/check-license.sh rename to scripts/ci/6-check-license.sh index 035283f9061b5..8f3999043b14d 100755 --- a/scripts/ci/check-license.sh +++ b/scripts/ci/6-check-license.sh @@ -53,9 +53,7 @@ acquire_rat_jar () { FWDIR="$(cd "`dirname "$0"`"/../..; pwd)" cd "$FWDIR" -if [ -z "${TRAVIS_CACHE}" ]; then - TRAVIS_CACHE=/tmp -fi +TMP_DIR=/tmp if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" @@ -64,8 +62,8 @@ else fi export RAT_VERSION=0.12 -export rat_jar="${TRAVIS_CACHE}"/lib/apache-rat-${RAT_VERSION}.jar -mkdir -p ${TRAVIS_CACHE}/lib +export rat_jar="${TMP_DIR}"/lib/apache-rat-${RAT_VERSION}.jar +mkdir -p ${TMP_DIR}/lib [[ -f "$rat_jar" ]] || acquire_rat_jar || { diff --git a/scripts/ci/airflow_travis.cfg b/scripts/ci/airflow_travis.cfg index 140ecab683f2c..31e8586c4cc15 100644 --- a/scripts/ci/airflow_travis.cfg +++ b/scripts/ci/airflow_travis.cfg @@ -21,7 +21,7 @@ airflow_home = ~/airflow dags_folder = ~/airflow/dags base_log_folder = ~/airflow/logs executor = LocalExecutor -sql_alchemy_conn = mysql://root@localhost/airflow +sql_alchemy_conn = mysql://root@mysql/airflow unit_test_mode = True load_examples = True donot_pickle = False @@ -55,8 +55,8 @@ smtp_mail_from = airflow@example.com celery_app_name = airflow.executors.celery_executor worker_concurrency = 16 worker_log_server_port = 8793 -broker_url = amqp://guest:guest@localhost:5672/ -result_backend = db+mysql://root@localhost/airflow +broker_url = amqp://guest:guest@rabbitmq:5672/ +result_backend = db+mysql://root@mysql/airflow flower_port = 5555 default_queue = default diff --git a/scripts/ci/docker-compose.yml b/scripts/ci/docker-compose.yml new file mode 100644 index 0000000000000..861cf9e8b89ce --- /dev/null +++ b/scripts/ci/docker-compose.yml @@ -0,0 +1,90 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: "2.2" +services: + mysql: + image: mysql:5.6 + restart: always + environment: + - MYSQL_ALLOW_EMPTY_PASSWORD=true + - MYSQL_ROOT_HOST=% + volumes: + - ./mysql/conf.d:/etc/mysql/conf.d + + postgres: + image: postgres:9.6 + restart: always + environment: + - POSTGRES_USER=postgres + - POSTGRES_PASSWORD=airflow + - POSTGRES_DB=airflow + + mongo: + image: mongo:3 + restart: always + + cassandra: + image: cassandra:3.0 + restart: always + + rabbitmq: + image: rabbitmq:3.7 + restart: always + + openldap: + image: osixia/openldap:1.2.0 + restart: always + command: --copy-service + environment: + - LDAP_DOMAIN=example.com + - LDAP_ADMIN_PASSWORD=insecure + - LDAP_CONFIG_PASSWORD=insecure + volumes: + - ./openldap/ldif:/container/service/slapd/assets/config/bootstrap/ldif/custom + + krb5-kdc-server: + build: ./krb5 + image: krb5-kdc-server + hostname: krb5-kdc-server + domainname: example.com + + airflow-testing: + image: airflowci/incubator-airflow-ci:latest + init: true + environment: + - USER=airflow + - SLUGIFY_USES_TEXT_UNIDECODE=yes + - TOX_ENV + - PYTHON_VERSION + - TRAVIS + - TRAVIS_BRANCH + - TRAVIS_BUILD_DIR + - TRAVIS_JOB_ID + - TRAVIS_PULL_REQUEST + depends_on: + - postgres + - mysql + - mongo + - cassandra + - rabbitmq + - openldap + - krb5-kdc-server + volumes: + - ../../:/app + - ~/.cache/pip:/home/airflow/.cache/pip + - ~/.wheelhouse/:/home/airflow/.wheelhouse/ + working_dir: /app diff --git a/scripts/ci/flake8_diff.sh b/scripts/ci/flake8-diff.sh similarity index 100% rename from scripts/ci/flake8_diff.sh rename to scripts/ci/flake8-diff.sh diff --git a/scripts/ci/run_tests.sh b/scripts/ci/krb5/Dockerfile old mode 100755 new mode 100644 similarity index 51% rename from scripts/ci/run_tests.sh rename to scripts/ci/krb5/Dockerfile index d5e7655803df5..cdb4cf979e05d --- a/scripts/ci/run_tests.sh +++ b/scripts/ci/krb5/Dockerfile @@ -1,5 +1,3 @@ -#!/usr/bin/env bash - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -8,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -18,30 +16,37 @@ # specific language governing permissions and limitations # under the License. -set -o verbose +FROM ubuntu:xenial -if [ -z "$HADOOP_HOME" ]; then - echo "HADOOP_HOME not set - abort" >&2 - exit 1 -fi +# environment variables +ENV DEBIAN_FRONTEND noninteractive -echo "Using ${HADOOP_DISTRO} distribution of Hadoop from ${HADOOP_HOME}" +# Kerberos server +RUN apt-get update && apt-get install --no-install-recommends -y \ + ntp \ + python-dev \ + python-pip \ + python-wheel \ + python-setuptools \ + python-pkg-resources \ + krb5-admin-server \ + krb5-kdc -pwd +RUN mkdir /app/ -mkdir ~/airflow/ +# Supervisord +RUN pip install supervisor==3.3.4 +RUN mkdir -p /var/log/supervisord/ -if [ "${TRAVIS}" ]; then - echo "Using travis airflow.cfg" - DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" - cp -f ${DIR}/airflow_travis.cfg ~/airflow/unittests.cfg +COPY ./krb-conf/server/kdc.conf /etc/krb5kdc/kdc.conf +COPY ./krb-conf/server/kadm5.acl /etc/krb5kdc/kadm5.acl +COPY ./krb-conf/client/krb5.conf /etc/krb5.conf +COPY ./start_kdc.sh /app/start_kdc.sh - ROOTDIR="$(dirname $(dirname $DIR))" - export AIRFLOW__CORE__DAGS_FOLDER="$ROOTDIR/tests/dags" +# supervisord +COPY ./supervisord.conf /etc/supervisord.conf - # kdc init happens in setup_kdc.sh - kinit -kt ${KRB5_KTNAME} airflow -fi +WORKDIR /app -echo Backend: $AIRFLOW__CORE__SQL_ALCHEMY_CONN -./run_unit_tests.sh $@ +# when container is starting +CMD ["/bin/bash", "/app/start_kdc.sh"] diff --git a/scripts/ci/krb5.conf b/scripts/ci/krb5/krb-conf/client/krb5.conf similarity index 94% rename from scripts/ci/krb5.conf rename to scripts/ci/krb5/krb-conf/client/krb5.conf index bbc802a0eb9e8..471737a964123 100644 --- a/scripts/ci/krb5.conf +++ b/scripts/ci/krb5/krb-conf/client/krb5.conf @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -31,5 +31,6 @@ forwardable = true [realms] TEST.LOCAL = { - kdc = localhost:88 + kdc = krb5-kdc-server:88 + admin_server = krb5-kdc-server } diff --git a/scripts/ci/kadm5.acl b/scripts/ci/krb5/krb-conf/server/kadm5.acl similarity index 99% rename from scripts/ci/kadm5.acl rename to scripts/ci/krb5/krb-conf/server/kadm5.acl index 691dce6c2bbdf..41d17385ff144 100644 --- a/scripts/ci/kadm5.acl +++ b/scripts/ci/krb5/krb-conf/server/kadm5.acl @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/scripts/ci/kdc.conf b/scripts/ci/krb5/krb-conf/server/kdc.conf similarity index 99% rename from scripts/ci/kdc.conf rename to scripts/ci/krb5/krb-conf/server/kdc.conf index 5eef3053b363c..c21095f418d4a 100644 --- a/scripts/ci/kdc.conf +++ b/scripts/ci/krb5/krb-conf/server/kdc.conf @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/scripts/ci/setup_kdc.sh b/scripts/ci/krb5/start_kdc.sh similarity index 75% rename from scripts/ci/setup_kdc.sh rename to scripts/ci/krb5/start_kdc.sh index 3e525c5e82ec7..6e02f006fa830 100755 --- a/scripts/ci/setup_kdc.sh +++ b/scripts/ci/krb5/start_kdc.sh @@ -8,9 +8,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -18,23 +18,15 @@ # specific language governing permissions and limitations # under the License. -cat /etc/hosts +set -exuo pipefail FQDN=`hostname` - -echo "hostname: ${FQDN}" - ADMIN="admin" PASS="airflow" +KRB5_KTNAME=/etc/airflow.keytab -DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" - -ln -sf /dev/urandom /dev/random - -cp ${DIR}/kdc.conf /etc/krb5kdc/kdc.conf -cp ${DIR}/kadm5.acl /etc/krb5kdc/kadm5.acl -cp ${DIR}/krb5.conf /etc/krb5.conf - +cat /etc/hosts +echo "hostname: ${FQDN}" # create kerberos database echo -e "${PASS}\n${PASS}" | kdb5_util create -s # create admin @@ -45,12 +37,5 @@ echo -e "${PASS}\n${PASS}" | kadmin.local -q "addprinc -randkey airflow/${FQDN}" kadmin.local -q "ktadd -k ${KRB5_KTNAME} airflow" kadmin.local -q "ktadd -k ${KRB5_KTNAME} airflow/${FQDN}" -service krb5-kdc restart - -# make sure the keytab is readable to anyone -chmod 664 ${KRB5_KTNAME} - -# don't do a kinit here as this happens under super user privileges -# on travis -# kinit -kt ${KRB5_KTNAME} airflow - +# Start services +/usr/local/bin/supervisord -n -c /etc/supervisord.conf diff --git a/scripts/ci/krb5/supervisord.conf b/scripts/ci/krb5/supervisord.conf new file mode 100644 index 0000000000000..165e5cde84646 --- /dev/null +++ b/scripts/ci/krb5/supervisord.conf @@ -0,0 +1,43 @@ +; +; Licensed to the Apache Software Foundation (ASF) under one +; or more contributor license agreements. See the NOTICE file +; distributed with this work for additional information +; regarding copyright ownership. The ASF licenses this file +; to you under the Apache License, Version 2.0 (the +; "License"); you may not use this file except in compliance +; with the License. You may obtain a copy of the License at +; +; http://www.apache.org/licenses/LICENSE-2.0 +; +; Unless required by applicable law or agreed to in writing, +; software distributed under the License is distributed on an +; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +; KIND, either express or implied. See the License for the +; specific language governing permissions and limitations +; under the License. + +; supervisord.conf - kdc-server + +[supervisord] +logfile=/var/log/supervisord/supervisord.log ; supervisord log file +logfile_maxbytes=50MB ; maximum size of logfile before rotation +logfile_backups=10 ; number of backed up logfiles +loglevel=error ; info, debug, warn, trace +pidfile=/var/run/supervisord.pid ; pidfile location +nodaemon=false ; run supervisord as a daemon +minfds=1024 ; number of startup file descriptors +minprocs=200 ; number of process descriptors +user=root ; default user +childlogdir=/var/log/supervisord/ ; where child log files will live + +[program:krb5-kdc] +command=service krb5-kdc start +autostart=true +autorestart=true + +[program:krb5-admin-server] +command=service krb5-admin-server start +autostart=true +autorestart=true + +[supervisorctl] diff --git a/scripts/ci/ldif/example.com.ldif b/scripts/ci/ldif/example.com.ldif deleted file mode 100644 index cda5d005616ef..0000000000000 --- a/scripts/ci/ldif/example.com.ldif +++ /dev/null @@ -1,24 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -dn: dc=example,dc=com -dc: example -description: LDAP Example -objectClass: dcObject -objectClass: organization -o: example diff --git a/scripts/ci/my.cnf b/scripts/ci/mysql/conf.d/airflow.cnf similarity index 99% rename from scripts/ci/my.cnf rename to scripts/ci/mysql/conf.d/airflow.cnf index 5ff549ca4ecff..4802d195a2b55 100644 --- a/scripts/ci/my.cnf +++ b/scripts/ci/mysql/conf.d/airflow.cnf @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/scripts/ci/ldif/users.example.com.ldif b/scripts/ci/openldap/ldif/01-users.example.com.ldif similarity index 99% rename from scripts/ci/ldif/users.example.com.ldif rename to scripts/ci/openldap/ldif/01-users.example.com.ldif index ed4a7ceb7632c..bf5baf5ad8a18 100644 --- a/scripts/ci/ldif/users.example.com.ldif +++ b/scripts/ci/openldap/ldif/01-users.example.com.ldif @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/scripts/ci/ldif/groups.example.com.ldif b/scripts/ci/openldap/ldif/02-groups.example.com.ldif similarity index 96% rename from scripts/ci/ldif/groups.example.com.ldif rename to scripts/ci/openldap/ldif/02-groups.example.com.ldif index 804dcb008bf2c..9d1deb349fa05 100644 --- a/scripts/ci/ldif/groups.example.com.ldif +++ b/scripts/ci/openldap/ldif/02-groups.example.com.ldif @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -27,7 +27,7 @@ dn: cn=group1,ou=groups,dc=example,dc=com objectclass: groupofnames cn: group1 description: Group 1 of users -# add the group members all of which are +# add the group members all of which are # assumed to exist under example member: cn=user1,dc=example,dc=com diff --git a/scripts/ci/ldif/manager.example.com.ldif b/scripts/ci/openldap/ldif/03-manager.example.com.ldif similarity index 99% rename from scripts/ci/ldif/manager.example.com.ldif rename to scripts/ci/openldap/ldif/03-manager.example.com.ldif index 1d06a4f2ebd6f..d4b90b73b8850 100644 --- a/scripts/ci/ldif/manager.example.com.ldif +++ b/scripts/ci/openldap/ldif/03-manager.example.com.ldif @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY diff --git a/init.sh b/scripts/ci/openldap/ldif/04-rootdn.ldif similarity index 84% rename from init.sh rename to scripts/ci/openldap/ldif/04-rootdn.ldif index 6f4adcad06752..59bd3e554db90 100644 --- a/init.sh +++ b/scripts/ci/openldap/ldif/04-rootdn.ldif @@ -1,5 +1,3 @@ -#!/usr/bin/env bash - # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -8,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -18,4 +16,9 @@ # specific language governing permissions and limitations # under the License. -source $AIRFLOW_HOME/env/bin/activate +dn: cn=config +changetype: modify + +dn: olcDatabase={1}{{ LDAP_BACKEND }},cn=config +replace: olcRootDN +olcRootDN: cn=Manager,{{ LDAP_BASE_DN }} diff --git a/scripts/ci/slapd.conf b/scripts/ci/openldap/slapd.conf similarity index 99% rename from scripts/ci/slapd.conf rename to scripts/ci/openldap/slapd.conf index 34df186e4a256..a404530b8d285 100644 --- a/scripts/ci/slapd.conf +++ b/scripts/ci/openldap/slapd.conf @@ -6,9 +6,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, # software distributed under the License is distributed on an # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -38,7 +38,6 @@ disallow bind_anon database hdb suffix "dc=example,dc=com" - rootdn "cn=Manager,dc=example,dc=com" rootpw insecure diff --git a/scripts/ci/run-ci.sh b/scripts/ci/run-ci.sh new file mode 100755 index 0000000000000..f2815bbd95979 --- /dev/null +++ b/scripts/ci/run-ci.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +set -x + +DIRNAME=$(cd "$(dirname "$0")"; pwd) +AIRFLOW_ROOT="$DIRNAME/../.." + +# Fix file permissions +sudo chown -R airflow.airflow . $HOME/.wheelhouse/ $HOME/.cache/pip + +if [[ $PYTHON_VERSION == '3' ]]; then + PIP=pip3 +else + PIP=pip +fi + +sudo $PIP install --upgrade pip +sudo $PIP install tox + +cd $AIRFLOW_ROOT && $PIP --version && tox --version + +if [ -z "$KUBERNETES_VERSION" ]; +then + tox -e $TOX_ENV +else + KUBERNETES_VERSION=${KUBERNETES_VERSION} $DIRNAME/kubernetes/setup_kubernetes.sh && \ + tox -e $TOX_ENV -- tests.contrib.minikube \ + --with-coverage \ + --cover-erase \ + --cover-html \ + --cover-package=airflow \ + --cover-html-dir=airflow/www/static/coverage \ + --with-ignore-docstrings \ + --rednose \ + --with-timer \ + -v \ + --logging-level=DEBUG +fi diff --git a/scripts/ci/setup_env.sh b/scripts/ci/setup_env.sh deleted file mode 100755 index a1d9a4c244109..0000000000000 --- a/scripts/ci/setup_env.sh +++ /dev/null @@ -1,166 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -set -o verbose - -MINIKDC_VERSION=2.7.1 - -HADOOP_DISTRO=${HADOOP_DISTRO:-"hdp"} - -ONLY_DOWNLOAD=${ONLY_DOWNLOAD:-false} -ONLY_EXTRACT=${ONLY_EXTRACT:-false} - -MINICLUSTER_URL=https://github.com/bolkedebruin/minicluster/releases/download/1.1/minicluster-1.1-SNAPSHOT-bin.zip - -HIVE_HOME=/tmp/hive - -while test $# -gt 0; do - case "$1" in - -h|--help) - echo "Setup environment for airflow tests" - echo " " - echo "options:" - echo -e "\t-h, --help show brief help" - echo -e "\t-o, --only-download just download hadoop tar(s)" - echo -e "\t-e, --only-extract just extract hadoop tar(s)" - echo -e "\t-d, --distro select distro (hdp|cdh)" - exit 0 - ;; - -o|--only-download) - shift - ONLY_DOWNLOAD=true - ;; - -e|--only-extract) - shift - ONLY_EXTRACT=true - ;; - -d|--distro) - shift - if test $# -gt 0; then - HADOOP_DISTRO=$1 - else - echo "No Hadoop distro specified - abort" >&2 - exit 1 - fi - shift - ;; - *) - echo "Unknown options: $1" >&2 - exit 1 - ;; - esac -done - -HADOOP_HOME=/tmp/hadoop-${HADOOP_DISTRO} -MINICLUSTER_HOME=/tmp/minicluster - -if $ONLY_DOWNLOAD && $ONLY_EXTRACT; then - echo "Both only-download and only-extract specified - abort" >&2 - exit 1 -fi - -mkdir -p ${HADOOP_HOME} -mkdir -p ${TRAVIS_CACHE}/${HADOOP_DISTRO} -mkdir -p ${TRAVIS_CACHE}/minicluster -mkdir -p ${TRAVIS_CACHE}/hive -mkdir -p ${HIVE_HOME} -chmod -R 777 ${HIVE_HOME} -sudo mkdir -p /user/hive/warehouse -sudo chown -R ${USER} /user/ -sudo chmod -R 777 /user/ -ls -l / - -if [ $HADOOP_DISTRO = "cdh" ]; then - # URL="http://archive.cloudera.com/cdh5/cdh/5/hadoop-latest.tar.gz" - URL="https://archive.cloudera.com/cdh5/cdh/5/hadoop-2.6.0-cdh5.11.0.tar.gz" - # HIVE_URL="http://archive.cloudera.com/cdh5/cdh/5/hive-latest.tar.gz" - HIVE_URL="https://archive.cloudera.com/cdh5/cdh/5/hive-1.1.0-cdh5.11.0.tar.gz" -elif [ $HADOOP_DISTRO = "hdp" ]; then - URL="http://public-repo-1.hortonworks.com/HDP/centos6/2.x/updates/2.3.2.0/tars/hadoop-2.7.1.2.3.2.0-2950.tar.gz" - HIVE_URL="http://public-repo-1.hortonworks.com/HDP/centos6/2.x/updates/2.3.2.0/tars/apache-hive-1.2.1.2.3.2.0-2950-bin.tar.gz" -else - echo "No/bad HADOOP_DISTRO='${HADOOP_DISTRO}' specified" >&2 - exit 1 -fi - -if ! $ONLY_EXTRACT; then - echo "Downloading Hadoop from $URL to ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz" - curl -z ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz -o ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz -L $URL - - if [ $? != 0 ]; then - echo "Failed to download Hadoop from $URL - abort" >&2 - exit 1 - fi -fi - -if $ONLY_DOWNLOAD; then - exit 0 -fi - -echo "Extracting ${HADOOP_HOME}/hadoop.tar.gz into $HADOOP_HOME" -tar zxf ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz --strip-components 1 -C $HADOOP_HOME - -if [ $? != 0 ]; then - echo "Failed to extract Hadoop from ${HADOOP_HOME}/hadoop.tar.gz to ${HADOOP_HOME} - abort" >&2 - echo "Trying again..." >&2 - # dont use cache - curl -o ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz -L $URL - tar zxf ${TRAVIS_CACHE}/${HADOOP_DISTRO}/hadoop.tar.gz --strip-components 1 -C $HADOOP_HOME - if [ $? != 0 ]; then - echo "Failed twice in downloading and unpacking hadoop!" >&2 - exit 1 - fi -fi - -echo "Downloading and unpacking hive" -curl -z ${TRAVIS_CACHE}/hive/hive.tar.gz -o ${TRAVIS_CACHE}/hive/hive.tar.gz -L ${HIVE_URL} -tar zxf ${TRAVIS_CACHE}/hive/hive.tar.gz --strip-components 1 -C ${HIVE_HOME} - -if [ $? != 0 ]; then - echo "Failed to extract hive from ${TRAVIS_CACHE}/hive/hive.tar.gz" >&2 - echo "Trying again..." >&2 - # dont use cache - curl -o ${TRAVIS_CACHE}/hive/hive.tar.gz -L ${HIVE_URL} - tar zxf ${TRAVIS_CACHE}/hive/hive.tar.gz --strip-components 1 -C ${HIVE_HOME} - if [ $? != 0 ]; then - echo "Failed twice in downloading and unpacking hive!" >&2 - exit 1 - fi -fi - -echo "Downloading and unpacking minicluster" -curl -z ${TRAVIS_CACHE}/minicluster/minicluster.zip -o ${TRAVIS_CACHE}/minicluster/minicluster.zip -L ${MINICLUSTER_URL} -ls -l ${TRAVIS_CACHE}/minicluster/minicluster.zip -unzip ${TRAVIS_CACHE}/minicluster/minicluster.zip -d /tmp -if [ $? != 0 ] ; then - # Try downloading w/o cache if there's a failure - curl -o ${TRAVIS_CACHE}/minicluster/minicluster.zip -L ${MINICLUSTER_URL} - ls -l ${TRAVIS_CACHE}/minicluster/minicluster.zip - unzip ${TRAVIS_CACHE}/minicluster/minicluster.zip -d /tmp - if [ $? != 0 ] ; then - echo "Failed twice in downloading and unpacking minicluster!" >&2 - exit 1 - fi - exit 1 -fi - -echo "Path = ${PATH}" - -java -cp "/tmp/minicluster-1.1-SNAPSHOT/*" com.ing.minicluster.MiniCluster > /dev/null & diff --git a/scripts/ci/test-environment.sh b/scripts/ci/test-environment.sh new file mode 100644 index 0000000000000..5c402d46df287 --- /dev/null +++ b/scripts/ci/test-environment.sh @@ -0,0 +1,19 @@ +#!/bin/bash -e +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +docker-compose -f scripts/ci/docker-compose.yml run --rm airflow-testing "${@-bash}" diff --git a/tests/contrib/hooks/test_cassandra_hook.py b/tests/contrib/hooks/test_cassandra_hook.py index 9cb0739993173..73dac4f3b42ba 100644 --- a/tests/contrib/hooks/test_cassandra_hook.py +++ b/tests/contrib/hooks/test_cassandra_hook.py @@ -42,7 +42,7 @@ def setUp(self): db.merge_conn( models.Connection( conn_id='cassandra_default_with_schema', conn_type='cassandra', - host='localhost', port='9042', schema='s')) + host='cassandra', port='9042', schema='s')) hook = CassandraHook("cassandra_default") session = hook.get_conn() diff --git a/tests/contrib/sensors/test_mongo_sensor.py b/tests/contrib/sensors/test_mongo_sensor.py index 876cb99ba4251..6a78b7d146498 100644 --- a/tests/contrib/sensors/test_mongo_sensor.py +++ b/tests/contrib/sensors/test_mongo_sensor.py @@ -38,7 +38,7 @@ def setUp(self): db.merge_conn( Connection( conn_id='mongo_test', conn_type='mongo', - host='localhost', port='27017', schema='test')) + host='mongo', port='27017', schema='test')) args = { 'owner': 'airflow', diff --git a/tests/core.py b/tests/core.py index fd0d803eee074..5a78244184d04 100644 --- a/tests/core.py +++ b/tests/core.py @@ -2187,7 +2187,7 @@ def setUp(self): configuration.conf.add_section("ldap") except: pass - configuration.conf.set("ldap", "uri", "ldap://localhost:3890") + configuration.conf.set("ldap", "uri", "ldap://openldap:389") configuration.conf.set("ldap", "user_filter", "objectClass=*") configuration.conf.set("ldap", "user_name_attr", "uid") configuration.conf.set("ldap", "bind_user", "cn=Manager,dc=example,dc=com") @@ -2274,7 +2274,7 @@ def setUp(self): configuration.conf.add_section("ldap") except: pass - configuration.conf.set("ldap", "uri", "ldap://localhost:3890") + configuration.conf.set("ldap", "uri", "ldap://openldap:389") configuration.conf.set("ldap", "user_filter", "objectClass=*") configuration.conf.set("ldap", "user_name_attr", "uid") configuration.conf.set("ldap", "bind_user", "cn=Manager,dc=example,dc=com") diff --git a/tox.ini b/tox.ini index eba94a89e577d..fdc375ea99b8c 100644 --- a/tox.ini +++ b/tox.ini @@ -43,39 +43,37 @@ setenv = HADOOP_DISTRO=cdh HADOOP_HOME=/tmp/hadoop-cdh HADOOP_OPTS=-D/tmp/krb5.conf - MINICLUSTER_HOME=/tmp/minicluster-1.1-SNAPSHOT HIVE_HOME=/tmp/hive - backend_mysql: AIRFLOW__CORE__SQL_ALCHEMY_CONN=mysql://root@localhost/airflow - backend_postgres: AIRFLOW__CORE__SQL_ALCHEMY_CONN=postgresql+psycopg2://postgres@localhost/airflow - backend_sqlite: AIRFLOW__CORE__SQL_ALCHEMY_CONN=sqlite:///{homedir}/airflow/airflow.db + MINICLUSTER_HOME=/tmp/minicluster-1.1-SNAPSHOT + KRB5_CONFIG=/etc/krb5.conf + KRB5_KTNAME=/etc/airflow.keytab + backend_mysql: AIRFLOW__CORE__SQL_ALCHEMY_CONN=mysql://root@mysql/airflow + backend_postgres: AIRFLOW__CORE__SQL_ALCHEMY_CONN=postgresql+psycopg2://postgres:airflow@postgres/airflow + backend_sqlite: AIRFLOW__CORE__SQL_ALCHEMY_CONN=sqlite:///{homedir}/airflow.db backend_sqlite: AIRFLOW__CORE__EXECUTOR=SequentialExecutor passenv = HOME JAVA_HOME + USER + PATH + BOTO_CONFIG TRAVIS TRAVIS_BRANCH TRAVIS_BUILD_DIR TRAVIS_JOB_ID - USER - TRAVIS_CACHE TRAVIS_PULL_REQUEST - PATH - BOTO_CONFIG - KRB5_CONFIG - KRB5_KTNAME SLUGIFY_USES_TEXT_UNIDECODE commands = - pip wheel -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci] - pip install --find-links={homedir}/.wheelhouse --no-index -e .[devel_ci] - sudo {toxinidir}/scripts/ci/setup_kdc.sh - {toxinidir}/scripts/ci/setup_env.sh - {toxinidir}/scripts/ci/ldap.sh - {toxinidir}/scripts/ci/load_fixtures.sh - {toxinidir}/scripts/ci/load_data.sh - {toxinidir}/scripts/ci/run_tests.sh [] - {toxinidir}/scripts/ci/check-license.sh + pip wheel --progress-bar off -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci] + pip install --progress-bar off --find-links={homedir}/.wheelhouse --no-index -e .[devel_ci] + {toxinidir}/scripts/ci/1-setup-env.sh + {toxinidir}/scripts/ci/2-setup-kdc.sh + {toxinidir}/scripts/ci/3-setup-databases.sh + {toxinidir}/scripts/ci/4-load-data.sh + {toxinidir}/scripts/ci/5-run-tests.sh [] + {toxinidir}/scripts/ci/6-check-license.sh [testenv:flake8] basepython = python3 @@ -84,5 +82,4 @@ deps = flake8==3.5.0 commands = - {toxinidir}/scripts/ci/flake8_diff.sh - + {toxinidir}/scripts/ci/flake8-diff.sh From 213061973bfec93a8bab42a5d267b317b85cc465 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sat, 25 Aug 2018 19:50:16 +0200 Subject: [PATCH 351/808] [AIRFLOW-2933] Enable Codecov on Docker-CI Build (#3780) - Add missing variables and use codecov instead of coveralls. The issue why it wasn't working was because missing environment variables. The codecov library heavily depends on the environment variables in the CI to determine how to push the reports to codecov. - Remove the explicit passing of the variables in the `tox.ini` since it is already done in the `docker-compose.yml`, having to maintain this at two places makes it brittle. - Removed the empty Codecov yml since codecov was complaining that it was unable to parse it --- .codecov.yml | 19 ------------------- .travis.yml | 6 +----- scripts/ci/docker-compose.yml | 6 ++++++ tox.ini | 16 +++------------- 4 files changed, 10 insertions(+), 37 deletions(-) delete mode 100644 .codecov.yml diff --git a/.codecov.yml b/.codecov.yml deleted file mode 100644 index 4094f35dccf0d..0000000000000 --- a/.codecov.yml +++ /dev/null @@ -1,19 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# keep default diff --git a/.travis.yml b/.travis.yml index 9c7cfd02084db..5bd750453a563 100644 --- a/.travis.yml +++ b/.travis.yml @@ -42,7 +42,7 @@ cache: before_install: - sudo ls -lh $HOME/.cache/pip/ - sudo rm -rf $HOME/.cache/pip/* $HOME/.wheelhouse/* - - sudo chown -R travis.travis $HOME/.cache/pip + - sudo chown -R travis:travis $HOME/.cache/pip install: # Use recent docker-compose version - sudo rm /usr/local/bin/docker-compose @@ -50,9 +50,5 @@ install: - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin - pip install --upgrade pip - - pip install codecov script: - docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh -after_success: - - sudo chown -R travis.travis . - - codecov diff --git a/scripts/ci/docker-compose.yml b/scripts/ci/docker-compose.yml index 861cf9e8b89ce..32e0a536c1597 100644 --- a/scripts/ci/docker-compose.yml +++ b/scripts/ci/docker-compose.yml @@ -70,11 +70,17 @@ services: - SLUGIFY_USES_TEXT_UNIDECODE=yes - TOX_ENV - PYTHON_VERSION + - CI - TRAVIS - TRAVIS_BRANCH - TRAVIS_BUILD_DIR - TRAVIS_JOB_ID + - TRAVIS_JOB_NUMBER - TRAVIS_PULL_REQUEST + - TRAVIS_COMMIT + - TRAVIS_REPO_SLUG + - TRAVIS_OS_NAME + - TRAVIS_TAG depends_on: - postgres - mysql diff --git a/tox.ini b/tox.ini index fdc375ea99b8c..77a3de9865f13 100644 --- a/tox.ini +++ b/tox.ini @@ -33,7 +33,7 @@ ignore = E731,W503 [testenv] deps = wheel - coveralls + codecov basepython = py27: python2.7 @@ -52,18 +52,7 @@ setenv = backend_sqlite: AIRFLOW__CORE__SQL_ALCHEMY_CONN=sqlite:///{homedir}/airflow.db backend_sqlite: AIRFLOW__CORE__EXECUTOR=SequentialExecutor -passenv = - HOME - JAVA_HOME - USER - PATH - BOTO_CONFIG - TRAVIS - TRAVIS_BRANCH - TRAVIS_BUILD_DIR - TRAVIS_JOB_ID - TRAVIS_PULL_REQUEST - SLUGIFY_USES_TEXT_UNIDECODE +passenv = * commands = pip wheel --progress-bar off -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci] @@ -74,6 +63,7 @@ commands = {toxinidir}/scripts/ci/4-load-data.sh {toxinidir}/scripts/ci/5-run-tests.sh [] {toxinidir}/scripts/ci/6-check-license.sh + codecov -e TOXENV [testenv:flake8] basepython = python3 From 5f8972afcaabf7f285e374ef63e0db3fd4b71f88 Mon Sep 17 00:00:00 2001 From: Riccardo Bini Date: Fri, 21 Sep 2018 14:36:09 +0200 Subject: [PATCH 352/808] [AIRFLOW-2952] Fix Kubernetes CI (#3922) --- .travis.yml | 32 ++++++++----- airflow/example_dags_kubernetes/__init__.py | 18 +++++++ .../example_kubernetes_annotation.py | 47 +++++++++++++++++++ scripts/ci/docker-compose-kubernetes.yml | 28 +++++++++++ scripts/ci/kubernetes/docker/build.sh | 10 ++-- scripts/ci/kubernetes/kube/airflow.yaml | 1 - scripts/ci/kubernetes/kube/configmaps.yaml | 2 +- scripts/ci/kubernetes/kube/deploy.sh | 2 + .../ci/kubernetes/minikube/start_minikube.sh | 4 +- scripts/ci/kubernetes/setup_kubernetes.sh | 1 - scripts/ci/run-ci.sh | 10 +++- .../minikube/test_kubernetes_executor.py | 24 ++++++---- .../minikube/test_kubernetes_pod_operator.py | 13 +++-- tests/core.py | 4 +- tox.ini | 12 ++--- 15 files changed, 167 insertions(+), 41 deletions(-) create mode 100644 airflow/example_dags_kubernetes/__init__.py create mode 100644 airflow/example_dags_kubernetes/example_kubernetes_annotation.py create mode 100644 scripts/ci/docker-compose-kubernetes.yml diff --git a/.travis.yml b/.travis.yml index 5bd750453a563..dd493363ab1e7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -26,23 +26,24 @@ env: - TRAVIS_CACHE=$HOME/.travis_cache/ matrix: - TOX_ENV=flake8 - - TOX_ENV=py27-backend_mysql - - TOX_ENV=py27-backend_sqlite - - TOX_ENV=py27-backend_postgres - - TOX_ENV=py35-backend_mysql PYTHON_VERSION=3 - - TOX_ENV=py35-backend_sqlite PYTHON_VERSION=3 - - TOX_ENV=py35-backend_postgres PYTHON_VERSION=3 - - TOX_ENV=py27-backend_postgres KUBERNETES_VERSION=v1.9.0 - - TOX_ENV=py35-backend_postgres KUBERNETES_VERSION=v1.10.0 PYTHON_VERSION=3 + - TOX_ENV=py27-backend_mysql-env_docker + - TOX_ENV=py27-backend_sqlite-env_docker + - TOX_ENV=py27-backend_postgres-env_docker + - TOX_ENV=py35-backend_mysql-env_docker PYTHON_VERSION=3 + - TOX_ENV=py35-backend_sqlite-env_docker PYTHON_VERSION=3 + - TOX_ENV=py35-backend_postgres-env_docker PYTHON_VERSION=3 + - TOX_ENV=py27-backend_postgres-env_kubernetes KUBERNETES_VERSION=v1.9.0 + - TOX_ENV=py35-backend_postgres-env_kubernetes KUBERNETES_VERSION=v1.10.0 PYTHON_VERSION=3 + cache: directories: - $HOME/.wheelhouse/ - $HOME/.cache/pip - $HOME/.travis_cache/ before_install: - - sudo ls -lh $HOME/.cache/pip/ - - sudo rm -rf $HOME/.cache/pip/* $HOME/.wheelhouse/* - - sudo chown -R travis:travis $HOME/.cache/pip + # Required for K8s v1.10.x. See + # https://github.com/kubernetes/kubernetes/issues/61058#issuecomment-372764783 + - if [ ! -z "$KUBERNETES_VERSION" ]; then sudo mount --make-shared / && sudo service docker restart; fi install: # Use recent docker-compose version - sudo rm /usr/local/bin/docker-compose @@ -50,5 +51,12 @@ install: - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin - pip install --upgrade pip + - if [ ! -z "$KUBERNETES_VERSION" ]; then ./scripts/ci/kubernetes/setup_kubernetes.sh; fi script: - - docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh + - if [ -z "$KUBERNETES_VERSION" ]; then docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh; fi + - if [ ! -z "$KUBERNETES_VERSION" ]; then + ./scripts/ci/kubernetes/kube/deploy.sh && + MINIKUBE_IP=$(minikube ip) docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml -f scripts/ci/docker-compose-kubernetes.yml run airflow-testing /app/scripts/ci/run-ci.sh; + fi +before_cache: + - sudo chown -R travis:travis $HOME/.cache/pip $HOME/.wheelhouse/ diff --git a/airflow/example_dags_kubernetes/__init__.py b/airflow/example_dags_kubernetes/__init__.py new file mode 100644 index 0000000000000..114d189da14ab --- /dev/null +++ b/airflow/example_dags_kubernetes/__init__.py @@ -0,0 +1,18 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/airflow/example_dags_kubernetes/example_kubernetes_annotation.py b/airflow/example_dags_kubernetes/example_kubernetes_annotation.py new file mode 100644 index 0000000000000..058baf69901f7 --- /dev/null +++ b/airflow/example_dags_kubernetes/example_kubernetes_annotation.py @@ -0,0 +1,47 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import print_function +import airflow +from airflow.operators.python_operator import PythonOperator +from airflow.models import DAG + +args = { + 'owner': 'airflow', + 'start_date': airflow.utils.dates.days_ago(2) +} + +dag = DAG( + dag_id='example_kubernetes_annotation', default_args=args, + schedule_interval=None +) + + +def print_stuff(): + print("annotated!") + + +# You can use annotations on your kubernetes pods! +start_task = PythonOperator( + task_id="start_task", python_callable=print_stuff, dag=dag, + executor_config={ + "KubernetesExecutor": { + "annotations": {"test": "annotation"} + } + } +) diff --git a/scripts/ci/docker-compose-kubernetes.yml b/scripts/ci/docker-compose-kubernetes.yml new file mode 100644 index 0000000000000..0cdb2df199b7b --- /dev/null +++ b/scripts/ci/docker-compose-kubernetes.yml @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: "2.2" +services: + airflow-testing: + network_mode: host + environment: + - KUBERNETES_VERSION + - MINIKUBE_IP + volumes: + - /usr/local/bin/kubectl:/usr/local/bin/kubectl + - /usr/local/bin/minikube:/usr/local/bin/minikube + - ~/.kube:/home/airflow/.kube + - ~/.minikube:/home/airflow/.minikube diff --git a/scripts/ci/kubernetes/docker/build.sh b/scripts/ci/kubernetes/docker/build.sh index b93c6b1715ddb..49d0fb87c3a31 100755 --- a/scripts/ci/kubernetes/docker/build.sh +++ b/scripts/ci/kubernetes/docker/build.sh @@ -22,9 +22,13 @@ TAG=${2:-latest} DIRNAME=$(cd "$(dirname "$0")"; pwd) AIRFLOW_ROOT="$DIRNAME/../../../.." -ENVCONFIG=$(minikube docker-env) -if [ $? -eq 0 ]; then - eval $ENVCONFIG +set -e + +if [ "${VM_DRIVER:-none}" != "none" ]; then + ENVCONFIG=$(minikube docker-env) + if [ $? -eq 0 ]; then + eval $ENVCONFIG + fi fi echo "Airflow directory $AIRFLOW_ROOT" diff --git a/scripts/ci/kubernetes/kube/airflow.yaml b/scripts/ci/kubernetes/kube/airflow.yaml index 4f451ba44a687..1b6e33cec5b3c 100644 --- a/scripts/ci/kubernetes/kube/airflow.yaml +++ b/scripts/ci/kubernetes/kube/airflow.yaml @@ -151,4 +151,3 @@ spec: nodePort: 30809 selector: name: airflow - diff --git a/scripts/ci/kubernetes/kube/configmaps.yaml b/scripts/ci/kubernetes/kube/configmaps.yaml index b5fa3e5f639ee..86c1a9756134e 100644 --- a/scripts/ci/kubernetes/kube/configmaps.yaml +++ b/scripts/ci/kubernetes/kube/configmaps.yaml @@ -27,7 +27,7 @@ data: logging_level = INFO executor = KubernetesExecutor parallelism = 32 - load_examples = True + load_examples = False plugins_folder = /root/airflow/plugins sql_alchemy_conn = $SQL_ALCHEMY_CONN diff --git a/scripts/ci/kubernetes/kube/deploy.sh b/scripts/ci/kubernetes/kube/deploy.sh index a9a42a7a12d12..75567c9acf6a4 100755 --- a/scripts/ci/kubernetes/kube/deploy.sh +++ b/scripts/ci/kubernetes/kube/deploy.sh @@ -17,6 +17,8 @@ # specific language governing permissions and limitations * # under the License. * +set -x + IMAGE=${1:-airflow/ci} TAG=${2:-latest} DIRNAME=$(cd "$(dirname "$0")"; pwd) diff --git a/scripts/ci/kubernetes/minikube/start_minikube.sh b/scripts/ci/kubernetes/minikube/start_minikube.sh index 5171a26313d53..6f843b89ec68f 100755 --- a/scripts/ci/kubernetes/minikube/start_minikube.sh +++ b/scripts/ci/kubernetes/minikube/start_minikube.sh @@ -19,11 +19,13 @@ #!/usr/bin/env bash +set -x _MY_SCRIPT="${BASH_SOURCE[0]}" _MY_DIR=$(cd "$(dirname "$_MY_SCRIPT")" && pwd) # Avoids 1.7.x because of https://github.com/kubernetes/minikube/issues/2240 _KUBERNETES_VERSION="${KUBERNETES_VERSION}" +_MINIKUBE_VERSION="${MINIKUBE_VERSION:-v0.28.2}" echo "setting up kubernetes ${_KUBERNETES_VERSION}" @@ -118,7 +120,7 @@ echo Showing kube-system pods kubectl get -n kube-system pods (k8s_single_pod_ready -n kube-system -l component=kube-addon-manager) || - (_ADDON=$(kubectl get pod -n kube-system -l component=kube-addon-manager + (_ADDON=$(kubectl get pod -n kube-system -l component=kube-addon-manager \ --no-headers -o name| cut -d/ -f2); echo Addon-manager describe:; kubectl describe pod -n kube-system $_ADDON; diff --git a/scripts/ci/kubernetes/setup_kubernetes.sh b/scripts/ci/kubernetes/setup_kubernetes.sh index ea559a02aa997..96f8f5d6af2a5 100755 --- a/scripts/ci/kubernetes/setup_kubernetes.sh +++ b/scripts/ci/kubernetes/setup_kubernetes.sh @@ -25,6 +25,5 @@ DIRNAME=$(cd "$(dirname "$0")"; pwd) $DIRNAME/minikube/start_minikube.sh $DIRNAME/docker/build.sh -$DIRNAME/kube/deploy.sh echo "Airflow environment on kubernetes is good to go!" diff --git a/scripts/ci/run-ci.sh b/scripts/ci/run-ci.sh index f2815bbd95979..1a65bf1ef6443 100755 --- a/scripts/ci/run-ci.sh +++ b/scripts/ci/run-ci.sh @@ -24,7 +24,7 @@ DIRNAME=$(cd "$(dirname "$0")"; pwd) AIRFLOW_ROOT="$DIRNAME/../.." # Fix file permissions -sudo chown -R airflow.airflow . $HOME/.wheelhouse/ $HOME/.cache/pip +sudo chown -R airflow.airflow . $HOME/.cache $HOME/.wheelhouse/ $HOME/.cache/pip $HOME/.kube $HOME/.minikube if [[ $PYTHON_VERSION == '3' ]]; then PIP=pip3 @@ -41,7 +41,13 @@ if [ -z "$KUBERNETES_VERSION" ]; then tox -e $TOX_ENV else - KUBERNETES_VERSION=${KUBERNETES_VERSION} $DIRNAME/kubernetes/setup_kubernetes.sh && \ + # This script runs inside a container, the path of the kubernetes certificate + # is /home/travis/.minikube/client.crt but the user in the container is `airflow` + if [ ! -d /home/travis ]; then + sudo mkdir -p /home/travis + fi + sudo ln -s /home/airflow/.minikube /home/travis/.minikube + tox -e $TOX_ENV -- tests.contrib.minikube \ --with-coverage \ --cover-erase \ diff --git a/tests/contrib/minikube/test_kubernetes_executor.py b/tests/contrib/minikube/test_kubernetes_executor.py index a8db7e12b7554..7d3947799dcd5 100644 --- a/tests/contrib/minikube/test_kubernetes_executor.py +++ b/tests/contrib/minikube/test_kubernetes_executor.py @@ -16,6 +16,7 @@ # under the License. +import os import unittest from subprocess import check_call, check_output import requests.exceptions @@ -25,18 +26,25 @@ import re try: - check_call(["kubectl", "get", "pods"]) + check_call(["/usr/local/bin/kubectl", "get", "pods"]) except Exception as e: - raise unittest.SkipTest( - "Kubernetes integration tests require a minikube cluster;" - "Skipping tests {}".format(e) - ) + if os.environ.get('KUBERNETES_VERSION'): + raise e + else: + raise unittest.SkipTest( + "Kubernetes integration tests require a minikube cluster;" + "Skipping tests {}".format(e) + ) def get_minikube_host(): - host_ip = check_output(['minikube', 'ip']) - if six.PY3: - host_ip = host_ip.decode('UTF-8') + if "MINIKUBE_IP" in os.environ: + host_ip = os.environ['MINIKUBE_IP'] + else: + host_ip = check_output(['/usr/local/bin/minikube', 'ip']) + if six.PY3: + host_ip = host_ip.decode('UTF-8') + host = '{}:30809'.format(host_ip.strip()) return host diff --git a/tests/contrib/minikube/test_kubernetes_pod_operator.py b/tests/contrib/minikube/test_kubernetes_pod_operator.py index 6ab69f31c6927..f808a2f47febe 100644 --- a/tests/contrib/minikube/test_kubernetes_pod_operator.py +++ b/tests/contrib/minikube/test_kubernetes_pod_operator.py @@ -29,12 +29,15 @@ from airflow.contrib.kubernetes.volume import Volume try: - check_call(["kubectl", "get", "pods"]) + check_call(["/usr/local/bin/kubectl", "get", "pods"]) except Exception as e: - raise unittest.SkipTest( - "Kubernetes integration tests require a minikube cluster;" - "Skipping tests {}".format(e) - ) + if os.environ.get('KUBERNETES_VERSION'): + raise e + else: + raise unittest.SkipTest( + "Kubernetes integration tests require a minikube cluster;" + "Skipping tests {}".format(e) + ) class KubernetesPodOperatorTest(unittest.TestCase): diff --git a/tests/core.py b/tests/core.py index 5a78244184d04..7b29bc6aad0de 100644 --- a/tests/core.py +++ b/tests/core.py @@ -98,7 +98,9 @@ def reset(dag_id=TEST_DAG_ID): session.close() -reset() +configuration.conf.load_test_config() +if os.environ.get('KUBERNETES_VERSION') is None: + reset() class OperatorSubclass(BaseOperator): diff --git a/tox.ini b/tox.ini index 77a3de9865f13..acbeb970e7ffb 100644 --- a/tox.ini +++ b/tox.ini @@ -17,8 +17,8 @@ # under the License. [tox] -envlist = flake8,{py27,py35}-backend_{mysql,sqlite,postgres} -skipsdist=True +envlist = flake8,{py27,py35}-backend_{mysql,sqlite,postgres}-env_{docker,kubernetes} +skipsdist = True [global] wheel_dir = {homedir}/.wheelhouse @@ -57,10 +57,10 @@ passenv = * commands = pip wheel --progress-bar off -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci] pip install --progress-bar off --find-links={homedir}/.wheelhouse --no-index -e .[devel_ci] - {toxinidir}/scripts/ci/1-setup-env.sh - {toxinidir}/scripts/ci/2-setup-kdc.sh - {toxinidir}/scripts/ci/3-setup-databases.sh - {toxinidir}/scripts/ci/4-load-data.sh + env_docker: {toxinidir}/scripts/ci/1-setup-env.sh + env_docker: {toxinidir}/scripts/ci/2-setup-kdc.sh + env_docker: {toxinidir}/scripts/ci/3-setup-databases.sh + env_docker: {toxinidir}/scripts/ci/4-load-data.sh {toxinidir}/scripts/ci/5-run-tests.sh [] {toxinidir}/scripts/ci/6-check-license.sh codecov -e TOXENV From 417d3be9523635cdadcef303a2800369022e3b0c Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 5 Sep 2018 00:46:41 +0100 Subject: [PATCH 353/808] [AIRFLOW-3008] Move Kubernetes example DAGs to contrib --- .../example_kubernetes_annotation.py | 0 airflow/example_dags_kubernetes/__init__.py | 18 ------------------ 2 files changed, 18 deletions(-) rename airflow/{example_dags_kubernetes => contrib/example_dags}/example_kubernetes_annotation.py (100%) delete mode 100644 airflow/example_dags_kubernetes/__init__.py diff --git a/airflow/example_dags_kubernetes/example_kubernetes_annotation.py b/airflow/contrib/example_dags/example_kubernetes_annotation.py similarity index 100% rename from airflow/example_dags_kubernetes/example_kubernetes_annotation.py rename to airflow/contrib/example_dags/example_kubernetes_annotation.py diff --git a/airflow/example_dags_kubernetes/__init__.py b/airflow/example_dags_kubernetes/__init__.py deleted file mode 100644 index 114d189da14ab..0000000000000 --- a/airflow/example_dags_kubernetes/__init__.py +++ /dev/null @@ -1,18 +0,0 @@ -# -*- coding: utf-8 -*- -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. From 10808ce932d25e7d3a5a435a39227c9c1e5ef3d1 Mon Sep 17 00:00:00 2001 From: Kevin Yang Date: Thu, 28 Jun 2018 13:30:36 -0700 Subject: [PATCH 354/808] [AIRFLOW-2359] Add set failed for DagRun and task in tree view (#3255) --- airflow/api/common/experimental/mark_tasks.py | 113 ++++++- airflow/jobs.py | 3 +- airflow/models.py | 4 +- airflow/www/templates/airflow/dag.html | 43 +++ airflow/www/views.py | 292 +++++++++++++----- airflow/www_rbac/templates/airflow/dag.html | 43 +++ airflow/www_rbac/views.py | 235 ++++++++++---- docs/scheduler.rst | 1 + tests/api/common/experimental/mark_tasks.py | 275 +++++++++++------ tests/dags/test_example_bash_operator.py | 5 +- tests/www_rbac/test_views.py | 8 +- 11 files changed, 763 insertions(+), 259 deletions(-) diff --git a/airflow/api/common/experimental/mark_tasks.py b/airflow/api/common/experimental/mark_tasks.py index e9e4fec2235a5..681864dfbeb3e 100644 --- a/airflow/api/common/experimental/mark_tasks.py +++ b/airflow/api/common/experimental/mark_tasks.py @@ -17,15 +17,16 @@ # specific language governing permissions and limitations # under the License. +from sqlalchemy import or_ + from airflow.jobs import BackfillJob from airflow.models import DagRun, TaskInstance from airflow.operators.subdag_operator import SubDagOperator from airflow.settings import Session from airflow.utils import timezone +from airflow.utils.db import provide_session from airflow.utils.state import State -from sqlalchemy import or_ - def _create_dagruns(dag, execution_dates, state, run_id_template): """ @@ -191,15 +192,36 @@ def set_state(task, execution_date, upstream=False, downstream=False, return tis_altered -def set_dag_run_state(dag, execution_date, state=State.SUCCESS, commit=False): +def _set_dag_run_state(dag_id, execution_date, state, session=None): + """ + Helper method that set dag run state in the DB. + :param dag_id: dag_id of target dag run + :param execution_date: the execution date from which to start looking + :param state: target state + :param session: database session """ - Set the state of a dag run and all task instances associated with the dag - run for a specific execution date. + DR = DagRun + dr = session.query(DR).filter( + DR.dag_id == dag_id, + DR.execution_date == execution_date + ).one() + dr.state = state + dr.end_date = timezone.utcnow() + session.commit() + + +@provide_session +def set_dag_run_state_to_success(dag, execution_date, commit=False, + session=None): + """ + Set the dag run for a specific execution date and its task instances + to success. :param dag: the DAG of which to alter state :param execution_date: the execution date from which to start looking - :param state: the state to which the DAG need to be set :param commit: commit DAG and tasks to be altered to the database - :return: list of tasks that have been created and updated + :param session: database session + :return: If commit is true, list of tasks that have been updated, + otherwise list of tasks that will be updated :raises: AssertionError if dag or execution_date is invalid """ res = [] @@ -207,18 +229,81 @@ def set_dag_run_state(dag, execution_date, state=State.SUCCESS, commit=False): if not dag or not execution_date: return res - # Mark all task instances in the dag run + # Mark the dag run to success. + if commit: + _set_dag_run_state(dag.dag_id, execution_date, State.SUCCESS, session) + + # Mark all task instances of the dag run to success. for task in dag.tasks: task.dag = dag new_state = set_state(task=task, execution_date=execution_date, - state=state, commit=commit) + state=State.SUCCESS, commit=commit) res.extend(new_state) - # Mark the dag run + return res + + +@provide_session +def set_dag_run_state_to_failed(dag, execution_date, commit=False, + session=None): + """ + Set the dag run for a specific execution date and its running task instances + to failed. + :param dag: the DAG of which to alter state + :param execution_date: the execution date from which to start looking + :param commit: commit DAG and tasks to be altered to the database + :param session: database session + :return: If commit is true, list of tasks that have been updated, + otherwise list of tasks that will be updated + :raises: AssertionError if dag or execution_date is invalid + """ + res = [] + + if not dag or not execution_date: + return res + + # Mark the dag run to failed. + if commit: + _set_dag_run_state(dag.dag_id, execution_date, State.FAILED, session) + + # Mark only RUNNING task instances. + TI = TaskInstance + task_ids = [task.task_id for task in dag.tasks] + tis = session.query(TI).filter( + TI.dag_id == dag.dag_id, + TI.execution_date == execution_date, + TI.task_id.in_(task_ids)).filter(TI.state == State.RUNNING) + task_ids_of_running_tis = [ti.task_id for ti in tis] + for task in dag.tasks: + if task.task_id not in task_ids_of_running_tis: + continue + task.dag = dag + new_state = set_state(task=task, execution_date=execution_date, + state=State.FAILED, commit=commit) + res.extend(new_state) + + return res + + +@provide_session +def set_dag_run_state_to_running(dag, execution_date, commit=False, + session=None): + """ + Set the dag run for a specific execution date to running. + :param dag: the DAG of which to alter state + :param execution_date: the execution date from which to start looking + :param commit: commit DAG and tasks to be altered to the database + :param session: database session + :return: If commit is true, list of tasks that have been updated, + otherwise list of tasks that will be updated + """ + res = [] + if not dag or not execution_date: + return res + + # Mark the dag run to running. if commit: - drs = DagRun.find(dag.dag_id, execution_date=execution_date) - for dr in drs: - dr.dag = dag - dr.update_state() + _set_dag_run_state(dag.dag_id, execution_date, State.RUNNING, session) + # To keep the return type consistent with the other similar functions. return res diff --git a/airflow/jobs.py b/airflow/jobs.py index 4daafc22ada18..0ad004809b7ff 100644 --- a/airflow/jobs.py +++ b/airflow/jobs.py @@ -1009,7 +1009,8 @@ def _change_state_for_tis_without_dagrun(self, models.TaskInstance.dag_id == subq.c.dag_id, models.TaskInstance.task_id == subq.c.task_id, models.TaskInstance.execution_date == - subq.c.execution_date)) \ + subq.c.execution_date, + models.TaskInstance.task_id == subq.c.task_id)) \ .update({models.TaskInstance.state: new_state}, synchronize_session=False) session.commit() diff --git a/airflow/models.py b/airflow/models.py index 8e54c9b2a1a9f..a631436c9a0f2 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1690,9 +1690,9 @@ def signal_handler(signum, frame): return except AirflowException as e: self.refresh_from_db() - # for case when task is marked as success externally + # for case when task is marked as success/failed externally # current behavior doesn't hit the success callback - if self.state == State.SUCCESS: + if self.state in {State.SUCCESS, State.FAILED}: return else: self.handle_failure(e, test_mode, context) diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index d202a977ee56f..c34ec375b6f78 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -197,6 +197,24 @@

    - {{ html_code|safe }} + {% if html_code is defined %} + {{ html_code|safe }} + {% endif %} {% endblock %} diff --git a/airflow/www/templates/airflow/ti_code.html b/airflow/www/templates/airflow/ti_code.html index 44942ca7d7e5d..72008f04682bb 100644 --- a/airflow/www/templates/airflow/ti_code.html +++ b/airflow/www/templates/airflow/ti_code.html @@ -21,23 +21,8 @@ {% block body %} {{ super() }}

    {{ title }}

    - {% if html_code %} - {{ html_code|safe }} - {% endif %} - {% if code %} -
    {{ code }}
    - {% endif %} - - {% if code_dict %} - {% for k, v in code_dict.items() %} -
    {{ k }}
    -
    {{ v }}
    - {% endfor %} - {% endif %} - {% if html_dict %} - {% for k, v in html_dict.items() %} -
    {{ k }}
    - {{ v|safe }} - {% endfor %} - {% endif %} + {% for k, v in html_dict.items() %} +
    {{ k }}
    + {{ v|safe }} + {% endfor %} {% endblock %} diff --git a/airflow/www/views.py b/airflow/www/views.py index 1fd4ba7df2b53..4cd66ff9241c4 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -676,6 +676,7 @@ def dag_details(self, session=None): dag_id = request.args.get('dag_id') dag = dagbag.get_dag(dag_id) title = "DAG details" + root = request.args.get('root', '') TI = models.TaskInstance states = session\ @@ -684,9 +685,17 @@ def dag_details(self, session=None): .group_by(TI.state)\ .all() + active_runs = models.DagRun.find( + dag_id=dag.dag_id, + state=State.RUNNING, + external_trigger=False, + session=session + ) + return self.render( 'airflow/dag_details.html', - dag=dag, title=title, states=states, State=State) + dag=dag, title=title, root=root, states=states, State=State, + active_runs=active_runs) @current_app.errorhandler(404) def circles(self): @@ -736,6 +745,7 @@ def rendered(self): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dag = dagbag.get_dag(dag_id) task = copy.copy(dag.get_task(task_id)) ti = models.TaskInstance(task=task, execution_date=dttm) @@ -760,7 +770,8 @@ def rendered(self): task_id=task_id, execution_date=execution_date, form=form, - title=title, ) + root=root, + title=title) @expose('/get_logs_with_metadata') @login_required @@ -860,11 +871,13 @@ def log(self, session=None): # Tasks in reschedule state decremented the try number num_logs += 1 logs = [''] * num_logs + root = request.args.get('root', '') return self.render( 'airflow/ti_log.html', logs=logs, dag=dag, title="Log by attempts", dag_id=dag.dag_id, task_id=task_id, - execution_date=execution_date, form=form) + execution_date=execution_date, form=form, + root=root) @expose('/task') @login_required @@ -879,6 +892,7 @@ def task(self): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dag = dagbag.get_dag(dag_id) if not dag or task_id not in dag.task_ids: @@ -952,6 +966,7 @@ def task(self): execution_date=execution_date, special_attrs_rendered=special_attrs_rendered, form=form, + root=root, dag=dag, title=title) @expose('/xcom') @@ -966,6 +981,7 @@ def xcom(self, session=None): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dm_db = models.DagModel ti_db = models.TaskInstance dag = session.query(dm_db).filter(dm_db.dag_id == dag_id).first() @@ -993,6 +1009,7 @@ def xcom(self, session=None): task_id=task_id, execution_date=execution_date, form=form, + root=root, dag=dag, title=title) @expose('/run') diff --git a/airflow/www_rbac/templates/airflow/config.html b/airflow/www_rbac/templates/airflow/config.html index 501ef66ec1873..a68c5a7aef49a 100644 --- a/airflow/www_rbac/templates/airflow/config.html +++ b/airflow/www_rbac/templates/airflow/config.html @@ -33,10 +33,6 @@

    {{ title }}

    {{ subtitle }}
    {% endif %} - {% if code %} -
    {{ code }}
    - {% endif %} - {% if code_html %} {{ code_html|safe }} {% endif %} diff --git a/airflow/www_rbac/templates/airflow/dag.html b/airflow/www_rbac/templates/airflow/dag.html index a58782d052068..685b7e9441284 100644 --- a/airflow/www_rbac/templates/airflow/dag.html +++ b/airflow/www_rbac/templates/airflow/dag.html @@ -510,7 +510,7 @@

    {{ title }}

    - {% if html_code %} - {{ html_code|safe }} - {% endif %} - {% if code %} -
    {{ code }}
    - {% endif %} - - {% if code_dict %} - {% for k, v in code_dict.items() %} -
    {{ k }}
    -
    {{ v }}
    - {% endfor %} - {% endif %} - {% if html_dict %} - {% for k, v in html_dict.items() %} -
    {{ k }}
    - {{ v|safe }} - {% endfor %} - {% endif %} + {{ html_code|safe }} {% endblock %} {% block tail %} diff --git a/airflow/www_rbac/templates/airflow/dag_details.html b/airflow/www_rbac/templates/airflow/dag_details.html index 2ef49ff16b760..37f51b53064ca 100644 --- a/airflow/www_rbac/templates/airflow/dag_details.html +++ b/airflow/www_rbac/templates/airflow/dag_details.html @@ -42,7 +42,7 @@

    {{ title }}

    max_active_runs - {{ dag.active_runs | length }} / {{ dag.max_active_runs }} + {{ active_runs | length }} / {{ dag.max_active_runs }} concurrency diff --git a/airflow/www_rbac/templates/airflow/dags.html b/airflow/www_rbac/templates/airflow/dags.html index dae0816507a4d..a0bdefab64e1f 100644 --- a/airflow/www_rbac/templates/airflow/dags.html +++ b/airflow/www_rbac/templates/airflow/dags.html @@ -74,8 +74,7 @@

    DAGs

    - {% set disabled = 'disabled' if view_only else None %} - + diff --git a/airflow/www_rbac/templates/airflow/graph.html b/airflow/www_rbac/templates/airflow/graph.html index 52c83ac0b8fa7..123d9e42a9212 100644 --- a/airflow/www_rbac/templates/airflow/graph.html +++ b/airflow/www_rbac/templates/airflow/graph.html @@ -27,7 +27,7 @@ {% block content %} {{ super() }} {% if dag.doc_md %} -
    {{ doc_md|safe }}
    +
    {{ dag.doc_md|safe }}
    {% endif %}
    diff --git a/airflow/www_rbac/templates/airflow/task.html b/airflow/www_rbac/templates/airflow/task.html index 6c843b28e3e5e..2b4356f49ec60 100644 --- a/airflow/www_rbac/templates/airflow/task.html +++ b/airflow/www_rbac/templates/airflow/task.html @@ -37,7 +37,9 @@
    Dependencies Blocking Task From Getting Scheduled
    {% endfor %} - {{ html_code|safe }} + {% if html_code is defined %} + {{ html_code|safe }} + {% endif %}
    {% for attr, value in special_attrs_rendered.items() %} @@ -70,6 +72,8 @@
    Task Attributes
    {% endfor %} - {{ html_code|safe }} + {% if html_code is defined %} + {{ html_code|safe }} + {% endif %}
    {% endblock %} diff --git a/airflow/www_rbac/templates/airflow/ti_code.html b/airflow/www_rbac/templates/airflow/ti_code.html index d38eb7d6c936d..8c7db0d9d415d 100644 --- a/airflow/www_rbac/templates/airflow/ti_code.html +++ b/airflow/www_rbac/templates/airflow/ti_code.html @@ -21,23 +21,8 @@ {% block content %} {{ super() }}

    {{ title }}

    - {% if html_code %} - {{ html_code|safe }} - {% endif %} - {% if code %} -
    {{ code }}
    - {% endif %} - - {% if code_dict %} - {% for k, v in code_dict.items() %} -
    {{ k }}
    -
    {{ v }}
    - {% endfor %} - {% endif %} - {% if html_dict %} - {% for k, v in html_dict.items() %} -
    {{ k }}
    - {{ v|safe }} - {% endfor %} - {% endif %} + {% for k, v in html_dict.items() %} +
    {{ k }}
    + {{ v|safe }} + {% endfor %} {% endblock %} diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index b532befa03625..0a1e18ae827df 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -192,6 +192,10 @@ def index(self, session=None): 'hide_paused_dags_by_default') show_paused_arg = request.args.get('showPaused', 'None') + default_dag_run = conf.getint('webserver', 'default_dag_run_display_number') + num_runs = request.args.get('num_runs') + num_runs = int(num_runs) if num_runs else default_dag_run + def get_int_arg(value, default=0): try: return int(value) @@ -272,7 +276,8 @@ def get_int_arg(value, default=0): paging=wwwutils.generate_pages(current_page, num_of_pages, search=arg_search_query, showPaused=not hide_paused), - auto_complete_data=auto_complete_data) + auto_complete_data=auto_complete_data, + num_runs=num_runs) @expose('/dag_stats') @has_access @@ -416,6 +421,7 @@ def dag_details(self, session=None): dag_id = request.args.get('dag_id') dag = dagbag.get_dag(dag_id) title = "DAG details" + root = request.args.get('root', '') TI = models.TaskInstance states = ( @@ -424,9 +430,17 @@ def dag_details(self, session=None): .group_by(TI.state) .all() ) + + active_runs = models.DagRun.find( + dag_id=dag.dag_id, + state=State.RUNNING, + external_trigger=False, + session=session + ) + return self.render( 'airflow/dag_details.html', - dag=dag, title=title, states=states, State=State) + dag=dag, title=title, root=root, states=states, State=State, active_runs=active_runs) @app.errorhandler(404) def circles(self): @@ -467,6 +481,7 @@ def rendered(self): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dag = dagbag.get_dag(dag_id) task = copy.copy(dag.get_task(task_id)) ti = models.TaskInstance(task=task, execution_date=dttm) @@ -492,7 +507,8 @@ def rendered(self): task_id=task_id, execution_date=execution_date, form=form, - title=title, ) + root=root, + title=title) @expose('/get_logs_with_metadata') @has_dag_access(can_dag_read=True) @@ -593,11 +609,13 @@ def log(self, session=None): # Tasks in reschedule state decremented the try number num_logs += 1 logs = [''] * num_logs + root = request.args.get('root', '') return self.render( 'airflow/ti_log.html', logs=logs, dag=dag, title="Log by attempts", dag_id=dag.dag_id, task_id=task_id, - execution_date=execution_date, form=form) + execution_date=execution_date, form=form, + root=root) @expose('/task') @has_dag_access(can_dag_read=True) @@ -613,6 +631,7 @@ def task(self): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dag = dagbag.get_dag(dag_id) if not dag or task_id not in dag.task_ids: @@ -675,6 +694,7 @@ def task(self): execution_date=execution_date, special_attrs_rendered=special_attrs_rendered, form=form, + root=root, dag=dag, title=title) @expose('/xcom') @@ -690,6 +710,7 @@ def xcom(self, session=None): execution_date = request.args.get('execution_date') dttm = pendulum.parse(execution_date) form = DateTimeForm(data={'execution_date': dttm}) + root = request.args.get('root', '') dm_db = models.DagModel ti_db = models.TaskInstance dag = session.query(dm_db).filter(dm_db.dag_id == dag_id).first() @@ -718,6 +739,7 @@ def xcom(self, session=None): task_id=task_id, execution_date=execution_date, form=form, + root=root, dag=dag, title=title) @expose('/run') From 935f7f755c2bcf0fcdac30ec215aac2fde83db5c Mon Sep 17 00:00:00 2001 From: Peter van 't Hof Date: Tue, 5 Mar 2019 14:04:39 +0100 Subject: [PATCH 769/808] [AIRFLOW-3830] Remove DagBag from /dag_details (#4831) * Remove DagBag from /dag_details * Fixing method call * Adding sync to db calls --- airflow/models/__init__.py | 5 +++++ airflow/www_rbac/views.py | 11 ++++++----- tests/www_rbac/test_views.py | 3 +++ 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index cb1c0a25d3046..3d2cbae33f2e7 100755 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -2983,6 +2983,11 @@ def __repr__(self): def timezone(self): return settings.TIMEZONE + @staticmethod + @provide_session + def get_dagmodel(dag_id, session=None): + return session.query(DagModel).filter(DagModel.dag_id == dag_id).first() + @classmethod @provide_session def get_current(cls, dag_id, session=None): diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 0a1e18ae827df..96f199667a316 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -53,7 +53,7 @@ from airflow import settings from airflow.api.common.experimental.mark_tasks import (set_dag_run_state_to_success, set_dag_run_state_to_failed) -from airflow.models import XCom, DagRun, errors +from airflow.models import XCom, DagRun, errors, DagModel from airflow.models.connection import Connection from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, SCHEDULER_DEPS from airflow.utils import timezone @@ -419,7 +419,9 @@ def code(self, session=None): @provide_session def dag_details(self, session=None): dag_id = request.args.get('dag_id') - dag = dagbag.get_dag(dag_id) + dag_orm = DagModel.get_dagmodel(dag_id) + # FIXME: items needed for this view should move to the database + dag = dag_orm.get_dag() title = "DAG details" root = request.args.get('root', '') @@ -432,10 +434,9 @@ def dag_details(self, session=None): ) active_runs = models.DagRun.find( - dag_id=dag.dag_id, + dag_id=dag_id, state=State.RUNNING, - external_trigger=False, - session=session + external_trigger=False ) return self.render( diff --git a/tests/www_rbac/test_views.py b/tests/www_rbac/test_views.py index 7078103b1574c..bd704ec5d2392 100644 --- a/tests/www_rbac/test_views.py +++ b/tests/www_rbac/test_views.py @@ -325,8 +325,11 @@ def cleanup_dagruns(self): def prepare_dagruns(self): dagbag = models.DagBag(include_examples=True) self.bash_dag = dagbag.dags['example_bash_operator'] + self.bash_dag.sync_to_db() self.sub_dag = dagbag.dags['example_subdag_operator'] + self.sub_dag.sync_to_db() self.xcom_dag = dagbag.dags['example_xcom'] + self.xcom_dag.sync_to_db() self.bash_dagrun = self.bash_dag.create_dagrun( run_id=self.run_id, From 68fc6dc17bad7a2c1689135904a50979c359b717 Mon Sep 17 00:00:00 2001 From: Peter van 't Hof Date: Mon, 11 Mar 2019 12:21:27 +0100 Subject: [PATCH 770/808] [AIRFLOW-4058] Name models test file to get automatically picked up (#4901) We had `from .models import *` inside test/__init__.py as a kludge around this test file not being named according to expected Python conventions. Renaming the file makes more test tools happier (and makes it easier to run a single test file without importing half of the test tree which the current approach suffers from) --- tests/__init__.py | 1 - tests/cli/test_cli.py | 6 ++---- tests/core.py | 7 ------- tests/{models.py => test_models.py} | 24 +++++++++++------------- 4 files changed, 13 insertions(+), 25 deletions(-) rename tests/{models.py => test_models.py} (99%) diff --git a/tests/__init__.py b/tests/__init__.py index 6cc4996f5e2f7..b8630ce2b40e3 100644 --- a/tests/__init__.py +++ b/tests/__init__.py @@ -23,4 +23,3 @@ from .api import * from .core import * -from .models import * diff --git a/tests/cli/test_cli.py b/tests/cli/test_cli.py index f274fe907daf1..55e0de349dcce 100644 --- a/tests/cli/test_cli.py +++ b/tests/cli/test_cli.py @@ -202,10 +202,8 @@ def test_test(self): output = out.getvalue() # Check that prints, and log messages, are shown - self.assertIn('Done. Returned value was: Whatever you return gets printed in the logs', - output) - self.assertIn("'example_python_operator__print_the_context__20180101'", - output) + self.assertIn('END_DATE', output) + self.assertIn("'example_python_operator__print_the_context__20180101'", output) finally: sys.stdout = saved_stdout diff --git a/tests/core.py b/tests/core.py index d9f1747b2683e..34687eb76f6b6 100644 --- a/tests/core.py +++ b/tests/core.py @@ -2616,13 +2616,6 @@ def test_get_connections_env_var(self): assert conns[0].password == 'password' assert conns[0].port == 5432 - def test_get_connections_db(self): - conns = BaseHook.get_connections(conn_id='airflow_db') - assert len(conns) == 1 - assert conns[0].host == 'localhost' - assert conns[0].schema == 'airflow' - assert conns[0].login == 'root' - class WebHDFSHookTest(unittest.TestCase): def setUp(self): diff --git a/tests/models.py b/tests/test_models.py similarity index 99% rename from tests/models.py rename to tests/test_models.py index b1f19e32a61a9..2dd4964fa5cdb 100644 --- a/tests/models.py +++ b/tests/test_models.py @@ -47,7 +47,7 @@ from airflow.contrib.sensors.python_sensor import PythonSensor from airflow.exceptions import AirflowDagCycleException, AirflowSkipException from airflow.jobs import BackfillJob -from airflow.models import DAG, TaskInstance as TI +from airflow.models import DAG, TaskInstance as TI, DagBag from airflow.models import DagModel, DagRun from airflow.models import KubeResourceVersion, KubeWorkerIdentifier from airflow.models import SkipMixin @@ -1965,25 +1965,23 @@ def test_deactivate_unknown_dags(self): Test that dag_ids not passed into deactivate_unknown_dags are deactivated when function is invoked """ - dagbag = models.DagBag(include_examples=True) + dagbag = DagBag(include_examples=True) + dag_id = "test_deactivate_unknown_dags" expected_active_dags = dagbag.dags.keys() - session = settings.Session - session.add(DagModel(dag_id='test_deactivate_unknown_dags', is_active=True)) - session.commit() + model_before = DagModel(dag_id=dag_id, is_active=True) + with create_session() as session: + session.merge(model_before) models.DAG.deactivate_unknown_dags(expected_active_dags) - for dag in session.query(DagModel).all(): - if dag.dag_id in expected_active_dags: - self.assertTrue(dag.is_active) - else: - self.assertEqual(dag.dag_id, 'test_deactivate_unknown_dags') - self.assertFalse(dag.is_active) + after_model = DagModel.get_dagmodel(dag_id) + self.assertTrue(model_before.is_active) + self.assertFalse(after_model.is_active) # clean up - session.query(DagModel).filter(DagModel.dag_id == 'test_deactivate_unknown_dags').delete() - session.commit() + with create_session() as session: + session.query(DagModel).filter(DagModel.dag_id == 'test_deactivate_unknown_dags').delete() class TaskInstanceTest(unittest.TestCase): From 1b61b95f77120d66c5e9335d49a588b209f1ad68 Mon Sep 17 00:00:00 2001 From: Jarek Date: Fri, 2 Nov 2018 11:44:47 +0100 Subject: [PATCH 771/808] [AIRFLOW-3287] Moving database clean-up code into the CoreTest.tearDown() (#4122) --- tests/core.py | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/tests/core.py b/tests/core.py index 34687eb76f6b6..f0ffe98e0512c 100644 --- a/tests/core.py +++ b/tests/core.py @@ -91,19 +91,6 @@ import pickle -def reset(dag_id=TEST_DAG_ID): - session = Session() - tis = session.query(models.TaskInstance).filter_by(dag_id=dag_id) - tis.delete() - session.commit() - session.close() - - -configuration.conf.load_test_config() -if os.environ.get('KUBERNETES_VERSION') is None: - reset() - - class OperatorSubclass(BaseOperator): """ An operator to test template substitution @@ -132,6 +119,16 @@ def setUp(self): self.run_after_loop = self.dag_bash.get_task('run_after_loop') self.run_this_last = self.dag_bash.get_task('run_this_last') + def tearDown(self): + if os.environ.get('KUBERNETES_VERSION') is None: + session = Session() + session.query(models.TaskInstance).filter_by( + dag_id=TEST_DAG_ID).delete() + session.query(models.TaskFail).filter_by( + dag_id=TEST_DAG_ID).delete() + session.commit() + session.close() + def test_schedule_dag_no_previous_runs(self): """ Tests scheduling a dag with no previous runs From 254116fcae90094c408209a06f65629e847a2e14 Mon Sep 17 00:00:00 2001 From: Stefan Seelmann Date: Thu, 31 Jan 2019 21:01:52 +0100 Subject: [PATCH 772/808] [AIRFLOW-3461] Move TaskFail out of models.py (#4630) --- airflow/api/common/experimental/delete_dag.py | 3 +- airflow/models/__init__.py | 33 +---------- airflow/models/taskfail.py | 55 +++++++++++++++++++ airflow/www_rbac/views.py | 5 +- .../operators/test_bigquery_operator.py | 3 +- tests/contrib/sensors/test_weekday_sensor.py | 3 +- tests/core.py | 7 ++- tests/test_models.py | 3 +- 8 files changed, 71 insertions(+), 41 deletions(-) create mode 100755 airflow/models/taskfail.py diff --git a/airflow/api/common/experimental/delete_dag.py b/airflow/api/common/experimental/delete_dag.py index f1cb6a952f347..365997638b64c 100644 --- a/airflow/api/common/experimental/delete_dag.py +++ b/airflow/api/common/experimental/delete_dag.py @@ -20,6 +20,7 @@ from sqlalchemy import or_ from airflow import models +from airflow.models.taskfail import TaskFail from airflow.utils.db import provide_session from airflow.exceptions import DagNotFound, DagFileExists @@ -56,7 +57,7 @@ def delete_dag(dag_id, keep_records_in_log=True, session=None): if dag.is_subdag: p, c = dag_id.rsplit(".", 1) - for m in models.DagRun, models.TaskFail, models.TaskInstance: + for m in models.DagRun, TaskFail, models.TaskInstance: count += session.query(m).filter(m.dag_id == p, m.task_id == c).delete() return count diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index 3d2cbae33f2e7..fc692e71aa4cc 100755 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -89,6 +89,7 @@ from airflow.models.dagpickle import DagPickle from airflow.models.errors import ImportError # noqa: F401 from airflow.models.slamiss import SlaMiss # noqa: F401 +from airflow.models.taskfail import TaskFail from airflow.models.taskreschedule import TaskReschedule from airflow.ti_deps.deps.not_in_retry_period_dep import NotInRetryPeriodDep from airflow.ti_deps.deps.prev_dagrun_dep import PrevDagrunDep @@ -1932,38 +1933,6 @@ def init_run_context(self, raw=False): self._set_context(self) -class TaskFail(Base): - """ - TaskFail tracks the failed run durations of each task instance. - """ - - __tablename__ = "task_fail" - - id = Column(Integer, primary_key=True) - task_id = Column(String(ID_LEN), nullable=False) - dag_id = Column(String(ID_LEN), nullable=False) - execution_date = Column(UtcDateTime, nullable=False) - start_date = Column(UtcDateTime) - end_date = Column(UtcDateTime) - duration = Column(Integer) - - __table_args__ = ( - Index('idx_task_fail_dag_task_date', dag_id, task_id, execution_date, - unique=False), - ) - - def __init__(self, task, execution_date, start_date, end_date): - self.dag_id = task.dag_id - self.task_id = task.task_id - self.execution_date = execution_date - self.start_date = start_date - self.end_date = end_date - if self.end_date and self.start_date: - self.duration = (self.end_date - self.start_date).total_seconds() - else: - self.duration = None - - class Log(Base): """ Used to actively log events to the database diff --git a/airflow/models/taskfail.py b/airflow/models/taskfail.py new file mode 100755 index 0000000000000..d9cc7af922730 --- /dev/null +++ b/airflow/models/taskfail.py @@ -0,0 +1,55 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from sqlalchemy import Column, Index, Integer, String + +from airflow.models.base import Base, ID_LEN +from airflow.utils.sqlalchemy import UtcDateTime + + +class TaskFail(Base): + """ + TaskFail tracks the failed run durations of each task instance. + """ + + __tablename__ = "task_fail" + + id = Column(Integer, primary_key=True) + task_id = Column(String(ID_LEN), nullable=False) + dag_id = Column(String(ID_LEN), nullable=False) + execution_date = Column(UtcDateTime, nullable=False) + start_date = Column(UtcDateTime) + end_date = Column(UtcDateTime) + duration = Column(Integer) + + __table_args__ = ( + Index('idx_task_fail_dag_task_date', dag_id, task_id, execution_date, + unique=False), + ) + + def __init__(self, task, execution_date, start_date, end_date): + self.dag_id = task.dag_id + self.task_id = task.task_id + self.execution_date = execution_date + self.start_date = start_date + self.end_date = end_date + if self.end_date and self.start_date: + self.duration = (self.end_date - self.start_date).total_seconds() + else: + self.duration = None diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index 96f199667a316..d6060629c81b6 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -55,6 +55,7 @@ set_dag_run_state_to_failed) from airflow.models import XCom, DagRun, errors, DagModel from airflow.models.connection import Connection +from airflow.models.taskfail import TaskFail from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, SCHEDULER_DEPS from airflow.utils import timezone from airflow.utils.dates import infer_time_unit, scale_time_units @@ -1424,7 +1425,7 @@ def duration(self, session=None): tis = dag.get_task_instances( session, start_date=min_date, end_date=base_date) - TF = models.TaskFail + TF = TaskFail ti_fails = ( session.query(TF) .filter(TF.dag_id == dag.dag_id, # noqa @@ -1718,7 +1719,7 @@ def gantt(self, session=None): ti for ti in dag.get_task_instances(session, dttm, dttm) if ti.start_date] tis = sorted(tis, key=lambda ti: ti.start_date) - TF = models.TaskFail + TF = TaskFail ti_fails = list(itertools.chain(*[( session .query(TF) diff --git a/tests/contrib/operators/test_bigquery_operator.py b/tests/contrib/operators/test_bigquery_operator.py index 6cc80c1eefda1..1deb542db94e1 100644 --- a/tests/contrib/operators/test_bigquery_operator.py +++ b/tests/contrib/operators/test_bigquery_operator.py @@ -25,6 +25,7 @@ from airflow import configuration, models from airflow.models import TaskInstance, DAG +from airflow.models.taskfail import TaskFail from airflow.contrib.operators.bigquery_operator import \ BigQueryCreateExternalTableOperator, BigQueryCreateEmptyTableOperator, \ @@ -178,7 +179,7 @@ def tearDown(self): session = Session() session.query(models.TaskInstance).filter_by( dag_id=TEST_DAG_ID).delete() - session.query(models.TaskFail).filter_by( + session.query(TaskFail).filter_by( dag_id=TEST_DAG_ID).delete() session.commit() session.close() diff --git a/tests/contrib/sensors/test_weekday_sensor.py b/tests/contrib/sensors/test_weekday_sensor.py index 55a4c4da4bf7d..0255aa3a162fe 100644 --- a/tests/contrib/sensors/test_weekday_sensor.py +++ b/tests/contrib/sensors/test_weekday_sensor.py @@ -24,6 +24,7 @@ from airflow.contrib.utils.weekday import WeekDay from airflow.exceptions import AirflowSensorTimeout from airflow.models import DagBag +from airflow.models.taskfail import TaskFail from airflow.settings import Session from airflow.utils.timezone import datetime @@ -53,7 +54,7 @@ def tearDown(self): session = Session() session.query(models.TaskInstance).filter_by( dag_id=TEST_DAG_ID).delete() - session.query(models.TaskFail).filter_by( + session.query(TaskFail).filter_by( dag_id=TEST_DAG_ID).delete() session.commit() session.close() diff --git a/tests/core.py b/tests/core.py index f0ffe98e0512c..8c407a7c952c3 100644 --- a/tests/core.py +++ b/tests/core.py @@ -51,6 +51,7 @@ from airflow import jobs, models, DAG, utils, macros, settings, exceptions from airflow.models import BaseOperator from airflow.models.connection import Connection +from airflow.models.taskfail import TaskFail from airflow.operators.bash_operator import BashOperator from airflow.operators.check_operator import CheckOperator, ValueCheckOperator from airflow.operators.dagrun_operator import TriggerDagRunOperator @@ -124,7 +125,7 @@ def tearDown(self): session = Session() session.query(models.TaskInstance).filter_by( dag_id=TEST_DAG_ID).delete() - session.query(models.TaskFail).filter_by( + session.query(TaskFail).filter_by( dag_id=TEST_DAG_ID).delete() session.commit() session.close() @@ -940,11 +941,11 @@ def test_task_fail_duration(self): f.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) except Exception: pass - p_fails = session.query(models.TaskFail).filter_by( + p_fails = session.query(TaskFail).filter_by( task_id='pass_sleepy', dag_id=self.dag.dag_id, execution_date=DEFAULT_DATE).all() - f_fails = session.query(models.TaskFail).filter_by( + f_fails = session.query(TaskFail).filter_by( task_id='fail_sleepy', dag_id=self.dag.dag_id, execution_date=DEFAULT_DATE).all() diff --git a/tests/test_models.py b/tests/test_models.py index 2dd4964fa5cdb..edea71fdfe172 100644 --- a/tests/test_models.py +++ b/tests/test_models.py @@ -56,6 +56,7 @@ from airflow.models import Variable from airflow.models import clear_task_instances from airflow.models.connection import Connection +from airflow.models.taskfail import TaskFail from airflow.models.taskreschedule import TaskReschedule from airflow.operators.bash_operator import BashOperator from airflow.operators.dummy_operator import DummyOperator @@ -1988,7 +1989,7 @@ class TaskInstanceTest(unittest.TestCase): def tearDown(self): with create_session() as session: - session.query(models.TaskFail).delete() + session.query(TaskFail).delete() session.query(TaskReschedule).delete() session.query(models.TaskInstance).delete() From 9ce630aaf8939faafc9aea7cb941b913f73e1194 Mon Sep 17 00:00:00 2001 From: BasPH Date: Wed, 6 Feb 2019 09:55:16 +0100 Subject: [PATCH 773/808] [AIRFLOW-3476,3477] Move Kube classes out of models.py (#4443) --- .../contrib/executors/kubernetes_executor.py | 3 +- airflow/models/__init__.py | 58 +------------- airflow/models/kubernetes.py | 79 +++++++++++++++++++ tests/models/test_kubernetes.py | 58 ++++++++++++++ tests/test_models.py | 34 -------- 5 files changed, 141 insertions(+), 91 deletions(-) create mode 100644 airflow/models/kubernetes.py create mode 100644 tests/models/test_kubernetes.py diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index b15fdb49fa7de..38abc5ed85a08 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -32,7 +32,8 @@ from airflow.contrib.kubernetes.worker_configuration import WorkerConfiguration from airflow.executors.base_executor import BaseExecutor from airflow.executors import Executors -from airflow.models import TaskInstance, KubeResourceVersion, KubeWorkerIdentifier +from airflow.models import TaskInstance +from airflow.models.kubernetes import KubeResourceVersion, KubeWorkerIdentifier from airflow.utils.state import State from airflow.utils.db import provide_session, create_session from airflow import configuration, settings diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index fc692e71aa4cc..4fcfa64aec4d9 100755 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -60,14 +60,13 @@ import warnings import hashlib -import uuid from datetime import datetime from urllib.parse import quote from sqlalchemy import ( Boolean, Column, DateTime, Float, ForeignKey, Index, Integer, LargeBinary, PickleType, String, Text, UniqueConstraint, and_, - func, or_, true as sqltrue + func, or_ ) from sqlalchemy.ext.declarative import declared_attr from sqlalchemy.orm import reconstructor, relationship, synonym @@ -89,6 +88,7 @@ from airflow.models.dagpickle import DagPickle from airflow.models.errors import ImportError # noqa: F401 from airflow.models.slamiss import SlaMiss # noqa: F401 +from airflow.models.kubernetes import KubeWorkerIdentifier, KubeResourceVersion # noqa: F401 from airflow.models.taskfail import TaskFail from airflow.models.taskreschedule import TaskReschedule from airflow.ti_deps.deps.not_in_retry_period_dep import NotInRetryPeriodDep @@ -5111,59 +5111,5 @@ def open_slots(self, session): return self.slots - used_slots - queued_slots -class KubeResourceVersion(Base): - __tablename__ = "kube_resource_version" - one_row_id = Column(Boolean, server_default=sqltrue(), primary_key=True) - resource_version = Column(String(255)) - - @staticmethod - @provide_session - def get_current_resource_version(session=None): - (resource_version,) = session.query(KubeResourceVersion.resource_version).one() - return resource_version - - @staticmethod - @provide_session - def checkpoint_resource_version(resource_version, session=None): - if resource_version: - session.query(KubeResourceVersion).update({ - KubeResourceVersion.resource_version: resource_version - }) - session.commit() - - @staticmethod - @provide_session - def reset_resource_version(session=None): - session.query(KubeResourceVersion).update({ - KubeResourceVersion.resource_version: '0' - }) - session.commit() - return '0' - - -class KubeWorkerIdentifier(Base): - __tablename__ = "kube_worker_uuid" - one_row_id = Column(Boolean, server_default=sqltrue(), primary_key=True) - worker_uuid = Column(String(255)) - - @staticmethod - @provide_session - def get_or_create_current_kube_worker_uuid(session=None): - (worker_uuid,) = session.query(KubeWorkerIdentifier.worker_uuid).one() - if worker_uuid == '': - worker_uuid = str(uuid.uuid4()) - KubeWorkerIdentifier.checkpoint_kube_worker_uuid(worker_uuid, session) - return worker_uuid - - @staticmethod - @provide_session - def checkpoint_kube_worker_uuid(worker_uuid, session=None): - if worker_uuid: - session.query(KubeWorkerIdentifier).update({ - KubeWorkerIdentifier.worker_uuid: worker_uuid - }) - session.commit() - - # To avoid circular import on Python2.7 we need to define this at the _bottom_ from airflow.models.connection import Connection # noqa: E402,F401 diff --git a/airflow/models/kubernetes.py b/airflow/models/kubernetes.py new file mode 100644 index 0000000000000..a18689eefd316 --- /dev/null +++ b/airflow/models/kubernetes.py @@ -0,0 +1,79 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import uuid + +from sqlalchemy import Column, Boolean, true as sqltrue, String + +from airflow.models.base import Base +from airflow.utils.db import provide_session + + +class KubeResourceVersion(Base): + __tablename__ = "kube_resource_version" + one_row_id = Column(Boolean, server_default=sqltrue(), primary_key=True) + resource_version = Column(String(255)) + + @staticmethod + @provide_session + def get_current_resource_version(session=None): + (resource_version,) = session.query(KubeResourceVersion.resource_version).one() + return resource_version + + @staticmethod + @provide_session + def checkpoint_resource_version(resource_version, session=None): + if resource_version: + session.query(KubeResourceVersion).update({ + KubeResourceVersion.resource_version: resource_version + }) + session.commit() + + @staticmethod + @provide_session + def reset_resource_version(session=None): + session.query(KubeResourceVersion).update({ + KubeResourceVersion.resource_version: '0' + }) + session.commit() + return '0' + + +class KubeWorkerIdentifier(Base): + __tablename__ = "kube_worker_uuid" + one_row_id = Column(Boolean, server_default=sqltrue(), primary_key=True) + worker_uuid = Column(String(255)) + + @staticmethod + @provide_session + def get_or_create_current_kube_worker_uuid(session=None): + (worker_uuid,) = session.query(KubeWorkerIdentifier.worker_uuid).one() + if worker_uuid == '': + worker_uuid = str(uuid.uuid4()) + KubeWorkerIdentifier.checkpoint_kube_worker_uuid(worker_uuid, session) + return worker_uuid + + @staticmethod + @provide_session + def checkpoint_kube_worker_uuid(worker_uuid, session=None): + if worker_uuid: + session.query(KubeWorkerIdentifier).update({ + KubeWorkerIdentifier.worker_uuid: worker_uuid + }) + session.commit() diff --git a/tests/models/test_kubernetes.py b/tests/models/test_kubernetes.py new file mode 100644 index 0000000000000..45bea723277ec --- /dev/null +++ b/tests/models/test_kubernetes.py @@ -0,0 +1,58 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from mock import patch + +from airflow import settings +from airflow.models.kubernetes import KubeResourceVersion, KubeWorkerIdentifier + + +class TestKubeResourceVersion(unittest.TestCase): + + def test_checkpoint_resource_version(self): + session = settings.Session() + KubeResourceVersion.checkpoint_resource_version('7', session) + self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '7') + + def test_reset_resource_version(self): + session = settings.Session() + version = KubeResourceVersion.reset_resource_version(session) + self.assertEqual(version, '0') + self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '0') + + +class TestKubeWorkerIdentifier(unittest.TestCase): + + @patch('airflow.models.uuid.uuid4') + def test_get_or_create_not_exist(self, mock_uuid): + session = settings.Session() + session.query(KubeWorkerIdentifier).update({ + KubeWorkerIdentifier.worker_uuid: '' + }) + mock_uuid.return_value = 'abcde' + worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) + self.assertEqual(worker_uuid, 'abcde') + + def test_get_or_create_exist(self): + session = settings.Session() + KubeWorkerIdentifier.checkpoint_kube_worker_uuid('fghij', session) + worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) + self.assertEqual(worker_uuid, 'fghij') diff --git a/tests/test_models.py b/tests/test_models.py index edea71fdfe172..264151ea22685 100644 --- a/tests/test_models.py +++ b/tests/test_models.py @@ -49,7 +49,6 @@ from airflow.jobs import BackfillJob from airflow.models import DAG, TaskInstance as TI, DagBag from airflow.models import DagModel, DagRun -from airflow.models import KubeResourceVersion, KubeWorkerIdentifier from airflow.models import SkipMixin from airflow.models import State as ST from airflow.models import XCom @@ -3591,36 +3590,3 @@ def test_skip_none_tasks(self): SkipMixin().skip(dag_run=None, execution_date=None, tasks=[], session=session) self.assertFalse(session.query.called) self.assertFalse(session.commit.called) - - -class TestKubeResourceVersion(unittest.TestCase): - - def test_checkpoint_resource_version(self): - session = settings.Session() - KubeResourceVersion.checkpoint_resource_version('7', session) - self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '7') - - def test_reset_resource_version(self): - session = settings.Session() - version = KubeResourceVersion.reset_resource_version(session) - self.assertEqual(version, '0') - self.assertEqual(KubeResourceVersion.get_current_resource_version(session), '0') - - -class TestKubeWorkerIdentifier(unittest.TestCase): - - @patch('airflow.models.uuid.uuid4') - def test_get_or_create_not_exist(self, mock_uuid): - session = settings.Session() - session.query(KubeWorkerIdentifier).update({ - KubeWorkerIdentifier.worker_uuid: '' - }) - mock_uuid.return_value = 'abcde' - worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) - self.assertEqual(worker_uuid, 'abcde') - - def test_get_or_create_exist(self): - session = settings.Session() - KubeWorkerIdentifier.checkpoint_kube_worker_uuid('fghij', session) - worker_uuid = KubeWorkerIdentifier.get_or_create_current_kube_worker_uuid(session) - self.assertEqual(worker_uuid, 'fghij') From 5a01be0e9b63664ece490573027491efa059aab1 Mon Sep 17 00:00:00 2001 From: BasPH Date: Sun, 31 Mar 2019 18:35:09 +0200 Subject: [PATCH 774/808] [AIRFLOW-3458] Move connection tests (#4680) --- tests/models/__init__.py | 18 +++ tests/models/test_connection.py | 276 ++++++++++++++++++++++++++++++++ tests/models/test_kubernetes.py | 2 +- tests/test_models.py | 251 ----------------------------- 4 files changed, 295 insertions(+), 252 deletions(-) create mode 100644 tests/models/__init__.py create mode 100644 tests/models/test_connection.py diff --git a/tests/models/__init__.py b/tests/models/__init__.py new file mode 100644 index 0000000000000..114d189da14ab --- /dev/null +++ b/tests/models/__init__.py @@ -0,0 +1,18 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/models/test_connection.py b/tests/models/test_connection.py new file mode 100644 index 0000000000000..939869e31b688 --- /dev/null +++ b/tests/models/test_connection.py @@ -0,0 +1,276 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest +from collections import namedtuple + +from cryptography.fernet import Fernet +from mock import patch +from parameterized import parameterized + +from airflow import models +from airflow.models.connection import Connection + +ConnectionParts = namedtuple("ConnectionParts", ["conn_type", "login", "password", "host", "port", "schema"]) + + +class ConnectionTest(unittest.TestCase): + def setUp(self): + models._fernet = None + + def tearDown(self): + models._fernet = None + + @patch('airflow.models.configuration.conf.get') + def test_connection_extra_no_encryption(self, mock_get): + """ + Tests extras on a new connection without encryption. The fernet key + is set to a non-base64-encoded string and the extra is stored without + encryption. + """ + mock_get.return_value = '' + test_connection = Connection(extra='testextra') + self.assertFalse(test_connection.is_extra_encrypted) + self.assertEqual(test_connection.extra, 'testextra') + + @patch('airflow.models.configuration.conf.get') + def test_connection_extra_with_encryption(self, mock_get): + """ + Tests extras on a new connection with encryption. + """ + mock_get.return_value = Fernet.generate_key().decode() + test_connection = Connection(extra='testextra') + self.assertTrue(test_connection.is_extra_encrypted) + self.assertEqual(test_connection.extra, 'testextra') + + @patch('airflow.models.configuration.conf.get') + def test_connection_extra_with_encryption_rotate_fernet_key(self, mock_get): + """ + Tests rotating encrypted extras. + """ + key1 = Fernet.generate_key() + key2 = Fernet.generate_key() + + mock_get.return_value = key1.decode() + test_connection = Connection(extra='testextra') + self.assertTrue(test_connection.is_extra_encrypted) + self.assertEqual(test_connection.extra, 'testextra') + self.assertEqual(Fernet(key1).decrypt(test_connection._extra.encode()), b'testextra') + + # Test decrypt of old value with new key + mock_get.return_value = ','.join([key2.decode(), key1.decode()]) + models._fernet = None + self.assertEqual(test_connection.extra, 'testextra') + + # Test decrypt of new value with new key + test_connection.rotate_fernet_key() + self.assertTrue(test_connection.is_extra_encrypted) + self.assertEqual(test_connection.extra, 'testextra') + self.assertEqual(Fernet(key2).decrypt(test_connection._extra.encode()), b'testextra') + + def test_connection_from_uri_without_extras(self): + uri = 'scheme://user:password@host%2flocation:1234/schema' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host/location') + self.assertEqual(connection.schema, 'schema') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password') + self.assertEqual(connection.port, 1234) + self.assertIsNone(connection.extra) + + def test_connection_from_uri_with_extras(self): + uri = 'scheme://user:password@host%2flocation:1234/schema?' \ + 'extra1=a%20value&extra2=%2fpath%2f' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host/location') + self.assertEqual(connection.schema, 'schema') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password') + self.assertEqual(connection.port, 1234) + self.assertDictEqual(connection.extra_dejson, {'extra1': 'a value', + 'extra2': '/path/'}) + + def test_connection_from_uri_with_colon_in_hostname(self): + uri = 'scheme://user:password@host%2flocation%3ax%3ay:1234/schema?' \ + 'extra1=a%20value&extra2=%2fpath%2f' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host/location:x:y') + self.assertEqual(connection.schema, 'schema') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password') + self.assertEqual(connection.port, 1234) + self.assertDictEqual(connection.extra_dejson, {'extra1': 'a value', + 'extra2': '/path/'}) + + def test_connection_from_uri_with_encoded_password(self): + uri = 'scheme://user:password%20with%20space@host%2flocation%3ax%3ay:1234/schema' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host/location:x:y') + self.assertEqual(connection.schema, 'schema') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password with space') + self.assertEqual(connection.port, 1234) + + def test_connection_from_uri_with_encoded_user(self): + uri = 'scheme://domain%2fuser:password@host%2flocation%3ax%3ay:1234/schema' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host/location:x:y') + self.assertEqual(connection.schema, 'schema') + self.assertEqual(connection.login, 'domain/user') + self.assertEqual(connection.password, 'password') + self.assertEqual(connection.port, 1234) + + def test_connection_from_uri_with_encoded_schema(self): + uri = 'scheme://user:password%20with%20space@host:1234/schema%2ftest' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host') + self.assertEqual(connection.schema, 'schema/test') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password with space') + self.assertEqual(connection.port, 1234) + + def test_connection_from_uri_no_schema(self): + uri = 'scheme://user:password%20with%20space@host:1234' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host') + self.assertEqual(connection.schema, '') + self.assertEqual(connection.login, 'user') + self.assertEqual(connection.password, 'password with space') + self.assertEqual(connection.port, 1234) + + def test_connection_from_uri_with_underscore(self): + uri = 'google-cloud-platform://?extra__google_cloud_platform__key_' \ + 'path=%2Fkeys%2Fkey.json&extra__google_cloud_platform__scope=' \ + 'https%3A%2F%2Fwww.googleapis.com%2Fauth%2Fcloud-platform&extra' \ + '__google_cloud_platform__project=airflow' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'google_cloud_platform') + self.assertEqual(connection.host, '') + self.assertEqual(connection.schema, '') + self.assertEqual(connection.login, None) + self.assertEqual(connection.password, None) + self.assertEqual(connection.extra_dejson, dict( + extra__google_cloud_platform__key_path='/keys/key.json', + extra__google_cloud_platform__project='airflow', + extra__google_cloud_platform__scope='https://www.googleapis.com/' + 'auth/cloud-platform')) + + def test_connection_from_uri_without_authinfo(self): + uri = 'scheme://host:1234' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, 'host') + self.assertEqual(connection.schema, '') + self.assertEqual(connection.login, None) + self.assertEqual(connection.password, None) + self.assertEqual(connection.port, 1234) + + def test_connection_from_uri_with_path(self): + uri = 'scheme://%2FTmP%2F:1234' + connection = Connection(uri=uri) + self.assertEqual(connection.conn_type, 'scheme') + self.assertEqual(connection.host, '/TmP/') + self.assertEqual(connection.schema, '') + self.assertEqual(connection.login, None) + self.assertEqual(connection.password, None) + self.assertEqual(connection.port, 1234) + + @parameterized.expand( + [ + ( + "http://:password@host:80/database", + ConnectionParts( + conn_type="http", login='', password="password", host="host", port=80, schema="database" + ), + ), + ( + "http://user:@host:80/database", + ConnectionParts( + conn_type="http", login="user", password=None, host="host", port=80, schema="database" + ), + ), + ( + "http://user:password@/database", + ConnectionParts( + conn_type="http", login="user", password="password", host="", port=None, schema="database" + ), + ), + ( + "http://user:password@host:80/", + ConnectionParts( + conn_type="http", login="user", password="password", host="host", port=80, schema="" + ), + ), + ( + "http://user:password@/", + ConnectionParts( + conn_type="http", login="user", password="password", host="", port=None, schema="" + ), + ), + ( + "postgresql://user:password@%2Ftmp%2Fz6rqdzqh%2Fexample%3Awest1%3Atestdb/testdb", + ConnectionParts( + conn_type="postgres", + login="user", + password="password", + host="/tmp/z6rqdzqh/example:west1:testdb", + port=None, + schema="testdb", + ), + ), + ( + "postgresql://user@%2Ftmp%2Fz6rqdzqh%2Fexample%3Aeurope-west1%3Atestdb/testdb", + ConnectionParts( + conn_type="postgres", + login="user", + password=None, + host="/tmp/z6rqdzqh/example:europe-west1:testdb", + port=None, + schema="testdb", + ), + ), + ( + "postgresql://%2Ftmp%2Fz6rqdzqh%2Fexample%3Aeurope-west1%3Atestdb", + ConnectionParts( + conn_type="postgres", + login=None, + password=None, + host="/tmp/z6rqdzqh/example:europe-west1:testdb", + port=None, + schema="", + ), + ), + ] + ) + def test_connection_from_with_auth_info(self, uri, uri_parts): + connection = Connection(uri=uri) + + self.assertEqual(connection.conn_type, uri_parts.conn_type) + self.assertEqual(connection.login, uri_parts.login) + self.assertEqual(connection.password, uri_parts.password) + self.assertEqual(connection.host, uri_parts.host) + self.assertEqual(connection.port, uri_parts.port) + self.assertEqual(connection.schema, uri_parts.schema) diff --git a/tests/models/test_kubernetes.py b/tests/models/test_kubernetes.py index 45bea723277ec..e01a88784cd4a 100644 --- a/tests/models/test_kubernetes.py +++ b/tests/models/test_kubernetes.py @@ -41,7 +41,7 @@ def test_reset_resource_version(self): class TestKubeWorkerIdentifier(unittest.TestCase): - @patch('airflow.models.uuid.uuid4') + @patch('airflow.models.kubernetes.uuid.uuid4') def test_get_or_create_not_exist(self, mock_uuid): session = settings.Session() session.query(KubeWorkerIdentifier).update({ diff --git a/tests/test_models.py b/tests/test_models.py index 264151ea22685..2f9670f27bc93 100644 --- a/tests/test_models.py +++ b/tests/test_models.py @@ -33,7 +33,6 @@ import urllib import uuid import shutil -from collections import namedtuple from tempfile import NamedTemporaryFile, mkdtemp import pendulum @@ -54,7 +53,6 @@ from airflow.models import XCom from airflow.models import Variable from airflow.models import clear_task_instances -from airflow.models.connection import Connection from airflow.models.taskfail import TaskFail from airflow.models.taskreschedule import TaskReschedule from airflow.operators.bash_operator import BashOperator @@ -3280,255 +3278,6 @@ def test_var_with_encryption_rotate_fernet_key(self, mock_get): self.assertEqual(Fernet(key2).decrypt(test_var._val.encode()), b'value') -ConnectionParts = namedtuple("ConnectionParts", ["conn_type", "login", "password", "host", "port", "schema"]) - - -class ConnectionTest(unittest.TestCase): - def setUp(self): - models._fernet = None - - def tearDown(self): - models._fernet = None - - @patch('airflow.models.configuration.conf.get') - def test_connection_extra_no_encryption(self, mock_get): - """ - Tests extras on a new connection without encryption. The fernet key - is set to a non-base64-encoded string and the extra is stored without - encryption. - """ - mock_get.return_value = '' - test_connection = Connection(extra='testextra') - self.assertFalse(test_connection.is_extra_encrypted) - self.assertEqual(test_connection.extra, 'testextra') - - @patch('airflow.models.configuration.conf.get') - def test_connection_extra_with_encryption(self, mock_get): - """ - Tests extras on a new connection with encryption. - """ - mock_get.return_value = Fernet.generate_key().decode() - test_connection = Connection(extra='testextra') - self.assertTrue(test_connection.is_extra_encrypted) - self.assertEqual(test_connection.extra, 'testextra') - - @patch('airflow.models.configuration.conf.get') - def test_connection_extra_with_encryption_rotate_fernet_key(self, mock_get): - """ - Tests rotating encrypted extras. - """ - key1 = Fernet.generate_key() - key2 = Fernet.generate_key() - - mock_get.return_value = key1.decode() - test_connection = Connection(extra='testextra') - self.assertTrue(test_connection.is_extra_encrypted) - self.assertEqual(test_connection.extra, 'testextra') - self.assertEqual(Fernet(key1).decrypt(test_connection._extra.encode()), b'testextra') - - # Test decrypt of old value with new key - mock_get.return_value = ','.join([key2.decode(), key1.decode()]) - models._fernet = None - self.assertEqual(test_connection.extra, 'testextra') - - # Test decrypt of new value with new key - test_connection.rotate_fernet_key() - self.assertTrue(test_connection.is_extra_encrypted) - self.assertEqual(test_connection.extra, 'testextra') - self.assertEqual(Fernet(key2).decrypt(test_connection._extra.encode()), b'testextra') - - def test_connection_from_uri_without_extras(self): - uri = 'scheme://user:password@host%2flocation:1234/schema' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host/location') - self.assertEqual(connection.schema, 'schema') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password') - self.assertEqual(connection.port, 1234) - self.assertIsNone(connection.extra) - - def test_connection_from_uri_with_extras(self): - uri = 'scheme://user:password@host%2flocation:1234/schema?' \ - 'extra1=a%20value&extra2=%2fpath%2f' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host/location') - self.assertEqual(connection.schema, 'schema') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password') - self.assertEqual(connection.port, 1234) - self.assertDictEqual(connection.extra_dejson, {'extra1': 'a value', - 'extra2': '/path/'}) - - def test_connection_from_uri_with_colon_in_hostname(self): - uri = 'scheme://user:password@host%2flocation%3ax%3ay:1234/schema?' \ - 'extra1=a%20value&extra2=%2fpath%2f' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host/location:x:y') - self.assertEqual(connection.schema, 'schema') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password') - self.assertEqual(connection.port, 1234) - self.assertDictEqual(connection.extra_dejson, {'extra1': 'a value', - 'extra2': '/path/'}) - - def test_connection_from_uri_with_encoded_password(self): - uri = 'scheme://user:password%20with%20space@host%2flocation%3ax%3ay:1234/schema' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host/location:x:y') - self.assertEqual(connection.schema, 'schema') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password with space') - self.assertEqual(connection.port, 1234) - - def test_connection_from_uri_with_encoded_user(self): - uri = 'scheme://domain%2fuser:password@host%2flocation%3ax%3ay:1234/schema' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host/location:x:y') - self.assertEqual(connection.schema, 'schema') - self.assertEqual(connection.login, 'domain/user') - self.assertEqual(connection.password, 'password') - self.assertEqual(connection.port, 1234) - - def test_connection_from_uri_with_encoded_schema(self): - uri = 'scheme://user:password%20with%20space@host:1234/schema%2ftest' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host') - self.assertEqual(connection.schema, 'schema/test') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password with space') - self.assertEqual(connection.port, 1234) - - def test_connection_from_uri_no_schema(self): - uri = 'scheme://user:password%20with%20space@host:1234' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host') - self.assertEqual(connection.schema, '') - self.assertEqual(connection.login, 'user') - self.assertEqual(connection.password, 'password with space') - self.assertEqual(connection.port, 1234) - - def test_connection_from_uri_with_underscore(self): - uri = 'google-cloud-platform://?extra__google_cloud_platform__key_' \ - 'path=%2Fkeys%2Fkey.json&extra__google_cloud_platform__scope=' \ - 'https%3A%2F%2Fwww.googleapis.com%2Fauth%2Fcloud-platform&extra' \ - '__google_cloud_platform__project=airflow' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'google_cloud_platform') - self.assertEqual(connection.host, '') - self.assertEqual(connection.schema, '') - self.assertEqual(connection.login, None) - self.assertEqual(connection.password, None) - self.assertEqual(connection.extra_dejson, dict( - extra__google_cloud_platform__key_path='/keys/key.json', - extra__google_cloud_platform__project='airflow', - extra__google_cloud_platform__scope='https://www.googleapis.com/' - 'auth/cloud-platform')) - - def test_connection_from_uri_without_authinfo(self): - uri = 'scheme://host:1234' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, 'host') - self.assertEqual(connection.schema, '') - self.assertEqual(connection.login, None) - self.assertEqual(connection.password, None) - self.assertEqual(connection.port, 1234) - - def test_connection_from_uri_with_path(self): - uri = 'scheme://%2FTmP%2F:1234' - connection = Connection(uri=uri) - self.assertEqual(connection.conn_type, 'scheme') - self.assertEqual(connection.host, '/TmP/') - self.assertEqual(connection.schema, '') - self.assertEqual(connection.login, None) - self.assertEqual(connection.password, None) - self.assertEqual(connection.port, 1234) - - @parameterized.expand( - [ - ( - "http://:password@host:80/database", - ConnectionParts( - conn_type="http", login='', password="password", host="host", port=80, schema="database" - ), - ), - ( - "http://user:@host:80/database", - ConnectionParts( - conn_type="http", login="user", password=None, host="host", port=80, schema="database" - ), - ), - ( - "http://user:password@/database", - ConnectionParts( - conn_type="http", login="user", password="password", host="", port=None, schema="database" - ), - ), - ( - "http://user:password@host:80/", - ConnectionParts( - conn_type="http", login="user", password="password", host="host", port=80, schema="" - ), - ), - ( - "http://user:password@/", - ConnectionParts( - conn_type="http", login="user", password="password", host="", port=None, schema="" - ), - ), - ( - "postgresql://user:password@%2Ftmp%2Fz6rqdzqh%2Fexample%3Awest1%3Atestdb/testdb", - ConnectionParts( - conn_type="postgres", - login="user", - password="password", - host="/tmp/z6rqdzqh/example:west1:testdb", - port=None, - schema="testdb", - ), - ), - ( - "postgresql://user@%2Ftmp%2Fz6rqdzqh%2Fexample%3Aeurope-west1%3Atestdb/testdb", - ConnectionParts( - conn_type="postgres", - login="user", - password=None, - host="/tmp/z6rqdzqh/example:europe-west1:testdb", - port=None, - schema="testdb", - ), - ), - ( - "postgresql://%2Ftmp%2Fz6rqdzqh%2Fexample%3Aeurope-west1%3Atestdb", - ConnectionParts( - conn_type="postgres", - login=None, - password=None, - host="/tmp/z6rqdzqh/example:europe-west1:testdb", - port=None, - schema="", - ), - ), - ] - ) - def test_connection_from_with_auth_info(self, uri, uri_parts): - connection = Connection(uri=uri) - - self.assertEqual(connection.conn_type, uri_parts.conn_type) - self.assertEqual(connection.login, uri_parts.login) - self.assertEqual(connection.password, uri_parts.password) - self.assertEqual(connection.host, uri_parts.host) - self.assertEqual(connection.port, uri_parts.port) - self.assertEqual(connection.schema, uri_parts.schema) - - class TestSkipMixin(unittest.TestCase): @patch('airflow.models.timezone.utcnow') From 8f14f641defdee963cbcca6930a3751697d8188e Mon Sep 17 00:00:00 2001 From: andyh1203 Date: Fri, 1 Feb 2019 22:09:41 -0800 Subject: [PATCH 775/808] [AIRFLOW-3463] Move Log out of models.py (#4639) --- airflow/models/__init__.py | 45 +----------- airflow/models/log.py | 68 +++++++++++++++++++ airflow/utils/cli.py | 6 +- airflow/www_rbac/decorators.py | 4 +- airflow/www_rbac/views.py | 3 +- .../common/experimental/test_delete_dag.py | 2 +- 6 files changed, 77 insertions(+), 51 deletions(-) create mode 100644 airflow/models/log.py diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index 4fcfa64aec4d9..6c454c9ac41f6 100755 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -89,6 +89,7 @@ from airflow.models.errors import ImportError # noqa: F401 from airflow.models.slamiss import SlaMiss # noqa: F401 from airflow.models.kubernetes import KubeWorkerIdentifier, KubeResourceVersion # noqa: F401 +from airflow.models.log import Log from airflow.models.taskfail import TaskFail from airflow.models.taskreschedule import TaskReschedule from airflow.ti_deps.deps.not_in_retry_period_dep import NotInRetryPeriodDep @@ -1933,50 +1934,6 @@ def init_run_context(self, raw=False): self._set_context(self) -class Log(Base): - """ - Used to actively log events to the database - """ - - __tablename__ = "log" - - id = Column(Integer, primary_key=True) - dttm = Column(UtcDateTime) - dag_id = Column(String(ID_LEN)) - task_id = Column(String(ID_LEN)) - event = Column(String(30)) - execution_date = Column(UtcDateTime) - owner = Column(String(500)) - extra = Column(Text) - - __table_args__ = ( - Index('idx_log_dag', dag_id), - ) - - def __init__(self, event, task_instance, owner=None, extra=None, **kwargs): - self.dttm = timezone.utcnow() - self.event = event - self.extra = extra - - task_owner = None - - if task_instance: - self.dag_id = task_instance.dag_id - self.task_id = task_instance.task_id - self.execution_date = task_instance.execution_date - task_owner = task_instance.task.owner - - if 'task_id' in kwargs: - self.task_id = kwargs['task_id'] - if 'dag_id' in kwargs: - self.dag_id = kwargs['dag_id'] - if 'execution_date' in kwargs: - if kwargs['execution_date']: - self.execution_date = kwargs['execution_date'] - - self.owner = owner or task_owner - - class SkipMixin(LoggingMixin): @provide_session def skip(self, dag_run, execution_date, tasks, session=None): diff --git a/airflow/models/log.py b/airflow/models/log.py new file mode 100644 index 0000000000000..fa3fed60ee240 --- /dev/null +++ b/airflow/models/log.py @@ -0,0 +1,68 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from sqlalchemy import Column, Integer, String, Text, Index + +from airflow.models.base import Base, ID_LEN +from airflow.utils import timezone +from airflow.utils.sqlalchemy import UtcDateTime + + +class Log(Base): + """ + Used to actively log events to the database + """ + + __tablename__ = "log" + + id = Column(Integer, primary_key=True) + dttm = Column(UtcDateTime) + dag_id = Column(String(ID_LEN)) + task_id = Column(String(ID_LEN)) + event = Column(String(30)) + execution_date = Column(UtcDateTime) + owner = Column(String(500)) + extra = Column(Text) + + __table_args__ = ( + Index('idx_log_dag', dag_id), + ) + + def __init__(self, event, task_instance, owner=None, extra=None, **kwargs): + self.dttm = timezone.utcnow() + self.event = event + self.extra = extra + + task_owner = None + + if task_instance: + self.dag_id = task_instance.dag_id + self.task_id = task_instance.task_id + self.execution_date = task_instance.execution_date + task_owner = task_instance.task.owner + + if 'task_id' in kwargs: + self.task_id = kwargs['task_id'] + if 'dag_id' in kwargs: + self.dag_id = kwargs['dag_id'] + if 'execution_date' in kwargs: + if kwargs['execution_date']: + self.execution_date = kwargs['execution_date'] + + self.owner = owner or task_owner diff --git a/airflow/utils/cli.py b/airflow/utils/cli.py index 32303cd90bd8d..6c3bf42c9458c 100644 --- a/airflow/utils/cli.py +++ b/airflow/utils/cli.py @@ -30,7 +30,7 @@ from argparse import Namespace from datetime import datetime -import airflow.models +from airflow.models.log import Log from airflow.utils import cli_action_loggers @@ -46,7 +46,7 @@ def action_logging(f): end_datetime : end datetime instance by utc full_command : full command line arguments user : current user - log : airflow.models.Log ORM instance + log : airflow.models.log.Log ORM instance dag_id : dag id (optional) task_id : task_id (optional) execution_date : execution date (optional) @@ -105,7 +105,7 @@ def _build_metrics(func_name, namespace): metrics['host_name'] = socket.gethostname() extra = json.dumps(dict((k, metrics[k]) for k in ('host_name', 'full_command'))) - log = airflow.models.Log( + log = Log( event='cli_{}'.format(func_name), task_instance=None, owner=metrics['user'], diff --git a/airflow/www_rbac/decorators.py b/airflow/www_rbac/decorators.py index 4ec2c7e8e9774..889a0b1b33b2f 100644 --- a/airflow/www_rbac/decorators.py +++ b/airflow/www_rbac/decorators.py @@ -22,7 +22,7 @@ import pendulum from io import BytesIO as IO from flask import after_this_request, redirect, request, url_for, g -from airflow import models +from airflow.models.log import Log from airflow.utils.db import create_session @@ -39,7 +39,7 @@ def wrapper(*args, **kwargs): else: user = g.user.username - log = models.Log( + log = Log( event=f.__name__, task_instance=None, owner=user, diff --git a/airflow/www_rbac/views.py b/airflow/www_rbac/views.py index d6060629c81b6..c44d25bd474c8 100644 --- a/airflow/www_rbac/views.py +++ b/airflow/www_rbac/views.py @@ -55,6 +55,7 @@ set_dag_run_state_to_failed) from airflow.models import XCom, DagRun, errors, DagModel from airflow.models.connection import Connection +from airflow.models.log import Log from airflow.models.taskfail import TaskFail from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, SCHEDULER_DEPS from airflow.utils import timezone @@ -2277,7 +2278,7 @@ def action_set_success(self, drs, session=None): class LogModelView(AirflowModelView): route_base = '/log' - datamodel = AirflowModelView.CustomSQLAInterface(models.Log) + datamodel = AirflowModelView.CustomSQLAInterface(Log) base_permissions = ['can_list'] diff --git a/tests/api/common/experimental/test_delete_dag.py b/tests/api/common/experimental/test_delete_dag.py index aaca6be47d781..3ab2c030a4e60 100644 --- a/tests/api/common/experimental/test_delete_dag.py +++ b/tests/api/common/experimental/test_delete_dag.py @@ -30,7 +30,7 @@ DM = models.DagModel DR = models.DagRun TI = models.TaskInstance -LOG = models.Log +LOG = models.log.Log TF = models.TaskFail TR = models.TaskReschedule From 89b9409fa6ed48cf92dec6bd564241020a7f43a8 Mon Sep 17 00:00:00 2001 From: BasPH Date: Sun, 10 Feb 2019 12:47:41 +0100 Subject: [PATCH 776/808] [AIRFLOW-3464] Move SkipMixin out of models.py (#4386) --- airflow/models/__init__.py | 43 +--------- airflow/models/skipmixin.py | 66 ++++++++++++++++ airflow/operators/latest_only_operator.py | 3 +- airflow/operators/python_operator.py | 3 +- airflow/sensors/base_sensor_operator.py | 3 +- tests/models/test_skipmixin.py | 95 +++++++++++++++++++++++ tests/test_models.py | 70 +---------------- 7 files changed, 171 insertions(+), 112 deletions(-) create mode 100644 airflow/models/skipmixin.py create mode 100644 tests/models/test_skipmixin.py diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index 6c454c9ac41f6..30d62983a8a9b 100755 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -1934,48 +1934,6 @@ def init_run_context(self, raw=False): self._set_context(self) -class SkipMixin(LoggingMixin): - @provide_session - def skip(self, dag_run, execution_date, tasks, session=None): - """ - Sets tasks instances to skipped from the same dag run. - - :param dag_run: the DagRun for which to set the tasks to skipped - :param execution_date: execution_date - :param tasks: tasks to skip (not task_ids) - :param session: db session to use - """ - if not tasks: - return - - task_ids = [d.task_id for d in tasks] - now = timezone.utcnow() - - if dag_run: - session.query(TaskInstance).filter( - TaskInstance.dag_id == dag_run.dag_id, - TaskInstance.execution_date == dag_run.execution_date, - TaskInstance.task_id.in_(task_ids) - ).update({TaskInstance.state: State.SKIPPED, - TaskInstance.start_date: now, - TaskInstance.end_date: now}, - synchronize_session=False) - session.commit() - else: - assert execution_date is not None, "Execution date is None and no dag run" - - self.log.warning("No DAG RUN present this should not happen") - # this is defensive against dag runs that are not complete - for task in tasks: - ti = TaskInstance(task, execution_date=execution_date) - ti.state = State.SKIPPED - ti.start_date = now - ti.end_date = now - session.merge(ti) - - session.commit() - - @functools.total_ordering class BaseOperator(LoggingMixin): """ @@ -5070,3 +5028,4 @@ def open_slots(self, session): # To avoid circular import on Python2.7 we need to define this at the _bottom_ from airflow.models.connection import Connection # noqa: E402,F401 +from airflow.models.skipmixin import SkipMixin # noqa: F401 diff --git a/airflow/models/skipmixin.py b/airflow/models/skipmixin.py new file mode 100644 index 0000000000000..c0adbd20aa4a4 --- /dev/null +++ b/airflow/models/skipmixin.py @@ -0,0 +1,66 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.models import TaskInstance +from airflow.utils import timezone +from airflow.utils.db import provide_session +from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.state import State + + +class SkipMixin(LoggingMixin): + @provide_session + def skip(self, dag_run, execution_date, tasks, session=None): + """ + Sets tasks instances to skipped from the same dag run. + + :param dag_run: the DagRun for which to set the tasks to skipped + :param execution_date: execution_date + :param tasks: tasks to skip (not task_ids) + :param session: db session to use + """ + if not tasks: + return + + task_ids = [d.task_id for d in tasks] + now = timezone.utcnow() + + if dag_run: + session.query(TaskInstance).filter( + TaskInstance.dag_id == dag_run.dag_id, + TaskInstance.execution_date == dag_run.execution_date, + TaskInstance.task_id.in_(task_ids) + ).update({TaskInstance.state: State.SKIPPED, + TaskInstance.start_date: now, + TaskInstance.end_date: now}, + synchronize_session=False) + session.commit() + else: + assert execution_date is not None, "Execution date is None and no dag run" + + self.log.warning("No DAG RUN present this should not happen") + # this is defensive against dag runs that are not complete + for task in tasks: + ti = TaskInstance(task, execution_date=execution_date) + ti.state = State.SKIPPED + ti.start_date = now + ti.end_date = now + session.merge(ti) + + session.commit() diff --git a/airflow/operators/latest_only_operator.py b/airflow/operators/latest_only_operator.py index 113977491cc4b..6fee50965d42a 100644 --- a/airflow/operators/latest_only_operator.py +++ b/airflow/operators/latest_only_operator.py @@ -17,7 +17,8 @@ # specific language governing permissions and limitations # under the License. -from airflow.models import BaseOperator, SkipMixin +from airflow.models import BaseOperator +from airflow.models.skipmixin import SkipMixin from airflow.utils import timezone diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py index a69eb535196ce..30f5290973b74 100644 --- a/airflow/operators/python_operator.py +++ b/airflow/operators/python_operator.py @@ -29,7 +29,8 @@ from builtins import str from airflow.exceptions import AirflowException -from airflow.models import BaseOperator, SkipMixin +from airflow.models import BaseOperator +from airflow.models.skipmixin import SkipMixin from airflow.utils.decorators import apply_defaults from airflow.utils.file import TemporaryDirectory from airflow.utils.operator_helpers import context_to_airflow_vars diff --git a/airflow/sensors/base_sensor_operator.py b/airflow/sensors/base_sensor_operator.py index 2e99704c86feb..fe96a36d8b3ec 100644 --- a/airflow/sensors/base_sensor_operator.py +++ b/airflow/sensors/base_sensor_operator.py @@ -23,7 +23,8 @@ from airflow.exceptions import AirflowException, AirflowSensorTimeout, \ AirflowSkipException, AirflowRescheduleException -from airflow.models import BaseOperator, SkipMixin +from airflow.models import BaseOperator +from airflow.models.skipmixin import SkipMixin from airflow.models.taskreschedule import TaskReschedule from airflow.utils import timezone from airflow.utils.decorators import apply_defaults diff --git a/tests/models/test_skipmixin.py b/tests/models/test_skipmixin.py new file mode 100644 index 0000000000000..9b1230409dc2c --- /dev/null +++ b/tests/models/test_skipmixin.py @@ -0,0 +1,95 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import datetime +import unittest + +import pendulum +from mock import patch, Mock + +from airflow.models import settings, DAG, TaskInstance as TI +from airflow.models.skipmixin import SkipMixin +from airflow.operators.dummy_operator import DummyOperator +from airflow.utils import timezone +from airflow.utils.state import State + +DEFAULT_DATE = timezone.datetime(2016, 1, 1) + + +class TestSkipMixin(unittest.TestCase): + + @patch('airflow.models.timezone.utcnow') + def test_skip(self, mock_now): + session = settings.Session() + now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) + mock_now.return_value = now + dag = DAG( + 'dag', + start_date=DEFAULT_DATE, + ) + with dag: + tasks = [DummyOperator(task_id='task')] + dag_run = dag.create_dagrun( + run_id='manual__' + now.isoformat(), + state=State.FAILED, + ) + SkipMixin().skip( + dag_run=dag_run, + execution_date=now, + tasks=tasks, + session=session) + + session.query(TI).filter( + TI.dag_id == 'dag', + TI.task_id == 'task', + TI.state == State.SKIPPED, + TI.start_date == now, + TI.end_date == now, + ).one() + + @patch('airflow.models.timezone.utcnow') + def test_skip_none_dagrun(self, mock_now): + session = settings.Session() + now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) + mock_now.return_value = now + dag = DAG( + 'dag', + start_date=DEFAULT_DATE, + ) + with dag: + tasks = [DummyOperator(task_id='task')] + SkipMixin().skip( + dag_run=None, + execution_date=now, + tasks=tasks, + session=session) + + session.query(TI).filter( + TI.dag_id == 'dag', + TI.task_id == 'task', + TI.state == State.SKIPPED, + TI.start_date == now, + TI.end_date == now, + ).one() + + def test_skip_none_tasks(self): + session = Mock() + SkipMixin().skip(dag_run=None, execution_date=None, tasks=[], session=session) + self.assertFalse(session.query.called) + self.assertFalse(session.commit.called) diff --git a/tests/test_models.py b/tests/test_models.py index 2f9670f27bc93..b04c012ff9663 100644 --- a/tests/test_models.py +++ b/tests/test_models.py @@ -37,10 +37,10 @@ import pendulum import six -from mock import ANY, Mock, mock_open, patch -from parameterized import parameterized -from freezegun import freeze_time from cryptography.fernet import Fernet +from freezegun import freeze_time +from mock import ANY, mock_open, patch +from parameterized import parameterized from airflow import AirflowException, configuration, models, settings from airflow.contrib.sensors.python_sensor import PythonSensor @@ -48,7 +48,6 @@ from airflow.jobs import BackfillJob from airflow.models import DAG, TaskInstance as TI, DagBag from airflow.models import DagModel, DagRun -from airflow.models import SkipMixin from airflow.models import State as ST from airflow.models import XCom from airflow.models import Variable @@ -3276,66 +3275,3 @@ def test_var_with_encryption_rotate_fernet_key(self, mock_get): self.assertTrue(test_var.is_encrypted) self.assertEqual(test_var.val, 'value') self.assertEqual(Fernet(key2).decrypt(test_var._val.encode()), b'value') - - -class TestSkipMixin(unittest.TestCase): - - @patch('airflow.models.timezone.utcnow') - def test_skip(self, mock_now): - session = settings.Session() - now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) - mock_now.return_value = now - dag = DAG( - 'dag', - start_date=DEFAULT_DATE, - ) - with dag: - tasks = [DummyOperator(task_id='task')] - dag_run = dag.create_dagrun( - run_id='manual__' + now.isoformat(), - state=State.FAILED, - ) - SkipMixin().skip( - dag_run=dag_run, - execution_date=now, - tasks=tasks, - session=session) - - session.query(TI).filter( - TI.dag_id == 'dag', - TI.task_id == 'task', - TI.state == State.SKIPPED, - TI.start_date == now, - TI.end_date == now, - ).one() - - @patch('airflow.models.timezone.utcnow') - def test_skip_none_dagrun(self, mock_now): - session = settings.Session() - now = datetime.datetime.utcnow().replace(tzinfo=pendulum.timezone('UTC')) - mock_now.return_value = now - dag = DAG( - 'dag', - start_date=DEFAULT_DATE, - ) - with dag: - tasks = [DummyOperator(task_id='task')] - SkipMixin().skip( - dag_run=None, - execution_date=now, - tasks=tasks, - session=session) - - session.query(TI).filter( - TI.dag_id == 'dag', - TI.task_id == 'task', - TI.state == State.SKIPPED, - TI.start_date == now, - TI.end_date == now, - ).one() - - def test_skip_none_tasks(self): - session = Mock() - SkipMixin().skip(dag_run=None, execution_date=None, tasks=[], session=session) - self.assertFalse(session.query.called) - self.assertFalse(session.commit.called) From 3aad18feba552de3318721a26ee1edb0acdb6f03 Mon Sep 17 00:00:00 2001 From: saurabh gulati Date: Sun, 31 Mar 2019 18:55:46 +0200 Subject: [PATCH 777/808] [AIRFLOW-4172] Fix changes for driver class path option in Spark Submit (#4992) --- UPDATING.md | 9 +++++++++ airflow/contrib/hooks/spark_submit_hook.py | 12 ++++++------ airflow/contrib/operators/spark_submit_operator.py | 12 ++++++------ .../contrib/operators/test_spark_submit_operator.py | 6 +++--- 4 files changed, 24 insertions(+), 15 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index bdb35cb1aed6d..3a0667f9927e7 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -200,6 +200,15 @@ The change was made to keep all the operators related to GCS Transfer Services i The previous imports will continue to work until Airflow 2.0 +### Fixed typo in --driver-class-path in SparkSubmitHook + +The `driver_classapth` argument to SparkSubmit Hook and Operator was +generating `--driver-classpath` on the spark command line, but this isn't a +valid option to spark. + +The argument has been renamed to `driver_class_path` and the option it +generates has been fixed. + ## Airflow 1.10.2 ### DAG level Access Control for new RBAC UI diff --git a/airflow/contrib/hooks/spark_submit_hook.py b/airflow/contrib/hooks/spark_submit_hook.py index 51a59d460022a..677ea9000843b 100644 --- a/airflow/contrib/hooks/spark_submit_hook.py +++ b/airflow/contrib/hooks/spark_submit_hook.py @@ -47,8 +47,8 @@ class SparkSubmitHook(BaseHook, LoggingMixin): :type py_files: str :param: archives: Archives that spark should unzip (and possibly tag with #ALIAS) into the application working directory. - :param driver_classpath: Additional, driver-specific, classpath settings. - :type driver_classpath: str + :param driver_class_path: Additional, driver-specific, classpath settings. + :type driver_class_path: str :param jars: Submit additional jars to upload and place them in executor classpath. :type jars: str :param java_class: the main class of the Java application @@ -97,7 +97,7 @@ def __init__(self, files=None, py_files=None, archives=None, - driver_classpath=None, + driver_class_path=None, jars=None, java_class=None, packages=None, @@ -120,7 +120,7 @@ def __init__(self, self._files = files self._py_files = py_files self._archives = archives - self._driver_classpath = driver_classpath + self._driver_class_path = driver_class_path self._jars = jars self._java_class = java_class self._packages = packages @@ -250,8 +250,8 @@ def _build_spark_submit_command(self, application): connection_cmd += ["--py-files", self._py_files] if self._archives: connection_cmd += ["--archives", self._archives] - if self._driver_classpath: - connection_cmd += ["--driver-classpath", self._driver_classpath] + if self._driver_class_path: + connection_cmd += ["--driver-class-path", self._driver_class_path] if self._jars: connection_cmd += ["--jars", self._jars] if self._packages: diff --git a/airflow/contrib/operators/spark_submit_operator.py b/airflow/contrib/operators/spark_submit_operator.py index 93dc08f1f77e0..45a70b64ea827 100644 --- a/airflow/contrib/operators/spark_submit_operator.py +++ b/airflow/contrib/operators/spark_submit_operator.py @@ -44,8 +44,8 @@ class SparkSubmitOperator(BaseOperator): :type py_files: str :param jars: Submit additional jars to upload and place them in executor classpath. (templated) :type jars: str - :param driver_classpath: Additional, driver-specific, classpath settings. (templated) - :type driver_classpath: str + :param driver_class_path: Additional, driver-specific, classpath settings. (templated) + :type driver_class_path: str :param java_class: the main class of the Java application :type java_class: str :param packages: Comma-separated list of maven coordinates of jars to include on the @@ -84,7 +84,7 @@ class SparkSubmitOperator(BaseOperator): Some distros may use spark2-submit. :type spark_binary: string """ - template_fields = ('_application', '_conf', '_files', '_py_files', '_jars', '_driver_classpath', + template_fields = ('_application', '_conf', '_files', '_py_files', '_jars', '_driver_class_path', '_packages', '_exclude_packages', '_keytab', '_principal', '_name', '_application_args', '_env_vars') ui_color = WEB_COLORS['LIGHTORANGE'] @@ -97,7 +97,7 @@ def __init__(self, files=None, py_files=None, archives=None, - driver_classpath=None, + driver_class_path=None, jars=None, java_class=None, packages=None, @@ -123,7 +123,7 @@ def __init__(self, self._files = files self._py_files = py_files self._archives = archives - self._driver_classpath = driver_classpath + self._driver_class_path = driver_class_path self._jars = jars self._java_class = java_class self._packages = packages @@ -154,7 +154,7 @@ def execute(self, context): files=self._files, py_files=self._py_files, archives=self._archives, - driver_classpath=self._driver_classpath, + driver_class_path=self._driver_class_path, jars=self._jars, java_class=self._java_class, packages=self._packages, diff --git a/tests/contrib/operators/test_spark_submit_operator.py b/tests/contrib/operators/test_spark_submit_operator.py index b865fda930108..be596ca6ea2eb 100644 --- a/tests/contrib/operators/test_spark_submit_operator.py +++ b/tests/contrib/operators/test_spark_submit_operator.py @@ -41,7 +41,7 @@ class TestSparkSubmitOperator(unittest.TestCase): 'files': 'hive-site.xml', 'py_files': 'sample_library.py', 'archives': 'sample_archive.zip#SAMPLE', - 'driver_classpath': 'parquet.jar', + 'driver_class_path': 'parquet.jar', 'jars': 'parquet.jar', 'packages': 'com.databricks:spark-avro_2.11:3.2.0', 'exclude_packages': 'org.bad.dependency:1.0.0', @@ -93,7 +93,7 @@ def test_execute(self): 'files': 'hive-site.xml', 'py_files': 'sample_library.py', 'archives': 'sample_archive.zip#SAMPLE', - 'driver_classpath': 'parquet.jar', + 'driver_class_path': 'parquet.jar', 'jars': 'parquet.jar', 'packages': 'com.databricks:spark-avro_2.11:3.2.0', 'exclude_packages': 'org.bad.dependency:1.0.0', @@ -125,7 +125,7 @@ def test_execute(self): self.assertEqual(expected_dict['files'], operator._files) self.assertEqual(expected_dict['py_files'], operator._py_files) self.assertEqual(expected_dict['archives'], operator._archives) - self.assertEqual(expected_dict['driver_classpath'], operator._driver_classpath) + self.assertEqual(expected_dict['driver_class_path'], operator._driver_class_path) self.assertEqual(expected_dict['jars'], operator._jars) self.assertEqual(expected_dict['packages'], operator._packages) self.assertEqual(expected_dict['exclude_packages'], operator._exclude_packages) From 47a744f8a5b62ac61495465c1b93611d507c790f Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Mon, 1 Apr 2019 16:05:28 +0100 Subject: [PATCH 778/808] [AIRFLOW-XXX] Version 1.10.3b2 --- airflow/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/version.py b/airflow/version.py index 0fd74fd4456b8..153b1ff2a0e67 100644 --- a/airflow/version.py +++ b/airflow/version.py @@ -18,4 +18,4 @@ # under the License. # -version = '1.10.3b1' +version = '1.10.3b2' From 7e184fca8c86a884f706ec789adc10973fc001f5 Mon Sep 17 00:00:00 2001 From: Asgeir Berland Date: Fri, 22 Mar 2019 05:40:59 +0100 Subject: [PATCH 779/808] [AIRFLOW-4127] Correct AzureContainerInstanceHook._get_instance_view's return (#4945) --- airflow/contrib/hooks/azure_container_instance_hook.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/contrib/hooks/azure_container_instance_hook.py b/airflow/contrib/hooks/azure_container_instance_hook.py index 988cbd9e74ba6..3f29c6de4af5a 100644 --- a/airflow/contrib/hooks/azure_container_instance_hook.py +++ b/airflow/contrib/hooks/azure_container_instance_hook.py @@ -108,7 +108,7 @@ def _get_instance_view(self, resource_group, name): response = self.connection.container_groups.get(resource_group, name, raw=False) - return response.containers[0].instance_view.current_state + return response.containers[0].instance_view def get_messages(self, resource_group, name): """ From 9a9ece5d2d6d3161abbd2568f52d0ce9f84bc27c Mon Sep 17 00:00:00 2001 From: marengaz Date: Fri, 22 Mar 2019 16:43:50 +0000 Subject: [PATCH 780/808] [AIRFLOW-4144] add description of is_delete_operator_pod (#4943) [AIRFLOW-4144] add description of is_delete_operator_pod --- airflow/contrib/operators/kubernetes_pod_operator.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py index 46cde7d0750c6..1e6eef6203810 100644 --- a/airflow/contrib/operators/kubernetes_pod_operator.py +++ b/airflow/contrib/operators/kubernetes_pod_operator.py @@ -77,6 +77,10 @@ class KubernetesPodOperator(BaseOperator): /airflow/xcom/return.json in the container will also be pushed to an XCom when the container completes. :type xcom_push: bool + :param is_delete_operator_pod: What to do when the pod reaches its final + state, or the execution is interrupted. + If False (default): do nothing, If True: delete the pod + :type is_delete_operator_pod: bool :param hostnetwork: If True enable host networking on the pod :type hostnetwork: bool :param tolerations: A list of kubernetes tolerations From 41fd951f9087e8cecc0e4255102c9f551e1acda4 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 27 Mar 2019 15:14:58 +0000 Subject: [PATCH 781/808] [AIRFLOW-3419] Fix S3Hook.select_key on Python3 (#4970) --- airflow/hooks/S3_hook.py | 2 +- tests/hooks/test_s3_hook.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py index f8ccb370f8ce0..a8e4d865e649a 100644 --- a/airflow/hooks/S3_hook.py +++ b/airflow/hooks/S3_hook.py @@ -272,7 +272,7 @@ def select_key(self, key, bucket_name=None, InputSerialization=input_serialization, OutputSerialization=output_serialization) - return ''.join(event['Records']['Payload'] + return ''.join(event['Records']['Payload'].decode('utf-8') for event in response['Payload'] if 'Records' in event) diff --git a/tests/hooks/test_s3_hook.py b/tests/hooks/test_s3_hook.py index e0f9e8a3eca53..62092343cdf0d 100644 --- a/tests/hooks/test_s3_hook.py +++ b/tests/hooks/test_s3_hook.py @@ -211,7 +211,7 @@ def test_read_key(self): @mock.patch('airflow.contrib.hooks.aws_hook.AwsHook.get_client_type') def test_select_key(self, mock_get_client_type): mock_get_client_type.return_value.select_object_content.return_value = \ - {'Payload': [{'Records': {'Payload': u'Contént'}}]} + {'Payload': [{'Records': {'Payload': b'Cont\xC3\xA9nt'}}]} hook = S3Hook(aws_conn_id=None) self.assertEqual(hook.select_key('my_key', 'mybucket'), u'Contént') From 950d059851898447db6537cd663d818ac54f5dc6 Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Tue, 26 Feb 2019 01:50:00 +0800 Subject: [PATCH 782/808] [AIRFLOW-3947] Flash msg for no DAG-level access error (#4767) * [AIRFLOW-3947] Flash msg for no DAG-level access error It will show and remind user when a user clicks on a DAG that he/she doesn't have can_dag_read or can_dag_edit permissions. * Change the flash msg contents --- airflow/www_rbac/decorators.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/www_rbac/decorators.py b/airflow/www_rbac/decorators.py index 889a0b1b33b2f..1db2dfeb363b6 100644 --- a/airflow/www_rbac/decorators.py +++ b/airflow/www_rbac/decorators.py @@ -21,7 +21,7 @@ import functools import pendulum from io import BytesIO as IO -from flask import after_this_request, redirect, request, url_for, g +from flask import after_this_request, flash, redirect, request, url_for, g from airflow.models.log import Log from airflow.utils.db import create_session @@ -120,6 +120,7 @@ def wrapper(self, *args, **kwargs): dag_id)))): return f(self, *args, **kwargs) else: + flash("Access is Denied", "danger") return redirect(url_for(self.appbuilder.sm.auth_view. __class__.__name__ + ".login")) return wrapper From 4a052013d4c28d53d288fbe159c3b39153b1fc3a Mon Sep 17 00:00:00 2001 From: davlum Date: Wed, 3 Apr 2019 05:18:39 -0400 Subject: [PATCH 783/808] [AIRFLOW-4008] add envFrom for Kubernetes Executor (#4952) --- airflow/config_templates/default_airflow.cfg | 12 +- .../contrib/executors/kubernetes_executor.py | 4 + .../kubernetes_request_factory.py | 16 +- airflow/contrib/kubernetes/pod.py | 6 +- airflow/contrib/kubernetes/secret.py | 40 ++- .../kubernetes/worker_configuration.py | 18 +- .../executors/test_kubernetes_executor.py | 48 ++- tests/contrib/kubernetes/__init__.py | 18 ++ .../kubernetes_request_factory/__init__.py | 18 ++ .../test_kubernetes_request_factory.py | 306 ++++++++++++++++++ .../test_pod_request_factory.py | 159 +++++++++ 11 files changed, 612 insertions(+), 33 deletions(-) create mode 100644 tests/contrib/kubernetes/__init__.py create mode 100644 tests/contrib/kubernetes/kubernetes_request_factory/__init__.py create mode 100644 tests/contrib/kubernetes/kubernetes_request_factory/test_kubernetes_request_factory.py create mode 100644 tests/contrib/kubernetes/kubernetes_request_factory/test_pod_request_factory.py diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 95f06ccd16e19..b83edefb4cfac 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -644,6 +644,14 @@ dags_volume_host = # Useful in local environment, discouraged in production logs_volume_host = +# A list of configMapsRefs to envFrom. If more than one configMap is +# specified, provide a comma separated list: configmap_a,configmap_b +env_from_configmap_ref = + +# A list of secretRefs to envFrom. If more than one secret is +# specified, provide a comma separated list: secret_a,secret_b +env_from_secret_ref = + # Git credentials and repository for DAGs mounted via Git (mutually exclusive with volume claim) git_repo = git_branch = @@ -753,12 +761,12 @@ tolerations = # scheduler. You may define as many secrets as needed and the kubernetes launcher will parse the # defined secrets and mount them as secret environment variables in the launched workers. # Secrets in this section are defined as follows -# = : +# = = # # For example if you wanted to mount a kubernetes secret key named `postgres_password` from the # kubernetes secret object `airflow-secret` as the environment variable `POSTGRES_PASSWORD` into # your workers you would follow the following format: -# POSTGRES_PASSWORD = airflow-secret:postgres_credentials +# POSTGRES_PASSWORD = airflow-secret=postgres_credentials # # Additionally you may override worker airflow settings with the AIRFLOW__
    __ # formatting as supported by airflow normally. diff --git a/airflow/contrib/executors/kubernetes_executor.py b/airflow/contrib/executors/kubernetes_executor.py index 38abc5ed85a08..83babdafa44a2 100644 --- a/airflow/contrib/executors/kubernetes_executor.py +++ b/airflow/contrib/executors/kubernetes_executor.py @@ -125,6 +125,10 @@ def __init__(self): self.core_configuration = configuration_dict['core'] self.kube_secrets = configuration_dict.get('kubernetes_secrets', {}) self.kube_env_vars = configuration_dict.get('kubernetes_environment_variables', {}) + self.env_from_configmap_ref = configuration.get(self.kubernetes_section, + 'env_from_configmap_ref') + self.env_from_secret_ref = configuration.get(self.kubernetes_section, + 'env_from_secret_ref') self.airflow_home = settings.AIRFLOW_HOME self.dags_folder = configuration.get(self.core_section, 'dags_folder') self.parallelism = configuration.getint(self.core_section, 'PARALLELISM') diff --git a/airflow/contrib/kubernetes/kubernetes_request_factory/kubernetes_request_factory.py b/airflow/contrib/kubernetes/kubernetes_request_factory/kubernetes_request_factory.py index ea46e1bcd385d..d1b6d4eaf4dca 100644 --- a/airflow/contrib/kubernetes/kubernetes_request_factory/kubernetes_request_factory.py +++ b/airflow/contrib/kubernetes/kubernetes_request_factory/kubernetes_request_factory.py @@ -75,6 +75,12 @@ def extract_affinity(pod, req): for k, v in six.iteritems(pod.affinity): req['spec']['affinity'][k] = v + @staticmethod + def extract_node_selector(pod, req): + req['spec']['nodeSelector'] = req['spec'].get('nodeSelector', {}) + for k, v in six.iteritems(pod.node_selectors): + req['spec']['nodeSelector'][k] = v + @staticmethod def extract_cmds(pod, req): req['spec']['containers'][0]['command'] = pod.cmds @@ -83,12 +89,6 @@ def extract_cmds(pod, req): def extract_args(pod, req): req['spec']['containers'][0]['args'] = pod.args - @staticmethod - def extract_node_selector(pod, req): - req['spec']['nodeSelector'] = req['spec'].get('nodeSelector', {}) - for k, v in six.iteritems(pod.node_selectors): - req['spec']['nodeSelector'][k] = v - @staticmethod def attach_volumes(pod, req): req['spec']['volumes'] = ( @@ -132,7 +132,7 @@ def extract_volume_secrets(pod, req): @staticmethod def extract_env_and_secrets(pod, req): envs_from_key_secrets = [ - env for env in pod.secrets if env.deploy_type == 'env' and hasattr(env, 'key') + env for env in pod.secrets if env.deploy_type == 'env' and env.key is not None ] if len(pod.envs) > 0 or len(envs_from_key_secrets) > 0: @@ -206,7 +206,7 @@ def extract_security_context(pod, req): @staticmethod def _apply_env_from(pod, req): envs_from_secrets = [ - env for env in pod.secrets if env.deploy_type == 'env' and not hasattr(env, 'key') + env for env in pod.secrets if env.deploy_type == 'env' and env.key is None ] if pod.configmaps or envs_from_secrets: diff --git a/airflow/contrib/kubernetes/pod.py b/airflow/contrib/kubernetes/pod.py index 3d7c138b7e179..f8c0bdffaee08 100644 --- a/airflow/contrib/kubernetes/pod.py +++ b/airflow/contrib/kubernetes/pod.py @@ -50,13 +50,13 @@ class Pod: :param secrets: Secrets to be launched to the pod :type secrets: list[airflow.contrib.kubernetes.secret.Secret] :param result: The result that will be returned to the operator after - successful execution of the pod + successful execution of the pod :type result: any :param image_pull_policy: Specify a policy to cache or always pull an image :type image_pull_policy: str :param image_pull_secrets: Any image pull secrets to be given to the pod. - If more than one secret is required, provide a - comma separated list: secret_a,secret_b + If more than one secret is required, provide a comma separated list: + secret_a,secret_b :type image_pull_secrets: str :param affinity: A dict containing a group of affinity scheduling rules :type affinity: dict diff --git a/airflow/contrib/kubernetes/secret.py b/airflow/contrib/kubernetes/secret.py index d69f507717682..73c51e900acf9 100644 --- a/airflow/contrib/kubernetes/secret.py +++ b/airflow/contrib/kubernetes/secret.py @@ -17,7 +17,7 @@ from airflow.exceptions import AirflowConfigException -class Secret: +class Secret(object): """Defines Kubernetes Secret Volume""" def __init__(self, deploy_type, deploy_target, secret, key=None): @@ -26,10 +26,10 @@ def __init__(self, deploy_type, deploy_target, secret, key=None): :param deploy_type: The type of secret deploy in Kubernetes, either `env` or `volume` :type deploy_type: str - :param deploy_target: The environment variable when `deploy_type` `env` or - file path when `deploy_type` `volume` where expose secret. - If `key` is not provided deploy target should be None. - :type deploy_target: str + :param deploy_target: (Optional) The environment variable when + `deploy_type` `env` or file path when `deploy_type` `volume` where + expose secret. If `key` is not provided deploy target should be None. + :type deploy_target: str or None :param secret: Name of the secrets object in Kubernetes :type secret: str :param key: (Optional) Key of the secret within the Kubernetes Secret @@ -37,18 +37,32 @@ def __init__(self, deploy_type, deploy_target, secret, key=None): :type key: str or None """ self.deploy_type = deploy_type + self.deploy_target = deploy_target - if deploy_target: + if deploy_target is not None and deploy_type == 'env': + # if deploying to env, capitalize the deploy target self.deploy_target = deploy_target.upper() - if deploy_type == 'volume': - self.deploy_target = deploy_target - - if not deploy_type == 'env' and key is None: + if key is not None and deploy_target is None: raise AirflowConfigException( - 'In deploy_type different than `env` parameter `key` is mandatory' + 'If `key` is set, `deploy_target` should not be None' ) self.secret = secret - if key: - self.key = key + self.key = key + + def __eq__(self, other): + return ( + self.deploy_type == other.deploy_type and + self.deploy_target == other.deploy_target and + self.secret == other.secret and + self.key == other.key + ) + + def __repr__(self): + return 'Secret({}, {}, {}, {})'.format( + self.deploy_type, + self.deploy_target, + self.secret, + self.key + ) diff --git a/airflow/contrib/kubernetes/worker_configuration.py b/airflow/contrib/kubernetes/worker_configuration.py index e8bc7384a0f82..60128f1479e81 100644 --- a/airflow/contrib/kubernetes/worker_configuration.py +++ b/airflow/contrib/kubernetes/worker_configuration.py @@ -155,13 +155,28 @@ def _get_environment(self): env['AIRFLOW__CORE__DAGS_FOLDER'] = dag_volume_mount_path return env + def _get_configmaps(self): + """Extracts any configmapRefs to envFrom""" + if not self.kube_config.env_from_configmap_ref: + return [] + return self.kube_config.env_from_configmap_ref.split(',') + def _get_secrets(self): """Defines any necessary secrets for the pod executor""" worker_secrets = [] + for env_var_name, obj_key_pair in six.iteritems(self.kube_config.kube_secrets): k8s_secret_obj, k8s_secret_key = obj_key_pair.split('=') worker_secrets.append( - Secret('env', env_var_name, k8s_secret_obj, k8s_secret_key)) + Secret('env', env_var_name, k8s_secret_obj, k8s_secret_key) + ) + + if self.kube_config.env_from_secret_ref: + for secret_ref in self.kube_config.env_from_secret_ref.split(','): + worker_secrets.append( + Secret('env', None, secret_ref) + ) + return worker_secrets def _get_image_pull_secrets(self): @@ -331,4 +346,5 @@ def make_pod(self, namespace, worker_uuid, pod_id, dag_id, task_id, execution_da affinity=affinity, tolerations=tolerations, security_context=self._get_security_context(), + configmaps=self._get_configmaps() ) diff --git a/tests/contrib/executors/test_kubernetes_executor.py b/tests/contrib/executors/test_kubernetes_executor.py index 7fac05b8baed5..9e969f390e896 100644 --- a/tests/contrib/executors/test_kubernetes_executor.py +++ b/tests/contrib/executors/test_kubernetes_executor.py @@ -1,5 +1,3 @@ -# -*- coding: utf-8 -*- -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -37,6 +35,7 @@ from airflow.contrib.executors.kubernetes_executor import KubernetesExecutorConfig from airflow.contrib.kubernetes.worker_configuration import WorkerConfiguration from airflow.exceptions import AirflowConfigException + from airflow.contrib.kubernetes.secret import Secret except ImportError: AirflowKubernetesScheduler = None @@ -162,11 +161,8 @@ def setUp(self): self.resources = mock.patch( 'airflow.contrib.kubernetes.worker_configuration.Resources' ) - self.secret = mock.patch( - 'airflow.contrib.kubernetes.worker_configuration.Secret' - ) - for patcher in [self.resources, self.secret]: + for patcher in [self.resources]: self.mock_foo = patcher.start() self.addCleanup(patcher.stop) @@ -590,6 +586,46 @@ def test_kubernetes_environment_variables(self): env = worker_config._get_environment() self.assertEqual(env[core_executor], 'LocalExecutor') + def test_get_secrets(self): + # Test when secretRef is None and kube_secrets is not empty + self.kube_config.kube_secrets = { + 'AWS_SECRET_KEY': 'airflow-secret=aws_secret_key', + 'POSTGRES_PASSWORD': 'airflow-secret=postgres_credentials' + } + self.kube_config.env_from_secret_ref = None + worker_config = WorkerConfiguration(self.kube_config) + secrets = worker_config._get_secrets() + secrets.sort(key=lambda secret: secret.deploy_target) + expected = [ + Secret('env', 'AWS_SECRET_KEY', 'airflow-secret', 'aws_secret_key'), + Secret('env', 'POSTGRES_PASSWORD', 'airflow-secret', 'postgres_credentials') + ] + self.assertListEqual(expected, secrets) + + # Test when secret is not empty and kube_secrets is empty dict + self.kube_config.kube_secrets = {} + self.kube_config.env_from_secret_ref = 'secret_a,secret_b' + worker_config = WorkerConfiguration(self.kube_config) + secrets = worker_config._get_secrets() + expected = [ + Secret('env', None, 'secret_a'), + Secret('env', None, 'secret_b') + ] + self.assertListEqual(expected, secrets) + + def test_get_configmaps(self): + # Test when configmap is empty + self.kube_config.env_from_configmap_ref = '' + worker_config = WorkerConfiguration(self.kube_config) + configmaps = worker_config._get_configmaps() + self.assertListEqual([], configmaps) + + # test when configmap is not empty + self.kube_config.env_from_configmap_ref = 'configmap_a,configmap_b' + worker_config = WorkerConfiguration(self.kube_config) + configmaps = worker_config._get_configmaps() + self.assertListEqual(['configmap_a', 'configmap_b'], configmaps) + class TestKubernetesExecutor(unittest.TestCase): """ diff --git a/tests/contrib/kubernetes/__init__.py b/tests/contrib/kubernetes/__init__.py new file mode 100644 index 0000000000000..114d189da14ab --- /dev/null +++ b/tests/contrib/kubernetes/__init__.py @@ -0,0 +1,18 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/contrib/kubernetes/kubernetes_request_factory/__init__.py b/tests/contrib/kubernetes/kubernetes_request_factory/__init__.py new file mode 100644 index 0000000000000..114d189da14ab --- /dev/null +++ b/tests/contrib/kubernetes/kubernetes_request_factory/__init__.py @@ -0,0 +1,18 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. diff --git a/tests/contrib/kubernetes/kubernetes_request_factory/test_kubernetes_request_factory.py b/tests/contrib/kubernetes/kubernetes_request_factory/test_kubernetes_request_factory.py new file mode 100644 index 0000000000000..e7f444abb14a1 --- /dev/null +++ b/tests/contrib/kubernetes/kubernetes_request_factory/test_kubernetes_request_factory.py @@ -0,0 +1,306 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.kubernetes.kubernetes_request_factory.\ + kubernetes_request_factory import KubernetesRequestFactory +from airflow.contrib.kubernetes.pod import Pod, Resources +from airflow.contrib.kubernetes.secret import Secret +from parameterized import parameterized +import unittest +import copy + + +class TestKubernetesRequestFactory(unittest.TestCase): + + def setUp(self): + + self.expected = { + 'apiVersion': 'v1', + 'kind': 'Pod', + 'metadata': { + 'name': 'name' + }, + 'spec': { + 'containers': [{ + 'name': 'base', + 'image': 'airflow-worker:latest', + 'command': [ + "/usr/local/airflow/entrypoint.sh", + "/bin/bash sleep 25" + ], + }], + 'restartPolicy': 'Never' + } + } + self.input_req = copy.deepcopy(self.expected) + + def test_extract_image(self): + image = 'v3.14' + pod = Pod(image, {}, []) + KubernetesRequestFactory.extract_image(pod, self.input_req) + self.expected['spec']['containers'][0]['image'] = image + self.assertEqual(self.input_req, self.expected) + + def test_extract_image_pull_policy(self): + # Test when pull policy is not none + pull_policy = 'IfNotPresent' + pod = Pod('v3.14', {}, [], image_pull_policy=pull_policy) + + KubernetesRequestFactory.extract_image_pull_policy(pod, self.input_req) + self.expected['spec']['containers'][0]['imagePullPolicy'] = pull_policy + self.assertEqual(self.input_req, self.expected) + + def test_add_secret_to_env(self): + secret = Secret('env', 'target', 'my-secret', 'KEY') + secret_list = [] + self.expected = [{ + 'name': 'TARGET', + 'valueFrom': { + 'secretKeyRef': { + 'name': 'my-secret', + 'key': 'KEY' + } + } + }] + KubernetesRequestFactory.add_secret_to_env(secret_list, secret) + self.assertListEqual(secret_list, self.expected) + + def test_extract_labels(self): + # Test when labels are not empty + labels = {'label_a': 'val_a', 'label_b': 'val_b'} + pod = Pod('v3.14', {}, [], labels=labels) + self.expected['metadata']['labels'] = labels + KubernetesRequestFactory.extract_labels(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_annotations(self): + # Test when annotations are not empty + annotations = {'annot_a': 'val_a', 'annot_b': 'val_b'} + pod = Pod('v3.14', {}, [], annotations=annotations) + self.expected['metadata']['annotations'] = annotations + KubernetesRequestFactory.extract_annotations(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_affinity(self): + # Test when affinity is not empty + affinity = {'podAffinity': 'requiredDuringSchedulingIgnoredDuringExecution'} + pod = Pod('v3.14', {}, [], affinity=affinity) + self.expected['spec']['affinity'] = affinity + KubernetesRequestFactory.extract_affinity(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_node_selector(self): + # Test when affinity is not empty + node_selectors = {'disktype': 'ssd', 'accelerator': 'nvidia-tesla-p100'} + pod = Pod('v3.14', {}, [], node_selectors=node_selectors) + self.expected['spec']['nodeSelector'] = node_selectors + KubernetesRequestFactory.extract_node_selector(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_cmds(self): + cmds = ['test-cmd.sh'] + pod = Pod('v3.14', {}, cmds) + KubernetesRequestFactory.extract_cmds(pod, self.input_req) + self.expected['spec']['containers'][0]['command'] = cmds + self.assertEqual(self.input_req, self.expected) + + def test_extract_args(self): + args = ['test_arg.sh'] + pod = Pod('v3.14', {}, [], args=args) + KubernetesRequestFactory.extract_args(pod, self.input_req) + self.expected['spec']['containers'][0]['args'] = args + self.assertEqual(self.input_req, self.expected) + + def test_attach_volumes(self): + # Test when volumes is not empty list + volumes = ['vol_a', 'vol_b'] + pod = Pod('v3.14', {}, [], volumes=volumes) + self.expected['spec']['volumes'] = volumes + KubernetesRequestFactory.attach_volumes(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_attach_volume_mounts(self): + # Test when volumes is not empty list + volume_mounts = ['vol_a', 'vol_b'] + pod = Pod('v3.14', {}, [], volume_mounts=volume_mounts) + self.expected['spec']['containers'][0]['volumeMounts'] = volume_mounts + KubernetesRequestFactory.attach_volume_mounts(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_name(self): + name = 'pod-name' + pod = Pod('v3.14', {}, [], name=name) + self.expected['metadata']['name'] = name + KubernetesRequestFactory.extract_name(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_volume_secrets(self): + # Test when secrets is not empty + secrets = [ + Secret('volume', 'KEY1', 's1', 'key-1'), + Secret('env', 'KEY2', 's2'), + Secret('volume', 'KEY3', 's3', 'key-2') + ] + pod = Pod('v3.14', {}, [], secrets=secrets) + self.expected['spec']['containers'][0]['volumeMounts'] = [{ + 'mountPath': 'KEY1', + 'name': 'secretvol0', + 'readOnly': True + }, { + 'mountPath': 'KEY3', + 'name': 'secretvol1', + 'readOnly': True + }] + self.expected['spec']['volumes'] = [{ + 'name': 'secretvol0', + 'secret': { + 'secretName': 's1' + } + }, { + 'name': 'secretvol1', + 'secret': { + 'secretName': 's3' + } + }] + KubernetesRequestFactory.extract_volume_secrets(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_env_and_secrets(self): + # Test when secrets and envs are not empty + secrets = [ + Secret('env', None, 's1'), + Secret('volume', 'KEY2', 's2', 'key-2'), + Secret('env', None, 's3') + ] + envs = { + 'ENV1': 'val1', + 'ENV2': 'val2' + } + configmaps = ['configmap_a', 'configmap_b'] + pod = Pod('v3.14', envs, [], secrets=secrets, configmaps=configmaps) + self.expected['spec']['containers'][0]['env'] = [ + {'name': 'ENV1', 'value': 'val1'}, + {'name': 'ENV2', 'value': 'val2'}, + ] + self.expected['spec']['containers'][0]['envFrom'] = [{ + 'secretRef': { + 'name': 's1' + } + }, { + 'secretRef': { + 'name': 's3' + } + }, { + 'configMapRef': { + 'name': 'configmap_a' + } + }, { + 'configMapRef': { + 'name': 'configmap_b' + } + }] + + KubernetesRequestFactory.extract_env_and_secrets(pod, self.input_req) + self.input_req['spec']['containers'][0]['env'].sort(key=lambda x: x['name']) + self.assertEqual(self.input_req, self.expected) + + def test_extract_resources(self): + # Test when resources is not empty + resources = Resources('1Gi', 1, '2Gi', 2) + pod = Pod('v3.14', {}, [], resources=resources) + self.expected['spec']['containers'][0]['resources'] = { + 'requests': { + 'memory': '1Gi', + 'cpu': 1 + }, + 'limits': { + 'memory': '2Gi', + 'cpu': 2 + }, + } + KubernetesRequestFactory.extract_resources(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_init_containers(self): + init_container = 'init_container' + pod = Pod('v3.14', {}, [], init_containers=init_container) + self.expected['spec']['initContainers'] = init_container + KubernetesRequestFactory.extract_init_containers(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_service_account_name(self): + service_account_name = 'service_account_name' + pod = Pod('v3.14', {}, [], service_account_name=service_account_name) + self.expected['spec']['serviceAccountName'] = service_account_name + KubernetesRequestFactory.extract_service_account_name(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_hostnetwork(self): + hostnetwork = True + pod = Pod('v3.14', {}, [], hostnetwork=hostnetwork) + self.expected['spec']['hostNetwork'] = hostnetwork + KubernetesRequestFactory.extract_hostnetwork(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_image_pull_secrets(self): + image_pull_secrets = 'secret_a,secret_b,secret_c' + pod = Pod('v3.14', {}, [], image_pull_secrets=image_pull_secrets) + self.expected['spec']['imagePullSecrets'] = [ + {'name': 'secret_a'}, + {'name': 'secret_b'}, + {'name': 'secret_c'}, + ] + KubernetesRequestFactory.extract_image_pull_secrets(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_extract_tolerations(self): + tolerations = [{ + 'key': 'key', + 'operator': 'Equal', + 'value': 'value', + 'effect': 'NoSchedule' + }] + pod = Pod('v3.14', {}, [], tolerations=tolerations) + self.expected['spec']['tolerations'] = tolerations + KubernetesRequestFactory.extract_tolerations(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + def test_security_context(self): + security_context = { + 'runAsUser': 1000, + 'fsGroup': 2000 + } + pod = Pod('v3.14', {}, [], security_context=security_context) + self.expected['spec']['securityContext'] = security_context + KubernetesRequestFactory.extract_security_context(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) + + @parameterized.expand([ + 'extract_resources', + 'extract_init_containers', + 'extract_service_account_name', + 'extract_hostnetwork', + 'extract_image_pull_secrets', + 'extract_tolerations', + 'extract_security_context', + 'extract_volume_secrets' + ]) + def test_identity(self, name): + kube_request_factory_func = getattr(KubernetesRequestFactory, name) + pod = Pod('v3.14', {}, []) + kube_request_factory_func(pod, self.input_req) + self.assertEqual(self.input_req, self.expected) diff --git a/tests/contrib/kubernetes/kubernetes_request_factory/test_pod_request_factory.py b/tests/contrib/kubernetes/kubernetes_request_factory/test_pod_request_factory.py new file mode 100644 index 0000000000000..ff835ed0c9ec4 --- /dev/null +++ b/tests/contrib/kubernetes/kubernetes_request_factory/test_pod_request_factory.py @@ -0,0 +1,159 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.kubernetes.kubernetes_request_factory.\ + pod_request_factory import SimplePodRequestFactory, \ + ExtractXcomPodRequestFactory +from airflow.contrib.kubernetes.pod import Pod +from airflow.contrib.kubernetes.secret import Secret +from airflow.exceptions import AirflowConfigException +import unittest + +XCOM_CMD = """import time +while True: + try: + time.sleep(3600) + except KeyboardInterrupt: + exit(0) +""" + + +class TestPodRequestFactory(unittest.TestCase): + + def setUp(self): + self.simple_pod_request_factory = SimplePodRequestFactory() + self.xcom_pod_request_factory = ExtractXcomPodRequestFactory() + self.pod = Pod( + image='busybox', + envs={ + 'ENVIRONMENT': 'prod', + 'LOG_LEVEL': 'warning' + }, + name='myapp-pod', + cmds=['sh', '-c', 'echo Hello Kubernetes!'], + labels={'app': 'myapp'}, + image_pull_secrets='pull_secret_a,pull_secret_b', + configmaps=['configmap_a', 'configmap_b'], + secrets=[ + # This should be a secretRef + Secret('env', None, 'secret_a'), + # This should be a single secret mounted in volumeMounts + Secret('volume', '/etc/foo', 'secret_b'), + # This should produce a single secret mounted in env + Secret('env', 'TARGET', 'secret_b', 'source_b'), + ] + ) + self.maxDiff = None + self.expected = { + 'apiVersion': 'v1', + 'kind': 'Pod', + 'metadata': { + 'name': 'myapp-pod', + 'labels': {'app': 'myapp'}, + 'annotations': {}}, + 'spec': { + 'containers': [{ + 'name': 'base', + 'image': 'busybox', + 'command': [ + 'sh', '-c', 'echo Hello Kubernetes!' + ], + 'imagePullPolicy': 'IfNotPresent', + 'args': [], + 'env': [{ + 'name': 'ENVIRONMENT', + 'value': 'prod' + }, { + 'name': 'LOG_LEVEL', + 'value': 'warning' + }, { + 'name': 'TARGET', + 'valueFrom': { + 'secretKeyRef': { + 'name': 'secret_b', + 'key': 'source_b' + } + } + }], + 'envFrom': [{ + 'secretRef': { + 'name': 'secret_a' + } + }, { + 'configMapRef': { + 'name': 'configmap_a' + } + }, { + 'configMapRef': { + 'name': 'configmap_b' + } + }], + 'volumeMounts': [{ + 'mountPath': '/etc/foo', + 'name': 'secretvol0', + 'readOnly': True + }] + }], + 'restartPolicy': 'Never', + 'nodeSelector': {}, + 'volumes': [{ + 'name': 'secretvol0', + 'secret': { + 'secretName': 'secret_b' + } + }], + 'imagePullSecrets': [ + {'name': 'pull_secret_a'}, + {'name': 'pull_secret_b'} + ], + 'affinity': {} + } + } + + def test_secret_throws(self): + with self.assertRaises(AirflowConfigException): + Secret('volume', None, 'secret_a', 'key') + + def test_simple_pod_request_factory_create(self): + result = self.simple_pod_request_factory.create(self.pod) + # sort + result['spec']['containers'][0]['env'].sort(key=lambda x: x['name']) + self.assertEqual(result, self.expected) + + def test_xcom_pod_request_factory_create(self): + result = self.xcom_pod_request_factory.create(self.pod) + container_two = { + 'name': 'airflow-xcom-sidecar', + 'image': 'python:3.5-alpine', + 'command': ['python', '-c', XCOM_CMD], + 'volumeMounts': [ + { + 'name': 'xcom', + 'mountPath': '/airflow/xcom' + } + ] + } + self.expected['spec']['containers'].append(container_two) + self.expected['spec']['containers'][0]['volumeMounts'].insert(0, { + 'name': 'xcom', + 'mountPath': '/airflow/xcom' + }) + self.expected['spec']['volumes'].insert(0, { + 'name': 'xcom', 'emptyDir': {} + }) + result['spec']['containers'][0]['env'].sort(key=lambda x: x['name']) + self.assertEqual(result, self.expected) From 39998eeee040288cfa6c587787836f096e602183 Mon Sep 17 00:00:00 2001 From: Felix Date: Tue, 29 Jan 2019 16:05:24 +0100 Subject: [PATCH 784/808] [AIRFLOW-3552] Add ImapToS3TransferOperator (#4476) NOTE: This operator only transfers the latest attachment by name. --- .../imap_attachment_to_s3_operator.py | 88 +++++++++++++++++++ .../test_imap_attachment_to_s3_operator.py | 57 ++++++++++++ 2 files changed, 145 insertions(+) create mode 100644 airflow/contrib/operators/imap_attachment_to_s3_operator.py create mode 100644 tests/contrib/operators/test_imap_attachment_to_s3_operator.py diff --git a/airflow/contrib/operators/imap_attachment_to_s3_operator.py b/airflow/contrib/operators/imap_attachment_to_s3_operator.py new file mode 100644 index 0000000000000..6126968c9c761 --- /dev/null +++ b/airflow/contrib/operators/imap_attachment_to_s3_operator.py @@ -0,0 +1,88 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow.contrib.hooks.imap_hook import ImapHook +from airflow.hooks.S3_hook import S3Hook +from airflow.models import BaseOperator +from airflow.utils.decorators import apply_defaults + + +class ImapAttachmentToS3Operator(BaseOperator): + """ + Transfers a mail attachment from a mail server into s3 bucket. + + :param imap_attachment_name: The file name of the mail attachment that you want to transfer. + :type imap_attachment_name: str + :param s3_key: The destination file name in the s3 bucket for the attachment. + :type s3_key: str + :param imap_mail_folder: The folder on the mail server to look for the attachment. + :type imap_mail_folder: str + :param imap_check_regex: If set checks the `imap_attachment_name` for a regular expression. + :type imap_check_regex: bool + :param s3_overwrite: If set overwrites the s3 key if already exists. + :type s3_overwrite: bool + :param imap_conn_id: The reference to the connection details of the mail server. + :type imap_conn_id: str + :param s3_conn_id: The reference to the s3 connection details. + :type s3_conn_id: str + """ + template_fields = ('imap_attachment_name', 's3_key') + + @apply_defaults + def __init__(self, + imap_attachment_name, + s3_key, + imap_mail_folder='INBOX', + imap_check_regex=False, + s3_overwrite=False, + imap_conn_id='imap_default', + s3_conn_id='aws_default', + *args, + **kwargs): + super(ImapAttachmentToS3Operator, self).__init__(*args, **kwargs) + self.imap_attachment_name = imap_attachment_name + self.s3_key = s3_key + self.imap_mail_folder = imap_mail_folder + self.imap_check_regex = imap_check_regex + self.s3_overwrite = s3_overwrite + self.imap_conn_id = imap_conn_id + self.s3_conn_id = s3_conn_id + + def execute(self, context): + """ + This function executes the transfer from the email server (via imap) into s3. + + :param context: The context while executing. + :type context: dict + """ + self.log.info( + 'Transferring mail attachment %s from mail server via imap to s3 key %s...', + self.imap_attachment_name, self.s3_key + ) + + with ImapHook(imap_conn_id=self.imap_conn_id) as imap_hook: + imap_mail_attachments = imap_hook.retrieve_mail_attachments( + name=self.imap_attachment_name, + mail_folder=self.imap_mail_folder, + check_regex=self.imap_check_regex, + latest_only=True + ) + + s3_hook = S3Hook(aws_conn_id=self.s3_conn_id) + s3_hook.load_string(string_data=imap_mail_attachments[0][1], key=self.s3_key) diff --git a/tests/contrib/operators/test_imap_attachment_to_s3_operator.py b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py new file mode 100644 index 0000000000000..0cde76cc94f1a --- /dev/null +++ b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py @@ -0,0 +1,57 @@ +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import unittest + +from mock import patch + +from airflow.contrib.operators.imap_attachment_to_s3_operator import ImapAttachmentToS3Operator + + +class TestImapAttachmentToS3Operator(unittest.TestCase): + + def setUp(self): + self.kwargs = dict( + imap_attachment_name='test_file', + s3_key='test_file', + imap_mail_folder='INBOX', + imap_check_regex=False, + s3_overwrite=False, + task_id='test_task', + dag=None + ) + + @patch('airflow.contrib.operators.imap_attachment_to_s3_operator.S3Hook') + @patch('airflow.contrib.operators.imap_attachment_to_s3_operator.ImapHook') + def test_execute(self, mock_imap_hook, mock_s3_hook): + mock_imap_hook.return_value.__enter__ = mock_imap_hook + mock_imap_hook.return_value.retrieve_mail_attachments.return_value = [('test_file', b'Hello World')] + + ImapAttachmentToS3Operator(**self.kwargs).execute(context={}) + + mock_imap_hook.return_value.retrieve_mail_attachments.assert_called_once_with( + name=self.kwargs['imap_attachment_name'], + mail_folder=self.kwargs['imap_mail_folder'], + check_regex=self.kwargs['imap_check_regex'], + latest_only=True + ) + mock_s3_hook.return_value.load_string.assert_called_once_with( + string_data=mock_imap_hook.return_value.retrieve_mail_attachments.return_value[0][1], + key=self.kwargs['s3_key'] + ) From c43a20fe23c4f62bfc84765fa99d95855c36b6e8 Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Wed, 3 Apr 2019 14:38:25 +0100 Subject: [PATCH 785/808] [AIRFLOW-XXX] Pin Sendgrid dep. (#5031) Sendgrid just released 6.0 with breaking changes, and I don't have the time to define how to change our code or tests - as they haven't published a migration guide :( --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index dba753593df6f..7eaca3a9fd913 100644 --- a/setup.py +++ b/setup.py @@ -218,7 +218,7 @@ def write_version(filename=os.path.join(*['airflow', salesforce = ['simple-salesforce>=0.72'] samba = ['pysmbclient>=0.1.3'] segment = ['analytics-python>=1.2.9'] -sendgrid = ['sendgrid>=5.2.0'] +sendgrid = ['sendgrid>=5.2.0,<6'] slack = ['slackclient>=1.0.0'] mongo = ['pymongo>=3.6.0'] snowflake = ['snowflake-connector-python>=1.5.2', From fe9df7301575f54607715dee0b170a1daf5e5dcf Mon Sep 17 00:00:00 2001 From: Andrew Stahlman Date: Thu, 14 Feb 2019 21:33:49 -0800 Subject: [PATCH 786/808] [AIRFLOW-3887] Downgrade dagre-d3 to 0.4.18 (#4713) dagre-d3 v0.6.3 has a bug that causes this Javascript error when loading the Graph View: TypeError: previousPaths.merge is not a function The bug fix [1] has been merged to master, but hasn't been released to npm yet. This change temporarily downgrades our version of dagre-d3 until dagre-d3 v0.6.4 is released [2] I also fixed a bug I encountered in the `compile_assets.sh` where the script would fail if the directory `airflow/www/static/dist/` exists but is empty. [1] https://github.com/dagrejs/dagre-d3/pull/350 [2] https://github.com/dagrejs/dagre-d3/blob/5450627790ff42012ef50cef6b0e220199ae4fbe/package.json#L3 --- airflow/www_rbac/compile_assets.sh | 2 +- airflow/www_rbac/package-lock.json | 93 ++++++++++++++++++++---------- airflow/www_rbac/package.json | 2 +- 3 files changed, 66 insertions(+), 31 deletions(-) diff --git a/airflow/www_rbac/compile_assets.sh b/airflow/www_rbac/compile_assets.sh index a2b61115c8570..a3a4ea2349692 100755 --- a/airflow/www_rbac/compile_assets.sh +++ b/airflow/www_rbac/compile_assets.sh @@ -19,7 +19,7 @@ set -e # first bump up package.json manually, commit and tag if [ -d airflow/www_rbac/static/dist ]; then - rm airflow/www_rbac/static/dist/* + rm -f airflow/www_rbac/static/dist/* fi cd airflow/www_rbac/ diff --git a/airflow/www_rbac/package-lock.json b/airflow/www_rbac/package-lock.json index a26d0b8894e39..c42050e276be5 100644 --- a/airflow/www_rbac/package-lock.json +++ b/airflow/www_rbac/package-lock.json @@ -575,6 +575,7 @@ "resolved": "https://registry.npmjs.org/align-text/-/align-text-0.1.4.tgz", "integrity": "sha1-DNkKVhCT810KmSVsIrcGlDP60Rc=", "dev": true, + "optional": true, "requires": { "kind-of": "^3.0.2", "longest": "^1.0.1", @@ -2154,23 +2155,30 @@ } }, "dagre": { - "version": "0.8.4", - "resolved": "https://registry.npmjs.org/dagre/-/dagre-0.8.4.tgz", - "integrity": "sha512-Dj0csFDrWYKdavwROb9FccHfTC4fJbyF/oJdL9LNZJ8WUvl968P6PAKEriGqfbdArVJEmmfA+UyumgWEwcHU6A==", + "version": "0.7.4", + "resolved": "https://registry.npmjs.org/dagre/-/dagre-0.7.4.tgz", + "integrity": "sha1-3nLw50pVDOEc5jjwoTb+1xI5gCI=", "requires": { - "graphlib": "^2.1.7", - "lodash": "^4.17.4" + "graphlib": "^1.0.5", + "lodash": "^3.10.0" + }, + "dependencies": { + "lodash": { + "version": "3.10.1", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-3.10.1.tgz", + "integrity": "sha1-W/Rejkm6QYnhfUgnid/RW9FAt7Y=" + } } }, "dagre-d3": { - "version": "0.6.3", - "resolved": "https://registry.npmjs.org/dagre-d3/-/dagre-d3-0.6.3.tgz", - "integrity": "sha512-1vAzNp7OR1370JtjNaVFiW04DBjPDq513cJnqNVWxIkZqB0HfIArsc5eriTY9RM9cVMUjxdCJ3z4of5f8HqbdA==", - "requires": { - "d3": "^4.12.2", - "dagre": "^0.8.4", - "graphlib": "^2.1.7", - "lodash": "^4.17.10" + "version": "0.4.18", + "resolved": "https://registry.npmjs.org/dagre-d3/-/dagre-d3-0.4.18.tgz", + "integrity": "sha512-7tRltaOfNTIkNEZYMCL8N3Q8bCre99x/mAJL2RbuUfPu5d+4f0KOHglZm+AzOG2Z/+S2HBDYciE6iDcDtki6Tg==", + "requires": { + "d3": "^3.3.8", + "dagre": "^0.7.3", + "graphlib": "^1.0.5", + "lodash": "^3.10.0" }, "dependencies": { "lodash": { @@ -3451,7 +3459,8 @@ "ansi-regex": { "version": "2.1.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "aproba": { "version": "1.2.0", @@ -3472,12 +3481,14 @@ "balanced-match": { "version": "1.0.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "brace-expansion": { "version": "1.1.11", "bundled": true, "dev": true, + "optional": true, "requires": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -3492,17 +3503,20 @@ "code-point-at": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "concat-map": { "version": "0.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "console-control-strings": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "core-util-is": { "version": "1.0.2", @@ -3619,7 +3633,8 @@ "inherits": { "version": "2.0.3", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "ini": { "version": "1.3.5", @@ -3631,6 +3646,7 @@ "version": "1.0.0", "bundled": true, "dev": true, + "optional": true, "requires": { "number-is-nan": "^1.0.0" } @@ -3645,6 +3661,7 @@ "version": "3.0.4", "bundled": true, "dev": true, + "optional": true, "requires": { "brace-expansion": "^1.1.7" } @@ -3652,12 +3669,14 @@ "minimist": { "version": "0.0.8", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "minipass": { "version": "2.2.4", "bundled": true, "dev": true, + "optional": true, "requires": { "safe-buffer": "^5.1.1", "yallist": "^3.0.0" @@ -3676,6 +3695,7 @@ "version": "0.5.1", "bundled": true, "dev": true, + "optional": true, "requires": { "minimist": "0.0.8" } @@ -3756,7 +3776,8 @@ "number-is-nan": { "version": "1.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "object-assign": { "version": "4.1.1", @@ -3768,6 +3789,7 @@ "version": "1.4.0", "bundled": true, "dev": true, + "optional": true, "requires": { "wrappy": "1" } @@ -3853,7 +3875,8 @@ "safe-buffer": { "version": "5.1.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "safer-buffer": { "version": "2.1.2", @@ -3889,6 +3912,7 @@ "version": "1.0.2", "bundled": true, "dev": true, + "optional": true, "requires": { "code-point-at": "^1.0.0", "is-fullwidth-code-point": "^1.0.0", @@ -3908,6 +3932,7 @@ "version": "3.0.1", "bundled": true, "dev": true, + "optional": true, "requires": { "ansi-regex": "^2.0.0" } @@ -3951,12 +3976,14 @@ "wrappy": { "version": "1.0.2", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "yallist": { "version": "3.0.2", "bundled": true, - "dev": true + "dev": true, + "optional": true } } }, @@ -4102,11 +4129,18 @@ "dev": true }, "graphlib": { - "version": "2.1.7", - "resolved": "https://registry.npmjs.org/graphlib/-/graphlib-2.1.7.tgz", - "integrity": "sha512-TyI9jIy2J4j0qgPmOOrHTCtpPqJGN/aurBwc6ZT+bRii+di1I+Wv3obRhVrmBEXet+qkMaEX67dXrwsd3QQM6w==", + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/graphlib/-/graphlib-1.0.7.tgz", + "integrity": "sha1-DKst8P/mq+BwsmJb+h7bbslnuLE=", "requires": { - "lodash": "^4.17.5" + "lodash": "^3.10.0" + }, + "dependencies": { + "lodash": { + "version": "3.10.1", + "resolved": "https://registry.npmjs.org/lodash/-/lodash-3.10.1.tgz", + "integrity": "sha1-W/Rejkm6QYnhfUgnid/RW9FAt7Y=" + } } }, "handlebars": { @@ -4960,7 +4994,8 @@ "version": "1.0.1", "resolved": "https://registry.npmjs.org/longest/-/longest-1.0.1.tgz", "integrity": "sha1-MKCy2jj3N3DoKUoNIuZiXtd9AJc=", - "dev": true + "dev": true, + "optional": true }, "loose-envify": { "version": "1.3.1", diff --git a/airflow/www_rbac/package.json b/airflow/www_rbac/package.json index f9a99fdc171d9..5c431895a7666 100644 --- a/airflow/www_rbac/package.json +++ b/airflow/www_rbac/package.json @@ -58,7 +58,7 @@ "bootstrap-toggle": "^2.2.2", "d3": "^3.4.4", "d3-tip": "^0.9.1", - "dagre-d3": "^0.6.3", + "dagre-d3": "^0.4.18", "datatables.net": "^1.10.19", "datatables.net-bs": "^1.10.19", "lodash": "^4.17.11", From 0c46bcabf9f9d845723ee87c917964b695c9a8ea Mon Sep 17 00:00:00 2001 From: Xiaodong Date: Thu, 4 Apr 2019 22:44:05 +0800 Subject: [PATCH 787/808] [AIRFLOW-XXX] Pin psycopg2 due to breaking change (#5036) --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 7eaca3a9fd913..db6a8bb8b687b 100644 --- a/setup.py +++ b/setup.py @@ -210,7 +210,7 @@ def write_version(filename=os.path.join(*['airflow', 'flask-bcrypt>=0.7.1', ] pinot = ['pinotdb==0.1.1'] -postgres = ['psycopg2>=2.7.4'] +postgres = ['psycopg2>=2.7.4,<2.8'] qds = ['qds-sdk>=1.9.6'] rabbitmq = ['librabbitmq>=1.6.1'] redis = ['redis~=3.2'] From eaaa5aba1c2459bf6e6ce15db85a97c70ba2a33c Mon Sep 17 00:00:00 2001 From: Felix Uellendall Date: Thu, 4 Apr 2019 23:40:46 +0200 Subject: [PATCH 788/808] [AIRFLOW-3552] Fix encoding issue in ImapAttachmentToS3Operator (#5040) - change method to upload data to s3 from load_string to load_bytes --- airflow/contrib/operators/imap_attachment_to_s3_operator.py | 2 +- .../contrib/operators/test_imap_attachment_to_s3_operator.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/contrib/operators/imap_attachment_to_s3_operator.py b/airflow/contrib/operators/imap_attachment_to_s3_operator.py index 6126968c9c761..3b9f143591efb 100644 --- a/airflow/contrib/operators/imap_attachment_to_s3_operator.py +++ b/airflow/contrib/operators/imap_attachment_to_s3_operator.py @@ -85,4 +85,4 @@ def execute(self, context): ) s3_hook = S3Hook(aws_conn_id=self.s3_conn_id) - s3_hook.load_string(string_data=imap_mail_attachments[0][1], key=self.s3_key) + s3_hook.load_bytes(bytes_data=imap_mail_attachments[0][1], key=self.s3_key) diff --git a/tests/contrib/operators/test_imap_attachment_to_s3_operator.py b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py index 0cde76cc94f1a..66287fc7f4de3 100644 --- a/tests/contrib/operators/test_imap_attachment_to_s3_operator.py +++ b/tests/contrib/operators/test_imap_attachment_to_s3_operator.py @@ -51,7 +51,7 @@ def test_execute(self, mock_imap_hook, mock_s3_hook): check_regex=self.kwargs['imap_check_regex'], latest_only=True ) - mock_s3_hook.return_value.load_string.assert_called_once_with( - string_data=mock_imap_hook.return_value.retrieve_mail_attachments.return_value[0][1], + mock_s3_hook.return_value.load_bytes.assert_called_once_with( + bytes_data=mock_imap_hook.return_value.retrieve_mail_attachments.return_value[0][1], key=self.kwargs['s3_key'] ) From 9446fde04ef752d98de9ce92b738301ce8dcc423 Mon Sep 17 00:00:00 2001 From: BasPH Date: Sun, 10 Feb 2019 19:59:46 +0100 Subject: [PATCH 789/808] AIRFLOW-[3823] Exclude branch's downstream tasks from the tasks to skip (#4666) --- airflow/operators/python_operator.py | 17 ++++++-- tests/operators/test_python_operator.py | 58 +++++++++++++++++++++++++ 2 files changed, 72 insertions(+), 3 deletions(-) diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py index 30f5290973b74..67cf5492656e2 100644 --- a/airflow/operators/python_operator.py +++ b/airflow/operators/python_operator.py @@ -23,10 +23,11 @@ import subprocess import sys import types +from builtins import str from textwrap import dedent import dill -from builtins import str +import six from airflow.exceptions import AirflowException from airflow.models import BaseOperator @@ -137,7 +138,7 @@ class BranchPythonOperator(PythonOperator, SkipMixin): """ def execute(self, context): branch = super(BranchPythonOperator, self).execute(context) - if isinstance(branch, str): + if isinstance(branch, six.string_types): branch = [branch] self.log.info("Following branch %s", branch) self.log.info("Marking other directly downstream tasks as skipped") @@ -145,8 +146,18 @@ def execute(self, context): downstream_tasks = context['task'].downstream_list self.log.debug("Downstream task_ids %s", downstream_tasks) - skip_tasks = [t for t in downstream_tasks if t.task_id not in branch] if downstream_tasks: + # Also check downstream tasks of the branch task. In case the task to skip + # is a downstream task of the branch task, we exclude it from skipping. + branch_downstream_task_ids = set() + for b in branch: + branch_downstream_task_ids.update(context["dag"]. + get_task(b). + get_flat_relative_ids(upstream=False)) + skip_tasks = [t + for t in downstream_tasks + if t.task_id not in branch and + t.task_id not in branch_downstream_task_ids] self.skip(context['dag_run'], context['ti'].execution_date, skip_tasks) self.log.info("Done.") diff --git a/tests/operators/test_python_operator.py b/tests/operators/test_python_operator.py index 557f021a7f49d..296ef841c5deb 100644 --- a/tests/operators/test_python_operator.py +++ b/tests/operators/test_python_operator.py @@ -378,6 +378,64 @@ def test_with_dag_run(self): else: raise + def test_with_skip_in_branch_downstream_dependencies(self): + self.branch_op = BranchPythonOperator(task_id='make_choice', + dag=self.dag, + python_callable=lambda: 'branch_1') + + self.branch_op >> self.branch_1 >> self.branch_2 + self.branch_op >> self.branch_2 + self.dag.clear() + + dr = self.dag.create_dagrun( + run_id="manual__", + start_date=timezone.utcnow(), + execution_date=DEFAULT_DATE, + state=State.RUNNING + ) + + self.branch_op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + + tis = dr.get_task_instances() + for ti in tis: + if ti.task_id == 'make_choice': + self.assertEqual(ti.state, State.SUCCESS) + elif ti.task_id == 'branch_1': + self.assertEqual(ti.state, State.NONE) + elif ti.task_id == 'branch_2': + self.assertEqual(ti.state, State.NONE) + else: + raise Exception + + def test_with_skip_in_branch_downstream_dependencies2(self): + self.branch_op = BranchPythonOperator(task_id='make_choice', + dag=self.dag, + python_callable=lambda: 'branch_2') + + self.branch_op >> self.branch_1 >> self.branch_2 + self.branch_op >> self.branch_2 + self.dag.clear() + + dr = self.dag.create_dagrun( + run_id="manual__", + start_date=timezone.utcnow(), + execution_date=DEFAULT_DATE, + state=State.RUNNING + ) + + self.branch_op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) + + tis = dr.get_task_instances() + for ti in tis: + if ti.task_id == 'make_choice': + self.assertEqual(ti.state, State.SUCCESS) + elif ti.task_id == 'branch_1': + self.assertEqual(ti.state, State.SKIPPED) + elif ti.task_id == 'branch_2': + self.assertEqual(ti.state, State.NONE) + else: + raise Exception + class ShortCircuitOperatorTest(unittest.TestCase): @classmethod From a76f1467cd8aab1f6a738955c91626931dac5809 Mon Sep 17 00:00:00 2001 From: gseva Date: Thu, 4 Apr 2019 18:52:45 -0300 Subject: [PATCH 790/808] [AIRFLOW-XXX] Mention that statsd must be installed to gather metrics (#5038) --- docs/metrics.rst | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/metrics.rst b/docs/metrics.rst index fb34b97e6498d..fc0c82e4d59ca 100644 --- a/docs/metrics.rst +++ b/docs/metrics.rst @@ -18,10 +18,17 @@ Metrics ======= -Airflow can be set up to send metrics to `StatsD `__: +Airflow can be set up to send metrics to `StatsD `__. + +Setup +----- + +First you must install statsd requirement: + +.. code-block:: bash + + pip install 'apache-airflow[statsd]' -Configuration -------------- Add the following lines to your configuration file e.g. ``airflow.cfg`` .. code-block:: bash From b24489f75c6eb86ea6186a3483ee09f2fb3926a1 Mon Sep 17 00:00:00 2001 From: Paul Bramhall Date: Fri, 5 Apr 2019 17:11:50 +0100 Subject: [PATCH 791/808] [AIRFLOW-4246] Flask-Oauthlib needs downstream dependencies pinning due to breaking changes (#5045) --- setup.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index db6a8bb8b687b..6fed3dd7179ac 100644 --- a/setup.py +++ b/setup.py @@ -185,8 +185,11 @@ def write_version(filename=os.path.join(*['airflow', 'PyOpenSSL', 'pandas-gbq' ] -github_enterprise = ['Flask-OAuthlib>=0.9.1'] -google_auth = ['Flask-OAuthlib>=0.9.1'] +flask_oauth = [ + 'Flask-OAuthlib>=0.9.1', + 'oauthlib!=2.0.3,!=2.0.4,!=2.0.5,<3.0.0,>=1.1.2', + 'requests-oauthlib==1.1.0' +] hdfs = ['snakebite>=2.7.8'] hive = [ 'hmsclient>=0.1.0', @@ -355,8 +358,8 @@ def do_setup(): 'elasticsearch': elasticsearch, 'emr': emr, 'gcp_api': gcp_api, - 'github_enterprise': github_enterprise, - 'google_auth': google_auth, + 'github_enterprise': flask_oauth, + 'google_auth': flask_oauth, 'hdfs': hdfs, 'hive': hive, 'jdbc': jdbc, From d4d47f0615b8732998f9d95195b396ef5088cdf8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 5 Apr 2019 18:20:19 +0200 Subject: [PATCH 792/808] [AIRFLOW-4247] Template Region on the DataprocOperators (#5046) --- airflow/contrib/operators/dataproc_operator.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py index e9684b987e5c4..f525db96836e6 100644 --- a/airflow/contrib/operators/dataproc_operator.py +++ b/airflow/contrib/operators/dataproc_operator.py @@ -731,7 +731,7 @@ class DataProcPigOperator(BaseOperator): an 8 character random string. :vartype dataproc_job_id: str """ - template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] template_ext = ('.pg', '.pig',) ui_color = '#0273d4' @@ -828,7 +828,7 @@ class DataProcHiveOperator(BaseOperator): an 8 character random string. :vartype dataproc_job_id: str """ - template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] template_ext = ('.q',) ui_color = '#0273d4' @@ -927,7 +927,7 @@ class DataProcSparkSqlOperator(BaseOperator): an 8 character random string. :vartype dataproc_job_id: str """ - template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['query', 'variables', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] template_ext = ('.q',) ui_color = '#0273d4' @@ -1034,7 +1034,7 @@ class DataProcSparkOperator(BaseOperator): :vartype dataproc_job_id: str """ - template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['arguments', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] ui_color = '#0273d4' @apply_defaults @@ -1142,7 +1142,7 @@ class DataProcHadoopOperator(BaseOperator): :vartype dataproc_job_id: str """ - template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['arguments', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] ui_color = '#0273d4' @apply_defaults @@ -1250,7 +1250,7 @@ class DataProcPySparkOperator(BaseOperator): :vartype dataproc_job_id: str """ - template_fields = ['arguments', 'job_name', 'cluster_name', 'dataproc_jars'] + template_fields = ['arguments', 'job_name', 'cluster_name', 'region', 'dataproc_jars'] ui_color = '#0273d4' @staticmethod From eb4d3ecc23c520ebb0ed32532b1669081c362e9b Mon Sep 17 00:00:00 2001 From: Ash Berlin-Taylor Date: Fri, 5 Apr 2019 19:05:10 +0100 Subject: [PATCH 793/808] [AIRFLOW-4240] State-changing actions should be POST requests (#5039) To make the requests POSTs and to follow the redirect that the backend issue I turned the "toggle" buttons in to an actual form, which makes there much less logic needed to build up the URL - the browser handles it all for us. The only thing we have to do is set the "action" on the URL. For the "link" ones (delete,trigger,refresh) I wrote a short `postAsForm` which takes the URL and submits a form. A little bit messy, but it works. --- airflow/www/static/main.css | 6 +- airflow/www/templates/admin/master.html | 33 +- airflow/www/templates/airflow/confirm.html | 13 +- airflow/www/templates/airflow/dag.html | 348 ++++++++---------- airflow/www/templates/airflow/dags.html | 25 +- airflow/www/views.py | 135 +++---- airflow/www_rbac/static/css/main.css | 6 +- airflow/www_rbac/static/js/base.js | 32 ++ .../www_rbac/templates/airflow/confirm.html | 13 +- airflow/www_rbac/templates/airflow/dag.html | 347 ++++++++--------- airflow/www_rbac/templates/airflow/dags.html | 27 +- airflow/www_rbac/views.py | 126 +++---- tests/core.py | 94 +++-- tests/www/test_views.py | 8 +- tests/www_rbac/test_views.py | 118 ++++-- 15 files changed, 703 insertions(+), 628 deletions(-) diff --git a/airflow/www/static/main.css b/airflow/www/static/main.css index 66f449aceee29..08bfc798a01ed 100644 --- a/airflow/www/static/main.css +++ b/airflow/www/static/main.css @@ -44,10 +44,14 @@ td>span.glyphicon{ padding-left: 3px; padding-top: 3px; } -button.btn { +button.btn, label.btn { border: 1px solid black; } +.btn-group label.btn { + background-color: #f0f0f0; +} + div.rich_doc { padding: 5px 10px; border: 1px solid #dddddd; diff --git a/airflow/www/templates/admin/master.html b/airflow/www/templates/admin/master.html index ec4573b5d4adb..7837c93f73e07 100644 --- a/airflow/www/templates/admin/master.html +++ b/airflow/www/templates/admin/master.html @@ -37,11 +37,12 @@ alert('{{ hostname }}'); }); $('span').tooltip(); + var CSRF = {{ csrf_token() | tojson }}; $.ajaxSetup({ beforeSend: function(xhr, settings) { if (!/^(GET|HEAD|OPTIONS|TRACE)$/i.test(settings.type) && !this.crossDomain) { - xhr.setRequestHeader("X-CSRFToken", "{{ csrf_token() }}"); + xhr.setRequestHeader("X-CSRFToken", CSRF); } } }); @@ -52,6 +53,36 @@ el.textContent = text; return el.innerHTML; } + +function postAsForm(url, parameters) { + var form = $(""); + + form.attr("method", "POST"); + form.attr("action", url); + + $.each(parameters || {}, function(key, value) { + var field = $(''); + + field.attr("type", "hidden"); + field.attr("name", key); + field.attr("value", value); + + form.append(field); + }); + + var field = $(''); + + field.attr("type", "hidden"); + field.attr("name", "csrf_token"); + field.attr("value", CSRF); + + form.append(field); + + // The form needs to be a part of the document in order for us to be able + // to submit it. + $(document.body).append(form); + form.submit(); +} {% endblock %} diff --git a/airflow/www/templates/airflow/confirm.html b/airflow/www/templates/airflow/confirm.html index 8084250c95bb7..c50a6e0bf0794 100644 --- a/airflow/www/templates/airflow/confirm.html +++ b/airflow/www/templates/airflow/confirm.html @@ -26,10 +26,15 @@

    Wait a minute.

    {{ details }}
    {% endif %} - - +
    + + + {% for name,val in request.form.items() if name != "csrf_token" %} + + {% endfor %} + + +


    {% endblock %} diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index 48178c85755f3..5daf7820c0a9c 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -96,20 +96,20 @@

  • + onclick="return confirmTriggerDag(this, '{{ dag.safe_dag_id }}')"> Trigger DAG
  • - + Refresh
  • + onclick="return confirmDeleteDag(this, '{{ dag.safe_dag_id }}')"> Delete @@ -159,92 +159,113 @@

  • + onclick="return confirmTriggerDag(this, '{{ dag.safe_dag_id }}')"> Trigger DAG
  • - + Refresh
  • + onclick="return confirmDeleteDag(this, '{{ dag.safe_dag_id }}')"> Delete @@ -159,92 +159,116 @@