From 7325c4f0291d6e69a0088539d9b851efda9acd19 Mon Sep 17 00:00:00 2001 From: rudolfix Date: Tue, 24 Oct 2023 11:52:59 +0200 Subject: [PATCH] arrow example (#707) * sends region name to s3 fsspec client * shows warning in config missing exception when pipeline script not in working folder * counts rows of arrow tables in writers to rotate files properly * adds arrow + connector x example * bumps to mypy 1.6.1 and fixes transformer decorator * explains variant column and other fixes in docs * reuses primary key for index in incremental * removes unix ts autodetect by default, add/remove detects in schema * passes column schema to arrow writer * fixes tests * adds blog post --------- Co-authored-by: Adrian --- .github/workflows/test_doc_snippets.yml | 2 +- dlt/common/configuration/exceptions.py | 10 ++ .../configuration/specs/aws_credentials.py | 7 +- dlt/common/data_writers/buffered.py | 18 ++- dlt/common/data_writers/writers.py | 2 + dlt/common/schema/schema.py | 12 ++ dlt/common/schema/utils.py | 2 +- dlt/destinations/filesystem/filesystem.py | 2 +- dlt/extract/decorators.py | 4 +- dlt/extract/extract.py | 7 +- dlt/extract/incremental/transform.py | 87 +++++++---- .../examples/connector_x_arrow}/__init__.py | 0 docs/website/blog/2023-10-23-arrow-loading.md | 141 ++++++++++++++++++ .../verified-sources/arrow-pandas.md | 4 + .../examples/connector_x_arrow/__init__.py | 0 .../connector_x_arrow/code/__init__.py | 0 .../code/load_arrow-snippets.py | 41 +++++ .../docs/examples/connector_x_arrow/index.md | 75 ++++++++++ .../docs/general-usage/destination-tables.md | 5 + docs/website/docs/general-usage/schema.md | 15 +- docs/website/docs/reference/performance.md | 12 +- .../performance_snippets/toml-snippets.toml | 14 +- docs/website/sidebars.js | 1 + docs/website/static/img/arrow_30x_faster.png | Bin 0 -> 145023 bytes mypy.ini | 2 + poetry.lock | 84 +++++++---- pyproject.toml | 3 +- tests/common/data_writers/__init__.py | 0 .../test_buffered_writer.py | 69 +++++++-- .../test_data_writers.py | 0 .../test_parquet_writer.py | 0 tests/common/schema/test_inference.py | 3 +- tests/extract/test_incremental.py | 9 +- tests/extract/utils.py | 9 +- tests/load/filesystem/test_aws_credentials.py | 13 ++ tests/pipeline/test_arrow_sources.py | 38 ++++- tests/pipeline/test_pipeline.py | 64 +++++++- 37 files changed, 630 insertions(+), 125 deletions(-) rename {tests/common/test_data_writers => docs/examples/connector_x_arrow}/__init__.py (100%) create mode 100644 docs/website/blog/2023-10-23-arrow-loading.md create mode 100644 docs/website/docs/examples/connector_x_arrow/__init__.py create mode 100644 docs/website/docs/examples/connector_x_arrow/code/__init__.py create mode 100644 docs/website/docs/examples/connector_x_arrow/code/load_arrow-snippets.py create mode 100644 docs/website/docs/examples/connector_x_arrow/index.md create mode 100644 docs/website/static/img/arrow_30x_faster.png create mode 100644 tests/common/data_writers/__init__.py rename tests/common/{test_data_writers => data_writers}/test_buffered_writer.py (70%) rename tests/common/{test_data_writers => data_writers}/test_data_writers.py (100%) rename tests/common/{test_data_writers => data_writers}/test_parquet_writer.py (100%) diff --git a/.github/workflows/test_doc_snippets.yml b/.github/workflows/test_doc_snippets.yml index b2a2f241db..e158c2d669 100644 --- a/.github/workflows/test_doc_snippets.yml +++ b/.github/workflows/test_doc_snippets.yml @@ -56,7 +56,7 @@ jobs: - name: Install dependencies # if: steps.cached-poetry-dependencies.outputs.cache-hit != 'true' - run: poetry install --no-interaction -E duckdb -E weaviate --with docs --without airflow + run: poetry install --no-interaction -E duckdb -E weaviate -E parquet --with docs --without airflow - name: Run linter and tests run: make test-and-lint-snippets diff --git a/dlt/common/configuration/exceptions.py b/dlt/common/configuration/exceptions.py index cae666dab1..f019565013 100644 --- a/dlt/common/configuration/exceptions.py +++ b/dlt/common/configuration/exceptions.py @@ -1,6 +1,8 @@ +import os from typing import Any, Mapping, Type, Tuple, NamedTuple, Sequence from dlt.common.exceptions import DltException, TerminalException +from dlt.common.utils import main_module_file_path class LookupTrace(NamedTuple): @@ -48,6 +50,14 @@ def __str__(self) -> str: msg += f'\tfor field "{f}" config providers and keys were tried in following order:\n' for tr in field_traces: msg += f'\t\tIn {tr.provider} key {tr.key} was not found.\n' + # check if entry point is run with path. this is common problem so warn the user + main_path = main_module_file_path() + main_dir = os.path.dirname(main_path) + abs_main_dir = os.path.abspath(main_dir) + if abs_main_dir != os.getcwd(): + # directory was specified + msg += "WARNING: dlt looks for .dlt folder in your current working directory and your cwd (%s) is different from directory of your pipeline script (%s).\n" % (os.getcwd(), abs_main_dir) + msg += "If you keep your secret files in the same folder as your pipeline script but run your script from some other folder, secrets/configs will not be found\n" msg += "Please refer to https://dlthub.com/docs/general-usage/credentials for more information\n" return msg diff --git a/dlt/common/configuration/specs/aws_credentials.py b/dlt/common/configuration/specs/aws_credentials.py index 6ba661ae88..8c4aabc4ee 100644 --- a/dlt/common/configuration/specs/aws_credentials.py +++ b/dlt/common/configuration/specs/aws_credentials.py @@ -1,7 +1,7 @@ from typing import Optional, Dict, Any from dlt.common.exceptions import MissingDependencyException -from dlt.common.typing import TSecretStrValue +from dlt.common.typing import TSecretStrValue, DictStrAny from dlt.common.configuration.specs import CredentialsConfiguration, CredentialsWithDefault, configspec from dlt.common.configuration.specs.exceptions import InvalidBoto3Session from dlt import version @@ -19,13 +19,16 @@ class AwsCredentialsWithoutDefaults(CredentialsConfiguration): def to_s3fs_credentials(self) -> Dict[str, Optional[str]]: """Dict of keyword arguments that can be passed to s3fs""" - return dict( + credentials: DictStrAny = dict( key=self.aws_access_key_id, secret=self.aws_secret_access_key, token=self.aws_session_token, profile=self.profile_name, endpoint_url=self.endpoint_url, ) + if self.region_name: + credentials["client_kwargs"] = {"region_name": self.region_name} + return credentials def to_native_representation(self) -> Dict[str, Optional[str]]: """Return a dict that can be passed as kwargs to boto3 session""" diff --git a/dlt/common/data_writers/buffered.py b/dlt/common/data_writers/buffered.py index 98f1b51bee..5c93e22bc6 100644 --- a/dlt/common/data_writers/buffered.py +++ b/dlt/common/data_writers/buffered.py @@ -1,4 +1,5 @@ import gzip +from functools import reduce from typing import List, IO, Any, Optional, Type, TypeVar, Generic from dlt.common.utils import uniq_id @@ -58,6 +59,7 @@ def __init__( self._current_columns: TTableSchemaColumns = None self._file_name: str = None self._buffered_items: List[TDataItem] = [] + self._buffered_items_count: int = 0 self._writer: TWriter = None self._file: IO[Any] = None self._closed = False @@ -79,10 +81,20 @@ def write_data_item(self, item: TDataItems, columns: TTableSchemaColumns) -> Non if isinstance(item, List): # items coming in single list will be written together, not matter how many are there self._buffered_items.extend(item) + # update row count, if item supports "num_rows" it will be used to count items + if len(item) > 0 and hasattr(item[0], "num_rows"): + self._buffered_items_count += sum(tbl.num_rows for tbl in item) + else: + self._buffered_items_count += len(item) else: self._buffered_items.append(item) + # update row count, if item supports "num_rows" it will be used to count items + if hasattr(item, "num_rows"): + self._buffered_items_count += item.num_rows + else: + self._buffered_items_count += 1 # flush if max buffer exceeded - if len(self._buffered_items) >= self.buffer_max_items: + if self._buffered_items_count >= self.buffer_max_items: self._flush_items() # rotate the file if max_bytes exceeded if self._file: @@ -118,7 +130,7 @@ def _rotate_file(self) -> None: self._file_name = self.file_name_template % uniq_id(5) + "." + self._file_format_spec.file_extension def _flush_items(self, allow_empty_file: bool = False) -> None: - if len(self._buffered_items) > 0 or allow_empty_file: + if self._buffered_items_count > 0 or allow_empty_file: # we only open a writer when there are any items in the buffer and first flush is requested if not self._writer: # create new writer and write header @@ -131,7 +143,9 @@ def _flush_items(self, allow_empty_file: bool = False) -> None: # write buffer if self._buffered_items: self._writer.write_data(self._buffered_items) + # reset buffer and counter self._buffered_items.clear() + self._buffered_items_count = 0 def _flush_and_close_file(self) -> None: # if any buffered items exist, flush them diff --git a/dlt/common/data_writers/writers.py b/dlt/common/data_writers/writers.py index 73e13ec46f..daf1dbbe17 100644 --- a/dlt/common/data_writers/writers.py +++ b/dlt/common/data_writers/writers.py @@ -274,6 +274,8 @@ def write_data(self, rows: Sequence[Any]) -> None: self.writer.write_batch(row) else: raise ValueError(f"Unsupported type {type(row)}") + # count rows that got written + self.items_count += row.num_rows @classmethod def data_format(cls) -> TFileFormatSpec: diff --git a/dlt/common/schema/schema.py b/dlt/common/schema/schema.py index aef2032261..77a5ae8e8e 100644 --- a/dlt/common/schema/schema.py +++ b/dlt/common/schema/schema.py @@ -393,6 +393,18 @@ def update_normalizers(self) -> None: normalizers["json"] = normalizers["json"] or self._normalizers_config["json"] self._configure_normalizers(normalizers) + def add_type_detection(self, detection: TTypeDetections) -> None: + """Add type auto detection to the schema.""" + if detection not in self.settings["detections"]: + self.settings["detections"].append(detection) + self._compile_settings() + + def remove_type_detection(self, detection: TTypeDetections) -> None: + """Adds type auto detection to the schema.""" + if detection in self.settings["detections"]: + self.settings["detections"].remove(detection) + self._compile_settings() + def _infer_column(self, k: str, v: Any, data_type: TDataType = None, is_variant: bool = False) -> TColumnSchema: column_schema = TColumnSchema( name=k, diff --git a/dlt/common/schema/utils.py b/dlt/common/schema/utils.py index 32bd4ade1c..f2075ce85d 100644 --- a/dlt/common/schema/utils.py +++ b/dlt/common/schema/utils.py @@ -698,4 +698,4 @@ def standard_hints() -> Dict[TColumnHint, List[TSimpleRegex]]: def standard_type_detections() -> List[TTypeDetections]: - return ["timestamp", "iso_timestamp"] + return ["iso_timestamp"] diff --git a/dlt/destinations/filesystem/filesystem.py b/dlt/destinations/filesystem/filesystem.py index 49ad36dd16..766f384024 100644 --- a/dlt/destinations/filesystem/filesystem.py +++ b/dlt/destinations/filesystem/filesystem.py @@ -131,7 +131,7 @@ def initialize_storage(self, truncate_tables: Iterable[str] = None) -> None: for search_prefix in truncate_prefixes: if item.startswith(search_prefix): # NOTE: deleting in chunks on s3 does not raise on access denied, file non existing and probably other errors - logger.info(f"DEL {item}") + # logger.info(f"DEL {item}") # print(f"DEL {item}") self.fs_client.rm(item) except FileNotFoundError: diff --git a/dlt/extract/decorators.py b/dlt/extract/decorators.py index ec3f2bb47b..eb8a08e3d1 100644 --- a/dlt/extract/decorators.py +++ b/dlt/extract/decorators.py @@ -384,7 +384,7 @@ def decorator(f: Callable[TResourceFunParams, Any]) -> Callable[TResourceFunPara spec=spec, sections=resource_sections, sections_merge_style=ConfigSectionContext.resource_merge_style, include_defaults=spec is not None ) is_inner_resource = is_inner_callable(f) - if conf_f != incr_f and is_inner_resource: + if conf_f != incr_f and is_inner_resource and not standalone: raise ResourceInnerCallableConfigWrapDisallowed(resource_name, source_section) # get spec for wrapped function SPEC = get_fun_spec(conf_f) @@ -494,7 +494,7 @@ def transformer( selected: bool = True, spec: Type[BaseConfiguration] = None, standalone: Literal[True] = True -) -> Callable[..., DltResource]: # TODO: change back to Callable[TResourceFunParams, DltResource] when mypy 1.6 is fixed +) -> Callable[TResourceFunParams, DltResource]: # TODO: change back to Callable[TResourceFunParams, DltResource] when mypy 1.6 is fixed ... def transformer( diff --git a/dlt/extract/extract.py b/dlt/extract/extract.py index 7f8142e807..fdca26fcfe 100644 --- a/dlt/extract/extract.py +++ b/dlt/extract/extract.py @@ -164,14 +164,14 @@ def write_empty_file(self, table_name: str) -> None: table_name = self.schema.naming.normalize_table_identifier(table_name) self.storage.write_empty_file(self.extract_id, self.schema.name, table_name, None) - def _write_item(self, table_name: str, resource_name: str, items: TDataItems) -> None: + def _write_item(self, table_name: str, resource_name: str, items: TDataItems, columns: TTableSchemaColumns = None) -> None: # normalize table name before writing so the name match the name in schema # note: normalize function should be cached so there's almost no penalty on frequent calling # note: column schema is not required for jsonl writer used here table_name = self.schema.naming.normalize_identifier(table_name) self.collector.update(table_name) self.resources_with_items.add(resource_name) - self.storage.write_data_item(self.extract_id, self.schema.name, table_name, items, None) + self.storage.write_data_item(self.extract_id, self.schema.name, table_name, items, columns) def _write_dynamic_table(self, resource: DltResource, item: TDataItem) -> None: table_name = resource._table_name_hint_fun(item) @@ -212,6 +212,9 @@ def write_table(self, resource: DltResource, items: TDataItems, meta: Any) -> No ] super().write_table(resource, items, meta) + def _write_item(self, table_name: str, resource_name: str, items: TDataItems, columns: TTableSchemaColumns = None) -> None: + super()._write_item(table_name, resource_name, items, self.dynamic_tables[table_name][0]["columns"]) + def _write_static_table(self, resource: DltResource, table_name: str, items: TDataItems) -> None: existing_table = self.dynamic_tables.get(table_name) if existing_table is not None: diff --git a/dlt/extract/incremental/transform.py b/dlt/extract/incremental/transform.py index c77fb97b9f..b1dffe6b28 100644 --- a/dlt/extract/incremental/transform.py +++ b/dlt/extract/incremental/transform.py @@ -1,5 +1,5 @@ -from datetime import datetime # noqa: I251 -from typing import Optional, Tuple, Protocol, Mapping, Union, List +from datetime import datetime, date # noqa: I251 +from typing import Optional, Tuple, List try: import pandas as pd @@ -137,8 +137,9 @@ def __call__( return row, start_out_of_range, end_out_of_range - class ArrowIncremental(IncrementalTransformer): + _dlt_index = "_dlt_index" + def unique_values( self, item: "TAnyArrowItem", @@ -148,28 +149,34 @@ def unique_values( if not unique_columns: return [] item = item - indices = item["_dlt_index"].to_pylist() + indices = item[self._dlt_index].to_pylist() rows = item.select(unique_columns).to_pylist() return [ (index, digest128(json.dumps(row, sort_keys=True))) for index, row in zip(indices, rows) ] - def _deduplicate(self, tbl: "pa.Table", unique_columns: Optional[List[str]], aggregate: str, cursor_path: str) -> "pa.Table": - if unique_columns is None: - return tbl - group_cols = unique_columns + [cursor_path] - tbl = tbl.append_column("_dlt_index", pa.array(np.arange(tbl.num_rows))) - try: - tbl = tbl.filter( - pa.compute.is_in( - tbl['_dlt_index'], - tbl.group_by(group_cols).aggregate( - [("_dlt_index", "one"), (cursor_path, aggregate)] - )['_dlt_index_one'] - ) - ) - except KeyError as e: - raise IncrementalPrimaryKeyMissing(self.resource_name, unique_columns[0], tbl) from e + def _deduplicate(self, tbl: "pa.Table", unique_columns: Optional[List[str]], aggregate: str, cursor_path: str) -> "pa.Table": + """Creates unique index if necessary.""" + # create unique index if necessary + if self._dlt_index not in tbl.schema.names: + tbl = tbl.append_column(self._dlt_index, pa.array(np.arange(tbl.num_rows))) + # code below deduplicates groups that include the cursor column in the group id. that was just artifact of + # json incremental and there's no need to duplicate it here + + # if unique_columns is None: + # return tbl + # group_cols = unique_columns + [cursor_path] + # try: + # tbl = tbl.filter( + # pa.compute.is_in( + # tbl[self._dlt_index], + # tbl.group_by(group_cols).aggregate( + # [(self._dlt_index, "one"), (cursor_path, aggregate)] + # )[f'{self._dlt_index}_one'] + # ) + # ) + # except KeyError as e: + # raise IncrementalPrimaryKeyMissing(self.resource_name, unique_columns[0], tbl) from e return tbl def __call__( @@ -180,6 +187,25 @@ def __call__( if is_pandas: tbl = pa.Table.from_pandas(tbl) + primary_key = self.primary_key(tbl) if callable(self.primary_key) else self.primary_key + if primary_key: + # create a list of unique columns + if isinstance(primary_key, str): + unique_columns = [primary_key] + else: + unique_columns = list(primary_key) + # check if primary key components are in the table + for pk in unique_columns: + if pk not in tbl.schema.names: + raise IncrementalPrimaryKeyMissing(self.resource_name, pk, tbl) + # use primary key as unique index + if isinstance(primary_key, str): + self._dlt_index = primary_key + elif primary_key is None: + unique_columns = tbl.column_names + else: # deduplicating is disabled + unique_columns = None + start_out_of_range = end_out_of_range = False if not tbl: # row is None or empty arrow table return tbl, start_out_of_range, end_out_of_range @@ -206,24 +232,19 @@ def __call__( cursor_path = str(self.cursor_path) # The new max/min value try: - row_value = compute(tbl[cursor_path]).as_py() + orig_row_value = compute(tbl[cursor_path]) + row_value = orig_row_value.as_py() + # dates are not represented as datetimes but I see connector-x represents + # datetimes as dates and keeping the exact time inside. probably a bug + # but can be corrected this way + if isinstance(row_value, date) and not isinstance(row_value, datetime): + row_value = pendulum.from_timestamp(orig_row_value.cast(pa.int64()).as_py() / 1000) except KeyError as e: raise IncrementalCursorPathMissing( self.resource_name, cursor_path, tbl, f"Column name {str(cursor_path)} was not found in the arrow table. Note nested JSON paths are not supported for arrow tables and dataframes, the incremental cursor_path must be a column name." ) from e - primary_key = self.primary_key(tbl) if callable(self.primary_key) else self.primary_key - if primary_key: - if isinstance(primary_key, str): - unique_columns = [primary_key] - else: - unique_columns = list(primary_key) - elif primary_key is None: - unique_columns = tbl.column_names - else: # deduplicating is disabled - unique_columns = None - # If end_value is provided, filter to include table rows that are "less" than end_value if self.end_value is not None: tbl = tbl.filter(end_compare(tbl[cursor_path], self.end_value)) @@ -247,7 +268,7 @@ def __call__( unique_values = [(i, uq_val) for i, uq_val in unique_values if uq_val in self.incremental_state['unique_hashes']] remove_idx = pa.array(i for i, _ in unique_values) # Filter the table - tbl = tbl.filter(pa.compute.invert(pa.compute.is_in(tbl["_dlt_index"], remove_idx))) + tbl = tbl.filter(pa.compute.invert(pa.compute.is_in(tbl[self._dlt_index], remove_idx))) if new_value_compare(row_value, last_value).as_py() and row_value != last_value: # Last value has changed self.incremental_state['last_value'] = row_value diff --git a/tests/common/test_data_writers/__init__.py b/docs/examples/connector_x_arrow/__init__.py similarity index 100% rename from tests/common/test_data_writers/__init__.py rename to docs/examples/connector_x_arrow/__init__.py diff --git a/docs/website/blog/2023-10-23-arrow-loading.md b/docs/website/blog/2023-10-23-arrow-loading.md new file mode 100644 index 0000000000..80e6744df5 --- /dev/null +++ b/docs/website/blog/2023-10-23-arrow-loading.md @@ -0,0 +1,141 @@ +--- +slug: dlt-arrow-loading +title: "Get 30x speedups when reading databases with ConnectorX + Arrow + dlt" +image: /img/arrow_30x_faster.png +authors: + name: Marcin Rudolf + title: dltHub CTO + url: https://github.com/rudolfix + image_url: https://avatars.githubusercontent.com/u/17202864 +tags: [arrow, Rust, ConnectorX] +--- + +If rust + arrow + duckb is a new data engineering stack, now you can get a feel of it with `dlt`. We recently added native arrow tables (and panda frames) loading. What it means? You can pass an Arrow table to `dlt` **pipeline.run** or **pipeline.extract** methods, have it normalized, saved to parquet and loaded to your destination. + +Here we achieved ~30x speedups when loading data from (local) postgres database using ConnectorX + Arrow compared to SqlAlchemy + json. (both use dlt as an engine, read disclaimer at the end!) + +### Load postgres table with Arrow + +We’ll start with [ConnectorX library](https://github.com/sfu-db/connector-x) that creates Arrow tables from SQL queries on most of the popular database engines. + +```python +pip install connectorx +``` + +Lib has Rust inside, zero copy extraction and is amazingly fast. We’ll extract and normalize 10 000 000 [test rows](https://github.com/dlt-hub/verified-sources/blob/master/tests/sql_database/sql_source.py#L88) from local postgresql. The table **chat_message** looks like Slack messages dump. Messages have unique autoincrement **id** which we use to load in chunks: + +```python +import connectorx as cx +import dlt +from dlt.sources.credentials import ConnectionStringCredentials + +def read_sql_x( + conn_str: str +): + # load in chunks by one million + for _id in range(1, 10_000_001, 1_000_000): + table = cx.read_sql(conn_str, + "SELECT * FROM arrow_test_2.chat_message WHERE id BETWEEN %i AND %i" % (_id, _id + 1000000 - 1), + return_type="arrow2", + protocol="binary" + ) + yield table + +chat_messages = dlt.resource( + read_sql_x, + name="chat_messages" +)("postgresql://loader:loader@localhost:5432/dlt_data") +``` + +In this demo I just extract and normalize data and skip the loading step. + +```python +pipeline = dlt.pipeline(destination="duckdb", full_refresh=True) +# extract first +pipeline.extract(chat_messages) +info = pipeline.normalize() +# print count of items normalized +print(info) +# print the execution trace +print(pipeline.last_trace) +``` + +Let’s run it: + +```sh +$ PROGRESS=enlighten python connector_x_speed.py +Items 10000001 [00:00, 241940483.70/s] +Normalized data for the following tables: +- _dlt_pipeline_state: 1 row(s) +- chat_messages: 10000000 row(s) + +Run started at 2023-10-23T19:06:55.527176+00:00 and COMPLETED in 16.17 seconds with 2 steps. +Step extract COMPLETED in 16.09 seconds. + +Step normalize COMPLETED in 0.08 seconds. +``` +### Load postgres table with SqlAlchemy + +Here’s corresponding code working with **SqlAlchemy**. We process 10 000 000 rows, yielding in 100k rows packs and normalize to parquet in 3 parallel processes. + +```python +from itertools import islice +import dlt +from sqlalchemy import create_engine + +CHUNK_SIZE=100000 + +def read_sql_a(conn_str: str): + engine = create_engine(conn_str) + with engine.connect() as conn: + rows = conn.execution_options(yield_per=CHUNK_SIZE).exec_driver_sql("SELECT * FROM arrow_test_2.chat_message") + while rows_slice := list(islice(map(lambda row: dict(row._mapping), rows), CHUNK_SIZE)): + yield rows_slice + +chat_messages = dlt.resource( + read_sql_a, + name="chat_messages", + write_disposition="append", +)("postgresql://loader:loader@localhost:5432/dlt_data") + +pipeline = dlt.pipeline(destination="duckdb", full_refresh=True) +# extract first +pipeline.extract(chat_messages) +info = pipeline.normalize(workers=3, loader_file_format="parquet") +print(info) +print(pipeline.last_trace) +``` + +Let’s run it: + +```sh +$ PROGRESS=enlighten python sql_alchemy_speed.py +Normalized data for the following tables: +- _dlt_pipeline_state: 1 row(s) +- chat_messages: 10000000 row(s) + +Run started at 2023-10-23T19:13:55.898598+00:00 and COMPLETED in 8 minutes and 12.97 seconds with 2 steps. +Step extract COMPLETED in 3 minutes and 32.75 seconds. + +Step normalize COMPLETED in 3 minutes and 40.22 seconds. +Normalized data for the following tables: +- _dlt_pipeline_state: 1 row(s) +- chat_messages: 10000000 row(s) +``` + +### Results + +So we can see **~30x overall speedup on extract and normalize steps** (~16 seconds vs ~8 minutes). The **extract step is ~13x faster**, while **normalize is few thousand times faster**. Arrow normalizer is just checking the schemas and moves parquet files around. JSON normalizer is inspecting every row to first infer the schema and then to validate the data. + +As the output in both of methods is the same (parquet files) - the actual load step takes the same time in both cases and is not compared. I could easily push the load packages (parquet files) to [any of supported destinations](https://dlthub.com/docs/dlt-ecosystem/verified-sources/arrow-pandas#destinations-that-support-parquet-for-direct-loading) + +### What’s next: +- [Reads our docs on Arrow](https://dlthub.com/docs/dlt-ecosystem/verified-sources/arrow-pandas) +- [Add merge and incremental loads to code above](https://dlthub.com/docs/examples/connector_x_arrow/) +- I'm on [dltHub Slack](https://join.slack.com/t/dlthub-community/shared_invite/zt-1slox199h-HAE7EQoXmstkP_bTqal65g) all the time. + +### Disclaimers + +- Playing field is not level. classical (sql alchemy) `dlt` run is processing data row by row, inferring and validating schema. that’s why it so slow. The Arrow version benefits from the fact, that data is already structured in the source. +- We load from local database. That means that network roundtrip during extraction is not included. That isolates Arrow speedups well. In case of remote database engine, the speedups will be smaller. +- You could optimize extract (both classical and arrow) by reading data from **postgres** [in parallel](https://dlthub.com/docs/examples/transformers/#using-transformers-with-the-pokemon-api) or use partitions in ConnectorX \ No newline at end of file diff --git a/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md b/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md index 5840fb3049..85796b0333 100644 --- a/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md +++ b/docs/website/docs/dlt-ecosystem/verified-sources/arrow-pandas.md @@ -85,6 +85,10 @@ pipeline.run(orders) pipeline.run(orders) ``` +:::tip +Look at the [Connector X + Arrow Example](../../examples/connector_x_arrow/) to see how to load data from production databases fast. +::: + ## Supported Arrow data types The Arrow data types are translated to dlt data types as follows: diff --git a/docs/website/docs/examples/connector_x_arrow/__init__.py b/docs/website/docs/examples/connector_x_arrow/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/docs/website/docs/examples/connector_x_arrow/code/__init__.py b/docs/website/docs/examples/connector_x_arrow/code/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/docs/website/docs/examples/connector_x_arrow/code/load_arrow-snippets.py b/docs/website/docs/examples/connector_x_arrow/code/load_arrow-snippets.py new file mode 100644 index 0000000000..488cb8a7cb --- /dev/null +++ b/docs/website/docs/examples/connector_x_arrow/code/load_arrow-snippets.py @@ -0,0 +1,41 @@ +def connector_x_snippet() -> None: + + # @@@DLT_SNIPPET_START markdown_source + + import connectorx as cx + + import dlt + from dlt.sources.credentials import ConnectionStringCredentials + + def read_sql_x( + conn_str: ConnectionStringCredentials = dlt.secrets.value, + query: str = dlt.config.value + ): + yield cx.read_sql(conn_str.to_native_representation(), query, return_type="arrow2", protocol="binary") + + # create genome resource with merge on `upid` primary key + genome = dlt.resource( + name="genome", + write_disposition="merge", + primary_key="upid", + standalone=True + )(read_sql_x)( + "mysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam", # type: ignore[arg-type] + "SELECT * FROM genome ORDER BY created LIMIT 1000" + ) + # add incremental on created at + genome.apply_hints(incremental=dlt.sources.incremental("created")) + # @@@DLT_SNIPPET_END markdown_source + # @@@DLT_SNIPPET_START markdown_pipeline + __name__ = "__main__" # @@@DLT_REMOVE + if __name__ == "__main__": + pipeline = dlt.pipeline(destination="duckdb") + print(pipeline.run(genome)) + print(pipeline.last_trace.last_normalize_info) + # NOTE: run pipeline again to see that no more records got loaded thanks to incremental working + # @@@DLT_SNIPPET_END markdown_pipeline + + # check that stuff was loaded + row_counts = pipeline.last_trace.last_normalize_info.row_counts + assert row_counts["genome"] == 1000 + diff --git a/docs/website/docs/examples/connector_x_arrow/index.md b/docs/website/docs/examples/connector_x_arrow/index.md new file mode 100644 index 0000000000..3d978deaaa --- /dev/null +++ b/docs/website/docs/examples/connector_x_arrow/index.md @@ -0,0 +1,75 @@ +--- +title: Load mysql table with ConnectorX & Arrow +description: Load data from sql queries fast with connector x and arrow tables +keywords: [connector x, pyarrow, zero copy] +--- + +import Header from '../_examples-header.md'; + +
+ +## Load mysql table with ConnectorX and Arrow + +Example script below takes genome data from public **mysql** instance and then loads it into **duckdb**. Mind that your destination +must support loading of parquet files as this is the format that `dlt` uses to save arrow tables. [Connector X](https://github.com/sfu-db/connector-x) allows to +get data from several popular databases and creates in memory Arrow table which `dlt` then saves to load package and loads to the destination. +:::tip +You can yield several tables if your data is large and you need to partition your load. +::: + +We'll learn: + +- How to get arrow tables from [connector X](https://github.com/sfu-db/connector-x) and yield them. +- That merge and incremental loads work with arrow tables. +- How to enable [incremental loading](../../general-usage/incremental-loading) for efficient data extraction. +- How to use build in ConnectionString credentials + + + +### Loading code + + +```py +import connectorx as cx + +import dlt +from dlt.sources.credentials import ConnectionStringCredentials + +def read_sql_x( + conn_str: ConnectionStringCredentials = dlt.secrets.value, + query: str = dlt.config.value +): + yield cx.read_sql(conn_str.to_native_representation(), query, return_type="arrow2", protocol="binary") + +# create genome resource with merge on `upid` primary key +genome = dlt.resource( + name="genome", + write_disposition="merge", + primary_key="upid", + standalone=True +)(read_sql_x)( + "mysql://rfamro@mysql-rfam-public.ebi.ac.uk:4497/Rfam", # type: ignore[arg-type] + "SELECT * FROM genome ORDER BY created LIMIT 1000" +) +# add incremental on created at +genome.apply_hints(incremental=dlt.sources.incremental("created")) +``` + + +Run the pipeline: + + +```py +if __name__ == "__main__": + pipeline = dlt.pipeline(destination="duckdb") + print(pipeline.run(genome)) + print(pipeline.last_trace.last_normalize_info) + # NOTE: run pipeline again to see that no more records got loaded thanks to incremental working +``` + diff --git a/docs/website/docs/general-usage/destination-tables.md b/docs/website/docs/general-usage/destination-tables.md index 8f95639f87..8e1f771e47 100644 --- a/docs/website/docs/general-usage/destination-tables.md +++ b/docs/website/docs/general-usage/destination-tables.md @@ -163,6 +163,11 @@ case the primary key or other unique columns are defined. During a pipeline run, dlt [normalizes both table and column names](schema.md#naming-convention) to ensure compatibility with the destination database's accepted format. All names from your source data will be transformed into snake_case and will only include alphanumeric characters. Please be aware that the names in the destination database may differ somewhat from those in your original input. +### Variant columns +If your data has inconsistent types, `dlt` will dispatch the data to several **variant columns**. For example, if you have a resource (ie json file) with a filed with name **answer** and your data contains boolean values, you will get get a column with name **answer** of type **BOOLEAN** in your destination. If for some reason, on next load you get integer value and string value in **answer**, the inconsistent data will go to **answer__v_bigint** and **answer__v_text** columns respectively. +The general naming rule for variant columns is `__v_` where `original_name` is the existing column name (with data type clash) and `type` is the name of data type stored in the variant. + + ## Load Packages and Load IDs Each execution of the pipeline generates one or more load packages. A load package typically contains data retrieved from diff --git a/docs/website/docs/general-usage/schema.md b/docs/website/docs/general-usage/schema.md index e27a87e803..ee73aea54e 100644 --- a/docs/website/docs/general-usage/schema.md +++ b/docs/website/docs/general-usage/schema.md @@ -243,12 +243,12 @@ data itself. The `dlt.source` decorator accepts a schema instance that you can create yourself and modify in whatever way you wish. The decorator also support a few typical use cases: -### 1. Schema created implicitly by decorator +### Schema created implicitly by decorator If no schema instance is passed, the decorator creates a schema with the name set to source name and all the settings to default. -### 2. Automatically load schema file stored with source python module +### Automatically load schema file stored with source python module If no schema instance is passed, and a file with a name `{source name}_schema.yml` exists in the same folder as the module with the decorated function, it will be automatically loaded and used as @@ -256,7 +256,7 @@ the schema. This should make easier to bundle a fully specified (or pre-configured) schema with a source. -### 3. Schema is modified in the source function body +### Schema is modified in the source function body What if you can configure your schema or add some tables only inside your schema function, when i.e. you have the source credentials and user settings available? You could for example add detailed @@ -264,7 +264,7 @@ schemas of all the database tables when someone requests a table data to be load is available only at the moment source function is called. Similarly to the `source_state()` and `resource_state()` , source and resource function has current -schema available via `dlt.current.source_schema`. +schema available via `dlt.current.source_schema()`. Example: @@ -273,9 +273,10 @@ Example: def textual(nesting_level: int): # get the source schema from the `current` context schema = dlt.current.source_schema() - # remove date detector and add type detector that forces all fields to strings - schema._settings["detections"].remove("iso_timestamp") - schema._settings["detections"].insert(0, "all_text") + # remove date detector + schema.remove_type_detection("iso_timestamp") + # convert UNIX timestamp (float, withing a year from NOW) into timestamp + schema.add_type_detection("timestamp") schema.compile_settings() return dlt.resource(...) diff --git a/docs/website/docs/reference/performance.md b/docs/website/docs/reference/performance.md index 20d3d9bb8d..8e9823cbd2 100644 --- a/docs/website/docs/reference/performance.md +++ b/docs/website/docs/reference/performance.md @@ -92,22 +92,22 @@ Below we set files to rotated after 100.000 items written or when the filesize e # extract and normalize stages [data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the extract stage - for all sources [sources.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the extract stage of a source with name zendesk_support [sources.zendesk_support.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the normalize stage [normalize.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 ``` @@ -235,7 +235,7 @@ The **normalize** stage uses a process pool to create load package concurrently. ```toml [extract.data_writer] # force extract file rotation if size exceeds 1MiB -max_file_size=1000000 +file_max_bytes=1000000 [normalize] # use 3 worker processes to process 3 files in parallel @@ -260,7 +260,7 @@ As before, **if you have just a single table with millions of records you should ```toml [normalize.data_writer] # force normalize file rotation if it exceeds 1MiB -max_file_size=1000000 +file_max_bytes=1000000 [load] # have 50 concurrent load jobs diff --git a/docs/website/docs/reference/performance_snippets/toml-snippets.toml b/docs/website/docs/reference/performance_snippets/toml-snippets.toml index db33235eb9..d028e9b145 100644 --- a/docs/website/docs/reference/performance_snippets/toml-snippets.toml +++ b/docs/website/docs/reference/performance_snippets/toml-snippets.toml @@ -21,22 +21,22 @@ buffer_max_items=100 # extract and normalize stages [data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the extract stage - for all sources [sources.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the extract stage of a source with name zendesk_support [sources.zendesk_support.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # only for the normalize stage [normalize.data_writer] file_max_items=100000 -max_file_size=1000000 +file_max_bytes=1000000 # @@@DLT_SNIPPET_END file_size_toml @@ -73,7 +73,7 @@ max_parallel_items=10 # @@@DLT_SNIPPET_START normalize_workers_toml [extract.data_writer] # force extract file rotation if size exceeds 1MiB -max_file_size=1000000 +file_max_bytes=1000000 [normalize] # use 3 worker processes to process 3 files in parallel @@ -84,7 +84,7 @@ workers=3 # @@@DLT_SNIPPET_START normalize_workers_2_toml [normalize.data_writer] # force normalize file rotation if it exceeds 1MiB -max_file_size=1000000 +file_max_bytes=1000000 [load] # have 50 concurrent load jobs @@ -106,7 +106,7 @@ request_max_retry_delay = 30 # Cap exponential delay to 30 seconds next_item_mode="round_robin" [sources.my_pipeline.extract] # setting for the "my_pipeline" pipeline -next_item_mode="fifo" +next_item_mode="fifo" # @@@DLT_SNIPPET_END item_mode_toml diff --git a/docs/website/sidebars.js b/docs/website/sidebars.js index befbfc0b2d..f88ccbdf1a 100644 --- a/docs/website/sidebars.js +++ b/docs/website/sidebars.js @@ -224,6 +224,7 @@ const sidebars = { items: [ 'examples/transformers/index', 'examples/incremental_loading/index', + 'examples/connector_x_arrow/index', ], }, { diff --git a/docs/website/static/img/arrow_30x_faster.png b/docs/website/static/img/arrow_30x_faster.png new file mode 100644 index 0000000000000000000000000000000000000000..f0c06ff5e1073acddc20b861a152ae7dc4ee361c GIT binary patch literal 145023 zcmeFaNv`Y4(k^yw;2FjP-UJN0d4cD~y^vV5$xdD82~(FXnOgpZ??j@E`y7x4-=- zRT1sK{p~;ePk;N{fBPT)`~MD({M-L^`(NSDf797J{%`;L|M@@u-+%kt|Mq|MlGhh= zoIRtz{T&zHKK&irq*d4d9T)$OVfVWJXPDiS_Fi=VETbL{!S6*CW^opUZ|7he4FAV= z^AC)0HuT?d4rk$wB>%}`+uPgu_arLc8$48N_I%#|j>+UdH_+67KHP(>?5i&QJI=#* zwd|uO{Ds5tZk*uFgMZu2zir8XqyK04$kp{TxN!^}R=1=q(ygGGt-Yy|j?d#7UCl#|0JrSFJ(DD5~z)gSt+8@Jg zf2@-KH_b8YX-l^5L=m-8DJd(B=%224DRSP5zqxe2NfrdX?;GS86mP=pN zaP;2QZ_E;BGZG>E>xSINQCKE#N8p0A?-8rP`jO%FVyS=L>wAz!|E$^s4rK*;fBo=} zXO&d{mPYfxO=;Fg|LEub-~1=^?jL^g&$@(F4dUfR0~lBIzDRyS`q!a9p8roS*hL^$ z`gQ?McyAG>-skRp(QR4wpX}HFh__x$pA3zY@7+khFf*|s{z<(bzrU}ej;C`f zofG&pqpydHz+K@lpH@2&9X;H!r)X_?&DtE2v!ac+;@RCSjytJ^>rUbI%sejTb2vRT z(c;pbC6$$9D~&BZ_PV<-g#5-y(xW*DXGKoCCQFLWJNrV~EW*BRC{7pZOegw`COTQZ zxZzfEZTCn?yyAo|P2yt#lR3aYGd>jY-!cAd;O3}U954b;C#RD|HT+o^42ikRvoDpJ za9T2p<cX!PaJA3Ny+-iG8z^nOIb&2eudQyWe*r|!-##>U>ex8nx;b}61 zD~>fHlTsz?hI}+}9t+$#kR3QyoKO7Wq~5PwD?i69sdt5X%(U$TizRzs4>Nz(FjfD%&kvS20#v8({E}$gH&)X&yiLHZaez8>EGPh^d*0zldn*NKsJ~*6Pe%` zbcaqjoccMFf~1%^^2znF#4p*(Z)rW{%DL$g17tR7dCGopQu&b@ZP&#|-g&0L2;{t@ z5ZfrZ>xok>E~KHMQABD`=#DGyGCO<(K`ieFlXnSCoj9LQxbo5MFXH{g>DjTNc66JT zp8Aj#ftPodzgT?6MKdz&Q8JElALm#Rp-vXb@u07YVsCjP(Uohs7gZ3x)W$bE< z35SV%)L|)NlG2IFH86-^dz~;YvvG0P?sSZoWaUYQ&UEv(q<{|xndq{eNudG2`;Fde2d3aRSl6udBJWiCLCBY$JYx!u(Sa6m>nBw+WQc>3- z{#36?ZXVjW7B2kwP>CjW zWy?FB@BJp+RWefxD$7ZsxW8v-QW01#3&rO>f8NuG&u5 zR77daRb0OApI1KfWwMZ-7yfxkLpOC|j`Wx_)$Qnx+t3o_Bu+?Jt-;UctdvBkOwpLE zaVEz(8>i4$S!kid5)fbJ$CdrddQeiJ#E>d~5#Z*2{I(|Ok=}xmb z-%f`_V=;AqxOPNAXNN&8h@4FYUx3|(*T(zlBv|4jF|~ZQMWWW1)8Xkjja?=#gw6EOeh85Jd!fJ1;Q8jxrwu^jwD7bnkKtjVJaY-J3$bb*4?FDz}P;oY%>73ZT-?&j4ZcbxDNOQ|S0y`mdK00v3*mD^@ z=uI9i+TP|jesnnRkF<(s`R?lbPDI_0cI)rd;Fe7@UhvG+=6<@9PIpavcD?^-q{7>W z_!-~YkD>QBh4Ikqlyc@^PM`VFJe1=hKdizU*#*X|_tk8VelR(!9o^{_F|)tf&5W+A zot(CtOKV@TXS2JU$(hzKOhvd@(CVim+u}oW*m`}7#?e10#=D52Z~C#_8{$w16epJD%qdpyEqHubRQJX`$4D39P!pxx?br?L`N%qedk zei-g8s!P{ZxpbkjR&y!rLpiUuhQ7J`&Vvf$Q?t2*yBL?TkRfM$keQHJjY5%gxgzV` zXzu2alF6-}1|8;-Y19{+_rcx8pxnih%AH&AGsLW3;ayDHhS%JoFDA_zk zbYMgaMrd=Ti^*dUu#L8{QcXNoS4I%e=4vgy7;Us*f`0f)cs)5uW(qRn#`Q5@4SBxj z2}=M;S9^;eD)lbeaOQl7G`m)?$5B^nq1xPK`dm;?YUBDAF=WyX?ep8t33B?Pl{QUN zcn+>1{+Q2lWW;lYX~lfRHK3chqNRhBCg?wO!&wO%V7 zEIS*ILepma6jKp&dz0j~*gvz~vb3UXB3+6;d!*wpNcFZ%X2n+|AXJ(h?4w~Oj)C2X z`r+N!(!AttdpXP1gt&5L!h*r>+jTXrU17qGQj9}img;EQWE&rPGL4V)G9C8R99eQ< zjgf$-0inA{?_};_&|RN#5tezf;z`{;1p z+@7Y4A;G)I*TLoKQ+NA0V(}wJ3gL`2mBHkgYJK{rj&X}QI=NHrmCxJ_gof(9do%SH zb4xh=9hjZhy<@+GX#dzgwqZ!ohxW4SQTJLgabqxDzRU@B*==tFI^DJhQ>JWJl*dPt zFU`fJ*LXf|@tdM&q+gBmvsZhCj`n>|-{tn+k6lAG#Y6|*2o11JKtD?4CFO&%t;r@# z^Ii{eI?_VgJ=1#aN z+C;{=OYDcT#dm|}CsFL3f+-~|Is9iXuCC4B!&b7O#+S0*}Ny44RPj$HR z-XtB4xs`JShYypG)SyPAE-}_qE%zN8%9-p;iPY@J7R*geKbWo^JmvHhq!!vcxJmAFPD+oQvRp*4g8jLIRmba- z$d{d;XnwuqMrBp`wCbYk5xW72lY>5aQD$l^oce%&CT_aCL~w{m;24$}S?o{NxO&%B zBakbHxVfozdu)%K@bdJi9Jf>M7AK(}Iq=|c{OdS;*e6`_<@hWl-1PN3@(6Jpdtfy1 zr@*DWW=rxp;*=i06uxo@XVnmbGU@P0N(wvZ+J&XV+X~%+_y$Z6a4ni|Rvd*ve`cjt zCrM;*x7FX?@*;8U{8kADH8jX%*oXFPOgA++1~#WnK`QXjZ|=_4%91306Y9TI9I* zrbPYoB*ah``EGvn9v^R`G$~uYlN6VzT#~tiNSykZe;)En=gG-;I_4;$Xd@`8g*>IypR3;wUsSfUqlVH5^-K>B4bJ~O^3rp zX2@gkpF{kpEH;lW$LmSWJZ3!>2hk8pd&K&!%SL8SuzU6pKdEuUdslfId9`n>$I#>; zk`{U?u~(3NrH5Uay2$uU z5sgyM$i~-$7zYCsMUl9Mq(|*Wv1T@!BhE(7gEdLKvkg3dRFsi3V_p-uwxO{^@mQSE z;kt5xEBAZGET7?806<2=!tDa?(UI`_t1((UaTJJl-H1Y&DP64!8aP-c6_+l2J7B4o zZo}<ez7EDq!4^m3la*r`7%)@t86A3*NzY$Rog~={iR^U-!^l=) zBk*Vdab7sMN*@A8fuUIq5 z@S-Ej!@ni)Rg3uW9b^gEo2+*9!B zxk2s=2`}fBiR&%p?dvU!iyZp#G?qg;ZRzNgEeWS)LFpXxfDfes@!9bTZ^OaQqaZ{U zcc3#tJkE0zi!y58OKWtepmvJ`x=%xEr@Yjb_{kn!&A8VFGSM3yQ8#vRe^YxQxZ(P{ zU97!VJK9FBaWrBTk2Y@ zJHE7?s%Rn_Ar|s5rY}yEg>f}Uh~kq*_t%7IVF}4s<#=tE{g^>mDem16Ed3YHB#EB- zOsqA6{l#iNOwE-m9f+zMP9<`uLfEox>aSbGOn#`lAi>ZKp^5To1!o^Ut;fT9oG#cf z-c?+SwpbL8z=R7K58-L>@v%9X7z@+E8{H_^yfTtGz*G+Ez#VL@whC*25F_j0g& zb$Fqm3XN!IhU;<;Xqvg5WN`7w_~qKLRl6DSP@d2xoNZ*BQ>qrLCF=pYI-k_jX_04r z@N`!L#~eSCTgkWgq9oz!PL5h`_uF&D%*(U7 zQxu^O+;QG6X2sfvPi>MHu4m*4IqfiOmf*O^DV%=WQgoYKGA9hnwNb$je^J%Fz-RMO z>OMU#FeYirDlYt&k?cn=48+aJ$Gai4cbiK-^Zdjw+D7{6 zjlpr_>MS~mc@`@b?5dZKu$R@%$=YzdbTW+FWA{l#pu3bvtfU<#V@h6D<0?c}9AOkd zT(IF5usK^*yX>+{#Lj|vOaCb>KZ5Z+FGcV(3u5$mI#QjhJ!8?Vg0IaW_T zfj%bJz_E11wXfbIE4@Cbl)Ag7$0z@y$qq@?B$`{~*mb%7P_VR8p6y<$4we%n*TlBz zue68xQwxqO)YB~i4<&f5lK3HY4I6GJ)7Hiaun)jScT-d3ZxW4pSV4XX#4i+0#e%I`40=@jr-0|2q z_*UT*^>Gc>D=_y5?P0FKE4+UR4PYH~oCMYZ#V>pbjl&Sx*yfzu2-Yb5dhUS7zQec1 z%*{lRTwcQALjuFHIZOOu8s3G5D8!a|-%>ad8g|6yb5nuH+o3-Ga^~_XxqX9gj|-pi z5Tsb(?(aMgU6`^fCtQ)pTVyycARWg#E$tTfI3)CF9f=INVl3+{{ch$!zCTd0UC{b-}BWotvKP=^QTeY#Bbcx&kn2)@t#6A8Xw&ESSRe1m&4f}5p0FK&ST z`;0YQd6nk_#6Tp3Tj2P3&~8XogH12WoCri8b-=R6T~qG@_nSG4zQey0ERvlfZG@kq zCVK~Y_q0NWO%I7a0X996#>z7*?ZDCi(bu~mE|ZA|B<(L}kNp<^>VpNC&T5NvluzxW zkZg!hNC0v>ypB(Rvckd$M4t?zZ@e*72Adwh+MtiWo;gnW9{&bhklE>aOp;I9CkO7q zB@GfCAo|d82SgvJoqYJ*piM7WNK*0b(@}aH0Y&`l*)#2LaxX233T%7Vmyr9=oU~N7 zENKaAvq-QCnQ+Dqcee}^UK*Qt8eixzzu(`FRCE^zNIrE0r;!g1MuYf4gS>= z@i#i7{2iSMw&!1K?mYD^?lobV{90?=gXsJBwI)ga_3Xjc_Z{9H$lqy<{CBjbzqI|o zwEcgz)c(@;|6M!vzjAFqy-ul>=Xr-hKhB=VkQe+|c!0q9>@d1n zd>qRRGm*bDubW1kVk;KV$Z#@Hc;>>>EIA$*nkjS035V%kV@-bP>3R3;z`vTQN0fHU z%V~T;cZ7fNEsmc=ycN4syTt;W4;&g18(pu7d+;5~Gk#$yO1wOTAU3z#Aztu!VawYb zKYOzdLVc5YHbylM7vDEk%ocmb07o-_ebQT+=Woi)bgj6>XT}6<8{AU%Xi}iK=ZU;c z;i+6zKk87ws-95|MMR%Y^>DnWJbk&*;~n{hj`@u}00L_wA)y0*o@Gdg7)n|xHcW%- zfzcw5<3I%CJ!^%D23Iv>I*6|NhHfM(KrMQ-7e%W_29Uq9D&Q98JrZEm3o~}{(sf?D zpiEWXYUdRum#-5&U)69)h0toH3cOqSlYz+&8a&8`taHXB?U1LJJ4P#7A?tGE5(zT= zE-&BcmhR7P$3!xZT%^!Hp57YA){DfB-A_Si$I>v;76oU?sWOeC^yFRi#%FNg4sG8_a!$6+gwvYds21fnWlzQzaZ-$Y0zZc~*cVuh5MCO|p%1FQ~W;p!9?T1+19DF4a_#FY> zTOtYukJ+Zyt(Q#9506~#Zo%s7l2N*T?GJpedGIuxc%?8-alP~r@?BNrF_9qQ z&D=2Lp(YEoa>bW7GFP5^5w#^ea(&jMpS%;ICHRS`TI^0PAqfbxlW9IA!M^9%(Xe3p z`3SGOEKz*D+?=?F0W}SpeUXEi>*R;vZHOP1wv@d5j#)06#`)=ugMSXON#MnXr;Q+W z8QO>0sV^R9i>r9BOirm%8YjP9jxPy&g;8#}=fi~kfG2~V zF35#{dEgxJxqr0<1mA5;e>tN7!KT)D6&3nG_#EiHq;B+*)`Zfy7)%!d3Sp!<hV+bi5G!f z3;*DPT%hyi=NeZ)Sp&0q?%YS zug5Jf%e~clxY-yh5Ubw)SV3VQC)CTcKi^LIt0huenCu2=g&$Oh8GzMGaFk^7>21<3 zKrHF#5F*!F`fc49Ly2VAU9FB7qI*J$T`v_$T@m1y+Du9YSotrlsp^FTUuZocV!=LJ zJk7X+QBvILZ3UrybPY)E6)~xST(6e2lYPV8Cgr2kW;WxJ5E;AQ*7N<9awasv@+O>8}rAEPKjV zD;`@&R!TIn@1-WI0bid<#0nX}D2DJRnFXF#)tB$2n%4EHxinpDm9T>|sE-wdwxKDw zK|B{l2`b2%DWI(&yG8b~`0DKfe~Sa_M^GM9S8=Vk9K;<71amvNLwk+li73hbcs}pw z8&H3`rq~5i{iIC@J}&<%&bzPk2wbt}z@vJuY;t`R6q8@dv4C+NIqhu?=@B^u5QKu* z%XuPC57Z7hycJUCXU<1EQUYAM5CVnN$4oFJ8OaezkN? zZqe;fe9|7e2rD>1B!glu!~u~^*l8Ovalu1OF0!{4?D27y=r`(H=M;kkCF{HiAQE=H z=C3H`$4qj`L(-8S=6t;Y!5R{_=W@hgoFPPuUh7DYNh8Q!5r;EDo=`jXp;e0&(h*+d zf`(c*rs2tUXjPLgW&m&_P@bdq^6Cav;RfYQc?N?d}Hn%8s2( z9xS~CTL}ETx(Rjd3D@GAW;~LbbS4-yyniis&+zbH%>eu@9qKIA0@c0Qc8$fF}hU><^f`+cHv` z)Ejp*v(hDb!X6!$3Xsgd$vS2#O$=#ARLLwRqynEWvCD-DM88IV80*K=5!oGRdq3+H zdJ5YcrecKM;ai=PZ(YwhSjefno_B8^uq(Zf@~yvm`Lh#m^(N80c*zg-#5SAgz93`H z%(?u$J(ib*mAFP#2oqTdExYN-hMJJo0Js`jac0^xp+R`Lb4a&vJ+!0n@e+j% zAlk0XCG!Nl)d<5xN;|x89i{)ttm~C4dGp}bPK@ruBH$Ff*BP?##0Eem;l-W2%W2#V zZ~;|tgK!b>6MU93#*F}rgDc3$m6@kPZV@%%)j2CVguXS;*ryjPipG_AmC^{jo|sMn ze#GlI?NjzRj!OqN7rJiom~k7Aki`sN1|?pKq95u&wrEU5wr@+VXr!0G#~NJ0$?_3- zJkv7`?3+HY8rZAAda~OxP2Lg1G9X4w#z*CL2Z0-04E4gP?Dz_(?jkxgd26 zkp;1Sq^3TI zq&UVnDJ;=ylL0I;xVoiUfP0Y_+V)3SCr}Rq?M^nphT;q@#nuDkvyAKk&W-7e9JlbH z>El8NMN!6SL=qd$tJ#Zj86oug>>|Y^Z4YZ9+0JDKT63cDPMOxl-#o)Dn%aW{l>}O* z(-D=?J#d=5xY!L7d23N)C|4J52WKMod|)c1hX~lSE?gLo`xb`MDkK5yi_#Y1API{@ zsLU;cn=3Nv@ln#@m-v{S$HTroK*qgl{zH{rn?&ap$h>(jP+ky|sHgs&FAfSJJK0-&Y!;(h-m0hqX+laxs+g$;K z*m|`BQ2g>ByegE8R@DnSh!}-E>*;_f7}0Ea#PbUU)7`2{lOcXNxSP0jHn3MI$MB8L zaB5#V>^&?-$5;>z0LvMS;$p*o0N)n8!ST&&y&fm~;zKL;ZDox%x-&tT9U8=VtlOIN zr7Xjde-d-zUIl-?W&K?BSe|bcUA~tzx?LwT2G#BZ+YKAI3~C~M>WG5n9>8APpLieU zL+)9dQrRw;)CBZ3h0;KRwsS_JWq;du7g%6mA>Gkz%c_Tb5~jYxyXhzYN~}Zl7?hjE znbVHoqU^94l*|Qh4j^`tnjxEU!f)>CqIHe0j0y7wyBV*%V#si;&~EE|Y2z*ADUP&$ z0>t1UP8#YO?^s8<*)Ign+a~{z0M@gy%~nVAkU(xaEGOBCV=%YwF~&7e)qwVS(3L!>PXPzeQA`A;a6^o8)CJMh#205{Vp z{#>l5mklo5G(sZwUr+&IU3^6aP->r>QA{Z+Qg78`_eDoJEdrRD3-8aMZnSXo%_RXH zT^F7oIFtUdBn1WFx(}SezZ0vyt(gj#w!_)EDO?#kKX6vDihbg&B-xgkfTBx*+=v@4 ze15SV1j{*drY*rxrPv}Rs9JQ_@{M(0tO)HQHXs;3Fes95{=lI8Q4C)x?lH&i#E!mc z5R<-V7I=V;iTwowCzaAC1`gu6%64vzFem}v#$%TM>B$1;Bf|0>ADP+L3}Ci7*$6mlLES~tp;szGWbD268H__72RM}enEH{nSNcj9vtHr z)yzcyt3}u)pvFg;HXwVCrw&l+i?Wn!g-ek9s?uuu13jQYS)(85fj@}Af{_8jh^K<% z6{D{5RYlW+c#{Ti-Uk9td-Ef{Bk%x@pO2S-!VieypmvbD`NVWA!aV^w;Lj3(b!Q4O z+|dD?)iZg=a)!y0T&Spoh{^}1?+>w2^_spI?(t%{z`1^+b6~-J!MB1hlnw4{Db$R( zvBJmhg^+!-GcEB7h;BeN#Q`wfhPadC#@a7-gpiK*#{Ply+&G6HNY6itpefDu&+snt z3C}*1WW_gNlyN@nhoa*U;H(Pq^Lm1A`1fe0vJwJ zZW!jk`rk?kW2riqe`56v^%JZ22WY3WWp+q9f1i5+`!eP^FdS_r5#VR5&8O@Xz)K9> z@_8xteUeh8dk!_Vc<$N#keFe%OeZSx+Hb9ggY{g~%B+$gv2Nbs5r~2O0|yHZ5+2eh zK9R8O;WC$5&xj8ze(W}1JaGE$jp3XCNkC#h_nX9+?E&SWgO6VLKv`{02k#SQ^?Rg> zSY6}m_jwmk${RaYq7DKiHqTo66?XiORhk03aRA4|vx?bzB4bxQcP*?Pm{dP7!@l<9 zj$mfV=C{@Z;78XIHa?rXz1Yp5ijN%zkc|CLNJgC?wk5C|09-kG574R){|%Rb-wfD| zLd*sgIFx$XIvz0F12LP(+&Lw=l)#t60wWITui-W}97*d?kV1piE$beR^L6 z-shlux+cfELsC#_Ef0NQw-ngzU0Mnohcf*7TWbL!|M-Ui%$8ZCa|C7!Kld108<_R> z1%Vby-i$ZHi`mWs_SHKM%w~afbAlVlzN@kJkB2$)4b-nG6u@o)v;@;G3G3-52OrLoI{TIA7R8XLZXKu<4uo`k9sK?|@IeH=7FB4LZD0LuL zV6|zR`*q?bITYjsRlLi7;83gL`V$WI_eeg|0`uz6aBiiFRfZi{WJ^{-1~o-6n6V!# zRab%86jdG-n`}Den%j2{;zQluec$H=%-IWw*}k$gE8PnY4Q61$%@a`L~_$laLuy+%$!Wk4kVCb#r!Fp~1X zf|xWgo%E6QQ9Z6-vOa!~oL5k|jGzA;2MZOb8oa^Y);3r~D%nl>r{7ytHAv-ApQB`x z!-#Tv?i?`6LAZSE^_aNl{hErtxheo7SQ-*yyXB4T=Hur+(;rI6r25EB`OwRQ{crVh zFv-{z*v?5?6tA$kR+9h#a?FgT+exdRVA(*7V<>J5AJ zzO!N$Fy8KjdzkiTtWTYvZ{~7+ZI0cO9kXpwxu-umGQ2gr8ZeOSMn?5p+rM>!%KyNZ8;cOhc{Mm%WOl}0{E*yYVdyuO*z zXtkSdf8A-o{HXs)s?Nz(#0dZmRD*dp1ZEZF_GgE$#?%!uATmA{Y zefQg}rT1IEjeb}2^%g@+IN#WiE~`pC@4xN%-Q_Ic&cuFe9YDUF5ghrK-K(pl)Moz2 z{&Lq0{8NAa(8wA0xB9qJWpsG}MiVc`q2Moq33cox!C3Y`Y~JjaeA>K!kb}vJ08aW7 zJPSE_FOLn5LO|(nJT^gcVM3*7AjW@`D-lY|Pc_5_UiqE+?NDim&EWd?Jik9$2Z+%E z0;#Q{EJ4u$4L|?L*uj@0II2H0c92d!kkjxxjogMB$l#9vSuPt`c7>du?xAFWjgx}U z+|P9PY4QG14joSxe~xQE{578TJAaLMQ*-(rn=Y42xAw(qsq?nSGpCE-IpDVb&ME*_ z8?FanHL!V=J^9EJKY8C(AUhTRVo84J3u7yQ#)w{wc3!wi1I*zor%JgVDtRbt z;#=#$IqU=CgxE}bV>1b2R3AC9;rMAqe(2&;^IKiqv=toKj5T(H4`{#c;bkC!rBDix z8v2%P)l2IOwS}^+{wQCCr;0zvvyis;^3s$?r0MV z%i>#WXnb!CKpOpFp2635+U>Ot*sS}pn;c7Evn#OGk8H+w9o${h-|65LJZB2YJRXCx z%HT^q+NufVxF+?By_+4slV9vgB5nQ&o`rnVH!scbTQALSvK91586Nt#tJd?xS(^Xh z*S>b0G?Y*3TdM#v00ZJKB1>1=xvQpd_{Wa2Io-yebme#bn`oi1`>*wHh%Dd+NIU~7 zGl#0!o&5~?_QQwu=}GOS9B_RfSrgBCHi#IK5gAUiWQ0Y;m`2yhp%S% z17D4>*xmv@Nj_@$or+(8|GxBL1C>`ILA0pZD_HcW_!qMBk*~%G|wW#D_W?@YabBm9GK%p9O7cakY&6N=D-N5I}m z5xC$DCRXa393zO*2>%AM?KUDVqlN&WQMKn1>k*1DyCD<@co?>K@Vy`eLE__bHB7NW zULd5-BSZzxu3_wC>xcr+;|!r)`{_SG3^6JbeEH7_JvX-{_bIfdjf>g|V{fNnaaD26hNW;LSYJAJ1j ze?#p(JtOoJgxTOd8Blz*>9xVwX10SGfFev$dQUygBRX}gc8mWibY=kB?s#=*mtYB- zfYV}`gQ{Tg0kn(dc1#U)5SmPR!>cEMTU3Z3H!t;?m9QXa2YxchRTy5+ERx~53MCn@ z4>sPjF&b1x^nQ6ewlJac(bG1APcf|%m^4x4Gwa`5u7+Rw*n`Zn*F+CgYr+w?RO`*- zJn`KL9?3+oTlsSI49~~O4cRiWm|=eWgyPt6>VAkk7Y!PoLvUOI#W^FxXJ!h91ccf$ ziUbQ?=HonPnp5Pd^vDDSWUq~IAO4^~c3Kt1&<-kYB->AV5Y}`5`Ydqv#U+*%ub$Tx zl6q9(I7oCtn5WG+j-)@t@gwkG_&n(u#;zD+_{u{3$%RBPNZA_q*cEcl?r}**1rN!3 z03nAvL4o;(0x%C_E>o!y{SrN^u}kjMnKHOY7%k+5gc|j>?1dLR#^AvR%((b8({2@b zoLP&L#Q93Z;p9og{a)4z-mPw{0%mgBK+W?9-5wemieoreudhpDbB5;&QDz+LBtx*& zh-g+bDAscYuWLcV1JvNiC4&|b8T@Ck1s;-#5x`aaNNw4;tkIJJ_;wA@l+q?sk1-T9 zp#_>T+tc7Oor%Nm@N!L$^cU7+!)I*h(KBv|ygdY&Kdz8k0~dE>j$g16a9K>F1W+HL zb>mn>&EmT?J(O;#M>U-T!ZHVJ439hT_Es0JrWpi^d%KT!@Hdd%2HydXOIspFZaZI2 z=t`oYj5^+et{f7=Xl`pZ6Rka9|5JMQ*t@S>=HA;QP04_#tL{4h)5r{TtS`2xva))5~zP^sVSC)82q>3KU`yTo|J=mHyi z$3WkdGNl6I(4Rai_TbWf+sxaZ+zj_=A3#YEVY^|327YiE{9NLjOH@KGVGKEZEWol} z1KqQPvGq>^kgqJ^*|r^x%Apt5^;V8=l>P^8F%!bjf3s zLh9y1H{~@FZ?Q5#%YN5RWLd<|#fLL4!!}2#TG=xu3N03>Q?b2xv$sxBMcHSli)Rjy znlIqz;xVkF6Q8!UZCx2u868efyvV$F&E9NSpWhVSp=sd>D4Sx8_tzT~O0zO%PxI6Z zwb_h!p%viyS@dAiP>%~P&u)hxaBiO!s1A@s)SfT2L&PWO)oxNQhI9iklLTha0rJOs z_r5Xk6^DBKaRX^U1szgPgnJkcI?5FoBe;w)b&65JA1%qRY8z`?n@4H{#3E*m)Ag+~ z(>uDXkaa=2pY^U(;|6j+fEavzUfJSyyvpFa04c>QL*7joYFVJAYKYQ}^!pk@S|Wsp z$ErO{qNzP{$b5K6StuRQ&Zd+qBpAlaX5S@}6eA2j(K(j$;kkbGsX6FSr(mZss4EcY zZ*$sIV42BKj84^F88_hF4i{-ZCTFNS0y-O1bh-;)Fe-o=w|NLL<=hMm&7(H3GawzH z9u#IlOK2l2czDR0GFqz8z18t@;UQ>0*J0Z?P{&I9h^*vP(>;4ydJV29%a~n4=4ciiu>RU$|l5$BZ=bg(BLaq!5lc$!#LRz0QyZ( z=9!c=@y~Ug_|!#po%U$4@BnFf6*hxsnOKlYq+Ca$W4Q`YqZ-sdIqmEA=Ujpv1cXJ# zfqXIF9LL+0huZcB9crUM?jQ7w3|Ov($~iiU?OaT0v-`QKO^)jQ+rxP~@Pulh%u5r+ zdcjaSEOF{Wug|EGTB^cRXuO~1F+1o0BDtdcIV=p|33{+$8dneHacz>r98V<0TXu$W z6Q$D--5$ngTofUoIlzJLGY0BrLGB`8&@?n)pp}Gr4MRq~)=({}{jDwD>V3UOq<4_` z1>qx^(4c0a=zR%GVt~@p6EmawYU6g`7$2KqSzx`6NadI*2tvzXjB94Jczn1-K&l5~~H`J}mEj z(0O+!SP%=6z)bxs|52`!8w%S)Em(nT-zpf&$edVTwF=?PSNKSTp+gNrSEeA`zq1e$ z?ltfP5;dXkXyB;tXyD1&@&0AKpn+GJ4LFH7j<6X9{)Q%nyin7IqNx<`L1n-T;EpU9 zczt}2SG`FhdnZHPK1k?jp~|2Y0D2Va=E>k_gt}a)=K|_^jf&<8F!aap9`1Uhx^%su znK`s#+EEPq1nD9irV(YzqL?sTo~|1q$t9^o@z^V~ULn2Z9X5v6V}nqmvP_D10&@D{ zi4`K!ITPS{)bkr9qn@8oozQdc9Mdn+HsCB3#Xs!i3fO`Bc3z=sAFBQ6?_;F=+1!di zU!s@n_zTJqI=~ub|8kT(tW-E#ZLBZT#^%up`_KbbKB&Lr&!}o&qffi-bVw1-<~4Z2f}R)eJqKpCrMehyZ?(PN zdyn$Apo{Nf9u2+DckeO^Wru^Df(CXYAQ?K8VR!93=Y8luR_h*WmhCjpLlLppvphy5 zw@@`A)N>j2Tu?Qr?gNh>J-!e8z4t0g{|mF*PK1F&jW_P_l2Ry3>x3jcGLO*EZD;`b zclN00+2);e@CVlH*@CB#?9FBOoXpqIcW4HY^m^+%>~YfnH+vjBu-45xooV6mM-pj> zRTJSoO~#N>iz{0%s{k!4R+0xxIsxM`y`s#}?OB8>iL%=Jaq<{HoE4^^R9|xcqEKv0Fnge*gx+valR9%M)PfD7<~z$vhOxB+th2H6tgqu zz<~1)(wb1?e(^L49*_Rg&M6FTQPnsUCE2p*sZDad4;ZjR{Jsrw12Rs~6OeeOZJ_F%Lv?IX{uY7hs^&`%KF2!wxX~aH$}GO#pv!3f_89lkrer z!A^y3;MMZe#l^g(tKANsAiCZ{$=^P8c#8ZS*XADM7EPBWW?ie6m`89nw+uWT_}(M? zp+>6H(=chgP0FhhsW(m!AGO&=y$4T?GRLJF;Eb)yNGl7aqU(ISFZZ~(k9>4x>|T7R z>)VdYgsd}Y@FLJHvc3i-rCh%oOvs#4R=~3^OclfnWO|L+#?#RTh6V#?LTb9t9HA-$ zJ3{GOZqQ{0tgssOj@C%bMTRfDDDv_Du=n0!O)cBsxN-zRP?{7A(gh)u(7UwI6;Mh- zCxs$`013S)NbjIDX@Xq>lF+M)^bRV$C`geiy$QZM9=+Fd?>+ba{`r1?_j%vrlgEe5 zp6oq))@RoGtTk(9^fc|&G_Fe{wN2DXq)ST1&A}E8)seI_A$6O$o3xs^KG?`m7Z}J; zhzm*_EoOq2kVOHD5siJ_T!BRt;_6aRsE59ermv5;7jVcG)>Dh*(O}TtD5#^T8L+ds zm^sG{J%Pq(XybtYM2$7Ux+o0rbX6H+HB#h3N>{>w)Ycm^SY10$4PdJUM-?yN zKq}IjKvfT0;5QmxZiX&mTDDpqXi1zbuqC&+iI%h!4(kDjySYHoD7dQ&7|6nhUD5C$ zF=tgI3_sx4(Jij?vOSPa-!%*|I1?dj$S3_2~SiLbGli4@SS z#*#kYr#`iiI1dQvEl|?ZSUpWuU?rFx@V+D%AW<^z9(uC&EP-UNQ0l=Fpt=y!734uU&D~^hM*aa5cCgR9DsA%*`9B0u`5qUQv~l(np|3od_n; zVs{^?HwF%z?<|c{MU#%W1&;qUGF0)o;)xW~RJ%?BEXEEZ?Q5ihMFFdLfh{6cWB|JA z4jiD1)zDJ$mBneIJyi@mJP^hPSVQMOobsfrhO`If>0E40ul;Ute(GIAs4=ihu{y9; z9|Fv={%xI%CXmvvt%@-da{@T}5A*WfHzIK=bq7N+V3YG-?3U_P44~K5fxElu{5^9P z*ce319@wMNNCMc6&i-#!ij>_Bg^`%97}^Q;2aXTX0PMy!QjdG9nf>AXD~t}XgR`41 z(iKQ<`OdINtpQr*po#=?)`9G8L#OM1%TNR^*pj>mV9~bib;CcjRDxtB^flDs+CLcX zkDL`Tdnb(cRaY}0tKskV6iACkkr<)@U`Kuao)tv0Hb{HGf@=XU?D*^IB0w_(ha5wU zfbInF{!32)$_hB^%oTXy@B4ZF%z1W%Xn=t{T@n}fR=Wmh_aChR*4Sfoq|{V^Ei+t! zyI;iTtuN%@1Dj|sgs5YRA&>BWS@`s_bni{V- z;G&7~iivun+;J#xUUA?jVt`{yDsr_2&aA@yQX~%H6$Kc$=)d~_`xtp^qA<=VoIBv? zi~hKyv;^=Y(mj2Czn!?Kn6R{r?BD+ZAp_E7FGnQK;l~qYrG+Jdrjv>RAJ*RCXG>&d zg+(QPFSo`1Uf2F_4$Le7#H9Y z9D@UvU;%<~z&QiWQ3nHm{;aElcC;te!C_p1QrjQj5P(=v?mz(+=Zi=0V2>k;X#AJVf<5%fR{V4mR=%s~!g7d#e^gu9$gt~txdaM%)2W)F64(u!{ zu8Q%%0jFmJQgd-ZAxNMm72083aK8vmOdJS%L4a~VR7gia+&{1VCmH`MKp<_g4k!|M zNe}votUpEJ&kFwmdcS~`)b{I+z9ejc#eTfY^&hw=J<=BI`umI8IeL@Yq3VipBmo%W z3A`9-1d|Z|Pv`{pY!fB{(;98-gtB%=VKE-=2$Z!Wf;4D;Am8+dx*$E(73J;-XdRUM zbyr)2ql-OoUkRZ2e*@59(VqzZq246G{|eDk!lZ-9{z;!n{k{CZ2h^mjl>dn89|{HG z1K=5nau)}>+snZbhayRUga$7_&j4k=Kn0HR0BR!j;6D`|+!oo#gBcW;7M2y4l?3L2WTixsjr5cD5Kzd-gE#tgW)zq9|JjQL-=f6k6bB*~5ZXAkhlyZ({mne)6e_wp*3PZ0t$biHBx|VfhB8zl>aVVDK`%csgxusA?$ZDN{W1l zO9Gpgk{IRB>%d?3q!v7*P{TLU2hYhgw z%O3S}BxBrh4j6k3pj#k+FID}c2>xD257_0NyO{0;CdDlviEsh~#Q zR*PB3D=tFP%bfbsGwlw+N6M3>FB!OI= zPMzN9w7Tf6`yu3->U2Cby{)p{UkAo|1|`NeO*RK6cKfamcoe>IeO{fvGGAEeJAd-G z{ukCS$Dc*qqWsg3@|9y0S~z^in6TpWM$NuW!ZdwN{un7Z)MkMYROpP@TS|7V}4$+y&-&D`dZfBGUEKr=UK z{S!L+F5J`cnLxM&97%=p~qB7RV2^B@2!v@#>C#czt`U?~(3U zOCtAJ8W=Hp>*#N01N@eie3W=tag&f$jf_u!ijetC*eSO8rZ~0?g=m3V&0DLph1JXM z18M4^N8S-9*8PXW5HNU|k+vP_h0gh|yNvSmr&UL#pt=)GoUEd==-95|xZAWvv zdg7q#U*`|a!O<*`gz?C5oU#c7S$v&kih+b5-5MFLl&$DFM9&C2(etE*2+j}}=E zKaT##PEjtWr(|X`#X~>Zb>5q17Pd(Hy0N)g*(o&wf;~KWR^q9n1@tMA1L@Wh?Dqb- z@iIij*N`YW>hK_TG=iS{p68OM(&~eFrFr3s4`g0%uSi!~AIzJEUQyinaCnbu4)E#JL<)lHK<0Jk=lf9-HJ{$|B? z?MlG7n&U*f)fQ3U$pDzEhv``1%R}Go(iZ^*8&+|b552v5RE~Qz*4kkvzr3OKO0^F3 z`5bVtRsPaL+^%D;YQO)aV$P8;vW^#RC&I{CPfO%~IjGwn_oLRo?D|1@J5uWAcOF^uJm&JJQ}TXU&-pgW=JP3WJkrfq zI5IBRNWS_)N^rzcQ!5|T8|iwPxSfhI&k;#~;j6a>cM{|M8^(MW3Qp$4M;@PAc57yv zh2CmBEfsFHT{YlCTe&stG-lCrt(vl1{g z1HSofs5@|GYs_n+uo^Sg^O923nph|3O%ua@xz2IWbJ+1ogIX?LVWUUQ1we%J@TLcx zG|uKQWH-O5dh~i=++}BP_;2pig z!@Yr*6l30IR7V*g-Yd1+adL5OPdg^|`pfmA6Y?0R^1@}RfDY5NF28f?7HyQj)@P4i zdT@jNP>At+sSz||@HLN2U8y6UgSO!|%nWR|4L#5hi)31v54Wy44Lq0j>;N}ctoYB$ zN6LF9*N2@UYqY<-KO_+D7ynvdJuh0={m#WJCzQPSqT_Dvwp-Uw8-`4>GQaLIBFs|a z^5&q;3Tk5*So+k#& z&h1ZpJJh(^Th=UWU8gXF0%7etrE3Ntn%R#Iwnko*T-KLglGvUWxmoGFUp;o#vxPHU z9agN3LziE`mn)*R2(44t<4&KV{FW>2*J>Kgil$s{(Rr?SN=v)@tPuPM3M*&s+0fVB zfi#l@B7e2*M80D@tUIfM5qPJWGYo8g%Z4v3ari0OyK_%x4UVbbAE1=r8ZEtN%Y4y#yQ7S0gCglpfFXGZnu4z9(LE8y`-phO{&!*QzH_Uka9 zJ1jpLSC7-Nv-HbmozD}?$}MnTC%zq6Z+Cgs6`O=Lm^V!Jntr)7JF_4FqG;9A$xC+; zSn+O_&r6cf!rh&N4y^0D(`CO*r-$`bYCEuz&SJJl3TYrLzdT%cIj?|f??x?!y;Ww|O>mAt zPc52F636o&#br%cp;BM2hy;ArU4FoR@RX%_!TOtT`9fi}+{JFnZzchRB|WkbFYn%H ztJCki)n!4nkt<-gN%e5;Lq>gtkA-!6^B=%D$Ase*)*{Xza}1~|$eK?liQ5n&C6!E) z;sIruRKh6{LO$TT*_{u^FQ*|TAbQ-g5h5yDzG85{*mKfd3SLi)25;mSj2>)HSPxr-aEg^aS5O12 zc7T2`)8Pj|k`rShz_{EKekxunbTsK{&5|=_y(}Kn0JG0ZMxxjx9D1ndiLdtF<{aKd zCxxJ8=yT{*s_`61q9gM)x(3H{g(F|R>UzpXFfYdrns0s$=Qh&R;-pw3G+!_`GtNU$ zLf(J&AgdLSBRa0W`i=Ik=jD{~JAC@fo6QdvR@h$@JfJ6W+$|i< zuRf%IdlC@!r|(1Ltlf7>JMc!@1Z|D21a6Z@m(cYlgtC#@^fs?e94<|a0#?~z7;|j# z+Z*=74`qY!=-Kz2*Ef4L*_W0u@aub%zm@LUM%EqzY@#ORDXrp$L6U>U^cg?`3w{jv z*Ps0+_Oo}D`UlQtJxe}@w?EphZV{(eZ*F6#CFLOYVE;tMRqOrX0iFEwz}Q^=y522K zNU)&@a{aoqYG)}B5_bM$HvLH`D>GAMoJxi9vu3J|=W~NoV}&Cxf|S|TEhFP3aUoYZ z0r?t!A4LHVVoS!%+HIjSh^Cl&bF@q%%`yh1eH3b78!LuvNvMy$dAOdVk5~1dZsp6t zopY)9xc(euQPESP{`LKfK;tye49417#TB=S-7YU3krC&9)88m)zKAtJ0^dC~OddZ9 z(z5T$Os`<#qFYyXQyk@|r=61z)QDj<{QMPiJkkB&*^_Mdh)1+aJ2q>#$%O_!rB79T z)yt47=<`@e3)yvH9*+=&!d^QA%qcS3u8q3gXguGJ1yt-xJ zjr(-lC)rv1J6ELK-h|Ui>$l6a5jMQ6t*zJU?i5iEOdkro@SheO%K^w(U%O@C?sO!R ztpnBWDZ1?2qHK^*d!?6`jE&+A7*j3Wwd;i*b3EP0v%bX!ESOKYuLP_BJVU6LkSRlu zA%J)<(XQ}c#?E$Kk-9X5`=pp9hKd?-E+0FOxFMemupSBej9t66H_}H{#(c;2D-Z5ut_fiLia)qga>gRhR|6#G4%lJ5#-}ylg;}Y+W3l?5?`C_ql$|kbQ?=_~- za@#{~WT0SkK5eBbDz3xUrE;3NS*B$)Yc<~O7TBhl%}ejF+L9W<(|xh~(n{mnbo(9q zByh~b$OEuldGJZ6)BR?n(%T)i#VK2%;e_E<`JBR!JIh78QKm=-$MQ;Fhids;E5ZIP zl!b$?oTjbN8;i_MR;vIDLt zK`Mvd;x6G8NU(g|H-G;V-(|<_*~w@Tm$?Vg)e`IxCH~9UoOP@9lDAj=vLBsf?`FIz zle@u6afBh{D_{;hBdR&c^B*GOWSf@CCjvtYBROYn*7gBzBqg9L2YcT-S_{zVyH>&L zcH;Mo6Al_h0u9aBS$f{kUjAD7QoNOe`KkUb`0GtTg|{SXD}XCHVabP-49Ip#4? zNupe#OLm-rjZCHrvpQ|!S6)s9-V;*D$=U6&SFL(bbgDIrF88f_m+Y+f)F{{+MSgUc zs(oW@;e{W3Rd>Rs(-Qy<`NZXJNt51&9crN%YwK@&1G@R&wr_s(N*ZD{oIZYeCuslO z6D8w$oft`(et*7f?@8)o!SC};tJ52a6*IrZ4boo+>H3Yc zU3?57r)~kyj3kfD9ZHHvO&|#YqqwHunq-h=RD%K6I#_P^Xj-xKCv~UsrBM@`Sr6o&6M4W69yH=^nTF$;^UmJ)~}rWEV7*Z34vA) z2~&yyERbE}`jnu*$WR5%B@-?3y4iQn{IXTPPLaCc(Unj8y56}xs$7t4!x_(h@wVci zeW`AWXo5iKitxbgD=o#ZB~K~sd_ZOn0yN|v*B4RgwOX~3AfHpwV|n#?gE}@%)dpoB zn=KMO)YGda20r<8$7Q_}nQ9Ks>>R^kiD%Zw40`FMVq`0B@9$-JH+MMgahEAfozpl1 z2_?+ctas@iHWWvh9$P062RmbicvK!ODXkC^?2$PETU2IVeNrLHg$shCnp^VqqYoF^ znv1z5DdBGpI>fs{7S(nGt)q;y>O1@buDY|W=v(XOi{d(%_p)T;Ur6{B#uci_S6+qL zW_i3idi?Go8-hAUr>%|U6gwc6S6UB zP+EGgF=?wW0SEOgbqW5A&zaRbG5%BMTJN>8WU&1f^lkgwp*2y%iKUnDa^!-boSo#Q zaS@q)opDxrHsq_l&4mH(yd`40@T2DOfbFpv*L%aM0>>*9c#c;7Fe zdDicd-tvf6l-&^o$7|EWPY3HJLTDJNtY~z5?w^VddS0!`sq&~GHHrMJ{3s-}m5h;? z7DBbVE(#Tk2oZ)q*V9`c;DJ%8>i15@YF}ip);J-*jxxDk!$)~NMbM#lGkvhw)#jrk z6W(jc2P9Y%AFDj^^j#v?>rzD;%7MC$Hg1s`*xC{-jqw0}!+dsIAfyE!b} z7%x*HJ?@R*sv{(PX@qxLg|`alY214n+RAgMr{?LR)47#0Nw?DF&TX&u$>FG%P@IdZ zSIvUu%9>QPwrk>MP^rP>L(nO0hwG)#iI$C;A$ujif~yQ{YDpTqGJD71ac_vhtC8eh z@9VwW9-pA@d_FNa=u~VTR}ou^eqUcvui9AN^Klg2_u-kLiR-?7y;eke8hXhU@rGm| ztVN|~Tq4x^Gn<=3{A_C)f?G`CkmdyYI~KC-p%WuLi|=d9 zBaWcf$)b-QG(6rvH~xhpe;#}DiSv_8cb(}K;S5Q%(mNqNdhs2}7e2AuO8zFh{^E2o z{*AMdC?TXQrk|A?*%c3g$xXO4`48-+z+XqTys`@}jcap9oD8E(*!H9jY#|fa8}6#` z>sH+6$*mG4D+<|JoTeM1dlFSA&Ke4}607?H-yMPD1=drv7Ko4AT|oO%K|W6gaUW$7 zF6L$9?NM=P@7<=)?pwb;)J36(S^FThje=Z36z+e|xlRkvoq%Qj141YNz4vNK9O z0c}x|Phu;r4984L6*X`)yGpS0qM6$2nx2<-BJMJQ^!!cCI7OuIaCSHphepO?SdKVR z>SvX@C`XjaJML42+>|oFRJtRd_FsjSW%w>GuLDmH<0NCFwmvpN?2F7p zaBqjuIm09yHW;}QON&a>HSyI$_;7EQ|S^dh-I^D>v!#D5fUL2h{!M zL-5Er$~}CoKgC5t9(VuiI+;wh^V;@2GoexU9zyOm zNGy6W$@Iy|;O2`Zq29#Uu;=T?n>3GoNmvagWN#-_+9cO|-P*K(IX<2nY%43*!>Jo} z&>~?w5*;iP!C5cx(aLpGgP&iM%Uk0c1>J5x*3VfE2uow#^*l>=e*NNoW(Q%d47cQV zI~sKvM358gcnVbtSVXPmSc9B3z_#wOzLxXaTG6{$6#S*puOTXu+{ZF>%nkIMDF39B zfelP;uZb^ImGT;uZv!=fdJ|h+)Av1wo31jlovY+yULjPQa|V2-&TC&ynPYX2CCV+X z5z_l2Jm_auhAMbtOZfcosFNy|o&xEMd-a^}dN1H>T*=p%_NemrG;|}TsVA666W1Ym zQ{utg!mmB3ye=W6+`Ucjp;jaE8B*9i%12WD+Zou-1*FK1ui3m7dwOO?gqjNenK>ES zn}{vC-^4Iv>MB$MN@9Icv?eGBQt%FQ`-qnArKi6m1;f5gn++ph{QM-x%9pYb>rqV+ zhTdu%{VfGr7s`J@yMYHJkaDB}Tw$GBxjl6+TFca==z^U~ddhdrqrZAxMB?pYZ%iw4C_QF+8|M}G&}V3pkQk902cx?@1NSoeOV4C0Q5Qf*%va7^j9R3cumpqodR=f9L$s92UBTgjZH2OvQ3EGrpS*}GecKN-qqhvJ;KJ<@HSL(8 z>bdo{3N)X#1LqB6G90T<;62@QN)`2vGdm=v71cjoD=b$ZhjBA*S6B%60y__DHoMuJ z1zHfYeNTHJzs{~6lIO*RbQ@}dHxy{Bc7nb>=rH{ab^{78;bAM70D)D$Vv3L`IGP*=K0}IIxgep%N z8kqF0XjAIvU(N5|I#}(3MP5>?)fWn@dnv7pLwk(KU&Vb=p^dy`bh^8D7$Q;}*KwKE zMC=~%!(qklkx_wXl1Jo6IbZh#fR7e9p}t)ZJ4)zEwLT*4lp5)IH~lSwmmw)RHL!j! zh@AD}EL1Jz?xJSJa^sP1Y2mEc#kjrYqtZ{g21Fy{8sDv{3NZ3AcX^m|pk8$Cgh8$u zI(qdcZs%y5j;XRoQ{a{1)CjprZehU~mKVP7^Yg^DXJ4Fwt90hw7p8#Lzz~cGueyz@ zIW6}m2^GGCZlkh5%SrdGOc2lRviZ5Y#v`%&?uLC;hjeU?;;WI-40mlho5NI!b<&y2 zcsY{vL@?9(lfMzU_H_b(leHeud8`$1%uK;|KHJ`P83-(rt(a%i8t)VII8G7BkTtVs zR8OmD(UcCqxPJFUdigrO*t?5aEIaHXQZQ=j9K%99n0KM@oY?FNMK66jj^X@!%RRLt zxt$Q^$rNM3=2gAjrb{$t{<1VtF3f8_0UWkON+Y^34@26PSIlVX;u=u)S5A3VDnnQBsVyzmWI`s*NaMruCmRoK3<&)jYWN_+w)tXT04HFk+gCd&iC zC7#Q4oj&>$oJe1MJ5=sYnKj_WX@9-s^wIe4T+nO-OMA%SYZ|sLm=}WiXodP@*rFid z(^T-;oXTXj;PiK+CDgQ5Su&{J%4&M2BWd(?YRG%)+(g>39Xq1-xt6C+$>p_6&Y{fp zxdWS7FAa4y8w8lKj<%|wqF+YsYk98Hs3~bV=15Itt0qzQ1U4~D77;6&*m8CPL1k*U zmr{8(BI(&+uNJEo3R~qHyKXRXW_X48*>HtGZENF4t3`ThH~mu%vcQgQcQ2_7lotk7 zGjZ{lv^SVXF&+0BahAp;u{B+oy!76P{=1`atIYab_WJ=QyM)_2&_!w@R+K{2`v-f2jj zMTgA|o^p<-XZ!ftOy>PVZ%uzH-H)B+U~?!7Pb5-R4j<(xai`6xu=25JHCXk6>J=E3 zCCH5VPUg!+W~#}K-<*ZUwJB^-y>?`AGPBVEXX=Mc?dRV;vm)d<5Hhkw8YUCeFQ9^6 zs4;g^z8zvsvlwY>GCBQ#r~Q?f8{b>=A@$~aGJ72(=^X3m9g4>rkKmKZBA~Byr3InZ zi72#j&?%Oq7jIFP>BU418*iQ|0kuKD>=v4CD}p~>)GB6r6E$Y*$ra6PM|n0vE9D5( zZi!w%PK<(KO4M9+s?n9EP&NeJ;=&)&)SU0f%~Kn0S3&+oDe-Gq)|F#-6FzbDyOfB` zm?V1zMP~4t+`;j%D=Q$smW<~LEEP<#PFC$oVL$`M*(#t;{IwJatE`}|vg01WmmdEP3 zKHmBw>AD)*iI!l7>6$XJsf{|&2g5R%m=It|y>Rhch<#@nL0SIGETV^$NRmL4%t$n+3Upm^Q2iD-|?J+Z6)(`!le}r zZHGH|^pz?%?g3k0;pfsW`>=J}K8;X@GB#v>-N!T%bVo8-h-&+NrXnRv5szXjhX}V~ zx$q+3W0aS6OHXkLgq^LceAo>aY~U&fZP_sOCVJ82FG(~EmA^b#nw9b|SbUH2PkSQP zROA`#d_!??zOrglNQmLC=y_Xig@>mryUdOG+duux1<-DtFWGW_KYT$dcf|r`vDvd4 zY|k$;DfY6Sy3y>nwU}ZG#2caa_N-3y4TB$uakEMb%xoi%mViOC6;MR1P7S^5aBnWc z=613{^zV-{0SSkU*4W%ZSum@`%Eju{t9l9(kSZ|%rK9i~v^~DU9RyJhRJp+2 z^U=g(k=?XhvjU;-QBMdNymF%676pfRc%l3+)M9xzsgYcV&f zETDRO8NxoF6MRMm94I4h=2AlF4q_T80-5{xvN2=%GF??aqoR+|3Yb$r+6aN-ZJErB zpbr`9-bg1Nn}gJ{O$J zdyerA0`Dr;U&Fak_0H-2aJEx;4Hh+C_*pUAXD0pereCjjWO-`xnT$=w3%;nxg~5ki zm?kE|3v(PgwahDdPOJH&XB0%g*9Lh~Jr~xcO4euwvnU7+l{^CN_w8TJy;**-Nl8~z zFFinnUsXRb!*p1R0%bw(|H#3HGVXDCl2ODM4u8cXmkz1jf>5XXa=%Wh{-YADY zbr9;hDm#6b1*zX%_N_;aHsJLn$OwlQ6vVWw4_N0R<`hFIt{ldyBtu@P|)#I?Gt6mo0B zEvwPG!4}lGkx|i+Rz8Z3{x=6HvFCHhEr;^*pU<^y3gs4qu7g*chVnwz*9tUGiwMSc zi1%Sl=-Jdl`sqwjg6fb!VVB6bQN~~zI11K6BXx(KJ^DmHwj&x}X*`#9A&H6g1)o$= zA3VTxZJ_I%)XvO2DkU?ELr$#Uwlj6Js-^gqab-uIOu+(Uz1qH{wTSwWOT!9XIebWL zeoIyINscB(>r3l)Me6fnB1a0D*6?E4Q$mQwx@rI1Jnj{K__Jw^(NkEDQQv?I$1m^o z#-Eo{HUrUC`?Dh+)!-(|NhpPG?|vPdbLpu_k>G5D|8>_CCkamCWSCjR#DNZ>K;WH#k2! zi4L`LJEpQicJ~CiSQWY_Ya}kmn4w-lT z&-wK3R;=D`L@K!?H@>}he*dCDiY^DE1HK4P)=Vv2@ygt}D^u2GvLUkjJr(`jJnk}a zIq2nf_#CURe*eSlEi<>7wQclL6SH$LH1C1PIR5~x+dhBPZo0Gl^253hOFjb?33Y{L z;e@n?tCT`um3wN1rp%^q%^W^tcNQC%Qng-#m9zp5*8^Zfa^~$v_?L7zp@r$ZhEpD! zH@6h@hN;|Mc`?RQ2kuaq1)Fc3H-r`vWBWKnf5MvvM zT7s6(4GKGBoO%-@S>Ti=Dp0%C)!5dZMqT}+=#npVwrdt9szPi9M>E!0P{zVlE)jyR zTNQNMkL8Z1$U!tO&Vjmi=-#rVFdfrSqu0H{ z$X1||o4DV?o-u0|83&~Ju?=~aP>PKK#AvmY`_nnYlAaag-rjrf zPKTy`S*4}A7Z+V95|_7`xqBE#%|<4R6t89e#Izoy?%J7Ram2a`H^uRcdod<2~VzDFCxcw=B;<`yVdg?*~K3< z$3{b3CwY-MPG6qNi+* z*)Rbi;i_1|lP-9vVuP51b3E#dsRa9{!30<9U74Dc54Pa)egRJ5$3uaX^lYnrDiH*o z>W=r0Pd4=)W1VNxp?0&>%x!qZ=p+Xe#6i%C9jj6&*eP&OTK43iUp{<6+5-=VQO-g% z@wbJjHKf9!>>;O}uw_?!30cD~o!#0`8z^iI3H8+ynOct*0cC3zxOqOKyB~=PKe6OmDSBY3Y-G9eG7!F+3ob6)asy2Iz*9eu4#m)#b)N;pX!6LD zuDM%GqhUn$D&SPKLR2WK#aeu#&Xb94eF^#Y1a*MSCV) zO{c{d;%YYz57j2=n-v&iGGeBbf)r-_As#|b_cr7WnTa9bBPx&irF{EQ45?lg6VXvX zRvT5`gJflaT6?F0z7xG*99xTGNXQLk1^yFEHjP?}Q!(P%EM2cmU&bBF$*tEP_TmIm zf7(6M^RXDdEL@Uwo!5;3^OV6)YwaTXG>?F!OyeDwCr+o59t3`!Vi&=`kqO@!!*8O# z2cp3eg@Jg$QpJE?7H$PxrpyS6_JC2nf1p}txb$u-j*;#9tCW0+&9$s1vUg8zr3#I& zW!<>4quPBZJC)5;%*^}||3Z<6-h-RvN}qtxx$3<^h~7?yVcSI>|IGB3U9w#hM_QLh zi=wGa3q$gLt^*eJZd$CTUOHf?y?qtvB`^+yugdJ;4hAu2gWOd zW2AJHq>J8bp>BH=bUL$cfxE``qd|D7%b}bxL{pF(>6-;OCxYvqdO6bzQo1-~Az`)Y zoZtQl75-8rXG*HbjB^AeeshWP%qV`ljq<#P_ao9)}-n^&*D z?4S;fJuy+m;}nFhoD4&^!BXNvz74v!3*8QP0#cmlZH7^WU7hC6+5>k`g`1A}Wuokr zvr?~T0~i^dlq%k2VhhYhNMpg9UQiBBXA`T2W(eiE8eGhKtVv8aZA;;ZDc?TFbhEoRf-^N1|wGn%y7G& zDpZB~{c4l2caFB{EQ`c##Fg`H%)ga%r)G)Vg*hlPmsT%VEnwdR$s+u$9&@G7`<=?> zSh4yQQf8|XsUX;-M%?)`{?(jd%XNaSyb2D~Mr5UVlp`Pp=KLT*mSemI`vxkf>@q!l zy<$-=f>}?l*jalDn&ma_^N6}>!yh$N?`MqTp(01%V4PR)bu$s>WESKj{J2;B=lK;d z8EB$5+Y6HunSg^l- zkc9!~(#*$vj24BSmM%#2bO)U&{`{6$FB6|US>pKC%79mp`L3dP%xcFu81^pHv&nm+Kc7u~XxSmMAPh*ZTnEU{Z|!;@Imv0l5J=Bx3xWUznIYm!pVbGYetJzV;|V&cG!T2J+!o!J!Zh*d<-%gctE1 z)L;$x=wY1Fo`bkND(iN1>EK#qTnfsuF00|GGUc14)FV|i!e2*NcrG_=l6?$=rs;iw zlP#yZAwO{XEdy!ORtAse;T*`d=0^Y9MtU!&CX-9|N`WC^V$)HowjgDWi>c%T4dA_% zIlJLo;{;EY`|HVHd5T#en#WW}J=#uG$H%Y`LRV;Eaa0D&g1czEzS8c)%8XX71Tg%= ztDtdgF*ukUW;n?Q8ll(0%LGQ8hER`KF|ida&V!k|a8_D#(ZyYI?OE<^mk9ATwgLGE zySH0h+XvMt=`7kKC#FX@95s6r)5iQw3$8S6jPIUen2q(D87F+=?B|FsA_6lfy;18~ zO%udp(>>(y_dXgaSD1Qx_bzUmQgV!IHz``|MwpYDt+3s+!E`(eko>dIw&JIk zL&5-;TvwL7S8cbM2v3j9E1^eea=RTtb(VJX#UowOi-P?w7>CDC>2y2uJVQ#rGeH~g zOO-w;_4c-PwR^xMiV&)L5A%Y0-IJr4eCcQg4L?0eL`tEk4>Xv6GZ__C>ZntBo~V>M zH}}MzJal#rPi?7X4TX6|k79idaV1GCA%P-MdpyWBqjxX*y+WKm3dG(rGJ$@f_YLOZ ziM>mN*uIF0Ls4(sbwd-Q6VPdpGg0{)jt(hSb?H!5PM29rL3&&NZcBM!TK&Z2%)B6| ziol8t3`1!J+I}S^WP+7alI^16YTA)9+pr0+=`P@lB{bxN1ar9{-0LwL!^QvxOVqg$ zZpqucmd8ak#B<)XJ$p`$ECy`=X=E!r1el@9X?040C_{bFBu=~l#3+S0LPlpx71gGB zdC%|#9gq_`e|@{dE#(LUqg|-pjFOX&VDGbO**6vXG;c{u5O7$&48t+a$60_^lAizM zD`<+W;RBHBK1p8j^6Rw7<<#UF;HDXvihl7^r<3hsn+q&}**lB5ix9k@hnDJVv;&V5 zk^q-C&!FkwExw6)QdSNmA(!5G?<>eyl&r#Z--KCtp6+Gzpq3*+EGX=uLU~BVJIStX z1Djga%%=j6E|1|0{5FBXYSXeTOn470#W0KDY!H()!F5oL*JIWQH5n$hYjOCl1lpw zDx2hC%zd2i)WV!3#k#e2jRBQwAB~+=3qHNc(k;uS3$tas2k}a-PrJ%>X)?<#%V1#$ zE%jMoKuA7}?lNDFzx(G)^vu!qDo9zlWH&`XT@g6)NJALTa1}-Ns7}4P$#Zg8WUAJL0`BL}foHQFM$kjMkBlj$0Bl)jyL`Tm5+=vNcxI zu5Q$OD)y8JMO60)nMf*el)O{RRnyO`vaEc(5-=H~<*1{2i=$$Klm6d`oEyL*muDNs z`PXsi--Y=y@W59ve*{=vuoQcptRcZWf>H-6^{~;bghgnhSU;5M>z%4aorJ_H{CzoY znLQ=Z{`(X74@J8RCQNMfvBgxPQOnI{JFuDNOmPu8v7rqh{T2@U)K*_P$HFXSpA+CI z_G~+uNUjP5xSY26o7e_@1J`b*&N(o0M`wzF>F-oEU+@F*OW8+MBy`JYQ5UQVemX-? z%8a#pQD}}U@%-`;hzEd2-fr+6%nNQ@isT}OuDs2++7wvN<@96#bF?R&4nfv!S~jG< zt|iHMRmjV4)HQtr7TU;hfDTnAjGtMM`#e#geulFjicrS}=-2c)@rijz=ZME~4bArOvn|rK4h4xe& zl&UrP{h94V-^qFOjSrAn!T!N`S6EpY!D1EBG;C63`nC%QLp-Trz1K1GDMk z{G3UVrIYef&-2m%-<$pN1A4vShqQfz&+4}Cx{5gn)>$xm0C?!zN{NhPzLlWY+20U) zwOfR>!18#Utvzn-Q)_594WYbk@`*{du%be**SL?*LhptT&p`^pp`hlKYDrOI0oKw#p_Y z#QNZ`q~#Lphd}T#FdA$T5{I0bu%*=J z*#Sye3E>qyczR^9)jGA~L-k5swRD%0;nw_yLVvS(K{IPusU0kj-$HozLes@+I1R5d z4mR52!pug$6lrU0q{=+EA($(LZ_wxDibe%7tk;-#-<_^|Q(K(dUh@QBk}=ofJx3kV z9nfQ1L7tMFp!~U$nawc+b#&5?`OD~~;;2Gwc49U#H!lOk>BLRUher8pg)^!fIMx*( z48^o_W>DsvzVN(#oOY}3MhHRX^TRte9%=h9GQ*C2JShZdBb=^-`I6F2o4t zrwX#q5R%)QsdlVhV4%mB>WCj>&0iYjPALpAyd(&n;Y{M}-*4*<+$wGu?tOmU&d64= z9Ixx|^ki|hpF%$$P|Ruh_C7EQ-rc%O47XP6H(=qj-LKCd=YB>!iCOUB-}eKeksk3Z zDfXNcR2}Wbjl-uOyR?K!nS6z0Ck;7;I%tCffapW{Ooh^dzaaQJjA8aDRL*Q!;?;(w z0#Bq~Z*+9}+Ro}!Ep-hsR6bhO;;FxIVl(?qe+EP@9(uG{Htuz>H>U3XoTH-Aft_7( znbIfJ@_W!j@Hr8|erkxtDUcpM)O-B7M1wnjPHfrJSW;3a+fZ!#31LR4&2%)c90MjM zrUVkoqfvM3)%iOV7ld_7PBT0xe0q|BZRw+Y15;{o&r4BZ0T2lMx;!aW)H9Ke>*bO8 zW4g^mbeq)2wX>Yy)3RrkDu^&@gK^2dqwRLqt0=^yn`OXCs{12wz2-iO*H0W6puD4D z({|^${Z7%vP;v6uWG5eHD&K+bWXX2hO79`E~AmE(+yH;Ph&=A8(MId$ccI}&J z$tJ1xzC?{g=W5V0-0(s4sCxkzeDlKOL$SJj9YDt<%nmaws|uw82+3F1+*w~tZFqUKohCVH+m`7%f`F$s=p_{Ka+Y}j%Fg45}F2*V4bk+txfP%)gp4ISXJC;DkjYo&?dVS~?U1zgiMl`f*g%=ZuuNR&<=dKOqRzs_ z*RCwky*;KoEXGm;E4Jf}boM{EI~LhpHnAtlahAd2?8oY-Q0k*`w!LB1^h5G*>+6BE z((W?JkNm^nQ7R3&A-6X~C@g?VFLXm0#N-d&3IL-A$NvULl(;9v~3$y?4T zOW(RYZO211Ab3F6^Gb)4(~wL*d68^iU9--wh8tCQn==Jd5{fyU$B38*T&43SeEv6C zxz$Erq<<>nFH>Lc*)LO9^K?&Fn6VV{soQ-FGwCmR&_VVb2yr!TQutCQL1l3rWm%7NQJ$TMXwu>*Gu>> z?lwT#p>z7ERz1D%yqm<|e}IWgZlw*|?K5{~B)d8RV`cv&4BfD4YC(i}qik{kc9b(D4j<~I8EGNHmS0o-{;r9(HIqyFJ#%*z(>1zf& z@?(P_CXBc$5O^Uhe!4s*_kaR4Nj|V)Xi8-kx=KU+AfqHu!Ho!|Y2%Jvuc-AWdhK_} z_oKzJB5QG#j>B}OQ!=VkBKD~dZ$&^0+3on|-r^pj7|V^}AwM9(eXbNpRH>RUXi&lW zIYGl>Zd&{uD$+?IlTlPE?+Y>leA*CaxfcIDPXR)L0pOh6iYayoFWl7sEX#qP^hhr^ zbA!EF@@{AABZGKA{e2LgAA#5m?E2|WB@eFwm6s_mkntoxI zlj_V(A4J}v67^^`iri+K=i7VJ=lGg8O-2^I7Zi0)Aq1HLL0IWXF%IJF(o(IGl$;KR_q>* z%0rY*tt!x!Y|<4QjwaA0a%=^fUdk&~AKYAW^N;SnO%0tLT>mO~m$Q>d>;XUTOWkD^G z7jlsoED%dsHztz@rh3G~@GNPmI@!b4Ku{?lrTCvyAveIb1#TV2l9r@E;M_DG=aene^M#38ozHzJ1mo+pUIyV&(Z+%as}+G9yI*+% zLcTmg86n9?bkICP4CflRVJ&LdL;)^zi3RrSJz?OC?;DT zgFm17iV>4|t8r+S#~qoj=1Y(U79(S_oo8JqV@|Q|47p?9bq`ol#RMK4PejpI4%PB=ddh8!01?w;FO9yi)< z5Nhr?H#Bf9Rhcw*b?Z$!<7kl&g88LIbxgGt&Pg}z<{r7NDcy(pw~9Z$LBaG`(b={k zI83ZeN9dr940;-@`enc$C9@026 zH4suIdYWwh#n~fDePt_WSa^mD>PvkPv&iVS^4_mj6s=}C$pznBFo&~~tLY@{RIF7| zR(^fj^;N4ra%Zq>fV~TrB{C9QRY+4T6C=C8#Om(L{_tfVR7BMndm`a_Bj{T?e&0yr z2cn-`=Q>7oa+@E3DmP|_@=)WqxU!%4^JTfEwTmBO2I}Wx>eHXRUp^Ra9Juxc&wSH$ z)v%vk^(kvYijN^`-oQ2O&~!-jss2WVP2=YXXk2uW;5|@fJcAiO0&W!B)g?ar?$$Tu z1PoQTU)#U{XZpujoF4T=sxyBaV(&EtlTx;NoQP>seWr=GuP3x6SXDm%Lap8~@?7Oz zjnreT<~|3%5lm@{`ik14X0~Yho4gBQ54|3|_I^YCqqBA7yKGgsP7WRW{3OO+QV!Y> zY1nIY%bl^(3X~~;y1nHJ9}m+P|CGJuQB9TsXzbgQF zFAKj?9?%uh^tMNMu$O@RxDkLDM2H?><=S6o_8z;#5*Qa#b441(+UI%k&NC_swQ;UI zUisDiPNZYsrj!M;Zfm${Cfpy+o#-?dFfG?1e83!{ess#geovJ4ZU&TRaPN!i>%ag) zk$a`vc{g6X3oRF5f8eYOSs>e}jEt+!mllf4k=l8zO2^+KgpSt8jF4Y*Fj0;yRPwN? zQkR(g^QfGxG5ulr+%p>jEnG5<>!ZhV-o>8n=mZnFC7iAHg)41G@@Q`q@kV*>2NK?8 zS#sTu6Ighu)#`I0#Caw=evEs@5{QxXSS-Wh`G*xL0jF)LKGhQSLmhgvVHW-Y_ga{m zj(yZiCP7(m#IKfNF^TE@pc)ov&W8Aau~BxYY3G)Y4e{=A*qIjX12#mxn z)$ZZFt}#Wr)tp<(;g3=`PRZBMADg&t7B1(in4ea*1cIP=>`ut8yVkiz%eBVaBZo#L z;zd9LmeoA&XIFN5R8c1iS-7?lP2Htq{TX$Mu3HV!T*OP8fPt-;iU!Y`8)6eF!e-os zHDOd$^nRDi)eHIaK)|BD(9{n~MX#uHE!~Vx+psX$_58HLWE7VUHEu3sqmxM6udXL5 zyOaAk?|~e8({`UpwiIN6WMkk9K2bzpJx?4%bd(V_Re%55F!4pAdc3!_f4hu^1L2h? z!gH#tK+2&L5M&q=&1rqkQ6RqRaQ3n_Kuxt|b^$m=J*Wc#oE6?6p6=E`WSRExapGe1Vdo3?EhZu^9p6JuBJpPeq zeg*)3)I7XqM;+1ARQbLo!~_^Eqg=Y6I3E(1xwQzzma*Eu?4s-Y7kvvue&hN3TB>=iMJDreOG%e_pmNgUhrCVm0_OYGk+Bp zt0D0Z%+c0CB@b`E(5p!e$_Y*)R#RC*L8}C0 znB~T_AS>)X-@$9talI%3?d8Ir1*2@wv!pSl&-kKuZ#>g!CG)9><077|>UBB*Fa(1F zS23UOEsIw~8sT-~eoOcuWXH{{_}j8}O=jLmnXdsZ zmh{jQ)m#%+?klFWa$3;R0`qDrhf+4X>XKTyirqYu`8VBnb9TCEkIB(DZ#cTSwV!&h z4s{K@Db>|}Q)#azR5i|fD^a>ZGIRhZ4%xcs7QT|x2 zm)-K|gB~$|RCWT~e81QZzT?jbmLmg*iL}%!7j$|I zCh{tGQDPMfXT{0%cbjiC*fp0o#QK^~ONo5AyUtsX7^w3{H!yiH0fv$Jf_ zOBd^&a_wkryL>*R{l)aDF=4LO(QW*p1aZLzvYDvrmFCmVGM~ffT@M?E(~F4nC_1B; zh!~x6M7LqO{?x5E-t6Y*^9n?CsJX0a)a#k3%nWUIj(_g zlf7* z0=RmDh;ityit4AGhmhQ~A->+s35pi}EkEAuAjegK=F{e*PJw2-j(c!q82SJe{pJ?e z$ylXn`%#_t*j@n+yCpfl+0R9VX2aPFJQ~au+;;rTvbY}~a_i^Vt}M<^eBs-8bOAbW z$h?5}X#0`zrZr3V@cO2Y zYQg4uO@f)pYs3Il4P<-{pKa;{F{h^~HnUJ^r1-EB_Yxvyk?hFH9%qC=+i!F&N*w?a zfuQH?Des{+P28G4B>g8S6@(s9j5D_!uloUy#wStnr*b|)FWmPM9^Xt7hk9;!+N}&RPc2?w&*i#K&&Ag9V@C!!ZHquDRfnyhmmf?r3jO7XCoZ^2+h%zNX zba*5JK!6yc1-ye4!3Rx6kkr}3A5SkU1Poij#(O72y*25OV6rKRAo+a#+WR|W<;p*R zeCPp!Q-X`yA^ksiSVA%$ykSj@%uL=Z6`+I7zA#>1Xw`2@JGbB=pC~D^9UVLkE7Hk(q@b}bp=C;vIzVWj<*ho_%JWDZz!G)c;o@k z0j`zR_A^-WKR8|xK13!=$i;y)iTJ@{CB6iN zPZySiR?On*fJ-htE#{RT`+(b$BkSkU5P+lE6v{m%_zj-r$;x02U^Z%~a;mqZdxcLe z908<{Js%->2t)+_knQR{g|bE*0hGq+r?0UikuuPJ{J_wGSA}(uyJChkGxPZ zrvQ>b0tP+Z$9XtXl7h%NJzv zrEeWYwLohd{9R5uu9?o$yzHtQ&9$@I5ncAhy0PMmMHY7aa%l$HnOxg%rcSICXIqp@ z@9*4j^18lqzlTM;riDelq=iL$e`9;t-0o6$K05#Y)3ca1^%Be1SI=bq$%G6)k&g1T z1(S1(1#h|>n6#xV)V1zzFKoYtiQb7)u^DxLC)PR&AyR6OLKM=W7E8N*T(C(xOVXTsXR*@FGPw)tk$VX@o<2>5%vvAi{B{~wS))6 zs9-67(LZAN->7-Tgq|;%)T&&FSs!^zxKn7A7UoTiCI`RVZF_HZ5{nmrz~{DM?RPip z2_=9K+=mXu{rz&KY@$sGy$GnX+`@O5CxTiXIP~V!6~_Sc1H-f)t(J%xTr(xiRk$C)i@iZKhtt zHX5}YvoaP1%TD;WK>dxexWa|hWKw;cj7b39r3_U2HnGXG>0Wk2Xlj%HWzeM2Yj&&Z zIngV2p!YaPhVbuFC*>T-vzif|;}tT;?)klm0vnx?)Eb_oS)4fT_FxK*%+9Uci^=d6 zbrGe+$A90;iBJ7-(NblyAb%^1URAzcenucKk87W(;|_h~)CF78Z88c#>wTwx)8n77 zMg8!OV1t+h>n$14Ozd(wl~(h34`H7N=_^L9?tWB}H(g(sfmp0>J)ZwNNoMs>RP%L~ zYwc$uw?gfR&}9>Fsix<5*%Ie`aY)AKSfy57WS7iG8=j&x>`#&(jF3&d658IyH z3V2>7X*Bq_aY}lS$CizPy%-ihQg{HKaMM{w{qOs4KocxB^f~>iXI$ZvGvZC&CsA_c z8rdwktKcu$ito-0^40QMlUD`0Wz$02Cg+vQ7chx3+@ZHNZm(g_C*|xDf=P&!>n53` zAsYu2Bj4b24`s9q3_fGHoyQE^z@z78yknuMKlXA2LZ<21{PDIQ+Oge`hc3m=z z%P?}_WJ-w;(#GU#tUvwqbeN>fdidUD@KO9(u5JRjmd$2mR_7;g)Uvlf1xtLZ=&iO@ z;*lB8OB8E8Nn%Qr{`)S?7_3`_LMsH8So6=TYYA{jIs3wI8_nanwZCZMbboYndbD8E zzhaD_Q?-I2x&RWo=Q3XBe&}ett@em)Q^D;x&>(&~%KjF=gHhSq$G?)I(zE_WlICnr z#bx`)nU1A2--=ZONl=Z&@J`S|YOr8jk2to-w&b)GU`Yb8a7&~-o2M#FeF>X1#QeZ$ z;^YjUe6rGIi1%N1&p#HGF8$8?$;&SkdZmR9C|W@jTOUIgO}r_<;9{!K6U-9y<*s%vt4zLYz$ z>IK@gDWK$(aG0(k1BikZ@cy8Bh=GeI*z7_q7HJj_+(dwjveF2$BcK?$7#x6d& zK`Pjz6$3rvp_U%U3ehyxjb%+ zYdCzzF`HWD_-4hE#gv*&xD?&HzGBRL=YdJqT1XyLi zRq*dG`czOt&R{kDheiGV!iN{DqzpGRRo19?c7V{#dF9sCtpVjf@s8;k<3EYD;sf?h z*|+qqhkxVTlpHirf`NF~a2BDgw}oIzm@R3g1#2}icgWuHB6Zjt#ueIaKPo8Po2Oe= zJFt3-U$x=m>%a0R_+h+wO7==@>9Tm^DgLfaOJEDXwwp==+U!7x0GB@P)^ISWybtah zg#Kkm-?vsm0HR%Tq@2%IaLWeM`>k37(qNq}!I?FQFx!SYR`n&TgsQRg=RNv7FN~_0 z?D4J!guuQ+R)qilJ}I(jL$~;iZK@$%*5GmMTZHqHm07fY4(M5HWxHboQCnNbvasYO zX4^wHP2F>`Taq%6T=Fc1?{30hUa=3&6SRbnA6VSA@Sn8!W?goi`OK>GhmM`{kRX{_ zYQ>M*{oLFMLk^6tx=LJNeT`XnE6J?SZKBc$l#WoJ>->!Zz%EVEL+8|jbj@`2c_PD% z)BN~ysqv`FNbWTR-G2s&xAiXRwH#g%7fKU2+H7@jSxsk~yb?Lxu3%DQctv{( z$q59(*+Y+fw!d*5pwW&X5-2k9g-A2Xg-~#`J!2W?F%|vlvXbHn?uf_2;k`?W*^{L6 z#354NzC6vcp1cK<%@dOg7x2$zO%M2qFjQ_~J16PfL|47mp^lf;=SATR2!*ju)%!|k6z`}7cO06;{zp{fx{ZqaCJKlbBNZ$;cf!5pptg0Ad*BQDo3nUVa=#S zli96Sw6{YGsW{%*rgp6deNKRWk%Yy#fqC@6G#S-F)&86N2$; zw}^N<1YiN(;z@{O>s0er&r8{ena+`1f~7Yo5?B-~4zauwmLkuTzn+dIToAd@2ZZ#u zLH;FeLFhy~Yq>Xh8(_VW9fG3GJ{Ha5w%yBIFPDAbqr$=^DOTYS$2`v8Oe?gt`o1~- zTG`VVF0qPeMjKSjr^xrB%LSHM9UQxK1{AfNS7*uTLu@C3N_B|6+W}Pn*;F3S1Z7QA zm<#zQVZ&&cxiX~gH*e@jdfe%SgG}#_>TTCvo^My3Zr5GD%pZEs6V$o?ZFLX^fHT*p zcYJxu(Fy3KT(NvIoPU4#G0NKYiluz*O>i^M#*>YIeFW-=iMp* z)E>DT9!@J+2}gzw7@lXX+MtP9_YGJ>`Mvy0q;9Z$f#&>{PI=e4<(EP+e^*24ZT~R1 z_SO})@^-oNV1DIGZ;|mgA)oGzP%GCHU|bj!tR?;CE_|dWOV^_>ez3#ZLm|igVYMoN z|2d*y41g=QK&PTZE16wT0=H#+Q|ko!`Y+F>uL|WvX+g8v*^=hL*S*=fMQIKa?{)>Y z`O_-`5%o_$AAPlmJQK5AE%5W88f~Lz~l$I%5G zgV@3v9H@IF&s-PRUOPWSA#n|de}&4i2~V=TJ(7XPoq+%KZ@%s&FfC~-iQeI4Qmx_x zQmf9SKzncMRmNo~e;(uOD_Bw}&C(D31P@)l_L+#ae7k&{kw36@k-08>vMvZ*s=(s> z-?F9;g5)1ut`He~0>4|Kp!8iaBDRu5qA*ULd%P(TfIeG+EU06nFLuuQ5P#_JSGb11 zf)b8tE``#*Xg*q@vOZFwVm?|Gb*5W(QSpl5vTkJt{J8c$vd&CBA`pwBqdY6)GpLso z@yDk-_mK&n;W4#bFemm=p0_tvMvF-XhF$y)we{hl;F4YfAV&I|Subjjjmr$BU412T zsrjz4Q{iDr2Hp4qVmmk7E)7eQ*bRII;2_gEdgo>Vq5UZKzi$@;3Y~4qs9nm1|N6Cn z5SwactzQ>Dr5o)j?5K(0Q8_$LxfN4)W<#o^4WSB_uX89EpF<5O^cyH{at!@9j5C$#w@Alte7q*uVL|5q)52cMM-hhYh7RJ z4?BHEB`&{8;Y0;h2x9S%MhgxkVzEUFRlRR5=0tYSSngJgc`kj~l&vhlk!`I2S=vr) za{i&p^ncM}{`)y1|Lq)o;A8?&-j#u@m2KzX3x_j-QY!9z?TUf!WRbcE^#CVc_p>7q z9me0bIokOWuyO=&*ag{1_FxvKi$m~KQyTYvw|MgS55n%ij`M4`*&=s4+*JU zNCW_m2axWUjOwn#Ghs6ML;k@)KFdWkQwgiG_y^O)-2~krxB77p*X}5NEU!77o+P*H zVz&Yy9kvzu6F2!4ev-a2)qFP9UewBO^A;<4}%tA=zl zmTMAh+bpz&M>R=mmjaqdIxd7uR*u$e0koA5|DrwoxJ*RP@kS~Chwzc;vj?u1=Z97B zHV~NWUUvW!#Nu;29O2&zbndoq^XHLvPI&p~>**yYfPf`nfS$lHfcuLDkD68jSW`LG zP$+r9$;xF{)!mqbK2@%+`2E{1(r=SN|S=yqBoV4He z;_Ntp$-atEJHIuQGMO%bdfndQzQq)v9L!~UkK!;}t%i{=^fzrpU*N(;!g=mr^l~yf}u})GwT?Jp|;k?J&KwFWKdRv=Fi zuAH1D3|Ub;1g+NtiS8$>0cdw}T4iq7hyA>Z#Vp^~;QGrNNa$2J7hqD7qA>uVsq_0n~Ihb=$lTin}u+=@=7pHWWo}ZP1sg!4f{#M0p{WENdI>U zbC~s~glWUAkJts+gAVYf;?vL9+`|l`@}GP%rIW^rj$rI%@K}*5jAPWu=ObuXWN#&l z^bZAt9c?RJg&4F8UFm<9pK(sDHHR(wJmlKCGU zCiDowX2-GaG!Yy@ZM0#A1n&=3++_K7?1sX%-EgN+|4#Ffd zI+?2|{We)11A3F(Qs3SMT$S`6uKLF8sgk35tNj^k-I*>f8}=l`^X9Nysmz?y?U0hnCBz z*;gdNA~=PqTxQEB)`up&;hszuHTfDThz;fC2o|0DFCl`%lFz=p1PC$JOPsIsFi5 zp6Th$*tHvH`}-QUPe0E{j9?!15ACppy{jG2_^H;Ze8G9$(CjYAx$uW~ul{o#Mb!wX zvN{1S_(tu1-iurLLrr>s_<{b>c1vAZ5Z8@;<;!Cf>xv{^8~r(39dTDWg$1zHZ3X^S zx-!r@E$cdj{44jt0#}hfJD99_f5`pH2n5xDBI9mbxb1QZ%0B1dMbR?R0qUF)bjGU# zA39`U4GgGE;!}AxQ7uRurBtf$tn5!;JeYeTGs5FP-0UO_zc%|~ng4qyJ@eu8hZ|zG z$K5X~{V1F>Bh>xvY1v7MS+z6EgDt;#r|(jOm<2dBZ$txMi%6owuz4ja}ts;DoKfxRL@xA&smr{8B%!j`KUP(1A0;ACgkEm7tz+ z*Yhd`T;p5OaQ7te6hts_(a^Sj)zZkCyoA=T;qBTqc%!#(k9l=+L22+P;R7_vfRCzL zm=si#zxGq)in4vz$2A3i(1kx5p<#Aoya-%Mo#h^pOV`7dv{2Q`7kp+Ch3)hKJ7CCAEju22)GDY9AgB8&j_CnW=u_SEfGOU{@*ggf z#IHvsf$qX_->ZN@N{+6lj|uG7`MKu6$c-V!M0t;m5s*SxXli9lrA9XcF_WsMMUq|U zn|(tQ0_F8zX819X#9YZXU4%oF?8WaQ|3i{Av|OF4k#qQ1DwCv_A5X#93pGAztX<0G zt~gpKG|gEg7y3i~Hd!`daFjk}0eS_z&}X{Upos1!d}X(v#@=lP_^nUE`NIYSUf(B~ zYA`(s2HG@Nzk5mB*uY#Ns1;qi+{UiuRdH;ySOnG)y1>tCa96BAr1 zDO!LBH!y|r|6u3|Uj(aOa0|+-*0OW(*IPiFwqzN$4PKxGGb!{fP-nrP)~|^Ql7X5C zQXZ>!jb7~B)3hG+kCHZ~cS9V@!Ca$W%i?~CsGk1)6#LgdT?|E0xRoJ9^oNd4Swz9T zLd>i_3p(18D=2}xO*7`yDX?F zfo1Oy+V$z>>PLufM0|bN345JnKDgm>J>c3pKjk468u*_PyLgTplA%_2g6p^EJRz}Ei zYpSW?vAzMb&@daVhoXP;d{Y1cU~L1gfFmb>2UrQmO$|6L{q4!r*M0ZOGw%PXpD_`= z@1wMU_{1G)d^->w67X&udyp&PEuk-}$|ro_oIR6(;=e1EdK0CM5ilAs`6d|}=D|3; zcy+~&uTY-kTd6>S9>5R9E8ziP;pIB)lax zcvK%`zNSH=!l&V-5e}gc;-71Kr6OMJ+HI~ouOPT9_T3$LhMq<9Q*_qIeYa$5|0E3w zV$)|ZD2CT2jh=}s^_j(eZMtzRcFLSLQP5E!v)Twm1;t!9*4J|kq5h^XK`@R^L@7fd zZ=o_z_op@PX+;H>XwHxS*u#(l!uuUyzbbgQn{Fbzg)G2ATr#lv7i7hIN6sQnQqDZ!@- znAGwiz_g61+9>RV$BC2aTmnGcMV*Tk3<1gmOnhKHObkN* zcP+*~(cQ<&A603R44DMp-V_|V5w31z(9KWweeKzQ3bRa4pp{9eMw~E!ER*|>EF+=c z8B%&wVb{ags`GL&%Bnr$R#Yx&z$C9p51F4+79BgbV}L^6BM^FmDmSUbJB31h=Ap5! zom$g#(|bvWZHxRwDL+*+gzWp4jQ-pbK49Rc7BI-@vyN^1Ex38C`_zRs~Qes zWKX7AOr7~tbkK0O*-M;rX@utrh1^acj_`XjT6eK;1^S&gZOzU6ejXkeJ4vIK$4e?4 zcS4Kv{y*k_KguVGGvH-z18-K42iScw`F-$6+*#ON32~H4PXu&hT*~9DIose)ZX1e6 zykAht!00ELNJU%=;x&5+%MeTcVT2R2o zKaMpw4=*{?nmJJ|1IQl?yA(NexsNY@Y5rGm6*kj!KC1MGhzSst@`{BIERl9tW7PP7 zcLP}bv1lPc8{?^ff>{J4(8#XPB&jc~8gS^}c_6EIGC-p~KZ~w1X zvGNS5GnxgiN0PrsEWmid*dJ_M__!Cukq366qe^agD-uZR0wI(l!g$y@z`eBC25!jK z(R`>QKp`{wU@;DBQ#t%$D)+g6Unu#Hh3*4ZR_D}p-E9V7OZl?;Y^pXN=&fi+0Q4xd z^^nQ|R*2?=p1w;09GHYAmmvF?j~DPzobc(!JRB1UZu?6lKh*hUkayDq!9VUj-vchq z{H=dk1QP z7l7|5oP2e8PH?!Xj(G1RmH$`L6$$7`43yzV!@!S8xdqQO{D{SPQc!t}`HRoJcy+4f z2nI@(1EVnwFf-7S67Fp>a9-^bL@Jk~75=IO+{@{CkSkrl5!kp327}hH{u8$>jt7@T z^{{@g;C}@GFra5;&Os`y5{wmT1h}E{IH26j%QOIUALL4Iakn%8+L(qR%)#x6NT*O0 zW+FbTmY|zk-v%E!!c`?CfrEfln~*y_%{Hj!7# zOo@wihzAl?fP|bXf3589S}D9nfQmbWjBg6kon(riIwT!*L)@r}@E~ z3p%Ru%hzZF?7V+DE^uupGdmBvz#9c0jSCW%N|iDv4%J6qFL~NLO{!kLQo_SLI(;Ad zZ2N*yzslh{CZjDX%IfVJM>HoQ zoGCu{JbZ6(w+2$on1Ydslr4&?>`@u~`U-Xgh>FRAj2>Y}B3f}eEf;ydbk2w(s(ck&!V3c}fm2$o3|XeYeNZo}zFfv6fgFpgQwHm{)rrogtfva#3=Mdv}*f7<8OY;30@b~4vuGkiZ6LmiGu(z>JHAs2oh8!Dt z9(*evjEacY>!Uyl#|bE6HN%y@_g8LQk^f*U_$l1?-uh_4HCH#(u*YR77=}?u>2i@~ z@*P||?4~u6Ey%1@_UrV^iH=Fd=A*3F>$@2HA<&;IF6MhY$hQ^ovLs4b#@i6Q{_i~* zLXkKF-mA_7%YmM)ddDlP>IS_v=K$T6mCX0xjm9$&`oiObd2cj|J$@YiZDV|J$Tb(R z*i(<2OG(bbKMAm?49?6ru_{UO)1KnLcN6vjgITv)z4mAh^Cb{A!JX`KjGUvnnk6z) zoh>X`-&e>%xHDSc7|&(y>lKR73913a6@>Pc`9O*h%XJ7)?*~~@RlyQNZ|HfF0a?!f zyM;I;j|PynmH}uthTm5D3!gRm)(cQYk@QKW-S7$aTpYwy_6$-*I8rar6;jGzH|^je z%~8#Z-vCyU?Pken2e6W}b@K0>6x-W;KQY5(B#a7<^{#J%UWtAWTAwuWl!eE z2<5yWrZyOL^7@+a-4f#0S3cQnPLvdaVg~sSIE+alYN&;Bh)?pf4%OAwZ(s<|K?DKF z$O6cGzU8)@@Lr2fG>TZYRr#@}ffPU&PJgmlB4-N3rU40#&rM)#3Y^n2k>UyvQo@`A z3+gT^5ko)_nV4VhDt5}P9*Gp%5-KimC}7}6SVDq%3N5xlwrU=UQ)4+jZ~&RXNUAsQ z0L*o~ExG#r8aP9P-=)$;^wEynms@#>B5=+AKjhCus+X7w0;%RmF#50q7%dMzOAKT@ zzxz;f{q84@hl~A2S%6~^P_vb@^gXEWjCwSsAXcYO&wq%~Wdtd^P#ZIgM^BT`b1mX;z@pEz2q7gk8-j)kfX*HSs6_5X5e3@cUy&iai)% zCxRf!vN}^&=NbIr!v{|_F@qZenOTT&7-Rl{0c>0I!`{9&yctJGA2nz4M_Sf4lvC{O?a4c?1HVZV-Yu%gMay)EzOwb5Su?dUygjw6MwucoC%MC&hrLE` zEPzt`F;zRI51RT8)c(N_))ONiK_?yPSjXeK z%Ttl#wVZ;WhwP>Z+p$;2VzZUa2JXH9hT*u&c8z>ud#&3kiIWL2LSUo-P_zmM5S zvc@(Ds~Lf-eWm!~VDd%Mx>bLw&lLFm7V&*>wc3C;xq>Okak0bJ_lELUEkNV7m@QZp zDGw1gr}`n7B`IkwcACbz)R{*&lYF~Jt++Eth(XB#`r+SqvX62 z9xynb6&`l+kJDbB54_lGuar5Z%0qPBO~NaM>0y_>70#|HQecH zF1$0#wpU?fQ{sB_lVd`dWA8;L5OKVX*hUE-igV>kU1yj{J330;XFlDzn7N?nJj$-% zTVfL7IV{*d6H@zne8|ZDs`P@z*uvx1n&Nj^#rO_M^tR|Aa8js}UB3ooeYr7o?01`& z4tVv(DLS$1=NOakA6`8My44V%9o@S#3+_r}009I414$}vAV{&g8r^PU!k9;?XX4g@ zL(Oe43JTTA-wi4V$S1wJc!~T#*KJ>=VkI?D5SAhz&lwIM{fdA)AlaY5R!=J$R$(25 zbm|dHB!-G2yg#I5yW32hAcU+4d&;`MaRuuJ6=+`M#$ofI_gr5GzRg=897tm~mUI(A zKIR>mQE!Zct&&nrI9r0woUTBN%VBEm20R0~O0EJGUF3f>X)e;?xv;D|=9m1UrA#%J z8IRW2q@$0IZp`*0ak~q>s;CyRlfJ!T4qKz#f3x7LQ;KQ5{Dw>UeVV_gzwBrXhQs&= z(o$Ew(q{$@by1`J1_AkMTr=uvFB>zpqP8b5NJW)?@b8;+gr~St{yDo~Bz<_?J(CmL zjb?L%dL+XyR#@?7WXg}I$Yy6j&DLZ*2S`#ob^r(i%Y>G0Cro}|JWoT)b!wdVN9+fR zQ!OV;;X_7m@KB3}?$(P{QCaSR-{nF;_76zgNqLPE0i*_>)d$@g=N7z|pzTG%aGNR2r%s@? zG2R7t1CQrcqPvZ#JK$U`E`NC;6Xs<;0F0$3Y1C&oxSK6nh_Ul-&bg_kZT&7dU?9jA zz|hG^pxJmKYYu&*aUDNwf+T&&SS@wpZR>A4ZIhh1OI;EQK zn#E7twTqvOykL}dPB1C9x(zF1QgBYPNK6?JmE33>lpSd=j@U26s=t9uD9dftF|s#V zxs_VYsP*2kCi~@+GR@jz)@=H`HyA;QYCjI$;t2x7444JW^rCv#SNhJCCRFSc(og-J ze8dplvfX-L(43~D0DO03*(%7YzIF)QMMiYk#ls|*)@dO`Ckw`Inf0bQzrpYYbbX}* z8CM80bKb{5t5|Hw%5Xp32GcYn!3dfb?R6CGUco&%WE^RMZCF(q7KI=$Vi*#tp8K)U zIH(=Kep?NEm#$d#j5NxXl_kg-xDG)@5z2!=KJa!J^j9c*7T>9NL6{9^GZVRjt&0pt zJ=>{l!X|4Gy7?`MU-zdHE+ULAb9k-IH5V#HJpPGlroI&xXxg4qTazfwgP^rdN2G(;u$wGW#wU4R{|X{D zFSOHkKXdWYf-Jha5)(npN1Zk9d8^7*`U((pkk^q+VYthaIoR(N0+PZU3av-q%tWz6 zWPL)iocQf~AdDWeamvr16f|-syiZ3AS?px1(~jr1B*A$4(XP&!?I*7dG2VSTM|QK+ z+DOLgNuEr8aeB)|}(l?cK$<4>|5(YPYWzgoLpd)mQL zHl_bAHeD3izVU%%)i*OU{wI%~yf@%@^j3da+DHgcPpt-RonD851*eti~-B8!qBT}}O3&Qz5Yr5c(q=A)Hd zj3ubNnkx3(fD+b92#RRTX(N;+Cv#P_r;kI;rFcq7php$-YwQSuqLex*h$x`VlD%Zy zdQ(1uTi(OLA!|@oUXpDEO%9%m@Vvq4`s`w?K2Iu7@?9#hn2M#BOW%cp>89@xeRxF; zUgJy-JxW)gSx?JPS5n8mcf@C6%vB8~X-APRXj%2wx>x%#jh{TtozcElP7%C{6Wr;z z-Ac1%>_N+FtULM4Ov2a;r}D~Ardhb|pIvG&rlMGyr?Jn18&PnA`tpf@$fGso?#Kz1 zy>!7!IS(8)i?Q3CJQi;WKad*jRt9S!q#aL$DI>>z;@22eInWG=#<_WoeAJ3D61{V@ zt(;Ixls0RSWc9Jfbqj=ro)U8@KmOqlpJWcsjcJ8sR(0TJbr4cH-mCL{HKj5Dc0$}DGi37Di`SL zfWhup28EnK7OZeo*QFg)1(&i?kJw0=api*mCl&ta;!u`Scd~RzNHDNwI#cey#yW#f zFz=eWPgDAlR-qZ5u%8-5is&M&(j(IX&&Wd{pRZC=8pNxKorf#swUsxUOWiL)5^WoO zhqEo-K9^BZ7bDD8zUZcI@3S?b2wUvj3>SgzJ7+pS&EOQVz@VzbZK$n20}o0 z<-^UjLbvV?OT3=cw80PANH;SOW>UT;9aH3f@Oc&hxZ)MY6R5&r;{FAj)%<{`*&T8t^en~y8L zb}-4GuUqo6)Qdl%)d<(dHfmmdH_m|rwV36*gzTPXQyVhR?kB!~+AKo9neee3pNO{P& zDX6qDGd;3uID6l2sEWDKQK|}u?ehR=gPCC|(pZ*bff}6)Pv#wgp`n>#lY{50Ys#Ko ziU?!=r=gJf7+vS~@-?r92QA2D*4)vOFbRfRlf}=AN|N+lvfu0C5#j3hxg+$yX2&Hr4zKstNZ5g}1 z4*yiYjN`#+QxiY4XDejQ_#FD_?m=&Q${>0I=hm16#~G=W+M3GOdpY+Ub34OdB1FK; z2rU_`>&!=NvPJ1{*Y9<);aLkuu##UV<4AhD4iBcE6?z?_$xxa8ly=HEsM85h_31ch zrmY2cpu>2Y5i(zv(K)lG&035muyk12Ew?pI|KBTk-70qIyH|}~lT9$MG@;0c(ivrE!Ryq`l zUw~W3^K=$uJ)_{s;Pn%h6)D=ADKvNcy&UA~lM$_XkwL6XcJPMrXD`&rdQCE=GmB9j z$0%S(NWjNSS-RKGhX}-1276$8&;=XhQZ2@B4cAKyaM=i62~f|VHLqV?btr{bJkFk^ zhX*AN;t|u?eR@M_sH7xQQbAFWpvLCJehU%;+v~7Z^#ztvFPX- z5i(mS6CUp%XCHORxHxRc=A7Qxqi=zWbj1ShA+_wA-q#{&4|5OB2R=UgK7L|DRyYaGLzk=T-wFUdLUEGoROZG+>77)vNF3-kM~ z&!eOfG5WESD%7WwE5?R|hNKFRhSCIU#ohSci=W#*Pmi32`BrI7q@-1nG?FWp>VDX2 zwGiUdDPzRVLrOmwkh*318ebIzAy69nndwa#9f}2Pl7d@s(u0uCt)FS)&B&oL5S*^` zr}?>p9tHHi{v4MQ7={#(eMv4sjd*u4-vyx(jgZ{93ONHw(*+O~j{(+u+-f}Ma~i-R zfyeH^(r*KWv8$Q)6u(AmS+QqrLZ)=9ouCylsN{UhaQ`;<^Bw2?H6t^<2|jYf!d!M? zAt9aR%)`g19a@OGkB0mfYiQ(ZBu5@~a2b&uEqeX*EJYU|du(;rx3X~*x66qO1~ev& zaD58jf|2xKq?7Sm-DZk>w9Fhuy%%i~m4l~$?s#}>qW*H&bG0CPNBmgDdx`O71>X~& zZLLdYEzrZR(hmE2FaC=KeG$V7YavshL7?xSVG~wM)$oF~x zz{Y$Roi!T{h2U5@+sw;lawSTf?s;U>_A}T1L{c>d4=(N+{+k(TDu8il?dkhL6elwm zW2ds=e()ZO_6rXnAqc4SzzB+n zfPm5rjY>&LHzEzvGn9aUBB+FvbPwGs-5@31AtgD4beDUb-p{??XMcFV&Gox%(fUT)%mAo2wgAyuu{bEk`ZJXF;R=;Ppmnp!`62wPiaJ@m8|k zJ)5~deBTCOfu2>^Ie^cw4W7UB8*}JKtx8-TGyhrRv#wT|+{zWyz~|BRo2lyyPjIMy z5E+wWo8L%WFrjl#f>Y1BzKv7mJir-$##5oZ+$9PA)#0t)a=vHsVJXoaXNA;-<0ae0 z7BDwfn7_%6!`IaqHcg<^il*j=eHy{gIqq#*Q7!suyHFB~n_pb}1yxiJe(rJVi{=x|^w59Gi zd>SxVog=A7r5U+Kx1pyv#;aTI@lBC?0+%s6x*%FYd#CPPa#5a>UXI+3C8cO#>?{X& zDa)4z^&OrGqyIDL*=6@z+&*}Z$dWmCkk7g45w@H#|8jmr+t@C?A`y6v?gfZGKy3`? zMnVz61l{4cF2HtHT?DTbdNuy=w<09hra!jKtQT~cF}UTu|6CaSIZ~@cvDf9+AMx%l zSz^K$lxsnwt0bn-bu}({C|rhTBc%l1St3)d-Y$4;UYyz&(;|4l%DKI|kb??hSG`RM zL*g=eh)8R^r~u2H8c_GS)ZaDZ`c(DT|>^>Gw`{DZ;K`2=uFjBW&s$%;?~D`o0HXAZ>?F| zgEpn+1TtgOFNFfetJkJNS<9?ai`r2ii;mR5;=9AASn8F#&2@?XcaP^h&@5|OhS#-2m!g^c=t4W|^i0J@`prVbSOEvx& zOf*bro(TA23LrJfJhA$L_)+?SBnxFRsAknAi^PtUkGAoDs<%xeOF2L{*?y zFhB%VO8FJ=TK4k2UON8(_^Zx^FbR7b<;f~~)4w4E0+80*FBP6AX|C>TYy?o`M!rNR zl7P_YKb`#-7U{Ue2;LnHl+q5dDlz#nQPAS2oA%KxRxmt+sa?(cQSXL=jQr$M7h+su zI!V-5WE+rybo5jj(bJzkVO*TTKkFCY2*KsmE{-vdDTpwRiRxg?zr19E_iXfjGw9DM zWf=W#pXZRV2PiN};rny=2pl_ec@5 z@#U)Gv1Ke`?9tD88$NCr6=&guz+{m!wism`pDrA)E7H4dk!wQsH7|t=(JdOYGAr5u zj^^V~9Y`~cUmE@e_7-3E7m1<7hyy8Ad>&))&&R!ysh0jr{6J6J20o5^JNHIaEh?j6 zyktO#84Qqj*Smh6t5EOAd7_7l^p=XV8a>olB~)M^wT9*h@Z(A8OBemJNJtUSx&*p- zBUgTw1n8?}s8NxU=yhTnM-!>KXaHd4N=j)`srM(FS^DA``5V4J5aroilO4v<@}l9l@V#O6ItN?3~}~ z_rN^nFauCxfv&c%J@0X1x#2l+5-XP%4v*dxhsSV~?d@F>1cOnyDo5}!L!AEKEE9C)eJEy9{E<`Wc_}+ODU93Z*Emd8DXq6#mh$+x zH~49ra?;1~Gz&k2nfkc?4+hY{`BfiW;DonNwI@^(Me!+-7Bg|!j~X~4;NzML#1ZC6 z5@Fr9#^cz~LWk33nXbIO{;ZK%yv2X3n9TcdL)+;^$~$mbP#?f=n9K47+pReG!ID%# zTHu=p)^fk);^soXH!o_$1N6>J36w;OC`{3;i@MrSW_6`ry=e6cnO+}vLh`K}s}d%m z=e76w(PB=uw+5fo=vhuMr*&!Kmp_ND^7_B4e3B0ygw?_Dgod96TWbF!&7W(v;D3Bp zB0hc}ny(<(S(uTXp#1bX;lY{x`moxNbq2M7(w^A%*#o+=?{{}=2AtL;lJu@7bwNBH zlL&NQlUB0wwV?Z|34Z5ta!8RPRS@`?Ax{49U%Vn$>;shrKGWw4d?IpIqtvaE zJ@aiS8d}jP;$v5&(cvDY?w9no+ZAp$Y=3Mt!|b(@>XTHJ*!Iq^o(owWTly6?$)Z&% zzdBY%u0r=u419%RSg$7yxW_r7;6DmPEel0@?4d_@bh5ez{TVpEmsET43TKREcK4^{ zP=Rl?mBu1$IS~n8Ptfzj9F9XB);xuG`4K(b_P?Hx*{iW0B=n0wPkIr2A1a2Am6i4L zG9Y2d_-3}|QrEXdxEIt)mtW~ST(29~H!@1r0A)H)yO=8=ARrZ!eX8T5APF_Ktw?&Sk7eRkd6I|hsr>_&8!+Z1Bsv1%qpHp> zE~NJ-zFSA0*h^86)Nk*7{mP?T<62k?#7kezIwRy5iRMN@!Ws*wemNg3Um==ZUyAx@ z>2hcL^P>X6Te}Zus*@$~+Nik8wt+eFlUGz+K1#Byw?r4#|L0XXnXTDKl=I2Noe#K#f&wRIMa` z1@%P%kPn!g967)e+l(6N>9K<&Fup8x#dHDnvN_N)?C$kRbOFtE&)yAz*aS=8F%C~1 z&qu}^CwRhv*XZs5D3{nP|M1VRt{bupr9EDW0SD(95}S#cg@#!U!_<`I_8t{@xi`HpS$z_1;Z0LUZi~?BgJX(IizM*xlukIMq=SOmy zrrZn*=+Ub)Dmpvpi{dm+GyRDf`Jm|+#Q|=$(R?*}{inV|9ZMj>RR_2KQ-D?Ji)+5l z0GI9b#lgE~@QHo0%$5}x=--$F8^V#nCv=~?qh*tXZC$BtOA_IAyk(q@$}u2rxdRw< z!QQ%597kxD%hW2=(;uVHkE75CMmOq83SJ^F5ha9PiAZ!FsN90P00Za(VF08Ll|}<- z8nk`sucHCv{aIZ_Y($hC#*!Fn6e>2_Job@_1JWDpzWICfW*2bV@JbSfYd3u@dow{) zo&N12bX{gRx*U&?EDAVJA9U=um36wj{vAMrls2|)0MVqMf&2HLUIlxraRP`IPAv?`d?Hy zY|k}0Z&t0q35_&L0c#E&8en7KnoGt+w<6{=|HK2_h#nf{4kjm%a@Yxt5d1d7M9#W7 zrIpMyoO))kkx$(9R;jm?TkfZ)rbV7Zq*S@A zoF)?pxq;WF3$&hps@~sVmg&GDWMBfKmLst@V8I4hRsb__@>uH0PCFwvnx9_h!<;eW|ZWvWm{vGUwb}9iy+Em9-!8ufE!KX1`f)0Cg^2PrEOw6>Ztw%Yy z*yB#tlq&?OoimJrx2}`-xB8q{3l3dH6pGLQ00QWC}@4mwz=ED@wWjO;CfGnAc zQ`jZwon^*mea?E^kRr-m;4Fn17ctBcUJnTJ|9y629JrVkfu>;(zkJ{n$W0;uw$ySo zsVkQv?hWk5w;rut*zYtEGIr=%CYetaOu#`ybg+n;{%=EsZh-2`K!12Eo59bb!2L6P zj>S&-%Opw@x9q;BI!pUV&D3*fzjr6V^wtA#(|OW5+as&L%~&ZO5HY~d#rm~n+C0Ya z8cH#6=#kHWuZ-GT2LL@oT)bB%cMKuw`)J4|=JqUj|JyWa`Q)+*c4t7~a8~{8GPwE; zUlGa8Hik!=nmv#1&xFd#Z;V*?+ z3sr%8@)s^-300cvbP_r0S?OohnMCR!N|2g?cV1WwGv4l~eD@8E+Bq;wZts2rZ`P zaqAc<%RZw!l=`_2p5O+_;#Whe+5c(*4zL4{jXYObRAi0vjI3ayFzvo%F}&<5b%*OZ zgxE6pcZ^LER$&cECJDdpAuJExNe8u5D1Km!z0Yd{fA~V&U8Euaf%$eG3EGdD8+IoSFG zhI@e>vx$%7DupjC+^+vR^T+!t*%@U%zO)C&jujq$S->7RF#G`HBNvLYJj=rJ>yDQDTQ!IMLA6dkGKH$!btqP=G{Sauo@@Of?` z^==9l8fjJwgL7#e~Vim9;zjbNCLnjCsy-I?CTG550QLF8@%s#y+2@oAs^|Srr z&=FjnQIWX#&f3x5DX;rBb-J!@^z&v42THMG-cv%8DG`*Zmenk62ykdsW6DU&U_k># z2Hq_Z8XEca^!>{|&sOB-iyGb9A`Kryy*snL#>rZirswQHCL|&V1bk)@;2SE$LdX!v z*kNT!MINcZmj-_8qbys( zJ5*;|X3!o*2VqdY?zb#HyGac=zs+6r+VIZ1SED_?uONvpPD=LzWvvnLoy(^E`mUe) z=P8Zh-QxPwJj?jMw*who#&=wCI{f-(PPFJnv1e3w*9EiFA_hTPluI5Jw)y_1nn;HIGPMj$ML$5 zj7oc+09tpR7$2vI&%O${&MsbgCp>y{d6Lb>JQv-pq`>bRG4@vffqr@S2BY0KF|ZcS zuCASOd;+fuN0Ao0~P8aXe11sA9I zoWQ;v>QsTgh81Y(Be}8CEIFhYKG3EHSEltPa?~3PF}?}7x+DlwK}}T_ZN1?f5F1G;*OhMb)%$tc zD`WMck|Z*pnMqO}^paj&m1kYIL7lQifl-Q6>Lb@bV5C=4r0OvGcb{C*>!Z;?OQ{Di zxu|_V+(D~TCtwj}5!1eotporoq5TWIj!EDdUCPZ)=HZeqZk%vy#3Ag6VpD$=cp4Du zGvu=1Agv%&^u*?wfbD2Ey2`f}NXR>Z9>`js^?LR!P)~4<8MuU4dOrG0HRrxncjSpH zYx~?+mLbea-@sslrSKfwqP93;5c|*j&@SaUMlvJK?A$W%ln*Ql0B{9;H zvNdnNOD-PrlVmFO;Nsm)&@5)J5*ChLo1!V=h}O*?>tf6wjW1Rt40a97=r2_h`&vCg z45sDw26D>S$27*pWHpujzVb%O-v%;dNg7!uvS@vPIKl9np99}ElO6mhLtmBqGoKDa zn5!R2gvS)>icWwNQtt>)6!&jY?XH4~+r4d4Ihz=QYgU(_t1tsQkJ~h_DygF@6mv2f z_&wo@SaJ0lyL$1!GySPZNT{kCss0M2h!Ie_Zj2A0dm2R&gM1aUVLU^%tzAp2s0&DG znM}_*#xeH13-S$bzpHM12B)eT`QH7d!Wf_i&T7@YKOyk&=j9dbC0I%@eD((@6mq65 zK$Fz#m0n#uAOD-8d#KK$@R+*kQo%2FUxaPPx*#P@Z#*fFTaWP@qZ*@Q-EtuFeI+4f z_{LmIGHv0@5Fx=pV>*Z`D9Rma%m(IL88WebRqJ2nh3grQ^*9!+&chXt3s>dz)__le zVtm!AkaT;$M{t&?2Q`KJ!X1JIfo{lh-LWO$7@S&OGirEkFzAOh=djz4R_s!0%j#H} zt#%x6pf_tkxV#sz#3UCrMMT_;bX8fwK9ph?>W?_9!k0V(v1kvubhDXS&w-bfk8RT) zr?#bOqT$V(b`w7Tv}E(~&}qz*q1EZmxX$%8VB3FWy=;%>^e_x3?$ycGOEW=n7u!-I zzr^f%8f!Ae;&w{5uVi=Ah*#X2o~jv^Ez$h&M9gh?4}UrgC>jCypp`l(C7ZpFW5}-( zPo+gwKb9MRbaZsi{_9+9TD(&;Gr{cXs*G;ENrt?iQN78~m#s|wOYt{<2%36y!oi&K zRNQb{|0!td7QRs2Q#t}ou)yr}9MSd%<#wU5T6h>t#Y*Av0#(Jnda-49@lbetsL5b{ z;n~8B7Z{2ZWnT)Cj?;w5I>IMMOcl|dWo0uUkPI%%L~1eRe;O~#;PehE2fwbFhG7ix zwri1sL$p|6M7SBB{qE{!KxEiyggu$=cXKgRva7#?!JCB;LHG(r-%%`7Zm=e4>MdUa zjv~S8i(9Yzz*?g_J^@R1f(!rn{m@OftTDQsjd_JaqO|_fJBjI=X*~C@QzOZe8Vr)- zMEm513!MLL8*o7q9~Hl4bvR`I=D;Rlc_n0L)rM`%xl8 z?#rW#W6N+Q^91vrEG7NlcNRFnW>>JG;=3u40Gjsu1SR2I!M44m z^)M%yU|JJ3EgTnUE@3T5dwl?!9LAigXXkrST-nUXDX>MgS+8W`g(zArza;!NxP-zi z9D)LBp_Vbt#FpE@cb^WB7XpL0Ti7W@L+RqpL5RSWXLhyyIe#DyoBN@O9i>aMyyh$G zW|RHhYn1rdLdmEE1(1X`L+devAX_$8M?gt=a8B>66x_(^t>5tfe#l*a+&m)RnY@Y{ z>Ov)Ha03jOPlcXNKQ>oox`tw?@G6@Ubp3KtYxT5da%5@o3#r)(awIZ>bFC^PnLD>< zwH2CONTT1DcFJyZ)4g{PQ#1`U54o0WTaXIm z#Veiz0{Jpl@9}(`Zs3elq;al5eh?xIBxmFTQdKAKf*c9r)L_CQgW(dB$8^HZ7)#z; z74)!cgk%g>e8FxO&iA}e_aEiX8(mKrn+5ASF^XAeW1WL)47BC}dB|+`>vgy$sS9r` zxueG}6<_rS{KnsQ*BPo#qk$iQL$PT*O+C|~*W){QJ$O&uQBL<$`=6)aX)?VAkGm4T ze$M$vIF`CAhFh=<=grw4!G(tzzqr^c?4|VhH=Dp{$jqmD6&K;B=~^Tpp<;EgHlpN0 zJ7sNjaO5kGGAn;ZucGbAkB9;2%*)_)p{2-jRh+ILr zI;E)(Ft&M(M=Nf7rruVnC86|QJ@f{P zv0Ayoej2&(B-6#~=~Y0|EZs#jQW^R^?7yLOuRw?Sg0=fm=J)x9l%BOX)Os^}Rodbw zXkCz$v4P$MfINC&?4k1(trh~aT27xI`)3Gsq{LH%1sOdIiVu^$<$-}QUX9Nc1(8w7 zh!0K+2+*7s_3~G+5>?8;3$zVu!BfVi8}Nhs8YzoT`3zq*D?q$^Mu@u055MrUZse>; zy~Ui=3Hjk*h}{79J%^%;U)g??s?TnngK{d1OY@(+rk(dLKrsw>2fD)T7h-1`@qD+ z<1??^m`*_Gsv462C^sU7_4#nn%7xQVo02>?k6`h~#pSyVcYNeE%5xvA9z2vXgLhg? zXvEjbmEb)|kMc)SY# zZ6YhzFlhP5DUgE&1lUSoYZ+U$PF-MaP1#(PoY%pwwT}=h-W~A)qARtEBA03=rUg*o z1P!N6Z|?)2o7v&7YP9W`_gwjvpCThA*V3J`f;F+RY^*n4BC3Q3FZUxc$~EP;xn>R{ zM!s2e#abGLCr)iQI=W0)OzdDXGTkc-pGG`M)m> zPDCJDSys)@uWmRBGx3rBzJ(KJb6)L7IGoWZjd_I-R{yl5>OJQWH>{pIe5Zp9F)}DEc4hhSonIcC*kBu~)%~v?MqEBCM-M-Vw z|LH-T&{LTy=ZeMmir>sFg(%#r&ZVvU4}8tyG9J*2NVSRe8w=Qte@?Wjsv2DVVv9NK z{S%oW`{YXq1GDKZY|2RgjT??45%zU6ZStNQuC30Vwnscn_R=y?Gr@9k9eV}5%c`4T z$v-$^?#FirV8Ot%+ZR>rpR0&9T(LwYI8&8XyFzs=&%V%B%0-fM$k~>```ytJSt8sA zOi7YSzbNKukiAJFV<=d8DzmM#e_}x$kk0`$_#R4=Ajl(=?VC(|f&nmckrKI013w zDQS1HTw4F5Pb$2E+OG9H-%=NP#rmS(2coIud0O?-Oy^zb0)__S(dMf%k@iZYt;gdT zm03<=izNt{r}-p8hFw|Cy0Tk$zS~tK`N>0`F)NwzZ-VJXaVWxZan`Y` z)F?65$ahSSx3>xsT4H|uLi-<*ATkt+57$vw=6)}uZ11+*8o;q4aYx~`70j-Y*%^-2 zxH$}IPX5`D7i*aR$kFXWZpL;0zN43`@U!?yFVOs3T)R$r#H=h|v8M+#Pu6$#0ET?U z$IxsS9~8wCl;~(y|G?Sy@G3TMVEqkORG1~VJnhhTPtBoe&lG~*O$Y9sjAW6uliJZ@ ziGR(h2DmL2$)AeYWk7K`6#^2=d`a{7U^L6{6Y%A*xz+afRtXSb5yVzz=I){#r+|^@<$oOIuLLh{W`v&|NMG+7)AYILCj{6|+F57{!B6D+Sk!C3H>D-FKk$EZN@`*e10qZYJd2$aB3eR%f0}zW;IqXt=J(7V3Z23eo>|hdHR%<3Iw30^@+HRl7)}s z6DsjnI~#AWE#QCHeHU5~x7i9X{%nvyF+29Piv)>FsJj5n4D66~1`3fv*Dc{7E-;WP zT>B3CW}UxfjO8(q`M4Hxd=!8kt{2tx3Nbh#tEHUdIdD2~-cI2K5p|$n7k<7I@;Y`4 zt74~8(HQcPIRc5H;sif*=L0)xUtAP=xUC__7MIR+LLNrVXiXQFYwn_19!_(@g(T_F z;+1w#+3yo0~YStM<-az7@2o&XBi*uqoE z9=b7iXdUAx90?sYu+2+}y63Rn;R*g?KHfSA1HY^}U8 z9XRiRS6fkM*cOfKdom4dY#QN$r7kg|PE!qwEe!7p!l78{C<*wWeCS$69J21Tg5;O( zWBlLy7DfZCV76#&)-Y-C;dxGu3y`B_NSeP31rcm~s_%-b>~XG{FetsuvBgpY(iWH# zJ6YB8wK&I|V2^xR$)p=f@Ue17FDl?(Qm`EtgSql`wXXJ`SxLo zIJYi}xP4lL5RgrajorCe{i)|4kcUuVbgRZlauRAryR_RnkmQnj zhvlqu~B7qS5&Gl~LR~|v?|suj8i^KwcUOq< zs+Pgr!%M~o_kVX)f*(;uetkXS8P~*`Vz42%VVIvOl-gp;dy|O`5vl8h zGOQ3{mEaFGU&PgUlo?m=A()ZSU^`qjeseK`dg=By65QcT1`v&W)w-|J1MGn&W45_C z#K)UcRo~Wy7`}_L0?DXptn7n@;^)}d9ofZNNw6~nW2JLhcD6JOc+KC zd@oA;l~uZU9NTU$_iyy+uv;~UMxQ%i2h5VwNfJPU4>H-Ay%=JQzTZy*1b`o1JOCUm zTBEeEe}@ze)ZxkP5y1LheV|6t*K@V)NoAAO;O`2*_p|$Jz?>O>faLeWdDq|ZgR|_* z!6OwQi!}#yUGiBuJxsB=X?YoS<>6olAx$pD1578;iwJZLei?C}&_b|XqK6Oltt-f^ zU#r-P@ZhpOh{ySM;3<|-qdon6Fd4LlYwxNv3O;zo8Ifs8Oyb@1XoK?OE!{^$z=fk_ z50Cu+@4oL9dY|*x128Cm0h+qI1_Pz`q{4-ok0w~oPDEWm4xgo zK(!|+env$_(QeBlq@foC*Aez{Z<)olV^Mqq;qKawaI4%*x=`DSC>0v>e{YlHhi>bIXX+e`_yr9x&sUZR|S4Aw&lSQz}oLi2TTpO%tV zKiw>1FZ#mGT9L<1$Pbc4f1~rEbs=7Su-*r)ivws~)U|mA-UZ-DEXLQfid{97SVI7- zQ(`zqyQ94PBA!sWhUcoPOh7Hbn_FrxWhlFhzj>USKud)(Q$jW0BW3Vi zqy|}1Jh_v6sX_UOOpV?YNMOsRvc@iqP89cl1zTl&4ymQ&=0J=fHQu>);sGG<>HE zfeZO%pCzY4h|LyDe+?=kwHmY;+?Iiib=CUCzcju$9+du~>RK(Fe&$j+&GQcsduQPJ z0X*|3Iba?oY8}p1%aBB-G4uCZJ^Y-Kk`mv8Ey$B(QBVCUB?~v1`H5#l+U8 zrC_t+R)oL#<8Sb(KM*?Yd3|gPMR4$??BIQY1@PtW*xX%MLYY0UeWMl=ruWf4seFRQ zy1Wtr)H&-_DdRXk@XllTVvAmCPQ8w~rnez`eWY1CvjkK5e3v^UUyrxs{$5^$P}{rk z{^K&hh*`5$_2-HRHuY`yY0lqO>V@_&K}7Gx==t~M`H3u}`<4s>j-7)~-cz0(!(OH1 zz`i)+n!LTo1GN8@tD#2E0US)7QXnn6)uIk6$McRGKo=+9aP^xx2%3rinFNfly1xXY z+5&#hEc|Qpr4_PbAma53v^h#XNxD-;j3GwO^WCWnC%*tenqN|UPOWWwxT^bhan1p~ zEr3oV6Wx~VJAIX_DP>vwiufFKal3$4`eje=M*aB*n9#;?w z?_$gyB;V5sPL2?Ev}?~v3@xDQM}zn$x|rL_A{DUih_BD&VG=kf$faZ2n5Ui80@;(f zYPXvUlbSBN86^m4pH&N4FDN&;1X3IU=P^m*q_U;+OpIRnD>@60Ww<1hWPh%{NExp& zm})VN_ol0G5-P1ZYyfIHAE}AOdb>l5O2PRD{Sfs}*>7L&b`N4o@)D=={5F22YGFnG zAFv}0tRz}JWZgqUVhk=)&Fm87A?;k4#Mq}o{CDm}W_S|u=TVw1)sN{WQhCFv>W-iu z+9lZtjUR7sUg3exKN(_T7LNxR2%xRrF1Sj&l*~?ndNq<44f*Q6y>^T`t!`q0-JdR9 zW#6Fo8OnFc#&Jos0c_A*6@!ptssT7uJvk{&exeSTWpJCu?x!)^+7IW1%cpDtp?SVC z$Q%hBR@$e)HoAXzbn)gH1G%z`KtXbKs^Fe0qEU${n?8yF3E3wJk_nWSEXcu`6j`gp zh1WgX8fr}H2^iv7jWZ4Yo|J#?2_7 zPwJm2J5e&`exahqP`SlHR<7{e&QeOahnez3Z-e>r0*ROqnf~T9?U8w^436jtZcSPi zOn)Dmh+wPpK08FCMdlSUm*)UQzxXSBJo6dNKX1Plr`s5jBP6eHGuHj3T0W(f8P0`g4u@Ba@S zbYn^zY*j(fEA`RpM>9GztL7jjf0U(rJX4BOxLCvB|4=U&4spL9Z(OQ4O>VwP;;qgp?P4WmoSMC6!|xy)_^zN6aWR4OM#JCB zdzp{K%Ym12K`*$lxXm5@M3sk090sq98bn0nBXPY!aPp1FyaYDCni!YtsvvzptOX5) zTd)AS{b=f?lT+#H;S`|D#!9usT0jiw&``Lw(iM~!2--A2)PPo3&t|^zG8Gi;BkhC3k@$^Ju#2E__d}f8C9jPc-j?o|+%d2Uha-1NxjoOlyLL=$j7Hv1MwrV5>9cFVm~@p*sX;MbHQG zp^aOf0MV!7>U&V#dAFAu#~zc54f94PBU6Se_tk6od~QT3;~+h&UgL3#w{U@)Z9;g{ zOc=aNBhZlWoLJ`Ce~2?~ljZX%2KANF*L(B*ZXy5Aje#J+7)4}`C{++Ha=zb*!v}cW z7&U;L)6(X1PaJQ1rIBOOIDXkqmSS9bkNMBNCL;H(R{^1~BK7Q0$WZbZcNmL6qyr}l z3k&WX6olU_WOFvr)Zu&cJ={n)bpa>~Um(vsGbsXZKKk&f{^C$ycr7R6%2m9H8j~N8H^w%EIw~6)N3yYxjNjR8eCjd9>H^_}yic(kJNg`H~ z#~a+NTQ=X~lc)kX8marfo?v{r^-!D8UAEjge1V=Nm2XQ_PhmZ17~#NtKC9Y8og5I6j9pPWG%5(@+j@)gN77LS_|)dEG# zG-Y-VG@YHLr^WtlQ^Q&NcfdBa4Pb0pW1iL2T@5B;ER;|n?30@LmLHw9@$&q5r>*Kc zIJf*A;~9=HpTP^V`Ulx`Md{!`p-#*Wi#~k>d4rb_G*$X#0?$6xp#2FGTwt`=m>Yx^ z{C(%?I6QYpDMA?MYiVw+(ca!|3MK(?9vLqUqqEc7fr)Z(u0;{)y~Jgsk{5n)KmaM< z9B5pce$Bc0AN#nHv}o=^dAs zELyy2XZ!EJD_9n-+ia~BRr#m#LWqp+2;@2<^#ogP$W2zo{0c|u%mjL4)@u`7$XcVUTH46bA8s+LWgFHzIM!| z2=*fsA#&CuO!HJ1UdcvD_y#VJ!vJi|?Pjf%V zKQ@`QSYUA)*}$mPN5Q7+(lvY0Y+ zm0DwOtgKDvk(7-$hPl>NGrADKE(e_Kku^E4J2`u)&tqt!hbO!l? zGn!<1QSANLXLZvM4Pk^X%}Sqb*E9N5$}K{S>1!$7hW>%Imkd`Vp>)%>XzbG0g_!G= z3nx$gw6}f0inD~P>Z+>F@lPdEJft`#daxXWI8T3jBt7$OvfPfNX0tl~#GK@>-QI%Z z6TBJI2q$4ZlJ=2jys=CT?{M!es9pdrLG-B47X-;n6^xKxwyFGZc^rQUI$BzI-l?7e z!8J8(EKcBc*Q;Q!i~iTv!ezV!EOTqm+x_2(P%UXbS{)4(ETs3OdftaP@JbWRqJcC_ z|B>l*xZ_6zgbLlm&cPXd@1&-V1xlu}kkz@*1(=ez8@}3>x>2EiorZ{U&T89a<;y~g zgV!H9aqRAIiWf^%LgwI@oJza*J0^51FTG$aIyKsJn@v?_G?>!JHWJ?*3OcW_I2c@nAy>=q$89ZbjI#`FIP(h`tv+7D>+d7KdoW1?zX0T7 zvXa}&l{AK{*2{n+4jkKGUKt*JVhray2>}_7okp+!P*odc&Yqby3jy<%Bk}DqP+IiC z=NWjW^^e79Tczp)5K>05>HcJ>Itx%Z|56uYcD=*spUm2-T)brK=|P@i=80#ny^$m)84$+ zhb@J5E%kmodj?P6I(JpVeY_216@Ah&rU?hV5zGAmbvJ~bH9SUr#GX_#4xINV-qd7G zh!AG1xe%6;ol^EutJ0-mJBN)k*<+acd}m9coB!E8tf_Df-_M==cMO5gw2?jBrKs`E z+FuWR)!<>HY4kiQun^B&m)9v|x(T+i(mkY~1gKy3mPG%#1sHxWs z=iXkauzsMFuT5H({tpIIO&I(H*`1^S#Br)>R*TyS)UG<=y|j!H=i6^h6>Mmq%iTsj zD?W%H;xZJ`#de6TcEI&(C5W*dfwBfsrBY&23`!=rD>RHBP4Qx--IgDk;6PNsj1c+7 zl{r|p_c9{@(To$7b zLttect!z^3eg7kRwV*nNcQCp&pWW(`w;s7D!gfooCF z8SMHX>H(9>!{uJgTXJYBws0=kksB;Y`MNJk7=YqbXYjy5 z#W>1w#dBnLF?f{Jyy5-klF`%`1_(3KCUair)bt~U=xkziB9#kor*u0w?PN;eDfLmh zVZ0M9m3)3y8qw3g3kv}Jdn6h1`gbCps*qF$XmPrrW0wlmD*-Md(EYgZJ2>Ymflopt zW(lM@R+HAcybY~f$f1(Qd1E6QL1s1f{7c=Btj-x%fAsR*LWV-TApZc*ZFVGTQFnoc zEb4_LN&YBX1B^(`wBJs7tzehIeL6wd5uOZk;-rafyp97x`QFc$G2E@2tfndEojIGI zp4zd!ji0C&ZMAkKOpO%b!9C~w(X+@k^XAJj#mI+DY_k_vdUYC^VX;S^+%GmPm9rl~ zVw|lDkV)_Q*C|wpq^>uzY^QeUDTCyuV8>mUO+>^W)^SEI*_k7b$IwWwuJzxMoIU#n zFc*LES+MMyR5HKHfvq&(;;1m<&R%oX64Z=StU_|miNo#sPQW_Upfx&GfydIU( zO5hlJk81~q&s;_8_>WP@<45JWpU`4O^fgF_VQdX}vJ{)-`*@eSzX=ddD=;otg;5AP zHtf6gnh6rWq$$5CXx36>!U@`wZa8Im3fH?g{629~a3ag#bquzQw(l}36862lCG5(7 zah|2EZhHeOxB4~}=gA!wn(b9;Um3piod^Wel3qjqBSmI9Tf0g`?vT7qrT%sorO%CK zZsK#~h7iZhx?I=_4UtdvXTpI5^aF4%t5Bac88yG78RsxJl!yGzc+CT!DefzckoN@v zJdIw#97$cB4rFi+vaVmsj=R=CyFer8GdT5jlb@oU1J(UQeW&Uk zsK(O&=4L#ZT4B31FCkm`W`%eV2Jy|!>Tb;l2XWoc|Boy2Q^@m0A~;r?EQiV?141i5 ziT^#{d&jx=oAF4A5@-M8R>>>vj7Nt?LaNw(2F7P!9!wCICZ;kgGpl(mv=qHhGseoT z@#ePd{9aTH4hZ#n5vsz;ici0iWeBrcK*aHBr;^>4${ssTS-{*4nBv+<6s^=TmAH21 z7g{;6k$g2?!ZBI7sV`>;RDKLNpsJ$quNff%#K~%otUrFeQR|gsVK3O|cPKPjJEu(9 zS3kx;MTeXxz67Qs zqn&5z+Na2sXG;1^2K~bxMj45St!PxX!>@9?7snl910Rr4wdBuJz8JrMa#7G~+w<+p z57#c*7o-?>byh*Fr~4Z-?w!rEW3;L>Zp8`Ues7%?w21aCd4vD}TSKDrH3tB&RUrVI zj_2j^ye25-uO?o7OODhR_)MWcN?tekzn>6xOvL%RQX|I-2q- z)}yk~A7VNb{H7j=))q#au=#T4x|D-Gmzs5O01Q(l;amT*aWS>G)K*rR8Tv=2iW@0@ zYqbJa?`|>*72p_^CWV-eb&1w;djr+B{K#>ONxPokxzT4zx7%A3CP@K_RK4wZWI~O# zV{_v8)jPqzs~1;BzXe=Y5}e?VfUt;e?%HvV(5_VbX0v38{S_K6p}IoLk7zx3i4jBj zu*JUq9*?V-LEkr}4`l27<~@wBC6aF>cQGjsnH?rw9b-pzzf&iwA^;1j`QL(iEAzle zn;jz9B(zvAk344y$`%J*l~^PMS%*LV2U)jM09-VC8W2=){7$p{^#!b{;n8)VdTB1t z{ouacM99iX7%D2#PP$2!2p~M)AUn8eNQ;x%$m{QKLwBw}a=kQ5T>!y!-GMX5+y0@$ z9<~!-%?FXYIIlqjRvUsSHi%qY1H`VRrEM(`B9SALiv=L@9P93&hA~|i)Wb6!TK|)3 z>J`aY+*8G0Ra*0f{8X9Lqjr;Oq>qCK^@N#(ORw?4Wlg534(-wkLuK~#9-@Jo3QVk)G2WGp(<1W_426_ybe;e_7jRt56=j6zeAaywt!ZDOSI5 z_wGS_%Fd^2jdwQfQ<)ELI$34!o;VM6Zk||rZFt=MCUD3=(wAgp**VqH=knRw6!c${ zAFk+xf&Qx$>c67=n2i#V#`UoY;{`?LA-GkkcBT`hS|(UcSxDUWA2p%IO+Zm*n>q7kKqm5Knwl)l$VgiPcR}Zrrzj>(p zEiqI%DPO(<<7za%jfY5*e8`pcmJHdkGn&Z18olH9>!-~%Q$j>;v^>R4iKamJWSY@= zK}V%W;&P|uvsD+N`Plw}VlpH&y=eI50J#Wlbi~p_=n4ivKW-b9#$0b$BAuMfCuo8O z%#p( zNl15hH_~8%Qqn0UNDHErQtv*f_xoGx{l|Onx@+Bg_GizYd1mIBXDC71V?e1cHh7W+ z@{+rwTrL!pS)T*z!4l*}@0MORfxu*_U6EWrdh@%z3;_cxP}E)4l({m`tQH~r(AxrP&O={V zr1J|;SO1thYzPdC|5i_9^q8(vH?W!M^DLT)Rh6z4zrkBaeMi!|c_Ea7CWAa%ExRfDD#<*gXf$A%hOER}@!HDqjK(*4P^r8VoUwFmpVWFqqW{g*doj6AKBpW^9M1~S zVtEKifv!1)&I8EJiGw@9Gi^M>)cgl82SNzd)iOAixeem$##@@PiZ^3bi+qnA^SufG zN4wCFLfoemehY&Ohv8NP8T=TPW)96AgWR<9tqU0GEK#@uqclnG>B(5mm$%#<^n1q2 z({C@@0#T`q``pbm<=gghEkY2)bdK-Q80^&?K0npXBwPeJ^`x-qmk34`v4+`OpO?TQ z;f-ATIvLK7XJ=sG-5ONbK5yhY1Y6ETKX13=3#og92kjI1R^KN)41;zqrVAJANq}M0 z$HV+H&Rsz~qlBqAN%0p;@uMrftP#Y~g{Rs9ZD+4j%m)tMeWz<*8)QT$81Y)=!0Jql7Uf z;#W=11CbM~E4(kw@9OK_Gho^xW%4@}hq-odH_)f*l-1{cGG*pg&CabWrSjEX{|uq} zh!o=(zT|d8VAzV;eva5d0qBWnqLWcA#jJUZ8M9J!${h3zrb-BQo%Ss{az?J^GidCz z&n-Oqek`pJAP|uK^5M`Yz$tM2Dc7$XJPLZUXfrvc@1{N0H|LNc*`LFTv9wS`c9_LTH_ ztYz}V?u~p(9hl@Pyk;q!!h|XR+=rpBIDC96*vK=(_L!;Ev3p&o{7Voq9wYEM%B^ix zDgw2pn~#_ee=^b1dt6MPav3%=)oVsAumk$KY$hAWOa^@S?yfR%Q9&Q$_Xo~@+Ld5v zqZCVr53PX=3KT#UUKBWLaY(3RRjj-Ie1E5UL5F!)Fgg8TI+whPqdD;Hi{Bd5B`W6l>Y`!>uhlB+ef&^-x22goWXL8a@MkaHQ!eVBfSAw1w(@ySMOV9rRi_mcRzpy zRW^W&I&Vv%HUaeGhj+)HoQ?CK6CV$bu%kGzi@rny`a z`adC(ko1Gk7KVvvyKg;Bn(XlH(olczzAr_2R-oOtZDOJgovOSe+0XhV^?l_!Ah}q# zsP@;=|Kxzf^_)iYq{abdf&Otu{iJsQ&|)9D-V;F^;nOOAn)xJ@>N(u6C6sp#6*9-& zOPZZU6LXlc(v8Bk5BU8pkHvlt@rE=`x``gXBYr$*bJFY)>@6u;8u2_fI$hYK1fh-p zrso`Yd;`o#Kf2Qe6%UbyXN zfz1xocyS?*S|JmiIf~h8a!e?YI5$o6*o7LQyH3WM?PJ0HK+r0DsYM#*y$whvOkfjs z+|lmLCvB@NC&V}#Vr(JDsv7B_^Ji?i>DJdI6pb&UFI;?x{1bt~8ArbD9lTDkC4H{u zxfr4jyKr&ui74m_Y;c*>xvxqa=S4OC3`W6IXw98C8z0xoZ=qMU{CtA1f|#XfDbW#$ z#Aaelo*gb$n4sF>9Vh(ErXYqcgc7^$452$5jnDPK9S+`Wm+m*iw?_3D|Nfq>3{4Ts z+c}TW&X!z_UqSIU3|r(?HUB5D-B=}1t0T{A8+1bt&JESP@}&E%NhfzfOp+v>MJP&L zROP)QpD$5t-uPsjCl+$cU5@{CW9?6kVLlDAmAc#tF2U&9Gd4{Q_*Oub>%0 z($2sQ7Rj`QkYyT zaXUvf+-c+%7_av*BBlp20_7V^@E2y?0c{RAO+dvaZSxu5=I58!9_#+AhENTlG((%k zWsDg4y7@A3^7%mR@LK)FPXz?|Gfk-97|>O21G>sZ-Xlxh?PjaQB5}7HC|xYCnXOCu zK?RKm4=MnpBmDMfb>>0aib8$~=mNYWJN~+^x5~L}IW6RL?B{F$`JN#Kfhap1r?ADW z5MV=101qBG%6W=|TPfQ^_|j`sRyZj8Tsr_b^;}PCGp`Lt)z*H6h|ca{R3X+;k9#Z0 zxc~p4cThPr4lsObDrHgKf!_oDi@<5 zsA|u_|3@Xi71|64Tslf13HUAH^&E)DT}p6bK?+CNhiH_iQ$e5u;f~p)JLv8FW8d=< z%M$*X|HIC)jz^|MgBX*Oeh}+;rIs1+B!BI7%#=kuTx}q)`gCvc^&(1tQvin+JII}6kb6M`G=UXvn zrPTm1368eIi&*JXA=!lZwzZ|wK&$a8mkivheexd7GN-{Eb=8c_`>JNHu_lu(^)!>L z0rJ*2c&K51`xr>AvcJ?EJ*@AVR|pNc&@fs>rRi$2cA6l?8WmtKV?W0JF{~fz_)9u8 zV-}>8@Zl@y72tZm-dt~;_6p#=B#HuM2VL#1n!TpET4dg%p_1WDoAIdT@fxpu zosV;50qBe08*r(=Zp2SKJ8%VaKxGX#8y?=U_J%J55txr)VwW2s*+>Jna4e`4jy{O4 z_`4`;I>e4yC! z(I}jMtk~f$_wYP6gNx7jkXmG#J?WBaIwctlW{*b8wWO{lbv9~qunp9H@t%M7NOD+0 zzdE^oP;anrIP)$y3k(Vd4;Nz@Zy5TwusW}P?}CVvO8MQO z%g+HJP9W1K83Ffd$s2cc1Af=FNh9iwfh&FAnTNCa)3kBTM=z7{?Ls=SC5 z8I+L}C$cXDHp#=*0r6Gf zoQ;j`qc){4J}ebx@+|j-!xICk5VD5Vfe&CX6%;bOK`LBM*Jk#!3cZRYddG1Dl$~vuEZ&w`c!J;4aNbp0;P@eoqBqkOztk z?D#hiys4U02IlD&nO_Lars01P@?!kNwy>;wEok@HmU*Csb0MdY&!(d)I36nv8CoNj z_Ttlzfue2sW;*?>DQv>4?hNOe&$*%8*wC<$SAz0mDW|6sWQ*lVRIbch6sHrLLcGj6 z6Q_leZeD{bzZXbJ`qNv%lUunQakBbN+UAPu8 zzOM4R0^8kCg`6Szq$DKI00>9yHL%#)LZ*8Q#DXw@G#dp_;(Oq+q2j#IGVH=g8yFC9 zAEe>&Be|)mEEj$tHy_IrZ10Oam2K7(1Z&vZ+o7UhbB+N>*U!Z8_40VpD4-QV0W@b&fM$Kmt`6%lrN*{>D*K^KOnvZbEyv@$BH!KTO{Ok zG$?=fvVAZ>;s~i2%eh*~5EIfJp2YJIA7Vq36Rv&$1o%5I@aj(s+p9CmJ2^rKX9=yV2 zeaYF61$8F=dhp`>{H~Rxb=?Nz&CzZ9eeQ6qSfiPsL(LK>o_F{;9mv{@9fp5VLZZXh zv%h}vn^WwIyNzS?A{aQD4DpqK$hcPjHeyC$2m#86u3iUn*d+n}6lk8=rAg_S&>h9D z8K!&O577*erm%)`#Egrcg@uJ7S^(;lj@DJ>|Rp;#9zCpDLZQK8a;U9Zo9 zs$yQ-+|>^zJh7^f!;{hY5^X<;U#AGgjk`WglKsKKw^qfT_Jf^m7RNhNWi>TkC7(NA zVYBy+YF-xKN7#F*ex<)M&32g@OQG#Iwo0erVVD(xn7^QLF^4vTbTU?xD7+Gqdihq3hJc@7gB!o-sC!Yo*rGI~HG()FX=g&UA z7H3tuv=G7vC*-eH7gA$tu|;a1_l=G%Ev)^jy2W7u)apf7J0rPgw=_fY1qcc9!q@v z`Zdqr7|2cYH%5BDV=*@_5n*s5h?X)?E3Y)CyHm< zS(oK~Elnb3y5XOG4jw*IW6RKm@2TKo_7?E^Ikv}!KY@l7CZEh${8oNQ(&rHIqbTV| zYV5kVpI%sOM$Jp|dgQ9$Y^=SdE@~jj-q0SEUzi;NwfFPivPUxh_3KxuqiDObs;tv! ze89lK0MS?r$>A2))o9xS3~H1kB_)LfbTT7A_`xh22qmRQ8^VKJB43Q{#u^9r-jaj~ zdgt~^@|D`2AHElY7c6;?Oe=*r)>-*(V1$5}@(4cge0y@vh!fgTU|2Avy=onIg>zGL&8N3KXt%6r+dmV&fIQ92g@$vd&QtfwZK=J}8QR+20Oc$+E zekV#~weZ@aAfS0>iA^gb89<&bxbZwe(0gUmMZ}b4bZmh-v}7rKHlA@WKPP(!JcVDr zNmpmsPt01Q-<&72a14!Vab)Hfy3XlRo^`is6&VY9KK^uir1&nEm_c`L+d08`q2Zf@ zS2TG8GEimL8jZi`X6Hv2JdN;3LzPI@S^b65R%Y+K1IMo(t*@x3T)Xzym}o-@?5scR z?9P~8x;O~Lf$+2Szc4`3>%{xcR@@(@~^l3$n* zY+JkiVN`P)E$!f(oAzV>?YX-U8WGNW1lqq#w;tjko~&K>f_2L06JEGP7ts8K% zsy~d3^nt;_?_hGa{obq=3WRehH90vFLrwp{u^Jtkpe`pT$1s(l)$iowWV-g~kPSz_ z_h>Yp5u)40rdF$$0x(2_yls|LjJ5u^PunXQwW;L$2f5Ml^(TVv%@m0Um&&`ua=w0r z)e8C#4lKvYeQl3c@wC&Sik!2FPi07hF5a7Rz^fb``_5{kRcmY9Twg=XX}5IfYPPrP z?B0eM0_(v8)-!)61*KFg=IyVao1wwk)l!5ZNseMb-oNwrP9>M2%qZ=aLbsL&WfVb? zg0y}NfsRsb%V)|(bw$NUd#<0LQ=8x3ZZ)HJTbxng8v|AvE%b3wVMS`+sHP!vSnL2* zx~IIFYN*1ik+S`?ZFf&coU8o8tW3lMNIzDa<382xEzW2B=qD9`IueEZmh!=T5QF8e zUpGg^yGH*;HPuM`qyC0QwF!9P^~Hgdj=eooAt{`;yy|^tZY(c+5^mfXL27{(tr47G zNXX7{kS{PEe~)mspk^9Ga6$2l?=pP+o@OY9zWV(`b>z@-wlcggK#s&jU}|2EF?~d* z2xG+e=Y}O%KWp#o438dlp0XjmF9u&%fa$u$feTQBeM?JA6;pW;{Ro!jUG3^Ad1#AQOI;lSn3V4Y07%2ou-xc&&eqkx+kg%HyA9tr z(QzJ1^{eh?xAkQL1Mk~N@}SFw*{4XgxXhjKiS%I`B1K}g;=-N3@ydx21r(W2{xUo` zyH|_jguju)BO`Gv)bZldVNrx9>ihhmbVnR-paqt^eZ+deF9q?XgCY$s8CXYcFnJw_ zQlTUOUea5B<8x9kDtzBmOgRYTj^jX9H)xYFXdel0I+HE`h-y*JT)^Sk+#(2Ele({O zn|(@RO3DV$snzDh`Zg8G-@B*{`F9sTk|<#uyLHayWLUdU+uNT%GvetN_t~dN|L*{b zSgm)#e_d@Xg6wRTo57%wsjV3|?PM`tdzV{1vJdRIJ4l1^Gi+YMajez9jb<3Bp18>A z0jr&-UhxH}VnqQw3aFOpaZ9r#tics2j^ywR&{_*>YSb=E@G{D8Lyffh5hI-Oj^Ef< zG?j}Azdk8z!{jyq(VogI4&x1IAl;4r5H6{^&S!E29|dpoZ#WVxyWDSpZk5klq3_1x!hvA zx74lJdSCp_ewPw;hWqkNcvX$nmi?=u&S3BgX!3yTyd6>Tx=3%!evX}&m$=b(_|0^+ zaW<`HDfrcrkm6YFXL*%;sbf^z_HDU0{!$-IH#7`C<`XqGdnkuAAp?tr>l~j@LfHr3 z0i$75PnQ}Coir%POH}AL_ItOglpP<-<0A5-D(CU?Ua1q?3TV$UTv4fK*-t3Wo0X^J z_)-ytTzoTS-jF!-fS$Be>?(}LuN0=#I|ku(v$E=&{4Q<9P}xf@5~^*){HFb8;gjW7 znWCQAJFGh71X7DFRzr=LjU#E;d`i2KdO8mpqSC4TI>ij&o4BM56~Zh4X!dHRNwyVk zO)i@+EL(nHiSh{m$=g?nQ1ZZa{;!O2+ILA+~Q)pu9OXqUmxCm>UZwP|WY);B;jX)v* zC67J*GH+>w4lkOtMvn8?7?~~_xjd%Gez*&KuOjV`Cvm2j!={hHI zs@eNBuTT!EM0eGYM`KQ=v*MH7PNjvPyM&9vcTvOua20;QIZPEcNXdK?HBJPR>Z^hn zyAOU#E^)Oz~|v32|to|rCk>iy*SRZ~XvX@9t(Er$`pKR5_ngq~ypG7jW!RQcN^ zaD=%Wt;GH~x;Vgjzon|G3RNq1DOcH2V9A3WH555Xl2! zbNu_dwndD-){4zQJ)8#!Lt5HiJ{8iewqHeJvOW7 z4;RW|ue-(5#OdXjG|8d8j=mG+Vr8{h_>DVFW~LT z%9Smhh{IyFYJ+1Csh~Xgm5wB8KX_o$Fzm8S9viHsC+9BVp`75K6S6PZxVgy*v*f3I zijH>3%>zJr3n{_vsz?HieE9&DhFoLTywL)GelcAzuF`+#lkz`PU9@xl#QJ8pU&I#K z3mdaH35iVs=f})m!Ta@{5jUqvrA(!LyU>Bl})v+Zvu zQQEZKDk~xg%Kg7cZB+rdbkDiDxx6rb(7|UnGMb+@)f3PEO&JTK;Uz+?0M!Q$-)Suz zuQaYy;^!!D8>-N))G#%zHgnI}e*DjG2=@Q;8*RJMB`L&TuD^;58eDitGwp8|R=>Wt zeUoAC!|nK0hgqHM<$bM)2dFYMmPVfq^w}m|U7gQ+1wx&FZR~z~e#^}b^_L&x=;nGY zfjOP*@AD|5?4uV?=r&4w9127n_VDbE6}HOIgj%{)9PfF^oN|4inwy%RM)}oNgym*K zW)AC}rXRE~FukYFL;GY7{G1{tOXJJEPa;I+X8=H>sH(U<>%vUk>_x&ktX^9BvA<--HY zH9wXW?nupy#e_dkQrO;7W)Ra_cZLyodSQOt50x&GQl_E7QK*#E#%e_Ig3?x2yxsq-wKu{!5qL0(^rCmI_k+b^ixD+zl@rPE zL*QIzY(UDx^~>i0aFYsJ0C+Rq@yvLD1=V7AlahL>pjtu|7kkT79vGk)2Z!<`I95e7 zVSMcRi@Q@9478Y{TH#K#`=$RtQqq4QsbjcqdXt`ORJ}<1bW#&b6@loo0`-CCeyOV+ zx0iu#eSOv3+C4Vc-{2G!6{doPM?lZRFLbNxZD^L3_@^3~Op69(seW3^r(2X%OjIoz z_ElAf#)kZ5Tm?9@vsWs1@53yuorOkrh)P6$#J`DMR^firFI?PiYPVHpEG^Ir5C{># ztgKS9n9R{3u_=Wc>+2R^M$c!r`c+Y>5j8G^lb?2TD4nrgQUtk3OBEK9adkECq(1jk zh4srJ;I6`qj53bS>o;jc^)vrr5X<}j{OCU4S{sV+nL5i;VLNZk{;oh`xt;3i8|yT> z9B+>u&k5qJ?t2|YwZ~oiK3?hPbZ80Rw~05%V=ST+p@q_;64I_3>C+%^lBsjY06j?p;w2k|Z3(&#M&W?;g!bZBfHj|Aj zflRERy6_=Bw!#ck24<5EJfYJ%9xrPo>~qF}4R8i!cA!u7iQX_Rcb? z{+IqQAiD3gHRrOSWv;B%ZE4`dVaf>2L`;6qnCE(=xHY@PLC4Mg?DsGFp_$EfeF?f} zS6BYmUJ_$MI+~$JJARr)lj)(-oy4KYFA3Jel@kl(Flxot`V-iZXyf#z=NgBSJSd?V zKaA$s@HK%AJDD_%0UPC)!_ad40q{wk%YuLdq31|Q?7I4sC*2VQpIaXlPI4{EPr;q9 zso1_8f&~fhetaE-5WU`}s$C8!{hO)8filCNtnu+N<@E@6{|<^UBvz5$Ug?@7{!Y2) z4ersdBQ^%rd^IhM_G4d#fCmiHdX2Df^HSLaf`DQ zP`ikMf7OYBMXi8)G~PWMB9MXbZQK4u24DX})it746RQCMY1-qf5lNVe+mnP)9_$-P zJ$9-!Hj7gs{Oge`TLo1nq$}HDkLC95m4tT+CNBv-sygho>W0O7MHJgP=6)_wM0<(TdtF6B@lD@zUAV61EP zyVBNF|8qaV|G6KIkvu;09koTn$R?+b_g}(X=dVTB5Y@@pZhWQdC)|~&*QupF^))Gt zzkU5N4Pal7LIMH^_A0yaSmZ4A1o;cQ+t7488pb2*k}ia}xsdfs`smp{bVuuPwzr*A zg~g&Tt>l#vhzil8AyFAM2k9C-orxu9=xKS~OWe_?hXXNPqdg<33MNBDbj{%hPOO~1 zKh{6`04^c!?YMQqDcOrniR(x(qC%&t`dc~g?AI?$e?W-mla`joxgCeP32=587)e%) z^f@{{et$e;9tURo?ud&oJ&5a#(nWqtN=k~1Q)}8m78A2S6*I0;Z(_mwH{Bcf51{C0 z9tJ)t1^@+K;8MUXm2>-dX~Y$#QTR>7KUN-wBEvJ|NQFoQv@k^n_M!=U0^ZNC4(bP) zGYMgge4P@=Ys$wgK2~LQu$O(}Qn*#48a#z@-xm~jBJ?$U#b~oZNE};9m1@3Kg0J#C z#kwvl1HChVkM5e|uqxyL2ZH;gs1w3LuLx=PB%rnswr^A|eXKc7VOpHET(=4K0%@81)54PUWrtGUxNjN^{D2c&Wg`SUIy4t)A6}y@mGZ9~YC-y|`P(3PY zPzzAsI0Hb$b(@mKFYdd2B}mU%IwL&VpIJQ)`gZMcG#($YH?VK8{06gzqT#L#OvZXc z)c`3RVfc3%c6fFU4!9Xm*~ElIMNLvUIbV1@ek}X$E10YZiIlf3u#(<$595oP6}c+Gsv2?U%1 zU$7{L>cYr~Q>*1T7GS2&lU>+|8ivz=&MqYsx((%yjw6w%AA^>OfGXHeTujPoKY2q? ztRK)y_*RG9hcE}9;i)|e3W^9o=CuHFSYe?sWWyv!Rl+zhz0;nQ5kYe31C~Q$1`!bv z;h@D83I389HemgtDUZtXbkf+hO-@#nYSAR@f{lxd%-?Oc?kOYfR6rh1j@93=L-OAh z?@{^>I?7(kJ35a)`Mk_cx%uAVdbJE;*}_`>N}Oh$EZVYXK;4w0+~L#9G6D@0(#E*A z$<1fQO-8pqTz=(m|NIR?`)nIUi(vJjaxm7Aj2907qDq7W3O6Ae#01&qL9FEA>_dw# z3HgN^EN?)bT#7i~w!X#jEyRIB;=mpdZAVdCYikQ27CsBA+DQ?xD1uJVzm`R1k;9;Z z9H!;w24PKpwC9qsNNa~n0K&K03m+y^HO8=qsbl#<5_P$CO zHk9V+xF^ezDoJ_A|6Xi6V{)_4&Wy^Wy#k#$BQp-b9&f*I-S5cgEE7%@8O~=N%T=!Y z9;kV@o1;e7>cwR`9V!i`C{ggy5va+39R0-ewjOGLvYzx{r3 z@i^e6Hu6rrA3iJqXVmI1 zx4jEs1V{@B<3PgI3ewtoz-K6x6-@#4f4o%={Tf--m~J@ZTe?(5LB2oLW^RC~|5ls* z`VK3gprMHFN#iQ9zn>kU899GV%WxBlrZb=d5=Rq{rh4No)`k~5g1dicl-`N+fN;a`22kC|+$* z&_v|oCR>5+noP|`&q>e{FQzA>^FPnGAGaLqyS`M|cHaLHSaE#@Y7aCThUf8A2lpt) zh~sIE*5dOEJ35{5#aU!v_}u|si{C)ma7$cYfiMoSs7ImMi_E8Ja-Gw-MFGik(!_Re zwmwbJjW!`M5w|B|Z_gPBeA01u9VLLbha3m?9usONjeHLWJ*6)-hK*sFgA`X8^RE94 zkm8>KT2DqFhySP(d1WPB2ORj?h#4lv&!1xfBy%k`3ln>35bPPq!28lRnY#M zZpbdMP=C>Ho>)k_d5C&WVs9^uKq@O;RYdQfd!T#2-=;NK z26ymfVg++V4?yQex8PGdv?fHh+u=20q&RGVaUsm{CB9wqQN_p^355-WF6EGk`WmMY-fJyWInX*?A|B8Bu4(P`4}C94=m+r z2iH!1cefI8oPN#ge(y(2x3l)(0B&`?zkx*#JOQce(@z%Kz?WS}8X%qOf&)7 z_0mgcEq=VSysrr7rQ>>FnkDyNsHO71Lk!ObRNXqTCiA|p?up8l-L9OMyHSmR&+3g< zRAgEsuf@zEvbu@KJsgXRB1nZqg?t8^cTd>A_av~7iG7|nc%;YuZpY{PBzdIyzRY;>R%1B3d1UuRoer$0RS`v{c{b_@N?2`eicv z=C2P!&Oont)n<>>aCvxv8ig(`81%(&ZE!xt z-%bn5u-vjQ``npMh9)$w`NFX?jMN<$yr1?yF?%tzDJgIlg^aYAlQuRsn03d1$D&+* zsIYR_vf%m_%-@C;6){ye*qgzIZCIsWN@hn#AQ&zeKs}UzFm43AxnUb1_w;&2^JisU zQ2yUO4#|Hx!VqOolAhIE_q|DhW>xOv)04iyOQHun0wzDb{?a|Nv3uAmdKdQ&*y8|e26M@p?haEsXG$l483=^M3Dhr zi-9RMd+F&r?NlJPJQbFgE0zl0DmID(R6tn_EvmM$&;n$(5N+FcYham4I9No=C~oFv zofhawKv7dwMM9W0Ic3y$IqQx5Da++gD5Vwn=gF9y`iv>%X1zM=LWQH63gJ7GFK-PS zfBaKlXS#R6T1rBVziHUqbdn|B4h>ZnDu8A9{78jg=;=+A9~??u8F+0t&=zCMz~b6n zPvIcn^Zl?3TVa}gdmIYJIs?TCf0`ut-KAV*w<~T(@+Ul%8?7m@l%NY_`t~U zWo!#{a1IstX@O9m9LVATI=a#r9$2?t3-VJdpGJL@8+7H1bjMdn@88=8s$PE^QeC`8 zYFPx^lgp%@7me3ZN%k7-<+Z6T$hU7UKwpG%cUgh^UFNnN4#XycZ7fVH>No}axAN!V zW7P+6_@&5?Gh!b2(HToGa%(+^hKpxE<{{ZyzHHJj$3wx7E-GAKUkBp`w7p<)SutPg zA;_fw(5|x|W6%KKeWkv$(?k}d#=OT2OQ9HJqz5ccMNr_%3?0_%WD!7{Kh>drtZnRr zN4n>~HxK{s&5QAt<-x{dopC zod2Fdkxd_H5`Fi5;u~H}&LaP1gZ2@|@$uM0y{h;>l_+8Rd?GkgUGqvbtpet>nf8Pf zOt5%vgP7&}Xk351nW^1`L|AP0hqv*uB^N5KG1|bEHY(2Ysw8hC9>Pye4Ucr74f292 zPCgW-{EYsadigX@yCiX02M8*1F|fkdj9mm#1WYg48{39r1eYSfJSYP{|I~LF!WXy4 zCQ3{Sg@>D6e$J)M`P!CD!}c2s9Yz zrxRhM4cYkXgK!MW(4j`XYgw;N3BMjV^_Gjey#TfPdyARR1HbM(866k}=wmZb$>g(D zSS;5QsQ^*EUc_4%z7%Es@(9#SkSB!;K|fwLOXv{;q8-9k-udLo6Hw-2vE7+N{guq-H5n2!HWl4jWi+Sh)RLBcH-)Jrl-u`sQtee%+Gt%_yxS7 zk{oJsuJLj!l#9uGTQ&DVZAm8`>8oKTQz^bR3CKJ zE)9GN9)R{2pS<1Db5i$b;Fm-4E1c$-ga80Pjnvuwpztl5z1hb`vALv65bUEzo74n9 z8MJU#l9EU``^TG^MMRAGtb76QskkUs+avvfRbK?_!1}s1u*rtjswx#(QJ@TykvV8j zSeMBEw`>Qq?PS)SSl~V3s&4V;j)hxq*`F4!qQxS>lGU-zZdEWN0*5Pjn)VCu zTLBSmf7E*(=srWk!$6={`>>DNO#vJzq8|)c5q}c>8LiDg%SjkAX(BJL@AP96u^gm) zL|!oU&V3U@L$-D}w|Te8z-8;mp!bNPC=wJeXh-vmrEi;L7W=N-elK@z7cO^pN)acB zz`eJb>Alu5dTrgbzXI|xyVDJC!HM_Dy&9rpYS8d#N+()9c+uf^ccp6f{&3%4gr#O+ z&kb*)i7$ZkA4R`?xuwC9HQ)8B%HH|S{%4-OAy)Z7WtOq8M8#9I|^3_Ph0R>8Ico~Pp4q0 zA$yV>vKChI%ZV;M9^@?0_Cl0dyJ^Snh36fCAtm82nx(&UZb8Dujxmslj#E&9mpz&^ z$}qnFJ!gT?`=eZNsyLtV?_KeW7FOJ0TwJAv`K#_%f?*^61cWH(+l?sT8U!$_pH{7o z<+rxkWz`&+++-#tX+QNGp2wyTpW@;pU-q0>K=_5TH@oUZf=msb0`mFy7hxazPFQTV zK8%jd&J1?2{pow>^kTJE0n0Q!ain>CU#Ynp)F|!;T?M?&93&_k92z1S^%zXyw0G4r zH}21%AbBn9wPz4d>kJ2XSqe5;7s+YAE3D#LBspva`3T5-jWP!BnYQaF;i`p;J`}va z19odLLXzniV1MtcaGICZk;BDJ0OU+Nb>DX;DL&73VR!PsaX_4+QJrljy_y_qBpQkL z{jKk#RiRhPLha$`?XjV1;)-T2pDx}!Xbt)fplq}uZ=LN57 z>t*_C)}xm@6$vH6ObVK5lA9|p3qp`l>`;Sge}cyXA8g+(5y z8JV^xCM4jTvVfTiSx^&9Bs`=bAPBxGwN%0NK$V5Im?{y6_FIq|a0-)Meq#Y={e83C zYMG>FWlNoC%ll`NZog!+H_xovQfu+iRlDMxpz~Uj3)FAvazpFHyWlfj_kJCcUo=>- zsWnw9+n+x7Lpnm?_wT|4!6~Bn((c& z34(A`xO3n;P$|TT#Cr{zI$5-&QSK6SA8Cb_O#rL%Bx!nOW}JPGqgE<(M1uRGtBZhq zEAAe3Z$dt!3~aL}KwOF%yuKfy7HFUKZ;~8uKXtY1Pyyc`)?%0-vcwRiWPA zLSwr8%7EJEa1ibM%O{-QHZS!6_I-S=skDBKIyZpe>5kI*)r$LVunH_lp!;6g)(VBE zUStw$gJ<5*4#m&AfaS%lsv)0moAGvz4KMl ziz>fQfpW@I$UyT&mkmEMkDlN*cfXZzpRnnh2PJl*U{d1sLi&3>^?pQz%vMh=6)^%I zDONA-TO)F;Ht-N@O`f+0EiEk(TT>PA1*k-gK;6T+3lKfEBq5084;EE$QCPkKr4z^Cu5E#&eOoMOhf++i9LY-61QG_-;6NrScjH?G zvD=y<)YqTqIFM4A9m20}Oy=OS<43^VtyaK$#yC3cbLz*gXDn1`=U9Exy+un!za!a) zes+g>^k9AFmbpe!5ih#bJ3vMznGwwSouO&U&46DnU&x5=i9$Ccf^^9c^86Q=#-M<< z5Y6wRUPe~kUsB+Hlezs|Y2xg>Dp&^LCkPDZSUD^rz?;L3pDPy3M;H*Zfi0g8lO;A!O;+*w0SE{vnlvED zy*F2k5zkTH2V{lIW_Zsk6})3#X)!x}>cIZ9N?2@5Gk|2Kxz~dnZ!<#MAk0S?xFp+? zBp*xu^$SfrvyHYCHb2$6^o|TY93y!jA$2TSco|4Yyb@j(oM;{z*olUU`{VMmGA?c> z7qfW<)iJ|JW8rg1`*IMW1yiensu>1f%B0|qT3ib(Q+!p5BAbZO(V~SG2eMEBTz`)K zdynh_fWS*XjAS+-Yk_aUDkuL+Q2JkpBP5)EG3G@0W*p6Ly2^02#U&Fwt{>s;aBZ2S zckg6<9QHQKrw~4}=@>yYIX?dVB1@ZHR8EwLVt>B5GR11!NM`tPj?Wj&D)XKkMp|9g zkE5E;6?)V`CTv`pvO$K^x#{)J^l`lF{r^R|*hnU=pj{R)OSETFj~o8;`y(}na5pmw zI=1ohLX?1S$-gH9{?^~HxtjpP@iV>TD~=A--1QW!i2k?YXq{cbyjE`>={n;0g?TP* z8)HJdA3u>eqd4=1or@+(0FHiu>Rf>}a4AY;NyGJkjSmtj$$HZOjwmdU9BI0Y*mEVO zr$-`2q%sBFQpgSw8AvEqfyT2ukPFt;X-K{S(-ewG<&*brFbrY%qeqXzdYA!RvlRvr zGLh)DLn{ipDSnk)p0BC7R_ z{^=ySnU9nbFIjso_?LuW@q;v|`{%Wkj;yA>g|hLy3!H8PJmnSjc9EN&e#k#EMpmJ!u;>5gV;YCz*?3$*MhMU4n^^9z6Le?+*+Qknkhek@r?&xw__bE`tB8rF&a z^;1VdZNz=>iEPMY+epD)3?|16pOus$_askQs5|giFnM`--8L*lxwtP1C6OtRxi7?n zs*|cUs1X8~r{J;&JHGJaA<`<8_cjxN3ypxr7Ec!^Mdc<{b3_egF+k6%PirmyWVzld z9K0mQ6d8I?AO%&>79C$ceT3@hvh;KElsD~TD7m;}H;I_s5=S=YX{VFh-{*ho0c~7whr=57+#o(;@ z#GcZTF|hKbtLr5ZQSg~$5aC=V1xfHTdI@l}knYWh*NCfUTq0e7=OOYhR<2B2G@2^# zj&5V`ZEUbfy8|Kcf;&-TgmFxeVrKPhC@G?j-L%02-Z@Zvl~YiN$R~_L-C^ZVhdKgI zb#w$wLPEmCb4u3-Xy3~XD;3_pX}f}Iki#loCfOxoDl!t`0Q{)n0V0hQsG{?6cQo@E z+7J0S7QLndDsHMLG5EHbR($ZL67#T<(JA{%6vlWg?dc4%2;;7BY{2h7HxUtFY^C_8B> z`|M;3n{yw$53bR_l*VGvsS7HrXMClFG=TyM8TP-IlLqfsN*1XZ3*_DZzGJJSphoX;pg2IO%yq`|BylQ2gNS40^S8088L;04%6e2K1^28E2aAt0Au9IWHW z7xAqsCvt& zsNOeR7!YJoi2)ReAw@zFr8^`B>5>*y6p)e_S~?{Lkdl<{?gnW@V(1X0grPw|8qPEN z`=9r`YxzlMIeYeg_FY%p{v=6hi1n6xvUyiSm5G0;A_*{HB@6r}??)QMP|FWRHfTFB zF8v$Zsm8R5e7qfjQ8!Hmmy2q(3V;ix%BgDDS>D(P0pNE~P&Xj#w9lqdVVKnPz$c0j z^da?n;UHR%I^UCf^9FPE4>>q)_v;t}eB@t~QoC^dp<&})Oq22xNV`78jy}u*dK_ci zcH-p@LU8IRzF?wug;ckSox)yQS|bs~EWe13G(S=3Ev+l>cmIoXCh7j5+uP@&6sm5b zu)epHuoJB^CJ)~FbDIAUk+ZpEZe?0p+D8(wF=ier&|d&HH|fiegtDGo205q?SVEte ztAGwQc?jYyBQB6=jg74}tN&gFtv@+55HnndwT%o9t7>Ru%+NXUg30$qKjsZLGeOt> zdHG1J@4<#mucvYld26K5zB%k~<21%uQDlrpC-CcmqS#zdkNO(o{BbL~3m7rdf19pm-Gk$t6OY}Q%{s*QT046Ka-^OE$#{oLBu1zJ_^95{_Kfl=V?Do9e zr(>#d%xl8Pe2Fl9W2H2J`%kP#6?2h$qdslp41FkJ0-*^CsY!9lTar ziDgI2iM81A(Cxg9`TX78v3u-xEh96Pv1?bu>5~(H13h>tgSpu}n`&O_&R{DwBjehD z_|=DY1{km56sWf3_OlU&l*Ygo#~GNk+knj-R0*$@Omj{H>D&*X7-t_EaSwwU;b_J= zuv8TIzCwU|4&S*4MCg+^hCW5^iMY9V?;7SH4`O29RO(V3-ff*a@rL$J!0jbRg#pr8 z{*~ANcaZm<1|aV@8<5pyQwhmN#xQ0Bf3nO zyoU|XpW?TieB6!=%hG5$9M{JYP*11|1&VWGQ2EhP7}#Lvnq)<0)YVa947|7NHy)8w zPypfCoFv=gXwnKmZ1~ST(C0rrubR-A?#H{Y*e=my?Ye+ZlCPbDIm_IVa0^L^RRjS= z>ysxvMi+q@vx@&!fxQ%52%ZzX#17FhAgXi+aaNN&3je`NKO7*gY=53NJ#sBnfNAF| zpn6t=l-$E99fm``An>vkORZq1?ItIWh!1idMrs3`8nfj}VxL?mrhUjr!}dJ!oZC*- zZwVChp>O3T>Xl9<>^9LA8_5rJW-t_ zDzvPo@Va=cCLoGI514)&ONav=-$6&zkO;pk+|hSVBkcr-+(U9%|QC<*5}UFM`b zF8W^4aQ&Netz~;P&NU{f$2$kb56J+-x3&%>$k}roLIE-CBH(=<0uc&E@YHWJ$Ox#* z9|CYOGBT2((>T~BGa(RE{$&eLBp?wugWgCQ)_ZV(V1s%u4+3rN?nKMw*zQ>xq?@z( z7zeCGYMpCx%Q|NI^(ooR57aPw8k`Mx^E_c$9^Tabg<`SDFR`K}8-Y|TddjIgN)z;d zlDM%sIbm~dX+WIZQfv@N7N#V$*`qJ->3B=Df@q1wq_t1MDAUQxiay1IhoBShj|?f; z&#fcEb0qdc`9f2j*~{3y!Ll7w49q$k8q}Jc{WqE+*6SH6+j{$6OwY@yJh-#v*6v={ z;q2=tdhoE3#_8)KPW1SJ?B>Gm&jHcc#^~q0d*~ppxp>`B7^MtMY^gHsOASTw8dZoz zLzlV7*yzy8iEo`2rJ=bT*8Kyer^K+SEL=r4fwJ;4-K71^M+={a#ZN}#w*ruB&mDBi z`EUC3)$f^U`2i^~Rz2?Nt5Yv>#K_*Rt{jGo`yZ}ZyUqp?Q15RCql#cElP23yw`LQD1=hsKvCuOxaLOeC%F1XxO*#pdi?i!S`kcr1>vZYwr zXhKYGDc&X-PLV{RLXO_#vquW#8F3Aa?Mj8N5--p1pa16T9wvnKQu)6bIN7^vo}m0m zk+#t5A)b%B`|iOmZO}~~J_m*IlPQcEw-OYpOf3+L8ze(QOQGDQh&A;j_diq0A0a;g z^D4;Z`@ptNv{7IG;ns}yv6PZ==ygQ2x)N55dPm-Vbto0m{q*1~*o7qRmx1DCW5YQF zyYI*(j)4K!h#f5ZMf(t{Qo9csuZRsa+Ms9|{_}Pra1r@<@prV@kIABtpv7UXza?FW zN=joun(@rcPa6SZy47e$XHwh|NPZp+cO&=9M^WR;q3zTjc$Kf$eQ%{IUysSh7OMHv zYXVx&1G_JsXb3shnVL+Aw}Tnf81j!M5#YaYOs^qF%jKk!>P&Lm zt_y$*kft(uZa1Gl`^K);h?3&?i zG9S@$+z}qUQ5%VgUg+{$LhM;I#4sUE9^JI!0a-Zi_{2mA6%u%U#o*99I4um5K1zIU+YsXc zmyCb4RrTX^Bk5APoNYR2VV~{al=Lep@R136^3S5M!eP~Sz!>`aEwX8$BqH6iXum#z zMfS@`%Y#v)Lcp?g|4Fl68IUnd>n%2?!&^?lDm*%#Dio>D%jebnknIK_yp_KD;={t? z>2|mCFgb2Cf(W+8Q$owp+R#F=sYx2#GI*m>+bg(XZkikE{|XdAWSCioUtIo37TDYX z1D=AUf6q8MRPIGSi%QBvLIHK{RRDs`t9YJ9Envl*y8!ys6R@p_{Pyj&#`vf{FiS24 zeWn4}z7#Mqd2jw0Ov)F45Ek4gF7=MApvBj+fU+>@ziX?5{osa~H}Z}7ei=?MWbx3m7D3!# z@=-3xbyD(s0%lgg|3j4rF#L*l#_=W`D&};ui|g>nS_BIV$QRcu+(L+^$GZShQz#y7 zad0|KVRrS_>}8ZtN0t^6RW`H+*+eQugvNT6fHN%j4!9JjmleJ3oyO zl8&-|OuS)J&F;cr1F1g%JlQC?op)cV?c0Fu$N|?h^B0bRsKXfSN}X)+Y_axaZHvTO zZ}Y!t3)X#68@88$3Z}tn=MLKrSHe8{PKQ4skhCrIEo7A86p;_zfc4^jjoME0u4U{L zqDQg+)At?rkxe{7Sh{UgGs51PAiOxW!ZwR2A*71;a!oBJaS}DtBhqQurcu+7-+R?l z$IKlQ__%mQyfF-q;Dio^gnBu+&r6iUzKXl*5Z_2VorS8JzPIolo@7g&<=Wgfv{HDv zZmBfCceW_6a{j$y(2-T*MJ7bRF3?<9dMU5aiyCtx2s!();nh6gEDHtO@CaokFygRf zC3qQ|WY;VE$#Wf`QP&2=uQ0BGuR#fi4G*?S5^`YY;^NAb#wE2eX#ODhs9}?<{UR-> z3xLCE&TvIbFoLh5j96*y(-INAuACcn=HO(nAXOTKj zu7NeuxwSD&?cQHSc6vW8w;5VOw=?;C?)787A>s!E>tlHr#1#sqz1K-LNzwS`a#Bi( zR%6kP4Nf?+%#MaLw!{2^os>m{KIvkHhLdI)+4sf}1`dv^dFYPQ*mIMMz@zBq%kM!W z63qg*LVBU3FWetwK*&>o<%wC5MC}*i;;o`X3Ez5D;}$F(UW@vAgxiUmr+jIccA`8Ctdnp;7HyR#nSrSK_T8g90J%gU>bk>#Prjd3$cL@uPJY+stxyPq(dptob5C-m>QlRW1>K9@abB(oP_o9?t%_j zt0{}5dFPA^!!F{`uMWz7Qm&p4K(hzub<3X=Zs8+Fc~+W^=Gs9KXp0_V0F~Y-PXI`t z??OvTk`i8-e<7FQ@c=slaUeTW74R*t@Gsp%sHHDhH|OtTh8KY z5onpsszUb}=UbztsL2OUa$Vj-EhAq%$gtm0xx56SX<-Z>mK3-rb>|7nVH`judR z+FB43l}Wtz#tLv{jSmHzK-*Ci*q)m4HtO1K-pQEfKJSACeocp6iHtZLM3%ie)|clw z4cOSI&E6ns{0wqy(!VkJHdi8upNAV0H|Na`QpuNrnJdW^kZ43kMP<0T1w%H+3Mjzt zw;-@Is(Tj#mxunWvY_Y!GU*=%8=lW9yHS&qI=14NCXFUlov<5>m#Auf*oDYJSJMnZ zC%9g<7+>hc4YCYc@le$GJpe%#8k=EQ5iG13qlIsQ>f3}^UX;1h`!|)ypNd1v5LI!y zLgAdjyc6;*fdz~FelyXEqezC?X+)9{CO1CFD`)pS8<^O6bK7_yg;TKa7Kr~*)3Q$$ z$0z#F`H!h4NnQpHU#fxVNx9AxAt2rDyu>!;rx|Dm3OTJo>9CTnBjwFq_dqFlAClbX zL`e;gbvbIJxc^FxMP21ZNbZYMFS&rQl@{AS5x3C>;0L6faIv=8PL{r6^ckRhk<6BL z3XpL!s5h4v(<~$QXlDq;(y#(2FO2|5uUVRdpPv91Wtt(_f*e*1BTX+|u+72#S;jUJ z${EW@lSe=_mYGR~6xqZy_U^8GmRha4m?ISXJEfw86k*`H@=gx&yYZn^exNfjMR-|& ze+cZ97~|TEz#{Ad5!r zu0ij~+B47!!EKd%5FgT8R^qpOk+ETGrMR>$Y=y0c)M;N*_Tm0@A}|tYXyl7FJezyF zJzyk^CxW5Rr?^~=J`7;Q2_%AN)!y{P)|g_S^Q> zfDsg?^2jXCXG3gmO_+^WxNL0xGvhJqC(ed!L!K{Xym!LUAEW1^yoR~#%&8t?(x8RG~Au;p0;kkT3E3i*#{!6) zWvKT^wi1ZHu7S)l=K!4SdanKAtuy8{?aVdlBl5%?H$NLXi8zV^rec$;mmAN}mQ-mY35`)eB(AffK$Js6Nf#pJVpzOK`#NE97CWLa<|2?!-VB@U7WR4pR@B zeTwtM%_cwhms!Ds1gvN4pJ!^Cn&@$kT7|bCc!1?LVIhy|du$}&t$WE%bI{gE zb6R!za$r!9i0j9eHH|(?+xG;UHEID4S?YP^@ppjoVghi4ClZ(J-4ei8iEed;9cdE? z4S)PsUxn~q+B_=FP_HpFcYYdXY8l3M5hZiS)D)w*(6P?Y=P6Q~Zofl9uDTa!ELT!o z;o5;k_kQ%j@#A7AoXby>f+Xifg;Xk+uN$hXKM{Xzo=bn=__Ez)5o#|nDoyTxZpql1Hu1k1>8_R3W^uvZU*6+{Hs`#ZtcaJ)MSGamp z_3BVpc!~rZuy4H7=&iG;?^*<;szPOZLGsBbF3y1<$`>{NCs<#HKbbmd^cn`2)=wVk%8uEvfrb*tFl{`scGh@It4*99s zMi=gtf#e5(`y8egb^G@!W8aNDS&;R1yU;T? zYpV_5u>p8@S)C&F{?}iB-Hi&PjVw6(nM=DSxHJ@dxu4(ey+KOnO^RM`H^IxvcSpfF#u%iO?ZgM;70y@XXcFS_@ zOf+mV9{?i3h^MPHu?4azSQLKquBdC}!X*jG<&Ni72u2;I_pM=DfzKNR!uz$-j$fHD zF#tNTsmZeCjYJIHTw($znDN|Fx=C=10pGItcCMN{Z@}jVt?~ZWOuIl7zn907ap?!d16O0vi;vT1Jxv(K0Es)+yMZA$$&=N+UF9%Lbn&wy!BYv7+AM?Lgs{0b2e++uO zyFZ21P>uRh-GAnIzX;ggkO1xh;)iahM=4?-8XJ|rcG9MKddZ$1i2%K8ZwVv(G*@58%au);h5dPrMibR^n7+Zo0t56LG(}!_fUkna&t3W-L!w02 zMCs8%WO7wz+d71LI zFuZaxJ6HaLuu{&{(BYxgOXpjM9My>%K`s1&YT7&2s_jvVc(9^86=+1PncS(~@Q~l9 zQ(28{Qw>)H!iz5b!X0QCZV}?j@chP<)>^F?(jZk)?b1;OL_$6fgtAedE_6^Ua8bqB zo-npO0i4d@_ED=uMjEi?_-(EDzXX?~LIDySK6V$_FYyX9Y6fcp3Bx6ry-tDt{N-ZD zM|IG9jY4Jj9*7d~JypeZQPJvaT&oLxvgc)blf3nb5Vj9Q<0gEE< zIYnOOkLAl=bXwV~ZIIu9W6INaQc+`~#fl%la)>VR^eKupCcYIl=jLlAJ~~plF8P+j zSLm$<+*;q^F{gY`$K=SOjHxwmJ5y=~FGGGpf||M~sYD8Hq!O#>;hF5-AWygTl;iRL zxB$;(b4$V@;Z+h~Rb%rFJrSyiNmG z;zk}PS-EA;*v7cdBlqq4-}AMM#i1yqLR^D#Sr2Q}FcIP>%G)hHJ`7>}n*vi-e;t3vm@q5v)z&L>_ z3yp%X>i+cWioFAcO-afFHc1MtT}HktHbjUn@d*`^xsg&M~}DEoHs-n zsd)VS?I!+yMV^ z?x#8ckm@y9ah0x6g@PiY%NtzwO}-}!Yf}xakK_;oqh{%myBQZCl;Ntfz1|1QEz}ut zz}j^d^$sA-Q*Te~QpH^dn}&jcH_B^}wpN@E9&O?QVP-Rp7}bNJf4C%g(@7DTJ@ueH zM9?vb_0UPBKgCc=d>N$_sqymkS^0+Yz(YTI1>EQ`=S)BmghisLan<&}@dW~O!_j_> zI8IyB%<7KzAj6y~t+LG5z8kq8N+URIuC9U|kP}LN7Or5<_&mH(4bQ7sj75q5a0PSUkEQ|6{NoNA5?t zl2ZD6_-7$vq_a=2$#mWS&o=pY!`~PI0_0~*fV^MC5Wx1!8MICqEt#AopAchdb2BZ- z3}i~izkmPEJQgtgORui;Vu6v^peDs7;KVx7Q6CD@{dKs|jd>#Y* ziL^y-uY!fNV8jRp&Lf1XS&&5rFlBV@k*P?uE&(^WrsZjzQ!6dn%ytFTCcvKPsq6FP zg-bP_H>u&oJ|CJSAM?~kZFd7JZ2+`P?N;ms{6~Cz+xPbnIAaR!*?j_4OrKDqQ-%Zf z(ljqfgd}uf;_zpH`|-@n%~pd$xfsI9_u~s7v!HlZKpjGkiNi7if3|0-fjEwYZOp@? zOO}{K;sO{_J+@txW>c~`SYLRDDKV$RDq6PlF(qb`;E`YrNJo3t?{x{q1~@t={BJ+{ z^;qUvG0jw8)<8!fDhGTFqXzx^PU~P(W4hXwRY?hpjv%Pc-$+O~I~LD~uC|_KSC8Q4 z=FWo1?L;u*a5nBfZP$n(5CD>SGE5%?zCK$sihg+zFq~{L7WA}b&yMyxDU4pW`4KQ? z=>>?ZKbrCd^ZSF;J`?R(3^CN&-QC*OS#$`s_!1<2$JN+{ibgI#o)nthL1T#*g1ma* z7U2T2D%aQ>{l9mw6vqlvKp7JIcwzdrQrAOsaMsWXftu}+ShLrNMd*t#&$(EyO8(fG zMN@Aa#p;1UG}=glZjVKR`wxY&GBQ0ta@T~9dc(}muYGiht4bhX4X8y-a3l{1q*$|HM-1mV7^b4O zh$(}@!aBj4F0Tj0wQb;tU{X4W10kWIx7aRAp_v-Oru?2IuvmI*{E{8=bej85%*4yR6h+m0S;L=4flox2{{y9s-HZQ0 zkXI>b8M9o!asGQ{i!fJ}pJcni`Y&eJBl0xrg+T8HTmm0kj6cLcYPKIhA+OWuTowf3 z5yCWPn4|-Mq>;N#yu@qgM1$aOc<$t&vhMa3^Ca(=sRuG z$H*a6!Vo#(=d-htB&t()QS^hc;8Csvo=;Q%VT+ToFR3JXt9)P10=CAOGR6K1@fK0= z2Tfs!8R?{^aDZ&!>+w|0su9V^{P2q%0jkX))nw7y2LaOlxpTnFN94og$2nlqr_e(N z0254=aloF63j+qlYO1PO!I_H-GFXopt(hjuKqWx0u={i+urfslKQ#rzytolVsCG6dUWn+*?CVSQ-E;tXGHaa z+wXuvuLqcxFy?(Xjj25z+DzdYPWW@B{#?_Tb5g--F`i~!> zWQ^Vd4?Y-@34u5HV%dOURoTn0I0nF_Upa3<4vGZikq$tYIjnEra9}3^0*=zs$9oF> z#Q1lG*2$CND*oqE#`j7CTRRXbYh+}a*Bs03cXIObSI=Cq>TXQNi8E(`RdBVQxb@~7 zo~L$hiw@W!aMmn1`i>Bu<&NXtC}XmI9|&Cs;- zmUEgJO@@QP&!U^f8zxO3i)g<66 z)Lnf#6O71=2Ms;|k88#hG;n-y$q3e`sH$7>XlDN?Itr z&5b1D9WCQCOIBz3U7%sS8%}GhWu?Z&|?E-GxrKW?lY)9>~V7f1sflJ-q-rlLrn>SkIlzOta zw|5Kz$JWee(~5sjqBa@i+ZjV$7b!B@^99y?sv`J_(%P3>KChF*_KaZURV+ne1nyC) znm>ak!YTBU$tci)l@+cIr|wyL$jU!paln(+Zeu>`z5k|un+Rfgw?~B)hQ-0Dxg~LN z{RyAa$je+XV4^s=_G0|ca!U^_TA`)~*JUnI5dk9wv)kc!#4r}Ir8wIeSTdgOzk69y z(R>`f^v`vtu4XN)yl%uwnkG@ggtf?}<0U(AZm>SC{z~US z1IO>1-zmxUlB<}XynFLTu9sKPfdnf93F+{elDB6Qyd)rEXYMRn-)^VBVdC43Pxb>& z1NO~W%`QiCS@Itc{;3%i_@42`4g%@DH*nzM9-B zca>zj{xi}dJf?47ILwni<-FoEB#vIa6h+?6ntibGP>NLLqT$?I>1+O6``= z(+uuPHAFBZ#*?U5;8!20+AslbnT_o^``?*J=Ykn!gS7=*? zT2eeYN7R|4*jgVs%pFa9+d2kU&I|WLm{;d(wWnQ+ogr_5qxWi1`UpU2;! zUS7Na+;;&rr+!ue_crkkRtE{8kBY@it zHfHsRU+xh*(7GjmeDSqpjf#W@*fMcHkFo=mNwHzd#uNiq8T2XBk1(Natwv7MFYeW# zeN@MgH$R4+kG+sM#@s3ioJ$3-^rbwf4LPNDT~kENHfn)I>GEuxW=dWCtu{#q7Egj? zJ7or%f`W~h5N$UccceZ3&8X$)djXdF0;+~)!)TMXdQrYvBr%~}oQExxrRb=wxc#@4 zWZeRH0$;sujM05LIDhZ}H0?Kn`xTk;u<-a+*$Ls{a;o((FZ*WZ6LQ$a(I9n&5yBH&k!05m)J8~nZ!SENZmd4aO`S_uOI_kMXW$ESnK~(7v?*Ii z`kB?$YsBSA%EthitFjK~NiuVRZ%IefPgWZ9tgE*@pENTjGKVv6QNL0#6K#6SoNmACHFFcf5vEhkA^3D3R80nw*3`7=af`rgI*)w$hrtY9PJiPwIT?50Isr^WY4>u` zK}Up0rP^=vlZao!PD^3@?5p?vSXJ=U^A~I|UTANQwr3xShUW~-hYXgDTluxCu?O#-Jp}Od2gNXcJ+{ktU&t zrEJ^GdgW}(@6{)ZX&QV&CNpHX_{`G17dctU<>Lws;XUi;TMZcrlNsQV2Up062O@ z3)-)+PTW<`alm0Tv9&ED<<#9Nd+U>O2WCqlC)o^MqhDyXrKgiZN3&GvOmok9zJ7HX z%95Ub$-8K9vU7#K}xMw17-X>>IU1jBfB>|PQm7{hkxTP zCy2XLKbKb)2y8i7*dfR}Ci@6+}K*XTSRGt8ewyYhEG74%yK# z={SoQtC``8_GU9FRvDCK#S^z2{m=?C{< zx;fdcdFFwCRTp?41}esss0vipN1myQmt`IQ(Wz^EnX$yX@em^JP1n_!Ga{Tk#A2H7duD`**}QwiZvRQk4OPxT=;0F^l1wcvW?ai1s}S08@7y@KLb?924O*yBZ-waghHX# zrE;u|b#y)(*nGvaYbWH-CE7}{X*H*b(f}C={LzcY{4NTcbHCjB5V%z)-i+Q?)~&j= zq@5cNmW_lD%;TIs-im1<9*Bs&@NN<7q*SwcT7vwYNL0orxmA=K8OTpJ&H~FJG z%gjR$V#6ItWyZfh%G)>Yf3Qp&s&qN6slgn$~9}s?F7*)>9x9upjC}Ja!9>A0!u=YW`ZeRK0tyL`< z1JC-n6>cE#2V?$0OwFe#2keHTUq>0M|9V*oKNqS-0Mok zwTY?AYtqYkR)Nc4<6B^t9zI^R1&hk9_4_2WTb*C?^;~w(t_36)hOHchg+v76XU^yw zy2F%S4?XO%RNVO~qZ}??W$ZBGfJX2r#4J7gd;dgavEB{eq6p=3a;G^bc+v930&34gjzoP8F&Koq!n^X2m4|uGzSirF{tru_ES$)B~@h zP=M9mC63FLYNWBR%)KW(Iw@2?lo)++UiuSUI_$d zcsFGv^3{qIp25W5`sClqFOtQwmKhz9vM0^l zS+BoSMl@WdATM9p(WiB7-Uq)y0pXYGFYG;!9!LmOg4S|3QN26qi{sQ*(o)?Y(!i-| zNJ84AdVpZW<#Fyj)#O&wJ+S1XJ#HE+!S3+%sgE7=QxP6&5;ovPZ%zp~CU%1BnZ}9k z_q(G&4#pxl#>aL`U+Qajl~*h^Zj1LMc_P+Hrf>ho;-m4~#7nW<#7ww<{L@d*4Vl6( z=XHXA$3H`H)s+}O`j(yOY~ogI`H*XyXBM%^l#`?IMaP~u%f5i)_x#aW7Lm)EV>%pB zdp=0X;dE#%7L%SKns6@kFY{}YczKlrJi-OwGr-K3LmimkS8)b~cXoD~d+6sKp^>s& zcS2+c5MX40upsPSiQ9aHg8DGgAThMeH>Rc*P5Sjw^9bo9a|FNR^LbHt-1>}lL;n`r zPVsaFXOZs_N!}c>NF0@bok%TfllfUz3wXZucgm9I` z|9S!Un=bUMaJ`Fo&3-9OarIrXrN%#HTzuRzZP#FDuO7(2w>DO!|K;>?davzKBlp7? zjLg2={mhYLfrN-J6VI4KfJO9|nFXPtb9O;8lEi}Ip-$@fA%fzKQxH-vFTm@DZ8fhs zZM{$v)zLm+a28}g0eFc;(Zm4f7mK19$X6(#;rDEH;|~BK`4y{g{kNg1yVqa55ckrB z_3w9MkKk^8M0%p{a@HKmJ!vWyR}oOveL%N-r}r{gkC}*(P>@$d`jO(tsQoP-16#kF z20j+Q72Ya&!~^fdxb*Eqt5W^cTTtdw6M2mA~h$TJ$vDs=@%#`tNGbrT2PHEy}wR^O=!BaAN${_pOeFS zy#QQ33OnGLN*j>C&%SjwVr`TwB@6N>D6$pM4pr|fex?Vz|7hVW<|rr*tR9$ycf)8Y zREu^E;tdJmyI^g?o9~~W&?>kb9V-moAohD4q`3~%Asle_mF|v?9^h1yGsnx5_zF{Q zVk%B<%$<e;XRcTLJy-ar$M+L}>%~OL?R!z~|baj(5bO z4qR_LRLM9S^c=wMCHrW8onSx+Rz%eOEb4yVQOW`hHX;of6cho1bCIU`cR0(sMRA?1 zL$>U!?DfA_hDA`D;FXciloCR|L3z<2cUJmR`+d8`xCWtta?riHg;3tG?yS^mWsIdB z3GJc{*==r{(koI0-;My>x?d^gW0dPzdmRMNO^0EOJnQ0mRN-HuhgDFh@A|D}|IUJ; z3Kp^i_{_NgNN*W{9?yBH9dMo!31J3hW*_5(1`!i9rlw{F@BhaINJ7q+0*o@oCn+&; zqnJaevA80g$)L=}QW*&2$7~L+t4&enFl~e7q`cqC>VRIQL?FC9Zq5H%ds~}klMI41 z(AkV{sQ(lw5n`cCvT}NQ3Dr`=oA!b!e=~0C7W6TbkdHFLCiTa(sLa4Sx}~_%J3E{a zN5r?EY1*WoGw>SnihhJ}T{*dE2fP|y z?z}C&StgZe-YNNhyHC3^OJIq3s;T_MqEXRjoi=i+noAQ zX-G80`w*1=H>AyU;Dk zBs7t-Fc_?;UL%nZ5^YB0YrTNMcl81#|K0)#oD}-h2SR#Be5?%evY9M-CftB4MsVe} z{KlJuhhDRY^F)Y(a|9b@QaaH3; z-nD@}U}^(0nXeKF3yyVkbXZBsJ}`eD2j%rH*i#z~_SN>48Fx3Xe-=t-+NPTAXHon- zf17Qw-#fuU2sjh07nCfI>%Twzw(mT4)3_9XW^!4|%F408-f={~>%({dKHwd*uyDC2 zZka7UA)-}G~_DgD4dWQG*(%`M$hz`|m4+(Q6UMGZm zsBn#Y=#Sa6iApnl`vPiYUv$`%`m_(&x{*P-PV?XS)5a}in^1^-s77c?FSb5<;EUE! zQ%f@lZ>ASZum#06@E-XD?3tc{bsH12ExITu(O9_Hshl7U>W$25^DH5DI#Uvofq}u1 z?yVJ-Z?+ghV7k-c8&;5i?nDreIZw!wMES6jXV0G50b{&AkkeRzg6~OFz{v~y|HZ;2 z`PS#@#v?He(gELepFOeDQm@RE2{ChNCUT2z& z8e^DX9q+dPF56No_e-{t;*8z=*Vq2DEkcXE4I9h0x_6w01K+q6MRU+*;j}Hv{~q)a zuIf!v;Nc3z&4|gGooXxL|G~qDrItcgfG4xQm!7 zJ-EbdC;nED&y0M*vaOODS)!}6MEyZnMKb&4@;J>&neND4I595jB=0LFUm#s^HET|` znkoVj&DS0IuP_$%28;JFJ#KWxdr)&t7t0iCXldO)-V9R~f)5u&dNc#&Wjxp0V0!l- z`k3F3RN@JX7jnpd^Y*RFhV5uGP2L&=+o3b~c77f)P2#O?g85|c0%&rU!#W@1#cAb; zaui^M7#7zq-pE!LMy4?Ri=#i}T2s&hb?;5%TR4qwSZur~5AhqKXP!>Gr0d8#=z08% z$o}E%1T%GMMNdcrQ6E2Emn@c>bvIq`gXhUIrfn%Rdu8ASC47UpHd}r*{d02b^9wa5 zVk@!KhQ-wxhowF)Q{+j-L8PUqhIdt7?bbZdqV_^8|H?H}|v zBx|L<@u|Buv26;1d52U}AEy%x;atDK;NPz8Yp@loq;rC@YJLCs0v1Zjhb>q<>k$q-nrIvK>EJ$LmTzEc%GZ5vY!mQmUAX>LiLlAT#y`j~)qV?l zWh$ALT8N98glmY=qiUX>7aLYryKag{M0U10enu-b!V6nhq!X^Q0w4coUN7A5&{m^B zVD8Q3_Q&$%e1GtHj(KtmBZF9MY;3-3xZKj8fKq0mr>y=G`1mgX_lMtdVucnIbR;wj zHc341K5fpJCY>%Q;a5E$f$osZb=xj=xirylawsx9YC{DTj{;lQ>`qw=_y%xr>lHov z!31c3JWT;LrSoF{UV!sBFVQC`b|D`L52fTr&)lBUl{8^i8GL+s;JWOGk0&qv(kqqn z#wGRBUn(tEBRl?h=Lc@bL4*XKQl6alPhB?i!mB6cIQYzu?JBdCw`U@($l7n39z}dR zm5qM|7dStfipSA5+b?~N4>d-8UoD=4{W21^`z1{MBF8h4ie-sQLq)fxuvl_i$5fxh z?c@jD#EJxSm!vry@FO-T`o?a+@?bg3}1epCtdWmA^=Aq4I#p?+@lk^Mg zMJ7I@Q#SMV+x;Q@>DHTUC6REC$I6@m^L(ae*`%V0U7xOXzi~m@x-b;b2MG>Z4#tGZ z+z!xhSa{&A?6 z`P*TzqYHvGv*Q6E>*3`SOENEEr~@@hmue?yN@5xTe6H-y)2%a*FW>!%r&0N@LKq+! zg2IXheIWt$ABV5s{z5bQ<99Q$>Y0IBw{O|9fV8_W?%j?V^3IfbB}~Eowr$3>k;QB&%p6J zwztRc#c4{}#u=5Uw&g5Z>mc|UYKr02ZFouJ!>o z*#j`*nYdF$FUB}a)o&uQ=4#0<^qQDNl|?y;6+kJ6b#Ct-f!d-rs$sjnk7dyJrdfia z6@W|wGB~q>YPv#j`CkK2ij>-cs`2IJd(Sidv4uv9iDEsX-f;o`FaKuE5;*kp?5Hv} zC4u>B08bJ#$CPJO>dgloya5emhsTa#i+1qCF6K|!e>NWal4Xpcvfod0;un|`h6|=? z19Upj2v7Q!pIpeJ0a=vhWaY$d>O(p;{Bjq?xd;vyzN8SRqwRaE)!}L*CKsQXW~_L= zzI#Of8Y#N^`h#4AybK(SOnOGcy4Zex#ok)>OcZ1)T1`Wvdw1x?LjKJda|l`)YJp%V zp8tJ{Yv1*OYUoAWj*|R0uG+y><7oL{>4zNJ3l5 z^zOg&ac|xN_XmRzpJ~K%!;FrHqw?R2b7OwH9`QX>yhRvGk*rwKML8#^lPj&8A<@0X za^0cgK_f$x^R{4r{j$+cS>@vmp{nyxQQPr#!w>I{pt{LMCbe-3u-CtkhA{ne zAsM*QS(N|TKQzzG0mp_N!%4gZMm`o!aseYyC5zHKVh!B%LYVcZ<(L8m%tZ&sL@d8` zu?wXu{{j<&2O8Om|EH_#j;FGJ{~W60DSH%>!?8oj7P2xjvNIzhJF`VPA}f1TMD|{h zO^Aw!kiD{J(IAv>{jQsHe4p>Hf1Y0FdA;uL`!lZfe!s6vN0d@#>A2U1xxA;(mx{rw zfGW3_lD+DA{9(pHR&($n^wzYphF)NXd|r4Ey)WrBZL*=A-)(K z(YdK>m-m^f2Rg{jfoNXmB<|>AZV6Vjqp}zzx(%5x1o^`gl@7rUGgkx zXbc&6I(v%)52-R|zGg znl3Ll43Tl2vu)Hf`Ya{qf}e^7pABj|#>!ff_~r@udli31bU4*<9l-H^W(+U5OA{&+ zv2u>V5dz7FSQ0!6_}$AZwZ7-C%&LZ;*XAxOYtZ3iX3DCc7$P6?k+~ESVRY1sNvEVI z;nr&53DmdeSv6UeC#4rpWl%b3)!X86T@SQOSj_!E9`sp7itIL1Uz1DxtJ247O<=yZo!>Oq6VB z;?1I-*Kgw&o)WPb*4uheYcwT5PEl9r+j*2eGepRi-;Xq>hHsEYo=D3husIZN7<+z(N zN%s`f(woZZxkBmfWxKqe!nP^H=28L7e@98Zc%xB8%!;wrS9Mdg#By4e(5)FRJIEqUobhph%L`>p$8@Pr)YJEV^{+K9dCvO*b16H#>l8E&)7_ zVyC$U1T+!)RpamRwQ{D;C;N5oC>%H0R?Om}nxJON#(aa%HqW5Mfe||kYI(o!V*t7H z4gGACZ(nAbZ6F^;>GV69jdQ9MuibGFfK;*msUA-U#j(L0kvlGSa=85MU}e$DX7`q?vJE@mJIY+A z6u!hcq8}}~n(OOleo6AQtUIoNx(vYAhuoLJA9xN~ieYpw!+-+!R+gJQr8?KhUwoLo zMc6X|6del-_Fir~JLPp*jO)52O=V*1u}<+=+wh2?ixlmr3+ab@g)xbLynguRRYoUH zcuqW@J(gg{T~xd6@jla7kw01Tw0W({=*0&)i;Hcht$4hIBUPSgdIkiL2G^!RIpcUU zPr2R8-ad~8ck~!NeHM^c^7>qr(-}dBZ9O~Y@u5HeKa0`+-`NPp zLIHWlpqkQL1<^-MjSk)*SIXGF_bCx;>f{;yR=OHBmHzqY(WR~VdnZCD z`u~nZJBr8uUf)tpm}q>sv>`P028NY9Mg8u)xw8{n)NC)ftY(mRz)B+LpE%i;z^lgcQRYM<75 zJvX)P88{d!Kwf5P(NO4^c2c}IxkT+EjYyLlSCn7`Q_N()^g(Xrj@+=HF;*Gdnm;7z zSdweV@mvuMF)bU%-qCqXe3leBd`zZ4nVQEug)!#@7krl3u#%G0XE~{l-B?C=KP|7?dikCuXjeI z;wh4kGw{n;H**U0?NAzpiy0W1p7Q2%AWdmOxdl=!v0k5$v0v#v+mX*%Qme(s#Kb8i zq6)OEr!>q(j(VP+O9gM<;1{|{jJW#v&9w;uK$Y}Z?T%&8l9Q7=xELEJOb#BQ`Fa3F z#W*uP%>jc43+AD-T_-)*D-QEJb2DD*^M_dA8a z2ZyU(%isfSQR3hLD%PU{CLBnet~pK1k|j72xXc4du?bghfO= zYTTJSb;hBO$DoQD#X+$9s*~JsV_yEy++r`$Z{Xz-hAzK3Io=Cz$IDN`2cbG_@Mc6V zs0k*s&OzZk9+1ZiUikxLnP!Tw3kwS~w{l6MJd)UnqN&;0loVM?sa@Uz8hz0PCUG#H zw#1PhJSbJcs>QX7aASz?alj6oz3$cX2ethnjlHGwCIN%RE? z;cOQb&J1kc{|*$J7rdmaHO0;`u^ypu6(^RU)_Fm_)R+6pPVDqO5MjGRdx3)5 zaBAw}FC6wa&g~oS2@Tkin-di0TVvM%sJ+>cGe(bh?W!;tWZ6~b`h+w>>~|3(p?*VAaHQ&R;8q)k_!1SGlJZig`B!7^cqv)g z%f*dI&9*cJA6q_+SYTL(a+j(X#qTEh2A&dVT6eIh6x^ErS+}4_Q$pwji8C z-@m^<6;u!cb2-#JS)qg(|BtdCT1L=(<Fpy28>*=I7*x7==hRnp7W*P3Vh^+=@Qz3C#~>X0n=vUPges(;SpQd8e;!G&9gh|ytGOg%gY z!8PKBo;-UdXx$nSf3l<={FO0o|He|#JgHmA^?7#5%41!Tq&LgLEWrPy`x6R3=PZLW z-mO>i98UkKx~Zg{`rg@kHO$V=+I^v0 zNqwgQK}G)WHaligb}ZI_&kIt;0l(PTSQ`MHAs^Dz?7z@D04HXzU>qgK%;AbJul<-^ zw3PW$F0w?gKGnlg($-7TI{WhY?~VQv4fQzYx{8~v4E1+3LnAKVl^HvZUywyw+?E9B zYPJCfAzeK`&U5ml-gHK>hwf%g(t;$)l{@_>`|$`VKQ3XgZfVHCG1OG_e-8sADx#XD zG5f$1m+U@yib> z29LC&n}(~iQ#Bd{c&d!+O1B;Eb!kLD)SY4R4rYx=UtA1&+>XU>DUKR0>nWq>a-vD^ zIy?Ih9vH;dn!T9F5NkF}K$lXlD)l)dNzxAJ{@b7dI3F_d(Gv!98g#u1+Pm&h*-Fx| zw7HyDU&j4`?vWpvj{D``Mv^CaP416cE|r5qX1%i zhhUwCS5_Cuc)PW0u^NiAq=d)=>nFNxli!s(j3z%4K8!!KPcS%&{@hKZ7h@X? zma8y-2wp|k%#2@DRP;Vm%#k&ADmuDM-8=}o%3mu7KZHgXj{pZOuoa?WCB%qws0lS9 ztKr6*EYasH82)Jl+YknCthQRgg z%ZPLQ5FN0(L|-EuL2fj(`FjTyVnKw^?JlPfCc^xLq=%Y!u)n|SuXX3=ACiCcQLe%Y zPD&ZaSS~^4Sm(9Eg>lIwAgZ?ZK;o4 zDdh?h_i^|vSf=^7=C(Np;T9hqV=9!5X}QlS%b|LAvT;VypM$q=pCEBn-`eJL$G(uPK8>Z_wLHBgHeTzx$fu6*j~d6lZWqi$CS_rV;M`?2~; z%fSSYHY1F%EVG$)fXN%e1BH)lQsbI}Bl@;PASY$$JmnO&;om4T4Iw6t_z5E3K&?eVLWCmJtellHxlsf1BF zW5VTnM~Rntl) z+idh+JcdG9MWui;`QWK1p8!gpb2={TQ_aD}HFA$fW&ezp^P(P9Y?M`$UnpyLyn6k1 z_^jdYDm0l+={2Y7^~lILzi&D*42&b$tE1yVK$3Q8HcP?#%AGi(#7?B=!`(Pw0v!&@ z-2|#bo8aKVHL|fV)d$Ki&-;imKZ$f1$WRWC0jMwUSLFTz6Vtc`$QdYR02rmiIvigS zV_;asRJ&zGdZ;`Wu-zpZdOn%-7ETDg{v+OU7e2c-Qs5FH6KdoJCk&D$--Lj zc^!^aixUYQk=8aaT}-LpyHE46GN_Tb*kqTBrzki5i~DvHAQMpf&S$t_S2M1CcxFPO zc5kvxcz^Y8Mc}f&Rmb&-sv4X7#|?<9+M8$S_lbjs$gTAFIl+qX);GPk(%+SQ`^`=I z~uM8TSNjx2mjs-+G|e2M{FB3N30 z5VDX_ZN{Kcc9&Zg3H2K?GBrI3Lv)vg1KKKSa_f$P0u!jBCIF!MNeK(V)gmzR}W*RMtBE}a?Ak7%HU!+p z=(R(H-*Bd6O-1M)w~6}OWiI1Vz&~xq>Q=FtrB8*OgUs9sm>447uSz-(>Xv?xTH&t& zHtE&H?zDg9t-DXXbr=d46YjKQga>a)68GIydZ{A8die0+iU*1h->^gCs>hyrO6;j$ z9d%w`NJxl@6}C59XcI>MK1M8$SEcR#NkIOBtD=TVg|m3`fABx=F7K_clk&zu8$^Pg ze&0Sa36`l_O6{Ubsut+r@TsY(_UPabQs#N1Ro72J4wGkgbKEQcWJMQKLsO_X2O%H@ z!}j$qq}zA@Qrt!nTn7ZxGV`;uZF(1FI6>mlx9lFSsOSUwjy|7t+d-^u+LTtnNgWlQ zfI2s57kUPUV=eV$1VM#p;WCZO6TBHfPFPb88JxGJm6>@$@w5ug+t#PQe=afb8)4wp z25$&hL6rs6r6cyEtC$cIP(s0M>WYA7$Z?lGH8vvo?VqjIb7$g5SRTI?-M?k<_%Wn3 zWcu7)hhUzGRmhssj3Bsz>mLO@>2-ilc|7?}uSB69oM}!@&SekBgJWTKR3xR;&1n&J zLY$oE`S9I9MP$=Q=hV2DfDaZHhRB8X_0_nS7i z4U>tIk`66RVn>4%o?F!l}*a)i@3ia z@$W@X!NQ_16o)Yr{1FY#vf81a4=HIQDRY)O_6^fCw)U!GeR({*wgWmmcN{&{@KX44Y6!T8UiU{yvvK9Hv#I?D(=3 zq6$t0F>{h;Os#5$T9}r&eQ-ROmR(XD(l-r=(M}{)&Dj04YuB=%&#N>X*xyB2O(KOU zC`;OWn|||sDu^{d&OTWVuDv(^sZ&RHnDU_GeQCaNnARnSKJS1b3FyX$6eJ%h>Mk6| zOM~|g2xKgBu!rJfs7=BT9Fro%B7w%B#eu=p>d;N6AV3jBMpS3ufR*c}>Z}NXZ-Ewn z*}8dNV65+Y{I9XTMT?kKPv$W~Vikq^NUELj?cCq(NMyh$;MPgYy3V%3R7cswB8~|u zAkC_my3_n0+1bx0;F*YocbT{VFIP2jWb-^e6XrR%5Y?K_ZI(M%tz z&!nfPBc9?6c)AS$tco)`D)8=i*LV6L`X`G9nNA&_2J3Y=39l0+m_7_~D?~k>VYDjveS!o`h`9O$n{2UBse;(`i9)SV zx@83EfQsVGvHie%Z&PtIu%=R7p?LNoY}^PDai5QpoL%P7&_Tf@m4CZZ*Ta5)56vG6 Vxt3aaW0MH}sGZkRDps((|9_eBPN)C? literal 0 HcmV?d00001 diff --git a/mypy.ini b/mypy.ini index 58d617a11c..0d7ab84da9 100644 --- a/mypy.ini +++ b/mypy.ini @@ -99,5 +99,7 @@ ignore_missing_imports=true [mypy-utils.*] ignore_missing_imports=true +[mypy-connectorx] +ignore_missing_imports=true [mypy-s3fs.*] ignore_missing_imports=true \ No newline at end of file diff --git a/poetry.lock b/poetry.lock index 5c7b40d6ae..15c257b607 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1286,6 +1286,14 @@ python-versions = ">=3.6" [package.extras] testing = ["flake8", "pytest", "pytest-cov", "pytest-virtualenv", "pytest-xdist", "sphinx"] +[[package]] +name = "connectorx" +version = "0.3.1" +description = "" +category = "dev" +optional = false +python-versions = "*" + [[package]] name = "connexion" version = "2.14.1" @@ -3006,7 +3014,7 @@ python-versions = ">=3.7" [[package]] name = "mypy" -version = "1.6.0" +version = "1.6.1" description = "Optional static typing for Python" category = "dev" optional = false @@ -4853,7 +4861,7 @@ weaviate = ["weaviate-client"] [metadata] lock-version = "1.1" python-versions = ">=3.8.1,<4.0" -content-hash = "518cafbab95729a8551e617a770ddd6487a0f4cbb4171c04f3fa7f5605e32328" +content-hash = "7d5b9bfb96bfd08e2b6843df885a3ff605abe603250db78e35350e18bc933a64" [metadata.files] about-time = [ @@ -5319,6 +5327,24 @@ configupdater = [ {file = "ConfigUpdater-3.1.1-py2.py3-none-any.whl", hash = "sha256:805986dbeba317886c7a8d348b2e34986dc9e3128cd3761ecc35decbd372b286"}, {file = "ConfigUpdater-3.1.1.tar.gz", hash = "sha256:46f0c74d73efa723776764b43c9739f68052495dd3d734319c1d0eb58511f15b"}, ] +connectorx = [ + {file = "connectorx-0.3.1-cp310-cp310-macosx_10_7_x86_64.whl", hash = "sha256:719750045e7c3b94c199271fbfe6aef47944768e711f27bcc606b498707e0054"}, + {file = "connectorx-0.3.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:aed31b08acebeb3ebbe53c0df846c686e7c27c4242bff3a75b72cf517d070257"}, + {file = "connectorx-0.3.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:71d2c2678339fb01f89469bbe22e66e75cabcf727a52ed72d576fef5744ebc58"}, + {file = "connectorx-0.3.1-cp310-none-win_amd64.whl", hash = "sha256:92e576ef9610b59f8e5456c12d22e5b0752d0207f586df82701987657909888b"}, + {file = "connectorx-0.3.1-cp37-cp37m-macosx_10_7_x86_64.whl", hash = "sha256:36c28cc59220998928e7b283eecf404e17e077dc3e525570096d0968b192cc64"}, + {file = "connectorx-0.3.1-cp37-cp37m-macosx_11_0_arm64.whl", hash = "sha256:c5173e7252f593c46787627a46561b0d949eb80ab23321e045bbf6bd5131945c"}, + {file = "connectorx-0.3.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3c8411631750d24c12e5e296720637909b8515d5faa3b5eaf7bb86c582d02667"}, + {file = "connectorx-0.3.1-cp37-none-win_amd64.whl", hash = "sha256:0674b6389f8f2ba62155ac2f718df18f76f9de5c50d9911a5fefe7485e1c598e"}, + {file = "connectorx-0.3.1-cp38-cp38-macosx_10_7_x86_64.whl", hash = "sha256:324c5075e8aa6698db8c877cb847f0d86172784db88ac0f3e6762aa9852330f3"}, + {file = "connectorx-0.3.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:027a3880629a7b33ae0c7a80ab4fa53286957a253af2dfe34f19adfea6b79b91"}, + {file = "connectorx-0.3.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a666b967958fcf9fc0444a7b3603483ee23a2fe39f0da3d545ff199f376f7e4b"}, + {file = "connectorx-0.3.1-cp38-none-win_amd64.whl", hash = "sha256:3c5dedfd75cf44898c17cc84a1dd0ab6ed0fa54de0461f2d6aa4bcb2c2b0dc1d"}, + {file = "connectorx-0.3.1-cp39-cp39-macosx_10_7_x86_64.whl", hash = "sha256:354c4126bcd7a9efbb8879feac92e1e7b0d0712f7e98665c392af663805491f8"}, + {file = "connectorx-0.3.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3011e1f9a27fd2a7b12c6a45bc29f6e7577a27418a3f607adaf54b301ff09068"}, + {file = "connectorx-0.3.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f1efb6ed547acc5837c2211e3d65d22948019d1653e7b30e522a4a4bd6d25fa8"}, + {file = "connectorx-0.3.1-cp39-none-win_amd64.whl", hash = "sha256:001b473e600b6d25af83b32674f98dccf49705a59bd6df724b5ba9beb236a0e0"}, +] connexion = [ {file = "connexion-2.14.1-py2.py3-none-any.whl", hash = "sha256:f343717241b4c4802a694c38fee66fb1693c897fe4ea5a957fa9b3b07caf6394"}, {file = "connexion-2.14.1.tar.gz", hash = "sha256:99aa5781e70a7b94f8ffae8cf89f309d49cdb811bbd65a8e2f2546f3b19a01e6"}, @@ -6457,33 +6483,33 @@ multidict = [ {file = "multidict-6.0.4.tar.gz", hash = "sha256:3666906492efb76453c0e7b97f2cf459b0682e7402c0489a95484965dbc1da49"}, ] mypy = [ - {file = "mypy-1.6.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:091f53ff88cb093dcc33c29eee522c087a438df65eb92acd371161c1f4380ff0"}, - {file = "mypy-1.6.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:eb7ff4007865833c470a601498ba30462b7374342580e2346bf7884557e40531"}, - {file = "mypy-1.6.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:49499cf1e464f533fc45be54d20a6351a312f96ae7892d8e9f1708140e27ce41"}, - {file = "mypy-1.6.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:4c192445899c69f07874dabda7e931b0cc811ea055bf82c1ababf358b9b2a72c"}, - {file = "mypy-1.6.0-cp310-cp310-win_amd64.whl", hash = "sha256:3df87094028e52766b0a59a3e46481bb98b27986ed6ded6a6cc35ecc75bb9182"}, - {file = "mypy-1.6.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3c8835a07b8442da900db47ccfda76c92c69c3a575872a5b764332c4bacb5a0a"}, - {file = "mypy-1.6.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:24f3de8b9e7021cd794ad9dfbf2e9fe3f069ff5e28cb57af6f873ffec1cb0425"}, - {file = "mypy-1.6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:856bad61ebc7d21dbc019b719e98303dc6256cec6dcc9ebb0b214b81d6901bd8"}, - {file = "mypy-1.6.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:89513ddfda06b5c8ebd64f026d20a61ef264e89125dc82633f3c34eeb50e7d60"}, - {file = "mypy-1.6.0-cp311-cp311-win_amd64.whl", hash = "sha256:9f8464ed410ada641c29f5de3e6716cbdd4f460b31cf755b2af52f2d5ea79ead"}, - {file = "mypy-1.6.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:971104bcb180e4fed0d7bd85504c9036346ab44b7416c75dd93b5c8c6bb7e28f"}, - {file = "mypy-1.6.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ab98b8f6fdf669711f3abe83a745f67f50e3cbaea3998b90e8608d2b459fd566"}, - {file = "mypy-1.6.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a69db3018b87b3e6e9dd28970f983ea6c933800c9edf8c503c3135b3274d5ad"}, - {file = "mypy-1.6.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:dccd850a2e3863891871c9e16c54c742dba5470f5120ffed8152956e9e0a5e13"}, - {file = "mypy-1.6.0-cp312-cp312-win_amd64.whl", hash = "sha256:f8598307150b5722854f035d2e70a1ad9cc3c72d392c34fffd8c66d888c90f17"}, - {file = "mypy-1.6.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fea451a3125bf0bfe716e5d7ad4b92033c471e4b5b3e154c67525539d14dc15a"}, - {file = "mypy-1.6.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:e28d7b221898c401494f3b77db3bac78a03ad0a0fff29a950317d87885c655d2"}, - {file = "mypy-1.6.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e4b7a99275a61aa22256bab5839c35fe8a6887781862471df82afb4b445daae6"}, - {file = "mypy-1.6.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:7469545380dddce5719e3656b80bdfbb217cfe8dbb1438532d6abc754b828fed"}, - {file = "mypy-1.6.0-cp38-cp38-win_amd64.whl", hash = "sha256:7807a2a61e636af9ca247ba8494031fb060a0a744b9fee7de3a54bed8a753323"}, - {file = "mypy-1.6.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:d2dad072e01764823d4b2f06bc7365bb1d4b6c2f38c4d42fade3c8d45b0b4b67"}, - {file = "mypy-1.6.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b19006055dde8a5425baa5f3b57a19fa79df621606540493e5e893500148c72f"}, - {file = "mypy-1.6.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31eba8a7a71f0071f55227a8057468b8d2eb5bf578c8502c7f01abaec8141b2f"}, - {file = "mypy-1.6.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8e0db37ac4ebb2fee7702767dfc1b773c7365731c22787cb99f507285014fcaf"}, - {file = "mypy-1.6.0-cp39-cp39-win_amd64.whl", hash = "sha256:c69051274762cccd13498b568ed2430f8d22baa4b179911ad0c1577d336ed849"}, - {file = "mypy-1.6.0-py3-none-any.whl", hash = "sha256:9e1589ca150a51d9d00bb839bfeca2f7a04f32cd62fad87a847bc0818e15d7dc"}, - {file = "mypy-1.6.0.tar.gz", hash = "sha256:4f3d27537abde1be6d5f2c96c29a454da333a2a271ae7d5bc7110e6d4b7beb3f"}, + {file = "mypy-1.6.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e5012e5cc2ac628177eaac0e83d622b2dd499e28253d4107a08ecc59ede3fc2c"}, + {file = "mypy-1.6.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:d8fbb68711905f8912e5af474ca8b78d077447d8f3918997fecbf26943ff3cbb"}, + {file = "mypy-1.6.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21a1ad938fee7d2d96ca666c77b7c494c3c5bd88dff792220e1afbebb2925b5e"}, + {file = "mypy-1.6.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b96ae2c1279d1065413965c607712006205a9ac541895004a1e0d4f281f2ff9f"}, + {file = "mypy-1.6.1-cp310-cp310-win_amd64.whl", hash = "sha256:40b1844d2e8b232ed92e50a4bd11c48d2daa351f9deee6c194b83bf03e418b0c"}, + {file = "mypy-1.6.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:81af8adaa5e3099469e7623436881eff6b3b06db5ef75e6f5b6d4871263547e5"}, + {file = "mypy-1.6.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8c223fa57cb154c7eab5156856c231c3f5eace1e0bed9b32a24696b7ba3c3245"}, + {file = "mypy-1.6.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a8032e00ce71c3ceb93eeba63963b864bf635a18f6c0c12da6c13c450eedb183"}, + {file = "mypy-1.6.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:4c46b51de523817a0045b150ed11b56f9fff55f12b9edd0f3ed35b15a2809de0"}, + {file = "mypy-1.6.1-cp311-cp311-win_amd64.whl", hash = "sha256:19f905bcfd9e167159b3d63ecd8cb5e696151c3e59a1742e79bc3bcb540c42c7"}, + {file = "mypy-1.6.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:82e469518d3e9a321912955cc702d418773a2fd1e91c651280a1bda10622f02f"}, + {file = "mypy-1.6.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:d4473c22cc296425bbbce7e9429588e76e05bc7342da359d6520b6427bf76660"}, + {file = "mypy-1.6.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:59a0d7d24dfb26729e0a068639a6ce3500e31d6655df8557156c51c1cb874ce7"}, + {file = "mypy-1.6.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:cfd13d47b29ed3bbaafaff7d8b21e90d827631afda134836962011acb5904b71"}, + {file = "mypy-1.6.1-cp312-cp312-win_amd64.whl", hash = "sha256:eb4f18589d196a4cbe5290b435d135dee96567e07c2b2d43b5c4621b6501531a"}, + {file = "mypy-1.6.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:41697773aa0bf53ff917aa077e2cde7aa50254f28750f9b88884acea38a16169"}, + {file = "mypy-1.6.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7274b0c57737bd3476d2229c6389b2ec9eefeb090bbaf77777e9d6b1b5a9d143"}, + {file = "mypy-1.6.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbaf4662e498c8c2e352da5f5bca5ab29d378895fa2d980630656178bd607c46"}, + {file = "mypy-1.6.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:bb8ccb4724f7d8601938571bf3f24da0da791fe2db7be3d9e79849cb64e0ae85"}, + {file = "mypy-1.6.1-cp38-cp38-win_amd64.whl", hash = "sha256:68351911e85145f582b5aa6cd9ad666c8958bcae897a1bfda8f4940472463c45"}, + {file = "mypy-1.6.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:49ae115da099dcc0922a7a895c1eec82c1518109ea5c162ed50e3b3594c71208"}, + {file = "mypy-1.6.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8b27958f8c76bed8edaa63da0739d76e4e9ad4ed325c814f9b3851425582a3cd"}, + {file = "mypy-1.6.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:925cd6a3b7b55dfba252b7c4561892311c5358c6b5a601847015a1ad4eb7d332"}, + {file = "mypy-1.6.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8f57e6b6927a49550da3d122f0cb983d400f843a8a82e65b3b380d3d7259468f"}, + {file = "mypy-1.6.1-cp39-cp39-win_amd64.whl", hash = "sha256:a43ef1c8ddfdb9575691720b6352761f3f53d85f1b57d7745701041053deff30"}, + {file = "mypy-1.6.1-py3-none-any.whl", hash = "sha256:4cbe68ef919c28ea561165206a2dcb68591c50f3bcf777932323bc208d949cf1"}, + {file = "mypy-1.6.1.tar.gz", hash = "sha256:4d01c00d09a0be62a4ca3f933e315455bde83f37f892ba4b08ce92f3cf44bcc1"}, ] mypy-boto3-athena = [ {file = "mypy-boto3-athena-1.28.36.tar.gz", hash = "sha256:a76df6aace3dc1d91b3f74640d617cd1b4802e5f348a22db2f16dfce0b01ee26"}, diff --git a/pyproject.toml b/pyproject.toml index adf6ff9b3f..242aee2aac 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -114,7 +114,7 @@ enlighten = "^1.11.2" alive-progress = "^3.1.1" pytest-console-scripts = "^1.4.1" pytest = "^6.2.4" -mypy = "^1.6.0" +mypy = "^1.6.1" flake8 = "^5.0.0" bandit = "^1.7.0" flake8-bugbear = "^22.0.0" @@ -159,6 +159,7 @@ optional = true pymysql = "^1.1.0" pypdf2 = "^3.0.1" pydoc-markdown = "^4.8.2" +connectorx="0.3.1" [build-system] requires = ["poetry-core>=1.0.8"] diff --git a/tests/common/data_writers/__init__.py b/tests/common/data_writers/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/tests/common/test_data_writers/test_buffered_writer.py b/tests/common/data_writers/test_buffered_writer.py similarity index 70% rename from tests/common/test_data_writers/test_buffered_writer.py rename to tests/common/data_writers/test_buffered_writer.py index 59a99a4ca6..85cfcb2d0c 100644 --- a/tests/common/test_data_writers/test_buffered_writer.py +++ b/tests/common/data_writers/test_buffered_writer.py @@ -1,8 +1,7 @@ import os -from typing import Iterator +from typing import Iterator, Set, Literal import pytest -from dlt.common.arithmetics import Decimal from dlt.common.data_writers.buffered import BufferedDataWriter, DataWriter from dlt.common.data_writers.exceptions import BufferedDataWriterClosed @@ -16,7 +15,10 @@ import datetime # noqa: 251 -def get_insert_writer(_format: TLoaderFileFormat = "insert_values", buffer_max_items: int = 10, disable_compression: bool = False) -> BufferedDataWriter[DataWriter]: +ALL_WRITERS: Set[Literal[TLoaderFileFormat]] = {"insert_values", "jsonl", "parquet", "arrow", "puae-jsonl"} + + +def get_writer(_format: TLoaderFileFormat = "insert_values", buffer_max_items: int = 10, disable_compression: bool = False) -> BufferedDataWriter[DataWriter]: caps = DestinationCapabilitiesContext.generic_capabilities() caps.preferred_loader_file_format = _format file_template = os.path.join(TEST_STORAGE_ROOT, f"{_format}.%s") @@ -24,7 +26,7 @@ def get_insert_writer(_format: TLoaderFileFormat = "insert_values", buffer_max_i def test_write_no_item() -> None: - with get_insert_writer() as writer: + with get_writer() as writer: pass assert writer.closed with pytest.raises(BufferedDataWriterClosed): @@ -54,7 +56,7 @@ def c3_doc(count: int) -> Iterator[DictStrAny]: return map(lambda x: {"col3": "col3_value"}, range(0, count)) # change schema before file first flush - with get_insert_writer(disable_compression=disable_compression) as writer: + with get_writer(disable_compression=disable_compression) as writer: writer.write_data_item(list(c1_doc(8)), t1) assert writer._current_columns == t1 # but different instance @@ -75,7 +77,7 @@ def c3_doc(count: int) -> Iterator[DictStrAny]: assert "1,0" in content[-1] # data would flush and schema change - with get_insert_writer() as writer: + with get_writer() as writer: writer.write_data_item(list(c1_doc(9)), t1) old_file = writer._file_name writer.write_data_item(list(c2_doc(1)), t2) # rotates here @@ -88,7 +90,7 @@ def c3_doc(count: int) -> Iterator[DictStrAny]: assert writer._buffered_items == [] # file would rotate and schema change - with get_insert_writer() as writer: + with get_writer() as writer: writer.file_max_items = 10 writer.write_data_item(list(c1_doc(9)), t1) old_file = writer._file_name @@ -102,7 +104,7 @@ def c3_doc(count: int) -> Iterator[DictStrAny]: assert writer._buffered_items == [] # schema change after flush rotates file - with get_insert_writer() as writer: + with get_writer() as writer: writer.write_data_item(list(c1_doc(11)), t1) writer.write_data_item(list(c2_doc(1)), t2) assert len(writer.closed_files) == 1 @@ -139,7 +141,7 @@ def c2_doc(count: int) -> Iterator[DictStrAny]: return map(lambda x: {"col1": x, "col2": x*2+1}, range(0, count)) # change schema before file first flush - with get_insert_writer(_format="jsonl", disable_compression=disable_compression) as writer: + with get_writer(_format="jsonl", disable_compression=disable_compression) as writer: writer.write_data_item(list(c1_doc(15)), t1) # flushed assert writer._file is not None @@ -158,19 +160,62 @@ def c2_doc(count: int) -> Iterator[DictStrAny]: def test_writer_requiring_schema(disable_compression: bool) -> None: # assertion on flushing with pytest.raises(AssertionError): - with get_insert_writer(disable_compression=disable_compression) as writer: + with get_writer(disable_compression=disable_compression) as writer: writer.write_data_item([{"col1": 1}], None) # just single schema is enough c1 = new_column("col1", "bigint") t1 = {"col1": c1} - with get_insert_writer(disable_compression=disable_compression) as writer: + with get_writer(disable_compression=disable_compression) as writer: writer.write_data_item([{"col1": 1}], None) writer.write_data_item([{"col1": 1}], t1) @pytest.mark.parametrize("disable_compression", [True, False], ids=["no_compression", "compression"]) def test_writer_optional_schema(disable_compression: bool) -> None: - with get_insert_writer(_format="jsonl", disable_compression=disable_compression) as writer: + with get_writer(_format="jsonl", disable_compression=disable_compression) as writer: writer.write_data_item([{"col1": 1}], None) writer.write_data_item([{"col1": 1}], None) + +@pytest.mark.parametrize("writer_format", ALL_WRITERS - {"arrow"}) +def test_writer_items_count(writer_format: TLoaderFileFormat) -> None: + c1 = {"col1": new_column("col1", "bigint")} + with get_writer(_format=writer_format) as writer: + assert writer._buffered_items_count == 0 + # single item + writer.write_data_item({"col1": 1}, columns=c1) + assert writer._buffered_items_count == 1 + # list + writer.write_data_item([{"col1": 1}, {"col1": 2}], columns=c1) + assert writer._buffered_items_count == 3 + writer._flush_items() + assert writer._buffered_items_count == 0 + assert writer._writer.items_count == 3 + + +def test_writer_items_count_arrow() -> None: + import pyarrow as pa + c1 = {"col1": new_column("col1", "bigint")} + with get_writer(_format="arrow") as writer: + assert writer._buffered_items_count == 0 + # single item + writer.write_data_item(pa.Table.from_pylist([{"col1": 1}]), columns=c1) + assert writer._buffered_items_count == 1 + # single item with many rows + writer.write_data_item(pa.Table.from_pylist([{"col1": 1}, {"col1": 2}]), columns=c1) + assert writer._buffered_items_count == 3 + # empty list + writer.write_data_item([], columns=c1) + assert writer._buffered_items_count == 3 + # list with one item + writer.write_data_item([pa.Table.from_pylist([{"col1": 1}])], columns=c1) + assert writer._buffered_items_count == 4 + # list with many items + writer.write_data_item( + [pa.Table.from_pylist([{"col1": 1}]), pa.Table.from_pylist([{"col1": 1}, {"col1": 2}])], + columns=c1 + ) + assert writer._buffered_items_count == 7 + writer._flush_items() + assert writer._buffered_items_count == 0 + assert writer._writer.items_count == 7 diff --git a/tests/common/test_data_writers/test_data_writers.py b/tests/common/data_writers/test_data_writers.py similarity index 100% rename from tests/common/test_data_writers/test_data_writers.py rename to tests/common/data_writers/test_data_writers.py diff --git a/tests/common/test_data_writers/test_parquet_writer.py b/tests/common/data_writers/test_parquet_writer.py similarity index 100% rename from tests/common/test_data_writers/test_parquet_writer.py rename to tests/common/data_writers/test_parquet_writer.py diff --git a/tests/common/schema/test_inference.py b/tests/common/schema/test_inference.py index be58adab53..24c97219fc 100644 --- a/tests/common/schema/test_inference.py +++ b/tests/common/schema/test_inference.py @@ -372,7 +372,8 @@ def test_corece_null_value_over_not_null(schema: Schema) -> None: def test_infer_with_autodetection(schema: Schema) -> None: - c = schema._infer_column("ts", pendulum.now().timestamp()) + # iso timestamp detection + c = schema._infer_column("ts", pendulum.now().isoformat()) assert c["data_type"] == "timestamp" schema._type_detections = [] c = schema._infer_column("ts", pendulum.now().timestamp()) diff --git a/tests/extract/test_incremental.py b/tests/extract/test_incremental.py index 5f1ab9279f..2b6d26ba12 100644 --- a/tests/extract/test_incremental.py +++ b/tests/extract/test_incremental.py @@ -351,7 +351,7 @@ def test_composite_primary_key(item_type: TItemFormat) -> None: {'created_at': 2, 'isrc': 'AAA', 'market': 'DE'}, {'created_at': 2, 'isrc': 'CCC', 'market': 'DE'}, {'created_at': 2, 'isrc': 'DDD', 'market': 'DE'}, - {'created_at': 2, 'isrc': 'CCC', 'market': 'DE'}, + {'created_at': 1, 'isrc': 'CCC', 'market': 'DE'}, ] source_items = data_to_item_format(item_type, data) @@ -366,8 +366,8 @@ def some_data(created_at=dlt.sources.incremental('created_at')): with c.execute_query("SELECT created_at, isrc, market FROM some_data order by created_at, isrc, market") as cur: rows = cur.fetchall() - expected = [(1, 'AAA', 'DE'), (2, 'AAA', 'DE'), (2, 'BBB', 'DE'), (2, 'CCC', 'DE'), (2, 'CCC', 'US'), (2, 'DDD', 'DE')] - assert rows == expected + expected = {(1, 'AAA', 'DE'), (2, 'AAA', 'DE'), (2, 'BBB', 'DE'), (2, 'CCC', 'DE'), (2, 'CCC', 'US'), (2, 'DDD', 'DE'), (1, 'CCC', 'DE')} + assert set(rows) == expected @pytest.mark.parametrize("item_type", ALL_ITEM_FORMATS) @@ -520,9 +520,6 @@ def some_data(last_timestamp=dlt.sources.incremental("item.timestamp")): assert py_ex.value.json_path == "item.timestamp" - - - @pytest.mark.parametrize("item_type", ALL_ITEM_FORMATS) def test_filter_processed_items(item_type: TItemFormat) -> None: """Checks if already processed items are filtered out""" diff --git a/tests/extract/utils.py b/tests/extract/utils.py index 2465a1b1e2..b109cdbdd9 100644 --- a/tests/extract/utils.py +++ b/tests/extract/utils.py @@ -1,12 +1,11 @@ -from typing import Any, Optional, List, Union, Literal, get_args +from typing import Any, Optional, List, Literal, get_args import pytest -from itertools import zip_longest, chain +from itertools import zip_longest -from dlt.common.typing import TDataItem, TDataItems, TAny +from dlt.common.typing import TDataItem, TDataItems from dlt.extract.extract import ExtractorStorage -from dlt.extract.typing import ItemTransform, ItemTransformFunc -from tests.cases import TArrowFormat +from dlt.extract.typing import ItemTransform import pandas as pd from dlt.common.libs.pyarrow import pyarrow as pa diff --git a/tests/load/filesystem/test_aws_credentials.py b/tests/load/filesystem/test_aws_credentials.py index bf9e0bd681..b4f367b4e1 100644 --- a/tests/load/filesystem/test_aws_credentials.py +++ b/tests/load/filesystem/test_aws_credentials.py @@ -52,6 +52,18 @@ def test_aws_credentials_from_botocore(environment: Dict[str, str]) -> None: assert c.is_resolved() assert not c.is_partial() + s3_cred = c.to_s3fs_credentials() + assert s3_cred == { + "key": "fake_access_key", + "secret": "fake_secret_key", + "token": "fake_session_token", + "profile": None, + "endpoint_url": None, + "client_kwargs": { + "region_name": session.get_config_variable('region') + } + } + c = AwsCredentials() c.parse_native_representation(botocore.session.get_session()) assert c.is_resolved() @@ -125,3 +137,4 @@ def set_aws_credentials_env(environment: Dict[str, str]) -> None: environment['AWS_ACCESS_KEY_ID'] = 'fake_access_key' environment['AWS_SECRET_ACCESS_KEY'] = 'fake_secret_key' environment['AWS_SESSION_TOKEN'] = 'fake_session_token' + environment['REGION_NAME'] = 'eu-central-1' diff --git a/tests/pipeline/test_arrow_sources.py b/tests/pipeline/test_arrow_sources.py index 759d5735c9..016eb1ee48 100644 --- a/tests/pipeline/test_arrow_sources.py +++ b/tests/pipeline/test_arrow_sources.py @@ -1,17 +1,13 @@ +import os import pytest import pandas as pd -from typing import Any, Union import pyarrow as pa import dlt -from dlt.common import Decimal from dlt.common.utils import uniq_id -from dlt.common.exceptions import TerminalValueError from dlt.pipeline.exceptions import PipelineStepFailed from tests.cases import arrow_table_all_data_types, TArrowFormat -from dlt.common.storages import LoadStorage - @pytest.mark.parametrize( @@ -111,3 +107,35 @@ def map_func(item): assert len(result) == 1 assert result[0]['int'][0].as_py() == 1 + + +@pytest.mark.parametrize("item_type", ["pandas", "table", "record_batch"]) +def test_extract_normalize_file_rotation(item_type: TArrowFormat) -> None: + # do not extract state + os.environ["RESTORE_FROM_DESTINATION"] = "False" + # use parquet for dummy + os.environ["DESTINATION__LOADER_FILE_FORMAT"] = "parquet" + + pipeline_name = "arrow_" + uniq_id() + pipeline = dlt.pipeline(pipeline_name=pipeline_name, destination="dummy") + + item, rows = arrow_table_all_data_types(item_type) + + @dlt.resource + def data_frames(): + for _ in range(10): + yield item + + # get buffer written and file rotated with each yielded frame + os.environ[f"SOURCES__{pipeline_name.upper()}__DATA_WRITER__BUFFER_MAX_ITEMS"] = str(len(rows)) + os.environ[f"SOURCES__{pipeline_name.upper()}__DATA_WRITER__FILE_MAX_ITEMS"] = str(len(rows)) + + pipeline.extract(data_frames()) + # ten parquet files + assert len(pipeline.list_extracted_resources()) == 10 + info = pipeline.normalize(workers=3) + # with 10 * num rows + assert info.row_counts["data_frames"] == 10 * len(rows) + load_id = pipeline.list_normalized_load_packages()[0] + # 10 jobs on parquet files + assert len(pipeline.get_load_package_info(load_id).jobs["new_jobs"]) == 10 diff --git a/tests/pipeline/test_pipeline.py b/tests/pipeline/test_pipeline.py index 48368a9df0..917dac75c4 100644 --- a/tests/pipeline/test_pipeline.py +++ b/tests/pipeline/test_pipeline.py @@ -9,7 +9,7 @@ import pytest import dlt -from dlt.common import json, sleep +from dlt.common import json, sleep, pendulum from dlt.common.configuration.container import Container from dlt.common.configuration.specs.aws_credentials import AwsCredentials from dlt.common.configuration.specs.exceptions import NativeValueError @@ -19,7 +19,7 @@ from dlt.common.exceptions import DestinationHasFailedJobs, DestinationTerminalException, PipelineStateNotAvailable, UnknownDestinationModule from dlt.common.pipeline import PipelineContext from dlt.common.runtime.collector import AliveCollector, EnlightenCollector, LogCollector, TqdmCollector -from dlt.common.schema.utils import new_column +from dlt.common.schema.utils import new_column, new_table from dlt.common.utils import uniq_id from dlt.extract.exceptions import InvalidResourceDataTypeBasic, PipeGenInvalid, SourceExhausted @@ -1142,3 +1142,63 @@ def generic(start=8): pipeline = dlt.pipeline(destination='duckdb') pipeline.run(generic(), loader_file_format=file_format) + + +def test_remove_autodetect() -> None: + now = pendulum.now() + + @dlt.source + def autodetect(): + # add unix ts autodetection to current source schema + dlt.current.source_schema().add_type_detection("timestamp") + return dlt.resource([int(now.timestamp()), int(now.timestamp() + 1), int(now.timestamp() + 2)], name="numbers") + + pipeline = dlt.pipeline(destination='duckdb') + pipeline.run(autodetect()) + + # unix ts recognized + assert pipeline.default_schema.get_table("numbers")["columns"]["value"]["data_type"] == "timestamp" + + pipeline = pipeline.drop() + + source = autodetect() + source.schema.remove_type_detection("timestamp") + + pipeline = dlt.pipeline(destination='duckdb') + pipeline.run(source) + + assert pipeline.default_schema.get_table("numbers")["columns"]["value"]["data_type"] == "bigint" + + +def test_flattened_column_hint() -> None: + now = pendulum.now() + + # @dlt.resource(columns=[{"name": "value__timestamp", "data_type": "timestamp"}]) + @dlt.resource() + def flattened_dict(): + # dlt.current.source_schema().add_type_detection("timestamp") + + for delta in range(4): + yield {"delta": delta, "values": [{"Value": {"timestampValue": now.timestamp() + delta}}]} + + @dlt.source + def nested_resource(): + # we need to create a whole structure + dict_resource = flattened_dict() + # add table from resource + dlt.current.source_schema().update_table(dict_resource.compute_table_schema()) + values_table = new_table( + dict_resource.name + "__values", + parent_table_name=dict_resource.name, + columns=[{"name": "value__timestamp_value", "data_type": "timestamp"}] + ) + # and child table + dlt.current.source_schema().update_table(values_table) + return dict_resource + + pipeline = dlt.pipeline(destination='duckdb') + pipeline.run(nested_resource()) + # print(pipeline.default_schema.to_pretty_yaml()) + assert pipeline.default_schema.get_table("flattened_dict__values")["columns"]["value__timestamp_value"]["data_type"] == "timestamp" + # make sure data is there + assert pipeline.last_trace.last_normalize_info.row_counts["flattened_dict__values"] == 4 \ No newline at end of file