From 4e52c92594b4725c31f851557a4896bcdb1d1280 Mon Sep 17 00:00:00 2001 From: David Liu <48995019+dliu27@users.noreply.github.com> Date: Wed, 18 Dec 2024 15:33:48 -0500 Subject: [PATCH 01/37] [ui] sort global search results (#26573) ## Summary & Motivation Linear: https://linear.app/dagster-labs/issue/OPER-1521/prioritize-exact-matches-in-global-search-results-for-better-usability Sorts by fuse score on the frontend, so more relevant results will be pushed on top. ## How I Tested These Changes Tested the original case on enigma using app-proxy, verified the best matches show up first --- .../packages/ui-core/src/search/SearchDialog.tsx | 16 +++++++++++++--- .../ui-core/src/search/useGlobalSearch.tsx | 1 + 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/search/SearchDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/search/SearchDialog.tsx index e9855337ff37c..e8605d16b4161 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/search/SearchDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/search/SearchDialog.tsx @@ -72,6 +72,14 @@ const initialState: State = { const DEBOUNCE_MSEC = 100; +// sort by Fuse score ascending, lower is better +const sortResultsByFuseScore = ( + a: Fuse.FuseResult, + b: Fuse.FuseResult, +) => { + return (a.score ?? 0) - (b.score ?? 0); +}; + export const SearchDialog = () => { const history = useHistory(); const {initialize, loading, searchPrimary, searchSecondary} = useGlobalSearch({ @@ -82,9 +90,11 @@ export const SearchDialog = () => { const [state, dispatch] = React.useReducer(reducer, initialState); const {shown, queryString, primaryResults, secondaryResults, highlight} = state; - const results = [...primaryResults, ...secondaryResults]; - const renderedResults = results.slice(0, MAX_DISPLAYED_RESULTS); - const numRenderedResults = renderedResults.length; + const {renderedResults, numRenderedResults} = React.useMemo(() => { + const results = [...primaryResults, ...secondaryResults].sort(sortResultsByFuseScore); + const renderedResults = results.slice(0, MAX_DISPLAYED_RESULTS); + return {renderedResults, numRenderedResults: renderedResults.length}; + }, [primaryResults, secondaryResults]); const openSearch = React.useCallback(() => { trackEvent('open-global-search'); diff --git a/js_modules/dagster-ui/packages/ui-core/src/search/useGlobalSearch.tsx b/js_modules/dagster-ui/packages/ui-core/src/search/useGlobalSearch.tsx index c17bb0674470e..013e1435c8093 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/search/useGlobalSearch.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/search/useGlobalSearch.tsx @@ -307,6 +307,7 @@ const fuseOptions = { threshold: 0.3, useExtendedSearch: true, includeMatches: true, + includeScore: true, // Allow searching to continue to the end of the string. ignoreLocation: true, From 29a91fc41c1c8345dc995956e72ae1f0f5758bfe Mon Sep 17 00:00:00 2001 From: Deepyaman Datta Date: Wed, 18 Dec 2024 14:15:26 -0700 Subject: [PATCH 02/37] Add a thread pool to process backfills in parallel (#26189) ## Summary & Motivation Some users are running into situations where they kick off a backfill that takes hours to process and it holds up every other backfill until it finishes. Adding a thread pool can alleviate this in most situations, where there is one of a handful of long-running backfills; in the unlikely case where all of the backfills being executed in parallel take hours, this still won't help. ## How I Tested These Changes Existing tests, plus changing e2e tests to use thread pool for backfills. ## Changelog Added the option to use a thread pool to process backfills in parallel. --- .../dagster/_core/instance/__init__.py | 3 + .../dagster/dagster/_core/instance/config.py | 15 + .../dagster/dagster/_core/instance/ref.py | 1 + .../dagster/dagster/_daemon/asset_daemon.py | 22 +- .../dagster/dagster/_daemon/backfill.py | 118 +++++++- .../dagster/dagster/_daemon/controller.py | 2 +- .../dagster/dagster/_daemon/daemon.py | 31 +- .../dagster/dagster/_daemon/sensor.py | 37 +-- .../dagster/dagster/_scheduler/scheduler.py | 5 +- .../dagster/dagster/_utils/__init__.py | 10 + .../daemon_sensor_tests/test_sensor_run.py | 6 - .../daemon_tests/test_backfill.py | 268 +++++++++++++++++- .../daemon_tests/test_dagster_daemon.py | 7 + .../daemon_tests/test_e2e.py | 5 + .../dagster_tests/scheduler_tests/conftest.py | 15 +- 15 files changed, 457 insertions(+), 88 deletions(-) diff --git a/python_modules/dagster/dagster/_core/instance/__init__.py b/python_modules/dagster/dagster/_core/instance/__init__.py index 25d251bc30679..f781a49e602e2 100644 --- a/python_modules/dagster/dagster/_core/instance/__init__.py +++ b/python_modules/dagster/dagster/_core/instance/__init__.py @@ -831,6 +831,9 @@ def get_settings(self, settings_key: str) -> Any: return self._settings.get(settings_key) return {} + def get_backfill_settings(self) -> Mapping[str, Any]: + return self.get_settings("backfills") + def get_scheduler_settings(self) -> Mapping[str, Any]: return self.get_settings("schedules") diff --git a/python_modules/dagster/dagster/_core/instance/config.py b/python_modules/dagster/dagster/_core/instance/config.py index 24dcd46c01ad1..2da173d909254 100644 --- a/python_modules/dagster/dagster/_core/instance/config.py +++ b/python_modules/dagster/dagster/_core/instance/config.py @@ -267,6 +267,20 @@ def get_tick_retention_settings( return default_retention_settings +def backfills_daemon_config() -> Field: + return Field( + { + "use_threads": Field(Bool, is_required=False, default_value=False), + "num_workers": Field( + int, + is_required=False, + description="How many threads to use to process multiple backfills in parallel", + ), + }, + is_required=False, + ) + + def sensors_daemon_config() -> Field: return Field( { @@ -389,6 +403,7 @@ def dagster_instance_config_schema() -> Mapping[str, Field]: ), "secrets": secrets_loader_config_schema(), "retention": retention_config_schema(), + "backfills": backfills_daemon_config(), "sensors": sensors_daemon_config(), "schedules": schedules_daemon_config(), "auto_materialize": Field( diff --git a/python_modules/dagster/dagster/_core/instance/ref.py b/python_modules/dagster/dagster/_core/instance/ref.py index 6a160b8d217a4..5c29d1d85b5de 100644 --- a/python_modules/dagster/dagster/_core/instance/ref.py +++ b/python_modules/dagster/dagster/_core/instance/ref.py @@ -451,6 +451,7 @@ def from_dir( "run_retries", "code_servers", "retention", + "backfills", "sensors", "schedules", "nux", diff --git a/python_modules/dagster/dagster/_daemon/asset_daemon.py b/python_modules/dagster/dagster/_daemon/asset_daemon.py index 85bed317ee1bd..e4ac46140256c 100644 --- a/python_modules/dagster/dagster/_daemon/asset_daemon.py +++ b/python_modules/dagster/dagster/_daemon/asset_daemon.py @@ -72,7 +72,7 @@ from dagster._serdes import serialize_value from dagster._serdes.serdes import deserialize_value from dagster._time import get_current_datetime, get_current_timestamp -from dagster._utils import SingleInstigatorDebugCrashFlags, check_for_debug_crash +from dagster._utils import SingleInstigatorDebugCrashFlags, check_for_debug_crash, return_as_list _LEGACY_PRE_SENSOR_AUTO_MATERIALIZE_CURSOR_KEY = "ASSET_DAEMON_CURSOR" _PRE_SENSOR_AUTO_MATERIALIZE_CURSOR_KEY = "ASSET_DAEMON_CURSOR_NEW" @@ -655,24 +655,6 @@ def _copy_default_auto_materialize_sensor_states( return result - def _process_auto_materialize_tick( - self, - workspace_process_context: IWorkspaceProcessContext, - repository: Optional[RemoteRepository], - sensor: Optional[RemoteSensor], - debug_crash_flags: SingleInstigatorDebugCrashFlags, - submit_threadpool_executor: Optional[ThreadPoolExecutor], - ): - return list( - self._process_auto_materialize_tick_generator( - workspace_process_context, - repository, - sensor, - debug_crash_flags, - submit_threadpool_executor=submit_threadpool_executor, - ) - ) - def _process_auto_materialize_tick_generator( self, workspace_process_context: IWorkspaceProcessContext, @@ -878,6 +860,8 @@ def _process_auto_materialize_tick_generator( yield error_info + _process_auto_materialize_tick = return_as_list(_process_auto_materialize_tick_generator) + def _evaluate_auto_materialize_tick( self, tick_context: AutoMaterializeLaunchContext, diff --git a/python_modules/dagster/dagster/_daemon/backfill.py b/python_modules/dagster/dagster/_daemon/backfill.py index 1fcc0c24c4199..23b5b8ccfdc68 100644 --- a/python_modules/dagster/dagster/_daemon/backfill.py +++ b/python_modules/dagster/dagster/_daemon/backfill.py @@ -1,8 +1,10 @@ import logging import os import sys +import threading +from concurrent.futures import Future, ThreadPoolExecutor from contextlib import contextmanager -from typing import Iterable, Mapping, Optional, Sequence, cast +from typing import TYPE_CHECKING, Dict, Iterable, Mapping, Optional, Sequence, cast import dagster._check as check from dagster._core.definitions.instigation_logger import InstigationLogger @@ -16,9 +18,13 @@ from dagster._core.execution.job_backfill import execute_job_backfill_iteration from dagster._core.workspace.context import IWorkspaceProcessContext from dagster._daemon.utils import DaemonErrorCapture -from dagster._time import get_current_datetime +from dagster._time import get_current_datetime, get_current_timestamp +from dagster._utils import return_as_list from dagster._utils.error import SerializableErrorInfo +if TYPE_CHECKING: + from dagster._daemon.daemon import DaemonIterator + @contextmanager def _get_instigation_logger_if_log_storage_enabled( @@ -46,9 +52,55 @@ def _get_max_asset_backfill_retries(): return int(os.getenv("DAGSTER_MAX_ASSET_BACKFILL_RETRIES", "5")) +def execute_backfill_iteration_loop( + workspace_process_context: IWorkspaceProcessContext, + logger: logging.Logger, + shutdown_event: threading.Event, + until: Optional[float] = None, + threadpool_executor: Optional[ThreadPoolExecutor] = None, +) -> "DaemonIterator": + from dagster._daemon.controller import DEFAULT_DAEMON_INTERVAL_SECONDS + from dagster._daemon.daemon import SpanMarker + + backfill_futures: Dict[str, Future] = {} + while True: + start_time = get_current_timestamp() + if until and start_time >= until: + # provide a way of organically ending the loop to support test environment + break + + yield SpanMarker.START_SPAN + + try: + yield from execute_backfill_iteration( + workspace_process_context, + logger, + threadpool_executor=threadpool_executor, + backfill_futures=backfill_futures, + ) + except Exception: + error_info = DaemonErrorCapture.on_exception( + exc_info=sys.exc_info(), + logger=logger, + log_message="BackfillDaemon caught an error", + ) + yield error_info + + yield SpanMarker.END_SPAN + + end_time = get_current_timestamp() + loop_duration = end_time - start_time + sleep_time = max(0, DEFAULT_DAEMON_INTERVAL_SECONDS - loop_duration) + shutdown_event.wait(sleep_time) + + yield None + + def execute_backfill_iteration( workspace_process_context: IWorkspaceProcessContext, logger: logging.Logger, + threadpool_executor: Optional[ThreadPoolExecutor] = None, + backfill_futures: Optional[Dict[str, Future]] = None, debug_crash_flags: Optional[Mapping[str, int]] = None, ) -> Iterable[Optional[SerializableErrorInfo]]: instance = workspace_process_context.instance @@ -68,7 +120,12 @@ def execute_backfill_iteration( backfill_jobs = [*in_progress_backfills, *canceling_backfills] yield from execute_backfill_jobs( - workspace_process_context, logger, backfill_jobs, debug_crash_flags + workspace_process_context, + logger, + backfill_jobs, + threadpool_executor, + backfill_futures, + debug_crash_flags, ) @@ -86,6 +143,8 @@ def execute_backfill_jobs( workspace_process_context: IWorkspaceProcessContext, logger: logging.Logger, backfill_jobs: Sequence[PartitionBackfill], + threadpool_executor: Optional[ThreadPoolExecutor] = None, + backfill_futures: Optional[Dict[str, Future]] = None, debug_crash_flags: Optional[Mapping[str, int]] = None, ) -> Iterable[Optional[SerializableErrorInfo]]: instance = workspace_process_context.instance @@ -103,18 +162,49 @@ def execute_backfill_jobs( ) try: - if backfill.is_asset_backfill: - yield from execute_asset_backfill_iteration( - backfill, backfill_logger, workspace_process_context, instance - ) + if threadpool_executor: + if backfill_futures is None: + check.failed( + "backfill_futures dict must be passed with threadpool_executor" + ) + + # only allow one backfill per backfill job to be in flight + if backfill_id in backfill_futures and not backfill_futures[backfill_id].done(): + continue + + if backfill.is_asset_backfill: + future = threadpool_executor.submit( + return_as_list(execute_asset_backfill_iteration), + backfill, + backfill_logger, + workspace_process_context, + instance, + ) + else: + future = threadpool_executor.submit( + return_as_list(execute_job_backfill_iteration), + backfill, + backfill_logger, + workspace_process_context, + debug_crash_flags, + instance, + ) + backfill_futures[backfill_id] = future + yield + else: - yield from execute_job_backfill_iteration( - backfill, - backfill_logger, - workspace_process_context, - debug_crash_flags, - instance, - ) + if backfill.is_asset_backfill: + yield from execute_asset_backfill_iteration( + backfill, backfill_logger, workspace_process_context, instance + ) + else: + yield from execute_job_backfill_iteration( + backfill, + backfill_logger, + workspace_process_context, + debug_crash_flags, + instance, + ) except Exception as e: backfill = check.not_none(instance.get_backfill(backfill.backfill_id)) if ( diff --git a/python_modules/dagster/dagster/_daemon/controller.py b/python_modules/dagster/dagster/_daemon/controller.py index e0016460d5921..0371c07128180 100644 --- a/python_modules/dagster/dagster/_daemon/controller.py +++ b/python_modules/dagster/dagster/_daemon/controller.py @@ -362,7 +362,7 @@ def create_daemon_of_type(daemon_type: str, instance: DagsterInstance) -> Dagste interval_seconds=instance.run_coordinator.dequeue_interval_seconds # type: ignore # (??) ) elif daemon_type == BackfillDaemon.daemon_type(): - return BackfillDaemon(interval_seconds=DEFAULT_DAEMON_INTERVAL_SECONDS) + return BackfillDaemon(settings=instance.get_backfill_settings()) elif daemon_type == MonitoringDaemon.daemon_type(): return MonitoringDaemon(interval_seconds=instance.run_monitoring_poll_interval_seconds) elif daemon_type == EventLogConsumerDaemon.daemon_type(): diff --git a/python_modules/dagster/dagster/_daemon/daemon.py b/python_modules/dagster/dagster/_daemon/daemon.py index 6a461618c0a69..9ae368c903cf2 100644 --- a/python_modules/dagster/dagster/_daemon/daemon.py +++ b/python_modules/dagster/dagster/_daemon/daemon.py @@ -22,7 +22,7 @@ from dagster._core.telemetry import DAEMON_ALIVE, log_action from dagster._core.utils import InheritContextThreadPoolExecutor from dagster._core.workspace.context import IWorkspaceProcessContext -from dagster._daemon.backfill import execute_backfill_iteration +from dagster._daemon.backfill import execute_backfill_iteration_loop from dagster._daemon.monitoring import ( execute_concurrency_slots_iteration, execute_run_monitoring_iteration, @@ -333,16 +333,39 @@ def core_loop( ) -class BackfillDaemon(IntervalDaemon): +class BackfillDaemon(DagsterDaemon): + def __init__(self, settings: Mapping[str, Any]) -> None: + super().__init__() + self._exit_stack = ExitStack() + self._threadpool_executor: Optional[InheritContextThreadPoolExecutor] = None + + if settings.get("use_threads"): + self._threadpool_executor = self._exit_stack.enter_context( + InheritContextThreadPoolExecutor( + max_workers=settings.get("num_workers"), + thread_name_prefix="backfill_daemon_worker", + ) + ) + @classmethod def daemon_type(cls) -> str: return "BACKFILL" - def run_iteration( + def __exit__(self, _exception_type, _exception_value, _traceback): + self._exit_stack.close() + super().__exit__(_exception_type, _exception_value, _traceback) + + def core_loop( self, workspace_process_context: IWorkspaceProcessContext, + shutdown_event: Event, ) -> DaemonIterator: - yield from execute_backfill_iteration(workspace_process_context, self._logger) + yield from execute_backfill_iteration_loop( + workspace_process_context, + self._logger, + shutdown_event, + threadpool_executor=self._threadpool_executor, + ) class MonitoringDaemon(IntervalDaemon): diff --git a/python_modules/dagster/dagster/_daemon/sensor.py b/python_modules/dagster/dagster/_daemon/sensor.py index f9daf37f65a3b..1611f911cca73 100644 --- a/python_modules/dagster/dagster/_daemon/sensor.py +++ b/python_modules/dagster/dagster/_daemon/sensor.py @@ -66,7 +66,12 @@ from dagster._daemon.utils import DaemonErrorCapture from dagster._scheduler.stale import resolve_stale_or_missing_assets from dagster._time import get_current_datetime, get_current_timestamp -from dagster._utils import DebugCrashFlags, SingleInstigatorDebugCrashFlags, check_for_debug_crash +from dagster._utils import ( + DebugCrashFlags, + SingleInstigatorDebugCrashFlags, + check_for_debug_crash, + return_as_list, +) from dagster._utils.error import SerializableErrorInfo from dagster._utils.merger import merge_dicts @@ -344,7 +349,6 @@ def execute_sensor_iteration_loop( yield SpanMarker.END_SPAN end_time = get_current_timestamp() - loop_duration = end_time - start_time sleep_time = max(0, MIN_INTERVAL_LOOP_TIME - loop_duration) shutdown_event.wait(sleep_time) @@ -453,30 +457,6 @@ def execute_sensor_iteration( ) -def _process_tick( - workspace_process_context: IWorkspaceProcessContext, - logger: logging.Logger, - remote_sensor: RemoteSensor, - sensor_state: InstigatorState, - sensor_debug_crash_flags: Optional[SingleInstigatorDebugCrashFlags], - tick_retention_settings, - submit_threadpool_executor: Optional[ThreadPoolExecutor], -): - # evaluate the tick immediately, but from within a thread. The main thread should be able to - # heartbeat to keep the daemon alive - return list( - _process_tick_generator( - workspace_process_context, - logger, - remote_sensor, - sensor_state, - sensor_debug_crash_flags, - tick_retention_settings, - submit_threadpool_executor, - ) - ) - - def _get_evaluation_tick( instance: DagsterInstance, sensor: RemoteSensor, @@ -630,6 +610,11 @@ def _process_tick_generator( yield error_info +# evaluate the tick immediately, but from within a thread. The main thread should be able to +# heartbeat to keep the daemon alive +_process_tick = return_as_list(_process_tick_generator) + + def _sensor_instigator_data(state: InstigatorState) -> Optional[SensorInstigatorData]: instigator_data = state.instigator_data if instigator_data is None or isinstance(instigator_data, SensorInstigatorData): diff --git a/python_modules/dagster/dagster/_scheduler/scheduler.py b/python_modules/dagster/dagster/_scheduler/scheduler.py index 4c83de5acc5e0..22e3882bb3b0d 100644 --- a/python_modules/dagster/dagster/_scheduler/scheduler.py +++ b/python_modules/dagster/dagster/_scheduler/scheduler.py @@ -197,8 +197,8 @@ def execute_scheduler_iteration_loop( scheduler_run_futures: Dict[str, Future] = {} iteration_times: Dict[str, ScheduleIterationTimes] = {} - submit_threadpool_executor = None threadpool_executor = None + submit_threadpool_executor = None with ExitStack() as stack: settings = workspace_process_context.instance.get_scheduler_settings() @@ -225,6 +225,7 @@ def execute_scheduler_iteration_loop( next_interval_time = _get_next_scheduler_iteration_time(start_time) yield SpanMarker.START_SPAN + try: yield from launch_scheduled_runs( workspace_process_context, @@ -248,8 +249,8 @@ def execute_scheduler_iteration_loop( next_interval_time = min(start_time + ERROR_INTERVAL_TIME, next_interval_time) yield SpanMarker.END_SPAN - end_time = get_current_timestamp() + end_time = get_current_timestamp() if next_interval_time > end_time: # Sleep until the beginning of the next minute, plus a small epsilon to # be sure that we're past the start of the minute diff --git a/python_modules/dagster/dagster/_utils/__init__.py b/python_modules/dagster/dagster/_utils/__init__.py index fce1525d93856..39808e9453cfa 100644 --- a/python_modules/dagster/dagster/_utils/__init__.py +++ b/python_modules/dagster/dagster/_utils/__init__.py @@ -844,3 +844,13 @@ def run_with_concurrent_update_guard( return update_fn(**kwargs) return + + +def return_as_list(func: Callable[..., Iterable[T]]) -> Callable[..., List[T]]: + """A decorator that returns a list from the output of a function.""" + + @functools.wraps(func) + def inner(*args, **kwargs): + return list(func(*args, **kwargs)) + + return inner diff --git a/python_modules/dagster/dagster_tests/daemon_sensor_tests/test_sensor_run.py b/python_modules/dagster/dagster_tests/daemon_sensor_tests/test_sensor_run.py index 6db5285f05648..192d26c9bb9a1 100644 --- a/python_modules/dagster/dagster_tests/daemon_sensor_tests/test_sensor_run.py +++ b/python_modules/dagster/dagster_tests/daemon_sensor_tests/test_sensor_run.py @@ -2908,12 +2908,6 @@ def test_repository_namespacing(executor): assert len(ticks) == 2 -def test_settings(): - settings = {"use_threads": True, "num_workers": 4} - with instance_for_test(overrides={"sensors": settings}) as thread_inst: - assert thread_inst.get_settings("sensors") == settings - - @pytest.mark.parametrize("sensor_name", ["logging_sensor", "multi_asset_logging_sensor"]) def test_sensor_logging(executor, instance, workspace_context, remote_repo, sensor_name) -> None: sensor = remote_repo.get_sensor(sensor_name) diff --git a/python_modules/dagster/dagster_tests/daemon_tests/test_backfill.py b/python_modules/dagster/dagster_tests/daemon_tests/test_backfill.py index 740e8bb9ae0a3..7b88aeecb6dd2 100644 --- a/python_modules/dagster/dagster_tests/daemon_tests/test_backfill.py +++ b/python_modules/dagster/dagster_tests/daemon_tests/test_backfill.py @@ -4,6 +4,9 @@ import string import sys import time +from concurrent.futures import ThreadPoolExecutor +from pathlib import Path +from typing import cast from unittest import mock import dagster._check as check @@ -58,11 +61,11 @@ from dagster._core.execution.backfill import BulkActionStatus, PartitionBackfill from dagster._core.execution.plan.resume_retry import ReexecutionStrategy from dagster._core.remote_representation import ( + CodeLocation, InProcessCodeLocationOrigin, RemoteRepository, RemoteRepositoryOrigin, ) -from dagster._core.remote_representation.code_location import CodeLocation from dagster._core.storage.compute_log_manager import ComputeIOType from dagster._core.storage.dagster_run import ( IN_PROGRESS_RUN_STATUSES, @@ -81,13 +84,17 @@ ) from dagster._core.test_utils import ( create_run_for_test, + create_test_daemon_workspace_context, environ, + instance_for_test, step_did_not_run, step_failed, step_succeeded, + wait_for_futures, ) from dagster._core.types.loadable_target_origin import LoadableTargetOrigin from dagster._core.workspace.context import WorkspaceProcessContext +from dagster._core.workspace.load_target import ModuleTarget from dagster._daemon import get_default_daemon_logger from dagster._daemon.auto_run_reexecution.auto_run_reexecution import ( consume_new_runs_for_automatic_reexecution, @@ -580,10 +587,12 @@ def wait_for_all_runs_to_finish(instance, timeout=10): break +@pytest.mark.parametrize("parallel", [True, False]) def test_simple_backfill( instance: DagsterInstance, workspace_context: WorkspaceProcessContext, remote_repo: RemoteRepository, + parallel: bool, ): partition_set = remote_repo.get_partition_set("the_job_partition_set") instance.add_backfill( @@ -600,7 +609,24 @@ def test_simple_backfill( ) assert instance.get_runs_count() == 0 - list(execute_backfill_iteration(workspace_context, get_default_daemon_logger("BackfillDaemon"))) + if parallel: + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=ThreadPoolExecutor(2), + backfill_futures=backfill_daemon_futures, + ) + ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) assert instance.get_runs_count() == 3 runs = instance.get_runs() @@ -613,6 +639,105 @@ def test_simple_backfill( assert three.tags[PARTITION_NAME_TAG] == "three" +@pytest.mark.parametrize("parallel", [True, False]) +def test_two_backfills_at_the_same_time( + tmp_path: Path, + parallel: bool, +): + # In order to avoid deadlock, we need to ensure that the instance we + # are using will launch runs in separate subprocesses rather than in + # the same in-memory process. This is akin to the context created in + # https://github.com/dagster-io/dagster/blob/a116c44/python_modules/dagster/dagster_tests/scheduler_tests/conftest.py#L53-L71 + with instance_for_test( + overrides={ + "event_log_storage": { + "module": "dagster._core.storage.event_log", + "class": "ConsolidatedSqliteEventLogStorage", + "config": {"base_dir": str(tmp_path)}, + }, + "run_retries": {"enabled": True}, + } + ) as instance: + with create_test_daemon_workspace_context( + workspace_load_target=ModuleTarget( + module_name="dagster_tests.daemon_tests.test_backfill", + attribute="the_repo", + working_directory=os.path.join(os.path.dirname(__file__), "..", ".."), + location_name="test_location", + ), + instance=instance, + ) as workspace_context: + remote_repo = cast( + CodeLocation, + next( + iter( + workspace_context.create_request_context() + .get_code_location_entries() + .values() + ) + ).code_location, + ).get_repository("the_repo") + + first_partition_set = remote_repo.get_partition_set("the_job_partition_set") + second_partition_keys = my_config.partitions_def.get_partition_keys() + second_partition_set = remote_repo.get_partition_set( + "comp_always_succeed_partition_set" + ) + instance.add_backfill( + PartitionBackfill( + backfill_id="simple", + partition_set_origin=first_partition_set.get_remote_origin(), + status=BulkActionStatus.REQUESTED, + partition_names=["one", "two", "three"], + from_failure=False, + reexecution_steps=None, + tags=None, + backfill_timestamp=get_current_timestamp(), + ) + ) + instance.add_backfill( + PartitionBackfill( + backfill_id="partition_schedule_from_job", + partition_set_origin=second_partition_set.get_remote_origin(), + status=BulkActionStatus.REQUESTED, + partition_names=second_partition_keys[:3], + from_failure=False, + reexecution_steps=None, + tags=None, + backfill_timestamp=get_current_timestamp(), + ) + ) + assert instance.get_runs_count() == 0 + + if parallel: + threadpool_executor = ThreadPoolExecutor(4) + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=threadpool_executor, + backfill_futures=backfill_daemon_futures, + ) + ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) + + assert instance.get_runs_count() == 6 + + runs = list(instance.get_runs()) + backfill_ids = sorted(run.tags[BACKFILL_ID_TAG] for run in runs) + partition_names = {run.tags[PARTITION_NAME_TAG] for run in runs} + assert backfill_ids == ["partition_schedule_from_job"] * 3 + ["simple"] * 3 + assert partition_names == {"one", "two", "three", *second_partition_keys[:3]} + + def test_canceled_backfill( instance: DagsterInstance, workspace_context: WorkspaceProcessContext, @@ -648,10 +773,12 @@ def test_canceled_backfill( assert instance.get_runs_count() == 1 +@pytest.mark.parametrize("parallel", [True, False]) def test_failure_backfill( instance: DagsterInstance, workspace_context: WorkspaceProcessContext, remote_repo: RemoteRepository, + parallel: bool, ): output_file = _failure_flag_file() partition_set = remote_repo.get_partition_set("conditional_failure_job_partition_set") @@ -671,11 +798,25 @@ def test_failure_backfill( try: touch_file(output_file) - list( - execute_backfill_iteration( - workspace_context, get_default_daemon_logger("BackfillDaemon") + if parallel: + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=ThreadPoolExecutor(2), + backfill_futures=backfill_daemon_futures, + ) ) - ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) + wait_for_all_runs_to_start(instance) finally: os.remove(output_file) @@ -718,7 +859,25 @@ def test_failure_backfill( ) assert not os.path.isfile(_failure_flag_file()) - list(execute_backfill_iteration(workspace_context, get_default_daemon_logger("BackfillDaemon"))) + if parallel: + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=ThreadPoolExecutor(2), + backfill_futures=backfill_daemon_futures, + ) + ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) + wait_for_all_runs_to_start(instance) assert instance.get_runs_count() == 6 @@ -925,6 +1084,61 @@ def test_large_backfill( assert instance.get_runs_count() == 3 +def test_backfill_is_processed_only_once( + instance: DagsterInstance, + workspace_context: WorkspaceProcessContext, + remote_repo: RemoteRepository, +): + backfill_id = "simple" + partition_set = remote_repo.get_partition_set("config_job_partition_set") + instance.add_backfill( + PartitionBackfill( + backfill_id=backfill_id, + partition_set_origin=partition_set.get_remote_origin(), + status=BulkActionStatus.REQUESTED, + partition_names=["one", "two", "three"], + from_failure=False, + reexecution_steps=None, + tags=None, + backfill_timestamp=get_current_timestamp(), + ) + ) + assert instance.get_runs_count() == 0 + + threadpool_executor = ThreadPoolExecutor(2) + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=threadpool_executor, + backfill_futures=backfill_daemon_futures, + ) + ) + + assert instance.get_runs_count() == 0 + future = backfill_daemon_futures[backfill_id] + + with mock.patch.object( + threadpool_executor, "submit", side_effect=AssertionError("Should not be called") + ): + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=threadpool_executor, + backfill_futures=backfill_daemon_futures, + ) + ) + + assert instance.get_runs_count() == 0 + assert backfill_daemon_futures[backfill_id] is future + + wait_for_futures(backfill_daemon_futures) + + assert instance.get_runs_count() == 3 + + def test_unloadable_backfill(instance, workspace_context): unloadable_origin = _unloadable_partition_set_origin() instance.add_backfill( @@ -2828,10 +3042,12 @@ def test_asset_backfill_logs( assert record_dict.get("msg") +@pytest.mark.parametrize("parallel", [True, False]) def test_asset_backfill_from_asset_graph_subset( instance: DagsterInstance, workspace_context: WorkspaceProcessContext, remote_repo: RemoteRepository, + parallel: bool, ): del remote_repo @@ -2860,7 +3076,24 @@ def test_asset_backfill_from_asset_graph_subset( assert backfill assert backfill.status == BulkActionStatus.REQUESTED - list(execute_backfill_iteration(workspace_context, get_default_daemon_logger("BackfillDaemon"))) + if parallel: + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=ThreadPoolExecutor(2), + backfill_futures=backfill_daemon_futures, + ) + ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) assert instance.get_runs_count() == 3 wait_for_all_runs_to_start(instance, timeout=30) assert instance.get_runs_count() == 3 @@ -2875,7 +3108,24 @@ def test_asset_backfill_from_asset_graph_subset( assert step_succeeded(instance, run, "reusable") assert step_succeeded(instance, run, "bar") - list(execute_backfill_iteration(workspace_context, get_default_daemon_logger("BackfillDaemon"))) + if parallel: + backfill_daemon_futures = {} + list( + execute_backfill_iteration( + workspace_context, + get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=ThreadPoolExecutor(2), + backfill_futures=backfill_daemon_futures, + ) + ) + + wait_for_futures(backfill_daemon_futures) + else: + list( + execute_backfill_iteration( + workspace_context, get_default_daemon_logger("BackfillDaemon") + ) + ) backfill = instance.get_backfill("backfill_from_asset_graph_subset") assert backfill assert backfill.status == BulkActionStatus.COMPLETED_SUCCESS diff --git a/python_modules/dagster/dagster_tests/daemon_tests/test_dagster_daemon.py b/python_modules/dagster/dagster_tests/daemon_tests/test_dagster_daemon.py index 4c656c6077505..1b90b97fea1bc 100644 --- a/python_modules/dagster/dagster_tests/daemon_tests/test_dagster_daemon.py +++ b/python_modules/dagster/dagster_tests/daemon_tests/test_dagster_daemon.py @@ -11,6 +11,13 @@ from dagster._utils.log import get_structlog_json_formatter +@pytest.mark.parametrize("daemon", ["backfills", "schedules", "sensors"]) +def test_settings(daemon): + settings = {"use_threads": True, "num_workers": 4} + with instance_for_test(overrides={daemon: settings}) as thread_inst: + assert thread_inst.get_settings(daemon) == settings + + def test_scheduler_instance(): with instance_for_test( overrides={ diff --git a/python_modules/dagster/dagster_tests/definitions_tests/declarative_automation_tests/daemon_tests/test_e2e.py b/python_modules/dagster/dagster_tests/definitions_tests/declarative_automation_tests/daemon_tests/test_e2e.py index 4a664fda6e6c4..402c0133cfcd4 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/declarative_automation_tests/daemon_tests/test_e2e.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/declarative_automation_tests/daemon_tests/test_e2e.py @@ -176,15 +176,20 @@ def _execute_ticks( ) ) + backfill_daemon_futures = {} list( execute_backfill_iteration( context, get_default_daemon_logger("BackfillDaemon"), + threadpool_executor=threadpool_executor, + backfill_futures=backfill_daemon_futures, + debug_crash_flags=debug_crash_flags or {}, ) ) wait_for_futures(asset_daemon_futures) wait_for_futures(sensor_daemon_futures) + wait_for_futures(backfill_daemon_futures) def _get_current_state(context: WorkspaceRequestContext) -> Mapping[str, InstigatorState]: diff --git a/python_modules/dagster/dagster_tests/scheduler_tests/conftest.py b/python_modules/dagster/dagster_tests/scheduler_tests/conftest.py index 80c87a0a281ea..0679622ec42f8 100644 --- a/python_modules/dagster/dagster_tests/scheduler_tests/conftest.py +++ b/python_modules/dagster/dagster_tests/scheduler_tests/conftest.py @@ -1,10 +1,10 @@ import os import sys -from typing import Iterator, Optional +from typing import Iterator, Optional, cast import pytest from dagster._core.instance import DagsterInstance -from dagster._core.remote_representation.external import RemoteRepository +from dagster._core.remote_representation import CodeLocation, RemoteRepository from dagster._core.test_utils import ( SingleThreadPoolExecutor, create_test_daemon_workspace_context, @@ -73,11 +73,12 @@ def workspace_fixture( @pytest.fixture(name="remote_repo", scope="session") def remote_repo_fixture(workspace_context: WorkspaceProcessContext) -> RemoteRepository: - return next( - iter(workspace_context.create_request_context().get_code_location_entries().values()) - ).code_location.get_repository( # type: ignore # (possible none) - "the_repo" - ) + return cast( + CodeLocation, + next( + iter(workspace_context.create_request_context().get_code_location_entries().values()) + ).code_location, + ).get_repository("the_repo") def loadable_target_origin() -> LoadableTargetOrigin: From ba3caea8c77e1295dd46fc7d925c15ffe1b46c9f Mon Sep 17 00:00:00 2001 From: prha <1040172+prha@users.noreply.github.com> Date: Wed, 18 Dec 2024 14:02:03 -0800 Subject: [PATCH 03/37] add support for incremental run stats calculations (#26550) ## Summary & Motivation For extremely large runs, we'd like to build up the run stats calculations without fetching all required events in a single go. This enables the incremental calculation of run stats. ## How I Tested These Changes BK --- .../dagster/dagster/_core/execution/stats.py | 30 +++++++---- .../storage_tests/test_event_log.py | 51 ++++++++++++++++++- 2 files changed, 70 insertions(+), 11 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/stats.py b/python_modules/dagster/dagster/_core/execution/stats.py index 7ae0756838f5d..0cea0fc4a0a6f 100644 --- a/python_modules/dagster/dagster/_core/execution/stats.py +++ b/python_modules/dagster/dagster/_core/execution/stats.py @@ -37,7 +37,9 @@ def build_run_stats_from_events( - run_id: str, entries: Iterable[EventLogEntry] + run_id: str, + entries: Iterable[EventLogEntry], + previous_stats: Optional[DagsterRunStatsSnapshot] = None, ) -> DagsterRunStatsSnapshot: try: iter(entries) @@ -48,14 +50,24 @@ def build_run_stats_from_events( for i, record in enumerate(entries): check.inst_param(record, f"records[{i}]", EventLogEntry) - steps_succeeded = 0 - steps_failed = 0 - materializations = 0 - expectations = 0 - enqueued_time = None - launch_time = None - start_time = None - end_time = None + if previous_stats: + steps_succeeded = previous_stats.steps_succeeded + steps_failed = previous_stats.steps_failed + materializations = previous_stats.materializations + expectations = previous_stats.expectations + enqueued_time = previous_stats.enqueued_time + launch_time = previous_stats.launch_time + start_time = previous_stats.start_time + end_time = previous_stats.end_time + else: + steps_succeeded = 0 + steps_failed = 0 + materializations = 0 + expectations = 0 + enqueued_time = None + launch_time = None + start_time = None + end_time = None for event in entries: if not event.is_dagster_event: diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py index b111e213861fb..ea234c56387b4 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py @@ -9,8 +9,9 @@ import pytest import sqlalchemy import sqlalchemy as db -from dagster import DagsterInstance +from dagster import AssetKey, AssetMaterialization, DagsterInstance, Output, op from dagster._core.errors import DagsterEventLogInvalidForRun +from dagster._core.execution.stats import build_run_stats_from_events from dagster._core.storage.event_log import ( ConsolidatedSqliteEventLogStorage, SqlEventLogStorageMetadata, @@ -28,7 +29,10 @@ from sqlalchemy import __version__ as sqlalchemy_version from sqlalchemy.engine import Connection -from dagster_tests.storage_tests.utils.event_log_storage import TestEventLogStorage +from dagster_tests.storage_tests.utils.event_log_storage import ( + TestEventLogStorage, + _synthesize_events, +) class TestInMemoryEventLogStorage(TestEventLogStorage): @@ -279,3 +283,46 @@ def test_concurrency_reconcile(): assert _get_slot_count(conn, "bar") == 3 assert _get_limit_row_num(conn, "foo") == 5 assert _get_limit_row_num(conn, "bar") == 3 + + +def test_run_stats(): + @op + def op_success(_): + return 1 + + @op + def asset_op(_): + yield AssetMaterialization(asset_key=AssetKey("asset_1")) + yield Output(1) + + @op + def op_failure(_): + raise ValueError("failing") + + def _ops(): + op_success() + asset_op() + op_failure() + + events, result = _synthesize_events(_ops, check_success=False) + + run_stats = build_run_stats_from_events(result.run_id, events) + + assert run_stats.run_id == result.run_id + assert run_stats.materializations == 1 + assert run_stats.steps_succeeded == 2 + assert run_stats.steps_failed == 1 + assert ( + run_stats.start_time is not None + and run_stats.end_time is not None + and run_stats.end_time > run_stats.start_time + ) + + # build up run stats through incremental events + incremental_run_stats = None + for event in events: + incremental_run_stats = build_run_stats_from_events( + result.run_id, [event], incremental_run_stats + ) + + assert incremental_run_stats == run_stats From e266da3399ed7d3c1abaa34f17713846eba6a674 Mon Sep 17 00:00:00 2001 From: colton Date: Wed, 18 Dec 2024 17:48:41 -0500 Subject: [PATCH 04/37] [docs] remove note that concurrency is only supported for non-sqlite databases (#26579) ## Summary & Motivation Phil's work makes it so concurrency limits work by default, even when using the sqlite database. This removes the note that it's experimental and only supporting certain storages. ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../guides/limiting-concurrency-in-data-pipelines.mdx | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docs/content/guides/limiting-concurrency-in-data-pipelines.mdx b/docs/content/guides/limiting-concurrency-in-data-pipelines.mdx index 010eb5a11e278..ab45e8b7e61e1 100644 --- a/docs/content/guides/limiting-concurrency-in-data-pipelines.mdx +++ b/docs/content/guides/limiting-concurrency-in-data-pipelines.mdx @@ -391,11 +391,6 @@ height={1638} ### Limiting op/asset concurrency across runs - - This feature is experimental and is only supported with Postgres/MySQL - storages. - - #### For specific ops/assets Limits can be specified on the Dagster instance using the special op tag `dagster/concurrency_key`. If this instance limit would be exceeded by launching an op/asset, then the op/asset will be queued. From ff569ad5f19d1792e640eb6afe9a161cfc4fdc6e Mon Sep 17 00:00:00 2001 From: Ion Koutsouris <15728914+ion-elgreco@users.noreply.github.com> Date: Thu, 19 Dec 2024 00:36:20 +0100 Subject: [PATCH 05/37] fix(helm): registry missing in postgresql templating (#26569) ## Summary & Motivation The postgresql chart uses the "registry" value to prefix the image name. This value was not already set in the values.yaml so I had to dig into the chart. Additionally, the Dagster helm templates don't use this registry value to create the image name. So if you do provide a custom registry: "myprivate.acr.io", then it will be correctly used in the postgresql helm, but not in the dagster initdbcontainerReady creation, since there the registry is missing, and it will fail to pull the image. --------- Co-authored-by: Mathieu Larose --- helm/dagster/templates/deployment-celery-queues.yaml | 2 +- helm/dagster/templates/deployment-daemon.yaml | 2 +- helm/dagster/templates/deployment-flower.yaml | 2 +- helm/dagster/templates/helpers/_deployment-webserver.tpl | 2 +- helm/dagster/templates/helpers/_helpers.tpl | 4 ++++ helm/dagster/values.yaml | 1 + 6 files changed, 9 insertions(+), 4 deletions(-) diff --git a/helm/dagster/templates/deployment-celery-queues.yaml b/helm/dagster/templates/deployment-celery-queues.yaml index c2f803376c99f..fdb016523066d 100644 --- a/helm/dagster/templates/deployment-celery-queues.yaml +++ b/helm/dagster/templates/deployment-celery-queues.yaml @@ -42,7 +42,7 @@ spec: initContainers: {{- if $celeryK8sRunLauncherConfig.checkDbReadyInitContainer }} - name: check-db-ready - image: "{{- $.Values.postgresql.image.repository -}}:{{- $.Values.postgresql.image.tag -}}" + image: {{ include "dagster.externalPostgresImage.name" $.Values.postgresql.image | quote }} imagePullPolicy: "{{- $.Values.postgresql.image.pullPolicy -}}" command: ['sh', '-c', {{ include "dagster.postgresql.pgisready" $ | squote }}] securityContext: diff --git a/helm/dagster/templates/deployment-daemon.yaml b/helm/dagster/templates/deployment-daemon.yaml index 91b86b5c7ec73..c63f770463254 100644 --- a/helm/dagster/templates/deployment-daemon.yaml +++ b/helm/dagster/templates/deployment-daemon.yaml @@ -51,7 +51,7 @@ spec: initContainers: {{- if .Values.dagsterDaemon.checkDbReadyInitContainer }} - name: check-db-ready - image: {{ include "dagster.externalImage.name" $.Values.postgresql.image | quote }} + image: {{ include "dagster.externalPostgresImage.name" $.Values.postgresql.image | quote }} imagePullPolicy: "{{- $.Values.postgresql.image.pullPolicy -}}" command: ['sh', '-c', {{ include "dagster.postgresql.pgisready" . | squote }}] securityContext: diff --git a/helm/dagster/templates/deployment-flower.yaml b/helm/dagster/templates/deployment-flower.yaml index 63b9c7d44a515..ca0ed535362ab 100644 --- a/helm/dagster/templates/deployment-flower.yaml +++ b/helm/dagster/templates/deployment-flower.yaml @@ -38,7 +38,7 @@ spec: initContainers: {{- if .Values.flower.checkDbReadyInitContainer }} - name: check-db-ready - image: "{{- $.Values.postgresql.image.repository -}}:{{- $.Values.postgresql.image.tag -}}" + image: {{ include "dagster.externalPostgresImage.name" $.Values.postgresql.image | quote }} imagePullPolicy: "{{- $.Values.postgresql.image.pullPolicy -}}" command: ['sh', '-c', {{ include "dagster.postgresql.pgisready" . | squote }}] securityContext: diff --git a/helm/dagster/templates/helpers/_deployment-webserver.tpl b/helm/dagster/templates/helpers/_deployment-webserver.tpl index 9427646bf4999..db97496d81408 100644 --- a/helm/dagster/templates/helpers/_deployment-webserver.tpl +++ b/helm/dagster/templates/helpers/_deployment-webserver.tpl @@ -49,7 +49,7 @@ spec: initContainers: {{- if .Values.dagsterWebserver.checkDbReadyInitContainer }} - name: check-db-ready - image: {{ include "dagster.externalImage.name" .Values.postgresql.image | quote }} + image: {{ include "dagster.externalPostgresImage.name" .Values.postgresql.image | quote }} imagePullPolicy: {{ .Values.postgresql.image.pullPolicy }} command: ['sh', '-c', {{ include "dagster.postgresql.pgisready" . | squote }}] securityContext: diff --git a/helm/dagster/templates/helpers/_helpers.tpl b/helm/dagster/templates/helpers/_helpers.tpl index bc654a1d1a839..33d862b260f0b 100644 --- a/helm/dagster/templates/helpers/_helpers.tpl +++ b/helm/dagster/templates/helpers/_helpers.tpl @@ -30,6 +30,10 @@ If release name contains chart name it will be used as a full name. {{- .repository -}}:{{- .tag -}} {{- end }} +{{- define "dagster.externalPostgresImage.name" }} +{{- .registry -}}/{{- .repository -}}:{{- .tag -}} +{{- end }} + {{- define "dagster.dagsterImage.name" }} {{- $ := index . 0 }} diff --git a/helm/dagster/values.yaml b/helm/dagster/values.yaml index 3e7c6cb41c5da..b16da4a504a99 100644 --- a/helm/dagster/values.yaml +++ b/helm/dagster/values.yaml @@ -767,6 +767,7 @@ postgresql: # Used by init container to check that db is running. (Even if enabled:false) image: + registry: "docker.io" repository: "library/postgres" tag: "14.6" pullPolicy: IfNotPresent From f7869db064400d43ff901da5b7567a0a5c50c21b Mon Sep 17 00:00:00 2001 From: gibsondan Date: Wed, 18 Dec 2024 18:39:03 -0600 Subject: [PATCH 06/37] add support for run retries with default sqlite storage (#26581) Summary: The underlying storage changes were already ready to go for this and it was just a matter of removing gating and tweaking some tests. BK Run retries can now be used while using Sqlite storage. > Insert changelog entry or delete this section. ## Summary & Motivation ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. --- .../dagster/_core/instance/__init__.py | 7 ----- .../dagster/_core/storage/event_log/base.py | 3 -- .../_core/storage/event_log/sql_event_log.py | 3 -- .../event_log/sqlite/sqlite_event_log.py | 3 -- .../dagster/_core/storage/legacy_storage.py | 13 ++++++++ .../storage_tests/test_event_log.py | 6 ++++ .../storage_tests/utils/event_log_storage.py | 31 +++---------------- 7 files changed, 23 insertions(+), 43 deletions(-) diff --git a/python_modules/dagster/dagster/_core/instance/__init__.py b/python_modules/dagster/dagster/_core/instance/__init__.py index f781a49e602e2..92c177d1d97ce 100644 --- a/python_modules/dagster/dagster/_core/instance/__init__.py +++ b/python_modules/dagster/dagster/_core/instance/__init__.py @@ -491,13 +491,6 @@ def __init__( " run worker will be marked as failed, but will not be resumed.", ) - if self.run_retries_enabled: - check.invariant( - self.event_log_storage.supports_event_consumer_queries(), - "Run retries are enabled, but the configured event log storage does not support" - " them. Consider switching to Postgres or Mysql.", - ) - # Used for batched event handling self._event_buffer: Dict[str, List[EventLogEntry]] = defaultdict(list) diff --git a/python_modules/dagster/dagster/_core/storage/event_log/base.py b/python_modules/dagster/dagster/_core/storage/event_log/base.py index b82eae23ad458..86d70c8069387 100644 --- a/python_modules/dagster/dagster/_core/storage/event_log/base.py +++ b/python_modules/dagster/dagster/_core/storage/event_log/base.py @@ -336,9 +336,6 @@ def get_event_records( ) -> Sequence[EventLogRecord]: pass - def supports_event_consumer_queries(self) -> bool: - return False - def get_logs_for_all_runs_by_log_id( self, after_cursor: int = -1, diff --git a/python_modules/dagster/dagster/_core/storage/event_log/sql_event_log.py b/python_modules/dagster/dagster/_core/storage/event_log/sql_event_log.py index 1bb1398ec7ef3..94e6d21d02ac6 100644 --- a/python_modules/dagster/dagster/_core/storage/event_log/sql_event_log.py +++ b/python_modules/dagster/dagster/_core/storage/event_log/sql_event_log.py @@ -1017,9 +1017,6 @@ def _get_event_records( return event_records - def supports_event_consumer_queries(self) -> bool: - return True - def _get_event_records_result( self, event_records_filter: EventRecordsFilter, diff --git a/python_modules/dagster/dagster/_core/storage/event_log/sqlite/sqlite_event_log.py b/python_modules/dagster/dagster/_core/storage/event_log/sqlite/sqlite_event_log.py index 69807391d07c9..7cfcc2bfecfc2 100644 --- a/python_modules/dagster/dagster/_core/storage/event_log/sqlite/sqlite_event_log.py +++ b/python_modules/dagster/dagster/_core/storage/event_log/sqlite/sqlite_event_log.py @@ -434,9 +434,6 @@ def fetch_run_status_changes( has_more = len(records) == limit return EventRecordsResult(records, cursor=new_cursor, has_more=has_more) - def supports_event_consumer_queries(self) -> bool: - return False - def wipe(self) -> None: # should delete all the run-sharded db files and drop the contents of the index for filename in ( diff --git a/python_modules/dagster/dagster/_core/storage/legacy_storage.py b/python_modules/dagster/dagster/_core/storage/legacy_storage.py index 28668cb5fb2e6..b44832c34afc1 100644 --- a/python_modules/dagster/dagster/_core/storage/legacy_storage.py +++ b/python_modules/dagster/dagster/_core/storage/legacy_storage.py @@ -416,6 +416,19 @@ def get_logs_for_run( run_id, cursor, of_type, limit, ascending ) + def get_logs_for_all_runs_by_log_id( + self, + after_cursor: int = -1, + dagster_event_type: Optional[Union["DagsterEventType", Set["DagsterEventType"]]] = None, + limit: Optional[int] = None, + ) -> Mapping[int, "EventLogEntry"]: + return self._storage.event_log_storage.get_logs_for_all_runs_by_log_id( + after_cursor=after_cursor, dagster_event_type=dagster_event_type, limit=limit + ) + + def get_maximum_record_id(self) -> Optional[int]: + return self._storage.event_log_storage.get_maximum_record_id() + def get_stats_for_run(self, run_id: str) -> "DagsterRunStatsSnapshot": return self._storage.event_log_storage.get_stats_for_run(run_id) diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py index ea234c56387b4..8f1d0cc194756 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py @@ -70,6 +70,9 @@ def event_log_storage(self, instance): assert isinstance(event_log_storage, SqliteEventLogStorage) yield instance.event_log_storage + def supports_multiple_event_type_queries(self): + return False + def test_filesystem_event_log_storage_run_corrupted(self, storage): # URL begins sqlite:/// @@ -186,6 +189,9 @@ def event_log_storage(self, instance): def is_sqlite(self, storage): return True + def supports_multiple_event_type_queries(self): + return False + @pytest.mark.parametrize("dagster_event_type", ["dummy"]) def test_get_latest_tags_by_partition(self, storage, instance, dagster_event_type): pytest.skip("skip this since legacy storage is harder to mock.patch") diff --git a/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py b/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py index 1bfbd2ec23fbc..e893bad1d38ba 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py +++ b/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py @@ -4142,9 +4142,6 @@ def never_materializes_asset(): assert result.run_id == records[0].asset_entry.last_run_id def test_get_logs_for_all_runs_by_log_id_of_type(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - @op def return_one(_): return 1 @@ -4164,9 +4161,6 @@ def _ops(): ) == [DagsterEventType.RUN_SUCCESS, DagsterEventType.RUN_SUCCESS] def test_get_logs_for_all_runs_by_log_id_by_multi_type(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") @@ -4197,9 +4191,6 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_cursor(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - @op def return_one(_): return 1 @@ -4234,9 +4225,6 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_cursor_multi_type(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") @@ -4281,9 +4269,6 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_limit(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - @op def return_one(_): return 1 @@ -4314,9 +4299,6 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_limit_multi_type(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") @@ -4347,9 +4329,6 @@ def _ops(): ] def test_get_maximum_record_id(self, storage: EventLogStorage): - if not storage.supports_event_consumer_queries(): - pytest.skip("storage does not support event consumer queries") - storage.wipe() storage.store_event( @@ -4360,9 +4339,8 @@ def test_get_maximum_record_id(self, storage: EventLogStorage): run_id=make_new_run_id(), timestamp=time.time(), dagster_event=DagsterEvent( - DagsterEventType.ENGINE_EVENT.value, - "nonce", - event_specific_data=EngineEventData.in_process(999), + DagsterEventType.RUN_SUCCESS.value, + "my_job", ), ) ) @@ -4379,9 +4357,8 @@ def test_get_maximum_record_id(self, storage: EventLogStorage): run_id=make_new_run_id(), timestamp=time.time(), dagster_event=DagsterEvent( - DagsterEventType.ENGINE_EVENT.value, - "nonce", - event_specific_data=EngineEventData.in_process(999), + DagsterEventType.RUN_SUCCESS.value, + "my_job", ), ) ) From cd526bd7986e552cc594d202b5b2cc20c7437912 Mon Sep 17 00:00:00 2001 From: Christopher DeCarolis Date: Wed, 18 Dec 2024 17:53:04 -0800 Subject: [PATCH 07/37] [io-managers-deps] Additional asset check tests (#26461) I used this PR to accumulate the upstack. ## Summary & Motivation Add some additional asset check tests which outline the behavior when args are set in various ways. I found this illuminating to build against for the downstream PRs. ## How I Tested These Changes Just tests --- .../dagster/_core/definitions/asset_in.py | 7 + .../decorators/asset_check_decorator.py | 45 ++- .../definitions/decorators/asset_decorator.py | 6 +- .../decorator_assets_definition_builder.py | 91 +++-- .../asset_defs_tests/test_asset_deps.py | 153 ++++++++- .../asset_defs_tests/test_assets.py | 4 +- .../test_asset_check_decorator.py | 320 +++++++++++++++++- ..._asset_check_decorator_secondary_assets.py | 22 +- 8 files changed, 583 insertions(+), 65 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/asset_in.py b/python_modules/dagster/dagster/_core/definitions/asset_in.py index 79b06ec0647f2..448ec1d3476ea 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_in.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_in.py @@ -77,3 +77,10 @@ def __new__( else resolve_dagster_type(dagster_type) ), ) + + @classmethod + def from_coercible(cls, coercible: "CoercibleToAssetIn") -> "AssetIn": + return coercible if isinstance(coercible, AssetIn) else AssetIn(key=coercible) + + +CoercibleToAssetIn = Union[AssetIn, CoercibleToAssetKey] diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py index c52ee4735a1df..a04248af80be0 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py @@ -1,4 +1,4 @@ -from typing import AbstractSet, Any, Callable, Iterable, Mapping, Optional, Sequence, Set, Union +from typing import Any, Callable, Iterable, Mapping, Optional, Sequence, Set, Union from typing_extensions import TypeAlias @@ -8,8 +8,8 @@ from dagster._core.definitions.asset_check_result import AssetCheckResult from dagster._core.definitions.asset_check_spec import AssetCheckSpec from dagster._core.definitions.asset_checks import AssetChecksDefinition -from dagster._core.definitions.asset_dep import CoercibleToAssetDep -from dagster._core.definitions.asset_in import AssetIn +from dagster._core.definitions.asset_dep import AssetDep, CoercibleToAssetDep +from dagster._core.definitions.asset_in import AssetIn, CoercibleToAssetIn from dagster._core.definitions.asset_key import AssetCheckKey from dagster._core.definitions.assets import AssetsDefinition from dagster._core.definitions.declarative_automation.automation_condition import ( @@ -20,9 +20,10 @@ DecoratorAssetsDefinitionBuilder, DecoratorAssetsDefinitionBuilderArgs, NamedIn, - build_named_ins, + build_and_validate_named_ins, compute_required_resource_keys, get_function_params_without_context_or_config_or_resources, + validate_named_ins_subset_of_deps, ) from dagster._core.definitions.decorators.op_decorator import _Op from dagster._core.definitions.events import AssetKey, CoercibleToAssetKey @@ -44,7 +45,7 @@ def _build_asset_check_named_ins( asset_key: AssetKey, fn: Callable[..., Any], additional_ins: Mapping[str, AssetIn], - additional_deps: Optional[AbstractSet[AssetKey]], + additional_deps: Mapping[AssetKey, AssetDep], ) -> Mapping[AssetKey, NamedIn]: fn_params = get_function_params_without_context_or_config_or_resources(fn) @@ -66,9 +67,9 @@ def _build_asset_check_named_ins( f"'{in_name}' is specified in 'additional_ins' but isn't a parameter." ) - # if all the fn_params are in additional_ins, then we add the prmary asset as a dep + # if all the fn_params are in additional_ins, then we add the primary asset as a dep if len(fn_params) == len(additional_ins): - all_deps = {*(additional_deps if additional_deps else set()), asset_key} + all_deps = {**additional_deps, **{asset_key: AssetDep(asset_key)}} all_ins = additional_ins # otherwise there should be one extra fn_param, which is the primary asset. Add that as an input elif len(fn_params) == len(additional_ins) + 1: @@ -87,10 +88,10 @@ def _build_asset_check_named_ins( " the target asset or be specified in 'additional_ins'." ) - return build_named_ins( + return build_and_validate_named_ins( fn=fn, asset_ins=all_ins, - deps=all_deps, + deps=all_deps.values(), ) @@ -189,7 +190,11 @@ def inner(fn: AssetCheckFunction) -> AssetChecksDefinition: resolved_name = name or fn.__name__ asset_key = AssetKey.from_coercible_or_definition(asset) - additional_dep_keys = set([dep.asset_key for dep in make_asset_deps(additional_deps) or []]) + additional_dep_keys = ( + {dep.asset_key: dep for dep in make_asset_deps(additional_deps) or []} + if additional_deps + else {} + ) named_in_by_asset_key = _build_asset_check_named_ins( resolved_name, asset_key, @@ -283,6 +288,7 @@ def multi_asset_check( required_resource_keys: Optional[Set[str]] = None, retry_policy: Optional[RetryPolicy] = None, config_schema: Optional[UserConfigSchema] = None, + ins: Optional[Mapping[str, CoercibleToAssetIn]] = None, ) -> Callable[[Callable[..., Any]], AssetChecksDefinition]: """Defines a set of asset checks that can be executed together with the same op. @@ -306,6 +312,8 @@ def multi_asset_check( retry_policy (Optional[RetryPolicy]): The retry policy for the op that executes the checks. can_subset (bool): Whether the op can emit results for a subset of the asset checks keys, based on the context.selected_asset_check_keys argument. Defaults to False. + ins (Optional[Mapping[str, Union[AssetKey, AssetIn]]]): A mapping from input name to AssetIn depended upon by + a given asset check. If an AssetKey is provided, it will be converted to an AssetIn with the same key. Examples: @@ -345,12 +353,21 @@ def inner(fn: MultiAssetCheckFunction) -> AssetChecksDefinition: outs = { spec.get_python_identifier(): Out(None, is_required=not can_subset) for spec in specs } - named_ins_by_asset_key = build_named_ins( + all_deps_by_key = { + **{spec.asset_key: AssetDep(spec.asset_key) for spec in specs}, + **{dep.asset_key: dep for spec in specs for dep in (spec.additional_deps or [])}, + } + + named_ins_by_asset_key = build_and_validate_named_ins( fn=fn, - asset_ins={}, - deps={spec.asset_key for spec in specs} - | {dep.asset_key for spec in specs for dep in spec.additional_deps or []}, + asset_ins={ + inp_name: AssetIn.from_coercible(coercible) for inp_name, coercible in ins.items() + } + if ins + else {}, + deps=all_deps_by_key.values(), ) + validate_named_ins_subset_of_deps(named_ins_by_asset_key, all_deps_by_key) with disable_dagster_warnings(): op_def = _Op( diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py index 3aaa35ba4a0c3..4f53f0668b03b 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py @@ -40,7 +40,7 @@ from dagster._core.definitions.decorators.decorator_assets_definition_builder import ( DecoratorAssetsDefinitionBuilder, DecoratorAssetsDefinitionBuilderArgs, - build_named_ins, + build_and_validate_named_ins, build_named_outs, create_check_specs_by_output_name, validate_and_assign_output_names_to_check_specs, @@ -911,7 +911,7 @@ def graph_asset_no_defaults( kinds: Optional[AbstractSet[str]], ) -> AssetsDefinition: ins = ins or {} - named_ins = build_named_ins(compose_fn, ins or {}, set()) + named_ins = build_and_validate_named_ins(compose_fn, ins or {}, set()) out_asset_key, _asset_name = resolve_asset_key_and_name_for_decorator( key=key, key_prefix=key_prefix, @@ -1030,7 +1030,7 @@ def inner(fn: Callable[..., Any]) -> AssetsDefinition: if asset_in.partition_mapping } - named_ins = build_named_ins(fn, ins or {}, set()) + named_ins = build_and_validate_named_ins(fn, ins or {}, set()) keys_by_input_name = { input_name: asset_key for asset_key, (input_name, _) in named_ins.items() } diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py index 0eab760bc93e7..f96462413a596 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py @@ -75,13 +75,39 @@ def get_function_params_without_context_or_config_or_resources( return new_input_args -def build_named_ins( +def validate_can_coexist(asset_in: AssetIn, asset_dep: AssetDep) -> None: + """Validates that the asset_in and asset_dep can coexist peacefully on the same asset key. + If both asset_in and asset_dep are set on the same asset key, expect that _no_ properties + are set on AssetIn except for the key itself. + """ + if ( + asset_in.metadata + or asset_in.key_prefix + or asset_in.dagster_type != NoValueSentinel + or asset_in.partition_mapping is not None + ): + raise DagsterInvalidDefinitionError( + f"Asset key '{asset_dep.asset_key.to_user_string()}' is used as both an input (via AssetIn) and a dependency (via AssetDep). If an asset key is used as an input and also set as a dependency, the input should only define the relationship between the asset key and the input name, or optionally set the input_manager_key. Any other properties should either not be set, or should be set on the dependency." + ) + + +def build_and_validate_named_ins( fn: Callable[..., Any], asset_ins: Mapping[str, AssetIn], - deps: Optional[AbstractSet[AssetKey]], + deps: Optional[Iterable[AssetDep]], ) -> Mapping[AssetKey, "NamedIn"]: """Creates a mapping from AssetKey to (name of input, In object).""" - deps = check.opt_set_param(deps, "deps", AssetKey) + deps_by_key = {dep.asset_key: dep for dep in deps} if deps else {} + ins_by_asset_key = { + asset_in.key if asset_in.key else AssetKey.from_coercible(input_name): asset_in + for input_name, asset_in in asset_ins.items() + } + shared_keys_between_ins_and_deps = set(ins_by_asset_key.keys()) & set(deps_by_key.keys()) + if shared_keys_between_ins_and_deps: + for shared_key in shared_keys_between_ins_and_deps: + validate_can_coexist(ins_by_asset_key[shared_key], deps_by_key[shared_key]) + + deps = check.opt_iterable_param(deps, "deps", AssetDep) new_input_args = get_function_params_without_context_or_config_or_resources(fn) @@ -126,16 +152,12 @@ def build_named_ins( In(metadata=metadata, input_manager_key=input_manager_key, dagster_type=dagster_type), ) - for asset_key in deps: - if asset_key in named_ins_by_asset_key: - raise DagsterInvalidDefinitionError( - f"deps value {asset_key} also declared as input/AssetIn" + for dep in deps: + if dep.asset_key not in named_ins_by_asset_key: + named_ins_by_asset_key[dep.asset_key] = NamedIn( + stringify_asset_key_to_input_name(dep.asset_key), + In(cast(type, Nothing)), ) - # mypy doesn't realize that Nothing is a valid type here - named_ins_by_asset_key[asset_key] = NamedIn( - stringify_asset_key_to_input_name(asset_key), - In(cast(type, Nothing)), - ) return named_ins_by_asset_key @@ -348,25 +370,23 @@ def from_multi_asset_specs( ), ) - upstream_keys = set() + upstream_deps = {} for spec in asset_specs: for dep in spec.deps: if dep.asset_key not in named_outs_by_asset_key: - upstream_keys.add(dep.asset_key) + upstream_deps[dep.asset_key] = dep if dep.asset_key in named_outs_by_asset_key and dep.partition_mapping is not None: # self-dependent asset also needs to be considered an upstream_key - upstream_keys.add(dep.asset_key) + upstream_deps[dep.asset_key] = dep # get which asset keys have inputs set - loaded_upstreams = build_named_ins(fn, asset_in_map, deps=set()) - unexpected_upstreams = {key for key in loaded_upstreams.keys() if key not in upstream_keys} - if unexpected_upstreams: - raise DagsterInvalidDefinitionError( - f"Asset inputs {unexpected_upstreams} do not have dependencies on the passed" - " AssetSpec(s). Set the deps on the appropriate AssetSpec(s)." - ) - remaining_upstream_keys = {key for key in upstream_keys if key not in loaded_upstreams} - named_ins_by_asset_key = build_named_ins(fn, asset_in_map, deps=remaining_upstream_keys) + named_ins_by_asset_key = build_and_validate_named_ins( + fn, asset_in_map, deps=upstream_deps.values() + ) + # We expect that asset_ins are a subset of asset_deps. The reason we do not check this in + # `build_and_validate_named_ins` is because in other decorator pathways, we allow for argument-based + # dependencies which are not specified in deps (such as the asset decorator). + validate_named_ins_subset_of_deps(named_ins_by_asset_key, upstream_deps) internal_deps = { spec.key: {dep.asset_key for dep in spec.deps} @@ -401,10 +421,10 @@ def from_asset_outs_in_asset_centric_decorator( check.param_invariant( not passed_args.specs, "args", "This codepath for non-spec based create" ) - named_ins_by_asset_key = build_named_ins( + named_ins_by_asset_key = build_and_validate_named_ins( fn, asset_in_map, - deps=({dep.asset_key for dep in upstream_asset_deps} if upstream_asset_deps else set()), + deps=upstream_asset_deps or set(), ) named_outs_by_asset_key = build_named_outs(asset_out_map) @@ -653,3 +673,22 @@ def _validate_check_specs_target_relevant_asset_keys( f"Invalid asset key {spec.asset_key} in check spec {spec.name}. Must be one of" f" {valid_asset_keys}" ) + + +def validate_named_ins_subset_of_deps( + named_ins_per_key: Mapping[AssetKey, NamedIn], + asset_deps_by_key: Mapping[AssetKey, AssetDep], +) -> None: + """Validates that the asset_ins are a subset of the asset_deps. This is a common validation + that we need to do in multiple places, so we've factored it out into a helper function. + """ + asset_dep_keys = set(asset_deps_by_key.keys()) + asset_in_keys = set(named_ins_per_key.keys()) + + if asset_in_keys - asset_dep_keys: + invalid_asset_in_keys = asset_in_keys - asset_dep_keys + raise DagsterInvalidDefinitionError( + f"Invalid asset dependencies: `{invalid_asset_in_keys}` specified as AssetIns, but" + " are not specified as `AssetDep` objects on any constituent AssetSpec objects. Asset inputs must be associated with an" + " output produced by the asset." + ) diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_deps.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_deps.py index 8b9f9056b95cd..a3cfde7ba0541 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_deps.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_deps.py @@ -4,6 +4,7 @@ AssetOut, FilesystemIOManager, IOManager, + Nothing, SourceAsset, TimeWindowPartitionMapping, asset, @@ -12,7 +13,9 @@ ) from dagster._check import ParameterCheckError from dagster._core.definitions.asset_dep import AssetDep +from dagster._core.definitions.asset_in import AssetIn from dagster._core.definitions.asset_spec import AssetSpec +from dagster._core.definitions.partition_mapping import IdentityPartitionMapping from dagster._core.errors import DagsterInvalidDefinitionError, DagsterInvariantViolationError from dagster._core.types.dagster_type import DagsterTypeKind @@ -522,20 +525,158 @@ def my_asset(): def test_dep_via_deps_and_fn(): + """Test combining deps and ins in the same @asset-decorated function.""" + @asset def the_upstream_asset(): return 1 - with pytest.raises( - DagsterInvalidDefinitionError, - match=r"deps value .* also declared as input/AssetIn", - ): + # When deps and ins are both set, expect that deps is only used for the asset key and potentially input name. + for param_dict in [ + {"partition_mapping": IdentityPartitionMapping()}, + {"metadata": {"foo": "bar"}}, + {"key_prefix": "prefix"}, + {"dagster_type": Nothing}, + ]: + with pytest.raises(DagsterInvalidDefinitionError): + + @asset( + deps=[AssetDep(the_upstream_asset)], + ins={"the_upstream_asset": AssetIn(**param_dict)}, + ) + def _(the_upstream_asset): + return None + + # We allow the asset key to be set via deps and ins as long as no additional information is set. + @asset(deps=[the_upstream_asset]) + def depends_on_upstream_asset_implicit_remap(the_upstream_asset): + assert the_upstream_asset == 1 + + @asset( + deps=[AssetDep(the_upstream_asset)], ins={"remapped": AssetIn(key=the_upstream_asset.key)} + ) + def depends_on_upstream_asset_explicit_remap(remapped): + assert remapped == 1 + + res = materialize( + [ + the_upstream_asset, + depends_on_upstream_asset_implicit_remap, + depends_on_upstream_asset_explicit_remap, + ], + ) + assert res.success + + @asset + def upstream2(): + return 2 + + # As an unfortunate consequence of the many iterations of dependency specification and the fact that they were all additive with each other, + # we have to support the case where deps are specified separately in both the function signature and the decorator. + # This is not recommended, but it is supported. + @asset(deps=[the_upstream_asset]) + def some_explicit_and_implicit_deps(the_upstream_asset, upstream2): + assert the_upstream_asset == 1 + assert upstream2 == 2 + + @asset(deps=[the_upstream_asset], ins={"remapped": AssetIn(key=upstream2.key)}) + def deps_disjoint_between_args(the_upstream_asset, remapped): + assert the_upstream_asset == 1 + assert remapped == 2 + + res = materialize( + [ + the_upstream_asset, + upstream2, + some_explicit_and_implicit_deps, + deps_disjoint_between_args, + ], + ) + assert res.success + - @asset(deps=[the_upstream_asset]) - def depends_on_upstream_asset(the_upstream_asset): +def test_multi_asset_specs_deps_and_fn(): + @asset + def the_upstream_asset(): + return 1 + + # When deps and ins are both set, expect that deps is only used for the asset key and potentially input name. + for param_dict in [ + {"partition_mapping": IdentityPartitionMapping()}, + {"metadata": {"foo": "bar"}}, + {"key_prefix": "prefix"}, + {"dagster_type": Nothing}, + ]: + with pytest.raises(DagsterInvalidDefinitionError): + + @multi_asset( + specs=[AssetSpec("the_asset", deps=[AssetDep(the_upstream_asset)])], + ins={"the_upstream_asset": AssetIn(**param_dict)}, + ) + def _(the_upstream_asset): + return None + + # We allow the asset key to be set via deps and ins as long as no additional information is set. + @multi_asset(specs=[AssetSpec("the_asset", deps=[the_upstream_asset])]) + def depends_on_upstream_asset_implicit_remap(the_upstream_asset): + assert the_upstream_asset == 1 + + @multi_asset( + specs=[AssetSpec("other_asset", deps=[AssetDep(the_upstream_asset)])], + ins={"remapped": AssetIn(key=the_upstream_asset.key)}, + ) + def depends_on_upstream_asset_explicit_remap(remapped): + assert remapped == 1 + + res = materialize( + [ + the_upstream_asset, + depends_on_upstream_asset_implicit_remap, + depends_on_upstream_asset_explicit_remap, + ], + ) + assert res.success + + # We do not allow you to set a dependency purely via input if you're opting in to the spec pattern. + with pytest.raises(DagsterInvalidDefinitionError): + + @multi_asset( + specs=[AssetSpec("the_asset")], + ) + def _(the_upstream_asset): return None +def test_allow_remapping_io_manager_key() -> None: + @asset + def the_upstream_asset(): + return 1 + + @asset( + deps=[the_upstream_asset], + ins={"the_upstream_asset": AssetIn(input_manager_key="custom_io")}, + ) + def depends_on_upstream_asset(the_upstream_asset): + assert the_upstream_asset == 1 + + calls = [] + + class MyIOManager(IOManager): + def handle_output(self, context, obj): + raise Exception("Should not be called") + + def load_input(self, context): + calls.append("load_input") + return 1 + + res = materialize( + [the_upstream_asset, depends_on_upstream_asset], + resources={"custom_io": MyIOManager()}, + ) + assert res.success + assert calls == ["load_input"] + + def test_duplicate_deps(): @asset def the_upstream_asset(): diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets.py index 1f29844b42a83..5fbc36e955cbb 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets.py @@ -1976,7 +1976,7 @@ def also_input(table_A): ... with pytest.raises( DagsterInvalidDefinitionError, - match="do not have dependencies on the passed AssetSpec", + match="specified as AssetIns", ): @multi_asset(specs=[table_b, table_c]) @@ -1984,7 +1984,7 @@ def rogue_input(table_X): ... with pytest.raises( DagsterInvalidDefinitionError, - match="do not have dependencies on the passed AssetSpec", + match="specified as AssetIns", ): @multi_asset(specs=[table_b_no_dep, table_c_no_dep]) diff --git a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator.py b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator.py index 09b0151444fa4..3299b270c437c 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator.py @@ -28,6 +28,7 @@ ) from dagster._core.definitions.asset_check_spec import AssetCheckKey from dagster._core.definitions.asset_checks import AssetChecksDefinition +from dagster._core.definitions.asset_in import AssetIn from dagster._core.errors import ( DagsterInvalidDefinitionError, DagsterInvalidSubsetError, @@ -118,19 +119,105 @@ def my_check() -> AssetCheckResult: assert spec.asset_key == AssetKey(["prefix", "asset1"]) +def test_asset_check_input() -> None: + @asset + def asset1() -> int: + return 5 + + @asset_check(asset=asset1) + def my_check1(asset1: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset1 == 5) + + @asset_check(asset=asset1) + def my_check2(random_name: int) -> AssetCheckResult: + return AssetCheckResult(passed=random_name == 5) + + @asset_check(asset=asset1) + def my_check3(context, random_name: int) -> AssetCheckResult: + return AssetCheckResult(passed=random_name == 5) + + @asset_check(asset=asset1) + def my_check4(context, asset1: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset1 == 5) + + result = execute_assets_and_checks( + assets=[asset1], asset_checks=[my_check1, my_check2, my_check3, my_check4] + ) + + assert result.success + assert len(result.get_asset_check_evaluations()) == 4 + assert all(check.passed for check in result.get_asset_check_evaluations()) + + +def test_asset_check_additional_ins() -> None: + @asset + def asset1() -> int: + return 5 + + @asset + def asset2() -> int: + return 4 + + @asset_check(asset=asset1, additional_ins={"asset2": AssetIn(key=asset2.key)}) + def my_check(asset1: int, asset2: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset1 == 5 and asset2 == 4) + + @asset_check(asset=asset1, additional_ins={"asset2": AssetIn(key=asset2.key)}) + def my_check2(asset2: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset2 == 4) + + @asset_check(asset=asset1, additional_ins={"asset2": AssetIn(key=asset2.key)}) + def my_check3(context, asset2: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset2 == 4) + + # Error bc asset2 is in additional_ins but not in the function signature + with pytest.raises(DagsterInvalidDefinitionError): + + @asset_check(asset=asset1, additional_ins={"asset2": AssetIn(key=asset2.key)}) + def my_check4(): + return AssetCheckResult(passed=True) + + # Error bc asset1 is in both additional_ins and the function signature + with pytest.raises(DagsterInvalidDefinitionError): + + @asset_check(asset=asset1, additional_ins={"asset1": AssetIn(key=asset1.key)}) + def my_check5(asset1: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset1 == 5) + + # Error bc asset2 is in the function signature but not additional_ins + with pytest.raises(DagsterInvalidDefinitionError): + + @asset_check(asset=asset1, additional_ins={"asset2": AssetIn(key=asset2.key)}) + def my_check6(asset1: int) -> AssetCheckResult: + return AssetCheckResult(passed=asset1 == 5) + + result = execute_assets_and_checks( + assets=[asset1, asset2], asset_checks=[my_check, my_check2, my_check3] + ) + + assert result.success + assert len(result.get_asset_check_evaluations()) == 3 + assert all(check.passed for check in result.get_asset_check_evaluations()) + + def test_asset_check_input_with_prefix() -> None: @asset(key_prefix="prefix") - def asset1() -> None: ... + def asset1() -> int: + return 5 @asset_check(asset=asset1) - def my_check(asset1) -> AssetCheckResult: - return AssetCheckResult(passed=True) + def my_check(unrelated_name: int) -> AssetCheckResult: + return AssetCheckResult(passed=unrelated_name == 5) spec = my_check.get_spec_for_check_key( AssetCheckKey(AssetKey(["prefix", "asset1"]), "my_check") ) assert spec.asset_key == AssetKey(["prefix", "asset1"]) + result = execute_assets_and_checks(assets=[asset1], asset_checks=[my_check]) + assert result.success + assert len(result.get_asset_check_evaluations()) == 1 + def test_execute_asset_and_check() -> None: @asset @@ -862,6 +949,26 @@ def check1(context) -> AssetCheckResult: assert result.passed +def test_direct_invocation_with_input() -> None: + @asset_check(asset="asset1") + def check1(asset1) -> AssetCheckResult: + return AssetCheckResult(passed=True) + + result = check1(5) + assert isinstance(result, AssetCheckResult) + assert result.passed + + +def test_direct_invocation_with_context_and_input() -> None: + @asset_check(asset="asset1") + def check1(context, asset1) -> AssetCheckResult: + return AssetCheckResult(passed=True) + + result = check1(build_op_context(), 5) + assert isinstance(result, AssetCheckResult) + assert result.passed + + def test_multi_check_direct_invocation() -> None: @multi_asset_check( specs=[ @@ -881,3 +988,210 @@ def checks() -> Iterable[AssetCheckResult]: assert results[0].passed assert not results[1].passed assert results[2].passed + + +def test_direct_invocation_with_inputs() -> None: + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ] + ) + def multi_check(asset1: int, asset2: int) -> Iterable[AssetCheckResult]: + yield AssetCheckResult(passed=asset1 == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + result = list(multi_check(4, 5)) # type: ignore + assert len(result) == 2 + assert all(isinstance(r, AssetCheckResult) for r in result) + assert all(r.passed for r in result) + + +def test_direct_invocation_remapped_inputs() -> None: + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ], + ins={"remapped": asset1.key}, + ) + def multi_check(remapped: int, asset2: int) -> Iterable[AssetCheckResult]: + yield AssetCheckResult(passed=remapped == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + result = list(multi_check(4, 5)) # type: ignore + assert len(result) == 2 + assert all(isinstance(r, AssetCheckResult) for r in result) + assert all(r.passed for r in result) + + +def test_multi_check_asset_with_inferred_inputs() -> None: + """Test automatic inference of asset inputs in a multi-check.""" + + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ] + ) + def multi_check(asset1: int, asset2: int) -> Iterable[AssetCheckResult]: + yield AssetCheckResult(passed=asset1 == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + result = execute_assets_and_checks(assets=[asset1, asset2], asset_checks=[multi_check]) + assert result.success + + check_evals = result.get_asset_check_evaluations() + assert len(check_evals) == 2 + assert all(check_eval.passed for check_eval in check_evals) + + +def test_multi_check_input_remapping() -> None: + """Test remapping an asset input to a different name in a multi-check.""" + + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ], + ins={"remapped": asset1.key}, + ) + def multi_check(remapped: int, asset2: int) -> Iterable[AssetCheckResult]: + yield AssetCheckResult(passed=remapped == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + result = execute_assets_and_checks(assets=[asset1, asset2], asset_checks=[multi_check]) + assert result.success + + check_evals = result.get_asset_check_evaluations() + assert len(check_evals) == 2 + assert all(check_eval.passed for check_eval in check_evals) + + +def test_multi_check_input_remapping_with_context() -> None: + """Test remapping an asset input to a different name in a multi-check.""" + + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ], + ins={"remapped": asset1.key}, + ) + def multi_check(context, remapped: int, asset2: int) -> Iterable[AssetCheckResult]: + assert isinstance(context, AssetCheckExecutionContext) + yield AssetCheckResult(passed=remapped == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + result = execute_assets_and_checks(assets=[asset1, asset2], asset_checks=[multi_check]) + assert result.success + + check_evals = result.get_asset_check_evaluations() + assert len(check_evals) == 2 + assert all(check_eval.passed for check_eval in check_evals) + + +def test_input_manager_overrides_multi_asset_check_decorator() -> None: + """Test overriding input manager key for a particular asset in a multi-check, ensure that it is correctly mapped.""" + + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ], + ins={"asset1": AssetIn(key="asset1", input_manager_key="override_manager")}, + ) + def my_check(asset1: int, asset2: int) -> Iterable[AssetCheckResult]: + yield AssetCheckResult(passed=asset1 == 4, asset_key="asset1", check_name="check1") + yield AssetCheckResult(passed=asset2 == 5, asset_key="asset2", check_name="check2") + + called = [] + + class MyIOManager(IOManager): + def load_input(self, context) -> int: + called.append(context.asset_key) + return 4 + + def handle_output(self, context, obj) -> None: + raise NotImplementedError() + + result = execute_assets_and_checks( + assets=[asset1, asset2], + asset_checks=[my_check], + resources={"override_manager": MyIOManager()}, + ) + + assert result.success + assert called == [AssetKey("asset1")] + assert all(check_eval.passed for check_eval in result.get_asset_check_evaluations()) + + +def test_nonsense_input_name() -> None: + """Test a nonsensical input name in a multi-check.""" + + @asset + def asset1() -> int: + return 4 + + @asset + def asset2() -> int: + return 5 + + with pytest.raises(DagsterInvalidDefinitionError): + + @multi_asset_check( + specs=[ + AssetCheckSpec("check1", asset=asset1), + AssetCheckSpec("check2", asset=asset2), + ], + ) + def my_check(nonsense: int, asset2: int): + pass diff --git a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator_secondary_assets.py b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator_secondary_assets.py index d06db18106ede..f171e92961452 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator_secondary_assets.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_asset_check_decorator_secondary_assets.py @@ -98,18 +98,18 @@ def check1(asset_1): def test_additional_ins_and_deps_overlap(): - with pytest.raises( - DagsterInvalidDefinitionError, - match=re.escape("deps value AssetKey(['asset2']) also declared as input/AssetIn"), - ): + @asset_check( + asset=asset1, + additional_ins={"asset_2": AssetIn("asset2")}, + additional_deps=[asset2], + ) + def check1(asset_2) -> AssetCheckResult: + return AssetCheckResult(passed=asset_2 == 5) - @asset_check( # pyright: ignore[reportArgumentType] - asset=asset1, - additional_ins={"asset_2": AssetIn("asset2")}, - additional_deps=[asset2], - ) - def check1(asset_2): - pass + result = execute_assets_and_checks(assets=[asset1, asset2], asset_checks=[check1]) + assert result.success + assert len(result.get_asset_check_evaluations()) == 1 + assert all(e.passed for e in result.get_asset_check_evaluations()) def test_additional_ins_must_correspond_to_params(): From 030be1a647a67dc25b3b518d0ac74afd3ae7e7f7 Mon Sep 17 00:00:00 2001 From: colton Date: Wed, 18 Dec 2024 21:03:19 -0500 Subject: [PATCH 08/37] [docs-beta] migrate - dagster-cloud-cli docs (#26477) ## Summary & Motivation ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. --------- Co-authored-by: Nikki Everett --- .../dagster-cloud-cli-reference.md | 21 ++- .../installing-and-configuring.md | 127 +++++++++++++++++- 2 files changed, 144 insertions(+), 4 deletions(-) diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md b/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md index 90d74f2c112e5..6c7521800425f 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/dagster-cloud-cli-reference.md @@ -1,7 +1,24 @@ --- title: dagster-cloud CLI reference sidebar_position: 200 -unlisted: true --- -{/* TODO copy from https://docs.dagster.io/dagster-plus/managing-deployments/dagster-plus-cli#reference */} \ No newline at end of file +## Custom configuration file path + +Point the CLI at an alternate config location by specifying the `DAGSTER_CLOUD_CLI_CONFIG` environment variable. + +## Environment variables and CLI options + +Environment variables and CLI options can be used in place of or to override the CLI configuration file. + +The priority of these items is as follows: + +- **CLI options** - highest +- **Environment variables** +- **CLI configuration** - lowest + +| Setting | Environment variable | CLI flag | CLI config value | +| ------------ | ---------------------------- | ---------------------- | -------------------- | +| Organization | `DAGSTER_CLOUD_ORGANIZATION` | `--organization`, `-o` | `organization` | +| Deployment | `DAGSTER_CLOUD_DEPLOYMENT` | `--deployment`, `-d` | `default_deployment` | +| User Token | `DAGSTER_CLOUD_API_TOKEN` | `--user-token`, `-u` | `user_token` | diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md b/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md index ff5280c31f89c..faf818e9e88c3 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/dagster-cloud-cli/installing-and-configuring.md @@ -1,8 +1,131 @@ --- title: Installing and configuring the dagster-cloud CLI sidebar_position: 100 -unlisted: true --- -{/* TODO copy from "Installing the CLI" and "Configuring the CLI" sections of https://docs.dagster.io/dagster-plus/managing-deployments/dagster-plus-cli */} +:::note +This guide is applicable to Dagster+. +::: +The `dagster-cloud` CLI is a command-line toolkit designed to work with Dagster+. + +In this guide, we'll cover how to install and configure the `dagster-cloud` CLI, get help, and use some helpful environment variables and CLI options. + +## Installing the CLI + +The Dagster+ Agent library is available in PyPi. To install, run: + +```shell +pip install dagster-cloud +``` + +Refer to the [configuration section](#configuring-the-cli) for next steps. + +### Completions + +Optionally, you can install command-line completions to make using the `dagster-cloud` CLI easier. + +To have the CLI install these completions to your shell, run: + +```shell +dagster-cloud --install-completion +``` + +To print out the completion for copying or manual installation: + +```shell +dagster-cloud --show-completion +``` + +## Configuring the CLI + +The recommended way to set up your CLI's config for long-term use is through the configuration file, located by default at `~/.dagster_cloud_cli/config`. + +### Setting up the configuration file + +Set up the config file: + +```shell +dagster-cloud config setup +``` + +Select your authentication method. **Note**: Browser authentication is the easiest method to configure. + +
+BROWSER AUTHENTICATION + +The easiest way to set up is to authenticate through the browser. + +```shell +$ dagster-cloud config setup +? How would you like to authenticate the CLI? (Use arrow keys) + » Authenticate in browser + Authenticate using token +Authorized for organization `hooli` + +? Default deployment: prod +``` + +When prompted, you can specify a default deployment. If specified, a deployment won't be required in subsequent `dagster-cloud` commands. The default deployment for a new Dagster+ organization is `prod`. + +
+ +
+TOKEN AUTHENTICATION + +Alternatively, you may authenticate using a user token. Refer to the [User tokens guide](/dagster-plus/deployment/management/tokens/user-tokens) for more info. + +```shell +$ dagster-cloud config setup +? How would you like to authenticate the CLI? (Use arrow keys) + Authenticate in browser + » Authenticate using token + +? Dagster+ organization: hooli +? Dagster+ user token: ************************************* +? Default deployment: prod +``` + +When prompted, specify the following: + +- **Organization** - Your organization name as it appears in your Dagster+ URL. For example, if your Dagster+ instance is `https://hooli.dagster.cloud/`, this would be `hooli`. +- **User token** - The user token. +- **Default deployment** - **Optional**. A default deployment. If specified, a deployment won't be required in subsequent `dagster-cloud` commands. The default deployment for a new Dagster+ organization is `prod`. + +
+ +### Viewing and modifying the configuration file + +To view the contents of the CLI configuration file, run: + +```shell +$ dagster-cloud config view + +default_deployment: prod +organization: hooli +user_token: '*******************************8214fe' +``` + +Specify the `--show-token` flag to show the full user token. + +To modify the existing config, re-run: + +```shell +dagster-cloud config setup +``` + +## Toggling between deployments + +To quickly toggle between deployments, run: + +```shell +dagster-cloud config set-deployment +``` + +## Getting help + +To view help options in the CLI: + +```shell +dagster-cloud --help +``` From 33ba996a7ea42044e91144ff2c298e920cd5078d Mon Sep 17 00:00:00 2001 From: colton Date: Wed, 18 Dec 2024 21:26:34 -0500 Subject: [PATCH 09/37] [docs-beta] migrate - agent-config.md (#26480) ## Summary & Motivation Note that I renamed the file to `.mdx` since we are using components. I'm not sure what the etiquette is here, so please let me know if that's not correct. You may find this diff helpful. ``` $ diff \ docs/dagster-plus/deployment/management/environment-variables/agent-config.mdx \ ../content/dagster-plus/managing-deployments/setting-environment-variables-agents.mdx ``` image ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. --------- Co-authored-by: Nikki Everett --- .../environment-variables/agent-config.md | 8 - .../environment-variables/agent-config.mdx | 262 ++++++++++++++++++ .../code-locations/redeploy-code-location.png | Bin 0 -> 85336 bytes .../aws-ecs-cloudformation-template.png | Bin 0 -> 328862 bytes .../aws-ecs-save-template.png | Bin 0 -> 97770 bytes 5 files changed, 262 insertions(+), 8 deletions(-) delete mode 100644 docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.md create mode 100644 docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.mdx create mode 100644 docs/docs-beta/static/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png create mode 100644 docs/docs-beta/static/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-cloudformation-template.png create mode 100644 docs/docs-beta/static/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-save-template.png diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.md b/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.md deleted file mode 100644 index a800e113a83ba..0000000000000 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -title: "Set environment variables using agent config" -sidebar_position: 300 -sidebar_label: "Set with agent config" -unlisted: true ---- - -{/* TODO move from https://docs.dagster.io/dagster-plus/managing-deployments/setting-environment-variables-agents */} diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.mdx b/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.mdx new file mode 100644 index 0000000000000..29d8d0b845016 --- /dev/null +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/environment-variables/agent-config.mdx @@ -0,0 +1,262 @@ +--- +title: "Set environment variables using agent config" +sidebar_position: 300 +sidebar_label: "Set with agent config" +--- + +:::note +This guide is applicable to Dagster+. +::: + +In this guide, we'll walk you through setting environment variables for a Dagster+ [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid) using the Hybrid agent's configuration. + +There are two ways to set environment variables: + +- **On a per-code location basis**, which involves modifying the `dagster_cloud.yaml` file. **Note**: This approach is functionally the same as [setting environment variables using the Dagster+ UI](/dagster-plus/deployment/management/environment-variables/dagster-ui). Values will pass through Dagster+. +- **For a full deployment and all the code locations it contains**. This approach makes variables available for all code locations in a full Dagster+ deployment. As values are pulled from the user cluster, values will bypass Dagster+ entirely. + +## Prerequisites + +To complete the steps in this guide, you'll need: + +- A Dagster+ account using [Hybrid deployment](/dagster-plus/deployment/deployment-types/hybrid/) +- An existing [Hybrid agent](/dagster-plus/deployment/deployment-types/hybrid/#dagster-hybrid-agents) +- **Editor**, **Admin**, or **Organization Admin** permissions in Dagster+ + +## Setting environment variables for a code location + +:::note + To set environment variables, you need one of the following user roles in Dagster+: +
    +
  • Organization Admin, or
  • +
  • + Editor or Admin. Note: Editors and Admins can only set + environment variables in full deployments where you're an Editor or Admin. +
  • +
+::: + +Setting environment variables for specific code locations is accomplished by adding them to your agent's configuration in your project's [`dagster_cloud.yaml` file](/dagster-plus/deployment/management/settings/). The `container_context` property in this file sets the variables in the agent's environment. + +**Note**: This approach is functionally the same as [setting environment variables using the Dagster+ UI](/dagster-plus/deployment/management/environment-variables/dagster-ui). + +How `container_context` is configured depends on the agent type. Click the tab for your agent type to view instructions. + + + + +### Amazon ECS agents + +Using the `container_context.ecs.env_vars` and `container_context.ecs.secrets` properties, you can configure environment variables and secrets for a specific code location. + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + ecs: + env_vars: + - DATABASE_NAME=testing + - DATABASE_PASSWORD + secrets: + - name: "MY_API_TOKEN" + valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:token::" + - name: "MY_PASSWORD" + valueFrom: "arn:aws:secretsmanager:us-east-1:123456789012:secret:FOO-AbCdEf:password::" + secrets_tags: + - "my_tag_name" +``` + +| Key | Description | +|--------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `container_context.ecs.env_vars` | A list of keys or key-value pairs. If a value is not specified, it pulls from the agent task. E.g., `FOO_ENV_VAR` = `foo_value`, `BAR_ENV_VAR` = agent task value. | +| `container_context.ecs.secrets` | Individual secrets using the [ECS API structure](https://docs.aws.amazon.com/AmazonECS/latest/APIReference/API_Secret.html). | +| `container_context.ecs.secrets_tags` | A list of tag names; secrets tagged with these in AWS Secrets Manager will be environment variables. The variable name is the secret name, the value is the secret's value. | + +After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes: + +!["Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+"](/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png) + + + + +### Docker agents + +Using the `container_context.docker.env_vars` property, you can include environment variables and secrets in the Docker container associated with a specific code location. For example: + +```yaml +# dagster_cloud.yaml +locations: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + docker: + env_vars: + - DATABASE_NAME + - DATABASE_USERNAME=hooli_testing +``` + +The `container_context.docker.env_vars` property is a list, where each item can be either `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the local environment. + +After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes: + +![Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+](/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png) + + + + +### Kubernetes agents + +Using the `container_context.k8s.env_vars` and `container_context.k8s.env_secrets` properties, you can specify environment variables and secrets for a specific code location. For example: + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + k8s: + env_vars: + - database_name # value pulled from agent's environment + - database_username=hooli_testing + env_secrets: + - database_password +``` + + | Key | Description | + |---------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | `env_vars` | A list of environment variable names to inject into the job, formatted as KEY or KEY=VALUE. If only KEY is specified, the value will be pulled from the current process. | + | `env_secrets` | A list of secret names, from which environment variables for a job are drawn using envFrom. Refer to the Kubernetes documentation for more info. | + + +After you've modified `dagster_cloud.yaml`, redeploy the code location in Dagster+ to apply the changes: + +![Highlighted Redeploy option in the dropdown menu next to a code location in Dagster+](/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png) + + + + +## Setting environment variables for full deployments + +:::note + If you're a Dagster+ Editor or + Admin , you can only set environment variables for full deployments where + you're an Editor + or Admin. +::: + +Setting environment variables for a full deployment will make the variables available for all code locations in the full deployment. Using this approach will pull variable values from your user cluster, bypassing Dagster+ entirely. + +Click the tab for your agent type to view instructions. + + + + +### Amazon ECS agents + +To make environment variables accessible to a full deployment with an Amazon ECS agent, you'll need to modify the agent's CloudFormation template as follows: + +1. Sign in to your AWS account. + +2. Navigate to **CloudFormation** and open the stack for the agent. + +3. Click **Update**. + +4. Click **Edit template in designer**. + +5. In the section that displays, click **View in Designer**. The AWS template designer will display. + +6. In the section displaying the template YAML, locate the `AgentTaskDefinition` section: + + ![Highlighted AgentTaskDefinition section of the AWS ECS agent CloudFormation template in the AWS Console](/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-cloudformation-template.png) + + +7. In the `user_code_launcher.config` portion of the `AgentTaskDefinition` section, add the environment variables as follows: + + ```yaml + user_code_launcher: + module: dagster_cloud.workspace.ecs + class: EcsUserCodeLauncher + config: + cluster: ${ConfigCluster} + subnets: [${ConfigSubnet}] + service_discovery_namespace_id: ${ServiceDiscoveryNamespace} + execution_role_arn: ${TaskExecutionRole.Arn} + task_role_arn: ${AgentRole} + log_group: ${AgentLogGroup} + env_vars: + - SNOWFLAKE_USERNAME=dev + - SNOWFLAKE_PASSWORD ## pulled from agent environment + ' > $DAGSTER_HOME/dagster.yaml && cat $DAGSTER_HOME/dagster.yaml && dagster-cloud agent run" + ``` + +8. When finished, click the **Create Stack** button: + + ![Highlighted Create Stack button in the AWS Console](/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-save-template.png) + +9. You'll be redirected back to the **Update stack** wizard, where the new template will be populated. Click **Next**. + +10. Continue to click **Next** until you reach the **Review** page. + +11. Click **Submit** to update the stack. + + + + +### Docker agents + +To make environment variables accessible to a full deployment with a Docker agent, you'll need to modify your project's `dagster.yaml` file. + +In the `user_code_launcher` section, add an `env_vars` property as follows: + +```yaml +# dagster.yaml + +user_code_launcher: + module: dagster_cloud.workspace.docker + class: DockerUserCodeLauncher + config: + networks: + - dagster_cloud_agent + env_vars: + - SNOWFLAKE_PASSWORD # value pulled from agent's environment + - SNOWFLAKE_USERNAME=dev +``` + +In `env_vars`, specify the environment variables as keys (`SNOWFLAKE_PASSWORD`) or key-value pairs (`SNOWFLAKE_USERNAME=dev`). If only `KEY` is provided, the value will be pulled from the agent's environment. + + + + +### Kubernetes agents + +To make environment variables available to a full deployment with a Kubernetes agent, you'll need to modify and upgrade the Helm chart's `values.yaml`. + +1. In `values.yaml`, add or locate the `workspace` value. + +2. Add an `envVars` property as follows: + + ```yaml + # values.yaml + + workspace: + envVars: + - SNOWFLAKE_PASSWORD # value pulled from agent's environment + - SNOWFLAKE_USERNAME=dev + ``` + +3. In `envVars`, specify the environment variables as keys (`SNOWFLAKE_PASSWORD`) or key-value pairs (`SNOWFLAKE_USERNAME=dev`). If only `KEY` is provided, the value will be pulled from the local (agent's) environment. + +4. Upgrade the Helm chart. + + + diff --git a/docs/docs-beta/static/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png b/docs/docs-beta/static/images/dagster-cloud/developing-testing/code-locations/redeploy-code-location.png new file mode 100644 index 0000000000000000000000000000000000000000..c89e91758ea16dae25ec95fd6ecc5e3f14086523 GIT binary patch literal 85336 zcmeFZWmr}1x;8xN5)kPYNokO71OcUy?(XgmDV0>ZySux)ySuw<(%*3Hy`J@cdq0Oh z@Bi07B93{_`?}+b^E%J*O-4!-84(W=1Og#{6cd&Mf#AGBAP53@Sl~Al(|Mi1Ul4Y3 zqVGY)!vuRE&}+~~VL^Fk&BG-ac?A<}*wvWIh6XqD&f{ZCfv#hlOQ*@QhK7cPGPNGy zH%w@l81nmCZW~QHMl0w)4g0UhTfTvyR#!IPpR@dfi9jGSg0^}8JYituRdzIO z&_jZ{STgKCF%bf`wGr+g&ESCo@`blo1XAC`Z&vKDm7r<`#oqk0MP3p1hB&pdC2_&~ z-CjT-3QIv)Sy|QA)va#?hB;qPLRe{Xw=H!*B_ALEK=(6}^dTnG*42f4hlCWHkU%Q< z#ZN4f<88V~$amwhjOSEd&jmwVkXHi!O!^^;j9fB zod}C?Sui3hs*|#^a{IslQgSj;X=$ZjPGHW{()jqRW4u>x2h1sa9$)hE5C#W_)}Hpz z2L}grOidA;ZloF6YewxJ%Xy`Ho9SpXc4x@^sw=~fJRr3pk&uE$7T)>C)y0im4-Sz) z#0NHT57N=v6H13^hxR>26qDXNJzt_38X7XPvflSI-SUZ|I=i@d@riO$fVo7qdN%I~ z=x9M!R?fXlD$4B>1L&H`$)%OSkT)gC8=|rCaeFp>oSzuKj_iOlmUwv=ACopp4xhG` z@35R!Zldem++e({JO&;ubr6(hZouF(Qcdor_JN&|OU;jwWC+X#ec3mr!ZWQ=Pw>I-+o@x@a%aq#~ww3=zb9MPoGtGrQUTpTyK?##B6=SM)M~Nc9&Tb9Ew03al z+CvxpK`qIZDW0;{b4>fS=pE?E@<@A@IRRw{WuEoT0|LNwhFsyEGf1l_HBl%da-B?@HFWyL0 z+C>dD11_c~G#O|(ik<%znFPi$^9%2eq(tt_KWs}Hw(~g{&C*X0$(VtGfyeFoMNcSE zSLEBz){l3r{BRL0Cd~4M$`B3*Q-M)Ym=cnbLSCinzK}oE1t3=&6{o)raC(#8CbdrF z{Na1F(45y#aoC$HU@jY+PEMUhk~usda0mzpzEnnkUv2kC(p%5$6HDT{a?*$iBj#W2 z{Dv+T%ZOHPH1agGP;bBYiiejsoKkxErA5cm5@C*1(zpQ<1{DbKxG#%t-eLVbI9qM= zAp7_e1D9DJenQCkVHb^v&-E+to7X}C!8-Lvr?ay|`0R`x-S!xz)ZW=_Y-?o$C(CeZ z0$%JS?0@S`{*S}wY2|Zb0daJ66mq=u+Bc|W8JSqSfb;#5-}B*Lb5xGg%i7Cg6lbctzGarq_eU?Ra=Pd*pQF zOPAUb6h;Gh&EJJupB1guZyltIwXRzU+o0>q%g*?zg0r0zjTQuIkGmJMRc`FM-NCr~ zR|cVL{ZZ8K^PLO|HJf0`KSjT|a@6Xkr5zp&}b$4?RXgH*%J((*DnDIGVqPsLrXe|8$QZNo~^#aNO6rDmj*{2x#5C|x^ zySc+MJy9lIF#P6F3@Q{H%H(Z6>0LO8L(AvOS9JD=6V71>g0T}6BH};XRnRFf#OdK& z1(WeANbKdIS1&%&+x1AB7*{dcRz&;mHB7>FYl2RwCw^4>yOSAEEBaBFZ4T#5KAL9ifGc?`d z5tD9buZ&j5scM=41WhE38VachERTrE4&e_q=X-X*!Du=m-w6G|TYJj-SOG_AdpiJF z@3p%tGoR$Tx;jn@>9{Xk4*Th!wmxyVT^}m>OC@s=?%#UDp%VG+O=PE8wRp}pI5C70 z^5#!<*4b`7?Rz#hHdg9_FZU+!2nN(Ufz=kWrc0Gh;qp&lw|et=d$2cx5_P%R<6r`E zz7s<<%)b%lGDkYa`T=aSSnu$8hqk(!9{%0CiV~ue$NL+b{mGBWIR?dR=5rNrxYpd9 zoNM4RJ?E4C@97_c-ng7^!WY9_-rte=zM$vLKr*)ziqr980D^=}zP$;V zWc*~UN!|3$NQ!So#XCG!^IX(7FL1f!J2c}iX4=PJm1s7#N72Nza0OP*9YN)sf% zwu>I5Qm=)lrK5W!Xf7zA8BXFxLP8Q(R#OwYcH17r?X8QUl3h>8U+^tALfQ>qj8hNz z?JWTkQr-Xd(_1n%-k#u7lK$wo$k^C10#}(_j@^TCxe5hrrt&2L5tJ=_qE?^r@q_W% z`8t&Z=lN#}WeM3W7h8RN152IHbS2_B)(RAgnWfLxgaR2o?pO#otOa&lqe{rdqhDl* zgtcXUL}%4sDz(W-(W~`*N@tB_w$ z=J(RvdGU05Mmjh&IGV3uR%x;TTc+2;bgTxstx{oVx@8YsJ64vcQf8y!MBubDB4Sb> z?r>y5y0tyr;xOx%Dhl;{#}qFSzdhe#6((eNq&2rVx~xSOnx3n*A?*&uE%GUlmuc~Q z#KV1O_`YI$Xf#(=Z^=nU4+t_ieC9autY+Dv0VMaoHg0<`xk#(LGB};Ydb&{MOzGx0 zSGi2i`}O_)R6%KtmCyan@!pwuD~*mRuy13<9&Y??XK6xk4lS=h;`i1Fg^!4N<6)O_N@!vsLq8d9sq!h;9xU!b^4=x7$5mK6nzdgh(Ed>gEy4I1aR5Pp; zrb?}rBb5+3UuBk|Iz_GOa9>5THtQQ0_MY|>e5Qx-RvrHOnvThO^mS47U<7xJ;+5r6 zHKEPAJHls!3^@LL?0DZ+UMW%~wLlGDJ;cV>s=w%;y{DoL|UEs^#k= zJi4uTKFSuWb8>LEbd_cFuoy?3h%-9h4bEvpts)=HWEhPki$(Lq!7hD_AQPb7gzfCB>n1gUKT)^WrRnT~G+Kb>b z+~>=qpci`+q=tpc$~BgRzv3soMnk&O315Ao-?lB!D`;$L zWq6PK>ko;wHX#ai4UR|egSLk*=jl}&g=cvnb_D8}>RVrz*dFVvQluVFPGBGqNQ0Puz zvl7gfhzm~mQEGRd`3;9j*Dqk~o#8OEr!=($K3i^y#rLAZ!cv0+wv8LtoBgb{4Ipcg zE%Ef&m?57%7|Y;Rq}2+owb@wTE!)5Ydsv92n&m1c^LfCU&fLC=7w@t-)BW%R_-CO? zgJOYjmT9%uv*&!j>NeT6hCdKq6=!RV*C?$$uKLmHY`2B`Bj1*}jc68GvIPUDGwuDY ztt!Mqvj^M@91fWlatPNWRn{ura`}$_`T!tdgOblz7F3gG&?;l@?VkbycZSwV^X_aJ z`rXw@NkaIlQnA0^DI6-{wH*&|%>^zDhh(xoVGhP|OZz(IDG%R@q9A7}m8b+Yx?J3q zMenO33^<)EXK2M#4-dpJi29#n$`rf1@E=|sO!?ONb1D`p@9o^Fscebn{mfz!yXGYW zS(>asV$vvfF1lP?si6RMUfFvXULsDNxY`d+A4gC&D zTeZ9fegIFbwni&rNCHdS=^E*KTQj(Z7it1<;+-&_0p1A-G>uJ-XuKl9DP?ZQ5x);T zT97OAkgiV;x2iUKZiV+wO~btSli`S?pD=0G_orw)u_9hGi-r=kY2qWy=gSud1D<`z%$Zd=n2;guV0az9PHB;+542$h*gTr``ze z8pR-@SuYUNLlG(6S917B(8YOFfuJHtyudtQuS24h17q2Nqo zY3x?-h&^0yrY%xhtYyQzCbA`sC+eeyts0F+mO*xJg2E;aX6~$8XSPwJrxzKt8U9sP zgc;=ynMJk->Rfb8jNby^!t?Ul1Y|vqO6)sWb5><3VISC zV$%cYk{z;27G$l;#WvuEIRah~1p~*NM6qxdeA1JP*p452@O3&uy$jTGP=I72fK)ULw}rk=^4qJf9~2IFv|43vMh_?yvYFvdw&~O^ z@5lM-?DKxkk1Krmp6jkW1HGH<2Hz5eyN*PKlTBOQ)Hp1^QmS?x~`UKxZiFU{O zVjFp}(Sdukb>d|y5r^|jm2&+Lfhb2LblE*&Dc&L>_;9n$gU{TCgT_FGq`1H}pj>VL zw%+L(X>TGYu&xd!UAS!0#Vkv8j&bG47w*PpYrpDsL4rZ@q7M{oszrk%4ofekyX8tV zujA2TS`P!?Q_pmfTGjq_i^qL9uNz8IigH+qh1v_ELp%32Kg5=chc{$GashM)SmL7#AFst$I5;2vhj#%urpQrOOr@oZ#3(@OJ5oijZ{M zBlmQg+$_U>cZdXhWZ>P+kanBgZc-PK(t(fjDSd(%!528 zt5l$fnIoMv-YPx)P9N>(h?qCE-bVHim2zq7lgf@EopA_|K z@vQdA>L&aIv&vPUAWw%?CDm)Kb&_HnZ;qGdxfl*rW#!hXDTNM8PKH}MYl}J{u)ySfJsM4ex^V@s?{{ zXj<3AXXzkHc~)Oy2|=#@0-E>*+e>HFbF0ycAxax@wg(dfiqzQfwPw%Odu~47tu%HA zq7q-NMo@?coT2Bu9@^17P28`n&FXH|_ ztM!Cwu7aubQ#JS$^TnR&ah}mnpBb)L{?XH7vQP!R!f*uPdR)EHS$3x~sw2pjuwwwZ8xdOF;3G9bFGJTxZ<`}*ld%iBZD@dvk(R1*8_DC8ZGMd9Iz zt8HsE+(74oqQ6g^Z}poBP$`t3t#{q;m#x&=2qghUMd%vomt;6aL?Ut!pVKixv3ebz zal#71MvvZP?vME|93Q@8>`vy9P8X~AdOqG|0H6p&XvWU-Q(#vRcACj&{0)NI2FGK+ zq@<$ZKmz3i!wwPs(2dr7XXKR>4x2Y3p{EOws@sOqh#LbjCB{e=&w5AmRlAdAzGq-c ziCAx0QqffPx^uj0IMz+oD7w$8LYjG8#Xy+{T>SvyZ?E=jP>9(u2BPUE%+}qYryy15 zeS>@|3!_OChQ=Y)9-z3Nkn!14H5SzdQyC=t8W^5kt? zhpzBPV{Iy#$-dOvwg^0J*d`?axi3|B^e+E_iObyV!2UW*S_6MI(he6e8GKd~kBT zYDs-Y)V_TJcIvc+Xvx8IZJSONde$+Ct7`2OT&CK~YkNJ2E1xH)vg5T}=@N82bMrDq zjom8Ezlk@Q*A3QW#=^LXuxEv*R-r(#ZHe0nvIs=4l1|nw9lL6@;G+N%zjC@VRbVoo zwVtgp$6W>Evs?NFVbcq27fQvmyruE>^_cSPfV5g^7HpWlb_TCPNDK`!nqE3%yxp$J zpFGb=u1JT_~fTx#tv-^L&tvpZA0|zZi+_u8y<4rM5bW1`J4_e`KMITt-}W_ z_7bh0jh-<3g!03{FygUNOFHCud=}H-;w4vT<(spNz*^_CFR&AR5jg_$_Iu$hCgs{% z0^=pkQX%>EO%!a7yR&r#DHa2$B?mLbYu85!)Zgw~p22wEw-ok0-*v|zzM<0s2Jn9X}fGS$1G1a>?Waf74uF7aSRAC1)3Z4e)TQpU4bX13j@b%=)ZxY%ka9A-<$ zxMaYY&C=OniC>fMDCFtFARp~8D0>5lo{1v)z(5&*8xg+!nbX>@Q1GkSGz-U zxntw+B(Rz-4h&!5^T?DqgtfTo9BfO5V<8e9AKyJv$!4s3dEJO?QS(_;AnQFX&MDXH zr(K`j@R9`|kJcf5kD$x;C07gjc&gl0po?|KkrIz6h(S;U{vr$Q67+!lZv84Ir^x z_A*eP-=1Gwo-$T@vn8sfEH{;zPJP-Oh(X;O&ziaLxZEYt?fm);z?V%zZEpLgUISsW zIy38Xh1YF;vk#q0B|ow{!O`^!@mHZ~t+;&>l%fBF+vh;B%H6&5T|F?y(jas@!lOV5z||EiZ(w z`!*ixX0tB>T(FShe6|h)pcB6KdlP2YRaPr}Jg%4UKo}G%P%QFoER3L#?EJdssU{$9 z-C(#neO{}!;{4_AtA8AfD*WFTS`g_heI3mbZ^CQzBN)(RHU)e zus)d=p( zoR$c~TO3H(^k0BE>3~bu7;8#LU#<*KE2;^Us5W)ih7K+)HfwMztBMF#?N9w6`>0Fj zxmc>hJDw$;bMf5YtS?rpmhQ}iT$sY+fz=uWG=$bDJK_~+7;PFyP@|+Yo`g5Dt9>j# zjBfEGyx4jUu(li8kBVjSEcBF=yDQ(K5y-ayirDOhkBMXm+w{u8&?%I1(K%Fj?72{R z)X^lEjV|Zg9c83}SAl`h3C#ObR$nc7UH4PS2MJhG8LkiK^Snlk6hJUYnB$*jBYHe2 zS}i9#VUrph@AF%Out#JY>PI@4LgHCI>=b+LrI0^ZEY!mg@t)?@4n>o~*=`T|CoRhB;eP0tb>^@kLNVP6o%{GcYO*!BIb6L4^v3ak7aRywcYM7;y(;PA@bJX zb~1L>%(X^~r2dSRBbD6x^n7iaO2f}VA(@o7pjxGii$xckGu6e6$71@$cq~0#JME3? z4_ZrR>GDB1nH)_h`{$LF6fJ_9u{t5OMvn`T8?9n@n9lEn_zN`_Z42yHI);{^EpE@g z`BSF5=iE-_9ixI!mm}qkfiZ!KklZal`kFlM>IYag%iS@@vu;dq&vw^@Sn{>ho)eiJ zKF!X}bxaH(4aCrn6YJYr(W;c;nRty7jA=F1KKz!&vr35wlx)TwB;wVh8KO?KsD@9| ztx`6(-!Z8^XPbW=_B>1fuse6*rCO$1sC@Q%<_B9J3JMC>Nxt7VW-{d@COs|+$wc_P zhbVP{ZM0+#o1epb)JOhVU#4%u_ch{B9oO1ny4~}D^@wG5!21+Moi~9>%HjKegR`W?{X!{K!ec9gr~C9C1`e5eJ$p&b%Z*UMc1wGleudxoALx&n@% zE72>l{&1;0C^+P4%Y2On`)RCEpy0Z!AnvOMYJ2e=$>;;B`=Q9U??i$LzUe-39m6_) zvCD*Pw9*0qYJ|tTo12QSe&gd5_x(`=@pEE2X(1tWcMliR-fSb~>jyI>XD!cHtKf--!5`ACpJqVFVW93OEhU1LA8g9O!7Q=LgQaJsrR)f{)da*s+ z{=iw`fDpG(qo2o2vOEBII_-(v=H)>E7wSwWr$J*?Wzq0jS?0x??~fqheCqwCVz+aE zT;?(X8%)3^9P`85em!zem71PiD(AY$W}|)t9i~}m{_457YgU#o!u1|C51Sz|aPFPa zZV128(;NMv?Hd?N4%`*GRExv-43D1SJ30hgRHDXG-?!Q1F;vbbXJRG}v4eLB7fBeXO{DX6cMRy1Z=&RJ@{) zjK%=)p&0;6wO*((`If+5O#1O>W^9A?x3j>7&NIDI!>Lb6P|o+o9;$LyIOFANU(Dw$ zui~r53Z%6jr@!j>mKjWlh<&inDP9^))j@x}%L>{#Gm4P{uWkn<*4gUxU(VfShC+A9 zLD0@fC8NaiTK%YQh`zw*vkzAU3h$Si!$RFZKOiyM)k`CNt3$#jRjVerGkUvawUCed zlU%y^9(jmYpNHwm=XW*GX%Hfh%L6=nv4-H@c~tZ5i%crX?f9rs>Gxf$0hR2UWwud@ z?EyE|_V%4kZcYySpM>1jS3?Dd_l}pidt;dpIS@akA-Sk9l6IJv<17v@#Qu2^_5M8q zKRNMFMAJNY6icxW+0&$l~_Ik=%IGk(_fz3VdC_e$EFne}{Q{t$y)r`4#W=UQ&`ina}E#RgI zjhejMF$UrNwAnf*FM1h0J;KaXdjff~%cQG4y$B=rTsL)I`4+v*eeV^rH@oGW5BRLb zZ_}1`JU}J6G*_(!;%Wa3AZXPUD)cd{)>s5?Uwhv7m1wnK>51OS%Y@#H@N%#H4iqU4Wfs}g#G zgn55iDJs@j`{H+q*Qyd4co}O{i(4AO&mdU^S5Dc;z~HJN{FLMEOC6XuACPvT?yu9O z8B*#T4_M9pfOaFpDAGDmmdlM3zY~$@{EGY8EeI8N7L~i&Y65Y3?gGp7ZQTipMF8@} zAPOEAfx&=+Kwmmc)%$AiDpm@qGlXPhq5d)a>7pzz3thdS(4^DnP(w#0}@uq(W$@a;JJP{2;BLs5SR`5qYy6=yuE10d0Lq}uP9-@nN9!LOyIMcuM z3*3`lf$)TsU{_L_Wwpqhsvlo=zM4mlWp`pDV~HWT*vP(d8h*N=5iF^c)U2P)=~tcS z>#p1DQ5Oa4Jp}&Kfq8X^s>;wXN7I|Zd8+$TWPL{^IK)%Gs z>kg$gZh~?wOV3TC(~Gsl-D2mu2#04hr`&Zj)`m(mJ+cg@(%k~wJ4Urua^$K9P7XwTV12Ec@QeUrkhF~A}5L;rlGH5jm00;eb(-!$=VvRP!dMC2-Ighh8 zKLMY;42@Duv2xMHG#-tDq$HEUoy2Be!V2o-WnM(7d7tj%e5HyVP~q^g#m2|u&TzW#f}W1gVxE>>f(FQzCGn>xG`GgZJ;x_;xr|Evv&eZCmsrK3mN$=& zQ?w0D17>a3pnMPI3Ux%sJ zF`5lS9xgT`>yl&u&C%S?HO)3I2-U>AuJBe?48lQJLifYs)*3)_9=VA|K0nxzkgIKR zkU;K=+wl@U$xI1yb*o@HKQ9=7e%Dv3%=qA1;n0pQ8vDAAgrtQ!8t1e zJYG)(4yN*B4!|(!m`%++;IfP|%OUE=@p1*TA8Zy}<}1~>9k8h5C4Gw{0LkmRVt58p zTSY9H<0nX2RamRR6{f+_e8x>kWX^Vr(Rnu4+N;ev@V5lB6!O!~H6}~-reHY(npVFF z@~6Uq)H)_~2m}AZ$hN&lM-lo8jx=I?^4ERcnay2YT(T z!`L5;qgh&7iX~CdVbQA;7r~+&KBIcLJr`ZH;LW4qD@KOPZ2yR|9tX(T3r^30xw08; zj4eg9Gex4(lLad>NdQBbW+*-zEZZb5+SBmu$Bu9V2(ic@#7Ey|vKwJ|BbzVGnn- zx4&PY-WiO-|59VxWlO+otK1=yS8JiW@lN!psljQqzZsXWw$8-*PWss={Nq4YyVLSv z9M?%5yI+K?ZHHtl)(hQQ0rw$htBPR2L+fmo zlGLXFa9pa2om&xCG-n_-eJmVadMvu0RPH1w|{TroY9&1O(Zh&0Pq_c=~70x#L2zOT`fKr0}2T~yz7qWyvYkmB- zq8A|lu2YJE-=X;21H|9=uTS^4kkp%Do`6g z??3U&=gSR^c)F*2t<)|4oRRO@H(ks*?{HY?6}{2uvH|OAfAE5ml2SvfngO$!xj(wZ zJGtp;?DYcU=7a0iVQ5Am>Rt>0O(Mn$gnekAQnZ+F?=JSY6wMEAa5(yEXEeN94b*lh z;saEtr`Dp?hjTXPod_0Zjm~q_0MPcnhBDhit){h!+v)Noz`Duivl?=~nLau&2Z#lu z>*NxCiFM}>!#VwS8LbR-)@zHnjb1xnApO6aK4C4BszYga^nLtr^x#raX42uBbfX9< zj&Pg#ruK&a5e;Ct3lL_~o%;dq-fRa2*~_3am=M2cp>pE8>dM7_{|y<22BQ`Ibg^c@ z&B>BIG5J8G;6gQFrK)24O#Ky}agg!K#`F+A4v-~Rt42SH!N;+<$k`u1#K0!nHfrII zL)L2+?bmB&P)Lr|6^y8NFq1(?bDH-A69mkZXqhq6k<=*oKsVv z!#bpJdJotQ%SCmKvzB8oQ-`X;iBwrV3!$T3@T-vhDa8&|hkaZCMf`5NNE6M_ac+Qg zc73I$SF70~t5<2dgk@;B-F*b0yJeT>=abay=a@A2oL%!%G-j-hd^9u?acl!%UXypc zhL_grJYLH^eUlhA8$E#`MTB@P=C3wc;ZLy9ZbX0QPvKD^ZW|wFfB2r-JBhBXlk4c{ z;1V@#@lpM&>3#=B6Gb_ic~Rl*uqIQ8>lpMD37WMApZk8w=OuTJydHdY?yq0l>=ias zlBBI;@(p~34wm18Wg^>kaKC+pfKKjL=h7?Z$Vq{XZ3JLbifR;r1`Hkufh&zJ85a)} zV&NbrU1+U4I>9=5?4D3ewVhdH(_Z%h0J9NU@BAj@-h8rLXN<$J*6{rL(d_=*+Nq~v zXM}8S`rtL@xDIb7>$P}K%vW?udflE-!s&#a>;Hq}{=a{0iEKp=?D-{zU^eUvh|H6F zmg^bz%@WXmS?jQYM?l0YJk4)QVluAJzZL0a3VQfF9LxPD8Awa6H%6&Oiz|XuiDsJi zkM%aJzQ^>OzK-lxhPS6{zW7o^-05vr+vc2}^PGU_=bvb5n!GDxd4Y0YxBGjDy34tL zZHMXZO(!d`A>oK(I~zzlQX3nAsHODkhdIskxS+txd>hW+0p);_;iVw7X0yA`0QNA6 zNH9)sG%2>O4PF<#t${Pn4_Sfi#wM2|YCeylQG z?soRT+2~{=(m!bu&gLNkD9|3xce>D1TT9-o0y?w*Rvf*gwB*_BjRJ0k@a9pB=xA4uk5SOk(E+d>P~s70XaCZu`2;q=0s1!*Bv)7G{ql0HxnzHT zavmO@*LLU!*NaOK=)0c!ZrKzRjI8e_9(%9#?`XyU3Hiy8Fb8Z}oQs5qU&0 zi5YQuah@j$!)FD@VE54GrLB4Y2Lg|Vd(NKKV}F_20oI!r?HZwNgELL5JsQV!BO^CC z{7x-5(QO*Xlo6#i?CckHN>`rji)ywlaQ5u)-UTKcxVeK*(Lx!tR_7vm>ZQ^%KmK4{ zAa?Z<^=kuhyitcCg8E&70^k5(5eS4{0Cn&Q3PK{#;_JD0I$7J>OG_hx1I|n}GwJ}%9Du%S!4Rf^{Iv=DnpKwVDj3%n&d-io7G7{&*p)$FAqfuq5mGI=LwVl?9zfb zO^MeBpy91YbzF36e{&tM+|jmt68`Jh6PF14561pWA?70k=v7h3 zaj^d+@cEZ*FJ)nHK%$d09UFgAX8zaXp8(Bj8uZ5c9~S1n5STalfXw!*tAVrnKYINA z+g1=qq_%qE?~uZO3x5QJF||PivL!FVg8tFtRDhjFeVq^1rXY${VqbYbpSPM5X^U(jJ$I~zXJ3lef|F&lM*UkfTf1M6& zerT>zK+lKYg~Y!ej|A)#Lg?=LgRb(w+?cRDu=!|L{in76HvHe?{Ouz7$pLm!wbgR` zqf`5p8rb~*cgX+$XZipBkV_=hH%dxKu5E3>0O90u9cdQzjz^{x52i%}&CMP@JU(>C3g%NMw_l`DZx7>K>>svpg@r$?Y z4|{cdd1ODVia-Yl{W$}BBeSJoLjPmWrH=EJQY~|19(WEv%$yx^9p7BtZdswfhWV~w>h z^8Gc&*G|aeEC&ViOUU8;-p-&9U7 zlK~&o@j2oCKLO^4gg0jEkHaq)_E4CkSfdE&?URGS@Bij&WmW@!*}K-c2=O0#8-Mxc zld7FKFSK9NMgE)5{q1zMk#@x#+g$2>5KCcLtX}-Jq5qC3Aei>gzy>*++G+hY>XB&Uvg>Ai3bI(}jWjjQnqb zZ}1xMVJ{9Qr~l05O$kU%Ky=w`VWbuK~!19{)1PM~rH9Z4DwvsRT$1+Ra8jU*m%+ zU|3Gz|8BX&UoS$gN~iFxA185(C9o0x68~%NU{l=J+3!LAqOa+8iNBFbtko%(wDkMS zvkAlc5Tk*-z^egDp`p&#mv7TQgoOg>u3$cmq~y>WsJA0$*ck8w0GzhcP*?ZHqW+SR zi3uU~>}&&WvB3@d?dRnGrc0e^rs3a|;=%E^(gn&M7=tmK3Gm2x*Jp`*hG=jLz$ zx()&h6%x@PKA?;iN#wA3NuzkUioNp+1N_|^x>aWi(sd%h!P@l^PcnYCo%vHF$j$xs z9AVt|wxrI(j7&QDjr>BC%>39G8Ui|na3qziP?e=MQi*ak z4nWoR4beanzV9MpK{;Lod;G6`YOO)^1E4~?mZ4-$On|l_Itv1Ce$$(+aHBMxDh@p4 zd}55}^Sp!y@bx!J$?gv_YicJf#s99T1u}uHK-30ayg{p04xgNy9Nx;V*$8&N*k(=U zv1y&6Q35ofxSIo7t-J0Q-42KMR;csUCzq1fnwrTtobK@bkwxC`{e#>m*lJ(F2m{Wo z)D)u0WuwJ;Z}%&pKZaF3yIb$F0|;8^$oQ-u?q1+_96}y0G~ff(u8&4Vv{)RqJ?iCh zgBx6d5}jy7wxd`Smt-tqgt-``_Ah94DjO;t`qkIKz2RD>|7|(Lo0p*wrRatuiCFm{ zFwp6@cJL^{#FowQeEetwi02z!j-UbXNwAvJ&dDh-DCixa*F@u%BH)kdU3x?hCE^Sw z4Ma`X&7#Hw4NU9-ZqlaGdBO~|k5E1|-`qz)>-RaKXo zctB<^ap&Q}+k5iXcBKFD{_YB2SydVOwjtsnnKbGeprcNt&f$RAt#DU;G;cj5=7rRh zm0vrWx3n1b$qP7HuC#X}KbtDp#J~iacunbBo~L|MDQCmEO^%4kO2e9B$hzo4%a4uhYC8NFr$!CoW4jHe zOVmP8iTTsD+uCOvjIcAw6WmvS@VYt5=P5=AGHNCNJ1jg*L+Z>ysxC(?HMzmDISxVj zNi?IaG`sk6R^$l1SLy&7_zCl~BsM=Y?>tZMstRVS?C>{zgm8g(2?%UaEBx598(2i| z3EOvLUm>;IF3g=t{V~kX<23w&#duPFhwzHGI|SDUXbY1_&fDJ^o-aO@hXDw4@#E-XacpV@S%ce;^Ez)W+!^ae*)IOVx!OKzZ}v<)lecI`7ENg%lZB0nI4DJ zF)g$Hpr0@T`dXW>TdmhK5wlU>che~|+!Xj2MP2;qk}1?fTd@H{bo8Q2;Ds(9inttp zFdLXYdY=sDp>sH&f|&JtbBks+KVCODS^lDG0En(Bd7!ztD)c9rB%qbn3+EnJVK{|~ zoi#MSH4rnJ@8G5fa^K&yd8lz>m&=k0k#N=X(K20P?!3D{6#2QhErL?| zC8?oArdTp0lWun_AT+c&Io5bA?Q%+JAJycc|4)c{Q!k(lNc6xoCq>^e8S+lhkuklW zkTY#>`nQ5q9l*dVFFdR$ro!gRcSgN;_a-Rk+1OOdH}96E-mAEvQngtoI1*NUKb@PP zbiUk$a6Fm+CJ|S@d*2&drnl!NtRe+z*sx6pHR#Xfc_=nJ%Qlm$(q8fbX?u4u2jEyv zSJ#yn^C&ClSVCor;nuU&qdlr#fSc)GLzFI>$$fij>KKKr2YMzxd=jSS5KFVLdlN>Z z_tsB~D&aHf_HTYp?*aHzXGh?T&Bh(;BVmDQReblp%I$p1f#Shq93VW?kk%vy$RneH zmPhe(qz3tuu6VC722NxEdi;lV;cr(rr9EN9f<(2(x_UVSi_H$`028Vs1b^lSLki|A z%8%*y((x>~n}SOKXus>*@+*)pI-gzo%mEtoH{#RgxpGwjmO`OxElmxtqn_fJjRfy5 zaY=UePQgE(nkid^s!bM{Mmc}9%p@HA?<(H&gO4?mH*(agwjl`GqAsaqo>1Thy%Fjj zG^K8t)M~QDO<*e)?dX=g@_lh@t#j7yg15|8WfDM@q9#1>d7mbR;g|kcwnkG(LyNEU z^!i+6T)!`vxL&>98So!gW2lTj$t1G}`T|iK94$qUxUR0C-gnGa8<5g#)WhFLTU`0# z=Yf;BTp&Niyba`Wxxj}q$pg058DczJGRTzG^kDy-WXl&0)jpK|RzN7g9lpZifWqC{ zPN)esq+`=JRqJefgX!O9o!%ok*EX2BDI$0@JN3G?FVtA+UUXln;!F%N%Z@W}(UUVqH};Y^hNh({0SReIM~AfD-Y>?vO~v8nUxB0Yp- zjXKg*QqMdyLbqPwvxorww6(o8pASeodymS^?&KI$%DuJUp-axWWp5FqDmQ*Y7rnv~ zPgSc{hw%vM3tuUVrdNUQiD9_9!RzQik}4snBD6clcv~r_+)dLH6U%KxQU5U4q5Ol! zr&ykd-BLGWfh`!&L}=d>6B**A@QeTV zef5=qnF{%Hab>!+YL5Z7?iB>m>!sF=|LJjn*EKo+IvpweZu!JKy>JHS_BS)#KVj_7 z&Z9`VoQ^CXTH8tw8#&+opm8F4v~CC9i4je&`PFud=CxaBBFRtYO0Q_t7rzqPtpTiz zw8~@g6u}vGchH-*G9DXC93Z?*gMP`V<|-(an`}9`9Bv0jKHrpk;e=pZ&^&u;E^OGJk$H?)_he9tCBj@-tB=i(G>PboA*H!xVRC!{pJp5 zKXrpMNgyqzp>D($d;Ssi{wK=1dj|zDS=@kbA>1_1-vW~El>hyBK+S2~*Vg+eP42pP z`A4NHjnsff;rrg_-y_-j*Z?+6C2m(Z)#`NqjEL7Q*qQ#YMzcC7h-#t5_LnTX>d9JD zn{HWsO6@LLP`RutaMNd?njBM6)F3?`=5WPfD=pvNLfj@?2bXI!e)$22X`r6_YgQo|qZVjD7 z_Xov@!^SVLXA8XC&@GFxHt(sQ5%asj7#kaVmTw$9AX-_Ky_T=Rd4~O_teIH}BB>a- z)S)=e)+Ug23DLc=UytlnPP8T_%3v`lq;mu(vY~HnR(n`mybp^|1t}$y(Llq)sM82R zj2Ov)xW*jo;AFu=QYYLu@LS&hmNqP?dRps3@VohPNr( zv&O@7CO+h*DbCaZ!j(kGOZ#MEawrU&LAFKMR0W$hAf3Bzn+@ug(Dp%tlU=#Sa`DZG z0ME&sA3i1c2C4f~hyD~G8p+SWp;;CQ|K-$veu3z5!{1%Y= zluO@|qF#N&pK8!eFH`@n4O=lls!?Fm@ocF_%HqDJI$;%$+pPg$>n_unU;Vr}+sPy@ z8u;?E1N1eI2%Li!6vU(3{_ofJXbfH+3gvR@{TW~7xGpWJ^_MSEsdUgjCg*4 zGE=KH9mX2*p!h44JDPnKpa?;G2%DE~zsFG-!|mJ#L6>eVu~qBz5(Ly?w3&-PG@I>k z;ZYBeMxfM+Q1aF-NA)Pe4^H^3dUIUG_6n;r#Hst|@Y`-&HX1w}k zkOwpxO3qP?40M%VCl%g?(swbDk3j2{p``_VFWd&M(KVLM z(vUNpNT+3lvo;TRoGcb&!ktRj^Cr`=A~rTI0n+Ksq)urJ^KolL#Oy||PByB4VT}N* z)iu_;s=HY9e5G4rY4eQWpc;wchfH`P199cAZ{m8kcgYpa z+94}E2AyZmEY;apLpt?62ZbXp&W|oYE&&h9`MR1L^bMKLC8f=Ra3F?Jmu!FN1zoVE z@QHf#*FWy@e{v`>l2378bd3+zZS_|f-OB?)wU?y(Z4j8i>(PIFc(mC#ds9XllweOY`dI1u5W_YqY)C{`FO;cBj@cqo~$gDaWXk>*sNh48-2f4bg19 z)wjV^K6@34+qo3UcUpx@bgES>p;!#2EidoteG*Yg);n7rOn>j%;_;P4qcS`UFR1!- z$5_Z+Enzs(r)MWqE&ON ztn8-Pcx%W0urZb0Qs6i?PqrvN(~*QO>#;)S#a9WaR|5ZfWr+x6VvXEdS}twb$QZI2 z^~Fy=a>Z)8pB)JT{yt&VE;GLat#-E%Dg4Q7_b5Kbw~ysk1qOtn5GB1GFOQXJvoH7O z{h1B>bA@|<61>T1}G<4)m#j;USQUhCS7<;+8|MOwS(eQ0;sMy}y*o%n74GtI8Wu!~LGivd%I zttnWLxs%&8-@yp`9c%E+JjKJv`EV{0v^J6}(>J#H-X!c53!jy$Hq%Q=4Jy1L?$sMg zW~weCr_JQN_Q>GzdU0{#M188lC-ZGwIMUr2(^&Yg_5Sy&|LdnNW-uG1N*p8sIH+y6 zF;ka)FEe3?Q=rr@w+}?aO)o%Ba zJ1==9uemqabs}L_0{Ylx4DqPMqh(b?-ftYu)*k{k3nxQ{{iP)zmcoy#fPzE~w?l83 zYMLOnkkoM&&^H9Fy^$xgfveD1ylx|RI+)6@3a9V~xhPbZ#|OF(@w9nnil#gF%+MTr zsnKH55QzVNb!kG~^g&>~J1L>m15{I}F|?}B0Pz~kVw=ae*~`NOny@h*sq`Khn^?+P zkt7CnWNl(?ide;BIZTjx^lhDYP2+JT0O&{2A_dcUn>*0c`bsicHsoUpvd*?p6^cl3 ztXex@B0KfQYIVM3yO`NiT8Zaz_qsVlA2%QSUYQ%DJN~7y4+5|%rMjUcEOk0;|9a{w zG24sfoX*OAS+)5S*BLwG?QgQIVKCy<_&F?_rAqN$VmBM7Vv&N&^+{gSrbwFQubReo z!YWGWs~=I;I}eXn$GJ^O>FMbT&n?hnx)tC5)@pN5bo4RgC-*CkA49*)L&jxZdGMrq zMAqT-`cX$`CX>LVFWMDFObqf{evOTvn=VUJSWbr_5_vNk1Y12%;Q-x=;x}w>MLE*6zVUzSe7Tr@)pk(c+dNjKdf8Eih=!yu<|Db_gga zmrMKmd0v!`lfoLTjErMy?e*4+ML*9`4;SC`o4DE#BWxE-bT_%e%Bht&@G448^tt!Z)sTA4;R^E^yfCt$}6S8l24%C zM3&EeEb%4X;8c1oDEa#fQDH-Gu%UhE+MAIaNA7c)&y%3VeR*#M8Tu;6q!ME%6F3W5 zy1Ds9#HOIXxC?|W3F4IBlS2w|0pDo|h}62qDK=aDNVA6VI#Le6CEJcJNYDR>MW=zR z!!VI;&rXBf~_m(=JIx-7F)0lfqYz&JK4zx%1Yr5_{lY# zUb4j3wt(sIk;{UVkcL6np%U%m9~>MM@3d5b*Gm7#1nCTOzkGBHmd+$>~Ct?xwv$@}8ejq;9UWpUh~= z=vn>0Pw~GV1wAInXC0L;#~coSEBOpy?XvBRm@P}QkiS^IIz&OU>#`H zXi*aWKZEE4vjPFu(N+hY6ZRkLSO}90N$?RP1n%%JEP!(0YN#|68N^rd&lqrUaHFoO zT4DcVH1Q==Fo`QvO&96ZKOg7#gGp><4VZU1JYCZ^P1kYE-&^7DzX;5cf%#jNQW*>V znLkq~1YMT?n4JwHVD_iW^2T)0O8hhM-#g<$7V3r1W zNHu{+(W@4g|8WoRe8C8hzF|Xe;{Hs_0Ug{p%`dL$$&ycB?&`?F_P@T|{prhFS=Yn= zd^t%E8n|=es?Si?*fFwT&#VK6CH@Tj_cwG9i~B$w5jSk5G5@(HVQfev2r+C^EhAtRggrs~s?$_dRk)zy9O({=*9c z?|46h^arg#SKlYG!Ol2JP=&v9hJZWC)n@U#Yc6Nd(OGhq6!#1m*UsYmzQ>TJAN!zy zG)$nsk_=S_*|_Ue4p$wy+=l&=_w_De)7-gL9r99)KnTPC`Or-}r;)T{_C)TXh&d{yX~T8_n@^o%f#~Kv9{N zmhS#iis}6A;4SY3KezDK!m~qTuC(E|=y;f!gbk=j2Yjp5{#M@J4*dtj2`D0KP2Okmkcgu;Q`;v@4$~h1^ z{T(j-d*h(JMfgcXVp?Uu$i{Yoj|RJxT**iv(uDv{37q;x4=mWF#TJMboleu)I0Ym+NR|kxXQwgRRZY|i|VszRU z(tp+&jK&nJ-#Gcqy;&mdJg|M^c(hRPR5P@oZlLIy>MoV@)QMj1LeM=e~1 z289gm={ep=1g6F~F&6XB5$BsM25$d=I%eeS8$_^5H=j>K|7-D%xDmGuAf+I$oaxca z2{HaP%))`{6|z?6)7!P)ycZ%z0g zW^n`kKQ=j>>vSwI&jLo$a5eFgp>k15#qoe zVMOmvga%eFF=2({A4dij2ADY#c7)IbH+b@P3EyzP`saD;izD@9!rfds8b+SB>)T6s z*d>nY_%wunY*rp9(5ek$K;E}L`U}5ogh$1}?~Ed(CXxO3Py$aw>7a821V0@ROjb-s zw9h)2Y~(opu_VDXPy0eyDfToN`N^IK5)D)Q*CpN#frbzdl(G}RB% z0e@`0-jj*{>&`)P{I3ll3-OSC=DIKbbRlm%n6?eTFPO4&VEwr_!TK;Jkc2<5;(v#r zJ=rM)e3}2Aj>^+RQ0cVI@pRKRi2m0GB57jwY4rqldmB=Qn8D@t3M$(B?qL!F?&$t} zvFW~fgNHxE%hSXSS=-xAtAme>kFuR}YbG<>7cx8oS=Yz_>Z5*Kva|l3F1a4u5pms8 z8vUPDkM1CrA>m^7@$~dOK079F@w|lt*g4P_;_sVH>fz}@(*Bm2*)J+8N)1U!NXW(g zwxa`L2L~iT&L6A9E{>0t+LI`MnBOwO4khS|@Fehl4GMw}mkJ-x z_;^mGe{!yP2m>T3mRDDcdoWxi0Z3?vAXodocS=}1@nOv^N5%Z+qy9-er&-S$yBLEt z!^=ub3tD~f)iK$N$p(=svenzFRnjDuV^?uWc(~kM{q+Mxo98OtLaS%)gXUMPZmf|LJ~6+S zb93{{%LjqVWo3Y)n*vmR{4^R*zYol5eAPY4ssO3IeI6dd!h}*cyZ3g(h3Ta_S)v zIG`HMI=krLa+D}2`KyosgN?hwHQWn(ke=KUC$I$2(Xybc8$*1@dR zl8nTT6%h_)^M8U2J$&DSpAZjsr$I&M{a8#;WKdKr)d2=D)gq`*HU!mJ>(d`I1ujeP znJH#;Y)Xmw?6guwi#pJ3HCI^C{$jhTUWw@jC)JetXOUbd21Q=C^ zi*3t2wf%h;&9cimfVC+++d+T00+_;#+7hD*y1h^|gi8JVIa?q*r$~!cz3VCD<>1bE|Bn&T;2=HK1j1g91Z6J9Q=qA@ zA*v)NlV%jEu5s(3>H`1XAvxFuJk$Spb-jVQsq(ovIqL#IpJNfwdQ_lu>mk!Xf zO+-X=MdeHcz*Z!oWjJ>ApkDp07ZY^8T0s(P6E`$BlPmTqT-ARx2E;wWw=Oxmejsf$ z)nd#0MOtT-axnU}T+UV#OZnf&>`|Bx-&>#drbnMT<2n_wYXVv4Ia?4Y#BD4`-oZZW zp1EJKZi5|Le*+;lN?l_38`2Z8Lu+B-YV9Ij<6M#? z`!lR$#Hy7*R)(!eLvsO|Px>MEw;4AUKF>QJP~hnJF$)jGI0S!z-lp~Pc1EkaF9;FN zi(2>uoDQ;vSU>yo;pypt*iG@WAN;ttcas=^xB{O>TKe%x9!+07T@)UxIhl3a z-8Ze(wm8f>t9h8D+sk8sWCD=5=Z5_WVr3eSTb7bdPG^8S%3{6+9kAN^L3+gSZa$6_ z*kY~?rpipxzvGqmbA0N84r+Ej-Wa5LL8}1s9FOf;%wV&#b*0`lGY5d0UJ~3|eRB9RuHEu?&X_Zb`^&otgAzb4(8wDz%kN&O~_ z!&IBkyC?b^v~0cwXN}1;mkOM_(|*@PX=z1u>mA z%YNy7N5>Zz7q^mB5^#uCiS}`?r2GKy$@dvIp3oO`SlP#?fBRg%Tcla2kSbN)4AYrn zfSo6X-5WB-KF9c|TL9`=hEZ&2(4+!Xg+ zKeXnba8@d;=Dxij^F>Bq3LRVkoR1kftz{>8yUu_Kz~O=|xh-WUyi z4gx4D(I)=wrDcEkt?68(18y$r7|x753WIhs+#mS$7Gld}BDH1qheL!82V+F?my*n% z482Y~GP$JiET-F#o@fR88yvQ~K#hQ_)*6Y5N(?9j79(#1+&HzMby9@om;GKMez$;y z)+WIXTKfoM{x*zTgU4#KeYbe}kJ9GJD-L6gLhgXaP-8stEgQ4EMQ9Rjh@v^0u(-vz zhCo0+AlcCE--y@|1K`iDB!{?vlL*7>Ulhq)u6}s+J^B1g?Gg|ty>Fe+ ziqgucH1lS@dKs`~A`8)-_jDKNknDxWT8skE1bm0A9lbidV<&?bTPFY*JF=n+jTM z-#Z|lC@AvjxW^VWewW})be@|l`wz&Sx)07_GsHabrB z;pM5qIY)C2w1}>b?vZ}I`ao3H7~pgygX-MiLQeYpwLK0XS^~Ae3gbgBium`<4x>b^?oL7gf6QcY^?Hp#MV5E=iJAiQhib0F z=PDd_Te~km^335uNkSo~z<@-#T<)V`GZz6}P)|$!x{@$ud8J%av{<7AqBO zD}FD%qmgA3gK^=Id1ZR-??>7GN2@^1gU5~l(L13Q{n&Y8z;vp{i#LBTtyCp=szGOg zz(S=SPQOP=I4*yXjE87iVlbT-di$*b;6qGMP}=Q;r^n$n<7o4`hfDQ@uenJ~ihEM2 z%=Y!A{zm>T@2=Xh--s4aO5MYwh}ILelCflfe)}xrOZu(RVuq3w7CB$&45ExDmdrW} zwJ1P|tk+Z0SfE*-fiYmNR2-00R<<=&h!hx!ljkUfIAAqeQ+YGpdW%ZVk`ZbsL8^Ik zrc&!96fJ2Y6=av$cgG5Kbdq2$n=Nlhp~vrj^8$#YbOkyVkMr_-k75`Mrbwkz|LAAJ z%XU8M@Z!*EMs>d={;sWBef_8bv)-XPg`R@^C!Pyth7#29DJk`>)UPdPH$o=iN6o&! zdLT^NAI_B>U7{2xre^B#esWJ;K{B0cQiYZEw3Mx|i3u&LJ(Xu*<1`Y2?rYl44FrW!6%C9xv0ENi z9gY-#6&D^}L7gzG?+JX&|M0TV)Z6J6Qs(ez{BhoHouf{JH&ip7ibjr%vIYs&T7S-W zykegZTHaroc)z5d4tc&BO|zdxcvo-y`@G?ai$@HamB}&>DK%{tG>R^u8Uei$uQvv? z1HeiC$=-pKn9pkur$nAn~gwd}`0^-r+;K+|UB z%=Kjn^?pypE)F31R4UU70#$5NP4C@Asbn@+va-3O{=|qf%G)F#cgbm)J{-%-nWZyV z_b<}vl<$OG?{8kJv6=hD1sT@K!Vn`mwQp{7 zFpsMuJOn%y{26?%CCpz8tuaTjH0HI@cW3L&B7bkvlUU-YzGlYayf}`d;_x~NZ~7Z^u};%EFa4!wqto028yyV43g-z^Bxue3nx4{2*56n{Y>`_``6kOU_<>5 zv;z#+L`5Nn;yJ7Ps<31DYVy&2uxZsocYOI$>#gVC?P5vAP-?2})$+dh!q8X`hq@KS z+w8pZ^Z4Tj2&2CXb-}zaXS$I0*T2OQU{%}fYE3*hB6#CDe?=p3aC0qPS5tjBYjxj+ zTWIlmSq!5y9X6H85ABSK8egvR35ie;YLp%#tb@&U1!k{)J`4m9q(b?UIP6}`0KK!` z&iPXP%d!xB=eOmIqC6JY9fwB-j>;*s-$k$wJj$j`sO7Ckp;^SEa6I#E~~xNg^J4=~iIifMN^hAK&6 zQ?^Ge3^}63ZIXAEfY}=}{jGPFY+Inv_~8rN0c1HyJ7U~ zCZSulr&QU_L~#_{b3EN&0hJscTUi)ByA#$vq@ALoSKXp8{GB|1l`iyHo?LM4Ij;`# zG^iu9X6PxWdyWgoRJdM;2*myP--_Yf4RAs3gMlG(TfU)9k(c59QV#Ph&;5&Qgqtyz zQDbu(63QKn$nxqSo%1@SoCawtx!f2c54HL=kDk}LC*I&Ig7vZ%auVs ze6(5oUM^Ai$_%^obKOKd1g5}A-3K}hn>q8GcX!9o>om#{dTO@ywym!;j z93;9+V5M<7s1)&o@nUl(`yPSI3Wf2Kyj>!m5nV&OY2~`w_ya;3vnM4Ut5r_q{^D0E z158sZUElAY!U~wgOS2qsnL9>0WD#5CovG{-X*nFuUcFjrGOizbQa}9=PaT2zBXa3* zZyLl<_>X(K)8P|EgdP#9A$IlL*iEoC1I}sZuU}Tu<~aKWAd;e=Fngiu;8D@u zdATA?w%4CPjRMkmEpFHH_mWyECOHj;mJ~(YhJBA!pVol>6nC~ z>wOsweFSwuFP;ND_lrr9(F9;oN%D!bC0 zy~g&6iG->sb5`hz3Nu660*F%mv!Bb~)CB4Qb0mmVzo%Uz)l!lOCR2oADG1io6Q=po zJtw~{j?jbTb)`;Wv&OZB;7T*CHcrBjAJ%ROHR%0bXPLw)lfeS1T(&sT{Ijsr<{tI> z>KHmjmGvEBKgMQ0W!&r$ zN5d6#&BZ{u+8p;i2aRhS;`L%t1c7(IOI(p^%SUTNx~nx#+v3PC2sto}u=-96B^bSj zXwx{o(L>^1jh-_LM#id1%+w4a?}P9Xv(qY@1O%z~9GO zrqL7t4AZy$ulT@5g{3|Ae}aNV(j5;qjOl}r&LoANC|P`wj}$Kop;c){*QZ?a!oVt> zyQVW_^#`Nwi2@7K8E#zr`;&urP?{scihp4Nx=;yueJ#9=qZoxU#Q#cgq22%MNZ#MY z)~Kt`g&?@^$KEZ+nWf7zgd9reDq|r^FD=%;FWUBgLSQvFwF$%po|8Jp^@QU`fo{>l z88!y(rlL#98|`0yNX!Rju{f9(*^X*m+LOj5b?mCHL-_KK0(LlLEi^ zOK+_1W48X2&hmrh?1yvLp@(Ny3k{KF>31dh7*x{X3FZ@|G7EQT*$O+`Wh!;TA`de} zJQ%O2l{ZFh2tS!g?kYv`819N6#q9 z`%`=NgjdXa;~aa39D>A^T+5eU9B!urD0X(|pGF=N1KSnz&G4yp2WL?xSi0`B8&ZJH zD-NSJ&sz7*g);rnCCFBI?ak=#^3xL&TYKDI5Pmwj&y7q_llb%uTaLl)mA`kexKhaT@`=26!_y!Q-QM>0Qy7NnN~)@kXlKUjXu;c0x61Xz zDyH?s>SaWosF#^L4Ia2v`ttKbf;)#Z3w51ij0RbDwvaj`}6^0cF z{GAv&vij(bt$UAVA3@pr1~tU(I*z9iqcA5Y1h!@j0(37H>V|%* z8ZJ<4gnD@3Ztgn@KcKvw0-^{@3-irW#?|t^*RsN z618g5fl2~yH=HDT&F^izfC}#*Qqg<@ke}C3`odC#zI^8k1zTa!ogyA`7Q4@rWz7(d zV;I4<|DaDL*d+x2T$c3<#Zo;;1Ol%t-OZTf_WXd?E!*|Cs0l!P+I7T+Uy}cjrasny z@*&rUq;nsg*qaBzl4;Bdn1<<03iEt$jEoJ4u^y`rS@fXle5w^&ouYUhPZSY#hfUQWbZ(;Hh9+F0J zc^Df8Ns|3km;0T(gG)K&0!Uh){o%|{vogI9DBgv|IPC(XEFHjs02&IRLxi(55+#_% zL*|yU>h)%UNSj01@2fYG8BL44UzA$x<(0+6#pQ-_cpu;4g6@wpXZw?~=7L^N-w5?O8o7@A$U{o;wcFo$ z$zrO&ufo>q?)byk(_Idz9#F^zD*oo6eT}p^+zC3G6cWpVw|sMVMfMUAoC;TIa!iXS zEbp5=9uEkMlnQ-@iTQ7g$ND6~tMbBD2aAKHeUWk##^; z&EoK#NSNMzAl(B4WtU&G=y{)+*1Y?L=%cUycu$+h_6neHDXn^ul%j^$Uk>tavWGxYt$EmC#V^oF3?HN!=oj#hG{k};pDBGS4bUEi_JORJGRj!D}9_utH-xV z4aal1+n8T{q>>65*;y6uz3z@*0*|CKiC8;^S5fp;SQ46Y=De7%gZJv5KR%8^9>oJ4 zzs3^Y62p(ARc#$GirQyEp=bFrI9%zU1ONDILT{8QeYQ?C{+pgp16b@xi zk4mK*;ZrVprGWxL?IKQVflsBypb;b7(NdmEtV&ztXF^IfSU$CHT7EG zdDJsA&)W?^bd3{re``^pdkT64<(Plwb(Oj8d_lzFGYgl#vom~@@Jy13yDz)LRGMFh zzP-$LjOMkDoI`i6{wEPh0*5{l1VQV*jjw!8 z1MJ$w;dM6-5z$HbCQ`M4?+^6u*J{3J48x)+n_}0_u4Rb3xcoVj$SxT?RDGwa4jME| zj*t3g7=@c!o{S@B=-4nzVVlD%RIp3p^H{*&U;qpkzw;&KRkIAe$Cr+amv{U@N8+k) z*jv!k8#cbnX67lb)zp+DcR$l-rK$!r6i8CP`mE>(Kvki0yKf+afE*c}HgCUny0i)? zmd*La=6$$T7r=g_MUYy**AcNhInp4g>998jD^j)-95}7{-T&4BPc5U-Gs&ToeZ#7bfMT6)zNJ(=WwwUNNrBG z9lJbUcL=P-O7gonX|Od2O~1aj-0iU2FuI-Xm9k1UYOlAwET*fRdu9S@IsOHg625PP zKAAnK&mb`i3Xxvt@2m`jS2^q%Vo9?DqXHoPOgO0UVlLx%?$IdbD~N8*Qma$e=F8fj zLA<&;rEzjh2|jx-|NX}#{HgST09n|TiQE_Ff$;l)Y0XI)-Hl^9j(P8F497+9+klPY z(ik>fJ6cO793#qnuVMB}dcA>%1Zr~?eaQQxQkt0(!YGS3zjfuRAuCvFWq%yBIuemz z(utgYS7dB)!*@(C6k}gW?lhDvn|%3+dEEyR`KYt=dFe!WXKPXv0JZq@+q47CS*c

jduI9Pyb#|0J6BeRbKBh33wq7pVoH#J6>knV#BRS38 z6Gg18Erh7dkG(fk>>1FY5Yik!CVntp97UatsS%f?TlM2v6dJYfSFlM0?_iF{iBv8E zAZJ0a$GG5zj6yV5PLN>`MXP5I_88y{=>;lIhCEGh*t6OdAM@-Dw5jz<8Ytf{Ifq|m$;A|T6Zp6v)<8aGVPamZ96wBI9MY}xBN(3 zj<2;V^`cD=r)x2Rxe$!f@{Gq%b~+VlU#Rgx8E*Zq|DU=#2_^bS6zrL@Mjd~-`-Tw? z1gM;Zp6^b48|{@#FuS>4{c6V;V;2q@PlP`+8;+fZv*6?78}z2(x}L88M%)Dy6}pzZ zq9Ltb!{%_*OalPnK#~3a4JHju@oP^ne!DdUHtU(-M*B@tU4#Q3E-A`1csjKjAn1mX#VM@~UjYS9P9R_UizZI*|DOR=LG zu+kybra~!&6LxMs*&)WJ-PZPy$~agVmp>!`GW?8oQ)m{`E$E8Hld?)Cc#sW1r5lIa z3j=U~=Fv92s=8gn1xxR&EMn1b^l(zE{@@szr;QGqXBCA|dyVu$woU_*Tym{n+G3J^ zTH8nQ_eL@gs|lRVqSicVol-U{Iw`y5JbG6Zpc`Gwfjv#ov<*bYt<=BNqGM9ONqK7y zhc%Fz%ba&PMRZc$UEDYn+l|DezhiK2_wCy?Iw+tZ?aGlW&@F-jz#IJiiWCE>daLOd zCyyxwSw5RT!_X_ekW7y0d?5@!U?~R8x2zWIzz2A&npA)uh%AFBfhJLLZJBI-z%x;_U8cfY0-MsXmwn~50Q0T@h`GPx z5qbXwO7({EUHlB$GIIb?rJ(7&>_)?7l%#RWc{U8;Mp z$5X+mEdSct$oszI;G3nx49iP4O)7HrP-Kn+8{F2q;hyloR=Ky%w)>ki{^mB{x2aJs zQ=l=)$@dz(o{?YDcPFB(M)SAUb-wRsUDeUG2>&z)SBPwcYaHkIQddz{U0B7`I1E>e zUDsw@$wWKHsNs-CO&|zgwLvDh40-8u^(JiP=LF=RU*5;+9arfv`v4Q?TSaOJ0UC{o z053%Mz_h*MHl`>0v^8siI+A>f;q5`IE?iuj%LBplopI43prG!!Nu(IM?#zD8r^4us zvN8O*^g~yT$vSog`ZT45!G4?fz2E#TN?_jxpbu~G+@%Z;kB}{pMGsDu18OLWo@;-0B%B>>R%SS-=V_okT+92?JEYlM;Wh7OvCx7`qf{c0*O%Z(=DI(_%2H{D z%+AifzED`Ux*G=M2jBG#2b0XT-M1&RxK>Hui>ze<8W#P#hDr{nJB}*jVe}aK@1BtZ z)!yxGQ$S`1g^JY5#K zy|-{G1drX7ti^PVU`W|`(WF8u>$QB$o+SFzT04bXGswECoqufiz74#vR9~4HFE-0~ zj<5bEb-4q#BjrXx&z*ubLHJeTPYin`Bv28O_FR_CV8c|BN}mr^n)c+`K=?XB;v$PdtYN^v+Jk>Xot~o za=!2Zq5B1?I@>;W4BpTi`8<$+RJQGKB6hpn%Qw3w;&tU37|^(e9%(ffOXKy#1Fn`V zLu@D(T|QYIsHlX1s(7KhAk5Mo>>abs7tfnB#80QUhU7u<*=WiApejk7D)7(|UPTne zW?5?v0d%?H);}cqOi92iPg;!ktv`v*Vbn5cwid$Zpl7dBe4eAxqakGU(s z%+EhMXX?lm>uejRc`LtM3#>^Or<0o6@6rYysD}F{sAmcF&NQtY^G8D2G{k(m4FDMQ zNr8lrh_uiu1L8Iq7d8t-sy ziGa!uI;hVRel8npT=Y&TvTbozHndVM!)r@Xu9{4so$SaU z(P`!ZS|MEbvr7B7A^{N=v)0Yn_qQ;mkOUg-Rm#<73aGB9FA*+gPJp$(K&Dm4XvFLI zTy5|TsATA1^gL*gLkz1|a0K%+x0%ffsD8CgDI?0v9RcnE^{j8X?*rr-uEw?fGY||GHd}h za?ge3tj*Dg?u*w;n|e3R>HtvQTyM2BxU1Z>D8?iVQOEBMXT|Sb1!wchDj!7qpm{oX zFRTm7EosmzJ;!Ig^R43vOo>{h#yEX!5~e(=3YkbCGVwaYi*}FgA}y_-qOK#+%Ui!n zoB1)Yjg803KMILtNTzba1A;oGRI$oW^h!A}Y*q^onXnFa z(PmjMHs$Tmzpt+3El?a*DpPI!AkyOC>AOLvu7B5+g-%O`Hk-~LcJmxxzo&##Fi~Q! zacw=w-^vkIzJ?O^>Ow zS$6$g78*OR+piM*=I6!gm=_YE5?SI^TP2-d`;j%g)65HNBw?Hj&KQy|_1G+dW@$U_ zM}cf%)l7$Z&QA(PL9IkU0xvS%_@3?4fVM@(IuO(|4FdX9CeH95`Y=H#_RrE!kjCjfK1)lvCwVfOk5J2n~4xfRzC;viUj#1yKln-^j z_Q#oi>K$E{*q&x&`lL*j`ZU~!J;%$FfX|g7ZgnR|7w1S8 zeo;x~jc+cfZ!)@EzB?<^`E+229|**xAPB4smib7Fz-SRTPa^QTAWdF`W!XWA8vs-r zr>d>nCx-JJB30J=sx8GK6Iag2Bl$6@WJG>%&~@@2MoXii%Mm1LwR(LD!Tef4vOH1} z!!R?O>%EwQp#h0RbUy+JjA^3}2fbTGqsq|P+LAY3PPJtnHHkpLGOiyRj z>oTduMwEH}7Lm+qt^yFC4H0x*aLYsAI#wA_P8O)412mN~isHB0#%g0~wM%$FA0ohYrvIAZzA!5Mn+>j_1nz#qGrRytcpqi&)(Q>tDsTP^B%F1fnp z!2k2j$}iPMc@Z33+|GmVpGbxuPY7t<9fN-9Xf{hc#g$hzu@uaiK)AbOtP2JsEs4cc z-UMa|sNa!@Fhb_$tUsy00xY2D2JPpBV$-EfPTVKP#$R7$qChhFbT-6M#}qJu6o&AwpOV`Ara-91pi2Ngt2z zo2vw{zX~zA?$_D{N*yLZOSKJrdAZF|1AKUDmf!C}ai!e(A3EQ*d_JYG<^BWd~+w3*KZyzk>Hw)}r=y>(dB zZJR#~NT+mzbeBk{q=ck&cO%`UA{`>q-Q6{GcXxMp4>iF1b)Vh+-S-~v{~W{273X!v zCra>&$>u_*_h<1H6QcYUb5zDHhQWF-?TX3HEso}SeV|vvr+S-rF1LV`|NmV zJb*U=;1;D?mu4y4)JU6MdVgh6K^8xSHIJsRTF!}yu<7ElrA7lN0ze)uVxeuC_R~&F zsbZc}-sR!6G%i=G?>yP72qI9_^``3y{PFO==^@O0zLm@?inB2`4|+hs%W}ePM|6F- zTCVg<^8$tnOH+@xhe%ZyjC%PbbFd)<32duPuGSk5(*=x-*2__=+JKm=eIz69#>q)U zeA%+`$$eQ&T|F~jx*Uj=CtLKSaMA`y+yXvuAV}Fj<9}G92XO%a*8q7-E=g9W7WG`R4fL$^ z-O^);vH5wy?->>t|L0p@!_VvcKL_BIlqz$08$}dm*{yor7Evvgr5t>P_PyJs`k>E& zO!qV*eiZ0;OPinB!`+KNvcS1Ixr(O9u1%!T3c^WM^J9rj#nrdlQ-EMBZoO}dr} zvt;h(^~Tag$1o+VKT|$MKDyd$0xky&Ex!0pdytH|T0kcNFh`;-vQ=VulVe56h;}lM z-`6Jva@tn-^A%f%@}>9P7rTrChRdQ}#fbsl%m=((ODsaG_p7`YS{AKITVd*zrMCi%iGZ z{7$#KlS6Dx%|7v~ok8)I)XKw!wn)a0wI&Il6K{XzQ7vNhD*ozdAGcDaGuo}rOXM{p zBGQ2q!Q%t?1;Ku7v-0xaNuyym()m3}0J*S$s!RWVUIBqD<<_Qy)dmrMw==9A-S(%d z{NhQQn)85iOK~Cl|DX}1<2I~F2WA;ALa}}=S0~}V44zs*lY)R=g(BcDB}ZunoyMiZ z&8*qNJksG73B?TEOSy+&Tg`(naP${kAt7_xj+;CVuB3QV<=m_(n(;B-a>#gcIJ*k_ zf1%Q=u$_{vY%7uEnX`%wfjG)$MNtz|=&A`wE6cKr(A#gV_IsLyl3+jpIff4Ftt2d7tt4g}oZc7*N zTrXDd0vfk>81trDuzzIExIu@ z%VT$83{{m;x7UDk)`%Pkok$RNwfQxn$Kzn*W3A0RfIj{z8dv;EvttE6TP2QA7u0;W zaA3=Ey0sHxp==0X=67{QBEh(R9_8Z85DoIzvRMO+|^8fAxxq?)X7}I3`ck1K#}eK$Q>az;er0uKW>4iM#{YaK zi}YkHLp3EskGuZUuXhxGWn4@BBt=Iv+M&E|&PsX=iZ;CfzzS|21#v6Gv_)2x8a67~ z1b(yInP?+2Zk5w5zdtlc?+KlN&zzZr^ejte8=kQhGP8&R^1h2AYW{HBt5i#%U1y!c zjKcj@x1uzerrCWdLhGgBkAzUW_no?L<#-w&+QZhsA3hXdTPw2dsEtA1<7-DAi@M<~ zb)#o`vJjjqp0DXG7hrJbQeDZHzSDad)rl6#62#T&YNHmP2g;F=5yz)Fw3Q}L)!B(M z?YgG0Y|QX&0Q-OM{|?Y13T$ijE<9OlcixfMl~r@NThmMBa~1QQg!8MSu+b^;!WHzo z|4YZPoLP|d{U+jb0rg~eS7cMF-~2Ij9~eBVb0wMz@w*-dRqqZ1`2E;o@7ztA`Cr;Y zaL6w}GKOtXZ3%5IpOW3Tn5%`NnW$6f^$Ba#q~`TkQ&j2C%7n=F=4OPyTG5`Dx}}#PPS?u!qeu5cNPBU??)!l87FH7Ra? zvT<+cywMWHXVVoO{YKb<+4R{H>%S-_q9io${8M^=b5_GRNHAy48wf^>YD5*IZv3i7 zSrrtTaM`67e1f1qHOYT$>hY_u251ilwVsUj!p?n?Ht=`HWrYajV$``-Uj%cnkCu1I zJE}Ub4Ke^t%KcgD*~g5d+)etmc_ZaXk(sXn0r!P9K#b72*rMQLGk({@C}*i9A1VVD zwR*p-di4ykuP0^T&Z+)2Y?Zoec5}oLwNn<#L#&Sz0T;fsdyc08I<_D;qF#e>v4i z(|MEoatkA+f8cVyBr~d9`is+StWm<;sk z!g#)}h_-s&0tKtjN}Ds!0N;>at+KdoyB{f!{n`~qgwG44aY!lN9RUzul3Hu=bbNjB zrPFMHuL4357%zm-|~M4 z%(rN9-j2|AnF9R!OUR;g9#g1(jp?C^f2$}J)ZOts#&A4~DvS&8L~I;A!T_1GI%{pW zbcAR$nIBRVC6t_C=TOD!?wjC@NIG*#Z1vWpx!8mLP@veJ%*{^rwqr z>Hg%^w1oWnTv8j6#F?XB#(ywT#rana7*=@^lFsLsXmk*@SvGzOqd)6~g6C-*YNeic5QpWG@#V&6Q1)wMEy&o=sPs{=2DI9*j7 z(a$G^C`+Y^dMuc6;Eg?99>QCkno_aj&*MIMgKyzVxELtcSFSdlAA--;_F6Zbz*1DL zC2rm5wBM<}{NttlE&B`5KW%p?-&9W~9GF%8ia#;3x`LyxbMU@7`XT6f z^_Q9H8#5f9$Pi0QUjjGkXne^sxvNU2_~n>z0Y*D2maVJv`DyFINW`A^VR-lPk8&+DL|MHu*f)$G6VOX=s${R^jMye^tqDudTJ3E<&^mwO0Hya1K|p zA})!%qwFtFedQBz>yecEY6ftKcg}YB{sW(IObd8{-8jFv0i)xWe@=)7hG?UO4~j@2 zZ_mQVrpr<0OMuNP!@Ol;iYDUzwqRb7!*X8C$L*;+>T}XN4)oRLkf*QdolWN6GlM$X zp>g#~LTeBCunzPJ1XOWJD-s0=yaF8xkiczW_Kh?{*|GglZ2Yvmv`K1EbIu=L|ls51Z%&$dG`T zw8dy@KA1foFMf#fGZ=Zx4O`c(R8U8*<23uhdDgjpAu%Z_#i%1QD^bv>6w5n0L%0B} zclBf>4!;?2sxLz9gn?1V6Ki4V$wA>AAoV%eO*-LHRbj=Vd5Ti6!4PE$qY- zXuAiQNTWXw5wQoIY@4pX%M1KX{V-1W&M<>=YOYL+lrn%#he5sL<8h(P=!cRZ*I9^| zG3St1>QcB&tSg`22|XT5A_7AwHCOMh{x8wRJr?aC>)6#gX9Pf%i~zu3d;~BU-_zQ7 zl~31Nv7IeOm7d-qo<+^UGB12hXzv-!vk$jcVhtXnzqB81|r(CjgY4T;cMc+tdR6Mg9I}o(fkCYQ!w~ zi$Nzd4u=k?IM3Ah**W>@>iJTwoZ2!#J}pc8_9*6YCQZb*OYyu2tI>~= zD@4=JsTR3|NZm<1{D5{&9l&#Ry*fQ(7iV~T5Qum}2=2}W7&+7_hi;V>?|Lpx4py?# zj;7DHaA{2S0a;V~vom_7%3qH#6Be-{I8|BkmVf?24zSF3e|!Kem~Zp&v^hU~K>nRY z6vH8FegbG}#Ry8urg5iM*JF@@Hda%32I2%7!f-Ik2g%ti@GWpTG5{${wOGx0S}lny ztljeyzV9+UKs;BSr6`3C=cdi>viTepK2Iz0$7yev@ogIe&&dhxoN zK)yaQpYNI>i9L3e4D$%(u)KruN3s08Z=q{%IB+I;f4^PwIBCEz zWn$gRIuNTMuzfs)jSx|*_pSe2ch)hldQZHhOKsBmNglO$>Ngx@?3_RQ3G|4kJ?_#{dWCtyi$pjE_c>3-vu*?^jt(OW|r-o7CVpzFaa{)Og?7tBGj%spncMmu>n2 z#37n6d{v?d)0?m;QwxjWAhaWshcMhiRq5qt*>W9_(#PFCtO*#es~s6&CMuk-kQSs~ zremWg-m`$mlDs{3D&TQ(k>#Z@jo1N)g4gx!$1BZ+m4a@I2WDR+X?W+4;7o(p7wfh+ zk9Qcx(5v~bOmUhXhgWg$+y+%Fv;t48XNyakda>w1UbBoq`Aq^Cd}|QaPnjxGxh6%E z@nysA0}RlEmOyBhOwzNJu9AfECX^xy#E!~(b!Y3D%5P3Q)+-Ch&U<5OOKX6_=46ox z0r>x(>>XQ-puRD6Wn}~o)}WXI9-C3_W@ls_*$+U~)FgUTABRm}cRow{fR{l)t3cuO ztt$$WU@DLQw|JJykamDEuI}nVBd!6oCbOnjyTzZqr)kw?e+l{ABfbo#>5NKRL)M%f zM7MfA-fijeyLWwigWzyEWtRK-IKe@^Tvto&yVG8&*8Tdc6N9)S0}~_Rcy`t&x6=pb z{Xa@6fXXM3#HsF&5s3JSy~R_(t92{Www%n)J5tpiClji4-bX(XaZobjl^cFmWc|ae zr83KbUXIJ8jT0Y_H#DcU5K^h#s;#zb=<+b6W+uZDRj9_O@x7oRVMsmRP)tv!19d}y zr((N|?0tYF6dry)V3uLH`TapQGo8~topExp#6ocwm5F;rwczu#oN@kYQtXkRF??lt z3E2%VatD7U3O$rTqocqK%sjsf{-HQ^lTI6C_f0&tmCC0ysw!@!+e+aU4cAuYZ7oVB z8(7I?E`k3i7f14!i?lAd^6u7a@kXv=VFD4LG8{Ijg7E_X>fN<6P=7MH>bV4Y@7EU( zGXZn{|DL-4N6PaZKhl9DAdW$(!6vB-wbp17Tupbj+&~_LPMkk|LsO$_JtNC)3qDzN zofETPYY7F~t|c~~_DW~UaYv0`U!FA^jp3Yj212*#6f9}xL(OhxOEhM=b~G{Zcbk%z zny!RD`n9>O@&jEiI6z1}FftMi_!L~*j@ktp#iaLyV0<|c77owvQ@FKg;Xry{dG;+sC2}ZiXuUJpGObb7kLHZO>)#=kyQ~JbmF*$ zj$*nX!?S{d!ut$3$bLJ79N=N8SildN7$NMU8sY}n@E^rhs6vC_Pf95>71QeZj~n0{ zj%j2?k?5gmGGYG&V0oA$&8We;_&HKkvlxHK(8_dOacoM|QQvkc-@i0 zlqL@ikaeu4l@6*jrh-ZCH%k|CUxSn$3@32dEq&n|a@@{0yg8ii2dc*51S*IN2r)HH zz`eZM7e~`GBCMo@N%-+V9I`v^-*-#QQ?j!2kmJ7Ac{E!B{G|R!(jSS51dzx1_Rfi` zYtIs|x1`-ZL733bdnsQx`_J2S4ySY)>}5XOZqdkHG_ODF2kxG?J-g*09zeaKYc|;u z`dt&p%7KitG}BX4zo#|3WvMZ#GR~i^HeXDIvu))>3-@WXgSlE>*Ss}>nmLbAN+IWE#qO2hM-40_G$&E8=JL{qM%mWU7&4>3}7Hv!TUGdch2s|3C5?^in6kS z(c?MeQssJOUUgVX|6KP%!f&z+qUY@x7q)7{S8LverS`5gOq|#Sie%oD2s?^v{^WuU zZVy+x_lx8H(D(^Q(U``}@5g)0Qm_4A=Dj^`XEki*w4g9CF*9=##LjmQnAGoDue-^& zrWHjx{UP^yEf0Gsd*gnW*BP7suXp~Kcy0t~%`QbZ%uUW{^Q%ZDQ<+XYr2D&!-YFpzzx(mUvqll7@sgR zBF|Syia1;K9$0L`ri`J;-{dXJ#(-Y_|Da^st@@<8V!Ga+>vzrA;ULwv|m8s&mEi8eKtW z^-29B8FE;7dMvLn}8+;wwM+pH^qVsG9WqSX6o&IZgK5)ahVnV0Iq({|k z(xrcbl2cIVTj8tuxX(yhS)$=jv45=296|g7*7^F~ia#RUZ#$C526xwc6XMCN$!-9< z5oW&)xMuT>Cs1B?iW?ow6vYar+Yt+S;RK(folwV!7N1(d4C-nhQsF77qw*47rg1(=zdmUb&vQKT0r(rM5i${N;sf^Lti zs9fj(U?1gt#lw}H&$;({?{S6y@Q{pr%w0)EON5X&c;ock4`1ZI6QeB1_&A;gYZ-uw zw^DRH2SpNd_}qtRE1ug-COGnd>qpYUG52FUM6&j-3xi49i zfGB>08_N<8>c23JMAW zPIFYh=ako=_I8mdG9gNi5e0=YkT(yWH~2(yT)%Z>{C-~KHI)4MfVAa)O3q_SmgO_hA)_*?Hau7u}?ZpI2Z&4c-^w>6TDt3`im(|F!LRQ=i49SY8l?WgF-+j z`(9I1gP5r%DXO}nU+X{tT+;9B&o3C7)uy4k&A!XrEze+giHWru_lzEA9Xw%wpAFs% zX7i}=_%e#CmLR z{3udX#<8jgdiLj=#K;jQMo|T$kF9lgT15-_k|?!qPCv^eXJ%`r?g}OqBLn=1W^R-I z^`OXZ9AXxRBOaWT$_Hk6R3lpc=t$x$xLjU;!F#$p#ijsa_65J!4S~=3(>nfURXxpn zN|=uyIeVc=e0Qz$mQj$A-+pk02OO>f4PK5kJT7|^c8HpO_+R|+mtM#c|0FB>Js}|h zu>$9JhW8m?%i|KysN-16%j50w{l<@s4Hpkyk3No!gp3R}$5|mSaL@AkLloi!?oT!3 z4V>?Xk9lvB@7|+7J>W837`)!5FA~}!cDz1|y#4_6+Km7V5ar617joOjk00-W*b@wl z=VYE5m7jGg{IOHG?ZgJ^PYHy5K#k{s@N)mS@C$*ma*PvOV@*3YokGu($V29kbM)a` zpoSLi>h9*SU2w2PL#hfZEI$Nk0(J}{;XAoa(E;UFI%IcER0wyW2a5PL@%>bpFc;g7 z)6M_d1Z&YuR??0uat(TMb3d{$*)W?6m)2ZH7;YBZjkBFA%~#AK3G!$ljERu(w95UT zFs26Mx(TJADA(IKU2r51m7_^WP9AMXq2tKIiXM80Jz1}(12CEKSG{inN=lBgB_F14 zwbiwe$qm*SgCoUQdi4MVC8m9(*kmk>jgtyi`8p^HofI-zphzB?X+N2Ll_iN2>+Iy@ zK*LYu`S3s>=)n#c3QOpkk(coAYhv}Ps5m^ip*lG^Nqzy}-o`yKhf_ek9sw^sFbtE_ z&G$%8OZx%b)=agMxw!1ALrrr*F1_wP&o-$vTMK)8*0Btc;OuN;KB~m=TEXRos|TEv zb)RiC+Z7PB&0INl)xG0T9aDnioq{K<%MKhh?Mlgmpfxr62DWtfFU3w>s!VbVc3$RBOOqL}^g}%pchv4da4eqGe4~PX{{iX@uP2HR>7#FO6_mu&PXBP1w zwjZ5}jYk7+gDx=u_!7{9t!n@J5}U9`{4h%%MDFK%`1q6!g-UMs)}9}r5d^$>m-#x@ z)O**Zr;C@FKEO2Dds^H5`)XxYVHc(#9xw-H@M=5no0|kr!7f=KcKE*xzZz|J{Y0DF zveBrZ+2U#lmHpjQpW=L^YmVH|SE$m~-85o43NzdbrY z>0o8ur|5hMXvyx%M=K`NQEaG4$*D=d(}|8CSF7U>Aea0kqVdlxMhRv#Jb*tSWzRzx z4S}7i(lg4z(y}lW8*+N?gYf!tE>igZr78c*EVQnw%IFh6Za~W2z4iG&&iTa`q?raN zTbC%u2^U!Wg!s$_N}^<#Do69TkzEkZaZGb7=wT7@)SkY&JTtRO1`(TqtMNWA6>+uO zLGhi}**S|<+G$QSjO*^DFM3bK*4WGT7A9|`b-Wjr7lmf|GBFTtPz}@I{=J49i!;9ysBLcR$)4DGmlUH@reC>(Z=z< z@rMSCjddMj_p{SFP?C_4X0QY-Ez?m>%ur=E4!&A0Krb?j_-*~6GrnYAFNL|N5qw;7 z|4+XEU!OI@`7Ou^YTlWr){Evs0`>#UW9Y@<=)Zqo<{)Mv3W9^VIs90L@XwFp!8vL+ zceMSY#E=_A-FB?V1py8xzni>5#Fch@gX@}H|10c`=esguAXAR>SC3w0Vr1+AV0dQR z-4mNRN0&0A&DQ_F8;66BOTop3+vew|SV#KkKy&c{$rAQvI=5n6{oiKR-G zOy1CSDQ}OHd*>HIK^6{fr2erH^xvd0=^2?Nw6&9wJECFoGI(rYfcsS%tRYbF?|;kq z_c63o_Y&B{v)6C_ZZG8BDUuocTOEdr(vT3^myZz!=`%CT<4w`FgOig~JM<50z&Ntd z8io<-c;sK_r;rEc2&$Z?R@)JqG81#+vt?y%(f`?bbi^4+*bgbny1IcE*!>#^2hPjb$#1N49zp~MV=)+3NxhDXNt4Z^rRqiuwN|hq+=f?gNgUJv z3xOMpw7}z3BlNQ5jSY<(;7a?C0}qs<-z>sK^^iJpMuUpT0QR}bd z;i9Uz0p}}iKXrJ@yDL8H+#F4xS3Q>V2))aMsT_m<*IIf}0YB(ANS(=E3^>zGNNDlg z|B9KSNHfsifFR7B?8NrM3P=YavXL9y9E-eVnL*M;$eHmeu?)kUNJ=6E0nPbc4UhOm7TW98cY{u(>g>V$92(+@CMzR@l+7kd!9_zzrmuf6)5xT&tILJe1b>p= zE5}UQ5raQQ(TFK6y1%zxwwUv(0sNc4nbB zPwH>tkm*=bD3HZ}xW=t*@7@{`rchYVb zfEjH_cyl;~cyRY>_w>=@$%y<)o3C8D6)$>(i2L~>3Of3n0K@&3UT9n3$cVg$hexLS zd0S;oLq#>(iO$wDED#^4o)2dl!v)(JE#6eL;^PM|EG$>nxQou)uWp>3Deu}I-1!@2I$BPUcKug*G$`q>4A#0BcBcFSER>Z6v z9aX!s?d%Kzg>HU()hif(U`TmwltOosi*~sIYe&5wNFB%XWt|oo5ETEXeLBnOd^?upg%50_pA`G6sDg7ZgAhjsM?|LWdL_MH zw#+PNi|cfqMK;Nzf=DG~tdO)D+!9FNK6o|~a2wYmip=7!H3+z*%<$NEnYtVFB9*cM)U4tYdk*lpS$4*!f0*lp<_^k)A_{;EV7&l$NlX%yt7{I!iMMGJw=p9YI@x(79_41mP zRFYH?ZzA`zo{r&VgnDl7PYMcR!RXKgyr%TJdOGUETW%vPB*Gx_ygVAs+7nNpMW}vS z2L}fSd?|GBV#PGBUb=l!5v0*)R{*7bySdvvLuERd|eeaKwHk@LXET zjDwG#1*%jUkpr?MT5O_MUG1>)>TdUh-8nZiQ*iqEoLgGvaoZ^e)}^aq}!~d0=p?*pKy#W$d}Z3-YJw8WjLJ|D+nOdbuRZe zOl8)b2?;dhfN~`$$vwVGR^7|r19hwl$)2$uZFsp=%nqg9pOX_?+vOJM7w9y0pBc-A z#YH)9adGbOCs(NnU8ecN^a;Cx@$vqhLrxm)@3*Thj-Mq?Ufv*te;VrYv(>%(83I+a z$NJ}ZVgDYad9q%EBIJGBIl!?!l%gEu>+MGIV$a;Cn9i8Z=1{Dj1j8F0O9ToYreL%RcPtSy_6bCl_aXs^9`r@=#S`%Xu-&99%_#t z%pKtg6@@Cy49v`;pDv_!@bNNBb76D(;f3p=Mn=5JyoCKPojm>G)nGq~{A*77}X3cE&eQ->G%7Ae6QBF@@-OD&Re8AZ3(%BeTkO%x8h zS2WZs;r)Sws>k_aqAxk$UY%YGaf0~0Gu3AfkS^{lw5^MgJ0ksZYroyfh*D)9$Cl5P z{qrbh!V-}~%RpaUeJ@K(D{@ryfOptF&Z;k5#M;oB#U0Nt@AaG=jhDkd(FFF`MOP*9 z8Vtsh$H9m_TgUh2>5c5^kyuz>?wp=RaoY_yR#1on?H@3@y1JH8gOmqMzc_ZadRObv zyvsC+v!|jW7GZo3bGlR~jo|N(d99-Idw4YsgEC5sPNW&jS4(0-ztwx5{;DD0#D9B9 zjYNcq)}UV1)+|euxXBI$kIlg9Ou!!~K7amr0%!ikcfPlk$dkU1^G425%||fW2bU2D z*$XzQIiyT+4AnqMMRG~Z73+9C&LHtg_;vj`yl4m!&#xS+k0cTxTlG#%Ew!?;LPXM! z26AkZj*d=s+_U8xRpY!vpS%wFg7cWl9|vnSbdR^>rfu;iSG^XmI7Z$8=h2CUv_?RT zyL+t@69tcj%Rr99NbbT5XAj>H+ydvo&B)o{sV1yH%Gh#PIqU=hIlgyi#Bo>X=gfU z_kB%!O7_u3TV8(Nf%b(7+%cQOopJ({G53fwr?ayvNW>iu&Q5o4U7p(N%C4H`mk=Pm z2Z5S8*48eo)`oxoHs0CW%LWy`Wm(l8h7TwX>+!cRN7Mkv+Qddi+L@8j&`hmCAcD0n zHfFd*W8R0dBCcEXFmwvv3JT~_Q}s&ME{oIC;*vrE<6EPH3Cx3)I^sG1Mr7`K?!;z8 z$Gib>*qy2i6;*k~PR}S|{AoT)^dXGkaoo!E^fZtsBjP6@ShPFYs;I^Sj4HLV4f&X4 zlhGPUkm@jmt$JgI{*B1~%J!1Jx;CRgH*diMC!NKSGy|CHOKXQUG_|y{%o|Y(qfz(yu-Sl{dYb)zo~z9B2t& z?Dx|2?)AXe#}#3dh>T5s|1P%J*X_RtvbK7G{Q_nGVp1yP^WpDv^MR^W?4K!ztEqr8HW39-2oP^(3hKua$wLWy?I8Ny|2W?qsEVDM8tqKEcy3tlns2`(0ve;t z2FUBJ-uU7{sJ#NOm#lN;4={kHdkk9#13H^#CE_{cUSxC2I1EogR=%g*|J5#)*ALZh zsTR{Q$-vA1)@HrKrlQ^GB|C+3MCA3-9r9e^AH%kMYHKH{+?f2;BnlFIEVv#pSR0o@C2{#acOpqe=%RK-^=Fzg4y=ar!%#-^hZxXj2i4Y zvmokBOz0^;83eUDD%d!)o#xpLbOCK7PJA$HJlmF z(Dk;4iNUnDtXG=XAtDZIv{<^ba%6ZKp0hsu_P<%UoC-7v#@KwUW72Fn@eHk_;qeQ7 zt@PLni=cOd|Eh`7UVqB&>h;TKq|kytOF9xsqTz~zS!|Pe(jwiR^AzeZNKT7Z!vLutcI}OaAZOj1r;+ z)O1GYXG)kk{jdrFDLPCSBs(cQ72V7IKnbB6+?tlwsEQWUNQR+Yhei&J*x$>CmZNxjtSbRIaFg`*(oklbTbOlD4NmwC2YE%vm9%bDxrrO>oCzBPgqL{Aa>hw#>6P#06bLB7c zbNw{qWj=*Dzlq@$f-|6amL140AzzV;Q}JA|)tuQvn`P56=fffLeNSYeVJkw7S&dm{#iSvR>mGkvcd& zDh}-FM8hi-C%#j`hTrxQS5j=fTRo+e+r>G3iVCl~t-(cK-37zCk8wmVV1ent!6>$7 zP>#l28l7Tu?`?Y4Me|;9;jS>`x*FYKba;65TS9cj2-WbxQ}G&?Ju|;MKM5EeTxr$G zo37$XM*1uz!Mp9Hkj5LdAmC12EH%Z69L{cd@%D!N!Vnv6UV>Ke!Y2Y?y#=E6YC940 zdLWLaBPBGBy<9z?uH=>EB501emOrG-RrLO`;v8vNJbSvscHoBOj-c6VZ-A~-UFxT6 zd|0dW4E$c$$0bZk^x8`B=on}b>Dxwm8CD<`&u zJTGfD>z(bi6;=Avfk2nZ1)EIJ&MOUb0`Y)fya@OI^lWKAW@b@lrof`wnF3z1Fb>m1 z6~VRjov<4dr^}bq8EwHPbP96LV(zy6DHZKuV`n}WOuN&f}KASX+uPGINq#8MF+Zh>UQo-o7(&q-R5NRLqBHcpcegLTZ z9`4q?{hHS89JP1c|8_l5(5x6S?s%@ev%>EsX6AYk--g5E;_btCNE*2x`CoSIE9A_d z!}Sq$qNom_XF~ch+M_iL{NJpF>!*@eBhEl?x7>sT_cNbu0cBSX!Ud!?-$SV5 zcf-eDX^geJtc%$x6~l0WYoA@$+^j2GWpN$k>fq!UU$4e&(WMAfmEMV5U$cB_-RIGB zsp-Ni-5uHwOQro9<#^ak3M0Kr-iXHpi|G!s_51E1>f*7~#Iv1qZD3k`Q& z|7@^Z3{>9k%aM-O-;1V~vPW8@w;-R6ly?j|Eg6HB|9G>Kq?%aMLA`ge6o$T0y!07g z|I3#^UZ*GUaWtT?r46dyak_s%P3JN7zoZNUMkX(i$B)R;|s-6Oyn0+8M-eN%JO0AdEO+e4kfI?dJ8 z@K$GHBLuLlj_28jWkn;$dGK48qbarfs+FilhW$^@E1<^jl*UqC$%s#<01|y}96aN^b~M{=w(VWEfhWoO}192%~OkS)Bnc z7imNo82M}pNtdxSJDnd!Uv21aCmWMv(^$^2M2Pu)V4LnvSd_symEDSD*qckd*aDAN zl0XL+OErHSz)fce?)!-ef=^Z`WZu{f^WZAY?9vkDvpUpZEQ17GnyK+FGVob__4{PU z`d6E@0yV-$Pv#%lL=ya*p0^uGaI_xCrt$=+EEVb$HZD~A9a;E^0DMt=Bpb=J96lBG zR`Ytm$Grte2+F>M5-Bk+re3paf~x=v4WaklNzVpFw`qhXeQ`Rk8|*3nJ-~`*c(3um z8P-oiWnY@*oM2H&ve1l;goHG;$8EQqRIonFG|B7FP4dxsZ+H(AuRJRi!)O0qKvLql zMVC+n?EWIAio-YmjAkS6981Z5yxS79j@j%UfJ7D#!@fMU4(x4bqqtG>If`Bf39&28 zyt$gC*L?9C&|zd!J?ISRID*cStRw$V7mov53BkdHjMQ#f90}<7+0uRjDkcGOcVMgu z7%Sm>WC1G1qhDKt_BDLqM4s>WuQ>Ts-5{E|c?Fx~uaBrELnh(lfM}puIKJU-INNsX zoyad6RTUNL>^PWe<@*|-#bZ3b_6CbMEj2Y_5W6qiM3k-+W6c;Z`M&oy8_xNy^D&;f zZzK8-cr+>FEnjB659B6&@LLw`_i>F1O^4_ zZ(1aw&{G5KSIH1NTyj+exA5P>3P6*X}2r;g~kSe|_7qcazx| zN9%r*7akJTM!!|byH#)Z(eGm$FHx`ZYA5-ij0`2of&jna8KhTD%5hrn6Fvi+1u(>v zPyB$Jj<=spgg%eU=2xd8&)RQ0TCB(p^oJeiiGEt_NEx&R0{w%qWey4oGH-Wv7ARE* zc9JIXpy;<8!SnUDcBIgDNCEb${F#U5W+S{A7QHTDb>ynx|xsQl?)`*_zR=3#U|eP4y8&9J5-s@--ai8))$ zL~<8t)BZNSbh=s**j(8YuZ_S#%A*N}(HF&(k31EJCzfI9{ZXXxA=*(iC~$`J@#XKM zi^DbcTsP^WHm~yNzhDK7r%LVoM1Wz>sCumN4=c3h#qbpA|7is!byJC%WCJ2zG=EvV zx9BT(27J}7+=%4I(uHa4$M)6ROx@b1)jo4X*hW^H*i=dwPs@yY`afHLpueAvv+1E5 zCS2#^Qy+7lChYK*kT4r5SLSW=xm!?Jqf;IWlaB`xO+>bp$KM{x(Y{G=qRo`7rmI**P@${{;;#65TL9U|yC zAK}CUj8kU~QLqFl-)3p9Tn-(K&pz#3u&@W{2W zV*vZe&SGVqukF+sMzhe{;5ZHq7>GHZSrP6m?Zt%-@ z#vPOH6YDMeoIm)nYPG=thz)k`Y^A`xhczh~MY%XMocI%#bpa{43WFM1?sX+!4pFDkT34PraUkg9*395`V!K1=gWjH|L&T!=oY@1J&ih#r9u{CW#8Y-T zH8+O=z4&~7%93JL`|r67G|dF$ZwP_(gBHO`a2w{FwP>_Z#>oCBm<$c?|Mo*+02Y1RvZ<;8hb5#YI{18jZ$He%W9(A zub3XjZMPI^S?9QlPkA(=1Ikr-1vHdtDGA$Kcmi$H(H8S;+(Tu#Ux<6YOTQ? ze{V#G`ufLDN*XxVzSnwsk=Ho8tMdg+-zT)vz`A_QiIx~+hVi}8p`l+Q{r#)B!AC7u zoOO{vIsHuvrw&xgQyv@nZpFtaf_R_5aZ+y>4LSatan?bP+hY75N8K_52?L-NKRrjSp!5s5!!)S;t zFH5tQp|}~`Ch?V-Q1S&L`?;HIyRMzY^PPi}n4A{loX&ZPgnWc>I_{3ye=Ay*z3i(IkKtEiq!Pu+;zQ z-(dZnPJeuuobp7QrtjcO(8PA{}nrv6XC zBF0-ig9~kTOl@~*4Hwah!u<=57%SQ>B9IvwpW9vhi^F{b!+{ASUbh=WeJ>-p_H$PJ z5bcmp5>-K4uzEHbKJ`RYQpwrIHmdYu6KF>#M`4ZE_-ajYW9(nQk~^wvAyeRBKI92K zaa%3PH=g%YbyI@mGx+*RPHo(vqKUu00J;9DK;>5+64Eqw^A!(ADEqZa^|lj(wkF-?gD&z zhE6P^jH;?_-@WOUToRIklyrA@yFlsgu0;wW-7G}9OG3K4yHmOwq(SM} zlfCyd{`dX7`yFF_a14Ys>zdaY$L~1dlk6KGz1)-UhZ@727Z7ypRp;ub`+Zw$#}HG4h38ug z3_<*Qg+V!%**T(h*@HSR{`T}{L!!Z*MgJ-Ja%kREG?8t5w9R!Z7$wL%Uv7Z;bp~iy zOa03o77@dk?Xd90IAgaol-r>&OYM9ye)t070}QXVccw~62(Mn?b?0Sf#@|y3p`xbN znnlY8Yh8u0AtXaGsm7uJY z(dCXqImrfTNg41SZgyjTCZbY)yIjSppSGu?#>trzamyFv+T^@cXJ?_m#CU3=>j+Fb zW&a`x`yVa92wF_?Y_GZfSLG z=#!V>8gpEHTo~(@cSjUyfL)UJHX+G!yGCD{v6JlCXP1N7?%yP)WqxVxg2?GF*=D`YJcJRI}1g=P9YpbNGe=Os^e-B!Q&4UnJ+nB)5%T%-$|!TFny_MJZ85a7Js zLr@|}oN8~q;zz}WSxP$EpU*_jX3D>ITaySs zAK^W0ZegA8+$rs^eT4z~sg>U6a1Y1inG{)S13_PReA7se5f=?m1T6n%>&*wW4nFn)MAsEZ+d-c*T)a$I{ytLi~D70O3zLEu#ATvpL8*q@xk(Ypi4>gB)g ze=N}?oUz4~-u>j<5VJFCGdF^1*hEJ(F8?-WY#>%pty*aOIn9>(kA52=>{ug^`iOiC zK53q08x%7zytrr+F~U2%`TX zBcgj7yzadR*+q-g@I;lFLWC22@5u!A_U?`rYDWyt9-G%HWidzlD*|?t(qg=Eo4bsC z*QiuL4<6;^UE}&nB5Ug8IoPvASnA)g1f5@jBp~RUxL!l1E73~0UWLDbFv(}Jy}T>f zqWbV^^MJsD%E-uLFa30DfK3hIPr|y=&FYfF#$%>WWO>V3P9t8mxiXiOC^;4ndI#Wh zfeq_%5%=@vP(&E^IeE@*r4A1eZgZ}sesjc%h3hw*NpU6Qcj5J*+1T0}$yP^KJbLx+l_{f+UY zmVx3Jo9TQ2;iq96`XjkG+VTFHuN1gLM2V?9Lwp?|alo~}AiT)M%~IJ7+7qYho&9a` zYlk6D79s5LB{k8#oO?Ey?xsRXus+6KI{{*$9QHvngfQqsE^C#i`@S0v(W5}hCSVAN zHXukGXUBoo3(x;|U;*2kE?h4%9Ap%<_gV+PX>Pyf$8enS>)z<~zXoUlS&iu1ksA1D zMZlOvQGI1he4qs8GiGU%v0=qeHTBN<1NM8v7MCMZoNmwE_d(idgg^_uC2s>r{y@j0 zBK@mv)Z$1%n?8@os3LV+XPsZ1^)JtG@*%2GU3}8Woa1}%M4&AFP)fp;V3*=b`?}k! z(p222q07$)G$p?kb~DlWmuI9Ke8Fexr!XH7o5ovFx|_7Lvc0$wPX?^^B5IB@vY}GY```c z1lQ{AEotbT1fO(18yTC_)&K7QE)urLmR^uwu&T6{uIG{R2L0(dE>n~`} zT6JH%w%Wa;5N&chpASp}_hza2WcU-%U+Ud7V;J!ViC8o@SoxmP@#szbloeZ?`Z>n7 z=bGN+(wpVnU$P-2XO%|WfaLN(l;fn&RT(;50zWM>a{O~wxT?cs5n$GCbaGbTS z2g57iq~L5EA@lfsKrnCjWEL<nuab|&TI#$tuLGdk?uYT@~; zvPIePH|AM+_wahX<+N``+%5{Y@GFpD=m8`o>VFqCUz#-k-(bQ4Yau*Z_@RRYww1vJ zdDRf%4~tJw-Y}V@C@>J=1IZ_^E7BqoSRn^kpGT4QSe6Xr>(hX_HE+D~wkxKM)RYwe zy#%Jd(Zjj(`mbMtHH~-Mt?+kWo^&kjvX|6Ve3kBQn1-QbK?>h4AeIItD4f zB0}?N%U`Anz4oWfHIpZ}>D!+!BSP)Ar~n*Y3Aq-==`}uG?rawEK_>V41HnloS1&UN z>pM7$b*^JxiTm_ABcK7|724CGoXLj{)Q>2+&(ZJVet|~zTA4qs4d0RB!?wCk^YilZ zn<%b(;Ly-9vYRK{>oiqj6c}5{g{8qO(lVrbg9z!I!SfaM^?1m+IAFzd3e#a4&})3z z8oT1z6FO#QU^I@vj_u-q+7mN1zdkyhf+>=?AyAYOMwf%BQE9)B%8}sv!(W3YxRr`B z%zD9$AoPQ{ybmVyFrRv>no#zO%JL?<3sp6Eco4KhDR9GvlMZ|C2|1t^=2uy-lyfZI zPv>dC-KKw7d2C&ACV-y2e`!>yRt5F&mu5E#4hHn<-Lb^V0t$EDBMM`C{Y186LfFyE z_*WsEdMzM7I@W6j0wW-=Xd`qL>*37sw2qTNzTWWhPKzvm+jZ`(GX|7Znuda5@H2+x zU_-igeZ>NLNkqeNILykAw(mzEk;n8yX_L?-p_Ce@^95yb6;*72X7Z1JI&)4 zA5;xTDofK#d3O$4Ow)leXOhiz$D@@)whY;!lT7bT*^nFLAm&GzXI4-9l%eKU*$y*3*9L*XqmszSznI7)R|xtE(ga{5Zfs&EpAgC8IMn&Sn# zoUaXhrWnB?&`PD2^W9`TTqg-{aa3Mz>RaqfvZ^!b+WuK37Szidl1fa6lo&C6QI;;k zfHS5jV7PJ0V_&vG^LqN;f?p#M0!iuB-5IyRg|)MtHHzaHf39Pc{?+oe03hX!s%&{T zPx~M5-QJ$B3V9MN0RvVk;68WaUN((=1{T2RM>vJb`B2|ozKNN}C7a8#{o1HR7@o_s z0Zc$Oa!P15{QE`sIwxZC#s9z5u8J?|2wPNeU7k81M=5(<*5~a6G(eEAPP>~$o^YUK zJbb?Ob}N+rhsfkl7z}O3;ism_JUl3Kz*v!NDq2R$tOJ;gP=Hh2jQ)C?s(XDIEdp{% z+!O=GO*w8X^I;IOVi$}DZn=NG!IReVo_zlL^@-}~yuLlButAoR5r`?F^=*&b`uBZv z6~Z12cYh8TT!E6ZJfo{=58|KSD>Ay(F);+|vs155+n(_v83?)o$h8Zr?V8jHt^I=Q z(9&wSGyj7`XCO`?=v|Qj-rP2vKs4awKbZI%WJ{N($_q?+3XfkZD;kt6U|lgY=~I!M zD_bdLZv4ukpXTwA@$ z(^J@PhUGndswCsXgYmLvU!S3bLd+9KLpy(6e&|O=ii&~X3Tij!RusxE4+3VmeFJQM zXH~r236w*V6#=;Q3|-E0s8t*sJz_%kU6KqTE}t0b7EhLOD91R$Ww03fJo^B*6+!~u z$=|9^{=#2=H-vs>W?ETaH;n#M;{u4gver7p_9v&`IcjJOcrNX}0?er6dX$NS_VlWr zhckG>lo%r;^0Sgt(_fL38vwCpSAI}b=)Q~I8%)D+8Ntjrwz@BODI`!AakY|s4i?KX zI42>4HbPXi3}9K++iORsEtBv$OnY$miiEF&O7=Z=r2_(QcyI3uE4L)~W4Q70Q+ieV zDAh{jl#~-HGdVp|`0$hAD(cJ2d(w#^4BS`Tn8!;2b;klp>79}Dm;x>^s7~%iCN{K? zsns#e41HlrZ#7@DKo)e9PCQ)PZJ`{hn4AhHAFjg^xsA&gN5eKOL4w3D1F{P7*LKS{ za^?i=JkliS0(Aa~4JwzHJ)g#+(6xw&Ui#yI(k~-VUyEGYEMeX@@?{YYRkzR&Xf-+V znLAb1Pgdon@=gi3*Sd9y6g`>_RXGkihyC6W=y&ZM>x3N!RZ+np-&IvISi8kYr|k(J9L3g_ z7Sn2!aRhS`wnMWx_GGqkD477rI&j5l?;3*7A!V(KvRUg^S`AwM zshCY}hxTq=&O-UQDVRQJD+y&rS7Dih_fsjS5gZNb9|Fx&o835C4}na+`x811k_8R= za^n}B%~I0zG)Yr8_v@bdbG#8^+lJ1-RH}g(NCqVFYd(zqt-c(>p93{^S|+`IkS||< zV`X1bhzktFhtL8BvI>X>e0zs8$}g^Q=ScI#&v;>I_#N#Xp|w_Pm|JkPgPIqlMv*+@ z!tS;6%RRS}mHh1#Yy5C`U7k)1I%K&Ww?nTm$J@__B*r7Z{u0Q^AoLIUk94eXQbhU>ne!+)0T?=RIlpGEsChv1&P#Q@(pu;D3`TdKi-ozLf zA&Fef^21YzifQX)HJ)YV@ev-|=l47P$2*4k8drhOas3pR{Nb-ED=R}=9L3r?$T&Sh zf7y8Uy+v%^-?w!6y-ZX(L|Xa@r$8-cjH$_%{zqTtkHIwl4@@UPO8D|K7E(>#uN#v;6Brc*f;l~_pB z!AXhczV+DX;OovPUAhy#=7v!C8k(SEM>nTh=a$J&qvu68f@81IQqv$A8LRB&0HI^L z9C>38MV0y-E}j5&93!ZUxs0r=+1AzlsB3jn!3<5_(B!%keVgV_vzHl=8(WiT@+8{A zp(l49u_ew{)bK--q$-lDE=?wqn1*Ut)?jwLa{HmeJNgDP+h6PFdY1e^L^P>qH)59i z#ia=%;U8OlDY0kfY9ayae<-SN59Q51h}#kTIj<7FvORLX#d~{q7e;5Gs!Q3G4z8-I z3M7(ZY-(Cyp4B*KR4e=Rb5KrpTs9;++8}DxYFPLi{p|-`E=gCkZCk#>KC&hvFyB{Y%FR;0bI zE2f3;dBpAF4O+J?R?z1$_o}_i8w_am&S%+r!9DUlTbt|Ii!n{Kl>9qfsfW7ADhuMl zU`58YoW<<-w8=P2;DQshqmvyHu5c8@MStQ#)Lou zx|rrrbH6oLGG8>huyX|Y5yuuW*ow^Agtw<%+tLzg*}aihig9n<)K+lk;c0S&heKaV z$uuzh`*gM!$*rQGRQ%^MrKJ3`(*KoUvDn!?VgnTdp9bsF|4I=maSLN$mNkjXk5%Db zzqBpi?MVU~ze)sVFF=)zo9JfI56#kjeds7ul!$(g?XnvAX}S;^qo2u*O|hgjH?S;Qq>pBCg+==tOhz*uW-1 zW6Qt%r3T1aXy)U^oO3=s=|?aQ!P)B9wurXJyNyehS^1I@{Qq!k<8ZLjf*eIF)0cj^ zk6QcnSLX*fACS+cP0Cg51rcs`qNB@QttGas((5GuyL{3_LqRXPy82z7cIWXLHlwr% zIB=<#o^CH#q8i_qkJ8HOC$ER0$kR1edm@Tr-y=KwWe|ZuR|_I~FJj)msxetU02`(& zqFXzgF9n=%>a2&AP5ghz$p4M5b;RNV2^cMP;Wf|um+vUrD6}$3X@#*xrR1WP8mrwt z@nZVd{rIc$qVmtI6wW@!dr93yq!-2ibJG9kKYmC=cn=hB1X0ddfVvMoZW1rfx`B2I z$j?F^#$J^wH|o>>@Sma_6hJZAoHcYYi1wvRr4spp;a^na2RNm0-+UOZpBqzs@BPf> zfIDK&SF%X|kFVs!gp-nzl2uR&D$g|7viw#jjw*-)Smw1JPR#e+F%@prEPN9z**9;DK5(X<12x7`Ui`b zvz*f!)sA@yRI|)qf>I*dR{4M4&UH4FI1mBQ{g5Kul(bHJo;WZM50WX#V>EOFb&gD{_!*5>j7vPxQ-S!Ew%MP zsqN+W*`VQda{oA%(LkVbH)={zORORZoG@)y0na}`&KFpd=H+px;W1q_q@Fl>VG56e zKMMT#AC|+fPvT|KjB8pdWRSh+z&{`T9sIAuuzI;#1!8)cKY`z)enCQo|0O;6MGWf- zGe@jl+JFiy2S>a&jCMu;j`IuQUockqQ0<_X+x7oERKZB#RQN^Wd=D@T9mvV&=jZ7# z|9x|^5Heuls!)Foqu05)jf*IuP@3nbVF_WB1!>?bXkRak)yP#?he3{6f- zbAevoH*`~XB>z6V3Emewfh|I0yC7VT1mKnSz!=?j1(?nFnM-8V)O@_lnfyL#E?Y|_ zpS6GuT&8Lw`QrgR?1`0U6&MwVe=MJuM_TEduRWZde?Y#HI^5{Xl~Li%0=!TFADlpg z)Vg?U0K|k9Q2{pMaUr2rLmQiWNflGQl4_Fz=1*usjeLBvN=jnV()=y0U)DDcP?3>k ziY<0cEe5#&(cv#m#-Z_X32SF|)#<4p`sC!9@5fm}qa&nzeA<3ynu!9`pRL`o;dBn- zqU+h$qo-K5`}DXt6f7*2+hKrC1$geEp$=|*g2b$n5|CM5-jr~-#Y5z={qct1Z0QQa zjGpetz<~kuGskPuW!w<*#w1z zChP13Y6=~Fbmdhk!^0yc^X;1ZKT9vKXuhLal_^k8-`F^ycKtTEGQUK}z!d9LLq%(3 zzS)!Mq~F|-)VJZ&B?Q{u%l+{~b8#*wDC!}3u1XIIRaUoSV5AducBY(~o?748f>*&} znHS*wxZpFJ>iA3sP`m3}CZunN_+rwN>Bs4rb{fQ=Mz|f*-nT)B4y;0>qb1~=Pz76C zwo{7ay_fX>kY!k=j=PSIo?a{h0PE7NmfGNt7$Rx!afrQ$p_rJKGmE5cK6cg*K3c*e z931dgKYtp^6m9BP8fXEh%KH_`ik_Yw9~kZT&CQufR=$blc$Z}FftleedAgq7#@3e2 zj2#*$fIP;0{|={ug+&w_ZeqSnai}QIKCcdt&z|I@MMTXqO0vlo?)EyZ0Y~+Y^|cbB z{mSu0z*EuDdgB3jBX+K=zzete%m`=N%%v}}0g`lI+Va0P@cq`Sdc&^4my;{qxr?s( zg@xD$Kdoftl_iH0?9VPPf^CKqKDi|xo%Z&>WfFSzVmES1?h9y8qs4PG8Gz^a&*f0K0p%csf-QZ&nUzz|!TQU!T z*UH~1ekc*BaIZ%ZlD3*D;>sUSF~L8$xwO&9!69-Y12mi63k$a~(c`0|y=A02G4aV( zZL@>Dh>%@o--zPc^~$@;uh~Be3suuWHWuSLA9MFUweV31kqSsieE61V@f4bK^@~XC ztc=1~sE7mP$-U3HmBRGmAO8CpY#@!8|O$j7jj>c!AI07CD^1r+|zpjokdq?;# zr3#De8b5Qrp@jx&tw^7LUA_rZhx4BI`|?uQ3mlxbW~$oT*)^6KFyG!bHR;HDOUP-O zBp7WlU;ga;n$G2u2907$I5fR^pg&tc{9Bx3`H(`uE^`GgiQ4z?*iAsTSSRgteb)6S zYr)aE3O}5F{$%xacuSfs;0TMJ+lLh1VHGEoz2tbR@13{T1enGWz25b4cE)wHTYH~V z2=$OPQ6fFaYDnx{N_dS`cB>VLJk?|RFhQa_R3 zJOGnlG)p;jk%19qI#wrYE-TfL=9x#`kOJjQR9f1@+GyjeF*_Yo_y{naFGTrGDrICQ zz|{}lWhiI2wK$Hz(<(~?b`qr-GRm~kq@A_JljQoDklUE=zpszF1DHB(UFXyq^LP) z(wid!K#zlg0q~GEuV$Eh^uGE-Z)#@dFP;RZ!#rAQRa*`RB*+9BCL^W9p^ct+dQGnV zg=|~aGt5oLfaoEPU%TEq3g9Cuymu7TF(vuu#JL>;o0|oi{_fL5IWy`B0N`e^&L%nr zf4s%#FB)^2NPs7LqPjDix-DT)Rco^ZSXDO?U{mEYmZ#*+$U#2saE8gmZ`kC~9bvH(fbO3G)bh0cB$Ra{X|a3ciEhIPbGlLse@Vbp zTx*hiY1uz(*Z>SEw#aOAgxCZLxEnj}Oi@Q-l0++~D|RsqZ%812(EJuXIBl*SmdllF zIajXNd!W`YCS%|J1kYxABik5-dC=@`(tECut)4cu1GKC~aw(Z9EXM?$0MX!CFc%#& zedhq!#4I;t17?@Gp4-Q6RGji~5vNM3>|mxqM7!3WDOG}3LjVE6a6WYM!W-&vluigx z;@aG-?`WxWAanLRT|4lw9za`vvP}=U<{FE&Q5H-C)tg<*KMvrNNwm_Ohplmj;5j7- zmg_b9A8O?E_745+3RTI6cpd*%>3t<^dG=xcqa)1pq%E&6FmNrbJLIZ+ZVpT6{tq7Q z`yiRN>Rjn)#NOFOkbOT9# zpHnrw`9dTS90lr2`0HeMadE{4!F;vAc&eHC1i!vFr#@SAY0Hlda({gdhe|r?r;^5wHk5&ff;A?~=9a3HAzmAx##Q9n4~ z-c7+-JhKgcedjm|fToEg+D>&X7#GM;1$^VS-~Bl;v4#f*(rYkNaZtfS2pIj+)6*MA zU4qE@TtvB90MmzWQE$B6Z%P&*1VwlavmDY6mf-I!UJBa;&Ypk$<;dwPvJ2Q55{TAoGn!g zt~HM67ktSF2Pi9Th)f6G>1+yZp?e(UpXh1;DkEI>x`IxvFAOkdh{zJJ+y}UDEA;e4 z>1cajfbb zRszi!^edck*Babj^El}CZd-||Q!zu5d7z2+o`are`n zj|?!=MrrA>Okc9SzC8>E8hiD6Xif``eXnb-H6}TN2giz_i+Ji%mKP{vRi(rdjHTni zdUtN_nA5n>5WL1WbZ&%^#)%=udf#NlfGKY-&F^-M(m8>XEBWKMfFU;E|9U=AnOjlk zdWkf@v}iO2H)f_2CZCUeaIz6*#bGY=$A|y3yFFHCSKe^xZbhf{`Nivt%bt7G`Dy`> zNxzb)Xg^f*dB^(3{U{XJhf??G*Fw08g*$r&r`HXnK2}U(m%U2mDS4f3KL}wMo$Mol zZ^Hr{CGZJE5cw0ff4ny9ij2}2793w<586BHbavmCSW z2%Uzeu3GD*gea600djqP8gNq3=_SyKlY-$kt|Gcrs=xK89xmmMKHGC?V%gzWd;2Ce zRVIm2)F>mztCx#rcUCEobUp%T+JzN9R5$}ry>6p7nB;H5A!pDcM}cxEdZUsj$#c}9 z1`YsB@&+*7t6V<=T8!kwM`Voe{gXRwR}tfE7EhE61;Q(X%bFZcLoSe3P=6!fmq}*o z$~1RY+Ih$ZNY8E(BW86&>TGpG^jq(Z*Rf+cQiE`2y^(oYq@!Pz44xfw$!E#o0qLO} zRwj2F^rbo=Ls9$YztR)Qm*hqgNj~)_LJFH75{@rD<_FU%MwdP)smS0%?h;$QX#X2h zI}pk~W!{+y{pxv(IvNEy2FJOMPC>MQnUI)QOfn?U>-?g(#q40EKGXZHO7z$)Y7Ie@Hk1|F}4)NrbCG6YGE z!BXMZH(f19G&2~_b>)qRbx&0hvQ zqQQ?|wep1~Q(XV?KG~UU`t4v8VTsLf{@;CI{2x-HjHpI`c-g{M2v%5i&6=;DGf}T% zU*Ns_(Ss}j-FSVTO`MHy-oBCy^_l^i8))<3k2zc5Wo@xY?4=>l!{i%Vsz)-}-2S;K zT3Y7V4WpOuGNcYFY)tUJy`}AHG-#wCB=C~im2h=h?}K=p=Kr-ZCA-ko7OYZ$qIs3W=0XCr3na@%Dfi8)R{iiQk6QiCzvbUZQ2}AsyPJpB| zWoE}4pLCcp@)I*1W5?820$!oOTC}=RI%Wgn|3=c<$-(@$a*c@ z;KTK%KbkB-5^jscwy8=Bj#ja*DTP?V&31+HkUqCXYyU>S-JA=b>5KAr;>a!_`H664 z?OZnsO4Ztjp71!LG`Vn5qp;R)Sk8+9k815y3gMT79NvP)w_*{vmM1T}RbzC;RXGj9 zpW)1u4^uY(Ra3lt3#3A$7L2vD@>xG|EwN#fPS%Kd%ECMpWEi(Jgaie5B0h(IAfRFy zSg5mGswcpq72!eVanL82n+1F~M#cBim+H3_6*Bnbvl$sUB-#x3gj~Sqe2L6ba$j78 zugF<75s-mq+JvjDU&PqbZA3zf36Cf9R#SG9|(X3|pgV@iYfpcoUU_?TAQT zT4O3imGDp3rz?0H(D6-z2S#`Me%ZMFBqzM}#i`eT2W0Qg@132f1pCqw-I^nTrLQma zHHp~@h|oc=9c_Nz@FI>#eXiQ4yYw1hUFvYP-z$sGDxGhtnj$zmLta$#m((EZ@ibxj zn3j{$6(A`Od+>+*k@pVB>BUGYupna9l>Et5R4k$BGn13x-}w^~?yF~D6UjnjQ>Ol; zmVv=4-Q2^AeWRN?E2_zIYN;Sxo)LIAdhzKnJ+tiQ`Yh2Lzg*@t0@!}XLvsrYxj)?f z-AhZz&@tr(0s>m)KClVgWUj{P`{ExQVViIGA_=)>f4@O!m?k zOR#|HUyJIU%r&yynMDK0zla9+tv=VE98V{DjJj32#t2g4UaBB5uK>q%O~TEc zR{G=cXYLX`rqmA&PCZpU%PoF@^1LToxK=#*=dzR0-8kKN>iUIse)3v`rDmOa|U%^4-ZXY3kJ|6DLpx5uY{L-^Z zMVdl;TYKGhnl8pz4Blw127KoX3}RvNRh_1h#)BXmdm|rVI9et8@-mD>7QLQ@Gc`)@ z&t6_sJR;<5JY~Zq=^S4xJzwn#xUy}wq-73+5X!c6s~g=2PZeG5w!ODgl#qFXj~wv7+td|_HQgjKn2gns^$wHJ z@=3J^o1cD26~TO=(hhy`rj{)eJd}CPsEXzrj2^!U8>G)dPR>w3-o6e@4uqzeE&Z%M z@YZgEJ2=@SiUs*#)U$0tut=9_?Irl%JFGT9feAiF+xwNy&|I(w8RJ$My(6G#b8$`Q zcLKqt^GDps`o(q)rza8q%selZ9Y@?nbdilrHyZ282XL8@Fv+B*v~ zdol7(ns=0~KK%{b!^30gDlL^+dLN-i6aMYao5S3yNvALVJZyQv4_6ki4d@5*4 zpq8OHCUpJ+z6@fyu$Q@q3@Hf)&{&~s7}6qdW;*%lN!f&xsB7v1cC+DRorEZy2|#+Nx2o|dy!wY}^5?QX9*3cf_)mo5S3L?v@iT|p2{ z5^nPB5;%3#$$@`J^fJmOD+a=KRO;gC68Rok+3^C~ME|Ef}P?b#|J5Soy^~qHMuc$ znyj&%cq(n9xuBazx~6!gQ_`n*MJnwz?|t5U74dA|!)ubTlDoH(JaO)IGD7rYS+P!n zz-6C+MZat2c@VFEgea57aBZPjCYr1-CG@r1O450(?&PHlk_@AcaR`wIZ@?xWRqg#N zlu_)p6PK>L1%Y}ywoaj1?1_R?7`&MDJ`4VnE>8>-1n(vZCS9zisrxJPZD*!}wadF| zlp8eICmxD5)cBRv);4yYQ3(WRc~OI>?YOuyJXjp;|M{#@WPvEwG-XD`5iU6VTJU^h zzca4Z%1-8x!LgF6M~aYL&znLP!AFj75HpQ?BaD6}onKK&SFz(K7$g;x7yR}P&MBL5 zeucpa{%apa$*~ouHGX05G!z*kT=SJYIMM@QXDI%^o69{G|?$M&C6Wxe25BRb}ubAr|TSrLhl>MkGGlt4ZR3bGv2#A`e^UT!_OW10vEF z%%WE{QFM4kDteMpo5*zMI+RGK(IzM!5_NVXgoQqm%(Y@&%x#%Q^Uz-sJ%$ZycV547 zIa#FSRbxFpYqIQ8ce+$P{2Ep%k$rFMb8V#FSP__4x7L~H4ukNhNa(k&f;(3bRe0|ULpVn_oQYy}W@OUp5}U(S zI@?SE_gMtRJyxVSUq}{1hB#OMH?6JD)jq6xiSQC3=337J)j;+#vaI6L!SAZCMc@ve z*99@DFE6)2Qy+RN9dH_*lToi-J|{95V6J^wU#pwZoGsB?O50pk@s9q$xO1tzkr9>i zdo$=#>6bZ1^(FIsyF)ae8T|t|ZMS+nlWj4{=2Nk<9;CLJXYQ%y^UNGm8Q+iR4h(w- zpWU-;_)z`oH?8+q*P#M?r)9crw&PSbdrjyapez#;mU(S9^~Mj`U00ElG0k3cZpCj$ zy6KX+%FI0G1u6dGBfTQpz#$*o3;SC@-_pzzUJq2UB=Y1lYN4J{t<$&O1kARl)NPp! z1P9zITb~h~oSfVoJAWRGacy@7#ijC`?>F?b|9WlyP6SJqH76tL^qyHSKxh@BD7p?w zPK&dJ>^!Of-or_*R%3?Yb@*Cen$6ivD(IylPs{KmJZsb5%7?2^ZI$tepZF!CBZu^b z_4aeHQEzx$NsEl0b#t?+$Oq5dF|eo{k3B^D`9N{8?eH#9DBCh{xW1QLMOGtNDZXtF zo)eO;BL7@sIA`{7;l5WVe<$}Pvna3!IAD`%*@<*?{3*xRvzhBk3gl@COq9}DR&EuZ z1gO~QmxcI^s=hQTgAHA`o$n38d1s0~)my4cm-BbQ_7X|&A*q*bPO6wk3?{O!crDBr zrg1&mB+|(_^|w}%(MXZe!Y$&^-v$XI8oE1O93`t|`Cjb@%FzrQ&DXsKsxviD5Y_R- zNB%KOv>zvbW2VQWyRZ-$q{rB`6QLsga`Lbq!@GnLWJ=_S&341)M~juOx5&JfcXC`$ zl+F_`o1J&i06UHNw`e@Y$mZib0+oE`0bz1}@$VZYvue0h?m=9XPXa*{XslXn*>uB6 zntd(~D|JM6OKy%cv^!t+r?#`<_?gP}+JobMyn|#WGHH;(BVmnF;H#nJ1;kM*u6(u{ zbL933=8_!zW^WLJw=xrgd2|`-E9_xNo-@v%=9b0(B=zk zaql{u2^lDf_(#(~zVd^H5xK2SWPXQ$-@mGx7L$+F$S|@g`U$W9*jrp5azyx!Sz{a} zic?|nHS`~<`fii8yRgkJ7w-%w$_&pQE0&lYv2Gol&SD;!lrGGuYoyiYEzn6jV}T=o z6vqNM=9Jk;yh1%gZeH`>eXF^P#ve=ZT;J%u*Vf5e<-IYTvfYG{R#%q8u^v6;cFJF9#0t1&fXDGiza>i8B(qfEE%$&)pR8exC4fK||E3eRl` zc)c!&g&{?~$@O^*uCo|;>Zvrjf}SjM5sJA^<6=zll!@qd?~SGP)!FGxbb~EV!a(~ zE*s4`yWI`XGu%WO9YdznVCE?xkwU9`ETJcBq$J0kV!B3M+D!*Fapz1d*tOSt2 z$VSS+p0MCu3xMJEFiM)?$WM)h(@-Wx941n0s|eMcZ4rX(f)bd}cs8h%G?k>}hm=5n zulq-Jc$^M*nLZX0?oWQbYa94}z6AP17YzCOjTUibQOp$w1PXlrskpm5l$^WQmUZ&y zT1O{Y4qtEC86t$qzu7A~gKYB*AFX?-K1Fj|@_qFW><;S5}zDF_iC+H^UO3@=yP>yEhrhBWAInIRpJ&es|-QhgZ-`{z? zTtEz0Y1(y-aXrS^l&Moc%3XBV7e$=(^mos4zTqe8>`4^KVC?m-H%pp`1I}qA)M2@% zQBNM6seIcvs-5L_OyLFP#q496&Kp|nII&xr9i6Xqt#16XX&7Deh=%5b39+fiWI_*%y-e1tNVh24wTvX}&rG}%V_{Zn$8_EFe z!?%uN9bj`J_&=dug()~`Ce+0phFrkymu{s=pOTAI1Ai_A;kAsVqp@20+U4*Amkp)U z)h{``c|vrhw-mM~j0hh~O#RnZ-sGAbDF4XX>1k=YQvJ5U!6Fi)|Wl0 zsg<($%;NYSXqzjI-MKvG;f3x$>4Jo@qM=_gq$2PG)UY@j+{rccjQ3VSUG!Cr5fu;V zR4~>0g?OIBnbpcpjm|slyn}Q-tmbi>P2DS$S-EpRq(s-C^*cfXiQ)tC3QmNaE->0R99HIQ#HBXoE_-&8cd1x8J8`=z`3QYEmk$#CigU|!3Rk!Bi=-g=hRSMR!D zIj7S{Hliz&F!a6;mvlKl&+16inpTt~W3?=lVN&gTqFI$4p@6xBfjPt%VRsWpLguE5 zKUm!%Gi`@{_)P{L)2@#sG)LfE$sg6J6KlpK720j1H%~`Balx5}9q2{}!P@*P3Oeb` zz-ghUANY*V2&J;)Ks!sJ15(>!wqAnOPYL@Tew#7yX=&u@q;P7Mq5IU&JTl=}}MYjOKNh8{Ae%Hpcw$#{$58Ye$PlTacW?X_h~6EW(vd8)KdD zd)FYh+YEa3mcX$70_BXJ3+j!&(2_Lixx=(%HKn8iwS>nDqTSB#DYgEBO*PUS0K=mGdf9EjItg;iCK>EdWyl&kn%@D9aiVF7LVDL8h-PTt7mSCUg%O4-r#{zgDUJQ^O}4YdejeMMvSv3AqxPy&<1 zwOJf3V#lnVK$g)K4gRq_Y^*DVfa`(zCLAJllM;Q>ub;ynOFm_6cDm$IxiVO(*n^8d zY+Z`%p;``<@T4`ipwX_q@u2pvIQK-o=-g|JJ4+FlPlS$WfQ^r>w@yl^0{@w~lWVY<5F;3llYMGhV=j?9Dh#OxTOryHt?! zdvqYgHqR89wrWWCd`C>Kn9Oc{ZyYpp`-8Q+(58$!=yJ+zI7MK}>8&cGElj=THQrjqp5E9XqS`F28sAoD%DdVwsm$(H(7UA#(7n!1Bppe?%$Fj}gFeKza>J68wm)$|sfSs$EzYoBm z>TbRRrlz#hk8s&c4h7Pq?S){G!|8>=1KXOosFc^!yvCj9Ip6l(%+EQPQJtD9Prh-w zewEJ1F3WbmZ1rM+FI)VvU#sy;Crdohm((6edRfDbLFZ{gotZMFP#7$v2~@pyzk-33 z8T1*Fj@kf0g!7eVH{;>HS12c>{#jW~bs`PX#>TT7K=4R-JD_ROE8J)>Pn+o~6KRre zK=#h?2HV@!pV&6t++_dd`1#WN%A z#A!|!LL-%2HVbcn(JggPQ@_i`>m?W>DfWDJ2|Y-!8TL*?Tk#{>Xr@%rmi6XKrklM7 zDZMl<^mbZe6PWZ&bL>}|{_35+OQ6xLwhn3p#+pOcO2c%+6cRTl97GFea(Bnm+`SXa zgpSVqfoPQ_db{cDq57eD~P9t+s~Ob!+Gq5ft8#|mo~sTCsB z?3OYByk{cROQVQM4M7UwH}|>tHbVJu{cS?V?YWnuKhtG-s7*5A;Tf{MMWq1c6Ukhe z9*0wjO>{;X%jLnmv5-7kk#Y%xlt4Hvs2mA{%-EQacuSceWR#<`M4E}e7Y5B8Bzp&x zfN%*(>gi0Zi)Rasy58xWn(LPT+QzXW5F zm0XY%J;H}9-v8CtS3pJCMQf8Eh*E-tG$%aeHu~@U-8QwYXKK1P9*=KLh>BNZa;2UAth9#D)(5|0& ztTO#vbxAwo2|^SS-jBb0&Cy2a+8HA}&O}HQG7%yOSxTOPXb2vxrHbqRw2E%wg9{rQ zXDTnkT8Z??gk!JW;P73Os8%<&sqKSHES%R6Go_7vQ1N)ptzVlx%CM??#N~)~u&O-d z>;+YiI(_GE@Mw+zQlW?)4p^_mEqRBBo;C!>3If=cb=td>AEaUyp7c*$zGZOlr96h0dGZ=Lp_s%=G zxhX3ISZLxWLgQqQqxOu{1-{SFmVTdsF5p?qeY-*5IYnr3Wi!WYFFd zE9KRLdEO|q*)3(o?r(Q#-!VQ(P3n>qA-M**v+*K}9m6v-y>NwFX4fcpH5JiIdjV(_ z4-d19L-3+1<=q!)JKyj-gozol4cqy#^3|n*{FbeE-YA=!KcbP>an4EB8TsCLXEv&7 zb`0@G(1;=+gzUb?x9z_3YhbmlI+Nj{ymhQRl^Cn9dIz&S)b44v3LRd2l)5GOJ-C$i z;r>a6B)ZbB^c*$P&ziS$(^T&xs@VJI&nVYC(^Byp2fN-?22WVfNydbAaF%u%#&w$u zSM)`_ufuC?-dz3cKU-rXXj=EIWeKIh>x{~L>5vc3Pm}~-90GTl+a2l8<~1rmr99eJ%Ocd+jp7`Vvx=O9Ka?Z9lyyS<#Gc1PQHemz4|GIy4SC~s;{uEeGJVf`*wO+ zWHqRxV& z{xPqf18-^824iniGp7N09L>C+=b-t)Qj2^2~f@ zFzE*)wBE}X6bb?Tec?L8*F7(OH}jx2t@RqJi>A!eq6Q}~OAcjXZVbH9V8WfeqfzAR zz9%c|wOt&++*o_)yUZA{&{uNLC1YK=5#b58;c=ES#G2iS}%97SDU zbJB*;NgLfyAF=CqFqmnnh`ymIJoPMAW!QH1n0K#yZSVph&0J0Tcu{$J&n3NAqaz+Y z>U=GtmR4QbUzPY29Q!4owcioCM|7eRm4y6J`7U*-!x_*{ATW+%V?BTPsV;A2%@&4{v$k5!L8mK)YPkxvU67t7sFaLX-}yUu*+CIm9i3rkGaOz)#r=)0@d)q*YN}&gkzd$K0|Wp=68!duvMvxL0Bt2dAW_FO-%;Z- zm2&*Saes53+DFCRkiTj^)5N1 zoZDib?a+-Bk&*%eO}>27mVHLBoUT`8mk3n#b@Fp`zVAfyaF5Cy{@AeEm3<@Fj8V;1 zM>ElCf#cGaD7q`gq^*&3N-GFY5KJF-?(k(_%bP@U$tR0M_g?5|l~}5X+IQAIFG;&j z{D7M;Mb4YzjKS9l5&WRn$U6Vrb4wlmE#5pK_K1jm%&pejddrhH8djQBRvj9fZ^I?p zCP&MN4rTJJSJ>T`a*G;MhM;N?`k*6C1p=1eI| zO(YmWX2cKfUwN(LYhl`^uv~p>O?mi`Sc<#mtj21J1d2~abvf#&bVg~4k5BlEa_hF3 z^qP&P)P8_b?Ee+%^6IHO36u6xr&fwtjXWAO6q>Uc0FXENcwF}Q^*1m$9CoSTwA zcf9@bq?Fe2^UXBrGBf!k)npDsBhtroo9_Pm#y%L zS0CaK&1M{&5<6pxQbI)qR-6Y+UPWPa+v@~?i&7Il4<(#ZdVt=O^&^lbI#@?3a8U(4 z)tt;EZ6Ibr%nF^d7Q`V6pwOq_ryt6%Qxc4Mc^yUjo+>F&BZ+Ma#B}~3+1UXwb6~^W z(-F||Y`bK6u+#U~EeJe1OVytenX{Qb)DSQtkUmZq;57N29$^Q`B^AOH{OzU07^yoF zh%VvHbv+u$I9=x1VN~&WDUjh?+1)09&+j#7p zjYa5wNo_&0TmUi3Diy@mBnE0?p&Qe-wtry9__H`0oz?QZ0|p|LlO2CA&1J#z^dZ1S z4tA3DTIAQVy4wo5Sbxt)VnV;YjZfW?5=aqbZ}a~*!udddA6Kz5W*YD$R=NnS-?lIX zBEP4BMCR#+sv1BF6vIE$uT`PkQx#(~6v!N3n0gmg{`mizkU-9i+DDf~DX4iX zf&J*m%TV3F*$)Nw$dskeK>P+Yh_CogQAE`5lrcD;hJPjFY>{15 zAU)`hWtFJR%-=ZWB5F{et4-xhLtS}(a-ZCJWCEc4)DwQcMYw`I59l<%y1~e2H$2GAWCWN#4c;7JXjE$r*qq`I1tI`a z30z{YEpW1d2HUcknQPqrqXl1q0rCYn2sqb^&)mcd7~#)2IsN{Iurs^_qha(0I$gpz z`hwxLAnmVli+cgG+tITBpF_h94~jXk^?0JBKrvY}m2dic+msuO0QI;mz@OXCr_sPhAaGVE=c^Ijg$sc{!pu`Rg#i3x30LB+9(PkMQ zGZPdaxamqm_CxBIJV&8xbumL9IX5t{x(I~l(vzPr>UY{88{cx-vh{)gp;LZU{&3ns zok8LvV8 zGR&!w9LT3-6UhanK3qY50(hW!Op*EGt0Jm$TfU2;jz6Xk!_f6l8Fg|khsXDANAs&t zt`P6ls|fGaz;yXT=Cu89ecF{S7Kb()O)ie;x(j`>hs?JV1Yosn& zKb=hRXkhjfw7&XG4>@jFl7B$t{xPh4gz%*TYrP+fUZ>{eCEC=*vtAYs(G(A0OicDY zdvc;8Hyz_VQDaw++1lzV{wrF|+A3dUZD%RLv>`u91hks*qL}@&rV7YbXOUS`KI4F} zO+e+DhMG6GkRfC_JE?jzi9V0}v(l{G3T?2OZ?_AKBpRn4iuYFUG(z)Z9t5w+37;v# zZV8-Y6aKn-OK=%3;W=C4fYOX+lsfluxUdf}u<~EN+~U30f^OZec6@-5+SoFFxt%gz zWiuRhDEFy)IQ24Ww8+JR7E#Odi3xU4@~*%k^W4UWH~)yJ}b`H zhC{3`eZNDtR*jGAAlZjtbj_yBp|^OO#M)>HOTFK|ge@{7v;86DXwtvMtm-MfnJ;5K z^e2iG4nZx>iaYn82ZCWcOn!enD+9LQw3R3WXTRb(RqRQ z{-;a!B*as>5NPi~-z_L+MS{A-Qp#&}Jm_GDi9Jt~#bfm=IU3tYxzQhnZR-VfM;0Am zrH3q%#cWC>JMD|}pQRf$$I9GEYwe9?7!8fHIeJ1)(tGyvkgnkU544Me?`<4rNvA;U zV@uliYGkFIh;gM0zAJt23@DDy3-aduMSd~+-p@|P19{I)qPlOjG!;%!$7M@{Dl^QI5=P%#RQZ0C3i!3hjPxE&+q<`cv@IH9y3|3wxVDyOra-7aKB?R+l zv1*I;?ny9X^|D8iL7vRsKblm7;D-$InfXU-&Glt^j(ORR6mys!ccImpk7m|jQFc0Y zH&9M;E7qv5Qk31YO*a=1vTSv97B zu82$B8C^~e+3&~TVmy`A3KT0?p+B>x4Cacorf`I^KHQFte_n_>e_~Yotw-Gb4oEbDUG{Sxz?97L|yx?SB z=(A@&j%dUMy$`z0Mi72VQnQxhL5{wN_$(OI_qsU_)Somx`_7*rl7 zY-1<3=6WyAWOA`8zW?e?gc;zuYrg+gg?D-_F1ajNL0g>Tge+OqAqbe^`|GLCMe=H8 zi$jlBTkpIQeV$@9U3)-DrUJ+g_+it&yUYIOqv-T)qDWf4X@t$-A}LM@2XO`}<~h+# z9_45APFLpDq`v{CX$>!cTVECk>p=M_tiPR@82u^o0WLyE8W=F8HgEaCO751RkOmS} z@*7MqtjReMOsHY^X*J+trAZp_O|gJOxLMN7?jdB48h1J; z(2&Uj?+KOt%?rz0Mc5r7LxhJgr+z`t#%&d~qSlu$xhGdJ6KeDEnT37F%ej!o1J8cL z%7;<}g#7o@xkAa?7RRWO50zYxI;NUI2?W6_x_Rd5$43Kq*N2&4?atVBo@iE)VdYVj z+2L7EeZlsYFLx&+c+xxF9BBjj3Nb9#NcEhe5H)*WbD|8(C7BOJpxY!)4*Vgh;l!)oCa~^H3Bn%k>1>lN%q&Kg4CL1yF3_80 z-$6OisYyO`fzhPZW49?yEJcWQ+vH%I8`vbK3Q!!9o0*bHn$!X1tbADl!)x}#U+HOy z&wV2M!Mx|$J3l1TQo!j}d`VvcIzf|&h6aTwvv*^t2tdp4ZQP<+@2i*q#}%{X4i1{v z0<5;4U|j)u1z5c=jvs+hn; zPs{fFs`6`f#@tmPSU(mSWZ8t^JQEnX7~voDvyvX?CWO2xya47ZIRWS*kTbs3?1&k? zEb~fnGW)D|>!%#Uk}G7mSgO5=iAIx+4IVQP&$>HO3*}*Dl&6i0g$F|FcO~eI>Me;&J;)|sLQO?{q1ot@ zKsO>SnoZKb2ofZFMw(%l9Bc>51$ZkF62nrOw=~6VilNJ{UW8eB5BPaWO%TLT(uT9M z-O3ga2vOzGh1pt$nVZ68?x@u~ba9D_LQ$k)EJbQ{*8QzFtlv#)FUD1g)OYML*%w2Pc`_&zFD$w2QwP)yU8dcvuyFc33dXJr#tVybbo;YG+8*kJ}``0ovu3FJ5^H8vpqg?9QaT1k%ouAL-Y)GlXC@P=8VB zKRU(-2>|hNM#N$O=u6CP+?Fc#QGY6A4i-uduXqyf&Ah$^!SXfD_BB6MxzqplmQIv@ zz+%KLeed0#3Uo8}WXYrkvkTv|Up)7D4cRZ&PM?0rINO^Vv)NE4*cmg6a z2eQ)ORsW>ri&s3}K=;)t$4#b5KG`I?;p^4j)-z_(-`cAo?mQaqYNa~$(;y%$7q-r| zjIN-hX^TV0a9G7roeI?N)N zq>{No2seQ>(?bzEikQWH6#ad{FP9j1HTw)%SA>}G7+UMT+ z?J!2})2n=Q``I0o>bpdD6h%k;9pM-Q7gehLi`;c@GJwWN#&ac>TSG3j>}dRR5u`oQ zH7wDKowrjvKkR5_=}IIgaDTk%?9xwl^|X3O27S@-)0Z1 zSSKt4B7y(#TiE6BbMWl$p00g{b>|=_jhJG$p$CH>OvbDxf)Bzb_e!ir5bkkZ$6pZT zFzAY&I$Vq%cD#ixt+a@QM{jwKcGdggi`b}_CReS-j-YFf&Rh2+hDx2K^96;Ljwl#q z@!0&2&iL+-HSW95UA5dVLt$g%<&S0Cw!h_KThp=IBvDbT6I=|?x^FSJXxgnzx~(b0 z7OUKCgfpC-{WgXMna?&X84~(zD%b3+)>BD8O#Emzj5kqfAvY^zY$-edfK?M0S17_B zPbd*W4**Q5&&-Oqu%u{#*R`2czLQVdRCPw_Ec>1ZW&LGpcPYzp`mXN#uk*+f z5v{!VV?nVUI=-25wwl}Ra)CQ5+Wr~08}`RW z_5Xy)N%}SV9UXncHvZsAYhpXu{t(sL&lT2V*Tpam4%b%))cKF;t24-e5+p2P9pM-D z+L@eJVqKo5RIDt|Wg=*(m9^Tjh;>q1) z+cG~9Wd-MnXuHzc4ErJbk)|f#=(7uZliwY zyu9B!-f&?=%U*lhwS-RtUaKRa!6dtxipQanuG5t`;QB2lH!7N3}|6*D2)2p-JOwRov5T-~f>Bw%6Zk^;%>i_5+ zZqodq+1;k^I?rPeFPH9j^18~G@7f6l!Zx)cfwhYr)wW_)xo62)?l|$xPN%+&5_Saf zS%lsllcQvn40Mf?f2v{R?GruI`I?_}b@+_ggc8yAXr9zHVu;vB=OIwVcCD_um&d5arIk@j*eHa7`&e*Z|-MMp3J1uBHC7JEYv)u%mAS? znMtihhE!52iUpUQXmBy96hn~X4NM|^Ih7E0QzhT5g3cpP+Hap0hT*1P``>0p5oeZBwBM>Jt`-Dj0_DlRre z2eL~0;GG=zx05ampjO-We(Wq|Z6p|ApEZY3Ch5GMUXod>ixXcNt!kM7u^>|b>QU7O za7A7aTD{n{Fpx_nogpM&A4Y1si)nd##%C)PX3>F^0q3)BI*&30OKmk5w5{!~j9qiu zzM21^dV;yZkv516p!%pjNEnlZ)8uRqt!|znbsq#v&=#9Me{l`&d_=fbkD)My9u(M#mUbhS_FjsmX^)%P z{z4<(Hr;wBL*o#{a^yv*p^dj{(aTM-kB#De;V`XK#4MTMYTFL{yG#iPm_5^@=e$NP z9SAS8C5F^Djg)(e?1fx6nt9iwF|NS|>#3XVO%8Q1D6&8zE%J14naM8o@cc{aJkbaw zX%}p3y@l87u)f&$5zJ1uq6ZY@s(i+2 z7TKt4Enc2^Mr)+=Lg#9EWuB2^(II;_N_A=>m$e_S^WxqEX*`)FM|+VTI1R zNevr(ZzK`y(?w~baT71K3Nr?RH>_9DZa=%4`}($o@}1+Dn?7{Bo@X8JvR~om;o&&B zEekM*3(B{}gRmVCPA@LsNv=-x3>}alJ6e6-6N@+g$&-m@Rmh~qcFVnGemk89vtMeH zs3mWGWc97h0xiVjZ~C7l*tr0<9=*Vc$4l^_`j@u-_X{#Qzzfso?+iPtG%%g{k%U(x zeub>wcY#=0#ku1RVMzVR$k{|$SJ`R7wj|q&R8;XRbF91Id`8Kvte|dX@tIfAVzZFm z`UE~5IvQ^I-Quu)g+rU}t035~s{X$R~T z9|){`cPBoOOo=p#3{F5QqxWa0N|LTw8f% znTzTUNR*4vuU0C-hIBqlzvw#2Lj44m>X?M0b5{g0IYlMu`-4hfEnfkXR@;N00`FmN z5HQMo^iifPOJr%S;iPt?#+C5J%NO}4E}#ncbjM577hn@z9=6=B!Ay)Lia}3I_>cmJKW1LWC;AWy9WSD6(V5v8 z?SXJ(B289EXb;>HiFhnSH1s~<<3qa89;I4ea%4+Ep}RH9>O>ih8YPQ@g85~SIBd!VmE}lf@o=IWp7QtK!6Umn z&L}{R0$e!BADA*DgF2qDGNYh!mLKpZ3IxTNf(X@1r!ACM0@wPk=5V{Q0g@IgKe>?o zd($}MDbK8933bn%dzRkW@S}NXAYP?G0zA-ejOXCh({F0vKlAqOTXsQlI&eyJqjSSf zi!+`c_b~LYhW^A&K9o4^F0BlCQ#m8!Jkaj63{1C9A2&#rJwD~tvnYC@+jzZ%!;4MO zGxOW^|Lo?!H>^q#XEQ}TP+*(-cbD{+$Xv;TL)|ojCd=92!O&{-`UB_x9<~2|BIqjV zJXKzKD4?F3K}hMIMN!~CDDaED@yd*#wrdLCrT*sVEuoCJdkwn8ZH`d zecIk$3g>;gMc=0X6Lj4L8~5kCSF^b}2=mz~FKY-7QsKG_^ z{jc^Q?)bm#0dn_U2b&8{hpd`Nkfy?SzKnB&Js*h*{w5(V%t7$(roox+U5N|yg1kKfV+KwuP`V8EV3E}p6K8C z_s{cyzvD3tJ+Ek6+S)qt^B>>K0J87ZuUT<=V$f#c*4Ev7Xx#D^a(o*T$Y3t!c_$b>Xaw3gsGDm(J`Xi5Andy NEfr1W!W-sK{|7gQLy`ah literal 0 HcmV?d00001 diff --git a/docs/docs-beta/static/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-cloudformation-template.png b/docs/docs-beta/static/images/dagster-cloud/developing-testing/environment-variables/aws-ecs-cloudformation-template.png new file mode 100644 index 0000000000000000000000000000000000000000..9b223bc5fccbc6dac584ef1a6d6da71fb51c8139 GIT binary patch literal 328862 zcmb5VWn5d|)-4<~!2=X`x8f}toZ?!%xD_bw!QHhuEncj}X|dvNMOunmf))=@Tzb>z zyyxCe=YO8}w?AYPcJ`Xf<{Wd(vEsBe6!EYru>k-8p0bksO8@|q763q>24NzWqBE0sM%rXm9MTrmPBxdr-Cd+96c5Tg9rYQMSZVG%PM^4 zeCFG`-@F&!0`0y!w=7d6GKn&I5D~G1Gg2cx@b%W4G~EX`wY513uZ-;(^g7*6x^|;m zt|WFN4_DjgIUunBLV8sBP!Je}Q5yX}{lSNaE-Nd`{Op;oEGHKi2Sg1`$xP)xKK=jR z6RW1Cretnj7-{m=E3OZXJN%!U`JW%=kfLsGZvL2^eF3+8#eQb>4`%(lA!))QgtXj$ z?}jvbcr+j~=J?pfAA*jjy%zgl&FSGv1W3@D!H4W-_|J4TRdcsh|^y1=4tx!92*iYuk@2@LZSMPhJmoK_hPXz>CQl0pTB}$)_kk6dIT-*IfjUQwE@|TC5vAr?KoL7ZPO7kJY zmf;^phnEW#4Gqm>a|mzm=t%F8Pf$ESb^$9DrtD|8M_0OBh_?Ok27>$+}L zD5ogopb6hi0lP2D`%=GZ$va%PC zcin%C-@SVW5gIsL{Ji~ewQjHt!Z2-mlld)Q+#t+fKZcM-y5K_}08dKu1H4?v&N`G` zn3C8w_*FR8Qf*tC5s!J-Q#?GpAM5LNeR2v4(HYsvq9Vy3}r-2nwrEgdnDf#<{ zEPdmsn~m4%JN$1QS^u!G&=Ub73T_F1kUinLI$1Y>7q_+=O5Xlr+J+R)g?V`S`l_0m z=Ia_5;E4Ew2w+AsXVy$oV;CAVR8>KQ_cI(Vi-&l$t)D)r_=Si(2_25SMqxukRG;ro z&3D6d0JW56jMvM{gj&hKP(*h3{fl0Nmo)Sm(NB=cTw7CiQeI*)?wD?F^XC;t81Q@?@R2#dTeOdji^8gzF}= z)M%R!PXLzH(g?)2v|`1F{JFn5_qf>CJ8}aP^6@Qn(p0-h|4RMEUN7fWG+Sj*1=^{S zQ!Bg2uh6-T8QzxpQ|X1DI0hSu98icJKipzhPDsf370XBQ-c2cBJyt#{M%lw8GLq_r z6(_;gbNPSr$LNs&5m~qVkcR_vwy2mW95T*qyzmDHYId}dyRk8qRYa2d%uocN;|ts( zJw4s_=4`vsY7je#^BM65TEX(-;>Y{FOfG}c4e&NSEK+Gv6l*gu!CYY{w`L@xykd-R~~u&9_(D zJUPX3_!RVAaX>1LbJgF6KdiL1wQ(walLxVS^~lBGe}ucPv}(Vec=vqKN5M&2bh<#H ztlj769ko=j$qNCCX6UKaVrPKw^lCXZeg%Y_JRPT{eal~`&>sQ|jghnRYJHlY{&d?f zPT<*2s(Ia7_F;y-kx>q(yS5b>Fe%Yo27#tEiZQ0$P@f-U@3$DWO+P`#2o}FmJ#q%KgDuCRgSRE~~k-2u#p! z5;TS;N&ekA6s<$5-j9mmgzz1Q5k_k*R@XPfZf}ZP+b}CNQ122mJFGLWQ6J1*MY*l#c|ll+*3xNiU}QqKX(P zdueu0eYpmFOO>W>n1u0L$OFmEf`hx#?)ZYCRz=o1Xh)Ab?_EE=SFb0Im-CANr=_OS zQwcduYXF#9?D7?jzB3e&%6f%LV|LI(d*#p=N?`m%)o2^CSFoJC)(h$VWlS40g}f)P z2b8^_m0rJ-RP=P}ey6C7yx*oEBBo#s;B8iUtR`PHe5Pvkq3@@lmD>~d?|DQNw2Y_U z;mD8j9D)?lp9xk@D-4#F4Er*KbTq8RV1qmj3MiC&GASOEd0pui2g1SApM}!>)7TKr}(9`U^T%PpW=QyHz zNCR5QLZ1}zaT3bO7l}zu0(w*R3s#jpgoC<)rnH}{q=IWQtz~Zkei%CFHCX(1#ri## zUn!*Sd>4{|WtkLwf2!zxfkJDK+>Qc9EiURhCOkVm-sq`Uz2X)=GyRdr;S_LRy zKA~-7!-vh@>520q^pbvRaUV4a8ZCXOmDAD}TA0gLLCxya-f}9#C@M6IL-~1-Jg>^mE|%Uz`3FY0^5wt!~;cARxpL#}9v* z^`4o<2y5d>g-P2o*Zh0`GCO;d7jbpCXT-Lix&fjt+%Q+!A9w-)-#(ZBbPGG=a;1~3g=#23>jVK1Oj6MA1 zL$8k$8l_Fd0gCb0e z(Q*O_>OWo_%vV*03KT)u$>f2Xdw}mK^WWqd58(}1q9N_Z$+}XhrmC-19enz%2c#mg z^3XIRYs0i}E=@jv%O&Bm^^5bU?J|CIa-}NcpXV>OvPZ7V8||F%gM~*xt*d*1(OnzP z`RX@*va07*y%W1iW?lxlsLnczuQ(=)6t{j~zier-v-HX7+{R+s+t|`Q!PRzX$Up73 zdnUjn_?snnrpyx-4OyB;c(rgyQp~ zZ_Bs!63T(XEH+f3(@95XLr7EkC|Eb#MvjMsymb=$c`YVkE#zl9(08Hbwxr zjh8#GO=l6QrJ%%|EMngK?E-*t+!KcOMtI~`a_PLAD5?;YRE%8=m#A3!34Z}TSe0I- z=Q5mQI;S~Q%k5LM&g7i$=!w4V_)4P4g%2OlB-Z4TgweUPpA)iGKBT1aCh@R)jR)hi zm93oiyLe1r@#wAWsw)=W=L%BmA)V?k;>$2b-^G6Y=&1CA_yfA=$1Bq=-Zl>YMrJ8x z#(vrDormByXp5qpSmrjfij9aI4@Q6o?BPG9X42?pYUn)y;Roos1gG%Fhr9HR2tu0F zLh=r#R+zdm}dzB-0(Sh+G-q+{PBMA|CGZJda zCb0z9H20kHK*+qm#SG*(zzG{Tm5jE~!-UT8F%avS%+x9D| zYfsVJ0SbAkvAC}v`g@arn6c#MBB`C!H*_#suk#)Ci$v)~wP;Uvo7Tzf)0<`GR9$KNIOP z^9_%*$}RavKbJxaUiWM;YD}|c<`9yt3-j&Y=wOb+j!_)QIc#6~L`&qjTQRzLt8w9> z6d#wmsH~lq`=q8gW657}Yx$g;@&{I7OjNYe`gG2h%fylLG^55hF7X2j0`5)S&4cIv zFf)?qJ!P-@mp5Gz(_p$(Ym@>4`lW_N|IMMKuQyAaV&t_#h9lC~3+=vjF4vg6PWW!Q z2>YJbE?fQpx`g|7m}GBf_`PDAe$6Wuai7C!&j@!5Vqsa1+|uurykl06!-BXiE^f~X z7%w~83&zkU=`JppH+u~#Su_b6aR}Jit>hT80X`q!W0MOLx2A+5nz^{cP#wwW+$1Qc zj|VLh2}3P_-P-Q;2Xx{`6~_2FE=@I8w$jFxPP zfJ=L;*Vgw6&JhUIDU<=~emkvQ@EphrW9)U`i(Zi=!-jw>7j-@&+^FfuSK)ZM5MtKs zh*_Ud7IdOb@31iRxH7^w9ON!aKFd@zz9X3do%zo+448cNT+|1drCwj1810a58oyJk1R(AH7F-(eZd6J zW!C8tQ!{Q|=2wI;x_K)SI!{ztO!B|)?geh1TR2x^qNHe@>8AjT@FpoliF?Wzy|mTU zldzc<7^#>dynmv$!eK>BKitC`ViX$*uQVd2RqmNRRyX(%2&w?%&?cRM^q}|1( z!L612i(Z?Duhv8AP%&^Pf?)yVzhdZuE6oVHaHkF05S>^}Hj%)4S0ZI5oKCDd^tUM3 zERmQXMkPu?*ENVyvs3PpNj*UK&$SSnsg4-Eq148$Tbqwhv|su zzNXreln2T6ZJHo@_V)G~9p`x_?2uuH+(fVs!jRVi?>&9$RsnlvP;2t>o^K4*5`9|` z2=(3y?h$2u^60fh(!t^Ji_!17(8wmBB0+sisTPkH?3mwc-H(Iz{(BgA-x||;WL+aV zGlc$>&ja#INC?wJIQ2@)wrDbkngA*id(7K?|5DIqGe_LFHt_y*m~xJdytIj<@-#4% zAKtYF3Iw0n1HcEYMm~JFxVF8W3=mid_*QEUvbs$928#w|7eh6+L6}}_9!X?W72xXk z5BY%=QVlUMlXw4|jq|>VpN+EZmqdDKUh&Zq@Vy=om9s+=A{R+r+4`0% z_=;zXKt|xn0$F4RUGTNcU`j3#tybORTP#tXc@8_OuQvLDr5QFXl?JFCPimiEsJsd+ z?Jz39+GW0Zlug(w8?-0dCl(ej>vYv6-DOcxF^ODpI?66Twlr#?*W38ew{55%!y0dv zN>Ra(pQ>&Hx!h?(YpqzVxwi~s=| zKPST45l%>`)WD7nlpF@6W|#fSl-CYV*0yu2Uzq*V3#4Obqr?rKLBQ4n&y!sCg$ZqX z+Qa2uBx7Zuy#KAuO3V$kLLVSF&%i4Uax}_W6Cu+VIZAG{xay_Qlr{BZ3No|KI%?KnS zHa%(@3~HZUjvz^bf^e`B!o^TKwOY9Zj^bYb$y9=F%rm{T##J5(KE@vkEb+!8Ap{8? z#J0-8jUs~|3yAb4f;QY*k2AqEtW`T@Z$?;SDZzBF0!+v3UIj+7X7Y?`SOmvdcww&n zXZ{Zs^#U*;29;?aM&m@sCyW$j91k@fzz@F1t3z{3y{5gfIu(71^3hweW3*K>Oy6pI=H)#{|M z2xU#qA=8!#CQK{|l+lA)fMWXZAIu;&aa7Bf=8!$X0A$v-h(Gx_kn!l0UF2$c>5OyQW3 z*+yIgs1Z-yJ}y9$8+#~F)~$8KWce9*Kl8c*GX&}3DB|N4=_X5&uNsG!hc$c-ONRMF z9s>(ci^EasTLOwKWnM+}4B`8SgyNQ}0cgCVi^2k=2>K#sud6HLZbXJ^MK& zXEsq_L+Ndx!NV-?_sXqVq+XjE9xdHyXJ-dY+I)7;C3Gc6jxvvBIRK$skGL9Ghe~=J zGu{&lSphwDo<{;BYrCPRB{sIU4iBmQy-2*=FATK=;lqXrkFiieeX=)HMy&L3OR>>1 z8Tk(lseyf6#A{IUoai9hSSTQZPBX5sVunK<)$!Se+KzLM3WYY(O$S#SP2hd3(``vO zEr1|saGzX+F?<~ll+0{2tmCiM1Usp0D1`QPt{ zZOi{Hm1Gm6&c*)&DgN)88;BJ332k^3^NHMlwLBDLPm9$p$4$ulzc3>d1eFV=CHV5c zF`3>Tqy+_0W^@q#N5Wv=YMS}9|t5) z)N_b}h-ib95V;YrC+}S)MjvzQy4`E)?pp8G{PQbLBvzL*=Kb@fw!MG~^z(~_F?W7_ z3#Jqg94(!R(DLr7hJsz!&cLSCnf;e*PV}h)=m(N(OH(txk5h$0w_i#SqR^kAVIDf*oq#qvMnF zz||RJqx&YskX;OOPnY^bad3c_@Gj8G`f(|-)8(O{C4XSG!^%ZfCZh1#86}6WIk$t2)LS7iN!%mR+~;6>SXDm(PzZ z$F`;}9`iO+Xw+;q-0h^oGKR&5ZfEe<_TfiuVP6WH>T=hU9=^7xbZ_IMlLb&fIzJd5 z4Xk@yJL2eY)YaWMe7*WINmH=zSlIGV<66{wI6)ei$)H7z+GK8i{-q^&_h?Ca*wFkE z=3@3aIjpySFD?#i@8BRkbv^iQ!a+gCn9_frZm4K;lZ zovj+1JG+T^N(cHrLBU7nbs z9MkFklF~A>mwkM@t9vw<{C-c9JYeEv&1p#I4`P;AB8gw;V@yj^Ud5f?XC2oAD&L3s zRZ2(B3q-TRnLdGINa3dkjHRiX18hNa;2Yk@h0K!Pf1~vsL?SbI&*eTJds3$X*`3jklg4ug z<&DV~^Je8Q&F?xZ1Zbg@{S{(#MyF@9w2$8ueT)fgAQ!Xi;o6d18U?py70+|p=2in8 zP88RMblf}{klEw+nzNnPKGu9upM8SO0d0a%*TO|0hdD1(!Le=Pi0Mr=eS3R%yFe=y zq-OqW|5&i=o#$$1zYE~f4!Tc$yfk3Cd--hf6`PJP4)#Ihmcw-Ya?*vX?~&!hw>QQG zOW%yy>a*70Pe4CA`yOpiNFLf=>H=Mx?dz&ql$T6nE^Q(T#$FGvNWL{DgCn0#$6Fi* zR!v;*pBEgC8=GIu8Nh}xo91_g4EX)gKuBXLx9cnUPU;R6sMwg3wB0%G7+I zX$Jx^0O}`H2O23&>j>nVhI!0=ERIiH?_M2YJoLwZzF3??wmY?u+!?Yk zjPCm|F$1-5H9rtJbkVkV6?UX+mYPqbDi|`?A;`1acN`viCFy(h`4Yu7MX5W!+ z8FE43Yn>%5UY}eOwopi+IdQ##XD7I3TxdJEvU zA!wy(ZW!j2-*R~6VP{YLvMr-t>L(xd3TQcyyP&<)!b*>8Pb$J_0_)>gmIJhToMzxymT?{YPVY@R^#p`T{XkRbP>bN2CY z>hifc<;rT!@R9YW?&~(=lUMWPkI4o*kUJ>M;o;CZf73DWbojCQKuYt{DL>?>d7~<@ z0NIm-_S3~xh;z-k*X`SOf5TlS&CA_dx<%(Pv2soC$NFTl^6KLuE^)h!nm|EO^QB#i zySs32HMbU8U*W8FM*_z}!6}eU*O%JDBhSPjvkM_}|94ZXNmy%hQ~G(;n}yw*pGBwE zd?hCCe5er5YK?|D4j}g1<|~VpQa%<+)IE)O0^3>ixAb^FRUpo%!8bcw?#Kc6$ba*t6fLyT?k!@I! z{x)oFq?2$VIHdb~9b703CUJOgI{;MW z2m~r&)l*I+2r59Tih82N$o5StJ&QFMVo@8%KPQLNW5_I_#03%QGgq5=^~E2EP41e^ zyPNL1Fz?=9ZhUjyr;tejzHCDp*@25hR7hbGJgg7RTsbmc+6?oxq^<@iXa0MnZPn(+h# zQH7%bmeQa2p=R!CLt(PXIBga@Te$DOa4nG>#TSH76a2bSM2SMh#(09w{zOYDPe-~q zG`w_qDuG4|3guuo&_@cl!tA&q^%&*J04yD%a%Q!{!YGC~RibQW9%YlXR%UnQjkJO7 zb6G=o(>`ykc@~pwH`B9?T2=Bl4*L*_I%ZUP`dCmXbcg`6>?18w-zC7uN3X1`OhiZs zzYywW{aW$rwb^h!Xlh0Sy=&ogFqzjUuwgS{HQ~-`_;LJyF4f-Ha6$~GF7va8Gi_dw z&`JLtMwew;ADYqMN`8STDlsv!00dGkN?Fb%nFCgci;E+orKQz{GjMXMbg09os0YYO z0R@pP8n7@;7;}yi-&+gDUit7ibYd)4BDKF?yioAiM4ultA@el-=FLApICb(hr#Q)%xHd{13m3sFjt;{B?kIC<ff&f=Z*QQk3(ESP*_6#CWvRg#GhQH!!VWD+GLP1e{{lSz*qhdf_@!&xMOEs)H zhE1G*d-DN;LlL$h;T}w_h@4Vv&@{e1@C({WF@Yb1A*7Ls$HdNQFmj<>CjmzGySgMvsHKnZ7+W4X8|qTk!;b5brCCSH2q-Kqzw(Rl2DJ0lYRulPrffNvt`$rWai~LDXXey1|+Gc z<>fKweijV$ce}pBJ@7MKUG+HAE&Enzz#fHx!AZK1()Aep@vAly&dyYN#ujtR#sF+O z)$5B?D%X5f>6cQ&ovmH$rRL^ZnW3SfvR3=9fuSMjlL%C2g5yK~G;j>!g-I;VQO|>H zek+)4Qm`P-{F2~b7gM4e1P1Ln4c$Z;M%!8Yj2+&aL&gqKi)z^XtFA%EKuHHvZb^(h zRCue|%3v^d?!2y{7I#+>sy1f&{JMeK zrb%Gl@5{@vx;ui=k{?7Qq|>CWbjg*KO30=CHsFeKdm(P7@y(z>u6T>I1Vqbcg>IL5 zfNkX0LZ$Bf&r0d<-@kLzxh}G^3=RrN3h~Ul>W(k`s?5mFRFRbpAKuAV3E=zE!z2<% zXAZs&?;(q{=%z;mQ1~6~H0=e$nTse;wn>FWW-AHc8lR&-`tuC~X1h_~oSrRC)?Ur|zR_my#_ zy-O=84e@wcqC3Bm0W>w`WPh@?t12VYweqPZJvkXjiI2rlkd{WI_k{cxU21G4$VgwC zVUu_hDObWHVP>T!-ZyHrw~Q+Y*jm6cGJILhIJV6-Us_d#a~s9X$~>}h_QK=z2MNiO zoymNrD*dO~1#6ORLA%L`$%!gfR^fAGYZU6-+?+3wt*7A~%#ZYKFuIH6y_}ye3Loxh z^JL+f`+PO5yA<))qx(Z4lAMin?(h>wxMp8yBnlSVP>+xBOOHp`eeX>~A5jzwwyFu6 zN4|*8fjm-ynm)1Bi7Xz1Kw)I+(9q$2S0KLf>m4WM*aV#h^bHw#n@DOYzgKg0rra$j zD|)zYlY^?+EaKv-5@`EWE~+oB!ZUcDz6Ivz3$RIPzAh_M-wAWuZYR&0o(4O-anRES zTop*QXklQQ2)5KqpRdRysXN=wzs2gKR3*7Yj@kBHlFJF^G2Up6%qoCJTWG& zEM9Fm?C=IBEj^zb3+wb{$Xzls3+q^PljBeElA2C+d>TsPdKu`IT+v;Dix>?b1+tDjQP zP^D+h-Eort{{7n?^nqkNOU2HhCF|&7UP#45r7^D-Qel~|m0Z3@a`O|*!jR?pqqj%L3PrOa&Oy>g1S6Z<4TR%e(hCi5~o z@~GnyJZIVDWxq&*MV6M9x91;#dSIDKmY-AW5#mQHOn==QxQOPjH(!YxBRe8HfARik z$mc_<3`2x0W)vv3rU^N61Qk1<1WULnl`t}8{3gJ{$_(p8!Mpgm@bf7L$LPN8^i7c}K8-}JthAci zm%NXV3V5bg@?5hJ3A4bDndc4{j}eMUSFZHtfXe2vqm2@OZ!%xuVp6*7QqW^}pgq{Ruh<$R*yZ0XDXB~%Y zZE|uPaOp)SrnVjaZUIq-atFe51|=EH)I#Vzw%+XXcT)_Ej3xqt0>3@Cl46q*^avsf3d}@( zA4{suTV6?U)|zy(T3c^&N)~-PJ#{A{r1&@`cea~%sMhAS{VqD3lA1b=YWDel6d@7i zr_p-CIjr4$3196zjhL*?&do4nl@+T`V{mZrO#x`@lXUn|?Xiof-+tl~l8O1P`7maj z5It6tf3FCJ|5+mARp%uWeud|fyFrt8c~_z*o@=`ue>ict7b;cZ48L?aU6s&dL%TuW zG7zD?6U&6`-)TrlSdc>bcTfG?>|xt2R^Y!DP%?LsSaKB))%>jAZsATq(Y$@|B`Pv$Cvn$)dQ z=9FzdXs(BUfPT@ov_xl;sLQ^`-bc$w z$z&ikfe7tibOtlM-bLnMXL{spy)D{MP{z&X&0lPWhY}J6%rh`BVB4L{Pj_B!R(}1u zd|u#N0F?v9d1PX8|qd(MYB*~J^<5`Psw7#AlTS=9E?N` zNOA?WrfA)d4}-r^|6jAx1i}YOBYyCkzkm>;N}4>yidrXv`UoNr*+q)=8hua9@b1oe z|D5q6RnCIkTj3HY>Q#J#f&3%Eo5f(kqd7ud$-|x(J62kd7-d(Y`ef?_n_8n|U2OZ1+Q-$&s;ID`L>HIl2 zr-CdE-y?PL?OQuLx85O|Y#0Hu@AWOA{1O1jf^_{+##MkQnFq6*5-hccAxsW61|1)Z z+CM|G=8CqKt*xLmR1GEfgf8$8XzKw9LzMp~8=^p6iv$J&8!V>^Bq(2Fh4jzB*B_78 zk^TKl`aTzUjuFkv7Z?V|!8NzIMkOA>!9~uGY(Ozgp*Ja<3PTv%<;jW(2odz|WM-<0 z#{PHGr%#-A=H}grDJhPsqJtca3`ABprg_(DCd$fkvdj>OnspQtz93PpWF1`mbu{Cq z>EB3Aidql}OjJaks5o2oUH^C;dhUiSqm70_(IE*(W>QdMyA5t ztmO1|fwAF*RYD|Y95<2inn$NQ&A9i|`GytgemuezFn|8lP;4v+FGYf%znC%Yg0J{< z@7t)D7_L9ZaS<)7RX)B2nF)Pu{QUd^Gw1ImKR2FLpHIKF{xLWAsO9CR_kGVT4iC&& zjgbsWNkyJ-$bHVG?(z9EvqVtVz~JCVK=0D&) z>xv`r$8R&{@Ob!kA&4P>ZSxn?UVo}1?ybVd#kGIJ=(K;uiVolk!2oGe)71hEP>a!~ zRMge=7_3-#_fnY@Q$(d6?&*FkY!T7YKkW%%c=}W~qy?MXklN5xs@|GxjQ~uGM7>NL zqsl86NJ;rh#qSm5siQWJ8Wkj#lflh46j(=cX+~N)nkcV*yU(j>H%6UK`Sz^3H{ei zk%dNYUigg~(dnFDd0m~Rn7H`t9pL?)L>%CC5Na5z6ss=LbpYZwC5O!q?Vf;0yu6fM zD`GdKb8=xX9KDUEmhk^}BbqFfK!y~DGn|h1kKY!onblAa z#2_)5&0RD1-^7HI6=G(F9vvOCeIKRT_@~Em?|AQRWF&O4!@Uy;)n3ku(#q3=(L1mR zS_v?JA_b7@e5N`&Oh|FlJCmhs1Z?to5{?oNeOu@jss}5XE;FZ+?!S{a++B(t^wZxw z@|Nj+tWoON?*q+Yci|5~C|iFbfCT8@B_Qsfa-^aa70_>T-Fa$z?GnbvnUSP%Ct0mY zd->u!LUkMcSMLUfhn10x#xQst@x|W3;XCBh$OT{e0U2J?@0Kap>2xVA0sB_6uvzxa zIkujBJrQ)|F7#SQ*zsU%V07lkDJ)EcS|ctcA^qY&e>~dT@ym~b5b296Yo<}41u`WUi!oR%IB z4tlZ(xfdlp?k)5EAcAfh5oQUA&NttTgES!3wyXf&Txd_v~-0^m{R3`Ff z@{x119nT;v0)@cPT{I$fxxt6skVm^ZQQB#{X$559+p$4(Anfqt*&2#Fr_H`~^dh5G z@4x7SlL5kE<>Yi~Hc<_aye$z2mIYB>H#1lV%+D-z7i9q5sn(kf(At-NHMTmW;NaX@ z$8f?1&f<_Xo2!%CDz?quJ+GtQ_~MRF*aesS_C$c>>1Lh?Rz6n0NzsjZt=&$dwUr=Y5x%5 zw-GthX9EWCJ2joE4M>Jt^FsOPC@F)<#T{j_&U13?xzTi4wrY??B(HDVNa+!vPuj~X zLvo78Ffm0_$jV4S5a0za+ruah?-Ay z5yc=qyH7_xO>V+R3nQf>=4x3Iqn-pw8={0u|p!; zzUQSg*VhZB5=GRkQ=rL|*@F^^cmahi_eI7-z!Z%xRp@-Q718D@>1;42kOVFLH;VwL z?1t6m7gGzWuExIZZ;XAq`#UKJi^I0&W_{!VSKHxa8Zi-n@n5G15wR7^ZNCz`O2R7~ zOC*8(mrCP9t_QULizhUkx!!*B3;P#Ul1COU$lDW8{NckU791sy6#Dzl+4-fh0o&^8 zs(^sNr#!angN&a~jG8?DvJdRS#(u{g(M|%RA3u~ZB4<&3hlKRa78p^kKe-GdKzzwj zP62w;x<$RZ_4u)gf_~lJ?yI7#+(5K^j1PtnYv$?K#dO7BagdVoruwd}iE=<-I}vhn zKNc34v+K-|IjlZQlFSd}SSdT1B-+?cX1%~6Q|ZkMWO1pXw-9jb3^A`Y&{GZSciuZX z@bv?VP^-L-LHU)z%wbk9{UXii^AB287iXf^U)EgH5A=k$kof=)3amH=W#e_&$kexQ zse`r&&qM{rEaWk8OSx;VFvRcLzqV)Q@Nm4M=Q4PzL$u-#`E{6Kenpqje2I^zilMiM zOrbwEMmmOwyBV4qYQ6s11nmuUXbm#{jcM{D5Qol)tCmZlcRT-WtBuGLiu zPWvJV+|p77W@kH*)%c?9FWr&!jWe=njjyO3$%5@SaU%JSw+s;hhB*U+!<@%G)kPM` zY1@xbTrVH*(W!3g)U3OjS36y6U87pYt!!f`VU`-fUC&xzdBNPCFRe8moRbG&ggClT-K*M*FZj%ECaVu|N?ytGNcYdL9ka(6PJ z)`+RXB_vGC$YXzTzROj+xA!^6epQpC&m7(xOQ012*RxfZKid63E1rbI@)sRpPs+yq zPqGIcJI&bS7}g<5!=iPHLdyJxwLQJo%Jw_rz@n&ER2mAZpGh(fXkxKSe zkt8h@ioHF7fJ>*<90D`E72)HE5qU5pxCUh6?%~n+jr(b8M~6vVvOlw=VH8mF#y^6; zP!pR#KP4t3!^ZC<-?Xv5)sPEz3r=`^lRBo;Oj$Lp@?ntvY4o;?>zL&T6_{TBNAJcBe zn3yKELNQo^Wx2yTX=NoPCi>jwpKe6d@}=YDzKD2UG8{8~NQuhhr=_6S!@MHmPN}Qc zsT{To`4h9^D6hn0!0LSvb{QF~{nDhgq(8qs3F!o?3pnh}x5A`Lo*bZrZQs!aXDluG zAF*?ca-3aG(xhc)|0Rd?mcuLnGN5q0NO}-ICUFE%zI3Zpp2QE8Y4-nW0rcxk9?4Hn z8V7^_^3%En5^Pt?%X%zdvi`i~F8^p{*bHjUVJ{|dCPYPoV79H})qo#Th*Itn4R*8v zvRJzJLe-gxiC*yp!R)AEAcEkP$rGJ${!kCb3s*)Kes!yN4@v0gSRf)WGrf|u+vI^&=g@16cqYP)Y;jYQey=KfQ`O;Z=A9G!o?KP3Z_0gKVxPSH=gPra&h|> zjf1Hfe*pf7vGTh!Ac@lu2C&23MRjTwdJiqkLKNLUy!|koS17r2n#y_7@sRcODTlw6 zL4y(XO8Y#s)SqVaQCnReQ4VFZ^+x4EARu)0U|}%$@LE~VRQilD5wGuFUg5>qNIh2| z7|@F%5d5&DM_CJOUqd;Cf)kV zUZJMu>-188e55JgzcV!>wJg5PXPpOOXjLLFkpSdq?$h3Lgyes$+Z~3xhfUT9tfq+1w(KemQ_F~Mo(UQjL)V+An zRsWs@nG2(ti(L2hlHqr`A)+?*Lf}Qxs$p5^Pv_-K2;3zGF2H|pw2R85546^ZARk*M zzVps}9fd;3;V9k*$etV2nq^*I;y=fd_v7GvRb+-=8;Gm1ZQX#f1R`L40eFmVK^IC{ z|5P)>r*Vxt&q&b}M`Rrtlb4rAf!BmQU67_4a@)2zdaPR)!Tnx%Y^6`MR)xKc>#HE0 z`_-~It-VXk*W2Y+6nQ#^e2TuGBHR?Cq%Z#$yD8R1kfF=|ddkVI-AC3&TdSux zIyi3N4HU2C$Vq2AIa*^`Tr|Qy-Zc-d!Nwa|`I+&=&w7~@F9MBm`OrctsoAD852}iM4xt?*qM0(L{mr^Y9;s=ukRe3N;(>1Q`(I-RNoW3 z)L=P)7y?`($-w<2zz@2Tk$79xvX1KCmXC4d7hBgZCz2$_TFFKZ;LkoP%BQF_(r)yM zeQ`Q=TMMT;@5E8VXP0KVY|qOtmUa+)J%ZCh3iV+@G8W-uKwR;5vF(krs+T&Rs-0GD zf0_DYQ`fP3KhF#5$m^xM{;8M7>qWLl|4978spw`mfgYQpiI33_E4T z`ib?*5ItnhUG_-Qop-5jvhP+e;H^av2YZJa@6Z>!CBdWI1E1REHkaM6`3FAOHoYVY zhka=1C~jS#hR7xZtCwjCAXI~7yv4ENiwlnKgo$G?V?6~G)kbfNb$=&TH+u55zh34J zDJB_C!I#!bC1Fn~ZnzB7EVZb|&znU#tJ%yEVKwWE2z%gSlIe~Jf+h_mC8dq%-l{?* z))d@%FkdsrC%<4yAWLECkxVQ8(S>H=Ilipl?5Ax`df21Z>(nNyf%7(;>Jy9zFMQxUsle_D(0GfQ*j#W~Wr?E`T(i?_G0^}^{{5AXI2 z^q}7+AO98+N-)m9B>o?&&N?irw(Hw7LxV7MGYnl4D$M{xNl8jcqbMmILk~T4BhnxU z(kK$r9ZCv{bc)g;CG~CY`+45?dk>Cdi~A2?@42pZt#z*7TI=ji+|-1dMLeSvkW8Gk zRj_;Vy+U`Cfb4a(k_Y0C#|}zjHJAT>vGct(z2KA-h1;DfcX3OhY9Q1t*qx(st48>)>N{U zp3(O0SLu;3V(m&&dLkbjxUr>;Y%C#e!;2}xV_W0`NvPCHKqtd8kb;Vq3gcXgpQFN0 zTiX$0sTi?nWXB_c={SSJv_c}mGqf4YXmQtlD{g^ltlgNC#rJlh(RUl6VR_D{m+%Is zDLY&K9&QEcv4{96#RVbJr}q3B;qqhr zzA%DySYtTz0d4cTVWm-(spPKf$(+x6G7%9Ga7wG=5MYS*eUWbVj^uanK#{>Kpen25wK~TG z_yok`Hlad&543Ws^jK-J#WA+2&R-*rH^G+k!SRVo-<{{QlOXF4>3)v#r6$Or487|e zN{g%8a=sjZmaR6joDoT@V94N2z~A$teNaNA?A7%%!+1O@Slfj4D10tSm671%;`HB4 zYio1{$@qQ4`O-`piUoVyd_wS=lbIg+SvMw-}BxFab_)%5U6TZd0D;ZjAI%X0FY8X|Vh^Rw<#b_A>G$?nCIBdH}0 z#)5YXvsd+;`UFJbR1u}pnz33c-8oXePBu2L(rr2~eSi>YIJQ6ZGxZzm&XDs=j3gi5 z;}zIUVT2!{zQg4Gj!>K~JZV^q31tu4bdq|Sr%yK`E|g;TNNgF%wCVH(4hE_g#wyZ9 z-6wy}KBSVG#O-%1AGa@!-UItXm;5@)|H3SfQJ_~wi!LTUW-hT0xjXvoTx_N>@o^^t z0-}M*8+~tsiX_n>FiW9rCnS!L5=O=Xlx4VGUn;G?NJ|jlpNgAYdY+Y;Y5i7#zrakS zA4?uABGSM;7`^sonjbnb`O##S3WmQZcNe*S5SlX&!jbRz8Mjqb;h02-UB%vGVQ))+ z2L`AnbZZ+gbx6Sbt(h6(iUcphNJ5ZkEX7^ftKX;OGz@Q~evaluiuIraBM&V%{2d+Z zBAmKNg4p{&K2SOE0wFgP6sZjA5|`@?eubmrxdszb6@K*UVNQ+770`KruVJs_YX{4d zCm+^tuE2$Iea~Rffoo!DxS3SLmUJavSY*4gI|67nv*x{dfWGKn1)mE&n&w`MLSrDo zV7!52_YJ{;&hGxn7*Nr1#IeKktC=a|>VE1a&61zKW1=qN!rnNFkw8Z4{4%Cmf2m`G z6q|SGGuP1_qZA%C#F#$OE-BXaBD1WHPRDn!Gd2pbE$d?RnXndbIq6%4@7h5=jF<_! zHrsk3+$n}l?m>x)xhuezR(s(3O59_+D1tZSJ5CGd7wILSd^++$8Y#@woX)xBI`ghY zySiKnD&$6La`bxQ!fWW8|s4WS)tH4D`Bz1%Zz34^|7UQ#Qw8tYTWyyuv$xtblkEu&{TTWzSXS zp=6!3gb$xr4BtGJ+f^B4#>^RlEoX%t!+MGQgU^+hCXLJfZUc_w;iBrK5b+qnkdKT zYBvXqzjqvb`lJ3>gpQfvm2=x7-^Y)~oAZ!Ii6xMxyRoREg7gt{AeC)S;=JeoSc>s! z+7Ia3AGCM4-4Vcw)>0A{47cG3c-4<0{)f%-7`g|-q7uQ3oYW>g(v~p;1P~J$-|aBr zxu&g7YBqM>`)cGAtAM5}nA1@%XGx;7TMCjX^#WxSB$NSpU?KWzC%(otsE0^^vC$>t zT{u?M71zs{tGL^*9_rid5PSWvLCb%2N&n&ZOMjR2Etosh3tC&v5>4rkSVcFyJ-t-* zJn9Z(rneK`ylZ*_q_v-;Y-umfIAjFO-Tyv)CL((LEL8mZyNcAkd-TJ2J`SBp!CY-m zOv+mr3NP-On^tux0Ez2ZKLmF|(Dta84EpydPv(FdaZ9vL z{Z0nsGEJb}2StN>L8k=(+fZ9=<dKhNiW0-Zd*xjuO<+&p-IW~9su zyzVX9N9P<&tLB}LpW;20{H~VkWsgC&sO-e_V14$V7jcW;^H8I``f$C~B)XlwEtkb#7 za71I5AIq1?JsE$Cozob>cAp7qh3?4(Q`$yFp`!+ZxG)5<)Z$!~*PHt3sh#(|Ss(3V zNGuVr6e8Lvb5FzwGBEfjP1p>wYrTl}g6TEe|lppN_CPOA-(MiBq-xONg zx@h~{o9|&}l5Q%XV_;lvU0P{1&AP1ZoL#G8Vx)I_DG=27oAI>kML=TCQlA+^CD$K+ z)5wd?RjZTjAAcK~*=Ucb#SIKgi$`w?su<}BxkZ!~W?R*%(Pc`g&$Z&X^T5|x)@1ew=Spo3Pw?^#ar{2zxbWUu7Czr8t- zGw2TO0D3%dLFD1fpr)G**}%fZ*H|q6n%OdJw_KVGV$L|*&{`E$pM>o3wRDl!ypwp; zfwv|pe~J9@SI^b?IJTK(mPzwiSjeglEwvegvS@Q7N20`o4i13>Hq+F1xQr|Lv4Z&D z^M)(8^C8cdMBEPe%;7E%U6@0-V(Mq>zds*bIUM|ysjWV6U`cMd%2?_%uxBNm&j2@| zo2jX~uJ_`$yeL@=VkcWBqyIUC&wR#6P<8NSP+Of>5_#!{dDg{3QlbzpwB0s3l36K_ z=Sa4wFo|xJ^Z2#Bt83zD_cAU0n<()9zU{f@Ls^7VeJd1Y7XpJxe^*a5@fJzsq{zPA z{g!Cdw(mml=88Ug_*Daew~>$enR|cR1rDhzivXjTxD##HKl;=E7K-!dq-zq_0Bp0xQh zTkai)&=_Pq+vErk+6k+R9+KK>hIp2zSzeQ~yUf4jIDGKH%*x{tEps^@PS-DVh4KQ3 z^kh&qYO`SM>nvBC#<1FHx_z+{Dc$)&y&Orp1?fe>bMb5s8q%+mU_SNBP)TexYn z0Y|@PuI}EQ?Jnh&RSkx3JO$2N2jvb*&<7@OwBc-lr^AeAg7#|&W%pm`EX$>+%O0Y}>gnmm6{kySlUoo{_Y>{GoQ{*7jbN@tyO*K>FwXR zf~|#H8o~C4%{OrNN>oDM$3WWM^25MYja%=1KrZ-`%h8D!3sQXm1!LS?9*6daO6SE~ zk(koDSq+4<&_q0bLYr&^ukAPwJ;K|7jxcA-($e=xlDh$h=zkW@;~bg}E0%t&u*lL< z+jUMZ&OQ+{FoKK~7U9zW(8i@=nlkR{D}EYw!PAM=<$@CIHX>pIeHaxFmJ0$Xu&;VC zCuZM?Ffx+svn;2;&(c)ayShBz;62nu9`?~y5MgO}P@7V+zBc&alb>1CxAo7~ zpB)%*V8oH}GSQ<6k7lRW+w9lfR8=)I?*xa04dMUrIQ&ERdu!)w&T@>JdTLGhFPF0c zSB==_#|5OMh30nv zc3}2%G^?bFvk`B=XWK~D_p->e%}t{hszA&3hy>%`)A`=l8GbH;aj?cyUX{lOos)pq zdfq+Y^(fs>%wlW5Ia7%1E$-^xu&`~zXKa#?$UFl!p@eq&EMVK52q&4G9=LeR8I|B1 z{oJhbOQj1TI;2GpL=G&I-+#ZB0$qF+)`Km9IxCHE2r2keo@s@CX#<5TnSKGN$LbN#iOm0D| z<3q=?cJTAw_r>(AL{U0lf=>|s+hN%rK7E5qBLO9% zpYJ+^L`AC>nPh#`GBb_!u#x^ZOtTiRCX%0C?QZRzlnJ4+n&-r-O!xqWCOfC7@u0cz z$m^jax7IRarO=)*YyfF-a?I4ef%TjnZ%u8ofWgIPqM3knDe#m| z+)wV1)8$e*@6E9s{iNQEw@aM4IU{88H#Nr|;HU+%Lr;urjk*bm<6;MPX1&({sXOtH zFjc$zlM~D`--I5QVz~)sXshFQ1KyDXpyDMvmTQs_*6hO4sz%6b``35V?VJKb> z?_|exzE?25vUpxYy>m)qqd5NtRQ9@JIo+j09XmwZb=?2u8z73KdJIqQ=jQ<zoJDi*yQaUWy)EOeB@R&Bs6b*r+wL%ILePHyA^rGSZP!;6n! z`0l#&$`KKa3#spqGy+ncUKWf2>0`JIhzO>I2gZl~QU>aM{-A2)MsOa`d@&Vr??j{! zy&x3RioeI`EwSr1x8C^K^_6^tC;R_7Be#!2$Y6=9oy#6W>z@r%t8tWxoj!S3Mfu8m zczF%=etrcsWmNS6!KmdSqCI)yK59NX`}sg9_#S9Ici6h0T>43e|CqSJOvmQX4Upe9 zg++uRsFvm2b^pPv4WsRP$2~yjo%1`0a6WL8FHj3`?s+5kdvkX@QIwQwA;`Od^8_a; zXl0F#V|@7|r~^5pz=(;88{dC4%|W6|j}8X)9g(WxSvitw+AeCftLb~jg1wDQPX`#yutX_n$ZLgmiihOR>+i<+AB^MdcQ7HRuu$58f<1d-0+Gld;? z@`23h;D3il0!knD$LD$Y;Qcw>^4yO`Z{Tc#M^VUaC!j>N~I*b^^zb^zpW zT2SEvXa@)KZuT4dv1B5*h!kU)gML)A_FBKDpkS9_F^_}vNP8_PcmOi+alg)#XPqrR z#D7-Yb(M*I9NP4 zC6!y9cgXsDqBuHb>+)Wcz9HA*-nr_i7(oyV`PPBEd2#y>@e&x*hCa|x zK4UzCd4U+BwJ={73;q2GcP`EisTwQlZMW&fu)o!~WA1ZKe%s9vF!___u=HE|u)6OtH^POb zKb(=q&A)R)Q|TgBxiC@@8U_15+AidX+6;IH?X~3e{PON14G^YIl852#XOrT?DbYba zxL1JKoEMLHgx2(6IIAJm9j`hn6>p$8W_WB#x|84;o6~s6IsH>uwv71jEzk)a8Hr4a zpx%5NxSxmrF*PM+Riw#7s4tEbP(}S-6rrozrB!zO?Y)qm{FMEf7qdN}I~vH7z6hK! zJApuUf9tMKOkKUS`=&_oNqqme1KVnopnk-eO9QiVIQ=Q#Kpt?7~2F5 z#|txtkf^v+z@VBsVst3PlGFvzc9wSf_k)c(X?R&9-vfj=kifRy*-X>bX;WEs)rd&2 zcX&1QVFRceiQ`-q6>sWp&4*ivI?uq*n05F2@ z7NDGYxNV`Vr&rM10N0p`^-B}O_v)nMt#75aR?={4MoC{&pr2sk>A(%RIaJ#+x1@GB zTR;c0fEOO_F1$t|nN({=p}Xa=F7d9a>716+Rv~4Q_6#`}V)Nx~+!phXmFWyG|1d{+ z+PiFSjgI5fQO_o*S+ZXmH9Q|9{@0edW11+E%Iq_4V`Ly`|EvqcNB;J_f;ty_YX=Hd=QIK9^#mFcn-_+d=_7_tGg!Dh{&7bYhxX^r9MeN2a=jfw&y?^p^8y1?9Y7Me-ztpX`OqgS(uN7PklQ;=*{@h*pm5v+i2kiy=@9_~2GXLEsg3SercSHRt!qPx#(p&sh(<69GX-tBPx6 zLvXcX%ZXdz%fy=VuHw(pgFtbdZ$UmA++eya?Q1qZB>2Jx#RG3=>hSFH?%AR@ef2%y5 zO=a%mba81BhuYn#GONyfl!Q(^Qc@xk#={mpll{0h@x9u@oAuEh`CsUHk_60{i)d+L z!~6{V<*+9{mUS1Y)kf1)ImlXe;6>PnVA9}3Q4rYDl18eFKDefy;#!C^@iQsok^toX zCqR4$CWa=A&Hs*4jY>{U%x4$e{aw|<&DAAYpyMLlwX)QP3e@>Rr=-t;9f9h=eudI4-4cEqz z@{Wehtl9c=MZ`<6j;ke>Fipwo9Paz9^3gFddg$OtP}N6R2OSL8X4G68X#Py&)!JnC zlX@hnv%pFpq{ED1a^aKDxI0a18ixoM)diZc2@6y2Mu`TS6@2L+ugAP^{-9LHZeXro+MkRkEdOgrd3 zWq;X~3$n$8+`Ut_=6!S%gH=+MY3I)qua|A9sg%e517TyNOqF?7&qPIaLi=$S2}a0G z5!VmZz+uiIC0M)bGnY99Iw2j^D+o3OB28OeL`0-YT;hR02tm3z9pK~@rQnFRUEUH} zlSM0HIFC^sn#cS5{rFDKr}u~pAfRNb$nba5R*K{n&xKM%2Q*d>Kp_%|TUS02$JzZu zmt*Q|iIQ3O-;8Xs?7MikwKrZo26jGywo5*T2=-KWcvYa-vbJ+@>qTI)l($(EA9A9- z)-whuO<5LA&j7$-*f6OW>FAwvDK&L=hppU`5v)%1w#)W zURtQq49KNi9Byh`R8XOu6al~O$S9M;2-Q+Y;m69a{SdqlA;p!1A++}(C~jc7fJ0;D zb+8;2L88ZbtGAb(5xaL6v1EV2oBpErhb^C1J6REmN)Oux#%i`7V9;c+rC}_o2TjO1 zX>y7OBc~j&L&PYB_<_3?J{iqQmuu*11PiXjKcy_U8I54?0`w~T|0;#+2+OKxTF|Au zP#7X{Zq9UgF$T+&y6_5qDbv-;=2O>wufS@Z1qm0scX-V@NiJ%W5Ff7!v=sMM>F5TV z-WAbUDsE3}Xs2mdEE>Ra({@%XIQv@?oKby+|YoP;!ooci%qa zc~@37KBlHUVX9iH6bBnDNy3A$yq;sC&-}fw^ZE};1whsCn-J-sZa%_$pYoIKAIi7n|@a*4y3ERvOpw<5tP{-r>x367rNXJ&NEkF7~w2Odo_C}8D z*f~2G9)MS{yZW>6<$+qj_rB|D8f>6~g2QbIDyAATM!X4vqPInahm!D~R(+3-({GH> zzOP+5`=qhQq~UnFjdIE$3DUr;W3*|bzkbT|RrExJ^$H-UwBB;a5+1BC5z;3WNQ?z~ zz#gMi(gB@|3Q$8z2$gfLe6048p$N9x(8W#By<8QxxBp(o->c?7K-$J$`D9Qsffu+3NSi^fRlF|F+Q&k~#eW5Cax*3T6xL zbNN9Uf;N?XrI1yga8tr2N;jT0-GSmnb(UAyQsJgFkXsu#{^&=rY*(nKS-iH|GyIY@ z4|^xv{oHDj3ysg(Q zqvbK^&p4IJK;esC(V>^iaz}XgBYsBiIqk-OYJyjZ!Wjs~D4lL^6u2lXM|2Sr_}$)U zaV!{l%X}NTeKPdRP!oNWQxiq`4%AdqvEdSg{5G;&j zP-@fx=5G9R8DK_0FTr7v_4T4a6>iw;Amje8ho}xC_2p%|kGy$6d$x{P)?1DexnZ^3 zLGz?-;7}=D@V$T1x_{3(MqWJJg>ad&iznw)n|HPHk-b zyz6z|^S<3_&+S&RNxE4h9YvlHUF@PTb9wGy6%`fyb-zG9SR=vd0=o-^Tgs^|EZI-% zhfPkviTPH*X>&Rn29}2SV*kPaMK;EkuZ&DU=dPe({FLMJNhpy0q13wf&|d0dQ+eP9 z&?evXU<|mr=_9zObbSibKBeD<6X_8Wg;78$zear=9G{G~u2iAc;V4v7OaozoaUplG zX}5VT;UE)G{R&|(RWu)^A$OIwY3&DBoSyD7{c^4l zT@;)u>?wm(>=9$A`~&1_a9`*&c6gs^T#CMH_Lrfd_p!0D^KVhM-zuKyb+N%l*?4$V zMPNhZ_)|^K!B6rp@4|aS4Ki(uqYN^)x78>rji#cfP?f*N#fw~3RufZ&m6Y}31joTP z!|j*L56-4D2OyfSFN!1Dn}zvK00Kuwb#?U*?>S)|XL_pzdSh?B(s8VV-`8~j z!*AkP#iq(>ylT#}nIYP^0-N`>k(tIMa>(SYEC(MSAA|P505Yf&EE(dBvr9Nf)yU^! zxK!0)7VP(&WKKyhsx-NsFUat^>eB4I`OuqUc##+r2~09ajStuZ8aT}UTZJ7^nW56>=f>5Vajc_5d$% zk4TB&E?7XVliY5MLYpOD9vO}%1XDqH$%#u7^VkZNu467~&IJ!G*4o+IZghV{;f>k? zTK4TqDv|)bP;20`RWpXeVW3!@2m7bM`gL3%N~?NdIrdcH0CYZ9XZM&3KCOrtI7nVi zJBxpXrUdgqfDMGNGVWh>ZhF*&f%E5&hp_O|PK2LM^G?unq# z%R_VyIv1T)Jp6!Jt1jaNo9BfsRJQqp^g)YQjyg5QI#?r1l&G{Bhqtx&(EmpF6Fx7dDk_`ZroafLPj(+j4%4hL zl2K(;+m!%yoBYZmo*pous&#yrpHd@h_uQ5UXuQ(hboNfa+@Qd3Np8Xp=t94)VfU@# zbx9eIMhV3o0U|wYB1(P4eigrY%k&_okl`xp^eyljH1#kjZFm7vJ~#|QfJ08jlQ>5B zai~>ZW7Kd|3#Bb3yxAUz~iOAtc zp-(MOfA_^}Or$tVzQ+s>5%qNY&6P9WzLxZ{9`N89Sf`!1IyKgVfo_Si z+A>JVnj0`4<5Ce0b{~;oizP6Dr!Mg7?~m6uxxe8-XSkjFLwwcKe|~?hG^{rF0=lGM z*zYLAe0o%gs)#K&-BOC!Eh+;BSR$wM~V-A^l15OGiUf*YM+C<8SnG1T;Cn+gS&jy+`WwV)b+vD#c{$Vd~`Oc3Mvt z!ZCPMuc%42shFt2mYb)Wr%5HufQfro7F_WYXPAioM)=z9@@rSc692!}O|F*_+7Z(k zeLCYHXklcM%r7Q53bZM)ao-sJP-!wO>1FGjBjx!A=NHF($8@@Ynf-G!)65gSO?oK$ zgiSfVjw{c9B3K=Hp)S}u`6C~6X#weiMy4-JLgE23g(n+|=WOjEJw^ z;llexdOGKx5XU6-P156+rP#bj%JKG(uaDui;7ob(Ij`B5ZE9ItW}Z{}^)H6$Z|s*F)vl?{$VsXPk@a8mL306VrFiK{u#b7HMbf9S#}^Gk5rtYHJbhC(13N& zAw>{O`?C8WgyUg6H7P3TWGZl+Nf_|&^I4h)1n9+tov?#*`mk=TwJ^*Ufg%25n>{K1 z&Q*7{==OM6#(tuoMKj=q^_B=mY+#y9?0?sD7m+2+HRUR(5-E@)xcARE zegZ0X)N+qN2{8uN5!e?B`J+4@8+|ZHoInV=4+;la%G#r40{)^h*)UPn?bkDMexPd# zEyQ?dAX)`>eRi<=vIj$^c^_$1Gw-+8dM#ny3AE021eRD4AJsH>$i*O6-gJy$+pl(J z)Kah}0%Pvr3g_Jj-A4hP z{41XI$DaUG&Uy!Wm6S82{pzK!&sGC9DH${}g=;?xw$II0dZ=>*?9^E()gX)X)QM;W zV$~3ex(G$JV1qvkbt`P2Ld&U=ldzD$>Kkt?aAm`f6#-t_e^;HUMA^ZU)en6(aijEk zA~KN_bW{Q6(@%O8>^DuhU|--c$tO0LNL^l-5e8bt7=yw@UXe$mI=D;0lHR+H7klmP z-=#41#;?=9wqG8`lstKQ1B^$S*~kmpUAjK#XWr|PGd9i)8jZHcj{;-iI;Ol8vMP`{ z+bpj(<-g~D=3>Ee1q^PR1%}Z1w^x}p&*a@)+St5>OKM9Xj=ns|rH}Y~r<6-vaC-`P zZ5z=MF#?|kqQIgUpakf=&phz_ITw#D!ztcaG^8--<~n>vp^R$=D>$SuSXiSjZge5& zCO{VU6!&li*wCl$=@AEI_#Gc+v_JBbCg`b?Lb>UlzI0PrPaI-NQ^-^yu|qJ*%vamz zhb3HQ;)y`*VUvhbAGxEDpEy&r;}Gd7+Nd?ML9M&8vaUAvh(tPT%}vv+2V7J2K=r9W zj(0SQdrME9z|fT5l6f734N?Sk_@XL9_p8ABH{;6865LvwEJoQ}xu~vS6f)qjE2ZsM zRrBt9-{Z|6Q-&5hYl@BXaPGr9v_mqKyQn|o$~h{Q78oK-B+z7C@<7O{3)@gCU*!xK zrFZ|>+Frwo#?zzWrh#*6tGW1(pXA#vUE4v+i1nC8rBAcJ zhc6i?(rL+F)s;(*NS~Tk6BTGst56c9rlT;a_NqOh29kq1yt2xw=*9)HR%hQuY)hgJ zYBc|?p9gXTPeW>OEDeWO2BPu2`C;tM#4%skij5&QxNN`&7sJQkuda3|*?zPkSmSfF zF`LeBQmJs^WxiqM7Z=ICC_<^`H7BminGK)PeL?|USo`t8Mp-M&ymL&g<9OD&eLfB6 z5{JuP{Lkc6cTMqgdayfr%1^rE-iGADF)qZ^pRD)M$ z@sX8YkvgpYg-ZWP#JVx1!NFO>+r+9mKbdv*3*LN_AT!f;wnAiRZR+*oxuzaqZb?u#rQAk6p7AE8=jM?OEKj@i=;sy1qDiiNPg8 ztW92E0@u8xXN6MCPjiZozo^YkC5>N3p;|Wp6P4J7zYbEvLH7YioX-x<aYG{Z!d=3NFn$z`A^iAZGaaky@K^&SxXX+>eD@(rj{&vyu$S*k27e0gw3) z;I{`W7Yy`QuxX9_=FUe5Pd16{rHkLy@%PrV2Vs!S7!#+aUI*zh1N?}56}$^Z*B-J3 zPD)D3>If84RgSkub49PvI0`0S{ih3jhg0Ne6j1p2bV_k;t4aF;PNE34vjvWi(E2RdDU` zL}B!6;Qs5yo2$GOnud9=tm?#vOz3D<(~uBea5(n&P}(J_No<6*9!1e($XaArK83bT zFr_vrotDgbQ3DaM{jjj{rn*=lDk+Lw`7Z&7*Yi2{S_d%ka0f7DC45X(ttT@~y0S?x zqC7Mwi)aCN;=_{!#NmMG6t*=s!}mAhCrh27t$oSDj3CG@Cc$&N$!}ZT(IZXk3DD!Lv)% z)Oan6f6=`#F9EX#>*UhFh1)~}4!Mef-N?kr1^-kEeqmm)bjH=R#Hg;>@phM@R& zHl-;WAf|~8ceTmxYR}L^brA!A@#c-J;}tJrwN@O!B$BbP$v+H>#it{?|L+Z*%kr}Z zx8C+z4Bpd@_1}Dqew@^xz30G7`cU$RYl__DO$4#@Jb9xts*HoWyTNwhu6-Ft%{mZR zFw1|_zlMwsn4pXh)675;@G)ntV0ac-32S9J)1k^WIizc3yZ>BANqdYg zWcB)D&$}0kCmRT6pPkNo=UHGCE7paKmQrfl#nRsEiV927lKG)eE9a+e;bM^^t7%4K=jl+{za_d+}e+E~Z zk~Ctd$@p1~73(mzWaF%bN{TTU>RzAdBgzf^6LCT61Jb4GvtCD>mTrUo>B4j5Ee$Iw+NWFb&ySPSNr7Rt0;CwRb7%#7Gn@PjH9rIESo7Seu z??U^?zwY!Yy+N{&UhmE~H{_Mu?nJnhMAzM~ZaIuDaSidvm!6D6M};f_B0y4tQN&Sf ziOGtLtskm@44-NMItde-a|s0kBhBCGypo$$C+E%KPa}c$1HrV~s-Cz7Mjmh4A5ib- z)&!wk?G|E;vOvUOdWbN#E&dp3J_}L-9dUajjcJTDsd<&cfH7Yj>55(f^@5|Zc-0t5 z0b@)L*7lgh<+}AIe-^!!92|T1Sb1xr>1%zg1V=D+C_R)Ex@7(A&J9PnnPZ2Z%SaxzGM?Rks>Iph7CaVfVpPJ!mi%V@Jsn9y`tm z;Seq+638_AkU31^Cf*?hZJ$o43;KXhRfPg@MPL*RhzdWq$ivA+P3UcxpRpJ$=b#vzMK0Ee^tuyUhw($(ZR6TgtrRo z_r<(Za3Nr#6?}oCiHeAD13PZ5HGi?ecD#$A5sgBZhSuR*+m7REj|}@UM0MQpR;;sF zLu+^?nPm`7K_`K)LM(A#**^e48M`0@W&i7v-+GsD(d7cbCAZdR z6C2gt81Xy=&k;=&xN~75={frTKvuwCvMGN)56=pIZpga=R*+-VHl;@<6%H#I%6v`o zsVQQU7Xiz4M(~1ND3ow4WsAW5DI-yl=vrEQRiORahH0oiyJ^B63+q|zm9{b{u<#K` zXx(`Pi`#h4-ceU_{NYR&^i)wwg66xDi~`rP)C>Vn*T*ig5jhh(=Kc=LcO4$U2(X>T zth?H&sHK4Q`7o8$R%`9`$m`_#7>S^O_EJ3- zj3H0}mqGN7`EW`JwW;!Xk@{%kZ{V^^;&X;Wu{Gy`#l&`&AcgCQ%ebMW`>7R(yLH72 zE~-?}rKj)QH-60d95FI)O&5fEZcba3=<=s|y4neP<*4))tD{ZqY5oOW$otx(U=f8x zNK|k%h)9=}yR6%rl%!LJ6}E;NJSby~e+3c&+e0R?O9`|o@;Q+jfjO6w=blP73g>u- z&^6`(+0xi+IaFL^GB0f4eM&uIzSQ**({JkGER*tdezi1n*_J+qC)BT~4-z{-m6-ru zct`}qEU9ZUN(gS?dCEZg;4)z9UZ&uGQA*)`2o>w5(xgM+*UPu(8b?__8+!WXydqGb zFOmKOi*yUvGdPhCMYY^B#RyTYws$5(-Rd7Su0$Kn9JHUFJm_ zfGR=8;5vvc&KP07NS*^rFg3aDe=bcE76(ZyY{#KP%*;4F3U<-z{aD$2f-3+DL?QTd z6aa{~@&e2QBYebz_vTQFmXMuN<(Kq#zp4ZCTWk71=CwnhQ0c%~7^!q1zn(zj_3(fc z3KD?BhCcOSy-ZtSfoVP3NCXgCqtPuCi_CY5WOTVT)VE7%`kI=1zv42u1mBKq!gusI zHztvqgqhH3@3Ot{L%}81zn>L*a!^fkR3zZGXGW{P!uI_4rSaS) z73@_|Q&W57_wNa-M`Fh+QFMEB*e*=86#J+jrT^sr@5AcYSE;~~E_-^8DZW>l*pwSo zz6boC&nMH*65P6vyBrXT%}$8FY4(WJS+=S0TZG3>jZt>}{r7mw&&nRR(Ut+r78qX) z@TcKu_4R@)R;V=U)o^Pu%~MB9fjri@tlTvWt9h2kclEep8)b^ZViRtU~n?Y!%tlxm=8_n%AOS0n6} zRNvzW#%qfnV5LuDgWe3_xcyW+)IAOid zlSu16SNmm+pRnrr{=uEQ;W1e4O+(RAnpP7iP}Z{f@is{azY1)jhnVWgJ0J8KaD zzJQ$IY!gk>)CgKybhM%WzZ=!!Ph8#BV^eex6{byY4aLU1TPqX#A*g}RpN}F{cF+3a z7)&aSKh~4s_}2r|k3+3iyxp{L1l<8Ytx_{b(xAb~jEiP~X0!=FHZA+gfK*{4X~WO$ zsfg`Z$LUv2q5+7MGnfqOR=TFGuNkSvXUbabl4NF0Ll(TC@j>0+hjFw!U&8(#*$FvVO?q9|lbZP}-*w zD*v-9(3=00<}S9O zbFa?9hU%ImKNeoj8nya*vL!?1;vxap(Woa+TLhM2TY5t5N?K)#iTL)y#%30n^S9Nb z;N(phcT3mFpyB;mI7Wmuye4Rl zC6$h(K9M*F`tNm#NFsv&z0{*{VKwV2BCvr^DVdW0oXHjR6+I7*!;<+^84rU}QhlE8cLA`Q8+7fHb?;e(h}}fbS2i;sEU24yrCfTtlfYX}Iy8uHE{d_qjJHNbsu+y}WdxYoHO)SR~W4^;f#u1r7#a2yGy$!j<))jQJB{9qGEJ@&sh!XL3Gs00>XDxOCj zQt*Z3(Z--aS-BDp^}`l59aMLC3Mfa*u?_n63R10!4xG80@~`*z*n56zWH}yh-f@^I zs`cDkm_GrKDU8R^rx)%Qua5oQ|R_y8X0%e%)VpJoaGaHol% z$Qh-l4t#orp(y~EuQ#*^nBshtVfo$EJuZb%P8v_|&gp_*bqa;p0_qJP(=E|r0KzT` zXep=tWT_P;gigwjIU%)j>^rctn$Hc5G8+YBuFBdLxG)4Ej z<-Kp1SXqmg9&j;b$Kp5rwK{q`9H@zjbejM4uwoewe9DYO2QJ^uynXE1U_$^$+2D!i zhM0M*F6*=}12f9&0iN3*6#`()nZb}-AUX5VVI#ay^<&t|ax`Dn^jj2s?BO8Du2Z4) z_?w9t&H(_0B|1UN;|nRFgo?rZJ5MEyXDHgY*^tUX$K_6U*;+qZ88Y?RHId`G?(~vSg!>2S^}} zt41DZBOew`ztuRecU$Lm7;kyK^Fiws=3P4Abyj3Ql6}4oSIhpWr($-Q7W93x`UL^tV6M;l>+_p^l!!LHsw#=^pObaArXlm zxpav!sAE~`BQCu%j)5?#y_T=4IMGrmP6SW$y>@28V=l$rttAY8@m&O138 zm7TT83;6jv9ZoT=^6?JBRY+s15Ymu*-D% z^qW&qJZ@zf0NM={n&O=fa!4T;nF%ow$LTSk5pSm}f_56-w&6XZ{I>)udpS0U`7k2+ z86al+n0I-+T#qW6lW|rYf>ESUUIBij3OFBv#6c5+s6f1+JIn+kg0|*kE&2b#i3!&v!leEl`aQ``t1w~i_xm{7q4p4O~a)#>qXER_j znk0jtPVBYcT-UEm02BQL*v$w=H25)OY!9cme|*>}j1hPbaQDA~*%XJ%0FCur?*Rs| zI@)BrMl`5r0dj!U1qdsQ(2=?tYz|I`+`TpadH)cX-~67Jl?d5yKglDrbEbPD7n-qSs>*-5nLc^Zc&D3tS*8#eLP(5Rner!wHe zi%cDoYUuhqto#5yJ{W+*@EC9%F-MV10bCR ztfvjrc^6=F`2|E0P;gFremC3jq-Uin=GdU#j7pHbp1$6Xl$|D1X0(c95cA?t1hS-g zp&lBP%`h4NM!vgsgOK;g-~6`+rh==882k9#L2I&tV5B)otHEJ3h!hNmAhFT-ATkgu zfblUPZA2x?_rfF*-_`AIuR`E8r^$PdcUSjcep($}n!5Zfe-oJ*8@mp8{5Ft6Dt@i}I0Js=BHq3~opz5wHdDsoBN%FUZ-0uOd8l7h!w5WYtEBrW zb_q#T9G7m~SZ`5QA$Y{ocy;9?jxm%lJ8)Hrdl~k|(@MM;6xlvDmRsSm<{<`K<9Xew zL;VGBLB;!!Ni>;6%_sTq7jIou`#co?Gw9pX*N5KK_@+};iQQL84WC$U`4p%d^SM#( zvME3RnItd%N@GUo%$uBOo!c&x*|N;8jnAUeJEpLm64+^iMN0!dr-8hsaN^ZJY450 zPJ63)FJd3jiNZQaG3(CVG=V>dzJk=6`XW{tUZNb$hEPD>Pe+=T_`hTI?3uay8jQAw-b2^}n`sqI zqAw!JPF19#U4~~b{h^7LoQPXL*d@yVYVd93@r@q?Pi^-r%`XuC?omsr4ifgx<%+!nZc;!PV&E+~{0& z&ny5R{P6!sT8%WdopkkVv=EjCg9mRjf=&b%BMHA{KyT||?0w;j7_Su0V-Y-s=u;@}VCB^S91wIWcUz_1S28-8aA^W0Tt6|?dH;E*m!3d>WqeOIgc;oBS5cPetHTrl zjbFd*uBzIv4N>S@&i^IVfLkX+Ni6AX(LSSVbIsO+-?B3QYALJ@vMy&4QUz zzp*l{Ss~Vbi@2Gh z1@7^&#C$TzuA+rggLGTf`-8d^o%jy;qPxXKR|5TO z3bCV;`u)4mi6?=1?@3;c?DXD>(`F;*Uu*Bt9Hz(X|; zhz$8lWZH_!IM*Y>k^glp9iNchp}ELxQku^QzqvA^xqJDe*GR?bGrEO&qj9lK@v)<~ z(4Bzt1Cov6sS7b28Vj&9U_>!kcvKf}2HwWj7N-F@ds-KyR6uX!pVyh#@PJ&R0#E;S z6?$NGtr<+XB*c#!lu{kJp2QhXfj{Q%vF&lHEupgR`bd_a0&vt?ek~6{^2F}IhV6G{ z<8zWCB&l0Ybi&?<7WcOg%+D==38VeUIPnda^BlMcuDPS#7C?nX&h;;57UO4PdLVdd zy}4^*lYX}7mF*KCWKCq(2J1nZ>A#9;AxxDEX#?q3Q|(T@48-Ma3-91W}7cdG5gr#Ad?d>QfW8BJ^<9rJ)>e3+hi8pDTTk0E(sj)GYu zPkUu>&%`16;z2o8I{D`3*I9nxC{kNuU)g4%vlX-(SbqbBw(??c93(_ZhC7m^LZzjM1~H$#-BR*>|*tJbF9`nn0IB+Y%P%_TK#^^(hGwT zf!&KsOF%L4Y5kn^92GuhFxd-3oZg=Wq5JAy97+X7lD_2;_T&uoock9&x&R2y?hQaLB z+^T_N5`Tj~@VvjrATSifUuoraC>K4%?E{Y2MBABYu$uvs!_Y>AIOz&@8AUDWXM5=j zm$gwHsD(!sMq3^;a!Q+R`e=dz${&Zf0Z5CwqEPh>onYa z!72^{wBCUcT5CLtXY;_Y?g@>1E9Zq~#y;q#ujAL?Djwly8|J@9URgW%AuL1LOM10o zwJNL1&ha1@OHNY6^st{1TZ=xO;EewC%u6lx)SmVSg_>#**b1n$wTE;|au!SnB3nHECj$Qq9*- zvYQuJZrI+6mlOe?(mWfp@n&X&rrmyZuHXP$f#jrV2H($@--6GFuTS$^R<>+|dq1jQ z>+F007QGAVEqrKK?#-$rxA!)-UE6gE91PCmo`YHF;+O1L2LQ5djnmhIN3YEIAUlwR zwR_r^e;vZWj4f;5t&WDzyTe@8KV6>pZI0ns3Nj?qlcmfnAXupWAyNe>QA}u&?{e(0 zmCj`85wJsDQ!&c8ahdG}I(Hu!RH`1{yVO;$1H_i^TdSk1-%xU#)SGYa8;t-_d25+A3X4|rX50jOA|87}&afh}A{5J9K_V9;~VmydmWb3e971a}U1#>93=+y>@?e6kJV>XT+|?nTv6L*+eq2mo=kxKIrUxeVFy34X@9<`)qe5V43fXocmB4 z5T-PoaNR_#b+om_k)P!JL;*LEJr>eMNkJBbB@S}tf}Sx?P_DjLLcD_091deUrK9x`o0W)1|1TM@AEj0Rc1HW^qD;W4|X1c|vxmys0 zC@S2J22pVUtUx+vzV;-q<_bpS!+KF(qWj#Ukdsp{e1@wEBEOrZDOH14bkqVcC#YFx zpmL!;n1mZ_z$Qb;{Eq&VWd0l4E4L#z0i6)2+5~>%?AQcEf=RP@?H)z`tPEe+g0Q-+ zU&~8Z;J=(nHX=siwOQgTJj04-ZekpdmwA$BgWG_I1bz_#PGpY29Lrcyu?I z=aG$>H6A;s6X-uZ*420`QXX~$b{n=A?t*BjA!YDjJ+FQC?e#oz0~Z@b++KfUjbj#i5MQ#ZJyBSYOhi z!4%Gy&);spwb1J66fQ++G=<;<%A|jM-Y(x~%Eo*TH()yuHCFWbBoK{^%Z;PliedZe zAe5gAjo`Z&dIEyOA$3vm6D8`Ew9Da!YvenNFpu9$-d_iU!LBdCdI|(EFIQu8e>y)^ z==uE8r26{$`iOCC@BFfGl1X)nP(|+4l<%@|nH7)-+Py~R28PSOgR|~;!XYY&8lj{o zRmLrfK)7&A!f<$Z!dlFt%lEKm@q(n5@U#PU{ybqDB3SY_=ITGy@cyy&31A*QAa9wX6u73Yy*_=!YziP zP53zrQYRmvobnVoEw9+3{vyaB-Hed`#qWFv+M~1Tw(1MD@B?y z2^?Uy?5HB!F)fXygHc;w`xAK_MJr~m9eCy6%8>Ojd}piFgX});L^_ne*NcCOD~Se3 z#QxVD8W9|}BiTMY=w}X&4$z_Tzo)b6{Y?3HM^);*|M7bda_xw3%cJO`=5%s>T58RQ z&&rNd0!jkj`=j*LhsYCRvO`HH*D;~aiI-TM8_yQK*E%;)5k1*Gu6~B-=unUvzlJ5R z7q_4qq=jEEj5lW*xU^nbWAcXeqHlAI6U-?z>b_h=(Hwxoe55(=U7piGy@tPm&#?_= zhoaiy(;DqSyr1??1Aw89bF}DQ%kM@#5$S#}h6Z>N8SqEc`a!`V>9t(g@R4EZ4xShH zwR5#Kf&ZBq3VS)D`C-o#YV`i$F_HCqf;|1fV_L>Q+KEW8BOah^=rU3Q;Ste-902QC z*(!Jg^VW@*DmC(vp~lb@t5DB+S$60*!fSixUTmC`R>ElzDFjD?Ai;nrinzd#i+;~c zO$EE*`=dHm`p7({8Ul;nrDEF0LFGD5246RQVZ?g#(lx22g1f}ffv0DHB}xJbE3=0a zA7Cq?=)E_%VFQfS++N5Z?#-Ms{ObH!=dTQRm1EFYPY=<4Mj5SIz6~o>#Cd&AS}9IP zph4$r+POOV;p2M=c$(eV3o(h>L$c`!_(fw&I>R_DTkiz!l5wU1tMseRO5N&)Y7Oc4 z4^|j+5#Hrxyx#>W0$rTGKP!c0+>Tk=Vakfa1pF?x$_~OGDxX&5dF!NcGZE6(C0VTa-o$!WkJ>1o5OY=ivT)D+`aRGw)v=wWc@u-`I4K2O=q| zpGU?P=Zp)g6=`9WKu58(APu+*nSE`6&mb-m(Wr8lm-n{JUMrWJ1_MeQBH4iL2tzTe z-K`z*o&RKwsAPV4eJlU|NMWg8O@@1C0F2;UL?2=l9!By*)f^Joc#tg zf&DJwiSwDenWheNz;R)XjQoRud#{s17%;HH`!H6Q$tp%O8Rrwg(I%8`cwCm4MTu?N zC&R?4^?mQ7QJ4vw0&j%bp#un)_RQ{sfs1)^;&FD&ShfuGG)#U|44I$Zu!;@;jK}Pf zVaM&SbxuL*X2vbJ4e#2u>)x^WpGFvgAiLqiB}{Xa^LlRnuVpx4D_J3*ch!d*^H4eF zVrvLbd!2lU#K(vc*GTf&Y-5#KR%=aNVj_a|0=ODrCot^005&?NfxJl$f-vjM!ErkQ zZEG`KTH@2Vm79I6;s!1H0}?_J&k|5{m{m-C?5Z{&9p>FONc>=s2apbJpp7+c=KLko zF%SGjTHGMZ{2lOvWV$&0zhpWVdKn0wv>&&^m)#C)dvw5epzs>v$hY8XWIa}4fm2>a zIIm5gM#3rbL@{1%PgAx$KjHf5xr3GyU#3^vURb0CEJss?JAA17<<8|6m+ zX-xjM=^sW877@W|7Q)oxPu>7Zc71T0v}`9x&y2m#z7G2pYWa}u1uFlcx@s;m3Sq!P zIFumRU17?mz%oY2MhI_HsOKA@`f_FtkMwLfP1<&e16ETm7;Ehn8} zKmRq=kg|s(9zy}k-v*;x6A5WXG3oQ`X4witT|8zKKBzc21rW+m)zgyk$n2EwKpTHsx9I~@7) zu_Nct5|_1z4m4)+m7D{FvkT4Z8q!(K)&ZPUGMw4e|EY$JfrckKmZemcw z&qZx;r}x`qF#Ep!(fr6pyyMkVPt>9qGUWyj;^1H0Gj`{YIon1DsWoM2-|4#Vksonl zUpA|>6UP@m0+$Ov%^h81b-%L91Q$+f;G^=>i9#WgMn@AK2$^f#+J&rY`0i4T5xB`cDYoOJ!C`BgrpD)aT-P#`Lx?nG-|Fw&ouD3ZS` zC(p@s|G0F%GqLJU#vbE&?jJ&qdthI#&Na_u=nSvYRm}Vq31P!rB|p@yWMqt`4dLEc z@m%K!UU@}bjAQb*TPuU4PtRW21^_cFdu*s9;MQbe7_R+-r_gb@DB_SxxYu^?X7;-N zQY`h_!c8KSW%(lt+k6NC7P$+3WpVne+Lg8=-J>baGK!=rLA0xPT9~qo`6bDY7~r4} z=YfW#!M;2*?T>u{%{P`ACqRVvU2ERg%eUl!<`;dG=ke(}hTPNQ!n{#%#vGF&8=H`u zwZZwR$0gVx(Ua)d!vQZs9OEU({{`IzwQ|r@EzjJ0hc5|nSgQK$-ShVDLql?E@!ACf zFls5+RJz|DAN;wi{*qKi&Y*E%pzg0+Vqiy7QnIgBCy}_3J+>jtX72Z?qLokC!TtQv znL#i?dn9rIjhCC-p>;%WR*v=LBgZFw)9OIY<}8R0j^AIm1xVJxct!%hF9Obqlx|d# z?E*q~Su_^-vo^4N81au`E6E0foSU@&&Sp~wJN@FLv|2|jqew8nXwu+4pRfGwHBARv zzsFBdY|PGkyskNImOAYHFl%;M0{m8Xih1VKAU>Qn_wsI7R6#r8Ia8jATqZ|#aHk@{ z`MwkD5kC^w^294(KPF@g>1)JHj|;Soyy;(O_^;HxUcPv08Yq(=CkXQDWbDGo<;cyJ zu6gO0hYoTwO3zR1xzozff1ucX*1@lle`hx<^#*FbnYNXY=FuTS(M8@*HZ`$uk$_@5FCzb*_v0yji zi&ecfcyQn3>U?`_(~3TE1Gv!N_hS(CuQE={A1|p z=meMak2p{4#o|afS&j+ zM-ZQ|Z7KRU!%lT})b+^&TYkt#Ml3p4s>`ANlSfa6Z_%+wH6=1kNsjZFa@x-ty|^521P1v3K_^-mE)G=BLvJq?k>9#Ek1uWa5~+Oa=@M&Me_ zs^7VF>onNO0fLT$HMAZpcJu}oWcl}u4By+f1z5C+$zfjz7$2H2WGgXRol^@MBqEhh zfdCnZgI~7#=GCH)Z}a2=H+sJ=d{aS~t|H*6u}mC-B(a}%>BnXbg3neZDOOA>|D&_s z{IAYB`gctA_Z~q%3bRqwNw+ZrNRyHQzBg-eaq;ZeK9%%(7m9C3k7^1>XLe$=UT11X z%Gu7%^o*%2X-Vl5GDN(Dbh_jn-xd@%O3D;+gSd1*rnvhF2hTlZw{LdQpo2kHt{v&&(JPOD3lbDa=7uY+-C~mHu zNvGBPwnMvndMbSeicVDf$c}Wzh)Wn)TUUracGXYFQ#|1T7Ul}g3qD$3d$aXX_fy@< z0PzbDgc@1JE|Bq?&?S~ZzMBA8%VbE{z_qD`0@0L5&Bq6hz55atEObg`&ga8GZBf8+ zLLHd|2c_}dtfaRfwq!VYwJptf(oAJS3@3_X{;X{0_11IKv>Ii3m4QMI6qA&!(!Q|v zuH3NlOszgEA^e7#lpey}u% z`E2tGP_Zc0sxOpiNhrJ}e_V6vZJDcn5%4Eox>W?!x32 z%%CL?&pHrhnOyw{V$r@w_(ua2X2xO@jswJ0S*>$J#%8#q~#R>31Pqg@V#<@~ad(SXO0P5~M$_OzUFNitSzV)Sc zh0j`cddd#0fTwELM#sGJaSi@w20ja!Tb1_+|j= z6O@9}S{sWj0j5#F)22CXj@(9&r#zb#zWu~1ZWClKssR?oD}L-t3m~2DK=R_TM^4%s zBct0MK(tnsOICKQ?S+Ck>G+TeNayb%s6aw92;g~aZLB^9$n9~#9FNsQiYK2-@G(+lsKKx6EjeB;OzHWI#sxaHx{hM&+qATGd|E4+Yyp%0bLCW=X)G> z<4NJp86v*>LPEIEpV~n1^6iT06ZnI0kY`$%O19MG3F_5F0cq~!=b~--X4CC!*Vr!u zLhU<&{K}ygnb9ZYSGRBd#(1`j0R6zg0iJL|u<*zceM!=!q2Z*CS_I~>SaxaQ6 zug;YQjh&}5$KMtTnYwwg-^xfM@x-q0wz66wlFwuTFfWpi5N6bzMX?59zsU5;ej^w) z6q0TKPilcr%FDHB?!;|k2KerXgb5sYmz1n;p``u>V_H5%tl;4U-_VY$7Bjvtmjux$ z0$#o8 z24kH5xJA)mjc|!_NWk_u@6M zc6>oZ?nR};Ss|^kP13WS?Tt(#OE)Bmtn4gEE0;3(c<*ktZiVU{u~dw0sO< zPa`*;gTURBOxq%N!sBxf-Ryt+s(B8>+ZL=CZX8xQ))JnFFl9`6rcA>^d*&bGUz}vH zKpnfOqZE!$(s9a|pqy%GtlzC2!)}_erKf^TMAz@6BUr2!4%Y4~Naw`+mq9+c+}Lra zcvO26j|m-ibK!Hkp98oQ^#>WrxuD7Lh9K7gV~&LgWC-(xNb83};?ccv()Yd;ZHS?a zb+Ng-z#wq*y(W(L4RArueyR&u0buAaPuNoNc5Kxc9NbGj@?!hf88dV93lrq72_W-V zK)7r;qbR8X(@-aHoCLz2@WJbm-MmzcwYl9I+#zjIx!e2NP)^i?sr>X zoPGgxj0o15jN}bKU|B+$Rsa2y;9GFQL zuHWo~#1_=&tgv6+;l^psVNa_4MnBoYpr~`?26hq0$c4TF$`+JV;k1Vy2nk@49T4i3 zw&m%H2JXze8{hi!u0E!u90n${|8fIxT|I#rTYb7aHROcV;lw`bsHMggk}vGTbe-aMdmfK*Diq3 z2}6P4i&UQ36ATFq4DG&;KYpH}kO}aBR^H=l_2avLOof%JUTkt2v;>R%J7}b8(BhW4 zp2KLDco~CjEh&2+rsa?bvjt}$C_~2kPljx5^uHOhvPXZ{I@n&wkm%;T+SZtRp9_XT z->=_kRP!5}5U6-~+;m&}c^^>WiTJ3F;^b$I@7#;ak!XRlfY-RjI<8A3$3<#al3o=1 zA6S@)vc|*aMnNq4oRA1BPK%Yvnm^hPmXL_u*47jtlXlv6`wd31@CXfx0?|pH zWA6E?lQi>MvES&68lM5V<0cD3L!Mbb$cI|svgt0_=U%LPpj04N%JT+MJ<*NDZ9Ff{ zDNwQv3?_4|hzS#9nqhDqL}Ef_o{mMZW%S|{xfu|w>~k!JNK+;oUKz#w z#{|?VQD$D7-54Pm`LIp8k=Bo@w{=97)sXRM{$KT*9QzuzX}4*;E9M73PJK8}l#rzhBh2iJH{wQJjSS9ES z5!h8r87Z{E{Vp?53BuRg#yVIG`Zo+Oh-1#rTL3G!I-PLy^ON&4n&k~$J|c2*aiGg4 zB*3#H3VdMK<6Ugnu>lX9kOoX&Z?Am^E|h=HB&ZmnFX?*tTVMt9dDk+U1~HNlaR1QD zO@CnF%l#X}VzELK@n~Zz*yV@fl7Z=W>DPn$&1vOV4gn`SVh+4g2j}?vZnaPv!I+XV z{tJT!kls+^37R4X-JeYJQk?~{3xfR#oKp*wX%Dl`yR!yG;f{t|!Tt%BY3(L--$)b{ z)6oly+GJkmTHG^b)Nw<`e)hpcyxA$EmG!xeeKjF(J&K(6Je_Df!!k29=vQA-iDW`o zfWw19^g>e{sF2}2z#|s}Vw@ub?67f=gXba&Mfu6_z&-*~`INLw1aB{SCs#k>n#;5f z<6RTOmO~JeG1H&(>!5dvI?A0vj&V@hM;15+cv=v_9M)lpB4{Ey3D1af4DpeYW~sG( zluH=t4F8D7yaNcZG|CJFh@+7C zRl9Zt+Be!*H?A?HAd(LHJTwFJ%vo0SZ#6#gnuna#Yl7Y>#?>vJlw@O1$vhq(OuXe= zJ}UP2>juCwN;Geotjkdy2oN%drz2{T7|$Y}G6wx0XLRZyZzcy%qmGP8fVZ{c)?Sa| z=^4MkP0fl4u~twVrzkBw!vA7k$4lqTAWn@AW8&Z}1rmlf3YRs^Ku(4T_J^1V8|K$< zEZX&~764}ZRGboIfMJJylM&1BCJbjo8211Qq=s^I)Ekz`Bog3PGikNkzrJOTk)sIo zy+9na)QW0?lST6oV><2S=$VNO8Ii&K*O-21vV^WZ;W_iGNFUd+2FfQ+YvF7#wg4f2 zR7{yDOC!~S#KUt=hES$B=3ZbRvFrW)4dINPN&<2gEO{(1r#(Lsb)_r{)bzJFj!Ltl=YSk^dRJu ziSTTM9%B?MGcV>Ym5>k7OpB+wU*Aw7p2%s9rs0O$B^E!zySWe#S%moM0@YeAUkbYkSH!+ZQxgEqzPWV{OQ)BIfVeUVj~~83L1G3u!3F zh=5o0j6R;`LKs?jr8Q4VxNL0%1&Z!HjT7X>aP6$M>)<)w0y<%Q1FA3@5RehXo&9v_ zo1(N)8Pb4dc^^?5n|7eS819`}@AEru37p@N(F+=((=!XQ4c6qmRrC3DDuRnqoYeq0 zVHhi1j>}c$G0CH`6Z>0F176Q4s)EXIs)sM`?y3OC5MB(r{-J{@OP*aIb(m9@@7LC- zL?o%BrtSlNw|ZHltGUNV8Fvc4Db2xc z(3%}9G8>&;@CddKZl$*yCrtFBInhK~z(L__6oVWKQ}A$;cFx7I4;W;>A4t9i1yWJ{ zLZE}hxH^ja{+aisrBs?4vXDLA@%)Vp9_y%6Ce5=W57MA`8Fd_pb^ z+1XiMRi)xg2EZQH%QLZ*d4l|uh5R{JAF+T|8_b0Wp#esaVz{^r?}~#O5sq&q@Ba#F zC6*B2#+k#yFsh~Lo(^_n_$&VYb7R5OtAt$?O+*g-0(=IyWkuto=?8u3ZG<`+Q0qns zLSMa;a7^mxF80|2?4aYTKDLgmK*myN4R?%ub%&*GxB#WI;K1W}b)qjLowIv25tVBn zLCT&Cu*6>b>{!KXi`*cP3Fip+V80giwQ4=BmEIidvo-ui%_HH5ZTV#0A%2?uj;N(2 zovYiuw}vc|cp1b)#;pJ5K^Vv^FdmrH-++iDs$mMud8s=JJNDb>*MGeRpL=$-H~#N| z4FA2Ig-7wsm70Nkr;i{hX(!PdnN8|&@=HJHTLIrX#e)VXSL>>MO=0~3WIT*?cCc)m zQ#L}txPSudZDxL!oHl}ao+kmEsZ?OnvM^8Q2xAiSx+gy7&=)ElJq zbJ0KNVP-Pp-!*5wB+O;kh`;(Tqe!^>777!tS1#6U;k_$PyB7ycH1eYRFEaC%(#aF9 zTfY>FzStcd}YG&-8zfhVPq<^dSGB%g0G3Zi4dDBw5N*QAU12i%zU<)7Q=R<{R~R z z{_Dz!2aHG9Zu9LeN%8qh*faemN`UgreGZS${aC)Z#}|V>`e55l{#fa7Dalv~x7%?& zG06=mjv!ME!K*7DPNBaI-H#n8II1PrYv-gYLXyrUVLr)2DS|dA%_I^HMX6^||5N-C z*kZIuo%+X-|2^g1YPMtt@qclVz(BP@WuS8mF9sbyU{mDO!zwS5lo>ksG>=TmKUtKz z{N>X}pZBG<3gERD6CCG4hwI13vLYWBih-A!{2Kt5Dg7LCOmJTPSWB~5;-rIy&~fNJ zRuN%N#NuQL*5q863B*}K$wt0_OaV4ckf1tt9jf_ESGy9UXlC}C>m}pKD$yxx89>tmqgWQZ2o>=;{Q7{ApF~h0aEJ5h7!yQmIE_@QxO%DIdLkI1H@H3)mgiaZW? zMzG?&1ub;?lQDdIz2K1(tcouAgKPq(Nj~1qF^=gPXHpd%ss^nKZLgM+2CZo%uMf4^ zP_fP#ZG({rK|qzu;g1wDb%6jE{`WyB@(V=uhRH8}+;T+a7nO{<;)n7yb?AJ26Z zG<_yAUL5-8KLhEmkxX8s_FOC%RjA5lDCD(6&bDi7?&}j-n zzg!0fOs#g)IrsT3bX#r6lF65D9KY?7+(%uPn_R|p^oE{Y#4CMoqi!&u-K!U7bMO1*Y(fU91g9Vw~h zT>(qp0#xRB=|&Dx@1M~h_TayxKPBfsqd&tzr%I3O5R)$lhjs1`{uF9vp6S_{J(aHt z%E2n$owon>P7x#~SN6rl(rQ)gRMaozDuU%sxgtyZ^2MMarE!<;CXO~^5ay|bhMZlw zM=iAnz@WufSrJ@!?#`x&`?FubrJ1h%4;a= z3L+5v2T;fBh94|}C1n6oj&e!V@3qHLdErmd+M}eBI%b7}B|~)3&lg{>K#dZ6V2FQ> z9ST-vF)NS20_8%#|EhystOLz$x8qTvW$SEw!{3L_gBX(a*`?2lF%fR_HNM3kKlz&F zl4!L-&;q{goF^qh#KORZ$3NfJgl%1das?*?J0)XWyERMRfM~>zkHMqWqG4-|+bfUj z?Ewe<1IW4MKoA{*4vs!l2)nUG11-0f1ElmuT3OG6nasAD4D2#|i22yFLzMPY5h z+Z__Bb3Ymy?fKmdX~{-P28mU|HU;VpIf3sf_o?KaV}Wdj)OVH)yQQ=&0ze%8Djx(( zE6QrrOt^l0Iw}+#6FRnat)`$&AKSgREZZ^v*JRXO@7GL*1OcS8hSWh>X>!%o~!j@e6@$_kGf{SJph0B zQ^oXq^Xi_%*UG?|T%BG9xY3cOXtu2nKrgHYH{lHJxq)~4G4l8^)Yu8U(CM$P5tCwv zVXk6F)OE7ndUxI3TTn=TJon7)PY26{wR30VK)jj8&%iHc-+Iq{=>4-&wQ+DA7Qbd7 z5Vx1x3fE*DWhHULEBIHOZopR%r!zKs>_vA=r+-su&u)x4swaEzBFkB=pBwe*2X8L! z9Q@Y%;(4_MqZH9N9Vi z9A?$f>`?9&y<=vTpwFq%0)nI8q6}!HwT;o8xfXCS`SY2L1ufHSAQk*6e153%;bZi~ zBy~^1LeLQzQJglU)Sjqtk9+8r!!d1@mE%U}^)a-qBGdt;wCkV+fSB##V>dyELT6GQ;0)N!Oyf&Q%v7j_^=X7SpAu8d z^PsQRqm~7!O`k5q)K@Ftb{&&X10cpmqfS&trr|I%6!JO%vY2 z;wxo{ZGk56{U-?VC|U58AKZ@Vrl(CRx7UXXhr^W+j{n$!EoD?!u4eHVL-wlzO3H4v z5-mu!AaI>^rE}Pw8bx4Lv^GE3j3_>|0aEjf53KYJM!*8%w#%AV7r&s4+XQ&3aK7{f ztF^KMwt7I$N&iMhHTMpT@)-S5_*VDo^RI{CviC}Z#!hc^TPbAAQ9aF?Gmu^QhHRP8#0FH^R@ ztuaD6(Od9t>{Tt8}h3^?eMttWQnaj7g{Si6EqP6sn!DT;+2BXE&sB>lS35%l^8tC z{T5^f!t({dj9>PVB!?O^8nZNnQ&~G$W&yxSfGM1)dC7w8XUA}2(2%17m07OtwCik- z7sfzJUl@nl@G_7BpNdCb)D5JO#_;)kb5Fl1eS7=xhI4w5IUs5uFMoUg8k!X?vwuQ% zLI|8C;uoTop~aAJ4x4t!C-swH(kEWvKT4V31!taoF0fvzj-+Jy-2t!0 zBN1(=4b;N)&xhm=)Ko#bp-1UAmJxw!K6CZf4zuPScM53*0``uEE*!%`HQ`{K-ACBF z3H4gaGvh>zBN-@!bMh~wC3$9Wk~~w0`g#yRw{Rq=6YvZYP2eS-`Fd|qov+WWF@n*8 zFR_AU0te)R3HSvN{&i=X0lp>Bqr4^*RPp_LoB& z$ZgvMsu0&Q5?+n)VeDtk=OxP1j=?g?4(qLNB9x%cb0`B%tV>*@>zRP^41DgxM`c;k z=KIY|6ADRW(wgFdx0hDuPWPlAEW)%P*FyAo(+*n^d?c)etiId{GJzLhwwrqH9B4mt zkcJo7>q6#XC$B&#dl4n)4S5HI%`1Tn^hR_T2=KElNZG`DS61wgOMpIg2pQe4>wY9t za{#n`ZiK(MD~N~?@6G2izS?q_gSAg9K%oU$kR*eG$tQk4CWMc+26P(DaLmOx7 z(LS1q4srb!Rq4O5iSp3i$%7eor*xg8kn$YjZ*^O6-W5{O@QRzKdFKr#R8nBfaLnKI zx>S>OR35V#SE{&zpGQ6W9Ce%~knO3!Tl zS_uHBf4-aG^sl@R%60-FINTM-NQ6TI1}A=ja0VLiV_HbgSC(%<#VQw}7IR?DJ}FT{ z5L#1YkA8ni@$fomki33v0v?&{!?GZ{IQH~g>C!}BPz01fQw&W2KI#C&Qyi#*5KEK+Kp% zva9qc;rLSj2*j5qq~7u7I{195dy=Ir==!8+UXpY|PgbOM9OU*j$1J%%=a9t+ds^TH zNuXB!+lAdk0fzNY>-U1pbB{-~s86RH%@81%LphZY%{_gRxb)XVri45vL$Cnl_$NSFqQb}x z!dHUaLr|4}%qiDCSc)ym9x@HUS7q*)153%R-v6H;nd3DzT)AXCSto&1TpcPBU5Vll-8!?^>LRs|s5*s>*Z;cKDY*5mcPAF_9h zhYSiO?Zl29KdOaj!0vbd(wn`FT!>qATb$~=2;3;Br6IGIW~XY(rpezG4>sIED9qJg6P1gt4C=aZ&L2t-w`rj_JkgPZ;^l4j=%QI zBB_0jNr0vNs1Ufb9UG7;1T766qqT1!)H)KyQ2}DsZ4x3jj+gjdtY$R{BCt(;8SKgR zBV^Qi16q!OY=LF(w{vmcKK!{UELV&5(ES7ca{$A*e4PsUD8N+zT97fA#<56VqS0ul z!2R31(8(7GVZBPFCOs_F_=S4-NUd6QB`}9VES)K=krVIa%~f?FivEXm_JS1 z9aQ>#waWkZu9ZTTpVT#e`XdsNPMgqejpP**Q2^G(unBA4{NZpvpi=Na0r{zE-zwEz zyDpAVB1^2AWA>Fd__reS;9Y&9rXV=M3)hV%;;!>V=KymyTx;^$ftrEDg%7qSCRzhW zm|0n!k2YKaApFHa*VqvF4+2a1QRxBQ3POZEm>^|RdCp8C! z@kX(K8WmJKi{as5(Ak5;R7=|7oR7621)fr4GhrT9l6ZOx&pj#cwT1BjF;u>yZhU@! zzl|mO0&r9=bNOl}gQHWS9ibqV^1acW9JwCr)dfOrY6srNp(-UtPTdgZMQ0P^dC(bt3)>s^%}lL4Xpj`G1T}_- zpM$PJ3iwAEoOg}Mm$U-N(IYl1N3oG4$C&f4ZtrVOeAYV-{(UEzp~0tuY^@kaB3_UE z;TMXRx&&uK@G&Sb+hNLZF1ig7oAT?&7auk7K}DziXwYmC|F{e7lEzIkI`MYLw3uY~ z>>Y%tsQGGjooTf4s$UaYW^-etoKQ@aJhjbvNO;g z^Y`)rp6%akKC=sd!=Hw&lolpymETd)(g4TF+O_~L_g48HQzQJfgM8oKZE6o#lOWgk z$rF(N-;QKDS7a!}@tqCealS^adGhQMg)yLg^0yn>n&*1uly6|vG;=ps>K+B7T9T1n z!%gy@p0vBls?I(y`a=uW_lHXK|67GbPv_6?ECZ&x@}dwZsbFY@GU=znw&&wxeWJ1G z&Q39Dk;Q}Oswl``CqTd6DK|IWLP^k=csjsaP^?woeoujBJx~=>2BCJ4#BREA>}fu;b$Ab~ z49x&&qjOI8=F_-M%k|=8b0wrjz=fyofb5drM<;xoumf1-y;y22w|>sB6%dU6a3q2% zUSlD1ZoUo$_it(b^@FGWmZtS#1EvIb{vkt9&iYSVEAzkoP-3A@gifH|%yP?$gk|h~ z6R0+I*a|NJdh``fsEv@rwu*uNv2R~Ff!+(Ky8Y(WEfFqU=+oD8XEz~9*NrW|Hi%L% z1uitO)k)Ns`+c)c{PhQSm7{0=DP~RlYl@GADnKJ!meHmTlknE&HBo7Yi?3pQTl1QE z(J>RQ%}m?nS2kA4fz)VOT?J+|`EQW=X38GxV+8TMXsBuw@D&QjZjKxCC|*-6GPA?>Arj z>In)1p8v3_^w1q%AXa#8;08$6)=8AO=%nNlzbq$7aS`wZ_vi=O`b1ydRZ#MZx?6Y# z`uN2uU_kIzTm^5~`v70m8j9pZ_AF(#B=VhHSvi(LlI zX34%k{O*i^(bgJo2mT$3lJ3th>*G@t20#}=g#ys2vkiz|LvvBzTk@n$Ot$Zv^W}E| zVv)G;laatoChA~dVlX{e6m;N2;0i@xV&c{Yxf8z|qp>I%&V0=Pm~0Bv=m_#rarTay zph(jEF240$X-}1)>KVIbpMhW-(A_RDqNgkot4Y0H@ zG_aBf>K=tSFTMDDrDx;rYBffmZJ*qd-nFE6AQX5HKh1w4^Cp)Sy=>rR`Lx63{wi>6 zz?$L7qYv`(ZiN))nOfqoc0d(Zqmz=8^ z7!o*x6oaw;;>&&Q>RkH*r;LGmG>?~l4=M^h!T0kP<7o7j(ca)90TJnfO?8V zJvShBu7H@$W!J>Y*M~-Iu~3IqvM5ANIPVlZyjS5IzBQmF%YfV{(`^G62em_0VkF7s zq(n4OUTUDa`*}}LB)0-a;-Qu}o*zIc{E+SI7G`hFTlFVyZGB97;NVui3VOKCP8Zr9 zyYJBhwm6T}`72>VqzBkP?+{DQOTGx`vV#>BgYDyBk5eyK{zacyIQ8pYPp!ACLPx z_~AW@_guNIYn|&{=h}Q&s`FW=9Z>B2X4N7Fhzzq(WK;xyq9aG#`a`7BcjtV4Fx@%s zQI^N`VZM7Q(0hS~zKu`N=TztWu*%zaumaE&^KOA?PKxA^%;2BX*Uohg)>>; z<1g>f=i+tiHZxYE62R5MXIz-hOJKi1vB^YxXxTBYo z?&p?^cfjtu6zo7lm^9OqSlH+>0o^n3q>Qwl86)GI$Z3BpFn7Ut%x|;)H zuJ-*nIT^y+AW{tmbLMa#z=r0o0n@|Pb%i~tOe?<6+Y zDe>&TEkQ+j^I(!kMM+Xh|EaXM|E1(6b*^p6KOY=>TI+~Pg^-vL{C+4HlNb=P_vOxH zt#fGPw{I^cH`<&Yth2xT_z}}9Q~agKhUJsGj+ozF0yek~)@)4Y&v=cDM>$)&4{rbi z*<_(<+XsxKv=BV|ohg(0lU9z43qYN+>b;N__@5RxRJV)eC}Tsxo*0h` z9Cr6ZW%p(CCEMF8z}#K`?sP$g0g|NEV_uL#Ak#$b2h40B`JywqB4!UzY8)%jZN8|3 z0L2S$ z_LF4}_jU2bs^574rpO@Cj_|zW)P0eO^@4j z3knHKN4;ti^Drgw6jz6BTu;}ZI{>I`*f~h427HomJ|ILqAwfk@D(-wW=zb=zWr!iB6$S+oGjKcHLm^TPBx)M^o-uhGC>$U*RCMLQ#SVxXpY1gY; za41BnLsFy{e~8Pi;SLGD3ki+!B7iPzU0>E;uZrUHdC_enj88h{-4>vab$q|J2ZDk! ztFGB;v4l+#0g$q^CH0?5VS;(yt0=^X40FZ*(cgblrQtOTkE zH1Y$=N48(T2KIOUq#9iGl;(ai@2T)AY+9vVEP+F{sAc~?g#5`Fi1a~(*#k}jT2!mx z3@PLaf)3k2;zHa&I2Yh+4A>mNrLP{)g~0$3pJTT^wyOuAupJltbva5#kUf(i3WUM~ z4Fci}UA5F5qXo!ay<_8sxNGgJZufOBpg0XswvP;r#JdQ+-W9%3gLA@WUthiPTeL$! z?`+(87K)GMNP^*n(c;>{L~IWNA2cNaY7MZ9)8y;QQ8rNN0qs?@I{>QBWAD+vVxd&= z=R=7%;VuBWFUPf0-z7t<8pH`1dNbFE^BvSmXc5>L4n0B-D2;5Xh7C>x1N~ zNBl`lAkYn$kPy7umwasXCc)cSTr2Q~x8(Dn7aO0P^g_PLybBL^Pv=iE<85YfvSgCH zFXx-aZXoV&n6SsnUwBR?=1jfwiRx}1+MS(0F=Pju*-mzRNg)$Sca+_aTX{h+HF5pv zI{OmsU3=|g>C4f&NT-_dC~fN=E&OA9X!5}Y@wn=^XgPW6|8N1U;n1#J+jKb|E5tQQ z>uw{mVR0U~+4SbhApM=BA=EH5fhURrD+=v?(eA$W@$&(8tXv)TSLyE$MLsXq78rDs zs{_?sobMxwieb#yTpyWH{4+QpVi37M2}-J|7|Zlh`}%Bj=YdEJL(b)C@F*TVHl6_f z;ey&u=x8&LijpW5N)CmMxt|RtPXM{}Tfk{^y=4xY#iqy6!Wp7^hD(PPO3)INe7R-h z7J7Z|ew|NOGD#=lps7^?CiV4T;+cl^F?L z2a8CoMj&&?SfReAF)aVq%wrv|jITX+(oXv^Z27M|F%A2g(RGIUttuGW z@IXi)v9fQJ(EDyNi1Mu)&wx4>=?3a2ke?Paty6;2d6I4($BTH9NTG~ynY@8hn?q%_Zs%?68w43 zi+e~@*c>E_pdVlHHkx$G2UL%0ClCd9rkkQpE&AcRE(wb!_pABx538i1?>w$a66vv zmp_Pc#yyV$=49aVwB!x4f$MaJVi0@`a1zKJ*c#n23ANDWfs!-ew4faoVg9f%Lv_FV}(O2z&P0EEsLH@14FSQzZ-tcAivK2_NYL_FoCYF z&$O?z%M+{UapqcgM}aG{#DE3fT`65rHaa+7YE;K>kB+9=PGyk=lO{HjJA`0xlScTc z=O1anKP|mUur+0ksh2jqfYiKqO07L)?vNpQ(<8zBWd<7X_L>R%%xF;C~;H<@;mY0XE@zn=dd@hIA`-fE%cwSa=lpXblH>LVBDiTR;j zT|f%qjPFTUV&dT(2n-abt{l`@dqch*wufX5gFM)K_#7%Kzz6b1tzzkV!Y1fOcN1Wl z&!aAppc{V?y5qN5I3oE5A@1SgH?}H-x|=rTQRBz?3PQ($(;~D%`Vl@K>qSk36oZko z0tJ`5SKY;NYMy(2>;sWn)*eRD@L4r%qBuRv@$kRgBgm@y`} z?Az>1!rPCi?=|5bdylQ6*Eo6ZTGa4hL>tIZdyqHATu{;}T3r+ma08}k5@ElaCzgQt zJPwF_`#FkciuJ&zNc9LP>1q^ipV(Xhj2arr7LXg}Yl_?fh3>D8O%F=1)jRGCLUY)u z-yInmY+Lvo$_o!#sKQk4QR;&}1ZkqglY&3u>xbGth|cu4NTf)Hkg_+u>QmH9djmKJ z;3RO<>)WQK;^-e$a7nO}Baz_F@C^Ehx|I9kq0>cX4Vvc)m5j;mK9^m5b@Oxn1>pe} zpyfVz$-L!Cbq42C<}*53^EeOYeVCP%HJx(a(f>UvJbYc_gM^{4!H;x-Xf<``_72?s zVrO0;8O~N(?(Wsy8A`Ez@nduURni%_IRJe9mO!T1cKBd9_DO)VuT*eI^~hck;LzYC zHA*vw>wAVQy1%6HAYnWSGKFF2#CL_T_&}fC@X4j8;+^SWh%hmf4ecI>Lg?H>DL2;R zCuH(>UQ(uULvVn5zb!|eSPuwA)+N52#(r1sY84Rs{7&?({kp_^?+Lg`xI%bYLK}Zw zx;B&IFs0T-zLp>qFKLJk!t$8nE+3%oH=u=CibJG;td?v!7laaIE_eOsw1xsqjX9ES zt%y(XRtIi(t;SVB`hUkKSp33iQ@5fvpBwQ0?|2B;q<<2+JB82>L!Cdy>=VYF~Hpsb`(>ZTEI56NU z3r#JiuTS+eMGvXje~8+z%B;vMka#<5K&^lu^kr}csK#rQ?=36FTGF^0PzTNvy|?Mn?mZg2%4p z4_Q%-Y~w02cCMeRzY2c0i;Yfew>CQsn4q$}_!Xx_%3XuCsWVXvQx8xsP*SK5O;)eT#sV@G7`)=Sn!zR$NRF9k3B& zMjSK>KI9jE%v_c4?e^y2qUFfeb%)A6PxeJRV>8`M%#VnWf<$u(=@*MpDh0_ws5zhO z4tnWI>6m%Hy0^Offc$O$D&0%S&b>x@PKCw-R3BOJtazWn)@mVo@m=5hn5*F9-Y;yt z!!go_8f{dx{pK!q&R;_FI@=_RtmD@-$;q2P_DQEz!p5E1%V;`RLf)N> zET6wKjeKYD^P%7F#QocT?`$QqPI>HiWVnS}sq`~vjO-o=1g!YKI}fQU5gtFM8yA4h z2&PSv8$!SA%#NKhk(BL@u1>=_kWXt&Msz?Rn1 z?F*B!{5eJbgYi*}5y=-tC*jxV`31zm;69+z%gnoUFg%n81cD=q z0D-XS{=9`CJ~o^m0`V)aX*p99PW;AO(Iqi3|zD zhvu@~k}Ora`M>}2B2Rz(QpxIYOv?Pf{N>GWsIlYtvPkXE({AI*(QGNgjLqVA) z%+SXA8spUmO6)Y6u68)?s~`sZi$lR|6egt_|Ne4rcf1_}Fs%#56Aq1YboUGHyiX|R zns2&dC4~8H8ge;8J$cXo{(&y^+L%R?m`y)@f^$d{cvvepO^1ho@w{mRd)}e@hWXHt z8y6;1A;AC-OZTiO-W9SPT`ws)?nA^s)2Skth9x zkD*t(y|O%Y&2#Y^&|OYKp}<^Qt9s5DAJ^L(Y9DH>&o#R;`nYHeN#(iozHiP#j`KEc zV{^dGy^fh}B3mb~k2KmsBHs1AKB!$34BTzAFmr7#<9a{c)Z3Z4GEo&pHpHS8)F!^K zSR*8JG{u;$L_f9IiWRJ!EbdB)zKk$)BRZdAuGV&K7o|*S4qE0!M{wG`IShIwOwo@% zDA<-7iUN`do(jyIc8)ML#$=#Xf`i@TjfuXk3C2Elv#7Bl`+ch0ZlXEyFuw<>@8ML$ zKgXMfD;Z`SSHF4oK<`zU#>@Wwq^;>ug-x$_=#;9O?~B8{!^KqR(9q+cHX+;2q|`Nj zK`mjXxOyc)%R>F9X0zPl!ag+L?OcU6iY=svgmqWy$E=f#^FRWUZ9A23ONGZTW+7&c(F6MZRHYwwbCFc692y zKGC8|R{Mn3&Crkg?++4UK3Nd$-)%2@G;I689-3v!ubaHi*(Wh}{4u65aVEf-``n87SsoB)wn>jUxY5^qJ+$3&xY_;`KboFa$pI5Nvf4R8 zAs{?t0RI(-Pgc9%RK}d)u!SVrX9Yxsw5HSAWu8+v)etHPvUJYi!xxj*oB&FagC4#@ zmI$q&NDGyV9UI5+<;J-yH=X9}^CpdG|C100bFrZ(WkH+%2?maZG* zw-Fl|@CCw1FJYfc%*V~DwCHUn)n+a<=T7~{DX z*}iP3U%%`7tl34Rv?^|^aS&L_ew@=+X&37rp{to}(==3`Iuotit`_t1#St(hmu-2H zSq@fdPQRL}3TL8;b}xQqIkPea%;0C9733Y8bkq?$qu-fJy%5?^T0QHH-#vSD=Bhqh zhPOPjJgqru-=3|GAX;amK;%88rpRnWnTa zj!tzqX)b37i?6_g(RL4n1G=)iH7MpUt!_7gQs3HYUY?7Y*pX3M70*X?vfRZD=g@-DM*YbJJ)a`< z!}+_f&%w+NII{`w9WUL7x9R(AF_kP9(VWoJqO1rh%Et_y*E9?n%S zzerC>X#fI|RKIj0>G9NHIR4*Bp=vtq*`5D9Pye?uk_BU<{D2r?qTEeu&Sym7-{}qkPb5k^l!d#n+hO8{rlmH{VyXw<%c!zfk8r^&G6xho$wIn zJuO|P^S}>>L)EnEvs?d5BOur$i@}p7P36|R@7LMzu?fIz>G{KN|K;8P{;i%&AqdE) z*HCcqU^prCA-;0t#2!Q;<7d?9>Zij0@BZ+g{Y8+yLNtjj+9#m=+FHx`BYRaUT~3R3 z1^5OA26_mNtLwe{PJ_HZVlHGC_KvnE+kZ2Vev_#SM=swoXsHG}YTr-+IE7y;@a%)| z4KQSknc$5FF71{Z4I;e}9}U`%a#4OO+!Mp(e{Z&7um|wn+gb$R;6O{;r?n*|Lo+V> zvWpI^inMHZ@LF6E3;XUjA}~G_S_OdGk2~Sllqmqsm?nb)wLvf7O%$y&fqgjh9yZ!e z>pv@&ZKi=rh+9CsnX@ZI5GR}=!?~oPu~M_@^X^NBjiD!>d7rk68E1S>`|FT+Ku+c% zz=Tzdd7}XqL{32gXX-F=aRl*`_dZ^4lA!7Jx%;)HfbGhg#O$eDb>8y`;ZrgoSbbie z$M)n&q4vcBg~X{R$fKfR))T7{4Q!C(G6W|`+zSZ*OaPj*wwr+b(bP-D6%O}P83CZe zR^sLx&FhoU>oKQ;HIg$m0&r^TK)~NuhUlChut~)h^*CIpC}vmQy<~KQL54~-L12K( zjEB>8d4ptF`S_tHr zq2PJ)auj#Ib41j^LK9>T^p2h2S0A=C zjTX<8DFg)J0TTCbV8MO{o1pSaAo0fy5M{ds1#9dVCttG8JN9qSKb+_ce>3Fo=PAlI zZ1-`^<-dI)E)1quV=Y2a_m7n-2mx(GEMm`Nhx&>0Lt$rh7l10uwj&Krs2^{hhDH~$Z2%(J zo|elAAPhDw^RC+glJMe1X;f5_VWWcy+nsSK{YUVU2DyBrh@ z&FcG(uCC`v=f;JePxlk{-r(zro0w$Ut^J`huvxFaoUFGiq|Z4P4GW8VDTT~f)#*3* zZ!<0;ky6)!R8_6q9f?9!bQDnWT1;Fm=vBL2`bNZh-##iQ z-kzK#V}gE!KdiDiY%NdSOx=UJ;ME1Ti-kf3NlCA<`$H*DfE+hxs4uMK?G_q#myEvk zh9csHvw$c94pSv7nkL*sDZ4O=1y6ebxJkH#o21E73y=8t>bJ{}F(-4B1LX$P6?Ibut8|ixsK;j6a@8m3=2L$j zA}=SYP_Qg{oAr08QEt0|+EZlsurvJ#8T?Xv8)3HJ`yyc!Uc2W;?ng!hw|rpQVwxzw zlr6*%hy|`7xs1%XJ{gGoZ_RTSyBSK8b;t8Vmi)=_ueMQjhg)xY+z`pX7D~Ta+=kabZB5eqoKnx!hz%C-5H8i0WLRXc34!jvWnv? zA9V{OBct)e2~eS$LKqrK36#kJ*4y*!yv*I}OnEuEkVS1p>+UWfs8RN-4IOACNwx7U zoTSQqBhYe?Nu*BB zVl9M2TMWjB7x0th-qjBJs#h#?zB8<2(eowQrzahw5D9Swn{`93kHe7vvJWA&o@bYRyS;I_0tf4{PbxFGr905KoqzPsKz`VGVQ#z zQEDo;cf2z`*cMDE7`&8UZ5kUA~EfpwCo)^7CaWg-5nR#z(dVw*SmG8=4rt@_F^) zgVxGI6T|+h+6Tj^?dkg2qiqg*Px^Gb5F#3Qhs0f~jpePe%E==N+Ru&N7x#cvky1aV zARyJ~r5`cXdu^8&TIx_7kr4c(cnU3KV=D@F?=AO3&$R{zU_BQAibTz~s>J!pVQ0Hq z2n-?VE6M0U;|4D3C3_GMo)^g=gn|P|5lTiVKVV&|%zG9&@<3#oP)1eEb@wPJuI0X% zIO>bDk>mV*eahn8j4$LWzve0`vEHh2Z-0_+UD*194>;tqT`ToUHLGk|1#j)y?3?vB zIm&J1YZcGrY1Uf+<8SMkx@Z5Bi0WmFL@>2Qcb$sKqt#wjbdn&P?r=DJG}kC}hh4~} z(k7VjZmsj7mCeCM>E=kW5g5E(VtNb}u-U2zqyWrH?#UmzZqr25Vx!+ysSBHyWoqW; z!9Zr*xW!b>`f>@|(;}l}x2E3YdYw&Px8+%id-ukcS(MoXSh|0`Niz2F@hQ9Ljs)^d z?KS}wg!k{>@m0RGX|amuv!lNfXG&@vPH?NICt1Lz{mn*zEv%#FH(zCgKWuECjsA% zxQ1D;d&#}78o8Uc8rtxk!A$fM0r>Z)jO=HBAL!pS|6Ir%W0qQuzawNKRL)LzBR;68 z=xA2k0Tdg#3W(MG!Ydj1?gIYeUz0!=&CYwLlRO^M(<BzI@0n0wxv;1UxPK3`kZoIO=q}M)-dddcuyeKe|npnWG6y z?`@t!!p;}W?~?ov7rq^X%or6YOh+PrW#E8Y+ zO*(kKM(w9tr>8s)M{fjd%UX_hgy-g!ab(Ou*Sq?XK+H?htZ(M>gQk*eG<9w%Ko581 z`|0gRJUQcfp2*j_xu9W^c`az)^B4%gR}fN%_PCD$Ccuz_L$gR(%elK-rNpW(SEIPM zmi1+crKS0vO>R+9m+ecK{hL_s{!N)QGx4yjtWSLgje^WURHlxOm8lZquDe9hDm>?uj zjbUn^?#!QD8Yi6brSL6xrPUiIsUT+)wN{8In~$gT>c9IH|LVf=*AbBQr9{8;ts1zilr zqv5+J;~)O<+^vJy;xeOIx8sx;UWYQ~C??0xa@im3Hx3?5Ry)-DNyG`67nxrf22NL4 z4>AdJH||X?NnX`E0;c?t^wz>e?AI{5la+z! zPt>eU4MEh;h5!j+qsek~NwZ|Ax*-sjGc-6m1VO$_1{dT3;<1g*$fpUr7k80wwkEaB zDjjpTnK^3bQX}eSil%Sf3bjp+Tz6!I$YDChip;D19ILz|9nEtAGGJK%S=(fo2=2GV zc|V%6DrZ7*1OSH@3=V(%`^^6eK)Cw&ZNxpScH6FUX1}*oZvAZT?2t-sZgsWX8jLRr z8^*ZD!;gP8P7a!UcfCk`@i>aWVyM3Ku;Yk+Ea!DoI)%MO>0d=ltNUzOYbq5uo zaLd*@b99B|ErXdF)s>`sOX(fm-TQm|7EzI>4hh}`jSja?Z$$w?&3R_{MjQu z{sDnaNAk+MmGi2kM2PY6!U7iC$JP|TYfQ~ay-!cM7(4e}_drw4a@qkm{glibEGhFS zVdKAGNof${ycRw^9Ibnd{z9%tIg!zk>Nje@GOKborRTx%_gGxGCE9tFU8rzbBA?)W zrAtwec}V;B<5@H%0H!dzG$8xKac3Ek@oOUPA@~~Aj$2<3?#B*dgmXuRr#T6&fuxYg zKh9EzWdCJA97t{e1u>Bm9`?GYwFWoe%$8k0toHY%!t}+%f46O$Z+^1kBkL2_D<0Fr|-*Km37)GZ2N1l=2!r9efGe>36#??Xxe{f$2>AooA50>;?j zuw0D=?{E9|kN7|OLBk9n589D4MGydLEIL!qz5my)8>fN%DJ`c$$N--?U)qu%G*xNa z&Im6_xcVUD^Z&wHfI;}z4{@A0^len-`zXuBh3o-!VjKXolhe-r`S-b+CBq4`c=6&z z8`>BCxHu=L=gyrwtSl_qbb%jK4aq@3%2;JoZ0wUaz}NVDBh=L50KFK>dNP{Q($WAy z=tEPlca{YIZ)E-flKxAZJXeCSa2kQs7Y&y=kl2tm&fFQTZ}??6sCw|>$%iH$8P*-VdcpFhN+cq&w)Tln7OH!e0eZcQEp`qfeM z5m9N2m@g(QI{G+~xpf6QTa6CyAvv~|4$-#FgEC*?rP}5Z#nnd=h|Bc;ehrd8V-^#F z=VC{*U`OTWv7Q{UwkX_hC`uh@r7l6Xv zoh`HmcI{#sj>4WY;h_+&kQ+Rp3;>q!bu}6OeTDzaa{sdhfdLf|K(A%?lh2@g$P5mf zDmkEY_zwWg7i@#hu=vlOar5JSGBv%qymw{<1$~y0m%o-ny^Fv_+y~scIVeu}X-eBC z^i<(N33qo#2Zo<+ym4xNbm+O?#}jzkVxx#hU1o({_tU3ObBc-vva++-fFkn%zq<^` z@xM-;-LL_`AVRg5%0L@_c>$yFYd|S0{XZ8hm=y=&^Zxy?{o;x&3kyr}=BOoQDiCvggb%qn0O}98tt#;~@a~?T8s3>0G0M;cbuF!V zee5P*YhagbE^_gfAJ$fS?OD#S?w6^=sRBfiXd1{&c-8dY=~_r^A6NfqyI9nI)~BAl z>F3p&=JtNzo*cU}RS}lroV)eI<-{9qn@8sP&gQbgKaD=Bh`7~}?KbY;o3q{=G3TmY zjG-XE$9KDAf1-5rD&Ew@(qePA^u_afiQY^AS$S^4tm18A1)k8*Qq%=}(yKSKl}ek- zkQxeu%@W9geM^-ZJd$zj&U)Bjj{4adG%@jr|9Yy|8tQ0B9B zgT0YS4)hx#)7D=LGBW;15Yrkq``#@pGY?Eq(70NDT5UC-77TsSbaV9v?C7kzE7iHg zob~>StbpBF5Lxt%C%?zdnmRjn7|4FTx6E>MC7GM=vG(_H5uJ-3g=G6dV6d4tDM)?yBYuN{CO9M#% zNS&)Q1*|FLz0`GL_0brwdYSrryGSCM=N-RfW4CZd#)}`7xvsJc*c_Y)(!<#Vf7fpg zR!0XvcmZHJTk}iYcHXSQ)_dQ~C+h~Kdm|@+D^sWc{O?J@s(v0SC}5V>Eo z71qromKR;b_h|Qc4geWF!@f9+#FzFMNBlrQfc$s@_~Hks3)HT@Swoyqn-ydWrIe&6Gh7`TYHf#A$=L)&E^Xy)*~xB zC+Sy4y-B6=!skssqAQUXJdQke8;n#!pZ#aOy&aL~A&F(?=j;@)3`0NS?QOu5sF4>2 zKmZ^s=_~>6zC)cH-{iz3-V!ODf9)%)Jq{#oXpW_7oO!>rA=Z-#g-)BF4Rwx`%Uha8 z{zgg*`aXjHXp=GmjQF2}8!Ptb`7+Yj=`#D_e})DB_=Chj&3A6x4)ZIgK#`APUZuUi zt)G8E7x49(Cz|R@gcdxBptVt4xD1U_>m0W|`qQ$@V|(_^8Bd+z!=W=vRaI46NHU*E zGQg}ySDy4s<|yYw!9SEG*G~zL;+3_I-JCwpyD-BGQ{?P)?1D@8SG$!@GG(CZYYSpo zke8iZwaE3au9*IGz;!gC_zp85BqV(M{?)7>0tN$k)BO5C`a?MY1)XyF&G{j%wkKXx z+bn+pGB7)<9MC(-ek8PAU0vq;yy|9$afj^pf@lN>$N?8!kfKhba^jWeWVwa9g;t({ zfuZXA_df~m#;U3UwXQAIIW6<-&=akj1LW=uEW{*^XzCbw5Hwmes*c4V~!fMLfkDvQmB{O5O0Tc;`wr*9t<>&~DG+*djuja1sEJ1;=e zdZ;XGO;P&1IukCv)x?h`U=3TV{PzmrQKiXny{nNfGrdsZBM9&5gqsr^ai1)Sb)^9WbBEBNX;no?%(qBx z6&-YNS38@miV?I)iX!xkVKkFsqzcHOy!T%ELG;XBt1Sc;ZA zR5W_bFvSPVLE1ruf;|d`zVF|EIJImOd2jGjKRPnjq!Q&SEgw1soO`r^7qqDRcg}EG zoc(RiP@Ia;=TD!08>@AWj&`mtz-y<6SMch47T(bUf=VB4g@LFrEsFPhK5S&&G$|12 zh27T&jRyy-K9QtoaRnCRPo|bGTh@_p(nrAo*4i7wjRiK70i`K&x|}x zj|2k0;{gZz6znGsQ@Dy>+{BO#*6zfwT9Pk9-tcIgqX8OdCuqFwxG765Kw+EcR_-!Wo1N`6o&(O#^8) zjhp?5kBlOX9zG+RpKofFaU<5RRxi&xy5N#m3*DJ^F-zjJ9++`MiygmQbPuu6tR4&1 zc56Hbxa4br=}9VPxcDeCC?%~w5zB|%U7oP~@9yn6|zDe!18%O{p`e8{o+N*(Tr zc?yikK&M0p3+N-O9RZQ3#^)RlKQOH!fNm!Cco|QYUwh<=}xG_H1sVX%l<`+7a`ArOJlTGbWvQ}H|6u7CB5q;6%CZPM&`#ufSn?-S`l zXc65*ItDsThpm~#Z9fbL>(hcO*YgQ~T`^Z-PZ*9PJ`>GE={}GC>$~K@t?H-@QBp)v ziQN7AGu=o9JtGL`COju)ZB))x-LyV-K3Fq+WF)hdpdKFYmrA#EAH_hY1qpP!oE%M% zYG2e{!lcf~p9g|pu0Yyba{MlS%)$aDOU+FM?6-?FoxOPU-cPr7wpQ92@;{Il-kK;L zZI-3O`ICQn{%S&MEgH$V^s_s9`w|>|_F_>YbgzJa>bRp{zsv}IC3QlB^2m7uF6VMu z0#R|M)JN1iA8nU6ONN}f78Mn50jc};hZ`>_3JSi`^4gb-#y<-%w!!=%;fQcHEbp?P z0)^Vn4?IFEH(@0T*|zlo2x+aXn{nFWtxC6(HO2Bz%q}BACQ5KQ;Y%!NSo=5dI$L}VVTImGcN|8gU{g_DI>EbQ!njOZDE zYQh)Rl?Tl-*ru*Y(TWU`6){#AjIMgAQ|)wCMgcbN**I{H3a$tr!`^|nS1nn00J5tI z6_1+%Fsw(6M9a?`4@&7y-0u{cAL9O#S>y!LTOAwg z|8Ca16?=6Yl{;xZ9B21IUCDItm)sskSmY!P>wDM3;@Y2vrXg{%o96asapRKCzvVOo zmz-hwVxgkN%)aq<9Nd#uJ!T(-i%B$p7wPXYN1$~Em9-~-<=0tThu8TeNP@=7TqT!c zYgj_&cCl|C%ntNLjsNUc)?j4Ge_YQZ7i66e!jn0JD9#_2&+bF)8b#!MB*}Y9pM968 zh!zpYQ$uUKvVvV)%!F!JvA}EendOtpfXmFppgb#(Q_h*50|3Hx6BniBgr3%oQ-06OE2m?-)Mv{- z$o}DNgRyYHC`?253#p#_0in?(UAepGZ5#!6tL!g*NAoiZUgvj|%INXw#Wow_XH(}q@VYf{{GMW3^52cfg#rknq3=W3 ziAPccK-$nYhBKXMxhAQ&>MM~{g|zsVgZL8W5ShQ>j82c zq6J8*hbsaChHl!U2;fX7ohFgMkR0!1D-kR}Sg2^+J+q+s(nw|y4oaMJcR*azZ~@Rm0a z(w~}>^kAZ5t426Wy&cQjl~~gJWc8{D1s=BP+hFM254Kn<;gb_G3bg{pUX$rFt-CH8 zQg6rnU)aWt;%D39-3M8Kv`~@Cv&hL;_x&5Ta?F_H3U~ZcTDzROe)wHn!~vpPJUn%Lojh!mzY1!_PyA<@UrpqU5@#t@SoYdAZI|oyj5K_>OjFyn>UG@7AURPt-;&yJ7JmuTdJ)>2yn3E5 zt8HFpS+;GZfq^y>;)Y|HVd7$58hmwLnA62ltUcY#g=&H1!&9{Bdb6{$YJ&HnN&4r+ z_nd27M3>=LW8Z~*A?=tsPh*b9#`+xa^OA_x?2)`<7o4CMEyNSl{)`Ngcgs}=(<@;` z`l}X`C&PxcqY3Sl9#)U2Nftev$wHGi>4tX`=0R{69|Z|vKF$WNXZ=IH+M^IafqJ?v z7`nbJL962t)36Gw`e=Uq?V(T-8|vWgA_A05gMFqIRTTt;T& zdyzZ24O}DNot2M8*Jg4jYpDt6e39Oi`y#yS5iuEMf62g zLL`LgnM6i_;wkr2m!WZ-Glj1d^js{*tD0No#3{b=aA`l^Npe1On7P}7Z&|xvM_Aa1 zM9dk?w5kU@!}K#JpYE$Wr;kj&afJ^JglgN%r;N_Hw96&%E%$9I$+H#$<8neBv4!Qn z^3`GHyyF!^*H3v86B05oE&s-0E5!Dq;+c^B%;g+|z~{9Lif;H-kCKc_4U@pqE=H`j z)lT~c1S9jr2lCevL&U{c->4$P=9tIzQl4SI{=gHMu*fVo)-}9&aqDdCduYY5C?&S% zE$h!z=LZ{(9N2F)C%n_K3z=053PG&{#(zAn4xeVT9~1R1U$OpB)zQaB4dZ-3P^-|x zxpfm{6EDK<-@AvBPI;I7a_v7NVi*_L2Vsm_Ce6dGA%U2PmbRHMjkiGwVwf>;?!7_OmlYZ$pk0EkUi^GR3zDp=C3I&M$SV zEEbt&yAcwizDi7V?|3XGw#MLaXJ5ZS)w6Y*@&oX~h}gvTczNF>eyZH$m_tdYseU|< z`*zdak>Nh!{cmKY^=`{V6y7jOZ^nGeFcBv)j7=zWG<+qk$x_R8@M*^`rUTyhs?efY z*`iV&>d^_7wesf5`JtSnwizRMdZPc~oyyv4Z-UyUU6A?K{oT*w71qV(la<{Q<)&1P zxti(%_6tAE#|}!JnGttFp#&ms?(75|1}Hzk6{vdrd_8s7{sg0V6ULM2>%ctg&y}GN zaNqS}yVm=;h=naLs?XUlh*z^~;n85%7h1YmBMgCq8gZXpUir~hhxO-MYSZTD+NzCv zdgBXwqqg{I?`MGcO~x99_3QgDCcXH0^YFdY@KT^z~4Rg zAq{ku8{d!q_O0TC)3Aqfc9BT{(>s1~$Xn+0+t<0NnPH#SKKzdSV^Q$@{rS`$8+r+E zNAtZLhdwBe+1i{`HM>U9=&ajlGgtWF9J8=fW0k|U6K$Ur0C`%Tp9RNn_a>?Vq>`M7 zsU-;@JDYA;DoD-^(jvSGSi`At^UMOSxD<;fb8p%ZJ(8_d1meNB0GFe)y01JoW(j=~S+TpWXzpgroV< z)5pSlZcC?$!j>W1v)WsFclK{xb0v{H>d4${6F^S_$||`9ob*C76^bhfJ%jS>*7?LB z_Z7G}f1BW9smy9f(xRvJ8IWl<*{NjY#tgqp9rnB(cj1=pP`qXR>M1W@O!ive)CfO` zJS$v)NxleBd#Oq}?;nug26!n#mP+-}+Q<~drj6}O=YT2Tce=5r2C>#uU7y3Xy@on zO{YJF|7rKZdOL-BC^|+Zw{S9T@84qv4Gyocz9!o3QUM+NNX2;7va4TzpDO7PK0^%g z4bz4gpR987ZPx2(bk$I}c?(%2lb8JFH;`(xX(@aP9F^sR_a16p^b&2M-LL)fdAbF$&&Z_Uur=&2*65vP*J7X zq<)`>ICh9X+BWXc(CFi=cN{ph9ZcV@MxpZcFvy!}TLlZ;i?QW^$!5{T+iu>T2;BXh zaaSisK2M*Ry=nVj;yDBXZSSTnVU#asZ6-;CWwr1s1(HSOa0+8|rTex1yqk17eO%&w zm~vMTkC?x%W6(7;kOcQQ!?To>ueVwb2h}A3%u&$YkfZW6RaOAoYXJ>cK4Ii zMYj!KQ2U=XViCo0>TJq#k-pmwQj#)67i zP9fx6{NlyZcv%f|-&|-b=XJs8?-Y3Cj8xB^Uv8BDrV!1Z=#epJoR$IuHEKkHo(@3h zGf*-0!8gS!^%_?=ojT8Lrny7#kR}Zjxuzu|9>3XkXmd);-2piiC~K9^$L2IYyptt8 zm3t_die~OjXob+XKju{q#L_G!;lf+8=sQ_M>_}Qd4cqsOs9!u;vcmaaN zV@N9DwI_5i+3MYMRA1O-lxoW&7~Z>kJurNJG7fB+di8c5B+|r{AUrmv!h?WstAk65 zner6p&aE+ikB#q)Mk-mFbv_Ve!Ub zGrh&;L@S%7ji&vb>UY-&Scfcz;!&)BJjwHETO70k$qZc4s0lJ~^oZXg(45|1?rp3~ z1MEi%XIjikhe)VJ@~nfA@2{Byph!QOzSDj>7lP3UA=5ZUWJ#ExN{+vET6>Tqkt)}3 zT}RWrwWCu?nNv?dU;Wr?H(q*rsHyJ53Ip9Uv9(r}^-qz#1I*>RPDVZGOwo#I?+a;A z2&M;~i|L94x>w@AuTy4Ls)rU*)E0l)&xc*33-O&!-|WR1p7{;{ z)WxOlW=TcqW7DnC#`sK_o|JjU^(mV`y4U>@o{t22Cjb{1_fs5vUZ46^ad=*tE&o1$ zEQ&;IJ(n3^rqo@FKg=pU?Bw}Qxmd?z1@jxWInO19dy}$viGBd;^7V$xeUovq^UeC` zqg_qYZkbAHHW|wx>gnkV-od!<%-&ipSTy=%Bmme}3-FkPaPh0hMK2q$a1O}^QxJ7j zpA!CZs*?Z|^q)UAOQf-nwlrll?|d3R{{;F$!x6f$u-5x4Z#wg? zrrBGZ+#bm`0q()@rXY`Ice&+{cX+`9FqD?J!27o9DFCFcd!L)&*rx%DR*YChg%rrAs0x_Lw z#q$Q^o28ZOI|WN`Bes0lwMmraUEZw^ck{WvM+(h!hp2Oixt-O+!n`4POQYp0{P%eWBWm(N{Qp`9df7KP#o@DIDsT-r;Hh~N#DF2mt3~7AN>qy9OQy0O*p&yq5Gy&FQ5wj zQyfj$spIARYm!^gWpu!s=k*kqlB-y=*#ciZyvERab%3;;2GE?}Qh zTbwKfZ$%;6;XYNSUz+a?+kF0zAhPi*VTat)a7*Z1NV%UAeEMvwl~MjoMI=Of`;kZ;1|^5rVN^9Tiwdbgr8!_*LKPQ0_ULTItbZUn>z)W&hGvSlz=9$6f6(O%#?*tige! z3Sn`t4_9kzAQkuR59gGI7J&@MNj02s`Xma{x&rFo2nP7i)a;LofEMG}6v@O= z{Mj@lhxMgfDO8l0rjq8j_@hsw-c0oh?bvh4aGT?q`}W0m`!&k4`>NRIbVmrYnvl+N zg}MXt3dVP&?_&8zE8ZxqsWbR+hVKReAD%m&+A544WW+%ryEZeI@PPmeK3Ptu`MGxd zIutw`kUG>rQSM4Kz76Wxvt8@Bwl?tA`#CsnN%$dTC-({{E7fx=>sM|&1~+ zT7KRoq*eyJY|t>y+=o-Cj<=+AV6}`76O$Xv*$5tSVUzSr12}61HudERm5N6TOE^)r zf2Paz98g10b8oV%!EQ9qMt*ReYx@GcS!TjFmXTaI41meAc_1mz_uhsJ9kOe;8xcHppi+7@8N zL;v|p=O1SV;YKGdF1Ek`3nKWbXBC+zWfM{QW4C1udr5oFhKasMjYPV+`_oq)&o6U9 zot9Kdvl2Wft(V95?trpAZ{-vvP;X7>iy9ORFQZZ6!rjf2WQ!qnn-+wGH;Q7VaTuQI zlY(ji>5rsNvq=#%bsS}aquS=2@$tjWFN?9uRj1Q?iN}PfcNJ}(T2Pj*4U-6$Hzv-a z$*H;5-t@aDK}a6OoU1;dHB@vtC^xKD5jRuOKGDzD`~Ez5Anh=|zdQKCQp?~kbAkn> z3llG98{QE_;_;4{0sKc|iP>CV->GU#PRjPJqJ0ZkPIqL~|;tLrxoXDOp zqApBReWf#t-9gB4Nb}P7_{`d?hV4$j{yElFj^~TGGr{?t4j8#jB9go%=F%3b+C{bD zXJ*3&qL=J6nN@4ma)cYRUOzH8goS#^eu({7T2DuCG{S+mqQZcJ^jFDpK&!HAh@=Cm z&1t?Cu6jf1HSq50MIPII*PF52b1t617d+k&7`mdjbhPnNT!`Rw9epUyPXJwWmd+R@ zi-=nBJdt50DN)JXCGI@@WWUT-KL zZmKr=#Yj-@)2qznMXH|W&N%5CSuIuEf?2L~l7aKj6^cHV6LnS|=Q`Z6XZhavy9ZUV zD8ox6P(dn-OJDTMX(`+u}_ zbzd8oy~x^Fd)k~TS}Y27H(``8ACSxv$LmV-lWIoi$CAR|u0cJp}tj zzz5ybC8~NhmSu7`mY#oaL>a(LDdX{i&SuUev+XS>%D zTK$>BLis?h?-|C)45wE4nb_(>BVMxymPxPs$&v%m8QI!6MaBE++S{$~1ThqXd76$> z!aSvAyI*BHwb?JBiTKv_gyK<|5;8~EKRk$@NXQDxQqfT~{Pc3Kqo8r7i8b*dZ6>*7 z!TR!rg9RIiYPuk1E>`q2|?$=9>+eMd}^B+B4!RPa@o)45Yu!Pv{QmjpkE0X)5^=Z7xX^YZHLbl8zyW{QlIdO zxcw>nXPS+5sZ60@Zlik?RA2j)KGzax*)5V~VZo4_Lw0h~@?5aGUt8PDHc2uf!gRC)af!!42s zLYBfP2)~CP(y#y~ZRkkUg%~E4_(q9T1Q5ukKjohffURvQ=&N)y8x0eTGmB+d;CW7+(FFr3&v zgd>wHnet8a#Zql?-o0R*xf}j`n4a*2)bSX5s>HSmxOac%{i^Qfp&`o5ZyVXtBOgt= ztu(Do&TJa1f$HH=d_Ox>Oqh3u;mN9=yK1;3y9cXnkU`U^z7Zm0F)6O76?ma5VC z9lCj!yfz+P7y9z^%_k3gm5aX6=RdT5t9a22+yoa`@{wo zUDO}f%nYJX>$+OSG1N4mirDgBFUBFAm?2y;ZH&D}gHeT=k=jWOZFEXnGm1iHYorX% zB}{(MSapKm{dI@?(7lWT*dz1!oexpiE8*yf`ybn07YBrg2}U0>V&yjYO6$8?6CqarZl>y_jm1}CjO=&c=AizWLB3|kqkEq!iDFM54q;$y(Zp#byzhovYmc;w8UC|EfzEn=-z#?U4an2`#fI(Z`(fnd$$&GK(&7WArteu zxJk7_8SiVR_Sr3%_$DA9YP{2k!eMa6{YINh)M<7&i7O|5l9cM506{Q)CY^e~Qq2^FC9-e>)mHrUFIi1E3UJFBTtXDx~Vmh(G7GQgKnI3HOJ|FyFzwUa`9c zCW*3>S|(Z{u3=0@a!2CJH1OVq23K9LpN~Ut1iPx&UK4^LhVJxQFlB7@rGRR6byhZ( z=*9E$Fr>$bZ&Hc79W2StV1+nJvHc)i7djUt^hRcUE{vMKmLXACz(`>HRJ}TNo&xmi zJ10h9ulilzvd{R}Bt&Wzodohtlv9W8r6XYvP7(>~pdUb+to@S5ao!8Y_^J)d7p8yD zwN?b(2pvSq4H20p(Z*8E;Thbpy`!^PlK0;et=k3Hhto1KMJ>qyTDbtkM_ZtOy>yu& zYuh8rcuZTRqjOXAUXxQ_2^kB#j_voL#L#%5Zs^x(l+VBd@cJ9F6iX+>sI@PuzHNE0 zW1R6UwOB;egNIQTW2&}03Bo5-+1#}?OboiB#tC2j$d7r^HX+ZrBX_{9J1%RQRc)Qg z+YFcyA8)zNw$88{Xul0GSlyEpgTBP0oERq&Yf&`&fz&qK@%!h0qoRGMLFjCDxwc<; zxcrDWhlv3&o4>HiSZqZY$Sq{U7Ro-o$TjWwBo)KXA zNFzyEN+A)pp%r4i%l1Jyq@Y=e$ifZu%fE@=2?;*!cFyxf_hy%#CnzNDa=tk&aK9Pg zUSXD^lhLSWc5?6x(Q}r`t?utC%xlGjou!m&eAB8IT-LEYyb^_O*;y3Ca2r68mq2kyP6yhx?9LUTQ6wBqaMCVJ2nK2M z|+=cmWE5DGK&lRHJ{y#)K;p&nuo*27js!ze^bL#$+lC2$i{ zG{uH2$GRerpmZFn8_5vV7YRWe3<)F2J=dq&w4QCU3)@j*Ad%`@dccVOrC0ADw5F>6 z2$0f+$&Qa$UtPe)qOJN~nl%97d$}7M>vBEe9}>h5e@|XF3JlxddCrtCojwW{6wO-k zenm<56yd+e&1FsGlM*{sHwNkBKH&Fk@phXFKSr4QmMRfvZ|-jfjqAwPLgc0}0- z6Czf}5q(MX3tHD7N4-%{4bI(@k=1EP06MJ=^)Jp*PlDgSrTwe55@_28On@W}{?d^v z-EE2DfGy})*AzcC9CZO}T-;7B1XXR3&?>PP2aAO`QDqvSueRNo(8iIVM41lU13X@Z z_ZMq24{|}u=K#HWQ8|c2_4?T0aZ5!l0w8zm<}J4x2=!|Q9nj7M%i4bXb)4$6TM6)r znaejNE_(?41AQO%cAgUHI2f@M^d;l+1lO=6%lLw1b%Fl*#9w}ntCYn#@w!doZ0P`D z8LhqSd_&(5t9|3JxtbaS;6#nj2GQxj|BMkBcz(hd)@+j!jJ3U~M0n!;a@qo+M`T7W z>|kaI;H!>_wKI4Q6$r z0xTzxQ0yK@F)NIHO^arj20_P0hSc;ouk#HT#SVPY#kV5JP1wX5qaFzntpEhZ35-T0 z5IUL7Y5-4kJVA^ok&=9o%}+bK6-1XMmKR~ooitcoi7goMiSe=ViLiIQ!{nU7G>6y) zp8>3zJVGfi@5D}6!)F#CHm|vT6~kYFDn?T)d!q?fQO9X)_zwd-u(Gv5K?(yIg~F2{f<#i|Cd zJg7ReQUSONRlTJTyuF2&WlEQdlxYCiyjR>X4QXx8@N=tNuP$8+D^C#Z!244(yhA$4g*x}+2hWun zi6-N=8N^^QMIA+GxC0kWq~u-**74S4qYJb8a*RfgXJ;#%QnIXLAvl{%mWD=%f8%0c za^p!fVIsq6TxtL!tpQ)GWbWlf3YN*0ioT?&yA@sT6#9)f84&+Q`_C`*XxbJY&=1&b z&n3K%Z-@5lsm~Fo`0fV%0Bb;if+CA_a;F{*F+Mr=As{&bgDIepUsL*(yR!v35=pX_ zAcDU0mK;&v`4B!u+pFbGTSOz6%-Gf~Y_rcxd_F7UHXz>maI%w{78z~^WQB^4pNfso z%yJhUZ;3?og5Ya?b~PJmqgN@JDa(Ay>H_qym1cZuOmar=YNNuz-(}H<&{#!XPS=h7 zWRhjR^>`y-dmMIYewmj0*wT~U*MWERf4BfHLBUp69i74AVO;Wi6uIcR0>g$)ZT(e(U^JoX#zjnN1K9y{zAwArAw$*8 zBHs&8Ijlaoe@JO}@Em8#8YgG(M^0Hq#d_K9HR@#qYQ5;HKHk%|3$w4AGX1 z*VvC$-)C#8bOy*srOb?`iAE2z(ncDZCWhhw7nXza8U8#y9+<)-%MHPi!gdkwpQl8MZAC^QxyKW?iixvEoEyEPu8wZS+c~H5W)?Sl zRL{`fLhC|w3rJ>`K%{l~hHm3A);&da#!aI;=U)lF@Otmu4l0PT;E~nA_k)QM-_qc- z&f;l6HhFh?qe`fw5w27ZS--hddUWdri;-+;K+yE?s|_v1E$ZN2j_jLsGjB--B=?aG ze`n~E9exRFB}8GDrMK&DMl8I1zCW_DPnTB{W&4Nz<2E=b!=`h*>c{tpWF7B*Iojyy zZ`5=MZpP0_5*lHOv4N!!>8RqqaOn&5BLtbNVEmawFuh&zxcyyg=dzPcQXUtx;YC9S zVW1QEcbu=S#fL95!CqfWzBHqw31MTLM0NDu1S6m-&-b-ua7$8E9fr1S`-{?(mVl7M zLHaI_^KN>LmN`*m(8>5xlnVbIHre9myw>x8ifZBK8&2;eB_xK!)@w+$g)jp8D_w=3 z^YiU>b?y!}M-z4B^G&aUkYBv0N8(9t^#CDuVVj?fKA(H9tD228kPtMG_=Ty9(_V|6 zc#hxZp0`~u?|9_ytO*LDZ(4<3PVp?njB~@?#(3xAQP0_&IfN`lBt#aT#%I#af=x-} zvUa~Ip4YAFu6np`dziE24gfo!JXhLe2_qW&US6E~tx|HI;2gcjMm~ma3!jg% z#GzZUe2)c%2b6#vKSEaGCF=3CqLp>xjlvnJ-$767a!40Ly0305<f>MSjKTf!7ocKhRFBSp`z)YdyzBs!Y^`%qN~$eX)e z3Tpo}P&PiX^M;f1=!~*X&uABye0jv~k=0Ds&aL!1ssG?}5`W{+)sP+A!zUKoa5Mdu z(C!Ak$848RHMEu&f}T!K(hG~d1^{SwQMPDt{l`b}*Yz9z#TUMP(N}zyGrjHQ+|_xu z8@3I9_}r)Ka&O4nuRoS^@3bYu?^$IQxoRDxOFyO(cJ$AXpyN{VagsWdi+B}#W=v=t zaUuPx{CrTJ+vri&xPMS{Cu}~z;Wxgi<`l}fe(YPlE{|HPm5kg*xLHIY$UzDmKS~bu& zeVxonz9k~Pj;j)T3360GFu2@x?iaWQ>W-|yhd(nbwfHL06uYO;*(#TM43bpy!@dLd zWN-+2(>sJ;%aLAac3IS=u^$n8&@_`_T~g+M2*$j zBUm)Dppo(0F0vPI^ED}!+jVYh9pP@1kXDw>%OzPdmlQX$q~u2vpDxA|Uu%)SMV)PV z!4u*W@xANlJkHmi^KKG@SRwbY)5_C7a??h~t4^E#)@2q$c>5LYw{?vRJhm?+Dm95r zjWUK!E(X?&B5vNEH>GROgk*X7Aw|Q&q=#w;kwv4y(jAz zJ=TXdHpg=LFF}H=%fc>p75=eGWC;X@=wepNPF#@Z zvvfcUQiLG$t7n6RD#pgK=;Y7e)qc?ja(F#tHS@68Mnkx<#k?;%66h7!!LOGTtc+g& zTF3FLb4BH;;Q>>S*c7Dfk`KD<8L9>r`aicd*#1@|Box29;dK!(Z;Ke>cIO?(W>PKm zka_Q#+z945W-z!)*T{0>C1V;|J06j>!vbg(6CY*xn0jEMdXnOiguJYGF-` zNFM;PkTrLB07yR?Wl|~;isxO+{qz~HUb3B)2ZQ7j?i77IM zC+qrbn-p(sie$h~h8a#9kj@rO{rQTSs)NL+yhbpG_p7136GLOE*=W<{Y+iEo%|~`F z(^-vd(TvYs$Y$7AyUQAoCZ9l z#y9ieT9_HA9s#59Zx6{0@ai>(m&mCIZFk=LP|;5)6 zNlVPa1O=#h(r9Gt;rEZX4epxJ3U-l_W?;ZWL!05x-`k1^*b!?jP%^!OmQ!FndXKN? z*Q}~Ka-U?eGLy&k&i1Bm;p@Ip7bxR|NWczHBcVhOEEL5*1xB7S7=D&vJ}{hUSXIs_ zbpI-XOb(fRGAS_;ewBF5PD|KIn%)vob+>EMp`j$2h6e3JvZ&^73aRFj0x!KgKRZ9X zQ%RKT3}}`kRf~|ukW1(Yki_cjnav=g6O_N|=Ekj4PSb)Ep`#MI!L=>Usx%HfF1zr; zqT_nC(>je9YwT6D-d}C47IaUmE0oQo@3LUEZYr*Bt3_%1f)Rn0`JWGUMw+!wO)IrW zVnpz_xPY3j;4Ngp--fz~+_y+@A)HsU6RYaC%FUnmgDETwhYR&9Fm&=8hlVF|hMsyJ zaB%M&Ofq)!-JhiD$CK9%H#ZNQh*|Wv+NxFgUe!DbQ~XxxuidGxf?afdIao5e64E38 z{RjK6-w*iZfWe2d{YeC#aF6f!kyp^6tNT`72I2mix`SCcEo*@5;Gc|CtirFSPHp#J z4kG^RrT>1R{{XBH{RfvaaAe09#SWddz^vvGo~Ogk^o~bm;CddY4-aVrGMF7 z{`HG?@B+RVr_rA8IOQZ!^yyd2PCGsxjKWt33sUuEzRv&mJHdYy5fg%crH8l*Kk1O@ zRt6F@{Q z$-Wf<--H90@xW*{y2=PaZt}N~?|iWE4h1U)SBTI)MIhsk!!BM{7P}9RG zDyY1FaLK}NZux>%tOleb)N6U@u=ENWymkQ*dS*JEn!+rm9PVAC7b2Ztp1xaS;oBs6 zk(Om5P7{Fiq5H8^&;{)l^^2l1r>xE#+07lab7ckHq~HmJ19$l+riLDCb2HWW{OWyA zA1vSO-Vx#UPr>durmlFN&!di~eiT@|q(}yB#=0)bp%(sk_Dhw%dTFs;Y`< z8{YQio03V)HHVQ|LAdAL#i!=5m;apRZa-2G6I)Uvcmi`P_s`(KK;;Y&ylisxE@1U> zZ00uFdby=BXIwm6(5cH$3)9$en)(SB1EY#Mx7$WIwnjp91wQggh5xhie@vYYUtnE3 z<~3@l1CvTL`0rN0xBRQpC=IdvB@DezsnY}i>}T72o5`@PMl&te5pkjbf9~!kw}TPK zy;E~i9&=EW>z=70ZbU=`5NfE90#AIT7I9|-JRJ2i#W3(V)~bhysBozUJEu~@oNlbk z$<`V-{1yjLu*oI}I;Gmm1-xRdCS;6M@g3Q+Mfc}Pah#=km=3hLrHTuzJSMeuTu zkal$Z((u9PB?m`;=nv+q)zQ||t21eB;B?=GJo(d%K^`wk!S%gy(ou&*&c}djak20^ z43oYjX@ybgBCn}VwwBXFBypLV;j!jJ>_RhVwCz$u?haZ#y84PtXrAPc&WoSyWW? z;|NF~&j)gIp+`Sw4n3t#=ptXVY^P`$Mx>{who%O{{`bWZ0D^J>TO%-E_M1S!QlXu& z?&=N!{gf>TEJTipy3k*>-}F4+E&zOVEC=)(oximHmD>_B{*A3~{#v4%WhwGi;`za) zRC$BeL+6*EQqh;v<}^mphBkNgFE3Vq)?cqdKZ`TP+1a2uZ$li2x8uHrOA^vyVk#$O zX?-pT@M&qqCpff1^KUyw<($9@3_=jY=;}tsCzUK}BzdDeoRFjy6cqICeRiNHnXCfJ z>`C<*@4(ngc|*SlH{pqunK28caW+{sHH@Jl)`=sdF68IKxth`QfFPi2D-lP=Ye^$64r%r{;dff!&T`-n z47h#*Y$>^cVDF2=xnC$an5aaILJmHbq7SFt+e%I+t=Fc%zrE%6Xo2KW0ZzrWJH2-G zOO}Nw_q}F+Q>4Wt?k@Js9*YsACOiLFg&ufBkNsy!))0?E*%e$WB>v*_p7dB|IXiH0 zqs7pGS`X-SPD<2c{oI7^x_CIhdq*%`ZeY_~_>#%-`m9hcO6C13#~y&DXH0asSdL|X zS_AwLgkfvlVOeYOp-SZ7Plv9r$Ae+$GpSst!D9Ekr<+&{Twd0!DpV_wwb57IMpaxh z(z^1r({{pvkAT8|n2k@Wz`tVKvTUW{vN(jJaPsMNk`fr?j1t+3V{T&1Pb-1gwzG+m z8_efyl8@+>~zyL(}R>bbCWg*VFke&7YySw3HGV4!IzgisR6R z?M{{o!QkjJI@>uv%sd~z0a1uNo~op+3jTU#So&$@Llph)+-q{=`<{%o=gsd(K4-Tk zJ@ug1i0egLVW<8p$8lp+^p$p6Eb|+i8;h&|DFQ-m_%z!N;=T=xs%Ooxboc|S%_e+Zh*JLiPcVZ&QA*U)d9FZEcDJ~ zq!I=Dcn0Z<#n$xG_g|3a_NOF^a0+lYMXBj1=1Qe;@~=*`Z;451^Cd- zyfDanz__co{rS;>P-0i87cc9u#95)i{kgR6zgJ#RI$$fy6|~u}0k*ObhE;$NpCZ4{ zOI+f+GnP{~oI$IrTW9;J5D6z?QBE~@3$O|q)kx3Iwyl^X3|dE)L-&%HKp33h!2aGf z@haB5#mB5bAdFs$y+TQb{y;Dh!=SpWn`~9Z6ZKW(unDo}7&?N3KQ+{a*^g=ui8RQ1 zv~L4?JKEZCd6!E%)h_pE%(qU8{$VQRrFPRkNVpD5!KYN?a5g7&d$N<1R}^yMeVLFPg2W_UwPmEH~uUmULVvv`&D zA3CBFzvSrZYUf44=4MY;+r`vK4+-G_))Loho7T%tKcAg#aceM1&}Ejo>DIhigOj^J zk1Xn1#7KL+^3>v1GT3o#Ap#uxjTB=9phIkJC z>9B~R1>?g36r!|`tKH+cnc7@7Ud(#U-UPgrt{*Vvabg7G2)q(CP77u$Pe)q^2HES} zhgkzGO5;|yZJHM@SG22FL=}WS_d*6%k^&>Usw-sC9Z-JaC>S4T5Yl{R#ePw{K3oJn zZ+&donQq%%sPX*p&df^$s82hS#vNW4to`$yYLtGpH7@DLcd(4^oacxM*U9{`_0m6N zsXyLS#mjpvFQTL4S!G#uz1DJTBu)1A)H*t5T zxCy9z+$T`tS--m09<X~geFxIa7#|bs3h;2(ndlPG9T>e;-vd0eo$*|HA*RFuJDnWNTyz!P=WG$7wfWj`GmP;vuKJ2J^ z^Is9@)heLIgVbe3YqHRpRngo;AzUBVab-|4dQjvMM|%E^g#&Hs7uozjJiikw z2tNt4pgiqswe!$c0DWTOsqlgrfHr9|Sg`j;QXK|(7_I%yn>qRNQuAv%I$Z;ujsg%0Nz2uSh+41}I$e7rvuB?`n#eG}I9) zI>9vnJ>=7!qEW&N0l>+^Z{G&6BZ+W!k2)q^@~HscU^7oo7r)f)(0RK#g|IV$ow6X2 zd!>;#6G8y&_PiTK&7PtY*27h~70zGFkV0NI4&1Gi4mrPaVS=LH73#EW!>*#MpJzV4 zdBa-i>HR9#*OR%txf)9zCrSVu5K<81ex%e-IPTE<8-B#~$(CZ1BzY~R{z_Mn;du4g zdX5$*fYouhmjm20&6Yu>GgSRQX0AKF9+fy=XmcE=E0}q*@Y^c)o@05eO&@q^bA$@4 zti&~MU-kuvS4|sUN_Rh;{ZXiN{bS;DUH4|K^DXJ!UJPRdlk7dSh(OwCw;hB@`}it;Qlq;({CtuIYQextbv(HB4E){Zn>WdIUs;i?$HG^og+gy~5&OaNEMOYcm+e;4&{`O(J zGw(z)kuk7$Z#9bQ1-v+ zrXCRq^dz<(DcSA+*iDLrCPea9)Kkp6Rp}yfL_aZy#1`KdsxuEUkOBD?j=;e`bYQXLOryMpsM8c2$2Ti$k zSb2p#BnNS83+H#~Qa(=y%}%$o?&(k+3f>FN-F#*woShMLDbC=nQcV?TzR5D!If^!k z*gqcD(n7%dpywAB$;-q)f$%&xL?p6Y=oyZh-Sil0X#IzWh;@dNw=*ag>%uiZAs0ey zdwrTf`Ec;Xj#N0Gktw?T5D%+2>{>##uY^_8@tWOtq=;tAnj_b!fskDfrmOfe5{I*` z2}^u=dD~#}iy2CJ?($rhKozA}u8p)@1+cM)p@yJro98PapW7^o*OL^|A;Tx9f^W<% zFzp+fvO*f@7KC`4@@+~Kv*IEFTa_{*nPo4DNoN}kDD87*NM4+?^D8l`-SAZv=TBbD z4}0qV)$xigBxG&jAdb>Jfpd#r*(miPqF3 zPxmX&nIwe0AWUT2u$@!kJfE`&Pj0vjeKy+Kwz**Mmlm(P!FE}jm&wS zBTV;hrv1sn@B+dSB%KsBQbA%HJlB@;x3D_H!+AIH~7TN#n*Dpp<{gz<` zdcr{E#J{+*o|h<>kEXqSc?1@KMi5{7cEK>1uqRjs8cvAF8(1@Q@v!Tq=@||jwRkdn5ibcJ?nBL9kaQu7`YFhAZV1|tD(?q`1wD`f)LPRc+PYfNhoRT;- zi~Rd>xTHEJW(M|npB1)PQTk%$IO>c4Kyd@3(mEPJcn`UQsxAZ&j^)dO1OV*!N&35y zjk}7(p&uK6U6qsyY%bXAl~hM1XMD*&@vXuANLQIbH6aMVTN0xgjr8?xrBMm{sw+8g z!U37MysRutStK1mRQzlv^;UZZMG3*_l$oJ6cGKff|DJM|uZ)-b zsFn~xTsi}((a`jJTHak@y()bmkYd6-R37S3t~imhb>-R~+Ye23qWy4GlFPvr7rvQn zOGs_0equ|hslK!E$AdtXhE^mJI|99i%SP9#*5TI(fmwyFH#UH_o}ORwS6R*XT!CS3UxqRg6N9u2 zJ$qrukc|T(dSXsP+qItEf)4L2h#C>%?#~WNe9<(F?c2auQ+?_?3bxu`EuQBS0AhJwo8M0;2H6EmjbrBy`%Mf3PzNy%-pAMf zc_S1}Md(HLtKVd*Z9;*~qbYbN!_eb$+}-qn<_#rC1Ns+h#_7cgiFFZK>rAN3G|{=i z_c&xKF-b&Tr$9tkSPKm89vD!(Uz7ZlUyPWrOdy&Usv2szhHI$>IJQ6qn_p-f`k`ga z$qfv!Nn5&eO9%NROuQm0pnV$PWUE1$0+89$lY(Uc!BP%Va zq#WOCYbmj#dZ>?A@$j%D4W%NeN=)*Qkm9&k5TB{Z_Z^zAgOPkkR>!Yi(w*v>&^|F) z3OY-1El%v!N*~mu3CKm^!$R)<7h7Ke6=l19FD30z(v6gKNe$f~CDIK7N+UINmo!L7 zBL)Hz(n_aDcSwVP%QJpZqNB>gA8E%M0&(FSmqhsr~Ttha;7*@W{6#;MO-R?CdhK zm3IjJ(-kvUY??R3S|@>Uvb3`j26(t{PT#*+`{R6JW4UvW=5F1&gm@EgkDE;AlG=~U zQtwjKaP^f&CAvg;|9Q)X?l;k4ev9a0Sg{Scti14>HNOFoIHB~0vfn)&9%uHEv9Z-B zum!Cww+EqX)q3%hHWGSiG)!LNTl)qLOkrwRlyT{qCEb+x_LGS*P5Jwqb^py-L|n1R^9N@_^e{fky>9>?Ioc7UKPr+Br~>T4G14%MDP5<%A6I0tnnEw_D({KkezTst zvfWbn<(1O&e%jQNB~|cUN#EDD%UgR;_)Z#q@t+FRzn(iU0ph>TdE4Cz?2m&FK(f1- zDoj0#RxgSHnT<6zZIJXdcQ3s*Ltb_GruMVcXDwin^BP-QS@}nz8#lJ0M;32q`li`$=eKug6Oy~52{RkpMl09fQxt$xfz{;uMFZC%*`0d%^%F0*tFuoNX zgc0O_9zoa&i$+72Kz~`Q3D*_4}ARkV;juTI9H(&blJ8t5<%UUX+`L%)7Ku zZZgi~72de(a~2}e^ml?lK5H!%R z*s3!54_A+b4>v+q{k;zqbJxi*Z%$--4MC)M5^DBYRZaYZ!$Vc0$e0*ZnR^w^&fsj? zlXgL1-glOjJ7g(Wc*wo|#P2=O?iAwJ=@}|XjYZ{V zj%|r`J8^8N*@>X9qS+EswBtsel&aH&hH}fk=CdhA*ctx!AqpDPF9n?xA04@QLF?LO ziy=UUjFoRG2t(gB8SL-x1P!Mu9DY7}{ra`pM2UW6bTpsT5|EJeY8|sNaqhX8Zw%#w zb2O;psKpH2`{y09NHwHvGJ46)0BQd6h<4^dsQFY&YN}~|Q&O5~aEe`=`TD%uEO#** zfkBvGfk&gEhi)_qU&U!Z&7oMb~q;l7FwP!XqnIgW2 z{eleQ;`28qe>9utL$rj9;+NZe9UUEJGgU9l9D`$Ga0pp2=olH%K<87?8l+(8o2QfL zu6-hW(E#?5@Y6YLf801S>YlVl9@qTkDlOpd6w>1G&+I$RXWdn3s_dxZX0HeUsgt+d zM;01(R-msJ=!K<03)C52^(a7ct{qDbVk)q56ib?zU>+Q(XzG}lBO@nEl$m(qfMwo> zTltltLH`cuoc>wWTf!NfbD9y7X0i}+)0km-R1OObJ!>7DL>CuNp_fv^Oh`mJI|t{y zw&=zQxN|0d`NAgzD<`E1$O_rsez`rR8T!*HzH4JRA9O4*4ahdODPsPI`$z9=j`4kE zsm*y(8=;J+{ShUP_+I0;laA|9_C%h4t9S);74y`j*MF+MJZmQ4aXCP z8zb+rK>@s-34U)HS8aky$>=97X6%n0kJNIYQQ#!5j45vWDQ-e$@*iXxemH4GaOax* zRJD&J7Q>=Z-v>Ct<7-JC?FogQ>Ex4+G2iQnsB{_4+GQdx0pR5_iC078>8pDV49L&w)6<#} z%RF3bgEPX5mH0+Gbbc<=@+$oi3nyoLjq%6*uB7XY308t@pKdyOXbY6)Z9yFczzL%@1N2NZvpU0EYBhe1^fP;)9^`xZDf;*{p*=2LYp z1(?`Cw`y7KNgn>FPIGp7nQ7Aj;FB{@ri*_96}X5Je?YAYh(1sR53)bc+%xs%@U<7A zhA%?L@E6N$9~`7e=Cljh`KG?oi_1Jzrt1px@iOln7_GC1HF(JnjhZ_>VG=iXA+-$e zvK*PJSpzT*m$m!l{trVR)v=8^vAahidi5uyr8yJBq2aX>P%~-*l#7lxx*azm`gP|s zyQwhm-q)CCovj|cfuq(7dAKs%vUK>O=sJinn=GlY}r?w;M_0t#y z8r9A$;Y3nlURi1gZh;o=|AZ32ZBT%FH*5WH3?KX_gl=)a4MmtQhM=8;r)nfi*mdZO zl?o*khPT&n7>9||w+|vyNN6bPp?6+O3tWW=2k6OGIRxOaw%V_yp?WX-_7S_WLtF&V zj|g!=?u`i+KB-vMeFS%|?z$dw zPoL7p0b@dR*|v0UPCt;Al$Y}J%5ea#AmFm}gO!S>(1|iGx37Klb}sIB@(cq*B}Q)P zx*3w*V+HB>g9zO#-VXw9YCHy<5W$ z3(~jmy!suV-KcJ;fr>>WhFXlg53|DU_mjt2Lil1nC$c&^#6Y1AUtBbcA>~0H9v^jL{Ip&R;s!AjEHwNq zz*6tX%q#~}w>Jo-^K8phi(@P)6kYfCr>6f6ffz=5U;=Tn2N)XzTycm2Yk*72z55F1 z_BpXXv^U#$hlWk8hx~#mW_PkI*GtfTp^0krqq+h($c%lPK#UfY^+%T)f17xd`!@eY z<5Mx}fk)jSmq&Z#pwiiXI~zz`(aHpr4kccXUDfU#z@}?OV1O)N4Jbn92UZUFWcYWO zV(#I<<&5kxuC5zgwq>brEZb!GnT#hUlH{~tu$4yHUTl}2xQ+P3?00CmBtqj;X&$)* zwY4|Bdps*rO9JBM^zH5WHnCBFwk{6M7(t)der}p)dJB)cv=2}vk!;MC@;3H046s($ zbePiaVInnT_d)uNAU!=PZSCW_3VJVp=h5y}5^$3nME_%`q#rs1Vzi8;WT4!Ta}BqL zm%jcRxv4_C6{c8M;GZB?&I&u%su!*)g%%B%7#<@g-l8-Curis^%yD*yFx0yl7?2YX z5>{CC635U;P#?h-DUe(R?SIzvH;@4jO8JXXCZ8cw|MOopTDbvE^9^J~Z0QLzDWN0d z%&Ki6c7P9!KB4;Fu}5QUBY$IO?a?}-&3g871#{LP%n4#JG3BH(a1?gmHD{>wNG4 z-ij`1&(3X)gGX(0`&nf#a&{IgOGq;q1zow^OiF}F%|1K`B?O;lFCr3yW~<}G&BjKS|i^Ji|rJ;P8bZ+zDY}cAj<52WhtiZcvBZ`wYz|dUTzu$sQC6*IGVlo%y znEe*)FHnyimG)Nx-Q1o!x^A{4K@8ol`@>&kQm@V*qYXiszDv{M(oA8|(fKQi83(SD z>~`cc2qTnCaL?t}PZEU^Gi<;Swy!A{!J{?TPKXnijC&sbX`w2F?|%F*i2DOp!MP;``A=oQ^E3=3 zn=tg+zI{oaAPWEq0Y?=lJy{(YiQ#H)Y4PF}ftwvR1XA`=dnNF>kWD(%T96P)NaHsQ zNT2qSZBtmK@%s8h=@wPBLYng{Ta z@eSS$H~I0MMH-|$Ubr2y=&c}#0&9R`FN-t)X$^i2iM}k@c7oX*4uVIrnW5G!9Bc~D zDkX~zpFDBg5yU=J(4a-5;36VQT?W<$>|P?$jc5RhJ+kJx(a=DQm(;)mh~g}O53OU; zd1NKOyd3%WZz~oKF_fGB<@NP2xHC;?xOeZ$l08Y@0O3C*S%->1k@5=ttO9s5BO=7y za<;z9!maUo(TV)m;G?W3ev?T6pB6IOto3VtTz>q%b^d3x7xAVfAoGj%OIk& z^OG?~5S?q85SCP_I^#HWPr9vpZ7A((<7tJ375bWeO9F3HR(!qm`T6g9YrdoGsfIuP z^a2fjxLX(sJcBj>c6BAS1kT{%*LbPZ&?KNl7j z-s7fMWy5DL9z~A`rRC(@yRqIsJS@h#zf0yerh>StGbZ4eh%J9OD+PNVYF>!>Pge?t zDaZieW~tskY3-j>doi`=eKwixF(rfu!*+LQ-KEKE%%ktkiM~QxNaeey>i-kT|HO7^ ze4q@>?E4Gdw-Vo1S`XI@#DI1CIVHjx+mKY)J9M)GSOI{8P;V5+2)AL2Yf)eK}$j0HcTRh=^mw!**q>dA5G5J7Ye$=2LX;UANXxHe%5jUnK$BB zZ(1fZ>XwGl*=n--!8gQLYWgqD;)slzY~YN=bd1nWk!HvgNaX%WOA z!Xb-k%a@Ur|1&2Rk?}yEG_=4{74V}l{vdg9_{>3}ex>!gHjLJ54Yo*2`Y4L}9!|CF z5+Y0>d_z3GXU|j4MFk18hJlS8QdY(#A4mBD^IaN1wY{9*aHOQ9&aMszhf9q`4-=-o zy0fM8Atcum!A?}0=*l*kCEe2DyQ96mc@w&NAF=$aZzrMMnuGgaD|agWkNuxEn;2gg z`F{?-Fm1%`cDIfDtkJo7sqzIMe+Lm0>eXM8is}gSz5{J{VwB1qt*K*@hj7%qcayK3*JSa7u-J= zUvw=lnt+2%=S6ODs#7)pR~=~ z`Z~|NX&~w@ra4T?h?Dp^(uVsv)GPs@hpx*7OP>b<3lT;Z(B> zcNT8$cTc>%MS#Tll#w6qz80CzYlVe4bM8qUqfJ=Ef$0^_ujVVSf|8O9(~_cqH^n=> zumYnaO7c%T*m^Sk<4k#3TT_${6DPkW$7}3re)Yj9O+BdWCa3Kt1Qu|ON5|bYf2P|K zx^0sC$gEGU?rxq9e(-m=ySh1>MWtfABlPWF)@R+xtO-wlneL)8QJaaFVm}8(dl!%S zO$wi5%YmB7p;^)Px1Y9zda7ie9Co%$^`F=qSCuWWPCnYO-qYQ{A7*W>;Br`GqsFZ`$kEY1VS?H8Re^ z5BjCn>FK9pe}=#pdE{r(yMKEDR9T_iG%vu(yEA9%AZ?RrD_>MMcGO-+Nn=Wwp-AIG>5|@y(@CfU3RUvO3D9 zQz8k>?ENwp;NR3+^R>P=&wti9B*UPKlg+69hsCa$JlVhu`=5NCvl3n&JmIn>9!Sju zm7k@+ZdfN8^nr-_dDhBZeN4!16V{jn_yW3!w6|QfmXP|@t5@Z~v_VJ-0)>X2UgXe_ zhE^EsBT&7u2>cIK5rUp<`%{tl4D1aAXxM8$2vhjDrO*dpr8NV+O;clLQkF})fs0nR zu&DHJ8Kuz-qV8zsuxuEx+I1uV5bO%jY{r70xg>-?UkajzprN5L(Zg|D#_+kg2njZp zBzfq-VXv~7Uq8#q%hNx6_=7;{a2A~Vs^Tn#6!KOUglcJNX`=&=MFAjN-b(92{+8Oe zj3O$2s>rkgz}uYGh(va6dVC7(f*HXGwLJ{#-c1R?l``)Jf{P$paE4OBxyS<4R&+w+ zXWR+_br$1$@4wwkTI2zO1o$F5>o@{Z27O=#M^OJJ+rk1rL_!i2Z2}Xb8D(5r^3|C5 zyY#j=LjMPE(1;aMLeeIx?5e;g9F+qT`AOYFH2Fziv`@G7mqP`)xyuI*U$3<4XCPR` zMI}VJh!9i!5%|5ybpQN&!OpH?*Cb#7M(~sg1mn{jnqxH(jIY60e;z8F2p_`08xQ^4 zFhsUYBG``LKLP!e4lwn8YtjfK3>*iUq!F%R8W*wIy}{JoF=nIn0Gfp>kj?P-DWyRJ zhR)y*CR`LpC9>`L>+{rHPz@-^(g4WMVN8sB^(r_wH+Oh_@ynMl+Y#LW@mP10#m765 zf|K9BgA$P2L%qk3agdRbjfiQ%pPI7>*h_wP^m~}nfuL=mG{}2E#PzOT<8Jyjs2sAK zI8s9pVgq{nzcg2WrCD6OJK;30&CA14hc@5$oSUUs>PI#bIh{`Pv{+5h*_dbi@&53s z{(k_(#Su`*w=|ON6AR={aEAaiyk3(P1AC^k69P7AJpSI(BV}Ph58xV-KX(9&W1|B1_?xgtS9g%U|gI*n~)=*)mv6(*80I|Cg4K{gfX=es=aWXmG z49%3W!hE6u1+jHGw{4%QqC-2N&e{NDU-SP}Dj&2sO#$U*pq^S2|tb3v6 z8AFz1wtl;)%jPW7c;_%TYtzlA1O|706t3!nw#QXR9&*v?w==u*ZM6JlzozfriGOv? zwCAkvth+1CtZVV1rSm7(BKH3nUgu#@WavDs`swZiFauk`f`!3XKW!mqcqqbMuyf*= zX7shLj`Z?$uNzQ1eLi$zVpM{2ZiEK4jt&;>ukN$;6_~dKA~7km8`j|91cPm`;#W+QMIMSE~0mTTW-X0-O5Q-7P`+u*v!P2ve~G#*@$W)jSzm6n|UXW((#o zY0h{6up4>)E?|%P-NL1DA_6~wka`EPaoySY`VK541;k#}x<52HIH>UCNs<@0k>{Ly zWPH4!Q-t7x?WaXS$NEkkk@$~-aszIJYn&Dw+hpRY#n>s6E8nw* zM#$|kEi(t`#imx!A*|W~SLcOAyW2CFaUV^l!U2~hZ59YI=mjc^hTMLi z3OETQ4^*o(-R|6+?K-5GAdQDbjE$nz{qmtP{~kzuHW$Jm8IO(yS@_!Yas0YBnPPt+ z0=@jAacg_;J)0K%J0V56#|q|S>Yv1eSMCIF3y>3ig(!Y{W_OL(a#o0UbfGec@DAzu z)*a#bU$1lRT#+SSF(B4yE;7P+~+4N$r9^MSBFK_6nDXF*qayL}=jd0TrEm>geIR z7WpiRSy?f_ZacLPcwwJxC-$S5MJI**<6}8Jz4R~}ix*f+OI_b2WVdwkBkiayywL8I zH20Uf(+kjCcMU3(Ir6|3bEKw{)8iJdtZ-i@HQOM!4Y6Vmm+7)o*V)kCoab`nY5JaS zIxU$gKN5lXAYpRIxU`ozB8WY{^~R8R(@d+jPO8&@;Kob7ZF2sRO5gQVannQYLRDB^ ze)B@zIdRuUlPLOPG;a6)$kw;d&3#crZzr4Km6zcbyW_^7>Mib3q^{* zw&OGLbs!l>BeK4C5e^;L7YE?xTFQ@bE$8ZtREH%F?57%o`bMkRT&Nstd=v3hxV+*EVnMO_ZgE@Q5mPSA0T7**I`6SQ+{QZ8<>>6Ui11fld7n1e0w zWbyRiK-${+Lzor9V*M*9PqDS_-8eW7sxyj^x#*=-In_k|S&n~b?RS@2J>IPE=? zXrOs+JUWoGHx(cB7`olgABQ?CRrU=K{Wv6K zM!p*MOR6m`rAd(5QlqhpDcq7sTHZTI&k>OFVDQu8 zKA}C*ZUH{gl~W0_7oz9IBw)2SR+o9ny6q9fPTq)7At9w$sHw)Id$SE~n0jyD!lNUa zTi!BWRvcv8eclB=efSh#X2RDk(UVh9j{011LpW>xnyN!q7fW%xH>BAxtSY9>AS~kI z3{lHkv^8oBBF|h-rcWVDxaX6+bNewZeLTvB=qWp`WpAO36nCwn= zLZIrE^#@cfD98k#h#H8shWBlwoY+eg_Ls&!X+=gw@eBt|K7SaG9+7xGRpZn?*-e4M z%u(L)L*kng22-Cuir81#mHh~|L++;})Hi3R$7kp5rg;i!%JFCl`e$V?FfdQ}WIU{w z#Ue2v|a=$nLiqBH51hu0B`1IK+@R-9}oOCP>b0wtSTp68e zYfoo0QMjG&=V(YOaGr(uJ8=UUs-pG$D=D=kdB}etNXa?W)&0D2Is|Iim02``QiB@s zL1+;(c)cDU*o_T&Dr-wf%|g&{u~{tScfsEkLuP4#+(=&o%23%#>ySYW3!1h~lM4|s zamFc1y8ur&id@v@!zvlysY;vpUomDZA2+)xey#+Wpx$2Wos@k>-VIzIfbC}GihuG; znQQ|a*{jKgX&Y)tA9kzp(KLFK&E2!jPJCq+yIU~~CmmL3OvXOBV|}SY3f%dYOF{t< z)j59lPd>AiKZ=i-xV{OY@wK7dy&br7SY$>W1>8ZpVc{?NJIbXc{GJPVFn?G%JNX8r zP^!_C9&U!rz zI@bMBe93$tUFqy*Fm>hogwy2A#~CO9@oTYqO=sky%s|l`p3CWK`}olU82y=`W0Sdu zr|Cl}|9|%ZPdHJLyr4!HoB!&4Fs3Hi7NQf57=Su?M6-s^UjapogL!&p-Tz8pOADzp#2hPs&uM8jhG=r|RgLM1s1* zV!bgY=+IP?$0%mQ*khr0O#Pjxy4?Bv8Xw@)f z#k_4}f>hv8KvvbTOj&+E83spvT`)fosTMAq(G{vJC}|b`1vPiDHYD@1luyYdTe`7?pU+ zkBN6!h^Tk7CMg8_cCF``ALI3Rj1bw*`~6mqp?VV-L2VeJkTVrImhDN$T=uT@++rs2 zrxUt-9BPyCxA~=+QOWB6%sXdafyNZI8TZ@)5xGdfELny&6Gr;=TWNw)HQkfK_bR?i z&(A<}2n3AGFD%43lFrMM#19JKOXMenb5{|#3UbR5w6j)d@RRabpaKhx&*)7oX4Ms+7~rwQp$Z#gIQTy{x4;NOkl`;87VaXPcTW+x|LN$b zRgN|1-p8$@3D`y|M zv*uY1s(@`o54@WA-UIz`gbxeYp3j3rMuIt=UU$3*p0?a={|HQl)F7Q-d2d zy?~RNeuO#K{PG2bHNq0VzdQo9ef3^MR~KA%<I`0qXXQ5qY&#R zXkydCh$stV6g?&fX4Mvb0_Ta9p#E>haS@f2t_8Us;$aa~AaAwk*DU$zi##VHr>{rN zM6dI&l@Grkjql$d8sZ=N6)Qmm zF>*-(6nJPvILp?5J9Ue}q^nbYjOeK%oA(=hyXPV`wt&Z2Tt;|YTZX>@Z^}b@IXSuS zy}iIvotT)Yt^eg*a4y-^)fLz)U%e_4>1sarmX{CDrXe7(=ne@9F-FiUM(LRa<>h#Q zFqHB2ZMe?*?7XVHSMP|>rU1nUH1j39o*!JY0Y<+?x4y&!OVB-p3WSP=R`2Z{a*U0I zRStYF_P~Gu!pGT3ZL`BEQE%m-cr>DH4>ELZlOyi;Rb{<`OC7q8y8CsD zGc+RNn_KC_|G;OWc%Y|^XFA&PA5ZN=%-1O?K?{7>*VjucD=YlOzhKVfDGw2H)-!}M zyVx!YS_&cBbJV+jjKZn3?w2+)%EH8j)&$)8^Qj$90nKXotsJ(fe5?7+Of}*-=0}b+&Ik+?w(kP8oI%Osf(r97WcRZm<;xkZ;=V;gX0WMia?pyjpj^Xk@p!|l?cCg)<)OE>hMxE7oaaup>Fvd& zDfXKV*Qxnz&-oV)dp~OjH%CK4PakYmBGm7R&;kuoB_$=Ee7MiTrOCKR{yK`6u}kND zM3$#*3OYu3c?n^4AH&-rD+{+bf-La9EaB0$l=bmRnswc3+rPAE$rT_v%d9v*WkY11 zL!=;48X_ec2jIQsy)42vu14gU~Z_;SQ-BCOPek4u4QJ#{Pg%<_EX6 z1llV^1uUI6xHh#NxCc9tIvzE`F_+haUEy~QJ=(>$G;ZGCh2Rt3j|>2<$xK|pF80$I zXOkiVfzWc$)0T%9Vmqk>kJ|>sUTlbYh~FGVYm!P72{c_MTOnLVp;!uUcu=Nla21sS z`JVMH>|}d71T^$X=gFFVf$@Rv5(qu~45GLP{LlG}AYpHTS~vr}!@@F*rxqfLJI8M@ zNksCz@PiF1+NVHrs%5nsnj3h8M*}z_Qak%SoZIVQ^<967mo+5>RX;^<^3dU(p) zJZYKR!GIfA4arZO&~hhee98B83y7;A^!jLYs_I2DCNYl97!aqoQ1&FkIT`qfU(piX z6+&;Q-H1KPT)0}ZqpN_a*0lQ)5|O2cEC-=ea|cC9qYwVN(!%iFtzR!1I)uih*aae^ zi>d47c&(8g?JX=^bla(AKI#uum~bKTR&?&`yTzlRuv;kzJO(AC6d_+PMoaYh)`@Fu z{{lCX4ESK=8G@ej{TX=-z&W5e;FPYWKaGo!4=iH>`vw$I;)$|zdCZjdAwNMhyk6#O zO&dB#?5nG}sW8zX7U~j%>L1djLo2PCus%9vp*T%g-co9pa+CScN)R$??Cfm5>QZaU zv#K0gXhQevt$As;r+CCrvHKs@NN7tZ`KI>4+}NbH1Zj^{gXpdDVN%`7jk?Rj&n-7M zJ$t(5!%5eA=L6N@?l*76VB2q073Yqd6J#HYYMAV1)S}?F;AD4D9pi+^$?(NC&-TpM zI2aoF>{Qo`UaJROB$JK&Zj>w|CMI76a~1TR>KnQS2VZLcB3N|c)bnR4WL11>N-|q* z+Nc3D^Yg|iA3FR+(tIYTt@vJD;(82fVatYl9I@`qG0|Y{ckL2W`6wR!B*i{`CGZ#M zR#m+}Lb`|Nm`)PX7Ru0#yd?~!W_io)Ml}7z-t3bqj8w=uZ%)f|!LPPj#wD`LcfOl{ zViY!5Z<&9JhK7elReit4eZlY45A-6)Yju|#X6E3)?avgfon%fLi9@MVx)CLD(c zg>hwCD{zt1*~Vf#^jyId*- zGhz>r3frOyLFy(Tp{-nK3FlxQ7P;ASx$!J*=zOV|(icm0{xJZ38aA4qvBaeJQ25Gb{{M;d14Q~Ya2(~UlYz$V~WCh!5?!nKDp-FYnQghPx)NF-c0!IHs0mRo6T zD)JvTEh9$IqNJ2S>tDVF5oFK{&&_`P3%IYHN(gqNplotw3EWryFCrabAg0WE^o%P0 zc+kMk&p(p>ott~MzTcT2a36WYsrept-w+kjp#0zX@bO|VYJKSg_%YE^5H#GV3D3`) zzx#kIZZSVMuH#r&cTFe{UcTF2Ps*Ez_VD3Ap4Fa>^JKpt{2RJLItGBy&vFQ2wMh$_ zRWlzgJdayOvfLeAR4ot|dtm}wLi@=&I3yO+Z(8Zhd`0CsYc6_Y?BZE6t>y<@dgQ|l zA$ITbNA#|j+yU(c)PQH0LuA>5-Rb}K0$4u>kgLXnjQ7^Y{2a`0#^c5 z_#WJJVTR|hkLfZ^4W_ohq2`p%7gW#7@&w*g#ECJs>P#1k?rqF%+N~-$cNF{LQ!m&J zSd$~Q6fN>Gl=c#V>|iqpbyj z)nG4DaU`}yE3IwSD!|knQaR8aj;57Pk|qquK~?2zObiSS^~LC7Cck^-d`a+f%ROAT z-C7i#dR2VbIbV8aw_Gss($Ye#vF6uMDt`kv;^xoIp{}LOpAJNK<1g`boa%{`X5#W}k`R!QCP2vf?sg9&5b`k+LAN2%eCa2<)KE`fw3Dwb*= z@~6XTU;~ouUG`VJ*J)*Wk4D_+b7kF&kq4jw(R%-b((DJ1LRX_&@|EbJk=WExx_L_U zaglN@)0H+UDj(mgq!qPv(@TGss)nYN^}>=iwJwx6xjvCX2z;bw)uU1(F|g^St6q%C zgTqm!));kY4l6AV#ym&yRAVRln)=?Tk&SYZyU|CRU-X zF9ka3$C(tD-s%rgzw@Huv=#InoymFv-`yGfK0Gt0T}gI{E~F1O^DQm{cIcur2mwU) z)tvEK7hk47Y6tQElHLSE+OPy3 z((KL<${XcCi>Q7)O3ISis_&zH2fx@OO6irq=dWaXy$pO9k?^waA(F?h7f!-0sWN(A zFOBZ;9xHV*LAS-+C>(s8I(s7uv>U?Dlq~zXJF2lhs27#>6D&z3laD>jW2edxyx`L5 zdaIq2el^Dvs?Zdum1E@m!c;`c`Gt+m;*}RIFToI6?%83QKn6%RYfSM`R)4$egB~%d zx)p&5>68ShdPe3wgcsnT#(^y1A5^KRjHwS&F$kXOB|qbkizJv-bt?Tx`7AmhKu}PS z(fN52rCzUpWUg$&NQW<;p!&;5E+s;C{Spi}GW92Z%7#6^dZ<&y{I1SitYQn42+}0) z?K7I@%#ui%fsFHqRiYlZBK%puQAe=CETdA3V=SEZtxS{ec{#{XeSsPNp-xC762XFG z66$s8VCN6nDHQ^9HFgPL>w7=1f)fMgRIvwR6fGTcxRGlVV}FKXPK;L9drX4QQFDo} ziy2j+cky4#3JpELH~wr zj3Ap*;sFu??Y|yDZnTn1bX0j?lo0PIG6o$HQLj3yQ5U|0C;aF%7Xa@TJV?k7L$81( zg!}-OaEBpVVIv}%qn9ik>Q*W_4Yo#HN;#iDe}2f|be(9i2-oq3?+%HuK={;@EOWtGLC3pb!Cm5ENjW`SH5;#AAWa%}0U;5glbduIVn0QS;aVMiO zX#|$E#aF$Oc-|wB@DxfQ3`_H!o(z9X|Mk23TXjkznUh;{UO8u zJ-EmO|4N(QzrOXKh5lP=5lpg3QVINf*u&#a@PPIK+$|!SFz!3D zkN*7kUzd;&c!z_!jt=o%T3{H@L(%#8_!=!jm5qj{ru5P?Gut~l9t>mlEE#EN;H_#W zp_;opJ1agSE#6O9Qbc@+)&-f&1eQ=Z$-~2g0fFf9Rn^ui#8C>5?Ike>l&IufIojBK zydFtCLI=JWr4}`=P8{r^V7kb>Epa^?tN=YDgA2Of!{?mJieXkeO249d z3>KRFG(owJdnNqzv>Ork4E(kxII0|e5~r}~>ino~>X{0SNdK10=3>O@D=Nw~s}k;8bBRZ63CfiFs<9nyJ|S+U z;~^FMg;8_8g5C`!KC5!NA49Pg12_%yG2IBwVgvC|IcRql z#(hSxyB`O{Ww$=aRoHv52~A}dH{^tP76g>N5_0LHvAbysUFR}cJ}Wih?tKyb4bry6 zYlbe5Qxd)w`>OCD<%cj!t#YV*s_0qvN*aK$kO_}hfE7JEW zQ%mc68E-Y`f@dy@b&!&oQFeDFr@w?ZWK9wOHR52HN#nq(!s$-pNJIo?15EBN83W8O z;^4UsLtZhQHXKXdW}i0tebhLu4rF(CcVoWs+ZWr&TxK1JqobqpYZi^Xz(?xK!n5Bz zAwWcwFOZKVsD>p_MAx|{>8fTDtaktSp}aLZNXkJi3ls9)rq1x33#Fh&(R^DNOMrJX zzlLpS$9DzSmgzI}HmNr!@Ox---PoG-9*+0-@ae;Zt#xy$f7>5;ZGQw4*Np~qVswya$s#S6h8UKHbL?Zakg3vu) zrz{JBi0}m&>8rsHtl~c5BygvbB*RIlQNypKO7W>5#D&xuj68Ug*Wm)wdOfrn-TNLi zEzVa;W{NU^D#CvQ@`iQt@yleeS7|QGWsA7KKBGk}ls8)BOEa-}R~KpV1O&3|2@l%Z zH{H{mZ_H@i@oxJ|k_(N+Xnv$sMASiN>TCE!syOq1+{<>xi-?;K+v`D6h6$}x1*+^` zCv?-1+)TmH+L@?j6ZW_tpYIDj+HJUZx+C_qv6`aHZ2pTYM%~dHZ?3iwtKBT2no8!E zzkcMillh-Fk-fTKp>y6-Z-9jv8{{|5=!6ML{y?MpMZKcr`H;EThuE?R{E(v7n_IM4 zL9UGT!JwzGao*MH?wbD$LW;Ed$Y6iL1$+MgM1YNlP}29sjJV5Z%hI)ubFYivpLx+M zM|R5i+F_`IZV9IqFGfE+1JlV3Pf(2@bf%{V6V$WoEQdtmCB>-1@FwM!zbGQ}Mjr<; zMZw`$oot^4g4Yfiy7$;rLKVbw>B|bjF3wJI)3kjfCy`89mhxYOw54;?tF=s39g2Jn z06p(iq}d4uj9{T{Xe+X$ss}E65%$x|p%E_#H`zSZqTY{TKsdfSDt}S`ZF#;opDwZ$ z`X!z8NR!bjmM$2R@NcW@*#qP%*muR^&I&LAo_Qc=kV34jtv4wZCm(RizAFVO=n3WT z{;w}f87q{2X(z#EoSmID%Ko5@1Wn9muq7#?#4}(S$=u-z^?0 zfjTfP`d|!{iHV6Hl&TioKtpRE9VMirqr*HP=YO78LWgJu!K|KBxHwGDGWGLk0XQH) zIS`Lf$XH9Bg!);IZYkPIyZ=v@9qYmjk9ak!a$A49)f#T?w)PyE+VOQO<*DVv)^Q$R zGXahqxnd~K`8P|9cweNC(E7!ZCb!_lksf_$WG0hK_5*ju7pS=B#thKsA_TSS8Lyw_ z(#QA;?(sX$dlkK!DBKP;^?x3Iw|XV`l}d&Xe!2j*vnvX@;DmdMe%#;Y&^!kMy`~EK zFd)$DD!^Momm!FsI0DhCGeolshKDcHaks)7@0WgZ2YL<$UPfkc!w=L|1)~R1Kj?Q~ z%d!uTC7;sMMplFU;^J(Jk9bL{`Mr8W)olTW-YRd{=h!zjff$h3t@~|5H`->F*9+_W zhLN#;XtxT94=36wa*Cc%rs^5ZHqv}RmSP_Fw}^c(`7x6Ct&8wfGY74b*$WAYJn1l^ zh^~KU#{Wu@;qw65!kjjyc@RV10FXS2AR6N7h5`Z#UZ@IOE88$F(eWrdb-@%RlI-97 z?Q1Ja0+ft$ij4RKDFi7N-m*B-RmnsN09>HUpT(n3Y8-=ZDU~@L76CT!LQ?Ggx%*_; zK5>6;&i~(fdD+12d*f~;%mwaVj3QWqWG>@E*5Jjd5`f`I8#W$Gt#=})Q2#bD69RKU zP}{(a=>#!Nk(LVcogB=}`L~-9=_6I-2WC~MIG_MO9$b<1|D)?GpsHHCt_49#IW!1J zcOxh%A>E>Mmxv%O-6h@9Al)G#-Q6Ofgp!94q`TvPw%+%>_rCx59A_|&3WvS-Q)|sN z=Un4N{()_6BD{Qj_c3t(=XXm}go97X!KNsV1)+vF0`w_Sr&&LL{^a1~lzA&|#|zqQ zAi2cRRLDIbCH=;!&mlfc9#*o-Lbu{aXD#WXaCN=i#(Rg70`g!%ck9_*7m z40guC#q|Sme{RIo_wTQJqf|b)vw3x)prDBLu(oPzXizA95G=2(e2(ZZ#K?sP18oj^ zx|x*>lBV%!Lw-=<^M?to#judz;QL-Bz%ulL8=Q%bQ6oDLN9tvzEJ5=c&N>3;2czs+ z%0l{l8Y$WvGdqLo?6yxvV9{Twl?N(l-q?LFXlOy0TyXhefuM;;FA3m=M-6g`hqKC4S}AZt5gMU4>gcWPZ?c*1_{BGp zyOMXeFa?=Q}rkjZbZoDiI7kZq(Tir?J3RBF47saW3_>Qpf|tL zI4P|V)W$-K-K67nIuX=~caTzyP3k53#2qE8)D z;}ajE>KgBaW(|2b8&wKia@MWZz@8i>?QmQDk)`alN9j#nI8xd6pr9(5%MCg$moK z(dm{FJ6YI`-!}V^fu+}a$5+f~!N9tV5JL=~pcv^>biJ)@2z`W(fUloA%cD{D(b!^_Yd9W{=(DZ0hF_Q}_d;{U>jW^45cvXFzDc@8zlc zpXI4b{8$kz&m5$;;d>OFUw&?ET<}-8tYzutfJgCDP8f8I;jxfnse-y4q$(wdUkc8MKoiBMOQ_Zk5ATRE=tW{ z#&l)pOsSTxkzJkY{HdaCS7hkSiOke7DRUt1vpT_%+_$eDZ7iU{=$|e3_t!f#BIqkRy1KA{NU@Rw|NAb^AT{qA%Uf zSgxFny}cN)VTTgft*0`c($c8IX##T7?_m7S=a&Y>!^CHBBR2B#CGNz{)WpW4f4zddy(A)aLRN24|`awT!n>73+> z=sa1zWhr34ifG)6sHZ0q$K;x+P_tV+`EhrYN&PhNe5tLVnPbo}zHK}vJC|ChY?dT; zTHasG3MD9t()Y#d{O6}v-?TK6wBexy*`v8yuR8CA{7&fpo+3WrsB9OZ_@-QWsKVhNBVE#PBM$ipF;mhWcg^Nx+r53DfY;2`vlm1X!>y(w{ z-{nwe1NRNVR<1TzRVh-DK~5qVMDchIf`#{Txps0k24d<}Z=Nrg76N2qSm0K|%mrnn z7~5sV9{M{4!61rx!?iIlsgMpo?$b-YjQkI=i#TC(ymuPex7+vjtZXXnp0CiZRn0Gv z00&gx-Yu?E&o6gX7&Y!o^^l6sKf1kp^8Khl^V|IwKRm*VBg>2yC;E9guNlkk+|&ob)g@!9LP`HJuPf<8`C14D(=gm;wpjgw~@sKU^Nn3V#m(utoMp zTN*za?A&tjz1){60o-E6lJ;oMo2-m2FEtkH2_0t>PVtO7?s1YvkLz}S)>s?&CV_Rl z;?j_bwDDVM#c!XhVgCbM1ovPd4;C}gS92Q{a2J|s;4Vs9Zu5G9qs#*hq(;(>lCSqD z@`(0P&Qi+ayUZy)=KxpbJqev{HI1p&$-(v1|uk zIa1Kk5x7k@(l7~DnW-Qd2l5p+wNM`~o`1$lz09vIQqQi$&5I~5MtpukTuJgl$Olb_ z^;aX~3nq5a43c3h#E&1pWE)-AP2tGU?w}AVF9BbqH)bjf$wQVAC|uhVA06~mb7wmu z%ruG+THbIm=2|5w{cA1myoZJ)Bqixr)&eFR7Bj)cX zMA>EFz+wJb)3Mq8iqq+r%X>z{6cJ}$XDp2>REAvo&|uL7R5ycR~*uTBd)aAIc& z4UEN&$W$AxBf|fmA!mVxd{EYg6xzCOJ#$`Q1Az+#!WAks;Kcxd0l`T&5|WaV5-ms2 z1r0MZGcPZ11_1t}h%gsrW&F|lKh@Mof-LA+u^OwO=Oqyt8Ci%Lgl!ft^8htAa{h#S z9>(wxg1r`28)bN5y|v3;>?V9Pq+hRvU`$Vt7n+PDi1w_$!pW#k$R=_ zZhI2fk~L%(@;w&B?gt^ple=3CDYaot?3`rhCEF-yf>*qYoSv7vC}9?l=(hZfF#fq) z(-;^ae);f3tm`z`Mx$K1nv~m!$X61R$AxXpjaQ zPuw7Cp7!+&ZEz3;0W5~y#i)`}1_>FPo}1?`W47%ejmx54|Im;;yOUg!gfo_qZ~y85 zu7>7M@zIWnDBfu*LGChjRIv+npYZDM+LK=nR@t%l5AW~uYtgt`{uR>i>qXyn)D;T# zxHIo<(UVX&tHoD`);Jc)!vnfwfsHde}D`?nnZYl z4uC|i|C#X>Eb68>WH`o59b%V6u%3>zrQahD1+LJ`XGq?=EdJrax}6pH!UP;tDHHIKwlXEhGT=yN*rxP z_x*OUh=@?>>FI9|y;FdG;XhtPw*gxA@SW-c(eJ|=0w(Klp|>K!Pyh!Qp+$&X^ebM9 zcyoZ3XNhHY^)5*$_ockuQWTBn@!{Smp_e!)Lca2tV*1zTn!E(VP`STuzYU&Yk0iL< zic-{GF|gHcLYa$yKtTJCALnTpFq|WPI=VX`Mvj`_+S;l%1}G<$A00=eI~!GBEWJ|I zQAtV3`)+Tfi~e}l6uk{exG1~MPW7&Gt1)UQn$)L<3gutj2+au`BxQPIUX#5T3MJCP zXo^@P9A!xVNLuo_wAwOHJ^eKqH_1Q8n8O@gU6`v2FE1Ew7I1Z91ll%=2_Rbe3<@&4 z6h^+Nm;iWh_O*EWA1=V|{=V;};T&kewmg`iq^Tx7ceF97`XLDSuM$ER`}}O9lkH*` zTF|_T55)=zx0yGWAY?a%8YD0(pRWKaUA5VAU&zWsxgyn1 zPcW>0x7>jmzO>_~+gaP`S%O>RS#$pZbFI4AFz3u=X=Y`fhN*Oq<2||9}yanuFY%gdzY7&kgFhI7$c#eWLwX-Fldx#;823J z<(*sto9K)P(EPWLkK+Sq|2dVIipqPimH9Ol$EIIA1F*v z-Q7DoP*@K;btj)FX$=dN(g6h$vSw!@dw4qLkk6!t^(E&5&s0ReU#I?M=z2hLA!X=p zxyPE?H+E%p0c2)-=HZsOoNsoI0IqO8W34WGckjUWK6%d%3Cq3SJZ9c)c;u%G zVc)b;zM9SB&Z_V9wj0AQzI{}^b2A7&dcVkBo*#C)aOe2XEui8EIbixaWhYwES^ zJyv9qAE!Oh1VZcff_AaypHq4o!#Uwt*ha&x2Mdm$S$%e~T6MZUp{m@p5ZvJ*^+zl1 zEi73+m$z;ESBMw&5^O1bm2zK#0h~ZYiJ&D&1kbXO2J=W#LBv1s{o>O>q~+APH5M`x z1EUbi*gEGuDqx~p)D?h;jLxu@ELag_5R>sGY3ecs&4{;2WZBd z@0Bt=h0^A(=O){{+_-(r~dbc7pbB;pG(DnHd?_omkv>JooNW0yh4*19soSm+9{>ks^(-db2#_*k)U_(285?+oKUf-Wmrn3^-z z2P&!{Ia(f`TBJqa<=5iDfNGSj{!vdW6yFLHM0LX_N4l=D;5<7Cd_GO)pn%jGWj37(73Y ztCv7#41w8wt6X(=r+kzyRNKq0cx)$_Z|Matb%kDBV>9S@AV2=Nj+$_s`+m0^#DS^8 zQz*G1(~BvZ^vaE{LRHqPq+Eol?H&40Y&(r9?0#~Kj+m3|P8Q$BK26avPmuU(EZ|T@KeJqRo~%l-}OP_Wl9J-0j4F zhr<0h8Qr&eP=7xu7&LXiXxY#}C%hln_QXOs$(}z<8L4|5qb`=e?2pnfH?$~9N3-a6 z**68~v4CT4l>vxQvfh1}#XDpo-NAiv%n7a@T>W2VFfj;wJuW}vEqJ}S_rzmd&%l5p zHTAjE6f0Y#xHE%p2d2KK*IjrtJJwqV2GRjm<&MsRkC6*Q0a6S)BTVg6wKsG3A8B3s zl&4Ua@59Jb`A@KzSF4{k&S`T_RaaIT6zF|du2&z+X~Iu1S`5-6;`#>MU%u!!b;M}e ziY6}QM~x6oCFuIj6S#ZZ z!e4Pms^>D)x5jq6De+HC@z+-hRnQd@*(Vk1&e@`qXv0g$-Y6%n(cx@LU37j7|0)MSoF@8dY;Yv+B zZ7*V8_|YsF|HUd;M}m+6ZuotN5Sx_PPdK4U!A4@H?m@`<=NnwIVK)?my%GW~xMAn9 zg)<&eaHn=-u`4M3$_|}c$Y>urGUodS60++R@(1UB^yDqm;e1r4jrH><9}Ygl*ai-M zt<14;8pq(vJMXwjOSb4Z;qY&zU(I+M(VAL>ZsINS4vER9|5ZP{L|QXvJow1@3V?o5 zwtsjIpTx-p70l`0R=>BMNTdee&cOm@>A%L;j0F1l32pzUiVEeU<=&1CM4-CaJv>y|>0T7=3hpW)jCq=_>5oz&U+2LiuZb`1 zekXhvq(bPWu5WsL?&*$Z*i^BH9Lgb68N^JZjx7p^86cXAqE{jV#BXghG|*8P6%$j<1=LEIc$GlG6^no%Z#`JTe@4)5 zbvPWfBh7X{+g~@$P#OT6lV2cqB2ROnZ=38!_{+A9`1mfEEleJzTREsuI1)9v^E7dV;A z=I^$HnpZ|1!2@vHvI5VF^txF^u$ zL|d?C_A4lmYbSH^Bf0U{v(ISzIi71X^shmX;! zr+z`Ob9BsP&rcz{QCh?269SIsK6YnSX4Ukh4_GKfOOCMl54R#$ApG>duQnny0GSb~ z2N+XKUZLy&PH zIJAn)%;H#s*(x7>Y5HKeAeJG z8-qH-f1VGA>F-n#B^%745NY7Y)X~{V**ZNv-TCdC3a1jA(kF&R3#D~g5=TeJ0rm8j z|G$RPGMO=~tX!c{H-MXdTzZ$pg3BXeB`v}0ia4rmYTwu><^kK5T&9Yx_9ILXgJENz zAp*aINedku6StFA9;gP{ye^tS<96GI@ z55P_lB|OyVu#u;O`5Fni|KRY@|5OSvIGMrblE1>Np@4 z*(CA*T{|HR<-rIH5Qf-79G=b!w5DsZR>BJh4Q zAl+zF{{7s*48DDW0R8`;$|V&Y5F@wu_M&8FW}>2_KLvuxX8W=0b1=+cAHU{$Pl5IDF&LR-#FA44iqYs ziIosjQBh&tzaI>Otp$*d4S*99`SU%|ee2nV0N~q|4b-YyaM7VDtNE7jrQS$pxnLl} z1RM-92IVaAc%amF)QIFCKI3?$6{-WP~wiy3fLshLKD`e8fbpXTa?U@E2Ma! zV>cyX|B&ovhfm6eU;9y6r_H(+D_yZ4+LX zyBr+PHMNFL8z{ASSVI)cruP^)p2tl$l&#Dmq#Q&Sz?xK`(C!uYH0 z(w;}?ABb&Tc87@17=G5l>UGPp;@ex(x9wN+RxE-Y{<)+3L+HZo-%FYd!dzal z^5TFUG)xq#!24><(Du`Gt+&t4>IP#N?L}#PsMEqxeu@+d>e;EhwomI-E32!!1270j za^(o|Shcu)b}4uWY*%TX285KSv69#=_aRtVSR^KP)wlGo0nW~?M1s`F*7zz3>jXRN ztr>uyE;iQZm+$ueT*q?XFWz9?TEVO_5H67VaQm|;?6()wMS@w^lf z-x-I4k{;s@^nINir6dDTCM(tgq^IDffODrq*w#jD;Q z3aU|=NrRF)f{W<(2c&@11yNTb{jw^oGzBhO;14C``EzuXG&K^lrrXjqg!1uS9T-DF zifa7%Tn_Xa6a8c)n~3U8irhM#sZl3-^ zHBSqfPGiF41U_e;-bE{?rTddUqF0xTKC!Q^Y&VAJr^-8d`u8-nS5%%#D}LQPg%`hb zCO*DhMuh1`jnU5}n; z`D=&H#V4oPtqKeb3_+j-jJ12n=y)O^XEA*F zGB+<~GyrAgdEpEW>C^~5YYLu=g$q=C@_fQ%G{Fa7F?#XIYqxLph5Y3)L(OJO z1at2NbFu%Ho0UOd-R&QK`#CzS{h1EY#LN{gs#5s=nlV9$^NJ-O{ZV8nGdauI8)=l` zl!Zd{3c?}b0v|g)++gE902}78{_cfxG8XuU>X*;IOE`#*EnuYI;wtAHRWs~-XJO%| zm5tRIeUYl2%2F)%+YZr};~6fCH^a>wc*SSTWct_g7T>YSb8K93NU!2|DYjFOOP!SC z?<@Ic!8HtDBNm6wqF5O&ta$*KOrepnUMq$(^14bA5pFoLb_M8|89LH34LjsM9pmZ+ z4yYJjc{6gfk3KwpQp+%i2dAxI#=!`nvMs+Bu4&9yOwMIqPU%dqn*RO6A_%(FT9kE&K!X{`5j4pSR6CXZioA4oh=d||e zo)e%&6mluiFbf71XEw#f!zYqN8PvM8rjN-rofLJ@q%Iy2^_-ai-gjz5$Dd`tJiHe|C5XwEiS7U;0=ccQ_ ziBLI$cG5Ao`B6{;@nGwRACQ=#=&MZt2y9r%mIt*<@M$=zfX}wzlP9BqoIxuHSWAWM zj%SN4j$6qg@DZRbUU?_8zdWetkH$!SX2d!Qq`3GxT2@q4I$cV3e!+m-6OUy*$MDc^ z*Jgt2R<+P6hTN;_XMr1>ob0x%o{6seo`A((vC+ng&-=ZdEg^(`m;3U*;^uZZLi&yz z!1dly=>(07w0^*cdz^8G5L8OOm1jX$(To^B(9V_?wYW>>CW8E0Pw*2TB&Qp(&|wYM zrU>4H>J1)joWI9mF-tY_^MIuaM6?uH1|UO~Wgd;7bTa^{nesaOKf!E&nF+s)-fK-) zT$~j4cpsXFc*de?V~)++aOo9x8#Lp%@kV9&@ck+B;^Rl3%Y3yr%=Z?YTq*1lP1LtD zxnKXXo=dDfBpoK-^aB75d}1~qz*jOUfYXdrc3oh>2bhc!K(kKalS?~w5-~53{o3#8 zsps5lK+q>Z5K3Q6_p6N|Rf>M9rg1b$d-zKY)3N5Kd8zD#UT%q=x$2kvbN4eGB5qW( zky}K9rw=?s!9MlhMbno9!8oyOEcp@u#-Rk6B&8S|2@0Ug?UF%d{64Jk;YCpvf-XBz z@c>ZQc-apKtc5CYeus4i<&MEY3_#M9<&H|c8#9?BgS%oRgM@@l2HatDbr>Ikg09DP z-dfbyxyc>4=+8RO6n3GT_44F0B**xSE*yEj!(Uv7H}2#$JI$H{ zt58_U$kc1$Y~aYGO~3D|ndg69yR<|jd}?t)qY2M)jN7CAbe@rhK< z4#CxHeTSpml4dX4NS31JpaKyEBT-=wgy!B+({C3a#n3z*^Sysl{{|f$aeo;h`7ttP zuO&6z-y=rh!ws2S-*NtvCfuO|$!(j+cfckS$)o=xYr_&1^_3We&=5u-90K|vwe(;j z%8y2~6f*muz653RNZW}y_ps@f?LA(YVw0K7`O1_am9W@nnHg>%A^1z62Du#LoE`j} zAdI$KdV6DtgicuQo0Jq>(}JW@Q_cBeJN|BxIKv;68w~*xU$uvXWV48KmpyF_k=(u) zUS`St!~3yyF|MR|rHH;Mi8NK)GFEyWs@+=*261N1+O@ab+}e72G}iu53I622QMPD6 zCex!U5RnSb!2|4!;@qLyyi{y(XXn3#m|ULOpPZ%jxf^OfB>AJvzWN;VPq+ZQ{{_gQ zVygF-rlr;;!VCp6BeUN)w6Bt&+c9K_hjs7> zkWxw+wJrtEboZf*;PWcTi=#*an-XZ~-W(qvyQWlvd;M|s*$3hT{P!jDjyB`^%l$vX zIzFU7aSYd{JFg8iLHqh3@M(E`X$m~mIh5b5hY>6?vIw9ZgD-Z621P_j0+t$lc^Qxy z`wxzeR9%enHWwFtAv)*0E06VDSBuBRw;kei7bxbF{Ado>hAy!;*d?v9^|HW_Fg-s4M90PUwSw+^H!g82zM| zw0tlyq(dTLVwq}441)-5LHGAG%D$D2!N#g|Yqri-3=IQv1MzXz(bZKhct-gW8NRxT zLQYO@srQ$v48KGQQ|AQuj+`HDP=nYZq#g-1=C-NJIK8s-NT8aIX87(%t}Y`W8Y+8C zEXC>;sIWzjqQk&K6$b{&HrJOfRTh(&!Z(*(cQ8o{fK^?Aa;1sPQ1T-T0G^L$s|`qO zj28!jVryXy$nMka0=VNfT0NiSDW(pn9j2kRv% z*t^{yh3~YsqC#d&AzFhXNW!;?n5jRZrA{LxEKCIvVw!$yFGqzXllU$N-@+DFqVxDQ zM!PxRN*+%9=qbD-AX>o5>WF#*1{C^VErRgKVqd?E{SO!5up<}U!XM>q)>7LEuvpeu z;f4og8>jXMs;Tw6sK<^!*Jg7`LFyzmkL|`;r}Q%H{G4Ng%{)~2v2MF9%qw;F27B|( zXQ5jwI|FR{~xZV$YkJ^L=+A@)9dEi{8ISp5Es}`M;P}; z6rKQ;K_~F0wEWU}qd29NeCu?->kAWlXl) z7dx#|fUvEii)|#{HdSpwczwDAd!(3T3?M%nz>*;4wYdd3>dt%Jln#Ka%?6dCb8FIx z|DNUmW)+8s8Cq`?K@i-whGsULG&D3su`1F)$A^MzO=72JU}O|y_J}V(Ch#>7@}w&8 z@$#;1q3cX?{^27xVLYszl{2OABxchI#iis|^sWj}!3 zs0&QTY=10l(&Z6#%%`Dvj4=zLx>hWp5#M_M@n&~_v3+9wrO=!!PlScfUxUt+nKsI7y_zhJ>&_IM}fgIsTtf(Y#fd(6sx)4~Gm0_rXE(PFnl3XRFUquC>KQ=(3Vdz%z_VuNlt0!A#C%`_+Nj7UJw)yf z!50I5Yu`Lsm9vl_j1=<#fM?-=MO*M>JVW$0Epbpeg>TaPSYck8;E`v|t`6fZs5txP zhK8BP$Kd;A8{KZI+3a+8rV<^8{*;|Abv9VHI`ch{8G#reZ;zImwT9=`VOg= zJP|j*&Xf?30e)hId}!n{4(s2Pmx%f7a)`eI|Cf9rx6@t9?fKS^=dm;&LkYUim`wkb z?xaF6(OqcYQu$!T7`gu^9l?)pZ~D3wo_ z7A^ZoFix9%ycG^y^Gn%7RP=pbe$qDD5cbr9X95B2+DH*EfWG9bx$O20fviZD`Tr<) ziz>dreG4+9RbULF=)$tFLFh{``?!7Q@YZ{lD0zLw-UcI7q zbO)`D%yDq6pLPWUq{AWr1^X5Oi$=!z_0JUR_sG6974NX3O~!I$WNm=|)^+|mJ&#&N zRMdu8*S(}#R7WZ^&J5YtS+Vcj3FyNf$t$n4|Q36^?*KCbwnOQq6eskVABtjhTo zm4}#($G{#^O3O%_Y~R;BjY>jaNoOFDBZBAwGr6nJfWG%z*#}tX9hCs%fLz}4OlGae zj9EW|*#%XsuTeUzBH9Lo-EG!q#7a^p(E+mBZ7?}dXU9sU|Cu)e{pBX^;mzcjBian> zFSocBMnPcPBwJ66W19(9(<$%>PuH63f`5Vc0Gzkav_jI#BQcp zvv)_J;)ILhV(B71w0)T)bC2Y)irQTeiHFmQUDk&XQ6T!F(nIPvkI-9Ay9=U^(sO#4 z5Yy^tW7~ctTZsL@_b~*9GP2HCbPrKvaK4C6U4jjH{Ye0dhZE`GKT_72n};A+`c1=Y z<3;VeSp-o2z>&>!wpbhzMqSqXz2~-gU|4oOK;l;JviPRv=D1?o3TG@rF5=Jtpj})% zmjDcgSa#1sW*!XStl_7>C&7-AZ?{WwW21_MUJ-(YAogKBKEG>e-lVd!TgE3dY{D^4G9svXB9^VYT^oxw9Gu&{U9I zp!-Sg^nN9w&UjT2VUoX}w@XnLP2*kPWs8MD!skusBw3kn0+SU)=5r=22G3D^Du!kc zMsuq{LojGJL&ivz%VpmL-(IxDg8M?Q1bz8H$m5)bD1B2I`yOb|aXC@3jR(VlBl*UM8*(=Q*=mNg_wqH0Xsf#IN6amz0D$upx_N*NZGt`KQz!8fxFs`aKZ zQ~##@LSR4!34J`Y`S>`A0-om_^2=k0ESUuHe*01(Ib;wDpd&i(F9%nuL52J415oYs zd#;>KJ5nZS=?U*Z#?Md}zm2xdESlfF>H&DsDkNa$+s)S7zgi)``F!iEonN#&Bf0*2 zEDk)f!&VybxSqRR?_&VrdJqm7V0(vF9;s!!WK%&3>y|`2apXj+uT4az?z0@4@J!EA zL$)cvdH=%GP4@iBALv2v}5*R zob`8rOXbHO2~Su#6}!en{R5ym9;Bht=K$Ou`CqcqsNvX^$~3)u?~ju7vFlmA;}&rU z0c+9JIOroHvHedvnP!r=;2I96Cl#>sSW#$$KpUcaARUnWghvW!IwE%{K?)d{m`Fs+ z3)DeuCd*jkK-YfOkU(_TZkIV8%Z{SBA9s@5Zh^7GkJNKu)$X9Ys!AMKH7J+q@IfF? zwL5tJAXaN-6a+33S!Fm!Fb3uzATns1n;?Z)O+ESo8CsvJwbbSI zSjpbisr3YI$-<{Varx?(aNP~Z)1&d0m$oJ6^^XAW;@bu2K%n{Q77tH`t?_IR8yE(n zzH1zWjipFowX#N_0Mb&> zA?57wtJ3&&oTKHNH}4zsfXSpe3T=C?#9b^~`7zM$TBVgr3(`$Ki0ML(ITSGIi=qiY zsn-)9nX0uW2d!I5w>rDKC9c?fpFTH^+jxqEZOceY6-r~vN2i8E)ssUb*yk(^y1JJ# zP{)fIiFS$qjpNcpaRU~VjT4k7c<(_4M?s7BXlR4Ki%2CX9dw)T;C*0OtSq$Cvyh4N zVquF~-Abk*K(mWt4om~#Q;4X}l=L&RLT00YAl&_=w6wHJe=JmMErFb>_mCFweZys6 z0ipBJ`lAw-qaefwAB){!9noJHdE^y7(Gz(i#UCHsW>S%!t-~p%y{#2PzD4AmZn+?!+_g)%URbTomaFi5%@~bZOB&=5Kd- zDgK5sq)~vl{ypZi?=(<~nFA?dRD6wrL5~lM9F%U~x({hZH6Q(U`{?0`0EOxTUKoyn z77Ky6x7lQwZjnUUVXC1dv4yLh!S9^^50GHM0v)|9%Ok zS4T<4*M)EY{0Mjg2~zeI)9m$7T6M+RD`Z*Ru>6MXcS1pAbvugZ4v?6BHTYjvf+Tnl ze%P#c0wEum6^PrpukC#cf(sxyqmn@|KKMb=%k9?t`fayf$kha&jJ|UH61?^ISq06V z3M`PF8*8Hd(VmW(LQEhW=4b-gj0`xXG=m{}T^`^lV3Y{rb^&%Nq#^d4D!-|@Nk8YvZXM9gP`64J`GIXAAeiO zqy;spE1^k7p5ZHApJ3AB8n?{JTinW+KJ9%4d-v#m za`O5Qexd7xsKIZfK#o9X*89wTjGOniYJ@uhTBN|MkvbLXL=&a8o39t`68_!(nTGv$ z?yP^VOQ|PuC#1Lj@Z7Iz_@!SBzOZxx!mGTi1GXPq-(U-&IG#e=BaTU(__L2SAW6hO9v?g-p%AU)4||7U98U@YqlY5vg> zOO Ghn54mDthj2C7DIZG5He?Ep|2-FL>6Wgo41rU(k#y^7 zPRoQR=|h-zvYT zHa7|&<~X^PrVF`GnEj@)2zNRa_W+QCA?PgAg@XM+UP0kVO9dU~3YUGx%W}#0MwYdsBJIg&hNI6np2H6o zHQH0yY~#Y>TSnE8HY`XNFLo{Ht^>z;i$98(!S>NcFT)#kRN?v|_hLS=J(<_3B*T_n zJdJtq0UjYnd%`5=lo>= zNWLwo;@$Ss#VSqBD#Ach`^@Eu>*2Gf ze(sJxhwek1A)wC)%+AgZ*`e_99QyqEDB=Jx7LTTRz_p$DX!CCG-QxVBXLK<~GIeNy z;=bRF030jA6^i+bhKF5{4f=_#NrgONKTYWvv*8s~4&`N) z7v{3e!NiKTQNL8^Le#WqH8aZ1)%tOFe+vEHbYZnK*#)Uo?*q|7o*A<4^k!1W2A0s< zizdH<9(2~wdgGGb4`Y8E{=)uk)!VKp$#jD^gY55BB`>RP1`rOlxv*NkvbF!1;kRmz zfgkMt5_5Rgys>E2CunK+-Jo}*6o^VYfR2`8rqfK*fY-ZCI zXcdGEZ<^>Rtk#Is66@}{e8T?mYp1>n)|z&%TTkjre1&Jpj>>&PJTO?)B7KY=H4wsGQA;&{%1}5#CM-crYgbc5Nay zcl^~k=KA+DAL3kP&|v*r=@lux#Ygs~U!@#aEo%3tKCTDfygyd5Gmg8WakEHSb*z^1 z-#AIj&vrOdE7H0=s;n&+o0hfQ6}bMLFb6)c^b zDCcXkPTguD<`&`MrREm7jNVBzTrm6bWlM925oR}s=b-qI0I$C8E7+?!Imag-x`kd$ zW)uc9b~R9fX}{AqYo=KyAuE?+(&FA*r8z?Oiave9Gmz%QBXK=K0}v#=kt!J=`=l(4 zFR!6+oE8el>sd(gz%h@F+lR+VW`YAO?tN2IQg8^xoug<14fYOzk9Zy+G)G`r1VZ59 zsHl;+G}H4OR3`PZ5*@bSBfm1>7*Uop$Ci+|yL<6&kcjga7Qg2+oc!S@T}$-JL_8&Q zEoZ-IvaBMzI@_dGENTcKCy+JDM>dyX!a2kDB7J}tNB}0eb?1?GX5zy<%nV-%Cemf}D=dcEe*4X(R)o+z5HMpT00XAuo^A z%28`ZqMr7wVb%n%nmr``K=@gQ7!ba2r_PE$sh??h$~*+51>l_S3IEk1E2yzqgOqN4 z`t*UKI>{XBiT` zA(bf!C)Sv8^C6C}lfiak9hNXR&?)Bpu}8CdE`!OjVsn`|Z|ksj_R}YyG=AX4$o|`_ zE&5tcN8kp33O_k5*~`}oGWz`NrR%++wf$xs6dKxIDa?uP=CH1(rwv;PeQ{qA*#;Zn z!zVoN@!Mf;UR@g)KJqLG@5v=zJorf+$5DCw zExqn~E6(Js&lL@pH1Cv_&&(-?R0Jz_e2&am5)jzoc8T{6xs z$ujb*o2RO5hZ>_KgF;$iA0D|;P_W!WrYBSlYaF)`?Y_k4F#YNb+!xX0CoJ;)Q=4^5kn(pizB#??yBQQ_Okm(Q^S&m|{}p85Hh zw={YjXB@<^IahFKb>BT(NN|)*B-*{854xt+2}We8pu;{b z^3cbfcSz2|fA_X@&<=`hMG*fNhUn;Q%oq}a%3-*WglGBgJP;X}5j>@%+>RG_hV0+$ zs&i!admEPP8ymAh1q$M~0{74g(C+uBKIq=df%nQ4Q~=Tsy>pK^11RcUkrVTFqt4~1 z_`p$lf_0a;nOu3aV(Mr$urJ<;4AEUk6|pNb+X?PwCo&_tJiL*}dsTo#2C05(nrj!= z_pZUAhY2|3yr0$ts8sY%w{+wkXxo?!wXR>0lX<~c{aVes5%~lY{3L@b zC{S|9*{+0-hi^B86j^k#Kc&d@=XTv`R!8{NEGLB|?)t(t+Dc3mTa@2d`h`f2bo89| z8ch{m$Tx1Bn5dCMeYe$=3wp{GI;1+DrdOD6^?qS2IXvF1*ZVWKz*r$nK zKhF(*SUvTe^?piI47xNG!*cl1yHCjr113(a{tw6nTCs%jQZ-ao8L(L-i}_3W(Ov{#J zMy*nAtfkfoexJp=hoxUc8!Qr{BfN4FYcb+9erqP(kpw?qt-atKwF-K_{V#zlHqx*B zJtr08OPs+dn$zA=2)ny`BWncMc7*h4QpaIGbDJ-0UONf8;)EP|OeQ_L^WhJA5oLpj z2Sa<+!weRT^nxmOhO@J($#-^;9@)`Gv2=-<)EkIDBEq13HO{W2HYpr!8O(fpDekLwh!rStEenfx5 z1Fb>jy6u~kA2~_qQIAY%HdcKaxtAnQ-b+6RyHCOAj&4u2Xz#Ad-G$E|Rt=1*Q8U7M3Rb~5 z&jvozQ+fQ%@Ruz6Nyo`Q2h{H-1jVd|7E+$yTGD|gl6tZz}w6V9M zB|cqXxVP3%Jq2^<2ta$K@I}Rak&@p=W`jl<6K9Wd(Q0Qz7t}H{C7;OP`PYxSa0UnV zoy2e}QRPq#Hq~8w@GiJuIe3|2t#BodcquspBTfKoh+HBukS)II;`%+y9H3dIo8v+b z0G;KhZ901t2i1K+C}`bL`h5~?kSiV)8O>t*R9Lj{o0xa~STOJ_et+*q8_i z2vaQ{je5+0ll1Pc=||3PzwZTlyqu5sC>+Rg*!+S5-D-puoRvPv)^tZ-sAWg`v2&93#h2tunSa5K!joFk`C#RX6T_o8Wl+~Xr#MDW#|qG zkxl^-q)Spjx$Tn`_ z_ubXyM6{{A%>>yxNzL0Xv(+o=T+N%^$ppbE2hvjknwe{Q0tkI(BoMg2nj zh2s^{>{p2%NZp)j-5tLTQIlm^3xxBSGQv1KMBIW1y=tyv>c@G;GsudH$K@Lt*j zwyG2z>bPen7&qS^6&HlX2M*2huNUnlAVTYo;dj|sF_CAccia_?SyRSo;@CC2g=xr@ zG`}1@p^sNrw}rr+UV91X8M7@@hi$w`3V0Omh1foU%IommlkoR=slCF+fSh~pDyi*y z0?!ITN!-l3m9tg}A6ER}V*4)z~3 zF&GY}nkt#7*`Lj>;^noj4(|58yMTofFNopV6{1vNe0sDHD54Y(sEFle&FG#p#j3>h zKbDtYJ{x-54!E8%nKJ$~paMHW198!WcXUV5#);S&kwaYG_7bSo7?j-?EHD;Y%05D& zSE39irezjt~(4`S#H*@D(9|JR9Vpr1PrYLb(`DQDW6G^l~iG85fd?e_d4N_M@J)x)&uDGCRuDg>ekMwe4e9gbmnVH7ir7 zIxlvx9I>=&1n{Q<E4XxsiMj4$@l*A^CDrNhusHgway$`Er@y(9^x3!|gT4o=Qq zYA1$U6r?omCC!|ime*3gZoux=JMaGD)QVzF-%?^v8OFn(Nlj^W^1Pzu?XMoj3!a`< zn=Rcnzc*SGPCp^zv)CSIeY3cg>EiCRuzTbCt{oe z56?g7sQ;6Y5{y+D)_E|{=LLriOpEB}-p_mbI+C|#MI}dd3-^1+3%d8gr0JL2t0{f0u+`f8&~5CwUPB!gZGYdLyneiH44aM0Rc$O=m5>Qok{qGvNq zfj_{OC7MJIAJNY%qB!$WhNVM-m6O-TKC-(jE#I*kcZtJdCWX7HB@FcsYS-pI+PYDx z;ZT^xR*b>?s^_XB@Yr$XX5@mfV0c})18g)4?igt=KER`f+P#Xe`eR|?<%_QaYihO| zdIPf8`}7!!uspSd>${jG^;(DJj|2NACn-QT??{x;Oh$q~=ZSt=R1{%GX6AvmD0=Nx z8etllni%I$^O?7leU^uR2GSy7z4opy9K#ZWb|9j62IM8JT+sfuU@c*pVJl6guh#J@HsoUsGz67k z$8|>?nP*I^!yEi22So6Q*C&EafjHvdv!Vf{;aiw6z51@oillu z%ofvrB{|VS?u|~vELhSKtzTAr((1}y2Sl)CX5GG=VU z`FW>|Rt;J6g}WuUPsM8KsEy7hMz3{>u2`57EThq#J~QA_ZbmICGW+1QwXVa}_EyJa zw`kwq$EF_sw0Y%lQ-ZAIz4@Mi7_2@yTS2EmZ=~VdU zaqP`@`#zi?CceXcl(KKZa6too!lYEeR-$NZKcw9tyvE2G#QHvYKz^NFwcQTt#!Qd& z2|#l5NuLgV;1YWIy(%X8(OO*o?c^G&f=`z>My51gz9HKZ%|Sdm^P^U3j){^i+Y9HM z(6hH=UZKt6Kk-u^vC9o3E0)fc$-4}JDe7ST=rB+c1k(I0p?<|?yYLfUL4l`$F!1lXB`i$9m!9>|Jb0Cs0V@B z%htVzrayXM{!)hFaYSg;uuk58tY~e0d)bb#Uo}pFunG@Pu+POLS}kD-kj)eTOKgHB zB64SMFJ#T`SeO39jQ z&yoAm1`Z80823&)M%pKZb;4fvrb*ngw>O((_3*?EC&SRoS8wqC^04oL(Bm*&YqjsV z3l`?&q`MSe`|M7|{p?|#rXt=9P7`J*9v2hY+$G7VTNJ5C+`uN)zC)+c7&k7FWF;V$0i3XgO)tvz?%cpgd zoflh5QMTci4^qz5jJa?dBWa{bo7H@3hB>kYCx0TMf+LsGuH4C4cfDiQ7E-3W_o{~Q zD$qSB0M9BWsL7gK$VE8gnMzaTlC1T$b6kk<_M^9eI}b35*gO#Z#*=YK0UI4J0=!gi z`p4Vz$WBGQvKURx053$DYSD9nEp1dR8F!qp|4sTvN~UaWz|Xl0yW#t~SoUq(53`>F zwmR^2>QBq^KR^apdAviSMabF(CnfE|=3>Pz?8tqz#X?(8d>fQC zRkJNW-BbKw^n-W&&B(Q8TfefkG7r<4keb5D#R2hqQKJ_BIQs-osevh4ks!ML=Oa6l zzVrG)3JRHWnD{If{}{d%pDS!-AEp0)1%C({>tb-G+K~#PT?6yybS1|Ma#u@m-r+x*v74 zv@Q~3@sgK2m)r-GmwFfl4EGF`$KthOrzEyF>ZG>;tK<*ZqYIYRPB0j9 zMvQ)@HX2CsM~Xil!tTTZ_h7yD@lU|0@+7b!Li18_&;DHYCl37cmMZB$s{79`T*DVY z1`nZ&!1%0@*SZuxdv$qzVnd+0xsM0wL}dL)GG1@|Ku=#E(X`%zkv~s ztMPAz)}n+ZP14;~Ys$effowr9WPEZ`baLad>)^ry!)~E)vBo+O(FInaS+zkmBp?`l zk(BGDii!CB|1vRSMBi1082ywM64UtA_%urPsRlnB#^pFZyx4yus=q+&9c0_S05 z<@p8RDgqmbV5+AoDk{?K3lIW=4sfW}R8~_|g1zLcE8v>O0rC$vd0K&2WmBlBE$rn> z8KsZcd9K@emv%t^%ic~^3_`Zksl!6in6()sI7O|7HQfma-6Orcq&2!4(GQJ>Q3@oX zq;PsKFRvfxCwpAfvo&^woS}~-CxI~M@t1gYx{uN>GwU+tExUzh( z5TEUz3SS=aIk`|Zfs)m~V;|c_%P5E1x&Cf189h?O1Q=>5pd|*2zyt}ypSCSvCDtS5 zLoh)>cGq0b&>ZLFf{RW`_b2XqxZPoHELSuefL>%!^8*lKIq~KeLW#{zd7_ zk?OGRY==Dsr*?Y@rT?97jKRIw!alys*5X5Na~BR7gs)wO;F6I1c^L2*J3>%M8F&mOYJs`UNOxIPPZ10n!WTEKC$-7P}o0MSDzQf+FTmpay z9KV)2QXH{cC9*%SA+LIJa#fV*81gCi!G&MtQ0(=Df?FfBB2KXkq$&ncE@RQnwLfz2 zvOVuSAPPA7dQ1DQOwVGgWVVG2Zk_3FN8!>@`@w)TS>HB-zsE==!N>dH6zqz}UtD{e zeFN{`09i~altO5}gJbK*ax~<}szrpI*o&Q+sn8Ek3dO=HTt4Se6Y3Pe+3hdC(b|cb zD`l1E@66l_b?)N6W*%J}J#VV9nDwo_&RdGg!N>mdyO{o*E#G=~I<^SObY=y5Y0SPP ztTYjZE9JX9U}@nKG`}j-lcBwSz!TAKb@a|O z;`8w%O%bxvOM(XX+tJR7rK=Lp_>SNCT_n|QPu{!Q@|Gt4NSUQ%+-Pv@{j}qD2>0%m z9=42MKi^iVv^8MZjH<1uxyf>p{t;f&s`lup^@Ec6fJ(+uOb2EIj%kF_w;^I9El(_eOSurpUJVnI_dk_S>Mqs!^!RBcl@7*7Al6@V zMh&anZ#iU^IkD&e{aep+6yXV_IK0VHKqnWwB3HJ646fH0YCrhxzdV!x9gK!T#R*zU zV_fouHm2I!<=S7!U7WV{C51i2GW&bb{n?^fE$YE>$K&ZBE(GwM+8B^?f2;kK0es`z z`k|FRD8{iA^Ik#}u_+&1hT?&GuRBSDACL>sNqJ~)NG9?;*nLH&(ca!J9&j#>=GAk) z$-CwbNU^&4>g0g1CBLQ*W8GpRL&b7Exe>(|YgD5k8p%MxL<)zI@&As@wEMyFc7Q;((x%-hwCjW+U@#^|q$K%@H z4bYD*B_Sa_PIk*Lu6|x}5$An5P7~`5WoRuS-SB$c#7s|gH340j^JhA0UYXQpAwTut zj~W-{969eG6mKpi5NgcpA|JvnnsU4ai#cj*x2hdk8oPCTb|lJWaem$rTYT*2MYoX+ zrRt_as@LrAb6Qv>h+j5^qj%kZzS^ol%60$n^f|CV$8@5^nDp{$oJ{|Srlxpf{tCy9 z(5!i^xq$GR2A$3l5x5e3J$XBB`v}}Ou>imD*qRFiuIHeE4b5{G7ayjsc&be_N8K-W z@6MCHyixh(5LG-IGDw&&@sfHCE7J1IfU@-@BQwn-;5qigT^f&jMMQqMqv!c;VM;p+ z5yh{buFQ5{aKvjq67;1+Bd}vBQoK@yN@ygT%~|{2$zE+y8C^Q#--|vJ@Nezm2<(`W zy&3r4{ymbZ0OcohYMNNuhzV)NwM5+zqmy!{j0m)TSNw+c$!LBl29?->KscpGxJxSl zcy04u!=Gl%PAJ6y&cV&K-LKtUA>Va>2d;eVRz$w#{qNNJ5)6)Nm%nNo#(H9lPDPgo zYJ;Fdzz_F7GYgZ+#UNzdnV6GNh;jD=ur9H=z+Fn!Jnu{5Mx%%8Qx!4HO5t#|4`kjL z%c1?vzDL56IlzUm7uaE{N%nf9$Hrud6w*3iYhUs~z-Ct0%Z$JP#ptk#UEKAYL2F~B zW72f$`D#V9?saI2*Rh|Qh@1T_Lm1Eb0p@A++^b=`kT_8Xc**-42ls4CSk;^Wj8%Yw zQ$h-d7JBAv4m6RAGamy*m{|bAn3F;nM?;DV3nkg+?U>f)TIx9LBEfxg@4pB0eO?e0 zl$uHsp>=Q6=;+~fL?0)(qDc?4>WZF>>?NSleP(}~$V+i=N&bemDi>>DV4z8hA9dl~ zOofiugf(^5pz(cyvQRWYv5av58#l~27CG{4Fk|Ejs1EX{It_J<1KL!xZKb3dDY0?7 zz=-QYA>XCQR8l;Yv~DdAT0&^kJs*8v73&Pi8^TLl6w)e$JN>r7m@&~?h$p|Q}1?I}IreaT34t2JuSzZN<9op@6t+=l}8ARtEhNwy8TuIu)CeW2vmoiDVVYzqTQDb=PX_uzZi$9SOb2{}V$6{UnJ^b7Nvr z>F_?xKrCEcCk(p!5u7Y#fj%$)AfuCBs@)O&#psDf@@LyALorU?f2p9VikP*c&Z{0T z*6)~~H*2M&p}7%eG>o68nmuxZgpeul^{ZDlY3vgBF5gZo0|ye9WY5ez36UQn{u`)7(BB_@)UiJJ}e_0lF ztt+JFJynkeNTxFI1t#=d%uEM?Gu016;!0c84p$Isfi`#*I!hw|ogv`g&H(@YvG*r? zNJdAw{~J|<-SkaVSH5u-StrN}{%sofKanOY14tIWJmZxh1(C%DN1K7hWb5k;;2YGo z!6t`_#ctx2hq1N`PygwG@;|>78-xS3aJC^ZzDb0p!@lQeLxk2|LK&}tlMoxdei$$Ja{!$*D4y&xgTL4`neY+h16~K{ z5QgG`i6rNaYOw8z_HyufN-7v^D%+;l{~jZM&l5&Ia1TFpC0w=51>{Zl2mg9BPRgT8 zE+0RDobC}^_{c^TaRVD95oot{e+deDdPm7$%gbaxK@%b>Oe`!Y*k!=`X3;OuR0U>8 zFL1ECLfm1$otAD_0QcG-KP<@j&FBDJA;c6r)K~Ai5-aLpygZb(uwP#ZS0;Jj3N%#U zP}l3}{|k)#2@#wKFv3G?pX5tugNL>Qp)gWjbOD(D#R@&?qblU;xwyDs3RF+<=LJ%o zh|55CyZd+0j$=Q*l7P3(m|j>_3>rQdX2=H8Gn35LI>qojs499HYXw|p8``;At6Q%w z&wA5DN=X8N zqd*%DnbTDkkW~~LB&=Kk`+A>2i0pZgQ_G36ECJB9Dn?N7?u=>6uJOwY)_@L0ixsVx z3c;8-fJb`kCmGa}`p(mr&CSgL^LbBtV=;g}pdHjC7%Fa4iP)0O&CON1Zx}Tj{d#O? z7XePJ9B>1wJ5O&GXlIq*gv<4MYo!Uxj}>Sg?D4HoJM1ohNforZPA}t!0H4F^^87@% zyTRd`sIcSQbk5|P5HN%O$%=#P(5dvvKdChO72MG8zuM3P)5OOGZRJ_LV`JmmbSOgr0ub*x4b9EXf!{0++P*o3p8<{djzEBl za{TMuOX2X)m%~1;E8$+LGDoaW15I&5e3B=EK@4+j$30#QMVZ`Jztb54>mVuq2sB(E zn^R;n2C_K!R4I?&av+%5x0`vHV*V)a%blyO#_dPv$0#ZZ7gjy6=L|K%rT|${ZobLe z8AuTg-WwYZrI-h?(kPq?oH~4O5CIAEf$>R~>hdcD?plR;YXH9m399n>qQb>FVmRS! zC{yYlj+_avD5?^0Kgg+f-z0g56Q84c7=Y{-4;vf1Xqq^V5qe9A;RHwLGBFSr6#Q2% zF3+};tMdN#nEbo&SyMq+@*)m}i5^?8Cm{MYqc-*c`9`cJKEf-uS<>5PrdNmTn*lH1 zMErR!e;h%T{QA$IuYl`Tu3{(=7kwlH9)SJ6DBFQ!dp}h;j_g~l)0Ead?~f*fal(AS zd}qQ<3m-&o_}K@)2FX#C_EA!wKfhh5*7_jJwShOmn8o@r z4d`iUPAB6`yBL#YwdBEHc2+OpL-;++uYU7`qZ&cx@D`RBYE!#h++&xIW^ynDXF(+B zqy9@gpaQ}NVo8t|0e6%0o1i5B0*7E7XE!+zh@}NUXXuNwAMGLTqyZBph7I^mON-1s#>Oz2=YFM}cFItRDplCv@n-FU+aQqIh;&EaiU0vzHs}#R;~`tNm*9%c0uaWI#&o-m7K&|>po(k54r3;b~&GdaUGL(Sd zXaJ9K0LD_Nq0*+U#7NG@^?}n8B6?ZxXnCu{cS^aK;vVaKei!tXok8KL!WXycO&M-h zYy{sn^p$9#Hpg1b|D((TSUj2aBjF`mW_nS_IRwzmZ`kY_Wx>@{ zJ44`NZd-E?u7r(&u-l-LPQnGAo0fMNQW-C7;JG!|o~;@1#sUoc&KJ z=P132IGNCm`@@`pRTe8~17rq)x`eR;LCZc{nzq-aj zMgi7(`XREhwGd+wZ`zZ)PmNl=k

Oh<0_v7m6Y#ri$LW6_NY2RAdJ>kM(oCeAqw( zXHSCB^Q7z8N{yh}h5~A-v-~|D9o;8*)%mwTe28X(RV;^grursT<9_1Kd2U2d?`d0Z zK84UtPs>+4Y&j$exc~gd(f8Z}R-avL)y!F#5n$mIdom9F-um9?{CiqZ>ly2a%?>BJ?{&@%dRIB0$jUB1&OEx+Qy?0F+loVY@O() z0~XH&N(^|rv|gTa19Jdt5(topk4r*Fm!rtSFfIRM>23$&3k$8%KRq5L-jyD<#Zhsm3`N1t4kQE3 zI}k4ST4!C?`96pJFBjmOhq*qAm~I#QeS?z%KqID~Au6%rx;o6Inm#;rc6D28CDdB~ z)fKuK@upL8CE($r&o6qHp9o@IWml=;TO&fZ-b8V+qlVu<6wA}jGPC1Q^~naseM_L_ zuJb<=?MvaWaM5DM%qyxk;di>~zXKGt2EZ&`O`fd&iz}}#n3JDxEAaM^rrWCJ+y+%m zLp)Wj8;b0mB5QSeyg;`kpG$kQp6LR}e=!*Rc7ppTp`=2AaiPpN*&5`xz8Yvem3<2P z!vX#ietkx#^~adbUf^7HB1Z>5w)GOg={1;@5uod_4<9`4455Oayn_A7Ujcf8=XK+B>QT1mpWYQK!|Z3g30wJX3y|4Q za?gRKBSH=4C%9CntXGzi;#TaJ6M@Tl&NK8DF5Ua^LeB{_d6v2%5=Im&tfEINKPseV z5CfM~g~LHmXhsZiL4j__I@Q*?7cfMV7d48pD?h(GWc)!~8rNv;xRzR(SFH2Z-jM8c zgL|n5QZ=x!p8aEqKUMAh8l}#fV%^xF!R4lr7KK!3>x=98^uoW>P+kfaAD5zHw zQr;{!i4yZk!ripTKVvD+klfug3#r~9KL(XG)<2XEWteI(z1k~QC-;8#!ezG<dmP_;tqkPi zxst5#>JF|)xlM$Rh-=4YJOfc?UDPE*k|3euk)y>Rom`*5<1Zvvse~v-o0oa?VR_K_ zuYO^7V=+fSXxa_BUZB|}o0{P~{ZG$;gcJYG!Vf4r~m<)JTrw1XcH~YL36>S;7HKN7EYSB5HRKcUDwqgt+!LDWuETN+w&B0>RC z>-IxWoWpXb7!Z*wL~CdHl2DJ%Se#_AAA2X!0su)NEQjxZC;*D=lu`o^TqvwsVSu4n z*!%lYsOZ-lN}PPWH3@}UScluVm-jpv2g7KP@^|CijI`24Cms#Q+p5}%smecvLX{I% zOjy%;#fnjkYh8K64e*=2clgr}cD<)7Ekhhsal%;)A);0dv1Xw3dexT7M+?AcuvwE^U-we_R(E`mk zIV*_!;4++LI{+!b^4?VoQm&bo+f0-saZZea;!5tEQ4X`u{wivnUf>YKk$-VjIZQQ8 zP^444nfM(I618a%dnG|3-4P!_nI;SenGY_bP-!da3W38AIisX_jB6pp~+?l$7FtyF!E} zz}kOjYy6&ZQU_gzq&vDd%?d1Q9nA1VMYmZ!Xw+Q3hp0d*mymax(p$Nnby!y9T0h+KcWwBv>Z-sH zitHr3H_mfRuh&9)@HNRa#gipQpPsDGWH@IfHn!dS#HYPuiZm)hOuhFBzyDUlosGC2 zPBEc;&vuj@$8ec}ex7#1W&YU{+<)GZ#Tqsh%3_VX6NSdGw7@HyFPQjcf(|@i1pThZ z#lcx?Y5ZW$@URsPoId`HrXvAF!KCgv2C%8IUTBf7di*6Phw1c#$L1`P-DDXJ(qm=t z7O>I91ap44H3C)y&;-ytK6}yWZ(^2o>}swI1KoK-OyADjhvj1UcyQX#baG^EcD-+8 zy5A~%sCCx%u7^teDdYvcOn0*B6!G(XS-M*!hmouSlY-{n+&dJz(cFm5kXnnOq~_^g zpKd+t&WuDYOUZlc_^I2_QMFL!yC(V+XAjJtk*-q|hs4ilbj}P>)H+aA`VjMJH3R`> zud;;aQIfQE<2x+OXUEOs_mPcWd#`xCIJ%^)jjGiA3fWJOQwg&c8?Fe?CtZIM_>Z5r z5H%QVMYe1osJUJJGO^xDOC7uCx56WJ$Q>{%MpA+@S3zT$@QfCEmJ zji(v0v6_#f$G?ycg#uX36jj7@*U@wFe=F;$LG@4w3CzM7DL;R0cV{OEpa!#~0HOx2 zL4+Vt2|AD8T3ahtU_KQ*Ia}-q6`9QjJqm7YPJgiS(E^S;v4B^Ott71jV?D0d7{}pf zpZ#}F2oXZiTW3SG6K5j!{ zoM?NfkW!%myLrAzn&-no9`IX$rNxGA{rKb?C8JBbKT=}(c`r(V0bG*$kv%NU;wEJ? zCu(@VkiuHOT!d@oeIy_oe*P%2s3yX8_(ypRCMJaz03dTxLPOAibw6H{CLnva%*@Pu z=|Z)8sE|-j8{gtF6E+TeUS|ussKJ<1x(mL)2vY|5<=ib%m<(*%l>3u z8fsl|i2H-!UOZ@TWhhrIfgKR?A2%&B#4*r;y>L&^=K*TxTh_hd`URt7({;!*j)!#4 z%^~Yw14CncWdVn@ceB&K)|!i{V!QKZ zxeK(R`SAS2yX?W7NkyXv3tpDLKh;AIpImDu4xY4;Yfsc43&}sM!lsfkdMp1<6}tC; z==4Hg(*fv#w#{Ci-SXToyF?LCZr4?leh)mMx>r!wF+XV*Qvtf_n%kd{Or2Nc8ee*% z6wkJGSv4#Hg`cGGN+N_JKi%rQJB&DZ=SeBQ@S`7DxK}@(w#i!8R!i+q!vioWknpRfA#{Wih(qDdk^Qb_v%XY<_&SQuys474FC*S%-%UgD^ep zE~7H>WxPJV&aj>KQkJl%PK&g1Yu_jD?i(8OPZ=+w$)&8bNZ9UpUcrD=;X0|P)@SH; z?(yn($J7}aM>Q)1uJ_Yr>Ux(eE_M;xm;jM8+tw;}#VMlDji`jQN9%hO`ono}7QYV% z+&^3QdxfUIJ#8yeW*G^vHFzU^gJ69o}@BD-5w}qqAAr#Vi6|9Oc(QW+*T#A&t9x+%PAkMeAj^V1$`$eXu zO9|F{&Pt}pz6gwtcjcB}AVPb4Sl1T;_yl4;5DvrT2xjHTR`FbZieVXS?P}=sd&LVm8gQlAF@<-?69d$0%xC#nc zIqFM>w;u}KQNCFe^dMreXs6C=l_gK}jlxitAQqJjowz5_ZBt2nxr+*dnK}LoC2vjt zEAH<_DeSbHA}yt_)V{L6vqW7#KHRne(}T{;+@yW(Vh;rBPlW>;O{jN8T+kl45Qgg?B8o78a^OuALo5w^Q8xW;j zB;knv1IW$dvoObB{oW1P^XmlIuW%N#aBGJfG&NLN6|#~po}bK8-6wM4=g6?10_6%W;-ENA=B1 zSkmx!5G{sSoRE^b0ly9*GX~N)AtvRh`%U^YfxkS zTmKMkE!K8tsu0&MDd*9-+gE|&t%L|~(6l&k$wR|unEXGm|zCx>sg z27vvV;;yJp)dRYXIHQnCS{1A>srYzZBF!>uG*tX z3#`K^^mLbhno?bdJh^r-`#xj^MSnYXifPpKM&8{E7rz{HUZ*z<2QshGGCZOK;}Ob2 z#|)u~5CL1Ivb77^CjVw85rRC5mv9A>z1Hh$ORo0gmIed*@@EL{_c3^|nLJ8)fs#eM zG3g$Mj373Z0Nr4w^szX7-irVi!Umqr*AMk8HXdG9;b(1p^y8uzntmEIwli>l`U$;E z`K01mL`IA1=yvbP9jSp~cl?rujBqyTS}p>wPJ<0?uMCO+l5J zkAkSGecmU4HVRt+9&V0cxN(=K#N<_iFqFeN%CJs^d!my}cjrr5cC6b2zzqgd>2?RP z3b~5DvF2oN;wo=}NE~wMjq=lubI7mN>#cCB#M2zQr=A&M<%TljMU+rT>w$5;zXs~5 zHiLGgjWE|>s7|Ni&`*E2Ey4(qpvqoJNe(`3j~>BotK-$Fx4P^6RC@tb^pU>uNBfeY zmVBwx_OOy)r7v2z&+Hm1)fWtu-ef!x$UlKMU$re<@8Hb*s9ah3&x~pNrWw?gzwm2f zWe!Xk9(1WQ|DX1rYkzBfOYK42F6{_g1Rp)K20jZ?o}FdwjpKY%@nzKb#Sx-$O0 z1E&hgBZim{!GfLjA-P_Q2t173_h}2l>J9mM?5m4Q#k8wcOmq@Tv3a;$>7|;9iIOwQ zD|)(&w2SdCdWgwiV^7Q4$Fx}T^YWo8cRRU2altN%NHu{g%0`_qz&`TPOt_^{D8s%l zK&23}%8Q-u_r5Qu{8iH_W-_cV{hFHZeJj{VAHzf6bkTI+a7MTynhn%*$1UI8zTXrx zwnxf$vlIvAZ7D5#1e{YRQGRv+>b(yw4hz0dWXW&uDz}Uf`Ot^EmpSy;jC}Rx#5QZS zmTqRP@U2;@ z?o`aHo|P1m{Z*w36eE=SP%+bhzD()@*Qma{J73MQK!$DoGygOtAGJ`n%iHj@c(u-s zH_qjCSHrl?_x26(pGt{K3*`@Cr|d)5%JU5QbxgbP|7{f7^c7?yJE9dbD98Pi*I$v!%EC_b zbSJw%BWiwVYZ9oVc6{jfzSSDnUm%>7w~!Kb1}Mljt-Cc& zl@QVWhZQ{_+qkEXX?oR1y5eXmSL(f;zU7qX9Bv{eUk=3p$Ko5b=Frin|7l}2=%~hc z?2Jd@{)XWRtZI#aFIl2z91iQfFJ*vGZ6~k4btw7bsjIPVA9Kn+UxDJoxldh^Mq$_D z@bFn&;P;a*N|@79Yzv?eXH=Mw4BpHEwYbh~4?QZG4 zgSUd^_^oRHpdC)!XuS8ImQxz6jAtslB&AJ7T>O=3-&kMa*}+mZB38g&z{-K9TDy$T zsW&PwHY`oPd=m@E0K~>&vSOT_A#=^jdkfCf2SmUqFS|H7?|U;t`0sqmM{6;JaZ;hu zK9B|cZCd%%GY8ZCtD?xaR^wiE-VKy=LIv`tZpDV|`!o%t%))ur`=F)Gqwe#u+ZhY; zT^;@lDz^=u;LFhJDcQk#7i#~qc&z2^Y#KeWTz_5L`-K5y!Ea-nf2Qf%2|6ypGqRo= z>*FXnDu-vvDiM>F6rUXC%k5=4IU=X5xOa}aZcPjS(6)E~7A+VU@S2^CHl3L~kAbs^ zC%sQLuudm}R@VQL3`dAO2u`UwvijtEN|QkFP;EfiM0u;FX_iRr2mcy}lJJ{A!u7m< z2{|Y(xlTOi*l4-h0G*9#!pWNAgwTaP`+Ve3n5#D*zFOZT#u~3^E9h3dj{%%bUh^eM zbqK!C?F;>!5N;=%vu)F#NcvdT(}#)lQGD zIG{Nn|2N78P*D8$cozi_a#y6HEBj#vb#80%%G|1bdBETcP%AM82M3J{DEvH zkt7;zSYc+`)d2{|`!pwuah&?>2E}?f;HN4j2E`;VGm%X4btL{SdSRh65ktrXXWzw7 zt+eW77gytadiFl2a|yk%gVzD8yM3s1PFQJFjC6iXFamK~=j2Lk(kW{r>i2`Or2!m< zaod{f?b+HRMIQ9kP7kQY0$dDHjGM!#Z>w{j+>YM~M_R>7vmhEzMd80Rp-!)2=m|B` zum4_vjF-K!hWTWZJ>Zk9XHE7nj|2nVJp2ZJ+RI*%2ajjRw9wj~NXuDm%)C?G9fG=m zl)AM?ej2Npt{!>ZVWNi>Mi+bda44zwXCgt=Q<1KpUmT4KnS4ke?6k*2L=1J_D~OD= zJ~h{8eHB@|H^lI$CqI*>m@y8*Cc_`~6-zWnElin+;_ zQ26YCo{JJb&^vurL0|OyrMz-uWa^x~0vQUNbRfSpVtF{rV;!?JrM;?Cy8J%U#-U|I zHFdhP<6~gfqQ{R-Dh7qc>cf3WUTfYF+EM00}1YJqbMbv9CYm9(sZIc3#A%c@Fo*y?-3+f|7u0q;eN$c?TW(qN`kwYceu` zlUJJ^BO{XO_H8^g(FK^!m3dYJ?h!KtejAtxSi6OcJZ3&TJUnd%Z6wGElu3&j6O!zn zuskeJ!5jmf?0mEo@TDS3V~0XP^*M zfn9!(`{clyg%xSdHiz~#ZS8knsF!nszg+XDw+$H}`XfuzK8ZP9E$uD6i?JPvRVtkw zEkLJ&TzXPjTNC5#YI>SrOf1GQH~!Ui1}{ZyHw)(b0yz`jbpkER6)N5xgXKUJMv201Blwh-XQhHYWBUb} zFO2=PM2}s?AB#<-Bi8(^%Hp0xIh(C6>G1CID1BrRds7I%L*>`9aDWvcb<(JgpZmq< zB=l7A7qxyw3wzv9TmYLS^Qphp`U((48(e%+jy$?Q>hJ*&R?8wqzQi*T*gM!Sb6`BR&IrK|DVCWmLsx0DMla3XNIwB*!J7$tqFUWyY{39*&c2CQOht zhrv|sfjot${-x=WWT)b1<;12z3KVgDcSri$OgKJ|OcQ);6M+|3vFUKfLh_0}@ZAGY9cC9q* zO{NdsArt)48~}@tRmC*QwB?L*y}R2FuBEeyyrl^%-X)!Jp8Ht5>SFis<1%JnNjY-)MVq@Sjg%=310}hBX85wOHrki+?}o^@ z2tEVJVB|e?p4L{%47hpsl#(w%F^lycs57cnRsqOA-h$@;EzN0rfd>mXr}|5nq`Q@L z^$uf)0XFfAObombG$_)>fCxlY=nvTv6xoPysO!$3$I4oj{A8>@T$YW)4ST-_ ziu8l(J9PgNxc)M^f?t9gsN(Uy1=L8evnK<#ed*8sdGrcoK$k-YU|pavdl`WrEJ0rK z#8sAuM^o%1p(;!V4+2pEe9k_cztq1~e|l-3m%vDKRq(_YtsRnZeR`ok=nt&Mf#V@# z>u{k)3K1yg`S2)(li|}a1H#jDwr>fP70(q9`et;?D)p-}SvdX$guzE)-8ltA(Ihk3 zM2dd+P8Cockx}?t1JcE3ed86Y<>%|``+YyA7(`i&{?~DFmjIbXJn$06iffxL>LiQ| z`Iige5DSW;7qdVUL&;|nq^Y<35`h4m?p(LE(bIzuRz1%*CV8V?#+tAHJOlp4U^4uT zfvy4_gZni8l5_vj8#Uc}+qxQxrQ&3%GgO6oYd46$xHx2y^6b;vIEajMAB6a&iZ^lT z<@@wx@aYr`y6kTn(O~%`3U2sGtXb}m`pIsSDHD|;_SYU={!aZ)tqtDVUv>-lu4;sy z;}b3}Lg+bu7l6Hel(8K^A?USD62%|H0~ipMw!2g0T%bO7Y<)-JixIx6#TpJoGD#6~O%U zSWU<$^3KF)Iv@-M?z2V3!}(R+&?FqdGs0aSc%$FL1BJ$mstxXeewp*lzHwqsW`H7U zGUR{wo7#4~C>XRO1Z@=v0k7J6ya)=Kl;v@z3^CNgT6P#|KXy?ojU!~aZqO#Ro!6FfC)#Rk!OZ7t9(3C_@8yW-)n-&pdPNQ(Wgq`8 z?WjYN2l4k9sBD+#|M2(%xz~&}a=7$$dPhSeGhJA1>T&g*XF=Uj!C3*Av$fiE&;F}q zV716ZSEw>Kx^=^)^O?`=b6I1vuI^-I}<1Ox}id*sGYN=`Muwg&q?*4 zWwu4>H(S`H-xzv(5kdEa;?aEdskJe`6B+9H5C_?-QogPXr;Tj8vq#o*&eDDt9RdzW z-#U*yCh)RCoj6)T0Zk%!rAF@fSQ^cX7xZ3k^l2G^`}yt8m`%^*>v8@^n-qC=DnLZn zh@(bp*ic6>niq|zl4z*8qgN1FbjM{GPp4V3(i%EkdJieS_kW~e8?0a#{r?$tK#D#) z0~7`ei#W7Ei4h8!`KZz)$OjS&*y%B9Db2Q6J&6ktT(b%+X6u|;04oTHu5B-Tf#v9P z;FFmI1zq7p0q={6X#`s9v|tp*R3(3!(jxS@_uUcT9M55KA(;&O$ZjVt=t>nCGtqWK zN^{I3yl$IKQ%g;KnxG*ddB4LbK&SCcWOT_n-5+ z%~g+A+8Yzdro_AduSm%U4c~9iIxmAg;S11rDegYhA4C9t_5_KlC{jLLRU`BD;pR8H zC|p6dc!e($6XZwR3o3xP6ek-P0BYUJcJg5JxQ#2Jed8{WqEM%cIVl4c69_QZWmFRw zcmQR341phpc<$^h4@jnlG6KkUm3p>MT}ie8b%}oyrD+q9_cmnefDyDN=;T2OJTVY) z^J(oRP&_+tw7Or7#4OMJU0j^V`uWi)qC~(JR&AZiP&5BEOo&Y=+v6aEOZxCuOTI4Y z(cG!6!Ru=CDHHfnd*D**nWEMaLJKxv=0{QJ;XbklPQ=%4Cr_7l;0kHpcbGq8BZ{1Sm431jU+K*RqrL5kKRtMZ_Sqa3IS!SkArvb`Dwxs-uq z;%!|_psr2hQd?K}JzJx}LdQCqEgy2-c){}QaI?theHul^rHO6vMY134+O7b>=0u|MYp-M1(!v6P(-r$|N=GnXvtr6$N53~GgD7{l zC&q0*8r{!L^WgLxya~+6>bpZmP+i=Yw{`dK{XY9BZEF}|FK7jnPm(k1bXLm75_ zETfPdPw@O}4ae@UXw~hm|DNh{mO(OBnM4d#pOz0pfNS3lG243==i@^AA z!1&@Xz*sid+yY<|^e!xbz|D=5eoTG16&4v;E+`(jS2OD2 z3kHVKE8yl+eCaj&b#}J+s8}QIEjtHI6tVdUjWL0%qnllS0;{5nDkof3r$h-`qBQTd$OlL(NO^Kjtxk_=e6yI1CSgGK|0 zicDM#2~Hcu#&ZQ6GxVhk%t<5{Bp-$`KLR-Xq)6tx;J2q#58h`Ni;W`ucffek@1o`HrPWiN8HLmD`z#{g1}#dI>B( zoT`1^Ho*5lx7OzQ2PxbyU|;%OmmL2|CJ*@dWR)*30+#D-5Zxn<;A;SnYBzsNna*e+ zN+U}*re7zgzR5XLYrvF`{q%6Xn#7^<29|)|K@@*s(U5y^%&jJ$@RLA(8aoSAf9zr1 z74ZAM;3FkQh_z>F`q``DXBDI`;S>Chn*aA=e%CNWs$0kT59$hVVLaT=kZETxqTwl# zJ^m*sAzK2^-ZqHZ9i20m5buDj?SH;*-$2NKADh$PkWz&DWGNr-jSC>aeX{VX*A0|T z#|FJk$&cATx3?n?czQcKv4MWt<2vTdFP#aogU|!<@Jmj+20?;R>}rcZY7Q%Z1hx1- zzXAEC<5cc30EcJ?(3WTa9)kz&fRR`nQhXRZf=f(1$~q81nqJwVEa$r_pAdaTJ6E>h z0#=dcJ#7M|Rg z+zH>Y24T)O-Tc60?ap@}2pAi49L?+X?jClY%VB50M8ZA(DFv!D-~k%Hbfp)C*IgJf zj9sKo9|B<-xI>^6h%zTBMTX5~OZgBdFv(qbeRDaHLzC)od}xUL$B!S0mSZZ~Vd3HV z05?F;QH0R|b3ecc#LN@AIQk^EHJpHp_ih||DorL={mUA#h*LM;5Cs!W;69qV4}o5C z{RM_1^sjHyBYkFv>#ZppIq%{RV{*58!_Vu^7*<~uL_R+HAU*mVMW@>KS3Yx)O}JG z>}i&TjF5p-?3510E#j|&4q7C-r6;meyceW*OsmpSE*rpgRiP_Mn|#euq@V+GN8zsv#X`46av#CsQ%B@@44?27PYh1ql+aug>= zVUZ#}5W&xiL4t@(9Fn9*QZ-i^wi%x}^j+f>4upCcu{rG>!c7o@eJ9p74$bm>ffgbU zPJ2J@Ke=^ti>5Y+aDp>i3CNVH-wk@kQ_h=j{%FyIT!m`O;&UXn+_}*WEtP7Dkp0h^ z+{wj)`1ZVc=&2#GHY*3)$mrfqmhVn=TT4=h75Jz>GI@n{xOo%t*>l+LME(yA5X zR8j}-@mBGU1xW#UwrlU;pt@!y@va-07*mgpvF@Ims@MfazU&)me9t;VTU3Ks zn!v0y>X8TzZpXxvIM~=|s7QF3c#R*IIHk8=RJ{fTM<+zG@GyeI4YX(^{<9~gIqBnt z(b-J*!ZfGrqcc?6IU;<0_Y5)~&L;Iz!si@(mkhEC6d~9 zaISBP4jl_od9B07maQBOu{myRSj7A}FNhmG;9v2<56fSr&9;5qWZ;d6S;s>m{#LKA^6(PSLM{O=w< zqX64<$t`1S%J4==$dPNn%=YMT45I?sl z$U9eh8z+G=@4E>lL<@_=!GzMGc^xs1q178~daphB&Mq~Ef%*4dHewb(zV+~DEVy%V zVl7#T69IGZu`(|t1K@8*54sOzUN3{)L_ihs$@)C41D|YGv3$);%WJP$d^kViAf(;d z)2Tbs2b?{>acxRU3O-H}GT_o+0U7p(wuT~x+oXaXW`c)7d<+s)iud+|>e{LoDxbsL zEh;jP_LoOf04e!-orvHPIby=e@=aQ0#bF+EoI~d=y{BLOr`aLpEo##LiT=k#4kdfT z?#r}&M*$UL4Ez&~c^^)96w|I*;g$CY{d*$*zqBWaurU7(MBO0MIt^LX)||U(Ig*iN z8yL$K26HIK9g|KCB1RW&z@IK*VQR79?WH91y2!4Q#HwPU)=fd}9}`&=ENJPI@TM|X zese)g>xG-?tCyqKyOvWgH*N2Ci+y|Td;$WWtjtrMdT1<#QF0k8hP4W)-kU0SoBW!2 zBqsC1yyBU^+a|_{p>&T_m3$#bk*D0%=Vba1@={Aq`taj2dT(riGt@zafIay0U6^j3_bclagmj6M!Ha)J*=>hZ3ZWT z;swcxWV3~%F)37C+1sOp;ET%grox80dY+ZpC8aOL9A+6yi?7aczZ>x~Y}%?4ekWXe zwe9gl;YEti!_fE#_1n=uAA z@%5jFBCLT4)!U`%z6Qu@xV=Ea9pjtWao(@Xh|#zAN%zg}E|#X^Zl&tvG^Lha^;QqmYBbMreG?}PN1q# znhqr7(XX$R$ZysGbb#v<&J%^hW{!>e6Eh=6vLf<{9EHUxkWZu|AM2av%f0;q<0Sk| zz=miA9o^j5UZ;$AyW5Z1S`Yf}S*TPl4(|pt% zpJrav^ULYmjRX9!;uhNa4{|<9#vKKnQhe6-W#n<_dcnnO#EbQTFj4m$?l%x%Icvd& ztai$ej9Ws&6b~`ZbI2)+7Zn#z!S3svbxUDve7qH=1yOD!$)T@a*4Izody$rwHV%8d zgKmZFNu@O_S$FKo)j%~3oeO;zK0lT4AQ%s*34Ijn0E2=KpUO??Js~zPlbrTQWx}qd zk19(mr`oNPKE^)C!-X&TT|kt&LGw8#h@QkBvj%d|($oE@qTjm>KVGx)zut|+d!-UW z$Pa3LiVw{!DoO!iQo0z{?NOv977eeGNfB&JNN8C3=+UF{4=}SmY_2zpjfLf8P*9MX zmX?u9kg~R@>lg`a(njkoz@}T_J{eq=I=p9!+em+xPvl&|IK9b%*fNUHNus1H)VZ#N zr!yc<<&8gs(9Ank(w{9?4a@bPbiJ89`Ok8KZ{7;PidvW2uJ?i!HN*zH?@$&w1k0#3 z*WhK;`u5K#!i-!pa&l58rUW3A3yhzb@wO@q%BlNv5fBj6x=l6)^0F3~R9Yn3$UjSf zqoXSd!mn7*ouf*S`y~zlnJ`l<1}D@b&3PVmv`fRqkBxrxs4=ol^Zja&C-wO7wJS41 zLS^J*_b8yezeSlh_)`XKeb+ns45R!ur>?0L`U-{_FKlh%ZL1iX zQ*hqMjFJfMXEpC;fi=X@yd->v;u+9^EMh7*~P{p#yH*3d4C@3xX%CQCUYX2 z#>}IrQ}B{=t7688pcO{T*$r^m7~c*Mb!c*gc*)CKt}(l8aGIK%&8hLTLi<=&R<^7& zl#{WYS96cObe8!}3v`5FGLeam0>9d#B$6OC66FZKf5?)>rlX^yPXsfBV z=EQy^w)^y!_8bv?jQ8pxDGtd4ZzCT?vDBY5sGkTd98Z-5fMax!k;)^h)J)*U|H)&}r)UrFd&w zQ!F{@mlqCSC`%E#3Y~B#v@kEa?&0vCXWM@dNQlOwvE#!t1{evYHIX@54y#1gAr=-EUbOXai5_oS@A z9f*v4=^o)qaKAQZU~W*NIPW{FpW)~8TSYAOw{M?-j4IXWNT_W~UlAjj0X7O}Xxy$| z#X{!t@)nJG0|z?w<3~kUXm~i5^XOwIkC{_!jCcJsGv}T@$8>A6JrgmtuU;-x-z!+; zP{bz9iLop*et+(^#D9qXYA$BfrQ)8;CmvmWH&WZ)mmR-jD$?`omG4Pp30lQt3JVJx zZV{?a^4(C{bUVxcM1W@tupze#a>ZVsIjPqYb8;#QPRBFJ#fODqZem%vxRkSp4!hzK z5GeWj)+Ljy7c(ajLPzx5=#wm7PCloM83Os|n$^|SN&x+g$~coq_CUBBsD9t4$$h)B zwN*=#B#W$^f2K4tr~U2PnV$UuasNLbX9|*9m-65=F%;%`^kn_u)Lc!!!5j#5-tT9e z_n1@=CId5lVWrd8*H4x2&r~~A`hi&o7iL9x5bY*$+K)ph4`osMA-l8UC(ixubGu|h z?4QXxluQFI&jiYNRc1HcWUP2zQB`4gC5S)UMC0E*k6jmyoOJXOI)L5fG5pA^+E2L? zBbO(5_jV(?h24OU_GW3vhb(md&e(d_Kd&+Jjjjcf$m93G&jOMfZ}X7#J4*v$PvH=0 z3x%to3cN8FGALtk*yLwekay^huV>)jJLQgh{A(BxbE)L@z(Pp@%vvlPmmZ00h&m*gk!p@6Vwe>gfY^H(!me$wTFE?_x}ej zqHuoZ$i?Tt&36R}H}~bA-Q`edatC4TJv2*B%D4)u<467YY9=tin-#By6RY%U7DaOU zhw5u0m@*xqEHz_*(*F2tYVu6LN%cpP?H?c6-O#4zrhP9BZaw*rELK|(lW^xiAqVs^ zqD{@i%Ky|c5-ZSceU_u6jN&z|YCw*Eel z_xB@IIatM(&q*&y<~gTns7PHR-IfiI#k=gIV$(S@sv@22_akVVpM>fr`Pv$7iAf)s zn2UXG!tn5GaX(=g>1GmY07n~Em%JOvR( z#WQzYshyZVfr0A;7Z=yY_mAZ9@$uNbj2)p0b%yg0-j{FsVMXxjEzO?Xj}{H-NuJbH zR7b_FnX*Tgo0GXsLqn_NGtb_{(a|oBG;L?M+3H+VbRIsd=AjAAwu_3wGdGiREXvnf z3a-(}*5&KbWXnE?hd3f3+XwxxWFINaMDC^g_pyT&6U!|-e+oPc_` z5#nx)^iat+yMM3pqrK5<4P*_kM_S5s_Deg0pKx$YD)yI(*EjW8Mn_8Y)f!|i)_p2? z{`v_S#Y)gzqyuADfy#a~BT#&vaAC7No;w zGgdaGf>7M3TI zc`aM+u~hVZftAk0UpeC58~8TUgMvQ(0cU@$G7r9guh{-kx#s&C+n0cb@>l7!0xK6@ zFU-syPfWrf4bJ?L*RJES8gXh zK5%W2vzky$2dPRg;C5v;*wH_G@xnyx{!eF42p4Z&$ML&{R>0=NDmpwkNX*D6S(HK~ zBS^y3FSI^NAK-l@UqHAR@Fe2sy{F^kt5=J; zS-$<+yaPlj4l6xI(&qTtF+9~r#(EU`b?Pk&lHGW^v2@Xo zQ8`s2ROY07!zTVmP}vuLSXR{ZG&hkw4u!A~+&(@_vn z;RO}zqhFoA-|~ArrMD;i*f~D8(7D&;yL~cP-FA<$4AyFXs1+^lU*j}|-Q4i!*}M+* zb2A-ut#4yqjD+)Up$AJ9wlph7%ndx_#4|$Or%|VEc0;Av*)d=HtaJhqYgooXTSb!w zKJjwJK`8A)p>VG7sByg~K=jcB-%zuRgGXP+u8SlszSKtLkbuBInqt}ho@3~|pQmYQxsD|b2M?2@S72}&Ry3sauln~L4r!TsIKYdsy=q_U~h);8gIRY zv>HydT}uz~qTJ6I-+mKmitt<4(AIr96dT`ZV`KtOLp(^?t2O~sFRa%-TuNW4WciHK&6paPbL_!#*Cprj?!{tSWARvI+YmslWDdvkfRNiOr8Tc-Pt>iev4 zbpAK@T)4OS_U|g!76};ET8Gaq-WK4W2%Sm&v~$yG(v_>K{0M8V*D)ddx@74G4h>M> z(Ruv%Hnih5P5WKOL(inpx>tZDLwetxUG=^42=D#;li{Q&ulPx0Vtur=mul_{Bkb=L`bfL@*@5A8K@XVQmgOL;M97RflTDz`q@22i7 zz9!FTw0faYTNE8R_qp-SnZ5v?0qgJS5QV_`MPa&=#rG)C$LKYS%l_;=7n<|@_?t_Y zS;gxl9zQu#(x?lU=Jh@t89`P-SY9HsX9*S ztZsw<4TOVUcCocB#J|WS7i508{k0`Dt&Ze=o_nEh|GnYh!=l?YeJ!!y#s?UPx6(+d z?nhmC@PXNPMOoB6`Q^6;^uGMrE1nc5k7PqWy;xygJh(wgwASkKWo%uC;+MGyrgr=& zuhG!vOqO`njmKpxRkquvuDc6DuKVXTim#j=d(D{Muy5HOB8z4@Shc%kcu)e@;`}l_Pv5{)5&G4@b8OxRuTN$-kN2$1t#y0Xy(ztoZ~6r80y1~Eo5Tyi z@l?~cfpvu52;mxO$_*Z&>n?p!9UVE)=NcYYi)9=<%0$i2F@3X>!FROxak%F^vEWtw z{$hgF9+AD?kkm`+TD^~}Z5({)5va1iWn|@ccw)4&DdMSZp~s@3h|w)Gms*$0=+P$u zRkbqlSOjI$4cik>ZB@yz32{~BF{%kwzCd$Kouqx-9!#TvxitERuk`|PjHa#kQcyO#`wiYWv2m|M z6sC{#vC@lsbH=eXCl^E7m(H)v{FHj?sT$_rU?k#@=3lBHgJWcFW)~{@i8oEq+D_zb zV7b&;z9Y>C95y!1V@)3OZ(XmnI4u{MZG;_c7bf`Hv~4RH^ zcX8TxHh4c(+~dac&ME~WBhQ6^?b{qojO|%WOoMSwhl?!VKC~%aQN7pFgtl|M!^?cQ z85x+`+=VE7vi(t(mGNiXKEG9-V#z2`Aj$0l|ztr7OFj^!~YXT4XYT z{yP>20(|EPD!#{6fdORL#Dpe#sHCJ9t5cCbGNjH|U&H>NjR>mAor`|TNWI5>7AVBl zVeTqj2`b7RRaR8vaPzz3EZ6mG&=Oxm zN>Z{c3RekV^PMLTBmVZ5x=vqnX%(j0XY7RWo3Az$B@%8*NUGr7IbCr-JihiOpAf^s z`4ziQI_;s`jRZS>9r9xz&AxC^;h)z(+{@S7UbO9|CiU7_hSc*`nvd1y*+$(1CBhH?08<7T{Ggb`c*r-=C^No`}_-@ ziBmH%p?3MSMk}(Rn$$RTt!zS^A0c8in;&;5={?eF6a{8NIg}#hoT0em{3BoP#fvoV z!K5ft7ipT6F6rMB34Sxnz#1brIC-}BxJw`(1Wml=eF)gTjY-IsSLvHrm}wq|)5c%B z;}0hs(`BWE`ju6VY?)lcvmF^k3f@^8Mh(7ocZK5ygP#YI*dTC<$Wh7 zS@VEo*t9@3)|cxp6ez!bA*?msEq5aemOuARyi@_84`y8&{c7QXKh(+-2ML)?@^~@i zppS2#P*sIPn+7TWJI{Rh@{pOa$ReELJ%6|H>G`wWOVcm^JoDA^MBzkB;SLBAmiPI6P7Ftxq4ssI~K|x)-D2QP}666!*2HV=&ApEz;YvyyE zr;^+D`nqDd@e_nFR1S}dhIqj#NYkYebtFM@u~W$Qv}usL^lbYbyc!a*+ct1Im&{wE zOnWSAYXmqGT=eQrrGK^1l$DM$$j94po6RGWKBM#J+yLDin%ac z<6=8|d&oR7xK5)aQdTyE$AfgW@~B9z+*@Q&V~UA0kas9qUz7Rz@#X6(Hxmlt6&ivU zr;R+9Mj0m5`ffK|nQdjL!rU(K4xg8%%J%f^51Z*d+qSBDG9X=o#B$xsAqu? z3kEG%Wtj9af)iwDNC7&A%02b`8K|T3EgJkOQ2t-sAjRxKdU^=&X1ewCwvx~WY2W_) z3=ckrYU&>?5pGi+p3hsV$uC>~*tl8eB}Pf^@S|Bor>32tGGO}6?5WkEJp-s?X;ptQ zM&gOww0gTkmGQb=b=lVKYRD0vOjuJRT?|yJrH69+oF}r5;=`29qh`j|hivQomWU&C zzC4cK9+%o*-Ojo0Yoly{RA$nv@>pVxNcrs36!G=;(N%MuSJ5XMmafLafB-DJ)Pa1oH!c2&= zw4+udO+jHHVt_~sDutZ864Po<^z<|m)H$desw!)8>Au>9(Uc{VKtG1{mV#{PBy=ZPg1$(PKMwJ63wDmMpdZJGVsRg{M=DFd)sqMMiVqr z4T_VJ#bUYdu}b=$MwpF0ZRc>OpEsw9ot6cwd}L}CciD*DnFv-6sR4+`#JjoJe6Mk< zBNL?*GcPashV#OB(vBQOV7VA|o?5!-{m1%G+ZpEzWpe0;=q(O|FHB>{^Rgb#+pGhz=L_9e+@<_S?2j516Vz};OTqB|$AFB+|!7!?r}gE+|!uvus-|l?->Y(?T<2qx| zB_FaB`(@>wsYm?}Kj^5%WJkLP<9Fs4nyfAlHqwZ12}q0N7n7l02Qh_cq%q1*CIus0 z=<3uuTOIv_0)0*f+0$`>moJpt&T%qUglk8J2HMvt&WAKvt&dS<2tMO^ER${x(2MsK ztH7ErH_1fw9}xC7<~oq#&6kglaj19M7%w{TiywFOnUOQST&O!L@^NW8sN-&kG0wCq zDk>ThMG`Ye=K1}cLkQ?Varp#SUwh$ho19gjCELJZe$O|`XxE+H zzM-*dYrAt0%IiyM8PqS8r?`j ziJ@dMuw4aE?1C^`KN-fV41*QQW$GGu8T7&*;Skf8I7>%)CoY?2-D&}0K&e{P<3WWz zfI>g)EgSd5?HkcVM-Qg^8*kibZ+}?I+ILN?tiUZd7mjn8#=X4r{1)1sRgcbg42 zI~hB!_16|kUAXuWqlv+m)SH*M+Hx{+oAqf^9ipEHBH$VA8I>BJIpUDUW2q&W2dzVZJNU#$A z4F;UJ9B^_mUc}<-2rccw{GT|>tH6FKeY~$}fd`DwE;;y8cl%f>0p;e|A%E8@S&UJG z%IJ#&h!fK8e--~Pp8k%sw4|xg$g1qmmG_LnQJBE)H_C9t>FuR{4Icj_N>W zz2M%`Qjl(MZ*LwM8NoFvJYvz9&fZIimjQwKzZceAkKo(wgJVn1)mY@`nr6o)BtY47 z`2`_Jh2F}92n#T>tEL+JzOw0vAyK#b4)PY)g8vdWP=vrS@*Bq^#|XgqfA4ArBxyeN z_&KLlR8%DYYx&zZ=m`^SuF##o_7yZhY8o1M!k!pGQUauRM1qZHCShmkLuL7gV)bA= zYk<(1E%~m{CJ9WTlY9kjv!aI&TS4~?qyV3XkGs1&^wUm3^Ng=bKni(3|M|Mzh&g;T z|5rzwe@)~OThJ(BppS#x@AsMwTDGC2*6X^kN@wptBv~ZTYrLf-R8+{wJC4&4D5eUk z5S6oCOP~f=oLpE~SbPL}e8<5eJpL=M!Wg7(%E#x=_K-UbGzABXOG(j*?ji&}@85#a0*~NMuv!kc1YmIkUt*C zZOjd%u^p)X^jN%4A%`r}Q9PP8-Gj$|*ht-V+&DXVlGDW8EQO$@r3E+>XX1;QqIh7Z zGhRrW5i?b$5`y4`#>Ph80wm+O4rG|fuq(*o<<(=+qRJ}PzH=-3H8W*+FVY%43oq4Sv)C;rHzMal{aRCs=8AhMM5d~qGo zErWw($!{-Da-!QuXwyUTFsVp2C(;k)AY-V)@2Cd=4rd z>9w#Qzxpq*Z-*%JKQ^mfY?Q;6^!afYMunVyVpgmF8OdnAksDP6x&XDipm(}dY`*t3 z6qVZbKl0*MgrYht@dHXFd5lg-nzg$1P?{C#I&;Eduj{e{r_{@Pbi{$gtT>5=YU`ge zv=oJA4#+l;ao(L#_Fz>h?Y$-@&YxV+2|Nlg zxGPzAdmb-(L0K4(CMavb2vU%Ls%OeM#sRWb7l9+{n~-0C{N<>QqS_pOdLJnL*>!bZhrnGK!S}G-cqr!jh7b zy1@vvNMOpuUpgXf=ifS_jniH5VB+w|IZOQdKC&}>fH5y|kYjPMF{DEy7gWEfpXe*@ zzfn@g#k#3r*97`hHd)y(UlnKlubhzjvp7m6K&nccOur;}Z1&)>Yq`i7(+1)wL0Swz zsuXt`8Uh<#ia1TD6$-h76E!`ZOhu{Im`l^`npr88Ppd8Cs8<>`1qEI?#1`UNFV2T2 z5(LvkTJsb_`ymqXifE`S^#Zd~wq3qi0SCk?-|F|;)00m9ewh8gG9-T@RdTIQ_+qTya0> zS%}IND>OHk5&)VTx-M~`@x4EO*FzdU?L%$(Nl?B}qRh%T;J^nNRcMh`GC>MgKZi`b z8cz2M0g0X2?i-|7tQ1Aj(i8irGkE%gNFo`7vx( zzqU9ZWn(>@CAbiCNt4u>Z&2a27>KKuw7h;yNF40W=VnM;f5_jX`U8j51W{+!o>rz8 zbsknLUUmLb|59U;V<9Dii0+|s7zO9OLosBK$;_sh%jnk*YRh(zts4%X`hUMrF*wsA zv!rmOzCDM?N0Z-P7-Wm~NIXr4l-t&4?^`d! zX{cYzRMeX*#{{aE#m2$*Hn4Eq*w@H>NGn2vVYx`&i|B;^`nr7|JY|AE@s0y#-ckTj zDJ=G&o(9ekGNh0}Kszg;GJO|o)xl*KR$68DE~Z6Rt{OKOUO}daZ`Ktadn+*tZ$Uuk z<+~*i%xrsc9DHGxmEGkZP_`n_81JD`dWK=TuiATy828hCLr+jHH&0!Lw2SuKO11lhX#$ z-!x+m|Ai{Yuw?^k`@G-UGXO=fUF#m){e3K|AbJv$2kvo^2zq$bz)`^bf>RGiUx~%V zPSI3BeNETcKKa9c)z%M{Km`7GB?AY+SD9l38$l`v%1<-^@CY=^J|=Xbh#v_w_T2JZ zQ?%-4ZL^PR_z#AHhtB~w*Gpo#-VqKva=1jR0`fL%coQ;!9>!t2NCeCNgkFNuZ!~6< zkn{adMF_xT9k`C3U}8TkxG`5FZ*ww&@aOTO+~OzppOM4iHK1Y=yKvp*n7giAOwve+ z^nZ|@y?(3CWb%6T@vD`}iAWgLsqT&9WE;Ox2zrNe>%Bna;~R{2SuB=}^w(z&!aa0;qrt!a><@+RHe=%0@s(*H%61K0QG^`UwZ}!& zTuCIPq#*zgK?ihYwk_*lxe{lfG9f!VyA?>TJ-odQy%Y+VA!;`cl;U}iO#=E3<#2pT zZ&LDz9zjI=hCg`^^aJ1Pxc+`s;hnv`3o<&vCW+}NWH;f51@%noMLlp>A!rsW(KY7S z12Q3Ir+60s&=nIy%Qc&`8tCRgYso0NLn+8*NgMvb{=huq zApQNA-RD3yh`c*EpsG!Ixt>$Fa6J(n<=Y>@0_flsM-m>e_;<3hvc}ox@4``(M53>3 zwkU#wLVb*AL^Om$EM^^6d2x7 zJl&zxJyFp~f)b)*^;r24{Nia^fYF%GFIuqc$q zUcHY!Susozj?Fc|x{7e$=jBz*q!OB1SxG22scan?F~TEEqPy!Gr6;XX zMyn*9O;atFm#8KP%(-8@SBl z!aojfX_fmt}- zca-_Np2Mk}?>^b+oJEoQ1qL~h;Xjb_8I8u#l2tQ%2nAO@@cJ6Ac0}d2?QmNs;I^tT zI9)^Mm%N$z^`EOrq=UxRYUA-r9fGB5^{+S<*%@KX4R7(xj2)1?pkpKiaQ1Da9E^@? z4A@DCtXk#Mb2p=tPFf^tW$YgvU}hz=(!BFqdg|1_<@njb5 zc)vFfqCjz6uQrfQC3N`DUe+uMjhT#2X%7nI0+|u&hzU88Z(ARWr43VJeG}CpVb}>8 zg8O+NBuAZ8Pntoa3-r%~M?^p)Tyf+oi7E4zvi>O$>+7hrE8uvdE!y7`u!ItwBMzlA^cr(-(9~O_Tjj4EWW^8i5`45 z$IjDGTgzq2mW*RnTg&10TKYtmNNB`Op=aV;O9tzgGws$A#dvpnk5e?}G>)5$rRjH1 zUTHir^G-?jfQXV9InM41kz+BnQM4z1>&(MO6E`80oJaEX1xDW zHxe*dp$X$HKhZz(yLYD6{11+P9()4%=t<;&6<&E%M905Mi%60iiI_mP(%(vPjYmiT5kS(lujp3ra}v{he&fxS2){`>dz%Ogb2$||7_MfUil;X9$ zjNr{Ilbrzoj1R82&3plSNgf{chKl=6c){iR#W8!N>iNg&9m9dW`P23Is+8|5U#iWjiQIqJeh;`_LQO9o zXVmFx9~ z;?SS9J)7_Ttl|3ndqn*WQ8u_B1g>ooU0}0hwXm>2sF7sUZ=VF^lU$zY$R{nIgbwfB z7q1n^Ei{f-spOPd(_i`MXYn;#ZjR-g1TmOm9`=)1d|Bmv-)5IpWnzb@ZUf17StlE<#Btc@#k?kS{5{8~w;u;rywp)@{1) z64lhkX}pD=n-?z0-FPt0Ie%s0HJR+xzV1*Xw*6*C$inB`P^z^m@0mBR*3L3+NIsM~ za2?pU~?74xy{jx1VEkpvvA#CPqadK)-kn@!%wrhisn$UAAg6INe;A-BT-PbS-D>8rOzsdW` zd*KcsZyLi%h4)IZGr4oaHv6bRr*QW26VhLEPQy3K%K6QK7d(YS7tpi(lT3De6YsQ% z`{SI!1oe0YU)L}VYdc$~Q1{#&GaOB-J@R+_SId`-xOePy2@*wR& z#iS2T(;J7^_gaq9l&XB26&v```8ftMwYLTIs6~PT8JSQ=Ck$R)ykKX}KHfMT9!clj z@w35xN`6M+z}Jzl^@}uC^r%O`OJWN@yOW(YOySq@=JlQ`s(7#6Qj3n9=p^5tN@L~Q z;(EkY*m3#EKuaM*I-$kgq78QXDDkM5ftI}Y=^`DVop20#;7<{x^n6JE#u`LXsp=ST zPO-N|L`5FNXu6>D^U#9;1lH51RFdRhav_!)dtK>72n`kzrxnYyA(k=xSU)k^!@eJxKvnH`|$E;$l8VWy8(g$8vk;mm+r$ z=N2mz05H>F@=0UQm%YEVv|UYh@|l%osSU@7&Z0{TfO%DIly7(WjA~zmJ#Hep#7cLyT;NfDokD5o2f*4W} zr#xQEYe|*ss*Q^LOXN@8>#?(HP1)ODaYf44-+!_Fj(Os+?^vx1HH2^&)+oX3YeJ2u zMGtbJ!n(FJNQGjh`1RtDfj)SdoA>g8$tmS_rR@tSTZ>7FSK+^eCgYVB9X{MRTqi@B z#!6_4-uWCmLQ=CTo;m7R5YM5g(BArCy0Hh_6tZ|<1`L#1sBZ54T(VI9Hlq7Iqn6U` z%e4em7-z&0O1;oo7>(oCBITVB^H}`mR9!MP6Y-HeUQ~8R@pU>K3z3GYcFqIuoky-& za*S&%(q(>M(}*ftxO%Af=XX(M~fe@LyAz zPr`3eH??zjPtZsc&Ru=~k$q3TwGXjF;9`O(9E_+)Cj!b+p6+mU(a_;*$h@UR@)opp6b$#L*cU@$a``?m7ml(MVq3UxSi2RBuBpco^#9}X)RXQsVJ6}vR z(i_%xhhtFLhMkLxjh~+w&VBKl-!qbNSsBlsJ)7#Ccq;(f;^B~nK}HOQ;pgufp)L^* zIuK_J+|b+1WWg$~evi|kB=q@#1o7oexP@HB6Bk>yQ;a8lp0YAZj{ejS&ux<03FWK}~dDULCkMmKx4%!24lUF7oR5M~F7Lj`xtXF!U zohX^2l%IIbV`?0$=2m$eFUetNq6+Wl8 z%*??$aq$awJ?Onoo;qe#Uf%*aJO@1)fV}kdhXwrf3=BrSh>`@M7;OgD*=$-mI

Q zrUITG!_=^-Xb2uD`9J{h0AKTc)uD!_G`;Ln4Ywj!H-^B3WBa>-!)s8rY+7davaGD7 z%C29LW`Cj7^`7ZaHyD z<(ruqIv-bQuHxS}2#RA}M+~y52g2?rv(x!Z-u$H`pxE=B_&p0aZxxdeu;iL93~;z) zW>zLyr0-+MmQ5xlegqP}9mN zJUwnkq@DnBknDC35?+$Zb9^L$A-nHlSM%&Ug&<9Cb8QLcB${=$%6U%W4T(q|W$Dz; zf?Mjj)kS@e`X;JTe{br?TRR0c{4g}@yJaeE5W3*5WjzIu9kytDe(w49}Veg6+x zZvhqM8g-9LC__ny2t$L2q(}~pFn|IQBArT02@KsSB@!YfB_-V{9V(%8w{#5M{GZW# z@ArNG-&(U=1D8ARJn#F&Is5Fr4?;qf6R>J$T|uLbEqi-=V3~1JVTa1ONl+!yXvOa8 zPH79^znV@*d*t9n1kpy)a8#-RXljCmb^+As0}Be+!9h3_+nvw`YScr{Y-5W6?l_~o zPY0gL@7n?zNV{z*Vn5|B6O~!Fn|?KpD^Ral(l9x`_rmnyHei*#ouQ7 z_Ek%{ksDJB!w#Df@Y!$ z65*6Q=zs*Fd8+acmpHt1C?R@uqS`hJ;AeT*BK8ZIt*xy?UJUd=8WaZTocB-v2c5|W z`1I~s{ki?`IklVm$qz^;hxCFziINOtgbqlCK;?=1YFi3=jx800nFr?>n!bidUXspb~yH>@kj56M9OzWd!+(#lEE9Ai@QhX z27P2*jRjl_wWrs{9ml@oRkqj5_RJo#;^kx;{DJFLhg|myZwT)R!mZh8zQ#9-z73l5JM8Q86NqaqRbBlz)?!5J%?96Z_VXaiw z*1iS3v(G9MpxY1=eoW!q_TJjO)oWI zq^aUAVW&_O}eK(`DTa-gJ%QTx&NSXKt8slEHqKqmxJ7v^hn-I{Z~;oGyQ zox3&q{&j1*(W}Du{yJTs)XqPy&z5mt4SX_~fA;Nk!dbL1oEOEGE!Cp+mt9G|$jKw<)7N#x`=$zw^X!vk_b zjq=Vi7q|SJmNXA9EX(?FXPy|N^pT!IO*M&_jVFYfAYpmL>#{9ydCp%rp9sL;E7gX@s{$9tUF!*HVEAcXG&cahwq&hK4qvz_J>{(q40gF7zAO6X8CQ$h>X_pO zXsGjbUH|+dCACvfHxDYkEud)zIN+)&H&g$#QpN#h{pog7HGkR`lVbU{g6>Lg@3XUC z`U1XzJ*SvhYF>Pp$pYdDfVfl{Q55%yU@w#eM;FC~S^b%b^6&6bkU2`4cyk%UXc9NwRv100E_`j z?By^=imC9lFbg%ilthtf4;SVOc9aaa&IVuXyB2~T6#sGoy!pQD_*jLmog5^YjFpJ$ zW~;|^qiNYS*$LfzD=zp6fw!$=AWD-~+4Fwy&Yk1AXWz+bJzYtS-i_||c6svrMs^0i zI$7MwK`~LK&)pY>U5@CtWN)FA;=amG-xTW45a2? z17WiD6x0{R&J}B2UB2FuemZ&|QBy85<62nY=2^XS^8i{3wa-bd=X&%mtArT2yT$6s z^U~vUbN9H6>X^WbNY5m>%(n?iugk^2;0W9buDBY0M+CGVpff7%kr1#-FCu%wZUoS# zepw3U3;j}3TBY&~1l&wz3p-A2^lYJL}q#>tTE5 zX|=c8;&X%5Zie&j=U{%eP!7}=%^XL_m^NGoir=QG_45*i?H zts}OyHM%Hv4*jy)V?u%n(?VYyT%lbwu26U0w(?uvIaI|xX~|jGZQXFETl1FDzTAo9 zycJ;q1Y|&Gtx)AUakVoJo0SGyJ(H8A-u!E;olUhxS0EmE^gPrw8tGAT0daOq3w)U! zwz_XebC??Eu8^TJWAGb%to~lldXk7IMnE9h$Q@@24}ye6jbpJaOzLgvYk; zBuVo)Nhvm*9eE{(!ke%4xBbjmD%8)X>#;ocrRl`3k0?z&3Uz80;wTzOgAWey8qWH> z4)sebzJyd8tYsX#>&>upaZbF|bMJ5I7{9ee`z7^h-7+&&X36uYm*=9AgId#Ab>2|( z&Q8;eMz5>B$CB{ULz|vptWuzRWQHsY^LvPs4pUz%a`C^!vAYnoJK;8bCyM)F=9pgO zFRN7IQe1Iw>eNm<{9FZIRW{BJa+;0p?13IC}*K}gxH+g+OzZU=`$jyA{rK?1%8 z+5#tV1+A`{qX<%$^E04NL=6DOJy8-CP?5VQNneI4FeymE8?g#|uCyZ{uE6IXnB`d8 z*f>4*MTH=uD%uP4!A`6^`^(dmzh>1qJUTnGfjs2s#l^g2R!T}B$F!!KR$bNTMyyYsybRbD<)JT*w(Kh<8rKKC#wEV_ofO~g z%k7ZR%1x+{27=@;i}$>0fn=Y1&?_PMTIPCuIV2tg2CBHlLW8u+&YQFR$t@qu9;%_+ zoS!1C-*yTC2^URFhtd|?6|&z(&Zas)cU$$Q!56Ve$k(2AL0WHD(ljp;AKb3Ao!=%($SfV5B~OBbrEj;-`{$qSvU4k?;5go-4SmJ4z zz(^r=);b5ZkQdqOM>3tmsyhijpC~&{S3@ab*B=EkE^8G_(to$d#0E&u$Zn*$!mlm=3UP!D?TEqEaosEATAuR zGn&$FFpIB&w>(2TmZM)aan~P9QCOM|MxASY@k7ULwtw@F>ijp#c6iF{kL~b1MNGgM zV2d@Sf%94~C0NRb$v<33e=v>KfU43+qjb}zp0i7;92OMppa9N+1svtWsfBoT0GZ4n z1`fg2+TYt#1*#8nRW-Hwm6hPj^J8`-odGoi6ji)Gbr|@VWgzTv!j1B+X9~))i3!0G zitom4Whxj0z9;a=3;c^Ml zy;&NBFG0`~r*ggq?)o9u;vgUZ!?&Z)o788CA&ZT>2ix_uF5nBwXSgTCq70i%!X^v3uTE#M3Px^8W2XF2e$MvZDQ5~aTYT>{;q_MhIYCQ_ zWa6>+VxQH}R4Af5DoVz`q49vb=_;2X2;VYSHEY?Lcp7NeGih)x42`f*$@)DXl&Fgc z^%l7F1%Z^+=`)Cnc!p!vp3wi%$F*X93qVA_B}4+6ZYBwVu>$Z(jN>CbMM1R8G)w*l zzt|r?evIlLff5q}$}#t+mYvsnZekVRGJzF7!JzPn%?x=qG zAR$vNHa416%04GeEzdlQjDJ{_EJ*=znH_09xWW1)WekUw!?i}D;S{9nr%dngX1pVY zXynM-r2E6+z)NBiGB63&^$b^SuNTZ5WZ4*CM162{8yJfk8{NaJ( zwS?ez&-=KunNXXo`cCho)V2*SXZ1bCuOsd!ZgIEe71^^6@sAP%Z0a+(PeSo)$q^&u zXT^a^qmMtofu7*9MA@WoKa}r{t-MkHAgHTzR)x4YmqQ#GJ&9mCeAoSu$ROrbu2!by zi?inh9Z#%WL$TN#F2{UsAxyl?h97}ujt~|ii(eJ4!fPVfj|wz>O)|NheyV5{yUxxl zuJXT0x&MQ6Ev4epChAbY2~Y# z84ebUBU`H!m>Y!aGs8^t)Yrb2QaCBqivK}MBv??&Cu|fR1C&o-GszRVZ>30l8m;qP zTUUD|=}Tdu(*|7Vyu1^rGavpjMuGid&Sn1f6C`k#uLLzd4wUZ)5Mox?&gwtr?GA3p zQi!<%%wP>($uRl1GSL-^zpA~jugpijjC-!d1DbU{j*0-_dxT<#dJIqj0-=laZ@xmI84$4YCK}D<-yvqdX*K$MY#@rk6Y5b#IxOhH9j8v(fZ@`F$_-H zoN#UMwwW>(bcjcwp+tcUs!Ziq?$Jqoiq%u^rHga1Q z=b6{xa+A*!Z_eSe1GQiUB!z1uU5QLzv6f*Kq?E2FO(W3={-w=^-fH$L~=v}r1H2Dx@WdgK9T#;EiL4Y4lkFQw)xZck5hLU?q#LX zKYi~aPxR1GXP9?X&yj>+!t9+hL8$ef+Y-ga*`dVY>F3IZlWynGAwsQt@AM1Aw}Nvz z`2|c@H~%!GWBP=J0{ZAmK>-widxWS|5}wbzh4!MS=$#V>+eQf+RKc9smFh3m<1dLf zhJlTl`Em9$Vb3!@l*AXfO65}BRt=V17Y^}^H^&0RJF*Ts+Jik?HWrRY^rlG%1o-$G z#rSu4F#!vsE(FVle+lD>OTlm-Z7XxxS9W1! zGZE8q#fuVK2nF(KycNrl7l)2^RUyF?X*M}Hug$4R4@FQaPoSbvF;7$ z#NaeBJ_xI6I-g930er30>JmtQUaqzSNgUGnL`7OV7WYh(^6+7;UHC1-l?UiAWAI2EWR$Wn`^vrRxW}`Fvt0L=FBn@`SkKJY2?2-BEW_x1c!rpqrF9G>n zbz`}{|J-C*Su$E(?pIc7DBK6!o$&qoS*a4_x!ka8pTQH=N+~JnBV?W=c)rb>-F8|@^78B_qD`_?5ow@ z$+pgaKYlnUvfL#%27ePWW$VD@y!w?KoC{x6y)3o{m3=`|x8kPz>zjZ17x!i{mwfYL zU2$LN)_Kg{YCu_flF%EPw_||D&fV?yC}w!}pUOW5Uu_5$w%Pl8m_)a+v9aNbivB4n zDa5!0VkdNfBp3UHF!O;p@zxwbl0or99<-~L&t#gEQHS4W35^7TLjiLs;IiAQvD%l- z3}xzS%+mhGu|UsaN_9kC^9lH?QdBRs`6CuHrLVhYTqWc+tZfc*my#XHC5d{7nc(q; zm_&H@ZpmQ3lKg>8|L}MTJM3Nt!!1lQGG?rIOmIRb_^qCCGq>xB4)2vmPgm9w*Pu<> zZ-;*zG?FLE>#!{Kh7G$@)8cg{=y^X_!MLRqp0U#~ z(dUYX2z1|15#xJ&1L;NE{k>J>-BX=F+~05{6H>`7%BwP^(p+^Ul<4Ya=GLc#mC3TS z>ja52@$1t`hgkQH&tAVe0@e1U6nOfRs%gs1Xf4)bqTl~W8c+||Lx#ERl&y`{wKser znr)@LW0kwtS^s_0SF-$p^0swkZ%X6(7{&Y1@o_Wq(ksvP@dy{MMTcO*nYSKLM8yxC zis^WzjqUKQbR79;EHT#y57bK;g!HJrxt1ufQ`WfBs~Vj6&i7w)e!bD^p8!uh_7KGr zOR5;DJ5F3$&%$&26uKWDD%bL9=NiwoRr=Ev9W465b0WF-b+2mChTD<7)Nr4kT;bwy z+QbE|OU?04O>I?@bO3pK!SKdWZStJ}N=vVC!Rk)eK4iw}{6@i)Xj6LpuBo97z3!ae zJejTD_LyrsXXEz6bFX>lzFfmGuz1~y; z3NS}!rgozkMMwA}m1mK`Oi6}LnwTHYXrrdorpYAy_ zVnvGk>fND)Ky+OKGP-(%hi=qdG-!-M(m>PtEMb>0QpIDY|K2=3w2_rm8JSV53^%+R zRhOCPucFB0=(RITGljjX!g}F9{B}N8?Fb>zUQYL#P$@L)SI6~rg?PHImsWMZs~{cd z8IvHI`ZQIAV&0qIRs_+ygAv$;Jw$}p%kDkCbw<@!LmTC5*xmWw*G?&0VM!?cTU^0$ zjmi^*xQxi?P@~hiQNl)xIFc8i{%n^^rM{(8HYJ4nM=?dreJHhYVcSNOSu=q;fl;)f zX2#gHl=}QT30|a%K3n=(0Mdg+cU))RN8qw~8nc8WG$8{T&Vg_8m($XfEM{lVHRgb8KXlJ~qKm;N+c`TyHdgp@0YpdcHaODgY8qWYehxcM1FN4*Eb$Spdag}?siOtU<}zaw^oVV&e+5^zgfKPk`XHKLc>{M8R!&l| z444b(jQEnSf$(wK-9uujH)%ACoWl~BdcHAkZ}L2!I+wVo^2vvcQ+KSkImO`{+cTcH zT|%;=4_ti^B09ziyj!c2Yj+Dh#yy;6v(f4Ac=5P4(!dUa>pxqx{d!(!n))HJ%WV;u z{%ke-VvK-3jxP$}`RDwvr_4c5ZhKD_BPEc14rX0%>HxNRIdDL)6;*tUDI@XcMdwJO z5*hqM^ea^g`0@@!kXT(|^+>E3MMjwE_xRrNtdm5HQePbBJYdoza!z`)o}52lrRT-_ zx2bOI=nQte#Bj@bG<&qg-=Ie@zBH@B4`7aK&8!g1t%OymRo>}!w#Hg*&!|Kv!)HFw zoUc*voBsf~$$o|qgkS+as^bpQi16vVT|)fAvEtspqm|9=2{h4DUDYWnAR=B6jp{40 z5xG6-o_1lK_SHSl$y$xad6j48bTu<-0`wZT0ip?>5yEjBuCEtmAbb9nuUQNo$G3Ph zd_3!I3%a|M8hD*{gnr!Quy{7s=CJ&2`KU4+>Zl#A?DXFQrd9^Ko;DF`3P%fK!lJOa zYWbX59AE(ooE%ZX?TO-ts9AG~Eps0>_k@tQ_5ipmOv}$2T7FsaZMJH_k-`XS0%8=A zoj{;0egRy@3qku9HYmP_mfSO0bCfV}KSc0&%FzaFrLt)oLwSoZYS7nVMeKp_n^nVX z#e~Rj8 zYSqaG|9c|iThY^ncvV#rOvn)zi*N zhiijf{5BJ&0RkwI_2Jg!sCJe0+IlPQ4Yb(339I5-cVOE5T42h~UliF+5)WF(v8oW~ zhcw-$;_DVW-+Wl`qBMg;tBh&Wa9j;2nO+w+?1gPe0~xvXT(d7*dYm6NL4|7yVcxD!rW8CXIGhklc|Sj zLzNx9{g<2NWcx`P$A*QoR`56eb^qn(ncB}lh>7p50U;R#RAj1#e zZ5d#yh6ROb-tU*c#dV~**{70_Zm;R}PgH&}oIZb2CSsen5fH2v_SM|+>*_1~jkq^$ z>jHN9?lio)kH2waU=sx2zP7-hXf&(Lrzm%K8?DroeNY9W)QK=H(anm%$Yp-l`J3hS zJN#|84H)03=fqYaytFIK=kYECW)PVqcTS}5prJ_%1s%TX{V23}zByiwk0EMIXV3(! z6QovZff+i1kWsW7AsV|z{!;7bY47Vx=jAxfx-2~e*Q3`{W1+VoxFHd`a+``cQqrJA zu6GF`ZXo`+y7!&SrPCvz2b6(o+>qhxzp~$wQ0+AAD^XN&=-v^2lUqRX%*UDh!Hc;7 zrA}FF78u=|&8gZ!S2IEk($e`vv(!=@+s5IDwDa@hot<^M;5L5(F>{m#p+_z~zO^@g zh5{8HfS7EF6`X6QD=LiOr*0S~Y`#TcC?-bHEcJ944j%7g)8#SA8ChIxxO!|$SW5ZKbpNg>Fe)3=Ep2z-7~aZ1lB6FtgKRQ ztc#JWY^3*_*3~!Ztq6(gPKW5}+N!CItB;@gQWZ;cLCe-;%t2|>Aw_N|TKIUWZ;<@kYwoE7GAIIP*HGq;lF z?RImW6jmI-S-`S#feRie9kuwseyJw3$*vzni1;%;$cvkJ>e}X1(3`Xxp>+WaI zfFr{6HM)Kz7K5qXJx<+7tU&E=QyY?*BxpnsWjfV)dh5vG@I3^O?bCNH`l={E2$sx0 zwT0j>y`IbBsio$=Dn;%?=My1uQV`PC;sI|ZF0ju|fNtyay7T|fLXtt8I8QCdKibF} z`TV`BYOO2vXmj9ZmD4%BPR)`4XodNVLcfPySNzjXt@GZ&01&Sia@{d|0eQAYv?EMOO1g6Y!<4grvyS158d4nf^Fd5%+y zcXx->l!ds^lw)_5Q=gc#ND*^rS9F7BIObqKzOYA^p_}*R@?ZK$AFKV2p-bwSkk`|X zf>DruFxIC{GtcHbe|V{u7)mC=T+1d|(c>gq{a5h)Ry2?c9n&_tF4Inr>Ta(wc1(UDB_wykXmKIY1!_JE|vtw74o?Edd@8VSTcF7qm>dm6%hT1Uv%pz43 zDUY!^WCR&gMOxF}DWgRNpKsUqi&h6nWj26!oRYBq@|{qxhWFjsCB_{%lXXs3-}dD{ zFnM|RJ{qMKxZS`=_`vOG)3x;kHgB~5yH_bI{4*W(oTzUJaV{n?boJq-jE~^;#s0Jn z!lU7M#$D+`CRUQt^JG4lhPGAm@?l1;Kq|U8Fa>T~qY)2E=_6D)FpZ_ON?@<8jz!%937KaoY4Gb}0 z(+8BF&N&&O1j?Jv{AS^C9uPWMi4mjlT#v*PXMU#!ny~{{=2MPNHRD3Ze4&qE-=Wej zHAq4|m3XzuSNG=AE!ucTPJ=@E=Tj7uedZj58|s>!A8fo7a(q?lkwZ{PK`TDHKFVe^ zI(t-Es9(+X7S7ow7 zB$C+zufLAG#o2`+u6(wwS)Vn%_PnCIw1H*T(~SR3?s$pOt7~rw@dv^U*n&NhVnJ9? zL^HE>=Q(8FH(&tvcIhQ0k8wcb>F=Z>v4(Jn08lirCq1%?f7m+fb-s1WQAQ&5Cs<`; zz}R6kb%n>IQ!rD!)i=RE8|c+JsRz5?F-~7vn?r&j7!}fuupLM=Y$ZX@Su3nd`avs( zAKId!N5JbgD|CM!id#7IXvKIFFKyFpJ%k>=p!;#RmdiXL3newTqtAL>x5knXe2X81 z2+57W_vbl)D_LQfi;G-80#fMyiwPK_V=iJoohx?h~`=#B3E z?z5zgXy|56`3yU4g}JXL3T+=tr&+u#{@OTHwB``6$a(qu!%OEuz4y~jm*+zdV`S6Z z*4bDg1Yfs=o%a^nh-eDGcW6vqoApi_YaX5b0?Gc8*Y;RlMubORAsM^dgW?_^v{(LE zNl(*WA6*N*Z_7Bp*zea2qhCPRM*{#w1?WtxRjl7QwcqRbG$p`_!M9Qq&e^*keq*ik^m;y(kYm>w?4^d? zInznDO+s0_B`?2|vm#C?97O2gd(cH`iCvtGuESA-9WSP?j>=K}jVt9}Rt>`U!m;{q zQL4i9EV?EjKxJ$>ekAyRfMY#F$cF59R*EhSD3c!YTgC-gt?*WmZJhsd_GCCTyJ4pyO zOY~sf2#jQ)Hq31|4;E~33b{?Hq(pH~rP=0pa`6ertKgHdP zpIlDcoK8v)uoNo$nJtcr+&XyQSucXYmLXqK) zT&*&nq^1$jxzYf^c^yO(&uYTGXABuFb2Nc+e%?re;`S?FxMW(!_(KRuAVq*UV>nL0 zmpuoGljKxyzcjbUIgo>Sw* zbanOC_9&~bTW%3*GBx)|(`iQqgOK;G5EP-2oBiG_#XJjq#5E%bDQ_3h69oi^3nr$` z*nDXlQ5WF!(2VbQd^L;Ko>IW+HQV0PV`*U@NwI}81e1sr4HAiL0 zOr{QN@LP(vv0`7g`wl9JYy}W=*K;1u-10migpT`&v2HNc1Rb|rMM6Bp4Q2lQS6 zr>-{d<{^MAo?h--w6g@wcz&tVC(xH z&rzP{b=DI~=s~#D^S%otG)BPgvJ7n9lMSx>*1-Fzj|3fwYRkAyJNj6S2__@Weq=SM ziU}DhsIcBIJWiObe#k!&UYoR)uVgmykU$`UMu}>+_Ej&H$YS_IU3QoOa1g6X6?NAJ zvBY4_%W`n>aqUXWmw+T5rK1DELmfG-Oa|!VBF>v*Mz)QI@t)^f)%+%%c=hY~1)Cs& z3DrqzA`?AZ&8%`*R=6wTZ2hg63*=O+9@djpy{_ijPKko{8CE$aU6Ean9?ydCvvkp` z>rtBbm1}-|EV*Oix>6qcKerBU2$mKx!eIz-C4ApU-uTHENiDQ+71SOXFugWv;ElhI z{6tN_I!*l32gHaM5Y(M}(Y$8b9tfm6<3QlG1yb=!-k0Af75Em={V+N!Bkl@JfQerS zOyBx^LSK}c_7E3nm47Co0%_o~l9++>_)A$lF)^9{Lh@gzdslv7zJE5^RAv0wff@AC z`p)%Lus9t!ijAQ*_tbQ%N0wQyjoYR?|3PL7ftcFf!VEOF9s!oC_19PDg`mv-+ItL` zP3(v4;h0Dp%5%;8Hs(wCq8FWlLqHpBP>Z}HDse!RHtFixu=kA}jq9$U-6sy+nt>(| zVALEk&p6(xj?82D_s*!G?#vtGAr91?iIMoEq6i^qfk6qOv5WIry7#HT8PR-A0wB4T z$ph@{fE&m*0`V6h(&znh#D^0f}j*1vxwDkaaE$-D(d-L)l( zHWL-4z&^GnTJb+fXN?~Hew32(b3M{JDu)t2?+jr zb=IX1Z5k81bcnDy-zp*NPME$WI@UhH&t;1`H=@KuMw>qF71&JLk6CJPy;_`Y8p9#` zF~9Eds@o&!7u52XVUBLnCPCO}`z%+{fMYQ>J0N!l} z0V!^c+44I%sJ*y1XqC?Y>IbEZC9VC!H}>KRR1Do1TzU---SXn5vu`1eS{X9f;I}(d z?#T__mxrS%36ecA%v)8HfYa-FCTPoq*#?{kw1AsV;mhyw+in4EV1d`GF^6CQsML{F zDHX&Z#}{e<2Qda8m@m59jIlinWE9CNt)cEp``&v!tVfGc;x4`+Pavu zV=VQsrd0sLtjRDk_GGM^?$MgNO?AuhI>|1OK+#tKjVj`T;7_f%!nkj};JE^maQy4l zVR5V*kak;fH6Fx7(2CD&y@ZfXb-^%AZiobY{Fy0Nk7R`EI`%$_#CD<^VaTV0O?e(q z7jYbQl)e1^x#BJyVOX@f(JL^!IaaJ)`HiRf9Dq48834bU=*2eaddoDW8>B7%Lvy}0 zaJtU*XxvOMZPsJIm()J!@;W^j>R{C`YT&d{Xa783oo$H<`|G;XzbBF@3FpV8ZT_`k_11T|sflEad@iDgt(6RuR9iALU|KF zV%tjGj1NjqO)%R}LXm`w)aW-ylU;u&Dus6^a2d2)Nw-S3=c+mx!DM{QMF^w#xxN}@ zaOhVJ$`aU^o$kM?zB6lE63#XhT8zBk|&Jy&=zr#yxf1 z4L1u7_lw${tt|r0v7AiKDr+s}%CzU(Ww#62wYPP(Y}UU#9Vy$F<6pb{zWL)@p!I3~ zG-t5Pa#)wrkeWWC1eQE}N zLg5g?SCCr((v8C9krAYA%|eFnQ5(0*=FTbJosk9GjkB!Vc2{>4(VfMIH{bb9;f7#< zrk-UWB&mIWdJvq9-##GoqIp+|mV1@)$?Ng5jys#!7V33xEHsr%h~Wr{2hH43hCZcG zdc%QFl6@N(I|bkztiSALUkZ`HGg1bkoR7Hs&TaStV-W%~t7qZ#EhJDw+3AkA6fUxk zT>Y2}Y?qIBTWb2oa)U*2Q(GwQXx;BFvYM)cqhbpLzz%*l8f-!Sa1MOy^fbr!?}acO z^t1a0VExAILzN}`NX}Utaqb>H8M)zyvERQ#NPVJLQ;S#WMfhf{xTj^mokUsC7Z;k^ zjWG)@De~^7WLQ6%lr()-fHvdL?VEM_|5O)v{XirW{;7!nPcF~xYaWQjoG_w5j7B6l zj*1lwl@+^}em$Wx=oXz%2j)S2x$qXs?$EW~B!PO(@b2V+&Sp7k7d)*H!sL5JzL>;yVhhoX9R1UOvCucPOo0DhYJg9I@WvdUWPkSg6+Z@PYf$b>OX}hu#H`n(V%a zI@>aCxSJG`azU;;uTR<@=u~xwfJp(K7S3F)1cU}}2|=m?9?1$UjPN!gnPB?%GUuF| z)JJ-lVNGX!$O?HTZ>QTe@jVzp{{ww8(4ER9m14t>?kJ zhtEx(ur^Gfvc(D9=jhl3lvR02(XT6>B0D1lB=61p9#h_RQt@zvA@Fuv8Ky|2ZuqSU ztt2nPWC(=`v1?}Q$>)7NIVt`3`$YP&2|~zF&Yo5zVj$-)-v9Bbo!Al^4UHc{38WoP zLcFWxE7*g8m?+R6VB6}r7^|ivblbi}MKOJZ;PRyy(HF{x3^$<^OH0j*==#2`YJob8 z?+QKinEwF??kJ*aeS4Z4sJL@Zk#1t?*B)Zv4AdsU(=-*HpATa%f@4=Mxk}reB>;AF zj*?T6UGi0jp@I??p2?D@M8FBaNmEKAk2B@DIY6=j&HS5eRNA0QT6zE_6lrKm1IRG5 zAqBV-Bu&~@3AzF;N5yL&3( zQ2WtcUel0*@0y4?vs6W;-o?gd`9lCRUJ0TxM3;U{egMmX1O?0>zv;P5FT-%S)lE!~ zVOSnmk@RYgbIzXqAf6KvI82w8L z@2`&N*gE;)LrGA;utHK(D+c|7ACCm=C0Es`)C^IHFX0jA0u^gvDY-fUKVOdatb6#7)k;+GW#w|;)u)%h_`Vc;>3 z$<@kn8r+3jEo zM2f54g2&lqfM~uE`IEL7s~w&A75#7e1>}Qwb{_}C5HLgd3f0b%sF4?UtBK$d2XKLH z+=n-ZGy;?KSj?T|p_O9<86xml=yLl`!zC8E0-~86L-|I-Cs<~~^(EK>8`uHC*D3j3 zQYp`?R?^`-wJujJB1;HS;PGya1fu0(%fl}wOmbOt>cr{sgma*SDJ*ZCt*Jt&L$k`ny!;h(NN@Jo~UE0N~ZXCjKtaKdD(Rb8CU2;9~oO zPL6QTc5&8NV%Acj&>xSI`B`8%Id9})p4+u6!kLUxnAqApVMN~oHh7|mDQI};|DWnW z1HpRb9TOo((^8(c37te$SDT)_l;=%P#MpILKhy+*)UEA3#$BDNFFCa(@=W2J8YVT3 zhYmXIn!;&^4en)ylO?nI?w_-#Co(@@o^NGy_V8inM^T@C?`t#+*Rz+`a9>IdIvUTa zeBNNrX{Az~b$NL#?mT&^X0sNi$G@>(#0Kf!fc-`5I^G7eF5W@#r0!>jshv=^$x}VG z_HMmFKW8M<^(g`(IYxtdD+c2$*b@8NZb*yJNACUZ-3Y)6-xzwjn)yYmTsR7Soi6-# zKL-k;+vYn55S?D%4So)#J`;G?iBcqX(~SlBmh$iWG=M`zDfxhol&}>dg_a&cNZshS z+gt-r{&AkHo^G>uiyI1K-&JpJ3oDTh;il}zP6!Al)2E=EZB$V{sPs79>wD{K+8=e(Y&F8v$|ieZh=)V|VD+d{r4_>%3yQs&#J!w# z8;{U|$6>C+S5U;L6>}xD#bcn~uoc7l10yEGgxSRpl3L0S?^PV{SY8GYLr)BiIoIw! zNc0{d{nCR?Y#9+?a99V@&_5-A-c%n@752J4gn`N`b|EtCFc}id(65Z8HpuEbfnTFe zlY%tKQ2!1797Q<^=bDE>xg+ft5~uLok2dX%#JSDOgdEQ(rD=jha4?pS5qxxFplwO_ z9VMPBSNm?BI-=`oaZ|aP9>8Ki(R`E(A%u!t3qOC`ibRdYReM;dy|Qu8qvQLv*w zs%>qRc5MqGPL!?wAvIAbt)a5q_gsCBQa7{vggQ3kyemsO4;8cND7l-hcMZ$e1_AdT zSGkh}S`)*x6>y8;9be8F8?DE#@7ww)!Vm$p&Nq8^=#Gf4zk+r)ea+Jiu8s*~l=f)z zMj&14d+T$Ia2865GMW|W;bFMd>9YPALI0MJ6|NhegP9G&y6sreegrtJt%#m*UzE$K zaOtVNSZUq9G%r3yY#I&vK& z#c0V9aJnakJ_pJnRIkzG5fHh1Vmn>Zm?2fHNbCK%bn$R+K+y`Vxm)i{TZw*JUT>%9 z`z_uk_>18cHQK%$VbEHd_k)h|)?+VX#bPZJ0C%+OVY}1Q2Ox@IndL4RtcQr-hM|(w z_BXh0aw*>!b?v|3c_Q%gRq~w(pfRvS$#NJgEZ~TKw(s4k#7wI{2^OaDmDI@}5?_HfwUBL01 z+<*cIiFt=qYb-JcJ-Bznj3T_RGSGumuUvP63my)f6GM&D$uC8bq-YE_qKp&a6}CMN zH2OrzAG(qtJ6KXDPm%?HrbKFDKlNdnhGZdx?8<`Ft}luJ;vfY8$iL@=9Wi0_t{w4{ zvY>n5(BQ*FXHMu`^z{b?9e|_t4&`Tpzq62`Q^O^&Xx|L5WQJuCtc4Sz@n-ZMq$he6 zMne)YdjKHhn0k}R<=FTt$ABxQoEgRrp~n<&5GAC8IFc1=K6i-JZzTU)okj!jC>^Zb zj4A&=UecefN`~$kN-kv(Yq44*&xCYof{ytWGgR* zL?{ESkJ>kj0t%<@WkE5(5@bFKu4CA(|E3TQ(h0-B*3SOds|vs5{OQ|}8P@hFAAX!59iHx|DE zP&H%FNyw>3>w7GliAN}Cq|*qb{XgJzQ)Mp|f!U$L3yAL-=RCds!+f`Yru5&(B7q45 zKvV@e>X&r(D0<<-k3cs7u}4#5G8`hBCF$k?fsBt%7+kOw1!@3NSj9&1*BONb&aId$ zIt4msl`89T5P;yeoq??&!L_smM4EPr_3B3e2g7hB z#U*t(_gPzGzb81*O1YX&T0u1k4Ak+91q!YGcaY!>?@e%}L}RK)5ODXONHv%9ysP)c z$d#D@93ts&zM9uY_q(K75)t?)cH%4acWyAk!h7Jnq^wLR@GVgQAz?LpQB5o89JuT= z*IV*tO;xAXA=_rQX&OZVL}g`AA`f_UzY*v=dITsM+Q1+mP%j(8|BKoI=7M@jG@z)D z0?&s3tnN~hkdDQ45K#o(KmtUxwjm=Y#aX-gHdthGY{jI;>Fz=bCOf6uq^L;~&WtLT zpPWc3ae@-96R^hY0QLJPpe@Q9qj;~&L(PQm>5~4pIHy!W`z-2HkbFQ@ zYyU$$`F9cRf?|QB;y>%>#sV6aF80p|AHvWsY0%kz`VgR#X3$|0aM1%4(TI?LNL7|8 z;fVVbhaf<^HB;@!Men~{0N({yY3^w$`!Z?;u1N2BL?YigxXOhU!5u+AqKui$`wZYKo2@7{6 z%(Fow!5WMr%2|isKko1i5g)PN{&9au_3THsHuy6*Eh@G;V3~J-(NP0IIjU^_ys(Vc z`{E~xQzjD#ko~&Za#xb)p)Xuf*ij6#qBZ~}Q<|I4Aq2CY$0?vB@x2eW|AbvND`Xjz z1(tn6D!>$pjV%J~l>fc!g~Wo54lchY{3oV!NB50U4BkWRl0gBvcxy$D157U_CvxqlykFyliZGf99@*a)C>ULO}ilv+S_*AoO}g#bg}_EaKx zm*96^({(I1HM@k2{($?u5$pl~Gj!kgfnenCUw(cB4q#tXIzq+V?7ijhiU@d6j|MHk zq<#fRsNxdLhtRPgTG6G@uQPgQ35)P&61t?BV&6pkL#+IF0rG+Bl#I0E=%0e$1o7@M zf-dg>97MWegA#!%ieL|NR#LA=je-P>!jKeov^^V8K)_=YRmA_#M5fy(L~(8(CQyZf zKi?w(WTOkmTq1CxEH1F!*v|oQMNP~n*NQwn0A=1)AD}tZ`(%mt1!+&2s_x(Qa z`>k)SSInnGGkU*Xq-vIZ= z0(sN&4)oTncmIOZJpi0mdbS3RS8BHrPc}~etMw&6|B>=f!SJ85fg%_h&5E~vgQ@{= zqf|gkWiv0}dZw+7j!voB0HaIG8yApHCcU|e!CGt*Y|phdadYv1St;i067e^mXt8}TeD|CibDu`fFp+;^U&w8D_yzrx+Bra2Y*{Ar^JU~ z^r#5%);>C2RFVJ#gOd(=jwtBJnWU|oPPgOEz;QErA%AC+F1GU~fbu|M;m+TqXQ;1$ zMdu-xc4`Hn28Y3EKGsoi7Volsv^@A<#Xyls6|!n9us6i0-mbB2luizh(#*|v<}=vX4Dda)b^cvZbVu={7H3-iABjn!VSCky(|%x%7@~mD2KhhEsCc})WR?06 zLvixJ^gx`&>pSh6;-N~3qJuq&pZEAb-2U&AdJ}5zIT5b}K~GRLOK63Tpze5`_xjC* zs6*W}C-7uxUBbOCfO;het)nsIZE?2GH~Qa1fvmi*F=WwHnh8uE(M9W+ty zX3IX#(!}57U~vDAi6F^F^cUXZ;j@taa%&Be#JO_gF|Escnj=$DnWl>*eOfGCHm1Id ztydxu&*;OW!Y_T$`{$}d%O4R`aG?Hf`-mIc?XDvF}Eo(GTm29b8wv zR0SY%!gb8^hpdHS%khN4H&aQ0#gz|1_BM1Y(c2?t+G7o)!E!EA=6A<)YBo_&%WSm# zz{h!Lotkm)D}m%v3d>sgb?0K+qtmK;P6wZ`aF(N=NI$=Xr$^{_byW|a&g`%J6ezxV zbh@z!It8v1TR+4ZFXkEeOgYnbIBW8KEadin-C8S&$~P-sQC}b7`%o*}nBCYPnP}<6 z(&%BMq_8Tvme@5r4KySV_O5)Zk}}sOW^Z%YKf>Og5V|jt?BRX4L<<``MYH?H=H|bV zEI+y~({|bDHwyQ02Q&;;o^{DTEwamu<#DH%8E~0WDyJ9g`CvpAr~6;PC0Z$iV(Qhu z%euwcKg;^t>t_FqPFC?ylV{pMDYC>op-lx7;dO1+!U&lC(|U4J+)@~d0i4I>0_3~L z)Uhi%1zakEP)PB;hwyxn6r7T76E0*7;-9YF`M|_;1D!)%5xR(fLBT-O zsa8S+CXHR+AkmxXbV>j6$|dpMhc(+Sbz3Vdk}kifPG%E>vqD1h>W$+12a62p>t+{Z zCH>Eh99~(Sw30=7oMJw9jtZ?F{89Ch_;mUHPMseS*X_ap96rOdN$(40vbC?@K8!?u zhXFuY@3Ak8LYjhh;i9IM{)~jKXPUL?+}s(kog&7;lPB+oL4UK{y!P|Ryp&GGbYFGZ z!?Zh<%yNk)F)O8JFQ54q0S0Pq?L)+RZ`Augp^pPw^=S6^|NoV;@dhBwdRr)#W!ELj zZpQC1!iO#3`sEJDhfdY|1vLF?;PdRul!V!W>g8d?4_FDAMcI{I+=C*!4G>a(e<_Ck8~fKl?UaXRjo!p z-rPx%_XFC=#C#ETO+7a3&$v&fz+GFTHyS1`U;bBFJrE8yQUmxOuU&x(rgS2;OmkBt zT*PKj7pxL5qc810dQCImmpeIskKE|fX6|Qjl=TJS7gnl0TnoN1yNqz)k))qEA3TUGv;r(gSrE!@X zS!->CLHn0UH<#}4yu!M>TnO)Y#NWtOFERRW3DOL9zAypU-*S)%to2%1wFdg&ub(|p z;>^I|xb6s`U#K(|3y%Uv5CTzeyPgffdBpNwFMH*g9z;v%i8pKGCPstXQj690-JX-t zdt^No@QD=!ZD%CuaF&!G?lJ5$!(A>ZvhzjQ;`0lQ?Z5TIR0qU`^5Qg{`@W(P1q^ESG$bTzm1B=?Ir_; zW!`;t$Eb4l{O^+RErjJgxNiIyDh`w$0TLfapKwiSI8A!)tMjX89Gc{S?pZ>tI)U-s zqYfdmhCFXO@n&$XKs3;M{oEjx1Xw*C1reYG7PqBcMk>0A;b_YMz$h4%n9HQOGo-;c zO|24PguPKDc^Tkm%PJylvS^!Bv=v767)3+Rg-UvpXYkub@~E|U@c7hI1Bs7f{g8XN z&6n~;u>@x0%e#_sUv1`-&EW5{lfwO}{FBQl#)T9hHY#Q^- z2_FWvNHQM&;x{Yvt9G>*`3A79^r7#6bSJAqtAAHJpJH%q?ltB6$glo6%c7q0$Fwf6 z_?s?2^JULrQlZH1#|oTyn>#rFUNg}nz5x2QNtsi1_V^le=KcdZxr=(c2@O1C?t{(1 z?Q+-SvV04_ar}>7dLsUZIgtta)tOD7SklfG+S0X-4i8#q4;NEMEiB`X;;PNgyf;T- zr9N6)=ya*&8y~Im!_R;oR6HkBT31h<6GgOl9UYB+u%z;D$sbC}4p6oy%^&YVKOs_r zTwSzK*-611sud9xzKd2uvC<5H^4Jw@o~qDWpnCd{LQROZWUZhYn){ma%@-{1H9s0O zflk85b{nJ{cKT|uawfTdQH!BU1x<{Q95__w@396ke772S*8V#5 zF4p(ffgyPv60NS#8GB$_T@UmtH|ZXFb!TbTpFD4nv46?PE_3C5&)B_U!HX_?-L(}e zjmh(=+b6C=hGvpyrYEbL!RC#jaYEALpY7#;cLeN~kB?lKjh3C#etWr--l;s4V`Wm3 zb=PI?8(q!G#p$XWhZ;4K4$#D)d+I@4da!mfF1Ts{Hu-F((>R8&EHgz?#WZOd#flnI zcvFUmug-s)E5}46%%>}%(Luv1LmLnBb5>9s=b#cc%HOLqDl-pC`{BD|^vqVj0y*C6 zOHLV>uDq0+W zZc*}dDCfcQ_Kl=j9Gv<18wBfr@<`U!%f7izR}T%-r(Ax^uNWScS(j^P{8-_q=i=8D z6;a>$y3L+QatTJso+9*?iE$CM)%a&O7y66Rz;pT6*3a?3_fsGD9E0aCj)aF_f#1fW z3k>qY$|bc9u@c5UB>lcY}L`3y9-F1+fIC@G=_OPxnb-picyelRI8{%;uo2gI@*}v5~d?rnY zGOQ$f_s<4KJ@56|8c%G$3p}1@NL!?+>up!BoF+4)0@l7o(QU-IYs+tDxV-5oac}BO3O*>_TU&7MBj!c9{PFJMdbBST z&Uej(HR`NtCwg`B#of#?u3hK-#qqT)C-t$uH=V^5&L=_h=la|B07{0o6WXOGJtD7I zzs3jD7K<$joi9ttT|9%2BLp2SCn7#FoUR1$piUPRgIPKq#u({L-z2a|CsG_!e<(d; z$$Ze)Sk=6jJjUqOSl#pC4(~mE#b3XO;g$~q9`Y_O|MzgoF&j=IK0Z-65sKL$&oGtb%C$L_Yg;%i{u6;n?+tJVb16&v z6vu8B6eeSn6iB2vB48R?B~gSa`(p>xKiC}VA;W848{PLAl_4=78qV>G~Q{Ph5?ar(?R1u z_)I)7jxWn!e+p+lZ=F|=_S_X>+}(x`ZoVHGY;tQb`S4Sol1d4U^+c(m;{^ZL^BB$n z;6q|5u9YOHAlGT$K}U<%iVv}&upbPxdA2b9Kh&o=HxVz$2b`mFUj+NeqjjsWEJH9ZcA?nHUPvoP4zZLb*AU}--fs}zGwqh$@8+%miYAm zfbkqIgiOifhns*Mhe?5Yu)aq0B>Cn;v>q8KN@`pEzd_P2f#|YkS=Ug zioE~;bB&ty`F*Inz10bAu}6!*Cm95P-czIc+y`yY!O=w0_A$bXZuMJ{kjJuDcD3k{ z&G%vGEZEiGAJgL0(?>9-exPYTw%9{JXKuZi<)!EJQ785UsB;g&u`X30y_QEk_}1uk zy}Cd^HuC-!-f;Kew{zVYb!(~w4ZpfknLxU=Kf`rCrGa3P@SgBptd)W^dfog7<)cpJ3z77#+SH7M9+i2^j{Exq#V;T6{*TL8J+;Fw{@~CX4J+2PnkMK(s zv{+Fiir0!J0nAWi3jbYTOqpQpXm7I6L+&(~WM#&=hWr@9& zD6oUxVHgAlgfZCg=w&V8H-dI-j)7F)bUeLS5lA5Qpx3NOVa*ux{3cN4^!wVk6uu#k zY7X|l$mNSd(De{|)(CDJNC=Src)Re~IN(5m%Q9qk101ASa`LjQ>cP*BbzW>94ZzP$ zs~@@oSNo&aYNm9@Bf#tQ5kS=D8SKmY;1pw*uP=BX_VJrm6a?;km+FiW^>}dg>kX&D zqGh!ZkanQy>wx{HT4n}xCF*4pIY72+KhA{OVu9Ee7`RR-W1fyFT!PDQzf4pVvHlWc zeozNBwk^O0W20PFKpS<;ka%9Exer{|Tz$4V2cZ8)j(V&9VBl?x1t4LK>bza6;Sd@{ zY<3>8DF<2rAVrvmai`_H@m}|{ilt{OK>yIfgXE!0A3prmLJ&|Nlrrlfn8~Cc{5AB` zB)Q8VDaeGQJx%Gsl4?=6_GPuC9Ee}F*c+$QeG)G~9l(%ZE5f;@DQkpd}H2s!T}YB}N4d>`dUz(Dh%VSmI~Pi;#G=p&Ccz!T@+ zOZ@Eg;<>(phd(jD^!2rv%VXdAvaA;>L@&oot8kQntw!zq_{)*^hhPjAt`wJgu-7eN(9;h?hEcq13n~l4$9b&S-9Q+uB@~S z5$J^uXG(vfxmNoK(t@Uq&J#E>gf@+MR~o-{>8<)j?mrcTqze_3=mkcuQoj#mONJEXCypNMRq5B7qd46 ztfq17INkx(bnuljp8s63=2&02y~+=pT$QX4Xbdb)JmJH10e*RYgwODGxZF#fY0d+n z$8L~bppC}7M_qYsFNIy3AVxm}Coc_HBrf7jTnjjg;LTzD-}at zcujB*r3x|f(p-!0X2XH`MSY_ok6oKVAM)GrR1SzqmPr(;S3^jJZ2-3mlOn1gdyW8> z29o~vAe}SdVksm!2y2#e{J~21{<0B7sRo`OE$#zE|BDE#kkr*OudUQ^=AAZb=CczV7#gv)|n$9&EMMC2-8AJqwU?oN?Z z4$pb?n{d;<>5gtOVl&hi3ZBA`fWO$Nvc{+u?6r@# zkm{|^N+eVxWSQf?-a}8h$u+*Yj+U++jL`r=ELcU{VFl^KzRB}E48_jDLs0VxlbVYl z@FQV*%eGV3Kqr)uO+!a71wxH_qt&`7NYfCiN6pQ`AuY{ff}Vc#v!N;i>if}e2d}c+ z<-4f6)-d{J9z0l>PZ9gyE6DbL59>W-HLGU%JRqStJorw^tb3-m*y&BR%Vg+<3Gjmy z2TytxtLJ;{|E$jx*|m+lGt1UIxMJ((_!3*J)(`zL)3_9eRf0715<{+ z$SDDdfI2xTrpkus9n@hX^4=JG0Gd*baYuShltd+s$4*j66Lv|o#_eNI;HgCfzPBW; zT(gO$KuA&I4u6zpYfTq9_EJ7)MGWm2d5Zp$F7A41LQ)6zUAH)r#+1Ujg zm~5-uDr^%m$U41swSxE!I7_)!CQo5_o9QN6LhSmGpOCOO-DQuqRKFCoVH2>sq9Le~QsP;Y>&uj8Ubc2#%tTugJ63r9M3RF&K zsPn_weQ+J+*rMe_B^JNy7c|=5)0}HtN|27i$*KJ2rA%Czqe@28Uk>av`@?pLwepnx z&t&dDj;727o3&Z2lzPL%Zz2d1p5j$0w83|YJHtxXoS`8wRxA^_7T zrtY#uru$@?VU~Y_NguVPc0)1XgyE_=MSd$Kg>*`yM=Dv?5ym8Fy>du45kDN((UKFh zcq#H${Co6=!$_PN%lN~1MmIbY$U;beAo6LJx%u=5^0H)(0F;fg`kL-Z&PhSqt&?ax zniU&zu$v4e>O<&#nsseO1T>fl(@uJHGItWgk+9=v;no1K=V&ITkG`EZGB;rWHlLB8 zA@i6z^4m^Db6ix_5o)?ZCJf%;uQ%vr%iO|PmBT|CGeUf%dDD7$nvAk%e1y@eXcZ;| z8)U|%VJGgB!LFb8qxwF&RF(Aok`WP{BDN$uteD257B5Z~a58CwNy?}xZEZ5Ed=F3`dN}&o((kw-tOZ48kaQ++=*B82R8hR$1zq$}m;@9$Mmvk+&h~ zMkArkg{(|o0wPQ#le!bZBVubys>#&SwH6}@I_9LOV^2JoXT>xgqe`v(3XceL%m}pKbyp$%KdeU71u}eunfuwjUoN!F5zl5!4ME8j` zKc}JZPpy)fUypFz-sm&~!#0N+@thY}S^BbE4&;wEg-}qgJ6Al_gwQUTYLPLR{}FwZ zkFmc(xCmOV-p*_p_FX3cZ1U+;BFqY>T}+rmZA+#NH6xRo+A-%xSfu+<9I=Prh<;%dr7`e8fn^xXuYdhqz{rQn#++cXRIg+ zT1p|RK}B`~T-ctkB}dCX%kh&-OMybzE;XT%?GGoYAzd7ICaFdHw5F|7`2=6lx zo(`4greP@!L@5*TbcbFJm-<`R{@;w9diM#YfL8l(O*fF52l8J~UZu%k8E(^nqITK{ z0EDYTZEY~68fvUSA&jORV(N>8kS}^%UPOR!6=g=qtsy#d_fW9`HEMq|jH*kV&`2G~ zPEs314&_&UXLCDsoaHX6qNgr#{|;xYZ1aPsdzu%#>x5-FSP1C`gUU{pXTSSEsVm8V z%fRzN{MW40eGq)I;+`RA;0aW1l^^bx$E&BdLXtCo?$X2`BkmW%-ZU5s-=KOHgS|RO zj@8{~DwssvlDQkU_41R#Il&X5v(9GbB6m^xejK)eovtZU($4UQHfxqBpI~r32;L~F z5OiGM?nGNM?UJ>!=0#%BSSf2N$;eOgmO25Y!HD18X(5!EwL5KwDEh+&x|xX81`2^j zLbpi|5dBa?h$k@po_`Mm0wIt|*}||K=XTw_SY7o8E~RM)R7?7S_%v73_|s zr`AI&nW%)!RG=)`r!Vsq*jgxN?P@yC?DS2VU9IeU^eHIHw6Dl@|F)0=tf)Q^cxuBh zlvZKWEk2eMg(kW6Gd3#3RTr$=$rsH&0!RDu!+zvaLTJC1AJm4YrSOF<3a2xQo8N}y zo;bjiBNpkWMW_^Zp9Dq)-Xwn8`u+6;-T!yv10?dfbcd$^6=e;AuwF+I0CibkYSU_o zF_`&gS2Oj9qby0}(UyM+8xxU+SR26vM^8lB+OvatbBbT;CRE`hZQ=i8XM-G36*(i4 z*9FJ@n|=0ovXFv9eU9SFa%FemAEl z*yJ`QJz>qlhUAm3ACqWTbfeg4rq*XOlVk_?T4j~r+BMh}EIN!Fpp?;lQi(B5>4=q= zYZF|~uICwDP=@5v#x(-}6P+-ED0un)E`d5A_D zhaRH4-vu6sY$ywJg)`bl#DXTFCpC~n=5iGGfZdYSixL|SL#-V7b5i}CrOcg`-T=%n zIsqcwT(+}5aCLKNu(#o#*K{J<3bki6?8Y#BxSOz`MCGVkb>Vj~pE`Ki6;Q%sHk|FF zEt3d(^1|>a_wbpWASd+@6TjpU$%FNaM2SW$9?%FA?_A5Bv<$AQtgM}xjgE-wpEVk( z3{2!=m_M&nJQXTi-59r5k4^V@6}wxPaBNaIw!SaYC4yL z;#X+mV=WOpBY8ojHZ(b6@!KG3D=4Zgn|fvJ$?7!>o`x7o6rC8B7E1RlI=A8H0%BF3 zGQ>bI>qqC3?p5ANm5VL4W~KP)$tKo}qyXAY+*rCB?UNk%c&syFX-gxst@7a>s_CrP zMO}Ea);b*%l#`xs6X7EG|2!c&P#b0+>5#)ymD~xnVF)FB!?)=xf)!ai=01#&BD}Nm zp4C2y8ka-tY^Nh^Dt3s--}>q_&KtqV+oZoLVobDac!UCOUR4#io<`4dkc`TzwH=Em z_J}@OkniX)7Ia+(_7n7{`7$wsg4k=GjruAYIy-t~(>9%%?jFcALbHIn^v znWo-YxWH~{k1EK{Bp?5B<=nhT%kxl0l>ib;iU5T&a_fy0lOgzl-94TnhTN!$42`$8 zfo!O@V;$J)#|yuXq=a-8|B8qu6>?Ic;LDcA(FsK6z|NUNf~uw&lv-X}YaKf90k^6W0E5ta9=ZjeNo77RUPA5YE=BL2B@ z>nuyp2f+&UiNcJIn+x=sb5nKXlWw;`x%4azYRUU-{PKcY@}NBL#GBkK(Uz|>gky3`?&(dZp=3C0}hO@78Db5FOw3;**I_gn*z0SB5 zV@{`Q``~|cRcPRiJk`t`=Bsv*yLbP_nb0Ex{y^eKeaCzDYR+3S@hp#jBRJ_DYTUJx zjpCEz9WawieP@V4;U&*6FBeHw#R-i_qY*oJz6@-p))liAL9fR65HCd{{ipjWa1J@f z6-(sT%UM{p4*fLo8I8dnZM@s%;*`QAgNVb@c0sEaee#0mHTt06c%OCQ5*}*ilI3N~ zoJ%FDx7&Y^JW+}eYR)R!-CgwJJs;uLbB)A4Yqw`l0%Sbb05TKZ=VLIm z<4ckSJ=~j}`aXYQ3luTfz+<@5j0)bNdqY+rFj4ZaQH>9*yPTI@Kqp^}`ae0U%c^1` zrqgPIJZb^d96TC*n1itAokeh!8sBM4%;e*lD{WtRS|r(CV+C0g8q!+XQ;$VgkeK1A z1{FtsR5zsdf>bAkf^@sErRHTX{Ez5pa`)fI{m^@YVXvzsgA_eDb$SGeZR&uY^!%O% zrQ#-Sfjni53?I6Ilou8Lk&^?kP9$z1^>|srsFLHqBHrJxrU0nkB!#TsU`7uiQd40C zes1B`@NUjs9vPOfh_ZH&TxDU&t42MYWX7L&lY>6)AIO{)=vdMQP!te{-a@`l=JGgF zn*$hyTA$tiKfypRA&E+~h0}=$yxq%4kkFh2qmZKj;`u2go>MS@iiLu^OP^<&?I%AS zl#krpv;F|&cDwfrs}L6wzciP9M&I@`j0{!j%cN4tHO=enxtWK@N zyRC=IQNsnbT2vmkKmR6CLG)Qb8BYL+ZxBrO%0c-i)SUwmw;HHXi2S>!-v*1#)51)u!(oFMqu z%WHM4w)~2aBN`CUgHuMGfKKRGRr=4FB7K};9Y}3;mJVf=ml`(JCqoEN z)og%&KXlfUqim*|Dfw9O0^17z+h3l2Dz(nyoP&*eeA^gN!;4?N#$&RDvHy%3RvcpB@xs-M z{Q?#Z_W=m32d=dNK>{*!Fl(tmFfXe(P$WCDd%0>QI$D9kY4%d|emopMs1t^B9~jgm zod@|_txcMO0>8Z!%fIy@jsLDK$ZYX=t)7UKH$RmR5C0!3{6E8IE&SYPad_+F+O)&M z@Q2zV%{$|e&rqkBvT8j)vskTcjB+wl3xk*42r5_$CK>z?rFTGy(R-`LSpek*u{8wZ zZfyZIrfjvOu6G@t6$8Gk$dywD1?z?##qTUVFfXLa&!;&hA|rg3^T^aSR^aAd?yg zQwo>_!5pIm<`~hQXU71{cmGTQqPjDf;5p;nxvJta$hj@|&qeG{Z{h0+{^g%-o{_UC ztl<;s^#V=dx37SKFvZMr$QG1dRpmad;yhMUCWeMo^ z-|7#{n_l?OpN+C)*@xeeeREUU-_zR8dGbS;f1fehI{s`U*GbNYxy;AX)vea^e%YqI zONE1Fwi^V=yJn{Sjg^idng0}HRa)*rZ9MvYyszveH^cSg;6@qGrn8HD-GvZM-%Pvx z`pCx*lMLF`FZ}&ck0+Mv-Wm>cd&2N9D{A6qt z(m&;Qdc*|RsBJPWgU=rGnb&;M1zt~I%c;j6U_z4N(Nj1l3z=?N#@!!F8#6xV>*Xqp`(|&ov?AA1 zOcwb*|9?CQY_SN#jzX$u8`meqC z=Fr}EUTrtl{bceI3&=&f07b} zmp>A}DaB}$;dGnN_D?zUiHi8aH@@{B(~3nd#gv1McYsxG@QdD6`rWSqW}=mKb%WGM zOY~_#-TMPhvrqS;bkE+#rSLr;BBoswXK~e69Bwm}#< z8pI-Kcmqd=cb}b@%SDDls?cUx0H4=FdLDB;!p8fjZgMHY^Rji%1)wbwzpYZ0L;?n4 z+|qaciq9WF4BU)AY&@uR?8o#fUIh4qbmdz?4Ir-#49yHse&N^xO---otxl%o_nJkZ z2e`AP3om$ud4P=lUc0y~7?d+Y4F|FCv#9bK!n_%fqveG(O+;-+Q{SW{!Y-&V`GW4V z3mAifoHQx~1pTkU+_P~3awt#=6lPm5l(cHVgvrE*f1sSbUVXuY00&dM+v4ZI8*9CnoQrm@JXF|g z{0E!igb0pql>Z9kz_RC{0O|oqq$RJuf$nnQQ=Wyk!Y?4%K#l!6fV@NfFs%TVSObYQ z?lMeQ0}L0{KKY0uS|w~!wPdyRh&)xTr)JNAcq2P&0z@SN;}Ubj7&tJoEqiK@KkmMY z3cxO3tJ_W$x}`$LM)cx?$QUtPv3TtMdgL-i27b6f^Yn=P=!H}ilcVhV8gMl>fJhlqJyIznNIgvc!4_ju}eH*KF%aF2& zD3-abWid*Nl9GT1hu{o<`KAhNnR&T^nn)s~;Gz=Vc&e%+D%Q=ZgK2cWPxjJhQlRN% zK~MT5$Lj{|_b`H=BbYuR8OtF5LSOxOmh%wheeZ&IF+aWYoO6ANjvDAh9lkJsSZw?- z`{T8Ta=^ZFAT{#5W{;;gyD33HuGe;Xm%Quws`uxGAwq%9ikJRnE*h|tY}Y_pdpx~O zbVqm;Qs^}HNU=ghKiZ4Urpt7jWXcG7~uU5-vzI!nBe>$(fS;64Rl|;*PS-1K!8x?w=th#!Ui0Z zt0o2oudrAU?046HtSP>)EoLFq<9PQ0HPUZ)iSl8+qR&PaalvlSl;!mCWL1K@=5|$Nr)ttBr9=NS6K))p_(TNYAmDN>Gw+>9YVw)Nm^=e z!hiE_9Y|MHX$m^yk7p+F+7>P)yf}OasvO^k(mX*kpDmFBCD;D3gekGJ0}L+4u}u6n zD0n&bStYyQaiKVmI#s9v6cHV&^o$xgQt9l#^YE@C(JcNEOiurR5C+RcOmkYk)5KT5 z%+QL6jmb}Vp!HT*d8=ideY(ZsrpL!LL_}2MCV#sdqn=HBQG(=AGzRbK#-!djeNGhE ztWfwZ6RuQb#J-LY&G6YNL5hjLw80Qm{rqU~)zd}MQi7CCWp zkBs}QVl^R)a<@yGkt*`;bJI+$u*Zm=dE+sRJHaKkb0m$3#jCYS(G>l@z7}s2eW^-l zHPS6;IF!=LwIebj8&nO>)k#%TL~G^~WL&WmdizXiD*9I=jb?dY0AsE1uiIc&5D>!E z-~AcK1^_8;D&aW_c~XRI_w}TnNM1qVKjKdhcQvN?TAO7~@YIR??v<(0ARTM+hr+1Y zd+9ml*$|%}v_(_Wa&JiHI%{Q07W*RHb8WY7quuL;mIzYiPh1lbevHET1P3~o;WX}d zm$E5E^gE(hf*fsLEd8M@gJzNr^=UTS8OO-%*J&^ULfTl8)9iq&yQ0&N+LE+A?g}N+ z8mIij6tX6RWP~Iw+8eFlHr4PDp)+RLYzw$X-Y<2R7-%1~y?K2d@F}ON%@)(4zXJzm zLnB4cmGwB7^-0AhYVDe5({NRJJ?tDDsdAR?KYI{KKfOuAhR)VdXQ+3So+Eb9O!hwD zO~y_M7S{aM1}?YfLITY%y@s&g+M-;x#XpPP8|c@S7ujCbHTiO!9wBOb^l(J*(Xv<{ z3b}MAscvl8kqk?Nht<(6lW@ux88s?2Zy5m|VyF`bt>@P4axDtWcc!^YwE)#u+=M8ba_ z{aBs0i<72}724uOeG-Z#*nRlAZ1lOx8VZCe3Xvam=37rB^GpdMnzt z_asRBor+qb#Sjj?5jH0SC0FmDNSvwxpmdH3P4&vNPZ?$FGEdo!(6Bv>+Uqa2<8>MLr9Ikt+d0oCm0UVwj2JE@wA{Eu z>_x|{7L`-ps>&%NgH$+qh5VwZ$g=YU{_@q$tHFf)=6?83l}7b`F{0QrFtxpn*Z&xv)!Q z+ju^;Ei;OMMA=FJI_5n5|X1RV$r}5$OwdA=f^C)ur^dvl_C?OVM`*XCPA7qQ z(MOsIV9v4c;)+arEd^^;G&G+IUX>yImjyr}B4e6DurP(PvT~~V9w>>eQC>WARL@HXE_E5`=%L|39>ToWyinL(p$0A#B{R`=j4Db# zJ2^uaJiqPQQa_RjkGY}*Tn2=w&EBAkiPOc7Y_uzSPT{+HuU<6^Uq7hMiL~JRl7~+2 zkq+EvF!jj{m0;APk6B6aoOj ztmzy^t@owqMDIBEx6f#Q`6lj(kF{yAi|Hw1gD;6vY+*w)ZNf`e+U98K=?HrL=-R%U zp9J<2iXe_r>Rj!c;7@_wpI?p=Gti<=WA>!z;E3@|fvXZD;zCq8=Epl*W}m#z-wf*A z&P%oLBTSFA{=quMBG=RjXKG+h7tg>b>Dc_LNfWN!0UfU*IK$}6(vZmG=m?v1L{0(l zL+neo5{fK~k{#&LOE|soW&e?B1JdCac$*!DuhZ%#H*a~muM8~pjuF$~W zPBHAjGA!DHv1#+?ko6k*6A^6CG1hQ)h`(B>JNKPtS>p9+Z7C zE-m8b(E6*(>X`u3=%2dqaEy!n@b*T^SMoOm`1r$l#^+#KzkW>FKL<7~?3C_9v!T^T zSy-h+0)`n7dSRPGMuRXZc=cnJji0{V87ZWc?8jF7C~cq0bJwag~hvd4A=*(oJ>7HAKyfi1VwD zofPrk@0|W2Z9um^)1&K3YYE4R)C8&aRiU!Bc?T_ZXHMZwkI#+;nB;s3e7y+RL5VcI zd8^_TI$7SC;q8sba&yn>A`Wt-AOH50SBo8({T#zAcWD8LgGJxuM&7k#u|?LvYi@zV zupBzP{gl-C$HZ&(e<<~3iI!YG;O(r74NmfezwX5M(hmUU2Fl@y&!61NC>3ijm}NZM zFm&p>J?;{c`})H!i~!=$*QM&&fDyCQYT`2BY@u z-G9SMwbAZTH^){v?Wfs3{Q0Rp_4RjVO_M#+oTM%yQNb*3F{dcxJn&j4bE;{yF>X(c zFyIw-^w1z~`nxv!7nQzCez!#WaP;ehE2tW%pb(kqgpCNyscg@l5-lK%%LC-L%IGFZ z`2Vg8U{LtSfeZ+L=LQKguue>JQ`|*QwR{GNoTGU|1()H?vNCG(TT`RXAN!%%1CCQl z3@@3H*lH7!vTNk71nDz2M&gCQ;KgK8KXWy;L^iUA{?^M&{Y8e#i48bZk&S*aI#a-8 zL@OJ^HinoG_<~UsreBFZ;y{cP0LLjUCWgJO;pS3`Mt|r8H;pr=>=++ksqcRlw6+xR z$!e&6L0%2^DS6zS1@tQi{ENv>V`BJ(8LSdw|ItJ>d?R0)P-^Lt<*~`#o&<$dG`VO3je-hpeTN|rtl1C&8A|fV4M!kOU1>-1#u(9@~{l3(S3qX z)qrirACP)D)(mi3fnYTdMLoT_>2_gn4vsFM@V~F)(P86&@>Wz-tm6?E|Mg72{1bCy z-^tOtsq3JyA}$^tf+ptPaW-i$LDD^^`Ixh|2K3^&YTmFvD~&E=n|xC<=ACAeqMIKL z?+cMZ4)f||nthfpjt6vnS;SYq-iv!|_QmkOjq7I&U>>>dsxH9-hD@6dt0d+LIB#a* z%Fr;V7-w#<8j`?i=&ni&LpeQW0=|zyT2ZpRzjs_&wOuE<3-Ft_MS$=?Hn70#tqxVM z#c+`` zOYPY)mvvgjsj_`8tWNdkvWL#6FPKeg)`(Jz6(Ti9X*#E>SKGc6hu3Tk%x4~ME|Dz_ z)rk(gYiMw5Gp_Z(TROjI^5Q!ke-bGR`Hd@979TDCXQ;5SHMib0K5mHR=g{~Qt#XWt zOIq|h=~@yeh5kuRHPMeCwNWbgN72`W2aY4D&=e{RoglYS?4VW-jM_ENL+UVi%(wx9 zwc`4h3?IE}cHf#&&yw>Y2fa~s&}P?hsR02;hjh(bvz=uDN1nHfO|Jkbza1D*bkPNO zKGJ&5b=e@{r`uhr0*p7`>n!(sfejCbd07enG~3`3#%)((x`o~(adDBu%zz=64w;Z! zABLLmoSCc2PvS5esS%2fqrIx|smS7jN8w_IyCYHe>Amidx$e;5-2pVqlfOr2{(UrR z{DG?CelYye0s3muI?cBdVI!$mQP`A@YT)Y3w5Gh1SD>6pyIO_~;pgy{hTX4(Mm>8H zf|wJ8r~lb4l-?YzhJ#zG9!1WqgSMXT2;5t_144jS0UNFYv_iwzYQ)q?FgZraK3)L_ zhz0BuBneu`j{c%g?S2lv6}4$uX=Xau#x-1<-h&%YnGOpC9QwJ6yP5w`dbB3PIL^Ww zac5O8xUKI-1pXWYBIk4#E)W*gy-QxovV+`X zMducrfP-1V7lO!emPVI^SNw&=CX%)NnT+LjJ8yO=s)kY_)YywV4N4 z*-#UO;)D%G)ImYrTNuWcf}093L5tdT0fX0C_N2ab1@K!$awUv+6!99Fry~5`}h;5j$D>O@2QMR6+>60&fDUBPqt=<1h8}3 zBnHA(ey7qU?|xfVtPv3=nny{j>NE6-MCw$MJoW5&V+3iRH1 z98qUY#%*$&<4WHaf(p!+!R%3Vv5HZ2-@9h;(f%^(&y##_6l0pxKdk_;dO+blTi=X} z|NevrA9V%!!hD4K%Lhmjq#*_qxiC18&==*dyj0kH9Xb~Q#ei_)_1PG!GNR!`ggpQ; zCDOjzZ+*qYfei}9v_vlDzwbiLGw}Ah-N6s)F!u2L*v}vs<;aoKXnOj2-%uvo@p|L$ zveW8W`*elc^#`2#Ii&4Q@tRBT7z=Jo${imRh8?@Uw;6pVc-4ZPC~)#_Y#DX+^!K!n zZ!Eu~9RJ*xjSMzmVK$>v-M?vt*mb(%^}3IGerqhG-ErviRRwedt)NwH%(KcS<&?`Z zSj!6rdu+a8jtW6U)S?w!W93&6dNGB82rS#igo3R{;|)o?c0GCi3WvK=q{uSzQk2;r zPqjzO@^nJ-x4Ofl-10Zna))SVN0W7vxt}VI$SY7=omYz>eQVNVIO`6qgr~#ptPcxk z)J5$EmSC+cSq0&+SX_yW;Ka}~r>W3m0&G9FlKU!?`2nZ*I+^l3_wzD|8j`OsNI!ea z9%RJ5>M1lRH1}rI{s!A<%^>6P*dLQ%+>yJu`_wrq2XDvxMWGZWOVeFOGN(oL%^})R zc^bplvtAxnz;!l#gWt@H)XXPG_0Kb9EqeFS8%=(CJYbn4Ev`EYyL2&kh~ zKsLn>e!m>mx3zu$mAlO=Lk{Z!&OQaPeqwm5=6KsNS#X%1X|^1J^|4U!D@zMS8omss z%|Lg&Tig@0uTA{+FRLzU;u>t8)1)7)3g~3*!g4m~LvHihye1@wi`+XyF{b~o!Ndm* zrdZ>re+QE$ewKmuuJtSCXtqW;ivp8f0oOblOH>AJh&V-XV=U2Ae$tp$pcQU_Qytwl zfzMCcZ@1w8vGpF{RR8b)c=mRzkT_OWC6vr#9;0jxqhuCR$taG!_fDDFiKI{z*_$#G z5^<1KW@Kdj@0WVNKkx7L`(KyKb#m!=o!9gEx}W#`cs%aMO+j<-D!AN`A}&!`Nvz&s z+$n(L-$B!Ntn?HbpZP#A9yS<&?5Inyqq6orK9o2asLPgg$8QG}L^nb`k?piK&yRVc zdDQpt8QB!po`Mh5j4ZZoU4|0i;HUYmDZBy788WjBNFIrd zovjt7x1#iNMABr|g`D$(W-`yCVsZ`lRofm2J&FoxXbgUlS7K!>xyrB!mR4$yusnC3Z#w+#g{Z{i^L^3#SO_NCbyCjjY_Ag0R@t2` zcWkt9QZ>JXifP=FMO)Qg&1X!kt>=lYMbIQkIEy57&YCaiJbpu!5j|vHWwH_`^ogMr z+s*XS>?@nw;b)L~Fwe2N@zN~%N$TF`JvMBCOp6dg`Mv^^-^?Hf9k2!UGftyx|C`W(^SqanOhsL5bCI`pf# zh5F~k`#q{>8fqnMr4wGtTMMl4CbmhVp?@x@ltkTp=BK@&zPNMg^AgWa$4_3i$D{Ag zg_%fyG?r7x;a)l4+??5?9`e7yE4aF!QCxIQShB_QV_RT|d)d!ypN#h|V=-qovUlTO zJM6Zto=g7uW>67YYQGv>)X&l~pep*tR4wGQ^e;d_*B&SaxvV+$YNji+)hYSnNg>uNPL`Zcnw zys!Lln;j|0^8&snmrq79#4kJiK;frY?ke!=4E@?R>!n29x67-u`r4><==|>fhoEfL z3z>JHxKxR`lpS=6BIyI22c*!J-8z3}N57?zY6!rftF#Rkptd-68F@@I%jxuSwtY`& zz=Qf0nu>{eQ$sa*LfOHg*iE4n2CN2>F`H6RQN7+l0Vksj+_pYwAt+Gxm=y|P;$!!G zROklfj~Lj_69Sy>vC!N}!#gr0 zV=*_G7gt0ra2GM3i+#VYcg6O9b`|{;NH+6nFL$?^K50=W;NI<^kRdAC>}IRJ#@6av z-EI#HSG94`WZ^v2Hcckvaa2Je!S{b=Exmp;e_>F5wRod0NH#DoKQKf`x4XoXzFy(& z+wCs1^5~Pz>Q+Bz$$Fn@J9-p-CyLj9bKQWr2osJX{dAfjq=BgQ!((P9^c6Y1wj}9> z#>}kRanIRbvdVJORxZu0D}_?!ixf<4iSqAtDM&(f)n2sD-hE@){T(fMLC{^7K~mn@ zfRBINPa19NzSnhs^!0{_XkO#n$qT7<#U9;eIM*H;%Kzu9P{LncsIU zEI6m-(e zACb{1vf!%|oGV#bSqWo)cW#fA_Y#MKWw9|WIe>G!fvx?5OYcHtZ^4?*%f0jh=c%`w zbk<&jY|hLa#EqxU-ZG-&m4D|Iu8_@wZ5onEr=L`~bAcg_HHxbc1*lcdb3aaAZV=&(Rj zYL(6(OZ47QcrM(;C?`EjUw?f_d{JSfu-FZAspui22^t$=`UR z-W|O`_UV-;>sp|>ek=2e?KJ+L9VlUXvxte2h)&zvT|2b<%$y71^kaCfL1rrIuH| z)~{GJ5W-UJ7LjJ+SVJr46J@=G>!07GdCZOFUsWH-zFcxhTdBR_mSmZ z4)l$OrZ+-V$?2vGT3HGI&DEd?@Jq!os$Uc@cNq7lAY)>Gb-mrvjeYj319(af2~hyG}r6Ar_8K3@lJe_l+3u_uWgh`^uA}!=whnWcO_3bx5}X9t<4uQP$yO; zh9y_ZNp?38Bw(cb#%HI~t&R3QcbE&)_VcSb^>LZw`vWg(_kzNf`jYncdd8Lq@Rudh z^#hJtc1T_o4y2FQT&+nL5vdF0bosKYN_9hJ%=FR9rNPtun2u|nf;+o29tMg_tslmE zQO;tKsjnJGJ>GaoN3x9^^4X`+>%lG6-fGy|m3leFF&+UN?A>oC&yYek9-&n9wM(S_ zF%5MY)1%hLWR7mGTY7epY;TVcjhgv=Fx)Su)4!9#LZN$#gM(9mso!wyjwE-Rai4Ys zCpP-_&ED+Qdnb+WcWf5xB{>lD@1Yq8iAe~4eJ7}|PnV18n5@H4$Jc6msV|cA!))|P zzj;z0C!(uw_JW|)6L2Rse~*%c{k@vLPhIVm#&=Doji})@pZ=Xc4>A2YIs}{|ek}u} zFCD>KausHDx%x$1rtt9so*1&X#RU%#T!R1aAu1@SCspn=6&^;WW8qCoNmr$&OXhvV zq0#2&EHK2uC;F;dJcQVnz4r7f)|-a?yui>iN%x$Is_*B>k4L>eN_Bnf5XTGXrg!96 zh;BKzPzwo{58tB&cO;aPXWs^OyG>TcqJn|+%hR044)%&OPP zW;v9wRTl*@qC-wai#Ou9t8KE;!$?m38pzbNGiiKZqm3l&s0;Li^oUGT@E8s`a912A)@bUmt z%4h=|K5M|AgbM6y2@9j%ArFr>Vb~R%L(oGzidxo94M$%EdR3Acnc=L|$9et%eYJsP zh9}0;evHx)U^v>p&i3_*{dV7l{GdwAG}t0raCHFxXuSPtb??hB@MiyfnSrjZKG6Vc z1q)X`NP#1>!X;141bBx2EcLX!NHXLz8Ce+lj_?D(Qcm z!!18d`hHL;{JWIo^EBK>=9zy)mHA8l3m=C=DWSJOZNSY0S9$N@hX4ROCApp*2x^bf5d>>y-w6x3>{i~-fdG+4X5dF- z8}|vZ_tm98_o;zspDu4!aRm>S9>}Q<+JkWLUQYR3w;`ye$13GA&k9DY(SWY!z|d6# zNhv8-81aV%UTZ$WcY;WjOgDZ5?*B8JD**@b=AV1wcMO^T_`pa_8{D$F!swR4Q>KCg z27B19BktBbD8Hw$@OXMZ6+j|V<*~@@WV*kAQ!KRb1mN>~t7?4qy$j81(Pei1$}qXD z6YHN%;?wK!X}A|%0DmaMrwJ$hho+*HK| z!YGbYDhL+Ta3Ux$&>Un{%uk=eSWbZH2b^CH{r9hhDrO_bMk;TwvlSnlZgi>asOVOr zsJmclzJ2LZ(BRF)i4zgEwY5{cQ_ufA8U}wUnmI6dZ2%NrTP2gMu_fxL5;r&SVC5wm>b;@6;V8rpIAFO-*N7x+L;dE+*l*&#d?7OwrUq;*J7URFUIq*12gT;!kN z{5R(ifw#h|CfR8qH)-M=lI|>upZ9<MM~XhRnVhmIT)&v@{%u*){PuGk_+cUXX8HVc{+y9CcwLPv-hVe%Opi6SaWTF<4C z3g>c{g%k=}Ft;z^q@<(XZu+JW**L>QNe+J0ssQR_M^&=)zk-C59aX6YnwLL+>fk$Q z9U!okUSWHV1U3OWmht+_PB%^ScEiJDiGo zr4(WroV@)ocIVD854c&Dd+*A;0VgemLiH7dz=CPZ3dngJK!&me*%!ZjF&(kwZ_x5* z!xRgC#qz7i{tfK^w0*#x8iCRZv}=FPffZdY`5>|{Pn4CFA?)p+E+B(Ui1-*}+={LK ztnr#-QS2xtHfZ~`nIH^86*T_kk2<#ftg;U&-3`f6kFkV9mj5_1;iZY(=3*Y+0j4?c zfOSs(wMq$aoeH@0>_SB~H3Jmrxk!Qj-!FS)Rrv)kBW`tm#E3}|0oQXXczsM2Nj|!s z`oMF`$OS|ma62+d%XJMu*6x|N@tMMGaw4#K2?7|Ehl{JOiEQt&ycZh+#K2z!Ya5#D zohkh}CUg*gKiMe9!`bmq<_a$|;1AUcBq#+O{EW8_`@B@F*F{A(wlc^?z%VP}+Y0`a zc0XLxIW1)Km+K^s;FH2Aqgs!2gYM<&bv-9F4tnKnF8~LiY~?dFea2*mI{3 zSl35*&TrIj!RY67>D@6joN{kSCbr>DGyrS2@Cy{#`Je`#ay%E$A9Zc4#EQ&&dqMJ2 z!KL;|I?b+&a5LLI(c<*~{p9q^MgVzzOJ!^VuTH>&e+L9pCNaS_3MdTtbH8 zJflBQXtATZESFJEt&@<6fmPCtME1MnIuEuu{R0B%nVGFA@{j0*Nu!Y=En!dQ7GkBH ztz(o=p2wZD<0R`;|39J36EQI!Gp34yjwMBS;%1K=-1u&C=*j5vx>ybBG8KKl@drk%*qE@~XDVl&e1xp8bK7q`Nf0wXT76VA;iOWv`!BsJ zoMQ|Y{_gU$Mgh!P{_jHjg%aD3l>437HC0WZ&zWC{mG9>HRx#@TC-HD#AqUXU2{S^n z`5kEp*GGhD_-~%4T8UI6k=$Am@f$66oK*WT9(%?{=ZR(_Elegqn-yU3$S7pc?F9&? z6R1PmpGw{xjf5SVnnc+Ss1o$10o9aii&qJ@U zP1;!oxbh_=#cc#qdIb{b)J!CG5ly~lZq9`O8q!Cj(Q@I)c`EF#u)GUBCrRqWK6#16 z&I+H3%*HTWmhiAQB|Kh49;N?$g+?RM4p9~$(au&q`|Eiq%xtv9k-s?hPh~yZS*M+0 zin^4}5kmqzJ|h<{DxG#GK<;wErG&dWb$nJG{ue0kfjj{p7ocTL9x zVBU94tr}YvZ1lUx2MRdx@RH_cbDd#0(N5SMsobRBPzwD2o^}we|CUCIJ20M#&gXXj zD+rk%kziCev<(_@Gh+q{RLv^}w*NXXLe%7dP5q$jiXA_E%&-h3+|~xCzzk%?1|+_a zjDlM5(En9Xz*9J)goj99{&DH@^&?{MM^SUDCzaEG2y6U_Pp*E7CQuo-=O!!oXEhkd zG>?sq{e1ZI@}EZ)VSG%Dq@n)odMq7qd+3#$g~Xy{T#pz3J)gyRJ?;Oq1k3X9+oLuG z%*f%$^@ezY2cu_XzSzqobU|fEF3Opkn;W3$N~w47@YM+*C+bkre7t(G6U;NY zypcC`V3JRs^b057=c@JGdc_yl^-w6jB3oYzK~yRVu2sAD6NiBGfya5e!zJy0V?zTS z8(TP%MnQainWvf9r9!S79tdOXnrmJLhJ1bsqb#k8B zx?`Br`|dEaDt>8c$VSkBqt&5rO?q3e%#!xo`?#)Q?dNj;yFKSb5des~Oy;0rFOhiS zB2F)#FLCoSsK&VH8IYQ!F*kfxU(MPuqjeK1K?|C2BOJ4*bB1;FTtBHDxl z5eX@l5)0DyylDz}-W#M9{@q%DAw*etZX7;W`5-w0Qd-u7&!VYi=4jOCWV9=^CP-GSZD)JoI*Z@8zpJ`P-efX3K zD|DJFrikB!IoCwjJCb^N3*nwN9RBg%^&&36_ckT)LaaA5N7pG?F_vjTOW()xoHm!v z%y8BXq}k{{(c`D~s6~IjRJ+TL1_?ruPdUSD7IwuIo@6l>TdRjB9`(Px|39&XdBcXR z!)lB|iPHeDph?fKGv&8`iB;{@zh9P@eRIlsvI3D`tPNf9ly*!l8+_I7wYgOHt}S-Y zMKBlWOtJY7+DY=RoG(pwA!3?QZ{R|27QSRY#XoM*n9#>6XK}*c5UpD9P)M)P!FuPe zr>3Xm9hL!mnV@mbfw$A$L4jFAISN*b4<%Zzf9a6J6=)twomt$SS=qZ)*7I8P$n))u zvAV=D`{v~$iSgN$4?uk$C9TryuVSBFS)LMzD!JV$b|C!#nsv-)h<)hw(-IFBVOjrr z-a(7b-|@B=L~38$N$7Z6EpGCn{i(KM-_yFbIK}Ptm-({?5YV3-p$QRZeSY1Sb`ZSp z5JuA+zw!PdI%qbbHQ=qG@&QxD{4X+~=|y*OWr!X1n7Hw{uxS-5w2Uf3_HhS&&kwv@ zkAjC<7}QVP&_!Vj{e%>ZRW?9bCCR$JobqS`lmgb(l1D{_j!8t&WVv7H-ZRY%c{zYC z=7lLx6szT9+T+~D>$)ZG>%94<(s|nJ6$JI`G7~w!$xR)ZXuj-)R$N4@tu!Q;Shi=KaL&f2B+rBqv@;}=$kgh-XQy*q- z9VN;lm*xzHW{0}ixF0KfJ!e_B9Eq+|X#_yCh3Yd)W6@vFnfWzJXE7WesNjhgiO;hX zN2$lXnA;M$TT0$tnVJnGqdYqGLstc8;N^*nhfAEkQh3VV7g#UyV#liAtt--`wHig; zDcoYrd&%WmQ6g3HOQ-`SANU`=gt<{ZBA00Ps*aNvJ(SaOM=gV(J_(qX;}6?|f2Hd3 z8b+56pn}K}{v*f0l^a`l?ig^~9=lI_)iphfG=*sT9@VJlnvQwpz; zd@3M;jup(-lM~00TIipwxCA*qOKkFFk)xt+n3&0~eORR*q?}MVASwB9l`bX9ge#pm zyRH~iTaXpXk4*X~6lh zc+aTPMOf#tl*I&5Gbts@?Eq~*AO1*Y)dT8D@EbY^0s|IIt&Gl}dXwbh?6axEyILq#-KqHmtrupEk{b@|YAvvPSZiAJh= z^2qkrw6hg%jO64Fsw#v3D_yzlL6Sx%>HY31lH^pKM=B(~#Pw;7XGnr+#*Men=L5-G z(u@Vn^KvCev5O0|afZ{@*Vi+<&t-+6U+HPC&UCB8?4nMrB@Dn!FnO%4KT)ddmyp^lk?9T0vLbJEO0+}m^NWe$CZ|*Jw?5Lz{W{? zH9}JUO4tL{Gh+ldf3kF*9AStW0p`Dn7sawlZkekVk2DiiZP<0nE>=2D>OOfd$*vG} zD@>x3YWS==osn!&ycmt&UbiLi43RPmvXz2N0WCMc~pBF6)4h+&))WMcvA_lM_ zD0xzlT1(0!V-1*hJPsr3CN75-WEb6&KWtTVTATqGKIk{WLWwv3D4iN)eXoB*EE8J_ zf$rLDnEBuucy%I?N;L!u)GXY}L>pA^<+F;SkX=kSI=&4G!S9!;S6^|Xb6+7nel|VO znuh91K3^WFtj&_d?kbqN-=g?F*cEYCU81?vY#krQ;?vx`>(!CZQPI<;z2dud1S9k{ zBQ?g04B%vgR-dnGx`*5edXD-wue_708b9wT?|lZe6~X<^GDByljb=aEqzAcvDckUj zw7P$`W;QUSYN~Z&%I5fZuFdf)ronGwx|Y&s|X8VHXFZ%@71sGp^{x;+0pq7f^s zDu8(?m@?EL^y|AR?b3Rsu&v6;Dhbd8eAuY_35tN0hz6p@>uIQJ`~B#GCKqj!mEHrE z^iSHq3?l@|biz{$o$H zQwOTpjY7`n7~4RvGC!N>2t!Z^j6BPPu8`?!1Xk5l)Q%x@SCT@bx?ZhP1xxd?2Di5_ z;7#I}V?{ZmD*=bY?cI)XvhAxPH^uR?t|qdpWwd^P?lr4LZ_O(vFD~fpY|WS@B%1%| za^|OeEbl?1S`gLvL&V@UJ8H*kF&m4^pYsA)h6GlKth$Y=F?BOpSfm^D?e!pQ@*zU)EE84h?N!NUZuKAnwm%&6jG>k z(~|x9^UE%QA-qR3R1m%|kg1vZr_t;2WBk-UhpcLcgNp?2ynT9#W7evn^>xvyhNb84 zQYA@$nAZFbx!Tb9d`Y+!$XN3hNmQ0b5WAGqJcIq-N@7rK_<5XB`5HN!>t=fuAKO5u zjWnl0p`1MrX~grh(oV#58BfZaQIl z7Qw=Z4>I)c@-r(AyT9)WL32dD$%!LA+0=)6LrBj4@)D^Je=P=q=rc!Mwv~863 zC9@%dg^pHuO9qj3`I35t)65a&bRF;b*6d0B`ug7fw~zETI;a5lp^yF(Ku=$9i}(KW ztH>@lg3O{eql!F^OG)iQYe^XceDt?LWJ?{&FaA6q z|9nRz;drz_KT;I>1#$siq699oCCcBy3bbyJy!^*XtRM^t@`vG@7Uy6_C~MdEgSz~0 zHC%_Vo6Ax6djydm_#%jt^ucm>BXmyj<%|z2_@PUaMQ@pUXk}xLu zXQY3YF$c+E;?e;PIW|i(p32 zE}&WSZhHTSBn~cp2|+ThrQ2yEDanNLAMAcEZa36L2yASq+z{uHSR;yKFdBQJ9xx=w zt)DkE8PfPPnK!q$hii*WD*TaVja)n(6Vvs<-JKn+NOBsQW^lcqr{V&O3704{F8XE) zS7rk=a2(+tX&%=OV%*Oe?HHb*(=FuA>;3`do>NczpIwxbbehp(aot3}DZc;1z~tRM zin3dS>&snD#=Af}8z zjw-tr?fqRfd^#`J_;~tNCv78G$V^*F4=ME|s^y~^!{XaZu%<$|&?oZniSiQg#1CN&O zk7v0=a4s8dGvJgUl-Fgv#dsW(V8IW30+ zuWleSD{Ma(b0fpC4R$vRO07HDB3Z>6)_L~C-oZJFn+x|afl}(WS>;h@dB+K2ry#HXZOG0Y zy~fK#VJklQ{4Hhti-ob;u}FtC6MVN-j9f6ney_w>0TPcQO+RM}^QyzBR3n(rHgXA= zzPW9zch9KWy{z7LpzsJFxKnYUASS!}!nR`pXi#08gP^s@FOn7-53ci6D1?v;ras~i z_maN@JDZ(&w|1AA2@CW{lanpo-{&KC7McH=ZKkY^ga8V9^KX$zD-8fd^XC?s1&F0k z6>6@W3;L0t>kXN|3_K1+FRtetgb@5K;wpYj`2~>)EY9K6G?=lZ94c{HxN>i{hX97s ziFI`)5;lr*NI4t=Cx)A~0UG8|6uu4EeDxXXk zh=A%!SVp&%F-`8iXl6t>lBSV>$jwx(s$|JRD zCZdjtOfPQ!K0o9`0Fs~jS#Fxn$t<l4IabNyXBR3Qz#YKY( zCSgztL7!g-t8A&*`$zBDyvNEN!9lI=)eXfG%f+H0Ac@2*8i=W+ReEE_gV|%q*d~Ay zkHX{4R|P@U&APWyyKg+nL>)qNG#FO-s*!jw-GmoBfGvQqh*@{Ao`iw!I-dbkTb=z{ zDG6||ouwtTu=M2Va*Esb)P2cjIYgqD01e9{i*onBGogKU9I4sF+ZqbfAXQL4>{!U@ zwH6mddmKj0cg&iI8XotZ!m2FQWyf zT39+N!SCLC&4WT5$CCuo@`aBR1wtym;r4C9yt8(U``3;nCB{u zRLpD(BV$Gz8nw2DPpN{NYWwxwS%jWa?%CN|y@R^*DWAS2hV$;E7Yru2t%G!+Q&jJ6 zZ>W#ecs0iDgUwfx_)P|c+kqR91K

97*PXk2i*_eAE;K8@p0-tm8qmpmklYiRvC2=9ByK?cEh;_kIi05|(CAv8dyJzY*~<5xO#=p#~b z9e+|0ws73fWHjpJrR*(vY)y~3TEJl%3qLqAps;w>dKqnZJuL@12;k{d&<2?#eZ zcxXJ$Tv4b~AVJ$3up&KUulQc`2x{70IvG!+{e4P3YI@Adyp|1@=}q7%?kSA)+XVbb zS4k1~-o*EfY3bpnhThwL$vpIKEj=PJ{n_;fEGXYCf6c4wRsURYknE-*e=|tW*^e@d z1ZnRH!+piFr}6qV_av%*-sYp`tfY88kXF}q_-h=G>gh_?Sv}u%^DpBcTG;!As)``1 zPx4D8-4jU1HK>pVlVy2!JAQHA`#xVj)MuOr90r(}4tKKk}-;!K{ZS|&+yw~s2 zGEKi%Ufb(}mkS;qDXR#nI4+KVeT}VW2-U3Re$LVUBs63&$#kX5-BNR7dk;Vb?(1_y zkyM2x*T^s8fwf!Hsh_JoTq8X#Da)FcF9NLlw#2d913FB#x!M5`E*E6_inE+x)W_l9 zU1;-y_`d>nXP0(BjN* zhbSTTi`3IKJK!fWB(rkl`(RN9#buC&zMhF#(5K5@E(1w^qChOTm-vD4+LzZF7LkOM zgTdW+s66$+=!lwKiY`X}cfMQ(#!Dqe#@khIHoK>#uN?||ahrX6H*E8cy#S)LCcdWjF(Wl7zm;|0D6otBz(OlpGqsY!r; z9L6x2icZRJ@Uyj5u;?V8ZdT!TKinjqp0o*#Hh{8sav>f|!&hX22tUBoA?-t%Ny;MZ z%-~7`s};FL+yTCtI-ICUhbHZ@^LXMCdHD z>-{eg_7|rEr0+}zGDWPeTKhmQ=!}ulY>n>+Cl0AmI-SBQIKq=HR$>226p>LfP@ibH z_z5HZp|20a(ILtKzO%mr;(C-7l{eH0L8@#@MwM3y>N1t{G~?rq7PnSf)KBQYnoVo@*ls3X z$vg7kNB{ml8?`YuK(zXhAVV>jzihfq+;m~HGY0U8`YzV0npZy>sxCH&`Z@uf|?#N`w>D*;F$g)BBHr^0W(Y4E~rgtveP^~*%?!}8x6JEIJx1~U? zP<1m^^qlb@Wu8A_nI+0_u{;-CCXH;5xtojbBwQ2>)O(!e&^*&g|B~m;Lf~*V3&jT6 zK;9w76=UZaw?CpE$iy8&d>?35V$UOz6BRBC$lv}}A6#>E%PGrvs;mUt$KX#R&`MOx zWcw|nc z9d%-4gRQ1bvdD;azn%GXp(%br{L9ZX@4Ju6p?Quzs@-0cmA!+O4k6I(ICF^(JKM9l z;3@B>kp|jzbV*OXHjz}+m$(QkBL$cFq$!V;Ck|s^1{3rsU*i&Vj&wLW?N{vYZ7)c; zQ}jqJ2J%=Mak9N?56KEeMy^>#R(kTp1dTFL>#A~xcyQ{@I@eI&S4G7q{aN75@${{; zpP8%Av5VCoA%o@;`K_h~)&(#ets;X~t5qwDSsu3CtqxHY`VAAx+K#VvGhwDTL0Bdp zLl)d`X>`oHRc|@eh&3TR*eCdYI3_r0pg);x@F{?V`vn?Gfu@X)ZSDn)aZ=N(-le9L zHt~Q-0tdPIvIIiZ@)xWhtnE7SLlc?x`FnZDb9x@J7rKXN19ZDj?@5B5IJQ~pD`x1! zu~Kop6)#FE>Otj3mZCG%j8P8aCNiV6o>OPjt)E)N7Ns*$pwfH7y}p)%UHqK%ORtt= zm`~5r$?kAwgqhWNwGk74m=kr9)4A)z_Uz&|_pVpY@|?#;j$Jd5NjskN(sXR6FyDj~ zn7h{nLnJyQ@IjLPVm|E_Yi?CLwSggVoM(%7=(Ow>0PoK0HpRx-O)meA=s(#?Opv2SkfTnt&)(x27ClM~9OPS1YK)&wz{$pReVqBcCncFU*O?U& z>`ZY_tQL_pcCx2VlNW;}j+2&4+EnL=s?`eZxcG~jUM0t;DOOf%^F#Lr7|6_f+uN@B zXNuN&e?FzKN*&YKEnS_4Fx;YejFiGOd*&c94WSOJAX~=Vqk8X(R)^ao0fHxc%5|_9 zl0$3@H%FdIHh??l%~7)JSgYNhENrKs?l`xMJbloSXurs0-fZw3Yf13ZTXr1qJzrWO z0nF&DrY~Yqt)LJN_6WX|ZF4Ajxk-UU;3bhN*J74V=rgEA*@!4nYbULAh^LG3=~_B# z3ET?P+W6leV|D5MC!es;3_en(aO)-V`!&Qb$1G*Q?>#*tD8AwvoZvp2719ym&v#tj zGo$-t<1sPf!>#Jjvo>I2=#;%&-(%5%QZ7QvM=+_CuQO#PUG?9Hlf&*dAG=V#bLg2r z65ZiT1q*W0HQR%KfZ;=Hz*1q58V{7`Qi$LMAJzwm&O3AoHn&GVov|~H14$JERAc+ z@$q^N4aXuzLwYU=-q2eakW3-X+&*NZF|PZs*_L))qiBSo(Z`#8@Z$rW*HsE@l2vDM zkE#{y9I%mF)+f-n%o-kO`pDgsP4{&3>;ExUc{)j4#_i|zeFhLVP`#G9*eapEZJ@d^ zcEVdW8v4iTOX3aZH>RT~Ek}y^+Y)}cm@)>>bDkzPddIXWVDn1zp7P+X4_+vWkax$D^j!B048V;h7@Ri{n=mHM2z?LfBz_wu%1@S6jh87b9a$S zr=S*S#<1x`lZ6}!A_(?=4>N14f9-CJ4S$IxaJF?o320I_oUF>YBfYa(s25v2rsG+HZP+af54 zhjwOSgk@`}cj<2eNyh^7-$SY7{&>ue@_|BH=;G7T{ks)c*;60Fp`@vWNVp$RBEsB6 zX;2a7^VyHW$x|54D*IR<>gMLeS%4@}F81J9W?S{7Sdl)Ely6|X40>m-oHo!^C6gyd z*pf*)WIN?(g~~TJ*pJ^n#|mJN4RX0ZpR;pole$Eugxkwp@%Gc;oK_KD z^*5ynK0KBdmUCX)zUM4fudOR_$13ccR6Vsr6eTdiVO8v$mZ3d;(eWqE@-bPOx-}gK zOw6Y7RJ9IyS^k&lXD)2dx3;2<1W!8oWX6r8O^_$V{ZOMVYKn^IA!+^Ao~~3MZ%P+; zB#x;0q5zr7w>=K|-XO+o7J_P0%n_>XP(SrPrkvmv9r|~O} zoH6V=LY8t)_-cK(T>VhIjCzOk_{dR4PVbgERr<@dI}?+yn`b1PEN{JJY5lNt_@Bcd zR0IKPA_~8Nodz92Hv&gFTO+fG@T2^(`9Wlcn(8_9wXBc(YMID+*cNe7!Mtp(82#tG z8z;u=`!#Sg?#iKu;?&(Gf3MNLXzHbmC+}nJWZY*WIKmLRBz8Mi<0Ch$piXqilQU;J z4}@{BcoT_0`EO1=A2{HD83`Tf(YBtG7Efn?hW(U5jjTKhA}heP2^9ofPLDLckNgte z{y?_I>U9e0RbrNsyFk~!W^lz4mml@wr`8^u4HB44ZuJctpDR%$$Tog$jpn0Vk4m;> z5~I(QB*Zd_`Mo)im>)!|Z77V(yEV=p6vj0O*5k=&unqg~d=!vMB8B+?n3^U9r~ElC z|NXwqO{j5d0UME(1@AJl3f(v3=|057aCLQ9&?lHo1-Ya=Y{iN@cq!q^j|!|ZzUvNL zjQQXNw`TcF;rGA$`v8JH{}35gFxy~v1%4U^(iMmyPo=;y5-eWc3t$6)C=8AEtwf{b zlTE%H{qQ{s1G9FkN*G=ammlS+gYKoX6#f5R7rZEEfefvj#lYAZ+yl>@B%XKxl$$ar z&q2vva`FvgPlrE@@M7~F0&?j5P?K8kuMD3~>x)k|sq54<(xlsS6sC$^V2GvyfE5#SFI>y^*5>;3%j_=0VEZ30qp@lK>O`Bgpjb#h z5Nj@T5U$Lvfa0j;br)M095fpTvLa=o=74IC4NGvl;pZy7*+(kv^o?AgNe0Hv);-|U z)H3$Q`;lpRX5xzhZes@_eY2M)(w7R2djBWsviA%>)EjkNew$t41{w1yS^pc) zW5#(|;h}33+ zdvS^VV1E=`5EjN^c0M`__@MlLo^FQA{o&e|0H+>}ky~d?2OneBB`+9EsTgMoJUV)V zo>LM5{J=T3E&wIZ%Gvq+e>yr5sG#tCv36<)j68y^oBHPS!xOV8geay>_`)*}(Sa(} zlw$Pw4)$?25ly1FZ*X8>CM;Q7s$|i9B@|grE%@g`i_R#9O~z$Dw60E(@i^q8ksU5hN5_ci!p@3nCYe4U?BvAN&EnHoI+ zRvV7nQvcoi_!Ok7uztdZ{5w_;9>c%t{n$%L&X74npnF`M*7=)nI0~`6fgw3|0wr_x zJz!XlsL0;s_9*H5XA?z1e=cM&wTdJBv9Dc4{ekhVg(^q>ls3ZA;acY2iVPnS@S@HcDW3Y6h zXdEpHk?D61(Y-q&WnOp4muxxWu=Cw(vnzD*%XNJ_;?oyDSp^L?lcCBJzHaW=kowF)Pu z7R??8Q}PWhqe)TIy>SDFTW9^6E&@hSEUVmC2L#iJxxCo-&#RuV6h;FMlV9SFLPS_j z3J=!&z;mVZ=wam=jGx3~&@v!?w+AHfhkpGcYc#O*9E}S3G0`s~#;bJ)@Tt#aR{(z} zU)GJjQ`wbeoa28kwS}|(M0B>&FsGR7`$j=isWUF~v?c5eK~&b%8GAihieoi(3Brm) zuNL*QmKAyL0j97(%JvLKUR&;umfnmH63X%0?N#SIt9#6ytCvp4$~}OTro+e-#NR!W zz8PoO3ZTsqJn<#;#~V=!r5IuH%vHI`fgCazoNY_$!M2#;v**T0HCv*RBNz2^gv!)o z;q3faEhoaNgW<6+v+xxzF!!<6iZQXZ&66aasV}~bpy> zdCYF;Rc@HLNgnhjWa(zeC&}DD4lNVrgeRS4iaFKIGuz(FAON|WhZjgv-XtanU2NIw zy#E7;0A$d^W8-ocrkh*n!I*TR5iG>Ch0%Z~$PxMH%bpRUqr~ycaK_R3;gFelqx8>- z5aLk{saw~*GPF*0{(@!#0GiEe*b!+U=$5swo122Y=Z(c+q_Ik#EIkRZv)m&}Lg$w6 ztpG29%=VBa&<+@Z>~=|i#2pa=K3fhZ$IqRc{4%rX#iCa=KwOxzkd;5q6s zW@b#RHZEzC-7qm{hypuL2Nw;1JmCvhhsYIb4};--Jk(ES8~W z&+qFRegdTQyP$3O^302^g9An4Z9PP*yU<7En@$ea(69RTKo3L%OdySt*w0o@A91lj zfxQaOQs6LP2mK!WmVuj~IMZ`Ci-#yloe29DCj9EYaTU#IhE$wnz{*isxR;9S=?Wl? zYv21nV8mUPM(^BlDnwAt;f6*S1x+#?YtBvNp4-FgHz8ECve6c_&0ftuyVZQwYila; z4(Ag|iXb*8=b!pLITL-l3N+n*O5=#@GPA^S2yZP9eqig>|K+ah5Q`aHV%`-XJ?Z?) z>F@-G?&+%z?9QR@u5RvM9@X5wcy9yx1rY>WKR}Z#=yX09r7_(J z%@g7>AN%~q&C6F-sWJhmC#R%5og)-gfRWZ}O!=2QdC+dbx12FP$#0PmgrUOyAd zA7%Cs8>)fZyfNyH`Lw`dGBS7+{A6G=1(5~cb+>7=+i_hZC4Ty=6?8jTFetz;b<@cn zh+&7Z2ICzSqL>xio1Qz?cZhGC=Qv6{zR|>F7nV0RM={P?$LsPfaK{b6`vvE7IdZyv zt7p{5{W4T%R1}tS$`wT@_!FN>9;&AD-rKS{^RVww7aR@ltIitrvBA!HJ-%J*(1jTw zTTRNt!Zvk;E&4tWOZd?>T5}v`#{EuF(^NNpN~a`3&W7R`x0vZlvo3#<6_rgKl_7rY^gP8QTi1v{GS~CXFH8}u zf;IG@{00ZLYTWn95{D~@ABV_$ZWTTZB2oTp7jsCV_L~uyBo;`d`7gDKpNS#M6CG?U zZ zhHz_DN!$s0h~A5RmZiD({Zo5Yj-Y2Tn7G&n0K%S1M$jnU8xh{{4NQi#sK=9V-xUrX zqMkb4`aqNN3`Z0%>NKmEWfR9J+5C?on~Z8+u%JcvG6n=~8@{#w+Vk2*cXc#dvbl1p zW?Mu)E?%>CwM+JmHxruGC2V!NHs?4Ky4-&4r{2G6eBjkJPI1(F_8$nSamtV(ruSSp za8&foRd;yAlJeN|^d|;Y5;g)M!5-Fj+!7N#jxCu~b>^6Oyw)lkO%z7BGrq2Nf3NZg z%iMUK(dtf`nsxe@r`)lY{{PP-vnHQEf@8Rl_p=d<+j(?fD&8J)L_DF@LDbzW3^j`# z&PP$0cr&lK_`DWt+VlBD)_r5fZy(aoa@(is5Er8@JSn(8v)61|Kj%xf_Wyos&_3%u zQGCyMmL+(#eNPf|h0Z?h0J}#{6TtX3tQL`3fo#r5?8iP$^B*vedU#(~-Xakvi0R4y z3dzXwJ0QtlUtd5FvK>O|4~;!)TBTZ87Z{45eLFKNb&D9M99Y3RI|cbR#NUJr-wZQv z=H%I=;QMTSTNP8!Jg39(HVG3%t##KwFwxz-ZiC}}Q}2>Ym%K%b69z9IG$A`nHH-ZI zj0Sx}BPN=N-nX;@su+$-o9zK7HqYtjFSbO@a17&NM zTO66Qrl|fZ=rvX#QM!8AzJnIEsN3MCFvi@q&NOJyTGO)ZY^9Kj^hTWjqh5P_8zxIDhbJmFGNW}PxhJ7TrX?k>KBh&6ZQHiv z`X=4+6VxmhK|rC&^Wz^6)2QdRmS5}e=zihsC3u}&Y})OmY}%GB%A>^g0W)kQm_l^gccZZ>LrV_rQ3JMzDtu-7_hsP2?( z^1QN+jG*5Q&$23#ORDYIe(Y+Rp}$`F&-?*By~^LYi|y;cX~47V2EE+? zG2SLF+JTa)P^fN}_l*scy7kI+HP&JS%}CbKpmHHkQA%GmF<+tZP3fm4)5JMEpY36U zbIjDP?=N$s!W1=qzl9a~I&uQu8%*0bzVlKhh<63_GI*vnry6IbOxifk)wMLPvtBm0 zJDJp(!We0fHD!Rd*yhm-wMCgC3yXHhty)Kw*iZX;avLNS;JxRwc3I>p=m>jM z9O|7TqkPIm)p}6&yU!8Xk!IG|#vaF@mB0M5i9Yu?1Np7VWZvnn7$)mOq7s|EZL1yS zs~LHN9q$nc`HvMJsfiCT$Ty9(c>~UYX~h|;rw^G}7X>)`r~3j&a)v_&UXt8P zTG+Ozq}mtjB6hjaRU`I{9s09coK^HH{I`}fi*+~54`P2?KjuYi;M+fsi`*8Ti0$jI zN-h@-a+$lq@^(MZy~Rl;uHXx&cjXzbwu)xn(;uFa0Cj*_N;>q})#Qtneo64GRinvRwY9}ee7p|B|3Q}H!paR$)6XBXAEHpQP3R9fqMFU;0pYLA7!-> z#@fn8Y`Psmc})RR-HWOt`>*#uc?pZiU=oVTT6!ck_Fqno42yk}8d=%Xe$C9lDk${y zQvz&_sRzN|z{XAb`n3pfN+#$_kusapo*%5fQOS;ywsUAH+V}JfYSPIab}-rZU-19; z`<=Z0O|F9Sw3{;}4kuq-5&PuA@vq!6B<2gL)@pO@z&le1z<^-{p`5X1le!`0?G2+< zW5P|m(_0kDORpKp@2GAsE*!zeOxC=@eC5Btd36$ea{AEsohR={&JK@{Jh}g0aW&fq}e`^4T zr>zoS;zPF1uoraZ>YxQZ}yk^yIgL3!PhrH*=E<`uD7rqI(q$#AF z-E(QE(p0BcAZe@c|8^lfFKc7m^QbYp(u*b7za^r_(XOWz@b`U&UHT;0hJ?RObE0TJk-fWZu}lDQpFn0$Oaxn zWlP;I1a&-U9wTXqpM5mKRHtR!>V2{{nf*2MjPClaOqI2lABK6~v|oJYn}&@<|F^p@ zQ0q;#(;2Ju7Q+F-KsZstT=U6&0l)K{n&?AEnES}~#s)RcG;rXY0^FKuS{RYXAynYh zo0lpgjYeVJnu)2oleRI4?ij4E9eld;C=(yLOJ+oFo9hKtbQr$snnPS@n!bstIc8e z>Yw%bSOp!@@gZ-H-K8;{%j^Ex_@VgP^K@=~Pf|ud(xn&Ql0KdG6byfJePqY%vvHN2 z?ZG>7lULn%d6xM+DkINKA0*3a-@N2Hz*1l=Y<(QaZg}p3Fo}%C|7OCSkWP1i1YYme-CI|>%lWcBcw5}v*_+2_q7VjkD=X8*NpU?4Gz~_ z$LL61RPw_~3vS&FvY(&&;%o`-KXfkg8hc)G`ppBih|{-TSRW_vHyroT?B4MyrC<8Y zFXwEeLy>4+wZ6IQC*j;mhxdw3vFMAbf63vGx2h(#!v(tWfZiG?GlHj*&@sEaRp=pF?$fs{T*$QeK4$xd$i?YWM`QL)1kq^{~U?Z`j zi?JpnhO5q{#os%mv2{N zuO3C%&?HQ~4>^aUb3r##Wnp~I+^VDVPKzr22mKu6{1cvir+}})THI%LseKdGl9tHPc+7w zJG^{-N^xo$zUmzz6~a81j1sdu?^JmkID-X?NL}-_{FHHQ=3}z>uAv@Pag4guGA8Ex z+6-oW+1Pt)%tj^rTS6f3D%I{C-n8GAR<|QfM4KZb3$LU-RL`wwr$+da%Laov;7s3~ z@B6v@3zY@>kx!2}5jSlf&BHGKPs%Ws4)^ES|LbcSVc40|pRCJ^hLtCouXgD8a&wQRQeD+E56H%wvX-v z+gu(lRQc`n31byO*z50kvWRacw_Y1?`n(lVtSh6caWqrm)@8z*7m`tWJS(bl${J_! z&zZ+X%E`0V5*>Z7dDmxcnVbF`d==_YT0(!;Z@qCt**lh7rQDmXA@Y6w)#=H5NQ1V$ z`4+#m&0Ru3P#9J+XNU{0y$a?x40aj`R-X95HTpk`oFgyKg~&N|?UfPsj>zDE^?R2$ z^~r%2LUm;W|3&A~vX+ZAS$RrqbOGiE>N6#lH&#VnG#Z~($Hr1hCZcfXsi~RXMCu4^ z`)8ByAB~@v^du(xGrJ2---aX;qY|j7Xc>WQ;^cpbKw+S!8@PHRn@hx8AY;H zRXK)NVpN%vsc)mcaNUUO+i3Y770L7DxFbrP*WX!rCzs6ymD93jWGP!;gwxWWOqnlx z$Yp0*`&o&^*pPI0WuLC$PBpQZL)3-dtp%JoCW7U!gtWr0XC!4*|L2lGKzM6M(ANJ= z@&bq)yTSsboiD(%3TMMjBcV`$d=mYbhbw@l2%+VoTD^ta9RatpCXUmLv@Ca_YNF5^ ziPo-KAp#r*4avkuZtZ2KT}EB#5G=n#Y9fiOf2sZYJb!txwLM6kCTPZmp`vzextow% zbAyaic`bMD?U(h_UI!pv41?EHiz!tmE}Yp}?Bz3YBDS==6)RoS+T9YmtwT#-kz58soB|8FKCPXw6+2mX}v$VqZ> zfmq}kwV|8G{U=V}1`D#LZkQmyH@yBZ?vv%r&lj`Tf(?$z*u-ak?v5nFf}nwfBEgi{ zA?nn?f;hL6EM}IUKYxq0C9$EVDCU%Yxfat-c_|`AY3T2#KY0r{v;y3a`U>Dbiabeu zS>)Ndxe$ELB6JO~3~h?eXW{px;~L@@GDgt*vq)2Tlk_tW}i*lR6R69?sCSf`iZ#N-fkO68^jy~i0zggO4C44t@{`2AC z+W$F;9ZV2lJHEvH#3uVh{n)~>|4FHxF(kr__yW+MdNI9XU4B?zF-bYXso z&8g0_6<>Wj=c}`>h_M@RFI9dn*mJAj{#xe*&@@v4@Qwj>gmE(JzyM4DHrIUDr|*-Y}{M<#g|Cz zPQ_;aBHGxe+uVe_1W#9yF$Ba53UuionJ6AE+YU{ghm?d0=M^yp498ZeFsV7XbQrHc zgtIf%Nk~PweEnu@M?2`UutP|w*aLT_2q8X?rO&LxxRJ3JxATce=*}A@xxY8{IoNT> z`1piK?ZE9Di&q0I7^kd>9ptu3Z@27nIdG+m1`CLq1Ai0$@>}W2evx18G%wiH^o8Et zR+a3hMC?S+{>tsR%)2DtzRY(^4F{aT&`Vm+hq+EPMFT8pH{Re5*FFbii1!{`Fy8yg z1CMRP59$zoz+L#Ld;@_D+FPekI@7o(Wybrh13&()3mviC|C{5O4Yzl>wK^nUtc7}5>; z!0+gCOZkmKb5;$gQCTkE)#^$&uCgTdll;GnuVREJh=$QvV%EL0JwHPCVR4Z zZFrSd(%I-U5p$ALJfUAz+H)wbgD+4*4j_*cOzzh@e7c8QNfA>p)W+IN%L>k$f>fkn z38(<5_9Jt&v>xRp|DEb6(l|#fF?>*5o3!d~EJ{Gta=iRme6+%pMm>eQ#`zW$F(m2z zgny4m8tzN?eaZJ{GhB?PVET(L_n_pNM4>wLboj6!t40({udo-W4dg*9uVP`}1)2cH zS;@|wAMx8P$aySD^?MH#2wnQE&^#O+I@uh&O&GWp=0eK|Ng0Q2u*@>ZrumG@D1EnA z1PcjPd7IAis3oLGA9caF0`V=hxNHPQn$BKOZJkiEBjd{de0cv6sJ zbBGRN?3sP7xFi8FmWU2NPW#EYzzrRFB1g{isBoBRr~tc@5*J^weyuHAY9vKZd2+xp zh7jl%1#nivrz>Ju7`+U`Jfwf`SQpTU>lZ4wa0)lnW{_)q^qx#`LiSgP(jh7yUf-`7 zpv1`;E>&1)Jr2%1GhjEpEZ(p^$$ICyW|XDeylVIR@*k8Q9i3H&+Z@Kv@f{EBu3mDT zZ+Vr`xQ0p_tmJIH4Rw?_eNOKY*LE@#QJ&5Zn15x@cXVF>$LTqj(VHzeF7sEE4lu*B zJDj`C5lx9~tp(fFN+vZ&OyK?&{IEHuV1*)cw)q?3ip@q&?ZU_i@3u;Y9HtktE)~LD#$b z{szR8S>6#g{T+K{f_U6^5EvZ(d3F6Q{kv=_-Ego%n<<1cLo5Ld*KNRx6MQKvJ{#$Q zyjJU07a#G&Ht7?XF%-DB$eYST%mbOvNawvcSKW;y-$q<`NU$VRC}(RQAU-3VEWv6ifVViv@t z%djp8baCiMpoI-*d?`PgG;QG}Tf(Mp$&xt6S%D*>y zsOH$wQtt#$>R2H6<8r%D>5K9ibF{o7O=9iKRbJeTVU_Y)_2ci#>sbGi5{M=!JD`r3QQC`YZ)WqEzM2l$Qc zm$LWLUMu{_KcBpL?Q;}yo0Rt%j0!aY4Fx3F^Q#!-4F5Ti2pSb! z_4n+>5<`juJ!q+zFxvZl=f3E=J9&^6uaF5b$LUcS@{tajh7=Sp@h2+Jm3V9Tm)`A!o;+l8Tr z1*a4^!ndEn9rk+1CvnDQelTD4ZJOX=e^@9%FSu&_VFYlCLy=IYW*Ol>e9TVB)PP@p6u%5~F_l&Axm-J>7k3Tc= zV#s}NZtjK`ju{JL}lI;+woe`U7 zBxR)@e$Qqz@%VS*^h+t(;D*P1JBkEW=kMve6 zyDl2TXZpbf`qBmR?SA-0)C;o?si#y%9?nl3Lza1t-c>tVn_gBPqM@4+3N-FyI$i^<@gtNbRt>0X&$@*DWSI~Wt&0JR-56O-O9NyvHzwMLkf^i z4Atl$HLww!5a;PSrR+yjhjO^n4vB^w*o>|@Zq2>#z-vwNNU!8T4b}90q3}e2N9t#?SOC&Dvu7t%xFDd9S9`$qL}=5?(SSe59Ry6(4UzvFE^ld+8JCKRnkY z5g#LL6olzRD)o!*HF?kUkWta`S&B$9yGV=phV*#e0tk_CglSz?q4p|fXJ@@gdVx`2 z!6x+vpk249M%YBZrCh32!4&9qI+Bq{fMkY=$~zKe>Bg-@TE?>ypLc6{@Qa@WMM=x?S@NlsL@MoCU@T3QBKb{R*3Bto&@U{M~f zNjECoG&IVsL5Oi(UZo4r^q6EQr8PNBkrQwUp6@>%iOvzWl64omGp!Y*IL~BwPo(v7 zuJd^2lDC7Ko(sS$dg05zTmEkNA&psC)?3+gU2U!rxcH7bn4Ol_zW&)V9DCk^L} zGyAxu+o%Le#6rRCug{+BqBnah)&x8IV6?_3%5tDt3`-NTEn_j`-2jpAfqEy z1Eh&*evCzQ7&eKx?G{)a&BxqmZ_!+qcu7%)ToIV;WfcdCLB8!l2w()5Z~B;>UoXBF z=jq7z>`}%^*o-pHKRtnGPEzGD9VlPEeqPKxSp+{CqSr>6oBHdw`HqU+X>K)P)+GX- zHlNl13baPsLu^f9AvPVNqq;(S14&z=_>e2%mDhc4bxv}WUAmyR;BOfuy(ubdK%~UP zHXNVTTdP^QJ!If=4E^|&mT0?y;Z(bULHp%_zn2i-pPgT)<^*4bd;NaGWwp}P;CC!| zbaYUq>lw;{8khM-U1O)xE;Y;*_Sw_y=ouas$oV+-j6voPcaM&oU@{3UV&IzZX}~J= zpZX(}V>Tt}pB|jK^=h^2zNQ?9nZ&AMo7TDnu!$1oGYP-b^Es|;WV{a?KL3_dO2K<6-9@)5 z7w1_}GbU+BQDg0uFB?$p`VD!oGm{8-V+L04S*(sGP)ee_QaJ?rbgh+;^=FXW6Jh@lKzH~>f5@C7el?#E#kj`p2=L?rb7l;bbG zG_kFgrQ^K(scV66)Cfzi6fP*8(ZE=55~~nTWPkX-`Y~h@OHyGDHgBHf^D@Mak9qvw zk3Pp%k`&m5?&BPaiY(ZW)aSN-c=f0*eK@lZwMa`hCg)8pzAc>_EpX9=h-5jP-zZ}> z2a->BtnOJ_#0iaKgqESbaFrA*_dhQg1q__oaNAe+lRA=&#H!^WR;PK}`pdy@S?jqt z;m8|*#(!*Qdrz=mJGvyhDkJ@g1YnIY@w0i;uYSH=A6OE9ldcx=<%5{*=qwEj?O7A@ zd<9nN-rZ*?A`Ik>09P!;#w?Cb{^_QsQ&KMqsu%-Hm?a$9aXGx_#I>2a3AnB$596|K zoS_#;>Udp(E|oo_GSZL`&m@^LW5SR*OyhYpGA3jwEJ_{I{b=!^G%Sx^@9`tBG4Bh2 z8P=jl=WLX9*T_iDLALS0j?Yt%VNyD7`h&pMO=Y^ZBV~@zs$o8pjE~WijbIqBl`H`x zd0z+hrlMKIN4oS=IoRmaOV-->$kATwGE7B^4E;7^wGM-Njd!lAKT%#9Z>%Mvl6Vfi z7{XHnxZ_%y*Uc=D)lq~b3|c`dHsQ9h|LkZejtJZssvaND$zg^+ZND3`$Fe*%0rr`*n5`Lsj!<*h~kQkwBi zkA5}0iL(qGbSXXi6vv|HLk>lmV)dD5QIJ(lJ#i^9w=F((=DNAr*^tolMGB-iALEAte5hKH<9&+||gd>+kAIC-1{^um%d?hYhv%Y*;2G~$Y z!f#`X+UzoH!76XUYwhIjwAXL(tS9V(KoS;zA~?aHMnok#b%bSnH`_o!ZSMeF<>4^M zo71z#!7_jcH>aHXKbZ1orKp2&Df9{=&ZfRpB_xNi)%YChy}{DWX4voxWv= z#dPPwzUAXj;8P6evBPInA^ezHKvZ0)WMXzL_n&M=Iu^jL0nVz!lPcqYMsG@?isa;z zcwvwI*kd#*!)6JfFt4|js{cb_V4<*|_|PYhKji+)+$AX8KE5zTtEZD>|bw>BM!c)$A+N?T14rO&) zzQo5hJLMqx`U8Pq#)+_{0l~|bZ=gqH$i>7+#z#E5rA=1Y5oRZ3_wQQOpZ)hypImds zyzsoAR3#z3l@6!xW))0=f896sIF8Wl|*T;t4l>S-4nlb zo7CtL*g{ZMh|g2+UV`bcykRpxn%^_c1l;c~ z`!+tF1VM@2yBQU9!~zC+ir`;Uv3Sm;7YH^+Y;a>@hS3bF;8YRt#r&Clblmq*r1wf3 z z)6kb8S`^-I`acMo?pI=;eADs?rc|Lc>T6$q5W5;44djdbCe=8r8FJH<#keg~k)0q6 zD)ToOlctQ;@4)M`?1CxWecK)wOV|laQ9yhnc$;KM-9~`3LO9rT>_(k}h7&L{)&3EC z4O-%VAmrQG8+xYSeAYR`_OBKIm>7i?|J@`(^NRrT*#I#0UOWyOzbNvQ00dKdgJLPRpJJe zw-QWNeW&poo06Fyy|jK2NaEx_W%%5?r9FdS~jxEuaxO2d+vGk0AUjcsjS?Nv;n2L-z)5f4PfNl4{99KH z*{7kG`gb=GzC^~io4S(nRE8cWdz>v_t!m$&9 zu;xH8x``O#)cvT(psz24K7bScSx@9==r+yDCUk~;f5e8y8v@rT!Np`KzRwMl)JeL0 zWQVyEEj@4cM`6NG4U|3V+vrGHLmtKsJT4Y9%7SQpx9g)lz+eYhKE3iJO*u@2I%N6{ zQ2xW<>`W!Z_snzgy-LDUqw57GfP**$pLIA8*S?Gq1(lqz3c zM2f-YvABFvb5ZH{_-MENmyMEa;>KM zuZpOQ09n!fS(VTnn}7d_kpw1EvhJqb$G>$A3`Dk<;Z)sY>fEF42GqZ6_c-9u=EjzB zbk9^@$xFTrspx}=noo$`OHLSdDn#W3CSpvqsK9)`X(!-~Iu#YY_LC?dP2q=zt1<pTMI-0Vw@ewqil_rjoE!GY}eo9EH-z>IIqYcQzB}G{s{0@7kwWz3ksFf+iOs zqs=`Ayu2HT4Q|CS{&>gMgN-v_zmy1rGpyGp+hwUtX*EArIU#vSxS_TDfv5q-`lqy* zEfcEfRzkm;YA+x4b4~ZqZJoTge*8<~BF!}2Ik5Pedu7l$^xC~?aHHk5<7qBIRz}9l zfqWZ!Nj9@e^5xr*nH(a@{^F~t@;h8`x9ymFg=OuD7xMV z&yI_}%YE*dH(ah(`U!fjxH&n&vTa(z-X6~dwHAPU-C&qCjqgqHo0s*D=Q**+CP{*l zD;T$T-M>z9WhRLWqo(nyy3Hc)j_S0+sVt!FksDATrYIedYkzd8O(|z7anLwT^5xH1 z#98C7Hp7oEIBxyEd(-?X-Vcr9j^Ah5lGv9#E#$tCf{x{(MrXZrb1e>akN5rWnxD*w z6YctwgzCL313k*q7~DlcoS_$0rhf^qPhSg@cu{LI$nT3L4$Xwf zCVqT|s@e&>+#jI4-ne3)xs8WrABtZ6#BjxJV^%$1v9HNNBow+Zu1q_Yt_@XL$sKHc zy?y7r3$4hV;AFx&5NT-EwZkQLSl^#TsI@(3jIn-GDtEM_WN>!fvcoZqEpVq@6=SWG zgI7eCQ5Y5R~~gpz$!^QLJq^;gS}E zq)4bneHWitNd&~I2X3w(9zUoOu?P0sqjl8spXh&+5>E5YilPBUNI$9-Kywli?=@5d}af^n_d^sTd5#8af%K*9vPBopcsViaHZs`Iwfat*%?F2$awy z{^4xk${bK}+4xfqAo-bp&avY})9=&Q2v>Tu`VoCuqC!1nLL9a@f_zgCm6K^x?nXbq z)9E$p+DOH=dbcA*GRWrra;3+Bb|8pb@RRQ3udCJ{Y4F zbqY=O6Y%Qi+Uu!M5Ihs}jg_Pgjlm@8)gK#IFp#x3ZvXrMOLNI%t^iER#8!XA<5)hHE2Qd5QVawkX09y+Y3ZAQ)&>0aHuzi`i_1 zJdm9`iS_INYj8PbW>iLyzYI1A&oIA?UGpn{sl7lagJ1@kAKWGVa)~agSsY4V#yC;S zrq`HxVz_&kr1FzWmpP5j=l1Y^Fj%zZAAWQV;q=%&@@YfhZ@Pq=mAvI?AXYki?3;iU zfAzYcH?DeLR?Ch=Z$Kg3 zIesbIrZgqs^8?DFjGidmU@Dd2XV*0atXjs6yTU|Upxsf!K286{95mxS*k>E*tVtvI zYU}R8^&utsUUcF4Rlhhl>(R3jkaPH z;>)R~m3@z13)Q5@M6EeRSGI)8jl)8v%S{VovMf|X*u{d2@4qF- zii9|tVC&lR!x`n~AUNVHa-Evh#k0x%g^(ft@Iyys!KR>N%L%(LsW8GQhAWv9-&+P! z>T5BE9@`=#VxrR_^M+d0qYz$Gjn)!8POu*JXexZqNvVnlj6io2bQ7YPS-zW5fbO2C4Y=rxqJ4KCU+%>~e50n&RQl0=wg1)Yqm^l*dQ%}01>&?B zT*D+4>uUh%-2Cjt{NX~eQs~aG{rb|d*YozjyMfw5yW{;3I)WZYklp)eO=+x#`#uDylX!~2FM^6lbI0yOvegd z1)H#dT2}%?`I7pC#KyGt37+74-;w#A!qeEbHi}*ls zVMT?@aMQwv7C9A}%f29sO~6J_L^FJ`+RCzm!^HEJ zkRRSkL)Pv~yKjYFyZBcvd-K;|%y`O0ZA|h|ZHptp>1QYBzU8RUC#|i`^#gV{Bu~O> z=x=|X2l)w5w_i`Rk!?O~Yvvvy+hJH!H*5+!vzuAk0KFabcP|!F@SUo@if?#Ylry*k zZ@*+FcTQ^hAr#TaY$c9=jbP`xqxWI^YfCz2|NKy7Y@tl%W*;k!Z`({ zQf~cFGWP6JiUcl~q zb5FJ8j+Yz+51gYHQ~5;nj5b}cBAHPq001betx+R_!|j8cDkWY-m{#lJ5DrT-Dchxl zxq#?xcCLtFzyKbh2?(w*5u8DLt}`Gv<0W*_!U7uoQHjB>lTEGemQ7WrE(ZqMHEEpp z^m#k#$ViCL>$gU>$U^3pKi67w+-`*wy#B(&ex@AD_D{L}{27t%nmWn3| zo^Av31W+@878?s$iMZ(}v=?yiRGPkzB%w&tBRLX1;o=dZlSC`tFjU8z`I}e|||oFG;URSR;gI$ULmB8xrz|{^HHCU?7Ua^%6No1Y-Tf z|4?@mpgx+IXuFFRjLN0eQKj@+G`Qhl=jH&n&Ak!olxs4=QOCHXuIFhz%Mih7$dR5*6X+sa zEgK=2%oi>d|7>y3UN%1W}$%8osnW-F;?_H=cO~=A@l<)bZ5Z z|1@**N&}pL&9U~q3SFCw%iT>_85urYe-GlfyA9m+#BehH@9jcFYyBj=a&u>P9f^MIs*Q2?qsrYKNzJ$ye9!W9IN6IOWO33OsKF3!* zhdFTZfUvUz`AoiKc=--X2t<5482Awd8ga0MN`y0Xxzgj8I&ru-HlM2{__T=4=ibAz zyDk10ZU;1FG8C&wk{=EZ#fu$$CVxRG+2vzJG|0Vn3+6-xZff+NA&ACmne$Lg5mBHM zn*x=K0c|}rzk{g1ysEG)ts?XP=ZFzhMOf{xcGjLCYUdss?lix(uQ%Y#Pde}!@yaDvQV zFZy6HJ!9O5V90;g{b=0}4EMny8%?r4M4TjGDrTySjM~`?!6S70Im%DP^7G@voy?<`PM*hI?DHt#ij)c{ zsj&VT13fc^V9_Jn(Fs6Io98KPPm$MjScpEEC;eK8`_e~278FL0MRA-=L!T2yAHX$= zxpMgn|LVd~R6An;(~hvEPlCRoj_udHuTd=gxNm56qZ?`7?pF+qHqMLKwfmp~e@4vx zd1m7_*b&9B?$YG{hl=iJWlvM|4^jVV_lmD~@SF^~JJz56(SJFov3Xb@Tr4Jb6Dqxi z<26QyIj$rKxJG<@Nchge4MF3O-`-5PnsmWjadYpI-1)RE*SF)C z(3d%u%|}t|;)B+|=qc3KJ%0}{M`Iechs1~0@w*+M6K6qYlXAc+@E)ntOu+!O`Zg=x7GM{N%&fsgJo*3o^@;7k@QGzhj)X1&X|H zaYfJ`SWjvoTO*BL+gWdp8e&VK8wvc_8(l5PJpE&EcD6TD!;94;^e*y=$ZcB!dkwZ; zfkLvcuD0FxTpYjgy?XcKXM8%t`&uG1$qgNsjz+BvpVKZE{U(PEg}ZK81;kD-yjE4q zM4g@!XbH~{ZpJj`P$ErA^+9hcXre}T!bKnrt_`^HFW8GitZ@KRnj!Mpf*xb~>QKZjAwJ}FRI zheRGr7YE&I4m|ua-weaAUeDo@@Ejzx_0ewKR+jx(Zm1g0Eq2{p%c+A;ZDlTKb0N9n zg^hnRBqq<$<85uL)U229rhYmgp#a@u2SCsEO>qg6-uJ@8u)nVDSR9uPU?8~_%4qu!AojBBI4R-0+fw2}`ag#YPXssz-E+~)@uSf)Q%Yb$d zt02)flb>1qTwB`YJwL#Du<=%yI=RzlJ?lDiU0YA*Hlwy#p)&E>TYE>^;e!Co9|G)^ z0nM0agks2{@gu}d6HBxGdcV|#rP-3Pl?kaT0dIR5;Sr9-Ns}C(z<9HtaSri#d@o@a zkVpCt6$y4bvlKgFPJ7pI`{t>4y@!A~Ya&U}A|`gwQ1%~vI+@~kDbvKhmG#uNFRPU?<$aCf#d3$toLO1wELY;ha5C{D8Iz6&o|h8Qf}+LGD9~w z#6(){{p$Xj%{ugaU&y^}#q5gam))K9Gt@V76S_w(a0IP%tu-7n(Q=L#s&ui<9H%4i z_5Qy_lz6PV))#bCy+B9le)!R4bHDzSL1|P(isaQO7(divUNA!(H~3H2CBx`b%AcR! z#er^aru!{L$_}1oN-)=*iOFATcr#2-W;_*GI(tBO{CK zI%PV=MU`+OtEqWeziQ84SKN<0`g^qH1P>)EbLpaMwFx6)pKCX>M%B_=w(Vq=f}<{t z|DOjwioeq=6-y`lC$sn< zw8URwu9}(u_}#Xz!LBdk;>Gh1)WZzt^K6gz*zm4Z$q=qSH0pma_H35WUWe^$dSwfx zJ9j|V*a@0#!WaH$)qs*PL59+1c{t)C7QlX&5Lu|JW+4X+I~1EtjK~!RzCzDi>Be2p z(Z60yrsr?$5K%-@;Hku%s=a|DXl{1*wrw+Zu#pt=v@HK3dq!XWSPOE91# zC0RS5qcUX_h36=Tv1*%VcUdmG1i0m(@@e9DsB!<0fRb*UVLFeZEP`s~p61?4&O(WV}}7BqGgq1XkS00NIvl@xPk@y z&Rh6aV=r`eWw_oc3o-TXkfC?5!{K17kBnC ztt&)8AAw`ruS+K>PR}wR>w>Sw7S&2TzK<&8mcHYheUo;o^$(vwq@iBaS>DIW$_yP! zk#mhAz_J{89r6I`8hX9<{LI0R`99M-96~EDq}%V-?Fs!>9}>Fn8)>@djrbniJvC?* zxW(z?ue_>xxn#n$oH_FHk1)$90oy@nJl|7rM?eu1Y^Z|{l35#4}Pt=WlARE zJh>Z7p_b%@tYo`&k1v^v{l|;i8fTp=qh-s2YS+(87*`S}{28VZzIp8`o`qBleZ&Tj z!V@d+N$6iCy4lQELR+xw$)#ob+G@);6;9*ydrh~M{ z{_jS`c}mtk&#sZoeu*H@oops6yVMRK=PV`|U=UN$*n|1+@V{98GUZNqWrK*q^#I)TkXP6F=yTcGt=fQRrepPBd!f z4|2|g*-kdQKVB;1Q{{Rq!rs;riMJ=-*GF#<>GpFtZnprb^gV2Gi0l0dUN4Ec&$>G}( z$qN1RWG6Smmi+51h9RPTU7uw-hivAU@tPt>@OO1gy!*R@Z{rxuueB`eI=qPUT!IOTDm#x3?>H^+3bsu1%DUWYnAF)WrIp`L!@VMq_g#AGX zM^geXVd4=OU$J`_bFrTBeB-9d!-}h=DlZvPetM@@CYPUj(*0i4Hn5d*Rm% zncWD>tsL~NHOo3cFXQ%?>@Iknssy}8x&+cxyr;XNou_N}9bbsr>vQZj2aZu}pDqWX zo;rruXNbOxOI&l?_S7s^qwoAaoqAwH@a%ema3MbHq}Ir$$I`fLFTreAz~LOtw0%wt z>syz?ci#K~^^Bt!+*im3uH0c}Y$I`J1r_4my3J+Jijk??MB+URATT0?lzsC@MeE;w zp4b`lB@PgXiu5{+?aVBQA0$;ol#7~$FQ)OK4i0y%)PKx;%FsFQTH=%;SKg#dMO)T@ z|4$1u?%|pq#v1=kc`n&(!}e+`yKj~{D0ZWZ?>wHcZ`R$!`z}R?*a*rt=^{fO4E}QB z{-hFX_5AHI-mAd4RJB7@)}E1hwxPnni*XC)wCgh}_oE`O?XSiZC=@j$OY~~7jO(d# zYCr<(|9X_;&LP7Z?#mk#ECIpOAJS*q8?UNm?J%znMzO|dG6%fpe`h4+e|2cM8B2-p zt2%6hEh8KXZ}1ZL3c2XJZGX+OzEZ-z7!< zK$Xkt6VCN?=I(eN-EXvstG1p6854Y_Lp}dXep>>--mcE5mCocXCNPNFI3EniJIKIr zRVlRDg%P_3pAafaS|5h)HZLA#r3{+@LK2*~u}P%9*0h=E1;--V4^qJ)4onkP^@qm0g!gS)1_>UHfO8}jvBSs;l{I2LkmnIypXZ2XtxHi*Xv%>v|%()OAiNN-%{dGE3vJ z5D7&brrN$)I^E0swpvzYii)9{nJ@{(KKb|F9+ip3YYbDvO~3Cpz50yfBITFCO2Kt< zBVY(T;~AFT)CADTP*+;`MOSps?LS_p%(+aR`_l8GxUrn~c=pQHym-VQqoHc`_m-Vr z!lK_9cl*?+`cjd!Er({-SmPx~nLaJH!Pw{8)|WJMoCrSlx?5kb8W3h?i3WPitix!R z_er)Sy-pdIrrI(2I^O7;)f8L3*EZ&QltS)N;m|t9Y2)0?cZx8n3i*x)#MHSZ>tYMB zIb}jrD}`PIQQywH(&~0I3LHikUbjwM0-B&K`yU9zIr`G{@yKhF(Q?GJ^7OJ?& z0CAoN$>%PhNxv&fa;bzeQ7k#=HHgKdi3jF2u&xJfauB08A47%&JCq!!sG$z)@^kK_ zjmIJzz}+xSG-@r;rNm&Gttsg6Px|r(6tfl5{u!6w5N$h{ksgH%eoH#Q%e<_3iQ|at z)X~%cYmioEJIibTDEuwe*6%!5z;@?}u?|;!mLq;$>Q!>WYfpm3u7Dlr!%GKISwe&mfTF1-X4?JR%D~B(hb>{P@;4)z~@Vs?dI zaqr)?Y#&t$!(s<^+F*+Z+Y);3c}$Em4(?IX;*riAo4;@s9418XDVChI{lZ={%fS|d zWA9kJD=LMEAp^O6Y9Bs2M5LZZ<@EnY*IUO$*>!Ehba&^_-3>Z)C?Sm?(j6+Pba#VD zcSwkopn`N8NC*f50z(d=4BhbU;kvK;eV+ICz5h8QFz1}--g~Wetm8PAngre5>sCQ6 zA(gg}o6bPu;>HoIE^1-4uyNwm?^T$`nn*>!n^K^hi_rFyBmO*EkLcc|&KHWpt`nKF zlf^x@j#bi1@Bc&rP7=WVq9P5mpL;P9*ahhZYGm6Bs{5 zI%IKJ;iD$`HbmtXmI2PSFDKH;OAYX`x*0R)1-?w;Q7~o8=Z>JR+g$_YU;m!MURZt8 zsW_uhwAg@&_nlaiCC~?hb6Z(Ia8<$nPzDkiIi-IjRn9*eM@qj15HH?KgfC#;&71U~ z#uiE%@mSw3`ndD%a%8#3e0E)Rd+=8T1;iJM&woM_h>~?IrS?;6FHnHFprx!L|JC#$ zu^=oxIxgr3?)T65egBaxd{3_Q3-ST2GukE0m&}EElNnY)m};~QG{^D3jC8B^^+S3i z76GUi!>BR7MjE5?xmd)0Y{$dCr&;g8-KXilFPj-2t!Qp>{InMyW+;jnf)rcieQ&wB z@p=PfUZ0t*(LORqVSo7D()Un3%}&KP)!8aAv%V`H$?!V;a4sr7VW{~?bWrs~5gTza zneLgypRZ3E!r#eVU0m34%6!BztO~_=aU~odhFQ&HGuHXNkc2!YLnZA}9VoSC6l{x< zTpU4Sx2XObFSm6s5pJwiM;c%$|Lhs*YfG=`oz%vmF}GD~+U-|quSIOH{di8FPX3%} z#;wxFq=C}F=cKy7O)e(9{nmYsLGjjEYrO9c`tTPwnj)$Bu+fVcSh0N)3m&dG9uc$% zGxU*ejvAK`K8hGw=-?}v`6=y0ZnX9HQg^d6XpkO%nqo-_~8&Ftd;7+FvdK;RN$4e!?wL0fw z6DZ3=?0{2N68iPtN5S5%a$aq^-S&_|cf#&Nul1kY2>G>x9#HnY=O5@$s{zwMVFJ zo>U_mZ>F}ut}KAXfpZ9Ub}x5B9OgjG5gtuWkImS4so&jdLfhY%YD(kQ7(yo4PC{i@{^n_lTy{b^bfFJro_i{h}w+c{?`3c0~t9yl^L1IbW;m3acd zi&wVmZ;a=E!iB2#BKwcXXUt?$;Lp+m`$@ZoEI##{?<%M)X5C9Y&F)NjPOUY8Ps^qE zx#A$p27Ta3-zkF)Bt*^&UjDjlzJ9N;PyYU8MF*~@LXUK)B?f_YzgzHiDh;s|IwFNv zId<=^x7SLEZm);_snu$j#GqH}S?GJn+h1l@Dq;=V6c_I&xg!y6F}2(Ghvdt|u7bHl zXi^I{nQHUN*`9Ep>Xsv1GZSOH?E1ObcrU?T5e|>iBb`U-y+h#wV){%GD9DZ--w50? zYJidonCx*>ZXnsm7P!C<46(ZbP|xi2>^AD}GJk)e6c9!$kYt9|fEMJG_kW!PJt$Ds z)aN@dsDN!5W*PLSK-IJN-OE*tSW_vBbi|5`Ocsk9+Erkpp?YD+o!Hii-`wZ)7CDh0 z{0wycuP<9w_=RmEc?!*bX}LnBZSs+VPi|Z_2cREAyZlY3I7DhYDF>u1EFeOfJW?+} zdSqCKg_7=dKd4)0rAEu;y0QWWh*Whb1rgxCev$#ZLL4AC!6P1LUwdhpAmxm={Gf2_ zYCkl%yZicQU7G69-9E*?w?-#Dcgu7VsLY7*Brvk~H@h3L_3^`xNG{bZB6H(}|9rhk z;MOuOr;Qyj0s>TfC|rmG+4R!<7ek>cQ%Y1CmGT0gsb+)e@N2!@tXmYw&lLmhQ5-SNYI5KN*{z6dk)u0- zzlt&*&Y6$&{?Ne(+~j|M_4e<;LXq{mRoe4-@OC5}G6d$n{t<@kwX$L;bs)MKhC!1# zGAk>KqRyeyx(l-|M{^f&jbGO1U^e6!?1J{&%YZzT?7ZxoQ-9Ef?om?wzj*2|ltyF^ zfKopjy1_pyK9UUDJ<>kHVTYZ-^sD{g@^LcvLt)yN9A|^}LoO*{@4)~LLOO8;z*@VK z0JOybz)fTLAHtJgW#3TdW?DShJXb%n91-<~1O5s<@ETOirs0<78r9J4kU}EFaFHx(<=bN0Fr8k#?{^R2-7}$8+PyVE zVf$mcDgX@E1nuP#5Vt;O@MTFy0{algrC~KNKv&TG(+qSQz`oW%SEB!Ry8b;F_Y9uf z$X&Af?)vn!q6}k(xRdgB3zLkF2HTD0oV!r!e{`K zPL?FAH*UpSy-?(uy(HVlKq^L5k!w~p;k7cdIN`XpEHvp}i`P6zI0e%}RlI!<)(H#c zLj;MMfk_*V8BuOM6Bx{Z6w8ET;M!$zU`8XU1AtQU&UT7~7iq0I1njNsrz^?|_LKjn zPyoXb`@;e+Lr{ANzX=hUeEYIT2EDw{jmjT$wO%NRvIQiIxkPF1uJ zlelvX1WB==)&2Dwg^i!(YoJ>puGB8*1)cphAGMdXthEB)NM0cj`)it@JQ~idP+n!g z!^__VVkPDBpb51zf|c3%HXmZ(5VoA{24oH+Me@6vsW?xVfeTSDxW7B#D5j$kwZj3U zjp1I5Gw4_w(+$?jzod)*ql6B-mBEqRuE}i@9CcV&&K>wfY5K_NMM9pfx2}M2}CaNG!!fX7?^J&SN#eeYkP+S)RBT#71za0F*m*j zjRGEUlsSLZTDA?SiPvr?XA(BSzGa1sS6w|^ z{gOjsKDU-J;q#{}AuU~J3G`mMjaey?$`fCYvabQ4@+(`6?K5fKioY!UkAY@PihTc` zSPIfm+3yNq##M$Pgdg~;!%mpwR{lo#RE~RB~4E%K>0A2;DOT`+}6#Puh?FkP71o*#I@ z3K^>&ts)xX10)9I^m(Wlfh{{$8G$q>9_mg^Zv^RD&jv=3fOkKAJD82VG&6(ocd*n2h!@`^23tM< zUZV2=XTMX>fsKLz>J1B8mURR+apo({=qk!d>BAu8-a!)YBp|&9Tif&Cv~iaJ5w2x6 z9_|aGA8<$p%>2Wf7PI~cn8`9bocoUZQfT>J@k%sNBRIZdmDz(YPv50yZHf3i`w{p2 z>ioHI26jpUMD|mDpCPq3|DzgiWZv{`{y!ErlAuhl6oL*k>%I2anm6#?YQp}g-K?&s%Gib8F)QR%rZJJD;^GJQMmPB?e6(DTz?VznhEMK+CpgC7cAqyrkTK|mjI zU)~`ve?Y0pu+~gl_Kn%+I1+f=;1x_lsPzFSm`K@RV)Xk17ZSwM1#tb~D)S#>3_)U;9mJKqZ1 z+~lbaJ(XkJqoDB}t!x;U1lDljKst1v5g(#Mnj5aw>@==oRV-{xQ?LV=A*#TE2}!;8 zIG^Bl7~|FucUv%1c?M+V=J#OH@`QK&538iNz=oOa1kWNNqF^MX;&LVb>}@p|lvUnY zN}`<>eI&sRMdS=@kgJlut0-2swZp??kH7O|d)hLD(mHZJuxNxL zli3|?8aU-`n(EGocF5$h!gsR{TcppVyV8SZgbb~Cr8>SV$J+Bt*u_{DFfrlOCwbMNTLXuG)+FdkK|oR2%)FXDVrOidLPgH#3_ z)j^h;jGhK5cLbDmIUBcb=FLqis1);vRg=DMzE07KT3`MMi3-9c=5sJstbIe<;2H6; zWo&!nk`?-P!JE;TNFw8Y+VjWHil0w2zO$$}K3NRSc)rUqz)&#o_N3eV+}nHN^iIJw zkQ1llpe{&VnJV=50U96&TG)x!-_IL5clHTkVJAABYN(+DZy)TIQ9x!tx}^ZJd2FZx zm&C-}m-FVQ34nHYM$x@a?RV!R;R+3a`jWuJkyx>!KVvj&y00}TNm*$;JdU=ET89i6 ztU}&yD2J7UAL^_aI?Dq%DIJC4YtygkTq!CnK)CugLct@#1~CMJ2^%V zBJShkLrXs=`XFKe9tv{GxQ4#>D`qs-y{&ZZG|#)w=%{r2m30HrE<$P3_!#a1?WHPC zNvN_|WDJ-*9Xm7X8n=vAPYVo&i+VAO1^`hi{k<(kj5W~1^?+%6T+^IUd;vdS;~Z9$ zxZRe+N-QArRDzXmrXn-XCJZIfWkq1h81M_;4_q{nC?68IOUyuSeyH#i8KW=-Sxb6W zvOq;lvDgkcEMA7Q0oB7xi#QE2CI$*7AerWDvjbLaoOdm1zu4Cyfj;H`0DTbXF>-U6EW|vTPr-K<#-3Ove zLyJqVT`jMrP+t$Dh>E%YHRZQks9g;5a)e$Kc+ZUaX2!v-nIvJHIt1|CgA`TsDYVjZbyTtz{1f6=r7DIE|*i#*jdO zS3BXhS%`|*XUjI)NVE)U9ydavB8B1X5UR>!Z-WWnYbe?aKgXpXJ%t(yu{FBBzGquW zBk9UDZ8xd%TOZ8JB9r)Yo=$x`#MzW_=UP}`D%}%ai$^T}a^k@1>ilTS*^l$Xv&pJ7 zQ|zRTKo@ywX_JD{vwhWgCH}@J(;s!NC1t0<&!VQme ze(2=yhDFCK?tu%<-=hJ?#ZTuxjPJ{gU8F)-=0`!LWCMosse9IIaw#d`;Do)puiyzS z?>Enoz(r-Ts>6^gM=;AkhL7q}{qRv)NT`YOXpH{l0-!cgL!5kJ=?bP*9KiR$V+&L% zUkj4t#Z$-N0}Twogd^3BPd4|g6-5FNRyt~-z6 zVWO<3BW5Tzc|-#CWPlLA2=9HlA9QTn=CD|Uw|u;fLox+BCQwYIJp?mQj9+xHZQaOy z+mLDJ;7}9PPg_&;muEL5`m=$8XBefvwHz~Do#-yc-%2bXXTO0Fst@hQ(c`HHcft6a z?Hg}|AhbhLforw@MjIuwc26w@^)6y-#8`v76k$;PMdR|oSgc?uLL>@XE=s#kWCbB6 zp8OEN3FR+!49bH(f+1wW7^I8jrpf5h@-|Ea)%^{3O-oQjp3aJ|>GfPYyH_H*$5ak2 zmd>>Hl)llD6d}4Blt7NkxAW&uhOZVen!f=eF2J8w7n=MOQ@9o9?S#ap0 z*jNyFx9_Yim+^IsC7MRLe}%hSyIkdf9CT^Y#EAO1;wcus@;3E|tvlVmxfv&EyS=Du{7N`B6l z9bfsrL}9<u5O)lnRjlNEuKY_o812Wv-7MNuzrI&2;jnK>7b>CbH>qt)25~|- zIx!AzSI&IXRfa~;ZY4tqz#t7n+;kppCzf!NEHxj2A@orvx_ReR$6)mJ$Hmg!msk{{ z&Asej2{Kd^v;A1$&GK*|M)qtH=crp8LU&1*t}32V$T>H zRPa(O2K>9CiCUB#ycmuMeS}Y(DDz$V2L&ZJyRMZLfjeFEsU0^a&iAq~og~qD972;o zmrNy0TaSr3MNMIIfM#W=q}xQYs77KdJ8`tviKw91CRUS{j<-D+P5N3E7R8cL@t}&3 z@0)kkj4$T=^mB!Rtu&pS6L=RRfr$6`ifx9K|IgkgnyZdVG9okcCu1M{6wU9KViOnD zFyLQ6*@`kptQ^CXAK$4N)0%pG8KG$m;e##J7twR0vb_Pp>5TSk+||3ch0_d-rj+_O zKO?c9UYnYf6;}$4xsGW+nJ1;qd~GN~P3a%!N1JVyt6uY7<*S~ufJm#iN`KzTGn&rA z@h_&hbRvGxhOm>2q4?h!+lp_3h8{*WAWqa8nvy7#P2hXoku$q>5!Q8|0y_z>QCYT3 z53u|i-Tm3x>)>=NrX`OGYdnBH`G?-Wx3$a3K8wI_3Pg=P?W-Q9PcJ=iI=-ZO*=I^ z#Bf%X4OMXWcmfFu2azs(ed5xT2>bZ^B{qT7 zKsz}FI`^Q9dfs zspC6>a&K$a!ir-{MAid08EK4WtV*BZVI3|nbq-Ov#92rXyze(-;1`K91>&ZhExtEH zg8@>-?YB#p1T7x9cG>=2yPQyjBIj(_p}9776wF{0 zrzjhAS;(oi!YJm?vhMByeebb;oal>Uy<{eux1*Kbv@iNd+^~~*dX@Lv1rk}%Xl#dl zob|aI8r4$>jx({sD$*|hcIa|&JDo9=z93=PC%gQz?oc0)10nT6Q%Yr<3#sq8%Pos{ zd0E8cMi9?%PK8Gf6SCOul^mF^3yfGp5aR^b)5+p+64cXKdOa^-sd@60e`&Sq2KqjE zb6ioeyO(tz|FztKwda#$N(rGD3-Y8&GIGs}bF|%Z(Xkp=x&d;ylQWICy6{9l0rdjn@j!@7!^t)EQ#7g1tG&vQ6qlPP#iW)()SoiWWB$m4WC}1 zVGR`-oJ@(HBtwhi>SiB12UfWt~z2mX0qRuPe6Hh_{ z!YMHtD-@2Z{^s`CzKjFU`3r_=GzhfJ10-X{zDwZ`1cxH3cf!H@P2{%$(P4k=m0qL* zJF0)uctqH;D*y&C6G%?2Fq+M%4@Jkx5<(h!%5A~5M=Id4F!w6s{ks1s8hjQ{I7&y` z*$tQ4SKAtx8Ss7?l#Tm9^teGj;C>@UHcVkcwH%L_+XC2}342Mn)5QUY8P zMbeitUkTp+HuC-=&tH8MR8ok3H|JL}@b-NX4E9O=_+J{2V!_}1qH?>C(>1v;*#t(> zLne%pS;I?^2W=e3+s@mAHb?l*4~_qEO-jtKNr4YPrc!i@N(F7xHvQroW7_C1q z5PAKhQm6jw9Ba2pnrAr(6j{&j9tqFb|NtkeWCD_vsq{y@lwgd%pJx z+xT56(3HB(a02hAV9(Y9qcIG+O%r~8{#1aBM*J~Y@6})D0V`pPIn}OLIWUejTZ@aI zpWw21I+i8bWbbL9?$~V#Sl9n zSKN7NX(`nIXp1)gP(L!Je0IF$LL?f;Dp90N|1{_??PPrpIY;~_cHGsr6Q$ISL+HHh z4edLJM$MOUE8DBvxDcEcrg+M2LECTdB%Kz7-2xvB?<38+M1Z;oX}9A3{JAPHN^S&A zG$&VD8Bcbk$`E{|VxSCyFLbp1^#>X~_vL=v=N26O2VZrafH3so3P81>rW3&Ii^oHS z517LB12y6D18C48Tn{in(Saz_Na=62LFHjC+351-MCA&QC4&L)sPn&n>jBKDV^)I$ zxml2_o2LLQXr7=L`ygMLAHue8n!ZfHBkOD*5x zX%xZ}Lwj3ZCSd75gqCDRK`bqo)Gr%wtO7WxQxxzi5()7kuTRj-L5R!jEC16_#)2(A2hq_39B>HDf^Cb z7QQ}Z;Ll%>Rt04vnkfMAo1gsB$1OPol&*$ie*qWR#3g{ydm?ekOFOh-KfioY?|8BL znbc@QIZLSPQr|D+3XIhqkKUSTEW5+T!>UUFpQ_2aV^9`P>hb+4=wu>6?jtHl>9Nr# zZ|A_{UykraOQrjP@T;tAaQcDMsvC+Ei?D)Rus4B4$NZe+ej-941F@B{Wn;!pU#9%&+-p`Ll3=n979ZaZ+kg_K%#CT3p;F;cF>&0#S37Zd`z z?PL*4Bc(hC%;ovd&LM>=D*caFN77IyuAxyi(?wQ)g$T#b%6fe-*%>0bex(r31*ZKP zpehP6!?O%|1OYnI@gv$ig!{@*gTym1JMnL4GAm|*$nJDBn7icFE@hhi?rFQ&GY94i@DCEzMK;PqdST=~<87KAm)FYXTKO+1azsu0W_e|U z3!kHbkTJ1%zQW(1&dR11}O8?^1z(>Mu|pezszw7BuSyV#CQC=_fP75>%=t zcm%l!q}B4#IT*eDTIollP$ux#vtI6x<25sU?=to3v%=m(zD8W)#_vY1L)*}ZbWPv5 z5aZX#_;Y)5t&o$zdQJnF#cp3aVE~R-fFklNx1vGD))qrNJTr_A2owr^HJ*s2@aD^* zi`B3P9{y!$@ti5JZhGZ|-sd;p6=Es-OLG)A9vh6*RRfkYuQsqcm<|O%DAnZkIJ(Vd zwDe6cQ>d@ayotC*7VdS&GS^b5-`D5S;!f^Ztb@;JA>PqxO#R3MNrZaY8$Q$+)~|a7=Fh#Zj*LT( zPrWumee$Vn)>K47)j~1>5;Ty#vIGXEfZX)%(&i>@EfXG<__>%C!vLy3T?2x_7Kbyw zd@=BV5EM`mptr#bD+WS$jiusL#k5eg?PA4nSKW}>F#_4T=zm&U| zmoku(8dk|<0=a$F;;_^n7)T9|i~*`@Yv2%D1=A*RygF*cdh3S0WIiFW$GUQt{@Z|mGIy9 zSB3(Mn`#1Y@qWGhv5&EcVxj8(DY5@Vv3P+=GE28{;d=k@BKsvLpZ4G@HC$rxd1JRm zJn|Uhfl!@q+-^9vP7r{ZOQ}u=&+SdL(9y^U|IlRZRDxNLYY62p%$dj0ENaV!YuNwR zZB7-FF8Z$dxJ(tThx-vb5 zY$sj%u__Y}7{Wc(M#~=Ww>8jmo0ncpx%fW}u`+xxT*E z5IzH)ukjC0A0L^5q2wKsHhJ5?T>LG4!g4<_M4V#SEa_=7`zId;wag?k0iSHYc=rOC zttEwWLdPuZ?w1o>TRu<{$eDfiH4vFK5~1;^`C8rQkSovx(S`5;7Y4i(Yx2p(ef4~; zIo@X8o#$l!MDuYUPh#L(gY3{t=_~0$ZMjIBsQj{{ zRvYam={^CVc_6F}c-CvMc@x;G&uG4);+`_k}! zjpH;ne1N4>L%bY00c9=KI+!*dh0nOIUq}9`9EuMALn3j4w1muX@0KJ8GVBcWJKUbu ziPE)!v(xPd1qMzw=tW@{I#OuhQ}UM2I@^P39>|d@NSJ8a^WK<27=$qB;@HS&g$zQI z42G_Ha071s*H&QwVNOL`7oXDg@i|9r2&6lBnPp~2V%N!hXw@Gc`wH5+lMFdyxv|s<}WmrNLGVsMybkHun@p z(eB+FvU?!-U45J~!t(QGM>khA?X=l}?~=igVfh)XdyGSz)%cB}IYtDBh;b{+t(LqWUF4{IhSYT1&^kFTt>^E`9(?j z)vhPK*!|U;CJF)iH3JB-so&8Sq2%9|XqLsDd%x*q8toj&3lK}Kt?8m4q-wLM6Rw>I zf>;y4`YO^JS~AHBMAK{kNw#CTi2A-}O}8)Q@-Oqlw{3}Yj%-Qu`~XdLl{D|EjPQYN zzDM={uDI>`i_YzkrUAYFZ{TZBap6(b*PMKNqn6*jA;g0ekkfv-0d%>Z(Ro5sYi@SV zqV-2*lt1&t9JQNIu%dANDI5NPwA@gdEz|y6W(hDM``VWf&f5V2#H+|s)tfeXlN1>@ zZm4elQtL2Miw8zz8E`=fXhku4dU}9$M{sjv*}5;%Rn(}|aU{xX>&I(+;K5u89W?QM zCTjK@jG38~Kt4|Yya7q$r|$j*d-F*GT1KTr#LoQxWXE8z04_-XD(JZ;s6hIsN|vgt z$>aIhUJpd}Km{c1%Sr1J7bBNs1O}n$54HotmMY&>3Sy@x6NP9Mtly>Khn`NLfwCW4 zjE$jOGDf2Y3kwo%$1T!cH~3)KaP+^zH`tVSj(+1k&O0Ai?%WmCGkV9_Vnsna=l#!8;vf1b^iN zlfyHpZHAY=@(!WjEOef5no50185Wdm)Q<5Qedr=cb+v%g`tEM=Zo_Y@k$CyaNdv9y z6&8`+zJ-qNw~tT`_#D5G-2{C*h_FDZef$yYKw8)sc+cR2r)^4%bY&5J>lz&pUrIp& zs+_m@M&*9ky8QO1QiaZeNw{S>@SGF&sE37*DadCEA(T*J;O14cf5#3~G#0Qhn<2NZ ztBMh|V3INVhy?<$ok!{cHIR2xEXtI#rK06qQxm9%zOqHAzsV6Anwm1=f+CFzTt))% z?ti-?nKR>im`HG>x|h2^YkD1jTCGX=-wLP*SpkKl*he8NAYxFa4?EV|RUltrp$37^ z`MK%|1x0fW3A>is*Hl&l9zGY=e%!y#^W- z;^E_`9A93qbSb9KaTm?6qyeZTQKK+i>FR|mDZ{x&N4vj;m$#D<&duAC9>=~HHPc)r4P1wqKw;34|ieJ?8eTt~tbTN2T^PxI9OS^xhAIp2>R-Ghad6qVLR+#Ew+V4KS= zV%>z2Jg@bOXg87WAhl2xCUV$0pcUC;4!%BM2dtL=3+L>>7MXm>2vYo#!33VkAD`2@ zC#yl}Odv#Y6A-56#Ti@SQD}j2NMu!23Md*XuEX<06n8`Hk6loJr^~~V-=+SXPBs|i zQewE`vbaC%E$c^`fb_mbSYu&fftMPC@cQX&GithAlVP>KYLseei$)6e$hV#zMe~5^ zl`soSU{`B=#$|#<40#)gGm_@W-N9($TTo~N=y*o1`NG!Wf^Hb2kJsypi)$!50-2?Q#?AtP_~G87X-Ose0J^_`uELT;Jee84mlWcTx%5n?dex>x50}65 zla=SnFEKo{08&wS68aBhBKW|c;`B`3K)sZa@mkX#a(+n(q;LrY$07k6!WiU>R5|%K znu3he+8J&Q4A-bVyGAA0J&`b@qggx7`Va3LL(|Kyzx?_KL{#`DQ!*Nuy&G;j9>B_j ze+wVts*7~HYltu0lc;j_a{W;rr$P4p67xJZPh1kDxV~)hxLWO69$_rW@Tp?Tg-U22 z0WN9fdwUx8LTkXI^k-Z3(}0Du#J)1 zL!w=6;9Pig|L4hp9l8p+&6cytka&f7=bm-r3}k-!xg5>6MSdzOsx6qwbtWtFSqd8? z59z&iU*U(I&ITw#0Q%ht{|8coirZ7zk`{+55Mu?yh70ccmU+b@c~NV1&IN@&YMQ{RPG%3j9t9zC0fySzO+;diYo%of3q% zI2`sI1QHFDmX_|P3(?j@Fi7BflhyXXe`kk-{x8Am7g-0-6t3Qg@kLWn)PE09@JGvC z0H#~M^(SipG|{bWK#eQhNSL;XEOPfThAn)p_Tvx;cMNRoh#LonL!b0rL|<4h4*mcs z?}TCX8C`uf29@&`fPf?0SbqWVnC3UC)qN2M#Df4jpftIM%_m-4#O23a*`Np5 z)zQFE)AkTPzFV8PdjZNk*l>;*Z`c7yxr?^;@pW-J-C^MxFA6|Uu52}V>CxqHpT(Z> zaZ->+sHSs1(4R5q{&stZ!3CzegG*=6FK7lx4uEteP;uYCnpAl$558Sow4LO?Tmbz$ z$Ya1sylknvzZ^OF&BTNTplaAsQc}M=8;Qu9|1Ne}(ozE3yYj>*ot>T1fN7?kx`d4r z!fRL+3J4c9DwXF{q>w|7j23_J&7}QBQNYZ_fvIgB#QD+V{MiWs4}|_B*XZDjwJb~F zpmXo%YKFku47mI#fr{ig`-f-0#sY2ZM4o82^2^JcY>U7@S}qhFm+O$mr`&Rt_X99&Jy2iNSfV zQpTW)EhHo~Hg*ahXA8XC8v>z%Ng)QT(=gzP!fRX93E=M_b&fg<7cz*rzUD*A-d(Rzus4=!52lPQd*Fp7kxAa7Kv56B zqxH#?Ck;Nk7Lzr`qljc%=SI&p4evj*M!4^Qd&NV?wUwciDO%@YeExl(E@n@p zHvR(#g0D>g?tzvApB*c>2RNul4K7YjRjdDEI>lV5LE`8yDCi7YFRDtP89@$pqQQ8$4@CA}3kV@Jl`Geu<)z72;y8xZ%?lTM>!tLiFJ5 zwnON{QMa(yuI(P@mqE=g`AoLKhy4c)IG55DSh-TeHF?`PVkl+1!}+*i{KPJqA*Ni% zXyqwj`M&K(W^BhD(cmB?_qlIeQRbFGwOgQ1xI|#(zXy^_k{*C=Vc(qXx%;Hm4m-^ zsw(Y@3D4-=kJLyyK{D#+d8hQ*oJ31Q7q$k)^B!Z{-+QB2OdM*L(jTyb^StPO-=!F! zmepjbJ8Kv?N2<#W12CgJk zLiKzP-zsDYnlG&MHl@P0;4*^;&7kAvk_6@3#8mRg>{SM=$tho`ZL4;Cdj_wOzT>-x z-1psqoA@6v6{ftO29M9~e*G&XI=FTD@_SS64u}Dji)xwz|22bZx$-yQegc)gGpzgO!i8sj3p3wk=EMc;Sr4YOF}h)jdE!4 z)j3b!U2^6LF&JPf8a;)}Ae}0KHL5zthazBFxBskoOg-WX#1NxXDLi5@Z^odIa9Bdj z8?FoW+gJG>#=%mOnJH-g=6wCpP~#&Y*{I)tsFf?u^Rsn7aW!2`>+OT_aDHjID7tv@ zT!cbcJy=#{i7pIehh*KDD%t+0EG8PE{MIe1=$LiwU8WIEv26y`WIUlSl|)_^hbUtc&ak4zz1dXC3>>h zX)gPGv~oWuztB1du>PwsaIXsl9h zte%z$%V_R>?*XlomxC#`j$FwnGIp?V3kw@&Jml^V-2cL*!R*lM953- zqhO_RiHjDY!*(`XJpxC3F*r+j)%=ufH$eBB;o}<|V!Aa=xiD@ESKQQfA=MSF|cfbAgEgK4D9EUGOV zB!w*J>EspW>DHC=4ThLwCiYI8HPuQ7?1c$V{s`;4o_W`vM3)YsEFxHlTG~dpa&t_B z(c-LjPO`G8X2cSZiA#c9KKV)9YDaR!SN?qfU9O>yQjyeY7!@5(@vM-B@=LPag zkhT`Dlh2v)NH;rAsELe|xlWWtI64@@!WIb33?`K%DLghG43=_~JIruKYn7Vx9c0T0 zawFX3hW$zaIk;CCc)=MJ^c|vD7ZOtWS0Yi$jkq-U_bj4+OYm|OJt>Z!KskM2hBV zfSX$V$bg8mp#-uW$#;D;_(`?4!KI9t^U9xqc87nN5^~Jgo^<|a`c31rMchH-mHE9T zJsLsCDa}BFoxThNz~{?(#8D{$#7f9+C~r|;y_a^Gt{hhZqMJnLapwCT(8KL+(EPr& zG3Doq7}-BS$09^iOGJ62>cX=}wMT8h?!fGU+-En+st+D z_hO9A1+=HXn&{Nm0jb#o-0VZarPjVc5(*UY`f8vieG@r#__DaC;EWEOzBIhag(f>G z3nm1P{rO_Ki0-|!j}HMcnN_vUto7A2nk*mub#d#EybG+(BymwOyBz~!*{y}P_o{9MFC$HXLC^o7VM`bxG~r_9`s zZqRoXn5=zl+>trP+N-Seh4!bQ1Rk;GwC~Ca>RpPez*a_&Kc`$LP(oKI!BC5r+-Wnzc z-F^W44WvOB<6$4w-&;H9XiE*$J`ST22vtuH^;kJNOP5~i*Zdy@v0NyLp65%gOWp~pRLk_-6z0u(eaUjB2PqXTEpj+h8)*Z6~Gpq4{ z?{dXR{%It3@RxF~UI4BdKK9plx1dnX4&C|Pdk&pLCbVQ`2BG%W?;O`Ftm0hj+O>Cy zqD^dIwCZk&r%(vIu{)od;Gq1NUm5GIdE3LiWr=9@Qb%md4O4Dml?Q^gwPoN&$IC{g zS4H_T(U8>u@yYqqW?Ei zJ2Y&EjWb^L6fNLB5}kvcZU$8Sc)|g>I8;`vJV{y<`7Tg#)&SW!T;0Ss*T$>;Guoux z`};uh;jlASJQLtI#Cd#~#KNP-DlONEB~JRJy>iD(lz?7xr{mn9LJL(HV{bDg8Y{jY zA%Fe$URt^ZiTHI}Yb%d!2bEG>5JGOv;8MJk-W16>zX7B8#^=xz4wmS6o;k5bb$YG& zeL?6#xi(dM#0-j*O2T24a2_>Ob{AY6oX~^>&E~P^xjwj~E`?npNy@Mo+*TesJ?@vc zu+=Rv1j5%m`t3b8U@gWazPus$qi{|&Tf{yT(b-&H2yjZCIE9zE#DiLbPj?7OVJvDhTQsBS zs-|N&*AoS0gS;9}sFMt&R4}Sk!b@40KfR`=1~25~*)v@9OhKc6Yvyupw8LNmS~-0{ z!K*UQ7(n7ebr}6CJ!U@I>jGdEHR8bepzgt()Gl94#ugZj!7=^|7l0)zfv_KJ8SLjX znU}W($mj}Ca;n)!D44WrvuDk6Qfz>wlhxlO zAN;O%v0vgH_p^(*T%`F8R(KEoixkZRTgK(0^O5fog9<;Z@MTkkRY8u*?m!14~-1u z&};>}-T~l)E(omqgszuMQZ)q3$j|r}E3}eCqHB4zFXQk))WpMy9a83Ig$cnI$sUyZ%zyhS z>;TuD$G%;5eaNTMFb%BvX}+z5#ozb!ZP6gnBxL~`W0xcL^<9RIp=CI{$|knS9H3!f z4=9ZH_x7UYh&e`gbaeE@*#Pp`PKBn^Q&I?-#t3w-K-!CV`k7j%8$O_zyyGzc?!_YN z3WCAE-;O;M+(ujsuJ5;RFophqe7$vClv^7$EZxk|DKK;?p$N#(DP2lPgCL@Ultb6h zjdX(`C@qb22}+5i0@B?jAo1F-829%x6=olFp zq3Ay;L4#_PE@VF7AMQH%5*wS%#1%eq|IZ{ogeO&f9y)H6=cFxt5`l9UJgqYWGOxV= zv0-y>S7)oC=9WL#f*q{~jbQUXFM$a&F2R>IN&V53U^40-nbgr!Q97cJb#`ef5IETvW&utglVTM8v!laj*##YdA%^O~#wLPFXz<8DZ@o@{YQ`s_kiFTh ztr9pR=!5CUYw&SZap$TUyl-4Re;xp)n>j1k*F6h>bZdwbX0bH=O8W|EF^imcAZSz) zK=w85G428Rfu~3Hn1mssIs9J`EmJ)zXis%MfuqJ-Y7;Cz>22ccu{TGt*Jg7VNlE=d z1q&E-6QN8X0At1l@D^xMHWi>B*83@$SK&OIv0Hk@=)?&$46w5sK?js#g}T zHg03o-jQ-g>J58qUy4*PFfKiP??ry$*{u$hS87)+r;H5lD8mRL*GZ8$;H|O$ zRoaQO90Y(Hu&pb6C28}f9tdsWGI4<8Q_~1qX|@=E@UyyQqyK4y9bUCokC#YkySFew zi#XfF0%ueO!0{umVk_b^Z!PrF7p%m4{^Hiq<(vNdBq-U91J53nqgAzH@S{M@MhAAn zQ)|ggV@6-V&zb*H%`x%E-eXh)fdE~l`rE=p`!x&P)pcKfM1P`OH^0I6bs^f#M(W#a zoDE-|N)ondq|0YXI!^#8<}V-qO#i+*lf&7(YQQ}f>G_B9!yu?Z117w5mtgFS9L3i< z^Rak$YLOo|85A$tG*$D%>RZ1EW;A$PJ*E<3R{>)@*CBPZ-rDFUl_e20t)UI7C1}^v z5{&gbbtyQ5s#ma;H!5hdqq2o}(}~rnxJQn8NWt*bKKf35l>o@7Z=A3K=_{&t%IXjr zXdtS?jB)kuDGJd6M)Op_+FKeJQ^$yVxU#Vhk+pySt`kZc@lCJA+bf>efOKqZ?0(I2 zU`qXJH1P#c^A?-|?-xOOdU{=Pz|EY06wwk52K???6)_q_X74k^MXTr;ab9RuN0D}O zn`1mzEX}XT;&+$tvSYa3R_})lA%oGf;@cv6U0$88TY8Ee;XdlODjPCS6glo>vjY@rID+yC)DHD&PP2o zEovVEz)sD=JWs^?@rqcFimpyXF^Jvby_TrhW$Ks|2mB;sI{*l18$VZUE81!1hJ=%{ zamw(ke%H%N=!*UxZMnH9`>ePya0W_PEB7D#qPdOk5*COPz`;WFpZx-Lj9^C{BUJAx zzeDX3ADE=Xh5^~e+}8GeCC*x9koCg0m&wZ8IV(2F`$X!BIR z=CoaYL_a&`JZ13gUue5}ZfZ67>}tjZp9Z9*-Q6vZb6f`15S!m2G#SMe*54^S?<1%( zP3WNgpJXF>916@#PlE$Ek1( z{!#hYPOln`njABkJrQ8|l$yY>!H%|@ij4p`f<$H8{Jokgx{);aq2>u7Y4it7mQ@)p zQBl#gPBg&GsRjkNw*YATg5iI}r5k_i@uNVK`CsH5t9vnGuU`UjXix$pzithnH~!Ko z1p1JPew(L7BdlTRi+(u}-U9^cp0Gzk z_#Du|N<4ccwXz3I103tn&HR=5wzvFG=aTEF$a(P}nS=-g@D$NQ&OQFjq@lzIf}bWh zql1z7Kz?QXaX_0bonQ2**Xg%b$-EC<-p&2~PDaTw`9bxrSC_-Givx2vI23tY!Zi^N zJ!YXhih6nvYD$Wk#Uu64&wZr>d`Vb*pT%kwem(!*T#kG#(S*+gBh4nWF^_G$4T6<2 z496>YyL!138IAN^O=p7bQQ}mO9g{SQ3K7o!m$5{p}~d-Pj*Q2zDbHKvP~In zk>dQ$7^~gEERwjz%j}fV`H(u-wsqA@;8DGEIhCO3RZ_9|>KXwY4_`6UC~7Cr2_F)z z>908OoF>;zs8hH(iAuAMKSsjKQ)sE--_bN2wP41*d!M7C=_D$ePX6I30@0Ld4G$cQ zhg|I)K^cE`w*5r53@3~nj|L)x4BM*OwGPM$zS{H!Vx)x(h7g`yO@mV(JpIYviq5|;_Z>yP zu-y#kyUsJDPz5GtW@fW%%E6Q}>J4zC%1OvLW^sl%i&`~lgi@G`5+nHFU(*|uGKo>8 zOdCEc_Uli=a0q|*m(N6`=GdFZ6M^h2pX~@0D;}U8-GBNn)*MU-fsq#^wC{Si`~I8R z7fix(+Y#Ws@_rob+~94xJIqN-OC$CdL&Ls*9>4SC3nmW_Prw^h)OgxQj4%WB zU&h|n?AuoUP4U%W^%wXpdP1PkLHF9|d(%v*ytKZ4US!!<>N3#qfD*78og3Ku|HB24 ze&g;oyZAIATOu#^UlT?fA654IqWo}rQ) zpj=w|HuPSHgN+x}zAKaxP%s|40J7wVMkdf#EUMU`l6LV3eAj-XhITm6Rk?K62E4+9 z82+>q4$#crXMRfnDKo_&xf;J2L<~{v!e1N8*U6)b&u5EaSMw}N*!b6+28nFkLfhSA zY3u8y$AtFa_O0K3s29}Ak8XVn{`8sC)2EyuWl5+5E$eU=`%-kYa0v(kfC-~}#{3Og z-s-2{WZWd4gU05due~&AjQwyLfOQL?mVCX@Q}Ge27;+x*0?7p)ckF0{d5rM5z;E{x z^Dlj5*=9q9>MzfQN??Mbk1Y=dHuNh`@g+92NG-iZFthBU_bv+8SAV|E`;h^T$3Qq{ z`+UabCdgD~SIq)%ef=?eR#Na~p>4;Rk&u2NiiKmxLhdSz`uK_4bZIvmyz?I8J#SL- zs5PfSvkq5NB(hoCQvL`b;)MNl7&BRvY;~xuJnd=gtb^|Rc;e|#msH+cYj?~xPaHhO zC-A>sxrM6C*w4=1=*u*J3un9ir?D*?Eq3QBo5^en>2S{fjW#N(HJqxoF_7=oK6z7~ zZ)wJ@p==ZP4tBQc{{T2(odaeA2nm>Mqt~aeUMVI42Ee=mN-<6kUDd$cZeUwDf>FUs z2pb5t`9;K*N%%D$cKxdlP&!>dl5o@}3FKj|@|tEM4zo8v<*NXR!|($|Sa)0uxnF#W z)+P(A{;k(%XDV$Vs|`l)wb@2-euA?o8R)311xq^d%eE((lQz{3EwU+P_RT3$-@MP~ zE-BM{Uj?N}DAZ#qzz@)EcBbshG-M3F+H21i*hXo4?ORImgYKQV=i+3Z9S7&j8;-pA zggxkQI-i#F&7l%IZz}^?lIZ%|JHM4*J!wrbqV*Ll6g#y9GtN!45 zh5s#%e-M?UU?671{-B4m+>dc~C=g+-+TO9iVgJoXQ>3`q7k}wHOk=xBzkZz}@#;mu zLpzhX!^!dFeOar*LPbA&0S5;WYxP=%cdE9#-v{{|zU~m6`Iuv6@z|HUPV=0fRSJD< z2&D@&7i0O?Yw*HVDrEEGd~nUrk=ln2!5Nb2;$4}%2#B6N|GG*C?zGBIAS@Uca;BZ& z4i-l;2Fk?*HznMxj&1psSrkr?;k@70_I4E5hvRP>)1slFahJ&gvzG*DTO$-$O^z=) znZ>PvAByOq&YPYMP)F>YTJ`d-_jKG!@> zrHoj;7`?q|H2TO*hTqnC*}}=;=&5}=+=J1$c_Qd9;;Oovq!o{lFpyLMogd+jsq|4= zWgndIA{QRysil5#Gi?_-?&-SuDfFH3*us8}38xX$TT&T|FMGTDPppzzWkJR$oA70A zcl3tJ)V@b}OHwoDfn+BeT;K3rn~iDE`*O91JOv$T!mUFjbF1I_(bZ@}#{+UaSdRXg z$$lpwSBFZnyuS_-O1IHN_Yfeu)w@3E2C!IbJPPybOk41)C#S!8)6xawZTr_D#q*4j zK+_%?_D~Z7!s^B4<#&#fn?SW6sORYF+6|gaO1;VfoK^04ZJ;W7b#>{*rddvWAfA5T z1s|BMYPuAfZ9E2m%k|8SVwRG5ls@4yW?xImF)$@Nut}3k95zS95ii&BO1nV2i+W+XB$gb9QTf%;dJzSmhYv=h)es?e z5|ZkYK_L5YTT!wfRn%rpifUM`Lk6#XFOFqXan=X~Ka0KoWwAd(RbCs6UzwMX*nev% z7XX5I_2c=G|7>GWVX({>iLVjD|2ZEd&DD}C(`4p&HABuHCi1C?0!t`h!%Sj4Qarqwou1&6ahy)e7Niw?_jUL|q zSABkOr}soS>A%_td)fCMB0|&4cg$ZmI$l`j^6A$)h=r-e0sv*qKpMfN3;->-kPDtHy<<9Jj5&M=$ zM;&ld(@3Y!0gC3|kT{Z(WffWZus%-D*nA_Tl1T-*iNHCyQvd9JzatF%->&l0vIQRsIPtEq%0LcsV zJfY=gB`4Eh1P2M@g*~mB9@p)b{5U0h!ynl4ANaa!C>Yx&dubo0 zmW;rjgB0`fB zLsE_0|C=U>bhyU0V;l%LZ&>8m6|yBo^)A3fFi+gAs58Yel1-a`tt6hfoT-44aSa)J zQ)@@?p^q==&9lQAx%Y9*aO#>n|ANDYPzEk%nsN70Cmz2=STQbrg%QJ@hTztS$b1CSh0l6zRsu$ z-rP0;+qEX>kk{p(hzw91Y7le?7ydqPFvhh8lQXc63y&Iheo@r}k6O zlpW)e6;PZ0supqwG{WA#OB5h7Xa4u~06ZUjqoX5;kTF#Cj`e#qjKACw-q<+M{LS0R zB>_qvN7?KAh45O5WrA{@Fju*7sFp{ZI6QxQoNfp#Zi?@?xoZ9kMFA%oek8~Xsby5w z0S*ZY*}UM^C3sEOjKN!pR|NPBuz-GO#jL;?Rs{ALf}1)}2JM*=E}s9CPeE#gD(1XL zdGoyYxO#Gg+N3GLMO5u1t?r@zB7w3w@D~@v0tF$BUXI_%z2Xe^kt2fEq>V&*1mHWb zFK|5#SlPj9ilZ)cMwMcrN|@j>q-YwcD1?+76_}E(9M|$D1iT&i$7LIw*GSV>q=AXS zMCi#MrCY$fV~j@9>#uzfgtF`8Fy{YN;{qNM9zrOXNDeGn5>nFKN85{wc@_#V>$^0@(zX+!6*LUgVNGwPWiu}4w~ycW^o|> zx-2B|Fip2kN+JH%T5289ot!aHooTO}r7F}C{q)WHXtCGMsOb!SY|K_OFGPs<7Rw+Y zz@Ifs=}*v+-+C5sxb{}#pe+FRX95EzwGR?Vn489ozfqa^->HG7Q8vN>ST2Gt>e6^n5P4*Q_XU)aG6=lPg^~9<*&@=>*!ZNN!YjS> zC7~^lQOuPPpvqdzmpBVn~83|gPMaJDK5Bj~&KtJ1vb8fJ!axVCX zJm`1X4UbP(q@Q>V*8;^EygwL^*U0lf@$5(1Cx%w1?;emKOx+>?QP(FXW%?79U$Mbp z%lPY~b=TcP-UyiNInaGulJh)iJvmPYlwE z3OhaN#X@iK|!_!$HXy!E(Sx#qbmP z^xO{6-~D0GbB19+%9aA|@bLv5K4mqC(4r2-pURyxvYH1$J&d zH$dh)`VJ5ddXo6Eq#2n3hB8=^MvmGnz@83q$BNRk$$M<0U0N=I?EJZ|M=`w0q-2brl%wYF~nYWh=7=gG!zn0Wz>j zJ`+yGZy`rXNC(=9(@W5QK;pVf$Md@3r3mH+2P?v!|2QO{{`9N$^q@=kj~|A;<#Jyz z;=Jyzp!LXAcqy7n?1%r17t$Ax1R9M)_Jc456>Q&U{srbdjV7Igjs8+VzlL8>`R2=) zIXO9Xu3NNG4B}xRQ)UKk2D5hOY3&%|^dD^oONresHS;Duy)N5FqRj6>?av=*8Q%W_ zjvl&4TCsnXX=V})feg6{;O?i1%oEpwik`} zql3BbsGEV@Sj1-yhQItZWbN0ubU*i1pt)fklvb0Y`5v}>r#(?n1rdNe%Eif3u$=$x_h}_Ky=;jsu7Iytp+0-Kp=+1m%VeQ^ z2J9%;;Lln<{Ny_CEpC^8jPQJR*;8{{De3mGR3-quKG`bn{a#nmZ>gi;y)``V|C{Bm zT@00L4W;rXr-2QZH?1DV(X&OlE3H48e`UU=k>jA~C*rgR1Fr@BGvT9W-u-lo9n&7L z!XM|1s`MrlksHfxpvtx`BgT*w1*LJbaDph4Dr+`+nfYn!(X7p z-JD0NH#jBdi!8mzHI_1!lej9q+hPBC7Z`0k{n^M=TzpA@ara*-GV8xmf7eZ!uPvbK;iWE!24 zsz*IqcO13m*?W*gy811L^SR?->$+S2x15JCms=lprOIOJRR%22M>-I@oP=4fw{jW< zs>*eOW>~Z~e6B9UpZ`Efn!=g9;+N#)Zj>0FGJBof)&F!a>&sGWyCM7dC zmtWRVTPz=nq4Reof?wiVlsv*J&cI0zGVsjIOzsiV1JQE+J%~KoB_aP*;=S-bYH zjXIPFG4!;@l44ZVqt2*n8t(#VtsOG7;(e`0|7C&?w*B zDwGR8^0We9T#Q2YPa@Ww5#pX}*~?(;YNz+~q=&woTAlJBuRbybbm>U3Qu7|+KU%d4 z>bxnFokJyIkFJjFm@K<1lf^)%%+~|dTR*C7e+$^pu%zFANPc|$&DhatxiNcRm{&iB zF3U3x&u$zWAkrI%lXCLzsHX6GC&nB_hz+kn91a>yiEO<1EK7#Z*Tof2mq+v4sw58^p9}CM+XY=n)8TH~f!Y?t0E5{!`_=6ArNd8BLPFvT4@zZ<>MSF@cLu%%#g^L2 z8>7rGRMVBl`c=G#~me0V`MMObl`cdO`j9Ki@{pZDVg6 zPUKCT^q>A|{8G}290DT6Y>S1Z)rkiF=8FdWWBOg$j;JecnD0FL%Cg%zG4SUha%a^2 zUyInV^g#fmaQhv;V=K`ra3`ktV*91@TQ>OfDQ_Y`z|3_#Ee4hYk)%(P589nnIHvcHx3R4I(X9uuKd!FnX9ub^AS$TC zsG@HhEf4yTZ2h$9w8^Sp?KV5f<24vw2xiJ43y$3-LG_)b8$%yF2%2nXJ01 zO74F5z-Z;rZTW??$Vj!~Erd8fzvifA&du<-w$sPgPAsiP*I1DskT@_|Zc+evR5#>c_#f=S%3GymS+hb^xB z8pEsEY>3hb-qg;C$3EhE+sK@(;oshon_zNsS!-G6z488;UTIztsrd)JGmT^#QPSl5 z6e12c3LmPp+-!7!sRk^^_~rtPFyc6pt%P+Q};hs`{2x- z$B!~!Zoj9S$byM%#o;g;YsYwK`Jo@2twD$*zzRBC`_6;r-#bBV5jswDp9>@!U&#E& zClkxQKlVS`+}w=km&Qu<`lo!x9wl7EGa=s-zgEw9}My z112SN^|^N6wGla$+ODSW5fQ=qd$aQE)Ka?GiizEW{QU~=658#n->j$47Ke3aJG{Oj zWAzHWn%y?ML-`)9#v_QVqzETJZuxe*_dE?{+mhrxXR<=PTU>nm@ea&@J3c%X9Shww z?T~M2sN$|l)L|q}(!ytXWpc^U5?a4}y%N&*rjiphWbT(jcEm-vU|)d|!M9Rex9)Rq zj58);6XoV>=ClgnHVL^Fz|3%|(Sv-!A4m*zb~v(wQ=g6yO|})PB;Sg}NyEGT2s2nJ zyB!1WHDLh99DYb;MtesD)BiAZKC6P5se4Q$)-FPuQ$fTt@>Q>Hp{Q3f1l>B z%l>`!MS5`cMf$NvZ?3=b63Rc<#wg=D4durq^m{{s3;I)irH0H2CWGJVQ%Wb6xm10B z$3|i3nE>YX?SNlH-4!l28gJ!q$-|I%#Ov&tM2D3d59&6e%)-hWuWytZ-2mxJ#(+=B zORa09N&JbF5nY3R#zIyvY!E=kj5q--6%HWur^(gqi!|;E- z-*6ls+({R35B!+w3a!Xs-ei(FCU`m?$AN|-W2Kq9KFQy=@Si`VlyH-a@?*>@)2?s5 zhZ0j2(LK*Dg!xOD#ph4#&I^_vC+{wdW3Qa+ho-jq*kaxV=R* zOhFd7RI|LIVp>r?oF7t50KNz+f?jWeU^hU4vN9-z=oYvwh2)kOh2 zg2ku3{Iqqie!F2K4W_AwQf!rHank9*iVCoC%m=$%LuR{B zg*W(qUZH0rF{EtDRw{RZSd|y}JG=oxg-l%uz6_vVDKTl=X3#CED~kBe`;x3~8`4(L)|=5$XuZ`1I*-FWG!4THX%oZLj} zu^x3PuALJ89B6JO=^3a@Eo>WU6c)UXy+^U0zDzXa&D87 zk&)>N_s_#P>v$`#C(2})FCJtPzl`D~F16%z%0FBHbT_CUFBt>)4+)z43feN(GEh+x zCZ6&h-(r1c7|ztji*Zm}ldqw|>ufludd6+2R=ONqQU-0=#%K$xpxs#*P<9<2)p&$||UE zhk_1ks-gJn?x#yh`OhamD%Y5LpDqDuT^45>aK5S{^gzxkPBVJ$wmldJ9jo=&mdfgv zX=wy3AFI(l$ zPZpwqjV+_?P`00xl8ie^U!l@~K=B*{0njweBrknYF9ZrMddJ|tsNl6cl z1(SjEE&o#klk3p+kqV7i9mT$AiEhQfO69F{&Q>yzKt&`wTug3w>#ZPT2V0X)*mEW_ zm>eQyWr;yB;mwezz_L3KO0!Bxm@pEiJ0eB=vi*uDTYOsZpt;PFaMn?&@S^!THg+>DiOxtIM6M zcqR@QW`oX2OoithTgB}G$tzE`3(=B60H4cb3Q>gm%dT zIkAgptAua?U-OB~xH^A3h79^bufAY3wdJw%Fz}2mJBX?PFmcg<-=FPId_Q3pnJkVl zcbrtu5?c;9Q=XL4#o>32`!*GDM?3G$i3-PFwxp*ned1#`@$4X-6&zWf+OI6snBTEz zo3W9urv~x9ZH!QQ44PU4h5&2|Gz)NYflYv@2_3$4yC(E08qnwLcc+fFpkT z4A-K;;xBOg1iikz7rOc79M3s($+_8V{H_z(tH}BRkH>(cI(jx@uzF&+({_A|0~zcv zl{G2WdZG#%M85EPbz|W}QE41d9DFgb$;cm@Y-1;5@R#q~r$g=EF^2*a++xz8K8Yj*FefgQ(|I zby~ownmJnhSU;7HllNKk**edyH+#7^4@39pgkPo`qI&^Fp&`Y^3*bdJ_{^)sR%0#) zv=B5TM#$=lgQc1TK$$j7M}x-qVcRHnHf|A^Mw&`VBD5VJco_Xw^xCx4F>gcmAAI|Dwl5UyDSpRmZlOqghRhlfhoA~Ec7?Y zOm!n3WsH;O&YUowJ-w$5q>D%K7-oiAS>Q-VkM0L70VHw7!f_h#-Q+l!u9#RvjQedB z`hJ}VMvh{b>N1&$3K7QkEN}JApJ;dX+VvPzB5+^sFIGWgz8^h7Vsluy?RuPP{0^kG z66qWSdqF|L=0!d89&b3`a8zFFbWDNi`C&T-E0-wq72hu+Gk>3g;)}v1_4_Z5{O6}@g5Eu z(WGh?l$8mY9uC{Jye*?D`%H#Vn<|eWm+(ZEUpnGW5VR)dreEF7IdU<+&H-kuB-&2G zSu4IyE-z~aU$g_7aY?A>VVav#6V!0C8Y;Niap*@Lwk!3EL}#J}HUR&ON0_p*Nt8K; z=C^6pr?@P7A!h|p->5JzI6?AjTz(^)TYij3PLSk-sqDe@G4{PQyUVy*p^VsFhu_n$ zi&jHaXgLg}$nU&ndYKXPIepOwx1reRJ1!)|x|oC1BXpyz5+S7rmQ3_UV~3F3952E-rzK zrJg@Sdy2S!b*uZHkPq}urMud}ZDc zc7?h^tta)MciYm$#~4#~Eb7WBQ*8Qtx(S(Q@*|Kqo#@BPnP1KPpg~Ma$7fxr0IafATQq|a%8gL(h`xpcP&79}ykHI5l8>ek0Lrh}@cCkQ(Y zg>AZveslYAH3Tk&=%>N!w6Itaq*=wEN0je^9I-J{s+5X>ngMJLFZbjV`R>A@A}&1- z9`v2UxJ1~nD?U>_fBl05o$_NJGI*B5U7Y|;)Yx=4#L5%Lil6lsD`Zh*vC`?yd>OGi zJF|qtM~2@#F_Tf4NF_%~NX!`YaWR(nNp931n7XXs3d~6)mgoWYHzGV4i4X|sqJMb( z4K6GJw8_uCFEu1rn~^x`!JlqS#5{!Zu!5gG8#-5@2A8C)Y>OEicRD4nKwl|JC)&>M z#$**-g*aM%E<~a2-o8p`Bp3p#TyPPY1XqVIBX@>)sig}mZD!v^%8AC@yUh?(@R$-; zPyI(_k{N}p4BVGm2J!S;Zs6rkF(+nBR8|98*79WU29rX zmrUzYeZy2rJ@^_nH^BY9eDeVN9Pe zeIKop8`BxWm1h=+LvD}D)*~=>6-CR^##D}R-UF|$nbV$npS zeP9VovfQO08digN=YXCRPou{`Q?dpoL66ctX%;5hP2P%OOG2iQ3U8?!^6};-f>#DB zxKfzmiscSm@j^t{@jL!k7NSa%tb^2$^PQd%kkLUsaHZ25Y>3VJJB=C^qQr)gIQD;{ z?a1t6>-*20xxm_(DR$G?;Mb^?*zAvq3X$H#QRjC)a!-?-tbS)3O_n{GmU1Tq8p5jH z=kl^6gatXVfyCi8K-83%UY;LlczHXZPiOofc2#3}C#vU-d#iKa;+GXM%;MB=r+GQa zMO-Bi0%E1o19*R#EI`~cmK&D5?C=*YQ4DqcS5(;+6>Q?-FmveMZLMS^XJopFJ+f@e7DJGuwXKh~HGxIG+Cb#ou@D)Jo^`&B% z`cB$y+GTg0#0vs_(@Xgfa-P5DJga0$WvirgvErBrn6h9~qLU%v;fIuvx>~3qAMwx5?U3Le4OM1(>`+Pt70YZ$C7BQWLiQlnPGAG9Be*g`awRMZnKPh(?! zj(|6<mCtu3c&%u*x_;cqrEiay_^#JOsT3|{bZZZbF)os<>9zvq3k*}Gdx}GD z_>h=+T#9sVm)v(?BGUWg#$v*di(pYCNrGMTGc!8rWc|Z^cDMsRBLNvMWDx0s)5*+= z@9|cQhOx)U$!!7x%NI%Mv$a+KY?0u{gmt=V>hJt}!w{eY%LEz7<5;Q?Bj?j*_vuK!EHSHG5mBtxsv9h`?Z~ z40M-V-mAY7LTcH0+; z&L6L=_h*p7UA#FXqzt+f5Xp<=dFNwBPwc(TRiyOcu*DNe7kp5GPj^o;m}BUvlea# zCqp=i+>nm5VQg=xjgQ^E;TJ8k63TKqamT@6F;d^A(M!C=v*fkuAVFrzOm}$oK)R)= z^Pu#)d1S@!gUgOj;bh<3T>V)tSOU2T|1)?6^T1bxe^cxhyLj)oOr9rUp{U@5M;a(N&2?e+ zD4VR8i3P?3dp^0sgnSB-yMw1fQw(XP+d$j3F@iFbug)Iy;X;Zjl$4F0eWNPx+zNK` zLsIy5(C_2DLeCv!eDL6BTYNP-$$E;zicX24Q1!i5K4Hufg2V0d&cLXR*Zg%2jcem~ zIWolqapZ2DwWV7&F`-M1xUnmVW5R{8(-{;)%m>^EG8y_(k2&BXwHzTAG1>83g9%23 zcC;IG%M7N6g^)N1={-##EA6TNRqpJncZ!k!sjNhpG0M^VE-qDqj=!5P1~?=J8Id{; zc{>r6V4fPyxIMNXFk}T2H3Cdjh4i`81>VoYwLg<~XYDjz1>+znS)M(K7bIoSFWc;g z`%K+m(nR;52kp+FGqEHA2-9WSg-!@sj^mdO4%_r3 zr%=<3UFiDd=0?gI5#w@x%Q134Ldgff(guFH3Fat-TWX9cCS`Wm(|%6ueHWr9wW3@b z6;xj{fuJg7-+sFrqk!o&t~c~nh=5U;)H}z9NIt8&TqX!dfruNy+Ltvh_3izxG@T(& z(M8=`?9-m#=I;M)poaPpzQu$5HFLEPQX}(KJO@PDJEK*b9h6Z^}JBh2+>|f$7DjssB;i}(*nEQie*6|Lzg5v@WJ<CX#zNe^h^=!F#z>5?Qj+ZHwga7SM!ht5W|}C($`!2IBcT=_o%G}+j@@nC=3lHxCtGCrN*>^8 z(UPRC+>nHGP>YSxFk^b3XV~#gWi#oLjr&%ZZPlB7npr=neg`u4d2;V>BpFNxDZ#}& zNXW`e*uKF|F~<+W2e@t_OW2g$>H4qRF)XA&O6t<;c~Es`{<4y)jvBJbrrs!raD1uN zXbxRZweFX)_~{)WAL>q*hGMKTCd}nOTm{39XIFG3i&x z%|3YxUjCNk%G;WCwWl4l1AVF3bjJ_}bxFny7gEawF7`K!AG4}k4d+E=Ey`?6@#PS> z)@pB(1kAG(psmwnc_PW?Br7kPa7h0!f#s1=&ar%W+2<7Om9zMk zb0@YVQ+5y*aEq!LpKbC`-BLq)9$mSz6tbf3zk|8XE5D7 zA&q8gBM_6LwW9B&ewlbe8f7ankI_lfnfD1Y%+;h}ZF;8z8Jun)T(5=jEk=$keRbrw zpe&exoinYZWI?0#RYml~J-=Z=mc=l+-0GLl4`dY@0O#VO?}hrllY8pE(U4t_i@KEz zv+PvE1Oe0i4oLKCfKj9|V50 z=t`9{1_j%?@0_@>Vlhs89~j{x)cq`b7mfThYd$6Iuky49*yKhZCXgkY5mlr~Xkq_; zl^S_eNlR=1oEL?r5Dw!ERFqpe3@oR zOP$h1q)wq5(2f|vs?f_fA_;`s(8)_A&anjME%iG0hGIvT;oI}Eh%-zZ6T!?F?ew0{ z`)Ebf-MH;^q+>~40S9-~Z}wG0Ohx2Qh4za`) zW1W$7lc=Z-=7wt$FTwBytywb_i@lN%W4IgB153Ynnav_P$J;*!S1}n?qgZ~# z)g_x$xWXZc&Fb?D%Xp1~AC3HUEu;4i*ZOA`P40^J54cbjQ)o}Jm2)sMt0{zfOt0-9 zwsYo@65GZN}3UzYq~i;)&_Qe5aPx>U-wMT@id|AVy>D=NE>kF08FRit)C` zk7-_0DRwwbm1A1~<HOybF(Z~b!6p9Pp?qqDB8S^LhpB> zurjwd33jPRxS|__v}_YXM0h<99Lu|!D4b$WaY#0%T80#tDgMOOUUZJ1y~x385<$u^8m32wc2`s%rypWI<|%j)kjj97{G@W%Q%7LiHiR|-2eUw+9sn5OyL z#{S|UP@_Bs|9K<&>Tq{OUr?z6OdGrxgb5+B1tlow8#MIsR9zxU>g69{% zwGW_dLc^RDah<^6A(Be54m7l$*?TX3tgflyKAV!6`xvaV(MgJ1fL!f-?+r+IS_7f( z_A*K$)jxk&;4ZSF>vsaoSw67xa#om9YVs(P9Xi^7SmGjN zeaR+poiOjOI7^_V0ao6+aB0t)6uBQ}ZCs~l>+qK}p5pJ{Y8(ph2yYF}`RYBYh6$JC z6t6p3f2o)qPLUqm7P?MX?MTY0AqL?E3D)4m<;I9c#_-WBqC*n=JHJEF_G)K78nOqWuDLv9t|=PH}8;=095%R9NKCamu%JfW}lMv{@ZPR zhc%N|QJ{|^!a5xB`7 zy8n-{w~nf6TmOfpq@+s$flW7xC?K_w?(Qy8kPbK9l1hUJf+8T@t#pHcbV@hU-M_g! z_ndpb^^SK8{!wD@z1CcFKF_C~#TRwZD}DF#)e&5J>rie8)t7xR-`%->?WT%$%LsIz zAoD~}lOWYA?UD7&?(OYaxrllJ`J?hpk07OV2-YbtBL=Try!lPQF37~j)Twq(Y$Xnj+t65`H>%HHzbK9WX3-pnQ|h4c8so16%kcBvp#c5)D0kO89D2Gb?58Absok-43Q z_$-y-3m3e(JQ%T1f6xUURWN632PhWa+|gr?tbZV}5JnnTFoh>wrTBO`{Dc#yPVDBJ z(`5$EV9BcV`s}yQt?lBnTjYC=7jG@sN20%IcDM8^L}#j9$f~RR`EEWluHM^zSZe*c zR9u2IszvHu*Kyko;rRpa{o6@`X0eCchFXSaCqYPw>s*WdW~0*AmuDGH9ah(+E8{L9 zhrv|t(uSb=E`R!m4c~=J^ZlI*05Rh?_XS*uk&86}#s0|^kmxcWp4hO0!6k_xEqr5U z4}{>J*OB=_6HdsNT)-GA?*mJTH&wIVdXUt7OFr$*oqM#A!spqoZEann1$vnvd~v^x z%&0Q}PdNJqT(>c9t6^rGz>gYTS{|jjxNQHzjllhogYHM74_L`AdD=kWc-0Bwb#acn&x zs4_s>w(M=7kX9?=wH}q{fQ`>htidJ`p&ha)50Rsh{ruH_V32{ki*0oX09>rXs7Gmv z^Csi3dooV88oFC|+@+^9NtRnNRW<_&+pvSCsb8B_)r)lTzm@H2pcdpEvQTMUu)uUQ z{A#j)TF0?GFOZybx7bOmG>-M&N6l-6*OAeK4A_=AyAS5(F|9QA`ohU9WJF5S@$-|n z8*aSaR+20S!!`k>T;B*I1*z=1w%_+8bJTMNWlRH$YHN0;>>A>`VwuoKfQz7`n;}s@ z{e%WV${Kf8vsi=#)GEpYL0-L&fYU-AOqS#zVp5E&w3*7S7zaW8BxH%*aV&HJ=y-jq z^G$jlmq#syfA(N#o3dm@_MDx#`AMg`@XHTNwD}_n{d}Q7pE_Uu?4Rc(MHABO%((>Y zfFW)vqpScI3!0w%8ZCI8_OjIT$kuIsK+{n!O){ofKq8@F zlH-@%xy*KTPH4HEfetO!(wKYYt0^uFLV8qx6(bjc%o!u5{Mi0_9rhthyXb|zpbWJO ztNp=Pvd*%i{%AFIx zKec}skyNc%!2GnlkX>x1M*NuJzUGNq(RTs8pkQ=cyIh+BeV8VzM#xA5ll%-#(RW_G z=Y12bWK{*U)b%qD70PQ?5(XR>D{#uG57F;@eOs$(r5QftvD!;ibEq19GDEaJKv(S+ z_{o0E;B{9Eo=A%87b2?LIOG{=zW@s+TPwg82&Ou8FGh+Kat3dV7UYfMTYhv&sI|NR zl*)3&nBmYxaR$vtuj7f0++L&f`@;bYO+^jYtcim;H<`fE)B|Wxu0JxV2-AwkCe5gI z8wEmuR8<%Ca~0WZCGs!!>-h={I*ql)i z*-l~ieZFGi9wr{p*<@~)LDcO`6(GOYhjU_&(Fc$T?tV4{)2_(v3%~$be+x95qY+qa@JAO#oFmw?K4^GqT7s$Z}Mn#VsAsc?gf0w?&)x>j#Rf0@Qn~ zj|x8&$2CqOss8?c&=(7KXcm(dLOjxGe$l724?x$Ewiaay4B?LEh286fRQRN{Y~MMI z@M`7VU5Y$pEsebF-hJO)k$2l!c4Z|L+&!d!sr)rC;xr?iaQ(H2uuA$Dx}|mW{#c-I z+@C9pWpEg|&uur4>$W!0#eK1}rCKNZkcGfVdLfJ?u7s{IMlASU*EHIcMfZ>1{S|bl z5rr7bbF)dtn(T|J!6Ge`bsqzOFZ& zvY;NdG0Ixla5GX>a=7_R87*AWCb+vt$&hA?Gn@Xk;Q2Qmp}dGY?iF<_c}_F<6W?~C zcx?0~sp`vWj;-gZy)sPdV%)(fmuSrbap7z2rIqA>WNVNJ5QT(tyYv3*ODAq%j)2BV zG!bZths0sTVI<-2Ltz(*V)(y-MlKNyr_A;?d zB$t`{j@tnpz=us*-@}cO9_zA4gj6%02MQ#7_|nerUQMKX&GROhR$H6i00{)yNcx>7 zzy7tVs$-hzxdlwWz}XN!NY>?fI3UGJPj`o|^=ChbGQ==*pY=Ln5l;B0D*OHpT7pLe zmDix6g4;q%aGFPVR1+*g40Navn>%Ug8;L=QiiM|?w;hENQ6XsTRY?c2n(<9R!laAr za0Lh+N0ArP2gj$9X1?}krVp@dnmT+|{RKlsekTzN+NbbhQAO3;ph8GV>ehh?{ysIS zNa4GVtdRSR^-wk6LMQFMa-ysYdRH}pmQ3)Ic%?4AtS0oqDoYf%qxE3~=B#IfzWdcC z*+k{qlQV$A)vTwMa6d=97Fv%a5zRTL?(9zD&HT7lG`wWywv*}V*kyxWe z1z)U|hmRzldTO_d;oU(r!x(2=r))M9yBGq|^%-go!C<4VcrC-68`I^VnQ~-r@zj7E zP^aFr28*crRf_`t`lz0r#ykO`@Xx?0-bsH3(r0aTCo_2$L<LNSJR;1 zak0pm6<#v;#e|xY5JSJvNy-y6RVukVW3v(Z6#R*@s2%OJO?p@oh(>NdZ<_IP+h6Ay zb#dH1~t44IE(L4<501k0m%a{~V4>x)885 zhWqjHG$`EnX&`-YlV}Aev_L9pqnHBGpNiVQROBM!ONcF@XD3HI*hO)B?0tzKf$>R% zzi1Sy=F8D4N=-$6BY#MIpE5hL3?NfXAiL;eQ^Dw>D7rOIcDg0<~J(K~- z5Iv5nM}F#K@Or|9X0?+vw%5mYj;c@CiLf-hr6h6d^=n@j=8S0;O|d=J!;RKyIErcO zAzJ#Sbe#a|gduBmJgRuwk(qb~H88PnGA9A42@8*fMT()e`y2PdV&C8wYa5OhHAMAZ z;lon98E_FX- zyrxM* ze}L~Un;>7|2=l`sC09f=vf|`DmixOCXiNegSUk(Y7kWlebu$fdmeW$5lhBQxe*+1} zw{Ydc(G{=Ubp_{!rlZvaiSf}+n=%K%3l$cM*e^CcR=lKD#rGs{SPj7q!JKJDSDakP z#T;m=(Lufo^+LLqLA&@?at@DB=Xa*@-4fKw&pq~k-tS$?L01hBGt2zBd`@kCHJ!YC zl0#9dj~FlGl?WR<>p)&?Y@+i=6K|G`F@2Zh}{vY{0I0{<5S}e(osiX z0-XX8Rk1JpkmfkRviV9yR#x^yPew#L1Km8#%h+aGV!pe(MprhO&U*C4Z1p>ddt3Bw z)=D3rx~Lc1`-fV3r0f<0K03Dj;c&&PlYhF|8cZaoinwISo=Gckm)!1yR(<~*L+=KH zvGO1>{_ds~W;*VEcx>K1hI~BGevg3hM}!t8O~p>DiaIrpnYB{$hYyMY1 zJ75L;4iRn0OOzS+NeNRBlG$)U8tVGiIvrtrUjMn_)N~ZDhx4#J$>G=hU)2!r4^CWP zdUuWOH|1?Te&8(I{ZOajWtuL>^g*lmy~nl%x_9G3B1aMEp)$ed=WB-hXS;1Ww1WBP zWofrl+ervQjmI=tN#Z+pUc!{V^C);<+Eg6=*3}C_4-9=y=EG6YUqG|dFld;bj@9YU zAh7&fGIQvbB!Zcof`LnLR2D^oj9nX!!6`HsjamBY{4M#W-#9U7m;n(q(`3Yzuvt{d zw&xIpy7yT!yvb^0;PSV+mIk$FJ$&ZexbX7vygR?+Qg!~XsBD*aY2~6%k12Bsb1^?~$S=wOMKa*KI$UYMpPbUr~|> zk6FCNprss4ZtT+1cL>_P52-O0`f03;`P;6aHOC;Srm9^6Og9}lQK=e_Hk{bo zit-tWAVC`sD5PJ1U=zsi%tnu<19ILV(v4=_h+65HNilisX$6IkPc=^L7OaYW79G36 zNjhOvKnl)isEXVb2Z#19C7E?RK*my%K!a)j{N1;sN$|~QQZ#@^Bya9p@FR0TE}`Xv ztHSQkki+pE!|pbR4OK3Q@%u@)kYLugz)-#_B)i7MqnH~3UjQ&jvA(p8L-rdD42COr zd!`Pt2>v{u7hzBQmi9jT{zM6z&iP?BlhOCrIlmPI>3L^_B)fc67ta|+j#PWC9LV+0 zuV4ojqwi?jKi&Dg0A>9l)vgp+`GPnJ%#?@%2d9k_WqWW{zEy+&1<6HUn zrBRJ-f#m1T{io_OPMvoCoKDTDfsc^`pBDN`w?-JJxd9D2vegez#j%4lnlHDDT`{q- zP025T2ATEF-1S?ph%5APbW_UexvrVX78kI?VUXD|*0rH{iA6|!Vla`JpH_r9(ojZe z>v^3Y4$Sp&mKg2Vta@o{m0OL-2H2R%__JjIHGgg!iVcnMZGTRWoWAIKuOWSD{`^N) zcxFMjp5a>&CRk_((m*iiPo6UQf1Ltu#LyP9)~H(rZ6R8;#?4}dluxY7p`{`TwHUsS zQA|-X$eWQvXRGD>Pn*$*#P1;)z)p4paQWXT&uuR@`N@%Nx5W4fqG6LBZaioBQo;>T z%;&oR=}}!xTnoJ#f6rXOw|l2JB2%>cvzcn4eDo`{rPDp4=6=xkRtQhCxNIENVNr33 z#FNiWX5u8rm>Uhka`f`5#l@FSrZ~^5fV(P3TbQxs`*4!GkR}%KDHFyw%N35PV>t-a zd9@zvFgL?7h5I5u_|Pxs>hY^R)99xz;CZ~% zYO=C|9J(k15zfRjq6_H?)|~olj%6y;;=X1p-?@?M2oK;lSYA!@(cDD(=Trw=-*Y-+ zka{k3J?a5f>0AK%#3r#D^kqZ*#(SS$VHfV{ErA31v3^RmK)1; z^vw4o6iQ8dsraj2@~=;Su2nD4=drxqY}KWTYJc(M;QKY#cVHW!FK?S#Ilf}%uOox!1=9W{nXk8=785DRt>Re1la z5Ws2^e<3_k_aL`k^k2oIK#l^qO5VFBYgtR^Md zv3^XuZcpco)kOA#y)|>Ji%cFoMt94f^-}=cd~cD-e16b?P>o(SS1UA@=4;qkAD3Q@ z9;5sO&9{>b>Er`8+06r1s~H!H!o+z3jVh_`OmU{7=2d*zQcjE%QLV3O@!5l9J)?7H;g~0PX;i6-` z&c&L9O#H21QF)@UkS9@B5l#wSY?fUN0Mf9tQ;oSfv`ggzOlBuU5J+uWLiu~Z9M=t? z9Cm|RXPKU`sE};wu(xmHHw(Wh&L9?XSU?OByq8ME`@8e+ah3CvouU)q-O{sdxN>bc z=>zsL-O5)iX`cI`Gs(X~uOhvL4u6V$4Z%8`PxaVVF?m8u`z&buEd}qnjVkaYWq^8e zzYzuRw3p)5AE3F%cD&f6xJuXi?^QS#0Uh&?9LEcWz}AkR1y&(`7K&LfH>d!LCBQL{ zWGMXGUyhCkf?0t9kKW^CJKo@%Yu8)e8D`*d8zZU*@pi+T_^|*k(~^6$w)UbTY|cL)t?CW3H%mD_X3UdAZ%q_QS zL76@vMJ)raokpCvy64T+2?`+|^A`ZAJAzKB6S)3%K22T#L3$T(8_*;gxF$g(<3)c_ z$b#W@ymG(Yr=^ii&ZFXq8!QiKI6D)$pE_+}Rc57%EEEnDT%hmeV zgDrmOt3DWjCl~q}os|mnHEvYkx9tb7{n7fJzjcismHc{bMXH0{#HNR(x~v;rM(;Z z*yX&}J=l?zF?#1bF@e8WN?(oGu6qwDId&*0WO!|M6ROZOjFwqUd5cET#B=KKiLnL# zA9(PZCGW3CnxqP~qU6II4yqmvm##TKK2#~pxb4wid1^f?b>6tDyL_7e?G9C?GzS$` zbzP3&^@2M8drx&k&r9U>>5>nxrc>y(jxKcG;C`1+ML6U}T$$DB`?v@}=;2Gwlo@Jn zBjHO^zgJTZ@be=p=6(_PdT_#z9luR37ZpBp{Ooz4Vp&l2LOzZs*ND5oCICHL@c4b_ zQWveu2CA?WR2Xl%f&qh$+ch^Byo!kG<4Nm^Vt^5Mf+{NQd5|x5CJ@R?j9meug&jg$ zu_8mFs5 zm*E0+dIBZnc%63NEZ7HT#2(ya}fEj*4MW+cU?bE5RGQRuZ{V zN7L*Vr&xDdY=3)XRt$eHkzo&V))+>Kwf$hEH0GyPc+18Hq5?0S)YapFIr4^}{gzxf zATF2-!M64ln5Pss+xC&<`2XV^k9r7@mGK{col~Cp55j|Y?_pbqf^%{S_Yd-R+S0dx z6heT%)(ZTo7J%)LZ5FMDNzT1NGbI=-0K*R4xgk@n?9aW%Msok5pWrc26Ge`Bw~7C} z34p%-qHKrXzse!t+9CP7m7YRt-0LHC+bs7HM|J%B&B>4A1EuH@bL%pLpA<3g?a7dl zAZ>elri)8Sbq?XBGPN2Yb|<=GsyA?xs_cUW4}Y5{U7fh*7-qhY3*n-7Xv69rC(!Ep zEY@MZ?Z3rsvGUyQAfTqyYp+2u_ayRV@>2TttZaiJPR5N3j>S*QgXP`%MefVRzS5gU zf|p}gdK|CkdbZRL;kRh&_+J!;-EP3yDZHVxSLfR_n<$;rJlk)OTRu8jl~y-LIy*sG z?T!=S?Rg)%X};M!W!`%(p5wBOpQFtw>HfNijVx79lym3cOT@hy15R5#iTb-G;@h5w zuU&*Rs_irMf9+VFt-XEcs8IEEaDT99>hzf#AiD zG_HLSPQg#1&r%Y=gpnQq*AGcM+d4Okfz*&Cu#gfV!>EkAv@OO3h`Ho077rKNhGzjT3}%mv_K6>g;hgH)2BTj{n521cI*CG8y#swE$g z)>jb;+`fL>Hk>eSMA+lkzg)ng1`Zbu1bpt(tnuEYb&(7JW|#Ad>Q(1k^*kCcvyyEG zy%Mi`KMH6afPgd2H4###)M{1gg^mf!v@J5do_{K{lpLsg%zF3zXbPP1%>;kwNlnip zIUnX9N3olffs4GDTJxUN%T)}2&Lu}SNKkwW0lXj0CVRe(jVOLZ9bgEBR-$Y*t*CLM zmVlpWiT8M=O|re|huG@NYU>nLT*k(7s3g5oO5FX#8NY!W;{dn=TpQN3!3l00;i!5c ziq1SGqwKNzGw+(i^D(Ycn*p2s&Af>%)|s<)W{Lz$D5eJB0Z9}HuN#~iaQ~+ZS>GMgH>CtsOBZ)@jF1 zCk=7MG5f#CGC;OXx1L>Hrbm!mNFrnohfMuQO1AmPVCVZXip@K1ACP)Y{m?XUW)^Vm zBy`zw9h*s(sGpJNo3uP=&S3{EWIcDLyMT9|wV+`}pQA>St{u<A$3e zfb(d7N0fw>PxC^OLrh7}-BBvTu#zRXz36v%LHi&oWmAB8VFGfhV#2`Rm98?XJ>IvQ z&r&}|VtOIr0J!_Npu1mP4^`uscqR|tjiQ48@8thWozWf;5E=`QGEdQ533LJ)ghRqs zGQ8W~G$b8wDZYv?KOWQnf|NaRisU=gf^QS$gqj^hd64zYl7|P1kq8tr08^{P^bs4^ zPD+^XA}pSL?%wI_Y)X_n>C0(7*>Lg5eBJ}&-mEe#MY#EYzv!Cfe(lLR@3A|l!lJrm zpu?S6G@NIkFM_R!ke1O!8y8E?BW0_S)pn;4N0 z)AnVEu?3BTtuwgj_0Jm=0;*?868R3Jtf+$9!}`wJ>vJuZ0f@;TYj3xJZB#A&W z$5ks#|2xQprZ^@@gOSw!N=hkwDxg!y&?Km~go)}d)T5f>jnSb71^vXP$H=45hd|#C zKtbS#z{bDslZdvA4A=BgQo0s7f?Gu(! zLx>mqzR<2R|Iij-c_Dmd6+P{F!^dBFWO$DcQSW&%gq3uurqt^no`n>7nYv;XdKd3Y z{YKUpq$Ro9zRxCOy**rwgMsIvg)5D%tuP&4KQ=suAaUuVpvA*|k>z6Uz@-v!zaK*CoicZFZMVNMd3Y*R#S8N1 zkFr1yQzq0)%c1vufbfK`IDZ0A?gG6S%}`8C%rF{8#=>{EgW)v(W)!4!7+DtzEVK=! z=__oV2H7pN1C2=dm`~#JWY))82mgJP<%(6;P0EiyR_+6F1rO?EzuE>Uh zgj56Tyxj*!{k^K}m7jR>wN%YMP4BJk0Ih9w! zXzwmKldUe%pl5Lp63o-~fZPc>(2k8D@P-5KhOx11FleGQBPC!QlI*8l&kKhd_hS%L zJJPO|5m8|b_pX8*>}U`#pkyEk+c>w5B+@C4n9HQBQmd}Fp7~zW>|b$jp&uj}KAz*j zqsgaX{|D-S18)a0Gz`i2{;Z8aJkW8lc(ym|{t*VD_~~;9&Yh!(tcTnVxsyWwCa|SW z76d);yg>=1NWjCuKV=J6ZjpLw8;?(mPlG_r)n|bE1L~wl0B=@q#-RfyxS z{r`h&jq1srlOP7o&FrUsC zX6*cG#5H)cImXEu{`F8H`A(!D?fym{Z7gGGx$XT^$|lv&ifa&0V$F_{IF5dLCp{T_ zH2rIkg#_@9;nvEW9|Z zy_Bv^NwArZBSBTi@I$(M5C*g9bFgZzCFgkxf>$~@sxowAXsT%&7X*l#jXKg@ZGP?G zlV=A9-d)&*f>ta57R0K_H$M-OR204?9uxW|FBJ)4e6Ozj!0(i`V%fk@{TgZ!ef0HV zoxTEHLsxMk5nHu?BV_iBw8o@(mJ8W>X%KOz51}ynQtgu#V3U{+1H_$}b~^$tKiB#w z z+kxJXqKaNdf52b$i&Y9SyEaXM!lB8*=#^@T=MR>x41HuZI?kp3oq?KL9)VL|Gp>R& zU#&+xgrDB}nj0VfG4TH1@mGyuf~5qrL5;ml)l!SD#}NReT3f#{9`k%qV4k)Re2ytzC<;}_sbddn)uba`|{P=raPue@@X%yJ?OS$pULEF zpy~%k9fl?zftG8{o5Jesb9vYHKdDWQbP$K#u2S{{It&d#fL8GLr=_D@&=$=3M@fvq z%-P6eib9`kx$wEkMOvybSXTH05Ex6F3o8NaHfcv~K$urxU`e^k6b@g_o z85%loea-`><7g|5XLMnYW?9*qD3CFbfuc|)4^^LQUNM61XaDyZ03o4> zKOtU}e?R(^KbtjpgX;zlvWy7C#LM9`TYq3PN($*B^R>&Q`fPL+pT5rK)L=_y-$#wWV)K6kvTuC3P}$o9dhz}${XpVB0wb-YI03?xw;@3(``xb~ zlB@I+g=XO2%k-bFA&>wW>^PDECJNw+`#qvUvfRTbAz2=AgwaJ6BYZtu?ic6?rtA%M z-U4!BW(d3WpT|{4@C{^DP}+Um2rvqGe#r&*4OeJd^;~i5ecb;jvS+n>o_EDN2%Ajt z+iV%T)*Ci~tBS)kk7o-T3OjGY`VBs&o(c8rkG+Zb)Su0fYSW29p+w2OVSRpV5jU$v zEx32=QdG3}0|VPG&7vtzo{jbjff^?9d|Phz`p{`ikK+4$qKaGnewLBa=ZK`$+Jm5= zs+#Bl_r0;_Mw<_=gz?ut^uDm$)2Ev`PK;>poL{p`y+}4&-#HmejMzA6{WW&EU)5nA z7}Vf>+drwu{c0<^zF0|Q*0Z9`3HNzmbWzNPXNHrQ`w=k#-IW(B7#Vx_W3i1+7H z&w_i7NPX47;yR__G@#n z{~+R+RY}mRj_ZQF3Vw{II7XGc`%V0YU#^U4njcQMUE4BtSZ4jNH8~Lochwc7u>#{# zdh|IghQ@VDz<2E3|1wU@4Sq#0RkOwcQE-XJx&r?Oi2s1#U{#*LjGM)_DIQsuK;XrF z)KWDB5S+VNrfMsJAD*LzT83&mb~C|GAsHDN(yFR4x>&>@4=rHFxZ4$sUWEdScJ1xI zUO%v5)69PlV#JuSmgw-V)y^nmYyL}8$A7`)wISw%^kEMqk_A3O=uM28ykr0;Y-5ri zJOL9gG&EFpTI}31j^-51_=sQY&lT4J88xvW1F{?BL#z8%pKs)4$|rH_j)6MMd^BIZ z%slY+q$Vg0UXJ9(x3;(E-YxMsvBkNpbvt;DbE*qwyYrAq`&AAl?)vID&+C+-aIj{0 z9tQYjM3yr!#pDsmKjMa+jJ-iYkL-+D{+1~nEuEv1`5tVSjk6p`228vAeSzjW-(V`Z zjMVV}UXT$N_s-GEs|Jut#!YaHVhX;@^lNESE~TGyeGesQ0ebBV?KGxg;$2Nkbfu<8 zRH1B|$^h?=&+bgQ7aVc!_~RB2Pp%Hj(x+)L4Z7F6y-U*EQus1Kh|f)ju2!h&s_>6r{OD%%RI86@;Eo zjDH`>bVC+@_RJpyIZCl==4Wf(0eEs8n3UGTznbzT0Y7TLRg_sJJuv35l^I_A{(*@< z76}O4)-s{h-k(X_#h9$;hBoLz)NZs_FyQXA6s1xoZwfyABK+XBTr#bdGDg4_OHB zvp6`QWx}s7{gt_$wl(kZTV?7`nJWfCUFtXQB09MM{5U&vLz=hHhnsdqAi*Db8(_98A+0*dVT0}VC2_1Z8J zA!%c=>o~EF4EVJt$e?49FiQshmN4pKtoJUsBcCsq?B;&=G!{Gp%A_1Y=l| znPI}^!S2V~0vRqOti-TA32Yg4b?>(DS}c-?pMt~p)c92Q*pz81ih}XNe@_H-GO6Zk zZ`l0@zW5(E{J*;bWS~Q@F)OmsSB?So4V1nIB(`i2OVE!`1t3KRa4HY?gQ2OkptUFL zbVEML*Q-*@l21za*_p0U15A-cvzTug-fseHdjpcN@e)Aco}Kmtsg~$RIB%+}$&yAY z=G}|&SpHDQZL)t`^b=e?8xX$b-#(Sbv8idtsJ-jA<~DK!4&FEwPq;^{hndtv^g z)n%~6g@i?gR`>aj8Cn6?`|4~sg|NH&o+=tIN7BEW!s~i8w-NwM|KY1l%b=t8!G5Xx zl_9vxq&k(3RnFVW3I$YQ9b|*Yg*u7MYR_#Ggf4J%NTfTS#AhP(KdW!1)v<6d_k+*s zukI0tUx9_6Oq{g=^P;<>NM|eIA|-e6OoUHI1;KA%U_udz ziotmd?!Wb~jgKC3-N9!8IU&M#{J$qf6+l z_X=Lhf0G&aQ|bj@kq&>&n!l{o$ZjuQ&rz$_*2Z3`+myFX#S<0Vts!Yr&R+}%q}x$Q zydN1Vay#-WZegZ-ucUD7&#L5QxE}Epyq+IKWw^Gh&Z@7KJ#UCyktrGXXBD^arVdBS zryw8UV(^jw#oVAIfib~l$;-+>(%mi{o@J3~+q(b^CSf3^kcKGhRTQ?Kdel3cJn4aP z<_em|%z@&a1Ub_|*h}rIKrEWr=dTjjICDOe5@KMF8R}?t$8oY_C;pCswD(>`Ib6#< z?VTaii9KmPHfj!8k`;V9b(|`?DF&KtK_aWR@tA5)WZ&ktWq3}|vNp$Ysfq6;@p;Am zxVFv&zJ3Xs^hPKT#eQ9#rW#pOI-dUOix*J=3WIO|# z+TQxmmwH2=&6Wm;Y4fDTUW#oX-Hywv9aeWhmJ6w{1_n583-Q8w-3;;2sAKwXU4Mgo z|9N_X8!iBGZ1n%_jE^o3+bt z=?nKl2TAFp`A_<Nkqp_pVT-mMZ811xe%pWKB#aP9MXZDU5V0P^< z`y}SRa2Tq<$xv-ex60fF`ZULN35|+tr3PJkj^u>+!o*?a^s9q;xeUMnSiSU1hJ{?8 zg};_AV%wBn4wKN(58)(c=}=0EdYgRG6;(tg7b}^WX_l-$@5jxk_={QOtq$EU&ODex zKkJuKWNPmcWln1&4c8r+1RDWd*|gN7%vU+htRFmioIf9K0t^wmp1ejqp>UjI%sG$l zab8=Of7>^pXrTp$vdm`rJyW#h?!L$Ihd1k0BQab)!UgEO>OS(b%m<{uqYQs-GD}t~ zu#!?_KKt>`plko&{m2IA`bJ@eMlt-BD2-0lLluBiD(5l_3oIQF$N}d^E}>R_l;`=8 z>C1tP2Nmtfr|Q4cH^n;P*7+V zeW8?ArXh3UT}RHk=ks7W`x0mzg))7vG+ZozR7JK@Y@CdRGG{xK=Jdi~OgW)wx`63= zr|yU{M~}^L|E7P*q^A;v^|^7^PoVr#t#mfgskDmxel=jkib3P+d2xWXKAZfsr~CLL z`qEGlsq9DHw&haSk>U7(Y8eXi?MoYAwrM9&sC2$(#rlZ;>3wF7J&ar7^FoC|}0kd9i2&e|c%y{71U>4J}70z)O z5OiodroP9l7A4;s!kQ>%kty6(m1C6GDQdkQNgNwhyD*B>&tf{Fru-@Ix}}&KbM+<-59V|xH&0wQu#u{Q76`}6IQus7YbpOFr%wT*^A}vwDF>l>&VJ%Ut{8yVRBIl_$SCQ$GgFX79vvNB>UH7z_RP2K84!UUJv~_z#!QrhTyDl4D1*|{=sO~@d=nN~1lCv2F%$F32Cc+d9`9c7ci4UnD34qc%5j+1AM%7=WCT>yh zowe)l)@#$Jb)KQB3M14Rd&lXIHsh3G{&St!iQ52^OPOI>a{^tE+jjK92%-Nt1_sW3 zU9ACaOwhwNDY?{gnV9yuBgIjUATsw6$~`34~z=LS>ocT@tO3*Xx|l$7w~>v1a} zvJL1IIf17+QK{W*uh{t8w5ynzOVCo%T0eSPZ(^y1DbH=Of(-{kf@mei{_*@#wAQa7 z%uz;JeUtemWVV|_+Gz2!Uv1-X3kbB{b%XnD^*w}X$1ca=zW1q1d>EsaOHRQ<8fk4} zeBOmG;Ep>xXAI{MaLH*gb`FA0Pq7z4LXMBQ(l`4VSst8VG8L>6%dHO4laCiho}oJ2 zp7+xxH*WPp{hli5RCaq@_$0+=S9G}gvy>1CT$rL>e0w8uS6P+g@o9tNTfUlV8;!SD z#b&{sYXdnY_W@s=vDPICAP5;Tj;j3cXu_NSc=phtt*=xW4ha&*7^9Sk1CTnRL!yiiP}y~sxR zuDN)?J*Ud<1rkihqveVW{dHOc|F?u;@CH^0iFClRQ9ox@U*+tyXNg>JsIrcr7; zB#g5XA(uJf11M_6Js@JJ5-A$i#tT84u5xV=t|{~|V9}}!!6HBnkUV9kI*CN;4n_%~DVUz&F}BmmBJVt;wz1U@f@62Nq<8BsaJzugV`*(*aeu6NE9N~y``lXd~hkH@t5+qTy z614N%%Z*KR?nVIrcF|i(%8or(1&{k?c+6hV@CQ1R*I~5}WuGF%Y4=wXT_^}Gy3>-E zS2ihf(2!4n{V#(QMZ%1h1P|FjO}K>O7J8Lx-1;C;P(ntKiIM}~TIFYrx4xNmypt2O z3I9=4ZH&|0A5GU$Zq({HM?Q)@7o5kWj=9@}Lmb-Q^zjsLI9yqU?|6|Xk;i3V^li^*7%>W4gqK8sqch5}# z2#TT1lR{0l35)XZ?*%at$9xA;k-p}^hr#2FqQj9?Rg!I3osOM5B-Y2W$p@^E!jlk@nN0HeI<^Fr)o;_%#{_6O}hsPRA5-9PpJJ{AOs$48fYL&%vT41mf| zfT1H6z0Xl#z^?BUi_tam)e&rbQ_vpo{?m-T2j9<_?^OPP?=vw`$7iy=&a}tw6|ROFzNk{JO10;i$6Hm@bfbM zzjFZ~xCHpdfge1F&hF|7c(#-w;zfN)a9aSpK3>D4wgnjWo1U=DG zG1D5+SCW?gshye8*(uEekg@B^KZrZrR)VG{Fu1i<9B2BsjT(L5Ge^gL8PQKpY%kWQ z?&AWkx7_JzHP7PuW>b<@J8zUXz7lfJZGX6w#0s9kW;pyD--?RKlRqyx#rcfHDtQYE ziT_{T6U|*zFwv0Jo=ZhVMYJa64Ic2_-Ob=Ag4QSrYNl}D zQ@s-v$4*dS8_u2lo~nyQiU+r4fDV=jDJq%EK4+m?dRhzo2|_Mfw@moETiTG(Rm`I2kp_J-#y`l`%E!1kdN9mdYk`R1GZ4b6R=(1yD185~xel1htr+r? zP&oqznZ<&f{O<7*GY#jEkdU9V4h{~E>zRqNT`>#+Cm# zH&rL811=*kYgfOdl*KQnIJHjj`N}-GOs#_bdrK?_7Fp5^??2- zsV&7#D%CbtF(h@{coaKLK&m?TZKuBMBI;xWXlH#NnzbVi@k*{6Wu1tT2=M>w*V;GK z+88Tf#+((}A+Ny*X?#ocY4@j20)3@zmha)=A$0)?r&sMzXRmaHx|UWjm;=3Vy0=(4 z`QdCA5~^inW+qLU01+>r&$VEdQkt#e?c0$_@0e+6o8L_po1j3lXnH25saMR3AS$)aGy$($jP%N9~d9@&+dea2cqffxwya`9>N0@)rseJ-kb&S za_?wixhifv!m^v6pJ$xjo3a1>#rc85qPK~OiDqL;$KK3BUF!++Xi4wFf+;1xBP!6Z z!md~Bu5W`(i|@8V79P%2d)mV2<7AOegQff%UpEnytAw+QCC-l>QYeW!#tjYSc{vTX zXpw1Y{`(gdQPl3-L+<WH;abve(qM{R^Cp8?g^Qtx%K3uMtZ;)uBm zAR>5XWtAI>TVAk>Ac|>EB*3K!!nq=2M29IKe&Gj+uut{ z>{d~e%ljfi_5C#;YcBb2JNnn%rIy79E~dz(PHej~=gxOz-?k101=zEEMZUl_5>++5 z%lSMW##@9%)X{Fz4q zJLPK8GTYyA-c_R?YH-0d)eq*AP75>PJYr<}Ft^*1O1VTf>Ra)ZZ)RceDZ2sB-Ku`Y z$FRjghPhr|zc}K#>kGf?>b|t+cjktY;7rP{u9awqZd*WPIVD~zuwC7vj}3OhMj!^| z7+2yQ0BJbA6OA71yu4%Eu|Q??K`ew8iJD$;1)%4T=D|ob2S!|F=gzLKhf=6J{8)Iz zx53n%R~=4^LbaCtROS{I6NvX4ufV3mh~b#Rq?qzRCE6dY;~{WGTo@!{g!umVFGM(8 z%SsSDECpp5-r{gncOss!F?ATO_%#JMt*4@HG%?!GZV? z6JxsbXuNA=^iHWaNT4LWYk?n|rj#l;;h(-m(rLTXdZab~nMsvnz@XTrgTeEBg#Ypi zJbHa?dvKV^YQ4Va<+4r$IcY$7oPCNs7q?Us5nd#3aC2~)#i#0gQlj?Oq=32LU;L*N zaqvBYg@sw~&aYZ3YpVvN6;Oo_XV|9*>%F@Ak(Lw|*ZLq?N?Q8X z8@zYo?PrAQV^0>yA{E5V5i~RBr^LrpKg4upQ6=w$wx9jrb~0i&;2TdNO-gFqEJ6tl zZS0PRdwO<6l49B3*$2rWQ!BO*Cx!X?&bFHUT-`GmjE=5l`y&aP=KF7R3vJOyEEuf-oyZvFho1q(}q@}w-1*8QDNkQotWdMnx8x;kl zLqNKvksbvpk&+r1B}7`fzH2<|J^ynapYyHdat%Fm&wcIq#ope)$kTSRk^uvgY%1kg z*VIqJ#*m73KBJCLxNP@o@ipvYz=s9l*nJ0R_2Y zp3koG!oD|}a8`8P+xp^9yhT%-z5_ZQJRj?qV;!CO^8XObNFp(7Ae< zljG6+w&|y=nAo`Yh_G38=iZ12C=;DZ>{>%PrRo!ZO(yY8fA`@S< ztg}TCJ!W(WHCwL)+=DtF{@t%FyYy*Pa5coIJIb~p2fV!D`p?XopW$#PGT7L!rcHBOQEvxl+mhiC5i_jD3bv06 z2T5XvjdSiEB8ok46wte{(I+ga6?mOPDXH+nZN4c5%+1$C6SfJ;*p3gKN$ODnYN<7ACdGMuJ0~mDs{mCQrw*aIJfHN!!97{tjY4q9aF7&_ zq#zd`C^}%_ZYVkYFh-i{>GnTYc_Hnek8yRA6>{d0apTmDM^8u9zAMLd$&1;gf)6S~L7_dh(x5{X0f>8I3r^IbH!wQ~f z(mOU>pEmV;{$T=!>{m3y?#@bBk42lHsj-3oNv2f~EbsMLe~}_=od|Y4nX^QKYAwUF z^OX){Y0c9nwg9a6no&^4OseD{q5ujSsUz%dE?S>p5Y2YMdY23o{0KK9Ve#^?@%=U! zy7h>&e>h8Sh`|fEhcZMyLUeRSdb_uwYHh$O1YQMc;x=$}h)}**#>FLwWrv8=$NUv* zqeB`Aq`TGh^`Bn^Pt7NI@f4bSr=}cIV^TJ~CYZa~jcxi^Y~p3vPNy)gEZuQLr?HUb z!y%dpOFvL*I;_4Zt8Mnqd2B~d(Z1br_L+ubDGxQ6=^DT}LFFQ^j{)-t#0$FkmDE?r z@XRR?UzU}KsBHJMJ+*)aWnV<lo(SF4b$4MSi#bW>%y+0wR$>lM!j4(m1MVf4D&Iq4s?HQ zLOR{H$*H&q#k>}!VNpSFsATu3zA*mu*|;CLF(Cc%(JIj7R*WudnJ&ywQul~2I8AXt4>mvixXj^sS65}$|& zavc_||29qbNog~qnO;OU?vcH2WN}W=w@m6e82QCPRdB;a*76q!U(iwO;(h*+rYZCmHwUH*HC_idI`F} z1Lroieggism}wHIi2^f1TD!5GW;UGV^(9t_HoLo3iFLgDBzz{sCaCs@eO`-We>W~@ zu1zm-NVCX}a65@mDaOXma}855!u!&^m%(4AKr>k0T7 zSy&jto6>B4&G|A|2L@o)6>(*h*)A^0^YdP;4T#>G_Pgl!Ns|)84eIEat`Wv>_M14K zbt+;P>71G)7f@GV7s6Q;O35#8V@(3t4bHa&P9DD8;R-Sps$I_^FET+D5fzS^caJ^2 zz)u3{B3wnORPzoc7)f=&jl z)1NPWQux-$HqO}o&%!2JS&#ZTDzA7X5k7|pDZyT9o+qCQymSgSk*AYFb(kZ*Prgxdsc+wZxfO?JW06cd6K2~ z1yt$rxyniby3BGIgWHUer>Bmt)84FFdbH+atJV6!0+|_+xOIjV4|?!(a=QAoe?-R+ zsoFi`J!EUPTYs&pcCq^{p!oA+3MOg?_n61j)a8!Yt-E^oH7&jsOj<0w zzR`9%?B%dx%+4h`=PVu1N*RU`Yz;~Mx(w=5&o!9hJwc`zhGc3A=3hy%1OC5co=3U0 z)>agSfA~;UXl4F%RX;om-)4h6wH!l|1p>KY*|_3}T-HDe2REA=pN?tG^9v1aWcOwI z`1oup=C;B)v0UV<-^sctR}T|z@Qrfx{?;z<>l|52wDEFY&_2(o#4h5t|90~dJHBKf zjXUOuXX4OHIqc;cYYXn&EQ4 zxNb7HZms>BpSSX?1W3M(2%m3I@vPV@wY2yAYZQ(o&|+TwaHEe21(9Jw8eq z(Hvp<`j&?FcDT40v>uLDc;I=o=C%}dCB`L?)%a6XG)DG2LTIMG1(o-RuuJxSLr$kD zmgMIzD8?lcsovU|II5CmX6(wJhG<}6pHon?gr}|99G{$~iHu>f-;^Ki|K2i_Rthf{ zV3(%9rHhM&Ef*}Xn{88p|KJRYyW3LC5C5<)yK;e?!-wAbIysrn-kE^dvF-Zm0Qm`1 z+J!B6JX7|xmW3SaFDdk$QnYQa3FUWo2KnX4Vu7@O2DS7gInU_s2j#jhEce*J+}zpT zj;XD9h6gba*zoVk=@x8<_VK|rEKbid3eTzCQT_;g-@BnkHtFzz$-ZD|z9278)M_>n z+1;yl4Hz=kQ_tC`si&$v(7&`ltkr&{xsX4^9d|OAx9S~&yw%6Eu$s{}50l$Gvt2(H z)Lxz=ENVjrNXG<6-zel^5QLKVowWUvBV|^^VCv&HvSr(&ExHSIZ3_zRD^=cZ?*l%K z+xsym`6d?Tc0zSQ6migp$OIggH;ZhwA3gBV^xU|UvsL>w=DBd#q|F9dU}Yp7NvR2goL4YC2s0gfyot< z)G_1MGhi2n@&!J{4RW8(3XITc=%4>~u3=tYz2*a{23x_L z7ZH&m|GuCvpJZ56K{VJK)!o+A)I{c!zcGhy{ggd;*q8E{hV}J`$&KfKz}6zEOSu16 zq50o+XD%rg6)W`qpN7#1XO4#T)HNY$*i)MS=%lVfmLiK~WF#PL2f3*_0+DOFlT|hz zz{xc{af&ze$Q|?i;+aCF*XX3*!wdl`E9=GZ=M=&GJLP|0r$2w~mGkYXKcx~(v(st` zUU4|UNDd+S`SWL|YD0ZhUw?l(O~MCL6O);RDm*81F<&|Y0it#rxMwajFPzroby^XFPD}7q~{j`xr|MA-#F?rsuC| zQ49D7$00@_`!Zi>2ZKWGz(`+i+2b>@QRa}YFns&1x#q(Fe}7p3lR7saHW|Rswz(L( zY*%oAa?e8cfA>!sCE~|oI|;wCyD)X~;%lT5=Tg_gMRmwU+58O0p=SFw%WG&kT-Dum z?Sxz4I#1-q+YU81t7~K|{GDUTxMm$tUc;B1vNSi_y37OrZtuZoehYx-uP|uc9tR#P zU_n?11_lIHhRvR7SycJRf1R2_fxWWavYu1|z5BqZd`(brXpRi9dbK0P0AsVm!VMiA z)zA0^lzTN#Htx;(g5`#})S*mA$*8=2;y2b!3wO7B4QObmkUb+K>^xOe`=Bxrzm6ey zv;rI#3`rG?aZS@U{G3BRR`-YUbp)eH$m=hDGNpUoY$xyt%uH+`kM9j1!$xjzyA0zK zF>3Tl@CsIG8QF_PlE$c-ttCmRgLeOpwu^zuJcg2zwR$){=grXJG+N_(5+RS%(Y1=n zgpHjh)cAR!iz>0c)p+#YfdAJCN%P<(dCAPach5%WNQ{j!OiYxa;uKU^F)UfuLEqH= zIj zKfc|xJ)B2CxYv^yn7X^0>QsGt`QnkYb5~&|QFsI{`@SK=&!1ottT(57MtV3uu8Fkg zbvA4F=8!^SeB0yakEk&;$&ys=`>{y6HiyjdzXqj6e*r&&T~zU_5Oyq?)9`ELEL-h! zzY=B@l}maBPFn31a`i{iN*#;I_K0zVE^%GKq0aLezBI9Gef?^--_`voiL{+57+6C9 zx+3c0Pu1OnWpJO+br$c}EWZ2>hKMcJXbi^Mr^1pzjsK1f|KmXmJS05IJ>(raik#BG z6|X-_1yu}QCGYBbIEP954+4{tnHyHk6+MB2;il9-&jJ8hZeJCg?#(Q-FE!b_Bv0v z{m)()lZT}%K_q!hZM{BJ?%A1Ob%0d0GmXe&Dx_`9i=czfFAo25J#%@`d4urhPa}{w zgsDAEVK&b`Oo1-3(}zl@QGE%X81)KLDeh6*JqDrUWfp1p0^AEaurztUn@@jD+uGr} zD8@5N48-a?IXc>Ph2qkSIetF)s{JfyS;O7Eaw0VMkqTv)5K%1o+;w?aag+EsAB%!P zv{L(1$ObE8LUUswJMSpnx|8-!s}f7^`Tdd=`DtHXa{eur#F_Dzo~JsZ3l=$WEz^Xq zk9lL#Zw%sn6%`XC$m6@i$GiyMQczIaJx&@J*r;(^<#JujO1L_VE8&(G!5v?+KY^VL zq0HokNqvQw;|EIpwU5;C`@Z38qB1t`H$69Tw`Y_Qz-uM9>>m!4fi11`aa*2mXyXe` zj4z*Hi+Y=h12|n^{$)-Td3t)to_E?lyih|`?X`1>u#&#S)4$p_xJ7cvHF*(}MxHk} z&2nDtzC|LDPwR|DD*Jvf4dPhSf{1*CqRS(=s zQkr&`hK2Duh=_=GfG0nrv-;VIw{};3@Kq2$w*r`7R7P0?dLC-dv>-lt`MGb%8Zd-X zQNK4jDG33BsqUQT7sdnHrKNkqLb?d-dD7tHL0f@VpLd0_L}Yu2CDaLQ{lLB{VAlqc zI>{;5NTpZF67}`FCp8Beev#kZk=7o0<@Fxt>SRS!zTqizk>1_diXGxKU@yZK7@Nvz`)7;K#$39+ktKXWHx^vJ- zS3yc!z0fR7x41o^VtZQkkOv=dA}e<~mww+tU{}zW&nO>ZMF7ZQ?UtuHb}ZJj*C{E>VD7fY-MeI9T&wM! zJuuWD46IW(_vdQ4Ch0!+D8H}#ejNoRrPl>yXOK%+m>i@UN#F3YwazU6I8aqEk}uqC zP@gPc+=((JyT7|?GN{RUeVQ$!(aCoIy$6scq#Hx%3c6N^~UV0K*1b@8Q_~&V+rLVu1O0u2#VrwR!6EVoN zvrxR=^vdG#<}U>(g9`i-w$njLKu+5E-fHI9dmp!L=d%OJ?KpJ}6NP~rKVJmBr=NZ5 zm~m^jteD$c?}xDk@z<{yZQsonRj|TSvx9zW{!4C|yUxDUrK|t#PYF~>>Dkx>t81<|4tevL}1Vg<)qG)2 z?ym_nl~g=CWqxpc0x4s8k!2dQ_L2STgoNE{$KXAereeY~jw$}kR2`O#PfVJm3@HN0B-rm7gqjE=$H z#P;TQ9*;Ed7}DQcRHC{Tsh-hlI|7$)ySg7q1J72J>FOnExk1JM+YEMUxDVj{87Up$)c45WsyA2!Q+krm~!U89_-7MZtBms`KLNymu z`w(hg$FX8RsUf94s6crB|nprBc(Ma$h90mUjN;iEoev%NC?+B1bfgp)l8EypytZsV$d)i>UVo>znT+I z9QNv9$-$_9a508NUn~<1*=ZRnkH=YFdNt_@ReE~*7*^-v$Q0HkRLpfDB_eZ>#_Y)z z?^`ZwL5hXR$+<6~D~@<4fyj;V858J%&*3LjlU&liL%};0(XgpI8;oSk_|`!|rQ1kC zqfP$5IycZwaE?w-KO(kxlRlgLYCK+U*GwqdZCe^|H4Q6Msaf=)G#z5E_z$16R)+4u z{2wMnEnu}v5s)`DG_VB4JdxtRaiL>M1A%-WqjUF7>S<{I*23+O$uH{L_0|jf^HA>p z(Q~!j0kOQ2>gz80f+V0XNb~R8h%l+UkIfRain-fAiF%3l|#Hu;pk;Xa?Z1p*H z*8Z2Apq`6Q2-Se;d#$s|d;K&AFBC!W`|%Qd87m?JfY!*5FRpPbms>Vdft4w_*vQd5 zHME+h>)_}}O2zKubB;8{Kv__rRQql}pk}L_#CzA6bMCtG`3<*X(|>V16QtWyV6_5h zhg@ghBp5ODN&DtO_*0X5LuX0|NOwOhHUCupmup1GJ1LVV_$9SP?=M+pVBudsDi#GW^ z8{0XZR&kLwpF-gT#H7{Zor-I*pqErrQxgjftz@NR{;oxwSoo;)-Mhv>SNY41=orHH zuXGPcb$R)#v~)L3Ov(C4SO>kH)W0Eq*D|uMcpshc<&#?u> zA;%{t>_U<_aLl0h!G0PFASAlQj~S>)rnIzh8{d;+m%T}ViP<6P9xJ4R8FbjVAK+$H zkdSJh5K9sn93uP-W7N#p-2cGIRe*wiEuqZ#-}-jtBQ)cuz&sHP-h(R+H0rRLr}w89 zU9mB1eJW^ZWTXq5ZV!+GtW*?mNnPXO>idGVtC5Fv)^D15;OFP}-owU*ePuW+4lEfk z%bl68Xe1!lx%Ok#cYhxL`xh3~kk)6dWlDH`V#BZP7W3pS2P%}^+zJSBAVkBN!hPYQ zUIZk+dr~&DIW>h(S37_% z*KEhyRLpLIrH?&WpH+;E4db?bSCJoMu!#2%{W$iQ<;FT$&uy@}XW|N{88pLy|J$#~&m?h9VhKl&uZ1@;U z^O+?9tF_p$v);4jlh7C-NA1Cl5f-KUWotETKOdwmzI2Hi}Ej2BD?rP1$hYwQ{W0tY! z$-q>J^B9*9YXP^3(CGO1whn7h=baiRL=_d$W{wE-S+Hfje|?G4LgW=yQU;KHUxEij zq8v0u@ruGNLaqh{=|tS@u+UJ#(Cp2c<*g;O;3ezKTg1ch#!IShfk(rVYj<}JcDtJ6 z3Xf4`TGbAh?0~*-?8c5rU^(;N9ZPi*336b5N!{bMr$c#XzGsK0*iM{T{F#X~9K^&lpVpmdH?2X9GWCr+x|L;cv zmQ558GQgI8|CG!v55~jLP;={uXoYX|Gg3i>y?uNtC?s%+i@fbv2xv=IxR;hzS_N8u zR@>Yt&|Q3JH$JITeqWS|8qtUPbz<0dczIji^GL+OF?IGva1p(1&V6@Ge(eNNDliDSb!O{tlJ5km<5s6-!@Blx9XUGAl{%8hN?lY<9v zy7QI3z@4N09#k2EQ%GotY8Bxzc&H z3=_KB50S7ot+YgY4$J8!tSEt|#?c-X0JL>L(aCCOZ-1p7%f+fK)eDB(>7e`k#;>|i zedz-Tr~x{KEjZb>(^TawBPi$kR>A!sBA8^=Px1d5>!B^wzT*_CcJOTP%t6_ z7r6)c!1I1lv^L{Y14@vKGK=zRQ5#>x*^C5WpaMU}7Znv~!VIr@{xZ%ZBnLszJ~2*= z?#}Mep#ZgnU4E!eNScz|(3k63I>NnqDARGNljBo%S$QYC{ON&C&i>>}9sxPII083|)T1q!NCn;j^PZomam*C%`p^NT80X;v z$#8n*YGbv??&}VHX1iHrsH1nc*8Pp0w(n75rif`tMOW0J0XA%+XTuN|@j?dk>|@!Zv%<($$M2 zi%rJ-2>~)l-+UbYHBR}R7Sz(A47L{nYZ68vE*jRYX?vSAJIEmv7s*H4cD^F&LsVS^ zP^H5WsJgXBd-Tw3?M@tm+bo?Yv81+i!IHf7&{{IJYdk3t@z!fM#kr)kbRky5r{I{9 zfigU_+ZhnRgzB0&%Xw#-X<=c*ghl43t5<kP&pFc|baFejo69y|; zKWp5l>Ns7=r+HjdW_z^+P>|DI348r}50XYCv+6GHv%yi#&k7ko1i3S6lEw9x#B)+p zq`h2+`M;p?y__i(O_LD(6l>GiU?6?$&~u+z$trqA%uwIhFw9M^cIe?w@CDnxZtn32 z(8u(@QupDJ?#4c;?&ysaq$>}cbhFuvBEJ8F`)j>8c2!3}HxBX5>DwIF%a{Qk^dLc=@so(-d7EaYKd01Q(^ z46tFmA|AWg>v#vIb7IV&V)8a)9dtZ;PswsTWRpVk8A+l^B@IyC4`?pWh{Y(VetD4e zC$!vnF2Y4EMO)HgsGmRzD!e5W?!6{i@oBcWRZ=vpog%s3eo*|vtMaQlVHDJ$^l#6w?kTSDd&!ydrNyI0jE-Wm3 zPtHO%Ha2$8IK9h{b+A(gQ@?!ePG<)w)E7z{mHVwKnvv^*Usx_A#Kqx!sj=EFg2=nY zST5HA3-$&E#IRwb&oQbMv2i0~WBrLU_v>n7{a>X{-EZLv4z`w)6yl!F#3v~?GX5xB zs_ABxJP~5!Mu4~GcRS_vR60TDNLnbLtujGq=()bJaoBl2WA%Wy1~d3B`8+)o`GbtEl2 zvJ#bNN<_Z3<4dG-c5-sRA}XT>EkPERVYM8QGd&u|L_L0CA4@PKIJDDXA51dc>Z7}h zXZNA)9*j4OLh09V*X!;Z{5zJWQ{g^z13>@%s+ntz6X_WE&1)+mwYziGI>+8oTzlNu z(A-I*bamv8=n+<5*($P9Y((;#QS^w5Eo`uC}J=2nfOzKOJ z1>)7KN%P$e(QEbS9{R=#J=+VW3)a z42EkmZnnpypIvNGX*wPb{wW7L()lN!+dq#RIDMY{l=`KG7VECgLg1?lXq1?V_oyC7 zf7~&OH6t4>X%$2f?oHxf!~ZHU0B+SOsEnpFkk_4wz)wHm0wBkzlZ!l5GZ_=w!OJ@L zh%_1~MXE#IH^ylnO)!#YSy4-&d}Jxtl;2gj)rO8h*{eH$58VociuEs!yX~?5=!qp` z@@RJT_aFXg{m+I%J@<&hriifr0Twp)Y{kUrC%!r`&8MB0Q7)H}N>w{8*S=jZD<*dW{9zU@7<^~wdEJ-dLw zXM4-fZ{2>Ez}u+?xFX?&(EJx(lUmuy<^A0fj$@s>B}1&-0(cL zB;2bshL^-dlEn5l8F<|bTheSiJ@tn`Ak#|znH!gyv)-%HtC@98G-pslsojgcJ#T6} z>BKF8puXT~K=9;OSC0nmF|tW}vga2RD0Jw}zbQEUlZgKJQH8%J;GK9`NsRvNPCS}c z2hwq*-{G(OXmlPY8q{>xI5@gOm(ebW%ZoEX~iVhUylhg`4(+(Kn4;mFRFg;ru3`TW- zC+JIlgP*_dj88^};hJW=$S#~|@d_+m^gP*n2#S}|*?yI=B_Qgbu6;;cTU+~!b*241 zSeyGfQ)XOU_22GSBwA6^rg9&R_RQTu%Mq2kLb|4aJ0}gJ%VbPXeisK|ItC^t5n!dl z8(tMrIDZX8y@O}=2cLuAU}^B@AIhT3U^3RqGqd-uZwP^g~eb9gHqxr&(?1~~M9v`GRWm5*p|8dEnpRPOtV z$@09O9`Lu2$E_4{*ebu1;pUYcmC!f9G&aE;8&keTkR_949oW{<2#RYk1^g4%-qrcY zRs1drF}s~e96cq5EA0*;rEcn>Jk>fVH-g5?5a&d9l> zk0S9jD}&bpgjt4mZ)sINWwUyS$xHP~5N$)$U$H>hnj@EyJn)Ck<4h0;+9S|+Xw%OO z6?Hs<5rC|<2OKRgMX_m`o2R!of;nx}tYGf;Lpz-D`klU9&ajsdRstI^jO3|&SX>;P z@O$NQ9t(7@p{kuKb2ZD?TqJ8sab6zGH&A`>ARG4!3WaiG!&~J0 zy1QVboYVC*mH;C7Wqf>7ng{|^c)U89(qqhcB%kZr!QD@9-yh!v6urL`8GzRTcU$S~ zml!oSue|<8AIZxe0UjQF^jd?Ma0qSMjTpDRjsl9#01w~*lVm;Y3HaTG&sINjxtIFE z1}LAJz|2}*3}pWi-=Ccuyf<#to}B>G1h0_L`wL$2+6tiiCQB8vx@iy*Sfd9hY}dF- z!dL-BB8%A4i@UtTvwpOmip(SN3}V%uOR7m(`dK*|n;d_aUZw;ecSlW!U-k0=bA$!D z4DKRS^`-(MJ;dSWYz@$7Q8P5zWjwY8A35=78yOn@xzxi3y*_$2&*KXuj0)bW+dc5Aqw4LgQgCCYMMG9!9FdME27;}tlXzC7BwPCmv4kX@=U={X zDxCeQ;}(!+%0h4XnGi1((x+E>Oz_0!LhqE3TOd$M@h;7>kX}$ zU*(4CuPp~^!=Kv+fx-VZz*|Pm7)8$JexaA6IB!@q23Ns!{POnpVRFedueiQ?E;21G zEjnPkeJYbZlWx{Fop%f2gf)Keg%uEdePL-!cp>XSp}N z1i;nHy+bcF!U+%#GsdFnHJdWhu9nr$%EtEzd;78S^0JD}^9eeU=U_vK`;?sNftcqd zFqBv*XTgJryz~@L-o{`+sjKgHgSFI7{HF}nt3R)7dm`UdE5qHvyenFV#rnt8Xx)Sm z_SP9#wQygKq}!+sxd_2rBeS`4O%|UQ3#|vhy83Y3Cmi(BIcEFf^5}!qSQ0#ti2PR= z7T#=OS*SaqEHf{WCrR)*iftEc{h1CN8VNcmvwl+-dURu+07AD`n%mTQmK{pGs@fI7 zmwv2Ykfwr9z@l?^fLm7WV_abJIM8Q#94XAxJM=zXOXsxMLioKT&*FlpI*w!qwpT{X z(j`6p+vD5BMKGtc1iwluEU@nG5%el~L?25!4WO>tyvvZK>&XXSlG{ha= z2RFlFb;f!>G<#>b6BNDRAgK&p+08C@-4g5$i`8DtJUgB1Gd%xr)xp*irOv_0NdR<( z%h`Qu)voLMU%yJC9VyGlC8}t7JkloE*hGR|UI0TW*a6EhxyMVfjwJYWgYt968|RYr zYKZM@V$MCAy zRaQwFj$b@qkxEkv?jU4@AktQ18aR)owzUuUyNQL<(mIXxeuaH}NfGuREx-{kuJV?a zL8wZw?5nkqT+fftJDGG=6gpL!bdg4>p{fJmU^&_6rF9+2I88oVvOJcFhyr>{U|$0p z1P7X`Sq}BQ!W!&&ZEBSj#uZ)&_R)J!aQ_`AuRtYccqO-*UISpYDfEe|PJOTrv++#k zF9544xx`FOe4a>11@Y-aka2w0O|@@^@$r}$)PLOA1(AFNE1pDpYr3ZAiy+u3HGECZ z!h@3aJ8bRS(CES#RQ|jQIk3+ESxG|S@NK>*~N6y z8W*NP%+UMH&{NhE{gleuFoD3g`+jvK1LNT_nTWEPZ z3@DC!*e*k!Y)sxq=5S(;%Za|G#FdR@WG1&F{*@f4KZm^NO*q2si(b6g@?E)QBg<2C z8`z=pF|Sr&$N9H94?8YTU9_L5`{IqWj9n;G_EHk_-4P*$L{Ln6BF|wXMz@gt_>6s6 ziOENje$t8fhdqAdGF|3$Q29~PzOP0Q4`~lYt>gII9=JO}V#F|Si_qGsjRN7n)Q1;I zAp zZIMwu%8(=G;4cQsjhNel3%)@8Pq7y$OTw#DZ6h&{!+gz%b$8)BD)<Sp8|S2J55Bf`8%^yo+XD%; zNVGpdnu*hH&J&wT*5wL?<>~Zw(|r3pHcYcbqG|}W5lKlTz%yWocX*y$md3#QP$UKO z9#2GkD^F3QG;3RybIAut>M$U|au#J1?~lI}^|(jm?83A=Aw}mUR0na{Vv7LA5q{Rd z9WzDe4Ms`;rvP>g38(^KA&MU~E!`PL6pOkUVmaQ`)_n9aX#i_O&UG4tS&;-S;B{jh zHIQWEi5}kY|1t1-MF@E?6c7T373tXhyiTj{c!!%dWaZ$K@|n^lN-3RQxtJV0f$HIn zY;g3yzj2Ov9BF0^FNp)@3uLZONQ2&*P%c4d3o$c(*a6stkTjvY)UN&_Bv6cnvZ@b= zQPXdAo;~j?UYE`V^HrWBYIel37e&$YQVg}FKykaqFKt!sq0 zb4Q^lFn_;i24-dG@ZAf-CKJ3c3!(AbmRm16K1a%*wAr*~^0>%9XIa(E3+pCB+}VZO z#r$n@hF{(Ws)Z~0?evB}lO@&w7!j6cSuvE$5$r_qk|_d*h=Szf&8C5oH^SLn>P2R@ z7%!~pq1yT(HcSwvV+QP^>?W3JG`;#Up=`eWMNknB3prXGAXK7pD_V)O8>?%uye*&A zWP+4i01%t``L&oDToUifoq<^#gOd}Bl7$8b0L|j!;ss6hlQ>ae9>nqKspSWICMI|& zR_;3jo5*}%m@^WTnG*D{d@uqbgU#<-?wGHh*i>ing&(tS6UXAV4X-=ownlA_6E6K4 z*~k`N5QZ8{Z|?cV(w%2)dZTItrtjt@9TCl<^cJ*TM{axcyM(&!DSi)NrnZI%^NQ$G zECt?Lf(s|61{)pcitGV9viThz_g`^{+Q1^Ed|2muJ`vBLUp*!ecJf#vX1Syv7q@Lg zfnNy4%=>5u_P(dt%Mzl?dv8=hL)3mWSLIMONx2S%NNK0iz&L5s1ICp>odo6{L+)`x-W4YqJ=(|t9 zZ)VGKwZs86Q&V?FC7DMPxwJdNZMi}*z%FtANl@Dind!byU(mgKDHVsS&Ovyi_l4ni z!?Ig6XYG{p;3$U+{1)EK77CpW^|3CCsai3~T9g^DGDs*{kblz_bW*Y&UunwjDx7?1 zsAGR$&_o&L1KreyjW&Sv4gfIl{E(Tg-(jAi6OB~zQ_ZlwLLV{uG_Jmf2Zuvz1y3@D zWBojSPfQX$@%TMEb;{0z_nS|Bcg1lhdl~x*jw-2JavaIB|4~Yo_}YF&lOqnb8FTi@ zR_jF^YAr@|&UMB4Zzv1dPterQCGVEkfa*BdrDB!mD!l;6<|!6t6%lzLW_17GtkHj< z^8X=A43Q!PW;1TL1FpZyl>dCU5SrTX1~*p|4fx(Bnvv3#>;K0$kN(lkQy1S30|n{M z^N?_Sa`nP{ z_@^t|+g~FhChj_BYzzs(0Mi*}^!}vtQB%UEG6p~blsTE9{>~o%g|t4<2QQ08)z3}_xMj7xUs*fN-Y{$JE|3_R zB>6_2|4r73MS;g+q_yJdY)kgsd{u!TwD*h5@EhNNc&k}o1#7pV<|n}{^THKMF7v@! zs|5!VKm0#5NF*I@^?_etzx?T1QM+IBV5Q)~rq$}^yl?h{0+xpH8L_H%m&L68nMt2o zP#|lK^iGc8_-_43%wDP(F|pWeIB0{mrYt@jOldU%174VnrlY~RRl zuX#W5VI#mT&;$Bg$9GlHdAqfs!Vd~fXgF%0SlDkWaPo9oT+Pbr8D5#66e}q3`7&1b z_Mon^!sqMQ{^-HU_}GyPz=SrOytJln1uSw@j)c7L|M_WuE^F0&V>OCrB=YcZ^VggC z1)toyfYNrC!U`uFi;*vh9R+g!DUFIeo1zcLn}KvqiuP8sk6p@H z6xhtIXDg4?Hx@09W==2H2jtDITeGsl-dDnhvvn(1%PMN#KM`)(&swCtI?UE}YhRq) zud3S^&${~1aFpq>k=YtF)o|pu;S>-sQlHpAxez!uSy!fj=Vryf8FS`@Y(Ojkb#Igu}|h_or^#GhZeZ<`$ZNtu0=2yO?dRbNV26 zmH0@(L+{cJT(Odk{qfiKP9q-^x8f2zeB}Zp8>J%OcLbX&@qY_og?TGHd3x1!R_+$3 zvLRFNoAl(j&d*1~6;*YK9roY+9a|TKE4J~c-p>YHHY^|xzBymCff-US?sxSs7$o2m zk&}43m;`QEjGYCG@!r+oyLt1=-5XbqotBBODzk{S3JXGdT>f z++j%w{RsnlJ~^SZ$?mY$2zh_RTEI4MZ_QCQQvx8c{u5pP`5@PW3XEE^F=s2xT|Y_Y z2U4L-Y(jFWs4{b!{}IprE42Lg|7$robEUhU1&R2@c$nC&bJK}?Ym3JbKxouI^91@f zJSwBdq$2j;w7dUx5q|4oNdZE*;Nak($@>S9kX1`~ZS76|@Xz0HfJsZ$&W;n%7#{bR zGlQ+HvS@TN-&NzLoAOzZa0yHPD((pnSgLX+H;j_0Ou||6S$~f_kmoSQ7nisvCp z8i)764vc4dI9L40DK~xFhnSeS^^$dVNs)xJPEhtn=KEu*Go+%ft}gBOl^T-jI)#pr z5KwQeKFblpnruL>zJC3B|6f&nm@Ox8k*Rb!f8-+NRKy{PLxU+c$CI0gw5X2`nT;f>%->YLjywu)WYO zSv6Lmg#yG-9s3oah+NO*Q{aK%YPYsVw6?0jc!h- z0d_|4Hb)7RZy2t7h)bqmapS(l_ElzX+~SvOq!?f>tKR z_hX06fMw$~6PIAjOyjFNl_k%3Nu(txo^F17xQwMiPTEy^ZMhS%M!6!sfJ3K$|sxIYmm^Y(KNwH+{iV|EwZLlVGm`6>A+iBj=# zH&h!x^9E93oys(id6_1`QoATp zrZ|d3F{6y8!eFIgdS2(OnIGp==g(n%KfcBK)(7ux@ArM4y`OFAi0{VFH@(VDaDc&? zm+u|}HNa077k5yhn#4Dcw3kXa3(fTr$kL~$JrZlH#-3*SX0YTu{NfLGAp!PyLAh!L z$hR7b7L-Hc;ln>WKwkkaWXp#~pu}2I4)O4Nk9t~O!y1tM9Vk1!RB-lMFdN%0*#^^P zlt)47hOPEX6?P5bOVOdE2m!=HK~sfbk1S5D8k$N%CtC%z#im&cwCBZZj+W;h3V`>AFevBXJ8@9!^H)C z*#d&F$O23gaGVZKlHLn5x~-(>*0tesCwPeoE(Xhtf`rXGVSChEpu)r13OcMNQ#Y)w z?>SfEJbctL50s`!w;$0!S3yYk{6A>WVXlVF;R}Ej-5-0hT@AVraPAR%Ab{x|2BP5L zFXm8mZO(MHg!e&Kg@0;QXcchzkrBZ3ctpnEq=dSPobOKaStzIiVKMj~)K03ciDOUD zq}Q&u@AD5VvxRq}D4dI}!ChWq(9tEb3(U63Y^>x6hMBC#r@dJ@5kb6fwz1Fi+(sDU zy?+(r@+=CbzDfYhqJBQR{ciNLGd#+xoY*9VTk?FxSbqER-etW~KrVGh^K$LJ$DT>m zAvcW7DaXoAhdXWGXA_-Kz39e%9;1yDH-;G5UW~Da}yq-m+~80rI)upKZRq zyfDbDfwaO-A{Sv#OU4jcgAaZTG|<+YipcqTZ;I9nZpbBm|FD7Qw~R9(G4SEvFp_a{ z?XH?(Dlb__L-+T-E!;Mf=2%zjQPpNZG8#=rJmFiqgDKbAGE{k|z~aPX^*6eEodiJ? z2(6!iW`lz$_MwV&$wkm)e! z@^fDfkHioEW`3lyI7yBRijQvYtNp5Fr7rAV+W8oEL9O;4l@z9N4yNw^==3;H2uXk; zVHfPQz?N){jg8H!Q!)6mC^fnc$a--n%{i$BYCtvDk9@t;&;6blD^`gey);je_(k>w;R>%RD7G=K%B@`Eoc%+?nQ~H{}eR^v(oxO~%$tcPT zQNv@a-st&gwXT$96AMqYEiS0i4*?E?Furcex0>3n-(g&6j5wQNe$J$Kt+|n?L|wfX z+H2>dU6Tu^Yj7U2Wew?*;ZhYSbmNsy9MKW^rHvzVC|Zy1OQlt$u(p+4Q3GWX5X*2^%OHt?g-P{0Av-AXPIc=b(8agRV+`B4IsnimUZTbq zt`n8Ck#@wMd0TE;a^y!5GJWddYLz4-13f%CUupehz8sQr&)-5+p-|u`3HszFIR>Ei z>ii}3t@i7|SGw&yQ>e_pbS<&UW8X7FURDdZ(x07u z)OB{F+ioc^aKcBn~kh9u3S`ukhRRliruxr%H#PjkPZjOp~wsM3%eiuJY*(y>FJ za1&nJEJRsqaht#9Wn{_fE_*VF@Tb7Xj&-t94|=3e5ugf1%66^kfA->$VFD){KfmF8 zdO=R3xb=nJ7;&vM!3}=zVG0t~%hEUphLnhJ5tu=T57A`~0~Dgr$1`Sk<1fEm zxUvLx{pT_5uWwicb(ao>WRol~yatf>2f3FSniO*l@izp@yS@>Irf;*Qm#e?AyvEkk z>ZH2Hh*frPzqw~UbqcBjK{Gt^e&-l-i;rDK2DiTwZ4B`~3;0~=ogmVFwkQlskX zo!@SSL;ReH$*PTpItyJ&Wsuh$u|MchV;#V0<^#4xH3+qimudH6uW>P1Wekbh{e zLVdd2dD?>jKU--nxSi+PM--2}njiT&_BgXG~oGX@HsL zT=3p~`YlqZXxe769;QF>ZQVL`x+Sc81T|4Py_Qkw^upqcZ2@hRI8ay6HJ#gyChTSY z3Tj)E&9~&22NHO^&@)6M!`ZBfUQy+#K7O&;tMG1O({f>EqYIz$K4r40A=8zu7h+T| zyDN1E=mh)LN_}dmmOSCX(3uiF!3*8lS%bKnoVliwXR-V(BH^!ti{G&lx?1iXZ&Mvg zD)|(dBpC`)$-C;2s-aw=d&8&5;h&u0S-ok5^1wc!yVUH?AxGgy;(H2e0NI?K>_u{? zZ^j_2@}${Hfv1gM`S;#W;$_$y&zKgE@kf zH{l6aZq!3UVO=ZB7Mb$n_um@KP?n3gM{yhRpWbmFC@EPDu~_l7C(b3L^tiRG#q%+d zs?_Blw@2mQjC;KS*Q{m8#|H%9rHECv+6DucCeR0qEFTEn(;uxEQ{I%u&jfjg&MUQw zm>P@u;sXWjo5Jgi&qWjeWxD=AT;!wpmzPu;W#!|~vDs>n18wx56k*ezxEuUaK?^mQIepd* zGfad33WvYL)zs88{JcH(4PABKi->fAM)-S?``w9~+|Em_iQ+3~Q+lY9tNT9sIon+N z!B_Yp;n;;YPi%hb>u1!y8@uW;?rod0cb(!obDqC7h@XDDWWtV_ICfLDT}c|9J)fSf zwrFb^Wl&}q*1n#+GCObA4ydI z6d0)?#LT~&@wug)vI&>xcJUR%QWAcccVYP5jITqQ@q+(`tNU4%|Mc6RRo#d=vYT2& zlx+AOdGcu^8N1_h-#hJjoRrsSCtNr8FmqD(Dk8h3*SN`ALXb52fG5w1RTaWaF{*R`6 eOGfo)7a;?ppftVX8sDmckDpJVcdaM>BOuT2_Z@wS%-0v|z=E#qPa9!Eg-g~XJ&+}Ysg=%Rikr2@m0RR9J6=g*o005s6 z0Kjr5z{h;^noy_*^M>W3qa+U~9j4y~0KfnhML9jF$zHRS<4gTSk*0VT{C9+*U|i)j z`iOQj|Efi|;M&cRPZ=^#WR|}DEUbNR^Zw_TrJrcwm%2}GHEO@KxYh2vNdoB!IsAK7 zl)<670gw9)O{I1#yO53}Br8O$M|+k1^5+XKXWnOXNXUqZEuPn(K2BEn<18{R=<$dS zYHI%e&*7F=XTQ%)64aLOO3RXtC0Jx}LjdGpY(*FW8P@;d52!L30TEFw0%1nT!_A$A zLPdZ2S~l5xwQn1HIX`{!rzPQ$rQ<%WY3r4i36=LnNB{#Q*R|;O$W(18>h3D5nDf!I ze;)vhJxp+qghWM2DHu0h+Upy<1Gg&vWcj8x_jaSTW~$leuJzpo)7=QJZTsRR$@S!2 zLb)kA6@9cZ;>~g@bKMtU4|=5e??XTzkW%V3(wyatcK5}xzw{LTrt`8b@-n`*cP*89uBn99r9!?bMu5agO zP7SN6sKg8o4mvciYKa1ZBsz?6EC{?A-7IKbp97ISZ_=4^QKM5+9NOC2t8+=_tyirS zX9GUta&E^~GBH<~x0aup-lAMTecIA=cdz{l$}=6hQ}>ZSQB_rq4G6&9cfZl9zY)pT zkWT%VRrG8mfU)A1k$BSqciX9kzX_}Ta#-f?>+1tmI~S;#nr8k<6Wfboy060Z__0oQ zPR^;;oj;+6$?! ze)<_1NTnw(|13$slz%*5Q~!HQiG^ZtMg}AM5f-gCqhimZW@-dh_R`jiiCLcH@LCk0 z!{_#NdL4!Az(!)dJKLN6b9?Ax`jFokDx>al{e}~X)xEH=z}pka5Vt#B7WLIcgA!IU zlq{q`L`?jnrsieYNMB#f#RaqsXT_UV7{c;c>?yATD=P)Ieo3?$VaxuHKv~NiWbamk ztBl}rae7T#4_F-5NPGaOwD`Qq1aSbs;%bLt5>RqhAA+Y zU=*N~mQ9BAPmT#F|6M0FFMqLm#@5%@KD-`P@$XOh&)< z<<)@W;6~r))ab2u+5FTdr`XnM)-_0bFc9z>F6+x+0Vpx3kbfrVfMdbCPs2zPGrS~c zeXy65Sn9ofKo`4((o#_ikB(1v482Djc@u{s&8J^~9e;$8SySY-1WRw^d) zzxk;h*sb8%=3?~vb-aw!m$Txh!ou5Gx6#qjW1GG{&us7SmBexD=7pc7zbugR zp-nirxjc48v|&;9!DcZZ*Hf%A2FRPb zQ=ek7pXA!ns8v8n`%NUhcy`kxv6rYq#JE#36R?E*6wibPB>}i=a9&q6{AHKo{LOc* z$d|B*Kax?3rvT&~6%p~09&zj2F;DP0YE-Y_erb>SCq*$orEb&PNmJ`|DazJBphH$- zbTkeehbJHsH-$8t{wIM(Z&XjHOFn+KljAKc-@lI#W*dPpyJJL?$}h8Zi?9SNzgp{{ z3NL^)FdyPNR(_F8o;SF&r(~-1zW$bb*Sfs(&i4T$nD_Q(Q>HbalK3I-XscXA6&2Lt zX440q&*S)(4`bZ%wPh_=sw*oimiu!x5(H*zXT`^DggDrz{#!6NfB=CBov3~M%8JD) zHKCqw-RP7Q7$_jB_EI5fP&p#zW`a8O9!X+eq&Ihob<%JP^XE#jm)UTIwK(0=SV?x( zDascyTOnkp;adkv#?|D6Ad=XbUMDU4$<3=jBQY0LGNLjD4w%j=Ex3Q7Wel``s)eWRhFC|P zPMhw!fLV%P*QmOfeCJ1WwHqlwXByqhf&#IFBP3YhV<4Xk1@*yyf=^aUi-`i?=RZ(@ zS*obRl*SXzbGa>6a=ZzO=x$#$c+I#Cb5d_JAj7ryH- z2opuN-+PY0f7{dMpVg74^&QuuZEv~NREl9VX?AwD()-qfvRP0Jhep-b5vs`J;MUiZ zziavGb$D?~0!yC|l85w!gjD5?QJLPtUhH<^7_G(j`1omT9L106(-Tl%pY2DT@wrD{ zI0_U@lswO733ZamIawdGSe4Zd1b%a?pL|k~t%#1NHWYqhNRh4Zqp-?U!?clol27Wf z;8PaeRrot6lkBxTMQu;&qNAfP|M`%JBgI0MA}VF>Zd|0;K#v(z;++WSAzTdkfQ)1Y+rsTV`tR)(ihELtTCKR>sq_nY+${*_ZvkIx!be-`TPt5gdAoY#S@P1 zWKlbaJZ6!-nF<9mVf^fY)hHSMrPQ3G`7U z*99Vov@Nf8eQ2qWSQ*5o-k0ki;v%-pCZws;C6BPO!o5#&m?CI+G~qZCqBE5Eu)cFZ z>g%fa5nqX@1qt@?@JQWkAPis$gd`a!Hn+t0m}VRW=M|gOJwDsSed2LT`}YfPI%NU~ zjd3&srb@jp5zvwssT6$A#E+bteya1>yQwZsP@jIPo0=#wxtF@Bo{)?pyWG-C?^F!< z`C)8HimT>hbLB6vVP$Og+J~1##tu7)3-KQaNeE*fk-YJU%HPOWv4OcpB7pUjUD-uCO`{L@VZFx0i037O{TW2UR8m%BttuxYujiOllF zS?X;h7Q#JMY<3!@lMkuw6~zM8tW9uhsUZllj@r|-K%anfhns2z?D?qYbyP4#E59|+ z1M7&*c4D0F_oB|bPX-UWZ991+k|7{p(wcy)n`xihq=uPf3o3@e<;75t)|U6xcxI$w zffEJe_clzFQ`^UGDCAda!x!ojl+|`;llS$rhA<$4aJ8|2Q4t7_+MALQk?L3X zR{VT*IsE5EN(ODt5@{azLEJ!#Uv5^jeqiw48dslcWNbPWExUtibupi^_87uLH_G$# zLEajh()O~u$p@_%3vH8Zj_gwF{g}cda=)|*$>D|gy<9HC95fr|SH*-%$L6 zk0D+#yIUby`A>pEzYc13kGxhxu|#QDJ=*GuWjGe%gpd0CmHA1Ma+Ys4MBRmt>o+sT z$qbcwvHh6vRFtB<7>JNcOkSU+PaSJ68(Xg>hqjRpm@SGHz{MfKA(khgcN2AI>B7$B&Ap4; zRo$A{7;1LHG4m@1sOaE`N_2=8RPkPKiTMB30)>{+6wX6%+DN9o!08C2>&!nL+FIe_s`;U+0G z2_x+ajx#T88h4CTGuU-^iZJ$t(KBGjBt8*20TaeOLW9bH0(L}oFQ8BVVpJC?SXk-L zpE?Kkvr=TyIVJ;10!bxCwL3MwTwNyk-4hI& zF;gIWAV~0nA6T>xP=nCnP~P<@`gQ&$sChsj%J#+o8W_C37vXlQCyNfGd=ksf0e^PI zigj|D$auLjs7}fDC;$0?RsJsf6K|Vm+5B1oXEecjJ1o3*!BRPqZ6K}hMg6280efhKB^uM?l zDehpqgQ7N8tqrM9s=^ldq@B)Bz_{U4A-K%1zMCG6&lKMJMRM}+kp&n*{;qrH2zKbV z*(GNmP_DF<{pXJ}<#QVwn|PV$UWg-r#;5T-b*xh;K)Rb5;EH7$>A!H0eFsp2ens4I zkk{7wFJY)WZ0$Dl?vm+aT`L~vh^YNSfzTz^Ls^Az9zbN>jaHhLEI+Fv=>vM0mkc9l zO5$6&+C{4*vfY@tBVf*7OZkEUz|!&x90=^}dE6(4P{)#Mf5Qbf!^OKNQvlrg2ocs= zmL&!$-}`9nwRgK4g@PTTuUqdLiImU}F#8jZg59Mm{_d4`fIpJ6gvk?TVOfws#NH1k zGZ#qO@sHxI$h)eQc|x;ax$r@><9}ww!x4d_k_DGcWI@*BO?oPtA<-kl3wlWACqg=1 z{`H1Vl^z`Oe1QeMsO=OFl@Owr0t5u)C`Z{0AXI#h=cAs(S3Ge-=Ys5K8p|r3XjI)p zh+uZB-EzyDV;WpPD*WkiK}`Zq2dE>2*jPYp=-ogfw@i40EVbP23#_up1yCKn@t@m= zs5~*x$hAo^a1=JFw3O7FINJ{5&bjxf85CT$fXghbK&)33S;Fs42s%AI{d2v1XY#YC z=+>U8_e;TJjtL5TSg`)9SN)QQO*u4`+GN?A3Hl~6hE?zJob5Na>YcYn6I)k6 z-;XPeLhIdx$*Nd(5f;6(BvI?^EyHag+TVnPKjLya-d%>XiJMIRgyTZM*{t$wHg9W9 zaF94AWH0#lqb84D|HobP?F#_(p5OpUKh&`lp#W&I;5ei?j4ApWAy2;EEyRsY-T1C z89D9+*!n37OEUsnnsR}M9ysc@dJpEIa32W!vxRQQiJx|mNkoT zL;*3U)`x`*sV`51WQaNm0VK|!AzvqJy0_6&xT73>N*FidG8SYc+g|!(0R8FkJuvt$ z>NN4Wb*(M`E`v~qVRuyXxSR>JJz2%ze}9h;7rJQyKxnra7$6Gm0B_>rr#q#>*Ua?9 z&4{#t{a}3Dg&$;0d|LL}XJ>7@$XDmz)5`3VswQi58y%DdN|7$?C{V z!eHr%+Q#^i_!&WUo;S%fJvXFiYWlMO%x{xF5x@+90w2zyvI_0Yeu6*;LVI$~iz5dp z1I)j3vzIEYm6ftBP25F~zHZsV`33k7h(d|=gdzhpOaUw;zXjecRlE%jGV?P8=)>;i`4~_4Ez>_W+^s3BH=0(lRE&=Z>{H?#BVHDK64WI&d()iCu35(aM*kPy2 zEh9bFK)!YU9%d`lnqT}JN9SdD_nUEe1n-eX6mZIWS|$f+6Oj-mW+BP9_ZTJ|Rg}@; zho4SJRjii3gY|Kp6*`{O`(K~fLx|&jy zf>FQ)DteCWYVapW6A**sT=1zPL--KF%&Kjy*sZHzF?8ev-UiqWEJ2iY#tg&oNg$~` zt*ie9fF8x`SU5B^^qNYH72y(4t;EF_oV{~BRoPET@^Pm=^ z8+k;4C<$Ovi#QC#ziDwrG)Awnb7fqyw>~_f}8IDk_&0Glm`G?2KxJ-F2n|1 z)ejZT#=XcHy0|BUux_8tMtm28llWSW3=ZC#@%^+whbv8bu0V}7O=#W;VaoJB?`%D~ z!DtujhE8OX(ZN5NA>dE9g}V7{tgH(2!Stlhp+dlx_Nbf|L=X7^AvOVA{*uQe;pp|yFF7uapX%62H?wRgk~OY8glpwON?dIwr5`B&#ks+T1EBL zV8oP+cgC6EnPj_N4f6ss8EODo-GR~!O_a!=jQ?yAnE!+`VI%I5JK>uUKYvjk1`5(w zK(KegO;XT4shNH?-Y{XKqPhPO;vD+p~FxT@@$*SFkvJZP<8F zxPVQbkI%Rc<(nbPR77Z`IT001<~RK0>zu*v1s5UcZCA)L>%Zw`FapR6f@Fm$@MR2u zF1Vzfa zmkWZ6ue22SFJk{cIXdiMm?(|8MUJ1q24mW?#Y%pCK{Q=5P=B zzvp*CVEVW>L^W%t_Du+1OzH{g3m*#pZ|?zNQd;f6XvQfuE(oMps>9>Ixr6Zy=J{VB zTJlkXzLHTdIsbisU+Cac+gSfU%Ypsh<=5KiG1ng z>o4U0Wpa{|5hKq^P&|0B_Rv2jhP>Ts<8CT!&*%8e!E<`vYr)I@He-BK<6Pvl#(N~m zqXDNQ^KdpZ_*&e2uQYOLC;fDOn(er@OkK6H&9a^DXra>)J)B1LvUO@u; zr><{Kt)dVk^m}SPjdV)rnNkJ;)1%{jdvv6XCiL3~WJnxc{Q01iVPIir0)##Z6n#-5 zHg~xD)aTk_b17qYcjn1-eNk%d<*0Q8Xaag9GIHS;Cz@LIYs|>>GrU9B^eU`G#wA(C zVEORdp>_DxW3*R=p=s&Sd#ZyCI-l;2+u|hj3PSVFaQ>aQsL1w#bj$GDmKAHJ!wqys z#{N@&&uy!G+io3Ghg;{?^Xf=L&V@rO_0vOz8N&an1?bUeIt{)bwSP$M8=n9i;|I=Uvh`bhT-JUmKueW38s#oJ=&g~d6>j+ULCn(UtjRq3|Y zSG{u=`;w}gzBgg?CH+k9{0aYJjWzFTueEp2ve5aJWaJG^l7`Em!5p$Ms^_;u##u#1 zw-LCztXXr(v?a z%P!r2j5S^64GuQdTE&OHw9Hi14EY-5;|TG4rIg*I+c%)_d^tU|-wBpsixFC{v`nxZ2-%WVwI697DW`6r)^CzqO zzFVMOZyVmTmEERYTc3+cM#D`epXEo8{i<}k_W35H^`g7zT_oOS#Z2L?q_073@vYdD z)3)W4jEz2Ejfphx>TibO)d~}deReNHpPmL2iDtSKtA;9^YcaU6#u?C_e`{~H%A<4E z*s^tocjK(DQ@3MLYIJyxsMz8FwSQx1cLR5Y#)ihUgnGu*VM9m1x5Kr|;$%mQ zd8JH;&DsTT!&-7DqYQE@!b14fb@QURW#)$k_3Sg-y|a~rWsQ+a{QRusgA{A-x{TB&P&Q}*ZZW4SKGHQ&ur7WzcvbO4nOo{@Yisyve=t&SejcIe!jQt zWVL(H?Js)$=jkh{2+BGqIrWIJu(;7ttr{lXJ(1uMe0+R1UfvfyF|JKU`?#rOyRK&y zA4-L%k@XT4uOVG~=evi8)0;s70SiyW=MRH-9cA3{rdR4e)ZR)o@)^t{KgmEkywl_M z#TQ4=##{^dGLQJryq(--+*f-UhNqnDZx^qOIU9R!U-(cP@BOmaTkw#%zOj6>mR?$Q zmX(i2S^8+~mQQAs-0oTeyEW1`@bj-{H}xFc%nlB1?_TfYt#&LfSnn+%_6{V*?XNE! zzvb+$E-YSjx$hqbO-7cSyNR|MTrXQMIlg|PE;;Iu*X(9je0rSkGjy~0kv_%Ayr(3-Wklc@9pa>YK9X7|R31sOr|rGNmcolH;nz&%MUGL#2?R{DUbmht=t%&DYn5 z-e>w#M<2XOy)r+T(xDt|@4nsUHVXI6WE#pW?C&m=6rG*^UYNITwe6X^GA_cH_TJM& zTDsF;QRQ>L#u3Y($qOXS3pxtvSwO3^P5*({*n2d zZKmepkbX;P()+aeO`PkCauch?IHP&<8moLavF1CEIVOYs@#@K>tII0Wj+&{<5sB8B zO+3wODu?~@`O5L8_aF0J8LIyXxHVQCs=e@;*9L_xJZHn3&2GoAUia$?t!FLhD#793DEFxC_K~jx@WH zQ?0;1cQk9M8i$0P21e}iOusBmG8U-kcYK3rXgk;1ebMXcyNf*?*jLYQW~U2HmhI#z zQpF8{J;4;zaBRwVk#OTZzAo4dBQ$++VFAC)2%scSPnnhLv78HUMWp%T@)A27jMn_Mt(Ffm^Y&VjoWCg&7nhKLf{YEUEE^32 zW65Fr;dbUI%Z?scpI3h&`VC2L^C#4J@NhYfbMwn%DMsZv3M>{uZn6uJKp^0Qrp_8J zPgxGCjbw6dK`Ta?Bj$yCL;x+JFog}Z`B2Eo-G1<$({s9a1T zhG$*(YM#1fgfFvUD}rGJFlaZ3<3~ked%N8B&d%R58$x2@V`!@E{l&K>Pg;8-ghi*^ zpUZvkVLw!o3-fzux^THNOm*J(XJyz;D>+9{$vhHHA1F@bvIgRy^^ryT7BgWMXITOp zQyuhse!fVY_CU!~kU4y1b(Q_;)2H9z3W|yW&~J!VS29jzytDvzicYo=6|y)mhujOl z8uT+X-4cIGGEnKt7ixyfP@8Gv#o43RYeClNI$VoQrIC3nqVJRu8-_z5a zNF;Kn5MiG~j)99?B<8pGT34^ZbyA--5%p-c&ynHb6W3?w=L02YZZx#CQwIbTG11Xv zp^=eOBNi5@J1jY=>}=!ZqZ(Nl*26Zeb9!p}_|a&xTZVJuPwmGNjSXxp5Myw0-4}ev zo&DW$TP=_#-udp%&eWH{;9zd_)-*DPJoGDO3`s&Y^jUCRxa9kRvxTbT)8l>;@AMF*sHXLmz!PC^9YYam` zJi!JRNlHlQIRferBNon1yO-oeF2BGTEhSo9*5_PjYY)G_>VUrN?C$R0^P|x*(iaG< zBF=4Sc#apmlKq)S@#6K|yyV*8tP7?(gqyps&pD^zImP|^Y*KuaMZn;)k#dvixhald z(xl!&n#Xf0)e zzNML&)_bbK_^mB2C86GBG)$}`PSD*W>_WV?O_nUTdEpT+In{Y5(21EpO;bFRRzBny zYZGS}mii9J>!~B$dvU@q{tQ{S{ zCl~p~w8ySQSFK0q<~Wp8BUS9}BR#0;Wj?>IeEh`nU_m}KDk32x!-PbI@&P@Y(3sHG z;?j~d4LyBy3mOZ!vDS#e(@_=8 zfLLWD7JkvXBfS%Sj2UP9@IS#2kI|s2EXa3;NwBBG-$MM79;5Uwu$Tn^*WRHL$t0r5 zBHv*Lq8uysV(F!ISm>t85-?>$dQV}1QMV#Ykqlo9-f`HE%AKzPZL3*z%FSSYeZ3F8 z%xujWrcX6BcNEe9$jtNv@tyUtr7`ABH33x%PrrqN(!{v@5mjfM*jRKZ;gQZOZwYk;RhiFL&nKLLTR$e~X+S!XO4-qtMGV{GhO7d4Bc^OSj%cwIF z``Lx)bguat)Sm{Q<;hMH|6=mm$SBKH+0!^}P#}XdUnBFSoH-^(jaf>H`pG3`{jxW4 z=oatLsHpKXxMWUVsm8Ofm45~*-G)?Co;1n^1uNzL>8GTGynm6&FElp1#PoI>LMSt8J_7SXUXapDu-LvTi7uKheS}}PF|qw5HYFu11M8)Jsj9qZ z+F-ga*AO2c#z&O$a&yCj?m4eFZY3ZP;KITpeQm&bhPad|9=@lQhj+$?HlPn=j_9AX z-O(0mty5Nbl`dtdfL!W;M5W%I*KUjHNgA0;K~4=~GQ`DjuyJF7zOY;RG3m})Qy?q|pojW8j5j9`$#fJIbC#M=hUp%)L1tDwwGQMTK#!cVSLWTngV%HV9@{3#Z$cyns)WaFpHId^A*4 zrW8jB+N+(r9@l%X1_jahS+&;_|Dfe3%!;8+{q}TXOxH+M&Db+7AUIG(`9ndFKxI;Q zH^=PhkV#5Lra%hK_Mge&H{YHxKVen40^;YEqVtIGYwZ_*lRa0X5{ONl5-FKqjVg)j z%vBWEaW?Ld$YK<(ajo30IF0=#@e-4v)Alu6Y7%!Y%$DDveAqw1&8=f(TDA`ZcGcKU zuUC-lvT+M(yB*GQKGNq+_+?bPh8lYt8<)T?Bvg8as}lVSh^?gXHpjEZrIv_=peEq# z^0l&JOb844GnTIc8lFzy2m~~$b?3PE;{&+=+S7sCfoCT2`rfkDg>;8k_O%yFAkZ{L z$V3S@M7E7Q*%7WV%RbbB>dcFXO-%fmE$8SMGBG_paL|1HZu0B8^M1E0%@$L1Z-_E? z<0q9o&5`NRXK&uT!Q{Z9NbFNG0umCZ7kMOyL2tgCsV-Gp@BBbbmP%2OGv?*`Xla#y za&@;Z$lv+)T18b=PryHM017oEZCGQG5)|jx({a)SA*q^JTr63E z$Fhf2dp(EBUi_HFtHMZYE~{M;RD==~pV+pPM`OOT%5i?%4P!Jte~s97+ZebHM1`f2a2*v~ehmN$oRU#A9qOE>?oYYd~QclEq%p{M6N!38w z#sN!s`NL_t7EHL_Z@iiV0t0#n4WHyTH!D2;JwB1s_J$xkG4a)GvwNm@My9B`0=w(! z$p?xpWV>9w$8FHy;7HQ$|X|_QBbj1l|cpj~VZLawot%+^^ z^U4RSGspTp#k1zjfHk$Pt>JM&F8afd{`MGPN46)Ggtgs8LnG0sEO>ZWUsx10Fz@O!q4bgAEd zU;qZ08xIG^>tjW*J|2Z#?XQ}?tPZ6g-_7zkTF*EQ#YCy&T>RzyLUHdCa!|ZrOI8+) zF=J_0^y>=*oHs5U{w9Tv1|2GS%6LpHMLB9E!4=vH^_HSBQmrXDHKx`pE;WDUuv>aWh9V*}L$=lhac*EB$;TPz*Www|E-Y-Gl9z ztpcu7qBgHZ@+T|d|*+C?*fU}t=E;;jWpzG?h;XjkkHOEUn^PJ zg5n02B!2z=*k{CSA|kIcLr0YheB7fR8B?HO1-|qkGLh&;tn-kFFbrT|yrmxr2pOB4 z{Q<|>3B2Aqm6IzSX=o6Rsn%^wyXsm-o5ij)@&#QSJxdg*^o&^#<2AKps-ZIcl?3Mq zR3}P(Aj17YdnUJho8J%jybCnlk_`7RyT@IIk?_F?f(up%VlFb0R6TER`>9m#N3x2H zPM~MxHBx%%A&)W8f+++0E#J>S4dm1}&6%y|zK%}y**ZUMZ7jR~{D@7CMi_wnlM3_m ztuBo~)#JLY7xPRlXLMza`rlDK2cb+yIn4`_YA!9#53l}0(MV7vh&90FA1Dg@3(iJ> zZ^bZB)bqiH3hLtn#+!HOe~M|a6x#HWSko^tiS(n@o_Cmfz1WP5g9_4BW|H zPJ@>QVvM|m!8>aE;77BH2y&%1Ov&D~af_#+^yS)plI!DDraObm%E|-cdRYP4^Oq@3 z=*PX@t65|OWno7J_TR7l&%cO?P%ogQjQLCVKxe_O?-3X@9ou;?l%|x}n5vFn&@#nK zcnuA^#`((#2~NF+!$8n`TTfo?yKI`X+2-J<*P;|D7PL;7Pev4Z5NUV;+> z!pa+d){E4-*jRi#l@A|6x`{v_&~!i%$o3-gD61E$|2NbG!s1Y{%YvUjRi$rG3Y1Ka z464tc2i*)U-wY|x*z`>Y3^W~>>>|4}>3r;O>W?zL}C?9Un0(ceMm~b)B0n@cKwC%XoOFpz~hG{D>y;-3GrT6%| z`kjk7J-}#-XX1DNQF!;$g>Pn@+>4X+7dmF1O{w0e?;8ADO|m z{kj2QPibspWtIQQ-F-L92vvGUQGinneTW+#5<-AEWObycd=KhUNg|vpij9GI;jY~$ zo_&48i6~S#;T+Ga>v?%V+zzIkjE$LDt`DkJ?lPg|1RiJc>g{RvzJEB*7n8q z6&u~(=xptan;nXADxBDdJ5xFzN{Q|@k{)!_fmWr%uLb{?;BoCAS|mK>end0p)*7~eP)S_7?h53Q&im z{=e{SYikRrM=brm5{%&M!Eu(v%*6BZ&xgq1O!P^9Y+Rg+DH*N}_BwtNow2=ReS3Nc zoiP#MoS)g4O4&HJj?Jmhi{adLVcN;G$BRMXcLXQafNRihxL5l2)>d?4;=@zvHl#Nv zdE1^YvlgN!Ko#2ulwrae7%o8sjV$-wJu;@p*3i_P##vvrK9{>b+h@9rdj^HA#H1%= zMUWD@25QO!(lJz>W{*j$vc6_Nalyu6x?9duSxqg0FIBX^Nj&gpa`_7rTn~>g>z#e$ zl`hkwzuk5dJG;92gIsMbC%?Y?Ib1=8<@1kAODAiU8P{hM=3xY{cRLe>-;C!{QHWvT zh`#{>5r%8Z*6pIuMMc=6?3j*}=)g?m-ZV}3u)7CUcAL6yDz+$|eTx9fs~z0}_|sZn zZ%DS-5vtpDH8wxZ**g+$d&F?_(c38oGj`!QeD-)#*yu@i=nG6|)yxcaXn6Rid(6a| zVGRuuLpfiKL96vXV_IrH9`_kR!kDTzReP@$rxbBk(`UGke1n@nAu_Io**c_*5-jRN zLysX47R>2a`Tms1u+ig($Xp?&({KPK*!h+0W0MRwR(NgTBCd;WQ^PoJYRmG-;yfK{+P@k!q^BFhfx>Y0Qi1uXch+5nK5i;*Z%jRsJd{CAgB) zcz=ER1qj6bj;?-3u?H_Lf^AwA>?}ls?8C54nK=cx9>FIAM zXlYrrV`6x?EUn(DSXjV6=-AluwUtaxOceCs;1Y2nnaNi5r5Mo{8}k@9&sTh->RbuA z9@KqY>v;A?Xj+}!QT?rMoM+83ow!ey9+?Nwp61yx(YyI6zS!7UPkk@#zS0Bx?fw0@ zgHn4uLgvqy_Mg))V`JJ^^zLgyA;^OR##?Fh^5Uv$wXZQYndFJ|*cjtW`c-FacvF7< zx!YbbGy7mpHcJhuRzh?w+$0!{l@tu@#3I1KC=^$aMl&HXF*b~_>bL6zA`m%ps<>hx zV5kG`kTkbVZhYt>F<})XTaaNFSNW}@OG$iBwVEmj@Up0?NbRj-I}~R0Nv|3M4%O;# z?6+Xr3uEuQ%8Xir-M(}zrnh7DoJEofuO0uOS|3H`{QQ;0H;~AW!AG0Bx2*7v(^-WM zg`Gz@#jwCev3IC!Il8e|3o>kC20&WJ=Z)G{NnH1TGNJTj;~M!@a5lbJLYwjnpBEfu31mZfoC?B=eq8MfFJcJ_2}iyArK z1O(YV?ki+vfwQx-KYPEnX}d;-%6m0DPXIzyQ)@;6T^St%Uwc?htYYC_xQy7v!Qr#2 zc8!m~a1cEt+XTc=dtYZTMM^Eh%VWL(1xs~5KT&pdjYt!BH`&?QO?Zm0!bCyOGA5)Q z8Yi@LI0T;+4;Fo-r3++)oVGZ_XKCXAp!GTe=azG*qK!tKJtXV>Gb- zH+?1Q@bIft66dUwY~x1n4tBu8nV{?fFWKexcH!gJ4;dUv&lxa1N1@di=8UlPu{BDi z7!Sm&vYh?hrRry-=Ast%m)t&mDq?NGQ=l3p^UEZG-Sd3+&2j-T@lN=!;S6EjQsZWx zA5;TBa;nsvGE5c(yw1eJRiPGXE8IYBCITPRmPg?|4JSagFFG> z7X`oXivC<*xYWUD9&wERUz9^SJD)!~XyX-d_!hg~0=-vV{i~nK^dPn&_4M^-8k-Qq@3Q%TC-0&*j?bjtW9VHNj{jrG1{fsLt=2x6@^u%ZlqG*y zRHB6IBilnjTK>qF@6;9dS91;;!s*!UrU>W`a-MO44 z3juk^E8J2sAtCL&%}o^zjbDn-7s#24;r_ZUOR~kAIdK2URc3B(?F=8cMvR#m(IHA< zZOc{et?~4dS)6Thi2`q$v_wAQ&Sn(kPsovF2=K+_u=ikam-f}wbwY?pfW2e+z!S5b zc*L%N#Ud+)z_GE=->hRk#QX63LOhmrgpxV)wH_fz|u=2>=_{;vB>*c^Tlg%wN zgqMpywnWp8!pxFS)Ip?|&c|EH#>SSNi_5^-rD}aL$uz!tqR_c2OMB^RppmDoarv3p zf7JqfZ$a4roQRmS_g`J_2Y_QLEsx|4Tit4^fBou1#;!WA_rKiUah{xgaR0sPlPFI1 zem}gfO3yx)p&s)&TpjxCuCceZPKUd#kJiGEdX>{{90ew>pv01aZcUe)t-TR80c+0h zbZQ@-a4LqwXYqI>j2lh(yu6xsQvA@*7zx-^D;^T?GI?F4V3@(StcvQ3*!c`rQQ~m5r0Z z+TBj*8D&TI=khKnVavxb3@AI0;hHX6jvg-k@{#^uLo!xaaB%qO2b zkWu{Wmqz`YwJE$P;&x_HP9=;0ekwt~hu>pYXNJnXU}UL*rM5Wbyv8Gb;MncR-{x!U z5^^p_2W z4~G1ib0)FepTx>00Tpt|nWzxQ?OoSE5H^?#ygFU{x)kZ*xWmxqUE>n zli8shTb?e%nqADX7=!+4u zxn(Yv+^gSQ9MJio!;D)^N=)UN_>$@#=e8;+0xF~(4(MHdXMIr7(p0DLW;0^~rNBe# zg%|1hFH~MtSR7>sem!JZ02S2|l>2h!)&&P!*X5NC+c%S*t7)pe)HdtO#xuFEASznh z+aoDW6~enEYGxH1)(Dq*X5E`Li}!?GjLm?QNt^=}E4#eDx!$ifBJ*f_+d2n+zs1PV z_1V%LO4pS^-pIwJYVF5n=J?ftq@!{qy;L3?N4f(+O+z!~!2+ScWlUnIi-$nJ$@0QT z)B9q-jFOmdu(%`y_*1JXlS@fYoihf zWJ~g1<#UPkQfwv_u{YoxKhCeG&5AGG>htVebg;9TS|Ck$sb3>h$06hJEkh58jKufd zyE~~8$ld7|zGZ9zlgV2xEy%Nb#_S?Jtqap6Tr@PxS0z7>JFi_@{Zf1XKFIHSO7gH! z9_O*BIJZj1Op|s80@f?SIir~-QF3&4Btr#)cQate>Q~_zi9BkDb*UmPV*S)?~k{tm0|xpo@XtMUsrRpz6gph59(1kY7(#X zntw%d>ym_Wtdc{YY-Iju$Qx#dSX%X;n|wA@pp}DOjHfy*pFME=Oo*T*TNR&8!Qg?? z0C8m-$l=UU94!S=Hm-CBAsUlaVV{$`%9Qev>mXyEqAHa$$WAhaYD56xKu4wJI!Ddu zuCRv%=R}=YA!1bgiQN47?jhJCklXk*%XgpYX9O*vZvof8E1gfD1GYihmfI`OFv+HK zH*_f1>DmP=o>t7*Nvw4vB7-sL{kr>_$l;=J$#Zi_sPPz9udfMY$Hs>wE8mpcFV39qp%VF zO7eqSdI;r7jb~VoEY!Lz4|_?T?=RHqf{BX-EvOj51hC_0A?$n2EB7g$%9GVO)Azn2 zEy+>1d_(UROy=7$BSr>lVE`VeU$8rNQmLv5*9z7dpmM8-iB1I1{V5oq>J0r4!rnS8 zsz1C-a*IF}8XR0%eTE+ELr&0FB!SGFFgU1(Uf_H2;cIL4B zZNl2I;gbYI4M12$t`sqHhC08{vz@PP~MsYfo#DZ5t=k8pjd%aWZxtB zZf{(<-H>A2pTF`g``0pF50a%sM$XM1GLdF({WaT2q^YI9r<74g_IM!wK~9sLT< z&e3I!VEKD8GO+wNp6NPdykC&lyV4`k3*_r^*so^X^>LzP|^6Uq@StJ=FG<#5Yan) z7~a7x5v5ScPiH>7LX&t~ahJ2T!6mT*wXleHFu6+uA*<5SK~tPwluuS7oLf`L?`5r` z-v{YbX_XFhw$yMB`SxSS^)P-gk@k7M2)w(@?;ys1{xL z>7nOu^%!P(k1vrL<{7AJn-T%D3gdJ$Q!xX&zs4>%WglezxO@@ovXh1tWT%;E4v#hv z>0r<^n?6bNe}N&jIpmF_u$PI1Hn;8&LWMrJ7nYV1~yJw>5!R>#sqFb!LjS`<_%^`!tiEL5( zJIPdBVG7uAoa)7RnME!FG*11S$i!=YL523FuG^^1%CkVf)2`J6yM`!J|Kso>0GcDJ zWluY0r4Y!Z#>;NA+;%(^B6#PQk+~GY0E01lE*tKy#_hp58vJm?HYJ_pp|twCx&Nno zau?^C4vvn7i|H~ox|vh;Chy2XiCK5wfWck1_Oh^?V#Ou0=~k)qvZ*I)@hg} z;ec#|h3Dtb+exzuIXN(V+R7UPw_SpFPoIn)cAJIHXQF}(m24DVckHv8!RBpF?Q6M{ z`j%7qmU4CEMmo9eIXp!hPtHH{891cn^BXOP!{moVG)<#l-&qEB4PsB7)~D7J5@({( zM9M4LdY!*s#iNF7f31;bDQcBb1HKx=NI)C$-EqTsyWY; zZ^Trn36B!Z;+-H`k3r2)RM9rrFcIn=10<&T%19H2hzKMl$+yU$Lmq=OU0vM?d()B! zi3U(O5~&PWkbJ#R#g6topudcEhGhP9@!rtW)injjoeD8AvD-7*uLjG~7s1(BN5}OS zb}F8aGhRU*)}%`|NnjBJNV=@uleRurEQHOvqkc@^KvahSVnnszD=JFPome}`0HoMX zw1GwfP3dWp>co#DD8kTK2>Wqa@Qy`HOu{hreveKoQ;w#*Qjoi+oV2vQlEi$3Je4iB zF**z6MlcG+c5r}fGP34cze4h)Jz;5_jE~iGwIoX}Q;+3`fZ zP$JasDk{4D$i?g+a~v{1p9s`2!4Z}h3I3?L%N-0_j*MEI#E%`BgSJM4>w?caHGW)t z4?biEIW+lv=QH{27+JkuN`BIdTk#*1V?c~g)52Lye@-ox)Xd%w#S4i>mS@#sLl^v>%!X(`u0^_wZ=CyC5$v&siEH|#XSu(WPBCY!y#@2Rb6QeKdPRT z#R~fVVBa%UR&xlZ=1d{8bH;=Dc0lf+aucH!eMvnOJF z{_SD+0ZNvIf)ez)h7E;%rlydqFvU>@iCxD(!CgXkAqybJLgtgN}%l}$aN(f5jt z3n;OfqiG_3fbU6FkW0Bnm?!E9rJ?_Rzk=bEGRX>*?YqNrz6qTTcqDoi5mbwG11nx4 zqv%isNc+HJ8DYmKt&0!@O!8$STxUNncG@2``CN0cVK!)4MMFn(L{}5=TN2D$SR9ED z6&ScP>LqM2x~o9O*?n2`_m<(_-#)=NkLJT3cJ-Tqg?_h4U;`{JSf)8RT;(~`W%T-d zmev^M-3~d30b;*NXpSh)8Oc$;Ko z_5S<3-jBj9S(6GFwE1T}vu9qB@H>y~ojr-9?g>=m0$S;ksqw2D-L4esJU8pZ1*0^S zF%g|TJ%FgpB)y!#C2SQx?g_rA?>)D&OY7SNk7t|*M_6nYn{o8hlJ1KlmP}Bpv*Rhc z9;WM0j3AX6XSKTX(DSzI{jc=`d<>mtbq9WE%Wam&rYZuB!;p2N<0{PkZY~ZlalPux z^d6I}`R}U@`S?EO)fCXLo^XkC8$SKJU@b;*xQLa!EKByu$q`M5?Tv?V;OPVcx(+`H z<>S}#Eyj`2YinyB#~58i$kintN10g!G~*t$&&(LB+-k!ojm`7+Ly`~z$h+i|N|NU#yHFHk0gEKuuf{W(Lyy_B-PR40ut>-Tr*fJB`!)uclTJu)p&#OK%#4sj1m=ZlZ5* z*WTBfj@`Uo-Kn#ntk3S7iM_6B*tye6C|_f9Unwk_SW&vI;~~wl zG3i!~3M9Idpsa)rGPc{8ZqJ5s*cC+G-7p(QRtW{n2f)kNUO zSuIuoZ`!U*A}ExbA+sskB;4Pq7}pfiJNoZ7-WQu^3Rx zib4zz4YLB&`^N4%_UuZd?d$U$i9YIMV@@PQ z57hzphTEh(%XrGIDsHB1Tt%E?pskyHUA51@vvuNZnqod;yP=hmfHpe;O~_vST#6vn zoI*VUyhjfd$%As6v-PH?TyGll!ttt|rwZ#hj{-ki^=HVG*}dl$40dh!mMNy$-o7Ol znsCG}I@+)u!fRA&IZ=(ge{$S%R^iW&Ol<$TJ>9npS~z>XsZ&o}Wgr|9xacBllGqP5 zA%{Vz9f>oEh(aSEo>$ScArIv55U>SOAf))96bL;^h$XX`xO{0$buzlsR78r6hp^Xg z4$qT?EjY`$DZ6$;X3@3xXrA&=RK%tdD>D^`v6}IuLSrO7*1lOFVCm-m%XPJujD>!^+ctuO)ugV8S$e%unfSt1tm+pQ8GFW`l=yS$b^Vv2)uQo$AvV!e9cZLmq zV+9wU;uy;%a1??rC6004hllz`vq*s8|CJ)?hAS$4=()S}>tYp_e93F^QE{rb_jjU0 zgBa&436y!eeL>X2{S}vXYQuh}kjzQn7BIU|XXHNND!CHFze3!9)DM}X)l|6urGi7( z+l|KwB20mH5&A%7I_JPlZTY7_3Rn7eB~{h#TJBWUUtw@Z;s`PAZ*Q3K2V_5Mt;~7( zO*YxM>~JM55>-7tUvl`K5|6@9e{Ow9!j(1)CN}qb5OaRuWyHWj-QBaM5Go>#Tu+t& zMFc%mZ1m@ycHB+^gj`&HY>KU#0G*}jqT>0MJ&N6s)%U!|$N$mb5X{!(^@kmxHXkzN zt@zGhEQZ@;8SMbQJ#;eS&vxV4{rW}NN&iVkvs!aRW1~ZQ8bcTXl`nFXO7Ui0iil2X zswGDCz4q(d{!H)_0IiAX;O{YJu>es8W9dx}S&O;dodSs~0f{1~LmA1#L}n_YE?W{2 z8Q*4u=Stzl(UTx{gk8f}1LPunz;|orqsg~WVRb{!B5hpaM3Nq$2tL>*GK=)7?O?H_ zwWlWX8l`WiTw4=*mi!B=Rha*_p+#>VIwQb`viJN0ct%r3K|$$6Oq+`8{Fr#V$B*nw zBUV-nX|>^BelOF%dq9s|WU)pe5?JIvwBspb3HDd}UTGydJ-hu;5>j!)MfK-U^of83 zk7o1nm#QvjrHx3btYHw-uV>UwbyuWRJNiYY0`gv-y=S4`_%R|{Dfr|j^%|)h+$BZ`*oAs zt?_IUQPT{bwvw;n*d7O50Njk7hzAV(vYNlysieEPgNG-8gO^+^IVqovR?9S&~gHhF&A5G z9vYsVEocXF|0^e?Fh)n>zI;IhY(Tb2YlVirI6hz=To8NAD$O2E!7?_oez#J{M1YCa zjAzGw^7FQKV6fd(_swH~qN?iy2~8pzP6O56_eyFBchv%-Gs)gb)aVkM;V5P|WV2#! zC6)VcO-o%%|A1FL@6MY0M(52RpO3LA@wKkI3j`H5n~XpjCv~9tWUU$w&$A>mdxtT9 z+@F?R%3TZX^~*4x^_@$)M!8)uB?gXr6u7|7whDhYQr)HuT8^r=zfn+wVG^5LFW9#W}ATH5A$)+vf;|_ql)qf4) z9W@t*B1G=7dDcceBlA1Vr{@%s)AkAZtkWUZrfYGd-^fAnG5?lDYg45G09W55pd!kg z^_C6(FLgMjBDguXF89_g7#&G-M!vlKYS%F}#%T7<^xjs?vHZEQ3N!p1xg9-c0_4~r zUx&NmM3w2Na?`>t&mUhYNO>NlanGp!WII9~E-3&N?wtG>Tho!sWY(Yhs&77*D>v=i8(Zg3+JNtturky>XVbJo9cZ4zAOIr-^g`1+9vwa8v&IsEG&H=tNs{ay$_uzN zqAiZv#5uXJI`xf@Z2$G9Fk3JnYT)VVz+KDvOwE!P*%V10_Z!GD5lA5bC3n3i3iq3B z2S9^-FjC9Q&c6GM)x^JpIWzAIew&EK=#|AR8 zwI@&tmRITN(zIasY{>eno=2|p&KDa^FnK*25JAE8M2RKWsow}EpR|6TrC0w@3bki; zYFW%ad0^yVaQU7aRnw@z*NMlgXHkwuy5?+#1eol$bUqrIEan{ zy`6_d?%U&e3a9C65_RAwLY)e^Dk(RAR6*O@ZYnqnq2%R-jSS1oODEXbJfA*|BM}o3 zFSJ0mQ}(ZZ5PZYM&8@nA`saz2eJvLaCI$^pw-C91W^e7T+eL%p+;R7q&*U`A#F($p zf8W0;0M|ZwCq+7MQ-hQzT#K1TtdAFu!l zCfhi;h@pU0&J2GmET)vrfCc5HT;zo9E3dZwx)Sf7IQ7@x59s^h7I44T+2imF?QIVE z0a~aauX=|-o6O?IsXlEq(p2GI=m|)`7(oVsBSaFYsHlGLT8C#aJ$6m2vssDqLrHw2 zaou`rE4l4Cv)uUAFLhrilI%aFSpNtvYK0~z2slv4o{pVf9MG4<@YK|Obng9X|4joO z9H7?$5biUCqQp&DYlotGYv@UMGzGsf2|^S|tkVgaX9Ws3a^4kwr7`yMV`A z!Jh!Fmn?i<;i>+{uLidz9k-_3wHUEcs=XVMk^JB~|Fr{?Q^!?T2bYF?HWwsc|cg@9*>K%Y1CBR8g-)Eu4G4{sE5D3Ww_*Q$GDoI~5< z2Neog8zFNq!8x-T)6tHVSFLQRzUU_f!7xk8okq0GoC#JuOBR0r*-p~)w;!^DlhF0q zpY$#d-t)hpz3b!i>U<}_Hvl!a?H&}0+x?L$^B|9*&sDzeZKw|TxV`_lQrnP&eDzQQc*okPS(m27xH&VGeN zVOJy2plr}N2gPq;lL~byd-v|SSYeC$;tUy#Rvrp1WWgLoxJ|6h0QCT%>L`4F?x4*9 zA@89Na9vk5#`N6izilDxqxy!1uX>h-L(?R}0As)uGHy5M|B$&a|6PzN$rFwnpU;3c zL?VipfsKn#Wg}r>^qq(N!t%n`X`SJR7!b3dR4LE!kgh0+5G#%+Qd?z`g@YH#b`a(L zQ7y(a<6Dd8a?!@+W1x#Q^4l(pl3a(b`SDXOu&O{X5n-3g{y}FGamdpx#SbYv-rnAK zNfx(PI4@ZsD1)EoqHd)CgjRol?{$v1=m0wKip45e5XcELd`Bh(%t4QF39$xZA3oKY z)Hl>uCH$y~ghrS`DAe7H5ipJ%*V{Xo)+a|09Vo+Oa&AwRJhG_v;~kWw{{8zvZNa08 zEKFwiA|pBL*dQ-gPl6^;)}__$;_ew8U};(EY}{f9K+{0(UJK$nrlmy|D|=T`qi462 zDX9VGMERXiBI!is}|mtAko@p^BS zd!F9@K7!?JM+VNn)D5mvq~=iy1l_{5gg+yA{p|iCToP_ctPxP62t|X30xsKCJ`@+} zx6{XDL`O%_TI1*>KT)7P99gs7(1A=j^xpW%iP};@e=|rJesdNopODR}1>;5ZUzaTa z+7O56`R(zreze+rCyMGe2luO8XV9mN)HnQh3Rb_qe!7~UP3cFkDw=&%%~N9^<`}{8 z7s!=9z%txGjNdgtL-rvvdYOdv|-9np(*@%dFatpRH!>mI1Ou66%wIdj~hV?A+ben{-9saDM00 zimoa8_ej>BX1wTS_@1?ww?hzR)LP`~JH3f)U8WV??WD!U-=C=0c0oF^V*IIJ0fDc- z8S=E2(jdv60CA_TPO1XPiB&t!K6qyz3yVjC9i5zv?%9ySS$6_E)%bbSZuV|k z_u%(^wNV3*13QrzAu3wU{2_z$jc{ohex+Yo?^oOCwzpDu3x1vAUzdTYe@jYZyQTCn zs*}vBR|EKu#}w`JwAZzOOk9;Jp1Fm&1&#GV%cME`Skofh3z<s2UMYP0)%J5hTlGJ;}qbFf^5{7E>WpL7ZN@OU81278ZzJp)zaPmA4v{&?mt4rhlC z6Cy!m84uNrIpE_y?X2DSo;Skqrd80YHNIZFS=re({?f8?Y?f3D3on+sWPgp(A1WL3 zc#$+GmP^(OXTF2K%PJUFNC_YGJr2TwYoYB|Srl-pL4~2ATYLh@XUS0;f7T2Mh@V`A zvM7_mhfjnQ&l%b{M9B({sZst6bEejhOYU$Rqo|ML6XPD_On>+;1kTk#pL8gJqMjuw z#|C8ZbEc5`+M#{9`QM6f*2+JdoQp* ze(iXX(0`w|F2m;iQc+m7BLn zoOrL+;w~uc$Gh#bJ+gH%-?7u`)F^ouac|W(V@h*Je*&#Mj`N!fIjFL-vUtSs$2Q|j zgT+OQXCUu3PRvU!e3wf>8dpF)8Ss)oYCd(|OMr4ZN7aU69{0}P@^#IQrE^pZC#n+5 z;gp|R)KTZKjWgV4vf2*aUI;iD;=Ea-A`x=dMepnvA3|gdG&QdiJ7MppztxP}Re2`- zFpzRFZnEQoy_>rKxOD8_l55gq;s5?{G?oOY%NyjeJAb)t^ZRFOX-*rjH$U;pbcKGtUT`P?;k&J;+?C2Z$jsfDYsbqa-*Z8MT=4RL`&SN zGgiPzfq7sq2lzu-nGuW_2$|x#z=)lam>A-8QzRg*da9bH^b=Pe4ulx1gNR`R$&LrEG~GRBlAb4~LiG zYI()yDcp4w6cqZY+2@T8748b3?D7igw76ZjevZWL+)G`0Xi^u1vz(;YQzoYXCBf! z7r^@e9{*KW^LZ-nqS{nkR8-)t#oP2={5SV;1b{F-op<@_$u_F>V^Kv-OA1r$!3@D4 zc8gdXyK#5Iouip20=c%n8CM1U&whPcpA2v-yi@%t2d9spPoz%V6rl^b?r`ws_}oy^ zoqDTDV~%@?dwU4Qd7N1>AbqycG=8?_TVh&u6ssY9GFRpOk)LbNb3ag#!%DsqLE5 z&*U&X1rD1Jmiq8wPotpH`{Tr8kZ^v;%93+AAhQ1WDrJGeS+t!^l3whZxf_`K545J% zJ2nb{KGV{i2$qiYtfx!sj8&mS@i!$tTD3GkH~1tjMd8W(<%+u}0ohOiPQ*k;Qc1ju!r*wOj4WfJQvzK|sCOCtT%^5_fNv0e6wzRHaf^`Tq_PtOoe!paq-dFE3S#xQ4%_P8b5et=p6m zwtQ&d&0>P4jcO2u>sj8f?40y>TPG5?_`TW9scLRR2 zYvKd?6au@@ib^yZHCkdep_n~*)V?wVxqDPV78Q7-6Cad(=s_MsvbR9y`14lj~K=q|G9qBdd{hDR|;3)$!Hnid*#Nd#1rkfL4Wf&%s(NFE#v zImbV_>8;Ia^q!Bf0^*K8S2#;LNSpN$SiQ*$99?R(GIAt5h$H2q{P7aAIE=bAvw>DB za1-=Rv9ZVa8bUpH@A8ce;ZOFCu1%C}-+b}Vg`|@PcN2ae`5t{QG!618mIX&FN$)Nc zqIH>mE_Uj)H?dUKXZ5_r@HIW%=FJ5*9-bax=@%ErKt<%nFub&tAKpF!n`8+Hmiz5L z&h6DYT#IB8kXwxgy$Fq~C!uFIQ$fz&9W&gzK_pDWW5>vC4RSORG5hZLQT50(KK?py9V_X&n}0ouZ|qBSC08 z6S)(A9O_RYXjEz9COf-#i;GoU<8~5+t`|hQk1h8k@dw6A^s>W#KN!z0c7B^j$Tc(d4bPcXfd~ zv9wuvkv}lu;MMEbe;h1&OUCjvsITUI)rk-rzz`wXU!<#S`J^z(z(whdXs=tvy{^-> zpYY!E8y4>@hBF^}5hXlg^&Gz6g!;|7p<{Yus=Y4k$XzJdaiY}XHv*I7mHyzPq2GPr z&D#3lcz5Ive)v}iZ5u>|7S`w#M=$JiJLk9R)k>4Y&*~CtT6P4DI&YFLG8?e!$|@wB zm5L>vWx6B#WBc)H=u$t;{Z}i3lq>AtRMO%^gv-$nGK_p9F(5N-vrJc@|GjTRmGh5b z(<)~@=P$cE!Q2aM+C;o>ecHkZj(`zaKf&iC+A`mx$eC7JgUS_XfT2g7&1YLOXTWqY z<|Wt*SQ5oX$e;pL$^jg@+fFJx`R(#lz4v(TJ zSRBKlu%jb%RC#h(RFDJHPoVj?gRK7n?}LK4=Y8IcCe}3qx(2(T-E#XOb{<-G7r5`) z#%CS21%qj30j$`ABF5QO#!|}l4#?5Z>7g# zuy+PfBYj}$PTAvWTic$5sst99sqYfA&cOVr0~&Q_?0Yyx%`cxfWy34}Mpi*pB_^M{ z<(0TRTEWnhQ9}}W4OILG00SO2<*!Z?pHr@829&;aZznG89ponEOd*J+f+aZu@Ex8kPB;z> zdf8S+?Px0^Re!oSTE`^k65w0r1yxyjLA8$J$n1NCD&o4y z$ujbYX**_&N`AQB9wZeE$5sd`eqsNFsfE*~+TpQkzLgnxJfPt{O*)+Il-i%ESCD6x zg|WUBB-+k_SiCbCphyYxLcbtIJwsmTiRL~G^z6umV9C&IpvgqWES-09nJYvO7uL&K zdf zqlGx(zuQpOz;bFBoX&SPsAjWaySWsL)&PfsJ{|%~xqfP4>A8-NC*HE+U?#N=YUn2( z?#n3>H50Hc3sew2zc?KT5%=^MqI^7%>5?daS^JKL){?h~I}4IVFaVmk)$n&KDeBO}1Dx_#6SFv}9_pqUF!qV#0v|cUPITV{j@wLb6FA zEt_`5pq()zt2)z%Cp+`aiht*$S*!i_S5AX})q8)Izxbome^0?H%M-Km`Uy&m?C^%J zj=SaLM_@?c04+JaRp2LTJZCuTY~X%;)A`<4ZT&&nAm^;bYR2XW#7(o~? z6-s9YY)O@owpAhTWCs6rF!Ijuo$<4zS9~Lo#7oC3xC^YA6dbg~D<#(;I3We((d6bo zd(cb_czA*!Gf*|i89WIoC$J`MEluxAriE=;qSSqu=w7Sct-|joQw1Izossy-s~1b& z!K(vS*83|1HG*c4H}I2ARl7L#u!c02?ZNa}2)>h#LK9_=(hBTD?Xg*y#dryn)-dbUiJCBjR#0Tf z#ky7XZ17nfRp=D#lX&UXpb|`KXTV!e7ag1>H<#|#a#2lRCa{D(&e zxrF2o%F(Ps>%{xtlL>q(T1)dx!|?5}kLK)#sdvi*sNXR|f9Rkquy)`+ND2WbsXASf z0N|bc--%oWx>$M|VO~p|P9vNb!A_exQ+sRnRY5_(TF^`#77ZfpH~!px`}Vjb-LF~? z4yWPTPtVCpMY2j2^;5<^noXEiJI{;h#ktOR<>!vQ!c7EZ>UQGYe5Om0WdO3@m?(QU zVTt|@Z&nc3h0T8`$n+d1dYI+2IVJ9~pbTt31HiZ$8%rl}SKC|PnLkzZVL2lzZ=Hu3 z!JQuGv9$cCF)G+z<%ssv1#s5Yu%__gfG&Sh`=4Hp#L5Y)kY91~O1xRkwTI(oP2+ej zk=8`9UgZONYAnkxiV6z5jyUKGnmqm3QABB=oiPPvZI7_ExBPaxXQ7QhSHykHRG;_JbU>OjAGepd6IB)bIp+%S;5 zt?ef%`M|^q&~Ru4zM!dSja-wzs(`6rmlBQJt0^dGACz3|DYsR-xB)DcUmClh?47lb zW&ZPzW9_nhtQWHt5bhgS7qeGVejYzvYEDTUM!|)=?OeQtRJ!W!vN&+Fl2)sds+fW_ z7dlv-NE+--rw%4a-UTeSe*@rgqR{TTg3Rn#Q4KH16fj3g)<_ji@$Ie5#3f+0>xbv- z*GqriJfwDbrM{}H6$_Am={JYKjdnKDLaUa0$i~{j5U5F~duX4+UN836Ek|3u1#n9I zv1MmCv#^xqgN|MOxeu-PL#w+3;*S}(s|FQFSt|eE2YlZL#=qeoyOp&23EzkZ)#n$K zm216`_%ZHs8cTwU{w_leEA_&r@v46w1v>r!eo(sR8`(cTnPnwGF3v$`TLvc_n zu{zZ7l)hva-75X0pIIjc-&o3h4h=p20lM|1KjTj<@tWMh#*Yz8%sc^fPmf!-T)H1$%4+oEuIn?|;T!tk*5C$w$cA>GZp%xld zd5kd6)@AHha4Mc<9kV;7AiJq@CC;95h&@L|ISxlhDHyv?vwE}tDY-`BkgRYacuYVdWFHyWBEpVy{!cx36fOslhq{|6mV z$ONV)Xlv`1LQ-Y=t=!P5=?oe?+EL_y^Chts%DOi0syfX!Hw_Cu@EWz)-1jBzqUdtd z#k-Z2vPP!(>9=ag6M;Edo2#9L;YRvT;$TLID84<>AUul=t%QvNkzb$SlgVOZ_g&>F zmPC7C7eHO$SUei2!3#-bW387cBolIRhp}tH=f>w2oj1R zkcif?FT<>NdApQ6y2Fh@d7Lbij(9IX*z@^Aa!ldqBlJx)LMgrL36udP_z3GH z2=OCGQ4AQ5P!g-t22&@=pnX9OhXFm3_Swe^$&ojZpvmq0RLeccATPDod7ylehsqzGMfR_t*jlJ%=5~LuZ(2fit+IjDvb+_(r)*yq}Gpcige2e zuO6052yN33{X^9fKIH*3$>QB@J^Ru?oZKDIm-Pj9@%9+xDN;Dk-W6_G8bPgUI#2(@ z@BVr}~`p}YSN9eYn@_hsiOZ?OQ zJpnOq=^Fauycue%Kobk-zQD{SVR7Nl z!Xp-&u$|%VSWI~D;#+baL@fV%OFHT(hyg+2nh}XEZ5AK;0k#NKRg7nF%Csl3kGq{ z8`8n-{Ng`a0IHDo2cQy=DYz11PcTT5MT-Uqks?HpbMXHT zPxD)@Vcr__rMw)lnGVyWCO{>PfW4Upwu>>9jt0KBs4n}8T%(Y|fYM|0 z#inb^`d?FvwNQ>*n6<~oE<}D|y@oz)2#`4%^};(II>iUz<;=8xHh{f)e8S?OlrV)~ z6jF27M%xKkR^^L=x^pI{csi8B?0JWo>$3E0f{LaQl|&Ad!w=$;=yu`sym~yz1lLz; z2XL`yz_M{k>3?{&l+~y3(Uc+`Bn|e0{Kg+7&Z0*1V6`vboy(tws60_Rh5q}j)H$!L zb*Qh2({0*RA>y^de{Pz-G14+N`o)VEX-EI=upgL*l|9u}fURSwG5=qz^WE$}7nn0@ zo{VBYjfq+3DK)F<2N3lmV6(5;1U9 zPWvdC`;a){;Q4vrN5hhI+SD&fXDWo}cJx0~w)MQAM%&Rduz&b$&VM6*QA8&<#eX9H zilT()aG2A0@h8~_=Jh}%sYjfMX-w77&?wr~r4lhX>jl2<)EFHb4Uy>yHyMxREx`@{ z-yy!>6A|X713R8b;6@!Vd={ao?MJEFmD^7S!FQZmH!ccHz_q0 z5VcS5YNd;QnyPYg;~_b52IwJSN9XoZCW++=;HQ}Z(RT^3X=n>231>ciWM_w&Pb@AL z7~mKgn)lF}Bg z1xJEp`_YdltUTP@I+@N^fxD4{045yJqU1z8Yze+_1}MXLsfA23tYN>^%lT!^Kd%@& zLK)TRne^Yc0`&GjugL=~tTw55rM1%aA-AF{{1XoPH^6Y*->kyx!}79AWgh3xBkIlw zfy~)Y*vyMlu#u)V#Sgj7DbtI;hwLdZ1;DUzBaA!W>53uJ>`J>aG0xTx{RPoDe) z!rk@jJ+Kr&;opSnmZ*d?+oC+@0iyT`@ZF}e6vD~E_md2ZjcuN;_8&TVeEad_oy+<} zSq)DnrhRy1WOFVY=93b-Sa9$4Ljc6iG_w|wxBv^V+=&i;oto$b*fS+8vPXx&n53M~ z49hjA-kRJk7VpG3QJI7$CH~)`8bgREUk1F}-umb1(gVQ8;9bmE+n@>rv5Q(`u7MA~=D8K9(q zC(WexGQPD)z^ZvVGvw0m%`Mv~F_)>)5e^IQu?MAg=wGcJL4Z+kK0n#@IbBXYTg}r6 z%S)3DsFnp#p^#J^5OThKh5??Y#V_|SHw!TST+%=rojaSrNZEa;AAkbtrNa`kt{386 zH!z_(AsVYT)0N^%)~?>; zYi+cT7(C3G?D*#z7`l5Nu8kc5f#*~NG4;ljQ<=9uTj44;b{>Fy-kuBZ-?)0_cYGCY zaR%lndM$v*l;au2o+%;$JbP%M9nIpCR%Jh=*r}v3^l+1hl;#tb21Y=b%pss=e1T4I zS8`&m?tSh&T=*Vx{wb2@x+id=z-; zx-Wpc@IJh1ZH9Cyfo~)~d?UX6^UKSR#gF1vs<)dhINqC*;GpH6w=Z72lz-tahM*HP zUFyUm$Yq27J>GUpwe#^9_-=P3yS@LF5 zufmrCzsB-~*=l?|0N9*ArGo((y}5g+mD|DZp(DV4`ENqS)d>^+j1(kp0#h%m0Zc3d z&i==geuwO`BYJ~+8kk*3V>|XK=#B1U+4rW(%DEo3{0@!ytGR?W`wz4Q5&aL^!kLo( zCEa=58u8B`*k*9XQ7}9;Sg@Hnhb*wYuDr{2ALaaIb}2R*NIjYWIi2Fi@r*J}y5L>( zc&54-a$)Ss_}e>OfPhkOHSTNk(DH8GtLMUnN_(%-+-6_HA7+(39KEki$Q*LMYxvPn z@C=~2n?MA&@1>bU?3`!%7eF5wX+-@>>D(~cMcLj{gTjO*J z-oIz`=5Z(Med9gjw`u|uncRK(CU|1bgoTULK%2BP9AiJ2rSRzsM@ehUR(=@R-n zKk^TuW_JJI#b&*1Y%|OHAMB5{`XPKJRdL14i)by4vczQP<(>pNx zyAyA9W4P|QaGr4e%hZL?b#CTebUYLr+#A8B6eVe+>(zDJ`-ZMWl4A@Mh(Q$RTYd8! z>Q7hRG|h>|MsK%V&1j=`mCFdgZLO#3v!jK29P;v<_&vMU*5N0avr?>o-d*=S<>Ba; zJ9C!V>*e5hA6%3n@@eZX9j=UEK><}*TM1$#z>pJOkjgr&p zMTL309uED`y$n4Mm)&IT%Z`8Y8|WCS=lEm(C}z3u=Y3s|>$)35;&Q0QRtFDdYMdzN5gjm6!$3V8|>qb`&p45~U9!J^@X_keOAC2~mGt$(D}0L_Aov zz8&2KbDdQai8_&XGs;}UT%0EBiPt{`$5|Clg_^cc@A~9)>ePed-`{s=R7F#ZYF>Ad z7&$+wV{=kPkEedVwt84GeD!E4v*{)A{Ev);VBOBLKyKZFh+!;Vd03(w` zYd3XM$q0RxO%Zx!Va4!}kFaIA!|upiCFy0*6_+ibs}NC>%Uh)w!q#TUr0g8yosG`e z;^YC1xt00(?lJk zvjw@Rw=_m={(bNrxOsWiH6kJU7h?T5Yz6Hw@<8rx?9z9bd3d?>f?ZOH;8;~65>@eL z%!U_S#)h+j-HJJvvsLvMX5)@l)+2O;sGzIs*n!(Hhu(^-n%TWQ0JG)RlStHNv^nN7 z%T^9ozD8-92`c5!6Y@fh(vr@BaSURsT--ilVN{!(Yqw%RpDB(c_pGxa(X9z|o-Q7*+J!@W-`s#jWSlad8dQYQb_1#LJ6ary4sQ zOWADkF_tP@LY1XxM3H2Q3#f2ITxWNQU71*EeoSh=QHvg~?ckMBhb!D0hwoQiN-Yi!HjS8a_< z{Jj`77*?`^C{gc59;<3%?5ke=vRD@dqv{}=e|{JQ#19ERwlKAt#lj=9X-6*mc3yYM zcKzg7l0x_z=)w-CDVvEF)h)GkZeymxYa{vxCoK)LF7;5`pFanCi@X;0rg(p63|Jg5 z`U{VLeWQ6`*Tt1T#{_=fo$bqr7oUBE}BL_yhfQO1;qy8%@>vsmm>f8Fzf?SZJ zhS+cYBS*>qD@RTLMTZbM%Jtwsa+EbINlHq}t$5Sq-BYtZ!aL@S+U^+V5efL54RO)$ z-)lM5gxnI{8bIU6U))+oW_INoa48ED{OBLQ*Jo7Owdl1rI`wh_hq86%fn0xo|HA{$ zj&W>vT&;cL+Y@iYui|BpXS>q}`SIlRrfVB6>2JtZk=zJ=HMFG~V{oTmO5GR|?I}C8%etsB>>d-i?LC2ee&i|9{@;-~UVKf^pH`Ehl!Ta!WA)3dwgp5)xKd+00B$ z=Wq`;!D!#!?+Uv}wcdFV_vcqt;*9|>V=HhhzSMR5n4@6P16WZgQ9ih|P6FZqXt8J`%yhaKPM^5MQBWmpk8!RQ65>|zPv97-LL zr|m-(qAbzZF={w?g4igBJRw+OfZd7sYR0KliF5}2bo;YsRsKc^s%ED0R~QmRd6HPV z;@xH*Q4fo2J7Z0>QmbK@J|7d!$@ZvNw}RYT{m=35MBQ4@<*}_Yam|W*C{fu@6cX%& z%JkXe2t)~4M0BEciu-hwhSg4g)@U20F171AlI!=u%Kyb+U z6_%yiiZh9dbNgSqZv@@*7n5!u`|#jRwl4PNn7~W!YoKI3 zorz9sT^#AR&Ac7tKyrak)sHAdGyPXF-H-N>WGzDUtM;TcaRnXQu##o_#kReB_320q!S8uFQP8`y|Fxz)xt$PvG+VfBGU z*6t5rd!aO8;J#y$DnIt{R7$*{QibP@&?hjKc{E4kPfqJzQ>2Iq#;`HT;YUcH&c@bk zQym>2cdGX3jeEYSY(^kR8%LUg_~5&u=`=-G5`u@IVy?}lqsuUEO}dP^L3$TJB^%=^~- z_#%v9qKLQE?!|ls7Lw?ehm+fnf>U6Jv8v*(Zicz#8ysh=$2v2d3N14y&kG+bhl{9{ zOXU**Wg7T)Z}8aJUqA28ywFts-eQ>Bf8d*v4p&!OsNJEUjmBmVhKrK(68F>Am4KdF z<6L+7%TS5N!H_nJ!d@vhOXN=s)EhA*Q2tA6ztd;uoFlL_ER}H?xVW8hDfu{G=k(msK zKz1AB9Z_yM(sC(V0otkm`~_u$-mnIT_8I`F{Mj~R8!wp-Z@9r`Xy9*aae6p0U%E`} zMq1jKP8I#Eaj*@TziXm;)1d)8i8%@IJ((Bi^$iUTy>VCp5XpI?r%fugf^Y`|_6DY& zbujn_X`qt!?j z0;eH-E5{W7ke#}9NPAhzL74{2u*~HLEpuOChk_|s$8w?L&5=8U>35|%0{chvF_+MF z)sCid;cRalDIehhH z(0b0dbnocf7rWP%gEfD4>4}=NxqBuXc5gq_#3H=I;{xy`&*(o9Mh!@zW8ezTN4Jd+ zo}JIU%}v6ihM`+{Vu_3*X%9ONhOa$JLDR+nfJj0Vw$}paCstQ?b|41h+6|trD(`*5 zTGaB}TTy*7AQi%5zP(!{aQAnmh26lFRtWEoh`Js>jlO9paN{Fk=_z+Vv?A4-g-}BY|2ZS zN;A&{mg>EHYo_`j;Q$#Tz`4CNCn04bOY=lkWpn-P;KVH;T(*nof=n=d4?R zFU2GrQkZ8P@|`2>$+M_!D6XG~1_X55(CwMUbyGKN^aui|U#M?Vh8F@{ufMm0xgt3E z>;y8-(AfsO3o}2FS3u^Pq-jnhd5ODf;irJx44G!!Z4(PlZUB>})Nld*MN#MF*IBEh z(rCYt)3mTTe0{^{C9@JhJL{FnYh228L+-ei&hXV4@oOL+BI070%lAcKUGBot)S~$o zAknWiS3rGSJKZW^r#6J=m;%WyRFmZ3rL2#cpMTr1+PhXilBijyl+4zXN)FR=TF7XU z5zz_$WPVTSG5u=DtmsU&sKx+5S;Da0eP*&rQS0M{SuYdx}0c zqt~s?H>`adZ9<>}PD~FUZJxH0%t~NS(wdIJ&LLve@oxjY^cm#5-4VUdQoWTP&=@Ah z+Inj*E15tx9ID-;1xIjvm`hkRYAATO-tZKB#y@Iy0jW6LxO?0^O=mp%nHn@rXG&8) z=R_|D+y1IN2p8alCs_js?VU|Rk+`=sECr-{ytpJ#SAw2xYg-r(F&)56 zz1&R@ERqacUGyA`K5d9*!X^@WZFTNmL+Bx1A1$&*08h-uRQ5jJ0yfh)Bk{96??P7> z$2D_koRNuUOXb`z*{*fd@1Gn9|20vldB1sc(0>s=#75NpZ}ovD&b6T z-B&UbB_|k5xgbC~{}!eb7&sf7UiioV6&hPKle-pKp0Qu%-MUY#@xpIY^YB>PQue6_eoj1scf8`(&v;O0{UQDzp4yYk>iBlKt!aB7>U3VOsHDr}ygM&IV{9 zWGXWnf(UD`i1>xm-rk*lkb6!Xj<~p4Pc15iTUPASC8ho}{Ns7tt&AywXEmoDjeysz zQ7@8M$=X@~fPah?`tnQDGpIA&%yr_s-av%B8e zMkZ~(gi-iB|7?WTu;zf?KpwbQA3N9@&y+T6?+KgI_F?&c@yD%gx@XzyBSUCYvY%sc zD*UzLHU!!AB>SRQT?J^WV|l{p3UWAwX8;DSytFqg0b z>@B!_v_8wM1jUH_D16cOKd%L`RI@p;@EhAo5o-uuY^{>(gT}h=V`+w&UN;KDSb0RTq5DYpVV=`xWfo z0q|O1F1-2iQO#^$**&U6Tdm238)tXX{rsO*s)E#fI~_NwSDt1E62nXs0HU(tZx5VQ zkKv|4$#mSN=tB45$9oT=vqklIMAW+Ip{ngHwnLRoSpthCO@%cm1*S=gzQ$%UHI-0= zSKAfG#*Ve-4Ai#4T#w$VTO>kJBD1?bTe70h;T8Sw2~i%jt0`Z%oeQ&=tzUHKING{=Vv=6CeCsf1d!|3i_;v zO8+-cPry?>plF`3O*vo^?zwYy_Ku5-Cs9roz?jWnOzwq-R+sCnZJ&rD0G z?0W;QW=U6atE(qA0Q1@ESmQMu--mrBq;L7y^p6l!Iqm;5kftRkFeQ5 z978Rk`>?5OE!ajKgVZc==h&J|wDtYa^=oi5+UM2_a%V0G(V994m}E9$;$~6)@L-Dp zdIgGnV{EWWe}+M)>CW3((jIVp}LyRZyw*MrG(L<=}Ay<62DAM^Jy zBfGP}>&97%wih=qE)qmeK{6^^9({|9Xl5Od@QG(EzyCK}jq`gT%dEI{q8 zspm23)va%23zj$u>>S(T$NPuK$9f1T#(EcO4$ONmk4lpDvk4?_-fA}E;E?b4#~)xA zt?Lhlq+w<^Qe{OV-G*YM5~qG~Cld)92Y5yVr|- zT|3@51F;HXMHA1xlBy=MaNfG+B`$Y|v`1gd#-d50_A8ZuCmboFQU`&#O!m0I-D+Vo zpuD|H3Q&P(%3s)!$16GofjSVWjy(n_i@%8k!ZkI-+^UfO2kHnsBhZSCpYUAPcx`*< zscd6a%`MS|0$2J;d)*0H6){XJ*82)c(O#?mOz2g9IQ)Hh!mJ?m^3^K7O{i-98Y=xq ztQO;(xQu#({&8y~hK2k(%z7fWGJbDHT>;mf5UR*gi38U>7S0C-z4cFA(J#5s#$F{x zkE_{3o>Cvr9-4uJ!leGX20c>OmTsl9KVKQSxC_m)MX|GRJ>_N>PL-lN($`0P)pHLh zdoMgLlS$NHH;B{6eJecQ3i+SOe51f~^!TrAzjg;I5%3K9DxgKfyR!u@y~1Wbt+3uR z&)dH@1L!p*dW6X~`kSBGDQ3KluMYS39iZ?&!ESZyejBN$hbnQ(II{1iWi_J6oKD=N za5f=<O2qeFY~^4oBeLR2<>@`ugF$wY;s~J&^ow2)_k@)kAkqhs7oBIgiP^N ztgH-)hr-i{LBFqbyLvdXc#z(WF+ff7XQJIA#b3OH`=-<5g2^HQJg6?3IwPn13LIib z9t@8UogSd&VsB>rP#4>pRfB22lrC4df&J64qey49g!E^d$Mt9qvdqYP0xo3h>J!7N zb7#F2#*c_kvU2E-7k;C!i~a#7*RWiMY*j)<s&|J{WxrsW;Dk~ovY$!0<|Zx z{f|`^heh&T7{RMI6Nb~?oXp|+oXk=~u8WCgv@57z4ca;8_4HKX{WgN z?rw0|WtsbqctM!(ub?~raC=tt`V3Zu=K%F@wz}dWyz7&|W5bgc%;f8-zWM%;3(WC* zI_MQG9(H-;12}$cw0*Q`0Qi+=Ia4@vbXOvBpO1>@p33kV9}+^74~B^E9$$V{pH%1& zaTGoMI?>CoEf&~&4LMHZJc7guUx~dpN5s%o=d!A{Co><9m&P519R~N(F!8pC-7HqU z^dX~`bAd3Gv#2){{R#hE{Hl(=dsI>sVm^Agbb%Y0!+603G2i%m8Om1oCU$r@uHIbU z}^u(jwR(X$`>i*3vV)>~S< zPiOKar^kla5Lk&Zn~&_gWwYv||P1Ef~}ZY}>mP!5)0=x%}lyK%ARYM+a^qWzX-?x<*u5Pxo7N#E+9(c0LO2 z6m+Wdy7=58^RVajzz4djG1MYsV=n=Ju3NWdLmmZq_kE;uOij`Jn4QWJ=6rRP_)K_+$ZDa;DI`Zfj3nC&d8$G>zt(0&^C_@f9n#jvwN zshe+wsP=r^bPshl;gsCgN5cj^@tVNusGkp7H?|=?`tSySqkZH2e-!RNlJ}oehl0+D zc4){hmLuBd%pn<@mqClr<(|7A4s$eg2h>7uo1>Q(7=6etrofP^oMKvUBRJ}dFLPZ zp;Yb#=)_w&0tCD41<6Uj(vU~Y1>+kvjHI~r;Zgi}_Y68=;rJ&-r~5wDe*G24H5bco zgUOrd>ESW()BE)umk$~zV9{~_N9(Btlh$gWl`0JP_#pO8AedypC)w6|or|~C5D=3S zAlcNRbX#&_^d4oLQZ^8YQzHh-GAJxb>|(rHlGGC!Lvp5Cly`?Fx{s)y{CK8Q;nJ~N zJ#lQxijVr!6&-wnKRv5hUHR4DNXM4I3S^-gxzi!Ed1t+F_WUsbo!8{n4ByaV^q8;hK<#lb<#_iXNUQAgQ9q+Lx^r`|3|kyHFWD= z-8S#PbX#oof9tkD<8`*T1Ko*6qqfw5jdo4tXyTFaQQx)_ARmlPxBF;5gK9K{sx~K3iaq9E>x^!@%bw&3QAyI|4` zTr^sJQO4_f#9GV7(tQ6(q=TRp;L{eYDJw5tQ!$%njJ}AG#OZU!5DDl0dkyQn}>hazt-6-#ez_bMw*zIc@C^Z`|GOQ#L1jOV&G2JrFIV;zN7@ zdk`l!(l+xvYJo2#<;RWX1;})0ANqZih`W7XqS1zS3+0Ax!NlMQT#dXmDU=Z^5#=Ue zmV{Vwsadjcrf>r$7x=~RhiB52(r?r^AwCN+_vS!I^_SifM^hC$o$`ye-mgDdq^G1D zZa7j}4KIsd!^#5(WSo6uJu9S{%0u_-+XWJNJ~akA&0$;(E30Q1{DSU?3*%s}%9EjZ zZy&1L!^7j$xKy<{NTr-6o^>h5K2BbE3mpK<^-eXu_cJFLnu%i~%Uw)BO~+deg?1gA z3=%FbJeq=g+6hD{Vixr=Yvcl~hw>Xu==&sbmXhZU)*bz0GyaZ{HzQBPZ`h)~Z5_XpK~y2J z%7jUfNljtf2@&Sg%gi=Ws?N8n*GMj`8nD>53C`1VJCj+F`<2gvz+|{-x^v6JX9M^A z560N%@Q1@pI0s{vJ#e1i%EDJ}SA%F>WDd<-dmxr}N1`7;)aus3QvKQrdN1jNX=yMSnsPUaW z#bZLe%yVuzIF$hDj=RrPj`~dvVb2Dhs%jktB%Uhl`SOOvJB2aU%c~If?n|-SJz?G@ z5iR1DaWL$D*{`y73X40oZ=VF2uo5DoZE2ivc+VV^Qd}>L-w-tXm`%J&JPhvWh&2)m zZ9x)9h;xjj1>yJc@OxKN4RF!JZu(0JNm;ib*aY5fYZDnEA3IWRS1>u6HhB~1erh&n z2=Q}~xNO(n>^yQ|304$+8)G~6znKNc%h~5RP9Hn82gqR>b>Yx6d*Ex;?+@wg$+lk7 z>#$SSP_L-*BM3N02Iuk6(2NbcN3Q1kxNTA;X+0Kd zD?Qlo@ntjTSJ*=}(6mE)?aT2?)4PNlf{)D|QrR_Ln8H*JXL~#5EYf^`W_sFryWTqz zb|JCm;`X@12O+F<8Z?=+S%;If-)yNJ!gyBqshvQm*R_>-%^Y&{sfFR_OsejtZEh{> zUti@#XP%kvV14-HzJ%9JtE^-Gk7m>FiJzVYsEzHmZ3pex2g}xq-$t%!b=62Fkt9u) zb>YeSBjC(CljS0eSd#w09{-pD;m)g6>F_$WR%{zBGJMymbsJMfmd<&}lEiCPu3HCSGe<=av0(FlQzV7`dA zQYa#M4!YSgM{h4luo>$T^EWzHd0l*~K@xZc2($0Uy3jdbA%A(IW=dA0tA@}oZ)m@( zU;T9yX$@e|o-z<>8?^Tq;cFnPAH9wz3GmUgv91kBa|7au#C?k8ml>!9&T=0)D&T;% zE;xr_mM1Q3-0`Fgf3e*4F-Avxv;{tqzD6Q}QCaI-n-S&oXm4>^mUu30#9JyZ2GJx< z&eOKu2XJ*+B$*egCgxX6ZV_$N#;?EsQJ59xBYtLzp0W{%)x9$VgZ0gCgZ1EkmU`l>yf8%YM9ZM>Ek+ImtKkV?B%+ue?Yv9X;Sm0^{V$ywL$1h!3yp|xO zJ#cgvON-v2)@Ic)y`XmH#QxiM{Ss{JT|d35fk=~mP_jy);&MyWJ}p$;VDL+Ao=?~q z{KoAq``ItHvo+2jMqvH&w3#L}H;7zDSAnQ;Ml7Z!(;QGG}TZz5_|cB zd`9*))+3NBG6&iA23}IsR1{)MNi;-}wJre3@~-Yk{NeT+E^?txrwaULMJZZQ?IkG> zAHQ)gxf_WW@&zX?j&oR-LW_;muB6tN1nig^kFwT6eIE=R&paj~#!1rTWVK#@&L5SS zOoY!+!vPPBF`l~iMA*L%Fl~`Oaj**&xW7M|cadRVDDfYxuYr;yJCmt?_3`T?wB0Re zxofUD$L@~_vh6$w$ym=MidwQ8hubh@^@27%K?p@oOgoF%W@+q1GKy((j-_w0?{HG+ z_ycggGP_7hd@LBwb>BG*VAin_kX0(WM`k-L&Msgh4C;VF*fP^dbLHpjN0goOp|1?q zFE%W5Z+onreL1^p4p#8ViPYNF)ukUl^4ED6#T`o)>=VLlmNh=`CcE;5G6L9UZ|1-T zmRKk}%+*vi<>2jS{yzH)3+;-a8UK6s_c}+N68zJz^gCZ8XV_%a)cuTW==o3=j1GeB zNkS`;FzomGB>Tt}cqKs5K(C^@T*o#q>mA%;TJ2X`C|WGH(!oKe9pHpeyzlxDlA<6)VlOI~0;^DsNxt|A_yLW(jZb~VxtpXerLsW*sr` zrVCb|b>i~EB=k`Pk(i{+)yO2aJ{LIunAhJ?i6~t>Ge*5iaT&9TJ%#K z-Z%zS(EiwzD?2<-{kjS5Brjh$1|DcXkJO{Yo8`G70r*T~dg|~m0v_6rK6E%`gRIh% zlI^AP1!5O}KGc!cN&;cl1y>f3Fx|!~jF0yDH{7X=WW`rN360^osis&qfefA<{PO3a zFXUbCI@GYN7>So60>%+P>H|cI56j3#2y{UqiC)A7*D)R3RTG1j-5Cc?Tp91l{k6+G zRgKM^MIP5D<0D6guvvBp*#gyW#YWb^bdVoX8V(hu8fZvA@XZ2bzhWPt+gK3(6PM}=Qb)OEU z@g^Wb*Dy7gaK4xaxRB1jmgwITxIiX1|6?6llR$54rIoE7C5X4StWm*`XoWdmNk-<~ z`g(D-�q4TUD7TR&JhBA^ah2Hd59A3oQjHl_(@c*s#BRW!bi8?-aym9!Szd^a<-w zUu7kVI>`e$z_q{YWkU#C9koWNnWVztmziBVd|vqa5G{(#+!{LVZlxL3gro81>W7 zl7dJ(>VJJv&>W89p9By0k%D+t2j+<1e@*~^!tRq)Oa4-bJTQB3QJm}o1bLDTxro-K z&$8zq<1#GF*Cd9GxbARU(!KZZ>n^K^8CR!fri>IvW7)Xh*SBkTe@F#R{wQ$0o4#zJ zRAxi=2r=s)OlQj)r@=&focH6I84Ejcir2<2k*(tUG1)ppTClEhacb#lQFdNUTO;rg za}1Bu;!r4Yve+!DTF6*4_3G8%TP${(wnm#f!?scPmn4W+LD_d?H!ldM;Xt(|lPIJO zUQ0-|hd-B}@UCOm4@MKMQLFjq_J}fHhOfTvz>*!>v5DuV0PArUW}tD{r{|H;H28gr z2Gk6cPWM>!6wzW9+H_Wjy)V}28pI5_E&Ft2wIovw z>L`4SLkDcr6E^NBrBVCRedIQ!x^+nSXy)5{y=yRGWhVQD-Vap`a1B4&m#u3<+6GDR z$DP&y7PVl9w4zXpg{aq3+)??ak_o&djTSU6HS)|4-B;c{tR-`Wq#eZ=BCl@oCe+Co zj&23r?Wm74no+1*=?m*HEN+n@hCMFiCzu5*A?cVnK!)CP>Fqm+cEjXGsggCayU35o z+A)ZK&%t5Cx9)q@xzqaJ8t+T677ar3_46nJU!)E(m}DQ0O@{Z5eQ@U0^<%Z!NWA|^ zOkF0Qz3q}u?APG6_gEDviZ9vMRoqy(zPTn@wTCveWS?T$OKUlvG4|)!IXd+8aayF) z98W36lvn8puiCAH$m4I&=mzREFBM7^htjS$=CS^Gnx-InGQsvJ{p^#3H9)ac7n(w4 zgfHo(U*em_eP7NL;S3G*791J(e|C?KU21*ioC1eXP_61w2V3;yx?YeD@V20j^P`@d zM6(Qu`<9!)a((|c*QxpM-@af|`H|7B?3c`uZXmwKahK($E@+?#S*eD}8JiqUyP|P|53S?(NzY-ma(R4uHVfN|-RPS;T>ACRk^Ut9&5mz=`QG09=s|&v z*BLb4Myn}B{IbW-Je z98xoxYpS0XBhS14cbs^dzw@b;A_p@$t@>lw^@S8&GB9?}1HZr@pZ>mb>iX(H!@&-6W^k< z@<}*`ffms&ML#i8B)TFbJ1zrZ8bSw# z3AIzoy4g6NJTD&cQ6cldKowloL3fe^gr5Q~_x|a>WV8IY|B~ykFO$tA%{8$gv+!+( z8V8eXm}%eA{A<#8DYV-ac?8|s{AmnW#j7HJqq#>WTUtX?L3bVg8fgInrYgQ(%(@Ws@AMa z|Iy!4dtotb#ZG|Ed&7}p>nhBo)IQPE&G8l-^e8*%-+O!iJpd-tW2j!e#rv-aW~4K6 zni}u;#6Vo$l}_cw%R}&A7imhuv2zlh%9FePyzGc@hPjC|=ejJo^-ikKKN)q?ui|8_ z66gEf16AH5vnQnd2>5+BSKSnBO9voIw1p(>yqJ8+ts^HH8UWI(cXvWUGh=r0Rr8lO z&`#*3SSEz87LA8C*;;XI7^BuRUC;za~!vHw&4!SiNf>t7Z?ogo#fKh6L5poe(# zEl-DK03!iIQ(awnGQ<0|B9t23-dMc9%;TH@zQc02?+0sWH@$V>-Kmft$3Pp~l1?Qv z2?U%!;fX6#kJkd=>-J<|H_q~VmF{bJoiCsMMXtP*oRYZ9-ZC= zgfW$o1BkC_wX~Wo#5al)e}_Z}%<_@5>uIv6y{KH2J~{?Nz>zq~#NSW2xuIelQMou$ z+#s^Kg@?Elm zU>YD?jSjR*lqXmh=D5q8TLoO@f9s1z2qaeYu8(k=a$tt zE&s8!KZSL@6MJoMt3ojVAWy+O`q=c4>kWui~pwC;jbvc z^nbv+&x%cGcc0FmH)grPt_7PO#BKNcV#=iBqU-9ku4&8keu6f%oYc=MVl}?~zF*%S zafnZUekZul(iRrhq79ib(I3t$^M7SgI|n@lt9cfJMNg6L2GCVo%&V?;)I$ z{BtO#Q9^hq!RQBKA>k_T9jEFl%S;WSTE{9x@2>IPSS_euw_I*|aD>iQ1qv^=%NH6$ z?Pe3Jb5&ikmEVrgJv}|0;OoA8wdyd`$W*X$3#oN9p4_-`BhO`U2;W1AE&MOCE*4#( z?)0e2Mi2$-+PM{On9Xbf->B}ONPND+z?0R*#l*4u|~)s$xx1ya%nU0~n_SWO^VxK>seZ7Pj7MoPOLf4>aikI4vI4S)PhqnJo*ihNEOZ$VrDG zm3@D+-E&`{1noxa+5G8~iC#v<3|m{=`x)Z}4& zqM=Zh%5Jb|r#AnEi>L85)T@VkJrv&akAfDacqfZp)%?VD1`*G-Ua7{R(!c`>*qD4vmW+TCHJ;i{tVTDaW z*lb=bKXhlQbi?oZgq+$QyP$w-hBZV%e|2m86x#LAN)@OVb&*|)*HH2ToDsJ29U^u} z4Wxv*QJgg(P1pTO$+51_V_Bm!(5rkHf=-18FzHPtX!?F&oiT09t8%oy z?6X$C}5datWf_W2n>1W=^a93=OKsC%RkbhqYvdw$v}U*=;`q-j7Q&x>XM*({CVb zRF)ois1dpi0taPi-QkJVm8H{5Bq<*4pA=1I3;%&YT^X8M&hz}32r5K}Z9Je^oYV>x zK$;%|)K`O+#}+cmMwA;3Xz~A5%GH!^@=59*j8~-;=9k(VrTQWa1I(@i#j!opz)iU zpfdFvCtD96aKKdnz;~Trw~>LtR;<=%T$v|YlXE&M)`t;1$>B-d11A9>ib)RKwLXYX*xo^mgX&o3JE-L;4kFIZxAXVQxr9pqhuu!W|{?4%}+v8l$V!hUfyuzAcURyj2(n96I$b=Kxzdq-c zwr3wmhAsg$eDS%5-yd|sz&!GH^MThNP7WfAYb6rL*(du5QcRXEWeVdejE96x$227w0$1a9p}ud(wd4d(k34$a$>x3v2+Jhn)XUBzWk3 zoodmaVyU>Kh=n3)6|u&SbRcm|Qf|V`rISKiz_`HU zEQ)5Lt~^%9K!}o6AIH)L{?76^h|fQJERf_tim<%m0XiW_rNCSyEU+ru3_8z$N1%zy(qVO!OEzW%{KSO9WT95<=o7ztA3ohm2hD`61$h_sF!7!cQZ?VLOs`$DawI!s z@01~d7MJz`!!)fXDr97e>{Ry(>f#&JeQkO-(-Vs(K#db1>#vcvvZw-fGv0JbMFab| za^j&YYFrwS^7tfjyLM;b$GEiHzD-fFrBqaNBE$;A57*yucnIvtw~-dT5=BXMb#)iF zY-nSb-y*u&MCQ!KAOeQynW@KbkOTg@SSBJ6p8k60-XxM*f}Y`1r~O!Hao`U1fxR?t zkYlAb%z|`O3vX=2vdZ-G4e5+NhJ4MB)`AHZB!Q?PwhF$-k_v>g40XOEtppLT%GCho zo%v~*EZ~`Sw&u1d6uKVX^*$gH#+1%u4J&hdTCkk36o9O>`EvkLxbz4N@YP-J9N)gc zn|G`Gh0Tco$9cO@1VYS#ro$JL)0R)J21sm`;Elg6m%Sp1y$<9{1U5e`8y z(I1m&(1!J&vX8s#*^vp*L7>efz;!!=XoQ>spFUY*XC{wwy_4#N^HsUWb4TH>@Nj7V z@-{C~Fs|-_thUa@&CP zj4QvJ3$j-NT*wBq*84kEe2xk^)&Kt=8iBTZ%!Y+G#(-~=@ifXGkm#SQcjL?=kCeS= zFBgAuA-E~W@OgV&bV8`G=+WJ-L#U$zRVL3uR;;xqC5fHQ+4Y!-l%gfFsZ`&!sR8zK zzyoeszNe)0 z<_Q*>oO^p$j@}B}5dT;!(!h}$zoi054$U0hyFRIcAzM-fAo5mzI8pt{%Vt61y-$wB zBEx>(gtG$%J0`ZmH4%to%IInA>xlriEIFrpP{+zJkI})5UGr zQJJzEitaYnFDE=Zw_<#fc*f9k=sh%_F@<=M@^vuPU++`Dt1LW<9@o{?ZG)$rWWX=LNAXzCW14EEF{BeAWj{kpMWCtV`UP<_itaibmG<(0+CpM zlQs)q|4D=k#|+T<+%hbSBi-pNm2kfbMN#I>(|Kbu`UgQbDuxZm$0_~ieg?R%odfOk zFTx5%QGyy?Ig$VT(8tc1S!$Ks{b9H^)LJlKd_O}u+scH)Z)sO62ZvziTKW}lHtyYB za6sL_&&{GEg(XyPvL_w1ZNYAM5EFd><*|2@7R*XFHhY6N94Nt7q@){H)YD^Qg+Kln zx1P%Dr;(!SkX>v~zLm~-qYAkZ;|tq=U6O%Z|51U#?5y`X^XiWn#Q5%|mNFGH-7MAq z1S@8q5bgC8>sYC}<8>@srS!X_Wtjr_97-HN{<{AZoy|V)d@Da5HvLNEcAQ7Yw0_ay zm(mYDbiQcQUWnzEaqV@Mcz4P9eDI6Q+nSRfb`8VtcYqztsQnc*Yo%B!B_UeW|BTk= zQ)n)XAd4cqJ)xZBs@oW`tgvV|UDhc0=VBVpmoahop5k3I_^6&vuh$@oVgK+=*oGZ% z;0WwU*2>SfH}#5Upf~Bez?MWhPnF$r8rpJg0`di(u+mavAMTuRb^9PVK5F0|*&g#Y zRH0U`SIZ$g8JAC`xR(5fb2FD@+YIX?!Yy(LQgo0<@^;m~7eyrZjp&-hf))dz^h%3s z_n=7AM=sya^aMtdXgcSsF04B0`29RL_z-KHdDQH1`QA&kgP=SWZT&-FB6R==aK-Mr zzH!U%Qmxb$R_mRKM@8PMl?FN^p;BJ=R)y!LMp_qk34VR^9*mBc*0TaFiKK-|fh&=h-oQq>b+s)vJ+t!A>A;DA*2o-c zUpYP8cdh;`{4$JvN=gSF^gMllsBe7DIRkq^R(#qFtuT~4L+<$f?rPooT!%ZqSOXYP zyg$C*+2UEO=C-3a=8Kn>D-J6T=32#qAWW4Pp!vJ7wC{#=_8#yM1$b0MG-soaW5i)0ifP$_4yBfd4f*CKK zI{K$Nq(Sum>Iaj?3)8(Obzv*Nwp@&@ zdKD(0xv${*S#_(0Cuh4IE~cyc^t3nRk~!M$>bylRP8u5>_0>@%X{gm0G&?TzFbX$jgTlPLhn(VymsEyS@#$CBeS;=Ui0o7$u3d z^5KW>f$et$$o6i2yxMK3y{?qWI@66_g;K!}a0Abm#*+^I<+caI77Jh>??BG*GAu|F z{nze-cwh|zE%h&en+FGZ&6gd=n>b|b;rp4l-MMNFNrWTS6M&)kWNMz=lAml3q?-2X z!idX;QJCxl1zOFNAB~Q-9mzQG>1{Ynfs-JA%vy3jZrtsgybWoq7+m>&(#7PF^k_4q zHU1m4Tg@+RgnJ}_%dtW(TX^_6H~xf=@Y^9ruJv0&gs1pKAhu)K2%nQCrEKJ`EPbs> zsuW8tdjRi{B%%?Z)}yyzoaJ{6Ng`#b%Lf)8PM;k6VS6O}UjD6g`$_HNUv>gspVnLC z5bW6+d1VZ)yc;;~ReD|d2YkbtZULKzU|=3_Tgw1~SB#_PmtVhRul;>jfM{qxD96j+ehKjToyO>7=$uyR)kLO%6P7$2kh-KTUAh8jq78OB8+KQUgk<8#0==D@k2MrZx3gy66h#$X z&qf; zmi!(7Q-36`0jvBd(e+T*IR*H&!IB|AzByhz&4vBML!{RI%Izo=1q_rn79k@jHTdFm zwC?aQ_+JD!AnnF-zmxPvW!bCDp?S__mU&4#z46 zUUiv97-)=Z#Kc9+`z=xy8>s2(Dp}&6IP%!G&-oC3!KJCoQIrXhM`dPbf9S=3YPnbv z9-Qj6>X+Nr=mzj0p+Up|)Oh8}FXSdGD=BZ+jA8&3mtc^5@~(g!Ex$ z2u1Mo*d35N9`dDVD38U&F+T`=Ee~>y@77(8UQOU*vtD|0iiM-yFGk%ht|ux$YMcQPUK6Ec>Rh@U@^)TmsJFX1oV}bZZ){4rI!O_R!Oz34Fv8lXZ{0T&7Xab8 z5{>4#`dznQS%mnyjdH`IzOlZ0-84XP;=ve8LG{AuE-k|?2GWa;3lP7A>q+P(y(HCd zPp*r4eq{Ai@ikr{ymcAW+_?qi`s)i><>~r>%sWascA+EunkyzrU;cH` zZoqzMk$KS2R3;Q~i1OHxwf&^_dTe{zIrr<=A^umqBVE!C{q&694!ZFwpQ5#p2`MRe0oFnicDthw zHxS;69IY!a9}|S?*W5&-Kt2S`cdSo#t}g1FsZSC;Pu^T@xZ6FYqyGw!57|p$;+tbd zO^`>RgncRZeJS?u)qt|&wMs`Fhqy3KBXA_!8Y?n51W+;ONg$=4xb2Q?<|N>aOf6z_ zfKCB@C>8coYqlBUrOLU)YnkP1C5^^bfVr)cm=^MBJV%WKNGgdL{r2^1v(V`(V_UM@ zy!)vkz-*s=QJ&i>HOYH*hcj+UZ^!GyHh9`hxUQX z>Pg>T+6p7gjhoIysLv(!1hFUbfnIv=>+*~4sCedt4LISOo3k3PUw?X5@oZ`@x#n!^ zS6p#S2s%94FfOj{0F_zcJbJQrxHSQ&lU(Gft!Z~fC+hi=d2%gquW zPAo?f4f$jrKr)f}Hr*<`-Da0He9vPF|C-8|WOBYJd}(m>%%k1UdbY$Ymi#JwI{X5H zc_;+ghRlbjq44!O6fz(9+^i-6Dfnb!_miZB@#4@PI@cXJyG;XhEfYX^+2JX4fsP)l z00+)~4YX(hpmP-FS~!E*L-H-6ydD>4_fw%X3S`YjA8tYrp1`_x}Pdsb!^D@5Ta9c0v;srSnTo$pDaZsJ7X)5=KC%&pOYmM2C(_4 zivBv`y;`2-E0P&w5+}!t1MsXy34AydqTDVe(nB!BaJEq z3HZF|w-6@B{b6oP(1&_{k!?Ys;7o-d+LR$FPF6E)nc#U(%;PYBg0=`&YN$}x4GpF) z294ft$ufUk-0k=w9V79BHPSFb+eTPa^szT*UP_7E$f?H0%J)(?5;&C8Lt)7gk-%a( z{R5Za<5qLWU{6G8zyVOU`juma-Dckm|2lxUMn(qpcYLEBSoD1P?D4#(qD1(#dYf1x zhi>)Q-W|x@N2ee%swi_RPC`1L7Qq=Z$R%Vad>w@gaRcF;15o9W1rW!%q@oXp&AP8R zxV5oeQh~y^T?p3XDVad_4+MQcK6s$PYo1dLP5bC|y}cuo`uM9i=RxlCUU+tN*M(Ac zP|0h`Zq6e&kRy1q1>s>%|6E0o~^w zUxM4^Z_XDel35e_qop%?CwAbmI*r+)VWe1!%l z1&0d320M5WuS{Pv23S^805RLd?=6-HaWxE!6(emcpx6IRp zAOM6SRJ3Zo<%0!(0O4}mjY>Hdyu7x4>@;&0q%3f8e@#&n&(Bv@S6g)zQn+5YA1RZ; z0lWc}A~>cus8f{a#A$qP;N682L!h9X<_Zv450L(GTrKtw<>ALpi26x`*Pu)Q$xqn@l>28!1 zc6^(E4HS2JD;1VxJ)b6FA4FgSpV~vM0^-C6Ta(P6&{#Xfqpb7v>sgFLujViw_r%9& z8`%H}U%|qcp1&@K3*3I6P@|uLL<{uB!KA$W22FD<8c9 zB?%w`;rhH`2*XQZjIk3Ubv71=-!}CC#cKf{mM2&)(-`P$UV^rY39zRMNg+WM>3$kxeeY95sQV3fDIMVC%NspLQrl)LXb=cX$MB?0FY!lsn8P#G1JOXV>`mdPt~XuEa2QEKQTkC z%GZ|(qpiw~fU}l&_49J@RAEE-Ul#?Ac~^}$!DW`SrGOVT5y9BT%EiJ}+a?B>qK%Q^ z)BSC`6a%?t*bRQ@Jx!E}ZO1nZ%A?#bu{nrd*UR*-Ib>FSv)F@qle7^Jibd+Xno29= zGFC$(og6G7C@%08PVWf;Zd98JwRUOhJ5D3PVXro zGySmy-=ly__hWy++APprExK>wxrHx!p22R2Kb=8PBtkKXOv6qpG?rss+pS^^|~uHB(JJN2>3dFszFA1*VdV zFnh`4C4rv*tDiZp9WKui0!=y-*S^mq1R|N=Uh_jO)l?_?8Kt)RBEG~#$Nf&!aFVRF zsiC4;>rqzw6sx7Az9j8Zh8t(|V7 zd9UC$*l=~8a%BT%Y#RBImk2$`PL*fG)3g-&_WjB~>!u%(Yst6E$@AZXki>y8*xsRjf&j?AM(PD^{U<#7H8FX_r0{DEB^z z5(S6EF>8D~*1gCLMx$5U zFITn&u^qvL3Ykpp4!S<-`^v(dzI^^BQZBG(ufnJJAy9OBy$#m9=Taw%3Q@VBgwQvR z47yg+lOU|dJ|@|v6Q%0~e~^Gl!?40<_^8#)?z;`1$?fpg%raHKL`92X5&_yatODT~Lp4}$Ea9pQJS18zAIQ-RZH zX?#Q)Z9EDQkgf0ylzjMWsxCJ;-i+&!4*00)Kz)Pp0gM4rE4FDvCwfE45-}t8nIRyY z0$ZQXj__$RT2>_VMm0Tz=(;w+?>%7#Y+WjmAI5lr*V;aSgwmXy=;wMZFIq7mCcM zL7d7HCUUFF`NCJd<~cX1c>Gi{vl1r z-5V8-hnFwDW74mGTqR)j{z>3<=OpsO+5=Z;B1bo-%CKdh>(Z(ckR5N! z&cevm^mJe_NhU9=!feU&;rhrhkhb_KlH6O^o42(O-tW4xSmDE~05B+3qDY~uG#}^@ zGsXQ)*}^d%K4aoaUI>2=Wcq3dCvs* zH%53X)uq!L4c=C$=pPlSd}XyRA!x!et8CTC=*etH)O@oOAD4u4uJ$Jdyy5}mAr8$6 zO>jAQ<&`^}p`5~Q;od~o3QZ!eP(#H}&)tgeJmR*4-&YdsHz+y!^^w^lH*jF1Pwc4* zkoiYUyh`bB)6Z6`kJ~{M^Ef#72(EtuXJ3V^oy@tmKM!(vM6mPfGK1K>@HqKEzwpq| zm5MS3Z)bjW@1b{zUg2`EWY&}UNOR(Jdy~t-^U-I^!h$Px5{SL{W$tdX_)#3ZXLK&S zd)Q|0++IU?|^cT7_B?`q4(`7F5+LuFJ1L)c#yeqP(b z@eRE}2$2QaF@49|%$x9WxzOqPYs2Q4q-^$%^#qf6X;mZJ>BV7z%3rHJL1l5-TNefx z8T%{o&#n3?pp;d&00+71TA63i^xlwZS+|7C7b4pGbePBPjme`k6N#g_0>vu z{L0g-@WYX5hDPa4uXFnI2MM@~2Vy!s;LgD9jZ`{vLE)kc?btKFBs1miM;+DyyhdaP z24gF1ht=Zh5~D0L<=Ne5Y`Pkl5%R$8D0B7zHfHmhSAyW8ix|)>=SKS5UPEhFo#o^) zexaGO;NG!KNbBt{D9I=%^+;PBUHp7s?>h#N_y|Z8f^NO$WS!p=U*zq# zp*nNiJmVX-x%1t9v8GtROOE^dE5TSytCZ~|nV@kNphUHjK2{s*Z}tNdVLtW1Db}}z zbG$gxgkAbDImkllM+M`Z+L9BYVa@w08daMzH$EPXT0{cFMKSjxn(|=$o*J(Y#tX__;M%%v)m_c zz_M9RvL`$AJ+C9VA9WG;?o#f6JLhP)(8E8T`X zq`U9}V*jhPN}(DVtd-s|D{;MZkdZ; zEXO=IKgaQ{f?;6itz5x_wtGsM*)%9^-M;?p-);mbPHy#EGQmyw{`N=U9iGcUSPR&9 zU!Ztl&mjwB{aEv900x8ax7r)dR|Cl1Ip@l6!exkP6Wf!}WCM20zu#Pr z&hxCX>1?NtBh3H7%q-I2LrbIL=tewWBG`2=UcZjJg-387cTFHS_4G&pPpjt(0eGOp2se9k{F%`OV(PYQ=u&a{t) z+^uq&Ct!V)t@`PQE|Zmv@Z*^|Kw}xi25+u19qWy>lTB}BGJ9KnY>Mm&(>3wu7vA*v zsFqSVDw#r2YM`7%ad%MsLMkcUpaGwHakbnRItt2DPNz_WpEo5Yn7jBl@D2+uAn&@|)moWzLvj+G-5M&*MJ*mq zeo~4xe*TLo*+g4vv`%PSXL9AK2AyBh-m4JrF-+8EMKCRMR&vy;bDQ(s#gOT-Q$Gvi z>`LFHDCO>w5;o>cwQ9mpPdR7Kn8(lW9abpV*Hwo4 za`O38bs=Y%k?Wt=se;_t(`VO^lM$J0MRqbO_SJh)d2(J)ltzp0QD5=&-+S2a{8Z_D z3ibOz{9aS&;KWaE-8f7u*1=nwdMj~)P+0#n_t8VH=8v!n+w>K|@v4H_5&WzfkrGF# zwj8T$W8OnS=nhqg>vE)^Sig-^wVUiS_Y9{Q{kGR*wOL_qHw7fk&Rm8Jr$1%7H~w-j zT7>s&liG&{PuqNZ=l$besCcJ#Y~a`y&uMk8x^2el}$DZo8kOs-W|xRW}u zul8&N*(zbt_CEVav)WF?Vn!@0+4rrD#wK6TC|$>L&MN(#NsWmKxxG;(9QJOR-6&aZ@B^n6Uke4roiWV4Wes^>sp?9Ju-LbJ zK8_UMom3-hz^GQi{sVxOw9^Vy&h{2^@(SQRey63IeKve>W+GadZ|GCOPKu%3nK;c< zHm5>%21`Ac80FBV3ha~{nW-FNyVZK&DF4}BLP7Juap#nsMWwFQOzN{4ncPH)V`&pm z8f)*xk0-Q7)uVS-CPG6B?OyO0w`$!UF1gG~UYqzt{kOoT_ z`){8tE7uy1rqB{jyLYQLjY4O{I$^JO=EM(VeeJ|*I5rkfYMBGIqMzD{hz9Tlzq= zZTJ9+EReC+{o$A^l(NCO6Rj6&+J3^zYSfXO_H$ujp$`g1LUEE9|JVHf(^_Kx-7qWQ zp;Kefeeg5gyu1H2i2tyT|9YXg143moJHm$q@*gYy-7x>%kpA!I5XBORMzP{8CvNTk z|My-n!sgR1pQr@>@5ldV>-bySQ$-*!A41IFNxzaiC|ou+1qkn6-umDViyhBk@1gk$e{72Z8_tA zZb=9ccC%jsuT|w77C(s+`^Nd{S`(rVdl1N8X;bwFwmds2;K`?p2BZIc579XRvDC`S z2|y^t5)f&(o%bw6JYOfk7F7dy*j?_sdwW3D#$vads*gldsTe%?9?z#y*;mK2{&7(I zjv)wCx&2nW))J6z!drU-zQmoMoBI=p>Xierf#`$qQPp8`954s)e9)O%rS5-k&VTFS ze~)85E(k&b3L_w0B2$7n)&P%9upwI z)%F5qqVAILYArY#&?up<^JWCH0}`$4js%Vqd-Rh>Rq=j^i19q6@$ZAUdZ7LC$M!RzjV%V3c?aB0!G8W`^)Z z)}1?dTJp#ZF98YKGN9hoaAnWX!8PEW=X|uCRuGMb+N82LFd`U?A3+O z0#^DB!$k~Yj7d}oA4(J5^ zC@&ib4Fn4GoFPZe{s1mAD&0oV+7c#?Hy>9X>*KHKP{RI~)BBGJ`+r}6g@FZluQcqs zFKXqoY<&ns+2+#}CmA)y7&;|rK%K;R*mwa5A5b+J1UJ2gzMaaGFC2im<}mf#!Y3UH zWXTlM(EPL{nhx<06eaKv3n{;z|8QpiVRmM5aTr>u@7m5!{xwr^JyY5Fp#boNlK!-a zole(yb+|Q|JW3CQPaE0sPoNOU@w`os7z#mCzehhqic*H@`5owjM1|>{!i54;O@e~jj<&wpdP9;4sBR782+dyMR>>Y4v>=>ydtD&p4th|fDd|7DrsuI19Gqw z;u+NywnpYcX5kl~i>PY=qweI@{E@{z^>G?B_w5592LA+nj<+Wu&S;49iC z-{XY36Y2t7-Fr(JAe)C3wv?Z2L;qV_L6FRxwkBU9imh&knUcM46uAImY#@S=#7@<7|GZy)j!?pry%MCe|osvw^8B;{}Plj$Ina5@oyCG#0Mn zAVKW6J%}ZFx3%P^8NcEUdn&->XA}8 zYUc$EJ`i7rF$-D6L_6}u?(g3+8-LpNL965g*}lEux9Mt)WqQ5MZJn;po*&gCdDnG{ zALnG}ZW_f!rN+c8<4EdmOr6mipli$R&Pw0JCSxW)U>8+aH+Hf-p86mlAkeivA|5b& zehxpXFSpx>iH?reI7kw>6h7$a=y-q5^4L;C!?9v`s$%h|rT7cCU1vLJTG+0Icc0i+ zKS~5E%s&Ot)Ao}>RM~I2xa|GHBUp@~6Cx5acG0PI8Mk>7Za@zWq(i|uM=-hpu?6%H z8n5O7&`ltW>AsRSgnlc%gh3<`mfT)412HlHqZ>u_<6Xdi8Qy@fzq)Ua^!j)@$uX&}7_IjhdI|viCfe+f2-7G?7Tj9uP%bq%f>t?g#YdeB27NUDwHl!s zo?_fW0G~COvuk?%3xiUviDZ7+4zd0QA;Y_MWE`NpLWAH1t<3uvc9sdz^!ZMSbfH-D{z8Ae&S zf$;dDD9h_TkAmDwQM%s*FCBOG-#Z({D%o#!kJUD3>;?|$+q63D>Day2;gf*nLExpa z1CiI4SaM0EP%jr^q%T3|@;HK(kUV*<_V|^y_PN%h?8{<8&k#^mr7V4WTHEE=W%wQirPJZ+_R{LG*=Sy7 zW|hDFM2T}4Ex-L#OHg8BjH=6O`0rA4LoQ=ko3w?Tf-+6HZdxEwF>1Qpaj?r&S>2>p zaPMo9uzSOZ)%ieOZ*Ol|ms-B2y=shHJl_nkW1F|EZT?Ddc2kT7R%RE==RG z{PnP_yPJ5p4w$;H=bvZ2phpDUufBzry_VbAAp(o2e>K@5fRp3QdurbTfCrfXsRd>B z1+2y@L4oX7}esnq;t)Y=KrVUC*nV#l`^tgh$BFIXP7m$B;zOcoh1YlF(I zG3nnj49CuHdAMUsHs&#>4$bhtc<&vt8o>U03mk+gDkKKjAf7CPP%6s!HK+;Q)_Yl&gbbcM3P zVp~WN;Mm+d5KpCiZd({zw>j9$%Ei@nSNJkBh^$Q^`G@DlSw&lGma_3?bC&YRm&__a ze5;Sg;Mx2{W_68E~X0t7VA*;!eu z->krxUYqZ}%O_m_#U;C%AwF{(h%+PK#_oD-80 zYiwgAU&duDW4?2V3w66+*zAX<6L9)U3%6}P+?=)=cF5j|_`*F>_l6FfVTZHQ%A*I4-9XuEN9g8i-!s+cX1NWT|c&!|%fF)F{Vm49{D{*GOm1}3o zl9QXen@AEs8}%Ug=rr;nHR;#v>)!VUl7NbAjv`}v{;}U9k{=sy5JVX|SnCnN7ttM{XwLkr( zFwLws{RhFyNMx;@ z$?VLY+R#+qd-|ohh9>*nAybz1{^?}rQ$h-sMjF5v$c{hHFr}|ouW&k8cg3(l9TkEM z$mRFkVPts8ME&hBrf`pmsNz|H^d|ZiCuY1AG=GIU&Q1`@g!yrTjX}+`bYHT|?>(Pr z3wkiy_vh10LmWyZ|7c)S2^q28qqZivnRn@NnxIUvmn{YE2E!^2n;k7g)5>}$#w=SS zd6S3j?Xcqn^>=f1t~zS;lY3LyCJv-m z=T>HS+HcP+&W9MuCTzaQAjP|}zhPEprFQDzi;uq@UFbGwRP}z*k2yE4Ne~f@%J4AHyJqWarvn(2 z;j5kL;tYSAM(=dBUg%--LR+py0#90MBpPz_K@t<5C_OC9Uy*Z->5r~0Ha+@+K~YWr zeBgK2;rxK1s*cK*F5&n%n>Na0Dn-aH5@48)2x(PJJnm#M2hzz#io^&e>*~fN@TwV} zKIO_YI6LH-k0^3u*S#g+uyb&{qb`>5EzVi;7W7#?39XR1fCFZlmLQ0ev&o;hwAS@> z>~Kq93*EmM6Bmb!(};5u#A$9PZOWC&khe#3Cf_uM3*$n`&kiZHa;9Pqzr?=T?|bpa zVtm`ntF5B9b01CyVn)3+A1|G5f={<}R)25REO}=N7|Rk7+|x&GO-BOec1K159&l(p zG9#zsABTp2ya*xpth@;xcGw;}FdLn=p%*SF3Bn#Nn6<_c<|rs}=7Zp@kzAGa*je4I zrkBRjdA||kOf{#1Kz$7kwyknKE7hrc-_+UpIN<)`?C+Ho4FGSM&d=Td(bd%@_)FF@ zNA*$%AmsahSg2ZGe=N)k#2{^+zWtgp85qTEah5-wtd{Ojpqo7Bv@t?EQeh+Iyqwjt ztt1l1YZX($jDPazgN;p%OK5_j;U(<~%p9%brdw-hZ0yPyESo-4z)~Ax^DJvbG!+Rj zm&KYEat8j3gNvwvp>@a?MHW^A)WjxmEkL!4yrPok!Dn-4V>=K5zVt38s%MQ&@g*C; zMwY|=D!d6B`|bId63>Tb5Ct#atKmlvg%4|y&*!RId1aFTSKlP}8YRdT%y9ZIPzDQL z{NpE~pFJq>3DXr)1kHW}!SPBvvyG{&FV{t|n;x{>q$BRf$laOstAA;D)AKFp;ZS7j zHO%ynLHgSZ6DH6QcZDsZX|gjvBs*)lhZcP0(}r7DN7RoU44gY-BGR3NHi4$0#+;=O z$p3gB2+k4Y8qN_DIp$%leO%*6pQ`z=Y%YRP0?~|WvQI=?sJo~*?TG@=+1|QUiRzy%Q3L}_zZxkZ|L;}-(4pf&nI_90 z5B`19f5B3;%>RO=s;^Qk+}{{l;W1qD!AjfIle5(gE$v=RbM)i{p=U6*pHD)Z+V zbnnUaQM+^xfXslImH@tn5*6O$eEZwj{Ku%xlH!D0UZat2qbnkDhK7bQDJfPO8X5;0 zKRcfmBd#@|q799VZju8%z5ol4i7^6HV_J8_rR3z~^uzr9;$g#Df4;kD>LW%Eg!N)8 zF$Eo8Bg77g_9&21P;i(RX+}{tzjKhcvn$c5bDK_lCMTLblK(o14KJPqXS*Xj#t0uD z|HVq`*G610koPdE9XISfVtd11li|;OTnIu`s;H_y_w4qRR#aC%&TGLzJ$FOfBRu+) z>Og&OP=fWoAHjKV%vCf85m?<=1uOucL|5&F7#k*>9=^1^?6&cu$h1CN736s;;X$G9 zTBYrmH-F@~Q)oBj7)YvkWor9O1@BoYwL`C4%EgiEOx63N_;>$<_RMnQ5Fhls%lyzZ zx92(|AbU|;U;|sp*R6SXzU+|-WHIa1_((-YST`MJ|7)HtO2W*cpFTC>=1Y8}8Z9UA z{`Z~!AH9j5X29ZiSR1%smRw*pKVM}Lxc%U|ACB2@fRjqTGz&X>Xu|LBJiZ{8KQ0Vx z;6Dt&lIxQOWtxf?q!pV$&;9-~j}| zPkD+mb|UJd|4&rquUY(z9}fgxpcfq&IM2xUheOvtCguaiWDY5bXl!bl639;Y&nF8Z z@r;kZS7npY+K~zD4Zu{qQuTTL1s8D#0Pg36a3W6TLWien%+^dzV}QC#HljbgE1C+j zkR%;wNAb2MZ!A}3U~^G<{aW47@SBR7ik7NsOo>T^1(4}~DONYU9FA6`{%M;Jd~FCm zHX%6VTgGdJ{~QshCLZi&PImTIYj=XGs;UvNk8z3`1Jh5_%`LgaGRch1=0l>j&(DRk z?Dcp516RC!AujZjSV6JK!~FLKr})OvFhKAJ{A+pia*5^E1?kAUirUH5#f4VloN66Y z0M6epVr;Cql)Aw;3kaaQNmHK+h~6v9EXi8QczgM%`MNjm&>(W0p7#z1>h3i zyaiR?-#<-|fXGu!?C6B%%Ce@B(eeeRm8KE3G_|{Xla2_&6YM=Tr79!@&~F^w^Ha|g zc)P0EVvMfIB8CJ$4qq+HT!bpkmiJ`$4=~}stFr}+MlIu}^LeVe2#iLaPO{edYN-0n zu2U1()d6_QXbrG2$n>@so>#+}7ra*<)dMB}CXMVzpK!GII@~f@0oMHm@94mcDx3S{ zNc;Ic{*2QeT8=1tFYeDbj#9gS-?^iztGnI1KV8MG>z2?c>V+B`HXE&`$!BZtE#BOm z9pU&T{RIjlHtvm{062DA#~5<^t>*{%rN4gNs#8drsWC?YmFp{%r8tB)>F4QLUp-d< zrg^ImLVaEYAi^sbWat0qymc`K6>2hV+EaU9d@6wrp@*p z1hHT3{$@Fx@u$=Jy*0^WvCafwYbhllk-L&*5%$38@Gx%@^yn185mYsuW^RydT9HGk zR^Aqeo{D$~{pIBVpwUYi8Q0(e2gm|Yxp*W?nHexVNkkB2+vBfSJvz>t8*Gz#*_E9@ z9=@LP136pljm>eg$y%FOT7IW2eGE{U75qHbyI!pAEk!OlLjWh_~V? z{GyRkYIMSHwDx2jw|M|y7g|jFh1j3$s?e1IjV=_yJrhX&37#6woiw>V8y=)z<4HEB z&#WfbxQs1MK1SQ0blzz*Zbql^xNTCJxmukoUvU}tPB3=~*`$R#-Y>ArmrI5>_>n-D zj^^}$P?SN2#bj+4J}7&<)>Xjy14eXb5IFo7Mn=9$iCXE7`)#AF-s=-k{%Y-Dws1{G zz-Bp@NZ^x1`vUov%;RS4|3=Lu-TVk|LdVZ!cwJ zf2;)qwB^kdew$VR47UOzRR_i-4PeY1un%8j55S0j0f^e&!eB)abU z4=Dvdly|4_P6|>8y(yWH@SH5O1kMGj0L|D?MJ1Y@^R1B~ke(}OaXH05k|$$a(6lXh zWUTtx>>ZWT7P?exo^{%h0H;K)l%>GtD?F z5d+ z)3>hN<2m|kEMgA;Cg0ZWeNn4UoUJ}71wPw~KWCic)_8ifWu~DK_ee?+rb(Zlt5ep1 z&md$rTAF9KHZ@+RJuBy>{(itkeI~Ke|VTEnN)I?4Ng#&`W*}!5Flvw}oyIZ!812 z-vn?_HNW!I@9OO;^Ceo=@F&^Yya*F28n9Whm>TQnaT)?K*A-g!wEsefKMa4@KQ9I= zDa_BSRo?2RmwjQGGesa8hvtvz4NuD*JP}|NY#LqnVJB>?#G4m|Js_ke>yU_;_Yv@6 zS;3c5gf=fH#S6GWksf4u7e%KnU%njoA%TRfZ(2NJLWk=bZvl7DvSy23zq2!u^|#Bm z6y@ViUz6Wl&G+!)H_h88-W4!Q_1y($EgTd){S|2QL30YML+-=6KrD>yBR?S2e`3`tqGI^X zAT~PAG!Jz4DH&QZx-}_u_e2=Pa(kaZ^$_xVpAeZ2fBX9N!=kezc8()^Bw?ro%G#nT00R-N@hW9Sl&e0vw20+zt3kb66 zt^iTO7s;mcdtpvDy0)h;U%CfsPjp9nI3?<_O z#VER;|Kv#+V`O+KPQ5F}PW#+tD2DCoS5Hjho?HBL)ASHJ-d23yu-Jk^AWh4eE`(Na zppDl@aO-sLi=q8-J1Jc;jm2b1kwh3li4$mc+`_58wpVgI6OpkR#i{CGyL zb6LI+k3u4!sa4&Xr*jfUSZb6#Y0JN$des$Rl<&6;DzF39#~KO2!Qf`U=r_3-p74S% zRw=rnrYk*37gX2=r$=1{rDHR$1n|Vyb+xWj#a^f1c5?wf{<`Vl(e(R^m=KD zm1#4rNy@yp7FYgMzGb%Z*jKPS4mmuIyz!(-mAH>&^(K+Jf) zI4iZN=~gH6mRONbs%C@aZAt*p{@b-Rs*z%22XJ>|EuIXTVJ>Jp6Uu?ujr69aHQ9RltM~~B2;Cd!LA-x)lVp{sv6}wdfjydHvkPJl^@=qPL z_ZyTN?_5eB)B1Si%7V+dJ{)rbX`%&DbRfNIEX3 zucHZ10!==k`$##WoQ=xb?WaItHuD4bchnlOl7wtpdpG)OljL09RxjdLxE-%5mAzwj z8Tt7r3N9~vfdvh3`vhJiDk8z$RGLW>6x9|lXjPu>mR+?e5g!Ew-G_N3Afhru@QKP5p-HZ})*it)Qs?e| z^e&*x*lyu?K>|1emI8H=_|g(TvMy6K$zYqgCdZOC`B!|lMx&8u`Ug%`DoRK|6t9SC z@VnD5_id3(=tS?V`qa^;##?G0H&dWn51+i`yKjB)J~A}Gb}|$yY}qBO^_$OOeXQXw zo3hvq$va&_3G|T(rJ)OMszwN$@>h7??BlDu!LlGM4?#`-t4?^z&rCC!IA65Y{y@ zao^Edw^6Ynd;5&+%I%A5#R-cQ1zib+GUZ6-}_g^2r#6=JfYh^1r|sFg6a@7hT`QE&c0GU|1|^XC9`im80-hrTvB9 zXh9)NcwYuq8M3w(Auf&7=3Oj#qJN?^&+!!qZGxGUPxCpENO@KneLnrTCL@9znp`R` z`~e3QifO5T8WEQ_AcVGoFHr{`DICDH!HcA>g_g|r{Hl}c?gkZ?+XOOOtNjM@tYUcI zGrl>>+K-F_PDkO1jS;^ok>`!~0gClPumFNEI{_ebj{JP-XM0gphP>m$C;kGOu(WjBcfj-^Qdp~inpoY{e85l^frv+^9~MHBi6h=E!o5u_o5f!lRv5(;q)MesA7biP>iQBqwuSDuCZq z=E2mF@adTEUDiMw&C+T*+?3~$w|Vrt6PFJf)@elWy{Nq?kiScKXY7^m890GJQ=n1q z8NEr-kK||bq{3IrAqm&?W1UyifB)r{XkVXYfQ;y&Y{JuVw#)_q zp?v?a@X|;)K&{(gZ~64nY%n%qYr{f-hN4;u-H2eqqON}MKp}oEnGZ#Ycx`q0F2|pu zQJHB~Ensgw`?1yH)OfM^tfzihtU_iYGt~W88)ShJ3Wsc+oaAk_tR%nxaU4{#A((ID^MTt>1bPZ<53EBXm2VGq#!UZR^v{VPv}LZO1|i*M^aupfz~&U5b$ z$FQ+Kg~)^Ekv+L<_Upf>v}*OEomtSqb~__2HIPcXEhphNoO1dZSZnn`&)KE6s0oK| zc|D;8+?)>{+}|Q|h{@=(n7#CdmKq=CuJIJQ3u~1jqbz1CCMCeho}Jl`DLya%O18W> z-G7fAe&X$@xBe+G*`0yM3Pc|x^oYkS+HjZ_2fI ze{uW7*lnY`-UU4pd88dc+=`3CvBy61GT0L=th?N6)jfUBit7NQC?o9BBIxP1w@fDR ze&LgFfr4kw}aL?mpVye&5f+Ro5q%aIHJb(&)7}^M;Umkn%#bvX4@6z9X zwNHs7#_@bSgW?P&XWza!f94dlQG31lGhMHB4*}XhKlRT)(69!DA1w$moXA}ByZmhm ziG?aTqbEfBm^P?gjxQ$;4$;CY&*8d~tuXd0iwW-x>>_53!Xi-1fC-|m*6wi*&k=FY z?@QLN8hQrUKEpVw2pV+;_*2VZx#Z&jAXef3GWB)r^VNI*JUaq;@L2Q`jf11Q?vpP{ zs0K4oqIz)31=_^k8SJMYI-KdABk>=}@{xhAxbEMVC;RcD=_P7&RmmQWCS_I~S3qB- zVqfT#!?4-X++cJV&YU?e_~qk0uM>7n4ug81)na`wqO&!TZw} z*CA@rC*1?)m*?Xn#hLAdgLc%sC%QMDl08j)N|^TJ{h(2s(cv#sD7(=bGt1O~aDW!r zw9I%`$coPAxrM=O|MRjEi;44SA})2cI`rnC{Ltm@Ft7G4FTGol)%u7VYs3v*g2|fa zGw zarZe7*MkEdcRvU;8Rg=c#P8hGLM@v7OB}TtTS{;F{DiT_s{Urg!*(R6`D(b; z(rUTWLH5=}Nf%I_#RBm99buo;H*GqT>wq+t7Jooz_yo#3?&;98Tu@s&Y{oN10fi>< zPD(t6r-1~0)O`G(b_G%rFPHX|Buh*KEEdlQd!6#P1$7sp{>BU$4PkERbi0j#-`L6X z0OmP!ctFiN31D^ruf&A8q9X*6d0#X;f->nG7U2Gi8Kk|>#ZNfarjYcsVI^KY%s+C& z$ZGo;_2Bg8yNdBy^JfC@e<$Q^E14V)c0`<$yh(ZLjQpz6?|m=l&Fv1gqR~A;y{`dd zz7IV>{(2SYen%i;$1EJnzms$%wC}cblEmu}vo-r>Q~%~3{uNg;>HYSs-PXvX&Cci? zRYeFXoz>Js8~Nq9jJs881U|wx&~%!`WXwc`GC?nZij}0={@hF(lti~ zZ*f8o4xy67!|u1;w$W-QP?X)D#F~6k%ddP#<(bwMi}@C^eu3PJueu4lZ(d&wQO5tc zuG9H3HRfV?3CwE$$g(fbv!e}+G+rduE%{UuQ-t2kj4#Cyj|MQi;o_c?FhNR+nI zfkBAH>k@QRad}R;f-|G^D+D>-C;tU6Eh16|@xGijx|&OsI2cc<<>vUg-h@6J83X&s zKVPeHmU-!aw)a%OQJ4km1jtt{JZ-UyCOMl(fi)TTXt0FtVDwPN)5?$&MSZyJ_@&S2 zrTpYwIju?+|Ge)-aBTD_vz&6Z8|uzBsO6@!>YztuMgY(E$Ip4dnPoNUnqmi+^dl`i zaPAsVt5-PQV80rxqhnU*TQjb|c>Zi+a$^|2ZcA7|Xo-O*^dWW=qkvi=#^!@ou3d7U zXq1q0eb$W76Fkx;zr)&&2>kU)f$h5-&#h}<9LypST_xp6CHovX;Sq_w;098cFQo~_ z%_Qto=Ux^!9HZ!3n~u1F{>?J%mk%`)7CHetj}=>r)d(^(lgU8&@H8vr0`@p=J!L1} zLIY`jIk;Cc^tLAD1f&5bv}p94^xa!nPh;;46I>zvXb)nW`m4cVAd&xZao$eibKPtQ zp}oS=n-`v|SygsvK~=drVhj^(6sg;@!_pc#;B04D9WEtBT6UbA-w+Aak98OZ~m$v@#qtFs741FEpB!lCRk95m!_2mN!) z>ilyfu=6YEir2!lXd1OUyXt~Db_0O*o>?8M^DPcBHwfI^Lr5Exe{woKzpxLBWQ5A& z|8ST7^1j*IAnU&_# z5{Kk;n}Xk6v&)~_KNff|+u1?o=`p341>rn?9F7lMhFsQNPIgB>UfD*e%??+3T6v@7 zv_5kuu&kat_H(eym>)}hZaUxd0)1mZSap1ca3axJb$Cz}V`*#IsYWU0tlke2rC+k<*G+5)ueFEPgIvA0(>lEE$VerGy$ zQ+-Fdqik3M^k@+6g&Uxz5SSE4xGC6(=efQ^Etx*(#H96oC?pPqR`=L7&@yrp9&*IM zraB{Ovz_=_E_EdND0!AF#7!VJhCVrJFFu@1(}d?KHwFPXsKUPGW45e=ox0llvC8xO z*0R>S)aQ$(-Yan5-}220{rQl(URWry@=zt@bZ3`&5nGwUla|+QByxgtfq?zz!mQ|m z_0GQE;`uEj*5+~tCX?Hb;#h6{w#v~i&j*GO89~#FuYvGvDgU{}+hqlpE?Q=gjT?)RQ93Dthmk0mDUAgCb9zz;Do`#s93@CCkS zff5LFe{BwkFD}$N|JjgI?3cejp%f4+ci=R}c_n_S0XL~0(}T;~Sphk|UiUo*OdITX zE60K_=4o{#p444KX z7w!P#NrQt>44qxUuxMBNX}7G~Q<FZj9|$d6jfDcs=ZkX801bKL}aOw;Lwak zrj8s-S4yv_CD$S6Nnh$0?-cD^h#13?08a{qAZe`kl;$}4*+AwkzQbzr%5ofezuLg- z3ndM^bhU2Ny(*=M{ijAoJFfw;83S#owXDxhLUdPGJdcj2qrP)61uXhJ;nm4F?kc6u zH$r6s_BW`Gt^*gW=+2JjL9pP?;&WzFU2pz8d!W9aOb03N#N-l->BA&^H~OL!kX!$m zR(8c6*25i}@6N)x!4ogv?`p~J_x9idBQa9!pkR%r-8LqTfKo1=mge=t@#ORRCL@~Z zl}BXV6>FxJL45TwF9*Yi{w&M*9jTi0rGF(9#JAjns;|6`@lkjo5p)hf!;rL$f;vB9 z;z(J9-+!w(LU0|%R;v>odl{F!+*|{BC`U*b;m-Bn^K~{Sa_Tmjt>QQ1oE;weIJ_I) z%q=n9kAL2cKbKGrx#o-~8FBo|f%L{x#dWt9)?#8Gn8mpa&yJ)+y?)8}5Zy@SG-~kQ za5dm`vap1xdhTxg-SbG8+#BdBs$8q56L#~Q`U2({bhr9Ue&piOx>FFg{n%y0QW+^!{5b4DTmN!kcoroqVwQz z>Yq+A;P(?Fec#(ct(J_->)2SZpb|?J(yFi8MIIESN$u}h;hBX;i$i&u1q#^%nKiAN zLAG9|^+@-%uPx_y8m>%tE61Z?&5TtZ{h4{~MZcqZ%GVq0_Jz{oZw|o`*YZ$m z5-59TNqH#xZ$PFt;45v+Q6y~mJfKn%1V$zYI$1yLZPBtO&e#0PMYN9R7q~Lv7OG4r#8!f<^KR%aeHM4?D4sbI5uX$poV|;r=2kC<7X;FFK_P=YYTK%R>c_5T2}h zDY-H}dH9#+o7DJ+c$9deJbLiznq7%TI!toyhLFSE=^Vbk6vz$tp2zI9hy?i#!910e z)@X3}Lb7%ADd-fI_E6@5F_Z&P9Fo$g=hD1V#htq3%(oxaO^R-Bhg}0g03&ju;3R-r zw%+;tg?(zU*4NK9yMMs%HW`Y8DcD3M-38y<{B}z)^Nid3W91C+dOSTJ+}Rxbpg$N^ zloIHDAk9w3`RQ=DKKQWf^GG6qe0_x%TjLtc*@wB%zS2iqj}}`rxLG`P5fr}MF6B7+ zSkmSzmnSI5)LET5dgRjnzI@jC=eFAhnC|!R-Z^)APMhNkOGk;&yqD3OXRx6_x0Yi# zlWfSZO-Yc_@(ZxuH%d!Y5HB8O0_GJDnKp-djvpgww>dl#*pc`P5Mh~a0EWOn{*;fW zz%-IHrFf2w~%`Sewwns zXx-(Z$nXgEW;$h4PF!71Nv+&Px!X&nHSNEN5u1RjVgVq8n4ks9-O)J5VC#-}*++!$ zV|H>6C`ANH>t~J%SZg10N-p9t@_DnJu{Fe=_xZH>AG_$+cXyJN;p&U1`)AM}bUsnZ zxGFCG{^uxi)Lt)lg$BoDkrS zCJbXv`I?N!WZAOK!P>TdILvb|&l@qLi{hZWPZC4ag?JQ8p$RFBB=R?5wcZLiR2^MzK5O?N7eG z)B7Q4hVtk1>!}`aCF_xoKZ>y#frmM<9oQWotAUu4W||HKPfH^{M02X4u}x6C*2|I@ zXu)BB{bETl7jDECctWgt&hwKk&24TimeRRD8KU;W^M(uT-;4_UgGZ}-Lqhq_&u6dnAgG}FDQt#*R(>KnOR3b zu2W9t4eg7g2%OWp*i{Zh)r=%Nfl|9(z^mqzKTv*dM{jQv&R%9j8eOjW`A|x{+=1Mh z^y00)LPfvY!(Q}DA-qRK6>;LytFM;sY+0sV=BQ?X0CAU7I`zDG+QpssUc!ZNzo7_z z5^@_fN!yB>E$L7jSV#84`F!!V)@C(Lp2X_JK-(BJM7@dlsFKceHl&o!qbl~JlYSG3 zK+Asv#D*<^wG*;n0-%Xc?^K{F6Ks_bvtkE_CUkb(6}sN&iaAxbpqoZSc|q z{U6qcfp<6lw2!39fHLm6%Bm{7K@^A;x-S(D*`i`?(FB!RsX!`HxNPq>QyuWTRGQs2 z)%9jLu;}55tU3=mvCJ{;GQ9AM5^%Avo$3h^9(P1sq-^xPhlwGY%Do?;Zv8sm5my80AsUsRg;d@}v@=PVyV zp+MWO^oZ0c7SXKvCFeIhRX-BCt3+3Wfs`V+G_`rkzjh0y!GtSCx-krp2E6yS6n1D& zn6l*%Ypx4n?UJOw&rT>|ff(TP^u~ZmY)q*K6o#v(VKH40ytqF684tH+K+6`PL9YK* zaxnzh4#sezERkF0yYt1+=L@U2z@3D_J0xE^fsb7Lu zjiOJ->MK6+@yh=6-Jjb%fHw1iXRF$S_7k~Ule@S)u09Ot#n^pQW*pO`_;e=`aDo>) zUS?i1W%c5d2Jty9+JPTb-xhtk#f&yf!l$w+Vnw>b=II)jg!2rUer0a~;7Ol$txODy z9{Rxtd*=_pG)ZNl@l1r0Kg44kwb)BGbNzoEYTqd9pQUCw!A5ddfpwSs(I@|M<$a*oUdGQSYDW8$~?O4?Hkk5+zTte%NynCtE1m)YXsNmrayYF><)! zm{Uh&W``l=C#`*$*Q7+5tVC%cH{Y1#1BQCy0XWTnRlQyPi?bQC#Jw6*8lO)-2WrY8 zup0fB(7TQY!99b|5BQZdr|F*-)t^cQ()U|Y82ar^jBkv>*HxE+l1qt?W$EzI7>~<7 zKryL(d-~u$@wrIF4%-iWp6-C$WcfW^R@lIEG>?9Ny7-;|D~?x;01-w`X~vzR7S|0a zJ30Bz$r6tIhs8|P2m1-mKE*hoc<(-eZ3iJ&h!S0Qo?YvT8mQv{wgjBKAV0i9a$ zJzXTYop%*LJRwpilGlNz@zy7Z0mpRpTUNHm`t989xGj1sp%MD>O{4*6gP!yTFMCtK z9-)~SsQdc8|KQGlyg5>VP{D1|=$$_dTBaE@$xDe|b=6JW6wbT7y*~F*sgl5sBwvtl z^^Ere)<{GYGOVv_;3*K@tKg7&LLg29Od`HRnzX{KTPRl>g_zejTjLxqMTZK0W_1S^ z``bP7pbyVE`HVOxC<-@u1wots82FIqrU4zheU=oz9$@(W-r#43p?NcElrgXH><`pf0GwOb{OECHAHn%-jRS0QLA>vW z-zc+3%OUv$P#)d3`)eP+c)!;)YbP2tQ?`2QaklsJynk!4`Ee1b)i3S~?c4fySLMG>eR*wOqqaK(1cp$Es0kI67ZaFw4NTDAl+BQl=!d<19?CQC=((|~;M5)S0h z0-Sf${Hmg`U6)K!$C8TkTG}9JEH$KZfmVYv-IhFjC&8ct3R_8J;3fpq9GF;H^Naca z!h%(8$N%tQxcz*5(W&u=Jt{q&&zF=*#`$kWAE@v)?TW_E#P~m1P_eD5L~4BdVp;zl zhlahI{$^`c({r1p<*1-dQOf|p8+!=8djScQFW<5={CVC@x%Rrz|0j(bM=E|jgWYfh z7g>Im$zFM0XCst0m*<(O76Wfke&Umn_$j+;cx<=C>v1tB`z|nOVRXAZu*|Hnya25U z&`JnE8XYW^!2|=q?|MzKgR&ssoc^7pplGLr7Vfus@Tc~&xP-Zq8iP>@tus|z%<0P4 zW&YCg^#zhmFHgY#JXqH6pxI2bYB8D!T7Gr$dmYY#endeSq>Q0tuY^I_ic>&nlpTb& z$H2N2n?(W%$-7A2jEq2bdFBS=2J$PZs58bnXuV+sf~ei;Qt1pCH=vfjO~qjUu?360KW|p1%NP)btgcM)5|Ia!?x;>RNoEz34i0p?I)DlowF(7?g&l9ljRucu z1%)dAR3@`(44T}H^`L>uAw2N@9F1JFNWas&OkO1W3P$^3(bkdp2kDx3klec`_y`sP zKr%)p-R<_4Em*V!UU!?W3hZ;G$-bfFC$jV%QrPd8|d z1KiY^dL0M~<3@iJo-_w;RrB*TOQx1xEN03nteB(nyIfC4?>>}UEz|5mxNa=(Xd6fD~%4HU{YEi{K(5}kIzO>h5_#>aQ&0*r$}W( zWFV;N$BymJ!j5S`deA3j+dP^sKiiVgjuz2Y*1EW|TKoM4{^?LOhQIF7XBHf_aJ1d_b?We(Xz_ zg#Es0#(IrV(@RI+ilGF~39vMoXDo2!O#Jr}rb|1(P2!#OtfcyT=WGl;Tc#nEfBe*OL9gi4?0fcyvrSh`>>1mzZ zCV|QA);euVnPuoM6D{+@lnM>YBj5551n+kBO7_PHxk5ceHG^{T!%; zEZCkZiEqmucvRtZZn-;IRm8gqe&Q~-7LXi6>@)Z0Jt-4_p1hg)cvF;Xz9(1?lfRs> zGV1>P3K2uNg`(sMLXY{Vo~#)k`G_!c=@OwW@YQ)yIY^fxkUAh-ygN~oVcNvMhu<$_ zXdo8QqNWV*;UJ^~bb5u{?{3%J&wqHjcVtW1++Yg!%n^lwHygLyZyN($0Q5v3?Ll=` zq)gV3~Zfqw>C4d&J|-M#9!7{3p^ovYhLN z;1xXFlctdBOj z{nZj0eGpJthn>SDB@wPd{o&=DZWx7}g&c*P`Oi?5y+2v;WL3Uq=Bwan*#6pF_s;@I z(zPzKzTg~3(r!s+3OkM8@DRPdK8^{g-0nI$;fk~{RsgR8A~jQW#rMP=v;2|O&>B21 z@s3RY_WJUYF02 zjTJ(+q;l)cfcIL00nU(=gldM}n_bDDLHDY;03$;{gyibni~JFdWG9}|3(B)M$iJcm zGY0orwH{Vbk~51Fo^B~-+T;@GbVt_O%s@)3!f7)@WTflY=)R6r7g0>f%WEh??{fqVc=Qc#^|w(bvf=M zLVQvbF^($v!xs0B7Lbh3&elK^d7PABy(Annh8i50EE<7_#o1?{%`6?Tkjq0MBYGf> zH>-m87|w832v)at?-z}!VD6T9Bd|~TKUjd=K3S>gE4Bw@M)<`6S7E|XW1RetWLfp{ zi|HGnZYp@$Z4Yj65SBkS84?d)Ek*=P*o~na0r@j-$HQ)mIcjH`Nl3#qS9j|0zMn{` z6AZ6QDbjCB#)%zX%*y6pj(6{G0Ockxn2c%!iVCxkQ6$RATyVT%4f$by8aXN&G1 zk*$+GMlqvxF*PBdee-xnEhxN^y}Xdgcbyt+JO& zzaO3RAI`oyei^Y2&ZB6yW?MHddwEEhpJo*_WhGPpBBUZ9$LxzL6qkmK^F!lqicGw~ z4@Llqw4xQ8zD6t8+D<$1EInez^DoAohv-k(<4YgLN6q$>@rcN-MOzZ@2Sq6~5e zK3A;G)Vf6Xy<6)UEWp>Mrg2=z4QWpVNAcM0`sjO)VV>U4%N`=Xmx}&rd#urNu1?z` z`BX>ri(Q|DHAD7{-inTh*WvJop&PmTg$y21$#d>ePI~?K9`P>jlC>Rw=rDb^S_r8* z+1MZfZW@iF*Rc`QYMWb(WSZfAmxzw+s`ELg+bPq)+}oDjYk6`q^nV@of%r(jN-kJQ zSzXTL&F$XYXB`z`xd-OLLzmKiS-;Wx{BR%CU)*(!UNYnsU?={$xz)F$UkV0;+E$w= z6;=;Jrv@w<1Q1~^d4}*yd^(Zn{be^l=DwT*G5JQKlA!zL_`I4=k<+_kZ@e5th+mf? zIfD9Z=mrn;``ZQ^lW7K^_3Q8$qV z3na#aU@rN>$@^@AwHXZ}7Xs@6eg%-!DD|_%MPjk)){8-!`Sjjw@Pgr=hM +lRac zMrUMla(rt2(jRu~Iytkbq18eTa~W}K^F{jM%U<^j z7gsr@w0OkFcxO?T?KbD}U~EPf2M57i99=4W%X98R?lvGDj8Q~~+K~8T7>+^ZtEG#@ zA?bDrPY21w=^IHo4k`UdLEEndJ3yg=`_1QflONPaYC_(T@3ESY+&AT}AwUI6>GwvhhAtStIn+O|; zvu3OMDcSkX0Drzs&PyYcrjT2w8xtOEmAL0Rl?RXL>(+a`F~o-zpYDqPZe}4{L1T{y zYeySJd~B@ba4=>8>u%I9H*YkrrAvrQ`o(?CXAAXZ*{zQDj#J%Z*+pEq%Z-V~-*P*Q zIGlZyAk10~xsxDa)L>RQOt^gIxdcPBP{ zB_AsPQ`L6FgFw!ll#yejB|j(5rrqM)+zp;^1V%=^Pa;Prr*pZgGuuSD55In= z?tf?C9cj8k!ngs>8*H-wXM3B;^A=}HD! z^Otwv4LzzmO>36RXj0>}zKBTE?}s{!eNd<1N1$Z>7MTkbQE{#w+ZP)f8yV#)Btgl) zn6g80u{|XFt+USFmf^*O{p(ne!MGV7@guy`Rr$iBk}Fu^KKawjq^xL+Nvt_z9sseM z>i7A&@U;0E^SF5CoJyAn-uyP3Av10R6Q_C9?Mh2Nb#NQn@;Lg%Yv$}l>vyT2AyVO$ z_NNAg{IZ|pT+iFsm2f6z-?VyI3UlV!ttWT=lb9nn%oEakzBnSi@kg6}$%=j@DnD5C z*DpREMMY~?PtT(v!l>+2zUJv>oi|^7=^>6LHU676dlp`$yEl6>r1X{XhZzfxM#@ze zTJ&p!ra>N+UVSoUx}o;`oz6QF^GW3t87OaFQ?YNXJ$m@kC2ti$B1!w9@8M-k=*?l#xn z<{nHh2BivX%`*4$EY#lG%^QjWfu$EQF#$>COi|!DlNc9Vp>@l1sI>_i&0p5`dVigS z+4wHHHS^0&{@sS`VMY^w*9#P`tQuSG#|B24McKp5E#$syTx(Bc>gxOg!A`;uf zkl;K*rbsjb4DVn-W>S1wj61?!AnH?}enCwMH9o!v4e2`+%eI~dEOL@JH6vwVV|q!% zcn+7vBoTkLHyyIrjL$b=RL&LtWg1Nly_ax*2qi=X6yV)3Fug!y7y3S#%=q5pt8ERu z5uLrAmtMjiomf;fmX)d#-YgdP$OsxYWIR)o!OQ2dg=?IIC92b!aP9NA1XRC^^m|sY z&ok4J14UNpO>D=y-e(@!{`NwS6fF!ZE2~3^$sRo0m3`2;gscG}LjgYr?zi(db6LJj zl+5;{Vt(_jkAa5oGq#c(eUDGsI|ojt4gD~Z>jhJdZpA1GP*ueG=W&&d=Et5ZH|%U;Msf`hSpp}>iB3}H~Ex-0{5#50M2R`z3uptFI-bl-VM(MNsFEALM@<Aao&34R(E(A5p$5MA}XSzllPNb1zi}6EF}`9F&w$a7<628OH*_V zndIe9;`H;``xA;rTj})9U1F_`W~pm9@k5^-hoHNudL%s{;ww z*w~RXs7Zlqwq6;>gr%l?vJr>ZFKyX+r9Y2#6j~A?y1_|0rnSDqkG=F4Ypxlz>3WV` zH2Q8DXh@SwKkFz?&2-CmLKE5O9n+=lWTNZ~{RVEwY)(a>RB~$equDlE-{`KZ%Lx9s zWr#-P!)5G7qSzkGP*geS)Ik?=JRV-}y#226Z*2{<`&AO!Qq$IcyPtBD!ZwlrFTc|z zd~uXq)?wRCR%AjtOP*6wxY@Y=G>M0+{3`*asD+j5+x_~pk}cQ{6{PjA1VNeRcV==j z2t?Id5swtBGlZZ#hz;KB8t^~}huuP?p{cF*%M?Y4uq*s0FJuuNls1ra!NJWA z+B2ni>&?ya-Qhi(f_6ZU9sj^=4W~U7@f+3Ak&&LjdHHIn&WWw{5qIy-r)YE)E|mF$ zWqQI719Mw6gPmAv-r6O_WmUVPS~S?02Q)TPTm)QHYRcg%d{~rr4n{H7EVCrW3MPUU zmX}O&57WQKFZpn@#FzW}@!xp#UzjBL(EIXd_*gX-RPy`N9+%{lnv~(AJZJ(2KDN8C z>e|+*{byVKcQ?i11$tA2&)S45FjeUPT}Js>W&oQ&`pGT9xVn#EQL1J#?Q^JA-Qxy| zNa*pVU^-F}^PZA{MNK^*GnS5TksHgCD{Uw9!~Xo}4LKRo1w+F4sN%p@Y5LXc(n1#? z{km&6Lz3h;U6J>uQ`Pn{&~5+EBlxfO@KVJyUEr^n(P4F&9WIp2u96Qh>fPCJbN6Q) z*}EE}S*p#+{zSs&{|H#<|KE?Jx;({@o+r$Sm2HLmi7SmM! zF+cz7M~Y-*og*|N6wg&FplH0JV@P$-{_OO%W@IW4Q#` zZYl?$0EY5YgH*nJ0)F|=N$Fqbd2WEV#}-4C0xw)}nzYx2EsTRGwY<-e;vFt4~J~07JPA0>`!Xk?DFOrD+ zKX0l%k4pj(JhQzg#G`n<4V2*M7CEgO?LiCAa4J zLFI^M?)xu7V-}3dn?Yzuw+H)fs-R`*{QP`L?^05`^)rLow)}i_8ylMlBk&uyLq#() zSqzMu1e;K47jJL05|{lqjvnk{oMUsMCWILh5;bf>V=PJ}BKr-3dJedzraA6Xz<^zb zb0TSkC(bKS`q}s#FWNX0t zS?Tp_uJ6Ug`)p*uMtN;*ZFeb9+q^TPuJGoS$AChg?Oy`0!D(priy7f*yiL zK%n?am5jZU$#T-e?iD0+j)qa%Fs=~lp74M?aXAkzXY>7gmdBAZy@*CP>mx|#^TXT5 zafwrJ=4op!t=JNqQmZ?JKyqOrC*^Uy>!Y_uCaZJn4t!L?Wq?MHQ+ffxwAi%Ut5=1$ zXlwpHK0dxWYHE?esqf$0o*IQM2W)OOYjARN&s{oqtpl{uLp}7N(Yda+uCA_q(ouxC z#Zr|VXvt@ETU+A0_Wv58*2{q2laPC_GXLKQhY{;a2b_n3M+N)g)SIz^%Jm3H0GW_> zz~*?!y59AV3?=Cs6TZifJMV63~rn?J|Vtyj{eppiatX7DFoRg}1}kJiVso6Cor(fxS1wgoCe zzR&n*MsziT%=`K0A)z%t(B~CKyUG(ZSGDp#1?C&d$(9iawua!4mOO~uPN5`Ut8d9~ z1cYQ!#a6e2-EA>307B~_3FvC;6P->`5_nB9xca)Z=;T|p7{T126OFH21jJNKZ_EofG5k3GTE5Z&_Tls*eus-L0QFJ& zw`!~);ArZrTlTW5*Hbkf+DFEM)IweBisVruJwWh!#{dZEQ=Dok=w7Pj&$;D?FG z5YgI@95XdNv_74TlKsI0GefiKxA|chH@9TW8$uBZ3g;0lTak)NVQ{SR@eVxvFXIcO zB_(DJdkj0M@rw4a!)Gx?+vjd^uzCCt6`f=Ql{Z`wXbb{ zciFwmWK5!T7%V?Tz6M?82&YSnH&nTyJ9{@KMysx_{?zyd`?_*d%Rl(n`_n)IsUX5r$_|M>;rh{rjw zVq(#owIn-Fuj8$5Fskf_JF}&|hCX$yqZE}Oo!x?=m)gM8lMVq&SEn2Pib|+RTFZ7! zQcxIR(g1w2w>!279x#Z7QF>d}cJ~0Vd++X!rPLkEGf*^qzuFtvI~scDbzFF{tsfeP zO-M5{g9>Q&W0djkKcH2(L6GWxux`5ea~OIb*c7UhA<{Fag&tlIQFczWWSlVauSMs7 z*4Hh43;}@G!63A$L0pb6OqeCyLC-Qe^jj_HNu?|SvDjzOAWqzl1QA*sN8 zjQh(K55DY_s7Y-NR+;ajH`Ih){2!i++1?|5)+d`qRt~CDQhm{XB4&4+Z^uQ`7^8lD zqb-g{?2%s7U5LV`A=nPm!zN+oSM$)yS(nZWL(Zt!35#+3YUVrJz7yX9bZG=yu-oDLJSri=_Qg3;C0rz-6@%U zL!QRO+N}zEJ*lL>(=HWTt|~iv2dA*X=;o0XTsgdZE29pJ>p=Z}eNmQ{wAw+`#6^aNjYffAC<=eJMOYU5@{<5f>jjj?n*kZ>r7bLEZ<3 z=5t}W@}k2dujQ^uEYxj0_}SSRO~fQItBOA7)eJkHS~bjBD{EOQB}NXIwm4%QwD1ty z|w6Rv<1AVfz& zcFLHM(Z*K6lg-p3qM{x^rKA}!QR4;)osi4X()n&z^f9ZLnr4Q`4tX^scdMr8-x}XV6z#R8{-!+wZXiVyd3B;>4-AB#c5T zHmWaimmCTepgsoGML1g#;?a_3u$f9m;Mb#^I)xDN*Xot53MSrwK@z6r-z{ z_J{S-)!?h>?FtXk?iOMXxh>FC$oA!)B^WPBN3%F*#{_;UGGA&Km3aeQs%S!!J)^Rn zK)xv~Qd?fj5aH~Nwi$2sZobzd@jUvjqe+*^97)*lv9>sLwaaeU$@*v0<`Aa!uX?Nr z@#=Iu)U>{S1tZQ$rL|scLzR(Bo&3BAwAiABFJCVxEDS?1hU zyf!#BJ~%qA1h%T)$0jTfBnvp%__;7xg5Jm1O7OHcg*5x~GRj!Sk;)Hrs1%Frn;wRE zqley3P!)1UueQIV*jqG)9+t91L+cH0&MrS~I11b7gx4G*QAVJICiTBwGVlrdn6uHB2b6kC^!4&T5@o`Vyn^eCc ze^F-Xvird)n;pFeFUn@F*}n>(;pCZh5`KV$5EjF112mdc{sQf9*LOR|Dh-WeB_dSI z5AQP~qL*-{6A=*stW>TIb^#xj|CtQ``-E-+aK3OfAkBU7H+UlbulTfY2Z&G0GScpw zKQBJ>W4z`5`ne;;W63e7u)m#Yno96Z_p1i8$Y>H^gFMJ!7+Q<5pXf5J ziW>?giC6DW#-n5ncuYiHy{n&w962MrU{Oeq`$kDw#v}fvPDRO(`0hB;j#~`W9Pq&( z%lTex`g>-mer8S|BaXhj1hdMo0^~XzgV6D0wdf10VA~vsk~#Pd5jRY~+CaIi*=MR5 z?`8~QyeJ?2{G7OJ<&r*iC=&t13pf02sxB_^{;Yd*f%x~k8IJbm4uNli99md{GVNTf zzj8g;-qz|MQwf!M=4_MRT4l@Gn-JHdVYQZlc9iq{)83U}n-7hbeW6MH!v}YcNUNrx z!bs68g*Lf2${3L`GwEb|v&IRijEaqqiil<0EN+`YD3C~-TxpZc?fz;rK6R$wHYB;o zz(ODY<%`4(DpAyT;rr({#b%MbK&6pIMky`_i~ zm`0on=63tNNK}1l$FO+&kRg8Ki&^FNkL<^k${|N5z_wX^CGmXJjD4q%g+6+m!`6M} zs^7Xb|u{z7sUhT&X=rS~eoiy?5l z)!xLedABx~MeX7jB_gEUafCyJ9i2CP`U%H)S zP}r;Qg&ZY1dA&GR^EeTBU5O1;4`O~p_X_0ufZs8>FuJqld=Y3hZqbKEVtI9#Dn(;L z(mp%SuFp4fLtSOl=|cuykF7;SMm^~b+8lWkpgEKyJDmW-9A1T}o1LBdeoU(M6{DFb z*vVqAp7h?(L6YLDPE>QM>LgU%AAYN@u3hbqH3FUQJObT~(eb5cxw4}Xsxaipn}H9Ua_ zf5-`E;ySq$xS2}OSvrvgB;VmF+NEhZB*x5CGlfHE6!{)m;(06}hQ-oZ8 zOmus4R?A1pKBXWu2gRN4%u7cOW;r16PO6YvxrdM2e_^W#1qKk1F!A;fmgV;Debee^ zB#-azHb`KmkB80}P+64LxIazldV+$#oPaLDl^9Y$Ih^*OO5BL92iEQpJ) zbGs}T{??LsNE&sWrk_q`WMm3+$=$Ra>Zj{AKgL0vZmB6j5G^o)ywZGPpo>RO6bey_ zfccY{V7qIOUHpcYjA){o6T$l7aAH9tCp(Lv*5Fr z$I#`*R~H;18i!b=1HkKD?8XQX5^})o67q1dZ1l#zN3q=7V61eJDkK2b%<1**G8sYe zQxPpOl~Thuc@CH=Ji-eUE&S%pq*Tmz+qrk!0bf4@3T)}t6~6nX#{t&CCu7SjR$W#8 zUFv=ROm_VDIT0g9oe_9h-MDrBmu8>xVNh%C-ekX0!LzMAW&6Gcv1DjTHLS;U{WOyN~bL_(H~`{sxmo<0Y908~Sl>)u2}1 zrKBPuC1J$r`G^_DhQ`z@Q)UeAA9r`FlrFULYO!u4RERQowk;0Ico7uwQ9H_Rb|v3_ zt7uF~AM3$L25Gn@j%=HN5CRhAZ`{uxr89SBD6z9+I)rq)WSW!Jan3`(i~oat7W~Av z9y>c=BUVdOJF)$sMUC5J%nJf15&QMKEBthWc52yAkfkr1{#$*hjYFWxL;0W!e*yU= zi?BBTfH;3DD)a9>gl2kd;#{4|PYj*G+o!+rkmr(zal)T$k^OVyG_A4gMjj>j)G56@ zrX+*}_(KDVFr1BPu>Eh&?w+XS6ej-*XFqf}_*HgfJ^MT{X+p=$kO>3qEQw^7wS0S5HIV8WXulLR57E`1e>!BE1ql z`9*HX)l}nfm+tS*hv&ocQ_|_~nwD-$Z-uT`S?=Fh>QHe!Br0cyckKq9%{pxKbhCIc z;x6(=kub&mHKiAVMA&4cV}^%!E8MLrWY$IUPW(CyIhQ^DUI5l2@7@ShbTIPx-#Ig& z!qz^FM=I9pacpXAW@K!1)7so@-r1ES8$2OOUjJX|IwDONZj{@1AQ|Mr#>@M2rF99I zWVwmP9`@HCQaz~W&4iN@)hTmu!VsnQ2jTV?)KQiz#CK~0DI4_T;uG(kULeuCyaJ9M z*E>ZfK-F+<%*5H)Zv?G`lSr9t4sc(Ax(Xol^{K?5&MsJL;WU%Rwv_0 z@pzsWW(AH>iZZ@C1P4}PRLAz8ZI%!}H^_xUeZTLw)Vc_KSzr;^wl-^IWP#Zk6>(~&ff9lIhv2}_8Mc=Aax-hC}`>u``g<}ak2*Tz(6 z;a4Y$RW>j{J0kdw+TBdbNJr*;{G_9UI45#ZwYwh;5$Q^}v>=`UZIYH{cV}-unhF7& zyyoUyE%60geII0S6bQR`zUT6?55WtN;)AEzH1NOl%LL#CSl`-6xf)$@VJEVj%leS&-2=eh686zOdOF)Xm zxq!LleSOIGKzrf%Nm5>adGNI_C5<|a*(G*W;r8SyB@Nw|*j;%3)K*5j``eQ!+x)Nn zqx9^KQ$Jq5P=I}|qm+AeL9d1r_r6)dAYm(W6#}QC5&?K9=X3$;!VThwg#Skwz)t{@ zMb3NyAJFxpU_My5%!kK@;oWedyG7!{rI$g>^${82n1HcD)E8PzIqEkb3TG5Qt+WPT z^Su*OBZwmrdm@q=l$dBgu9`*oOBDFAv?y)6NcZnnwZ3wuviLp z$#$^`OP>ySt(yejyX$17$1L2urR3hC?s!kd66fQjVdjccRQ2aQPg@N?p;8js-&nKe zr-sQvWNodl6VI{s2Xhx8L#S3oMtz1&E)Y)+AFs@87t;(n?i6ws!(vO?7M{iwG3SOM zRMM+&%wF_x`J@hIAl_{ZKdH$k=X+)N{P(MZp6Tm2zKIVU{VczExGa7OlD{ozzuY5e zzS(%m*%NK^yU{~am!!=ybbAu#IHsTfs{{ZLnH%q|IC(2)8LIFTi=y&VfAUA;NFOJ4 zyY8`v+6gTVR^;E`W&*3VWQD_UB{?1;VRxwlb;0M)f39@$zc@L%;d**OHsP2?Pa(Ql zVTFZ@|F5p|ifU^6);L0>3xuMe2#80DfYPLfB1n-UMHGs4w~5e(s@^X0GGq7DVr>nc`uvk@K+_0a*5W zQMR6ukv&R~=VFWEL9UCHxjZ^M9~Vd>(Pm|WwTLH!G}z z>$19ukKO&s1wk6u!T9VefZPw3TRv1607K6(s3zIHPAtRf=HebSX`aV?28-E|T5+bLWx zp+U4V(^$~aF~T5@`b}qf(DwP=@gsDL%M4Tdg&0DvdsD-Fp!ibLYMvB&ZmdyB@~=<* z!&iE*4B1QHV~8eMcrAv#>~H^pyFcV+A0Yd_>ia-<7z%D^WK{dBkt}(miCbs;4lD!j zsg9D4Hra|0>hs;q<5iPHG`jk4sSuN$u~qI=><{ai|0`C+FtgvFS6Cs+askaaa2PeQ5&W~;fOv#w(Y9%maTND@s zT5GA19&A<1M+jM-IaQ#lJSGmBi@QaF5W3p?PsG4n_OHolD8)ohHzTNh9Qa87< z^i<`k4ZLXj(WXI+2dD(LsF$r0*V^lU?IDT7q_j?qU4j2@M7Sw!Kg`EqIkaHl#Ww=l zTpDg@mwzQ;^-v$bt4&Iq_%{R{i3ZS$Dje=$3&(cq z6imWKaamFV6B}}DJpKrVopN<^>&ViTUb6>hRUgbTh^ncb4hS&WU#$-E@$r$~d_Cx8 z3M?8hK`0ced7&5!q=hql>UZoWEi69AXU)>_aDdfU##7h4Nb6!nzO+lQVfFIVi7Rz7 zDuE@uKr4M}jdsa~J$ZUt{1P9&q@-5BO~GwFY&z|YYofC9Nmd~oZoT|fU`gGRZ3Viy zId%cf)?iICv(ZZF=nAJv-HIa zpgUFS1_su$uZOW>TZ1NCgfj-b24m3wABP-fYz~_2rm$raMoi3P>myaOw zNUn4Hg=~L+fBldUr5gom2`ei;Hkn=SD3yaO$yC-t; zi%Lp5A*?zQ5)$q2gO;nEJLpX?uy3ujdLXXLw2Uo%3TvVM^8Twk32;}gNv*-ISk>?R z3Ktoj);@rKZkX6P5IpT>Vkag}zC#9!$itW|9(E8Vwb4aQF9%>3xEH!UuA(tKRi7rl zWHdFM1dyf#_syT4{;^gxwvE}q%&`NE&UsM4`}y6dcue4YHv2Q%hq9ILDHmxedW6~l zpUG&agFu2)Au=-BddFuGbfcr~SNcvjj6DZHG&B^qOC39HWvw2t<<5f%6%%4Rr7!Z= z;^a=4`f%{GKYan(hg_fo(IEnV3lMt7%VDb zt_RJ)R2{IO|NO;^+=i~^jn*_dw_fL!@qdVXHQ@Law)fp`K+l$W#=R;bDcJ#1Hd2}T9_$i3X(Fll7OHU+Y?>ZCv-xGuw!F$uKc-DM<5^2A^&RuayhrIE-->{Dp?KJi zAOpyH6cy`S7pwVSx4;`i4cD|RW8KQCidu_L&}8^yRTV5v+DW`j&-#ZdI$OK4Eo|)5 zUfwNB2){FQ(2&TwD=b)0WotZ3I9a0_o9o;k#>+t2x8l>p+!-@z35m!?MXZH2-an5NX8 z?N2y`SPaKhvstD=4#~Dk;iMe?CAVpr@&+~Q2;oNe$&K7o)#k87f;a)N2g9wI^4(nGE6TbNt6 zShF6{zjos6?xTTIm;6&Ms5c9=?R!Wn-AnGl(h>(3ijl!;I)4ZvjO@Uo#Bf()oC`#+ zt2)jjY@CZ9WsTSyx<`!8DBaWc@wvFQwMF^8w~^o8F35T8*s=cpe%UuntnBP<*E){+ zBoig&fTpefC7a8*O0M;25yowr+u>bHI+}&=o)>8sW?Ozfx9xD;(3qvKyM`s_ij06A zK_jXDOUP84mjpkFRH5adqkjh?6zwHFKd)nMW0jWesQy5bho@$%^~PLram|TlnxlNc zT=A_^SzjOdRO1zjQ#1?>3V@=LS{Bf#vOP{p-?c--TcXh_ZMptgCr!9825_#2ku|J9 z`ehsw7q=Zs>F*c0X26)=2hnf_>SwLIhzfsl%6E-%OMG~YG0LXl(n+?HhA95OXa-kq z{$ZTbW8#@iAxANv@&K4~x?(nhy)0>gn17u$vdlTkVqZP`Ol(?LC zL~MdZ2}Rp1m8LaNlXYDyG)!LqL|Vr`Z#{N#Y<(2Uiplu6;=Oh3;;0h_zRB{sGWp`9Cd{rNMqnpWd(Rwkl8+$nG+;hs~i=4Xx*n&b20fYah9uuMDq;d@|wJ@S9NF_E!m*hkU7W zNCO|;ov#KwwCdxoPUxg@WhDAR@Y$Z;q&`=|Yb#k=cXV6W zCrHL;TBi23cm`5TLR<-WBq?*d$y|Fim>`8Fuca8Iu}BPI<;;X}yA8b1)jfb4wv+rW zt%Uz!e7Ml9;EnS{2*U(=t*|AokYM%qS+GB1{rFF#nxv4 z#hVNL`>!LDnG#2%AZOL$C(r@x$YnBGYETpW-Ew3B%}G!6SBcth3{~jv>6rtrjgV5O z_kl0g$CukO1y$)&`AiJgjdi|P#fi;zC}%RI>MaLd+-!?}Ka;+nsj0#Upx>-n(xnAB zh?ziE+Z;XhUW{EjR|_(5-KG2=zpFMmT{@x(35U5vBPh|i^oenQcVK{M5pb!w$}9c)mIh_|2fRCoxFwrt8dD-%WB!B~eH1P$(6sCk=in6l$`=3r~ zV39~Qa>LQ;_U%|vJrt^O3CXV-_`HW$O6%%LI~8zvK$}!mzQdSG(U!X)D@)ODo*N5Y zM3#c4#l^-w_feZaZt`afBps(cU!eY!Uz@OtHw8&l5Q?2O&3AJk`;^SjAOrQR94?=8 zSwx(PE=UqySf8PEKE2s!V-xrHA_Den)+=2Inl||2z8K72x?C=k34`42=7aqtp;QXa z_T%1kp7+(yFVh2Xo8?5shOqx!!Ly+dL16#zDkEN+j&%nS5 zdCM)yOT{xsjEHU&h&#}R3guLE;v?9U)rR4>QC{Bn@6>vprb_-LyP@mnAT{r*%O%L` zEUK+{>+ylNf54qbp2U!3W@HXfyQ{@lXIJ+BMmXK70b)Pv8oyuue0#9D!u zZv!cjd!Le$!k-6C_@X54TB@`5_36p7&%(f^Gs4BH6y@c0PjBD}BikS^H+S;*B4Q6i zkUeoiJ4%!5H0$#+hO5!J^z*}rPYcC0HD{NW{Ji|u=?-!8h~0B)YIIQO?=6rJv6OHB zuU+$hY@D-7z#xh^dxi>zHI5mAj%jp)yjQM+npK+z)U3|pjP@BxXq^xdW=?kXB}oelr?(|J zUB2?!z3W@Gq)JX|iG;-KI*JKc&k>MayQt)p1@0_f;fouw1iJrcZ4v^cZKV4@ICuv5 zPbm+qGmk3Oc6|%xBIUkH3JH;vjCY@5GAcIE0T#%-yvdO%9U*NmL`;Io%F26L^NEU( ze-jh%4?71v`y382bM@*~O~3Z%>+#EB6VgTt_MxGwyUJ-@bB#h5BM>7NN0!=avzNhB zkm79RxZ;yENr|uin@<1xNs-n!%D{EIi!YpT zGIN`LQmMYdjl^#$I#z8&%#9xi;$b0pGI_PQY@r#mN(EvGd9_>QyL3D#Qs@NrCSbEgr5 zxrMD9z^H`y`Oi%~h(5SE1qT05196ygKd>&H?YszS#?GSo9Z%$%W@%Rub z_1^ZojOn_LY}I-4?*($yC=U`zYr{;CaT1B7?a&g)6 zV`UgNH=F>&!+yLFKZ&jE!w~D5!y4J|E|_-T!od)UD-h0d|MDUG18@WU0gC^ESV zL+A-V?P)fgN5FV!JgB+6TE16!uqJ?BM@0r3Qw{}|bb{egYZ4Ij5zZbOa zMXdgs{4_a(by)VG{Ul80gr8T$(T<^|40O{J jRZ{w*$YAF7^YB;;dNNx=}@Vl;e<64!DbL@Wsq*<@y literal 0 HcmV?d00001 From 0b6bf0f4954c5cf7eb99256a258be0d7d2b469c5 Mon Sep 17 00:00:00 2001 From: Sean Mackesey Date: Thu, 19 Dec 2024 10:01:39 -0500 Subject: [PATCH 10/37] [components] Make dagster-dg help messages include parent options (#26590) ## Summary & Motivation This customizes `click` commands and groups so that help messages now include non-help parent options. Before: ``` $ dg generate component --help Usage: dg generate component [OPTIONS] COMPONENT_TYPE COMPONENT_NAME [EXTRA_ARGS]... Generate a scaffold of a Dagster component. ... description ... Options: --json-params TEXT JSON string of component parameters. -h, --help Show this message and exit. ``` After: ``` $ dg generate component --help Usage: dg [OPTIONS] generate component [OPTIONS] COMPONENT_TYPE COMPONENT_NAME [EXTRA_ARGS]... Generate a scaffold of a Dagster component. ... description ... Options: --json-params TEXT JSON string of component parameters. -h, --help Show this message and exit. Options (dg): --builtin-component-lib TEXT Specify a builitin component library to use. --verbose Enable verbose output for debugging. --disable-cache Disable caching of component registry data. --clear-cache Clear the cache before running the command. --cache-dir PATH Specify a directory to use for the cache. ``` ## How I Tested These Changes New unit tests. --- .../dagster-dg/dagster_dg/cli/__init__.py | 2 + .../dagster-dg/dagster_dg/cli/generate.py | 11 +- .../dagster-dg/dagster_dg/cli/info.py | 5 +- .../dagster-dg/dagster_dg/cli/list.py | 9 +- .../libraries/dagster-dg/dagster_dg/utils.py | 120 +++++++++- .../cli_tests/test_custom_help_format.py | 222 ++++++++++++++++++ .../cli_tests/test_integrity.py | 16 ++ 7 files changed, 373 insertions(+), 12 deletions(-) create mode 100644 python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_custom_help_format.py create mode 100644 python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_integrity.py diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py index d8cd4041ec23c..de007a61d537e 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py @@ -7,6 +7,7 @@ from dagster_dg.cli.info import info_cli from dagster_dg.cli.list import list_cli from dagster_dg.config import DgConfig, set_config_on_cli_context +from dagster_dg.utils import DgClickGroup from dagster_dg.version import __version__ @@ -22,6 +23,7 @@ def create_dg_cli(): commands=commands, context_settings={"max_content_width": 120, "help_option_names": ["-h", "--help"]}, invoke_without_command=True, + cls=DgClickGroup, ) @click.option( "--builtin-component-lib", diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py index e8111606e41a5..bb9642105638d 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py @@ -18,14 +18,15 @@ generate_component_type, generate_deployment, ) +from dagster_dg.utils import DgClickCommand, DgClickGroup -@click.group(name="generate") +@click.group(name="generate", cls=DgClickGroup) def generate_cli() -> None: """Commands for generating Dagster components and related entities.""" -@generate_cli.command(name="deployment") +@generate_cli.command(name="deployment", cls=DgClickCommand) @click.argument("path", type=Path) def generate_deployment_command(path: Path) -> None: """Generate a Dagster deployment file structure. @@ -43,7 +44,7 @@ def generate_deployment_command(path: Path) -> None: generate_deployment(path) -@generate_cli.command(name="code-location") +@generate_cli.command(name="code-location", cls=DgClickCommand) @click.argument("name", type=str) @click.option( "--use-editable-dagster", @@ -112,7 +113,7 @@ def generate_code_location_command( generate_code_location(code_location_path, editable_dagster_root) -@generate_cli.command(name="component-type") +@generate_cli.command(name="component-type", cls=DgClickCommand) @click.argument("name", type=str) @click.pass_context def generate_component_type_command(cli_context: click.Context, name: str) -> None: @@ -138,7 +139,7 @@ def generate_component_type_command(cli_context: click.Context, name: str) -> No generate_component_type(context, name) -@generate_cli.command(name="component") +@generate_cli.command(name="component", cls=DgClickCommand) @click.argument( "component_type", type=str, diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/info.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/info.py index f509963606b3d..4152567a74611 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/info.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/info.py @@ -10,9 +10,10 @@ DgContext, is_inside_code_location_directory, ) +from dagster_dg.utils import DgClickCommand, DgClickGroup -@click.group(name="info") +@click.group(name="info", cls=DgClickGroup) def info_cli(): """Commands for listing Dagster components and related entities.""" @@ -21,7 +22,7 @@ def _serialize_json_schema(schema: Mapping[str, Any]) -> str: return json.dumps(schema, indent=4) -@info_cli.command(name="component-type") +@info_cli.command(name="component-type", cls=DgClickCommand) @click.argument("component_type", type=str) @click.option("--description", is_flag=True, default=False) @click.option("--generate-params-schema", is_flag=True, default=False) diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/list.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/list.py index 055c5a827079f..fa8679f3dc728 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/list.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/list.py @@ -10,14 +10,15 @@ is_inside_code_location_directory, is_inside_deployment_directory, ) +from dagster_dg.utils import DgClickCommand, DgClickGroup -@click.group(name="list") +@click.group(name="list", cls=DgClickGroup) def list_cli(): """Commands for listing Dagster components and related entities.""" -@list_cli.command(name="code-locations") +@list_cli.command(name="code-locations", cls=DgClickCommand) @click.pass_context def list_code_locations_command(cli_context: click.Context) -> None: """List code locations in the current deployment.""" @@ -33,7 +34,7 @@ def list_code_locations_command(cli_context: click.Context) -> None: click.echo(code_location) -@list_cli.command(name="component-types") +@list_cli.command(name="component-types", cls=DgClickCommand) @click.pass_context def list_component_types_command(cli_context: click.Context) -> None: """List registered Dagster components in the current code location environment.""" @@ -53,7 +54,7 @@ def list_component_types_command(cli_context: click.Context) -> None: click.echo(f" {component_type.summary}") -@list_cli.command(name="components") +@list_cli.command(name="components", cls=DgClickCommand) @click.pass_context def list_components_command(cli_context: click.Context) -> None: """List Dagster component instances defined in the current code location.""" diff --git a/python_modules/libraries/dagster-dg/dagster_dg/utils.py b/python_modules/libraries/dagster-dg/dagster_dg/utils.py index 57be0cc34e53f..0fc5f980b9518 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/utils.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/utils.py @@ -6,12 +6,26 @@ import sys from fnmatch import fnmatch from pathlib import Path -from typing import TYPE_CHECKING, Any, Final, Iterator, List, Mapping, Optional, Sequence, Union +from typing import ( + TYPE_CHECKING, + Any, + Final, + Iterator, + List, + Mapping, + Optional, + Sequence, + Tuple, + TypeVar, + Union, +) import click import jinja2 +from click.formatting import HelpFormatter from typing_extensions import TypeAlias +from dagster_dg.error import DgError from dagster_dg.version import __version__ as dagster_version # There is some weirdness concerning the availabilty of hashlib.HASH between different Python @@ -211,3 +225,107 @@ def hash_file_metadata(hasher: Hash, path: Union[str, Path]) -> None: hasher.update(str(path).encode()) hasher.update(str(stat.st_mtime).encode()) # Last modified time hasher.update(str(stat.st_size).encode()) # File size + + +T = TypeVar("T") + + +def not_none(value: Optional[T]) -> T: + if value is None: + raise DgError("Expected non-none value.") + return value + + +# ######################## +# ##### CUSTOM CLICK SUBCLASSES +# ######################## + +# Here we subclass click.Command and click.Group to customize the help output. We do this in order +# to show the options for each parent group in the help output of a subcommand. The form of the +# output can be seen in dagster_dg_tests.test_custom_help_format. + +# When rendering options for parent groups, exclude these options since they are not used when +# executing a subcommand. +_EXCLUDE_PARENT_OPTIONS = ["help", "version"] + + +class DgClickHelpMixin: + def __init__(self, *args: Any, **kwargs: Any): + super().__init__(*args, **kwargs) + self._commands: List[str] = [] + + def format_help(self, context: click.Context, formatter: click.HelpFormatter): + """Customizes the help to include hierarchical usage.""" + if not isinstance(self, click.Command): + raise ValueError("This mixin is only intended for use with click.Command instances.") + self.format_usage(context, formatter) + self.format_help_text(context, formatter) + if isinstance(self, click.MultiCommand): + self.format_commands(context, formatter) + self.format_options(context, formatter) + + # Add section for each parent option group + for ctx, cmd in self._walk_parents(context): + cmd.format_options(ctx, formatter, as_parent=True) + + def format_options( + self, ctx: click.Context, formatter: HelpFormatter, as_parent: bool = False + ) -> None: + """Writes all the options into the formatter if they exist. + + If `as_parent` is True, the header will include the command path and the `--help` option + will be excluded. + """ + if not isinstance(self, click.Command): + raise ValueError("This mixin is only intended for use with click.Command instances.") + + params = [ + p + for p in self.get_params(ctx) + if p.name and not (as_parent and p.name in _EXCLUDE_PARENT_OPTIONS) + ] + opts = [rv for p in params if (rv := p.get_help_record(ctx)) is not None] + if as_parent: + opts = [opt for opt in opts if not opt[0].startswith("--help")] + if opts: + header = f"Options ({ctx.command_path})" if as_parent else "Options" + with formatter.section(header): + formatter.write_dl(opts) + + def format_usage(self, context: click.Context, formatter: HelpFormatter) -> None: + if not isinstance(self, click.Command): + raise ValueError("This mixin is only intended for use with click.Command instances.") + arg_pieces = self.collect_usage_pieces(context) + + path_parts: List[str] = [not_none(context.info_name)] + for ctx, cmd in self._walk_parents(context): + if cmd.has_visible_options_as_parent(ctx): + path_parts.append("[OPTIONS]") + path_parts.append(not_none(ctx.info_name)) + path_parts.reverse() + return formatter.write_usage(" ".join(path_parts), " ".join(arg_pieces)) + + def has_visible_options_as_parent(self, ctx: click.Context) -> bool: + """Returns True if the command has options that are not help-related.""" + if not isinstance(self, click.Command): + raise ValueError("This mixin is only intended for use with click.Command instances.") + return any( + p for p in self.get_params(ctx) if (p.name and p.name not in _EXCLUDE_PARENT_OPTIONS) + ) + + def _walk_parents( + self, ctx: click.Context + ) -> Iterator[Tuple[click.Context, "DgClickHelpMixin"]]: + while ctx.parent: + if not isinstance(ctx.parent.command, DgClickHelpMixin): + raise DgError("Parent command must be an instance of DgClickHelpMixin.") + yield ctx.parent, ctx.parent.command + ctx = ctx.parent + + +class DgClickCommand(DgClickHelpMixin, click.Command): + pass + + +class DgClickGroup(DgClickHelpMixin, click.Group): + pass diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_custom_help_format.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_custom_help_format.py new file mode 100644 index 0000000000000..7819f4935d5a0 --- /dev/null +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_custom_help_format.py @@ -0,0 +1,222 @@ +import textwrap + +import click +from click.testing import CliRunner +from dagster_dg.utils import DgClickCommand, DgClickGroup, ensure_dagster_dg_tests_import + +ensure_dagster_dg_tests_import() + +from dagster_dg_tests.utils import assert_runner_result + +# ######################## +# ##### TEST CLI +# ######################## + + +@click.group(name="test", cls=DgClickGroup) +@click.option("--test-opt", type=str, default="test", help="Test option.") +def test_cli(test_opt): + """Test CLI group.""" + pass + + +@test_cli.group(name="sub-test-1", cls=DgClickGroup) +@click.option("--sub-test-1-opt", type=str, default="sub-test-1", help="Sub-test 1 option.") +def sub_test_1(sub_test_1_opt): + """Sub-test 1 group.""" + pass + + +@sub_test_1.command(name="alpha", cls=DgClickCommand) +@click.option("--alpha-opt", type=str, default="alpha", help="Alpha option.") +def alpha(alpha_opt): + """Alpha command.""" + pass + + +@test_cli.group(name="sub-test-2", cls=DgClickGroup) +def sub_test_2(): + """Sub-test 2 group.""" + pass + + +@click.option("--beta-opt", type=str, default="alpha", help="Beta option.") +@sub_test_2.command(name="beta", cls=DgClickCommand) +def beta(beta_opt): + """Beta command.""" + pass + + +@click.option("--delta-opt", type=str, default="delta", help="Delta option.") +@test_cli.command(name="delta", cls=DgClickCommand) +def delta(delta_opt): + """Delta command.""" + pass + + +@test_cli.command(name="gamma", cls=DgClickCommand) +def gamma(gamma_opt): + """Gamma command.""" + pass + + +# ######################## +# ##### TESTS +# ######################## + + +def test_root_group_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] COMMAND [ARGS]... + + Test CLI group. + + Commands: + delta Delta command. + gamma Gamma command. + sub-test-1 Sub-test 1 group. + sub-test-2 Sub-test 2 group. + + Options: + --test-opt TEXT Test option. + --help Show this message and exit. + """).strip() + ) + + +def test_sub_group_with_option_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["sub-test-1", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] sub-test-1 [OPTIONS] COMMAND [ARGS]... + + Sub-test 1 group. + + Commands: + alpha Alpha command. + + Options: + --sub-test-1-opt TEXT Sub-test 1 option. + --help Show this message and exit. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) + + +def test_command_in_sub_group_with_option_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["sub-test-1", "alpha", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] sub-test-1 [OPTIONS] alpha [OPTIONS] + + Alpha command. + + Options: + --alpha-opt TEXT Alpha option. + --help Show this message and exit. + + Options (test sub-test-1): + --sub-test-1-opt TEXT Sub-test 1 option. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) + + +def test_sub_group_with_no_option_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["sub-test-2", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] sub-test-2 [OPTIONS] COMMAND [ARGS]... + + Sub-test 2 group. + + Commands: + beta Beta command. + + Options: + --help Show this message and exit. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) + + +def test_command_in_sub_group_with_no_option_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["sub-test-2", "beta", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] sub-test-2 beta [OPTIONS] + + Beta command. + + Options: + --beta-opt TEXT Beta option. + --help Show this message and exit. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) + + +def test_command_with_option_in_root_group_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["delta", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] delta [OPTIONS] + + Delta command. + + Options: + --delta-opt TEXT Delta option. + --help Show this message and exit. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) + + +def test_command_with_no_option_in_root_group_help_message(): + runner = CliRunner() + result = runner.invoke(test_cli, ["gamma", "--help"]) + assert_runner_result(result) + assert ( + result.output.strip() + == textwrap.dedent(""" + Usage: test [OPTIONS] gamma [OPTIONS] + + Gamma command. + + Options: + --help Show this message and exit. + + Options (test): + --test-opt TEXT Test option. + """).strip() + ) diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_integrity.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_integrity.py new file mode 100644 index 0000000000000..5588c3dec1c10 --- /dev/null +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_integrity.py @@ -0,0 +1,16 @@ +from dagster_dg.cli import cli +from dagster_dg.utils import DgClickCommand, DgClickGroup + + +# Important that all nodes of the command tree inherit from one of our customized click +# Command/Group subclasses to ensure that the help formatting is consistent. +def test_all_commands_custom_subclass(): + def crawl(command): + assert isinstance( + command, (DgClickGroup, DgClickCommand) + ), f"Group is not a DgClickGroup or DgClickCommand: {command}" + if isinstance(command, DgClickGroup): + for command in command.commands.values(): + crawl(command) + + crawl(cli) From 70ec3d451fd4db92455a95c84eb8a6050595e290 Mon Sep 17 00:00:00 2001 From: OwenKephart Date: Thu, 19 Dec 2024 10:08:59 -0500 Subject: [PATCH 11/37] [fix] Allow a backfill to target a selection of assets of which only a subset have backfill policies (#26161) ## Summary & Motivation As title -- we were unnecessarily forking these code paths, which limited a pretty easy to support bit of functionality ## How I Tested These Changes ## Changelog Previously, asset backfills could only target selections of assets in which all assets had a `BackfillPolicy`, or none of them did. Mixed selections are now supported. --- .../automation_tick_evaluation_context.py | 42 +++++------------ .../dagster/_core/execution/asset_backfill.py | 35 ++------------ ...t_asset_backfill_with_backfill_policies.py | 46 +++++++++++++------ 3 files changed, 47 insertions(+), 76 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/automation_tick_evaluation_context.py b/python_modules/dagster/dagster/_core/definitions/automation_tick_evaluation_context.py index c6bdc353e0006..416851a12540e 100644 --- a/python_modules/dagster/dagster/_core/definitions/automation_tick_evaluation_context.py +++ b/python_modules/dagster/dagster/_core/definitions/automation_tick_evaluation_context.py @@ -197,19 +197,6 @@ def evaluate( ] -def _get_mapping_from_asset_partitions( - asset_partitions: AbstractSet[AssetKeyPartitionKey], asset_graph: BaseAssetGraph -) -> _PartitionsDefKeyMapping: - mapping: _PartitionsDefKeyMapping = defaultdict(set) - - for asset_partition in asset_partitions: - mapping[ - asset_graph.get(asset_partition.asset_key).partitions_def, asset_partition.partition_key - ].add(asset_partition.asset_key) - - return mapping - - def _get_mapping_from_entity_subsets( entity_subsets: Iterable[EntitySubset], asset_graph: BaseAssetGraph ) -> _PartitionsDefKeyMapping: @@ -236,18 +223,6 @@ def _get_mapping_from_entity_subsets( return mapping -def build_run_requests_from_asset_partitions( - asset_partitions: AbstractSet[AssetKeyPartitionKey], - asset_graph: BaseAssetGraph, - run_tags: Optional[Mapping[str, str]], -) -> Sequence[RunRequest]: - return _build_run_requests_from_partitions_def_mapping( - _get_mapping_from_asset_partitions(asset_partitions, asset_graph), - asset_graph, - run_tags, - ) - - def _build_backfill_request( entity_subsets: Sequence[EntitySubset[EntityKey]], asset_graph: BaseAssetGraph, @@ -373,9 +348,7 @@ def build_run_requests_with_backfill_policies( asset_graph: BaseAssetGraph, dynamic_partitions_store: DynamicPartitionsStore, ) -> Sequence[RunRequest]: - """If all assets have backfill policies, we should respect them and materialize them according - to their backfill policies. - """ + """Build run requests for a selection of asset partitions based on the associated BackfillPolicies.""" run_requests = [] asset_partition_keys: Mapping[AssetKey, Set[str]] = { @@ -406,9 +379,9 @@ def build_run_requests_with_backfill_policies( asset_check_keys = asset_graph.get_check_keys_for_assets(asset_keys) if partitions_def is None and partition_keys is not None: check.failed("Partition key provided for unpartitioned asset") - if partitions_def is not None and partition_keys is None: + elif partitions_def is not None and partition_keys is None: check.failed("Partition key missing for partitioned asset") - if partitions_def is None and partition_keys is None: + elif partitions_def is None and partition_keys is None: # non partitioned assets will be backfilled in a single run run_requests.append( RunRequest( @@ -417,6 +390,15 @@ def build_run_requests_with_backfill_policies( tags={}, ) ) + elif backfill_policy is None: + # just use the normal single-partition behavior + entity_keys = cast(Set[EntityKey], asset_keys) + mapping: _PartitionsDefKeyMapping = { + (partitions_def, pk): entity_keys for pk in (partition_keys or [None]) + } + run_requests.extend( + _build_run_requests_from_partitions_def_mapping(mapping, asset_graph, run_tags={}) + ) else: run_requests.extend( _build_run_requests_with_backfill_policy( diff --git a/python_modules/dagster/dagster/_core/execution/asset_backfill.py b/python_modules/dagster/dagster/_core/execution/asset_backfill.py index 5d33dc5246b6c..e66b7f08a05cc 100644 --- a/python_modules/dagster/dagster/_core/execution/asset_backfill.py +++ b/python_modules/dagster/dagster/_core/execution/asset_backfill.py @@ -25,7 +25,6 @@ from dagster._core.definitions.asset_graph_subset import AssetGraphSubset from dagster._core.definitions.asset_selection import KeysAssetSelection from dagster._core.definitions.automation_tick_evaluation_context import ( - build_run_requests_from_asset_partitions, build_run_requests_with_backfill_policies, ) from dagster._core.definitions.base_asset_graph import BaseAssetGraph, BaseAssetNode @@ -1533,37 +1532,11 @@ def _format_keys(keys: Iterable[AssetKeyPartitionKey]): f"The following assets were considered for materialization but not requested:\n\n{not_requested_str}" ) - # check if all assets have backfill policies if any of them do, otherwise, raise error - asset_backfill_policies = [ - asset_graph.get(asset_key).backfill_policy - for asset_key in { - asset_partition.asset_key for asset_partition in asset_partitions_to_request - } - ] - all_assets_have_backfill_policies = all( - backfill_policy is not None for backfill_policy in asset_backfill_policies + run_requests = build_run_requests_with_backfill_policies( + asset_partitions=asset_partitions_to_request, + asset_graph=asset_graph, + dynamic_partitions_store=instance_queryer, ) - if all_assets_have_backfill_policies: - run_requests = build_run_requests_with_backfill_policies( - asset_partitions=asset_partitions_to_request, - asset_graph=asset_graph, - dynamic_partitions_store=instance_queryer, - ) - else: - if not all(backfill_policy is None for backfill_policy in asset_backfill_policies): - # if some assets have backfill policies, but not all of them, raise error - raise DagsterBackfillFailedError( - "Either all assets must have backfill policies or none of them must have backfill" - " policies. To backfill these assets together, either add backfill policies to all" - " assets, or remove backfill policies from all assets." - ) - # When any of the assets do not have backfill policies, we fall back to the default behavior of - # backfilling them partition by partition. - run_requests = build_run_requests_from_asset_partitions( - asset_partitions=asset_partitions_to_request, - asset_graph=asset_graph, - run_tags={}, - ) if request_roots: check.invariant( diff --git a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill_with_backfill_policies.py b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill_with_backfill_policies.py index 68a9a2cb714a3..08930878ff2f1 100644 --- a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill_with_backfill_policies.py +++ b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill_with_backfill_policies.py @@ -15,7 +15,6 @@ asset, ) from dagster._core.definitions.partition import StaticPartitionsDefinition -from dagster._core.errors import DagsterBackfillFailedError from dagster._core.execution.asset_backfill import AssetBackfillData, AssetBackfillStatus from dagster._core.instance_for_test import instance_for_test from dagster._core.storage.tags import ( @@ -37,7 +36,7 @@ ) -def test_asset_backfill_not_all_asset_have_backfill_policy(): +def test_asset_backfill_not_all_asset_have_backfill_policy() -> None: @asset(backfill_policy=None) def unpartitioned_upstream_of_partitioned(): return 1 @@ -45,6 +44,7 @@ def unpartitioned_upstream_of_partitioned(): @asset( partitions_def=DailyPartitionsDefinition("2023-01-01"), backfill_policy=BackfillPolicy.single_run(), + deps=[unpartitioned_upstream_of_partitioned], ) def upstream_daily_partitioned_asset(): return 1 @@ -69,19 +69,35 @@ def upstream_daily_partitioned_asset(): backfill_start_timestamp=get_current_timestamp(), ) - with pytest.raises( - DagsterBackfillFailedError, - match=( - "Either all assets must have backfill policies or none of them must have backfill" - " policies" - ), - ): - execute_asset_backfill_iteration_consume_generator( - backfill_id="test_backfill_id", - asset_backfill_data=backfill_data, - asset_graph=asset_graph, - instance=DagsterInstance.ephemeral(), - ) + instance = DagsterInstance.ephemeral() + _, materialized, failed = run_backfill_to_completion( + asset_graph, + assets_by_repo_name, + backfill_data=backfill_data, + fail_asset_partitions=set(), + instance=instance, + ) + + assert len(failed) == 0 + assert {akpk.asset_key for akpk in materialized} == { + unpartitioned_upstream_of_partitioned.key, + upstream_daily_partitioned_asset.key, + } + + runs = instance.get_runs(ascending=True) + + # separate runs for the assets (different partitions_def / backfill policy) + assert len(runs) == 2 + + unpartitioned = runs[0] + assert unpartitioned.tags == {"dagster/backfill": "backfillid_x"} + + partitioned = runs[1] + assert partitioned.tags.keys() == { + "dagster/asset_partition_range_end", + "dagster/asset_partition_range_start", + "dagster/backfill", + } def test_asset_backfill_parent_and_children_have_different_backfill_policy(): From 452d71399b79c6e855b02aee3b7cbea8187ba3ac Mon Sep 17 00:00:00 2001 From: Sean Mackesey Date: Thu, 19 Dec 2024 10:24:29 -0500 Subject: [PATCH 12/37] [components] Expose generation CLI params in dagster-dg (#26478) ## Summary & Motivation Exposes CLI params (if defined) for component types in `dagster-dg`. - `dg generate component` used to take the component type as an argument, but now dynamically generates a subcommand for each component type. - We no longer define `cli` as a static method on components. Instead we just send the schema over the wire to `dagster-dg`, where it is automatically converted into click options (fields with an object type take a JSON param). - Because component types are subcommands, we no longer use the special `-- --key1=val1 --key2=val2 ...` CLI syntax to set params. Instead they are passed as normal params to the subcommand. - All component subcommands can still take `--json-params` instead of individual options. It is an error to pass `--json-params` _and_ individual options. - User-defined parameters populate an isolated namespace under the component-generated subcommand. The only parameters we define in here are `--json-params` and `--help`. If we need to define other common component generation params in the future, we can define them under the `dg generate component` command namespace, where they will not clash with user-defined params. For example: ``` dg generate component --some-option dagster_components.dbt_project ... ``` - You can see the available params with `dg generate component COMPONENT_TYPE --help`. For example: ``` $ dg generate component dagster_components.dbt_project --help Usage: dg [OPTIONS] generate component dagster_components.dbt_project [OPTIONS] COMPONENT_NAME Options: --json-params TEXT JSON string of component parameters. --init BOOLEAN init --project-path TEXT project_path -h, --help Show this message and exit. Options (dg): --builtin-component-lib TEXT Specify a builitin component library to use. --verbose Enable verbose output for debugging. --disable-cache Disable caching of component registry data. --clear-cache Clear the cache before running the command. --cache-dir PATH Specify a directory to use for the cache. ``` Issues for followup: - Ensure our published components ship descriptions for the properties in their JSON schema - More thorough testing of different parameter situations ## How I Tested These Changes New unit tests --- .../dagster_components/cli/generate.py | 17 +- .../dagster_components/core/component.py | 11 + .../dagster_components/generate.py | 2 +- .../dagster_components/lib/dbt_project.py | 8 - .../dagster-dg/dagster_dg/cli/generate.py | 207 ++++++++++++------ .../dagster-dg/dagster_dg/generate.py | 9 +- .../libraries/dagster-dg/dagster_dg/utils.py | 45 ++++ .../cli_tests/test_generate_commands.py | 28 ++- 8 files changed, 228 insertions(+), 99 deletions(-) diff --git a/python_modules/libraries/dagster-components/dagster_components/cli/generate.py b/python_modules/libraries/dagster-components/dagster_components/cli/generate.py index 8405b5ec04e88..4da9481d312b2 100644 --- a/python_modules/libraries/dagster-components/dagster_components/cli/generate.py +++ b/python_modules/libraries/dagster-components/dagster_components/cli/generate.py @@ -1,6 +1,6 @@ import sys from pathlib import Path -from typing import Optional, Tuple +from typing import Optional import click from pydantic import TypeAdapter @@ -23,14 +23,12 @@ def generate_cli() -> None: @click.argument("component_type", type=str) @click.argument("component_name", type=str) @click.option("--json-params", type=str, default=None) -@click.argument("extra_args", nargs=-1, type=str) @click.pass_context def generate_component_command( ctx: click.Context, component_type: str, component_name: str, json_params: Optional[str], - extra_args: Tuple[str, ...], ) -> None: builtin_component_lib = ctx.obj.get(CLI_BUILTIN_COMPONENT_LIB_KEY, False) if not is_inside_code_location_project(Path.cwd()): @@ -52,18 +50,11 @@ def generate_component_command( sys.exit(1) component_type_cls = context.get_component_type(component_type) - generate_params_schema = component_type_cls.generate_params_schema - generate_params_cli = getattr(generate_params_schema, "cli", None) - if generate_params_schema is None: - generate_params = None - elif json_params is not None: + if json_params: + generate_params_schema = component_type_cls.generate_params_schema generate_params = TypeAdapter(generate_params_schema).validate_json(json_params) - elif generate_params_cli is not None: - inner_ctx = click.Context(generate_params_cli) - generate_params_cli.parse_args(inner_ctx, list(extra_args)) - generate_params = inner_ctx.invoke(generate_params_schema.cli, **inner_ctx.params) else: - generate_params = None + generate_params = {} generate_component_instance( context.component_instances_root_path, diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component.py b/python_modules/libraries/dagster-components/dagster_components/core/component.py index dc8ca5e0dab20..4f5d7e1e829e9 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component.py @@ -22,6 +22,7 @@ TypeVar, ) +import click from dagster import _check as check from dagster._core.definitions.definitions_class import Definitions from dagster._core.errors import DagsterError @@ -87,6 +88,16 @@ def _clean_docstring(docstring: str) -> str: return f"{first_line}\n{rest}" +def _get_click_cli_help(command: click.Command) -> str: + with click.Context(command) as ctx: + formatter = click.formatting.HelpFormatter() + param_records = [ + p.get_help_record(ctx) for p in command.get_params(ctx) if p.name != "help" + ] + formatter.write_dl([pr for pr in param_records if pr]) + return formatter.getvalue() + + class ComponentInternalMetadata(TypedDict): summary: Optional[str] description: Optional[str] diff --git a/python_modules/libraries/dagster-components/dagster_components/generate.py b/python_modules/libraries/dagster-components/dagster_components/generate.py index 5f6e9a369bf83..56318d59e98c0 100644 --- a/python_modules/libraries/dagster-components/dagster_components/generate.py +++ b/python_modules/libraries/dagster-components/dagster_components/generate.py @@ -32,7 +32,7 @@ def generate_component_instance( name: str, component_type: Type[Component], component_type_name: str, - generate_params: Any, + generate_params: Mapping[str, Any], ) -> None: component_instance_root_path = Path(os.path.join(root_path, name)) click.echo(f"Creating a Dagster component instance folder at {component_instance_root_path}.") diff --git a/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py b/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py index 02f8ca82273c4..4222ef7ab9f89 100644 --- a/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py +++ b/python_modules/libraries/dagster-components/dagster_components/lib/dbt_project.py @@ -2,7 +2,6 @@ from pathlib import Path from typing import Any, Iterator, Mapping, Optional, Sequence -import click import dagster._check as check from dagster._core.definitions.asset_key import AssetKey from dagster._core.definitions.definitions_class import Definitions @@ -42,13 +41,6 @@ class DbtGenerateParams(BaseModel): init: bool = Field(default=False) project_path: Optional[str] = None - @staticmethod - @click.command - @click.option("--project-path", "-p", type=click.Path(resolve_path=True), default=None) - @click.option("--init", "-i", is_flag=True, default=False) - def cli(project_path: Optional[str], init: bool) -> "DbtGenerateParams": - return DbtGenerateParams(project_path=project_path, init=init) - class DbtProjectComponentTranslator(DagsterDbtTranslator): def __init__( diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py index bb9642105638d..e3812213cdc59 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/generate.py @@ -1,10 +1,12 @@ import os import sys from pathlib import Path -from typing import Optional, Tuple +from typing import Any, Mapping, Optional import click +from click.core import ParameterSource +from dagster_dg.component import RemoteComponentType from dagster_dg.context import ( CodeLocationDirectoryContext, DeploymentDirectoryContext, @@ -18,7 +20,12 @@ generate_component_type, generate_deployment, ) -from dagster_dg.utils import DgClickCommand, DgClickGroup +from dagster_dg.utils import ( + DgClickCommand, + DgClickGroup, + json_schema_property_to_click_option, + parse_json_option, +) @click.group(name="generate", cls=DgClickGroup) @@ -139,82 +146,150 @@ def generate_component_type_command(cli_context: click.Context, name: str) -> No generate_component_type(context, name) -@generate_cli.command(name="component", cls=DgClickCommand) -@click.argument( - "component_type", - type=str, -) -@click.argument("component_name", type=str) -@click.option("--json-params", type=str, default=None, help="JSON string of component parameters.") -@click.argument("extra_args", nargs=-1, type=str) -@click.pass_context -def generate_component_command( - cli_context: click.Context, - component_type: str, - component_name: str, - json_params: Optional[str], - extra_args: Tuple[str, ...], -) -> None: - """Generate a scaffold of a Dagster component. +# The `dg generate component` command is special because its subcommands are dynamically generated +# from the registered component types in the code location. Because the registered component types +# depend on the built-in component library we are using, we cannot resolve them until we have the +# built-in component library, which can be set via a global option, e.g.: +# +# dg --builtin-component-lib dagster_components.test ... +# +# To handle this, we define a custom click.Group subclass that loads the commands on demand. +class GenerateComponentGroup(DgClickGroup): + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + self._commands_defined = False + + def get_command(self, cli_context: click.Context, cmd_name: str) -> Optional[click.Command]: + if not self._commands_defined: + self._define_commands(cli_context) + return super().get_command(cli_context, cmd_name) + + def list_commands(self, cli_context): + if not self._commands_defined: + self._define_commands(cli_context) + return super().list_commands(cli_context) + + def _define_commands(self, cli_context: click.Context) -> None: + """Dynamically define a command for each registered component type.""" + app_context = DgContext.from_cli_context(cli_context) + + if not is_inside_code_location_directory(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location directory.", fg="red" + ) + ) + sys.exit(1) - This command must be run inside a Dagster code location directory. The component scaffold will be - generated in submodule `.components.`. + context = CodeLocationDirectoryContext.from_path(Path.cwd(), app_context) + for key, component_type in context.iter_component_types(): + command = _create_generate_component_subcommand(key, component_type) + self.add_command(command) - The COMPONENT_TYPE must be a registered component type in the code location environment. - You can view all registered component types with `dg list component-types`. The COMPONENT_NAME - will be used to name the submodule created under .components. - Components can optionally be passed generate parameters. There are two ways to do this: +@generate_cli.group(name="component", cls=GenerateComponentGroup) +def generate_component_group() -> None: + """Generate a scaffold of a Dagster component.""" - - Passing --json-params with a JSON string of parameters. For example: - dg generate component foo.bar my_component --json-params '{"param1": "value", "param2": "value"}'`. +def _create_generate_component_subcommand( + component_key: str, component_type: RemoteComponentType +) -> DgClickCommand: + @click.command(name=component_key, cls=DgClickCommand) + @click.argument("component_name", type=str) + @click.option( + "--json-params", + type=str, + default=None, + help="JSON string of component parameters.", + callback=parse_json_option, + ) + @click.pass_context + def generate_component_command( + cli_context: click.Context, + component_name: str, + json_params: Mapping[str, Any], + **key_value_params: Any, + ) -> None: + f"""Generate a scaffold of a {component_type.name} component. - - Passing key-value pairs as space-separated EXTRA_ARGS after `--`. For example: + This command must be run inside a Dagster code location directory. The component scaffold will be + generated in submodule `.components.`. - dg generate component foo.bar my_component -- param1=value param2=value + Components can optionally be passed generate parameters. There are two ways to do this: - When key-value pairs are used, the value type will be inferred from the - underlying component generation schema. + (1) Passing a single --json-params option with a JSON string of parameters. For example: - It is an error to pass both --json-params and EXTRA_ARGS. - """ - dg_context = DgContext.from_cli_context(cli_context) - if not is_inside_code_location_directory(Path.cwd()): - click.echo( - click.style( - "This command must be run inside a Dagster code location directory.", fg="red" + dg generate component foo.bar my_component --json-params '{{"param1": "value", "param2": "value"}}'`. + + (2) Passing each parameter as an option. For example: + + dg generate component foo.bar my_component --param1 value1 --param2 value2` + + It is an error to pass both --json-params and key-value pairs as options. + """ + dg_context = DgContext.from_cli_context(cli_context) + if not is_inside_code_location_directory(Path.cwd()): + click.echo( + click.style( + "This command must be run inside a Dagster code location directory.", fg="red" + ) ) - ) - sys.exit(1) + sys.exit(1) - context = CodeLocationDirectoryContext.from_path(Path.cwd(), dg_context) - if not context.has_component_type(component_type): - click.echo( - click.style(f"No component type `{component_type}` could be resolved.", fg="red") - ) - sys.exit(1) - elif context.has_component_instance(component_name): - click.echo( - click.style(f"A component instance named `{component_name}` already exists.", fg="red") - ) - sys.exit(1) + context = CodeLocationDirectoryContext.from_path(Path.cwd(), dg_context) + if not context.has_component_type(component_key): + click.echo( + click.style(f"No component type `{component_key}` could be resolved.", fg="red") + ) + sys.exit(1) + elif context.has_component_instance(component_name): + click.echo( + click.style( + f"A component instance named `{component_name}` already exists.", fg="red" + ) + ) + sys.exit(1) - if json_params is not None and extra_args: - click.echo( - click.style( - "Detected both --json-params and EXTRA_ARGS. These are mutually exclusive means of passing" - " component generation parameters. Use only one.", - fg="red", + # Specified key-value params will be passed to this function with their default value of + # `None` even if the user did not set them. Filter down to just the ones that were set by + # the user. + user_provided_key_value_params = { + k: v + for k, v in key_value_params.items() + if cli_context.get_parameter_source(k) == ParameterSource.COMMANDLINE + } + if json_params is not None and user_provided_key_value_params: + click.echo( + click.style( + "Detected params passed as both --json-params and individual options. These are mutually exclusive means of passing" + " component generation parameters. Use only one.", + fg="red", + ) ) + sys.exit(1) + elif json_params: + generate_params = json_params + elif user_provided_key_value_params: + generate_params = user_provided_key_value_params + else: + generate_params = None + + generate_component_instance( + Path(context.component_instances_root_path), + component_name, + component_key, + generate_params, + dg_context, ) - sys.exit(1) - generate_component_instance( - Path(context.component_instances_root_path), - component_name, - component_type, - json_params, - extra_args, - dg_context, - ) + # If there are defined generate params, add them to the command + schema = component_type.generate_params_schema + if schema: + for key, field_info in schema["properties"].items(): + # All fields are currently optional because they can also be passed under + # `--json-params` + option = json_schema_property_to_click_option(key, field_info, required=False) + generate_component_command.params.append(option) + + return generate_component_command diff --git a/python_modules/libraries/dagster-dg/dagster_dg/generate.py b/python_modules/libraries/dagster-dg/dagster_dg/generate.py index 46a198f56f7a7..788c9c898dd8c 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/generate.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/generate.py @@ -1,8 +1,9 @@ +import json import os import subprocess import textwrap from pathlib import Path -from typing import Optional, Tuple +from typing import Any, Mapping, Optional import click @@ -148,8 +149,7 @@ def generate_component_instance( root_path: Path, name: str, component_type: str, - json_params: Optional[str], - extra_args: Tuple[str, ...], + generate_params: Optional[Mapping[str, Any]], dg_context: "DgContext", ) -> None: component_instance_root_path = root_path / name @@ -160,8 +160,7 @@ def generate_component_instance( "component", component_type, name, - *(["--json-params", json_params] if json_params else []), - *(["--", *extra_args] if extra_args else []), + *(["--json-params", json.dumps(generate_params)] if generate_params else []), ) execute_code_location_command( Path(component_instance_root_path), diff --git a/python_modules/libraries/dagster-dg/dagster_dg/utils.py b/python_modules/libraries/dagster-dg/dagster_dg/utils.py index 0fc5f980b9518..8bcf29db6b277 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/utils.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/utils.py @@ -1,4 +1,5 @@ import contextlib +import json import os import posixpath import re @@ -329,3 +330,47 @@ class DgClickCommand(DgClickHelpMixin, click.Command): class DgClickGroup(DgClickHelpMixin, click.Group): pass + + +# ######################## +# ##### JSON SCHEMA +# ######################## + +_JSON_SCHEMA_TYPE_TO_CLICK_TYPE = {"string": str, "integer": int, "number": float, "boolean": bool} + + +def json_schema_property_to_click_option( + key: str, field_info: Mapping[str, Any], required: bool +) -> click.Option: + field_type = field_info.get("type", "string") + option_name = f"--{key.replace('_', '-')}" + + # Handle object type fields as JSON strings + if field_type == "object": + option_type = str # JSON string input + help_text = f"{key} (JSON string)" + callback = parse_json_option + + # Handle other basic types + else: + option_type = _JSON_SCHEMA_TYPE_TO_CLICK_TYPE[field_type] + help_text = key + callback = None + + return click.Option( + [option_name], + type=option_type, + required=required, + help=help_text, + callback=callback, + ) + + +def parse_json_option(context: click.Context, param: click.Option, value: str): + """Callback to parse JSON string options into Python objects.""" + if value: + try: + return json.loads(value) + except json.JSONDecodeError: + raise click.BadParameter(f"Invalid JSON string for '{param.name}'.") + return value diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py index 7db275b8b0b92..7bc7ebc634b0b 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py @@ -1,6 +1,7 @@ import json import os import subprocess +import textwrap from pathlib import Path import pytest @@ -153,6 +154,21 @@ def test_generate_component_type_already_exists_fails(in_deployment: bool) -> No assert "already exists" in result.output +def test_generate_component_dynamic_subcommand_generation() -> None: + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("generate", "component", "--help") + assert_runner_result(result) + assert ( + textwrap.dedent(""" + Commands: + dagster_components.test.all_metadata_empty_asset + dagster_components.test.simple_asset + dagster_components.test.simple_pipes_script_asset + """).strip() + in result.output + ) + + @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_no_params_success(in_deployment: bool) -> None: with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): @@ -195,14 +211,13 @@ def test_generate_component_json_params_success(in_deployment: bool) -> None: @pytest.mark.parametrize("in_deployment", [True, False]) -def test_generate_component_extra_args_success(in_deployment: bool) -> None: +def test_generate_component_key_value_params_success(in_deployment: bool) -> None: with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke( "generate", "component", "dagster_components.test.simple_pipes_script_asset", "qux", - "--", "--asset-key=foo", "--filename=hello.py", ) @@ -217,7 +232,7 @@ def test_generate_component_extra_args_success(in_deployment: bool) -> None: ) -def test_generate_component_json_params_and_extra_args_fails() -> None: +def test_generate_component_json_params_and_key_value_params_fails() -> None: with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "generate", @@ -226,11 +241,12 @@ def test_generate_component_json_params_and_extra_args_fails() -> None: "qux", "--json-params", '{"filename": "hello.py"}', - "--", "--filename=hello.py", ) assert_runner_result(result, exit_0=False) - assert "Detected both --json-params and EXTRA_ARGS" in result.output + assert ( + "Detected params passed as both --json-params and individual options" in result.output + ) def test_generate_component_outside_code_location_fails() -> None: @@ -297,7 +313,7 @@ def test_generate_sling_replication_instance() -> None: "params", [ ["--json-params", json.dumps({"project_path": str(dbt_project_path)})], - ["--", "--project-path", dbt_project_path], + ["--project-path", dbt_project_path], ], ) def test_generate_dbt_project_instance(params) -> None: From cda92cd5d55ce330bc44ca922f654f2a91fdc4cb Mon Sep 17 00:00:00 2001 From: Brian Tu Date: Thu, 19 Dec 2024 10:28:21 -0500 Subject: [PATCH 13/37] Hook up op selection syntax to frontend (#26488) ## Summary & Motivation We want to hook up the new op selection syntax so that it is being used on the frontend. --- .../ui-core/src/app/FeatureFlags.oss.tsx | 1 + .../ui-core/src/launchpad/OpSelector.tsx | 3 ++- .../src/op-selection/AntlrOpSelection.ts | 19 ++++++++++++++++++- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx index c9244178d1cfe..bb481edf6ea67 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/FeatureFlags.oss.tsx @@ -7,6 +7,7 @@ export enum FeatureFlag { flagAssetSelectionSyntax = 'flagAssetSelectionSyntax', flagRunSelectionSyntax = 'flagRunSelectionSyntax', flagAssetSelectionWorker = 'flagAssetSelectionWorker', + flagOpSelectionSyntax = 'flagOpSelectionSyntax', // Flags for tests __TestFlagDefaultNone = '__TestFlagDefaultNone', diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/OpSelector.tsx b/js_modules/dagster-ui/packages/ui-core/src/launchpad/OpSelector.tsx index 0d367ea3fee26..4d10777dcef95 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/launchpad/OpSelector.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/OpSelector.tsx @@ -7,6 +7,7 @@ import {OpSelectorQuery, OpSelectorQueryVariables} from './types/OpSelector.type import {filterByQuery} from '../app/GraphQueryImpl'; import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorFragment'; import {ShortcutHandler} from '../app/ShortcutHandler'; +import {filterOpSelectionByQuery} from '../op-selection/AntlrOpSelection'; import {explodeCompositesInHandleGraph} from '../pipelines/CompositeSupport'; import {GRAPH_EXPLORER_SOLID_HANDLE_FRAGMENT} from '../pipelines/GraphExplorer'; import {GraphQueryInput} from '../ui/GraphQueryInput'; @@ -85,7 +86,7 @@ export const OpSelector = (props: IOpSelectorProps) => { const opsFetchError = (data?.pipelineOrError.__typename !== 'Pipeline' && data?.pipelineOrError.message) || null; - const queryResultOps = filterByQuery(ops, query).all; + const queryResultOps = filterOpSelectionByQuery(ops, query).all; const invalidOpSelection = !loading && queryResultOps.length === 0; const errorMessage = invalidOpSelection diff --git a/js_modules/dagster-ui/packages/ui-core/src/op-selection/AntlrOpSelection.ts b/js_modules/dagster-ui/packages/ui-core/src/op-selection/AntlrOpSelection.ts index 614bde690e347..80a6042aa9881 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/op-selection/AntlrOpSelection.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/op-selection/AntlrOpSelection.ts @@ -1,10 +1,12 @@ import {CharStreams, CommonTokenStream} from 'antlr4ts'; +import {FeatureFlag} from 'shared/app/FeatureFlags.oss'; import {AntlrOpSelectionVisitor} from './AntlrOpSelectionVisitor'; -import {GraphQueryItem} from '../app/GraphQueryImpl'; +import {GraphQueryItem, filterByQuery} from '../app/GraphQueryImpl'; import {AntlrInputErrorListener} from '../asset-selection/AntlrAssetSelection'; import {OpSelectionLexer} from './generated/OpSelectionLexer'; import {OpSelectionParser} from './generated/OpSelectionParser'; +import {featureEnabled} from '../app/Flags'; type OpSelectionQueryResult = { all: GraphQueryItem[]; @@ -40,3 +42,18 @@ export const parseOpSelectionQuery = ( return e as Error; } }; + +export const filterOpSelectionByQuery = ( + all_ops: GraphQueryItem[], + query: string, +): OpSelectionQueryResult => { + if (featureEnabled(FeatureFlag.flagOpSelectionSyntax)) { + const result = parseOpSelectionQuery(all_ops, query); + if (result instanceof Error) { + // fall back to old behavior + return filterByQuery(all_ops, query); + } + return result; + } + return filterByQuery(all_ops, query); +}; From cba213babe15f0c5251de6d9f961dfb423d0b9e0 Mon Sep 17 00:00:00 2001 From: gibsondan Date: Thu, 19 Dec 2024 09:50:19 -0600 Subject: [PATCH 14/37] Regenerate pyright pins with a more reasonable fastapi pin (#26582) Summary: For whatever reason the current constraints are producing a fastapi version from 2019 that is getting flagged by dependabot. Give it a hint to produce a version that is more modern. Test Plan: BK > Insert changelog entry or delete this section. ## Summary & Motivation ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. --- pyright/alt-1/requirements-pinned.txt | 39 +++++++-------- pyright/master/requirements-pinned.txt | 69 +++++++++++++------------- pyright/master/requirements.txt | 2 +- 3 files changed, 54 insertions(+), 56 deletions(-) diff --git a/pyright/alt-1/requirements-pinned.txt b/pyright/alt-1/requirements-pinned.txt index 9c55cd90ee281..e7d36fbca5958 100644 --- a/pyright/alt-1/requirements-pinned.txt +++ b/pyright/alt-1/requirements-pinned.txt @@ -1,8 +1,8 @@ agate==1.9.1 -aiobotocore==2.15.2 +aiobotocore==2.16.0 aiofile==3.9.0 aiohappyeyeballs==2.4.4 -aiohttp==3.11.10 +aiohttp==3.11.11 aioitertools==0.12.0 aiosignal==1.3.2 alembic==1.14.0 @@ -24,10 +24,10 @@ backoff==2.2.1 backports-tarfile==1.2.0 beautifulsoup4==4.12.3 bleach==6.2.0 -boto3==1.35.36 +boto3==1.35.81 boto3-stubs-lite==1.35.70 -botocore==1.35.36 -botocore-stubs==1.35.82 +botocore==1.35.81 +botocore-stubs==1.35.84 buildkite-test-collector==0.1.9 cachetools==5.5.0 caio==0.9.17 @@ -66,12 +66,12 @@ daff==1.3.46 -e python_modules/libraries/dagster-spark -e python_modules/dagster-webserver db-dtypes==1.3.1 -dbt-adapters==1.10.4 +dbt-adapters==1.12.0 dbt-common==1.14.0 -dbt-core==1.8.9 +dbt-core==1.9.1 dbt-duckdb==1.9.1 dbt-extractor==0.5.1 -dbt-semantic-interfaces==0.5.1 +dbt-semantic-interfaces==0.7.4 dbt-snowflake==1.9.0 debugpy==1.8.11 decopatch==1.4.10 @@ -95,7 +95,7 @@ frozenlist==1.5.0 fsspec==2024.3.0 gcsfs==0.8.0 google-api-core==2.24.0 -google-api-python-client==2.155.0 +google-api-python-client==2.156.0 google-auth==2.37.0 google-auth-httplib2==0.2.0 google-auth-oauthlib==1.2.1 @@ -145,13 +145,12 @@ jupyter-events==0.11.0 jupyter-lsp==2.2.5 jupyter-server==2.14.2 jupyter-server-terminals==0.5.3 -jupyterlab==4.3.3 +jupyterlab==4.3.4 jupyterlab-pygments==0.3.0 jupyterlab-server==2.27.3 keyring==25.5.0 kiwisolver==1.4.7 leather==0.4.0 -logbook==1.5.3 makefun==1.15.6 mako==1.3.8 markdown-it-py==3.0.0 @@ -161,7 +160,6 @@ matplotlib==3.10.0 matplotlib-inline==0.1.7 mccabe==0.7.0 mdurl==0.1.2 -minimal-snowplow-tracker==0.0.2 mistune==3.0.2 more-itertools==10.5.0 morefs==0.2.2 @@ -169,7 +167,7 @@ msgpack==1.1.0 multidict==6.1.0 multimethod==1.12 mypy==1.13.0 -mypy-boto3-ecs==1.35.77 +mypy-boto3-ecs==1.35.83 mypy-boto3-emr==1.35.68 mypy-boto3-emr-serverless==1.35.79 mypy-boto3-glue==1.35.80 @@ -210,7 +208,7 @@ prometheus-client==0.21.1 prompt-toolkit==3.0.48 propcache==0.2.1 proto-plus==1.25.0 -protobuf==5.29.1 +protobuf==5.29.2 psutil==6.1.0 psycopg2-binary==2.9.10 ptyprocess==0.7.0 @@ -220,8 +218,8 @@ pyarrow==18.1.0 pyasn1==0.6.1 pyasn1-modules==0.4.1 pycparser==2.22 -pydantic==2.10.3 -pydantic-core==2.27.1 +pydantic==2.10.4 +pydantic-core==2.27.2 pygments==2.18.0 pyjwt==2.10.1 pylint==3.3.2 @@ -244,7 +242,7 @@ pytimeparse==1.1.8 pytz==2024.2 pyyaml==6.0.2 pyzmq==26.2.0 -rapidfuzz==3.10.1 +rapidfuzz==3.11.0 referencing==0.35.1 requests==2.32.3 requests-oauthlib==2.0.0 @@ -264,14 +262,15 @@ send2trash==1.8.3 setuptools==75.6.0 shellingham==1.5.4 six==1.17.0 -slack-sdk==3.33.5 +slack-sdk==3.34.0 sniffio==1.3.1 snowflake-connector-python==3.12.4 snowflake-sqlalchemy==1.5.1 +snowplow-tracker==1.0.4 sortedcontainers==2.4.0 soupsieve==2.6 sqlalchemy==1.4.54 -sqlglot==26.0.0 +sqlglot==26.0.1 sqlglotrs==0.3.0 sqlparse==0.5.3 stack-data==0.6.3 @@ -292,7 +291,7 @@ tqdm==4.67.1 traitlets==5.14.3 typeguard==4.4.1 typer==0.15.1 -types-awscrt==0.23.5 +types-awscrt==0.23.6 types-backports==0.1.3 types-certifi==2021.10.8.3 types-cffi==1.16.0.20240331 diff --git a/pyright/master/requirements-pinned.txt b/pyright/master/requirements-pinned.txt index f3383f88c3e96..4705a9adbdaee 100644 --- a/pyright/master/requirements-pinned.txt +++ b/pyright/master/requirements-pinned.txt @@ -1,4 +1,4 @@ -acryl-datahub==0.14.1.12 +acryl-datahub==0.15.0 agate==1.9.1 aiofile==3.9.0 aiohappyeyeballs==2.4.4 @@ -58,10 +58,10 @@ billiard==4.2.1 bleach==6.2.0 blinker==1.9.0 bokeh==3.6.2 -boto3==1.35.82 +boto3==1.35.84 boto3-stubs-lite==1.35.70 -botocore==1.35.82 -botocore-stubs==1.35.82 +botocore==1.35.84 +botocore-stubs==1.35.84 buildkite-test-collector==0.1.9 cachecontrol==0.14.1 cached-property==2.0.1 @@ -89,7 +89,7 @@ coloredlogs==14.0 colorlog==4.8.0 comm==0.2.2 configupdater==3.2 -confluent-kafka==2.6.1 +confluent-kafka==2.6.2 connexion==2.14.2 contourpy==1.3.1 coverage==7.6.9 @@ -173,8 +173,8 @@ dagster-contrib-modal==0.0.2 -e python_modules/libraries/dagster-wandb -e python_modules/dagster-webserver -e python_modules/libraries/dagstermill -dask==2024.12.0 -dask-expr==1.1.20 +dask==2024.12.1 +dask-expr==1.1.21 dask-jobqueue==0.9.0 dask-kubernetes==2022.9.0 dask-yarn==0.9 @@ -184,13 +184,13 @@ dataclasses-json==0.6.7 datadog==0.50.2 dataproperty==1.0.1 db-dtypes==1.3.1 -dbt-adapters==1.10.2 +dbt-adapters==1.12.0 dbt-common==1.14.0 -dbt-core==1.8.9 +dbt-core==1.9.1 dbt-duckdb==1.9.1 -e examples/starlift-demo dbt-extractor==0.5.1 -dbt-semantic-interfaces==0.5.1 +dbt-semantic-interfaces==0.7.4 debugpy==1.8.11 decopatch==1.4.10 decorator==5.1.1 @@ -202,10 +202,10 @@ deprecated==1.2.15 dict2css==0.3.0.post1 dill==0.3.9 distlib==0.3.9 -distributed==2024.12.0 +distributed==2024.12.1 distro==1.9.0 -e examples/experimental/dagster-dlift/kitchen-sink -dlt==1.4.1 +dlt==1.5.0 dnspython==2.7.0 docker==7.1.0 docker-image-py==0.1.13 @@ -223,7 +223,7 @@ execnet==2.1.1 executing==2.1.0 expandvars==0.12.0 faiss-cpu==1.8.0 -fastapi==0.1.17 +fastapi==0.115.6 fastavro==1.9.7 fastjsonschema==2.21.1 -e examples/feature_graph_backed_assets @@ -252,7 +252,7 @@ gitdb==4.0.11 gitpython==3.1.43 giturlparse==0.12.0 google-api-core==2.24.0 -google-api-python-client==2.155.0 +google-api-python-client==2.156.0 google-auth==2.37.0 google-auth-httplib2==0.2.0 google-auth-oauthlib==1.2.1 @@ -327,7 +327,7 @@ jupyter-events==0.11.0 jupyter-lsp==2.2.5 jupyter-server==2.14.2 jupyter-server-terminals==0.5.3 -jupyterlab==4.3.3 +jupyterlab==4.3.4 jupyterlab-pygments==0.3.0 jupyterlab-server==2.27.3 jupyterlab-widgets==3.0.13 @@ -337,12 +337,12 @@ kiwisolver==1.4.7 kombu==5.4.2 kopf==1.37.4 kubernetes==31.0.0 -kubernetes-asyncio==31.1.1 +kubernetes-asyncio==32.0.0 langchain==0.3.7 langchain-community==0.3.5 -langchain-core==0.3.25 +langchain-core==0.3.27 langchain-openai==0.2.5 -langchain-text-splitters==0.3.3 +langchain-text-splitters==0.3.4 langsmith==0.1.147 lazy-object-proxy==1.10.0 leather==0.4.0 @@ -352,7 +352,6 @@ linkify-it-py==2.0.3 lkml==1.3.6 locket==1.0.0 lockfile==0.12.2 -logbook==1.5.3 looker-sdk==24.16.2 lxml==5.3.0 makefun==1.15.6 @@ -363,17 +362,16 @@ markupsafe==3.0.2 marshmallow==3.23.1 marshmallow-oneofschema==3.1.1 marshmallow-sqlalchemy==0.26.1 -mashumaro==3.15 +mashumaro==3.14 matplotlib==3.10.0 matplotlib-inline==0.1.3 mbstrdecoder==1.1.3 mdit-py-plugins==0.4.2 mdurl==0.1.2 -minimal-snowplow-tracker==0.0.2 mistune==3.0.2 mixpanel==4.10.1 mlflow==1.27.0 -modal==0.68.26 +modal==0.68.41 more-itertools==10.5.0 morefs==0.2.2 moto==4.2.14 @@ -383,7 +381,7 @@ msal-extensions==1.2.0 msgpack==1.1.0 multidict==6.1.0 multimethod==1.12 -mypy-boto3-ecs==1.35.77 +mypy-boto3-ecs==1.35.83 mypy-boto3-emr==1.35.68 mypy-boto3-emr-serverless==1.35.79 mypy-boto3-glue==1.35.80 @@ -409,7 +407,7 @@ objgraph==3.6.2 onnx==1.17.0 onnxconverter-common==1.13.0 onnxruntime==1.20.1 -openai==1.57.4 +openai==1.58.1 openapi-schema-validator==0.6.2 openapi-spec-validator==0.7.1 opentelemetry-api==1.29.0 @@ -461,7 +459,7 @@ prometheus-flask-exporter==0.23.1 prompt-toolkit==3.0.48 propcache==0.2.1 proto-plus==1.25.0 -protobuf==5.29.1 +protobuf==5.29.2 psutil==6.1.0 psycopg2-binary==2.9.10 ptyprocess==0.7.0 @@ -473,8 +471,8 @@ pyarrow-hotfix==0.6 pyasn1==0.6.1 pyasn1-modules==0.4.1 pycparser==2.22 -pydantic==2.10.3 -pydantic-core==2.27.1 +pydantic==2.10.4 +pydantic-core==2.27.2 pydantic-settings==2.7.0 pydata-google-auth==1.9.0 pyflakes==3.2.0 @@ -513,7 +511,7 @@ pytzdata==2020.1 pyyaml==6.0.2 pyzmq==26.2.0 querystring-parser==1.2.4 -rapidfuzz==3.10.1 +rapidfuzz==3.11.0 readme-renderer==44.0 referencing==0.35.1 regex==2024.11.6 @@ -552,15 +550,16 @@ sigtools==4.0.1 simplejson==3.19.3 six==1.17.0 skein==0.8.2 -skl2onnx==1.17.0 -slack-sdk==3.33.5 +skl2onnx==1.18.0 +slack-sdk==3.34.0 sling==1.3.3 sling-mac-arm64==1.3.3 smmap==5.0.1 sniffio==1.3.1 snowballstemmer==2.2.0 snowflake-connector-python==3.12.4 -snowflake-sqlalchemy==1.7.1 +snowflake-sqlalchemy==1.7.2 +snowplow-tracker==1.0.4 sortedcontainers==2.4.0 soupsieve==2.6 sphinx==8.1.3 @@ -578,16 +577,16 @@ sphinxcontrib-serializinghtml==2.0.0 sqlalchemy==1.4.54 sqlalchemy-jsonfield==1.0.2 sqlalchemy-utils==0.41.2 -sqlglot==26.0.0 +sqlglot==26.0.1 sqlglotrs==0.3.0 sqlparse==0.5.3 sshpubkeys==3.3.1 sshtunnel==0.4.0 stack-data==0.6.3 -starlette==0.42.0 +starlette==0.41.3 structlog==24.4.0 sympy==1.13.1 -synchronicity==0.9.6 +synchronicity==0.9.7 syrupy==4.8.0 tableauserverclient==0.34 tabledata==1.3.3 @@ -620,7 +619,7 @@ twine==6.0.1 typeguard==4.4.1 typepy==1.3.2 typer==0.15.1 -types-awscrt==0.23.5 +types-awscrt==0.23.6 types-backports==0.1.3 types-certifi==2021.10.8.3 types-cffi==1.16.0.20240331 diff --git a/pyright/master/requirements.txt b/pyright/master/requirements.txt index 1d9b6d64ff895..2cba16896ccbc 100644 --- a/pyright/master/requirements.txt +++ b/pyright/master/requirements.txt @@ -107,7 +107,7 @@ wordcloud # (quickstart-*) apache-airflow>2.7 pendulum<3 types-sqlalchemy==1.4.53.34 - +fastapi>=0.115.6 # producing a bizarrely early version of fastapi without this ### EXAMPLES From 1e35499df105c0d1cfe371c89c189a1fbe74d8c9 Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 11:16:55 -0500 Subject: [PATCH 15/37] feat: allow passing config for dagster asset materialize cli [#26069 continued] (#26578) ## Summary & Motivation Continuation of #26069: > I was browsing the repo for good first issues as a new contributor, and found #16802 interesting. It looks like @sam-goodwin's PR #20979 is nearly complete, except for a change needed for one of the imports. I used that PR's solution, but also resolved the import issue. Which is a continuation of #20979: > > This change adds support for -c, --config and --config-json in the dagster asset materialize CLI command. > > Our motivation is to simply materialize a whole graph as part of CI/CD and use run_config to specify parameters that scope down that materialization for the sake of CI/CD. A single partition of our asset graph can take a long time to run and for CI/CD we don't need the whole thing. ## How I Tested These Changes ## Changelog - [dagster-cli] Adds support for `--config-json` to the `dagster asset materialize` command --------- Co-authored-by: Jimmy Shah --- python_modules/dagster/dagster/_cli/asset.py | 15 +++++++- python_modules/dagster/dagster/_cli/job.py | 36 ++----------------- .../dagster/_cli/workspace/cli_target.py | 35 ++++++++++++++++++ .../cli_tests/command_tests/assets.py | 17 ++++++++- .../command_tests/test_asset_list_command.py | 2 ++ .../command_tests/test_materialize_command.py | 21 ++++++++++- 6 files changed, 90 insertions(+), 36 deletions(-) diff --git a/python_modules/dagster/dagster/_cli/asset.py b/python_modules/dagster/dagster/_cli/asset.py index e1589f6514876..89dd9f21ebaf6 100644 --- a/python_modules/dagster/dagster/_cli/asset.py +++ b/python_modules/dagster/dagster/_cli/asset.py @@ -3,9 +3,11 @@ import click import dagster._check as check +from dagster._cli.job import get_config_from_args from dagster._cli.utils import get_instance_for_cli, get_possibly_temporary_instance_for_cli from dagster._cli.workspace.cli_target import ( get_repository_python_origin_from_kwargs, + python_job_config_argument, python_origin_target_argument, ) from dagster._core.definitions.asset_selection import AssetSelection @@ -38,6 +40,12 @@ def asset_cli(): help="Asset partition range to target i.e. ...", required=False, ) +@python_job_config_argument("materialize") +@click.option( + "--config-json", + type=click.STRING, + help="JSON string of run config to use for this job run. Cannot be used with -c / --config.", +) def asset_materialize_command(**kwargs): with capture_interrupts(): with get_possibly_temporary_instance_for_cli( @@ -48,6 +56,7 @@ def asset_materialize_command(**kwargs): @telemetry_wrapper def execute_materialize_command(instance: DagsterInstance, kwargs: Mapping[str, str]) -> None: + config = get_config_from_args(kwargs) repository_origin = get_repository_python_origin_from_kwargs(kwargs) recon_repo = recon_repository_from_origin(repository_origin) @@ -137,7 +146,11 @@ def execute_materialize_command(instance: DagsterInstance, kwargs: Mapping[str, tags = {} result = execute_job( - job=reconstructable_job, asset_selection=list(asset_keys), instance=instance, tags=tags + job=reconstructable_job, + asset_selection=list(asset_keys), + instance=instance, + tags=tags, + run_config=config, ) if not result.success: raise click.ClickException("Materialization failed.") diff --git a/python_modules/dagster/dagster/_cli/job.py b/python_modules/dagster/dagster/_cli/job.py index fea9f907916fe..134a3843b3c1b 100644 --- a/python_modules/dagster/dagster/_cli/job.py +++ b/python_modules/dagster/dagster/_cli/job.py @@ -16,11 +16,13 @@ ClickArgValue, ClickOption, get_code_location_from_workspace, + get_config_from_args, get_job_python_origin_from_kwargs, get_remote_job_from_kwargs, get_remote_job_from_remote_repo, get_remote_repository_from_code_location, get_remote_repository_from_kwargs, + get_run_config_from_file_list, get_workspace_from_kwargs, job_repository_target_argument, job_target_argument, @@ -60,7 +62,7 @@ from dagster._utils.interrupts import capture_interrupts from dagster._utils.merger import merge_dicts from dagster._utils.tags import normalize_tags -from dagster._utils.yaml_utils import dump_run_config_yaml, load_yaml_from_glob_list +from dagster._utils.yaml_utils import dump_run_config_yaml T = TypeVar("T") T_Callable = TypeVar("T_Callable", bound=Callable[..., Any]) @@ -235,11 +237,6 @@ def print_op( printer.line(output_def_snap.name) -def get_run_config_from_file_list(file_list: Optional[Sequence[str]]) -> Mapping[str, object]: - check.opt_sequence_param(file_list, "file_list", of_type=str) - return cast(Mapping[str, object], load_yaml_from_glob_list(file_list) if file_list else {}) - - @job_cli.command( name="execute", help="Execute a job.\n\n{instructions}".format( @@ -306,33 +303,6 @@ def get_tags_from_args(kwargs: ClickArgMapping) -> Mapping[str, str]: ) from e -def get_config_from_args(kwargs: Mapping[str, str]) -> Mapping[str, object]: - config = cast(Tuple[str, ...], kwargs.get("config")) # files - config_json = kwargs.get("config_json") - - if not config and not config_json: - return {} - - elif config and config_json: - raise click.UsageError("Cannot specify both -c / --config and --config-json") - - elif config: - config_file_list = list(check.opt_tuple_param(config, "config", of_type=str)) - return get_run_config_from_file_list(config_file_list) - - elif config_json: - config_json = cast(str, config_json) - try: - return json.loads(config_json) - - except JSONDecodeError: - raise click.UsageError( - f"Invalid JSON-string given for `--config-json`: {config_json}\n\n{serializable_error_info_from_exc_info(sys.exc_info()).to_string()}" - ) - else: - check.failed("Unhandled case getting config from kwargs") - - def get_op_selection_from_args(kwargs: ClickArgMapping) -> Optional[Sequence[str]]: op_selection_str = kwargs.get("op_selection") if not isinstance(op_selection_str, str): diff --git a/python_modules/dagster/dagster/_cli/workspace/cli_target.py b/python_modules/dagster/dagster/_cli/workspace/cli_target.py index 9a8587c1182f7..28ecece2accab 100644 --- a/python_modules/dagster/dagster/_cli/workspace/cli_target.py +++ b/python_modules/dagster/dagster/_cli/workspace/cli_target.py @@ -47,7 +47,10 @@ WorkspaceLoadTarget, ) from dagster._grpc.utils import get_loadable_targets +from dagster._seven import JSONDecodeError, json +from dagster._utils.error import serializable_error_info_from_exc_info from dagster._utils.hosted_user_process import recon_repository_from_origin +from dagster._utils.yaml_utils import load_yaml_from_glob_list if TYPE_CHECKING: from dagster._core.workspace.context import WorkspaceProcessContext @@ -800,3 +803,35 @@ def get_remote_job_from_kwargs(instance: DagsterInstance, version: str, kwargs: def _sorted_quoted(strings: Iterable[str]) -> str: return "[" + ", ".join([f"'{s}'" for s in sorted(list(strings))]) + "]" + + +def get_run_config_from_file_list(file_list: Optional[Sequence[str]]) -> Mapping[str, object]: + check.opt_sequence_param(file_list, "file_list", of_type=str) + return cast(Mapping[str, object], load_yaml_from_glob_list(file_list) if file_list else {}) + + +def get_config_from_args(kwargs: Mapping[str, str]) -> Mapping[str, object]: + config = cast(Tuple[str, ...], kwargs.get("config")) # files + config_json = kwargs.get("config_json") + + if not config and not config_json: + return {} + + elif config and config_json: + raise click.UsageError("Cannot specify both -c / --config and --config-json") + + elif config: + config_file_list = list(check.opt_tuple_param(config, "config", of_type=str)) + return get_run_config_from_file_list(config_file_list) + + elif config_json: + config_json = cast(str, config_json) + try: + return json.loads(config_json) + + except JSONDecodeError: + raise click.UsageError( + f"Invalid JSON-string given for `--config-json`: {config_json}\n\n{serializable_error_info_from_exc_info(sys.exc_info()).to_string()}" + ) + else: + check.failed("Unhandled case getting config from kwargs") diff --git a/python_modules/dagster/dagster_tests/cli_tests/command_tests/assets.py b/python_modules/dagster/dagster_tests/cli_tests/command_tests/assets.py index 3cc4565e0f711..59dd764b890af 100644 --- a/python_modules/dagster/dagster_tests/cli_tests/command_tests/assets.py +++ b/python_modules/dagster/dagster_tests/cli_tests/command_tests/assets.py @@ -1,4 +1,4 @@ -from dagster import StaticPartitionsDefinition, asset +from dagster import AssetExecutionContext, Config, StaticPartitionsDefinition, asset from dagster._core.definitions.backfill_policy import BackfillPolicy from dagster._core.definitions.time_window_partitions import DailyPartitionsDefinition @@ -37,6 +37,21 @@ def single_run_partitioned_asset() -> None: ... def multi_run_partitioned_asset() -> None: ... +class MyConfig(Config): + some_prop: str + + +@asset +def asset_with_config(context: AssetExecutionContext, config: MyConfig): + context.log.info(f"some_prop:{config.some_prop}") + + +@asset +def asset_assert_with_config(context: AssetExecutionContext, config: MyConfig): + assert config.some_prop == "foo" + context.log.info(f"some_prop:{config.some_prop}") + + @asset def fail_asset() -> None: raise Exception("failure") diff --git a/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_asset_list_command.py b/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_asset_list_command.py index ae1f33fcecc16..482d04b7e5364 100644 --- a/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_asset_list_command.py +++ b/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_asset_list_command.py @@ -28,6 +28,8 @@ def test_no_selection(): == "\n".join( [ "asset1", + "asset_assert_with_config", + "asset_with_config", "differently_partitioned_asset", "downstream_asset", "fail_asset", diff --git a/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_materialize_command.py b/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_materialize_command.py index 4d16a01ade5eb..1a730fa956b42 100644 --- a/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_materialize_command.py +++ b/python_modules/dagster/dagster_tests/cli_tests/command_tests/test_materialize_command.py @@ -1,7 +1,7 @@ from typing import Optional from click.testing import CliRunner -from dagster import AssetKey +from dagster import AssetKey, DagsterInvalidConfigError from dagster._cli.asset import asset_materialize_command from dagster._core.test_utils import instance_for_test from dagster._utils import file_relative_path @@ -177,3 +177,22 @@ def test_partition_range_multi_run_backfill_policy(): def test_failure(): result = invoke_materialize("fail_asset") assert result.exit_code == 1 + + +def test_run_cli_config_json(): + with instance_for_test() as instance: + asset_key = "asset_assert_with_config" + runner = CliRunner() + options = [ + "-f", + file_relative_path(__file__, "assets.py"), + "--select", + asset_key, + "--config-json", + '{"ops": {"asset_assert_with_config": {"config": {"some_prop": "foo"}}}}', + ] + + result = runner.invoke(asset_materialize_command, options) + assert not isinstance(result, DagsterInvalidConfigError) + assert instance.get_latest_materialization_event(AssetKey(asset_key)) is not None + assert result.exit_code == 0 From a1e8f854ee32dc96f111041216aba309afcaa9ea Mon Sep 17 00:00:00 2001 From: Isaac Hellendag <2823852+hellendag@users.noreply.github.com> Date: Thu, 19 Dec 2024 10:56:40 -0600 Subject: [PATCH 16/37] [ui] Limit the amount of asset info required for evaluation list (#26571) ## Summary & Motivation We don't need so much fragment data for the evaluation list, so let's require less so that consumers don't need to overquery. ## How I Tested These Changes TS, lint, storybook. --- .../EvaluationList.tsx | 10 ++++---- .../EvaluationListRow.tsx | 24 ++++++++++++------- .../EvaluationStatusTag.tsx | 17 ++++++++----- .../__stories__/EvaluationList.stories.tsx | 6 ++++- 4 files changed, 37 insertions(+), 20 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationList.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationList.tsx index 18eeaf18b0f7e..25daf6c8e841f 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationList.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationList.tsx @@ -1,15 +1,16 @@ import {Table} from '@dagster-io/ui-components'; +import {AssetKey} from '../types'; import {EvaluationListRow} from './EvaluationListRow'; import {AssetConditionEvaluationRecordFragment} from './types/GetEvaluationsQuery.types'; -import {AssetViewDefinitionNodeFragment} from '../types/AssetView.types'; interface Props { - definition: AssetViewDefinitionNodeFragment; + assetKey: AssetKey; + isPartitioned: boolean; evaluations: AssetConditionEvaluationRecordFragment[]; } -export const EvaluationList = ({definition, evaluations}: Props) => { +export const EvaluationList = ({assetKey, isPartitioned, evaluations}: Props) => { return ( @@ -25,7 +26,8 @@ export const EvaluationList = ({definition, evaluations}: Props) => { ); })} diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationListRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationListRow.tsx index 4a52253515d1f..34ba66772ec19 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationListRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationListRow.tsx @@ -10,20 +10,21 @@ import { } from '@dagster-io/ui-components'; import {useState} from 'react'; +import {AssetKey} from '../types'; import {EvaluationDetailDialog} from './EvaluationDetailDialog'; import {EvaluationStatusTag} from './EvaluationStatusTag'; import {AssetConditionEvaluationRecordFragment} from './types/GetEvaluationsQuery.types'; import {DEFAULT_TIME_FORMAT} from '../../app/time/TimestampFormat'; import {RunsFeedTableWithFilters} from '../../runs/RunsFeedTable'; import {TimestampDisplay} from '../../schedules/TimestampDisplay'; -import {AssetViewDefinitionNodeFragment} from '../types/AssetView.types'; interface Props { - definition: AssetViewDefinitionNodeFragment; + assetKey: AssetKey; + isPartitioned: boolean; evaluation: AssetConditionEvaluationRecordFragment; } -export const EvaluationListRow = ({evaluation, definition}: Props) => { +export const EvaluationListRow = ({evaluation, assetKey, isPartitioned}: Props) => { const [isOpen, setIsOpen] = useState(false); return ( @@ -39,27 +40,32 @@ export const EvaluationListRow = ({evaluation, definition}: Props) => { ); }; -const EvaluationRunInfo = ({evaluation}: {evaluation: AssetConditionEvaluationRecordFragment}) => { - const {runIds} = evaluation; +interface EvaluationRunInfoProps { + runIds: string[]; + timestamp: number; +} + +const EvaluationRunInfo = ({runIds, timestamp}: EvaluationRunInfoProps) => { const [isOpen, setIsOpen] = useState(false); if (runIds.length === 0) { @@ -95,7 +101,7 @@ const EvaluationRunInfo = ({evaluation}: {evaluation: AssetConditionEvaluationRe <> Runs at{' '} diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationStatusTag.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationStatusTag.tsx index 4e89fbb089f9e..2e2b21a794ff0 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationStatusTag.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AutoMaterializePolicyPage/EvaluationStatusTag.tsx @@ -1,17 +1,23 @@ import {Box, Colors, Icon, Popover, Tag} from '@dagster-io/ui-components'; import {useMemo} from 'react'; +import {AssetKey} from '../types'; import {PartitionSubsetList} from './PartitionSubsetList'; import {AssetConditionEvaluationRecordFragment} from './types/GetEvaluationsQuery.types'; -import {AssetViewDefinitionNodeFragment} from '../types/AssetView.types'; interface Props { - definition: AssetViewDefinitionNodeFragment; + assetKey: AssetKey; + isPartitioned: boolean; selectedEvaluation: AssetConditionEvaluationRecordFragment; selectPartition: (partitionKey: string | null) => void; } -export const EvaluationStatusTag = ({definition, selectedEvaluation, selectPartition}: Props) => { +export const EvaluationStatusTag = ({ + assetKey, + isPartitioned, + selectedEvaluation, + selectPartition, +}: Props) => { const evaluation = selectedEvaluation?.evaluation; const rootEvaluationNode = useMemo( () => evaluation?.evaluationNodes.find((node) => node.uniqueId === evaluation.rootUniqueId), @@ -19,8 +25,7 @@ export const EvaluationStatusTag = ({definition, selectedEvaluation, selectParti ); const rootUniqueId = evaluation?.rootUniqueId; - const partitionDefinition = definition?.partitionDefinition; - const assetKeyPath = definition?.assetKey.path || []; + const assetKeyPath = assetKey.path || []; const numRequested = selectedEvaluation?.numRequested; const numTrue = @@ -29,7 +34,7 @@ export const EvaluationStatusTag = ({definition, selectedEvaluation, selectParti : null; if (numRequested) { - if (partitionDefinition && rootUniqueId && numTrue) { + if (isPartitioned && rootUniqueId && numTrue) { return ( { return ( - + ); }; From 898ce98300da3bfba7a02fd6b69e38f9ba3c44fc Mon Sep 17 00:00:00 2001 From: Alex Langenfeld Date: Thu, 19 Dec 2024 11:25:29 -0600 Subject: [PATCH 17/37] [buildkite] use longer commit hash in linked builds (#25872) 6 is to short to do lookup in github ## How I Tested These Changes 10 seems to be what is used elsewhere --- .buildkite/dagster-buildkite/dagster_buildkite/steps/trigger.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.buildkite/dagster-buildkite/dagster_buildkite/steps/trigger.py b/.buildkite/dagster-buildkite/dagster_buildkite/steps/trigger.py index 3f6087cdbfd38..fff5f317adb89 100644 --- a/.buildkite/dagster-buildkite/dagster_buildkite/steps/trigger.py +++ b/.buildkite/dagster-buildkite/dagster_buildkite/steps/trigger.py @@ -28,7 +28,7 @@ def build_trigger_step( dagster_commit_hash = safe_getenv("BUILDKITE_COMMIT") step: TriggerStep = { "trigger": pipeline, - "label": f":link: {pipeline} from dagster@{dagster_commit_hash[:6]}", + "label": f":link: {pipeline} from dagster@{dagster_commit_hash[:10]}", "async": async_step, "build": { "env": env or {}, From 37f6bef3d784219a29b17028d39fbeac4722cfd1 Mon Sep 17 00:00:00 2001 From: Nikki Everett Date: Thu, 19 Dec 2024 12:07:40 -0600 Subject: [PATCH 18/37] Table formatting how-to (#26599) ## Summary & Motivation Adds guidance on how to add line break and lists to Markdown tables. ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. Signed-off-by: nikki everett --- docs/docs-beta/CONTRIBUTING.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/docs-beta/CONTRIBUTING.md b/docs/docs-beta/CONTRIBUTING.md index c0b8e52ba5712..3d446d91f5e60 100644 --- a/docs/docs-beta/CONTRIBUTING.md +++ b/docs/docs-beta/CONTRIBUTING.md @@ -102,6 +102,10 @@ After: | `DAGSTER_CLOUD_DEPLOYMENT_NAME` | The name of the Dagster+ deployment.

**Example:** `prod`. | | `DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT` | `1` if the deployment is a [branch deployment](/dagster-plus/features/ci-cd/branch-deployments/index.md). | +#### Line breaks and lists in tables + +[Use HTML](https://www.markdownguide.org/hacks/#table-formatting) to add line breaks and lists to tables. + ### Whitespace via `{" "}` Forcing empty space using the `{" "}` interpolation is not supported, and must be removed. From 3fcec9f7f31afd6e3c39955fd440e9348cdcb390 Mon Sep 17 00:00:00 2001 From: gibsondan Date: Thu, 19 Dec 2024 13:30:07 -0600 Subject: [PATCH 19/37] Add flag to TestEventLogStorage for storages that don't implement get_logs_for_all_runs_by_log_id (#26604) BK > Insert changelog entry or delete this section. ## Summary & Motivation ## How I Tested These Changes ## Changelog > Insert changelog entry or delete this section. --- .../storage_tests/utils/event_log_storage.py | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py b/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py index e893bad1d38ba..0bbab79ac9b3b 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py +++ b/python_modules/dagster/dagster_tests/storage_tests/utils/event_log_storage.py @@ -504,6 +504,9 @@ def can_set_concurrency_defaults(self): def supports_offset_cursor_queries(self): return True + def supports_get_logs_for_all_runs_by_log_id(self): + return True + def supports_multiple_event_type_queries(self): return True @@ -4142,6 +4145,9 @@ def never_materializes_asset(): assert result.run_id == records[0].asset_entry.last_run_id def test_get_logs_for_all_runs_by_log_id_of_type(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + @op def return_one(_): return 1 @@ -4161,6 +4167,9 @@ def _ops(): ) == [DagsterEventType.RUN_SUCCESS, DagsterEventType.RUN_SUCCESS] def test_get_logs_for_all_runs_by_log_id_by_multi_type(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") @@ -4191,6 +4200,9 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_cursor(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + @op def return_one(_): return 1 @@ -4225,6 +4237,9 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_cursor_multi_type(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") @@ -4269,6 +4284,9 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_limit(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + @op def return_one(_): return 1 @@ -4299,6 +4317,9 @@ def _ops(): ] def test_get_logs_for_all_runs_by_log_id_limit_multi_type(self, storage: EventLogStorage): + if not self.supports_get_logs_for_all_runs_by_log_id(): + pytest.skip("storage does not support get_logs_for_all_runs_by_log_id") + if not self.supports_multiple_event_type_queries(): pytest.skip("storage does not support deprecated multi-event-type queries") From 881f64f234b6964dc2388fe3610c02f121fc7f84 Mon Sep 17 00:00:00 2001 From: jamiedemaria Date: Thu, 19 Dec 2024 11:36:23 -0800 Subject: [PATCH 20/37] 1.9.6 changelog (#26555) ## Summary & Motivation ## How I Tested These Changes --- CHANGES.md | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 911f8fe2c6e34..1d6dddc474e12 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,5 +1,38 @@ # Changelog +## 1.9.6 (core) / 0.25.6 (libraries) + +### New + +- Updated `cronitor` pin to allow versions `>= 5.0.1` to enable use of `DayOfWeek` as 7. Cronitor `4.0.0` is still disallowed. (Thanks, [@joshuataylor](https://github.com/joshuataylor)!) +- Added flag `checkDbReadyInitContainer` to optionally disable db check initContainer. +- [ui] Added Google Drive icon for `kind` tags. (Thanks, [@dragos-pop](https://github.com/dragos-pop)!) +- [ui] Renamed the run lineage sidebar on the Run details page to `Re-executions`. +- [ui] Sensors and schedules that appear in the Runs page are now clickable. +- [ui] Runs targeting assets now show more of the assets in the Runs page. +- [dagster-airbyte] The destination type for an Airbyte asset is now added as a `kind` tag for display in the UI. +- [dagster-gcp] `DataprocResource` now receives an optional parameter `labels` to be attached to Dataproc clusters. (Thanks, [@thiagoazcampos](https://github.com/thiagoazcampos)!) +- [dagster-k8s] Added a `checkDbReadyInitContainer` flag to the Dagster Helm chart to allow disabling the default init container behavior. (Thanks, [@easontm](https://github.com/easontm)!) +- [dagster-k8s] K8s pod logs are now logged when a pod fails. (Thanks, [@apetryla](https://github.com/apetryla)!) +- [dagster-sigma] Introduced `build_materialize_workbook_assets_definition` which can be used to build assets that run materialize schedules for a Sigma workbook. +- [dagster-snowflake] `SnowflakeResource` and `SnowflakeIOManager` both accept `additional_snowflake_connection_args` config. This dictionary of arguments will be passed to the `snowflake.connector.connect` method. This config will be ignored if you are using the `sqlalchemy` connector. +- [helm] Added the ability to set user-deployments labels on k8s deployments as well as pods. + +### Bugfixes + +- Assets with self dependencies and `BackfillPolicy` are now evaluated correctly during backfills. Self dependent assets no longer result in serial partition submissions or disregarded upstream dependencies. +- Previously, the freshness check sensor would not re-evaluate freshness checks if an in-flight run was planning on evaluating that check. Now, the freshness check sensor will kick off an independent run of the check, even if there's already an in flight run, as long as the freshness check can potentially fail. +- Previously, if the freshness check was in a failing state, the sensor would wait for a run to update the freshness check before re-evaluating. Now, if there's a materialization later than the last evaluation of the freshness check and no planned evaluation, we will re-evaluate the freshness check automatically. +- [ui] Fixed run log streaming for runs with a large volume of logs. +- [ui] Fixed a bug in the Backfill Preview where a loading spinner would spin forever if an asset had no valid partitions targeted by the backfill. +- [dagster-aws] `PipesCloudWatchMessageReader` correctly identifies streams which are not ready yet and doesn't fail on `ThrottlingException`. (Thanks, [@jenkoian](https://github.com/jenkoian)!) +- [dagster-fivetran] Column metadata can now be fetched for Fivetran assets using `FivetranWorkspace.sync_and_poll(...).fetch_column_metadata()`. +- [dagster-k8s] The k8s client now waits for the main container to be ready instead of only waiting for sidecar init containers. (Thanks, [@OrenLederman](https://github.com/OrenLederman)!) + +### Documentation + +- Fixed a typo in the `dlt_assets` API docs. (Thanks, [@zilto](https://github.com/zilto)!) + ## 1.9.5 (core) / 0.25.5 (libraries) ### New From f4ca96044f23e8c213f0aa2f81b88eb6df0daad8 Mon Sep 17 00:00:00 2001 From: Isaac Hellendag <2823852+hellendag@users.noreply.github.com> Date: Thu, 19 Dec 2024 13:46:49 -0600 Subject: [PATCH 21/37] [ui] Fix googledrive svg (#26606) ## Summary & Motivation Storybook breaks due to the namespaced tag in the current googledrive svg. Fix it. ## How I Tested These Changes `yarn storybook`, view OpTags. Verify that the icon renders correctly. --- .../ui-core/src/graph/kindtag-images/tool-googledrive-color.svg | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-googledrive-color.svg b/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-googledrive-color.svg index 2d94beff46945..5575d924a946a 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-googledrive-color.svg +++ b/js_modules/dagster-ui/packages/ui-core/src/graph/kindtag-images/tool-googledrive-color.svg @@ -1,3 +1,3 @@ - + From 15bf8946b5bb3f9da8ba8ea6acca019c2ea12442 Mon Sep 17 00:00:00 2001 From: Alex Langenfeld Date: Thu, 19 Dec 2024 14:36:00 -0600 Subject: [PATCH 22/37] [record] kw_only (#26600) add native support for posargs records to help with cases like migirating public APIs ## How I Tested These Changes added tests bk all python versions --- .../dagster/_core/definitions/selector.py | 36 ++------- .../_core/remote_representation/handle.py | 26 ++---- .../dagster/dagster/_core/snap/node.py | 2 +- .../dagster/dagster/_record/__init__.py | 79 +++++++++++++------ .../general_tests/test_record.py | 57 ++++++++++++- 5 files changed, 125 insertions(+), 75 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/selector.py b/python_modules/dagster/dagster/_core/definitions/selector.py index 5bff31e3a57cd..7f3ce30d3b36c 100644 --- a/python_modules/dagster/dagster/_core/definitions/selector.py +++ b/python_modules/dagster/dagster/_core/definitions/selector.py @@ -130,7 +130,7 @@ def repository_selector(self) -> "RepositorySelector": @whitelist_for_serdes @record -class RepositorySelector(IHaveNew): +class RepositorySelector: location_name: str repository_name: str @@ -167,17 +167,10 @@ def from_graphql_input(graphql_data): ) -@record_custom -class CodeLocationSelector(IHaveNew): +@record(kw_only=False) +class CodeLocationSelector: location_name: str - # allow posargs to avoid breaking change - def __new__(cls, location_name: str): - return super().__new__( - cls, - location_name=location_name, - ) - def to_repository_selector(self) -> RepositorySelector: return RepositorySelector( location_name=self.location_name, @@ -318,21 +311,13 @@ def to_graphql_input(self): } -@record_custom -class PartitionRangeSelector(IHaveNew): +@record(kw_only=False) +class PartitionRangeSelector: """The information needed to resolve a partition range.""" start: str end: str - # allow posargs - def __new__(cls, start: str, end: str): - return super().__new__( - cls, - start=start, - end=end, - ) - def to_graphql_input(self): return { "start": self.start, @@ -347,19 +332,12 @@ def from_graphql_input(graphql_data): ) -@record_custom -class PartitionsSelector(IHaveNew): +@record(kw_only=False) +class PartitionsSelector: """The information needed to define selection partitions.""" ranges: Sequence[PartitionRangeSelector] - # allow posargs - def __new__(cls, ranges: Sequence[PartitionRangeSelector]): - return super().__new__( - cls, - ranges=ranges, - ) - def to_graphql_input(self): return {"ranges": [partition_range.to_graphql_input() for partition_range in self.ranges]} diff --git a/python_modules/dagster/dagster/_core/remote_representation/handle.py b/python_modules/dagster/dagster/_core/remote_representation/handle.py index cc59a21ef5bc2..4807570492c8b 100644 --- a/python_modules/dagster/dagster/_core/remote_representation/handle.py +++ b/python_modules/dagster/dagster/_core/remote_representation/handle.py @@ -11,7 +11,7 @@ RegisteredCodeLocationOrigin, RemoteRepositoryOrigin, ) -from dagster._record import IHaveNew, record, record_custom +from dagster._record import record from dagster._serdes.serdes import whitelist_for_serdes if TYPE_CHECKING: @@ -84,19 +84,11 @@ def for_test( ) -@record_custom -class JobHandle(IHaveNew): +@record(kw_only=False) +class JobHandle: job_name: str repository_handle: RepositoryHandle - # allow posargs - def __new__(cls, job_name: str, repository_handle: RepositoryHandle): - return super().__new__( - cls, - job_name=job_name, - repository_handle=repository_handle, - ) - def to_string(self): return f"{self.location_name}.{self.repository_name}.{self.job_name}" @@ -123,19 +115,11 @@ def to_selector(self) -> JobSubsetSelector: ) -@record_custom -class InstigatorHandle(IHaveNew): +@record(kw_only=False) +class InstigatorHandle: instigator_name: str repository_handle: RepositoryHandle - # allow posargs - def __new__(cls, instigator_name: str, repository_handle: RepositoryHandle): - return super().__new__( - cls, - instigator_name=instigator_name, - repository_handle=repository_handle, - ) - @property def repository_name(self) -> str: return self.repository_handle.repository_name diff --git a/python_modules/dagster/dagster/_core/snap/node.py b/python_modules/dagster/dagster/_core/snap/node.py index e56fe51fc32dd..2da0681b7a665 100644 --- a/python_modules/dagster/dagster/_core/snap/node.py +++ b/python_modules/dagster/dagster/_core/snap/node.py @@ -208,7 +208,7 @@ def get_output_snap(self, name: str) -> OutputDefSnap: }, ) @record -class NodeDefsSnapshot(IHaveNew): +class NodeDefsSnapshot: op_def_snaps: Sequence[OpDefSnap] graph_def_snaps: Sequence[GraphDefSnap] diff --git a/python_modules/dagster/dagster/_record/__init__.py b/python_modules/dagster/dagster/_record/__init__.py index 58e1c5c1e07ed..66d819823a76c 100644 --- a/python_modules/dagster/dagster/_record/__init__.py +++ b/python_modules/dagster/dagster/_record/__init__.py @@ -35,6 +35,7 @@ _NAMED_TUPLE_BASE_NEW_FIELD = "__nt_new__" _REMAPPING_FIELD = "__field_remap__" _ORIGINAL_CLASS_FIELD = "__original_class__" +_KW_ONLY_FIELD = "__kw_only__" _sample_nt = namedtuple("_canary", "x") @@ -44,10 +45,12 @@ def _get_field_set_and_defaults( cls: Type, + kw_only: bool, ) -> Tuple[Mapping[str, Any], Mapping[str, Any]]: field_set = getattr(cls, "__annotations__", {}) defaults = {} + last_defaulted_field = None for name in field_set.keys(): if hasattr(cls, name): attr_val = getattr(cls, name) @@ -57,11 +60,9 @@ def _get_field_set_and_defaults( f"Conflicting non-abstract @property for field {name} on record {cls.__name__}." "Add the the @abstractmethod decorator to make it abstract.", ) - elif isinstance(attr_val, _tuple_getter_type): - # When doing record inheritance, filter out tuplegetters from parents. - # This workaround only seems needed for py3.8 - continue - else: + # When doing record inheritance, filter out tuplegetters from parents. + # This workaround only seems needed for py3.9 + elif not isinstance(attr_val, _tuple_getter_type): check.invariant( not inspect.isfunction(attr_val), f"Conflicting function for field {name} on record {cls.__name__}. " @@ -69,11 +70,25 @@ def _get_field_set_and_defaults( "you will have to override __new__.", ) defaults[name] = attr_val + last_defaulted_field = name + continue + + # fall through here means no default set + if last_defaulted_field and not kw_only: + check.failed( + "Fields without defaults cannot appear after fields with default values. " + f"Field {name} has no default after {last_defaulted_field} with default value." + ) for base in cls.__bases__: if is_record(base): original_base = getattr(base, _ORIGINAL_CLASS_FIELD) - base_field_set, base_defaults = _get_field_set_and_defaults(original_base) + base_kw_only = getattr(base, _KW_ONLY_FIELD) + check.invariant( + kw_only == base_kw_only, + "Can not inherit from a parent @record with different kw_only setting.", + ) + base_field_set, base_defaults = _get_field_set_and_defaults(original_base, kw_only) field_set = {**base_field_set, **field_set} defaults = {**base_defaults, **defaults} @@ -87,13 +102,14 @@ def _namedtuple_record_transform( with_new: bool, decorator_frames: int, field_to_new_mapping: Optional[Mapping[str, str]], + kw_only: bool, ) -> TType: """Transforms the input class in to one that inherits a generated NamedTuple base class and: * bans tuple methods that don't make sense for a record object * creates a run time checked __new__ (optional). """ - field_set, defaults = _get_field_set_and_defaults(cls) + field_set, defaults = _get_field_set_and_defaults(cls, kw_only) base = NamedTuple(f"_{cls.__name__}", field_set.items()) nt_new = base.__new__ @@ -109,7 +125,8 @@ def _namedtuple_record_transform( field_set, defaults, eval_ctx, - 1 if with_new else 0, + new_frames=1 if with_new else 0, + kw_only=kw_only, ) elif defaults: # allow arbitrary ordering of default values by generating a kwarg only __new__ impl @@ -120,7 +137,7 @@ def _namedtuple_record_transform( lazy_imports={}, ) generated_new = eval_ctx.compile_fn( - _build_defaults_new(field_set, defaults), + _build_defaults_new(field_set, defaults, kw_only), _DEFAULTS_NEW, ) @@ -145,6 +162,7 @@ def _namedtuple_record_transform( _NAMED_TUPLE_BASE_NEW_FIELD: nt_new, _REMAPPING_FIELD: field_to_new_mapping or {}, _ORIGINAL_CLASS_FIELD: cls, + _KW_ONLY_FIELD: kw_only, "__reduce__": _reduce, # functools doesn't work, so manually update_wrapper "__module__": cls.__module__, @@ -219,6 +237,7 @@ def record( def record( *, checked: bool = True, + kw_only: bool = True, ) -> Callable[[TType], TType]: ... # Overload for using decorator used with args. @@ -230,11 +249,13 @@ def record( cls: Optional[TType] = None, *, checked: bool = True, + kw_only: bool = True, ) -> Union[TType, Callable[[TType], TType]]: """A class decorator that will create an immutable record class based on the defined fields. Args: - checked: Whether or not to generate runtime type checked construction. + checked: Whether or not to generate runtime type checked construction (default True). + kw_only: Whether or not the generated __new__ is kwargs only (default True). """ if cls: return _namedtuple_record_transform( @@ -243,6 +264,7 @@ def record( with_new=False, decorator_frames=1, field_to_new_mapping=None, + kw_only=kw_only, ) else: return partial( @@ -251,6 +273,7 @@ def record( with_new=False, decorator_frames=0, field_to_new_mapping=None, + kw_only=kw_only, ) @@ -303,6 +326,7 @@ def __new__(cls, name: Optional[str] = None) with_new=True, decorator_frames=1, field_to_new_mapping=field_to_new_mapping, + kw_only=True, ) else: return partial( @@ -311,6 +335,7 @@ def __new__(cls, name: Optional[str] = None) with_new=True, decorator_frames=0, field_to_new_mapping=field_to_new_mapping, + kw_only=True, ) @@ -429,12 +454,14 @@ def __init__( defaults: Mapping[str, Any], eval_ctx: EvalContext, new_frames: int, + kw_only: bool, ): self._field_set = field_set self._defaults = defaults self._eval_ctx = eval_ctx self._new_frames = new_frames # how many frames of __new__ there are self._compiled = False + self._kw_only = kw_only def __call__(self, cls, *args, **kwargs): if _do_defensive_checks(): @@ -470,7 +497,11 @@ def __call__(self, cls, *args, **kwargs): return compiled_fn(cls, *args, **kwargs) def _build_checked_new_str(self) -> str: - kw_args_str, set_calls_str = build_args_and_assignment_strs(self._field_set, self._defaults) + args_str, set_calls_str = build_args_and_assignment_strs( + self._field_set, + self._defaults, + self._kw_only, + ) check_calls = [] for name, ttype in self._field_set.items(): call_str = build_check_call_str( @@ -487,7 +518,7 @@ def _build_checked_new_str(self) -> str: ) checked_new_str = f""" -def __checked_new__(cls{kw_args_str}): +def __checked_new__(cls{args_str}): {lazy_imports_str} {set_calls_str} return cls.{_NAMED_TUPLE_BASE_NEW_FIELD}( @@ -501,9 +532,10 @@ def __checked_new__(cls{kw_args_str}): def _build_defaults_new( field_set: Mapping[str, Type], defaults: Mapping[str, Any], + kw_only: bool, ) -> str: """Build a __new__ implementation that handles default values.""" - kw_args_str, set_calls_str = build_args_and_assignment_strs(field_set, defaults) + kw_args_str, set_calls_str = build_args_and_assignment_strs(field_set, defaults, kw_only) assign_str = ",\n ".join([f"{name}={name}" for name in field_set.keys()]) return f""" def __defaults_new__(cls{kw_args_str}): @@ -518,39 +550,40 @@ def __defaults_new__(cls{kw_args_str}): def build_args_and_assignment_strs( field_set: Mapping[str, Type], defaults: Mapping[str, Any], + kw_only: bool, ) -> Tuple[str, str]: """Utility funciton shared between _defaults_new and _checked_new to create the arguments to the function as well as any assignment calls that need to happen. """ - kw_args = [] + args = [] set_calls = [] for arg in field_set.keys(): if arg in defaults: default = defaults[arg] if default is None: - kw_args.append(f"{arg} = None") + args.append(f"{arg} = None") # dont share class instance of default empty containers elif default == []: - kw_args.append(f"{arg} = None") + args.append(f"{arg} = None") set_calls.append(f"{arg} = {arg} if {arg} is not None else []") elif default == {}: - kw_args.append(f"{arg} = None") + args.append(f"{arg} = None") set_calls.append(f"{arg} = {arg} if {arg} is not None else {'{}'}") # fallback to direct reference if unknown else: - kw_args.append(f"{arg} = {_INJECTED_DEFAULT_VALS_LOCAL_VAR}['{arg}']") + args.append(f"{arg} = {_INJECTED_DEFAULT_VALS_LOCAL_VAR}['{arg}']") else: - kw_args.append(arg) + args.append(arg) - kw_args_str = "" - if kw_args: - kw_args_str = f", *, {', '.join(kw_args)}" + args_str = "" + if args: + args_str = f", {'*,' if kw_only else ''} {', '.join(args)}" set_calls_str = "" if set_calls: set_calls_str = "\n ".join(set_calls) - return kw_args_str, set_calls_str + return args_str, set_calls_str def _banned_iter(*args, **kwargs): diff --git a/python_modules/dagster/dagster_tests/general_tests/test_record.py b/python_modules/dagster/dagster_tests/general_tests/test_record.py index 08af55d1fc1dc..8d15cfa20ca4a 100644 --- a/python_modules/dagster/dagster_tests/general_tests/test_record.py +++ b/python_modules/dagster/dagster_tests/general_tests/test_record.py @@ -364,7 +364,7 @@ class OtherSample: def test_build_args_and_assign(fields, defaults, expected): # tests / documents shared utility fn # don't hesitate to delete this upon refactor - assert build_args_and_assignment_strs(fields, defaults) == expected + assert build_args_and_assignment_strs(fields, defaults, kw_only=True) == expected @record @@ -830,3 +830,58 @@ def boop(self): def test_defensive_checks_running(): # make sure we have enabled defensive checks in test, ideally as broadly as possible assert os.getenv("DAGSTER_RECORD_DEFENSIVE_CHECKS") == "true" + + +def test_allow_posargs(): + @record(kw_only=False) + class Foo: + a: int + + assert Foo(2) + + @record(kw_only=False) + class Bar: + a: int + b: int + c: int = 4 + + assert Bar(1, 2) + + with pytest.raises(CheckError): + + @record(kw_only=False) + class Baz: + a: int = 4 + b: int # type: ignore # good job type checker + + +def test_posargs_inherit(): + @record(kw_only=False) + class Parent: + name: str + + @record(kw_only=False) + class Child(Parent): + parent: Parent + + p = Parent("Alex") + assert p + c = Child("Lyra", p) + assert c + + # test kw_only not being aligned + with pytest.raises(CheckError): + + @record + class Bad(Parent): + other: str + + with pytest.raises(CheckError): + + @record + class A: + a: int + + @record(kw_only=False) + class B(A): + b: int From 0b4099f6641533129cff991e9276483a3c61e796 Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:49:43 -0500 Subject: [PATCH 23/37] [docs-beta] migrate - kubernetes agent docs (#26505) ## Summary & Motivation Broken links converted to `/todo`: ``` - Broken link on source page path = /dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration: -> linking to /dagster-plus/deployment/agents/kubernetes/configuring-running-kubernetes-agent -> linking to /dagster-plus/managing-deployments/code-locations -> linking to /dagster-plus/managing-deployments/dagster-cloud-yaml -> linking to /dagster-plus/managing-deployments/environment-variables-and-secrets -> linking to /guides/dagster/using-environment-variables-and-secrets -> linking to /deployment/guides/kubernetes/customizing-your-deployment#per-job-kubernetes-configuration -> linking to /deployment/guides/aws#using-s3-for-io-management ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../hybrid/kubernetes/configuration.md | 185 +++++++++++++++++- 1 file changed, 184 insertions(+), 1 deletion(-) diff --git a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md index f3053801d3c1c..44932d1d246ac 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/kubernetes/configuration.md @@ -4,4 +4,187 @@ sidebar_position: 200 unlisted: true --- -{/* TODO copy from https://docs.dagster.io/dagster-plus/deployment/agents/kubernetes/configuration-reference */} \ No newline at end of file +:::note +This guide is applicable to Dagster+. +::: + +{/* This reference describes the various configuration options Dagster+ currently supports for [Kubernetes agents](/dagster-plus/deployment/agents/kubernetes/configuring-running-kubernetes-agent). */} +This reference describes the various configuration options Dagster+ currently supports for [Kubernetes agents](/todo). + +## Viewing the Helm chart + +To see the different customizations that can be applied to the Kubernetes agent, you can view the chart's default values: + +```shell +helm repo add dagster-plus https://dagster-io.github.io/helm-user-cloud +helm repo update +helm show values dagster-plus/dagster-plus-agent +``` + +You can also view the chart values on [ArtifactHub](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values). + +## Per-deployment configuration + +The [`workspace`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value of the Helm chart provides the ability to add configuration for all jobs that are spun up by the agent, across all repositories. To add secrets or mounted volumes to all Kubernetes Pods, you can specify your desired configuration under this value. + +Additionally, the [`imagePullSecrets`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value allows you to specify a list of secrets that should be included when pulling the images for your containers. + +## Per-location configuration + +{/* When [adding a code location](/dagster-plus/managing-deployments/code-locations) to Dagster+ with a Kubernetes agent, you can use the `container_context` key on the location configuration to add additional Kubernetes-specific configuration. If you're using the Dagster+ Github action, the `container_context` key can also be set for each location in your `dagster_cloud.yaml` file, using the same format. */} +When [adding a code location](/todo) to Dagster+ with a Kubernetes agent, you can use the `container_context` key on the location configuration to add additional Kubernetes-specific configuration. If you're using the Dagster+ Github action, the `container_context` key can also be set for each location in your `dagster_cloud.yaml` file, using the same format. + +{/* The following example [`dagster_cloud.yaml`](/dagster-plus/managing-deployments/dagster-cloud-yaml) file illustrates the available fields: */} +The following example [`dagster_cloud.yaml`](/todo) file illustrates the available fields: + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + k8s: + env_config_maps: + - my_config_map + env_secrets: + - my_secret + env_vars: + - FOO_ENV_VAR=foo_value + - BAR_ENV_VAR + image_pull_policy: Always + image_pull_secrets: + - name: my_image_pull_secret + labels: + my_label_key: my_label_value + namespace: my_k8s_namespace + service_account_name: my_service_account_name + volume_mounts: + - mount_path: /opt/dagster/test_mount_path/volume_mounted_file.yaml + name: test-volume + sub_path: volume_mounted_file.yaml + volumes: + - name: test-volume + config_map: + name: test-volume-configmap + server_k8s_config: # Raw kubernetes config for code servers launched by the agent + pod_spec_config: # Config for the code server pod spec + node_selector: + disktype: standard + pod_template_spec_metadata: # Metadata for the code server pod + annotations: + mykey: myvalue + deployment_metadata: # Metadata for the code server deployment + annotations: + mykey: myvalue + service_metadata: # Metadata for the code server service + annotations: + mykey: myvalue + container_config: # Config for the main dagster container in the code server pod + resources: + limits: + cpu: 100m + memory: 128Mi + run_k8s_config: # Raw kubernetes config for runs launched by the agent + pod_spec_config: # Config for the run's PodSpec + node_selector: + disktype: ssd + container_config: # Config for the main dagster container in the run pod + resources: + limits: + cpu: 500m + memory: 1024Mi + pod_template_spec_metadata: # Metadata for the run pod + annotations: + mykey: myvalue + job_spec_config: # Config for the Kubernetes job for the run + ttl_seconds_after_finished: 7200 + job_metadata: # Metadata for the Kubernetes job for the run + annotations: + mykey: myvalue +``` + +### Environment variables and secrets + +Using the `container_context.k8s.env_vars` and `container_context.k8s.env_secrets` properties, you can specify environment variables and secrets for a specific code location. For example: + +```yaml +# dagster_cloud.yaml + +location: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + k8s: + env_vars: + - database_name + - database_username=hooli_testing + env_secrets: + - database_password +``` + + | Property | Description | + |---------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | `env_vars` | A list of environment variable names to inject into the job, formatted as `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the current process. | + | `env_secrets` | A list of secret names, from which environment variables for a job are drawn using `envFrom`. Refer to the [Kubernetes documentation](https://kubernetes.io/docs/tasks/inject-data-application/distribute-credentials-secure/#configure-all-key-value-pairs-in-a-secret-as-container-environment-variables) for more info. | + +Refer to the following guides for more info about environment variables: + +{/* - [Dagster+ environment variables and secrets](/dagster-plus/managing-deployments/environment-variables-and-secrets) */} +- [Dagster+ environment variables and secrets](/todo) +{/* - [Using environment variables and secrets in Dagster code](/guides/dagster/using-environment-variables-and-secrets) */} +- [Using environment variables and secrets in Dagster code](/todo) + +## Op isolation + +By default, each Dagster job will run in its own Kubernetes pod, with each op running in its own subprocess within the pod. + +You can also configure your Dagster job with the [`k8s_job_executor`](https://docs.dagster.io/\_apidocs/libraries/dagster-k8s#dagster_k8s.k8s_job_executor) to run each op in its own Kubernetes pod. For example: + +```python +from dagster import job +from dagster_k8s import k8s_job_executor + +@job(executor_def=k8s_job_executor) +def k8s_job(): + ... +``` + +## Per-job and per-op configuration + +{/* To add configuration to specific Dagster jobs, ops, or assets, use the `dagster-k8s/config` tag. For example, to specify that a job should have certain resource limits when it runs. Refer to [Customizing your Kubernetes deployment for Dagster Open Source](/deployment/guides/kubernetes/customizing-your-deployment#per-job-kubernetes-configuration) for more info. */} +To add configuration to specific Dagster jobs, ops, or assets, use the `dagster-k8s/config` tag. For example, to specify that a job should have certain resource limits when it runs. Refer to [Customizing your Kubernetes deployment for Dagster Open Source](/todo) for more info. + +## Running as a non-root user + +Starting in 0.14.0, the provided `docker.io/dagster/dagster-cloud-agent` image offers a non-root user with id `1001`. To run the agent with this user, you can specify the [`dagsterCloudAgent`](https://artifacthub.io/packages/helm/dagster-cloud/dagster-cloud-agent?modal=values) value in the Helm chart to be: + +```yaml +dagsterCloudAgent: + podSecurityContext: + runAsUser: 1001 +``` + +We plan to make this user the default in a future release. + +## Grant AWS permissions + +{/* You can provide your Dagster pods with [permissions to assume an AWS IAM role](https://docs.aws.amazon.com/eks/latest/userguide/iam-roles-for-service-accounts.html) using a [Service Account](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). For example, you might do this to [configure an S3 IO Manager](/deployment/guides/aws#using-s3-for-io-management). */} +You can provide your Dagster pods with [permissions to assume an AWS IAM role](https://docs.aws.amazon.com/eks/latest/userguide/iam-roles-for-service-accounts.html) using a [Service Account](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). For example, you might do this to [configure an S3 IO Manager](/todo). + +1. [Create an IAM OIDC provider for your EKS cluster](https://docs.aws.amazon.com/eks/latest/userguide/enable-iam-roles-for-service-accounts.html) +2. [Create an IAM role and and attach IAM policies](https://docs.aws.amazon.com/eks/latest/userguide/associate-service-account-role.html) +3. Update the [ Helm chart](#viewing-the-helm-chart) to associate the IAM role with a service account: + + ```bash + serviceAccount: + create: true + annotations: + eks.amazonaws.com/role-arn: "arn:aws:iam::1234567890:role/my_service_account_role" + ``` + +This will allow your agent and the pods it creates to assume the `my_service_account_role` IAM role. From 4b55bea797d2c7cbf54f72b267c77dbf4c0f8b13 Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:50:27 -0500 Subject: [PATCH 24/37] [docs-beta] migrate - rate limits doc (#26502) ## Summary & Motivation Broken links converted to `/todo`: ``` - Broken link on source page path = /dagster-plus/deployment/management/rate-limits: -> linking to /concepts/logging#structured-event-logs -> linking to /concepts/logging#raw-compute-logs ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../dagster-plus/deployment/management/rate-limits.md | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/rate-limits.md b/docs/docs-beta/docs/dagster-plus/deployment/management/rate-limits.md index b4364e74ff276..7c85eb8fdaa11 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/rate-limits.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/rate-limits.md @@ -1,6 +1,13 @@ --- title: Dagster+ rate limits -unlisted: true --- -{/* TODO move from https://docs.dagster.io/dagster-plus/references/limits */} \ No newline at end of file +Dagster+ enforces several rate limits to smoothly distribute the load. Deployments are limited to: + +- 40,000 user log events (e.g, `context.log.info`) per minute. This limit only applies to custom logs; system events like the ones that drive orchestration or materialize assets are not subject to this limit. +- 35MB of events per minute. This limit applies to both custom events and system events. + +Rate-limited requests return a "429 - Too Many Requests" response. Dagster+ agents automatically retry these requests. + +{/* Switching from [Structured event logs](/concepts/logging#structured-event-logs) to [Raw compute logs](/concepts/logging#raw-compute-logs) or reducing your custom log volume can help you stay within these limits. */} +Switching from [Structured event logs](/todo) to [Raw compute logs](/todo) or reducing your custom log volume can help you stay within these limits. From 1cb5882541f8de6698b5ca15a0d635e0f9e0b31b Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:52:37 -0500 Subject: [PATCH 25/37] [docs-beta] migrate - managing compute logs and error messages (#26500) ## Summary & Motivation Broken links marked `/todo`: ``` - Broken link on source page path = /dagster-plus/deployment/management/managing-compute-logs-and-error-messages: -> linking to /concepts/logging#log-types -> linking to /dagster-plus/deployment/agents/customizing-configuration -> linking to /deployment/dagster-instance#compute-log-storage ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- ...anaging-compute-logs-and-error-messages.md | 104 +++++++++++++++++- .../configure-redact-env-var.png | Bin 0 -> 87970 bytes .../masked-err-message.png | Bin 0 -> 52841 bytes 3 files changed, 103 insertions(+), 1 deletion(-) create mode 100644 docs/docs-beta/static/images/dagster-cloud/logs-error-messages/configure-redact-env-var.png create mode 100644 docs/docs-beta/static/images/dagster-cloud/logs-error-messages/masked-err-message.png diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md b/docs/docs-beta/docs/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md index 01279273021f2..ec61dc4fbbee3 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/managing-compute-logs-and-error-messages.md @@ -3,4 +3,106 @@ title: Managing compute logs and error messages unlisted: true --- -{/* TODO move from https://docs.dagster.io/dagster-plus/managing-deployments/controlling-logs */} \ No newline at end of file +import ThemedImage from '@theme/ThemedImage'; + +:::note +This guide is applicable to Dagster+. +::: + +In this guide, we'll cover how to adjust where Dagster+ compute logs are stored and manage masking of error messages in the Dagster+ UI. + +{/* By default, Dagster+ ingests [structured event logs and compute logs](/concepts/logging#log-types) from runs and surfaces error messages from [code locations](/guides/deploy/code-locations/) in the UI. */} +By default, Dagster+ ingests [structured event logs and compute logs](/todo) from runs and surfaces error messages from [code locations](/todo) in the UI. + +Depending on your organization's needs, you may want to retain these logs in your own infrastructure or mask error message contents. + +--- + +## Modifying compute log storage + +Dagster's compute logs are handled by the configured [`ComputeLogManager`](/api/python-api/internals#compute-log-manager). By default, Dagster+ utilizes the `CloudComputeLogManager` which stores logs in a Dagster+-managed Amazon S3 bucket, but you can customize this behavior to store logs in a destination of your choice. + +### Writing to your own S3 bucket + +If using the Kubernetes agent, you can instead forward logs to your own S3 bucket by using the [`S3ComputeLogManager`](/api/python-api/libraries/dagster-aws#dagster_aws.s3.S3ComputeLogManager). + +{/* You can configure the `S3ComputeLogManager` in your [`dagster.yaml` file](/dagster-plus/deployment/agents/customizing-configuration): */} +You can configure the `S3ComputeLogManager` in your [`dagster.yaml` file](/todo): + +```yaml +compute_logs: + module: dagster_aws.s3.compute_log_manager + class: S3ComputeLogManager + config: + show_url_only: true + bucket: your-compute-log-storage-bucket + region: your-bucket-region +``` + +If you are using Helm to deploy the Kubernetes agent, you can provide the following configuration in your `values.yaml` file: + +```yaml +computeLogs: + enabled: true + custom: + module: dagster_aws.s3.compute_log_manager + class: S3ComputeLogManager + config: + show_url_only: true + bucket: your-compute-log-storage-bucket + region: your-bucket-region +``` + +### Disabling compute log upload + +If your organization has its own logging solution which ingests `stdout` and `stderr` from your compute environment, you may want to disable compute log upload entirely. You can do this with the . + +{/* You can configure the `NoOpComputeLogManager` in your [`dagster.yaml` file](/dagster-plus/deployment/agents/customizing-configuration): */} +You can configure the `NoOpComputeLogManager` in your [`dagster.yaml` file](/todo): + +```yaml +compute_logs: + module: dagster.core.storage.noop_compute_log_manager + class: NoOpComputeLogManager +``` + +If you are using Helm to deploy the Kubernetes agent, use the `enabled` flag to disable compute log upload: + +```yaml +computeLogs: + enabled: false +``` + +### Other compute log storage options + +{/* For a full list of available compute log storage options, refer to the [Compute log storage docs](/deployment/dagster-instance#compute-log-storage). */} +For a full list of available compute log storage options, refer to the [Compute log storage docs](/todo). + +--- + +## Masking error messages + +By default, Dagster+ surfaces error messages from your code locations in the UI, including when runs fail, sensors or schedules throw an exception, or code locations fail to load. You can mask these error messages in the case that their contents are sensitive. + +To mask error messages in a Dagster+ Deployment, set the environment variable `DAGSTER_REDACT_USER_CODE_ERRORS` equal to `1` using the [**Environment variables** page](/dagster-plus/deployment/management/environment-variables/) in the UI: + + + +Once set, error messages from your code locations will be masked in the UI. A unique error ID will be generated, which you can use to look up the error message in your own logs. This error ID will appear in place of the error message in UI dialogs or in a run's event logs. + + + diff --git a/docs/docs-beta/static/images/dagster-cloud/logs-error-messages/configure-redact-env-var.png b/docs/docs-beta/static/images/dagster-cloud/logs-error-messages/configure-redact-env-var.png new file mode 100644 index 0000000000000000000000000000000000000000..4efe4e0b654a9a5000c3f988e6bcc27240b2b7d8 GIT binary patch literal 87970 zcmeFZRZ!gB);@?6LI?y11a}C*g1ZF>E`i_{+}&LhAZT!RcWAt^KyV8V4UGgEcXy^a z=Y0SA-khGQxtXfDn2S&}-MxQnuf6nH&rX<sa2V23;>vJv$j{*55SgB# zK>sp1o<0r-_d>`*OiW2yOpIK~(azMu+5`?xDl9=ARYPR}FH<`1&7jr*LRoO(vq$~s zgoXVm$^JNp+T=vPF-#F+Uqj6(@31g^Vd2=UDW<`S-&}%(j{eJIvEp?GMB(SYMIp&uVhzq|I9=IygGM*tgGZSX0nxg<;~F(tTst393J8I=Nfbsb)ZU z#bBW%sq~)D(w^B{)IQYkK%-G4K0bTunw{pfMW8(2VmT$;Ci ze+ItVi@rNMmUA$$&ZSgrVgoKC^)0GHM8%RuUy<2hkRsDOWQYSKY z3^G#Zr-2tyQa>(3S6Jb3gM=xt{R49xL{IeK-v-{zQm8pNBc+@yQC^4Y9O3@)#zH$; zssAh_!-300AP^97L^j@Bef*`}qk7=xQBb(w*GwZOOqp%sXXuM_QANXNtjTW_UeJJ6 zzcmRTgiHQ3BDb#O!MP+&?{o9(Qu)BY`f3#=C(hn0(7tlLh*2^?p_r_+T%tF+zOy2y z5S29eR@9|9V)xIt&R1?B4jS%o5qt2O4QzIrs&9lX9&J#B7cmOR=2IeK&-(+67qTQL z_AE%n-qmV7YLH1(5{R9fVxHd-XWfgPUmp=)1LzvC_(LU~q>~|YY zL=z?{g-aLiij(<{OU%rVwW9rjM92cA6G;P3n}pS&e-bMp40f_r6PBZdjFARPC$G?eE?iU%FXl{%<^=oO#TC7);5`|Q!F>J=jwWRZbpQ&ztQ1#QVBTPgs zek7$7?6+FS`JQ80%w-v&l}DvSJNSWWT-=#B^{c{%@bT*{Y-c8|uLmD@f2(ZKoS}fi z+C^?z{Ywm6k_k3KSy{R8f(i^$l9AHF5)2_4uhqgiyKcQPc7J)+MDx5VT6?khs&d6% z0Bv(M?kMHT&|9z_r>a}~$m&Y074z|9Hg?3bmq=s?A@5UTB$DOcQpDpl;Jm~c?KCS1 zKZ=r~-odsB)jQIZquIbJdWDa}jdh8wh`kUN9d=KRJ>b7CazfP;)0F=~8O?&kg1aQl zIbllTCOfy(u=GzUc4=%OrE*G{a%HV7BzRCY!R%eu0HHBMEhg``VLC+`78*M$?pT&M zV1E{UDlXA?GMXQ;Vku&pvLyxSIg5F%pHx*j)z8%;RbH2ls_B-sD4w_Oh)qXZYFS=b zTG!WDD%aCk@=qz0g4Mjl3Y05K0;V`DDO}L%Q$15}{BHt27V(bra0uDpRRpR@0%g2P z6x14(%}S!xS=3n6g^PVP+th=~4Jv-ltXK|D760bR9nTrBlAd)Qj~E}G)hTb5Jq>sG@5U$Gf)J z+1b#!<5F!{t=Mm^ZH+RGg=W30u6@wIBTrIJ8YV~pB(e%hAa94Z6FC#z%KO#i%B;&c z)i%qQv==mrG+3%D8xI=B8&~IW7pCSG0RAQ-M2#^H7Bq9cd;C%DK#K)*2Tv{#O=yzmvEO+qU%HPF>aLkr0Z9|?7y|SQ6jOO9~ zlLiQJ$~@)T1Tt$ezhZXMCDhdef`BJF3_4Z7Pv8$=(`L@5Hb*ym{3bi+4v6UG8%RxO z-uzlwQBu(_zzm>oXo3M;Hc=K-G7@3JY?^sV>yGL%b|SQKKi)T4^n25POus#lX=k3xoXterEofaNODNRz77*kPF*;t*|GOFoe{nowN1x zN?|+mYzZ!e2nPAEy+MD8&WvG;KK@+#xhB~O`IG<$Lmh1$%P^I-*4%*o2Y2JElcSeW ztRa0R8)p*k62qb9VV}e0LUqWl-wq1NU{CWrCwa|k2`Tm(R~ygWx_N`u9i~a(Bs@dw zs=PPOW2j@TBe6K?o^-W+yotSqhJ)=&(<=KyMkm#sdw@$ibt=6)Q-G_P@7hPke(-zz z=s4BKu{S>S)9+Q2K`|d?WEB=NXM|~uhj&QEk} zud`d6=L<>a8+*Ykzpu>BQ6hTiZq0gnSP}whHPQ5AT zsk>^lxZ+r<+t@ap+?4a<3F0&wk~22#9p2Ey18j`#q>^xXZ4wWrFYlrs99gP}7j0d*fScL8}>NYwh(0b8xO*!QDi-$*2j$L?7=IPycQj zGF#vAl5s$1qG8HuyVvBGNv;WZ5-Q_z^9Lu1SzPGc%OLEA zg37^`Qf~$atMQqlh1kZ7+sJF}bJy;DlZB20to^M;f)1!Bm9=;#jaL?Mnn%_ZnqAI; z)i00^Es9;r!L{mccJo5jiYnBt`7Lj3|7Ui~{t5QCf53BG(=OI>{*S06Jm zOR?J@p;RN0?U5nTd?a9p;XREWhm18@MJ0t{Vbi0f)Aa`?7CuR7BN~WHeqiw%BcgeO-#SMmK?g!&e6h zOS4V-51m`0(`2rGd;4d50uS%640iK^CEdtQNjuz?kJe>p?~101AY*b_s%}76vR$;xbXa0&!GaD2-RgKBMq@Fv&MIQBUQl`&>+I6O8@i z1IQ~Z=J?KUdAV)VInyKSR$xqitSyVIaZ{kYT5s1J8`VonP1iz z9j{|Dvb^cMH|xC6slH_Ufyu;oOV^!RV73)vtcnJpUgu_jH~vgk837&%O9bwp zKUOqeOYuHj8J+)*0y$U+FqN)j}|sh9uec8nmM271~e1);q=p2`h;I zeCEGIAm4pX9QND<>?C(NZ1_Edmx$meG2w=+P17dIhFm7b()K9W1WfA!B&Hoh8!za+ zk!z2HPo-3N@%z`Evb9H>OvxufVB4hS`+DpOrJ#@vDkfxRt06Hu63wKBfX5IO#*@t- zGL%V1ObE}&{o(%kqqBxZqoo~PT zAvX4dpF_}x4}Tr)zh15+<6Yez`WZhy+}C=Z6JD=RLM-|`sV~g5Ut6w^BcZ7{SK#0} zrxB&PNIb{?@p4~V`uLAT^mZI%<-bhjU+Z=1ixAzsW>EV~$XQCqY56&Ue$sTX-RD*w zQQSs;YunM9GN{iZCu!JTcb4Y0N z_DV)Uee+-I9(pYVlWm56-d&&4fCn5q|N9i5w28%-UDNJr!`ztn+|vAgea;ulnD_O_ zIitb3k7-$a_)~?DFVz&hFuzM(@83Wz@ObBB4HS(;1|9U$Y9zCrXWZX_F9b&oD1$Tj zGZLrh27glhZ9V@!Me_i{>9T>MEFW8KY)eL^px_4NoCe~l7m>J$EMxi9IoJIXu#SVg z8TmOr^LhjM)Y}{djo4hVSC$jUs4l$f%=|SqBWsxd?VwMq^z$L`0c$cs#6yVd2IQ1; zk40?+lR^H!Qwphf(>crn|FZA@yo5*Kv#5!DwGk74kr56`3<^d;jJf&_>$4(>uch|Y z_t*ov)|_RGxOukf7>8@=M#Csz?LB4$dBV`2i?%o}K9q}&8s=yGMJ{lDOUqcB8Igp% zzs(Rbi-7sq2ApaKk<*aL!1peLhNmMLP1FlN?7LtdmoXW9qp$<0%i&^bQVVP1`}^C) zXX*5?{Obbu_nYiU#6o056~tg69A*vz+q3sxAp05E>w3TwAx^Di_ae#5ScxVRk_`XA z=1{hsNKxhd-f7d`cdWJo0sT`;We`Dck(iv0eFFhbVHi(bO? zeA{@>tOpnQ=e zN&8So@*?IB0}M68A!vT?BtowPyeWDqR$PJFR#v+;@1zA=$5sJTnZH~o3~^p^#KjY% zq3^uIhPmW3V_kSUlwx^gPOfA=lpLpa4svJeD{+|nvU=`r=-=}6G0j>)lJSVjMmg*) zbyBTl49sQZi_b3j_GJb#Qo=H7e~}sNXA$r>$j<;XC5~J$w?Q70DWc{7-en{9>^XoG zMoNRq8LQuEM#rE)67jnVnPLr#_OS=@0;X7@)L3s2<>qT`iq`uL{si-S%YDX{j&gzd zU}Fk+8{sW>;l7k&ax~b_tY1V&jD+?oxHGOmoW>x4Yxb>b)={} zbs7z91M7c@z$LPEdM$|o=niv=f-RI8JRPkRde@#ezWo8LPt5E2!x=PmAVaI6|CYIb z6ancO;BYun)9yX0DsAFema=4+hPX{R&qO9kDJ)W^xN5$07zmg#QvDEd{!p_GswTi?HpmLC3&v(3pRL-%ZvfVZ0 z@B+zbB9q&wjEQz#Sr4nl_Z{cn1Ie9EV;z*y0ypvCtsfzWKNf7;SdCsQ5e87>Z+Svo z*gOVb(5u!4M-ZNW*O@E!ElmG$+2(n7+&3hF&lPGV%%JYe1iajD@%a>c{vcj`hkg*GmE zQ5l|tb=!cw3#L@=#1wYgtLf=h3I3GFV@htjhgJgRjtAreH@sZ@tNlhX<#t1i#`Bdh zx{V}xA~o4|Z;!mH;Mix>HJyKYvic?DwOY)wMS;A4qNZh><`ti+9rgyl!*5Nc9j%bp zY+j?14N+VbEOF8}lD(>AM1R>2^~W#2*~b>_vvF|bKzPAntNrO(?^K26F|Z2r^QFFi zhXlHwO{*`5Hbgj00K_t5K74)7r++Pn^E)xKKsJdDzH4eGw}n`7SMf`nnE&QlzstmB zNW0|2!>wIf$szuKg@iENe&}~vT*l(k>3Q>Nc`3%wJ|%?L|L(>FGhLr-Butmh_MgvQ z#yp1vAqHzV*v8eY(Y65H?~YN?uv!Qu@f_FtDU2jLXvba zIELEF9$+lK#z;>RF~FMPcjZ1LSD5;5Z(nuIOmTw|_H36m-t&cxLYJKlNvimNJmMB9$m*};9=P1^v zGn5^&S#axLu z++XT@toF?A6>X?j>8Uc?j{nl8Huu<`7+a51Oafld>P^Z&$7B4`p_sy_;`_ksQX8zq zwR8>wDbYDxosM<0jXJOD-3*x3T5Z}kpT(Lwc4Mdm&%N-Sx%M; zDsSMM+{KqyX3y~6T(Fz%?~P2`_fhWF;pePS-7ODQbwBhF>}3GD))Ij6a~)@ZP9W@Nwgk88b5h}*m$?r*z=B zlze}97y=neI9J-;&$8A)Pdubb6=W5HVG$E6d#6RI;`Ai$I=_>NZzmXg!=oZE!RLm~Yr~ za-kRPmblirHrM7=C6{tidH3bTneXiV)@|M>a(#u(7mxP8U0F)RjFpt9|1No~XhgA= zn*C=83ynJKrP<+)3+?U^Gf#gDh(Vc5khtrB3oo0ckqN+rR>Q207eGJb22zyed!g%S;a=uVmw0@>4&k!G*2$ zwte>?#OA`K&GvYq(P_^UQ}A|iXtYo?E5IeiKc+2l^kQi&v4ZbbPU-hdjnypaGJ)AN zdb6U7MvbBB?{)-sIk-XfLhI=mmo49g`ooQdY6q1S(*w6dK|kM&Lat?`TW`8`rq{Kj z3hHNP+t2qxhBHl8yV?qy(hW>qWOqR`sX+A(vM^GQm( z6w1=2<wGOn8#Oc^mR&Mck}Ic`n@)V$4Bc=Q?cNTq222x1o|#HbgNL1WLqZx;us@!&&#|dSae;>{5)YIgaO}l6o)d4Jnz_yA-2o z4CRP*$nG}iLV$h%hC!-?fPv)?In~-%)5vE#^s?ApW8l@oC+juokhKyCC^5m zUc}2)@7Y@oFr8fbj$jnhr!yA$?+e>4;4}G>_RMl%r#jsc&X&&6Lw=W&V|pVl?1e-V}K(S^Cz@F#9S@R2C%Z&U|3z7G7xF zukJV+dvvdP;eF#luUne0!q}bFz3l$!<-M*!QX$AI#iK^Bv#qG*9<^ zMT|J2XyJ4$m|DNRb)sduV4~H-sKmDABa7}uO6%>x?aBDvSJyU_WIaD0<2l>LQC}9j z>3NjCsnss@-GwjVqodYZtn&v=S_jb7k-&f8E%Nb&VupprDJZqF!pE*n*7xxq^Zs3& z{1;js?rFE84m!fm4$1=XU2aP$%pelB=#-KcJutesS)&^IKGz;K=Z?Ds=H*rWkV!PD zZod5PkT_kkrX50RcSy8rkdvn`xMi9AbVC7iM`3@eBx`cnM^Ua!oZrf&Of4zdCyNEu zkJmg%n()1&@}W@U!a+*HyNo?!WFJd){faWv5%zqQJ}%|@9Sg30OD*?i(IUDm&l%Uq z8FKj`6qofuW|D$0Ie~0o>#|#?z(68yo5Y`Et#A2&u3skZKIJXB0>=1uofq2LG$+H2 z58HiC#wF2B7PaHb6J}<2zbzO;lETYNfS)?MzF`m*OuklMum$MyH8z4lE*tjy2e&sw zRN(B@Wn5)yiHQUf2b+`B5 z|5)`?d@^_DpiFLi$^c^#LlB*i{BRx+y`Nh?W;-;*(g2RfXVjo1e9y&1Z)0&jwP`f1 zE?XW}^t~9bs{Mv`!DXI)v8xrp%%z)C&wtZMdn9PLgy4m_H5u` z2hpBUVwpYzRS~|dYsJ^^GM&F^tAA*x2G*0YObn z^rBuiivV@9Y7rH%3DwPJ=Qc_?I8?rR+^+4k$7|Uf|F9ExhX8n;2n?*uapSj}UFc-) zEp*9J1is|ooP%}@S!jly+>Jv!*@q}ai}5AwfrIYAH~bK}-HB|+)bYH{NLyX486>fw zL659xCEbX6t9k(0{nRG)XF?(G2z_3h>=&vL0)OR^x5%~D_-mdR%!!j@7g6aKsJ~=3I zTv}Z>g|GSRsbJz5eOzBZf!Sxw<`UP0Ok*tqK506|?dh_j)9J(0c?!iVd9LG|>QBDA zVm>KH7ta=+v)Y4o@9mCvcS_>~^OJg`JanC7iR>m4bwKZi(@_ag3v}>iKRHXe&Ir@1Jvn(aaZz{lCIT3x1)a zLhEl}qT|;eA03J6{mDeU)@4zon&>npxB&L7CHP4G`NWlF<`!rX_tCfKjCZctCu2CB zRg|`aT`QI2LjMr=z+B6lZyR_x)564?(b}olhN|!F`{g(13r73{V8ge&?Y@MQ>F*LJ zr73gk6%KZx?R^Mwd|SUb6|~yDzXIM|sA$Iv9bbu2%yxW<(xfwSh`l6PUoW`4F5On! z%ePh6rMLup0iSe}YE>D#TXnd)(RO!Y@0No6*Xdp%zX|&NX7oK6l0H=m>2X~0*yPa< z5+ZAQ9L;WhHw&V-X&sj*mDp`28mlP;Vt0K7Q?e9H5rIOS9U9(qnD zG2HbLSkIh&p#RjGto@6~-+|3+_^NZ?u@tcHx-&PU`FgSl#g$UG!(!63Sy$WzRWWTv zt0^DwcG+*4q za68knunV>QoZxj5b_81R-?BY+8k&v?vI*q~iRGByG8wnnTj6Wm_XXN-j|oRdE>-7-h+|tgW zIeQNZ4(xl&Urx$-Qt{cA#g?adkM=XzP0Uhn-XZVJ4-=NaK7s! zoNZ=RVNHUU zDrSh3oY}<~w~lu;eh}_~nhXHmwg?@3DEZv?=d;U8RiZ=nf&pvU2@b~QyE-<_K!-`Z z>(!8~^=?dIJodF}Z6nXV0%VuF38|8A1T5oeGb#65O)2BlhhDNnR@rm>QHEbgme!^R z)P`T|QfPakLPh?|DbA3gu)tAVfs=%`y$Q$ui{w`-K&`^0x?yOKHp$>RHDkwK(cM~N z0ie<)ZoAG&biw;0ML!{q)<=@$i})F%jtBk&I7tC%4e2!p5C524;3~CUk}(}iUe?lz zX0c718X@tEX_;YQhQiA~@=N<|*Db^YGP9Cv1rQFCeCr)Nf?M_@flR`y^ex3YX>Daa zPia6*59fP=0Zzae)0C)%lD_tX#TTVXTz2gmZjs`jR&*A!tx}@Q7%di;);3|q6^I!# zz|jkWd%45R$OfVNz5qA*@I{9;S`eB|-L$!Ybk%A}rdn~%PN^RF(TCtl2%|nSuFa3N0d05W22wI!67y^wE4 zXOzq=oU8TO3_q6+0vzY>FSzvruKj8&xZ!MQ$&Z1DZA_X}kwNp1l_9_6#Hmb`@fI?; z-t!c*qv#u&t_Xsp5gSxXXA+rptLg!!s3@qMZ_A9N3ug^yycRtVtr!-hq=&YuQ50^@ zXZ16BoP{sRLQhFK010t_A-335F(z7Ay@Q{I>ljZ8>z5zN@wxceK4!UP0PgNVWEnk1 ze*ofPRBLzJ@bXy~YTu$5>?tE&*bqw?U^EFFt*kYy^P2`fGjqf_Ot#w}e&fE4OqDyQ z+9Dy7$as*bc`0@?J3cJ6hc9&!RkEcPAB;{T=VoBna$C6&*c*RViK!z5S^plO*>~qg zva8rPGD&_2ZGVPt$-Li^fEr_=v$$rXcb4Q8PMa4LV-kl3>_b&Rg&+`!sa+uh)UeRp z(7>QyS#G0R>_S$NubY>~VZ+2PP?1x=P}|7 zM)~ME>-2V>jR`-Zn40WdYOR(zMj*XGeAe5!Bw|Go??+B4zBssGo+J&ds zL!wN5p?zi(aG}wzBZK4T2oDP0yLc8lLU)H)6k}&bo+2;Pp68Grua^qV!&hgcgXQo; zQgM`955f=bHC}H(Ed%_BRX|EKe2%xJ6+`LymoBbSOCog*CzD~UcR>7*m!EfqpS#l~ zdK?8jPy_{8y4Nu!G2TzLIb-b1q7e~KebD!Q<#A!%m|ORiMmB1Sq)DM;EUTk`WM;+q zBdrea@-SW%dlLV)Igs}j@>jLqIs6qaP%3REsriL|nx-YH(0%JI7?}Mtc-O9JB<=O7 zW2KIyFYAm9m#1m`hhcrU<3-C8a^oKI#Bnb#E{|^Q5{F-^n0fn{_RHF;P0~jG$uya-*n6XK zq%grcGTaKZo>G%AWc0FgY*x=pWt0yX;Cwi) z{Y~ID*5(JX#<#=2W+63b#wl>C`WyJ}8Ap_n-6K@VJow7ZK0z1Q46!?-hJw zXbZXy7;BGN?d~J4pNWB5QZL^W`QDk^vQzZsU?m9{GOYJo&Hw(ewFf-XE1w4Wi1T)u zb-UF9m;#sMHAbXMW<%mK2EK2Vi1lqYi>|a4ksRw#O=EWb6WNI~IpZWKr*sX4*zhtm zOn2%uW9MSEYVY{W5Vm zD8E7Cz|luGWn?~aB^9%OkGV1UEiw%OQ=Vh4VwO8^8wz>Em$fWooh?n)phD**`sq+*goBf>q;l@Aw_R?w1Sm6IPQ_u~A~$lfjMGC=h07}r^82aI z%~nmUBI*FELBfnQS+vOqLuY)afuAtAGz%22|xRp znsC^7=4hWEfW08)spkbysA6KSE^%5dR#qN=y+>B4M)auGbZ?AFyRjKU#k3VHaN}*+ zc`B!3X2L!n*3h}wVv^=<+=9o?9%DtqcTf_wnLM5^hQ2s)iQ;yWl)%uDSPLmzI5-69 zYCV23>KRm>nJMA-d+@&S+4uf0big-&5r;e?>CcR|=g`GIzkJd}j;$(aP{A)ff2|>s z__|gunP+<7`>sk*Y(`}hX#Y>CQTldtlvmO?-0E>384Eye1+beiI}~y%)X-P(6F%eF zFMfhhr%t|su4)4o;%PM=b-j7|w<25no_oaFrhb-Ioo+>KHszIZ^G-swz{&aVlF zOh*usXvH7{kgBk<=LFKnS?>;obn?4PQ%%2CX)|e3_8ZRTVrf}B*Fq?;tBmY%4xw;7 zWpbPMTYMVcaTS*Z=-TZ-`_}uLn_a55d4r_W{G(yVUNQVB7MCisT%)L+?%C-bNU1u^C zd~7E(l4^FPS_DrsqTPCg*u2$+;=B9jdg3bgTdni4HNR-cQ>;Ju(VWacMm#Jx==EUCIyu-9RlsXvren2l_Z z*jAVR2Bw2-LmfnJITRbF491#J2hqHka5RMZc0NlCw#IsrJ|>dsF$pZ!g;Mp3;h3HQ zw!T8@V0vKaNAhB1gMd5ay!j|Ln9m@G-uZ*70hyVy2(~|zL;cVw!HEBVpZovux>|o| zbYQf9#rqeg`X6Bpp#kqr8S%p3ET(@w3Hsn?Sf&Bt{q!(3i&X|)S_3lo64u587|Egj zkFF!G06WrZJcYD(eXcOKYz+UT^%Y9T=P=*?^$xnU8gIj2ZSTMLOpg8xi+mUX|A)%| z(ZT;#Jh>Wl>6*bB5}3B2DFO{Vcuk?{2r%ESfkvJP)u{ddz2N_ZaIAiWCoWm1#5yj_ zw?kFQAxEp-B?}u^tdD;~JOB9xIXXJ=!*2D}+1(x)WgYo&7k-)w>`e*=#6$7!f{*AH zqRE-BcVR|RDbD~MUMCU@QFFtt+5vGAq zKN0uxJA%_;Wnwslnx~Wqj$-=1(He9|(tc>Xu(Sb>WcZX4fh5{6>67AP;jiN6KX$wQloIc0)?sP`D^wS%jy+fFFj!j>nRtp}H?sZzZyazy{2cea zU86UYrRI1vt6!ixB%ALDQK2LqX6Yb~H|Um>9ZlygfHJr!6HQ>7!MrrN1!AzG(0REv zs8ym@F_}^p&*w{!-zxLs6MyW$i}Z*0^$9llix6=k7@8Alz?HgT?qyIbkL^QwF_?%S z9rqEIQIP8ezoYIi%}HPB&|YTfLS5LK#(70|kpl0pgcIc++T(Zj7QMrm z>*WBVe)FZ@}EQUm2v`Y=b;)Vbz5{e**HpC z|AlipbhK(@JmclG7#D83`W0V4PC;FV>51d2Od~7RSXPN;ldyFfmH}6wnpWw!_4|;b z?B;B8tY8+SQAQe|n>4y#AQFIMvJWE5hjz*XlBf7Gr{N0&Gd2qaRXIm256a0 z$JCn0w5U@1_$!S)an(*1W9as!!S>xYwNU2OB@z`=xqgS}0r1hUp}%@oKPHl>{l}~B zkijzxyMc|!GK|j_o zu&95K1%Gh2N{H@sb0o&D<60bg*@9Z%yJn(JNM~JrGnLcI#{Ruz6fv7qFOnv3!)e@T zqD8$zhY?EG`_b5^m(i zRcNDYD$Y50HmNbzUH0aBeW#BR#R_IBm=h{>`I^d3)_Fy3TekTwy$+XVr7Crr7Z0Hf z6-Ju{J=+c|Sus0L%N3_uzejkFsuEfEO=l~ub3&&kgy7+ zX6Er#*`LQm!wSIuR51kz!Jf)cih)jLTSHn#b^a!&kMYvt_GmM7u1XMbd+8^|Wxcpw zsVFU71NN*hJKWzj8l=H^bD(z?w2Qw*dJhyT&v@+r09J#89i`!9c~OI zZW;FjSelK{P;$4{si4fu(YK2=VpbN*?e$YoE+M^sd%1ax67U^aB%FZ>drBBBxJbb` zR_~o|%V$ps4Vdzad`8e@6G?`$eF%8}m^?v49#=n#tgSkg2qz0W=pG+#tDZ1457Q(6 zY)Y@oJ@mn$h!^{%?f~q>i5I{~ME*GFN(I2ZPMq$tiJHy(A zFSj(5i(at2e>nsNs81J04m%zxxG(ovE*Ec$7k$R4e0Q&4-eVH^h_u0SCq830w`Utm z1LdojC9#_+5D1>U<~Y z3}u|aG73iK>cdX>0pKYTZ}SIUuR_fhFU<_+7qG7c(YRX=^}#W z`E+dc8olWZ>n9diX?X_Qe5olXU3*vVGXLL*l=p_$%iE5#jx{wm6IU%B`xX4_oie+8 zC?e;$QKoU!IdJKP~`FaxU{Njl~NPf3ACJbp@_Dw?UxO?Yyac@YJZ;@A5vEVBG)v+Mo z{72xnY8>t^K-~W*eqdvnVdyt(ezn$M#@OJEBQ*Lk$iy%~j zdG{;88QJ>MaI(i83a`}qcB?IX+5ep*oWD~-Ctaz-I|+W!V2Xj>mb0?AKTplFwpY@| zm=Rdx2n_1m(d~7BfdXhsi4YZAFv=JygCH74p9#xkX5#DMKiKdV2Z`^j#F%{+p60rg zn+fl9faSr|N&c>aGs*G2g&ZFXYD$ger3uKVSW!9o%07X1R22+Pei*mi^1z62P?{wQdi9}A?7)Chqx4bF>D zjw9qqU(z?~9A~b{11Kn-5hI@F2)j7HVnKcq@}0cG-cE-==Gn!5Br^1hWr-Hn!Cd|9 z#rBM8+OS&sQh});4FBsK$NsI6&|B^-@FCKGMum@Ay;3KZVe?0p(e;Yk-l7p^71SE~ zP7EBaVR^#5QfHVZFKf9`0`xB9)ZCYeI`Sva+5bfi%`2_(jGxXl&LK^(B3*5Xe{VU5 zYeC$uB_o5T@&wYeOOmUeR6Hgfa!qSqg(q)sxO+MFKmQlNUr{H(ns((^Wx9qMTe>gw ztC*w$#)9a0{aG(6iK$?0QiKcF-cVhEk%8b@EUk$9)}?T{88n;5{!B1}W#Rqb5ES+X z!%S!pMWKs7a;Pv{@lm)P7~4bMzzR~eIsQ06M?=j4kr)p6E~rEIc^2jq|38x@`9(5J ziJ{S&_9be}%bnEm}VH z0z6s8nr`!n)wSul8-<%XlwPe|X-xpF(9&^P^f-)D%-NEdw3&q;>Xz`@a#gekQrOKn z(ykM(p>56j?cwaerha)mlvyL4Dd3TBIjvIQS^v&)l$VnR+B^6{CU0vkPVuh0rp}?4 zeg^D1NMZqFLMo%2=jpCmCcFntzmM1HRm=u+#6l*@_||VcUB;5)6f3tX(*!rE=IgB* zZw1@FvX2#0m}P2qGZm8BMQ82#UNRR;Kqt@np9ip&+dNgFw-%$_To#9Z9Hc za&d~nn8Q&!UJrNIPelCOce_k=E(@++<*9DhCQMJ==+d-7(sIgOCG^rGnR54{(~W?H zOpSn_mVVPNiLHL$gcFWCwD2a2xm?O(@U$zIgf=Jpf}c*yC-jjAqw9YZSxt+dMn6LM0jAv2-=^e^4+nO%issGe1H%m?0f#UkL;`f;qwYP(BrOSm@kA zyfY(|rPy-0$0VHA8A^+xQ{Q@L2qfMoGwM}{@oikDLw$LN?TCYc_0oIimBm!GUjO;h zaF*|lCZFE8idu!%{)lVRdD2;pk$w_7(X`uX`d8okgV*;E_CyZvxo}^2z@y+r-P0YL zPwMBL_ioTD$d5O>cFi}bX%k?*An|=rL;Pl@4_hU9iksy461!RYCtXtCEn5rlj`HEg zfn(h{50QBc)q9aQk$EWjoWEVbF5m(W_D8rp6}P4_=+!#uB8JL_%bFR0MK?E>(wj=} zFAtVD-hR+F!}tl3SoV9w(6PxXSmM^9x?BXath1b><1zkN^0CDPF{eg+e?8QPOMVBH z2<){xG=?uuLcZrE>o*U*Xtua?SXd3;<(?@lxM^P5IJ=?YX`8+V0=i`h+;kL~Ol>vp z2`lhiFDd#+zI5MvC`S0|`G2wZmSItL-P^bzB8Y%WDIq8VBGTQU(kV!n(lvCago32f z-JQ}B!=NI~&0xyE!t%=1&N7D-BY*BvqZ(r$H(er6&3(CeM>xa={o z>S}_>CV-B&kCrL0oKso@FXRYbEJTaXp8M`i*Qg}DWa)?-npPR75ys{QBuF0mw#&TcnT zuav)`Cn#7kML?}5ktJ0*vbZ$y>G3Ebku_ACWHX#kqKFlJGV7FR`F2na{tZbw@??3w z64KLDC*GqxazjNAn-4TDZMW{6P8cj`c2!(FNfg*YtE}igPpj~|R;}^A( zkJ$_{<=^E@$Ii8444s2$Udq|^u+w~_v$sSTG` z*IUd>Zf zYY-&G*S=a~jeT9qrB!;CdMCz(GhoSq`ITsPkC)zSfYIdgDkxpp9sYLgl8B^U*k3C? z)9jj%_@0RTj6o=c2S{arMzU(%f#477c>|F!shVOCjT^wFm>Lgh3b>y;`y3z_#S&8S zf2b!q1y%>-AcNewwr7ER4etg8^V7miL)OLf)?>!HrOH{VA4;YDZKTC!#;+%HcSL7b z2D~EzU{lS|W}h-qey@IjW%cBe7E7!i1cl#e3be`x+Z^84vB^*e=@eltU9v_$*|$>f zOb)G}OYFe5Ix8(KT4Fz1>4AW)ch-cGXEZA&L9g{x!P9F#_l=&Muq#fhdTj-0nO0{1 zcznV8IR4EpK+G}Q&g$IshE1|fTRg1L`qD(dq*dZlZ3XgsmiO>B?41|?kKHH&*K^^% zx$0j0Lt6&$SiyCd`!bBriSW-_HUg}}M^urd;DNfOjQaay?meNmP0Y5^j=6l{ zdO4DPvdp(vlDaH}hpVdp2%-am@h&?F?C0tBOvEzz#5=8ZYi-}D_(y3r0ffC{H7Bw? zGIkxoMK+}d-Z_Bl=>x!!^Os~yR=p-$vP`SncdLC~n^(>@OoDdPwL8vuSS_)`-<|rs zj7@N(5^cStCHn65H%Km-LsJmKHRiqz(<4C{#7}bXEmJTV0W?HVBR90f0&<-k5Z63r zv-f$X{Q~=OoCO|?io3{~+eoL2Zu0sJM>XW89p)=zqf)mBh zU-`G!R;D-HBxlT9zlRWCkMFmX_BxEfr#n*Y2)0;K(_$#v2)v+MzE;Ym+%gIFJySrqLRtG{bVcy2U`JV*Fm~kaRKXJB%)3Na&)|7 zYf?f`t9k!a6RgAlZPyAgoH}$6ekt>@Do8yr1*)^Q!wu$1F1o}z4R$o^395u*H>}=MKeXoO|T=g{H0fhUTvHf36Q-J2BERZB>B{Re< zW^FvpaCqlj@%1oyGjevy?AsCl^ZK$co5Rc1mS4TqYL@TSZ=Mm~{LD&ob;JM7>PCIO zG!SB%?R+}~DO9v^3C;czj+IVLh|}0Cp)mAe{VY4|19K#GwJmKvnR`;%D$;)c5a`hV zY{F)KAZN+H3$&{BOi!wVHuw?P0Cd0FHw)@IsY9t2Mu8R=qRo3|+-J;tSCf)dM3SxV zIOWzVf;iC&(ZwbfM80o6_G>(+A>dXU)--?}HjTM4k5}DA`E87qTBs@%e0xXG9YnmJ z2rzyo$Rpt;+smnFnGzu@7(?$~V)8(w+t+dO0ipNl_20%dd*EsVskLU#hg<1pUjub= z4BC|>=3FSi(0W5u!pM@AlFimJEqnFkNg9OPDsGMgNBOvef;aAX&-}|we@N^8d$gTv z&kJ<1$+@$|684Jb$Ob#=B7j z-A7ViaE&0bOV1ucQZ)_$()OK%qjoMrcs{9o#UzXooHKm%3}^XAv*`PEQYn0mgHidU z#Vu}g!*t>2zd#SWxoq2$vVIx)rwc8=4quK0WpNcr=)RAt^jJaghi_>$2b#Ydt+CK@ zKmD5DD<*7R_^RC{8sRoZ=H{Q7vP1qn@~e9-k#w{FMv|RDv&-jZ9{`#5Gnji$6XKk? z9!zEJ5tFR_VkD{R6Vix=Sh-neL#)ob&dn@3?h=SUoO3T^%`w2^mHjxT#wF(e!N-oA z{k$E=&wQe{G2m84ZX5D@(s=>MhjCTN&=(c2 z*6hZrH>GE<6sBDo#a|>BTgPgaUT@?tW ztxBcNuEA4zv~%S;hvnI=2r>@mQcd*~3$*y|joXwKIa`m_hLn`AM46ZP+O30gNcw!x zh(j4?*n>k}kHITbH#9AL(gm!nySP%;Kx^yM4N!qFuQ{0;ZbCH*?wf-`C6gwu{wMlY zbOxP*Y5U4Y3&YjjNijRe3X9Hr2f?HFI(fInu1-|N8LK334CDdN5Mkm89;#a z3GQ`*KVtndb}cN;PC;9b;~Oodi;tV^y7*lyg{cU|Cemzpo?bjKaeIf>MLNAEo`tK6 zGRihb_#_4GOO$KWu{)Dd1VNF%lf}Pk!hvkr=UnqR@?t3vSr=)jsvHwK3Fyl;7-J|O z8wa#c9GXWxneO9HpZCU>+K%g(V}IJ$HS%m2tElxqYg9Nu9P8{SzOfhD=o@A0GN)$J z0n#&MiEY*ChCO6*PozrXJlCrg3TP3^c+X`kziO5h;(m}?1$jnRk zti_~S6pLj^00(}a$gzn|c{97 z$CD>N@bcP^3yiKO?VilI&k=wM#A?&bBYL!bsrtK2I^czkJsV|K@mo4COB)3=>Ll538Sr<@qDe_SM%tO8Dr5{9gcb;8V}-aYFhV z-|aHSX4cvCIUgBewrJWDlJh?%j87vE1nC+r^9u&me;eM3>wDUjv*5%FB!HHCt{ZAs zgO_^ZiF7G!I=Y#(E;UvXGqet?Sif(5*~0?JBAp2Bp#BuY$Nh?f>YXD(a4P_3&sl&< z$-{rGe3M4eli>sWmoLD{@gsdP%Wh`xTY}R54Ft4|qH6vg=n^l#!?Y2Fj zu!B0+prS#l=#hEFE+;3ef)Ylo2X!jv>aMMmjy&A_L6PC6XCk=ap0rUc*NKpCuM$Er zqAhYVA*Nn-Zni35rhUo;meeHo6z5DJR(u-kcJt@#%sntH4MO~IEO8RXGR?`Z+w!XR zhkafi>llib!&q*pbz!&^9j_HK1tG4}pAkAoTCV7J?kYQ0)Qy+I3Z)~ho078QW(65f zCZU60U*7$V9R=LI#Hrts+mRqzOTQ8$s%xEBJ{kF?@4~ryt*Vw?*xkm*N?a2Vd@XJW zVlMOP;H)dzP@m1yXId>Djutgp0|w@dONcjtvAWt8SH`*(TGlQr@Bl^2_O)vBs#WO? znwFV+Z3h0H>u)+}N2^k!G#z9*O>AchN)I`L5?q}kT;~ew+3E1v*G<^z9^WHh#*t1Xorpw~8zEe8wBiAm-xI0NCBKD`W%B{+_ zahhgzZ{if?GO;~cAUT#2o!Us!g__cO?k%Fq*sjP-fgc ztx-9?wh$$LE^EInevT(@KAcpwaM_-*1L($xxb|o_o>r3s*`QQ&_+@vuFxB2@F}G50 zogf!7_nlk)(pV`Zwt34>JH!86;~uARQ92DzQ6P?m<*gFSPotbI-!u$LUUfT^+MWYS zEFaEa$T!S*<=*r<@&6RoURyEV82QkyaMsyek93tu%%nP5(%?O(b=#T2^sRowB9ENe zPdM^tcs<+ZP;U6yP(=YPRF6)`?w1Gm)Ivh}R34B5aIFL}U$UOtRlWC9^`~=+0Yrf7 zgwXi0;BJ3n5Vj8ItJt4PwF=X}e$a+OIcS*4p1e}^Gt3m?RLXPupj$|?Do|`HcY${A zPpRAeLBWDpZyT?K@_AiYzg1;w8S?%yC0ga8%fZi$G*}UcIp&^SVNcGWTFDsPsB`YD z_@ei9UAS7i>YG#ectXXX#N&G&7#0)$Mo8}H%k4oXPP`w?$WJsvC3sPgVI8>e7<^;ptd zL7x}8c#|^8#oWc!nsy=CCr<&k5x8Xl&h$pL_LIXBYLz*rCtF>T)!be-R|*>}rMvpP zCqJspfaId41(4jLXjx?Kpis86LTW-|igEG4WPu_aOzA~Dv?4U1K0kMD2(LXzX(c~)ZrQ-{lLr>d4#jA0L ze%7#8$bo7VF=wqrmDJF0{h?0Tm(}`8BU(MLA7L6>+D0$!&NE3zl5<@3Ws%YKJ)=AK zc}01*1%;una){YR{dngOj^1A1tk`cN@;u0Yf#lmdN#kU4b)vK%emkdU?gnDA>K&Fn ztVr0yB}I9kc~OL(tCd-u>>qAg)cPx?@hEjfrCKscesD|edz>h4B-Q|T6+c~MlhK_3 zfIQ1yVRE&Ox>x$v^m6MGrk#$lr3!8 zYjhkjZ+KUx9#X@Wph811es6(i+cPY`)dZBPdMNu8`9&3#=F}tS)}kkBk)9`4=~_fO zAS~4>He2Mz`7@}V|WuV?2^7N1L)~!q6U-eN3l5+oEBKZp|SVcUw z`}A#i&~^WKs}YOI6e^=pRr0<}vIC1>!TcaWL`<21H@NsYdi=@=D|_++j2sZd)UX7PllexSs95TWk>xx2gp| z-(oHHG1aTV0a)|7Zgdaacz}+tAvqTw*VVy}%60%i=mkJc&}?q>G}XcE7+SZ}Wpp3W zVDsW?xAGI++EC%GjI_KJGTr+*IH@N?GlNf>X%rMiTSE$~oQ&|)tTJ<`>f4i$#<*G|; zA+?ntO$8g%MI^uUBtWiqk^_Ws1SUEC#STwdaspgwt;qQi@jnii&KnSHHZ~jBQ|Z#N z;XoFoZpEFUi9^o=C(-XS{7}lAkRqY0gXUEuY0fQ~GRxU}61~aBPY|{F<9G z|Jalg{~X}1g-XBqT>@!+XAZ3fb9D+bkbpJflq#ZBJ{uE%9?Mw6Va-V8T{mKv-I%2s zxdrJQgDD>HX%ENb=QptiVwC;>>*~83KO7tBA1&(3yq)^E}?(&!Vusojy?r zXmNZBvp5k#JSizp<>}AZ`-E1Uis0}UU|ry88_)8ifv9yKoSJnw=Pu}26BwoEpLI|n z@$TX6KIb+v^f$?y+U0lPqNbrj^>N!wC~bXN@EameTVqIfJWYO3%<9G4R%EcbASc9K zHS+As9vPxt*u+i*#n@Z_!K2e^8wnLYfvb;Ksnn4<7R3#@hmAYxLA*ET#^CS16yCGv zPq+*Rr)p@zv8H}^5}S%%c1swJq&K_u<|n#V$A(#IM2k5^Q?)ynxM4E@YCr1AqIEH` zGzTNeIL1JbmIG-Kk_)OPtZVa z#+=icw+2WDeSx@4MZ6XF<1lyf_IaTcfvT6eUfbM6@(P0&tPJhZj% zolxy8Hwmdzu;JwWs%X0aVx9n+>vFIhlrCw#mMo3Z+|Js4G1Y!+$?YXk zgOd5m#vW{?!vX!X-A9F=BH2w?(Di;0SjVqA=9E?uQ!Vw__&M zw{)=ENB{x7q%w>gM*-;Y-Z3pBW+Ayye$d1hIiOZxUDK2@&NY+=bQacKt=R}BqSW^R z5biDWzxpws&DsR%&iNjK(|nbmkKD*9wbUwBf0F*_x*|)5RCq!+pd(`*`!cp(mtlp zSDmW74-VmxEn$ONHiMHqh3%pvx-VafgeFQNxzZnm_D0-;sxU=0<)( z6}w?ek|-M?T*Rd5wGE(?D7*{6jO*h&Mr@XERf7^vBCKFOgc7dJ>oZJ{u zHn}|q^o00~#i=Kf5B(rkTal*VwjFAGVY zs2tIS>%^>Hu1s_yK1m_`umSf+2r-qlg(OkQU~JWomP4>V?P^Unx0}B`W7M!C7fQ55 zZZ1=ZpgF4MF0P7<+2|It?ED@rITr92OeiIr?(Qg5O%(AX)9{$8)gGP2OYedJ# zE0G7ZRd(Qf_d&)_v&;;Ny$k)R4M{o+;-4tqx3byvd)f?Vi~aH~%yWeZHd5t?{%Z)0pAOF(7qDvpaYx)K^aL zo)II)9q!o0mCdE3K-p|?e1q4gVClK)r%Qx=sas2EW`Vn=_e7DUAPWJ$mn#iH^crQA z85wvf13|<}qE_|3RqH!~L@)lxFQ*f4CEWmV#e@&q(B^8pGjcz)B9hW&%PG>`ITJs$ zX$k@eBtl#RjQmd4EMT8?_UZ#lxld;rEy5DZ(zfbha}K56X^az#TC>GWU}yL7L^i!2 z_TFbPw>zJVS)&tJ(NBwr`YjzDy}i>c(lpo?PgIi`Kd+T~4BvBWZrjLRQi6FH#TJy$ z?72hfx4A90ezk6idZEvOBv7>(Kecem%HV1yH0?0?-uJ%3LxuUBWufdBs)qUM0|&`( zEhbFH$UbQlWmK}G$<^=);bH;V*x{4Pf zQ@X^zm80nduSL00v*gNCUfu7&Su>`7-d+b}8TEA^>-VWd*EJ?<-_+@QgBm{a3eJSr z@kZ1?`M~GOS6;xAoR#AvU@tjnKGW=;fYkK2)@yxdou5;lP#Zo@(JNlhnilwE+tdX) zu0zi1F&E`qWCFGt-XZO0@2ID$a0?&T)Cc$nbcB>g(YedTjBDhs2R(+c!_S?f7YkD6 zIl-PX2gJ@-Vj)6N^&VqQ1tugayI!mr}tFd0W7wiJo zdvdz51KAzs-iny;QiL~6$N>tiysU9{vFzeTbSoL&^|3U&+>;rQJ`cP7450P)icXV9 zJ!rC&RTnG=u_2h-v5)>}rm*~C%v!ti0zKYBU#H<-Y87bQf zZ&DUrhe6u5EXI0%?0!WfW!Kag=s^jr<3PS!Xz=5@h#Leoyz>Z9)bKypx+)fy<2!#u za+6^{M)esxHumsKRf(tCx2gi=Z&$rRPW0M%lQkQAB3k{8yuZ)>c%PH?mM)YIoH&So zc8*p|az<7sPBT!%%$zQpv^395&Lku4zI4}c3rBmvmk#a2&YKIre}bIv`mS)&M86OT zhdGBhvh`I)`Jrc`YT%~7*-m&x5@FH=42z@CV=99SMOs&l?z=q{h_mm(%;ql#wgd;LmO@EwVa`BrLF59-pu~RQHG=f)Je}b@D|(W)2qMakhQcE!1~fIh(|ZoK zB#7+3`)4myi8&VqVadv2ONVwoY5{3ft%UM)6iRa+%3)Azumf2Q=e^9%lq{~;TVdtE zvY5%ljXzC@L2Ndhbm|$R?Zcqj`Vc=+?pLBhtJKKWnB;+I$W(WkIQk#Pf_$okT(rr| zVH-UowWUd*NDv2|R?)#}4fx`Q9s^=cyTSexOhe!;QqMkvEff$Xw$WE9=S{G#Qhf*F zZyXo^NYjv>$!T&`;P3xZ9(SA^n5Fo9QBZr}$FniZKA1tj+iYOFPyHLkqa(ud0-8-< zBn;DMJ-=ihq^rKutGB()xo7{3D%P`-z#%GDCCX}7349P*lX#=|cO~%F<7Di8x~Iu2 z@6;aLN_;CO9I+DeLuzSCF>PTnQ%Et^e!Wg64g5;`{XbO>pjW^9*6WxU)5q1Aua@uu z7=43?N)ukg+|vYKJhPV(tkh@|Nid*YLEZ*zpn?r5+_yb)#fY@w4IG% zP35;xSdu3@Q?u>C#KTez!vR|H7n)tee6 zui`Xp#(Pc(Q}dJ&x(wA3q&NPF#D24yvRnu(H|1kHs^y(h_BTwE1+2oDjn{$biq%Z5 zM^-+Wbfs<)7tiKZ={l*$hccb!>_<~-d>FsvDextNRb?^woNEotTvo%Q4nKYNZhot+ z-BqiTX8`)Z63et38TGMdf6wS2PpWrtc+B&wSR&t8JuS^E=<{-=4>Qk@^k~CM&(EmI zc|^=PuC_7w3+T4!9bPKEYfkM*dHj3fem|*BV^h`)zZ#~EF2wqV)k(vO1Rs82u3VE$ z`t<2RBqi@hX~K=vcMK?hS1xP${4gu1DTAI_cXd2NT~ZtYrum7(w=9#tZ1Wtxt0||F zob0-Egw>g3d}1D2B?TI#D+x;ptlb^!n=Wsu+>DT5&E0`>@9fR#MiyvEEFp$Bxe~B=!_rpXVCj3r~PGb#R8`O)+^X@WP|u}o7~syR8)x(v5IFTiktjs!D>c9m1)Q-fCELIcekzROve z@5!b{U^%W9YtxNj7sRyOBPl1ZTkv>4-!YPxxw;xUUpyL}1FOvrrlq)G$$|x%ktJ&7 z)@H@8n3Vu6EB4}I^if?{9H!3y*X%b|_pUYAO)&zd+zZ&CX+NB>+C~d%+9;H#ZNYNz zJ@}EKsX>#=N49j~uwbY0lm{IlG;lpiUc335oColB+TQ)()qyY#otz;y370M0{Dz<}*yjb6OnAUaj<;@T;SJ8P)vi{XI z4c%}QmFYqUYc+#+sqIv2`ec-^zgzL2KJ@R6^4T57<&pKV(^8(X?lmSm&7lZGB?T6U zJ5g(5v@v$D_i>Wk^-0tYIi5` zG5+&8DV@aXz@Bt5CRck)Hta3_E#(Xm4Y7+~Hpi#Ej&hUC!jiw8+n-PVcu8{dfHWQ( zbFX@yqWrC8&V%bB`Iy>?C9)pjakPJVlyw_o|=NrDqU&(Lk{_~C<z#&?W_2f*jy-AV|qk+}Eo^WZ<`?=d5A8vVc-c!TlVAL9e~tHL*( ztegMSFJpZ(4qUX5RBJY0XqU18s8#->yIPr_m)sq1lHpJw`DG7 z$i9Vny0CB#qS}VZN*PFRd`P-@_N)2I&*{cJvrTVWx%@CPv&F94j>X`MuS{GPP<_EH zJ&Lu-8gKon7ZxY03j=23Ws^>}5c+(LQr}#yy*~;~|IEo1z;ut;B(!VH2^*Kj{d?;H zwyZlc$P{v>yMzyMUQxd1dA?Nb<(lAgJ}K(aAmR>T0w zptrdm#4kt(;<~{%fNFB+jG|e-_d)zZM6SAeyrZx421dJyVV&>2^2XY_Bu!}bk;%@w zO_ld;zwMueIg;Pl420x2##!k%XKB#cE^Fg5>tpr|*HI$!O*9@!rhJyH-=TMtRX^Ui(zU)u4}Awe*Cj6 z!heR;zc;^Z$y@fT!B04B2RIfQW+_>OZhz1Mx)2-sjZ-fk@BC%Y`n+oj4-;*7kEcNuXEj9K9`Th z5eVkhibO#gm)O*vUi}s<#V(a6Z5NYsfB5*=%oQJk;qq{ z6dtQ;1N}#p{L>E2aV%QOVjdS7R?}bJv~wy8dcKWlI#HVh8mvT4R@HBB9Q}46e>_QW zUGt%#tK}wv9G-W^q=R|CEsepP-&0K8i=JUg<*|Cyi4uo{AD;qko0Q9EeaZgs5*g5u zVwDa%G5N!FHZoQt1$8T1(L%c%H9j2?;-avcTYnhzKN~Z^6fcj_gy}~eF%6NAWu~FI zz-MJyE=v_88*6YV;IkjZ&yT{>{C~!0=-JPZY^U+sM6Gy{V=_KEAjvYijz{*oh`UcalcMN* z1nBLyt3}Rj!Yk_zKX;dUmz;+0y_1P-k^(Ajq=tpu^bNL?&uUj@Ei(OS3=K)`r@`2n z-dp)1#Mxh!{}08ozhf9FhI#arpC6d^6BXFN9XP1*3Ihu-!gUP=DnTC# z)BWl7VL%kxRP;if$csL z<4&F^#cn0OrCIXoC6DaU@P%Qb`|PmeDGHQLfN6685t6RCrb;ESm%&eRwo#!29zMv!i zkyp%}@Ph9Vyx?89p0PJE5o%Sm$TQ)oVF~+izJkmizi1} zejF}}dntsLu`Bk%(@;%K&9rMi;PT$yeV$%g@6%-c4O*JLxE9m_x-ns;F3Wj^$Q)JP zFvCj1sCd(2ta^V#T=j`o1*ze(pL1_NtgFJHId87&>Jk#N(x}xPlizE46iU#K!^jRf zHE$OZ!+7R?RK%Ji=G2VfGVAGj%|z#c^o{O1K~cTPM8NXo=v4|AqLECwGU883n9dm^ zDH~z|IqVl|y#*;c(G~;edMC8raXv>w78z4stRgn;s>hH-LyB~dJoikK{*OOdQyaQ= z$yk=soS`2Hl|5%YoNX(V285A{IxVKf0D1fQaDjpQCFJ9>F|MOouFnb&cLY{@xYbv$ z@z*YaGpHB)A4B`o_+l7th29Si;{;r;$(~^Nlf%ym%>xh*4V+>sowJ8>isHXPvZ0Zz&E?0vriNfEBPe9{ZP>ANnq7u7}2>O zJR1xgH|q|+-i0Ru4H%PZjcQU><9V&CALua!+jJkFVj)7uTdb~j%T_*cUSa2?8LJn3 z{*j8e$mTr_(}uv0Dq5of<1*3f6s!hs0)y7-N)$reSn-L-J2l!l|J&38)E;KhjPT&* zS3O32E~sLTe0eN*41`Q3ZFRycj26^iGCq!@`F1F@`GZzvJ(b;wSl>`oe94nx zWCodnU489-;M2z#{%qa|>#tuLxkOSpS;QHQaPhP%b(n%uKZWN=hoRgMse1YMVp5g^d;lj5|^F zArnfPe!WHauasU={Ja0W+*K6FXK=*(A6{{koT%df9 zy)aFrjh-nSIW-4^gtcp1RJ^Xcl=m<@3n_=tsqhnVCD zu3ZS0bymXG}NyZ=EKL-l~=o_=MD5lp1>00s_nS1!i9l8%=WfvI5;n2tm|92H>T zZ74G@=6SaR4su8JPrn;%{&^#bD*^NPK!h+Wm@kcSlLNdGkaam}@O$}U z7)O3|sWgi9v>KCJrn|)h3@j;}J9_;;9M0bnT>uWSaT(uAX=2>Bz%0Mzu(XIf!5FZ8 zd<$$`oX=gDwP^=0EdmjtMvHOeJC{A0UEF`I_5UvJ?>+y&miya= z|KBWE!W!E%T-4?shQ()8uH_VB-X|uqt0)8`f_-=eh z36J?w=OO^z*OkQQD@!Xah`OKO!GCuutCEb#t)9}|Vi73fWR6st;XhIKT>Lup@5FZg9Z>3UI zLrT-}1fU7JdmXqKMJ1WNT$krl94TZh7-PN=_aZ?1t0Y3O=<8RT%0}Iuf`T3?iZpiL z6Kb_PrMI+_eWrp`Z3;#4sLaPd$&-yJesoyhhj=i)(JxF_{Ou8bKY8K<@3zi;ofkL% z>sO!|X7N|}DGl|jt8)i*wM?Hi`>m@7#LP_Bsx@Nj?1E0uJ8g@#m<+f!3WN=WSte{- zFrS@?Z?zmQbD89FZ6Fi0AWCq5PiC~bmk~t%{SfO}dFa8Tr+}2vD}5bF&9WXY?#|Wj z;ib#0{z*Jb7?GT^Ym*Nb&SZv{A*S`nxn}F&(Dm_w)9b_g)teVge&(09=-u3ZxdyJ`o-DXI3 zm+u$$$n(#NjU_R4Zs5M;<}PYT;E14-a1MC;bkE$wQ5D#PjR59=h|dAzqL_reQ*9?| zb@=Sy4=;NI>Jd|B=mQ!)D$IC;$Bl@l14eG{ov?TvHJE2ydX#dxZljt08p}=H`ZNbMX`g)Zh;v3C3b7Z4}lUkABnSeRz9Az8^$;y5Gd+$ z$XdiUVUAF;E+dq$U6?PV_;?witl6qjU>x}lkgg@s-JQg=bhq$gfPv}KmDVv!Wd>e` zpE&dwU9p@@nJ^Fo&P2K$IBx+VnSX_wPvU(izanUWT78e8mKq!_UimA#O!% z{x*s~o^UQ}=O3Yru9(A5$;@`>Tu#2|0WKJO7?v3m^KfmgE>AK@ac$y_ ze=%?aZ2QTt^KGDN+LdVJOOHm*NuS(zv~;8hS@>t!JZ~tb3mJ%=FA^m4`v+`Jv_99Z zw|xWWJd0XmG3(hpL-#>WcS}z>`~qDz;5uHY^-mU)1kJm-(c+1KhFEXaSyY^6`2p+Z zmi*`exA`e^;Y}fRN?uz!+LnuwI99#-7xB^G{@yihvlQDvc@1FJ#z=)Hwv(2sW&_o( z;VX^D1uuaM&_|Mv50=A#;&)FS>`VbF=6x6I+p}-%`nB3X6^W6MYAg{b^AI%h^ZnaO z8rlM{g)gQ#v&Y-Fo_z%ioVF^ZjynOgqaIEf+S`Wp{IfoyTi%VpmCUB2V~OORZ@1&M zX5;v0saa3}iypwMNN;op-^N9{ZM(~ZRNDqWVS zEb`vj=hKbxjl~YCiW?G#&VXmOZ@v3rce5s}a@H&eTZwlo)qtFWPZjdQb zZTPc89>A|6j!1BOS=hx{030RDlr&8(_!=On!wg36bAv@1%qsEN8|?SfpqUU^NRX>y#m}%3tTeotpAa7 zZ6}`qS`O0qj%-P&N8i-V`R;uhEz_Yl4%(;y>!14`202Y3a?RRxDC7w&^4{a#k~_a$ zq)}QRvb#NV&dkB^cT^YfkYWN&9L>D@WQkfLQ~ao7gFUG^XL3?+DQ%Cff3FLGIxnTJ zdj`Z+V#>?H!_DdV*=#j-}rrhC4mtnyL`P}RQy8v@f-g1`xDJNsz^sSRx&o=YDL3k+i! zmH4J!REDE;dPRLt6qA9zi#-{wGM^?99a!Rh^?@0npcS6s{&%tW7~gVU1wz8jW!-%p z2~GiGt*MQg>-J4M)`A;l+0{TxhUaOw#olx|h+=W9Vh0`|ru-jL6QB z|KJ0WNt86G^*aI6stmg&+3k&}kSGx=gPOe<+=b;Dh6YX+?B?KnIYLYEWw~N9-U}UF z7XHl*HE3tr@AVH*6Wy{0UfNBzp3}_=quTD&_UXL#)o|N`Bq?98b@JbL%(Ro<^}YC> zJ#;824&4Alqm9%?%xRt#FAYBjO63X}Qf`#E*_3rDNi9%^y8|t`NMC_hm3}O-GKa|c(>sx43o#mt;&sHk(J37p7LBr40_5IbA zClEVfk1oOGFPVM^8nXUE?xK(M6H3OY-s2JpaWC?;&=WbMX%16Bo1RXPb+KPc7iY-Q z>}9lD*hHeCdL{T=xzAt~HG-T7xIG2*sBq=2a5ts7F)82TaoOtXeHLFj9zC)3nI1RT z@60uF?5@e)mqMOuu_o z(2ehX$u)|zDQim4_3BMmdVYUfckI3GWRuP6Ocm`ioDFgf+CMj_p*2T3+|;x& zvM^8o97;Htl*(jXCM;hIZHg3Sn#4a>%9^s4Uq+?W$ejPnok*NwQ(krH#QAadAQbCa zzTj!-f>Q3UC}CtTLw;}V*KEn^tKQVNoxCXD!?)(>r}2j7HC?JB1DG;>po^sj?nWe} zBKzA5&z45klgwo9yQ+0H)=*Uy6GA<&;X#g^^ksv>brWz2r-G%^TwfL`G>H9l(sw&{ zy^YpwF}0En5`$|yZwX14)zB&nc_vMs2VAGE34d09b0yv1Q(@!BDZo zn8B~$C?fAwznoZ*#~ccI{aMr8nJ~3&pcPZ5lXMquLzLJu>BwO1ly#8Gz^CRH(cJLO`8kA}!oO|I|_`YP!(zJevFPeMz+x;hVT2sm&h;EVZqYa2x?bz2M zAeN>=5u=a#;?SN-5@)f-cF;*VW^{?H0(q8;UL+1zERh%O4R|FF+kInz-en>L&M z(}yo>g^X@Abt5u$)4pCy82qLbac@d_jPCV5>dj#x$O^Q#*6#`E4i_Zo^yVPgES%yT z?(KRnL@?qD4!q=<8t<_*?{Vy}Yj}S`JwEqzgbHRTITv`BA`hzTH`OY?Z;w60Hwi1< ziv1U?F7XgMpG%_1{XQtSH!`14%%I|H)$txhlh(UgpAAuxwnUxA zaU+yUwJPziAJk&4DDw$W(9n&(jl3@t6z8Sy{(bNsoT?hkfZD1Ow%S(g%>v)ejorH` zO+q)^7ROCW-vUpNY-pa(DgD-qN`kAuOx%dZWuC!ex6qa=vOt>j34!$x%J3jMu$%M9 zcKFs0et@&TZ7_E@docml`Eot`KJshG{b=o#n~{VU0ozMjb1%w9i1}xH3YmL>wzfsapEcANy)^pnO| zz7y9*+$i2mB`=E^+yY)%Dqp77ELh^!#VJ80Pp3%zMV3C85=D&1I)F7p9R0-{)adco zF%E8U?^#=!NN@`!nx0*NV&sEO^{Ke{k-&bJl<^!NU3XW0NPt0wx*;| zJ{7MubJsoI_!=I;$yo*X2hU*sk^|##_~Q^|k@s z+wj=I^jevu+5XY&dc(DjRHr~yxP%ka)ZIbpC}9E1JxhAIkS^;W&`cA`&GE~jq03v< z5F5*@k2P~OsAS=7BZh8U9ZB5bS(96#txUnuZ{}K#ezBOR&w^cxoH_pxKn-5!M zoXY>A$VTdl6JmQYfNW(B*#^DV_`EvdXPsd)h8MSnvwwQI&)~YJTd1XgY9+<}B~7<= z8b*BJy{`_vFRq$wR`l6OR#rkTGAs@X{UxOM^AMcgG%+NLXY=1x=%c!?|H~U}_OsCZ zP)&L@=+>XqnZY+_a&l$~QJGm9q)VZBbBl%!xX|Y!8HAcJl#|i3v=;!WG)|ctdLp={ zlpPQ0p7tF!yAGxH8p>^CAwI04pw2g*pDe`Jg7QK5?y4{M$ao|yr$pe-hf4c%`FTOY zKRhd(zQ}G}$%ZZXkLr8VtqX?Q`M-C+gOawFwV7#wck0#yDnATb!Yy#a%e*>1TYVW7261%VPBfzPnsrlrvnQw6X1Un3@|U z(&yu{CEY|3GDl47)&#Ji#eNPPL_$y8kKTkN$%!+n9M7;t9SaR5Z$BGxg&F=#UimnC zUEy%cpg;JL^3|tlE5QrV@jdY@-R^T2k9!kAjbTbV3M%^b&e#p-8Wx2M8hm zVVCdz9rv+c!TksxzyxOInpxMmKI=Rc_vLECeL89hB`}~Dnrk=IW;3uuc2Z}Uhy%D_ zN!UR9;4a{3ZJ?z{h&7#DAr%NZ-d%juy=9tNx4XYFE{_&5>(5k?t?=)PwM$#4`tkjF z%Ox98o&uh}3KzGye3P2K7lkic-3;ne{J%6;=iw2#$H18milx%|&< zX2$SkDa3}R+4^vyS%J(XHv7?}xXgHR9+jXJ-)X>_V8R`1RWIPqxd0_3$?gexbw`0z>9=G_OdT?-zvh3f z-oEdQI7l;f+wrm41GZ`rd%?~FZQl$tv_(hU-h}u7VEd3;Y4uMb=w}pTyakhQ{l>>) zr;e&9v(NGI+{WlIDhBoWwl5?VHoZw>j@MyKr^5^TD-kb2-)I1a&GeG2vTVL&@KqVl zw=2tZ+ABf`KXxn9#EI`wS`xu%SM4SdUj!Nq*M#5t3C27|6>`hx|9QDVLwy(dAAp~w zNkT)U^-3Hjidp#oq>t{gl2$n^Pw5Z{xK;8p7mEpTq-I}T;euy){uMnj4rTUYnK2-b!6Zgr9@lV~+nuy& zq&PGZs*^y&>{UcbYagx5%yI7f_=~aec7BRmgEx7D2i%Ga3TrxLIrs(wVnyP(rU`Py zDa$Mb?b+Ti6?g6GRSSATp>fs66lwqOQ?B@^-_)Piqz8v9?dXiG)w7UKlP!?RVHYlAZKZ^GyKjYKf;!M;W~cb?Tl+*eKA`& zmI*?gtPobW%^-KBQGImateZnR{usMQb~2U)IALZK*$o$q2>RZhc$s0#wtdRut=6-h zFV1audNNv?VU25l?9I!ep0p9Alz~8azT<@5pD5dq^(@0&wXnF$TJg{lo4u}WpUU2A-u_iF+e&m_a+NX==SsV zdXDEDG z?o^((5r<9P-8$%-p3!ZO_=-?84YQ{ei|1nxM_5I1l(mPAsDw((W@EiqZ=FFNxA4NV z@J8FDv-Wf^$vTi_s&4WCNae5sKT_syZhkoNemSXcYDoPTLVNm84Kmh=gy8fYMc#;; zR@-a!hHj|20pZ*NzDLLyz`0;zvtOmbyeeIyituTTJ6Nx=b=*T_?iu^No`qsjYKE z;*k4;6WYGW`mAeGo23aQR?Kbn*hgQfSG|wE@K`s${ZJ<{bSaBkezcN%J^d8~-6u`+ zDCi`SK^}!O&95mC<@>QO>!nls#U8a<3_}33u!7y5{PJov8`3k?7EGziEKEIm}L5Fb1})^K1BO&)2<;(k;y z6Y`5Dr|#PvQVBIU(y!+TsU|iOoEmItx<-nLb}{4gsp78-e3CaKfAzB` zUwNGpDZ?{mGpUX)_MH2Rt_ktJND547z59HLP==iIlE$Z~Yc%CyuZMRnCYXKho{vw4 zJ%~yVY^N4$QNu8b6^Q!(3bP@N0d`wM2;NgrxW|;sJnQvq_gLda_andj0D}9Zms3Q0 z9^Ib1_PSx0-WWgN?MP>Xg44rLhVmo0(o)buF5$2l(AHinb0Q9{HEIBk~E+$jNKqK63`IbBmbE-OX$*oYd_SJ?`yWCzuPVyi0Mg862@De;Rfih&D>CiptCzKs`I{PV`=oSdI(Sdewut;UJ;&HQJAMRiwhT_{CX2PcS zI6f$XnzQEVq#P#-`EcK;z5P)3MI}Zzc6AHR{#ACXLAxw6>7q7jQxtWEW~qn5LhrB* zy`Y`98kQcAn4GQ$MeLq_cRt@ig77v>O-`+Nx zI(Ht)O3(G`_nYgIW2h4MH*!l=UYGzCtywOQvFl`it6=-`x6!WW3TuTEps+sL&{p(6 z88@8!A?frZXl27)h>~?|{=LrlE~8Cygb;!oEwM%3XXYBC4V2IC<92yNorb)KY&MF0 zW+$d_7dC3h=z+|_H`Dd0#hDuYgnpxCQx>McCGy*TX@d@uG}1I7pT?ZFKfCvwr7(tZ z#!u1!qx~O@6lafyHnX;iMRm;z3?7SCC{zSGxNoJfrOzFdPnVbi!{m#)Jeacu-9F)E ze`Ga9lJ!~uY9-taq7d}6`LjH>J~cjIjFD628grHSMPJrf^*{GIgJG$;! zi1kd4lo99=K{#b;dQconeE^x)m*T(a7l{kE4?8RaLKwBVx^L9qInwXBW-rb?Ji%B@ zzBmaOEO!?g5*)y~!%dNR0~lz&ju{H)_~I26m|$2u(7~dD6sijFJi6YVd>B`KoD%%S zXnE>#pQkB!5cav6yvpBmtx9aOZcnH|WcVOhrB?)!-D>Z?Fp*fnVIbQ|_)~agn+LjH zG&MJq#j&?!pbOTSblOZzMaGEC`}2Ql;K`V|-~Paq%-o>`{E#AX4;Wb#)ihDADc`&b z_ITvYjBi^FU?_v;3nDcA+RNOX@{&DECs|%GgqdSLiQ9`)N=wqKv>&qHD0(S9qu%bVCpR2|MJM)VN7^cVreJ+hlb=C;z6w?Hg%C@KnjKs_!OnybA=F+c`2b5N zJ{oo91sGE%43WS+h;c>O`#kw;!*usf#sej&hRziG=`D`RD7}D@cj*!j{aF01I#$OS z+_u`&xrk$Zv)E?{00`mVA(&2aEiJ*3ty@pbNLmrnnMOg7Z&^H*oVE(u6LG4C(TkGd=k zO?JO|-=JmEriV|dz(=gsO9|J~5&}CHew~t*i--6`7p6REBx|c0m^cjlN>xf0dfu@P zHo@S5C+*$uD+I4VCMw6>MXy=ef7fH3nq+lbDbM^s7qmB(2k2UuMA7cdnmy4+VnKQ%V%UP-k-?|O{X4eRb0UlETww!$4sNigrJmB6+k#<+9 zW}OE#$}TAs?wT~GN;8B^pdzr{L5FV?=fwZEVd6gdabQim>$^X~H=>^6p!+L+2SwGX8uh+ z!!VrX%nj9G9bBn$^K`(8YHYUK$VsL`_yLV{bX0jb>L970e2Zg1W?do-Qo*(7JF@$C z^5c_795PDZvPd3MN}F-&_5_Kn6?K!2X^RkfS3k#*_i6m>PJqQej$;0nXl)5uyhzt- z$I|)cc&H~dE0vBzD|_TB=SU{cN$}PiZVb9PEW&!{cWz%`k$Q!VX0HFS+woRkT@2p= z^A`BsFh592rAV(Z1uLN%X90|L{;-|b)h^lQmf3n2g+9{rB)eatM0G>x32IIDQ4Z zvfW4-6uw%q>wyQR?!INVe~6Xd>XK%pI}U_m(e?aBKQxlWo$q+A4|~B5{bA9_fy@q> z>#5YOpCnhyE2D}lBza^)D~UU5N9*kabo#j)sY9~)Ss3`w?0w+!J%3TmUWN&9O?+pSMmrJCQdr1!7VJF z(vB@@ynT_U@lhj5Om*u7N0ygAT~I7^pHemZ>VL5q3fC`B@wX|sYCn5`g%h6_>g6^{ zp==(sBnu&|ti0Rmb6A2?{{=Vx{q73^s!efLU^OUE)!k-69F;KuZcg!9 zd+Ii7q_Q{p_CSI5-hFfeO~Ji8{{SdDj=*34F)EGh&l3Jd$c32L8zj8&|73YGa;_}+ zUYa}HoOJK12h-1gHFoLx&+NDhq+twWjh1-Did(YqdB(S4lGZf;_3vIjGT^8JOjch% ze7DcBzxgz0_CJ5>7oSdu!19StP2rz&|1W(3d`EiDu>|wP@%}ge4+D@N z@7#`mqbvU50f5g`x^O=Fv7PgR>pzeD`+SfOcH?g;`u_ku=bwN198$WFrdRae{C@+0 z{LIJQ`47Jw;PXk&XG)r4p2htqOZ46lKzltGolSM3IIc z{r~fXUOb1CE@<=W|2O{+0+64bXJ7ucto=Wy87Q0pmT2~A?z8_q@-IH%8i4Na{`VzN z0uD?fSE%=|h2H-?vNv!I+W1uee{^x599X6<5fJ**zK|-76Q39>=W&E#u}TJfh?(7*VOr*6BoVvzh_G)eLr&sP}xZ;GD|o~ zz|pTPdYMLY03H(nPUEcDf4syk!!KkWvhbJ9X5cmK7%Ta+og8ODJ%0bJBo1Lx02;Aw zE#fz-^DSMyUvP7%%_VA0KUH?+CRgc0pQoCdHhS5hGQ!$U+v!mM%8q}Z~k3Gp0CS~21?9L>hYx_Q5Nzq z5HhP@jU?rTqh_;4EU|yzud`4R1PK5}6&)8E=YAcACEWJi^H|*-;eJxi1Rt`PR$7H6zQ2nw@LcUL z%Uv;Z|93SjeG0CQs*K8kaHTeAPHIiO+jYOAw6a{G1|W?YO{D4x0tw~VfV=bUMAO-T zixw>ah;=JGJuvCk+ch1aOLQ+Mm^yjDqIIHw5++Q2vJgpOU-X{&jr{a1AKgsP1 z7p8t;o5q|ITX^n#Bsx_;+$z5vp{r56`SI;T8e9L6PRzo2f? z`l$M9p`&|o2-Wx%a6RAz_Rz1&V=bK7#7413fYG5e(8y2iq^Rz$EL40ploy4wDVL%r zy&Nk{0wmsscddL*G!RIN{XY1d<)8a1V}a5(*=~IlsljZo+Ij_WFSP08vtr@8qx6kp zu<5qA_3UiYZhNJp&-x<&UXt43!&2Y*-g37YLUbD<8c^udD=*AV0Corf2bE|UL|DXM zCuwF6I%zH8>$)%pSjBJ}n)-xVSq<}kA!%;#|E*wQ#s8c%-Tx$)lDgV?#^4<_t6o-@ zfnL~$13!ae1I@aH-+K!)P{n&_Q@`SdHShc5r?MHwizsI2ryaoPv5|7CE~dsi zJ>c*QAp9p{shP#?KRdIDaM0ce0qV;HMp2v2+=R&3WT%FDoslcj?jsHy8bkVqAb#tv z?VK<^L8K1A;*W2CULWkw{-MKgotcwXCq&!`2!!}0+vC_O=PsPVlW708xea$8V30S@9Gb;%3-EAO{TXzNiQqFnZF$1O!dq@;v1uF3B#KuY$A9Sm7%h`j zMU~)D*^f_OGW<_%492)3d(T0Nz_fDmmgQ zQG@H;aY3)FrzGVt@=!i2+6UhUXMbW~&_}BR{{%S9U2bdwh)V0^AwM0gSAOns@dM`5 z3YBYrKl++zNGIM6R>xKsRzenQe$iT@86$48`0vGK1)OCRUncj3`?o$zR0w!Ceh3JW zzoy#HEfA7Q_&ZAs4JRt?SVhgBmyA_I;QlxQK;+>33xDr_-59Nksk9yF(Fpw7v9r{Z z$fuj9m0;a>wNj?1-fWVPEndSQX~vqrVc2YhnQf*l+9gbOZS=Zpz07lNsRqfm$d!?!0|b0VYBW z*az{i?weGX_yD1v$d338fx!uBtlz@5@#{=r zGB5H>MiuZLjXiJw0nN8Z3m*<{KM#_}&Jee=$IYU*D4uXFXX28U`cj1ePwedB0wYZ9 z^K?bV)M_UTw_+r*lRr&DVEN?icektCN~V18k0Sk|Z$H$W6VS_=eTsuj7lwwB#DO#; za9_%~cANT;7uADJ}srxjpi3nx1=-WGwV4eKfV zkP;v#n1UfDt!EMCE8WjOb1s}wRdM)HHc9x9Q_S$sAt>6T@Swh1fsK^akePasrSaOi zHra)4SmT$u7VQLZ`lq0WvEO7P-CKjs_8O*q(1aha7mMBt)XOM0L#G4pzcwnN0rcW8 zfRNcm`Q2aOJW!@&WxDL3-{5o3_dU>yXzdMof&bgb?dN=ZtH` z27qhDgtG-c1T4aDd|Cc9=`4oAQ3*x*>gohtdon3=rm#riel7Br>lc16(t<4@)(+}} z#TY&cADgUvz1X!~?Ui`R3xLW_j_ppDz4T3CbUXxl1ssLP`bJ$Zr)3)OK=o4? zTnKMGHY3VvGu8VbQP?6Yu`cC8X58Y zyjtH42deb5d|y7lzC8PC0K^tH1J}Cb+G&k`7YYPhi9ZMt-W-vz8?=(%*e1zB)bBn! zIjtjUPj|9O9hrRv{J_Gu?C8($uB|2airqtJ>S`aa=6~?neC(08G6x2|B7Ia}dWl{< zIL~fMaz-7iK(Hzrl6^2x3FPZYJ2kCxdlyau2(Pl1OgYl$Q}U?$`HOS*?a!q;=vIJ( zwvtIW8Whxy;QfslCB|qYg}dKflNW!TO8D+N#4WwEVDZCt;1B7Q46+NHj7I!+c{rit4(OE|f3 z`{&ObbKN=CKtSuvt4R{<4^BJeNEUa`vURSz*=^G;sxANeUD9*_Ds&E#H3RsS>T0nA z5om9H^C)ctWVbl;d1f_yhzP^fxh`ERfThR_I)T^`7?K=Z0SC zK_@RjL7U{qOY1a1+@yJr@XOR!*~H+XN8xglIU#vJL*0F8Ke6J!P|A#ZZlM9#>!n9+ z=@J^`09vh{RFjn-Jp4kwm;Zq&;B7OS$x%e6Y(;7Xa+k;Zp>xNsn}YT5=F7 zk$)x#@9JH!0i<}u0doE>nw3Uz&wq8QFto%}VRHYJ8Nbyn0zP!M;kGbS3_L~cHb)4p zo5Y-3@PxBBI0bI#%s4zYqX!+%lGSxP)I=Se?4hJd9tx;^>{Crw8(hb&=9|X?^oQnB zZz^w0AS2k3K`7;R&5T#*_=|9Rb7%|X0Z!x)5e=k31f5$@aXJ|E#+h^bTL;qcZXA;} z2&1FrZAE}5ZDW#zo#XV{XD?ipJ$etA|M_{de>_6mNB)eXwkpr&pBx2HPc;} z&)+xl0H;a6d~6nA_Xt+L;(Oj-vl%O+mLJK;IGlA;EY?oXJ?d0*P6CAi@I|~DTY!i! z!%~F;evJR>KVAS(?V#u80mnvmGmQ|bXoW}nzvxbqGqdV-$O6vIp1cDBowophpYnoT zBRHMalJxTyq%vW;yH8kuv$y8e&pXoChVH3i>#=_2r9HqSt3{rcQ$&g#G&2PMj=C-1 zf;h|Cv&B|wD+KcWHhOas*$~clRMS}$axXw3ml{$P@*rTu0wr_NXed4CEROF3(G2=4md<~!`>cgX$8YTk{r!==NJ%d}}CIe8C;W-~5Pif<>F z^M3fLEKqTY>l5@{_R@hnA*yEC5b@d^MrUmcQ`Dr9-*={K+T$HYIw2(P9Q~2HoJ&MF z(VMv*hYglt0s%G*9N_edc&3(!o1hXf7|i%Y?RxkOoGz~cOqN&}AP9YUkda&h;`Eu8 z@N+UHnf(zyal64MTlIU~6>AYf$m^Shwz|az_@ue&lkrQWn=%0-4Zu)Cs*#w7w(tOA zx)DvOR7qj!;iml=Xvn9pj4V|giVWw-UlV`7s@1uVg=)-fe|Ew>5Ys7ai4xazAvDexVw() zq3@U9=`!ZA)%;)G@Rt??@HfC2dyHdZk*Wm@qnp1nu?(wgg>s#kRyPES3=XNEjufI| zD+<-3$u=fUVH2S}mkP2B*M40KB8a#1k3Qw!eI`%6VPimC-MtS@FwqI4Xr_I3VLgO@s*_P&!?0ltPU88o}!MuW#au8~2 z(GABjY_ani&i+3Ll@<$$A=}Xitc-Sb1JD6_bWEa6+K$J+Pc;B4f8mhb2$zu zqOA=agtQ2*6nzcVCel3r1I&HW1Wj$6Qva>X|4-(77Ou2` z1O?~?F<-1yRyOg#OE*k2`Z^|z%bgUFJWg&$q~ORIzKg;~iw${ZQmt%84|*!N9U$Jv zN#aIXipN~6+`um4irv(`e6^%r7~-eMpu>1{8f?q0Waj(rnF|*2#o_ub?+)Z~b_{Df zKsqxYQ&Z$tV(w2?*1+PJFDt^9%p9fw4POhM_%4sQx_Nfk<{(1AJ@(m?eRtS5!co!J z)nGrecKRcn$GMZwGS6L0e0*IV)-bUC)s-+dPrrogF%v{=L5cuJz<}=mkEMoE6FKR1 zj$wBzGXDV*B^(&%=Lb^Pbd@YxdVZZm`O%FBv{!W;b8=Pc@8(?H@|j-j zNj0;vaj=2f9O44EL_~RJ_TOxn65g8X=j{{WA@+H5YasQO$Qz24l+y+RLpS;g z^nE3yC))G*uYz7QZraL;LGl=@)Yai627<0>M8J}gi$RC^x!K`>eCh@~N# zsKA7ZCh}V^V-wBU5l+-s<^Tlqdc((aMo~$FT;WLA`dbv>emIevFx=Z;$L+tL2mwsT z?0qb6pbK`0AX2`Dr}2%)hZK^I-I~{6sQ&UlT(fa)nlAEmVm%!Xt~bv7`aRJ+-u~Nk z$6MpznJuQe0+;{-L?Gd)sulB?JG({w207yXl@V4a%VoVP8uH=o73^ry4xn;Urf zm_Z1rpXee_QE)Fc=*AGvULZf~gx{=Q?zONabnEo9XW^}3UTw2L@k}+ZE*7lZ7t-c& z_ZMrK88Ds9H>+ty4_k?-x3cY%xcwpRYj6~)kpJ+@H@r6yiBUuJRUE@Ul76M!?LOv5 z;w^g}gbD^pCsRe?J(mf>>tpGOv+@#$$I)h8`0)I;GZQ#>E|q@{!Jrg0Z+t2KegD_K zAe)C_gI_+CdJcF8X6@kkYtxb?2*?p97wG&f#QS`V|NNSN^$-vI>LVwUF*q0I!}6 z*iJU8SjtP1m-IH{pEmaxAP>=5&;F1eCLcyx&WP;0QU{`|I*L)!z_s~A=neDoBj~@z z;82%X)})x2xA!{D>-7%=8-<6tDfo5oc?>=kpW5AF*N}S^u>FQc;dCrOZ~NtP^IoW7 z9ga!m``lSM#m%cls$<%C@IGX;a)4xV`PUm;yjphOIBRylv&EM$^lyu>3-|AfJ_#>^ zZ+)40Bb<%W*mlj@Iej#bHF8NkeF`0B#jgm922wbq&P7xXCQ?o{#B#GT>pBd1R{a1Q zNE_AScJ<c?m4)Qod{Q5* z1LZ}07EWwdHiMnr#@4v`)RqZQgy^YAM2WEI!*<(-qO zQ}2A5869+ZnlED2A(bz*Pw>|JZU>O%V1t1)R@BL7U054 z$Piq3XGcFhu}fd`h-5C=`IAP4#NvD;V?_xV*0`k-dV&s5b5fY{uUIetdhzO40TeSwplo{_ zsd3EG?e6Ayv6->}9rey{0A-H%3(CP%D}|gM_6|ow2AmZqETg^D;Yp~!irc$6N-r+G zt*4uN*la~b;aO_G={{WtUM)q)9+JW9b|4Y~Cp2Xa*zFoM6HElGe7s_LqP*R}eR5%# zJpgbQ)a=Vlv}B$d9(?R)xpu!q?|DR12*@9Bs#HfBku>b*X7*UOHy`%CXzx=kYJH#ax0TTUe?l)jkF(Q$2Z z7pfOnA1#ES&60P%FMm2L-H0droW`VLp%~5m)b zM71mCKgqV4t*Z-N^9;B3>ef8VbXycpIvcMG_C0uE>{%3-pJvxP4Yz2?(_r7-sL+Dq zSJaU+4@IYxsrd{?tGy>N!v(nkC*wDuG$YS|z!K6QPa8l zh^3|5p33j>ttO`gczujPrkrO!a2SvR(ozLZJ$Z3!n{aaI0#hW44AJwS5^I~vWhC2G zIpCz$U8g|}8yjp9guj%!4oFB2Q>d^&-oxfPQK(UX^Wkr%sw+GPj;*kU-4rvIxwtZi zwQtaNKa6`Dh15Rt@zeBA742+T)|EdPUoy5x0;UFEud!@bV#enbhNMEHk}KUxO92Va zAt}>PHN>>SCaPqy6#BO0Lo><{;b2nZ6uM42we`tw$_vOl`GEk{ydqcSohY*mulCwe z&4g1p5IU}_n5wrdfx2P-$GeI3{q}M)NZZJk+bWk%*hNoIcGU@xgSDzMS^F3hry8$K zS(DsnHz7cvMhI=O{Ycm0S#3?J?ZBPu(UZee7ukJl6DA)6xcqGQRa0 z?^O9oA~^#oUyAO)FM${Zbw(l&S-4B`O5*TE;VYLFllQ~!$6hlmGF1FoR=4B^U!LCb z8X37(>b>)N>-ZWqHw?MO3x}zPyM%Z2?`|k44jlRAeY9`rPZV9Qm08DfosaZ)P=;LR zY_A_m@pseIRXyC2c>SF;qwJc^HW}QQ_EM{K88Xk z?QY#9{C;_$BR%)lveYJuZit}uTcU4?eL%QhDyE{r+yiC?pT=9`2E2;Ew05s+XL;9= zz>?~5ryD6eK(>XHxzE*2dt<^vCEc+HscvVdVb#{%>>-{pJVC7AY+*lem@?z4q7YA} zv&Y0uxi0*te5}vh$9F@=XW`}=mJoVg)BaCIFUE(nmFFSv`1XQKY#kbQz8-Wx3-a$Z zgF`2AGXyL3hH?aPJ|MktRkJVK{1-T<#JH+0aL2dOwz+{bn%|ixU2kaLEa@-f2&L~z zM??{F=eQ>9a?gaddp!Z!YD8-O>{rn`)vtE;C|9!&$h4DE$b=$u;loeU+KNk!?;pu2 zMH);8`nebT>RI~%2gEIJP=)1e z%rjF>pj)R!476J16zoqZQ-++DgAAy;{%DObsi9!g6n~d*eAR1XY*YqX&$&%Q5tQE3 zc+xoTb6zkwHmjxA2Q{s+mr0ridD;kto&(Y$8dUdcnefx$PRMuH%G_bs*37{H_-j9& zV;jIU^4{9jL`8|AxCvyaal{OI^Ler;u0}7sBLPbb-$A0sg2CyR8AXnnk-rbTS1{mN zk`}!i_h0-n7IU_eXAk0dU7O93KO~Paqw*qt^Bp}b?e)ix_+2}N`4o|i|F*D93 z{8S5#r#`jLY#Z9}C0a`OAK6YKz}l}A5zk@q6zm%t4|nwV4uiZcwjB=NNqY1iOw)3! zG?>TM9KKr^@E*NlO2{&oQJjPs{(3+=!c?;efz|~N1Aj#QG7&Ln?`NUIj_XFvdN!PQ z9{cm7-x7D29t|Lwg8LzmV`1{hz7*@ELVYt>#7$YT?}D<<`^i@04wt5WfW``pVnl~t zDE)NGa=BxUH->Phty|p(fV5)Pts6yE`2E2I7aSG67BhkbuNA>u6((yOtjSIk z88_AEy#d8!nN&9)f_ZFgjcvFIVD&- zJN5nG7{%zRa!HJ7Fh_qKjtRmO$13k`oXr{ZB#O=mBC;CM_S@PuPs6UL)fzacCzJ}8 z>OL5B~WoQA5aYc%3neekZB*Wr*R#ZajkqW-cnXIE#(MKG&y! zfw$14hFuI$W?Ch7H)AjGZ=QWo2(Q55I?F=d<<#x(GDEf+$hSV9q1xD z>P@UGNny2aaSL6;w21psAGDP602?Mdfw9rvUkhw%|Dmd+Jfn}E-XLwJoL93d zn3w?J?eIO9A{b>tYMwj!!@6uSZ)(Xw>=(_~a3rW84_p1s6yC^L#YyaOL z+QG-PNj}EJ+2NkIp=m)}7=yQGABF?&y`K(HV*9qRy$@_i^=%=}oN=5=UHUu2y`n&q+x z>@2F_i1IZ3;oZ$LSrd5h({$+Ogf$`EuQScpHyB4Gq{FwSBqdK^2CBJM=vlycc88nP z0o$w`W~qXhGZ!ofaLQ0Gt7n;6A-r%XM2}R5qMKdXw9RY3_rgw7S?lXvbV?U)G#9nz z6C++lah-aj8mGT=g_?CYO*$c3!=Mv2pBTi})QFnH{B6A0Pl58ZwDshSr`c+-KtUEg zR*-*YdSO^eV|p>r7#Lbl>~9GRv@b6(_fBKCxw`p*UE2+L6tv#Upc@uk0Y+6k)^w_g z<1?&VzOScrw4h5clk(maH}qCHlS^D<7_GzP?Rhq3Z^sDG%&+~n)JYuMzyDrYRr)Szr(niY17Dm^pP5avgn$`U zjTLffs=hHWz_*^BkAN7@s&T@^n~c0q-dn5FUGJADHL;QPk4}!rx&u6P(4@%{5Le5e zMPCDPuaV7X%eZ!J4~uxD91m!+5)P!Pmy8;>%=q8emnzsR7B^bfx^FoLbR9COVorSv zM@;tOq24X8PA!vetba(ukj-UOKUW%_VuscoZ1zqGMjx!qSG~iJ%wP~&-?j@0v)4Ma zc6D%qd4lLbsRCGxQuD~D0DH5mbYQ&tENgqY67dVyIvmtFy5qS7CYe74K z=FlZG4yhmu8xYpZC@{5iXxM82~vkv6%1Dg9xtZ{nMTNk7413? zoGvLz6+mu`dE(wN4D`$KB745Ag0hdp&%P8n?een9R=S!66I^tO4MO%~yN+FS#W!NojG{8@>tfovnt4;E%VA4W+`pWW-8a(d`MOCWtr^D@c zNS_Q79+6@l4ejtr`Z~Kw?U->?7qSW zFMUGdrjvnjhbaP1Q-Fbxm>G)3V`aTdujq^2Z+Aut95R5{#hET~BI2`oHIv1@tFMi? zVu#M2!w;pmfH7&BpSY)HcH&MmlggmsFViI8T4ugaw=!HZ*0(49I{O3m;jpi$`lqQ=5@i|&af<+9%KZnraO zjD>Aay-O~ze=j4CH?|GAXc zO^&qZo^er)ffSrBEAptO1ZRCkw;Ca1^>$2>>uAQ0IN_I)QdC7Hi&Gfq*lZETK`U$p z`qq!T0a|Di;YzoWpZ>zwsch!^gKW5sH(cmYkzzT`-G-xys;Z*x?mo}=Jt-R$QL@P1w0Q*2ztK|NLYgVYCV!Toyvs z$M&8aXybjK`Fl{xtqx}~CfIbFUYGj)e%Ao|Tjn~_YWZ2~VY(htd6d819^z6Y>FqJ! zc1?pQob?N&G*4jl)Qcn2(!(&%=%6Amxud95=S7Y!MHj=?7ny)&Esk95dZ!RFr_y^D zPM2S*4+t{#qyFZ;0w?=al_^Kg^GsMd-qr|)h+EsarNu;_-Xza0h^J7>Q`kM07X#nI z1D{7UU&E8}1tE}vo9jF6?DLJ2w&|6sAl{+I<7q9KgEy7Mx&hO>?d)3>TTjKFd0jiL zFBaq(DM~BlwM%KFWDl^ZSn8>mSrYP?KiHf=$eOhGbo4pkW^}_mR^#XzAfu%-uNqAz zHq{dZ{%$r&nTKLU^7ybXt|v37cWT(l9wC> z7lrPe$G<&sAqjgMcHPT_e)w1r-ipvR-9| znwn+#>3yPykn}z(PWKi7D02yd&TqAh7Bh&3;*CEgw6V)aF)=GE0#1mQJ@+QHK{&ICb!X~=D)nxMpM;X(zpG4gXsvT#IqkI;=fB^=Cx|b zwVNbsgT|KEa~69pkqXg{lnf3C`Z^I9v-rZk&B<&YpF*m=>9QZeP2GDJ50CS)M%YHO z%^_4?tt+*-PgG!{b3 z3oeqdS$K@!fwNXSfcq|mRjt!Qv##*!GhF$bR|8gl+r=A4QNGKFrnaR1c`Q=5xv8mH z%=z&rmSj#6i$sxJOd^Aj(GoY)jtMxoP-`sOx;sbX;t!A2kKZSc_<&&1t?77P{X6~L z$}xtv8jm$!&Os{8S-XHk(i^0i3aIIj7rrs5WK=Q zeUW$dX78_(7u>1wZ&q1(rhI!Zmoa`8_lc~B1<4ZB80h3lL-`JV*Kv*!9;5(irp{>HpyR_mq@T6&wt%jodeTC7$fK zIROqY9io`N$aTKcBWDEID%m9!#Up`Hv5>K>R&^2a>Quqs9h|!FE+@(g5#sIKmjztV zW|_@j3d@Kt&bBvtd)8svRU#{)&8aS}&Lnj$$qS3Mmc?O{{M_9Y4zzM}9V{>h7aOu> z=X~NUAm>iKpM%vtxT0ca%MWK_H=1)XY4A?$?f&Ji^PtzAwy1C1C@=Go;9eVk7?nN! z)+$DUIM`cTs#K+hQjlsUl*!b2V?3Ua7RsNo`5F6Xkt9`sN`Tndskek9!e{$M4@f)_nCx zQ32I9jpFGa_9GRN>+1NntSgpSR*=D`lnm3WVaBWeu1> z1kH1=eDwGRq;8KLph2q=jC)(&=uOGN!*MgiJdOMCwJvpx1zS)E9a(my8mB+{b*aN^MpkH{~rgp4qHVtf#R%TNzs=VVJU&Sa>X%`&R6X+wHj#;HBre zdH4+}HB!v&?Nd#h)$4wvX!@HxEpsa>V{Y8EKC3YS@ zY_>&zgB>T-e#E)M!8xp?4e&v+f_knjZMEDEKP9G%CN-#9FZY=_0fSw#NRb6i$!W_1 zPva%%!6?2&V|uW5AS!Qb)&Blu?mDMT4~gnei}mVM%x;lB-z0ezO8Qmu27mvp(72*B zD+V(+1o(PA6_E|b2Y*&d2yyi4Yj6BhHt)Q>O(bSKBsnf)!%a;srEbeJP=p~ik}uwWAt&uUV!kb5|b z#3`%a!a%;peCshDDe3g4%F#0+j^C&`wY~JoUU*v*^Io40pK;}TX7{pA0mxLbqR+dw z5P!d4aIbbVTpg{Z^5TG+;Zyh6l7Pr}%IlNKG>AU@I;+*ZM z4v8vz1L*Q_oeXA3UjG6=o%Gcpzew3(aYOU8FDgH-oj;S5*l3@0Jh;5q6fpP4-;gWN z0{Y|DgIzPIJP%7y3DBdLS3&1xEOl^FOZI9M6NHc(Sywj`Mxw+Fk{>Em75zW<-a07C z_KP2WM37QJ5l}(F0HsBwQw+Mh1*AKch7|!(NkKxoYiU?&2?;@3q+=J5lGt6k2)(R*$qEIt{QJlP^-Ix{n>}JxBSCvw#2JF(MV1iKr5>M#n&7k0;r5z{MQz|xY`@iom>?*$!lE^E%_767lQDNNH8{sW z2fO&hrf$Q2h{A)iKhG-l@#!Lf4*Gz1pXhSGxzuFBCKSYvUY6XA+NffY+qZoSw{eD9 z;AJh!M;W3EFj-U0wMKkmZjVN9i?5qx^*2#3+=bZ|K793MAOqsChl!Qb320ppju_q; z=9Fiuoov*#hbT*llS6z|iTmtl+SGx4I6N&eC4EhQ1Ry!%(SG+mage!2nXmNj)jl2N zVzKNSEl3+Rz{XgxmKPbK#3B2MUWc6M?CV}$z)y2uc;&A{02|08reJkSPHwo53};V$ zaS9{TbkayaJz=!Idkd~BNaHv+c!7=8EWBtuu>OBEu2AZ4fjmn4!{N-EEqIUS8T(Wq*_5nS<7G&=n=&!H;{sl4 zdv&YmCI)E{!KAW|Q!9B6QJx`Kh65TgMvi23YDpu_4vO%4t)ozLkVWo_0o#SyTndXr z%f@*$We`}Ip&59UR=}gW9uj4D0xH{&#YDUSv7YbN#we>cxkyMr(>z8xVWaP&VGh*OXv3NYk!HN}Gwv#hUMIDwznVOEWbfJ*G?qHHKfmE#y#;Q{=O|0`%BcoCuUPW6BeK7j zq2naA)7bXT%RcouJgpnF(1J+p7|=tSZ-mFE$n%uc+b-|)*%yJF2kPE~b&r{Wy*f@qbfLZc_tl`a!zH@#5zDx@H4eQco1V?^-5et|!8vqhI6|nzsjwM9 zz#vx({|6iP@_xQ|vOx#2b@S+hz2Emii3H;AV-*jbVHe}+>HX65YFP3Eb zrXjz0)OkrdPEtwf#{jzi`w3PV2+hd+fZ-C-kM&d2fFocqk9fhqEl^3q)%z1L!h~K}=IN59g0ze0eA`p1iIj zIi9_aWb7e2p`U_eU-;3E`k_XW_#08xDrG|FzJ3et$2(uCf9#C7*NG$kx3x<12$d9a zoQ8)8!@eJV30``Oy!%Wi&4J0&DDNq=I^sBGG7(ps!iW~293?)~IVtJ z&~9d=khLzsMq+#WEStGLip50NvQg1x|_b{En zHK0whW{+TeOXBNW*QNVD+LzsS_WRsJD{wA5$dA9-x0N4{KM%Jkf>N&T*qon-%zr`t z&qbQ6f(vk*k#QlwazO4o98PC_^CTj&=7A3>Td; zQ-)u2UVGI{Ej?}hp}w34xIb{}jy&5t7iA&x0C%L$?!x|~Z>@LQxlfd+oQM#|)?xnp!T6V&?42Hl=R>o)+^&r=J1 ziQ~wA)?p9`Kg^v}~9M5I^kv`1n80+;=gB;gr&zzD7xXb{E$F`pzxMl`kKb zq7rL~TysFOe;8y7KzZg)gML^dsl68H`7E+eU^!ZbEVJBcbV83@5spZq z|9RlR#UNOsOv2NT5xb_$lO@ylfu}q`lOmXke=L}%x*F8c8{ZsGW3xv{#?kz;#s97} zZ9TBqE;^oU^z%s_4@PhM+*c9bvT6;ZTl(D2TFi~QsITY6+sWr-8f%{Dy5?1)0vlA{ zX*^p3E>{FZDw4TGPUv8a%WA{`S%tFNS-s*iO39{c?oi{&nf&NYgC9EDFxY8o{)&g~ z*VC@(stsT%XfuV@J=3wRufzx(9qtvF6-UrO_df*lA712N3+&Amf76h?K#6Hg5OlX1 z;qJAX51YyN%8Yk<9q(c=OK&s`K+$TMuEzi*)olAm{Cl~*GP*stskkhQZ8D)Yz>qF( zLlwyhA1G7y8E*~+P_P_>-O2!`s%ok)*Dgf`=A=B-=xeX4wx;Lx&I|wx%M{ZkhZ$}S zuIC1mxCeIResW0K&C`UBfDWOCx7^64;@lP_u8|Zu&f9B&cEwt1<+A9PWM7%}iat}Y zYA==5EI(h9Cc|cVtaWacISk5jpOE`NBvxfKBC9hW!dLY=-n~mhweZP_^0A{S0Fy5Q z|N6al&0eTvxunIr&YGdtbi4GftZFIE#dnzI0X@ms)e@_T!t)AJ?ebR zpr=(bYb8_vabbLJl%re7MowTS5R|0t9B+!B+f?mDSDui=i}Z6_`w~!4Su_m7o}uRO zh7V#_G3N37$T#M$StNkDPz8ZjL@hSY7Yv?m0hzn(AnKKo6W7A)WZVjfT&tYi`*f-J zI@X0M0O4>bfoABIi^9+r6I9O#Jvu&x2M?FIZ){t+9#)mm(|gUQp|r~;LjbKttQ$AEm7|Ania(AUQp?vV$z~g!tl5qC+`N<>TnB1Ur%maQ*g<*B8m!c>~ULAT9A)Oo_tXH6W1@R^46aa3&exL7l z>Z<#?7Xo3(U!DtMPMelTl$M|^O?KJ*%C^tIWImU-k=gjuerWG(r?*8MuVK#oCE#?P z-9r+~`ve#lcpcT0%RIo{HbMgAl;?pFk&`EfU&pxYUZ38KE@a=hb7W+i0skzAZ6u{j z9s9apXtzX;&+>5)_$U1dz$8Sz)>k82eQxf~zPfNWXXCBrwd^?8S!l!OBhaOMprREG zvWylia2%=8wj~7Ct8|U6;yUSB^XqRd@@vNqZf`4PeNHjjbG-FX!8Dyz2ARN?FLri)zirw7jm+M|_yHmN{U$O84j7@^&uYv(&d3 z$am;|)-XpTc6GAr9%R9>`|GT7vZsCl9mWVGd%T|Kkf}S##=M7M#&%F^5CvLl<>-;> zolXok98+>wi~5`!wC7IlA@FbM!JoX7P#tdp8d7>io-Eq5Nf{Qaspii7Ma%GJhP-l7 zLEI@+;u8=C=zmdt_Y;?uy1}^I(O4f&4gg78JNSvp#FRPdgL5J$e92;QfVr>;d)7gC zZSHVF)0ZK1X{bmmOR2(Wq0B=IhR9v?bRKQq2tbF2j6+&JA8xSbPnFFJKPgraJyRqYTF#&t$Vca4_-@Hv zwjmSpXrW+>(>I^YObJ~+SRq61b?hhKk*p7C^VH!Rc|B8NH8MzpP}OrAuhn5H(8ao* z7@scmS21;Sm>M11VEVLc--cW8VMc)J=#apgQGNZ+ht~jmz4saBjrl zi0ov1;M4R)ZG@Y54TpIaG0}$hy)7JMhi|g`76!h1CT28__g~+3-A1ScTO0p0_3>Fa z8$Hx?ZPp7bssM4V*&TVD!P@PmumvK^iS>3<^wGiOIi}54@Y=pjnHXO$v~8 zeCkZWNg%lchYky^WnO5}tqD)At|f(Ok`H1gOIif0{shVJ~N> zpx9xt$!>Z+cFo6s$h=^&@_GGE0zy->c6ZeQgY6|5mnj1=Oax}H!XI`SXM5c#SE9U^v!T>&+!(F7Ha17^MP8-NA(Z6g_7u* zcu`bQ%-RaJn^l8XiZn_dt5(=1hsHuMeW)M~9;c2wgOaT@l(JMX*weaN!7;DPy?1yn z5(z~M;lYIR>F$ylh;F0(YF;#=Q&A8mD+gSBF5XeATbcRSrEQS;!8SFK*sqv$gJ z>CgvD_4pWexT2ElP6YRuMV9cwF*03do_;r;VT9W7wQdZn$XZD99rQad!Fau_eAJ;6fY8~q;Lc>r%jZowT6y&__mSNS@FUt;lVHS+czz%=`cw_EA!W0EEbfR>}L4N5beGop+OhOZf#^|?jdM3sA_i? z-k)r)RLbJ@7;4YwOl;1yL{=H*7;@F5C-3dI&}f}xAyu-P@Vp4GgXqT-cgsOSe9C)E zxK~oyvzyQNJr(HQbZ@ZASXS^Zsrm@p$at;WkJiu$Rq1PBw-Ud^0=-hF2~X8=xiS`{ zzV$FUy}3hKVlNKicWL_pR8M$_#$bFAFW9UlUDg$jyLeRi8`}6~%+jN4T8ES26LES+J+tPsM~cqAnW&;5DWp*;%Y^vI_dRs`&v4 z%L1O5nji9^&a-|T>$W)>>N{m|b>TST!X2NRZcIuZW7)OSNzv=QvXDhDX|hk*Dv6?W z5&AZ@7otlWScZ=Lagy{wKrOlB0F-qVEtXH$Z=@1saq^y|)8*EV=s3kqWYM=PJ$RHe zNW`JsQx$ben95w`$vaG}WF}%X);RgB%026eel@*W)ogOjit8tZm!%P%R^^;$aK*5} zR}!5dL*0$zvV>&hH(~siSt*egMB*w!02FLdzhIMT@=zyVWuNwwUZxt{eBWbL&ANwW zud!)jhE_lOlEjbq;{|9p>4o>+fhHwd|FvDu`I|tnzlT>@@st=*25a8 z3Xe(2h@6NqM(8OmC_*9;I*BsRy$_}4hUzTCG~bv>i{jn!QomHKi{;d^t5ED;tc4Xn zsZX(u_0<-FXi5$mbk63o4o$w+bTV6NmVac(vpd&+lj2PquSxu*YWJ4rxfe`cOmN#p z^5X1m2uXQ-oi+!t1(z;)(3MNs+sSVeDgk&Q(!AGLMyZ)x`77*Fq))p#TW+5aQSOm@ zBWFEKu}1h{{Vjz{Hia9_bCWx)G}lG?Hu=i*o_Et&9hG#wUXH^;yFLredDVP&SNZ62 zAMJoN2F1sNsmhf~9{%uQzpopBV5dcYa9JOXl}UbLyCR$!@6D6Yc)wpyKit^&j7lFG z@+m1w$2ODw6zAqRRe?VyECw4Ih}48v?Ru<60fKuJipSNV#yUShTIcl*zmR%}QlUbQ za_hvSY!uV7@`}%6=*)tO1yZYSBdSiIq=2ukYVHNo;SMDJ9fQR{u0!l#mX}8rBHp&a zZnKd@n5Ga3zKzpeKaosH!DIuiAq%*AV+$(=m8V*eR`{KIKsW zE;e$(({^iFnd!Mu7L~QJWa&ZsTiJsn_Ik)lVo7PY>oV7T4R)|R(_%gcv%i4KPsY1v z7V^BYofR=eY-f?V(42@`QUB5L>c0ao@)+-K@ z3y=OCqvU^-$Rcs0?ixuV>@IU&dR294@5Ce)rixGRYF1Sjp}*`^0z#Ng6OJW)=ouP4@6 zwF`Nc*%d9gp{Wz^6?B4bz?=G}Y3hz@N6QZhsL)%C)|OZeRkT(? zYWQ8a|Fj2LyweB?(nH^b-ce^dcYLgLIQu!(kebjPqYTa(I-}DS6Bbg~15j4Ob`o6c zJ2P_yqFt;o>t^TyxmR4{de=EK&-6A17JQS=1TwP{Z7t`f!vPnE*^Ur(e_*e-H~76I z_tBimZnsI($h?v8^r(&3VNUEcfeE$%_Pkcqk0`_aBN2JXY0K2b^;=Yg%%|N--V@Ut zSkL{-zLS|qUbEiRlnJKduGcIzwY;&_t(&Wn-%l=(DOGylP<(hco z<33*Ms?~cjP0)R#=&UoGkXK()cHoV}Z=v>^AtH*>iswD}FPvxGmMXYu&XcvM2+Jkk zGqK2f(^cU(Nxjd$cJG9EI;nTarV=qdoI2o9a8Od1n%EBfV&`>s}JYo3%=3k z%;TF+^{Iz3{vgjuJqohRynE>L3O|=bce&o51`$GZby3^JYngFN*N|YhM1tv}cC}0P zK`C^0!B~dsN};DL^UE^my@fawM_d`A4k{ZYJPhbXTMAp6;RN5Kp5G$6nSv6o-)jLJ zfABI=ulZ4IR5WF$VDDQMTIbm{KXXfuO%@{b0hw4HoZvK&(cQK`;ZGGoQ{Y_-O~@!*`^ z#gJ^A?ucH(hg{>9E?FLkEU%y=A{h5@cp3v|py52F+pX(~^gp#xKvI#0tT-mhy;;~^ zIy9LBy-;io7t@hknL=W{!u$|cEqbHsJ#>7}ls2&nG{mg8 zmIJTCT*7U{EOSW2#-rMH=2t#kEN8pu)|PEcciXDzuF1^)q0q)cdg+2PQ}?7z2#B8z zRt|s=6$Fd=f>eS(u7UT*XRwPN47}Jy>&VCnmspQT@lKRz8V6l|cfNtT@a{=s)|%<- z2oS;9oHtQgN^nOYmhW>=+6)!eU!R_21w~R-3gr?-sI&w&uWdJ~nrW6gN`sHv2w|Cbp_Ul*;Adq^DJs zkJR*By9V+`C#wD_M@(HUe<%8(MpsSYJNXUO(Zg^*OzYyh|3C;edlUSgvTcsU(>_0^ zo!1fGdH0$pL$94YnaHs_I}^1#y3lVSbImdQnEr-(crixD?Fajf^;7?L)SK7MjeVy_ z8S#iI3QwKM4=Hu+7<|+&j3Cd0PC<_2L}TN56fC2NMzb%G5{W?$SQ5ZIOjta63$I(# z-D>xC=tA_}>j^LpPTy8TYdB11ogZ5PkP2X&FcQwsg8H>_N$9!ZI^Z7 zx9_}qRmb{9H-M7OSd2!eAGlO`HJnREd-iSqPA)r*#U6wU89avv=bYxQlp|C@LPObMw1azTOhLzW{K{H}MSO-1C65s}G7ZVLh057sn@b#WLyaWJC~Nq=$)>N)y5+LY<<5I%{xhhHVMO?l zvv}!=UUvs}*S!S36kQ;a+u1+KJ^V_OeRnbmV+0`QNRD&X0Rn*xk~Cn;RRBUokd+m7SuR0>I*Twl_y(M@tKYTkTYJE zfb~_%M|X5EuLLqQtsS(rY^<%-uJ!M<$mJv~I9&JI1}Rz@9TiGfGAU|jOHS_Ech8gK zSnCQjt=c2=ZIg$yGZUT?)qY989jc^e3Lz2N&C1a@Yj&^xm{K%mcaJq*)9P_m*SQr8K!Zh-lx;t9?4SPSi?F+6!U-_F4_O^A-^-aSd-I2z0#h+ z@ZyV+NTc5a#ZOg}_Z_z(2`=dn zTM{b*iGlXLreHB|d=XTdr&V5A#Ln+rb@ImVDv4B_)EngCh$(r8oY4E+_g%8-aavhG z-@0DQPiqCZtcoQo<_7Q9jrq?my{G!)#~&fbKb8|-l;3k}E7SH@!ZROaxk|Z_biS3z zw&~nV9_j;R>?YlcOT^CFoCAl21+|n6b8m&+$>y$r7OF?_zv)NPm$acEeRuHK2)$?H zJd}W}9DQ8~K)N!JWLi=wZ$8Qdo84D~isiCay(GWexPTF)MmtH*z?-9aZBZ~O$D$?^ z(_E?0XD<&jmbZW3(QiuPX0b>Eoc2Q5zqn3vfan@Ch|<$xRO{-L`0e8Scc3rJL@@X2 z12$ErCsqjX9Y|x9ezUy3J+?_3U>s(f;(k&6-)HK*O-@T2IPU(S^ZO^?9l>u`#mmZn zPUv^M;yv-l{vJPig-Jx@V%^q@n7_T2mec{DHloQKaQ@h)<41o{1Ecc7p5gpaqwD7Z z(9VKHC;P2g+`leV6R4i7zmK9aKPJlju3w*tbbth(XCd`wtE~fSLsIvfujklKt=bv|eCTQ(Gu` zj=wI40V`Na2Lst3=jZ1Ei0l)AQDJM6e-yd;W#0ck-QXPquNN2Np8N)5VCQ)-{a_$p zYixm=(Lzjx>4NKnfn!YYANvd_lRu#m-1YK%yabJ6J&({%i-HrkMH{pc#BmpY5$1pQ z1yryg9Q>fpg73gVz22jO-|zRojQ>yX5xGu$Q(Gj<`ooH1U)V9Lq)7^rh5zR;z)#*$ zh*z*SSF+DOx!K=dz)Eh$xp-dwN&Oe{Y;#X-cw4s6+V3eKFf4KUV^tMSm}v2eFHC~S-w-@kl+st3(DNGa`90uzfly6kL2@_@Ego!dz&dzxL zyFF!UI`&uZ>sJ|A{xpyOR_@hxAYZqf&6@)?xZJx0nA8iCIJ2NSaea()Iqbj-0`8I% z|4v#AC?2~Jov5y_cSI>hZF{wjezgl8A&al=5(vBTpcdsZR4%W`fHYXZYJwIi@jUCOe#4~6HXH#K(>&AMbjWeV2pG2 zgnpK)PqHRn#+jqfn9OJFeO|+Eq(hhf+`Nza-oCLPcrrZz9$*dzBIJUr%6@TtoO7>| zpN-253HynPz;Qc14m}^4^%K(3x&0Me5b4WD|EF#y6;}pofYh0lhk9KNMt+TM1@Ae6 z9JZmST1CioyRX)ZP$;7=@DGC-pzsyI9DB#I3iYl-&TO{aQeLLa zhk)SH1nbB8md)<^->MrO??fm8P9VtZM>YH}um7@bw8njFq|fYIQoU@ephh)a(ujX< ziDPX0#U;Ud&=w0yArI#@BmRTUf5&Y6^HIR+7gBy7j!_UR8gwG zx$ehN3eDwGbbeBoHkdv{a*9(!q9O%y=)|{Kr0YC0@#)T(7qi$8efUGTz5pnk?Gf_j zg|or3NdrCrSZn~g>y-cqVwzhQxVa6=kt2O>!Uxe;>_1-19>9Zc1D&8J_jl0M z4&B9|M#iw5aQ7X&XKFg0;etJj*A~a-gOyu`if2d=6VI8fKD3P^?a>Z^^*nUWKdL+t z`flF%_5gDy5$ztZgZpcFSM(>tBD{OkKmzSrBK?h4@tz`_6Z)Tc?-sL-{;awGJAEs( zp>r*?(vh~uu_diNek`QBFWz~~I)JjPkdUSp6j-=oOgwvJ%2vA^sI|4)UZ4fA!=gg& zH3?rS3Gfp=X9=|syWD$#Rc`r3EY)h#MtB&&tgVms*Fr#j!YNkU(?a5lT+RsFWsveL ztM7Jaxj4vN*c#qEx?HYX#9LAe->Y(TjNPKSGep6rd%4;oxOxkL(~{zs2$epf=C!{c z_X8L8Lmo(9hKVTR`f{3+mpRi@8v^7_{kl*p9awv-(X!FbAe)k7oK~0Mmh|Firaw8W z_U-pctTpAJ(!fZ$#Y`jwniX}h+}Q2C;l=?Hs_yGWnPYeTT!}6LCx>Dv17LnPSEWwo zS`L5S1pM6*rh?i_>Y1OGGhz=dKvO|KerK3!ry?5mX}L%Yp#egqm7ZSvW{;nY$GMPE zk6m$hY;IQQ6RO`cDoxNBzLnZy%>Re5h|p5tNSq@buu&WtzD=$Czj91#$7x~osB`r5 z3hozY&+<1&!Zal7yeQmn^vInzf*K);B@G6wHpYO=0p}t$_k_k4* zvpv6}I-zbTzThxua0Fq9Dn(!{uc%+$f2{@bT-xdnxAav7(;cF~DF2t3MT_gwtRf9&l5M=QgZWsn-6f3QU#jz?aEf zCClyvr|iUBxYFiIdHlV3;;qeG4mpSWsdWNqi_Vx^Fpt?!rF}D3#2(id0L7>#{GT5pmxcGP>EC_fQ#Am1XBATU%GuAM^ZI%R zR-laYY}I88kA`DJ7svo3p@FaUw}MJVIjFf!hrT>3Is@64Bxf2LHc7Nzn`J8mAgxDy z($WJv;V+w}Cs6l&OvHmXDh(iKsDB_!e~9#I{;@k8_advx0)Sf+BSpSBh|!Z0>URgMx$E}4p@rstTHvy#xLnwDAjYr4h(&BE1%T`FaO z8syyZ80AYBrR(Mulw= zC>DIn#qI@4Zf};K@~rt300xx63aXf=y()+M)k`S%?xTF=GDlRR5@?Ep-|1u-?$D)j z+}s=eR)Oj_?bPHWqS#?TeL6c-zFyA_obLNNnZl`s^fd3-4ONGv#;aWeKn4MsSGQjr zpky@%Jp^D5%a&l_V;IKra2msAM@jc6fMuKvIA|5?i{Jq@3RpYFJ&z2ZB*uA>3GlY! z+E9xMGj~`EIx^QiWO7^&Y5Ci=NQNUpNlL`>UsW*dOjLi#jSwDfn>@3%GW}b4X3!SF zGOxzRO!7(3ke}0%M^Q(q8oiGr0kNW2{W-u8ty5;Ysi=L)=o~qYkh-y)Iudrk0@UMN z*`C9k>@hG~2|We9xc|kX%wdl8$S}ArXAi3q6-UWaK+p6_xzy85=Y@|J$tfEC&vqK$ zfxADMoK3MJiV*g*&1VLYJgCNc=;2Pwab$Wq-Pc2}XHiu+68}A>f=WyiV8@kT@uW82* zBe6-5*6gHcGQ-U{eut=jpf1h(7?-N9?$zrvE0j1a9jp6!lL476(&Ot1>7>>B-U)CV zdmjgRbmrIO7XcpHjlGG+5E%{%N@n#>pc?qO==AMZx+KFgXKVlnazDeUoxaSr=lrqx z=iXznl1T%X-rvL4>9jwv(gOAX*HULx?!NvE4uUbOx=!mBrD*Np?kJ$?aCb!sC$i{N zjadPBpH;z(f!7|Ws_1=SKh+8B(6U3eVB5^ileHbb?nD&*yLT!Aii0J74_g3zdtQZ> z0dWz3Jdn4GevSq6cHrH^EbqTE98|+srA!1W6{OIR%9EF3E zRu5VrDYMrXhLOPTQBT^K1S<_nSG)@bd4k$wkcd-%MATK=mtr*Y#-Jv_MS`gj61S?WZkN>iWx*Z+rHegIcA2=wU=l zj%v20*cFuzkd-rVt1P31h1>yXyIO8(V_QVYLVrlN?fcADMPM`JUjGSfA&9|H0df zKDtjQkZdNSDJ9`D^b?KG9VUvYgtW{U@w>?k*$2 zXzxeT_q^qZ=%}(HX?ka3e`bkx@4(p0s%-Lok?#~GY&VsM{ zpTBvCB+6ZeqXZH)rIjr9};D2~V=QU^nCSIQ@dU5s1hlfs?n4fgl|K`!;CxY4P45yx{NFc3P#0)vJ7TxuVZVc3BjJw{_kl&{V&uX1UJ@}JLf+CHBLoKJALWL zCH%87LQ4X|Ewhuc-RVEb7(WkOMC2pK4=wf6>j1Bi0*thgJ5lN{jtr^DvuB|uzuH;< zz1>@Y8Hk9#jZoANBHYi9{pZBB?!PSzDC(&WoNkhF57sMxEdB9(a6}XS`Z?}uA&696 zt=-Y)`|F0|`U;%$TM^=hFaD7qKYCgE{9P_3 zXu(%*R=m>2`M#4sMCf>*H9&B;BZqqLukqPXToBEauqCU+{p;(CkF}`^@8;j*vxFcF z5>{C8e*InLht_5zYZeMs;TB?+xE>k^(y4^VpAc ze-~pFaBOnhivHf;4lI^PG%oiWssFUWuSb>^29`r3!ch3Hs}Tm2${j-M>rua~@At`z zr~>_0!r0>V{exe2>!vrD(fP55tbYju0)}EP2<{rI1Rwl$HNyL=&`Z$lvb_>Puv}VZ0S_9Gx{Fs{s65dOoKaxz=;>+^3s~*es zswofcV2bHs@yf8OcqnSvX@-0wSH0-LR9=okHpvNn<%_%o-xug#@f0a0-YzE$(@FpYH#dep7G)q?y*? zkTH?>VD6}CY&Ge2VR3q{N9AV#W8R0~UOHSR)p@8yu!$&Py%9XoloCyv@qHQ9CO|Z`+h#|a}vXM z>Cn?LE$=knDXhMDhJj-$F*Wq5Jtavd*UMjPo<)tN!Kp~t$`{_dJ|H%Y6Gc)XQy;S8 z`o8km%Hsb`Q2OxtI+c;H$xk!T@=Xrq1hQR(%xnuMRnab@_O2x-5DHR(oMoQ+{R?zL zc#bxXA?GWB?p{Tv{%&9@P=4~gxuBWDn$bd|#JSKi_v{s2lF56lb*{;c%mDhE`W^Lg zs@?lLjvOR*vG1_@j25ak1OD(i$W zncL&_+@>l^a9~lFh4=g>bgB3ZD8o?5`5)>_rq#ukTUh4<+=CU7JnhwujHVkWp*-s)0BOWvs2-} zA18Rs=2jL72($zzi(E(X<1}0pBBj3k+ofoS+Xqojw@rTjO3BPTCKczfWKx)^NB|@v znEuc2HGQEO0b=cRfD>;POtjto;wZOibsE&O4w#wbbvliAeg$=ow0YTk;?r-E@ajgP z&A5=-u+Le^I6kG`IP&GhGe;V)W`^G--Hfa%5_I}D-XS3}>c86K-W9+-2wWU}F!R?t z^<^Hok1aiU|8A4pv4Bk3V7dSQ8rS?hI8v<>8OE>k*u;K!JZK+$t0R7}+Rm(1>yZwe z@xkD1r?*#}wK~2_q2LSpUa;b^g#B#sF34<=-4Sf;-Qh;Oh6JZmG*C zVl$NA6eK*ya)C}#GMm{itwm9MROyXm+EaLx@@niLnfW9^3+~peIyVf$d!!mPIESS0&GU_y3dmuMlca$V>Bx>Ud zZDxLV)Oxx#B4EJx>KzJ2mZUr-vD4587&8z5yhRWOE4D_S8m=~qy}+z4Qp%|hYW1dl zI4P56KlA!?ZDTm71qV&vtyZ?R1MaM@cDE1gw}{cln>eV+C*)++>t5-Id(}0yPg}!2 z+(^p%a_Y<_hF+77GGxeT8C21@TKGO|<@EV(A!gdp%ii~GHj5@F-f@^bT@Jl?iOVDK zYq8Sx>({4i_b_3T5B6GBR2W;Q%MH`&(K2+`uGJ6wVegPW2@AX9@^b7q(_O@raDR8v z=B|rrV9i&z^1e4mEZgE3Qx}*VPZm{s6x$6$F_JgBY|ik#Lu|W^(Z282(p@O2c%I0c z>roQD^v%pjl*eKpJ6IVw#$sjrsm(KxS^0t5!=C`B| z?L%C*^@UrRaxDSDY&@=?#aqK~ITx#;@)pC3xBZP)(G4H&TJ18=ud46e+#0Wf_*-s! zhNDl#aHz>L=;a6Y)vmbpL_YCuLwWAal#hFH8+IE-#4hv|BTR3XHcLwvEzEH$POjS9 zBO(2Cp#7rvS&|CtnH;^Ry%~9{TjR!GLUSV0HQ|h5y^gW661*hc}jw zJVRR*Dm;I8q@L5=u|dsoWqq>4#Ui%yFC;#0jH-Ary3_*7#8Wt*!I&-(;=}z0b%HJr zg`9$dTR2iSMDSLk*Jj<*?o5vUO}F&Arj{e7^5{GH*2s}CHa*=v_lBme>6wIsFah0L zWv0Ws3gD0=Kb=BQjcEA6Ye|ISn94aV>7fU3rYBeAgx6A}dsSwE=e^Uyy1*&{#8_FOQHdt*nKwJPH(ac4??bx^K3PRVKEk zJQ9BSB7kqWHE^CR9po<)764&B+JoyzY7f!snV;X45adNHtM(OJr*|FUgk2fleFB?Z zZGML0qVD^&M%lZ&(LyKuPPHnNj!{ntA>$)1`T5upA1(YHrr>>#Qhm<)zDlEG9<%x# zqdfd@1cBB3;ABsx)yqu%iCYett(z<`quo-%%@Vhn>!Yg}yJ_4(C@ z+^(46mYHs}#q<=fl9i5|$AKI7@RY%0xpmxUC~~ulj?nha45=0NY=zXE{bi)b4tkKr zXymQk>eNC)-GW89t+|frzhv2s>hW@R=c`)?O#*hqEXLCD;eAE9X!suf#pLoz^-(3=rlejl|nkTE&lE~q{A^sv+;Vob5FNVz~k zjnCOtRPjz*?k&xpyKU8pYAUcBXRxO9GO>cpJC#C32*;BxmJ16E{AL@mQ?z83`(JE|~Ld z-s>fG7#m6sV^U*9EH3od@%e+h*m<52w5%;MWR!KciPMIt&~XhD(Ga zhK6`_N{w1SlYh2(FOLwgZxV4hBjZtZZILGkD;xaAqI26}WTRq_xS;Cnv`q{__3ns> zM&X2o)?nU-O(a`~FK>X%oxPa(u>3kuL)xorjhd_5;c17 zyHtLg5*n*#AC0i0q`l)XAdv06C}=O1x+h}+nr-k` zsHWzvV>np0ON}P8qVGGEEv{wjYSEfkWQC8J_Owb9k*}Uxl2U< zski7hoM(@he@wAFSf}rkW(%)$&ew`4_pK^TjIMQqoTPzwiB&L}jp1(;WH8UPB9Y04 zsNu_7X#UU)P7A4AKV#9CHgyN6V6qCkUJ9}_Ap2w)(+@9?nG(a&B-0RhU+QKBP>;Hk zA8FlyLD9ICO(fmZO1OMoO zqz{%`C@vCss6Eih8ik;-CS%CEBD@XHHMSAYt^CL30WT`mVl>Ly-w{gOJKg8ZR9W0w z4Rh;SFxKwMkinLb1@i3i=V>18J7h9z6pv|G)qx5lmUTAOlWrQdOBRV%hr%hTI#@0i zf&(M%Izobzh(d~AP3eEUlL&Z{UvzcEsu~qt+C%pQACbp1Hz6KA*1b+%P@%oh54(Y_ z=GT#pm#m72*s_}|@z}P_MbzQ~vB_ zQdIA@tZTLyja(rztrh81u0hkeR;mmM4hZhTGD|Wz6RUNd(irpB7Pgx z`CXj85zi0b2;^|dp9-kZ(oe7uF3Svh=H!nZh83K_C%*79l1e&25c^EN#lH{q&@`2* zW9SunOU~rQg&Zv(&aWBSRWjj|Wu=Yd*y5O98<9#$PIS65unSYkGiYJ%b2~s#F7n-q z8z>^-q{q+J#W_GLm&J=hrhde$n;j1|nYlq1Fza+WHmE$_2{Gn)`y; zn(*T8{A0`RP)xf^OH5uo#i|-@rc@uA{uac$+h{@bqhRXk5OSNT`EQ%=HqJ8t1N7F~{O_R~F^kYd!Y_ zDU(?q!(t-6G>xP4TTmsZ-JR|d!}sZFR#3Cd7Tks`K^RNg%eT4M!&6NO``qLQp@z+) zeJRKQHbicyj_iV*a;=Ail|jupkPGDeJ^oG`kEc7oQw^HBjGZ!Iq49WFd*me%rj|7M zgrnC_9bv6&Ku-jNd3+6z@p_s_?B}OC_0N0kw{*@}>3Hs*4%73} zpejTaP0>pgsO`2-Tw_-(425YGW43__wpr`=-lH zhid)*jVvRSC~GuzNhRwTOSUoO=2}C9v5w5xvJI0YWUCO83XM@@NHZ;VgHn;ner;nn zT8s<|Ls`G4<=)?`xqkDaFo* zQ)gI}io)JVM-#%$=HFBOm%r@1Tz-}gU}R@5_)|rT7)G)rM|FWRDte^fRp}W?oTXED zAq*%@y*Ol`Oh=c4=%bH#1@)uF}l6d0$LehPL_ndiQlz2j!^3n=oObl@L(+ zJf!aaAI)p9w-a)9jplxTOk&$;KhxcNv)Sf*==BT;i*&vRMH;B8I+egU+mcg&fFk`L zqnq>etchXf1Yb(|M%&Wz8gN*B7i#oYaQP|cTYig@w ziNe}o+04QHZ5yvgZ|DB$w2>mF<-*Os@0{hTTZz|=Zu#cwLSvZt@vos21+PV;@M^Ta zHeYFXww0V}_zcTHHEdWrc;dZDv2F2vH1b4*V3gt+H~Rg?iL#H(vB$TQ#bIZT<~)@z zJ=@|n-MT;1(kEwmCUz>r4F0vsnK*J~eT*2-7xiSQMsGFFpZ>5^+?SdSvnEr>39Wct z*%MGCK~GA_0`u);qI$(nDI9dRc{hc*Iav_C-%-DZn((gN$3=e5Oj>q@1Xb|8dgV&m zErsm2SKPNV=kq%)$Z$rXmiDXUsmCX+r zT#K}lt}i;BH5NAHA9~^qtjEq-scx-ikulKUj;!5$0g_bvv;p8>T*H0eb9OyZ5R(lU z)lprDt^Cl(ZT>!0f8)l^R6T#gTGa=p)Fr8?^NwrY8Vn#xH^uVK4C-OfZhdfDL&Q3! z<@Lovr=oVhiyvFXBa##HM{+~BPG3Fh-Wbk@f#3jZfA-~9y7g*P_TP^NbeHl;7NOSr z&>Rv(4}nXB@QOr6*qo~Oysa>QU*&=Eo47a4-omeejff}S-c~|6v8EQL_1%gV`9Cx3 zrn^1>Jq}jP?DC&y{0M-h%&PQ4-)a`S&K+9q8n<#9_UT8>NRUp%c1}{3Eyxee>QaFH zIxAm77T7!<@gr6b-e%5s-%vU`9zAMJMoy4Z|F?)SM(I9i|MAPZZX)oAy; z+-Z_;gm7z_czqdb!+N~*Kr}bqV7u#rCPw}*{FKA12Of>rXm$yNNYz)@%_pOdJy$}G z15WHbJSj{SOT%cFR3sz}W{v>44pltRvCQN`{oj_-QTKDiTU5W8(N^wCjEuaji0+*v zDSTZV=~H}u9F)@j43E^fFjwO0UsDtS?BXQr-vbdPFiDTW!nUW$*!YfuV@Jq`PihzTM+lwH^GlnB zeDi?~cLymX-9GH&1UjeD(KR)TUrQ&G20bhZ)nS${0ba(V)eFoxwIf=9f@65f0(;w^ z7%U#@z82I!t6NN(zYHK(%VsWLH+>unB)Qz*!R3Stl7cQ!L!{Y5`1S*WXAzk`y-4WG zTvUH_PG~DkiqA@4+(C_|GZxrg?SdD~H_NhKdiUs^_Bv--#OLp)R7@h;A7)&8r2G!; zWjjzg7oBzX8{8PL=h~aHE8n0KUj8sM)Bz z9*S)U#_BIsF?8wJT^O+&lUqCP;o)1F+h zezwm;a8C@fI{0J^@Xn(G&Y($PB%`rRx7)Sh3nz0DrNK+yMfxdRi&;~p zeB}tZn2%-h^uL{Gf2Q}r+3*IAQ}>}gv!fC4>-%&?$`-7GFuV*(-dtS0^LzbIZzWxl#z+s&<)}#7VgBs$G8+$ z&M~xaxI6%o$n!lJQr*+oyK)}+SV>>RFVY>gZt24X-zSCNp*jsth-f#((GU5AM*4fB6${_J8_7yl;p!G z4{j4Ib|YC6Hfa_UttT02d=Ci-|F@~288+cJ*xh-qKsHHq31~t4Xr%d0oI!fMrhKmo z4=YQXo>$K|HBe4-JwYK^5Y%(iI`@A(_?1M@)621%abyY#2_6@^$VMe(T-NFd{L&Gd zhj)mSFWI3vo-Gy6-@m?^;S%vw%z|UZk0m}`8dkG+T03tJVSHOMpvvWAlxpm3llVHM zw*B$QHC7g*l$RwW{z>Bni+yatV<3Ont&vtFMZ1c7{_3|rk(%%4_D(RGjJK-^xr0MIT_v$Dm?31VDS2huPMK0MWUl`M{^quUNbXnh)tY$ z-$~b$l+kj((N8{>P!*lfz;!?0EUBT0f5!TvqOm}sW2KKGs>%FTaktl7piIyn3Lff) zLk=E94C*XKW2c7o-c$Chh6oC-Bz>0rT{5=;U_A-JPecqIQP?=3(#^$++oydA0W`C7 z+)yEK`rk0-6^}VqNF`S6>9JDAk?)4pQftG5O%=Nx)%IMgIJ8e|LSSmFlGwVrChxHJ z%GzOJI^Y@V4M9j07uTtrx$D%c7ez%EtpT)qv0eGIVcSXW>XK92sW}>!M}6xxcgaC& zH3@w7|91e#4+R{Uz5^3oKzJ!S#R_uBGKFt10emo;<;R+kS#<@jI|VWduwp*8dHlGH z5fp|%_@DALePFAX{?zknw*bZNU2NjaCZ0bpI<#@z-5&8mb0xqGv^ZKcK z*zbcp;5Z~%rU0gbZH_i=4F`%#jLzU)!Kzr~X}O=*ah~Pal1(z4Yfv~2eh1>v`cIE^ z_BVo~K=vXGA45j>W#zR1y!OAXy??(k07QsZ=1T$B`JZ941ixm@{QfF?IL1+UWm$J8 zMSC*g_W}Mf0WWsYj&XVptFHaxSIp;qsmBmRBqjjo0=prRD0Jld$*+@-zL0@ z?>st&tOA~qun!iS)#zAfuh;w#-n1+(9o?(uhl-x&O?fce`DS)32YZQ3jIyOqeo_NQ zf#$@MVUlVYH)$ZSu5f1Dm%0$&ttaos=?67P;WAfeOnLb6$a;#PTm=8&-oF^l?<{zU z9A|RMO6(bQNcQ(Qzhrr5dtk(tLwb6FggI+ASioVMZw{!L@4(Z+Q0T7ZE#a$hO+Ffb zr)P|Xn3#Kj-Kr=>6>&8+b=TzT?a<&=!+Aby08?)ShQ%IAEZnDe@l2sP!Ne3H$`iHq8asgewTc`FUY9DW zpm0f%Y(Hgd8-)EIh!KxlGdp>OBqikoogpDKBv+-DOO8|09-Wz+)%-$clODpmv)wvnE_ub8gWGlxQ`|+m7ii^4B)0)o zo2y40T6^B8>th2#q}=*wEXN0Z^d2Hl)l!&T8hJ$b&7cG;Rr_XHQhn6j=`YSLN?S@6 z!t{ky#y8iptG&aG<+0O`9j2Is!>&-tvIasM4XZzLO1=x`=!ZbtQRD;r1ER88JA_|X zJMs^{J2>;jC2s}eG`&$-Bw*9#)geScQlN1tr z)pCN_pz*sa{L4PxQ%W=1_&}xV#Nb0pypp6*xX;IF_)4v(*|Rfhiy>0QJm=@;$cX-G zLqjK$Jy$w!p>>XJwvHx7Y{6HF5(tjZFzT)Njs zfE>vh^2&-iht#m%QVhVpptin?Ndc1Zw!z_{?H~Ot6-On7YAmu3%anN&LJm5`uG4J`#0m1Y*vdm{2lCrQoR8eo$`6qX`_o>z z2zI`2=+ljje0%dPl^VpB>+bp1$ra(pJ@j>*#&b)fZ$^gfaVjOSoDfy5)gkZQjL0jb z)_vpcbY$YE1}|!zV`;-lr%XFHb=o!75JmQ0u8n-3#Bk$Ak6~-T^1Y=*lG7$F8_DBM zBBPue?%5nE6Pw$u9vJ5K#h32c2K-D;SQwEJZkPWD<8lI3 literal 0 HcmV?d00001 diff --git a/docs/docs-beta/static/images/dagster-cloud/logs-error-messages/masked-err-message.png b/docs/docs-beta/static/images/dagster-cloud/logs-error-messages/masked-err-message.png new file mode 100644 index 0000000000000000000000000000000000000000..d5e27296cc0bb2dce5f2b423603a182fdbe7d282 GIT binary patch literal 52841 zcmb5VWmp|uvnGsd@C^jF;1=B7ogf?c;O-FIf=h6BcejnZySuwfa2TF5^IkLO%zLhH zz8}4Nwd`KiWvlA0>QF^_2_ytu1TZi#Bq>QzB``3Usn2mT9L(ph7j6YR7}ytna}g0m zDG?E3MF(3Gb1P#oFv-vaHCT1!VXRE8*cd@!cpd2@xfppcYLa@enQ(owZ{WhAOsFbn z#y`3n!lA8>N<$1``}|2=u=;;{)nsRvheLyID4?Cz)jC`|Y%ks}1x()^57t_Y!4w6C zF~e^ak-%=mf?^-5i3g76SYb-=hI|#l#N5?VV{H^Yz zGnGP5u?rF`G>9`muWK~;5f#jhT!D@n%9m=Fo&r0}_$zxhy&t@Ao8s>}K(o?s7JysU z8YNREl070AjfHmQH$<=%%psOAHx0#r1=yOp5Jmua_Jr?}fPfDv(T-Dybx!nG!xVm& zUD%BBPIHql<_;|yBI-=oEhSKhh<)yBB_F0-?@jxMQm{tqy@iSN@X<-*I1A^%i#|yl zU_i54JQ@LH4OSmfpl6Ff9~HS4Zf5?KM5G(El{!l{Vhu|c9zJ{xT3=3X+YuFc)*tFG z`+{H8;JC%qhol14iFShg{bQgCCQXL>+>Gh!8I*fWsGn}NLwPId79h=S&}B+KxV5@| zoEX*bob>@hs8O}B;N$X03Wd&8V$x7c?r=2H$D;`E{z&c`KWPc$Gol7TQ{UC(%1W8F z&2+MM>f3cJ9axc2YKEc!OelZTo&+>pHQ&5$Ygf}kpwXHuN+^EkwyPrcKoULat@FoYO)eSiwW29Q$??hFUn3DMkL%i0@Q<&rA2aR%d| zl?}n_!Q_JBK|#XA^dy2)_~R2>omN~1sfA@Kc0r@ZKtpx?82%C^`R5^IiwPVMAV7ld z>z`vUbfXJS<^Q@!0PgEqT7hrs*)4qfiQj0)u%^U zm}eVp8zv{t&colXa<7O^!cV@KsI**cAiANeBBv0RFz`jlxj5o<=XV#HYp}h#8(72{ zxJDzht%mAX0rL-Q7=bmU0;1)Vh}iofzn`mF5;JG!gd#Mxnjh*!VwE@|_a-R!ulQMS zBKJ>M_)m3|ji@{!8NKob*wsereERxw>W~Xz1WDUD^Y@}1tuT50qqoCB+&w;hcx`pH z(pa2}oDZBnKkhIQ3WqkPeffUys*;Y4F#N3E!WQb<3UQO~#i^XAyZ{^OU<_MaOyhiV zg`Bp2n;3tA1nVsH^YhEwYzd(N`|Hs?_%Y-4H)R)8Zsj+k*ELB8CG;BZm3rukxMHBnxS=1xglXwi{m&Vx)_?8n+xatqWL@ z78Q+DtdQ(BhWRISE#b1i3fapavLWg{h?%H0aY9nQA8B%fCwk4G3-Qw_;fzPPC%kFKcL(TXwwghud*iDUXJpd>t57y;@h6k0Na-AL7~Q z5pc**L=eH>Q)9%EWvNKwF&Z(DF($f9OTw?BB)^@YTZiagX~>cvpcbKFV{o88pevxS zhDL|JeM2Ai-4nba>yK&97gmBdCotzI33W=C6MN3iEj29NDMc@hEhJS+DO0MfmDvv* zQB5$V$r{G}Nn4A;^?Qs`ft-=tmW(5oF%C48MV$)33nL=`6DyJ;q9IdKke;)a*Y-nI zl}+s)7^#d|Isw!vYgM>!I}w?Ww$QZrYhl$;W1-YQZoxArU%C$T5GhcqDDj(PwIFeZ zZ%F->`t1AcCtk!g&B@AdjaA_flmJP4l*j{{luS#a)fj<{Y68XH8trNU=HZ@AX$%Smo%5GsKh9ceC-4v10iFVJEJwZvVlQ@L&3G=0(?EKHR~?o zUi}wDyV$RWUuyRorzo>1bJ#~oCztb`IUmEm2B?NHmep(78(SSDHfFY8O*xnC^0sk$ zHw;#KJ%g`*Cm{2DKZ`=M_-PBUWgRyw`%(6zvnJF&xn{{Nz7egmbFb+RXm4Z_C zm+W!jd!g{GYN1@=Bo6l%EVAgD#OaeqTXm_f5o~QWEj2Tv1Gccwxs7GNWe=Fh>VB75{&WN0|gtY3U_0>_+ZF^ z&Y|4i^C+v6EI8?3j7Weh6aFWI$tZ)WOx!1+9Q?v}v`g zKtI-n*G*d3n%f;*?Xa6|ojUi09=`6^bmcAYmK7xx_0=uZ4UW#xu9wY}1(b|O7&Dk; zK2W&9x=-HlAG}Qu&K6A_`cCR~M0K=ufILAGD-nQ*HTIjDtQMk0+i<-ifnI@ipE@5? zUkxyhY?`ef*3b{<KVt#tTdN~?S$QaKHgj*M==EtmK@xPxrvR1n1!l@%Z6wZJyDJDOQX+o zA`oCQS?m{kOarHLkDkAx_J(TUI0`IKxG0@Xa~f(}X^XARx+OgxUmv0$!DFDikhjTv zk=9Oi;}~X_N}WqD&*Wuq;ePUxwi^kHpO_{SpZw}YJ^x)bc_l_%T1I{~b3uUO=hz98 zjuGELVnjRFgvnaGJ@x=z)Fhsdr~fn5MPR^j;9dNk&aqxI(;AJ94-91HFigyYvGpO#h7HD$+6ZU74=>x)Y&wY;?*EUz0!^Yvj-o+5Xkj(IE zT2*XTrjKYo`3Po%ENcpy ztS4+$fLC%l@6#4woI01fLixOiA@#YOv;MKk{E>B|{@~bT_E6S`Gl0!#RQ9LIz}SHX zR^7qmNh$%m$07bm`Wjc~q;n?7@M6`S`WE*PZ!(MT)!Pi>3EEzrWbfiA*=FJ<~YncsyX-XPj%ykpxS(*&^&H zb{#fID^XFd)vvDFdUAdOU&a^@O0v}y(IC^AZDF^t?&ThJ9$oO#$ZCHV7gOxtkj@(s zOSt5?;?Q3I+FaujcU@R3m&K0AvAKfYSWr3ATIxw_Z#lhCv>Mx#@e=u@b??%9ZoJxg zfqH(lhST|JNoB3SlBO#Q*vum93r!!E*46$(U78m=m#^2Vx!NxCS1Yt$KE2<)aiEA2 zn(+_us(TEbDDE3PZ_jY&+gRx6dAWF*idcxe{0Si&kL-vHj^-v8x_*O}$OyYEVlh0dV?O++`x_s);i+M>Kc}}*fppDf0>qPUap>C0Dk#RkwW7?bL z_@E-Hh9INqv8mPeaNXOv=xE{)RCkGXk+8AYtoPn^Bs5Rt;&XO>cgFkv{n6kwFHpjj z=$5e4P3dY+X7ROXu4sQ!HcQnNT6a@$J}Tk^{XW0K z(OPwyus7dz+Hd=1YBBg6^H1}`-QzU!0#Oi>v;yY~&pIKuFQl0&q{G=c0YtJ*M{ibDa+ zV*A5GoADKN45q)y-u&kOd}0J1QUDQigrvyPl*8Vv7I#T&EFSpcJ8rMXa>LTq$<&{D z8{Fo@51CTR^Z4}2D3#)AIeJva^_fs`8mmj0$jO1xe2(G3z=O@fpgu?7pBwJy1_lO^ z4e_rtFjLu({~CiO|8vo&y%hosOb|>;R7k}Q{7eVh6R?Ou2xrs1UZ9xY3{q}0vuCV_ z4y%`ZkqF0jb#f( zNDvhj>Ob!waQwcnz*-VF(T>AI_=AyMo%Jhs3;lDOpMH>U8wXU`EO`bdls`1Y4L^WR zKF&$$|9S+JA8N4R5h{lvPW69G_(|PA^ZtMAwqPL)`kd2=j)q8th+|M=PQSEB_s-e)>_qsAk=-P|^?3|4R~nTY+7Ju9rt|qld2#%$&|z&$tn&ilTIJHJ6+1 z@n01G(soZwxUQ44^SHq~G%O)f z|1NA%FsO@Q4qGZKR@)l!7ym<{RIGomKK_sp_}_#@|3%e5=I1N41v$v+@p9xBV+syg zHFp95LF~Xp$;2V{pT+-w9@zzcGA}CTkJJB_svRP@dS<)FYlw<0O_N1zkH4=eT35P? zW87i!fBR~15aOSzYZ4fi4-Y{C-gV9;iT#ndz z(?%5B<&M)kn58I3!SA-?E0M^fL;vUj9RELZQdwh%$(qG9^gwWpN-q1y5B@ivW{DJa zfW158jqe3{HcSkHo;}Y@t8=$K7Qd=Cm?YeP-2}rrX5N9NlKF+%K_^s}cVP=3HbUd?P2H^MP97*P< z#~ezEL+^_=Qd6u}-N#(nbJg7~xCm=Z5Y@;W^>F+Cf7&=#ND(z+xjckr^ z&`hn{*+buOozq7*S@7F(qY+WXYI6p6qow?hJaHx~*Tai|daM0kCo?UWVv*>}-xchn z*Xm3KR4bK|E;fC#ThI6R#U!FL6@AKpRYfAOjjdh;9Y(I|1l-Pfhm(5WnHCdH<|>6Y zHm1+?e?9L}X9)OAGpxsnX6o$6{=;QIe=u#14xeoDft!4C$;Vv=`~wZbvrx@ZaREE zHYS-fIXT_>L{~tY!MS);x3(K?BEiT+999QN7-DwLY9*dhy}J_>xlz-*rc*n{mcyg8 ztmHDO$-*r5^_G()slB4}T^&A3BLJz2Ql8$Qt2MhF%_k_H`O$X=<5~#4Bbj#FJ;x~S zKVSO8ljdx%)LR`72{CD7Co3mrO&Jw0-^q?w?zI54CQf14jK~-<9*^mjISBjGifMXy z%YO`$sCpuOfO{gT7_;VgTN>k$c-=qh0u5#&<}>*YX}pjj z$#LXi*g3|dYCqmRTaboX_R2IHNHM9Ez|!tE{}`!M{DN$t|M*<11+OHw-egE7{XaCm zXpnWKBVo+*{6x|er~Q{V`mcuRO}fSI$L_LYmD*`WPA(tsi-0{Or{#KfUnita!}d~u z(QY72s+D->6eI;e1%T5NE9=YcNuJEV`(2sR55U0oPpEN_SrRX%TEJiCB);RgZ*EUg-yF*Wi{ zwMSLXUBvM0#`=f;m)aSS>t%`B2O5;j_n4A-Zz{{-%L|zDYCz zW~Re84Jz%@8aGi`^Q9VNDP75Fz(|;_36aM4fCij=iHC`JE=TbOq#di3IVJ#C?BT(5 z5A^I&PmoMJv$4bc?`fav=UKXsi*=A~yx7dApC#rb?Rt25VAO*d%#_Iz`3{j>XOLUn z-~^`L7p$*teDv@Akzlp>7K`%k&i4*Ron#YPT;W+tBl;ElMq%vx(SA5Vx$)Nf7Z@gV6) zUXkB>!|2Kd2n9#ed6Z5Uuf}D}Q7(cPf+N*0A_}7y?ZM{btrf`EtrI}gu6)bG1(kY5 z-SGXdSLW08$Xdki)9)f58qSCpe2jYS@+}TOfqOW}qUyi#%B3e~gV zbjl!Y&K|`wTYgjkFhAEb;;62d#g4>%p|-RcKA|pT@Dz|sjHLg+_u#0xXm+a*^cu0E zh2UeE^NTnAPD0p-U+}qg(O}0@){a^9<4ZIbtgpyRI~Y~_RJiiSK2GPtY^qlU$w(RO zb015GLX}OA(^#n^!n9^|JH6#r8!wG}$YA!zY>3jvVyAwVx4|LuUhbQxp!mc`Z+R<)oMwd+hLD0gU8j+@a9SWMb5%v zy_9QzEM;6Nsj6uA-T;;o?9sg0Ch=MlaEo2Cy%?|sb=6Fn~j;|0F7Y=l+X#YD>_^(l>ukR|9 zrhad;*P%DJ^8v4FiON7uP<)31Ezmmu4K`YFH8GM}cWvoe4~M2i7Qp7wVE!A-|mi;v4FHC?y`3ID_L)Qel#Ld4k^Y6$mw;}3wkEsPf^_`D#;ql$1pK zTC=-EzXc)@e;c{`buC-i+G~kDeZp_vOwr}V+vqGo=9JLfoS?Z_2xFAv-_qbX#r`CXG#}X;g}Pqj(xv4 z%x6Ra{Mww)Lm$=FK;$QUvR6!o7_An9yE0yF0zzKT2qot3ZJJ?Y6@iev95PDf()i(G z`SB5PjdbZWPUp-X+zV8;?w02MpC%9)sw>wRregix9fZa;HL9a!LYb1>$$>B7aVV$EA$hiNx(se<8CUyYJX-HKmOn<>_ z(ofF*ucWjK9;&d(Hm}1J?+f99sLf)n)mMSUiHmi)hF=ePN7oq5z))D}jHT9;4ae6L z*?yT{+3Y2x`NUU83-qbCY7){Jv?vKy?~alubHbZBOfQXC7tA{3OyVdBK@XT-c!Vdynx* z0vXGo%1z{~aXgASvF5?#O=JZYoAghKb6o;?W;<_SMB&sBKnwL0A6?vE&^uQWad(EuNH`}xAL zIgsazFSwuZjspI`d}eoD=1|#+9@l&DVxefg1<*|L8~|CCj$Ko_Z(~Jb(A`d7XRL*Y z6;U~x4RHu`+ebDV1t~-iJfcE~ugKAw9robd2U%Q1E=#=ETipII5ORzD$@6J>df!@M zOI;~!z@4$JJ5ftKI%aZ+guiq>vV<; zHnq4vR6MT?a74Q_g0wkerN!k2Do(%bXrZMJKq3v0{;Oj$qMJi1b`xb2zaHyyw28^u zdHO*mUB1*NNQ`Zev+1`lbKL1{S=`@84PY#|_kT>n4>R3i96w&MXKx<*qnyy<^2#~> zwwdL$fApht?N~u(edYHuyXQ-FL{zz+M6}IjF6VIUbQ3h9a+GueQs?w$o2L}d;Y8u+ z#}jIYn(}ubw8f%xNJ81$%*fM&$Muv@SPEyT%wEcJ&C4RT#2_O+Z&&ho4k(F1ac;?e z?v+NnlO!no>v#Ls$;(ZNv?AGD;d|UE4)@5&V0}Pc{^Vx8={AJeNZ~|EIN|M+o`4A; zW9xJ~sbm(D$^G^WN0Y^&IWMQ=PKC?CUh5v|7V7U363@0ouIc4BZ`QpM5rZDLCz(v0 zc9%E``4ov1=EA%YOzD=nyTS^PN{g@+=0QX<`Gg{6NC9DcFq$^D%Uc~ASJ-rctq%)G zJrKYUBa?Nu@Rz%BX{F_bc;0w1($4fKtc%8G?h$8X1u?%sSI~1HVq5MZPPk)M;}|5s$!EK*VMsQ-_66Yu9db66?gHT1{atDB`N4{85*sIa^~ySt2zH z1VpLv#M5epoh*L*>0neA)`*0WHqHMh)`7EwLaxw~@knCaMTwU=Dd3@FWFJ;9{*GCq z9BfE&)ezI(H_~5{!NWbD=IQEHv}s#=D2vIu3=pFGO?66%mhTSd2oVf}j>b{HSI z-aSo#c~v&E^hBNah={x6t**d<3Zb7U`HVNT>^) z_EG>*%_HQ>D9X%JK=YlGZ}vAth)fA$T*kogBpSAMk6ZByi@%RT!*#dGoijU%^o$`MN#z$47sIZk?=V#aoXYLoA#=wDkD++Ux@;8eUUEv%>*A z=1RZbUZFFV8Mu===;J-PW8vCjeIjRRM8!Qhvme9=Jk&rn;{t0L;y`Lsh=rOFP`@ex4 z?tjyGzfADCEH-8BRAXwbuu>zHt<^e5rvjdCO*nWjY)GG%eWuj+TWh2OQb)YF02q|=f@w`p=uStYK{9HpNfYUyuIo|K+SRc=z^3mW zx7Pu$^C?@P;b53hduyYSaPMM8rZ?~P?jZRdo`5^@`Y!t?2r^g1gS`_eglVevK~^N*T%{hA z<_3{>_Tnr0`*XkJ3b$#L1>%x$tY@0;o5hFMR)>BYTHdp@U)8?P%X0!LDSk_jukhV? znVO~xFy?vZNoP7&w)u%aj%c%8hVm(I#I*NUtIIP$^|_*ApRfDush0H{p1&^<94jH0 zct!d>SvWrWXJRcV3^tPV7cq&-IO}iZS4lY;mQNgF3kIS(cAd!-E5UMg<5+X=mn&{5 zY6@DPP#I3${1QslbCOQIOKxsts#2zfw@2u+@rctspTF1(ajZTyjO%X zuxl@_hpLG%UfPXvE=(hiFarVl%sY*`Fq2!ui6Fk#AAIlp^D?ijn~Ee9iQBrsQ4~@` z`^(Jws0!E8*Lkt5^;)mnF&uwq>w_6%x=(arKrYiiH8MlL8SIPEK$Ci=1smP{mb+%l ztlsYh8Ix7@VMJmBceP}lzX?wb2`7+t;L~c=@;WkUD+8p2yS;deCD%jK@$7ow#?m`c0jCe-NkyFd81Q7;orCO(*GbJOi@+aX|rwjU46qT<*{5Co6l$h+jZQj)8J!PgSDRuXm=wdgnLV zKX;C9=1=3<9AR#vOxpds80rw=n}@&Pvit2ioacWgVM;y|Ds>s_9Jy9GubFlHJcAb) z+pTC2JJXU|sc1guK9&O)IN_ScelPvVJ~iQZ6m&2TA4L6z=;M;;->-i+o8Dr-$tVxJ z8Hw4gwRq@Z3K6>Q%^lx5oj=QA;#C^9paWIU*Dpmt4}`gVuUf=aZzqWKj>qVKuY2l^ zNcOQ8Oy(_F756lD2b|3xr{2#DS<{Ullb)WJ>X#Q+x?{f%;lVE}^e^xBg1;KvKRQ88cLxY-&*DHd9B%c!35q{F@ zq#it+=fN?nK+DuC$)##q^gE(MX=>{p7b-OUoU#VkPAN7Weo)ZW3XUBc?-&0Wk+wNa zUskF8Vf02ayDGJ0oS>JV`6FnOkUtyotQF4xoQf$zjkzZm`R4B3ODvmQKkE_tiTqIp zL6W(h2f>qSVo%i$c6srGgUeC*`Sq!y)%35Y24*6`e;^$G)DUi!(7&O8Jl>IFLoA(B ze~i3IQ6(2cv%KK^U^m--m53vS1L{Z@K^lGEI$7YlxM6J2uOCj9m5sUTO!nZO|6ud9 z(p2*Yj0&r3B)%6EsIpbBEwlM#hLIz1_~h_w91_pElFO6`c!8t(**U_x`bX_?Yt0Mo z6i6E0QqM9)F{uCNn~+F_ zw)TPJmft{v%bl@)gS-B+(tgI_a69N(#AUVmCU9v&B7Js>{dT&2w8lN$o*V(k-|Gz{rJuEQ?I2B8G1;*@vfieBr8*`&D-GUyG|>+|obZZ$rQ zU!dTz1pzD8h1p6I)!ytHUtJng0{}yZ$zTqR4D@=5mmSU_i!+#~X45nCbIn3XV%*EI zM#Q*2mt(XS%QWVSogYu;LW!)QRXt4S9A}x-=A;$wVsJKVZ8Fpn802BLQm)2E>xJi~ znH_vQis?%W2%<4j^Cz#ur^}IfdQhpf=ib)FjaJJ-Rl4oK-jkJr08Dq6-2?iSmeHke z9N+CY-KK(fF?5SY5=JyqM-%B;4ADlb}df1*j%GM_}) zSPK1K3OjnUzSWYgU`*(PYGePTQ^-1v_JOL8o^X`aMtS+|7CQLqFUvN@X(HE^!0-uX zfrjkq&id@K*KK2&>=xywPdsf+Yztp2l~i7Ez>u7ZL926T_oK(}?c`ae2P+r)bgoWW z=}fCNnNYX0atEOc!ErUh+$n&bF2_tX9lWorT9vq`^?odKy=}y`$iXxzkzTKJI4;^q zzmFN;pZ1Npd?-uQ^EK1Ncu3HAg?ejs(tkbVU`Bv`}D6LoV!u(hKM+ya9P{MAAJA~yjY2+C;gJ5Xjd^yfi zg|#6b;%Y;lcyU6T;)#9(d%SLRvu#ZrMP9k_C>n6`q+ivOk?2XGg{|l5w+v-@@#Nsb zToe$s7is1K`W9|C8Ze%As9~?Wto36Pv#E)xsc|x0UTx{uGnoL3(v>>m{M%WF+NN*W zV`|2&;+8hoFVRib1gq=t93K6j!mm9%5vny5<0?4X!Pv_>*ay;u=;ZMpBxNJ~=jj^_ z0myHl*mY|H6YM{sT(bRdUn#*R8<1xI}nh1%74ClJB>wbttJe?z!v|4ty4#!xg ztIE~h1%Yu;K(bP@UNZ^_du+wbO2R2Y<#G1o8moKm$!U}Z3t&KVu&x%-PJQ&K|Lndp z!TImgZM;rGuO{T~YoboaZkwRc>((=ySmB<@D89sE=mJpiy<`8uw_OIS#1uvQsT*3q zv36h8CvwGk|7myR{(`PUlW2V7u)#rZr^%DYrKLIe^0w?lm8lCgU<^|zVhYG|FeO8= z7Xih9u+5GB;LuF=SE1(y!nwJq9OK94ajW0@tf4NuL^D*yyMet+q30AK6#-x_+;`e;+U zOl&UN4}Dk8>Oq@Da18kFD8W_Nu3-6$7QyQ8VHX7BHpP!{&H$M+@WCyA(QWt?UJXM~ zttP}8kFdkxNGbW#6C%boW*K`a?|vuaf$5jR?YyFy8y^nE2BHzA@WO_-ay6f;<5^Ru zy(MbQG+MEnZ_gsgO+QTbQ^)VDS5)Hnc`P4fHFK`Y%N+ufTv<1RWj;$~=U1&xPq+=2 zb6Szkb{`D!OGGziD%bu{;awgN7qp{=)met}Rz_vq>XG{fHq0EN)?F#2lLgQDf88ag zvRTSi>2*#c!#>*v%de}P)k&ZbDF?tb&`B48toq&diZ5lv7s~&lvH)4{b9#hr8jCeRP%)T1O zf9*&-3qhp(Yz!cWs&;VL?Yovf=IUJimrwMGL_ZgdsSUnG9nZijZmr5 z3hBwD9uh^+e62Lh^c1x7C>5H+OtM(^&< z&Wv9DvW=VS)i=g-EZ#&HPd<#!6nz1BZlxD3YP#z2YnV2;E*r_EnwKUugBF zZM8WR?amEu8ZLZ=em}s~Ic_Z5pWddlZ|Jlv3M~g)LO*1D!d+#SeZlba=xtX6lKM;# zNtQ*Q`O=<7ZRuBOI}SME{$Tp`Qxrmr{jiZ@572UZ2eM&Z*#vwIsDy@~*n%UjJB#_Mn~ zese^CV&6$nz)MBurO68|B;$;98_#Ye5cp~kZbV2R5eOGcg zEM_GS5oQ2GuJ?}(j&y%MZ<>I#5Bhym$jm=-gAsWcuN~&!7$28&#>wdb z{`l%0Yq7XffJ-%&wj*0e9d z5ur7BCSsoTb%%^-I|k$-m+3M>>c6{7zo4^O0BC?q@4HxopVeUD>XWd6zW5|HGfb2( z$EjNZB=)&bQ0NmtmrI`Vfeao~tW^=@2h%YPX~d-R+^g5?cKyF*6v&bE7x=)ft*%{4 zg%YpBjl0h(!blP&Plaw%Kf@s;zgp+piex{qx4YTiNO|01-~6*VW^V6LUZueS2Lisy zWE~iWO2W(bPp<_s3;YYpyTp-N4e1j;RmE)~6_P}DV{InmK6&it+{&#^ zn^fozkJoW`={kQ%D-G7U*;F*1{%HsFXM}M3GpN-KtU5hJe@@HHgpdD}oQ#bN=E2K??Z-Xodj-ur2Z3Db0Pq{D+xX!y)}@gJH*(IX5DH82 zFL@QSMKoo8CpsSLbQjt2&<0#-@Iz6wKrfraSx_NxW9-t>dr)jGn8wGsKpS5K!>iX~ ztp(qdoDCGshHiSjJ{d>^Jn~<9>Daf$k^$x6Eii4*W&*FgS~mm&=;XR$VfJw7ZgIch zR06o%7-5bfjcEZY)sK!6b{z*UPjRORV%7&^P32L+^Q*}NNE)He3r)ZH3ZJd9aa~O& z@-(T+u3&LAI-@e!Qn~f7Ht+sgC*0I{uKoo?8O8Z`;mlWOhS3J7?xk+M;M$wV)~YJh zP4Ci;@TR% zcGnu{CUpYrfwo(yY-&>!GlQ(=L&mMsL^6q51I3row2r!Z8&%eIiy+#E)&SZh+M9C4 zb{Lw4L>B=KK0{9H_f_?Y>rVxvdV(5pxXyjchta@=<#?^uSS@Aart#ol9(G0>yW55~+S_}dv% zKySS^S@Ld8OenW;!gea7YxKrtx)-~E5CO?(!=U8v2c{#h{fWEccqXfY1RU}ixkuMK zWv1t~;9l^Tg>C|Vu)i&IS~WIk>;LNOl#DRhM?B!Qi@xps02j^G)jm$@XscUwgRxF! z^|N?i*P{3ACS_AcG5mvOm{BmUH{}qSD_>_Mfm3%`zlft3f_-{{c6$le3A?2Q9=
hAjk?GXm){@(ZDZ4w8O%+*H1e|;{C@4n^8~I zJ(JhHFyvc<(JfI3IoWc}tm`~DT|+wNI1Qc2*b(556paFW2unKTP$eJieAbV3dLJA8 zgjZ!Fy>OD4U%1=J#NOJ^?Q$%bO=C25!x{H!F2#Z|5-wJyD zNzMdo^QFwDdyeO7Rbjb3{_<96w(F9BIan-=!Ot~q>fc;*Sp*~SYhF26&}ychOGM_q zN-o}5bQZJNq%Z9`y7Al}^CW-v+R1X*u2VES1rm#Ew$EYDh<$GpAN#y2?_8~1OO$XDUAtW=k3p8@IFwWyXx9D&upO~nYH z7s|}YL+k9TCi9!1M_KHOw3dZ{;S{uRZzXGGbN@--b8$p*~Yw;0} zcP`mB-g}mKZNZb1?)A?WIVA}2H{D8WtLGRS1%WD(`Scn;KP*9=BD+%9zj@XfW!I)< zHx>ht3DE^M-ePHZ3HFeOHX>{mwBF0(89HTCDhXGbt;=_g8EvaKzFjqe0Z#nP){qeE#}ZxDu7;Xrp;qe6UuLTm!25bM1r;$e}&YCIWmx@e>!$hoeMYtB&ufqg+H$ zJx}HE*x05(4RalPFd5mouiN;N*j23(NM01gyNjih(yJ-a@W9!*_GIBc{@!}?Dukzy zLiOtWOcxH#oJno`n27>$ez4490@`+YC#W--d9zfTzeSQZm$M^KJDCdoJzpxcYBG_X z%I9GoT<)FW{(gkW>Cp}ftNP5%HxC_;~#`k96r9ob~{e#bc4oUFS0hF@9wwxz`~oYp!)#YG^UF!!vp^Q zvG|GLh+i77gj-J5e`$uL5P)be>&+lHVX8&F9S#{VOE(qW$vLY%OEd+8UHh~X%G93& zk<gfZ#+Rpq%%XI3{by|J} zkV-^JhGGo3mV8<{PXI05)k4K%xl^%fJH+-Rqj~=~;`a`iCqGOvTfI$Z3#$GwtXv}r zmuX`PuTS)_l8%D5$F;bgN4bg~$U z*QByJQy`P~Y*NTD{7-|e5FU7&=)!J-f$6e!|MPI1*=J>`-ecHsWFV{{HWJsTQP|Gt z>x)K<;kV6$QB%q6NL!=84hwQ1o?S!HWraa5%QxXPwIPI`AWTC7V!yFs=|G|!i;CA#Rb*q=?LE?X=)k_y5+pucF)v47;&yj8JmUAjlxX+dF@g<-A6)I2ndnqpX zop#Pe*hDTG!O&{t-WE1unda{LVg{5%=Ss5Vdu7*oI^L^}xszL8Iuyr*?KP4yiNu}B zCu4)?$=GMl!?Ee0a3{=#fj`X{BJZrd)oSZWr`*g__?Pb*u|T9oq3ABg_ED@5+4{eAwFOz2y>G`_W1EyPLI$=zAwIK91N1;6= zO?MwN93Yank$h5cDoda`dKdYiHasKfx?79{%E<&^NuOWAjSYlaUH0>hKv#(oK!}DFp&{w;xoVxLg%XcjoSj1_JxpyoF(g4~? zqKcJoXY9)hUW}PJ>aX^}*qzXg>FZ4rX#3&}FX)AGMb)>bE6xW`Mm!WBm9DN=^D7Pq z>*Av?fIHuiuA!p$%84m{iACx{i)I6Us(Js?-g_M!V+W0PPNZ zR_88sXtdl=+>-oE13avTZNGWbTa6l3U$JRDA%qF`_Tb5B>5sAyonO^LyI&l z4A6#RQdQ08r$}f9%DhQZXpQvd%7Sl_!51UNH_=y``R>CV!NNJ{uq4&0V+uX2A;H1hw2S@+Wp5c2 z*S2?i2MZe9-GdX{g1ZC_E{$vO;O;I78l-V|Z`|F2ySuyF?VP>ObM~ow->2%WT2&3K z4^7Wr)5d>{-<0|x_W~p0n4coFrf)$#YnrbgH{vf;DMtDMmQP{SUO@hAMOK?z=)v2W zz&Uyaz-}=7zK7URvZ?G7^>4x%%@$$N@XN4y`0g``(3$rxVJWI9ISym_1v@b%tffNE z5=>QH_~ovwSd5l~z#=I$-a!&7?nM7GHUFmgrycdl&4sdb_8)?!X#^A_kr>*`S(5QOJ03^D=(x|YIytmk)K zOi3>MN#UpRIYy9O#P0_}AmK>{75loWP%yi~CiA!$jN)A*u;@_POiBlcb7Wqm0+v31 z;n|SnED;@BM>IsLb{i5~92=+C7A7PfcqvhzX2w)TsQ?(|i5S$`K9Ov8xQS#|25cgL zN5{)|YP7Dv{azwZr5HBXgC(|RBEf{02IFm97FCXM;?I>|Ly{dx87gIn#VVb>h>Rj^ zLBu_DZZ5s5rIwX-@Sy@Arl26&U6b2zELxWYOJ|uGSgvw<7;3|+XWh*!N5Iij2ufS{ zhDXM3Vq*DA6YE}Gwf9TPzb0yn9+B_SUZ#{O9-J>MWh2v^Wv;?MUN+xrM69=v5d&8( zMF1f_OJG6W%L|l|^qPlNhA$vSMbK!|1ED_FY-*4&-@`vY!xJgc=5Zn`(kvsEjZ9&` z^N=%R^xdc9idLE3c5;oR_8H7^pEJnJ?^fs2kp8Xr;mFM?@*i#cR+b)T zzMT}2EQKniV6mUgSZir5`P~PF58V0LfWW}jCQPP~eom_%6K$1}Uyd_uLn>Yfq3)H6 z5aSbt&nq8n5gvnZNQ>F3Znz9x{NX0rwM{A%!we~9!m#CW-*$$U7>y?v;UcT7tQ(&Qj^Wr{U>SD6j=aAE zo20H}z6qNLm)m5ha@uC&*-=vk6Tw9)g@+_WkWQp)Rfyrq8Ot^>mfi28I#nsQJBn2a z>21muy}L>FPzez&p;LZ9BHG|{1P!ayJ1q3<-F$90@BS$-h6+~iW5|umL*Dagz3&E@OO8R$pzBd>C+khu?bZB?)m&8lTb>UK z30f+hSsn?YGYoVP#l$WsXRH2TY}*AEP{wbzIC1GcxW%adwyFiL3%PEa;p_WX5;9aKnA{9LCet4$Xmq{2<1%9DM{+ zZm2Dna(G0YXgdia3CM)BAmK8h^O_Y}ibiM4H(`mg&beoTK`RWSH*(oY$9BA8C)mZ) zRq4RlZ2puBn&nVwC#_0!)KTn-j2g8uwr^vf*j(4+bJNibH}DJDHPsp3I;@yQZt>>+ z=}X+4yK7ndBEiVMh760@M54LbcKQwE*RGi5h08Ek7-Q%AhT!?%5inBt2`u zPn(j$&HW6hRDHmHee_rqcc_C`*5Tv6)|Q?bT5;g#LzhYGufJ1eu#B5G5RBN2(C$(> zqOE=;!P67u*HJtu^_&{>qa{bhPs?d~jl;V^7%35KE~ZoO%kzF!RV_;A(nvJ7kg$ev zP5~<&2_EU3x6MhO3G4Mn>?!RjB22Qs&Q1&m7ZP2Z*1#n_+dZ&OGF11AeC*!IOgP3S zKJcvJkz076B5GyCZEg(yv|ucpZ|=#ggn;Xpcj_2;LsC^lK5*P8pBTSYy@IGbcRCkX z;q&nkTf5H)*!4Jw;qUPhnBlX@U71w$nwdqHn6==Jk$X!ee(sGm4bL9=;VwJk7((KT zGn1r3qS&A_oI+{MDWmN~{=$Wh^3$5x&5J`V>IWvwjOA8Ye1p6M7CvY7RSRKDVdAwW z6P)}mJkPo|UhsGHSi3`H0v586YEDx4Le60$%v*bN4NO0xv#3Yg=u}`2cR~45!0%V_ zSz>R$R zLdpgVnS#DF5_SLignblNEqL`G4oet8ktIk{X1A7HA*0%4f6I`*4WcCX7m^Zr7oOg8 zFh$SxQE^}%fdRoLollYlz>HY>y|uu&e`SEuufhE6(Zh4Ez{bucfbq+-Uv?jw%{mhMBgy6L_>b zktiNy$qwQE`EM40T9hFkuXcd|3EU^0ZWPS>TztS|7--2+m^wW z(VJ)>*!0y6Q?IoO9-C&PKY9lVT|tK{|0C+*isX0I(q-*DetLZzH%#u;i=*l1kg85r z@SFDAO}}VYJxafgUo}NzXT*1W=M(4>_HB3u;nY~i5ab08H_s=|ytFU(gD-))?{!7m zNQ!BE?!hBejSBfe^Vjv*CC`9k_zr82r`7VlogSlVrh!`D*6LZc87Xlz0h z+vW9l7f_{(!Q>`j16;WNuv&e~OjUK+65U+jRr8C&57EbfN-b@$Bb$7s?mMWi3t{bT zp6CP^gjj1l%xu`6UPU&m;?;CUnjfI2KkBp{r>*VfuK!IG{vD2=;w;%%V43gn)Fy^S z|K(hH4kym-v8B^mH-bwk^lhL%70cMp;fAQRV^)YeO9Q8Tn*O|)V$WP<&MT#eQt-Y1 z#|G;>4DE(b7WT3RzTzZ6^wLs=!78l{J@|`vHx70WtTXunAC~rRPx7Q0eaIL&V_xqu zqQl_wN~{SBx^D1k)Kfv;-(XI8qA*kIA0Q3EL&Rs9vj`sbJWG}`1Liz3YT7n}qC?@|OaI_l+Jh2yrS~aY z9BN&^^m~DSnS-7~SiYuuVXD#E&&=Dxi4Y^cAeU)9PUeqeTtHTmzx@p9;t!DFl*hMq z8r?`2@~Gc^s!CLC7>Qm z&}Me+vDoPZJYBASW!hO6VKkc9ypF{p z-=8y5yH+EK-feOVdUxy%%1z z`hF%Ir~Ote#Z%f5B%4#|xhB{zV-iae#-pd2^4X2?BGSRm?y+nt|F;&h{M*F~tlDSe zEO3!;vWX>U9!rbb1Oir>tTNA`pxxXxJ%zp1i@@1WC1`SYv?5;KE(s`z3WXoCM~Zv=LP>ux5wQ8aNjeAdCU=ZC@ZVZ@;BH= zsPs)OH*FgRSrNLcjJOo$<^&qMjr0qNHyC+1Cmn13yzIMU6d4$X8#8H>E;3TKtXfj? zz*qN$URKxD8*&Y^1(Mi)pcdS6?_D^tatE1|kWN2}EI{aBtgT)EaJgGaP6ab$4rhaSRgnS$hOh00JZc;BITi-1Mz z?jJ}8MA8vh<@O5q22`i%S79X^-nzT&=&2X?R(${d;=TxWF*)3&WJ$5Of8WDC0aT0i zF7u4fx}Q1C!7t~i8Ad#VBLSIYGbPGdj~{4g*Kknx&X!kOYr!`Dp^*qEuqS_svG9*i zfCE%GxAts<9RRix_f9$o-4bC!gj=z5w5yQ4kD@M5ySqH`DWLClkKQ1l z5Qf|ty7@hXb=Y>PJ2<&b#8qzDmwv?mrTp_DahoG@y9%2DDvtz`IBG!UtHrVy{hc#- zxvvfs#J4Gb_CNNWe0wKYz*~OqW$TJTfF#HAun0V0s|fiB=NkbB4gB$NX1sJ#l55x9ct6R4oSd zFjNi%tYQ^F>v+08!#Ps-RlRWp;CcbdiC~qmlE%t2!qk#@!|Po{Rb?gQ*&=E6HP~-e z-ZGVq;+jFMRfB(5Lc6mK<5(%b++d8>31iXf^Zv=AH}A|=aY0}#g6`UgPEL$bn>faVvmaO^17YO6RI>m|+j-b4h#Z`&q8Y=&8WuERYf z{V8iM#(zeg&QDZ4aHQ{gvjLZ0N1T2#nOV%{7(9_lK21as z;A#O2BO&eqBVu9ucfG?F4}ve6Zml8)zPp*EQo|7hbWZ!mPJr$zurX({*g(`ACyta9 znoOfgJ3Zkk2Y*%t`)j4VueP8%OvzbJ6}f%yx(^(E2N^fc7l|k(H6PO7#H*k|n-4i9 zMt(ue@uFsT$7Mr<_El-n}jt+2BRIqNyBYEA#owTv-}x%Vz7>1ZI9L zL{^3D9zkrQmqI;y0%Xu$w0zDH9hzoFcxr2HZJh*zWG&#ZsXJ0|A}E)!zy?cy8{LHF zz9y*mnzUmEM2aN}KKQEb$fF0#KfF`f0GfoEKILgHsNb!3nQ~`50T6h#zrrNpc3oxK z?#TP1B!FZq=|@B?@2&2Lgd(1vN3qMcl!!X(J{lq|(q)f=1p)z|uOD^pQ79?k*foMz zw}MQyka%&5tLgpCa6@Znp23I~M`et>G_z93)X$<%eo+;irf=TNL;_+~k}N1eCv9zitBrNaO5!%38wBf}plJFBUC29q4WNO|l7I)fSeQtH8yZv#&dAxT zTh+VLTRNBLDT=j=azpiO3rCYMnBh09j28NJt&V2ruw6j{H7nC&u&arWamZKWxk??N zHQUZ?4P=FvrYqxRHXha(aaSth#SMH5aN;iQ+fbhWHQ(21a-4PFEw^{_87>3j;19ls z{Cs=Ch@;qCsinPZVlWn*5MH+uS?^1?vgC%}qP}(1sBjan{0M^(^VFin?3o)X^RPGA z1eMLC;pHdhAaSW$CZWWm_Ir-9(z&^RZ-D zgd}xE*2wq7fp-fZg2DiPGeqfPp!27Wxj3JaINgJ!9L{vUv~RC%jcP=_)c>`~eimCjxio|BXJbc2Wka$(Z@iW_i^ z&_7!S#0L97N>2DZR_h+^M+Nmo;JCZdx#hU~?`*&~W`l1d+90&ePOr4^*xJiK`;1J- z>y>=T4y+O$rzdoMkHY@YZ+~k%eXRI?CTOk7bgjvY3JAaT%ukQ_ zc8M&1zz9!p+_IUije7*m-7nYyW*_b}r9Zcw*vdP#4GKchGSP>K`@e_0-$%w-noC#u zy{|NI5Y&Pal;+4tfdvhT2!RUwHk_)`cppH|Lm@1e&5yW;5x?{LV@6_!>I-;%+(ryt z8JeYDWcckwscVdCWsXe-FYIseiXIMhTF{X&`%g8?a|JGNS2eyRn?6^DL2qH%?X@eG z_j#S)W<>XIEQVhNB*YG9oGxbf3674`PO*-7qj5K#BS__QUlk_`13(5ZA_?{KxP*I) z6;8N=A9^;5sCTx}V4+oy;EwL+BsgmDhnxf;0t4E-oC?0arv(4@uRnV7K#PQY=q^W? z7f%c&MIAxySjPHWCAoJ;Eu3j4DGO!s^_(ruh8j-P`haAE?WkTPk*8f>EtB*o&;sWM z_t{w4i==VCcqbuV=CIY=V^-mJjk!@1gb zEV)oZYyIXXm_%fQ(?11U*8XY&FSrhCBY#`4KNczu4Gc8$?MFqu95GUq5&!GH6I|fy zp#&uqIOFzj5*%}gP@&>BmQtx}+cJD?qOP#es3x$P{O@-Df4rE3IcE7}WbOaCwT=ee zLN0MD#jhMjimCxOkoJKgH}9{E(7)g2KeGEhc)Ha(=TV6#fq#^3D;{QT3njxBvMM z{?h-o!KAqwak1O~6a+db-~yv|zhsFFmjp|8#LIk+ez!}#`@a+2Le+-hu%!P1-fKYf z)o%H;IuRu1NFZI*F$L%QS;bLBhszybtLXm&S$J=WiVAsB!jr&?`adnQPln3-jT?c0 zKOf|Xrf@c$oFcMble;V_@shA!9 zz8aaDnXw=K{Q<$f%^doxPHo{Trhxos1dP#C>8Q}5RtiJqFgQq{&^t$X8+?|APyN$6 zTRb1t%z#2%#bD~%F)2sqc5=X*&`V1Lj~=D%i6@B?Q2VX z5h173%eM3-I4b%_k2Ro>$lU%_qugiMOKX}fhCJ-~eC~(L0FVH0E^{OsY?acC2lto! zgJ>o5hA_i3Ai&@mlcQSN?=RD|;kX=hdGs2g4r<$vFB1J+xMRZUluJ}EX#P!B>xzTPMPv-Ut)$&&g1EkL-+Id(4VG5$Dd+;L7|ib%z~yp zo18a6hw8!P;>CKe+ZtCwBcZ{P{7I+-`3Z zhNFT)EDBr9V_&DiZoJpCr&Wg1oYdEYU{`%RnWbGLq@0jPWc6axtLipyBQ{9L+}VhP z%mI_byoD`osUfKg+7I;mR4+nhsWxA-!Df=+)EU^)f7Wv zSQBYBFG*Xy7Si082ImiTzxXo1pch-(yUh!Vl$i?@-e#8=WyRhvzkV`3+|!WTul?+u zHlEht+j(f3pn%^aWthwOWUb{6F8+`cO;muX0@*YGsVWcCIr)@@3m0*bUfG0{azg%1 zk5^=pd`sPC?LI;c0)^9+yeIVvt(1Vxv%wWU1O@Hv3#f6yR@d;#nK{}ezZl-L*etfG8o(9ac! z&Y=>!0Ew@=b86dIoD6=E+Se{9bQ%Ta0wuCbl{Tpt9(?YDV){7-6bDNpLGZaN$MQt` z`+YaXIy$n?O1I_ocvug6mD@)_cKEv(wi{hT)7B=X-$ zZA`E>u6A@jQqu@v z#0Jf$lo?-TFg|PqoBs))+l+#mQZJ~p8r{{!6De@rZS{b_TFo)DT*rN+rL&Ss6Wx}ZCA8RSb)tk zb+JgImBddf&a~lo%&3{(n$xgH!tS7}ndX~Zu*VotkGM~@_$ksCn(!eq=-Zz)&ELz< zf4{7ALfXx!m$(Qbx&))nHftQMqkAbGj8Bnzt-_kWlN&!OV=Y(Io-x$6&TgYXpD<&N~@Ci(`>YFd(lP& za1D9)%f!dZDnU@8?r_M9VmYLVa@obj_-r%ia9QoXHGq6v5oh=N0YF=s-PxcqHtaVF z=ayXoKkdILl|oVy@5Tuf2A9;CRaGl%vEhMm13cePTHqJ<@okicc%Vy@=6Td59R42- ziTVu<!v8(_=I>l=^pTT)mt?q{G@DtzEy#9a>!Y1U-wKpgyAb4o?)%wOE??gSrRRou z>9}Ik3c`mN^cp^?FUVkW!a7scpHSmZa6KJs{CIV+YGI7EQ(SJDG7FrRbvy>yo(yHI z)M)2n+5G-}m^|yU?vVs9997zlt1q0u>ta*#JU^&XhhIi4@U@X8gcrox;_x!hfYSI@FnU?c8KixWOM{JzY&09^x<3 z-5^c#bqoguqVUMeT#jhcfZee~eZ(RRSUu;>W7~A41Q?!iYCrl?u<}0!KitmHFE$&a zWdZC~;FB^IT*taq3x9MRo<6^Nd1lZF+pFX%ma8(KKZwbtQsS{yIhpY{gFLMyqZDU< zk~&X}l|NkVPmB_Pxhkw}Z9_^!$2`X=?yg(!(@k-Jpt0=UFwwQj89E(rp74hT+XF-Q zfs0Fo87m=MSGgrqIh7t>%==`c(7nB4ZKkC$FtaAbdF)r2U5BRv0Q5=wIz4n%!f`;#;7*(HZX2f2>)YKL2gwSFk-;d#~N#j@=*f{C?{$ zIC6n=Rw=t|_$8(2cBA@^e68gCdGKpCvLao7(`@LNem`qTR@A@4{l^)h9dj+04rmPMzDB81Wtrq z0k)40%e5k7BQtV%=+szE9=sfu)Uc9R#p{phqVue^lO7ge8+tUvAd@;kHdiyx6GN<; zfcG{bF3ic2u6>f>yJf3f)d>}apvVHhB52CD{>SZv@f(!6*E&E9-hY_iP@~S-(CHjpiy5fn6|)F>H>+ zh^En3Zrz&%l6P7Rg~AMs*Gp5=O+x zT2A&BMQnc$tLgcSu;0dnK3Lf60GTwR=2=0uGbb&)9Dw8;`Du=Tm6|qI=hgdhE(-e_ zJ=lih=G{@%GX;bB9FgKQo+6=>rb9PaoPAdkOLourZnIn^043u+*`Ey;pV_3*e^%dr zb1Ci6AZQY1BXRTNH!qa){Vn3Fk^Bu2%OCM$2#xlJP`(JGwc2mGnn+`VL0?#|MB#mx zC*JBLlVjf{DZGCg511zr$+-8-F0z{ZbZ{wb91Vbkfml02#x2BLaE4it^A*9DL5>`h zIzXkjNAJcaUTG2IUOqg6E%#x?v z+7~k%`y&&*%(#X0p?b~hkRm8_vH!tOG`NZk&q3!>V)0z ziKe1*11X_w(G%o#`VE}y<5U1jS>5AD6En9d>u+j(ZnA)?OXD1tKY&R}jqrFWgR@1I zh(E32?py9oZMbiX(_T@JE(a)hB(Kx0RC8ciPxq9=k4P^E z2+56}-3p4IZ>D(Noia9mT*;2_Dl;3ep$mBiTQU9|1*Q5d3L=gg+&;)srH6Mc?2I)~ zlleC4MMEdN(gA{@JbG*ce_>#^#yv++Cl1 z;HA+JZ80g;+ZHkQ@@~*2HtZLP$BjY6Zv?fQU#;Wa*$$J;qX~#=*F>{LAg=WJdhrct za@0NIJ874l?b)*0Z$g4?<36N13Z%3486nDV{Q@tg<>Iv_&H`XFqi_#}4R&d;+4F?_ zibtjX76b-~c{4a?3l|}-gN3S<973AP?$Q>D@$hp6-)NOIdKZH_7U$+L3tQg2=LL$| zWbb(#+jp&%C>CPt8AC)F(e3vjjxi5Rd6njwJ6fjEhLQn^VLL1 z3jeFK6zH%�ng`1%~hYX6rDMYN>x9$PV4)_Kc#G$pBVtrPex}3yB+Y5V7fLG93+P z3V>|i`vo;R-Q3$dt+n+ST?KyL&``6hD8bQza&PD%4fqDnu3qlqMeu69B>=_BAl(-^ z6veU%WA`Q!`K86N^JlX}jGlAE_mS13k&y>;IjIddyPJC!OJ4W16;v=#qI=H)wYhoB z%xUDfQ-P zv_bos{wXHSS!~t4+#G4O^xG>{WkOyiu9;N1(!0%;2A@?AY=Mm|t9X5!*13m$hUjSo zI$TT1+~Ae)7dh_&|5Rtm2)VBk56ts_j%lbA3}Suve~$&&eki5B10(eQ^tEUDWTNN7 zExESL;+sNB$W|X(g?_-|_;S3RmMvch)57HKfrHj?7j2Wf%mZS<7vbvDSApuCMhSF= zaY6>rFezn1ov`BYYU1i=i`~^t#3gxI3>l;02jFy~=HoZ*)nft7J{KmPOjS`>+M{4g z!iv#>$q}X2N@M@|=$*bImGE4U_a!BL*c08($9U7q*rM6I)S&6f!yu8fnVP+!7xK+S z3o&G=+6zJ>YYh6#Nj7SliopP@IZR~tYRmpK%_08}r>I%Wf#5h$<+0ys?@_u=PTPmoU&~aD}=~HZIr7?Ifo0UK>RpxbqbA=k-ZSwL& z2Z?A@g@yWFk*0GImWkeA0Sf*5P3T6gfqw-n>JU?_kY-Hs_kZOE(+n1>4AjX z|5U+PhspcMFtdKE!n|mCue~>3L$2jRxo;C-PWC7zsE|?&CpO~M^9+^9=k_elqnA#t zVIfd43>ac9S?wUh?t(rrN?|ufFD7{Dtb)98RG>dPW*%cRyx(Z40|x>f&V-H|2g`i& z>L^0!sc11BC92&HDaBj9>ojVO(rz!be~0gf_B4!7*Sor@d(L&29?kLvp9T>nN+<%AiWN{#QUvB4s71h|Hltsl2s;!Q4@)t8nt&cx zOk+FInwz{JgsE~)&2Y=epj=cs0`vg8*r4z3c?qA;b2IdA*%lg|7USOYsnAs<6!UO?V4NGhWvYl>L)kcbHfOOeQi=1B?LO@KQ5^r=;HcsR8h@0cB#;PZ^vBw z%c>QWL6DA%p=Nphf?yhQwX6V z{46K;lN~0^v1oihEx_gFDFHV=8%EdRUAF>t0%0Q2}IRk}8!9{1(6n38s!>F~E%sCJh|15v?lF{bV0>fD;4$Ton5xPywxieVk6KY3M? za&2bnUW-JF}9-Ql8TIyp>da)Zf$j@+nR0dB~pLZXQ5SGE`z*A4O$v<3^}>Eqw6Xw zECFD~`}F3}xL#{xH!-Kr>KB`rrJF`N#a#|XxyossVC_Gzbn}E8g^H_30N4pb{D@E* zj`}uE;t%iGgTubXv+U2=_N1rr>3gL%+Ul(&0O+EkKb9??@bgBc>~nV&VGtsHL%70z z6{3-`98CG~d3jKWA(V2S8{$iZMqKQXOxQQ*ty_ZjXa^PG1|haJJNOv8%14grO`hQ}D6<$L07XH^f!fj@I&zNb743?o{xvsVhw8%Dfr$Gt24l{IQa*8 zO#w5Gsvr~ZezYol#yj#nxH6d$fwT+RDbUq-3ac7E#Gg%ivbp?{!catPWdM)V>1fLk zd2&YL_~Dd{l1YR8Vba;Q{Yb94tbKw%VzzeG#)Ee@E2blzQAamt9KVzA1H!D;VRQ6P zS2#t#MAuUeZ5c>Pp?_Sc@DLgR1ApxD*viLwrb^sO#8O7mADWy7kd->!eaiLTq9@d- zB^EeE2}jNgnT9Hzboskh*18ySv+Q=FJM8cudfzR-<9X3-eqIyHmlgDTj13b8PE&MV0Y+@q%M;DJ^8v*Y<|$|sdi>idnlc-)C1^t=Vj7( zS+cCQmw)%tMHzF8gK_z5I(MoF3RcI$K+~U9c{%+)#6zO=G2}@EG{=G=KgZOl@Qqz@ z)|UIOrrNT*_>E_16Gf(^?IvT&&8TaFS2Nr>SUY!x;_M4BAM0z~Vtt&8cWN%RSq*)V zt`C3Jaj^y^fkqJ$4Av$v!D(M~m$k<%;3r}ntKA*9*31VP?2-;;m=i{=GKoNM|Hroi z(kBHpIPs4?gZ|Lr;CzE9E%aTy#QG4X3$W?fC3hVeJaI&csSpmna$Eh`--c8pAbrec6r6K465UZQSE6&n{J4<44jiL1WX z84`vt#i0?W!01`3is0aX#4i=NqDxB1dbnDOlX>X-a?OH~`cVYZ=^cVZv&L{`#+CD$ zC6z@vM`F9()Lxhv=UDe8;wk2anvKq0OzjPrL=@M#xKhD`+P&5f^bqi`V@{@6r6{#R zt3cb7^|{`0;~KR;qCR+sW?WfO~I9OEe$NuyNF$VRL1j^{tJp}9@@(wC1qEg@QCJwZG zNFt<{(hoPUb{jaM_ zm##d8U8&!X*W)_Wfy`A};cK!qp$3(v?wmn2vr4%R+HV78=!>>b2M3fIswg{X&~aII zn$Ru+$x?yzk`(>ZxrCJcqT(ZDG2W@P5%hp<;hkFqA5=NLPABYTZ6`vG*|L?}mb2xy zGNbK|MJSeqQhJ9U*kE6veY=As*b#LEO@YbLo4^4@U7$z}(u7fSEumJJw2vF33qp>! zrd7_v2DMkq8w(Ex;nIbo1rJ6|q_56hs74Y9Xiy0qW5XqXepw3~hpH}%0 zP0YNz;uv`}ASUHa^XS)=%jxoLY61DLV~&IDr(S-)nXhNOb%b%*$fgVqGWtZRX=6(< z8*Wv%22XCs6x76&9$6xfbFV0+E3vScUAV)gz*lnbSY2hafkG7F2Q!>V z7-003E9XYGm1l!Ooi-1sEKs>5QtMe!rdK^8I-~Wt@K%(>^Y;Zi=2eJlB(uhR7yd%h z>;BPIG(r#^Rl6?InylH_ynR4rg$P^q^r+&@RT}<0MVTk1?hkL05%V*j)=4?TbSW3e{S?-uW&A z(XWwXKW@LpKMhztCcAlGFl4CSVK70Z_q-xI$!wieY0ue$uz!jd>i8c9Z zwg*de3pV*<#HpPxjO8x!#zlD$#BZNF1&7+{h&6u{#~fvIjlJmlpd=QZ+59*=@oK22 zDBe4Cpd6q1;)FtGDjdtvu5a6)djj;H(_GsAtCO4*o`yM5CAK~kwzYz3o;035m9$pEpy1D#j4Q!V zmStNzRsM~ zvo3bdc#l{LOG2NM@I~<(5SNp^>t!HQt(-&|mTZbF1T&SkmP*hV9 zGI^d|@qG&KX4?{@+&Z%Tlf#b9VE(?SfH^Fs=jNEEpYzT$58h;383F_Nps@k5V|E3Y z@+tj`09WGBQRs+Ip_&QaFXhcV9^U!o*b4D7!kDWK`Y?Di#!iQkc*hI0R?!lVDB&_w zZ)jGChI?2Wo7~??EhdeLXZmOi9B52zy=P!4Ffde@tFI=A`uT+!c;wb4s1?N|sdoh= zomah=NG^OZ9{ZT@X}#dx;Y4mW=&Xm*9=Z&e;N+Q%iSCi~t>DyoWLfwjPnX zdm|eM0ZW73h$Sp;^JQq|0<%gc9kYoVV8nuqh!vSJR}rX*UG=Siy`Kj5Nz49tCNAkUX)-TdMyM zZqw`Pu~vrT!DjR8Xyi`LLHn8CQ6Ra!t#&2pjCYD!AwxWFH3?wb*Q8~=bR=#Xpw*Je zD3OKP@50dLNMid?&D?vH;ws~dQ0wZxrD6SFTM<%-GdNPFdE*^#)Nr;Qt||rPwI-)2 z7Pv+y@!~o$kh33Eyi%dYI1kuxgBfQd8DE#6`iU%to4Eh(*g3+-^b5wo$?xY41Tv3S zxnIeZY13P5W%?znyQ-Z4V@UCd)te*2vagX{L%fY#cOhrjlx8QlM1)&Y$9l~YmEj9? ziqT}J3!6XMhnnN^n;Io~T|R$SQ|LQJD^o3XOKu`9$Jl&|S7c!-{mJ73>N;xSH8aHe zb+xNy#W%w9ECL^)VXC1TqoMjR9A5{#HRoGe@$j!FfJss6O0NYG)tWkAT+<$vuX2V9 z^^iDdZIEn0hCoTU1*^O<>1Jcu+i@lR?l}+Yoy9m=h10lV-5aCAQAkHH$6HksP_Ht4 zM1QI=z4Xh59AeXmjgc4gm*b5Vb%MjfNUt}s@oyPVVKiH-v{q?EcLSQTnX*5DU2puH(r1;N*y%qPo$9v zUIWLyV-Py>DlFZbF(fVd^GMA-R$G_OT%2GnZOSfrJ(RPZTwiOlA3(jNa>|PNQzfDY`4AJ>uo?*)StDGfB|SZaxSg^}TO`R8u2 z-Z3nC!XrPd0=p7391UTEJnPVuul0!0OE78*dod(Qa-8bMrggjwQ;>}EH@qb}bHc|r ztWJKttA|B#b%+F&5i%W+i5QBW3osV~9Ym*tkiTkO_;Sub;81R=id|CdvY>7U@6w1$a9W__y)FTVH zPUw=+BuL<1LCFny;tZV`f?)ansVbQJ>wcW`LXC~?J=t)s_H6U%rgOQLg~I8L0A|3F zI)e&JmyBex#w~9Rw=2YEc-~;V^eugfpxkd*ncrp?k=!>YLJ4@fZ9L(aGuXeL;td~f zbrETfI6P*{W{k{E(Wy#9wzlh;w?6Awb~#n3>Q`yDNHDs+ZjC*jmvfYq6@ZUlX)q8C zv@-C6G7*Sx=;JSDewoM^_Xw1EfANkX!oW+r&@c~fi0!LU|D{Qj;9|*l&pW0kvK>|) zv6SomyyZB7+^g^Sd|9XgpI3|P)S5Q*NU?#serTKBuHi25ivciL&VJ1rp{;}-ytcOn zgWfyK4>Zj1w^z~jy-em;S!LSS{zzACbp;p3g4ABbOj-tHp&I70eUZ}<_A-J`I)R2> zjb~`A7n{CeUZ$l-EuiT$X)lXr<~67MO`t#owO6K(K^`39e2nQE*-BDrtW89cbcx^E z!F`nq8%A-zR_9;Ia2#;yDH%599~@6lvEr z3? z0GSEPnX&?kZ$^;z@!_pThM;8V!lU4|z5SIIs<-Bk`wb=u+TwE8%=Zr;<8xBa}d}xE&*oN)B1fZyjLvL&co5GHbpb=m^)O3 zELQ~eauqGMKf(wUf3q!Qyh3?9qDmE8(l z-i_ytKrg>IQmcnb9_zeLd)uqmn6ei?J?QVyr#XIYWr@BcNB|=G%kn z#U+ah;yNlj>CBlpTkZRf$k0HWQmXq>6$SyO0f%}qBNOzAEE^CRUcFzft$yQjOG+a7vcWI8RwZK%iEAe9xu1u7DUSBy`+Zh3mCef2=;*}{b z<`yjF?^hd)rx(#t=LX3w5nN#{iIk>u%j<~B)^BpY-MHjB1>;nitZ=#0XBN;F9nCkQ z_v??uP;!(lSs0MoLau0)E3F8xH1fmsFZ_?iAHO2yhy}+*iI?Uf%Re5f)rzgIqKR2g zG^nG@vJ5!REn13YBE;rQO-c`AvQ(n@;hoVHV%P}bm+y#~u6k7&82dZ8bbUa0+0Yd% z4JE`tlaN8eok!M{St41kySv`TcWSK~ zIzuVL?8d#04j+ptHw{@C1ELmIKg)7(mYBlnqEL@+ArP!sOlTC2I~`;xv9RHM;*Th3 z3EDPyvFvxIs9L{zd*7D+LArxm-5bhB2o09iB4&HP%}AC`^BWSTyl9yQG!fZ^BdvB+ z6u_~cqA8_1(epRT10r8NQOjX?8`rS~OkX6fN~_L>$K;*!km501ZcPyh+tiJ-)1ne? zK8^d#7#z#;<0voq{f8?<*TX0_%fYC9T>K))#n2%7)L`}( z9J?__vD0nkl7vcJ(#-NV;w8r@$%?U}dM@+867$H&>64u5u0Ruvgyhv@52xj^VVpp7_88o%|f zw|WR|u@vy>FEEYaAMvXF{x(?fK?U)N=Kk`bCT-Ba-K%A#BLf61@nrth;Sx(Qdy{I> zt(f}Y1s@$thAVemI|I^zS z!%Pxtxg%)0QB5A_6yn*Bk_lOu##Z%4O5u7ecJr@8W90VT(_@V&psiaTt`|uXvz8dj zWYLwW$?o!orBZo*bQs9qOY?;4z&mJ^Hu>qWFvKWIZRm&k-c`t5m%nWv`sJa$Hz=Tz zmWkxh+ii<)Cr1}!G2FB_d4ZrE^Ku(ggNpU0rud>@ucV}~7=uX!mK1sc+B>UyJ%9P5 z-o&>ki>f4*t2}Gzj+XuduLtr==y124Qt_t#4Wz|oXezC9m7t0hPd`exo|mySrj-;gm;I2N zxTjHnM5wRwB86K@=00_Ct*^Plm7rQKUm_qXO9@=pmqw4jX2@Ldn1d_L&?qA7E-j-U z*H%M%hmnZ_nZV;hl*t*9jbqMxB*HKC@;LsHK|LzN+Q&?Ryo2%}TUTstx{uvDU5TN# zt_}%Bu9PF68dg7bNa_DPA=TR}Akhm9tDvi7#_{xE3O|N4=S{Wol=E$DhQ^GweyCSH z@ID!Lx-8%K89NrK+Af4G(;ZEntqpMW7KUH7O%J_l5j=tA#}_(apkVS;LR2g95hB3- z8WP9r|3H(B^-EstaURT!LHtf$`{quZO(@|~4Z^x%-&&w?f^5F3v`QUfz>eVG@7~jR zO(q?`r|nJ%GW->x7)|ywKKqDx-_P;GQL^GE3!Rq;F_oTf*QGo#*KMIwU4F+}7G;{8 zdUwK_dQ{;BPH8d(G&pQuZ3WZj5iz5%V-;&yJidKOzus|&6g!TFmmQPWyghC=Y;8Lc zUz$z0xUCvxlc}`y|L!<^b&L*bJ=q5A^4bB%eP`BjWg^(2!U}d%n!Q$~P*>iOTG8^{ zj|e+ByTs?11q4M&aa|hcv{JV`d(2^)`pZFKl&CHGa0XzgrN-mCs`(`_jRUBIE&KdU zxihg)xJmOd$SH8Y)p|bkShqHq->6#-aiWuPHrcS5uUU2xM0_c&gBO1%`HfnH;Nfk9 z73Gqq*aGw3hW56q9vXr77ooL*K*6Tgc|jYpB8Tcqz=*xcQE^rU11#-!#`qg^N(?6a zf4XoHBG>!mUmZ`zsm~)|*xA9DkiV{XwgUOyP)n5|9PU-348v0BWvqGm#q*L)muzR? zl3yQn{wnp|8p$WOD&4)WMMA8P4pM$?(D|xYS^>(Gv|*^x`LIdLIh9PWuOO=VcV`vy z3}b>u7fHVR#zgE1uupYI1sZ;387fXcdgpxy^dz7riv2v8=Wgr)F&ALdEM{n2)@aSr zX-=SXR1|bmxk*FQy&8YW0m`i3d}oKoa+QS#(~|{P?F{F~Qp)DC?4G!#^5XGg8s{PH zV8NR6DrMl0p)pu3EAWcVq1FhrBU=2I?~ z0wx1J!mrB$Q7{#?-bph!tmFDRYS?XR=sT~s`g(*iX8x%`OdFhpMH#xJjwj7_Ve81E zqVh&u+XS){PDz55;)G*a^L9@p(iE|BO{(>;j1R6ZoUik=ek`_SVZU-`2BF;3tfzG( zv3+Tku23OhJT)j`a)!wsb#+~pTz0z^!f$cDPx64hwP|fHKg~OA^O$9yW^XbY-%Cos z>)wc~oy->lvT1QwEkTA7lfjAmo)bbtl-CO}zLJ1=;_L1?-}G6EfR!v{<1?C4IJ5{s z7+MAtZP>~?rC?~Z7cV_fNWGE=$o{#y@$&Q1^fqMgl|&0ThyK}e+T@TuZ@-S#??(~< z;%u6PoK3{9D}A?a<1f((f$Le~`TVu_T=$tg^S>MhxIbYB}7PpI%FoW{H^qF<-n8{9#xXH%WvQJn+}9)_Sw|@I9=+h4)_J>fHYG+-urU z6;q84w^91^Dgz8ZV?4$6EJ#p4L;V@XmQIf{ji(#BPptm)t_qO8;WWH|vHyOX0i%}E zOj?5aH#zm6ck(ZdwMQCC^=vCJ9n+{(wBzdw_r&}8*QAk$jIY&*n%B+f{{KGuk2CR2 zK^O1^C!vf+{flV(>|6flkc1IfuaZ{b{NDLb-u>mNixk|R(EKOOAR^|2`G1Z6*E>Ck zuIf6k@w~H($Us{xHvsdKRQyL2%(xVxpw~I(ofO0bn2rM%16Q8g9eI*+`+G)PX)%+e?lm!7D^;;1?blk57!h7J^Wa&5Mt4vgM!78Y%i18x zO7yuZ<4hVnc8wtxV;WKrj{z@k#Jvvp@wno{Sz|Th#&Z2~6scsK#sV_lz5@-zLCRdc za@-@U60*e#4O_Wp?13MG`0dwFQpZ4HG>vn;g7f- zME2kBkyJ}BsiWvfNM6>2NfNw#e47C^;z^T;{=CCLr*`~guQreiA0<(2vYRmI)Q)0S zWo(}%d}naeIuWV=`R*{x=4eh92rCpRn5+;*DYW4uBO*pqwz>`s1)`HMdwL8$s=R0S zyi526S?8Q@I*~Tvb@P?W$~|fIf_B(W6!eF_hCZ7~IE~7-T8Ewy*xli)Jik6rwYga} zx4HTwI~YNmj8P`1I5bsUPR^KG9<(Y1|$*^!*}(4GI>q}Tk$<6g1R`MwE*LS9KOf0Tezi7pQT zes4KB!@`U?w9^N*xogQq^v{pm?ZL1|$|?FZ9#ZsXM_bZP$EEv3M$mOvcWqjP1j?38 z>iKLRSd~fNE&g}Z*VyQGiTCi$w!$+~=~R!u?lraW86~QAT*tJ981T^ouAl7ux;_Hw zU$GnDwa72%Dom;za>DaCs_n=#vtYT6yJoXMj6yX@1m&`gw;YzkClr$^VhR}=?DKcK zP=l(>pJrSjkMkL^MiqH3r^9Vg%qC7-zQ9ber?>*I>g-^*i>%6IP?dv-6k9~Ab@c~1 zA8C+Cby}Z@hE}IA{+)g|zOBB%taL4me&-n`X{%gX>Qb|^;l;fiAPy0h!g|nW!XS`n zV2CAR(7<1uADM8<<%-jnzV_G^d+4o}9-Jrk*MtNb=oz-R+1#IDvhY6!Z&5u|Sg>L- zy|I7>eSNMu_#Q25hB&_Hq zALklm0%B2R(9PG(+xC5fJH0IyJ`?Jd(AQ^Zq#Q)yPYS^&Vsgq)KtibiSSy~HX67Ud8A9`U>a-M4r z^hA(2WD~?Pc+@Do@{#?k^5b0~`b4n<5~WI+Ps`ZuFh#P$exlL#AlvGM30*3XQ^TY@ z0LUbhbkgfK;KKSx?Cs06G_EE+|2imI+2*iCP6`rfE$7SrbePM0V2HaDCUB!%@`NlA z{R;v1U)?gm2%1YVEmJjs{gt;Xfn`rJW6K~L>%{9STH!j&Ny#B_Ml*z1C5!ap#)!@B zK>bDByBH(m-LmQ8np(}XqMwhwVn?OE-bVpuA23dXEofE=F zhrKA@g}$%Mcw#j-BnLTvRKX1F6Xb5M5f_3-832hA zDn01k-+&FmvW0F^@9mkd)2~dKYZflkZY z{%SaTczrNVT;*~`;%*jfg9y#vdL(+PO;h%vye?EGjLpC-iC4Lia-nQ7G!VaesmWRs z>yAsMT2KO|xP!`QI!4sS>YU78T1r%;&Fu`3-!$pkN=)46fl{*ZRZYp84wOeiI~tA4 zVTq~jiW-gRt`6dWCbuuRR&jlN8AYV{ol>hhuY)^VkuIH^`blf{M>bzB@~r%Rmxe)k z=>SL>xo5#6rST1Q87kJA$YdR3(z}lrKRSE}x9_i;j~r8xaTVJ~s1QBh6G38{2S54+ivUryr6X?L zOtTa=Wn{C3g0yJvsQrhUqQ{+ai=32wmTL(Ag3p0;EV+K+zB+j zQAE2UfEDtnvE>+re~bhssX?h)Mqcwc8abI$mx>F$w`BLI@`kq!l}#vwb#o7)j{^%P z@-Wdijreqg2C-1osZ~&jb!%;WYgH4XXIyd5MsGSQI1oO;havMwfysrYkP|O)IW+H zm{?lxy;rVlMw4C4wA&PBMw2KZtaqhH&QiT~B@Ts6w$ozf?r%U*eakKhTi|(3UfU)Y zJhno3usis38MRtzQ@@3WW+J=U-Fg49joGYJquWslkE_!n>#bm8F4NfwdM&BCw-3;F z0ICxx6zXV27<@f=cG^D3K26Rr6JG*}*`iilp!KwLbbAjfF>qSgZFRm*x^us+W0-Y0 z53aRhzTffCX8ZbPxD;2|hKp#71Gdd~o?8N|>g(<=r}?rtb_ zFvwzcYdoF-WiNyc)+3!=I!FNfLL943hZi$jQnF3e?*n1h{kCu+W0eaZOHl|^2A}8T z*Ixf!C_%t)?XWT^duHR-5gqN~@c=avKsn*@=-|b7I=xkmC&s@~ zwlseNF-u&k_9u}BvJl+5A6o=ft8BU~ft8Im2;Rh09y|OUok^?@6naE9ubXt^sdygQ z#RMD~{o9(>%|F&}G1$&yvR(~^F~4Uj94y$NvT}1^uQ3SL*?21!sPyW!5Zs>!z%daU ztQPK?@|>mDo{3h8q#UE8z!vZaapquD@wQEfNgL-5Cdi({+AhVgcRp3G!FBa|i=mlzEEB&k{S57cxYN>|oZ?4I3ySC%4Kix{{K<4A3blK_OXV0c zYU7T_t!IEE$N%Y+1yI<1R4f95qeva`Sgh+LKbA zHK2r|_(%=0ZwAU4he~3+*;B0mjJL7a&OMz*u6C(NrZs85-2ePEMOnKeq*_Y*% z=tBlI_(4Yis5bBf(g15U3ar_S>IU;YU0g`7$;9l|+W>8ffs787t3{P%eDLJ} zqsgGwz4HO=pkoGr_p*k=cPk0`P>l>9!}IkU3=G-|{nwKvBtYomMR&sAp^;Mv$5zLt zkDC52`-&{!G_AY!Qt{-d+sS43w|Q#scq-@RXT7r@QLCcfJLZd#_-VHx?6zXxYd;x2 z!Mcn{2uYXKM*CMa2fPGbHwOFbgXQ#kxM!Djqps&iLJw_ji)$Q~&|)$3ufV1-0$3~$ zI}~zXc$))E{VVTc@n08T>ldiU$rNm+brfM8S!H3~%K%Xwv$mPj>;_ zw+*jj_1ZB#)TNJEp4shzZiE1wiB9vYBDmgQt{mm+p}Y3`o+CciXwR1lr#FZmC%Dkw z0&ftlLijQhfrUkNv8VNFf1xE54NaF%K*{ad-osDF@A}7R+DFamnX(5rVDld|?(gMi zMpFo%0@15;db~#@^;6fxeM2XHAF8J}yl~YzO^QB-SgOfmdHxXLW=OvZY}nb(Mf44?jhXa>Gfn= z&l{=uVhrjsTefoT**#YtS?+_LHuthkoqNq5uDN8y8vXZ`45oa%n&F=Dw1Lc= zOTR}9=H6(eV~|c zm{9e3Y)UF`PpB&kn zUN#aQYP0!Y(65M#siOU?yuBMlx7c-C?z9bdkvN>>T%c(&ZyrRw-Q`J(D2VpgZ0CM`l{drk(+}c+1S5S~c?{2amO7ivf|^BxqP)VYrpmGol)+@_B5G>t1_g-G5&n$$MO~7HOmW(xNxWm9pDF-RUk;h7sH(t} z`B!9N&G5{`wp+o!q9%vV@<*ie3S*x|P$oTP1SFUlC@w<2yG-?f~YZ2VyJHV54R+9@^t1rA+kfHC}e&bl~ zn?tG#P+qU&_tcu?(NydBy$lZ1sx?I@`Ml;ZA?o(tzXRF=yrw$>MQ=s+4s3I=9G(%y3{P z8uIbu{EMT)QmWLU)0{W?Al=P`I%efKy}VpRpV?1Ic#Blzo2Z?k1B%PwGKM z!A~QYh90}8xksKys=wvFE^six=WNmtKGG$X$H9Gd4N}OzOJ`v*zX@h5MHj|94*|lZ zY1X<_f7kxJq4{2*2U?k29R^D(YJ^d$iGE9h=?zPK(Z^7mqx6(Du*Ax<^A2aEhssBs zt9i|{WO&9>hqq+;_cxzGj+Z|0M`*R_Q%A!d#>;XKnso})VZVA+@qbye-k-T{E-22D zL~AqF=s#t1wB{?hsfPtQz&hDHdQd`IIHR935OOm$ReuD()q55FP0DSV5oJvZCikKS zQG-Z8SB<_Wk{=?*t3Imbt5m?>==GWOPq~!bo?m=Tvp9r3JAkQwU1}=8dfY@&w~M7P zAF4h+ViFelvzCqgC2c@rkDbI;H$R2Jx&u|Y4KH>Q@5Z@In>BhWb;82-OU!iG=r*O7 z>J+PMP%L#hThaF0KTMdm6CGD)TTqp^&ztPOsD?EvQj%>(sDE{Ij3N`yTka7^gklSp z+7MmsdRJ$#4&}~!U|}hl7Cu!7VjZ^#>rPE(c_0ciYJ+H~>g^_8bb^t#&X3G-fW(^m zY27yg&c7uSoyyneN|V=jN38FN$h7iOP>g5Ota{<&Q@&cI+LwIle{~NMUp%z8X^Ow9 z9QO!igRF$vdmo*zxGg$ON=0J@BV{DGH7J?)8*z*hhyTe5NZccliQ9I+7>CQxY9dby z5j6beU#!*KQEM4lqsX&N13~ zs_8_c?anNa+)x&JG@ERR;~7G!!fb@?`WOmYt+@i*uG&Hka!U`zJX^Prpti$zsF4~Z zO)-l0GIxKUD%8G+^Pk=~DN$fUV3-ux{va*+X*GfyG#g}7Zj9XQA<2aeT~d!oMv|&a zojVvIOTAN;!nuo8URbF%AKX9?Ab%z9@1XWgd4Jw$dh3fIccaNDFBzo1Y}KkXBUF?} zYLU?cMQK3E1R^7AuoRh2MopG>R-R;tjgCY~b0C z>vaU+aQl@!6LAcjn#>I-6fpEw2RpF3Rumds;e!oVE?+ zc(rFlY4cqJ&I>2m4Q64i{Z67l3unH^7Le`D)8om1SMm|1k(&H)FXf7<@SUPV=Nud$GM3ycj9sOI zg8-yi`@@=ex_>N1-+U~3zj!ACrl#KT4$S!_Q_Au9#M>gDUc7+eLJe0c@u);Fir7Bm z{<8djq0xmc=6co({FB_n87e{^lD;2&LG^l?8ITsXRn|5uhM-D-5rEsEUJzz4+vx~H5XIR*`B+xgdKTFX+%N0BEBbzhqj!fjhVRd!gOCyd z2xGar;I;_iqm@dFkISviDz|w!aPsk5;v!2pd3rnwdhR_P&iWRk^ERc7VxbhMLLl?E7g8jmam+6osYpBbCOK9W zWF^i7I$s4!$YK}3Zvd{wV1$FYP_K40Y!GJr5=wjo^ZVEhWykN+KW)N6D>hpvqdW21l*|-X2XFj8!5_(58YzCiHJ>2&%49ir&~$G`w*yAxWVc4=$;wZsX{si_SfQen6r|APWd+4tflR0 z;{AE)#@$gUy_+6K-U}dwfFaCt(q;LjlXoyMC(&UnJ?jV}G|->MRW>B#KoX}BAZrrn$aa=gZ9tpIBfo$ac_NNu}6i=%`?~OIoEcZi^pqznq zMh)RhQ?b(|mRJLAHDDPKrIb;c!pyIJmZ|A5n8F!fka6K*n#}#!(1fp+;Z-NYRi!7K zqX_(N=E0u#)z881JLx&8=S0{@YXOjQxe7O!`lYoxOHmAlJ)IFWx`#vU#vniSHQd(W zo>IB*ut5YAqg7jjpggYX{9q6yVDS~UiPO#~)i6_o<)y*Lg+?1ciKBT<>Mc~*fimAV zdr%%9KR!A`*}YUjo4_gWOKnI?AT=y#`3a@Lrr^rbfaX14fgR_;`C}^W;HJJ`(EBkt z*7diEJI`#K$-ra}>V>B`@y;L9MdBTd24e3IAYC;*={9pWN@CkG4KSmMBxj@e1*x>B z#*YTSfOeQ+HYO#tbC%q`>Eh=>?Yt5`6MbxnuI{cr37h#*@+o(Twl1PSUBD(fb5;)C_M?@W%_P}2_OPIG z9r*npMj3!S_cq3;pgU}PxQ0|s|B_)wq=M!me@kW*ie~WYzM9`+wdFk_h#NGs?*ij$ z@8KKdWc7kh90=!Qn+%y^(;Fko>TRzNf3C4!(HMCZgxY-eCeD{rxg?;*Oq#*ThtF65&*5#asi9W0F?LshYkY|#sbr$FUn7Imz#0;?gDo8D^ z<~^b~%1|h58TuaQV6&NL5`DR427t($lYVJk#=$>Kd?H~8} z2`t!8du&(H;mqL^qpxS}lVnRc8gE)$PP2aD&G8PYd$+-OUj`{&m1*PWZj2<&nBa(I z2=8+hmQ;leT(oI7k$OWp*pwf+*qn|xNjIkk4BjZrD+uwev z@AWu3sWay|!X4N%<9gQ_c(QU6BhP^wawH>IfW~l zyabBDLia3wgZrg6n5*PBs=75m|JFm$e{5UN~9bimgdmicT;qbnH zyvx@8tQqmmvMwHNC|7YMij-7yMuy$-qM#&&i@!ICgO)|x6r?mdL|;1={aXBMsEiU8 zCp7xIy=sGaCMqnLr4rovhw>ISgC_lM#*J)AI~Cz#AD-= z!~?3ty4@MkSu2ofE-$L3;$7ViI`TrxV*kLhnNQ4qWBgNKlL_^SOKLr+NYZIP?z8#~-@23~;^lIOdF8#ZJcO zu#XhL{z_=EK$u;MIwL+5w{+Q;D2*T{!|LVbg0$epm%PS@^hQr(Hov)ctFs)Q5JqcW zM*Y#u<>I888}qqsru&kOM$Se{APl7>_`7KLb<5h%+HbR?cl(AGzl8CPg(lm@vBdq~ z7IAx^QGie^R!FqDA4_azZ+!KsRb={=o@DxxR361h?(kbYh@)YrJBFMFz?eYmeT>;Q{MBdO6VF)2Pg-oJs;lhMP(cz4Ja z>4!xlCz8ALVi$lt97Y3-K4z7^jct}b*`IA_!!xR;6#quCglx7%F|5dwz~wLWy`p!q z4z}3?sN2ytHInRD4p>LqRH0VBv_60_>-RKU7F))G!X+f4K?*^tDJ=gWRFhfx^J+>hG{7!Sa zzRWPT0g$Y2FzMqgmtn?kUfAZ`3DqcEn|+yaTwDKI`wV4=rSqM<&!r@egar+C3gE3A zz&~3D1^VzkCBFuq05Vny`OAz9?+#5&3>|v6hupLmbzL3$9*}Ru(8U*(6Epj56zZJ< z6W=>B-cgRqV3mBf!Jwjf3Tk@hxk>yqq`!z@zN<;jhIF&PYPYUF@~Z7Tc)5-xV6}{u z2!(WNmGM~(h|T^wxQ?~(1lAu@BvbJgLrhcMvy4{ess?_qVbuJ{=>ITU{StjDYM?pG zKGb9~wA-p7it37N?W$YECHPZjW3Ci#WD9GFvhODp(xpHPAw?qVVHPQtN9GGZhMTh8qcUF74KjneHgoH~tI@ZRaGolly z!EwLNv*VnTvq%~c>93-x#Bf~z1Sa2N7#9cv)OszEWwQ+_`(^M7N0`QnFn;WUrQL}* zZ{-XOgXbrP6dW6oLaf{mKWsrA%bt+@VTGwx#4Q`RUR zE~TNxzY`=ZvdK7&+$TrJ?F*p9NfhS_Ev4J-q4t{ZquA1!A$N$*K$x{m zuX7o8C{4qev9++ZCiy_c6+XA*RXKHSLXZQR}Hy4hsrTOp$}<8v`34 z+TADi*qar`+lH@SRN%Q@w9ru8=X8uAjvOp`;@`_mczaf8RMCW?cu1|3ou{tVxzy?I zdFkI-<5DQ^gF_YUe+U0+!;%g5h%p{-zK?d#jCmgHrB(nH_9zu^m zu9{Ay@FW~i*mQpVX(GvFnpvitI1*66Iv(yLyS0e&XIGKmH;UFXaX=8_H+n$nF=3B8 zJ5Z1y&A9Umz0xaJq67@~@}(yP9t|~~Px*KH?2NH!G--T9i}Ndv_b{`36&Rw)q}57a zoF*Rl@;k*q@{sF2#lwJ?FunY`Z)@(us_wi5XsH^rkJJ;YIG}MTJYSLi0Rge`nQfie z_UYUVa)S*QF6HZG&vx>`R;eF%41xZEeSC0vVqwQCw5qX+cw_;u2pi%n@|wwo#L6{n zojH&R9X1mMVYj2?GCKScH5rU7U0nW%3P2kxToMR3v#L4gf(>O6swLx4-*$y#Fj%?C0r`6zyUnbOEj<)I* z+3j%Kz}sY24%X8{E==Ro!?4OW_#i$r$##*rCPW zU;isL{wDGTSqW#nr-V&$f{<1GK1l#eqs@lX9(Z@d81qtU_Wa#HoC1=sP@U>gk?;1Z zMxKpi|2#zHgIWmH${kve+`O?!7fW=xB(*JDTKK+^QdPIoaw3hZhf=ndRc(W$!;;AI zkMs6$Le=mEJJyF_N{K)>Y;X`ROV;^8qp`kJ%OE{~49`UJnW3#+%a~0V&HvXI!TY_? zaQOWPl1tFGTPq&Fl;n5A8z8RQ+?^c7Xn&%Y)TsEA^B+w@4S%}sa0{&Bq!T!+E{gSK z`MLmTG^&?WV`%%S2F)S_SMVpU6j#SDP5C`d`29|LZjg z-V@KtbM7dBiA`6CIi|xoFJhbbMM_Yym(+ zn2B@5zqOb1scXBU=s=!jenSbo5G!JlClhZPlPO%-*!GI?mZ%0m%UBO*@z@{*=bv^+ zJcmK}v;f)plu?+d`S9=0dJZrq!&hCJ&-?rdtoJJ%wxNm2?CnCwl}llV$Q@w2_`eWC z(f1o1aw1&(&mlxDw6(`?&hR0F&zgzG>RnOssh#Ah4~fVCNL} z{9;?k`4E|p6HgKT-XL!$iM4${#47zMIQ9yIYTxBHTHqSvsOW10s)TmKaAN^BbP! zaw^e81CKx~ii9W;&xsH4FVu9t?T`%jg+_|!OE*j94u7E9{=jN;2+@0Wr~=SWALnXB zrH)qftA8aB=VO#AX+BY`;Y10r&yBQ4^eOpzaIs4)^6%(7v`xrM%i^TCrr;T>KirAt&iggp4e+{!R!H})L?Yp{_OWz&`Y?;@rS+7F5 zdqmNzF_+2-z9G!D8Sv|cZhXWk+XY*!l@3}^XO z>0eM30+k{Pm)m}TT&ho>0A$@!Jb3v>&2~3c8%)o+a)cVrq6fjn`dGj~#NxP|NdR;3 zX|m945$TtvR%!FG@El?`k%+zdHwYE}C*Tsp?L6i7Mg;}d_u775Vgf9s}5Zs;G<5PvB&+Oo`zLoFb*xIs7a`&%I9v%;r&TaqQ*1 zo{eYTud;JP;!$GKOR|#jl&!p~-##j(lPMEI_B7^ksDm*~`6(P$1V#EU*SRc-|Fdidq$p(nEQ2n{i=oB*ZhETf|d z;DuA(u_}~lw|z9g^X7mjqlF;JU|B&7My4n^j^;1uApp@|b=~22G%iI)7pVpz z_B_iIr9(aXz5bYhMavwAaQ>*)6ne&_l2YdsD87;_)0uCCm5JYIj9nrr(X4G9c{lRGES$G7><_`0X*z?_Pl+|g+Twqe1&>td! zu-%o?%$6bKD89s9oTu7g11a9C1G$s=nkL;{UHY97>6(z4 z(4rCb@AYBrC?AZ0IwtD~6Qb{p&~|9l6fC=RhpB*oJ$8)A*3hl(f@bzSTf1Y7X}{_{ z7@X6>egGM%6w3ly~aKma7+k*0Fwg4ghk^} z@tqvv{AbHDn)fUMsf&cUY0gR$M#FSQ$hc9MoBe~4hmS+9@_FJwsJWqx0S9I%CVK07 zg?hLz3@eu*dg?tRT#T6dqZJ@c2dc*^2vEC7~Pl1?N+SNvFm5Sl#5O({PBi0%4& z39>#n#*u(L*1qoqYzM+mm%^2asWkUnxu^z6l=#G2%lGYiw!HgS*+)fl?>lbSxC41C zhlm3hL{T6@dC59)tkEa{Ak-B1A7wOG1Vl}DeQ_w;)b(I^tR1~EmG6BFg9VXQ6%RBKbpWAt$Qr!q<{ z)8MMKJwQ>Zb)>jVP^QOSrO4NG)#MeH3sSS4ZK^K@xLRyBWL5 z)HYKiy}8<~HRyHO{l+{HH<@EFnW}B(QaW}9nOHn4j~2m~LJq7VQrd+|bOgvyanQfQ zCz3CI1=jkX0q+DRjMkRSyGM$kuFbz9>}0F0b-`Zus{!|GzgTjchaPxfm5-e_f<{zf ze6C=%{NzmlYjV$=_zpS5prYj2GVv!5L9#C-G7sqrUZp;j*F$5PGFpEiZ8>}6EgNzP z420K|qqz+i(J1PpHlm{1D>63EZ>9*ZK~h%OZ~f-^$)k4@8eVA-^XUc66EJo(sRch_&ndiQ+lbxZMEI= z!Q`(YG5f+qxSj zG*1OuBM9J5nhE)rzOwp2Z(uwnoE9#24*mN*TOtK~b_bZh-;TdKNB?sNDo{nzSS2}z z>DYkA>Hc_;-aVJ`n-O^3La0OY<+GNvV5rGoM;)iVIBF9AHq()y8bdw8QiJ2*!v;&O QUjRR%LQ-GK1ay4<4`HYHhX4Qo literal 0 HcmV?d00001 From 908eff41716b93af4220260aad9d26c413f3490f Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:53:50 -0500 Subject: [PATCH 26/37] [docs-beta] migrate - docker agent docs (#26506) ## Summary & Motivation Broken links converted to `/todo`: ``` - Broken link on source page path = /dagster-plus/deployment/deployment-types/hybrid/docker/configuration: -> linking to /dagster-plus/deployment/agents/docker/configuring-running-docker-agent -> linking to /dagster-plus/managing-deployments/environment-variables-and-secrets -> linking to /guides/dagster/using-environment-variables-and-secrets - Broken link on source page path = /dagster-plus/deployment/deployment-types/hybrid/docker/setup: -> linking to /dagster-plus/account/managing-users -> linking to /dagster-plus/managing-deployments/code-locations ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../hybrid/docker/configuration.md | 38 +++++- .../deployment-types/hybrid/docker/setup.md | 122 +++++++++++++++++- .../agents/dagster-cloud-instance-status.png | Bin 0 -> 154835 bytes 3 files changed, 156 insertions(+), 4 deletions(-) create mode 100644 docs/docs-beta/static/images/dagster-cloud/agents/dagster-cloud-instance-status.png diff --git a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md index 1294b30313cfb..99111c0fa0616 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/configuration.md @@ -1,7 +1,41 @@ --- title: Docker agent configuration sidebar_position: 200 -unlisted: true --- -{/* TODO copy from https://docs.dagster.io/dagster-plus/deployment/agents/docker/configuration-reference */} \ No newline at end of file +:::note +This guide is applicable to Dagster+. +::: + +{/* This reference describes the various configuration options Dagster+ currently supports for [Docker agents](/dagster-plus/deployment/agents/docker/configuring-running-docker-agent). */} +This reference describes the various configuration options Dagster+ currently supports for [Docker agents](/todo). + +--- + +## Environment variables and secrets + +Using the `container_context.docker.env_vars` property, you can include environment variables and secrets in the Docker container associated with a specific code location. For example: + +```yaml +# dagster_cloud.yaml +locations: + - location_name: cloud-examples + image: dagster/dagster-cloud-examples:latest + code_source: + package_name: dagster_cloud_examples + container_context: + docker: + env_vars: + - DATABASE_NAME + - DATABASE_USERNAME=hooli_testing +``` + +The `container_context.docker.env_vars` property is a list, where each item can be either `KEY` or `KEY=VALUE`. If only `KEY` is specified, the value will be pulled from the local environment. + +Refer to the following guides for more info about environment variables: + +{/* - [Dagster+ environment variables and secrets](/dagster-plus/managing-deployments/environment-variables-and-secrets) */} +- [Dagster+ environment variables and secrets](/todo) +{/* - [Using environment variables and secrets in Dagster code](/guides/dagster/using-environment-variables-and-secrets) */} +- [Using environment variables and secrets in Dagster code](/todo) + diff --git a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md index f16e0dc8e25a9..9144b7cb623cf 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/deployment-types/hybrid/docker/setup.md @@ -1,7 +1,125 @@ --- title: Docker agent setup sidebar_position: 100 -unlisted: true --- -{/* TODO copy from https://docs.dagster.io/dagster-plus/deployment/agents/docker/configuring-running-docker-agent */} \ No newline at end of file +:::note +This guide is applicable to Dagster+. +::: + +In this guide, you'll configure and run a Docker agent. Docker agents are used to launch your code in Docker containers. + +## Prerequisites + +To complete the steps in this guide, you'll need: + +{/* - **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/dagster-plus/account/managing-users) for more info. */} +- **Permissions in Dagster+ that allow you to manage agent tokens**. Refer to the [User permissions documentation](/todo) for more info. +- **To have Docker installed** +- **Access to a container registry to which you can push images with Dagster code.** Additionally, your Docker agent must have the permissions required to pull images from the registry. + + This can be: + + - A self-hosted registry, + - A public registry such as [DockerHub](https://hub.docker.com/), or + - A managed offering such as [Amazon ECR](https://aws.amazon.com/ecr/), [Azure CR](https://azure.microsoft.com/en-us/services/container-registry/#overview), or [Google CR](https://cloud.google.com/container-registry) + +## Step 1: Generate a Dagster+ agent token + +In this step, you'll generate a token for the Dagster+ agent. The Dagster+ agent will use this to authenticate to the agent API. + +1. Sign in to your Dagster+ instance. +2. Click the **user menu (your icon) > Organization Settings**. +3. In the **Organization Settings** page, click the **Tokens** tab. +4. Click the **+ Create agent token** button. +5. After the token has been created, click **Reveal token**. + +Keep the token somewhere handy - you'll need it to complete the setup. + +## Step 2: Create a Docker agent + +1. Create a Docker network for your agent: + + ```shell + docker network create dagster_cloud_agent + ``` + +2. Create a `dagster.yaml` file: + + ```yaml + instance_class: + module: dagster_cloud.instance + class: DagsterCloudAgentInstance + + dagster_cloud_api: + agent_token: + branch_deployments: true # enables branch deployments + deployment: prod + + user_code_launcher: + module: dagster_cloud.workspace.docker + class: DockerUserCodeLauncher + config: + networks: + - dagster_cloud_agent + ``` + +3. In the file, fill in the following: + + - `agent_token` - Add the agent token you created in [Step 1](#step-1-generate-a-dagster-agent-token) + - `deployment` - Enter the deployment associated with this instance of the agent. + + In the above example, we specified `prod` as the deployment. This is present when Dagster+ organizations are first created. + +4. Save the file. + +## Step 3: Start the agent + +Next, you'll start the agent as a container. Run the following command in the same folder as your `dagster.yaml` file: + +```shell +docker run \ + --network=dagster_cloud_agent \ + --volume $PWD/dagster.yaml:/opt/dagster/app/dagster.yaml:ro \ + --volume /var/run/docker.sock:/var/run/docker.sock \ + -it docker.io/dagster/dagster-cloud-agent:latest \ + dagster-cloud agent run /opt/dagster/app +``` + +This command: + +- Starts the agent with your local `dagster.yaml` mounted as a volume +- Starts the system Docker socket mounted as a volume, allowing the agent to launch containers. + +To view the agent in Dagster+, click the Dagster icon in the top left to navigate to the **Status** page and click the **Agents** tab. You should see the agent running in the **Agent statuses** section: + +![Instance Status](/images/dagster-cloud/agents/dagster-cloud-instance-status.png) + +## Credential Helpers + +If your images are stored in a private registry, configuring a [Docker credentials helper](https://docs.docker.com/engine/reference/commandline/login/#credential-helpers) allows the agent to log in to your registry. The agent image comes with several popular credentials helpers preinstalled: + +- [docker-credential-ecr-login](https://github.com/awslabs/amazon-ecr-credential-helper) +- [docker-credential-gcr](https://github.com/GoogleCloudPlatform/docker-credential-gcr) + +These credential helpers generally are configured in `~/.docker.config.json`. To use one, make sure you mount that file as a volume when you start your agent: + +```shell + docker run \ + --network=dagster_cloud_agent \ + --volume $PWD/dagster.yaml:/opt/dagster/app/dagster.yaml:ro \ + --volume /var/run/docker.sock:/var/run/docker.sock \ + --volume ~/.docker/config.json:/root/.docker/config.json:ro \ + -it docker.io/dagster/dagster-cloud-agent:latest \ + dagster-cloud agent run /opt/dagster/app +``` + +## Next steps + +Now that you've got your agent running, what's next? + +{/* - **If you're getting Dagster+ set up**, the next step is to [add a code location](/dagster-plus/managing-deployments/code-locations) using the agent. */} +- **If you're getting Dagster+ set up**, the next step is to [add a code location](/todo) using the agent. + +{/* - **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/dagster-plus/managing-deployments/code-locations) guide for more info. */} +- **If you're ready to load your Dagster code**, refer to the [Adding Code to Dagster+](/todo) guide for more info. diff --git a/docs/docs-beta/static/images/dagster-cloud/agents/dagster-cloud-instance-status.png b/docs/docs-beta/static/images/dagster-cloud/agents/dagster-cloud-instance-status.png new file mode 100644 index 0000000000000000000000000000000000000000..507a67f8f8ad9965e328d217e84e38fc14893f3b GIT binary patch literal 154835 zcmbrl1yr2NvM5Xl!DaB^EJ-CbSXWnEQ0gek~LAi?9pLqI?vNlA(-K|sK}LO{HQfQ1IPJZWYhLO>u| znTv=hNQsCLD>&Gim|Gb`KuCrqs>5g~4`F3$$Hm4@K%&N?d7$U1e;}!Yg;aWnM*>Hl z@E$$D4{0Pfz!F=rY`-Klx0IGRB3z@2J{-h|rIO&Rh%Kh*j*d3rtbf0H-`;$+_apT+ z;m0!1TR#&bbHJ*lnkz*KZ4rPRDah&Pme$X=b24Dyr)ef{EZs70RFc2y{bC>g2X|AFhc7+@WNrl-ZX(YkKzquMS;i_wq+ud&+Y639DOqI15_%m z%TTPAcOHCNW;Kfk3*p&n-MwNaiRq^b1@?J;dk-bkOspW3&5FZI_MOf`l}cJVg7KZ_=)>v`w_v zU%u_ zVIsVc3qeIWI==#96^u{k9sN0(^!lc#h7h8a$NE0RWoRVw*)R;ksqc@HxFqikl^QRgL85P!^)E=VM=^V75m6Wtm>iZbTLb8uhr7qHCqG=xDED%o!FwoFx?T4frJa-d{P0wduGYd}L zbG_~mV-;IN`S+g)D%=3;+QR6ktZ#qO*FrRWBvN08dlCe|zB3d;l!W;38NRFrP~rF? z<&BBIdp1;pKjvrpd58wTlg}LUFvtNO6Hsz~EbEX>h!ugYb~jD|$=NV%Z=YcUVgb0q z5{LlhUU+*!9aLL#2sp9aSWfbHQX}Udg|J^nBqJO)n zCzD^XA>t22$KEz3cC5hsFE}X}>4Aw~b~G^6BG|fL?`dZJoehcXNaY4no5*y!P~Aw! zdJ4B3&+~sldvWN6WOY|=qFfL@A^3}bB;^En!Qpp92~!$C=L9Lp%}a60a3b9)QEd=> zlQaTYhw9le7658PS;C~l(nBA^Mtfa04Zkpki82# 2QhcKz5JT9KS3- zSun5g^Z_n`ZkYGeyZj`FG>4c=HC^2mofQf1wiZc{21xkX??Lk>g)Fpn?$#(aV;gOwB4E2_@hqvOre$?~la^y0O?|Z4%Jkt(fn}@NfZ3h}2ZM>6H8RyK92$scSji+=?g$2@dH-8P_bEbeovW=*(8VI6Z&8 z{8dE|GIW2HiQRjy$eqZ*NW91xvPg+4aaeIW97UeZ;n2GWvYWu`cMtgwk2jS^zO0{d zgK&-T%2-AKv;iQg}tc0sP7<1 zz)K*CoQ}UJfhv9pb=CaM0n0Yni>*ey?4)L)w|j*|kO8&i?cytwdw=%jWS$ z##UR;eMLm{h(_`~LllFp&WWyjMPkL=vYCy6jpGXLvR*4s6T&h_>)K_uXXKT}_T=Wp z!qkZJq2|%Tj`2a_ZvEs@L1Tq$*`3l^Ek9J&=d5OaS$-lfKJP0pl!uptz5V2q#(}f@ z_vuUlOg#O(w;XqS54Kllu%Dsx-dQ2-!?Gf5L#w^xhp&O%ggHU<=&vy}uD-3&d&dJK z9f$(cVft8OTAMCX6?-20GqyniIMF+Sl!Ma!qldZ2qT8dJuiG@3LjpibN~R{hy#3zM zEyF@?T%)qOb|5o6PrP5kCe#bVn1~|E0zV`gkH_Yw$#YU|a&!OTJ)kE{lfY46meR%J zxN<9O%WyzF+81{Zjf1v=Et61y%}4&JHE2C(2hJ7U8hs~>UMfv0KCUEwl`4e1^3!R` zN007fIBX?EWzi2v#d*_t)4*a7x(HPTaGq<(P-!R+FW#CGTZW&b&+gM$!gpo1LWe@a zPh@!%g(p&vy-^!#_MTH*2Y6GSZdR+@tI|FIU!)G{bw&*ex1^1%YML$g0^i-k9}EWB zRXX$ZUn1B;^-RT#tcN_Z2;xDsp2SpKroI(gHd?2s)zO^VUKVLXbphrlXfRS?d4?@# z*Y#7LdEb5lnMK((XJZQyP;Rhy8ZL>pQLf%z(q289kdJbzWa|<$G-;UWH|3pcDr!v1 zvs|RMrP9~_Fd0q1v17NqZQvTrxaOjFq~B%8SklDTYx5*D*1K(6+vOS8XM3%DH-vzT zjmt=<_v=^4Ig!OD)=zm99N72ti#6}{>9m#`daZ{ZDD*$w=kJYVj0}(P)9=vpy?!5w z!4yEI8_}3;Sa!VM#q7o0#N<$eu2=?r@>HEH87Lhnuh8z%QEu6-tXH1Pq;+LntBJ0W zYp?iDT9Ru3+Ihe{60io?H0j~!H2Z#^?5Dt6v)#09so^qnJi*$cH`eC2!L$a|(|c*_ zCOWI$yKLS*oyfbApO`%lJr*7UgLy_=d{!w|XR3K@?usoc4-fgoJ(K3L*)G#6K=HSF ztJpP_IkVL+OFsQCHE-Abo;oA4rhMA%KVE(wAvqJN@6dRzx%l{OXHKA`t}&;~SaxP6 zX>KOKd@CvaU4LssALz3(V>GMcu5z}2X_#P$5QBuz$i2EmXMW26&3)5;WICypiYlYc z#p0>BVd`7qcqSZ=V@=|W$5YwsLV;uZUUbGCzrJtRmGFvKOhkz9)vpt~bW`mWib`pu zWN!wJh8S}J7mD-Yuu>BBah)xlRy)22kH@H9yha+ZfyR1K6+o1hq8y}qpO07 zs)`Xt%JzM)5AF+Vx|bohM0olO?GHQS*#eBWGW!Q!^xk~uCI{Mqjjb;Zuci4M4R>9w z%dXF7(vOG_F23h46Aw)HTeUAZS&~^a{7c?8PiXf$tAeMUenGT5$b|1c&QC-lLaZJ` z`Xxa^5M+{2qbIs{2?-_K%?UxhpsTH7!}Ov1Y0ZC#zBo34K<+frMQk>xSj8Rn=9C; zF3$kH7Ix|LARE1QCy46L^Q-+88%Xjklr{ZWU2`1A(vzw1hGsQ(-T2>}se4gvM&7%lMo_a_$ofT90|{XfW^HBT$nC~Q z`o{=vaQ$~PBPsD8L!2!6NHt^?h(&B2jEUJ9J~DhH<%cIGCgyc8GT~Md75@_){Ed&) z%*n})n~~Ag)s?}Og~8Url#z*xi;M9iGb1xIJ$MAYqq~ihfg8PzBiX+u`LB6IjU5dg z%?>)$K=7pSVEv4e=M zH5k%~|KH5|6Zk(n|0m!dI5q!`lj|evf1>=4p8tUSy$5b3V@F#n=if9`wJ~?%XXa)6 ztLFa+rTK3#ekN8hu0MeOv-N*MX#6F_f42Tl2zdu{FdYnjFO8q+PYnOr_UCwB#^1dE z4_Ww^p#9MbmNb5NUdI2HT7LL)!!Kaf6oilx6;g3~a}0#>Qt89$M}dQb14*B$3M!Q3 z27tcYEp#yi1cZ{3-Ubu99D|Yz$MvcL!dX%2{H|Adf|7#Tu4Ge2S4LNoR;;Aj^#8y~I8ndNF{nkHGWdTgP!Mnm%R`3$SQPxA&_cn|1XstRf%%U`0WG^J ze&8Rg1Oyy8;7vFWx|wMe%|BKNazT5ln13kAzxORk42j`NIWhKG^&jdKm;>2#sIh_{ zAqjYH&dIB{2zXpLu1rSLHZ_C4X|ho5jjst@YYoEvQzH9kLk+W9>Ig8J(NI#WLY4W| z+IDpPtX6Ihtu2Dk^F4th`&E}ap1a#~^A9(WT?fc+s;xUdAYOtKD!eb27?XSld&6;Jbf3JYOp%6>yhEQpJV6%tu_Dut}1iHoL8p@~f z3$Ptwjd>y;q*W9Jwii)fUQ;N21ZHMZ?hG4CHeWkQsVY)M{b6IWKli^Pl>Qz8FsE_QXK_Ft*WyBX z=Q}B~zvYC#lBnOwXug@LqcWJ7@Ti|X?cboSqKL*lyMd)<-Hl<=(2&Smg4^XV6Jtpu z4dk|&tCb`C4$~!R?&A?+btVQ39jz23d2AIF@$quAW1*3;vEp`Wm1h{{-{KC%degum z5kYMk<7f%r%G!(MJ@s1?X)2f!J z$zeWXkL;m;F_xp_;fPx(dwz7WqRcg44 z5$*!8C!ginpCNi`LK^hr88Cx@-j7J(njjYDOOa6Z(j=Nz*J4lB`cZ8MG6vt-A0L>V z2H&CX#TVfinV5Xq8BVP8Ke6Vb;iSfLt|G3%V43jm%#^>9_-}cz z3UPD<%R{bUwowE2m5;)-2-!spOgK+Oi!HhvNCSbVR8=;ygyztxcBS=hXm_-^#?!TZ zeTrVT%AG=WU9(t;Vym2G{??J}5A74Yh64+Sa)a#};d5jv+5ZoX^!u%m5UFBdjZDyh zq$mcZ+yxGzOo{S?nG)&sN~wYjw#Kp5G3^Ak+%x6cYRt?^0ZENw?=ru+U3xv&Q*Ga< z6Cr5H)sPFmzm1HHBvs|$;83rrtsO<#_{tOo=y7+yhtz69k&}}8Ha)H4KeAI-$9mPn z`lmAcb)K%Y8XwJ;zhmJP|9dw^N{~42c2^~eE_JpkCvS#NP|0#OcDh8BJm1UxL%iCo zm3wA`*x6FEv_qp+;xZB*h3AackY0n10~<5$9~eKO`NvJ@;SW7$mmNkhD=CxnQFb1ZIh@|qcpDqG_#s6zB#5JblHVe5#RAzr~pkfiB!hf!2 z#HD_~fgzyYS*(kzdvK1ZOI@H^T z&1$P~l?nZ3Qj3t=qTmm-rdh`mYwc*1*D}3Z{+nZ>trnsRS!X^v7+s4Sm5X)e9`m&2 zdcbnXzT6NqBJOSBR5nM4`5nq(_zjxfjYRYc-;0QX#JNoMUz6#kti{ZXztxT+wusLz zE|Ka;_V2i)zzfxFZ5y0CcFETwX|g9rO0yppUL0#WcgAA8w|#KEi-Vc{BM58)R-xjm zL7dBvZ`z#HQuQ49Mjp_HiJC$uHscrvT zp5+G#Hl2)OBA=0{j&bp=DVx>>0Y8_!=g*YV?z9qq88l^-Zf|&0au~E@0c*H0d92k* z-^MCRNl8(yi+EbX6ySK0eH(6)+KOYdDH%e#?l-g0>=cCN7aG^HQ*5M6#KTLQ&~wMg z#<)cIpDq!uJfC*3WkDYJKNk?emXT~91taWkjN2J^@Ru!rM*_Bfm1Rx4ljN++g?iZ~dK5SpIg4 z=~IXJ#HwTbvdrB0@A+(jHk=!hMfM_6wzHxRa}~+nJT=Qo_Jn_s@}0ktIV19(Pi&@A ze<(L?(kQk*)?W0W!Z*``V2`y@$nj;g>vqHQKrOC@KyV_urr@fnlG|=iwNEfs7qWEw0=el;givO0_hg?M-6>uXJlojS|8J zJx$kq0*&{74Cxt@A3`mYgpeMKrd*#aH4a#_tkYu%k6-SL43s0&fzMdtYFL#b1wCeF zW^#I@7=&t%{gL?NWBe@;C#|Jvr2l?TfnF_#?JEA3gC_ppoBt+s*m_X2*WH-XMT*{O zD#o1-=Cum{3Axl{!!A%6R1evw%X;Pm`#=F zwnQDh$!WmtI0)#c+n=CWp~KaMGMC@hHh**2b(`CEe0;3bwAVcJw|W~UN<4kL+XT-; z&_d;nvYwD^qyNdB+Crf>szLS}5H?IOw+=hRL!juJU$-0r_qQXHqxV+t z$G1o>$XmP2?3+m@stxa&;^*5a)R~#Dh%g@YoxavtEh{2htrjbQOvdg-z}~ZDIwb)D zeMEm`G|*$PEH(c7H`VD&YJGl|B5G<{TG||tVYegI;robH*m}#kq*?35Prsca(hB#h z-JuT0yK!IJMUyidz^0X4E?%TBj2K|8j}(yrl-FkMA}p=d3Df4U4aa!BC_73;^N|sGAxHh zkDH?r&>bH!y;hx_uIW{Qti#=k>o5_ENfQGj63de7K5=7In{GW2!!;@R8TfuF#Sg?o z!SCE%pRqaC%Soe&@Hg^7EejPO#wJ~^BSnL6Z`MsFMBw^*AG2+_dEUXH#=J`K8RY zO{7Mm#T?)!K7}T>Z_)UHk55)<1e(9z4A02YsESy(Sop)Mn zspW2;q;btYc?j>zobgD~P{93_@p$HnQllQxeW1UyosX>5F7i5I711YXutcO{2NAZ&S^k7g5-=(TYQF+Lprf;_>xx@mOSf!_+s z*|xGCT8-;L5fC0uq}^K>ghTg<%gBHXVat5StyHe%%(WdW!dtsLo@KgN#>?U*j(oJB zoFS*@^WvWCBeHrFU*bjuTxvAvflg(!G`YYUTsD*==Ht^HhBmilSQE)2L1V&iWuz}B zJ8bcOifNaQ`_l3$k>+Q+H;nS!=z|XXW0r4R?rIlJYOdaOFbYx5B->I-XxeviK7~Uv z3uwdso&0z($C@Yy9`tO@LmUK($_Sa_`xNG6Jy)`Z;tcRM^NC=~1f`)J@~fgZf{i3p zj+U2srKBTb_G&ebb4B4t^(u74J7xPrvh4bR0PD>?R=1Vin z<0xE?Bt?`cCU*wS8c`b}F5x74-NQuha;^HEU4uqg(d@U-dQ_!q<#Ab5a_66V{>X4q z+dwuIC@*U+C!`Aeg$^gkK_cO@vaSh=EIfe4zyju%>(1j@2Lc7xlxP?PuA{`Nj|2$} z>@U6Nvku4aD)6;ZP>>S%?sG8Qt;_arMQs!FagcV zTG78JX;4f7RWFUbWn2+mZMJ&eG2l$RTBvi(9?ez8FNq^2W9$2on?!D{@V7Q8l0%_v~z2KZq zf2yigXpmUWmPOl&b<=?|^rmpA$z{xBgroB6Fj7!*es#mK#ymuF-dk4*JH3lJ_Hl7@Y?nSQ zeX~#|bCc${&Us{PEXj4vM>?AC)NSEgeb1N+K|y~0kPwVBE6K{Y&qJGiHFg`23z%O} zsv;v1%@;}%rz`c>yvjsuM>JVX;e%hwL_z}dbi}zdTa1bYBg10^%e5M3&k2BF)ug}N z!Om)}SZp})|5>F-MwT*)a=VrInK(IdL4GJ71)n+cwjpEj=r4Rq}tQ0RDFov`5pZ{?hCcscRTcO=g^@ zcCz?0Tj(t#32J+Fw*MK>b(LYPu0VIIH)$bo_;Y634!F0j-k{rVP5(*~qC&;xgS@T_Gt^Fd z96EB(X7=)2)ePRdJ(bjUmqaQ7SbjY93L5%SwPfYA`XmOJ>15Ou&ylyvltrDj`_>di zEPX)2ANE8-OV<;4wy|o9{QS17ZB6bJkvy=|MGwjZ%on`51= zE!Paz(?!;Uc$FNd#4_XlQz zFZam|jmM6Cw{w^b^R?+bp zk`*25w)W>w);}v922RL&xZADn@#6&T(f4PLsGsLGq)_^?H?cwY@3*;K9D18^QREvCfdIjeasb2BMvS$Yb9i z6OuTf$^f-gwUpufmctO5FV-D`VnZ@2jcunFUH)>~>uPt@;f&7}q5X`fT(xx8;mrF^ zI*~^8$FcHbh6&W{QX|W5OuRB_RHaU{64(o&0Q(s%Gc|-9lc!s(oX>U#qE;SBU(k2i8xUn=6 zZPatHJ8AhTlWY4LLWV!_Lt z({zD@_N#5aQH^6@qn@5wssWAFh}_ZV{LFLKqK$my-ekJ)1%yhk(0Fx&H=X(vwQh+U2qO;|k$7QeN)h_*Zm*RImBDmRdZPV`C z=~DG^I-Zwb=uY!GHZv3Kmfo}VI2$6HNFu?X+81_+&y z<32E|g>F}MI$ITKe|Bk04PFPViIkTs*g77~y8CAJ)_~PPswX(giWxn3UY!4O)X-mC zY_Y+)R4T#_NWkM9SF^Ye5fhA8x50=JXxddEoBnBwQkFuy@$H%@>MV~Ylzj-n>=fB< z?=Ao1k15Cy#9JON;o1vr4mX| zdw!?WZ4o8*aXgxctb>{67wlMbykqj^;~|PQXTxf%~*oF=#zHHrC#bnzfkX(TsHH}4R);|J^CF_8FfsP>9tM5yYGX% z+uWxp?=J?3tS*E?z+uAF?RT#ba>+K6Q47ubl1`rQqr}V4;g5=2;!!7um*WDy>9-|M zSGixgIP7dSi!wm~V@8Y~ca098x8~DL-=>QcMcA#E^X7^$D+bCsp6IU9JSw$g*g5%Z z?aPs7CT3?_obigi1+9@pT9EJ7ZrQI!1@!M(@#2@7(k_v*mO_b9qCQ zd}n*5>Z1x-K)WpQYe_es0sA(g@i%ontt)5MQAr#8Am4S|9oAN>@k~%=gV%B+E{9Dn zDrI`=iB{g8d(i_7ZwR~P>~fI5ezQe>Ge7t?X9^?ClC}Z+HfEL#_p#fk-)C*Kv(ax8H|1Ce#6GRkZah_l8;x$+q7cBnZ4Yke6!PrJ z<=8fq=lxLGei`R=dtiE4@1U-+K;$(@;xIMZ^Me~!>2h*HenN*p82))4q%fIyz5;pF#RaajQDYZxs$&S>-^B7CH^z=IbSR1QxpnAn z<7^t-uRapRzItEh@LpYWjt4U9wb;IQ_>MX*wC>&%uILNDwX>=WAt9Na0-K%lhSAFR z`M&V}y^>5HhGj!n#YvwQC(Hlk;++BlH3!ob5;h2Kme|N$u)|PGq{&PlC0k<_G?3%4 z;ykxx_saRY}|Kz6?Hg3nrg$c}{;KKhU$W>ibW@h7zB)ZMjSpoelk9`Y9JyMwI zQKRtK_2E!qgAR%B)bjb7r>UNV1Cf36GY=9Ie6AXSPcN$80DO{2Hk0{Ja(A$)(&Zyr zlKK4;if7~#cs1)WKEfs@^Mqn zA7N|_i(c!2@-E}m4$=Pb!@EpLTLLDfzTO$xhoN}i_}0_5?-MQcs)I*uKD^&aP3n?4 z={4^dg$uOpsTyG9Bmf@s<^X%GKv={j_@0-p`=jl(+s?P3-cB2m1^Y2wp3m7HxAb0P zT}zjRE&6LpUF|IT+U<_MLK@*thJ-%7p>?s)o68D)K%i=*_Z{Dz`ym2j0jDicZTfjp zcd5?qSgZB;r;n2I*DW%e))n-XF$R&=PBPqQ80K# z{_`szPJ>d~ZP%ppK|OCPS28K2lLkvG+v)5eRWJgZ?l&VIKaKZz9@=jc95|M`B)RXM zG|D*f__~WExIY}24puJ`0@Du5IXdgT7wd{W=-Mj?XkD62{2TUP=8##(t=i$(trw|@ z%pWhu(WxF7&_)28Q?v{nKP;Pd8(dCsaa^us9fpg&f82++Xo;+}Kb~sWdb?zvRe>+q ze#(>Cxg7kQskM!T4w!0O^HFYpywz@gNP8)uN4=_SxxDlnSCbT>U4U9_FsF6g9fPwZ zi>Q&)cxSfxd2ZM43W37Aig`*IesxlKsIthM((C3ctJO~(IsQAZ;_pBv4?@p?r|W!I zsua7cXtkAynzM+S=(F2CEdhm=1lRUD>G^9tOtsVbYIzrvodap1mwwfX3{ut7d^6ZR z3~ztwNtmrou4aNcBNy@J}cO z+Uf)@ei;%PP?>BQg5v<9B-Oo3fSZqZZA1 zIVd~;KN~VwU^<@3>~YzKs{Pe$U+XbYqp--O zI9!Jim#<%SK8$yx84yhijo4ICBqnMNGFt5j_1kr7fs`PW>$%zQe2dVQ)c$MpDzgtj z)-_c)MU3pcct%6$j2mPK7;b^%`G%|)%h>)4t$A~}G^4aRdaF&yh}&%qVUBo^$o&yW zh&l}Rx{Z3v8{vazbDQZ{+TDMpi>E$U5)gwt?q;_Jwdcc}*JUwA9`8D4*$}2rgq$IvB*ezNZN=6(CHSacaID9!1zuftl`p#{dSM? z(b4@>CcbEEvf|OULpDzOQIAet&p8U;`FgT{iqeB`3r1wT=yw>-Kf7vYvoV39T$$$9 z+^{L`yS(asXMJMq)!=S!vz%Am=~z(T@%kbS4%elV=#un)Y6DJtvo_wo>DFw*@n~C$ zrQXW)=(gpI=U8)F2{M-;eDV1FbiU$hBbE|_zF+O^@ywF1dK3^#ToJ|<5A{Vd8{~7` zjleXvw}6{Qp-Deq&!^cKaRa`x`rjdsBq1bB$pB|zxqinO7tW2nTV^0N^y=*Hk4$<1 z524pN3E`yB$zuIZ+RO9K%j4+{&0{!i#Wn0_VmiT;BY~U855i7cJa#BIgz$mufYrry zBzn{n$*9$DG~?hNTgNOY zhTd;encL9OK-QA=?+!n=7i? zv2lKirEd7#1&O5Dss}>eMMxwe_q=V!-Zs3^xj$m;_;tg+U*_A?1}MSH)G6Mbvn%KnUtzB)O@IW$sm+jCJ$*s_>6$c);MLc=W zbvdoHI7ba4fb-!1iZy-*(Hujrv`6Q`*qq%o*Z~I4yU91%--G)a=A@Pdf>)j5mp{72 z&M|pheke>CbT7z#K*`K2EUCo{`W_NuRQFa`2X13|w8usfWnc=__K{RyXK%fFViE18 z9TtYuj`ZzhTlrgV%(AJ+f~iurOM(Kug7Wv+1AS$wEIW5NQ~8lEImoxT(T-Q7n7i#y zKO-87z7RlL#!HrKMt%&ZjLAZ#7(-bw*iuIfRAKfww?H*3+1n{f<>-%#d-#N?4DwXUhAK3=Jeyz z<4f-?R<8B@9x-@xiF~`mF7HjV(&}!BczsBH-{Irae&1?JCP4&E{SFhZBK?jgd*K9% zIXv7{Rx0*ZyZQ}J@Avf0=6U9}+zu%09yCo;qZ)HTiYUuO0O%mFGZ^N)2PH-Hv6dJi zP&YlWOfRpmYnTp?Y#dnh#2%I1YH_b*zF1Q~ZJF{)sF)YE*rCLFNI1SlG2b2*6t&a> zM|-N%)ce?drgpC}W(mFurHeplEd%m^`VyPDn=MT<6BsGwfn)F{#Z{~+B(gJ`SgtN~ z%e8D8pj=R!Olmi%J;ONY@e5K6aHZ9iL%@{3tH1RSA%cxJTfC>c+>_eaX1=N`h}ics zY+zf`Te0u0Npgkm^_#j})|nV>@LiIr#~VJoSMS~Y(Nt4#&*-+!bISIy`+-cLecSow z7Azq%o~0RdVbc^xxg)N#AGXXif|{;A;u8+P%>xVXkF9s8w~hRvk1SR_dHTLLjq^WV zsr)#R8Sg$m3XRE}6T$VSb{0P{BTAc80*idBEO~xT{I!RX-2^M>w2nuK2br+R3d;lb z#@}~OYFRGb3c6oc51i!H-5dZ=gfM~vXNfr-!;>k^0$+4pSIGE~`KDphGYAWgxx<*R zk%!u+$h?-@y<3TY(DEvDwKyM{KxvFsRl492%hW*o!wIjPEvFKN`}9YyMNM3WQ4Qb- znI?_d<`6altl_afjNr|*VneBYdd&p#RZMMZU%03}X4ULN61FI`wIZKC$@NBV2O$>2eZ|7#PVi?Ugiw^_XB$m00 z#e;Uw5Pr(eStEh)*`6HW?GucJHDEx@X)%DI9x%N1{Jd1Ubl$SX%LbQw&A4#K_)$>Q z7<{)V4R+5}uuf1~kFRe-&o?*;3raaB$xe@*LcPvk9)l@Gv(!OCV1k zZjKD6=cl$ZnMvkPocfy(w>D-AnxDdHjT;*1g}e?;ifDUp8e{N_zZCD-NWqT=x^_Mi zz2AY0iVpR4@Y=bsS*CDEUJv-nqJwusu^FCO6+=ucT<2=3%Ty=4Kk)>4-rb_PX1dQVIBb6dSvk1BG@F9B@z-CRcU09qdC`P2p$b*&fZ(_eYSQDgrL~U*^BN-1zm_EJoY%$yew!3yTY|4omZfc^>>U zmflIywqd*LXg?f%lm}JLKf(pggMaGHoGa<}MEH1a#>m(Tu-JTXyrA%w<pN?Dy=;uoTuqu^+6F%Q| z&l-HHn3d~Tgz2nTN)pAFQ@`QddsE{&$Lw98Sw0IcXq=iQVRHF@Mc9+n&LV7q?P?#_ zUNVPYH2qSGeKr&sdjtzZ^g~z*oi4^mqg}seud!qdo3A4bA7P>u5k>oB5QW>}5EZ`k zio_)PXvJH5vpTxvmJnqILeh?(gv-A+-r#qS8piOf1_}A?jlhUU#gQz2M`3K4{YVt7 z=Xloss?W3J9I7{5)i0Ztl5-~DRVIXTG<$f11~NICE!{gnJxHIm=8jdL(t7knG%-Gz zz`mHVDGgyodAi6M$Xej;eW2W|PDMyu?zz`wkVu!kj9 zaWuvp(&*uspnQ-;U!%KNna1s9IzLv)VE29D1qV^XX)LAJ3OJUm~1JLXvg{|i!g)!;QLSuHg{i17oNA9JhZ1l?{iGYdv?-o z+Z!VIdLC|oaw3=TTp#@AIvmHjMpQ%5;G{!~C*(dO1yB`Mpz9Fj*ouD4cWN@vKkTJP{TIp*2}RVY{pG$ zzGF%iF|=xNmK%Q>Y!!Gl_#Cy;*}Y3}fCXO^jx_7NA8*A~V7o|e--?FL##XY0!n8UK zxk?7Oa~AMlU6jFmn;|nC80;}5y5WQpqPFwhl-qVZI@i8_e87!_W^-$5yK8^U&up;m znkcv)Btya5NH`mwT&)E3x4*8o>h)PI)=5<|8SYfHp56QqcnT%s-IU`@XujXg34Ryt zoT0BaZ?4{1KO4@&g!0%K&SQU8J!e3dem_8TdCFsJIBj?y{V=O-JLEj4Yr8A-8Q#Cy z*dwDsL@RJWh(rD-r@H4I|8Pc36N;;<&}$qNz%@xKqM~;z63aT@<=$&N=5t(T98DCB zdSZcUiPUrWy>;2y%VY8r`a65mx+9yrjgZ&p%A;qD?sdXL9oXQl<38-HJ6N8RmiIn) zCyfr5Lxs4eLgx)|LaNl(TkND1mu?qdv30)2Bg{y%2JN2jBcTA#=@5iIAUb4>q zm`n8>7kp)#oI#;1)VF=#o7KakZ;K3jRZeKIVpZ6d22{7AA9IZDkIO$B_fOatF0`FI z-q3s~{(x$0Q+G&MxkWJx{xx5iMB7M;P#%=sL|!s2-6reCJlQDk-~((k;UM^*VF=Bt zHG`{80b!dGSCs1*j>5|aPsCfmn$H&51+__$0W_f)h5G|Fk7po>3L;KHfF3m&nKJas z$6lKx9JOK{fCGd9CFHY}Z5A#Yyf6zf`VS)eCCU4JLq<5D&8c{maZ!a8e4 z;mYowl%hI)jTRg0ZO}qDb&`}6@hOxX7u%X8mEL<1_czg=Up42uC8dhbL0S?eX9v$lk5$<@B!5V}F}D4O@Nvbu3zF|yI~ z*sP`MW-WSF2D9-;YhOOJwSiQyMCf?un9?z*$+|5m0s|7la z8&qH*N`vMynW>9kc0_~l8xCWh=UuVQa11>1HoBMCNQrHhE_>7UZP*_nE{kAqkh~>g zxiJ)v@X-UF#?Z?O&l&4qIr3Ws*Hy_Mg!BCD926;?S;J9wEN5L zGTrzx>>scW!`XK=o&X`=!-v4@OE0s z82Z5noCKHblx<47+@ay`GC0ZEhj%Apzh9L}j@vAwc(4gJ3~_(>8BaTVd<3&vfxx^k z2zSLav<|ebI?*jq2~3-g&mJC4=Rv9GU_zR zY6Cvel6)4rT|#ANAQxqOdBYx{TeIfp+V;3yV82o52W|O@>9QQmtx&&tH3~H z*S=(-?W@z)D#%}mX%1f=0C31fdDnLHOD`S_1)+aQ-4>8~|9$`Si%OHuj~BqrZFLQ= zZ{!2HbAVfZ7w)1|B8veFy3B1KfNw;`MlcVuEJ5wSSNO&?kVp7WtK0caH>S$64^OI` zSEViAda6mJQ%zLlb z>4iSA)q}Po+812dyXf~R3=xHPy2c_pRGE}SsDO^0NU8!wce%l?UzQE3SaOSb{ zj^VlYu7K<7u1&G6*3oJ&nk~|qU?biqPDSG6XaZVNK~^IC-86eallrq~ft*}66D=*K zeF^NxpRb}tD>0PoUZZ2*RFReFRQ0-lXdZRQ>QNTCm~sTOgsuDH{C)?nEC1A`@Juql zDZ~ZG-U+zBs3&tM4Q z?%`$w_NeFd^n2-Gzu|HS-}KS!&w$nHdj5I7TDPV@4t-9iE|#wYx+7u}!s}&b0pX8%FIZZoI_9`dwxC+ZH&E?ppy#hDRQcg2l8h+9KJfrGi7? z2zp+YBQDq8mSfZsMJu(ELKGE4n!J7f?;*&3AdIDdX(-S_&T-FGWQNx3+~+wG%}@sZ z(+gmA%_j}~J0#E8!;F3FnC_fbJSSysqs+AUa}1fV$J46n_v!dI&(nxXDd5qJw<8~Y zH))sFWARa`&w@H>2^$!#?wYgGB;bN=6u538Q+f7N!g&U@QbFGq434vg-JTzYxvEeC)2_8bPyVni0e^L(-aAh^+<<& zcj$h1y7#4L=10h4A59Bixu)aAq6`%{M;G9T0nRR8@P+R)&9=|Q^Qhdkl#gfK)-*g< z!n<$?EVHu8v8AwASh}Jof3C4&Rvic8Gy{Z-!(ZUtPyWGzI^wATRH#jnB9DM`&G zn_7=PBR^KI_J2?|IftE2q-@cb#?A4bSQO|tR2S%kK6(870i4ROAHCrxZ|+_b!a$7D zp;{9*yZ*Bq4bJwoRj%z;O4bxOYHN0f;E-Vm&gn=H^5hxMLwKqAC!gImewK+WrR;~z%^Dar7A&qV78ki645gu)X+YUJ$Z6fxmQ++DZCKVm!ODM;8AJ$m?KEk zT_@jD6S~KW-o??D(aKYcnc*xFfYd^8b!)%;#h`qvSY4p@f|tc+t~OlzGlI=kp=Cda zK=~^`mh8nE6e&cmbXs__+=?C8*k8H1JHJY>ELhqZr%exC!!)o#5 zDkl;-{$R-EUY^0|N&J+b(#~`?tujPA8!XqstMCdG+NHRN&10{#$z?mZ;O)L-zyUJZ z-THPkZ3+FUg}-cGFISJLGLKCuT!FNCX1^Pxo*CeEV>u8J9X4Eit%6uXG99w-bp)#4 zUw3zz4}!joD)xt0s^u!YX;Y?q1QYhtZxBu(8COACX$2n<{XS$k3Xad8d<>j*UeA{3 zH+UvE&S)J3@Pc~H_^Ont2%;%)Tm8YxnO%kZGU6wMl~hZ@x95IoZ@p!Pb6oq{tEH&Nz{h|-~y;cIl8QP_1i6WHbM05`2D z{m{*i&BNOxqs^rx-ON>-twv$@Qo?mZzY(~ez<=v;a9Tcue^!~#g1WXCU1V(XV&qnw`ormuJs>B)$sdEMOyd$%Tw?I61u|@1+i*x0=r>JS3e3-YxJA0 z2v%lNi3UiMXHE{9Ba_m^jcpBRy_hl>3 zY+>M2^Kh!Jm#156UVf6(v(@&;>JPF8RI{bM!g=tkeJ6^N%GD|A?}=0zg~<0j&O&bw z&7Z{7AKa4syf9#JzPSYA#Q`Nv?lIgNT`hJESK3;4S=PMdC1Rn_0`-O1+gsH$$9SulkX4 zQ@g|Sy*fi+K1x%+HmtOYEz+(4I&VB&v%2wehAi!Pwq%p0ZVFrCZco*&Lbp1Msu$%~qI=J|Ls0AMWE6My&syL>;j84<#PByhq! zd(8}-QRB&C&Q^?&f#e`RZSL{KSt=U}{leK-i{Z1?#b}ddSqY!&NkTifJi<>8@vg&R zyjE`q!_Y;sH`igKJZc`hm2FB5ESf^5ZIPDK<*|^ZgVN{SlwL1HP(u|*v!rwdwzwKE ze;QbjQob|vv=52k!9uot#C&%uB!iX_zCBOSHXsW7IE01EZY%XBPjnP~=(_$;6~Rj+ z-=>y_@i@Cp>AT{#o`~N&kGtZ?wSx(!RB&g++AmW}AZ`7-pJVVVwE1#mKQ!kN4yoLI zB0%AQHuSNM?M84VzyK2)Eb!WcDIcg3z`))(R2YX3TubU9&B(L{p-ip>n++`6wVRVq zLZ$iweEFsfDMhjpjYhYE&RH0DzNcI_<_W&2;<(I)0=i!1hM-d}>7cDM{2?A6-cgSv z5y|$hjIPF%M70wZ0jF8! zm9^jb!|kR9U1oHV3)W}+Avn}hdBwz$6E#zVpel;jzzzB zZLuIH;W4S{9E-DSH(4bSgzXlB6Y{tul6{jI(AaRl`!X5MkosbkSkPMY5WC$R<1k{B zT18B2YlWNB3DE6QVT<#VLlX_$>O|F10>g3u-f8F;Cj6aKo`v^N|BE&fbl{+N^s#yT z2uDa?riNY*kdV|Pynof-Q>hMTwytj8!1Tbi+6}qx)iPC9!z1W{-pz-zU#3f*^IE)I zy{#G#J#YrJ9wP>5%B8F0adz3#QT@EnZuo2yp4Z%Q|3nN(MH0(ZSPTp}&nZK7CzfQ* z_U8E9MzANLx(Kd$S)oAq^(rpZ)2%HbauW!T&rajxLdkeV2ca`Z8JI7NTU>zrGN2$S z9OsksUnN`&$SN}|Q*1)g#|Lhc2BlA=D(Ue5tex2W=CU*O=4ki)kh|b{n{%Ydv3Ycu z|H06fZ5TW>|I1X>G4qcqg|b62}}`lN|!D6q(3 zOkeuLqGPHvQbtQ$G|SQiER`-;tuBo)RFEs>TIB3y!j$hLu!xUY4N$a6ZG^sG#5sn5 zk8$JuH2#1>(0leO0do?SSDytF{BEa1;bt>r+g?y>fo0|wr;zH+r^eso;>b!#&z9MY zvi`uL8R~#477~x#^U*~VkPW{9Qvx@1L5PVGx?piPhv?Y@5-A;b%k^~M%kZR*u9C?` z?>_Ib3bu`5wd&ZWAPDGe`e>yOiVl9$MsZa!D7ZUrODqmI=2N9VK0wL5XxnZJ7V{Hoh@W|Zcrfd*{T zI^a_uB3r8|p+N4^UuI$`r@DwWrg&?fXlVR>$uv4ecD#pu?(o5+K6{MdS#9It`s_uU zq0FOKcOZyg^HF<~lkXSF@;xE&_lp?e+n|$Xw@;>u(O*T40i{Id@7h)6A(BDzi_2|x zEC$&TK7gtBu3o!{`%i@uPw_l1a!6n{o)<)#&&pz5*7=Eu`RGVaH=OfjM6E|@2oGBy zQ<*SCnor3J)wLY{(dKtgS~f5jRX9GpyJcAPyMs(#XQBUYd>gicikgT=r6ZxQEV#_e zcN!n3=a(%LQ~qvXX!DeVqoNUPxs!qK048I2teDJsD2J6f-EJ!F<#Q~Yp(HjPTN=dM z!6>`C9u~b;gxZ9b{u53FJ`>cVJ~+l<-&Tlp=4my?A(Cz@`=iM-9biVPXS`piASD%~ zSqWm$%;S45fao&yPHM-HlpsnoO26PRT;5Nv+N12=xpZixMsb@#%!n=27(cVWZyWRy zi#P4()iC(Si6Z)7*~C&yD-WUIj3hw1kxltA^m-ZzbqP$(oP=C0Lq+af?c7zV=R=IrKu2^CchU1iZn; zqFuwL-8LXV%#iLKaY(ww=yBIQZJEA0zHS!0sI_B;nMdcK+A>q)xSx_^dFc+)STTz} z6J`E#{9*)uY`3f{?R9gRDHTyFdns_1wsW1yXk5rWdZ>S;w*3z zu07Lf^~(%q@KYmy3U$7Q*DhRTm^b)Fi>Co(Q42L zaJjWgZCzS=Nt}&K-$trYZx(%&zEIfYaVqu9?H(A2O>esFud9_^?bJuE2pgCbQyna> zWy_w|y=Osh>08sbG=v89^!iG~XH%}d?Unt>uQ=+`XS&wA*@#|cgC0xSz4y%1IHNPo zyeV_mC(pi(%I`c~OF2VX=q64*f2xJ;h!mc@DN^}Kv&|$G-OahGt=&rf0Sa8yq7z7P zTZg*&W`O1sq$>8({mu$Jc`3l7ul@5~e($;`(;hx}x|YEJ;X0Y2^qfT}h-igZNO-g~ z>L*44G%Ih;j!+47y5-y^hk0;XB7$p1!g{AYd)FpyIg)f8okrzgcrVE^eR2UTP{}i@ zBNj$3s388wWfG?9puio<>E;Z ziLAFF1?TxA9HGkq4EnF#Wa1$lxms_09Z9$(tq+HKB8c;Tr-ljMI=e90oFKKqj zNSx|%Uv!<^-^Xj?E1vh?HS4ga6RMz|@YyP05E^~9Su(p>`N23y$*n|_e09+e3Bk{s zY20tP6Z2G+I8!>`WKc%moT;+yd)|A{e^X>Pv*?>*CNuDw&;U3$oai8i+QGfo7icIP zl|(RVR_}kIZajf5-2#BFoQi{85JnKc#fgMLmY2tO z$|nqOnH++IkaHDJcQ?FVnx^{!e7y2RR-|^e$4mFZ?j@MtUA*m+t(QVu-0>6Rw!O_M z+>+{AMOzNc7!QQfvi~Mlp(;zKQulw>=)qr{37&(k!W$G)SR0W~n%CC9WI3<+`YSyt z?Hi9BOdc;Z?~EXV7b!L%6?t&uW~y*s@2-Y%)|vG@jkBn#w$^na5LrojGKhNH@9CE; zU>W`SeW*ljo&b)xqwQV2WMXAsmQc`RaBAR2rAm(6hbRRvSmV)=y!7Yja;6VIiw0>d z-;wz!<;04<#Y1G{tRnPBD`4jw5c~J>8cj(!23HNlvQP z80^(4CJ175^>WNxTv5i^GXKX@mU$T}VMY=?URtZLBb)K}r!;Qq!mp@PJ)big(%daq z!>UoT3b z=#u9aHbRrP9F^g(6jx&2sgw{98M{`sWERB+THkB}=v~^iMwxm!R0KMk@&s*#x{e7R z8y$iB&0sYTZK|W|;Bv9&9+>wJ zNt`eEbo+RXTAKMPs>xh~xW}*Ux70^nNMVOr#!mhszeB`6c7D@L!Yx}0C-Ht}U~(Vy zy64RYgT1A*caRHSG7du|wv8>`#~`tNmftx0IcKY{=avP$j!Z5z4Si{3D11JTjZjYA zU2V|yXT^SEe&AiocM-4FFTYBX?zffvQrq&^Rx+f>Tg&lJn19s%EA$6j%+T)#6B-8~ zfa~;3HS#zGfqiNMKz9DX@TDTIh{yMe?VHPOsDU`tY7%_Sv)IkJRW=%71jNl88gp-M zCJj*z2U|(Ogc7y{FA32%b{ai!4q<2A1~9?S)!Da#?PzbqBB}LwPlnnR-g=-0^3($k z6G^YT@n%8$3+*g`c)70ee1v&R%r-=n69qSRcZSF#r``8*C6ZmRQ>ypqc~lTuS^Ukq zu+@+e-S3X3sl=K=52wn-Jk<^8mPgBG=ab`mCJ4GNUM%LL!BiOM3K77-_v|#6&ga(_ z!WCyR52nk@uO&(AqaHySy|)uMCT8uwJg-rL=_^5=PB{lNIAcrujwo$)QMfm{zJEiG zO-qV?a4=hL9_dW|U^xj&Gq?LRYGSQY#e>F-y=_b<6bq4WDM8MIH!$(@gS^CqJ6B^W zqPPNEAlmVQMBlXx!l}eEQKFxn*+BxhM9i%v#oT;xw@yQ(27TGxv`oXgsQ3^BCcK!xpE#HBK=3QmYcl+$hujkOg(^xH9`e zYVwqAz#H}4|M4>$&$$O%5`c-?hl4r^*fqlw$`>m?wn9#&o z>(&0dho{Nm=wE;nA81zj5Pe!&XWLv#!@b8pX~eHa?`FQ zfCq%vr%X>Thx_;IK1Dye7sb}WC0=*}0p_=>$VKzb5qb+; zE#}-&lwP~yR2d=8nN)cJH#6(*vtPLjwUm_ry<_~LQ_)uMM_FpZdwEpmZ+wSjBlGqA z5{npN&3LSlH?QJ9a-1e1Dp*FuPIclF67GawZ-zRqML936a)FQ z_j>GF&lD?bSdBX&Scx7;tFW_>&=kYtmpEOzr}h}emXij3m}4O8Jr66B|_bVM^HJ&>UiyLuRY)H)-%eeTEZMvBuSnQoN!ai;=Q_d-HJ$<{ERiH_p}44x`oR%(6SvWe59r7{KWWNKX-y z6Cy?3=f^L46NlNmH}zHz<;>< z{e!W{ij)x$J<25zXjE#&n;@YL;92x;SIdbJzEshvv}0^qU??s5U2U&Kl%r{`N1)lhn^QedGxXDvL`jD!J7bxK2I|}O#K=7nfE&C z@i4U?5Ve)WP#%}!n;U6U5P3}dgjw6_ejpFW+lS*V$Cd}7Z|*hheVH9A^FKwM5QXh6 zA#Cx8#j&?+&l5hEE>AQtB9zX@m>;;*Ip@XAX_UVvSpzeFJ>Bt4;|AQ`s@+aPD2kNd|(NG2%QwbHI@B-Xp3n&)etWYMR zsCiqweBThk-Cls+q)+9w3r>VZ}cOAzSEszt4|{{=u1I@Gf) zE7mBopKNUkl2P{11d+nyuJ&ME9=4UnD>Szt0VK78pDc9Lhy+Ul;*8sNc3vUmg6Nc- zODaUb$to;+OG{*AXSb?8KOXEUj9lj<1Vx7?nTifJ0BQec4}=+DKuQ*op}sA51Y&RFSm;5aUyi;_T39Jp$dUeR{M^JV)6?@Ga9)ZOAi553e({cvYSQ&D5+Z;?yUm9xyG<;t)yOj4r37Q3 z$@AEP17~?;Dzxj_GJe}Kx6Rm8zYGHy9gfF^Zrl<%J2dRwc|`OvdIIVdLn&=#)Hl~) z8N;GoFDaDnK zgc1~n-iK(XOuSt|;~}Ay56LMlcDY#$8fAU24<9E1w>RGkoL~}hF-NUtu_2|0&_KM^ z7L_N07_>!ar;}chZ+x(VT9(#R9L8kbCwuORD$_N11b{SNQ-fdJ!0)0H@iotr&ddY2 zRz+!Ho8JIB3(vD8%YeN&;FhH1hG);$c|Z0upwcQBec5cXL|^E%t1M&{4T+(}iDWJ@oWvog=CQ7|m7JTL1 zWW_~ab|mGOgbQxl@u>#q^7^HQtnnfY!JG8+Aw!T6-s4wWynC$v9gv2*R92!G;rA!B z@5g9=Hx4dhST_)w&AI zK9fMqA6I{4W749CjnuXFKWXwS)Pq8fO=B=KB25nL0K*x2f`NWTNz3D) zgKFu4Mh>2cdl4Uwm4L0=&;`x-59gnn_A8`k3ga)Oi; zoVNI2g_KU<{p#!+vTk7ng9>GGR%5>QvpsHLm8vrJt9s!Rjc`x=+!AI^+$cs}fJ3Tv$NGjuLFq6{yunnG1*>D9cL;un%eDl9S1K0zCFy z5)@JBgGE(4Aa1jVPTG37)rWZQZ8?{ZNT3*QUVQ(HBewsDB=wit3Qp&+&T4TMT$(Hq z`40~aqx?0cf5S!%7gQn809e$jLHwM3S(M%LaeE^Rsh2~5m5|Hl60N2=t{4EM`yS#7 znex&!oO{ZZ$a7#)#+2S8+6l}%sar~#4}k5_IG}d%3grYLPj|z4GoszxZ|b_M^Y4wJJ^Y@N8>~{=~eXF z;2!v$RjGeLg!}I+tL$YdkQo})*oNwb(O3B1Dl(CKes6w(@kYdU#gWkMWk*_=J^fFO z!k2&*7eldbjRtg)Y%R9T)coSe+~Aw6ynI~K4clpdiqI9+Ag_!pwJQ3O4!I+io5&}v zkCEW^qMww41k);AX2fm*_KhbG!(o;L7y0s~^-*3n-@6 zyfxf`pg|FIe&ftA{@6W84HX``JcPL0lwsr|#8d|Cx1KU#SPv9LxY8x8xe##Q`-(}# zggFQ3b!w$#!lJC|Jdv78s)$u%Owf;e-%)OqT9Wf-x6V-M1Nfui5)yHdB^{;z)ia*j zUyEQ0u$i|m#2H&iJ?%4l=eVFgvNah6Iu&PP_gb^*x#T*WIgd^0pZg=Ci&(_aFVlp)08_pXihSDI0_t8z){^EQ&7X#G`(K}}K!)Mm zJJ*1v zlzrbF!YI?zd??QDw9mH)ZQCW@maP5FoAe)rnE(Ar7?;1wY>K=0aiDs`PFn}oV2M48 zK+}cr#FUl+3ckl?C&|5YC9q1_0LQOyl%bS1zD*zWYK{CqlOfIKBn+=6B1d0*d9~Y_ zxq!>P*Aoj)3V)tP0g`u%MBJA?D5nx)J8d@oA>3F{7cQv(HB=BQ3mzlvUH+v;$!tm4 z+nOU(Yjl#pQoYd)o@^2+TWg$f#>;3ZtOj$x)chqO`XNV-Xc~Z5@o_4U9H+CU_yZGj z*$h;IPr67+fjiUPJz;jo^d-{&ebir3m-@DYlYmU<;Aj1YmHpvkFps=~J zvb@}8jY+%AB$!19IA5LL3ecfM+3#wRVxYb(RTffw`QFR<6_6jjIx-84cx9U)&%#K( z6;w}O(W(0%+j5WM>m;Nv_LeW4FYtGUk`Sin5q92&XqM7%HFe{A+ho1MWoA^I_&Z=n zb{U{kwE(Cs2I7aGFoKb1yB7{UtNQL^B&WIeBX5AhWuvbA%-_<@t@>;%vN+UKFUDE_ z{=fdjOYOfu6cxk>npx0m(zPt`u!DMNjH{hR@4xJZ7ogD^4WdMU`SkRaS}t}r;!sxB z10{UQfPCvFJe3VA=)ZH8~NG+Tx9^k3>XP%z+-RmXl@ zLkXR$4p54}1-Qqx8d=#9(pP6>EP!E=yMA;MdWtYmcEF(TVK|phFhuLHfr`Yl*pXpw zWX3$4vq+;j`A6Pkcb`;r6lF{Sg3WGYeOYQ>)c0y?IX2_xUI)RYE`15bZ+44o-1iNp z8$DgFmnzKuEcW<6$@)%sfAfWQC-R^qF!W~#hD1|zsRC}WKVE$4{@WyeDz>EkJ_r)- zBnZ?&icfrsl+!I+E4s<-^=Z#&fYPH3GxVp2jk*izDp9L4OgIy`nal$<1a^URA9&*n z76{gpwkaw;B#E4{=E(xc@F?saGmksrM*@-?%O1g2GOvJI0@%%+5i;A?o2KzY-#9cmq%&K7Q|0ik?(Z2YyxS&V(L+|ZO&9`13 zr68}4FC#Lnss?EoxXoAle5!q4pWjahQ9*ROxmwkyRWRPvE3wY<0%xq@Xq6qnJ7=}* zDofA8WaBYV9pT6!O4IFD^2_jv!}Fi{1lWKW|GUW9u=_Rm@}$_7nkL^-=QpGq@QGjq zSbk3mA`a9aCpJC%)iyFRqOjqC{WtSxm1KNl`m6<7Jf<8ODU;MsT|}>6ec+taBSg;#Rdi|4F(1c z#nX{(Ekg~U<tF`p7o@f7N^krZgm=9aQwsSTo96%b{hXUR{#0r8{5I{{(yb? zVu}E(+6T4sBiPj~1qYQcEx%*gJJ-vpkUk-OuVi1JJ!Yu6|FChH)!B}DeRCl2=PH{h zE(myW?_#9xrO%hPL^Y9T`W0SR)x)e_Ec>pz)|aPA+FG1w(f|oHp97}B{6@q`=S=R7 z&|2&!8B3~q=`E}9Nm%PRnh2 zEgZpuBrIBZ);_9dt##*txc%JPV9sD3wbzw^&MsCQvS}O-5BWL)%rAF;!tW$IMMzEm zGlIXLq5pSCw+PN(H}ll!g1&7GH9NX)j55y@Pjk!^PteU+R0vmZ9^6hDMOG|kk_;qM zU@bz3u^5!YFIHc6T=R52%c;0Dcxj{XspUPY%`$QY_t$APT3lD@sp4oKDVQ6r$8I#Z2-^<% zSM8(fs4m?srfAxC3e<7}NMWOFba}GLh4d=h4_xzsX!!by*1C9JRrkXC9t+jXeeBAK z&exu+BwPn9@OBR82+Z>g2B{i~vO&44WRC?8Ta(__D>V{h)|;ot_yb*Q1aKg1A0WP&meL92#mL9r`kkn9I{TAOkMDbB z-H>@afy!#p+hwQN+}|9>iJQ}^=N06_C56bm85=DZ{|jgaX2ORD%{-GfgEsdN#jFBtCb<%Rbr(@EagRk8 zx0R<4{Rx};g~~VQ)TTccBGW~tU}Je7vq8k^@zOUenp@T#@RhSX+^^3$EPmg1i2@O_ zEO`XmFk4Es{@aNJM{992xp6AGQh8L!Z(glZ8U}ph53rz4_38?RioK`9e7==O_#=cs28NG;yu* zD}jwSV5#zZDcb{ z&jeiZlpwT1Sl|hjE*s1TDD)_dLwxK=ufw@HmmVc%K7a6~aDLA9C#O6fr|^~sivpZa z&iB~ms;gYht+PSgjRuE8DalQ+Wxmt;1t3PgF_bB>7qfS^J1eE*2+*#%gT25nh}7yP zw{0aFFvgl%R_qAj-IS3g!&y>g0K{pPQVS7x0QRCT+f|Mk!GFCUt&zOCfz|m8{|9Q{ zOf+7X!(WIRnWO(0el=>^v)_qzZP1u{Fn;HwB69Iy)sY<$IcFV(rb7yB{L9IO@bD9r z1hy}23Y(dKxm^BlEHGIp8(SVO$6p8I+6DNoc=zgdzq?axmZ|GAF;t5rk-XH*4kAsph|Ak$xiUmOHK3Jjszkt@`_h_x3 zSn&E^K~xQaztG1Tp9kMAfwsNurf{ZF?El!fWfrdA2) zXN&&v()HI*Y~E#YeWu|U&C2S&^A0BL*nIiE>==g}yhc+R5|uf2gb$h>@Y)md*JJ8mRF^p$>(y%OK< z$<&A1ijIzdwoPF<|7xnWf%%kw?8(1=lA#X73dY(=|Jm;XdU=VC2ygFN_F_KuuYdL* z4wEWOLrtz#mlJ!nH`N}u7YHn({n0$->!nbU-s0w~y$0u55yvRN`at;QXDX`QS@$p? z$?ETVsr1^GUhMtCKXZU%$v4 z-vjQA^PVz-S34DTK6KL9?0gpI+t4mXhII9*>S1ZJ&dIw)9?WeVl7%FGt~8I8!)HQZ z_0N=C{J%bXa@4zR69b+A5xzH^*GAR+J?^!F36rdKUzE=Y`&vd1J`l(wKkTC;qljC4?RR_a zaS|I!2Rl2uNk7YJ+OJSbKOf6G`#exHMeFC$bL_PbBSApClpJOyaD}zX*^5&@j)a%l zPK0>#9dteJoUw&M45I)K5cUNKcX8OYh^MRBQlP z@f%_MoOK>#^-KGcNMWA|MlUT}ubh-z6-1q7Is}U*p}6BQnfn}2K5#FHAX}GMMPH}x zFl2jms}LHoDMiyBf+$Kx6Yx74P47gAG`qL!_f@5})pSuIXIUf#a;J~4&UVQj-~G-j z+b(JNMvjDD?MKEb31*tvGI>N3moOpw?2 z>$vIcZZ$if!_n`a!z!qEftjugGvFK!V*8XLvYxVd4ra?pF}Q#nuny1~=U` zU9J3%lbiKf*GBvfD2;jo=k}gLYSr2Pvllym8)7TLIU}x`^}u?O#Hjd-*~RPn9wSSB z7PPf!!IjdN5-HZMzXLyq!@(x5l9hAwnG#N6JcLv=2RQ*30JWk59n9QRHj zU9+;0?K^GdRfoge%dCF|6i$h7feSMO(x0gfje;g?vw;xvvHV!)gBZUtvzFs>dA4Y; zy{3j`>)|G!t0|9FMU(Lln)a3KinG3o+`WjS1VT5(>|eP?*3S3J%ALkzw!?xFEI%A>MlE=8)r|2T= zzBk9ao=;~w@*_OBF9%=$7|Cd7SNdf!J#DsHn~w^Mc}r*_gHhwJGfV$tr&Kvb*)42B z!y!GGcuZ$mQYJkJz{=x}1YP@YXw;LR@!Jh!`q&yc4S-@UUpdmeSHeB6cAe=*zvtn< zcI?i5z|K|(Et5`lyWDs2Z@WGtyltQ+Q$4zO>2&Ok(z*a_bvre5?IE*gkrKF(C$ejg z7q1*XKYJrslY0AZ)aq`tx9}eI-;3_o`(4A#j_bEe$Ydx%3U<{O>%*B6p?U|78$*dJ zjV|zLLXCmMF*qmfI{$WjdYX{mT*QIgWO71Oa#AkTQOoV(yyQ^C@bbM~!>(IwK#8^4 zC}FBMpm@a!&2@n`ZS|gyLt|L&1m)x{hGzrl)xH~W2-pn+2 z=83>8r~fePQS2?!^Scp|tImDB_NMs~#5Q*CK-AU*l%1i23$DhL8MOw?RiA)sfh}^y zYw=FZ;f5gnTK$I%(>;XGxvt{WtzC<_&$~Dh*YBhmvU(b%2FJ12FI|7NFKCVtUY{)M z>vCsIO_augt^XA1@YcPv*;C)>B=pN#+e$GkUM!yc~rsyjtTMsrl;0*`|F?rxXeUXpd#6c!WD zT4d`PxE|)ziJG@>{rF_*oFwS^zCZO%wujnI#UP1%0L0o7bCv=;wOr@*Lsf88_G`m# zlMJ`e`N>z8M0fM^gXOaMv)Qgs4<5R-b`wSm9{x}>*|x9lnHSLKiR%3>?JRh5@QV)A zBxtwscK)&*oRc!=JsNcgl%t5BbBt5ui2^Fq(euoh|MX(oHd`JesIUhr)_{A;kc;ky z{Ye~oX0z$gYqg&IZ@{)115_3fAdysPg{!~6dW^g9`qtx@Hn4kN1|O>}9nG;E)C-!A zb;r`@+1GDc98M4V2uB1$Ha`)j%^@Q1;>coM197;!t16b?(39A-_Uf)K0LO}}E@?7* z!ufKyyrW}@Cj(KWI^^iPvo((7o+oc&?IQY0joTwMnH}ohFPIWN&KCb4>*r7R)4!eC zx>0Q$e#6HK7loG^-Z(9_NK>Q+xG&zd;;!;E^XHl^{_csBp8&v2uRKj71c#L>-Yza` z%LDMU=Hd61vXfQ&lqbVv{h|w5-eO&6-935>m>NB`Q`j!}@o6y+dEI}-JaBzw3>(Rz zI-k6ucb7G8Cr}dP)xUkHPS}aNdW66Fl)5DNpUE*j;oT*7UX8xZ@O~m$u)q1rGWgcd^HYm z-mBec!}_RJo4BL(j+s$%M!l3H>b)m$P_N9X=zq15>k2Xv^ya`4(@nJ;Q@>i@9!o>5J1ZM*1eR}fhiL_tJFK)Onm zt|&-Xs)Qy2(t8Olpdz415$PpFT9DoeB@m_e2%(20QbP-&2M9^_!*}-HW1Kzq_rB|l z^LPDWIAjDuo;mM%&+EQQdC=|jaN1GPp9sP!(QhursTykO$K*SVOJjqormFuq1gwvW z3KYx8GV3*ObbNaLzUSO!zP7g24M6M*jON9Ysk_{xD}WoGt2g=mi8GRuM#$UI*|M*H zzMIkJ?Vas4+uH0E6?g=n(ulagb?4E1@8ZiGFIu9dWfD;=XyePqxOHsOjf1(%y1>J3 z?%;@PDd)7Kp}P6f+ZNK@E-q!3m|q3@HMdw!cA$%ojhm?6xXc%Y-U$I^$~)*e5tEmKhb=q*K*&{xx#j-D=S*ZB8KHMzfKg^jcPaLK_xx#Sx%U*dorJ?r%tZuG1L?P zdyjqj<#)b(Tg!`F6F|dJH&-d-`Mw{}HcJI|tH<+o^h@Ke5aj;Id8nl-zpmmJvn5)V z*2@M5{b8LgZCrDf8lT8V%MMDVH_&qK0V~GQ8@q;Uke(d<#!_V`+5A>>0L@JWO6nZD z>2}cYU}xI{6kWTxcG9swQ(CCVjtNJfAu2^zp=HI7+L+Z1k%>n(T_M&C2p7_s6(e*yQpc_Y(j zX$MG42K|9w=&wP9C2HIWt=kTGD|2uWCOJp%iI32Jyab%cp}wFs)BGhgXA+FQN@S51 z@qx5POM#XjXmhRaqsd$_+A47daFK|O;10fL5wsEJXWjPcn*(8s2p?17Ft4$D>+ute zN+uwu&|)Zg;b2JLVDv#1!0@P;c$?n&hebLn*aXIUL*m|P;G6-k0{ZTXDT-+vzo-S$ zK2?nJ64jYhpMBxKIA_wcE)%CqTyWL1zs73SK@sjGww`fUD zbN{(6X$}{s?r_O5=b(Ot-My#V%eY3~Ehvj$%i_^bY*X1cw; zXJcyB+a%z+c|GpbYg3S7A?20k%97vTTlr^;&mWr}n18XBr2VKFNHUze$a}Ugx8_pAP&5z9F2YJs;qeR$~|YJ zrMngMes9WA$(+DO7QT^mUN!p0bxz2tb(l+U)&1?fWDOl`a0J&(=NT@u$`Zf%{=TXj z4a6EwLsY@HZ~Rq8SPItWJ;rk$@Fx;L)IY--bp#eaWApsGGccoUU^aJs^)>?{YcbIB zS)X4w20$O$05#W4-DRC93D38!l}GIukf(H2tZDm=)ffOCSWmZv0XK>*+P2!Ag8UucJg5XmF37vcD=vc@}a&1+UA)kN>?#G1hwl{L7)=`+ywob_ZK*4 zyZJTpFEZxi)YxMtJUXG-rV9#Q9Z@Z{N0!pQ1nXFP;9#{f-FU(jW#K)2>gcTO1WoWL zkiVMqYg?m=ZMPR+ZUCOL!~+Nb>(moZu*%<)n()S`CO_BDB`0cQ>0>A}{YtSWi=@0< zx~?yaCX)`_f%K&pI5A!#L6!2pt116=X~2Riho)BUo97cb6s-M_7Ikv~Mlf58rC+x^ z*feIsZZgLz?b_$M10mx*0kI+_H~%{_v! zyT<5-n}-N8>%f1%u7Bu1l&ks}xie~7=3MSIa<*L)zy?Xqy=B{rgEC+DW|+n1 zz{&EgK%DuM!8$Le@n#RujOPz>mDL?qUVAu)imp{!FQd-xVqJwM6kq@RX?9H<^&tA| z{*o>^)v{kfi8ie70T`vvt40e?#Fg}QJ^2ni7!CQPFlM)=oh0CgXwJ4U54de?FK!XV zx^{Ma7<++JAq2=q-`Cq4RE%z~b|-#46DR|ph_JxY>4n8HR7j4CLcq3tH7PEF+qGK6 zyl=l< zBS)@Nzz4boI3sYo75i6B+f1!HI{+)V%dCpquSH+IGK&^l-D;m1Y4Id9(>#e+q->ZY z4iEDyc1E#tsyT;4#?9Z-mla5sV4RQFG1L`&Z?clPhd**_KrT+TMhg?1Klng@kvxf_ z)x6}|Bo-GF%@yez0L~2ibOuNiZxO*P=0$J^JQrQv+lVD|IOO(9;QW|+rGy=N_QU6Y za&upaRorxU-R?VVzH(lhg!gSEz3CM}^ttHdWZ*MSd2q%krb=B~wLa5dIBEh&~ts;KDXSGtaMu5x+%@R3EpMtj}-Li(Lk zyva4ijcWmeK)Cdid*V>aB2R1^Ycr$i+I@Fr@5SE_f@$g2nxd3~Rhs*bVTkC26oBH$ z(5m%k0jr~d=x?&VeJL$e+dn_wYnJFcHFatTI)wdA)YniDnE1nIjV!IB_Xxl_EeeR) zzcsNc?7hxa^RjJak&Y}2+HtaF-EUE(&Z$ZZ<5Qn_I#hQXjP|P@0!g%70Ete{H|psA z$eYz5p*OcPM&qG$aT#pua)@~T0*)EisZCw9xp)0MNFIpz@UrSe-~)>04yDy%W47%J zbN^b%gzanP;EGBc9Ir|oy4>*Ej^pLA`>$>`ma`O9I{0Xx1yXh2w%(z2i<%y&c*m(z zELKK%!RGcr&~+(q`&_lQynXHHOnpemwfF|wPV4sYASOji_)cHc_< z-A1?Pq*gEfX#`}JlMr=+RaQp2$Lex~#3XV@3fpEv%)rhW{Ne6ZwUggPbUtHpO%iGi z&s+cVZIi}*c)tgb>&<^OHfIOuA zr5ZzU=q%JhqTAE<2&di>L*5$&IgvaYdtIqBj35Eg#+e>IXm6%L^@HfXfJrjrYqUhe zYEJO(8is#Bk%+9+ugup31C@S2mO2HbdKwQxlZ0cgDar$aW8?917pW(UfXTu)PEvJw zMc)%9>ze?=ZN2^nXzN5!OJkik(>Xw zwh5hrCO^GQG@Ou~N%PGyM^xPZv!efB{zo~F`7Brpqe7tXf@j1+sDV<)=aPHFSM_dH z=)!#NMECf;x!NX2oZ9lzFOc@?-s`gnoNtLtcW!XVS^rb#+}10RX5E~-lK;m#jZZGR zFZDxEJ1FZ$5FOFti6Suh+iRE7n-f=7<~1CNzZlf`SAnFV z?=AqeJ?jNy1YP|>(|2y(zdEnu;%N$e$Z5z_uD^*{{J8X9YHMM@;u~M_H;s@hBU<{| zx@i&*_;fO^#RDx%O;z08GbN~CU}or2Ulji#W405;70<670}Nc^m?NdmyI-}G*M3g} z!qM7uyW`(9?Hs5w=roQ~Prt;LeVhm5{rSf_fD7wfVf=@00A%2H#noBeoIi>iwh}S- zn}<360?=%azRI!F6dlaA)>qPo%1Y{S0>Aw$b4)3UetOVg(aS_E(^>Dkcy!$)Kj*%< zb{XD4y$tn|N3COVFvl||Wf$FobV#GHy&PH*1kk74$s%L&^JBP$o*8FvwE9Hg>&;yC z`j@hScE<)ir!fnqFI_ExkEjRNDfM>`!Umv|%cFcf_?xiC(Ngx|eIbhmn@gHo|4mf( zzqUjoKTxUtA=#>b^WhyaX}N$+GU`EelKmSy>VNLK7ybY@D{+k9@#+6%*Zns;bGQQd z5V*k2@c$1~vw!}af1b{*e-_)>fA-Gte@ekWrQn}ZaPps004NgwX>$JQ+5YJY{wW3j zl!AY*Z@|RtpHlGurxa8^aoQe`)1QsL$YZo9X&&{zuJ~fd0eJKaduE8Jh5!8g^AKY; z_7FQZr&oOc9(#MwCh$A>nMH9;nWWu7hTlT!?FUwedno`r$Qz*49l?r66AEX5X>$Z* zkCr|$G-tdI!Eqa#1v?0Xs_)fKyv52K7^4ab4~*Z~64IOR1H{|LT*nhCaedu!ydT*J~$%eE@JGUM%2Spg+L>D+BW9Y?`WxmSKG@Y zMdcO_%;YAD1;8g34|8;p@oQbgZVk*q4<=w~AY7^2;vRiO~07LCS55WJ-S*40(}z0`X8!Ad8ZSQ zF*%|=j6DMe!E3#$d|)a|quANN7@^tqsH%=5)A!v2r?9Odf)&6^5-V2PZ<~Dp#v`W! z(2@Zi@N^fWal7~b4Sw>!N%42z7oG*f%YUgIH~_>;S{w2@@t%ZQlRTWchD0F!akx;EXA-7k?Bz&aWC zvvQly-1J^&Vgbr1QDq(+(k|Z>hN_UlAiBdYU2hdFWItioezZC4JL3f6Jv(Ij> z2J%?y-Kv(E+9wK?b*UbM@FM~NYAyR(pNju~T9Y4?+jV;t9f!O2Mjohj0AB4ln{x3Q zzUYP2;CqZ&ZXI+?x}p7R1u^HEkILGEvqStF4Lao zWgy*vMS}RuJDMunO}Fzf(-LimTD-$B7Db<5mZ27lr=9nOGv}f+0Ol({-Lu~?OxdT3 zy*6H1h{+FEd#6;Bs7M}Ktk2KzezK;78y4(_A_zv8R#ykDYsTMVNpBFsZ3I!y2Qq9S zb|a2tC&0zx-ywN|D&Mo!)@~@#Niu5))Mp89Rebx||{B<{HH%E_F0)v!Z-u zD01az2I*zE4+#v`y0qIFGw--Yhe;HUh66MmZ`amm?YhyMqpCxC1PS%qMdC-|dcu<nbt6tSQvYm_!Ep-&cVdtW4qVkQ6;m>cnYp$G@f)-^*r~@e3-AN7K|R;b%+ln$CL#D6E6%oDdTn6|9t+gQ1b&UVc+%q z4T&GQBTGuy!=+8l*{oG^Y0#08ucPSH{w{VFu=4QV=j$*sb;I2oPxU>JD^j-kPh|o0 z_(&N81e}ns0ScHfKJdz@LJYtf9cYrbnszAax)5?C>_ACQH%&mbz zPeliZLXI1udE!$xgbCxb5ZkMShRR7>Kz0%j9ka=6w zUDOescw=IF_o#SpcV~>PV8uoO-gagFd32064?BD)3lcHz7q891CGG3HkaB8n(-tT# z(Yqu~OV0vbJ;Ge?!HU~oxJXr|UbnK3M75ishrHBx!Ccl~L>k*%5W2mFq>-_OwzhN3 zl^6Rouww>Igd37#c%Z^S%!U_$|&6cnh}!N+-4@JDVhx zRPDyn9@;(z=DUgqZ+Difbf~M6Lt<8V2 z3UWeO5^N;{QAt8%!gla;a(~c1K<#n7vZ_TL1?Ve#3y9s`Q+=3&!__7kpw#gZ+#Es< zZlnMtyS4EvnahNbqbc!iC_ek)5s66F+rDoA{{KOz)d zNybTL8%srk>&ITyZ((Y$bOg(BvIT*5apTrCBc3_$^2tzmv!&@~6K*|oi3Y4GY3sfe zX)kiLwl-!%LoNY{q6Sh5R4U)M{i4Oc#gkb|8o^GMx>IOgrCBI>Xoo#03lhtWl~rbo7= z)ep{#iIyQZHFmF4f?P3a3V{9ZsJS%4?2yB9l3>z4=G46z>4!ti1s|Et{JR7Hea}UE zS;^gtV;j?FxP^yr(Ow5HPOC{U`Ch%ybrs-*=@O4~AIYR#y_jqj4i;N0(29(BQ*H%E zgAA|T)G+;B1~B>zY2Hq33D{@>!ktso10-BDAGd0Xp(_2rQf!AQb(=lC){B~AiO#x0 z0()Dn+%%b^9!!<0cXm4=Ugw^((snt|n?&I$@7>f8VJ)u)3j1#NDS2+7^WeoihIvks ztwUK(vk&x(X{Aev#3-#H>dUnzYe$Ss*;}ATR7)xIz1DgaeVLs?Cwh`n$=v=@erA*t zV|VP^S6lkKE7zzSkIx+SPsxOYKstq?@E4Q>vqAAw=!v=(zjZu0a z)eV;t5k*XSTOY1iYLvi@iflf4I)z7sV7BlpT$U!}&x*-G&lCx%^+=YGVvH}*L*U-s0)65KG4C}ETDTX|v;9zN`O_dWYT8!?4yP@_zr~|AN@v1Qg zD=*jEIJ-`+23t%@SzMSc=%%Z1(`2N%=569kCD0Z|8fzwjaV`|$^EgDgNZ7^RM2T$o zgQ%X^%~T~g;C($2XpPyMK(j`VRQ{Y8(vW4uqLf4|(8t#W?j?&URQ(>jh>{m%PwI7Z z+Cw5?misGF=nozO&tXr}+IdH)6$!Nu_t#g42Y5=v3jhCNVk&^w5oMzvb(JB z>NP;lrn}n6K=gw0#{3FfMWhJ{%ul7|?}cq1F$;y=QHHJNVLRIj6p9mr17JrPS(yfB zp=DRv5W6@f-$@7JA-u&)vl+3)9}UoeMP=(Qa#|yGqe> zWiPwY0-u^eyWDo0(*sw=g^l^NQvNcw$q8y+KlFjCGo6^K?T#gWZEwqGt#(Z+>RX&nv_(&)O7Z)V9)`NAZ^b_dVWOgCynItbM`n!4B1( z$yI~*ws@mgYuc}xE*Pf=;fY_!XC7^xnG20}Pzd`JsYVDoSfYgFqPf#UGVfaj-5yV^ zC8dNZM5+yIX63|Jy?a{^sIe5IJ1A6)&SxQpw?D|GwlC!Yk7&amE;AxtegF%6}nTL9g zW%RWNNI!k#_RtsJ6$jhJa^7PTzc^jTv*VB)=EZE%xejAr00|4}W~lV?4JWqf!in{i znOTe{AQk(O(;>;^3x|;LJN!cb%L_ou^jfEF)<7H&F>mCpCdZ?#kE`RJfLd&(O`RDK z$aQwkKf65u7nSktL(0rXedYze<=n%3!Qa>DaC-RMgnh(Bk7tr})nCgOBVwY}xSejS zY9(t?{AapTHyW)w+uJrYTk?AS8`%Ott;$<83Z7G7-MBP&aCPWYzJr;o(@Umz=u=JGkxOpugp5^XXG#*KocE<(NPd!S9bHW z05Qs8mJTOdtV3u;N3SMHXsmbh--L1I-7g6lo^J>-s;3c7Fd|zo?EAYK z421h=ljIM6@Bkqz+(P?7{oKg|02h%t(|wBkW~S}jem~m=a>f$TOCJC0E+iIsXDG>o z0nMBoe85V7oa&V8tGDP(Hh!p|nHel>W9VM-d=IBv)F(_88E2PyUC$Mq>(Gj!m>n&F zX_Ck6%+#t_9(UiSor%NfD)36{xlOlahY>Rar3QNFTbixFFS^@w zM*q`p+iVDXto-4NK@&|&Ek;(Z%y_CIg7wk%dL^b*jOJM4)pm3o>hzl>{=35v*eFJD zyMrwI#cX-U^8Zo~(K>64p?YtDtscFZgDSM|DV zdlCfdm4E$o)|i!W@mIW()N;wGq{(KIkPj0qF{TGFM3@TT2hJ$q(V1&PbTD|B&%*Vv zf_!63Nc6ZDx@qR8bMAmbmB5d{?Zvo8Ri=1zAge4)0FUYDuXLEf*oW>v^j`e-_3dO* zOJ=)1DBp$UM1{k4^US+0KnfRsp{VZL@lUae4R@y=xhr>0-HUe^?l$*bGWtPIH+kXd zeR;YsjG4W&o~YIL;jJco-<8m;?F@~MsRv6vB+53)wY3O8_1_4KXBrbitmC)CJZ3!H zIui=Wmib;%g$tckvbh%(Q3r`5*=~c2$zYtgLMGS4q9gIGPXCkdRd!;LER_z^N;?IH zZ(4srnW#f8;uGF06hfoT$=;IMwM&G(kRmm-WYRW7)^)}C3Z_~CQFd=VU_}I@@gb*? zJgRU(&YL8&JC5IIhXh=Kf7B(_Z$|R+M8chdhk3Qd@qifNiKO1*!>~lxM$+i>+v`aQ zfot_b(wnd88COng|JvBDkAo);ByanFJ^qb_AP0yWf%si|W?)DjI^xQ4-CsM~`$=L; zUdxreq61}DGnH0}#!McT-oN%Iocsc&(;NgzB30ihL`4L+luIB8yknTh!^M8V;xjWj zknpO``hfMgC4JM_D~Civ_IC0po$}+@mx;b_b_j{r(csb4Nt3B?rvnm-r0Bwc=Qn6Y zgr^+lkD@tJG!8`T4hognV~Re8u`wFVgFD~EZn(w9YKk`8qJVJDRsGhKRi!&uP`j9l zr5fmmn)dl_e%QVJwdaqKWX2!0jda4zsIH6`?#5WR?*sl!}>Ut2Be4kDACtX4BELYe}^?<`@Gjc1yyGCo@?s>FxO2XKvqhzr-+nz4`IAm#yp%h%!=g+otah*58j8AHeu3UX$ZxA{0!l>Zd zWLfOxjYbVsav$K9cnH^Erp+6^;q4?RA_{{cPrr5Sks2gT2(mi4LR7?ID`p8 zFVsK%{LL4fKbZ(_zyNA!x!X)&WkF`gt)VaIMz$(%(4<8Z%m=iI?9>Y^#BG_jdmFDN*l8K+%8?p%5cCDC?OjpDM^RVZ zWhUxZJ^}2}Cma$zBv%gmfmvp>KOoXod_*~%(@2oj;cow8mn_|Huu|*ORQR>PW|=iS ztvI3JW3t|oUcNtyk(KbXQis#BP3tzJo8R=V@j$W8S#JhCXTWD-_U%Gda0j`eF}+|I zt1q#ar(MK!lyWqX^L#K{uhEE$|3wL<6wy*fK_&}Z2*ZL8%m!GRVO+LUYEDc}vQjgg8Y_8-~w4^FJ|uokkyeksz&{ zhbc!56D7eM`<#8MX(xnwA_kEz%2Ju(8fgm0Ob##e!YItU(2*h%p!Ue7geqk zBHKiBtF1HFnlk-$BGb?@Qnc1%C4GxKqp-`s;o<|P(5*s)-`)(IQJ>8`gygX`IgON( z40?vYz6FG7Ptm3)0wyKQ_BJ)qjki^3@{7ak9PdWpFdmk8#oaEIf%-*ae(X8XW-5yH z*@^KikfRbp@?WJRjV_p5-TnS^Tc^YW5|kc&zCKe7GP726$ndIgikJc{xJM!RGyIKXdt_n;X_`KWZZNX1ArKGb z@RxOcD72R^MhZtV9Pq})uhtsH#8BBqm*230SD#bt=9*)->)qx5)BaZe*ZaQ9pSpXX zSZ%*%sTk+BNudE zseR1!wzi5Ko1lSrNBg9bbtv3M*IoDJiRkt(c{aJ7u>_Z>ss6Q=r{W^@2%MMhOUrV7 zg!yFsyW!oFl1;Nd{++w64_yK@cn5^agy)!jm#?t|;ca7Fu&2etsGhn%hIg^VE$xvP zZsqz^xfI0`K}YPjb}H#jLnb&~Fs~7~x!m#Td~po3CDzA(qFGSvM6-Yxgv#5YRgSUv zak?`zkj{HqO31o#%8tb&`nq_JR@q_pRM4!?K*A6nox>&Lx_XU3R6Q{yHSW-9=05#Q zXT85vH)TD-{CvQuAPWCF|H3`P!s=m)LG#Fqbj0yDcbif4G>Jj5C<5n^%(+_4DcU@z zS74C?H}6j%R#>Yw;yJ!E12(#|eA;=yyku?PKji8n_sPZ9S{YqK11xy0p4giYqe#b! z&CeSj9KFp<%e!mXDA`G`4MjpwZ;31Lhf03+A9zG+Qps?KBdl=^HEV@ewfZv^2F%A! zv1~)OyX}TTeo7CKOTf6-`L39+S(GN^+|U5UmIJZ3;ci(<$PO`u5-UH&!Vt~f z^$Mgl_%u&@Y|y<%qA6);ROU(qcgV6q=nm;GYa!5k)BFYmEA%Fge&P4ej~QjCX+HJy z8@L~A{S3;l&dmw-6LUMkZu=jg0iQt*cV=*3pienKa+2`&!UXxgl z`^x_d3Nnw)vijs+;@T9@T+n}_bICi4W3IBFQ+Y_y?O)IGy~bwqYWmY2VWbU&22WXh zDNPng*L$p89mabb}RpuiwXReQL zR9wh~oAQE}`SYei*!mMzc{TS>$n~4%8f;Eg?|&6eL5MA`vWsb)d(JmEAjqPlS&-mx z#>u^)z%C7pb%PQ6bRJT^n_n(4cz{3o`obB7OPe-{hx>YhA4*N2@{+DSnF5-UPd=)9 z@!KQ@_~;kbl9V&(Z`@RxrMPt5 zP=O-AvM!BHF$K-3a^N0HR!NKtB3TT!Deew1^mP&ktXjjn;M$y|zbxmUAuz&zTmMOhFE^wa zt>xo}5~nup$AIJA{6~5cH>bRsAdd-ZHzD#i_eT)ce5E#X&FD7FLf*xH-=rOV_m1#a zJX@cwu5~7yuGI(Evs3v?hRXJ+%vO z8;sr7TXrJ?FCa5bT^kgqURa_mVxfA*p~AB5+qEv;`k&G1(qjc3dHThZ%N9N>BhqZA zIJ<~2$jl8pJo3>%)#*RTU&d}Dj|t3gH+zN!0=0P&2~TaA#aRMh%h9$^0#GK&l~U)? zn5XK(NNC2^^6S1V3hhbw4DLKFnxK6i-lKd8r_9%Dac1B8^+mQ{ zbt(*EcdGKNR3Y}KK+psK-PP2G4l_?p>OI2(WmiT^B{2|6+?p=MQz_Q8Ha7_D*E+PO z!w|K<|4;-ro!uUJYHhjLNg0cc@x#~sL{m3{LP+Crp0UY-&_##4`aMf*?)Or;2Ap_%2d>`_pQk<_DJ9H4}z5Gr6`- zW=q`Sp}5?=(UhE*wJ7iZ0j0EagZaw6MWbNFfHup6S4C2e<)cRJBWSD|Hx8rHdcu4@SeELS78v=@7%nwqx_xN4}-PsEL5`jr$i%23w4a+u;b z7ni{FIY1{QttTOZHSc7Cgf)Vp+!H_#npX1l*RJVxcNr^9`07yiAnV|SN>*0K)1oq; zM&#yimG)faPb(nA^!KQ1^_rOw-zNF(pByIj@6I3}$ZGK&KL;6Dq$HoNtd=csd7tHvEkhT$Y{h)(s0_fq zvj~VO8zZu8*Er%-8&V73aojxb%sE;HZV8ZW-=^} zcWK_LH(-O6#tjxoA<4L zUFn(^x{y*(Hl|4k{=85Xvg#DMus>gof0C9A2|MREw10VEKc2W4&Uewt3njnceY5qItt&T%sBChJtvWJ5UG zThN~K=?FQI(-kRDV?O1w&wRx_*(WQkAbhIW#EaOdAp~~xa{;2}*D8GaBH4s8C{1V~ zNIxW9v+-fmv*qUK1naxXJ;`Oa#3=s#=Z2QKWdQQiZ+|7^hz*!DDd2{rr^z!~di9#0 zoFdw8&1;?`I)*v0Fv8O=nTIBWnnt9^!L?!~XpX0=p3jWy7iTn|hM<1KaGFWNNr?*< zgJTu_;mq8A>sDAFJq3j1v`&eJBNm|5c4JnjL;MPdi z@W#V_DX~6eD?8I*jbye6VF_C!pwI?JJI1ZMF}l5#^om$%i331CF8O#waE&`Ncm5iG zK;H%)Xkq35Mhp7~b?#;&d|X zAv4>q@s!ZZ?A*(*&ifobr#M_2374yB_+fN?N9Q&SUkD8d4pSJ*`Bej)Hy-djZF0TS zJ8jCCa-1TcZl=v~86jw^y*);!wTMk#a;iYP6&8|SVCM2=L|}qHl_}$lJe%}RP^&iA zTFG$1Y)4uoOlqoNCG1P?N4c?5rF+zGG0zA7y4^8fU{EIfCT7}&SF=r>rdo0F-d3jZ z4_lEG<9rKIViuN_TTw8cpCk!eDXH`|_Z;_W?NE7$`KXkR*w!t%vFyj6O)iyl^~v`U zHE*Z}xMJy6mbKp5A!(aH6wZe?pb@W;Lc@XPBI7d8<2fczPCgqeF%@cLl>Alv_*dIQ z+m>dF0O_YQR81kzN%#D41IZs(3$yWQl12IoO?7pC;6|g(k#~0AJ1>bxZ*54?TurO^ z>eF05f8*79!HAkwNZ7)`r_CYu7^l&9w6PNhK&)=}u`z8<>7HT}apez|XGFHz>n0NW%de$4UoZE+bPNz;3{uNjR zXY6-!ua&Y18GYl=445>twm|5zU$}9e5pf<*(MEBP7F+PJ2^nsZ%7g&tEq2=B1l+3s zC;3t%sfnWGT?k*MWow6Ky6py=&wz?aqmC+3D{H{3_94 ztr{zF6W;hGi zod7MCdk)`ua!LvcZrT+^FB#N_hFqVUI=@?&f5zZ?4Xr@Isa+c@v7J4g`@$u@;H}h~ zz5Dte9%=gi!sXeSMy*2*AQ$RddKu0!0xy7FNf9ip-CS-v7t*slB54Z|$>Fy(tg#;GyqJe|cqm=F4dCn+~365pIIzpniFfrtZJo-I>wZ71v6Ff09?5p%(c<8`kO<_vx~G zc3_<7A_e#9VgW30e{-JXKIj;^rOj*Uh9T#uU*`8~9IBc6{O> zKe&`!%I3D?CH?ELZ;v$gi|;PB-;+0{DElcc8G5^mMGsYI1&?TPdLEXa2Z zVf{_fQmL+WsFS@vUVvRdvoatYd(E_iA_lFZlm3TBnIiYGD*BX7j`aL%v2CG>{KK$nS!rHA0p;~(8;dt0{x6T?>Qoe`^?`;vADLG+Ft|`>xpECt?9XhzEm@*Gu~^-q zW3MlLNXM3C-{xS+nuKTNiuH5mKQ?d)xG9DlR}(ZSwvV_mRu%q)Bdz>1v}435b12lo zuviaX_&Olns8{$k{Pt*%@g&~A2zKQGTQPE^Vs~|d#|^a;&gqERpy30gyA1P9N5oFK zq)WT`X70cOcJ!X4>zo}%=7SviIctZ0tNk$`uN`~nVMVQNeHiGvX)qbUktnoqzk4}h z{p}BZCo4f6`s*iAle;A8L;=Pmk6m@oY%#($-uN`VuJxRSV`taP#*FW8d#|2bu!s@2 zs=RxB8{<2%a!y%RE`Z)`4S((S=1KNtev(H3U{`qCwx_N85!YsYj)cs?e!l5E(b882 z0=l)CXsn^x-t%bcLENi*MpKN5G+Cdr;lf#%6qtSp+0=j~6gqNhyKjxkMebPy_go3t ztG7{SctApR*L$v)faGo~Q7gsd8Bu0FJ2gYu^GmL;)X5Zr_M-6ppxi`v7pD<;^x-b{ zb^c80E(B<3^(t)y3SL&h#h-@AKsE;>Dy`Kv1L4sAv{NBx9)-qk!JY?&NV?&N_VHEu z>-}Svj|lG%%(H*FC@^A`f(X)0HC!u@QiJ-+f^}*#c4wam>)`_9Nd^YP?o$Z9Tcy)I zvo3bEt|TNkXhS!DI(TiuVyJ4Y%9YJZVGVHCm>W?P>mCRRiF?!^gitzMoE^}lTR(x7 zQ6M7_e->uE_g2YTgQy?ln(`Qrxt?jPKl1+p;bSR>O;m?#4aVrTHSY5X;=f+gQVVaJ zV&S^5W!Eu9a$r_sUQN&L!A`fKQD@9`ks8MlsbjsP18BP@HHloAmP60Crh{+V=lqJ) zS!p!zbt&gh3JG3l{}9-3>dV%)+9mSYgMf<3>V`a8)Z(LMF`{W7I5UlfTX7gX3E&A{ zqczL!fRPrCAW|)OaEje`J9lUzq&ipA&r5(8DUV{I6q5iW}jIq~O$@j6tem)j;~# zv;Qz4d*634Mbax=`2H#1w{9Lo!xrfsU(+;!{(NWqpm=-~I&4$bQ zD3u&Kv_kVVI=k7-nR=2fU27sFZhwakw36;Ki=^>)ef-=_Nk$Mv3ErAk{AU~N+%^>e zYuEI7BFy)BKj;y!=BV)72I^gtriJfZqWx)nPvyr8&XgJPQA ztimT!EpdCmgYUeQosQ=<&7y_i!@)BHDy^DlF5K`w@dU3w{3ivxOw`NM=mq9kcOe$R zgXiL$lnd=*^K*gO+CZJHX7b)~wCt&%5yuf^L{yijDhgF-SSAX3HXh9Cu%knx{}#xT zbVm6N-r=_~uC!4H_iVo4FJ)7%vbp}}^yYS%%uWBrWBj)qQP?yIhh^q>;j%h)&7Si; za#od=!8hm5o78ybP5Dzx^!ihlso6>fH4WLBC@C6!X=q2o&V(`ey3`_GC%HsBWMQ#p z$)EdyQYhdS!_zDOO9H~p8D4OSbmPxS#^nO4R8CA>-6}w~k!7oMN%3vpocAO&fz!!f z8}!Ow-zM6vDKURzlq*~6wg%#r>VEq`FC8xB-vqaYlaVy|K2MZbhI`Fg(TKGR2E7yi z7AY<7h{&6-2X3}Hi!W7jCh-AVEc`NsC?0z@XPW4QbEXSM0IOnHQ;ij;?x*4vFy2fY zeq%oyTnqC_Svr$C-7CAwm9HEv;VtQ~zqbOYd_z_tz*b z=)?Acaew*F!Kr?G@yl?dgW=+EIyj`?%$^Ocoi3faH6^6ar~5Drc_HP70cqC%^Xw%5 z{qgILJw6KM&~S5D6ZNV1ctKfu1z0=wuXebh1Z)-GS}>ZBl-wQ16XD%RK2|?Bq|2!0pXNYPCu~bj2efKSH$J|~@<`YOIbJXzjn`6NW$^_JYyuyx6nm|fHrthG6=ni2 ziTU|13h^zUYXy1}zuVrjwdc_Y$serDBj6DmghVE`*}Q~;76)ups(Q!$JR5gnFLRfn z@zsZxFMK~T>Jtk#C-R&ghuHlDP2ZW&Ud;P=r3go`i|=S@D(2%W zrZYEgKP>X_Fv(R3WCS)lY~8j4?bvF5JCW|ecR0~xckXhEIz4R=iJ8fx&Z}KMIntAj ze{ILBf4yc*oEQ4S-e%zbteVGhRldcMjqER>7vP41F7wi4RTd?nG?g#ekw&p=>po6h znkr6sjau0n4>TVANU1F-l&AdCXL4D(TfMfR!H++3oGgzW*=$%G=h2qRuF{4w`c9xk zli17S&gJ^8^N5qLPQ@sU9e3C8g3qLIbtMsZoQMQJIwD^(-@=J$i&NB>t2D0E!>M#pZ=htW13ML4cyFXSV zkkvsY)eLr6_d79^`v)jTcGLFIKr`gH@pc*Prt~pgh@~?|sd$jA1&lsXUIE)d9Y{3q zYPq!{%oQG~G}~viCQ~v{6jFRy^S^eIIWzIYe}ZQRFTSdOQ_s07tY2H~-LUF8$he9d zzcMUKIb_bu6mb9H(w{c6R@~ni$OqiGzx_fTuAY?JL46)8HWadX|8qEo(YaTlQ?9@D z&JK#Rkj(%$>%Uv^-XZ|`-E(?~Pk;KZW1ubO1zbY^gT1c~i#l!pT~Tq7QV|IWQ9`>>;~Z|P1DZD+|dlOOsVjjw&itrOAD%TBuqz( zindmCd`%;{W18LV@*nilAJdJ%Tg|ExieH6UQ{L{(7*Mek3EzCrSWc(clOnRJf;aM5WV_-MS_}fRd>ZK*O&S`qyjqbQCGp>DBpgG<;-5zKcSZ)iR*aP#kv>q;T#PO~^;%UT_9=1Oaek3pN#3HTd+Q*-I+oE#p`o_G!ho||= z+=qE)vxm0TAAshoU?|yvX?faB*w2eGT0vJ}PqSE;oU=02&H6K2ErwcsjPiIB_DV~O z;Nvyt7O^S5Nk!MTg6XPb^i7pBAxq*1V#(o;Kxw%?I3XV-2!$uMyxU;>F0JHUK=5|O zY>ja$BB6^G?feX#)(ytHj>%_L@dq_htmc-JnrRT)gF{=Sv!!f`f|$ot&HdQS>;XMb z?=20n1iiv+JU{WVLe$Y-EtZF2az0jTaelX3G4g(Lw~O{x#Uth|#HVnhw5VH&D3O%o zD7TV*2DNHp#gf$cp=XS2+GzKSTX#A86iC}1d$Ok_%+Kq+kQraX`ko~yv9Q-Cxptu2 zt7X*HG0=7YKtv2CcIW&h58CZRnGd(`w{q^P%`REylIGM$jl1-;q28$5^yTLZAT0De zZOzBZt=))8!#6&Zt|d7RDrFI{%<=#M>Ku(gi`TJC zYTS5J58vH{DWV*=G$}mxj%dxEXp0>|u^{$T9jkkB7npTLxEa00&EZh&>WEot;MiS= zyr$mqq-D)hiIpM}rhL0E;;!3W#Ta~-3RHR#0Kw}L>j*h5q~qPB31LsQFufON74npk zL*-rw3`@lw|5{r4(_;aZI{uW*bgVmZjN=$`Hs6YS`kSLEci*+?`&qC!D+N}UT%CgM z{cOv<{TDUxgIbkp2vPK{WZaJqbFHp56g&sVDQbAmZZpS+yAZndX~Gl>JLT%09mxC( zwjP``wKswghRV=n)kS4P>6ZzCDPMYWv{4^B22I}zhKsj^38S&1y#-3fzhg0k9(>c< znXld&;$EAjb39#~SMn}jG?&q8AKMp3)$XShJ}*c_F9k0O_ZQS#$haP-sI0Bsn+tke zbX4Ulqm+_5LBgt$&2_FrT6x%Ku;7^ zON5t-g2A*eA&r5iOjz&Sc6!`IMqgf!3i-` zuFz=}<+1{jnQ}Xn{D^NaH5JZrS)cdZYmZXa#Xrw4pF$oT*!MEy##mQ>#@1}e9eKri zMQCmvH6#}5#Va0tO0aNR2HRs**gzHW!2%QNE&73Sp7$Ac8iiA^2DyxK!j2#ogOAh5 z`X8DK=2)Z}62gzjqH$aKN|TATW-ji2{AK2-8ry_SIaCNQ#R*}D+2sR5Wn~uMC5r3p z5pHUqSLVnWZ?73X6wZnZ!;WR{wHfrc> zTcujRe&wM+`%noxm_BD&DT;lmPSw}2!IvYO8GbG(O=NPTQm<(D?N@lr)rLnpktau- zk3YTK5*nYLZ4Pg_(9*_P%rx1cLqvkh0gyc4?^|L+*q$}KoM8ljf!k7ub;^`_(qj%o z*O9^|-@~Omy8pRdnVgJl?fM-8K|b4Y_jdn}S|6W#xK@Zezd;PlWnbZ9 zYqMHtnjjSNBbd?u4OZ|{QUB}S8I#%;8 zt6IjjqA?d^`oWBbMzhPG7N?(DZ&e%ifXIZ#^0_ZcyS9ktQSGPlAoY2Ifkf_b(W!R3 z;q5zDq`WAL9b}Vy%S@Jvs@k8N#^ygvl^r~TN$cqC^JA@+<3}PM;F9v+z~eeubg(ClS35qTlUX3ZH4sFMJYOz2q(s_#w%-r-i#f*4Q5i%V^JU6^TpH9XdzLU-3 z4Xx(d;US_DVv$q0*iqf4x(XcJse41SQR56LZJMr1-?C;L3`q!(P689N(ALH}0zMuCueuvZ@k@bUcCrgsxlK z^Pufzc)6QK?;O&wFOtlTbBARnY2ZpM?xo_$goi`nY7eF zf*eI=NU7zdzmpsJKQ9Qbs|2q|WvA{q)3~@wR2CZ?Jh{8AKZ)WWw@D+_d|DgJol@1U1|W&hMC zKpbNc=|^-?;~T-~^qUM{b8R-&#iIFV@n&tYbgU7aj_Lrb#BmXy6d9v9Sg5`sI?g;$ zb*^h#?12TDY$-24bqSpvXcu>h`^8uf=SQFIb^1~gM3FMG!>kq1HSxK=l@U}2H3=>4 z8b?D-P2%g*M&SO(s({Gu1TQLZOb;daW)YD%W}~#c1)_t|1PwiSP6buQ#iff6WJ9KZ zposm4ORn7a+M{ZsYZHTYU%|-BI<(WrL~ZKO7IdrUKIKqJlXp=}yH_WxT%kBd;$h*u z)bFULl4aW-CB{0Iw>H*5!#Z|JxSq%d5_ImX>Gu@XpJ7SQq&`V{;k26rQs()OX@!;p z9A@%fI2WY@#q1_uQi-|I&y+lO&ecaLG9K;wxsD|%mpdhbx}>NUXZVsme=w*zT0HK# z*)Q+>u#nph9CV|%1=K&f;J;tz1u_-|&*~6aBi65CE)mlZz)~ItQhn&lQWqI{Ps(|N z&!*|y{mp6Oi6*7T>16d-6zPf9_Ze>p=ODbOeer4xwKhyHOCIGh`Ypy zpB5T;EpUf~>_$(s?b^I+-poZnOt#anI)JWXX4TUCSYT~al}Pi#AyrfIgf>JAr>>7p z6cOdRw09m7Y^m+_!mAD+QgXnOi5oY*a~l42L#+v*l??%i z5A&W~wzu!!w=5XmK~@u*S>*Q!R9MAZS7U{TzbL}lYY=+&5kIcf&UDKgLobn?r6x+1 zD3#ifJATCBXC>xL$0etEQ230D>Pc&f6N9ye&1)v>dHd4kB}>&lP!{2Sf;4NvsCE+w z-^D>_iK()>EbRlutxL&gbr*)k*waiA#WN|5lTe-%NhuIIrrucS-6}S!MZo9}3#kU< zo%lj7%{=|19r$kI=+|3U#Jq|v`x{FOlJ1{J3?q%(STtJ;bqCP(3ehs*nQG45iPn=q z=FrLX8G2Q|e4;t_nGJCuZQL3IVd~@dx~W}kvwT76{Gn2Bx#AbJ{WR|U>~@*UYI!?S zaoXed&2WfPvOwPJ?}goedTB}Ai?ZPy>>&5eL>Z@3;SxZ_Pp70kTVVI$wsN(Hv~ZW< z*T<1FgfYIuH1FX{H}G9lG5&QG-j4TvLa74*BE$j1YeIAZxbj2NLwew-T%)Cp_pKto zwv6KgF*>IHcJz}XIT?1X(M?vDo$UQjeJ;58Yw0gj?c|=`LO+1|{gYA=D!%Y5G)`+x zP~&Hx$;}~I%3D8iOYu7n_MaYib_f;O&($E(`VZUsZvTLI{oxWxfBY*L{|&)6JwIc8 z!5(<2$i1|i&)D&BEdG<6e1Da%OrLG#5#hJ(pWVvdYqv&SEvhDT*uR74`=4+0&sP53 zQvb7+|5IN6;QN5H@_%tEy;jotXB~=vrP7`Zc^^RoyK*OyWqtb7yC-7emmdCvW_+gk zvKdiugt4>v`&a$TuXfFN#uFvh4b0Fk9WJ2LN`9G;D>y!}bNS1ae^3g4KEqdZ&#^!s z3`8c~`LCD!mme0dH9GVbIjw#%_(=~3sF!rS$7Q@0MO5T}(RF_(%HZNh)j(};#}r)p zNgZ+P3O~@HPm*Ro|M=4U>$@cnE&`&j34gKgyxITsmd-vBC|^qUOAmfR6n=k5HzzO@ z$EGV@f8MJ9vJZdwpx+b#yL<0*VIT{C7V_`?GuQ*H&Z3|Z@n5!5UC8RrzqtVZ@`-=m zy;~MQ^(Q@O@%lNx4T!@wFjMl?6tJIrX|NtoNN$Z%@M{tI|LjU7>w}d#Xd68L=TG?l zPQ8f+0+`lmy~k8M58_x9p{ZnI=Ba5x?WM4bQm6H0b#G|Gc__dEOc*N;wQ z1OoWa@%-z;{C{^mzq7Uf9Lc{Zv_GuaKS%QK?8HA7^Y2#Yzgx^d2-ts8ApbW<(pc$~ z1I#5urpV2#f4GAV1(~P!s+p=o0}Yq|?p^<iIT&9#}?)r4M8b>U;5Lp6oS!RcP^AF#e6!$Z%G5pDOj7R3XRu?fj9U z|FrUd`?s&i&oX2fDsiVqZZVat71ve-#ou24pFAex#LqIM7*U6rH~!?}tqp1g$_yE5 zStWV4+v!gn)+eR(X<2{P2l?D|EBM%0NCp-tGFGlwfw4A=AJcPSqHnBpbsWP00<`Aj zw9qK;!f>JT(aX|>5b-xbi)VX4z zfwJw-j^bHp@B&a%(cP~qAcw?xiI^(*tUI@~z{nUhl_Nb*J($%iOZlD;KH@0NPrA;o z)ryMObm|YOER}*vD_LF(m7hZPqW}W=4D@FO4b8#i~ze=-WpC z&?_llszwmT!8EJi=F}QG%23yKXrNs^pdShbnvbY$xy{a$-?VW6H=b*aV~`*^`Z56D z;U5sj2ITb=BA@L$Xy#AaHoRyGsYAN;u`q*1I^m#e$|Y8sW$TG@?(;K4{40rQ4ev{* zY@iGryt6zYb7M=Q%9d3<`!2}ohx!I6;#b(F6H=6s-J=c?{|V`ztfmjH^C1d~V?7{Lj`FdcxfuOo|N(i-Y1aWO-%cZgyWb;ahkK&ChuARvuf=Gy2QB3RV4 z^j>5Hsb**?h{TT7`_Mtlr$u66XiQQaEGtJ?02UR<{y_vcVE@d6M zBKo(1u5%ht(Jnpp@|#6+Xe0~JuLuyfp|@)sk!Brn`9@CCcSx?ePbNH!o%{rye9tM$ zDsj49Y6dGpat6Ph~C zVYa_MRl9j|I{x7#-jgu{o(%|^IOdDiZeb<%-q3$EaI#S7*a5^|SF8L~EX`tvv}oM- zt}?5$F*H)zxIb8XZP$=O`xEiZUf!>o28&5^MCk4N?_|f^cB*1# zf;Og&*T$>RXge^{km$o<#Td1~V8)@j-6oi$x_fb}hPq=!V*PL}*2uE1wl^Ag>%j9Q zL&v$!B*Rzod{BW=gNvh{;A=vcr58Co2cO~Cck1?{U8^rM?q+^z1-kg#$1jRvDD$4w z<|^pKl&WIDOc^84(h%F~I^oFPSBQgXO8?mPd0L0lDrn2D^~h~k$|sP1U&5QQ%ZJ|z zl~llu@u)O4OddpU1PQI@rC+^M#>fxu4#9iBE6`(O*~!OleNUnZSB!iO}Q2gF>c>R3wX`RZpuc>}(1 z0+Sxsu6sx7$TDc!wgeEv?$4datVc?GMUsD8()-ob?q~c1uhU@3gC4gir;S?Q6PoSj zaY?gffz{--EV#Q$q2>K{$Bv3y&n0Xh0;BM2rrt*YVJ$lxKTHM?4`D&tIyhIaSXMFQ)(?k2M67SiRb!k(Td{eu{_Wuy}8K>PchBWDrv!ML3d$EDFDRF zW@2-;#dywtu^aT8rLCP@q7~?QBeBA!59=d6b2Fe;;hB9ig`7uu(>8qgHwA5i^Uy~% zBMd1dB(ob#<#LGf9eNIqcD#@?%8)X@v-a{p#3@!(v%)kWj0GVoR^cn@7eqO4MGG-4 z5y$TQuzSEa%%1JiRNY4<AAi<&<9xvRI89@~yuT{PqSzwjUB>lN68ko{8nzCgYPtu1Iisx7XQb@7HX6)Op zX;a938c$?}PXFv>!NH81pwV`A8zq4ig=TY9ZmCy)vP0N@u4Z-f7=vv`Xxmozlz9-Q z_NQyg0ifyw5ofgM^T@X(Cj?9H)f)hhhY1!lhOUKHE<6MDKY|ven_hl2;mzS$LWaK) zdX#_`cbApHsS)f8&?%)gb3woOoe6+xZoN2p9hqX7)Dt_c%&Zl}=$7PJV?Fv)lR2Z4+<}v5D@wSVMxf_7$dZ|D8DzWUPz^-?w@zg$9TG%&c zreBs<0X1#c)`)F&NQ#{Dy;RC8#cCSPI$)`GV_knH)%r|xGC6&ef?Po5F`*o8iMsXh zJ4HCgy4LrC(!33-gYR6?=TxcUUado-lglv)^|?>cI@?p$Y5X?dbP2UM0GjK$iHxmw zfmIZgf5rhIBJb^5rs0yyh232oLBmH}CfxWm-bLP>QhmFP_ElGb7wYS4g2F25(1+df z4WCZ_R%e3rW>1jB1;m+gdv)+EBMHDPq|$gl%S)eGRpWtNgw#l2ANLsYfIF3AKkx2n=r`+!zDj$E4aASF z5#_oyPi+|q94k{YX>rTG`?kTG>mO?A8t!Lp)dZ3?3F;HCQ=epU9=;a`WS*>)ihbsT z&azHbH*%g)uDXDDgaob22u`xgz0&1@yhzZk(Ng1iqY)JA6OT8qN}$Wt^7N$7^dXJ* zT8hR&)Eb!sJko-qAbqrn{wED(HyU&ej*T*WCZe4%G(>ZqQZDeDlkgX zZ2u8&>-A;m65r_f4XFR8Y|TQ(4k~hX_D_HiK9JU*R?9t ztlt`%G>3p9*Lt+{3HlUsbAM+*KY0XBEZgv&gGi8X>^L62`Qd1-3L01zuU@QE?SV+X zF)lVI#bYRGLNd5=+K1ZQeBaR$@97VH6+WdX?Ok%PH#;xYU5G6N7$sfk!g$q(44rb@ z32_-&*^tpP$BKr)aM%|yx*I=H!0{p{eX|IBKPGXr-E4v$cJ1=tZp3iK03xCsyQvza zdCvos<=Aks;{67I|15=j;cHp*a9yVqdXVNvY&EYx_=H70`}!ba$qi6+sf>9bt$I6G z#IWO>f)frE^0Dh_B;v7xwt?ANWjw~eayR)H>3eJlGi#S*0|Uub8=2wN_4rFW&P9A# z&@Gn&y2q5kU<#EkWHb;k#o^eY+kICoe3kA}Ujh;Hqv(fZ%*v{I9w~WNLzSBK0n~Rw z85P2+Pve%e{Y2h)plM(;bWGQ^K1rT4d-UZBHWpV8qU++CE}c9~2L3A&D_nd1i6rr7 z>L72Vygm{^!DMl^XU5WgM>N+l1O!SJ#SakTRlwLA>T7r>Hds(1@vR6c`9ikZt8e8W zhG`8n2Uw3FW<|Gapljpoo*gl))kG)v9|TcO;kPZp61-ZWr&vSZ0QBkZ%?a+!nKo8p z@$EOUJtsO7MN??0Ct7EzuC+$kZn7S;<3{3r^oF_S8SA4?cShk>=LV%g95=lDXnUHN z$FbgNf)02Kw?vYnn+^rYbQMR}0XhK0Hf`S<}&j*DL9|TfEjAxrk(R2fPQ17BOySTiLT&t(m;+bj?l2(4$h=mHCx;G$% zG~IEjXz58)a*PauCOq$Y#@IOg#TC#(40L1r#L?2n06bQaJO3oZ!9>$|1vFMeb9McP zx89bLCAw_>R5A8VzGSN}gbcKhd#UkQ_GUA5RIMvV|0-ZTOb>wHGl6}b;~@t%aO#)Y}QHK$rXnz{ArVJ5)HTaEM)n;vk0#(>!8qCT^UF0(L9nW`kr74- zSPeD`wpVPTGmx1*mCJDL$`|%sFAJeZZ_CLl=fCp&L<-jqc{w>Qy5(yYrGV3;&MZrE z-W-Jnw^~GbJWZj8w=af{@1=T{x^+bH#iH}p+H(B$pO(&JQ{C7u%2I!3S z{WLre4^uztEzK+^@c)YNyP#m05Rjgb<|cKlm7|>YO34ymlz40nBnz8BV>W^I+rkHJ(dff_Sl8to zL3;6rl^H8;2`ZZEX%@7=C#~FVZb2`0Ia{R*c#M)$2)g*&svP4I-cx5nh+AiJ4rV~Y zIrYDw)`sStg0ZLAT#z4VT$BJ7Pjqwgcyn%K?Hz9R`H{2iWRH^grn6U6$QrAE_u0y1SH9xhC=r zy>-Tl--AroLojqFBn>8hC!r=}s$H7!plW=8?C}xkh>rdIe7asC#pd|nEJxV&c?vXa z8Dn#G(bDGOO^RYDqLo#QB{+f4kQza4Pr9fon~N8g=ulJZz7#saDsmI?NqODG=5>ukuBu z!}55zcym79^Np`uRSv)~cJCuO#yQ^hS53mMs{hwr{p$~2r*9$KFpEA0jD}!{DC04En6d1I~>OP6mPVjvE>MZ_?;d20^j~T_=T-oAm zDh3(eR=YIJsVX+kVR~mkNR4@QL%?=?#z)+lWz~*dBUi^^c@^q2LRIWRsDd_Ou>=Ge zbyU1=$vr!$47dHM^fOj zE)|aH=A(W@c;dUltB}T#H8dw+Btll;ifMqXviRPfN?N%|XFV6S)|d6rj_PfNi?pWc zBf5DW=l3g52b43EO;BsUvFmj0u$YM)*yRb@j>p=?)a)*E+g5McFm02$JpSzaAV4wH z!}Rhx@b@lQ$>{ZEe1UDdsq>XmWVZ3rsoGZTM~uvtFWYXryw+(CWeO5KeMZ|9!(`jI znjlhvU0^3{+xN9N1DEf6cHj>??Gz8l4Y1}BEn!_|nqoBcJa(L$A|&9U;rzR|F&l1u z)MD@S@g;ZGLeutOz%@+2JdoqKo28TTPBmAjE(0f1r3mmi)YS-1cqoAhY^IraPZs3E zsb3J}I6NI!b?RL#b{4TD95;E;GqCCc<}eP3emW&XTJ*`rk1eTX1uYi=iZNVsBNjQ2 z4ih>Grkk!*bcEZQI*$7ge@#Re3!cY41+GK&l|oEoWeF(^bC=9 zpY|&D^?~~cHB@HEVRiz9!Rq(7ABIq4XOzm!vpCH#dZ z`tqa5@s)S)UR}(GTD=u(8`SpDwlYxQuDixxANSEhruXT!0LF-?kFU$I8AwOJ>W19a ztvrqLu2|9Ay@AoSn?i4iMvBe1#tQSU4dIO1R?|to1cN^;D<-i0}Y@+&T{) z8y!l5kzDv;XNh%1bF@Yl$F8vRQWzJ1l&4N2+PY%beBDY-$}8NNCcXxEg9Kp6DZce^ z5gwQclH1FRiv>hRx6LK^on&xGDxecwc)Qj*LkyX^WQR0|8x09k-o6Ra4S|c@OW_DA zZ~5ucG$y0BE&0%ePHw);TX)P&j>GcapDO}{3X0weAhj9OS^#@()x zU$@8Glp>x7$qhcNh~6z@iNlX7aHIeI>;G=hBbah4`SNo#+pSXGc}yEvwy*Lr;o)6L z4_{ALpN>^BN<8AZ5eH^Fx;Cu*=b?!Id8~@X?Z*x+$N}SP{9;TTvtkSTX0@HD2}O@_ z*~m^BuJsVRnXZY<{Fdqo&$Y|W+};WI&UWd$@AYI^_giRdC1~8ZU~D!;N)fD{lT!~ZEBT!D%v^r|>K}Td zmk7xEdPAniJm(RHQ~o_lcqS$$tfCadCDvpjfJEK2_2X4WrReJsfLHHcOuWvZ=KF-rHdbV(w$^6kzEK&MB zNxOLs$xtSx%LDo_1@g^JIkn)|omtr3sTId|d~NG0=WuI&@Z=v_gR@V`_J29q?w}R# zf0L{}@L0(ICPcvhCe|LXyE!~BKRHM+$x=6b&t5_RKg3XpIAe4$Y!yGaoo8?1_3hVR zF>HZJdOyA~=Wy;{P1g_8bNU(4x%K$^xi5(OhP`+{u|1!Ja9%;)Pp6apsDPgB=#vW< zpf_GWVfuf+2vQGhvwQ$YSk7SQ)!(gHehAK)cR_el;4;7{5-JSnO8}CE+eH>*B78QZ#tbl) zs=1x!5M{Sop4&fQ_PqEt4FFESaJVSLuzuV8Z0feUO!4>M^4jUw=CtZ8(S1IU z3%|B7WJ*Zsdh+-pbPhIn=LcMn^w*yO!F26cym;TYBuqX4!j>jhY}3P7WZolC^t|gf zpG`L(pGEK8uU3|qe?K+p0g|oOWyVb*`GDPli>c{~m{ENZmt(zu5kSrqI!5Lu$ZZzt zP1hHfwGKGsPPXU09=nhCeFJJ4FWK6^zsNLQvrVsU#->~Dgs)rSOso(x>p=RiMHRU) z*AWxxaU+2CRK~4L;AWA6#QWi*Ap>U}^z#8A#C)>USHJIcE#-a&aI%_*Mi2ZtBu?4R z+QY31?gw=ynq-yZKbt$V*XzZXTS2$FKRLIky8zVYd4IO+;PX%g=@H|yUH+6`F*+)k zF#BuYf0V51Ez=c{l~Mf&fhsD{JemaLEO5e?uMB0O7>-W zv9r%{XUF08Ham)aXL9e_sIcXnf1U{ytK*X|2-c(&_TRtv@0VOs9T4m_88&^ZOOT7S zNx0vlJ~S=xyUQz?f%ml|W6yP`j!dqmH$z3%`UR{1L%aFQ^hDnm%nQG&GMt_GE8Y;J z#o-*7T3a8<*^7AP>E7CM<%bXV<{NxD%7crB69OmBL}cklyzI-DTjRu7*%Xm3CE%>_ znGY8k+`RPrDUoCb@0zt&Rt(C9#^$?GDSSnw^|wnMFJ8O!BR9^xb|!4Y$qF$gYW!+S zzlR{Q4{mLhh3`|bMtOzbd;Lo-zWXZo`?V18eh#;T4+xqht#M^es`w1WaWRqR~9F8*x)*{_UdRGY|vvyjELlo-8ulq^*&?Q{aa5F5!bw{nZgO9^gZ*Fh+ z=x(W_HM(N%M^r-XML@hyg-c8m+n(rodXfMPpXHuEI&O+nr<6-{chG3D4m%E8%m0FV z(_QSmeP18rdfr1?!uK~gi2wu#K4=`OVBs_dvS_bSenf=JITbd0a|wR^>BLM|Z$0J( zYsc^Cc;ow@|MIiN74nlWm!@J1J7n{=N4LB$K>Iu0LFP@f;USHys+}1j^+0kDNBMJu zH`wE7OAoc_$#yADg3A)+N1L4h%=HCf+1SdL@N`iHdft;Y{9=2mV-PQ_KM5mmYET*jUrdx>Nk3}4v1IM*6Qa6@GJ@GBv?xVwwL|D8DJO;fO|h<70Ge4B!8N1OOdWtZD$f z0dvNd4)H@Pc0_{t)pRdOP0fSyH)b4~oA(Jt?zS@zFnN8#fo!oa&#w*th)3{mEt^eq z&I|DxVS-^>`F8h5Hbm?0`tFGB>#n2*pN_e9!WgS^^FfQOoO>fB)>-^LANMy9&~d!! z`ar_M`BoxOhb2BfSdA4|@BPyF;Z6+zH1}A3wXEY25I)_XkHhCRYa;*?iS?cj<{N$# zakd~AcYpCT?MYJp7Dq+$D*9-%WqrJ2&G$@(Vr*+H zO8dDD{uCtRa4dq$=)k|zD2v8$#2SSUW*~d)n}%aJ^(uu3D0lz`uz+Phn{J5*VDBRW zUS-yHWq3w*ljS@>!4Pj$?o}=gW~jK8Me&;X(s5SD(oAG1#ph4?-pp;~VzJT=-I#Sz zk1|XcaKV(7jj$e`sHU$ip>5^kM1_HSbGvC+m=)qf7BcW?uDBEMB3+5UlvS>JeNfoo z^JkIxCKbE+Opo2g_#cqPNlP+vaCYqWO(Ijyg_U`mcYcx*TAT^3ts~RF_RK&#;oAA? zlg)VvzY&`X{qUmB~$eMY~jlb z>+Si5LoR0Va7D_|{cbt$+9~g^^Q0J6TMN;Y~O$$CwUljIS1t^9;0nrAw~t??pk!g(1OX^ zd3Im+=b5Au39qFQJ7lqpiXW~Nc272q;!f=EqL3A|LDZ;womL8SA6N4!Z`Ovq9J9=; z5D-0V9v_K6ul$_sMjNYW6pT9cI-u{74jq=z5B}8z85f zk?YI)`n0JEF%bi~8hPVWxfN^e5cN>RaHUH-1nW1I6eNCmfW9Nmn@C)U3OZUaHAY`5 zf)ysqIdNI5N_ZGV!%v|{Nw3v2;4=3L%P>&A*{tuo*pp^PloB+;_@x0l=Bn{xwohYI zZ^ShKJ)E(4sA4@Xs3(sT>$z9!xaqr9F>x&LHU4S4=yLXYX#5dh9(oWGe$j=9T!^bm zM^`Vi%C6D1VC~h(57{Dj1sqy>sh173P#uB%LT!|dVLTj5*cKy53~Vu>=TR+fjUEg` z0oqn{?R+GJjg$Cty&t=w9%XtFpto}l_`=LgXn0k%G9 z_#xwv#DD_w5&CA}@d>wKZ@LME5HEbCpdrq%YNuZgYSiC-j{&>)X&xWtiWxHNQ8}a^ zupC0FCD^D7-QZ2Y6n2QaHcWfWG2yOLMg=vO7|nG`6A265{7AuTO=I`jYf*^+ z6X2C%#^mL^;i{JIFN5i#lgT$?Ew9w3A0(*F{YGEzsD}r;Bz+Oy3A45Z_v0c2-c1Yx12A+4=5;EQ;HJT*o7yZRZ=W( znO=wjs}4C{<(2|Yn}JEv!+_@94;SPj7`|wl=vFw% zEO&@u{f5e0>(@LzAMPH=bn0kcyAqWlOB?TTkbw-O9!|3kAwak~+t1hFb39rWDYDh* zPjO5`9P$Z??W=5b)y^eWR=03!R*GR&sXSu+IAlcx}( zfL!@>o>+bvI%;@`WL8O?>7#|MV0+5fJWoGXPTMnSBE@oAT5V%;P%Ls0k?odbVge&h zp+#e>sEG&-4FsZvvzko|JLyf7o$qQr|J1NSU#%5X!`X(vy4v&^wIIKE>&xqqnJLNP z$hDxIw~>dVmKD~84ZJsHxa>8lJVhnM3- zKDb66y?gagc<$p`%DlLm&=!6!NI!^QzpeH3TeX z8Y|^am%ARScnkT3Dv5CC<%&Rxsr9c_YDR3l?O1#s&)mh;v=BvUQ)Gy}OlS$Ljz9LS z{AUu659rBEFY|l&WAB_0P;q^#5znpI9bVz*U7a=()k7OEwb16Ro-7>@WVH#IV01`s z+Xa010qgVW0gi;|{0bk*v)0G^X%eRlRv~PVIA(66X|GE(A33~ef*8Ca=}gsH+3j7I zrUN13?uR22L>de&2~c_D*lffxc8!)u5qY8&>ninX^@v5kPY z$-aQS1m1IkhYT$r^a1LA(P1@80Dy;y0HQ?uIdB-d@ZK>9MiplxLKtI8`#^8WnS2K)eABZRI|zYRN>hcL-aj z2)xG*@AHTrtxMFT#>|?990AweR|O!8seAJ2<>s0rh6C5K=_@RBT+l$5znxlTCHD|) zvV)!Mllkm#&1(o14WlJYMD(cFZbn)5?G}hoKpgY9U7LHacrHD=I;u&wRxbQ|bp7}K z=F&?^9jznaTF9HZ&4z2FNcp`^Jr1z-;B{IZ=Ff3kEgsm6UMcEPoGM?=BL)y!%XIx} zyY#jj+E!6@ps_Z-i0L$x=JnG!C;$}M zenN5gQan#h&V8hHU-u~g3bl#x12xaFfODC!NbNb%9u5vy&;3~mS*Sia-(C74DPA=? z+?slmc%%H0tUPqI7>yu+3A&ei4jTITRqYP5j^X&VAx;v;{6*I~B*05jfO+ z2GN4HG>YQ0Ve5@ar&o6T^jOQAoQoglFq|tb;6l}+mXS>%&ncWbrmi8l>3ZY*w zOymI-QZDzs^&$~82aDGDNg{wmbMF?(Nk@libRs%uW#AzlyA!tihL_2L#~#q|?XDsA;uh^aohZu$8{&u`_F|IPeC! z3?@I51`h0;e=T9ZR4$9x>baf%!LU2|Ed$pz>3>HI{|x3FC# z0lPb*A@Y_QI?9|wE9Rz}vg*JZ%1|U-Sw9^l5l0g^#K_7b2HPblqo8E9APZW{uO%W? zH+%J1LKbJ+Ld#_|L+N#z%W4T*qjOi{eLCD$6WLK3KxbcGafpE#6Op-(8U>l`G$FBc zri-m%fQ8ZTBRhA*d_0W9QHZ`J$1hTkG8VH|u_kUf0mmw!;%ST6CV6|-%jV-01bHbW zc^lsEJoOjuH)U%IS^nL#5*%|fdNWcG=BZe9FpT^(xaS_=g(MBvn?+{MmB3cB#J?4g zPd=(%-p_>aMxA9a?gbsm$UOU1t&DrB**Bj+{-nzwuj`Cef*9% zF^Cu+(C3?Kiyo~=?4Z%Rz_su3I!}eI=iz7Eg25;eoWhuTr2V>VN$<8;g z#ceszEZ=UqJff=BRX*ylQnt!_++bv%F)#mit0RmZr4y0CSEzqtUmGAG`^~LiVU&!a zg_%==vjj5v`Fst?0+7bRcZb${SaTmK!KXb>@#8&DT+6eaTOCRV#$jppyr*EL_*%?# z7)3dhPCzmtWmTguWTh_JNTiYF^b9?D_~)wQF~WXS|2V+F&!Lcej|NLGwwI zu~5K{z6a?Q?I$De0tbxUH}=Wpwk-k1yR0sWZYmL)<3--*MaM8HzCLrMfz zzNW9G#T_Y$gzwEAtV{+>24F;`CVWYoP4|;&M?J3~N z{7Njg{a7#GDEobjrY&l$B;iTOrDNW25mxtMQ8eIY%+nm;hliH3;SMLC0fSOG(4iX!1uI zT@G`v>ybPS@Aw-sbN8Er>8AX7PdEwZTCH#Ln$&-4{kXpkvK}(tMvlv5Y)^wtg##Lf zndI^`)F}mRsJ>lWe~l9?A@?ox+1nPBhFWG%YNEL8aj{Cw25|a05iVKkY7KeCR%${| zE6CTBNQWXfH^^IsU7L}UgPHZhcAsC`b%;UJ+A?8^YVcf{9R}*r{XL@~U+oaS+IV=$ z3ZaR@D|Wtq@*>yTodJ_98}up)pd~h|o32c#qJ}M&XJT;IQdbXlM}$Yl>!TUC%)7tZ zeY2nFAUf#n@wkHCL9Cf#t$$z>)Ncu{FQr6Aa1!ecsofN?2-ASqpB@8uU> zw>7_2K?k4L1|TT}Y^#rVi^WTM1tEqyaa9_5?+TggKb3*7Gc|BF6t%O^b?0l!ti2g%vlqo8Rh?h{uK2mlUj^K$;C0_(hk3)4TmttTzWb z6Kws~Jg^Wt4%h@J)Yu(uF!`X54_+^HmpRT~IWKBdwl)q6ooqjfdxqQNJfJHmhD~CL zxD_w#X!_-I`=HTse+!R-$I(WE`DX`{IQyu*=m=%c)m~HgDE~BC&!Z=lzl^6psbhT{ zlzr=_9rt*=g_~_G%JkPTV>>f~cNGWC`aXgu*n{YVr)^I-G?Db2kDV9ECGmuo9l-Z5QIL@PDx%O2(n{_@q+{HBn%cwh}8&bz*pJbO=(R?_spPz&$iu z2XWT(&`TXVcZr7lWY156o^@ZGj7W6-(>e$MeaTA)^Yka=L-{(l>Ti06;(F9Gv|b{l z4C~LyWEe7cORVz-+w3|xMMrUummbfK8 z0ws|cavxSJ9H;g5MgEhd2TV;zC7GS3 zdzWhQ`}=l}GYry*a@BCM82zgvep8azG1fCMi$@>Vw9AO}mn%P0JxORniSCY@Rd=S3 zqNJAIdAa^T(aD1KN+c>!&Zl9s{-&+!ux`R^)&zVWk?*vKoY7^9eevw|eDy(d7HNMD zJ1KX%9Y|~_rftoHDyfU)q=gK|KYbR_qzfIheq9pvOBy?USMy&=3k7Qj8&Ya*2(7G#WK8$qoF3+pU5khe+LfAd*y4vwt#_N5imL+;!ZpKXk|NSBa>qSk8iNVd+=~no*YBQrIvG>+t zRc&4S@DUZI6-AInN?Jha5(Gq2k?sa*=@u!GQo2(>x;wVg4br{oP1mMjv%kqX$8+?# ze(&pf-haRIpO@EOthMHvbHqLFaSyE%F8lS?F-Z8!K?l2#*1aBpCmOyJLpm=c%>uqR zT9U_sik+;iK)kpg?K#}h**)H6NrWLIy{C&wKqi4Fy#qOy;pr9& zS534J+*b?9lgGPW*w_G|bBJ(_dpp|#tIeX31aW)Wh-Ta{jJcf@rzv@o|3JNbZsP)~ zFfUQV4JTlC9WK?6d>6t>hu1uH+cX$2$QoXdD38aVr2Alp1afLFn(5aV!o`;tP(&+u z@@?_z1Qrp`!^zj3-B;JLv1rpv_Cah0-)5J!Z%0W<>#hJ|FAT4GO53Y^_Om5`SwL3~ zFwzay1#nm|4yQUYXF=_!87c=m zhYpM$l;;RCByRaimAUHSJedZeAr;G}vi*cxh}FhEM5013d0sG7s9oxKK4}YE&22yE zU02Z&m+5eq)$rX+BQNRGjatc3cRv!FIPVZa!mzIS@ohuTFTL0#JieFkHE`|ew!8gn zW#~s^xO9Xo50sShZgdgGwZ7ZGjKf0}f9_$V=xEs*f_$jH-%c&b|Zs5wH( z&wgXv58lJguJ#CM!T_L>Bw?!8Y|u7%AXNL?cR;Ol_&yq!b}~xuj*X-GBiW^>i^v~)e zxcLoTz#zTj-8D|As}0v)pGf1on`FM09&5{104SyI^w^3=#c#3E^g!%C0O7(zRwc~) z>!7PGBj@1}yo027?6?B84&wdeI|LNn4(1dL?=Y#5+t^NeCA;d{r9@9u5tX zm*IcZB;}1GaHL%cDhv(IK_il8IGu)`t{|msaDtsb2+A5`wys`M>O|q}loia^uK|S@ zU-3|VotP^s0Qa{%Hq?QB>V}ZIJ-oo!^E&m<)KhdG;WXt&jhwG4naHguC~4f4;hJsQ z(*$AxtP>(d7truzOQOq00xF8Y{-FHqAp}B6Y~A#619r9P zbc$L9aWvzFBQr7SuvJHanoAnd)ZJw8b>3#T$|woT+JZGpQ_3nb&dS1xh2reYND${5 zsiTl`q4s00V}Qo{w8ME(M_aTf!7Gwax@v1B#TRTl_yV=iOd)D7MyBMYQObY{g-s2G3# zl>51P#OiF>(v7xhc}X{l*WE>}a;-;DX>?<6WkA@B-EF+oYW7Qjk}-bTXBJY&$BytI zWS$&8PFIp33s!v~bhmOfcCDYY{VkTMV8tkeAw!5|5*O{IuR7mt8e$3@i`0LJv z_OOO3SUldGD@KH};LKZ_#^>T5HOV3=Hf*h^IPMf@3%imcUHK}Xn{a}}&yTGLxOW?^ zI?WxtTY`vSj{=~9T)b#fOyR4pOI>%{D|;~ok&6mWu-oHwOT-`;xuIKVXmMw#>yhAH zPUf@ib@m1lk0TpG+gR?+U&B}T>%W*}Ca4-@UX(2>5fz=cEXtm~<=?77)?2kv_))cg z5x}~@%47E!P+6oC*r?-T9vFJp9#pRh$8nhN=3BR?NBIuO@-nH&oaR}zeZ7|cI3)p) z0CdU3GIG*d$crujue0*d^cXmgdZUBIQFo9;;bQm$E`QnMuPd3Ej|;x(sp49)>$L>h z+D}#6VR#_cp!D6vPC~fK1c3D5uX4M_ek0Zbc3$LwdmldvCq#u~UQfNFmLnZ>=2D<0 zWW+FqB}3IhlkA37I0Gd@KP@t&Y2|X?AzTBg8_Aq3=BLy45D#MaldvT3ZjK4Qpn3nC zg%&n|Dp;LvR8SW^g=5&=%&$2gS+U_Zl&ISnRvSnHE-%xqDZ*NBd!64OX%MCPLdVJo z)atO5wBR{6ym0kq;41*`S1lb&mVOswrZ)y6ltzd8({$r~3SjjQ#m14V)r_$q)Kq-%({wo|Qi zaE~+x*y!6)2tC4MF>cGqs=&|Rzh26|z7D2MysU(6@|Mg8sPKev?JjQ*Bq7*bPivqU z)`Ie+?#rIG?fYHsu7S{#qcK^JaQxA`z1(N-()yF4>3QU*Wg_oEcJY<;?Hgs>PJF{; zxF2m178T1v-B)&$M>yKWR1{WX^bgiYi{#74K&sne0kgn#-|<a~agz#~+D_4WMpFyne^7bl&Nb#wHdDEL7qWn_@jzac>I?K@Uv(Qd= zV)qaFtCO|^i&-6lPwT&dirl$CL1(n$3BKT|1c*P+3RmJ>)o4)9>{t399cOgn&&KAs z$2e#nC+PO>_-biIq1xNa)DPKwd3=r5B+EepQueWGGkjr!f)lai+hwsloaf@KbxLNX zeV{z~&c^?c=iIDj!`{j6*G&PQ3bZ#aLqWx1*6L_h{1XHcVolB}sjL3l8I(Ju=ZFq4 zyfhtIiS0*=sS9kxi@{Wg#tUy!x+(&sX;PPY=bu%u-dgnXx5ISK%WdjPhygpzw?{Z#W3C$P&`$~z z78M%R@s26>L;QD?c7qPt3}19L(K2b)-tX>;oe&2`d&0<&H{bbm1TsY=BaPxaPoO*V z3iEoPX9DZ_veJmAl7Wy0rqDTzG9=@WoIND^(j%~NvWh8gGoIH(TrCt8f(1dLa?9V<yuHO0+?m^tk^7Rn~M&-%t$v@=6Vy|7vCI_iMRZ~9*R5e;%nKX489 zDZ}loNPJ+ig*`#BH1~uPhdX(uczvgKcTAkj@#y2jW!3>BsR&m&q+79MOY3iq4DRj* z89d!WciCN9a0MAHY8MY`g&Lz(;BVnQ?acn9UJ_d_GqY4|9tmb-c;+G|fka`Y8 zVerQlYwlx>uvvx5Uj7MFFPF@~0zRPCy= ze-z~}PU><`jf`_{cTb$GiZPUbGs)=yr?IFt;Rfd~bK1fd48Bt+?Ep8x@P@uhrav4b z2n4ynsI9?Bw9adQk#$J!z8q7AbtRJcViQGPdB=gt?`(#rTJ}mo1zuLq+!E7-XF)3) z8{V?>q>~`3geE-N+susZfJWkt!DjLLVrxA;jKC{UoceLgH>)sd4s0UxQfrQxvNJCF z@sKfoz%mF!^=nBV%R$f1`Y9UqAM{9l_)Zwu$OqhoVB8$YuG-y3rbxKH51tiYgF31QJ_Ald=nyZ z!|PDf&F*RJB0KZ!c>_kTpGbWXUnR@LVhK{aZK}E(4Fpe2fC0?qV{T_Y8P6&3 zjwUISQ^B(VsRpSNxOMdemCDOBi6tjCyVb!x5g-dW?Blyg@IQd9e?#JO_fWb+pJ)I) zJNKLM9BzlpW|dJ;j|P$Kdutm0P@yjXt&c#(BJ`H#m@4hiDd^yCkxEqql++XXMIn0Q zKER716il+ISr5%MCh|~JpxYmjk#5D;^HrtU!ib>mAv*uM9sbC0IX8Q736YgupQA?R zHN56L<7}6wXjc;p>(&^eR1-zC;>%lT8wF~T@xRiqgyr*IEN1@&F_iZ$NQQhuLa!f6 ztqi%G_7#8OWZ{tM@2@}FHPzYvFfx$jg?%oz6rRf8avjkf#hvJ zPyK{n{vx)Qm|5xXvKYOGrW*J)f6*iMk!d0)gdhR}fSh?6r?DqdRCGxfIR@WOQfuQ{ zL0#kqea741F+O&KqwU2@WvufnJvHmGSsel`fyn?&khO=LwPZ)@^-|?Iq!b?toGy}_ z>ral>y7B&s7WW4Zo|}MzJU@gqCc3PX0)oV3p?>?^+c09Dy^ZZ8L3%(MR9hz)obzSaMifYj6F- zmcT+W^d^OXdE6!BQEMZGMUvC7iVi?^B%-vE9_R==vZ+iR1w#?c8U{Ru4Sa}eLjgmo zl&qM}5!VM}W_rMQCLd4yM10?RgyQ42N_|HJ)zt1=sxYJD(70yy;WB`kyM_%_L$m-> zr(DhQnHRSnO;41TGpokAn=(6>@Fz4 zz!BUN;wfMMTEL+@+zqsgK7IYXxTdSU+Tepx;s<0+BBq!jH!rI&(C6p{<7rG-{}Q

)ly8GD(}%s6D{ZE!0_yI!?I(Cn@zO};;4KN4veDI5DJxj> zWtKLK^>vVwE?2;rbxOK9ai(f4qZ#e$4Y*)kWK=Ar4oZ$};aYDIK;=OtrF)g{q7Ai= z*XbMNxCPnYS%1`k0T9_4e62R@Gw8yB9P4oy-_e-lb@&=g>TzqivjqX#I$Jkd88%mF~1$aDGxi zcs@HQFN+aG_$WI}fJTGYljr6nH>JwvX&xuE?U@_Audz`yNxIYjo|?C}xdV(s6RM21bajgT=(SS!-@Q*g43$^H)3v_*w+v8^x}} zJ<8w8HEPVzXW{5*Wgm<@O&=2~6G(-cagy*a@8 zx?0Zv?){3f45M;-RE1!=8`@m27T#Iy`nUQF$Z)eF?gDIMo!f^4Y~XrKr<9Y9;El4w zdYuHjVTrh=#jdT>34WcqCbTj6lfz!EP=3X|l?j%IW|e%24s9QziZ^UH>p^0%9pXod z2`~p{Oymq=`xur4Q^cQ4Oky-jiu-1ZJinHIWAv^p@xj5w)!vY6fJbv^rlD?e?yK*A zxC^vL1lv2YRqNdqdWz1<6hSxjm*I#FNta)((6$|V)0=1jIT@sazWDYL?%N|Yp!rJz z?fN=n*2HtG2Qj^QT`=XZV$@cJGFKd0Fe9p{IU5c5NYl~_Ae*r_E~7Khi<2Vq%oH&D zPrn)9`vLZfeh6p+W>Cn32zK6979=kcD73 z#N)`V+@LFVjNkt{i`E?Odd)_?_`&w?!0T?WU&*QUJYctARzRI?R;|aVFE?hQ~j$VFFJA zh8vx4kMWSC9zijIo)^fwP{PeNSBpoW?fet0mM=(80;j~@E5I|@JdCOl-5#Ws8oS|K5tnwy|tmOOWF33C1`j1f+= zR#BNHAJ3nv_LG-6eOx(TQDMLWU#3J}-=VEouj_T61#I1R{_u)AdRVkW3cdX>C?6C` zzwU)P!-Gl}c^iFwEJ-H?A4Q_5gt{O!;G&wpN(-hR8FM2uB zdF3eoU@vo{?AS0wW=Onke!*;a&&H_sQYI1_GnPK8x72NPerj8}IiAxv<$$2ZT3@e2 zVCxjr9p>+8CO8dq&(4U{e;hGvdWSR66CQbkDK4pk%;-!bA z0b_#muUFH=f&*(96zr!BM*4D9;I%fo+Y9HXWFH(D9ROA2V0L`X;(DGcQ*ZHcJMFx1 z*ZaF(_iZlornMjAW4H0w^>1vE+L&oVSm&33tp#psBl2M(yC%FTOeViHDv`9@F|kxwoY`Bm_v z-F?$|x>l5D^e%fw9=8|Ai;HFc3`hCnm}kAc9h}TC^leyI-OqJC4uA!gXn^!0$@X%% z^We4@1!R9?%U6r~QV6S;p~7N+rw48i->F}UUaO3g^oRrsr(R?WoQ0TxxqffgjNC*s zO&{h5)A&$s0)+WBhs4Q(dL{~bWV-gM4(m^s9!=@Al8C5d*B19*^u_nMoZksPA9iRs z>5g4|6ht~z?SR92x|ml1{Rljvsa(5Em+na`WD>7@B40^UNRKbw5|Yrkb-tb1S3X&b zVdu~wk?DPn)SW8caW_jRg#QrT?PL!FikLoZ+~=IU?e5&nLSNdwVXDy0dPq1l7OWy> zS-)SQlNQ7wQxa*|b8Ord-IXE!t4#+&sZyz8J3vvX)>#U?t8>mLSyJ$k^II}#myv6| zG6z#Qn^XmL8{wSc&RbNG_eAQ7KZVbIQCy6k3tdPP3z90<eaZ?+zq1_}UE?^2yd#;^6o*_R23bHA6#&L`36guGR*gaZ zm}zBoqoMjM#}`@=7ws-)mEXh|^a=8$kz0^M3$BguTO}H2%d~=O6lt%@*~XL=Y`W^i z?8?*HkI?2tfZ-Iu-0a3;%kMcJ|5afdv~vOM{+iBNW|B9MS@2^+@1&@>%E;iy{LP4-18aY&OeRrmZqvig%-k3#9!!eJi&5C=gjDf*0gco|DG#Zpj~3?PD|jnduqBp7rWMkd_RRUA}q3A{^!s_Ci%j*U~bpvU>y z1$Ndzwt1vcO@=fc%T@?_`?Q2)^zU@;e=1(TV(bIyZC{am)B}rkah;}jvpSgqylU;m zEwM5er4CO8>Kpi6AxZK@07ZYK5?a@tUH$UgNnTiIF}}2{xF|k>)z>PWwoVp3~=fVzsL|JnNO{`7Kor-#Jj(itH?F~(}S8EE!OrV=zwH% z%?yPZ!cfDBZ&4b4nK z2kjlwFDg8U=R(&n><5u$t;wkVfOMmdAK6{iax;lv_srbC>1iUL=?%h6Y;XuwjE8HT zMs)*7FZ>HN8&BiKSN5!?c>p9lZRrj!dRSNsZRVvpu!yK6u8xF*E5m<#O0NAL{md1w0S#R=jHO--o24!^c0(!9bJO)yhmgbUHjl#!VKtTD+9QkF z79G$nI&Tqk=^xNCpJ|aB?!V2u^og#Zwd>HNTdixbU;l9Kz(8o^0>IfA9|1Z%dOirw~bEwmtvd{0Ny} z0qQe^gb$77;+UTusDDkMb}_{a{Pk0RHx!=(Q1;j6K6S_Z^GnF(vVkay0|tLWn| z?;w=igFeV0`79x!&i@CyU8s76jRCdP6zX24-Ya_jhBEwWt!3`O1`n4{YNYg!y4C)CtCy0%4wgkfy!XRf{_38fx%z8DPV1P$ z_jUfu?fozR7{Udfmx(6mzquwy9EjEZ`-%SjL?F*ty=h$O%v`do@a1gGzTAO-IrB&|#Y_v0Gl(A;-{s2+ z#CPn?Cwzi9$87Ch*L+ud09|}BkN`!yT~s2o8yr)%@a)(u+8@u9XPH#8z89~SyNT6> z$_c17tkL6acB$uxhvX?rTET#jQG-n{U&8qd6)TwP!F#s9N(ex#MH|p$XH%|i;mBvo z&~Zh&Bka*!wraEh+xScv)c^?D?4)SER)j4;DViBsXN&=LI0b-X0OVIh;CH;|ws*U! zJaLr`Nn6Nsj!dD6eB_2yCPHqd&J|bZL_n5_ zMivY9r|CBJ6k6I!RggAVGj~Sc%te>@UH0a?T^|F~Fs^^|^7E~l!^8{q z03W?zew||~^FPCNyBEop&;*fA#XS1>8oc}TpPv7y)Z$sD3T%w3n~QYtUr#PHh3h!) zQIP4G;C6C=ak}9l#C9C*$NuS3#Kit8$x=_utBqmA4$ z-I;4%aajnZmK5&1!!&@5ql40HwjSmK`4%6s_UK_1ql8ikSIY!eJ?G_Vx5mA?T#h@| z#21)FD+!cnZ74^}n1F#e4jj89v&~7+-~sQeb=uF(W``}`WnxddT9t%Rh01{0_C1FZ z3Km*|eu(iiVGzV^AP^V zVYS09V)~<_9#F;@GfI4hOylQxhg^YD9=9x9x<(Rzb}w1ttIZq>#YG`y7SMYn+^tVV z$7z@sE88>Te9o^Cho9+|&80s?w_hgz;r0wnH#~0QU48=~fM~26^n(&m_Zsh`FtrO{ ztg42hgpA!K(Xl_v$&ONhm{gPSOt`w#CF7nZJM$$d^2^bd@8Z{F9KoZ*2Nsi~QYD}Y zNY-YdW!oN7>>29;swpPPy*Ryf4|1WoDxfYr@9!Zbl4Rs!)KVJlSWWP zqoSH+829b$)ANt~DWO3}(>I{BV!=lbn!fBQ1BM2v{g_=vVPmixJs&)Rg2e~unNV{Z zX_iZ4-We408`_92tTeN)uqox@lL~4Cfbjs8Rw$O5U@s__0fTh=5Ydbx;}QK#NQwl$DUgtcBVXxCs|bPvB#I(Td>?L4q!+5tq%gY}f})T6oMY ztc#`?vjnU^53!IaN{ojHcJ0yb!lAru1|;ddR`ys1%zn17G5D!$j!fL>1NWOtF_}r2 zWNv%UiJa^dTx!0pLpq>#GYmJmF$M&wSbbG z4k7n{9HUWfFIk^5%G}*{i(TMs?V|^kiXEXhQM5)jsoHPe`VeTJmR!2^5%PX&YHIXdYU)$8)Hm{J4jQu z5<}jpml!g!pbX7<)}nSNarQEyYMB~zM&Jkr>0KB44duq0->#7VcImnhdC!%FqEAmh zre+v9_q|^Hw$uJKm?KoJ=W#Bgt zG*_7+#bI!vsF`LhqWWjyj3&2`G0F0^BQ;=B^T~W8w&beHpwXwa}8?GZr)vAoA7HmJpsD#c%pPY9^JpMJayse1Ni&;BjQKfJ5{3Q!RxAu}X?AHU1_2 zJPA3jqcu;(YFiVA3fVJ)0%%B`g{S*b;g;EII34q#kpVKU)OCyvP)M>Qk;5+cE$j!s z>C{Pbg}-wfWD%^wsmc}hd2gYa77sX^ZHoEoI!WBtb1_DCg>7&u4rHa0EdDqB5d1Nv z9K}ZT+l(`(6@6$^kfYiPyLHz*BvW8*a%#ce=_gAyf;g(NY`^1O{^EN$8V^*Fw7 zJMNtZOMEJsAIKMRc$`}S%Fo|iUKFV0M0$`8&@_Y5h{r}l37H62f9r*&yFu%dL=Ge|q9*8U$K@1psHOLw_tEtTFq=?< ztIJ0*+$2RY00)E|+lf>?79JKTRc=Tyse2HE*K{b`9ADj5!uY1r7&PRl2MmzXgSxVh z0_~jYmwxBZY%dA?xZ8e1$)d4Rov`OH))s`Am}X%}j7&NegukJ8tPfH6mJ2hCEHhD5r&h?YltrfdlyTY{3 z2gf#gi!LUFmpY!Y+jk`iIKSaLzdG`wPBnRLXzj8fYPMQMJq`nL$lCL3SI9pIk@LfG zc0YaMMnPDp53La9EPE$-ns^x}^$J*PGC;~r^VjA){N6%V0?*xZ)oO-a^4DMMRXj-< zy}YyZ`MDtVO%s^}wlNG|ylO;tPW}NYi&wc!9LEqU)-o(LG0+O7eLUsE;`kYl{x&&M zjUFdzqlb&P^s!>f+BNK>4i{|1X~E94myw*-_W9Nz-LwtN38r;>_Z9I%dV~)j3B$>= z5erV26cJ&5mnq9LecDu#neqN!o~ocqgN|@4=2?El_L|~TmM_e8>(6plmYh%qd}pW0 z4sB8V+NZrD+?5PgR!Z)5E+Yxd(U<3!;<9-uOiPhAZ%yIY;K={64MHiC@7uqYdXXm9 z9o?)O@%&{q3*@K^gBQpcIhxw+W~IO$1X-6IKb4UJ?_`Ou>FVzx7F7E*w|d}Yok)cx ziQ>X~eBV(QhQw2-9M>VydTC5CQ-*&lf`ZMvx|aOxY~ZQdVlBt|bRF+QAuQy*`wP02 zh4N15S&q|J@yGnOAoFz_@vhI*@Za8};H#-MH5S)jX~e03v($2ZER=_p^CK|TmTGy8 zZ#o>TtJIr6YyI77eZQ~gWf38YAf(N#pE#GLEjn7R()rnO!j}@?p?vuf48OOq-+8x%DPamA9D-hY=_UXIn9-=0S20)I6Y}eOTvYp^wzYX8esEkgUImYw zTQa3+GMMD7M@!Oix$oe6M5ApVp)NX0)KQurpN?jofvVm3`^e5ZhZ$=bZTIY3L!DHq zQA$1B7Uv=qe2t>}pj-D~nhx)dW+HQ1%#t;8<+g`^PkcESR>|wC2kFwT z-jQe?A)NkF6A9LepLv{kBgKbHdIV^}a$Tir<2$Uys3U+^JclTgZ+00ciQvQjS}bDO z86(&%|7#5f;*K$ro}K5gh5_5jlTY%Yc=j|(hT{(Y@s=~Y-xwj`Wy`mqem4%5j(^HY7= zmL!;TNL~awMyQUTf%w=Y^{c7eXZ`jUDgFq&&Zw*Hnw42N`)g#g4MB@I>05$rcs!Q- z*$Ss&+;4v}TG#}hVnMOW6>&8#1=bPtN=2Zj<)rzur5|=Vr{X zmos{0kWTSX4o**r$L6GyR6}>2%OTIlAxq7NcS=RK{Kzz0V1Aj_n3t|M`3X5Mk-g_- zi7v)th@1#q8JPU0iR3;Kf7@HjeA}s38iF_8MQBp>n&i%|dX;PQc~vk|9H-0I%sX3+ z{A#{QWR7L(csnjiU^<@Z=x6J}s))BT z-NIuMYGa%HP1CfNpfp8|At%o}okKZFHD`!JW=c086Oa4-Shan;a+}|xCD4RbtHxm? zxPzKIB;hMsQ3s=1WZ}wE<-D4hoBIH}Lg7cbID#~l+-jVm63bGKP}Ki8lnl4p6cc&9 zh>Gmi?x;QRsE1mA?(<*c%;ofFTLkSSBR=Q>hQOrwN3IW)CbPPYL(020WOAREM9N~+ zN&{KeujOWAxcie};QR5OcRTX4=u_{XzRR2xxyLz*y)FFCIiQCs!Xzxp>7=$Mxl7Eu zl01&KSE976BjC$PoFq!qrjmD9$_*?6Q>4|2CLKj|!n}+;d{+3ChpW3K4mSEI4q(er zw@;oYSi##m%Q-b4TvztxoD0W$?8rZmT>oTnp7JuQU*>4E$>$(ZA93nt0>+G%JMR-7 zK~&JL!56q6$Ujt79`PM9j%3k#8~yU$(U-_gwFRwO;}oPDT7M&S(eq^nwB9)5!4i0m z-Naxpf6el}>%|Zmt%t*wpK3*#J4553bQ_hDO;!E=W`{hoy|`7IqeWB0x!=TQ>Ns?d z3ZuK*;y6PNH>-3IzEq^r0|T&V{Ck)GjHNt@#gpscrJDKbRnZ9Mms}L7>iVAAm!UeVdKgVFdYlTo{LUEc)dn&INi4T`ZU z8$T=(w$Uv!Of5#a`xs0X&!XvYZWrZ_3a}7<+YP{z+%5!%Yh2ELCPjF*^V~Y6E3_nt zh?6Qxn)HwO?$5(-_2#-UU$R`SGhO)P7f*y#YWg>iGYi^s0Xk`=h>AM0da}){319uu zH8MZcmAx7&=a%7fez4JID6W>$;Ik0|b>E+z!F#35QkG@EQ9WEfbyu<0(cf!~D*vh{ z@niBjI1Vg6uFx1cqBU=pck5R0`8+4GBS{c~qc*+tNj~AQXYXClydOW`?$c*BBt#rk ziEAH}dMLcEiF#}HE($l`!&AnQ2|U#9Skf683rh}E>tMLlJCj7iffi{2b+<~+p74|0 zP75p89hQ&V6R|2M@tD}f-`A`S#5f|DDuHrs*yNJwO_YxRa>N2n7U_bf>A5zx;;J7{+@Sn zk$sq)W4j^k1cgB%I~;zdXW!s&o+}YX>9z2A>srTU@xg$P*MIukPpIYUg+)!LPV`Tm zWi= zhT+)h4w_Y9~%Om5usbVpc1UAIgD}FEwAD?FVd6-AMmP)&YM`FrM4AmDnXd zVfY}BO0?Wu+P(JUf_I_!+11?%FhNol?(s@3&xTHY~pIMB81HgB@|Q zBe|V0TEsjDq%6=L-2^-~&-Es~^T%LXVScR#$iW0W^y_Ij^Olo-i|ngO3_A?(<^T}ZI| zgd&dJrF_FGpzxa+P?P6yJAtsm(-K4of3g}M>`{>1MLh*+?&ZMK7ERSSwB~A`lzO;n z1`(Fs>!7;+k!l8z5{;fDR^6yaoA*~vW$>5O8yEOE5S+l9dWTMnS?AVR!&QvXg#zXJ zE$t`7vZsYH!ZF>G{fo#^sD#LgaG!3Sc`e-mJS=m``0M=r zRG-5iXhbepa`-?)+N#-3s+5c7HaS1Nlem)Ad~RIc;6$Qu^xB65hjHdbz3VYmgG@vW zFKrC8EjcuNTE1O4&MFhZ`{*n2$OB}ZK+{HY>6KakrxgtZQRVH<=c zv*c@H{v?9&yZF&a2vYCgpX$~(*Aya%S+V`WkFeIAzYwBDY|&q;l4|9P{>}YK zqZpVst9shu;|(n{RZ4wTrk!RLa{0u~z!TQ{Jahbj&)Be-=&#@4QavnXvF51)80&WY2>&UG_vPb0fy%}DMeXn>H*^_>yZPa{*ct6ex-eR`yzw$~;0l-rPf;Mr z&(ohN@WOKsZe}hOW5ePt)hf)20Cqgaz`}0{2G5rF`;l3IVT(neFQ;nP1@ZQ=frG%$ z0Uj|I3a*i+?G80ZGE#}76O9OOx-t;%t<>)`9x>7#!Jz0EazgLmTvL)~m^Us#C=dC1 zQ_~o}|K-v_t0hDgIJ@;g_FFBNfRb>S?9)RioqzTa0cE9OFV1w?vxuDNCSl1aMr(8R)mBzqdqw*dkZ65i^SsQD zOol81S)^$1u@kH;#ky)dx))8ow8q$vg$4cT;R_Oc*FyWZ#5?#UUC_|TsOQcQ70(Z3 zLO3TaC7sCL+XTcRKSb#|aVm}%wUUr!t3w1MCe;I%w4G~@V-@lgr=Er1UlHuQYz?N@ z>z<4G(o4*emoWxgOgG$NQPtn7M?~V^TSlUY>VI6;f{VivJanFRk*UA1d5W*OC%AUH z2pitYVL2myp+H+YgsR<*)h=n`iE?!kNoItac?6x@J5Y(c3B0W#`o{UW3x6(%S1ryz z`>Y@@f4423@#5s^M&DjrG|q_PYY{3va4xUfD`v=3D!?(@BZG`-&9_z=m5*<#So(R0 zDSYA;E#yVM4VYqoIqus0>2QoQpX3wUU%vMbyD;`fXb#UyYLtS>kj6-L8*@Eee!{so zR@GH`{f_c}sL8j3+!9`8D2o8jdoi~t{GM7T`1m1ZeF;E{PT6Aj3bc^ddFOLWrW(vQ zsbF*)!ko$HU!97hf%?RcEOMLZc2J{C1gb1!7GU1AOT+k zknbt$^nn$MDvM;2EhKzIDN;;Zy4W;qe)e^g)#JpfX?z>+9d7_Wiw(_T4%vQpmwA@KeZk?ZZ`iCr?M!mnyqzDR@yyF*OCKY)QS40S~ zRaY)H7db05Ldou6nk-ZHlyiaP1swatMQ|ltK0$o;VyJo_*#fTgOd-g-K!jN-8?b-d{0UEe5?KbBnx``(BC5@VeTx@Dp0;Ueam zGx`css=iw%e=97>c_ybiIyQC3Tx)?sfPY&u$Z%01d_pkm6q?5Oo!g zm|W9`T`L_&0-I1$5JsJn@xq<>%<(79HT*92rw6YvHX5Y}pUSZo1iq&23h^L}?F~f=Dn-Zq>V6 z?bE<^oV?(}Dt!q`9o<;z3wiXEZo^f( z4>U@Ri>MOS8c(}ezsEnRdKKnSFdl^eB_Y{{{h~3CwM7QbiOy7 z@aG7(OY#k?+)^E9mIF(e)2UGBU%}zeZt3S!`D2Rn`?o5mcvHzfG5L%K<)=sf&)@yO z|E7>VIk=j4QuO41zsvt{^}qN2cg_Ft#`$-z{o|$j;kXI?d&d56XY7M%^X{TciHP{} z>@qTk?q;ZDk`W$GxdxEI_ z$UcHeiCMlpMC|czC@~?wL5O0u68rA`6@GzKWScQoe)A#XXEgT7g~vdU&Q!6(r-@X5 zP8I&4#mA-s7XR9`ZS{ZPWq%+Izj!;LC2DZ>zxV#HcG4T6f7kr~a)$riYyX#3`~UA= z`#^i`AcpxbQpN9!#~uDJH8hlMoe$g7Sp++h_|N5uMnbcF~Ih>km4^ zctaNLSkiU=^IU%r5}F_4w>Ue2Ch{L75jAcaV8*4@6B1IwUe7NzPF5ZlPF68-&My($ zAy*jxax-oISIRBMbk{}-E=o1g!4(!$x-||pkN7!hc)}wm_(NdrVXf$R^kzQj_~p9I z@74TqsS!J#7Wi{Lm!+dme$Em^WQ2r!NNVgyd!}lzoz#qlP%c59Ryl;+8rwvhigI<@ zRXwjguA8|f^B>Rtn*I_Rx5L)>${p6bjZ1@DZ%*>8;hLNl=enm!8(kYOl`V2G?4L7%9y5zknQZ%TLG7%gfX9HuUB+gtw;VK;OY0AT~|9IA7 zGEWT)3LAXA(RByC`9VrF_x+AF@OV?YY(M%q&s-^O@Mz_>?Ye5!wnZD`F@=#z*?=>o zQ5}fKqB)7qBO3#tH2#H;(9#{iKGZ1hcEX)1)bYwwDd`x_SJ4Z=XWC00M8{)LOMe!j z;eyeOuqWRP%sSLwODjcQ?>#EEscy8iSx#@@%ryZB!MU|%2iDE#17=cKeZVu#ALzq- zRp^uJ&i)t}eH01Z7^EOuMO%s}AtBtq{%c6#343)&mssUF4+W&|W z+UT#G!)FoYtiIj!oQz?m^0VU(X3$s1Q!QA!5bON- z;oGhNX4)+sBbv&a*2Zre)R#3^6Lo+50SKmwDK9y#PtK;w7@v7PCeORFSNH8u97_2e zZ~OiARXpGD+4Ur_PhjwNMle`11#=5@qVYCzONM6)d}hJT$+q<5?B9O9!Y?)%qxm}i z+RtnD3K52vn^NCP6%77GdDvSvSSd<@ED2heNb8%!q>C z3e}>S*-@y0A(i^P{v3m6Du4y1SMGl48p@r(3lus50m zqJIKzOSFYVIz~$n36(;h6V1g=Epm+Vc$yz9nZ5%GIB>SpOAe}w9SV7l{-I;#HLGza zm5w{Y@=1~r0(>HF-7n^P@E8=MH>OM%JED5!GP$`ot~8y;$|yNMH(D42oeEAjpI)lhYsCh)s$T3#I883s1I_R5t@^+( zV6WO${r$b$(5~#|R@fZo$M+2pPl->>zdc6R@0^O~K56McesX2O1UBopSy8XGSH$mv z>$l98x3O|?7EquJ*xK0&Cgb|doSmE+rh#FE2+QYO?Z2l`5cm2(--tHcJDQI zAsEt$FAFM*)A+ahD4+LMkNaCAW40XzYqy`2r5k}kdlElzG8x6SLD-;@-~Euw?TT#y ze6c;QK8za!B_{q&Q#_D_i^C%1`htFk(by>7uKbZLDep+0n$hLc!xF%?(7{je2zKL~ z6YmV8m6Tr5f4$3{=|ug$WcDF!5_E(7Zh>NAz9e!5=}<3$0R!b0DJ9fsFP9sYIZg@*4Hx;Hw5Ih%2CT@@Y?;nv)K)0Nf1kuuZa z(G#pSQ14_c!6p9~|HQZ0{}!?+Lpr33{*a+@gIHjlmuL%h5n%MiAncg1VW|zD;uW+7?U4_FYsPU7wCMRh= zL%af>QRQxZi-^+F9npX`Z`L8dk~1)G?UwEyrQuA#lJ;d5Ohmgt+scL(6!Um|dVME9 z!L=hRt9~h8qh`H_x7tOaLS}s9N@9`OSg`?(WeS}NxubLz@$uigjQ!RqNv2Ox*QEG( zq>#`dp*1bkcm3H+0i?x6#|SV1H>M6zY~S-vpr9xI%hP%0 zO3Xm6<7}*~G(SqmI}%0y8;PkUWIw}h2fkj{~VNq2WQqeqW1w*4;O z_kN!D_aE$Y?L3e3IKIa>_D+9RYMAa13nG0>G>c*Z%dLS{8lKT4k+sh0m_o}dz;;l~ zsYQJEX>p-xZnsA*84Tb-%wRpj-O1AAhJbL@g-CuwJ6iwqu>3lr3fOh^&@VAjO($1; zYJSJ#xzmsH1w(yY;nPLUIREZ9()a%Re)MkMy=)lb)g7k$(=tqn6fC@SU2okXBz6Br zDBL9_>}TS;#ajXhUsSuKU>om0IX&OYdhj2R;+-n6uo$-^wNsp>w2`=f33x_u-QKtPZz zyJm?td;NBbe~jd$qQao&-Rf&3zeZag*Z&-aPpSM4r%&>h`d^Ws^X~X~qto3x)1~f( zUJHELA!$YgguYAv^>*@)0Ij_n!=FE|HA|m8r-?^~5RwO%YXClXCBQy=$IaP!dhRf& zL-_*h5zX%a=(bu_hEiTw(E6Pw7ZC?Q1a}!ZE0xPC-wAN$Bf;3@Pfc~D?*ndMt`Afs zOINO49&oG8p%JZzqE3&fc<8j&hk97!m8Cj7q-pyeub<9uEuF@0)`W2m@X-2uX^rUzF6fPsLIdt;_u%($Q>pb+Js*2(3#R4~V6YVm`|lM2$eH1PmcK-q`kZ?W zgmZWJP!CaC$u~4|KJnQEk%r0&b3&JEX>dZe@t)+srFUi1DTh)-|BWFpo4S$#SnI8d zQy_K&$Z5>>M=b&Tr-}&V{wKPR<}InsOFOx|V%Jx39_wK?xTxn59^Ly?FTpo$H2%l_ zY&xw_jF`6}v{=;of;Dh;>P5wO@0L&+(4dfspTndGX&eh(Iq1~fKUr2Ahh+v+^oIO} z!2*I5-mBgL2ua(VdX?&HL7-hJ@3O24iIs_kUxT>GRqul1xd3^!7e-aSjl{ezK*DBz z(+V(0)#odb8vr58?4F$+v8~cTe0Lr2l95L4X2gqm+;(=`6Duq8lbrnIUTXL+i#Xkrdz&1qMDqDQ`N5a;S210I%|P+HJp zhW{Rc&*?5a_P(&YVm+?8D_q$9rUwDrW!KYyAzasHk$LYl!|&m>A(^XJ`i#FT|L6M9 z{gfft(c`9>h9|Yv*>&`zW5b_5#U(Y;&Bf{bm3zCw(3}jea;Af_8{T~L z>L_BSMeNV-BAVN~43(d-zF?YVwegVXxQ`Na&A4wz^N@ijeR*GYJ2WJM82fkROBTOzZ47K-?S%ZkV2zH#R z!GOE-mtK-oF0+(qA-!hR7s5>-LgBN)nmFmB?|L7kyMdRaj)fLHNyY2<*$>SmT^hkk zniQ>ffu>Fx)*$$wS_Y72zNo#E5gGn(g<$GTU!$K3ih9k?yFN_g)Urksf3F!oe^<=6 zq5an@kxSf?ZwUoR3@*Z|>bm|R;5io4D^1_y&YllLU6))AjEFZ1Sqi9yH{a0J7Lgq< zxvktGAF}&)uR6)Z^~0# zo{ng4DfmbRFZY+lttPuz!?PA*@oL~hhV+}CXN|x!=dD`Ef@AiSKDoh%t1hiFXnfBR zM$JnH(y=c{^JV-;@03W!!;r%Z%*aD3sAh$6hpKQ(`jUT-(W`HCKYJ8K(^Qj1`a(E< z?RKTh32-v2r1I9(IV{Wpiirqlo}c*VU&#KVh;W=|SOV(@28HB!(o2B=^AX^YQghT< z0y0J4kTlWwdr{hXZ^n3>vDfQuFstNfPwej5l`2dWXxAH`E8r1u@Fvr@lA-~tFvdBb ztNsDbYomy!U!A!D{7JtlJi{d9J+@vRMaYDG`R}~>0HC~_8Svx z4f4UtHZ%Xqp@sT+Be7I<4A5l7Ab?Oe*a7C6amj{sZdmnx5?RGmC!e@qqRCC`;d5E(+UobNm6TzMLR zWCHs|E#u>No(;64#lVweE)J`<7_ju|0d02v6 zzBz>FcSqn@`nHeGmtndX`hOX1&UE*KLqDrk%%~8ESOmyxx~pcSRNi_k>M-rCU0=?B z%^MChCv#7a1V7llBsA0%Fp>&X+d^LFdB)F6dhPhQb#>FrWTF7VAijwH54yKLUSO^R5eqZK_(<4znGV=Qo|f%J*k z>F6DLM5IJhaKil40-d}elegkA4QDGw8V7*M@uRT4z|)ksGpJnLRR=Ivh&5j}Q3t-k zaeP>$P|% zV6Hll*nB)OBk>eow{Y<)J ztBtQJmDH5|-Lzv>)DsT)<-FZ%^87TcA61~ z^4S1d@Sz{gUFE;(5gyTWYQZ>a23UAs)6Q8tu(oW}8N?0VQaC6NGz`Y!gJZbq|ABjqxp|d@4(924SdI5%EV0-)gpRHAu41qCnRe)XXygoh#os`)6 zk^FywvY(G{pEgkHy0#DBMPv|In1L?h|G3@gBGYy~+Olla}6kw0wa)9IxQX|IGK zp2uj;QlQEK&AZ1SSI_*MRoxj%Dy<{2_CeA))O*e%sj%)<`~bty+CgAap5n$#3ScHR_GGur=R*MhqJ|XI-j(n%EpF!+UWP zr{m7olmvvY9>idLetqSvcFDZN3NRi_c8rs$iJG-|gDqz~cT;+shw~jju|}uEqG_zk z^bc3)nOzJEOUm!7?A5zA_hk59>-n5)%`7r>TNpoQ;mjBj!{n)C@Vq)b4HH`#XP1~Q zlA#OPZ@u|ZlEyM}m%nB{Ki7yiY>-HIEApLJ2z(bxC+fwi%ZhrtwgxZUZqUD=S~d%Y z%3A+J^NYDHfgDS^u&InxZC3>qK{v+-4qs6e+$^3A6Q`e6q}nGnUs-!J8)$+sQ%O?c zhVzH7xVDJ+m(2^)yi;J&-d=v!eLaU;NdWjVx~}yp!AJnauA9EF_C`3zm|b}7%B*zY zV&)33JWF{K&FOfe+MI8G+FJDU&S`Zq(%GpDaQlqHT-$xjb~g8m$t+Y&ey$kHLJSHH zv()zmVv-i1zx8O8Xh7#D)?KlAcCZ;b5`(20)d0Upbl=WQ_V?JH2A=L&8#x{Vj#WyHwy*YbwLoqI%86`{WS zCR28k$5WneW3qv@t44tk-D${xyYcKR&fJwp=r-(?8x(UsfZ(&dxhsr{*T7g#7&sk~ z&zt}DrsA_bC`sLe%I+a}7LS@*nyL9S;4Qnsg-)I{k1bO8zbDx&qBl39y-W+QF*D*C z2LJ%I;MoDLEt;^J(-;v!bQO~pw_bGbf8b2bz=rBD-a>@HGdJL0dA7kD?EKR>vk1)1 zKJDe2T&4gbS8&h3Y?EoCk25k)3qk$ z1#cqz7X}0NYnx>`H?>@U+{87})jSyYWjM(3PH}K~OaamFrKaUo`0ytEdD4pPLg%Xq zdw@HRwtn@OPlrBg5glQ;y#jowl!iWaqMpT9-J9abQIr@ zZ}D;k=s{xCn(}>+KP$Oe-LBR`7u;u91msY^tw!ijUt0#kcL2kHw777hY`dK+;`+n} zSBv)TyTxvi4}jiqG_7{3qd!ljFO6?Ca+eSYyAyi^ zi;n1&8!|O7;dC@K{^JFZXupyv<$|N}*)-)m(A12?Iv#-ALMVJp3qL8X&gJi`3~#u| zi8CWsDolzIZE_r^=2!Y&#cFwU|2fHevF{6zix2;`{ZLTfGJ`H*m0WxSkvlC zND8rI@e*q~KV`dV@69D<@q#`FKsH*aB9!vi(F#An!l&)pXbVlNtUjBPr~@ZiX{r3# zH--A4-6TO&?<f6>EKe-pvTk1>+t7#$jF!pP8jtTC%cuODSt{kxqNhLqN}Bihtx3}M zRuBz`Q*fcsDqCqeViC?|-o%I}Mv)16JJ7dEvd-l8e<6@$Wd3p3rI44_Q>QP~5HR;j4GqwKv z6qoyMvZ#`%2whnm)*p*v0fWM9)kMo2e!sdF_QG`-I|exsB8IkqnMDz9h0u2HV?$>8 z{^i%wE#GBm@#xOQk`J>riG016Gv>0 zT@L@GykG>$F()^JZZ@y4i?cIXJj{(O%PXTmirU*R`&WSs-K!lBItJA0#!J zr9((JuT|fExTrB(W9^~UGaMG2{`#dSrfu{3`eOetcBw7;10D{jy}kvf3#o;(>J2q{ zGz&5wJe=oXexStcUtIyCA$fo-2)OJZKCD6{k=%@F*sT3teq#-;UGsMqs_8+q8(OMdJ^ z8ZtHzkBoecVL$Aq^rvlGP~AN+DY0Bl)iC<^v%bwDP5LWs7HvsLEUv_G1+YNq=a%ZttkiVUA=lfx)sgevfma6t(Q<-7c-R7X!UEyPRF~hShKNuC`E8W6C1%jqCCJfuH^wAHuC6kSZb-%h?xU$Sn~&X zG>)$lC^0>>9t&mlUZ%fAe%9Gf^~?CA+5j$8HNbnkPyt!=#NAob`W6*eVz6&A^X15# zjQPevX7PH|#$h2vj6gjL-9}Q${E?==2drOxDL)xRtoxl-pw9S63vXe zYXrHnnDrh4tzylSVSOH_F=V(HAFwz`MUtz{xuA}S*1p=M4hzUz%!PofF9W#OHB`*8i}n}`=D6i z`hh#Y=es3oi*2jqW9^QYLEn{HO==)2mdBD2lmozu&WXv+c#90jSFR&az+B>x8l)6c1zx zAzAp<@6&$c_vx;y5}YxpSn&4#^wKEsm-TT@uztJvpwZugHymPC6Lj{dxN`rUXz4xX z0;MF56I#gqDNLPkU@0T82O2w$AfiSRQYUaYC5ejfrOS(4|M5d!wZ)xgepze2NBp9o zszQDU{2#v)vqD<$#c;?dS#c=_tEl};3f_KTg@ez zwE6;;Ohw&N!cz@zPGGd@9~k(|LZz)+ots*JKb@hfTC$qx`RGbv%ev%zrY9B;E0ity zf7?gEgo5#{%Q8>qOr`$#{eQUB6lG*NSq^;b&RV7$^fBtYkI7``lDLeFd0z?jcK{wQlqg9b?L(ys$B zbt&@9TLQcAvo)xlH~EY>B5s|xnEK@y{ouK8{t9p;=RY=6k_q^E{DFpvJE+VvTpZW| zpPn&|bvj4@Zf{D**q4uUyCH2kR=^6tVTV7c#aewsQja87BbyZLvHauQlcPd9+~~@i zoe?j4BdCe{x*QcVp(?!8it}$++9faQ#_C2)knx4cO6-4GUDKcL) z9dIv+lj_ONSNgT9<^zdkb1`$-BmHp=8U=3H5fE!6PvqK`0&fc5nao;7x~8hzhj;Av zWWyD6Mf`M9`Nf$F8n|EHXyMu_H?i12yhVovvXM-@!}&U1dQn!2ynWFOBq!83>R;B} z2{VaEX8u%-rR{vSkQUfEax#r9Ua!$LL_A_El7m&JPqajSkusimoOrT%+M!q&akKJ> zdMzXFwQP&Bs|7$^I+98Be&oNqrBiyidnmad=Y84Qq#?)h5j`0$yOqo{9$SKTVsVYH z)Ce4NYJHI?-#OPI7!PXhG5E_;m}@~EdVVF<18d(vN6T(qB4+p%CfJ16oD|oG^|-CB z+_tuiH-hbiOf1M)uw=z{fB@F`cm7E~_ecwgV)RTv4wbZ>Pi?JLhTrny>Z|Sit*^l1 zKuC}{5{-x zmsTZr=byocYCh!pUseDp({&^bLbt|Ju1LOj9fXriC^(^G>QN!2U_a)yUG(#Mh{$MT zVv>)acoj+OjsIdvq2C@u3ZH|wd5*82 zdOT@ZarhH9}J*l4fIv6c9W?vq76$gN<>Nb*-#w>1Bo1p*4sUkJiE2}& z>nsNR+ib2Nb#mW@31vZa1e}?a(RP!iln_0tT0G{F{D9r;*OjjwXS7|z*(4f|J&A1F zfyD5Y2L!?idqkU&39`ieG#`J>>BN}>3nQBYrSu0@l6rt+VBO~JMn1WlQ}20FMBy~ApSDYRtYC<{pFENi*0CXVvt{{p541*q7+*Nz{T8zY&vXEG zkjx`hfHj-SdYhyL*MG;de*!Rf8MU8dk*sLf$xoTHi=df^LZjvDMd^0cDL?O;X%1y{ zG@o0aVQRB)HgPHqXy`nHLQm}&HB9JuANbV;eOnxd9QCx!H_n&4N55pH67R{Y#?*Cs-e>tQvQ2 zVDk%MX?OmYHYiUN;5=q=W;RaFC(3&fnluI(XP_gR7K-(G^bylmqo^)+lhXG>>1GNu zy|disZanwSB0cmxG)Ma+@C-_;G)tLAQZgtf>T}K8Tq5Lg9%U7H>YbD{RG8^ETy|mT z;M9Wp7XD+JtGKXR*BvM5U0ZQ9STn2$eEZK7a0ggIIO}My!=HXfkI(zl)AcKV)a#EL z_oIHErKx{@0Bo`C>#56O-IhXCg~1Vp8S~Spn?q(T;zt&lzBe%!nCj?0!cT7y{;xD? zxzP+1eO$-?;AApKByU5OGyP&Vp9|9duxL(`n=3aecawxuszT9UA?~V?=9%F61+kk| zQAFPvFzeL-T3rT(r%tWd;+E~SEqQrNr^bjtn@f8m75r62kAc(G$>d+!)#d?{f=no# z#D2W%c`hqSdPU=OS!9OO51OpraE)~Qj|4!@=Q){4L8HM!d}z14&k4$^rLty7fd!4# zDUPpfEv<@#Y6DS>HO9wRfBZZbMiH&VVO3gc&?6J`)O8eSeiH2GK2@w0{aX1Uu;QDC zMNP2zrz24m#(XjUL$&k>({4a0TjNeRrw)R#b>|G-tnt1Q! zZz-&;%MC(*w_G!Q3y_$pU%z8hHUvugmx`y^?H?Z1nw5B+tOG7v2iuNto4i-gn>le3>g{AS)u z-=WmgtN*Goc)@ga#Kx}cHGXot{^yg;bm#Rj1*ntk17Y_lWu){S#>4HnAvccWptd5l zRoeenpr0KCi^x8=zZ)d_2LP;bY&!1Q+{Hvd-3&pP_e|PQ^+)$e$LSgnMv-?v)u-$;Otp<2}$r#ah_+rEJxC4ZSOI^$JABB`a;Cs>`Tf*mV&; zRdF>A@I{qH4ro||9^Ot){wiVu#Q9F}&C8GwD1-N@G(tusSe<~Smje0oK(|MiK5xO< zm0!V;v`NJSP6TB4ZS9NwUqdsy+w|Y-TP+OeH5mBW^Gd2cM>v z2`S_9@;rudgB|4nfS3j{y;w2cL+DUPp#8>cT@vD?BhN_GLcjr#;*TWd}~!*wZ~Tnf`hcbF#J89z?U zTf7MsmuO~K;q{1tceG;gQN3R!T1AL#m((8P0Umijr59hp1x}}bBYGAPe!qIT^Sf`= zuYl^wRLd{-lcPniD?M*W?QTd6&px%#JHf`WP>xO_k_<{O8RAM14%>4k<}dJl~!)X0Cg>++m?gb6U~F$8Lc^N<(Ju2o`f z&!SL}v#m;bV_Mj$d8$H>ZQY{f`oip~DBUj5xm;rNlJHdPX}mYNXt`cmuZE}!_{;RG7Nvjjft<3+PEsSKq22nqc}rV{Q1YuSAJnf1IuUlgOb5xFPOIoV5BJRr{Ox?bh(F zPD+0DxJmGEm{|Q_F>LLlsL7cjk>1vP&n~^ZY0>Atmj2 zL0?5a%5sO$B>m!A*3Q-pw5sd%COk4=U8v(WU3>FRyag^ZojYRjBk7WcKX>ctVA2YU z(?a9Rc+r6nmn1FD<`p(DtJ{i%D=5YS^)Nt6;=M70kdb@H*>KGqPm=RkSJD+8cO;I{ zJ8snlbL~F8*FbK3{~4kyB;p*XnHHo7%&|uMc31qKM&|7a$!5s_&R^myfHV1F&+chf z4qkO`TD|I!#LdpO;U*;u$;at(BYpwVBZU6J#C{BLo~S)wF4P@A!VH6VfJ#7nWMLbYN^xH#xuDK0h4$yGU<1WE``wQwi~=nAr$v1;>v76s(y`or z^>>tx(5c#3iF8WP$j(aU-Tr8CRBdJt=xDAG`b_tO6AuOJgyHbSh%E64O zUVUV&vIfk2RI+^=`=g$CzK)+r8peYu(-1IK#(h;;WIFH0+_b*CZMfqXi z<@noD7Qpp<>OrTLv6JzZSc^KYx!i;F)q!W_DygwH{n2eg;>YAP%f97jtVJRD@ie_b z-hoIeh;2w``sw9|0_9XXw)0lktskQzPxqw)OAadgWo1Yau8qJcO~%CakA(Cz0eZ`i{opy z!Z~|b0%YG=;evsbc-74?n`DPAzUplJR6~Vs=CQI~lXc*qH~MHLrZ|c+5gQVLUw3PP zj7s+zFH?A=QS%(It4P{BO`%{lh6W?;lNrxQSSG)$O<1HLMz?j?@8?fZ@jLEOd;s(q z_JGTjo7*E$IvqgwJ;|;j)Z%kl7jMOahS_3tCx7d4mnF%&f!L2(8JmsGfVRuPskKl0 zjftb0_(kx_Z)(k7W7u#V5L{GYy)~J;z=xPdVyOtkX(HBk7E4amC7y`Sur;&-NAU=- z@Lm*Z%2FFx=>MbQc*L0B7~Z>|mk8bj*f6Q0BS0L*c7P$ez>wG|ODok~v18t*a(9-t?Co6tU3Mv^p-CNcM6VIaO*9vM;~(aoKeLMBqy{=jliy|!O>R2A2oEmTVX#qNF!uM1kMhACB;yD zjCA*DIFanAxCQ&7c~@Uw$}_D_K)idbBS{*uQ3oPc&p(#&w^srB%C@@OpomyGvJ zM>p*ux1g8T99(!QYYq>h3^KpO5_1(chQV-DPAmcyN@L2P*Dvkd#{0&8TE85s{b%&_(<5Rdw&5ZuZmmL;pNhPz;>PS0cMk z%YZyB;}<(We(8bmLoVcnksUfwD17QXUR!2cA&VBb{F@`r!S5j&|Lols#IC^|7KleE z>yY{!(tB?B1`$!!FC!{Q6eeuAR4v~lQdE>v9bP~YVa70!#NG^EQn%Y|`=w;2?ob`q zvb3Rs{td^EDf}jeaM@s-1Ki(%ZKq@i@mN5651V8VgWMZq=&W2X2^mJ^A?;H9m1p?0gTEjih_L%n?FL+pK2z zJLLD-a5G(enU9ZD66Vf~XL^I%tmn$N)1W&`19lZh&n{@b>^M7G|77IHtOcs16$2d`b7S%{d8=JD^s{aNci--TJ z_)`HcAe>F)%v`J06BxGz=Ct-&(i*ca-V`!w1kx@^vN9z$Z46Ll*-2F^6h zDZzW}=N7jJ&9-B11xTrUiaL8p&!+j?dGKwFzPb<4$`QM3k9(COR5nwug1yl}2C1B@G?p zjd`8paE?buT`i>Q)s#>ABvK%}Rdyu3x{mEaVfa&2&bQV!Ze;nX`$KEGtVUDWx zoqitF9yLG%y1E7X`2y6s-NE=H(t>i0wq$s+pl1W2wx(_YbPf+LOj5lx%tEiR`2~^j z|C*G1e^1v0$kAXV+{WgCLP2(U`ODoZDPE7~UE!vF+k<)gB*k`@2Xl^oj)|OwDl*AS zn{zDy$z|$%{@nL2kG_b#uU_Y_ zbiid=YeyTwQ#P^!lJoq_q&ol(%Q=HqnVIU1Rbcca$$jCe#Ho@TIc4b1x|01BQXDn9 zr9Yu%jxUHWw3SCznrjd}^{pxX%}XR_e$zy3#f1Zb@Y8_o80zQ&YO6OsePwbZ$=nNj z(__=mT>cLLC* zMhhATv)e7{e?AB4e9u)7cjKnE@UtxlN_ah?CfY{>K3_A(PhN9> z@xu&MJ{bhj9oX0!;B5J1R1RaFj320I7!O?fDX~6Ozf68QpM6UK;D2Aa)v~F49-_zu zPpjH4_1E5~#{%0^Qh;<)($IzkIrbzf(Q8cE1CRKQJure}y=xoV4=&jCiYD6Qv#Vr070{<)6Y(_vu$M(yG z9nb6oyS``n$}YFmPtZ6e2gCM_?mvJaDJhy9fmPI!-FJ7`7ETf~TBS!5%p+<|zj=M;ZW}=8>7N+IlGQj%w)8`M z_rWP5^VnphQ*6edUhAK4YVtRw8F?fGBm%*AsSe24Y;Sl@duU^8rNbZ37)|u~lqtt1 zE07*F!Ts6visOmX{V|Yqzh}+81vVXq4WD_H2wo5o7+*96o{n`K_@(Qou?Zsb>A0X#w55+(5cjHGmF!s(c z7$OatP$CH*03v{z4?s_Y3TS0$0cWIqR94kU%Gjs;=!=%M|2}S^nb4WP-0pB=V335H zyK~S(Aj<5%zyYJ`IM1y5NnQOHAH)n_NNSx;7Qcf4k=sEfeN@Mz9xk`Rhg{sg=%t>Q z`#bVNL#C#e02`b?8rjz`2h3lkj8 zn+(ls*LDYm;;3xQ$@I$H9put}%^f_7uH2a##sk>S#TwV``*dIbxgB=*n)vxM(5}hR zBDlKX(-6&4)0F{9L$dYF4oHUc<=9nK$J155SqtwOL_TcU!{A^5i{e&;Grbw}GQGXN zKFrf|=c#>u0o{O?%8Xlq9|oe<@YwugG`}&_be;uwU05RW>>~BuGel0fEGo2Lg!1kl zUWJRvySuNo$~Scwjb+*lOHG%WdZaCdQA@-QM}b`-%dMUpDls;S?RSg{Ve8dOmR)q>roS)pSaajE$;qM-2J5K^G2n5r;YF}WXx z)aUz_4zJeV>jB)k4SdN1(FlxA>)dO*uG=$gu7B2OTT#Yuwpj{^Uu^rt_^mzCbq{xV zkarV9xEbNFck9ba5OH(diMZ?5L?JqfrG%}GQGDWQg2unq<&RZ25 zn3HBu48UbJJb6^`;?q{0qAr4TJ8tLNlA(?loQIyb-F$ATkv<@3C1Q zM+xIh#nI8U=;EhV4{1LG5gA+-t)oqB)IS0rx*vU7Y6{GhF>?M8s)UDMTpe^DVONmD+13tY-!(y>0rv+}DteL-4kg2{=FTTAYMHGuTla{+V0QvS zu?{-MT=VWCelyFo*X8a9lXK}T)$&Y4c<-#AN2VSYAdkV=8=dm@UWZohQlU=USw|#J z7OhF~0PMM+$pEkAF??Y(RlSzkcWJ(HHdF6pVmFqPFWyr)2$4wmHGA>VBU2dB+H>!P zr*CW04n#t&+RTxAK!7(5^MvKW0dGatlIs96DAZ40Eo1Y?Pt1rp_L!KnQe2F3^0K3EhS<94_p-DVs^;HCcN zcCqwta`Uf)7m7Q&Wu_h93RFXzmMK$NS!M=Cx^EJ`Doowgw8gZA(gL9kr^U;6&$LOY zUWC%@29iT#uZ(9rpS~?mjXk>o>g(lnH!f}Idp5M^1v>v20aG~X^$vcS_YT`s(`~+lu=P{6avr=> zBhMi%f1x3%WKNxij|;C8%E(%P(7itGGDDjc-yoM zkT5i@2BphIVz{GF5qJU#{v;LpD4HhiqUZI(#le}`K>>BefX;1Ye9s5|m|ZZIfra9J zbrn7AY6_g}J#jOffIrdZH+tUTeQu#MTLC^3Cjgt5^o;+is+h+hSM1b8QC_Wslqu1w z%O-w@?j9GsGcu1#gHZd%7X7q-H#W`aVOaBS<}pQ3;0$2=VhX#taJnJJebRNV2+Hrt z@S3-}_C1T7IkT{s3W+AtQt;_ntt1lMUuJARvcA@=g0u3x3)Yz)iXuk*_qa zMz2MK*ZYxM3FJiI+@k+maDE!Tmq?ZJ&iLe{hVG?FIWEO)bg#(yhiA-rR<`h7NEz%Y z*5P`j>hzA&&A!K0r%ds!I-G(wd4A1);gQMjmT8{hYGHT= zW7#%#&9=YMm>cP33HP^YK6e#k|71F$Q^i|-&sT`ibZU6s_f+(J{NX~6nD=HWB6Jp! zg$?11fU2q-)0beWgRlJpEJqu(zxe&N`Q177&olKt$P6(Ob&AC=*$hqTd)sAu^tML1 zuZ^Um$u6rB^Dn4euho6pjc#VR8s2%_)f4`bU12exja>{Krv1%r22I?(+`qkHg1z*l z=JRb3Cb-hkXmn1nnNd;c8RcL04Ht($aYA9Pi7Qz$TybPatrM9hAMqEnb`9f%Ahv0S zyk)8VIoNKaLBs9ac-u{G|JbHSi%xqQIsz|JiO3ln__22YTey(5x^d5rJ<6)uDA^aQ zIec8LB-?oQ)!<-DXXTtylcLG5P<39|P_W;q+2cCAg|p=%u$ zfeQk*c;@gF=I!VQ0%=?8H40Z^l%s1D>M0{<2nw&^aUcg9QLEGKXO+aF;MC z+)a=8tQ*Il8m*FPz7D3t`|lvT?vY836Yj9b9i%K@$8O@G>+}2NYM#>yJ&1K7Dhy`0 z*$`flde~D=s{e9EVe>9h7zB_f_!jV3Uy-<-Tj(FTE5OMRU8zPShcg<5*DEbeb)dB7 zdqs9R53Uw#ZE|7d35SPa^)wF&SxC$?r&?bZ7ue-LIJ)ubf8q&smHoiZ$2K6iD%L^k zca@zv!#iVaLRZPH`@K%h0#3$jG%~}b0xhurfn3IARV)|nUlq&8>@?I1T$)r4D>8o> zP+2+a2^Z@2+e?P4ycmt;Ws*-h-A(F~trB*`l_2tdn*n#USIIm-Pm?M81zc#l&k(L5 zhC9%haPff_Tt%)vyPRw*+@hALqO&d0*|NACM6R!~03mb{`~r3z)fd=-`3GDt?*cU@ zAo%u!?T5Gd2nmlLc25kR8^9{xj|LO6=TMwx4!y|rc$`slz3{j!;(^mZr{ zd+d|h_f9~N`eAencodK5UNXOBGYx%JTRIiYC64*B0!?!+Y6dhSdjob;uD1%Wt7c2} z&Sv$l;d?UG)_p%3DrAV?9N_DnH>3-B^5rki-0EOJzRXvME0~7L5Uy-+c|+~RwWnh> z>zKS7t^OZp?;Y0E_N|R>QA7d728amQkX{6(Hwz*i=`9p7AiXEFfPmN#Q0XOrAfXco z5PA@i-lP+{^qK&H1PCE_vG>_$yT5yW&v)+qhbQ{L%35=cIma09c*i^7@_pShvQ(lX zWjq}5fnAMZD(C4r!j&nv`;4Q z<#t(?k4(b%-ob}%iZB>=Iow<8nWBVE?5s2*oZ$JAg{4DhlE*d_vPTRa3Zv?4y)5?` z1i1CaeVUmee#q#xRaaYV8Fy)+-@F6j^ST|0>luHS*2DB$47|m7J4I^NL_i0VQ(GT* zk@ymQ{5(?> z@g({lcw4Xl1iO;D_?#sCmsyqF9-(=$gPxw3vN z{3D!;GMBATY0%NPS3QEzYRHEBWiVZ_SRiVSq1T_OY&KL_6D@<;?i_{NfYn|-Ew{B~ zgBhr{#jE?*Y^`mgocEP}-U?1jvIvG_TSs$!bNPOj+z&kosaRb?wStcKX%+%kBH9eM zftQM{4}B99^FN>1ei7L26dw-hQ{@>0T0c6xC#yPm-uwElU9P_z%ZTyXenDQBliAZgO~iK}+?lnMOBlO_>0o{QNT@Bv25k#bVQ9 zs@S>?^h>Qs5kGUxD}WifK2{?lR+%MxQf=V94b>hL@~s4Z({?rBT$4A5K=e;2xjuz=ib@d6YD(# z+N^pp%C|K1KH342DE+Brfs~U7Gke$et-K46yGJkcuQxhJqkXtmG7U@$4gwSBm>bNsW z*jHw0Z$XbRi`3e^sJ+w{g{N^^6OZ9OhM4#e-R;5r1(CW~*ITLAbZAcK=T$onyBr$Q z81iHq{M`Haz5CrG5hV@*n^LxuNP!&jR<{d`9fNhk3%nb`78@X$4h<;TX0g!WvUT_M z7Yg(X1LT~_B;iA5$T890YH){Q??Ln#Cc>T`TA+NS4_4Cg^kcrTAb)1Im@~IyQyukl zAv^N=pJ8(PA3cu9?tD|$6L!rE8ydH=(sjTTMzNO;a$Vx=n#WSAoyRMC0Xov`#q3vI zoY@7-;~WnDHNkvy*)_VY`Q~&<_k=f9K&<9NV%8fLc?83k zivY1;zEIz`DoNJRWvmDk%(&m+=Vs|~V&IGVaCq$f!(J6m)G7v=8B{ba4-T(inDlzz`mEMnXdgrY*C!BRa3h( z5p($ov(%2PCH#0a2OVq6q29@vqr6Kq(Eqdcm8@@C+`rxp19REHH4{R-%ABq#eG+S= zpxtPO*Qu`k*_x$XlOE9%IGH%2;(@g81B42v-_P40xE5u(Cj(n1XhX*6m^h!cRPMfX zod`T^EUe#om*~}HrkU1gNnK$@1Jmbj3Wi<9bFkehSHT8Qv(S&49~@a=>uV*{A3c4S zNBE+}*v7E~;Bx)AH}5Vs{|T!2`~I-a)$v79DV~!}spX<=mB`Ucs|-)A5fgD#$D=n()qlWw+mzNF z|I3jSj+Jem0`_a~Yh(O-wvJbHNh*l}?&0rat-2FMAb8%JRp~At>;|douQEmC# zWi*P}fjb7{YM&CFdYpr*CgQh;Jc??cm3iZP`;B-D%(GA(ql!})x8N}ckHM-ECtQ$_ zUcYn<(AGnL z|2{Lw5ZlA`xoGX3M2l>Vy`Mu>?acLnrzBDj$@K?i`&VsC!30Wt*UmGGKVi2ph93+Y z1N!{KMTS?@O32Le2iY1Yp%(7*2E?6VJx#tPs>fK`fo22TJ1TcE&>T+;PV*CbLER%* zx_d}|o96uUVEpTJ)aBK|nET4Q@#0@u@tIR}YuamwIJDW}&J;Ru^>K~$4>07={pJnU z^F-W~U7AROZ4YcCX$lj_;b#k$El1k_c|WH zJW2kZit$*h;V_(}MGskVyG4=8y`@Mm|CfQkH%m{EEt0-8bJ_5I%JGwoZ=Y`rKu+9G zr5{*R6h#8ktiUN-zN7&Ud`Zsy0LPwPV_fKhZ89uT3_d5(5CDFZu2pPI?^DTp-aq)x ztZN5hy6C=Yrie}sAuaK)E6?u+Y}gaI6e(5SE926u$zOB_MC$in%TKGBDdK3J*>xx2 zTpL10EAUoRHJi6Yd?NS8gIoZ)$l@B@ZVhZL zgwKn?H1l!9nrl#fzi_3S{`cwbUpKppCt>^h>w$oe2UM?O`GWO+>cciGXQL|2s2g@^ zccd<-E@5&=Q-M$y!58iNd1JjpUID}b@t!e7Ty;O#jpO3pc^}w{3q4scZ4AQe2wYPSTUwx_@$R?A~KInCK z_)5&3XWM*>KnG^pCEoVttcS>OKBm_bLl{FhOFrKvg;svhRPp}Ji!u>4-R?y?BG-9U#s z7vl50IPMjrpaBi@guCZFzi(E!`r!NcjM%~X9Y9evS?v*z0BvfxrmoW3!5(2Py~8i( z!jzCHy0d*R&b+`;R6CZ(vfArip)qEhRti_XfV)B|X@RIZ)-iF)ErrS&)ak*iWU3fb zOJZe1=b0gvDP&}^w?GF{z}Tnh{k-rw&zhdM(%e~*=nTW{DKQgHIpTP!N@JUTvdgpijP2v zw6ffA+dj2KQ_+3QWxTRM!ewMh1SRRZEbLVi-poIi(A5{C62))!9{9?u6Gv|$@0%8_ zyM#l<-&z1ZwwZjbgzz4P@Z>Lg1ST;*{$w7kOzCxtt;H|Euln@;9d`k5@`=H<;T9$A zDGv>Avu7=OFSCF$h4K0&5`4x(cx=;`P7({<{Q6>H;$ZVill9>HC* z=urje7nC)8U)lr$UnxJ;*Y?h-E!n=PUQ%t6rS$P%htJU(b1PpPu2=SEtZ`f<7Pz`N zorZ)+mui{w+Zq@8X+|ciH>Ez~KL2+_`1$713voIj9e1^GVE7jMnC58vKpoq~-|t2( z=cSkjU-Hc?Y4uuZsSjx_^L^x ztCCzWGF<5KUoj|fVcr^=`3@YS1pLoEpStFD_q2Yl{``y4@k|dhY>Udgi(Tg5cX{+T z(KO5g2Ex%}O^=z8f8g_HcI=ja`}5~!FQ>bm!o?K-bw4b@^z$8N%)*h(;jpKI`eZ|L z!oLjYR19tK7H$`Dp`{pXZ4{T_{FAJT7x*XZ1m1iM$Faa)@o-pM++XnPUx&f~gMPmG zAnhN9QY#$oS6&ixi9%<7U$Jw%>NLO*X!i!6tBrA4FjMof4AQ4b?SCTiqbD+>IRT9; zq_hqFA6G`|cD2U}KnG-FnEzN;U)THB{an*wq2)KqZNKAWGG3xwsVUQ~k4a^Iyhx>_j_C$rn1Vm*wECRu(@GPYHF_j`j=xAdY|T0i{INe8YT~SgF@S zd^KMmA>T~$_|<$ZFu&#Z~8#XZ!1S<`}Yd`Wm7R$!49$crt|M(oqI6yXebqP zp%zha^lx)f7v>NdUxDA=*~@QF1$1&zvbQ~29+OGy<&>+9oaOk?N9-28b&~wm-?Tzx zE}p(~^|$XtaRX0>9ZhpO^ZV2NIInZEb;Terkw%`xG2%*|SpU`#3fcI1U)F0$Cx!2F zjAJk$;AFDrF%P?D*F?`;__@aZ^1sbaYLxa;rBRidfnNIJG3bHc#I|e_;6UN$}EqXTd_cizf9oeVt6fpZUKnepwA-z>=ihUiA1N=yzR0>!>Dd9K{sYlbSlf7XS->8>VX2{}S*=eA zK)<-OI_CQF47jd=rXiMbh{>3(;bt>FnKfQvP&$kZ`m_W(a{+m&QS zU=b?7gzt+K#3Cq0Bj^)II~!^kMeY@avvVl$3DLLpwYpVGK6&O)kp)uo9Y-qeu#q|O z`SkNu@-3{`GfVXys^gkhZ%;HVNI5FLZPuy4?skWr>F+1nf4idJVlyc}+B^fK3!oxQ zERlB{F1nL^2#B!sb^u=3DDP=ZiC!>#6ak3v>;cAUn;H9ly4x<*_d6XV)u)dSVFqN$ z&NbmPFz}i>&@%Cv#@@7pwehTx4mMvTYV#PPSN5^&TEVA9<2PsF8FY5rJ`Ewh#GzN4 zlehdHdsMj%aN7$x0#!@m0z0y5dW7CRR2I5E4vwBoI#-uK6UBVXSib@ODXe8}wnvfj z1MCfA^Of!m<&^j5X2fZbV2Eqq?}#Ya0fQ2{D=HKcc>DL`;GTeARoTuxsdYm^->$7<^DCIv%!;irnn_X(XxFj0u&lYR zXGi0qNAqn!FCDqrrIw_)T63>`2VVRrKo@lssoO}2sP-u<$Nwz(3C4|-Of*ON-(2>o zf#A200_|?-FYWFg`r}7B+4>ny z+~+#azssxF6XMy~^)Zx=X;e>JZsEA>4&`T~GQp@ozn4YG+-i0*pb>zw?#Uu7V2<+{ zhlGm066zMxbEQXYB4{~?v(xUp7S&(WpL{*G=OIEk7ayEy*O_rZg>L7dvVNUFkzk+O zw;B(FA|&zpnEDyb`1WF_B0K&Lo2&n~t)rAK8hpeZd-O^jq}u|#j>^2gwCc(tLxoxJ z&Xs%g$+eGai=@iuJK{#)Gj|;9^QM=2uc8f+fs}$G%XdsT^D2k1scAr)k$GgpCNkH^ z%GjMTLCQKso7JD#>PY~fzB)}K&t4;GlRxB)LYW?w)oE=oF7$}2usvu1AJM8=GesLl zTJYJ=ZCTedDpH7@)&j4M&u63o1(F6`$ARKcQJ=LWT--5;Q@BUY=nk#Cp?qlCqfFy! zd?_%RW>hkqrP`|30g^bz8q04$BpC8-_#e7w2WifJMR8py(!Vb96-AXOu*9o1BZ(#^ zvXw8Vq&;n`M1J(pCzf(g4Cnf=RKFCANI!MhB}{`ctKd|~cNHB2_GgXNsyw0xK_hPy zDy!ac$bjaVAZHxPy`X##V6yd8{JQ1aI>0?aal8A0%eJ2DjIfhUFT-;J`BrQ>6PbE% zvQ}^J@Vz96zY$fHnkyLP{(s*?`X??i=H7w<9d2%eNk+TZRB6@trBH#C*M`|z9QlTI zuaQ5^$H8L;5ZDw3hIU*$Yc%K|4xr$ZJoy6vkXtLWvce_E}tt;Hw zjZob9a)qkIUK07|*FFHwo+0*J;0nNIIaa)i{z_IL_r+5rR)>SEZUIw>jXv4|gs!!V ztXBE%9?av3D~J7CnZx((Uy9r3`f-{xUmvT1rUmtIvc=V@59U`gbIesprA`=6Hn9&%lY0vk###JC9QL<-E*dJs?+u9c(LG?njJbDCEM3Rpb0fWUfT) zfiHfdZKz7!h@n(b!pYc`0c7|6x#l9O+DkP+_{K%Tbls(zgVAyaa`ap8m7xe2-zFB*r&&}!&ygZY}T745P4#+l6#6NT@JN+Y*f;MBIWtoow{;6IpnlWc#g zE?PJCJfW|K;yD4N?akqXR#Rw5<%6*I2&WJ22a@058$*Ppqlr+|GVQ5d?vBis2v|Wl zJHDP8S!hw-uDtEy#WUDZl`~sCRHA1myF8~C@Va}+{?cx4U;q1)S_64CZ|8xxGg=|c zYsk|)doR{JRm!?{Fb_1Q#U&%>>$X@_+X=6#T)N~san}i1lSAyDQ^myWN3M_6RfX7< zIFOJ&2vJc z?%$Kq9!>OAs=}$CQGuW+Bc$mvDxv;Y&~#cr`*dpuc30tDXTe*}=d+REG za?g)ozsc0?IK{yJGUu_fawl3S|L)cqXe7_RIpCmenWq9fwB*;lSj~OR`yz`P&_5oPJ}^aSEr=+Q{%fzI!L5WYkhVW<5Z}XjRF=B)0C%P)&xp+!eLQw%(5QJ|@?9Th&u?F#+Yhv98oL!E5l@uf2{Idz-+0*5rsfWq{11JFCtbD`Gp+;A=WGV@YM zc`THtmDxL`kU_q;V+zN}8MJo3o(Tx8aUG!o454e~YY6-2zkUYTUBwvp0_vknO^-NL zCiOnih1K_*y*h#?#nQaq`4Zt$ZM*^SiUU7RGF*|AxF+RyAR5(=4YFQHb4q$3KVgU8 z8MZ*P;nd(L@HW6d<*yWx1V7r_3hy~sTv~Cgk1r=hh7|26q{urWn6{TE>#{9jwEeK{ zUu2xSbXqXCUosFjSz~xI;w>{smc<$N0i1VLff!hW^sqP&yCJv+gSU~qM@Z@kRGVsJ z&IJN1q*|0LKjJ!m^PIL4mBjpVD{D0&nzvG~etkQ+$hH0^+yS)%yWk4s}jR=Mh12XlL zp%b#x-QX0rnKnKVk10&;*)Ns>hgE5a_p-<@*Sfq$f1IA}OT7&_MCq-Lme=$|ULjYf z=Ic2-b)bFM$cEg(+y1I>TZPoEFN({#--aKp3&dAR3<9Sx!=(C_b9z)P$?Z;Lpt=%z%voB*k)xpPIv6{E2HNM>2X$R}^MCX^u7TJFER0O#C|} z3eI5Gnf~h1%hAEy|B=_QC}*fhYWT~elTZmraOkiFG?X-~{I*RvIaKTxXV7<5zV(Z- zs!;Nv@Kx`3icbsv5CQE5HPY(wzwnsZm0b6ioc!$CBH*?O@MR%9BvNHTVs^FQvFyiw zd#g8shwdi|dVQ9b(HrnH+NuIZxS*$KgE33OUr!`<##~Q_y*Vqfl|rCD@rT3BD^=kS z=vU-sKa>`@M=yQud$U$8?him4!veiF=hi+gI=D4! zvTk3e!#^*-eZsjB~UomWO3);@}yHa=*tph?IO=`l?Yhc zC0PZ)BnC#&j$Di$_3W37pUP3=Zwh=`2%R|y`nZrMCIcX+Zu+NlIl1KHiqGF5k1 zKC2hP$dR8YLqp&z2KPP#a+l2tdp>HIX|34Gr;BO^Jvxv3N_!NBfmtcXi+HxXTelPC zM#>!l;k(HGGK+^G?z7+}MpGb9FVI^QCXS11SFP}I0Sswa!Q-x^hK@svMafGqu1r|# z_HYGETj&uV)6S&IZ^=!460Po2#{d$&D60lJOSZ?{;|4=D#fS?tGWd&Q10_a%}Nz~eWSmx&b@KDSk>tw_WN@3KAW0>Wavz>Oaak0K^dQO@>D#}%*Vi`0KO z0g05FbpoPZC5%ZfA+ozgp0IrpK@eD}gaDe;i|i^K!Iws)-F~{qu@)x{@0kQ=)cKK0 z3WbYzN?JJ4P2u&M$1)mDThYqW!BL@Xd7kd)#&Zj5n_wPN#v-`iiYvcm<=5(!RHdLs zDXTg+LS+f~jP11L@4@Y&9INdPw3@8PyE%O$@l8mj`|w?SU%A-^ilLnyuXPR&$r_F- z)zgv}5;JEC_ycRQXdHU-49uQ5tsL7qsQmE=Jc;gUPs&YilPNk;j?&>BTfE66Z#6vS zR%%d~r1%gacGBE?$Gf2*1J|+ltzxI>YZFH6_1tH5ksir5wfMyQ1n0?GWz09Nh+Aa6lT4c?^4sQPX2!|uG=hWGRSZJdgO7X^ zr0I4T6X-zqrFDxIe7$m3dpnUamhZuA^GPUTb%u4}@D(v%Eqt3n8Z+_@spK6bQGFu+E6 z1Na+I$oxrC*m}KHtrxyAkjWOsRTgP2Z&M+{9RJsz>7yvJ5RZIKFrdC4-V%V( z$16%`>1)aKrsx}*w@mU&U-ZXa>%H6C$oxFV_o^P!0`slgDg}ru$y!lBKzaxKM1SXa z`8uZ5P8zy)cWa|W?|}b$LqK1h5%VQ%PAS;S%1$&*q9~(1eKwNR0Ox{jV^+uA4L3fE zR1tPFqjnv5;z5V-C)vidKwb$cUHwC@mCML5PajJMeSBMRF1Y)b#$^%$-|3HD<=y>G z=-CHXDKxl`UiV*VYjJMcnwR+Q-MK!I4L~ZFp5!1rm2%PM$tuVM(@Dt5(q6miEdh)piL%NG zxJ~qGaTRuNXh+0xYE>>-MsE@9Csf(*Py_oyN}@?*ts;2bce4`>`VZUXkKn%B=Qv;n zqeJq``)6eiU(p~1zUj?Z#h%f}j)nk^ay3QXAHnokc7C9N7Qt}x$4R!e(b)Jw4>Hji z=(<;qKeN5?GDL%Qn_l5(7^|al) z&rgZ#LMR!PPiG`MS+bEca$ilfdUp4aOht>he0H^~g_8u=4#O>6Y@--R=H(t)J?Xvu z>pY;fch2-ngJe}!o0#=Xzx)I4^~TBBLSt zb?d^?M0T1;SCI$2q`t zdNmu~npG^&YIi3~N?sT@!30Q?yP-(5VNt0{0^;nuBqR+Z!RRTUR0S}3Yad1QQ?7sb z3^;I6Z<6`aMF-;?2_ri?izY`?tXnX1Y58iAC`=$_$4J4^c5iEbHZnW!#3&OzVtuft zLY8$~b$s)}nUvM;p0$aZ7U+UOSj6)}mkFB#={xXjJ%gB9B`wdiX!v1DX!rW^fqDi4 zme2n0g55t3fR}*=ELK?xC_SG(ej}Dv$vIr)S7HT&fvI63rsFJ|2(OT9K}e8*VW$AB zXgW1qq)PC@Nba3jzAZ1PI~c&Rg=-q(ahs4U9>z`|AD-i2Ae5Wg{)3C+Qn-~)GrNR63jNMmDnx2j?nlJ z&LX_Q#_vs|P|YX4e4q8zd1O<^*Icnt2X7`j@LRanvE25T8BV-m<@5A{6V-$G=*JPs zmn8g5k#wx*V>TaYvcAz5TLa>L)~avkYFR$8*S+)t-E|(Ju-1)haAwcsUQ@pZ4t>hf z@8b*zF9mJ@ibJr4@+J50l`psilEXt->-~4i4KDJ|_@Y)G)(>BL9PDvdH1ylAVaAUDOg?ANatFFcs z;*sV>Hwk}|D4u|(8<{3cNmKqUnmXNpD5-746-hKRjyfy@jr;{@rDz&!Dyb((+f*yY z4Y(Jz`w95CR0(7jzLfC3R@Vs@DbTr&PkCtV_?2)q&*F1VZglpLYOw#AD-s_*u%vLk zV{F|Dov0!0N*j8&Y`!!7IIquPR-W2G-O+-}n!SmHn=7qIOsPsHSS(t8KMBcuxD0e# zI{=uH&|UScxa{<<`=OB)v_+vAVjmt$f=vfEgF`r&-{wyZiWRK{=JR&=)QaJf6owai zS94mJ&l(NVtds`MVw-RPFIkKZ-(5YlE#d2NK2o#(km;vXnK0F`M0V zO8jd-?)^{>^K3C-Zg2EnzVliF>x6QW2LzDvYVNB1lCGN8 z^b>xH3x|61JU=(U0jO;`O^!<+K6vcpW+#IcDAO`K8-G_Fa!eg;talohGAsNx*Bip7 zQ#w&*IjGMGX*lpjXb7=JRNuAS^R7d%d{|ApdKg`#UPQFqmSHAJF>A*j)7N8(;dpA| z_-qkWYA6)%D+GDrht&$nwe-ZkFfjB3@_v|A_G^K$F87u+fZU{cnfhAuDizW3*i(Iu zMz>hI4~RSJd5($%7)zSuC9*zn?cHU$vt;?pfr`)ZWI=;?(WznUDUo1U^`9W6J~JWbn){mL~sJuYPh1wGjSpAs~Eo;;4f2PS+t6L*RX`|QNwv1y8+GI^9}b^ggF zx>oyAjGn((uYMR=yLE+2^m3&Syj(Oxb^(Ay!N&=l5orq#cm3W@RR6VdN zGXx_p^BhRO+MKn3GT-eEhYJo6n8y8%Kxh5I2aBBV^Y~7AR<8n6-QX9Or-H1;oA`4? zU(`$x=^Ehsb-ry`@pK~K0OT6N6|*i%aJQcx)mfQvDf{!RK!I*BSQk7t2#hko zhbS#_CmDSI#eNpL;M`Yc|J_~J*E(m6RU`^_2N_+M&)F9=eJ$;}EG|vUe;-A-`o+HLqs>E#DM{_!Z+lS7NIdG&_pfpc3 zz^il8KZLZu|9ON;9=?h$2vh70eC`L12C66KYinEbMl6p&kA4y@q*k7#IbWYT^(IYY zZ|Py$`R(Zj5CpGC{p5Nnmd>C*EESM3m{7!QqXY_6qX)Dq=^*@}aC-*gJCqFCc~}9r zroU2IjBrj8mlY9swtH%8%xXe6hxR$)bHLcbJTRR>Vt9++*`pQ}~ zkcVzqXp{Ui%GT;Nh$;ZyQcS*XmYRz}HU?V0fz_WNU!S1bSStouXZQQP#^M`@?omo& z!NAyFtl7nu8tS~b^T>vHfXZobt$rL&?AZus>w4{Hi_xo#*!{!y9Y3)kpqMrNg2e~A z_4U>K&mHRR>XhxdHrKt*r1(h84NI%+gKDp3TSsl<*^Y=HT)R;V!&!aA%F;J2&1}KA zcEjC`P*PQ7y57WUW58o*9Sd3r(xQhu~4Jc5o&SnYU= zSy@`U+HdE?bySM@xM0)SPQ5qWl`q33dtH%D;jGEsEd%*Ric@_o)eb^3iH3@5JDkEg z4oVuqwuQ6pHTicf(8(cQp<5g6Xup-|>w7FEV1g}4VRXIaG>so2;^RWJIj&ycklUbV z9|uFAZFg4mZId^pR)>mq2&Y1pX1a>+hrY&@mbIZ>#{7VockYmZEdSJ$XekFMn}zLh zGLog+KHtAjzU&>w(y7q(SG(VFGp>$oFp+zWnik zcBl|KUp-sSDetE&26T)w0Ukfnxr;M^tmsBZi_Uv}Dd=tI%*4 z2p`IT+ONA1uE4D} zl|b|r9(JKCuJ!iLT)j@aa;;X zuq#k$Qd0>b85UYBTMlg&tHD|v$)rVtOV~>5hqdt_ofnOjjrVkpWzysHRo9Cf;Pe|u zgM^)WQYRqq^NZ3y#5VPI%dK8DQk{Fr@FUq4S+|d}q~&-hkeu1gozs}l$UFzk0A1m5 z?#IS$&Q5L+xD;kIpeaf4KXhXwqU7l=6-0;Vi`3855PSS_M2_J8^+#7bc?QV<8Z7%O zhr|5b9MBSQmEJ!pZ8Df73yirPT<+=ioT7wvIuRnIoEitv&gHqM=|f3 zVrjQ=6E(xsB1fFqOWV;oqytAapZ`im7z1Tbp>Q=jO``;$d{n~Rz?k$gjrg|h|KzFN zPsNBthOsn>O{L!LO`DcS6+9$P)`a~Xg1EFr9_PtUgO ztvimP>ln@oT{dQw59E(BD|@lr@nU_Xq|t6eN7Usf+I7lyPuQ5ZTjIW++ZO=mtYW}j z6?OY)`C9Cu(@x?uiXPQ?o-h#NzGcBz7o@kN@4J2MINd%8w~DNr5xLRqQXUe=Dr9<2 zR%*aA*3_}DEVt%K#jr)t`rwn>AFXVUM4fsN{gtZGG;CHOuxQ-Txq7}p&wU&yuZx*f z92u;|2<2m2^~rw3%4eX0g22an9sZZWMp~2niIon?%8z%g>x5xYd)rK1S<2wN=b5YrI z-n9`R*3|z*dd*Tz-w?0OvUfa-bYzEa1`W2y5~q)oOp_(n18h&m=IRcG1DVaph7`Cm zrcNU(`*uyb4s95pS#Zc115PJE79(&JD7Wuk3tgI~0dwj6(UTBw0aS5C#>(vVC|;(i z?d1X;&C~r`@$0~p_8I}`&L*A;RW4l+3b4;L4g{rR7t}Na zF1`}h{c#fHa8M^~vvisVz9tq5`2d$ITx>5A?KG*|p78`a8b$Ur*iA+5(LUxA$Q|#N zr-qa*Bs)EiwKtm=6Xf+@1XxrhTJWeHtjvxvSu1PJ;PK&02H}X|p(<}(+jnZh`R|1% z=knU_XJIZe9^h3t%M@Vyd&-*Lti&VC!IGiCqx`J$ohdK**z>lz9Z5AWe$muWCxc9( zMutYxUJQT%>9)p;MC@cQqw2+PCZOHNDsaayQfLTfmgSznC?M1JlgSlD0Kxv`zlECn| z!W*->{rMqI%pa3n90zhf0ZiWkJOlV=rvdwNq_6iOe?LMnK_3;h z4Y^wFKL6>2eRqjB8t^FR#_?J+Q2;2)_3?76K+YOal6vA(5|}PP!nVVIa48;Yn$2q6 z=<#A@8;FV3JqG9JF(f5~uv+S~P`*E9O%_J4o+xwiO+4<0eJkG0uXCtIsPx*Nm$3dW zMFwi*6d_|k$u8A*q4kOALs7ZPsS?{(H;tVEb%q}f(O*mEaov1zlk?%8IUJ8M0_o!KA+ZHf5P1UWKCNC+J0<-`(JS)gY zjGF3%M@2G1)9P&^+iEseEZ^>5gaEYJ)VHQypX4>5R50q?0E` z9}-$4b9eq16lG8Tj)Q8dKLpr(E=8d)#YR@3#w5-FmjpD zN5I@Fm%)IhJKW>7?O04Wj1~=I!fH}Gh7xmLRmrsGEgwAaGOG`S%O6ZrFb^7+Ixw@c zO^@qS+h^r#pvm3v9GLt){P=^6#j&4n7sx~Zb^Y|LxK+3))tUq3Ib2H=N0sI!&%k5%;75#@Ac6x-~(rWfX4w&w-n$MZ{-$DzP4-1X8m7np8qc{ zI>7wmT|_;yAuRq4fZNT_8E4XDB}NyPphSwyDu(F3EdMB>VW4+xm-vEO zCKGWx;BopM4WCxD)8ve}rUTDiFK(4s`{J)?Ko6+;;(;Tm*ufW{yLP6^$=@8U0{Z#% zWRz*S!=lGiwl04l;Bt{Cv~S&W8WGKJC}Jx#KAm998Y1C5Ze@^fKrpe7d3q1JuJc?m zC&2YrT7B)kHUOgBOKqJX#($8`rphPm1@a~d^&F68#>w0Vf7*iR5iF@(iZ^cWz5TI` zv~SA#QWu!%)8exuw$n%q(Vw>a9&bAC(;FO# z`MSQL8Ha$L;~R7I_mZOO7B5Td+4llSG4C1GiV5!)O9FazpPk|x;guc5X^Ac70+=I~ z=jRvj)Su9hh(HkcH?B%*8padH6m%e7fr-5-(vvFRZ^22(2r60&Zxa*cPxr>9U=e3^ zkJE8*CAMH-{?7Yl$eTU0@(I&2^m!c6@zV;P*(kig&B{>-RCt+s=>7&-lwpMxD5@Q0 z8s;(p`K}L^_`>1B)wjgte9}LBB~3x2{7t(9i3H2)DVBFpVU0<2_$;rg z2f-OeAWztL5?bb6US6b(CUuENF+AWgi;$^Rk31oU>#&iFdiNK;yv?) z{t{knPq?>HakEj^|7-aEeTm2cgecc`&V1Hn`(8MmwfcuFOv>7aTwRA{+p}%Y+uAX) z5q-}R3m)?K^LyIc8}|SzGGns?IZqqC1d5HtR&ZT49-=t=~K)c2D)JIB6h z5FhrQ10TXTaFh#&RO68z%Y-AoSq2s=IRD~e>m2#3dlFmU`wld4zPRrdn;)GoyeBR> zNlk)T0~VHf0`3Jbb-o5d!g7E{HY+P|oBpNV#^PA??aG%w#1E=Y6cUO+vg@%WB75Ju zQ|tC`F$RWL!18WoeADvwVh0$*4`?CH$*$YQ0EHJjDen&hM~}L<=cq*Gx3s^^wF!+V zYMgrI?mGJMOQYZ^F9X9S;2iFQI8C{Pnp*)S&TPc`np!0nz#+kux=@dW-sN3KtegiV zoit=7TJ#jQg6k;ev8Ta>A6x*I3VX2u&`;3Ta~4AZ%2QPdK(m`n;s%L{F?wM>l5JrG-)O!97Nq@xiF zPCwp?-wZA@eHkKj)EjT)^wh4av`1PJsKE>atAdwx7GPf;xg)ZhCpoHeU+>KvlcLX* zkx_547F>o-#Qk3_a=$s?7}H=e&T$h55`#eGzrgdOLylWN-L?Kuc*8B9X{`otNwsVL zfd%CAESpW|Gf8s;m05Q*2dSq$&x z2nC;dGkIVIg7Imv{dZg{1YcnI=Y_~AfPwebMD(K8l7$apXIO&`g{;G!ROhbLBgo)G z!Zm^eOpl0zCP3?rb6S233et6@z7PnH%K*-&Q1J*b{H2-0!s^HWGZ}v1WZ?RjVz;l% zeDa6eZD-EP`hSx1uYS8&Z~tWVmKS@sT;2&>DR6VY`l;#b6Aru2dDvv! ze|CFJsRnS#+)}e=OT}}58{5i|7tMzLewlpod3f$wBBx_x^DVS~kGwXHgL^Peo-6xhC|qT;iY~H?kHaOx*N6m5K;%ULldRjv{$#Ql4e#aTWD`PUJl!o5)$b88OEPOAQaNo|lQgV4W?#bQNz>U|h+BU`_ zha43P!O&p^BU@uVnxb8jd?LMwMq1@f`Aur zTX{f^PYY&H;19PaZa|H9b{K`@CijN+1!Sp_|Dok02JfOxiz1qdDxV?itrkpqh?nrj^b|ObEF=jO0 z;Q;RRCNp!6W=L4UjvkdGBSSV80oN^BPMIBPOme})&{)EH;?B%<5|43>wxS0qyacoq z(I`K$Rp#9MNRqX3EKziLcFtsj1j&`biLIdWPAu&oPG{keaF7BPBFh?&uD~IWvp{fI z#|kV9wi09v+9^nvis&J^8S^9xSUbmX8-o@{kQA2+ zyui@8d>|==WR2HBHCNP(pCspyOi*>eCbpR5DsQJ4FwtE Date: Thu, 19 Dec 2024 15:54:34 -0500 Subject: [PATCH 27/37] [docs-beta] migrate - Insights docs (export) (#26509) ## Summary & Motivation Broken links marked as `/todo`: ``` - Broken link on source page path = /dagster-plus/features/insights/export-metrics: -> linking to /dagster-plus/insights -> linking to /dagster-plus/insights#available-metrics -> linking to /dagster-plus/account/managing-user-agent-tokens#managing-user-tokens -> linking to /concepts/webserver/graphql-client ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../features/insights/export-metrics.md | 150 ++++++++++++++++++ 1 file changed, 150 insertions(+) diff --git a/docs/docs-beta/docs/dagster-plus/features/insights/export-metrics.md b/docs/docs-beta/docs/dagster-plus/features/insights/export-metrics.md index 219eee2060f41..1944b206519b8 100644 --- a/docs/docs-beta/docs/dagster-plus/features/insights/export-metrics.md +++ b/docs/docs-beta/docs/dagster-plus/features/insights/export-metrics.md @@ -5,3 +5,153 @@ sidebar_position: 200 unlisted: true --- +{/* Using a GraphQL API endpoint, you can export [Dagster+ Insights](/dagster-plus/insights) metrics from your Dagster+ instance. */} +Using a GraphQL API endpoint, you can export [Dagster+ Insights](/todo) metrics from your Dagster+ instance. + +{/* Refer to the [Available Insights metrics](/dagster-plus/insights#available-metrics) for a list of available metrics. */} +Refer to the [Available Insights metrics](/todo) for a list of available metrics. + +## Prerequisites + +To complete the steps in this guide, you'll need: + +- A Dagster+ account +{/* - Access to the [Dagster+ Insights feature](/dagster-plus/insights) */} +- Access to the [Dagster+ Insights feature](/todo) +{/* - A Dagster+ [user token](/dagster-plus/account/managing-user-agent-tokens#managing-user-tokens) */} +- A Dagster+ [user token](/todo) +- Your deployment-scoped Dagster+ deployment URL. For example: `dagster-university.dagster.cloud/prod` + +## Before you start + +Before you start, note that: + +- Metrics are currently computed once per day +- We don't recommend frequently querying over large time ranges that may download a large amount of data. After an initial data load, we recommend loading data daily for the most recent week or less. + +## Using the API + +{/* In this example, we're using the [GraphQL Python Client](/concepts/webserver/graphql-client) to export the Dagster credits metric for all assets for September 2023: */} +In this example, we're using the [GraphQL Python Client](/todo) to export the Dagster credits metric for all assets for September 2023: + +```python +from datetime import datetime +from dagster_graphql import DagsterGraphQLClient + +ASSET_METRICS_QUERY = """ +query AssetMetrics($metricName: String, $after: Float, $before: Float) { + reportingMetricsByAsset( + metricsSelector: { + metricName: $metricName + after: $after + before: $before + sortAggregationFunction: SUM + granularity: DAILY + } + ) { + __typename + ... on ReportingMetrics { + metrics { + values + entity { + ... on ReportingAsset { + assetKey { + path + } + } + } + } + } + } +} + +""" + + +def get_client(): + url = "YOUR_ORG.dagster.cloud/prod" # Your deployment-scoped url + user_token = "YOUR_TOKEN" # A token generated from Organization Settings > Tokens + return DagsterGraphQLClient(url, headers={"Dagster-Cloud-Api-Token": user_token}) + + +if __name__ == "__main__": + client = get_client() + result = client._execute( + ASSET_METRICS_QUERY, + { + "metricName": "__dagster_dagster_credits", + "after": datetime(2023, 9, 1).timestamp(), + "before": datetime(2023, 10, 1).timestamp(), + }, + ) + + for asset_series in result["reportingMetricsByAsset"]["metrics"]: + print("Asset key:", asset_series["entity"]["assetKey"]["path"]) + print("Daily values:", asset_series["values"]) + +``` + +To use this example yourself, replace the values of `url` and `user_token` in this function: + +```python +def get_client(): + url = "YOUR_ORG.dagster.cloud/prod" # Your deployment-scoped url + user_token = "YOUR_TOKEN" # A token generated from Organization Settings > Tokens + return DagsterGraphQLClient(url, headers={"Dagster-Cloud-Api-Token": user_token}) +``` + +Refer to the [Reference section](#reference) for more info about the endpoints available in the GraphQL API. + +## Reference + +For the full GraphQL API reference: + +1. Navigate to `YOUR_ORG.dagster.cloud/prod/graphql`, replacing `YOUR_ORG` with your organization name. For example: `https://dagster-university.dagster.cloud/prod/graphql` +2. Click the **Schema** tab. + +### Available top-level queries + +```graphql +reportingMetricsByJob( + metricsFilter: JobReportingMetricsFilter + metricsSelector: ReportingMetricsSelector! +): ReportingMetricsOrError! + +reportingMetricsByAsset( + metricsFilter: AssetReportingMetricsFilter + metricsSelector: ReportingMetricsSelector! +): ReportingMetricsOrError! + +reportingMetricsByAssetGroup( + metricsFilter: AssetGroupReportingMetricsFilter + metricsSelector: ReportingMetricsSelector! +): ReportingMetricsOrError! +``` + +### Specifying metrics and time granularity + +Use `metricsSelector` to specify the metric name and time granularity: + +```graphql +input ReportingMetricsSelector { + after: Float # timestamp + before: Float # timestamp + metricName: String # see below for valid values + granularity: ReportingMetricsGranularity +} + +enum ReportingMetricsGranularity { + DAILY + WEEKLY + MONTHLY +} + +# The valid metric names are: +# "__dagster_dagster_credits" +# "__dagster_execution_time_ms" +# "__dagster_materializations" +# "__dagster_step_failures" +# "__dagster_step_retries" +# "__dagster_asset_check_errors" +# "__dagster_asset_check_warnings" +``` From 68fb4d8de7676d6f1419deb9a2631f75546ca985 Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:57:25 -0500 Subject: [PATCH 28/37] [docs-beta] migrate - code locations docs (#26503) ## Summary & Motivation Following links need to be updated: ``` - Broken link on source page path = /dagster-plus/deployment/code-locations/dagster-cloud-yaml: -> linking to /dagster-plus/deployment/agents/docker/configuration-reference -> linking to /dagster-plus/deployment/agents/amazon-ecs/configuration-reference -> linking to /dagster-plus/deployment/agents/kubernetes/configuration-reference ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../code-locations/dagster-cloud-yaml.md | 302 +++++++++++++++++- 1 file changed, 300 insertions(+), 2 deletions(-) diff --git a/docs/docs-beta/docs/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md b/docs/docs-beta/docs/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md index 7790f2cf5230a..09ebc00dbc154 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/code-locations/dagster-cloud-yaml.md @@ -1,7 +1,305 @@ --- title: dagster_cloud.yaml reference sidebar_position: 200 -unlisted: true --- -{/* TODO move content from https://docs.dagster.io/dagster-plus/managing-deployments/dagster-cloud-yaml */} \ No newline at end of file +:::note +This reference is applicable to Dagster+. +::: + +

{}} /> - +
+ + + + + + + + + + + + + + + + + + + + + + +
+ Name + dagster_cloud.yaml
+ Status + Active
+ Required + Required for Dagster+
+ Description + + {" "} + Similar to the workspace.yaml in open source to define code + locations for Dagster+. +
+ Uses + + Defines multiple code locations for Dagster+. For Hybrid deployments, this file can be used + to manage + environment variables/secrets. +
    +
    + +## File location + +The `dagster_cloud.yaml` file should be placed in the root of your Dagster project. Below is an example of a file structure modified from the [Dagster+ ETL quickstart](https://github.com/dagster-io/dagster/tree/master/examples/quickstart_etl). + +```shell +quickstart_etl +├── README.md +├── quickstart_etl +│   ├── __init__.py +│   ├── assets +│   ├── docker_image +├── ml_project +│ ├── quickstart_ml +│ ├── __init__.py +│ ├── ml_assets +├── random_assets.py +├── quickstart_etl_tests +├── dagster_cloud.yaml +├── pyproject.toml +├── setup.cfg +└── setup.py +``` + +If your repository contains multiple Dagster projects in subdirectories - otherwise known as a monorepository - add the `dagster_cloud.yaml` file to the root of where the Dagster projects are stored. + +## File structure + +Settings are formatted using YAML. For example, using the file structure above as an example: + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl + build: + directory: ./quickstart_etl + registry: localhost:5000/docker_image + - location_name: ml-pipeline + code_source: + package_name: quickstart_ml + working_directory: ./ml_project + executable_path: venvs/path/to/ml_tensorflow/bin/python + - location_name: my_random_assets + code_source: + python_file: random_assets.py + container_context: + k8s: + env_vars: + - database_name + - database_username=hooli_testing + env_secrets: + - database_password +``` + +## Settings + +The `dagster_cloud.yaml` file contains a single top-level key, `locations`. This key accepts a list of code locations; for each code location, you can configure the following: + +- [Location name](#location-name) +- [Code source](#code-source) +- [Working directory](#working-directory) +- [Build](#build) +- [Python executable](#python-executable) +- [Container context](#container-context) + +### Location name + +**This key is required.** The `location_name` key specifies the name of the code location. The location name will always be paired with a [code source](#code-source). + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl +``` + +| Property | Description | Format | +|-----------------|----------------------------------------------------------------------------------------|----------| +| `location_name` | The name of your code location that will appear in the Dagster UI Code locations page. | `string` | + +### Code source + +**This section is required.** The `code_source` defines how a code location is sourced. + +A `code_source` key must contain either a `module_name`, `package_name`, or `file_name` parameter that specifies where to find the definitions in the code location. + + + + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl +``` + + + + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl + - location_name: machine_learning + code_source: + python_file: ml/ml_model.py +``` + + + + +| Property | Description | Format | +|----------------------------|-----------------------------------------------------------------------------------|--------------------------| +| `code_source.package_name` | The name of a package containing Dagster code | `string` (folder name) | +| `code_source.python_file` | The name of a Python file containing Dagster code (e.g. `analytics_pipeline.py` ) | `string` (.py file name) | +| `code_source.module_name` | The name of a Python module containing Dagster code (e.g. `analytics_etl`) | `string` (module name) | + +### Working directory + +Use the `working_directory` setting to load Dagster code from a different directory than the root of your code repository. This setting allows you to specify the directory you want to load your code from. + +Consider the following project: + +```shell +quickstart_etl +├── README.md +├── project_directory +│   ├── quickstart_etl +│   ├── __init__.py +│   ├── assets +│   ├── quickstart_etl_tests +├── dagster_cloud.yaml +├── pyproject.toml +├── setup.cfg +└── setup.py +``` + +To load from `/project_directory`, the `dagster_cloud.yaml` code location would look like this: + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl + working_directory: ./project_directory +``` + +| Property | Description | Format | +|---------------------|-------------------------------------------------------------------------|-----------------| +| `working_directory` | The path of the directory that Dagster should load the code source from | `string` (path) | + +### Build + +The `build` section contains two parameters: + +- `directory` - Setting a build directory is useful if your `setup.py` or `requirements.txt` is in a subdirectory instead of the project root. This is common if you have multiple Python modules within a single Dagster project. +- `registry` - **Applicable only to Hybrid deployments.** Specifies the Docker registry to push the code location to. + +In the example below, the Docker image for the code location is in the root directory and the registry and image defined: + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl + build: + directory: ./ + registry: your-docker-image-registry/image-name # e.g. localhost:5000/myimage +``` + + +| Property | Description | Format | Default | +|-------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------|---------| +| `build.directory` | The path to the directory in your project that you want to deploy. If there are subdirectories, you can specify the path to only deploy a specific project directory. | `string` (path) | `.` | +| `build.registry` | **Applicable to Hybrid deployments.** The Docker registry to push your code location to | `string` (docker registry) | | + + +### Python executable + +For Dagster+ Hybrid deployments, the Python executable that is installed globally in the image, or the default Python executable on the local system if you use the local agent, will be used. To use a different Python executable, specify it using the `executable_path` setting. It can be useful to have different Python executables for different code locations. + +{/* For Dagster+ Serverless deployments, you can specify a different Python version by [following these instructions](/dagster-plus/deployment/deployment-types/serverless/runtime-environment#python-version). */} +For Dagster+ Serverless deployments, you can specify a different Python version by [following these instructions](/todo). + +```yaml +# dagster_cloud.yaml + +locations: + - location_name: data-eng-pipeline + code_source: + package_name: quickstart_etl + executable_path: venvs/path/to/dataengineering_spark_team/bin/python + - location_name: machine_learning + code_source: + python_file: ml_model.py + executable_path: venvs/path/to/ml_tensorflow/bin/python +``` + +| Property | Description | Format | +|-------------------|-----------------------------------------------|-----------------| +| `executable_path` | The file path of the Python executable to use | `string` (path) | + +### Container context + +If using Hybrid deployment, you can define additional configuration options for code locations using the `container_context` parameter. Depending on the Hybrid agent you're using, the configuration settings under `container_context` will vary. + +Refer to the configuration reference for your agent for more info: + +{/* - [Docker agent configuration reference](/dagster-plus/deployment/agents/docker/configuration-reference) */} +- [Docker agent configuration reference](/todo) +{/* - [Amazon ECS agent configuration reference](/dagster-plus/deployment/agents/amazon-ecs/configuration-reference) */} +- [Amazon ECS agent configuration reference](/todo) +{/* - [Kubernetes agent configuration reference](/dagster-plus/deployment/agents/kubernetes/configuration-reference) */} +- [Kubernetes agent configuration reference](/todo) From 5a4dc5d9e9125226c73ec794bd66e7d847bfc9b8 Mon Sep 17 00:00:00 2001 From: colton Date: Thu, 19 Dec 2024 15:58:10 -0500 Subject: [PATCH 29/37] [docs-beta] migrate - managing multiple projects and teams doc (#26501) ## Summary & Motivation Broken links marked `/todo` ``` - Broken link on source page path = /dagster-plus/deployment/management/managing-multiple-projects-and-teams: -> linking to /dagster-plus/managing-deployments/dagster-cloud-yaml -> linking to /dagster-plus/deployment/agents/running-multiple-agents#routing-requests-to-specific-agents ``` ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../managing-multiple-projects-and-teams.md | 391 +++++++++++++++++- .../isolation-level-agents.png | Bin 0 -> 136487 bytes .../isolation-level-code-locations.png | Bin 0 -> 72075 bytes .../isolation-level-deployments.png | Bin 0 -> 82144 bytes 4 files changed, 389 insertions(+), 2 deletions(-) create mode 100644 docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-agents.png create mode 100644 docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-code-locations.png create mode 100644 docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-deployments.png diff --git a/docs/docs-beta/docs/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md b/docs/docs-beta/docs/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md index cc59bd13364a9..6b88b8104a216 100644 --- a/docs/docs-beta/docs/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md +++ b/docs/docs-beta/docs/dagster-plus/deployment/management/managing-multiple-projects-and-teams.md @@ -1,6 +1,393 @@ --- title: Managing multiple projects and teams -unlisted: true --- -{/* TODO move from https://docs.dagster.io/dagster-plus/best-practices/managing-multiple-projects-and-teams */} \ No newline at end of file +In this guide, we'll cover some strategies for managing multiple projects/code bases and teams in a Dagster+ account. + +## Separating code bases + +:::note +In this section, repository refers to a version control system, such as Git or Mercurial. +::: + +If you want to manage complexity or divide your work into areas of responsibility, consider isolating your code bases into multiple projects with: + +- Multiple directories in a single repository, or +- Multiple repositories + +Refer to the following table for more information, including the pros and cons of each approach. + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Approach + + Multiple directories in a single repository + Multiple repositories
    + How it works + + You can use a single repository to manage multiple projects by placing + each project in a separate directory. Depending on your VCS, you may be + able to set{" "} + + code owners + {" "} + to restrict who can modify each project. + + For stronger isolation, you can use multiple repositories to manage + multiple projects. +
    + Pros + +
      +
    • + Simple to implement +
    • +
    • Facilitates code sharing between projects
    • +
    +
    +
      +
    • + Stronger isolation between projects and teams +
    • +
    • + Each project has its own CI/CD pipeline and be deployed + independently +
    • +
    • Dependencies between projects can be managed independently
    • +
    +
    + Cons + +
      +
    • + All projects share the same CI/CD pipeline and cannot be deployed + independently +
    • +
    • + Shared dependencies between projects may cause conflicts and require + coordination between teams +
    • +
    +
    + Code sharing between projects require additional coordination to publish + and reuse packages between projects +
    + +### Deployment configuration + +{/* Whether you use a single repository or multiple, you can use a [`dagster_cloud.yaml` file](/dagster-plus/managing-deployments/dagster-cloud-yaml) to define the code locations to deploy. For each repository, follow the [steps appropriate to your CI/CD provider](/dagster-plus/getting-started#step-4-configure-cicd-for-your-project) and include only the code locations that are relevant to the repository in your CI/CD workflow. */} +Whether you use a single repository or multiple, you can use a [`dagster_cloud.yaml` file](/todo) to define the code locations to deploy. For each repository, follow the [steps appropriate to your CI/CD provider](/todo) and include only the code locations that are relevant to the repository in your CI/CD workflow. + +#### Example with GitHub CI/CD on Hybrid deployment + +1. **For each repository**, use the CI/CD workflow provided in [Dagster+ Hybrid quickstart repository](https://github.com/dagster-io/dagster-cloud-hybrid-quickstart/blob/main/.github/workflows/dagster-cloud-deploy.yml). + +{/* 2. **For each project in the repository**, configure a code location in the [`dagster_cloud.yaml` file](/dagster-plus/managing-deployments/dagster-cloud-yaml): */} +2. **For each project in the repository**, configure a code location in the [`dagster_cloud.yaml` file](/todo): + + ```yaml + # dagster_cloud.yml + + locations: + - location_name: project_a + code_source: + package_name: project_a + build: + # ... + - location_name: project_b + code_source: + package_name: project_b + build: + # ... + ``` + +3. In the repository's `dagster-cloud-deploy.yml` file, modify the CI/CD workflow to deploy all code locations for the repository: + + ```yaml + # .github/workflows/dagster-cloud-deploy.yml + + jobs: + dagster-cloud-deploy: + # ... + steps: + - name: Update build session with image tag for "project_a" code location + id: ci-set-build-output-project-a + if: steps.prerun.outputs.result != 'skip' + uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1 + with: + command: "ci set-build-output --location-name=project_a --image-tag=$IMAGE_TAG" + + - name: Update build session with image tag for "project_b" code location + id: ci-set-build-output-project-b + if: steps.prerun.outputs.result != 'skip' + uses: dagster-io/dagster-cloud-action/actions/utils/dagster-cloud-cli@v0.1 + with: + command: "ci set-build-output --location-name=project_b --image-tag=$IMAGE_TAG" + # ... + ``` + +--- + +## Isolating execution context between projects + +Separating execution context between projects can have several motivations: + +- Facilitating separation of duty between teams to prevent access to sensitive data +- Differing compute environments and requirements, such as different architecture, cloud provider, etc. +- Reducing impact on other projects. For example, a project with a large number of runs can impact the performance of other projects. + +In order from least to most isolated, there are three levels of isolation: + +- [Code location](#code-location-isolation) +- [Agent](#agent-isolation) +- [Deployment](#deployment-isolation) + +### Code location isolation + +If you have no specific requirements for isolation beyond the ability to deploy and run multiple projects, you can use a single agent and deployment to manage all your projects as individual code locations. + +![Diagram of isolation at the code location level](/images/dagster-cloud/managing-deployments/isolation-level-code-locations.png) + + + + + + + + + + + + + + +
    + Pros + Cons
    +
      +
    • + Simplest and most cost-effective solution +
    • +
    • User access control can be set at the code location level
    • +
    • Single glass pane to view all assets
    • +
    +
    +
      +
    • + No isolation between execution environments +
    • +
    +
    + +### Agent isolation + +:::note +Agent queues are a Dagster+ Pro feature available on hybrid deployment. +::: + +{/* Using the [agent routing feature](/dagster-plus/deployment/agents/running-multiple-agents#routing-requests-to-specific-agents), you can effectively isolate execution environments between projects by using a separate agent for each project. */} +Using the [agent routing feature](/todo), you can effectively isolate execution environments between projects by using a separate agent for each project. + +Motivations for utilizing this approach could include: + +- Different compute requirements, such as different cloud providers or architectures +- Optimizing for locality or access, such as running the data processing closer or in environment with access to the storage locations + +![Diagram of isolation at the agent level](/images/dagster-cloud/managing-deployments/isolation-level-agents.png) + + + + + + + + + + + + + + +
    + Pros + Cons
    +
      +
    • + Isolation between execution environments +
    • +
    • User access control can be set at the code location level
    • +
    • Single glass pane to view all assets
    • +
    +
    Extra work to set up additional agents and agent queues
    + +### Deployment isolation + +:::note +Multiple deployments are only available in Dagster+ Pro. +::: + +Of the approaches outlined in this guide, multiple deployments are the most isolated solution. The typical motivation for this isolation level is to separate production and non-production environments. It may be considered to satisfy other organization specific requirements. + +![Diagram of isolation at the Dagster+ deployment level](/images/dagster-cloud/managing-deployments/isolation-level-deployments.png) + + + + + + + + + + + + + + +
    + Pros + Cons
    +
      +
    • + Isolation between assets and execution environments +
    • +
    • + User access control can be set at the code location and deployment + level +
    • +
    +
    + No single glass pane to view all assets (requires switching between + multiple deployments in the UI) +
    + + +{/* +## Related + +- [Dagster+ Hybrid deployments](/dagster-plus/deployment/hybrid) +- [Dagster+ Hybrid agents](/dagster-plus/deployment/agents) +- [Managing deployments in Dagster+](/dagster-plus/managing-deployments) +(/* - [Running multiple Dagster+ Hybrid agents](/dagster-plus/deployment/agents/running-multiple-agents#routing-requests-to-specific-agents) */} +- [Running multiple Dagster+ Hybrid agents](/todo) +{/* - [dagster_cloud.yaml](/dagster-plus/managing-deployments/dagster-cloud-yaml) */} +- [dagster_cloud.yaml](/todo) diff --git a/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-agents.png b/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-agents.png new file mode 100644 index 0000000000000000000000000000000000000000..2fd662c04df8a9f37e67ef01985d39c7012d60b8 GIT binary patch literal 136487 zcmeEt1zR1<(k>7nNN@`-!EFV1hv4oO+}+)RLvZ&%aCf%^cVD=>yW3q!_W91ad!O(A zg3B}OS;Ne9_jGl2)mv}X3YC))MS#PB0|Ns?5Em0x00V9D{g_7duLqrpg#szIzPoQ?HamJ*E2# z^Ux&87ZXoUbmt;&E%^?9yfvPQby18<0|w$tFB$W+nzEZ#%mmGn!FPPh{IAL;SL; zj7WG|c)-C_vZ~0i-;CEHgPBi!9Z?HddK?0)O*r40JfgyN-++G;!jC=fM?MY})cH;g zoNEno$rnq(4jBe2h6s@snwtm`n;*`;)4mc1-Cv;+*78llH=Lt4Z-1cJzGL=RT*GUG zEcZ)Wqj!aL=jU^U3i?6$-k%8OtvnGQtDZWzOMvc@p%u*k5s6rVAAcn@P`zM6?85i4R^n0AnM)N-IzcfDRq zrPmw^9%4agO?T6#pq-W$X8HSuH!km8FkRqhqaH&j*00Z;U41;cKYs;uCv76FkFD9P z>#X&`D1Bl0k$^0*hIRpi3;PIL7JwVz{qwzips9o-#V1t7(2`J$Zs-k>4TB9X)KEiy zdty4GP%)@nxJmg*BL%=dntFnzzzs1)^!66Q)|)NXi#HcsaDn=Q=Q-Vq)s(oDPVtxV z1@Tc5%w!g1k*GRm2?_}cag%g;vxAUE21fN4`rDq{_S>6-Z1KT`kj1bw!Gn3_xhnFq z3N1w+sF^V|6XX)!C0tVnE0q_%E8I|$R(e*tFBU0&R2nO}nUG@^N=1qg)f8HkZ4o;x zFxby`MC3{MEWsyunu}CYRLoH*QnFpFT$G`5sQ2wX`8&i98C5ZMU+tpK)wM-4%IM{E z3Rb>(OFsMe(jov5{62+#`rfM@F_tPN%{Rh4GLkxyDqB0QfvEvsYhUYZp*@Y0Dw4I$ z8qQW?2&_^^s?)QAKejzKT6nCju!!2*JAm91b9CNIpT(Xu+H>E}Kd_$R`OZ}+I(0dY zwr@4VRdO3!)N9Bzm4Y!0Kdk3v^~~?p^z!*x=9%G*qCcg-p#LX-iXdU3M7`>Dl`iKl zy$?L!-98OP97GV{pnX;E72G`T#qTAQl9JY!0;V#jy-PMvwwJn;K9nX*lON_DNlcwd zUS{Y^tr?ygc1c;S^`*n33&2#yWTVxSq>zl+rXMPwPoAf*>8Y_WcF~L}n^T!rxmIZd zzB@!d6g)IOoOSrR(>*rNU0^WEhAW!pnD)#XWvOiCZwYUuX|=kbK6idpd~`CWc=UX% zI^QtYVOQyh%zotDcq-9KW4A0?|Yx(YTTjv z=Kb){c=)OE;q@1*Q5hpbin8S9dy}5gS$4T?SlN%{Z$~fVaVT#LaIXZ!B@e=1l5lE??R0% zGM=7$Zr*R)k7N_YqcxLVi6@)QO;+k)>S6`Z6>6MuFR^;Mxr##4>5F$~hfIi__D94- z+g1#hqPf9U{Hivg!c7__RaQ2(OzjbTIuU(=f&Jzssh5maw#5%E)Q<)- zj-%&V30ih1*WBT)PHupU=Sv^0pwGRUz;-R4`-AN^nmL-A@&Fa`GD{t!#$r=t9|8U! zKW4gT%u0OdYN=>xJUhsZr7dD)id)%G)SH{{N$9L|)4u!o{nq8F|Dmc}rKG<0<`DecFVlbhEnu2s zb_Rcvx3-F)JZTn*K!)A(GUPNS5xOL(E#hnMyJJK%XxR)o)r z+ZX4YTv+o#3mj`q*4x_dI>(H6obxLOK=%!-LmkBQNj_yKjpM6crfwz?lisnNEPS)L zT7(Qqj@jAv+~+^SwQpZ*SMp?eU3j3oTiiVVdTs}+ zI+%e6B^UB@U;wO@$9EXk4s6^Sf?tcBMMoBn`8rECw$o>$2-{O39jwqyC?88$2&L>M z6~~MU3M_yaZy!q6!z1uT(s1C$S+PoNq~BD5m-huRCDwKa%=4IlkkEpF@Kqxm{Si6E z;8VfQ18g<{w61JfM7U1BRo67c8Cy!~{pyBWnj!8^_^Sq!>_Fvt~+aj%w0UoCY>l^m>Li`bPAwR<^&IfN{HWf=;cB z9QBA?tt_n_I9+*2em}tpI{$SUKtlBU5l0Ii5;bW#A|V@lBO+G%kMti&c;SeMh`8+y zjX4#BKmFAl^dAq2siUJUCjj8$;zIAjL~mnn0$||a-~fDN1TZqvfu5jqaI<#QbEUI( z`0%HbfAu45p{Up;*rCr2IB-E z>3`h%dsAfxBYPnmE6|vZy#HRBzZ(DN&A%FQ1Aa~YKSc3I&cCmM1kDS_4fvl$vA3n;mQA7#ANn%*Yt>!2ipVMo60)M{iqu{x2ev5%+0y_uD=N@k^%94eh2 zPHA#b8C~)q-&hI5Z3y|&j~xAa={35$8@1{C2CRxYGePG|;-QSb37x3v7N4zwboy=C~ z@Ga+mZm`yl^SVsFKA2&0L!kK81<`jH_U_PFd4D~;|FV!`kznCTXcRK>Ge^o_r6T?6 zbO;#@L()kgWh^=rlL=ww2*E>AZd_!mNdDfqk^@4blNl2*boU^qz8a=fA1V8VCE=Pb zYfbA@7R(u6+JumXBe&waYYRAbTNzwH=PR@JnU8m=HZm8jH6lKzk`LS?k?u8Q{EW*d z*?65<1%=k?U1@QrmI`==sS_IpOqjpx6Hu}y`}16O+RCnSWSn zb}?e~pE5@MFy7|jG6|1BIEs%5&cvXm;+aXuG5QGzAI!`6y z^{x6EbOC4>V=xd)24s^T?l*^h-@#fKTHV;cPTr_y(GSa_>M@^Bz69Bk_{i>2;L#5Y zT96CcF3!Ix2Y)9tOr_dHnI;>DVq;vvDuM?dstnthHpWlcz$;wRm`wHQX-zYF?PhfK zaWuWaDQJj%VX0PLwc8pYb;C>A>(^6rpk>DLVOM)R!8tFnG4>Q~`)IL8njBV3ho07v!FNkZ zP73S%^F7F6DH(>=I8zzU8KP`B3P*ZomZcYE&36Z@!fQ`YwbAWQ^%Q0puNQJEcE`tf zK9O+*H#QbrJWM5T36GJsh+m4deH*}Zr6lTvyi!?sO6?rzKxUie&BXO2Oeic;=@w6? zr7{&9Rx+%D`iHd?!-6=Fqz%WUwN9bdLn55|z9yPaZ?;RY>?2$?QH6fb9xK@6tq@qo z$TqC@p`?|`47sVfg|yJxe3-|_s-Q<_EFX}Dkj=-NE?jVnSMZ)8pC!SffPM6! z8a^F>t7%KDSR+Wwf^S?muLrwmUJ#4S0&N^V#qvjA1+3*vu)YqY+PI;lrcuVomz@p| zpr&C=B<4O;#(!}gzOWGP>p#`{xodCu7I=M_mfIZ_>uHs1@DP&EV}O!M-Ds(NmYhZ| zgEAaTT4t{o+6ycz`wz-C=1XlNtquokXU)$f;4(yY?z?2g!0JR!>4Av*gR{)7^^|aD zz{;n-dV)lslT3o?qX?DhToW4mYulNJ!O{#lJr<3#z@OrXuU9F!&6i9JmmAraO1WC+ z=p#O_VarNwb%3TuGW0KTcuRXE@XE9rcCj@GJk1`tP5;J5_J`fzK*q%FnYg6>fu7Ku zpC9(WR(?_XNj?jYjv3O|fS>;X_jwaGB~St@2FS~ShJu*|gW@>S-7!$2=52oYGPIO1 zZ%_?oQV`WVdE4Uiy)}@VN>m0bEb?o<4+WsWW)bPbE6byyPs%+qIjx#QcoUg&&UXas z9U1yjNBute)VW8G)QeHsZiuQTqvb6ON8LIjV2LnI{isr6^Umm->mz@#0IA2us3P^i z>d~F|sU0#;EG*jH7H=4s2oUl9xr%CGVBuJ_pY7!JF;7p`n=_oBno{5rX{El`KS`~g zrVF>@5OApv==mM;|yK;??!wBsQ3W16)l_!D`sX+k3Kgi z5G*0)tMd88**9_yN$x}zJ=kGFSh$)trgtT4#})40p7u*#2EWR{ccfZ^qI?P;Lfo)D&gA*FojY* z%D4Fx8=H;oP`-W?qsED~Ph^|XXEL3+B~56!cJ%S9`rq-6?=rF+2__(f?v3hmwUh=o zV)8Yt8|Dp3uqEPjuBc?w3hr)skLE2r%b-HGf(WH3qCcg&+!2K`d-={_R${E|^Pp0b znUgw8p25N6u4@Wy2=I71p_-NenvI*h?%S;vXhpzAa@veo0Y*o*FNC;Je%a@H5 zDk7OtkslR3d8u9^S2$u0T&##IF7MKhOfOF@ObOIvQr)tSv&k0yg%plUte-CtDW77TTgMfw!a;L^L6>HhdzzZ|0aAjI?vXaD{Co*`m-779)7eX<+QK9Y%sbb6IdWFy4 z+i~R@b<>dp_}pF(+_>y5-#T850NBBo!z*aDHw`|u48!woAZKGae_v$d^H6*8YFcKy z842T&*-z2=O8fl$oP~~qp8BoDWA>!qf!Kv$u&f3dCyp8=>Vxpn*j=f;$2RdTw{^Fd z*YbhE!*;}}>2<{`az&u_g#*^Gsc>d9LcpK2-$nabQ`N%f<`MY*_gTkiuNRjD1>cTZ>{&w71+%oSCs=I=< z10Jt278ot?XDl_7!c?z-y%Lteblem-IMNM(m3ZaGQ-mp%i2?k+y)KTdfo|@_UF+y` zI>~ThF_s13M02y(Idp{EJ-oZmf*i49Q2%B6+TKHo1^Ua!ul8^>(x=n_xt?C-JPdNx z5LA+)8hX32K>yu6U?47|UhHe?^CUvnAw!8Xp`$pF=LL(XiHSQZc2CQdT*!dWyN8H5 zjWf;2kiKLyUux8TG&_hy^D=1e` ze+OGcM6iY3d`5?N-5+vjAD8UOj>+*WV%XP)w*n=xSa{3u`B22&cfy8u(<=*>JLwnP zR>Y5!mspwmbzuh{iw(yzBNncF#0dzNJ>@cFKr1{4CEZU;qOTUY@Bq)^SyqHhS}|@+ z9A>Xq6(e~*NVcuQgSTq0b`r$nDl8l5M4BWiO5dL=#y0kLGVqOw>3*TxRU0z5y5rW* z^6PTW!x6Maq^auD9m(69ULFm(A{-;bwq+9U{o->;symfvZXjQ!2TvbDo5wZl^WA|M zmEnaGrk(n*w6vb8qVlAEO5Q^!@By}ydg|d+{?n)&>;j%F2PN>#Jz?^k*f>A_{ek~K zpbx1eKiSJfxav}&8kB#}Ry_x)il#^njWL4!@fGTqb$gYMUE3!RTsSqTUE6Z?=4aBvtoPF?WJEy>D(o0AGG{0~;mnc*%;^X$?0tQIfHSDDW;CnV%Xm%PUt5eahw zBgWjJ@2Ud4hQe0}+TX7X4sjmgKj%6i^~3Y>dobE--JVgZdTw zB~Q56*`o~*DVko_R4SKvBT&-6Akgo7{#>R>;`X`T6KUFJO-NPxzV72A`Nr6rgpOri zzLoXY0D}lOp|?OT|jI={o(( zK?qx+cM`cH%^D~Es=Hc<1+<;b`D{(-tJ3=W*gT7*0ui0jSe5vU;A^?04J$=P@|Ij3 zE~n5O0?}9#27k52r{KWay%P0FPI)J-Hf5`gxs;}_rTjBao5x*Tx7$?C{=2Sr4S>%D z$%Ur-ykjbfi1YY`*BG+x%VnMfRCX0ci zl;~q}labq_Gm^MjBl+rYQoaiU3;mWSOXDKl>!k}h}lha8?h^4_*#^S}&U($~E9~7)+Zp^+C1GmSavZksh zF8BLC`OH(m2Lceh6W9?c!oxs;B7Jo(pddVW6fHbCLfCUCETVYw3*hBj@GU`XO*k4t zCf}Erd+%&?b`D}BtgMHG{q5`u*h3j1E%_{HjQ#0y1IP0w9|VAf|Y35GcBT}dVCBAK){WkF~*U78j5 z>vzCRV;A2rRssH|#@YR>gOa|FHGCw~(OlD72%n zTGz~a-EuX3LQCfh#OXN4`sk7Jw4~vTw;*7eQuM^_o%OsHK&dg!XTlJa5o4TtIMN}02EC5k(I3r#;0s|qHyAtY&-)mo(>R7u}L(pmoXcv?%{UmNHE6E42V zLNU3tWp?smOQurK`{#zTPG$%sKNeyJ#)bv90x<4~^<*Q_f37Eec>$?(MG~(fu17;z zt=4Lx2BQvlTs_kr6(z4u)yZ{8u-0W}QXH=T;i31jN7qH`VnK*DY#K~*~W*|(ttSi zOkelRTu4Ze!jB7T|2Ks%+~N~gsb1@uhD*Ur|Dvw#w#7uc7K(})#}P1YDosWuKb^W0 z+BAm=j)5dksM1-l;ya&D;aAHIh#ukQIeqs4{SV^=1};G4A|M2jxrkNnA58R()+J#L zitqR*K(?A<)9?i9#SHmG>fGHEj7D zU^wxCo3FYzLN&(;s79xfq5&cC#M;2qE#)AdP4Ev^bGnf^QeT1{=FGGrFwE$B0%px za^eGw8Wnf|TID3J0ojzoSX6x2Dl${q7PLOj-@RdU0_>4mWhhWaT?Xl7X>9O)rStFg z^D~42;c#eV`hQ}HTt2@}Mb&MhM8_^fSa&h-7n3QUi}&D={2-MGNc$KHMKL7?rf-Y` zp2P9?sr<z;GkN78IU|&5H}0x-M`|)$ul*qDvoeC4%^XT&}e9mTdhenm&N>Q|EgW zNXAfp^Db$_Nk;}tauhjz7l|wYG85B&ke=E~(l9<>G|>A+OKH(J{`)_+1pkFIe4K4w_zV$zo{FKg4^C0@HLWmsYFEi)6)D##KJn zJ-n7CV0KI`eT>__l5W4{n5`~UFdi2|VRkU2fvdye9K$N3DGj1BfSe-3@SwhBVAM`1 z09@c-jQ-v^7>psUU$U5gXWy5sV3QL_{~EBpql5IdR!$Urj1NnVfM`WTii>isFuZDx zLP`5aQfdvZNKZU*ihBI~kV-KS%fP|G&Ru~^*4yO78*RTcDxU79rK4|Xia$&1 zNLe#3A7!UH;*u20BCq>P)cm@O*SUW!U2Y0k_b~GZ;kSQjO|%>2&FN}Zx&Lg5DnR>D zbobA7^|;RM58#d~r|-7uql1eZii9R)-{~7%v(~W)(A}vZmt{4l%ABwZ>ivl7U!L+m zXq!yN>6nP+hTop*%vfJ|5Ck@UH3Y%YJfFRbtf!9Y>arfX>dV%L{_k0bE5`Ki4~!*`t%A9xriYiM1qI<{=Wc z=G5t2T;o~sfvl9fgCk$@bZ)aTS(C@UGx5<#oD+_VjevmP?cT3=*l0)NfU_ED8LqJ$ zHsu$@rbW7(+WvrLHk9tfd{Sq#SMX+PL*3QfpJy@D4OQ~><%W&rcs}d`^NJKxSLifQ zZh16fuc5&w7p$X(`KUsEYvV+4a}|#cY6fMf0}UA4V}yhoP}t%;Xx}uJ^C8LLXY9Ni0_zQ+duIUhF_OfUj*<#^anNQse+)W=fk%-Zm{K-~_^D5Y0a*N4Y zW-e6|lDyb-PA<6nj#7m=P^$_NKs-Im56oXomx3@z`2XP>Nl805$l#N1$a%@N%oaus zAqiQ6P-E4nl^}MYM<@(560DQ!6DgG1_cbNuzEyrl4ClpggZJ+sCC7nK7Q_`YznHdg6 z32R9R89Q#my1btY)M%YiL1Vao$QWv1CazfpyeU+u!Rgkg4o`_$m#+?wU}BF?6Lm|^ggPQ=qU@o;*U}h2gPt+tNrOn14q|Q@O{iNL2|76=p_c5 zIAd&g$01@WW9xKlWX2&Nove1pk#wL0MmEMK=2X2KuGw}Y*|g0Q|9mFlT&k1)*y;X{ zV_mZWjch%mW~Xy#JE=)42cTkC`3R-W`V;f1;+qDnRcl$RhYqO*k&HJMFL9TKvknGq zURGe=GxRZ9^C}OIc1XJCgY8f17G}{f3j=dM9?~wo$p0eti8vq*m^|&Tr=3Dl!XWGa zWjzETdP&IPO+wpHXi9`Jfz$0P_HybD`G#fEs6Tt7yD^4roO4?O+ZRyrT{iwFA!0hROUz8Q6 z7@&P7bLp~qXp`XkgSjAi7Nv3xEz9YADp2Yn7%}`VL0*uwbT%&(o$=m%REs09pCcwc z9f3pxkxhuzp}jZIKpUrTMT^ZUVLit+{rOzUD$51O#ezOc`AZH~aYCrevUE4l5Po7S zY>3HXuqDaHLV4cF`-Dk8MwTAmsPbLeVjjai-6~y~0AX+deE8_L!t@c}_6M`=euzUq z5D&^H%K8wlmW@?pI5TSbQyr@9C1MlPWN^z{!7w(+t*0sjR(Ta|cx#Eekjm{--EdM%hK>mkh$vA|DJI z#9_xDQj$M-9v&7Lt>S_6&^_ZeNFUvsYa#XR%kuLODl*1C=#dt&9(WN|?J{^v5xW!7 zzHN$kC2v2zb6_faGNYU+Zrx( zOoOq74wa$(WR8f=3o#whd~*VneElTeM~CFrmp#89)d>bM7YlcHjk<%scGI*L(>kDu z&;B{;%YZjZ%N2SP%tS+#n1WJ~IExz{3zB>cw8h-phA)a!cvioZz0(o`nQ7!xVAx-3 z&OZ%G!HrDHh9*31D~B3ObO)HmG>u`By*~rSSHj9O8Jr-&e;QuELlGOHBy&pde`NRSTobgs$iYvSI&vEx(sTC-}DE>1x~(d>Bih5P{_ z^%*;%TwO+k_A{SDU~en8poV2)nPMhfXg57n~-_Qm7F_Sj836t2sLc7vkZ zsME>t=LO03Ud*H=s+OWl92=chvOe3PO@ZS;GV)jT9Ol!h$gp^tPs%-QLcfXw{H{O= zekPv%9GgE_u!ZDXfD|Te60LeH|J>FkGDF#E;vv)<12#u8IX)}o8hs6|t=(ff*RAkpd~{<};oR5FT@*DN7P2fcmO*GEcq_L2 z_IltIQT@=Pc0^6gr^~7I_#CnTB(I3iuhm#q_XcH&v>_t0&Qvz!8%mgTI)cDg({A4Jvx#1uz5EVuwi<+{%z)7Y3h%U; z?GX`DUOB47hEAR=q0**!D0o^g<$5S*w)M4^H04~1U?!)8 z*GMAnR|9a9@1rFR>2JM?{8H+&REJDy?sq>JAc3}z3M_jFRtdXi2KNraU$H&52tRC! zrF<7*^ZgK|W+tT`)At=eyGfZ&b}6vikjG@_^5VDG9XXNL5XSmH^NPc*24U3{uB7kx zfKxwJGBpqLOLMM4;BD=$E_hY-=s5VJjg3uH@2bqm46t}rPj%mdf<9K$+J#EuAa116 z^;V4(Qh|Y~3BNC5tD@}0ZRu*cd$9vVzS`M;v!DP>Z5t$ol(Eda$DqCG>E$L9jWHEV zs0@w@gm)oIDBAu2^u~M!$E6I-V;V>P6=SYuo`)2Ta)7s(TDz}fenm1;2`Od>- zYeg)SSJjRO6%HS2`TnMdC(UMjb@bt46>rmy^LLt`bs2F4Rfq0z1ax`HmP<)wQ8TYy zlX;TICqA9A>6UeXYqzzormh~OlpsQPODDQymj~4MN<|fTYX{0gYRC?s{FBWP$)^95 z+VoFC#^2>>y-83;AT$vv!SQfoEPN~m`ac$l<$9O2Hx?2sAh`DEpM)37f}(!{n97hY z%(Ov`sg>^Zt!^oc*5F@BhY=DKL=(E*T)|@Wb1In;kGlK?d>pYM8K;%R`XddW^)-%% z0h?6vfa-58X1&xGSeWUwpNoH1N#eFHe8p*-U=%-Bn~@#D)Hl}HQ}@xlP5}k)*Q+JN znR@2kQjIAlH`-lyf6T0kG=I}j_4PX}OvdAx*xq6rVNNL-*a?Yr6-_5z{Gi%HdUZTt z-#}LqR!jS`nidZ1dwq209G%A(A}DF^=k+r($`n!GW#Hq>RlBWtR06E8D|tGkm<@5N zTaDymzYF~`y87CXnIQRXuS&T#uZRC)|8iQR$6P%~Y^UIk302)WiEI)bGD zksXBpZY0r*&pIK*-COYZVTx$&#_vg`DjF~I#nA)U8?#RjQH};tVXh?01Cpu$1JrVV ze}Ab3#r%5NAovvO%+i6-b<$r$g9>i+jZtlFVSlH3i9q>p%~WFE{AXS)wQ;4yhE@p} zII50W_i(Ak#Da7}4MQsk>Nw_C-Cn+T-+cKsU;=`Jrug`1WXQ4Wae}&7%i+%%tB3d;x@HS=gF-e;rCeoMPT9Y#Trz@}m#M|80uQU?L;%)#_ za0vAzuwbLLi>ZLDYI?}hB@#&cFw1?LNN>#>C(~u`1$KkFrCUs3KIO0V)CE8i87~ZF zbpKTsLMzcZ#~Bp}dRa~Msu`zztdi@E^cm^tLGs)NQiBpHbddRvN0ZaTMm*t5zUT#w z_6hP=Sjnrykt#9su`#jXXqLW9yFVg7KqWlGrdO}D4l${*KfQ4&`!5Kuku)d^T zo$MU3mSOF0%y!3Q`mXe}TH+HvlYw&1FuA_`*g9RWW4&#f3gd9f0-el$KaIjCqVILa z&wSeRMhRV*v|&wt(+QtoLGqcv4L18jm*fJ#+l&YThOdmQq}vY5n661OSVksNcXLXI zO5GxpabeVG{BGVLB#u1X5?W-?>?A~Yf}(=GiV1j1W=iJu9x2Yv=n+45NiMiwYdZ(w z7ylj=G(aF%7{{No7DGz#w(zFUOM~O$x|LGIo!O8|j6{5V9a1U&WeCOWKYEscCwM#X zAjyhDh_XFQfwI8+OD}PWN0?rJa>&WQYTqEy(;&3is{h7PAg_sWp@X=2$7CQigK6dn1Qf{GuQP3EqY8bp7KxStxdgvJS6 z5l(;JFj<;ER83-JpSgl9}5q z%MAHz<{qSLrcvQPQ~&4%KT#cIyfS&8bU?V9In{zJmrQq+;Q;ERx*TV@a8rN&Ch{H* z#ncMs9Xqp@W~>K+i`8oSZ_>huSPOc!QFQl}i!M#-kdOECR6Adr zEGS3{cR~D3q<|dY zBo`9PSZp$oa?_g}6#QZX>n`Vhn|V#L&K3C`%#f6wSV5lOz{!tRz?bg4vG6NUU4KH2 z6h_UL-}hDhi&H)ne-1oyr5yzDfTZKL0w3M}mKelehvMO7!pAd!3U_d>e7WI0#4n(3Hus`b zn$rM`Iv@Ru1d>;)B(699iw3en=)Ep`W+#IPs;hvy1d+Y{??JGfDTwLQAMaMl0CA^l zWxzK4bE@Z=jnEWSIIr7d&PzWHQiFa-2wH0Td*gDvrXPpWRI+}p3$#xvCyBG(b6kO1 zOIoJ-6%&q+>c$r!+H&Ee6-LIA-B;f9OYA21#>voQ(Y;$F@0o>Dt;&>@d#7k_dex2` z^7!iQt@Nn;@gnWMEf!2pLQwq(O&`6&7{jTAz|{OqO3?7@lbcR>y%+-%HHie5Nox&= zj?uU|=5}kgyTwFDca0i7;t|j^F3&6f@c3S}?>CJ!M+Qy`Rz0*BCh}(w=LUk6GENC1 zS=He9ofX8WTMeiIV0?WeP+ZaRNKB4S_EBW}x8j0PTF;U8wzt_JA zBIsC9$30qR)6RsG`phHDDQak$+y?vZVeOGC?Qf|yDgI84r? zfeN3uYK<>@#OwXbg5QoIs*m0Mm>AHtxFS7hT5N6~{WrC7S-?~H30FrwaCzeoB~XG4 zN?52gO{r1*_LTh9knt9H8%E^ea2S7WzD{G9Zr;#DhwE}J>Xmd3O;_Jlb^VyB5?6s{ z+p|{Zw&h1aoJqFl)xmC?u*x}FEhyzoy{7Fur%^K{TA_IP*~(BGlL*U;Me`&J<=GBz zc&HB|H_!-F0*Qj-o!Y;^gD;>B zDbH#aBse7GzHsmFIvek^9|9(BdPHXPNV*_vdSotm$4ksm3hBH&zNrd#8z{~zypg=# zzfAbPABbagt*uA+3mzP$g3&S^Kiufs{lQX2#9(Zf({eQ(tWVs&$th1>2vPoNZWr;B zD*hs0r&^*ELFF7&9?o+lxe@tJ4TxuaR?BF$6Z1`5!va>Xv*NuScvI&N`zK|kq#)i2 zP{uSbak-xHXdjq<7u?gkh*hB|rNq1pq9fA{`#>>>`1tDTD^)=EWbNkINVWHA)v0`) zO8YN5YoW#RZ5u5uThr6R?px~3Lm{ju1iqWf1Y;-p(_QP;@%koQs-txS_v;!qN|W;R zYVYzV=!xr`LY*o*Ck;NMChk-1fwZ3l0TL+jvgn@KADy0y8(M?jB3G(F9PI~;$phV8 zEPX!TF&jSKBGddSTdw>FPU^#eWNKYj7-Y94g6QwGH24YT_hfgG(#cjM5rpf{WDB*H!zRg|2hSxH z`VDD@Zpo1nXEU_Z3tPlG9j>>?E-(d64sSUJ+tvE7s1vguWbp+p+fs&Tu)Nr9*zE+r zy`i}sFXwA8y@E@mr@p;?X^RVDd(Y2>WgVk$I+Iw}&tpSRRrs0H-p`qCCBIIdD}9Ud z&KS9eo1w9|FG&fhQ5SVpTS|Z-Ak!TU*zJNg_Nkj zn6}aySr-CS#+akIYb}>{%M~lc`W9Z0;R*vq-8kPn1e{~WbKXt2mH4k5@jp<{1ZQ`r-VW>A2yUr!6KH<(3Vlu zr^4^hr|6q@?Y_5fK1@-gC{EQBV$B>bg_t{;66wfR$GvIeXwGuvdbPOnri~;eoF|vp!`ys`{7LJ-UcR+CpkU}(&zY#;fa=q)YsSFk^=_^ zcif*O12OoRBBG*-Rr)=JQYnnsEC^o>$ieG}6$Z+J5b@28X_O>k{qW&g#a$*rC0ob? zobOHwsm!v}mKzJ8EvnvYD@1o6Y_ys|90_{9_Ikw$ChlXVH~>CS=qh<8%F2@MnFDtU$$p)Ea9}tc7J0hmS`ZVco@QL;-Y6JyS{8D zyB+!Cp+(;Veq=buQuc* z!mtY<;wyg)15HfKHxQA&)yVEcS)ztORgmF(Skb{$69`%D1o&M;l0@37D(D7zlcZft z-NYk4GOs{T@Ifde;0vTm9$}*DZ zwc+Gfp-&CmW!*pt9m-eu!0sq;t4R;alGE3*F%!62y)9KwY}>ReZ@+iD+{x4H3Yxgs9?C=R6*Ox1ye|a(uT-wZ@o>MJ z6&fgI9j^Zj8LvB_iOI?#Q(9F=ULiPB^Mav1_U=@tqeQxQVxXF^${+`U7(ZQ`Q70ta zYv;iG@TYXM$2KNUUUigYt`5c+mWr>U|UR%?kVzhd9 zhA9>7g4tH-=O{|Q)dk+EJghfYY3zQY8^QGq_ zYIFsWdjrtT+T<| zc)cF_$*T7DHZ}wuc80UnYmCY6Acj;DFLy@bHc0v^OSN0-mrTGF626N@DHO@)H*Xvp8(SRoyP70W`pDSt5G4KSdMnMD1au}s9`{^)gFa^% z${(>EVAbEUX<6K$TE|pqizAOqsV|Lj>`${<8`J2GGo2X`xEM$<>5_)+nM{%v;}#KN z!-t6AxILstw?XN`EA3zu`z%6s&C6L_pPZlywt|_>q>0jO#l0l|j<_0njxZIDH!kcd zc z{K{&zAbi^XNRNQW9({ANELmP2goKywxtrm#LX64RRC!IVX~OGz#vhjDbTA!*S>O+i z5Ig&N+wqEW>Hv|2L8CIh*%uY6J(iK4zIA=R**D~nCy@{f>*cCdK2*L{T-i8yEN7Vn zrdfJM2%q$nCudVM@Hluu!=cC2`C99%i#$-4@oBEB&vA1YV^DK04lbUVE~vYgH?MJy zeLXgDyn256L7yb6f6DTDeLaOs0K>VX-Q%8o8=xL-?}4qQj;18BC0vV)*2JD>8^dan zxl_s(lfAt!-BCgO!bCr#0gSgUPj}6A4aC~o2H|+QuG!QuLoOFIn3SF28P~71c%r(T@T0X>3HwP*c|9+>UxaMBCE&>z3*~F!Y5aKUd#B5&Yzv)RL9#aZpDm< z8>Yc%qrE>KDpor#X-bYDJ->(PghLw(h(}$kd}rf58`bQ21D!GFEf)U0^t8Zo`9i5U z2nmBuH9k!FOeQ9J;M7Kx?Jkt?38DvyPPj+da?!lZGZf8hKb#{A>aLBZX%6(OfL#d5 z7qv#uBN93##Ie~4j!Of_!zNHV^u@(dv8I>fuf5Yx0;$$%_``PnXP{EEqgj*rHQD*g z^J6z=01T>#6JdqCOCHKEl{P%Ys&?rK|^*LtJt z9=BCSTjVGqI}P6)7PWQ&jI;GDUR}c^W4)GG;}=iK5S@Pr&p!2N@?~4 zS1Z{lr@2QQ18C_G`0xrZL#;k!^=vpgdz~0tk8+Qw1ER3eU4s$J&%cZQAl}-Di7T!z zL>cR;7Z$OxzrFfqe@!EgvM%v*YBLlXIP601Woke8lqU~BrC?KiL2W!e&9K*MSR$x3 z-J-qFsflubKBNB?Hw`chb2MEPkWD(BA)G@wKgTC*T+_WINjN<8mvZ)mVGNaKkKm}b z@g^5*6)BJ@7SOq$6v4P4-&1;di~S$2&N`~9HSG3+Af?h9R2u2-?r!Psl5Xkl?k+__ zV$-$h?vU>8?ykEy=X~EC_cGuQVhr|L>y77~^EaP8!RJiFY37(j3cc?s`SRR#A`%jq zN?P(`E@kCmcAP%>c8%Ws+VX6=?^AAOk9Rs}YlmxR`}v!C zcgRR)Gwa~9G#Axn;pX0EG1w-g)GKu#rcgR;-uTk2CSlm_MU9rFsC`HCT(Gamh?N7| zcxI(9IrH%ny}0=mf1#sRv(6fnnU4lNe8fyK*0#t? ztt@9#Q+V&;)Rk^NvtL(C_9&uh0*=2Z882zTpjKxtUMM8Iga37me`ajdo1MPpR9jv$ z){|U{(?~x00A7@OD)^XGI$#liT~w#kWP8h<_(w^ymg+!LXQPb>*Yq7vwRNkbHh8Wj z%XWrWP>c;f7a&jM!5^<`8YGT{1No|k9GzmPhB8g<54J{TKWG}?Z?kYA_&OEn{q-Ku zBUJiQQeKR3mIzEM1}3eq(2U9kSzU+~7dGYq2f0PFYX+L=?dXoK>K7Fg8T0>glNZzP zNr%l4Z(dU@Mje%3eG8r^Zv8oJsTmCH%N;UI)9|qz&&l%%*Qv2rcPyEPBnRNUsoQ
    f@(wsqSiAOoCR7auj4;ApLAe*{kJ+Q)FW4r?wS+Iit!hX zx{dG@?w`|@oUb3>nkpQyw&V+&Jlzc$be=#O1Ao?Ajkru0l4 zuoPaa5G+tOP)=?>G9#-Eyog>gLn%pspFRTn9Hhm}J*BrOkT-j55?Q5IMfEgae~4US zBbgh8ZfEoVeknyquf6F@!+2* z+0rS8jt>2W?+&lc?0b%#nbutq3sUNMnktU`N6(odLKzjRe-6i^sZB|<5+SLfvF!kJ z^pBQU4h@ZSVSjVHG(OHC=`m(nELj_o!RO5j)~E>+jmD#)p;Z_5II5Y+&D-*Q3-lmxvNu*b}QgS&tTh$*C$6~{sw-hdy69_wsh?A@H+3QW{XfSjsw>_8x z=Ygm!Vyb1~ZBN}(?q3kzdCIh`2JHebqF2`ArVq-bTHhwagWM;{yRaRsX`{6+U4_Y% zN_j?PIpIeQT8=DrlQgDLRoN`c6;@k1xd3HgCGi6Da<4Xi04^Q zy(&pWu*CUoknP38#RcDHa@Fxl6Uo7OQ-%nui^Jkg+#$C=U8P>XH#PRD1zanraE5sK z`~V$^Ih6#@i2kD$yR7h5(%>oY@64fG+^BYElNU2Li^Ydj=C==@RbKE4SJ-q&bfxwi z>^yl@Ubym0*D|)?PY)DCUuHY_a-Vp2RXK^her_n#h{zy99%ArDbs;;t z$3Ic8wz_4r;@8$5((huX@wkLl$CXfeJ-)}ZSoiB%RVm9Y|Vf=p0 z-f5S)aOQt{0b~dJ#bL&|gt%Y`R0T=LvfRtu`e^ZJ?e#O!#rGl$bw9MKqJf!S(7J|( zTH#Wl{!syaFj45Q1}ib8!@X|T;PzKqTO*YFFZSkZs64NxR2hGB0E?_=KmN#EWgM7r z2CPX2+ug<=HMc>q)HCc=Wz{uqR~-;`p=LZZUJg{2OM zBWs<(QdBZiQLwTgrgUZok3Z`ThCLDTGP@mtNcJm(gAIHrX6URHzq?@2lS`fxNU~-g zjmK`0epZI;UFqwl)mem2IjkaNExAjMR4w!^EA(x?t<8DVA2qTc+#oH3>|Di!zFE$? z^aWs|uih1FbRXcJm&BmFOwC%${Z`MjY%5e!W$Q+?=_-FN$M5_HC+UR!wEWahA_;G1 zsQV-dEd1fVECCh`5E9k_y-(m4qg19*;Gouj9ItV*f9Bl}dH=YfNE3^w>1w+c0Pmh` z74MhJt$*1DqA#-b`w6g{@`>9leIg~1gG+;TThkekphX6?L?$wmi|2@67^t5KyNAB) zyjA>aV(;iVLI}-de^rBibgs=ZDHUlzZz1n7cd~mC9H+5Y=6@ZuI4J+GA_s zGar)>e;>7nrNCA$Qe%wz#i|`09)v%5=D0wW&pqBgRs6YI9|SNNS!<(^q0ekIH-6b0 zj~nEz&UtlCHX7ThEm~~&Yo9k3CkP14=*->xAFq;sW_dkZ728^6@}5( z=gGfxir!7;nlx(;k_z}bZ-l1Wt@onR%4Ii~o$ailFGPR4yH6YrlPW}e)A9`*aVW-9 zzcnJ!-5veA)tg-+OLD)eO`CyCne)RDC7=K0=ZDhj>BUAzxT(FxZ#pIiQY1#_uZO}( zyHgA1ikllpz=md29g1-OEQZty^tIF4!befJ)jqp0xZ?8K$a2ke6O;mZjK8%6|IIpBFLi^zs=@VbD zQma74JJc{NrRh3@ATSlnTMi?p>UFggAgni4LbrIqTX(ErtsE6~W~g2XONNyCNJ6oh zr9VuPrS4R&c@N}3B@cSgh|tdgo_vp1QwMRz7Bo8lVOKiOA}Nu9aLz9$VbrqOpz(_W%$n$uO^||EQ zM_o-X_3_bOA+lOc-|>p4nnOyO|&Fri-MMG&A|W&oRuJ!GhZ znc6oTnSS?`F>M<;y;Qx-w#SLKn=&Cm)9cpbYWF=>Vs$%B^=~gdefjusw2ukp{V%_@ z>a&`YzUzYz*PpqAEKN$8Xh#SRb@?1p7wsF7q{`{f?$?Ev*ICw^R?P0T>w_w5Zhk^K zWi&VodKcy3r7DK7XA~^fZm_SoRYq-&(?^X(q>QBOYPyg!wnqX_ez`?5*2!q>2lCMy zeRalEugA^Ipdg}F2yr(xD=b&rR46oaip>}mI@Sg8@lQ?%x}Kd5k&KFkol<06Wi*3k zh1R^8c;oPp&JECz*Vda;r817ic$J@OZ8unVfPovUu8nyut~ZKO#~pNcZXsEF_^!@| z40S?5m#Fji_lKqr=Sp^{EAxw$Ui0R6UZYoYngK6rlSXCeSvD(nPhgF{O;@2IBgB7< z^j9yv8-|@iuH6q7JySoHNCd4pKkkXV&H*GmQ=%>uPbw3~z~udWX9}huerC1tZk`QuTQO#+678|MIjN@-GL%n2&u7NMYaWO_4pdz3jVp% zg{>ykDwJblty?de8*cZ!mQH-E_k|cAnJk`X)r7C#;BDMsZS2tSl}+JC8gq%KPR=Z2xtrR3Tqd?}c0aE_Wp*msx>5^v?n)g$x^5EIZ7FRYkbddtDHo~X zAYq+f<+0O!)GOp>Rcics@Gg9ClNX8nrVVatVzA|8WUE{FDe<5POl@%QM$w(hlg&zF6YC#(O}J3Z)_ExHeYE`EDMh8X7>U zQf(dHyg6;MFQsB{Wx3Zt0TsVvV;N{BWx>dzI`ZxDE{%amarb20JO+%WOrc(bQR7jJ z%W^OwYsBvXM1P=4l_r3#LVUY*HD+0sGhFSn`@wY& zGoHpx)QOmhu*PK67r!C(Pe8NE@F{+`^q^hUF zk2W0XAy-9|7s+@0grcH6Q9sd?B@1RX;x>k!ZtrJ`xU2>{_ydCy^Q2rEveA^(;KMQ` zT!EA!2HIszoT~_PkR>Q8+%+RJ7Hjw}#r|UU=Q@`?f$sxuS7>J*?DI zIb5=uVa=}7taT!Q0+kA-r0Y}c_XMpn(PdHtoHS zqH%S>n7Sw%sVLCZV0~{Q1;i85I1@LMHVhmdZEHc^-uGg;H?nI%ulGsIKkJDg)yOo} z1J%HUbmh%^p?3!ZdbZ0%FL~hdF$;t4Hqzt&5L@sCR zxxBKnvb;vwo9DwhH{UhDT$xVphr||e?Y_tJX+88{cW2ke1S!wR>i&`nej7qlGJ77U zT|Xyy0a(am7TGts<-z*vvlPC73#eAfbyb6om1m4iR$zyso1(PV#btTbu8YDx!DWi~ z0(1h%CHWp2Mq(`J2ZSpn4p6NxmaY-{z4Q%ku8;%6i*~B$q_>CSA7rhY0|RS%fd&-G zg5mC2TH}7P3x(redZtA;=7wkYYq_4Nn-;@vHThf}ACCRh#K1qC3fqfi@5hf}G%`;An30;`%;Ewsse>CLpFqj3YiPXh2^M^3G8;UH0}ps5CjHN_T%(b9C+ znBBWU!uW9djju?&l9y}y)VI5JlGt%%R<~<*-?6ta`YDz42Nwa^1X0S2Php*#Z0!4D zjcXG=+Oc+{P^$D%jMz=tjfv=!Lnl?_9G%g~Pz2 zWV~h&T~n?i&>_3xsZfo`It$YLK1r85h{IuPwk&w&q~KmVEo>&|TiVr{QG@pdo+tgq z1V(p^@`ioHf~WThhJ?I%$P5j#2D5!qNNnA!3ROC%k8|n4Xrar~>guu8F*IuD#iuyx zb6I)xmyjGlAFCaH$ovyoj;5+Zaep1z$VT>YOyQ5E{F7PnVsMICVjXP!zjW0b z-$~TlvtE2->LEZSR&4cp+Ffn;muQGAu*SI1V6mD{0MTlRCsO7uH%~XkejCJr9)ONN z_}my8emjRx4>fztk0UFf3(csp~GY zyUTp_O#9O4dk9lG`%`W{DX?zyn6Vhv0ZP_(T&20Lu%WhAbcoFg=9o`vXotp2T&X+3r+0?ab5i-a(2Zhl8>I}P zg)a4(F!F%`D91A7^38?L-M=ej3uxJIqx4@ZU@@^*;6F~T-i2)!05K-TV3lSdq&rmy z7yn{2gjBX zv8BHCph#$9KSFv@7EKwZ16wx#-zlz2Mt4G^9}>q^{IsY&+p)BtbWBEPKs|D)GDS2c z@DBg*cL2vpE$ss)02HfY1XbXU{j9ZGFwrMc<3!gZ^ruluqQwO9s+Nt%kw~F^)qGQU zNzyi*VX3WN?O_L>x)=Vrd4a=s_Palo*AFrOpF2Q*1@lUAbpXGxy26RB-B_L3JcB$H z{#?55iOSW=yd+cD7NG)Ad}T-$NSCtXv-H?>Uraa(`!X>8c?;&Y(N9o_M_A_ zQA`iA<8e0I=63jZkm1nslKyk{ic#o0;fw_!Dlrk?_g!kO{J{$VWS?kyp#P2d3=#fx zZe%HFVVUVqXQQhK+}<8e-vhi>QXuSMkwv?mBmi?esO_d^y5&e;uW#~~k^Vi9ECi9Y zo#8_Q0t5Q3fI|kBS(IHd9ksA^cAY>LrB1t^Yu_jTIW&1pz?rwR<6z=QP9xI6ZHyER zfl))^%cfWwU*h4}WT_qB+|<5bCp1{HLqKLR65qD#1)7?gSVGp&%KB&1xd5>*v|+{Qxm6*VO!NfQ5ZhxbTOVnMLDCbanf72-cjtIGNW^IG8_kqMe3|$_tTEPOMg`#>jO%|ZP79wKa#itFiX5?!2b<`kHUML zbz>R8f8bH&aMsRdFb})GkzGjS7Ik7vL zC?SmBi$LMTip=;;0!67dA9Lkg)(0B#)|#!UF(1^hZ=Gc-{_97A)ZeAHJyi45HloDe zb!Vinh*=A=S_DJ->@96SsqUaJ?DuXttu)$)1Ld}T(?0bn?XwA9H~fsJL%db@!5cUN zJrS5NgyL-*)+g!W9=hL8gU<4=RNQ=ciwbjRS3$@0_-hA@4ox+0&{>yiY=tTSF;m+_ zy!)rF)O!mvyM@!bn%*K0m@g!Je0%_<;poVG_wb;Y$?4QLhKkQ1jozjHkNECejkbF& zqiash=(uxfo2Xskv`r=@hSuo9hibiQ?Yc~Z%5cg z55%I2fv&iwNG36D`e%9YuY?7vXbgraEp_Ql>2$tsgm1OLHRu_#}i3icpH!nY#21+ zp9USIKX7-z;cc*}`NVu((gzI`DgV=QVV2Vum+QyhKqM@3dit2-6*ePNtHpXdfJ_Sd zi>2!k%WAzS?yxhG2hb`CNAoosQ|-QCBEee_`g#qDR~8lO4MQzv1qAZew-}RfLW0+T#o$;K8uNp z#)V4>9$+}(YA%qXta^j|@~u_`hw%QJp1@LP>Brx}PsGY#C5`sq565MFzzAg0DKXbk z3bUR2Q^kb~-%jN3z&tTE%NaK4yGSt|tWJxAb7pC>m%&6v&lv&J{>)Z42^77gPbc@=0i}IRN z7Q24m@FGO*{@TFRk*xysufJ6Vq>knQ%4~lc z{6!{%J@%IUHzHE#D~xeG-|TjklP{AF#GjJC?@t!aV;nCxQi)hxgHc)twIp5 zI5;H*+ifo|dDFGFHiOM33#V8zk+Kp|mrWT&qaploeH5Rp_T3msHjAvWpw40@YSzGE zzS_*5HuxiM+*Q!~Pedo3K_9;CHlDP^+p$CN6`_j@8SF+l;bCytO>xoNsrlQ25U|mV zV`RvF2|@FG@q#wk{|Cw=e$B-6MRqX%yX8|0`Oq+^cWp*&X{m`1{-M(W?stM9QS{*Y zma49&2rj4Nst_VFa+4_+%nUusz& z5osW|X9o4>o2_5qFQMR~PREr>%s-Tt`s#oJzvnl+li&haJgl~|KRf-18k4FBAWjD= z6kVKLT)moN2A&jbR}_l*F$Dz$hiPEJXk0E)U?Rr%Qpea+Xg5?Zo_7MT-E1g@;XH+g z>WYMwRRyoeN=qxlFwtqO>z@t}!I`i!@Pt*z}(ZrAhg zPxc;Cv`ARA37WE`U48}jXI_`PvD{^N3Gkmkf2NeHkm6|Uq*C?#LEH)utajA0EM8GK z97+z5E%vlSw~@eZIptY}M5Y`rPR_WpGMlEWV&G_n50&Wj31mF&Y(q~B^yCv{VdpSf z!U4Y$!5=p0*ZHPdci2oJdsRs;f6*+=~7DbIG za68PuS3WN2)6D{+zY@f|=_N7|)i5k#fq{dw0EJLXN8#~kXouAqGsJ?$O@~ql!RW2e z_ZDePrXOi^TeLK|z3%r4lxobCtL_afibG& z(Gud{Rfv!xB?2h`i06ec2O$cUx3~9P@9o*f7K16vPEKhR{J_R=28SY}_W`&rghmn> zpH~`KuwtAbHoU!g_FDs8PD=W9&ZWIMfA$r%JEt4qZBv5gugoXfpWqNa4TBGs^;l~~ z^q!Q24%Y#JO|x=g8gZO=0RVqdU+5Kf65nmA&r%)Xn)u?a2e82=r(pZ<2#Zp4a8d@n zmXBLBn-V}qXfTA3;oa*#+eQ3laL9$-_TRqbdIvBGB>B9wq-CKZL%{4b35)69s7iM( zRLIM%A?!_Lrp@@abANR(-mQUmPtX}L)?f7uIW0}AihW`BHWpd3iRRR)JE9(Do%ItQuu4#To|+<(j^;m;Z*8d_Dt zXQ+#)#Hy5so_M!P6he8opFSq+rQIJloO z_oQ@z9L#)+o?-qdF$Cey&399*&S?Mssk*MK)$|)9wzXfQ8=jgux|jOdFiPxD4_dgd z`!tZyP~T+uw?^?1K~PI4$v<0H`}6QhmBufZbFuVRSx%RH0zhpki|%Ik8_Dp#kxVWr z)Mzj$v4%L@?bR9aF4A?lU+?0lR)jG%R}P8FaF9a5)O(?meG!E2M~w? zIsj=Uk`?XpqaMvh$2e0-1o*+ffWx=-`Bv*>xp_-hTpZrcL#8>LOV_l{`G*A?Eik1M z&Y0)6aJugi{(RN2l!rZ#K4{fV`E`Z*O(nI=>u+XL<(Ksjnk&>3&Z4h4Jmn%vFGmNW zaOHoSwl8>7%r+R+Yj9*M~^yD(62@7Cy6&JD2#Ee zJ^yM?K=b9pBr6>Bc<}zL+qY2aJw{tZuC4c*ma*(s^UUjqfQflB?WVVOepI(Y3S#1b zinhy;0Iyf!I^3HtHPnB>LQjkF+a#t=5N^OwSPkBKhI3=l8s%i z{-B2qbq!o}6_-bi-Q3@n1bWz5DDN8N7;buGXs`Dc$c{=ZH@j&b;0fjjL34@1G~rZj zBQuF&+os6EFurILWrBjN|6M;0vwgrJDfe9MIV76um70)&>0y-+N}#KNwf#fVAr+*; z78fRtxDBC3=XgFf>JAsxZ=2GbuUyey2KYrKH`KGPyh>{GyQ{ph?1+I%X~AQj$?4-H zy0vJ7c@d;z+Z@7R%TX|*+zE$l9oVW_T$Wq*gtq6#Y6TdbDbp;JzooJchb-^Pe z(|+1BdHxpCq}3Gm2@QSVE%$4I$o~!vh}HO(GhqA+Js88M*i+PrN)6?z45|cY3@@T} z$9bw7@a_^Hps6)mlDN9NTdco_87_IyY8!|r`-#N}`Jn=0bbVrom2(l|uy2R~v{s1C zUN9LEXeZF$lzg>K9r2V)N>xrL=d+bZ;mWPAqsfXy+Vo1yq%5EHT*FqO(g%D>jZ$*V z;#Dh!zW2>9A;&r`5_eLd+(jMN%i+D_M9nVRgPXS}^2;4rN>|+)N^OnbnmGG$Z96i7 zW?cVurJ~Gw^95IbbvuLUm$8ZEp<-BgB(d1e4ntx5m%)q!1_NHLuzL(P;5`0%(LUX; zkyv&G$A3HF$LPQu3Uo=+vF>8QlmxZ`Rhn@eSuN{EE8yVxCg#^H;tX|C!rN-8f9!2z zFi~E6NiR{M#Ygyg;blth?8~jxEQ4!MXH7e`FbT5W9_?n6WpvQH(rIwsV&#Br6P!qr z6pAizN~|M(KqSF(R@DdZ=1y_dUZK!n^D2f9_G2L?(h45Ceo*;&7q%In6a-v({y59P z-{saH;%pblret_3B%)o#i|mFhRw>uqzKT~WHH9;?&-Rue#@=RL%H@>eo)?j__Xo6w zcKcw#n$XWj z{TW;fo;7sAL6A?4Wy&jN{aGREC0Yn~MHg7Sm@6ElA;Y-bLZIk0i=+MB>(035Z?rX5 z&k_7h1G*YlEbA&7m8Pcj0iq#}A4(TvlGBxIvdUBm9%-=elPl|`(>PPBKV63gV7SZT z>OA)f78A@~F5Kig9-X7{$*a9Py%b$oq24b}kDGXqrDOnlHL7b(Vd`D6uNAKtE%-&o zQ5u?4s24XKYPSgepq-=a)}_(KNHc6Y;c?%u2dhZk(n-EDgO?_%610u^( zlMHH*j`PdSQbWI|^k)q+s~~4)n8i&-$PTvhPGAt7)WD^F#Rs;h8d2PX=3}f-8#TYE z+wVS=2Fi(%1$AEENp4D4RMJNiFTuCJ+#nki+l_DOUJ!<3#BjE<_ zdQ)||(@92$V#3m#HrywFU0!)m2KL}IEdLm ziG-@ROjVF{-t8hh-~MQ&*nUM1Vlnr_=^0E__tmQQy?LhGSAOWPDDg1lCOu!Ru0^0< zzg?w=TG-^x0x^uWsJqy?{45cN{;HS!M))(g<;vin_wr&$Tqqps;sh!1F82>ZGwRk^ zrgl6}dg5>vCa1=RAPq2`GtDxk@(P#)5U7*|O$M_UuvHR9i_MPIs^>U))SPRdrp|l& z`GQ9&3DG+Y9$q|6YI>U7&cZJLjH_od8J*ydTY2Hnv0K!f?F{HL+U!cJpFcns-s$7I z{?f;kNT80bu`1zq#VD%9{{l)ef9OY_t1<}Q^lmBdiuxk9{XQfAx-;^jfUU9EiCevd zaB$%0APGhg1=GDEIb`79k)C~nx)Gzum?0#-{4r4XNF*>?jo(x3iSqsiyqwq7O_V5w4GIUWv&Jbjj6Bn%&U-3MRBLI!7eFgYON zAY{!3)zAk>!WR}+s#V7L6IRxl4kfC8v|21PaL73pB9=#sBhilvqy#AnlGeT)J~yuG zS!6#CiL|Kmw9a3F`|lu6o}n4r;yqZAo*{0Yz zmN|8zkUa&3f$#GbfT2*sz}4K!i`L*%+c;Vg(XqI7_pxVbMxC%zuFTx8fAYahQ<8sp z*E*p|;MJm6U())w&ETxwtd>M{vu(S1vv*3U#Ew}pSE_JCfDu;+ArV$idzNWun3T*J zPq~-=*<%z-0eL}|@K25w`;U?97@-A@@GX@NV&OTRGH;5+l~PjIf$ii7NL5f3kxa}a z+RepdEE6TEof$F5I8JAzXL);pbmq=h=^Dt(@<4$2%e@^5c4r!g8F!5%KkEIm8yF$zL25 zXb3TH5bDGE`NLA>mn=Ci0RjvOR9K3D z>)daaJs@5=kj7$RwEOqhn0YW+yblwM3-ZQ$nrRZ>5CjaSxV4hhO2O`UVQ>koVD8o^ zKKTK^rK+|3FbWBBF?&M62UUv@vvL$+oS~j2P3#U$kdh+pHZd;_dDXpil4a28-C7@Q z<)-(=mY+U@%gr^eL?#E=GOb1hB2Dc25piDNJQKF_IS!f*loZg9RvfX+{x12rF$gnX z7BS)XpKe(9MXcYpYKimsQ>qN4RbSMb=z6Iut)@!V=t!o3+jARam#N_xKSr^(7 zcSYk6ONo@Kj25Ls@K8Ccudui{dq7I3t0dO9hnr8dE8Qllww&Ib6l-P2GQaRR{lkr`q}imF{VP=d~}dP;%Ww(*3d zC$eqyLx^(c!ywzNx4*kqbcS(kexm*i*;}S^yWEuz#pi1=8pTU`-`>n~D{BGR0Erw9 z+v1~HJS4yd-NHm474XkuMIZ@XZuICn0ngFAlk!y}R*B>+6mwx+Au@jDwxkf%=DfC& zHvn@@M#v|^)M%fHcV~*Udl^-z8>`&Wg2xRGYCe>w=R?iN;Q$j$<**fTe zC}TS8oPaUIg^!_?+syOdxjJkm2KC~1*xl52`u+-W7xBX#6{8N1;+6fx_BCn6CGZPl)EY> zVUJp-f3-|L-+q9ctvC_0ZhCfdG8GaQk*YATIPo->EI?vc^~;pp9+XK4N}nAdmT3@C z9>-?rparL?8*|BJdr+}N^U4%Smj?Ght9-jPdU;JS_PG5Pns}`t7fTc1Dd6z z;+{mZlG)DjTYg{7S!e|cDWNua;kO4cp1%5cBGG~@7oZbBoO8Yz&oEd#ub#!7lI z(WzXCcuBx+o05l5Eo;6`dpzw#W&~W8yS;+!T!lQzbATwPII|y2q8TqzrlL^D4cQ#Y z)GSf~Qv%`PBrfOUKNp1d9UT`yv{PA-l9%ob!tSsweSuPPfK4_c(5}X6vuwFEotU$3 zyV?c?1ZIB~eJplulGX8ka_TMw!jjNz_O`{}+Sg6kTu${!R{hL=2?lJSO2iH(_9T;cdPUi23F6Ip)>4O>L3zGTTl3kbETbWN_nxn#p?`nJsg z9+&Uwy94nMJHYx^$XNp7su&+Vj#nQp04O2}_|manX`unQ=K}lUAQF>-wS+FAq~s$oRg$PwOA~;XG5|PTCG!Up z1<~fc9?iIGK8KsD4(FQ^B-Mg#OEnhB`|f~YbUvf&jUcQ8Uq-IsqWr~!=(RjUgExm( z)@;sa{7p=Cmoh5)XK2fHu#wEf_?!RL>SnYh?`He-Ty-77Zqt`)jqB^g)97d5ePyGf zrfCdF1G0J|zXXSigy5TCFyJ=7FWHfTISfs6@l^bKnOEFZaZBkNA}sBF$9o#!ei!kH z$>{oUj&gYmaPSBLFR5H3f4R|l)Qy6Wy4Gw$)apPjW?th0T(_+7fQ*aSxEEZG0EOY* z!)!4v4YUNO17!JPKuc5&jRaEaim5Gx=7Nv@fg8Z+gn=%yE&$5sO1CTXPrM%W_WCCa zb!D8bfH|d7=b)Bgu!y2ptX!PmW4GCdn&=3pk}J**gNg}F-Ge0M9Pt+RqQ?tEjQu;7 z+apKV#j@v(w$m}7+fFI)5n6B!0P5ux?SCW9BgE&KH&UVkQNp7^dY{(DV_7&>Jcj*O zlfl=ue~VqTQ*G!{jniP{rmgiyg3FZfvR*-^=){AsD5y|Y^10y1GtM0gE+UOZ{6iQ5 zgZ&#swNyp4|QVER>sS0(%fDe7` zjQ}M4`vLKO3rHlH^YsL=t`X2EROBj3$O?0GNg{)&t4{ zBjv~M8-R*dM4@IRI2+i#2mI%(J0=Ou?0pj1*Wjr^?$V*M(0{||l+u@2^ZLuvOk@Q^ zS16WPQFRGncX47YOelIG^%lc42bX=C&(_u2r5ShoC_H~jj#GZNHZvc#bu|r>K94>v zhWik6{qYG5!S{6*FcJ+j3j(b~o8TMJ;5K{zeCDDqH3M-&)G?aAJ`@6*C43|BmYhlq z)TlRSPd6*cP6yKzvEPtfu#whvJHpU93{n}5jerR_O*mbqs~Z~fXQIMT%vj)Vm~92v z*F%Z`A^vksz@Oj%4sy<*ym@*aGbHhaT)z4&6vo(ZI|GSS(@gZhP_5ELKb+N&6twKN z(>{eDu>HUYDUPz`*EXUQ^M(CBQ>s11$?ij_wc<49Fx4NmYWF#9G*#+#S`k3? zBIJ%F1PSJE)HmrUYM#4!65#k~BCsE@9!`${fn*fhDL+1qq}1{*0Us}IawBszt#@cp zA)j5WERc-UvIYL=6Z-0rz8OP0fxU2k1H)xChBIV2DkkP0VjAjKpL{I| z=sXGZi0!CE1nb5`rDFh|kl;jo?+Z(c7@!JYPL3iF+w^@njb*&W)4c{+4sfs)&sZux zVmYW8kV+-JPnYcLdXy*H@YzViGO*h>;uSyiht|hRDk0ySC{UGZ`UtE_C2>^xaO*$* zDMj7@8A_KBm~J1KQN;jPp%r-#HYIRSq#XVvJ|fr|63M z0!i-g2Kgv91e;Yc{Pbu(E->L9ArBgTT|0awC(sgoUEE9Z)%8Fa3I)*$$$*#Wtt-%j zC84n%bo%Wil5P20Qu@3g+H}Z9Po$#3#%t&aaL5&pw;F~MC=@ACQ3Z2qwYqrSVWCOZ z?FGpLQ8)~`;I~wwEZ9v)aQagiexq@qRw0>6 zV~sI>1m=@UECb`m@_{aPTN_^oCtqXI?SP@92E8w^4E*ib>r}i2PSSGP(f^Hy6Leb^ zytErJybI!bySx01MO@xF-Jxhdo;V+O1o{jgch^c5Bww|l#DH$C4hf>|+wavqc4&CT z4>V902U$sdUhbbbY_Z^HtGeEI)u-I^9FN@G?2i3bPO{Y#lO7-v2_r|f1C~ZVIK-s# zmTH<1ghK5OtLV7}FQAH>kA&f{h3$ou-$wl5_Igy)>rW*RfF_X^RT}-#iieeiqG#A0 zW~?b51{YGw_k8=?XfR18G#_}kT=Y!rzGFSDS%fvb9G)&W(7!``2Yz?- z??=OypUO%8yKXZlv@bZE#Nc;cx$9n>E@`?dnx})7>g*m#pSfC1pU%-lIlg=9W;ROm z?y+wVk{?tF+M@lCJ0dc5ADTP5S$3DJG#M@`p#=JqP`d2;3G<0N8p`zv^Bkjb`-Kwj>_5QxDR*^!V0r%2mJK?lw)O6LTR{P^sIQgM zw@`QBFtIse?Ew@Hn{D{&yf2mOZ-A=CBjZhAgel$M-@(7o?#Y$VWXbr6|8_L%mmWPkYs6lM34Bgbtah`GF(BN2FtV9ahSNKgwRP9J^G2}uO zG~_tuS3U*4IrDV--kE>Y5T=8%8CN(@E7Kwt93W_NsQwHR;RyOGRz)w|b-bR>=!n}~IlZ`JZ| z(g+|@+lkb7Rhqp(=Ss!{w^zlS`mVA+#`&SIo=#Na87oY@guTLeZZ{BcaS#i~D}!pR ze}cbbUGS%4?VvOG@;|)~ap~b+hlj$YQ+%2Jv5BMIYsR3M9b!!X{Fh}k`=DLsjSc%^ z#@H11{){h(u6;n0NZzqqp2ylDB~oslg@Liu=yKkakecF*SOCdZUN^M-GiqoGiiNfn zQ^{4E+FN4P*>^pM97s<)U@A7%T(N(immZ_55+WSg!RvGjm^a&9CN6^cRg&i#|Gq7Q4W4>={Yt467xN$+A^rg^k%9{rPRp zXK9xvBX*=+VACu5}G-IQqM$FohE$B1s`#p@gm zYnrwy^E>xpn8JV`z9w1oxKeu?{#J<_J{g^{+99&_`{s=(ego9d6Vs$O6{n5-;`4|ohvQUOUd9fr9jZc8(z7W`m04Sy-j zb7N5&Xc-k7pC~XVtEO>&Pa7iov1E_O3db!Rnuus5;rWn-O`zRk)Du3diZ|`B#nhS? zMdE=l-|X7xs@5Ew)t@XSb;woH)fX=0Mj4LB6Y-hGJzD1Q^&pl;lE_au!Rcgt#g$e& z={x2l3STW+QjS=2h5Je0ETuuF`HB8*#<&OF#}R}HYR$1v11ipsYORKitX6fHKC)k{QY~trZX3_MsAB`1^P_yMyQv6qzeA{jF0D z%6&|=+lU89pK?Q7g{3V1D6(UE)4=%*W3$-?e3f(9db!m!PoS0rBW`Y5i5f zsBhe&ju1|zikaQlI`q z=+4dAE-ak*Ql3jFD|N5Tmb9stj^U zB~mv*nZ{3FHay9k#Z=PLdf!OalkUr6L_^O-k8V{3tAmQDY=(ec-GSn_)(&kG7n8Gd z5FltM5F5X?(RaJDwoB!fxH@Rhd>vDWQn@+iAyDUR_pUP2!0Ne1t=&VNEC<~qx|Y6* zbTG`&4~*7!IvjKfbQmWceg)a^NTd)}k9y%TdlD4m+82No@Bk(`-G^|e-%|1}=W z!pAsPZYxt|OH~xz#OwQe_F{%Vf8ZLpADd43vH<3%-^alNtzQQ_+Yfb|z#hy7Q+L&bs1d){9667Bf#Dh;_DZW(Y@p57fHCeV37W>TbHpHKF`?;SOLBlJj$o_ zaAw$*W$*ea-0jF!_Vq_$5}Z$vs3v#&$Wf?fCN)&2Le4{e1j3bvIM-z4Sq^H&)vYYR z1$z|SA1K(nEtQS9KHi=U*e1**l-Rw=VjNC;T{1FAK`-}>PaIm(dr+H9yJii}5WP&- z``35kPbn)@NJHdBcEaY;ht6RV`$$Y04k}*-`a;CIrf9ckAZ$rwA?wCx1l9KTENEXQ zy(p;_+c#)>b%TjFH=9@qNU8uQ$h-S@Jo?{j=^Ch1v!e+mh>^cxVDOeaYIkG&-LmO zf@=6I+ZGN=5{7srHqS>wSWqH}VI+d}yu-mxA>eC?(##b!y8R4IVpEJA+r;+#quMWYXA|-)5b}W5fP4?kmP3JXATbkgV9Sf4i z2vG*~^(kRY&|u4S9JW77cgTq;#yNMv=C84L zfyA;zCzDQ&S1u@t$7&hLrLBl_uMDs@!;UJ=_Zrb^g5M9HU=w8-Z)! zdwo}6>pmr3y@<_|Q9J~DUUG9ML^UZ9TWxmPQSRy>8Flle= zW{yx8=r8NsnLH3;06yCbV{GHEm~hniQMFGXShbJoxbhJ8$#GNM%Y$J)(q@#U&YPZ^ z@zirIyENFv8C<_f`XX4($Bfrz_ib$LZujF~ATJ&a?^su-Yw1fF)bPYZ>V7^`H~Cq- zoQ<*f;3_j3uyI=Fm6R%ZI|d5SB9RaIu4Ws4P_Gq!F+be;RF#_*meOHL?Rniq>vfSj zYX>)zP1Y&d<*bV)+`cb>7SP>;HxeeR*UU?n?g8_96N6PfWu04+ezIKYw&##6`}DO( z-oO>N@TD2-J8NucZ+ZcyMLy)o*Z`y2%WFJm2p71-)NLHO;-po(_8UxGx#W4o=h^)v z)<$ibIQluHnbQESKm#y-jCc%JyNI>LtPlMT%ofSf&U5RrPrjP)wTF9e8O4W>`wWAX zo%9FY&8}-8NsmImoM?!pv;_rHAt04qr}pAuqA}rk?sh7eKm9)Ga$})kbi09W`x)U$ zhuVgL7;FcH#DSVH%xcM@b1h(jcgTviM49)uUkl1wZ>Be^fKwa2Hx#VL^wGt%<_S^bNL*PNz?i$LESgu3v$k=SSwBk@BG#32O!Ooc4Oa@9#HzQ>OS zcj7Ev)@bf~G!vn)J8)5t`;}O&j5^H1t~u{{gtueF_NU(2eszoHWTb>dq8~mc(Co|$ zIQ{fIf?b>tG}h@xu9gQbL@Ax^s6-Q@TwLQd%`6%z!G=1el15j84r#`No$6^D*^F5# ztdx*DJoM&*lF3+kI%F9e8;HHt(4;*DPu3HgJw~sND!DG5KGVxZ?LUNeH^Hjv9B9V7W7&H+^U6X{ zLK;Xh98Szbp1U3I_!li&w>Vf`bT6d1j9~_SNf2xh2EP5C9Ic1 z)c!{LwVT+)H3H!c!1)peHT5=xJ&p6DgsyZg{ zWT=BW%S!5N6!2{Xjpl^SPRjnkJFs~JLCX2D?p2N5d5>glvy&~}^KAb6!#f34<>jfl zpZxajoADIQC#_D`r+8XR(`iqqv~!xf9Zclqa1M4=trM6x?DTfy8G^PaYoBj#Hp5B2 z{+Intm6dZ84YZYT(s*rKv7pI;lLX@&Id!+3XP&Fu_NE=Aoei`hIwgNu4oLmv^& z8xHG+GIGhjhb~?XTKs#~{sIO;MUE#~i;Tyt7iR+2IAjv_yJ!x=4VgRJarA^hRI;#gom6%Qu}j~B%dLLYK#)Niayc9Z{;x>eBvb~< zdItN^-f8(??A5=2*k6poi2~$j62nw(w*_aMjoxr8jq4dW_55&bHp2I_0Hll=OQcp| zpin4?uJzNe)e1o+A*ErIPX45W1Me$A1-Gi|ea16ZiKs6sK1@3M0Y${YZe} zHt&yc6afHsP}R=Iug{Hy-Yr;P^|Dkk3PsiixT8t}rAEEcNa>hu?0 z%0@`UVBF|HCT`?Hwb>w8Ck6l_d!{~(?%dXWF|t5r_y5p=8bDFCTdvnjY&4P_NvqLd zZGp|>uD(`*;DttL44`@WX6u-YIZy3+xBIW6@gIMp(Wo*dzbM=cUa(_CeEaqtOwB1E zfHGe1BA7wElTdbzZi|W+hk1tzRkGSdc2J8O=S{^3Xx^o|Kl&09{%(H{)ISe^frK%p zw*wXru`feE2)?7BRmZCUA6+08tNDIF8k^glJ)6^6mTk6Q{A&n?iZeje!K!zl5{qaT z*BFlze(rdU6Nma5#8DH0hFj4AfNqp!Oac>l0dW9?M^na^N@VI^@H>QyJbr`8An*zi z*PKhf(onKGw1q&w>y($@m)=N_;(?V~~(0a_3+svKh2f7bo+ zqQ@3MyytTAGKoOfGX#o5rB+kKo5f|l<{5&;Ds>t9vPRbsNJqBAc^<#)whg@Npetf; zH}46-WB_FjB8jdQ#O{0b!G2bW1G8D6SElMq^6VQc6fkmY0%guyUy>ZjO+}7uk<+DvJh+lLRv{R+;96HPf}0hN*94PRf^ zEfS*6bl~Yvo;r7as1#tKLqVXYT}Bib0p(i^tldKgvt@r4?#COwYA{y3`C6?lrg(PW z$qRyOBB#*fKnk{uzqTOkItuOm`$14?w<|}v<1CrC36)RH7b%&Gx?aOZ!QAE*Upz50}$}00MS1`3-8icB%vlndNWt zz_&Qfs(zS=i?YqO?i4DYlD0;pJ_n%<>AJ)({^t_>`&p9^>4)wPu+5>3fDE_cL`-bO zDE1^aYOKX7lT4paFbPN$I+p<%zAQ!0teql!7f31u>RNAtrOs;R9biX|{uOvv=M70p@6$7RF1pZ_43pceGTANF9uTT=jCDjUI`91|ifJQn%C~@HCV4l0=c6LSmmd!7{+w;;x*0+Iv&bZ>F?j9S|@%6*@7nwH7PhkG1d-=PKdB%BWVK@un94*L?4sJm0kqLcAr>Rb( z!!-vkq~0zFeIEiGTVG~E`zXWC_6ti3NKh`G1ti7%+TVY?vOyMX*Tcpk^2_A|IjpkuJ859jI5=3uDxOSvdQi2-- zT2d%^Y^_?$6qFY_Z-X+rzf`YxMTP0r1FcQr$Dqk)_LBBzxTRLOTySEIsE7RoQP+}y z>ld_-Q_1v}T{{9^3gpzX^uo$*9s8(@@pL2ks?`M*0^LhGm;ET8z>4J|e0P8l`$(kJ z7jnyCFi})Ik#(p%Uv{{~80CX>+q?Bv_OaSaw#1OfWX`Zky#?KnIRy3VAqQ3UDsH~d zP@)3dz@k-b?BX}vzPM&;^e+GKCn~M$R4O}+!&tmmbh3P;BL`D@Ir7Cnq-*H*DEU3i z@d0TNvg{Xg`LcQo^}mk!F*1zEphQ=W&qtVG5oL~%G#;%$5B3ov_4r)LBvN0vce3>4 zTF!eUBrODfL;`;?n{So_KF>Eem~C^fG&$M5YqbG{uPkt@8I=%+Wws3<^Ce69odX+g zK?@QO$^pCHr+w3FGvdk58Ms{=0$%R}oXOww^!WQ*c%=tD%PSW75iH^;6~r#9za48j zZO16d|CZ=N9Zq5`6|>C38dprS*6@dSerA)@B~QG}*q1t3Ef%u6H1y^07!5;E$OtY7 zZIBm)ywizYGOy#UKk$Rk{jMQQ-kBhrTE*dust3C$_nmG~QQG2sK;{Q&$zS80UreUArB&)j z&T{#gJDtxrvej%47r-T-6R#-L2nAB<-Vq+QCYJ1b+>D4%tT)GP_XT={w5NM%j=6wg_5XaA$)|@4!@SW|`0wG^_`{Eny}* zTvr<);dwSqLqEj=t1Y#l21Qw!kJ_iDc>-QSlSo=5pLOT(^1YjMD#fBN2W1G|$v+@8 zdkK640ws=+Y=GysG_hU1EUYfREY^1{*sdJ$axG)EOoxBFg$D?i91!k&-qAqGPk9Y; zmmqgbxEZLc5e{-vYs=s(ADHc{nw6woPL7d4wx5iy(-13#RTca+Isclsy0wr5?J0`SaPWs)(l7+q3j$)BG0i zrBk@`h|6_*D^zVeR8if+aTb=!TAFgZhW-fCez2kM?2<^vy8Bgb*-?h?PHx7E7gg{s z6Jb%CtAG3*Mwhm5k|EdsK0o}`X2Zxs0_L%-<9r*EN~lh%bAnExKz?8XF*dxx{$NJ3 zK9tCh1WKGZ!ONwthIi_+Kt7+aOjV#GJ(BgE7Ci9(rBQk8~nEFM%HKQez?Hg_*-6Za}Q{m6cL+? zZ4m24n#T$NnA1~uFy7{!Q>3aM=m5@8b{`>%0RcNqTgDTq+z3sMc zj1^#_@>89*KZYw5<*i3johEwmr#u0nfY6t34*EaDz>CdDP{cr!E=a~Qc@cjsqOw|b z$H%Ho`>3m@iXzf{ChM-!a`a>0dAbPUY_@}|!gA0?G?BQa%2-*EvNc`_r}3tg#GQ(T z^aG*k)lhzs_x4D$QMP+=SzU>VEJ=A#THez^T{VO8l-PXhzS-tE0r%JPf4c1d*g_m5 zJjHc$FnSNHX9Mny(J_a~_~v~2^5Iby1my7@7f6rB9U}U%p^p|C(wD)Bpf5Mgy`UnY z)0;ss2yOqTp8S4;vOCC2+1FXgosEFSDJRoGW_Q?^zZ~nucP0!(R!JpZH&>7b( zH|l>K7ExX~8`kVrw%^sM=`}Zi;s%LCbL!ds5n#Q4#$wyYky&tXboMsp^@npnnDZ%} zpTQ}SfdcH{piO7uAU09X_Vr72QRdCLfV8B(BnPp+lX5eI^DsHtEZbvijkb=8kwRaC zt($679)Uh-?z}AAb;=J|N~NoEjVo4kX)Tr)t~>5-DPeR-M&411uIQgAdHeUk2dtW9b-u8r82X~4>YhZW_I8~rt!Gjceq`v?6=*zAyLSdSw@-2 zlzZEG$MDmZuGQ(FrNYO2#XY&gPyJ(gF$&t~gWa$-CPtt|Q)z4|7h$y)O%_xS+;(cz zW-jfZ63LdU*P1y91+aLx`IMO`PC$s!g5v;67oI?qxOpbNPG~MXwdA3Suycb} z4PG5oRaqeQ1s02WnX0L??9@;cRWo(kRr!O&7Y62n+(-SV=BuDn*=W#oiP$MD}=7vW7Nbs=Qpu_9kTV2&tZs!SsL zR~lqioGtHGJ7CNhq?ih3DaP=vLygT0rz-Is?BI}POI3GF{0^Onr2F>jfKB}`-^Tx~ zI486axm^+g@km_W)os%*6Ir4RKq~|EKH;`cLevr|V`R$pQ1?yO$E&4Cyl7LFONOK( zb$YZdq+)uUj>cw^hweudGN__X3|W$fLMWD8c79vyHN#lBPvyk-6pCXgt;`MsT?|&P zSeOs@Vz*}PmV5UHwq_3c5>Hi<1}5`!kvDzLZB-dG43{Nn{iv9NF!m-=pevVlh zEMor_4E(dWcp=+)+bHN6-+KzM2ooOw0bHo8uy z3NG$^0{^$ymu9TTA7FiCMfEfCxM!0F$-<9aDqXH5|I1f$7xmQ5L5+KRN@+BjTFj9C z1hlk%Smr1(KD~jYsrew;LdRsfASMKdL-EIyfAb+ z4Ej^kO&zO?gLq-`_4Bp8gybT9@rt7aI_)Z%#)3i&6R_5`>tR60hSs3R>~n(kyRzFM zAhlRi_usw|@!Auxnz1{+1>qGrP`wB^?0xsgt#rVyuc%83CG{;D+(c2AiCMEQ2#Itz zoI$yBHFR*t>2iNTe?Ea+j^=POhB-F8>!RB{Iz}QyJ#=2-vdVLo ziact2Fv3XrDu?V&H|am;=kMnhX)r3^XN^JNi`WpgN=4RGp^`LMT))>6B9vSnZpsTz zKqJvdb(W`vWZiYS7{Pv&SX8{&$fYx>__124_bX=@Oj^xuUB&WslJi9R<+0e|F9;(o2Y|qzw-_)HX#(tg+&J8jxwG#vnXj%#Tik9TJCXymLy0&25d!Z zNPp`D{$pqO!HWbI(%zb&e-hhsDh@VhXSwMDAtOMC!&25Zt7|Nm3o4qgf6vQ-*K2b# zbNd8eyJLWX!Ayci(_2wrH!s&XM=V0b{YX)1g>`Aj&Oq9XM>8jv-Kg>J!Tiq$Zb2}x zFXLX8tbwZp6_`;;C4RMGhABA#L0ADcIUrvx+yPV&NvS|t?|fO%6^^F`Hgf_m2I1b) z604aIv9^mEM@-~u&YU+p$kXfY|F9BpEq6y!78^>GXti3Xt@%e27<044RJCANK!s1`9S0CK(iPO#wu8D#k+vhn0!k6 z_t)a#xLjy)0BRLY{$yjX1W_A8y^=*IF$0Lnu4D#-ptWjmv-ifa@ik50If=Z(w8?^J;nZ9E3Y~J6Hxvi)SQ=y zcTggUssR8NCLlS&flfdmAdwsisk)n}s~tdOgFv$jO=twJIyL~Hso2T)Y(gLIIt?Xo zZ{ETTP{TfhI#Gvn zZdqNIBYrwx(;HaW2(XF5`3MIjB30q*XK4Qh`6r5(I0@oQb+K_`0J4yG0hgbZBwmAQ zQgFEgA;n)Vnl9XFn}yRsRaiM#k{U%qWVPg?U_D-n`cY((WqCKv?=E2us)WKe3jQ;P}^&6`7p3*0G*VnrBj;9%kGtVBE*d!JU z6lMI3D&uiJU>hNWN2mS)>I+r-X_Pc~1cSi-mZG&B3EZIjn z5a1ll)l=ov0O8f=R-LMnyn6}OAt}l}$bRQf34XI)hL6MJ-qeHtKLjqdf&OVOdx|{8S-k3}yEW5F|MKAqv$_m}4!Lfd7?)x{=E)=GwZY zW~;kfH?%IL_z8+#~2+w7?1 zdAY%cVY27r)x^mq%EbL3yX~(z!CQwR)&7{yvR_rmjIBpp(qZ6Y-5qIjs%7r-hxW3k z6KJn^r$Hpv!x%{pss44nC%B~%8TPMf1fy{X5kd}P1K(bv=ev1~ERlwWJc^@@=<1eQ~j<#=3W~B;Eu9}YzE*q;l34f0v$xe zB7C29GkEE|D9;)X8V=d5rD(O9i4>@IVrcPNbS-KE3D9U&op=}A*d!0l6kDSoPHxRm zdaeI}8&13~ZVduMQW{h$3?(S)`Kf0HvV+xD3L7Fg09={e?Gjy=cddiTlS*&qE z0I)V*_oTAVDiKuSJY}T5vvHVZ9Vt1j+Vf>UUq_IX%^@ag? zN?SXJ_h(X2GiiCbgKaW8;JE)KJNsWPXB#7Eu@isbe}gvoVEIh-^a_Fk%|h zAV#wuMU)`DBOAK|X2X&1hv2ijp<=Dc{ZE5}euyb8kxpCYBEb#ZVT{y@bj8hJC( zEccd#yky>0zQRB>dw+Y-1bNs2tFxy_rm$4L>OnO9n#UQfK#?C$ts+tHWuOv}v}uDX9UGOsTTTYI6{&f<6Fh@2DuCpr`tE8|ZJ}PSop<<;BeKp!}Ym?#SU5GLw|_{wabkGyM&^;;m(aA^Wx75A%7Dxy zxiew4TP7X(-j@J00@d&~A3^FP$KHHpuC1W-4}7m_XSuZR1^~yJwnf|Y2GV3a&}@?f zVzNf{OQN3hKFi(JVPO~A&}2ltw$wt5daYOaBk1(ji$1SYh7au`xr02z909qHd_2_c zl50K1_i$~Q{yH5Qq$t<<3i=y9r=hdiD`wQ8p{t}tavWkHK3|0)TmB$w*F3Ivh1#Nt zXR~m#HAJ)cN~={zG%uSaBv(G4I(Brn#Q*Sk&aHuyspq57@7FypZ21KZcB7aSGE*Bu zVK`juN^wddKA*`ALf$8GxsV{W2`)khlJd8;l(ED{|D;xj)CYO6TANPoVUT90YPD~w<|8kqTt3N z1?*vP4uHdLz%NIP+^Vu#)jVck3R4zasyg~uY5Owqw;mftEclBb zL+zVgvlg9<((E^V+8Vwb%U(^M($>d@wRS#8LrBCor-<8A^;A2{rd)z&0+jQJ8+t8^ zCK*lgEOE)~E`l>wTsn2uTCrkuJF#>N&89FH<=OV9=9HhmPv&w|eSF`IqO=;%`PU-GK_3HJ!f|oj{xt68ke-gW)j((A& z`d3H06UFrpIW=zmj97liyrvXe+@TvNmy63%$mQ9d$i`BnN3~(q8L!tHRiTiruVu%x z+`y|X$(n)3R2h`9-4>y`4SlnhhFQ)@)f(NqzZ0fsHAQWz>`|hGInj$~LeTo2_Rv3E zi)rN=kwkn4jlm)eIkH0pNr^2s`=p!zZGjeje zNd6fO$M6>@l?|11jOI2*_vXRRtuAB)FE!C=@O(bw1lvB|)D6T4;ZA5{BWxIYiHOgd?x?gT-`SoIOB5zp>9e!QA?a9=PRi@uXRgW&wxO8B~ zJc)*BaCcTS>UxXYpDQlWRN|jqE|l5}g}cAF^_iYBX0b_xT2u<-ndl?`WV!9Thq3(7 zCl}ptjA#h(|Byl9BeQ6GyI?_~a%toheEUQ5tvRA_T}tK!>;4>A-SQRGFfrQs%R}X% z4ls#38Cqt}X(iIvc9K4SS0+%9(z}HYPjz>^x(XuRx$FukuFi(9YmQF}`ioS|BHra% zYKOnM2D^zs+zT4UO!1GA=cnt<_It81Hk=KyQCAHr&Cb;o4-XfU(uX)Z+$bk(kkgd! z0&FFKx8Qt=i*sNAL7p676G7paG77|)K?QTD3^pV4cClMZ*+IUeCAbPMJDYXw-;EQ9 zgb_%LH~f%Fwvd>reRE0>$0yA51Fbn;5_#M1?bvF)-aD(86+|0vQmZkULSe3U7+~`%` z;_tzna!RaS$kA6Cw@XM}>)%VW)z~lwByk5MsE+ycnV7w`qB7mU<}^QG<04P}A2$$y zHr#o6;p1!gR7*CtiH*iTnsp_2Q`<~jM%$vp2M3p2-)^3D(Ox$+Bw^Nd^sP8b{jN}M zPo1;4J=f5pqOoQ`Xi8w8#UV+jDV5P`&8g~k&{ynea9do}c5bDkqg!!f@j5z9iBL=6 zV89z5t|_4rW@OZ=sQM~46k=Ra69qDyc!4W4rtSxsG;TQl=Nx-9VnSVjm7Z{Vq2|&J z_MIMNnKVbr4vD<6|Gi}}ZPLy;5}Z(z+5I-K7AqW{bs5mVcHz)CY*g z9~`PpKE``NFuSF$t`;R%J*YLMnH^28OWM`yES5i1K~@7+)r`EzjXpo`lZn^z-M2`| z80)04Df-r#*^PGfPF}z+*xd=E&J zqV)Tz0~K6jAbJHMz4){Tk`Eb7W>^v`jl&<7ic3P0YWhWRqk94Z4<3Z`4|z0iipFLV z*0%G~0~?wkprHS4`g;}>lcWp2{X>LBXc)7T1{LGEcRX7o1|OUkNa7+U9jWo0VKS+g%b{Ga`0RwN)OwDta6%_W{2~qoZxxyrI?n>B@MROD{Mv{ zHwn9{ZA16f{p({Y8tw?fe7#53{PWhf%inbu>|RYy=Ww_n_v0-j(xx?kz0JKSg%(2> z7JtAg_hNMsz8MR4RI~rcps`9qdJ`)c((G02MM5)b9> zPq5i5naWNM$?fK&mj!8nZ&@F}GseQGYOb zAloGIB}Ed}SO$sZ&dDfKjSZ^w=ShjcgTIFid@;z^J`x)E=ElMJ&EewLD0Sb?8C9gH5wXEKHD7Jl&Yoyd`a(v5+o$0xEphWj@bMn~DJbA1WkXVYDnuNqm zxAazPHbqS;FoC@XtJr(V33e{F6&2>HOBtB3F0RFYP24aViO`l!+1D6rB6m+t&m;`^ zuHHoErteYf%=|UR&%ND=5#&v=3Hm2|{e=d0XIuB&Ym2Dm!CjDNnOaMIw#Auj;j()+ zf7VJny!AMYHIlj?dxs=oT`Xh#a}Pa>dokd#l21*;OkCnoXT36R=%?5Eor$<)@(Y4GoI0g8EHRzp!2;R&7eanR%rHDdi$uVxM^!U!0w<*I%J+ zf0Zc+fj099Sn;%YGzL6>zk$B6xB3I&I9of}@DTi${hPgKzt+Et>}L9UxqPz}R2_|a zZBqhd-QThuq{@X6eDWnFfBi z`=yh0$?1kPi%kdGO3$=ODYAQ_zCfT*kOqEDWZB%@9%X+ zyjBQx;fCn`LYPEJq+ zl*2TzUiNd@@AslpfMdhG(3l9{UVvaSv(7EY23XH|Z(P3_y^#z`;I>O(L-HWje&oIC zJry<@D#NALVlOq-UF(Y(H=#5h!TDtBn@A%sckull5+9z*(%2?RSwKSWmAQ@c!y<-c)N7f5oCt)Cz0aVv z79P>TStaqju6fLa%mf~z#@PW>qDgO3^$}$LNeyq(ZV731ni^5a&wS~Vy);u2d#1RT z(0ELy*#%j}6q(%o0&3O^sxRcm?0s+#{%&e-P$6i(&ILyW>T{mE-zs)jaDFq6>k|L* zCD2f~rYKrGH(WK>>1Yg>t;{m;w+gdX%^$Am1|ByNOQI`7HqmINY=<K@? zCJckBtSk~RS!D>>>U0u9_NPSLj~CSOkvLRi!!XG?Fh;GboU<+yt8WF65@CCY41aP= zL^+O;!T!2tc_mCRNk$jZ`#Sq6-8-G_TlcHli{0q>{LQ8Wtq@C0XsqW=UmDUzR;^c3 z$&L!Hk!pc*@~9o=D_@mB1)j^PvIVr{`7Wc?4dIJ{<-b=pwS)ZSZ9 zhgFw28D#2|JN=n(ayraR1;;~!j7+!vuNytm_}{FSe z$P{bKT(r5CRTCo>j{!I@e;O%F5Gq-T{(u5+U@KDC?C?Y9k&_6ezS;)X?~(W36&wS; zjnY{rNeBfL`-n&Q`zb6##}+Og@{UIGznODjru|GXR#~38GfWZpp*wj<~6=)p@h6I%$`KxZ#$*n6l26ERj^F_s1JZ`^It3F1`fha z+Ja7zl=|hbn&{G5^|wzQr!UuYBAUNYh{kc%c1g#7Ih0N# zlCImV!3rp>x84emY}AMDS854j|81VO!=kee^SMf(nl*c{E=IP+($0W{{pbB_+TICQ z$K$X$RK(#wLRBT3L8MZchCbS*M{b#jbpx4MeLqydP=pT7Sms>)^dRO?Wd;=b?FP=C z;vw7KYE#Bis}ny?uiOK1|JH!7+|1#x65{^zrfOIGxRk*4*7Zmd73;Qd8=$ z82?l3_Nl`5c*h~ZQu*tZYw-_PrP@4dERHX@OZ5g^mt7^H3|7k0MD^nyGj7h?1NBj+ zmwxRKg77h>og1Y`l(ezS8L=Sls6&s~p2ytntPWnSEVeL6g%Rz*7=Jq#C3-Dt z|CA1Mzhz_l@aq#pFacSTo&(AW2jra==^J&muG?kerH^Y;8qR$^mxIu{qymj++5B0q z{Yb1;zn>)M;1Whu4cNta;S)o#Slxxj1h7)~XU9*=asTPH1q9lt9p?tM-M5pwCra(B zU#%y4a$NX#8u~37t<+tOn)F|&yPuaB&CtDv^sVElOnI3zB!2WZ0&?kic2Q=?mblkb8N1okc=QzM#qN9sqdn`l_SrTw;+# zi5aQ;H0-513BI7{m=3zx3KpBaDP+v4g2bvDovNjVucC*mm9{0xJPleP1%NUz3Iy$~ zWki-r7#R2l!XX~5L_$f^FqXU7&XU$U?w3WoG}rSk`C@UyDK{@vRL-FWZuT(iCUA$A zJDs-%Gdw1-Kq@%^3{_o539c)AlOt&WK)LE{DZ4NnEMyCc5^&vd2 zIxAwdJ7pE>mIXrB_xMWpT{o5*x7P_eJz=QAl+DvphpV;D>x{jJho36HB`Nd`>BAIj zJgL)Vo3TcCgdN39`;iBol@hRI==aMh(_BurI0R4cr(h{HGb8S^vv{Ddafv_G7b)7`_5l&!pF-xWUK|+=1riCIaIfa&!M zYROGOXZ)^F1teQ9d$%+oRoAYM$eR{jZpDVTqrOTOefRV=ENe(6LYCcR(JLz1_C%2@ zMRw&*V|N&!)Ri5pY9X%w`v9<(2~`y#Az+!8V2dG7q{ZbBjo(>EB-p5%J9ww=GadJb zo;T1pv=y5;P%ax*X*7m={Dpd{PP^mn?^EkToz1OO8dE&=QXC2;+TD)mBe>#3PCcE z*=V!blcEDIR~=PZ(G-bJ#g^&ijt55cyPf-&xfJ5kFola4(s5~TYtQTXUHYoejV>rq z=rkeUp=j&(`gy;qtojoz{R;} zoH*^76bj(a!Te7afW&f~W{0|#nt|-tl>9%0)QK=;c8CCbpKh3Agq(}R4>ODrA+}I) zqp*B-*eGI!*g%o9^D(3&dp47g?ivpnC(I1HPzi4hUkYs<}d)A5e%AaXjrJP++!fy73EkgsaBA`}H$xG(xTTSoJDS z)qD6jjaAN@@aq!ehFOk8oFGo8Tu?~tCCDnHSVz9Md@!@Y^M~F&%%$C%OS{N#bGd7i zGKU)hn^}b`lWt4X{iL!%J}kzc79kY;A>{26LDbLR8vF2$bZBWK0$u3nZLPl2y2Db6$?ojy|PuQ6L_#M$K~X6J(n* zF)WpY_q9|9>(8PD*Rhzj#d-VF1`q_|jWgc4l@yVi)gI;``l_){lA$amvsJ16u;G*x z3ijQ4SYI6&den%?%f+Hq|LG^stZPJCUJg0kYY{h_8WbY3RB671&&)ccP(O;bVF~<} z*|L*g{RVGH6AX!ARhgaYrfIlZ2?_{fcWC+bi|#Zr42GTh%g}M=e|jO8R(gUcUl9XR z;WU!M*8B#kcw%2$!CuY{5((;lS)(ua$IA^RvB9AsXMK*3wwo-GjrM|Vxzo2u$s5QMh*UDC0QUV8WUonoOpAupf0doWdD=6Z0a2$64nkiU2vK7{O_ z6PTAOFO>+ZT=C8coXS_=!-UAhgsP<#e?N^6ZWa(S#@e-iA70rb?ySbsBYVO>qp4xZyy3rc1Q-Yw=z_I>jh;L7I|2st_nP zB+g-I%kg=*m`L;6V9_u>5`+A8P516HXMwZyJ_c8gh200O);Jt#8dQf?V>}D~u0cb6 zbvkWjkDPE~_gwDLCfBpMjj$EN6MLes2CVayrY)wZeCvhBUy-}kee!PHpF}J4f4^X> zcM0b05^x>lG|-C0bZh8)OXO~}bOXEf9YRk3h;EhA3l-Msi)~yH{`Bh8(g!6b>3)pu zv7Bu&CLd_S*j&+|i4PqVaMYh!M{sj-oy1zr15)}TK3tkgtp(W&A$nZH#f+F{^ zde-9b-*18&McRmJ0`Emkr>r3?Pv>WD%^-N>+A91Z%Y{o1)mmhllva?kS zN4K|g@DimRjKO(#H~i=s?3yvq=)0d}}5ikkuUPS6|dC zp1hU7wwR2*N4m6PrQSkLF=>RZoR&C9^c|y!vl4@xV8I!sTv$g1aI^sl9O$DTH*64; zbj!FgG7f{Snuuc5RF?wrFz1&pZyOL|kuoYVsDCan9)HvSe>{C=(jn5_ zNOyCnLxXgeAl;qP-Ein`knRrY?(S}ouJ_j8|9!rB`9Wc3?%8XvwP&qoKNQS$uMD=G z=Ms_%_sj@S9p=zRB+U;5_+4jO$%Xu~nlotKUJu-y@=w^m6RiL-naYG~AY$e@^o^&X zDV7$mZOe-qF~yt84@l?T-(fvd4+)CvzKfEfFV02|oCN*O?-kTR|2pDcV-h^~^>~$cFcVhF9Ut^(1sQEimw4QkfsEOK|70{!dXUjz2ZXsS7(izoO4NK6KNDo(~e$B3At zDt4K&v-Pe-8<>T@MZhm_T5oje$G4#WUTKSOu}1JkHk~a8=cYF{kPO_W|H1(q@z0Zp zN0e|;pIr1E4XcMD0wSAwnaigfIWr zPU7#*os#Mv9b_(9T9E$wipcHCpGb;q4l}&aQkx0??k@etzsI-Jk++3+4Hq<2Ol1D- zhSMu$rSswD3>E4{Vgt13hL<-TM6h@!Z#DBUR(#hb8LL|dcnj0P+vOnx!vozWlhCz5q(4i zNILPT<8%BQ5$T|(rgc6Rx%M1~DhAI_`z3gF!>I)ox#Ecuqxcnu1W4RntpDlT^+6CE zk{3pnR{^%rWaJ=K@eURE_3Pjik)Wgs`dQ^9Y=+s3rd|HcHrHzodKI`NSZTUhpb7UU z9RpfSYrhU8exxXilx}@&;_jkqNGRSxpo2J}9Gt*1Gh*^x#O_&uw}N)n5xCz<*s+Yn=H~jV%B+txfQd!R*9keNnKYfcXXPXH@E6z|O7+fe!j z6(e{OYuyulHK3j1ae3k}|L$FuH;@cw+8rd&t=_bUPJ2UF{`MK+@%Pg*D)ZE;QqA2@ ztSU9aNu4fLAxdf$dfIvBx`QsWNb*}yqW25M-nMDTCwaC)KOCHiQZ;=0W{7Q5^=#(Ud; zMlO9dM&nFZ%wOIFNUSdx>VAY{zr;M4+|5m4rq_OGF%q18giAILH#c=knd^p$3FxqU zm4*MI+Ao#t){&FU00sA&t|AlNm&d%@90ZemA50et4_O~#e~LM--jmFfZ_ps`fJu6T z_7wyTPj0MVc?hnm>Y2u3yYN?2ky8+&DECi7*P3W$YpRp5SxsUk?QJQwkTTGwXi!0( ztu%s~TGS?L(S9oXZAO4OSiJ7h?VF$7`S+jP6w)1el2wM7M}NkOaWAVyEnK>OPh{*E za^5f{lt>sDW4Dp>vk_XYvTY?=l<>)RqMQ6^cCS90pZsSZPWnJD)2c z9=DB>29fhyWEpDuwDN6gVf@1Ze?w&QEP(7f5F!K4%`3938T&cLMEB7dOxiZX4JZNi1=q zmK?GNOq@L$?Tjb=XCr2=SIeQcSafJp;6`3FU1+HeYBprGk zj`HjrBKg@Q8;7MHhfhr&ki)jNLhKnH2UxLZNBL{CwO=&PmG?>Ts!UKVUaI3Y+uXxu z2GGH>>m4@I(Bch=M+i*8{Al>m@JZ}Ydra}UC7f-{s*|tHmCv8HpNGM2!iu?4Wh;?K zGKcn#RiTNIJpEMgy~AYPP2-A@DSu|>IhC?Sw;Jd}e|?9R5bC(PG<`XiMZKJ5G>Hj< z!27V#6CBz!vJuU8c`!i%yAl)f1~FLzl-_3qoglG(=n|e}3Mtw8#^}>SlBqd@W+gI( zf3Q-rw+O$u_~y<9OC#(xzfOnd=Iw%^gc^azHj2g zM|jMEpE6T9DTyD9h38uFPmdFCtg@BoKjbL!yg{#ZO=#@HYr?=p8n zdZNP>@1D*dTnazO{CC}m$vXIbL_z{&fnT8mrZnYE6-2m0iC$N7e4BP}_PJ?&h8fBL zzOWt7nsKiSCP`o!j?j!rxkCS2^AMlwZ#@EQY1U_Oho9CJ`hv zsIn;$CQhHq9k%P7V(TnA5o#&z=qVPk{08w}!(t7xh|Mg&PmI}G7+jKF^X?W%6(f9LQ6j9PHXC7=#a9T4qarCqnd51&Rj%L=?-YcJ0risO z_0xkLg;xf+;97jnR%6O^%?o)Nw`4R5BD4GI>4&C}W16415)tBlSr)qFI$%@u-}3Te zS^lA{A0x;rbwQAea0kV($e8A$?V?F}!zYnKVKPf!)}7}9w~a1zzMwa(^i%^94!9~1w>r^PXeSz_T|~>D^`C$iKKZ_Rd>KKcUaFv@due2L(c}TGjjRY3 z`Db!C$!Sm(dbaDb8;$9RI6BA$HN8)ej(tSNlebKr7t``exrOY*>)EVLQrYW`iBc^p zo6JV5BfSV~=cMa)a_u$GULu=M!Drnax0m=eaQX(6a}+0onrC)?;aFvGik|AHldz|w zg-sPJwirPJ>nkGZ-8(EkaSs3AdcJ|eWVv?$mR7(Vt`H{pSRen3tNTY_;R&tJx@@Qn zy9TR9qJq1i88Pl3+wz{l`Qid?UC?MdHw(hHh=QfJ+u!0!fy^-c;mpM(;VtqAc`DT6-zZ?i_FvHL^6&mOS7`#nz%&$d8SSHCjeo`y? zv9xM`H7wZVYDke+B|+maRday~y)_UXTAPq{+UE#lfu0vHDD>OYYxt{wdJ3=Fr4w)7 z4J+76!JP2A@+^qR`MzbVraQuQ7U;Nrby-xiN~@Xmt`%LV^y~;r!P&v$BEEuUMEa9|Oh_$oi{uejYB!`)fYUV{KJT!7??jDTV2ni9TXSiTvvVq1Qc%s@EvPv# zrveq<0YgVSB6o(qsB-C_;kn?PEU#fyhie`Z-@FB?1h5mbZ+v%wRBDrlKL=He&8cEz zVH*GL4d9puF+ksIQTM8-7|$TVFvrnv?Me(u9uurpc(-KysDd;TtVpx^%_q?dlc&b` z762}@sr81irYjAys!rOxEyC*f30BM62oR2pMf$fNb)1qF=1oLurVm&bhHXB<&c+U= zX($TAl!$$sg>!OP*<8%7=5c>jkIkf%4YqA6pbAfGb;LuF*~gpDol;e7JAXhAMJvBD zqxdjgw@j|tCO-(T;#$x9W#4~~4#E9Q_|b11aqj+f3rm8gFG|3xiLyr-9WHxj={kQI zG}OPff=Dh&RBiq|JH1=LUmNV~IQ$YXW1%Jbxq1)PBX{;PAdd#Wc>0K~--HNlu46?xjq^RQdS8Ky^aFmWP5UGki`U)8 zd!b1}`V2GyXmsIr8Jf`Av0p(N(s`1~FfBZfA-lixW zYap|SVUI`2SX|Lys{%HLEUm646}`h>qCf267-873n0c~#S8-y*%)NhTi`A%!l=gw8E|oe9?30{1$&u?_wBK9S zQ_IurB3Nk>ALF$`tE$L}L$(2Jl;a(rfm2={B-91EnlOC#z3loFWl>1b_dGmU63RZ~ zR-x@4-tMw$aaW2VK=upddXsD1CNk#pvW7lQ`QO`y1>0hDa zU=uR9Jzm4<1wv7psN zpmBQwm(TlO6IuEPnHs`p4CO-3cIlq*`K6LB9&uQ{Ca ztoj5^sINfW%qfW(T?0HPevN&6kPUJPhPFWg6hTE?Kkl0&YeXdn529%ALWQee+fBBZ zv=Ve}&)?E*9gB$d?aK+9hm*H?t-Ax)@BgeybLs?D2dy;T+zK)|?miKv6?UH&DOGFn z52&ndqNa68~Qg9VXpV8Ep z6aJ!`4j6HrahMq3b)#xoCOisS7o@G?-NlE5pV4Z9q7n6t$wmc#o3PN*X~ahpf0uE>{l!z6SCr4j4&gD_XP>O%(b~O62p! zQ1kVmx4{Z!(y~on&pfRA&&;*h91#)aX(iH_FcWC>JL1`~!VOwEe==$zjtohpbVwVn z^QJ;#-rq@HH+%P#ul#YVQX9t878wYT5k~tb@D@o)n7WEq0=xRgIXF6 zidy{^s;ZQ&UiV?g)!R{hrjw>3lW^f-Jl6Dz-x~6CV#}C!`bp}`{0W2k%<9f4mU|zg zszgAVGWD$~Mo)vI#NBrR62kCyFh50G10_`4duoq4^aFy3xC9W=R8=yHj`HHkm+$pp z5!oM|7&3;!c6LBjMCd&a?)Uva>qc;-EkLF6hH5H1TYZsCZTaC9mwrQ@Qj!l+pSBgw zLX$MREC8UW3#|@4&_j?Jxk$3A_xJMsi!n)|R6bg-DIcT_BG?t4WK9ihW zT^q;}-J}(W)t5)jU4voFq*WO_sd}1_2)Zc!BXqIOPw9BCS|siz8wc;AkbY@J!TXQ9 zblS2O1_j+>Q`0gMtXk)|SyJSY88++WOV!Gf8Ag4XTfQRw%a^wKK`i= zF*;xW_Fn}PAV!hX$&c&(5>WdNaim{F3iB)4fj?Hap0&<-#}NiJ4}x~}p`J9f`-ce| zJ2Z-OAwyhkM!-Yw`Ek-D_IuKug0m|lVbnP06=RZQHY-R9toj~@8kx-jDDr|aY5O4n zKJ|Mav$v1s(TNOLnVmN&98+MJ>)hE8XrUk08=O4f7&9?L7v{xkEGcYFO*R($pTWT(ZV)FnsY zd|v;ZHfT%;W3S=gZ6zQ238fonH?* zUHDxD)IiJa_FZ}Lp$Dj6>*t%SHLMsk>-gc!1}L0ZB!1%JUU`t0CevT^f4{lhz>8il z)c0#wLAFo%ge@)I_8G*`i$qnSJQ7F3Zq%Q^d>L^UHiPD^n)fK|J@Z!Y&Bb5n=iEDx z_V0%-jr2whqFoVEROvUo7<$s#Yx<>ez4*;xx1CGdhYAJob7$N@)VdGQ`B6>P*Z0H+ zcdH&(F}{AcKT7=AV6R&e6v-i9{;~3HH5{0y&$o9#p47&MUgH}n!9NA_3}34U%(_2OKH{rVVN-RdmVC8K`HB)3C<%gy-T0?&uS zXZ-IA^}$AyBaTA#ANL_^Os$Ukyp`yurs+LLM{9o3)cf`{#cZyeHMH@HbR@82Yw6Us zim21)g6jZDFWG#6&2+NYtiAGpcK?}M{a(YXc+KJ~XM8bs3O(v}vI5XriOXe%g*qP% z3tqZhEBzK#`nls&BvQ-$G1C|0M=^EmnrJ-7$Tr0o8Ve`1^V#X$9y=$)*>`Q?2xINW z3_709m;vb|ik`neG8{!zs>FY19w?Vc(8vMEUUKyU_n2=rDzOW36 zE*OChzrtVjjmvB-}WMkq~djf4w$MIoUb&b9hxGX_===L1c-#>c8_k4kZ6HXvE zp|%P2A9i+wBpHe|v{NFN&+xJj1SfCgZsM%3d`K)!W1>w{l89!iT?|jOFbVqI%Jc1z z#fv3xJu_7n^lmXi#S*?ryc7(*3E_b3?707^JEGCo<4W#X zc1HbsY5Vm5Eu)`5mw*57gs7rakP0dq(Fe|?+vY`LpttpJ!>>D((3q(Ef+`1u>wV$d z(BMs!Dr?6|9k-0W;8;Q+)E1Kskid7aPX*DBw>FxxjunvY6H13zegV0AA zdP39LQm)9z!f1w}-qE7mQ5}v5Ww_Vsw9!yQ8&;Zuq^C!`5%If~yNa#pI;&Ig5_#bJ zq|s=$o8)Qt^FwERb!8R8hmVn+rm^HF-=A@qWesK@)%v(F0#%-+k33(~Z|EOF%kdYA zcoj=sg?~)DVLQBfPo>m<~D25BNF{-SD- z{%94>Gn`!SU0l~h7m5_@HKgORlH>I~oUEPw7pm8}xTsVD4<`|r8|xpO=0skH5!kg( zbdLB}U71PW2wnlNvZsOK`8gCB?9E|9f4Vq0Gq1U87QlcKo62;`uK#`0hvCi{&n_4b z>Iv>mL_RR5@+2-cHyWLtnryc4P*zkMM3N|ENG zKSargB4CT5C8(&@Qn_?PA=>ht3+5aoLH4%GEWSD=WwceI5OS<>M*)x`Dvmlew!BoT zLM${iFxAjIB~uaYeG^4=;2jrLRbt_r?juX4w-rmPBhwa7LRCXCX*Y-}N5Unqvocdx zgPenkxp6#>^j8w-f#UV1lX5~;A0JC2wiODECuoCwLtRUFFsE^VuuTxNBnVKkjVb7r(Z-hoP4qDg)oVdX`^ zcb30~g~w?%sJbE?4fzXm%ZoRJ-o*>Q4lZ<2S2iQlPkkRGrlVmvTapsTtu}tDSNe5? zq|on|r=LQEMgy!uFT{49NNgh`Zrzg< zu^9T{7{))8klHw(@ydMC$*|_=y1`ii0fMaFj{Wo=!?6s+E_0j{Z}1pW`zbd=1rM!! zGo!iAfLHM4<(;u4`tg~V82c~NLx3+?t$qY5uZjShtoj7R_NPVlLpg#K%5%TEepeZQHfyeaqsBwqNLiw{T#& zI9)w8&T1>P#z=}FpFIDWf=V)4o|H!0EH>lx`cd4t6|vgY6@AYn@EE^}TmnxrM-0}D zMckZE_kx#-gy7^j4?jYQ;QXvDaCeS9{zu)7#5JI;Irf!lS|5q8dEbk?SA_fUMT?azAKd93Q_C881o-UcMNOtSKAp9`0RGdQq(Ac>tV`w}ksVVWbx}~*PHcV(-IBl)=qE)5jZ4ePDW#Ea8qI!)bmk6)^(?Mus z-h@IhT{K+e|DEb7vP?fPnPG8RF}pvcguyqTg+g&%U)pZmfwey0L1RfBJo7_RY<4sc zk@#krc}zno%SJ;_O-)dhluL0?i4FC=$Y=pRyXnjN&Guv_E9~QaOklsj3!sQlDLZCV z_JMp~2M-jm7v?17`WSNST}iNIPr z71#i8gBAP}+As9khRC<;Z_<6pPwN6NJjXXCF-;OjigfKqYWIG+ zuT34lWN28C5UFL^vQmlpWN)ix!u%!1-NF}r@>pJ!b1{?$n}QprUH;pk;c#Gsb{+ei z+x~43B4MV8(7z3m9V8RIHDaLJ`)xq+5vqlJKAXyPp5jm8odUn}1fM#;X+==H3%L+3 z2+o9+`%m5vAt|@pL(WEc6Yv^6(cs5Emv_{K-e_ZO{uXS;XZaDJ*~M;4eC6C zI2?3Hjno;7*~!2VU-xw0iKYi&-Yk({CS-gz4#__D5X2_yUS2#l+rOA3)Y{rg4&zNV z408Ik=UATa^h-x>rTizQ8q*VL7SK%IV0a+`5^xNL5AiNpj1`e7wx@34Y8}>$$($I? zW={0&)}#H=A3=miWKsHB4LK9%Ah;*IfqW94>@zbl$6x|U`T<7f5JpW6$7n|92_((_ ztN3#9tj?@$&Bbpmp1+0&@E2d1e&QlxilP178SL}|bye9+QsaI+ClQX67^*e<>Wreo zS1T6)Zc zZs%Q#RCfoN4efHZy~;45yNq1j_o#VW9DUN49TO80p@?(XXjwD>5B8CEJTTPRYRK?v z5x?F0K!393B7nLR?a?a5JV>btf37wa!7$hje#zFb0(t`$lMBAfYfp&-zoUjWH(rtD z5Mpf;gier}R`y!4Pz5N!c&ucv(EemsNgl4X2~+=XH=otz`mr~uG280v97F$&cD6E_ zh4`E2t3&KfGZ)6Olma9Z5N>KWBZ41=YN`+m-C9Li3;U&CFyD&(RKxyOABN)7g!CH1 zGO1+1MyFe(eOq0X!FUX$(Bxw+AjXCuj_#! zDFvSt&IU@TYRm9ecp5X5!wz!NZX{5Av#W#3LKQO2SinLhO*aJhb^R zh%o3U68U8l&V+mBj+q^JWrl2j95GQD>o4~Y0tI!l?sLx0fus=jwdW?tY4rcZG?M19 zqkjY)xhzo9V%tGHg8xd|4ys&N9xBs}@{Gpmpo`J2fI#m^SnyCTXKCo(9VZlAq>ISp z2OjGB!yA{k@_>iyEin!98nNA%zFO)}-{aYMDGhFx?Om4;Y?rD9`IIvJsb}*6-AM#l zcA8Uo$(>j&+B4WP-GpGp)n*ltFQ3b6`#x9gOrMtE^_(2DI7VM+rStFRw!<0@|(h(ft(xV;V69Mq#f+1q85k z9P-5?o07K6$`k@5P4rb7R5G12L~Q!G2N2UnkPrjPx46iWv7jsOHINf8?$nRS0AzN( z3Mi-lkQpf)<%R-0MLXY;8I3rD7|tS)CLNzW*q?uTBVk# zZU0Ws=ODBwt}1Q~p@8o}5F~t_sDVG;U=L$`0oHi9JI*j@aD6tj;dmbf>=XN_hDnN)7E~3iIejPKQWsTAB3eu{!CA4XDGeO6d^^a z#^v&BqZgop2;mL~&aLD|1%E}afEE+XCLoJDRl6&wz<;@{CV1N=By6-)G5)o&7mar4 zq?!W(>oRpxQYlUKs@-a)e7QIZ3x!(xj?is!dt0yTYO4s~y`t`5vLoeE@9Z{F{+&9h zjzPF}aN+}WK;G8S=c;rHkRDA-Zl-0j!SDL`Skv;D@9}Xkx+4%BRTX0_Wvv9l8Zt64 zm|#gIN5eERcue|wq3HEKcubChqJZ=fcSdGX%IO%lY7nxJW|_CpMDQ4?%{$p| zZ**7_w-+U1wsPlVOVCBN+<(iEMjz>rVa{wE9q2j?T`|+S~Zmx?i_r-M3A2~JTGal!!8RbIVK(b|h=m_|k6I!5$)hC?F2qso z5G{`kmaP$9OT3*&891JpGTXG7*LFuvC#MsAnlOe{sH4t-bJ(;Zb99D0G@_n5M_G?AU_ zN^D8i(-oVNFut@WYs6H9bK)A>WZuW{=V30&Y1!q=#*=H!ajzU?jS8Qj4xOW4SZw3 z{2GJXij|NP1Yex74R-92{ktCMKHu@@OfSBMgE%k)+L0n)Llx_uK$KP!Oxv1u#S8GM zfDkwhj>GEydr-c5LEjMvFsK7L=~1Eb(1?i1yIxi!7O=B~+j`-EVbYZP_s!@0Rl6Ud zw>gs2oy0+ntsyRLv*H3O(>)I}ZfYl6s(@Nh&3dU4WEM*A7wCEKc-meyc=P~j93kow z`IFWb4GmJrR63Xx6Q{o(YMSzMtneMA@_V~tjoMqno{jQlbg1cp*DF^&tqoDW)&8MMQ!{H@h3IAN+#bn_WMG4O;LkEYM#R zotQqB-RakfOaAkPs#`sEsWHG^o4+6t&c?<$Q1gqYNYO+a!#9(~;Ftn&J_Fq-$QQ9N;XGPYL>!%4}cK8@nP#ki$ajA&Rbsq8wKz&il zb}IxS0!J}eRscSJzL;bBkhgSojw7Hp|3}JQbxkVDlpo+tt|fnoPBl=Z1X?MEdpuy~ zG>KG%cXHtQ5RaZs&@(&3katu^$nycCrx%yQ3ky$+TkQ8(%aNnh(uak)l8{^xaS`Ki z*o<$^CnaBLbiI4jHECnt=&$8VnUc_@IH))SSgtAKCG1-ese=C2?J9)OxQ>7= z9zmsC*eb&%dN013sq{Bc&IgWu8~xz5&sKv=rZqlnOLoclk3x|O5Kur&mRx51Oi0cZ z4S4s-w|hRD^hXgXRqAi+%uCrnOvoS-r=2FInJ`p%zkApuPsoLlA?;Tn4x+~*i`oa= zwF0Bo1OBeWh434%SbfDf)!;H${v3i9rwh3~gQ@yjZ3CL+-||8~p@~&BlTndW!|Mnb zg~@4MZTE^(lYbiUNt$I0#N|UM?HEaQ^>tecl$_&5AaF=Ag)mQRE#y6IAoC?3J103~ z6lR6^@hIKP#@D!29vAU5fS|GV=FgL{GMj*}@_q8yxqS9m8$?F?)Ww7m$#ktgF<7}4 zwX_;etOWA#-DxPo0%nz-KV3ZiV|JCL-sr=5Qk~6L=AZ-QLzhmI)E)?m@oIY@R;k6= zCKR1nI?#S{1V-nLf#Pbr7X!mpi@UnztHYo67v_`~h7CgH`SM*7Uj&Y~h)u8)DTTu85%`?e)LN zLIzfTfCQM#uO$YfMi_iD~;A+y;52nE&ovTf2{Ou zxz>=cvH1qsxkFY~maQMR(`i&-;Co(=hd3Yti_+gLDYSJ%Pe zPN=f&tOZ=Ym~WUY`~F=PG|)16T08DzXRErG`uS=;;ei8HL7$P3>2eX_?kzl1MMmC- z2}X$PZp7xqq{Jj4x)N5!ZX~#mOkZsk9f%=AlW>rfyGRZv z=^5?y_Usb+6F)eGv%`F4N=jCOR=OQMr`M zb{lS{NKtHw(rUHE{&>FNPID2h#efOs94BxcZq5eXzz@8$S$q`penU~KZ851OZq)D&qJtRw$PVBuyg%Q% zy$O)VxNgK(`|`w}wP_?aOHOpeSa(*k7pEQM{~%#*KDnTfo=xv3!v3idQLZg!=ioSE z2AEc&x_&ToC@rYH%2?a~Q@4XsK2|#cBHc?C{nU7=2#E1`+MM9uxk}-ZvnEm}0kQ>g zY(+*GBBR%6fsGzD%~=bk9>xGcdPF zqiqz0ZaMReP#~2RWFt0RI1;-(0Cdq9*yRiGoOyo!k4X*S_&?m7Z~rH32qX&V!zE8V z=j3w}fahP&H{Jr_EW%IK801>LiuqwLIzNHTD>VFvJoy|^r8;vpYUR>@29&QgY6L0O z4P|Yq>$Mi8)hcS=Aus*q%j5FWOY4h=qz{HO_-#mVc&Da>ui8@G2jqU&m zGvp&%d32>DV{n2O^8v65QFf?8Mv0YIlv>WdWljv~aGf@=Ge?|pe<$hit|m@ss(4|< zNEbp(T`txml>49b&tU*2gn_r=RJ3n)tYG0;W4y2|(PSz&%l>}d@xD6{oR(lOtQy!v zup~|P<2Zhxd6myiWXg4V?o;@OtrQBRoIrMZg`-ZdapuUB8h1lkPFWUT462Z&HtEZW z%g41sg`x7Q(WC-&D#Za~fZQeF`2&7WR7Gw1uqSoGvFkKEt=w_-zafSa;x~ed!IYBX zl?m9%K?F?>N6J9J6NXWLln)M@xi~$2bl*W2!}M4Ne*qBQ6XG?g;C{mexGPhK<8sE( z>&&Xg=gMZJ?~Y~20`WSDo4d)ZX4}{hCZlX1Y6ik)wMx4nTxf4?o%v&?Sbn3+-xP>0 zf&r$qY$ep855e)SRwB1~z1R~9UZ4jZT!%77hox+*;kU#e`bMV7x*w$ojbYATDyM-M z6tXfSC!sP%rG{_s9_~rSZ^ctP$u8gs1DZU85ADhU6UK*;dk!f0^;e~9epl#EXjw-U zF!Yc0CI9AqU~ZhS)%oqQso8+H&u22cXzEM za0x`6P*u-)Z#8x>Wh?5;#|Ot&Y_S?Iw0y623LvGrj6z3+g?STMtRkgEU+C4TSnfH{B?-S;yGO!4!!L5GsJg7+)1IdY z$i9%X`ahbAT$dLn&KW*SB$%yuRDlYY&poNh>5M+{{RE?QGD{8MC#+s$5AN z?s6adja-BO%b4_%!czqnX%)I1qZhP1eBwh%sD#F|8QP_)2fi5FJD8V#N&7ibM=kTi zksYfMf2||Hq+?$*t}VSm^GoDqa)l>P%?~!g<4c;MxKsa5`)5fKqAHk%^FTFq=HE6s_(fTl1IdqtPVVdFcQKl9WSw#^L~3X=zAP*$w^Iqase zTjkyz93D~|__*#n%BBMa;{qE0Tuv0?9lZVF;H)ajBi7V*|)b-t0i`{xNP#>&C|Qbq@OUGw4hc1BogT1-H( z))!sxxIDP@AosKTu0!(LU^s$%9Dw_1*f{K^yq zkJXU*fkX4hm;wJ$#EF^51%pbR^dDZV0xA1@{)vG%RRExgtwv^^`VN;d_gEiCyIc); zy-T}G&v1~AXGB=2QzF@6&=wWvtmF?w$=A7D9cR_i2^kv>LK1PVzsN+MYLi8V7VJL{ zeK$;G4Xg{NI;}`D4RP|{``6m)LW`mVSpm19sOZP^pm0K6p|6^?KQJ&T!j zJ)R#}JF`){$OnOIGB*mo(grXuo$STE27(i_fj|Zc8?=N3B!V9=!4T%}pEj}(M2Z3A z7F%CmU+OC!T&Z_(f4ag=l}-FRzdR{8pyJ&(Sa{zE0)Cl~I4pACT3CI(U7veJYI+js z-37rx46>CNj3gR|ojc5MuhVwGb=~0%?YvgVbht7kWiwKSg%8sonHnRYoU|6J&L{hC$0SGI#n z$)H&sS)K;I{gs;YT1$_nm&F8;KG{Xb#GyWV>&H>30Wc9m!lc~;8MK@yI(*Xj-Gfs} z&{$4!>|+G-+C&P6m}dnW`dJmOen*`60U_mu4f_S7O4<$4{H<&-);1N}|EPd1{0?FxSl4Vo87#SZWu%7SNd!+)!sQC;w?+@9zWd>hG0g#FLRVM3aW{~fDkEu zJjs{jzij0=NoFY%_w^0|^q+BT4{X$9*`nS^+k81bgt7PNLCv&N1pnU)0OkKQJRYJ; zOa>ZSHrbGQuhzWzai>F^jospl2JwBDH68~-iQnNvP1CQQ^P66T&UEY5qQNQin|3rr zUgx1hPxp&=BvC{uBp*|oe@0knid&`g5u+*x+{wSpLVhY$87oGYEh&FWpwkf?+*P9N z>TvO^2&~fEEFcIubRdv&RAV9j?)UrMCnn<&DzmBF$EUu)Wqx z`x$Xg(A@tV!3P}{i1ga^du*!3Ak!>nP?el#?h$P4D0)Ou+!&Y3U@V(LS|-pyQ49}d z%|xoTj7Tf$KTiNg^mQj9Gh))DaPJR>9RH};B5FZr5oH;EFR>|`E|V?wP*rnTvCgUQ z>iq+Vh_>^2ByAC2-TR`e{W8^LUeJXY1a>v=3uiJf7Gvik&*Js-WACB`;;r_Mm6{jnO3BO|_GJs| zeq|Du*A{8!IEpqVmV7$&cz(M>sTzi$Tv}9Iqw#p$Sa|QvShJO}W-{tzVYt?akVv`) zhzC>8r8T6Wtv@#qDz!m$rEFug=*&V!SYU|Ve&h74xY~VFY#qwWimV~wJ>pqGP1iS` zMxUPa;?^vSJWQ_r20Cc3n z7kU}TX9nvWMBnp}gW%~I8_5|LUoQFxrhn=1e=b4Ys%gtUys?_Z8kg}pt z630CtJCwMLWyS%SgUEQc^!^~D5gv`a5WEkhbK3HQ6KEH?z)Qj5CPlQO9bM>cnw*@R zrdD80kd%d(zStZWS|#(sie}=cGiIo67(&Bh)#`SJpoQ;{4g9itw)r#OywU4}vlRN^ z@yrfNG>>)Wi-Q$2bkc`@#ZsxTvUuNrN@cPLLDN>3WGNuL$K?pf+tSMUR_!qm_Xe*j z&W0TKGW;F%Xd2n^w6}Xch9;o_WVGDE->g+Bq*eiA@;v9SnQaxp14XY$7xw{UI@ewX zF|w%?$Y(IC0ewIHz_rvt1*U~8$IM~pLDD|gHQ-hT;?^3;6#O_wH%qQo3)se!1ww*i zMp_zRs9M;J!u?nA_UrM3Q2tHsykJAve86xaZ7IivfRCGdka);D_A8dU&y;!6Jqx2>J^!x@KoS}@VlsIhDQ6nsia+J^_Wl0O zP$G<(WPGaF2*#1rm3`~pafOoHkK^PjY)J7FelMlS$LybU)CI}76$paCEC6*7HhPI!9#d~ zW0fFvrlKx?9-V^_bR1mC(LAk# z;EKt=Y?>iWx*$NAe$kA!*ezXjHQ2==M9tLXt4X6~IT44lYBlZs^?Xnkv@v%z79ZE$ zKktPt7anRzzW1sj+*`tyTXO?38QRvfw|T|$jdCK4GuL=g$+0@mTvOY(sB#bxk23`LA8I6gnHYeyFtg?|SVB?f7jrk`{TTtGBkmni(>>GLA$C1x4iyNhljB4tS?D zG3Q0g$TIxhp=qnl*;>rd7hgJyxHP;T8AN25Z4SKsm7KGnq7@$j3(0lmtJipjNxRec z7O?x`1l9+Ot)JfKY2LqunUB&kwMgC1fWwtc*z^oDVvMx04eDV@#evLATmnsm0! zRcAAlWHr$`tM{$4$#v5+VY{;~eZb2?q-8|*gd*`HDtY%}3 zsK#UW|B>|-P+2I?+k&8kN`r)Si==cT-7O7L(%lWxAssI*-HkLT(kUek(hbrL-y+`Y z{rx`=$D=6w?#|B6?#w*T%=4ba+y^h_Yef+tV9!1o67RmCfnYw{5d}=@stL1rZ`At8%v`-XT3*E@!x!$Q^qZBA?(4-1zvZb$CTz~}m<8I6Ug6`Y z$*>S5OVPo-+4RT6Z+31lyyC5;yx8 z&YDMJ-2C0<2y^~+7SCpSI31>@^2 zs;%*S=xXGl>wVJeh;$0ed>ix#S_KRM;a&na6%LLVB*2FV^*+vwmUYU!_Ox<60MI{j zr0(s6fJ9GDWz(RuV}f4M!$Y=MvgR zVkuM+F3Emq(iv`=BL{H^(h(p`mI(lWN=rL)s;aZ-W!!=O!*XVFA2$; zWibH0(UHi=;_6U@ecz3&f9ZBWR|)s~K;07j-YZNdy^};e1}4MCTIRcy8@O^%piD7d zQ%EI}>b=vdv+;8|*%n(HObZ`2=OVH+1{Kwf#tM`aGX#Tt@yNek8WJ&ipcK-5oG1`{ zT2JyE_X$2`szw5z0E(>g1pH7-1@yNy^mtzqud{H<8_W4F^NL?jzJMOQah;p+0?Q2V zP=Xd!^0H=#w@ORMGRVdJSL<0@uW_;kiw!}zgsV}?^P04n9fcLGWak#%J7X;iV`{?5 z$TCST8jkL}jNU?!^%}h6$rQ1z^9L>7$YQHVA!nzhs2?k@3+h&>RiZ3XIB81D2RB7E zJyPg2r(*+ypSyPiLh6^O4cE~b1gzavYALtv4{f5_`(r|uDadua9xM*tG#)(PQKOHf z-PWR8n!+p{>f)0aDQvwg1bm|yZcCz(q~xH%z;;Qr7Je@7J75H08Q?;Z8BvcbONb$$ zW5n{_UMZ-d5Z56sYG;tf8j#ZP7QQXTAG148jz#<8IywsTB6hvI$5N`1c7Al%x{>xH zCzC2PwZz_1!`l@+#F%`V;-?D;_@usq0m;%wf^Z(JgLqMKfyb^-vQe3FB(TOt0BOFc zSt?I>dTv>xgkyz9Hq`7zMYjiI(35t`OF#S77ewSdO^ym$+Mhj4FFtRBc;JewJuGdt=o5OtqHMn zg@Km=N`b^nU#aZ7uTa&S-7DYmqPm!Mf#`%@4pi6FRoeixq$lCO6B9bnm;e%)u`q=| zG=igR)QWHb+G-tj5_pQfy{s_kl?N4pUafi@2mp;j(^+`+3EuWrl5igmy;wu0BkO~y z6HesUn#(gkVz~jqjo@NKxjbF-g5d&OjC>6dGl?z?xW%>doOejR+(!*MbvaNX`HRXL(Xwj ze9FLngtrPY8nDb-PgH%+APBT`!hO|q<8SnnW#Ezu2o)+!Lk5=jGRsAkjw8+OJNs4h zV49qk#i}_yu|7Gk#LXJ_RxJ#Rk0kqMz=iyL7($Ugr-#`ht3_Cz&`k$@uSR7!&_76x z40~dNbUsZs)Ra+LJFq;b8!@rqqKmnA(Tn2vw!XHEFbyWfa5SapM+{~&K0R}N`L&&s zjnHe@u#0k0I_E4@#xLhZzEkX@Wh&GSF3jCABulNV&YIbc{7!oWMEGm~^c>3TvG+r4 zV4Xw=kqAsE;FA<2-v{V;G+oOApK^}S1S-a`TYd#Cz{Ppn z@2%zLZz$&bTyI$N=i$Fq&4|8Q5M#@&>t#04>7w?BvhW$7n z-uU_Yeg_E+s@Jbae>#`w-zT>2xA(VoZS%Vu=2PeRZ!~LymnzzKtkxqYdlQ4Ihv1lz2|u=QMB=k?Ou|e=)G%^0nCr~=uGX8w!(o+(omO{l22c* z$@!3ca(cQqbjccfI-wb#%ZaeP3y4xajz%>Ue|LQ-a}-2p2dCZ^t0@J?f*@b!VtYt* zW@e@v-d&`@+^{!pr_vPq;X^7AZBH9Za)IrXWj^L?JG{>fdauLX6$P1okEHN!eCiQX zt%#$$e9EA9J%Y0a2Z>(t;hMGlEbLN8v3z#$Yh?No<{Dk-a*8h&tL5)I0ZaXEzFv2L z?ED*kX)q(7QnSmG9mQfb<_pl>El)ay<6!@6UvH-tw2aKPS?L*?YjUl&pWv750eb=3 z{_SLW1um^$fVgoc@R{OFeiVs;f5PHXf>ehP(5sw`*& zK?M?(ilArU1@N`Ab%G89li2OnWlg54sDRrhqe)z2R4OP@w6&Jz8Mg%*xdhHgC$mL< zuc2011Bss%a4Ha2ru>lzN~90}FqH2u8WQh2o=Wgg)5e_NlJ#~F-ET!DeFf-D(Q&-t z1NWOxF31-If#)WJ3j#eFV|J%B8yvDh$K*(mI=(&}mgZYFx%iL zgAnxm!Qu+gzXzJCJMIEXM>S7+*Mvvul>N63xqCP4>9m&=o4vr5lfkInJYw3)XS zJ3VR&$_Hrwm?kS|iqAAT4|{Wl8scRjL6I;#zKAXFK7AF|LOC`3MD(@FvKxJh`S!Hd zv4i`!A!>y$0~ZMFKNe;I26>I*6JimPlDD4$(4lNZmdzNa{DE*>R_Tr`-*mWf8@Q;) z&+yU#(>)?AOv*PwF!DHiX7u*v(y|A{?zVB1KgH8&lLJHK{KOkQo6j5CCWOLm0Yd-I zhs)m}+X5bqp+l|$?tSK9K^J5*0D%PZ;acY0iN0cFU}VN!{949zT` zu^Qc1$>^laOutVI#Fqj)@P!L74UO?ngWh-FYdI$0gGGWiQ8?$vNcg0+3GuJgo`hqeLMlz)$6^7OeCER-$e1exf8XTy^@f9kO;4Abs>bSua^0|60((t6F z>5~-3{Cawn_qb#X?S13n1U(GaEvUnXLeE-%!@!O{fuER^ns))A|qY03qUQ^asNydP4q06r0KD`r;V)4Q672zM^e_uK@zpcU?Do`xZNnU>Q){-A-p68HbQi zp*|7%lhF`}gg)o*qJP#aTO0e*@dQM;d-~QyTOVTCtN7%IMha*Z>Y56qD}u~Dk=j?3 zYk@Czc6M1H9n#~=6S%?ddSW(-r}s?>f06vhBZ;Trajfjf{0OI27vq`_Mw*&IVLU^r zIJV|~+7VSSAX{FN;#J&4?i>7>*xMn(h*Gt-=AnPB?Fn3yn^Qbt(M6N{E0L6$VrAc< zfMOmo>rK46phh_`mfp*>X!RHI!D=TaIpY;fhi2o*@z9F8u9KAKfNY2 z@+N56D=F&o(9xs~QUp?^et~pYWy)OEi@sHXSKzM8nt&@5GT4EGvI2pngHOsvlG?~9 zkvecePC8;iDv)?u%_!PsHf1q4a0_A(CDr3REcxZM9tXPqp+%waN3>r>ro*W#-8~(4 zcXw@`=6x^<2?!4lN2j&doUz7_!?7|Qr@QNhyZ?H@X29?M&6?mvjEqwxrt?0Ug*&@9 zSxBjK3scWYi$f|kUS?!gB6oeixL2`2nQ4>Fg=Lo8GbZli*+u1ka#j%Tw*G2NH=ouMFeJe&)TcB?eaB@z^hqL#};G~*SHvB}HMjbEeEPoMS z8p7@kF7(bT;$|QCHlGKcoPn^fn|=Q*gU<|}@F7SzHW&{iv-@t(f|iye4zLRq;0AzU zK!bNWT9P!u_1}9w^7fnmQ-du~Mo7Ltk$L4!OWFk9E$9&IE27^%FtA&xP+!i#%ZV)p ziwKGITDY%#gMs4Yv;#eh=9BT8F$*I^))QMaB&c%ac_EH_bn~KaXIiinf1V;~)R^jR zkeghKZWD*R_Ez3DA`eE%|68c*$tK|rHMRWGY78@KFQuf%F^@m^BZ0OMHPMqL#9~BOkd|Xe9P@EC(<9d$uaO{IHduAu#504YI7H1WU}fP z>pNf6?2d1n$4(Xis=!53jemp57-pW(Q`^~3YEvc-a+e&<+**wcKsHAiN8PeBzrX;yGSzbacCe*kesu^Na+*f2Lnc zGLyUOXb)!vdHj zk(!uNGiHjlJ6d#ufKDQ(=0l4+7R^PQ8fCtJv?Ajv@0AwL<$#J=X#EQAx;4(zI=c%L z%C1oTQ={@tu2z=|rH))HwhHsh{x>rQ)k{#&0@ci)>SRsOOefl^teVOjP+BX}zRnLh zk1oLw5dWrmDpEvP>BI^L4#K8^3hc;F8>BC6Nh=_{I6W`>WwzSm^4h&aGD>}1d(A|1 z$hstXeGNoE<&&Qv`;a260?g|q!b{1(WQZ;M5cYye`0){>LajcRC4DuMu)&7UBs_RHlhk~jYa9ZD*cR5N$qHzGU3Il*+tqT z1~_CbDs3vX8>R>{CVwRUxDvno`Td7b6n0Qh5vOjyk>z7f$_Mg~%kHPPF}$iaQ!Q?Z z+Yfbre#6CL)(bqxqSV&pIePpQjiE|Ti>tQSvnOw@;i4Q-}x7Q>E@%w3*Vpo$}=r_X@ zDCP?US{4YfCG3eit>#GHF}#~Oxmf>E#($5ommiu}nPFZ`(Okm7O@laY62tqiQKKGh zuX_?}(#^D_^v|0?!7uwua}@4+dm%MwaYX)j@zKr8TRG)ifeKB)JrlA?P9=s6IN8vs zBsbsaVZYF13P^?DN5*buGVCWqoimG}9g)ta?lHY`)9PL3yo{5|ZEe-$vcGu3n8_;L zs)_d_?DK+{_knMNj%5OypjHS4=`4R}_lzHNcW^{|qiFV2TAf(I7ox{o0 zmR?1)My>y8TD7XmN9ha&_4py4$k$>Rqyp5hilp9Q7od{Q^4O{bL&l~~avX_q>ynUh zmGI;P6Yf%0GaHf!eCK#P#jW0#4wbM@8Vm@Bcx(#rl%~qp3GZZG z-fWont}vHbnIB&}89S@-x6yOgm_Br#t?$##$@+HZ^0dX?1;xnjZlM29w0xoP$?Ue| zJ*GO{Y=DV=P;VVHJo!FO%q@!S79H@dZzG9^!o#b01SKVJYvzax9X1CSr7`n00y@l^ z1e$qDyNKR;up3^d@|tv>xE`tnLtuNAMt&TpRHRA=X0cHCnsQcrEQGc$bBN!o5+K-Q z7*u<4F+vr|-WX_VPPL8ktcI@&%VG+^<0G+K*WAX1Q+t(aNza2}*i(1kZ!~uee1v`} z3Tipn@UqBI387}J&Ik4F6gMCijvUI*9M^k@shPU)I)n0cs(D#~#&!OThP`Cr3c)lR zq6wo{)8)h0bI6>`Sa08z2j4MpO2~tdH+w^9w&|L7<-qez2(RC5DhFBpvR|c?d-cVg zaJ-0R0eH6>H}h2pgrT)P>>s26J(MZwgLvCo$*1^S6FBjMroT%zkzzx&x%;XZffk zgAwQe9mKc_8kc7$1hL5`Q&P)C91{M%3Is`hUJH^KuF_^$!;ims-kNPw^P@O?!vhqC zfeY7J_;GXgkuw{uI{;4*(+cxkS3GL^w>ky#Os+FM{a3HP6v59lxyIx3xEcL0rFS?0 zsjb|d>6#JXCx$hVYEAKvE`vJ2cOle$i7yKJgVKFIPuA6Qo?-=A(00Gm4$_MOD8O!W z0iLhJAca&Jvdl)KPflvQ%b5=^h>6IWzt+B_{s=7a)e_GE^`ASc2cB>AE(ZjVzX`kZ zEtdPQc>=eqD1qmY-5XCg0{rR?km5&2Mvfpf4G3XJ_?*1L#gA;83zn@yb}`zfKn)dwTe_we^C5H*Ic|Kl}0WGFJ$ZO7N!_sROe z$|8lKfsgb|4=3x`?NHIx=U)d9z9P7`G2%2m!wv*a?LV+kNkRu09e(G%Iqse`US5NK z6Sp7oDw_VjzM?2}20eiU^FS)OBqATbU!5Ns_a!hK0hs5`G8Y%uI*79`gKC2S4Lodv zrnePWRz~lWtq1J1T_KtsZw?+H@yGx$%e9%b9|A@Oz|PB;>$c}P?Jp`;84V3iwS5*~ zWm&xgo2rC1>!h%oPra6YgsxuZ_TugeH*gvnneiWBoWxESg zA(oU1N?+lwFuO5sz&U9RIf<-$&{E8rIW4cleObi0eaT-W-gjzmcer>MC*<{t>D89U zfytlk7lilq_a7e3SIF0>vl%VXf__H9D3KtTQkSH%xXb^s_!WQk4EaZ z>OjBqqYVWr@t&SvxvZ9R-$D3tg*OZYHA&Ug)q5aJxm<`umClE&uRX4uz!pKOjPAVs zS1M(E!SQPYDVbVLE^;8tny2&CN3q4jU9HiH;r#NlcZN0_%g#W9?Kq_}3w{lpF)|< zWQ9Aqf_NU|e`kZnkDutDVUwQ;hF3uXN2Pd8>rnjOju8gI@%_YQ)Q97D`e08gDEa>V zd#Wjroe&w=3?T3zgwqE>^I{89e3W>1j z5DM)C@)NlnNeJd};Zn5!0Cr#m z*a73^u1LnA0bh8uDq?a$sZb~hxjqc#xIk4^5Ld%tvr5Nv9nKP~AgEGmfBvjX`Wa8) zCHyB0Xj)xe0lwL?cMr{H>thKI1e}EAS_^SVh-zv@>S2FCH10Kla2FnTT3DXNaG&D5Ufr1@@0JtdDNz2WJgch&Pmj}hu z>@?yGmT+ZHaam%P=Id+&bTANJ4ueRsjG8-f4#g^ByjkBv*e&I9p0I^IpZZ5RnP1@7o=Uv4Kkk=Uq$x5IgDP{|%+ zAooG>^E2qIwlO6KQc?y-GF?5*1;0`k`cMEtJd}bmSoLclrB6nnPAd$w53$4m^zHfB zFaN2zl7hKe)d|Qo2Epg|Iod4+7s^U9sJ1iB_l^ z5R%iIAtpv^-K`$iGKCq%QSE^%E#TK3!}U z!Lr;mkExk&sHg7R(>0}TM?)gQq>4~TvR1bef^wi9kX|@PS&zr1g718=%oIAVTX3qD;_1bKR zSn})4f6mdsjPdJ1{?5>3dQtKEkX<{%ODesJS5daF%J_i!PnSo1$b$^j$X|TPAl}^z zFm{%^mw`qd7sp#k?uToGtkaqflR|c?%sfW(%}qavc&HZ_@S_4F+zz@a`sMdaG#h&I zb5l(|wiCE>a9k~T#EI7K8E9~9=4JMEBI8-7^(0v~tAhK*Fm)mfPIe|lp%-=m1DXa^ zP?oFF=xBXtwdtmA=_4m2O1g^f$J|g7YIJx3L)2$wN}S;T60ClBd`U>8rl=k80E7@E3dVS(2vLFV*OA}WmdOgx$Zm2Vfoqb3@xAa81UoEsSbD1rpp`PLR79n0~$XaHE2)cG%~43Kk5 z2oxGU{PhiqHRqRiZs;2`$yO*i2<788e!N2XNJt*q?odAbUZfS0?3w>raGsdvREb0O zEb##&EdvMBM+9DyyFpLV?PsfUsvx8*~ew4$&h~wI8QejY4e1@St3Yw0;KXezp4g@u}}9*}}g49G z?1#$Ol46}D1@{tt1{JZy@mD?vRf)rTr9$OZE1KAF@|~AwcoGHvPq0y$7hLa+)dMKO ziPP=L5tS6@HKyTu%?qAH?{78^r@=yqD~C0y@w|r8v75{$-I&vR)D3tMHz@RK`FkDB z6od2D`7{t-*7odZyA9h&v)C!+7#N1b6pCFI-ZeOVKgzMMJ-|G*>opa~d zR)?`T)Rk@zPoWd-;&6&hQW}zf62!fq2ErRYy5wU#tp+A1^bx6yHyY{vx1-k}?{C#hZWrwpH+i`gh>CK_XwGh74>{f-& z54(|HVKrtXwuV=vBB_qW+p{|{0x9nf^2S#nTeqd(-#gjPtT>8Y(sp)j^1kh!4xb^q zwBwc9WYKC#{L}@-e#LLx{t>UlWdKXFG}6la=*>57+h(33J_mUHrGVgq$ycaNk@vlX z`7F@ZNAshD`uZGRm0e-qf^@wZ{!HbhVG`u=>1ct{%W2hEQ_s3KD6GY1-!4tKlzQ|R zkwRqG_LK2Qr*~uUHKToD)J>KsAH_q0?&)0bYlpSf{{lNJ6|*9@rcL3B@0>v+BV;@` zk@=y~WxjgYWn6l=sQ(!x2mkT_4KhA^>m2fW+J(aTERCYewnl?y+I~WtI$K%D?bpl) z`I>inlewJ`>}faWuVky|Ls7N%-yx1%B$e^j{8k=SM(6pkdk|HRDIYS>l?fQcyazB= zo|1t&oV&G|im!=AEau!6kekBa0%HHR$scRS2Mv9(ge-Y@%+E*ALO%fq>D0Ea*^aHB zH&~%oTIrQ5HCzvMTR!PCZ9H?ILR#g{VCB53&NVC7RA*mIF6?`cHt(uhFkYgkb<1;2 zQQ2R^tD^NWOQvI>=3>Rw-A-~zBv=!tV?LO29=puL0-B>)!uAp_7-G>p^XA^fc*7f2 zX~GK2E)=F_A+5L9w=PEa$HLV7Ha}coY7xTkG56LFEU%}Js@UpAcsv>WuHRifKImy0 zPZ%FF9gE!SXnHyS(RsSD^P#6znq(HNt!FgZ;SGH$YyNnbS^O(iQYDZ%l3_~YuueXw zNxPYlgPf%r^hiOg?D!sqkGSmf^VLoI8pxZxn9sJCW~i)!apwA`+b^FVIkNjNq}}Td z!TZ^AL7#-bIe(r~cm9JDm#`+Ss)g&9=dAv(mE7kH2SqpEmPeBwCArIJ&A#n;;wrd$ zdsFC)6luFsL1uGX=NTP;p-o0NV_$cUH%5h5o$X`eF$bg# z+i#gUd8JPq`ExuE{hY7k6>f+Ej?M#U)Ei}aoL7t|i{xy-UNcSO+htA8=UnsTudFr1 z#8Vq)<4x_ZBWA1mB*wOW7t4oVPw420b$D7k(>Tg!JHyY?$(m-Ex|HJGe%EFX+|0C4 z?iU$b@INN|!{&OJpt!p6s5TUOj*T1H{I5YRT~Mbv)p#n>b6GTEE{oAQrde;an)T7uJmSiklQ1^s9-TT zT|3AwZ$`jg?ZSn7fm7Z7cymQGlZ-M)i(tXUWIrzDj6NNv?_g}XhqhR1EJ{oLl8v5m zL?qzyZU$a3)Ob)$tO`+=WXf!87Bd<#m49+&Ca83tmW-nGNz>f~iM$;6MMLj2uuYz)ToLirwAY4m<>>ioyNsscHm zJ!yY@FrXFZo2~@eXdi&eHv!=R%IvFK!NHYvB(c(BcSTCWi*+l4-wMd*N184(uq?lT zFnQ%iUF`npd%#E5TKdYJb9%zZlpd0IDhUOxqE!Sf%7c*Ty(zIHwTX@94kz2iGkz-K zCrL{;c#QcDlkojp!l4JIOV_d3D%Up2K~N z4?$-$YR{Rp0biS<)4YRg|SQa7U8@I)3+o;H&K8gN#E$ z(S;0C03=wJ=Z7G^2{hn2PXP3>lAw|?Ux%N))3uA97J9`LngJCOUZOS%--v4JXtcE zm=E)t>C1J@>u6aK*C^8UG&`MD2Qq=`P5Ze1@dD_*m5bFNQzW`xSW_68Bt}W^_3Chm zxvb|Uv&_y4pTGK|cBeahjL>ong5mPk3iEH4N6|4j8TM(#E}R!#_)ADTE&KG$@X4q| zQ2l$>*64UKF_^a-&r*y=@+L|%m!7-p$$NKwb*C+w(6K2xu_h*;s-Rhu0j^h3bXo1a zTyQ^nY3anSlm_FLu=G(05>D@gttdE|85^pTcw>Aq3%yk1B2}~VnZjRr!dSiXXV}DXof@=OhbIkV{3wh#&8#paWq#CT zuArr5irxFhIU{-7$TWry$GV9~NOI*3gMyfq#BXijgJyu)UgtnaWbiupNuu7>=YTtX z{Wz~;hZpHrCBh$-0a_y)=p}4=(bg~U_*efUvUFZ$v%b5v;Z}7+uFZ46c{GUd^d{3a%BU(!%ur%%DLNR!IFrWhy+M33Wh^vc8$dA7Y{8aeiydsWU zmi%{?5p*y|m{{%k;aUNhYYBP9!J$KgnR~q;jx1tki?_ zJc#-%A+>JC5w3tn6$zj95W9X?#0+%j@V&JH_Tuh~kDZ8*p)&b3ObGw)pN_zVF_S|e z6OJKA6_Wb!Y33#fje&^TqRaJqLifxcS+>_)uT1CfwC?1G06cvOsPKUMaTO(@<@nLi z2}3uo4Jggx!aY7glZ)%oElqwW&Kc%&1{??DIYthJ{C{`pez0~DFb$_hqu-Vid_=sz84)@$f0Z5gxs6^>0t*QzNA#&yb@#7-yv_Udkb+^Duqo93 zJVd>sy<|p1@v|0lgsp{7<4@eJ(}q>1uC|wY3#1+JPfUvoU)US(b(*d;5yeM~^y;+K zWHRla13=04?7hwYyHaFAF!F=IhWmppN=O9-r$k*;m+>(AB13Z@KF=zMGATS^?X*RjPlwfcPL)&l-#&?!!2Fy(X>O5@6B0h;H(HR4_`9>ALsf_-I0 zHZr__V*XwU3sh4n1y94}Zo`h|{}s?rAfOQ_e}aFTT}SY#qJFvN!A?<)^W8}Lv-IDW z7y-O&Utg!PLCU{#wDEb6$z+=WQ4#&`8f^r=SRKogX}D)lw&HM?fUctig8EILzwL(o`*+B+-K=C}t3;iOZ5#@B#2<-zOCZ9$-bH>KCm4 zI}OqZN0{djh@(^w!k>fp0DhSR3fkNK#Xq$Gxcc#eh9GrwQ8XZ`^3MiD@O{411QxlA z`&;BrnZ4yv%LpE|Q6$Bqk|W)}d+&9;>*d`V3fNV#|6PL{kO{F-uTY%|+&?q@eR3yMU~2o^ z^djFk|NYpAh&#E4uFSday?W#SM9c}j|KBrl_XomTIC@5=+o>34;(ty*xN1SoAZF$Z z5cmYU_ujmB^RGZ3z@vwQi7oDD{KqbuSwX8!`TIZAhkz6_#N?sL9Y>$Gs3g&;ef!7=4K7)k~rIT3tE&H1O zJ0sEvkvk0#3F|KSSE7nfpijPefx1#k4AlLsvs6M*dytVSO+DWJ-(%F}3SB>wJOpu`A1xyM=Oe+)`ToB-zk>#^kjU+( z)45jH_p4SU73=>MNf0=nlA*!=f4?Wo15eLU>Gl~>Lg>#k>+ato+@Txa8=gRh|6L!r zK&+sfT#gxsAjm8z#rN{S@csghh7>bO3jSR(2%!D605(jT3o*e1CB0fKM!aH zEztRJ85A_Bz7UMD=#tQz-bHXI4VArw4Ke;zD(Xkx9@iHTcpjjc!mgEZq_UB4d0yBU zbST(P4mF2aJ%+JaYS+|$5fei`y`DD;Jl75yD7cqiNb`7lbFJ|&i?zy>mHX96m}{t$ z@0oEjWH8!V3 zkz~TL@v4(xi|{loCzY8Y^1cuQoT9JMmhR6$=ITL|@<(K6^f>nB4SXIa=c98qpwmZtbiA3y(-z{YsY` zHPr;vXIlh{-^3KnS!Rpt74z3Hx#A_yn8=^rIEM1hWM|pj^1uQ)|b7 z`ywoS`5P;5Tu@6G9jR5uEMXkI?#N^l_4cz+>%awCxyDu>ssM~}YYqH1sbI)T*@rUG z|EP-mo%KZQCwk)_Gx{|AcSt!*oKU(=5lR8^5s{@Tb$xjiv_5L2ytzqs2B~|-Pk&6p zudQ$K>3jS|^N*3>>kjACi9SDVjy8)79!i{YtsC2lWkdaJggI*|0+*qE;i2-ST+Cc& z>kyV#j|YD#EXp5AP<(ub`g%!Rqm9u_gQ}a`L$h;p%^D@{=Z1KXf{;P=NX6Nv*M4KU zM|yff{;T%?Tye|rQ0~u>Gno#v_*I{x=IOv>5R1(Cs`~Xs)WTW6(nmujRO&)J?>3Af zO>C?p&StD=Z#Q~7Tz_t6C27pwQB8GaK|=)N%w&!SJ=E+a&u6CaNxi*Vb|VVmSd65^ zoi{k0W-_=|xXyYSF3l=Y3O9z^juw8Ba1m#5^~{o&3f($vDPnP8XoK6}G+x(mvON3! ze^j0l=GPVn{-d__zA&G(yGKKek5_8T&sO_q3Q)A9wElwP%aLv5K@pGlq4KLT_&>wq z5M;t4Vs=s7UJ5+pdFNSL7Ue64iaWZ~+_@L&`uLQ!=+}F_Ktf30x$XzlUqK0Cu#&In z2cSNwoC(s;k6;ZyMw&nTqHNP^TIImtQtqLQeSsa-OA z;J@-i0S}K=IlT&MQdxeu34t6_`66gPnXHE;i-9597?LrdvQ&?p$P2PqYK&$qa=|BA z-?`b}p`kKV5b+Qaz#;PNlTl>`SmMx(Bh@RD15$L7MLm@UmRS43^8()FZygrz-BwLZ zxAl6Epk1Ugs)t1T@&bdJ8%(q|zpA7@Rx-Xl34>aWUKR-?B{o1!%331DU3*a6@`{%F z?QDS*sxVQK>~$vbu|LjWY<0!Ve{Z&BBq$9@Vj)lF!8Du-a-hEFf?qIetwKU;S2{WP zOSJ|)=XN3l*>tyUA%LM_s?X7k^)Wb*OyWBK8`M-_H7JXqV0 z1pR#6iT;!OyigVQ?gsFo<@9LaJ5kNv4%_mfLeX|3;fvvy#kM_KISzx0;k0`us{6t1 z3*50|_9Ej!nnWuS5ohl7`N8ZZZz2JiY|c~WR4T)Imj}FmUr^q7z^ zaHE7Myb%N*)O7H*A~9GKKR$VRN$bsmcP-b4q3%l5>9ZSy<8U%BttRfBG>-_?yLWG@8 z+$6cklr0`vUY;Mc-sIBX1&{C*Zjj47)4!{o@2;c>=Dj5vr1-M&{MT_WsXJ?a*sD=f z?w#V`5ahR-V&-mZ@OVkSCkA@ZXoPVAYUzHGTfpc1nOYb1imtREnotG4K{<^ zr_Ylf6S&E;OzNW@;;y7}rr;@gA09sNr=luBTmKL>S7U}H?ZYw}Ku2z+RZ zkyiAEi7wNJ+{5j|gN_&in2d3J9wM}Bsvz049qUEnlq}D}u)R)bZb>Z?*HN0`oPMVMq0SuI~PI=*>lx-rJA8A4T!B z$0UL;73KS7F;Ntkh4FbWT?>tf1G_lK$&}KwmZNlHmfpBm(ar9cJ*v6Mui~}cA={4P z6q7Dn#Rj=44pFm?^O~t375jt%;~Db>d~?hBDlYr!jBPe(JlOFJ?Py`Jk(6S7_>BXv z?%T=tY3{+x&O6oPJ=mS8WcDUwk&$~JBNU|Bg-&K{Wk^D4)v73?31AQc=}z={E18a4 zF*!9^%EHl}n(>-MV-gGxX}8?oG`t4Xu?Xy|3Fu|O4C(j4VB)<#0s$AjPbdsAazu}} z=JE&v*}!IL6d56mkjm+so7%;6TDo&YhY7D6^85@*CiSv@p{cg}v6hxPSmnM5< zX}fUj3Qv`26vcK>BXRE;VT);-{Ev}iPex?3729dV*%LL7(aqQJ; z$JX)zKUQy+WlKyVvx0LDF&B!n@tUf=wAA_d`E3+D8inZLYJWB$&vwkGIL=W$#O&~Q@!i^;_-~u6RmUIc`M!;j6fLez@iV`VmF#MzCr3UKSV`o3#84+5U+X&xg zS1g7XQza08R+cMaUj`-T2pTjAjmOX0k8Za3SNg9vTxG^S&h!oq&vC{mpS z4EqxeU=#$&bu(_4f95W4)TBnRm``;9LZtrHS%&kJ4__D#vp!&GLKB@-4)R1d?AG@# zhhJ!1|6G}S{L5%m7xmp&3Ye^xJ^d!;a~CFcE{ zdn+E+b!CB$eWbyXP5{}CLk0F^+H`3q%LO-VQUxmS_SJd)Hr#}+-OR*~ms6=!YPbQqVoQX*t3L@6fs4~}3ISJg0?GIP5SRWlZ%@)++ zpt{O=rqu>E3g}WCJQC7PC`sIm9W!J}uWLNhoWI zZ66}J%aui3@EZ?$fbo(%n!h^3P)+PPOtu|}>}Q?VkP8* zc~bjzCct(yPtKV({?pI2-nxL(yb7v}Z&yceJ6b&6xHaNc1#gCq$8n}ww*AbE zm!0$d?$>5SwxCEs$1}W$va1)+eh7%GZ?=%GSj;BMkTZc_B?0^{m}aY%?-sN5-M|ci z%5Wk5>f(p5lHiHinTa@#fsaaWDRzus?f{39gABv*Z+7KZvVHUH!q^H{~Amlx{A zI3-ob=h+-J7mN)ccO$Xd9g#t(#DFHH$cieK=JKp_GOB3HClemByAczJ!aUY;y-R4E zkrbQ$%euadg+_R&jOM%-dK^3>hqPMnN#y$a9tmCi$J@7&=A)zRT^!%H(Xz)32jM$B z)aR@g8fN+Q#y1;V`G)gBo_nfYkBFy2=$egSTG`$$#A|y*zHppju$4?U^2(E@Vc^}! ziq>yfzai2TC7nYa9uk zm9|SI)AmwS%^PY^$8;WNe%ayE5_&=fH@S6n6!<*;Qyv25`wuRUZWDv-aGj0h-V^cRV0cjC;1f zWm$WO$9nk$(j_#N_uv-@rUY3b+R&#pUy=ty0QErj0;{kVL_ z>Sqcu7Z>d-WB+s5Y=cKa_D-7W)$~jE>ys&m9%OcL&Eoxy!R%1W<<&*sVctWIBvZM` z>RC_4qPk6v@&+13ETD?Cggt^ETGejlPniM|ztzS|g03wp?)Pj+!!!noVz3U2O0uO} zX2WM;WDXOJk=N?&wN3gD5zA=d)lgpGNx$`MBJ*t0Zn1b>cc#a|<5 zN{bAB=E*dXqIDvL!l4=PsdI_nEav@v^lt@alUw&bE|#PZ7yx z9{2S{1oQu(cEFLxp!SX`M#TTiYv3Q*2=KIyN;3D=UMd~Yt5xh>D*vkoLn<6{x0Q z=?P*8QAu=VKN;0^NJ;pia1SExL2XQ*LHkp6YIr$&;GNh`g{EyP2dW0ohZ49;OgNi& zZgUKU1kt9!Fu@1486V$fFeidxC(bzqR0f~ZJv6Z1*B6H6f8m@_lspjdw8fT$PBz0| zj)XqoF)@;e?|<=VY=7a^Q2-w7nmwS)B#}Im0)i^-sz~L@UMG@iQ@un^KD0?#+P4@| z77{21Sa+`X1=qtnvdh19UkJ8rI`>If#HC!Xfv=-BhH@>!^9>s7JkYrO>Qp*s;??_j zLx5IqV%rySsEHt#y1?QI*QcHN%RYISxppf_hx|5gzOr6L zdJ2`nmIu3MZ|RQAK-{K6AM~X3#72TO;AsHvM|Yje^a>}`z*0zm*FJs-5f+$bg&vLA zc+3~eR{h^whuUpI%L=@Vj51g054TH7LxA6yNj0a=%lznv{Cpwd`JFXnX=6UfKKT`6 zBc2}XoTKk=U}v;^^FH?t?T4eK`yX^1P%n$C_%*GQ_vpf9`loUSq*tB2%0TfJ5SS10 z(p|j&EU^|tRsL6kCy)eWpPKRLphCLwtJ9)EuIFnSnxMK%%%Nx0K7jV~UC~im`)_9K z#Y2-;b(8w$lFteze8WOw&gLT=))X17%ug0bUYPo80hW_;CYoXkmCKPJ5}Sf6cIUO+ zJ@?=~Y5{BiFpqT-bh)`S^M+@O;qvjAp-@*JS$;4x76?fZboJlyH5@u^fB79eCf;-T ztGqWlGeYuteK*BI>07dP=I_n;zbYlPmPgn1w=FoX@wRmp#>vLmYG+VIiA+1JXRjD{ zX?`G%#mUIU=$m6l@ja{&MeFk8Oh2105M}#5G0VyRnDzH>KyQzV_07j>l%nIgq%IJ7 zmb&G#12QofRz^jnP8+TwnArSX| zNfPN<-0=-L911inCyCd&?O019u12-9CqojSEeuzMb(K5ypfSKbw+pkfb=az5|1y{R z%rmO{gSf7J-8H_|L|JyQXDf10fVR$CTO@Z2Y?=$N<|MDo7aX^~bb#%=aaMhEIG?J8 zw?d{sL;wEy%?6~nMQLX`?poYQ<=o!+EI!rxQG2wfeq8v!!BfwDZynBxWNiGjv{iP9n(7j{-r=liXNg-|lKd`w^^LDLa*L-{J$z?cahKLH za6Whq7@^!&(;-1et(QWM;GPANMuzy=&!xD3j=eU3jGXj~h2xzm?VI_+B<)IX)1mZ3 z$rW3C2#2NfH0xvNMUclI@JiyB@U8XAdsv|iYIuv0Mk&u@Xr2j?ajv4CqH!ieN*~E{ zS@=1FijiN|qJ`n5;ynvt@O1x*iIDNsxmQ(iL zAF+YoEw8_%MQ0sH&~tVUh=2Xp+ofA0q%*!_oFi{xd@e$9Gm_g{-Vr`sXOetwHS+5N zAEPP!Y#|V{z4XU)aG7T%*U+0^b#B-5(^xk}f2k>MM0-d*L- z1kbU}hZ?!>Xg}04NHQoM!raH%JRDm;onzNJF_+7GLmoM;Y7*MU8`Y9o3WxsQL6Lzk zJo7vjvC=7zqRfe_@2ec`Fem;rRN*1~Q7vmYRpe_A>X1Q9LcOs?3)_E3#v z??{b^UQsMA{3Zastly3{m~DDp`A)v|{qZJ;2kxM~=lbp=VW~uZE8|vW_&@#K6GPc% zNtC<}>OOkzGp->mwt(^<#%SKF$E%-MZU5df?Gk?n^rS9^Z+l!(mfT66#n^hlcGn)8iIj>H7oP19m)QG99z{S~Yv z<@8GZ7JFJ@&JyY6n*i}SOn12MHnUh-O(5C5pH&dB{&dl^3_NUPUsPb12K?74bjx|DStF4=| z0dr7`9G9_(`%Il{SOV?&a%h`x=`6CB(vYrAroT1%n8Vq+qhKRei76i|GuI~ZatF_7 zp^AF9L(V&0FG3s>bl?-D8B;uXD!|urCmbj!sGfh33GA=+jw3bZn&+1DYlF7M*wxFG zctiUadHxr!;m-xou&-cJe=m?n2=2JGXce3xbc{%{uF#)JS&q7vPo(Hjr=*;8kAsGlm;WA_z zyCRB}bbveXGqmE?*TOFC=!Q??+ge$z7d3d)^-49n!wBEK+sW-8H470>EFRk>8(#j} z|L_64@~^8Ggszx~PL$%bx+Q=)E=6I_;q7RwwzMpdZB!b(sgUV>s-2}YSGki=YTw+6 zqCdEPilMPR{GGEoxI>*m4 z3Kv;0>hu(Xb_IKMV@Ea?7MkklZ9q*n=xD%qSq5tqQ20;XazIX7X`9n%{1sp1ax%XXT^-4=$SC0@6 zR*w~zC3uxZ4V#ZN2xvHSHr14iA4`2@5~T2^-jhwKJb?_VBFBG}IJ$|RG&f`rCZW^y zhr(x>$2la=7}A>Gvi5MRKe(uNNT%ea*QoJ-($gm}pc+kOOZS!T9QP5>@;p(dYhVB% z9;JfZ-K~7z;Ggy-0U##bEPl{??~fGZAT2G&10Oo;){!*LE(fu9)eTgL;0CQ$js}ZM zv8lG$IZqoDni-JnhF=jbSR3x#&3?lETmGWR=D|nXK-+l&vygBo?1l2pL(}MEmZT?< zlm(kJnLU2)(80OfHM?icHx*HAzsZSiN44ShdtQ}Y-mdqE?_mTRW_;hs`~&~*?I^Mx z?~_q*FhT3RAP%$I>1k$T25&-EDRsNAPcd%Nsm)TTtM`Gw zDYCf#Di35hV{7PWHfW=QtJyfr^uSg<=7YP`0COoo!vE(7ll(>ALPlx8>#12i8hhk& z92)vz+2;C91o1_zvZlI6;Y@t4=xvNLdW6L#>Kl}tMHiJt0ZV(xJuMqW1GS5cD8AZ9 zaydRU_KjdfXSCcisE>B~mQ=C+3jIn;XxxDdnPB1AIf#7%LXPA_A|(sZ%V45 zh=jPKWfv3emg*rYydJ8?nH21Eue~&=gQ)4jq|7j+U`oZAZGRPgc!~7oD=1nhTyz5S zN+ZekSOKIE6ld6UmYM858PIG`-bUZ1jABhDStQ540>03>k!zP<7c3akC;#}65V=Cz zMBBcBi;u}NvY)lvO;yF3hb48z&%qXYR4*)}5!+kdAAxl#_{NyZI3}y-{z_%6E-ZXY z2=`Km@M8o0IU=v$`Af0)AN;vsNpPZeN5oe9y`;R=H8Bzcve)r(e}pj2yl%R!t!p)O zr(6@h4>i~(rFpX8cOa9hpE0SX(O-y&G$1t{Ign@MV=`p#*?2ZrDTF@y)dk>%GYNJO zvr~U7`%PvBWmqm?L&`xEjVTJi?+u-H{>rZ_Jd2{UG{!AEholm#X{W3QV)Lq^LJL#1 z=ZDNP&jV~-x@&Nj39^fKd)3ol)iE&lihn*ZaoQTG7U{A}+fsAWzW++5R(9z=$|!KT z0DKbg^f&$r(W^JnRIpdEhIfiU>}tjp4(k^$1k)Xjr)qkCC&sI855L+dm#!V@9rKPd zjNa?{G&5*yn0yvgM#tPq?Do_dQe+d@FVaGjj77Gpc#C6FSb+*6F&B;M>{*TOmjW`P z|H6l+S#GII-|MlJw*V3zhrami)tHrwP7uDn_c3ETiD`XVlIaT_III6}6%P*$37DSj zmf-h_GZrRMX(wfVex9=f0QXCKNH4^m$#mSFB>=rFH(OdFjN_=OVQH!g`x(JzW0`K=EQS3EqN z_|ZpaMV%lt3Oe?QfFZUe7dE~30~{-R!lxkh-1YBkA0wV&jR;03$7EMdVxJ)@l(@l( zR8{oUgwiOSo<}W~7-Px-#*^8}GWjcn8fiP#&k;oJ$yFYl-xdjSp{K6=GdkiO1&Vff zV3`10X{#B@b6NUll?3OH@^<{5TCaC*)TEj`@(Vd`RjUu1SA0h6B4gpIzsdEtTZ1v{ z-(nQ_ovQKgVe3lA3lzL>vxt>%!4@;P$o+BtEVn#MQ(9HmGbA3veO_b^Jd$)|Z0#s& z>LS6s(GeQ|x*5T1_VbGVwgzhwo3`E{HjpJNuc;XHTZqM!p883&-nR;I6pTS3+;j>$eG;hpqX1t>i@t=&TqDW1%w@UTN9 z6)b%k#a3zxAD)X4DE^c%1Rs9u&3_9PfT&PQ|D(;rh2$w7$6pIv372A9oOItEcK)(e zuXGW-hqIol!QFa}%~o75Pj7kDDPRz3I9)q^lD-zAR{JUrW;WO$vxMii$r&zm!pM9; zQ8v_|2m92pN~KM?&0_`NT^df}|C0J)zI~xh`7p^`^!%-PXjTX@Rh6jKf&O$$uMm_i z{F6TpMzss&uBx1-2kTTbQLt-Y3h>M=6NtYhbv^e{4l;4HRjcmL$`(FWZRxPcJU*RZ zvYU#GOpt>NV*DrXe$!ig)Ot(5Ab*L_E~XC}5I|bpIfpR&=cOU4f$-7Jny`gG-tGE` zUcg}95sjerMwNzFeT@u8ENywGBm54ZqJL%Nby1-oZ)T#K|Eg~VyxK++e0&Q#7@0eH zoAqiGev3MY1BHt|b39*LTS@zJl0ArsXFkY;v&T?v%a{4#xpF4d%_0v#{ahw~-G?vwoOCih)3`q$OK0gq1HTvo0$lXvN< zG=z}LVKCHmEI^D!FrDZQW|T+`^2M)E@fKuL;m}E$VV;1<^-wvhVwYYXibuzV3}r+k zTjjeRCsyP^mQ9`&c$3SQKsCIV10HjzT9!Q_&x$0lm9dNSXPjK>zS|q1hP~pnP zvrRJcaOJnh1K&AV0w*#@g&&dWJx{IfFQY!CwNT}j{bs`n0TBeKNWmdx!oIaVbaLLNUD;ff(aXLy{qWdTsfY2@lX zFzO(#RmE=GnH~L+b-bo^w%?A(7F;y;m`~sI-&C}g)XkB{Tr>cMYS27arlg9^_l>Y= z2@^kJm~dSPC%))!#YfnRjRn*j*tRRbo4?B*44P_NBR;3EwXN7KIe2wPAfN5 zv%o3jPdsPJr=M!N$Y87Qk)PJTtlhP^h~ys)d1$S^|6 zPn-_Wi~8cQ3R*|W?}jT8N>6|`$?qNyuOR7?3*>S@%a%=en&3+OkMUn{7O0`Z9A-rM zA}sErl|nT})OE81pu@T9z8M@Io~u)jiBQDK8jSBzYnS-(aFNU^AZK218@hR92vV>p zMSAVlD!jv$a&(9$3eCkk+({9Af_Ni_*`On3eVm5Kb;*&m_Xp?y)@ zmln=??f$dr3W|~TkQoD&3^)dlInG%mYOe#>9n8&oY01y&$F!rd!%_Q7U|2=Kf{k+` z9{tD9_J4JYQummCFx=-Bb?yAD@xU|vxlkz)kwOQf%R;K@;2p5$wGqs3WPDP}-* zz|^(Q4~}Q^A-cn&Q&XaPU5q#OnjA-0yO~Y!<0>60(}8bx|6tUU-cWm{M3q%alKSkv zvLJGt4x9p6aMc@hy1=8W3{IjcQ2nP>PRxEVM{xWrwsLZuLJl7JHJp9QP_m)zoR0s> z)}Hg-uYX3q&X47(Gaf#-!@IInuu6SF?P&Y4`YghX*Xr-*b0AUE6v%jd-A##TVq!ew z`cNhZBsO!aiXZsj4h9uxhc1Q~E9eh)t$(dJ#q4A%Gu zVf8iYg@wby-JMO2KES8U3%l;+&ke(NjUkGUyVkg{x7*$b_MWD1;U77CV>C1n zRe=dwl2rb&iRap+#M+NI&WkoNcM_OM&FGmBT?j0Q2jflUH5YgU;F^0S=L#IWehQv- z|0TGnm>wwmn3nVQ@Lf{;z+u>~YFS}oYmnf7_hlf8AP^WsjCq5ZmmItPhRVwF4`y7hI>{j!<^Fd6Apz?ZyeqWEp7$-~ zcKJEX@?^3!*z9@)2lW6fz5(3>d(`b4CYyDu8EkfpksNxuV51{(7Q`Qp1TSBR?&JME3v$e z##n>YO29Q6_yVCOTD@z#y#p3CU&M-HJ^Q=cya7=nhy0<(SadBQxRRglx1%;U-WSDN8Miu zyqv##`&Op_w2T$f)_=g-N*2gms-E5}kS|+HZ)#i?d$pHZ(-Q9>Q}95~!zw|ROw@WU z>6WztsxzhRB=*Dk8SaO;_1b1cm9pNYD@_<;i80~SK$MrPeOVS0)|6D-@Yd7Ntzr7m z1zdYHsJN17QxpZH>Uf}06V_izMBQ~zG8rl z^;vz?pSd{aJ@FyJ1Yq~fB_qBOCQ-N&1#+5azb?O?cFdMyg9AXU*$q*J04Q5AM$?Sq zYlrnAr}H17VUZp{x&(C;#X!Oh>BT-4!>-9IvkLI@FIB;p6?(9PzZhWcb^yS?5req8 z{aK3#<0LDf9yEKh}9%gNWB?Yy)@ln)bh>o1Zj`OGJJvl=}`W|TIz8KS( z2gppuDVE@3+o>M?7_D&majQ#l=M%RhoWyEI7V(kFT}9?aVszZp>E%yUStVD>Me@A+@A(zrq0W3m6gwj*( zEBC-|7xlKok955~rYXS33%V>n1<~zk+QQZl)~F(c|CGQ-SRsVybyBL@S-KFLqGXPH zU|6(|ss!CbYW^5qhx)gG_>cvN^oRs3`+u}fPdA)RKlqJcet$ph_IfGiy_#U5gvTWL z6pU!V0RczxK+ihxBsT$O(&J=I{G0Y zdzLu2DCR?KF52~Oksu2x4}Bc!Y%!GAdYEmAKy3E2lmCP_aCQ*ydIt~JI$MYw4{L_~ z2o}_$=iAi_g!D`M;X^Mu9$3J>C_?Q7^6Yg#(H6ZlpDu~aOOx&N%q zYp14IiR(-qaee8I^u7se>f)-70l2Wa<%!eQ^5u%lzBrK{?)c^U$_{5Zvr$e`Nt>_jQ;QWD=`gkm$Hq5q&N|ND`nt#6Hl*O8fOwz7t@}xr5|c1 zys~bcCVrTp>48+_FNhvde8(Y*zzMf+bCwk3@jAY?r&#gv~{LM8Cn zvJYke`Q$Ttii?|oCRUJh!Sv-mP+Lpl&=IZ0Zv?b>hD8Qn2GzAXZC92Te0NsOJjvfT z<^)s{?B&V27g#uZsoD?#;Qi$3Imtq*@Ln#S{&nv5;wpb`S2bZih}fFpTWTALr$;kf z@?Yk5e@_r|FG>cEt<5(@fH>k1`<+|D^la)6UQ{N>k?{r&Ab&Yx&9#xfdz779X((G; zL$4hH=QK4rBL;dbh0N+J1-E$4h+66AF2C#$5UH$HU_WAc&l|C_{T>~D)PIeiaJ{O$ zNrIxX1vIKpJKq-PC?}zlP8%#*GukCRq;?VW_Cu5eFFyYe>AlKa^Xvay@*kC+XA08E}!WAl5b~A&u)YLv!J+$VJ+hiB_{&hC< zvNkX>*x$<^1w@Ul9&=VVFt4R}`h4`bx}i7y2uK8h+!BjTl3tT^gzf3^6PuyxH)nwA zVC1zqu^1@r8@=H&lMmBQ=+$xvzg#vlIoBw#ArdwR9w!k?i_AYiN9isoH1+A92VhFO zd>iz}URBYy=}&9GdpPYu;bU2D*_XY3AlP*sN>>1^C+06)U^mFc$cK#1p#AT80}Rxk z&qy+OqhJFG&!h6YjAip&qUY-(AMI#gLwP!6k)~4-HVj5nH7s0g+UtuiC_pg;6t`@b zFp7!VqD`({yj!(krdEDxiW%|yuW zz>ta}jHTDv*?+DuV^zxJhCf%Iz&8b9%btT?KGqTyw7>0=QmUg$DcaEYsPsPWQU!O& z$Z0H^scf#dyzO>*pZWP!lOCn*4_$0DZF_ZX%_y%FC_vA1A|@@ z#s$(i1tlY}8)X(#DuXbnja7q*1mDSi35aRh`7kdtocyeX*ENfP2d?C{SF5;cB&F-n zSoxvK&MoK4(krvL7$$tVANh>W+?3&sQDA9c)k%uqJi(e@oNi^eoSgLwQmWD;>N`u} zg>G@wdo{D$cVxxwXrm$^@4sH3DDI(8`A~cVd88I(0Yu?I_#^*7zHp!kkwjb}7=xob zoTF*jC5g+=4A(?m&B6w4+;h^{M?d5hIonPhUfC~u?)ZsSG-8$fbl8;ODCrK~R` z-S3RWg761CN^ebjd+2HzJKKL)u{T|Shz_#pAb}6LzZkVt96M#f{5fTa{%H@zl=gX} z8a@WrWq0Wvb*?08Ye)~bYrxOXENk`c%$u{W^2WD*CQVXWbaiwu_GkQ?#+Cy7k2#si zr10R2-U*^bGibC5`6A1$zX=uN6e~4O^S(4x*N*3TjRuhy+}p_U<}{UCdg^53ByA0J%g8%=YX;>%X;7j`5Sg*FV?GC7J6 zbx;R@m#;u4L9(a{W$PSt${ut|y&WX?nrWJcT!v-0_H6nt52TA@>P`KndmLY{@SkZa zimpyF#~`z%nq+Za$F-$r4&%)F^j}7EWwz3GT$8hlYY`8hPDbkja9&s#7!y> zs`9tPMd&e$SFtr?<~xb5%b|NFcCRX80q|c#|0chUU*R{HFK+G@%k)Ym76d9 z_FyueN9#ryzmgy<AD-E%^aSh!YW$C4nrees+YfQ=>KnVA)c&4f>vJS_)bW7(sPZZpu zLX#{_t;lYzbg_C~Y1$7IPA)hnt7gnu=&;VY(}q{>=(%cAkJoR0g{SX%jsv^^jvy(T zMczCvhUPyGv;Sv=0IV%tJ!JQQzKAG^&xNP0T6hCyt5}a|N+jM~@gRn&@Xgd z%^x2}2li^A%!?e5-b##3F`eRedA`3b6a1bYh^zWBzSzS-W%O_cWCKOz@l||9K zE~jWW&pFJ3W)F|R{(d*r6R!=?lkIeiZWUvj#`8h#O$A+D7owfS~ zN0R;xVjFl9`m>{jopS}_t#1G!^(DLjL==fmdt()$FwCY!nafbXAW(58b6O#HdEoE% z66(-k?!0s{|AK(<%Ep)NL8Zv?;Wna z*<-|ZC@W-T%8f+WSCU-x{si*)3Y%U;*l9g3KRs}@CzZNj5t-Zam~`bHJFHNBKjiGH zXBaZr@?F*`AA|D%gh~@ z+*V*RoQX{PIn(ny75tb_;!{oNf2mv!^4oIy#xmg-rp6w%DS9M#eM^E(BBN(nDJWoL zWlVZ>OEO~oRs+ON@hJ|`wqu+eS>qQ+l?LPW7~hoDH440uG#e)iRR%o|X%`RRj&8bF zc!K13+j6=imkQiHoLrQrU3fFhPS0hgYH{JStUkIWHEbfY>Kx)yM9_Ou(&!h}n z{kiOU&8h>t@Xe%sd6=~8CurclC8Kjf!Tirw*3;G_UKC0Ct;+``PU5?XEiUkpitK1* z(mhFkKi|Y9jXdYdQaGh_(%3RMNnfqJpHP=+6pAhTdv&p?TW*U32HKb#q4ShOBUh$E zMDMm=UXbMgV}$sixCgCRfB4fiNRgxud@`c)dSO8pOGgLvAg?OFSC{+oSCVOa45jTp zr4t?PG$dwxRaJnhWY_GW4SjP9@Z%sms|OmmvOPDiejQHdDPfV z{zyo6Ukn&LEgLd%`T+h4_a%B%(VjO79;F9&lgTT5`=2ZT=$%Ov%DFamZ~K6@@TKjM zIPHu@Q@eY@*2P%W(7~c;E+d#}%$@jTVkLa5-gDNskP~>@2YbMv+T8p%Ul=jp=dLoi zMGI0~Ub`q#cmWluD+-)vj0s8Ylu`Kfzv2Z9$kmKA{z7&8D>C`$AyPHpm?38*hW^5z<`v;r{Cdhq%kuBwSFZb%Uw3; z7+NIpy=ca zXjq>$ut+lFvRf$Mk0+*GXrc{K0Y0Ov6}G;VJpPQV^RWViK`+V;Cs{H@6` zchneaaqI5-TzZ}|>HRQ@Y(j6;q2$R$8xal|+tVBd=T=b8A^_kE`<{{y)-fORAay-W znaV)G?Cuys(glfDfWBFb^c3*;OTaN2?!zDdSGvIMSBqJ*APQgeqpR;&Q|A(m-$SeB z-jrd@gG9HuvzA&Jy_$CTusS?d`c%s&cU>ogBowP-N4}7YOHxklz{34?@uA|#_oAo`r?7zJoY8I^Od7{{}Ne{5|cK1uCIC{X(Mb>f@y*| zg)sM(8Pw#Jvf_H*n}yU{zxAZ%60>b?7J~NI{f(u*DgkqCo-so0rfP!Y;vVjN?KIb$ zF)~pl)ffERNn8corf1;Dfvg=ejM-S15Oh@V5cqr0xmsM6F(Ul*ZQu8gO;)xx+vl_G zmP4-woaNcx5TYXWRz@*DXyI=3VBvRmGMD47DgyW3pDOebh*nW@>sjv7hs+G+s>b&S z%?!S`pR?Cpdm2E(GnN2@u)w>a9i#}_SV^sO{o!^RF4;pKyP3>LO5tZffn~Ft4 zpz8O}n7oe?4}~wGSf03oz|F6K3`DQCvay8Z{G5fkNtGiK*|))Ecr8M|^Uo>|u6Fa0 zbA`91FQM4VV{P+o1>Z26#lXvQ)+_pQmY#zXvL06GR3J;c4$t^G`k~ld>+G-{2TPI7 z0q-8N%BL7)({x*iN_Q%PI5Cy4Umk@jLg{(@ZmME5sP9$Z_d8gdj4+Y)*gzZ&L_Ym- zT)wkU2Wli(Rpe=8x|`Ynns}32MB5{XC6J7SnY2`u9)@5C%Pxdl@Yfy$(JIwK>4pi^ zIJH3#PvtKN^5=>j3Kz*guQzL3g{~F|;M4A@k5(wSLLUG@~;cB$z%d<<`k(!ghZIcl^SMY!Mf~a-=H3(-mB5n*<9oONF{Xy!xy6C^ASu{jfyL^g z{yiYp6L^GKj;NBJ#+xn5m^B=&O#U#%HVL|a0g0gcMr4T4VM!ZL-&q$Uuu6d0(V`XM zXV}zH^*#c}wN!v4Q8Dn^R<>%|*=@t!sQg!ZlD5(lxX=`$rwnP44{5$`0!&**_keEf zymQVp%MW<75dNp#2PuBYv)k;nJd)m6QmHX_THN}XTdZ!^QeQiWHUt)_X8w;xAHq^OR$hveJM-c#mrLCDQV@RsGjE?GLC_jl@AX~+^iRh3{l#ib)1 z-NFPYqC4DKNIjekDQ4m~2_DhdUtx4M)s7_Cb6~eT7#s3)uG--`Mq0g5=lK1BQrA)v zE{{St=Dmo`j8h-Plm+(}K#K`BS3{4Q5l`l2Q1cX3Zv_oarh(?vQ{Pnd1!$q-^mdK47&n~RzPi~EyhNz z+UN00j>v(u2y*dl(cSA2t|iQ!d{oXH7)vxBi>dw+YT?w{X& za`Rp~`h-;G6)tKx!G%<|`YWTntkm^CANtieS)~A;T45Oxe(>D!JzL(j<}6&SDU-T@d65yYJX8U4nCaf$3NFb{D}Hbtdn%i8%L@g&RKOJM z6FK4eQxLOCyVo^!`Yj58=$A-60559yyX!oXCmlzJHHBqv$3iJIOYf;{wub<+W{inU zuuD@0@fyk@g~j51w|x8~lzNhG5@h>z@+&rh6HJ>+Z+KS9>@q7gWBxXkV{FDX3}A{t ziLT~JOaSgTueL>1K!w)x+S)XrAJXPorog5Xtw1$|&N3p2bj}mDCFh@4uit$-Rpk6L zdcbiWlj)YeZnwN9rtl}?#4-MJ-m$M8g!Cvy;gdLdul~n*4()YnN@$0 z^>$>CWX&fM;M6l2#B;(DnKdo0nZ(>C-fcpL7%qO?f|3`cVk)wat2H!ZxTjKTh4Z=ccTD`_05_YQdq{mH= z=3;xaoiO>chSLq3zRGsEnD!2BenG`;(e`nMIHjhR1u7c#t%?h8?8XL0n18nql9Fb2y{JKM2yz?sL)NQb?@o*Dz&jY0cBue>G$@b9-r4WgZfThApe6bpsK0c1Q!x>=O*dRC z2RRa9N(1f03(wWGiVqWaT4*MhhVs_8iQvw9?Q-by!6g8n$qK7x0HrlCo%=-X!Tg-& zlQY4dHNq`LOn{^Xns@N@zk6lg7d%^OJ`Q6_0)_RKcdTUl8+CMJ#HM4Wue&m33|t0% z%{p!v$!WJKJ8Nr(Q+o%mdac4ll-y0vY!A8B@RQSK+w${KCz(@r^GKM8AvEx-PPMH#xhB91T2n3}b1SnnM4pnQHLh8+$`@M!dW|_>-U9Yb$vw@Ft zqi^)mRhj$5P z&o~E=(UCp;6&1UmnW2lLJ5viD#4J}F`Ps~ zc7CMV$$bALxCeV|;G$iPhWJ(E1(dGrbW>ro#M2#-+heK0K4~teE!Nt8)&D zE#<{XL_Ir}cX|!UCmF9fV=Dc903nrIY_s&-!@508X&*_Qnr4iwAgSps)G_gTxnbd` zJ8slagER7md>NA5-Kyt!{?*oHyDj!Dc!+laK3((Ys$Yb>y`@*%9njTVFNtcibgFjI zXd5hNvL9t;O~#tmUvd0mQaJqATB7s0|61xEX`_X?BWIeav0t#fsOWDH*+Swk%LJD) zGWSwAz#X6g*8JOs&kl*;;`5A?aEX)mH?1XlUOScM3A_(Tjn3mG9wDJ%)fno%dIJMb z?;3GH{WVx&rCAH)MyvqD`OwRip@XzH^;PwWrzeea^n?1JCSPMF33wFC+j_F6N)@_* zQ2HCW7->-IXGj=}61g7P7Y_4xhHr<)kElJ$#jl!<;PmRfjN!sElGiwA=cbOe)MqTG z$$St&hZ#LA*h?V?rC;ars)cpaYZx`N{Du1>iP?gJ6aF&SMeOh6yC_oS;Oxm8%g^?x zEeVizUC>A7d!4wiwfs7iGND?&qhfa@;KIVUZzDI28`yb}Ke`q`>aXVH<074R=?)aNcP#h2>IcWQ)@$GJEh#~uUTpd#?*H=v|sUyU<4Z68R1 zdD1+h0{Fq!$LG5P)o7dmrgE==lrcY1NwEVO+TXdG`h==3M8D=;N~FOXt< zfo9&uyE3?5{trqmus{25k0`D_CVN%}ZL!d^qt zs0EL4dnEs@0{->IvyIP3MVO81B|KHzF|%pX2a%+)pZEUynGRJx%-Th~{Z`|)ksXf) zuwxrMLBfR3HQjcX8f2-?oL#uT#LKDe7YQaiUP}^u>mL|R8T#5&X?LW_qtl>Aa#*vipTZ_Jo{Tl|xZ0^ZBLbD`&_rw@11!oMnW z1+@S1q{A_kJv7sfY8AMh$j1d&ZD|t|`lQk^&&dDmwRDAQ9hQ2M7-=F`C(Y!nygRI_ zljW}Z`wjVDup1vC0I}*;ONNz7`uD89QaKSo>7O472V7N}j zZOBG4`K)t6MqmzL4N30{z}~|7 z<3aj;QdGMEL(H^*&!F-kkLY3jf@*+{1@B~cs;!Ckn+=qOZ5-4@fc>yQgw<3tI-d+-hKd73VkW4($zr3Cc{=_Cg7 zbfDqjtIy9QVKu-9DF5{#wK?uV-L>>Y<%&JSRFL?=Xr`~190UC(=4J-^3w{|d27>Td z7?#-v(*Wd@IpE{=XE4}e`u)~^eEsg1&8mDObm2-XuR|kvEp6zvC+=9-V`GJC1?kX+ z7Tas^lU-?}cHlRrJ9C*XK;}DCrU^yq*cG66;P!lG9@vdsGhkB{Jke?LJ$GJN*-MQr zue+Loy|^s5KEMk4jAMaQ7j1Zjp;{uT`aCGYQut6K%Jg$ajs811#YrU#rBt(NS;BkN z(G zUeafYr?dAh_^EMuYRgG6o2|XO7oT}yY(}_^yR}DwU;AIJ7=c;lxChj_2<(MfYig{h z&+zV72=zWAl`C-#CN7apz@b`B6E(qZGyQ=A^cfm3_N9A6WkO6u zGFdjHuYDSCQtws{MmJQ!C*YhWk)BaZZ$4ah+8FY>}IF#!O zXegs!%-M=P2^cF99Hz}4yg$GQO}xt(OxW)r`JDkNiE9h` zOqxtbc_YeNoec}}#I0vWe6u8w2d1MDhly?%t#R~MMdkF$WT1{I+py#&E$KFnyibMcQ&sIVf$l!@K$nGwcmz^xlR+6wy zQ?th(U>fuxEJDO_>s!BgU>>j7=Q>bjP-^_T?ffq(PdY1r-DVrC})P zQj|uzq+`gTJA|RT5l~XP99p`&q(g>o7~*W-^PcOR|9hQJbMcYA*52z`_w#%1yVR^~ znriGdCs#8^3H3|f`rNiMidU=_Lkii>4hD-rFI38|X-|Xdew2NDe=?uv{``@T7A11t z{rX^?zNN#c=96d3&#AhjEKPMR%Q^2nV(8OjlkJh2M%lJY?~3(t2wcim-{iT1uI@YE16+*Z_(<573XZhw!!%jMMpJ#+4L}23QuTj9ux2#pZGSDMpYf?xkA}Od zt1ucI{r5ay)Zp3Js@VTl1m1jprrA}}0XJtYp(q*o|^h7l;U z5G<~#QGKw^!SlKky!P38LP|B8D}_ZG8}>9^E)*ixA&r*hqiZi%XW_;?oa;mMj8TK& z_OO$O;rzp)qd|0HWJROgcN^A|dFu+QV}i?6=>W>BJ(y7yqRGp0Emx|6p6y4&Hzok;F8iSg;~1gcXX?X%IfYz(XH2XI_+`RKkRpQI3&W}C!{=W#q{GDc z;RMJi{5^h`|cVMp>RQtD%KM9pRv4|5-)G)}1$Ru#cIkfU;$#gN^Smtrd#`nv0B*KD*wRDhu|?~%_q*BaB7=8%cyQtRU@p~b z-~xkZ{UT5Dei|uJoi1cv;WzCY$>9gZ+CfyJLs3k~7k>K<86&A~r1D{0<>>Jm0b&b{tc=&C@!g zz@JwGu3i9_#7GUhAiw!VDB`!jtcU^(`gs;N5En9%q&%JMElm3YpYMBO#Wt?B9FR--=yb%m$xl`6 zx~o>5gSzfrCPBHo_GZUxDvAuZT!s?r0Q%>DZHeX#1MwH#?82(2FS~Zy=p2JyE{hr& zWRvllFiCVC)>S~UCqE;KQngei!RANyc@><*WN`s-p+~;+)sOZnqmkGGlk!9XM9OTe z%%ps?1W?E0RC#t`;{dv@OTDhTpw>v8=BRcdOxc?WXrrHVI`x-3@cVacHJ3A0qlvD+ ziC$sgp-Q#=&ySWr3eHgL-M)SBOoiiP!g}|sOVI>pqFgibBh)$mFhBa)a4I=NqA{c9 zFJ?POxzRP^8Vpq=&lyH<4t+Cdf3FjsEL%M!T3~@UHTHgaz*3%VqXJPSko0o)=EKTn zk&DI7lm7I;#m!712nh@?Wu2IterlVLXKUbcv%%S^BIwie13dt7#AN|_t9W}w50<~t z=X+r_AXT#WPE4-ir)R{iy6M%?YIF|TAAZ1MDD?Wg{?oRLcnZTX;QZUygeXTuuYNp@ zj}dr-&+S_J!I7CYx*D?;MLwX=b<_zZsT)Bag5{ zC{_;=%6zK#ilvrPqX@QS21BEyQFiyxTHjT4J9ZkgIDp-B~ zZ1FF7&n$g(ih=ZLE<4Z#szvaHnP*P!`pRMXBn|4ZF1RiX$Ntj#N@vR2r?6;!Yr)6b zr=%A;az!ZVY^&2w`{SA=yfq{}2HQ`65(h@uq6+uwYEsXiauR$X+-`!6+Eq4CY_9r> zz1TKzE02OY)J)n>X5KY-Ky%Id76mNsUkOa+|Fb@sGKGl$uO`z$0VVa_3nTbNwdUeGTW5isn|pe`?S|C58EG zW2w{hWq{-Xx}2h^lO(~ysob^Wef`M|qGA9HAPbTUU4m=0Og!3PZ^{mGN4&WJJB>jB z6~@@k!|uiW)(V?0AMSw%>jmS?)AO>Z81M&62tm_mXLVqON;|!Db$rzGl2(Z&R;wMa zq}=EW+I6%(+Z95v!?la{MX8o0x>pMc;aH{g;!*y2;u%E;HvBaxgYEh5gw$NU<(RU} z<~^BI)F20Dgn8%x?C}5I?~D#d?O4f-ytPjYpI_yq3OD}(`9 z`$>(yF=98x%sf3n#OYz{16=n5dTR1S(=aSVcrt0gzD1f7AZ!N&x=QAIJB$Z=U(8nh zsf(WN)Ju1N3#xF@EKAGo#^HWAKdLs>ZoVFe1{ONCKcZ+L`-QyNNAO4pyB+Mw^}Ykx z_e(I<{+cHsat*+q9T?h4Ww#YOAJ7AH4O)!xXTX$967PkuW}!D&f7f1*O%{7=Tv~nH z-N@NvWh>NdaeI9u{+PaRR-T@`KQgLcnQyL(m`A-xB~hrFM()e*n|?e!@gyUEmMiAi zaCgiEJclj#mjh0Lm;Xui*VH`5Di#roLe*C82w16p}bo+Tr zJ(3=tO?!KJ_qZhWROF(KI_^!e2`k4|jh;^0fXz~|d8$(*E-B5geVUpz==&N8-7t(i zt}iHg6*}QI{7U;MYjM&~FM8X$L<3BT(;2|>e&s`5tlY<|F(=1#SCwo(Q1r%HgKo%o z?)1E(eUg9-L-CxzOKcZ@f0b+o` z=r9{#Dg4lM?&@Z9v-YR_J~1ijaj!Q~e9FT~&4dc+paBvY@0LsvD#j&Bw58bRC~BP*?NyvaZs?-#fWM4Jy{&Zj+#4ACU4xADEQ+;{!iEHEABrR`b`o^WsIJf= z8puiIrIbT`18d+R_oHmNOJ;wP>i2Y%+OYDBsMfhKWv`4D&R2(!g!-k30!!{!@tm&zsWTmaTn=K#{K8kD<0q)KxK*T`v!-2G2^@NIW zV`(o-c!zVCurgW!*4n7HkXK{r!uQ6>C7Y=puZ++fj?9_E7YheVFE97C+&CKBuR~7L z75*$PqP&v*P-u2{J~LwS+G_LrlGZDniY&HZ?m$>aNi~Ciyrh!95ecGz030euVhrLf z#wAW|X%*A$BK~2;LN!1r4ENWMiazDF7R-HC&E|*$q_&F2uwAma6vwmh%@hU)#-&`% z3m^IuY8s-Rpw1R8;15521PML2ALG9kd|5wh92xK!itp{ykgkwt6<=s4_8xZe1eVlW zc-wlvJ#eIH5WheVSDv?Tj2)O3L6lZ6W+aVv)Bdo?a13d-=@PCO$Pjd1jA3Ko0hB1Y zOA`#Jx4;E+>e0TY%3(iyMFKjZh(lsswSz)dL&fjuZ;kjqVG^{K*)2T+j=#XFeRl@m zundUSqDK1eiI$&p3r(8Q?>Dp6B%Yg~+ZqyzesP!Iy%EbVm-0j3^>{BZV$<}Xv^%?P zK;W-tKrm8_PGq3(#O6%1b3DgbQqks7783&63G#7}TneEtn^s zBjx`w$s=k=ky;Q2wkCSpdbT~W>L=aFEsZ@+@#?HQWBM4sj;MmrdOmDS`x}D3CQpwt z%vb^wW(F6M#H*DsC1Ez0BO-m(f!Gv7HNpwNq^c$&66h6|+A@DH{(A*{Hqph}$F=9H zY|M*EYDmNM%#Z3OO3lmSr=dQ;RW}CWRY3oaTD%{(VfMKD%D;KGaH5@rOYw4WBr`Q# zF^LCA?EbUl$p4Mc9%Pk~uCQ8U1turo`7_7M50X}K0N*<(-G`!^>a6wP4aSDgOfdeP zr9o`Nq1)Di$lnu-+@-F-fA5m)4GbD>H<`?$u|&DOmuj4;d;h4-)QjFF86T!tNvZC zrS3-Mc>;O)teJ~RWm3R?>s`f{wn?Jf&5_mn&)!T-ek#S@>Z&E`rRz2e+F$dd$w}9q zk1Qvv-Lb}|P93~>f1#GXPWVQ9Gv*izv{Jg7(n6m`mnB-`US%95Xl`nbr%-5uO^Fub z$7uN9KSaUeLNu)2#8|}cmEGbU{3vaKue&bWqMb*>8KvcIuAuGYazyhTmo+OQYmZ_# znYFHuD)3>36ctu(R+Gw~-G$&t%cUen@{LY2AQ=%Txp zmoJ0f>>d5S!IX>A>=cs0f-{VsFKK32G*(`AcX<`!&*z}iq!Fx3cy}QHtFnUjHAUz? z?LHxU8uHWEmxMoXcr2A+;$70s$z9W-qTHRL(&?kS&k=8{%4fc*_xz-b*8Jr*XGURF ze=oyx9EwBWNfZyD%PyyrPH!AdY0yt&mT#4krnGm-S38)5%XI5qXAE`1ffV7UnNdd+ z>4i^T`@^p5g(qKWldGI}_ZbZIV)&@4Lri#Bl?ikL828y9p>mO@p61SL6lx<+E=s$Os_#Z9pt>PsxYqvV%!Xnjl|Emu5CKaV7p``yGq9_3KeaxU}Ph%v_LG%jPDOpev-ly=Lo*FoO1fmi*% zoMNB2^;6NrCBP+EIK#)}lejvSw-`_YL`1n_)_@9Ia>AbLl{1z2p=jP!`)qBkkRlbkQi{CU4|FbA1>Sb)xyp1zlX(!LaM-iwTt9o)}STW~MPheBy z@6=0f5+W*>^m2X_le!_vGq-^oHtG^ae0wd2MIa*=aE1A_=8>-$s4MdgkyPrlA42m8 ze;`}n8GDv#i}ocNUokrLwQ=ly4F_2gt`N0M8&%$QB(UPknqu3}OybYy1t%RuSQz}| z>;hh)$zw>828iD0D^qyAb(vNzE1m57P8dd*vU;BrEeWkNt`yyI3+?+&L))T&1XNyF z^&QX_Z16^W-rrny6Jv}_+aO=-GqcjDfIDe@@p!ZBV^zbES*6!>=M%{@AqRB)>Ck8p z9N(X!yNP~`1{>#m(66X@#Z|$Zwcxi1Jp~{zV<}r}yNq-%K7(?daXC9c_Lq&Q1I#7O zTo1+|^xCflFCTk0LpGYy%OPF69^090z+j8(=`klTLjK#!xhASVw)~W0*tsO_aLLEM z>BXKlxE7Ap-!*m_h%s+aOc)5NQ!lFsrNh8r4Ij9Ty&8=4~+uvtJV~f`ey3eG0%_YC`esKv9?V5?{x`4VicD5 zMAaAdoK*3y;IMdC``rAyyI?M5jfJ z$(rboLfxeT<#xm+=h#|W;iyalk92kP1L&$$YxdH#GKplu-jWi<_00^+%l#rst4V)r zhDOWyPtopF&tY#;}qvyEw3}(gS8!D^|cPpwsE6x&!PLEll-GuIU3Wx|yUB z2xuS)>#8EdLkj~YKv`Tp5f69ILYjiNn58AXAKOZ(TJw%}q8Se)bB(a@H1^W=Ue@Fg zq~A7ScP$#+uk4g4K11ErD^>}FB8Xbz!)7{?#N?PN~<7r+cwX1Z#!Pi0TFRIzFy*r3rcRKV&u zVT-(8%9rNU1k=VGpe(hEXSx4Tc>Cuu!l0E?nEfoPP{MUuo1k8ft3XS*B=;Bv-7i4B z!u!Pl_s1MQUIjnl?e!#gDOK*NbC(n*pK^sYRO!S z1CgsGvhu!g|0l)-#B-IMRaW=K`S^Rqq~Dp3sF+!(Qa^@UyK`)L?!sm_+*+InjN@6J zUyw2KNz_O|FR8cvaHJ>NQ9vnHkXMCv-D}Imo<68qn~0ZvZ|io|(p+o&#Z0FAgPl4x zlp4n&<`c^NiGSG)V)Fb4UK#QStrz>vwed;%r;)VnuJ4VFL&T8WWj2<+f868XS5%%l zURMB?fGf>fA%7t8Jzh(dj?jD9)5Nd9a=KRgMkc|W@#Dp8hkPS)p;NU~f(F7Mogqn-M~ku2*Gf_l&6$ih30zI>${}=bv_Ia?Qnv%ObsEPS3WjaZ4X^#ZWwn43q?V z?xj0PTv?NSp7R@c23qsiex7oQjPgG2rm(xgos}3&!`~5KQAMC@xhiA1RyOEk%kEfJ zeI_imS`hrl0-;JVp!sk35w^mSbh}nBwj|OB_jP12_QY!>(6FA)ck7vvDEc?k zJHK+m9@9DKeVmY(@EMi?9GU(0ySM=V!;@nJ02FU64_mZ7b)4XXpP$K&st(1fq9qNODOU;Mp8KttdXSig zN1njGKFHU2u#{>Z2}AeHy=VAfp~YEt_%&N?&N0jOS17Y>cLM$e)Yajj3(lsP`RIJ} zWPz()lWp+6n_~LMZR#zbhnR;5g7-njEK@hrg!;KcGjMs-Cq)`C(@m0vX<#{IdBR=8 zPvJg!0Cp2y(3z*UZQ$ci>*nHnz13H-35=CNsKP6ObbNO^o7*8bOxHdZW ze0$bL2%k6_V{t4FhvrT$2rgH5wO07D%x3a9Hh@ETExi5xL~a>TGz)|GCfYQM9#rep z$FKr_))*7Q2)gn^8Bilv{4_yN$~}JI8%1!-g4IWzG)1j(Jnj7pX}!-a@XHsLu6Iov zL@Hmz@`EZOwjuZiZ~nNJrp37F3i;k@!jL(!z!WsKE4)DOBE-MP(Q8A^YmO)7;B~>y zeXi9!!U5h*Qsw$!VHmi1QY*TXL+@v4*5i>tU)5xWJLgZhYxBdIEq1|?g{v>haGG~` z{W1RZkAmM3ioWOiJSN;!Q8997v1MX+1w#fW&d*AoIKK+un8di$L?Sz_bBjcUD>JWd za0dCkx*znP7M+Qy|Gsf;pmj6%P2=1H0uP_-^Z!}M&PG9|sX($dj+ zq4i`Ba3^t`V6*`&Q^qZtMV+|0bdRO1;CDQ$aCl<{{mVpvQp5Kq1=Y#rPHhpwpyiNzsP`iz z?Mu4ZkWAx<*wk+62^*^ab9pqx7QleX*BSa}*-}!&+KA}$YTHH6Ubz$gX(R7@s~1l$ zW@&=5Qr(>rj}EvdL5sMuLA$V^Q(}sm6N?kx#At4+7&Y}9~Xdsg9jinTuWNSZJOozKZnOYDlA^fFgjj^zc& z&ueFTDw*VZ2D(c^F!>n7Xtlw)`RA$vguXhhaa}z#bjEw7dAyHgZ#z7bh99$j!&$?} zK+ZG31)x0xG|nD&c25+y$#tWw)WdbJquOjIH-& zI`C{$kp1+JT$>3acB?x>Ypa>^Oc%toGM&$r(Pi5C&t&jU@7uCg3ywdnTz^9s{b-f? zx1Q>6M39yz64>&`ZJsM^-XQFgfZ(%lIaP}FVonI#`X}(f4QWw2Vb{{g{l6AdKkMgJ z$+xczyFN4>Wu5}->_s=l)3n3 z){t`yRhFJj3X?GLhGvR`cH{=u<>zgv6XPAtMYq#Qa1*Cv8!0P6KJvav`eNxHw~ZBF za{qVvzmxX4ZZeZ?P+s++%9E@0g<9I?;PEhu&dM$1zr=%8(e+;Ed_laV{Eag!0K1fNlLKBlANDjs3aeerao|V@a2j6Ap*|AV7>7geA0l{Q6>D z#X^wppwvR2D^%5lA*n$%J;_a?Vr4wBojrTAi#Lyc@Gh%slJ^nC$R{Re#ck6%$dv`? z(Y=^2+zaDJX7B_8-O2Hk=-e1`cbN;DHNzI5Ss2{rdXHC16BE4 zW!{o(isIO7U<75jb6i?O$oVV8&2k4f{qtR6xRBWeaZ%+p@U(k@cQ4f{3(_{D;!cc6 z(D0p8DY)kW5GUkjwD5t$21}R%)Na1!Tgf_jCY7oF0ZA-)cQ#=Lct^T00!)$fjWa7p zoRG4$CvQEK;*CAg*1*$Pk8U!}hDRYX_@0^v9ygqVw`+H1i#_WbfC?QC%-s%~IUfF; zINY3Ct>_;e3pDFp2s8ugQV4taVaw@oa0-xN@$9cS88v?_D)=y~Xy#PLwK=OF{c_M2 zhld0Wdi;=)&y|4ciwqP_`pxo#PMEbPYO4Uq$hCSS=&{{8Am}SRv+%HrW}H)dY?@si z5l@C?*>nEX9$C}xyVyH;Vq&iOe}yAMYz!hwVY8YW*dVIBM|e<;t_VvH%$j8wdhb9- zxJN{v+7^1Dc7bYpz2{Jwi%&#^C6Bc%lKadH54FcXh)+BUsBQr(jQex(grQJ~lE@FO zP`O1@C9(G*m*`E5Q90|)6#}7E5*qc6KtK39=^Sxz6 zow+_hbu4)=YYJTT=-1WE+=s~Y6GkJ8oDg}S~4(T9YE->cPe zJXf8Q1nZ0?Rd}0Uj|hbb$-F1smat?EhP?6^6iX&VEh8NFaW?`T-6@MdDRtJlGZX$+ zx9ap~+1Ib-V&1wO#rx*WaSjMA98OrMXwhx5t^4x9_T!7ccb31m((zhr&7x4{q3_o- zwG>?1EXoQ_ptm8f$TJi!Hvvk;Q8NQDpO0n0WeB@mGWY;p&two)AI-l)h=cZq?+?i_(MCw>^ zkl`7^oO?+JI@~$G-bI%QWwW!LF(&|*=?9ubT-osR#>M+9@J2Y`l>m&*W!&>GL)R}B zIWpjRt%nH@?f4bn2< zi3h-Y*aT+Q>RDCcV}|K2hj?W^$0oNWNV0A%$RH>GJpUkIv5?Z+@C1HyL$dSXL)2jD zGbh)~e^ktEFR))ZcP4+0rG=F`9Y>(|%1vS(s0D&x3Dk`VP}o+d_dMHP5at!IB;-c% za7jlK2I6reB{AQ#U{ngsck>L4=Hhw8&-<#Zv`F)b+KEY3!=NfRCyAm|^@OrJZRFCC z>G!YW;)K$;t|YDcS#`O4 zO!0ar$K*rTtb}mriddix>R3Rx_&AzwK4vTP0N*E=*e1NX%o@VVn#GZM`P)ks&|=kp z*;DuMuxUcW|8(X&&So6PsB-|yo2BF&S`q5hAVx3j5|wYN?cq#zF7PdV<4W3T#&6E%y|qvLJ;|5LzRRFa%=0q&z`_IM?ThOsu+xT| z;QiJ5ql(e$pAzkwTtL1}X6t=(cd#Ho)y&oLy1uZicQbMgX*TlPYamJ2?$2K9`Lu3u zu+J+4RP3r8+Bkh3)VQP| zhc?u|es=XcBMX5?zXla(Nt2VWDL=0tb+=A%cu}D@a&cdh%^cw;oSkzo@4q>vOi=3i z$63rkjSAyp;N=!zXbg`$M!{Pfl-*Fn>I%N-XT+jqh<^Q?(6LYycABeC;}GxnI1&Fi z#6|Lc1($?6!4Q^1%+Gy^?z5uWnRY34`be0C6ROoi01Jy5A%H%`dJjjEug3^c%gck7 zS169f1sJ9deY1L7ui-SO6q|LTo@dXX_be<9 zvWHwyS@{I!!eEuzz@jY@VCD*dVIwpqOk4*Sn?-GzmXo9mr*(R>ND18rn`*#3gu;#+ z@d^-Ul&TJ{=g}c?W<4cU=$Lp8siwV7`OB}PgAFyHhI2LdRNs$qj=O`LE6H{{b2RZ7 ze`w?BnllslrwE}al3fc2YgQjWz6L@d*a4pWIVsVn!{5M7)AY|c6oes|av5*G@KSNw zd0AphvrOkrz1qn6fpC?`yN5BA8qY>|$NR&>f*96;!Uel`g|@~Jhu%DTMYHBi&%W}W zy$yH~|77Q}$%wE5!92q78hT=(QU*ko1Xw(~OuNx{>)U(oY<4$aW_mlRy7|S;z?mv3 z1} zzpI$jDTb2G&Lm^5f$^sK*FToLEfS@d0+#Cj^AI?9oITxVqn~ohp_vMYVvcA+f;+blTl|_G6M6htgc>D{ZLf-5LRYtdCn@wPb9g#`t z1Y^Q0(Q&$;1is-O@6Vn*Hu~beLdcRD!hb-;OO%=zZtYu=n%c0;z0Wu)H)gb-r*TKi&%Z(;CmG>AT9haa~` z@>L@tM>-eXt|Z}FC-!y@=`LZ8#d|?wtrY78qB4RN=-%=-7w}k>nW6`Poi8Fby?Y@)z_ug}bzciTK_R^fM2Rt1Of(Bz z$?q)TqrWEH)fvO{UG1g3arSbfOYP_lO<514_VND@S@jAdA^@ezuwfJnb6~+`Zxxed zoGzJ3m1FyXO$u(0Rg zJ~0sDRdkyiv$~d#ZRMj(t4qG-+X>j zZsZM1Qgni_K76-Y-H0G$3?QeJ#ELucng(7G9v>-h%z?rwqbvVU#br`had3-+l?eRZ zQ2EWbhJX#?$3rd!^6$>2t$&&nKa9e;#MZ|p!FM+lQlQkvDGk9Yr(nM$JprqyvEf29 zB?ZZUClF8C&22qO_T3x%4KG3l>_bGEY!3bt6QkYmlI@3Tlw3!rXAlRA?ot zZmtkX2eqs5tBA2x?+H1EBzoyNP{q=Ftzasu)GMQv`e$#b5vMF6VM2yzi%^I+FjW|E zeU=j_q*K-zM_XRs2Lv)cGw`|o8u+B*`(Hx3$|`?2xU?+$JCv_i+m_O_aEpcUndh z6B9n_@qt2+=(E~2Ox1+0$}i~AYUg)flOjd#4i)dQ2+M|_B2W6HPAc4ag-MEEBH`Z? zd<<+8=@*BE5FG8k*hpfiIr7&wheWo4b>_U&$|_QE$lPoITlk9R%}UVyINVKFHc1R9 zD*_&#m`c($FFT&hoT|pYKzw+9rAV4u6O2f+pYcmho~!})(qozd#{Is6(a6t7=)~N4iY zJ7C8VmFkJO_Vzg4QlXpnM`bUb|Iq??+V_GDGpi?SD-;_iSX^=C0A&o6c|25rSTfP0 z&<~V)t*3!PU+*kYTeG8YlnCaI<*wnw(t_I=&WjPlf4e=*JVr#`7Z*UP`#JyjZx>iQ z`QomYwh0jZ=878ipN`s9-DJajtvDFQyQnuPXYFtwQt^TZJEvfWK|!3=Wo78al;Hgk zwkT;E=I!*U9V~e-(Q*33kmYb&uD)bsEs%Ok~UEKmDk9XO#aut zwWGR@YR)qGj~0`BR-!wRkY#PlW1oBtpEz-N!qD6yU4 zU|3r3i99Zyok5^QS#b8f3fqnSf8gK|VQrIbNGtBfu7_Ajf*6N<(CS!b&_&qI>dHzfq8%^)sj~$ zVYB|qQ?POhBPA&8(OTgY_^PQimd|$w4*euoQ$ct&+8LXK|`d}Mzy_I zto1=FPqk%AH31I-(U|JJR8j$d0aOj;gV~i@Vo+-v|6ZkE5s9$aeVm{CpM2DJ6E_S* z!56b8px8#BulU#I`B7vVM@mDNViOBG^t$h%H<(|K0a4R8#NV(+<_4<)?i3ZeET;9p zy)$kIPip7ObGqmYpm#480;ewwi$*Old5}5Q@2`o(7x2$BM{z_+{q$(gH?bRcqE8Q|!ekLeM(9mqOz{w5ga?YVTOPe^-Rr^>B4~`0P9yR#o=2#{jO%-+(0eXf*MS5ZMKc$tsZm*h8_e{+ zCaPVt6Y7%We> zdh%bVj7#2o?)`C^oD0S2?Ev}~f|5i9tk}d|&WP-Tq|Y>*ZeX94sdMX!pVP#oTCcG- zx+dRMn~nBvXY>I>dX3$0UX7T)Z}@%^H`l27K@&#!FV6M|-C2{*66X8GqwGhBn zlro=VQ=2^PEL+5x!NhAm=#XthI9J$vaBDaF+5>WBnEDEdJ0~ghm810bv9UaJw!?^A zX=%&nDb-4Kh9HrB`>B#WIWq|q2xmyIw8yMURX?UMjOL(K19mQ8w-z-({%O#4UOcqEL zRN|YGaGyb4f07VfjD(_6pL~Atg~pl>ipoVFJjHP&rB-|0`C<7_6w#!VjQRJYS^0l% zqN%qiL9A6g61PTl%J*EeLn@XpGU^B)gt2q%N^wxX!RxTAlO;@VddF&?P8+<4-Y}fu zM~R_=N?H8wg33(mrEYd?=25+|-m0c@BnkxJ(5(TfiY0`L5TrjI0=Qqho|+|1em=xp z#$T9uP6S%^L6%K)f9_-8DT>N?Zeeigmdb4rlFm+VD8?KIKa(s7B8f{c&u6&6(0iHZ5Zj(ax}ats@^>CI&|4?y4Yo^_<50r zjQW@fh@mR=Ib-H8n;J|Eh95o%CEDzXS=h>nm1@Ri7o7qw-U;L5iMR)bzlRDZW|BiP zry1;k|QWR$?H!zW+0!oOJ=^p#umh00Epu3+edbv1Z24Q4&j1j$n0{75S`*I$4}dKs__8c=*;BmYFe;s?gKIa zHp@{N1G=SFij>bq{IA=)E=_&4Q};KYkkZ6a(j?Tm$O0P8J~?|wGB9EN`(+qLN3l`8 z4kv;%lEMrJDp`YU$KzHGy#!wi9X)ua_jWW_(J^~uwY2`J${~OiC}pp^O_P%3X({1S zO=;>m2B#;XN1E(UI?i+;hT%e?J{TCLc+cgSc&$A#x=7PV(0}kAu&jnYXMB|?2k`j% z$5k#`b_cnnfy8ItW<@XKD#d=9iik0h*3Bkg0R$x5z%XUR4}P9%%4cc#|6E*4jJU4C zLnaZM9y~sWpNA6E+ec~y+WH`oguvqQJ#j4buciJmxW?U>B~;GhNg`Rdc%Wr=r%7~~ zlpX~do34cA#Im;yHi~FM!4m*7$HNFRN62cjV#RRjk*bd}6LDdm6;a`*HO3m@(g=mY-u z<3DU=1(A8xoBo;yjt`D#C5e~2Q~wIt{+BU8jk;<)q1e6n?sPdqL;8N5{^6Mxh_b?y zkXB&SOgRgfUObu2j1e|Ao3IrY0F0hU4aBDzg>kv<=RJAd=r-8oW=gf{4}avJ(O-p1 zmi9_7SmuAB{UML{qn}5^Vw-(^*z$9HSF!p&l<&oD0A-qhV;*hmVK}Gxg=!mCiP-u_ zkKwo2@!aQ>Lfqa0*!c!b*6bf$kT>iMeoZ(Xib&mGHv>Y+WURX+)fn;HH2iiPy`v&P zD7@+2_cF7-`j3yj#oxzdzk7I|+q!v#)pz5S6~%UTnS3t19}^^M`{U3U+9wK;q+Cs_ z$u0mvLtZ97S`rqMZdClm7W~(4n{{z?*SrqUbDu#x-X3{d$5NQx<|hzxgl4C+ zB)Oy**9F{Cd>`Qk_yu8%Vfzn99Xp$PM0QtwN;dD98CmKhqk%0ue;-UxC5P^iPq(wr z_8xUDCXI-kMop$9>2{fPNNSHdB-P9}nYFu){{n{x-$6bHr?Ze&=rfzFf zvv6-@f1K)fa&j{5d1x*(Pss;u!ECN7xuT!Ky3|wl9Qbv^taK|BlsrO#Mhl)fwlUek-#ok~YCNESzEN)?#GPLKlmVqzKV~N4 zWe9Jq=pL7lh%hVBa(#h!V80>RqBG+0j+-GGXcMrQk>kwE0>eJM7gxo~P5D#zXPNX#J-UebvfhR#|w$p$-TFW+dJ5?g79MKo06bE9Eg07t= zPO>Gj4`hp2XZzv;2XA?4^4!yiF(X%lrcdKcyDYoq_Ma4Ya^df4ns7lE=-NMerWb%d zSwO7N5!HBK#G@Gr<9V9%DxXpZ84a7T+Kc)ON*3fo0X~#9Bn{N^CvJ-EtF6`YDOlZ# z+VwwK(SOhFJtIM5d0pFXH^`8QZv7=EMKwi{=05NoYvkIo^N{C4UO^6h)>t^C<=j1K=dN> zWRf8kgk;T{{Yc6mY_Jqe;Noo?0THgtY@;jmKgL;~zTvFnr;oM)EOxF`e;u&<;!W28 zKn|YwC3CoJl*)VX-CXbBYo`C`J^y>$aE0P+8^3s{Izd`(J168-3-75aw;=!RI>?$SC2}O}UzviSGC<16|#K zS~j|VnN3apwc>8urz8eu(!7@}DG>c>;25t~`(g4=-R9qaS<|ADV2K5wp}v!4M$hTl zW21Nc)n0EN+7&p0c`I(HIN(>n^^!SOh&#w%M&%V9Tw@n)aJH+INL82m#}hB`3Sr(V zKHkbe!3k5&%f%2RE)KB;x6okWFcXG0n|89~Jye=?%@DkXe1{@LbVfF~HyD8<^b8Dv za{s`;8s6Ryq8Mhv)GX36IezhY_r4*BBc;oz>Up=5dA)*C76>JN~;hs6m5@ z=bHL_hvs!B4y2}COkR$jJ4`|ZYgtT~g$~N6WjKEl3t}Ow*;CoIls0(ZL&0mTuzT%* zAx15c%0bghFl4ZI7slu)38zfnXQxMF8Lnx1Qqoz6C|NF!x3tf_>8e98-^SgAg$V3* zP1^t3?_?;v<+i2p7BAMX+JpPxZYp0qg{PRf7~&p(u-6lLT;-lblo|dX63D;dFf~*R zW1dIcuuEb>7aR}?uC^%_)^S>9fl3noO)GJpqZrJA{gDSLAT=cEPj;Zo7cB<=Rb~uhn>Ill z>LCgnZssvrYH_y0J?@Ptb8A2&dc>s8tfa*k$ECA4&)!nM=oBdexvb`-A>^yZ&et*+ zs=BB8IZwqdDoD6nsV8nL0Ma>%(wj$SUH)N(xU6pV`IiLN;+Q5cB2zK z@2O(8aO?b3iwy#zH8^L-Zdfu5Kz}#(XtnkKyRlzHp(Km)U)AoV|c6=J|ZxEY3 z^uArIJZK;i2Qgi0e^e9Nolwl3QG7F&?ygR!>$PB@82FOmPsd^#*>%{V9r^gm($$c; z&IYd33`7?V)Xscj*L*ltcq2vNj)d9*%^&RUf;fgFprkUDiQuD<$_RFO^(c6nQ~QBf zIxHx#{IJL0)Jw7ct0}6s8*zM0aj4iXKTAR23G$Y)a1?QQ)IM8F&ko(kpHVl&T8qTm z&dHB0oNWJ89e7Gc#>~qek)dz#9~H;HfDNKA;%PZ+4Ei*CPJRSDD?t2*6C5DHw0^Nc zBuztO+H$x_SgDUu^od?;iaE!Mi0t`%(lH-#*GSF@`+?pwdTM6-Qa_E2_4}PE_i-;&-W17B+gV1VU`a2j~max;KHx|u>iHR`Cb#n{x`gP7O* zAE)`hTOj^Zk|=>oYIVeYkMPnghSHv%ZVfq$9rwQu76NSO{G5+6-x`h*U`{azzi~oq zqAh5>KX~PFHYV%gVU&HgNz<#|L&0?y#H0MJXtlw#kAy-upAQn)g*HGQq1#D(Y0`le zpE2TryCpxQ_e(no<&Y>m5t0cip~VgYb{=s_eyw7i@8r|4ii0)Uh0H_QkQ5GuSw_?% z;?SdRdR1uZXf7XcA{H1m>4_PHu}%ICi`j}1(xFEIEyu8jC3l>hdU;&lQ-usz_nrN} zTM*A58$*2Z&4K9F7<0GMQzx^tE;Ma# zkghN~ z6PafkW$L$FlkNZaOBWICC)3uh?a4sc0sCt(cF8R-ZW@b8n}fv0qm|(ZZMfhH;jwL#xyk2B}EtmNdBv^ua1kVTid22L|_0#!jTzTB_u=%=^Bwx6zL%b z1Sx6hhCymzNF^0S1*B6-LWY#?5Tr}G!Ef=r=e*}U=jHsqzZmw9VfO63_FC({?kimU zobHxNwrt9arEk8sdc!}_I&SA`Ko@TN`vMyx9HM->K>9*T z6&j2>wL!>fIaOmotafYJq}Q!8KP~9!%1vvXxR`G3CW|$^kPAQ2+}%_X`%*9ZKeL1G zxE&F9D^-0)@uSl_TOh15$M{2W@sl|x%*!S6f;iN~CBJyRtXW;TbIi^ImreIDPO!wg&A+s zX&H~^lza1is3l%9HGFC70<_WJ-ojH%Kp+&@DASQ6ivBK7{KrzBuS05x7vX!%Dfqn= zcUAF0C>_tWbR_`?0Qy8{PR=1NK_H=TJL47CTpyoNX$q;N=Tb3OXV$x zQ(Qk*9{c%nVe08t>K`2P&*5NwRT7sRsM<3hjdOHrj`HY`2~dodSkB|xf|t5nQ1=8^v zY`_t_HGn2&--F=1V&x-MwY+(hv6l0lSJ!$%!+7@r)e(FEmTeVLtOMY@b}OY+COb&^`9*wOGRX9>Z9UM z+}r%(H}gt?x8ho0R|xf5V&n>}fZ&?@_QDi;)(`(keUVye0*N&G_&zEWfQt|(9Zx?> zQsK#Z!xquRezl)&N@3Th2MQ&Pf<+mpNRV(AL1wHfZ@nrS7gm5BBzx{ssJ2;RJ#r(t z)LgGle{wsNb-wIRp9s>Cv$|F(Yl2?XUq zFUYKa&i)b&O4A47fp^G<)!xQgv9KzHy{sGxW4hZJmFQ5b!=+r)fqmEFD_(eAO>9-j z707Z?)pIoK4+jb9muqDwr<+@M9cC-2Qe%K$^%E)Q z>z>=)7rc)#?ISV@&wX$dyLU;QH(3zokXCrhCWjc;CoHZ`PCZ#ejP_n4GPV z{bi+~H$}ktn7(4+UFgiGtd!`p@YZ$4KVWpMI__FHV2VdwW%n4~g1y98vh2W30qIDQ z6?)gBhBET`JnL7vx;bPdn1YL-^H2Sx>(|m8;+ox9#wHApW`vV}FJ4r_o|+BHE_axo z>w{}=y&MkbnhJIrR74BVg$rduK{>PJ!i^tdFYQ6kKeruuYwcYl$8bCwW&D9sQt`tP zdaD3lgmlFV4Gx>K;V*Fn?Z7j7H#y(up4z05{NxYhS@IPVW7q znsl?oC6B{lsUxQ5>z_`+;1REalGY>0PE&xe9OcZtce64b)yC=A`xS$MRyCy1PUIFg)hricw;20SOX z*4}ike#O;(snHM~dvX72Td$fV&lz{NZS%NU@^s2kYQJ=((DWM}WzBj$`Pwzj_}Y_7 zF=*6qg~`=}TI>&OYPKpC&jTQ)nG`W&`94PfxeQ&jZdzhO1$vW`mSvY77JX6eo zC(f@xYfy_fz z0shIk`-9>z=#rze|H&uvtn$Ic^vJj+$XwU^4f_a@B|7d*X`i~~$ezK7XqCkJ151D+{%IU@n0o8Nm`GrKvQHu}lt zdIMk_XI_V!<;f>%fGTLK(yr{id|I~)$u&Z{ zUB2df1oOV{%}7MAL>>27U8(5#@}C9^|16{l@y!tZH(RimB|%O{(EuO~mo;1d0C9_n z#pZ=6D}}2z0&AiCyk9p!zteu+Wz_m^8T01;%ZmLEd#=z}hnZW_;~&6O@=i=WJt_Jub`aD$9GiHQST+h_C{f7onP=E{=J&^Wif^sf?PI z(>1&{;*+b=n^5oL4Vw)ZCQEDkIdCSr;9zNN@DLR?eGJmY*wIwOX&Cmc?TX)NSa}JU z0S+jpr_qlVV6vYN=0Ge_Hjr%nbaEjr)Pz5cpSAm<8~uaZiNt}4*R9VEtCHFBZf9(2 zaPm5DBZHlZjViAG0=enPR|nMhoW;VOxp8Pzh=_{X{hcSlMhosU&dCDjc>j-HH*8--~0E@`9DFhiv%7g7(&N( zcQWJk7!6gs;Rt_XifGY_XC<~GJCqh(o=}F$=IqTg_CN4T=LKrvID=}pmP5eHjg(-X zFC=32*q4rn|0TBoE7!aMv*gmp_o!@D-4+sNZ`e`FxFVY*6b1K63Wl}WWq}Ab!ZOK+ z5hA}x7hRU%omb}~&#YC*Zw0Z=#j(5AQ-v-*$<^uaXd+|M@|mxu{{8EHpZTMvreONs zLKH%x$YELysSqW&n8Sn0tsex-3%A(s;?TFOj=|MloNt9jGal02Kx>d1=>Hyh_jY^O zo{vFt%0a}c4$k|YHZQOi%n_TwzvlrkqneHLbx~@d+ktk628KEVFNHElPdw6Vam8@Q zc`SABkJw{jtD5vGez)>7>N>B+i-d%*m;R6MN$L`gPgFSJ4Lmn!ur!*MhZDAd*&w-&nJk1w|aGtZ(nAEKz?YD*W_wZQqY?8Q;5B9*Bt z4((YYFS^|Ams5Yo`6{Nc0e(L<#2)Z7^SAt;eMqND-0gL&*i4OYQza~Q*m|Dg&?ypl zz+8Z#EM%e<6lg46td9LkJdEa|HLmxQ)bgyBZRi-Q(6y-@jtW>)HuiVre(ojo18Om5 z)kNeIvUQBR^oe^DKQHK4$vP|5Jq5F4aebSrWl=>{=cPV`+uqu$L2NLTw{m-_NLGJ$ zl+0V);4JsCj77vCv)K=E7Ph3@a#E0;5rOF|01X37Ic5l-4Fs%DlWQ68vm$4%H#=uoa*$gADB47~5d1S?(k|RMj zsF8NHbu57L_Q#J;S~^xARh4w#VhlVVhFE{A_%pwO68q@ALd*UjM|q(c#hWM|v`lI^(9M(c0 zUr~5u=a$-US+igvgv+TCDIAfJp6YAvGZXc#bGI+bP$j-|cRuk6j(O4j373=u{RkAjtL>507LtLRD z{$stJ%c>!LjirHR_>C5Xl$=m{f-(B7qJtax@JIKz1XK9ez*F&j@5TMXyVQ5dSi`ma z?qe>u6C44%7gHd_E1U{qdT?g@a^I+Qb@~l?I&X>PptHM;l;fl3dCSH&+mUsO@8oT zq`UQp7t1hb1&rpc7E))Iv%P7^_xcenGbl&I5;_Sa!+w&ajh?>pewm`)I7{o^F%}#K zJ$LroZ-T5-%q;h3bB+rSl}bsmkx^ImP_WQW8|pa^ApJW;2b&9p7p4z6hL1qiKC1T)B3DOBMLN}wLqoZX?Eqds-g|kab`qpafec?$Y z5n7i>{mNzYCS`)_1*QSho@dNI_?_X70yKSw3ME9I(-G%*EE3 z(mqauF5ZtIOZ$g-G?rcRp1oEUdGZaam!evBFSCf>L89+i`~^>!!u8_mO4)g?$MI}e zx^S88M-*H!37%DCPHTxybhX#+B}*0*5DdU1pL^Z7ZW2h)5USqGA0Xa@$*xzHnJM;8 zEiIh!&2#V`>T2mdA9ot$iMh)Vsi2-An6dsq?3A_l@^Ey?`~cQT`EHDpyU>#shJErV zD2z^tk&d8azb;TitF-t*q17AJP0?C|F^>S9VS2TWE7_WyCnNJE%=nlm{h~{G;=xOI z$@q)C(RnpfNeZufR6nMo;gqs9FR#dp!t^2)QE?gyE&6ubx zsX{5aPLuJV8p7C-73##%PA@r6@PH7wn2hPK`>fK~r;>%O zM$H4fMCCu9Xg*07^|4f%30%5oCG8={zS@Z5^}DwNw88@&mwrVD1q(?jSjtch zQKb&jZx>n^_^Ozh|0f~A{oLa|=~NAFet{w_r8j?XY5tQ3eg&V0mD9{ng`hJ&O0LHz zgWg|cqmg!8MocUA_=optk?sZ0QQrIg-@Lf4M|)Zzw_7~9Ovv5XC}aZ29Pjw~l?o-% zyDaSJ#xc7yllhm+@^7R77oV;49Nu}Ha4Ze}gb5NVm7Ye(!uNzvw$z-nkN4Aj_GlQH zHG);%NRjn;Xu_~z(6;IDrW%rFjdNQ*3SAUL_3OLAmZC0WME;}e;MD(Xlc1ZLl%fYg zY90{(!9UOgQ2#|l_Qo9aSDao9#!99NcIif_iw zVKr0<{QH$(DK;ZT0mWj2R(QPb1#?cl4q)bxJ+@FBBKjI*1h3)f6CEOT5@V%?uH}9!5(!=L zynb(=gub@V@m+`kl~l@AbL+dMsa*n}7zB59#EjZ?#EUa`l)0aAM~prqZPi5SkfG#_5vRdC{QJaTX9=$e8#JuKKLzXVG%M`5|+e~_5upH+4U zO$#TVOc{MY^PL+{kVB=|42D(Cy}sTNylq6?3nwfpYaqTZ&W!a5uig*m>>WgxiTImZ zdfj{t?rT$gG1lbXcNpiNeNhx?Wd*rBBs{%${>HrhNB!{eHo>TMS@KusmiEVr7Y_vj z(gMESMmJzoxh%;_W&AxUw@V(fE}3s&Zg=At;yM&}r^JEd&GuriX?p_y0XBpOcNHBe zZaT&oBAa6|p!gcIun~2y{CX)&f{8yMM9}cr>y*P91*zEzQFvg&=^#O+}f|}RTy9LKXS%_F*|BC>Cyy?$6-@=zuGQqTD>D-s-?^;>R zz+0*OWZL|~;VQ2tJ!*W^E*aT3P~3%~I9m?uO-l2|Nb^$y(3R@$@1VJ@q+oD`6cFBzK{f!Tg`iZom**-|4cd2(%B{ry*4v|( zjR*ALu^?veJr_3sk41Z_$*z8nx&7@V`L0K0TK*?pQp~5uo`Fgl7-B{vQe>C5kmsi- ztK~$@or4<`yJXYe{-U1)`5;PD9)o0b@BA} zjsyuZ{@`ipiu9|YCJSDT#}MgD9;HL`Tef({IUng0wI9u_h@QJ3+p|hdvo6cjd{||= z^y`f=#jY&j$>zS_7rr?Vme{8Mr(Bn}e1@Q6qaxe1D@|G*DG&LP{ev=gqIoYS0xH&; z9(?`g;-GUjn2(pASwPtWe5mC-`yDQHzmK|HkU@qB<7OcdHYVkUB<64BJ-5}qz$B*b z3RySR!>&`Bt_CjYS$V!oQFtuC!AwTgXR^|`i!dj7!&j<{MA9%(?d0zb8&?>wk&N1E z9tBY;k@{2f)y6(+w+@Z6vqqWWs~-KMG4b~w`5z!sbkCyq>wfCAvB>BZh^DV_4w0D( zgIKkdhbiLPyA$rV6*Ehrvjyo?Eto-MELK$%aR!qM+-?}$ilmB`oP6c?FqMBinNavF z2wr@vU2yFvnq7HjT97Zch4~obVI8B5VT}5QC-nstbs2plfYr*JjoA8(%Yq*z;@s-K zzmP#ctXph)grvjAXjCC!cRDRt?$1??ZvCMrtq^5}pYVL*4^LWPd z)|(hA-<+Ilm%hM9d>BMW@Suau(Nys0FXA;GGYx&8G%|s9ia=(|PlGR-jFtljxSWBv zXZ^@|F#ut2_cUu)3-iEvHqQa%Fz%|st8X1qArCckP!h|*BKCVGF0I+jwKGj2Y=H{9 z5KBWc8jessqlb(k{`ZX96lc1E=*X?4l+a%Lza9t(W~a=QSbr5}#dLBgLfl*y? zE3nXSp=$*+i0(%ORP9zVW&A<@fxVKyM#O$0fIT2@KyBaI?`YzSzS!PtS70PkY)%2kuHXt=^mwrOd_(&A8?VAG$MBh z=T&Z!5L7*1Y;RY;FTI%7&@PX(kVXeB(T0fTQq*}}s(8TC`>dAQ;p60kQuYqjO9!o9 z_0LIWjSot^$edwyIL>0)da=10ew)<1*Vh5h;~j4JKb<81=j&gQ`I+Z^lKqvkvH5LY zSXe9M^V4Z&j|O^@j+zxt%C0-n5u%I?ct|!`kD4?=ud%B`EfXiZFOSZUV<}bRr<;J? z-xBIlHI1$Uu48tMoR_m6CbIN?P;nR(vL5MM&GJ4d>ieiEyEO1gcdp;VMW77+DJ_bM zTUUZfgA*=`Qb*(PB`^?pQZZJqDQTyHp+F^5)~8o-{q)6-&kl`s=U|GVV%sUJ6r6y@ zEnj{!X)troYEohvA}phwkN_W5}f33mJ>9TJ=PqHJP!pwUzkkJ)WGA95R})h>R2 zE3FSwc-YT;?XCd<|60igkfZ6W)&B8vSyPKYd32r8Efpx?sMf@kI)Oz@i^S0iE6^TA zoTYrMy~-xG)~7RH7S4qZFH(LPg0Mmo84r94rAek6U~^aIbcc1oQ?RmKzzT(g1e+q0 zU(pku931jaf22;#Z)#V`^+4#pksOj89xC0a)&dla0*sK@xx}?@(%Z6})e!X_JTj$8 zABIXexB&DEmVj?iPw08l|6)j59Ck#U;^nsjTuFF;*{=*0*jzwX@zwxkyz;!6ZfkDV z1in+nOtdlz?0eP(Z-4X7@i_&wa75^b3 zJ&2tZ(bi{mu0D_D1mtxFMC+nUVDVYfEcXO#MPWx$&O&SYMe1=3f@Vs< zTvT6^gVH`NvOVjdf3EUbD?j(|@vmj9!oUpP{9Eg+WMt~C{;Yu;MfM`wb3am%V5Ca7 z>19fWe+-O>vkocO27=j?!Q+fJs(QBMScJ@_N)ABIm{7~HNwMi|=?Ne$-|Wj$9}oXD z$uF9o@JOPx48@G$zU%awTF~0)H|q@sH7(Yf=Ey~Dt9R=+JUgCJWBF#eG2$n2waqp#b8-^`46 zYh)YjhZz;Po*6Vt&fhkxwrp%>`c7QIta9|l`PDy?5Fcghdt6s{HNdPL z4=UP(vzmrU1Zj0S`Y;ui-b3$AEwGvc_3;dEqJ(E9V4&h7l8vh=mqHnuwS}22>fJvW z$@v8ZkO}S^GVe_a2>3SI$|~u>mqmGUZq-y0ov3^It6Gi9zwO1z4=95ajk%0u^|yfk zKxwi&Y;+(iIfti6dVF9--MmgkooQOwm^{@O>keI*My&tK1#_yR!d(elrYb zmD+5eek1Z&gH=?C(Og@3w!0gB0Z-akn(zYQf%-t4LKmj}+E!<|%-jbme<2IZm8(ud zetsZ4CbGPwu=^QKH5zw+CBNaKwk}MdWH73z^?CDv4N-I`{!_MJI@z{duC7Z-3NuZR z;Popk6c&m{{Qj#dy)E%PYK(_C#W=~i=A@wmJ|m4II7=y&5=~+)N|R&yjsPuJ!npq4~s^Jj0x$|1S*u$2)?ozY3oEKJ?WpprDr%QuveTf}5kV&;U52t_1gH%ao4 z7t0K}GfYQUhPjUUMUwsZpnK7J?do>Hh9vKz>Ve(G*P%(+notzkQSFm~l3X)NN0oEV z{5H`5BTq;mw)|-a*h|;6 zRwsyfM2SAv|E9Vo%f+$qg*Sp9)rF3qw9msvo*wI$JNiB38B@aya@2Yf<&&!0uRR<) z+CKh>h*oXO2>ayF^kUMo=c+N~4*SO&=@LN0fPNSgo|krX=$nvWNV3`euKP>!R`5la z+hx+FCw%>#;q7d#rRO56T9%NXdhRV1gHi!I3*8OXl?m6`u7)3ST7jG2yxeOl#_U2WtS((&f4p8)dI+ql1}=ywZ2T9`bPiwg6UX z^gknR{;K&6b7f(neJBh}4whLIr(6~>o~bz+%Mfl+NUQA^YM&upJ=R)dfuOM^cMvTZ z(NxZkIiO5t49|X067;n7nwnQgxAYh$1B&rNXbTke#qw#F&bWF6QMQ&YhlpjUU8(W+ z@CiVtK%}vaQRUU3Uk3!oU-@ncv!(S8zd}Hdj%$6TG{4^&sY{I%Vb|jGRsXp5eIa1U z#Y&8f8k0}k6LRv6m4m|!5mZs}i~y}c`d#hrio(t2P%_`(PkGvc_ruMChA%csw_5f3@qLI;m~ zqx+PfH-2zh{R)>NDpXQQlJ%6iFFP<7o!FMKH&y+r}e zJ7_Mta;x>NU9uGu0$IQ?SO?>{rF%mW+-x}1!@DFI~3H!lYHS@IA<`6l<9p@!Qxj&N5Nn5Y+JJ0ku(CS~)E zQR<95Qa%v;njMLEghBc@qP7*xXvjv-h2{P+aUOpr)pMWP;bj%HTpM9qj9mH77`}60 z9l9agkatmmItlBc2h~U-HncwY)&L;Ninq?W=kG}VDn_KbA$8fmFfYQ7$*q9h1M#+y z7(6+l-D-&ILx7SF&FgI4QseJcAFLS?D?U))^*(oJ_pgKMaH7we3pc;w&RdiIH|7aE z1HN~{e4-L9>0dgW%RK(KI_ZjR9j^WLl-aYt`$zttRiqBMisRbvTq>_P7t8qv)^_&M zui=I$v!<6t+2>At|2k+$!D@(_V6c^N;sN;m0dD&)cR js{ME@=bZOI&=-g;6C~qfA+)97WbhF3s&Zds4E_EeJk=t) literal 0 HcmV?d00001 diff --git a/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-code-locations.png b/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-code-locations.png new file mode 100644 index 0000000000000000000000000000000000000000..558b74df3d24493894a51757b8cd2a1655b0d678 GIT binary patch literal 72075 zcmbq)Wmp}{(k>7Pmf#ZH-GaNjySoQ>cXxMpcXxLP?(V@Mc<{T}=X~GU?0fF7yO{N? zW@dW2ySlsTt+#4IWTb?@LZL$e0Rep#6%mjF0s@T!0s{7i_zXBhn8s%f1O&Be!p|=w z%FmB0V{2t(Vr~coBodO41g;RLisHTbR>TZJ=#NiyLo7n#9Y6JnR0LcWQWQ7GKN&9d zYfKM3CNw#IF)15q?$#m1unHV>BX5l^PBbsCDmb)?%rcHj@6Xfa^*ZK*Umgd)I@LE^ zF48f8Nbxj62(b!5qVXnT1Hql?^>=qtnh$`!DgXhq08vm&6YB;A1ON#XE_RI`aso+~ zH&qbpy*ypKDJndGpuqx>0&f#`h9~)0W8_x3Aef*473T%7ev4iB6psOc$4=gg zCwryq5seEklzJ2he8M#c1AnZnFnL5_Yx2 zS$V;610m@~K0j&nQVUYHLp4U^aPAoF3g}(-O=O|kWV&VhCsBn4CF~kWU0)f-X7GKD z-QAHKOE@%SyLd3SIO7(zr`-ZirC&l>LOWD@hj|1q)>0n9^zI}iRLgUjI=dIG21&%W zX-2M;_G0sc$ezKDfT7wb+4*XiplSjWor5SBOKl4Cb>H3oDM78s&Z$U)I zjUen9Niyulk&VQK%K58GFkS6{N<=0bbEcUX>l=f2_djK{B3ztHcZvu5WYrkrbGC5; z11V+I5TSjVs)qwIoem#Y@n3ry1FBEB+?zQi!*Jb!{uIoEHswP+1s2%-MFp6B3v|sJ zRn7(u0xSj>hVnB9E+`rglx??dH9C@?Ts5Tmr-W?u(@)?%@YY`#{N%SVn?Wmm(zd9b zL0x&coxuWoNWb~vLV(NSqUS>%@eW248GzT|!Vbei`Sj+nkiv`k^haZ=!BqH&<j@R8hN{SIx^*JDlC4k6`})car!-ws^q zZMM~Cjpz>3`FS(IZA1Qz|Zg}kxA3+|OT2VQ}UjuV=>S+_dDSrV+ z3={3y(+Q>BPgeU0`BU(xrUbVUQrG}pb)kx}IW7}STBv5AX4HCsMSr7iTeZg$JSI$0 zNL_!+E}xBt21@0(rcX}aoKT#g7o(nnNw#l)J34#0b7+Or^e64YZclDmZEJ50K`8jr z_9VcGZ6RJkU_d@WR`_H1d-i^l4fr8uPa=dMA5s>A-2Zt;a7TZK9U;Vk#})xk5rmIg^>`(!Ee3!ZX^0lNGqz>`d@kQ}bVhluP zM3D&ECJAy0a&a?M1&gDghWdt$S9<&I`?mYLqs;L^#h|5-^FgBpmHA4t(sFGj1mq0J z>IpIlUlMM~gA^)@zZCB%NGiN2Jd_HSJ}FF=-A&6d@TbB?2&?n2O1Fue6zLxo+QV=r zXo+$2UF5@-m6Wm;3zqGdDwbp@o#Vo7OP=~k&@;mZl!3Ezp) z$)a8Oul~tpjw1aDW(?sp`?MFPD04*%KXYgcb&Jgv)uqeR($n)L`O}v(<>jWOE}Lq5 zIF?h#<_i(K^n>Pwn)S=FxVpHxk{*+(p&Cy;-xb2m+{cW0b*E_Mw09mO2h4CM{UZ^oUd?mqk+n+m&7 zJh}C?n2<8WG3vT&yi9b>`u)1KdtBFBG>*EDv8BGVdQ^0by03h6KeySOZ=V1+D?aAIH+%T&L+?=-A@Tr6M`6A7*zQ)H*^g|Okn+l-vQY8D&57U`|iWe z!*~vEJYp-+jcBsT(oD5BiVmtjRk7M{jx{EC7iVEmDm~HuoZxAZi{XfvsNA2qd$MQK zqtlSn4%0Y#?)sy8u6hUkmBAe-MaU$fW+6iZ!b6LLQIt38%@(>|zWRJNy&ZyuVYaB6 zs^twV9+yW%qvFltktBKu`UtQ@p4)-8X3Yem>G>I=eH=I&jvEx6s9qFaGF#N7i0vYd z($wPEA~w>?v2>(2NQuxv0$jOb(`j?vbH{VACOIbHGlR4GGZ^&0(L&Hj21<7rE(0Xi zf3E$c4k^f2nW>v|IBG2Do6#|;_!3JM+h8yq!+X@aCh?ll&b&&{M*gHPWj}GLk)UC7 ze#;To?%+am^>Xc{5vVn&zR;=R^>Do3L9s+}SLv@rTw$(l*j)NU(TkU-r)R!@-lWWn zs-BFJ!o7>wNYX4uN|sl4(+PNU9+d)FJgjZ8wW8t``=#UXX~N;|#IgKIsh6fj%~W1Z z>(P+Y>SZ3s+^hJHt~>u#;6#KJhLcsVC4yCJ)3V)l4_!j-b8RQpbt8j?p(WMwqvvX@ zh4LuYXjv0t(^%8Jg1Gzcylc6Lr-Tig)i_*c2752rm=jyPolLgDF^{?}6^9A!+ z_rZ5-n;6wiC4>2e?l_Lxcbx}MdG}7w!;dwUN@b1pcPGG?KAC>o;52iri}P4BT=g~B zl}U@R*itO+*TEMtiJ!{?J0ijdXY#g+pAW~f5WHHQDF3KD*(_WJW<_{yxcEBe=0lo} znxR{wFx}VpH`r%OBr0=H}G#;G%m4#&zSN}CboON-r zGyi4!Vd>h5nzDJyh4#gi|&9gqCBM`*G+=LY%Uya#|IurdjK`^GknKWLM^) z{jquLwtL=w;n?^>Ws9b=_?dFIpe(iSXglSf*c8z+o4ROW0&pkW;Mm&)pI!KQr!Nd|P>>`m?%w8GdO4sXUs22__TV zJ2C>)&guOV(gtYC5`;&Cg;84?is3d(C$`&brv%MiE*+@Yguf6~fFHi%E*0H`3=GI0 z5Az62$IUI^Ror0Y&QZQbWPJFC92eIsOiHZvFCh0bY#bajY@9c>Fr+8A6n&whUyqPE z*q?QzDt{saG5Bm*sDlNu<2V_Q0p34=^0WxTL92D;KSj} z1}It>+Uw#vTbNtgu{m?%|8)l&p!|NC1|RpYTkOp^@l_;caQUrl4RM*Mzf*t5=Yqn; z#pSRyFk+Jv5c+p@z<-?hKkV(T*=T5-oSdkg=&7x2jcI6ESy^em)6vk;Q339tvU9Pt z*L9|{v?KV}L;mME0)}?_wkFp0CRUcX@6XlMvvRQK#K(Vc=zl-|>ZhTz$$wh1wEOq4 z00X3Xzd}Px{hj8&RkJrS`hQe=zw)nYe~s&3&2hXxj7`SG+0a~7zyzQn>;PTkV))L$ z@mDkd*OmWt^j|dSZk zflb=h1kjxByAZi(IcWZ`bN{}dgXUerf0OvXrt`0}fcfNt;-LB8wZR32)bdmT1jGX* zD!{AY41Aga?i{Cx5x6-csV=0R{j>Id0|OYI7arJ<6jv%bXG=f8_P_ssRQ2|j1%ZnS&+P&G z?^S``8;1R_tpZN|e>Q9>RJ8^2&mrx7_vvuEM?bZh%;fw$d2lA z|Ca>s1iFbdk=#<@et~=xsE!w?y&#Te4hbP)|CWKn$+vQuZfSJH61DmtF#+M0nh^4; zzqddkym_$Uyq)90qiF%E}oEIQ5R zhHLFY!;{@hXibiU9o%Q-Tt8t$$p8xY^1ev@`RVylowljomGk)b6*POP2*&doH><_h zjpmXeKObLI@lDf?CsQcF@UV}%*sZ^O5Eqy~5VX)iEdM~?nwA>3n<24}FIe6=fvaqT z(XW9V5Z^KC#1_vxQ#xYenqW{f!nS5fDMm=5SBWgH^&N-({3ZR%Y#5d4 zN*r{i!Li-}>+wvi`QapM^m4a-Cjn<65H8C0`OFuF_wPHS+Y_?d`bPU1K9$xWc3~zM zXKq8|kI#z~Iz7F(cK~6qh!Jg$FC@rpq2z3+5OHY?f}_2rO2z=6Y|B!rS{LJD?e4kh zut&lW(_U{Xsvbf=f!R67vG3njg17f4M3yupF|V@?9{M16B4uFlytR%L*xcR2$7Kq9 zA%%)HEYdtDerk)cM5?@Jdxw#@0R3ol_`}UEuS6DsR;X0mjjF|}q0vC6Af0cBTxZonU(4T+^y{DIS(=tNxlNr}F zXF+g(mRo1mab|6FJmNx7fO#q!y3gMQ3k1Oe>hp{g`E9C(3cNnO-NU2)snC%xN-O{E zbxCyE={%)0v1-xtX^-S_(;JfbAUZRXxT>fiKe;>!1NA|-cJ;T`N~J1~oN8S?NsT-% z$6o@*3?I}HkaA@$Z#e*I9$sXUCan`4-YvZg77f0y+yi)`Lw-4&AS|4PgxpZK| zjpv@=B28obrR|Vb`|CP*NBIUi?^v*X_P79 zsPICZjbzcHS5D#z_gaRDAw*?qtwO?{ZUs?Msl=r94h~7hu?Dk8kl4}e1hY@M`fVgS zA}o3_F<{`qLXKWfl$qHGF-x)G&|Y_S6u2q1HTBMV#SkAhCN4maEgg%rP+xw^%?axB zJGqO1Ht(3H?~N1$@GqHbwzi!F*`KGUp`cZ+a(^yElhxxI)# zN@ruK_pE~Wh9#t_I&b~i%7?FNnqzC+#TNMrNFT+#%r(D_ZMtz9}M_w{IEeLZCOY*iq#D@%^3o@;43uv6ib9Z=O z_f(g{Vi9AHs;rc!jMq2wUd-NRHN`qlhoGP(h6Tt}AC%Y<8MBt**pFDi$ z|Gvn?K-qwCEEvo=mGLtv;NfSlpIMnoX&9xr^Xb zzQn5CSdvV3JzRs|Y^Tx~y2les*Uv1>uP-;4QwXky=X335l1N*@x{S=wDsp4G=ps0uX(94TmkFlDeT;d=jiR z<=mM@`-l!ZOmBtulIhX$i1=es1=l$jAL5&cDvs(pFggnZFv=GNAVVz(e}bUS3?*qT zZJt`9SG0bDvYI^L4;FKY+vm1Uc~rEvEwytVUcIyY)u^TZ5bbrBeNUS$S_7&@<_S2% zFwM-k%1BM8Kxm${*cR>TW-gegpue-E*Cs|uQdNjiZN8ut3w^Y7A+ltZ+n7-wxKwbm zk-@VMiB;M$>&bKOOPW3&Q9n^EUX75GhW-=mxrrX<1Ijg{l3=EEq;z!nLwA?GH_|Hc zd9qZ+qhsu%i#rTVr^~Ba>h%iqjZg=w3E!eqfetZ6FpgH|29nKfBU`--^a(-QG^-%L zW*Gs5u#}B>g_b`;bW6_pQ_oB!F1D%|)a?f-*6jn4?8?H@Y*}S5dvay#bOj4FRavTW z7OGI=X-F<^Ys1q?Zb=lVHAy5PI7QE<@$Py zt9WKur-QU5r+>A6X+L)`YE2ql9XYy6xO>Eszrevtm|HO-*JZ{hCQkuLKnzRuD?u>z4z) zF()7pqcno>lfQ3I+$|l@xjy`%mZ>JTor1L%SNY0fm#W4TrNMJWD;l({qxAEhjNk0N zFQw_{C6rH#6B-2>;*D}`o>(R3<4`LnjgccZ$+6f55%4$t7##uZ%A^(#Hk^!}?O+L2 zSmjt_OEBu+C2<&HEVbIH%d9ssy3{2}8gUAkJzcdw*S;EW3!!BdheBX(9A5rfsRZ04 zq|I@Q_w~s#94;|;-vtH&+w+lpU_Gp$w-<`6S&OK2y|S~OZU-uaFaBx@(%xgcTea0@ zJN3)=POmeqqlKZ{WW}bKjyecQ_wn&QzgPqWm!xyZ%kv=FI)m_1;eg*mHaX=~&za5D zZ{-c+GMhe-hfpJdov@hBCdsV{pnqP=2%(-kRa)&!1~|K~hV>Nwp}Qjmv(@TWK9iJF zD!Yd!JGRYO{Y;q^Qzk3(Qgcfu^i`ulN?B&eVO~0*jH__eHIgnOYx4H2 zaLH`iK#=047?+nzVH)2hn^?l>Gzn5KsNwK(j7GHiaD6~ExHyocy592j807n=CX~L7 zFZD4jpHhhgTO1RL&6f%yo>aKq0+5Wgk~!j^LFn4dmg$J^M=CWiou3>JzjFOlpvGb$ ziJFy@r_W$^goD5P?kK|?101P&_ZW^rNZ)Gl>RDl)h+7`h3>r^`*Yr%F7Od>`RxO3C zY$B)WmgyJaReNdnXfTU}-epsZl)IFAbeQN@D`l&fTaPLC73%46c)fTvW@hxqja~#1 zBD5+V#an?iwnva>R&(;XR$E?b@SWw%fb~i@O)!1?EmEcT3pW;i70yPzwEE->mZRo& zE(XW)Ok?puGw7EAF1xjPonHW)+xe*(_O`7z_7$K&_#hoVDiC1ELoUKYsjdfk3&X?) z2fkC)KNC(!E}UfpVU`{nT^_2t$NOiy{3X3FVSajL<@)X!`d}8C)x1!ow`@#1@+qNI z`nKQ>I3}b|Ey$%I%>A@scU3P@uZtJ-k0-0hdlY(54(>VXg*D+=0{+|@2e;R$x6NV6 zd>ev{xl&?sg)0nR>Q9u+boLepjaGEi2pIv z3vZm}b~A83m+vON{OUf8*3xhnB(`cHK1gIX#P{+{l-&>9T(*DVQQb5>@8r#hPp5~y zysFvn{y|2#7@+aE+1M~&90qucgejeqP&&AyU@X>1v_}Ju?-JDGDc2?+-G5^?9It0u zro$HLtHVOl@)HzNJ{SOBiX%q?D-@VWKAz>_6eLW|UVN2FqO+(#i_=_V;q=(?1HGat zf;U!@cueqo`Xg-Ok;i7aR6u5PnuqVIBz4^Sdbd&0c7@b7Z?hJahT2C)M4PgN6;%bV#QKh06>4@i|V7F=AseIgP0nY3gUJD z%&Y7q7lg`{rVbAM0OeJx_an@f&4b5GZGliU6C-S}aN*$=`UiW>@(uVl%Rv7!a?Un1 z=_;vYck-N76tB!leyK!;EWVmZ`eh;l;FLzV%?`Ghp7#GY}sr%h44K~j9W zpB%l;8Ivlj`5m~?JQ?Wz8ex8d8T)}i(EHaWSC!xp8NqyQmcy5Pje3WAIX`XJc@ni~Hjzdqm9m;{+sMYAa zJ6SNb#j*KZ1(foI+PYj(w~>{X=bqiWVwMs<1`m55)a3^F12%W$!v&BFr73ei6odW@ z1k}aY3^K0I=-cCzQdfc}ennF4iY=DQJEog&FGVC+TWox356MJ-35SabfF9!*j6beu#X_qy9du?*X7D}7; zg4wnxv$R@~1zjD~(rXcd2eD9TLXVSNC72R(N=Em#Xj4wP&$1yZ8Axg#iHBMhTD-_WqQam%SS7|3eU zZq)@D9$XR#9*$AS(8Et38Mp3pNF=HPNEMH3zUK8w#wL;<)Wk}bQBkjUNk9TUYx^f( zI!@H`Rd;%!E{D<1*%m%BJ-9lFJoAH$1{u)_Z9f+r?w}f*8QIto%V9yG(1GPuS#MQ& zIA%zE^*?x{)*v9QZ;Fn-`+LI(qhaL+1Afm(S2bg-Ggk~YU2_|Eu@@P5TX`e&B2E7_P&Oz7Six{ z-s8B(q5%Fo)97Okzts81N{!hKOmMw5t-^Ea+g~#6ZUKncw(==t{_iB4xA!<`uas7d zh22aA1=|jG#-`WcFV-A9KnrJ}@c}7koY_PWKL9m**?~V!jW@OcKx=6D62jvkCg}@I zygJ1KbYbZEW=EF3$I6Us_&4sGB|-Jn+^-NUn83{hY-?0wNs#IU)O;MHN$UQ*9|4LJ z#8Rr^k>wwYGykHSym1?GUfMsa4A-fG3v@hA`w^;9Kz>0c(2R?|5rw2beHXB|+ zMd$LGCe`qgvf0_$&49H^ijbrbqNy5s@BB!r$AKbF91qn&)G`hsu3-og7UA$uBR>*O z8qe9I6Cr9+kOSfX3-d&P>~?NI2z}mBB7pi~tEKhto$wD~64)15sQ;{)8k5#mX36OH zbOvsp%QR`9-L`0%rV-RL*y0>ZzpBY@moPIkQ!Ehyg;FuFWIu7nDd^?V=9_x+L)2$q z485$7Q@cU+YuK#48HO%6C{MQugvz_{ZBthx13rw$dmoesX$ha6dNf66N4DPsBEMr7>S?C9?vrNXT@r@!8VCX_0qiYZeGd$RrzG%|Nj2M30;_5|?o z>2KKj0Ux$!bV&CKn%B(c^`hydCX0O}`CJKQlp>HKiY-bfv$T4VZc?HB(=pY`)voJG zgFzZx&_44EA}5?8s?Dmr&Ry+njDx<0UfDP4Ac(P41p9~I zf-6_EuBp#Iix>1>-`K*^9c)PwCi58VLr9VT*|%n`zrs;_+ff!ubz5)wP??^ z_od_h3H0jfHn(I_Iv!*JoPrB=foI(JWd=b-`nLJ~Dsq}P_n5VP z7tb0yVwcqFZ}Jw^VDCCg9uyZlCD+gFgHX2&Kqt;w$pQ!(@m#nx zjdgzjm3QOYbJN1r(Qu-l9LEZSG6ANdnQYvnCquF?c~VY7T+&oy7V~uJi1<-K34wvz zYtMq(EV!qrEHgntfl>vN!_nY?SwC$7z#a2hx$*E1O`_)I?kd~9I~-dtIhiUtEG(|G zI>QF=E{3yCPqQ%DP4bmW)krfO?(P2H9Hl2ZMDG)Od&xzhbg_FQ;K1J1V68aNj;teO zF6RfEyaf3k#7WH<>Fl~2W~&-^=U0Qd>{$rFAjN-<X zWd|_WX0SCSTrjsxn+ zcssvG3=tU;rKwz-@}iO3@*8IT4q0?->iI%N_>H@WDx6iOR+{d(mnXL+p9IT`!T|gZ zvP=7;*I0q|T)D8B>+5w=0m~5T>z^ysmCF+paxG#RD5X$lb=ynv1WN)}HA+ky497J2 zM6pq}QkTgEg*FS1d7{Sm)u$87W1v<)a^hYSy^LP`5Bt5v0GjJ>_$3JnO>gKHqaxpvKHn&YMua9D{0VBZezCo_P8iZH zsv4#WBhz5XzMifCC*$PQ#I}PwjeEn)8I35N5%!Wq0Ip|;#4Ds@^r%ISP-m2-_);T? zm}D=znb>=+OHm5_G-+g_B`*9!7q!lZ?*DVRO0yf4n^3+*8f8Y=fC z*Dz>xJezDQ>iax~M0HH@&_%}!$9VfD{q10Ab^MGC0Ou98`J2%JVYQ~Bo?)>~{7tKGc1qmc}+uV+wWb>RBan)ycoHf|AlM45P4`tRAC zn*n(ok+6KzaWlDSh`H0Vq8gT!CetMUrQbD$0Dl!a_34q}aXUiwYV92rXPX)}^whBq z+igaHWB6yLrsGd;c3|fq$A3{`u%XL#his+LBRfh|vpAMr^^6*JNGMZ`arOExqgA=l zo`~cT8b4={seeKc8E1cH-keSoru)K$hR=$PSQBOR^&#}L*9$r4qny)-@@8%@yI~Wv z*6w&1^WHbEf#>43YJle=6(31spGP-8ChK8yA@h#DyX8QfuB|!2tv)5*w;anWLfg_l9PPMUj3udE!BnqWAU2+~|bsQ!LF);9;GFAZB%8 zD@)wmdB;?yqDkdVo-fI4!Oi@qAcK|Wn9xTZiB+@xBwX15QdeFz#*?er(r{a7o=0u5 z{q8J{4h3$~M|0Xe3^hh|o*JtYw?EawolPDP_9zg+`-Y5|D67%#PT$sJSZe+&J22Vq zfvu-D(d&Vy<&1k9taPQG*a0PX^5lAh7?viYP18)*r#}~c2*=g%QKV(kZg1wg!q{Xg zsY*;dzb7sTUY**eJ%4lXjc|QH1<8yW#HLQAe?g<&lbR~alB5a187Hn9BBIKcs$^N} zhEX;u$t5baF8(}2g-@AJcH$j$+0N8PpHJji>t@LL!YR zqEI@E`hXD-HOjlzUo0ydqKTE3)h0tf`fO}ev*!bmU7?3TA#*)mgm{m|}A1Hl~|UAk&3hC>N%< zqCV!&5vCc%-KOksz?n)>X&)P>(;p^08g$wZ9c=gx2nKGth-Z7fz1AB~V!P^ZtH_}t zT|0jMHVzMwzUdWc1XU*Ln3eI=`O7J&#--0<{2SUkK>1r{0SQqzZ3v5eN=`3dY^8O- z0}E+GiyR$G@SpkGq`_99rp`{DKK%IePgLOTMrMWM_|^=@itKF%{*5jv=)LHsQ~K!B zqy&+%cex4d)c9&m2$9VtukAN&vmFz(aV#tiHNEG0gQHaN`Dqo&Dx^C*Bzi<09jEe- zZX2d6G$6@!;_2j=ZlUxu4IY(sYynj7hG&n>CwPt&%AWBR3;dFF(F*wU-@5ykk`eY^ z{}m{KM!eqKD-|$$?1L<32w(g2RzGUdO0$)E4XDhxRz(lE;{It;w(9=#aQB)GE$ncY zMm7^f3uomgI}l(v(q;gbKwfi;O7Q!8K+}62-WvJb0vSW0rp-l_9&bX&>CH0S8q7tl z`Lsano&otnqzk7pkj}6nUz?jG>Z@Vq*vRX&@+%@s*M&JTMb~o{ywO%M;L)td8GY>^ zd=wzi(?}rj)>pfkDqAh5z6+V`V88wXgMHkV0oeCJSqp#+H~tEC`fB(2!~DQii-Q38 zqQV1`PRavE>ldvR2CIlQSVwC!@;3hnQ8c3No?2O+ZgdbQ`8-LbAZG&vSn7X7hrPWW zKv{nLX;wrmd1s1s50YB(5@Q@7WHA6LX#%3#Vtm39qVUrPY6N%d52%W9KhX`Py z!h?Sj{>x7kCI%WK5bktKX8(W@4&fjmA*Wf%Roa@UH?ucMs0)*WzqP401DqHLVMjaW z@9=OyKBw=A8d$DBpg*>xg9;jHz9J0(@C9{l(n|yG-y#1J2=P0JKC;xK`hfRjcp%rB zoi!^tSn&poK-`p1^RqV6%>z7xtkd5J)7?*1~?8ZR#cM zy+!8vKp*PiRBH#4-QpdWIX33k_u1->gv8Zp#M;D^`1%j+!r=q(sG+#A$SGP~TvDxc z@`YSoQ`-?dAL$;DZVSMtYxaJfOZD8AMHe*%#VE0SeIMe4G{DEa-qbIL`|z;z)Na3I zma|8I0y4d)T889y2%yI45{OwmpK9SOGKjkflp3G2ACA`rShO*T1I(AL=U4j3PSBq? zpgg#+kN~EGf9cyl(u=z3Kp<^G_Kx;>(`$Zeh>D8MC_$ELD79HxlDv9yc)XmFsFs2$ zlJPgJ%oPAEl6(s~aun_??)6 zVj1`k{paac0%`Nwl$vh3kZs!L92wXL^1!c`b_%?#|7ms z7hnYor1)E7nyQi<580KirbDoVNvRkk(4EY9OGZ+0mjN6}iA03$itF0_bl3fK0%FpB z^1v9V+-hgW@OYo%f{lY+0l8%9*qvhP0#c=w^3iD~;!UqIe~HGKjs^E4JNV*R4tM8c zgdPE7O^g`i<&QtGR%VOY+Bie+3t;Q;eLp^8^*?{ukKGfH)xmW)vjMh=62tNQJz@z@VyqiJki@2sNW~{e^@8dYE{HvZr zJL5N_;j&lFu4OA~x}j8lo~~w08EY;ITKc+V$3}z!0D|CCm$M?22sVD?^r7G`_TB9& z3K=%hAc^VZ(Z#9S9W2$`sj*9|=9?4}ykn5m1(%e1r?#-iP*=7#*1!6X1R8BIPG(7f z@{zo}6W{VQs83I?>e43O4wlAVt5(($Z^ohU&wy6jE)KUwPOHAfFuj%lNBiP-9d&Zl zIqZ9P?RIVOvY`(?mB#JGvo6FTsqVFcJ2|ix60@qH8*}qgoFd0!%AHML^Zj;tp``!( z2l(Pe!Q1N^^~uqjwcX;g6(`wc4o13V#(%sa=`MC}bO#IKvg$M}sCS*fLIJFT&3ckC z(E5>Zl!C}7CK^itU93nq5dvgf2p#ZGz=etV?2Atad^)IKZ2RgK`s@c%JlV`^f4sR{ z|M1kHK-WxI!zee|lvxnz*E-rt`RDoW5~RgT?Pka(oEnFdhAJ4~Cx}dXGKgsHb}x8{Rtwvr3(a>h_ zdo2N;=fS8Qr9ur*gYJZwOZih@RdW_cc$f=rler{=!xrkzMeSMtf&|GMD_;VJVnNhv zs=#Tj5+Fux`e;Cb-eiieXsRIDL~;6V!Mx?+z4v3}*QF>wf7q+1QQpeB(yq_9nRK4h zsGNhM_q2v!3KfEqCLaAtBKMZ-32_HJ%arG9?cb!xLPqRg%)i;ln7&=5X0Ld;KZLL6a8zR1c_YTdUQbCOyhc7#(&Lw^&7yZ4 zAKG#6{n-A7iA*i0vOpupM_{XobZgdRCW2v96@B1$0xEB=5RR1bM*u*2ms||_?K3{V z+jt*e9-u^Fux)Dnh!3hedJ;7A+p_rQcq4eyVKX+_qdJldVzSvuCqM0t_H4Gcw!6;hEs<$==V7C$e#SxsXGVeRU(l{N{fGBCR z+JyK9RNpL_aT;h-VuB@7@bVvo6E{kcBdGXq{BfbQ0Xp+2do9T5GM|nmQ!FT7+k6>< zL0@%cIs4-fhLm;eWXab#5GDeAGP4a`Z#TkZfzi;%+e1(#EF%^>cq^GPgz`Qcry1Tv z=MmK$wfAko4}Z@q@%D~`+iKh(qWGo^7 z5~FB}DrUaoU9XCF4!Hjq-pi?1>le79!4LtUZgI)XAClC1l*+a_0ANA5z66Mt21iRyNv-*?jj z&V}{uW+DBz=Wv$m)sQn`0~t$)jJiGt%$slXCvyTBy?@D&lDK;rGYZ0N?Cui|ft5B* z*y_HbV6d=@Ao}h%dzBj$CS)zy#B&HP^aFIA5u5fD%(+Gf zQ2>EfQ&W^vCna~bj7{7w9|v6sIO+%e1O$2&`W~m)lfk2KTJCU;SC+uk#U%Q66e?)C zL5UuL&57O{10o01bbqlyGEqCMinRG=2a*7n=vqohLL@&oE)ojSB{|)6F01G6izK}y zSdW_!J^yBsbdnMCBJ0*G<%o;DJQC5(@KK%1LR2IBgoTAksAk34QEM0F{`7TxSlUlN zMe#_%?cRx`8i?2Ff8k}Q4Jfs1B%|zyo2YDd8jw=ETh=>f*rVOKh#vk85gO4>gUXq6 z;+BHKt$V@@;Dn=(((AC-+OPyyQ7vYuC&!q*D5L5Tzt?`c9ESLo{tsyUT|(!jiQ;Ht zo;|-dAFa~xMHMAq==BdZ5ecFC<~OP#Qz;j!mz~$Tz#_P7F52 zvB%PeV$lSQ`Rv;HF}W7C9wkWOk`fbp1AT~q3`oRWCRtYaq;lRQqm){PD{tCfkmI}k z6NOwoGII1{o0$15kzPO0Jl;FSbwc>Kaek7qJT|Kn#{uQ85s8S>j)&w1Cd{lFW375-a@w=1g?^7KOHosG9Ls>L5WyiY5mjv*^I?m1eiYke8VrDfqZR~gV>6c zRM5qjw9nF>5n`XgWyNgCEmE5TnLL}+{$iLJ4$E@(w)d$gZ3^oK^t*~ENT_AWXqzTz zobrBgu9_}(=gu?+q#gb)gll!VY&D;KK5Cq+v>GxW;9{?~j5@UA1|+)$>7Fc z6RrS;5&=l&#)ECiENLzM+Smd}Vx?QHxt*lO+cJML@k8<>ht@qzK(mU~_Ri8U-y3hM zKWVCX57Xa_6q2a{DN6^AK2rW}hJ+?E8;iK!2B4m|BL4jE4u4~{Z|1$~Lh6#SG`f^q z`X#q!)88BUW~`tdEWK$);q4)F1JUg-C~xbEiSae zMVS)?IsUKxauM==J2?vmO9{Zdtkj)IgcCtd6y_28cQKcoYqgmNK7DU8RFzgD zHhUZ&8_%cisUiIp+ky}Ig!`x5VdC~{0zQcGDn53`lBYoAvfC+;>GdgrZI@6P+uZqs z@X=j?S-sbpC4dI3KF_HelHqwk=ryH_RKXP!n`<6A*S`L)!l;hONn46aVEux{6A=|9<2^-jj9Z&D<_uQe6nz(t)NS?W;#OE6%G-jyu9Uco{B{sw7+wykHh{4l4S(Px*_J{|~D zUe&{6sV8P zo%B#)GvC>^u^{^O>z0lz?Oh`$%>@YIWvnvX`pJ_Ejn~W!CJVL9-1Ry+h-N+(a``-t zuUT)6Oqo+LUH9TnHID}vvjl!Padv>1Nrotg7%Cq_5`%6_f57?%KxrhKHkO*nYUCWU z-y5_6Sn1J<$8Gug?+I1vDZjdCQ5@WS*A5=6GQ#Jv3m^#WTjJ0IOPC|RzR#~9A zzJt6X#*t=KYCOx5fTqqN1|jl~zgG&+@#Qai29MuK_JGW`S_KXH=H+|J$=JSCOy&GOaNmMo~Yp}(M=;&d+ISWUN8E;pasAqCttRQUIXnuV;Akx}Q2 zqq&m;Lo3r#b50sF$3nbPRp=UPF7U)xR^*13hxCbUWK+XUHt@xh)7roXhKM0(_rFjf zqOfWl+y4NhEu-HzjPadXkZTMm0K%a7PKbKD!F9D6(5#Z1QK_O;lWt*v{E@0swDrM( z{t>eyw4FH1fwW$#@VXWUj8_Il6OD*h#7qArwc^^Piz_i$3!T$d{OdGwW*V)yofv7_ zarcL({OGd5h@YDMgfmnC-}q*75@WyQs=?2JCS`5&_c}UT>uq#-ABxPg9{h;%_nilr z6pp!be}YWPeuB6rI|fahu@V*KWc8Q; z{CkBGS;n0UMHF8r>J0%+t&1vo>uYU{=?-LOCmwLgnshPS0gMqIpnKOsTTlAmHm~K8rTBHF!kE?0<1GWP`acT8Z#A^ko6g#p&oE4L=bej&pZYp46GRAh=yrHEL&A znh@s|b;$-uco8)a6wgOM7Mq%nQ$F0A(5Dk$iOX}9?}&`3_xQHD?NXj{*Ji&7T`CvI z2IH9y5`dxkz?Kl~7h;p&$HY#_i%!0M1GPV5@EKGd4h5v8XXb%(E&?Nu$w-V>| zC!u@z4qOy6+$A@F z;5xW8fOB%y2ia#~uo*K{yl&d2U`a*Di>nAZICs*|n{4uA(6w+jo<$=?E+N=1tu+ld zIsa<5xxX-_Y{u`!4pk*)|G66m7j^ZGCGgW-^vlaVO0H~=JB`kKAY`?7J_Y_I zKUo*x1uUV7yKKW`p1a9f9*^VrK`4@|M@`-31h<`Ozho71N|5$aJrnekce>^|hQ7X} z@}cX2z{{C=j2~+VmbGS=lxT}mz46Dfesl)#rGY|rflw+Op*h+E@NLGOdlUXS~R3XOvE=hK8l$f zCx}`$2k92;&DZg>a1(a?PK+gG?!D)mgq!ABHRDtsjT36hN~7$V3driQok zP2?+%nNW>vjT$riJR0=n1tkVetpu^y^m0*h6jIT=gLtE8uEnA*XGvMLxBdN3@`3u3 z&6syihj)HdZALS5afEG`BMap7UtA>sEgcYN(_r#4*q%Q{%o|1lL5pbES41aLYo>Nc zWi&V7g$bc}@ilU@QRI?T7D6%*3R;$# zS|hHco+^U^TKN5qq|@47OG3cQ8nK#HnRnn{QEOBv$y#@9PZRdDUL;0BLXaHJDv=_a zqXux?r6pS%2$eD9;??Pb(ku1F1a@lT57}LBey}q5=cjjmik#hC|Ds&2gSSwuC-QyU zjC1ptLc27RQ9V+B;dEp{D2;=zIt;=bi-+B7RS1v5aU`kqXCi-8=UjxRt8pY75yyc> zErjq&nBGuXb&>l~W}SVPD(vp{9v8@HVm7RjyQPVfix%wvVe2h};%K|IQ6RVuP7-|Z z;7)M2Kydfq?(Xg$+%32!xVr{-cY+3YIL-6QxA(47)iwVHrfyks$?7fnEbxHuziWyp z--jyNs`OXkBEo2-4{+tU(6r>8*>RVTUxzr^jz9E3(Gm&~O(~u$+_i`{b=iZydF!d} zw?4Dxjr6E+b?l#PgSk<-vCZW_99eZ228mMsR-4f@6mz(=EqpIb5r^nHl4+Gzh93Ra zH+b=X_#*gyWMX(o+ORxMIU#vUT2b6kJv6;#FOGtmiF$;M&+aK(_zep&Uv9{^O|G-7 ztbFhF+sg|`ic>ib-RjBPlZ4QNSkRJ zM}#&?--6jrE$(?$JqSBudB37M{3{>;06AqRT3e|Arve$EsknsZl#QaH4W;T5$@~?Zn4sekJ0y zpOL<;o+lTZmsoDwmE|f6sw^FVgxhdPYtev;om8WK_iF78@#@k=%SR0$Oz0lhMlV>7 z&*P?#FxG11uP^$*ZiCGG(%e6My41ud-CQ^FnHEvI$zBvOd3Z^}CGTl*7q`PkJ|n&J z&#Uaj)K}d2@a3K}i~67Slx3e$rN&N808(`xQQT7%N+hz@!lZ|qnq;xbCO&l*ra3K| z1!e@y0%7<5-@SdUuOBMC1Zkvk3TDho%un_ALD0j-oq@ZaS`acwGhjR8gQV zUqm#!yFD{<|9gpfF4frEXIjfyv+)N0F)y4b=0C>T=%Kqk564C9?Cp}=juo!(wH>9D z4+uKwrrsAhelJt|in+jXEJ6j7#DY(wO%WbVuDrUPajlUhX{KGKRRyA7NKD1H1 z*@g-GHWu`J$muKu1*9I5t(YK0+;KT~!-;&yKYw8W@V8F5{AX#kBTxG? zJi)rVZyL|!Fv(s9Q9__=BGC*J#XkvLz>hfR#WmPgiP)M0Clv|$j=%R;`h33PW3-~6 z2Lv0%hZ?64>aNh~AD>j~wHnE5rK2{4NtK<7dG;O+>7&@-a(Cf|0AIYk46t|ppbdI4IY-)PJ|I>^Shi~>f#o`8@=XEiH^*KmV zLEyn?1+BJjII7I3YMRGv_z^KX=+Qyu=VF!2D{CNn_8|BKx` zLIB8Uv7_ex8n0tXff{c%bz$*7Otopz2h!`dBUDkG2$ z*n$pvx}#~53TpOJsDS1U{gf`*2sytxLABV`%*Xv`KIW~N5pT+w( zr=F@ck6O75CUmx0RQb;=01Y3?*IK`si$C?TBJK!<~jx1pJj)Y3Rr>F;JvE0%!9-xG-P@F!Psr;tYqU~kxM7Lv$M8COr zX6w)&2mq!RSrEPXttQdv{}VjL&j4KW_;}et2H3>9E|Q=qM>Edb7n!&u&zI0YR}ZZX z#m>tQzuVNxQ31%C8UY$hsMGRCYmFM#KT**48B-1|p#d#Wai~arT59{_%@6g| zWWHEyR8P^LhlZ>9SmW{uj8g&VQmk3>^EJ_?qUz?0kE#g`tu)-z;hgS*>Fr_8C($k( z$*0J= zwxDz_?RO{HYP}gCha;k*vM*%8q#KPSO->a`Tj3Ns29Lz80-~^`nem%GpXuhsPgI9< z+UZNUy<&gN%ltODJ2-PYbt1W&cAp@`EzR zhOXBjhFr8G)B59SzGwsvtMMq_>NIKxk@H2kmax5$eS4MBy$Qc!n3Uj*o!q=t6 zL6Nx#7>xl;7|yZ$>S;-U~z^9%hGNzjW?QES2#FqN8Q}@;Rqc z2NSwIxb7~}#lW}F4QxGr&&raer~feD#FGvI`z8Xh1x_J$qh>VBzeOOv+nBaj$70bw zaQNR~j*E&uz78bXD#eO|ojA^dEuazjoE^qr2Lg8r48xn>OQIX-{0lXQa(i@D1Prk= zi6;i>@foE1o143-CYMcKyG72l@ZuU+3&6P@6otBf)g-~i4XOO^PG%%P)PCgJ_Zk~g zRwu<5WE}p&^YEMZ1R)!O|HRw+wBQFqS*j3Pg|Y0jN2Ik*vuxZZI-sJ~Mg0dcFJk-; zVy@*)SU_f>^>hJX!C9c$M(CtZaoN%X3&0JP6?zj7h*-6xYM}tQ#{Y{*0>s?V_6!b+ zFNPe$s7OD@xS;_1KTag4g{Fi==w#<&^l_T;F!}fjfrO+)yfP1xM_jg#yu++ja~7Ea zwKk`1+V3-NVSvG%0NDCs`FZ23sTp*~hYrlE)rk`Ihl;s83T11hfMFE|>0NV}k66(L zr)xi!ldBZN!V8N{_4X2{0V)*K(z%M#BHSnasQ4_NE;qE1L%W^wxeKyXI7XY;<*N%~ zr6yOx#&`Ux%{_U}vR!md$)X=c3F#qaC8rhbNu+x{g_cnlDh%Gb#vH-S~@7*_9<)MRse ztBj_wIzG0u*)#wUNGrD(>w6lsHXM7x&!yLNU1@Vp26$%E+iBj8@rzOY=I*#kc*Ebr zp(M7nU~5IWb{~fSEr9Q}HqZj}?=&lOgy97EiEis@tB#ac6ipr;3bvWyGXPFgTUkTI zgwC|Hdn5yRqn2oBDHsSJ+t@2Hbm0Lg#WTX9JOgSNT0%Mv16x}0yO^@s?b&kS^3q47 ze)GqPbxHK|VD>RhVZdHn(Kb^pt@z<;R*s7uNjxDtGuCW$R?+8&&e?TxO`>B()=z?i zlfNcLZz6lXVfZfN-Y1t52|aW`+ys_Q4AILU?hDvAJ@UVHlnph+PY#`5EV2Ffsdb2Y{r2mm}UK${sG{_UHE}d_x`GBDb z8WifBE00kB+>P*qTrVmV`Fzl99aaqwo)Z?5e6RQw%EV-DMO={fDd$_MEjuxNsg9r< z6>}f-{6(~YwUPC3%&}O1@p-^CTE58?CfHr^JH>32a$VWwc@2UFQjGD}deSN$k(${F5r0RHWSd0||6f4+uN-*_@_W(KQ2#B(12zR~$BR&c+mr%Y5}+-&mX<`@GP}*@Hr?GR?EWyAxrBC5-pta-TeJO-x?g8|+@-wQ zw%MCK(wS~m>TBdN`8Z|2ks6_!$I$lM7=>k@McUiN@CdpxV7>+!A5pD22TJB;5ok30 zt-~4q{BIFU^5YZuzbY0I+RVM^6q<^kN- zZ`L_0<}vB#d=&)FYPL>ddoILMnhfR2Sanc96sJAqxm}drp(&5t3dF0i-a`!g(+&!5 z6cJy^w}JJf|AxlH@c-uLQ8X#!9V@BcLROK+!~!bJO)hpF-PS zsIiH~Z&ZLLSPFExeIAdI5GMLV_~R{l)#W{d&SZ2?9tM(tjwqtVzO^Z(WMe*^r%7e& zzfh?X_FjYYyWckQ@$hREQVRaY>nuGN2fm=X#%hmcz~?88M4mO`_p9a4d<^cySDUdS zl6B-{Cs*rcpl{}B6rASU^rp4aeae3F-fVw}BoMf@7MIV{a|0%NEb_54C%w7z9`F8< z{xnGv&-a17C6v(Un~|7qv%#y)(XBS7C(*50D;X_?XbW?{1?LZ;aB%0QMw%Xy5XBC? zI&Wo8j@CJhDiT*0|Jk`Xnf6MMAN7r0qEBwrjjs^cL}s=?jWX z1Is_+R${gcimXQF%_L%7Fkni&VI9Y+lp}Dn$v>iUf@9cN=47(6*4v(lxCr?jxrK`; zv^PU>i`vhwa&Zj1B$E5uJ6MyN-@F3pO_oR9C>lq3`94HB;z0kxjbjtnJ&BYMg7}Ff z>iZF(&Ht%gb3Ijk)<3v?wBHTv#aT%le{0z<_V8wG$!+d7@}p6nb4;(Frxr^s_CG>@ z)rG&L{qkpjZOm$JTMN+OUk+=1ZvEwTDeK;E@<~PwZ|GZG{0m@^FEIvmx`Sii=e|5@ zXPGoqS3P*r-rk4wFa6-+4V{BtG2RqiZ;sX;AH|Ly==G{Adygcpg^1Mk2XKWXv{Y*- zkh6hbVlf&48AO0r2aiL++^?uAJO0O)sZc=#p}?(fhiN*?roNgSH6;Ui)7uLxdtR`Q zyKZxfo$t9Dm^Ddm<+S(rH;uXGXFWi;2LWxLCJ-GVen`@n5oRmP0R7Cx2l>tCz{tg@ zRQK9;)Q6KsKQAW~ZqXu8&G2hh8+V;(bv*PT<8RU# z(mGi9#PP9{4;~Nvb`LMWEpexxF@r?A!>r(fMCGA_GMi`^GS|E^_yjKRv1PyNAG=6NT4VGP>C2JT_!{uqGF{p27setixZ(55)p` z1=4mLVun|jV4_9rug`3hjVkYLl-`C z?d%=M7z+&|!Kn;So;8&G7M!6S9kd5%yM`8|v^dnEarQU1TKhkre3aM>jqw3o=0k03 zV3a@oK^6*H?w$*Zuyl2eMAG38;$`Gc@_;`sX9Hu9>3lFAPNct_a0%O9nn)^Ij9#DH&r;o272g+P>HzrmCLN*i{D=r2#&uUn3NlTgBY!s5ry zY?yYF!^^I0bnkQC?K5jnm5j${439HNZA>)xjIhw{Xgr!ZV-)7(rXeL~GU4r`tPyKx zu=HFRch(EvK=@=7Zhj>PsLO(@9f==5sY+Qg%bGtY!S?nKvA^Q_Z&G~q5v8baM@iZt zN#3<`%PX@i120%JH>tSXPuhL!UzC4^ToZl6i!HBq@NO7qNPqdpf;kmjTEtg5sCawt(sT76!nG7aPd!HJBNfw=7h@Qv+ zsbqkA+j{oW%?T}Yo^qjd2*bqwY8`B&H9)5K@&a;pgPM|Gwtg1Rk^Om;hQBEK$A1dy zP9$@inDbMLN@FEhao+uu%QE!}mexu=+@-!6gmH`7*9F{o_wkgD-qpT+yL){WpH1l2 zD`s*^pJ?mgZvdZ&+<;$<{I5MW>SFVzFDeRI?P-lN9KS=*2O`XA{KKlyt?gK#vyFK* zQ${A$!&J3?@&9 z&qpo@|K5%OFpLQ3^0oI~R7i|d-TJGjMc`u?&6xYoqJ2Ie^~Y>W=~`GjO=%Vm zyJIxP#kd)sFIHA#)`mmwZY!bHnxuS^@T)PA6bv2RN*#*exZQ(X56PgVuBnQGW>-@> zFfHuDb8t|;YT(D%+-@c#M^P?^FX~DL(ujofKZ=O}lu3N4yXw}7x}J&FZJ1a}MeCsCvgL$3IOWTuRJPENcJ@+ejID9K8bn$ZvCQ+ps&#L8vfhZG z{Qx+lqz$!>1f;OGeV0W81?s;|u46fY6`FrJ5l(%7f{K`KVE9_((U4aVgfkGg)MQwO zA3)dm5alDNp;ANDwm+Wfc&$d&3zmfAe@hbyz7cm|sc$A`JyZ{`*G$(VOTnj;(9rgo zzV4hV?*v5@$Z8vAcf-$uj%$8~%r)O8><5g88M%nsZHK^kH&)NrYBVZZp)-j$R(}kX z`ZUHQeb;gSpdMh}P)1n?3(THMM9auv3nkjQV&V98Fsov0z4aQ7_H==5K3P|RA0Qgn zwmzzgnA+#B4oAB+ixo#Lxr)223&Y*ojf>St zLlcWOh-L4_g!5kT#b@6OS<>lI6>@6wKd~L59WquUVKeO7OllE@!W~yVL?+`9$j_ge zvv8a@PRBe^y~95m?^>}7g{OZSX(TGHsSbCQhd#=jxYoXxd%cN2-%Hzjyaqf>5qMmm z#(oPwZ!7dUZ^O=oHUiW791xi#5iILu=~;&(Sx&Wiz2!p_+L;f2Sr5BMmt6`#PpDXH z-1a?AY8yUJ#eD3|99WM@Cbf8LG{4n2ZHAnPqazUt7a=FiFFsp5ZguQN^pPs7MgI9* z9#TDHRv|k?=VR;=HqPyH`}25}K0%`U)cI*~e@aNJOs<8r2IO?Up?;1rCzXes{z+>E zDqO*hEM;yxJ#pxK$>28`n(BYw!veU^UQGo!1k`vL>%opRKMWe(39?Vbnx?KTYtx{ zNm#-k?)Ku!IU3XwN`Dy$p(%f$p;nEJfvcvv5UqRL0!bQhf`3v%?n8Lzq0>aXea|$P{9vFXAVw>Gr6Rq;dt1AGq(wJmpf-o{)R!lb{=ZP>`H193f<_sx0qe& zQP`;V6+{Obv%9nbTAZkHG%IHbHdjTIV%K>op{69G6o^M)T@nLqcD&ZMeNC@0-QZ!c zP^?jx@xIM6m0=(Ky~4FQ0!O8|?xEPMCLWHbGOu(4ZS*++^gF|*SNFwv>qi$Y{F&!) zJ|@Bvcw$zvM2|wg4sduwKJ}t~Wwzo@dvfcZLi+Kkv|OyAn$ZTHLQDi7-!cC1jA9+v*#_5Jky>bs@ST+84emI9QUVj&`5vv%KXI`bJ|`@1 z=lQ%Ln^aWL;)O;91>U-&^5WTfCW_KJIrLjm8d>Dyj`h8Hq%qj%C9q8KNxl*i?w7rqj$!_Tq2a zM)cQ$)!iL{1C)rLZR*^XK^M+~@3nX-EQMa_fxhWtSYNn#xt3&?#yN48%*&sbY!X?Z zQE_DcL$lS=*Ra#Co(Nt#+X{(8N|ojN6!)7?_M@}-TtncWb1QMfRas%ec~~fC%q|bM zOYTB~`59Cbm2`gd<_y*R%CTaVWp^bNp5EjV9lsSGEwgCbac;UTZaZf>4~A2EmFFSz zvyq|e`x4*h!>VZ3scjVZhS9V6fMz(BbS!XSeO|V%$`($k zFgUaYgW_SsK$3z2wbw&j&5q}0NwE~E$qI1N(A{5|m<9D3#Fy0S-$-%&aDM)*aE1nJ zPB)^I{^-IsClfbMI@s}Z)0L)_zMfbQ!B1S*JO1~Ri&S$Aou^Bs<JMORnaD^7Q5C*&D(ktHVNbymptOgGPP>~^BjI{TszhwTo? z=%MB6XXOt+BHBzew^JR&ay|_(OwKCiGN;P{swt%lt?l(N`4>vNla9sq=0@i8!KY!g)}jSL(Jxq>mvQOoE%3hDA|7Geh;)xvl+DDo zJZyjF{560sgDwd`#_FXaG5!2Rea)3E$9j8D9rpskgx_X4MhXh!yl-gOUH}i6>FkaX za~4KR-=f3)X5o8B)}87|$+G1Ce`;Fx;L7;M6a7Y? zj^)8s@Wr&jcaL;cq5CJCgY`)btprZ2->LfI&mbdcWAv0aifv#-o0eucjJbS=}SG>QxukNOA?nagVQ8TYJ?bi@^a6a zSs^_JRB&Lus7G_LEbQ%1J8{NdYOQPy?U2;T20#QvuF1VLVl zQ}F9s#Uf=+Ef|<&Mrih{E@ryKLqhhe)8X~*)s`MY+O$Y!9shU8DK6CzWooD+(I;*! z5Tke}29ij^7CJxP#XCqucq}C-DaOA2&MR(#Fv}u~%~BSO>K)uDmdfRLw!R;fY6>X- zjMpQchZn%FCbSw|eZ+D<&bvmLgn~Yfgb*?0ZXuCnB&f5tXchD3ltY`8tv05Ggf1uT zJo7P9_G8!_JWLGxhxC@4m?GLY7;x-g?nqaUheCcHv>^<%7j;<5wxuz4g>J z`n1l$*|d2%nVL6d(T@j+C*GTS#lc$vyNe?DmN|=6vPDNGJqNpHV8ay(xb*hE>D}8P znum_@88=IUYyrUT1W#JKy)N#d?k9OJ!HakJx@pOfFu)@~%m81ljKUB4JQZN4V>?M8 z%=_gitH-;GB!^HgF})64p20sKyjRvsq!X~Ds`-olx<#JgQiW;5OKpn@1`9N(4$!i_ zi*s_cl)z7_X@xOo_G2G+E?`qko{sruJPqJA2_X$3ivIF;vVXZpsBIWuq@ef-MT#2qCm8ObpWn~G;&*_pjmRL9#T zs~_0Wcg7`fVj(SdDb+I2dVZ#u?5tB@J=VRr2jP^X?Jl{P52#33klwFH8rWFHeVWy3 zp(8i~_7^mVyg82^J@1o7v%S1)@Omv+DTy5j;KhI2?$^M2kb#nuOk<{NkGM)YbP>Zx zq_n=^zm3^_=Cfm}crfN7SZf?Vc;;%lr1DxZV{MzhVgp8nV*n%+yP~$Li|U(iwaDSO zbIuy>p-W?31=`Uh2=A#s!X8er?mU`N%(nxH_L-2;+^yh6BA7a#X_QkJgiuWm^~w}{ z5eu=RK~&qB<$%61&fQY1P8$BcmQ$fa9NnU~X$@CR<^`kr3T@>nv+z0U^jHNBeY3&r zW<>FA@JXPq6gY%uec;YMjrd}LV22>&x}-N{FS+a&aJIoU-er>mGhKD66US+ckNd=H0LiGt zB-vT)TA;x-E$pX(`F5Oum{CqsyN>A&;rB$YC&g$`6RInnO#hvsUPJ-1?mjnieEZh$ zWsN@)6)2^%fRWLhM*-W>eVSD47kq7uZd7lx4iMa=IqX;S_= zoX(<|sd%&U$Yyaj>NB_@kc5q#jBxDEz4x(9sZ*dIQLuPtVkj3oH{&J3(4=L86NfbQ zJUaBN^bAVz6?&FnJZJgCD@ha}H250}wA6um7qMYWJw6p!2$lpvz7GUH&f<&KMraXB z##=Qo8}y~gl-1;>s`4xK|FNy#2>EvWxNC```;p0Jds89(fvMPfpdaHAP?`-kihFkA zh~lpwyn)X;17eAI*p^T!QUq5qK8%jj(xpsyU!&U#KETv~Lx! zJ$}dP70h*}!YFH2+LTtB=eO)S4&L?1B*WhTh4)1~gpbMjEnx%+ExsesBnv~x_;tGrDXJDmI}l9GZs!5oPogkV7r>Q zg1ko5wjD;O3ltvfk_UM0vHK=L;3q=w)?*Je`0SA#zYE^8b$>g;+1BW|hLsa9=*=AC=_Lem(}v>(jTa|quqz&kC(6C{692ONaHgDm?&|fV@YLlg z;l*5g$|ZHI9#OF(0ad!rNqvr$Kp%wLxqdSDRtJ82|x|W_~vqNuD_lv zH;iPutWl%!KUHVAAC_!p560`G`!2k@49?frZS+SU?q)u;3jBOL|KiVesh=PQMHUMQ zg(KAa%JnC}G+UgLd$M~T5sR7_6za^`EuZD-M&^_IL`^(vwI;Wg@)YObnLZc?cBge& zYyGYFAY``sM}e!_rXvTn;n#_M#dhk?@wQK=^Cb9-dT~19c0SzTL*JlEZO5;gUa%OG z@0du5Y0GtrJAKav;+_~kFzeTQEmG)W5c_?j_hJ_y$HWLt1XMQbKCjwhk$C8TN@$4! zA=YA22v8Vy$Pn3u)5u#q-?T-vozRo$R}iFDVII|su8I|rBXLK;*RaGn@vJ-9TRbUjwU68uFFc_8O_N=wKm3;bKETxM>9$@VI*;Zmz7 zzix6KEwemLlg2nef={9~ggg#OcMqR9to()hxpV*P!6gJ_)8+nG3L^NMg)^tbtg|M_ zJ}@&mBnB}gMkuNu;7=Vamd`ib9sYhXuj^x0xIGY)3{Ld5e;;l_RpQtb zTg;Q2^akA=TUM=+%5Y3!YX1VP5hmh&n{0}PDhfdiid-2c`#86N-NF{Ca zV9xKNywHq_1!MSTT9|Zx^*=_a*#W?=VENOlMwWq3p0#Tbc*RG<9z)>ZLu*X$P?~S6 z-VhBj<^C%FoQ2EXVNKg+l{^n(Hnwy;1OgeCUrz|y$kYAh?nJJz{gfo7`ZXPqnLvzeXwzp7;ct~K@ZGZRy)? zk&h%xEj*qAiIGrhL6g9FzQ1u!}apTZ>rhJ!fc^he_K&t%tC{yPQQ28AIPN=W9hY<2dnjaSpP5w z<|-6Q4W*i7QU5ZZjXzy&K?($CNJ~q0Lgxz@pi7BqGTQ4Byo$*C7`HaabF{^*{A`ki z+C;bYnT#qb{mip5%uAKiWtVjPWpwJA== z(S;@^w)))K$N#9o+;+QtBrDqR>-6=c4YYA3No`Wtk*W-;c6C77)8mn2}5y*4Ycr1-PS=YPCTE}rxlm^!8 zybt$cC*iAk<6@p@gp%4)i%V6x#xJQLFjha@CI9Oa7+1$NSGjO&GMa$kmt4_gUjzwI{rMtv z4>3lS$gPmM=eb#Yhb?5uYPq#zuCarap9r|sh57PTliejX`75O#%LPZgf=LqSUW@e= z9j3t1KQdu@<@M`%+h?%bI2B>vkcq>(v54#l~#E zBk^aZBw@wltc}@hii}#WlY^WgUU}X7@dT;{0rrf{mB?zbRFHT{s8bDt5wxDcPAO&sG~euq3Ka#wo7`U*Sek_ zs+Pv>5pIfnIW#g8TUhB}P~7p>*ik89Y<4Uf(MKX8s_@M6#zd3knv!|8$tDglDMW6dJ4+~0x zIdtIMbDY$wMU%C=={Q3 zJJ>-4%8Z;b0%pTddJd~4vcGu#vCus^(B0_0nJ>?e7K0ydUY{<4PXNv*e8ugt@?Cu`~hWA%EBAtU^vl+B>p z&-Q5tUJqncizv4CF-+Ip{pujM<6?OOx#KV8Qs?NhjrH3>umoAY+ek>8OWr4P3l3Pf z-7u;~?zL(?z8os;Y3n@NNAvRd>t+-@PRCMJ9q93X80V`t-l6Tpw}>Cl1Xn4y5F%BP zX)Q6ihZi^eN{bTFZd@NYw)M`iT{5wvR#+Rea3J9&3e#L-r*X3=p4U!}aHG#7H+g%B znx;tc(g)gJzfKOD!iyJE=D1=V&+BCfNNYjCw>6#AFFS4^*`Rzt=mkUWkbQnUu7g?Q zheIPw{L25V0uqdBD}d?-BgDGi- z?=k!@YPZR>>I!hIcr&h6M{^a;NQ5#ef+)eM3*~gp+XqCUD)1G;LN*z2c1S|FfnYqI z{Edo0+eTL=WLCYqErgkDg+n2N1@LbgwT;- z;GI&|-3xL(_88)bFV-)g)JrCCT0mo3+qs)dGK~;ZvFQ#M2L@`Ypg|#{eo`qnecok; zhWuDgiuOfi5x8D#evB8>ay-K{hhbis^Ew_fr>&Q=V{Cz0Iu2uO7sqwuhmjT_mksT} zuFpIn8z-03U-?UX>+E->y0F?o>6=!LL*&WDIqw*X3Z;|Hg)E;~f~X9G-!HLSF60Pd z535;|aYwI6f8{w3bSGJK10@m@ve~Sygb#hHYTeS+tu+j*Os3af>R`}O>12tOXG9d~ zqa@vghc!9XeW>{9LTtYgdRBMe8TU;;1}^%JXXPd9^StbKJalcQK*DsI>Bozmzq8cq+V8_M6g)?gA0jMKf=F*jRWUVxs(~4n;Ev27WFvj(R7q27~Zi zafr!ilg7F&X;W=geqz6KR#WldHWFQTjXEZgD^sC+&te>DxzZ@VL1#ErUcq)}ve543 zfub1DfTm-84yyxrB|mUjm;cy!hu75E{&>or%3+fgoP$lLq0}=88V&}rhKew>-{om{ zceEbYj_hnjFvyZ{ZaP3n{@gr)?z&pCsQUid$umr%!D^Wz<-9__Cz!_Y{!@4YHZsH* z5`xU{a$VjnXmOD-$JZ&M$uL30R~OR=!bL|@%~A~KQB=#%JC_*_K9V)x-fJ7*CJR31 zeV_C;yt#irHm<;E(w-wjn*#25ReLU?Jo?Pz38h}6#fK(w%Y?{n{XvTos15ygNc8e* zR3)GwF%u$#!=&oRjLe}XjT5mA;u>%fs0sBUjO*U_`BVA%(RvF;JDm^mL?hzR9(c(z zTYb#sW?W%#wd=eL5QBP84sn)SYcv8|Q6vWz|Z;A9R0OI=jc6|g$gg>}qnqAIL z`Edyy-kOqd@t4dj>`Ttt)xHQ^+`v&7=D?-yFJBOqC~$5UH!#k0L#^&ij2{hZ+J+`5 zb{KpYr&jlu>{~r97u-jZUYx2r4dI}|>K}n*p;uSSE}4XpT|8Q zbp1sAy)4B=we-aHP|^v*B^CnV?GeC&Zww++GU3P3@MdiZT(RjagH>N~JOo>?BJTqr)m25#^(gW6b9 zffw2MUE$5#H_&h06D%y;NbdvFaUOu&xeU@M$|NTSGn6S zjm^T`UZ)(K0h7fl5g(QMoe?iEWDF?L?KpEt)Y4&%HppNSXrG;A>!%B>q*Spf&e*f>-!^1$TyQDuwl zkL`MUUGMwC;BAumFzzd4FMr&WG3OAgP+gzb=LQPsegQY&ZJ0~LNc4@J(;C+C0&^f& zAn)%SmQ|@hBHTylE$-e2+>Qe+#CJA=edT|hpa1ts`~}L6fw1j@$9iB;(a9T zWCeE()&+{T&+~Pf3w#jCF|pYR5<@F7$GT_TtEZ;l#Kq0B4UkNQ*4?iU$)i57P4#4Z zUWDoT`ptJ^t7_$D@i@~TW&wSq6(Y|EVnRV41T4gNRfNbIA!tOEgraW+ev-QA`8Ceed=r1YhN=+w>8A=)4kUOedCQll`>85j z{x6x}IQ^gaJa2FKsD&cN_2M0z5WwqN_1kl)7xc61S7IqB;3- z@x(iyu2BEo^0!)RRWDVkC_M^$8Gz1rS#uj$S8ka2vp)2l7A%)1g5U=?1;mKTZu^{D zvm~})T+;u2i-`U3z@Yi9N#JhpU09Gb^y; zq$zYvY=F}S`7B}bVswlzey=>KR|Mfk$b9weJ9~1uy0oLwIsIOp_P%-#hmbj@Khg4o@^V!nxytsSH z|BcB1zEQ-%2s@FR^@WQ+*(~NvQxm@OsVD!9cH8}~%6f&w-f<$nJN z1EGTHgdN!*HV`S4ixDZm`!gWD0}!LR=_Uu0V})}BH%VG0WGIBzEaHCRV;DD25b7Q~ zJQX9RhOO)m*-X|&_FfuzRuG7|nz#xb`V+o570MEv4P{E$*JkuCq7mUM0NuF{qRP-| zwAH_!t1T_}w)hbyu|~F-M25enF?a_3PIx=g8Qg$@tn>A6qYx2EyC7#b`$L2SuxmWm zKZVveXA?tVbqGE?flGb(&Rnr`x&fY92EUuEqHs zv@cp$HMLQm-}H`ee%94*wndG@?2xe3%thNt{@?ggBSc);2PTW@x6~hmgDz98 z$_mG0GoP8TLY^ar>H;Ge!1N-ycIs_Jh)jb%>w(f^N8jJa#`54w7Vtm)6~?xRO@^xr z5)AZ*iyP|>beDtUVFG!>j)Qm3{eVle#+`ID2y`uG2EovapzpLz9KCO-UlHP}5Fu$g zFKU+DA0}iebmQQ^obY@S%+|1wsJWh|=?G9AFArsvJ&y=p-N;T!;)r?Z#%Tej&MS5U zW^4xlR51^~rU#m{rN7C=7kXh#$G%{@K3P^B?_%2|hwVZ%iY>;wr9$t`7s!)Wffv)! zr^1jQ%nJFLbN~lU=Akl0Grmm<4;PdZZmZFn{r4VX5+{$o6$jlpqTpFVcvQ7hpUKT0HwweEyIUVv9WBt}LV(32RBv}L)hwsV2a)t9K& zuFzn+wbl-in`yNBhONFq{Z5#WG?W}vWSPW#9D?`mYjM5Rvca?s$eLJ65MS^&G3}UK z2nxPkx%czK=Rpx!(pEi?*Ki_rK2&OEa@H_aQO8ObR23sAT9DiH9f0)u2LWZ9oh!`!vr?9DCJViabo@w3QLf$nSjh#3^r$A(4n``;-@sb?`$i> z7-f)boRSAT@U}oArwOv3C0C$mdcm%TsSSCVAsPB>Ue2^hNvF3Eefet5ES5aIcQ>2v?hcWz@8&t4^M2=feq->*hB1IUu4}Ei=A3I??|vjax3BU%dymol z4Cp64Ct4{#mI0FfSGDiTuRdidfv7$vu3BXJeDfFwJJ48(hX29y|KPD~g%7JLPqG1R zb7E&_Mcdz7YC5RZ1YCi^`Gqc(J!(e&tG+0d23nHaJr9R1m30gfr)~Re#t!){Htz!`dx1hX9D=|AbphJbLw0OZlC9i8&m-DnQmnY3`Evr zh9L3F79@Wx6kxphx;lCLr_YVs!XrD3*0#62f*>)(wJs^nN>?vp{?Uc(7no`tGYG~| z0cY)x5N1n(YRBrCee+$Foh~M~Am#|l)D3{M)T0_U z5_HE+c7gvB=eX9j!~dPkK&=$0!BF~=s6JzEfW@`b-9F3x&#nXcf5?y~9;Yv$>SgWt z8j8oj2sdDdkFf$WW?>(=mjp;??7#sUlnm-D@i}Zosgj&!7+hOdlr{RIWDnXpH|-aL z%J=a>HpvcLF5Ai&Z`*MzBB{vt0WoV5`=`Th03^^kIuI0EhEwG^#J}}j8UB^!O54ZX z7YHy0P`}J#`1-tzPPn%mc1@MtLVkwgq(Of;)Z%Go(KZaIZT5*E%P$eyxmV0S)`bzW zFam$IX($_M1O<2!I5cWs^55n~2wiV~O^IaDEY@=)IqL~Xq?P0){a3002p!2-F-(Ss zE*$%hmuAnJzthk%%%D>ML0)1A#~~zv%A(IKQU6;7y}tKJ=SV)!-0mw?$ARY`U-ZXz zE=LEv5|Vs!%7a>cv>6Q@W^|W{yYU900p)$CU{}5s9`BA0At~j-Ai1@L2fMwvtL7{d97cB;Vn1kPPC;kuD zy-5W`_Mem}jb$!m4H04ZFIE@n=BjM*yoKr|HdTR8hGX-(`ia!ny8cD7@A%m-gm<&U z6662Hw}@5Y8BM2b5rSPPJW8CmQ~9 zFd(!{C8lRaZ{R5jQ9%$nOB|q1J0}oyu2kJpYbVkFe(q4Y53A9i@R7cpZVrwK^E6iN zIRt5394(b>nJA}#%#9&Q3n;+A06^t(E(2#dX8(T4c@(QD-#1|0>9v3zy6ANz?5`O< zNbh!g(!J+QWVMv7t(h}r?(yG?I7AZ;KT^+qD1AU+c9AO-OmTNMWk}i-D8!k7*OyLA z{5(1@S0R;$HD6Cb67D#y?|q;qMn|uu7S~?bYcDP2|Gw-F4j>a1*FUNR4}>wxo){I3 zRE0p?eSiO=rO}Li4>xAg6XQzN2;iL9!})|f01a+USV5y$??)$s=)6 zs9UWlFVt2jZHI%f!jWW94{UBHAV=7n_Mpr>WDWm%G6BzRxrA(R~m8^hw zNEjIvMZ~WQOzSA&HQ3gs)h@mXy8=(0-Gi(bFU=>f1!q0Q00KEIRB?%NEoKkXHh81U2JDx|;J9 zqE=7uss0Oz|L5BfYdBdZ3S&2V`PREfj0uqb*|P2|vOEZ6r`!I-fmW4NF=A=~4goKk zsY-7F0O5Ix(=`F4>GWh2LBNpC_!uLF?tYKm2e&g(P>s0MyB%<$u%j9W<)sU{_W_*K z7r2S*vpq9l{)uOk;0TAC5u>Bl!EB|gjy(h?oQzk|!oLo%0-O<>e+vAvmO%i#4x@9f z-#Bfg(4tD~srSu5D5f{`yQ}mD(8Hs$j0$BV#|r#Uz#jbJFdNc2oF-`^uBb!klGu|i&XoeDiuYB0vG);+vjwU_x=|)l)~q? zO6}>8pzH22aP7)@8(rf|?59F)@@A#Lh2&SO0Je|`CJ;3SG|%g>R+_c>)`?{eGb>OT zF_IeE^{TWjZb^nh$f)H>dKEd!$Cv6n=6L~G2PKW?DG6A4Y&^PegE61|Y0+--{K~@KvLGCHUXnr3;qAV;m3RpehNZB2U;YkpI#G{ zqJdBX(;d4=>e4vO!t?-wvThOJ3H9%R^jQALDnTf`k6)eb*Zv-r^6b zR;E7mMd{;!fFi9j>iwo=fzrt$-A@~80%zlI;{zSr9aF=Ht^Kdx+t?FYquD5O>d3;2BKhMt-)+mg(x(My#fxk9w@+RKB`geVFOY^F8kM zo0m$@XrqEG;f+xZL_xCk7mHq<;(ay_X8*;O)8tbqC02uOb?3(Y~FjNAA@n<~&`^cW-_*1N0+qtSMsq~-M z0d8WF?X~6rf+Vgq>n8+eo=g+-IeeMKC`0pscB^LVWCuIb@A{xkvJIGT6Z!_m%R%q3 zeNSowV08bBFy4TJ*FsmFWgXK)O?!!-OCY=)X?NwL(ugiV4p1ISG zo?o7V#_XtB7dQSKrMyC%0f>qpmx%u4i~gZW1!!3n*gY%H_#Iaz8>Noj0L=e`3Yq{|7XQw!e|^XT z(H24QQNu2X#mkPx4YC0O5CtjYSgpmzhJF2^ci$n3N+M{xl7n$FXM(5^^bjTX8N1~M z6=kR3FF6XOUj5Nc0ob4v8rm43%a6=--ZGF47!rOziPAxXkgiX74Xl+WVy!OQ?S1XR%d5X_2jEiIc4hN6&Bk(0A21n&g zfha^1>Q2b2r_~tDj+cTW-Qusz1wjOCn{02l!Cbb(UtW#=v>L4-R>P~D}AE5`f;S8E|*GR zQ3MECfDk+71KC5~92?C((hHg(Tgm5BfC=q)%>~x1RRw!C3y7rBmMN7G+57MoaP#Pp zXLZ%EVJDqvLt$vpT>m+M)-5uNIzeCtUg#cqQGV|`LU00-pGKqY z_p>#Rv$H@*sX~R&4dm=3-nmb|RT9pn#zLS5ag{#GO?$Q5*`IPWOAG`!XGkp4vfkmE zt8@&6e3g!zEKk&P3iYIJ0(3~m4)Dnx;6roil30iH0x103Y#d2iJ$PdmOQyyCwXX0n z$x}cYY@WOTyieA`hk80I4lVmu*ujs2FL2CA#Y~08US;&!Hn5G=RWQVf2B549|4WL1 zV#R;~x)qIRZ8jzYL=_}m_dsryx7q^?kCg@x<6#-_2+FbtrHFUUCR3YD2$sNQ?3IwrOwxUdBr1m40HA6-_5&VW`< z0y*eQzmPXTvBN&D(Nnyt@t+z69(M^RrZ|^B2r)1b3wMIX6eDP+6?+INkZ(Qx4BNy1 zO(JVi_sg%@MG`fw=%0@jQ$&2~9^rNW7u)afoT;t=u)CmY7(hp)4 z@1q67@6OdYe$_w&Ac_^BM5?!~q>+b9w`@fZG(sA}Z4{T$g? zwYZn92zS!=)DVV+wuF@Voa=aEs(P_x%_WfH1~xKSZ^# zbNdpD(muu}gsb`CzMlTeVuMEo!BmxmS`~EMKnWb8%V&plL<8+?1)G0te`@wJmK42h zY%Nl>cyIYh5Ga?kqDjZe0b1g(L_Ne|;qz$R{r^)VU{3@m=@Q1tIPFJ)iw-jKxb|B1 zkb)3KEz(^Rni_ln=C$B39}p%)<YT9$>gewj633SvNKH&By+wckh&yk8+^p< z7^0l52#7#2UXUTX`4quE72y4?E@uJK7yQUa1QlrK0`3&e5u~TLjwA^2y-;-Y}>=N(I!06Q80 ziso*!6`x}s3y2Fwb)#&hw4Q=|%9VxoyLr(0(^`n%5jD`Us3~HZA7hz&1)gn>NJ!VIvRm*M~c_tJ@G8XYL}DgAZr8`Y!8DGVeGea_;W zJ}Z18WZ`=00-y&oYMcn=MV1WkJ_}3W_qXMyy+6Zp1?&#W&GO---Nn>6j9<)>vwM*n zBwGP?TlGlB5a1*WIJQEXV1f?Xd*bMrra5bvP&GED719Zd>L)Ho{v+8OC|+K}%@q*- z&kKXRB5r_IvPE6W5qIfT&7%76)i9}akh8Cr|5ql_|d_i3L0T#3Ia zJ@)A7bHb+Qbf<&HOS0eu6r<&krw-=6ir{7(VH(L(p-wVnypYnL*wT10rSe@*vmHw> zT+_QmVA)}3iiz(rB0WgdtsyA{(PkA4H&YTt%&9&DCNI?ZFUriEy*={hZ`!mC1^V5K z_OUt2*HTv4dv`e|ltj)Oa<*nCNYf}ztPU->=3^IN97Ccl%i(ukP`P}ZUEEN}BVLF3?@C!ks;*bctp_w|X(OFV+rAA>g5T_7&_w8)Sn#(js`*TBZpOP-}U*;hr0C7B!0&3i6EyWv9dSMu+BuoPv(qu@*cO%`V_tX z^<^~Z!Q>d-c-XQ;vM(Ge(mHf-`2~X9cjD+i^9Hxa8}9HTNH7iKh_e4{!d1~S!9O8O z9H)n57aR%HkCz~=UrRDb-MuYC9bBL%KiBDu;If6WqN6^5P^eHsx2Z_8F!7=v{Tf#> zt7li`Fbq`bXdxWt)ByxA^~&A5AXoy#=RudUhW0PY#$S>OTUn^w&}T&UW`avaV8Ay@ z%oCJkA>}!wuo$_0)?nd&H!KH)bOARZggk-AG?V_iBuRbQwf*`kNL2?p}Q5?CbQT zocU)X^1=a~D@P^Ww-XGDc1&|*3UE9!u3qa>755O}=tS+-;Q&Ze?2di?ogX{f&)-w{ zm-6m1sg1~v3e{2gmjgZLuWzonS$C2bg~}5uxKL>6{WW;6+ompQ%-<~`GN02 z2*B&bxFH~s^Cak43OD;C3oZanTYSVwsMxJ*P&~(DH|A#2fy3!#p`GDYklY2VzMInJ zchRgYT&$$jG4+^3V8WerBM;nToEER(MQLx+KW`Q620|c+Y-pygZ$|G{Je|SC?a_mS z94g=&_vjc+Zep|<2E(7qt~Z`9icwblx?3TPo`+=u5=tdCK~*6|h)i_GUSs!p;3vV8 zl4|s7U^$M`8!bv7!cu<1-l6eY_JA6ro^_q6c- zdiu}M#z)@7Gw#?y3~D5%Y@&#_$qxZ2PW_?vYbaIHT_|DN6A`u`bP{U7Xg&%x6og*wHX+BCbL-Ps%+{qKFYbsp!E1X_?`zjLs>asB8g1NKLmUM5}y?U0Dk zv!|%(bC!VuHzx`-oXM$ORj7MF6g*~#2m7;?00k2SfZ1sE&und% z9Cn5wCsG2#iFQmdIuziJ$;eHN1`Gw;>r*fFk^p*2wAD8U{_wcCO?5`0;W7!V8-a!lc9f;8GLG!If3)uYD<42*X#Q`At1P7Bl#Ca zIQ+D!f}v$B3AaD#qW+Ugqht{RHigRfRzC4Iaji}1-2Ad>p4TB^l?B384RSZnv3$9qZHmoUZw9mXkwypN2JX6BmN?Fu-K2k0F1 z?rD;}o=z(MUBqJ(`j>Xl+`t_aUG8KQG_EVes>IN|9}G%W^sZePOC?x z#jK&RAhx!k4RBaD+a-+N_o(Es!$Eo2L&6}e|Dw|9!OWHoN1b^ zv?@H;r}kUO`nR?-Ee0ca6UcC~19U5u@EIIE-`t0^OuUhm6oN{~5}DheBwj?Jq1LL0 zs}~L8bykNYm$$*PGY!jA6j|TL(rSl6xmXPIp%=3Be{ZQgQY?ZdB6sT-J^oChmHV?$ zp%#aW*u4|kC-5Rd)#p<9fs#TMzj^xRZ%Cg1M$zo}kWNU3&~SudtzOV>& zl+3VpV2=^^9)slmePMQ`koR^JbBciCdv04Qe&>xajmkm>Bor>@WC>PV?}jSoVJ{Y> z)B*ZGe5Y6&u1qz=R8rihwoBBL7&-X$r~Z9vY)j>|LipAFI^-I=QW)C&C!pQys{P%X zeve!~4I=ZqtjI~l>ZGu$2LSEs+7nxP^TF&&oBf8ve z2F}Xk&L@V(qC@3ZH{-8DZSYt|`kb(IjTM61ZM6YUi8hrPVjbQSnTOEBS%ZW3q>KqHf zpU)@ml8rNocr2^EcJAkUw$E^Pyw~u)D`iC>GDiA9B9pc>vzw)dH7_2(zeV-Tt>RC^ zl)nom+m7-!n$7ILaC);>kfSvdrXt~Zd2C#?H{5!sQouRFn8LlOOLAhxd!Vrh_c6z% z^Sx?Qm#k`&j_r@dZS)tqjf&Z>$#?%KK|Q8$DtM6%2QFCXj_4Frhir?Bud*+^fj%pC zxWla;{VqvrGK9G@NlJ8G(KQdNOd|hlqeWByh|bw2OA%}Bi6hG@{>ek>FXvyHlT>gL z2-n0ixzC%Klo!rP@VB)%4s{zXGu&=k?i*DO?pGv`e>OE-p}nqL{H<1H-|cp_+F8He z5P>B-ah>-qFHD|4@18%g$KS`xY@&_7+@Viv=6p#k~Myl*5>!WB=g%pdv zv#;t_$pM=dghiy<0L}e}i}OS=O@XZ2VmFmC%z-=mKgpszV8KUt!yYlf}y_DEk;#y8V{?iqF+^(sek_j0pO6jHp@ zxgvGVoe0P3F)@^ndS}+xKYSS3KQu@-rYj8l#}gt$4JfcN+vXJoz_S6ql9j+}mb#mU z!aT*L3$oKT8Z}z7y5W7lmQ`S1g%HabYz~&2ggGqsFF1UcgE`9G-4DiAch)uzQep81 z`gT!GN(;<)8BOXJyp8KRuZFVN3~O{T1&mF#ZExHThvg2JDi}I*SCU=sSt7Taax3Wb z_z305$lNJWn0U>ZE0b;iEJJ@ux+pVa>rm9FyFa#Me+Yf4#f^q`1tbHXP50Fs^?Zs_ zreLgRF*MSca>er=a1W@NM{X+j zr8bM@zgzjqnRfpzf4b6S{sY$f6q~5S!AVgDM8o=rYF>0)fp4~2zP6G(Ph-PZuG-%M zGv()oy$2_8sCh*{SaDY(0+D^FeBDI%T_9u%H&$Ql5`H)Z97ODF0CLl9Y#F#&>vj7Z zle!8U@;|S;G!v4H~wwCE)yA6go(#+_!hucxzb=DHW7^lH5gu`~wK z>6IXQYU1P{H00YY2~)x*HGL|D9l;x2!0io)^__j9TH zI)`Wp#iXekwtEJXo93-%^TrCcxPnY?X{Rj=wg%g!!*ox(MepunN=3o;9X7_r2G5TJ zq=I^Gws%UX_>7DU86Hh)g$u(=i>Q^ma#^dnIj`3^bLA&eIO<@!a4XwE#gsSwHJVNW z%nR9iI<_Cbb1mSwDGGe8#DA_K-sC9WZys^~y4d%Yh5zc7X+b@&*6zY_@siT)V71nX zRvL4r^j(=e#iAihYSm_-jSwpc;Q~9#=#T`Digrdm7(ae>MX@~muN8Rm1~AJtEb>>S z{-Lor#Eix*?0YU-g)4&u^3n_7a3sTCM~FGLy|C!TqHF;Bd7su&1S8{^9|KI4=teM> zVVtNmW^4qTAHCF@#P|`#?Swfzi+Sm}lt z$HRpR2do!a?P5QA^ou%~-;6R7as>VAkQbK?%e>pS2}3=+t1xUgB#F#uO{#cG@Vr)%H&b z)W)$jmn``X{e_8I>k>xgi>yO?!?EX=Qiq&~ih(|~2od=0Y7-tcEzV%l1~{DLZd+)?{$#g25c z%47EWAJ?(_vt-3cP1WUj-44Ptc^V9{5xuX~ALOH}ZNO@?*oNz2fTT+Db0zw?8| zXJ2xuZ`_Y!mYVJ*!{QieEPmGOw^>>12GiKYang{Q7c#9uaTAI?1FFmdW0UIBoJ-m& z^`skX9sC#tIhziqe`OXL@Xh&MPnC!=PBkA#qnh2PzpPRP!1oy+M+uGn>7#Hw_{kIw zH$pLL+8FGe5++}y+9$k(7Tm2u1X{r%4es4iqm>H6$>(cg1gxZd1_~>)S-?>g+Axp^)9q+u)2exuWn zuwR(cm!UGXGKS7i(V-2rgjHQ<^THxSBA!tlHA#ppTlwbvpwbpQ@K^_MV>cmTQ7~gC z?wdXP@CAAbP!zkcucqDq2g5<~z=kzqotOKA_EF@(7K%&Fegi)djl0qL-JJM59F1z7 zuG3;0Wy=!&Xq(_kR?;~`+XZtUSDW9 zwEtSRnS1MyX9sFZ%pkFT z3ni5LqWUV^6HULl)Vo~CkBm-wm9usB8v!N;68GX%8mNL4(xtZ2h%X8yTne_=ZR-yC zGcCC;swv9^H!&z=09%D*4{4vw8fYR0h)?1Rb*q$9^rjW_$8D)|nKTpUJ$}7k+N-vqu})--QlhAP zO`5T!aeg%WLT*c5{S_!nAJ~u5UA|XMsIrF5=U+I<=5UhtXw8GPMn_QO0uc~THefgs zIpPOMA{mM#>u=kb8iTHl_Yzo??SDAT*7_|Sa^2`wF22|eJvGRiZ1R?fxWGp!Pc_`z zgQog0h@N{(EgraJ={?*AtA3Sr_B(xfAl$)4JG;0B-B-B1=EUIc*DWngc(DQ9vK813 zB~aAedH_7bf^#U&fh|aZ%Yo;2oF>_Jd1uJ;7yGmHbp7g+*{M@SGLC)5(&VYP>9E** zwa(oa5uKx5b;8oB+}$0I_LNIJ3kmc9kST?f78XH5kbq^c!V{F~uU2VyY$;sYL_iH+ zhzodE3*3Va{>BCJ)E{P=DR#NvGn%(qV=wh}8 zcF6H3>Pca!=fQlrEVv`bECK2@66eTB@o`LXJXympdTYCX>nH}JKhYA8)X-v42(56wIGq0Wc8DeigR`;~JfK#nuH+_g6`e4c z%81Qtq=cSQ$XT=VIz$wd&g82n_2Lk6mXtf87$;ZI%-7_6`ufE-G~1*7%;EkV!ZS8eoH3duL@BMM97E?QQX0Sh3= zCt+4s?cWNa(q7Z&^HO(yb;y)S_yfNE*+F0wHfD;Buk3Yr6VyqKHouPVNhvmqYc_wd z=}iDoom}@vCfxD9c?E>=(SaAz>6-5L$h| ztw3>{Ub*18ED)rVteP*Df=%M$DyukkyL+E=Y(c4-Hw)+w`9wz*wi>-a^>?Q-7!~0; zMJ@);;+p~@SZsh)nB0#rbvk#EF5Mgj7Jj*PEgq?i-SJlvqX8(Q#Ou?at-nCo7I>8X zjZ7(I_+CFn!EfVrw|gOo=H2Rem(i~J{Tr;FPT7}{31&8lBWB5BBQI+BJnyCtJBGsP zkW7k}m!O#JDc&cz&<>5D{(YJcF#QmDE%Ur2$&=?d8BYz8{~~;!q9_Z!X-Vy4H6!ys zK%?*9yUQcWxdnN=aT{9b(H@7#>S2Mv?k^{*a*wc#KoVXx= zUDCKIUM(gGP$?jz9PfQTe^Z-rhY%3jYN@T!l=y=wd z7+Zv6>wSyJoC6P23>P#?Y@vb1rN*VcSq`RV$Q-wc>&^xO&ldlRB7Qi>R)n#*f91n1 z%Y>q$S3;%b&lV=?I&cIKpHZJx~+3P2uJDH|D39j3rVr8wC}w`PytEox6NXLKk9pC?2a2 z5ig+3sy6XTaH&`C9txNTLKk*Elv2ntROw0P$C|0lk0>MC=haAkwhPE3W6y`|MB9)~ z47wuY&JA7@IoNaFJP>z&<@3^-*87{L1K2_TWdtyO?MEti+3Jw>!IS6#YE|&c_r;0y z3fhv96^Y>WwV?yk;{vRni15AnYNq0;NOBQoWUPIx8Dz4M16O*S9>(@BV}Lg#7EVyR;9_;#x{c;QQo(yL<&@a_ia+Qds!M9EJ-iM{77j9 zd2K=J%({_2r{}Swr;Gg(RV%|dhL}Gjkg=f5i6qDifr_Xy$*=f$c3qmAPZq!Bso2aX zT2{dwnUxU5%$@S2Ol3px6;cX6{VAg&a7&XpK~RAD-9rISMFTUF0s$(FmT8MM(D`_S z48BB?mzNTq^(8inxmT5aJdkma1q*Kg^D!+@X2-#Mx%IMI2*!u#6Y}YcsMf?rjz; z{(6$8*qfj&VNe)!{(vAtys;V&S1iW0@6Px26sLq(U%ILMjQ-!1vKrju2o>%P+iwO@ z@c?2+pnYNtN=TT(?OWNPu+;vDt#KUCVtO`HCaLAw86XxYw@#hH_cPJv);-j%_I=~q zjm@d+%X&YDB?^+??=5=M1xwahlx566U-tQ-GbIWiXLHr>cj4dIm<*LH_0Hy^7^PV!?R3iUTBHk(2yQ4WzYoX^@v5qzt*z4CSuQk); z@fE-i!lr}4nAdHyzt&GfrI|JrW1S{;CCBeY{jPL&xtq0Sf*Z7=0^MNI-b64@YLTs1 zbs59;EBsb%4m#Nm;(;>dLo0U;|B29(_y_zM{($*GDjZHYGCL@S;2)L<*y2D>#0vWT zK&XG|U`>KXz4BzL+T2hpkpqaupsX{Zdv6pe(NYvC$xd(;s&;qWik9-p@~mwphQZKx z<_I#%oWV21Mw@lJ`_b9#K1YSa{WkIi^y^wg$qMbNEj=PCk-O%ep=gbL$>;sEZ*k0q z+B%Dn=0mKDlw;-h3p+hYN5+j zR3b0z>pUWAPk?Aw4+EAyO(R9K3b^N^^;gj&s*2Z`7}by#Ct?R`B24CyjFS(~6a&)F z7r!~gA@I(PNkQpnZ%C+qjCd=(75de(N&g3XNCtk`cpv&&zc3IXCSI&{zA26U5-v@h zP+qdlXh1Tj@V%V5LAT%+b}an*&GDn=DAA@Xy*{;kR(&M-IELmF#_%EQg5ZbCsuF_O ze2kR7bf7#&_KJ+@BGN0m5p3j^5%A;bOCGd};MuQv%u6nv-3>j*r6wzs3FV+F5OLwk zax2!W&GkORQ@06J=CzuB31f9flxaJtY{0CKkNA+NV(t1>NvMM}gZZk7qmNkw*6l@> znGBqQc!p`_TfcwznJGDnIJ`hrC=5wz^3|W>N*YBg1bsf}Q8XZ-QLZwI_^ck~UqMAp zr8k(0hho?P=yjwMhR&SB11_1n6v~jsh~+De)v>9GuUnJ6*K%d$1a#6vX-=~QO~)0k za$9*dcJXz2d{jEzwD$M6bq_W9+;7P*?7(cJFv?~|ZqiJaN+s*uV4B8n^@@;Vu@CRZ zuNgglF7fdn{34q`+x=v5B;(GS1MlT72QKUKSuom5Aj&s7iD?bnzV{j8#@p#hY(I-8_r7hx+BjvfOR&(%1luo6>zyA=7k8g9*k@6fdn}+nxOr-_-&G4E zA6n2fo}D!8(->XAm_QbWpCB8B^vk}YYqN`#Xrpr!VbVKv-KT$RvItK<@nWe!gMshG zu`RVYZlE;Qgb1{F(KeYZFU1~RcjHZQx{capzcFbW0td8Bf~-SyPBGnON7fiGQ`MTx zeSW9`)L*{xU^A?-+;+^9Kw=bEq09TEIKyp3U+GFo>4;mUb%mxo#*8MXXz-5hJw(f-fREWPjH4q>i%_?)vy zVbUaYJ*R)%4lPnouGDWArH8=4C?DD=5x6tCjpO<`iI!f)_A~emi8e5qVV`YFLb|Q| zQ6)SE2XD!_M>&_(0;^OdsHE=Z&e%ykk@xL>b$F5Wef%$m@v^}W{jQ1u`@QWqa$`JG za<#kNecvupKsa;NUhetU>0bRbyE9zA4ocef>m_R4@CPlQZ7D^^70oFIlc9Cp=YC2e zK9@Q()y}~g94C7)1Y@uMO?d^C&Mtya8%3!)eQmwXeO>BuXVdeJOV*e30C|+v|3ef# z%64z+5;oUtk+&|m>j6y%%J%3NihPvl??~IOjf~EIN0I!Jd05IBn=_QnxJ7goye_lO`9!hh z?usJ(;VpjT>RqVO&0ml3UBnY}y}3|}&8av0OCG}6XFw48z>*<8 z^G*I_umk>^%=?4N^f{hAR!EEFyCnW1`H-l}z?%zxl$s;W$h1?zZ}B|; zq$G1{vRXOb3D-T8Ecq(B3JpiZpPKgsZ8T|b^a-|2t{L95DzNT=trG@fm&kEYtBgCT z^)(8$l@ialN@QtVT`$fDou+ZJjF{4{*>C1OgZs;8{CbYvGosz?>{+lqjBd^k6<$`% z-Y!Y^yB`VXtvKcJbB?PQxrZgBUf(^rx{945U$LsRtAAL35oLHPVyn|AfA2m;rwDgC z<4K%7lLAaMlWNz&`Yi-?rA9Loe2RxWAMmP*Hw>dk18!3xF?{gXD(@cx_@*J zx8HRYU?bXp^T3m~Oa@-f_pPY(sAd@z{e_byfmhpxc@&0n-HBd;y+8MQGHHT)$P|g^ zE*`uTtBUol@?CF%pvv15eypu(K)hq+I6?hMs$dpfjltYF%fc}p5mLfu2?Y*+~=At4CE_BE@U7x z_a2?^_g`&gzS{=8>{~?>n6)!_k~IX2FxedWXpMZT%-1O^6s|oPb+lEWNfA}Fr;$4| z9?tYI!f}O0paCPJ?G$lvnS4pEbqGDVKR;t&{4&nJ)}VHlyOr`i=iF-Ry*`;H>lmFA z;hCH%L7pGX*5TBTqf!!# z-=~YpRXZvkjOp|>3j4+B)}5&<(L!$9L)dCSAl>kAJ zd=MhAwQ&92lZonLq#NsJEU;bTTpyUnlgAte0qByOIAr)eDX1!ovd2U%$5eZv*?o&g zpV}VB+*B~{F8z_X^I=K8PaT6X{Ju7eEIEMa7mb!bLZOO5W17gZdpwvBfNMzs2l0mB z?nn+|`(*2LEh%TMR~ zX@Y%LY~7m#J9>qGe;#=D@O2W!c=J}5f?!=GAY^Q($e$zsXs+CQh|Av+dMnADnn5FB z=%-dCc$FAM((%tM0H>`%2KNSGt)_=%M1nYGES2ysi+;rPy314X2^;Tk%xN5xIPbrG z4_4_Ub*<#jeU=5Z;pYL_ai_pM)|uS#$7h=PgA<0)#d5pcxaYc^I0IEOsb2NF{O5Ta zXBP#KpS>uZ1FxF?k~LAxY1%(q+qE-W$ywa7(vf3Nc zUeyPfUL@NoNy4^6(?_{lGpG>|RqZgr{#7%^92t>k=ZJcU!F-F7Nmd8(Bi-W5zwnvk z@?phqNA;wL7tR&-axWbT5Ej+#1h3n%QPgmZC^5mb!O^ zGgVCb%(=OhM=b2?*%!{av~+v3Qs}YfM$T)UOfNsaJ*7-dg-Y? z!RNDmW{G^J22j?RMf)r!OKhVwAPf6`kF!ErY>ZcY8%oI9GoA*2SLiVz9|Us{$8z75 z?vt@Mr%K{atK)(cz#~ZchoyqmAE2%ccXeHgO6xF`Vsd`&4{46@)hG-zW;Q*6p) zL&=xHir0V(7xelIfNcVw0nWP+R6u%GY3+IQH%=A;++DQdv-1>0*zpaUB8S3Vwzwfu zaEHm=0hL&=%J{$!aB}Oz#MO{}g6LuQi!Hb+s~L$emODozy`% zBR$c$Gz-R@GT^&aXLspFw~19f0dWMy2H8Pb<834~%?zLqG&F=~lZD!K`HJ@S*(ZhH3R+QIRKYke>Kmd027e}q)zBQinLtuFOWUDo z`UV*vzPJj2y>Ggp@r{GPp>#0&e{RToLD9KyYu5gMx3=P4hLtA|(KLAhkUyVsL+Ef` zeD#sdIb|9zWa#XROH@C#U{I}|CVC_nqAVeJszR%Qf92!9Tg_&~TG}R`!VE zHofVpjCPCM0MIH&@&^>(WN#9U%npF3r8^n4_}ldSpWlA>NsJxm1z|&!23tyADdzqX z8Mmb%uYDX~CYLS}(fI3uM5TNP{X}x2_7xKM{}uO^QE?^F8fZdDu;3O5?he7--7UC< zKyY`L;I1LKy9IX%-nhFp65QS2Vdmbs!>spyKUOc+VKt|_PSviey}$3Py_2Y~{vnwE zdbC;;?;t;URyd=9L;2bku3FF|Pg5LdS$iS7=Nl*t=5&sDrvH#OIZMUJm1=%d%WKM4 zeTO4{>#utikusd)(CQLX|IuCoV-ds#ffOYzTv=Y+fM1k&AkC0)OVvI=4*j11OArYfy`R;^cG}O~;>oHd-4Lu# z`ml$k4bm<6rBY$|P!y&TpLv$7V_W(R2erC`EjjAxUB9?snJUN0>D1`u-QRXUZ5(HQ zS13!I0>BrNH^4A}tP|$M{*5ozldqyjuli2{hb?Im;}2FJJ^B$Lq<+WbDpg2JS#vn+ zwf$igG*-))n_t`_=C+*Pwldf~_5@_J(P%TA^;VInI;={pZ;zEZ0ae~0)?Lf}sRSF= z&CVX<)eQcI{7*rNi2<(us+HlGKt6ophO0=pAUEZXGe+ zp4AyH+_L-*$(euZ#rMjY@5KeYAdm!sldWTN3)$a7$^Vf{UX$p7&Be3eUxRe|BK^LO z{dyrMNwo{c;Z7oHTAss{DAQ^s?5e#eHk5w4GWfmmsnTqZj8eHdj=5pg^f~;`-L^r$ zA5!CXXm-CDZ+T3S*esrj*3>zo_m|8;%Or@AescMB53u5G=HX;p;-HYvHFVHIeD2r2 zI&fS7GIKaNAr}%~TF~ote=L%?4=rn1%$}23?FCSAMQTGWT&u3K`MS9&oNutYXi?*9 zOKxUhmIhL@BH4|)QwkA6D=Jrz>)Y2RL+7QU#|__n&c7olI9O`^49gYv<~eBDSO7wx zDLA9OFVIeG_6L;YDYhyZYXVnbPC$qhbs>dmoG3EUE0x$!F5a+iCI!S9zG=O0o_+IG z;M`ZA%=edv?~%wMr73)!A*(ilV~lk9sg4?rZl4;RoNg`#5bw9>lbK|wX3EMJrsOWC z;0TQXQ2Bab5^(5UkK@ypDabDX<3h_oAezCIQLVqr-@ zXelP`Dv7uAl#DKAxAVu!zs>E4LtBI^9glJ7i&i_r-)Zs$0MZ6FBmnB=>Q)OQyz3vQ z@?jis42!HGM-r*=5C1sNZbUC})%9=lT2Th2Mg$_8k4@@2z`S0&JbaU*e-P04aMZj5 z;LlAPU&ZuX^hXV_&O_w%8gEbPHa5=6<2<-(rRw&EqaK=^%v_G}p(TgP8#?K+!PkID z>;z71hWE~AlH)-u5r}ZmBN6sbrkI|t#x$mKo%pEp0qCl%Q)nxU+A;?Ii4KMmC-eAm zscECGoG+Rvx4#eTz%t>jQcwTl37=$I^IK(h!NmB8jI+C_WzUg(7gSG`6XQkitIzC3 z%iH!1`;VBp((%}*XJYgNfaYf-KT1TY5)ZVI|5+`cx2sCjAZ20L9yo}% z`vI7|DkFp?a`(4NbIsFi3-iq@qJ47_^-8~tVQ>_EKz_vLs)aMcEl#2w%Tji|4;g7J z&(RW)nllYxzFlmO4M6cTdYe1##~-wD z&)W@!n3@p9vJCx|B7GWwJPt^U0f>Y*=icr@u3~DegIibrk}|ugy^dzXX$2c(rtL5M zCiw?__uGVBiR3fQ856{> z!AIaHh@57CTl5a^AGsv3TRm@BFn{mAgneF+LKK=n{*DgWqLvFjT}DGxuE!74VxEr8 zMo5y5YwD#Bpd!1CV|Zd)d~~0Zv*VIP@CL1~Ivvpl<2(T5p+`3#JKJz4>6xVnk1_;N zGLYCmx~++OWto%wPJ8P8P^fW)=29?XEYPQs%Nd?C)}SiBr9=GCF$$<&F5KhfB7xL2 zSsJ`XUIR@%}xBe$XV(9RxYe9@9Y}x2>1yZ|8M_ zzVx*)Z%+*;-EGGPmd9VpwbBliw<_JZ@`T7!b}WzfKK-B{Gee`&P}JZ^6Ll8p9JcQ` z=~pSIbr@+R$l4{7OH0Yb(LmM?^MCf(!e5<&U7j$7Z7YDIIiVYnKmIQ3^S8ADzpuUA zwqE0X9`gss%3{O#VW5Z$9%YheoM$P_nenjuHKVf*#^R>aSK4 z6qESay!$`fe3qTc_DAxOJz9_i6DAm@;bKUO4nKrbHO&xmRd8XGyCJmL2XG=L^VcG5 z=58TvLFs>@V%?=4<|?$=(ueqqluKMS}Thg=LIh#2GY+!V#*q$CmvH`O#6+nSL(xMr+r^mYVWi zes1pg_n-y)qaGGQof{=MWJMK$jWbH|225-kb5`UpCU3UOJ{;{9b~0-O=`|NG04M)* ztUr?po$JfdAe@*@IMpAAm`WtvBLtBw&xjxpO6_vmGV<_U3aML-)>1Dup5ptboS_DH z&@0v{97Vfz{vhHWi7UayDU>XIM}!}ZYcs2@8hunKIdp5U>NmWE##JJ+(js%Q;5dMm z*W+mINg^qg0JgYM&(E!a|BMWF$ivdnm1}Z(>KCaT01{t-rdaM)`X>&1FJcH84ExbI zklhM1z1w4z;T`_JzHIXxgv%x;8v_Ezc8*jbP& zC{r~Pb0jJp$xO|BRCag}l*BO-S@WH23;Alx?or~4_H?wI9nF>1+X;tg>AOF!-|sMcpS8bb>T+R^3YpPjLn z>bK=)?jt4S(tYp;6K-O#9roi*Xj;k3{b!=6PSMrH1*Eo_1b>d#Qtz?P&&-`s@^Z%P z_p1(eNi~oQ2HB+41A^V5xQ%&KuO;eOi!*k<2rq>{Jo@1Nss$+n3)LT%Ax5@08-!RW zTeQN(%j-r|otOhQ8mi&Yb)FBd&Uol>;4G}kp&=OrW+PqevG+j=VL#f3vC73qh!HI4 z#C@EVkxc8Mo@*HX^LiiUeJXS&u@=osM@>T!g#Z&tI_#C>S5c9gQqXAeq_9D0OF-!o zcRuz=K38o!zlTP{9m~7yDT|(_#?ST3^;7CxoA%ODc&}b${tupaQjz)O!47gs^yM=I zc$@KYmUootCP8A29tgL*Zs=yH8n=g0g27^aE;&bXSeHJppuPWj@l%qw*L)gXzH z&L2OZP9j;59X?Ft&UlYLw)9a|cw3YDGy9suFNVpli|<{*r91Xlx9B_J-CqI>sXwz! z8g^I$h5X)-zdGq_$0rzs`m3zp-^Kl2`HTyC*VS$wa3SdmU51|*~T135Q9{tv`4tIuwXqb|JChfLZ$8{T|_ySL=4}laZ zdD7WC{&z9*|9w7YBxv??%o|Hx3Dt+lxPVRl@aK+;3inP(P_3=KXXfYbN2EP}`|r_t zjl{qXfjm<#o~#su0%zXzbO>J!K7)DNiq8gok+(M?kSNrkPY$sb=-~e|msil-@$E|= z1uEq_I)CP04wVXtw^Od+jsvaTntprm{H=P`kWmZ^#`v@te~sz?4dpQfPpYLZd=$Z8 zX+PQr=*Z5^%lKE7Hr6@UKaClyer4>!Cq*~JJNWc=|J8c=MM$dopJ=YIL1O-*L9hRr zI)7iB$Mfs&znNm@f+DX;OXyE6#-upU;u8s6$}-E3?AP?XGd9T6gp$*g4(Gb)pA0~K z9v}u*p!dQKg7ta+uTjqp-hfm@SqCjvzCVdDlmoN6I2ovh&xSeZr}D) zKrI+kmdj~HlOLV?U7pR((o2$6rVr^TnPo$bcVT*Ba?@5-rTo+GMYOyo- zx?HKr=WE}1k1Q#Q3$-8J&0&7^IrHiMY_g(K%qB}<$gr}6)Hb-{iR&73={+~`mOYwX z^&wN-;2^Dm7lMSH)p&?vtWu$9>@?B&f$wbT0R#N49x}&_X~u9ujjiPqJ}{jU(J)Iv z;mQg@V&xtUirPZckL-WVU?BovJ|m@}m_xp>7-G_PqT$-N6C1l4Kz^f3bgE+NP2Az@ z`E%NQ(D-zOXA-kFkziT>9tg&!+H0x~ZM!O6)T=6>WPhLXc$Qb=;^f1{<5$fe6U9tk zFOFXPRiNcDf;F}EI!|$r`E~to~1Z@qK z;yit2vCw4}(z!$tRNgp{Z?runcxXBl+vWP=Xjzwu`#g2e)7mE~y~2T@V0(kkk;;QFao^I;tC=-e*5N5}T&{5G{0pyU@Q4w_)|Vz__cf^R zj69ey)5NtD{f^_58ZcdD&?5o{uhU$cIa0BmoCO?ZB!WhEFsh$dVej8a-doYXxv4(D zJ}-a&yn4h4NQud#&9q0PLz~?13Erk&5@N*nH`ObMx7%E}k|Qsx)W$GfJg!JL65T#D zT|z}`_NC5JZ#Rr770Ns*RF)^^zO}gdR#c+PaYAo{m;Dj*J{L-1Z@FCY=!9vLMK-Jy zTFHz4AwNAR%@N3BU;+rfN0%R>Zn)747&E@90Msna8P z^m{bQ6|ZWJ>ikc&?uLdde7R-9LZiiG=N~5gUfa8G9d)!aH*TLHHn5lq#Qr!UMo_ma zoe4{FaES(1Y@w--wWD~j4u8#jc8Ftc#(J@L-(Reo?at6@aTUnHl`GL{TySCzvEZ*E znDX$1kQG;BhHYMyoNEzU%;f!#Mw@~m^`0VrQKaLk8T@NGlbfl##b0u<5NQ{*}_ZmuJ z(rmHUz))93r;ffHxxsh%Mqz0>)p#Z3ZRJv~C2B}ya9kEDn{t6l>}~T|Vsil7Uyp(D z{1`ugz5d@TvbVPn$~L2Zj5x(ZBAQQD=aGEq7Sxz9xZ-TLmrrp@Bb?&TGOjC$qulyw zPnOJh=hppiZ8ZDogd_(KiT!W2A*QSr)Iy7v@6vvBAiE*MCttv^pwA4^mTq?k+ig$h zRw=g#Fzud)kd=h_6Jw+XjOd7_JoVLTy~Sp-!nJ|0cxp-B*{)sLNV!)5mDZM|FV;lx zx)o9a?_i|w)*Xnl0-pE5jEGL&`hsfF6$QF1EFP|7?6{&*92pKU8ewt#MbOd1xBJ*X-!EFb9;BsZ{I@5)2TMz)z}Vb(I3Cp}Ip%wuyV|=z zY9Me!oG@#|<`l0MrD_eQjJikAp>~^OJoj$VOvwP}_yE3#RFl_@Z$C#9zhOd3WSK_$71fCBeS-wc$$*aYZ>>iOromTzehW=Yzvwq_iND2kZ zM03bO0I`56lP@VKtPnHzF>2?AwI)Q2#}yoI!q|p=wEV%2FXhm-tR`%Ucy6I$DWhVt zvKNVwg=ygGy0P)~T69_8SDpVjL)@spP`?;!DaN8Ox}a^kc%y9h=_ovVoT%@DFv1479vH? zuAj{LWK+`Z;#D{^8}GUotuIOVRro6&H>@+ZEnJE)QJk--yPEVbq98{VL6b{A?pMs< z!_~b#x&H2)^MD!7`J!XJy>(D1G&4i0skt{80eRG<)JOE}**76hn$#AgY`=pz_?a0W z+XAi&?z$w7I0_%GNx2G}Lz^16C*ur4U>bxJ>a0fJfP-UGkmOWq9YtGEzbLwwXnDDt zGbbi$M#UaCuc=PZd6-;~TSY&m$q*T7sIEn62vHzKz6VqO7JboO6q}HXN!33fWe=AN zuGSPxT$SZe!hXIH&K^ITkhZ0WkVdwE?y945~e(&lq0a&=+H(Hz1KU|LWnSYx! zo(@?2Serw|vKwnRbbEsmF+MR)vAUXWzqRXi%GJ}LB4j%Hcq1)w=F-6qe(gYio^*VA zLeO-IUNzSaPYGkw@P}K;QXD-ivf(q}1sUtAV`ONkEPitNTWWJsI|-Pv zUBjDBm$o$q7!Xq7&eg{aA@xP`7hSJhcGdsHDMzo3KJ(Am z4tUl!7-lV%R8<*wbq@?k>Co>ZPL`~1ZE95z(tCs#$_d2aVLCqSKdh5OrgVPviiQYE zMhXbHR#8Y$P%`c>656$Aqfd{0P+tWvy+6|=D$`Z)&&DJl$~^sC zG;smqxSBR*Mwk-EL&`GJ&(QSh+ce-Y6H>9Q8NV|4H8?Eds$;n-63{S#tz)xzgsZBrc!Xi@M*`chUDxK? zSj%brMOuP?L9X?Hs|^0g+0*2?4f-#bm;M|-A2f8NA4&9k%x-6B8ns^pV&EXvf*_H8 z1HHd|u%$_#ew3wLtIDc8C0d@Z>Ogk$gk^nQ@FWJ11PT^Xe}pn!LHKzd za&CvDB2o1aZa%GX;l3Y<4v&UGp}m)&JGT~=?;02GBGYEI&Ohc`*?ozG3-%LRG;^%G zmbr=l+*7rrp{(H{!3k4>U@D&yy=p+jpIJtxyg~J zz5a|w! z_g3V1X4K!Ih%L2y?^uLjnYpNNa5>uusp{(y5|H(a_%vB-Iy(#QvG-T0NR-T2gq2IEk z2HX~&tDZpnvt1Wh&I2LDqkw3SZYooJk5Yc_@fq?rO)ui+_E#`DB)zfmr9F52Gk*Yg zD}ijW6@{M_Ap#8_RpuK8#2xa%eDw;D#4(3r;T_^44)979%;S8suEej`SHhGDP$ zaCemtWPROq<_fIZ{1e}`Hr^GCy6Mrdl)j!}JgEsmD_k&>=^#b1u?Ta!pPQGs9|ub3 zy3DGcRGv#C==yM=f26FQW_xOkh^jPSXY0dlM4#7;hLE_9g6^+RqRiPZ0KF55~Qj1e_T*jsHa&n}| z^9xWr-OEi_A3{=bod}P0!n%;M&>$W8F?vT3CiOV-_h{%N8Z9?sSt>7;(NLq|^{Ppz zzxQMcH+Kt#nzJmf*jZ$~bbF z8$C3dpu`^fIf+0v+S~#zwqQTZ@WMh$?b8Lh@cApGrr)UhZfFn3+S{gls@{|hXcG9 z%}w%WhYu!K^tz*z$kY96vSKH5N6hxkNKpf$L32_&Bk5cHQEc6D^QD}M$*rA*WlrR| zJHfp&x^2aWlX>=9w6-GZ>Y7UCgAdX!kpV-u4PAa_-9LQs2=UP4a1P)6XKxZE;5Z~y zZ`i<*?7We=hSIR~e-e8fo_eB?2vaEaoN#|CkPPvlaMbB_7mvsO85&W$9ZwJ{x$a&=awE1;@FVUP zVoM6AON+E-f&`<;s7V^d`7u>fbT^4`J!)`A6LgS*4?ortFe?z@oeke zA4)A5tc-!aVC7sDkD{rfmhl`sT&gDdavS`KVf=4nkxzT%y^e(wLHb8x^49G(JCxn2 z7P$Sf!3|g}=1;!j+Yj8NUBb~aqU@=RKb!LF2G_eXO+hd)9n^gao~jfxXSV0NJYf;A zKD4K@1@jO6pzo$FS+gvaWvb++OLypuIk+T*c!iNj=r7Fel}`ssP-Hmr7qiHgH|#zK z(cw}%_nE1?Pd#U{VmcnWLmPxh00gAs!6H)HBMI5LGa2c(tT}GBG)zZpxzxGhvP<>%jwkMF$J-0f3QX!e zi!1^Z6`DhBQDfBjPtVSXFMXA6vHR!sj2t1xux{Mh@0BQ15GnJ=JqDH6R0UuI<6RwH zKFw$@oV$hNP&dUKU4Gtpx)QxnYJVRSSEEi&5ilFotnmkIE&`o8-n{u3WQU+6(Yu4Ns?=H`-l$*ai| zl%}vsEW}r-iqO1b_A0- z%;Eev9f1uuf(w<>a_-IhQss%lMgBQnN2WPDTG73>6iC2U$x=IBh9e~0r*VBTFdoo6 z-`_EYAIg_WMI_dr&gUhNvf?*zsN0$otxKaC;JK#Ywj8vlD%=6a(w`|L2|*)5gA>zY z_2lpeqBP=Wy!Th+$wUZ-=rQbF;kqg2oaMop>CkAH4x+1NNFEK!mj%R;jqb%y(rjy( zqa%xkqYl}c=lK9w`B+b?O@RLfVSVD}IHns!VDtbxDFZ~#?ELZ8B(pu+-_~ABv zYHN%LHSb*}PmIqcI-!rZHmo+b3w&O;Dep$EWe(`h;$Rg#8}hLTX33cFrCVGDF}}g;?iA$Yt}lKS}8ZZr$^08~P_DzM3+W zAtxByHz(g*=bIaw(_*6_!Qu&WIeK=wJT5^H2uFE`#Q9O-v%eX*+2#KnkCR8o+R?iN z9aoifXU2ri;ts1ilaXOnL&;A{PuNXYoPyx8*qh#{HZHR)SEsR&ZVuz1Q)mb_g> zRkR{balPWz%yMkLfc^}jY`x}k1NMdM9V*MyXdk`G=&T5KIWb1`C0)8$Gt)%S+yLP6 zaY)EMRmnuyJg(9;VmXUq>F`i-@#W^`6}bFqO3XN#DpR zT=a#lXhR6 z1DKP?L@>gC78V?6>qk7glQD%;W5A<`#K1^AbN_X0r{;WbZBw=+j>pkJ<2Fy>Z}*)d z)1;Y)Eg=BcwI2x$L28bqqmf-W;tbE`#tvkTd>cR4*$3cn%|AZmpXjZ*c%gu8dYvBW z77qNjAQL`fj>zknI}}a@;X!8x!ySz?pDQg>7KZf=Oe)vf6iDj;WB%!?V@i{~SE-?B zsxU@;dx(@-9u+0uk^e2Ey>v$UF~P>j=0Wl6>YZYzT$aOdjh)$|W?&bxV4E9L6PasP zpm3)5D@9MiJ$#8sBaISekbG}(xM(¬xBr9NieLBq3a6|LlOepbRCSvGRK7_2@+n z<)r++YZu7Sqpm!s`>Sq-;~P<>@INDpQ%M(~;ldc4{1kD&0mJlT-bULs-9|fF`Q??8 zJ*o4x9FFS4GZ@Cfy)BT?_9o(1G1rGVBa%)8&Mk6B=Uw4D##7P;AP+`w6}_%H9;!%; zm0YxLkQW*sXd3WA#!aG&gW&>)DY;lULbVs`A=A%tMR8sGZey3Lw`4G392$-rwf9(J z9hjoGf?ATwQaOo6PToP>3VNfCw6)zbJX{QDHB9k54Y3+u0ToQHgt^py;__J0elF9w z3x2TvWCV~Iwtsw8{xjkbNCjIpWF(?X?Rov8YwzY4#h0|^pt0s zV$FAXJjpoVDvbB;5qEygt|jSe%@!A^oEa6Y#5J7n=1Uo%m>|S{iQamKlmV2-K;KmI zJD-MdF&{3ue=hAA9|5U}BPrWZWLIp{G;&+Lv$JdTn$N0Yai)1O!RdQ->pnF+^8-fP z%E5^)!5Bq$LaS%=xq0@Mi(bTSxv1u&V%Lz~?)B~yRPJU_oK z^l_Q@zR&r7` zT!wph(aF7sZEd*-iDolhuWEu*-1PCOU=$8uiH-d}{EH%fUc~qb+cUq_Fm?>ZvGQuf zRYcPt)Rs6bw*+_2UdJnewPyieP;kh$Q3u3x=Ln|hIHs=a;9@eQb*W(4;|U>%ui>dX zOZhW2kBKLy9=wVJe4Z*K-oQm;<*T#*K#TtbR@=0%gT!3#jP*Xe=;Z%=ux@LBbIjR;Wl%(`{OKlE9jH_Efs(0d+ zwbx&PM!rw*ZU84C47~WPSCFSd85uB2#7xcV&U8$6#rM+cG#(tx)MOh7B1^3Th=+l2 zU#Qbc%nu4&J2AQ$bp@adxf;~e;wiV0Zd&P2Rerff1_#GDNLUEI1=w~p6lf`m=BW24 z1!#|8JK**nH}4m=K*i;1DJirN~n;{7Y4_#fGRE{ z8zZElu@ED311rMQj1##9&ATr3+sI>Q7Yo19VTXq0a5NThbRx>qxP{w1m>CzES(0i9 zuL5WbCBttww5mKKIux`fJsL4x!&9_)`pVq9rN1t=VF}|vI6cWya46g71stRilVe6o zd!#vbEQ@3UESyucR4Q2_Wcg&+z5EZuk}CMvv>#+0{p0@R2_m&-{wCr>;=|hN#}pxJ z2M5|{oqF#9gZuwbS7eb{ueT}L%RsXu$i^2$BUz&%=O{E`_6_cz$OYvDBKCP5eQR{0 z_M<5+0RfECSX(dZP;h7?x!~YFXz|qMdchyC{`G{|sb)!Te$p!qgLQe?h|gsq)D`66 zv1T&1R+GLFM_NLk^k?d3+H_*~v`kD2&yRMypQ&BbbkT#fd>?$eIJ+wA^Ce9W#-W?5 zaiIQu@<`elH|+1$eJhi7)xOt*|^!4}(xmXcRx`HRWL?AiA14XVBI zgB%)(FypXP*Pi*(I=boHx?yFXc!`Jp3^sBEra)?-48g%Dla5l!F@zmJllN|(4e;LW zIF7!xEnSzW(X7ee=z?@9{jzYVUeZnZJgd-B+SjcfeItFi{$s#le(kNgZJBS(T2)n1 z-47u@7cZ!#%^2U#!ug-UKX5uwh`^TP(xGdcjb_60DU=t;6%K_G8zf&Y7!p?RYgX{P zqRnw~j{Q<^lLrpg8zc}Xd#b$snLy?xAk)+={zt>bH>GPVHvEg^g#Fl)E+oaA`4?McfJ`ww9}p>fYMQV#Fmm;1leg*b9V+ zmuvbXMISrNOtuiK`n*OkuzB4m=i=)JI$Vi6L-o)Wu)Od_LuK+mo#@t0VC>U3&etS zQs5x}6ku+OlwB|{ivrN!*dn2YLHeBAcjMELH}N5v6l(JvF@*@g2)fCmPUX<9qU%si&Y$>y#&> zQstws2)u@Bw|KEVyn^TvAh=z&Xh52AmD!9I$!hbw)e7*O@~Y(mR4_6r z)D+z3Ma|0@7HlpXe$#-EN_bj*w0@&bH-uv`>X<;I_AI%kg%}9h`fR30QYjXAxUQmI zJlLU70z+D!?vU3;S)g5;yV;iPQ=NUNFg~AOMDN05;FH4^ zF6PA}j**kgpY{ldNfx*D;8{f*!XM-KNLs!uWVkR+`Mppexoom%3n`l0^;e(JWcvNn zGKJkeO(KJLo&{&&AAw4z@r~E8Fzf<&WO>zAn-CoDLh0G)(xwLq_si$j)NlGZO*D!` zm4oX&Ireh7Q&lndzzu*jEaPptQ+6#;S6M(l+h@n(SddgmqBRU4{6w^AvB_U%q(?bB z*}p^~?d?_^ZCF>7USC=k9aN|5&KT}oj>c1aR&c=75DPLBp&Zfmj%@9MIu!}@eS$@r zM%$}Sv3|*X&!^3raATct9K*BUrI@l@B9t>5KkTZtHJK1y$IA172gD*i;}Sq9z2knU z#`0pW8W4|e&u#5oT#rKhmBswh)#rZ1b48qpAn<)X!fgs8nV(`5Yy^N{nxjoNS;Zj^U#&gk z7`HPzvuE5FZb?!`63HbpX2XN^5@Gk@rtoKod=(p}6>^$_Zp=t6dbKJEmMSbcf)!D7 z;p~}&1Ot$2N0$j`UIi!)ws1|}H~28(j5^&(x*)n)Uw9bRo|l+X+RCS1n-Tl#aB32q zh$963rCb>$(j$5l14mDD6#Zv9i%RMpum0XxUl+2we7n`yVL*83r^77-s74IO%>dUg zk#4q_LV?7r){%nzC^9kvt;ti}aNuJXW~@HGqWK_?t?fx4t194!-MZ^ZMbiV|YCZ;N zFC}ohKpetD*{{DsEe8$bD<{GOKFh&v%s5G;n_4|GSMrp_k^(HHLp8TK`(lI>!+Hl% zs8p&b8a0X}6ckQAQiU51=H+RHQxQd1X~1c7h83VuhAS_MRg{Knu@-eDWBxEwqtRC{nH0_Z>Uh>;>|rJSVLHXZxP)rst@57KTLG367T8eCT?)f3w2>n zoSqVL#VggUqeE;N6H$dcJx!dBxA>{r290*f1V2Lj;F|X9-HS-YUef#Q(z;$%*598* zgney~6{{i8c6UIctct8rnm_LO0#J1VT(9C0Lgu-Zw;)Bix_$*&pzB_G^1d0le}-Aw_c0B*tr) zp7Q%?A~!4;jnbOCl(ypfVAP1*HXFM;4M;wbL5-Ep4~L>?FvLH;};J4 z^TzoJX%%J*CEJpR z$?Rl)wr;&&kZ1yGH;8_|G>~fw0tH~a7tLYNdyZZuCtYP-AEvJ~pPlB7<{#6wlb)-y zC^fHRH&gi~o;)^|7|#d8y@?Pu9B~f_4w*5kjhjQ3&u#~k=yPW(`6<|hYJ zL~;V# z6oG&E?uAu+Cu*0d`aXP(Y@4%jRM9Q6l%MQlbm|oAbE)EKyi4`F(MNxxa83mIN6y4s zy;BrI`JW@qUD+ru!ax6`#%_-SN~3VbdET!`ov(nG0-D3P=Z4paU;!cJ2@*vJa2N6V zja$3qi`{Uw5Ryzbh2m~sx8}IeV+{PsXPPGv=?JCn+aJxiq`2#AU*wr{La`sn>$Ky! zl*{@eoD}bvI3fIRq=mQ{8~LDUpkMU3ExG(pe8h%GQ*PDqaYcXjxF`#O(mRBFtcBsx zjg?12yJE(a{vOZ$t>@8Yn#1}HYsN2M2JXoB*Wgb3 z7q&&KZ;$47=xcK{{@o{>661pyt%(J+NI@& zoO&fMngLjn`7jWdv3IPeA(m2SzxoFbzE~jZqx2Tyq4FJ;oyg6{Pk&gj7v`gS2WQia z!86${#ebsM$k9#t0sVMKVDgJ;+|B^=tIii?SN|uqbLG^tqfMVBfKN+e6D0Lz?S^G^Wf%6{FwnYN#Ey!1wjF*i5lA!4SO@aVD6+7MOXo(MCs z=xN5-bV2Wq(C8T~WG<6gXsQlz06!lrz~NhbId9%c@ZbD4DOrXEXJUUU5LmE0z0m^5 zjz_io;F7a4lZ5+ArPT_L(-c$#yEJ0DPgk9Q(k-oIfkP##*RMX%;F~2OPqd&gdvXN^ zX|v&YPKOJ{2WosFwFIJ9*uJH|Dz0HDmUH{BJidJMb5{P~T95g{gx|-#3M8dm7VEkG zUIqApryzHpYMzJV%h2lk@5qYxE?>vUuy1Z`T`Z=0zi!Xw8KU9ZXr@&a#HApKJ?mGW z)qwy29CffC-qj6iNJ2~42##){^m-$5LOjtoy8U1qH^TU_A0orCAZGa^;+xQT@3n9h zG+&0UY<&;q4*&}JFOc&*!J7!}!#d)y(QSBEqfTnB*jTBQ(dMDadh3EDAmAma>nSz9 z-q+pyK38MF>aW91ytgDOokQ12t|++whV!xaXhZ&YBKwoVx@^nF1-8sWh;^;#?Hxay zF&3QIF((xPH&27i4NVnC&#p`Xc>#P+SQdt#3|_E4Lvdi{zZWpzT14{s^inlY%DZT6 zGF0d3n|5?WDdP8TF|io`cyx8>@~je4%1grTo**QalUU=&KDtR&l4(oYQ{t0h)573j z14EuQOd*t4KYC#ZpYRT_RY|3F_K+7;u$gGr|-_lV|c8h6>5j;R%zG z+0N}?g>K!8s{0eS8W7!Xk_Z|uev5(OL z(uVJaUkix0H=*8b^)iUQ-Z6>n?HN{ZREYV;SK3ygHG>fm*Y^6Wx~7Si*ov%0F4CYG zeQEmU`1N8{nfeJQ_NnNYD~cd@za;k7%LeyDTX$%SaIfO0PBgm473DM;_M3KPYbLyr z?2g^?>jzrvjEmKqt;*Zmj14W`b0U3K1q(PfZht(LtDDGxPaKKm*L|^0F(^XN;2A$1 zYjGMvQ^Kf*+kOtKoG=?V>eBT)Sh&gyF~dtbM&ioUTyKNt?lhmNA72K@QF)z@qLwR zy`$g>r^C5PHmx(KKP6}wxcE4cOoUCjP=8jH#%6^FkHbFt^G9$EGz1_iiur!Z5dqsI zLb&B#sAZ^1r7B_vpAi=L$}6wV{fzGBsYRWxs2=jme(g7IFD5mZ7vq2i^V$%qw&eQ> z>m`7Qh*FZ|=H~(;^f$`A#l(q8iALIu0%=kwR#Spg1LEq!HV^g6J-rHi&>QC*e@3hU zUD0_bznTIC1cyg!=vfu~Nr8MM+)JI9_X$ehPH%pt#v^k7TRvfQqDH>cNYHAPCz|^( zd9q&DH2M~z)KWPf#)dFK-3WtlPKq&>Y_elel0J(YY8g<+mIJW#u52Q)kk>EQ(t7BP z%Qwny@hnNOkvU{Zzt}8Cb~C#=LtZ^^IaDJgDTv_2 zQV^kX?NRy;nBF7L0wksFDYIe;J+q~F|MxJTitr=H-rvI> zVF3|GYSV|l_;tY4obTSCOp=5@-ld4*3BJt@Qv!i7|C!78p10F?n4ox!Sf2Syjr#|mT7+d zEu@s#ly-vLisGx#FI^`+2FSS7oKw?ZyunK9Z}AUU`0vN^d|Yi%S4W~x=bV#%DlgXR z-IJrrx3Wq0Pf!%IAVpT9{N{LJG=Dn~ae@#b6vfbz>c|`20}OR;RS}=DN6=qWl$#mJ z3AKvh3mXQBxpKSy;$$zW)pYH#kp~&z97z?T|t3MH549-6T^*$IN zN`zcBexd)rzc2ao72qJgi`e<+ju#jI|9|~GzBGHbU(qUe@GIa)LPS=$Oi<7F{{g~t B7{UMm literal 0 HcmV?d00001 diff --git a/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-deployments.png b/docs/docs-beta/static/images/dagster-cloud/managing-deployments/isolation-level-deployments.png new file mode 100644 index 0000000000000000000000000000000000000000..27cc815b9bf4439621d8b92989a8d5a508a7780b GIT binary patch literal 82144 zcmeFYg;$){muZtmm!LA&lbXRe?iPmR=xK?jAl~SgHBEx9hRD z+pf0ia+dZEKuM$?LW-Xc8AUW26A0)0QEzKAxp5DGEDwPA0-&LnBG(BD2mlD=&vgv$ za{?qw8_LLapC8X&73A+BaZv%35F4cJ;fX%h@3JdgFihS63Uh*2K2V4=bvdEsi$_DE z6C|z2QN1vAiN=QKe|-=Ld<2<81HKRMh{*v!OCU!>76zCQeiZ(Bp<$d5bb6k7KZ@;3 zbfHV%1zg-@Ob-7_+)V}uZ$9Cq`0IisDzA`?+~BdDd`joy3em2|Q2iKt%hgvG=2tbQ z$(<=y2{baqJfDsJL7vewPgTQ8-=Csa`~@mK8P^&5SsWisQ(TxbewpmrdH7+wP3;9q z*j4vujE!Y%Mv>N)gFXq!rRKab$qa`FR=?yZD`N@+Grd*_^0 zFNN3!!@#BDc1&&%)f3DC1VRffJ73jj#HO^5Za=V$k_M3xQVrJ;pLl?mI=+%gek0gP=!N#?Css)5*0S$`D?mdj0OiHIa4&Qud4JtMfzp2zfNjI&dzcJW}Jj4A_S z&K6DxfMP}!87|a#EgHadB78*IfBA6;P#b@-J$X#^&UF(JDwqd%+=qM|Ca@Dp8G?Ns zdf6LC)&>m-6AeP4h2;Q2S<>7XQ3GAYL>jwhD$$;>3;SYFwqsa8(sz9jys0cpYSzjp8#e90A-m9XN`G{pP z6=EzR&hT>cQe@eUV;3U20&e*H#?|&ehWieDz@PPzTxb1&X!xVcnzR-8)hDt0-Wt6X zqTJhT{f9NCJ4!q3T7cWS(m4~=C14}awXZTBCmUb;;!doRI zILt87u5Im5#+@Y9VfbOeVGRjxL#(i$OqKb{M&=+El+;j-K#jgsRG-!?!wy>)uygg6)Z7)-HobMENu<<6lQ&d`&%g}O1eZndGc-UpQT zW$cPa6I;i;1ipiRgfH`d=kMA5Rwlq$%$`CBLoTEw1iJ@zQ*cvnlN}>OpU0Mn4iq8+ zlZ!AZGie~paDb^2Z_ay5L=m;Kjl2!D&2kBK$&L`9%Xg98BUerNj?y9SDy|?dQjD3* zj4T2}%OqYlUN&};E^lrS+Ca~s{!(|xeaCiZYw&YiP$6_N{7lebUU{yfjFfCk5eYRj zwpzS&JW~7(b&z~{AyVO{yrlfI{C%-t@uU1$$?b$RGk*%|H(@pYC8-vXqXNBye0vnm zcug^GzO!7^lA>bPLcx-qVuhk~#Uq`}x8z7DB1FiNS_P|_p5o7b zy|l;-$UZ_LLOHz}-^NlTB)LbJM@CXcQlx6fRo|#0*4oxOnrTeqr3hy1u!Mas(VwkS zL9NrVKs>QNF<5-8tuTw+-#>)j7qNHTPn*M?G}w1N$Un54;mlz#6rQ>o$2_o@VK2Fh zDeBe#IF*b&j5w_0Ve!o4(e$GE{Pmd;O3shckIzrYk0MZjKS8H@L-B{>4;>QD92cRX zZ-?Iq@G!$wdil0adhvS+B_t$uC1z8YQ<0KPlWZlfB#$HsQ)PxZMiNqHl2#b|Qfh{$ zhMkhvYQ5>+)A_$qc=MT7N1Q@DdWU|fd?9IpzUF6*nW2+fblJS(g5r&0+bq%%+7aK8 z;nAF3_-@bG0!M+~=;wFBsrISQERp647JlZ47HSr2iz@RM$Hm8|^K!?}CrS$q^Bp#o z_Gn*@9UIR??9%ocXRB5&N@8nbr;EBw#uxfoCtM2bK8KAW=#?31ICeEUx0Kql?pzMc z415~8Njf65RJ1g)3|fZWO&P@=Ug2&STj@+cYM*QC^E-&%ujP^KM;jbLpVq5NAi};BI=Ylly1Ks@IoJ zMr{=yH4yoSFV=S2pM`^dY)#t)`lb0PLTdoY;e?@;V7U>ifYE`qqkelZBTGzY7w%j4 zoA)DGpg7ECvTM;KlljR?tvA{@{&a<^=N!u{?k>*4&~&<@Jz2pMB4_>Iq9e11v$thV zCI%_-V|U{gqnr){S@w->y4znR%^7-@$%K}wfi=l^sO6Iu{XlvF;BAS+)F8YImaMf{@rX9zN#!Gsgo)WWF z#8HY~oIu1zY9WS+@)|1vF-U+bTWlhAx^wz)8r3Ar1a7i-N^kPr$8+2eT#BE?o6Hvh z5-Y>Y!}K9}xyq9@(+&sqc|Rt#P0EmB=wj;hC!%=|nwKSB(px_-k+e`h>V35zy-<%= zw>iDx2y1n4VYqz0@=_1f>{Xj>SNFO<+-akkr@1ZnS0pbp*D`1~60>6tMp z@uI7xqNQ=~AUBjWi~cIZE3@VVu{MK4gDoD`GS^&I_CoO7w*NTlaC_uf`l#5=(4=ZA zr>gm2z-je7Lul?*_)Eu~|1xm&+t+taR^65uR?Q6yc2`|Y@zqb&?Q~c5%oYZgbPErj zOEDHogZP6b4VVo>4R`YX^0mhw7OPD962)|OR;bU4d(#)$=T4p~!#a*;Eh|=#XV_-) zW-{)AZ*++IX(41RFx~1)Ysk~L0tHJ_uGJDm}Z@u!Jp)+ zts*E-oI@q}`o;Y!_$)dBwj{9aTX^qe&U)d~{!j*nSF;oCFXcy@*^9u8Z(ge|zK+?s z@TP-ic$RNi?rMAL?9-9h7FG{uT{m%#v{2F}xfL8#Pp*4E_IwmH>K)t7z&DAlMNSuI zottaVeV(|VzjC6dZ5(%D{I$6w{*pNARLMT7>Db|Z7RtE8foTEVJe)_m&x+*1+J z@qOR^(7buWJ!dC>Xk@m$NyEs}WtGio>FzdRl`MD!4!G zbPKvWA<-d`68)dm`<k%zkPpDw` z0X~ZWR$HpkLQS5Dxr!;_=ewKf%$iu7R?sp`;{$ z8e9efAR#aSP~Z{-_!kgl0)YNg1^~#xPXGWaCJ+Dzeqwgy^g+>t^vKXh4pU{0EaUh zxM*QuuLE+nFt@a0bLJ%e(}E3L{{5PP81$!!y%{I5vZOSK-^$hi#6tgp{sS==0tf`+ zu+=wYlNAv9mpJ%8PGVzwduuiZ1}7&cdZ&-{R<=eAjI6Az3?G;nn3(9mE$HlAEbVoi z=`8I?{^{hu`Vla&)3Y_Pwl}e|1pV$;N7u^1o|BmP_dx&s^Uvora5nk(NS1d0nihD1 z48Lm_80kMS{NKdvO$`4ZV!vzt5&JW*e}?1u-5Hy-iL-&Zihv2&L)d{o8rMf=R*pZz z{I8mSf6#v<73>Ub`K>I#g7#ehUY36e|7Ydjf`5io{`ZiKjGvhPbIkv!`VZ0Hb6}IQ zH31K&^IM5rj2sOA>)pTFb1?kY@PBChpXL1X7QCKZ2pkOmZ5vz&2A_iq0RSF=r~t3L zGsLk5oHLdJPFtM+IFbLVTkL8Hq!_kzRK9mG@e>F^u@XedA|FL9*zB7ro2PWp<^*iR zbcIKQd}ViX8PZ=!IVM*-eXdTv-HrcP!~W4|wC(c*i2*4&5D4k(4Fdh&9_tN2(%Yrz zflg=$ULG_w=>O~EEd-JGw!r_R5-iaf=KZaU~B=!I50{V9s@BjZU{?YOOM|2@_ zxVII%0sp^@2M&F{QpXR@%&F~gFrKERJ@%OBw{7tNKp<#oz{$@cnJUC{f{%qGY_1e~tthu=BfRF`Ov{#dj1J4il2 z6S7#5$X~`Oq72RRVBAfy+4YI7_1;^dyMQO4dpF`usdPAkA~PLMsdP{4sx7R8OX|W= zMZ-0ABmU~OzcqWSONJyQ$|0yQ&>!hCA&24_4f+WYY~D{t&V$DPbN@i1L7;I5WkR>< zeEZhcu3~AoO;EC8TDFmU5j=K*0HN;!ZJ_ekM9ztzQC7LX!krPiiJd^jM=IU{54N7+ z>|f3omIq)+414!W(RdlxqXLvc0Nt%bEl>D)Wz8yY+Eu5CLvH3{(M=^_rZac#WaeMX zhUNqvUQ8GTC$zb~0^cTZfaw&BSqeL}RYZuwd9R(M?H3=&8Q7W0>zDMaYR20U21W*{ zW(NW9-JMjPWduRmb2e^CB!MWI9QE~{qwvUAHkuXmlc!MAa)KI$!@ zPQ}-t%h&4|h{aJx_bpu|z$!{=2zlLPNN|@D@={Ia2G3dIMt;P5gF&geaaYv4emGOC zQ1aHJ*9rw`vHdAXm5ul>fBQuUz~}{w{P6Sm5?iY{ zV}CzrI!9mlRJ+4~Th2HEDT4h7ij08MT>v#Zn~G9JHnmOTlS31+H@Qqlp6Rj*rs1*> zv4{vLHHcdKvvRYW$)>Of;Tk7d_?io6Xrt#H=256V^cg@I`a;7|c!58Cu5_ zh?#tE_-yBH^rh_44O^qbenhhhdWB_Y=;B%~)zk6ue1C#}_2s+^s%3|x3BB9{^r#Ko z4}Spgd91ov5cH7DT|)L{B?$O@OP^0 zjFp)+98CNOieqE0lSH!AY#%t4kh#m*e_5dvi|uk$DfD_1kuv1a_9>U0ObYu_zHN3< z{b~+j@rxaDBAr43t%AkdNRU6wg#%8a_@|9ge0l|yg%qV@wAs1H? z0-(dWG6#_Oi@ke()a~`JT+cZ)9@@&T4y1fh{@$yIi#%BSvP4<7a^SjKWabnwCkfniLH=UDlQG*d@u8u}pg-rVaN&cM^jc z>wVDDzh?R4%A~(0K6L2pno?^tKnCQxr*PCPHU1Q$WQTS76%k?1)o<#}{rWVO?N1Af z011R<*p||MZ&BEk2|H=2Q$^;}14H{^6bCo&f9|H6C z7nE#Odz3H%&=YO!PA0D@8iq_j_V2P{i5f!<6Ssn?dQsdemqsLLVpQf)jCa^J%y)?$ zRJ6iE#dCS^cU=oCSy-kIp5hOVH_~DmWN|Thz%0rZ0V!S&cx2)P!>MT&a&wraB1P<7w^p}bU!b(qrIwuogZ^j4Ogz1#}OmR&CQH~$V zpo0|zt7D~z0iV`$yD(K?$hRY<2)e&G;E_t3cf9fee!{yh{4eLv0om)_(Re=x3p03} zQBYHze>9J(5Lo;{Eba^XjJaG;SQ@LdEY6O}DiaKqc)NKVZE{HeKb z`6UwPmAMK1p=oOGL$n1Vscc^UOwpJxVN-^+;JYc04@R3z2@Z}gr|E9gs}ocHVILpY z+8txV^28DcEiVfDTf!L;h!Ec{S3bNJ=$_Y>jyz_0K(}f#xzc}gel&7z=y^9?u_`R# zdJxJlOI8pZi*uu!Rv-q7F%rl)S-nS@5Of(gER1`lKOv|mQK8j3=pWbynwZzw zsHk!7NaQv*S>>*`$o36F&Cs9lbyE%-Te4BldNy;KYR!60TCz6Wy}epuS~NNo;hivVGED0BKr)EG}F$ znMXQ(vx7)DK33r=X$Skt{}yq1XVUPiR7flrykok2UUEks@RJ3w)Z^%ouI+6}5f|b0 zJ$i|lK1BvUz^7A3NJ%>5oxlB;QJ^u#eEeAlv`@sxV_Bd7xluvutkC8~UgMj2v3pG$ zp$tlq6A+Em9v|wY!X0`8#0waxl=9#ADoBlFuv*9bKEOVy&SW18+9o&j)l2q}fqmvd z66?2Bi+uH*h^fh{t0MDfvA7y#OFOlC!%WhLJPj{4LeGS-%+r95(R@xCr^%Kr?v3?f z8gklgXU=_ zl-~0CxP>Ysus~aJRque(*=+;;46VMm_-JfPqbPrJE?AJ$N3SPCMp0atJ=IAlr-D3f zIOS(GoVp5a<%WL@HMR#Guo%P(I9Z^+N{(*0WzO%C>fkR(+v+tpMeiqxvwI zFi`~&{?jyn`|h_8WgyYy-gd-TgU?YDG-fZy2_+*#B8c3)21xwlyRa)M$#Z(-^%lBo z@6gGABJb9biN+)yI(SCoIMFU-D=noHthw|^yy(j@x;)T5CKtC|mJ-`UOkoI;cJYjN z+H%DP6+En!=Pfzk&d#J>OBY#d3pWsMbj};0re~9+-cHqeVjidoV)zvLbe+&O@sicL z{bWgbxgmArESbn>*`bm19x7A3NHRoC6H*!#Jg>eWR9{1bJJ<}ly5_&)l6cW(4`x8q zuKI+JNLv3m_n*N zz#5to$2!!@}_tU$Pq_kcwvex`;vPT!jp$umKBOY%7MqYE}>p? z*bsHU8eCe_I@F24)-e8JyMcPWlTBDO)(s&+&Z|&scc4g*>2O9s<9v-mC7ZXsP=ZrY zxmeJrADa!-pYBaI+aKEF&h2=R${#YKKhT(d5na%*l3ua`CikUKH#EM{0E`F$8f998Zm8hn*?0=|#Z@KWC(N7NCw%}&4lTX@or1~se|il<9>6JQr7yOm zN=CRw`Ft*$(Q8TpFcqHgMNDawtBiv&q3`LZ9(hC2HkF)gt!*A(rcpXF;u1Q$(Ly7P zgl}A-0&Iz{viYI_%ovAK%2AW`GL{z{qk9zWOM#-6jp{G?tfHdXBV^rL12dw{>sXub zoaT5^QK40_9AzO%F4o*BQL6;?J6f2YorcPN5E9+m&nxeuqFbkzKCGr7wuKy!SBs+@E%+ ziD?D$ih?=0|BNTw6d3RF`r{qyIWx;^X*y0-k?`#WK6GhxSNoLPK~W33@%)KF`6Moe zvM!+Mok!8%S-LeQWbfx@k!K4NVD7TTvM}>OtW2%N2KkGzQ*Iaqy>y)Ij_K;7Lh)oN zc{;m$iX+`pJ`f(0i0Nj?#y^s!r)NN`9VmJ4PFv!j`Ox@n6-)0>EY%u^|M1Fa$C2x?X?pD`l1dq&rhUDesb{nLl$))Pu`%}I;AUJb zW)pe4Vd->JrKTLvtH6qg=ZGX@6CgHAFW!=1rFU0}Wg=_sQT{Zw7eJ1nv7j{gfC7m= zp>(e<94Yv`#okB9<$5@1<&DO3VIz=>{6Ec(7xV_WALL4R(V!^!Fr1Z^@U;HWR%>&s z%2s#wB^F$E2-$usVddWOYe z2VZrUi?&_=Mm~`4YWldi0ZuCKh8ik*aK<6Erc~7-naQP5$3&g@9OW0%;MLUW)R+5P zRPunY$BgLQB5rHBdo*H#`k}G%ji!u?Ev6I!R;-<3geS0x_cjTiDE%{kbrl%bVZ@Qk zsgnDzU&8v)yj7n-bJ~xWS~}D@uhrb^l>l^=(J5VUFxIoScwa&J`0&0J0~s61T$S0r zo3%PGLlL*y!iF&Ng2JHE(%DN3*s6P+O!~{4fqzWYKMK;=(!PEiV&+*G(DIZM8qGxz}G zlGvi}*^--D)w=Wav{9piKba$pTEGOMa>_8Y8p&hsh1lw)KgsU|0?e%t;kOr1D-@4s zc)gg^h{*sJ{DoNlM(Wx_ygw0>(i$0d_a$^aAI5e($e>V6R;rc!P)!v%IF(R!GP`N2 zrZ-TX(sAOoDh8*F%7Q#_ehtksim)7zi^#+1Xc*9o=zP^0f2GbqG-zG%m!8iYMIV!VKag;? zG~kNlhBq1^1K$M{Hpzhqikq{IzcxB>o4dg#H5AyA6>b4TR7S38qE-3xCq|^=|Y~=0<}V ze6x%E&ZxXzlV=tI($nADGqV8)X*DvChW`QthJ0Mm)QeRAngts4awN~!csd%71ZlSV zqohZgL_8P>2Ek120A0<+j|T~#LF>c!8%n&HpFSQU{EBejem{fVZ)E^prfncW$r5_< za`={ zX;mC9>15KhcQtQ=XvqPPRh;A^26#$#fdbW}g_V(L;x>Ysw>bZXy|j3=_o~b6PuFq5 zc@-m>SXw_$^RX@A$DM)0ORj3aKfC266rlT)?k(?o;Nq-Zg)k5#dAkqyJdXi$`sKqY z8qBFN1=o>?2*;LP+U=q(EYuLwgL@Ba)<6~*AzL&Ps{c=LZ#@=Rn%0^GP9#OJwaUe^ zRDQVIB2AirCmI~$4eb{h5=7e+;jIP_LgRMsuXjJ6U76Gm_xv?GR6kGEmLUNHVT9hv zYRvJP`Qamd4gu|v!xt^L>wR86Xx6aGIL!Z^mXANr`uQ%@cIdnLqBe@g)KmNDW&)dU z^aSi@vfLPuB|Mf>DrccQz5R)Or(o~O5GnU?$_l;A2UsBS8``|bu z4UT^5mJS{2vfm|IO~!x3p^_SSDnF&(fq}>n;0fl;;4r^fHIYPHWY^jh04xxD`d^3K z>Cqvx`FJbd?-!8~<%STot%OkL1^=hh(hTa{nZgcIg`jmAXrAkJyUiYW#iA_6D4r&}J@Tkj-Twsle-T5@F=%{rV$XXQ z&o)4a$)hwXlyoE&oNP*GDpQR`#9Y5;-Aw4hP1L-8!4i3fR3O+fnzdhO9N9pI{2Rkv z2cgkG0=FX-#R1ero{)R~7jx077`&j*b$tC=eY9H`|Gkb7uLA2`8kkg&F{BR0iv_R; z!OS7eciw1r`@hPqU9Zn4rXJVaVqBk3Pg;Iw{HmWESUFv>YK}PtN4@*Kb^9l?a|kDA zU5-p<$EHkfI}d6dUSM6C8)9ig^e)Q1&YOB$!C9{*)6zX%}v>Iw7t-`&YAN*om0w}oOktO<oHPwnWi+3u6pT%apKOtfFHG?fMq;`+_C3rTCmO$#3^V#yV~>ii z2zd#ZbWbJ?X~PQiJ>ZrYZynd|R{t!9XWJQ?tF=ph~ zDXJhI-B$I3)~V4Oxkm^qx%_37YwFY~Qd9N%$xgrZ$sTEc+aU2YX3=oQ>`4+el*#X} zj0cD83WDK9gmzkwVpxcgZX5hrT(yvCW! zdd2MYezj=(-Ek`VYOfK3!L-jb26Ry*yep^XH%ha=-K`bTItB#kcBB=)`hmqU+PHsY z6i?NUu@yHH+uk<=x(xle54vXDPIn(!ZcJhrSst6KBT2GFjSQz94lnsWmwK+l2N!A1F3=RZw`G8jhNVC3A1RzX6z7@6p~2zelMs&xF(6gxWyvJmEJvy}e{Q6|u>cR!~-mYa{!ql2IwaNBid?tXH zpWHSXugpxYr%1fk>%>x(&5)q(_=*i2t}aTv)SfQ+bld9Z>Mi{+Irw$YydK|vy_K&; zC6g_%%R*~JupF8174#xGR`206X@o)+rTCd%qd<&7p&-lZg(v7UL1yIZVS(MS=URxjn&7R1{LQV3?=*!z)cpehAcvAPDdR^=i;N z+rTZO2PQ0ix@|UiTJ^)i;1eMsmiu!~wLi?Fi zWN9sYSH3zT7X@7D^N?r~PXcD=ETK##-35+!)~}7v(ZEjlqYB$LIj%Wmhe{Sx{T#wS zl-u*bHtLI$8MdAN*8C}F(Q5iLCilGO#ysoC`>Bh)a`~qwrsG2x$bB$qr2g}JJiGH^ zsnWaR*-h5(L9+S1W@A%7bBvB-xGCM048EwbSo+A4<>f6_%oVhOaff=rR7y$}v{-IR zD6-MPkgySiTzv5cCWu<&+>j?KOkds98u_abFT-(TxGQ^n}H4!jq*1siQln=Pa|lZ8A3lih}kYglmmp>=p-P6azH<{Tt$3-vZ;PJ*YJ;rz-rCpSnWb4G?9^>K#l@qr8ll zy1$iL7cG~P$8yhqCzeDIM0)1q?IPbKQW2AT^TFJYH#bM4Ou{U9_mxd;Y_btmC~1#T zE=Pe*bC$RGDuEhJ&uy@lYIrNSs!Nw>yUKKLv+8P>&e|;XJSn(q(Bzr?vL1|;j`!l0 zs5sbzkJ!BTDP026dmCppDX_WIz6Q>pab%T=%^e2-v zvP)eB>ppi4hHKn1WZgD&yWeaxzI&GLd3AbINDuop=oqEqAJ|*@S3QPIc330l{NRl5;_`QW{ zeC_A{c+m4U*DqjQpd2O^S#0~-LOwjf4%8*M^BDV`4|p+w*F zd-;s)3Wjum zazrnA|MXdLkw*{)1KuNPtnGPC^%KW9^p2P3M*51Z;+Qk z?0u!deZEPAh^n4|+V9C`ABj3daIpnH*|!CCxdnTD*8bhJF5$&;-Fe`S*z+skxsBIM zXp_QIHBk{7;*1xt$L;CIBk<$;EcWht{F#%r^#F8`#MOLDOI^vdxgzo4Olq|5>kDJ4 znH~g{C?El_NNP$Do1u~#qM*$d%?RIZWaZ`MwYT3x|0px*u}l%yVNRsjo3`qe)^0MH z8qW(u^CpXy`j_?PH2Kb0zDh~c!KA~G92h+@-hPFhusTE8aX5RrMK_>IT6vF#=M$#p zIiczFsh62;P=H5E@n9R>jglpS$VjNUR2LDq44eSVx6Rmbl)o3elCD{`aIwn@6=dC& zPLUx0L?q7>{LX%N-Q<)3Sf;4usQuolNSTM9-CgR$YhPf@$-MuUDz%$5m3se&-m8s# zBqOS+y!=413%^MAQZSlCsh0I#UKXF7@sIaT;|1~S_6-^ZK}WSu2>U${A4zP<3Zb3j zKwGaK!D&bc!eHHzm_2DQ&XV^;q9dQB&!MXR;@Xca^x<%8Wxmk+c1CU__hVT{-X`=m}%IsQnkA7E58BSlc!i!P2Zd?{?O(2AY z5@jKR(9NBfx+Rg`iA2sRyrJu<^O-T#N{q1N7Lasp=cD7Y5*GJLH9~odoDDqH94l&M zUkKaYygYlo2#~FnFbY6i?ye>BQ&PbWBX`?S$HKpnZX%5*F0@`HxFwK0_!trY(fWuw zGx$v$XY(_#AV9d;F^6d6@te33Rk|!?7oY4;d#aBK4$NAOvNCLW>eJd;tUsI=o7axS zJYszhDL3;SgJ7}V>&gx(DEOpCmmys^uRiSHr#u~4EemUpUq8Q*7j7EhC7bc&!NFV7 zdy6R|_Yht{5X(h|reHpU=f1C)9e3F)sp+dEMNR5k z7B2AJ*cJ>-5UHLM+iKT*?m`{sjoFgI!E%%fgO(@*q1muC| z=yp%_$$OPvj<<6SJ&D`Bo|r{1hWBj^w`R_bLk;g-Vp*~Hdau2$)0aGn)Xi#vOi#g;_iKp9(yE)Jg@Ws=!^ZeU(oLK<^26@ z-O^y&Ar>~JQj#cEZmzJjiSh>Zm~>Uj*f_u7)B7d#uW+A_^?ph|o~;G)gbYrXVCB%S z4h(p`{lSUdK_LJm5$K~nNp)7HptBr7tJ_e%l3A;BP};lC>bB2p(4QQ#(n)(X36aB{ z`bdXq@i~5JzLQ1EW|=HX3w#u0ZJWqV+Uz(6cFtMTlq}?>E1HNs@9nr-1W}AdpJGi0tulq8LNDZ;uX4tGe9$5h&C8Z4>&J+sutbRrL_{R z_7-8HZd#w5Im)Oo4nYVEH`jujJ7YYHU_P!}ocV;#Eyjz5!=K_T`I1R68fq7b)Gg~EREx{W|#_8mgtXDe4b%*jRz;v;9V~w z4gDU2r`hR&T#I2y9u15!@W>iN_NzwC@N^T`>&;NZ5_#!{W!{|@RG>ZtK;&mbhb~=A z>QkAbUs6~q53c(7ct^BAVgTpNLKYG$`|#-v@I8;Smw#rn0-0U?c*Ya-<-2d09Lu!6 zxGEY*(O~g|?YDG+qGEE0uwg@c3Y!oSF+C ziYY^1Gye2TZ1stJtU|k_IDIRI>=V@KhUto_o7Hb6y^ilD75?_%@D+UDUH))+AdZ#4 z3E9Q8Gt@2jFD^g@k+u>z+W5Yo>ZR~v^NKxY7lg)6k~zo)nB}w z7(uwxo2ak;rlKtp*&gq`n!b}xM@KpS zi^;Lkt$`*DgSqzD!YN>%J{IAhgRb9V}Clz$Kc1|scPz@%T4 zGd>P0r4%15?XZC&u>C0sNmeeOXXu7iSu@%s%- z_uxZF7WcN17ZGuxf5GugNDXU#1kH|ECRK&@dT}t#J`P3v!C`)pK%&5dyx!M_i-Hpd zj!18g^OitHV5Y;->_m3}qQ=rhnD{#do=w=@pKxN$U@Ip24ZHp#?w$)uW;9{bBJ`bl z?yx@QNc`4nGj-rWix_Cd&9}5`k@+rBPbfd-OG+e;;6hdE=@HX3gKqW|8wI$$kPl_Pm6f>JE&x@x~nv5n9UY-w? zC;P1zFytd$$;U`g+8T9UruVYHLBiJd`o)ROM5J2T>184(sP*jNxo^McG^f^Tgr2R| zh8w(D36I#?9QuBwqB@(+&VE?$thB<(v4#-+RB*I`4NcDm7rLm z$w@7jkAzL98VBZq=`<}ABm|t>oLCEThHm1YvnXu>ngY|~ezOwK1lPfNZ5%CWQk3XW zG-uOnYA=GhsR=UC+l#CA^^?S7-gTZGahLwG>eU%G7WNah3unbM^g?V~-IIFZTudXgD5fm&qlCx(S@U>ouk%nJfT zWjgdh%}(^I0u;CJw^*i(5OWUc%;YAuryN*CCrRVLSIVp0UiZX<-;PP~)jNVTU)w+K zP$n#?NA3)Rk;8+tvx4z5ntWy6WNe7(n+DWjy-He!p?|g^)9-B%p6>-Q+Je#F$Kbqv<)9 zJ406-BJV8N9gj2fB~veT!C*6R$@BHu>2RgxbBiTDn|<&=R*TcwTz78kU6LvgFi zso!c8ujPs|+0eSNWBY#F@HhS?sUxl-EyHEM18C`m!?;~NVn7b%*-Rb)0O zOYVItxcaf8<~OeiM)K8^xT1WbnME+IS>SQ*G(S&oS7wMS-wkHSL;oM9k(N-W4w|tR zLE7~{S3)F^{nap+6lSxf6353&4Uz4yFK*2qk1SyS_6;poSWR9Y%i(xF`r+nC803|S z(cN64RKX-uV>(0KT%iwETdJk}>s`m|#2ELh1f6Ov2u-{YjK6@>HuarOAE;#G!9)ha zxqx`!FLUO$44ZWBVKeNc3A%B@-klJ}G%nmD$DA2xAoQ}^6HziI9|PGRGrHL!&EkC+k+AVDEM-X--oxuYw5Oa!fOm{ z@~Yih$6Zg}u<~wqm7U>b!Pc0@L0;sdbh0aUBT#*|=MM?^Tp0R2wfB?FI1@QIoC8Wg z^W1vt-)}{7Q+mBTA$6~Ly^c%`B{J057RVKRZRN07WFGAf_`(rOA!T^SV6I%L4S|f$ z7AN|i()H_ij)5Jo=NqMN;dj1-6kpSl%GH~?OL%};7L=h;mmU|JKi_`@-n>IDCb0nJ z&X%h1Gqj%jYLZO>c-PPK{V<>?c_QkNz(MaP=}*>H4d%>*@a z^*It4l1Y72tSkJvcjElcm($E(BF)E3<|p}#vW>o_^oRiW2MsWz;k)iq)kzRlnJj96 z;Rw%FJf7$h7;xHG`JPSpPkQ@=4j}`$-;(+EBkBCS$^Jk#LJcFxR9}g=Gu&IzD@loT z9ZB~(j!rEhvzvF(0gWZ>4V|to>ik(T?C03%=r$ z2Y-kh*04S(R<0tRDc4Ai=d$W}W?`AQzuE(HRzTmQW=ce&iCa-l^@RK$-v(1!kO)WQ zHmtgaE%i1BAmUUHB~^vfAC1V_B-)?u3kal+vhInAQ~i$1fzW7pSzu##Zws!ttwXdh z-mJZt$z5@jH4y1r+K6y?0nd!PQHSu`kqNO(_Nh?O7mf)m8@6usv4tP`w!8x|vX`|~ zF3&et`D?9URtf`OggL9okJX3h2Hpdxr7Gu-8XTTJdqZ5K%h)!DkdnWQNh_chpfOCk zf|Fa#pAy+TOf_5WwpgUH4sqvi->H7I$ZH<+5OGsF&U80G7~gyK6t;dHlw%@{_o_nh zU@PI?p_3a|jvxMWi@N?97!28Og_7q6>+4Kr)qYQ2g7w|)l=V)Yx(*1b=?l>P1RXAgOhwFpUz|-wrS>M z@v>Ce!{jA=*giN@o_>l|q7qEOQs7Vd^4gLIxov?L*ki)r>gZAepZD5#~JD$4f?r8eE3`sGy3xDs=McnO= zr|jhNXl`)jYIJJ#x!{9DE?*)U2#pMU-)LHAHWv}keYH1MP;ap`2|m`vK>}H)KA%jd zgLww03uXKgvk-i<`;FP`-uCBzUUmNQo<<~*mzG%n&Bs$XS)gK#p`__#F8tk2s#W_$ z%Q!gfbLOHQwx2RzU5EO}hM9zFuh6`|2TbF3PZ4r@4A2mJaxY|R{Xz_wgd3Em^aSU{ zKetgmht`RaW{f~;IZm3ppaGC=`S%vS_|%zPFin=<3dJ?i7HcG@KcnRO+~>eRaBLu> zdeZPQd&J&&3Zi(tv3Rihh6qgH4qy2hamF^djXKR?h3XdB02c%;V}|X2-}5~uPA!_> zVa;;b+V2T|C_0)ga-UZkJ`Y};|Lp6rAP}L=WEM~L=27-aXCRH6eXQd7k^4}gj^_EU zPnJANY?cblzbw$fINZM_0muBdS6r^JKIIi@-!kFuG8zqzESRu=Us-l0qMw_=j5peB z0dcHa2gbgy*rT8C_+b-z%nd?9w|ryZ2WjMX2x+hOev`!c^~+Rk5uO%0%G6__+Q9x` zlFZjs7|wW>ay^jX^S$*wWS3uLv^PXitNXpJBgqdgP`F~LGOyk4a2BWAt#pmy$d@t> z^_KNFuu61joLU|?@s`~jFkMrXI{b=I2l3KtK54#(z_kDti^ZgspI~P5Xjfr$`D&7_ z(%I38-1F@6(qnSebIW@Rgwm8KkMsO`C;BC^J(TZRt{Ql9**#K|5`Arv`OU-?smpwu zniC%*FaIR%s?xr9)vMK8xm7+g*&9tK=qj}&LMmS4Dy90WAr$;8Q`e(?d0DIUW6A4qMw=Y&JnfDWczdk|K zy4_iF8H;qJjl%{2HY3&{#!@9q&|@g12&`5(91gJ>u-A%noZn?KZz}*nAHXV4wz#EI zt2bw9IPe}@Bts&_h%@64M9NaBF=UJ`a)Qmee|bDhl^Z>}>7`Qd3QwTdVDo)U{={y* z4yc&D@GFgvcDM)vU_mUxkdDU#zVmtPZ2vqyl)2a`rq(Wbl(&i`Xp!dn03CQU{r0I& zv++)y%WmQDM@L1(0^WgN74TxKog{^+4E&{dWjzc**c}F?{zmFv_Y@?leuCWizP=P6 z`9V$6J-JjcnLJx0gIcPr525G&;ATw8KV08U@iaIKL14C8_2f`CWO1kTl!oR{&SUKl zU_w5X9_&l8IFh}TpY>{aKeKV@XvB2d>;7O}2g5)g*|1~1=GlTrSTj=nH9y)@Mdla8qy|J=M4`lIheZa5fegQ)i_{P%> zZK#hRT7PvR02g1P8JrQ(AMfA`UTE)F!Ct~$@MtIT;erIB+#)ZF+VahyAP1YL?DN~! zqbS6noMz9&G{7iP)|x9)me5Mdf(Ots@2eCzYjsHUJmH8+Ey{%4U33NA*>kHGt$jD` z!@D5KjilLMq)%nz$6Iw~xb#Fah&@*F#gD_$}%2b?!T@Tc#Qt!y18?Ia<0@J2|rs0nm?IyQLY z^6EcneIiMCO$7KvUXydq9|*l|A2b@waf>|P&U-PD30t#&_sB&R*drmK?p4MR*x$j* zrp?B`6o>?(oK|Wc{z8D0s72!Z^GG44K;KfWY?gjfxnKG0YeIZnT$dU$mc_A^c z^>nqh7uLZAcfmXXrq%9&T8G;q27a5Tl&-V!OuKer8E4t;=Y(a90MuFd)%9>gRZvQ+ z_wOwGR7z9J?BP|A(+{>ZKjth!`&9d+!98@H+65${Eq7xNM+jp_a=)GB75}Ydw_bcD zN^!214kxE_3n2URJYrl@V0=f6p$q*BM}K?eUb)zE+yaqI8ToDBGYxmRsF1!7FH@oT z!-jT{%Adf3WVQYwicBWodbP#jqbU**ivdWO+h@HhQMJKbc*($xPkv}xs`mmJ$fCHX}k}_aQi&&bDglQ-hmB8o< z+CmS%$743OVIs!^`;JApI}Jj>tPj&*HsL1??Yr`&iibnO)CAO%J8D;h8GKNugvn=9 zG|u$Kze+ayRtut+6~PgU2=&sRjTj9$UZJT=_)r{6{G$HsCdwjddUVhYmfujhF#c{~$Yl|>Sqk_fg$ml&|0hr7SZ5y)AxSb)Y}=O(v=*>4VUfV18}R;4t4KT)#?C?{(tQo z$|$yAZa4w@$L`+^ztn#0 zKAbL@x153a5cU|wwCZXB{k|)qmH)Hb#UFgb`iGahwW+-<@3ES|aIwhMHG32l#koqI zQ70eA|9&+}nL-*B1GCuf#)ee*ClGdtWlT-aGhr(C&>kr1HsWVpkF0)Dgs=!l6x3CJ zC%lNhJz0_*Gy$oI8W~Z#)aiD(C$8Kq8oRWUUoZBw)EM59uCL6850v^Xnw%^)1YBR7 zyD~k&rMXqP25LWB(TFwKJX)|*u8!$o zwjVC)(qZacMVHeFN5+D8`lKm(fh;XmJ^QanFR7?pvQI7damRwQ!v{{7r>V;^7tSnB z#X~ROYA1>qc(1KZp#X05LR`3r6=W2Iq)1Y z-Vjk}bpDBfIgxE_tEopt)cK42;`4X5gUKP8{5qGGGKOW$PPSVLdzbf2E;mfkM_7{| zqocf*2QZR`<~mQMMg`mwoU7)|BT-GuTl5P)#8=6FAJ(Pe-JwZcqPj$yYa&7AEKQ;; znsz8~uT-sR1xkK$2}I%goxdG0l{%3IGb?I$2&X>*KK<#F38(WlTM2$Zp0Hs3!|RMz z#P$H0cRrvE9BDxJ>8uZNyky!Evef`!`z%KCG|&&OUp9du@59k~GLwPC{UPK>VK%;@aH1Do3K?V-~M->$EfLbVw zd_AjW!;)ih@^n<;dDrjdD+T|_;GuA$w(9=GS-IbyY{b~(X=y3QeN#k$u!^_eE?e%W zLX$HV+wFt>^}Cs0b{$P)6lQOeEN_$PJ#Wcv>N$2Q?l(uAqntkgB7LIP?&@H?5dJ4y zJe4;`>1-`t7dILUg!%vD@j%Gj<&GVfyRFsFLVwOQKAU4OJUemVEqp0co;c||lkFiF zc28J2SQ)4Z^otX8E7|&Qi82vIayj`n{#}vctkN!up8iy9bi&HAtY?oqB3GhJbR?n@ zro^$cdvi8U2)f*>AV1SQ;k2%g=u6G(-0XK-S$nf<`!rQp_{p<>a<}EEc*P~7*dWea z6>yC0l#{&~X}pyk7JB!bYYgwKaI^kPY30{J6pMHrBkWqb7dX5bgVdewn_5ayxYS!z zMFtP!UYw3D-G=qKr(G;ZI8oK+iGoU%OJqgg$Ja_eay{Aq7JgT|fQi-JfEkwf`OsJ< zmQr3ML%(>?GC}gsou4^-o;z%Omn^L2)7+O%)pp)WwepGNixo1U>8T}n+Ve7DX&^i2 z02G@)B_6Ow>#55YXc_Yk@vLQ?Gj4UTDwdine25Uy|40eIO*8p$Ubn3JC#h5k-6;w? z$yHx#w?fUyntB-%;Uj8p!=ZaRSB*hU`yeobZ02&``t!xG*<_T6+FNMcC&bqS8;k%! zWsZJdEtAB^A`O36TGjQMic7BmzY*~Y*vp4(j3XGO6gGWqeGe{rPf%#n(>QJH`?DBz zTA53(hCXK5zChL1?Ey8&t2S`I`lUkH}|WvX&qH0+%!6kU9a3j%q!jy=x}Np7t3Zx39s=PR1V_4 zQ)SYPQeuV#eUU5s_FdfiInR1s^|dpgNpwi5P#1xxlP!JDCcPn%C(1QyC1jTtZm2kf zyY>XKLz@|O$F>P$08hx?IJVYpO=V21s#>YM^kX2FLHRMzoK|IL&?vzgzhKf@s zDdVIl?;;nuS?T~ygTNvBL&2*Ofg|ZFDQ>x$y#r$bnC@r2FHCEmpRiUP(p?^|@HOXu z_Jl84+|!?AZRj5OYpcado+8RReDE|&Ed#wm6S^SvG+xB zOIY%+2k5$a#)XdVJHF(@WSxEgS4N^x4IPfor(}hbl0DwZJ~NRU;l&#?xYTRmli5l+ zSLvLpp?lqVwyhC#kDYR>seg*IjYB+VVlZ>dGhqQ50$#B1cph+Cu4dQoXl@Lfj_LN8 zrVc-i6a!YCmCjrXRQKCgJ>hKoz?%h(3s-IPALy#atmzg}f z=G?AR<+JG8M=!pxr{&E$=g%(Vj;s83igQ(MmGwvF8o&sx47cCqA%)G(YgQQH?= z0|#AiuZB&K<&jOJox1bATxhD>+DwkpPD)w7Xhn3ujH;60hO84 z&`Ot3h^<*_YxBJsXC|y;NEXqdX_jYy>4&Ym^%4=U=HCep%Q2~c7tt%&2kE70ER54D zMix6y)#n~M7(bhzOSvfTT<{2O`#i%AO*mYd#&<}41wu1m&=1b}SGr6U&n&-*q0`^7 zJ?x)s@J}!Getw>{9$9X<4sOLF2Bs3`fB}u#J3GYfrYutS>vD@%+AexZ&(mtVL2IY} z_qY$_La*7QW4#`VLFt4E<{KDm^9LMmACcZp%kV!`8%5y?ja^^uL@jc?XTp(%dQX}n zt8gZh6szCF)>}Mq5KVc4BcAj@H#*OCGs^P>|6Z1;6^Cn&Txs+Gvh+72!;Dvl0Qb}! z2a4qUb55PQQy^zn%9Zg>XF8@}USxs=GX5A1GT;G!?dPnocAu=C z^18tZCi7fjj}_W4rS?DXCI(|xf+aFpBV6+NC1qn?h;Ju|;ago7G`2iedLB*&Jt1@a zrd6Jg9@tq@eTj~f|12tR*&9z<79*RI21bG>R*^~<@)b}PnUMa^~9~!^*Y?W zq-J)bqkEUT5T9(GuryO*MX%8A4BeFdtMtw9{U9P#IF;V@;UHHDW{6~*dqm-XlyS0I zAT2jp7RRLtWyHyMY;ni;*BDgt8R@5js%(I(TU6(*w;A2$e(wr24hX-51Cs?ZS2(l@ zfK(i`QoN3c=!C5NSu*nd#tF|ojo;_HgQ?&--*}h@7!ky!1{rLQLYbt6yNnsxcT(CZsBeziqvx88V$sIyrrOHP zh~Zclk^0>bE_|BoZEsp@6a)00YyrPZWW#cuHjX$!^&mg+6CY4aIqr{>_aArMt+F)@ z@jhJ$>K8K=_W!D)!yNEJlNUlmw@!glmtzUtuLwUgG+P{=eHZdEV3Cr<)QfD?Qr=4x z|LH=Ka=~jwUlvL;a?*T2uh-)D4a5$AYL>7)^o^dvdmKT2@`(M9DdPP30V@O3E1zx$)BNK)bg2szu@#u3x`KBb>90eHdqX7qbdpr-Yq+DS+k%m zeb=86*4_6R98KfgYlK`OUhz!f4V%x=@QstJvl*cF2xVu#M|+WcD>r`Z>4l<+Nn4^F z($ph-DEpeh^Bxt<$!2T8QkqcT3brC6Ibu z)6RUriNS;C%XzEwxa|eXZtitK2osNPSAf}k3OS?G*37XK`^nn3jD1B4N+-{tA5nig zCGNVTA;YqAx4p}ZSv-}_?JKkjIZ-O~qPo)RR6fXRS00bu%a=Cn-}P18KktYCY3TM6 z`{QrRm)$*-4OWh$fgk0$Lj|nh6Ya)`sk}8cYc1$|B1g~A%Or+D7VGmu#;jEDjf#E@ zcs?RY>?xGSeWL1yMb#o2#&Kf%)JHy=E?O)-`)9W;21>f;JYFbhH zpLfh0!gWNYwPBylh?n3yk(Xrl^@ZJG5D@*m9uy18T=foDER-w2L28;llY#80c3}l` zoaRjoTfvw9K7Hq|p)yTaYQJ=BT_7bD;=IKJqpb#U!Q88%s>wOy)Zq`#aD&C3^J;7D zdd+NY&H6Zel*Cs#eNz0!rOK%&K3Vgcr(MGF)%Z-iW9aFl+Co_@B7)L(MOzqzTWBht z6;bthVUkwZe`skZ`GeI6XX)>w3nnKsqRg%`?X(yMv_o~8?{A!ko~r$+sEf%hd6lEp zl0o*G8E+r*DQFF%9zn!zfJU^3HfRi)zwh)5Nq`=(KrbH=-i!WRi@=Q8uctcLx`D5m z=mGLH#zbX}&u+!+3JjZ{s;XrXuCn;?Y!)M!D|3p;2ny+G}9r!_|`;V;? zeCX#*+qlC^3wh04gCQR@xCx&mC^#MbJFm{3km5@pr!uigr3#C$ET;ht{3kAfng*j1 zKesuDVEG#Jp%gC(@l^*d-9AaKW-m<7rE50#?B(@rgUzO*JOTZd$G@=NrB<{Sd+RR6 zW;3RLSLCz!;SGJ(Cz=vAO&$UQe{O}Wr!qIU>IfJsiUH-+%81l4#5Q=c`hW=FvAcC6ATCVM;TCamF^(QTr29bbl%h@k|l{$K+cV@T&* zzeP(;Ih~cO(!zdmuxulxa@POgUCv|$bf%0GIX8t4$^(;=o`(S4rHYJKtdNX6p%RBb zfoj7!o5s>B!^{o~YzS4199`X`8vJJv&D!2mW4~C6~^fbmQR1`?y=l(_SNgN|aV< zx>;J{&Y@d~8!v)5=Ss8YuEOr-xE}=gPV*|yY)JxFRIQgJWYQ+C%fmclep|d&6Sfl$ z^PAH14ChaFlV@LS z(qC%WZ7Y(78M0#I^;wWO@^Wf64NIY@WSC>K<3Dp>7375GC$Tr0 zmZa7zE=^22Ed)b2!>x|E0c($36IAc3;QK*3jzT@^I#-WMt`3BS$I7Gn zi`LzYnjTiIuTGouD6MlVlhslq)lYi#QhF<>FF-qK1SSB(xsFsFZ6Vf&xF4GXu6ZFB z$Ol`md1$dXr498zrtD=in(NnPb?!TTx`D!qgm@%{BPYP)i zoS>R9(vU;wmQCm?VP!(9{_Ih^llcS5`K(uegWe-=lzX`0?3!D-%dP}2Cef<(;PKVL zvW;?3#Kbc*;?1u{KB}v48JtAp4pp2ADWa7V{RYbjgJW>gr?#LDnuE%Q)d zn%wpS9jU#&qpxjHu5?;NDI)+CDU&dpK8F3JNSa<6Xtm*-P&uy$Fr&{6XJV@GKkk|c z5YSLjgZ^y`%$TYCO1-hRP<1`{z{!+`BcnDZreLNT_p5Dn*GB0@p}gom)52l*seHu! z>iV=o3T4FDJ;!uGUh*L30><$R7T2A%q#_U{p`*G3weAOi#Cprbb7qzLMYiM9ggjfu z;Zmt&N?P~{0EeEs39*Rp6VV7kd4F?LB9X4`>#(fb_Ej}*{m(Ubc%%;+YpjrZ(Gb*- zyBr^t4{_efRo3{TWrf$xUMNNVL- zk8&OO>o#)AWsN$@A%8ohkQq_=eI0(fuM6`;&Mh8Dt%7{I;HnY$6ot>jUig?nTt}z5 zhsAGpD?i2E<7-`19JX?IY~fVwI;se$d!II+$w@ybhw4al%1hvWc5ZiD$j=XRea0TJ zHoj4i~M_X39?Q`4vl1*r>k!-^OY_WDzh3SCUFxRigluprT%&gNq+ zjFQxVh=YwHg2ieA#9=tS6gW40F?z}wa|BDR+UMf58*$5Qw9admT1IwNAJn^HoO(MH9ghtL$jaK7P92iW&_Q zorcFvqZY_XB<69XY4bT;zo3ElXT=}{Iq|qb1VV6~PnWsbU@d5WJP8{33x8?`2q;n} zoKR>uP}RvV)TS_Y6TBOwh;He6xg~B(+iXI;7{VS7Uw9E>ewuRWo^>JJ6Ns-*Ze`Wg zM}Pxk5_jsKdkE&NE@v&fq(@rw%DTp39ryw^7kumH5b+pG#YX;qwZl2L`HH?+$}#CC(IEQB)Pq6vA3S zUfnFtDqfp#ts%iI0@AO(7ogp;5Y-lTQ(@RUT7}fHRh+r<!!)U$W9 z5G1*gc?;UiMMj42Kl> za%n0|7GTK9BvOKqZt+G}~F z1JhH(dJ)~nzEdiQH9N}X637LeB^&WaNvrrwjsPx^Yw*Nx$27OZ?6j*3b8)%O_?j5T z6k@SoQP%XA(W1YV8MBG$8H3!5T*^cZO@o8!eV5QpJkbO;{Ir8c2_S{uH3hBc<^m&` z7DcD>kufp>tq#l7pcNJ@?W5c_B`15e_=5)M-nG`R#1sn^`2A^1BNb>MiW(bher?5a4a=jnEQ!><^8c zU^GD5_$j)U-RLhvhj!yb!+IOaN!cy9Z1&iDC){~e7>8tpPI`Op`@1hSCp;2Y-->sY za?$?acjsfoq5~~1k5rn9QsNZgT7Aen>eXu7Sswd3Gt)y6~WoXy(OvMoo&ttMn7WAje`!tedI%$FZXu{++r zL5&ZAh*f7CR=5G?apbjN{TXY3(oG$C<23&CPUsFt!)R0pNL;cHQu@rhq!umsUm~gP ziw|#*D*YJgHK&t`opd^jVOxkO>q}WEP62kRHors45ZN?KOVhB_ayF|en`7Zu8yf#3 z76Dg<&#}U(5Nwrb&i7P1DCzUtstM5@EzexsqSiN;;%V%Y@Jp$kYq5f@>2y;@hnzXT zf6eL%WCL#mq2HV9_cCQ=&h4qNh$5vDGJY%&0LE%~L={>I&y?c+k9pkcoYOWZC#a|Zs-gpStJsm zwA1we7|^oxYy&^v z83Yv20o!i6V*Xd@?=5U?$IhEEW6}kFKA05I+B#bLMe(WsQmzL>R1h}78jh(Y;W_|< zF}{wm7K5=K)`}PLIYtnH4mT$;tI2e3ECs&MemO#emRfbdcSV6dhzgyRKXtjwMX=w! zqZ){a!~JN1rDCHoa?nZbroA`#hQ4`ET5uR&z(y;uFkuy}PO=_uzBX?iJ0QHVhnex` z#usKJor%tL5r|o(A@rO$juB1;|8BM{LHTL-chYnZD}>9=7u8fn%v>Zb&%^ig@So>G za&i%oySb%BeBj%{N)hnn5j5JU;H|w_7e=e3dh{Lbx31;!fs9*!O%$Cy)pIv*S;_g2v(#9z-C(Y=jn2q16;ww>`HW#|BOQr{zizX@X*>#She;|e7Te*%;o`wzXBhYCz!NmzGoV;8d z5U8S)v$LX^L;IeXHO+C`;GOq^4vZ0{@_enja@eO-FCUeowoEhTu}-!1RNm(t;a6RD zg5jQKqwnTV=;cFywx3RCZt*!?|C>L(K~?+v2kQz9id7X)SJ1$C8MW;YU6xt4*ndLz zCe91r`vKZTdrpI=jM{&$`?6atL1n}xvVcO_{(Ncd#+o(g$e6^o`M3U&t$y*qnq_Kn93U-J|7z zjpV%&V?W zLnO^?{wA}O^SNzBaDoXR0j@|wiVjnBMI8y=ErNA%JvPW58ta(hp7wu@rVvm&KJ1GK z41c&Q%!e&$W?PqQ%`OtbmOCyrB;$;{>%jut68~hqvO>nZv$VS$py4m$()~lzHauIP zBor)wQT+-`&;N6V?bth|k_|%G@@A4q%q9mYG+5Z`d+)x)Ooc{kKoIB9)$e!xX?-kz zXr6O&v_ve@~^ z(Ai*BJJtp9c?xDgE~gfx#}$Dp3k}XM{_BEV9-FQEAtcEbYp!P^bJFLL7heJ;54R!S zGPm%nPy>PiLQgUQ}QLY8^6niL}1KM z%x{P3s>Use2b-|ekhN`4Y-nCdOGW}q5h%3Bt_N-D{^lB>_sjyQT@HQ znQKvko+z51fLMD_;Q3q>$RJN_LB|PTMY-|edIu*(!SG+$3m69^lsnj4c9Iw2IMk+g zsz7*mT3oLBX%IN3ph~s)Lvl)8?~4pB>9;X8eOilbcQ&VY%ObB6QLF_t__;*5VR8!( zo*0V9h|#%2+@4@{%q*k6*XzZI0*-l3B)>6$z;=#MP@oabfNAU&n=Iq zN(}6eM(acIYQ_V~M$sIg)QXAH#9<5^rIfNJt*;R$2Fpp2S$RO$8~P0`e}202-Oz%i z2a9tef^au72t}1*+d%Y#R$DFBljjFGCy@s__6DJIuh?Jhj;^Rwtj)qx z{RXi5(ULmPAK8^&OqMaikG^YqL(p(y5P=laU=aUzttRW8?t$Vqi3lQ~qix_-W&xa% z#yjY>4n0(V2|h1q>ZO;U_Mr~y7l&of3wQ63@H$s};13#yPiZC!S)35gZ2hDo-LC8X^5{hE5q@gwo>oy(A#2z;+~&~n^?Nwa%R%bq*KkLjQ7!f& z2I9tXAc^pr_P^*dA$qLw1KW+p%`zbC;S2`U`$iy*DbuC~8{l50*D7JEVp8@YtXVjS z+iEOdveWIMaQM|p$at2gpT+a*3BT|`UC`u{_fO#=rH@4b1i*dc4vGd+9!@HMIBl1*PDf7_5yc z2VTS3#fD|3oI$dDU$V&!q&nhGrmU;PM3AO-clF&Vz(C@F=pYMp6s?6Tfa~OX>iYk~SJfAU&)z@%8 zoi?DNDI+T|-^UVeb#YDE$ert>D1s;{562IovL~Lx89yi#MxbN=01GWQI*)-Z;Z&I@ zI((BX;Dr8dNys^X&1QO7s@24J7IfwYuptkI8uFHj9yf>L_w%eWmq zaC}FC_?yr{UJaj3AI)2$VvcKbsC6V)JiM;ze{AacbW25&Kda-ZQLXRWLhIc#!UsiSOGW#=#3zFdAeu<) z3-f5Xri0PsAZ%6Bmf^UMmxl7nUpk3$XS4myZM^0r<^<gEM#9}wXz{BM40$A$ zq6!7aKLC9o_;^zK2lbE)?jI0Stj^TRCw@$pfw8nQ(7iL}I)ssEgsI*4+>?Dj9*`&) zF4_OTSJte1fawWXrtWyGiUyL%bn~v`|9nMWp!^?GH{hq7mjwQ$kvHesvisR8|EVir zWX&Xkfomh>vh^{vfKo)Wi}ILSuvkD~@JvPvkO{rBi;+>>c`D%*Ln%6gjiG8FS2@7x zRF2Ck#ZN*?`qk}fw;-I5O_TUbxD%N57j!mJwOU^|(gAKR+2Bvux0ny&^uM?|(2m3H zzAjh%wr{1-)f|rnUrt(oY5IS!|7b_^4M93Zx7jf6jXV9%=Sz$wb==?DUP>EH12#d! z{SAaLHruEp)%~sYcjW}`l-TmAk;)2>Qbo<-8;qxxOJzp(vns~#i{ywVW~P7*pce*< z{BpdKT6+e#z$w7WB88M4#H)F8IKw>_S&5{SA-x|bFThf{8!o8%BLE$+Vr|k(_KSr0 zv)Zl;(P}l){|62O{z`eBR!4HVpZ^)59nnY`g8Q4RJAij~EaHDTSPUZOb7j5(2Kz^} zy6vvv^^^V187XILU4k!J9ak_8xre{ch276|+*KfAJhUpp2+BC0P{3iiia5&gdf&dq zV4zIm#!#!I;v9&VUVrGg(@h+Md;7_`d~7LIE6uSWN4@5mX|$N3wK^ng(#gHI3{3x} zs&qT1qi5X-Eah*FNwJ}@|NA-*f?1|e6e zN5dfZVZzFw2q9sT#sg-hA;7;}Dxv#(d=z4*U*ezL50G7rQ>1Jr;q_Kas`~6x^*@wK z6br(R*ShqWx7o0@b4%5#DuaIZvo_`|14CQ5U$opPLf}m8E-%lXK{2UfvOGPfgGuqw zPeordu=zivSjWLpUw6ueztH)MB9M{*g*|Jba(+oBtArb%E1^U@n&)S z9?n$H-Lc4i{{WP&Vz{qZ7vkEDw23qPM`70c^`S^M^|LhKDW(Q2@rnkx&f^9Ni`;m6 z`~Q8d`M|{2JlR5Bl)Ek@OdNMP0VL(~zb|^p?hAnldlIx-jEv7gZKSi0k_!#Px0hUl zsRO&f#}O@5j;yW-^SgQx!l{qI=K1Joj1cfCfC$XL_yI9D8qXej?|8bb5z3xj`X{PH zDerlJ{7@fM1T<-w>s>V0v=F$PbAOT7`X-eKWo{sO2#N_v`R80^7b$ z(O)5Hvz0n!%YPN{#omGBfX6I*II6KqyG0Qol1iL=ST8ArjfBZIa&MPxx}jDwum>4V=LW_!MHj(&nNjzRr~`r^$6y zpnWO|M=1oSO3mz{KkXfr@HEsJBa{t3)@U2ib$qzoIl0Q=Q_2w$uS)H5+i*|eH49p9 z^Y~Zx@|)KIZsz4E$#V9^?$72KIh)bT@r<1c$q_j$HrNo3r5^@`y-q{D2|CV9@8*7Y zT7khp|NXDEp6x%srvz>(1~Ebhz%>`DDdxu0!$S?Q+ry<`Rbhy!LZ$QQB%DkT8e#nS zXUlo!1hOZ?{}6GIAGX+g^=s&l8iXr;_v>ykE0`z?Y9C!Z`vIla_ma5P_ti8P!X|F)dQ{HJ0MPK7w9nDp#RcdkD zZ+GWP9&G;kl`Br&MU%5Wbq$9T^M!*J=2eapUKDguZ7$&+XREOB$e0f>{Ea)nG?wBvG^4$g&J z{#|*wejw;o3cyr7^}U;Gzdc{X(_F;}>4Tw#gvY*zO$dLnPwxs6z2wC`Iv3|)I={3ZED;)46A9@kkUAN$t?{mhU>_r&?N z`2^vlVPvj-@gIvrLD;u{D3MFevj45zRH1M@As)kGSu(Ht?=#bS-Y+VlL?JftzVLW# zA)^kL+tS8?0R2tehN|cbhX4`|%P8m###~b41P;dpfAXoAFq}`hh3-bo;U5t7tortR zkKQdeL(`$JnDjdODIj0d?ZKEY_xF!SRVqJ8N%D{a$-D5Pl{fnFCH{$!K}p$dR<%Qw z=MhJ!>&4SJ*4kX;H8^@&hcqTVA5PoOj$t#fmI;5Xv^qJ<7CS-8A=gB{1u;PU zMNT1$XHNsFwYeEnd6Yg?AXXprr!Y?M`K4`^Omw%239*vfGO0UAgx>_#ufCV(-+Lzd z>qtLShW-}J3&J@`hFAB-j}8caFk>Wq<-bwD*7{_T$2cM8>eR^Ui9U_5Zm<6CvAlMw zgpX5@Hko!~-jjSoH71?7v^mSXY2B!T>h*lrDm5#uR*l%oC90X!*-Q7e=H-nstF#^Y z$@rno^2IvIuVt;a?Y2z+p0>C||Ak3G>5LJT;MNT<-z7D*m&@tgO|Xo~^f^dH^bm#JT&;j9Nkh#&V?s(y)IZYOL41PP=PO zYVZy^y}AC6{FU))%%DdvU|=bZN)Isms=hf`AsPdH0BDnOxjS06iIUR}5O2jf8>5xx z$z(i^#X8A7#X1m6{TIWpAv^G}KY)s^G!uAa7dU(2z`Y%M&5#>a|{u9QHX(rrs|S9;~IC|nJdk3#{_nhe`x?9q&g(iOGcjjl9V zuh0TrjoKVsh+cCfrF`b{uny*9XAe3WW*t*~VT;Q+89fIx4Y=EU?D!Gk0%>+)BYBLZ zArxibBe(CuyUqJ))~dz!vOS%B(dl4Xyv6M$h*Ca9`!b<+#Q|}IC9K^F)jyh2W0K2a z@<8#gsJydYe2S9?Ufd=eLy6+!!Kf!-# zobno_`pOmP#2hqppVVu+#gRo5yUu^3UIL{{Sun*3Twz*d&gwTn!Qh^u<>T{wI%ztL z;92gArIs^y>79Ji4?~gxSxSg3lTMp9*7j6ixN?cURA=@Kq_5K=Cb5I*tEv6wF9Cs= zE{U@q`2&uh*pbjgvdL)bjCMe+eKN0$_2(FBpw98b>h-$cKpIM*OR|?1ZNcQ01k#v3G)IEqgkf0PY%?uJ_y>4wt;Sgzp#&aVkNfJX)r`IN5WG;x zcdTqu(+C1+PIF0+8LmNW$r;tg*YEJ?Fl)>Kq&5qmq zx?G!^EGKoPKy|=u^sKfz%4pC8l^=**&5l(Qy@xJD;LdZ{I3Dn=D z?cyfeyceqeeST@OLv{sGTBfqn@n0W@H0h59B$M9ngYfA=p3cCZIO_m6310&mpEB7M zkNh5%_M(oT6M}#&C4?BR8UWrHK?MKO7p~H%pKqZs!uPh%CtJ_L?X)1GA7MtZ zJfe#}@5i+yEOR=JoCh--c#Z$1X4so8d3{nTD{r4^YB*MJkwa{%4TLbO=@$R|Hl$Y? zvQ3|^MCwHurF5}(*#hSFTyHiSOH-LN`C_`N^OcN`-1Z}{%XWDFzf9pTd`gmwPE+2L4Fgm^k-i=F;X-HPlVtaWt}U7#>R zq^kc;e84h5-hicxMy@oFg6J5;t>*?}8en{Hc2j4~W^&vLL035?wgjzbhC1LeMgjE- z=16s;KRcN=<9CF+5vHzU0#&At>sb4ZzD^!Iyny$0KJc7U>hQl=!IfDAq4H7vNEb*w zGv5n}-bpn02SNf2!ef!}Wz<`sn0+|S9vtS=QZ05O1b)0!WAfTsJkG~7A@YE0Q4se| zCaI*ztO(C%0xw(}+_Ojz9_WM3z3lNvO4#|Z7}&IKD;Q|}9SC=d2L~PTmit1z{IepF z3zvF+JY$WkqMBfVS>*OH@Vo~C*pbL!!Ga+P1(Kt^+_@D#(A>i-ntRRDH$0O0W=ZF& z1@a$t`EdglgmbRFJ17Sin#O4O)igbDzCm+i>g1*vnelx?abGeG+zO^0c9uEzFBe8) z=~e7|X{l@Y$%2x}ULI^c(#b6cRiVN8Kt?vla56<+t=UjIXB!TUih`=FX0CSkV#NKj zZqSx&K9=|1Sq_`g>H5K~T$G@eM5?n*!C}z-)KvrH#9{iXOYHQ~ywo2&RxPpxRNG`< zmC2A>|6}8AX0fE%UV((qDU-$NoTmeFUks`zeylyN)Y|r0E?9weLZc#H~x_(+5(Y`*oTtL13HecQDxj@a`conkkMxTY_dCAh^Hbu?Dueyb- zV_aT@gJuvBjS%f)jNC^9qtUpBLE=y4V|e`D$X5%WG?Td$mz41(;~awI;LR8PK(|g$ zs_C|IcNb&ewu%kzE2e`eu8VL1?EuxK(7f+Da5wCG;~6+CT+WwbV#ZjoYvgC^Wv(5f zP>1)h!{lf`j>uP-K&Fy@2*d^fJw)J(UR$gEsiG)sXXt10W?qSY|7>`*e!v}3oNPFc zzljBx)f<7sm;zMF-~2`ZQFP}QQ>Ow;^SU!|?jU1+G-DGZ1%C-SLLetnQa?o}`Yh#s zy`NK#Goc37lghO9A>J$3sEZeWCq@faB1ph=k_TF&1b?P$VmQbn4*n-d*RN(Xm!H$V zYr$SVf$CtE!S`$VGr?y=4Q-s;iyg$Vb+WoMV)Eqy=i}u{i=dM0^3*~v8gp(PktP3h zZqdbnZ%Z|qm7XR%uW5JJd@YW9U&0e~+YKE$?2=eEE1&EcR~$N4;?~mJTKN}(){Zk9 z+CZXEi<$wuF3Ayo>d~^#vE*4V`^`g_BW&KOAFhH%>+c-Sf>vVN)V&%k1q4F6*v?OM zCEk-R6)S0=+admzc)3a{7(V(pe>cYlkukwcE~NJW(qR}Xyk4a)6X~-#m>#Vnb;N!w z?=XB)YV)I%CS|u*#mhUcdOI!n72CUnNec^x3(<(4B-Gbo+x6}>2D%q>uIrEC8{s@fD+G0uZAX*x78LoczJA<3>?rzvL^+sAkNvM7?FK8rguyac|33E#j%hWp$k{3{H6BU)gqjy6_&UE} zQc2+(O}%~=EDB6(y`l9`FFdjTa*^on=s}8e$b0kxQXy7XfFe%!9RY5f*rq)Plnsz* zB`{}$@GM}=Imp-VEp=uSAE;WpN6TC zpgLB%@%Sy=)|BH}TnF^7VT1t>u{N9^~A@7i6xEGL=`I-!CPGu^ja25B?(Cv7%RpvH?86VV4WJEVijg-o9;OiY2PT zFUj<4r|bkx2u!;i?OOTT)_*_M6c$|DU!2s%3?L8T8k)?nZdzIR%h20kvHh{p!eq+& zOsWMx3rAhlQzg%TnH5g)$nog4Lu~R1Pv{6u7zH?&&M=;(G*EIFj!Y_}YceDUZAcKm zTnUgB^O>b-JogOWI9NWdg*1PVk*HGM#2j|Nx$3vSxJGFCQN7%5>%aIm{6H7`N$pxa z!`weuF9e^{G&(pXB?2S`wL}J%&c=AJF}y11?d48S2n6LDB610xR1^GRWt+Fqa1)oZu@dh#nA)2L*DuR46vTiwRTKqZDFIxJf3H1 zYly8x`ukrmZ~FtD!aHIz_$m;y9!mU_&)~-#NbrByd&{Oe+O2C8Ru%yQgoOkT?(PJF zTX2HALvVL@4ek!X-8DFYpuvK>ySsCm`+4`?@2+~zA2@ZY)`taEG*@5JJ?9u>j#&qU ztFxGcsA7a*xK5*~#}(WPzo?BLK!GQ9{r=MP(IJT$9Uf<&)mE+RGM8&XFG7vOC5Y^~ zb*?r+uUtXysP^on{(43}HXcrmIkSA@E$j>zX~;b~!Zsiz8QtL$lre0PyCfRwcEPr9x^;H(%Vi14s1 zxk>D*@Q-yC)eIB(5B@yYpuXio4V}~Uw%sS$*Wfl2^;8(ntN(9zHyZ&h;wc8cP&F*^ zUNS!1>aZ|~bTt)<)(H}YK(7rfzZAae+6F9boaHG)xCL5Hq!jC*1t7`REJGUu`loeotffyn+3n2I2quj(FA}M!texGNXSgKkcV%VCJTS8 z0o{KdfZKDmcq%f*2c!Iob~;(MT(QdR`nm^+Bb?y>Bmi+zHPTIXR2|R8S3MyVD4VGM zO^BC3r6Ah7(&flsxZLJm>LQ>T^ngk0b$JE7j(i`r8TqEcT)RqlQn2>@IU*Bcbyze6 z3kTj$sorcVJ`E>PGM>^GUIBtOHq9@vD9rx)#OegsU{^co8Znn_K(0Ef&Q81&62xhK zX^E5i+A*l}I0mjQCwyt3WnTX~buY}k-CwqT&wp#mb7)|2Ip}_FP)lH_nwFKfY#g!K zA7@+kFE4=Nw`}1tKi>^LN4RgbVDfrr!^(1$R7^ijwx)3_m&YrX=uF+t(gK)9V{hx; zOkL??RaplQ{nz>7y-8lUB8#Fg%n$AEYLVT|o2vzFX1LEYVT!Z`ypail0fJo3C&?S+ zQHd7`v_(MK`{(x(Gnk*n>&e;jzc%N@maqOAFH49msA(3-nza{>XjT0U$^C{!|BVmZ zV{p2hat0}>-1ESRT93}N0ddgmoo{xQ`wQ8J)}Q7}FU5p__4r%>VYpk+My>f2I}muD z`kp$O96$p;<%DbpcbO>3?)lYT@hJW7E!_9$FWx*{MgN>=H0UG{A1qNRRce*~hpVd~EUmAAfW)mq>2XSD(2)>kyXrpFfwT#5;H4LbGHThn`b zGsBPplrI(rV1_Wv3%)%_Ckx@KD)LRk?!hI6<8%4+g_{VD781l5tGdu&nft-_RyfQ1 zxfzEFu|h8bmqYnjyUNbbd>(BRe0NFIHvw@UFseC`*W~RlPPf zD7sdocZ$=ZNkG8oYa*qnW>!!}8n(WOOak(;j?nT6g@Su{l4g+V<@1=V;WI<+=H{|N z(5|gC^JYJ(dBJn;VhV&G8-9xr>H_HbJ3u0fkz`n`P!v;Dg>aTa0 zDyXq>tMU4v#%7d)w`|8a4T9(_)3hc&+I;}8#AX;3f5y;Q+!tR?fe0^`Y!1W zIlv9!Li3;^FWP}jvA@AA-{Y*yDr0ry<(d-_{eB9zi%@^S8Ubiq9{w%IhT7U9WPB3lih@f_F6n@O`Kc{*$!Sx-U$h7ri0c6UVjo+9CvkP6ht$R)m1 zi{X3TGYJv8a;1(z&#;A;1u8DjCO(X}N=VH&dL>bo5*a+X3vr31hMSVjU78u za8W&Q9LD@wc;D~)=K)4pz{I@v_@smwcqDaU{pQ{9oS`oR_t9WhHW}`iYK&xYQy9Nb zk4NNW5#L@MKT#rdAR=RnDA+b_?hkglv0m2d9qfktB{fYPO%ZkEe6r7_TfSVw(T+Mc z1Kld+WJGm50TNRQ!*%A}HN$z$1bQfZU%3zn1H$xYfb>%&Sjq$hEG4>qBRvG0XQGb+ zUbRpl%$j7>fD}Y*=AQT?-=m*eZuk5_4>W{`rndbnc#Jt}grb}dQiAKJ~lBe0@Iu_+%yFd>n zeh`u)Kl@U7`JCK7`NQ`I!dJ}WN7dFejktkz=eAD99m~Ij@|*|w zNJ7Kq(w%APB8J9-6j{i3(+(=T4o4P-fBpi& zwQkK#-12Z0z9T-<8Zf#%|=?-ue*=yGO zU%ANWj?Qak0v6t=guYc|kX$I?@4;11mQ@=va9<&gpup1q(3~QxDUAn3D7}zH*Udtq z$q=1Dnp=vuQThDUEdQK0eX!;`eh*1p{V%b9A{0Q}eCy~OG$j9Gjzffi7Iu#V1R43- zeWE4Pc~r6DNF)%Je{Ln3{}j~*qi4Cgp5CATAiU0HjPw(dGYAVMYQP<|Mw>_n1Q&o+ zI<<5v>lr@Y&HrJeeO*YS8EplNpSSJLUH;U>vWnQT{Z!#*4_4~q*B$M6>5cYJE9Zx* z$j;>6)c8?(=zjy|w*BB_3LUWTKxo2y>xP`=t{Mf^`}N@CU;b;j-1xS;?X+_MQSArZ zyWXfNiN%+{$O`6ETadm^H<-xxHtZ~x$XzsyPvF+w#$saG8~aUv41AKf4;Y_xRT*kR zU{n2l@cl?mJ%f zw{FG5RmKqp19s5H))DO}wUw*k3ZS*e0#LO#AOR_AG6WENDuN2qe=Qvxp0Dr8Wjt?0 zN1XJ7Y{?I--3P(V^m2t>I0Z?YYugDD!`QSLu{N` z2tw`!i--FcoHq;5d<*S1_9qOs`1iO=(@4cLN#lQRDrx*V*6|o$|L`+KA)6yX)$ znitLqDw=;9QAf*#3dp$3k^$6|=?Ptqq`GP#h_}g;!Q8KdnaHErY9TI%T|z*PkqG$H zm)5Mkf9msJ(Ie~r&3_84IP=I^1#z@9z3P{jkA{`A1N&~LqN@&n?=KVI=rF%C%a9X4J z7f~5?`2JYCZZv%GEk6SA9BPw`xm@GTdWN*Ys5GwxN?gqcSOB_fQ}>n@w_~jR z@xIY|pb)KHnaNw!lUIs39HrArvxNxZ%doAvfFT2+uraX4MDsB03kab5<@xVg$!y_p zCO4hQngYQ1JEb+CP0d3X^#>|~sG!wfZcjQj)IPPlUKG@s&&c#XL09KXCJh2HOiouC zpLhGKqGbN3jr!_~=%m4KU-AB?wc5Cc4}*dHUr(1>thxWuNjuI<1B~{w4?7J`j+a0ReYV zmCC5+Cl~~45XO)3bXBDh<`d}*18wkldzObP2~wB(Zp0+e=)ThKX|y$%Fp(#Y_R8M8 zd(;HVY~5FP0JgYrV5z>ka!r2W5xxC-=Qgl6scd zNzlq*wzC@zbm6aLhRr9UkCP30mR_+y)0n~DxP})MI3W%smC9wLj+j?1!bUsJk8`7u zG!bc*?Lk^dNLey)+In;MU9-ye-|cfkZ?ggq={K_4?e1hyYf-_((b#M|UUPDj+CsAO*0zHsj{_`JOa66qG+EOZ+MV!Y_P;iAxU z&&s~OPC}5NpE}G$XtK@zm`PL*k|h|8{;+(=*PSCiAGihLAVngouRp6=IcC{W4<_+9 zwpJ6L-fbXQY6GxC5d{{f#a%#eEWNp?0C)BTgW}hP@A2Nkb{}ftC9Bu)Egn)=u5&FP zqmUX%(37uKwZQKY*$$G-+VOkPUtn2L7g=H(Rq+gRPjfct1h$byK{iUJvsFLLa3&QN z4c7#R&_9D*&u6V#3Gw5M4^@Nam+#Ya$h3++ayBFZ+dUb6_rXR}V0ug_ombX1L4OTtf7K^cN+A+R+~HWMHb_#iTz7L8TDH4VDSHhXc}JP!X3RAO=6S|{`-~02Z5^0tM1l+g`L)$Qu%gA%uGSp z5`H~PUQL1$GM1>5*%|>C_*@n7PuPP999rIrYI{VcPBSXUphshyk zo7s??4a#R91pQM^v{=Ce>_xBPb>!pf%M&8ShaM;QFFwlr%V#xiR=|m!HrC&-l?6fG z_4dAfheqA)pMl}0UlP3_PKX7^f9Y#d97yn#9k#;qBKG7SaM=qN-jzYc4Y$iNxm%6_ z{VR9pknEp8CDZ5q0AtCEb&d5c`9M6S)+!BB4vu`Dc)mKC?FhyrXGkhWRRRCw-b$-` z-8ps%Ax>nK+Stvs4NP0rn&Iqa+Q$f)&T*Wvzs9OrFoaq>)ZI}Kf)IY2n@(7cm%4Wo z@t=_1g61c_eCNGwGh2g2E^10SzwChzdJd4l9k!g1Su8<%Klm^j}qzWtU449s`7x z=!e2U{Jxqq_susYBnh3gEEr#Po8B4}e>)(M^@#DWMz;7T-V6XiUtD3_urSnBPyii6 z-~tQ!en$LKf1%V!vWTTexEBjL#5T~q_@-Z);YnehfX0IsNEhl%0!)Pjk8%RbPL<1H(># zaL4ox-AkW%@8xJaO)(7gyn`!6W>$^NLAv}%1fxJOi}8Az;XZ)2UsUQiH3PD}jVt2i zvXJ9$f}NtjWKp9~2M=vH3Ermg*pzhs`9iVSF>xgMcNHB>2@0G!=>&vSTd5F6=>D^> zoz%Y`ZLGYqIVznOqlQRVkwNOG0%{%x>&Q9n33|_QIgG2gzjRk52@LAu1a+zi=o+HH zVx81rJQ2c6Nd$r9#vG+FpjXXShf!L>Z5$9|JN#uHm=nT}q&U%|vk!TiNAWz#A9bb4 zJn@3T$O+`s#vDV^Ij-N3f(kH?{LQ2T)Swe^!^3R zlaH%;H|#Bs?(26j+f%^S^qjh3wniS%$H6Y#{7>TKBHJkk*IkbVeskzb9#{WDI4J0MyL`n)SCFek%4=nlG{*KMaa*?6my8B9k8 z|4#WaIu+X0VgK4R^*(WbtM9+Xqa7>#qs`Mj_@oR2Fs13LKqSiNfHs+IL1#AXp z;UDLA*CQhCiGXlqeuv3F_`Ur+z)*h}Xn5kg_?C3B9gLBb;xYTR$^wPp`rI-}(1^gX z+2DuW$b#PXUO#G}X{P)06GQ5%zh$*+)30Z#HSx}~)l43n3g2ilU0^+QAvM8r5!WW~A4d(ae*$KMKxABn5B)ep+_ld=OWRxiW#I-NsDy`q z`O)4we$)2t*pv;l{a}8H)KTt`UjrW?QA=rB@1;@9cN{?Atw{o~(R(x9?a}-pD2y-S zYkom|RLi#kzkxQKcY>u<&jG)y4QPOzpgRewVo4qD+?f*Othm&C-?AWU{TCFfef@3! zxE;jZDxR8A_+aw8;=b}>t0E$Ey4QnLmt42KkG(Qj34*HCGpxAoy_~S?pSZ@ghP?LW zCKeP;-bDojU#$?ENBRhC~KD-}AGqG znI(x@mj$Lbe@Z=o?{hVNWTj~G?!Ho>P_e4<{n=3fl{fGT+(RnC=}!W3h+#8A`jwjB zc~$ki9UsIreeL~ovYp=5A9QmeUYN0w%Iz8E9`N}=h*&;#FEYzA`fa_R; z)8Pe75b1-&nm$b^s{eqSO=1t7;eh)F7*zW6zi`5o+0!QfN8sC??xO~=3leN$;94k{ zX|}VRTo7{H7hmz*OCw>RyUi|Z@@V)0wK+7t@d`XaQ$P{HiGPQTQ1=A>J=5Q(&ElI* zVVHQP996=O)c;=U@vK?-vb#7H@obJlPydqlI47I^}np_HipQKXgBh<}jdfgoK0ZhGJ_QnrqE$ z6IWIa>U6g%7Mo0RF`UNU*EZ!Yh{SyP7h>=)33Hja4=9=~ipKrF2lDR`If0Ppc0T>1 z{QdXu{_{mw9E9#e>#90O^`Axf_iDTbD8Se#!(ev)4JG{7m(Ex)uwBoqY83u+#eeUT zSJ+krm<@D!SaFocr$%1-~i5B`1ioL7!tzl-yK z8yN6#Aq8$Mk-O{={eQo@FiJKQdas}T|9abivhDw0@c(UI|G&K8p@MBHK-;kECWKPu z^7Bt8K;;nk2(E&t^?pQg8NYRi#9ovWNBggT71S!Ct_L%|D`lgxBT!*W-FWXgWbZ+NaI~IJ- z2QJ?Gk?$LU;|c!k3%&_Q?-yMsKG_F-q(?UM-NcPFYPo0G5h`W#M!FY01KhzCKRP(_ zRDV11Z&U#gxEM3MkuS@LuqU?qL$4>JQwR7jOoP9?t4kkTo@M#O7N^=u4P;bqYTUe^ zSGZ6&fch1mM(`s&)Y_;ove`2J+wCYC(EX1DWkg$%#Tz#OGAp#A|7cBmB@MBPy=-m9 z@U%Bu#$aj*C|P5XTh`XVdI>Da~bez$9{{oqF; ztJi0d?-+6R3j+5O0bKcVpp&_6NA?NT?F#OE-^6~ovm*n(LWitY#{+-jBN_LF{6IQ! z*P92fivGk!TWJ@%2N9_5uE5=meSxk=-HqEZj1X8H9dC{50nBhRQo<$U-~k z!_scrhxnsCBm@FY19#HLj*g8ha0f2kBIoDhXKQFl2h!|OI=bVu#*NwCEAFge!~Y@o ze+cpG>|`wQZFzGKw((`1K@uzTEID<=f9^j#z!A*ieVAYG+}>TId~cyf_P`;cQ+TlR zQc2!rDh0NwAmMa(>b*Om<{UbquKY0laL}yuy*@RxdGXe*A~XSN^ESKSa}X85{4nb$ zf-JDExL+QD{KtiyUYK?fTT-;fp~x9DPo&Vw?*@z4ybfl6k>MkE&;ql^ID;&JumTF+ z1QhwwrdkP;wJ&X1(T?uMTE3c0k8>4g*5571KI39vN(oLewZRA5M3U@1G}xfoDem~+ zO7aD>dsVjeBpAhxf*|-MSIFWlTW`{U5m%wwT%CqppUbj5N2cDnbsva5euT9)ccQ2` z_c%`^zlZ}iCjTOe4Z0WMy~zRoJ6>C7JbjHg8m5q!u4*#e`-jh&8<9K{f(+9K_Bwx7 z8V^}Ca`oDBMR2B5&}b(gQ^pErNiB=9{>cW0u25+IJhIXNc@th7W~nlwiz-YOI2(|V zi?HBsyh*Fgf}Mf+3|+gcYjqF+kA^iljY!&gasN6f9|71Z7nL1UYyLNI86vNP@<9gU z5xhI=gq9@G!8h|L%4`9LKm`zf`A+x@3k&BIjd|b_d9W?NIYsJcjzd6PF$0SBQvJK@ zVml51>q?*zd~GTa%(F@AOE%$iH~}B;X+lkq52}q5{TUx|cE8%E*SU6)>hy>xINe}k z&thH4%@5@Ju5O5MAr|9PXX zCi*;&iWK1d;f>w@!|RZ?qJxWUt;2vR`!T7nWGV&>uaBS6r*rk&nX|MoDmLX!dtzmQ zRbpQ8-yT!@iW4=Gm5!BuZHklHLrU7u8!jjLfdpZ^KivD@P=R?isW7kV%T4eu z0^dP{eSiB?qk0`Qsp-E%1kyoa7Gx=DTDA&!LKBW>f?@cxbsPKc|3e(#fB`TdxI5AF zIfUhG=Q!gt^X0F5=CxoaRNHOCMQ3h)w&fOg6=LfZJB^=Soj+}l!WKMyXOmmMy52eV z3a}9(5AN_-5>mp-rZg2!Gb~?nK0BS9)!o8p3T-M#5>5z}&A{JGe42h1yT9LTpjG_# z{#NR&kB;67Q4bVJ9Ni&P2N7MrDM+iNfOvlzOj<$dbNBsJ@G6yB@yHR_eu;LmCdB<` z2K59xX`g$pgw`TITpY&7$X*+<5Er2&GWmSU)9a7_ej&j4%`Fy*+B9`Qdvn7qqg04=jpY{vn zkEG&>i&_mnR{n5`zc%*^PdY&AR0vTlpvZOPrM7iM&vcc51CELft~Jfbh-}l{@k}Fpd&Q|JU_}sn8wRC7mGR!~3T8BUKVO^Mi3y(}( zX{|CkL*%UXE>yIv6nlLgYNG2*+ilH7r&~CGRE*XL44rA2QcdNw=W@o z@B3b){Og6&zo>)Po6*a@gdY9RTG=DQFZH=BQn`k^=4A-Orc z*2a~Fw@|O-RF=%3ku(x}Wlu3bV?3ZS=CVr5FrbiMTgfsiyxdB3;dMlYU9%HwCcOaA zy3_EIKT=OT$aSRD7))7=RRqT@b6a6Z9wf51rwsr46>pcU8$Iv$+Q+ghI5j$&BV-E? z#0`xoAsDGXygffKmWIhmEi~=w_o^{nMwhK^8HPm>(?I5DJ-zxdZUnOk*#SEY6Qr-8 zd#~epw<_&Yg4Uc@hwcuQp_n^(#yC#TrVW&(Kxh2?FUR$R_{h>#QVH7!p+;Q2kuJhe z49ImfSfgZ7(M72$1sPVcvKSV)WQ5ks(5~YUqV78qC2Jg>{j=--|2Vc+yd7hqwd(Z#PY?dm4g(yiDH3Csnq6qOc^Uz zoa|#qxHCN|tNIl)Q>~G4@x)qhRn#W$(?mbo7zjzEu?3?J6V@e^KK)b=zDGHA+avHd z+}!!IHIxdIC;NBMj%B@OKC+UON@3_OF8FY$I>k33qygl|Mz64oI)?l>(z7&n^g-j8 z3Oapg+q{Kg`pOE1@afG)L`z|Cz+IV?E>NimsERrt3hvI_P;Dj!)O^*Xu6i}uC_nd# zzNUoy(I_DU_r`y__Xd|r?Q(BOr4#Y4r;QmdlesnP;thvmM%zcY?e`X`$V_@jJ!tVR z;^vR;Zk~--mS(u*I+xt#ptINTl^c&5sVl}Z9VLy{LU^yaxKo-@Y{kiOF@#qpsmNE_ z6$LS#mu0O|f=v0UTW^!>pvAv`HtQOg`BSPwn&2-MPc8IK zJd->lyG&bjIKRtMTdu-x7h!P=qO$v=C-g9%OeR&hB9qTpM^yLdAc=l9sV##^5I|5YKICBFrJ^ZIIZ}r-Pyq^`ck7jA+ z-P&x&a6b-3avz3H-$xvErM#qEe(!`i?o6y75Bp?q5_RcXn|i3; zraFSRRM}vu#jMbrLnOuyGOTxxQq?&Z|Ad@ic`Sz25g>wCn3VY&M31~D#VVJ4cwI!MJA(~;*&zK;@ zX_o*Lk+z{2rvkg3)ms?Kgp zA_b^-8%`0T4CYO8d4nbh8zD^wpu1F{`T6NHr0V0I1j&XU%X*>f(%m82DN9Y>BFjHb z!3kESv<8`$7@lKTItRiJ73q9W{44|#$zMy_&wSN8Jm=D38$N5;8ZNkTdX8bNa5?4-#S$7vp9UV$aFCrLcE@Uq*GAF0j9**c1USlkpKO@{VNW{JHG{xZ0!u z>NrHaPhZ`x4~v+l{TU~_v(0g=1^4ZK-UJG@LB&DPTi;A;T>W4f*oaxqA#x-IVzF_F z(cP)*Vg^V39pVpWMa(03oY@5-YyNtxu=QSQNC;WXTB5EsUXt~61(gWodRvpeTGT*= zt2n91ex2E~%%oEC4=?a#=ROT%j?!fS}fnt%1Z1yVLJ6SSs^fdpLM zqf=!y^N)uwHAVwE`?*UoBltX&jn>PILDnx45_~hAHlgz*3omyX?4wyFVHxv{ajUs# zw5qz6tohDgjLH=r+8C}1Ocbp5g`+n}S{mR)_Xm^zd<$hU|B886J8CQeHx7g8eb4Fo zI|L?vqHA8v+Bsdjm0_6B<)T}TucZA)$%!owSGGe6YZYo5OG3`P9j} zNkAT7>yNv50oGP3ja_;qlZVE1A}2bQnE`VM*Ztzcc*08nCMx2ZD3oOYETiK2ULF-7_DJh8zr%ko3{zyq z18zo=C(CUx>&Vyj^4%}*mcIt1QJUl%^tS^i2F&5>K{nK4scU9XSDXlhc6o;0T(-V+Qe=%EI+=0Ci(&;mz^9X0Q4a>s(0hMZSd=mt+FG z?baMd7To$ffe_a|xH7Jnn|3>!p$^a*?3vrUgi>M!HxJ^Z2p8edSkeUN2bt)UH!YNM zVtrd(RX6p^dnQqFio2l{Ml-1*`(22MyqpLa3ep3HA<|-1pVWCg0J$1Q)MQQpEUK?1v3VS{wz_n2g)Z1^;msrVQ#LRU#)Fme7e|q1aDX4) z5^DihR@qAfQIKIwotixsNQ_?GSn99Q2nttnWPv6O=?zK#q#2y9&gg6jtxHwT z_U9~YeK`S=_A$aP##1hVUAG!-dn)nA^!as)OFOZ#Lg~CBVDG6hx_o`}J0N~ux>8;T$op%_tMf)P!O|E-j zfp-r~F+P>xIyP$yD?Hafwd2SI=1I>y0l#=sF<2e4GNiR^S3}}POq#WKD4z+9gVJ3m z`=+#QI?N1us>TH}wYKc*m(?=nkR-7^KpC-1QBP=TZeN2+?NG_23uuFfx6(v%0zy$G zQ!sIgk$Z~FlNiKm(2L1TQ|rcsiv#F2TkC9$>9a1TNwZ&u#y|glsorLaJkaL+;N$WO zb0*mL3YeuHHs}yuh#F91NCWc@{p*x$CbydOC?GDYTJ*&VVAbmelRfiiWAn_r9DHl? zCi;`Y^5rceRyLSO^%9TQ0@uj!bhXPbN(RpBX-o)t;O?Sf&6(&h+e#H(uiqI?HU!%c z!>5X&EYd+RdKa)5`jY8+t=2|OTFBIv05>25Bc5v;7xH4w&3fUz*a1t1<@(kj?NM^6 zc0$wE=QPxuq35%B)v{AfzE!|AKqfb8}>Z0Uw^3OyyVQ8kAMh4)wu`yn4b8v}cz zWD!srmKenwI0^l|<3u6=LA*|@%)KDf9AR!DW67F=@3C@kcjK9gGg(BN#Zw8*x>%EV zLruE|EP@P2Zdi=j8fRLUryi#kGLh7dEM90DGs-<}mKu=EH?Oru51Jisx|qvq$iG&T z>xaprq4D*t)I1*%yD&`n(v==UGudjJ5QSg}FvZkgU$dsv6u-~pYSOU7u>H6*I*&g4 z86b0LP;DM{!i%G{2vlq-J32E0eqvxtP=UQN%< zMc=&EO}Xl-1;n&PGYTeKeOI@ALFh(XoP^}zJP2g_slv#?!(&KP;rN-W6_K@So?fr< zP2OXv96hxg>tOSqQYd{CCV=BzUWU^(R(WWO6|`uusBp*r>vWjXEW5IB0l!T~ZMZWsSL>D>5dGca@hbtL>+F{`nkiHN?dqZ}o%hOO5A?~mh8bH~2 zG;sETT3)sI157ZUD&PnWo5%+U>=FT|G_eF~CB;@k%PEBxCrh%y&Qv&fWq~S6W(poV zteyhEnW*TNfBPszBp_LE`-$)!ky3@G+N=5aR3hMMJny{F;#`?J8c!AH`O&yLxO3ou z_~_@Q!aDNvr3MbkOzV<+t0WPHM}JizsV<3+Dz+9TUNit4%hM2ox@zL6xZyV`6*ZAo zyc~ip14d5sI<^17&CkwLAg@FaAs)PL(G^h1GzfGU1S@~$;;;SY%h3y1qbSnrRQ>Gg zqwVa}LbTG6TRvf!!$BB*$eF60ft(RAFipunV8xx z0%~ve5&%Q%hIhnBrMW*H4T_(09&{wH|rVQZaY7K^zk|xt4pd@pePnXd@B$0f1pS0l6X?%qG7R3~3 z$2rmA?fv0Rk@HCrkpa_$Du5s#3XOcGL9fu=j;Gh+@_k6Dt+U^esaHo3961MHa9NJv z4O=2kH(2sG%id!rBHqRo-lDxCgYVcu5MPMN)1c zKpP$7VN@Hx?#gHBuBoKpdf2_xlXc%MSh-z1ggp14H1yBfzVfH1k)41!#ngL|F!bsZ zxbf}M%Jx`5*E~y0n8@FC%}iW@)l_QSJBi}QKc8>jPHkGd?U0nPAeip+J{^C339e=PuN}?d zd)H1b3+Yfk-6mB=0N`mtRc)|H0m6y_US`ZP8{I#fCe3en8QFpv6)QA3JHGQ#@&O(? z$znjc>QubWngG-^pIaWu#~D&?w-N1FA8$Z^m&fxk>nHe7sAwKN2bz3f^ zGLpqqp(aOycwerIv55)d+Ah_3XFvo^(z$`oSeHmWi|ZQ=iq3z}^BE9UKqx_gOJ6zA7#0<*3lU))=~}(5YBFuF(3l)5z>ewOb=t}Du0$!!5P_i+<0`m>BLe3G?P@}x7|f0=esv?OUQXWi-Ruad zbd353+wI-wS&R>*Pu>q@p@>!rvJDJbxTSVYEU)32sxKgApurR2yMkttWIP`OP^TwF z;_pPz2N0k7KY**)WWVmN5@6r9*}RikD+4y)1mw1FMN=d&8ujE2OogvJ7g?7 zw~$V4`S!uX!Z43su6a9(gru0j5Q*Vi!*Ye%&NJNdx zFjTWU#l$FCZIdq1YDD0ds=AaTI2lkP5-5Tw6(u(p2zmTsypp*oZ0>4ahP`=;9RU~o zSPpA#w#j;myA2v-Pn(VJ9;cIOyq<2pWM798>BsCwK?BZq%U?T0CjdGbMLs<<#aUbEl+|HYIF^C zUQ^{*-Y-@vb4~x0B9U&FehU%AVbTvcKdI2^Xx|(j7)Rvh40n#;BhCYQpt#2_F}MQl zddaZpaeo=klxxf|7}Z3tlIYgs9svmyEWswk6h(AJ?>w_Jh^=%|=1xNK(#!Z*0N7Cx zPyv%qY16_Qm@){#jg?8BY0BdB;c9s<3G=9~e5dwjg;|Z?AxSbH;{pNrZ)6H#>y-!q z5CtlQ1GeH*c?Iz}vfz(1ne~!Ji_YaoOaV1J4AgXP;fe8kxd`GE*UuCO+fva;7X`By z7_= zpMzc}Uh%Dg@!oBwCG$}q!`5uYA2XJoM)DQJ7*Kwj{fQ_KN2PE-m5`X=e4ACV;fNA{ z%@!Ro)r7I)!RUY}uS5h+Zc4ZoHk3diJGz zk*FY!D&BgGQu(S7!i45!VmzwwxVN*pSnXc}Ag^j{26c72&n=tu%IZ$oe*HxZmNyNq zlJN4>|MiCj(V)LnL-=Dg3dYVLOJ-Rk~stiQaHR3VTMkdPQk@Iz^sI;e6v!#;7wADPY zI1^hB`RTYz=BucS?GMj^bCdwpoP5bQPweqCq`3~;ggWf=BvLlvWFmrT6=KDfHG4Ai zcH)u%kswIJ>5Z2ez97;BKz8QuH3|_x!U%s46U=>?e&JCA{CUhcV%?5fiajqeLGmQz zD5$dqjNp8L#=-F$d)eF=A985GD*X1$brJroQ><2Z@qOS0Cw${LQ@hbu0nlfxk^}x- zyFjtB-BpPttG_M;1A|_Z(uUxj6SSDdGf>S%JmOUkyg}t1*IWRH%+VtPmvpwU4>A1Q zKvhvjXY<`j$4iOcRtC~%0^~nN31`Cm$Oa!}r)t$vtH98z6M?xfV*qwQc?OF?Tj{WC zrTUXJ(OiXo&~N3wXv3|y>QZ37&8a$7P}pS7y=;?KUY?F%xk2W)$3-S0n{e!CUuEK% zmL>7HdDc}-UzVqcm8fRoaivPE#!N-=W26owkeb6~hW=6aVUFbCa9ws|wTV2VX!+&2hQQp9!QoLEz@qjFIdUwnC}Pu~{VZ@9B(o zQblSU_h`k_Nt;od z^U`7bd!X3)vDOT^U6+mRGIwTkA_Z+wqR1wta4VchTZ_|KJydCHzuiiem4_DX*uE5? zK#S+|B2b&y>5|DgBA&bHXnli!(AZ&F*#Oj`eD@SUf+{XZDtnJtISf(^DCETkgg#-8 zEi8+V0B?~1?pN1Yps7UzK{AlO%{&IZfAjhzX#* zkCK)komfXsw8vz~Ei>81RTkDrdd#QQ1Q0*|P@qnh)e&cpRj-~@N*@hBwm>Rk>N?^a`8AhY${RMvL^dqRiLo20bhiGHtUn{`mkYzOH(wTDB=_@Pi#FTvp( zOjb*#yz+Pno{KPaY84U8fqw)#-29!fAc!zV2J^RY+!vLeQ6|_l?Ha?-!=)CL)pdvC z^rMJDAOkH=%P$BC^qu7ZD0U$zF5C5heXrw1L{&`MjyDY?$8aug87@FqO0e${(AcVlGe17>+=7__1{O)YceP=B2Z7jCpzu3-^| zSZFdT!BCBu^%S^n4j3m)y$5|hQLX=aj?O?=>9HuJEc217&&fJzSX486u+p1Q?HpZR zWT{CXHywQEDPHB^$M(1Dk2(Hq+t<00nw=Fgvo@z0uAk<)>0DJBm%B#2;h)59@_#xW zg-PvQ71z}He)u4qHz(Ad)EJ)pW!7se_Q;D*`ghrH6HVvM{XqK~&)yY@M**0?LFL5e zj%!6n&&LP-xjLJlDBn#3^}6EJ2WKpaMW_`O7Ci34TJ;b=nC5*qj=Zb+eDmt5O9GOC z$^MEw>vmB9JicD*Mhm3I{m1SOW^>-#_=VI04Aoq#YwaXZO7|8ChZKlg;y6}3E__R< zl}bomY}{YrY(Ng}Nd3%DJKt!XwKV{1A?@0Ug*XsRx zE1pQ35>F{doNAK4LB?qh-D2<1NMlTYAD|Z&UN|4(_20jrj$vp?jN~NAK|%+0^Ty1 zASR%>IWvfcWcg2?)&IlNRk$_X^?jPrqZ>wdDxDiG-61I;B`L`08YM`lDBU55q;!`^ zw;-L8()s?ppZ6cwwQD=^J)flKX!J9xk)?2$^+I%FP@{}TG$$9$Pu1g+msr@my1G2G zY1=;2T4(mS?)dO7Z}ZUHn#DERV_0zedc}G2_|PuZR_if|l-Vh}JL$Q;<4g|T-;z=X z0j8#NYyAVN)6kWqufOYb4j>;P{rHOLa>F)gyn$bry&YFhaRmGmAcK(01U`;}HBPUi z_M+sKjs!x)_f`RM3bb4r^cdCU+25t=H>w8DU*Ug9o2Gp8)aaHdmrZ>4uKW@<4&%;0 zmGLiqg^YJF4zN}GT&WgLMc3jcm$_O^33>5$1V{Q_g=)#ra%wV}>uV+{ho%PGQ49I^ zQu-tGeP3~b*Aou08z)k@Nzh!iMYVheV1D5wRK$_xYdwM0S$!lVus_=z@;F(AZf&C> zqvd~S@x1(F9MWQ-(_^E{)*9{lyS+{7CwP{eQ%_mYW-zx_63A6Ri6O%!#_O{CGyfGu zHC^WMT8{j*lLR^A?&_FN=HWa!VCnCeOm&*JNN>)+lJJUeK()r%R=P#`JnZ!WlOHnv zvl`*+)i8oTwz8i8u-b<_{Q-4B9N8!dL7OPT^D%Hs4ef(J5ZPJ;gih!G1rj|pni~Sn zCzK}cYbMx?toMMtJzV4i0F+7w&|x5KSRajGUb)`!lwf4d&3ytB1?Xf+0BXB;ym%?+%o3-Z~xe-H-{ z38fN8(?Do4&K9K=mUK0zDNIi^SplK4A;D_PPCrh3-ytQYhfKJ11oOZ}Es- z{}%UYuvIEsYKY;cmg%MAr-BjwY6B~aWvvl~pRWYV*dCqG4R-^OtGE?fHIX9s$2T|h ziGxdtI!*riq^_lVjXhgq?KxZ{nNfdB(C36UrRtp9DuooUyWI}AJ$Ez;KC()S9#=*kER>nZuL*arT)!O^R_dl&dH8fOdjRGwNBL)k+R0-57QM zplPzNf8`7k`FI(gHGokm$=`wMF1 zl067_ncyZ3X_MiIw((H?3834#4U50x1+}ojaJT(&?Y;d8$OykiZEp z+Y?4jC(H1v?Rp)^d())XXOm~Zbq*Q@)nG#R0G!)!orLc87Ym5bwe2EpSO)+ELTc)* z2k`#|7MD{~)1X5nm7f3v$dTSi7s?%W0nSj*6jV@*rO2EAM^tQu@FOBkKC^0kVxPx| zsL=qBpm3u7^LMyUaly;Vo(>awq8F)12)fw9@M$0h-|Gt>z}gybK{KG@0k?HTsw+M_ zv@D8b+(ufj$Fi#sdPcvY++D|kK&QCNNN6A4Pc|c@^_r(V1idzJqDLKRik51C)^HeFhKtE(S-?_T& z@}vLN=fhFXVd(O<@B+?y>~E0l{SBa~*7wO+7qz4<{u&+T)#+xHi<9vipBM-GEeb_0 z($&-rPi<3| z86B(3L5$gyyqy!7o}7t{&_BxrDw_+T#Q}uQd}JD*$sVPHg@4E=$r{}=X#b;w*jud64Kmwxm zOFjRh<~FnXl?O?aui&QkPYo*0LDMy*H?=UY=69#9g~JJACb4wq9X^e}H_yA4IVqlD zBU}4>!aD77FS5TWW7U@{P7&xQ9FE4k9z0!MD%|{b!R%B1++$jlwPlV+cb%e1F4+2? z?*oVt068G<0zpMUoF5t$NWe`qG#AMV8H*8f5&##c0@>Xf!9MM=9;FIRbcUen)76b~KU!`9tm%v}4Kc{xEmh9g68|6y zi2N;WVjc8+y!ZL{@EjiFBRWY#`o&s*jVhwn`y*4>`J*h7;8QFnqMU{R5IoA2=F3SbkLV|^J8?F0hlJ9ZM8lsFG$0+u^{@R5}oo!CfrH8T%q!pZ)FZ+87efG6O5Epa*?UF|e~lEYn?zdKP5t5F>bETr zAx;uZlGqv0wU0?}`Zf`Od~b)iF>YQ_dRmTf7Ce7y!Yz$BRV9>djazm~S6=hb{~&UP zr4o(?OOt&Za88T-A|6a3^*W^c6Tz#5xUGUYrg0B74&dz{N`S&HQH;1kWjMG1C;ea) zHvPVcpJi=Cjc0HgeCZP98}fd2B-K0sU^yxZxT8EN+tmnz#lKp06(n1?ocl@%xhml@ zpqai)t=;7L`xgBPvA*I}5?jpiT@?kIoZCsS#>K!-zZ z47c0zr0WnsSviR83}n5NH0pkVB=Q&h>14ajtmWPCFo@TB8x4n^S*kB91;~I^1?=#t z5HxQRW}{Wemv{|$Q`pQX0D@0*N^UxwD(ygjC1y+l1sHk@|A26Uh(Ur0(5%kuIbqs( zr5HURFHwYl=4(m~9>Gsb0w^YoC@Bz0AE%$3yR?S38;qc>kzX2!r>{000Bi0ZHvXPn zumutv-_2g8TJfV2LS-oOTcX%{Hl>jHr?Fh(whPT_&&7>uprYYM|CY#do?IlSJGze~ z2M(F9GFZ0$fKA@OXuU|)&QBG4jkHWQ1jX?^-~Ca&5D|jr?c^M$l#m4*1fQXpu<|ki zIJ>Kh!R@0vbliIo#U4STw8+JBf=q^PK%Al9+yz5=^`08VWYe7mqw58Z%1%s{!B;XR zl#&m5AC*%%y9;HI z9v~F>xc3W6_bzMepi|Tij;5eBnouEvpH4jZ2jW&|1gA=Dpy%SN-%D)wGG5-!=W&0s zGr8}ojhh23^+eo%AVh1vp@m~k^_Kq4EgBGlP#LB)ipm|&hu6sDK45ZC_j4e7|FdO% z)LXnAPXKr?xUkq+dwzljIa@`|Gt4;#)0~EAeW?onRKGc?juC=Ej#k#^HgJV2Fq4CY z?3VcDH@)*H*9w4syMp2OUr}6(>Z{%W)XK;+3gmi@M|;js-x-m}-B^iEFhdd=kYJUxAi|o~ zt|gDd;k+szaHS8BFriavBtWXWQ&y6#6N+xi1iA10faBIFn1o7~MUR1`F zGzr!>O5I@+bfuoMHfX*IW>G$2K2nopXO461XnOr&)UR+Dpt4YD<~sVro&fo+8D5{})mv9Twzr!u~l|`oU4CQf=JB<)mBzf&D zxh+>9zf2%W_faP`mN4p7gEBso=^sZ5q9#gr<43CVs{NarWTn_9+2$Glv^!`+4;ZWE z-_}(^Yb6B9Euw%zG!wz}nCk>|p5R7|ZWL)ZfzX67j#s{`$hh68bajMpF}p!45l&s8 z(-7b_II9RKR;JmZa3eV+edg9Yr9kGCa7=Y>COYtXysR6nTQ0jHzpJ)Et6N=)?s9?^ zEug=LoK7?ml50PLRKNa6PG@2z5N6ZB+tjg8fVFblV~f-EWx}N?Qg$4<+HvOb_lj4r zlD5}`YNRX#>}#K=dfX<0%n|AdG(0^}laT0@=PIW%Du{r_V@c(OBXNL< zu7CSvHL9^@*@!HtH~boD%)H!AeJ0V<{_HCIn%tgNLKFHes%NxJ$BZC| zP06!Hpp{XST%J8!1`bHE34UAudUovtsT`pE#>vdvAj;) z$YcX4Vc`|F4juG8q_)o7jweVds<_)cf)d7ECG^vfFoK+?Z0fsl#V~v>Ol9-e*!S@d z-;_4YB6&2p&8}!2=MQx`4eCQkwgb+_E1P$pD3~VH8&}@At*vd#rFFp&V-n3Kd{zQ;X?$9Cp0X|? z#flL-o>}cNWv%t`;cjGzL(()iuRzi)A)6{2G|mFOgwTYsAuE&HpRb~ZHGaU>)xI#P zRZHjnEU-bzIt+O!D{A%Yg*V1wXo42Uw>9VqaAQyIP84j}%#ge*Cpd{3cdA;Yh!V4f zgg|CvYQCbNW0x)qa{o)z>36m>vU1JoZFvt7k<|*FMzB$y`?v*ly~RiH!ib>5>yO=S z1jiJTT$b|@U*0CwR z+^$2`!KH^IBa?Gm8O^@A%ofuZ^`f+4!p6Z%v-YQZGoJHpHkgS~%K0z^37gdVub5B5 zPjYVPjViUdxWfSD{WYOfySFN0i20~+QqDf*`L1x(&*LSvbo}F7O*_|OJy_14wa*7S zt=@N`Lz5B*ZC1pkp~~bx`$0{NDB$k?w&r(d>4IMVpD!Qe;fB11h|qx_Abym0?`12| z5)CUESBCAlcp>hF9khLsi`YrWhm57jLdKxFhp+)s$+V=& zx+@2rOS$?g5Pf}>+0m1C>bohHIYOO0ZiV6Jjx(GiC+QL6)p`z-CPnQCJroAh`8I09 zy$NS}Uyt9Kj+?eW_Y@Z7*@y1e%!bBx7`-Sjs8cJKpEIcU9hfS#het@VH2%nD#H}p6 z*R%hX_8RXh+K~Ys$5(9j5DzKv_?guN_2@ zyQ?d4n6n!*U^ZFWkp>d9YMaaY(XlM@h<6dn8wvZ-p@+$~ke z5;EF1_R%YIskkC_3j3&Kaem|3N@erexLn85so0FePajVCs8rj*6oo1iTS~e;*%Rovz;l#Yzz z@)WDV)3st^)%d!|3xDpw_QRkegR7Pk<@(ev?g}N4Wnm(DAVhY_yXwPP{zG>$Pp+zCneN8*rtSap9jC94FYVqtkRP)r^vT0uEKW`qk^oHn0&%+@vN6m zUK*&Cq!=D9V;!FM%dla?e92Vj>o25^o17HLiot|#=4cs6g6__Qq3EB%AX=tA3D~#X z!(@Zr@81MaWIHFZJ{6T3FX{H1RuMaC_}*ZCEizoLAC~J}q#|Z@nocAc91zazeJH=m zzO(sZv!H$~c)5jDI)Lx1=a=>X9A1w$l6vjh)Q(IDZQUr6G5#p(W8+7On<%|slQY?@Hag# zqQ^@wEj02{%0_WW{RoMe$5cOWtKV&(PklPgUk{cL*`1E_BB8lXm@FDocAa^nX5KeE zEPTr5Ju#WG(vysBSgNuhe?8c6LyWblIXBTkJz{T|!243QvTw!^EH{8Arkk%w4D&q{ zVpKcWY!I3byq4!=gEevsc+rj>t|$i@{+v9p8QOjP!1W5HUD;s6tfBk|{%e)bmnOmu zcX~c|qWjMV%?08$J5CmYUZOOk~EStobAa(YTD)v^H zHCWA*6C^Arhn8nFc!UaoSA?qfq zT+n-&BFbJ4&)x3-@$jnvsV}zq7yo&A>WIfOgqok%3ZcZ-uT-FZ9h(>C^q4TPHWc)(?{&7=3!j+Ov>j4zLv`L$%-;TTEIHf$&uR-E z+6%W;yWJsGo-W6%dGX)+{_q#3C$3MHG0AoQIJ7g}P@w0dUr@(s-w)C6&d7UKXJI*h z*s`!`_EUB@we&)GtpFEoJjPP4G+fZd0pe z^qh$G4X3#=+VrtR@7o}z?m@|omv}h?pm|(mp{_<_RMW2`5X;A>YhF|=l?U3wXeHoK za8jJ51qoyc7FtBC<_IsPmU*;}-GH_Bvh+W`PJo`^!fsz*FliGi{1ow}BV_6XWrQKI zCc$~_hh71L4CR{I3ToreoRm!h<+%9bFNHL)`)Nb?%@4{&<3i^khBfhTe_fjqSE}C7 zOWA5vI?0m8FKO#tssn*)crw>|Niru<4iYx4xP&-Z*eT`$qGIePiIu298ZKrmY|i%I z?wWkiKGWDt_(FPwq&0G&1tBBjL|T0E4h|w1LWdIa1cdSmMvZHGC1k^;$RhH&1h*%NT3r>D`Ijb-NB$oBJFcS74yrxOzCcCR8y0#TZqJ581KLE7&!)#2XB zf>H7oQtdhK=j`RnmM2G+uUsix>3-}TOkqBr(X28L)gJFm{Y|h-LOSbsx59|R?i`_3 z%NwOFLXBCE)k8@&2m3&4O6<_DXVh)b#{6ll$+l zkk#8N95(jxCww!BW((mvrNYMSeYq0C4z0ago@rJ-%Ky%KIGSlF5;hJ_0f6Qj7ehqS zk#r1&f$A{FnN6b8r>?Fp*7)Z2pZK)m6pY+#jSBnkiVhX;qi0nPGL!Tk7a zSch5U{?9m$j0Jf%3|PQAylX<|%&Xbl*-G&APW&&~nop09Qe82ujUd#P8rK-Lc)H8` zX69rnPilGe7Y6y(PrM~a0*PD@*fAMF!MsktGGWh$`lC#FVU@Z@(pxh7(7+BCxfsq%@w%#A=yV22{Mf$?SnCF6!tJ? z_b)lp+?oO<)m@r&)D_1MQ;fr(ByyOYYyQsmO_A*(*@ClWYScz>&v zXdkEKe)AI+M?vT11`ea%d%RT~*dpX1>{J?$1@{Oj(iLL|YYQhUg$4nf{Wb(XQS5Z0fq!-C~$DPSLluIh(&smFTEB*$1 zLxE5cAzV*Yn?^y|?JZ6X#pSdPhNXnGgQZV}rU`lq@y%7Y&!7EKQ2w+rYEhmKPFq%; zx)m1KJqGcyCYVYE8bM%XQ^r=@EG@?t?$Hkb$cxZX&pYtp3fWVCaFA!$zv7M@bdhaSP zwv`HL!moPLJ^Fht`1yus80%e4ULbw>msQ&j^*=s(8CPhk=#Nc&P~F`txSL)LgX)Or zUZ57*kN5bca73p6x-hlBmJP!BZq>x*RDYmezjP%Oc&yF7`qS&Xr#+@Z>-*f=3lkl9 z;T0Wd{SQ?whSyW$oyF-JF_xusu(O`S%uI$Cc+CubSdVP|Yz?R~Ath9NXTDNF+nzyH z5p|PE8-z1jE0E-`802Pg`Ke}H6ay&nKm(?@&TK+AEEEux6!5@W9k3Bh{Mv1DdA6lm za_WeT?77;B%n9iATd{Iyt6maILjr&$pvj~Z#~c(C#8C>Ah`#wK9U$)fyWNaPKJ!_J z57(Qj1FQH8c1?QjR68(iE5$H@5y5YzXM5;Q^NVdq35gQS5FDRi+<;(MeSammf*{Cl zNqzjtZ)E>O6PJ&JI+>?25U8gM25a^qjuD{0q2mI$#ElWl(kuuPTiJfTHNvp*(q}Eg zgB#5_=|q`6thx>U5si03OPBcvJMr47X~!cQ5@hNHuk;Tnqw$(`@4H^!LK3OWkIf3= zoYG$y-JY$PCH=b(MP`0j^dHc_`p<9kxCRb0J#bN;p#npF>0c9(95xr#P*^X7;r#=o@ zyl{n(J{HG0q%(vd*TFsYczO?D8GaLk(ka}w*uLvQbEQ2f&be0+>f_li-Usb=xbm7V z)vcaaaceAj4asnU0wLD7b@oOL_>8X)Hwn)QIY%W_6TS6O1zp$X2WQwgyl!#N?^fUS z`=Qf$G&SP-WH*-@RYCySGz8f%26e7k0QAEduSi%*cp$*0oGQd}*)tMa}gSRFJ)0CsM6j z9MT8<=Y7ZLGc-2-?eI>}JShrrj@;bdQb8M8Z^B;4$;olZVf0XoSu^2(M{#AH2kbF} z0|QFF$HA+^$!r9qyl<}8oR7w{C4FG=^fIgnG)OFdPX7n$DL{Nyq=BDSpcF6MC|<%* zA{jOvT!%6V37w}_X0VnS0aCSe_NU7@EZY4qr#PrDHi4w~y1)ltA6H*rlj0)F0$%s} zL{OooZ5ugx1DcTarRnrt3Cnubox^t~9RySA-SWszVWej1pgg)}ZW`)H1<<26rlN}#<0&}I1~G@n@Ka_0t7*B@)D@Uu z3zNLED0?o>2WEsp74P$N$O)W9e5F?@8mvxcgYAf;ir=d*4mmLq%-`gEYHm{_k}U?& zOL5rjjS~?Oy>?mRJfEIyU#Rt0?vyi@ih|l)Pkm(cNo+gFKB3Z+=V(_2=J$fbAdLcR z>huY^L}PI8vBw%I$Z6uqsaxvqXifyoMl0O)8Y)namDyv>L68*}3x*IzYcE37Wp7c|i!ThyKY8eeE z{@FhY6Ag;Mo4HM!w0x198IcPIw0yAGk0JQzQ%os!OV@OAE!b-@fd4U`L_~9uZlU4A zFws~G@V}~_jH5>5oK^VWoTc!7M(^rtIAoo3*#QMt4si+YtXezfhaVN-f)X=u5ejMy zW}=3B*!tvU<5C`*d2$D26Vanwn@Tgrg&tDnQ+cj+Dkb7yEXcrl7QQhRIqdXvaCcLg z)0u{30YA?w=q}ic?jozfD8=hOsC-UTu%ud>Gx}H5=wGE&rOXDZYC5r1F84G%rVk0wJz+rGvD~typo8PLz}vDb+TgK1Qb`)aJ#rAnV=t?2S%))i9@$;OSA{mNB>TM{_>o=L-9Jb+e&um9ox?4O{U75Ws_q^#V z^qcioqO{*Fo?>*HF3=E!(Cn?x_Ik8xyT>P^Ag4ouAt-oZ6kk{P*P7kdLfwMm67n<$ zAB!3R%T^S<2*=>`SG%$7K2Z*TUNlYsZmy_~TXlH5H_5_jTx*kJJx>7RXeZ`*+fP&k z{ThZ-AedA;?ltgOeJ7k}2inEuQ}5S0D> zIv5mM<}uqX#Bc%W{Y5vr{4DDaS?#t|laGFc$L*(i&G+&?H|xzOi&!NCi60Hvab0th z>a$)P3yWN-3Hqx>$B}cog2uu?x1ot`w#zdnK@gSIw|-BA%!yspC8iGsXB3`wY4Nn| zb5LQ1yOjo+Vy$nF5jD?!WbcPo6ZOAHlZzYjgS4NdOKE2gB!j6gDPZLIF6keQog+1J zQJo%hCDNY?-mHu1DuI6XOvQg7v;;12B`8FEX^mYfJ}(ENoh$GbkR@=wY~Y@$dTAKp zkSD5)SGC4-6-CIjd%id2A;KC0Xc6*F>TzL7rA|k`n$H2rz{m#R<4Ap4aFU7!?{6L* zxm1#g%Z-0usIdyeE$R+=cetoM_r;7QFZ7VF{W=uLpMXku9~A;Ul&{i1A!Y%MWQ3W+ z)ka2|@^>K4pE>`0VdxdLs-40Kz9cp_ir9|4zW(Z%b*p0YHth2WU_Pxbvi|13Hr5gUM|od3(hDsh@+NK84{2Vg|Q4m z6287@(M*JvtC@N=ub=8UEF-S}vQ37)eo@_z;C?wwU6KoVo9l&sSk-xl;TnJ-cCipu zh>#7X3AL9*qd=Zrjji|`~F z3oF-&$CX&c(=$oQ638Vzd?^$OlUL@1DHE=M-j2PxK3sjf%}-jPlheSH!F}b2_4Ou! z4SDhoBvHG~BVQ%$Q!v1Sb4m}uah7OF;E+#yNfW3zi{6|lAbG4PbPy$%+^u)QbN;Hi zA;7#^Xe4kc;xrp88@hV<2o%jb;CkNzJJF#l6rG*X`C@;j&gbvzZXNX~{xuysi4$hn z9p`2xki&P@3S8+#($)iUMFi5 zxTKiKLGs2~Z}V=!b4Iy)rx$Y;x?f-Q+5c*CSL}8Ix(Gb{Gg>)O%`-?$diYfyk6L!0 zR4(_XlpL3h3h=)4)Dt49#QbHFbW&& zBWrMti;K(TlH1;7(KOJ1K>7LtC}0_}!)7IG>bQsyVElz@9<@6HSm(=L^}Y9anfd_- zQ!)!M`tas>fvh`iSOD4lbJrEga-E|0%w4&f$gU8CABs#{WWHle`hVk56DP|1U2QDd z6eV!l_+Hmry;OJoShWu=dE;xyT^^P*{@ATMY!`)&jmJ!vwk6FuUPWvK54-h)`{;LW z`!7snoC^i+EN&E?OWnt;U@}}RfDe!o^|Z@(iD9D*{z37q)mw&EDJbqS>MVME>efuU z__OM3;K1+I28J8mua0;&&F#eL6MynP@Al5>r-*q=t7xnRpRI>%gQfYw0Iq4z{dA0u z<+bt4UA)$XL4U#l1l_$|KnbIxpAU(_n434)PeMVG^ zkF}LOSxff7|m7~#{ZLO0@WQTK`JA+^gVrfGpe^0vlR*2;$IjxYm!gl?7QYD*vM;{IjwN0bGRf(5i_LYcx_(1 z)>2z#lW*rNbEh4FsJgQyd1wsEIw30O+Q#PF+YTr8JnTi@V?cA?)$uCun^B~Ge*Ev= zid5|Uy(2kF{TSko3yFFgZKw0@_*F_ZT#VDGJ;E#*7N=lHX(ikin!u{|9oYS>zY}`C zp(jIamDTaXMB;q^#t)IeX!!HG3jz15bW&yo;M}XwD4>(_)dI|rg=C;%C?Bc~SeY>c zn>NH8gljJS{=Q4N);>fuG-Mnq0Z~XXYlUlTKnPzX$2LZ(K7^*(s zbA;t~?ADr`+-tNt96ok68d6HMG0ZkASeaV1e1-GdbTN<31cjZk96q)&AUNk2SR>L2 z5#6|--t6TerYIYO1R_n4xt3n@yBuQ6p$PH}Gu6h>+3q2bNy&P#gwc!r=iv&^p>XB_ z5daB05hj;eU26_=(~)HR*W7dLlkn|i6_pU!U`#5DoduZYKIL=?A`9B^ zPY`>b5H3P=`+;8zj@l(F-*X2lB#|$HqOG+n1m(#g0z3I#{zPVmh50IW;b85oJw|r< zX*q%YBi7m*Q(2ga!Z8Qdp!FD=>A#cTmB`68c?Wav zi@1mdeKIF=U*z1VU%rScJQ$*%K~XLO?zqWzq&@+gzB~Ut1v5$Akj^OVLPbZL$c2X41H($~>sGwTol$`M2k!YO>Q4kcSw6O+1cjt~)_zuJaBTN$H3H?YgIFSx5U^u}h%GlaHF?rccT=qS^??wfK$?v}G%*y^ygWQ>y=)Qo z0%f9H%VHW2T}$0zsR{?~xLk`rj? z$48+GX182wgcn6NHN<622!vv^9KTDwA6&w49=_U!HYQmF#fXR6LMow*8;|nW5nTtEHWZ*OV4kCBOWlN?DkUyDMn8a*&=q}`1-$V8ANkco`=MU#a&%pJ z`jpk|EcUV)m$48h0{3n4Agvz-Uu&|szmWcWgWqkh;E9`=6~O{rc~a*B0(Zf`Ju{d( zPqMN{g}}iXe#0qcwxFVSz3KRexd(d$)Zfk>;>DK4v$gBxDV$C=B0W7fboTlEiAbvq za!?*z6OS{QbpF0so~{NS=u^|Lci}JSh6v|z@HOO+Li~{vk-ITNtKi>0)Nt5*4IRK) zNs^mFb=^2kqs4%sFc@-XgNk# zZ=rBFlbQl$fB5OETAWjRuwn*tmfT~f6Lzmb2nu>EvF&QrgZQo^opNrk_*4ixy(dPmLT}eHm3>`F^VTvj^tbKoZPkut zO_RL09GwhfRI^-V!0HdY6b1WLcAOweMa?gF6B{iedu1n4P{sD=d(V{PrS$t3()i>r zwT+aua>qGT6|xygaC1EHUQVrRKkW6UrNWdd2^r*PvVS;$Ai>z52@tmIhja)h=s86p zE=-Ky2ZaOi=4FOFG!oV2z2jtrcy33CO73K5Fea`f2G|36j2aHM6f7LXQR9UA%Gy({ zNP_hhGKf7DQ`Tq_q=s8SYnmtATISG<()A2%hLJ3hYDrR9{Wv+y$fP6iAr?X0j{?I% z`ygiBMIB^M;2`7;G5zqvW^ss}st=8dSlDmWKXaf z#!-2hJx7WB7OaH;9T|dIO(x|($KRo1oau#2TD6!qdnD=U_hLSG&kguyKfy?oMr91T z(<`ohpC@gD8`i}&p3+r-XD$heg?{SP1+Qqr-*v`d90q%aTpTXV!3kD@#XxW?u{f>? zA6^qfm%ApH3{2UkQ09-NC!@v}{c_U%P%259I{0jmUjc=SK7 zqxj$xseq_wW(!?jtGxdNAYO!n7RB!?Pv(nB-)SNldvC_+W^t@5`Ai6$ZivnaFi5Fq z2_%!a%e3$DnTT9am(P3=>0)A+g}0+&Qh7QC9T|wcAnr-0$&FojtFW-cO+Chh1+n3t zrjkUMME?jQR$!k`-SEpqnhdo*^g_4*wVe@|AgT)t`a{c5bVf{!037%hM1pMP(mFpv zZwr1Gr^T5-!Wl-+8gv3dp$v;omX^o4gC3QtP#U>TYgA2Np{xi8xa;gM_&k{##lk-}X^r{7`-3n<1d#E)W;bdp`^ zJZX!RMvHn0=7z5X2OGf7%jtfVfHRbe`4eWFuE0m(el&RwoMsZ;T&JH<_uk;+!?QyM z6=_8cDJApxSxLL!Kg>r!r$F=+NBTwD%-$#ed(u%Lrz;>OOWnVd7x#2B0U$?q-3C+7 zmk70*1p*XE!0KKBtW9rheo2X8%|sR6QBNxzVuE}9@hAQKiy;3QCMqQK5yKJ`7B20d zp|09rV6MjUkF7*WygTonK ziZ6b4H82L%xC~FbBRuU4Rj+IAjdrS0-!AI z{?I#m%AVCekfuCXy>o8*8|NhDYL@2~dQ7rZS$QW;jiK10D$w7i3b zxu}G4d30en@MXZ_kA6W?+eS>lgXsk_JjLmZx_mDunZU`7@lt-by_XWac~Sd4%S0wQ zkN!oeWtY|(;lEXH$+yL(SeTx)!mb)Qohn!Y?J%TTmG;oP^5<$)4%0%;$oN!^5lG+d zvSE6;dJ5vTR#;L#t4eJ_zQpL(4!?yTZ+U1<8u^DE%(+J1;|3%v412lebx>=xkUcR9 zz0=Fnsn626<15h`o_5&e1zU1SWle-r&AvrxJJQhPGAEx4NL$Xb!>@i?1U<0GU}GW77|*pe zA4@j`2u1gs0?TTxtz=+>>w;j`7p`1iY8N-kEgZ|Fip>THY`5C%oM~j`*Lwc{>C_3@ zB?OVXw&q~UOdK~13)elHCn7-JeQ;1APM~=oHBL+iVUm4e*&$v$%vv~5{sl3;7Q4e~ z`XlqlOTYV*ZZU4W$5ttuAMJlZU@g;4c{tG>OCphcO`0^4O{fH0wA^{Lz7 zSdRZfv&>TATk+c3_x3~@UB*uhbM_I6{=%@>Eiba!??#PN`_6ey;O};ky4C(UW zvJV#ry7V`7y?sFfrM={HmVCXC$~~uK=r2K=+L#f47-+&=>Z?WZ>&6&-F(WqcO|aWB zB7p*5*ydYH7DYzJ5{9QE@krcLi)oTfGd5WX-3B?;h1Vmoxv<33Ck+U~vEh)VmUsP}6c5hvjF9=^&AplKXPaIGe+D zfuqw_nTeHZW^x&%-PI{VhgaxDW1mvR{ROmr(Pb07RT58 z8!8`Dcc7YEX^ctFBX`@6Z8L2vp^Z=-%i9b{Im-QU)Wv(f>Jg-Z`M(b@rTeG;&E)9l z$SKm+HXYgMI)aJgSZSQ8P(EeSz#B=#<(H34B5l~|IxQQf#iP3nBa)XL4%|2;i9eeE zPAt&}sx($-v*aXW?g0>=$Jiw+Va!)IqAGr55}Xs|4qP5wr(_v>okVHC0~cR}=^ynU zs{o+MkQ+JEHlAMUz)O^MS9B&zF%&@fQh(1Z&4i5tY#4TR+-U855;eq!xx{FbBXcn& zmrP0aO%+HHn9uERDm$NK7=DyinzUe%e}{@{IA$n=WZ&%acOX3>sV*zQok5pF8YH44 z4}R?EQc9E+{keS)(j&3b^-CKqJg|$K^mZOrpsXQz{$(zdQ9hsEs1%}Cw|$f;=K=#S zNEmx8Hn`RPiV#T@i#T~Uzv0)&kpp>GN483?QUf}sLWHkb57){6_%*hY&T;VYB|BZ= zEeX3~_^^I5o!p(lXc2kn`RR|(7Kk^W3t}R@hij@78Eq;7WVN_Fu9g60%9}SQb&TAG z4{_~vJ;JZ@F%QxjEE^nhkL_-q znyuTFz6MhuA+r|x8ShK|gsn)2r~Mnf#=pS`p*PzL&GP#Ph4ynl zj`~b~kPsGv7`UMrrvb!V2>c*^fc<{%wvDma;(;%9*`f|ZRd^lR<_X74EuS*AX�Q zWVFdz4QF>Kr<`u{7#O%zQ^@o-by!_>ZD!BN;!=9|*Q_VKuMMQ_@ma!c@-I`$zYtz@Kd zF19}R96HT(`mW0#-s@Mm{vux*9?prNd=^#a5flDShbqxQa(AeJU0jylU*iAjdh4(# zx3+zlfdPgdN>UiQTNKG*0O@Wd1*DOZlx~o2lX$^#GywDtphelsK6B`d*1jsM9@GrDn{ee9DCnW`i4| zgee%KQ|S4;$CiEfw2kV`Idq2;=^lNUtYmn}psCA&A0%oi3zGtyh$V=_-=DKbYJ;pr zEXy5@H%{$npz!lxi;UVB-`QfVD0%nw@!d20`}&<>5t=2^#0&N>v^G;pvrS@@SuBkWz&ohN+MUqtnybIaya3id`MQP9y zd$}BCs)VOzmg%MC@M-G@#vk<%-?Visi}$}Aa2JMYo3H3;*`0%SN$7R`MlMuc*T31< zvwj^WDbvd;eYb-+NuB4`Lhcv9G&H&Mn|}D!5P#hJ*lDN^0Dr$3(tWt~wc13&$+->> z2Q?DSyr5f;wnt90MUvnBvh?_^T7oX4zy8YMFBqYpU-FeWzp!JJSSpRUvLxi&n@2}b zv-!qnxEb?x@6Vo4PP>&h_=pWYz3>y7vKlS!_{a;1+`Rsp%WOVUMqv>>@wJj8_h=0*4I_KpD3gSW%TbjNmWv?8y#5t}G! zwT#}OgrWyLK~~9jXZEj)v|h|62s^FCv1}^a8j7DW2cj!y;Cfq4Hz|*Ergh0G-;m-( z5L>2w&>k*N-@n(59x_s&Ld!>s=o)+8FTAKsVkfks-9tyK&>?4r}b z%3U!wS0h?t9U#;MIOkRHQqG%Tzs#%mB5SXcb%rOFx}*~Y>&2D(7B^@)C`X4N zmrUw$uwlPgoe#{?yX)cSOt^?oDz&cj4whZ(=?i^)uQ|il(C(^fm{R`ddPl(8Zr! z3q(!26YN^fx9~5cj1~D@Up#PHlQ}#+O_{z0bR-l3?TaEnAw@ob+4=h?*>QRM$x*0^ zSobkKG%OoHp{>VqOSfjv8NGoa#kp*n{ZP*hyiz@lRU~(flhf3yXiXU=9moQJUEGD5 zxJR9eY(A8T?ILf9>BD=u-fSCkhv+514OJ$TsobB*X2LVYyq-zgrpWN%9HP^!mH98K zHM|P`vAFn=pjYqq{Mu{l=-9ZEpEHW^U&@Fh)H0OgnpsjBPqEa4VyAjiZE8FvFRZg{C~=}i;rVsOyD zz#lAVRG4m;oUE1Tmfx3(_gdM!e6BnTIPIqJ>5Sn_PLfjw7W}R^xTK4OvL3{gRl9)r z@yKdlOhK3-05rh(FWhUL;sHC5fbY7aL|Viwqkc^02JiP!P}_aHHx40yWCg63#=DXA zym1eUOTUQA-vcpzYTym4uKj=q(ifBeth(P|W?SPOI^jmLdLrVi(p|vhQ=KqCE#E0S zI@B4#ZFZ}g|D;8R#}X<;oZf-qb<&>}ntpWk8emMpYL_TuFKBd~)(h@Y$MAZ#9i5qg zN-p4Fy9}|oSdQY$Z_S5xqDPOF?2Xuj99>VJlumTSKuFj~3e;ICR3+Vm9B)70MynT= zH$(J<0>IZN``&> zzO3Z(^w$rMem+^Gb0eX>YH;j56cMs~k&oNap`oY&H%DKGD@7|L05|&B@S#suB!`0P@uYG7!2Hk_n6Z z!r{=HAS3wZbQc1dddR$%h-Hmev$Qy!1oq+tbC^P>lD1%t;8}@Y3`w9;$g*(0cSutC zC4c~hF4Sxl4d+imUHqV~C&Qm=#%;Q3O^eY7E%y5(P(|<_x)bEgFG>}Y@!mZ6c}->x zUBIe-L!w014tlm-roF7@@?(?J1D6;5<^8A~5v!h5KNYcl- zZm(FiPJ!Z$b@rv@(R3ZiN$TmqGng+#V;f=qS~mg;C*|;xLD%t~)Ye0-sE_Ydh1uYh zydWYzC19a=vOY@xKIFKmL03pqA?2yQh0ckuXrA5HqdhvTx0xd7&jEy#tndy5>5O@h zx6n`fheKlztH%iJCf(EWDu!2ZXQnZT%ShYu2wI;KvuexkFebcfKPq{Kn^g1BZESq0 z8!YB{)i2H>Gn6005=79T9R6mZ*;7kPqgbE+$3y^k1xdiGyFfEvOjMc#Z!eLGE6NkM zAfA>GuU8GP;;BsMkl`kS3OAu4e<y7Ul z;${g(;yu(L$yctiK3=Z`0G(k+J1<7lWsSxR;{?g5N!z^cNLqHG(M6}y!-?JfkVhcZ zU#?lp8m$sya>>GvUtU;vNOu{02`r=Zp{q&Y(ef)3OFBD;W zw{Cu~U+#kun*|nBFP!rCENy9ePJ$fOP8cSca2L*+)h09Z+n{p|-gMauK0OJ+QTWwF zAZlvri=VA~xfzpy)Y5sO;0%aqA?GIp6BRAqADmZ_RQBhky#b@J?oy3?)F9}#ADIot zvbIT1P9kSs=*TEfP>7W{T<@4*f+C>`s@_3|$g(#SmIZPdIJwENDijfZ{6|4WiRH7Oau_Esc=j89(PhhM zZBUokMBRm(_1R0KyvRIsU4~dZrlw!;*Q1sq5`6Ptg`z>ZWfB;13_4!v76zf)?_+lR zgdtkfdD6D5wcqv)0a?@+KcwykOz9ANtG)!*3W{aIH#B)rME5?Ac|Y#=%n4|N`Ih?$ z^5|EUK$&pGZItPNqvuLS#~aLQvYlducIvxZLxSTpMk-mBfBU(4fxP{71^B4 z=W&BUKz~z8AC~d>=@nFk&_^4qXeh2{-_-Ty>Rj!JY*h@v7JA=apMsqrhHyM;(bv?) zxUTnVwOCfYWNF^Z1JUyxj^V%GV2)63StP4BT7*WsqJ|b=eGJfUzGfV0sZCnKuu@ug*^{*j@>e`E5$M%_UIC4gfwr&O+;>mQw1 z&jGX!Kfi}p^1d&)5#Q;ug|Q6$cbdFTE00d5>n!j*JUnD$fygn}57i(W(2pDR_BbTv z&6Z?YGNaI=C6-W6D|>63Dopv`3Exe>Ji4oWL|dyWfz3q`{A?U4peQiCc)4P?)*S7n z_f$~0^lmA{?@HqoRp{4u z00<1Io*#ycSI-=NX{o&K{Gi*{zxwPS%XIhD9ehv%6@0T;D+@$ue1crtH`M1{%a0s! zB`!4M)jhc@XmcGQ>3pJ+4G86{#LlQ07IGbKG)xZoRU%lh51V4FDoXfhF9WskZ?iZUC6`JGG}~yr7HM zWPERDu;ea9HukwV$lJdBe$p}i(jxO@--Al>y~QR(WXDg5sp!4PGG=Ov-^Hotkh9Q^ zhT@;37_cCVB{9Nv(DDx9!$_6YZccw7jRH9?f1;5OeU_{>}^<*+VCfY8H&JQg6&fp=o zvei`B?2Y5O1_dq$Vo-F} zHx9g9mD@!Eah&%Y95uAUwweR(9Z}IX*d*B~DS*e@^V?%mWfVWLb=m#45o0Rk;VhAy z>JekzGY^NA5Wcjny$pQ`Yg(ngNpL=jJ!-HS)l|<=5~O2cYrb@1F*A4UTZ9$S(E0CEQj}_wdnPj`$&Pkzf-12`aj1yyER# zg?%C?wvm$`@AF||b0anTAZbse0=bkh8sk$Oaz!8jR~*c72JSrab0ho3bS66@Vn5~& z<&H$^cHc*#N_g;U9GoT#LvIbi zRpAYoPRt> zs=UN~dEqH&Akq+eb5<(-&J`=UO6iU$_!f0BS3h`xSM~-h<|B#s!hmjAcp;HSzJd9r zaLcX)gvo2oEt1!KNQ%LxQ!&$UL3OaV9>kUP6hPZ^QCH=f=f-L+$9c+@d@YRB`dk(U z!OuOGbJr+{aAl_v>ZU_2a)tWNRRWx2In;ZZ#ICUBjFE>X!~W z!POZ}f$W|Vi+U#pgAkY6r>a!$-eSHW%vus=1^YDKp%&haPZIl<2cOi#FcV8XyJLQr8+YQ;J(^_jY3Kn8nlUM zr8BCGy4kD-+^8wG`!%Ztj^9>?jafX5HRD1Atg)1QZ970h^v`6jcP7NZTC%?eI%@sl z^~;pde#nin*GDK*&ljOlW7FD^l6)khZ>!;5-;Y0jc6s>zzRT2x@H&fLa)d^ugvqeP zdQCik`ey6UEMTom-H^FQ!sMS2OP|#j%2&{;AnKeq3g9#h7>T?SSV2)svxwWue3&}; z$qGk3JydUfLw3a^D>nIAOUhX=9HFxA{Ch)I-$$PPeyt#s@`?N5lGk_Ubq*~(dlL53 zkm}HzFKx>PEO7VXuG)1L8Lyj*3_W2g#fimT&Qq&f{d;Bu596LtWs3X|5&jYQ!a{w4 zVBI{4=oq>xub}S5FAwF!;NmL{_A|bWOsLu^;!h z3xJCo{^rmxuh#_+KCqOXy-8)C1+!>$$O_d^A*6$=j{r_1vq$N;M#alVni&HO!lCz~ zcQ|5*nbp$*P^jXvhNT4G`QY>Ki0b_GC1ZJ;A;=51?Rm}P;_#XJkhHC2Y1u{H9nbFU z_^DuxIsZ!vq0l*Aud$f{II@?ArD17+1y5|Bb~ zxw)gaRr!R{vq{3QlA6clU$kZI~%e_1H>*#+}Mm3nv$)63QE_>?5P$w>5 z%0v=X0ZKstpU54O`fGM~=D<7t3MBs3wcnjAlZb#w<_uoN=HYEL<*v>aZy8zbv9+iI)X{A$Pd4HYKW+l_Ad z!gFaltASKd{J1XP4|z-iLj=Gt%-0B@=P}=Z7$6l^T|@LES6NdDvkE9+!E{?<-Y~U? zBBV}g(R15VOme_0o1ZFi;=`;@&xb#{!m;OnwM$yf&nfcT%#qxzJm#P{4WF|YdqtaM zGnc%C!`~r~gNcQ~lk7z^_&yPBBrU^t$gb*{c}XkGYl$VOl8%K_>g$t;r`TDDpLiC^^Rfh~84k4%i191@!0&4B&FfM8*)Z1zN>%3pslB<$j_ z3-yUlb;k)jyOKDX#?SolPE6I1)OCBS?Y{VG9MbdM_NE|>DjTm@<0fRDr5D(JXuR(w zpltA~=_0p-@WQF^%cE1Fybi3c*>QT!*?Tj~+{7HTvYnMo0poR20NW~~=7dku+k(LP z=(|rAVV!c}jM}$eSFreeZPYwZyE>dS-t6;f3oH&+FyOyotCT@I?4+y+mJsj z>d{s&whf%DolBcXb3BtRtZ+Jl#SMxE>f)UiD6zIu`#5fp7cnIii|2N^h>w|(}< zIW$P}hL$MVV0S+_mwv7Cng0X;XOz@alm|TBX4?1wIuoKHzhLLb}RD?#U;(%IV z{%2>O#6J9I{1rxg&Cz0;5@gFS2`|=a=2E0%nLOiJLfv>w&r`MRjf5H6Ls=j0!Um`l zqyBNrCi?tZaU|eV}rfjOJ>%NhSBPT{jgh{J_TeRwFA_ zI0{2*Vfgp!xu$`h?N{z-WUj2Q5l9YqcKUMl6gx$7bGxU&g*Fx{$`0Mb$)nZ@Xg~Yj z^2${PCN{aahtiwfNfYEKqpbT}$!`1<$6w|e*xPzM^S~=$&Zt40t;88c5f)^iNnqB` zA|A4%^FT3?OxX(+G;!JYAyHyg6%bX44aYVVff>G1rpo@1ccFT^cMEf)IeN`*g42Q% zk30%X`nww{dnWAdqTEmsM&E2QN-g2}va@W1AuW+pt&F+;X$W(D@kJK8NP!EP2(y$^ z&pSD;t5*h_Apbf45S8yT-qglnMeN7n38 zxqC*Bp!WTqA=}Q!-weqn(c&S!vlWp=>-_L6!KhR@rpEHzm(Mzf53V)x<>t?@p0=z^}(5KnF`o>xPBt2f9~Xd^V{^JX;18#@b-$?m5U4% zkjQlTeNEfpurpCSqS1Uirjkj*t##9%-k(p+)eKDRQ|{s@|1&y2i&wU{Q%R*0iP2Zr zMXEN8Zp&(b{?5P{7HY*L(R58^me`Z|jtu!mu8cS;rhUoxy>-V-i>eJ*Qf3=1Q5j6& zcrOg`^_si3)z(1|@=V>)qeGG)EM8L&vtINjIa>yO_nQ^|t9S6~&YsNxSM zF?v5q!@xqvA4KI!$ByeB@u2I;(|S!!JN$tf!X?4Sq^uff%nwNfc;__¥YzRxL(G+{dxni;`v z$OIP8b4iNz=`d+jmMVT~F$k-ze7~j>#Pro)^B2p0CN<4?x$C89cPop~=j(Dd5s%Hg z&Lx+`Hg3gJ=3##(&28n6a(G2@h=d=Toot)Q&^aXtC~;(wsMho<6j(w^Tz|6_oGzQt zZGN}ZKY8ju3MH>0ciP_s@YRNM^3xyM-C%L&?QWL10Fg){C#7WJ<{C(wS0)zd8SxERvVagg5YYXx6<$55wmwtmB1|>?V3W0r;y%GewK=WpMR@556l} zw=@|f$kk#3UT?myut|JuL6w800h4ot8dN971x%~rrq}S~#x;(2z1}TIC!27H zlZ3qsubRSq)TGaitdE_lw3s6e*DP z!=!GOnx5w0iz%sv+D}G4Dszh2BU9^R%dy&#Zoexl1EtEa#4SAL9gpG>>_JJZ4@|ne zf#0JAsfU0@=1`1lmR=Pnq86I&YB^6!_9R4w*IbHqDo^nKh1&Z={_#pMhq%KsO71`+ z@g30_CA!9>h>QBG>&KmD2(|h>qm@1lgSHsped=R@82$9OJ%3sLCt>}4e%+-YZ)l(n zQ*fkp{O&PNl<7P~&QTxxv=0E^>PDwhbgO(z_lO3*%TjB>!}AOyLO=O#EPVe*K3SCl zo>Y8)iZ*vZ^xriH01QO|GQ)k!cP>xk`buu{K*hGl$+MForQr%Ch7rO|7fG5R_MIk5 z1^+GqaxegN$eiYFhUtD+T>i2*2VO9NSAefRgn)XsNdc2IYrfeZ7h{*o@-qK5(Mwle z_xnBq9YuITSETxmNHD=^QjiU-3mnU@vIbz^{^wQr(@VliY6T~L9O2XCk@jY!L{Z4} zQ)*QQynNsu4^^^H|841^;#~v*u^6z>=JGk!|M`2+6PPQQV73PZ@#Wg0JjYFx{7;fv z`nAJoCu1#7HX06)FB97E=KP!RReJ58d7XTi=i@X{hUgq#8{=DgwK&a@{Ch+Y1km1M z{z8@LN6s$B?-7xxk!L{7s@wXwz)_y7&P&%2D=Ur^VrwM8z>~iG@Xnl+@Qv|cLhamh z9RC`tD-PNZpd1p!aGls(*#!@xI2X2y8GQTT5=v4Ld^qpFZ7A8|-LI6I&- zCHETo;!-{~C=b6ha4g4fvVVb-lTj(;L)Cl?HAe3D)NxY$U48hEVF2IY@pRRu_}gtd zPWa#P_T)ORt8lIx-4Ymz9_#B$(K4Hugi`onxu!}S>+k3Gnf7ljJlxP=^!p?pKJ@n( ze@xBwHIg#Xs;}dn$Y5-{?Lg`c(anm-68$aF|>f0B{qoCIQ+oOp2R2QBT^{1Maj|rD#U34GT_`Fp?peqQVv#B%XQuoZU%s5e<%_}zQAJoVzo zNK3HGa+j9g#eAQO!)}1(6sUKkJFcBsU_VY6jc08-04!K<7d00R@s)k!d+QIx<}t37 zbvr6l$N;~yqz-$){A12f8;i~&gYN+|mf2fg1lq?6NFRcse+s1kG$tg_?yQl*S;hnw zGxHMY`b73A=6PcGk9Ni|FN(y!Xz*6>bqH4?pHz!d7O zm1Cq#HJ`WChqr!%O{Kfe`{0bMKqgHtvuFx69+mPcX7W<{a^7Lzq*?#;e5QGh;gL<( z%;l`UQwj#GE_)sEuifN#P3Px;zs}*eAecv@@)N_~qkT$(GoZr+V7R0Hz^?hkHde@&DWV6hbVSQ1+3>Iv5(Pdcrk4J~y-%>r9}H z)opw!__}7$z>1UF5ptMRE^{ugU1)E*F_3Bd_HnYkfdIo4h1+c+x$9*jA~16-uJblu zLDw1o$WLzS1x*YIyS7A7NRnJWKD`g$%%0>*;};G_B9PRMV41cN;=u94uhmO^>YpO4gWR!a&dMr&b^;jb}(8xz=1O2wjK)KtMTWxo=r$Sq>kB>TM2CU%S1}LF>uj z2MUYG5ml$5HN1w7mS6&>>og>%P5I%B|2;E}!MSvP1G%m%F(Yx!J%UnPn`>Tkyzc$@ ztTdKP)Row$>Mbx*wxewJ16&8!SCrnbwlY8Re@XiSw->xfZMsdS$!2h0b@t^0nUO8% z|2Y=_e%lTZF!zBr;S-l{#Ik_I9u9Z}DD_jWq%-6v$lzBp`6hF$s^kP2Tff+_4xe|V zb~7XBU*L?5y3Awn9Z5Vnx3~YQfRq+Vl=6QLA#9L3J@g$N{9aQ4&AT!HqsYsDmEaZf z+W3orz5eiH2_cG5v|%Jh-`5TT6A>*%;+tQ{CesF=n<&GITDo|!KN6jzAV$>WdGt~X zm>fO+p74KqaSqsE4L$d;epaCxb7BU<+qa6OdQr-)udZ1G?(~oWtTcWL9Q9o&rO*A| z;jOGtjNz=3k2T!qra|3S+T$Yaz=dLmBF?EZjW*J!hz!>zPBCuvHRV?lLXWXhhkP?g zk^c9I+HhmwhN=euhs#nz%JT?q9!+X?k6p-6N^)eqN2tVd!ZT4=O3s2z>p?p;HZ~2- zK?bQH4malm=&neV&7#cQP!Wp2Kx5r2RQNuki+aIukiavB50oFf#_%tHzl02Iyr`Mw z07!sGO2rmIrua;a&|phCG`Dl|Es@~hS##DX{>b960$8fAfApmIQr|1a6dNGc7o7!p zTlEbCFcuu3*ZIG}l0F!8-K#2QD+MtTlcA|-Mr@|KwuRS3%VsUSQl?nAy3Bk`8oV$5 zm>4Yx!xzC6eBtUbdjJKVa%O18tQ9pz9PYU%xX`bk*SB>u<#lCz)HGG`GK&wiM!v{I z_3!_{m;j_eX;RqTqma22B!rdB=Nn({%a*0%V2MHfZXjRz8O@%~?q)dbh-&TqG*T`n zwU69Q&It+ab0xV;${*~VxQ;()8TjuFenqE?{Lvg>WAV9KiP4Y!mg<1)-){0NZ8jvQ zCvCak4iXo+@RX}GUC&ywqCup3$0v|dAD@Ty%&8Uel0PI_^mME_92Z&nU%s< z?=rmy=|1SlH0HK_b!lB^6rNPjjdtN-qG=5E(x10EMHor_=(tzLjNTj4t<<_m6x}7i zOh7oDw*?m~Uq8s((vWBL`)`NiA0CPnR3rg^bIo3Y*0-jpMl-yN44OlM>A<&UP)578 z83d6dUeyvGVmWpQZ!d|L@=5 w1$P42$O=f0KNYM*|F8G`dlY~=F9yEBBg|ZEVhc3iLjt~JrIaKq5yk=k4}u1$CjbBd literal 0 HcmV?d00001 From d9ddad45f4a199787b54cb940cb630dbab2e458f Mon Sep 17 00:00:00 2001 From: Isaac Hellendag <2823852+hellendag@users.noreply.github.com> Date: Thu, 19 Dec 2024 15:11:01 -0600 Subject: [PATCH 30/37] [ui] Move stepStats out of RunRootQuery (#26610) ## Summary & Motivation Move `stepStats` out of the root Run query, since it can be expensive. Instead, query for it at the `RunMetadataProvider`. This should allow the header, Gantt chart, and logs to load even while the step stats panel of the page continues to load. ## How I Tested These Changes View a very long-running run. Verify that the Run header, Gantt chart, and logs appear fairly quickly, and the right panel takes a bit longer to load. ## Changelog [ui] Improve performance of Run page for very long-running runs. --- .../dagster-ui/packages/ui-core/client.json | 5 +- .../ui-core/src/runs/RunFragments.tsx | 14 ----- .../ui-core/src/runs/RunMetadataProvider.tsx | 60 +++++++++++++++++-- .../types/RunActionButtonsTestQuery.types.ts | 19 +----- .../src/runs/types/RunFragments.types.ts | 18 ------ .../runs/types/RunMetadataProvider.types.ts | 48 +++++++++++++++ .../ui-core/src/runs/types/RunRoot.types.ts | 19 +----- 7 files changed, 108 insertions(+), 75 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/client.json b/js_modules/dagster-ui/packages/ui-core/client.json index 9b8c430dd2ead..8e9e8ae0c4b40 100644 --- a/js_modules/dagster-ui/packages/ui-core/client.json +++ b/js_modules/dagster-ui/packages/ui-core/client.json @@ -123,7 +123,8 @@ "RunAssetChecksQuery": "6946372fc625c6aba249a54be1943c0858c8efbd5e6f5c64c55723494dc199e4", "RunAssetsQuery": "53c1e7814d451dfd58fb2427dcb326a1e9628c8bbc91b3b9c76f8d6c7b75e278", "RunTabsCountQuery": "5fe1760a3bf0494fb98e3c09f31add5138f9f31d59507a8b25186e2103bebbb4", - "RunRootQuery": "1aa4561b33c2cfb079d7a3ff284096fc3208a46dee748a24c7af827a2cb22919", + "RunStepStatsQuery": "77d73353a4aea095bfa241903122abf14eb38341c5869a9688b70c0d53f5a167", + "RunRootQuery": "4f2633b31ddc71c08d3a985be30dc1bf21fbc462287554f165060c51a3554beb", "RunStatsQuery": "75e80f740a79607de9e1152f9b7074d319197fbc219784c767c1abd5553e9a49", "LaunchPipelineExecution": "292088c4a697aca6be1d3bbc0cfc45d8a13cdb2e75cfedc64b68c6245ea34f89", "LaunchMultipleRuns": "a56d9efdb35e71e0fd1744dd768129248943bc5b23e717458b82c46829661763", @@ -135,7 +136,7 @@ "RunTagValuesQuery": "0c0a9998c215bb801eb0adcd5449c0ac4cf1e8efbc6d0fcc5fb6d76fcc95cb92", "ScheduledRunsListQuery": "2650d8ebdfc444fe76fcf8acd9ff54f9ecacdb680b1d83e3f487cb71dd0c7eae", "TerminateRunIdsQuery": "d38573af47f3ab2f2b11d90cb85ce8426307e2384e67a5b20e2bf67d5c1054bb", - "RunActionButtonsTestQuery": "d85a7e0201a27eb36be5a7471d2724fe5a68b7257e6635f54f120fc40f8169c0", + "RunActionButtonsTestQuery": "5d358c3360e683549b885108c3dbb7c1d21d8afd790a5ee963e6e9640ccdbfe8", "RunsRootQuery": "091646e47ecea81ba4765a3f2cead18880b09ee400d1d7e9dcb6e194ee364e51", "RunsFeedRootQuery": "ef8eb6ca144d661c6bcd409ed878551851f15dd1c0aa8c03ee9c68c1c4c301d1", "OngoingRunTimelineQuery": "055420e85ba799b294bab52c01d3f4a4470580606a40483031c35777d88d527f", diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunFragments.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunFragments.tsx index c0ebb65c183fd..b34d76e07dab3 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunFragments.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunFragments.tsx @@ -46,20 +46,6 @@ export const RUN_FRAGMENT = gql` } stepKeysToExecute updateTime - stepStats { - stepKey - status - startTime - endTime - attempts { - startTime - endTime - } - markers { - startTime - endTime - } - } ...RunTimingFragment } diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunMetadataProvider.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunMetadataProvider.tsx index c99bfc4106a6f..a3d495d021e7e 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunMetadataProvider.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunMetadataProvider.tsx @@ -1,11 +1,17 @@ import * as React from 'react'; +import {useMemo} from 'react'; import {LogsProviderLogs} from './LogsProvider'; import {RunContext} from './RunContext'; -import {gql} from '../apollo-client'; +import {gql, useQuery} from '../apollo-client'; import {flattenOneLevel} from '../util/flattenOneLevel'; import {RunFragment} from './types/RunFragments.types'; -import {RunMetadataProviderMessageFragment} from './types/RunMetadataProvider.types'; +import { + RunMetadataProviderMessageFragment, + RunStepStatsFragment, + RunStepStatsQuery, + RunStepStatsQueryVariables, +} from './types/RunMetadataProvider.types'; import {StepEventStatus} from '../graphql/types'; import {METADATA_ENTRY_FRAGMENT} from '../metadata/MetadataEntryFragment'; @@ -103,13 +109,17 @@ export const extractLogCaptureStepsFromLegacySteps = (stepKeys: string[]) => { const fromTimestamp = (ts: number | null) => (ts ? Math.floor(ts * 1000) : undefined); -function extractMetadataFromRun(run?: RunFragment): IRunMetadataDict { +function extractMetadataFromRun( + run: RunFragment | null = null, + stepStats: RunStepStatsFragment['stepStats'] = [], +): IRunMetadataDict { const metadata: IRunMetadataDict = { firstLogAt: 0, mostRecentLogAt: 0, globalMarkers: [], steps: {}, }; + if (!run) { return metadata; } @@ -120,7 +130,7 @@ function extractMetadataFromRun(run?: RunFragment): IRunMetadataDict { metadata.exitedAt = fromTimestamp(run.endTime); } - run.stepStats.forEach((stepStat) => { + stepStats.forEach((stepStat) => { metadata.steps[stepStat.stepKey] = { // state: // current state @@ -370,7 +380,18 @@ interface IRunMetadataProviderProps { export const RunMetadataProvider = ({logs, children}: IRunMetadataProviderProps) => { const run = React.useContext(RunContext); - const runMetadata = React.useMemo(() => extractMetadataFromRun(run), [run]); + + // Step stats can be expensive to load, so we separate them from the main run query. + const {data} = useQuery(RUN_STEP_STATS_QUERY, { + variables: run ? {runId: run.id} : undefined, + skip: !run, + }); + + const stepStats = useMemo(() => { + return data?.pipelineRunOrError.__typename === 'Run' ? data.pipelineRunOrError.stepStats : []; + }, [data]); + + const runMetadata = React.useMemo(() => extractMetadataFromRun(run, stepStats), [run, stepStats]); const metadata = React.useMemo( () => logs.loading ? runMetadata : extractMetadataFromLogs(flattenOneLevel(logs.allNodeChunks)), @@ -379,6 +400,35 @@ export const RunMetadataProvider = ({logs, children}: IRunMetadataProviderProps) return <>{children(metadata)}; }; +const RUN_STEP_STATS_QUERY = gql` + query RunStepStatsQuery($runId: ID!) { + pipelineRunOrError(runId: $runId) { + ... on Run { + id + ...RunStepStatsFragment + } + } + } + + fragment RunStepStatsFragment on Run { + id + stepStats { + stepKey + status + startTime + endTime + attempts { + startTime + endTime + } + markers { + startTime + endTime + } + } + } +`; + export const RUN_METADATA_PROVIDER_MESSAGE_FRAGMENT = gql` fragment RunMetadataProviderMessageFragment on DagsterRunEvent { ... on MessageEvent { diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/__tests__/types/RunActionButtonsTestQuery.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/__tests__/types/RunActionButtonsTestQuery.types.ts index ea344cf423ea4..cc5a89e67c1fb 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/__tests__/types/RunActionButtonsTestQuery.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/__tests__/types/RunActionButtonsTestQuery.types.ts @@ -56,25 +56,8 @@ export type RunActionButtonsTestQuery = { }>; }>; } | null; - stepStats: Array<{ - __typename: 'RunStepStats'; - stepKey: string; - status: Types.StepEventStatus | null; - startTime: number | null; - endTime: number | null; - attempts: Array<{ - __typename: 'RunMarker'; - startTime: number | null; - endTime: number | null; - }>; - markers: Array<{ - __typename: 'RunMarker'; - startTime: number | null; - endTime: number | null; - }>; - }>; } | {__typename: 'RunNotFoundError'}; }; -export const RunActionButtonsTestQueryVersion = 'd85a7e0201a27eb36be5a7471d2724fe5a68b7257e6635f54f120fc40f8169c0'; +export const RunActionButtonsTestQueryVersion = '5d358c3360e683549b885108c3dbb7c1d21d8afd790a5ee963e6e9640ccdbfe8'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunFragments.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunFragments.types.ts index a50b77c1b0de8..e75aab9ccde18 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunFragments.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunFragments.types.ts @@ -49,15 +49,6 @@ export type RunFragment = { }>; }>; } | null; - stepStats: Array<{ - __typename: 'RunStepStats'; - stepKey: string; - status: Types.StepEventStatus | null; - startTime: number | null; - endTime: number | null; - attempts: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; - markers: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; - }>; }; export type RunDagsterRunEventFragment_AlertFailureEvent = { @@ -3186,13 +3177,4 @@ export type RunPageFragment = { }>; }>; } | null; - stepStats: Array<{ - __typename: 'RunStepStats'; - stepKey: string; - status: Types.StepEventStatus | null; - startTime: number | null; - endTime: number | null; - attempts: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; - markers: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; - }>; }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunMetadataProvider.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunMetadataProvider.types.ts index db950cff62923..9c7d0454624ba 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunMetadataProvider.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunMetadataProvider.types.ts @@ -2,6 +2,52 @@ import * as Types from '../../graphql/types'; +export type RunStepStatsQueryVariables = Types.Exact<{ + runId: Types.Scalars['ID']['input']; +}>; + +export type RunStepStatsQuery = { + __typename: 'Query'; + pipelineRunOrError: + | {__typename: 'PythonError'} + | { + __typename: 'Run'; + id: string; + stepStats: Array<{ + __typename: 'RunStepStats'; + stepKey: string; + status: Types.StepEventStatus | null; + startTime: number | null; + endTime: number | null; + attempts: Array<{ + __typename: 'RunMarker'; + startTime: number | null; + endTime: number | null; + }>; + markers: Array<{ + __typename: 'RunMarker'; + startTime: number | null; + endTime: number | null; + }>; + }>; + } + | {__typename: 'RunNotFoundError'}; +}; + +export type RunStepStatsFragment = { + __typename: 'Run'; + id: string; + stepStats: Array<{ + __typename: 'RunStepStats'; + stepKey: string; + status: Types.StepEventStatus | null; + startTime: number | null; + endTime: number | null; + attempts: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; + markers: Array<{__typename: 'RunMarker'; startTime: number | null; endTime: number | null}>; + }>; +}; + export type RunMetadataProviderMessageFragment_AlertFailureEvent = { __typename: 'AlertFailureEvent'; message: string; @@ -488,3 +534,5 @@ export type RunMetadataProviderMessageFragment = | RunMetadataProviderMessageFragment_StepExpectationResultEvent | RunMetadataProviderMessageFragment_StepWorkerStartedEvent | RunMetadataProviderMessageFragment_StepWorkerStartingEvent; + +export const RunStepStatsQueryVersion = '77d73353a4aea095bfa241903122abf14eb38341c5869a9688b70c0d53f5a167'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunRoot.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunRoot.types.ts index d32da61f71afd..8c196998cc90a 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunRoot.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunRoot.types.ts @@ -58,25 +58,8 @@ export type RunRootQuery = { }>; }>; } | null; - stepStats: Array<{ - __typename: 'RunStepStats'; - stepKey: string; - status: Types.StepEventStatus | null; - startTime: number | null; - endTime: number | null; - attempts: Array<{ - __typename: 'RunMarker'; - startTime: number | null; - endTime: number | null; - }>; - markers: Array<{ - __typename: 'RunMarker'; - startTime: number | null; - endTime: number | null; - }>; - }>; } | {__typename: 'RunNotFoundError'}; }; -export const RunRootQueryVersion = '1aa4561b33c2cfb079d7a3ff284096fc3208a46dee748a24c7af827a2cb22919'; +export const RunRootQueryVersion = '4f2633b31ddc71c08d3a985be30dc1bf21fbc462287554f165060c51a3554beb'; From 40807bf072d534c270a593381c7e2bf1c9d3d31c Mon Sep 17 00:00:00 2001 From: Sean Mackesey Date: Thu, 19 Dec 2024 16:16:28 -0500 Subject: [PATCH 31/37] Roll back deprecation of multi-asset sensor (#26390) Resolves #26223 ## Summary & Motivation `@multi_asset_sensor` was deprecated with a message saying to use `AutomationCondition`, but `AutomationCondition` cannot cover all of `@multi_asset_sensor` uses cases (like running jobs), as discussed in #26223. ## Changelog Deprecation of `@multi_asset_sensor` has been rolled back. --- .../dagster/_core/definitions/decorators/sensor_decorator.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/sensor_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/sensor_decorator.py index e571e075c4193..9c98321c1c9bb 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/sensor_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/sensor_decorator.py @@ -4,7 +4,6 @@ from typing import TYPE_CHECKING, Any, Callable, Mapping, Optional, Sequence, Set, Union import dagster._check as check -from dagster._annotations import deprecated from dagster._core.definitions.asset_selection import AssetSelection, CoercibleToAssetSelection from dagster._core.definitions.asset_sensor_definition import AssetSensorDefinition from dagster._core.definitions.events import AssetKey @@ -248,7 +247,6 @@ def _wrapped_fn(*args, **kwargs) -> Any: return inner -@deprecated(breaking_version="2.0.0", additional_warn_text="use `AutomationConditions` instead") def multi_asset_sensor( monitored_assets: Union[Sequence[AssetKey], AssetSelection], *, From c9233e2228034f32577b9edb1a683c5dbb99c068 Mon Sep 17 00:00:00 2001 From: David Liu <48995019+dliu27@users.noreply.github.com> Date: Thu, 19 Dec 2024 16:26:31 -0500 Subject: [PATCH 32/37] [ui] use jobName as fallback if only launching 1 job (#26597) ## Summary & Motivation https://dagsterlabs.slack.com/archives/C06QU0WBCQ5/p1734625511029539 ## How I Tested These Changes Will test in dogfood replicating this situation --- .../useLaunchMultipleRunsWithTelemetry.ts | 15 +- .../ui-core/src/runs/RunConfigDialog.tsx | 20 ++- .../ui-core/src/ticks/DryRunRequestTable.tsx | 1 + .../src/ticks/EvaluateScheduleDialog.tsx | 11 +- .../ui-core/src/ticks/SensorDryRunDialog.tsx | 3 +- .../EvaluateScheduleDialog.fixtures.tsx | 138 ++++++++++++++++++ .../SensorDryRunDialog.fixtures.tsx | 104 +++++++++++++ .../__tests__/EvaluateScheduleDialog.test.tsx | 47 ++++++ .../__tests__/SensorDryRunDialog.test.tsx | 48 +++++- 9 files changed, 377 insertions(+), 10 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts index 9249da1025e91..77b1891f512d6 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts @@ -24,15 +24,26 @@ export function useLaunchMultipleRunsWithTelemetry() { const history = useHistory(); return useCallback( - async (variables: LaunchMultipleRunsMutationVariables, behavior: LaunchBehavior) => { + async ( + variables: LaunchMultipleRunsMutationVariables, + behavior: LaunchBehavior, + jobName: string, + ) => { try { const executionParamsList = Array.isArray(variables.executionParamsList) ? variables.executionParamsList : [variables.executionParamsList]; - const jobNames = executionParamsList.map( + + let jobNames = executionParamsList.map( (params) => params.selector.jobName || params.selector.pipelineName, ); + // if only executing one job, and jobName isn't defined, fallback to jobName from sensor/schedule + if (executionParamsList.length === 1 && !executionParamsList[0]?.selector?.jobName) { + jobNames = [jobName]; + executionParamsList[0]!.selector.jobName = jobName; + } + if ( jobNames.length !== executionParamsList.length || jobNames.includes(undefined) || diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx index 7dad84a6c971d..713a6650a95f2 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx @@ -17,7 +17,7 @@ interface Props { runConfigYaml: string; mode: string | null; isJob: boolean; - + jobName?: string; // Optionally provide tags to display them as well. tags?: RunTagsFragment[]; @@ -27,8 +27,18 @@ interface Props { } export const RunConfigDialog = (props: Props) => { - const {isOpen, onClose, copyConfig, runConfigYaml, tags, mode, isJob, request, repoAddress} = - props; + const { + isOpen, + onClose, + copyConfig, + runConfigYaml, + tags, + mode, + isJob, + jobName, + request, + repoAddress, + } = props; const hasTags = !!tags && tags.length > 0; return ( @@ -76,10 +86,12 @@ export const RunConfigDialog = (props: Props) => { topBorder left={ request && - repoAddress && ( + repoAddress && + jobName && ( diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx index b9859d06be68a..ca7e773b9c6e1 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx @@ -71,6 +71,7 @@ export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName} runConfigYaml={selectedRequest.runConfigYaml} tags={selectedRequest.tags} isJob={isJob} + jobName={jobName} request={selectedRequest} repoAddress={repoAddress} /> diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx index 7262979f9473a..4cb739a3d7447 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx @@ -190,7 +190,7 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { try { if (executionParamsList) { - await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast', jobName); } } catch (e) { console.error(e); @@ -198,7 +198,14 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { setLaunching(false); onClose(); - }, [canLaunchAll, executionParamsList, launchMultipleRunsWithTelemetry, onClose, trackEvent]); + }, [ + canLaunchAll, + executionParamsList, + jobName, + launchMultipleRunsWithTelemetry, + onClose, + trackEvent, + ]); const content = useMemo(() => { // launching all runs state diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index 1047d13d9a391..83cad592e15a7 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -196,7 +196,7 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop try { if (executionParamsList) { - await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast', jobName); onCommitTickResult(); // persist tick } } catch (e) { @@ -208,6 +208,7 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop }, [ canLaunchAll, executionParamsList, + jobName, launchMultipleRunsWithTelemetry, onClose, onCommitTickResult, diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx index 724b62da00125..da76071aa8ea9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/EvaluateScheduleDialog.fixtures.tsx @@ -79,6 +79,44 @@ export const scheduleDryWithWithRunRequest = { }), }; +export const scheduleDryWithWithRunRequestUndefinedName = { + __typename: 'Mutation' as const, + scheduleDryRun: buildDryRunInstigationTick({ + timestamp: 1674950400, + evaluationResult: buildTickEvaluation({ + runRequests: [ + buildRunRequest({ + jobName: undefined, + runConfigYaml: + 'ops:\n configurable_op:\n config:\n scheduled_date: 2023-01-29\n', + tags: [ + buildPipelineTag({ + key: 'dagster/schedule_name', + value: 'configurable_job_schedule', + }), + buildPipelineTag({ + key: 'date', + value: '2023-01-29', + __typename: 'PipelineTag' as const, + }), + buildPipelineTag({ + key: 'github_test', + value: 'test', + }), + buildPipelineTag({ + key: 'okay_t2', + value: 'okay', + }), + ], + runKey: 'EvaluateScheduleDialog.test.tsx:1675705668.993122345', + }), + ], + skipReason: null, + error: null, + }), + }), +}; + export const ScheduleDryRunMutationRunRequests: MockedResponse = { request: { query: SCHEDULE_DRY_RUN_MUTATION, @@ -94,6 +132,22 @@ export const ScheduleDryRunMutationRunRequests: MockedResponse = + { + request: { + query: SCHEDULE_DRY_RUN_MUTATION, + variables: { + selectorData: { + scheduleName: 'test', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + }, + timestamp: 5, + }, + }, + result: {data: scheduleDryWithWithRunRequestUndefinedName}, + }; + export const ScheduleDryRunMutationError: MockedResponse = { request: { query: SCHEDULE_DRY_RUN_MUTATION, @@ -253,3 +307,87 @@ export const ScheduleLaunchAllMutation: MockedResponse = + { + request: { + query: LAUNCH_MULTIPLE_RUNS_MUTATION, + variables: { + executionParamsList: [ + { + runConfigData: + 'ops:\n configurable_op:\n config:\n scheduled_date: 2023-01-29', + selector: { + jobName: 'testJobName', // fallback + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster/schedule_name', + value: 'configurable_job_schedule', + }, + { + key: 'date', + value: '2023-01-29', + }, + { + key: 'github_test', + value: 'test', + }, + { + key: 'okay_t2', + value: 'okay', + }, + ], + }, + }, + ], + }, + }, + result: { + data: { + __typename: 'Mutation', + launchMultipleRuns: buildLaunchMultipleRunsResult({ + launchMultipleRunsResult: [ + buildLaunchRunSuccess({ + run: buildRun({ + id: '504b3a77-d6c4-440c-a128-7f59c9d75d59', + pipeline: buildPipelineSnapshot({ + name: 'testJobName', // fallback + }), + tags: [ + buildPipelineTag({ + key: 'dagster/schedule_name', + value: 'configurable_job_schedule', + }), + buildPipelineTag({ + key: 'date', + value: '2023-01-29', + }), + buildPipelineTag({ + key: 'github_test', + value: 'test', + }), + buildPipelineTag({ + key: 'okay_t2', + value: 'okay', + }), + ], + status: RunStatus.QUEUED, + runConfigYaml: + 'ops:\n configurable_op:\n config:\n scheduled_date: 2023-01-29', + mode: 'default', + resolvedOpSelection: null, + }), + }), + ], + }), + }, + }, + }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx index 98e691e5086a9..68d62e5701303 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__fixtures__/SensorDryRunDialog.fixtures.tsx @@ -55,6 +55,19 @@ export const runRequests: RunRequest[] = [ }), ]; +export const runRequestWithUndefinedJobName: RunRequest[] = [ + buildRunRequest({ + jobName: undefined, // undefined jobName + runKey: 'DryRunRequestTable.test.tsx:1675705668.9931223', + runConfigYaml: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx\n', + tags: [ + buildPipelineTag({key: 'dagster2', value: 'test'}), + buildPipelineTag({key: 'marco2', value: 'salazar2'}), + ], + }), +]; + export const SensorDryRunMutationRunRequests: MockedResponse = { request: { query: EVALUATE_SENSOR_MUTATION, @@ -81,6 +94,33 @@ export const SensorDryRunMutationRunRequests: MockedResponse = + { + request: { + query: EVALUATE_SENSOR_MUTATION, + variables: { + selectorData: { + sensorName: 'test', + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + }, + cursor: 'testCursortesting123', + }, + }, + result: { + data: { + __typename: 'Mutation', + sensorDryRun: buildDryRunInstigationTick({ + evaluationResult: buildTickEvaluation({ + cursor: 'a new cursor', + runRequests: runRequestWithUndefinedJobName, + error: null, + }), + }), + }, + }, + }; + export const SensorDryRunMutationError: MockedResponse = { request: { query: EVALUATE_SENSOR_MUTATION, @@ -335,3 +375,67 @@ export const SensorLaunchAllMutation: MockedResponse }, }, }; + +export const SensorLaunchAllMutation1JobWithUndefinedJobName: MockedResponse = + { + request: { + query: LAUNCH_MULTIPLE_RUNS_MUTATION, + variables: { + executionParamsList: [ + { + runConfigData: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx', + selector: { + jobName: 'testJobName', // fallback + repositoryLocationName: 'testLocation', + repositoryName: 'testName', + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [ + { + key: 'dagster2', + value: 'test', + }, + { + key: 'marco2', + value: 'salazar2', + }, + ], + }, + }, + ], + }, + }, + result: { + data: { + __typename: 'Mutation', + launchMultipleRuns: buildLaunchMultipleRunsResult({ + launchMultipleRunsResult: [ + buildLaunchRunSuccess({ + __typename: 'LaunchRunSuccess', + run: buildRun({ + __typename: 'Run', + id: '504b3a77-d6c4-440c-a128-7f59c9d75d59', + pipeline: buildPipelineSnapshot({ + name: 'testJobName', + }), + tags: [ + buildPipelineTag({key: 'dagster2', value: 'test'}), + buildPipelineTag({key: 'marco2', value: 'salazar2'}), + ], + status: RunStatus.QUEUED, + runConfigYaml: + 'solids:\n read_file:\n config:\n directory: /Users/marcosalazar/code/dagster/js_modules/dagster-ui/packages/ui-core/src/ticks/tests\n filename: DryRunRequestTable.test.tsx\n', + mode: 'default', + resolvedOpSelection: null, + }), + }), + ], + }), + }, + }, + }; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx index a8a2b55c06eb5..10eb96d2d2f3d 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx @@ -10,8 +10,10 @@ import { GetScheduleQueryMock, ScheduleDryRunMutationError, ScheduleDryRunMutationRunRequests, + ScheduleDryRunMutationRunRequestsWithUndefinedName, ScheduleDryRunMutationSkipped, ScheduleLaunchAllMutation, + ScheduleLaunchAllMutationWithUndefinedName, } from '../__fixtures__/EvaluateScheduleDialog.fixtures'; // This component is unit tested separately so mocking it out @@ -160,4 +162,49 @@ describe('EvaluateScheduleTest', () => { expect(pushSpy).toHaveBeenCalled(); }); }); + + it('launches all runs for 1 runrequest with undefined job name in the runrequest', async () => { + const pushSpy = jest.fn(); + const createHrefSpy = jest.fn(); + + (useHistory as jest.Mock).mockReturnValue({ + push: pushSpy, + createHref: createHrefSpy, + }); + + (useTrackEvent as jest.Mock).mockReturnValue(jest.fn()); + + render( + + + , + ); + const selectButton = await screen.findByTestId('tick-selection'); + await userEvent.click(selectButton); + await waitFor(() => { + expect(screen.getByTestId('tick-5')).toBeVisible(); + }); + await userEvent.click(screen.getByTestId('tick-5')); + await userEvent.click(screen.getByTestId('continue')); + await waitFor(() => { + expect(screen.getByText(/1\s+run request/i)).toBeVisible(); + expect(screen.getByTestId('launch-all')).not.toBeDisabled(); + }); + + userEvent.click(screen.getByTestId('launch-all')); + + await waitFor(() => { + expect(screen.getByText(/Launching runs/i)).toBeVisible(); + }); + + await waitFor(() => { + expect(pushSpy).toHaveBeenCalled(); + }); + }); }); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx index efb95aa30ed1e..b217a38741e75 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx @@ -95,7 +95,7 @@ describe('SensorDryRunTest', () => { expect(screen.getByTestId('cursor-input')).toBeVisible(); }); - it('launches all runs', async () => { + it('launches all runs with well defined job names', async () => { const pushSpy = jest.fn(); const createHrefSpy = jest.fn(); @@ -140,4 +140,50 @@ describe('SensorDryRunTest', () => { expect(pushSpy).toHaveBeenCalled(); }); }); + + it('launches all runs for 1 runrequest with undefined job name in the runrequest', async () => { + const pushSpy = jest.fn(); + const createHrefSpy = jest.fn(); + + (useHistory as jest.Mock).mockReturnValue({ + push: pushSpy, + createHref: createHrefSpy, + }); + + (useTrackEvent as jest.Mock).mockReturnValue(jest.fn()); + + render( + + + , + ); + const cursorInput = await screen.findByTestId('cursor-input'); + await userEvent.type(cursorInput, 'testing123'); + await userEvent.click(screen.getByTestId('continue')); + await waitFor(() => { + expect(screen.getByText(/1\srun requests/g)).toBeVisible(); + expect(screen.queryByText('Skipped')).toBe(null); + expect(screen.queryByText('Failed')).toBe(null); + }); + + await waitFor(() => { + expect(screen.getByTestId('launch-all')).not.toBeDisabled(); + }); + + userEvent.click(screen.getByTestId('launch-all')); + + await waitFor(() => { + expect(screen.getByText(/Launching runs/i)).toBeVisible(); + }); + + await waitFor(() => { + expect(pushSpy).toHaveBeenCalled(); + }); + }); }); From 234e10c81f7f8a860631b0df3ab48a03a0b8233b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 19 Dec 2024 13:33:09 -0800 Subject: [PATCH 33/37] allow different `PartitionsDefinition`s within an `AssetsDefinition` (#23824) ## Summary & Motivation This makes it possible for assets within a single `AssetsDefinition` to have different `PartitionsDefinitions`. I.e. the example below is now allowed: ```python from dagster import AssetSpec, MaterializeResult, StaticPartitionsDefinition, multi_asset partitions_def1 = StaticPartitionsDefinition(["a", "b", "c", "d"]) partitions_def2 = StaticPartitionsDefinition(["1", "2", "3"]) @multi_asset( specs=[ AssetSpec("my_asset_1", partitions_def=partitions_def1), AssetSpec("my_asset_2", partitions_def=partitions_def2), ], can_subset=True, ) def my_assets(context): for asset_key in context.selected_asset_keys: yield MaterializeResult(asset_key=asset_key) ``` For this to be allowed, `can_subset` must be true. This means that we don't need to change anything about the underlying execution. If someone wants to launch an execution that targets two assets with different `PartitionsDefinition`s within the same `AssetsDefinition`, then the assets are just executed in separate runs. This will help address a common complaint we've received with respect to our dbt integration, which is that you need to do a separate `@dbt_assets(select=...)` per `PartitionsDefinition`. ## How I Tested These Changes - Added unit tests. - `dagster dev` with the example above, launched runs, clicked around the launchpad, looked at the asset details page. ## Changelog `AssetSpecs` may now contain a `partitions_def`. Different `AssetSpecs` passed to the same invocation of `@multi_asset` can now have different `PartitionsDefinitions`, as long as `can_subset=True`. --- .../dagster/_core/definitions/asset_graph.py | 4 +- .../dagster/_core/definitions/asset_out.py | 8 +- .../dagster/_core/definitions/assets.py | 144 ++++++------------ .../decorator_assets_definition_builder.py | 60 +++++++- .../_core/definitions/external_asset.py | 2 +- .../multi_asset_sensor_definition.py | 19 ++- .../repository_data_builder.py | 11 +- .../execution/context/op_execution_context.py | 2 +- .../dagster/dagster/_core/test_utils.py | 1 + .../test_asset_partition_mappings.py | 2 +- .../test_partitioned_assets.py | 111 +++++++++++++- .../execution_tests/test_asset_backfill.py | 58 ++++++- .../utils/partition_status_cache.py | 98 ++++++------ .../dagster-dbt/dagster_dbt/asset_specs.py | 1 + 14 files changed, 338 insertions(+), 183 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/asset_graph.py b/python_modules/dagster/dagster/_core/definitions/asset_graph.py index fdf5b528385eb..57ac569366530 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_graph.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_graph.py @@ -90,11 +90,11 @@ def owners(self) -> Sequence[str]: @property def is_partitioned(self) -> bool: - return self.assets_def.partitions_def is not None + return self.partitions_def is not None @property def partitions_def(self) -> Optional[PartitionsDefinition]: - return self.assets_def.partitions_def + return self.assets_def.specs_by_key[self.key].partitions_def @property def partition_mappings(self) -> Mapping[AssetKey, PartitionMapping]: diff --git a/python_modules/dagster/dagster/_core/definitions/asset_out.py b/python_modules/dagster/dagster/_core/definitions/asset_out.py index 050e1d9e03ea8..8d81cf665bcba 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_out.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_out.py @@ -22,6 +22,7 @@ from dagster._core.definitions.freshness_policy import FreshnessPolicy from dagster._core.definitions.input import NoValueSentinel from dagster._core.definitions.output import Out +from dagster._core.definitions.partition import PartitionsDefinition from dagster._core.definitions.utils import resolve_automation_condition from dagster._core.errors import DagsterInvalidDefinitionError from dagster._core.types.dagster_type import DagsterType @@ -217,12 +218,17 @@ def to_out(self) -> Out: ) def to_spec( - self, key: AssetKey, deps: Sequence[AssetDep], additional_tags: Mapping[str, str] = {} + self, + key: AssetKey, + deps: Sequence[AssetDep], + additional_tags: Mapping[str, str] = {}, + partitions_def: Optional[PartitionsDefinition] = ..., ) -> AssetSpec: return self._spec.replace_attributes( key=key, tags={**additional_tags, **self.tags} if self.tags else additional_tags, deps=[*self._spec.deps, *deps], + partitions_def=partitions_def, ) @public diff --git a/python_modules/dagster/dagster/_core/definitions/assets.py b/python_modules/dagster/dagster/_core/definitions/assets.py index 064507025685d..1eb903545f868 100644 --- a/python_modules/dagster/dagster/_core/definitions/assets.py +++ b/python_modules/dagster/dagster/_core/definitions/assets.py @@ -36,7 +36,7 @@ AssetSpec, ) from dagster._core.definitions.auto_materialize_policy import AutoMaterializePolicy -from dagster._core.definitions.backfill_policy import BackfillPolicy, BackfillPolicyType +from dagster._core.definitions.backfill_policy import BackfillPolicy from dagster._core.definitions.declarative_automation.automation_condition import ( AutomationCondition, ) @@ -107,9 +107,9 @@ class AssetsDefinition(ResourceAddable, IHasInternalInit): "descriptions_by_key", "asset_deps", "owners_by_key", + "partitions_def", } - _partitions_def: Optional[PartitionsDefinition] # partition mappings are also tracked inside the AssetSpecs, but this enables faster access by # upstream asset key _partition_mappings: Mapping[AssetKey, PartitionMapping] @@ -229,24 +229,10 @@ def __init__( execution_type=execution_type or AssetExecutionType.MATERIALIZATION, ) - self._partitions_def = _resolve_partitions_def(specs, partitions_def) - self._resource_defs = wrap_resources_for_execution( check.opt_mapping_param(resource_defs, "resource_defs") ) - if self._partitions_def is None: - # check if backfill policy is BackfillPolicyType.SINGLE_RUN if asset is not partitioned - check.param_invariant( - ( - backfill_policy.policy_type is BackfillPolicyType.SINGLE_RUN - if backfill_policy - else True - ), - "backfill_policy", - "Non partitioned asset can only have single run backfill policy", - ) - if specs is not None: check.invariant(group_names_by_key is None) check.invariant(metadata_by_key is None) @@ -258,6 +244,7 @@ def __init__( check.invariant(owners_by_key is None) check.invariant(partition_mappings is None) check.invariant(asset_deps is None) + check.invariant(partitions_def is None) resolved_specs = specs else: @@ -297,6 +284,7 @@ def __init__( metadata_by_key=metadata_by_key, descriptions_by_key=descriptions_by_key, code_versions_by_key=None, + partitions_def=partitions_def, ) normalized_specs: List[AssetSpec] = [] @@ -333,11 +321,11 @@ def __init__( check.invariant( not ( spec.freshness_policy - and self._partitions_def is not None - and not isinstance(self._partitions_def, TimeWindowPartitionsDefinition) + and spec.partitions_def is not None + and not isinstance(spec.partitions_def, TimeWindowPartitionsDefinition) ), "FreshnessPolicies are currently unsupported for assets with partitions of type" - f" {type(self._partitions_def)}.", + f" {spec.partitions_def}.", ) normalized_specs.append( @@ -347,10 +335,19 @@ def __init__( metadata=metadata, description=description, skippable=skippable, - partitions_def=self._partitions_def, ) ) + unique_partitions_defs = { + spec.partitions_def for spec in normalized_specs if spec.partitions_def is not None + } + if len(unique_partitions_defs) > 1 and not can_subset: + raise DagsterInvalidDefinitionError( + "If different AssetSpecs have different partitions_defs, can_subset must be True" + ) + + _validate_self_deps(normalized_specs) + self._specs_by_key = {spec.key: spec for spec in normalized_specs} self._partition_mappings = get_partition_mappings_from_deps( @@ -363,27 +360,11 @@ def __init__( spec.key: spec for spec in self._check_specs_by_output_name.values() } - if self._computation: - _validate_self_deps( - input_keys=[ - key - # filter out the special inputs which are used for cases when a multi-asset is - # subsetted, as these are not the same as self-dependencies and are never loaded - # in the same step that their corresponding output is produced - for input_name, key in self._computation.keys_by_input_name.items() - if not input_name.startswith(ASSET_SUBSET_INPUT_PREFIX) - ], - output_keys=self._computation.selected_asset_keys, - partition_mappings=self._partition_mappings, - partitions_def=self._partitions_def, - ) - def dagster_internal_init( *, keys_by_input_name: Mapping[str, AssetKey], keys_by_output_name: Mapping[str, AssetKey], node_def: NodeDefinition, - partitions_def: Optional[PartitionsDefinition], selected_asset_keys: Optional[AbstractSet[AssetKey]], can_subset: bool, resource_defs: Optional[Mapping[str, object]], @@ -400,7 +381,6 @@ def dagster_internal_init( keys_by_input_name=keys_by_input_name, keys_by_output_name=keys_by_output_name, node_def=node_def, - partitions_def=partitions_def, selected_asset_keys=selected_asset_keys, can_subset=can_subset, resource_defs=resource_defs, @@ -771,17 +751,13 @@ def _output_dict_to_asset_dict( metadata_by_key=_output_dict_to_asset_dict(metadata_by_output_name), descriptions_by_key=_output_dict_to_asset_dict(descriptions_by_output_name), code_versions_by_key=_output_dict_to_asset_dict(code_versions_by_output_name), + partitions_def=partitions_def, ) return AssetsDefinition.dagster_internal_init( keys_by_input_name=keys_by_input_name, keys_by_output_name=keys_by_output_name_with_prefix, node_def=node_def, - partitions_def=check.opt_inst_param( - partitions_def, - "partitions_def", - PartitionsDefinition, - ), resource_defs=resource_defs, backfill_policy=check.opt_inst_param( backfill_policy, "backfill_policy", BackfillPolicy @@ -1044,10 +1020,20 @@ def backfill_policy(self) -> Optional[BackfillPolicy]: return self._computation.backfill_policy if self._computation else None @public - @property + @cached_property def partitions_def(self) -> Optional[PartitionsDefinition]: """Optional[PartitionsDefinition]: The PartitionsDefinition for this AssetsDefinition (if any).""" - return self._partitions_def + partitions_defs = { + spec.partitions_def for spec in self.specs if spec.partitions_def is not None + } + if len(partitions_defs) == 1: + return next(iter(partitions_defs)) + elif len(partitions_defs) == 0: + return None + else: + check.failed( + "Different assets within this AssetsDefinition have different PartitionsDefinitions" + ) @property def metadata_by_key(self) -> Mapping[AssetKey, ArbitraryMetadataMapping]: @@ -1138,12 +1124,17 @@ def get_partition_mapping_for_dep(self, dep_key: AssetKey) -> Optional[Partition return self._partition_mappings.get(dep_key) def infer_partition_mapping( - self, upstream_asset_key: AssetKey, upstream_partitions_def: Optional[PartitionsDefinition] + self, + asset_key: AssetKey, + upstream_asset_key: AssetKey, + upstream_partitions_def: Optional[PartitionsDefinition], ) -> PartitionMapping: with disable_dagster_warnings(): partition_mapping = self._partition_mappings.get(upstream_asset_key) return infer_partition_mapping( - partition_mapping, self._partitions_def, upstream_partitions_def + partition_mapping, + self.specs_by_key[asset_key].partitions_def, + upstream_partitions_def, ) def has_output_for_asset_key(self, key: AssetKey) -> bool: @@ -1398,7 +1389,7 @@ def _output_to_source_asset(self, output_name: str) -> SourceAsset: io_manager_key=output_def.io_manager_key, description=spec.description, resource_defs=self.resource_defs, - partitions_def=self.partitions_def, + partitions_def=spec.partitions_def, group_name=spec.group_name, tags=spec.tags, io_manager_def=None, @@ -1504,7 +1495,6 @@ def get_attributes_dict(self) -> Dict[str, Any]: keys_by_input_name=self.node_keys_by_input_name, keys_by_output_name=self.node_keys_by_output_name, node_def=self._computation.node_def if self._computation else None, - partitions_def=self._partitions_def, selected_asset_keys=self.keys, can_subset=self.can_subset, resource_defs=self._resource_defs, @@ -1700,6 +1690,7 @@ def _asset_specs_from_attr_key_params( code_versions_by_key: Optional[Mapping[AssetKey, str]], descriptions_by_key: Optional[Mapping[AssetKey, str]], owners_by_key: Optional[Mapping[AssetKey, Sequence[str]]], + partitions_def: Optional[PartitionsDefinition], ) -> Sequence[AssetSpec]: validated_group_names_by_key = check.opt_mapping_param( group_names_by_key, "group_names_by_key", key_type=AssetKey, value_type=str @@ -1772,27 +1763,24 @@ def _asset_specs_from_attr_key_params( # NodeDefinition skippable=False, auto_materialize_policy=None, - partitions_def=None, kinds=None, + partitions_def=check.opt_inst_param( + partitions_def, "partitions_def", PartitionsDefinition + ), ) ) return result -def _validate_self_deps( - input_keys: Iterable[AssetKey], - output_keys: Iterable[AssetKey], - partition_mappings: Mapping[AssetKey, PartitionMapping], - partitions_def: Optional[PartitionsDefinition], -) -> None: - output_keys_set = set(output_keys) - for input_key in input_keys: - if input_key in output_keys_set: - if input_key in partition_mappings: - partition_mapping = partition_mappings[input_key] +def _validate_self_deps(specs: Iterable[AssetSpec]) -> None: + for spec in specs: + for dep in spec.deps: + if dep.asset_key != spec.key: + continue + if dep.partition_mapping: time_window_partition_mapping = get_self_dep_time_window_partition_mapping( - partition_mapping, partitions_def + dep.partition_mapping, spec.partitions_def ) if ( time_window_partition_mapping is not None @@ -1802,7 +1790,7 @@ def _validate_self_deps( continue raise DagsterInvalidDefinitionError( - f'Asset "{input_key.to_user_string()}" depends on itself. Assets can only depend' + f'Asset "{spec.key.to_user_string()}" depends on itself. Assets can only depend' " on themselves if they are:\n(a) time-partitioned and each partition depends on" " earlier partitions\n(b) multipartitioned, with one time dimension that depends" " on earlier time partitions" @@ -1834,38 +1822,6 @@ def get_self_dep_time_window_partition_mapping( return None -def _resolve_partitions_def( - specs: Optional[Sequence[AssetSpec]], partitions_def: Optional[PartitionsDefinition] -) -> Optional[PartitionsDefinition]: - if specs: - asset_keys_by_partitions_def = defaultdict(set) - for spec in specs: - asset_keys_by_partitions_def[spec.partitions_def].add(spec.key) - if len(asset_keys_by_partitions_def) > 1: - partition_1_asset_keys, partition_2_asset_keys, *_ = ( - asset_keys_by_partitions_def.values() - ) - check.failed( - f"All AssetSpecs must have the same partitions_def, but " - f"{next(iter(partition_1_asset_keys)).to_user_string()} and " - f"{next(iter(partition_2_asset_keys)).to_user_string()} have different " - "partitions_defs." - ) - common_partitions_def = next(iter(asset_keys_by_partitions_def.keys())) - if ( - common_partitions_def is not None - and partitions_def is not None - and common_partitions_def != partitions_def - ): - check.failed( - f"AssetSpec for {next(iter(specs)).key.to_user_string()} has partitions_def which is different " - "than the partitions_def provided to AssetsDefinition.", - ) - return partitions_def or common_partitions_def - else: - return partitions_def - - def get_partition_mappings_from_deps( partition_mappings: Dict[AssetKey, PartitionMapping], deps: Iterable[AssetDep], asset_name: str ) -> Mapping[AssetKey, PartitionMapping]: diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py index f96462413a596..cd5d477265db9 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py @@ -580,7 +580,6 @@ def create_assets_definition(self) -> AssetsDefinition: keys_by_input_name=self.asset_keys_by_input_names, keys_by_output_name=self.asset_keys_by_output_name, node_def=self.create_op_definition(), - partitions_def=self.args.partitions_def, can_subset=self.args.can_subset, resource_defs=self.args.assets_def_resource_defs, backfill_policy=self.args.backfill_policy, @@ -594,18 +593,61 @@ def create_assets_definition(self) -> AssetsDefinition: @cached_property def specs(self) -> Sequence[AssetSpec]: - specs = self.args.specs if self.args.specs else self._synthesize_specs() - - if not self.group_name: - return specs + if self.args.specs: + specs = self.args.specs + self._validate_spec_partitions_defs(specs, self.args.partitions_def) + else: + specs = self._synthesize_specs() check.invariant( - all((spec.group_name is None or spec.group_name == self.group_name) for spec in specs), + not self.group_name + or all( + (spec.group_name is None or spec.group_name == self.group_name) for spec in specs + ), "Cannot set group_name parameter on multi_asset if one or more of the" " AssetSpecs/AssetOuts supplied to this multi_asset have a group_name defined.", ) - return [spec.replace_attributes(group_name=self.group_name) for spec in specs] + if not self.group_name and not self.args.partitions_def: + return specs + + return [ + spec.replace_attributes( + group_name=self.group_name, + partitions_def=spec.partitions_def or self.args.partitions_def, + ) + for spec in specs + ] + + def _validate_spec_partitions_defs( + self, specs: Sequence[AssetSpec], partitions_def: Optional[PartitionsDefinition] + ) -> Optional[PartitionsDefinition]: + any_spec_has_partitions_def = False + any_spec_has_no_partitions_def = False + if partitions_def is not None: + for spec in specs: + if spec.partitions_def is not None and spec.partitions_def != partitions_def: + check.failed( + f"AssetSpec for {spec.key.to_user_string()} has partitions_def " + f"(type={type(spec.partitions_def)}) which is different than the " + f"partitions_def provided to AssetsDefinition (type={type(partitions_def)}).", + ) + + any_spec_has_partitions_def = ( + any_spec_has_partitions_def or spec.partitions_def is not None + ) + any_spec_has_no_partitions_def = ( + any_spec_has_no_partitions_def or spec.partitions_def is None + ) + + if ( + partitions_def is not None + and any_spec_has_partitions_def + and any_spec_has_no_partitions_def + ): + check.failed( + "If partitions_def is provided, then either all specs must have that PartitionsDefinition or none." + ) def _synthesize_specs(self) -> Sequence[AssetSpec]: resolved_specs = [] @@ -630,7 +672,9 @@ def _synthesize_specs(self) -> Sequence[AssetSpec]: else: deps = input_deps - resolved_specs.append(asset_out.to_spec(key, deps=deps)) + resolved_specs.append( + asset_out.to_spec(key, deps=deps, partitions_def=self.args.partitions_def) + ) specs = resolved_specs return specs diff --git a/python_modules/dagster/dagster/_core/definitions/external_asset.py b/python_modules/dagster/dagster/_core/definitions/external_asset.py index 57e7f3ad6fb4e..ef42bee63e384 100644 --- a/python_modules/dagster/dagster/_core/definitions/external_asset.py +++ b/python_modules/dagster/dagster/_core/definitions/external_asset.py @@ -153,13 +153,13 @@ def create_external_asset_from_source_asset(source_asset: SourceAsset) -> Assets automation_condition=source_asset.automation_condition, deps=[], owners=[], + partitions_def=source_asset.partitions_def, ) return AssetsDefinition( specs=[spec], keys_by_output_name=keys_by_output_name, node_def=node_def, - partitions_def=source_asset.partitions_def, # We don't pass the `io_manager_def` because it will already be present in # `resource_defs` (it is added during `SourceAsset` initialization). resource_defs=source_asset.resource_defs, diff --git a/python_modules/dagster/dagster/_core/definitions/multi_asset_sensor_definition.py b/python_modules/dagster/dagster/_core/definitions/multi_asset_sensor_definition.py index b98f6ba5a8253..a30322a421fe5 100644 --- a/python_modules/dagster/dagster/_core/definitions/multi_asset_sensor_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/multi_asset_sensor_definition.py @@ -265,13 +265,11 @@ def __init__( self._partitions_def_by_asset_key: Dict[AssetKey, Optional[PartitionsDefinition]] = {} asset_graph = self._repository_def.asset_graph for asset_key in self._monitored_asset_keys: - assets_def = ( - asset_graph.get(asset_key).assets_def if asset_graph.has(asset_key) else None - ) - self._assets_by_key[asset_key] = assets_def + asset_node = asset_graph.get(asset_key) if asset_graph.has(asset_key) else None + self._assets_by_key[asset_key] = asset_node.assets_def if asset_node else None self._partitions_def_by_asset_key[asset_key] = ( - assets_def.partitions_def if assets_def else None + asset_node.partitions_def if asset_node else None ) # Cursor object with utility methods for updating and retrieving cursor information. @@ -744,24 +742,25 @@ def get_downstream_partition_keys( to_asset = self._get_asset(to_asset_key, fn_name="get_downstream_partition_keys") from_asset = self._get_asset(from_asset_key, fn_name="get_downstream_partition_keys") - to_partitions_def = to_asset.partitions_def + to_partitions_def = to_asset.specs_by_key[to_asset_key].partitions_def + from_partitions_def = from_asset.specs_by_key[from_asset_key].partitions_def if not isinstance(to_partitions_def, PartitionsDefinition): raise DagsterInvalidInvocationError( f"Asset key {to_asset_key} is not partitioned. Cannot get partition keys." ) - if not isinstance(from_asset.partitions_def, PartitionsDefinition): + if not isinstance(from_partitions_def, PartitionsDefinition): raise DagsterInvalidInvocationError( f"Asset key {from_asset_key} is not partitioned. Cannot get partition keys." ) partition_mapping = to_asset.infer_partition_mapping( - from_asset_key, from_asset.partitions_def + to_asset_key, from_asset_key, from_partitions_def ) downstream_partition_key_subset = ( partition_mapping.get_downstream_partitions_for_partitions( - from_asset.partitions_def.empty_subset().with_partition_keys([partition_key]), - from_asset.partitions_def, + from_partitions_def.empty_subset().with_partition_keys([partition_key]), + from_partitions_def, downstream_partitions_def=to_partitions_def, dynamic_partitions_store=self.instance, ) diff --git a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py index da29ce7539dcb..938957c149f04 100644 --- a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py +++ b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py @@ -255,14 +255,15 @@ def build_caching_repository_data_from_list( asset_check_keys.update(definition.check_keys) asset_checks_defs.append(definition) elif isinstance(definition, AssetsDefinition): - for key in definition.keys: - if key in asset_keys: - raise DagsterInvalidDefinitionError(f"Duplicate asset key: {key}") + for spec in definition.specs: + if spec.key in asset_keys: + raise DagsterInvalidDefinitionError(f"Duplicate asset key: {spec.key}") + + if spec.partitions_def is not None: + partitions_defs.add(spec.partitions_def) for key in definition.check_keys: if key in asset_check_keys: raise DagsterInvalidDefinitionError(f"Duplicate asset check key: {key}") - if definition.partitions_def is not None: - partitions_defs.add(definition.partitions_def) asset_keys.update(definition.keys) asset_check_keys.update(definition.check_keys) diff --git a/python_modules/dagster/dagster/_core/execution/context/op_execution_context.py b/python_modules/dagster/dagster/_core/execution/context/op_execution_context.py index 6b3d90105c8b8..8b1466ba00f23 100644 --- a/python_modules/dagster/dagster/_core/execution/context/op_execution_context.py +++ b/python_modules/dagster/dagster/_core/execution/context/op_execution_context.py @@ -292,7 +292,7 @@ def an_asset(context: AssetExecutionContext): # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"] """ key_range = self.partition_key_range - partitions_def = self.assets_def.partitions_def + partitions_def = self._step_execution_context.run_partitions_def if partitions_def is None: raise DagsterInvariantViolationError( "Cannot access partition_keys for a non-partitioned run" diff --git a/python_modules/dagster/dagster/_core/test_utils.py b/python_modules/dagster/dagster/_core/test_utils.py index 117dda48a1102..8b76c1474981c 100644 --- a/python_modules/dagster/dagster/_core/test_utils.py +++ b/python_modules/dagster/dagster/_core/test_utils.py @@ -90,6 +90,7 @@ def assert_namedtuple_lists_equal( t2_list: Sequence[T_NamedTuple], exclude_fields: Optional[Sequence[str]] = None, ) -> None: + assert len(t1_list) == len(t2_list) for t1, t2 in zip(t1_list, t2_list): assert_namedtuples_equal(t1, t2, exclude_fields) diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py b/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py index 0e7c8881a5af3..c5e56d3748d37 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py @@ -251,7 +251,7 @@ def load_input(self, context): exclude_fields=["tags"], ) assert_namedtuple_lists_equal( - result.asset_materializations_for_node("downstream"), + result.asset_materializations_for_node("downstream_a_b"), [AssetMaterialization(AssetKey(["downstream_a_b"]))], exclude_fields=["tags"], ) diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py index 8b446f2b36baa..398d9dc796167 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py @@ -16,6 +16,7 @@ InputContext, IOManager, IOManagerDefinition, + MaterializeResult, MultiPartitionKey, MultiPartitionsDefinition, Output, @@ -58,6 +59,7 @@ def error_on_warning(): def get_upstream_partitions_for_partition_range( downstream_assets_def: AssetsDefinition, + downstream_asset_key: AssetKey, upstream_partitions_def: PartitionsDefinition, upstream_asset_key: AssetKey, downstream_partition_key_range: Optional[PartitionKeyRange], @@ -66,7 +68,7 @@ def get_upstream_partitions_for_partition_range( check.failed("upstream asset is not partitioned") downstream_partition_mapping = downstream_assets_def.infer_partition_mapping( - upstream_asset_key, upstream_partitions_def + downstream_asset_key, upstream_asset_key, upstream_partitions_def ) downstream_partitions_def = downstream_assets_def.partitions_def downstream_partitions_subset = ( @@ -92,6 +94,7 @@ def get_upstream_partitions_for_partition_range( def get_downstream_partitions_for_partition_range( downstream_assets_def: AssetsDefinition, + downstream_asset_key: AssetKey, upstream_assets_def: AssetsDefinition, upstream_asset_key: AssetKey, upstream_partition_key_range: PartitionKeyRange, @@ -103,7 +106,7 @@ def get_downstream_partitions_for_partition_range( check.failed("upstream asset is not partitioned") downstream_partition_mapping = downstream_assets_def.infer_partition_mapping( - upstream_asset_key, upstream_assets_def.partitions_def + downstream_asset_key, upstream_asset_key, upstream_assets_def.partitions_def ) upstream_partitions_def = upstream_assets_def.partitions_def upstream_partitions_subset = upstream_partitions_def.empty_subset().with_partition_keys( @@ -136,6 +139,7 @@ def downstream_asset(upstream_asset): assert get_upstream_partitions_for_partition_range( downstream_asset, + downstream_asset.key, upstream_asset.partitions_def, # pyright: ignore[reportArgumentType] AssetKey("upstream_asset"), PartitionKeyRange("a", "c"), @@ -143,6 +147,7 @@ def downstream_asset(upstream_asset): assert get_downstream_partitions_for_partition_range( downstream_asset, + downstream_asset.key, upstream_asset, AssetKey("upstream_asset"), PartitionKeyRange("a", "c"), @@ -427,6 +432,7 @@ def downstream_asset_2(upstream_asset_2: int): assert get_upstream_partitions_for_partition_range( downstream_asset_1, + downstream_asset_1.key, upstream_asset.partitions_def, # pyright: ignore[reportArgumentType] AssetKey("upstream_asset_1"), PartitionKeyRange("a", "c"), @@ -434,6 +440,7 @@ def downstream_asset_2(upstream_asset_2: int): assert get_upstream_partitions_for_partition_range( downstream_asset_2, + downstream_asset_2.key, upstream_asset.partitions_def, # pyright: ignore[reportArgumentType] AssetKey("upstream_asset_2"), PartitionKeyRange("a", "c"), @@ -441,6 +448,7 @@ def downstream_asset_2(upstream_asset_2: int): assert get_downstream_partitions_for_partition_range( downstream_asset_1, + downstream_asset_1.key, upstream_asset, AssetKey("upstream_asset_1"), PartitionKeyRange("a", "c"), @@ -448,6 +456,7 @@ def downstream_asset_2(upstream_asset_2: int): assert get_downstream_partitions_for_partition_range( downstream_asset_2, + downstream_asset_2.key, upstream_asset, AssetKey("upstream_asset_2"), PartitionKeyRange("a", "c"), @@ -490,6 +499,84 @@ def my_asset(): ) +def test_multi_asset_with_different_partitions_defs(): + partitions_def1 = StaticPartitionsDefinition(["a", "b", "c", "d"]) + partitions_def2 = StaticPartitionsDefinition(["1", "2", "3"]) + + @multi_asset( + specs=[ + AssetSpec("my_asset_1", partitions_def=partitions_def1), + AssetSpec("my_asset_2", partitions_def=partitions_def2), + ], + can_subset=True, + ) + def my_assets(context): + assert context.partition_key == "b" + assert context.partition_keys == ["b"] + for asset_key in context.selected_asset_keys: + yield MaterializeResult(asset_key=asset_key) + + result = materialize(assets=[my_assets], partition_key="b", selection=["my_asset_1"]) + assert result.success + + assert_namedtuple_lists_equal( + result.asset_materializations_for_node("my_assets"), + [ + AssetMaterialization(asset_key=AssetKey(["my_asset_1"]), partition="b"), + ], + exclude_fields=["tags"], + ) + + with pytest.raises( + DagsterInvalidDefinitionError, + match="Selected assets must have the same partitions definitions, but the selected assets ", + ): + materialize(assets=[my_assets], partition_key="b") + + +def test_multi_asset_with_different_partitions_defs_partition_key_range(): + partitions_def1 = DailyPartitionsDefinition(start_date="2020-01-01") + partitions_def2 = StaticPartitionsDefinition(["1", "2", "3"]) + + @multi_asset( + specs=[ + AssetSpec("my_asset_1", partitions_def=partitions_def1), + AssetSpec("my_asset_2", partitions_def=partitions_def2), + ], + can_subset=True, + ) + def my_assets(context): + assert context.partition_keys == ["2020-01-01", "2020-01-02", "2020-01-03"] + assert context.partition_key_range == PartitionKeyRange("2020-01-01", "2020-01-03") + assert context.partition_time_window == TimeWindow( + partitions_def1.time_window_for_partition_key("2020-01-01").start, + partitions_def1.time_window_for_partition_key("2020-01-03").end, + ) + for asset_key in context.selected_asset_keys: + yield MaterializeResult(asset_key=asset_key) + + result = materialize( + assets=[my_assets], + selection=["my_asset_1"], + tags={ + ASSET_PARTITION_RANGE_START_TAG: "2020-01-01", + ASSET_PARTITION_RANGE_END_TAG: "2020-01-03", + }, + ) + assert result.success + + materializations = result.asset_materializations_for_node("my_assets") + assert len(materializations) == 3 + assert { + (materialization.asset_key, materialization.partition) + for materialization in materializations + } == { + (AssetKey(["my_asset_1"]), "2020-01-01"), + (AssetKey(["my_asset_1"]), "2020-01-02"), + (AssetKey(["my_asset_1"]), "2020-01-03"), + } + + def test_two_partitioned_multi_assets_job(): partitions_def = StaticPartitionsDefinition(["a", "b", "c", "d"]) @@ -786,7 +873,7 @@ def assets2(): ... with pytest.raises( CheckError, - match="AssetSpec for asset1 has partitions_def which is different than the partitions_def provided to AssetsDefinition.", + match="which is different than the partitions_def provided to AssetsDefinition", ): @multi_asset( @@ -796,8 +883,8 @@ def assets2(): ... def assets3(): ... with pytest.raises( - CheckError, - match="All AssetSpecs must have the same partitions_def, but asset1 and asset2 have different partitions_defs.", + DagsterInvalidDefinitionError, + match="If different AssetSpecs have different partitions_defs, can_subset must be True", ): @multi_asset( @@ -807,3 +894,17 @@ def assets3(): ... ], ) def assets4(): ... + + with pytest.raises( + CheckError, + match="If partitions_def is provided, then either all specs must have that PartitionsDefinition or none", + ): + + @multi_asset( + specs=[ + AssetSpec("asset1", partitions_def=partitions_def), + AssetSpec("asset2"), + ], + partitions_def=partitions_def, + ) + def assets5(): ... diff --git a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py index b0c71d4084069..765204ab65ebe 100644 --- a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py +++ b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py @@ -17,20 +17,24 @@ import pytest from dagster import ( AssetCheckResult, + AssetDep, AssetIn, AssetKey, AssetOut, AssetsDefinition, + AssetSpec, BackfillPolicy, DagsterInstance, DagsterRunStatus, DailyPartitionsDefinition, HourlyPartitionsDefinition, LastPartitionMapping, + MaterializeResult, Nothing, PartitionKeyRange, PartitionsDefinition, RunRequest, + StaticPartitionMapping, StaticPartitionsDefinition, TimeWindowPartitionMapping, WeeklyPartitionsDefinition, @@ -562,7 +566,9 @@ def get_asset_graph( ) as get_builtin_partition_mapping_types: get_builtin_partition_mapping_types.return_value = tuple( assets_def.infer_partition_mapping( - dep_key, assets_defs_by_key[dep_key].partitions_def + next(iter(assets_def.keys)), + dep_key, + assets_defs_by_key[dep_key].specs_by_key[dep_key].partitions_def, ).__class__ for assets in assets_by_repo_name.values() for assets_def in assets @@ -1685,6 +1691,56 @@ def my_multi_asset(): assert AssetKeyPartitionKey(AssetKey("c"), "1") in backfill_data.requested_subset +def test_multi_asset_internal_deps_different_partitions_asset_backfill() -> None: + @multi_asset( + specs=[ + AssetSpec( + "asset1", partitions_def=StaticPartitionsDefinition(["a", "b"]), skippable=True + ), + AssetSpec( + "asset2", + partitions_def=StaticPartitionsDefinition(["1"]), + deps=[ + AssetDep( + "asset1", + partition_mapping=StaticPartitionMapping({"a": {"1"}, "b": {"1"}}), + ) + ], + skippable=True, + ), + ], + can_subset=True, + ) + def my_multi_asset(context): + for asset_key in context.selected_asset_keys: + yield MaterializeResult(asset_key=asset_key) + + instance = DagsterInstance.ephemeral() + repo_dict = {"repo": [my_multi_asset]} + asset_graph = get_asset_graph(repo_dict) + current_time = create_datetime(2024, 1, 9, 0, 0, 0) + asset_backfill_data = AssetBackfillData.from_asset_graph_subset( + asset_graph_subset=AssetGraphSubset.all( + asset_graph, dynamic_partitions_store=MagicMock(), current_time=current_time + ), + backfill_start_timestamp=current_time.timestamp(), + dynamic_partitions_store=MagicMock(), + ) + backfill_data_after_iter1 = _single_backfill_iteration( + "fake_id", asset_backfill_data, asset_graph, instance, repo_dict + ) + after_iter1_requested_subset = backfill_data_after_iter1.requested_subset + assert AssetKeyPartitionKey(AssetKey("asset1"), "a") in after_iter1_requested_subset + assert AssetKeyPartitionKey(AssetKey("asset1"), "b") in after_iter1_requested_subset + assert AssetKeyPartitionKey(AssetKey("asset2"), "1") not in after_iter1_requested_subset + + backfill_data_after_iter2 = _single_backfill_iteration( + "fake_id", backfill_data_after_iter1, asset_graph, instance, repo_dict + ) + after_iter2_requested_subset = backfill_data_after_iter2.requested_subset + assert AssetKeyPartitionKey(AssetKey("asset2"), "1") in after_iter2_requested_subset + + def test_multi_asset_internal_and_external_deps_asset_backfill() -> None: pd = StaticPartitionsDefinition(["1", "2", "3"]) diff --git a/python_modules/dagster/dagster_tests/storage_tests/utils/partition_status_cache.py b/python_modules/dagster/dagster_tests/storage_tests/utils/partition_status_cache.py index 9f4e8a9b715f4..fdc37412975db 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/utils/partition_status_cache.py +++ b/python_modules/dagster/dagster_tests/storage_tests/utils/partition_status_cache.py @@ -10,6 +10,7 @@ EventLogEntry, MultiPartitionKey, MultiPartitionsDefinition, + PartitionsDefinition, StaticPartitionsDefinition, asset, define_asset_job, @@ -79,36 +80,31 @@ def test_get_cached_partition_status_changed_time_partitions(self, instance): original_partitions_def = HourlyPartitionsDefinition(start_date="2022-01-01-00:00") new_partitions_def = DailyPartitionsDefinition(start_date="2022-01-01") - @asset(partitions_def=original_partitions_def) - def asset1(): - return 1 - - asset_key = AssetKey("asset1") - asset_graph = AssetGraph.from_assets([asset1]) - asset_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) + def make_asset_job_and_graph(partitions_def: PartitionsDefinition): + @asset(partitions_def=partitions_def) + def asset1(): + return 1 - def _swap_partitions_def(new_partitions_def, asset, asset_graph, asset_job): - asset._partitions_def = new_partitions_def # noqa: SLF001 - asset_graph = AssetGraph.from_assets([asset]) + asset_graph = AssetGraph.from_assets([asset1]) asset_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) - return asset, asset_job, asset_graph + return asset1, asset_job, asset_graph + + asset1, asset_job, asset_graph = make_asset_job_and_graph(original_partitions_def) counter = Counter() traced_counter.set(counter) - asset_records = list(instance.get_asset_records([asset_key])) + asset_records = list(instance.get_asset_records([asset1.key])) assert len(asset_records) == 0 asset_job.execute_in_process(instance=instance, partition_key="2022-02-01-00:00") # swap the partitions def and kick off a run before we try to get the cached status - asset1, asset_job, asset_graph = _swap_partitions_def( - new_partitions_def, asset1, asset_graph, asset_job - ) + asset1, asset_job, asset_graph = make_asset_job_and_graph(new_partitions_def) asset_job.execute_in_process(instance=instance, partition_key="2022-02-02") cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, asset1.key, asset_graph.get(asset1.key).partitions_def ) assert cached_status @@ -127,34 +123,31 @@ def _swap_partitions_def(new_partitions_def, asset, asset_graph, asset_job): def test_get_cached_partition_status_by_asset(self, instance): partitions_def = DailyPartitionsDefinition(start_date="2022-01-01") - @asset(partitions_def=partitions_def) - def asset1(): - return 1 + def make_asset_job_and_graph(partitions_def: PartitionsDefinition): + @asset(partitions_def=partitions_def) + def asset1(): + return 1 - asset_key = AssetKey("asset1") - asset_graph = AssetGraph.from_assets([asset1]) - asset_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) - - def _swap_partitions_def(new_partitions_def, asset, asset_graph, asset_job): - asset._partitions_def = new_partitions_def # noqa: SLF001 - asset_graph = AssetGraph.from_assets([asset]) + asset_graph = AssetGraph.from_assets([asset1]) asset_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) - return asset, asset_job, asset_graph + return asset1, asset_job, asset_graph + + asset1, asset_job, asset_graph = make_asset_job_and_graph(partitions_def) traced_counter.set(Counter()) - asset_records = list(instance.get_asset_records([asset_key])) + asset_records = list(instance.get_asset_records([asset1.key])) assert len(asset_records) == 0 cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, asset1.key, asset_graph.get(asset1.key).partitions_def ) assert not cached_status asset_job.execute_in_process(instance=instance, partition_key="2022-02-01") cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, asset1.key, asset_graph.get(asset1.key).partitions_def ) assert cached_status assert cached_status.latest_storage_id @@ -173,7 +166,7 @@ def _swap_partitions_def(new_partitions_def, asset, asset_graph, asset_job): asset_job.execute_in_process(instance=instance, partition_key="2022-02-02") cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, asset1.key, asset_graph.get(asset1.key).partitions_def ) assert cached_status assert cached_status.latest_storage_id @@ -190,12 +183,10 @@ def _swap_partitions_def(new_partitions_def, asset, asset_graph, asset_job): ) static_partitions_def = StaticPartitionsDefinition(["a", "b", "c"]) - asset1, asset_job, asset_graph = _swap_partitions_def( - static_partitions_def, asset1, asset_graph, asset_job - ) + asset1, asset_job, asset_graph = make_asset_job_and_graph(static_partitions_def) asset_job.execute_in_process(instance=instance, partition_key="a") cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, asset1.key, asset_graph.get(asset1.key).partitions_def ) assert cached_status assert cached_status.serialized_materialized_partition_subset @@ -795,36 +786,35 @@ def asset1(context): def test_failed_partitioned_asset_converted_to_multipartitioned(self, instance): daily_def = DailyPartitionsDefinition("2023-01-01") - @asset( - partitions_def=daily_def, - ) - def my_asset(): - raise Exception("oops") + def make_asset_job_and_graph(partitions_def: PartitionsDefinition): + @asset(partitions_def=partitions_def) + def my_asset(): + raise Exception("oops") - asset_graph = AssetGraph.from_assets([my_asset]) - my_job = define_asset_job("asset_job", partitions_def=daily_def).resolve( - asset_graph=asset_graph - ) + asset_graph = AssetGraph.from_assets([my_asset]) + asset_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) + return my_asset, asset_job, asset_graph + + my_asset, my_job, asset_graph = make_asset_job_and_graph(daily_def) my_job.execute_in_process( instance=instance, partition_key="2023-01-01", raise_on_error=False ) - my_asset._partitions_def = MultiPartitionsDefinition( # noqa: SLF001 - partitions_defs={ - "a": DailyPartitionsDefinition("2023-01-01"), - "b": StaticPartitionsDefinition(["a", "b"]), - } + my_asset, my_job, asset_graph = make_asset_job_and_graph( + MultiPartitionsDefinition( + partitions_defs={ + "a": DailyPartitionsDefinition("2023-01-01"), + "b": StaticPartitionsDefinition(["a", "b"]), + } + ) ) - asset_graph = AssetGraph.from_assets([my_asset]) - my_job = define_asset_job("asset_job").resolve(asset_graph=asset_graph) - asset_key = AssetKey("my_asset") cached_status = get_and_update_asset_status_cache_value( - instance, asset_key, asset_graph.get(asset_key).partitions_def + instance, my_asset.key, asset_graph.get(my_asset.key).partitions_def ) failed_subset = cached_status.deserialize_failed_partition_subsets( # pyright: ignore[reportOptionalMemberAccess] - asset_graph.get(asset_key).partitions_def # pyright: ignore[reportArgumentType] + asset_graph.get(my_asset.key).partitions_def # pyright: ignore[reportArgumentType] ) assert failed_subset.get_partition_keys() == set() diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_specs.py b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_specs.py index 8c80ff498c353..0f3ed7eca908c 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_specs.py +++ b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_specs.py @@ -62,6 +62,7 @@ def build_dbt_asset_specs( key=check.inst(asset_out.key, AssetKey), deps=[AssetDep(asset=dep) for dep in internal_asset_deps.get(output_name, set())], additional_tags={f"{KIND_PREFIX}dbt": ""}, + partitions_def=None, ) # Allow specs to be represented as external assets by adhering to external asset invariants. ._replace( From 09333d9ae6025668e4e6e82bd2325e0feb03dd1f Mon Sep 17 00:00:00 2001 From: OwenKephart Date: Thu, 19 Dec 2024 17:24:37 -0500 Subject: [PATCH 34/37] [components] Add custom scope for custom components (#26594) ## Summary & Motivation Small change, big usability upgrade. This lets users define custom scope for users of their component to take advantage of. This scope can be anything from convenience string constants to complex functions that return raw python types. In the unit test, I have an example showing how this could let someone set up a custom automation condition constructor, but the possibilities here are kinda endless. For example, you could easily imagine creating more complex translator methods without having to fully give up the yaml world using this capability. ## How I Tested These Changes ## Changelog NOCHANGELOG --- .../dagster_components/core/component.py | 10 +++++ .../core/component_defs_builder.py | 1 + .../core/component_rendering.py | 12 +++--- .../rendering_tests/__init__.py | 0 .../custom_scope_component/component.py | 42 +++++++++++++++++++ .../custom_scope_component/component.yaml | 9 ++++ .../test_component_rendering.py | 1 - .../rendering_tests/test_custom_scope.py | 27 ++++++++++++ 8 files changed, 95 insertions(+), 7 deletions(-) create mode 100644 python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/__init__.py create mode 100644 python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.py create mode 100644 python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.yaml rename python_modules/libraries/dagster-components/dagster_components_tests/{unit_tests => rendering_tests}/test_component_rendering.py (98%) create mode 100644 python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_custom_scope.py diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component.py b/python_modules/libraries/dagster-components/dagster_components/core/component.py index 4f5d7e1e829e9..dc1d77f6dcf72 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component.py @@ -49,6 +49,10 @@ class Component(ABC): params_schema: ClassVar = None generate_params_schema: ClassVar = None + @classmethod + def get_rendering_scope(cls) -> Mapping[str, Any]: + return {} + @classmethod def generate_files(cls, request: ComponentGenerateRequest, params: Any) -> None: ... @@ -233,6 +237,12 @@ def path(self) -> Path: return self.decl_node.path + def with_rendering_scope(self, rendering_scope: Mapping[str, Any]) -> "ComponentLoadContext": + return dataclasses.replace( + self, + templated_value_resolver=self.templated_value_resolver.with_context(**rendering_scope), + ) + def for_decl_node(self, decl_node: ComponentDeclNode) -> "ComponentLoadContext": return dataclasses.replace(self, decl_node=decl_node) diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py b/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py index 04a1ce93ed03a..6021344d9cf39 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component_defs_builder.py @@ -42,6 +42,7 @@ def load_module_from_path(module_name, path) -> ModuleType: def load_components_from_context(context: ComponentLoadContext) -> Sequence[Component]: if isinstance(context.decl_node, YamlComponentDecl): component_type = component_type_from_yaml_decl(context.registry, context.decl_node) + context = context.with_rendering_scope(component_type.get_rendering_scope()) return [component_type.load(context)] elif isinstance(context.decl_node, ComponentFolder): components = [] diff --git a/python_modules/libraries/dagster-components/dagster_components/core/component_rendering.py b/python_modules/libraries/dagster-components/dagster_components/core/component_rendering.py index 02b47e71be6d2..b766edb09186d 100644 --- a/python_modules/libraries/dagster-components/dagster_components/core/component_rendering.py +++ b/python_modules/libraries/dagster-components/dagster_components/core/component_rendering.py @@ -4,7 +4,7 @@ import dagster._check as check from dagster._record import record -from jinja2 import Template +from jinja2.nativetypes import NativeTemplate from pydantic import BaseModel, Field from pydantic.fields import FieldInfo @@ -51,8 +51,8 @@ def default() -> "TemplatedValueResolver": def with_context(self, **additional_context) -> "TemplatedValueResolver": return TemplatedValueResolver(context={**self.context, **additional_context}) - def resolve(self, val: str) -> str: - return Template(val).render(**self.context) + def resolve(self, val: Any) -> Any: + return NativeTemplate(val).render(**self.context) if isinstance(val, str) else val def _should_render( @@ -70,7 +70,7 @@ def _should_render( # Optional[ComplexType] (e.g.) will contain multiple schemas in the "anyOf" field if "anyOf" in subschema: - return any(_should_render(valpath, json_schema, inner) for inner in subschema["anyOf"]) + return all(_should_render(valpath, json_schema, inner) for inner in subschema["anyOf"]) el = valpath[0] if isinstance(el, str): @@ -84,9 +84,9 @@ def _should_render( else: check.failed(f"Unexpected valpath element: {el}") - # the path wasn't valid + # the path wasn't valid, or unspecified if not inner: - return False + return subschema.get("additionalProperties", True) _, *rest = valpath return _should_render(rest, json_schema, inner) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/__init__.py b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.py b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.py new file mode 100644 index 0000000000000..55075ba403c82 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.py @@ -0,0 +1,42 @@ +from typing import Any, Mapping + +from dagster import AssetSpec, AutomationCondition, Definitions +from dagster_components import Component, ComponentLoadContext, component +from pydantic import BaseModel + + +def my_custom_fn(a: str, b: str) -> str: + return a + "|" + b + + +def my_custom_automation_condition(cron_schedule: str) -> AutomationCondition: + return AutomationCondition.cron_tick_passed(cron_schedule) & ~AutomationCondition.in_progress() + + +class CustomScopeParams(BaseModel): + attributes: Mapping[str, Any] + + +@component(name="custom_scope_component") +class HasCustomScope(Component): + params_schema = CustomScopeParams + + @classmethod + def get_rendering_scope(cls) -> Mapping[str, Any]: + return { + "custom_str": "xyz", + "custom_dict": {"a": "b"}, + "custom_fn": my_custom_fn, + "custom_automation_condition": my_custom_automation_condition, + } + + def __init__(self, attributes: Mapping[str, Any]): + self.attributes = attributes + + @classmethod + def load(cls, context: ComponentLoadContext): + loaded_params = context.load_params(cls.params_schema) + return cls(attributes=loaded_params.attributes) + + def build_defs(self, context: ComponentLoadContext): + return Definitions(assets=[AssetSpec(key="key", **self.attributes)]) diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.yaml b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.yaml new file mode 100644 index 0000000000000..a23aca62cf535 --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/custom_scope_component/component.yaml @@ -0,0 +1,9 @@ +type: .custom_scope_component + +params: + attributes: + group_name: "{{ custom_str }}" + tags: "{{ custom_dict }}" + metadata: + prefixed: "prefixed_{{ custom_fn('a', custom_str) }}" + automation_condition: "{{ custom_automation_condition('@daily') }}" diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_component_rendering.py b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_component_rendering.py similarity index 98% rename from python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_component_rendering.py rename to python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_component_rendering.py index 6e036492363c1..e93fbd4a9dbe6 100644 --- a/python_modules/libraries/dagster-components/dagster_components_tests/unit_tests/test_component_rendering.py +++ b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_component_rendering.py @@ -41,7 +41,6 @@ class Outer(BaseModel): (["inner_optional", 0, "deferred"], False), (["inner_deferred_optional", 0], False), (["inner_deferred_optional", 0, "a"], False), - (["NONEXIST", 0, "deferred"], False), ], ) def test_should_render(path, expected: bool) -> None: diff --git a/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_custom_scope.py b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_custom_scope.py new file mode 100644 index 0000000000000..8a8af764d332e --- /dev/null +++ b/python_modules/libraries/dagster-components/dagster_components_tests/rendering_tests/test_custom_scope.py @@ -0,0 +1,27 @@ +from pathlib import Path + +from dagster import AssetSpec, AutomationCondition +from dagster_components.core.component_defs_builder import build_defs_from_component_path + +from dagster_components_tests.utils import registry + + +def test_custom_scope() -> None: + defs = build_defs_from_component_path( + path=Path(__file__).parent / "custom_scope_component", + registry=registry(), + resources={}, + ) + + assets = list(defs.assets or []) + assert len(assets) == 1 + spec = assets[0] + assert isinstance(spec, AssetSpec) + + assert spec.group_name == "xyz" + assert spec.tags == {"a": "b"} + assert spec.metadata == {"prefixed": "prefixed_a|xyz"} + assert ( + spec.automation_condition + == AutomationCondition.cron_tick_passed("@daily") & ~AutomationCondition.in_progress() + ) From d4c99f587f088ac8ed4f7d24648b85fd0d1baf08 Mon Sep 17 00:00:00 2001 From: elementl Date: Thu, 19 Dec 2024 22:53:55 +0000 Subject: [PATCH 35/37] Automation: versioned docs for 1.9.6 --- docs/next/.versioned_content/_versions_with_static_links.json | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/next/.versioned_content/_versions_with_static_links.json b/docs/next/.versioned_content/_versions_with_static_links.json index 5b12b5f2ea18d..9b2fdbc33b793 100644 --- a/docs/next/.versioned_content/_versions_with_static_links.json +++ b/docs/next/.versioned_content/_versions_with_static_links.json @@ -610,5 +610,9 @@ { "url": "https://release-1-9-5.dagster.dagster-docs.io/", "version": "1.9.5" + }, + { + "url": "https://release-1-9-6.dagster.dagster-docs.io/", + "version": "1.9.6" } ] \ No newline at end of file From e7533d674e0a5ab5920fef6416f959555345812e Mon Sep 17 00:00:00 2001 From: David Liu <48995019+dliu27@users.noreply.github.com> Date: Thu, 19 Dec 2024 17:59:46 -0500 Subject: [PATCH 36/37] [ui] fix job names (#26616) ## Summary & Motivation We want to fallback to jobName if its not specified in request.jobName ## How I Tested These Changes Will test in canary, jest --- .../useLaunchMultipleRunsWithTelemetry.ts | 14 ++------------ .../ui-core/src/ticks/EvaluateScheduleDialog.tsx | 15 ++++----------- .../ui-core/src/ticks/SensorDryRunDialog.tsx | 7 +++---- .../ui-core/src/util/buildExecutionParamsList.ts | 6 ++++-- 4 files changed, 13 insertions(+), 29 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts index 77b1891f512d6..f9c6f36fe9899 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts @@ -24,26 +24,16 @@ export function useLaunchMultipleRunsWithTelemetry() { const history = useHistory(); return useCallback( - async ( - variables: LaunchMultipleRunsMutationVariables, - behavior: LaunchBehavior, - jobName: string, - ) => { + async (variables: LaunchMultipleRunsMutationVariables, behavior: LaunchBehavior) => { try { const executionParamsList = Array.isArray(variables.executionParamsList) ? variables.executionParamsList : [variables.executionParamsList]; - let jobNames = executionParamsList.map( + const jobNames = executionParamsList.map( (params) => params.selector.jobName || params.selector.pipelineName, ); - // if only executing one job, and jobName isn't defined, fallback to jobName from sensor/schedule - if (executionParamsList.length === 1 && !executionParamsList[0]?.selector?.jobName) { - jobNames = [jobName]; - executionParamsList[0]!.selector.jobName = jobName; - } - if ( jobNames.length !== executionParamsList.length || jobNames.includes(undefined) || diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx index 4cb739a3d7447..cdb83e8aa6066 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx @@ -170,9 +170,9 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { const executionParamsList = useMemo( () => scheduleExecutionData && scheduleSelector - ? buildExecutionParamsListSchedule(scheduleExecutionData, scheduleSelector) + ? buildExecutionParamsListSchedule(scheduleExecutionData, scheduleSelector, jobName) : [], - [scheduleSelector, scheduleExecutionData], + [scheduleSelector, scheduleExecutionData, jobName], ); const canLaunchAll = useMemo(() => { @@ -190,7 +190,7 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { try { if (executionParamsList) { - await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast', jobName); + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); } } catch (e) { console.error(e); @@ -198,14 +198,7 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { setLaunching(false); onClose(); - }, [ - canLaunchAll, - executionParamsList, - jobName, - launchMultipleRunsWithTelemetry, - onClose, - trackEvent, - ]); + }, [canLaunchAll, executionParamsList, launchMultipleRunsWithTelemetry, onClose, trackEvent]); const content = useMemo(() => { // launching all runs state diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index 83cad592e15a7..d54ec777a3f9a 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -105,9 +105,9 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop const executionParamsList = useMemo( () => sensorExecutionData && sensorSelector - ? buildExecutionParamsListSensor(sensorExecutionData, sensorSelector) + ? buildExecutionParamsListSensor(sensorExecutionData, sensorSelector, jobName) : [], - [sensorSelector, sensorExecutionData], + [sensorSelector, sensorExecutionData, jobName], ); const submitTest = useCallback(async () => { @@ -196,7 +196,7 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop try { if (executionParamsList) { - await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast', jobName); + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); onCommitTickResult(); // persist tick } } catch (e) { @@ -208,7 +208,6 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop }, [ canLaunchAll, executionParamsList, - jobName, launchMultipleRunsWithTelemetry, onClose, onCommitTickResult, diff --git a/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts index d5264bf96a6b6..41f621f63f131 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts @@ -15,6 +15,7 @@ const onlyKeyAndValue = ({key, value}: {key: string; value: string}) => ({key, v export const buildExecutionParamsListSensor = ( sensorExecutionData: SensorDryRunInstigationTick, sensorSelector: SensorSelector, + jobName: string, ) => { if (!sensorExecutionData) { return []; @@ -36,7 +37,7 @@ export const buildExecutionParamsListSensor = ( const executionParams: ExecutionParams = { runConfigData: configYamlOrEmpty, selector: { - jobName: request.jobName, // get jobName from runRequest + jobName: request.jobName ?? jobName, // get jobName from runRequest, fallback to jobName repositoryLocationName, repositoryName, assetSelection: [], @@ -57,6 +58,7 @@ export const buildExecutionParamsListSensor = ( export const buildExecutionParamsListSchedule = ( scheduleExecutionData: ScheduleDryRunInstigationTick, scheduleSelector: ScheduleSelector, + jobName: string, ) => { if (!scheduleExecutionData) { return []; @@ -78,7 +80,7 @@ export const buildExecutionParamsListSchedule = ( const executionParams: ExecutionParams = { runConfigData: configYamlOrEmpty, selector: { - jobName: request.jobName, // get jobName from runRequest + jobName: request.jobName ?? jobName, // get jobName from runRequest, fallback to jobName repositoryLocationName, repositoryName, assetSelection: [], From a14cb87a2caa69ce10d27674e7de518721bc5cfd Mon Sep 17 00:00:00 2001 From: prha <1040172+prha@users.noreply.github.com> Date: Thu, 19 Dec 2024 16:41:32 -0800 Subject: [PATCH 37/37] add support for incremental step stats calculations (#26577) ## Summary & Motivation This is the counterpart of https://github.com/dagster-io/dagster/pull/26550, but for step stats instead of run stats. The step stats calculations are a little more complicated because it relies on intermediate state in between dagster events to be held. This includes the `previous_attempt_start` time for a given step. This PR introduces a field in the snapshot to carry over this state, which should only be used in the incremental calculation. Another change made here is that marker events (e.g. resource init/teardown) can be associated with a step, but generally occur before the step has started. That means that previously step snapshots did not include these marker events if the step had not yet started. This PR changes that behavior so that every step will have a snapshot if it has a marker event, even if it has not started. The output for incremental vs completed run snapshots should match exactly. ## How I Tested These Changes BK --- .../dagster/dagster/_core/execution/stats.py | 316 +++++++++++------- .../storage_tests/test_event_log.py | 55 ++- 2 files changed, 245 insertions(+), 126 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/stats.py b/python_modules/dagster/dagster/_core/execution/stats.py index 0cea0fc4a0a6f..bced58da9aa75 100644 --- a/python_modules/dagster/dagster/_core/execution/stats.py +++ b/python_modules/dagster/dagster/_core/execution/stats.py @@ -1,6 +1,6 @@ from collections import defaultdict from enum import Enum -from typing import Any, Dict, Iterable, NamedTuple, Optional, Sequence, cast +from typing import Any, Dict, Iterable, Mapping, Optional, Sequence, cast import dagster._check as check from dagster._core.definitions import ExpectationResult @@ -12,6 +12,7 @@ ) from dagster._core.events.log import EventLogEntry from dagster._core.storage.dagster_run import DagsterRunStatsSnapshot +from dagster._record import IHaveNew, record, record_custom from dagster._serdes import whitelist_for_serdes RUN_STATS_EVENT_TYPES = { @@ -47,8 +48,8 @@ def build_run_stats_from_events( raise check.ParameterCheckError( "Invariant violation for parameter 'records'. Description: Expected iterable." ) from exc - for i, record in enumerate(entries): - check.inst_param(record, f"records[{i}]", EventLogEntry) + for i, entry in enumerate(entries): + check.inst_param(entry, f"entries[{i}]", EventLogEntry) if previous_stats: steps_succeeded = previous_stats.steps_succeeded @@ -115,14 +116,154 @@ class StepEventStatus(Enum): IN_PROGRESS = "IN_PROGRESS" +@whitelist_for_serdes +@record_custom +class RunStepMarker(IHaveNew): + start_time: Optional[float] + end_time: Optional[float] + key: Optional[str] + + def __new__( + cls, + start_time: Optional[float] = None, + end_time: Optional[float] = None, + key: Optional[str] = None, + ): + return super().__new__( + cls, + start_time=check.opt_float_param(start_time, "start_time"), + end_time=check.opt_float_param(end_time, "end_time"), + key=check.opt_str_param(key, "key"), + ) + + +@whitelist_for_serdes +@record_custom +class RunStepKeyStatsSnapshot(IHaveNew): + run_id: str + step_key: str + status: Optional[StepEventStatus] + start_time: Optional[float] + end_time: Optional[float] + materialization_events: Sequence[EventLogEntry] + expectation_results: Sequence[ExpectationResult] + attempts: Optional[int] + attempts_list: Sequence[RunStepMarker] + markers: Sequence[RunStepMarker] + partial_attempt_start: Optional[float] + + def __new__( + cls, + run_id: str, + step_key: str, + status: Optional[StepEventStatus] = None, + start_time: Optional[float] = None, + end_time: Optional[float] = None, + materialization_events: Optional[Sequence[EventLogEntry]] = None, + expectation_results: Optional[Sequence[ExpectationResult]] = None, + attempts: Optional[int] = None, + attempts_list: Optional[Sequence[RunStepMarker]] = None, + markers: Optional[Sequence[RunStepMarker]] = None, + partial_attempt_start: Optional[float] = None, + ): + return super().__new__( + cls, + run_id=check.str_param(run_id, "run_id"), + step_key=check.str_param(step_key, "step_key"), + status=check.opt_inst_param(status, "status", StepEventStatus), + start_time=check.opt_float_param(start_time, "start_time"), + end_time=check.opt_float_param(end_time, "end_time"), + materialization_events=check.opt_sequence_param( + materialization_events, + "materialization_events", + EventLogEntry, + ), + expectation_results=check.opt_sequence_param( + expectation_results, "expectation_results", ExpectationResult + ), + attempts=check.opt_int_param(attempts, "attempts"), + attempts_list=check.opt_sequence_param(attempts_list, "attempts_list", RunStepMarker), + markers=check.opt_sequence_param(markers, "markers", RunStepMarker), + # used to calculate incremental step stats using batches of event logs + partial_attempt_start=check.opt_float_param( + partial_attempt_start, "partial_attempt_start" + ), + ) + + +@whitelist_for_serdes +@record +class RunStepStatsSnapshot: + run_id: str + step_key_stats: Sequence[RunStepKeyStatsSnapshot] + partial_markers: Optional[Mapping[str, Sequence[RunStepMarker]]] + + def build_run_step_stats_from_events( - run_id: str, records: Iterable[EventLogEntry] -) -> Sequence["RunStepKeyStatsSnapshot"]: + run_id: str, + entries: Iterable[EventLogEntry], +) -> Sequence[RunStepKeyStatsSnapshot]: + snapshot = build_run_step_stats_snapshot_from_events(run_id, entries) + return snapshot.step_key_stats + + +def build_run_step_stats_snapshot_from_events( + run_id: str, + entries: Iterable[EventLogEntry], + previous_snapshot: Optional["RunStepStatsSnapshot"] = None, +) -> "RunStepStatsSnapshot": by_step_key: Dict[str, Dict[str, Any]] = defaultdict(dict) attempts = defaultdict(list) - attempt_events = defaultdict(list) markers: Dict[str, Dict[str, Any]] = defaultdict(dict) - for event in records: + + if previous_snapshot: + for step_stats in previous_snapshot.step_key_stats: + check.invariant(step_stats.run_id == run_id) + by_step_key[step_stats.step_key] = { + "start_time": step_stats.start_time, + "end_time": step_stats.end_time, + "status": step_stats.status, + "materialization_events": step_stats.materialization_events, + "expectation_results": step_stats.expectation_results, + "attempts": step_stats.attempts, + "partial_attempt_start": step_stats.partial_attempt_start, + } + for attempt in step_stats.attempts_list: + attempts[step_stats.step_key].append(attempt) + + for marker in step_stats.markers: + assert marker.key + markers[step_stats.step_key][marker.key] = { + "key": marker.key, + "start": marker.start_time, + "end": marker.end_time, + } + + # handle the partial markers + if previous_snapshot.partial_markers: + for step_key, partial_markers in previous_snapshot.partial_markers.items(): + for marker in partial_markers: + assert marker.key + markers[step_key][marker.key] = { + "key": marker.key, + "start": marker.start_time, + "end": marker.end_time, + } + + def _open_attempt(step_key: str, event: EventLogEntry) -> None: + by_step_key[step_key]["attempts"] = int(by_step_key[step_key].get("attempts") or 0) + 1 + by_step_key[step_key]["partial_attempt_start"] = event.timestamp + + def _close_attempt(step_key: str, event: EventLogEntry) -> None: + attempts[step_key].append( + RunStepMarker( + start_time=by_step_key[step_key].get("partial_attempt_start"), + end_time=event.timestamp, + ) + ) + by_step_key[step_key]["partial_attempt_start"] = None + + for event in entries: if not event.is_dagster_event: continue dagster_event = event.get_dagster_event() @@ -135,19 +276,25 @@ def build_run_step_stats_from_events( continue if dagster_event.event_type == DagsterEventType.STEP_START: + by_step_key[step_key]["status"] = StepEventStatus.IN_PROGRESS by_step_key[step_key]["start_time"] = event.timestamp - by_step_key[step_key]["attempts"] = 1 + _open_attempt(step_key, event) + if dagster_event.event_type == DagsterEventType.STEP_RESTARTED: + _open_attempt(step_key, event) + if dagster_event.event_type == DagsterEventType.STEP_UP_FOR_RETRY: + _close_attempt(step_key, event) if dagster_event.event_type == DagsterEventType.STEP_FAILURE: by_step_key[step_key]["end_time"] = event.timestamp by_step_key[step_key]["status"] = StepEventStatus.FAILURE - if dagster_event.event_type == DagsterEventType.STEP_RESTARTED: - by_step_key[step_key]["attempts"] = int(by_step_key[step_key].get("attempts") or 0) + 1 + _close_attempt(step_key, event) if dagster_event.event_type == DagsterEventType.STEP_SUCCESS: by_step_key[step_key]["end_time"] = event.timestamp by_step_key[step_key]["status"] = StepEventStatus.SUCCESS + _close_attempt(step_key, event) if dagster_event.event_type == DagsterEventType.STEP_SKIPPED: by_step_key[step_key]["end_time"] = event.timestamp by_step_key[step_key]["status"] = StepEventStatus.SKIPPED + _close_attempt(step_key, event) if dagster_event.event_type == DagsterEventType.ASSET_MATERIALIZATION: materialization_events = by_step_key[step_key].get("materialization_events", []) materialization_events.append(event) @@ -158,129 +305,50 @@ def build_run_step_stats_from_events( step_expectation_results = by_step_key[step_key].get("expectation_results", []) step_expectation_results.append(expectation_result) by_step_key[step_key]["expectation_results"] = step_expectation_results - if dagster_event.event_type in ( - DagsterEventType.STEP_UP_FOR_RETRY, - DagsterEventType.STEP_RESTARTED, - ): - attempt_events[step_key].append(event) + if dagster_event.event_type in MARKER_EVENTS: if dagster_event.engine_event_data.marker_start: - key = dagster_event.engine_event_data.marker_start - if key not in markers[step_key]: - markers[step_key][key] = {"key": key, "start": event.timestamp} + marker_key = dagster_event.engine_event_data.marker_start + if marker_key not in markers[step_key]: + markers[step_key][marker_key] = {"key": marker_key, "start": event.timestamp} else: - markers[step_key][key]["start"] = event.timestamp + markers[step_key][marker_key]["start"] = event.timestamp if dagster_event.engine_event_data.marker_end: - key = dagster_event.engine_event_data.marker_end - if key not in markers[step_key]: - markers[step_key][key] = {"key": key, "end": event.timestamp} + marker_key = dagster_event.engine_event_data.marker_end + if marker_key not in markers[step_key]: + markers[step_key][marker_key] = {"key": marker_key, "end": event.timestamp} else: - markers[step_key][key]["end"] = event.timestamp + markers[step_key][marker_key]["end"] = event.timestamp + snapshots = [] for step_key, step_stats in by_step_key.items(): - events = attempt_events[step_key] - step_attempts = [] - attempt_start = step_stats.get("start_time") - - for event in events: - if not event.dagster_event: - continue - if event.dagster_event.event_type == DagsterEventType.STEP_UP_FOR_RETRY: - step_attempts.append( - RunStepMarker(start_time=attempt_start, end_time=event.timestamp) - ) - elif event.dagster_event.event_type == DagsterEventType.STEP_RESTARTED: - attempt_start = event.timestamp - if step_stats.get("end_time"): - step_attempts.append( - RunStepMarker(start_time=attempt_start, end_time=step_stats["end_time"]) + snapshots.append( + RunStepKeyStatsSnapshot( + run_id=run_id, + step_key=step_key, + **step_stats, + markers=[ + RunStepMarker( + start_time=marker.get("start"), + end_time=marker.get("end"), + key=marker.get("key"), + ) + for marker in markers[step_key].values() + ], + attempts_list=attempts[step_key], ) - else: - by_step_key[step_key]["status"] = StepEventStatus.IN_PROGRESS - attempts[step_key] = step_attempts - - return [ - RunStepKeyStatsSnapshot( - run_id=run_id, - step_key=step_key, - attempts_list=attempts[step_key], - markers=[ - RunStepMarker(start_time=marker.get("start"), end_time=marker.get("end")) - for marker in markers[step_key].values() - ], - **value, - ) - for step_key, value in by_step_key.items() - ] - - -@whitelist_for_serdes -class RunStepMarker( - NamedTuple( - "_RunStepMarker", - [("start_time", Optional[float]), ("end_time", Optional[float])], - ) -): - def __new__( - cls, - start_time: Optional[float] = None, - end_time: Optional[float] = None, - ): - return super(RunStepMarker, cls).__new__( - cls, - start_time=check.opt_float_param(start_time, "start_time"), - end_time=check.opt_float_param(end_time, "end_time"), ) - -@whitelist_for_serdes -class RunStepKeyStatsSnapshot( - NamedTuple( - "_RunStepKeyStatsSnapshot", - [ - ("run_id", str), - ("step_key", str), - ("status", Optional[StepEventStatus]), - ("start_time", Optional[float]), - ("end_time", Optional[float]), - ("materialization_events", Sequence[EventLogEntry]), - ("expectation_results", Sequence[ExpectationResult]), - ("attempts", Optional[int]), - ("attempts_list", Sequence[RunStepMarker]), - ("markers", Sequence[RunStepMarker]), - ], + return RunStepStatsSnapshot( + run_id=run_id, + step_key_stats=snapshots, + partial_markers={ + step_key: [ + RunStepMarker(start_time=marker.get("start"), end_time=marker.get("end"), key=key) + for key, marker in markers.items() + ] + for step_key, markers in markers.items() + if step_key not in by_step_key + }, ) -): - def __new__( - cls, - run_id: str, - step_key: str, - status: Optional[StepEventStatus] = None, - start_time: Optional[float] = None, - end_time: Optional[float] = None, - materialization_events: Optional[Sequence[EventLogEntry]] = None, - expectation_results: Optional[Sequence[ExpectationResult]] = None, - attempts: Optional[int] = None, - attempts_list: Optional[Sequence[RunStepMarker]] = None, - markers: Optional[Sequence[RunStepMarker]] = None, - ): - return super(RunStepKeyStatsSnapshot, cls).__new__( - cls, - run_id=check.str_param(run_id, "run_id"), - step_key=check.str_param(step_key, "step_key"), - status=check.opt_inst_param(status, "status", StepEventStatus), - start_time=check.opt_float_param(start_time, "start_time"), - end_time=check.opt_float_param(end_time, "end_time"), - materialization_events=check.opt_sequence_param( - materialization_events, - "materialization_events", - EventLogEntry, - ), - expectation_results=check.opt_sequence_param( - expectation_results, "expectation_results", ExpectationResult - ), - attempts=check.opt_int_param(attempts, "attempts"), - attempts_list=check.opt_sequence_param(attempts_list, "attempts_list", RunStepMarker), - markers=check.opt_sequence_param(markers, "markers", RunStepMarker), - ) diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py index 8f1d0cc194756..8db0dc8f8b240 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_event_log.py @@ -9,9 +9,14 @@ import pytest import sqlalchemy import sqlalchemy as db -from dagster import AssetKey, AssetMaterialization, DagsterInstance, Output, op +from dagster import AssetKey, AssetMaterialization, DagsterInstance, Out, Output, RetryRequested, op from dagster._core.errors import DagsterEventLogInvalidForRun -from dagster._core.execution.stats import build_run_stats_from_events +from dagster._core.execution.stats import ( + StepEventStatus, + build_run_stats_from_events, + build_run_step_stats_from_events, + build_run_step_stats_snapshot_from_events, +) from dagster._core.storage.event_log import ( ConsolidatedSqliteEventLogStorage, SqlEventLogStorageMetadata, @@ -332,3 +337,49 @@ def _ops(): ) assert incremental_run_stats == run_stats + + +def test_step_stats(): + @op + def op_success(_): + return 1 + + @op + def asset_op(_): + yield AssetMaterialization(asset_key=AssetKey("asset_1")) + yield Output(1) + + @op(out=Out(str)) + def op_failure(_): + time.sleep(0.001) + raise RetryRequested(max_retries=3) + + def _ops(): + op_success() + asset_op() + op_failure() + + events, result = _synthesize_events(_ops, check_success=False) + + step_stats = build_run_step_stats_from_events(result.run_id, events) + assert len(step_stats) == 3 + assert len([step for step in step_stats if step.status == StepEventStatus.SUCCESS]) == 2 + assert len([step for step in step_stats if step.status == StepEventStatus.FAILURE]) == 1 + assert all([step.run_id == result.run_id for step in step_stats]) + + op_failure_stats = next( + iter([step for step in step_stats if step.step_key == "op_failure"]), None + ) + assert op_failure_stats + assert op_failure_stats.attempts == 4 + assert len(op_failure_stats.attempts_list) == 4 + + # build up run stats through incremental events + incremental_snapshot = None + for event in events: + incremental_snapshot = build_run_step_stats_snapshot_from_events( + result.run_id, [event], incremental_snapshot + ) + + assert incremental_snapshot + assert incremental_snapshot.step_key_stats == step_stats