From 1dbc04c5429fc9045ce2724a875d42ac2b6a48bb Mon Sep 17 00:00:00 2001 From: Allison Portis Date: Wed, 3 May 2023 12:45:04 -0700 Subject: [PATCH] Support Spark 3.4 Makes changes to support Spark 3.4. These include compile necessary changes, and test _and_ code changes due to changes in Spark behavior. Some of the bigger changes include - A lot of changes regarding error classes. These include... - Spark 3.4 changed `class ErrorInfo` to private. This means the current approach in `DeltaThrowableHelper` can no longer work. We now use `ErrorClassJsonReader` (these are the changes to `DeltaThrowableHelper` and `DeltaThrowableSuite` - Many error functions switched the first argument from `message: String` to `errorClass: String` which **does not** cause a compile error, but instead causes a "SparkException-error not found" when called. Some things affected include `ParseException(...)`, `a.failAnalysis(..)`. - Supports error subclasses - Spark 3.4 supports insert-into-by-name and no longer reorders such queries to be insert-into-by-ordinal. See https://github.com/apache/spark/pull/39334. In `DeltaAnalysis.scala` we need to perform schema validation checks and schema evolution for such queries; right now we only match when `!isByName` - SPARK-27561 added support for lateral column alias. This broke our generation expression validation checks for generated columns. We now separately check for generated columns that reference other generated columns in `GeneratedColumn.scala` - `DelegatingCatalogExtension` deprecates `createTable(..., schema: StructType, ...)` in favor of `createTable(..., columns: Array[Column], ...)` - `_metadata.file_path` is not always encoded. We update `DeleteWithDeletionVectorsHelper.scala` to accomodate for this. - Support for SQL `REPLACE WHERE`. Tests are added to `DeltaSuite`. - Misc test changes due to minor changes in Spark behavior or error messages Resolves delta-io/delta#1696 Existing tests should suffice since there are no major Delta behavior changes _besides_ support for `REPLACE WHERE` for which we have added tests. Yes. Spark 3.4 will be supported. `REPLACE WHERE` is supported in SQL. GitOrigin-RevId: b282c95c4e6a7a1915c2a4ae9841b5e43ed4724d Fix a test in DeltaVacuumSuite to pass locally "vacuum after purging deletion vectors" in `DeltaVacuumSuite` fails locally because the local filesystem only writes modification times to second accuracy. This means a transaction might have timestamp `1683694325000` but the tombstone for a file removed in that transaction could have deletionTimestamp `1683694325372`. ---> The test fails since we set the clock to the transaction timestamp + retention period, which isn't late enough to expire the tombstones in that transaction. GitOrigin-RevId: 63018c48524edb0f8edd9e40f1b21cc97bc546cc Add estLogicalFileSize to FileAction Add estLogicalFileSize to FileAction for easier Deletion Vector processing. GitOrigin-RevId: c7cf0ad32e378bcfc4e4c046c5d76667bb8659c7 Support insert-into-by-name for generated columns Spark 3.4 no longer requires users to provide _all_ columns in insert-by-name queries. This means Delta can now support omitting generated columns from the column list in such queries. This test adds support for this and adds some additional tests related to the changed by-name support. Resolves delta-io/delta#1215 Adds unit tests. Yes. Users will be able to omit generated columns from the column list when inserting by name. Closes delta-io/delta#1743 GitOrigin-RevId: 8694fab3d93b71b4230bf6f5dd0f2a21be6f3634 Implement PURGE to remove DVs from Delta tables This PR introduces a `REORG TABLE ... APPLY (PURGE)` SQL command that can materialize soft-delete operations by DVs. The command works by rewriting and bin-packing (if applicable) only files that have DVs attached, which is different from the `OPTIMIZE` command where all files (with and without) DV will be bin-packed. To achieve this, we hack the `OPTIMIZE` logic so files of any size with DVs will be rewritten. Follow-up: - Set the correct commit info. Now the resulting version is marked as `optimize` rather than `purge`. - Clean up DVs from the filesystem. New tests. Closes delta-io/delta#1732 Signed-off-by: Venki Korukanti GitOrigin-RevId: 98ef156d62698986bfb54681e386971e2fec08b8 Unify predicate strings in CommitInfo to record the information in a consistent way. GitOrigin-RevId: 043a6a4181c112b9c9a45906c1275fbbdbbb1388 Minor refactoring to Delta source. GitOrigin-RevId: 3625a5c44999139ef4976c62473b233167a4aa83 Add Option.whenNot Scala extension helper and replace usage of Option.when(!cond). GitOrigin-RevId: e26244544cadeeff1d55862f840d4c6c5570e83b Introduce DomainMetadata action to delta spec We propose to introduce a new Action type called DomainMetadata to the Delta spec. In a nutshell, DomainMetadata allows specifying configurations (string key/value pairs) per metadata domain, and a custom conflict handler can be registered to a metadata domain. More details can be found in the design doc [here](https://docs.google.com/document/d/16MHP7P78cq9g8SWhAyfgFlUe-eH0xhnMAymgBVR1fCA/edit?usp=sharing). The github issue https://github.com/delta-io/delta/issues/1741 was created. Spec only change and no test is needed. Closes delta-io/delta#1742 GitOrigin-RevId: 5d33d8b99e33c5c1e689672a8ca2ab3863feab54 DV stress test: Delete from a table of a large number of rows with DVs This PR tests DELETing from a table of 2 billion rows (`2<<31 + 10`), some of which are marked as deleted by a DV. The goal is to ensure that DV can still be read and manipulated in such a scenario. We don't `delete a large number of rows` and `materialize DV` because they run too slow to fit in a unit test (9 and 20 minutes respectively). GitOrigin-RevId: 1273c9372907be0345465c2176a7f76115adbb47 RESTORE support for Delta tables with deletion vectors This PR is part of the feature: Support Delta tables with deletion vectors (more details at https://github.com/delta-io/delta/issues/1485) It adds running RESTORE on a Delta table with deletion vectors. The main change is to take into consideration of the `AddFile.deletionVector` when comparing the target version being restored to and the current version to find the list of data files to add and remove. Added tests Closes delta-io/delta#1735 GitOrigin-RevId: b722e0b058ede86f652cd4e4229a7217916511da Disallow overwriteSchema with dynamic partitions overwrite Disallow overwriteSchema when partitionOverwriteMode is set to dynamic. Otherwise, the table might become corrupted as schemas of newly written partitions would not match the non-overwritten partitions. GitOrigin-RevId: 1012793448c1ffed9a3f8bde507d9fe1ee183803 SHALLOW CLONE support for Delta tables with deletion vectors. This PR is part of the feature: Support Delta tables with deletion vectors (more details at https://github.com/delta-io/delta/issues/1485) This PR adds support for SHALLOW CLONEing a Delta table with deletion vectors. The main change is to convert the relative path of DV file in `AddFile` to absolute path when cloning the table. Added tests Closes delta-io/delta#1733 GitOrigin-RevId: b634496b57b93fc4b7a7cc16e33c200e3a83ba64 Adds tests for REPLACE WHERE SQL syntax Spark 3.4 added RELACE WHERE SQL support for insert. This PR adds tests for the feature after upgrading to Spark 3.4. Closes delta-io/delta#1737 GitOrigin-RevId: 8bf0e7423a6f0846d5f9ef4e637ee9ced9bef8d1 Fix a test in `DeltaThrowableSuite.scala` Fix a test in `DeltaThrowableSuite.scala` GitOrigin-RevId: 28acd5fe8d8cadd569c479fe0f02d99dac1c13b3 Fix statistics computation issues with Delta tables with DVs This PR makes following changes: - Delta protocol [requires](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#writer-requirement-for-deletion-vectors) that every `AddFile` with DV must have `numRecords` in file statistics. The current implementation of DELETE with DVs violates this requirement when the source `AddFile` has no statistics to begin with. This PR fixes it by computing stats for `AddFile`s with missing stats and have DVs generated as part of the DELETE with DV operation. The stats are generated by reading the Parquet file footer. - DELETE with DVs currently has a bug where setting the `tightBounds=false` for `AddFile`s with DVs doesn't correctly set the `NULL_COUNT` for column with all nulls. - Throw error when stats re-computation command is run on Delta tables with DVs. This is a TODO, we need to implement it but for now throw error to avoid calculating wrong statistics for Delta tables with DVs. GitOrigin-RevId: f69968961dcf4766b6847a191b66aae7f9ff295d Remove the check that disables writes to Delta tables with deletion vectors Given that now we have support for writing into DV tables and table utility operations as as part of the delta-io/delta#1485 and delta-io/delta#1591, we should remove the check. Closes delta-io/delta#1736 Signed-off-by: Venki Korukanti GitOrigin-RevId: 17e7e9c6796229ada77148a730c69348a55890b9 Regex based table matching in DeleteScalaSuite Use a more reliable regex-based approach to getting a `DeltaTable` instance from a sql identifier string in `DeleteScalaSuite`. GitOrigin-RevId: 1d0e1477a7d22373e8478d7debc3565c092090da Enable SQL support for WHEN NOT MATCHED BY SOURCE The SQL syntax for merge with WHEN NOT MATCHED BY SOURCE clauses was shipped with Spark 3.4. Now that Delta picked up Spark 3.4, we can enable SQL support and mix in SQL tests for WHEN NOT MATCHED BY SOURCE. Existing tests for WHEN NOT MATCHED BY SOURCE are now run in the Merge SQL suite. Closes delta-io/delta#1740 GitOrigin-RevId: 1ddd1216e13f854901da47896936527618ea4dca Minor refactor to DeltaCatalog.scala GitOrigin-RevId: 53b083f9abf92330d253fbdd9208d2783428dd98 Correctly recurse into nested arrays & maps in add/drop columns It is not possible today in Delta tables to add or drop nested fields under two or more levels of directly nested arrays or maps. The following is a valid use case but fails today: ``` CREATE TABLE test (data array>>) ALTER TABLE test ADD COLUMNS (data.element.element.b string) ``` This change updates helper methods `findColumnPosition`, `addColumn` and `dropColumn` in `SchemaUtils` to correctly recurse into directly nested maps and arrays. Note that changes in Spark are also required for `ALTER TABLE ADD/CHANGE/DROP COLUMN` to work: https://github.com/apache/spark/pull/40879. The fix is merged in Spark but will only be available in Delta in the next Spark release. In addition, `findColumnPosition` which currently both returns the position of nested field and the size of its parent, making it overly complex, is split into two distinct and generic methods: `findColumnPosition` and `getNestedTypeFromPosition`. - Tests for `findColumnPosition`, `addColumn` and `dropColumn` with two levels of nested maps and arrays are added to `SchemaUtilsSuite`. Other cases for these methods are already covered by existing tests. - Tested locally that ALTER TABLE ADD/CHANGE/DROP COLUMN(S) works correctly with Spark fix https://github.com/apache/spark/pull/40879 - Added missing tests coverage for ALTER TABLE ADD/CHANGE/DROP COLUMN(S) with a single map or array. Closes delta-io/delta#1731 GitOrigin-RevId: 53ed05813f4002ae986926506254d780e2ecddfa --- .github/workflows/test.yaml | 2 +- Dockerfile | 2 +- PROTOCOL.md | 54 +- build.sbt | 5 +- .../io/delta/sql/parser/DeltaSqlBase.g4 | 9 +- .../resources/error/delta-error-classes.json | 36 + .../io/delta/sql/parser/DeltaSqlParser.scala | 34 +- .../io/delta/tables/DeltaTableBuilder.scala | 6 +- .../catalyst/plans/logical/deltaMerge.scala | 17 +- .../delta/ColumnWithDefaultExprUtils.scala | 1 + .../spark/sql/delta/DeltaAnalysis.scala | 162 ++++- .../apache/spark/sql/delta/DeltaErrors.scala | 32 +- .../org/apache/spark/sql/delta/DeltaLog.scala | 11 +- .../spark/sql/delta/DeltaOperations.scala | 54 +- .../sql/delta/DeltaParquetFileFormat.scala | 2 +- .../sql/delta/DeltaSharedExceptions.scala | 24 +- .../apache/spark/sql/delta/DeltaTable.scala | 10 +- .../sql/delta/DeltaThrowableHelper.scala | 37 +- .../spark/sql/delta/GeneratedColumn.scala | 33 + .../sql/delta/OptimisticTransaction.scala | 28 +- .../sql/delta/PreprocessTimeTravel.scala | 3 +- .../actions/DeletionVectorDescriptor.scala | 16 + .../spark/sql/delta/actions/actions.scala | 36 +- .../sql/delta/catalog/DeltaCatalog.scala | 28 +- .../sql/delta/commands/DeleteCommand.scala | 9 +- .../DeleteWithDeletionVectorsHelper.scala | 83 ++- .../delta/commands/DeletionVectorUtils.scala | 37 - .../commands/DeltaReorgTableCommand.scala | 55 ++ .../sql/delta/commands/MergeIntoCommand.scala | 2 +- .../delta/commands/OptimizeTableCommand.scala | 52 +- .../delta/commands/RestoreTableCommand.scala | 72 +- .../sql/delta/commands/UpdateCommand.scala | 2 +- .../sql/delta/commands/VacuumCommand.scala | 3 - .../sql/delta/commands/WriteIntoDelta.scala | 7 +- .../commands/alterDeltaTableCommands.scala | 16 +- .../merge/MergeIntoMaterializeSource.scala | 33 - .../sql/delta/files/TransactionalWrite.scala | 2 +- .../spark/sql/delta/schema/SchemaUtils.scala | 253 ++++--- .../sql/delta/sources/DeltaDataSource.scala | 4 +- .../sql/delta/sources/DeltaSQLConf.scala | 9 - .../spark/sql/delta/sources/DeltaSource.scala | 20 +- .../delta/sources/DeltaSourceCDCSupport.scala | 8 +- .../DeltaSourceSchemaEvolutionSupport.scala | 34 +- .../DeltaSourceSchemaTrackingLog.scala | 15 +- .../sql/delta/stats/DataSkippingReader.scala | 2 + .../delta/stats/StatisticsCollection.scala | 18 +- .../delta/stats/StatsCollectionUtils.scala | 635 ++++++++++++++++++ .../spark/sql/delta/util/DeltaEncoders.scala | 4 + .../sql/delta/util/DeltaFileOperations.scala | 5 + .../spark/sql/util/ScalaExtensions.scala | 6 + .../_delta_log/00000000000000000000.json | 4 - ...4d61-ba30-9549910bc73d-c000.snappy.parquet | Bin 609 -> 0 bytes .../_delta_log/00000000000000000000.json | 3 + ...r_4ae9b93c-0d65-4b33-a40c-9c13f68a4763.bin | Bin 0 -> 4557145 bytes ...461d-8581-136026bf6f95-c000.snappy.parquet | Bin 0 -> 8473865 bytes .../sql/parser/DeltaSqlParserSuite.scala | 59 +- .../delta/tables/DeltaTableBuilderSuite.scala | 3 +- .../spark/sql/delta/CloneTableSQLSuite.scala | 193 +++++- .../spark/sql/delta/CloneTableSuiteBase.scala | 75 ++- .../spark/sql/delta/DeleteSQLSuite.scala | 1 + .../spark/sql/delta/DeleteScalaSuite.scala | 38 +- .../spark/sql/delta/DeleteSuiteBase.scala | 10 +- .../sql/delta/DeletionVectorsTestUtils.scala | 43 +- .../spark/sql/delta/DeltaCDCSQLSuite.scala | 16 +- .../sql/delta/DeltaColumnRenameSuite.scala | 54 +- .../spark/sql/delta/DeltaDDLSuite.scala | 69 +- .../delta/DeltaDataFrameWriterV2Suite.scala | 8 +- .../sql/delta/DeltaDropColumnSuite.scala | 54 +- .../spark/sql/delta/DeltaErrorsSuite.scala | 6 +- .../DeltaGenerateSymlinkManifestSuite.scala | 6 +- .../spark/sql/delta/DeltaImplicitsSuite.scala | 1 + .../sql/delta/DeltaInsertIntoTableSuite.scala | 102 ++- .../spark/sql/delta/DeltaLogSuite.scala | 2 +- .../spark/sql/delta/DeltaOptionSuite.scala | 22 +- .../apache/spark/sql/delta/DeltaSuite.scala | 168 +++++ .../sql/delta/DeltaTableCreationTests.scala | 3 +- .../spark/sql/delta/DeltaTestUtils.scala | 74 +- .../spark/sql/delta/DeltaThrowableSuite.scala | 25 +- .../spark/sql/delta/DeltaVacuumSuite.scala | 22 +- .../sql/delta/DescribeDeltaDetailSuite.scala | 3 +- .../sql/delta/DescribeDeltaHistorySuite.scala | 24 +- .../sql/delta/GeneratedColumnSuite.scala | 118 +++- .../spark/sql/delta/HiveDeltaDDLSuite.scala | 5 - .../sql/delta/MergeIntoAccumulatorSuite.scala | 18 +- .../MergeIntoMaterializeSourceSuite.scala | 94 +-- .../spark/sql/delta/MergeIntoSQLSuite.scala | 6 +- .../spark/sql/delta/MergeIntoSuiteBase.scala | 20 +- .../sql/delta/RestoreTableScalaSuite.scala | 217 +++++- .../spark/sql/delta/TightBoundsSuite.scala | 246 +++++++ .../spark/sql/delta/UpdateSuiteBase.scala | 10 +- .../DeletionVectorsSuite.scala | 30 +- ...DisableUpdatesToDvEnabledTablesSuite.scala | 186 ----- .../sql/delta/optimize/DeltaReorgSuite.scala | 113 ++++ .../delta/schema/CaseSensitivitySuite.scala | 4 +- .../sql/delta/schema/SchemaUtilsSuite.scala | 584 ++++++++++++++-- .../delta/stats/StatsCollectionSuite.scala | 29 +- .../storage/dv/DeletionVectorStoreSuite.scala | 3 +- examples/scala/build.sbt | 6 +- python/delta/exceptions.py | 8 +- python/delta/tests/test_deltatable.py | 2 +- python/delta/tests/test_pip_utils.py | 10 +- python/delta/tests/test_sql.py | 13 +- setup.py | 2 +- 103 files changed, 3894 insertions(+), 946 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala create mode 100644 core/src/main/scala/org/apache/spark/sql/delta/stats/StatsCollectionUtils.scala delete mode 100644 core/src/test/resources/delta/table-with-dv-feature-enabled/_delta_log/00000000000000000000.json delete mode 100644 core/src/test/resources/delta/table-with-dv-feature-enabled/part-00000-fe8a3447-d858-4d61-ba30-9549910bc73d-c000.snappy.parquet create mode 100644 core/src/test/resources/delta/table-with-dv-gigantic/_delta_log/00000000000000000000.json create mode 100644 core/src/test/resources/delta/table-with-dv-gigantic/deletion_vector_4ae9b93c-0d65-4b33-a40c-9c13f68a4763.bin create mode 100644 core/src/test/resources/delta/table-with-dv-gigantic/part-00000-2bc940f0-dd3f-461d-8581-136026bf6f95-c000.snappy.parquet create mode 100644 core/src/test/scala/org/apache/spark/sql/delta/TightBoundsSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DisableUpdatesToDvEnabledTablesSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/sql/delta/optimize/DeltaReorgSuite.scala diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index f024bdb335f..963b5fedffa 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -41,7 +41,7 @@ jobs: pyenv install 3.7.4 pyenv global system 3.7.4 pipenv --python 3.7 install - pipenv run pip install pyspark==3.3.2 + pipenv run pip install pyspark==3.4.0 pipenv run pip install flake8==3.5.0 pypandoc==1.3.3 pipenv run pip install importlib_metadata==3.10.0 pipenv run pip install mypy==0.910 diff --git a/Dockerfile b/Dockerfile index f8bda1df7fb..4b5164eb910 100644 --- a/Dockerfile +++ b/Dockerfile @@ -23,7 +23,7 @@ RUN apt-get update && apt-get install -y python3-pip # cryptography. Otherwise, building wheels for these packages fails. RUN pip3 install --upgrade pip -RUN pip3 install pyspark==3.3.2 +RUN pip3 install pyspark==3.4.0 RUN pip3 install mypy==0.910 diff --git a/PROTOCOL.md b/PROTOCOL.md index 73c104306a8..abfd4de21e0 100644 --- a/PROTOCOL.md +++ b/PROTOCOL.md @@ -24,6 +24,9 @@ - [Protocol Evolution](#protocol-evolution) - [Commit Provenance Information](#commit-provenance-information) - [Increase Row ID High-Water Mark](#increase-row-id-high-watermark) + - [Domain Metadata](#domain-metadata) + - [Reader Requirements for Domain Metadata](#reader-requirements-for-domain-metadata) + - [Writer Requirements for Domain Metadata](#writer-requirements-for-domain-metadata) - [Action Reconciliation](#action-reconciliation) - [Table Features](#table-features) - [Table Features for new and Existing Tables](#table-features-for-new-and-existing-tables) @@ -475,7 +478,7 @@ Protocol versioning allows a newer client to exclude older readers and/or writer The _protocol version_ will be increased whenever non-forward-compatible changes are made to this specification. In the case where a client is running an invalid protocol version, an error should be thrown instructing the user to upgrade to a newer protocol version of their Delta client library. -Since breaking changes must be accompanied by an increase in the protocol version recorded in a table or by the addition of a table feature, clients can assume that unrecognized fields or actions are never required in order to correctly interpret the transaction log. Clients must ignore such unrecognized fields, and should not produce an error when reading a table that contains unrecognized fields. +Since breaking changes must be accompanied by an increase in the protocol version recorded in a table or by the addition of a table feature, clients can assume that unrecognized actions, fields, and/or metadata domains are never required in order to correctly interpret the transaction log. Clients must ignore such unrecognized fields, and should not produce an error when reading a table that contains unrecognized fields. Reader Version 3 and Writer Version 7 add two lists of table features to the protocol action. The capability for readers and writers to operate on such a table is not only dependent on their supported protocol versions, but also on whether they support all features listed in `readerFeatures` and `writerFeatures`. See [Table Features](#table-features) section for more information. @@ -564,14 +567,54 @@ The following is an example `rowIdHighWaterMark` action: } ``` +### Domain Metadata +The domain metadata action contains a configuration (string-string map) for a named metadata domain. Two overlapping transactions conflict if they both contain a domain metadata action for the same metadata domain. + +There are two types of metadata domains: +1. **User-controlled metadata domains** have names that start with anything other than the `delta.` prefix. Any Delta client implementation or user application can modify these metadata domains, and can allow users to modify them arbitrarily. Delta clients and user applications are encouraged to use a naming convention designed to avoid conflicts with other clients' or users' metadata domains (e.g. `com.databricks.*` or `org.apache.*`). +2. **System-controlled metadata domains** have names that start with the `delta.` prefix. This prefix is reserved for metadata domains defined by the Delta spec, and Delta client implementations must not allow users to modify the metadata for system-controlled domains. A Delta client implementation should only update metadata for system-controlled domains that it knows about and understands. System-controlled metadata domains are used by various table features and each table feature may impose additional semantics on the metadata domains it uses. + +The schema of the `domainMetadata` action is as follows: + +Field Name | Data Type | Description +-|-|- +domain | String | Identifier for this domain (system- or user-provided) +configuration | Map[String, String] | A map containing configuration for the metadata domain +removed | Boolean | When `true`, the action serves as a tombstone to logically delete a metadata domain. Writers should preserve an accurate pre-image of the configuration. + +Enablement: +- The table must be on Writer Version 7. +- A feature name `domainMetadata` must exist in the table's `writerFeatures`. + +#### Reader Requirements for Domain Metadata +- Readers must preserve all domains even if they don't understand them, i.e. the snapshot read must include them. +- Any system-controlled domain that requires special attention from a reader is a [breaking change](#protocol-evolution), and must be part of a reader-writer table feature that specifies the desired behavior. + +#### Writer Requirements for Domain Metadata +- Writers must not allow users to modify or delete system-controlled domains. +- Writers must only modify or delete system-controlled domains they understand. +- Any system-controlled domain that needs special attention from a writer is a [breaking change](#protocol-evolution), and must be part of a writer table feature that specifies the desired behavior. + +The following is an example `domainMetadata` action: +```json +{ + "domainMetadata": { + "domain": "delta.deltaTableFeatureX", + "configuration": {"key1": "..."}, + "removed": false + } +} +``` + # Action Reconciliation A given snapshot of the table can be computed by replaying the events committed to the table in ascending order by commit version. A given snapshot of a Delta table consists of: - A single `protocol` action - A single `metaData` action - At most one `rowIdHighWaterMark` action - - A map from `appId` to transaction `version` - - A collection of `add` actions with unique `path`s. + - A collection of `txn` actions with unique `appId`s + - A collection of `domainMetadata` actions with unique `domain`s. + - A collection of `add` actions with unique `(path, deletionVector.uniqueId)` keys. - A collection of `remove` actions with unique `(path, deletionVector.uniqueId)` keys. The intersection of the primary keys in the `add` collection and `remove` collection must be empty. That means a logical file cannot exist in both the `remove` and `add` collections at the same time; however, the same *data file* can exist with *different* DVs in the `remove` collection, as logically they represent different content. The `remove` actions act as _tombstones_, and only exist for the benefit of the VACUUM command. Snapshot reads only return `add` actions on the read path. To achieve the requirements above, related actions from different delta files need to be reconciled with each other: @@ -579,7 +622,8 @@ To achieve the requirements above, related actions from different delta files ne - The latest `protocol` action seen wins - The latest `metaData` action seen wins - The latest `rowIdHighWaterMark` action seen wins - - For transaction identifiers, the latest `version` seen for a given `appId` wins + - For `txn` actions, the latest `version` seen for a given `appId` wins + - For `domainMetadata`, the latest `domainMetadata` seen for a given `domain` wins. The actions with `removed=true` act as tombstones to suppress earlier versions. Snapshot reads do _not_ return removed `domainMetadata` actions. - Logical files in a table are identified by their `(path, deletionVector.uniqueId)` primary key. File actions (`add` or `remove`) reference logical files, and a log can contain any number of references to a single file. - To replay the log, scan all file actions and keep only the newest reference for each logical file. - `add` actions in the result identify logical files currently present in the table (for queries). `remove` actions in the result identify tombstones of logical files no longer present in the table (for VACUUM). @@ -861,6 +905,7 @@ Checkpoint files must be written in [Apache Parquet](https://parquet.apache.org/ * The [metadata](#Change-Metadata) of the table * Files that have been [added and removed](#Add-File-and-Remove-File) * [Transaction identifiers](#Transaction-Identifiers) + * [Domain Metadata](#Domain-Metadata) Commit provenance information does not need to be included in the checkpoint. All of these actions are stored as their individual columns in parquet as struct fields. @@ -1019,6 +1064,7 @@ Feature | Name | Readers or Writers? [Deletion Vectors](#deletion-vectors) | `deletionVectors` | Readers and writers [Row IDs](#row-ids) | `rowIds` | Writers only [Timestamp without Timezone](#timestamp-ntz) | `timestampNTZ` | Readers and writers +[Domain Metadata](#domain-metadata) | `domainMetadata` | Writers only ## Deletion Vector Format diff --git a/build.sbt b/build.sbt index 64377ed99cb..9b29c449d5d 100644 --- a/build.sbt +++ b/build.sbt @@ -17,7 +17,7 @@ import java.nio.file.Files import TestParallelization._ -val sparkVersion = "3.3.2" +val sparkVersion = "3.4.0" val scala212 = "2.12.15" val scala213 = "2.13.5" val default_scala_version = scala212 @@ -204,6 +204,8 @@ lazy val storageS3DynamoDB = (project in file("storage-s3-dynamodb")) ) ) +// Requires iceberg release on 3.4 +/** lazy val deltaIceberg = (project in file("delta-iceberg")) .dependsOn(core % "compile->compile;test->test;provided->provided") .settings ( @@ -221,6 +223,7 @@ lazy val deltaIceberg = (project in file("delta-iceberg")) "org.scala-lang.modules" %% "scala-collection-compat" % "2.1.1" ) ) +*/ /** * Get list of python files and return the mapping between source files and target paths diff --git a/core/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 b/core/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 index c2e7a0bf1ae..82f626cb717 100644 --- a/core/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 +++ b/core/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 @@ -87,8 +87,11 @@ statement | ALTER TABLE table=qualifiedName DROP CONSTRAINT (IF EXISTS)? name=identifier #dropTableConstraint | OPTIMIZE (path=STRING | table=qualifiedName) - (WHERE partitionPredicate = predicateToken)? + (WHERE partitionPredicate=predicateToken)? (zorderSpec)? #optimizeTable + | REORG TABLE table=qualifiedName + (WHERE partitionPredicate=predicateToken)? + APPLY LEFT_PAREN PURGE RIGHT_PAREN #reorgTable | SHOW COLUMNS (IN | FROM) tableName=qualifiedName ((IN | FROM) schemaName=identifier)? #showColumns | cloneTableHeader SHALLOW CLONE source=qualifiedName clause=temporalClause? @@ -210,6 +213,7 @@ nonReserved | CONVERT | TO | DELTA | PARTITIONED | BY | DESC | DESCRIBE | LIMIT | DETAIL | GENERATE | FOR | TABLE | CHECK | EXISTS | OPTIMIZE + | REORG | APPLY | PURGE | RESTORE | AS | OF | ZORDER | LEFT_PAREN | RIGHT_PAREN | SHOW | COLUMNS | IN | FROM | NO | STATISTICS @@ -219,6 +223,7 @@ nonReserved // Define how the keywords above should appear in a user's SQL statement. ADD: 'ADD'; ALTER: 'ALTER'; +APPLY: 'APPLY'; AS: 'AS'; BY: 'BY'; CHECK: 'CHECK'; @@ -255,7 +260,9 @@ NULL: 'NULL'; OF: 'OF'; OR: 'OR'; OPTIMIZE: 'OPTIMIZE'; +REORG: 'REORG'; PARTITIONED: 'PARTITIONED'; +PURGE: 'PURGE'; REPLACE: 'REPLACE'; RESTORE: 'RESTORE'; RETAIN: 'RETAIN'; diff --git a/core/src/main/resources/error/delta-error-classes.json b/core/src/main/resources/error/delta-error-classes.json index 56b3462bece..cca1642c232 100644 --- a/core/src/main/resources/error/delta-error-classes.json +++ b/core/src/main/resources/error/delta-error-classes.json @@ -17,6 +17,12 @@ ], "sqlState" : "0A000" }, + "DELTA_ADDING_DELETION_VECTORS_WITH_TIGHT_BOUNDS_DISALLOWED" : { + "message" : [ + "All operations that add deletion vectors should set the tightBounds column in statistics to false. Please file a bug report." + ], + "sqlState" : "42000" + }, "DELTA_ADD_COLUMN_AT_INDEX_LESS_THAN_ZERO" : { "message" : [ "Index to add column is lower than 0" @@ -1023,12 +1029,24 @@ ], "sqlState" : "42601" }, + "DELTA_MERGE_RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT" : { + "message" : [ + "Resolved attribute(s) missing from in operator " + ], + "sqlState" : "42601" + }, "DELTA_MERGE_UNEXPECTED_ASSIGNMENT_KEY" : { "message" : [ "Unexpected assignment key: - " ], "sqlState" : "22005" }, + "DELTA_MERGE_UNRESOLVED_EXPRESSION" : { + "message" : [ + "Cannot resolve in given " + ], + "sqlState" : "42601" + }, "DELTA_METADATA_ABSENT" : { "message" : [ "Couldn't find Metadata while committing the first version of the Delta table." @@ -1328,6 +1346,12 @@ ], "sqlState" : "0A000" }, + "DELTA_OVERWRITE_SCHEMA_WITH_DYNAMIC_PARTITION_OVERWRITE" : { + "message" : [ + "'overwriteSchema' cannot be used in dynamic partition overwrite mode." + ], + "sqlState" : "42613" + }, "DELTA_PARTITION_COLUMN_CAST_FAILED" : { "message" : [ "Failed to cast value `` to `` for partition column ``" @@ -1557,6 +1581,12 @@ ], "sqlState" : "XXKDS" }, + "DELTA_STATS_COLLECTION_COLUMN_NOT_FOUND" : { + "message" : [ + " stats not found for column in Parquet metadata: ." + ], + "sqlState" : "42000" + }, "DELTA_STREAMING_CANNOT_CONTINUE_PROCESSING_POST_SCHEMA_EVOLUTION" : { "message" : [ "We've detected a non-additive schema change () at Delta version in the Delta streaming source. Please check if you want to manually propagate this schema change to the sink table before we proceed with stream processing.", @@ -2100,6 +2130,12 @@ ], "sqlState" : "0AKDD" }, + "DELTA_UNSUPPORTED_STATS_RECOMPUTE_WITH_DELETION_VECTORS" : { + "message" : [ + "Statistics re-computation on a Delta table with deletion vectors is not yet supported." + ], + "sqlState" : "0AKDD" + }, "DELTA_UNSUPPORTED_SUBQUERY" : { "message" : [ "Subqueries are not supported in the (condition = )." diff --git a/core/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala b/core/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala index 3180a357776..e24bf68a1e4 100644 --- a/core/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala +++ b/core/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala @@ -257,7 +257,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { case replaceHeader: ReplaceTableHeaderContext => (visitTableIdentifier(replaceHeader.table), replaceHeader.CREATE() != null, true, false) case _ => - throw new ParseException("Incorrect CLONE header expected REPLACE or CREATE table", ctx) + throw new DeltaParseException("Incorrect CLONE header expected REPLACE or CREATE table", ctx) } } @@ -273,7 +273,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { val (target, isCreate, isReplace, ifNotExists) = visitCloneTableHeader(ctx.cloneTableHeader()) if (!isCreate && ifNotExists) { - throw new ParseException( + throw new DeltaParseException( "IF NOT EXISTS cannot be used together with REPLACE", ctx.cloneTableHeader()) } @@ -340,13 +340,35 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { */ override def visitOptimizeTable(ctx: OptimizeTableContext): AnyRef = withOrigin(ctx) { if (ctx.path == null && ctx.table == null) { - throw new ParseException("OPTIMIZE command requires a file path or table name.", ctx) + throw new DeltaParseException("OPTIMIZE command requires a file path or table name.", ctx) } val interleaveBy = Option(ctx.zorderSpec).map(visitZorderSpec).getOrElse(Seq.empty) OptimizeTableCommand( Option(ctx.path).map(string), Option(ctx.table).map(visitTableIdentifier), - Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq, Map.empty)(interleaveBy) + Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq, + Map.empty)(interleaveBy) + } + + /** + * Creates a [[DeltaReorgTable]] logical plan. + * Examples: + * {{{ + * -- Physically delete dropped rows and columns of target table + * REORG TABLE (delta.`/path/to/table` | delta_table_name) + * [WHERE partition_predicate] APPLY (PURGE) + * }}} + */ + override def visitReorgTable(ctx: ReorgTableContext): AnyRef = withOrigin(ctx) { + if (ctx.table == null) { + throw new ParseException("REORG command requires a file path or table name.", ctx) + } + + val targetIdentifier = visitTableIdentifier(ctx.table) + val tableNameParts = targetIdentifier.database.toSeq :+ targetIdentifier.table + val targetTable = createUnresolvedTable(tableNameParts, "REORG") + + DeltaReorgTable(targetTable)(Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq) } override def visitDescribeDeltaDetail( @@ -406,7 +428,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { ctx.identifier.asScala.toSeq match { case Seq(tbl) => TableIdentifier(tbl.getText) case Seq(db, tbl) => TableIdentifier(tbl.getText, Some(db.getText)) - case _ => throw new ParseException(s"Illegal table name ${ctx.getText}", ctx) + case _ => throw new DeltaParseException(s"Illegal table name ${ctx.getText}", ctx) } } @@ -537,7 +559,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { case ("interval", Nil) => CalendarIntervalType case (dt, params) => val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt - throw new ParseException(s"DataType $dtStr is not supported.", ctx) + throw new DeltaParseException(s"DataType $dtStr is not supported.", ctx) } } } diff --git a/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala b/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala index 63a1ec15602..dc96cdd0c23 100644 --- a/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala +++ b/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala @@ -339,8 +339,7 @@ class DeltaTableBuilder private[tables]( val stmt = builderOption match { case CreateTableOptions(ifNotExists) => - val unresolvedTable: LogicalPlan = - org.apache.spark.sql.catalyst.analysis.UnresolvedDBObjectName(table, isNamespace = false) + val unresolvedTable = org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier(table) CreateTable( unresolvedTable, StructType(columns.toSeq), @@ -348,8 +347,7 @@ class DeltaTableBuilder private[tables]( tableSpec, ifNotExists) case ReplaceTableOptions(orCreate) => - val unresolvedTable: LogicalPlan = - org.apache.spark.sql.catalyst.analysis.UnresolvedDBObjectName(table, isNamespace = false) + val unresolvedTable = org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier(table) ReplaceTable( unresolvedTable, StructType(columns.toSeq), diff --git a/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala b/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala index fbf8f9d2483..f8c53ab312e 100644 --- a/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala +++ b/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala @@ -424,7 +424,10 @@ object DeltaMergeInto { // Note: This will throw error only on unresolved attribute issues, // not other resolution errors like mismatched data types. val cols = "columns " + plan.children.flatMap(_.output).map(_.sql).mkString(", ") - a.failAnalysis(msg = s"cannot resolve ${a.sql} in $mergeClauseType given $cols") + throw new DeltaAnalysisException( + errorClass = "DELTA_MERGE_UNRESOLVED_EXPRESSION", + messageParameters = Array(a.sql, mergeClauseType, cols), + origin = Some(a.origin)) } resolvedExpr } @@ -536,7 +539,8 @@ object DeltaMergeInto { Seq(d) case _ => - action.failAnalysis(msg = s"Unexpected action expression '$action' in clause $clause") + action.failAnalysis("INTERNAL_ERROR", + Map("message" -> s"Unexpected action expression '$action' in clause $clause")) } } @@ -625,9 +629,12 @@ object DeltaMergeInto { if (resolvedMerge.missingInput.nonEmpty) { val missingAttributes = resolvedMerge.missingInput.mkString(",") val input = resolvedMerge.inputSet.mkString(",") - val msgForMissingAttributes = s"Resolved attribute(s) $missingAttributes missing " + - s"from $input in operator ${resolvedMerge.simpleString(SQLConf.get.maxToStringFields)}." - resolvedMerge.failAnalysis(msg = msgForMissingAttributes) + throw new DeltaAnalysisException( + errorClass = "DELTA_MERGE_RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + messageParameters = Array(missingAttributes, input, + resolvedMerge.simpleString(SQLConf.get.maxToStringFields)), + origin = Some(resolvedMerge.origin) + ) } resolvedMerge diff --git a/core/src/main/scala/org/apache/spark/sql/delta/ColumnWithDefaultExprUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/ColumnWithDefaultExprUtils.scala index 52a4d190154..1e7e8dcf7b4 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/ColumnWithDefaultExprUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/ColumnWithDefaultExprUtils.scala @@ -194,6 +194,7 @@ object ColumnWithDefaultExprUtils extends DeltaLogging { incrementalExecution.queryId, incrementalExecution.runId, incrementalExecution.currentBatchId, + incrementalExecution.prevOffsetSeqMetadata, incrementalExecution.offsetSeqMetadata ) newIncrementalExecution.executedPlan // Force the lazy generation of execution plan diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 7c8b56fe56c..eb17f222312 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -53,6 +53,7 @@ import org.apache.spark.sql.catalyst.streaming.WriteToStream import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.CreateTableLikeCommand import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.HadoopFsRelation @@ -76,7 +77,7 @@ class DeltaAnalysis(session: SparkSession) override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { // INSERT INTO by ordinal and df.insertInto() case a @ AppendDelta(r, d) if !a.isByName && - needsSchemaAdjustment(d.name(), a.query, r.schema) => + needsSchemaAdjustmentByOrdinal(d.name(), a.query, r.schema) => val projection = resolveQueryColumnsByOrdinal(a.query, r.output, d.name()) if (projection != a.query) { a.copy(query = projection) @@ -85,6 +86,18 @@ class DeltaAnalysis(session: SparkSession) } + // INSERT INTO by name + // AppendData.byName is also used for DataFrame append so we check for the SQL origin text + // since we only want to up-cast for SQL insert into by name + case a @ AppendDelta(r, d) if a.isByName && + a.origin.sqlText.nonEmpty && needsSchemaAdjustmentByName(a.query, r.output, d) => + val projection = resolveQueryColumnsByName(a.query, r.output, d) + if (projection != a.query) { + a.copy(query = projection) + } else { + a + } + /** * Handling create table like when a delta target (provider) * is provided explicitly or when the source table is a delta table @@ -182,7 +195,7 @@ class DeltaAnalysis(session: SparkSession) // INSERT OVERWRITE by ordinal and df.insertInto() case o @ OverwriteDelta(r, d) if !o.isByName && - needsSchemaAdjustment(d.name(), o.query, r.schema) => + needsSchemaAdjustmentByOrdinal(d.name(), o.query, r.schema) => val projection = resolveQueryColumnsByOrdinal(o.query, r.output, d.name()) if (projection != o.query) { val aliases = AttributeMap(o.query.output.zip(projection.output).collect { @@ -196,11 +209,38 @@ class DeltaAnalysis(session: SparkSession) o } + // INSERT OVERWRITE by name + // OverwriteDelta.byName is also used for DataFrame append so we check for the SQL origin text + // since we only want to up-cast for SQL insert into by name + case o @ OverwriteDelta(r, d) if o.isByName && + o.origin.sqlText.nonEmpty && needsSchemaAdjustmentByName(o.query, r.output, d) => + val projection = resolveQueryColumnsByName(o.query, r.output, d) + if (projection != o.query) { + val aliases = AttributeMap(o.query.output.zip(projection.output).collect { + case (l: AttributeReference, r: AttributeReference) if !l.sameRef(r) => (l, r) + }) + val newDeleteExpr = o.deleteExpr.transformUp { + case a: AttributeReference => aliases.getOrElse(a, a) + } + o.copy(deleteExpr = newDeleteExpr, query = projection) + } else { + o + } + + // INSERT OVERWRITE with dynamic partition overwrite case o @ DynamicPartitionOverwriteDelta(r, d) if o.resolved => - val adjustedQuery = if (!o.isByName && needsSchemaAdjustment(d.name(), o.query, r.schema)) { + val adjustedQuery = if (!o.isByName && + needsSchemaAdjustmentByOrdinal(d.name(), o.query, r.schema)) { + // INSERT OVERWRITE by ordinal and df.insertInto() resolveQueryColumnsByOrdinal(o.query, r.output, d.name()) + } else if (o.isByName && o.origin.sqlText.nonEmpty && + needsSchemaAdjustmentByName(o.query, r.output, d)) { + // INSERT OVERWRITE by name + // OverwriteDelta.byName is also used for DataFrame append so we check for the SQL origin + // text since we only want to up-cast for SQL insert into by name + resolveQueryColumnsByName(o.query, r.output, d) } else { o.query } @@ -266,10 +306,10 @@ class DeltaAnalysis(session: SparkSession) cloneStatement) case u: UnresolvedRelation => - u.failAnalysis(msg = s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case TimeTravel(u: UnresolvedRelation, _, _, _) => - u.failAnalysis(msg = s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case LogicalRelation( HadoopFsRelation(location, _, _, _, _: ParquetFileFormat, _), _, catalogTable, _) => @@ -349,10 +389,10 @@ class DeltaAnalysis(session: SparkSession) RestoreTableCommand(traveledTable, tblIdent) case u: UnresolvedRelation => - u.failAnalysis(msg = s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case TimeTravel(u: UnresolvedRelation, _, _, _) => - u.failAnalysis(msg = s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case _ => throw DeltaErrors.notADeltaTableException("RESTORE") @@ -419,6 +459,18 @@ class DeltaAnalysis(session: SparkSession) s"${other.prettyName} clauses cannot be part of the WHEN NOT MATCHED clause in MERGE " + "INTO.") } + val notMatchedBySourceActions = merge.notMatchedBySourceActions.map { + case update: UpdateAction => + DeltaMergeIntoNotMatchedBySourceUpdateClause( + update.condition, + DeltaMergeIntoClause.toActions(update.assignments)) + case delete: DeleteAction => + DeltaMergeIntoNotMatchedBySourceDeleteClause(delete.condition) + case other => + throw new IllegalArgumentException( + s"${other.prettyName} clauses cannot be part of the WHEN NOT MATCHED BY SOURCE " + + "clause in MERGE INTO.") + } // rewrites Delta from V2 to V1 val newTarget = stripTempViewForMergeWrapper(merge.targetTable).transformUp { case DeltaRelation(lr) => lr } @@ -428,11 +480,19 @@ class DeltaAnalysis(session: SparkSession) newTarget, merge.sourceTable, merge.mergeCondition, - matchedActions ++ notMatchedActions + matchedActions ++ notMatchedActions ++ notMatchedBySourceActions ) DeltaMergeInto.resolveReferencesAndSchema(deltaMerge, conf)(tryResolveReferences(session)) + case reorg@DeltaReorgTable(_@ResolvedTable(_, _, t, _)) => + t match { + case table: DeltaTableV2 => + DeltaReorgTableCommand(table)(reorg.predicates) + case _ => + throw DeltaErrors.notADeltaTable(t.name()) + } + case deltaMerge: DeltaMergeInto => val d = if (deltaMerge.childrenResolved && !deltaMerge.resolved) { DeltaMergeInto.resolveReferencesAndSchema(deltaMerge, conf)(tryResolveReferences(session)) @@ -672,28 +732,7 @@ class DeltaAnalysis(session: SparkSession) */ private def resolveQueryColumnsByName( query: LogicalPlan, targetAttrs: Seq[Attribute], deltaTable: DeltaTableV2): LogicalPlan = { - if (query.output.length < targetAttrs.length) { - // Some columns are not specified. We don't allow schema evolution in INSERT INTO BY NAME, so - // we need to ensure the missing columns must be generated columns. - val userSpecifiedNames = - if (session.sessionState.conf.caseSensitiveAnalysis) { - query.output.map(a => (a.name, a)).toMap - } else { - CaseInsensitiveMap(query.output.map(a => (a.name, a)).toMap) - } - val tableSchema = deltaTable.snapshot.metadata.schema - if (tableSchema.length != targetAttrs.length) { - // The target attributes may contain the metadata columns by design. Throwing an exception - // here in case target attributes may have the metadata columns for Delta in future. - throw DeltaErrors.schemaNotConsistentWithTarget(s"$tableSchema", s"$targetAttrs") - } - deltaTable.snapshot.metadata.schema.foreach { col => - if (!userSpecifiedNames.contains(col.name) && - !ColumnWithDefaultExprUtils.columnHasDefaultExpr(deltaTable.snapshot.protocol, col)) { - throw DeltaErrors.missingColumnsInInsertInto(col.name) - } - } - } + insertIntoByNameMissingColumn(query, targetAttrs, deltaTable) // Spark will resolve columns to make sure specified columns are in the table schema and don't // have duplicates. This is just a sanity check. assert( @@ -735,7 +774,7 @@ class DeltaAnalysis(session: SparkSession) * of INSERT INTO. This allows us to perform better schema enforcement/evolution. Since Spark * skips this step, we see if we need to perform any schema adjustment here. */ - private def needsSchemaAdjustment( + private def needsSchemaAdjustmentByOrdinal( tableName: String, query: LogicalPlan, schema: StructType): Boolean = { @@ -750,6 +789,58 @@ class DeltaAnalysis(session: SparkSession) !SchemaUtils.isReadCompatible(schema.asNullable, existingSchemaOutput.toStructType) } + /** + * Checks for missing columns in a insert by name query and throws an exception if found. + * Delta does not require users to provide values for generated columns, so any columns missing + * from the query output must have a default expression. + * See [[ColumnWithDefaultExprUtils.columnHasDefaultExpr]]. + */ + private def insertIntoByNameMissingColumn( + query: LogicalPlan, + targetAttrs: Seq[Attribute], + deltaTable: DeltaTableV2): Unit = { + if (query.output.length < targetAttrs.length) { + // Some columns are not specified. We don't allow schema evolution in INSERT INTO BY NAME, so + // we need to ensure the missing columns must be generated columns. + val userSpecifiedNames = if (session.sessionState.conf.caseSensitiveAnalysis) { + query.output.map(a => (a.name, a)).toMap + } else { + CaseInsensitiveMap(query.output.map(a => (a.name, a)).toMap) + } + val tableSchema = deltaTable.snapshot.metadata.schema + if (tableSchema.length != targetAttrs.length) { + // The target attributes may contain the metadata columns by design. Throwing an exception + // here in case target attributes may have the metadata columns for Delta in future. + throw DeltaErrors.schemaNotConsistentWithTarget(s"$tableSchema", s"$targetAttrs") + } + deltaTable.snapshot.metadata.schema.foreach { col => + if (!userSpecifiedNames.contains(col.name) && + !ColumnWithDefaultExprUtils.columnHasDefaultExpr(deltaTable.snapshot.protocol, col)) { + throw DeltaErrors.missingColumnsInInsertInto(col.name) + } + } + } + } + + /** + * With Delta, we ACCEPT_ANY_SCHEMA, meaning that Spark doesn't automatically adjust the schema + * of INSERT INTO. Here we check if we need to perform any schema adjustment for INSERT INTO by + * name queries. We also check that any columns not in the list of user-specified columns must + * have a default expression. + */ + private def needsSchemaAdjustmentByName(query: LogicalPlan, targetAttrs: Seq[Attribute], + deltaTable: DeltaTableV2): Boolean = { + insertIntoByNameMissingColumn(query, targetAttrs, deltaTable) + val userSpecifiedNames = if (session.sessionState.conf.caseSensitiveAnalysis) { + query.output.map(a => (a.name, a)).toMap + } else { + CaseInsensitiveMap(query.output.map(a => (a.name, a)).toMap) + } + val specifiedTargetAttrs = targetAttrs.filter(col => userSpecifiedNames.contains(col.name)) + !SchemaUtils.isReadCompatible( + specifiedTargetAttrs.toStructType.asNullable, query.output.toStructType) + } + // Get cast operation for the level of strictness in the schema a user asked for private def getCastFunction: CastFunction = { val timeZone = conf.sessionLocalTimeZone @@ -759,7 +850,9 @@ class DeltaAnalysis(session: SparkSession) Cast(input, dt, Option(timeZone), ansiEnabled = false) case SQLConf.StoreAssignmentPolicy.ANSI => (input: Expression, dt: DataType, name: String) => { - AnsiCast(input, dt, Option(timeZone)) + val cast = Cast(input, dt, Option(timeZone), ansiEnabled = true) + cast.setTagValue(Cast.BY_TABLE_INSERTION, ()) + TableOutputResolver.checkCastOverflowInTableInsert(cast, name) } case SQLConf.StoreAssignmentPolicy.STRICT => (input: Expression, dt: DataType, _) => @@ -767,7 +860,6 @@ class DeltaAnalysis(session: SparkSession) } } - /** * Recursively casts structs in case it contains null types. * TODO: Support other complex types like MapType and ArrayType @@ -990,7 +1082,8 @@ case class DeltaDynamicPartitionOverwriteCommand( deltaTable: DeltaTableV2, query: LogicalPlan, writeOptions: Map[String, String], - isByName: Boolean) extends RunnableCommand with V2WriteCommand { + isByName: Boolean, + analyzedQuery: Option[LogicalPlan] = None) extends RunnableCommand with V2WriteCommand { override def child: LogicalPlan = query @@ -1002,6 +1095,7 @@ case class DeltaDynamicPartitionOverwriteCommand( copy(table = newTable) } + override def storeAnalyzedQuery(): Command = copy(analyzedQuery = Some(query)) override protected def withNewChildInternal( newChild: LogicalPlan): DeltaDynamicPartitionOverwriteCommand = copy(query = newChild) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala index a97e5f81d59..4bf835502b8 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala @@ -1959,7 +1959,7 @@ trait DeltaErrorsBase new DeltaAnalysisException("DELTA_UNSUPPORTED_DROP_COLUMN", Array(adviceMsg)) } - def dropNestedColumnsFromNonStructTypeException(struct : StructField) : Throwable = { + def dropNestedColumnsFromNonStructTypeException(struct : DataType) : Throwable = { new DeltaAnalysisException( errorClass = "DELTA_UNSUPPORTED_DROP_NESTED_COLUMN_FROM_NON_STRUCT_TYPE", messageParameters = Array(s"$struct") @@ -2275,6 +2275,13 @@ trait DeltaErrorsBase new DeltaIllegalStateException(errorClass = "DELTA_ACTIVE_TRANSACTION_ALREADY_SET") } + def deltaStatsCollectionColumnNotFound(statsType: String, columnPath: String): Throwable = { + new DeltaRuntimeException( + errorClass = "DELTA_STATS_COLLECTION_COLUMN_NOT_FOUND", + messageParameters = Array(statsType, columnPath) + ) + } + /** This is a method only used for testing Py4J exception handling. */ def throwDeltaIllegalArgumentException(): Throwable = { new DeltaIllegalArgumentException(errorClass = "DELTA_UNRECOGNIZED_INVARIANT") @@ -2396,6 +2403,12 @@ trait DeltaErrorsBase ) } + def overwriteSchemaUsedWithDynamicPartitionOverwrite(): Throwable = { + new DeltaIllegalArgumentException( + errorClass = "DELTA_OVERWRITE_SCHEMA_WITH_DYNAMIC_PARTITION_OVERWRITE" + ) + } + def replaceWhereUsedInOverwrite(): Throwable = { new DeltaAnalysisException( errorClass = "DELTA_REPLACE_WHERE_IN_OVERWRITE", messageParameters = Array.empty @@ -2570,9 +2583,9 @@ trait DeltaErrorsBase def failedToGetSnapshotDuringColumnMappingStreamingReadCheck(cause: Throwable): Throwable = { new DeltaAnalysisException( errorClass = "DELTA_STREAMING_CHECK_COLUMN_MAPPING_NO_SNAPSHOT", - Array(DeltaSQLConf + messageParameters = Array(DeltaSQLConf .DELTA_STREAMING_UNSAFE_READ_ON_INCOMPATIBLE_COLUMN_MAPPING_SCHEMA_CHANGES.key), - Some(cause)) + cause = Some(cause)) } def showColumnsWithConflictDatabasesError(db: String, tableID: TableIdentifier): Throwable = { @@ -2745,6 +2758,17 @@ trait DeltaErrorsBase pos = 0) } + def statsRecomputeNotSupportedOnDvTables(): Throwable = { + new DeltaCommandUnsupportedWithDeletionVectorsException( + errorClass = "DELTA_UNSUPPORTED_STATS_RECOMPUTE_WITH_DELETION_VECTORS", + messageParameters = Array.empty + ) + } + + def addFileWithDVsAndTightBoundsException(): Throwable = + new DeltaIllegalStateException( + errorClass = "DELTA_ADDING_DELETION_VECTORS_WITH_TIGHT_BOUNDS_DISALLOWED") + def addFileWithDVsMissingNumRecordsException: Throwable = new DeltaRuntimeException(errorClass = "DELTA_DELETION_VECTOR_MISSING_NUM_RECORDS") @@ -3080,7 +3104,7 @@ class DeltaTablePropertyValidationFailedException( table: String, subClass: DeltaTablePropertyValidationFailedSubClass) extends RuntimeException(DeltaThrowableHelper.getMessage( - errorClass = "DELTA_VIOLATE_TABLE_PROPERTY_VALIDATION_FAILED", + errorClass = "DELTA_VIOLATE_TABLE_PROPERTY_VALIDATION_FAILED" + "." + subClass.tag, messageParameters = subClass.messageParameters(table))) with DeltaThrowable { override def getErrorClass: String = diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala index 5400e8aeb94..761e37ca013 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -42,7 +42,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FileSourceOptions, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStatistics, CatalogTable} import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal} @@ -177,14 +177,9 @@ class DeltaLog private( // Delta should NEVER ignore missing or corrupt metadata files, because doing so can render the // entire table unusable. Hard-wire that into the file source options so the user can't override // it by setting spark.sql.files.ignoreCorruptFiles or spark.sql.files.ignoreMissingFiles. - // - // NOTE: This should ideally be [[FileSourceOptions.IGNORE_CORRUPT_FILES]] etc., but those - // constants are only available since spark-3.4. By hard-coding the values here instead, we - // preserve backward compatibility when compiling Delta against older spark versions (tho - // obviously the desired protection would be missing in that case). val allOptions = options ++ formatSpecificOptions ++ Map( - "ignoreCorruptFiles" -> "false", - "ignoreMissingFiles" -> "false" + FileSourceOptions.IGNORE_CORRUPT_FILES -> "false", + FileSourceOptions.IGNORE_MISSING_FILES -> "false" ) val fsRelation = HadoopFsRelation( index, index.partitionSchema, schema, None, index.format, allOptions)(spark) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala index b5981f168c7..f49f35d7689 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala @@ -24,8 +24,10 @@ import org.apache.spark.sql.delta.util.JsonUtils import org.apache.spark.sql.{SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.DeltaMergeIntoClause import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{StructField, StructType} @@ -40,7 +42,7 @@ object DeltaOperations { * @param name The name of the operation. */ sealed abstract class Operation(val name: String) { - val parameters: Map[String, Any] + def parameters: Map[String, Any] lazy val jsonEncodedValues: Map[String, String] = parameters.mapValues(JsonUtils.toJson(_)).toMap @@ -59,6 +61,12 @@ object DeltaOperations { def changesData: Boolean = false } + abstract class OperationWithPredicates(name: String, val predicates: Seq[Expression]) + extends Operation(name) { + private val predicateString = JsonUtils.toJson(predicatesToString(predicates)) + override def parameters: Map[String, Any] = Map("predicate" -> predicateString) + } + /** Recorded during batch inserts. Predicates can be provided for overwrites. */ case class Write( mode: SaveMode, @@ -123,8 +131,8 @@ object DeltaOperations { override def changesData: Boolean = true } /** Recorded while deleting certain partitions. */ - case class Delete(predicate: Seq[String]) extends Operation("DELETE") { - override val parameters: Map[String, Any] = Map("predicate" -> JsonUtils.toJson(predicate)) + case class Delete(predicate: Seq[Expression]) + extends OperationWithPredicates("DELETE", predicate) { override val operationMetrics: Set[String] = DeltaOperationMetrics.DELETE override def transformMetrics(metrics: Map[String, SQLMetric]): Map[String, String] = { @@ -174,7 +182,7 @@ object DeltaOperations { object MergePredicate { def apply(mergeClause: DeltaMergeIntoClause): MergePredicate = { MergePredicate( - predicate = mergeClause.condition.map(_.sql), + predicate = mergeClause.condition.map(_.simpleString(SQLConf.get.maxToStringFields)), mergeClause.clauseType.toLowerCase()) } } @@ -188,16 +196,17 @@ object DeltaOperations { */ val OP_MERGE = "MERGE" case class Merge( - predicate: Option[String], + predicate: Option[Expression], updatePredicate: Option[String], deletePredicate: Option[String], insertPredicate: Option[String], matchedPredicates: Seq[MergePredicate], notMatchedPredicates: Seq[MergePredicate], - notMatchedBySourcePredicates: Seq[MergePredicate]) extends Operation(OP_MERGE) { + notMatchedBySourcePredicates: Seq[MergePredicate]) + extends OperationWithPredicates(OP_MERGE, predicate.toSeq) { override val parameters: Map[String, Any] = { - predicate.map("predicate" -> _).toMap ++ + super.parameters ++ updatePredicate.map("updatePredicate" -> _).toMap ++ deletePredicate.map("deletePredicate" -> _).toMap ++ insertPredicate.map("insertPredicate" -> _).toMap + @@ -230,7 +239,7 @@ object DeltaOperations { object Merge { /** constructor to provide default values for deprecated fields */ def apply( - predicate: Option[String], + predicate: Option[Expression], matchedPredicates: Seq[MergePredicate], notMatchedPredicates: Seq[MergePredicate], notMatchedBySourcePredicates: Seq[MergePredicate]): Merge = Merge( @@ -244,8 +253,8 @@ object DeltaOperations { } /** Recorded when an update operation is committed to the table. */ - case class Update(predicate: Option[String]) extends Operation("UPDATE") { - override val parameters: Map[String, Any] = predicate.map("predicate" -> _).toMap + case class Update(predicate: Option[Expression]) + extends OperationWithPredicates("UPDATE", predicate.toSeq) { override val operationMetrics: Set[String] = DeltaOperationMetrics.UPDATE override def changesData: Boolean = true @@ -398,10 +407,8 @@ object DeltaOperations { } /** Recorded when recomputing stats on the table. */ - case class ComputeStats(predicate: Seq[String]) extends Operation("COMPUTE STATS") { - override val parameters: Map[String, Any] = Map( - "predicate" -> JsonUtils.toJson(predicate)) - } + case class ComputeStats(predicate: Seq[Expression]) + extends OperationWithPredicates("COMPUTE STATS", predicate) /** Recorded when restoring a Delta table to an older version. */ case class Restore( @@ -415,7 +422,8 @@ object DeltaOperations { override val operationMetrics: Set[String] = DeltaOperationMetrics.RESTORE } - sealed abstract class OptimizeOrReorg(override val name: String) extends Operation(name) + sealed abstract class OptimizeOrReorg(override val name: String, predicates: Seq[Expression]) + extends OperationWithPredicates(name, predicates) /** operation name for OPTIMIZE command */ val OPTIMIZE_OPERATION_NAME = "OPTIMIZE" @@ -424,11 +432,10 @@ object DeltaOperations { /** Recorded when optimizing the table. */ case class Optimize( - predicate: Seq[String], + predicate: Seq[Expression], zOrderBy: Seq[String] = Seq.empty - ) extends OptimizeOrReorg(OPTIMIZE_OPERATION_NAME) { - override val parameters: Map[String, Any] = Map( - "predicate" -> JsonUtils.toJson(predicate), + ) extends OptimizeOrReorg(OPTIMIZE_OPERATION_NAME, predicate) { + override val parameters: Map[String, Any] = super.parameters ++ Map( ZORDER_PARAMETER_KEY -> JsonUtils.toJson(zOrderBy) ) @@ -499,6 +506,15 @@ object DeltaOperations { case class TestOperation(operationName: String = "TEST") extends Operation(operationName) { override val parameters: Map[String, Any] = Map.empty } + + /** + * Helper method to convert a sequence of command predicates in the form of an + * [[Expression]]s to a sequence of Strings so be stored in the commit info. + */ + def predicatesToString(predicates: Seq[Expression]): Seq[String] = { + val maxToStringFields = SQLConf.get.maxToStringFields + predicates.map(_.simpleString(maxToStringFields)) + } } private[delta] object DeltaOperationMetrics { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala index 066b7a4f3fe..8bd1561c689 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala @@ -189,7 +189,7 @@ case class DeltaParquetFileFormat( isRowDeletedColumn: Option[ColumnMetadata], rowIndexColumn: Option[ColumnMetadata], useOffHeapBuffers: Boolean): Iterator[Object] = { - val pathUri = new URI(partitionedFile.filePath) + val pathUri = partitionedFile.pathUri val rowIndexFilter = isRowDeletedColumn.map { col => // Fetch the DV descriptor from the broadcast map and create a row index filter diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaSharedExceptions.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaSharedExceptions.scala index 2666fccb97d..1bc01a261dc 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaSharedExceptions.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaSharedExceptions.scala @@ -16,13 +16,23 @@ package org.apache.spark.sql.delta +import org.antlr.v4.runtime.ParserRuleContext + import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} +import org.apache.spark.sql.catalyst.trees.Origin class DeltaAnalysisException( - errorClass: String, messageParameters: Array[String], cause: Option[Throwable] = None) + errorClass: String, + messageParameters: Array[String], + cause: Option[Throwable] = None, + origin: Option[Origin] = None) extends AnalysisException( - DeltaThrowableHelper.getMessage(errorClass, messageParameters), + message = DeltaThrowableHelper.getMessage(errorClass, messageParameters), errorClass = Some(errorClass), + line = origin.flatMap(_.line), + startPosition = origin.flatMap(_.startPosition), + context = origin.map(_.getQueryContext).getOrElse(Array.empty), cause = cause) with DeltaThrowable { def getMessageParametersArray: Array[String] = messageParameters @@ -48,3 +58,13 @@ class DeltaUnsupportedOperationException( override def getErrorClass: String = errorClass def getMessageParametersArray: Array[String] = messageParameters } + +class DeltaParseException( + message: String, + ctx: ParserRuleContext) + extends ParseException( + Option(ParserUtils.command(ctx)), + message, + ParserUtils.position(ctx.getStart), + ParserUtils.position(ctx.getStop) + ) with DeltaThrowable diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala index 5eb19562908..350d12a1cbd 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.NodeWithOnlyDeterministicProjectAndFilter import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform} import org.apache.spark.sql.execution.datasources.{FileFormat, FileIndex, HadoopFsRelation, LogicalRelation} @@ -73,15 +74,6 @@ object DeltaFullTable { } } -// TODO: remove this after Spark 3.4 is released. -object NodeWithOnlyDeterministicProjectAndFilter { - def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { - case Project(projectList, child) if projectList.forall(_.deterministic) => unapply(child) - case Filter(cond, child) if cond.deterministic => unapply(child) - case _ => Some(plan) - } -} - object DeltaTableUtils extends PredicateHelper with DeltaLogging { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaThrowableHelper.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaThrowableHelper.scala index ddd64abdbb9..c86b306a242 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaThrowableHelper.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaThrowableHelper.scala @@ -20,13 +20,7 @@ package org.apache.spark.sql.delta import java.io.FileNotFoundException import java.net.URL -import scala.collection.immutable.SortedMap - -import com.fasterxml.jackson.core.`type`.TypeReference -import com.fasterxml.jackson.databind.json.JsonMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule - -import org.apache.spark.ErrorInfo +import org.apache.spark.ErrorClassesJsonReader import org.apache.spark.util.Utils /** @@ -35,8 +29,6 @@ import org.apache.spark.util.Utils */ object DeltaThrowableHelper { - private lazy val mapper: JsonMapper = JsonMapper.builder().addModule(DefaultScalaModule).build() - /** * Try to find the error class source file and throw exception if it is no found. */ @@ -57,33 +49,16 @@ object DeltaThrowableHelper safeGetErrorClassesSource("error/delta-error-classes.json") } - /** The error classes of spark. */ - lazy val sparkErrorClassesMap: SortedMap[String, ErrorInfo] = { - mapper.readValue(sparkErrorClassSource, new TypeReference[SortedMap[String, ErrorInfo]]() {}) - } - - /** The error classes of delta. */ - lazy val deltaErrorClassToInfoMap: SortedMap[String, ErrorInfo] = { - mapper.readValue(deltaErrorClassSource, new TypeReference[SortedMap[String, ErrorInfo]]() {}) - } - /** - * Combined error classes from delta and spark. There should not be same error classes between - * deltaErrorClassesMap and sparkErrorClassesMap. - */ - private lazy val errorClassToInfoMap: SortedMap[String, ErrorInfo] = { - deltaErrorClassToInfoMap ++ sparkErrorClassesMap - } + private val errorClassReader = new ErrorClassesJsonReader( + Seq(deltaErrorClassSource, sparkErrorClassSource)) def getMessage(errorClass: String, messageParameters: Array[String]): String = { - val errorInfo = errorClassToInfoMap.getOrElse(errorClass, - throw new IllegalArgumentException(s"Cannot find error class '$errorClass'")) - String.format(errorInfo.messageFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), + val template = errorClassReader.getMessageTemplate(errorClass) + String.format(template.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), messageParameters: _*) } - - def getSqlState(errorClass: String): String = - Option(errorClass).flatMap(errorClassToInfoMap.get).flatMap(_.sqlState).orNull + def getSqlState(errorClass: String): String = errorClassReader.getSqlState(errorClass) def isInternalError(errorClass: String): Boolean = errorClass == "INTERNAL_ERROR" diff --git a/core/src/main/scala/org/apache/spark/sql/delta/GeneratedColumn.scala b/core/src/main/scala/org/apache/spark/sql/delta/GeneratedColumn.scala index bd7e647e14a..04c51b2af47 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/GeneratedColumn.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/GeneratedColumn.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.AnalysisHelper import org.apache.spark.sql.{AnalysisException, Column, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} @@ -161,6 +162,37 @@ object GeneratedColumn extends DeltaLogging with AnalysisHelper { spark.sessionState.sqlParser.parseExpression(exprString) } + /** + * SPARK-27561 added support for lateral column alias. This means generation expressions that + * reference other generated columns no longer fail analysis in `validateGeneratedColumns`. + * + * This method checks for and throws an error if: + * - A generated column references itself + * - A generated column references another generated column + */ + def validateColumnReferences( + spark: SparkSession, + fieldName: String, + expression: Expression, + schema: StructType): Unit = { + val allowedBaseColumns = schema + .filterNot(_.name == fieldName) // Can't reference itself + .filterNot(isGeneratedColumn) // Can't reference other generated columns + val relation = new LocalRelation(StructType(allowedBaseColumns).toAttributes) + try { + val analyzer: Analyzer = spark.sessionState.analyzer + val analyzed = analyzer.execute(Project(Seq(Alias(expression, fieldName)()), relation)) + analyzer.checkAnalysis(analyzed) + } catch { + case ex: AnalysisException => + // Improve error message if possible + if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { + throw DeltaErrors.generatedColumnsReferToWrongColumns(ex) + } + throw ex + } + } + /** * If the schema contains generated columns, check the following unsupported cases: * - Refer to a non-existent column or another generated column. @@ -179,6 +211,7 @@ object GeneratedColumn extends DeltaLogging with AnalysisHelper { getGenerationExpressionStr(f) match { case Some(exprString) => val expr = parseGenerationExpression(spark, exprString) + validateColumnReferences(spark, f.name, expr, schema) new Column(expr).alias(f.name) case None => // Should not happen diff --git a/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index a952afcfdb4..3ea014254fa 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -139,7 +139,6 @@ class OptimisticTransaction (implicit override val clock: Clock) extends OptimisticTransactionImpl with DeltaLogging { - DeletionVectorUtils.assertDeletionVectorsNotReadable(spark, snapshot.metadata, snapshot.protocol) /** Creates a new OptimisticTransaction. * @@ -327,6 +326,17 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected var checkUnsupportedDataType: Boolean = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_SCHEMA_TYPE_CHECK) + // Some operations (e.g. stats collection) may set files with DVs back to tight bounds. + // In that case they need to skip this check. + protected var checkDeletionVectorFilesHaveWideBounds: Boolean = true + /** + * Disable the check that ensures that all files with DVs added have tightBounds set to false. + * + * This is necessary when recomputing the stats on a table with DVs. + */ + def disableDeletionVectorFilesHaveWideBoundsCheck(): Unit = { + checkDeletionVectorFilesHaveWideBounds = false + } /** @@ -552,7 +562,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite newMetadataTmp = RowId.verifyAndUpdateMetadata( spark, protocol, snapshot.metadata, newMetadataTmp, isCreatingNewTable) - DeletionVectorUtils.assertDeletionVectorsNotEnabled(spark, newMetadataTmp, protocol) assertMetadata(newMetadataTmp) logInfo(s"Updated metadata from ${newMetadata.getOrElse("-")} to $newMetadataTmp") newMetadata = Some(newMetadataTmp) @@ -641,6 +650,9 @@ trait OptimisticTransactionImpl extends TransactionalWrite val deletionVectorDisallowedForAddFiles = commitCheckEnabled && !isComputeStatsOperation && !deletionVectorCreationAllowed + val addFileMustHaveWideBounds = deletionVectorCreationAllowed && + checkDeletionVectorFilesHaveWideBounds + action => action match { case a: AddFile => if (deletionVectorDisallowedForAddFiles && a.deletionVector != null) { @@ -651,6 +663,18 @@ trait OptimisticTransactionImpl extends TransactionalWrite if (a.deletionVector != null && (a.stats == null || a.numPhysicalRecords.isEmpty)) { throw DeltaErrors.addFileWithDVsMissingNumRecordsException } + + // 2. All operations that add new DVs should always turn bounds to wide. + // Operations that only update files with existing DVs may opt-out from this rule + // via `disableDeletionVectorFilesHaveWideBoundsCheck()`. + // (e.g. stats collection, metadata-only updates.) + // Note, the absence of the tightBounds column when DVs exist is also an illegal state. + if (addFileMustHaveWideBounds && + a.deletionVector != null && + // Extra inversion to also catch absent `tightBounds`. + !a.tightBounds.contains(false)) { + throw DeltaErrors.addFileWithDVsAndTightBoundsException() + } case _ => // Not an AddFile, nothing to do. } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTimeTravel.scala b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTimeTravel.scala index 0f0456c2c88..a4b4f092e23 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTimeTravel.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTimeTravel.scala @@ -89,8 +89,7 @@ case class PreprocessTimeTravel(sparkSession: SparkSession) extends Rule[Logical // If table exists and not found to be a view, throw not supported error throw DeltaErrors.notADeltaTableException("RESTORE") } else { - ur.failAnalysis(msg = s"Table not found: " + - s"${ur.multipartIdentifier.map(quoteIfNeeded).mkString(".")}") + ur.tableNotFound(ur.multipartIdentifier) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/actions/DeletionVectorDescriptor.scala b/core/src/main/scala/org/apache/spark/sql/delta/actions/DeletionVectorDescriptor.scala index 2769f2146da..86594937017 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/actions/DeletionVectorDescriptor.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/actions/DeletionVectorDescriptor.scala @@ -227,6 +227,22 @@ object DeletionVectorDescriptor { sizeInBytes = data.length, cardinality = cardinality) + /** + * This produces the same output as [[DeletionVectorDescriptor.uniqueId]] but as a column + * expression, so it can be used directly in a Spark query. + */ + def uniqueIdExpression(deletionVectorCol: Column): Column = { + when(deletionVectorCol("offset").isNotNull, + concat( + deletionVectorCol("storageType"), + deletionVectorCol("pathOrInlineDv"), + lit('@'), + deletionVectorCol("offset"))) + .otherwise(concat( + deletionVectorCol("storageType"), + deletionVectorCol("pathOrInlineDv"))) + } + /** * Return the unique path under `parentPath` that is based on `id`. * diff --git a/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala index 1f6639d9934..a66c0c332fb 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala @@ -34,6 +34,7 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.core.JsonGenerator import com.fasterxml.jackson.databind._ import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} +import com.fasterxml.jackson.databind.node.ObjectNode import org.apache.hadoop.fs.Path @@ -521,6 +522,10 @@ sealed trait FileAction extends Action { @JsonIgnore def getFileSize: Long + /** Returns the approx size of the remaining records after excluding the deleted ones. */ + @JsonIgnore + def estLogicalFileSize: Option[Long] + /** * Return tag value if tags is not null and the tag present. */ @@ -579,6 +584,7 @@ case class AddFile( */ def removeRows( deletionVector: DeletionVectorDescriptor, + updateStats: Boolean, dataChange: Boolean = true): (AddFile, RemoveFile) = { // Verify DV does not contain any invalid row indexes. Note, maxRowIndex is optional // and not all commands may set it when updating DVs. @@ -595,7 +601,11 @@ case class AddFile( } val withUpdatedDV = this.copy(deletionVector = dvDescriptorWithoutMaxRowIndex, dataChange = dataChange) - val addFile = withUpdatedDV + val addFile = if (updateStats) { + withUpdatedDV.withoutTightBoundStats + } else { + withUpdatedDV + } val removeFile = this.removeWithTimestamp(dataChange = dataChange) (addFile, removeFile) } @@ -609,6 +619,22 @@ case class AddFile( @JsonIgnore def getDeletionVectorUniqueId: Option[String] = Option(deletionVector).map(_.uniqueId) + /** Update stats to have tightBounds = false, if file has any stats. */ + def withoutTightBoundStats: AddFile = { + if (stats == null || stats.isEmpty) { + this + } else { + val node = JsonUtils.mapper.readTree(stats).asInstanceOf[ObjectNode] + if (node.has("tightBounds") && + !node.get("tightBounds").asBoolean(true)) { + this + } else { + node.put("tightBounds", false) + val newStatsString = JsonUtils.mapper.writer.writeValueAsString(node) + this.copy(stats = newStatsString) + } + } + } @JsonIgnore lazy val insertionTime: Long = tag(AddFile.Tags.INSERTION_TIME).map(_.toLong) @@ -678,9 +704,9 @@ case class AddFile( @JsonIgnore def numPhysicalRecords: Option[Long] = numLogicalRecords.map(_ + numDeletedRecords) - /** Returns the approx size of the remaining records after excluding the deleted ones. */ @JsonIgnore - def estLogicalFileSize: Option[Long] = logicalToPhysicalRecordsRatio.map(n => (n * size).toLong) + override def estLogicalFileSize: Option[Long] = + logicalToPhysicalRecordsRatio.map(n => (n * size).toLong) /** Returns the ratio of the logical number of records to the total number of records. */ @JsonIgnore @@ -774,7 +800,6 @@ case class RemoveFile( @JsonIgnore var numLogicalRecords: Option[Long] = None - /** Returns the approx size of the remaining records after excluding the deleted ones. */ @JsonIgnore var estLogicalFileSize: Option[Long] = None @@ -831,6 +856,9 @@ case class AddCDCFile( @JsonIgnore override def getFileSize: Long = size + @JsonIgnore + override def estLogicalFileSize: Option[Long] = None + @JsonIgnore override def numLogicalRecords: Option[Long] = None } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala index b22962eddeb..0dd093ebb22 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala @@ -94,9 +94,12 @@ class DeltaCatalog extends DelegatingCatalogExtension case TableCatalog.PROP_OWNER => false case TableCatalog.PROP_EXTERNAL => false case "path" => false + case "option.path" => false case _ => true }.toMap - val (partitionColumns, maybeBucketSpec) = convertTransforms(partitions) + val ( + partitionColumns, maybeBucketSpec + ) = convertTransforms(partitions) var newSchema = schema var newPartitionColumns = partitionColumns var newBucketSpec = maybeBucketSpec @@ -256,6 +259,18 @@ class DeltaCatalog extends DelegatingCatalogExtension } + override def createTable( + ident: Identifier, + columns: Array[org.apache.spark.sql.connector.catalog.Column], + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + createTable( + ident, + org.apache.spark.sql.connector.catalog.CatalogV2Util.v2ColumnsToStructType(columns), + partitions, + properties) + } + override def createTable( ident: Identifier, schema: StructType, @@ -348,7 +363,9 @@ class DeltaCatalog extends DelegatingCatalogExtension } // Copy of V2SessionCatalog.convertTransforms, which is private. - private def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + private def convertTransforms(partitions: Seq[Transform]): ( + Seq[String], Option[BucketSpec] + ) = { val identityCols = new mutable.ArrayBuffer[String] var bucketSpec = Option.empty[BucketSpec] @@ -364,7 +381,9 @@ class DeltaCatalog extends DelegatingCatalogExtension throw DeltaErrors.operationNotSupportedException(s"Partitioning by expressions") } - (identityCols.toSeq, bucketSpec) + ( + identityCols.toSeq, bucketSpec + ) } /** Performs checks on the parameters provided for table creation for a Delta table. */ @@ -539,7 +558,8 @@ class DeltaCatalog extends DelegatingCatalogExtension col.dataType(), col.isNullable, Option(col.comment()), - Option(col.position()).map(UnresolvedFieldPosition) + Option(col.position()).map(UnresolvedFieldPosition), + Option(col.defaultValue()).map(_.getSql()) ) }).run(spark) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteCommand.scala index 89df2cd5d95..5a65a60ae99 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteCommand.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.delta.commands.DeleteCommand.{rewritingFilesMsg, FIN import org.apache.spark.sql.delta.commands.MergeIntoCommand.totalBytesAndDistinctPartitionValues import org.apache.spark.sql.delta.files.TahoeBatchFileIndex import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.util.Utils import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.SparkContext @@ -118,7 +117,7 @@ case class DeleteCommand( } val deleteActions = performDelete(sparkSession, deltaLog, txn) - txn.commitIfNeeded(deleteActions, DeltaOperations.Delete(condition.map(_.sql).toSeq)) + txn.commitIfNeeded(deleteActions, DeltaOperations.Delete(condition.toSeq)) } // Re-cache all cached plans(including this relation itself, if it's cached) that refer to // this data source relation. @@ -440,10 +439,8 @@ case class DeleteCommand( def shouldWritePersistentDeletionVectors( spark: SparkSession, txn: OptimisticTransaction): Boolean = { - // DELETE with DVs only enabled for tests. - Utils.isTesting && - spark.conf.get(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS) && - DeletionVectorUtils.deletionVectorsWritable(txn.snapshot) + spark.conf.get(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS) && + DeletionVectorUtils.deletionVectorsWritable(txn.snapshot) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteWithDeletionVectorsHelper.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteWithDeletionVectorsHelper.scala index 47ae2e734d4..4202742dec3 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteWithDeletionVectorsHelper.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeleteWithDeletionVectorsHelper.scala @@ -21,13 +21,14 @@ import java.util.UUID import scala.collection.generic.Sizing import org.apache.spark.sql.catalyst.expressions.aggregation.BitmapAggregator -import org.apache.spark.sql.delta.{DeltaLog, DeltaParquetFileFormat, OptimisticTransaction, Snapshot} +import org.apache.spark.sql.delta.{DeltaLog, DeltaParquetFileFormat, DeltaUDF, OptimisticTransaction, Snapshot} import org.apache.spark.sql.delta.DeltaParquetFileFormat._ import org.apache.spark.sql.delta.actions.{AddFile, DeletionVectorDescriptor, FileAction} import org.apache.spark.sql.delta.deletionvectors.{RoaringBitmapArray, RoaringBitmapArrayFormat, StoredBitmap} import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeFileIndex} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.StatsCollectionUtils import org.apache.spark.sql.delta.storage.dv.DeletionVectorStore import org.apache.spark.sql.delta.util.{BinPackingIterator, DeltaEncoder, JsonUtils, PathWithFileSystem, Utils => DeltaUtils} import org.apache.spark.sql.delta.util.DeltaFileOperations.absolutePath @@ -75,7 +76,8 @@ object DeleteWithDeletionVectorsHelper extends DeltaCommand { private def replaceFileIndex(target: LogicalPlan, fileIndex: TahoeFileIndex): LogicalPlan = { val additionalCols = Seq( AttributeReference(ROW_INDEX_COLUMN_NAME, ROW_INDEX_STRUCT_FILED.dataType)(), - FileFormat.createFileMetadataCol + // TODO: when upgrading to Spark 3.5 or 4.0 this should be FileFormat.createFileMetadataCol + FileSourceMetadataAttribute(FileFormat.METADATA_NAME, FileFormat.BASE_METADATA_STRUCT) ) val newTarget = target transformDown { @@ -170,23 +172,24 @@ object DeleteWithDeletionVectorsHelper extends DeltaCommand { val dvUpdates = notFullyRemovedFiles.map { fileWithDVInfo => fileWithDVInfo.fileLogEntry.removeRows( - deletionVector = fileWithDVInfo.newDeletionVector + deletionVector = fileWithDVInfo.newDeletionVector, + updateStats = false )} val (dvAddFiles, dvRemoveFiles) = dvUpdates.unzip val dvAddFilesWithStats = getActionsWithStats(spark, dvAddFiles, snapshot) - val (filesWithDeletedRows, newFilesWithDVs) = dvUpdates.unzip fullyRemoved ++ dvAddFilesWithStats ++ dvRemoveFiles } /** Fetch stats for `addFiles`. */ private def getActionsWithStats( spark: SparkSession, - addFiles: Seq[AddFile], + addFilesWithNewDvs: Seq[AddFile], snapshot: Snapshot): Seq[AddFile] = { import org.apache.spark.sql.delta.implicits._ val statsColName = snapshot.getBaseStatsColumnName - val selectionCols = Seq(col("path"), col(statsColName)) + val selectionPathAndStatsCols = Seq(col("path"), col(statsColName)) + val addFilesWithNewDvsDf = addFilesWithNewDvs.toDF(spark) // These files originate from snapshot.filesForScan which resets column statistics. // Since these object don't carry stats and tags, if we were to use them as result actions of @@ -194,13 +197,52 @@ object DeleteWithDeletionVectorsHelper extends DeltaCommand { // we join the list of files with DVs with the log (allFiles) to retrieve statistics. This is // expected to have better performance than supporting full stats retrieval // in snapshot.filesForScan because it only affects a subset of the scanned files. - val allFiles = snapshot.withStats.select(selectionCols: _*) - val addFilesDf = addFiles.toDF(spark).drop("stats") - val addFilesWithStats = addFilesDf.join(allFiles, "path") - // Every operation that adds DVs needs to set tightBounds to false. - snapshot - .updateStatsToWideBounds(addFilesWithStats, statsColName) + // Find the current metadata with stats for all files with new DV + val addFileWithStatsDf = snapshot.withStats + .join(addFilesWithNewDvsDf.select("path"), "path") + + // Update the existing stats to set the tightBounds to false and also set the appropriate + // null count. We want to set the bounds before the AddFile has DV descriptor attached. + // Attaching the DV descriptor here, causes wrong logical records computation in + // `updateStatsToWideBounds`. + val addFilesWithWideBoundsDf = snapshot + .updateStatsToWideBounds(addFileWithStatsDf, statsColName) + + val (filesWithNoStats, filesWithExistingStats) = { + // numRecords is the only stat we really have to guarantee. + // If the others are missing, we do not need to fetch them. + addFilesWithWideBoundsDf.as[AddFile].collect().toSeq + .partition(_.numPhysicalRecords.isEmpty) + } + + // If we encounter files with no stats we fetch the stats from the parquet footer. + // Files with persistent DVs *must* have (at least numRecords) stats according to the + // Delta spec. + val filesWithFetchedStats = + if (filesWithNoStats.nonEmpty) { + StatsCollectionUtils.computeStats(spark, + conf = snapshot.deltaLog.newDeltaHadoopConf(), + dataPath = snapshot.deltaLog.dataPath, + addFiles = filesWithNoStats.toDS(spark), + columnMappingMode = snapshot.metadata.columnMappingMode, + dataSchema = snapshot.dataSchema, + statsSchema = snapshot.statsSchema, + setBoundsToWide = true) + .collect() + .toSeq + } else { + Seq.empty + } + + val allAddFilesWithUpdatedStats = + (filesWithExistingStats ++ filesWithFetchedStats).toSeq.toDF(spark) + + // Now join the allAddFilesWithUpdatedStats with addFilesWithNewDvs + // so that the updated stats are joined with the new DV info + addFilesWithNewDvsDf.drop("stats") + .join( + allAddFilesWithUpdatedStats.select(selectionPathAndStatsCols: _*), "path") .as[AddFile] .collect() .toSeq @@ -293,8 +335,13 @@ object DeletionVectorBitmapGenerator { candidateFiles: Seq[AddFile], condition: Expression) : Seq[DeletionVectorResult] = { + // TODO: fix this to work regardless of whether Spark encodes or doesn't encode + // _metadata.file_path. See https://github.com/delta-io/delta/issues/1725 + val uriEncode = DeltaUDF.stringFromString(path => { + new Path(path).toUri.toString + }) val matchedRowsDf = targetDf - .withColumn(FILE_NAME_COL, col(s"${METADATA_NAME}.${FILE_PATH}")) + .withColumn(FILE_NAME_COL, uriEncode(col(s"${METADATA_NAME}.${FILE_PATH}"))) // Filter after getting input file name as the filter might introduce a join and we // cannot get input file name on join's output. .filter(new Column(condition)) @@ -312,7 +359,15 @@ object DeletionVectorBitmapGenerator { val filePathToDVDf = sparkSession.createDataset(filePathToDV) val joinExpr = filePathToDVDf("path") === matchedRowsDf(FILE_NAME_COL) - matchedRowsDf.join(filePathToDVDf, joinExpr) + val joinedDf = matchedRowsDf.join(filePathToDVDf, joinExpr) + assert(joinedDf.count() == matchedRowsDf.count(), + s""" + |The joined DataFrame should contain the same number of entries as the original + |DataFrame. It is likely that _metadata.file_path is not encoded by Spark as expected. + |Joined DataFrame count: ${joinedDf.count()} + |matchedRowsDf count: ${matchedRowsDf.count()} + |""".stripMargin) + joinedDf } else { // When the table has no DVs, just add a column to indicate that the existing dv is null matchedRowsDf.withColumn(FILE_DV_ID_COL, lit(null)) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeletionVectorUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeletionVectorUtils.scala index 03c0e276cba..2a784b5c261 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeletionVectorUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeletionVectorUtils.scala @@ -85,43 +85,6 @@ trait DeletionVectorUtils { protocol.isFeatureSupported(DeletionVectorsTableFeature) && metadata.format.provider == "parquet" // DVs are only supported on parquet tables. } - - /** - * Utility method that checks the table has no Deletion Vectors enabled. Deletion vectors - * are supported in read-only mode for now. Any updates to tables with deletion vectors - * feature are disabled until we add support. - */ - def assertDeletionVectorsNotReadable( - spark: SparkSession, metadata: Metadata, protocol: Protocol): Unit = { - val disable = - Utils.isTesting && // We are in testing and enabled blocking updates on DV tables - spark.conf.get(DeltaSQLConf.DELTA_ENABLE_BLOCKING_UPDATES_ON_DV_TABLES) - if (!disable && deletionVectorsReadable(protocol, metadata)) { - throw new UnsupportedOperationException( - "Updates to tables with Deletion Vectors feature enabled are not supported in " + - "this version of Delta Lake.") - } - } - - /** - * Utility method that checks the table metadata has no deletion vectors enabled. Deletion vectors - * are supported in read-only mode for now. Any updates to metadata to enable deletion vectors are - * blocked until we add support. - */ - def assertDeletionVectorsNotEnabled( - spark: SparkSession, metadata: Metadata, protocol: Protocol): Unit = { - val disable = - Utils.isTesting && // We are in testing and enabled blocking updates on DV tables - spark.conf.get(DeltaSQLConf.DELTA_ENABLE_BLOCKING_UPDATES_ON_DV_TABLES) - if (!disable && - (protocol.isFeatureSupported(DeletionVectorsTableFeature) || - DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.fromMetaData(metadata) - ) - ) { - throw new UnsupportedOperationException( - "Enabling Deletion Vectors on the table is not supported in this version of Delta Lake.") - } - } } // To access utilities from places where mixing in a trait is inconvenient. diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala new file mode 100644 index 00000000000..c8d98cf4cc9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala @@ -0,0 +1,55 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.commands + +import org.apache.spark.sql.delta.catalog.DeltaTableV2 + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LeafCommand, LogicalPlan, UnaryCommand} + +case class DeltaReorgTable(target: LogicalPlan)(val predicates: Seq[String]) extends UnaryCommand { + + def child: LogicalPlan = target + + protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = + copy(target = newChild)(predicates) + + override val otherCopyArgs: Seq[AnyRef] = predicates :: Nil +} + +/** + * The PURGE command. + */ +case class DeltaReorgTableCommand(target: DeltaTableV2)(val predicates: Seq[String]) + extends OptimizeTableCommandBase with LeafCommand with IgnoreCachedData { + + override val otherCopyArgs: Seq[AnyRef] = predicates :: Nil + + override def run(sparkSession: SparkSession): Seq[Row] = { + val command = OptimizeTableCommand( + Option(target.path.toString), + target.catalogTable.map(_.identifier), + predicates, + options = Map.empty, + optimizeContext = DeltaOptimizeContext( + isPurge = true, + minFileSize = Some(0L), + maxDeletedRowsRatio = Some(0d)) + )(zOrderBy = Nil) + command.run(sparkSession) + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala index 7e599cfc498..d7df847f427 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala @@ -258,7 +258,7 @@ case class MergeIntoCommand( deltaTxn.commitIfNeeded( finalActions, DeltaOperations.Merge( - Option(condition.sql), + Option(condition), matchedClauses.map(DeltaOperations.MergePredicate(_)), notMatchedClauses.map(DeltaOperations.MergePredicate(_)), notMatchedBySourceClauses.map(DeltaOperations.MergePredicate(_)))) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index 70e8e327b9d..97ea00d84ce 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -109,7 +109,9 @@ case class OptimizeTableCommand( path: Option[String], tableId: Option[TableIdentifier], userPartitionPredicates: Seq[String], - options: Map[String, String])(val zOrderBy: Seq[UnresolvedAttribute]) + options: Map[String, String], + optimizeContext: DeltaOptimizeContext = DeltaOptimizeContext() +)(val zOrderBy: Seq[UnresolvedAttribute]) extends OptimizeTableCommandBase with LeafRunnableCommand { override val otherCopyArgs: Seq[AnyRef] = zOrderBy :: Nil @@ -138,7 +140,34 @@ case class OptimizeTableCommand( validateZorderByColumns(sparkSession, txn, zOrderBy) val zOrderByColumns = zOrderBy.map(_.name).toSeq - new OptimizeExecutor(sparkSession, txn, partitionPredicates, zOrderByColumns).optimize() + new OptimizeExecutor(sparkSession, txn, partitionPredicates, zOrderByColumns, optimizeContext) + .optimize() + } +} + +/** + * Stored all runtime context information that can control the execution of optimize. + * + * @param isPurge Whether the rewriting task is only for purging soft-deleted data instead of + * for compaction. If [[isPurge]] is true, only files with DVs will be selected + * for compaction. + * @param minFileSize Files which are smaller than this threshold will be selected for compaction. + * If not specified, [[DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE]] will be used. + * This parameter must be set to `0` when [[isPurge]] is true. + * @param maxDeletedRowsRatio Files with a ratio of soft-deleted rows to the total rows larger than + * this threshold will be rewritten by the OPTIMIZE command. If not + * specified, [[DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO]] + * will be used. This parameter must be set to `0` when [[isPurge]] is + * true. + */ +case class DeltaOptimizeContext( + isPurge: Boolean = false, + minFileSize: Option[Long] = None, + maxDeletedRowsRatio: Option[Double] = None) { + if (isPurge) { + require( + minFileSize.contains(0L) && maxDeletedRowsRatio.contains(0d), + "minFileSize and maxDeletedRowsRatio must be 0 when running PURGE.") } } @@ -154,7 +183,8 @@ class OptimizeExecutor( sparkSession: SparkSession, txn: OptimisticTransaction, partitionPredicate: Seq[Expression], - zOrderByColumns: Seq[String]) + zOrderByColumns: Seq[String], + optimizeContext: DeltaOptimizeContext) extends DeltaCommand with SQLMetricsReporting with Serializable { /** Timestamp to use in [[FileAction]] */ @@ -164,18 +194,16 @@ class OptimizeExecutor( def optimize(): Seq[Row] = { recordDeltaOperation(txn.deltaLog, "delta.optimize") { - val minFileSize = sparkSession.sessionState.conf.getConf( - DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE) - val maxFileSize = sparkSession.sessionState.conf.getConf( - DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) - require(minFileSize > 0, "minFileSize must be > 0") - require(maxFileSize > 0, "maxFileSize must be > 0") + val minFileSize = optimizeContext.minFileSize.getOrElse( + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE)) + val maxFileSize = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) + val maxDeletedRowsRatio = optimizeContext.maxDeletedRowsRatio.getOrElse( + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO)) val candidateFiles = txn.filterFiles(partitionPredicate, keepNumRecords = true) val partitionSchema = txn.metadata.partitionSchema - val maxDeletedRowsRatio = sparkSession.sessionState.conf.getConf( - DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO) val filesToProcess = pruneCandidateFileList(minFileSize, maxDeletedRowsRatio, candidateFiles) val partitionsToCompact = filesToProcess.groupBy(_.partitionValues).toSeq @@ -191,7 +219,7 @@ class OptimizeExecutor( val removedFiles = updates.collect { case r: RemoveFile => r } val removedDVs = filesToProcess.filter(_.deletionVector != null).map(_.deletionVector).toSeq if (addedFiles.size > 0) { - val operation = DeltaOperations.Optimize(partitionPredicate.map(_.sql), zOrderByColumns) + val operation = DeltaOperations.Optimize(partitionPredicate, zOrderByColumns) val metrics = createMetrics(sparkSession.sparkContext, addedFiles, removedFiles, removedDVs) commitAndRetry(txn, operation, updates, metrics) { newTxn => val newPartitionSchema = newTxn.metadata.partitionSchema diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/RestoreTableCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/RestoreTableCommand.scala index 24b700ea8e7..8d7a76d1c93 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/RestoreTableCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/RestoreTableCommand.scala @@ -22,17 +22,18 @@ import scala.collection.JavaConverters._ import scala.util.{Success, Try} import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, Snapshot} -import org.apache.spark.sql.delta.actions.{AddFile, RemoveFile} +import org.apache.spark.sql.delta.actions.{AddFile, DeletionVectorDescriptor, RemoveFile} import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.DeltaFileOperations.absolutePath import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.functions.{column, lit} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.IGNORE_MISSING_FILES import org.apache.spark.sql.types.LongType @@ -120,26 +121,63 @@ case class RestoreTableCommand( import org.apache.spark.sql.delta.implicits._ - val filesToAdd = snapshotToRestoreFiles - .join( - latestSnapshotFiles, - snapshotToRestoreFiles("path") === latestSnapshotFiles("path"), - "left_anti") - .as[AddFile] + // If either source version or destination version contains DVs, + // we have to take them into account during deduplication. + val targetMayHaveDVs = DeletionVectorUtils.deletionVectorsReadable(latestSnapshot) + val sourceMayHaveDVs = DeletionVectorUtils.deletionVectorsReadable(snapshotToRestore) + + val normalizedSourceWithoutDVs = snapshotToRestoreFiles.mapPartitions { files => + files.map(file => (file, file.path)) + }.toDF("srcAddFile", "srcPath") + val normalizedTargetWithoutDVs = latestSnapshotFiles.mapPartitions { files => + files.map(file => (file, file.path)) + }.toDF("tgtAddFile", "tgtPath") + + def addDVsToNormalizedDF( + mayHaveDVs: Boolean, + dvIdColumnName: String, + dvAccessColumn: Column, + normalizedDf: DataFrame): DataFrame = { + if (mayHaveDVs) { + normalizedDf.withColumn( + dvIdColumnName, + DeletionVectorDescriptor.uniqueIdExpression(dvAccessColumn)) + } else { + normalizedDf.withColumn(dvIdColumnName, lit(null)) + } + } + + val normalizedSource = addDVsToNormalizedDF( + mayHaveDVs = sourceMayHaveDVs, + dvIdColumnName = "srcDeletionVectorId", + dvAccessColumn = column("srcAddFile.deletionVector"), + normalizedDf = normalizedSourceWithoutDVs) + + val normalizedTarget = addDVsToNormalizedDF( + mayHaveDVs = targetMayHaveDVs, + dvIdColumnName = "tgtDeletionVectorId", + dvAccessColumn = column("tgtAddFile.deletionVector"), + normalizedDf = normalizedTargetWithoutDVs) + + val joinExprs = + column("srcPath") === column("tgtPath") and + // Use comparison operator where NULL == NULL + column("srcDeletionVectorId") <=> column("tgtDeletionVectorId") + + val filesToAdd = normalizedSource + .join(normalizedTarget, joinExprs, "left_anti") + .select(column("srcAddFile").as[AddFile]) .map(_.copy(dataChange = true)) - val filesToRemove = latestSnapshotFiles - .join( - snapshotToRestoreFiles, - latestSnapshotFiles("path") === snapshotToRestoreFiles("path"), - "left_anti") - .as[AddFile] + val filesToRemove = normalizedTarget + .join(normalizedSource, joinExprs, "left_anti") + .select(column("tgtAddFile").as[AddFile]) .map(_.removeWithTimestamp()) val ignoreMissingFiles = spark - .sessionState - .conf - .getConf(IGNORE_MISSING_FILES) + .sessionState + .conf + .getConf(IGNORE_MISSING_FILES) if (!ignoreMissingFiles) { checkSnapshotFilesAvailability(deltaLog, filesToAdd, versionToRestore) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/UpdateCommand.scala index 9c7dffb4191..2086bcfe532 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/UpdateCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/UpdateCommand.scala @@ -219,7 +219,7 @@ case class UpdateCommand( txn.registerSQLMetrics(sparkSession, metrics) val finalActions = createSetTransaction(sparkSession, deltaLog).toSeq ++ totalActions - txn.commitIfNeeded(finalActions, DeltaOperations.Update(condition.map(_.toString))) + txn.commitIfNeeded(finalActions, DeltaOperations.Update(condition)) sendDriverMetrics(sparkSession, metrics) recordDeltaEvent( diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala index f9a7495b7c1..d9bb8dc543c 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -114,9 +114,6 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { require(snapshot.version >= 0, "No state defined for this table. Is this really " + "a Delta table? Refusing to garbage collect.") - DeletionVectorUtils.assertDeletionVectorsNotReadable( - spark, snapshot.metadata, snapshot.protocol) - val snapshotTombstoneRetentionMillis = DeltaLog.tombstoneRetentionMillis(snapshot.metadata) val retentionMillis = retentionHours.map(h => TimeUnit.HOURS.toMillis(math.round(h))) checkRetentionPeriodSafety(spark, retentionMillis, snapshotTombstoneRetentionMillis) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala index 4a950d95832..db3187e969e 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala @@ -198,8 +198,7 @@ case class WriteIntoDelta( } } val rearrangeOnly = options.rearrangeOnly - // TODO: use `SQLConf.READ_SIDE_CHAR_PADDING` after Spark 3.4 is released. - val charPadding = sparkSession.conf.get("spark.sql.readSideCharPadding", "false") == "true" + val charPadding = sparkSession.conf.get(SQLConf.READ_SIDE_CHAR_PADDING.key, "false") == "true" val charAsVarchar = sparkSession.conf.get(SQLConf.CHAR_AS_VARCHAR) val dataSchema = if (!charAsVarchar && charPadding) { data.schema @@ -236,6 +235,10 @@ case class WriteIntoDelta( } else options.isDynamicPartitionOverwriteMode } + if (useDynamicPartitionOverwriteMode && canOverwriteSchema) { + throw DeltaErrors.overwriteSchemaUsedWithDynamicPartitionOverwrite() + } + // Validate partition predicates var containsDataFilters = false val replaceWhere = options.replaceWhere.flatMap { replace => diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala index 69ef3ff65aa..29e067cb20d 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala @@ -219,15 +219,19 @@ case class AlterTableAddColumnsDeltaCommand( val resolver = sparkSession.sessionState.conf.resolver val newSchema = colsToAddWithPosition.foldLeft(oldSchema) { case (schema, QualifiedColTypeWithPosition(columnPath, column, None)) => - val (parentPosition, lastSize) = - SchemaUtils.findColumnPosition(columnPath, schema, resolver) - SchemaUtils.addColumn(schema, column, parentPosition :+ lastSize) + val parentPosition = SchemaUtils.findColumnPosition(columnPath, schema, resolver) + val insertPosition = SchemaUtils.getNestedTypeFromPosition(schema, parentPosition) match { + case s: StructType => s.size + case other => + throw DeltaErrors.addColumnParentNotStructException(column, other) + } + SchemaUtils.addColumn(schema, column, parentPosition :+ insertPosition) case (schema, QualifiedColTypeWithPosition(columnPath, column, Some(_: First))) => - val (parentPosition, _) = SchemaUtils.findColumnPosition(columnPath, schema, resolver) + val parentPosition = SchemaUtils.findColumnPosition(columnPath, schema, resolver) SchemaUtils.addColumn(schema, column, parentPosition :+ 0) case (schema, QualifiedColTypeWithPosition(columnPath, column, Some(after: After))) => - val (prevPosition, _) = + val prevPosition = SchemaUtils.findColumnPosition(columnPath :+ after.column, schema, resolver) val position = prevPosition.init :+ (prevPosition.last + 1) SchemaUtils.addColumn(schema, column, position) @@ -294,7 +298,7 @@ case class AlterTableDropColumnsDeltaCommand( throw DeltaErrors.dropColumnNotSupported(suggestUpgrade = true) } val newSchema = columnsToDrop.foldLeft(metadata.schema) { case (schema, columnPath) => - val (parentPosition, _) = + val parentPosition = SchemaUtils.findColumnPosition( columnPath, schema, sparkSession.sessionState.conf.resolver) SchemaUtils.dropColumn(schema, parentPosition)._1 diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeIntoMaterializeSource.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeIntoMaterializeSource.scala index 10a7be51d46..fe28f1f2818 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeIntoMaterializeSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeIntoMaterializeSource.scala @@ -275,18 +275,8 @@ trait MergeIntoMaterializeSource extends DeltaLogging { // We should still keep the hints from the input plan. checkpointedPlan = addHintsToPlan(source, checkpointedPlan) - // FIXME(SPARK-39834): Can be removed once Delta adopts Spark 3.4 and constraints are propagated - // Add filters to the logical plan so the optimizer can pick up the constraints even though - // they are lost when materializing. - checkpointedPlan = addFiltersForConstraintsToPlan( - sourceWithSelectedColumns.constraints, checkpointedPlan) - sourceDF = Some(Dataset.ofRows(spark, checkpointedPlan)) - // FIXME(SPARK-39834): This can be removed once Delta adopts Spark 3.4 as the statistics - // will be materialized - // and the optimal join will be picked during planning - sourceDF = Some(addBroadcastHintToDF(sourceWithSelectedColumns, sourceDF.get)) // Sets appropriate StorageLevel val storageLevel = StorageLevel.fromString( @@ -328,29 +318,6 @@ trait MergeIntoMaterializeSource extends DeltaLogging { } } - private def addFiltersForConstraintsToPlan( - constraints: ExpressionSet, - plan: LogicalPlan): LogicalPlan = { - if (constraints.nonEmpty) { - val planWithConstraints = - constraints.foldRight[LogicalPlan](plan) {(expr, updatedPlan) => - Filter(expr, updatedPlan) - } - planWithConstraints - } else { - plan - } - } - - private def addBroadcastHintToDF(sourcePlan: LogicalPlan, df: Dataset[Row]): Dataset[Row] = { - val joinSelectionHelper = new Object with JoinSelectionHelper - if (joinSelectionHelper.canBroadcastBySize(sourcePlan, sourcePlan.conf)) { - df.hint("broadcast") - } else { - df - } - } - /** * Return columns from the source plan that are used in the MERGE */ diff --git a/core/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/core/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index 0b0da61fd1f..e7b28ac4485 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -196,7 +196,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl val projectList: Seq[NamedExpression] = plan.output.map { case p if partSet.contains(p) && p.dataType == StringType => needConvert = true - Alias(FileFormatWriter.Empty2Null(p), p.name)() + Alias(org.apache.spark.sql.catalyst.expressions.Empty2Null(p), p.name)() case attr => attr } if (needConvert) ProjectExec(projectList, plan) else plan diff --git a/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala index 820d3cd0db0..2434afa322c 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala @@ -518,8 +518,8 @@ object SchemaUtils extends DeltaLogging { } /** - * Returns the given column's ordinal within the given `schema` and the size of the last schema - * size. The length of the returned position will be as long as how nested the column is. + * Returns the path of the given column in `schema` as a list of ordinals (0-based), each value + * representing the position at the current nesting level starting from the root. * * For ArrayType: accessing the array's element adds a position 0 to the position list. * e.g. accessing a.element.y would have the result -> Seq(..., positionOfA, 0, positionOfY) @@ -538,65 +538,109 @@ object SchemaUtils extends DeltaLogging { def findColumnPosition( column: Seq[String], schema: StructType, - resolver: Resolver = DELTA_COL_RESOLVER): (Seq[Int], Int) = { - def find(column: Seq[String], schema: StructType, stack: Seq[String]): (Seq[Int], Int) = { - if (column.isEmpty) return (Nil, schema.size) - val thisCol = column.head - lazy val columnPath = UnresolvedAttribute(stack :+ thisCol).name - val pos = schema.indexWhere(f => resolver(f.name, thisCol)) - if (pos == -1) { - throw new IndexOutOfBoundsException(columnPath) - } - val colTail = column.tail - val (children, lastSize) = (colTail, schema(pos).dataType) match { - case (_, s: StructType) => - find(colTail, s, stack :+ thisCol) - case (Seq("element", _ @ _*), ArrayType(s: StructType, _)) => - val (child, size) = find(colTail.tail, s, stack :+ thisCol) - (ARRAY_ELEMENT_INDEX +: child, size) - case (Seq(), ArrayType(s: StructType, _)) => - find(colTail, s, stack :+ thisCol) - case (Seq(), ArrayType(_, _)) => - (Nil, 0) - case (_, ArrayType(_, _)) => - throw DeltaErrors.incorrectArrayAccessByName( - prettyFieldName(stack ++ Seq(thisCol, "element")), - prettyFieldName(stack ++ Seq(thisCol))) - case (Seq(), MapType(_, _, _)) => - (Nil, 2) - case (Seq("key", _ @ _*), MapType(keyType: StructType, _, _)) => - val (child, size) = find(colTail.tail, keyType, stack :+ thisCol) - (MAP_KEY_INDEX +: child, size) - case (Seq("key"), MapType(_, _, _)) => - (Seq(MAP_KEY_INDEX), 0) - case (Seq("value", _ @ _*), MapType(_, valueType: StructType, _)) => - val (child, size) = find(colTail.tail, valueType, stack :+ thisCol) - (MAP_VALUE_INDEX +: child, size) - case (Seq("value"), MapType(_, _, _)) => - (Seq(MAP_VALUE_INDEX), 0) - case (_, MapType(_, _, _)) => - throw DeltaErrors.foundMapTypeColumnException( - prettyFieldName(stack ++ Seq(thisCol, "key")), - prettyFieldName(stack ++ Seq(thisCol, "value"))) - case (_, o) => - if (column.length > 1) { - throw DeltaErrors.columnPathNotNested(columnPath, o, column) + resolver: Resolver = DELTA_COL_RESOLVER): Seq[Int] = { + def findRecursively( + searchPath: Seq[String], + currentType: DataType, + currentPath: Seq[String] = Nil): Seq[Int] = { + if (searchPath.isEmpty) return Nil + + val currentFieldName = searchPath.head + val currentPathWithNestedField = currentPath :+ currentFieldName + (currentType, currentFieldName) match { + case (struct: StructType, _) => + lazy val columnPath = UnresolvedAttribute(currentPathWithNestedField).name + val pos = struct.indexWhere(f => resolver(f.name, currentFieldName)) + if (pos == -1) { + throw DeltaErrors.columnNotInSchemaException(columnPath, schema) } - (Nil, 0) + val childPosition = findRecursively( + searchPath = searchPath.tail, + currentType = struct(pos).dataType, + currentPath = currentPathWithNestedField) + pos +: childPosition + + case (map: MapType, "key") => + val childPosition = findRecursively( + searchPath = searchPath.tail, + currentType = map.keyType, + currentPath = currentPathWithNestedField) + MAP_KEY_INDEX +: childPosition + + case (map: MapType, "value") => + val childPosition = findRecursively( + searchPath = searchPath.tail, + currentType = map.valueType, + currentPath = currentPathWithNestedField) + MAP_VALUE_INDEX +: childPosition + + case (_: MapType, _) => + throw DeltaErrors.foundMapTypeColumnException( + prettyFieldName(currentPath :+ "key"), + prettyFieldName(currentPath :+ "value")) + + case (array: ArrayType, "element") => + val childPosition = findRecursively( + searchPath = searchPath.tail, + currentType = array.elementType, + currentPath = currentPathWithNestedField) + ARRAY_ELEMENT_INDEX +: childPosition + + case (_: ArrayType, _) => + throw DeltaErrors.incorrectArrayAccessByName( + prettyFieldName(currentPath :+ "element"), + prettyFieldName(currentPath)) + case _ => + throw DeltaErrors.columnPathNotNested(currentFieldName, currentType, currentPath) } - (Seq(pos) ++ children, lastSize) } try { - find(column, schema, Nil) + findRecursively(column, schema) } catch { - case i: IndexOutOfBoundsException => - throw DeltaErrors.columnNotInSchemaException(i.getMessage, schema) case e: AnalysisException => throw new AnalysisException(e.getMessage + s":\n${schema.treeString}") } } + /** + * Returns the nested field at the given position in `parent`. See [[findColumnPosition]] for the + * representation used for `position`. + * @param parent The field used for the lookup. + * @param position A list of ordinals (0-based) representing the path to the nested field in + * `parent`. + */ + def getNestedFieldFromPosition(parent: StructField, position: Seq[Int]): StructField = { + if (position.isEmpty) return parent + + val fieldPos = position.head + parent.dataType match { + case struct: StructType if fieldPos >= 0 && fieldPos < struct.size => + getNestedFieldFromPosition(struct(fieldPos), position.tail) + case map: MapType if fieldPos == MAP_KEY_INDEX => + getNestedFieldFromPosition(StructField("key", map.keyType), position.tail) + case map: MapType if fieldPos == MAP_VALUE_INDEX => + getNestedFieldFromPosition(StructField("value", map.valueType), position.tail) + case array: ArrayType if fieldPos == ARRAY_ELEMENT_INDEX => + getNestedFieldFromPosition(StructField("element", array.elementType), position.tail) + case _: StructType | _: ArrayType | _: MapType => + throw new IllegalArgumentException( + s"Invalid child position $fieldPos in ${parent.dataType}") + case other => + throw new IllegalArgumentException(s"Invalid indexing into non-nested type $other") + } + } + + /** + * Returns the nested type at the given position in `schema`. See [[findColumnPosition]] for the + * representation used for `position`. + * @param parent The root schema used for the lookup. + * @param position A list of ordinals (0-based) representing the path to the nested field in + * `parent`. + */ + def getNestedTypeFromPosition(schema: StructType, position: Seq[Int]): DataType = + getNestedFieldFromPosition(StructField("schema", schema), position).dataType + /** * Pretty print the column path passed in. */ @@ -616,6 +660,24 @@ object SchemaUtils extends DeltaLogging { * result: , b,c:STRUCT> */ def addColumn(schema: StructType, column: StructField, position: Seq[Int]): StructType = { + def addColumnInChild(parent: DataType, column: StructField, position: Seq[Int]): DataType = { + require(position.nonEmpty, s"Don't know where to add the column $column") + parent match { + case struct: StructType => + addColumn(struct, column, position) + case map: MapType if position.head == MAP_KEY_INDEX => + map.copy(keyType = addColumnInChild(map.keyType, column, position.tail)) + case map: MapType if position.head == MAP_VALUE_INDEX => + map.copy(valueType = addColumnInChild(map.valueType, column, position.tail)) + case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => + array.copy(elementType = addColumnInChild(array.elementType, column, position.tail)) + case _: ArrayType => + throw DeltaErrors.incorrectArrayAccess() + case other => + throw DeltaErrors.addColumnParentNotStructException(column, other) + } + } + require(position.nonEmpty, s"Don't know where to add the column $column") val slicePosition = position.head if (slicePosition < 0) { @@ -632,53 +694,16 @@ object SchemaUtils extends DeltaLogging { } return StructType(schema :+ column) } - val pre = schema.take(slicePosition) + val (pre, post) = schema.splitAt(slicePosition) if (position.length > 1) { - val posTail = position.tail - val mid = schema(slicePosition) match { - case StructField(name, f: StructType, nullable, metadata) => - if (!column.nullable && nullable) { - throw DeltaErrors.nullableParentWithNotNullNestedField - } - StructField( - name, - addColumn(f, column, posTail), - nullable, - metadata) - case StructField(name, ArrayType(f: StructType, containsNull), nullable, metadata) => - if (!column.nullable && nullable) { - throw DeltaErrors.nullableParentWithNotNullNestedField - } - - if (posTail.head != ARRAY_ELEMENT_INDEX) { - throw DeltaErrors.incorrectArrayAccess() - } - - StructField( - name, - ArrayType(addColumn(f, column, posTail.tail), containsNull), - nullable, - metadata) - case StructField(name, map @ MapType(_, _, _), nullable, metadata) => - if (!column.nullable && nullable) { - throw DeltaErrors.nullableParentWithNotNullNestedField - } - - val addedMap = (posTail.head, map) match { - case (MAP_KEY_INDEX, MapType(key: StructType, v, nullability)) => - MapType(addColumn(key, column, posTail.tail), v, nullability) - case (MAP_VALUE_INDEX, MapType(k, value: StructType, nullability)) => - MapType(k, addColumn(value, column, posTail.tail), nullability) - case _ => - throw DeltaErrors.addColumnParentNotStructException(column, IntegerType) - } - StructField(name, addedMap, nullable, metadata) - case o => - throw DeltaErrors.addColumnParentNotStructException(column, o.dataType) + val field = post.head + if (!column.nullable && field.nullable) { + throw DeltaErrors.nullableParentWithNotNullNestedField } - StructType(pre ++ Seq(mid) ++ schema.slice(slicePosition + 1, length)) + val mid = field.copy(dataType = addColumnInChild(field.dataType, column, position.tail)) + StructType(pre ++ Seq(mid) ++ post.tail) } else { - StructType(pre ++ Seq(column) ++ schema.slice(slicePosition, length)) + StructType(pre ++ Seq(column) ++ post) } } @@ -693,6 +718,27 @@ object SchemaUtils extends DeltaLogging { * result: , b,c:STRUCT> */ def dropColumn(schema: StructType, position: Seq[Int]): (StructType, StructField) = { + def dropColumnInChild(parent: DataType, position: Seq[Int]): (DataType, StructField) = { + require(position.nonEmpty, s"Don't know where to drop the column") + parent match { + case struct: StructType => + dropColumn(struct, position) + case map: MapType if position.head == MAP_KEY_INDEX => + val (newKeyType, droppedColumn) = dropColumnInChild(map.keyType, position.tail) + map.copy(keyType = newKeyType) -> droppedColumn + case map: MapType if position.head == MAP_VALUE_INDEX => + val (newValueType, droppedColumn) = dropColumnInChild(map.valueType, position.tail) + map.copy(valueType = newValueType) -> droppedColumn + case array: ArrayType if position.head == ARRAY_ELEMENT_INDEX => + val (newElementType, droppedColumn) = dropColumnInChild(array.elementType, position.tail) + array.copy(elementType = newElementType) -> droppedColumn + case _: ArrayType => + throw DeltaErrors.incorrectArrayAccess() + case other => + throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(other) + } + } + require(position.nonEmpty, "Don't know where to drop the column") val slicePosition = position.head if (slicePosition < 0) { @@ -702,22 +748,19 @@ object SchemaUtils extends DeltaLogging { if (slicePosition >= length) { throw DeltaErrors.indexLargerOrEqualThanStruct(slicePosition, length) } - val pre = schema.take(slicePosition) + val (pre, post) = schema.splitAt(slicePosition) + val field = post.head if (position.length > 1) { - val (mid, original) = schema(slicePosition) match { - case StructField(name, f: StructType, nullable, metadata) => - val (dropped, original) = dropColumn(f, position.tail) - (StructField(name, dropped, nullable, metadata), original) - case o => - throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(o) - } - (StructType(pre ++ Seq(mid) ++ schema.slice(slicePosition + 1, length)), original) + val (newType, droppedColumn) = dropColumnInChild(field.dataType, position.tail) + val mid = field.copy(dataType = newType) + + StructType(pre ++ Seq(mid) ++ post.tail) -> droppedColumn } else { if (length == 1) { throw new AnalysisException( "Cannot drop column from a struct type with a single field: " + schema) } - (StructType(pre ++ schema.slice(slicePosition + 1, length)), schema(slicePosition)) + StructType(pre ++ post.tail) -> field } } @@ -921,7 +964,7 @@ object SchemaUtils extends DeltaLogging { names.foreach { name => // ,;{}()\n\t= and space are special characters in Delta schema if (name.matches(".*[ ,;{}()\n\t=].*")) { - throw QueryCompilationErrors.columnNameContainsInvalidCharactersError(name) + throw QueryCompilationErrors.invalidColumnNameAsPathError("delta", name) } } // The method checkFieldNames doesn't have a valid regex to search for '\n'. That should be diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala index ceb22f1a93d..3745f402516 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala @@ -244,6 +244,7 @@ class DeltaDataSource sourceSnapshot: Snapshot, parameters: Map[String, String]): Option[DeltaSourceSchemaTrackingLog] = { val options = new CaseInsensitiveStringMap(parameters.asJava) + Option(options.get(DeltaOptions.SCHEMA_TRACKING_LOCATION)) .orElse(Option(options.get(DeltaOptions.SCHEMA_TRACKING_LOCATION_ALIAS))) .map { schemaTrackingLocation => @@ -255,7 +256,8 @@ class DeltaDataSource } DeltaSourceSchemaTrackingLog.create( spark, schemaTrackingLocation, sourceSnapshot, - Option(options.get(DeltaOptions.STREAMING_SOURCE_TRACKING_ID))) + Option(options.get(DeltaOptions.STREAMING_SOURCE_TRACKING_ID)) + ) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 3e4c9d445c7..7fdc74f7adf 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -1157,15 +1157,6 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val DELTA_ENABLE_BLOCKING_UPDATES_ON_DV_TABLES = - buildConf("deletionVectors.updates.blocking.enabled") - .internal() - .doc( - """Enable blocking updates on tables with Deletion Vectors - |Only change this for testing!""".stripMargin) - .booleanConf - .createWithDefault(true) - val DELTA_DUPLICATE_ACTION_CHECK_ENABLED = buildConf("duplicateActionCheck.enabled") .internal() diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala index 32677ee67a3..657bfca37c0 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala @@ -343,7 +343,9 @@ trait DeltaSourceBase extends Source // Initialize schema tracking log if possible, no-op if already initialized // This is one of the two places can initialize schema tracking. // This case specifically handles when we have a fresh stream. - initializeSchemaTrackingAndExitStreamIfNeeded(fromVersion) + if (readyToInitializeSchemaTrackingUponProvided) { + initializeSchemaTrackingAndExitStream(fromVersion) + } val changes = getFileChangesWithRateLimit( fromVersion, @@ -498,7 +500,8 @@ trait DeltaSourceBase extends Source !forceEnableStreamingReadOnReadIncompatibleSchemaChangesDuringStreamStart) { startVersionSnapshotOpt.foreach { snapshot => checkReadIncompatibleSchemaChanges( - snapshot.metadata, snapshot.version, validateAgainstStartSnapshot = true) + snapshot.metadata, snapshot.version, batchStartVersion, + validateAgainstStartSnapshot = true) } } @@ -521,7 +524,11 @@ trait DeltaSourceBase extends Source protected def checkReadIncompatibleSchemaChanges( metadata: Metadata, version: Long, + batchStartVersion: Long, + batchEndVersionOpt: Option[Long] = None, validateAgainstStartSnapshot: Boolean = false): Unit = { + log.info(s"checking read incompatibility with schema at version $version," + + s"inside batch[$batchStartVersion, ${batchEndVersionOpt.getOrElse("latest")}]") // Column mapping schema changes if (shouldVerifyColumnMappingSchemaChanges) { @@ -691,6 +698,7 @@ case class DeltaSource( createActionsIterator().processAndClose { actionsIter => validateCommitAndDecideSkipping( actionsIter, version, + fromVersion, endOffset.map(_.reservoirVersion), verifyMetadataAction && !trackingSchemaChange ) } @@ -866,6 +874,8 @@ case class DeltaSource( protected def validateCommitAndDecideSkipping( actions: Iterator[Action], version: Long, + batchStartVersion: Long, + batchEndVersionOpt: Option[Long] = None, verifyMetadataAction: Boolean = true): (Boolean, Option[Metadata]) = { /** A check on the source table that disallows changes on the source data. */ val shouldAllowChanges = options.ignoreChanges || ignoreFileDeletion || skipChangeCommits @@ -886,7 +896,7 @@ case class DeltaSource( } case m: Metadata => if (verifyMetadataAction) { - checkReadIncompatibleSchemaChanges(m, version) + checkReadIncompatibleSchemaChanges(m, version, batchStartVersion, batchEndVersionOpt) } assert(metadataAction.isEmpty, "Should not encounter two metadata actions in the same commit") @@ -961,7 +971,9 @@ case class DeltaSource( // contained such a schema change. // In either world, the initialization logic would find the superset compatible schema for this // batch by scanning Delta log. - initializeSchemaTrackingAndExitStreamIfNeeded(startVersion, Some(endOffset.reservoirVersion)) + if (readyToInitializeSchemaTrackingUponProvided) { + initializeSchemaTrackingAndExitStream(startVersion, Some(endOffset.reservoirVersion)) + } if (startOffsetOption.contains(endOffset)) { // This happens only if we recover from a failure and `MicroBatchExecution` tries to call diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala index f54457740c7..7ef02938793 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceCDCSupport.scala @@ -224,7 +224,9 @@ trait DeltaSourceCDCSupport { self: DeltaSource => // skipIndexedFile must be applied after creating IndexedFile so that // IndexedFile.index is consistent across all versions. val (fileActions, skipIndexedFile, metadataOpt) = - filterCDCActions(actions, version, verifyMetadataAction && !trackingSchemaChange) + filterCDCActions( + actions, version, fromVersion, endOffset.map(_.reservoirVersion), + verifyMetadataAction && !trackingSchemaChange) val itr = Iterator(IndexedFile(version, DeltaSourceOffset.BASE_INDEX, null)) ++ getSchemaChangeIndexedFileIterator(metadataOpt, version) ++ @@ -302,6 +304,8 @@ trait DeltaSourceCDCSupport { self: DeltaSource => private def filterCDCActions( actions: Seq[Action], version: Long, + batchStartVersion: Long, + batchEndVersionOpt: Option[Long] = None, verifyMetadataAction: Boolean = true): (Seq[FileAction], Boolean, Option[Metadata]) = { var shouldSkipIndexedFile = false var metadataAction: Option[Metadata] = None @@ -318,7 +322,7 @@ trait DeltaSourceCDCSupport { self: DeltaSource => false case m: Metadata => if (verifyMetadataAction) { - checkReadIncompatibleSchemaChanges(m, version) + checkReadIncompatibleSchemaChanges(m, version, batchStartVersion, batchEndVersionOpt) } assert(metadataAction.isEmpty, "Should not encounter two metadata actions in the same commit") diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaEvolutionSupport.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaEvolutionSupport.scala index 42e0ada2192..12ad1c574f8 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaEvolutionSupport.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaEvolutionSupport.scala @@ -97,9 +97,11 @@ trait DeltaSourceSchemaEvolutionSupport extends DeltaSourceBase { * This should only be used for the first write to the schema log, after then, schema tracking * should not rely on this state any more. */ - protected def readyToInitializeSchemaTracking: Boolean = + protected def readyToInitializeSchemaTrackingUponProvided: Boolean = !forceEnableStreamingReadOnColumnMappingSchemaChanges && - schemaTrackingLog.exists(_.getCurrentTrackedSchema.isEmpty) + schemaTrackingLog.exists { log => + log.getCurrentTrackedSchema.isEmpty && log.initSchemaLogEagerly + } /** * This is called from getFileChangesWithRateLimit() during latestOffset(). @@ -178,7 +180,7 @@ trait DeltaSourceSchemaEvolutionSupport extends DeltaSourceBase { */ private def resolveValidSchemaOfConstructedBatchForSchemaTrackingInitialization( startVersion: Long, endVersion: Long): (Long, Metadata) = { - assert(readyToInitializeSchemaTracking) + assert(readyToInitializeSchemaTrackingUponProvided) val schemaChanges = collectMetadataActions(startVersion, endVersion) // If no schema changes in between, just serve the start version val startSchemaMetadata = getSnapshotFromDeltaLog(startVersion).metadata @@ -249,26 +251,24 @@ trait DeltaSourceSchemaEvolutionSupport extends DeltaSourceBase { * end offset, we need to double verify to ensure no read-incompatible * within the batch range. */ - protected def initializeSchemaTrackingAndExitStreamIfNeeded( + protected def initializeSchemaTrackingAndExitStream( batchStartVersion: Long, batchEndVersionOpt: Option[Long] = None): Unit = { // If possible, initialize the schema log with the desired start schema instead of failing. // If a `batchEndVersion` is provided, we also need to verify if there are no incompatible // schema changes in a constructed batch, if so, we cannot find a proper schema to init the // schema log. - if (readyToInitializeSchemaTracking) { - val (version, metadata) = batchEndVersionOpt.map( - resolveValidSchemaOfConstructedBatchForSchemaTrackingInitialization(batchStartVersion, _)) - .getOrElse { - val startSnapshot = getSnapshotFromDeltaLog(batchStartVersion) - (startSnapshot.version, startSnapshot.metadata) - } - val schemaToUse = PersistedSchema(tableId, version, metadata.schema, metadata.partitionSchema) - // Always initialize the schema log - schemaTrackingLog.get.evolveSchema(schemaToUse) - if (hasSchemaChangeComparedToStreamSchema(metadata.schema)) { - // But trigger schema evolution exception when there's a difference - throw DeltaErrors.streamingSchemaEvolutionException(schemaToUse.dataSchema) + val (version, metadata) = batchEndVersionOpt.map( + resolveValidSchemaOfConstructedBatchForSchemaTrackingInitialization(batchStartVersion, _)) + .getOrElse { + val startSnapshot = getSnapshotFromDeltaLog(batchStartVersion) + (startSnapshot.version, startSnapshot.metadata) } + val schemaToUse = PersistedSchema(tableId, version, metadata.schema, metadata.partitionSchema) + // Always initialize the schema log + schemaTrackingLog.get.evolveSchema(schemaToUse) + if (hasSchemaChangeComparedToStreamSchema(metadata.schema)) { + // But trigger schema evolution exception when there's a difference + throw DeltaErrors.streamingSchemaEvolutionException(schemaToUse.dataSchema) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaTrackingLog.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaTrackingLog.scala index 212427503b1..7bfb23c4c6c 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaTrackingLog.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSourceSchemaTrackingLog.scala @@ -103,11 +103,14 @@ object PersistedSchema { * * @param rootSchemaLocation Schema log location * @param sourceSnapshot Delta source snapshot for the Delta streaming source + * @param initSchemaLogEagerly If true, initialize schema log as early as possible, otherwise, + * initialize only when detecting non-additive schema change. */ class DeltaSourceSchemaTrackingLog private( sparkSession: SparkSession, rootSchemaLocation: String, - sourceSnapshot: Snapshot) + sourceSnapshot: Snapshot, + val initSchemaLogEagerly: Boolean = true) extends HDFSMetadataLog[PersistedSchema](sparkSession, rootSchemaLocation) { import PersistedSchema._ @@ -201,9 +204,15 @@ object DeltaSourceSchemaTrackingLog { sparkSession: SparkSession, rootSchemaLocation: String, sourceSnapshot: Snapshot, - sourceTrackingId: Option[String] = None): DeltaSourceSchemaTrackingLog = { + sourceTrackingId: Option[String] = None, + initSchemaLogEagerly: Boolean = true): DeltaSourceSchemaTrackingLog = { val schemaTrackingLocation = fullSchemaTrackingLocation( rootSchemaLocation, sourceSnapshot.deltaLog.tableId, sourceTrackingId) - new DeltaSourceSchemaTrackingLog(sparkSession, schemaTrackingLocation, sourceSnapshot) + new DeltaSourceSchemaTrackingLog( + sparkSession, + schemaTrackingLocation, + sourceSnapshot, + initSchemaLogEagerly + ) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala b/core/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala index c81280365ae..539ae2d893f 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala @@ -243,6 +243,8 @@ trait DataSkippingReaderBase } } + def withStatsDeduplicated: DataFrame = withStats + /** * Builds the data filters for data skipping. */ diff --git a/core/src/main/scala/org/apache/spark/sql/delta/stats/StatisticsCollection.scala b/core/src/main/scala/org/apache/spark/sql/delta/stats/StatisticsCollection.scala index 6d53a1d6690..3aaae025bac 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/stats/StatisticsCollection.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/stats/StatisticsCollection.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.delta.stats // scalastyle:off import.ordering.noEmptyLine import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.delta.{CheckpointV2, DeletionVectorsTableFeature, DeltaColumnMapping, DeltaLog, DeltaUDF} +import org.apache.spark.sql.delta.{CheckpointV2, DeletionVectorsTableFeature, DeltaColumnMapping, DeltaErrors, DeltaLog, DeltaUDF} import org.apache.spark.sql.delta.DeltaOperations.ComputeStats import org.apache.spark.sql.delta.actions.{AddFile, Protocol} +import org.apache.spark.sql.delta.commands.DeletionVectorUtils import org.apache.spark.sql.delta.commands.DeltaCommand import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaMergingUtils @@ -150,8 +151,7 @@ trait StatisticsCollection extends DeltaLogging { * 2) "no-nulls" columns remain unchanged, i.e. zero nullCount is the same for both * physical and logical representations. * 3) For "some-nulls" columns, we leave the existing value. In files with wide bounds, - * the nullCount in SOME_NULLs columns is considered unknown and it is not taken - * into account by data skipping and OptimizeMetadataOnlyDeltaQuery. + * the nullCount in SOME_NULLs columns is considered unknown. * * The file's state can transition back to tight when statistics are recomputed. In that case, * TIGHT_BOUNDS is set back to true and nullCount back to the logical value. @@ -409,10 +409,17 @@ object StatisticsCollection extends DeltaCommand { // Save the current AddFiles that match the predicates so we can update their stats val files = txn.filterFiles(predicates).filter(fileFilter) val pathToAddFileMap = generateCandidateFileMap(deltaLog.dataPath, files) + val persistentDVsReadable = DeletionVectorUtils.deletionVectorsReadable(txn.snapshot) // Use the stats collector to recompute stats val dataPath = deltaLog.dataPath - val newAddFiles = + val newAddFiles = { + // Throw error when the table contains DVs, because existing method of stats + // recomputation doesn't work on tables with DVs. It needs to take into consideration of + // DV files (TODO). + if (persistentDVsReadable) { + throw DeltaErrors.statsRecomputeNotSupportedOnDvTables() + } { val newStats = deltaLog.createDataFrame(txn.snapshot, addFiles = files, isStreaming = false) .groupBy(col("_metadata.file_path").as("path")).agg(to_json(txn.statsCollector)) @@ -423,8 +430,9 @@ object StatisticsCollection extends DeltaCommand { add.copy(dataChange = false, stats = r.getString(1)) } } + } - txn.commit(newAddFiles, ComputeStats(predicates.map(_.sql))) + txn.commit(newAddFiles, ComputeStats(predicates)) } /** diff --git a/core/src/main/scala/org/apache/spark/sql/delta/stats/StatsCollectionUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/stats/StatsCollectionUtils.scala new file mode 100644 index 00000000000..9e4c20a2564 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/delta/stats/StatsCollectionUtils.scala @@ -0,0 +1,635 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.stats + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.language.existentials +import scala.util.control.NonFatal + +// scalastyle:off import.ordering.noEmptyLine +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, DeltaErrors, IdMapping, NameMapping, NoMapping} +import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaStatistics._ +import org.apache.spark.sql.delta.util.{DeltaFileOperations, JsonUtils} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.metadata.{BlockMetaData, ParquetMetadata} +import org.apache.parquet.io.api.Binary +import org.apache.parquet.schema.LogicalTypeAnnotation.{DateLogicalTypeAnnotation, StringLogicalTypeAnnotation} +import org.apache.parquet.schema.PrimitiveType + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types.{ArrayType, LongType, MapType, StructField, StructType} +import org.apache.spark.util.SerializableConfiguration + + +object StatsCollectionUtils + extends Logging +{ + + /** A helper function to compute stats of addFiles using StatsCollector. + * + * @param spark The SparkSession used to process data. + * @param conf The Hadoop configuration used to access file system. + * @param dataPath The data path of table, to which these AddFile(s) belong. + * @param addFiles The list of target AddFile(s) to be processed. + * @param columnMappingMode The column mapping mode of table. + * @param dataSchema The data schema of table. + * @param statsSchema The stats schema to be collected. + * @param ignoreMissingStats Whether to ignore missing stats during computation. + * @param setBoundsToWide Whether to set bounds to wide independently of whether or not + * the files have DVs. + * + * @return A list of AddFile(s) with newly computed stats, please note the existing stats from + * the input addFiles will be ignored regardless. + */ + def computeStats( + spark: SparkSession, + conf: Configuration, + dataPath: Path, + addFiles: Dataset[AddFile], + columnMappingMode: DeltaColumnMappingMode, + dataSchema: StructType, + statsSchema: StructType, + ignoreMissingStats: Boolean = true, + setBoundsToWide: Boolean = false): Dataset[AddFile] = { + + import org.apache.spark.sql.delta.implicits._ + + val stringTruncateLength = + spark.sessionState.conf.getConf(DeltaSQLConf.DATA_SKIPPING_STRING_PREFIX_LENGTH) + + val statsCollector = StatsCollector(columnMappingMode, dataSchema, statsSchema, + ignoreMissingStats, Some(stringTruncateLength)) + + val serializableConf = new SerializableConfiguration(conf) + val broadcastConf = spark.sparkContext.broadcast(serializableConf) + + val dataRootDir = dataPath.toString + addFiles.mapPartitions { addFileIter => + val defaultFileSystem = new Path(dataRootDir).getFileSystem(broadcastConf.value.value) + addFileIter.map { addFile => + val path = DeltaFileOperations.absolutePath(dataRootDir, addFile.path) + val fileStatus = if (path.toString.startsWith(dataRootDir)) { + defaultFileSystem.getFileStatus(path) + } else { + path.getFileSystem(broadcastConf.value.value).getFileStatus(path) + } + + val (stats, metric) = statsCollector.collect( + ParquetFileReader.readFooter(broadcastConf.value.value, fileStatus)) + + if (metric.totalMissingFields > 0 || metric.numMissingTypes > 0) { + logWarning( + s"StatsCollection of file `$path` misses fields/types: ${JsonUtils.toJson(metric)}") + } + + val statsWithTightBoundsCol = { + val hasDeletionVector = + addFile.deletionVector != null && !addFile.deletionVector.isEmpty + stats + (TIGHT_BOUNDS -> !(setBoundsToWide || hasDeletionVector)) + } + + addFile.copy(stats = JsonUtils.toJson(statsWithTightBoundsCol)) + } + } + } +} + +/** + * A helper class to collect stats of parquet data files for Delta table and its equivalent (tables + * that can be converted into Delta table like Parquet/Iceberg table). + * + * @param dataSchema The data schema from table metadata, which is the logical schema with logical + * to physical mapping per schema field. It is used to map statsSchema to parquet + * metadata. + * @param statsSchema The schema of stats to be collected, statsSchema should follow the physical + * schema and must be generated by StatisticsCollection. + * @param ignoreMissingStats Indicate whether to return partial result by ignoring missing stats + * or throw an exception. + * @param stringTruncateLength The optional max length of string stats to be truncated into. + * + * Scala Example: + * {{{ + * import org.apache.spark.sql.delta.stats.StatsCollector + * + * val stringTruncateLength = + * spark.sessionState.conf.getConf(DeltaSQLConf.DATA_SKIPPING_STRING_PREFIX_LENGTH) + * + * val statsCollector = StatsCollector( + * snapshot.metadata.columnMappingMode, snapshot.metadata.dataSchema, snapshot.statsSchema, + * ignoreMissingStats = false, Some(stringTruncateLength)) + * + * val filesWithStats = snapshot.allFiles.map { file => + * val path = DeltaFileOperations.absolutePath(dataPath, file.path) + * val fileSystem = path.getFileSystem(hadoopConf) + * val fileStatus = fileSystem.listStatus(path).head + * + * val footer = ParquetFileReader.readFooter(hadoopConf, fileStatus) + * val (stats, _) = statsCollector.collect(footer) + * file.copy(stats = JsonUtils.toJson(stats)) + * } + * }}} + */ +abstract class StatsCollector( + dataSchema: StructType, + statsSchema: StructType, + ignoreMissingStats: Boolean, + stringTruncateLength: Option[Int]) + extends Serializable +{ + + final val NUM_MISSING_TYPES = "numMissingTypes" + + /** + * Used to report number of missing fields per supported type and number of missing unsupported + * types in the collected statistics, currently the statistics collection supports 4 types of + * stats: NUM_RECORDS, MAX, MIN, NULL_COUNT. + * + * @param numMissingMax The number of missing fields for MAX + * @param numMissingMin The number of missing fields for MIN + * @param numMissingNullCount The number of missing fields for NULL_COUNT + * @param numMissingTypes The number of unsupported type being requested. + */ + case class StatsCollectionMetrics( + numMissingMax: Long, + numMissingMin: Long, + numMissingNullCount: Long, + numMissingTypes: Long) { + + val totalMissingFields: Long = Seq(numMissingMax, numMissingMin, numMissingNullCount).sum + } + + object StatsCollectionMetrics { + def apply(missingFieldCounts: Map[String, Long]): StatsCollectionMetrics = { + StatsCollectionMetrics( + missingFieldCounts.getOrElse(MAX, 0L), + missingFieldCounts.getOrElse(MIN, 0L), + missingFieldCounts.getOrElse(NULL_COUNT, 0L), + missingFieldCounts.getOrElse(NUM_MISSING_TYPES, 0L)) + } + } + + /** + * A list of schema physical path and corresponding struct field of leaf fields. Beside primitive + * types, Map and Array (instead of their sub-columns) are also treated as leaf fields since we + * only compute null count of them, and null is counted based on themselves instead of sub-fields. + */ + protected lazy val schemaPhysicalPathAndSchemaField: Seq[(Seq[String], StructField)] = { + def explode(schema: StructType): Seq[(Seq[String], StructField)] = { + schema.flatMap { field => + val physicalName = DeltaColumnMapping.getPhysicalName(field) + field.dataType match { + case s: StructType => + explode(s).map { case (path, field) => (Seq(physicalName) ++ path, field) } + case _ => (Seq(physicalName), field) :: Nil + } + } + } + explode(dataSchema) + } + + /** + * Returns the map from schema physical field path (field for which to collect stats) to the + * parquet metadata column index (where to collect stats). statsSchema generated by + * StatisticsCollection always use physical field paths so physical field paths are the same as + * to the ones used in statsSchema. Child class must implement this method based on delta column + * mapping mode. + */ + def getSchemaPhysicalPathToParquetIndex(blockMetaData: BlockMetaData): Map[Seq[String], Int] + + /** + * Collects the stats from [[ParquetMetadata]] + * + * @param parquetMetadata The metadata of parquet file following physical schema, it contains + * statistics of row groups. + * + * @return A nested Map[String: Any] from requested stats field names to their stats field value + * and [[StatsCollectionMetrics]] counting the number of missing fields/types. + */ + final def collect( + parquetMetadata: ParquetMetadata): (Map[String, Any], StatsCollectionMetrics) = { + val blocks = parquetMetadata.getBlocks.asScala.toSeq + if (blocks.isEmpty) { + return (Map(NUM_RECORDS -> 0L), StatsCollectionMetrics(Map.empty[String, Long])) + } + + val schemaPhysicalPathToParquetIndex = getSchemaPhysicalPathToParquetIndex(blocks.head) + val missingFieldCounts = + mutable.Map(MAX -> 0L, MIN -> 0L, NULL_COUNT -> 0L, NUM_MISSING_TYPES -> 0L) + + // Collect the actual stats. + // + // The result of this operation is a tree of maps that matches the structure of the stats + // schema. The stats schema is split by stats type at the top, and each type matches the + // structure of the data schema (can be subset), so we collect per stats type. E.g. the MIN + // values are under MIN.a, MIN.b.c, MIN.b.d etc., and then the MAX values are under MAX.a, + // MAX.b.c etc. Note, we do omit here the tightBounds column and add it at a later stage. + val collectedStats = statsSchema.filter(_.name != TIGHT_BOUNDS).map { + case StructField(NUM_RECORDS, LongType, _, _) => + val numRecords = blocks.map { block => + block.getRowCount + }.sum + NUM_RECORDS -> numRecords + case StructField(MIN, statsTypeSchema: StructType, _, _) => + val (minValues, numMissingFields) = + collectStats(Seq.empty[String], statsTypeSchema, blocks, schemaPhysicalPathToParquetIndex, + ignoreMissingStats)(aggMaxOrMin(isMax = false)) + missingFieldCounts(MIN) += numMissingFields + MIN -> minValues + case StructField(MAX, statsTypeSchema: StructType, _, _) => + val (maxValues, numMissingFields) = + collectStats(Seq.empty[String], statsTypeSchema, blocks, schemaPhysicalPathToParquetIndex, + ignoreMissingStats)(aggMaxOrMin(isMax = true)) + missingFieldCounts(MAX) += numMissingFields + MAX -> maxValues + case StructField(NULL_COUNT, statsTypeSchema: StructType, _, _) => + val (nullCounts, numMissingFields) = + collectStats(Seq.empty[String], statsTypeSchema, blocks, schemaPhysicalPathToParquetIndex, + ignoreMissingStats)(aggNullCount) + missingFieldCounts(NULL_COUNT) += numMissingFields + NULL_COUNT -> nullCounts + case field: StructField => + if (ignoreMissingStats) { + missingFieldCounts(NUM_MISSING_TYPES) += 1 + field.name -> Map.empty[String, Any] + } else { + throw new UnsupportedOperationException(s"stats type not supported: ${field.name}") + } + }.toMap + + (collectedStats, StatsCollectionMetrics(missingFieldCounts.toMap)) + } + + /** + * Collects statistics by recurring through the structure of statsSchema and tracks the fields + * that we have seen so far in parentPhysicalPath. + * + * @param parentPhysicalFieldPath The absolute path of parent field with physical names. + * @param statsSchema The schema with physical names to collect stats recursively. + * @param blocks The metadata of Parquet row groups, which contains the raw stats. + * @param schemaPhysicalPathToParquetIndex Map from schema path to parquet metadata column index. + * @param ignoreMissingStats Whether to ignore and log missing fields or throw an exception. + * @param aggFunc The aggregation function used to aggregate stats across row. + * + * @return A nested Map[String: Any] from schema field name to stats value and a count of missing + * fields. + * + * Here is an example of stats: + * + * stats schema: + * | -- id: INT + * | -- person: STRUCT + * | name: STRUCT + * | -- first: STRING + * | -- last: STRING + * | height: LONG + * + * The stats: + * Map( + * "id" -> 1003, + * "person" -> Map( + * "name" -> Map( + * "first" -> "Chris", + * "last" -> "Green" + * ), + * "height" -> 175L + * ) + * ) + */ + private def collectStats( + parentPhysicalFieldPath: Seq[String], + statsSchema: StructType, + blocks: Seq[BlockMetaData], + schemaPhysicalPathToParquetIndex: Map[Seq[String], Int], + ignoreMissingStats: Boolean)( + aggFunc: (Seq[BlockMetaData], Int) => Any): (Map[String, Any], Long) = { + val stats = mutable.Map.empty[String, Any] + var numMissingFields = 0L + statsSchema.foreach { + case StructField(name, dataType: StructType, _, _) => + val (map, numMissingFieldsInSubtree) = + collectStats(parentPhysicalFieldPath :+ name, dataType, blocks, + schemaPhysicalPathToParquetIndex, ignoreMissingStats)(aggFunc) + numMissingFields += numMissingFieldsInSubtree + if (map.nonEmpty) { + stats += name -> map + } + case StructField(name, _, _, _) => + val physicalFieldPath = parentPhysicalFieldPath :+ name + if (schemaPhysicalPathToParquetIndex.contains(physicalFieldPath)) { + try { + val value = aggFunc(blocks, schemaPhysicalPathToParquetIndex(physicalFieldPath)) + // None value means the stats is undefined for this field (e.g., max/min of a field, + // whose values are nulls in all blocks), we use null to be consistent with stats + // generated from SQL. + if (value != None) { + stats += name -> value + } else { + stats += name -> null + } + } catch { + case NonFatal(_) if ignoreMissingStats => numMissingFields += 1L + case exception: Throwable => throw exception + } + } else if (ignoreMissingStats) { + // Physical field path requested by stats is missing in the mapping, so it's missing from + // the parquet metadata. + numMissingFields += 1L + } else { + val columnPath = physicalFieldPath.mkString("[", ", ", "]") + throw DeltaErrors.deltaStatsCollectionColumnNotFound("all", columnPath) + } + } + + (stats.toMap, numMissingFields) + } + + /** The aggregation function used to collect max and min */ + private def aggMaxOrMin(isMax: Boolean)(blocks: Seq[BlockMetaData], index: Int): Any = { + val columnMetadata = blocks.head.getColumns.get(index) + val primitiveType = columnMetadata.getPrimitiveType + // Physical type of timestamp is INT96 in both Parquet and Delta. + if (primitiveType.getPrimitiveTypeName == PrimitiveType.PrimitiveTypeName.INT96) { + throw new UnsupportedOperationException( + s"max/min stats is not supported for INT96 timestamp: ${columnMetadata.getPath}") + } + + var aggregatedValue: Any = None + blocks.foreach { block => + val column = block.getColumns.get(index) + val statistics = column.getStatistics + // Skip this block if the column has null for all rows, stats is defined as long as it exists + // in even a single block. + if (statistics.hasNonNullValue) { + val currentValue = if (isMax) statistics.genericGetMax else statistics.genericGetMin + if (currentValue == null) { + throw DeltaErrors.deltaStatsCollectionColumnNotFound("max/min", column.getPath.toString) + } + + if (aggregatedValue == None) { + aggregatedValue = currentValue + } else { + // TODO: check NaN value for floating point columns. + val compareResult = currentValue.asInstanceOf[Comparable[Any]].compareTo(aggregatedValue) + if ((isMax && compareResult > 0) || (!isMax && compareResult < 0)) { + aggregatedValue = currentValue + } + } + } + } + + val logicalType = primitiveType.getLogicalTypeAnnotation + aggregatedValue match { + case bytes: Binary if logicalType.isInstanceOf[StringLogicalTypeAnnotation] => + val rawString = bytes.toStringUsingUTF8 + if (stringTruncateLength.isDefined && rawString.length > stringTruncateLength.get) { + if (isMax) { + // Append tie breakers to assure that any value in this column is less than or equal to + // the max, check the helper function for more details. + StatisticsCollection.truncateMaxStringAgg(stringTruncateLength.get)(rawString) + } else { + rawString.substring(0, stringTruncateLength.get) + } + } else { + rawString + } + case _: Binary => + throw new UnsupportedOperationException( + s"max/min stats is not supported for binary other than string: ${columnMetadata.getPath}") + case date: Integer if logicalType.isInstanceOf[DateLogicalTypeAnnotation] => + DateTimeUtils.toJavaDate(date).toString + case other => other + } + } + + /** The aggregation function used to count null */ + private def aggNullCount(blocks: Seq[BlockMetaData], index: Int): Any = { + var count = 0L + blocks.foreach { block => + val column = block.getColumns.get(index) + val statistics = column.getStatistics + if (!statistics.isNumNullsSet) { + throw DeltaErrors.deltaStatsCollectionColumnNotFound("nullCount", column.getPath.toString) + } + count += statistics.getNumNulls + } + count.asInstanceOf[Any] + } +} + +object StatsCollector { + def apply( + columnMappingMode: DeltaColumnMappingMode, + dataSchema: StructType, + statsSchema: StructType, + ignoreMissingStats: Boolean = true, + stringTruncateLength: Option[Int] = None): StatsCollector = { + columnMappingMode match { + case NoMapping | NameMapping => + StatsCollectorNameMapping( + dataSchema, statsSchema, ignoreMissingStats, stringTruncateLength) + case IdMapping => + StatsCollectorIdMapping( + dataSchema, statsSchema, ignoreMissingStats, stringTruncateLength) + case _ => + throw new UnsupportedOperationException( + s"$columnMappingMode mapping is currently not supported") + } + } + + private case class StatsCollectorNameMapping( + dataSchema: StructType, + statsSchema: StructType, + ignoreMissingStats: Boolean, + stringTruncateLength: Option[Int]) + extends StatsCollector(dataSchema, statsSchema, ignoreMissingStats, stringTruncateLength) { + + /** + * Maps schema physical field path to parquet metadata column index via parquet metadata column + * path in NoMapping and NameMapping modes + */ + override def getSchemaPhysicalPathToParquetIndex( + blockMetaData: BlockMetaData): Map[Seq[String], Int] = { + val parquetColumnPathToIndex = getParquetColumnPathToIndex(blockMetaData) + columnPathSchemaToParquet.collect { + // Collect mapping of fields in physical schema that actually exist in parquet metadata, + // parquet metadata can miss field due to schema evolution. In case stats collection is + // requested on a column that is missing from parquet metadata, we will catch this in + // collectStats when looking up in this map. + case (schemaPath, parquetPath) if parquetColumnPathToIndex.contains(parquetPath) => + schemaPath -> parquetColumnPathToIndex(parquetPath) + } + } + + /** + * A map from schema field path (with physical names) to parquet metadata column path of schema + * leaf fields with special handling of Array and Map. + * + * Here is an example: + * + * Data Schema (physical name in the parenthesis) + * | -- id (a4def3): INT + * | -- history (23aa42): STRUCT + * | -- cost (23ddb0): DOUBLE + * | -- events (23dda1): ARRAY[STRING] + * | -- info (abb4d2): MAP[STRING, STRING] + * + * Block Metadata: + * Columns: [ [a4def3], [23aa42, 23ddb0], [23ddb0, 23dda1, list, element], + * [abb4d2, key_value, key], [abb4d2, key_value, value] ] + * + * The mapping: + * [a4def3] -> [a4def3] + * [23aa42, 23ddb0] -> [23aa42, 23ddb0] + * [23ddb0, 23dda1] -> [23ddb0, 23dda1, list, element] + * [abb4d2] -> [abb4d2, key_value, key] + */ + private lazy val columnPathSchemaToParquet: Map[Seq[String], Seq[String]] = { + // Parquet metadata column path contains addition keywords for Array and Map. Here we only + // support 2 cases below since stats is not available in the other cases: + // 1. Array with non-null elements of primitive types + // 2. Map with key of primitive types + schemaPhysicalPathAndSchemaField.map { + case(path, field) => + field.dataType match { + // Here we don't check array element type and map key type for primitive type since + // parquet metadata column path always points to a primitive column. In other words, + // the type is primitive if the column path can be found in parquet metadata later. + case ArrayType(_, false) => path -> (path ++ Seq("list", "element")) + case MapType(_, _, _) => path -> (path ++ Seq("key_value", "key")) + case _ => path -> path + } + }.toMap + } + + /** + * Returns a map from parquet metadata column path to index. + * + * Here is an example: + * + * Data Schema: + * |-- id : INT + * |-- person : STRUCT + * |-- name: STRING + * |-- phone: INT + * |-- eligible: BOOLEAN + * + * Block Metadata: + * Columns: [ [id], [person, name], [person, phone], [eligible] ] + * + * The mapping: + * [id] -> 0 + * [person, name] -> 1 + * [person, phone] -> 2 + * [eligible] -> 3 + */ + private def getParquetColumnPathToIndex(block: BlockMetaData): Map[Seq[String], Int] = { + block.getColumns.asScala.zipWithIndex.map { + case (column, i) => column.getPath.toArray.toSeq -> i + }.toMap + } + } + + private case class StatsCollectorIdMapping( + dataSchema: StructType, + statsSchema: StructType, + ignoreMissingStats: Boolean, + stringTruncateLength: Option[Int]) + extends StatsCollector(dataSchema, statsSchema, ignoreMissingStats, stringTruncateLength) { + + // Define a FieldId type to better disambiguate between ids and indices in the code + type FieldId = Int + + /** + * Maps schema physical field path to parquet metadata column index via parquet metadata column + * id in IdMapping mode. + */ + override def getSchemaPhysicalPathToParquetIndex( + blockMetaData: BlockMetaData): Map[Seq[String], Int] = { + val parquetColumnIdToIndex = getParquetColumnIdToIndex(blockMetaData) + schemaPhysicalPathToColumnId.collect { + // Collect mapping of fields in physical schema that actually exist in parquet metadata, + // parquet metadata can miss field due to schema evolution and non-primitive types like Map + // and Array. In case stats collection is requested on a column that is missing from + // parquet metadata, we will catch this in collectStats when looking up in this map. + case (schemaPath, columnId) if parquetColumnIdToIndex.contains(columnId) => + schemaPath -> parquetColumnIdToIndex(columnId) + } + } + + /** + * A map from schema field path (with physical names) to parquet metadata column id of schema + * leaf fields. + * + * Here is an example: + * + * Data Schema (physical name, id in the parenthesis) + * | -- id (a4def3, 1): INT + * | -- history (23aa42, 2): STRUCT + * | -- cost (23ddb0, 3): DOUBLE + * | -- events (23dda1, 4): ARRAY[STRING] + * | -- info (abb4d2, 5): MAP[STRING, STRING] + * + * The mapping: + * [a4def3] -> 1 + * [23aa42, 23ddb0] -> 3 + * [23ddb0, 23dda1] -> 4 + * [abb4d2] -> 5 + */ + private lazy val schemaPhysicalPathToColumnId: Map[Seq[String], FieldId] = { + schemaPhysicalPathAndSchemaField.map { + case (path, field) => path -> DeltaColumnMapping.getColumnId(field) + }.toMap + } + + /** + * Returns a map from parquet metadata column id to column index by skipping columns without id. + * E.g., subfields of ARRAY and MAP don't have id assigned. + * + * Here is an example: + * + * Data Schema (id in the parenthesis): + * |-- id (1) : INT + * |-- person (2) : STRUCT + * |-- names (3) : ARRAY[STRING] + * |-- phones (4) : MAP[STRING, INT] + * |-- eligible (5) : BOOLEAN + * + * Block Metadata (id in the parenthesis): + * Columns: [ [id](1), [person, names, list, element](null), + * [person, phones, key_value, key](null), [person, phones, key_value, value](null), + * [eligible](5) ] + * + * The mapping: 1 -> 0, 5 -> 4 + */ + private def getParquetColumnIdToIndex(block: BlockMetaData): Map[FieldId, Int] = { + block.getColumns.asScala.zipWithIndex.collect { + // Id of parquet metadata column is not guaranteed, subfields of Map and Array don't have + // id assigned. In case id is missing and null, we skip the parquet metadata column here + // and will catch this in collectStats when looking up in this map. + case (column, i) if column.getPrimitiveType.getId != null => + column.getPrimitiveType.getId.intValue() -> i + }.toMap + } + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaEncoders.scala b/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaEncoders.scala index 76721250512..959f4215feb 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaEncoders.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaEncoders.scala @@ -83,6 +83,10 @@ private[delta] trait DeltaEncoders { private lazy val _addFileWithIndexEncoder = new DeltaEncoder[(AddFile, Long)] implicit def addFileWithIndexEncoder: Encoder[(AddFile, Long)] = _addFileWithIndexEncoder.get + private lazy val _addFileWithSourcePathEncoder = new DeltaEncoder[(AddFile, String)] + implicit def addFileWithSourcePathEncoder: Encoder[(AddFile, String)] = + _addFileWithSourcePathEncoder.get + private lazy val _deltaHistoryEncoder = new DeltaEncoder[DeltaHistory] implicit def deltaHistoryEncoder: Encoder[DeltaHistory] = _deltaHistoryEncoder.get diff --git a/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala b/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala index e23e7472c12..fc0fe15adcf 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/util/DeltaFileOperations.scala @@ -450,7 +450,12 @@ object DeltaFileOperations extends DeltaLogging { files.mapPartitions { fileList => fileList.map { addFile => val fileSource = DeltaFileOperations.absolutePath(qualifiedTablePath, addFile.path) + if (addFile.deletionVector != null) { + val absoluteDV = addFile.deletionVector.copyWithAbsolutePath(new Path(qualifiedTablePath)) + addFile.copy(path = fileSource.toUri.toString, deletionVector = absoluteDV) + } else { addFile.copy(path = fileSource.toUri.toString) + } } } } diff --git a/core/src/main/scala/org/apache/spark/sql/util/ScalaExtensions.scala b/core/src/main/scala/org/apache/spark/sql/util/ScalaExtensions.scala index 6ed02caf0ae..5223a8a50bc 100644 --- a/core/src/main/scala/org/apache/spark/sql/util/ScalaExtensions.scala +++ b/core/src/main/scala/org/apache/spark/sql/util/ScalaExtensions.scala @@ -35,6 +35,12 @@ object ScalaExtensions { */ def when[A](cond: Boolean)(a: => A): Option[A] = if (cond) Some(a) else None + /** + * When a given condition is false, evaluates the a argument and returns Some(a). + * When the condition is true, a is not evaluated and None is returned. + */ + def whenNot[A](cond: Boolean)(a: => A): Option[A] = if (!cond) Some(a) else None + /** Sum up all the `options`, substituting `default` for each `None`. */ def sum[N : Numeric](default: N)(options: Option[N]*): N = options.map(_.getOrElse(default)).sum diff --git a/core/src/test/resources/delta/table-with-dv-feature-enabled/_delta_log/00000000000000000000.json b/core/src/test/resources/delta/table-with-dv-feature-enabled/_delta_log/00000000000000000000.json deleted file mode 100644 index 5ac362dd2cd..00000000000 --- a/core/src/test/resources/delta/table-with-dv-feature-enabled/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,4 +0,0 @@ -{"commitInfo":{"timestamp":1678403656146,"operation":"CREATE TABLE AS SELECT","operationParameters":{"isManaged":"false","description":null,"partitionBy":"[]","properties":"{}"},"isolationLevel":"WriteSerializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"1","numOutputBytes":"609"},"engineInfo":"","txnId":"9ce289f2-84a0-4cd6-918f-139376c1d897"}} -{"protocol":{"minReaderVersion":3,"minWriterVersion":7,"readerFeatures":["deletionVectors"],"writerFeatures":["deletionVectors"]}} -{"metaData":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"x\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1678403653515}} -{"add":{"path":"part-00000-fe8a3447-d858-4d61-ba30-9549910bc73d-c000.snappy.parquet","partitionValues":{},"size":609,"modificationTime":1678403655890,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"x\":1},\"maxValues\":{\"x\":1},\"nullCount\":{\"x\":0},\"tightBounds\":true}","tags":{"INSERTION_TIME":"1678403655890000","MIN_INSERTION_TIME":"1678403655890000","MAX_INSERTION_TIME":"1678403655890000","OPTIMIZE_TARGET_SIZE":"268435456"}}} diff --git a/core/src/test/resources/delta/table-with-dv-feature-enabled/part-00000-fe8a3447-d858-4d61-ba30-9549910bc73d-c000.snappy.parquet b/core/src/test/resources/delta/table-with-dv-feature-enabled/part-00000-fe8a3447-d858-4d61-ba30-9549910bc73d-c000.snappy.parquet deleted file mode 100644 index 71f64fa364151522c3dbc3a3fa3e4c75d9aaa5a0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 609 zcmZWn+iKfD5FKwW2vQ2A*<}TS3PDW*_EwVZBr>I7=tEnI2_@}A+7eds*b&ji(aLFp z@n7^?@(p?H7nP1%H-7LwEOXA>&dl!V;8VbXJKW{JAAa5MIh+yP-DG>w@vYIG8)uciF zm80r3$Mdp4`FxOLl_vVt$I@I3iV0?DKsp-<8BTO^_kMU59Z3KwHv(?3GHF@JosQ}$ zALQXigt^WX?c#cdwW`V=D#u!nbgkv*xmVv!(d&Dru4YN?wY+hP*~obPuXC@^`O?$fNiwU7(mSsB5$Vpk)^8VNJ`t!x&`vT;P4GI6Ic&lqXqXjGnIbSJi=ast}!3|Up zs!n^UEvDUEnYQoyp?eU|(ro00-IK14dZQSFBTakg*Kqkg~{F|Ypuii{|K;Zlyq z{{{U2zntmq{{Qd)0s;T&e*yUe0x>XyFermDI72WbLoqbNFs#Da;TeGu8Hte@g;5!e z(HVm=8B4M4IE>49jL!s2$V5!cBuvU=OwJTcsZ@4qreRv9V|r#_MrLAWW?@!lQ+7KC zb21lmGY|7JAM^Va@GHndEX*P-s$zC=mS9PiVriCPS(amYR$xU|Qf0detFjuavj%Ij z7HhK(>#`o}vjH2bk=>Y0*p$uKoGsXrt=O7v*p}_o-tNGT?8MIO!mjMb?(D&y?8V;f z!@lZg_vZi(V|*rHLMCEj zC9#t-8Iv;wQ!*7(GY!)+9n&*|GTND#nOT^X*_fRa+6F6*&A8?Yf8u`!#lDVwo5 zTd1Yoimlm(ZP||P*?}F|iJjSnUD=J@)x++|UhK_2?8|=a&jB3BK^)8>8fp*YaE{9y zYq(bH?DgEhjoie|+`_Hg#_im}o!rIU+@rnrKJMoM9^@e&<`Ev{F&^g$p42J(G|%uX z&+$Aj@FFkqGOzF|ukku>=%#&(w|R$md5`z`fDieIkNJd8^~`?G7ktTAe9bp}%XfUw z5B$ha{LC-vnSjLkTVt9W*NCSXD)Vqzv?QYK?^reI2@QffO5(=r{?GXpa+ z6EialvoagAGY4}jmz|q=n3wsOp9NTug;!@9hu#$WQ#tFZ{}H{LUZz$zS}fKlWe# zV?dw+0sjTcAMig0VqgYgPzGaghEPa56hku%!!jJhGXf(r5+gGTqcR$!D~27Du^5|i z7?<%Fp9z?diI|v4l+;ef4|WH06t zF6A;V=L)XmDz4@luH`ze=LT)GH*qt!a4WZQJ9ls=cX2oOaIf~+`+0x|d5DL3ghzRd z$9aM$d5WibMrZAFJkJZf$V;f*n}7J1{}>S1{Rejc45Ywz5C&y124@I{ zWGIGa7=~pyhGzsuR3tkxqcAF?F*;)~CSx%+<1jAcDZZV637LqAnS@E1jLDgTDVd6? znTBbV&Q8w^%*ag4%q+~xY|PFa%*kBJZRcTL=3{;qU_lmQVHROg7GrUiU`ds-OS25i zvK-5^0xPl-E3*o#vYM*fHCU6iSetcNm-Sem4cL&4*qBY&RL$(>Y{8an#nx=Ywrt1t z?7)ufq|SC1c4aqqXAkydFZO01_GLfz*8qDU2XQcma43gyI7e_KM{zXAa4g4Zygh*v zIf;`wg;P0=(>a4PIg7J3$DYf1oX-VZ$VFVtC0xp7T+S6-$yHiyui;v*<9cr3MsDI} zZsAsL<96+^cXAhZa}W1&ANTVB5AqNX^9Yafn2y^gc#@}hnrC>H=XjnMc#)TQSy$|< zyvFOi!JE9r+q}cOyvO@|z=wRK$MzFGbjoF!l zIhl*OnTL6qkNH`E1zCuNRm3jJVl2)QEXh(V%`z;@axBjZs%TeYWmaKTR%3P6U`^Iy zZPsC3)?$!m&waMPhE!@g&+|C``$z9ydJ>1KE+|L6# zXdmKX9^p|Q<8hwgNuJ_qp5a-Yv(NJaFY*#E^9rx>8n5#PZ}Jvz^N#M?_jsQV_>hnI zm{0hW&-k1#_)@Rz*L=gbe8>0vz>oaI&-}u#{HE{r5B}sY{^lS4$sj9xRIN@iy=9F7NR^AMha`@v)xRPx*|``GPO`im&;GZ~2bz`9UA; zPyEa;{K{|q&L8~AU;NEK{L6m~2 zFe|e$yK>k$nTxrZhk2Qg`B{JkS%`&ML`CglEY1=v$x zaW?00F6VJR7jPjLaWR*0sg~Kxxq>UXimSPXYq^f=xq%zGiJQ4aTkUP!&K=yzUEIw* z+{=C3&jUQDL-t`F;ZYvrah~8wp5keq;aQ&Jd0x;(`w}nn3a|1Suk!|P@)mFN4)5xo zeV-5bkdOG7PxzG2_?$2JlCSjIe#5tX$M^iekNm{X{KBvN#_#;WpZaD0<{$p$KL!MM z|H0iq12HgzFerm5xE+Ea8H%A9hG7|w;TeGu8Hte@g;5pFj?Ng2$ykidIE>49jL!s2 z$V5tPCt*@1V{)coN~U6JreRv9V|r#_MrE=yGYhja8?!S9b21lmGY|7JpYq!USdfKS zm_=BW#aNsrSdyh!nq^p42KI%B;ewtj6lB!J4e4+IAh*Wj)qs12$wMHf9qx zWivKc3%ey-u{GPUE!(j@JFp`=u`|1{E4!(?-Ge>Zi@n*0ec6xwIe-H>h=Vo69?D@H z&Ji5RQ5?-N9LsSW&k3B!Nt$d=;Z#oJbk5*R&f;v&;atw+d@ZmSauFAE372vimvaSI zauru|4cBs=*4rDnk(;=gTey|mxScz=le@TEd+fd3$NfCOgFM8;Ji?Z`zCMkHt+B*@9{n#@F5@ZF`wwE{fy7~f-m`sula^= z`Ht`TfgkyapY_H5%5VJ6AN49QRo%`gnh za15^qc0@*EWJY0BMq_lwU`)nhY{pSsJ09aR0TVJ26Eg{uG8vOI1yeE=Q!9;~mg$(D z8JLlon3-9amD!k`Ih51R#oWxpyv)b^EWm;+#KJ7XqAbSZDq)voDVAm#mSs7XX9ZSd zC01q?Rkf?JI%}{dYq2)#urBMdJ{z#18rhB6giYCu&Dnx2*@~^%hHcr7?b$&c?N02> zF6_!~?9LwS$zJTuKJ2T0c7G1wKn~(y4&hJ^<8Y4PNRHxYj?q|q9LIA4Cvp-ea|)+& z8mDsxXKI!`n{zmq^EjUixR8sum`k{n%eb5?w9;P1)m+21T*vj?z>VC*&D_GR+GcO( z4({YG?&couiSA5Mke5-f%dw$?Ye&T0-;a7g+cmCi{{?c#z z5C8HX148=#59$6Hh=Cb|K^cs}8G<1d$_~vi49jo~&j^gjNQ}%VjLK+=ZpUCu#$s&7 zVO+*zd?sK*CSqbHVNxZtlQRWVG8I!Z4bw6m(=!7zGLtgfS(ugCn4LM8lew6id6<{^ zn4bk$P=)NmEW)BJ#^Nl&k}Sp2EW@%ar}B0MR%9hsW))UtHCAU0)?_W#Rvo)8>#;r? zupt|E(jKZjl#^{W}n2g2PjKjE$$M{NMCuAZfW)dc4GA3sVrerFn zW*Vin(=k0WFe5WDGqW%&voSk!Feh^{xANF|nUDEdfCX8Ig;|6}S&YS5LM81|EX^`3 z%W^Ew3arRVtjsE`s%mz1)?iK6Vr|x8UDjiLHef?GVq-Q@Q@a_PvjtnS6l>M!l4|-;T*w{9L3Qb!?7A?kLLtV zOixrS@Gj_bLB8@Y*_ zxrJM~joZ0HJMCTE%{|=9ecaCjJjg>l%p*LiWA<^L;7Ok1X`bO(p5u95;6+~2W%~-R z@*1!625<5fZ}SfC@*eN=0Uzp-{g_Yql+XB_FZhzL_?mC{mhbf5{=kp?#LxV~ul&aE z{K236#ozqHzxrnfg!cXq?fajB7??pAl))IBAsCXO6xt5MunfoWjKGMD#K?@osEo$w zjKP?SWyfY5#$`OlX96ZFsriJ*_nem znTxrZhk2Qg`B{JkSxANLA}q>cEY1=v$xER$*0EV|CVGP1a&< z)?r=NQ+>Mu8?q4_vk9BB8Jn{OTe1~ftBu{3?bx0j*pZ#snO)eG-PoNy*pt1~+wQ}@ z?8p8bz=0gZ!5qS&9LC`qVUOe}j^-GSUXimSPXYq^f=xq%zGNt^90+{$g-&K=yzUEIw*+{=C3uLJf$ z9^zph;ZYvrah~8wp5keq(OLT(&+`H=@)9re3a|1Suk!|P@)mFFj(wN+c%KjWkdOG7 zPxzG2_?$2F(tgF)e8abV$M^iekNm{X{KBvN#_#%J|Ku49jL!s0XeVM~CSg)0 zV{)coN~U6JreRv9V|rz0z(k#QWEXVS!z>2KI%B;ewtj6lBp_+Cr)@B{nWj)qs12$wMHf9qxRWrLeTd*Zt zu{GPUE!(j@JFp`=u`|1-h zxSkuhk(;=gTey|mxScz=Q@iZl+{3-x$NfCOgFM8;Ji?<{6&lIiBYQ zUgRZS<`rJmHTybm@Fs8ZHt+B*@9{n#@F5@RvHgTk`Hau`f-m`sula^=`Ht`TfgknB z{>(4@%5VJ6AN-`_r`=5arm_Znn!5Ew&7?PnFnqe4L;q35?z=({* z$c)0MjK=7U!I+Gt*mfMoWjw}b0w!c4CT0>QWilpb3Z_&lJ2lfVEz>bQGcY4FF*CC; zE3+xPor5`%5B`v9o)%X+|51Q z%YEF>13G9Q;$a@)Q6A%Qp5RHI;%T1YS)Sv0U9d0m5-;-#uksqN^9FD77H{*8?%MZw zpAYzukNB8R_>|B1oGk_AmbCAO7V(2846} z;oLt1F))KLD1$M$Lf9c0ilG^XVHu9$8G#WQiIEvaQSE4q&KQizSd7g$jLUe8&jd`! zL`LMGrO>>y4l^?gFV@cz1fF-*^m7>fCD**gE>S)?O`0w z5gf@;9L+Ht%W)jf37n`&_GC`sR8He`&frYW;%v_0T+ZWsF3>`I5f^g_mvR}Ga|Ks& z6<2c&*J_=;o*TH4o4A=fJjBC1!lOLK<2=EWI%S{c z8J^`ip63N#(_ANfh2?JxYwZ~V?5{K;SZ%|HChe+&rk{=>U}1+oJ(2!k>hgEIs}G898I48t;< z!rKuTk&zggQ5coc7@aW~ld%|^aTr(e?D$N;giOT5Ov0p0#^g-FluV`6b{eK-I;Lj^ zW@IL2W)@~;HfC22J1287H}fzr^D#dQupkSuFpID#i>bI>f+bmsrCEk$S&rpdffZSa zl~u*A%4)368m!4$tj#*C%X+NO25iViYHT-QQ#NCBwqQ%PVr#ZxTef3+b+9|K6FajD zyRsX*vj=;!7kjf0`?8<<+XFa|gE*K&IF!RUoFh1rqc~b)?6Dlj@tnYkoW#kT!l|6b z>72otoTb_J9M0uD&gTLyC52)_R#`2J@g24)ZjWiSS32!*smF*L(4EW4j*N$q4z&J;|^R7}k@Ov`jk&kW3{Om=2wVOC~icIIGC=3;K{U%766x@F(y9p2?V-sb~8Ntl$$n4Bq?lBt+l zY3#I2$Mnp=jLgK$%)+e9#_Y_&oXn-%b{^(sKIUfu7Gxn7W)T);F&0+|yCh4oG|R9o z%dtEwup%q5GOMsEtEswOgEd);wONOCS&#MEfDPG*jn%|%%4TfN7Hr8@Y|S=o%XVzf z4(!NI>TGvmS9W7}_Fzx;VsG|gU-n~v4X_7t5C?MzhjJK)a|B0n6i0Im$8wy;+Y>mE zlQ@}EIF-{loijL-vp8FG?75uB`CPz-T*Sp(!lhisZr2WbCwFl-_i!)waX%06AP?~{kLajKl2N}@*BVN z2Y>PxfAf$2+W#03$@@Q&_df$MFoQ5CgE2TmFeF1Uw8Ge78IIu@fe{&rkr{1Wd?8Ow1%qs$_O@reI2@Vrr&gTBc)qW?)8UVrFJhRy!NBGY4}r z7jrWY^D-avvj7XKkX@KXSd_(BoF!P2rC6F}SeE5jo)uKluEff$!m6ys>a4+%qg6zY4&u^;7rcqY|i0a&f|P8 z;6g6aVtWafav7I%1y^zvS91;5avj%m12<}uy_s9MmD{+TJGhg(xSM;pm;1EeKEQ)K z#KSzoqddmrJi(JZ#nU{)vpQ#==LKHmC0^zgUgb4j=MCQEE#0>7@GkH1J|FNQAMr7t z@F}11IbZOlUfHkthHv?f@A-ir`H7$Tgb z4dF+U5iAPccDi?Aq*u{cYx zBulBZU4~^@j^$Z_6J?U`w`Q zYqnuqwqtvCU`KXhXLYf=vKzaz2Ya#?d$SMwvLE|%00(lA2HQh8l*2fjBRGd-r-%|<9$BhLq6hTJ+Yti8K3h7U-A`S^9|qf9pCeVKH8u7 znP2#o-}s$B_>;f*n}7J1{}>R(`#*~JKLar^gD@zAF*rjoBttPY!zipBj^P=B5gCb* z8HG_9jnNr{F%`>>%{Yw9c#O{kOvpq`%p^?8WK7N!N@=HJYNlaYrek_$U`A$QW@cel zWwWz02XitPb2AU~G9UA^01L7Z3$uud+QnF$C0LTBSej*6mgQKU6ads;tK9 ztihVB#oDaHx~#|gY@mjABQ|CeHf1w5XA8DuE4F4EwpBa3Jv*=?JFzpnuq(T%koJg4*a1zzMOUgi~E9zfaZ~2bz`GFt#iJ$p}U-^yS`GY_8%l^$j{L6m~i0b~M zx_<^@UmC+cTF&LAv7@Khzm+=^%37C+H zl-N$fq)f)-Ou>{)#nep0v`okJ%3xJXAb6MF6L$)=4C$Rw+pZ!3$ZYZ zuqcbMI7_f3OR==d*kxIcwnR_{~4G;7?i;noFN#Jp%|KB7*^rz z@QlESjKs){!l;bK=#0UbjK$cDqqufF#%BU1WFjVJ5+-FbCT9w!R4O|)(=aX5F+DRd zBQr5GvoI^OF*|c8r=5$rnTL6qkNH`E1zCuNS%gJZ%r4FnEXh(V%`z;@axBjZtjJ2N z%qpsCS7UY7U`^IyZPsC3)?$!m&xrv*(g1KE+|L6%$U{18AK_6R<8hwgNuJ_qp5a-Z<9S}-MP0Hl^9rx>8n5#PZ}Jvz z^A7Lwp6=TZ_>hnIm{0hW&-k1#_>!;qns4}4@9g*dz>oaI&-}u#{KoJ6!Jqu4-}WE= z9g?9Knqe50;TWC~7?F_}nNb*((G=Z|!I+H2*o?!t zjK}y)z=TZ1#7bf(Wilpb3Z`T#re+$ZWjdy324-X?Wwx_0E3+{>b1)}!F*oxtFY__K z3fKi%h=o~%MOlo+S%M{5ilteGWm!(;?Fy{OO03K(tjcPv&Kj)ATCA-)c3swEeKuf2 zHezEoVN*6^bGBehwo+@m4coFE+p_~ZvJ*SA3%jx#yQ_!YlfBrReb|@%*q;M9kb^jw zLp0PL#^D^nksQU*9K*33$MKxNiJZjAnqp7oG*0IX&g3l4<{ZxDJkI9=EwmSLF_&;D zmvK2)a3xo9HP>)0*Kxfz*c-Wto4JKsxsBVogFCs4ySYbu?S0(O13bt>Jj^3J%40mv z6FkXNJgqbKS)Sv0Uf@Mu;$>dpRbJzD-q20^7H{(o@A4k+^8p|75g+pjpYj=>>xKQ2 zulSm8_?GYZo*(#;pZJ+y^ws{x@BG1^{Ken=!@vB;fEeyShWl3_J1~PVD1$LLLog&m zF*L(4EWDS@fCD**gE@plIgGAn4dANh%&`GsHkjo%Yz%{Yw9c#O{kOvpq`%p^?8WJ+$QU`nQ9YNlaYrek_$U`A$QW@WLnG8?lq2XitP zb2AU~G9UA^01L8^3fo0kl*L$_C0LTBSej*6mgQJp73_+v#LBF~s;tK9tihVB#oDaH zx~!-Ab^|tKBQ|CeHf1w5XA8DuE4Ee}yDi(XJv*=?JFzpnuq(T$ZI_Bo#C1zzMOUgi~E{)#nep0v`S~EX9i|uCT3<9W@R>JXAb6MF6L$)<+bxMKMSxR3$ZYZuqcbMI7_gk zO4+4ZhGkifLr8@sayd$JdMvk&{SANz9v2WpT#m_s;}!#JEHIFh3{nqxSY z<22r$z=@p1$(+KeoW|*#!I_-J*_^|84j-r{ZE;a%S2eLmnrKGI|R37_&ApYsJ@@)ck64d3z|-|K_@k)QaP zU-*^Z_?49jL!s2$V5!cBuvU=Os*7mN~U6JreRv9V|r#_MrLAWW?@!l zQ+7KCb21lmGY|7JAM>*S3$hRktB75c#aNsrSdyh!nq^p)tj6lB z!J4ea+N{I6tjGFlU^iqVHf9qxWivKs3$|n{wq_f(wcD{hJFp`=u`|1{E4#5fd$1>a zvA6ozec6xwIe-H>h=VzVLphAYIYJ}tQ5?-N9LsSW&k3B!Nu10noXTmOt{L`B&f;v& z;atw+d@kTZF5+S?(NcREmvaSIauru|4cBrV*K-3mauYXei@lZGxScz=le@T^d$^bT zxSt1f&_2Y&Ji?iU*siT<`rJ$HD2cp-sCOb<{jO&@9{n# z@F5@ZF`w`$pYb_g@TFeaula^=`Ht`TfgkyapZSGf`HkQCLqF|b{LMf7%YO`rrGm4_x(HNaE7?ZIWn{gPI@fe>8m{5uA z#7x4ZOvdC)!IVtJ)J(&)Ovm)hpp14VW@Z*p)MIGz(YQIqV+oWiM`#_62FnViMhoWr@Cr}_2*F61IE<`ORDGA`!|uH-7N z<{GZmI(t1ga3eQyGq-Rnw{bgna3^c z%X2)>3%tlnyv!@S%4@oA-{4K&;%(mHUEbq;KHx(>;$uCrpYj=>^95h>6<_lW-|`*b z^8-KflRn#D_?6%Ioj>@KzxbPf_?Q0}5ZC)ZuJ=C!F))KLD1$LLLog&mF*L(4EW;_h z9f1)UiIEwFQ5lWV8G|tyi?J2Qj>~wA&jd`!L`=*iOv+?T&J;|^R7!29VOpkRdS+lo zW@2V$VOC~icIB{hG8c0*5A!k~^Roa8vJeZi2#d0qirXbvlBHOhWmuNwSe_MFk(F3k zRqU#)#_FuWnykgzti!sj$NFrbhIS)1W)n7LGd5=nwqz@|W*fF;JGNH`yCXZXGrO=W zyRkcauqS)5H~Xls-H-h_fCD**gE@plIgGe6<2c&*K!@#a|1VW6E|~^=?4vx!<2=EWJjK&I!?Qfc^Sq#o_9b5C6<*~vUgr(o(_ANh%&`GsHkjoNtD!1#^g-FluX6cOvAKH$Mnp=jLKwZW)@~;HfCoI=43ABW*+8cKIUfu z6|@VnFpID#i?KLMup~>dG|RB8%Gu>vffZSam05*VS&h|MgEd)8we32r%X+NO25iVi zY|JKX%4TfN7Hp|jc5Aj_Tef3+c3?+#VrO72otoWfb3D%ryvR$u%qzUgYrL)-_D$a6ZQkKs-s62f;6py*V?NFWwr$(CZQHhO+qT{LFe2va z=W-tBa{(7}5f^g_mvR}GbA?vgtGJqLxR&dfJjBC1qNDaP9_I<3|m;@QlESjKs){!l;bK=#0Ubie<-U9L8ll#%BN%Fd-8$ zF_SQJnVCgd?QG1>9L&jF%*{N^%Y4kw0xYOPc3~D_ zQ5IuymS9PiVriCPS(a0Iy8#CkzpAFcMjo6q?*p$uK zoGsXrt<>6X!?tY4_Uyop?8MIO!mjMb?(D&y>Sgz4ANFNG_U8Z&_fAKf}@Gt){P(t^g(ET$ogD9vSjKLX#AsLFH8HQmQj^P=B z5gCb*6~&IqXpGJnjLBGx%{Yw9c#O{gC9o4R5fd{BlQJ2TGX+yJ6;m?}(=wgX+ZmXV znV6Ybn3dU>ojI73xtLpd?7Ym!{4BtNEX2Yr!lEq3;w+((b}5!-8J1-^mS+W4WF=N+ z6;@?6R#y$XCTp=a>##2Cu|6BHAsewVo2aSXjLq4CE!m2#*@kV|j_uij9odPU)y3}0 zZtTt;?8#p2%|7hQe(cWy8fXvVU=HC>4&!i+;7E?*XpZ4nj^lVuuqSd7Cvys?avG;| z24`{>XLF9`+VeP{3%HPrxR^`0l*_oBE4Y%YxLRxMwOq&b+`x_8#Le8ot=z`#+@YQJ zF7DMH+;)?e9sU1Xn*2oe&JVs<9GhxPyXU>{^4K! zvjZh65a@r2y#E=PK^T<57@Q#(lA#!yVHj57?C^}hh>XO@jKZjl#^{W}n2e>^b{xiK zJjQ1L6EGnYF)@=cDU&fdQz)gKim91~X_=1cnSmLZiJ6&&S(VMs&K%6iT+Gcp%*%Yt z&jKvSLMm(*VNn)iah707mSSm^VOf@Ac~)RWRkACy3ahdjtFs1cvKDKz4(qa>>e~(2 zkd4@wP1uyp*qklclC9X9ZP-@r?Dp)yj_kzF?82_>#_sIFp6sRGb|3a-KlbMU4&)#X z<`53$Fb?Mkj?^f7G{5*dZB; zp&5o@8IIu@fe{&rkr_o%?P!e77>vnSjLkTV%Xo~>0488UCQ@QM36nAzlQRWVG8I!Z z4bw6m(<_6Wk(rp8S(ugCn4LM8lew6id6d`A$NVh7f-J#;r?upt|a=M2XitP zb2AU~G9UA^01L7Z3$qA|s+e7zC0LTBSej*6mgQKU6z*SJFzpnuq(T=+qj)OxRblMn|rvI`?#M6c#wy5 z*gnFeJjUZZ!IM12(>%koJje68U|-}VUgi~E!|0Q+*49p-5 z%3uu65DICBVrYh8ScYSGMqornVq`{PR7PWT#js;C7GpCG<1!xOGk^)0kcpUBN$jLd z#^g-FluX6cOvAKH$MnpgjCLkwW)@~;HfCoI=43ABW*+8cKIT^eyC4g(FpID#i?KLM zup~>dG|Q;0U5@2hffZSam05*VS&h|MgEd);wN=Ni%X+NO25iViY|JKX%4TfN7HVm? zVr#ZxTef3+c3?+#VrOyZ?5Ui_>72otoW|m;@QlESieyJ- z6h>t1Y{k}W!?tY4_Uyop>ST9j7j|Vgc4rUv zWH0t+ANFNG^|uFbAO~?Uhj1u|aX3eCBu8;H$8fC1+2c8Z6FG^KIfYX>jng@UGdWAM z?Kzywd7RG$T*yUS%q3jPWn8Wm_DZheYOdj0uH$-c;6`rZW^UnDZqs&q2X}H8cXJQ- zav%5e01xsI59^41l*f3SCwP*lc$#N;mgjh$7kH7EblJYbtGveRyuq8i#oN5YyS&Hy zdSE}~BR=L6KIJn$=L^2%E57C%zU4c;w?FVBKk+la@GHOZJAd#efAP2e*njztfs%Xw zC-?tnUW7@Khzm+=^%0ZhPz zOvJ=YqNH{*CT9w!WGbd+8m47Bre_9bWF}@-7CS4mF*|cGCv!13^Dr;-F+U5apk0WC zS%gJdjKx`kC0UB4S%zgV$^ zHe++PU`w`QYqhc4vK`yA13R)4JF^SBvKzazhkDw**qeRWm;KnE12~X_IG95?l*2e& zBkYkJ#nBwYu^h+ooWO~k#L1kZsrEEZ=M2u|EY9W}&gDGL=K?O&B6~5Ha4DB@IahEc zS8+Aha4pwyJvV5hy@{K-gO($U{8LBRtAuJkAq5$x}Sd zGdgRZ<9S}-MPA}%Ug1?<<8|KPP2I9@^A7Lw9`Ex3AMz0&^9i5w8K3ioUfQqtns4}) z@A#e{_>rIZnP2!--|X-F!Jqua-~7YB{Kr5k+F zuq(TE22HAr-ghM%u!#RQ@If|n>hGRKS%koJje6Az>B=Z%erD;=QhNWV^!{gH24PSJV{nFGNQPo)hEZ5M9K$mLBQg>rGYX?J8ly7?V=9&%n{gPI z@fe>0Ou&Rp#KcU(q)f)-N@1sDDyC){re!*&X9i|uCT3<9Wwo<0J9986b1^sbFfa2l zKMSxR3$d_@*hN{4#aV(SS&F4uhGkif86{n(!aIFN%l zm_s;}!#JEHG}0c$(Hz6E9LMpTz=@p1$(+Kenr2Vu49?^%&gLA>84j-r{ZE(Ovr<@ACm4@(~~N37_&ApYsJ@ z>XrSPZ}^t)_?{p5k)QaPU-*^Z_?=R zhGrOsWjKas1V&^eMphI%Dx)zvV=yLTF*f5cF5@vi1DJpbmB>!aBuvU=OwJTc$y7|u zG)&8MN^fUiMrLAWW?@!lV|M0XPUd26=3!ptv-7h63$hRkvj~f_7>lz6OR|(o+htgm zau{Zm$FZ;1S2XG(yoFHNl?9 zNu10noXTmO&KaD^S)9!|oXdHdZ!h3NF5+S?;ZiQ+a<1S?uHtH~vDb1P*K-3mauYXm z3%7C`w{r(~a+h}7d$^bTxSt1jkcW7fM|hOScw8s!lRU-KJj1g*$Md|vi@e0kyuz!z zrt9_%-sCOb<{jSUJ>KU7KI9`l))V_FpYb_g@FidIHQ(?p-|;;^@FPFzv;Bo%`HkQC zgFpF;zxjuM`Hz88d;h2Q{%2qYVNeERaE4$=hGJ-jVOWK;!!rUSG7=**3ZpU_qca9$ zG8SVqj^f(!7@q-5z=TZ1#7x4ZOvdC)p_Fzire+$ZWjdy324-X?W@Z*)0*Ks{Ja3eQyGq-4~y^Y(sgFCs4ySayZxsUsKfCqKR zKFlLL%40mv6FkXNJk2va%X2)>3%Y1u;$>dpRbJzD-r!B%;%(mHUEQ^95h>6<_m>-rDc@o*(#;pZJ+y_?6%Ioj>?fzwF=q!@vB;Kxy268u!n@48ouc z#^4HJhh!**W*CNLIEH5gMr0&LW)wzcG)1>#FeYO$HsdfZ<1s!1n1BhHNQvzvOv+?T z&J;|^R7}k@Ov`jk&kW3{Om=2wVOC~icIIGC=3;K{U$+jz@-kl+DCvh^Ta4M&9I%jaEX4$hj zhjTfP^SOWvxrmFogiE=M%eg`;?NwaOHC)SeT+a>M$W7eLE!?VY_IB>zPVVAv?%`hU z<9;6CK_22^9??V|)fM0TVJ2 z6Eg{uDw&;}DVUO}n3`#rmg$(D8JLlol-bV0tjxyj%)y+@#oWxpyv)b^EWm;)WEW-; z7G*IOX9<>MDVAm#mSs7Ww=1wBE3q=GuqvyuI%}{dYq2)#u&(Oa_1S<8*@%tVgiYCu z&Dnx2*-EYLHf+mwY|jqt$WH9cF6_!~?5-YmPxfMO_F-T4V}B0dKn~(y4&hJ^({OtP zM{*QLa}39F9LIA4Cvp-eYl=OU(>R?oIFqwDn{zmq^EjUixR8sq*j~b=T*l>G!IfOa z)m+21T*vj=U~l9mZsrzlD@mAGl+uP!5Ew&7?PnFnqe50;TWC~ z7*Ubz$c)0MjK=7U!I+H2*o?!tjK}y4Py#z46EQK9Fe#HUIa4qtQ!zEuD6O52>6w8U znTeU1g;|-6*_nemnTxrV$Ii=q%+CTW$U-d4A}q>cEY1=tX_sPYmSI_zV|i9!MOI>E zR$*0EV|CTAYqAz=vkvRB9_zCK8?q4_vx%D8&DfkR*pjW-nr+yY?bx0j*ioJA&g{aj z?8ffw!Jh2J-t5D^?8p8bpn>)v4(1RJ~@u04N*|OSz28xq>UT%3jSiT+4M_&kfwjP29{a+{$g-&K=rm z@8WLm;a=|JejeaK9^zph;ZYs4kMjgi@)S?=4A1f$&+`H=@)9reimuw%c%3(Rlec)A zcX*fgc%KjWP><}#e8Q)E#^-#&mwd(7e8abVr}y>;e&i>9<`;hDH-6_2{^T$I<{$pm zKRZwc@Ba+`{|w9^49Z{(&JYaAPzH>n2?E>m`RwF$&}nq!IVtJ)J(&)Ovm)hz>Lhq%*?{9%4TP04(4Po=4Kw|Wj^L- z0TyH-6}F48D2uT;ORywMu{6uDEX%PxE3l#}*_By^RauSIS%WoMi?vyYby-jK?FMYf zMr_O`Y|3VA&K7LRR&1>{c3ZY%dv;()c4B9CVOMrzclKaU_EK-V5Bsto`*Q#Xau5e| z2#0bQhiimAlA}19V>p)MIGz(Yk&`%?Q#h5=G~J%TnViMhoWr@C$N5~qg;$uGHQ$FK!zR*ki z6<_lW-|`*b^8-Kf6F>6{zw#Tu>xcc5zxbPf_?Q0}D5Lw&=>8d)K^RoQ?BEQ+kPOAp z48yPt$MB56h>XO@jH0M^G)89(#$+tUW*o+4JjQ1L6DXmbh>4kmNtukvnSv>qim91~ zX_=1cmBG%)Ow7zI%*t%c&K%6iT+Gcp%4_FieimRs7Ghx*VNn)iah707mSSm@vCFa? z%d-M2vJxw^3ahdjtFwk`+O=4lby%16Sf35pkd4@wP1sb;?B;C2mTbk=Y{Rx}$M)>N zj_kzF?4quAH+E+a_GB;iW*_!tKlbMU4%8rfFo$p`hjBPZa3n`@G{U1zgBQT+Ah0%4J;66&aZs104;%08) zR&L{V?%+=CvUhV2_i`Wi^8gR>5D)VRkMbCg^Mp>?r+AuYc$VjQo)>tLmw1_1cvaWz z>%766yv5tR!@Io4`+UHMe5A+r6F%iLKIaR*Ntl$$n4Bq?QmO3JOvAKH$Mnp=jLgK$%)+e9rtEeO=43AB zW*+8cKIUfu7Gxn7W)T)uF}pZRup~>dG|R9o%dtEwup%p|vR#E$S&h|MgEd);wONOC zS&#MAz;4J!Y|JKX%4TfN7Hr8@Y|S=o%XVsScVI_$VrO_?v(Dm;V?jv-{8N{u!7-7?i;n zoFNp_4#m(6!>|m;@QlESjKs){!l;U7M`sMiWGu#J9L8ll#%BN%Fd-8$v69$HnT*Mq zf+?AbshNgpnU3k1K^g5#%*-sz%52Qe9L&jF%*{N^%Y4kQ0(Ljng@UGdYX1IfrvOkMp@e3++W*%q3jPWn9h`T**~j%{5%Bb@qC0 z;6`rZW^UnDZsT_D;7;!1Ztl@udms1n01xsI5Az6*@)(cv1W)ReeVS)@mgjh$7kH7E zc$rstmDhCLzQLQk#oN5YyS&Hye87i%#K(NXr+Q{T=L^2%E57C%zU4c<=Lde|Cw;cR z@GHOZJAd#efAKf}@Gt){P!{)}#r-R=9fUy{jKLX#AsLFH8HQmQj^P!-j>t%i%qWb? zXpGJnjLBGx%{Yw9c#3ZaFaZ-X5fd{BlQJ2TGX+yJl~UVjn3n07o*9^tnV6Ybn3dU> zojI6Ox$NA`!@SJL{4BtNEX2Yr!lEpu;&us^WGR+r8J1-^mS+W4WF=Ns6}u{{u{vw8 zCTp=a>##2Cu|6BHAseZ&-GoiqjLq4CE!m2#*@kV|j_uXK?#NE;%r5N8ZtTt;?8#p2 z%|7hQe(G-z;6M)IU=HC>4&!i+;7E?*XpOPQavaBV0w;13Cvys?avG;|24`}XX4`W( zm-9HE3%HPrxR^`0l*_nWE9{kA#noKHwOq&b+`x_8#Le8ot=y*V_73jkF7DMH+pNo<9mMKM}FdGe&JVs<9GhxPyXU>{jvY@9|L9e|Ih0G z&%g}ApbW;~48f2JWrt=MhGjU0X9PxMBt~WwMrAZcXAH%(V=*@4FfQXUJ_DG537LqA znM6tLWK7N!OvzMC%`{BQbWG0-%*ag4tSoj`W@C2dU{2;@ZsuWL=3{;qP(iy83$qA| zvKWiA1WU3MOS25is+?V(6Ci2XQcmaHxja!#RQ@ zIf|n>hGRL7<2iv7If;`wMN{o*oX#1X$yuDuIh@ORoX-VZs73Z-F5yxx<8rRxO0ME+ zuHjm)<9cq;Mtc)Ca|^d}8@F=@cXAhZa}W1wpS_<4c#wy9m`8Y&$9SA4c#@}d+CIax zJje6Az>B=Z%e=y?yvFOi!JE2e-{u|OV|)eJ37C+Hn3zeJl*yQ!DVUO}n3`#rR_W~Y%)pGy z#LUdXtjxyj%)y+@rQCKN=4C$SX8{&uAr@v47G*IOR|&f$OR+S|uq?~5JS(swE3q=G zuqvymx?O`cS&Ow2XYVxa|nlW7>9ENM{*QLa}39FoW|P|IFXY$nNv8G(>R?o zIFqwDTXXEWoX7cGz=d4I#azOrT*l>G!IfO4)%F^$K z7%02=RhGrOrwZkzyBQPQ(F*2htDx)zvV=$&-*|8aiaT$;C z8NdWg$V5!cBuvU=Os*7mN~U6JreRv9V|r#_MrLAWW>Ho<8?!S9b21lmGY|7JAM>*S z3#yP^m_=BW#aNsrSdyh!nq^p)h=VzV zLphAYIYJ}tQ5?-N9LsSW&k3B!Nu10noT_Q|bk5*R&f;v&;atw+d@kTZF5+S?(NcRE zmvaSIauru|4cBrV*K-3mYLmU0Tey|mxScz=le@T^d$^bTwBJ6!gFM8;Ji?H>m{5uA#7x4ZOvdC)!IVtJ)J(&) zOsDjA24-X?W@Z*p)MI9?O%iJZjA zoWiM`#_62FnViMhoWr@Cr}_2*F61IE<`ORDGA`!|uH-7N)*5>)*Ks{Ja3eQyGq-Rn zw{bgnXs5l4ySayZxsUsKfCqVqhk1lYd5p((!am7UJk2va%X2)>3%tlnyv!@QYG31Z z-r!B%;%(mHUEbq;KHx(>;$uCrpYj=>^95h>6<_lW-|`*b^MgLxpZJ+y_?6%Ioj>@K zzxbPf_?Q0}D5v*-PVavPW)KEtFa~D`hGZy)W*CN5I6FKeFd`!{GNUjmqcJ*TFeYO$ zHsdI+9gp!DzywUlL`=*iOv+?T&J;>%r($ZRVOpkRdS+loW@2V$VOC|cvoi;CG8c0* z5A!k~^Roa8vJeZih>F_9SezwTlBHOhWmuNwSe_MFQI+hLMGrO=WyRkcauqS)5H~Xls-H-h_ zfCD**gE@plIgG(_ANh%&`GsHkjod zw943JS&rpdffZSam05*VS&h|MgEd)8we32r%X+NO25iViY|JKX%4Te?7IsUvVr#Zx zTef3+c3?+#VrO72otoWMDVAm#mSs7XX9ZSdC01q?Rkf?JI%}{d zYq2)#urBMdJ{z#18rhB6giYCu&Dnx2*@~^%hHcr7?b$&c?N02>F6_!~?9LwS$zJTu zKJ2T0c7G1wKn~(y4&hJ^<8Y4PNRHxYj?q|q9LIA4Cvp-ea|)+&8mDsxXKI!`n{zmq z^EjUixR8sum`k{n%e35H!IfOa)m+21T*vj?z>VC*&D_GR+GcO(4({YG?&couiSA5Mke5-f%dw$?Ye&T0-;a7g+cmCi{{?c#z5C8HX1Lg7m&*T0X zm_Znn!5Ew&7?PnB+783849D<{z=({*$c)0MjK=7SVaH@F#%3JGWjw}b0244F6EU%p z*h!g;$(e#FnTn~IhH06O>6w8UnMs-LEX>Mm%+4Il$z06MJj}~{%&!7=K^9_R7GY5q zV{w*XNtR-1mSI_zQ+c}rE3y(RvkI%S8mqGgYqAz=tBzfl^;n+`*pQ9bm`&J}&DfkR z*pjW(+HS+PY{&NOz>e(1&g{aj?8ffuVfSP&_GTaUWk2@k01o6J4(1RJU62#@j@kMjgi@)S?&jD42p zc%Bz{k(YRxS9q1zc%3(N)4s*qyu-V^$NPN1hkV4xe8Q)E#^-urzvL^v<{Q4{JHF=! ze&i>9<`;dnzwtYN@F#!qH~;W2|1nTr_n+7OGcbcFs2z;K8G<1hilG^XVHu9$8G#WM z$&Sn@jLK+?&KQizSd7g$jLUe6ZwD{|6EYDKGYOM28Iv;wQ!*7(GmX;P>6o4wn30*7 znOT^X*_fRb2yjtIG+o+kc+sOOSqKFxST7vQmgFMT*I|o$MxL6joie|+`_HgrtS6)?&L1+ z<{s|lKJMoM9^@e&<`Ev%G5a`A@FY+1G|%uX&+$Aj@FFkivVDbDd5zb3gEx7Lw|R$m zd5`z`fDiS^e#|F)%4dAe7ktTAe9bp}%XfNjf8a-c;%9#0SAOGn{@_pk;&1-pU;VQK z<@5f}=l{>Z48ouc#^4OWkPOAp3S)<5IEH5gMr0&LW)wzcG)89(#k6BFHsdfZ<1s!1 zn1BhHh>4kmNtsN^?G#MOR7}k@Ov`jk&kW4SOw6n-c2;I%cIIGC=3;K{UwtZbhj^Grc$CL@oF{ma zr+AuYbk;t{^Sr=|yu{1A!mGT->%766yv5tPW8dXH-sb~8XxHhz%ZKc#I|kQ-mz`lwr$(CZQHhO+dA{&8T0)CStC`uyQ}z~ANY}< z_?ch$mEZVXKkT3U#ozqHzx>BQ0nUGb^UuHx!k`Lf2WJR|WGIGa7=~pyhGzsuWF$ql zqcAF?F*;)~CSx%+<1jAcF+LM8p%U4NnS@E1jLDgTDVd6?nTBbZPU-Co%*ag4%q+~x zY|PFa%*kBL%{-$ju|EfJpa$84IfO$wjKevCBRPtrIfi37PUGzfoXAO>%qg78X`Id(oXJ_7 z%{iQ_dG>rR;6g6qVlLrQF5_~p;7YF2YI_aWavj%m12=LLH**WOavQgE2X|_hy_=RhGrOrwZkzyBQPQ(F*2htDx)zvV=yLTF}C8^aT$;CnScqI zh>4kmNtukvnL;V;R7}k@Ov`jk&kW4SOw7zI%*t%ct{iqw=3;Ktj6lB!J4ea+N{I6tjGFlU^iqVHf9qx zWivKs3$|n{wq_f)WjnREJFp`=u`|1{E4#5fd$1>avA6ozec6xwIe-H>h=VzVLphAY zIYJ}tQ5?-N9LsSW&k3B!Nu10noXTmOt{L`B&f;v&;atw+d@kTZF5+S?(NcREmvaSI zauru|4cBrV*K-3mauYXei@lZGxScz=le@T^d$^bTxSt1f&_2Y&Ji?iU*siT<`rJ$HD2cp-sCOb<{jO&@9{n#@F5@ZF`w`$pYb_g@FidIwcglo z`Ht`TfgkyapZSGf`HkQCLqF|b{LMf7%YO`1(D^Uu{4+3vFerm5xE+Ea8H%A9hG7|w z;TeGu8Hte@g;5pFj?Ng2$ykidIE>49jL!s2$V5tPCt*@1V{)coN~U6JreRv9V|r#_ zMrE=yGYhja8?!S9b21lmGY|7JpYq!QEWm;+#KJ7XqAbSZEWwg2#nLRJvUWL^X9ZSd zC01q?R%JC-XARa=ExR`BurBMdJ{zzh8?iB)uqm6dIa{cu-HNT*hHcr7?b(4H*@>Om zg?U$zJTuKJ3eW?9Txl$Uz!x58+S_<8Y4PNRHxYj^S92<9JTsL`||Ma|)+& z8mDsxXL1&2a}MWnp61&NxR8sum`k{n%eb5?xRR^5nrpaL>+JR1z>VC*&D_GR+{W$P z!JXWt-S!^t!V%PxZ`x&KG>iSA5Mke9L!y&ky{_Px@?s;a7g+cmCi{{^D=` z;a~ov|MUX=SIG5012YJNG8lt11Vb_uLo*D+G91Gzf*p~O7@1KRmC+cTF&LAv7@Khv z*N(^dOu&Rp#KcU(q)f)-Ou>{)#neh;r)4^(X9i|uCT3<9W@R>JXAb4Gb1^sbFfa2l zKLc2R1zCuNS%gJdOvUXIEXh(V%`z;@axBjZtjJ2NtSWX@R%3P6U`^IyZPsC3)?$!m&xrv*(MO*D{+|C``$z9ydJ>1KE+|L6%$U{7=Blb}q z<8hwgNuJ_qp5a-Z<9S}tMf(yj^9rx>8n5#PZ}Jvz^A7Lw9`Eac{g99Nm{0hW&-k1# z_>!;qns4;he#iIxz>oaI&-}u#{KoJ6!JqnN|K=b5V|*rHLMCEjCSg(~vy(FgQ!*7( zGY!)+9n&)dGcuDh+gX^E*_fRMDVAm# zmQ^{sJS(swE3q=GuqvyuI%}{dYq2)#sIFa)_1S<8*@%tVgiYCu&Dnx2)yi(oHf+mw zY|jqt$WH9cF6_!~>TdU7PxfMO_F-T4V}B0dKn~(y4&hJ@vxjp8M{*QLa}39F9LIA4 zCvuV|+fz7|(>R?oIFqwDn{zmq^EjUixKNAi#azOrT*l>G!IfOa)m+21T&MN+25#gg zZsrzl9l=@XL*k2d4U&siI;hW zS9y)sd4o4~%f8J!yvuvM&j)=RhGrOsWjKas1VywXF*2htDx)zv zV=yLTF*f5cF5@x264(iuh>4kmNtukvnSv>qim91KY3+1O&kW4SOw7zI%*t%c&K%6i zT+FRJc3$RVeg?1r3$hRkvj~f_7>lcfU6Q3(nq^p)au{Zm$ zFZ;2-2G|2Rh=VzVLphAYIf5fOilaG3W9@Mq&k3B!Nu10noXTmO&KaD^S)8po_FT^6 zd@kTZF5+S?;ZiQ+a<0%ydlgr64cBrV*K-3mauYXm3%7C`w`+&Jle@T^d$^bTxSt1j zkcW7fM|9La#^XG}lRU-KJj1g*$Md|vi@e0kx?*4DHD2cp-sCOb<{jSUJ>KU7J+vS3 zF`w`$pYb_g@FidIHQ(^9-r4W@fgkyapZSGf`HkQCgFpF;zxhXh?SJ%#?m+()b^p)6 z48ouc#^4OWkPM~Jb{K|bIEH5gMr0&LW)wzcG)89(##AgjHsdfZ<1s!HFd-8$F_SPU zlPS5Kf+?AbshNgpnU3k1ff<>JnVE%ImCer19L&jF%*{N^%Y4kw02W|D6|xJn2#c~9 zi?akvvJ^|R49l_{%d>(i+Lc(DRalkPSe-RkleJizby!#R?D}lLhHS*fY{I5&#^!9n zmTaZgb{n>3JGN&Bc4Q}ZW*2s4H+E+a_EaysH~X+J`>{U%766yrtXr9p2?V-sb~8AU@dKlzKl`GUEX%PxE3hIfu`;W$ zDyy-&YS=Ydi?vyYby<(~*?h7qP3>lE&K7LRR&32SY|D0R&kpR!PVB5Mc2{;| zclKaU_F`}LVPE!Re-6+6{zw#Tu^9O(O7k~2)|LULZ z1Fk^-6?gs5zzo8m494IL!H^8a&T+Gcp%*%Yt&j1!+ zK^9_R7Ew{V7>lz6OR^M8vkc3!9Luu;E2@%RnN?Vo)mWW1Sd+C_n{`;1^;F+(z=mwZ z#%#i-Y{uqn!Io^r)@;MJYG=1+2XyoFIe`;7iIX{nQ#no3?HQcOS)9!|oXdHf&jnn_MO@4!T&iXE za<1S?uHtI0;aaZadT!uGZqjCZ3%7C`w{r(~au;`V5BG8(_wxV`>X3byM|hOSc$_DA zlBal@XLy$9bl$$ei@e0kyuz!z#_PPno4m!_x?|tvJ>KU7KI9`l<`X{UGd|}FzT_*t zw%_nA-|;;^@FPF*Gr#aFzwx_%*gyG;zxjuM`Hw!_4)k9M=bwQYgh3gM!4<*|$xsZ< zFbvCZ49^IR$ViOLD2i%FV|2z~OvYkt#$jB>V|*rHLMCEjC9#t-8Iv;wQ!*7(GY!)+ z9n&*|GTND#nOT^X*_fRMDVA0lyDZDG zJS(swE3q=GuqvyuI%}w=U5m9@hjm$x_1S<8*@%tVgiYCu&DFwg$yRL5Hf+mwY|jqt z$WH9cF6wG`V|VsoPxfMO_F-T4V}B0dKn~(y4Y7xE7>9ENM{*QLa}39F9LIBlCfbuY znNv8G(>R?oIFqwDn{zmq^Eh7%?1fy!#azOrT*l>G!IfOa)m)>s_ByWT25#ggZsrzl z9E3%~Lkzw-xw z@)v*e5C8HXeX%9bfBIiZ*Z&N{pbW;~48f2L#n24Huned0b_7ObBt~WwMrAZcXAH(< zEXHOW##KB!J`*q@6EQK9Fe#HUIa4qtQz^BbhH06O>6w8UnTeU1g;|-6*_nemmCMe} zJj}~{%+CN8U_lmQVHROg6|;-81WU3MOS25ivK-5^0xPnTD%(|9mDO0CHCU6iSetcN zm-Sem4cJhP?8a=urfkOMY{8an#nx=Ywrr>Nb_aH3Cw68Rc4aqqXAkydFZO01_EkT- zKL>Ci2XQcma43gyI7e_KM`^S@hGRL7<2iv7If;`wg;P0=(>a4PHOrpOIh@ORoX-VZ z$VFVtC0xp7T5hl4O0ME+uHjm)<9cr3MsDI}ZsAsKv$t~xcXAhZa}W1&ANTVB5Au)> z+edhm$9SA4c#@}hnrC>H=XhQh?2Ej_%e=y?yvFOi!JE9r+q}cOyr=v213u&9_rdfBBET)Eel&QqDgEGYEq+ z7=tr}LfWAinqe50;TWC~7?F_}nNb*((HLDZ?3j$j*o?!tjK}y)z=TZ1#7v^3b}}Ys z3Z`T#re+$ZWjdy324-X?W>ywEE3+{>b1)}!F*oxtFY_@!1605+$U-d4A}q>cEY1=v z$xER$*0EV|CVGP1a&<)v@cc9_zCK8?q4_vk9BB8Jn|(TH3AH znr+yY?bx0j*pZ#snO)eG-Pm0{?4In!-t5D^?8p8bz=0gZ!5pHY_An0T2#(|^j^-GS zN*|OSz28xq>UXimSPXYq^f= zwZY!VP29{a+{$g-&K=yzUEIw*+H3FQejeaK9^zph;ZYvrah~8wow85!4A1f$&+`H= z@)9re3a|1Suk(g(+P8R{cX*fgc%KjWkdOG7Pxw^N?B{&Jmwd(7e8abV$M^iekNm{X z{GzY+H-6_2{^T$I<{$p$Kl(y@p#Mrc{|ameVNeERaE4$=hGJ-jVOWM^ct&7EMY1C^ z3ZpU_qca9$G8SVq4&ySO;@b(BkcpU>Ntl$$n4Bq?lBt-QX_!{&?DWjQjLgK$%)+e9 z#_Y_&oXn-%b{^(sKIUft3$P#yu`r9UD2u7MU4kW9ilteGWm%5pS%DQnjh?D3qyiJZjAoWiM`#_62FnVhBB z_8iXTJkI9=F61IE<`ORDGA`!|uGA`fHP>)0*Ks{Ja3eQyGq-Rnw`se*gFCs4ySayZ zxsUsKfCqVqhjqk0%40mv6FkXNJk2va%X2)>3%tlnx@=$JRbJzD-r!B%;%(mHUEbq; zJ+L405g+pjpYj=>^95h>6<_lW-}0T_+aLImpZJ+y_?6%Ioj>@KzxZ2!?7#d+UuF;V zUm5rR49p-5%3uu65Ddvs46QJBScYSGMqornVq`{PR7PWT#!yT<7GpCG<1!xOGXWDa z5fd{BlQJ2TD}|ksshFB+n3n07o*9^tnV6Ybl-170?99QO%*EWy!@SJL{0v|L6|@Vn zFpID#i?KLMup~>dG|R9o%dxyF*cDlcm05*VS&h|MgEd);wOL1X?Ru=w25iViY|JKX z%4TfN7Hr8@Y^^qSTef3+c3?+#VrO72otoWYM$YKlqcs_?v(Dm;dO8EP?(j>-;k?gD|Lq*})ltAsLFH8HQmQ zj^P=B5gAF5?I?`OXpGJnjLBGx%{Yw9c#O{kOsGV5VkTiyCS!7@U`nQ9YNlaYrc-)5 z12ZxcGcyabG8?lq2XitPb1RRXm-(2V0W83REX2Yr!lEq3;w-_EDrJ{u8J1-^mS+W4 zWF=N+6;@?6Rkv%fCTp=a>##2Cu|6BHAsewVo3N>x+0EI4E!m2#*@kV|j_uij9ob2p z?Jn%fZtTt;?8#p2%|7hQe(cWy9H>F|U=HC>4&!i+;7E?*XpZ4nj?;L10w;13Cvys? zavG;|24`{>XLAncYMwox3%HPrxR^`0l*_oBE4Y%YwAx<7wOq&b+`x_8#Le8ot=z`# z+F|eHF7DgSPw_O*@GQ^qJTLGfFY&Uj*jIUt*Lj0C zd5gDshj)38_xXSi`ACoLCw$6he9jkq$ya>MH+;)?e6J7oM}FdGe&JVs<9GhxPyXU> z{^4K#lOM4L`mdbpe+Fg{24ye?X9$L5D28Sjg|)*mJR>k7BQY|gFe;-lI%6;%koI%l8f1zzMOUgi~Ec*GT0fJiJ6&&S(%O5nS(i*i@BMHd6`f7?En^FK^9_R7GY5qV{w*XNtRM+ zy9~>+9Luu;E3y(RvkI%S8mqGgYpRxAn{`;1^;n+`*pQ9bm`&J}&D7j(!Io^r)@;MJ zY{&NOz>e(1&g{aj>SlLm5B6j)_GTaUWk2@k01o6J4Yr4HD2H)4M{p!ZaWuzpEXQ#? zCvc)B*^@biQ#p;(IfFAfi?cb0b2(4*?FC%OMO@4!T*_r!&J|qARa~t#_FAsvdT!uG zZsKNc;Z|U62#@j@kL!eelBal@XLy$9c%Bz{k(YRx zS9q1zbltweo4m!_yu-V^$NPN1hkV4xdSXB2Gd|}FzT_*u<{Q4{JHF=!e&i>8w!iQz zzwtYN@F#!qH~;W2|Iv@v1N~RQ^*;kM2!k>hgEIs}G898I48t-U!z+Rvk&zggQ5coc z7@aW~ld%|^aTM2%$M{UZgiOT5Ov0p0#^g-FluBi%W*VktI;Lj^W@IL2W)@~;HfCoI z<+O7#H}fzr^D#dISbzmth=o~1MeSlN&JrxiQY_6fEX#5%&kC%_O029Zc2!nmb=F`_ z)?#heVO`c^eKt@-yAd0+37fJRo3jO5vK3pi4coFE+pB}!k)7C?UD%b~*quGtlfBrR zebm?P$Nn6^fgHra9KxX-#^D^nksQU*8e@;;IF9E8PUIv`<`holG*0IX&9rB6Hs^3I z=W#w4a3L3QF_&f-AX-tGR}2xsL0(fg8Dro4G|>?QPu79o)%X+|51Q%YEF> z13aih_F*33Q6A%Qp5RHI;%T1YS)Sv0UeHDR5-;-#uksqN^9FD77H{(o@9Lg?pAYzu zkNB8R_>|B1oGRc|0+8F z49p-5%3uu65DckMc4&rSScYSGMqornVq`{PR7O*DI|gGi7GpCG<1!xOGXWDa5fdwk zos`L#oGF-+shFB+n3n07o*9^tnUvYi!mP~3?99QO%*EWy!@SJL{0guOupkSuFpID# zi?KLMup~>dG|RB8%Gu>vffZSam05*VS&h|MgEd)8we32r%X+NO25iViY|JKX%4TfN z7Hp|jc5Aj_Tef3+c3?+#VrO72otoWf zb3D%ryvR$u%qzUgYrL)-_D$a6ZQkKs-s62f;6py*V?Nm-(2V0W83REX2Yr!lEq3;woX6WGR+r8J1-^mS+W4WF=N+ z6;-vXu{vw8CTp=a>##2Cu|6BHAsex=n%GU*jLq4CE!m2#*@kV|j_uh&9qmr+%r5N8 zZtTt;?8#p2%|7hQe(bLS_COBeU=HC>4&!i+;7E?*XpYfXdmP7e0w;13Cvys?avG;| z24`xPJ)3hlm-9HE3%HPrxR^`0l*_oBE40#H#noKHwOq&b+`x_8#Le8ot=eX9=ML`V zF7DoPg;#lv*Lj0Cd5gDs zhj)38_xV5%?MHmfCw$6he9jkq$ya>MH+-vi_IrNdM}FdGe&JVs<9GhxPyXU>{?T9i zA3d-x(0`TP|1&UyFermDI72WbLn*W!hG7|w;TeGu8Hte@g;5!e(G|mv$ykidIE>49 zjL!s2$V5!cBuvU=N^YlMN~U6JreRv9V|r#_MrLAWWwEm|8?!S9b21lmGY|7JAM-PS z1z1pp?7}RS>bJ{zzh z8?iB)uqm6dIa{zLTdB3(hHcr7?b(4H*@>Omg$qMU?2X*S&D_GR+{W$P!JXX2-Q2^y+^7Ba0UqQb9_A4q>9&1`cX^NZ`G61kh>!V%Px*|`^}>G1 zSA5Mke9L!y&ky{_PyEa;{K{|oZvWs<{^D=`;a~ovhgAmpuZr`}zzm|Gb}$BK2!>=R zhGrOsWjKas1V&UOJ2Im%Dx)zvV=yLTF*f5cF5@vi6DXmbh>4kmNtukvnSv>qim91~ zX_d}S&kW4SOw7zI%*t%c&K%6iT+Gcp%4_Fieg?1r3$hRkvj~f_7>l!nO4_AZnq^p) zau{Zm$FZ;1S2WX%@h=VzVLphAYIf5fOilaG(V>Qkm&k3B! zNu10noXTmO&KaD^S)9!|nrqMFd@kTZF5+S?;ZiQ+a<1S?t+H2h4cBrV*K-3mauYXm z3%7C`w{wSf+Pk=$d$^bTxSt1jkcW7fM|f1n?BhJalRU-KJj1g*$Md|vi@e0kyrQf2 zHD2cp-sCOb<{jSUJ>KU7KGY-oF`w`$pYb_g@FidIHQ(?p-|;;^=%f9KpZSGf`HkQC zgFpF;zxjuM_0RU8%0T~Bb^Xu448ouc#^4OWkPOAp3S)<5IEH5gMr0&LW)wzcG)89( z#$+tTw&O4^<1s!HFd-8$F_SPUlQFqc*eRKcshNgpnU3k1ff<>JnVE%InN8X49L&jF z%*{N^%Y4kw02W|D7E)om2#c~9i?akvvJ^|R49l_{%d-M2s*+uqRalkPSe-RkleJiz zby%16RNrpEhHS*fY{I5&#^!9nmTbk=Y{Rx{XSZhuc4Q}ZW*2s4H+E+a_GB;hw)?Oz z`>{UwtZb zhj^Grc$CL@oF{mar+AuYc$VjM-oC(#yu{1A!mGT->%766yv5tPW8dXH-sb~8tKLaxe zgDRLEoFN#Jp%|KB7?$A}o)H+4kr_e z5-iD5EUhwjS(amYR$xU|Vr5ogRaRql)=*8m7HhK(>#`o}vjH2j5gW4!o3a_3tA*W? zt=O7v*p}_so*meco!FUO)Yb0B?(D&y?8V;f!@lgt{v5!88e|XV5Dw)q4(AAt?yQj^_kTv?p;gr*JB#aXM#kCTDRr=Wwp(+4H%83%Q7kxr9r(jLW%#E4hlRxkhX4 zbzIL4+{jJb%q`r?ZQRZs+^JpmZtme;?&E$Q;6WbZVIJX89^-MI&`J9gPxB1V@*L0e z0x$9sFY^ko>Y9C>H+Yk`c$;^4m-l#|5BQLe_?S=h)PBb2e8HD|#n*hpw|vL<{J@X; zWPj!te&siQ=MVnmFaG8q{^dV;Kz*S9s=NMYUV$^He++PU`w`QYqnuqwo`k%13R)4JF^SBvKzaz z2Ya#?d#jJ#m;KnE12~X_IG95?l*2fjBRGd-r-%| z(|!8^AMz0&^9i5w8K3h7U-A`S>y7=E@A#e{_>rIZnP2#o-}s$B^wa*u-~7YB{6{aR z2=reK=bwQYgh3gM!5Km!?NAKOFbvCZ49^IR$ViOLD2%FTc67#IOvYkt#$jB>V|*rH zLMCEjCQ(v58Iv;wQ!*7(GY!)+9n&)dGb)pvnOT^X*_fRMDVAm#m9@*UJS(swE3q=GuqvyuI%}{dYq7TK*mYTt_1S<8*@%tV zgiYCu&DlaN?N)5fHf+mwY|jqt$WH9cF6^pqc6at*PxfMO_F-T4V}B0dKn~(y4$)A1 z7>9ENM{*QLa}39F9LIA4Cu)*CnNv8G(>R?oIFqwDn{zmq^EjUiw9sC}#azOrT*l>G z!IfOa)m+21T4%5425#ggZsrzlI(d1-64QD1$LLLog&mF*L(4 zEW2KI%B;ews%BSb4c25W)@B{nWj)qs12$wMHMX0uDVwo5Td*Ztu{GPUE!(lZI@lfA ziJjSnUD=J@*@HdVi@n*0ec4a_?ExIfK^)8>9LixF&Ji5RQ5>x?_E?VNcuwF%PU2)v z;Z#oJbk5*R&eCjq4(D9yYq*x{xSkuhk(;=gTey|m zwB6ppo!rIU+{3-x$NfCOgFM8;I$|H?F&^g$p5!T><{6&lIiBYQUgRZRwy*Fiukku> z@Fs8ZHt+B*@A1AK*bn)LkNJd8`Hau`f-m`sulYuA?RR|75B$ha{LC->%5VJ6ANLMGrO=WyRkcasHfeFz1fF-*^m7> zfCD**gE@plHOwB)5gf@;9L+Ht%W)jf37p7DoXjbjYER>I&frYW;%v_0T+ZWsF5p5f zvKMm+mvR}Ga|Ks&6<2c&*K!@#bAvY8o4A=fJjBC1 z!lOLK<2=EWJjK&IqqFuop63N#AC%aFZqhE`G#-#j_>(_ANh%&`GsHg&Hm0G{K;SZ%|HChfAm7CK>yWt{ux+7?4S(B z;0(c#48_n4!>|m;@QlESjHJkR6h>t1Y{k}W z!?tY4_Uyop?4-_i7j|Vgc4rUvWH0t+ANFNG_SXP=AO~?Uhj1u|aX3eCBu8;H$8api zX}mpw6FG^KIfYX>jng@UGdYX1HOHRId7RG$T*yUS%q3jPWn9h`T**~hZLi^4uH$-c z;6`rZW^UnDZsT_Cuy=A7cXJQ-av%5e01xsI5Az6*@|cd>CwP*lc$#N;mgjh$7kH7E zcv)BMtGveRyuq8i#oN5YyS&Hye4vN+BR=L6KIJn$=L^2%E57C%zU4c<*9ZF}Kk+la z@GHOZJAd#efAKf}=&${cUPc+{zdEk}8JIyBl))IBAsCXO7@A=e)(*$;jKGMD#K?@o zsEo$wjKP?SWyfY5#$`OlX96ZFsriJ z*_nemnTxrZhk2Qg`5C|hEXYDEtRi+%7GrUiU`du@X_jGGmScHVP(`~EE3*o#vKp(i z25YhwYqJjPs-9h+4cL&4*qBY&l+DCvh^TaH^)+(>a4PIg7J7 zhjTfP^SOWvxrmFoL`&^uT+S6-$yHpMs7>}}ZsAsL<96=gPVVAv?%`hU z<9;5{LHiI7^9Yaf7?1M=Px2H`^9;}GoPC}bc#)TQnOAs~*La;bc$2qu+rGoQyvO@| zz=wRq$9%%4e8%T|!IyevzvdghrD~cVJ(HNaE7?ZIWn{gPI@fe>8n2?E- z*iOQvOvdC)!IVtJ)J(&)Ovm)fU}t0|W@Z*iV>p)MIGz(Yk&`%?Q#h5= zIGr;z)1JlIoWr@C$N5~qg;$uGHQ$FK!zTiu~(rfz--}3(q-NUjd+ZIL9ec!Qd+qP}nwr$(C zZQHhO+qPdT^l0M;X2i-;zTbQGccnv*_oMzS(%O5nS(i*i@BMHd6`f7?E);wLM+T8EXram z&JrxiQY@`9c3GBVc~)RWR$^sVVO3URb=F`_)>3V|4(qZW>$3qHvJo4z37fJRo2!N0 zlC9X9ZP=FW*q$BOk)7C?UD%b~)ZOmEp6tcm?8Cn7$Nn6^fgHra8e$LSFb?Mkj^rqg z<`|CUIF9E8PUIv_wx@6^r*S%Ga3*JQHs^3I=W)Ik*bBLci@AhLxs1!Xf-AX-tGR}2 zxlZft4cy30+{`W9%5B`v9o)%X+^s$KUhd<59^gS9;$a@)Q6A%Qp3q7A6i@RE&+;74 z^8zpO5-;-#uksqN>xO-kw|JX(c$fEhpAYzukNB8R^wfUF=X}AJe8ty%!?%3L_x!+* z{KU`tVt?f~e&-MVCi2XQcma43gyI7e_KM{zXAXskVs<2iv7If;`wg;P0=(>a4P zHOrpOIh@ORoX-VZ$VFVtC0xp7T+S6*X|LjHuHjm)<9cr3MsDI}ZsAsKv$t~xcXAhZ za}W1&ANTVB5Au)>+edhm$9SA4c#@}hnrC>H=XjnMcu|+^%e=y?yvFOi!JE9r+q}cO zyr=v213u&9_rdfBBCA z4SoMNbpH&*zzo8m494IL!H^1Nhh`XtWjKas1V&^eMrIU7Wi&;%V=yLTF*f5cF5@vi z6EGnYF)@=csgl{rnSv>qim91~X_=1cnSmLZNtx{|%*t%c&K%6iT+Gcp%*%YtuL5>K z7Ghx*VNn)iah707mSSm^VOf?_dAkBDvJxw^3ahdjtFs1cvKDKrj$N1aSf35pkd4@w zP1uyp*qklclC9L*Zo{^0$M)>Nj_kzF?82_>#_sB2_hc{jW*_!tKlbMU4&)#X<`53$ zFb%gya3n`@G{5D)VRkMbCg^8`=w z6i@4neU|5Vo)>tLmw1_1c$L?9oi}vTzQxhK24-X?W@Z*ilteGWm%5p zS%DQ+JiWlLpYSfIGiImlA}19V>njh?D3qyiJZjA zoWiM`#_62FnViMhoTIt+JkI9=F61IE<`ORDGA`!|uGA`fHP>)0*Ks{Ja3eQyGq-Rn zw`se*gFCs4ySayZxsUsKfCqVqhk1lYb<94_6FkXNJk2va%X2)>3%tlnx@=$JRbJzD z-r!B%;%(mHUEbq;KHx(=vLEvapYj=>^95h>6<_lW-}0T_+aLImpZJ+y_?6%Ioj>@K zzxbPf_*ehzfW`#^{%h>}pMe;dK^T<57@Q#(lA#pZ4#Thv$MB56h>XO@jKZjl#^{W} zn2Ke`W*o+4JjQ1NCS)QeW)dc4G9|ZDFeOtlHPbLH(=k0WFe5WDv$EJ(nT^?*gE^Ut zxtWJ~nUDEdfCX7dh3z6N%3>_e5-iD5EX^`3%W^EQ3U)?yQj^_kU`FoP(l z9gM*lf*~1-p&5o@8IIu@fe{&rkrl;`%4m$v7>vnSjLkTV%Xo~>1WITpVqzv?QYK?^ zreI2@Vrr&gTBWnoGXpa+6EialvoagAGY4}r7jrX@^4j^Bp9NTug;ry!?G;L@~ps$ti;N!!m6ys>a3xfb}iOs9oA(%)@K7YWFt0a6E;;dyE$91C0nsI z+psO$u{}GmBRjD(yQr((josOUJ=u%B*@u1EkNr7-12xDV%pn}gVI0m89LZ4}%`qIy zaU9PHnrKhrWKQ8!PUCdW;7rcqY|i0a&9moo0T*%+7jp@hav7I%1y^#FR@-a1mg~5l z8@Q31xS3nHmD{+TJGfK3?A_eMz1+wBJivoI#KSzoqdcbL_6eTkDW2vTp5-~7=LKHm zC0^zgUez`GI&bhMZ}B$o@GkH1J|FNQAL+6Egira5&-sEc`HHXkhHv?f@A-ir^~wIs zFZ{}H{LUZz$zS}$!m&xrv*##oo$o+|C``$z9ydJ>1KE z+|L6#XdmKX9^p|Q<8hwgNuJ_qp5a-Z<9S`MFY*#E^9rx>8n5#PZ}Jvz^N#M?_jsQV z_>hnIm{0hW&-k1#_>!;qT5s&Pe8>0vz>oaI&-}u#{KoJ6p`Z3I{^lS4@iy=9F7NR^AMha`@iCw9sh-)-`GPO`im&;GZ~2bz z`GFt#NuTX6{K{|q&L8~AU;NEK{L6m~Xzu=-yMG0;12YJNG8lt11Vb_uLo*D+GMvKO z5g3t?7@1KRmC+cTF&LAv7+Z1dxQxg6Ou&Rp#KcU(q)f)-Ou>{)rPOvBre!*&X9i|u zCT3<9W@R>JR}MQTb1^sbFfa2lKMSxR3$ZYZuqcbExLtxJS&F4uhGkif84j-r{ZE z;a%S2eLb)r@(~~N37_&ApYsJ@@)ck6jo#Yt_?{p5k)QaPU-*^Z_?|f9${f z$AA{T|6BO}XCMY<5C&y124@I`v_mm8!!RtvF+3wMA|o*}qcEzX+0hw;F&T@o8HaHh zkMWs+37LqAnM6tLWK7N!OvzMC%`{BQbWG0-%&1IuW@celW@C2dU{2;@ZsuWL=3{;q zP(iy83$qA|vKWiA1WU3MOS25is+?V(6Ci2XQcm zaHxja!#RQ@If|n>hGRL7<2iv7IZ2c4DV)k_oX#1X$yuDuIh@ORoX-VZs73Z-F5yxx z<8rRxO0ME+uHjm)(|UUYH*ymB=Z%e=y?yvFOi!JE2e-{u|O8IYzJXb24irBU`U2y zXog`}hGTd|up=@OBQpx4G8&^Z24gZ7V>1rpGM?hw37C+Hn3zeJl*yQ!DVUO}m|AJ< zv`okJ%)pGy#LUdXtjxyj%)y+@rQCKN=4C$SX8{&uAr@v47G*IOR|&f$OR+S|uq?~5 zJS(swE3q=Guqvymx?O`cS&Ow2XYVxa|nlW7>9ENM{*QLbBxB?<2arZIFXY$ znNv8G(>R?oIFqwDTXXEWoX7cGz=d4I#azOrT*l>Gp_TS3uI3u9K7|_c5zm@kt12HgzFermDI72X`LfN4ihG7|w;TeGu8Hte@g;5!e(HTQA z?O2S>IE>49jL!s2$V5!cBuuJgc5Ho<8?!S9b21lm zGY|7JAM>*S3#yP^m_=BW#aNsrSdyh!nq^p)h=VzVLpe;t?GYTwQ5?-N9LsSW&k3B!Nu10noT_Q|bk5*R&f;v&;atw+d@kTZ zF4AIq372vimvaSIauru|4cBrV*K-3mYLmU0Tey|mxScz=le@T^d$^bTwBJ6!gFM8; zJi?+JiWlLpYSfIGiImlA}19V>p)M zI9?O%iJZjAoWiM`#_62FnViMhoTIt+JkI9=F61IE<`ORDGA`!|uH-7N)*5>)*Ks{J za3eQyGq-Rnw{bgnXs5l4ySayZxsUsKfCqVqhk1lYd5p((!am7UJk2va%X2)>3%tln zyv!@QYG31Z-r!B%;%(mHUEbq;KHx(>;$uCrpYj=>^95h>6<_lW-|`*b^MgLxpZJ+y z_?6%Ioj>@KzxbPf_*ehzfHvO$ZM^>(h=Cb|K^cs}8G<1hilG@sVeN1X&j^gjNQ}%V zjLK+?&KQiTSaxj2VO+*zd?sK*CSqbHVNxbza;8v9I~7wi4bw6m(=!7zG7~d13$rSl zot-(Dlew6id6<{^n4bk$kcC*7MO4%-#^Nl&k}Sp2EW@%a$MUSeimGH+W))UtHCAU0 z)?_W#W*ydLJ=SLfHMASCF`KX{o3S}tuq9iuHQTVQ+S%>dfgRb2o!Nz5*^S-VgFV?x zz3o2i%YN+70UXFd9Lymc%3&PN5ge&e_GpgbSdQa(PT)jN;$%+YR8G@$dj@B67H4w~ z=W-tBa{(7}5f^g_mui{4oGZAJtGJqLxR&d9!mZrK?cBkg+{NA8!@bfI%FT_5gz3+9_I<3V6l%p*L?V?53iI%%KcX`bO( zp5u95;6+~IWnSS`UgLG$uy67fZ}SfC@*eN=0Uz=aAM=Tx+Ryl$FZhzL_?mC{mhbqU zANWz9?9cqdul&aE{K236#ozqHzx>C5cJ5#QYv=vXzzo8m494IL!H^8a&$3qHs*&B8P1uyp*qklclC9X9ZP=FW)ZXsEj_kzF?82_> z#_sIFp6tcm?8Cn5XZPm-4&)#X<`53$Fb?Mkj^rqfw#RTR$8kI-a3UvhGN*7Vr*S%G zaHeM2vpI)zIgj(XfD5^Zi@AhLxlGIL6mff$%U7?i;nTp{d`48_n4!>|m;@QlESjKs){!l;a<=ynXoWGu#J9L8ll z#%BU1WFjV35<4lAF*#E(B~vjq(=aX5F+DRdBQq(porPJMjoF!lIhl*OnTL6qkNH)= zF33VG%pxqxVl2)QEXh(V%`z;@aw>0EU`1A9WmaKTR%3P6U`^IyZPl^svL5TR0UNRr z8?y^8dn%`KI%jYuXK^;?a4zR@J{M@Ay@-psgiE=M z%ejIpxr(c~hHJTw>$Sn&$W7eLE!@g&+|C``$z9ydJ=$yU<9;6CK_22^9^p|Q<8hwg zNuJ_qow3jI9MAItFY*#E^9rx>8n5$)ZrZnan|FAZ_jsQV_>hnIm{0gr&+O-X!Iyl+ z*L=gbe8>0vz>oaI&-|jV_BVd#5B}sY{^lS4m-(2V1z3=USeQjvl*L$_B~;Qb#nLRpvMk5) ztiXz_#LBF~s;XvJXARb5E!Jio)@41`X9G55BQ>_0uqm6dIa{zLTd_6Uur1rMJv*?Y zI@z7sg$sj9xRIN@iy=9F7NTa9@r20h>!V%Px*|``GPO`im&;GZ~0E|?GOCOPyEa;{K{|q&L8~A zU;M2<_Fw*EKu7QYj=ujHh=Cb|K^cs}8G<1hN}=sA49jo~&j^gjNQ}%VjLK+?t{8Sq z#$s&7VO+*zd?sK*CSqbHVNxbjaytc6G8I!Z4bw6m(=!7zG7~c^i=CC(n4LM8lew6i zd6<{^n4bk$kcCv(F2bTL#^Nl&k}Sp2EW@%a$MULRS7aqtW))UtHCAU0)?_W#W*ybF z>#;r?upt|l%p*L?V>)i1;7Ok1X`bO(p5u95;6+~I zWnHna@*1!625<5fZ}SfC@*eN=0Uz>_9@|g&l+XB_FZhzL_?mC{mhbppAMB6(#LxV~ zul&aE{K236#ozqHzx=0w&ffo>z5f}AffvnSifzYXT*hO3CSXD)Vqzv?QYK?^rLa>n6;m?}(=r{?GXpa+6Eialvof2q z+c}t%xtN=In3wsOp9NTug;-cc?4m5j;w-_EEXC3+!?G;L@~ohWb|qG36;@?6R%Z>? zWG&Wa9oA(%)>i|&AsewVo3JUHu{m3?C0nsI+o-MGj_uij9odPU*@a!%josOUJ=u%B z)yMA3e(cWy9LPZ&%pn}gVI0m88flN>XpZ4nj^lVv;6zU1WKQ8!PUCdVuxD}>XLAnc zavtY%0T*%+7juc0+RM0{E4Y%YxSDIYmg~5l8@Q31xLI55t=z`#+`*mP#ogS)z1+wB zJfMU2As*%t9_29}=Lw$VDW2vTp4B<~JTLGfFYz+3@G7tII&bhMZ}B$o=&pT__xXSi z`G}ACgira5&-sEc^~!$DH+;)?e9sU3$WQ#tFZ{}H{LUZxY5(GH{^4K#V?Y=8-^Kki z5Cby^gDRLEoFN#Jp%|KB7?$A}o)H+4krV$^HdAxE z1zWNeTeA(@vK`yA13R)4JF^SBs+--NJ=l}I*qeRWm;KnE12~X_G}s=(p&Z8H9Kn$s z#nBwYu^h+ooWO~iWKZT4PUSRC=M2u|EY9W}&gDGKw-<0B7jZF{a4DB@IahEcS8+Ah zaIMza>$!m&xrv*(g1KE+|L6%$U{8LBRtAuJgyV=NuJ_qp5a-Z z<9S}-MPA}%Ug1?<({=j>Z}Jvz^A7Lw9`Ex3AMz0&>xuo8&-k1#_>!;qns4})@A#e{ z_>rIV+5W<>{KoJ6!Jqua-~7YB{KtT<-v3>_{~3sZ8H7O@jKLX#AsLFH8HQmQPT}nc zjL1lg%qWb?XpGJnjLBGxtvGgE#$$XYU_vHhVkTiyCS!7@U`nP^YC8?nG9A-112Zxc zGcyabG8?lihnOmg$sj9xRIN@iy=9 zuI}0Q`G61kh>!V%Px*|``GPO`im&-bZ|!${&ky{_PyEa;{K{|q&L8}#U-obQ;a~n^ zKsWc_&HXbF12YJNG8lt1ghJY(7@A=imf;wl5g3t?7@1KRRnhF|jKP?U#n_C)xQxg6 zOu&Rp#KcUZq;@hUX9}idDyC){re!*&X9i|eCOb2;Fe|e$J9986b1^sbFfa2dzg>U@ zS%`&Mghg45#aV(SS&F4uhGkXGF3$?A$V#ltDy+(Ctj-#&$y%yy*I`}OV|_MYLpEY# zHepjXV{^7(OSQ6Fvklv_9ow@5JF*iyvkSYjo4VUQ*pt23n|;`q{n(!aIFN%lm_s;J z!|dT4!I2!r(Hz6E9LMpTz=@os$@UaZ*S3$hRkvj~f_7>lcfU6Q3(nq^p)au{Zm$FZ;2-2G|2Rh=VzVLphAYIf5fOilaG3W9@Mq&k3B!Nu10n zoXTmO&KaD^S)8po_FT^6d@kTZF5+S?;ZiQ+a<0%ydlgr64cBrV*K-3mauYXm3%6>U zy`4L_le@T^d$^bTxSt1jkcW7fM|9La#^XG}lRU-KJj1g*$Md|vi@IcA<`rJ$HD2cp z-sCOb<{jSUJ>KU7J+vS3F`w`$pYb_g@FidIHQ(^9-r4W@fgkyapZSGf`HkQCgFpF; zzxhXh?SBmD;rqXb?|%kjUW z7@Khzm+=^%37C+Hn3zeJRLSh*Ou>{)#nep0v`okJ%)pGyq|9~}W@R>JXAb6MF6L$) z=4C$SX8{&eA-gb(uqcbMI7_f3OR+S|uq?}|yj_76S&5Zdg;iON)meizS&OwMm%+4Il$z05>Ja%5@V}2H3K^9_R7GY5qV{w*HNxKwF zvkc3!9Luu;E3y(RvkI%S8mp^@U6Zv~n{`;1^;n+`*pQ9bm`&8wZpP+p!Io^r)@;MJ zY{&NOz>e(1&gx=!WjA(b5B6j)_GTaUWk2@k01dPUaWIE)D2H)4M{p!ZaWuzpEXQ%Y zCfE}>iIX{nQ#p;(IfFAfi?caLbM1MY&jnn_MO@4!T*_r!&J|p#RrYGG;aaZadT!uG zZsKNc;Z|U62#@NReViwFlBal@XLy$9c%Bz{k(YRx zS9I0B#_PPno4m!_yu-V^$NPN1hk9f`<`X{UGd|}FzT_*u<{Q4{JHF=!eY8LEGr#aF zzwtYN@F#!qH~;Xj{@DS&y#IT7|1%H+GYEq+7=tqeLoyUYGmOI8;TWC~7?F_}nNb*( z(HNaE7*nzA*o?!tjK}y)z=TZ1#7x4ZOs3>^3Z`T#re+$ZWjdy324-X?W@Z*_2 zb1)}!F*oxtFY_@!3$P#ysjyvyMOlo+S%M{5ilteGWm%5pS%DQ*$*#;QtjcPv&Kj)A zTCB}Ftjl_;Z#Q5=HezEoVN*6^bGBehwqk3xVOzDc+p_~ZvJ*SA3%jx#yR!#-vX^?> zeb|@%*q;M9kb^jwLpYSfIGiImQlsqA9K*33$MKxNiJZjAoWiM`rs?(!&g3l4<{ZxD zJkI9=F61IE))IRumvK2)a3xo9HP>)0*Ks{Ja3eQqv%Q5|xsBVogFCs4ySayZxsUsG zz&^-BJj^3J%40mv6FkXNJk2va%X2z!U*JVv;$>dpRbJzD-r!B%;%(ir@A4k+^8p|7 z5g+pjpYj=>^95h>m0sI#_?GYZo*(#;pZJ+y_?6%IT|exf{Ken=!@vB;fZp!ExBF)x z24)ZjWiSP|Log&mF*L(4EWSA^Roa8vJeZi2#c~9 zi?akvvJ^|Jj9r%HSe_MFk(F4PRalkPSe-Rg)2_wZti!sj$NFr*hHS*fY{I5&#^!2a zw`41}W*fF;JGN&Bc4Q}ZW*2p}yRkcauqS)5H~X+J`>{UNxR?95p9gr5hj^Grc$CL@oF{bBKE=~K!?Qfc z^Sr=|yu{1A!mGMwU*`?pGY!)+9n&)dGcpr1 zGYhjao3h(En3K7fn|YX*`Iw&tSdfKSm_=Ar#q8oN!ICV+(k#QWEXVS!z>2J-%61i2 zWi?i34c25W)@B{nWj)qs12$A6yD^)vDVwo5Td*Ztu{GPUE!(NR-GLq1iJjSnUD=J@ z*@HdVi@n*0ebvwI&jB3BK^)8>9LixF&Ji5RQ5tQJ;aHC2cuwF%PU2)v;Z#oJbj`45 zau#QE4(D9yYq*x{xSkuhk(;<#TkNge#_im}o!rIU z+{3-x$NfCOgFK|e_7NWCF&^g$p5!T><{6&lIiA-A`ywy#GOzF|ukku>@Fs8ZHt+B* z@9Dn%fDieIkNJd8`Hau`f-m`sul2@$%XfUw5B$ha{LC->%5VJ6AN5*dZB;p&5o@8IIu@fe{&rkr_o%?P!e77>vnSjLkTV%Xo~> z1Wd?8Osph!QYK?^reI2@Vrr&gTBc)qW>7{u6EialvoagAGY4}r7jrWY^D-avtAJgQ zg;a4++gEAO{GXz626hku% z!!n%0+YuO%krbd-r-%|<9$7_AMz0&^9i5w8K3h7U-A`S|DS>T463$(1#cL^bc z5JCtcgdid!A|fIpq9P(9A|fIpA|iHu?Y;Njd+)vX-h1!8?=@%c`9I&U55Jky+^^kl z+;8=r=kMJg+#lVa+@IZF++W?_+~3_l+&|sF^tb1K+<)D0L1DpqrT;@=!3iq*kA(%N zsr0`oEI3CMg@pwdsVY!daD{4vg$38CAyim!o0`Lg1^1{eQdsbiI-`XJPpBtWSn!a~cQ&{km7PEx~e`qCFSWu8JEI3%f|5R9T zq>@F21;;7>-xd~}qLSjmg0ocq-xU^IpvscMg3DC%-xn5Kqx#aqf?L$|KNJ?+rPi{- zf(O*`KNc1|rtb2>f@jqCp9>3K(qKhl!5bR+p9%}!)A+i=f=@K{UkVGp(rjg6!4F#a zp9>3q({fc|!9QC2UkVEfs|yPbSNwl1EI3Bln!St%U{u@&7L@I8@;_=1OnCTt)57 zRk|T_Rdg^{)ke%!+sRxF8#7mP7jw03!d#u*%+<3gbM^Ny*U)CnHQLKu6Pq*FbRTog zZNXfN{miwpCG!F13U0+*$wB7IZ_QjKL(Ema4RcivGgr;F%vC?aTus|CSL^o7)v*I} zb&oPv-;T^RIL2HfJ2BVzICD+y%v`e*%(bu!b1hFY*V?YkrtnSIKG2Rem~i zRi43IHD@wc{aMV_bT)Ifp2J)n=Q3CKdCb*!K64FTz+592GS~P;%r$i}bIo4DTnm>n z*Yah|wRSo4E0`;OC39u3Vy@z=nXBv?=2~|xb5&o*Ty@toSK|%L)p8?qwco^CT{kmV z?=8$Va4T~S-^N^Hw=>t|9n3XzCv(l;#av5wGuP@p%E02brtl zA?B)jn7L{nVXlTpnXCCR=4yMKxjLUiD#K>`Z?yBd!D%# zUtq447n#4rT)~%_EBOj@n*D^i7CvRJ<T*Y5ASJ^kr zweDNys{W3->b_^L#vhogSD zxt9KBuGN2-|I1wAf`B~_*zkJ`ZqHmDJ1|%GD0B7g$XtVC%r&wTbB&KP*VN9;H9Nsv3%fAa z@+5Pu?aF+Lx#GJqS9Y4Yig#zOvYCKAXTAq>RnIb4-JZZ zLzt`fQ08hljJcW*XRfv*n5*+h=IS|$x%$^L*U-_-HF^wlO&rTy)5kH_-0{q{cmi{+ zoXGqn<_ez7T**_ID}O3;m7K<0<)<@Og1O>XGFSF0<|@9Lxyr5y*mLIB zGFSC=%vE;SNl!O)pav-_1?l<1Gh5Q@NLXBb~|%T-oacmcQV)fUCgy~ zH*>At!~9<63g5?E>HC?h=mF*`eUP~-9%8PlhncJP5$0-ml)0K8W3INxnXB^&=IVKp zx%!`CuA!%yYxEiBns}DErk`W3x#yW{@df5ud6D@`%oTi@xstCiSN>JzDtV2$%3o)$ z$~Ty+=1u0Re~Y=A-e#`WcbKc=UFPb3kGcBZXRg5ym}}%i<{JNqxu!m5uGvqRYvEJo zTK2Ky*{fGI#%oQ#O+Vh}258CsfJrCOR zpgj-T^PoKs7F?vNK+v8C?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhF zdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*# zo(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B* z?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD z(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6i z2km*#o(JuD(4GhFdC;B*?Rn6i2km*#o(JuD(4GhFdC;B*?Rn6iGgo>8<|=AuuF?&e ztD=Lssy1S-+D_(b*qFJRyO^tO6XxpdX0D!1nXA8txrR1luF+oRn%JDVru&#{ZVTpG z>}Rf(EtwB6S8yxlN)9qterx6`8Dg&TZJ4Wan7L}UWv==W=4#rGxmvepu8tj;t9z8W z`gUZl!7=6<*@?Nv$C+ztXXcunV6KH-m}_~Gxz=`NKE+(|-Iyyo&0NL1GgsLRbFJHh zxvFQGt8P!`YMf)Pmc4@ZocTO+b?wbuy$j4Wun%($FEZEIzRWeb#9TA`G1vSub1m)9 zT&pY04`8nFDs!a|WUit$<|;jixhf83uBtb#$4s6Ggsvq z%vEzHbJd^4Tuo;)SL-><)p0I!b)UyvedjaR;04Szav^h#U&LHf7c zg}DZ9Wv=1dm}~5I=9;{Nxn}NUuKBx|Yw2$0TD^z)z04K9kGazKGgr|A%vJgzb5%UV zTvZPuHmAQt0W3I8^nQQV7=9>AFx#s_3 zuBE@3YxN)I|1wv&AY{)&_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tP zJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm z&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9 z_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+ zWY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?I zL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tPJY>&9_B>?IL-ssm&qMY+WY0tP zJY>&9_B>?IL-ssm&zUQ|0dp0#Ggs+`%vI6BTvZz}S8XSAHEhgW&0WmZwh41}b~9Jc zrp(pf!(2m~G1q7>b4_f{T+@BbHMa$GE%r0l%9hLrm@Bvyb0r6vE59{!l?*Xg`8Lc| zIm}!&+cH=E2y-=U$6T%3GgrqB%+)>0Tzxw-*Weg)jqDV%=gh~MYieianw?;-gfw@*r zWPTEJ1y5$KpMnXC02=IS_?xw_9|uDg1O>XGFSF0<|@9Lxyr6#u65Tk zSM_zwRd+peHQvBnEjKb(`%TQ%bu)AI-oji1w=&o8ZOk=xJ9ACm!CW(UGS~cF%(Zkk zbFJRP{9fh?-^X0(`!(nXC2@=4yD9xtbqiuC~XStMduw>Uom6 z`k!L1p{JQ^^cm)wc$T@QpJT4M=b3Bq1?F0Lk@-u^6?~bwlCLmV{#E8Gd5yWsUuUk$ zH<+vDP3Ee9i@BQKX0FzEn5*Mm=IVZrx%%E`uE7tOYvjX_J!k$Ab4`8BT(h4r*TSdF zwfq@#t$oh?3+9S{$z0j5n5+0}<|_M!xz>HlT-Dz(SKar_)%XK*wfx9j?LRSB*U!w= z`wMdo{K{OzzcJU?@60v%2XoE*$z1b)G1t=H%(eOt^M9Eu?6qF${}8t4VS667=V5yu zw&!7c9=7LUdmgsuVS667=V5yuw&!7c9=7LUdmgsuVS667=V5yuw&!7c9=7LUdmgsu zVS64f_&^h>ussjk^RPV++w-tJ58LywJrCRSussjkbMGMhr?5Q_+w-tJ58LywJrCRS zussjk^RPV++w-tJ58LywJrCRSussjk^RPV++w-tJ58LywJrCRSussjk^RPV++w-tJ z58LywJrCRSaKR^<`Y&O79=7LUdmgsuVS667=V5yuw&%WuP#w1CVS667=V5yuw&!7c z9=7LUdmgsuVS667=V5yuw&!7c9=7LUdmgsuVS667=V5yuw&!7c9=7LUdmgsuVS667 z=V5yuw&!7c9=7LUdmgsuVS667=V5yuw&!7c9=7LUdmgsuVS7%zjk(erFjrAKbCqt$ zTooP6Rkaaw)pjyh!^X_j+{IjNn=n^rH*@uD%3S?D%r&$bbB*>g*Tm+`HQmQtb6YUi zVn1`OY{`6pxq@3US8|ZK@>?@k$q;juZ^K-b!^~B)Epye6Fjv!d%+F^C{+v@5Wr&Y33^4ow>?pm}}i0 z%vC+hTy=XgSK}OWwd}=Q?eomlwKsG1E-=@?KFl?|$XsLlGS}o1bIt6>T=UD!wX{ET zt*$UXfVsk}%#}Wnxr)}9tMnk|syLXrst#eU+C!PE;V|ZEKAgGQj$p3NBblq`DCX*4 z&s;-CGuP-b%r$W=b4?$|Tyw`W*WwAxwQ?f!lb9=bGIJ$QVXpkC%vEw4bCsXYT$N`q zSIwEsReu(9HJ!~|t>-XT$GObaeI9f5ozGl@7ckezh0Ha65pzvl%v`gVFxSGR%(Z+O zbFE#@{0ioZU&&nAtC*|!YUV1thPl>V%UsphF<0I7%++`UbG6*aTE2Du121D&JtPnm3uN{w?NedYidg-(jwfcbTjEJ?83r zpScDDLxn@6Mu7yvTYxy(gTKk;&7t9s^lDV>9F<0@|%vJUcbFKT9 zxvIZouDb7;tMLcsYWb15+J9oMuAiB!_ZQ|G_?5Yae`Bt(-mX znQQeQ=KnHRxFBNBBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+G zBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP? zJYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE z&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry z_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;Cc zV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+GBlbLE&m;CcV$UP?JYvry_B>+G zBlbLE&m;CcV$Yc?y#aF-wKG@ghRjvb!CX}vF;{ITb2V(tT+Lm~)wT(9b#^mX&!)`P z-@{x(n=#jDFLO<7&Ro-d%r&}vAnJd3FbCnD+SNS%~RXNOD zHQO>*{Rne4ZO2@#+cQ_k4$Rd(%3OUrGS}c3bB*l8T;t=+HMKKy%}y}a!Y<6UJjq;Z zyE30*uJ~@um7Qj;;@z36Y=*hk?ZI5tv&>buCv!EVy=pVnXBp$=BhoE zxf%{*uI9s;tL+Hp>O7LUdX8eQ{`Jf?bTo5~9>ZJ{$1>OSam+P$Jaa9cz+5XQGCzsA zf+sUq@)YLEpUPY%r!iOg>C9Dm26NS%$z1hkF;~;s%+-1hb9J1{T;1m}SKs-}HFyDY zja*=Bjy< zx$56yuBNw{tMwh`>UfvAy5D22zW14H@B`)=`H;EBKVq(_kC|)s6Xsg@l)08aW3IK& znSa4t@h_Px`xSE)f6ZKF-!Rv@Z<(w5JLanUp1B%-V6K)QnXCOL=IZ*Hxq5$Lu7O{f zYxpCvL*8Y z<_d1bT**P^%5TkFB}2?rz72C#4l`HHw#-#O!dy+;F<0yM%+;|2b9Ik0SKp4zH8{px zBRetI_&9S-?aW-W6U?=+3v(?`GS}L!%%_+uz8iC8r zGFSRQ<|}pc!dwfNGS~8D%(Zqo^DCGu zekF5duVSv^tC_3p8s=JeEpt_0$6R&SGgspc%++!ubG6^ZTwOOaSMM#%HE=6)4d2FG zW4ANcd4b0>4n-^E-@cQe=OJ22m}eTTU^-es=t_n52iedZecfVoCK zWUld#m}}}|=9>M4xfVWUuI0~|YwdI9UocnvOXkXc#azW-GgsL+%(d=Y=BoaVx$3@W zuErmjtK~=LYX6D3x_)M^-d~t&;8*4v{*AfDerK-9KbULgPv)Bci@BEmX0FwLnE%UM z;ewbwkJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P* zm_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^ zkJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i# z^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJ zJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4 z+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P* zSV6drxzZajS5Z52m2Svf6&=h~wGngGb~0DP#>~~+#awNhFjr?cbMT>U-FHMALX zjrKCv#OBO3-N#&WTQJvRKXa{Y$$Wshf?F|Ha*(<5TQgV55ObAp!(5fa%vG~3bJdS9 zSJQUP)w(@%b?m@g-J{Iawb;~aCf?8RK|^UT$?H*@taFxS97%r(5oTx0t( z*W?m&&Fsfq^UKV&v_EsLt}s7j^Lz%1LFy?AL zoVnVLV6M(1nXBh0=IUS1Tti1Q*XS|KHE}F+O&`ZxbH_8+;t9;Paw7ASm@9ZPb0tq< zuKcOYRdO10m7mUBm1i(l&6&(qe-?8!oy}aW=P*~txy;pl9&`1b&s>8SFxSY1%r$-y zb4^{$T(g%j*TSXDwR{qd>?bA z?`N)}2bin$LFTG>h`FjBX0F;tn5*GY=4yV7x!N9QuFfZztLI7P>VJy4hMs1w(Px-z z;#uaJevY~3o@cJb7np11MdmLtSMX)#O1{Ee`B#~%Yt^1a_s=s5dy6>5*@dxH=`H{KWe`2n#pP8%o7v>uHmAQt0W3I8^ znQQV7=9>AFx#s_3uBE@3YxN)I|1y_P1?+j;p2zKZ+@8nndEB1I?Rngu$L)FCp2zKZ z+@8nndEB1I?Rngu$L)FCp2zKZ+@8nndEB1I?Rngu$L)FCp2zKZ+@8nndEB1I?Rngu z$L)FCp2zKZ+@8nndEB1I?Rngu$L)FCp2zKZ+@8nndEB1I?Rngu$L)FCp2zKZ+@8nn zdEB1I?Rngu$L)FCp2zKZ+@8nndEB1I3%bkW_B?LS}vA znJd3FbCnD+SNS%~RXNODHQO>*{Rne4ZO2@#+cQ_k4$Rd(%3OUrGS}c3bB*l8T;t=+ zHMKKy%}y}a!Y<6UJjq;ZyE30*uJ~@um7Qj;;@z36Y=*hk?ZI5tv&>buCv!EVy=pVnXBp$=BhoExf%{*uI9s;tL+Hp>O7LUdX8eQ{`Jf?bTo5~9>ZJ{$1>OS zam+P$Jaa9cz+5XQGCzsAf+sUq@)YLEpUPY%r!iOg>C9Dm26NS%$z1hkF;~;s%+-1h zb9J1{T;1m}SKs-}HFyDYja*=BjyUfvAy5D22zW14H@B`)=`H;EBKVq(_ zkC|)s6Xsg@l)08aW3IK&nSa4t@h_Px`xSE)f6ZKF-!Rv@Z<(w5JLanUp1B%-V6K)Q znXCOL=IZ*Hxq5$Lu7O{fYxp1n<{H|Jxkh`LYhrWe zn(kw+xh=wq`K_6&WQe)Sw_&czVdkpYmbvOjn5$_!=4#!Z zxjJ@WuI^Fh>f4dI2FI9dWGChtA7`$qotbNPg1HuUVXoy#=33j8`4n@-cVn*XG;yuJ)Ul ztLtXw>b-@z25x1p;oF#N>~`jwyo0%B?qsg{yO?X~ZsuCOhxxtC6~2$T()Tl0(F4p? z`XF;vJj7g84>MQoBh1zCD04MG#$0WWGgs#m%+>QGbM-&PTtiPY*XT3MHSsKSO+Uw6 zbI&u^;tR~R@*?w>m@D`)b0uG4uKcUaRq`5hmA}qhm2WUt&6~_s{}yvKz0F*$?=V-# zyUf-79&`1*&s>8aFxSY3%r*WIb4`8BT(h4r*TSdFwfq@#t$oh?3+9S{$z0j5n5+0} z<|_M!xz>HlT-Dz(SKar_)%XK*wfx9j?LRSB*U!w=`wMdo{K{OzzcJU?@60v%2XoE* z$z1b)G1t=H%(eOt^M9GkHw*1~(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCk zC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;n zdD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&IC zo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI z?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j z(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCkC+&ICo+s^j(w-;ndD5OI?RnCk zC+&ICo+s^j(w-;ndD5OI?KyL$H(;)!cIGPGkhv;4n5$|d=Bn*vu7-`7tGSE0+BRXX z&Ti)F*_65ZdzfozGv*rYWv+?MnQOX_x#qTDuEl=lTG^8M0CNSmVy@&MbLF>Yu96|< zD&K~=Du4Cz)$)SLRd972l1yveV2}ygPH1%`n%xJ(#O{mbvQoWUj_J=4#oCx!UKMt7~uO z>Rn*2fqj^3c#*lr_GPZgCFYvhkGbZTnQLi(=2~50egJcYSD7n)AafP1F<0q9%vEtP zb5$L}T(yTXSHoe<)qFT}wH?7+okucP&r!_Pzn-~%@MPvnp2A%DQ<N}sg1}|W)kqen?{37O>x|q3UFJZ2QOPOo=GUi&locR^Z6~B_XvR5%z@zu;#b`5i_ zyOz1CuVb#d>zS+Z2IgwHk-6G$Vy>>6nXC5}<{G$_xrT3JuCd#hYw`}}nz@s?=I>&z zrMsDH^&aN;GFSLM=1SksTtyEsSLuVyRq+sWRXxmHwU01Y!=udA{1|h!J}p+!dwfV zGS~8F%(eD8^Dmey{v~r|zhbW9ubHdt8|GT~Ept_W$6R&aGgspe%+>NEbG84(TwOmi zSMM*(HSjBQ4gbbmW4|-k2zWKg|DSu5dxho~P`2%ATj}dCH!r z?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2 z%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$b zr|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj} zdCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyk zo~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCH!r z?0L$br|fyko~P`2%ATj}dCH!r?0L$br|fyko~P`2%ATj}dCHzMS9$~HDr#r0(hZrb zqJz1rHe#;YPUdRZn7Nv}n5%6Q=IZQbuAWVqtG|c2hBjla(O%}7*qphh`G3FZCiMhtdnQLli=9-;gu7zEgYk88n)^=q+#a!{-m@7NYT*bRHSJ@16t=ogS zs%M$2ZcpZFoMWz*y_liP~2QydIAT>a~r zYv^d^8a;-&CXQvU>EoDd?s(=}Jb}4ZPGo)(a|KUkuH-4ql|PlaN={?0^3$2C@(kvx zIg`2S&tk5ovze>)9Omjcm$|ynW3ImQnQQO@<{G(>xyCPIuBnTeYxWZ6TDX+CmM>$j zwab}b!CdhxnJar0a}{6BTxHiV*Sc$&tNJ?Ts=J=K8gF2(mK&L?{U+w>x|z9pZ(**1 zTbXP4Hs%_;ow+9OV6K@vnQQ(o=32U&xmNFCelK%{?_;j?{mfPL0CSZ-$XpcKOb9KMRTz&5| z*Wd@tHS!^Ijeo>kQy(+e>?h2%@F{aGf5u#EpELi0x#C|kSN1FBD*l?e%D!Q)b>A{q z^>@rw_dRnp{=i%;!Wy?802jlgzcY zEAuJlitol;*=gn~-krJ1W|(W;9?Vre%UpGPGFRgqbG7WnTE&xG0Zh_EOSjC$6RyAGuPq?%(ZeN^OKk> zcrtS(Phqb7smxV!8grGO&RmseFjvi)%vFCDb2XjKT&?FYSI4=`)qNgw^_|aLgBLK@ z$c4-`ei3s`UCdmwmoV4DrOdT_8FQ^&&io4IieJfG*{hhV_-f`VyN0>eUCUh6*D+V! z^~}|H19P?9$Xx9=F;~~k%+-4fa}C_eT*J39*VygMHF*bf&D_ac^LH`V(%sCpdJpq^ znJauBbEWTRuA&E+tMozUs(6UGsvc&p+DDkH;Zf#levG-=9%ru3Czz|}N#^Q*in)fK zX0Fj^m}}x$=9+$vx#pf{uEiIaYvo1eFELl}W#&r0!d&@RnXBYA<|==kxhmgau9`QQ ztNtzKYI>WwTHj%=j(3@>`#t9Bd!M-mKVYtr51DKHBj%d=n7L*@VXlQwnQQqo=34ul z`4`L;|B|_~Uoltl*UVM+4Rfvgmbt3GW3IaInXB;!=4$zox!QkXuCAY%tM?b?8u*pD zhJRzOvEP|%@(<>k`IEWk|6;DCznN?GALjouSGXW!&olNsW6v}8JY&x@_B>4wZz(ZO6*8!=aH zCv!Dy%v{Y~%+O-GuO(N%mwUoXpOl_4`Qy0gPE)95ay~ql(`xXW3J}InXByx=IT6>xq6ObuKxARHFPv{jUK~X z6UQ>w^l{8JcRX_~p1@oyCo(^Yxq>G%SMn6*%Ad+yC8sf0`RUA6c?NUUoXK4EXE9gP z+04~?4s&&!%Us>(F<0OD%r$rcbB$ccT;mrp*VM(#HG2tjEnLc6%a<|N+U3lzV6OO; z%$2>0xr(o5uCi;GYu&ZXRec?E)m_hAjW;k?%Z<#{eiL(b-OOCQw=mbht;{ui8*`1_ z&RmmsFxSkT%r$=(b1mJ?T&wpmzn8hf_c2%ce&#BAfVoN^WUh*bn5*hx=Bj;!xf&j2 zuI9&>tL<^->U@H^dY)vi{->C0=xOE}eTKOvo@K7-=a_5mdFEPtfw@**Wd0I!1z%>a zpQnXC03=IVHtxw_wDuD_>{SpKVz=7&zXP0T=6fNEBh656@SfKW#2H@x^J1Q`a9;T`<}TP ze_*bbADOHDC+6z1n<{H|Jxkh`L zYhrWen(kw+xh=wq`K_6&WQe)Sw_&czVdkpYmbvOjn5$_! z=4#!ZxjJ@WuI^Fh>f4dI2FI9dWGChtA7`$qotbNPg1HuUVXoy#=33j8`4n@-cVn*X zG;y zuJ)UltLtXw>b-@z25x1p;oF#N>~`jwyo0%B?qsg{yO?X~ZsuCOhxxtC6~2$T()Tl0 z(F4p?`XF;vJj7g84>MQoBh1zCD04MG#$0WWGgs#m%+>QGbM-&PTtiPY*XT3MHSsKS zO+Uw6bI&u^;tR~R@*?w>m@D`)b0uG4uKcUaRq`5hmA}qhm2WUt&6~_s{}yvKz0F*$ z?=V-#yUf-79&`1*&s>8aFxSY3%r*WIb4`8BT(h4r*TSdFwfq@#t$oh?3+9S{$z0j5 zn5+0}<|_M!xz>HlT-Dz(SKar_)%XK*wfx9j?LRSB*U!w=`wMdo{K{OzzcJU?@60v% z2XoE*$z1b)G1t=H%(eOt^M9EuT#&QpIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?x zv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoB zIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T` zp0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7 z=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npU zd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?xv*$T`p0npUd!DoBIeVV7=Q(?x zv*$T`p0npUd!DoBIeVV7=Q(?xv**l}-hjD^+L^0#L*}aJV6Li-n5(vvxf(WRuI4W0 zYTJalI=h*xXH(|t?_sW?&6sPnm$@c3XRhf!=9=4rxfc7GYh_F31I!iNin)@5%$47o zxk`qZt9%>gsvKsnnr)e@euTN2wqvf=?U}1%2j=P?Wv;#*nQL&2xkh$kuJLi^n%bGU zW+#|yVHf6Fo@B1IU71fYS9~|-%1$#^@$Sr3Hp5)&_F%5+S>~$Slerq_n5$(k=4zj3 zuCBeAt9OC92KHgD;YH>e+n2c}mzZm2KjxZWX0E0EnQL{0`2oxoUS+QIfy`C3#$2Tb zF;~UG%vE&=bJZToTn&dYSM%Y_)pi7PbsouFJx4KD|9a*cI-0pgk72HfW0`CEIOdu= zp1BrJV6K%DnV-a5!IPOQc?xsoPi3x>)0nILbmpo&gSl$XWUl(Nn5*e*=4w5MxjN2e zuI}@gtM7c~8oYqHMlNKo@r#&i>SE@ay@a_IE@iIe%b08Ja^_bsSNuxm%3j4>#aA;| z*)`0y?po%mzK*%-u4k^s83GS~ET%r*Bsb1lBWTq`d!e~Gz*FEdy273Rvn%3LL{ zF<1HP%vJdYbJe`bT=j1;SJT_f)%p%|b-c@5-S071-}}rp_yKc`e8^nmA2HX|$ILbR z33Dxc%3RBzG1uDX%)el+_?OI;{ffDYzhc^Fjvcu%+>xA zb9Md9T)n?A*TApLHT)ZMjs4DClYcPR%%99P{}*#D{mopf|1kfTx%^|UJTd+)vX-h1y|UwiMp_x^8# zeR55=D>;T|g*~sZ=N0z6!k$;y^9p-jVb3ei@Dl1WvQEz zhcZ{~Va(NVICC{0!CY-eGFRtO%++%=bM+s?Ttmk)*XVJ~HE}$1O`pJAb0;#_;z`W4 zax(K%m}}js%vE|Cb5)$qT*)(-D>#$6D$ioBnzNa!{v76NI+wXx&ttBR^O>vr0_N(w zkhumgVy=;knQQzK=9;>cxn?h8u7%5)YxxT1TDy|@Rm@d#HFK3;!(8!enJar8a}}>= zuId|@tL{eTYP^ZLT5e{p_FI^%>sIFKy^XmBZfCCHJD6+iPUf1ti@9d*X0G{rm}}`? z=32dv`Tfkb{sHDHdyu)J4>4EzVde@SVXmr2nXC3O=4yDHxtgC~uC^zctMe)5>Uo;E z`k!I0p=X(E^f~65c%HeYUtq4e7ny7ECFWXrnfd>iYuzi%Rr)G(RlLSr$=8`Hc!Rkr z-(;?ux0tK`ZRTovhq+qcWv-6*n5+AJ=IZ-^xduOEu91(JYy4y8n)-yfWhGDW?g!>-{E@j@eqyfnpP8%c7v}2y zmAM9fW3J)fnQQD1=9>JIxn}-iuKB;2Yv~{6TK$*#f6TRhgNQwk*z<@zkJ$5wJ&)M) zh&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@z zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k z^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5w zJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l< z*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ&)M) zh&_+k^N2l<*z<@zkJ$5wJ&)M)h&_+k^N2l<*z<@zkJ$5wJ!h`64Vf$2&Rpq@m@DjH zuBwfhtG1K58a83B<}T)H+myLFyP2zJGv?~=VXmRgnQOF{xhA$?uIWDJn%k1O7W?b%(ZT7<|-Xzu8M7#D>=km!M4m*Im}!&+c8)D2y-=U&s?oLFjvQp%+)>0Tzxw+ z*Weg)jqJ=^RUOJ)wTCfR!{N-;d<1i~9m!msM=@8=(ahC<408<~ z%Uq+!G1tWL%r$)ibIqN|T#F|$*UHJvPhqZgr!rURY0On|I&&q@V6Nay=Bhl4xoXa4 zuKIJBtLa?kYCVs+I?iXV?hBZ!??UDpyok9*E@rOrOPFiwQs$byjJXyrXRhTdm}~7y z=2tOS$<@qNehqWQuVt?6b<9<~p1G=TV6M6wnXB<8=4!c_x!P}GuC7~|tM@kM8n~Ug zhVNjmu{)V-@-F6@xtqD>?_sW`dzowXKIZo`*ZK#TtL#DMiax|#>4%vse1y5G9%ZiD z$C#_(apr1%g1OqBWUkJqn5*Y$=IVcjxrUx)uF>b1YvOt4ntp+~=3Zp3#g~|C@=^f^3eV4g9-ea!r_nE8j1Lhk1 zkhw-aVy^LznQQ73=9>MKxfVWSuI0~}YwZi>Uouz8SIkxZHFL$kVXo}A%vJo3xvIZs zuDTzXtMNzXYWaz|+J9!Qu3wm|_gCf`_>H-Se`l_-KbULsPv)BWi@E0iX0D}wm}~W4 z=KnF5t2_2QYR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wg zJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*% z&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ z_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{ zYR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9O zqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wgJZjIQ_B?9OqxL*%&!hG{YR{wg zJZjIQ_MExOHe{}7J9DKsVy>`*xvDm1uG&uKYS@Iin!A{*ZBypz>}IZ>&6umdhq;C} zXRgs+=9<`oxu*M=Yi>*CTI^@8m93Z$FxR@RnX7b=xhl3{uH+DN1=})LrWeM>ALdG0Zh|EOU(>$6OP~GuQM9%r$o+b1k04Tq`FtKZUv0oyuIL zr!iN>>CBZpgSmn;nXB?F=BhcHx$4hhuBLODtMxqQ>Nuadx-Vd^z6+Ua@FM0KxtO`e zFJZ2!OPOo-GUi&ioVk{-V6L?*nP0_RC08?7`8CWHzm~bO*D+V|dgiLWfw}5#WUj`W zn5*Sx=4!u%xw>v;uHM_2Yv6X~8oqR2nXCU9<{EmIxkjI3 zu8HTFYx)J|ntPGC7GGknm6w_SkGa;p!d#`VGFQcG%$0ndxq>&CtMX0es(FjK>fdIr zrgxaD^oJ&)P*m_3i#^O!x4 z+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P* zm_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^k9m&S^O!x4+4Gn^kJJ&)P* zm_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^ zkJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i# z^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_3i#^O!x4+4Gn^kJ zJ&)P*m_3i#^O!x4+4Gn^kJJ&)P*m_28%vJIIl+Rj|*jhHL!V6Li-nX9&wxf(WM zuI4W0YTJ~#I=h*xXEWyN?_sW?&6#Vom$@dkV6N#t=9=4*xfc7GYh^3u1I)Ee-J1|$rj?C3P%3OUrG1uT2bB*lGT;t=+ zHMI+K%}y}a!miA1oVWaXND)&tR_LOy;UQi@9pfX0G~kn5*er=4w5U zxjN2guI>w%tM5YQ8oY?PMlNQq@k^L%>Qd&Ky^OgQE@!UgE0}BTO6FHFSIO1PRelX~ z#jj~+jlyq>wLZ(y#v8=0%|Cgy6nnYr3;VXm%QnXC6U<{G%2xrXmxuCY6rYw|AU znz@^~=I>#yrF)rc^*-kJGuQeDn5*nT=88VVTb$ydx({xx&OzhSQIx6D=ij=8G8XRf**n5*$e=4$zg zx!QkbuC8C0tM^yt8u*R5hJRg;B&p3Ru6zlXVoHfOHUUgnzEg1M&q zm}_oJ=34A$u9dBr4=~rdt(mKIkhvbu7jrevF;~mp%+)^6TwVJxSMLII4eZNY!;8!{wjXm%E-}~4{>(MM z%v?(cFxTn|^8=Y{{VH>n9mHJGHRehm%v|9i%vE(LbJZTkTn&daSMw3f)pjIvbsoiB zJx4QF|1r!pbS!g?9>-i0$1~UT3CuNjB6BUC#9S*UGe3p7)}6{+rKd4h#p%qIJcGG{ zGnuRMEas{?o4M-GVXmfgnXC0Y=IS_~xwUfX2 zy5DE6z7Lpd@I&Sr`G~p3KW46}Pnc`=Q|4OujJcLSXRfs`n19J!C0{XD`Pa-9|Ax7; z-!fP6JLanXp1JCNV6Mg=nXBa|=4$_$xw?K~uHIjnYv4EL8vdQR#{OWg$v>HE<}c=& z|C_m%{$Z}wf0_TsTJx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB# z^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN> zJx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5 z*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4 zggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%v zPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB#^MpN5*z<%vPuTN>Jx|#4ggsB# z^MpN5*z<%vPuTN>Jx|zk<|^BexuWgNmEMTC!Vc!D+L*a&JDICt6Xt5}Vy?DLnX9v# zxq3EZuKph88rqz>MthlSVhiS)?qjaGEtzYvpSf1HVm`oJ>$YaD(n02`*oL{1L(COy z%UqSi%vG};bJdS9SJU>))w%<7b?nGo-J{Iaw-a*>jxpEB&dfDF&RkQwFxTt^b1m%3 zT+5TpwYD4cDdsBKow>@VnJc~rb7g0ktGFj~RnIb4-CoSqILBNqdox%2JacvJ!(6=! z%r&qta}6&t*Vul{HMzuGGy5~w{4#Sb9l%_xE6fjMuJxZx|*U+)dHF_L#O&rf$(<{H0*xuz~CHnQQ(Y=32U! zxmNFEem`@qe}K8l9%QcQL(G+an7P77n5*hh=Bj;+xf&j4uI4A0tL;hV>U@g1dY)#k z{%4qL=vn3(eU7;%o@cJ<7np1AMdn(3iMdu@X8u3sTK5WbmA=Yc6|XT@@^$74-e9iE zH<_#EE#|6!o4K0aVXoG9nXBVH=IVZ*x%xg}uE7tPYvd#58vmHNraob=*-x2k;WOr1 z{+zkizF__(bCrC>T;*RgSNt31%6`jS#qXG_`g`W8`+>O{e`KzfpO~xtXXfhqg}Hiv zWv+qWm}~fV<{JBhxhDT)u9?4>YyNNMTKb2%R{v%GA9JnWAZgE&_B?6NllDAm&y)5% zY0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6N zllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1P zJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm z&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9 z_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&y)5% zY0s1PJZaC9_B?6NllDAm&y)5%Y0s1PJZaC9_B?6NllDAm&zY-iL*|OMGgo>e<_bHQ zt7>EBs_kU1hE15Oxr@2lHf65PZszLQjJf)Im}_Wr<{Ir~u8A#}Yr2oQ=C)+6#eU{m z*^2o9bFJH&xk?9_t704GN)9nsuq|^{4l`HHcFa{j!dy+;Ggs>l%+;|Yb9Ik0SKm&| zH8{pxBRezK_&9S-?ZRBM6U?=+D|0PRGS}K}%%_;EWOwE&pJuN39?X@UVXoqy%vC+h zTy=XfSK}OWwd~DY?eomlwGVUkE-=@?zRWee$XsLlG1uf0bIt6}T=UD!wR8Y;t*$UX zkh#{cGFRC_%oSZ@uJpmo6&}J|RfjTH?P1K-a5!@{AHiI0M>1FEQOwnIG;{SI!(2nh zGS}#F%r$X5b4{PXTyrNf*WyXcwQ@4^QTF+yyj`NwT`vT_byO6mCFJi8di!uJ&7)tLs+g>b;G*25x7r z;X9aX>`vyIyo?T*=p&D|mysD&J(Tnzxv%{%z)JdWX4M-({|j_n52uedg->fVl=g zWUi5qm}~rF=9>D1xn@6Qu7%H-Yx#5LTKj_em&{f26?2t;&0O(sm@E4&a}~d1uIlfZ ztL_KpYW$J8T7F`#_Me%n>lfzg{gt@}eq*lT-e-J1|$rj?C3P%3OUrG1uT2bB*lGT;t=+HMI+K%}y}a!miA1oVWaXND)&tR_LOy;UQi@9pfX0G~kn5*er=4w5UxjN2guI>w%tM5YQ8oY?PMlNQq z@k^L%>Qd&Ky^OgQE@!UgE0}BTO6FHFSIO1PRelX~#jj~+jlyq>wLZ(y#v8=0%| zCgy6nnYr3;VXm%QnXC6U<{G%2xrXmxuCY6rYw|AUnz@^~=I>#yrF)rc^*-kJGuQeD zn5*nT=88VVTb z$ydx({xx&OzhSQIx6D=ij=8G8XRf**n5*$e=4$zgx!QkbuC8C0tM^yt8u*R5hJRK+Zzsy`q2Qb&_3iAV*YyB#7l^w)f z(KY5uAIx0gATYDN#+#U{xoRI{u7<~%tN97$ zYI~BoI-g>$o~N0s{~6{QdX~9HpJT3x=b3Bz1?HN2k+~LMVy=~!ng5Tu*1f`9rLQtq z#cRx!e4V+1H<+vPP3Ee3i@ECGX0E1pn5*?&=IVHlxw_wHuD%bLYw$zn8u^I1#y@7R zsZW?|_EY9s_>8%hKWDDBFPMMHTqR#ISNYe>75|30vfnaS@jK?K{+_w&eqgS~ADOG= zC+2GZnYp@tVXod^nQP!T<{JK;xyJrruE{@{YvwQJn*W=*mi}R`)qk1)$6V_-$k_9Y zJFjv#|%+MK z*ZeYbEgisIt1HY8WUlqA%vE*}b4AyfD}69?g@-U#)uGH)dl+*y9L`+LM=)30k<8V3 z6m#_)&0PJ*FxSwr%r$x(b4?u2T+=5o*W8KBwRjS9t(?sK6y{oYDsz>d##|MrGgtBq z<_gYauFA8RtLALxsy~Ohn$Bge*7KOF<9z1ozJR&bjM=dT(Q{f!mpD_zvb8yOX&l?_#c*yP0eL9_CuQm$_E&V}3t#t$%>I${u8{=tInv zewew!N0_VXQRb?BjJX;fXRhWan5*qc=IVTkxq6;vuKs73Yv@_#8hwtrCZ1=m=@*!5 z?nUNWe2KYMUS|G3=34g(bCtfzTotb|SMqh{3f^F@$~T#-<}K!`f1A0Q-eIoRcbTi> zJ?83upSk)zV6MRrnQP=D<{JN)xu!m0uGvqSYvD8ITK=55*1ll=C3BU0#a!iIGgtf@ z=E{D{T*dF0tNMH9s{4Vt8h>Q2mY2`nX6(O=1LAR zSFkN}RSq*(&34RHKf+v1+cQ_|4$RfDBXf0+GFRVD%r!X1Tq8R(*Z4SdP3^*5vlGm< zuq$&dPcqlqZp^2ct7LcPDxYSq_#Vuaonfxxp3GG}%UpGPF<0XpbG7WvTOGam+PwJabK-z+7`DGS}is z%(Zed^HZ2>-Koq~dKz<8oX%XyGngwllesF-Vy>FAnXCRB=4v{ZxmwR-u8#AWtNQ}x z>bsD+1}|c+k&BsY{1WDxx|F$QFJrES%b9EW3g%k7lKEB4RdO|Rm0!bL@oSkYdmVEX zuV=3680ahqy^s0*%(eaj<|=!TxuOp-SNdV*3Ljyvsz;ft_A%yac$~SKpJ1-GCz-4BDdy^V znz{O)VXmQPnQQbp=9+k(xu#!WuDKVPYw;!KT6vlI|CnptE6i2;Dsxr5#$3tQnJajM zxhmgeu9~-)tNv}~YI=vcTHj@^j`x_W`+er>`+&IyKV+_vkCiw0u27Y6%;oq5S><{Le{FAw6{$j5AznN?4ALd&9m-&CpwSI$~J5Z5x z>|m~{jhU;qlero;VXo#b=4#uNxjMUhEE$q0O0Vw3oRiwqUO5KIWR+lDQW9 znQLV$<^#;NZfoW$9b~SGZI~-L#9YC)%vCweTs7MYmGgsF>%+8s3D@fvd_UuUl14d$wRleuc%Vy^nPnXBm?=4ySHxjNosuI~4ltM3Em z8vKyCMm}P$@sF8n>J#Rg{gk;DK4Y%s&zWoO3+7)kSIJk*RsJ<|#lK;$?6=HS{EoS* zzh|zxADFB0N9JnziMiT;X0EPZn5*|!<{J2oxrTpduCYIuYw}O#n)!>l=Kp4{rGJ=f z^@i2lhO$=Yc&B?0I0%1A89W^T3`5_B^oXfjtlGd0@{2 zdmh;Hz@7*8Jh111JrC@8V9x`49@z81o(J|ku;+n25A1nh&jWiN*z>@i2lhO$=Yc&B z?0I0%1A89W^T3`5_B^oXfjtlGd0@{2dmh;Hz@7*8Jh111JrC@8V9x`49@z81o(J|k zu;+n25A1nh&jWiN*z>@i2lhO$=Yc&B?0I0%1A89W^T3`5_B^oXfjtlGd0@{2dmh;H zz@7*8Jh111JrC@8V9x`49@z81o(J|ku;+n25A1nh&jWiN*z>@i2lhO$=gd{MA#+9B znJc{!bA=tuRkbm5)pjyh!zRqt+{IjNn=)5tH*@uD#$5e9%r&$*bB*>g*TfdgHQmQt zb6YakVn1`OY!%pZ=32KkbCnJVX?U<{6gt?lwXRg*An5$z) z=IS10uD+d^YjBLYMs{Yd@p0yw+J(7hCzxwtSLRxtWUjT{m`^cR$?nWmKFwV5J(w#y z!(7EZnX7u1x$5>}uEsg$YT28)+UJ?8Yaiz7U0|+(eVJ=`k-5h9W3I_1=9<}`x#pLd zYv};yT3um&Aaku>Wv;S=m@B%*TYvts?o-^0FQ<* zKEhm8k1|*7W6ag?ICC{W!CY-mGFRtQ%+>QWbM-&NTtm+?*XVQ1HSs)iO~1fgb1yR2 z;!Di6@^WC$nQPrE%vJg-b5*>?T*=p&D|mysD&J(Tnzxv%{%z)JdWX4M-({|j_n52u zedg->fVl=gWUi5qm}~rF=9>D1xn@6Qu7%H-Yx#5LTKj_em&{f26?2t;&0O(sm@E4& za}~d1uIlfZtL_KpYW$J8T7F`#_Me%n>lfzg{gt@}eq*lT--k#^}dETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mkp6Bg( z-k#^}dETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw z=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mkp6Bg(-k#^} zdETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mkp6Bg(-k#^}dETDq?Rnmw=k0mk zp6Bg(-k#^}dETDq?RnmwGgsM$%oS~CuJlIC6?QOJ)yB+K+sRxFn=n^%7jw03%3Ph@ z%+<3QbM^Ny*U;w7HQLKu6I(FXbRTogZOL4V{miwp74re+TDLWGl@2mj#Wu{99Ad6u zTjr`9X0Dp;n5%w-xtg|TuGSrxt7Avz>Kh@x;#yRF{*_*lA=b5W(ALi;^ zV6K6EnQM5FxyJTmuE{0ln%SSZ=9igk=>X4TXoJcPNb z4rQ*|!O|YvOq3nm&QK=1yd;#gmw8 zbIo4HTnm>o*YXw2wRR=*tC*|gYUV1xhPmR`GFSFG<|MFxS$( z%(Z$S^ZS`={R7NZ_8@abA7ZZb!^{;v!dz94GFRHXb2UG~Ty0M>SLajA)$=rS z^*_U0L(ekT=yS|9@jP=)zrb8`FEZESOU$+MGV}j2*Sc4jtMpaos(6jLlCLvY@CI{L zzR6rQZ!uT>+sxJU4s*4>%Um7rF<1Bd%+>b+a}9pTTq7Sb*Z9ZGHT4N|&3?*U3!gF9 z^5@L8_674VnXBY0<|_Z1x#Hh2SN2=xDt^aY)!#E$-4D#w_#<<*{KQ=CKQmX?FU-~Z zD{~F}#$3a{GuPN3%r*HZbItt4T=Rc3*U~@CwfZmf|Cnq22BAFC8!}h4ow?E*F<02ZTvZ!0S8XSAHEhCM z&0WmZwkdOUb~9JcX3W*!!(2m~GuLP@b4_f)T+@BbHMb>mE%r0l%2vz=m}}kE%vCzb zTov0eS8|BCf^C_ra+tYlwqvgP5$0;zp1E3gV6KiGnX7x0x%zfuuE8bg! zY8U33onWqoU72fnlDXD)V?M=PCA%|M`80FI_h7E<409FtWUlI2=BnF^xfxRcY(PE_GPZ&MdljYkGUq7m}_Q#=9*t-uB8K*YjuVBfy}jjmAT3eVy@^K zbEOYvuJ91%sydXpY7b+shQpbw`3UA}JCeCNk7BN#qnWG!80H!}mbpfcW3GwgnQQt4 z=9)W^xfV}iu9cISpTb=0PGzpr)0nH`bmmH)!Cb+a%vE_7bJd*9T=nNLSJS!7)p{Ot zb)3&!-4`%d--XOIcoB1rT+CeKmoV4VrOY*Z8FMXM&Rok^FxT3Z%&%gulB=1k{2Jzp zU&~zC>zJ!}J#$sxz+81VGFRhG%++!;bG6^XTwS*^SMP1iHE=s~4d20BV|Oyw_cGV&ea!D?uJsQvSJ{Ki6@7@g(hoCN_y}`VJ<43Qk1!4a}7PqT%*r1*TnP8HT?o}&ArH6i!U+P%FE3E$6V`PVXo3wnXBS8 z=1RWKT)`X6Rrw}!)x5=A^=~s*(>u)7`Yv;IyvJPK?=x552h272A#;s<#9ZSaGuPB7 z%r*Ndb1i(vT+5#`*V-4%zhthGub8X+Yvzi7!(7>KnXC96b5(!OTy;M%SL2V&)$$W_ zwg1dqUB56_@2|`?@Edat|IS=ve=yhNpUgG$7jw=3&0I_WFxTq8%>QGq^&1rIdBL6+ z?0LbS7wmb#o)_$S!JZfFdBL6+?0LbS7wmb#o)_$S!JZfFdBL6+?0I2>&i^ae^MXAu z*zfJulevf;}(T^MXAu*zfJulevf;}(T^MXAu*zfJulev zf;}(T^MXAu*zfJulevf;}(T^MXAu*zfJulevf;}(T^MXAu*zfJulevf;}(T^MXAu*zfJulevf;}(T^MXAu*zfJulevf;}(T z^MXAu*zfJulevf;}(T^MXAu*zfJulevf;}(T^TG!04F!8%u;&GP zUa;o{dtR{T1$$nw=LLISu;&GPUa;o{dtR{T1$$nw=LLISu;&GPUa;o{dtR{T1$)k1 zWg9YAw4J%q8!=bd!CX}vGgoaVb2V(jT+Lm~)wU^fb#^mX&t}Zk-@{x(n={vFFLO<7 z!CcdQ%r&S>~$Si@6%-n5$)P=4zj3uC9HUt9OC92KHsH;YH>e+mE>>mzZm2 zf99HBX0D|Jm}_-~`GL%}ewDe(4q~q88gr!&X0GrM=BhfBxoQt%u7<;ztN94#YCDp- zI*($mo}-zo{}|>PI+nRck7KThDMk+~L6Vy=~wnV-U3>rQ2^($ko$;&kRp zp21wfnaovr7IW2{&0O{8Fjv#L%+-1xb9J20T-_HiSKo!qHFyzojarT+Up}S1{MwmCUbVu9B;ntNa?~ieJlI+3T3Acs+Af-@sgTH!@e_P0ZDDGjp}y z!dzXqGFR_y%r$U3a}D3YTw`}K*W_KyHFGy}&ELaZOZPI@>V3@bXRh@RFjv`w%oTlz zxzZ0aSNI5XRXxgFwU04Z!{f}=`~-8gJ;_|1Pcc`|)6CWX408=V%Uq++G1tWN%r*T2 zbIrZTT#GL;*UHPx|HoYGUSY1%SDCBgHRejb&RoG8%vJd&bJe`XT=j1=SJOMp)%q@T zb-c%1-S0D3-v`V!_#tzRe8gPiA2Zj~C(JebDRV7+#$3yvGuPS|%)ex=lCPMn{A=cl zf5TkaZ<(w39dlKG&s=psFjwP`%+>M}bG84>TwT8~SMRUPHSimA4gbzuV}CH$JulkxqCGF#^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44FWU2>Julkx zqCGF#^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44 zFWU2>JulkxqCGF#^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44FWU2>JulkxqCGF# z^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44FWU2>JulkxqCGF#^P)X3+Vi44FWU2> zJulkxqCGF#^P)X3+Vi44FWU2>J!h`64Vf$2&Rpq@m@DjHuBwfhtG1K58a83B<}T)H z+myLFyP2zJGv?~=VXmRgnQOF{xhA$?uIWDJn%k1O7W?b%(ZT7<|-Xzu8M7# zD>=km!M4m*Im}!&+c8)D2y-=U&s?oLFjvQp%+)>0Tzxw+*Weg)jqJ=^RUOJ)wTCfR!{N-;d<1i~9m!msM=@8=(ahC<408<~%Uq+!G1tWL%r$)ibIqN| zT#F|$*UHJvPhtN58LF?IS_?p1!)sq0!Uf?F!X|_e!XeCW_C7-hAp{W-5fKp)5fu>; z5fKp)5fQQX-h1!8_uh5cdvD9$dq2;``wO0A3>BTmT&1ToSH&63RedIN)t$v$!P(4} zIET4%=Q3B*dCb*%K67VVZeyDKTL(EnAFmu&B!d&%_GFRj= z=1M)zT!kl?tNBUhYI};gI-h2)o@bb=|5@f5dXBk9*E83|^UO8<0&~s1$XtsrG1tn= z%wJ)yqF0%#^fl(Hc%8Yb-(aq~H<>GVi@6eSGgs~%=4yJExmw?2u8#MatNR1y>idwn z20vo1k&l^c{1fJy`jok5H!#=2XUw(yIdiRj!Td|+D*1}J%D-l=s&AO9_FLv^_>Q?^ z-!oU{2j*)0k-1uaVy^a|nXBs;=IZ^Gxdwh?uHoOAYwQo^n*5WwX8vNX`M;TK=^y4= z{g?TF%vHQD&&RjJcGgo~Fb44~`u2d&;6*gtA<}T)H+l;w7yP2zJbLQ&r zVXmPqm}|6`xhA$`uIWDJn%j!G7WRXxO9b=xslFw9(u z?U^e#!dy)|Fjwo2%+;|Ib9Ik0SKrReH8{pxBfBuy_&9S-?aExU6U?=+8*?pBGS}Mf z%%_;EWDn*lpJuMAJ(;U^CgjhV@5NlPS?0>@&0LLh%+<0FbG6SiSJ%GG)w{r41N$-8 z@FH`K?ay43OUyNM0CUYRGuP6A%(c40{2=BkUS+PbgPE&xjk#(LVXpc^nJaP_bEOVv zuEG(_)qEs#wH?J=okufQ&oRu^e=Kti9miaw$1~T&3CuNpB6H21#9WIfGuO%~%ui*m zqSKhG^mOK`ID@&W&t$H;vzRM5o4FF_Fjwwe=4v{RxmwR>u8s?stNTLc>br=!1}|o= zkxQ6s{8Hwcx{SGIFK4cWE0}BfO6FR-iuu*dRdNk;m0!zTRo5|B?e!sl&in@EirvUu znVXoa@n+^~xrMpfZ)L8o+nB5OcIFzmgSm$9WUjHhm}~NG=9;;Ox#sU>uBH2!YxRES z4=`8pgUnU-5OY;N%v?2(FjxJf%oTZzxl)faSK$ffYJQTr+MZ&r&Zn8H=Nabef0nt1 zo@1`j^~^Q#JabLIz+7`LGS}iu%(e0|^H-Rw=vC$_eT}&)UT3cAH<+vLP38*TVy?v7 z%$0kGxtiW(uGaUMtK)s<>i&Sa`aWc?!H<}0 zF#nRdO1@&Q@~@ez>Ko>&{Wi4jLN$EHT(R$&EAs<$HU7w4Ek7|=`_IhP^$Toef zzcJVF@60v!2XjsS$y_skG1vUx%(e6nbFKc%{6FR@UKjZDz@G>HJn-j%KM(wQ;Lihp z9;~}mk^d9;^T3}6{ygyKfjHJn-j%KM(wQ;Lihp9{BUXp9lUt z@aKU)5Bz!H&jWuR`18P@2mUHJn-j%KM(wQ;Lihp9{BUXp9lUt@aKU)5Bz!H&jWuR`18P@2mUHJn-j%KM(wQ;Lihp9{BUXp9lUt@aKU) z5Bz!H&jWuR`18P@2mUH zJn-j%KM(wQ;Lihp9{BUXp9lV&xym+TuF7`is@a&i>N}V#vI%pgI+?4mDRVVEu_bd&_c7PpR?M~7&s-~8Gaq2CqHUO~bdb3!wq>sB zA?B*vj=6$i=1OeOT)7eEYTALhT6biwj-8mRdz88Qc4n@@G3FZCg}KJZnQLlS=9-;g zu7%y0Yk88n)^=w;#atzOFjx6Bb5-rhT(vXI)vyI(CNn5%e|xylY^uF5s$syT$Y z>JMeE$YIQtI-I!*M=)3Ok<8V06mxYR&0IalFjxPv%r$fzbB!L)ToWfS*Yt_ZHFpwo zEuPF=E2l6&mAQ&eW3JNEnXBRq=BhrEx$4ehuHbCuN}R)7xpSGT={)9YJ)gNcE?}s(hHaY93*(`bU{7@)&ca9%rt?6U^28By+Vr#ax|F zGgr?u%+>!aa}7PmT%+rmYvOt4ntp+~=3Zp3#g~|C~T-9$d zSKXV;6}-hb2WU&T(R$&EAs<$HU7w4Ek6bRocYhp z)%6Q=_5R9S1HUoX@bAnu_6KuK{>fZ3e=*nm-^{i24|A>l%ltp)Dqa`%=V5;y_UB=L z9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU> z=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9 ze;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y z_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)Sd zVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L z9`@&9e;)SdVSgU>=V5;y_UB=L9`@&9e;)SdVSgU>=V5;y_UB=L9`@(VRkjgxRkkx% z&Bn}C-@#mwO_(dy$y|j^nX9>rx!N{kuFh`e>e-yR`g@pbXba{V?PacsEtzY&kGbZy zVy?x0=33dB`2cejZNprpgUnU2Ept^5F<0Go%oPkXS7LkS%8f8r(+palatiZPnXBkD<|;j%xhl?JuIe+HtL`l3 z3eIM(#5v5BJD0hd&SS3D^O>vT0_N(zkh%IUVy?l9nQP>dus>&hDRWI-#$2P>zJ$ddgf}lfw^KgGFRp%=4!l|xms>vuJ&7*tLrxA z>b;%02JT?4;X9dY>@Mb-yqmdZ?qRO^dzowLKIU4zpZNpKRs0}xl|956Y$ z_!4uiyv+O+<|=xXxk_JSu8P;0tNIP*s(X{Ug149}@iuei-eIn$cbTj8J?83opSij} zV6MIonQQPP<{J4p?9Z8h!dz3IGS}<|=34lSxt2d?uC*_if5}`WUoltt*UVM*4Rh6g z%UliLF<0z+=F0rQT#Y|6SIbY#)&4Vcb^XFzy}vToz;Dbo{5x}v{lQ$5e=^t1U(7ZC zH*+oh!(6NXGXIaciq}Q_dBmSb{CUKmNBnujpGW+8#Ggm}dBmSb{CUKmNBnujpGW+8 z#Ggm}dBmSb{CUKmNBnujpGW+8#Ggm}dBmSb{CUKmN7j9$iT@k%=MjG%@#hhL9`WZ9 ze;)DY5q}=>=MjG%@#hhL9`WZ9e;)DY5q}=>=MjG%@#hhL9`WZ9e;)DY5q}=>=MjG% z@#hhL9`WZ9e;)DY5q}=>=MjG%@#hhL9`WZ9e;)DY5q}=>=aF?AG!=^Y^N2r>`16QA zkNES5KacqHh(C|`^N2r>`16QAkNES5KacqHh(C|`^N2r>`16QAkNES5KacqHh(C|` z^N2r>`16QAkNES5KacqHh(C|`^N2r>`16QAkNES5KacqHh(C|`^N2r>`16QAkNES5 zKacqHh(C|`^N2r>`16QAXRfl1n5(j#xoS3MuKEtD_r(|yb}w-s|O_A}SY*31W(t7seMDjj65ifx&zdWgB| zwqvefn7I<$GgoeextexhuGSryt79kT>K$3E66Y{i?p)?-I*++p&u6ZV3z)0>Lgwnb zh`9zYX0DM-m}~q}=9;>Uxn?hCu7xX@Yxzp%TDyw*)y!3L4Re)W%Uo60F<0&N%++uM zbH#3CuFOr$)p#>=wcNs7?YA;l*KN$zdpmOt+`(MKcQV)5UCcFkH*?M0!(8+CGS||5 z%(Z$y^9Pu#_(A3>dx*IzA7-waN0_VrQRa#~#$2h#nXB*wb2UH7Ty0M=SLf5r)$Keb9H~fTzwxh*WgFYHS#fYjeo*iQ=c-|>;~pq_>8%h zKWDDBFPMMHTqR#ISNYe>RrL*X)qcxd4c{?W?0e?Q{J>m|KQdR#Pt4W+GjnzQ!d$(- zGS|Rw%r*QwbB+DMT$6t?*UVqcHUBqrE&an>tN$|pkGYE1Mg4iypGW<9)SpNFdDNdr z{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9 z)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@% zNBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNF zdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!! zpGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFdDNdr z{dv@%NBw!!pGW<9)SpNFdDNdr{dv@%NBw!!pGW<9)SpNFIdhe5#9Wo_%vG~7bJce+ zS7a0BN_8?R6?qaUC&6umRo4I;6XRiJp<{H|9xkh`LYhp|0n(kw+xviLMv7fnC zwq`!STt(Y3SLq;gRcy;#)kDlxw;gi@!_1Y~p1E=(%+<65bG7csTpc?xSNAA$_3g}D zgJaA!vI}#Kk2BZQuFN$%!CVWwG1u}WbFJ;pe2TeB_F%5^Y38chleubVn5$tg=8DZS zS7vYKYMf)PmVKD3eV)0x_GPZ#1?C#qkGX~ynQLr+=9*k$u9*XvYkrxzmJVdD)fMIk zF<0>_bCn&;T$O9gRdWb))gQ`Sk;9lPbvSbsj$p3lBblr1DCX)snz?$8VXppTnQQ1c z<{CYoxh76vuIUq*YwjfGT0EJ#R!(7lDsvT`#$2VRGgrkK%vF6RbJd;2T*2AQl{kmF za_2Hv(|OF*dOmY?T){Sv!d&B*GS}2)%r$#Cb1huKT+3H7*V* zzLUAe?qaUVyP0d|9_E_Am${biW3JWvnLof>#Sb!9*+a}#`7m?UJi=V{k1|)}G3H7= z&Rm5jn5+3o=4yM2xjLU_uAXO@tN&T%8hVboM%Odf#PiHG{Q`5%y~tdPFEQ83%gkS4 zuA*0&tMoPIs(78bs^4I)x;L3Cc#F9bZ!=f!9p-9!m$_QsW3G<(nXCH)=IZ;9xduOC zu91(KYy1=Dn);NvW;Za`!e`92{5f;2eZl-o<|_G$xyrw0uBvaCtM*&wYWR-1V&5}Y z<_G3#{E@j@eqyfnpP8%c7v}2ymAM9fW3J)fnQQD1=9>JIxn}-iuKB;2Yv~{6TK$*# zf6P_9F6Pf;{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT- zWBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemP zJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm z&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1 z{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P z=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT-WBxqm&tv{P=FemPJm$}1{ygT- zWBxq0uDFf4$~I!I%68_e*_gTNJD4l733H`7nX9lVb2WD{SKDUH)!EHlJ)1LEe-CpF zZNXflz05VSC38*pG1uHy%(d9hTq|2MA7HMcZJ4Wckhv(HRh^0gt_VuWv<9!%#}Kvxe7-x zSM!m~)pitfbso)JJ;yLt|FO(9bR2Vy9?x78CotFaiOe;35_2t{%v>v{Fh7;KicVv$ z($kr%;tb}hK9jlX&SI|MZ01Uw!(6#@nXBnM=4w5kxjHUjuI>w&tM4M_8oZdfMlNBl z@k^O&>N4hix_gV6Nf^ znXBv}=Bj*{xoRF^uKGurEAkj~r5m}}uP=34%oxz@g5{v~sj ze8pVlUo%(LH_TQ0Eps(|$6T@RnJe=Hb2a|RTrEE_SNqS*)%6Q=_5R9S1HUoX@bAnu z_6KuK{>fZ3e=*nm-^{i24|A>l%ltp)Dqa`&=W%}?_vdkc9{1;Qe;)Veaep57=W%}? z_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Ve zaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc z9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57 z=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Q ze;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9{1;Qe;)Veaep57=W%}? z_vdkc9{1;Qe;)Veaep57=W%}?_vdkc9$#17#$06^F;`_fbJc9jT=gBy71@NjQk~3I z*p#`NyO^tOGv?~-X0D#inXA8txrVl2uF+oRn%I)Lru&#{ZY$}Rf(t(gxnSJ5`i zRXWIA727gb^$>H_ZO2@}FmolgXRh1`b2aV2T&+7YSI17w)ji5weLFMP;23j_?802* zWUks7=4#lBxni@-mD!uQ8t0g+ zWgq5hpJ%SFeVMCwfw>0uW3J&v<{I0dxh9vGYvusvnqOwFr30C3b%pss%vHR~TxAC{ zSLGUW)f~cH^@lQ7XvFxU8{%r$ixbIo4PTnkq)*YcIjwRRQrtC_3h8s;j$ zmbt2~W3JlknXBOj=8E0OT$!7gtMO*$YPp5E+HYm9uG^Ta_jcwQxP!Td?_{pAyO?Y8 zZswY~hq>nOWv-?Bm}~WZ<_|De@q^4&_7HPbKFnM-k1$vLqs$d~jJZ;eGgsjW=4yVD zx!Rs$uFj{KtLGW!>VKBGhMr@t(e=zV@jP=)zrb8`FEZESOU$+MGV@oMtLRncDt(Q) zDqd%<>Nl9H?oH+j-eRuA+su`Fhq;>GWvYyNNMTKb2%R{v%GA9EG2OZfAI zKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7 z`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7c zgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7tn02z z`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7c zgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6 zPx$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg;OC^MpT7`16E6Px$kMKTr7cgg@&0LLh%+<0FbG6SiSJ%GG)w{r41N$-8@FH`K?ay43OUyNM z0CUYRGuP6A%(c40{2=BkUS+PbgPE&xjk#(LVXpc^nJaP_bEOVvuEG(_)qEs#wH?J= zokufQ&oRu^e=Kti9miaw$1~T&3CuNpB6H21#9WIfGuO%~%ui*mqSKhG^mOK`ID@&W z&t$H;vzRM5o4FF_Fjwwe=4v{RxmwR>u8s?stNTLc>br=!1}|o=kxQ6s{8Hwcx{SGI zFK4cWE0}BfO6FR-iuu*dRdNk;m0!zTRo5|B?e)yna07G2Ze*^^P0ZDJGjp}v!d&gQ zGFR7a%+-54a}C_VT*G%V*VtXmHF-C4&D_IW^Y=2>(tXUedO!0An5+0f<|=!LxhfxK zu9`=ftNu~uiaf?#smGbC@C0);KgnEePcc{L)6CWL40H8A%UnaxG1usN=9+k(xu#!W zuDKVPYw;!KT6vlIE6i2&Dsz>-##|MzGgtK+%vJX$a|LfPSK@8v%Dux}P46;S>wC=A z@ji2Pf52RQA2QeAN6a{T7!dz3IGS}<|=34lSxt2d?uC*_if5}`WUoltt*UVM* z4Rh6g%UliLF<0z+=F0rQT#Y|6SIbY#)&4Vcb^XFzy}vToz;Dbo{5x}v{lQ$5e=^t1 zU(7ZCH*+oh!(6NXGXIaciq|FA9jmhcN&54oKTrDeq(4vk^Q1pd`tzhePx|wuKTrDe zq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhe zPx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk z^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wu zKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4uZsiZ$o`tzhePx|wuKTrDeq(4vk^Q1pd z`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDeq(4vk^Q1pd`tzhePx|wuKTrDe zq(4vk^Q1pd`tzhePx|wuKTrB|<|^BWxhmV4t7c>7s_$T~$R^B{>SV6Mrp(pc#awNh zF;{0dbMU-FHM9kDjrKCv#For8-N#&WTQS#SKXa{Y&3u5lind{{(n02`*p|7f zhnTBwJLU?8nJck9bLB>ut7!-3YTc2!I(A~N?osCI+nKor$Cztm7v>rtXRfJTnQL}} zxfXU~uH{MQTHBrZ6mymA!Cd9j%vH4~bJfl;SHoV+6`N(Q%-+n^ILBNq`!HAgJacvJ z%Urz+%r&qda}6&t*Vz8dHMzuGGY2r&{4#Sb9mrg(E6fjKuHseZDm$3DD%Y5+<`Cwp zKa{y5hcQ>`aONr;!CcKpGFRJC%++}`bM+jZx~*U)jyHF`XAO`O17(tk9n3X+Cv%P6#axqjGuO;L%r$>6 zb1mJ+T&wppe}K7)A7rkwhnTDKVdkoNgt_V;Wv<9$%$0haxe8A(SM!t1)%Fx~bw15p zJ1)hY@j7!=zrkE} zZ!%Z#7IP)uX0F^j%+>TRbG5$5TpjN-SN8|Z)%PKD4SvL2BOf!@_$SOY^(k}BZeXs3 z&zNiZbLLw6g87%sRq_>cm4D4#Ro^gI?YGR;@Evo-zGtq?56sp0BXhO<#9Zw^GgsFy z%+>oVa}E5)T*JRJ*VrG-HTfrV&HTk&^M5nf(m%|#`Y-eUn5%eQ%AcqFdCH%s{CUcs zr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqF zdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-! zpQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s z{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ z%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcs zr~G-!pQrqJ%AcqFdCH%s{CUcsr~G-!pQrqJ%AcqFdCH%s{CUcsGgsM0%vIUWTs0dr zSA7R_MK)osR3~#4Hf65nF6L_6jJZ0ynX6}W=IZZZuAwcMYqXcSCbnd*=|1L~+lsjs z`nJYKKTunPLSL=?<)v*(Eb&oPv z-_Fc6IL2HfyD-=IICD+y%3QM(%(bu^b1hFY*V^vPr=b5W(U*_svV6K7vm}_{ExyJTquE{0lnmK^E=9igk=|JXM zU15F@a}}>LSJ}bLRk_AoHHR=){h`biIgGhdhcj2<2O}YvKgvnm&=a=1yX+#gmz9YP^}bT5e&k_FI{&>o(@MQIBg|F*D04*~ zW3JTW%vE@TxtgD3uC}L`tMh5*>UoB_`k!U4q34)ubUkxTJkMOyFEH2Ki_Eq75_7G* z%={JRDteWx0oyOHgo0PVXmfknXC0Z=IVH#xw=1KuD%bM zYw#oH8u^&H#y?@MsZW_}b^~)Qe8ybMpEK9m7tFt8u9B~qtNd%`s``exYQJT!hVPgw z_C0fDeqgS~ADOG=C+2GZnYp@tVXod^nQP!T<{JK;xyJrruE{@{YvwQJn*W=*mi}R` z)qk1)$6Uqh(*8W{&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf| z&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ z{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q z?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^ z)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$Nx zJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf|&(r=q?a$NxJnhfZ{ygo^)BZf| z&(r=q?a$NxoVm(2Vy?<|=BnA4x#~NZE3yf5r8=3buqksjcQIGnX3W*u&0Ia3Ggp5P za}905T%*0rHL)dgP4_X^+*ZuB*w0)mTQeVEuA*(2t8|dLDz;^=>LKQ;+m5+{VdhF~ z&s@0?=4#r3xmtH*u8y6Ut9z8W`gUfn!7=6<*@d~r$C+ztSLT|XV6KJTm}_~Gxz=`P zKE+%mdoWk|G;>w$$y~KF%+;_LbH!$vE3-FqHO?_t%RbE2KF?fT`!ZMW0&@-Q$6UjU z%r&+@b4@NW*USOTHNVVUO9wL7>I(CNn5%e|xylY^uF5s$syT$Y>JMeE$YIQtI-I!* zM=)3Ok<8V06mxYR&0IalFjxPv%r$fzbB!L)ToWfS*Yt_ZHFpwoEuPF=E2l6&mAQ&e zW3JNEnXBRq=BhrEx$4ehuHbCuN}R)7xpSGT={)9YJ)gNcE?}>=i=e3-dv9$~KfN0}?~7;~i_XRg8%%+>rPbG1FiT%AudSI;xd)&DGW4L!$P zqwAS#;(6wpeu25>USzJtmzZniW#+FiSJA7?Rr(rpRlLqz)o(CY-J8r6yv1CJx0x&V z4s$iV%UrGRF;~a?%+>t?bM<}5T!SAm*T~1rHU0^6O?}E-vm2Oe;WOr1{+zkizF__( zbCrC>T;*RgSJgMnRr@V-HGIchvG18H^8<4={>WS{KQUMP&&<{J3v>1U%3K4#G1u_# z%r*80b4~uqTr+<$*Zkkiwe$~jt^UjWKjtc4m+|Krf1dH@8GoMf=NW&V@#h(Tp7G}y zf1dH@8GoMf=NW&V@#h(Tp7G}yf1dH@8GoMf=NW&V@#h(Tp7G}yf1dH@8GoMf=NW&V z@#h(Tp7G}yf1dH@8GoMf=NW&V@#h(Tp7G}yf1dH@8GoMf=NW&V@#h(Tp7G}yf1dH@ z8GoMf=NW&V@#h(Tp7G}yf1dH@8GoMf=NW&V@#mR!>(%%FGX6Z{&oll!& zhPg@ynX6)3=BgfIuDb1*D;Q?3#P-aU8)2@d9hj?iN9O9-iMhH*nX7MS<{BJhu901s zYkZu!rgmkn*$L)a*p0cCCz)$)cji;fRk8b{V<`YvLw!Hbz|$EFjxDn%++-pbM@ZNTmyG7*YKUpHFg(s zP2SC1Gxspp{JqSzbRToA-p~92<|=-Wxyl}5uF8j*tL72rs(+NZB9Adw>T%{OJi%Pe zPcm2AQ_R)*G;{Sl!(9E(GS|>^%r&~6xh9@xuIU$;YwktnT6~GQR$gZQ3Ud{`%3P(d zF;~Uw%vJpcbJe}cT)|t+m3W)Ea_=x#)4R;o`W|z2yw6Uouz8SIkxZHFH&c!(6rBGFQWQ%oY2dxiUX6SL2V& z)$$W_wg1dqUB56_@2|`?@Edat|IS=ve=yhNpUgG$7jw=3&0I_WFxTq8%>QGq;&oYn zp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O z=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNh zf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA z_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{ zS%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fC zp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%04O=UIQA_2*fCp7rNhf1dT{S%1!4 zWg9V9Wjk}#Y|LEs9n2Nkgt=0k%vIQwxthC}t8Fvp>g;B&p3Rx7zlXVowqUN&UgnzE zlDVe)m}_n;=34A$u9dBs4=`8JHq2Ez$XpfMGFSBwbJcCfT){ANCAMd-+z4|u?Z8~E zJ2F?tPR!Lk%3OUrGuPl4bB*l6T;t=+HMJ{q%}y}a!fwp9Jjq;ZyEC6+u97{Nt9+Wd zs`g~A+8O3**o(Phv&@y*o4Fe2n5$(U=4zj3uC9HVt9OC92KHmF;YH>e+n>26mzZnj z0Op!sX0D|JnQL{0`9aK8yvkf<2QydY8gtbg!d&%-GFRj<=1LvTT!kZ;tNBRgYCDR# zI*(?qo@1D+|5)Z4I*z$Uk7url6PRoIMCO`1iMbX}X0DY}n4ijAMW-=W>FLZ>aRzf$ zpUGTxXE9fBHghG;VXoY{%++)rbG4q&TpbrMSNDa?)prqd4PMM#BbPAO_@&G>bs2Nb zUd~(#S1{M|mCUtv74xf^tK=HyD!-Pws;*>$rTds`^?v3LFjw(|%vJUfb5%af zTs4m{SN)^R6?u%gQjarN;R)tyev-M`o?@=friUJbdVgiEf!~;G_;=W@F~6?_jRTCd`%UWUj)d%+=h*Ty2{% zS7$eK^=!^u{XNVzv;}jG_A=MRmdrKX$6Rw;G1p>0bFFO6e1N%%wqdT)LFTI1mbt2j zn5%9(<_d@ zT)hj-HLxFZ4KFg+*#68lxx`#E2Qb(CGIK2*$Xu%{%nxF&;#KA695iZht2`b_4kJBzu3vzaS#4s+$sWv-_4n5*@C=IXeBxw_+Cw+{9dsH#1kuEzH$^D|2<-#$3I(GuOZ!%r$%`bB*1_T$6V**UUZ4HGeO2 zE#1dltM@a1fVqkvWUjJ@n5*((=Bjywx#}NfuE=A|m3o}H3QsUs^OMZg_7rn9dpILXRgc-%+>fKbG7`$T2z8Kg_lIFZ2JHt9V`BpXdE~-k<0FdETGr z{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~ z-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM> z=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0F zdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx! zpXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETGr z{dwM>=lyx!pXdE~-k<0FdETGr{dwM>=lyx!pXdE~-k<0FdETEhSJ_6)RoTv5H5)Tm zeFt+zHes$*Cvz1xWv=Ef=4#uFxjMUhEE$p)Htew3oRiwq&m9KIWR+in$j1 znQLWh<^#-Cv<-8W4l-B8w#-#M#9VdTF;_6mT#4uSiO*=4G>yFITu@iH3k1|)^ z&dfD9##|%2FxU7vb4~5aT(c9*wXhp=El)Dn+V0G!n5$$D<|?0NuBttmt9FLD8untY z*er8p_GYfeIp%8Fhq>D4nX7AG=IUKwu7UlSYj}~l#`b5f$tC8RIe@w5mzitnK;~Lq zVSW&E6|XW^*}=?JxyD>IhcH+Dq0ALIjJZ;WGgsjV=4w8Yx!R6muFj*GtLGTz>OYpb zhK^&d(c_tG;soZJK9RZRPGYXblbLJf6y~QgSJ7$AReCyeRh+?G)n_tS-C4{PoXuQ` zbC@f4E^{@V$6T%FGgrq2%+-A%bM;-sT!R-g*T^NzHGU~`O4YPf;9VmC5Z<|gK9yqURLZegzWTbZltHs3KUt_L{*O{yO4d$wQlevPom@Dx%bLHM)uBLaHtMxtR>Uf{Ix<6p9z7Lse z@FV6L`Ix!JKVhz^Pnm0W19L5W#$3yvGuPS|%)ex=lCPMn{A=c_`i8k`zh$n5@0ct0 zJ#%G#V6Mg=nXBa|=4$_$xw?K~uHIjnYv4EL8vdQR#{OWg$v>HE<}c=&|C_m%{$Z}w zf0_TsT*d1O{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN z3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*G zyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*| z&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#% z;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN3;w*|&kO#%;Li*Gyx`9Z{=DGN z3;w*|&kO#Xxym+TuF7`is@a&i>N}V#vI%pgI+?4mDRVVEu_bd&_c7PpR?M~7&s-~8Gaq2CqHUO~bdb3!wq>sBA?B*vj=6$i=1OeO zT)7eEYTALhT6biwj-8mRdz88Qc4n@@G3FZCg}KJZnQLlS=9-;gu7%y0Yk88n)^=w; z#atzOFjx6Bb5-rhT(vXI)vy zuF}()tKtmisy>ss>ds=W;B4kfoWoqXbD68@JmzXWpSe0NV6N^9nXB(2<{G@1xkfHw zuJKEmYw9xQn!TL47Or5fzQlfdFGmafw|^hWUj@Rm}})_=C3eU(W}f=`Wkapyv|(JZ!lNgo6Hrw#axNEnJf1W zb2YuoT&?dhSI7Iz)%^i;^?k@(gC8;1$j8hz{t0tUeac+38<=b1Gv-?UoVnJ%VE!d@ zm3+lq=de8*g|@0ly}!~bWfE`Rbb96$|!Uz{;^#u#H9_8nu4F~%5U zj4__>&+|Nph=_=Yh>D1ah=_=Yh=|x-KfAlTySux)ySvWy;{FS+7dLZ-e`K!YPs|nk z%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK-AwLiKdC1R0 zejf7kke`SAJmlvgKM(nN$j?K59`f^$pNITB&qID5^7D|Nhx|O`=OI52 z`FY6CLw+9e^N^p1{5<66AwLiKdC1R0ejf7kke`SAJmlvgKM(nN$j?K59`f^$pNITB z&qID5^7D|Nhx|O`=OI52`FY6CLw+9e^N^p1{5<66AwLiKdC1R0ejf7k zke`SAJmlvgKM(nN$j?K59`f^$pNITB&qID5^7D|Nhx|O`=OI52`FY6C zLw+9e^N^p1{5<66AwLiKdC1R0ejf7kke`SAJmlvgKM(nN$j?K59`f^$pNITB&qID5^7D|Nhx|O`=OI52`FY6CLw+9ebLOhZF;{iKT(x=TYA7&QbCJ2)x|pkT z6XuF{Ggo#~=IZKUuHMaGgsvdb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOo zy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7NttSYfW^1DI=VmHC0pRl3Gp6$dd_ z^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{jUC5algBgH z%n8gjesidzaW->B&tb0g zxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f) z%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ow>&EV6Le< znQQhg=32O$xt8x?uC;ra-^W~~_cK?;1I$(ZAam6|#9R#zGgtE?%+>ZNb9Fw(T=B=5 zEBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw z>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z z^e4%lFLH{sVJ`e`K!Y zPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK-VLuQ1 zdDzdxejfJou%CzhJnZLTKM(tP*w4d$9`^IFpNIWC?B`)W5Bqu8&%=Hm_Vcixhy6V4 z=V3n&`+3;U!+sw2^RSGgsvdb8Xy%x$0(_t8q`}YMEoM_Pv-Z zJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7NttSYfW^1DI=VmHC0pRl3Gp z6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{jUC5a zlgBgH%n8gjesidzaW->B z&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP$y{Yu zF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ow>&E zV6LeZNb9Fw( zT=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E z@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j z6CX3z^e4%lFLH{sVJ` ze`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK- z5kHUkdBo2nejf4jh@VINJmTjOKaco%#Lpvs9`W;tpGW*W;^z@RkNA1S&m(>w@$-nE zNBlhE=Mg`T_<6+7BYqz7^N62E{5<045kHUkdBo2nejf4jh@VINJmTjOKaco%#Lpvs z9`W;tpGW*W;^z@RkNA1S&m(>w@$-nENBlhE=Mg`T_<6+7BYqz7^N62E{5<045kHUk zdBo2nejf4jh@VINJmTjOKaco%#Lpvs9`W;tpGW*W;^z@RkNA1S&m(>w@$-nENBlhE z=Mg`T_<6+7BYqz7^N62E{5<045kHUkdBo2nejf4jh@VINJmTjOKaco%#Lpvs9`W;t zpGW*W;^z@RkNA1S&m(>w@$-nENBlhE=Mg`T_<6+7BYqz7bLOhZF;{iKT(x=TYA7&Q zbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMaGgsvdb8Xy%x$0(_t8q`}YMEoM z_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7NttSYfW^1DI=VmHC0p zRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{ zjUC5algBgH%n8gjesidz zaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP z$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ zow>&EV6LeZN zb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4 zN?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$Ts`kISN{jhHS{5K zjef*j6CX3z^e4%lFLH z{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>I zW3JK-Q9qCRdDPFNejfGnsGmpuJnH9BKacu()X$@S9`*C6pGW;X>gQ2EkNSDk&!c`G z_4BBoNBunN=TSe8`gzpPqkbOs^QfOk{XFXDQ9qCRdDPFNejfGnsGmpuJnH9BKacu( z)X$@S9`*C6pGW;X>gQ2EkNSDk&!c`G_4BBoNBunN=TSe8`gzpPqkbOs^QfOk{XFXD zQ9qCRdDPFNejfGnsGmpuJnH9BKacu()X$@S9`*C6pGW;X>gQ2EkNSDk&!c`G_4BBo zNBunN=TSe8`gzpPqkbOs^QfOk{XFXDQ9qCRdDPFNejfGnsGmpuJnH9BKacu()X$@K z7xnX~pGW;X>gQ2EkNSDk&!c`G_4BBoNBunN=TSe8`gzpPqkbOsbLOhZF;{iKT(x=T zYA7&QbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMaGgsvdb8Xy%x$0(_t8q`} zYMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7NttSYfW^1DI=V zmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiXT)js#*T6B% zHGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8RxW3* z^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@SZ(**!TbXO{ zHs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0 zF@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$Ts`kISN{jh zHS{5Kjef*j6CX3z^e4 z%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of z+P}>IW3JK-F+Y#_dCbpaejfAln4ib|Jm%*yKacr&%+F(f9`o~=I1d#kNJ7b z&trZb^YfUW$NW6z=P^Hz`FYIGV}2g<^O&E<{5=I1d#kNJ7b&trZb^YfUW$NW6z=P^Hz`FYIGV}2g<^O&E< z{5=I1d#kNJ7b&trZb z^YfUW$NW6z=P^Hz`FYIGV}2g<^O&E<{5=I1d#kNJ7b&trZb^YfUW$NW6z=P^Hz`FYIGV}2gGgsvdb8Xy%x$0(_ zt8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7NttSYfW^ z1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiXT)js# z*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8 zRxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@SZ(**! zTbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B z$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$Ts`kI zSN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~ z%l|Of+P}>IW3JK-aX*jydEC$AejfMpxSz-UJnrXlKacx)+|T2F9{2OOpU3???&ono zkNbJt&*Od`_w%@)$NfC+=W#!e`+3~Y<9;6Z^SGbK{XFjHaX*jydEC$AejfMpxSz-U zJnrXlKacx)+|T2F9{2OOpU3???&onokNbJt&*Od`_w%@)$NfC+=W#!e`+3~Y<9;6Z z^SGbK{XFjHaX*jydEC$AejfMpxSz-UJnrXlKacx)+|T2F9{2OOpU3???&onokNbJt z&*Od`_w%@)$NfC+=W#!e`+3~Y<9;6Z^SGbK{XFjHaX*jydEC$AejfMpxSz-UJnrXl zKacx)+|T2F9{2OOpU3???&onokNbJt&*Od`_w%@)$NfC+=W#!e`+3~Y<9;6ZbLOhZ zF;{iKT(x=TYA7&QbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMaGgsvdb8Xy% zx$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s&c7Ntt zSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iEb9EiX zT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpdYwl9! zTD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xufxS6@S zZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd z0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r<|@3$ zTs`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcPHTxHH zE&R<~%l|Of+P}>IW3JK-2|rKxdBV>VexC62gr6t;JmKdFKTr61!p{?ap78U8pC|l0 z;pYiIPxyJl&l7&0@biS9C;U9&=LtVg_<6$56MmlX^Ms!#{5;|32|rKxdBV>VexC62 zgr6t;JmKdFKTr61!p{?ap78U8pC|l0;pYiIPxyJl&l7&0@biS9C;U9&=LtVg_<6$5 z6MmlX^Ms!#{5;|32|rKxdBV>VexC62gr6t;JmKdFKTr61!p{?ap78U8pC|l0;pYiI zPxyJl&l7&0@biS9C;U9&=LtVg_<6$56MmlX^Ms!#{5;|32|rKxdBV>VexC62gr6t; zJmKdFKTr61!p{?ap78U8pC|l0;pYiIPxyJl&l7&0@biS9C;U9&=LtVg_<6$56MmlX zbLOhZF;{iKT(x=TYA7&QbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMaGgsvd zb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq(rk0s& zc7NttSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$m@9iE zb9EiXT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`VuIWpd zYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F$(xuf zxS6@SZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOzFjx9r z<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ**VLcP zHTxHHE&R<~%l|Of+P}>IW3JK-Nk32edD735exCI6q@O4KJn832KTrC3($ABAp7isi zpC|o1>E}s5Px^V%&y#+h^z)>jC;dF>=Se?L`gzjNlYXA`^Q50A{XFUCNk32edD735 zexCI6q@O4KJn832KTrC3($ABAp7isipC|o1>E}s5Px^V%&y#+h^z)>jC;dF>=Se?L z`gzjNlYXA`^Q50A{XFUCNk32edD735exCI6q@O4KJn832KTrC3($ABAp7isipC|o1 z>E}s5Px^V%&y#+h^z)>jC;dF>=Se?L`gzjNlYXA`^Q50A{XFUCNk32edD735exCI6 zq@O4KJn832KTrC3($ABAp7isipC|o1>E}s5Px^V%&y#+h^z)>jC;dF>=Se?L`gzjN zlYXA`bLOhZF;{iKT(x=TYA7&QbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMa zGgsvdb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3?8sCq( zrk0s&c7NttSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V&Rp>$ zm@9iEb9EiXT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_-xh5`V zuIWpdYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM%3g5_F z$(xufxS6@SZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TYo4KOz zFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AYT;qQ* z*VLcPHTxHHE&R<~%l|Of+P}>IW3JK-DL+s7dCJdIexCC4l%J>kJmu#pKTr92%Fk1N zp7QgQpQrph<>x6sPx*Pu&r^P$^7E9Rr~EwS=P5r=`FYCEQ+}TE^OT>b{5<97DL+s7 zdCJdIexCC4l%J>kJmu#pKTr92%Fk1Np7QgQpQrph<>x6sPx*Pu&r^P$^7E9Rr~EwS z=P5r=`FYCEQ+}TE^OT>b{5<97DL+s7dCJdIexCC4l%J>kJmu#pKTr92%Fk1Np7QgQ zpQrph<>x6sPx*Pu&r^P$^7E9Rr~EwS=P5r=`FYCEQ+}TE^OT>b{5<97DL+s7dCJdI zexCC4l%J>kJmu#pKTr92%Fk1Np7QgQpQrph<>x6sPx*Pu&r^P$^7E9Rr~EwS=P5r= z`FYCEQ+}TEbLOhZF;{iKT(x=TYA7&QbCJ2)x|pkT6XuF{Ggo#~=IZKUuHMaGgsvdb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)Tu8}3? z8sCq(rk0s&c7NttSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G7;|+V z&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l<{G_- zxh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYWtNjM% z3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#TE#~TY zo4KOzFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EMbB+AY zT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK-X+KZGgsvdb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D*xd!)T zu8}3?8sCq(rk0s&c7NttSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3q0H5G z7;|+V&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJWm}}@l z<{G_-xh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uyu9oYW ztNjM%3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$H<_#T zE#~TYo4KOzFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN%r*EM zbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK-89&eXdB)E(exC93jGt%xJmcpX zKhOAi#?Lc;p7HaHpJ)6$Ggsvdb8Xy%x$0(_t8q`}YMEoM_Pv-ZJkMOoy_qXmV6N_cn5%D* zxd!)Tu8}3?8sCq(rk0s&c7NttSYfW^1DI=VmHC0pRl3Gp6$dd_^*VFa9?V<~hcH+3 zq0H5G7;|+V&Rp>$m@9iEb9EiXT)js#*T6B%HGC{{jUC5algBgH%n8gjesidzaW->B&tb0gxy)5KkGXozXRiJW zm}}@l<{G_-xh5`VuIWpdYwl9!TD*+8RxW3*^(&ZP$y{YuF<0f)%(d|v=Bm4vxf-uy zu9oYWtNjM%3g5_F$(xufxS6@SZ(**!TbXO{Hs%_+ow>&EV6LeZNb9Fw(T=B=5EBgd0&^|B$Xu&0F@Kr4N?u{E@>iLw>NV!7d7ZiH-(aq$ zH<_#TE#~TYo4KOzFjx9r<|@3$Ts`kISN{jhHS{5Kjef*j6CX3z^e4%lFLH{sVJ`e`K!YPs|nk%v{~SFjwEN z%r*EMbB+AYT;qQ**VLcPHTxHHE&R<~%l|Of+P}>IW3JK-SwGMEdDhRfexCL7te*rZN&-!`R&$E7>_4BNsXZ<|u=UG3``gzvRvwoiS z^Q@m|{XFaESwGMEdDhRfexCL7te*rZN&-!`R z&$E7>_4BNsXZ<|u=UG3``gzvRvwoiS^Q@m|{XFaESwGMEdDhRfexCL7te*rZN&-!`R&$E7>_4BNsXZ<|u=UG3``gzvRvwoiS^Q@m| z{XFaESwGMEdDhRfexCL7te*rZN&-!`R&$E7> z_4BNsXZ<|u=UG3``gzvRvwoiSbLOhZF;{iKT(x=TYA7&QbCJ2)x|pkT6XuF{Ggo#~ z=IZKUuHMa4A6?SBsK(plDW#RVy?=onQP-U%vEUxs7 zdY@vhfv1^k_!;IJdzQH-pJT3>=b3B%1?F0Mk-1i1%KABTmAt}S<*zbV)oaXE^Ez|Y zzrkEhZ!%ZwTg=t*HgiSaVXpMM%vE@gxq9AbuKo|0Yv@Df8vTg5CO&4a=}(wz?o;Mk z{EWF)K4-4=FPMMHTxDM|SLN5twecI~s{59?8oy(%mhYLX{Rid>|Hxd)pO`E7nYp@u zVXnSknQQPj<{J5(xyJusuBktnYxXbZTKJo}mj7X{wSSra$6Tcwa(*KhODj&d+mxp7ZmZpXdBM=jS;;&-r=I&vSmB^Yfga z=lne9=Q%&m`FYOIbAF!l^PHdO{5*KhODj&d+mxp7ZmZpXdBM=jS;;&-r=I&vSmB^Yfga=lne9=Q%&m`FYOI zbAF!l^PHdO{5*KhODj&d+mxp7ZmZpXdBM=jS;; z&-r=I&vSmB^Yfga=lne9=Q%&m`FYOIbAF!l^PHdO{5*KhODj&d+mxp7ZmZpXdBM=jS;;&-poXRpgkfI$*BaJaaV^n5((STy0&<)wv0C z#k-j+yD4*Z^)OfOX3RCv%Ur{oGuK!jb4_l+Tr>U5HNPcuEe$Z&>Q>ALnX6=L<|-dz zuBvUAt7e$F>bGUCrV-|9-Hy3BMwu(RJ#(dZV6MWB%+)i-T>U#S*U&g~jqc1`6BEoe zy$f^AO)}TwuFSPE#a!#VF`s6xvfY`ha)!A!?!jDjv&_}FCv&yTF<1Lu%oUzzuH@d# z6)Z4U_dd+kx5!+B`!d(a5_663$6Qm(%r(0|b1kee*YW|(wYJLqK;|l4W3GyWn5%l7 zxoQt)u7*RHtNBpoYCDX%IuB>A_z}#NJ(9V)j$*FfqnT^q80H#2mbu1`W3I{LnQP_* z=9)i|xt301uGN#7|Bty!PGPR{Q<d}Fjx7j%vJRo zbJe`gT=j1-SJRu!)%q55b-c}7(RY|D{VsDA-eazw_nE8z1Lhj~khw-bVy=mgnQQtJ z=9>GIxfVZTu9eT3YyAu6UouzOSIkxUHFItJhPmp#Wv<5Wn5*S`=4$_exxzm(SMn$3 z3Vvp;?q8Uz?^os;{EfLrerK-nKbULkPv)Bai@6s5X0GLbm}~7{=KnEQ>4w1113wS^ zJn-|t&jUXX{5;su@P7h75Bxmv^T5voKM(vo@bkdW13wS^Jn-|t&jUXX{5FRgq(^>VUaw^UT#yV6NsObG3CbSLY_o74K%Q?551s z)x%u9n=#iwFLMoV&Rk=C%r&_MbItTK*Zh{uwKTw7t6MQ2WUi8}nX7z=xvI8du9{)y zs^6Bmnnswbbvx$j7-g>L_RN*ufw>AhGFQ(SbM^1UTtnl`HM%o%O-wM?^e)UbH_2R! zyE50x6mzZb#(bK&%64b2${FU`xCe99%`#Wxp3K!U$6W1uF;{q=xsrP`SFpfb-TN?C z-y(Aj?#o;wOUyOCA9GDDGuQ0?%(bw>T+0VA*V-!c1DUIIjkzigVy^0S=BhoIxf%{( zuI59TtL-r6>O7pe;zux7_DJUHI*Pe^k7llcW0-6BSmqi#j=3g}XRetOm}~w-=2|+5 zxmHhR{y*j_Ifc2(Pi3yE)0nH~bmpo*gSnc{WUkh;n5*M#=8B%fT8FQ^%&RpwPFu#(y%C2Ir%Bz`c<2B4xcP(=@ zUdLQ5*E3i94a^n3k-3sLF;{Rib9LXsTz$7P*Whi;HF7(1jo-msQ+G1g>|M;Ya5r-; z-@{yM_cFhaxk~S6u8IektNKCas(pyL8Xjh@=0}*T?NR3He2lr`k26>H3Fhj0lDT@H zVy=OwnQQnN<{EpJxh9`uu9@eVYyJi1T6&SWR$pTNGIN!@!d&IAGFR1W%vJL`bJf4W zTupB>SL<8M)$ulSMc-kr^t;Sec#pYy-e<1<514D{L*^R&h`A;{X0GW^m}~A+=34xW zxmG@BuJtdNf5}{BUoltZ*UYu?8|JF}mbn_gW3HC(nXCN=<_iDFT*;r9EBKkYx_@D= zzF(PZ@Hgff`JK7O|6s1EKbdRxFXmeKo4J<%VXn1*ng7RJr5o~op7-;-pXdEN@8@|x z&-;1a&+~qs_w&4;=lwkI=XpQR`+45a^M0Q9^SqzuH+22qyr1X&Jn!duKhOJl-p})X zp7-;-pXdEN@8@|x&-;1a&+~qs_w&4;=lwkI=XpQR`+45a^M0Q9^Sqzu{XFmIc|Xtl zdEU?SexCR9yr1X&Jn!duKhOJl-p})Xp7-;-pXdEN@8@|x&-;1a&+~qs_w&4;=lwkI z=XpQR`+45a^M0Q9^Sqzu{XFmIc|XtldEU?SexCR9yr1X&Jn!duKhOJl-p})Xp7-;- zpXdEN@8@|x&-;1a&+~qs_w&4;=QjkMc|XtldEU?SexCR9yr1X&Jn!duKhOJl-p})X zp7-;-pXdEN@8@|x&-;1a&+~qs_w&4;=lz_yDss$K9WYmIp1B$d%+*|EuC^}b>fD66 z;@!-Z-ITeydYG$sGv*rTWv=1PnQN?%xhA(@u9<%3n%|PSmIjz>bt~qB%vG{AbCnM< zSJgJmRWr<7_1iL6(+G35ZpU05qs$fEp1IOHFjrwm=IR+^uKt~vYiOLgMt5eei3#SK z-i5j5CYft-SLRxoVy^Ywm`^iT+3w6$Im28V_h7ENS>|folet>vn5%s+<_gaorg15{0QdC9?4u?M=@9L(abe)408=1%Uol}G1uhr%r$cY zbIqT~TuUc0*Xqg4|HoVp7WWj{{rS3x{$d>FJi8VizJ$Mdgf}sfw{srGFS2@<_d0RuI^iytM69k8oZ6UMs8=W@jIAn>Q3gG zy^FaP?q;s#dzfqOUgq~PSLyxCRq+6GRX@mFwGS~@!^6zg{0MWkJ<432k1<#LapuZC z!CYNWGFR_Y%r)>da}7VkTw~8N*W`1|HS;`k&A-50OD{6l>PyUDX0DP~n5+C%=Bj#) zxoTc#uKG8atLaVVYJH2jI^Jfk=sV1newVol?=e@;`^?q<0doy~$XufzG1tV$%r*T9 zbIpCqT#KJE*UIP2wf+V3FPW?CE9R>Fnz=T9!(4UWGFRhw%+>NebG84#T;U&?EBO<1 z1wS)a_b<%V_bYP^{>EG*zcbhPAIvrNCv(mI#as)2GuQGz%(eC}^Z%HubVI?<3w~bk z^Maoj{Jh}j1wSwNdBM*MeqQkNf}a=syx`{rKQH)s!OshRUhwmRpBMbR;O7NDFZg-E z&kKHD@bkikvGRhS7yP{7=LJ76_<6z43w~bk^Maoj{Jh}j1wSwNdBM*MeqQkNf}a=s zyx`{rKQH)s!OshRUhwmRpBMbR;O7NDFZg-E&kKHD@biM77yP{7=LJ76_<6z43w~bk z^Maoj{Jh}j1wSwNdBM*MeqQkNf}a=syx`{rKQH)s!OshRUhwmRpBMbR;O7NDFZg-E z&kKHD@biM77yP{7=LJ76_<6z43w~bk^Maoj{Jh}j1wSwNdBM*MeqQkNf}a=syx`{r zKQH)s!OshRUhwmRpBMbR;O7NDFZg-E&kKHD@biM7Ggn29xvB%^s?9T3LxH)Pi_F#5 z#ax}6Fju^rxw4xwS62^n^=`&o1HH^Oyg74?^)c7v7R)u%&s_6cGS|`obFFU0e2}?H zwq~yKA?B*uhPi5nnX7(V=4u*YuGa0Ct7DY8qT4f9dI#nz?8sa_W6agR6LSrXGuPOGpd299B_;bWO=>^SC{ zJf68`PGGM26Pau2B<5N@nfd>itK<~sDnFIEs!n6Bn$wx9{tV`7I+M9t&tk5QvzaS; z4s)f?Wv;?`%++%~bM;@qTtgQ!*XTveHE}U>O<%%XbC)vL;$_UWayfIYU%~uJ<|@02 zxhk(_u8r3)SKYPD)p#9qwOr3!?Kd!2_(tYR-o#wN&CJz(3v>0|%3OoDG1tiL%r$-o zb4}gJT(fsE*TUV*wR{hAt=-G~KISUDpSda?V6N&1nXC38=4yDDxtbqguC_;+tMf7D zia*X=*(aE*>q+M7eTumTo@TD$XP9g3S>~F2j=5%@XRi4dm}}`p=30G;`OC~z@(Od6 zzsg)yuQ6B6>ฦHvN$y}{(F;~ai%oTlyxzg`4SK&S8>Up2J`afW6jbCqr= z`gzgMi+*18^P-;@{k-VsML#e4dC|{{eqQwRqMsN2yy)jeKQH=u(a(#1Ui9;#pBMeS z=;uX0FZy}W&x?Lu^z)*h7yZ2G=S4p+`gzgMi+*18^P-;@{k-Vs#SMq3V*b|1)%#-C8nH0EV%# zv9YnSv9VsUv9a-ZRP3?GI*yIU!p6qN#>U3R#>U3R#>U2epAYvhxIWC9wPyB?)nAC2 z#~RWyW*!T*ikZh6*E(k2CT1QhW!so}tPJg9=CQK2kD15HCC1ESQ5|CDv5LmU%wv`6 z7&DJmK0an1i|rIMkM&>Yn0c&vU1H|38cQ+rSVGsBd8~HbV&<{pyT{C9_2>~ZkJYzl z%sdwC6*G^O&^u-xYYd2)$4cxIGw&NSkCm=p%sf`+{xS1dtJ^>x~e@Euv#Ks1)JIPY?nTiS*h#I$?)`@k|Mt_U5 z!`!Xymk)ts!b$clbpgK;Uh{6*ca4YY6ZArQ&A(?pk>9YN!5=q?l|oOgq=%WrY+Oz{ zFHs;W?9)yulhG>5ET|!HY(0v{n}#jC)^>ZNquK>^mwNO4m_g)FY(y~19cLx#QjvC`n)nF*TQ-&O~I3NI#E5nT0i4E(I8&BFj=Wa4xLtHME-; zE!5Viz0{HK!gMElWBr1G?hq?MAEk_giQ*J)Iz5}17cKOcI4jK6+Io2t*e2{`_fiM& zBcbh`vK`~1>Y^Kx$KPjs@);HcZ`=>oXZ^eK3;q+6ajEFEM8+typWVr2=F#3R~D6)NXu#c+fj)pD@m-=h0>98h?wqOFqP&1kc@9);s;9@)iCN|8Pm@6h!JM zoliJfO;XDvQ=qUwv!y5ouNYSGs@t`Ux@tqzL~6meX4;b-u`WS3x0lsdAE*q03F0Vj z9Gyr^iKh9poVn&gZHc@BtQOX@o2YI0&Tx-s*@q2VJ%t?UBJVOcNDsRoJaV5}FZDOd z2l!e1&i$hQ5y_&Iei|o(nOVy&=K}eJf^1Q$1YRaA=T)++8a370sJ_&QZ^pDF+hQGp zcvrG|=s@WYp*V~~^cX^pCi+vI8Ri^qzPuPL6IQWnsg3xSaJ#q5-e(+CkD?RO8U8$T znY@PG3huZMtjGFuGo41rSpg~JVJWIGPKPDEvUUZdidr4j zlIrpenI>cltaZ@Njk7xI-IQLiuQ-q!LMITTqOtx2XRn>D5q9JRi&DIZKgii2x}GyZW~L~;}r?^5CPYphQzRFgg@FDZ%)*v$}_+m zVLrQB-qbM6)Wj`$dT@gwKA ziD}7Y9HbKnHY-KqdBXf&A-kASQZ0)rNLBdiOf9l5)-Y)7Hn&>o?UXp!S?tF3qWcm9 zqrv`gXQVk+n;=gH(}Y>4V!_=jAsvXdU@>vMiI3*DlL`cD=}5cnpo|ip37TJ zbwOzZMKPX}=pF=!`uT&Lq2>r}v^*Y66sEE>s5$ujaFMsvUTLgRH=xbZc77MLk35JS z4UW5~t#kS%-r*k5kBR3|;J^sVH!5XpxAIy7=9l-ex~-kUSKO5Jt1( zsfqa1aJo0!o@Xplm!g%@8h!(_ncR-;3ii4OtRwny|r`AUv00s-g*^$&(d_p+MQ|y_BsxCm9w4B$Ob)(JXkVUZKK`EE9D(b9K z19D~*^X2fynEQo>uV1*hNuZ>lr)Y{WTueQvDv{~ccHaJ zU!kmq>%~pnHhL$qH?sW0j%}XO9Qh(}g&V9#-N*g#srS-;V|-9Qqwmr${vVT!OogQl zGPs$o?0POGA1o*q{Z)vDNr)fA z$mAqU31+x+toiz4Wf@#0uH`n;TZkReE`OhM&^)T0kk5eg!e#avbql{6KJXsf&y82= zJM>Zd%Ku>gkV&u<0q&-=2tBJp!aO3y6{cyTR8-cl;8ZcIYqjLMprO!&Z9%oh+lO&p zXS5xO{X$qG(jyFYS~wD`{2b znxM8&pKU}n!&`=JJkgFfB((O6iqyz1Sw?-&o%$LP8Aihsv^B)?)mg5NG?CD(B!9VEo897*RP zsHl)%%qeM>)hftUKy{%OTbF8xHwl}2t?YJ2oZ1<6lX~%enStaGEFl=_jglWVKTxL2ukt@pU7jTN0#kJCMIZ#Qc%GRW6aTRxm>aas@^-LG*vB5Ej^Zc6)80Azl5thNiS9@b z_{Yq1@)h zQeFc#2%FjM)GmBqc)&YiA2&{`=g=kTDu0u?Lq5PB2hZHVdaH-Z7Z{1LUk!9}0*})A z8J#R<4lTEw9~2Ubu_dXpc!jXCSIz#YAbGryS@GPA>){O5}lPU@K>1YV}i9@*&^k`yyl;}@!rkk_1dGaE#R9MNbp*G-~!)@M9d#`aoJ%Wx)r}=Zt zCGskEGw|H|majik0{B)8xi54?{Eq(l$(&SXS}miT1>_KNv-zn)c(JgASH>=HR935@ z|D-y61Ew+A9BUP{bvsy{^sY)z*hd_|4W@?^Bcn0C>`XEhZKkY(1%k#dr*wQ>xY65U z?=W_&`_Uok7=MyEOJ2aP1lQbK)?NLf@&vvRUvuy2PsF$AhyTY(Vy4hi%jrQTAsd^M z%8M5W3wyL(%3#!rh?Q#a98-_vv8F)_x3$$?@2GTv-NoKqKY9=`G)nMCIpfSkZHhb{ z%ogUci>Rge%5b%}-ri(vQ+J}h(gFSmbDTVloeM6yu60BAl>5*ZpK$^GmI$NI{&(k> z`A+sJv8}ug3gG#vVNdjofBdOTDeq0d^9*ay{uj z#DEC;!yIIe(PViNP=uMRN-e;(aGAHtUTbVrx1b%;Zhk*=h&+a!49>Xct;_l~CTk&}QFLTWZWl?l%ll3pI0G776SDkU*|MTRA7U|dkwZD=*o zTPUqzd$A+eh3-!Dj{5ooogrp|HcB1`5``)3bZRy}FI?y?u~!(Y)%9qTw2j}%>?IFi zM*`bDWjXpq#f3LSkGoI%#Ixw7|Hk=Xe%8Lrzra5s8Jmhqi)Rcod)e(=Mn1J5Dk_!W z%P{51%2>6ard!*puQyVf!IolMt^?hP=oi z(|LrQ)gaY8h>{BPG*gOXu!=zyx4KnJud6hKO~e*lYq~wrG3xAhb9$M5wSn>wkRXg= z$5Dy+lyI6i%bsg2RF|L?(rSJ^vx(e>?F{y~mUURSl~d3WFLEw@gYcqz{v+q9`BHl$ ze*m9_@9Zz?AD%2s>7}tV7@5`VD3_FvFUS-nOJHS!a&9H7s$Ns64eN`IxMp-qqHQGl z@s4En(16?@Kw%h*s4=)4PV}bQGmJUve6(0v#;;=5k{hut!FG3-wNF2&9EB&uGu(Ol zGI1@s>ECf4n2)vR@+lj1Ee=WvvQ&6{R|? zCD!E{(oKjKQ7gZl6K8hTy2-sjUtu6Sgi63ig=4)5_GDw4It$H}7V=A&736AcePFm- zEmPm4Sn#lDbEjyBxENjWuRFKRd)g!UDR?QoVLwox@$cbJ@2{QINU5el8Klg7b|x2@ z4=WfHaf@4}^>RujSXHda)u!tcjiRQ$;IuJCEnb#D4*{_KDTogXM|h*{@y0}TDw-k9 z;pa1p$z|B8V2!)M+N^I^cENq(LGCDhf;bbM^DjA9&70aC`2lz=JZE1~@9>Y|7ca7Z z8?#CPS+ypMh*PNPII%B)=rKCorP{}FRCv- zFdXa+w?`Ud)d^^_G>xCd%q16MOM>ODZmrV|Wh*qrJ)A`!ChX{>f7ZERUeT`0x4}K( z5&M*SiN6Wod!OuY#!vMxN-Cw~(=Zvx%vkmyr<>O*pche!!_s0ot`c39s2Op7J%=}& zYJ%Jbh(bIoQ9W=F_VWhWLyZyYXf$4$$WLWvkaMv4!6J95wNhWBY=E1^?c6SUA8{}` z;vaWTo9DDk@>OtCxWhi69^=o$zTont8p}^$CX>^!S%K;j)#-;_q;FoBOpx@-c8yILlt3 zuHe_hTi#v!q47k0fnH1R`A^I@@+bB;NaCijQtRoJOfZ|6lgmpNAc{n^U&>+3iW)1| z0Gv>d<*BB)5VrQ(+Z~NAYIoFI>cfHt@k%M z+svKXUiknxA{=K=Q|ItYq3hkSJ>$OWqi0gUzhy%53l;^x+<#UwJ(ZFcW)!n(aOt}K{eq&whq+*ZyYxBTH0-m4r(XVRqDz2VFr+cvEc!7$5^sHNm1ZT zQRNoU8nHZD<*#)%np?CT@@}wSIK&>KPU2_7^WJ6qnsH0Liylf(_!rD;@;&w``0DOT>xB)yCUy&>wb~wal)CWUncieS zY)~-7O|VAk>0Zy-;) z&-=_XGQi#jAKcH@cm0?04<-{+acSv{M3yMKpUcT-7SxK$B|sUWJX@KH4RQ=?d-d%` zMl-b~YAbc%J273!o>-rtzYDElI#R|!S)9Zv^h`pH=KG7CW#%evt-KLz5q7Y!T6SHdq1ptskXrNYnT})^tb5SQ?Q0FxhbReflsJw{q^A(mqgno3XQ8=7 zTOqFo>xE71HfkrnH?+LNwr!kJ9duE0`5TN!-pBmlsr%A;qkm97!|&oR?jN0uNEM~= zGdP*e>{>24A1EjkWlKhe#47_i-nkNoZMR666smwlGEGrNXja1-pt-U9E-cN)7oYObfC#);@@H zJ6qlKUP@m$P#nS~(4&ZP(FA|8GtHc(&6O8|CBh1JHMJh!6mIoQdyiqMhmkFv;vME9 z>0&p6+wMK!BzLBbw_`oJciH3SKK@LBk?tg{ND~{Cf9H|9Uz3PEJ@|T zsj!e&%r0q^RV$z>Qgyx-QoYEO~6MJ!e>4C(MXt+Pp8Ea0^Cd<>n zEMYFYkXnMT2z76rZ5Ufs6YY^K{xD;cr!XhD;9jw=>$jDA@R9hGdr7|`K184VZ_ZEi zua;C!3DO7|*vwRRJXe_4D_|EfimRnjIjIs~m8nVA#_9*W+td>DHi`)2MTzS{1EPO4 z$RFyAFh^_S<%wXbFoT^#&BqsqOTCr$8e@aH8Euz#@%xyAx1HZ8q!vRZrLuelrV3dds}yROB(h zW9~`otbRec03Bkly8FGIe&!%;s5}CU7RIv^sj2vkaJDzkUSup) zSE4o227WWMo!o`(3l6wPtmFD=}<_^Q{)jUqB@MW-%%`j)!Tz zjCK|yhngGZmkRO4n37~!tU^%Pt!DkF*HIe4#$t1>72S@Ai#qvTot|bNZGb!&3>QYS zW2p)FWWP`m{yY5VC9_i*Y1ND|b+knRm5^@)Phvc+I}2KH=ZOAKo83iIGB0jnYe*_-ss0 zGA~vjDD2W!DVJEU^%6=MSYE8mRipnS>O}SZMou%crPfyN06Ga>*`8D% zd_V}jVKy?xs4|))Df~=EB^O{?u*_X$t<^UwTi_0HH@BZYL>!Aw_-CB+=4I`gd<)za z9o~>4g;L@f$O5~?7)5+P`ykMcb#9E=RR@TE!;x=w4y_Yx; z9rkVKl<8;}Wf$BKJoY~2}I!e>G_m`u&7vq zD?^tjDo0iQnoe!AzSc-?23iVj*$z}EyldFQ19pD{s>2YH#_%#TiBzze!5nwKwOC)K ztb%LBjocP`2eCWa=O1*AnkTd~@_BGsxW?Y1?&1%_$KG@MmGMsfh`vfc_&-b%G6j}8 zNaqq(R-IJxKuRpk(R3+-i7NP2oa$yRt*+bv@+sg57g?9OfqUUS?~(n~c&WZY zAEeLxcjg!Q4@(xLbkkTF^vp_jm`lvZ6{L$2C8E-PIj53YRjVo22K9wTY%{7Q-Zm7y zcv~`hr~vhsAU}*j$}BinT*xh~+J*AQcW)L%T+38$F zzNmm-#3^o;*2>A1Kvkh8TbrtnHwv42g5Aat)p#UHJ$S(MCm}X07~zh##_JQ6sc?ok zhnr6?CYD7j{WZ=8bF;Qx-Uapv2ic?43H(fW&bwq^HEybR&;#i)|D1V6zQaBSU);$0 ztz$}Zh>Phsg3d~iQEorKQ^+i)m6XeZ3PKgOI#mm=8#eG7+s%zuYC9Asb>_P^epRwJ~Zgwc^_`ab#z#ThP<(V-3&;E5qSPaV$51o=i-OX8NkLz|^$m zvJTb>2D_Cq@jc;w?~r}WIH{gR7o;ovb>=pC4|^0mabH-k_4mpr_)YxD{iTxD@ zcw%BS#h>oXHs@)JyaB;6*~(D`hiAMU_Q0 zB#y7g@MKd=2wJ=Ct&VyZr912`_TvW8Lx~a5D1V%jXim|l%d^2eVG+BOT8Xa-*L$1n zZN^S@FFGI{;g2(?$#dAHz;$m}o_=5P;WIJd-qIoQCHn6Fa{iggv{Z6hkWt9O=Ad%p z`NM)t6biZGp>P0hm>g-g5@_G)9jx(RKQcJh0f1LP6xcyP*ftc$v<+<=~V zpY!QwL=e64KRBPw@7gcFPELqD5w@iC8RQZd8RTMTWlQEcI#V> z^kzy+*jDVob)vfxJtN@vcc3{;L-H6P3zJxdnu)97d~dP6%vhzaMH{6p{0?R}xgR?e z9Cc4vXY}*RWq3`z#oeVJ5>KM%{wwF5`BD2S{{Vl4By0*QHJ&~sysS29}oEQPb$b4WlE4`u<}7Ax2jcBudURFjl^bLOS&!5A&U1Ur-uo&{xSr^1jLS^ zWPDOM)th0@G3Kj_(K2Zjzn0lZZozg0yWD-&LH($50-h1ibC>CB#I5L#|G;@{KG$B! z@4!dlEBk}`gC_}-d$^s>Ak?ggl=AQtQ<$W&QbAd_f>lMYuGE5c#fDrHx&_fXYUjr} zoy~4qFS#!mC=6i}s8RU1aDq44o@UHa=c0wu5`G1}Tpb{ww_JCACu;Y19lTvy`3B#pELkVnu`EZfUEWUP-A6 zYl^kG`g9|rStR&v9MO!|B)JCwLVp%g!*CRi_Qu;2jj8GkG)J1xFJ_jJtFX1f26wZy zUEihbg9pW<+zI*&aXz}_Uv+MpceDrcWAI#f#lE9H;$Oqa`)y-JaurAEB!bV%kYpZ= z3JST!tde?Jr2?!XR_AKbb%};iW52o6%50~_$(=zrp%>ei8i)@GhkGOKvBm^-GMXmM z;^#68$tBo|KzG+!hQ3uX;U3ZA4%0SqDmv?5aITowwcGMN@JM*dzNFsZAHq-GH~XjY zS51miN@@5EOlC4WmMh5X7O;xw#g)>qoLGsgO4lT6NA-N(X=(~u8(9SLg2eWq0Ny_w z z=5j00PKaYWQ{C`hVIOaRJ=hqojznXn3H)Sc8aWG_8!T`&Yq_o~>!2ZSDy+-z1(J+D##77>ec zrRj1+rHJ)wIGkBe}3`Qo$@F1<5 z(aNIdP;$fkVj->=U6LppmG>(<)y)62I&uThSZL0+qT1ncVJEMv-P7o!4nTvY;rvKu zEI9$294PKgOVt-B8eA^w+&bDIwnjVr-OhgVkakQy3C;=^*eld^{C0TPduTr~UZ}6p zd+8JZjrmFb#gYao+|*WjJ(H3R<`naC1?VC~@u-x~I2BD+t08lsp1`wBDFJU2w)Z;P zU5xH(Z`4m3#1CagkfX8jL83dwny$}Q=D|hcQf?)^hS(5o^0zrV&Ar+I`3N{JoMz8a zm+-6M4bQXh8@~Ds1=3qSWWJCQ_B;6JCbLrMX_bsHiHMMeD*P8qYjR#~nF z{uAo34XDO=^RT7Y*6v_*QoEv_QXhT*GngEXjSR-PvNcIpl$lT!7jPQAoY14S{zhku zxkKA6?+1s3W9&)lEPf%p>|L{O8F$r(=!x`#f6crnKVjd3AMPJ3iJn494bzL6xNLMz zB5y?bg&o=~r7?0vzzQ{3j;e?AVH2-~-P&lcc0^sI?tE{iA2|pc8YH-*tZ{mxG6hZ- zXLIxDMa0r*g}>TaZ*J1I$veSb;Q)JtI*y+X9q*#;8aGrA-Isj+8559iu`u}Tez$(< z|CD4fm6(>xNM|8(M7jKYPC>J%RzfZV$_tg*YSe#tov^;w$Zlq|RNJBsQYXGE)06Cj z4G5q+%tHDYMTV0^g_}vM#DZwCzsy->uGKcmTfh!sH@lxYgdYn}cxUYM#%1*yx+UG^ zA2LtK7uf6Io%_-Hs{c^_z$9V{E;XH=$P{JuNhgm!>|{4{Y5C-Wpr}xS zEkl*ZD~DCRns#lYzS;;ilUnj^nGR$ptZUH21y+9@D#IWW$8a(|iBO^${v2n%xma5! zuL5g@jqDa`2fjPp=N+_<8Yk2<=)829zsB4m?_v*w$L@3MmHtlo2)~LyxIc6fB1MGz z=^Vn$s*!RYKnaCenkt1eVFj;>UEQdq)d8F8HZIHost~>BIA-bFfX|0KC+(bFO@g& zgZP>IPX8kQMM?dXP8u_VmRZgYatZm^f>cqwL|ED@XIC<+sx?tjY@r~hTZ@azA*ry&uN2L?|8Rk5B z8M_wTbnjRX^vB9`_)2`oeWbq~g=eYt`35F#NO>5p|Ln3J_>@+>e{SjaA+R^Y3{b)I2w zHB5C6vZTYj&72|~>|$`my>8vs?R?DGEQdPbtQ=6=hH42)#g4ISBm3SzLJvc!3Ctx(xAK{EP$7>VisbGdM zhn-I?#+QXFy*2g*W3##)?UMHK2brVf3G7U8&b?$^)o&_y-~;h7_ndx3yo*BrixZi@ zHB3$pa3LK_P+4&@%_;Cu?DKg9dJ6tGV7vX$RxP&RjRT7tuEw z;170&nqV(Ie{Z_j3lBL$wj|XfR%w$WEnZ;B&%x-XeRcu~J=wHb|TK?aVH6A9gS} z;vTn7>*th9@Tz!|yF))99!Jmoz-25nUi8p^;=C|lYwzVx;G6K1{YxdqQ--O% z^mZmAo0=2ll?w1hnBrt=LQAEU{2FEhxf$CY>~!~92lOM#ad=uh$6cbY5;r5yzwh|wGcAzc zf>8LvM$~T{3zK=N?6gKkH4Dli<>vDqV zoy@LUPq_~mAPi=QQzP-Qq3lhv6=SBVq6L!1FK2Xe9cBbu+#S|#eZO)D9urS;XXy*X zmFSv(%eiYl)Sk#Mz-!?>`-%F7{|x_lN$eCxYBfE|BxU1sGI_}YSdoBsOIeIwQDI>X zk>l#oJkd02;kS0$n;o?-a(B>M=*JGChTbTE91io4J;spLNl1}q z@+z}{)Uf5jDtE25QQxBMfV;*0+#&iHaWXpNpLZ^s*R)&mUGPwN!oHwhiP|xCT0t*wcH+b6uPk8sor?M zaG*EDPB2EP<4~eBg`dvMCg)*`f+g+>Yqh>!*#x(VJGs5|0pdtx`==boyr{YI4d4m) zS)Y1_2jLs|N?1{v>?vy|UjKAJwnuhxCU}!lWQmW9b9J&1#W)9)*I1MVc!` zGepIxieKHSW!BXi%1uBEp*7o{>WFszn zGBzog>dvs{=<}7uaGAJ@TT5>wwnW?gUCuu9pmtO~0nP~L*~`>5{8o6!dtg5{o~y6W zJLx0;mH9#b!IA{YUEE5i6G~P{ig`GSE=P>rfYQ zH@xoMw(l8_)TijB^oIYyd?vqRzk4#b88!`+eASbc&r8BP;t zadYW~#FA*auRH5ZL)$8wV2@z2hbbFB6`u7j*jJ3}>TPsSdc;3vUXpLH55Xt*oAp!w zt0aXf#WY+7Ix~?y%IW8I3YbN-;&N$FPN>9IrE21}!+IWXH#G#c4HBhzUSfKX0P7zN za)(+Y^wG+AI8mI+&7kKH^P@%nQfH;PM%y572HS;Q>^|xsel$Gpowm;zm(;81rgVpY zz&s|OW3Pg@ZfJecBjq>5#N-@Krz40ci=V^EZRXbs$;CiPp)6a0s)AP!|MTkD4UEQW zbJR*|$Hy_9$!=J$ppQGi8mtdjM#8b;1a2}tjhGdw{sKocmutGb4j95#)};2}R(Qxe zW}h_9su$1|={kR#xko<2o(3=6*VcRelkyGz6#sHb>6Ap8D7~M_$!6x%^2!B35urF+ znkt7^3R$m)%^CGn9yOH&z6~Ri@t72JcY9m?^g+r{I6@rFji)CPQ={qrY-gUiNLwne z1Z#v1>}F~^zAN189k7oW$JNv5oOFr5%G@OHU=IS{eP#vvTP1{F#EAP%V?^>Om7mth zXlBuJ$hkp&p%7b)Dv6g3%X^jWYQ}$R9n?T-%r|FRk?pX!pp)Cx>Z$ip2Ef7MaBd_$ zmY5Jt@)c*MscH*k4J;RQb{%ElTf-gRZhOCRNIiy5N@w{C%oXxFc00K1KD3_bFO=8t zz4(dyM*k%KMoIh>PHHo~mPyVAate9b0#p&acv#9~?1~1f)Ah|$qFKhc?DPSJ{v=nk`kY32dW}|ZA zc|*!8Y|}<5l|dCHmaoBZWIc=znz${j)_QxTBkUq}=X%rqh(Xa1KfxJgj?)t5DPX!V zo1I53!k30Cyw&!4W0SfK?UeTN2bd$|aqM*9xEC!~zoB^WzUXt$=zw?|eegd!-_2jz zKRFplC8T9DQd#gEVJn*gG7%SA(XuY(F-^T1D_hJWv z!>(=&AqGd1HRiKFi<1FX11Xj7o*44KsL|?d(P_H6JP{73E7X zWytba<)Ert)2gl4R~o@)VoR<4+BUT!^+eoTnT4*bL{!XVs#l> zC9UN*GF!+U*zRDTd(b+npHR-g^WtUh8hwkn8$IwJJI~Em+B^9p_$vHh|4>Qr6d~@V zvk4=sN}@ax#TRC1vJ}Py72GOTb-k8S7d8}|a4qQ8MEfYt@9cCldue^;fnbP`z>cEE z;fdj7Z<;;Jn5!;COQaS2YGysT3ELK!?jFn14=Xl2B|6+i+9hs8xBYw0BlD^DQhozI z2%p*S)Gz#BnAA&Yr!g|9nNfBr7oU$QNEXFP1f|__RwccvQWMq|>vN6hW<<-VjW0U! zrlj?d0q8G4b{K{5G2wV`qCM4^q0T|`rN#U*W)-;>+Zb$iw_CgPeab<2R6N0*q0bYS zqpSW+=Z^V6dn`W(uY`B(N9rs7BmC`Qc5(w((;-62%9BhUlEMlH#oUrsS-pZ%1y&bp zadqj2M3bnw-^yuc#%Z19ZlIUYmmNqA!4txf-dKBrF*j6kp8N*h z3ktYJtm1lUr5vmzR^@8awTb!>?>BV>vyCRo@jw!Kuz>21!*HlK!X9mmS0|#W(hPnM zGoM_HEelq!aep8 z^%Q>@zV_bRpNwznPxMzx%BN(~kQuPdK{hw1l~*sI6oJLX(p)*Z5>Yj(;d4$slh>Nc z0%#+MY&<35J;L5zKYNfdR2_jvOXK;8%v5p)HYb?pF0z*DE0r~HgSeU7PVXZ2MF;#N z&T;d!c22$ot_nBVJJbXGarn#&?6*d!enFA+o5z^sB#xyEGP+r;9C~ggKP)5`<4V$H zi3(9=KepV{tfMuM8-wOTE4CdKhj$LUdOhtv#sGCN8vZ}~cH)4wzyNUg74o_=Ihv4> zV{$ZSBQ!!IG(saZa*P~v&pk&h#6m2@qF9t~Ar{3V%n?JFd+xb1S0>l@kNlqhwHx@y9TNL_tJCQkEi{)8D(mqx_nLnL88$R#7WWQ>>p}sA>*Zr{hr2IVl zI{q$L>i%La)0Zp%iak0#>s5<2)3u}Z{0*Fq%}uo}iXK^`t+vg7XNnVPV*k^KKTLRp>}pPC!d?l z3m13`?H7!f)y2}A-FKR@{3xs9FN50s!D{s1lvez^!~f_1^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq|4+^T=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa1_) zu|m3Xw5q?Rv$naOwt>8{uxYz_CFk2FO1QJ9+Itw9y04^n4{An~BePNQ=wOUH);dKW zr<^68+qtm5xVSvMD!SIc-nq%VMY~s6YBDXH;mx#X8*|jT(!B11CMy?ZdHh;XxJ#^3|46CCFFSSpeZl|d z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|NleE|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXT=WAwBCyYr{{kG6umvao8qMkVCyCStgeC)t}DvbwFLba!qB zm4mY(@z7wHJKP$fk5op9qdQ~j6N;15)1ou|bDRszi?z$;tAuOY3Dv}WQZhN5;!U-u z88g(G((LY>=J7Jk=EqsE$jzx8mYt94Pm3>7{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzX?FC+dx|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({Qs}~ zZ=D~_pS9oRKZSqVPPJ0rP1XnnZ(Un7Hc};N^RC=%Qx42_iU$RQ-67UceV8&_9MKtB zA6Xoo9v7Y9pX{7wo~fN9Um#rEURGV1k549q6TM0HWMhgtRhrhF(ZuDfEQucvQg^5G(He63T|Zx`>SA4dFt{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TrmJ{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(A3L9!UuoaUKMFs$zg2DichU)0 z^1AjKhM=x1iQSEwP0Ie+R`E8$KzAo=kUm%$A`a~gs}C#=Ne_#T^pAFqGf&V?mQNGT zY|pOF&o4?Y3$OIX+Y^k5>Lh7$cS>`28D-PsIGE)o*5i7r%onrHqWY!c)%1<%ZJ+s--e1{D+@>?I-m%y< z-96gN-^bbCJWxACK1?{WJ*ql3KR!7rJk>kHKHE57y-2#Odu4M?>1Q{_L2#QJT6gP_ zGF^;2v+Dm9kEKsWPy5e0&zt;z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TvVF{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzX?7vz_P#cfe7$;;%Uu=2jN zYvTvCk$&s8&0pnz*^2Qh!Rqc>);fA0WkYe3PXBt#VnDimw4=YPv%9&MwvW8OaA13I zb!dJ>a#VP%cf5U)ajJTTbawar#w{<+y!e{HcW<-;{Wc{O@9sqP{l$an!_oiz$DAk4 zr#1dR|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({Qvnf|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1!c0%2jhsLJ!#k|JEZ-5;8t%3rfT;=h9b z+!d`=^wpKM#C1A->b^z4bhBtne}J>SxudqLyt}YhyLYu;en4_?c&K-TeUx#mdc1T} z_tfU}(#g(?-QZH!v#!y7JZ%0?drap4^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;Ng1;Q#ag z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQpWCL@{5(q*g}L{dT^LK$Qu?S{HD8q9WZ%a>1;4s~SbypNDJzPrbXKo>6}{8- zqrQGWXESq4ZGgPJuw%PRwOhVtvUj+jcYuAcaj1HPbX51)#wt(D?D+J+anG|{{Zhpf zuj%;p4aLprtK;YUbhW6L-`iQ=?5p*YHxssOx30F! zcSv>#ck}kN_cr!Z50DP-9@-pUn%Ob26`bhW*6F&VoF}@SOY19&tJCYE8~mG{Tg^MP zyX1R?`&<4$|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({Qvta{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzX?>3N*Y3KQ>fJ2mF3nY5^zH?Nj&WN*js1s}Sf zT3_hjDBp`eb$+dv70c6qqaJ=wXEn2z)>~d*=-X~wZJKY9Y#nar?O^X>?56H1?cLq4 z(aVD}BR)JZ-D50EKT)y8(>qRmZgF9HNpyvOwR4?$gLbogt8hnqXT|^L|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|9@A` z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXSVcM^ruz1W^*B%l>;VI8iU$}ytV zIk7&a7?++Eo$FudTw-3KT`gZH+|b@s-IDYF`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l|cD@c;S${D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD-xh}6-F9S5 zS7T{bH)$R#pUj?)pADXOU$S1+-%#Eb-|Ku>e^Pv&ejR=1FLiz~mubu8e}x|H%GIj* zn#tPXdfo>1#>S@V7Sh(;?HZ-rIaA|30?pmm()EKBLp;1=)}xCt>DcHLf1Gocd9HS$ ze2H*HdsTI9etp9K=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpZ|YD$p7d6^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqf1?-Jw;7>&w-j}!HxHH%Xa9>I3!ZeJ zww~3WS6&id?YvRHRlJ*i5Pjl*?tE>2r!AF#5tg-oR{!KHBrAujdTZKi8|$eXNE>%I zZ8k6EY};4~c6L>34_#CC74^oNUFFmgwgGB)d~5@$!X!4-Z?h^pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fByfu2LGS`&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc|9Pq_UE1}UtIO-M8{(UT zTirXXyYzdN`^5)457& ze|GSucUmchqk{lKu=^bq!XPlt&|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6aY zmH7YsfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|4;1N&A9Tc?A-Xm;1c%=>uUWvamdZ+t-^HEu4U&eLtgWFiY>8hO_{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`Trx!QQ7EtOfc3x#TuudrJO5X z*tw*>ytpd8HoD%w$+^Y6UAt4hN4T$@QO(R}Cv(EN-aLDOu~2{#Y!+_m4Y0R2c2swjcJJ=h>|OHz`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}1M@&Eb% z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD4~>Tf!`%_qNPUztS{&0ETc2Es;Z$#$ zJ;Ru(&X(qM=Qe3MKg;4pLGHe075WmT6hG=z^%uoA>G#o3{;$p-=3m->@`}PL?P^u8 zymzvG*w^c4Z)R+%4v@C*?%3>7?w0ZY`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l|~>^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD4-SU7L#<)@aAkxz zvNNhax;QR9Av)PV%{kLNN4r42Sh&2svKpUHNG67pyvgX|wK@ z&DQ01*$y%Ppa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fByfD0so)>&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc|4!~8Yp^~<87dCz46hF<4oi=Wj`ojpPB2f_ zPLt0R&S}rDF3K-Ut_;U}6YPn`Bz3Yhr8~8W%IR4g&k7RvaVyp5E19^clh>~nZ=`QW z@A)4(pPFB2-^kw!KefM9%kt&PzhMurr@fldOYJSK-|gFMTyC0e5pNyv|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6Y? zaQXlIfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|F^LQ>N_ce#KD~*^*+V^>4DK9{$b9M=F!@5@(IGp?Wxrn`Ps?&;YHqM z_LatXb%HdpJE^&?470oAD46cX)+{|y9v9Qj{QBAA`ShjeRsRj=ZSy_tL-|wTi}vg4 zyL@T#OSsHiZvSiaPRpT7)4ig7{QaE+%|o=q=sn zGWXH;mk$&UX%DTA$d5{n4UhLuvQIV6P|udm?_ShgT6)!K&_> z*4jG%pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBt`Oh5yh0=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJe;u(;XT!Q*v01uhG{E29+0oop+g;vE*r(mEIv_td zIW#=NJIX%RI9@$TI<cP^X-6I;aJSMZ^69d~l-E#Ev z6j!{oS-YMVD zK8`;NzH+~{e$;39$wX73$TRB~HI_K3F7MG+~L|6OQ zIX9R$Yq!dG2zRyjRQKf%BoBp;c#qmo7*DCsNY8a&Y+f!GXK%*u1n;{aTc7D)Dc_1e zc7Cq;|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUNUF8KfafBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|9?rBMa%tvogQXSZ8f=<(7Ro)+92OJ*)-h3+uGjF z*g@Sz+O4~1qm}z+dVEk|xQAP&evD#?CwA<5Tya)IautzX}zPruY4?i*7>Tg3;sX|D{qSLbl$H&Dyo$K&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmY%;i2u+3=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ|8xIq=R0$$ z_KUnsSl<3qt&p#rtQxNAt!=MoY@lu|ZQ9+Uk;`o}CEhtu-90Q#-&fJagE~e%vKW<) zj>h<7om0$l+FA0s!iDX{)#dqB$+hA2-c9x`#_j5z(mmb#ni=KHY<4^+nCs577U&C= z7sQu4i|e9Tl9mzwpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBygbKL4Np&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc{|C+|=I7ei@^`}0_UG#N{Lkc{a0PE= zdsSmib!}T2%P_fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~xKI{bhBKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|8JS^Y9Gj-2%ooKRo~`6CO?P2dw<&h7%QkNORIL*Y=m;%OpG@Q zBzJR5*0)s@ap#U&4=#qJL!)8-aA$-$QX3_Y7RIzER43=BC1-}`co*0g8<(qBN!NC- zZzh(LvdQt3V5&RKnxW5BW{Yz=bL+I2pJve_KX>^5{D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z#Eq`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}EnU(sHd z-xA(!->*K-KTEy}zx95!e>Q$s|CIjeuF$MhcC$5NAz0THt&Ma^*<6%6+tve%ozg+k zV1I}+)EuS_mq!RA+as%^^W%~e!jrwz>@$sX)C;7GyO%dtmgBPt@x)+~JK36|PgSOg zGdeTtSw)gQ9;N<#Co}o~{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^ZyrW{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(&&e+eue6J+H}iLr_rs69&+M;^ zZ`B{ApS!;|ZTWZBiB}4`?i!Y$ud9gSMjff{Uu>0b6Akotat4`$wIT9QVOV=$bx3|# za%6b4cbt8Kak6@vbY}OQ=KS)a?6UaEV7xoQny61wCW})#Q|qXhp2pEEKXD#6Q;q-6 z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|9_s$|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXT=l<-WuuzDeXIawUO>AhpWZ+xtNCVkcYw)vrKvftu1_}lGR zE9qTj4N>TE{D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z(}v{D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( zkG6BGdHI55Vfcdgvc1@NQ+-Ezzx#1hm0xCc{6o;VzgeyRx6%<;>U8VXi?z~qqCWnH z&L(DmZ7X>jVPLyswQIh6vRAl|x4(U$afo`DbY%DF=GgN1?4wNtp+Sx=OUVzm~I(*+<(@-bCo%Zdnb;w@-Eq zclCC+_cHcT_m>Xr9?~3I9+4dt9~&I+o@AY>pP`&Bp5M8szO?YtYa-vj(Fx4kv{1fV zh+6(X|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({Qs!p|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih!Qmd{9LhO@mn_FQA0x)!n7Ny8ATyl?P-8 z$A<<-xJOyX>c=Z5iKlkXsGZ`x)Qv9nJ?9$J*KU*p;kNdUivQ35=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJKg{|6{D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z%zLQ^RTA411aE=KmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|C7SW-V}SPF-@Hz&Fs!@ zlJfB^jpqlMyU5D**OWqB(kbf?i%-)pqHp}~ouABKwLj#)g#X$eRnL62q*vJ6Ti@<$ z^iwyJw(JgQwkvnYc8PZj_H_5Q_R|kg4i*pX98sIaF{u@u=-bZerlXxFyTYaI71h=G zbqW8U|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{=Xmc|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6Zt@Fv=mjLGU0X=-;`Grf$nS#c6P?xxm!JyRBmdFQqI zjpFU}z34;#Q|Alw8|{1fC*jw2S+zX>H|Y`f^j5Qb8NJo@rM}&M&8FoR+1BxP!4B>& z)^7Tq%HHCBodfEF3L`x{GW}y5%REuD<zzYxA@ zzpIw!za-1T<=($`52L5Ln$)Y?yIHT?Alo?JG}yx3+S*RvLD@yzt+QvX75k=obdYa2 zhnuE$jBE)fwx?9%^0Sh2!wWtBKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMW~=Ku5m`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQzsR^uy;2(AozUD^2H9Gav?dH*HnRr3w)ZTUUn!}gQv^Ze`NyKt%Zi@nTPuKp|a z==N+@E!WJ}j@Jt|a5uI#)wfW#7Psr{Q14u*=^l~h@9XI1L7E{SE|~4;YD_*hIVBwD zon`a?`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l?qf|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1!c+3NYyMcvC9ue>Jn;~Rs(z0C^syOl_s-ihmni~psM zMNj%qJI|WWYcI*K3U9P;Rqy5>B%g$zdtck%8B5h)q-EXZ%|GP|*~;;%!J6*c)_VE| z%EscRoh@p)*fv$7oqg5W!_>5WWnDO^9Z`+UMm|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUM(mCop%-JDmt*`=`;T;uxIje4NmCWf86>wAm)(+8u6{r@?SnNMm@%g+kWw=Y(& zxGP($>T4=%i|chZs5dI4bn{5|w{;YA zXHAv&5VUq^H7p;Vj0i`1qwLYf7?uCe|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|KF1M|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUNk@1E4y<>{FdpBK39 zrIx2(qxj;DouIz8xFfwQy4SzodC+`V`=9)n@MQZ`^-TU;@?!Xk_qzR-@vi!T^hx*g z=Bx7C?8o@$;CJ^=>mPjuWo2>I&YHDQtec9_M!w{1ZpzxWvLfu<4yp#{Lz1E4FmJd$ z!WgNJlKB7pfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|IIG{pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fByeb%`v5wofzA}>8@j)r@P9fqSv{mzOJ|-y*awo zzr(r9yjQzleo%P0eWZFce1o`EmAH{8jL+`=j-<{=4$0 z_)lkrdZnV9t`Q0Tx{hdWq)GDTg4}LX4a|2+28DyYA@)#Xm^xe<(dGa1|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|2G=` zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMWWJUlbwV*<-P(X#c^6-PX;$ z7KM%4O{)I+R>?NuKyN2|kTF;tA`R^hYxw{CfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|A&?QfBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~xVc2H~thr6bA zjBY6>igxGp`mEyI^up*8{|e`7^E&MY`DWqP_V((|{GQ~#@B!~3`w`<&^$F>z?laB8 z@`dc>cyaKi`;PU#{;~3z_*LiI`iG)Pe~ViGZ>M9fq;=&r1fgB0>XUDnY!deOwz9V| z2C6$rgSvy8y-NN+|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({Qn^t|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1zx7VjJA?m?EJAFi0HzS zp>6-F{>xWPRtZ=4*0R?z`luU9n{@j(Tb2W|?PLBw|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({QrRg|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1zx;_hsz z`W}iV?%UDp;l+q_WHibj?Tj(UYNyEKgtOXnstfXqlgq=ayld_2jhob4q}#i9HdD)K z*^GE*Fx#DD&DG~A3&e$;7wWutEiIxYe(8K4tag}Fp4<&T7_{i6CN|33LC{MGxz{>%7JT~S)4yL!{B?47M2_YL~F{D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t(n^^pR{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`Tvr>xgv|(c9eRjVo*9b8sZOihMB{)5%Nf3RC{!FTz*1wa(J3| zrhSfafqJoYdH1Sjd^sVT7*7f&yHl*G`ZQ&RII}amPKw9VG@9>c<6{UXu%9N&8;) zVg70IMfi>Pz5SE%tNMrZSNFfBN7*x5E$$WccGtJ~|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO*==^{FKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|Dv*y zD0Mckw<@+t2Sz*jgPg(U5N)VDOc>rCQXQ5bnH(J+=bd1mY@DW^DV@{3pt-2LEW0uu zA53s3T9fq2$`o;GXIeeIh|^h7;y>=B=6o%a7YTX$YV}6`cJf~Mq4%l%h4GF0z4TM} z*JfF{Jo`895%hFdvwG?LfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~xGDg1x_KmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|AM$~N31t2Hc9(OTlw2K1I?YZ zLGoZ>NV`w9e|}(cNO+icqWGCu`|m|w8!OCnBP8IJ)ggnyc)jYy=}i|e5igZebN1<`L0}={Sq$=mb?F2J@lRm z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1!cDkA@%|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{(rZ#M!i3Q;(F6?jF}1U!Ihm8lMrI?VfL4q+h07DUR<fYdo*M zB)!^wqj{@*H~S#|B>3F@+WJmks{A4@>+t{i|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|Nl|*|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bzQT&zu7p>^8 z;;e42rL80P5jJf5Rh#8oCIiCly&dgcjosC~qeJG*-RGMZ%U81395fr{$I|2=8D=X^6J7`ZSQLR zyl>Jk+|1k39$;**?kMfr-M!hf+&kMZJ|H;QJ=8ivKT0`PJic>MZ5OAfPIR8{I+vQB zc8%-{H?}udx8`>wcZK(Q_uCH|53B!^9_v2YJXJoEJr}`riM^`PKYG`%C^$Sh4L{t(NyndWY+KeeHh6X6lyG zfbRCq4&^S{ZtRPc=Zob{sqit@VnR_ERN{o>=4|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+c!lZgM% z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|Nm3}3+Eg2d+jIrSK*I#dG&AJBk38g=Jm3B8|$lmrGDMbnk~w$v+d#?f?eF* ztUdL;mHos6ItSNAad>J*$M}|WqG@ZV%Z_kfdtr4+enoP1c%65HeY0_^dWUpZ_ul5d z@`3E3_>tgI_X+DM{Tbys@x{(7_2S~q^qq+R&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmY$-pa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ|2^kJ^Hc2$ z`5WQ;c4_rXzARZD{_FLydm5{$y`W+LuI^sfTwmUl-4fp(-09w9-KRgGJS0BSd9@|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({QsA|1@9d?vO= z)p*TdZFfCu1ASvvsK4xmn5 zHm;>&^Hh$u^%ZAlQ`Pp6HDTX&ST#H!k&FySd86$y##r?fXw5hr`l7RuDQTO_vaoGCu-Yjflnf4scth=B z#&C6nG_pIYIl4S9J0U(fIL$rNI!C`ixmdisb5%XQn2=74Ci#<{Ddtpdnmj|`|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ihz_pymJb|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|9_x*D1Rh*G}qSsg0ODer`j;zBVN!cr#sBC3^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqKRM_B^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;P$Yxih@SyU%{Wcu0Lj zdbImQGp}5bEsS3XUUnB-Z|d(T?~5OIKC8bh>hy=G@qcq#^KY#quOxKa)vLAgb&@{e zhTbN2e`70k8);y7r)Jl3_iV3tpJ0FYKbk8uzI8Ne^@%YG#*nvbpiRV1c{P zdO?3#SuDQUd8aOmkJ2jo(yyH#%trf7ZiT3Kyh$-Xmo^slyj_kympd&s&Gc@ROjVxa%sr_=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpZ|ZE$N%U5^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqf4O~? zajkm2bW``1W^y?tn;K6GX1Fu0+4>x1t~jr=pw5a#X&$}i7tRv1)IO3c;mh`$>ihhs zI?=Sm5V?}ipY4z?}P49C3tZ&>e*v#G18lZ2l>?rQq*}dMg*gM@XI>0~JIn+Et zJ4!xQIKDlxvh&juCp^#N|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|IhzF-{$}G|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|G&VvSiM}js(WoSp`4gaiYEtC z+^N(_)vi-37H2)&`Cj8#}$^O;&L;Xwoue)N? zvs^9f755I-cl%oX^v#ql#Q~k|>m7<+(%qsx{k@(2%mcK8+dJLn z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|IhzF!{Gn(|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|36bbN4lVUadTODWi~#Z5KMF@S(EiC%2aV$XGR?tv(hAb z+)thPW~MEYbK$l2jq2_Ez2w92Q|}A=8{>QRC+XMjAIgU6Gg!^-W%btASNe+m zI-Av76kDg;MLYPrIJ=p9YJ1E32?w+XRYrbzVur_fmVKhZ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXSVlFI+*|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|9`S{TKCN6?DG8VqWH4lN_V_9L7%8h5+`@2)OQzAIz5X0Sx#a;uBGyPA#0znUdmrh z-U#3J-m^b6K2^VvzUhA7EG>V@mc`41f88EdPkl9|m)N_re!W4lak^==g}=44owF&~EW+G#&pr3#dR=+llcTvZLX)Er8le#yp%EIPkz<7Bo_mfM z!a^*>@)Zm5{Ug8U|9CF7t}w6Cu9dGBZmNUgwls|H_9N$kZfrcNCel;wv(@Z;PV#E_ zruUBhLFZ%rGv!P1o95f{hwP{L*Wh>e59=@UUu^|>C1I7iZ_zIu5Ut}6at3!dHa1hY zlD2JksPugI#0dBEO#6^dM?YHO|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|IhzFM&$qV|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ{~y*ISDuue8lMrI<(_9< zU|y_UDqkU7RbNy1>5Wm~-{yqfyNyVFK#JQL)y#ZW@@zQUn`6J)c~gH!`9S=*`Lz5Z z`#SzM_`&_j`qlhh`$PUq__yv+^h$e2ef@sUfbKfRAa$^`ajRBa=325tsC&EHMrS|W zR1OjO|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUM3ZTSEEfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn{|_yX$c~AR3r=!Rwazfl($14F5H7CW;_}psuJL{6 z#%^HTriRkp?bK>oK0TQc&h%#4&vs_(bCg%bH=DQ1_p^`UPlGSqudQ#*AGDw3UxnZ6 zrNy#zdDO%2<@E0MHTtOoq;=YXm68uh)No5rvv=s|`tFJ$?$_}D`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}0l^8fk& z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD@1GqM9~vCt9%CJ6o}`^BpCO!8pIbQTMUm@Y?s(m63}3xb3ff84a&z*#o5hE%6^0&ed^}=FNx;R?uFLRc6dle-aFXeJ;*xLJVHB0 zK2A8Pwu;kJJ37~QoQt}yak=VA*RLpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBydtG5??c&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXc|Bk^f?jF|O=Kk72@}a^J_0dH)Ju$NU(;d5euHmQ` zNp5>tH8vlYj1MPx6YWWz$@&y!syMBAsC*=QJbp5G#(my;$$Uk7Lw;L$zy7e8n|>aB z<3XzrrzA@42hUz^1tX(u|`?>Z-TE#q|6md&h3zj@lP^iy z@E=dGdv-*lt} zN3_S~CxvI~=ZY88m!sGHx19I79~yJj&!w;0s`@^!lV8HdTVl7Je{@0VDT>Xigu(TYLQS`fG=B$2@9u6G>VA^h9$X!sAC(*%p5UEgpVk?npQD^F zj%qF`$7EN=R|nU*H&{2Dw`zCDcM12__Z1JO4@ZyrPdHC^pEF)mUzT2P^XlEaNInTm zZ=PLszSnEz7qMynSN@s(9sd`s=&o$7YV!a2|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|M$`O|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO*%Kd}^wOkBL zm1u~sI$L%%V+U22c5nBp_RSAW4h|3ZjhgoFMv$r8~q>)h#8h-M-p6 z-!0iI+}AtMKDcwZew1>octUe>IXpWvK06rcUT9rnj?u1^uNJPWuP<&&Z;5XA?{x0z z-e)|hJ}f=fCe>4Un!FHZ-fMQ=c~>u#PsFnMviv6dF8(q2+5OG>pZTZux6J?N|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|NoD`|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTAycUX{sTlR~C8vK^HU_GSG^E|M+9Ka3**@Ia+s)pqv#)-ja$@p?iTgGOltZ(_;v<7$?(x>i=5Xyy z`D|fieL-_j=qvU}w_8zs9&Qp4-ydY-HtL2;7JMjm> z$L?p=m*zLxck++I&o%#_|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{{PPf|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1!cFKH7k@muGgu3+?3MX67_dbMW0 zcCuc$fwz&pX=e+48)bWO=VsS(&upLgfMBS5n02H%OgmmaSr}fQQJj^Y7hT|A>|ENt z!njJkR=U2uu?q6rk}$m6i|hwFvHqx%h)*@omb0@t@vFg`?mN~8=EvG+@|VIl^|uB8 zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBye>DgU4U&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXc|MyYt|Kc>=B}S|MBMI$_)yny*$?D;n-rDwhoelJjlugAg znyt(2vYp~xgFW4StOLxU+F|mM!m#?d;-vJ{=nVfX=e+I(#>MKT(iN>&U6cFCjbY&3 zW`~`-^+3t>sfQQHb;I{c(Z=Hct8Cp;{Wsi`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ|JdjM^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;S3#DZBHGO8s7{+n=l7 z^8Y1&hJSni*(-Kd)>lZ&Y3_7uhB;H4B|j_7u3su% zN#BUx_WA$(fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|KD-=|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUOf?H0x-YAMZYzp1{{jdtznfrzduJ zuIJbnbzJ>&#S^b-#+MVaiSeXhvOC3^YEILp%QJ+T^^?Uj>GRP`{wogupa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fByfg zUH(7+pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?|JRIMeOD^lkE_q}FOzS=@4O%FpF6+l|5N@H|8D**SIAb1R|!^g*Ra+y z*VWdSHxxFhH!HSEw~co6cX9UU?rrR^9wZ&wcB-TEZgOI1d8gZU=Um-UE)w15vT|%T zE*>9Da3@-m%*ommd8#n2eyDgPeLQ;7f5v&f%m3&9^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqKilB{^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;Nfx&7{}b zH>-E@50a0=&%7_~Z#v)UKPo?qzcs&?e`J5f{{}0#D_N_Ut7&V(iQjAHjjIQ>tb8hI~Y}~5eA>GyT|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bTRq_A%|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|4+%MCey;{-VA$Y zXO{k~GFzO}yi&fAy&b`2YNW{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}EnZ_WAt{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^ZzF$lfxqE`s8AGdS-OCKhn9ddxlJ$!DhW57nzVJ~!xA;8$ zDw^*va29qK8H?4W(z5oSO2~UAV%W!%?EW2DAE+qekY>|zi)@>C`(S5xH)}6*U+qBo zVBzrk$YNM}d~~uu+&Qy*wlPw@P`aeOw7Md{D&hb0|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|Gzrq|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ihy)=Z&`~ zbSCPPl*!_h=HBxD?4kIP;Bog!>lyQT?IrmY;f?yO;=T04Xs-Xc^Hq1gu|QoYEoz%; zN!}*^go4-87CU`(N$D@j&3feq*+%iE!4~c|*7oMk+HUe*!oKwZ#nAMy=tzH|uW^2dm1sk{}l7JH`qLMCq^YSYBKCHc8?549sZ-QV?ykn3>ORr|?LpO{`4P!6;c?zc_Ng8IKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NQ^qI{%;l&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc|7pqy@tkI4 zd0}=*JSMo(z1q6Yyg|EJzE!xRzO%R|y)Syuf7p4f`-Jhd`keG)n^mvndGc;pc%Rs1 zXP#au--~tgbNO5LzxdDKZ}&fIMRR3sRe5z`&3dh3-E{qELw^%z^X}HhcIr;juI(Px z-ueE?LE)j^5%w{i<8=N%|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({Qu(>{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzX?6U0-R;pLgx+40EWLiZADjCrMY zwS1j$Lw!?mOL}{Br+<%gU-v=dVf8WTiT0@~&0k2e@HH>D-|ZCoCrT;KYrZMJ%YKZ1 z4t{h0XZ>mZt^FsjD6CwsQmmG)5v}E~>#X11(AY%XT-v(bw%Re@CD|j~+uPqhsB@@( zgu?&l|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|9_;&|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXSVR5PqRK07%c9-Qf(ZH+W9)Gm?72v^ou71yTMM>qMm zIJbB2H11LFlOAm2>d`z&o(fa%1v~4!rsvALV$pnDewKY1e-nJ?{%HMd{-*s;{!{q7 z{zneShU3k^j&C=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJe`v%1 z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpZ`C!JS;mh9u^$$o@@;_&(zMAM+z6#7Z;bNS43C&*E-jCZ!&ICZ#%2l2X#2l)(HoB zgYAtwo9SCA+lo6jdddIi|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|G!Jd|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTAYrJQ$k9&YM)I3Z(QXVE8U!PQ* znw}Az<)7zV(7o8WRJ}sFs=cQ2^Ba>Oyv+;kyE~EofD((3HZ#jv*|YKNV2=B$^``lb z_JRDd@LByu@pbxb^n?GC^K18a;}7*O>EE_T)hq9v^bPxY1MGD=gY?15#^PqpmZg^Q z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6a28}tA9|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|8E!UVqr(Pgk z+`84}xtClM`reIp(78;$&&K z%v)~v==9QiD}BX&P5)BP2F6Oj|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXRq*yaE8|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|KHHs#N1rlTHa3Bsotg7 zBi%dN-#^GXw0necjC!1OQfpPG=XP>#=y(^|Zs&5{Q?3#HW4^5| zs+*sfSmEiOZJ*n5^otZ%yu2A(j?2c!6M~8EBx|xcMVl&56Q{&gr_XoGUub zsB(06Sv)ov=Z?1~m=m>0@?>F3eQ$Ap`cU+U|G4vH_Zj1P^(E<*_Vwzm{JrGEaIW{c z{Z(hazCc+hE^3-`N!G^y1cKYs5>5U;|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({Quq>|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1!cD)MT=8ufr;opew% z*x%UMth<%5t-7PMOS^kzXRiLa@|8Hh`M#{PU*aZM z;!IAJpzIR%0&FTABaP|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|9_dl z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXT=SG~OGk@kvu`+c2$-2uir>L6)wJET(cEfX!=!PD*CJBGfWVv2_}hnGiX z$HpfFr?{tCBg}KO^W{;(=z2_XWqNgVoqvOKbN5!`4)reS-uAxg!TjOmvG57+Y5Td( zi~7sT>*8C@yJeAm5|_a|x3a!BYwZ`g5%~Z7fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn{}2wZXu03R z>DBFR^i}&w16sKnm@CPUQ1!O7wayN@uIw%v&A#P<*}?JQ!BOt9)(Pe*+G+9#;hcJ8 zabbE%G{(Quxw?Cuaf5oZbZdKib!UE0a$oqM_ptp~=L!93Oy?gBYIuGg(E02j!G*6Xj_ClNmuerJPu32cG$fYo^{-)sn^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;PGd z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzX?`OyM@p|hyF*jTDAla{wa)iW29KB46Gx8=@2T~UUJYO_VTO}2f!bFiDc zm$k2Xpmwl)xNuZGtT;YBIU4St>73miX?dSrR`Lr0xq=X1=E7^1DJ&e_VW)^8fk&{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHDml6M;|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{{QFxSI+$I0%M`N zNLt)3soMOXL>HQU19N277Zj@~rZxL+cZg1^u?xyV}?<*Ww4=oN$kBo-- z$2%u?hZ|?AXGpM5;w`;ZC(A6H_4K)_5QJiPETD_ z`iN4qX1R8@Uc5oDk-Mq2g}IHky}YxqTfJwoPkKN!)IZEQvOCN;UOibF-kwpNm7kYf z5MJzEYG2X0O21aQUc9LZ%GQigtVSr%t>F3-t_tZ{D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z#>) z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+c!t?qlqhw5DE^LAcU<=-cD_>0%rOFFIok0OXYo0ZE|v(@7@gSFlDtPRYK zv`ytLgl+2Wik;G3qdon7oCCT;jl|OFpN(ewa~%FZ|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({Qs=W|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXT=vhlk5mh@g* zRG;K!GB2#W@9nzti{2-lW((-8$ON-^tmv zyQi^_dVn;vJ+wL^KPEXYJjpxNKBIG%ex7oHcyZ&FmuFslP2jsXT7h|+7Rq-EQ9Z4g zp3aD7`m>y8yZnFtKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|EaFGADUdXfLwJ`VIwTsRtda2A4tLD4%$L#0$x8Q&7pVr^zf7*)j z%EGGkYQ-AqTG6`x`p$;kO^nUet)=bS9jjgPJ(9h{{k?7;0~KgF5aon}l|`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}EnV~PLI|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|Nr6k(JING zO49HJFSB3kBZ|X8uq6Q~q1{uU?^8DP1L6&0oV=tGlkT zzPh2bNxNCKRlaSqW4MdAhrM@afBhikQ1OW7=(3xg7+b;Vu5F!bI@(3DD_mZWEykte zqY3^*XHs{vF-4s!@&Eb%{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHDM=k%K|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{(n?GkjKfRVd6byr=1t{OnFVrn|I0& zvXA4>f-l`~tnbVpwV&nRg#Xol6n~}vMl1L$IjeM6GuBYolGbepRfF@5lg+}dylw3r zJGMGJ|ZNF+jzD_bI9PDjuZ`RpL-&WaC+@;yQG_w6-GdRTU zSVx;(?L^rUPOnE4=cMOHqx{j%W!)3-jgY}J-&BU!5t=u8g3w9u_jC{9UFi$?h8 zIOlgq8Kc$9q_OSQ)pa@lpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBygYg#XX~=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ-w$u}0{ga3sNbzb;sed}az-{Y zo)tXn&bH>5uWE0~?+72%9~GabUqoN~-#R~Ze=>emf0zDfmsQL29!amTx7XM1*BPL% zqYM%UH$zG_+cMUI9bDbo-88iQWK%e#KD;<8JvKVQKgBt%JHj|eJzpBtUQ&(8uT1#= z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z&<&{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t(o_CGycW%@Jz>S@x5_Uvk8eqnM+$p7d6^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqKicE}^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;NJr z%R8QajpBPt`A^uiN?6 zf_!1JC|v934>phfSV(+F;DP{d*IT+|F)(}(Gwv;tthkEB?w{)*)U;jYo;O^nZ zQR=bM3GK<%@chi=>~N&V|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXSVzRmyV|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|L=4z(p}|p(QC$)+mzQN@<8i@wcY-z1oTN>brwCK)`-_LtN2165 zC!J@y&l@kPuSjpSZ&mN*A0~6d&%Lkg`JDy&LS>P-xLH!R**~!m^mIk5k11*WWmy$4)cz;Pww#l`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l`Fg^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD zTgvI8-JDaNpN)z~2bZ~Ht#RgfZGt>em{i|U+?C!N-S0o-JkovKcv5{vdcJ+JdO3eR zc`JO+`_P`-`CR`>nJ+GA>hhPYiI)Ve`;R4-Jln_w)~yx#H)| zyt2x^kL%zUx3QL(t@e*B2tDhSi&fLrqc#1to%OmK7#pdZN?WvBSKH+~CA)@udi&T1 zbcX7ODg1x_KmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|A&kGfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~x&=BV=6?1cD~;52uHb&h$yHcB2XTvlIMT%BGQ z-QeHs+}gdvxJ$iPy1#v}dN_Y9c_Mt;d(M8b^RoWB@|O5sQ7_IED>a5;f(^y+wPuif}u-YWwJlQ(j&fCe}wX>(bk8*&>|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXSV zV8j3C|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|9@b4aCUfnRB)_&f^~{{nl?f{M>xN}u(%{06J6feh0rGG|$`~NvB zc2_o5RackRY}cyR&DT#h3^(yMx3}(Wr|+cfD(>0v|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bjS@Qq+|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|L>LU z8y^@P>>h3%Wge@YAfFu$Q~9b)b2$cDQ_$ zaBO{iadJ95I@3Sf8QHzixI`TzUD;k$U7KH@+!Ws8-EQC6xktZGc~E?~d9+Njr{Xkt z!Og7K%v^g{E`(3&&x$Y8Z=&z~ADy4OzZw5i|CIi2|E*TYS4vh1SM%1e*XpdRudi$< zZqjU4Zk6%>`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l^dV^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHDZxd|q?riO5?xpQ3A1EALA66Wh4vUWWPj-fP z&os_fM@kpA7gv|&S0q=3*Lv66H+62&Z&&UV?`fj)fh>+64HEY$D>YxxGWj(juiq&? zNI#A~^S^Yy>3(PYsQxVd*8X1ok^hza8?NB3WUta$OenkbiMKSjm0@;w90d=!vGu5#Xiv$h@IpPOcr|@9ddL63`MCR;@um8W z^j-Ty^;7<9@_YD)_m}-|X9ayFWfgI?re8TATPGeA@c;S${D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHDukZ5z`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l^eD zZDek$Z6R+XY+vtG?3(Tw?c*Qd4DB9f9H|bIj&Dz@PR-9q&I-@-F0e1|T&iE8TqR!H z_~ngR5Z@Mr?%h^oKA^?&qe4>8DxOVeM|1pFoj1Gh7$2w~OP{r0RA1-cCO?EfdB57f zcmB}-QvMZJXnK{sv%Yb^V1Ud2=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpZ~v>#sBC3^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqzqYxawt>8nuxY({v30s# zw3EN9vuAf7;{bK2bXa>tbxeL-a#DDzcZPjd=RExa=N$b?QQShIY>WLIYK<9>6RyER(yJ3yXRVt zd6DMImkVA!t{9(Ah$i}zoXOoO##D8hG`)SKdOUwJc_w_`d&z#K^M?Mm^1k>{Gq?Oa z`zoFvEN~ZEi_FE^QknnH|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|Njqx|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+c!&-$<8-*km&C4UuXweA|mTI#ye z`t9Iq<9xGZt8iOyM|+pf9{S$O{^CK+A!R2!I_?H1x|VgiX=~@oj&M;uy0|PI8;$eF zI}^GSjY;ZcX-a!*Rxy!QsZaquehu<3oaud$iRxPt+{=biuCADb7zvMWg-8oUz?;#&~stG_k$4x+A|U zxi`Gud&qvI^SJ(`@{IU=^J4jO_ImtQ@SgjjHP`%H`%0cKEU5Ya{D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^ZyqV{D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t(KcqiJzxuyBe{}yc{#93yR%&}!ee-_FfN&jekUhAwvA&tImAGxQL#b!G$40Q9 zYg&hx9qnkjE1X!LQk<5Kh|clPcSdzb8<(kLrE%?b)eZU0$*tiX-d*;+o%{8Nlt;wJ zn=cn$k z#_#GM(qHZJsz=@{=^gg<`q=|I>*#}&!Q#e^T5g$X@eYCR?rs_8ewryCB6RAbieu9g zqEr0SoDtn~jPun|(&%qYZr z?REJr;k~*jK1uoi{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z(~Y{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(pZZ@oUw6MXeo%jser*?5OY>#P^00^3 z%kJIjtM^j|i0d>1OC=i;tHGA8X6<0=+U~L;>{lOH9Go5=9pxYEoX|bRI87ZPozsr2 zF3c}U#)MaTSKHThZqRR5ZWZrn?kw-g?u#D`9(EtIo-m)*o|9h`Uans&^7P$^|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+c!J)i&2|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|NnjGqwc517wXs2x9x&zVZJC?94_^i*~>dU^j=DDv2P=l{WCcp z7%1)#OEtIDGuXiiQ^h* zi8nMimA7QK$9D$zxc6BPnh$G_$xjGR*J<%Wnnkbq{D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(uRHvI{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TuWp z-!|S?KaxIeKd-*Z=O+uoh2A22ac8N%Oj$1WXnK}n)+d&N{;q5dG!<=#tO{G!+Z5ZU zJ4d_udpY}d4>S%|50{Q=hgHYtCnv+hGrhCzk(~?mOO!F-r$ z-h6vOXQ94GSu8GX+VY=FhahICWLS8-cd|XabEbZ_GE%&-xwyPEyCS|SxYoViy2-pnyIsChxTlVa2huot)K8qJ zy8M6sKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|4$qIfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~yVsLx9;wJ%q%=WiwNg&%rz?aw=3>GPEZ;=<;avdNaj zZSapPSUpWq>my4-|9b6Wy>x?UBY#t8i|#hY_Ug{kZtb4cKKTL3(C{$tNPAf4c>QE$ zxOirBR(W1_L40v=>Hpcc69%pY27tpO#|X_m_uO+Y#2m2@3$YLju@FPCoO8t7bI(2Z z+;dNk{zRtGYpf|iC?yL^?Yiyjor+zvJ>tEBeck=7gY-j`!^I=p zk=3Ytbb4|$)*t7bZJwu%moE{nXu|TkUXHwmocp5n zvi_R#hWJ)nRPW_w`f*hGUpTe-oz}=d32n2WT-aNbEDrm4%i4X7e(I{yn%&y<`o%`s zX7QH6w(j=UPWrCO9^zi@;A%)dG#wTV_eVG*%~9HD`D9^iI=J4w$+}IyQ@K}s zpnbSX^QY1*dcp5GubR2`rd$Z`H8aYYy;;fZaE|w_J=d70&X*Q+f7X8&|7A$e+Lo90`j+eH2S0nRSwKy8pbSQyeASX#Zq5<5KFbLLQ{c^6?|W0d>!%;dn(@(#0Y<7m~9PUudmPc2T*&W!p0{D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z(BZ`2YNW z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En&vwtV#_N|TSBO`)*Huw|YZ^!Q_=)qdnQBkTneak0v7FR*7&Fyb z((LZ5`kUhW?8o?*;5YYA>u>!(Wl3?FcDZWBeC2fYXf1y|XG3#SZ3}rDp?|YusrPnI zjBp>%v=1~a^)Siqj;fC>j?YetPYwA0{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z&=V{D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(W36%e*~)q1_%^66 z&%^Y(DDrQ0V)Gs?kslV)=E?He-t)=CaFX}BJ=K_|zAMe>KB_-0KF_|2zX`r~f3$wl ze^dSx|8D=SmdKY*my1^PS9Vr6*V5LLHxxE))N<>dmh2en-tM+x?4z2}f!)FNp~Vr| zQSq_C@h<zg?pO& z%ZGZ8B~OOWde7SvjY;b3($wzldV29e_EG$4@VWby^^N|$@}u}m`)l<_{#W{Mw1mI3 zvz)o2wz9msuvW8fsq{8U)NpH0vv)Lfb$7|=_OACU4#*CU4-JlRkFxmx{D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z$?5 z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}EnM=GPl(XCsJ$-VTf$oDUF0`qb$l&=$_=9cn~-rdRl;X~eI_LIi5>hsdX z?v;9SF(rFDo*sPQeq?>Bf3AEbe$)Q1`XT>0{Wbc-|I7K;TtZt~UQSrCk;>J2aB897Yvc0aMsGCS?w^hAeu|u|V zyj!rRySKHUet^RN=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpZ|ZL$p7d6^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqKeV-~!*V-4I&%CI9M>G9dGc9;-&{~$ z+`BBfD!kUa(Z0pFL%mzNzk9HLw0I(WCY}(y znLlVh%fAYLG+p^mPe_&uMQ;ULGFDS%Y29vvdgEgAY^!*?U7nw%b<>78p|U|g(TCSBECQ{PbBoZTMZ72M}OXg#Vw zp*$l_XkV;e&R{1mp{?{;VWqYBbe-LiGxqF=UZ zyk@YDyMeW_zPYlMxLvz_#sBC3^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqzeCRd=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpZ~vOsz`pY6r@ea9DFh zc~tM%4NT}`qJXc?3(z7;AZ!B>n{C1hDt5Evgq6eX?cazCk~CRclRs9c2S?<94%Z z%bfqu|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|9`8L|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bJ>qzr=baZof&5-vI%;tdd;NGFh5#dqZvG(!CN$RQ6 z>D?LiImP+eMe(J?c!bSz10KxBkAMO)BbbLi{{JPYw{byTTM~E*DI5c z!^-=@u8r^1M*69nUoR*YW{cv*K_7QntFPWqSyf!KUAtO8-zeq(^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;Q>z;{Wsi z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ-^5p)txZkaQPzdsn?1|Dd;28^ga>Q^e)h&Qx1Rk!7LruRk<_>VY`n@?-c$u9~oH?Nj?@6DtL-}6fQW1~{Pkm~N+dTuc< zn;$O-7P^b9#d;rQS+Q@sQngCHM!I&y|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTAj?e$+|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|G%!In44&-ytSY; zJCr;3c1!jQ_xAR)4=@f^50#GSj;xO}GrPYF(Q&#=zX&sQ!IFKw@=uFkJdZ;Ed7 z?{w}pAJ86=9~Yi(vhsyqFL^c0y*KT`cuy^*kGom*>|###Z9F%a=gzkl=nIua;^KD6 zYMFfbbfst&pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJe^rP7&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmUIZU;n|V#F~M=}iPkClY04SmIqiAX`23Reis)+pdgmtdHtkONUg3e};WF(# zm1N-yUeA8j$kjKcqICHDL3eCoNOL$`Zgxi|9yr-8W4~MDul${wbs6FY`?)7?V zF)e#Ho)OG+XIZoLIm)-<-1f)nm;AT%&**ReKW9mk|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+b}kH-J!|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|G$(h z3M(}I%2j)7ChLS7cpKZB8(XQ{Njr2q)w>pZWP8Q?2K&1QS%>I{D@TgQw4KTp4kzVW|zel&m4 zev|qC{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^ZyqJ{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t(zc&lZMZLvIpKw{PuiejBRb5kBr(3_?sMsvqGTt`W z-rdRCRo_F|OWd~|QVq?ArNg5U{zzw(Ia)he9xIG%&MN)hg-H-z?uGVsMx@><#ogWY z{l!DsWAT&0v+ncOM17L-x;V9cyPBSVkbV?>>VNKhWqzZ5FaId;|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bDYxw{C zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|L2wSdkd0<;UaIb-N#r~?JM=`R;kx0*3QFmgwc&#j_G;HS)uP;Xa~mSYA9XT-B{mJ+>zZK-yb~WK4v|sKdU@1 zPHbPPCg)Sqx1;I)2hK<4r`qT8SHd?9|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1!cHzog{|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{{OeVxyih6zPG?$ zXe?3}OMSX!>g9`-vQ^?Wg079_c;TX@e&oX@VLMiAjtFJ1q&2Efu3GQ(3w(i#-QXUhZY@ewnD=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJ|MMRIpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBygMWKQ_4H`ktL%vTpk3%kGT|B5BEW#Z+7 zmE2XVHT1QW^~H_aO{*>PZPNbH0Dl)}pgBkzEDsTeHdc99&rXgG9q$C&HO8o(bXIq6 zeL-p@t6DGL zFx@oT!r#W}Zw}CQkp~Kcnte*McVJ?Lhk3SrwBe{HNN#s(eR^?bc5Zw@aIt%tb(Mat za-(=ldwX?PeqZ`v^r-)Y^Ncw`dr5vpnB2TkzSVmt;s5jh`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQpC0o6`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l{R! z&9G-0v((wrobH?Y`{Kv!m-x5fPxo)@KYdAM8FBe`#cJhz^>nRhJ%2-IQ*#S#8@azQ zpwY|Sdq%QPXnF_QmT{PBOGkId*2fnoWv9lc2WPtHS{LXSE0>8^wbxWPAI$^!kJ1qwLf8^WZD@8|!=hN97ms zxAu?ful(P1iD+qmIcG(4Wo>nNEn&SzDL3h<$=0Ff?P%-9?y4c})9qIuP#l~c8XplH zR;(xW8R?MEZ;8N)!bV?(0e3#Jbc=F&gTE~|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|DRy+|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6ZduTGREb(8BU#oO8R_=Dgh_fzY0{VU}g@%#3N>gW8|^pEH-|6gYb zb7^fkc|~F6X0=l8t(z#}CZ1|-ld zpO&5xo#UVHTx4FVT`6B9+|b-q-qyP_xi@^kd&GX+;Q#ag`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQe?sN|^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;Nf? zdbXQTzf`=EO^&AoZ@bg25A=_ePsPvMFRQQf|D``fKl{Hrf0%!1|H?}UOE+SU%5ui3hX?RfptYe=k z^uy>A|1;-H^K0#Y@(;q#Oym#$!CT4}jTKZ$TCH2NUZ>a~+c@4l*vj3`+Ckr0 z*-hND-K*L+-#bNRInq2vJ5D}PIHeg|j_aMBoEMJwF0rpLu2!#?ZtD2|{D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z#$E`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}EnZ!T`n?uzdV9&{hIp3t9BCWtS!FITVSZ=`QU@A&UKADW+NpUGbe zUpL>CP4B0q4S)AK`yWG4my*P;Z{4q0HCr=YC)mK<*xFp*O4&}_q1~z4HQyuME85rJ z-#N%UL_1tQQaGj=RgUhRoQw^}d1u?_8ROMUq$|3x=Ku5m`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQzq;W6^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;P$B zyCJ?gxZSenbYh&Q*lRd?q1rVm7q_>VhJo6l)4 z$}bDAHF^1FuSnhtOYdX5GQLo2>AP-jJ+GLbEr=Hei`>OlAAMP+uh_3$rCK9jJ6%88 z$luJ_(%e?tUfxOAwHa6r>J3hYghRby_Hbi_I#L?dolv{Qn9Pg$|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUOH1^j>h zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|7W=8Sm)~(DVK^@wpUlz=QpLdMR)r5IuDqSXphTJ3(qw#l)c`oNglrG74~~Z zseUX~-Ryc!@ohFYo)^q_7g!7RMap8aPrFRDe7;h;O01QbCi08NC zt4s1L(yOEE{hOTI%saJvAh}7J+qjV&5q{;-@0?H zdHQ^1fw-{!yZSF*GF>KG-e1XC#au&MTV7w-sM)mKqPI=bKOErgVh=P1se`2<-GQ}L z9G2Pf(ShUg|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6aYviSe}fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|Bus8R8A33YsXb*=jWy4qf7iNoU6_2wVUMI zggcvi%B1&jl7>%tnf-#%Q(u+xZfZTPcsH97&kSa{v#mM$x5`{`Ui(Y+TmEPIcl4jX zq_d2l$Q~fZ&oZ8Af?{=*9V)x95_X$k*K+Dqk|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUNcDg1x_KmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|A&Z&w_=!`^jC6yEB^_B}?TJ}jl(ll8O3^V!6BQt-Mv)taWi ztIQB*wx3p?=U=7YMBn>AI=`5|X@AOp3;#7sluP%POI8e5_ExvoGS*W!ls4_udh0^V zc8v94ch|7?(M^T_&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmUJ-$p7d6^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ z=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqzkfTl8kP@FM?@q2QO;=dWNoZGPB^>q z%L{u!a(Nhf*V&PAs~Su9bobW}6^~_4#?J=NyA!QR`s>P6aaudQ`XK)({WSXA|H}Es z{9gM}{zdq$`J?=+_iwU9xU{#Ny`r(Qy1KMhw{EQzn`COdb)dOBTDrcwVu<{I{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`Tv7j{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzX?LDk@VNIEnc<_~v9m?O1O@@V1YW=!ez&Px37LNBl{H$wF~De7*i z?tn?R0wdf&Y>7srk9~mHdtHee*;4bMM#WkMJ+= zUwa8-X>~bi#ZIbME97k5SP3?9RcmWqQ+5>fmjBQH=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJU$6N8{D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z$3r2d0Cf z!Tu0us5wj{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t(TSVLV{ha~kF4{nOkTAG0%L98>a#(14N865Zg6c|Ry3^}3i*vIJ z;){dJ+^ei>^&6F2#5>x%s{8T>(?_Ex{AZjA=1bZu@?>F3^H%vz@BQS%@DuMd`%B|% z^?%Y2-A{E}{GN63KY`#bWr_L2@IcS94>N4_Xvyi0uTLsY%}$Tc49;~gurAgwQ?3%PZEvV< z&Tmieith6tbRIRI(4LVe2ro4+m#_8SNZtzH@!q#TG(J&3lfLZg`n#gZeu~@Rcek_t z(FJ8GQEdBG{qj}QH6#8%|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D0Z!|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ihzl%URFdP}@}ALfEF!${l-pvU_NF z``D&&plV5nbw|`k6~|`B$0r4+x~E%b>gOsKh!?k)R#)cNq&Gx2`?ouHnfGZA%8v?9 zG*6e$^^`om;)|?~zY7}oC#%(eS32>Zc5&4wUpDO<_4E1v z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^ZzA>|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bJWpj0HEqOg*!)B9G?QNZC;f|he?`|0CK9boTP#;_znjH}z z6&&jxZ=Ix{s+=yK*`8CKpI?+-8eQpMi3E=`#7$GFWlPtPH&W-#I{{fEzB3Ci=#gNvJU^B|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{=aDQ|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6ZN zr>!WjEUezh<+?p3*(6lGt!>TNQPrj0yFKf@i~X_#;)8=j-6O1{^kbFd#gp1os?+i_ z(sQEo{fnGS%`3HQ7ed-o;}gpYWS+fN(MsV_<|cdyoY@n%-U?**m%u~q3` zD7E-qJGYvb&rcUb3;jjTVw3;R|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|6kDf|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUNske3#gYgQ)uInlG zCWfkREot2j_0Gj^*`D#80|qShcgOIr+Eg+-ROZ-&tTT z)cF7WfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|2vug&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmY$P;on9mm+Fbh3Zdk!X3NI9sv>RDZC-CxY?tj2?;Pyr z?rH6<@24Cf9^4*M9iAVV9upnspXi)oo~E54pCg>#j4v(!g2+q$^Er%1Ag<1~26&8!#np7N@g zw^OTW`Mc?iXr@2QnQhL|zLn<+{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t(O~e1^|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|KF59_1fh3u=D=01!E~y zlve2a)vFe3X6wWo1RJ}XTU+VdDLaTex4Tw*>FVyfIx23>;`p8*aUZr){V63AUuY**lk(TosnIn5U1x?lQ=28v z7UnekfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~xOO8!6ppa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?|GM{G(u6;Gt^K>vssBhq*QZ{#=$rM6R}I#5*ReLx zH&!+mw`#Yow$FD;ca8S&_j2|%_ty@R4-pP;MwBCaqmt3#$=+CdoN=~#o;1D->dT8T zyDpA`Tiw{YM^BW8#k760dNzMPofu8>Uw5XO)3kTx8N$rw(~|$s|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|9@7G|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+b}O1=nd?>oCOeo|ZM_ikams92o!iI)xfy8WzG^);1s#0}bws?G8((`}>e{hgd$ z%{{celaXim~zm!}HTn(5^SJ^nxcpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?{}~DYpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBt_Ne(Y8D7e=jqCpF!? zdVaAWTNp127Q20{W%a&FKXKJ|jcVV@ z+arvT>L_V+=hkBiFFPytgA3ijx?B&H>%^$NrMe@(JH0=8$bZav(tK8XUY;mSY9^Oc zdT%HEfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|L6bn|M~yZLjFJhpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?|H6CEE{%`XO8TOkQ-52`&F01Pg9YwFYmvTK=_4-N zE?=#buad42t?jSxY-DbxZ7FXnY~Kthcj*mG28DyYA@)#Xm^xe<(H&ho#R-`kj|n{Y zEX&s~R08qx_NwaI{KoW_=nnsG=YI1c?J@aD;n`+F`BLwdWOB&==l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa1{5$N%U5 z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWqpWAO5h5DXUb~Ec)#q4ZO{B1DTooCJ07bpwGMeTpplKC>}^3h8ED$W|_+S>Z^ zM#5&z7Ued*{>gxF7jK|F$QY~+k%o3weOO^Z`Ggq-|2tRlKwKz^5#m~D)Jh_+Rb|9hP_RbEy8WQ{`LT4 z7j>XCsN1JDivu$&J}j`^qb)~2L2<<~?djE-`MK!@(Z&8{&Q<2M+Kuup!X3?B<$b*e zlSjiRyk~6wKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NQ@F4gNp>pa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq z&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?|4e;B>UERq*NdsywD{d%hC9=mrO#I8h~Ku~ zS3l;zq`yUf`hPqBnM-QR$jb{WH7l2^_tr|*3peyOwYM;~QTs~+I=$Y#FtUAOGdR$- ztiyC$Ia+kucZK(P5896!{D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( zkE#5B{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`Twc(RQGKCd@(Vb6u%x!b*EYH>NAv?;;i=b>Z|;l^!w;X{}<;s^H1$> z`9ERFX6bUd-ipb};p*O6_Ik#K>ZZ~b-PW~M?3n5C?t$U%W10GaiX|S_9#I{YADbQ@ zo#dbDoNk_}ohx4;T-;n*UfH`QxgosSyWPIaxKHK(^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;P$);{Wsi`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQPr8Td z$BHMjXXEFCiS8upb$zNbO?QX6l%70tOYx|y0yD*DEo+Jdq8z?erS3`bd-OrbG&(ycB*{3aAtE( zd4BJrv?^m zGD&>Bol?D>PftIHKJq_xJ~zM8zLCEder$d&f9?H|{1yJ|EnzQhET^t0t=z3v%f-5x z5^oZy?$(y3@2KeF?(Lq{-uZs%0nx$!q0SNJQQEQc@xn>XDdlOsGm>+{^Sz7gON}el zYor@G{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzX?8*2VP|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({Qnz^Te3UiyMz1Phpfl+CzWT#=i8U6SMtf}l;~}L zy7Phgk@l(lx$sr*=n&Itm`V)Cc3I@Eo$u! z)z0~D>7LQv{(jB@=E2&b@)5#O%`xS1y%UpD!qdDn>~oCs)r+J{yDMt`KmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NQ?e z3;sXE=}XZo{$yv0`L;G){y_Ms`K0`; z_hs^R_&@Il`)A`<^$+Q3oztCHk1zQD{D1yG|DXTQ|L6bn|M~y?fBrxJ zpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Zzf(`2YNW{y+bp|Ih#D z|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}EnFODw@ zu5z!nZq#p4?hx;8@2einA5EW#p7AF*FPX1sljSMG+s!-W`@Ij7Pr}cdZbT9YvbR8&i%&{ z^raM0T%qk(t(vcyt`lwGZ|rPtZl!G}?;z~l>{{;8+bh{O+}}IMKEya&JyJTR8&!`k zPR_>0{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t(rv?0f{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TwW8XIkg#7bq8tm$g?`*W@>(H%GVocRBZ&4{DFf zPYBO6&y_FsUQS*M-|*hD-!a}-Ka@V{s``tf&c2JA;3v1We%CwYA5m!gRLkal(|*ya z{+iA@<_6lv^5();&9>$Cy`7R>!#%vc?0t>>)q|u%y5aSRVq`Wd9v$%i`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l`GR z^8fk&{D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHDpJbh?pRSxKp4*;ZU6fy%UKw5E-{9PA-mcvx-zPlSJW@X1dpdb8e9?Q^ ze$9A8eM@?$d#^5wkFzTNBBus5A8*cCIWbbP1q3$K^+YPCQ7Q?dP@rYog%m3&9^Z)t({D1yG|DXTQ|L6bn|M~y? zfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWqe~iWd=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpZ|ZXe!Oy$ zcxroEbw++pdVX|~f2nh&d5w01e6w(Sb7y&P?}6lz@Nw^H`#Ix9^=0X`F0bD#itN3( z3_f-%>kGYBz7v~vUNt{okS>fC`HP)C=CWE}xu39VvqrggZ~bJWa5Ha9ds}0Bbth@p zZeTsA7@Q4>hX%vk;THd&|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn z|M~y?fBrxJpa0MQ=l}En`TzWX{{P`R|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1!c5z0~GvF&lyiTNq%Y0(+}InMd! zMcSqEmBKa6_2o^y+mbuOd%XwjM~uhSr={n*7wTT|YL>@u28H{cRq7urmH0(Fr}{RZ zo6d{o`wN_f<|1ve+(%fpS-xDUw@R``xVE>xy^*n*x}~&j*S{W6?2-+P2L*%OA=Xfx z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En z`TzWX{y+c!AcgExfL-sIy0K(&vxdR-)eK^dBXhWxAM>4-^qXBlHM}*^2SQ) zD$*L=TJ?IxhS{d^7Qr@de`|ogi!xB;|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#5qvike|M~y?fBrxJpa0MQ=l}En z`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|KFq9E8jQWKRU=i z#5vqNQaeUIPB^hSxg6UYmz*7*=Z&{7F|JUrmagxj`qm=O?unD&VK=p&(lg}+vDZ$j zUeBkd)1r6%8O}^|mNr|SBYfL@U;fzpCHXD<)BD^0&sb7jMq0jGv0k}YJzFbYFWAuC z)Y?MdM(HmOX!-yAfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn{|8k3fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~wr<-4YPM0@%BI{TXkX@|&%3r991%Tc}2 z$;shZZ=8L$ah^I}x}>|j4vXuuD84m_-FvJ=e^^Pyr`l($=ktl_r08{jsx!@eSDPWv z6lOJ_mtXb1Nxlz%^nS5_GyYWnmj3IOsFyC5%T|n64pw*9vewf#R5lg2Xt%ET|NMXc zKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D z|MUO1$@%~MfBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t( z{D1yG|DXTQ|L6bn|F=!Ik9P8Rb@nj#()N}27Y=HMmBV`@l9AykZ?t`~F;*QXo!$BM zg+-8E9*4noZe-o6$I3lo(mqr@mOq(38$It&bS9avYg6TE!n@4}c5JAvnArCgXP>6t(EoFm9@n6TBX_~=l}En`TzWX{y+bp|Ih#D|MUO( z|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ-!$d_^Z)t({D1yG z|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;Q>n z+S1?F+1}hq+g08}*sB>_4(Sa|hK0ku5%x%9lsZ~Exf@e^#aWpjUl;`Lw8+C+Jh@Om?)e7iS2`5^qr`_%s2_)7go`o8<2{<-)y`y>7< z_}5*+T3TODSy5cMU9FPybyNO7|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp z|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({QvbL{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bn|M~y?fBrxJpa0MQ=l}En`TzX?_5F>U&CD&eZRPESotjpFVL#1Ke zVYOWxojLIdf$NU3JpC-i7cXott}e^3O0SJ>^lx$QFz?pxmmd-yYn~{d=}kyp3SaRi z+f$6U)#=g)-G}uj#b?=<@z=rs+#jr;^jMibJ1n+?qg}^3L3fog zqSu~Tots~fUL0NKU*%kD-l*Lo-yz)H+*dx>do+0>e8!t#zhu0kPL`&0Z`JPL7A`bg zNV!hRjm&Nqb~N;+AxBD%jGSebr=d9w&6AQoDcM_Q*)23)L*r6XFSX3eEOTMuLc@iW zD=9ZJZlrc2oh!{;XPJG4eSrV(d;IwS{D1yG|DXTQ|39O?F1;zdm9|E0e!JIEIp>_W zF6i$n?};DeJ&7Ln){Sak*(_c$E$tK8E_{}a#$&;FZNi-_-L!8Rx79DCuY_;Xg=o=V z@|G+2od?!K{d?sH@uz$%*$#K=yVbq&`P^S~~F#9x9 zzEZHVp?EkLsg1g0rEz=0m{e~{w}jj2Y&7T3dkd9CXUSUD?<)_)hxvN45pLGEs@vtA z;;y-;J(iyoo@Q};7$mhLH!U68nekHP|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ z|L6bn|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|IhzFk@)}ofBrxJpa0MQ=l}En`TzWX z{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG|DXTQ|L6bn|Nj#HPEVr-ztL-| zG&?O;tKO!xiye7qqJ&*_wc1_Qian;T^~y$}FYAv7g2CF5J6syEM~yLcT$&Ij)2V3M zpYdiZbI!cApf4&*;&Q%{tcGj#_3B1>v$$n$Ydi98VK3W{qu`(xyN9L3J~C4ESmOWl z|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD^Z)t({D1yG z|DXTQ|9>v<|M~y?fBrxJpa0MQ=l}En`TzWX{y+bp|Ih#D|MUO(|NMXcKmVWq&;RHD z^Z)t({D1yG|DXTQ|L6bznZAfl{8!#-rNL>mn)GI+MQqKbys5vt_y1X)0QmtQNYn_u~(Pk7^g)OQp+pzcHW=N<+eM`f2pJ zf7QEIx$fMsru1oLMx4#ROui1kt>3BME#E7yn5){Fyk6MIevJL#=UU)CDuwpG5vd1K bEF7l%fBrxJpa0MQ=l}oj|NqW+zkmE6)b~&D literal 0 HcmV?d00001 diff --git a/core/src/test/resources/delta/table-with-dv-gigantic/part-00000-2bc940f0-dd3f-461d-8581-136026bf6f95-c000.snappy.parquet b/core/src/test/resources/delta/table-with-dv-gigantic/part-00000-2bc940f0-dd3f-461d-8581-136026bf6f95-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6e88e78c6c48330d8ea841c7dca9e4ed76a9970d GIT binary patch literal 8473865 zcmeF)&ud)w_3-g??_AAX|D66hn=+KHODQQBQwA;6WxHu9ly)`iZ1OLZrVWi3E`%mt z$cvIy8!0$31d^dz)!* zd6~mIH-@e+9hy49b#3ep+sV1>de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~ zz3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY z>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj} zde`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9) z*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{ z^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4Gq zuIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-? z>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJ zUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q( z*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXb zyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bo zu6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy= zcU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHF zUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU# z@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGU zy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg z-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWy zb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~ zz3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY z>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj} zde`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9) z*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{ z^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4Gq zuIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-? z>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJ zUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q( z*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXb zyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bo zu6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy= zcU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHF zUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU# z@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGU zy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg z-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~z3Y0{^{(q(*SoHFUGKWy zb-n9)*Y&RJUDvy=cU|wg-gUj}de`-?>s{Bou6JGUy54oY>w4GquIpXbyRLU#@4DV~ zz3Y0{^{(q(*SoHFUGKWyb-n9)*Y&RJUDvy=cU|wg{_}NR|IPeX{_fuOH|vEXzzavQ zv#i2*=Y0S7c4vKG7vBfI4?GVJf6sd!cpi8jcpi8jROZCFkJI_R;P-;x3w|&7z2Nu4 z{3mSvUhsRt?*+dX{9f>T!S98=PuX!JH|~Dq*7t$$1K$Vsu!kSF^E~i8@I3H5@H}vz zASmFXpViRXb{qL=6;dWl}5m-yV`qmQEM zX1bYfrkm+zx|wdKn|a^Eqc=M1X1bYfrkm+zx|wdKo9Sk{nQrE;z+Hj60`Eoi9!T%U z^xjMFKRtN4wLPLeqCKKLqCKKLqCKKLqCKKLqQ4*D?*jPy5c_`tijVk+kNAj>_=u1A zh>!S)kNAj>_{iT;oVu%WSLLqCU6s2kcUA7H+*P@&a#!W^mp*^#^Ort<>GPL9f9dm= zK7Z-+mp*@~6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);= zggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I> zPN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1 z>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LI zp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe z6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdh zI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4 zs1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);= zggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I> zPN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1 z>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LI zp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe z6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdh zI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4 zs1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);= zggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I> zPN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1 z>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LI zp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe z6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdh zI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4 zs1xdhI-yRe6Y7LIp-!k1>V!I>PN);=ggT*4s1xdhI-yRe6Mk+w;cx2vUcUY2gF8zL zVejecQgiSdz3#oV-VJB{xYdsWKL5e=(YLp+l5Y(^4}KmzD*b-&TlAoX`laZXqF;)B zDf*@8m!e;aektyq%ZLcdNzN@w4T*+CL!u$kkZ4FWBpMP8iPmeDkXcTOO~xi;ld;Lz zWNb1v8Jmnv#wKHvvB|iu>`?4b>`)vgyT*32?gZVZ6MQ7Y2Ul|M<93>{W~>=&#+tEa ztQl*@nz3fA8EeLxu{RcYuY;2+Z}9Ni5+A8{dUB_6x0bXV!F(p{yyN_Um+D&1AOt8`cCuF_qlyGnPJw#R)u#$!CjV?4%VJjP=@ z#$!CjV?4%VJjUZYc#Ow*jK_G4$9Rm#c#Ow*jK_G4$9Rm#fya1^$9Rm#c#Ow*jK_G4 z$9Rm#c#Ow*?9J8QTOL&aOc#Ow*jK_G4$9Rm#c#Ow*jK_G4#~mKyF&^VF9^)|{<1rrNF&^VF z9^)|{<8k6K9^)|{<1rrNF&^VF9^)|{<1rrNF&=w;z1P=!eZAM$dwspv*L!`v*VlV} zz1P=!eZAM$dwspv*L!`v*VlV}z1P=!eZAM$mwkaZS9^1{H&=UewKrFLbG0{Ddo`EO z9y_jfT~Wa z-s|hVzTWHWy}sV->%G3->+8L~-s|hVzTWHWy}sV->%G3->+8L~-s|hVzTWHWy}rI| zOuV_;o2$LK+MBDrx!Rkny_(Bsj~!P#u6A7QxY}{G<7&s%cDm5@feTs7?1H7 zkMS6f@feTs7?1H7kMX$0V?4%VJjP=@#$!CjV?4%VJjP=@#$!D8`g*Ug_xgITulM?T zudny|datkd`g*Ug_xgITulM?Tudny|datkd`g*Ug_xgITulM@;vN7@IYHzOg=4x-Q z_U3ADuJ&p!pFMV5?YP=;wc~2X)sCwjSL3ncYRA=%s~uN6u6A7QxY}{G<7&s%j;kG4 zJFa$IZF}sv+HtkxYRA=%s~uN6u6A7QxY}{G<7&s%j;kG4JFdQj$9Rm#c#Ow*jK_G4 z$9Rm#c#Ow*jK_G~;V~ZLF&^VF9^)|{<1rrNF&^VF9^)|{Cm!Q59^)|{<1rrNF&^VF z9^)|{<1rrNvDepoeZAM$dwspv*L!`v*VlV}z1P=!eZAM$dwspv*L!`v*VlV}z1P=! zeZAM$dwqS`7kG2EH&=UewKrFLbG0{Ddvmo{bNTGC<7&s%j;kG4JFa$IjmM6w9alT9 zc3kba+HtkxYRA=%s~uN6u6A7QxY}{G?Xlx($JLIj9alT9c3kba+HtkxYRA=%s~uN6 zu6A7QxcU|z<1rrNF&^VF9v|cJ6Z!TVAH3H;_Tif!9G#&5Yo{Cb$}Ii%;jdBu#o~{V zuT}ZbG$)!9 z709E{b9g3tCVD1%CVD1%CO%#HZQ!?o-v)jg_-){~f!_vupY_y`HDnE0L)MU8>A5^} znd!>H{qCce1?wexiC&_Y=p}lIUZR)y?9%>UmU4&e4%r>DJ7k~K^GQ~pO7VH;!#CsT zC3=ZoqL=6;dWl}5m*^#Wi8lhx{|#ajvI*IQY(h36n~+V&CS((`3E6~fLf(C)G#ju1 z8?XTzuz}wOY`_L=zy@r<2C@H#r+il5XZ3wn-)HrGR^Mm!eOBLR^?g>~XZ3wn-`=3M zH?TLbH?TLbH?TLbH?TLbH?TLbH?TLbH}Lni{iShl>+!Z8Z|m{49&hXMwjOWm@wOgs z>+wko8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE78xtE7 z8xtE78xtFo|MA9Tz0S+|`@g*Li_v_Hx>hYW2VtVu`}U9i&<&f}Aj{B(@I3HL^n1bYM)QFgsWK2c8Ek@^i1?j^i1?j^h|W8 z>Q>jivYTyp>+bN~;k(0khwl#G9lkq!clhq`-Ql~xc6;pJ(!Hg7OFJVwBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBIBReBI zBReBIBRix2+s^3O7e1O*qf>wAp8W9jZn#vc zWHx&2rJaUEL!u$kkZ4FWBpMP8iH1Z&q9L(dUKK3P%-=Dr`_}P;5|aP;5|aP;5|a zP;5|aP;5|aP;5|aP;5|aP;5|aP;5|aP;5|aP;5|q;^2jk^m4siFW1ZUa=lzH*UR;C zy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UNKVFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl z^>V#jFW1ZUa=lzH*UR;Cy?i#%%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@- zTrbzl^>V#jFMqw$%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#j zFQ3+Wxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<<+uYu9xfO zdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vdG_^ky<9KX%k^@-Trbzl z^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-TrbzlN2!*ad6Uaptx<$AeZu9xfOdbwVHIrVbATrbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@- zTrbzl^>V#jFW1Wlsh8{JdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v zxn5r80V9Etz(`;uFcKIEj08pkBSG&;D|9P#D|9P#D|9P#D|9P#D|9P#D|9P#D|9P# zy<9KX%k^@-Trbzl^>V#jFOSg6^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#j zFW1ZUa=lzHkI>8Ya=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9J^ zQ!m%c^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=rYcnO?4!>*ad6 zUaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$C#gt(WWNdbwV%m+R$vxn8c9 z>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn4d9y<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH z*UR;Cy<9KX%k^@-Trbzl_4010m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v zxn8c9>*ad6US3yvxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx z<;#s;u9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vc|Y}Xy<9KX z%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzlv(wAV#jFW1ZUa=lzH*UR;Cy<9KPXZ3QuTrbzl^>V#jFW1ZUa=lzH*UR;C zy<9KX%k^@-Trbzl^>V#jFW1X6^m4siFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#j zFW1ZUa=lzH*UJZ?m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUjF2VdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*Z1Y@^RHu z2~+}=KqXKKR05SiB`^}G1S)|_pc1GADuGI%62vuEb*}1M)w!y3Rp+WsFW1ZUa=lzH z*UR;Cy<9KX%k^@-Trbzl^>V#jFW1Yn(aZI6y<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH z*UR;Cy<9KX%k^@-TrZzhdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v zxn8c9>*aOo<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m&cM` zu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v`EW@u*UR;Cy<9KX z%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-yz1%YdbwV%m+R$vxn8c9>*ad6 zUaptx<$AeZu9xfOdbwV%m+R$vxn8c9H=&p7<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUaptx<$AeZu9xfOdbwVntzNE|>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUaptx<$8HH)ywsAy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-TrYq2 z1-*Q)m#^Nt`%FDLHJ99c>cei>KmJ#u{YR0$dh^=`-x0neqC6&+28Y$ncZBZ<-w}R& zmRYl)%N%jgVSOQdAr^zQ?+D)!z9UXtQTWmDqv1!xkA_0EyS*c7q9$shChBX|+>Ngx z(U53JG$a}l4T<(omi4v`M<-rpBeRj&$ZTXbG8>tV%tmG-v(aPc?}kJ}q9M_cXh<|9 z8WIhOhD1Z6A@Q`&NSc;S%cf=1vT510Y+5!go0d(>re)Kz>wMSwuJc{zyUur=-{j+O z;MR?FBi%?h(v5T@-AFgmjdUa3NH@}rbR*qJH`0xCBi+cW$i3*vJEXmE-y4G?^m4si zFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UQ_`%k^@-Trbzl^>V#j zFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFQ3+Wxn8c9>*ad6Uaptx<$AeZu9xfO zdbwV%m+R$vxn8c9>*ad6Uaptx<@fID<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx z<$AeZu9xfOdbwV%m*=`(u9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV% zm+R$v`D~z<>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZ{$Zz= z>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZKCSg~y<9KX%k^@- zTrbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzlt7W}hFW1ZUa=lzH*UR;Cy<9KX z%k^@-Trbzl^>V#jFW1ZUa=lzH*UR*ad6Uaptx<$AeZu9xfOdbwV% zm+R$vxn8c9>*ad6UapsqQZLub^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#j zFW1ZUa=rX=>g9U5Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9pu| zFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy}Zl=Mgk*&k-$h` zBrp;f35*0rg5FIlbSrc#bSrc#bSrc#bSrc#bSrc#bSrc#bSredTrbzl^>V#jFW1ZU za=lzHkI>8Ya=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9Jk(9895 zy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-TraOvFW1ZUa=lzH*UR;C zy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cz5MPodbwV%m+R$vxn8c9>*ad6Uaptx z<$AeZu9xfOdbwV%m+R$vxn8c9>*edUUaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUaptx<$AeZu9xfOdifyqa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;C zy<9KX%e$dou9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vd0pw{ zdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9FE@I*Uaptx<$AeZ zu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfO{nX3#a=lzH*UR;Cy<9KX%k^@- zTrbzl^>V#jFW1ZUa=lzH*UR;Cy<9KPPA}KX^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@- zTrbzl^>V#jFW1ZUa=koP^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU za=lzH*UK~Xa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%Lk#C z>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZe)B24Trbzl^>V#j zFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU@+g1#xaz3{DuGI%5~u_!fl8nf z7ztDYl|Us>2~+}=KqXKK;+m^ES9PxHT-CX%b5*C8>*ad6Uaptx<$AeZu9xfOdbwV% zm+R$vxn8c9>*d+#<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV% zmp@bM<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m)EJ6>*ad6 zUaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZ9!q+;Uaptx<$AeZu9xfO zdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfO!@gdwm+R$vxn8c9>*ad6Uaptx<$AeZ zu9xfOdbwV%m+R$vxn8c9>*acR)zi!Ma=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU za=lzH*UR;Cy<9JELNC|L^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU za=koTy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UP`Gz5Mxiuiog5PW?^z>CR3!92|e4 zvHvJ?50btRz7SDWHvGznT^awk9`2vkZ4FWBpMP8iH1Z&q9M_cXh<|9mMy!Hz(`;uFcKIEj08pk zBY}~?39=JpC&*5aogh0wZu0RLr&%RgC0QluM!J!1q#Nl*x{+?A8|g;6k#3|L=|;Md zZloJ|&9JxGcu}sGX?rbdgkG+f>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUaptx<$8G=dbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*e0# z`|ee}Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU@?6);^>V#j zFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzHpAGbKy<9KX%k^@-Trbzl z^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-TrbzlcRIaXFW1ZUa=lzH*UR;Cy<9KX%k^@- zTrbzl^>V#jFW1ZUa=lzH*UR*ad6Uaptx z<$AeZu9xfOdbwU+E$iiaxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6 zUapsCUoY3o^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=m<%dbwV% zm+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*beIFW1ZUa=lzH*UR;C zy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy?l^*xn8c9>*ad6Uaptx<$AeZu9xfO zdbwV%m+R$vxn8c9>*ad6UaptxV#jFW1ZUa=lzH*URV#jFW1ZUa=lzH*UR;Cy<9KX%k}aIy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH z*UR;Cy<9KX%k^@-Trbzl_3}FPa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH z*UR;Cy<9KX%dg(h%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#j zFJG_qa=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%Lk#C>*ad6 zUaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZ-VOC~y<9KX%k^@-Trbzl z^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl>q;-z%k^@-Trbzl^>V#jFW1ZUa=lzH z*UR;Cy<9KX%k^@-Trbzl^>V#@xzWq@a=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZU za=lzH*UR;Cy<9Kvr(Uj?>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx z<$8H`dbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*e{pUaptx z<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdU=Lku9xfOdbwV%m+R$v zxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v`5^Riy<9KX%k^@-Trbzl^>V#jFW1ZU za=lzH*UR;Cy<9KX%k^@-TrbzlKi$#G^>V#jFW1ZUa=lzH*UR;Cy<9KX%k^@-Trbzl z^>V#jFW1ZUa=lzHkMfs~tDZ`r5~u_!fl8nfs01p3kw7I-2~+}=KqXKKR05SCuDPmn zRp+YCRh_FkS9N;1Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6UY?C!u9xfOdbwV% zm+R$vxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$v`Hh}lu9xfOdbwV%m+R$vxn8c9 z>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vd7XN>Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9 z>*ad6Uaptx<$AeZu9xfOv80#l<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uaptx<$AeZ zu9xfOdbwUc9O&hGxn8c9>*ad6Uaptx<$AeZu9xfOdbwV%m+R$vxn8c9>*ad6Uapr{ zJ-u8n*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UR;Cy<9KX%k}am^m4siFW1ZU za=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFW1ZUa=lzH*UPij%k^@-Trbzl^>V#jFW1ZU za=lzH*UR;Cy<9KX%k^@-Trbzl^>V#jFYl&$xn8c9>*ad6Uaptx<$AeZu9xfOdbwV% zm+R$vxn8c9FZA-QI>-6P-@g9i(fqKyR;?Ts)u#9BD_5>`!*cr^Py>Z|80Mshn|O?hn|O?hn|N`?(L&_nU{H)mwDOq z(B8p5#dgU=WFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC? zL}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`G zOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8` z$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~ zh)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A& ziO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&V znTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8= zk%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf| z5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk z6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l! zG7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH# zA`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57` zA~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8 zCL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`De zWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cg zL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC? zL}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`G zOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8` z$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~ zh)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A& ziO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&V znTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8= zk%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf| z5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk z6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l! zG7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH# zA`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57` zA~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8 zCL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`De zWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cg zL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC? zL}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`G zOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~h)hH#A`_8` z$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A&iO57`A~F$~ zh)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&VnTSk8CL$A& ziO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5t)cgL?$8=k%`DeWFj&V znTSk8CL$A&iO57`A~F$~h)hH#A`_8`$V6l!G7*`GOhhIk6OoC?L}Vf|5&c&a(e?LV zd-eS2)Gy+5tE;Qs@XV}@@;{R{^5B1&|IC-9i8vUHGbv7p~^vud;WwWwb*{p0f&q7fRQ5gMTp8le#yp%EIP5gMTp8hPKVSC)A*#TFW& z5gMTp8le#yp%EIP5gMTp8ljQDkK-?&_%k2=LevTxp%EIP5gMTp8le#yp%EIP5gMV9 zKe1d^rxSlZ+FwBQhi=kfXfQMw8Vn7F21A3P!7wy)&0I6r%r$c~LL)RnBQ!!IG(saZ zDmL`T;X^ZDH5eKU4Tc6ogQ3CDU}!Kj7&<3%PUM^jjnJq?BQ!!IG(saZLL)RnBQ!!I zG(saZ8lw>!p%EIP5gMTp8le#yp%EIP5gHBA2#wGPjnD{<&JQO?>$~z*?RyHe}mCed#WwWwbIS)P%<>0`8DgP%%^t6+^{PF;ooE2#wGPjcPPPBQ!!IG(saZLL)RnBQ!!IG(scq!wWP*BQ!!I zG(saZLL)RnBQ!!IG(sbvHS$>_pEdGXBcC<$StFk{3N%6^G(saZLL)RnBQ!!IG(saZ zLL-0qB+&?s&MredaXoN;+ghptDMredaXoN;+6ljD- zXoN;+ghptDMredaXoN;+gho>|LL)RnBQ!!IG(saZLL)RnBQ!#z8ja8hjnD{<&aFG(saZLL)RnBQ!!IG(saZLZd(blEPj{x&wNRmh=U&--z9l`p7bU0CCPDm zPI^vyP8u*4{%o9wf`@{Kf`@{KLSbiF$e5MQ%4TJmWv!aa(H;sO3LXj`3LXjyz3qe4 zZbuc9qw_rpNkWp4BqRw*LOSJ7s{u4X12jMbG(ZD1Km#;DTeqJ4XNb+pW@WRoS=p>? zRyHe}mCed#WwWwbIb;7_5!S8Nt=6s9t=6s9t=6sK*P_p?^c6$JP%%^t6+^{PF;ol{ zL&Z=rL?bjpqZ*CS2#wGPjnD{<&hM}2j=4gaQXoN;+ghptDMredaXoN<^hS?Yl4Tc6ogQ3CD zU}!Kj7#a)>h6Y3DM3rWaMredaXoN;+ghptDMredaXoN<#Mu|pfghptDMredaXoN;+ zghptDMrgEwMredaXoN;+ghptDMredaXoN;+)S?j@p%EIP5gMTp8le#yp%EIP5gLur z2#wGPjnD{<&LFXoN;+ghptDMredaXoN;+ghmw_p%EIP5gMTp z8le#yp%EIP5gMVh6Y1}p~28#XfQMw8Vn7Fp_yyuXoN;+ghptDMredaXoN;+ zghs`N*%%BBh6Y1}p~28#XfQMw8Vn7F21DmWm1d4cXoN;+ghptDMredaXoN;+ghsYT ziAHFIMredaXoN;+ghptDMredaXtaq&XoN;+ghptDMredaXoN;+ghpu8q7fRQ5gMTp z8le#yp%EIP5gMTp8jaBijnD{<&8DgP%%^t6+<*aBQ&bf2#wGPjnD{<&&>fjnD{<&ejnD{<&3|8le#yp%EIP5gMTp8le#y6&q$_FfIU1o68le#yp%EIP5gMTp8le#y*%~Dpp%EIP5gMTp8le#yp%EIP z5gMV?Te#4ZRox8@ZrO5c(J%>kw*SAUy>%`K#RUh^7y3SOX5qCLob;SD zU@ZJWFb@R}1rG%e1rLS7&XU^A%4TJ=vdpqp&4oq}1rG%e1rG%e1%=-BL29?7n*W>f z5|V@@AxTIQl7w{1AI$=2fCgxQ255i=Xn+Q2fVOTu`47yRmCed#WwWwb*{p0>i3S?Mc=ilJht7%GN}p<<{QDu#-oVu(g) zghn+Qp%EIP5gMTp8le#yp%EIP5gMV97s>}3p%EIP5gMTp8le#yp%EIP5gMV9&l>rx zkghptDMredaXoN;+ghptDMredaTWEwvXoN;+ghptD zMredaXoN;+ghmh12#wGPjnD{<&3|8le#yp%EIP5gMTp8le#y6&q$_FfIU1o68le#yp%EIP5gMTp8le#y*%~Dpp%EIP5gMTp8le#yp%EIP5gMV< zLo`AoG(saZLL)RnBQ!!IG(saZLZcRq&pVrb(x%GwCF0+G*M}ZJIVs z(+wgbAR;2~BJw645fKp%2>1wiL-1Xp z+WwJio&DwheSgo{TQm}lL?h8iG!l(OBhg4S5{*P7(MU8Bjjk4rL?h8iG!l(OBhg4S z5{*P7(MU8BjYOlpqLFAM8i_`tk!U0uiAJK4Xe1hmMxv2u6i=02GsMNv#n8pj#n8pj z#n8pj#n8pj#n8pD;>g}}Xx1@geO~Fd6r2ayUlQ@Z!E*S6lLNHJO1yBG5P>9$Ws@+}LUD;ikX1S~5J47QW zfC4Ch0w^%_+6Ss#NA;cGDNi9$NE8xek9Xi~eS1 zFk{FVGKP#HW5^gXhKwO&$QUw)qLFAM8g+_BqLFAM8i_`tk!U0uiAJK4Xe1hmMxv1q zl&?f1(MU8BjYK2SNHh|SL?h8iG!l(OBhkp;8u?oze{1A#jr^^Vzcuo=M*h~w_v6%} zk!U0uiAJK4Xe1hmMxv2uBpQiEqLFB{Lo^bNL?h8iG!l(OBhg4S5{*P7(MU8BjV_5s zqLFAM8i_`tk!U0uiAJK4Xe1hmMxs$A8i_`tk!U0uiAJK4Xe1hmMxv2uBpQiEwP++7 ziAJK4Xe1hmMxv2uBpQiEqLFAM8VwbVL?h8iG!l(OBhg4S5{*P7(MU8BjYOlXMI+Hj zG!l(OBhg4S5{*P7(MU8BjYK2SXq9Lr8i_`tk!U0uiAJK4Xe1hmMxv2uBpSt2rPmB` zF?2C>F?2C>F?2C>F?2C>F?2C>F|0Uq&RjGSjYK2SNHh|SL?h8iG!l(OBhe_bVcq3o z=wj$%=wj$%=wj$%=wj$%=wj$%=y9SB&RjGSjYK2SNHh|SL?h8iG!l(OBhg4S(i+vG zk!U0uiAJK4Xe1hmMxv2uBpQiEqLFBHNi-6TL?h8iG!l(OBhg4S5{*P7(MU8BjfRLu zqLFAM8i_`tk!U0uiAJK4Xe1hmMxxOdL?h8iG!l(OBhg4S5{*P7(MU8BjYK2SXsBo; z8i_`tk!U0uiAJK4Xe1hmMxv2uBpP*yMxv2uBpQiEqLFAM8i_`tk!U0uiAJK)v!an` zBpQiEqLFAM8i_`tk!U0uiAJK4XjF+tqLFAM8i_`tk!U0uiAJK4Xe1hmMxs$A8i_`t zk!U0uiAJK4Xe1hmMxv2uBpQiEwP++7iAJK4Xe1hmMxv2uBpQiEqLFAM8eJnAiAJK4 zXe1hmMxv2uBpQiEqLFAM8i_`oqLFAM8i_`tk!U0uiAJK4Xe1hmMxv2uREtKUk!U0u ziAJK4Xe1hmMxv2uBpQiEqERIpiAJK4Xe1hmMxv2uBpQiEqLFAM8i_`yL?h8iG!l(O zBhg4S5{*P7(MU8BjYK2Ss1}VxBhg4S5{*P7(MU8BjYK2SNHh|SM59_X5{*P7(MU8B zjYK2SNHh|SL?h8iG!l&((MU8BjYK2SNHh|SL?h8iG!l(OBhg4Sx=SXe1hmMxv2uBpQiE zqLFAM8i_`tk!U0uRicq-BpQiEqLFAM8i_`tk!U0uiAJK4Xw)kjiAJK4Xe1hmMxv2u zBpQiEqLFAM8i_`oqLFAM8i_`tk!U0uiAJK4Xe1hmMxv2uREtKUk!U0uiAJK4Xe1hm zMxv2uBpQiEqR|zik!U0uiAJK4Xe1hmMxv2uBpQiEqLFAcSTquiL?h8iG!l(OBhg4S z5{*P7(MU8BjXFgm(MU8BjYK2SNHh|SL?h8iG!l(OBhl!RXe1hmMxv2uBpQiEqLFAM z8i_`tk!U0uRicq-BpQiEqLFAM8i_`tk!U0uiAJK4Xmm+55{*P7(MU8BjYK2SNHh|S zL?h8iG!l&lL?h8iG!l(OBhg4S5{*P7(MU8BjYK2Ss7o{wjYK2SNHh|SL?h8iG!l(O zBhg4S5{)`VBhg4S5{*P7(MU8BjYK2SNHh|SL?h8?h-f4liAJK4Xe1hmMxv2uBpQiE zqLFAM8a1MkXe1hmMxv2uBpQiEqLFAM8i_`tk!aK_8i_`tk!U0uiAJK4Xe1hmMxv2u zBpQiEm1rayiAJK4Xe1hmMxv2uBpQiEqLFAM8g+?AqLFAM8i_`tk!U0uiAJK4Xe1hm zMxs$I8i_`tk!U0uiAJK4Xe1hmMxv2uBpQiEJ47SVNHh|SL?h8iG!l(OBhg4S5{*P7 z(dd$BBpQiEqLFAM8i_`tk!U0uiAJK4Xe1g{qLFAM8i_`tk!U0uiAJK4Xe1hmMxv2u zREtKUk!U0uiAJK4Xe1hmMxv2uBpQiEqR~*%NHh|SL?h8iG!l(OBhg4S5{*P7(MUA9 zS~L=kL?h8iG!l(OBhg4S5{*P7(MU8BjaG?9qLFAM8i_`tk!U0uiAJK4Xe1hmMxs$X zReH@37eg0A7eg0A7eg0A7eg0A7eg0A7sHA(=gdVT(MU8BjYK2SNHh|SL?h8iG!l&> z8`fPehAxIKhAxIKhAxIKhAxIKhAxIKh8`#C;LJrM(MU8BjYK2SNHh|SL?h8iG!l(O zBdt*_8i_`tk!U0uiAJK4Xe1hmMxv2uBpQiEmqa7cNHh|SL?h8iG!l(OBhg4S5{*P7 z(P)ThBpQiEqLFAM8i_`tk!U0uiAJK4Xe1haK{OJLM5FJ0R3~{F`5VKj@0;=ZE~jsAlNL2fC~He?8shlUx0F_Tu-&`vE8L)4vbj`ou#2Zn{sn zFMazHJiQ;hAG{yDALK@PnrGlqzWK@C?gj1z?gj1z?gj1z?gj1z?ge_as>8tv-=q6` zbbpWTdvJVSm+wvTy-Tra8|n@9hI&K2 zq25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra z8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&w zy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO= z)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4 zL%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC- zH`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aV zdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$ zs5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9 zhI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPv zZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz z^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHi zP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I z4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT z-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb z>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2 zq25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra z8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&w zy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO= z)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4 zL%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC- zH`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aV zdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$ zs5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9 zhI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPv zZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz z^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHi zP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I z4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT z-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb z>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2 zq25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&wy`kPvZ>Tra z8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO=)Envz^@e&w zy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-cWC-H`E*I4fTe4L%pHiP;aO= z)Envz^@e&wy`kPvZ>Tra8|n@9hI&K2q25q$s5jIb>J9aVdPBXT-tfD-H(YyR!`6=t z`#=x&xxUYQZG1EI6A_NynUDN;_VPaeKKOm@e@D4eZTg8@P6=q z@P2qZZ@MqJFS##ueal08KX^ZQKX^aX?#b@S?#b@S?#X%!z07<5p36PiJ=s0kJ=r~3 zZ{Ovf?4InN?4InN?4JD1&pqJJ_%rtc_X76<_X76<_X78V{5&MylsDx~c~jn$H|0%x zzhbUnHrU5~Cu*Q5JB zBHy<(m?z?icp{#NC*p~CBA&?iQ~LhRwH?EGBA$pR;)!@7o`@&niFl%lC*p~CBA$pR z;)!@7o`@&ni8|}6cp{#NC*p~CBA$pR;)!^oh9}~Qcp{#NC*p~CBA$pR;)(qIZLhz- z_4l`)tK`f1a=x4|=gawWzML=T%lY!Hd^umvm-FR(IbY6~^W}UwU(S~g@a23tU(T2F z<$O6`&X@D$d^umvm-FRA`EtITFXzkoa=x4|=gawWzML=jxxGG*rH?P?%lUG?oG<6g z`EtITFXzko@*#XVU(T2F<$O6`&X@D$d^umvm-FR(dCiye<$O6`&X@D$d^umvm-FR( zdBvCW<$O6`&X@D$d^umvm-FR(IbY6~SA02N&X@D$d^umvm-FR(IbY6~ck<wR|~W&X@D$d^umvm-FR(IbY6~ha}>Ocp{#NC*p~CBA$pR`WjEf6Y)em z5l_Su@kBfkPs9`PM3v_%Jy+?uO3zjD<$O6`&X@D$d^umvm-FQ{U(T2F<$O6`&X@D$ zd^umvm-FTQd^umvm-FR(IbY6~^W}UwU(T2F<$QTJU(T2F<$O6`&X@D$d^umvm-FR= z_;S9SFXzkoa=x4|=gawWzML=T%lYz(FXzkoa=x4|=gawWzML=T%lY!S74t+q5l_Su z@kBfkPs9^#;E8x5o`@&niFhKOh$rHScp{$2_uO@OuF`Xro~z```EtITFXzkoa=x4| z=gS+uoG<6g`EtITFXzkoa=x4|=gax>nlIzML=T%lUG?oG<6g`EtITFXzkoa=!dCd^umvm-FR(IbY6~ z^W}UwU(T0Td^umvm-FR(IbY6~^W}UwU(T2F<$QS;U(T2F<$O6`&X@D$d^umvm-FQ{ zU(T2F<$O6`&X@D$d^umvm-FR(IbYtxm-FR(IbY6~^W}UwU(T2F<$QSuU(T2F<$O6` z&X@D$d^umvm-FR(IbUA!<$O6`&X@D$d^umvm-FR(IbUA$<$O6`&X@D$d^umvm-FR( zIbY6~^W_)#a=x4|=gawWzML=T%lUG?oGuF`Xro~z```EtITFXzkoa=v_kFXzkoa=x4|=gawWzML=T z%lUG?oG%~Bm-FR(IbY6~^W}UwU(T2F<$U>YzML=T%lUG?oG<6g`EtITFXzkoa=v^B zU(T2F<$O6`&X@D$d^umvm-FRU@#TCuU(T2F<$O6`&X@D$d^umvm-FQnU(T2F<$O6` z&X@D$d^umvm-FR)d^umvm-FR(IbY6~^W}UwU(T2F<$SsC`y9rX^W}UwU(T2F<$O6` z&X@D$HDAt`^W}UwU(T2F<$O6`&X@D$d^umvmxmA6bJReG+HFXzkoa=x4|=gTX;oG<6g`EtITFXzkoa=x4| z=gawWzML=b=gawWzML=T%lUG?oG<6g`SP`VIbY6~^W}UwU(T2F<$O6`&X@D$d^uk} zh%e{M`EtITFXzkoa=x4|=gYswm-FR(IbY6~^W}UwU(T2F<$O6`&X@D$aVzGDcp{#N zC*p}}o`@&niFhKOh$rHScp{#NC*p~CBA$pR@;!GQo~!g+rROU7a=x4|=gawWzPy_+ z=gawWzML=T%lUG?oG<6g`EtITFXzko@|rK_%lUG?oG<6g`EtITFR%D=zML=T%lUG? zoG<6g`EtITFXzkoa=x4|{~TY=m-FR(IbY6~^W}UwU%r7a=gawWzML=T%lUG?oG<6g z`EtITFXzkoa-Uz`&6oebefha{+oyi`gY(1HU%Ya`_3-|)QfG^++_<}k=kJ|I5o;UTpsXkRcl8@vg`A9yJkK`lyNIsH} zJa=Dz(D(ItzQFSZo-goxf#(Z6U*P!y&lh;U!1pxy9;(6o8Gpu~@n`%Qf5xBjXZ%_B zyT0$*^ERHh@w|=aZ9H$|c^l8$@CAGUU%(ge1$+Ttz!&fZe8Jj|;hrn;T#4sOJXhkm z63>-*uEcXCo-3(%BA$pR;)!@7o`@&niFhKOsI$I`C*p~CBA$pR;)!@7o`@%kPi&Bn zPs9`PL_85s#1rvE6;H$y@kBfkPs9`PL_85s#1rvE zJdy9Y>+oEq=PEr{$(Qrxd^umvm-FR(IbXh=FXzkoa=x4|=gawWzML=T%lUG?oG-8W za=x4|=gawWzML=T%lUG?oG)+qa=x4|=gawWzML=T%lUG?oG<6g`SPpza=x4|=gawW zzML=T%lUG?oG-8Wa=x4|=gawWzML=T%lUG?oG<6g`Es9M-p!Zu<$O6`&X@D$d^umv zm-FQ{U(T2F<$O6`&X@D$d^umvm-FR(IbY6~ck$(XIbY6~^W}UwU(T2F<$QTBU(T2F z<$O6`&X@D$d^umvm-FR(IbY6~*L*o&&X@D$d^umvm-FR(IbZ%^zML=T%lUG?oG<6g z`EtITFXzkoa=x4|zmhNK%lUG?oG<6g`EtITFXziY$(Qrxd^umvm-FR(IbY6~^W}Uw zU(T2FkVHHYPs9`PL^V&u6Y)em5l_Su@kBfkPs9`PL_85s#1rvEmFFrwSLwM*&sFl} zd^umvm-FQnU(T2F<$O6`&X@D$d^umvm-FR(IbY6~^X2_~IbY6~^W}UwU(T2F<$QU? zm-FR(IbY6~^W}UwU(T2F<$O6`&X@D$eEA^0oG<6g`EtITFXzkoa=!c=U(T2F<$O6` z&X@D$d^umvm-FR(IbY6~^W||X=81SBo`@$}z!ULAJP}XC6Y)em5l_Su@kBfkPs9`P zL_Cr2x$E#;rROR=SIL+2<$O6`&X?DGIbY6~^W}UwU(T2F<$O6`&X@D$d^umvm-FQ{ zU(T2F<$O6`&X@D$e0eut&X@D$d^umvm-FR(IbY6~^W}UwU(T2F<$U?od^umvm-FR( zIbY6~^W_y^&X@D$d^umvm-FR(IbY6~^W}UwU(T2F<$Sr%FYo5d`EtITFXzkoa=v^! zU(T2F<$U>N?aOEGJ2U+wAMDG&y6y9aXEj4V^&X$>zq6P3xkvhad4JDfFZS{d|AL?R ziJ#xj;ocA458e;n4{v)Z_a*lw_oc3H^;O;v-Vfdn-Ve2VvU{?7vU{?7vfe^3Q@^{v zbx(Frc29Osc2CyZcey9KC%Y%RC%Y%RC)aO3=g6P&XYK{=1?~mz1?~mz1?~mCs)`A9yJkL07y z`YQQIK9Y~*Bl$=^l8@vg`A9yRu1D9S>(TY-JP}XC6Y)emk?#rZ^|>7W-oo=ld^umv zm-FR(IbY6~^W}UwU(S~g<;(eUzML=T%lUG?oG<6g`Es966Y)em5l_Su@kBh)8J>tI;)!@7o`@&niFhKOh$rHSe9v8n=PEr{>A6b2 zoG<6g`EtITFXzkoa=yIb%lUG?oG<6g`EtITFXzkoa=x4|ulaJmoG<6g`EtITFXzko za=x4|=gVupoG<6g`EtITFXzkoa=x4|=gax>ukz)5IbY6~^W}UwU(T2F<$O6`&X?DG zIbY6~^W}UwU(T2F<$O6`&X@D$KEJ%1FXzkoa=x4|=gawWzML=T%lYzNzML=T%lUG? zoG<6g`EtITFXzkoa=yHaFXzkoa=x4|=gawWzML=T%lYz;@a23tU(T2F<$O6`&X@D$ zd^umvm-FQ{U(T2F<$O6`&X@D$d^umvm-FSH;>-DRzML=T%lUG?oG<6g`EtITFXziY z&zJM%d^umvm-FR(IbY6~^W}Vb#h3Htd^umvm-FR(IbY6~^W}UwU(T0z@#TCuU(T2F z<$O6`&X@D$d^ulU^W}UwU(T2F<$O6`&X@D$d^umvm-FR4d^umvm-FR(IbY6~^W}Uw zU(T0z@a23tU(T2F<$O6`&X@D$d^umvm-FQnU(T2F<$O6`&X@D$d^umvm-FQ{U(T2F z<$O6`&X@D$d^umvm-FR(IbXhyFXzkoa=x4|=gawWzML=T%lYzdzML=T%lUG?oG<6g z`EtITFXzkoa=v^PU(T2F<$O6`&X@D$d^umvm-FRo`EtITFXzkoa=x4|=gawWzML=T z%lY!bd^umvm-FR(IbY6~^W}UwU(T0z^5uLvU(T2F<$O6`&X@D$d^umvm-FQnU(T2F z<$O6`&X@D$d^umvm-FRU@a23tU(T2F<$O6`&X@D$d^umvm-FTE4R1UVPs9`PL_85s z)XNj`L_85s#1rvEJP}XC6Y)em5l_SuP4`@-=PEr{>A6b2oG<6g`EtITFXzh#_;S9S zFXzkoa=x4|=gawWzML=T%lY!5d^umvm-FR(IbY6~^W}UwU(S~g=gawWzML=T%lUG? zoG<6g`EtITFXzjL@a23tU(T2F<$O6`&X@D$d^ul!6<^Mm^W}UwU(T2F<$O6`&X@D$ zd^ulU@#TCuU(T2F<$O6`&X@D$d^umfjW6fR`EtITFXzkoa=x4|=gawWzML=jeV@bl za=x4|=gawWzML=T%lUG?yynaKa=x4|=gawWzML=T%lUG?oG<6g`SOrNJP}XC6Y)em zQNJzML=T%lUG?oG<6g`EtITFXzko za=x4|k6STM#1rvEJP}V+^F%xmPs9`PL_85s#1rvEJP}XC6Y)emk?*-gaJ zL;Fi#ymCP^^waP8eevJf%lqUizc26a-R#9)-r*zh6F>3u+g{51!TZ7c!TaHDPwT$q zzU032?f%yL!TZ7c!TX_hPj*jsPj*jsPu5%LW#02VgL|@jvU{?7vU{@LzRNw?J=s0k zJ=s0kJ-L4SxnKT_KXWf|FK{n#FK{n#FK{pDEq@QhoARc-DR0V~@}|6Lk9;H_$w%^$ zd?X*qNB%zC-@jMy{JaFdfG^++_yWFwFW?LKf{HKT3-|)QfG^++_yWFwFQ|RKqUTCJ z==+HHGyaS}3 z=SnJ`h$rHScp{#NC*p~CBA#e1PgGyU6Y)em5l_Su@kBfkPs9_&6X)`gd?X*qN1gJK z&#Cb_H9n`t^F%xmPs9`PL_85s#1rvEJP}WH%=1K^C-OXz=ZW}ozML=T%lYyv_;S9S zFXzkoa=x4|=gawWzML=T%lUG?oG%~Bm-FR(IbY6~^W}UwU(T2J^5uLvU(T2F<$O6` z&X@D$d^umvm-FR(IbS}6FXzkoa=x4|=gawWzML-~;LG`PzML=T%lUG?oG<6g`EtIT zFXzkoa=yIc%lUG?oG<6g`EtITFXzjL^W}UwU(T2F<$O6`&X@D$d^umvm-FR(IbZJk zK8Nw;d^umvm-FR(IbY6~U&WX6<$O6`&X@D$d^umvm-FR(IbY6~^W}UwUmlW(C*p~C zqA&48JP}XC6Y)em5l_Su@kBfkPs9`PL_85s#1mDXtMpu@=PEr{$(Qrxd^ulU^W}Uw zU(T2F<$O6`&X@D$d^umvm-FR(IbY6~_w(g^IbY6~^W}UwU(T0Td^umvm-FR(IbY6~ z^W}UwU(T2F<$O6`&X@D$gZOg3oG<6g`EtITFXziEzML=T%lUG?oG<6g`EtITFXzko za=x4|=gax>xE1q6JP}VcizniVcp{#NC*p~CBA$pR;)!@7o`@&niFhKO$oJfJc&^fO zm7c5Q%lUG?oG-tUFXzkoa=x4|=gawWzML=T%lUG?oG<6g`EtITFR%G>zML=T%lUG? zoG-8Wa=x4|=gawWzML=T%lUG?oG<6g`EtITFXzko@-|=2m-FR(IbY6~^X1)qIbY6~ z^W}UwU(T2F<$O6`&X@D$d^umvm-FR(xz8`}=F9nVzML=T%lYz(FXzkoa=x4|=gawW zzML=T%lUG?oG<6g`EtITFXzj<_;S9SFXzkoa=yIHm-FR(IbY6~^W}UwU(T2F<$O6` z&X@D$d^umvm-FQ{U(T2F<$O6`&X+fQIbY6~^W}UwU(T2F<$O6`&X@D$d^umvm-FR( zIbZ%ozML=T%lUG?oG-8Wa=x4|=gawWzML=T%lUG?oG<6g`EtITFXzkoa=yHaFXzko za=x4|=gVupoG<6g`EtITFXzkoa=x4|=gawWzML=T%lUG?oGpU;f(4dB5|)@o3JjFMs`>W@!DM_~1XYl5f3v{Fc=3 z_H0&SC7p52#7UgQNf(TF+_gag6hHwKK%uvElkUpy%I?ZE%i8ZOCO`ocKmim$L0Xg+ zrA28`T9g*w|DgqUWp`zFWp`zFWp`zFWp`zFWp`zFWq0Lo{Sml-7X7p6pGE&H`e)HU zi~d>k&!R_e-unqbBr1tYqLQd2sv?wh~i_)UBC@tn^hq)NK7`hm`7`hm` z7`hm`7`hm`7`hnxfV6jhw6Y>ik)}vfq$$$)6262l;Y;`uzJxE~OZXDLgfHPs_!6Is z?JE_!L?h8iG!l(OBhg4S5{*P7(MU8BjYK2S$afM(b=uSIV(4P%V(4P%V(4P%V(4P% zV(4P%V%TuzoVjQu8i_`tk!U0uiAJK4Xe1hmMxs%#Xe1hmMxv2uBpQiEqLFAM8i_`t zk!U0ud7y5PXe1hmMxv2uBpQiEqLFAM8i_`tk!U0u)uNGTBpQiEqLFAM8i_`tk!U0u ziAJK4XjF?vqLFAM8i_`tk!U0uiAJK4Xe1hmMxxP|MI+HjG!l(OBhg4S5{*P7(MU8B zjYK2Ss1l7tBhg4S5{*P7(MU8BjYK2SNHh|SM5C)jBhg4S5{*P7(MU8BjYK2SNHh|S zL?h8?sAwb_iAJK4Xe1hmMxv2uBpQiEqLFAM8VwhXL?h8iG!l(OBhg4S5{*P7(MU8B zjYOk)qLFAM8i_`tk!U0uiAJK4Xe1hmMxv2uREb8Sk!U0uiAJK4Xe1hmMxv2uBpQiE zqERIpiAJK4Xe1hmMxv2uBpQiEqLFAM8i_`c4eKr!Ll;9ALl;9ALl;9ALl;9ALl;9A z!ye9@GZ&3SBhg4S5{*P7(MU8BjYK2SNHppYjYK2SNHh|SL?h8iG!l(OBhg4S5{*P7 z57bqnk!U0uiAJK4Xe1hmMxv2uBpQiEqLFB{OEeOVL?h8iG!l(OBhg4S5{*P7(MU8B zjfRLuqLFAM8i_`tk!U0uiAJK4Xe1hmMxs$I8i_`tk!U0uiAJK4Xe1hmMxv2uBpQiE zjc6nqiAJK4Xe1hmMxv2uBpQiEqLFAM8g+<9qLFAM8i_`tk!U0uiAJK4Xe1hmMxxP5 z(MU8BjYK2SNHh|SL?h8iG!l(OBhg4S>K2VeBhg4S5{*P7(MU8BjYK2SNHh|SM59VH z5{*P7(MU8BjYK2SNHh|SL?h8iG!l(!(MU8BjYK2SNHh|SL?h8iG!l(OBhg4Sszf8v zNHh|SL?h8iG!l(OBhg4S5{*P7(Wp~25{*P7(MUA<|B6PxQU7Fp;H{UwGOV8b`TA{= z{OHwJR8=+k^WTBiYr3l;lRsR4_O+9JgFF4t25i6vY`_L=zy@r<25i6vY`_L=zy@r< z25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6v zY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L= zzy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r< z25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6v zY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L= zzy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r< z25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6v zY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L= zzy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r< z25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6v zY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L=zy@r<25i6vY`_L= zzy@r<25i6vY`_L=zy@r<25i6vY`_L=zy|(&8~E(`lYN7`s`#HDcHsBG+W-oX5DAeG z36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAg+av)(x`9=4`4*VW?8$baPA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIBH?9D!q1*R z**Ewy-+}-0{bT4KL;o21#}EmT5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG z36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DCBgNZ484*nZf7-ve(0 zC_q9aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) zLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aM8eB}gjW>S`C$is54;Va011&0 z36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG2`>W@b`{q7VF!K>ybYiL36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG z36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9F9Q+|Dy;Lv4*VW?8$baPA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|Vo91|%F@Sm%cw_&xA8fC3~$LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) zLL@{&Bt$|aL_#D)LL|HlNcf?`IzQ~d?}4`g6d)lIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIBH?8~!Xbrqe%OKE18)N;Ktd!$LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) zLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)!pne!-Gz02*n!^zZv!YmLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) zLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@}O z%YcL*F0Avz4*VW?8$baPA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|Vo91|-?|- zzX#q1P=JI;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^l zNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_; zh=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=i8`2|rd?=Z78mJ@7Vw z0whF2Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&B)kkr_&tSne%OKE18)N;Ktd!$ zLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)!pne!LksKtumisb-Ud*Bgh+^lNQi_; zh=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRp zgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^l zNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lmjMYsURdXc9r!))Hh=;oL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) zLL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#FI3`qEi!a6_f!0&;#0Tdu15+WfIA|VnYArc}X5+WfI zA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnY zArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X5+WfIA|VnYArc}X z5+WfIA|VnYArc}X5+dPcK*CQJ*7;!veh<72pa2Px5DAeG36T&9kq`-y5DAeG36T&9 zkq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG z36T&9kq`-y5D6~>5`L<%&JR2Ad*E#V1xSd5NQi_;h=fRpgh+^lNQi_;h=fRpgh+^l zNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_; zh=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRpgh+^lNQi_;h=fRp zgh+^lNO&2L@Y98Le%OKE18)N;Ktd!$LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{& zBt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|a zL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D)LL@{&Bt$|aL_#D) z!pne!bzz+!cHsBG+W-oX5DAeG36T&9kq`-y5DAeG36T&9kq`-y5DAeG36T&9kq`-y z5DAeG36T&9kq`-y5DAeG36T&9kq`-yu&O&hd+uc4;NXG(*?pJ8AZ+bhc)~qXHYQNT1wff7{-1VWVwY#gF9q6uFD+ZOb86T}$lLwcxr@p6Zoi0-| z?h{pO`-jTe#!pqPB}2;D#ky*Z?=EK#e5Pt0EmQOA_g1Ya~Cul?Do zwYe^5um4=tTGT3MONLji(SMRcqR?a(4RHtJb+PH9cRhT06h5oW1m$RcqN_DrYml zQne<2e>vOvTUF~snVRvxUA4A+uADvcyH%^_2g=!7zgM-!{9rj-@%5^8s7%d?->+I5 z|8hCI^AD=l+#f1uFZ^NEYX6mTHtmn9*1j?|+yA&~t@*3v?1n$7S~GsQoIU)fRqK42 z+Wmi4wRZira<*(lN7Y(>WjQ={ZH$xtdO4jpI>u9FZ0{WtDC)# z9Q!xQ>2(uhJW|H?x=Ati{a86YH#x@nf3uw4J2l3UKVD7`Pmghb8QXJa#klTomDAm` zW1RK3%jwN`#MocP=COG(?*55#I&xu*D}J(^u3i-53_dt%)FcgpF( zdt+SkcgyLlWigKb>2mt?@)(bnvAJ<&jGI1RPWx8HxbO?*bl|}lNBzBWx^i8N2g=x- zY+`J#DyNG!#5ntB%IW?`V!T+!_Vi6L?)m%W^vTUJ_Wo=+9lJHgDOZ=%^-sijwv6pJ zpNw(G&y~~rcgDE%=gaBwT`^Ah2jz7BvoRhoWBd8%V%$8uoZh-G#zp_IoNn765M%pal+)guW87E9=F}}Q zuKBfcde@W~XZ*`@`r@=0&zG@1V`hxIe!ZOTxGl!zzfn%d&53c+zbdC2=EiucjO~l_ zW8C)Ta=QG^7#IKRa{B6BF^>Jsa=Kt~j7Q4Y-nS&izJF6rr`#9g{AaX8Z|~*x#y`EH~n5YJ@j;p3;%sN zoxMB8QU9TwKC>sr17&P(dOpVH>*aLQ3o*|AkL9#|AjXSjY_B>P`A~vG+ff z)1FsiobsQ`>49T0o-Jd0=7|`0{6RT=>QszN|4TU?eT8GPd7(BgW1D zwVXcCALF9`R!&b`jB)fImD7cnVmw&J_6sAfh~MA;UQVZuigC^#m(%T|V;m@BGvLtIbC~4jAPmz<@nsZm=ERLELj-y#u0gbcu~xAN0zy{ z{qC6C*XH@|dt%;~b2D*i%xgyF`D4pso^f5ie#8SYpU=6ydS%SJM(6qIRWUEWK3{+L znwTez$@7=i#e6E~_N<3v-gZNt@7xgc;<5Sq8~b7&J1);3*%b4UoZADNW9}QD=PS0x zJpaag{mCa{9yuY;@7xjd{+!$UcgDQ#raYhaOw6+;=Ift)Hs=1En;Z7Vy!+-nU%xNr z6_fJy{rh8{d`q6+|5D7Sb8e2k9P{?cc|QM0%u8;~*Y76jNz%h#_v8}q2?dH%+^m=EOK+eEsI@VxBTP&qt1l`E1VZ2gk;|zqvz*&-^`c~=G?w?Tg>Yh zMGIoyduN`%cxTMhFU;oo1OudHf@J{=f|}AI-UWbzIDw z`tp3?gqRmTny=qCG3HU5@_gzoF(1gedE(ZXo5%8e%(R$iZ_d{@Gh)7&bNj-qnD;!M z=l9(nbMKaX{n0yOp0YL1=gyD$Y|icH7RJ0|Tb|!?SIkSF$k%VVJLU=7^L*5jn2+b& zUb{5r%}?g}>&s$Zv?E`?piV=l1QZV_v^A&!1Tn^PH#i^%Eb8c_8Oz z)59_E-IeDfHpaaAnSA}KzL=-&&hyioVm_C1)AM-DJD<(-m$t^dY)`&^=JuE;K9}b^ zcf@=m=VtuVF>l$M=Z`!SbI#jG4~zK^K;k5 zJpWj}{@yV$k362|hsVaeKj-$G8)IJgYM$@DDdt%x^7S`Qin%}M=CR2!?>?F5Bd5l^ z;#9tV_4Jr0zn14`X2yIv=jQI&F>gPe=LhG+yyQ&2e%8F0$Dhsfrx(P0H0S2VMKN!B zJmZi1}j9?dhvx-qWAw zPd*rP@0H!+{hx&7w)n0LID=l4Gn^U{HQ{qaX*o^UD8=WmYrc+TzT zx5T`;-C6j2>k~0A8d2uz6(k^FYqc zk>_LHdtIL2u|MY3qx1F89*B9`^?5$&<(SXq+-yD^^Ug7Oe(lkim)(%BfADzB6UXNH z*%L9J$hlekTFhI<<@w8JV(uBAufOeF%wulM^Ihj-K9qAa;mw#gPRR2|-->zeP5Jsu zmtt;D%=6xnp%C`v+?*N}^O~FU{I2U`o-rw3|KbfXpU=5HV|>iJZprf<6JlOIIbT2S z=9nkln&%sCiTPB{?Tfd@ylqOJFP|3k;;H%iS7*dLc3Pe6kxuPlvu_v}2Mdwkr3#H0S1?jWKVU zpXZ1AVqUl)UqAb?m`5$l^JgB9`9RLiP1|B_?#%N|+hd-+C|}=xD&~thw^uzK^Pao% z{IzFd?(NCf_w0#z%H4T>U~kN4b8gRkA?6*6^Zcn7V_teszJC0{m?tdB^NojMKAv;? zts^mSzBkVwI2QAwrTO|3uf{z3zC2%eD&~VZw_i9N^ZI3ZKK1pO=iHyK-~L9-135Qi z`eWX^JkKA#81w1}^7R)6VxG1l&zFrD62JdBH%G6HdFRSJpEo+@Wxe_O=f=c5aaEpA z9vAb8oSQ8-#=K>9o?kaH=AH-h^=l`^JZ4RvpPL->p`4o~Q)Aw^HqQ@Fk9qDlOxo?U zVs1Z_=euXeyf5cw;vF%sY4ZHBc`?s;xXjIng)yJcxxIQ(%)8d-`RSgRmv6||-+fQa zlQ!o0OZUcnD(CjBWifAiB+qv)k9l!lzW&CQF^_#T&mUP8^O2m}0}saBw<*t8tc!X6 zWBK}%P0S-V=lPu*V&0!~d;cRbuX{Ytr)`RP)|Pzzlbd7i&$+o_Ys|a1=K1<3VqUQ= zU*G>^%#)wU^ZR$kd^+dm*shqjZ_o4j&&Isu$$b6Z=VBheBhPQ$7xU4ao2~m}-t<(S zkA5lUg*)^0>t2p|)YEzX#^IO`a?fd@<+tv#-UxXLp|8 zd^YCZXY=)&&&51tPo9sw5cAob+Yi1O^N#28{LEW1FWsB3U)&BcI^p>|KR7bx<2ko) zyDsL<`||wh>tkN@LcV^&*qBG}&-1?VF(1sieQ83>>tD?Cl{d#c=Rm&x)GaX&gGc=hv@@dG4$E`iFXB zZlB2W^Q&XtmvggpZOm&<=J_iR#XRFwzW$E&F`v)5y=P<0yI#xlNsq?7{B*wl@yBAG zbSBTQ-4gSuoZD-*#k}ooo}b+w^WxX@_4hm#^VoBF{_@i?AIZ5rdw0xzZ{+!|Ju%Ne zpRd2^`Itvu$n!^Eh*%>M$F?!l)3#tf6Pa7ZeG0@^QMt` zzVK4a3$M-B?;G*q`28Q1=Tk?;d?4rMiP14P*X8+`8)BY4I$z(6i}_;C?F%=?yyyBn zzi(p9y<_tAM<>NR<%T?;duz;Rb8bI3HRc^-^Zb?>F)tmLuir8&<_Y8TeAMkRAJ4hH z_Kui0-#%o*Jt6)P?%%?FcOzV-@hL$fTSUTCKC*YAK?$c9n zjTQqI_}s;QWQE0WQ$J>;;p8U!*b(gUaesCU>$ljoakxUu zU;>V9vo~JAjvek-FX1jfd-)13(9(JXhj!WTGO#7U{qQ~9vd12OfU~sJXX;uLm~Axj*cM6{qaYdN@rB{&lA}!TxDL zH=J?5FbY>``7j12&e=_7*mc4E`6R4MviHouC0g#!!jVgMr46>FxL>ow?N{vY4)~Fl zb4zd_&7NI`C$G65@xaO(_R}@^hL)Whu)JjtY{DZM?!`7-qs59JKEGps4Z!X!_m+LQ z@1FhU04~#FC;~@w?8i~q{=j|v39NZ!-#&%&wD>uLgHP;BNq9QXeIW&_p4lg_;S4RV zZed@6y*~>ZU$~Ft;5sc^p5SDWee)UiymJ5b0_#icjuKp<YHa7hiDK zJG-t17ibx(!=Va$Q^P;O{eN)3-UzpRvR9hmEG=y<@b(w`y&5)ExgYI-#W#DR6Hd{x zOAD{n*!Ohs;1Bl={qQF(JOgmN&R!aVoxj}Aj=);+Fa5n<+#2B`EezvuxPd)30b3P+ z79G>Dx{-ay0_SK^x59oUyK5dcH*x>604r4NQ%*Qdi?j>&HnZzjU_%S{N2_p^maSem z(aP?VuuIK-)d%a^*cW%;5-p#0;Yd5XISAW2xKD=Qb`5*)A^b?ogCjW5$*zjQlU>|< z~96wuIJwR3gdt3|NXvX-rzhfhAVJzkp1Kno*v>}^9`#E?EW7(LyO;E*f-3+tiT&9 zj3e9^m2jPw)6H;ll>NFD_87SzXovM<>`@I|p{2STj+xlEdSJ&m_usv6mzmwEhYPgK z55l1d_K9KGGRghrDBLo|-e-cdv<#c!?P>PrDcCf_{l*L|EbO1NaEg|88@x8l{$Pg( zt=x|-!k@G_UxMRv>{>VMv~j=hfwlAOjq7lc7Hb=D*v?+wgsltQ&uznM2m75L&e39I z5B4vz$M<2gll#svtXN{-jlgMI@J%jW7rXle-LTC4>nU8N<;*#paI@beVb==xgIBQ5 z!+v}XmuT6Rfg`Kz+gaGQ#(gaZx39A=J;9H(e13)lUiQf%Jh{Prsst+~d;dFpL(9Vt zSZ=bnRN)aH_p%1pXz^8t&$rkY8vZHnf1CTqM!0W>-K2ucw76)2qki_DHrT$)eW(N0 z1lW~baGn( z!k$C!SEgZoguP^eE3~xC!LcLu+j-a#<$lNkcOA3GoN$4b?aOc|#=f%xTTZzDS%q8T z>@F{yrDZ|F+o$YPTd;}mm6BIGusCDa2jCPfBSCoWoW1n`9=!Oo+&qLoX;D3b<4JZ$ z40c{}{}hL{DfaO*xJZlS1suL&*QH=q<7N1Unl3ELX#Dmx8?9>bwV_PZ03vj%RJz9jF?cBG$!P*Y? zt#`Oci`oYq*04LPu(gx>=Nhc;VxRbhbF@e`;H`@OZuUMUY}RsrsDc$e?9HuknihO( zqgTiN*@158<$j?HuF~>B3n%*6O}((IpZoKESf^+28H7u;+&93H0e0mmY#Zc$Z47Q7 zVt*fpA89!^2?q@9*=cxknER1gSUJLeItSm-veORBQTBiX9x-w+mf#vKR+i!OG4|IL z*lpt8vIh5!v)_2(GA)KS;i#GYcnh{qaNq8SHIwYy0XR>KpCBBZVqZFdr>D6u9Kxy@ z_Q@!mp~Y1U_F35bPhsOM_mMNWPRo`goV2oUreMz;_g`sPZ)11d!WCLR-odeX_VFC- zuycR$2zM>8>z?5PEkiGG$id!Jf-Q^Oub1H#Cwrv=XK88sg1490@4sQwKfFVBv<{19 z_JsK7xPMx9HNtCd_Pr)}aOKakp#}b=g+~p?J?y1+*tyF6Y$vQ;W54Z&i?lH4;P5(o ztPi$&x$hW&)f?&!%j>GOF?t=-qFUqdCfXlR4 zy@aF3>~B}FJ;uHD2G*RgXEJb}7Q^>&FwTDR08gKCugSxz1iQZgXK3+Tgnei1%Wtsp zocrQCT&LyqC!D-szplcbB=-Y9u>O)g`U_WRsaD|4*s&D*mJ)VcasRD?yVC5=R=7aR zd>b6PW}nc&mK*LbyWo~v_P!oCOUrOCyq#fh*2AVd?l%Tuk!Alhz$seVN8z=5_J=We zFvtCv8UCcj`6L{FVAsyT&PVR|XJPFVd!r35(qhdHhx6=Z2W)-jer^d?7ufHX;T$bS zJh1{CwI#Lq>@D;F$!+4U=Mik6X8c&(4U)e8^y|5&e@ z9~{@SJ9c2_0QXP3uy&ArJO~$Qkqp7%A$Hv%Y&CFya0IJ|*_&c;juzfH>>pvTBw+I> z_wyI9!pMGq38!gcOvBzW_QVZrFmc~~2UlsicMm7V*%gnl%gnte59=n_-wJSvmb0&L zWRgAe2HU2%H&o#EY4(#(_>qO#tiIRX8)Lljc)GkHn>j9oE=WCu#YXm9uN0tOR#>G zUF(J`v_uzQWJ)^ zeeAmt*tEs{Zxj~W?CujdMa$wTytczWa}E#sxxY@rpR^dfg5$gF(QDWl;Jz&bYxmf< zvT%_WwHzD{vOAw(>pu6-&#*egK2d~ov`Cd;{{eg7J8TYffA|3_4%wTlaGDnU7?U@` z{#i#i9C5$U@NeV(Y5C9yC!*{o73@0Z{=5a&#n^k=;1VtOJK)F(yRr+m#kpV8!tJN* z?>hLAmUI1ZAiG3*_GMUe!@j)&=V|e?3I}i5m%Q+FhWmnqRd?)@ zTX2RJS39sT%ibS=jrZI~f^eOdEeCKi$G&+8dmgy|I)e3&?2Z^*q2*&7jyfg@^N5gQH zmaRrO(ZKFA!7jz0Wz`Jp8rc`8;1VsLX5fgD-E4(zP24AKaJ!1VcL9E+<-sBxXl7Tr z;K>&5y>3|9%KqVjZ)iEc4ofwAZUY`^Bivs$-~*g3qwIZ3I7`d03f?xdH@Cv3G440oU}0ka?0{3Ww0FU46)8gV0j(XX9o?-h2_n{Y9 zBiWTDI8Tf9G927wf3LvPKJIN_uxg7v`weGkFENz?_E;ZWpk@02915}T48fKI?tg~i zmN2`^2xn{Ao4iJyU%S5vS!V%J;X6fGlGcrD7_Iu8#X|5z@N0JF2eB` zyTb)LPq=?_!`e9e_$pkaMRFYupR(&DY)x=~;Dgm?>`gmxjuzfs*niGm*@Mj&+|P$# zMUwqK45w*fJc7NK?1^L8km9~O4p(Wpmw*#j?1~H6mFC`a3G1%e->%>iEoX1w$PIfY z1KV!7H{8Q`L-F71%gF=$NXw2q9Jpim7vRY(_w^#Iyk}p2gKud0@(#-!`}8L~^1yw% z3fE{c@B^PevLF4z?kDcm3VcMpFVF5%!ev@~tKjG}`(i6>FL0l4gEcSgW(}OD#bp;9 zEVB3Zz|*hXhkIdFiCv|KGql(kgne)99|qW1=H5OE*J+s>gOl&;r?tA)_<^T zXW$AggR^k#lfBUfJHEJIwZmOi_Ob&m(9*gDhrZeGmSIbc`(Y2<^1~iqgR`{MY{1)f z_T5d`^vnJ47A(ZS_V@ef_QNS!76b5F1N+QAJgE4yygq?o7hgX6~O;u)2kP;u_A;B6SPSb{&lhw<2loplxJt{1GMvz`n?7JyC->)Hu&#@}rv{g3xnGAP-R#PSe-HPs<$kRZ zZtr1#Z-O6bIoARQbnIC*JlV_rNC&L!V?XVLZ)n-6g=If`KnIWLxflI#jTS2d@c97y z>k#Z7keMCv+v!*gA0F_4Uh0AEj)QR?qDw!VCN$Dv#+q$$$t9=7inRrz~LqK z*e7gtao_O`tC!h#e&8G}>c6nx&F)hC`?&uV?q8I!!oxn*45w+4ZiT(8?D}@tu*Ut7 z2CmYwwHr>Xv-^5re6Zl}*H!I>bsOx9dbmW(r$IO(+0DbSZIk=tDBSL2?=``Xv^+4w zfh~5`6g;`jy>|vy?y!H%!Z);>x53iSp0mRvyWATW;TkQ@mf-UMd$${Q?{OdWz z#X4N3#p(te-DiK>gzX{jt=q8XfIZ`f^RyV=gM(rAlYMylkb6xSRz=wT5jaDO-ze-m zVqZRijZyB4r*NH?)8}yVnEg5ldt%%VT*3Mi_UJWSp`|(l$Kvc;S=e#P{dW%TO0YYh z-~ui4&v59BeWD0k&bhxV!7Uf;eeZCVmf;V0JIUT$g-w^-Z`5FsV*jkeDO%ba@X(jF zEB1#*creZVm9A!Ok1*_d8(iEqh}ZT%^UC77l0F%R1P4$NgMCtj@CE z4Zt~Cj2K}5J$rlvHs`qS9D@}P?7QP|nihOejrWn=J&kU7;{MeFS7|vj2Pg9EH}kOT znfpNptShh|JK+*7+m_+T3;Xs8Y%6kKTZP+S*_XWVBQ2jL94N6*Zo!jp+^2S6WtqJ{ z0N>E^FbK)hKOVbw2tHVVZD;Q^aEFDX%YVc?!SrsRslOy+z&OuUCr#VX1GAhb~PMoVc%(o zEv?-DXy6t#yQ>?{(z4J4Z?~~e^}!}S??_(hVbQ^^AA(b~j10qT8unHrJlOeXxoLtw zX;C%9@h*196zuHg{%Ho*YT3uFaFG^C8yxOo*Db(S9rp)|u)3GM$pz>XfFY{P~@?z?y4DlPZ+;KUHSA_Th(+hv8!6)3@tX=VBb9ZM+a=Qb8qj0>$J>i;p77QSTF2xaDUbh z>lfLzgK&kGK?59fvNw*xjwS9_$KWm(dwCo#(9${yhnCszreTYl`{7x*WraOH2WM%i zvBTRQ_FV^TTIK%N35zv$_cENKWpM>wTW6nHg9p9bUwh$CS`2Q&@eTIq7VMPVxA|f1 zCi_+ZF4CeFgu^~|=K*Zp;{N#%R&TRUMByAQQZd-S!`^oaoBiA$p23P;_U0s+~T&Bg44vr?+kNaTz8Tai2u;!e7dkD_c;%68RUa&72;prsz1rw~g zWS^XXGqkvxf_*9WehX~8;yz-9>$Gf{hm&dc%>~$V&HdLRtiNG*xZnycAKh^5mVJB` zc4WA}Sckjr*mV*v&@$wMLs|Bw9oTZu{rWE4l4Gyz!C6|`Lh$wj`+XQTJ#s&K1dAv3 z#4((rWmgqQ@{U8RR7 z#lQady6GK+l?{Iu9|rh_mh+>qRIum9;E_h|jb^w;i?d1iT*=-&1G}5J56;4UDt3hp zF4JPw4o92W-yE>Lg?sA~tZ8M>EW>$P413_9n*C%Ao^In_vjMBx+5MYvh8DkDu&;xC z*$*2v+!q6IotD%4aI%yA`T+KHaX%1&_1)~zC|seX`UH+?*|$z%M-TVkXK1XfD!dY5|bMUsFz4-|?4RF8l42wbb&lfmFOM3}k8)APb z!-EFy$3EasTAY8u@nLpt4R(%jzh8&7qwI|h|1s{L7Hf@g*vMXPf~{lR&$Ym66Z@SS z&e39|1NM)z$2(!Onfp#Hte9Zm)xl|6@S&96Np|-Dx?zg@*CDt{%b5{4G0lEsgk3Y- z501k+3;Xc|T%u*$G#r^_-?qRuEB7@k+&;&?G!H-0@_7Lc*w`nX@Z>!ADHp7?v-hvS zH?%xlh2;W!ix(boa4#iXqs5mGK3`;C*n!^+CDeVO~v z5v*~uD`Rk;7VB|1xWfLPfTum&+b&?$Dtq=4&d_2s4g1#EPj6u3I`>_7aGjQcdpPN3 zZ+L_~8{DttVZCH872pajEw6BFll}G$cKEm-s=!@a?6FU{K+E=TIJC{a^8;IUxc~Wu zTm0-U#eahPr)5D2Z||~CHNz%8+f-h)!eWnI-wvl}8PULNLH5>ecyRyEa>39+;_X-DlPXsaN?9*u@1Wu+^2-Z zXV3Uy+XeTAJ-9u|ezFfg(y}8A2QJzD5qL7ieLV^*uh^GQ;2T=LoWe5AK79_4TyvjJ z!Zlh9T*2oz>_^wI`<8ok2JXwS`?7GE7T-BIddI%_1lzOR=bvHCJ-fLG=V@_Sf`d8s z-gkKVf&1_Wta@ZuRpAURHfpf%iT$Gv8}r=T8~#(=KP_{OaPpabOa*%i+@H0;`WJR> z8(g7fumg@2*&DlH$1C@%TDYskUe>_{T3Y+z&>Q>R0Bk99KWu+^KDhs#fW;@fdm2vBvS@+VzSw8x;K3^Q*YofvEe0KM{F^=Mgq=0++m>PN5Bt^% zT%<*96%N=_T~dPO$$DX z*sEm!JVG}#ala6QtF(NG!wD6;=?r!?bANsT>sr`*QgDfu`)N4R%C5YHZEEhgNEi3w3$D>(bg51gf?Z5`g8V87phP5ro#p zrq~nPaEg{)yYSjH``#WrIP+)O5Q0Bx;R(ZW3wtR7J7>9{J%+Vb_S+MS1t=kHv>Ia<`OVZWW-m4VF*+`nXDg@b+S0Z!8*{RDd#+4Tk3;N5U`Cma5A+`ostR|!AT@<0U# zR@qgp@Z=iz-Zogd&i>H>-_UZt3zlB?oE9G0;NI8^*JyFp51&i+?m^hS$$iiO_xacr zqi~rPt7C9R805aV0@rCdy#^=u*{{8@C&c~0CagbTk8Z&gTB`kUEX=+YfE|b2e+S{N2)pwD zF3>W62#1c?C!(+=%Kc>wZaHS}JB72f44=W>m~PdU$8pGe)kRM zXfaZU{a5U9@n7Kn)7*D9!isD5-6l9q3qF9}d&BNlqZ@9ye{F}Ww4CXL6B+iKZrF9l z{h$ukW!aDW;1Vs{2H?m&`}PoQ%W+>DhT9+5myGZuEuT$r;E{cD0-k*0J~ag^^X&Z= z_=c8;R#-l>x6H#M1@7enT%*O;B7FYBzTkr0MeZNnaNjGtX%#Nh;$j_+me_kFY=7fE zVrKShBLGnJ%W9o?5D@D@r(PeI9#V?AOR<< z>)VRZ+4lOE2|!aE52 zHS84wZ0_WKeiT-8vEPruX<8V~u(z8%F$o*A+;`8wRa)-N!igSsg$;J;xcAs$T`&8a z11`~Wb_tI3v1gWHTR-;(58SS2KUsqxY1y#>2L{;voABfy_w_AUImEu~hi_>45`d+F zeR>}r8RkBH0M}?S5P{D}*pH&Hdz5?i3EXF7_npFJT6~|u(J}VLBy2ZvpHIP>adz`H zoTtU*EgUqn_h#Yg3GTx=ST)J6dV({w*m#D0Q|uovuyLAudkL=7GFOI^Gwfp@u*bsv z*%z#zW!Ki=3N3?mIA&#UZ1}Hm|8v~0Ho{#t_Hq+kpry414$ZUQsbPzq`{53_Wr023 z31?}k(Zbsf_FWxpTIBw>4;D^#_W+!tWpM~zTVkIXfd^gOUmM|1S`3cE@n!bt1nhKk z-!=_vSJ<~KaFG@@D;)N)JLh5RD)-L|uzHPs!U^YSk#fQQb@sj$*zD#0a1~Z;us3_* zG%fhGBVNh==|eYca=)+xS84gM3nzT+rXcLv;{H4Y>$cf@4&f3l_mAMn4!bf2+x*fMTWsv>!2EL(X=N&Bf*#r0RNQis!2-j$_l84U^ z*k222+u(FptTCu~3BzWp24MA^50;5;pUe&OIT`;y|n#{I{* zFDPNv3HxL-oT0^4E9{H2_qW5wQ|==gxK7KKZaA4>-|T@sXWW1F!uoS|haRrb@^KK3 zU9gW2!;U2P7o%|3CA-c97ibwW!=V&=(-dsE;(mPwZb`FOX5lO?Z8muOn*H7mn{K!t zU4+FgdtwPr(Xz`8uVvWxJn-Nh_YLdtCoMc1a6HRi+Jv3=+|O>q+8q0>A1>0uum^`9 z*kk*!^^yCIFsy!J--*CETGXSkKhN$ufz8j{znsE~0{hfCoTf!O3434I^;fW=$o)zNGpWqTLpPu1JncZB3ZSUMCOK^LIz4sk{q~*Z}9Qa^Y zRpH4`?!7fw`NjTGhi_;(-|*ky{;TY{MtJ0#d!q`j(c-KHKCiKNx54fo?t>k0U!7gi z1(#{Ds)eJ!>~A{QF8;~i>$0^U)-?QCW(MFqErtznP{DpO0#7$`uNi|?N_PJ^oT0_< z1ng^KU!I1ID(;IGxK7LIIXKzOemxI+TDTu@!1`A9s1vTxQoRhv)a+X;u%nIp?^U>~ zo!#k$3$)BjIMl&Du?1T+++Xg%EuHLr0XR#`a1h?^VsAcxP2JpY9Ku4&{&@taXlakZ zYd!1_ad=S2{n#1&NsIFfINr;yO~KAS?)THMwx7N67B13a?G6s>*~|B^b%6W1M_4_` zewT-Hv>17T{X^{WSJ-UezOxJ~hS_&3aGDnUvMKKfyZam6Fv|Vw4_u|?jQDSH|3>y3 z1?(E*ey|DFnb?n;;Sw#|)No{+eY+jDnYpiN;PwgjrEd6IT%*O889tw7Uzmd3R_-5X;J!I_lNB!0;=%?;ZR|Y@ zuzjBU&?2m{vnyS2o)+tFIJm(6?t!Nr+}qY+)gpU#1J2N5)Cc>V?5EqXaf$n`UARum zz#g1*u{VTZ&ocKbVOZ~GFGb)AEiK1zY=!;y1a^41A4{ig524gLOyjZ{u)@ zma~&^B+8zdhHb~(8)o7582iZ_{7B0VI~+J+_dDRpIQMlYtUP63UWRXI`LY7b1pD+F zJaWc;+6&icF|Y}rpR*rr!R`z0)qc1y$?gllWm1*!8XRzvqU6q70wAe_&zFYQ>G;GXpZ@-1>w9MVX$vgJ39PG(*fA$FL z@7cA_aD|q^7dVzq=#AvmC9&xYa2ChkX$V5N%v^ccROWoI0g&Fp~$Jkr9wxPWW4ShUfmJK%elNKI395=C-9I$ho``IN}Yi7S)hKsZ? zc;N5^du$E1PIBL|0jsCjcQ)Z1E$Umaf12Iphs`tGzXV`~g?(xtPSYZN0DEWI^%2-$ z<^CuNS83UL0w?C!eW$R?#(nh+*3GjoCgBn-pHgtd&ThVjZ42BdZ{cl z4lJ^(p5RF*_ugk%xy1hQ0^iVbz647bd#((REOT%CfNQik`-0Ei?A&w88w`z#c&54^s=9H!qXevYqYRRvio&#h8Dkl zuy2!nc>p&0xGxUDby`l3z{xH4Ya{I0=6+xt*6*-KC*TS#)zfgy&%R}W9lP9rTj8z% zyK^2c&@#UOhxXVfoUkRx{iO?T*=O%tfwQy>ufp3Q_GT|^I^cdo!XnK6>4Q_WwC})c zhwKl#@L+`du^{|Oi}Mg1KVsJ&!pr(8;pKytmZQpR@ihcVBwxzkR{le|n>`RLO5%*8aXC)lCVV`V|^+H||$9V11drv7 z_DUT#H*-JV@IT@HTiEX#;WRCbD%jh~o@jv$YVNz+;3_TmI^aYbyP^wrwR7*$!nzLj zHyvD}Sef3&+c1+%e44jg`)%Pi(c41$bDYInjv=c z7M!QW0$1}L0C1yt~!7-wAeU=eWUCjN3hYzy*&okX_sEUfJAZ#YHE;t#ww$37$eXWYMy`)dXKNsGZII6luFZHAq8 z?%UL`c7c7X9WK(Mrh&r_c4s$iUF80`2Ua`TC;H$VEmC^ezr@}*1e;yl9}dHcW%gzx zoTdf8PTcEe|1_f;R=8i7f~&NAn1K@>c9Ru$t#W^EgLP}{JqvJ&mivovWSw2JW4 z4*P2acKf-v9K(IP>^CQHnHEC{I2vF-K8Njl+_zuCnjri36`ZHV&ovy}XJ5*|(;@B) zSy*+zKKTG=XmRxf`@-z~1=x7VeWVE2Y1#4yCnM~e@37~H`>zjJA7yt`;R-DuYjEtC zef$^31C{^&T)k-cUvU2?>^db}pk+t}hvMu_t+3^k`}H=sCBa_lfU~r;b-~+b?DtyO z^bg-kKiUh63-&}moT6pdAiS1j-!s62mw%QGqwps!JY#S?#a5cbDxaC?FIJUQ}~gV z2WN2LgMw9X#^Jy)g&ZXmR!kpO@LYpJDeq_rVvq zufnbl-5 zzs<1khkaQM8|&N`+u=Gbr#s=~FZ*>j>=FO$?|ue!u)g8XBH9O6XsI56V+!`IA=uH# z{r523rDS&+;Q}r5COFi@J~07XRNP-q!7a_~eHJ)N%di#RZeeephfS^AZ!Ew<&HlLv zr)X(+!E0^o4{ms{o%^v>_>&gr>u|h-T`OUyhWmXVtnFlP+<}X5tiYk>Q~dssKfe*6HJXxWyBBSY-l1=wcbzE*_WhuN3j;73|Mzr%qM_Q_9pa+Ld2 z6;>MA`+wjYS|0wwa*Vx2@xS5zP25W*T%*O83O*laUucEhX6_%`;JyiVlLju+;-U+V zPO|s(!1gKbL%pzOnq8@f^R!qWgo88e?*@3Wt8|S$1 znt|)I49vnw8+(Hd_RMp?Vu$s1_L2jx(9*I5#}?Rcmtlv4`ymh9wa6Y@gA255-+)6- z_MJ`Gvc&z*7Tn@uclqHgEeip7dzpP|A2zwUzdC@$3cEf6r)U|8!fPJ())RPemHW+8 z_>&gZGdRA+?nuJUb?%>1u-3~yehn9Ck-UY&8|=C)Y?a&}A*k->k!)aO=KVa_;d*TZ=__^<{!BtxB)#1c0yQ1NL$NdMm_cX$~J@&UI zxJ1j@7B~`Q&!}PBKKF(WxIM&v(g{D(vO^084%q!Vcrwg=y$@C%vM&$7H?({if@Oq# zdITOh;y!JJYqS^`htH$zM-#C7n0xg!+!tf_S>Q4)zO8Wdgne-yw#T{8FTk2pcC!=C z)8f(v2NUeQEAaFg_u*AobOi&dk9x(89ahxY4*k#?6~HBH4b;(u$L2XftJ<_ICRT?cL`fE+z+SWmOJ+N4V)e0~cvg`-Q^=cBkV1!2Q2)|Ez@7MfQniI7f?AE9`$|?`wz6CGHP3u;Puqxf@Q? zLiWJkGW%yQYK@V4H`7j74D(t3V*!98v`6#UWWbZM-C0g#A;m8-eatgLpxnG-s z+rQc0XW>U$&e`BVjXi6JCx5sfS%j5!_R}T!hL)XfSpKpHJn)G4H~)T45bJP_7AqU@ zc?0|FChS)HSy;B=zDD*NKU}88&>kFBvLElm_9pJz!>~rhz8!({wD^g_!DjZQ6L`9X z`@$)#YGt21hcmReO2R%hd;b+|Y~wz14cBSel7W-$?3-EG)4}~$4%Tbf9ZztDmXFVH ztdo7b2s^sCzbL_7-R!z|xIoL$2OQF}H&tOv5BKXexJAcasl!=X+8X{(+3K;nBkJLH3dkb`Ehr+Yf6E?6(7OkroC6 z93Ezmjlk9s?mNa{^(gz!IGm$JeFF9y*4)PtCz;TBPS;?>M{O0UONR zA35PFEnAo2#00x<1$IqxUtNWDQ|yagxJ1h*2}h>c&0DZ-hWq3W+-_m-4Zx4IJP5*p zS$5R{JZa_Ldk8D%*guZo8(PlCU}{8?u}<~jTUDY@VTA6I|aKJxDTe`J_ozv z7B16b^$w0MvcKKKb|?4NM_9AOp2@>`S`5FyK^ObUD?Gi-y`~JS-0c1eoT0_AH4(lR^@Z*Q|V8)4H9_Zucy_}M?raEg}pDR^y{ z{b2?k3~)bYg+FO=ZiC}{?Ais`8RUL{5!UXrH@e^=E!Ny{IK*D|z}5rq=hk6$nEh@8 z&e3AT2m24%8{;>x4U9+2h;W8~Q8vbwG{|$SO61LxR zA5y`Z47;)w&eLMO4G!M1zjwgXS?+CJuU zal$PncGohTrDb6S-hN}BT7ykx?ytPCcxTsd!YNutw&1l2d#fKF{NR2w0Dsb=8ieDY z?2ZH2`NjRyA*`*kk4NDmEs`-f{LQXAg{?L256)oq4|`J*&e6h~g8g;&N*Xr*azB3y zE5twl`#ItM4o=g;n1j6y?1@L%p!l=seuk^G+{4>?DZ{!Z_O}XLqUG!t z98s}nzF}K4_l7#$-okz&{vX^wEjt?FKr6ey37%AQUvGw$ZS2cx_=c7*?XYZTpYDW5 zI=D}F!!=qA=-_h=`%xe4?&MxQ0QYsV`-b2$Exw20XgB+!5w>f&&zoRP54(8+&eP&@ z3J&VndoA#EFZW?9tmy@0Q=Y~>>1?#Y#r7Q zv1=t*hA1h&fc_i%iQJ^BDUr@3#-!`d14 ztpZ%6MXd;jE$q%W*gDJo^E<4zvQK=%Ia;Kuuz!xd?*}&9xIg@b74z)PivJh)PYbDp zy>|9b6>M1GexVhv((<7VPB_?28rZeS{dpIxbF%mJz$IGl_rj4ScBLM+xwu~&gxis!9l$<6dw&Er?s6ZA!gX4WRgF|8VrYG2P$o={=+!A50yuevn+Dh>D5&L}^HbuD~{eZhRzR_YDpIAMT$Po<=wxXD>Cu&QtDZTVQR1{Z^9jCb@afz?sL3AO+Ai8J+2oOTR zAe!he0t5&UgXm2Ry$KjV2+^~@efC@bwbygPyB2dM*s!?|9>DG=_VXh+MN6Xv$DY}@ zPGSEG_wEb0Ak7}Vg3oFBy@4aI>?0{y&v0))z(2I|Kf#A@>?beqSeE;`3|#xpp2)%( zS~)-A%@6k3Z#eYH{pT;NeX$QZ{wCf(tx|dL$~Sv^AsqPO{-_u(`em3lli@ERVg`H0J3j@BUrD*_;mauOR!9ExFWntJ+%DylLTeQd| zJmqE&Ps0sm+&|60AGGS4gLgId(_c1dEv@c%;Kge8j(xbJhWq3poL9?Ua}1x-YV8D8b?imwaFdsNbqRmb@_r5P*R%KE z!Qlq(Z|~viM)p=4PSbMm3@4h{D_&vznA@Mf`^p>a*4gtu;1n&BUvR92{q+a-w{qW` z^Y7vPx3OR5!soO!7r+r8`%V$8w{x$Rz(2HFEQJp{*mE>^tdsju4_xbKf3Acxv z@+AC0t0fEGoo08O!NZFCu?x6zhW+IVex_CTEo?^E&r@*kEcXqMuxpNe>j}Q4RcRVt zoM(?_;EpKwKUp|$fqmo?KBd*;H>?)f{qnc){+GD79Pk${b@}jqjD5Wj4li@>bi&mu z>=757rsdNOCsx@9%3&kUeX0U>ud(~8;S?=LwQ$U2ud0Xr>)gj1;erHvfexS3GTjPC zHrU^Mu)fK?(Fy<1>beU)++uGD!eiUq@Abm9JM5l*I76%00KB=&o;wVO_P8Gp!`eQ3 zdJKM`Rd5nsIbgq-h69J(H%8#1Blhh%_>xxc1$ZvWzOV$h9&`V@42#7cj>Ab>*(N-3 z!rrw3dr!GP*@CmQcz5CLGxo$jJb2E1$r0?iV4ppPZCbug;P@r`;5i(;;{N^;c3!i$ z-@wfcr&8?MH@M-R`~DC3gH|_R@a_Y9>n}X~ z$o+oKzmNBCvsdK7&$L=DfXyfNykfZbnfr+n*!9BxS_Z&@>W9;`)OW*)FZPWd z*!bq&Wx(zq_PG$8qUC!Kj{UL^jlh2Si+}!KJdDBxIe)A=Cg5{gj;G*=gS}=3)^oWx zXW<`O6-D90JocGIcr2g$_ZVDTz#dwKGqk#0gEtG=+Y)f7i2H+0SSx0)+<_lxwXz4V zIN9?L;Xn!ZlS#P9#h$U?OIr1w!E>eTR~K-roBQT#SjyOUZ{Z}Zw0n3$V_$lNz2)5J zJi}R9#?tV%hy6JN4_0vB{SJF7+0Q;L}|xv;Z_eLf$)rsby) zj@GgdJ7HfP_m3{v;breEgDqODa(Jqqy|xl=XyAUm8vdYFaUHzd$R4SOhnu+nXoM@9 z*#~s^nO1kLu&J~A+Tq?7?hiX*S1Wr}0KTQwY7ky*V=w4~JAB+v^}~7X>~90`DXsd3 zVb#HY9fq4ax!1?xFIx5{;e9`Qxq`!8+{Yqtb%5P552tAvUw{+c>@Q2O5#&C&0=s+I z&*N~4md15B*2})N0sH&7cW=W5277cDKBwh(ACC01j~u~zh%R?ZZ>Im|x$0Eb4n|FmH(%s%)6KhP@m3a^Z^w`bwN823jXaM3t> z^*4M;tN1TGH^E-$_z&^^C%K=_gJp_6TL33%Wfa2`)9g1Tuvc;4;)b)d>}&A$47;ZS z9*l6mTm^e(*>h`Qo0bVL9G_!PH^9Mp?t7YHXO#V-1-_=G$p=Rl*ta`i-y-*AU9e+` zeW4q+XpvrcD#ji*;D%-HpF;2lt-6Nb-4*uJ5qNl&d+!)r8E4;^fS+kqG7Xz+>~k}4 zugU$_EbLlmABw`aw7Oq}7ZdCq%W%gA_sLZ_ZogimR;mVnh3d(jr$w9UQRfxl>Z z--Gve*!vIR@GkeaNw|8Cz4Zi6({gYIC-&JZE@9(<`;}|heaN172d8M6yoX~)?5~fo zKgoUXGhA@Yewl{PX=#3gBNqG4JFK5@uYJKkv|9Xu4^P>1a{eQ{|1<7KbK%-^_UC*! zL#sd$ym`TX=7d9++}D@F+7#imj_}6pKx^%d*mBV)AIQXCyLnz9RD%izmxk^9_%h*_Z7k^T8@h0n2Wv21^Y|6kGtUl zH+w-jd``=B1so}3f2)FZjeDaO{-M>i7d|X!Z)t?bJlyX!!?hLco>n+RtC$botYpva zghN%_k9WaZHG8@nexOyb7hb7hzcAoHE%%KBa8Vum_7HqYD|Z;4^Rh3D!L9Y&|4zWt zz#g84leDsD;E6`|t~uD-#QjMW&eGytg14L56U*?R&V5N7_O!6iny^jFR|1Z=vJY;- z!8Y#icVMTFy?r0PrX_g@N88z}k6~X2_iHDxqm#Yx9JXjtm++LIJ$nr|baCH*2Y=A& z<{sV+u(#Uqa5wk+&v0dsz2X&qrq%KrZ1%9{eZalF+)sSLu0HnHANZD5Jvo0H@84j* z%!NDpxo;|f^Fr)9Mer%D%1U50z`j@tHw|(x8vI4es0ZF3Vt=ZH!^7MMYT)V-_R~5z zO-p?PoCvdTG{MFw_pTP$J;pxQ2B&EGZii#z>_dLoKf(P&04|th@92TgX*uqLBU9`( zAy}X0-W-H~XjL=iT4c|x!juuV(DBOG66-+Y3D3GPeNuyccbJ_BFV z@{@(5o9x4%uy2d|$8XrN&E6^h3En>~mII#JVXw`H8+N&0FN8m6RqTX!_t+yYczB=t z4>w$Sz&=n8Khx^20yYoXebsR95%-6+uq(-4RS)0NYPAtwJZ3M@;SP)Ysa81eg#FD2 zpVF$Y6IQ3}*IjVa8TWb+{-R~C7v4W-FYkxL7u?4N;Oa|u$1t3xWjqWguGnA3VB?zm z;3Vw6VLzXSQ?xWj;Mgtu)*S4=kr)9Cj3Jy{|0>c z$bPZ~kJ;SU?ZUNB?1_CiLo4SIy!p&NdklwOxc@wXwKV(SIs8DY)Fr&~%HDnh2Qu6r z-N8j~?9~tOC9UE%JeOrJe1Tivxu1T8<%2!@1}AA{e83Z*>^EPq_lx_MUpPz4e$Ic2 z_y5iA$%6-fxL+=SJ-_U^#js7wL%Ho!G16Iy=QPiJ^SSad`?UAH5_SR-?@eLM((wH_=i@DkMLm=d(JaF*3A8A z8m`sZpEGcVR)KeTvxWWa6AraAAC*AR6iUIv1bNg-vIZ0!?0tJ{VEJw zwCLmT)DZjbB-}8}eYt`^Xtfl9cSqPA^YCz(`>_SMa+LjL34W$k_X=!|v7g7`-f`|5 z)?wEK`_=}0ORLgtcyW?Fx(jzqasRUq=S{PZ9KolwdOU`eV)viIO*7nE=kOOTbyx6y zgnj)64$pG$Ou^N2?2!jJP0ObZC+67)USK22ed-l-2SUiA(8m$;Ar z!UZw*0>|IQ`=@0(4~{IezZJmx3in1a{6nkj68LbHy~PcW#kt?p;Mz5IPX(NzRjdl$ zG}&`&;m|tw<6c-xu%{c~2U-Q2;gt>cixxPr$$g^_F4|(>?tm|8f!4@rQ2c9}-&+fqu7u@$B z!XLD{Ny59A?5!v8@D=y_XK>{;d&MREOsnN<*t}uSyMudgxu3X)U3cuSkMJ$6dY<9M z6#Hcw?zrc^=?%_%VBdL%Pia;51*=E)#UHrI=3a9CGrWIVMswl)C-$d&IQ-0gpa`yh zVLx@kXC}r{!3O zBOmNFZLt2yz1a@`(5lD}AAYgV1mLl6?%#uO?GJmX56;l)wjbX7Wp5jVL-JStd?y?X z!&=TCOXVp1K&zE;c*Vh#(<&`;!GhzZ?3<@>(9M16 z1?((ipTB~yY5BQ z_fr*cUL*Tk6?{sozFJr{v0r=Pre^N-M)-@Cy=Hh{XD@Gs!!6v$d~kIuyQ33M(=y%# zC)(Iwx?#h|eXtjHx3ixcaEg}30XWvdzBL5}EfS!ec@1>z3f!9`?jCoS~I74sZ6d&zf+kkNeLAtQqWsTkr#|QakWU zKYRN=90+lLbO;v>uvZ_$m$ZtXz;lD_h39bV5ckuUune@^|ylvZnl zuv%p=8iAYQ+^bRei7EaT0Fb^jZ>=lc!vBCXH40dm_ z=dHpiS|-=v*cSWiI_%%(zIPKY*kQlihR^ldrzQ?_mgnwwYXu*g3>^W!f z*a7#W7jW$%`|}l?p;h1(-aKMIOTnQe_w|pkcFex{1V7Nqm4;U=_W2APIN|;~3m2WT z4}Zd!w0ii4=g!zW<-f%HKj;3~0m}t@Z9bf&m01W+T(TECVeb|9GcGtw%ZD4@zGe@V z!-F^6-&Me#TlThU*rw&M7LMPsSJuP96!)u*u=AchUx%-0nQDcj59}Eq?0e+CuM>9I z>{nf|MT;JUr=Hk%d*Oy>?#uh(4_Yk^z`HN(j$wE>&HY#yu6$*G8H1l`)jbKD8TRvO zxc80wh6wD+vTx17x3nrIcIUGDZr~IxM|W^6 zkG<*v_UCgSx8Z^U_JSAqoR;ZVI8w;|_6F-k+#4V853R1h;KO3}mS1?x$^Bl=-^2Sa zVfW<08Ct~(;7u2MZZRAx<$k;b*4*srQuu*ZK@DChW54jg0gd~{D!8beeY*y}q?OwX z&w1Dv8sOFn?thzLsbmkgz)4!!ZSX`DdsheSt>*s34`*rdcEj5>?1>(Du$KE01NPLh z&xT-|majoL?qwewfrItj-;cu12KM#|_?njF6dY}2ubzQ@P28`|!j5M4!YFLfq88yP zojn_a8(O&UUxhztb+ZQVwz9V-;Ndpz_c!56AA7|P{7kFmJ=kn#&pU*BJGh@n!mdvC zR|~$SRnHl`=x4vYfIGUlZ@Px_0_;1t@F}gz?qSuz&_-J{e#>;xZr{z z_Kq_6oR;HqI5Ny$Qwi%M+?&<#53P#o;KMNcOg%g{%KdvITsy`d(%}rPZd>8aarU-$ zI5ffiK_{$DvR4M+2U@KJ;gu=&{607^&HZFQT%_1D1MnrSdWYe;8TP9%+#2D&c^sBm z_T5Q1Nh?jk6LaiK5!gG=ea<|brDbdZ-j1?AFTsNg+;^|Qo<;VvIBe6>unxzU*f%%e zV2u0HZP>ZYKEDfJ)AF+qM_1T~k6_;__m9W0BhKD=3R|>T=kU}Td+in6U~<2H1Aow} zI0f&nvqv7_;RN>|He9*EKJWrR)9UUOHaFRQS-5wL`@;{|was4j4d2pg^%q{;VJ~p} z*LeTC+)w4fd3)?{1@I}Y`ifz-&wgD3Hyv=VyWuZd_B43^kiEPD4j*wJtAeYO?2cME zP0P3!P8_qpG{A<%eXtpJpRk{|z$sc9eQ@lQeX9fZpKsquua1Ex$c*YigHiZ_ zR;fjJ<(|EL84f&ff3yl0J+fDu@FlI{33$$CFWiD#pSYjif#sP!y9Xy}WgNm2FYGr- z*qi3QsvVR6|zp{HS;lT{|%h#~yjXn1cwrQEThvQlH^dlU6=f39|c7Cv5q~U8? zn%>~(C;Rq0?EB)r>7A=zV-{AfKu!nQuhF|WV^5G9!brr$8^4I=+51cyT z;haBK-cq>I!M;%jKhvtj1Dm<*bCqy!9`|3>uq&T^s1ClR)qOp@Sis)V1a}m2pVZ;J zBKDd#_>@*_?XW6lFY?1pPVQ9z{-WhQ2=AA$_xHhJ7x%aQaCIqr>mZz_?41N;0H z9BAbJ`vxv*VjoVym$Z6#fajXoJD=cIo%`b#SX$U?GjNhtW)_}kWiS4Oy=~mje8X8< zK7QeCAA890-{SqZbAOiydpg+L3Spa;!(uqz$zJJ#gMRK;-LSKZJ--~jre&%Ejt1B> zRj{v{`@UM(5oEve!WJ!hBRti^zS|5p^m1R`3V+aQ$p`QDu{%29VT1dzF1WIv{iPdz zrd4+@Y=+p+4Y+rJ`-TD7HORg-1mDuCGz>2eu}8IwFU38!iKOu&gr_JJ+fnBqRQ1G}f$efw~V zmZL*Brr4{FVgC&G@e{Zp!d`F=pVKmZ2}fqx->zYOj(g({{-M?NJ$yLN-eSXJQSSGi z;o1du&nujvRqPGkTx8GvfI~~%kAJ~hj6MAWKhP?e^Y`)om)S3J;lK*_jRkPgD*JX3 zd`T;J2|O2PUnqrJ*SP;JgT-VId*CFk>`HiIoxQ6D_9nPLse`k$cpKpD4faG6Jh;hy zNek@RVxMh;ZCbwC;rKTDpdSwIaDN|woxAMqJ@7Rx$v!x`$6g(Ref!+64Z@BC_QDa^ zqD76uQ-|!?ak$}#`~E5TgH|^R-c7Q%&cegT-0#oBl@@!&BK%CNu_XGP}22RoPorPnM>_eZh-{$_| z8!mWa?~wlv@1K@q2ON23ugQn?7w*kM_=i?SPWUj*KI4MNUb%mF!?hXqP&u5T)olg5 z`NrN>4TrMaAJoFyJ9}k4{6MRfMtJ3eJzs|dpWIKj!bM-~86SK}tKLp{?wkFp3vT`4 zzBvfXFZ*sUoTQc34^POy@#i~XX#n=-{ISXzhO@Mch2d=n`|}t)n9F_lB<#s!KbwYa zS{fp7JfD4Y4h|M@U%CK03)$zF;A>ibmf>g-`*0lg6?6Y+!VV{U=LT%iVr{`wCG54k zaD$8c^?mq*R>eo~ZYg`@7#?$m$&x6Bt z+{X&wYA?H^7*5kNUIHiT*ufM_ORLm3zAl{-Kq>13qkHKk>t3KJM$f;o5fgL=T*ymD7MXJJ@GKaHx~} z&p}x8vk#8I541{+!Yf_u?Gtby!2Qt_T-428Jp*6TDn1L(1=$OuaBC0u(~Ge5vS(v( zl2*nlJkiH~vj%$&?pqRYmX`fZc)Oq7vjY!?xL@9bJp=5yhpD)m3aeCB-x8ou=kkz znFlya%ZCkbTkN41c<_Y#yI0tA%HEcRZCVaL;P@GPoN`KLj_ua&L{mU$oSX!TTBZ^$9rq#=UbIuFkSYX5cg}pR;h{oqZq*8z0=K7Gd`% zyKfmz(Q>p3$G+IBOxXX;eLMjd{IC~n!RNG0@4%5?_P0G)mw)rmcY<*U|Iq3>2_NRL zx17LZjz3m=XK-yUyXO+l&?tXxc`;^3Gd&<9?pf6w6gQziBk5iBG~KZ{=^AqY4MiA z+hy#DGI&tqzQhB2%GqZtVVjn(YB=s;AFP9e72My~!_G?f_9pn6mZT0xtJtgCU|%)& zYwfV3hP}`aTePSEJXOn{4Z;m|-1qmvAGEsZhj+c~t%LAzJ@@;=aAgB~#VGtttL1Un zY-G=yf_s~|pHQ%?nf)~a-_ojQ9$wVhFBjmB7Vev3a9%6>&I)`=tFkp%wXrX*!%aT! zWfT6QWpo?fZ)bnng~J`(2M*xsPWICyI895v1tg_*#l7nSb_dw!uHY0c-#2iq zn|&w+`-9v+JirA#>>W?=IW5O8aHN;LCIjnz+?!eWhgL4dYijCH}=Q|!;(@ZdD}-Mz3! zv7Z^RO-sW79G_v|9D;)p?n}e4bC!L648Er2X9AASu@6tfzIpB+XJAK^y>kw>XtAR3 z)B=0$65O!J{rWQeL9608yt~96G2!7D_a6zka+!T#3x1~6-41N7u>1Dm-c{}o4`ElF zz3LdgrPb;Qytu|*a1M8v+)rJ?dF$+N*YGK=`tD$rV86bHn>M)DZTO3py=QoTlfC>E z4sUTEdxNXD*&QEnnwIe|II+Y2@&g;Y+y`_17rg&H_VZjgMN4A=9NTB#DuVq7+`CKQ zf`rx7~_UaIPNvrrEJa^4rI0Coca6df? z%Po6$98S{8n1Uzn*l!fy5c3ukHBpNF^a**%N!-~;!|G1&9Sp1TU$v`nnQahpB8 z4hNsO@7aW%&+Hf5@HH(>dvNrHeft3RrMWLl!j4z=1q-%lku!KI!ydkX8{W8ox`IDw z)pZN+X4y|u@bEkL-bc9dgMH%(ex_AP8a6-K=Q42b7x!OT*!9gm^ai!#E{9*5q ze}MP@%YD)T=gD9H^PNzW51-O%tq@i@>_twv$??ZZx!^Ba-rewiE_;7D9M0qZwgRrs zXK$^B)3hAa!ifU*ih9^62Qjc$yPX4%>L?w{Z8(CJK=&7_RB8#oR;Pw z9C5Ml^ul^6_gX*vL#xFB_|VOsGYpTFaX%V{Yc=-gF*rl3z$Cm`&VDuxhdkWZM_{dj zeRB?epp|O@Ua4fCUxEWw+>-_cO-FQ$P2XAO51Ht{dJDv9I^Q;Q{WQ23$SJ9tpu|T0RHi#1Q+y z2y6^ixWqJ{gjI+PRV10snV-^0P z)%6;DILY3UfXAk|-`j+1r`bI_aE4Z~J$O^G=N`hL8SclEuohuYTkr#|f@ko`Ec?X; z9GK(2@ft3gXWzbsFKOkzhv%Z~3y*N?0{6d9uq?8N({Pejb_SkUV()r~y)o`jKH)4a z-XC~-nLQ!@E8hPK_a(WoXO(?6AGT@vDum;4_CY5cT;u-U1v^dl_A>aImSi~`U1zVZ zgnbF_*Q#O1276&0Y|*0X;i*mbY$M#T#eKgHf6(ft72e%uZ*7N%cevm0ge!O1D+2H{ zt(JqZx%b!q(%<{blGg|8`+uw^`eB=v*8x}^u=fnZ+9CIsVc4Qo(>QD#vF}X6lH|Tj z!6q$>5m-NFmwDKxWpn{n7W>l@tex;qUdFSC6wrJIH3L97K$LFwIb6;}>o3xlWuztf{l!9$qW*%U5%l>Y| z+8ytNUSNw>x392~VsFdBa?kz22W-+(`3>t2>?^;pO-sJxAL0HZ`(z%h*}Rh}fGt|} z7Q@C9`&9`n&)he=VUw0!4c1@SwF=m#WvL2QY4)62SbOE2F)wV<>bU_nGVI;Wu)J}9 z)&iTfH27dW%f8tG+q9H+!RnoTz8ls)c;}}FwrDkMz{V&0V+fWn?mLHIlNM_P*1y?n z$6%Y5^$A%0uoq9m+Ar@!W?+j}AG5F_|JI*(J`jZ^=a1FhB5cy)TZVN9`{62V(^6%^ zDwlmV0c&}@Q?LbFw3^z1jePdEJy;63?>mG|TCS6@UdXPWz&0&=XRs<_FTaGfV%~{e z!xpU^cd+4PAHRpCg!`9A*rX-+4C^lT^E7PJ()b3eQueKPSab7^`wO;c75#yYGWK8j z-*8{!ek2z*X|eNRy`0@&1lzQnIAP^suPcSM3f@VS!4|EY9@wa4pRI(Yiu=!M*ra8! z4%VyLQ}wV-OM4TnYS@o-SgYlo>NeP-RlFTG>evhYuz0zj4!|ZY*&wXfvm1S|P0LL` ztQy!`24Ss{clL*2i&mad*l1#39*3ow``jtmq-8?Ey3U@Cz&0&C^RQ}RzgU2^R^Dlf z!4|EyS74)!y=)B@ANLFEut|$-!g@P4coN5|H8^(?|1y~xZlq^w|THdtJXr;2(cd&!!p2qg$p)mS#iVqAbVapY|}DX z0jnYQ*D6>W=AGVJ*rL^?7dA%Nn;T&XbHCFJo3v=Hus+JZ=!0!qaynr(#y;8wYva80 z+znf_3iQIp1pAo*%Ov;p1F%WU<`Aq;vAe>sP0Rcktftw&CtywS&hRvB(duCaHfGp6 z=U|C&e;kEPT56YIeU{x^hHYAk?{;8W;J$4iHfcFL zg!M)C%4683W%UGBOYHgQuomN;sY}?RRpuHtmf8F6U|He*>K-;}(QQ~?W#4^ut`fQ0PBbBz8=`7<){x zY{Tk`y>SoLu6bws0Jdo5PQu0w`+^0_E%(2tut`h!0@m-??JL-(rRx?}DfW{TtljgD z_Yt;em3V@U2lkRQERWpJW?+++uPm(F?1P`MP0Rf^te)80<^RO}XWmIVV2f7O`LOZA zzE%iJn)^a0Y|^4!u>Q)Pb;C9-{pGOAu-{a`+8gh*R>Kyp_G@7y%U)3r%RBeWjj%~e zo(}6D>=Ui9P0OnfR-f!Wov`-BJC|LsMXRPDY<#ot^uqGPeOW(j(y}-J>%Z(W4BNDf zhG8ZD_Mh*ur!iQ|`C|!8!WOMgr(wgvULS!am-~%5*rdg^0PA_|b4##I%l9&@^4W*t zuvWl34<>BUs$&B-3fYghU@79hW*0VTG52A;n7!x-wrQC;hLw~3{RGxZcqeoYTeP~p zgbf#a+YKzG+#lS*CM}f@uqo>L5KRlGA+0$a3tE`^P1_HGT98t%_Lut`fp6|C2? zZ`QyzEu~&q)v?bvz?zqLewtv5R>Li@QP2L^21^6?ogJ`Ai{*#)M)uln*rsK@2Ubn& z#RjZ3^G+lLTeSKZgbkg2U<8&H?(arnlNR3uthcfsPQf-URWq<^V_%(xH6QO3L}80o zQ;V?C&i)pIrGxvvRoJBEdJWb)+4Tf$)3Ub-D?fYr4y<+YPHYdhXyrJBjR5<25|(c6 zUo6<9C3ptwLH6?t*ruiN8dg2*Teq;*%RBCS*rHYR5jOhRf1hA6xF1QwCM|Xb*8AE0 z@32kF$tSEr>~%k|Ho!Xx`Nz0VD`zfj46@JW!!pGEXCZ9TGU$Z$VfK^@wrOcEgVhN8 zQ8}!Id8fJ(wrCZvhK*77!a7*SxSy_vO*jw6RZIXBPJ7J4f zo&ao2u`dToC;=!bR1o*sa0T6%_IHN$=phP4RqG>yX+t+pp&W0t*4!7|7F zLIgHxk$G63XAdvHHZ4y}u!^#Gt-#s>@0`YAi&oxs*jQxW*nnk;`;u+gq-Aaw)?@5n z`>;*R&=IVb+3$~GZH0F_PGO5y$#d9PWv{t{CC>fY4Q$dq&N37i`lq-wmr{_U|58vv_CNfGt`*gka-@y>kebQ|^yPV3U^GF<3ui zHz#15mf~qxowLu(z}f}xe9XcYtwK@QxMaUugyo9+wq@9)s=D`wn4AaetMBO47a;MJi$AgZ;A_mQU^n>R^+WR6VSJ zvHO}}o0cOTR^RMZZLs#kJMnhdqE&$(Hh$Ts1F*>7`13vX7KBY&j6PV;VZZK&ZCY9e zVdY@o8-}&qKNim@Y|$z<4jXywxl^#@b3d+Nla_P@)(hB!^RP|J#R9Af*&Ac9R>V8o zE3ic?_Zn;zvoEZ};^h8!12$<1Z^L>CyS)qBv~(T7%Ef+i1Z$=kU%@^u628v9@hwrRP4fK@qr`xC5rcqjP+TePaqz(xi8S{9Z{?h8L*lNR+2 z>s9R8U)ZLl-|^3Iznc9f57ugUr?n8aXtiGq8@22eE?DZgUv|SLEqUdz?q#2-fNfe{ zt6){n-ct)}4ZL&dg)Le&HNr+C`%W_~P288Y!X_x-F7OgtwV55`$ zI0}oO`SLe0gf)YAGS{$0tKK`<=x4vWhb6>)vkjZH>^{T#0K4`I+q5jb z!D^5_=L6P;cxUVjwrKVI0~^Ea-8ug^?vHSPmJ6G-G!(#kn0>PdwrMFXfz>Gcd?~Dr z@y<^fY|(1i0~_P)kCm`YaNk)2o3vPUus+FN+W_0NtT(}GioLi6)~0zU(gs_!`e=s@ z#XjJNWrq8^0Bq9Y>w)zM`(Yn!(^3_J)hzq!Ags;tPQeIl(Q0ZGHs;yi#$kza-!}!D zv|KA#Utrf~VVjn{c~~v7moLKF67R%futh7!Ds05q$Jbz4=Kf_JHfafN!uksP`8I6R z(zpk!Rraj|Sc~(HI|*B~idwL-#{PQ>i^=`S1#Hq{U%~o1yZ;upX*o&3D#2d&2x}X> zlX!wHS~=6OvB^H0fn|&P&n#@xGWZGW+w7@t*rugj{vX`mVLx)f+Ai-@=ff7Q;)SrW z$6n}!WuNX5yq8rF_@XTKJ=8`jh8c`vX{%j7GpUfEyYU@gNty&teetIIFgcw=w=g(b`VPR{>} z`?P3zu>Q`zSOD9!1K!!58ytA{q&$YJm7fW`60>evsPwA6OPdM>-!1KYF|8?ee_ zp9#TQKJR=C!WOMUBd}4xem4qBA@^+)uu03|6s#AqSI)pTEvvJzDrV1*!kUwJrWRp~ zR+$)Vl(6@$!s6ooY7I7N(G##<%D%e^+q9JLz{<_Ovg*{ADa(Yb$XgiTru9oAdeuUlc8mX>x{wX*MZ!de^ecml9R zt5^^=eC)Y>u(WeO-Vd9!qz7QVgFQG5+q7JSVb#grI1X!m-r1gnEn2x1Y;>_NL|_SU z|2qepw1gL6y_?-$f^Ay5R$vumKZ(Oy5AS%_VT)FY4cO>qFWH8rkNeqO*reraAJz@_ z!6VqF<^C8}{p{_huomK-W4TFa?{ms0UadV$a&JO-ug^tcKZd zUSVy7cUrTsMXUV}*a)*%e8V!z{qiqt(vs)+7q~yhK9L98w7eF;YMi~N7}h3u=duL0 zXw~F~jY;+$4VEeH%PL@#mc=SqpJta@*rsLF3oFI`)BtNUyc1}KEn1znz($0<-UrJp z_ZuCsNsFrs*5}ygx?!7^?;cpqvkw`t7Ui9X5Ny$^V+b}D*pEkGS>(QE3^r*oCt!Vv zy=WS?X_=XURgC?87S@(|ClrM(THP+f#tM7eGAygyAFRSAEtMv$$JtjBuuV(;7Od9T zCwE}Y2#fwg1aiB!TCtv;$@!(tz(gXM(#yL#B9#n%Mu zr|gG1Y|~QJ2CFmn)pl4r=bZvSY|&~e02>$VZ$Vftx$oxBk1U6|2&ck|&{d@toX=#kX>YjaT1=b#T$Grwy zw2H37#v}Xh1}rxBBipb^i@gi$Pwf5!*rw&=2v*PRbr!6>@J`|swrJ(NfQ>Z!>=i7p z+<)G{CM|<0SkJJh9$=f6_9s}qu^+v_T9$XJGq6RgcosI^*$Y2m`QU!~8#Zal{=)hv zyW#j(xKGPX9<09DTMA+An|Jn$VT)EC7i|2nFS}v+6B?!XqUTK8eY!+vlGO9l58$FNDu z$_cDjvge({HZ7Bvu&QEzy@s`F-s!!AEm~dP!$u8zvkglv_dCz9NsIOh>vimlZ?H{E z&Ihc#?4w_>R?j=nKd?osK+gY%`wi@8xv(^HUta*5v}_i^dK0^=1h#3JFNIYz`*#_v z>AW-Sfh}4+RKi9JduI(St=u2i!6q%W4Y1zEZZ^R-EyXRc^0Cjf!CE`-e6+(BtwMg- z=wQDKz|zTmTMul~a@Ys!e)h@`Y}2wj2&*pk{1I3S@XpjIY|$z+4jbLpkqd^RP`z`68@(*_UFl*2g=JRoJ4{*cxmY>@VxE^mE_637fQ>Z^L?syCZv?hzm6(H#ID5$gENk4)F2N=(U(2v=vJb{#o0fYMR_pBT8?ct( zo#g+|VO_OV7lYAyzWP_{?QZ+$>fL*{FYYjLhu{vu9fEwgOxz*3LvV*?jI-y?n8`X_ zYrZYpBz8za$-h7)Ko;Q8^4tDv@*|%@YHYI<)^&ZbR zBK}wZ{yx7?i}oF|Wh&OUI^ek^_;zgRP4aCl&mU5J2Zwy8KJabX<|xCrKH_;{wr@G+ z_ba)+L$)Lr_*N%8A20Il*z!~2+nn;ev&?sJ#&_;2d|S4uuJ*0Zd45pqTQ2y0ZiDZT zEn$;yb;c5I26@NMpSJ~-t&aD3-=#IozU7hMmoEDb*|NFnTRrhS*zoPx zGQH*7Jo7wa*LU#3cl!2yTef+$zV$258;*U;8^1q3^&PUM=)$*p=lSZDZ^xFDTi@n` z=l||~2cLZB_rbSio33Zy`itlHufFA*-`9Nj4%uSAe5)Ux=l%M2Y*~{3&AS<}=Q>mj4p}yV?_dO9a|nC;JZBQkmvkMe=+%-M3>)c9w4w#q;?b-$69r ziOu(I*=DHFw~pcYTd{A6<@YVhcgU8Da^EVB=c>xLW6M^JZxhe+^m^Yx0^gZw^ljND zvc?d+>GAE@Qrz#`r0{%g(07o^cT$aS%Qh3EzI7VU z|Hgew!0)>!eTQsu)4o+Y&uiy=JGLAy_%<0l&tLK#Wb&Qm72lR^64!m}ES`^T`j%{d z|F!KqWJ|}MZd?)AJw`H4!OW!)5=W#c_rGVcL-}w&N z^6q@ALZ0g<-;OPpFTPC?&&%F@2gQ76`_s2&n~Wddx`gMmf4-%Z-$zCM@BV$Z3`F}@ zis#RQn|Jq5lU zTONvhn`)ldmHG~9_|8$8Z_73XmA-W?&sVB_OC7&Ys`DMPWxT<+s^|H4lW)hCPVL(? z@cg#TchJapsylsKw%PCYt($nB+vi)F`TgR6?~pC=!@gAu&&`N$$Ci&V-$wJib;5Vh z%6G1&d|S3DpY^TVc)m05TiW@3X6QR)%iOYW)xq=VHQ$acgB!k0C(mEDdocUJ0JfFJo?bs4=?c4P6y!Y03(9d@s?|oagsekmX z2Y7z`>{|x;ec_w$kS(hpzSR)VlfQjCw*33`ZH9UNBmakg-|(HTDBqTC?qYoF5uVq? z`<79De~{=qWJ_L(Z#BmAFwM7POG1WkGtTpoEZ@ODzVn&m+pZG*;T&P6wk9Nd^@(xSNS&6JddgM9nA2Zp?cqzZC)FF>sg++wD^`eet+KTJ7kOM z@U7-~zS-s5u_e9Nw^`u%bieOlk?%wf`L=A+XMF3B=TD=)Wr^Q6{PP{Mf5r-=(%sb&GWBI-?GE++i!e_Y`MAft#)}{`QY2JW$($i+2eWk ztM6c+?<~Cgwrmsor0+bR{EAJe*ahPJ7h~woo{u`bJyV8v8Arrx4Gf@q4piz@|}Wq-`(Z%Y?CwXTR-u9an`py z^ZU33-yvH}=v%$;{C(NCV@vCrZ}ZCY%MIVb8{aA4_HEf_d)K#q=XvIVZ~5T&vxmMz zwnU%!R-Zf{IP>k;@^azZeDS>L+IR5Hcg}8oTec~2zV#2!Hy(Ydm+M z{U*AClTIwVIFaJK<97XxokvuPq^(|5SekI;_$d=?J-zu8t z<0-x!TYl1fn;4#VX7~-yvJF+I*{2p3imoc5I31_HEL5 zKG^F!2>8xxzi-Pn%|pI*I?vCIZ^_{IrDMKBwru|MtulEYO!;orm zJAI43E!#XU`PMl+Z&>v$x%~ck-FL{AqAlMlkLRm9z8zaq_I;atp8vDHg95(ud*s`) zP1mV!UC8tMbKg?L?`y7nhitJozEv^L^X`2+wk$pPHYGeyc=jEX@}1FF-A<9r8|d?z;1w`H55WZ$}q=WnUL zrJCQjr27upa*^p<)$m;9_;zgB%JXe%d7fVAJE-G3GsV6w+e9kgx}N9#<-VnX-#=CQ z4%yOJ<6AZI{G`seV@q+PZ_~u{wPxQzGv7&V^=;W^qTRP{;rU;uZ_)g|yT^CP7T4!n zweq}n(6?jD;jnMh#`FA9-$6UySswRo*(Pz)x9;Hi*tBoy9BXc>et6+p(qT)3+Jn`RTXs zV3hBa{Q0(QvmWvP`1i+no)+y}#`*nZtnZL5lHgnY<9Sb#Z^xF06yIio=XHVaV3O|~ zW%#yiQ;_XjPw{*u*SAdb`=kQjAzQ|ae5)Crf0y`nZ0RiXZDx6XTj4vH<2%*WzAfAA z*ZS7;JkM?LEerg9vB`JHmU!)3E%My7`F3pi=HFEhS_b-vR)@7uD?*`jZ~!Sm8(-?GW? zH&%UzYza1et1X^SZTWU=iP-gRwt3#W?>pGxJCD}4Wt;kA-+Gtl$EUt!kKY$w_zu~! zdgWW~^E~<1w`0q{d*9}O=RXg=1Iu^1o_$-kxqJ1k4|!hm;aiUQ{lS;-kS%$?zSS|$ zL-~*XeYPY-`8Fp!ABpiDobsK|INz3S+7f;1GoD{3`<8QlUy4Z*{@*tSsM- zE%Q0P%_Yxc@_h$ad}pZ8w`H5xV&D3j=Pk;&-0=JJa^E3aRF!Xa%k#||-;OQm^}fv= z&!-!G2lsp@vcG_;zf`U-E6C z+l+4d)}K88+V(A9{Jwq9cgU8T1K;YK=aomk9b5KJe48JhXP^5He)-PArEkkNu{XZ; zAJ2#He2e_o_&@gMe1~k&Prg+I&o5qlJGPX)`!|&M4=_)$G2shnLpn;n&(lG z|H;1}!|(f}eTQs$j`gi#dES`d+p*;|$+wB)d2y=mAfE562fi)aq-Ofo2|S<7_AQD0 z{x8>e$d;Z0-ztgcuE@7zOI@jNlg#tOGT%W8-zli{ZP{kI+P6;Sc~YHkN#plp4ZcIR zd^h=40naC7<8VF8dDI620bI74Upu!?$D0 z%a(6b$n&OM-$4=IIotPb*{0;sw=U-S#<6cH;rD50zC*T5UHDd|JeO>3%XO!=t zn(y4j__l0Q9q(J$@cbarx770c+!Ws-Tf#Kos*dOJ8NMA`MzVaHdY*se_zoKQPFucj z%QjbqzI7wdD@uGz6Tjb4zC*TTRrprTJfExb?bs4i>)W*Oe6Zekp!v>gqi@SL%`Lul zE6>kceM=j^FYWLhvSqW&w`%8k(Cgc=WxC(D>EL<9knf?-xc4MZMxQd>wcc!Z~B%2eqXcW zJ7kO9^Q{JXo@aeKwk#d_HbXp5IQ1P2^PSOi-+g)g;eV zoNvdLtpwj@is$LczJqDLGn4AuvQ1>VZ#~2F{!HI8%kQ7EeTQsm%=4}0cz#mg+p(p% z*tePI`C6&(V1e(XmixABGg0YVFY^4a+P8%KzPrwM$QIY&TP^Xtw%NC1%c1sdmU*7v z?mJlFJIkHEE!!ma_|~gDAM5iiYyAFez<0=&j$z+wo#!_rz8zbt#(kR&p6^Zg4mSBt z&a`jKHVd=9^%l?L7JSP#zaI{LhirLY_N{h!uGf4!wp?!bHoH78+x8vo@ty5m-cf@%fOj$WqJO5;oGsL>DsqBtmj$ zz4(?Ben0uxa*z)k}+nn*dF5>_5@1OIXqbT2&Z3<$2>kFQ*#QT;@ zexH=&J7mjvif?tr^Y1j@jxC)TzRfkyZ?k*{H+-i$*SBSx{e0j0mgl)ezU7YJFP8WY z*%DvoTix^ARQPsm`Ka=39M4;8eFqPG=c?YfWt;LQ-};f~J1xHDiQi|o`3~7K*Wp_| z^E|rSw`0p-uW$3h^Ot_#!7JZs9`bG3=FIrkZ#*v@^DXcEe&e6-kS)QKZ}q|RsTtpn zEfMp+%_q-$7kvj`eCKh=w`H69Rp0uX=f~^5<%i!FZut(`vby72{qj6{-?wASKkM84 z@%-n=cOd@_{$F*S`nGIyckWw9@Vw^Aw?v}N!Hw^bEqV98RTR&|2j7k@3D3SwG|xv~ zeFrgo=kwjSWt+Ay-#V7(*FU}`j^9^^{1^TQ2-%GUR6Lig7{C!S9|xEOE-{HfkjTe| zk^xJY*Hl0!acM~hB$;>4Gl7s0l>?|0E}MCPBP6{LFsWRoi-91GuSF`r5~i;l&;ggH zNCtKIm`fpCcajX4Oqe~=K{K!OHu(KExa>U z1cZcqmjFt0=_msnA-5HPY2{K?4Fqj`ZNC<nTR@F-8NUY{AwLg*`NyU683-o$+TAN)2~+(6=t(XIUw};U zPVO%d5)w+JhnnUR9|br^Kw9zd8f1r2npG&0n`eYpdN69Og92% zl}khm5UlaFzE;2z=CK{n>s%VT0NLQ3;~pR+q^KWIn_N~00Y^xR0n8Scf1^OK&DVa% z0ZW*!NkH##xt|7Pmv?IBfRGTo0H{4Kc}svJWN8I3`&<&%f#86zjcx*#FrV9iwp`lx z06FBH>jNMpr1A(*M_hJK07ppnIbe>t%wGb*315r70W4vL?f`wt<;?+d#yc%fKuE~N z3!u)qsCU2-vh@j=3ohwDKyb;|X8r(6n8?T|4}Ha@KN^s0-g$}zLP8o70CmIVBnfbY z6sH2_mdjcI1b2KbH50IenaBq8J(s^+KpgLM7XTq4t_V;MTxv@JN629rU>>>TR|3Hk zUt6vQEMXGs0R7BmtO1Z0-uY?*LP9z;pkBG$v;mHgs!qVXaoOtzf_J`_(+60>EDQkp zgG<~nAfLQ5JOYG-ypI9ui;JEB93huefcfT9HVXtld~JIku!PA70sYHmb{UXA-icZR zLP7>M044t|{y9Ex0gjNSUBENI*&s0ZW+mV?al7Njn21ns+8IfRGTm22>1} zo?E~X@^BBBST1#sKoG~*j-CNan1VMz$8%Zv03?BTlD>hEknvwYC35+eXb(q7XB1$P zxZK77K{8*fjt4AZ_7ee}!X-BakW}7TOanqf;xho1#>Hd-j*yQWzyw@c^MN3puU!=a zmN4ZdfX?8uqX5a|oy-a#BxJ4%P+44}YXL{dU_D^6xx6$2K@MMQZUHP|&RPMT%cZmf zkUZYm=mJ7Qf?h!7bD8P~93c@yfGOb8Yk;7TuRV?emN51I0A0l8coL9e-YJ{`LPA#O z09C>zc@c1g{96J{DVLuWAW(d*YaOtJx!VME8JC(JK+1XNU=Iih$+Li};1V7Ij*x^? zz*KS>IR}C&zV>+uSi-d30J@sX^&KEJyi@T2goNxq0jicu)+^u$nSTdN9haCdAgJeS zLqC8e%>iAt5RbP)%Gm697j@dNN>|xlE@5K?`4tOb0As`Z58n zxjbb9(#ksxc|b_WNdchRxD*uwj*zudz_fEoDF=cMzBW+_Si<~P1G*oBzPA1dSi+>f z0D79s1W{%5Y6cEhwwSri{5@tCb&*ceu*}!?^8rhkoFYK4a9Jz?WR-W~%7Bm%Qvs+o zF7H)w#c{ua!3emN45bfZpVi*#^iK@62`pAtBM-fZFCV&wLN@aN^~5D81RNpL z#ejL{5}|W53-DBuWL8VAfTmxM_m_~UD%(|{$+=PaP* zzr#Ps_60y90CODzAt9A3fQsa@y9PKyvNr(}#btgQ2%`B~>>gkVGjssx7%p#zfW-1n z%Lxz?a&ZQzI4jCnT*it3$>yD}5+Ed`qYO|vTy81=M@UsQU~;+a)dE2tU(0C# zEMXR!0G-bzP6JZFJHu^2NXUB!pbEL@Zom<8*$bE=E@cBiP|Vl1hX6~Mj1fSWaG4zg zq?C7}CV-HTfhj;KF3&T7Bcy2_FlAg$7lELhuaztVmN4t9fUe+@wgE^b?@Vq1AtAC0 zs46Z!`+y_l!2+h5OWiRL)bO>VQ@|3Y-~!OKTvo0CspFlbTOcH4{2oyCT)rOwM@Z*0 zU>dmGz5+obU#tEAEMfM)0NunT_ZN_6-dU6ckC2e~C_uGvF)@H6X+V!~shI<0ly?pmfRK>9B|wdF30D9| zNWwZ`#<`4a0>MAN_PGsM!nExHdV$MX7)zWGw*93YU~jAXw#V6WM?z%x^BB*SK^S0J6?I z_eDTRNNp*gHn`X_z!8#P37AbTOVvQI#n%$+085zB20(9f`Dy}Whj-dF5E63J2B=*w zm7RbiWUm`Adt9>nfMB1mEersbFtNjcKHxGu0*K|Ew=p0jL{9+fkjups;0P(31pf#8C#tseuH zFsWyNzT`4_0mv2a{9OSdAw9Q%y5{2U0Y^yPBVca096kfVEnh2m11w>dKLCBlCFvWG zd)^uQ1wul;CCNiME*(*TBjh#)Fb`a+;(_3iuk9xSmM}RffPUh#m1L}{93;~XiUIQ5U@A2RAVH5}= z0IB~6Si&4l0y>gQ;S3;Ayt6U~goGq70xFuz_!8g<`B?!>442MzAc*B_cbkAEO!W?+ zM=H3BENX!>t(zp!%071alUjG0~nC8f251r2CJQ|P;-YJa(LP9nZ0F}ul zNCq4s)2V>T;u4V#1lfG8FB7nYdCUfM4wr^JKyrEKxBv(VDJlk39+%Zpz!8#C4w!r{ z|0;o?fUo^l1C}sdb$~A9a^C<*5%1JA10f++1FD!yUOV6jS?UB#373Q(ASmT)qkVuS z%;x~06_@s5K+1UMdISgwsT>DXIhWlDz!8!?4VVfp^Rqxu$=6~R085ym5YSaz-j)HW z=AD)`ASC2s15h$&uw0n)%bPZvN)NaHo2 z8o8X@0*;Vk2bd- zm^Lo?u|Uwy*Oub}OPItYKzDE%O97;lcfQhqkdTfHKy`7s$pRc9Rk?ua=CYR$1U-B$ zrwFiwSttQ?FPFG7K>B!RxB>_Xd9MOgKNnpKI6^M#0W-j*tO*DP`Pz01UFL!G6q<}tp5Y_7?-puK*o7zas~(q zk$FJ<MOt!X8#7z^IUT80a@Uk#RniHB>ovti(Jer;0XD62TaJN^$Q4= z_}bMEU-MR3KRAYt89^CCphS zpf|Xb<^ZzEI~#dGNJvl!s4Xs2#egFuLIGx*OK&+4?C`b6O287Pz6Q{{T#oAi+2ftU zMj#|)wHZ+RT#{P>N65c+z#MS-=>!7H*SdNDOPISpKp%3c83g2rcMgVukdVAlKpk@l z#{ox3!X#i$xQt8#!6{$+oCPdl+7j3c<1~K2nkV_fV$RPJ#PCl46c7^fJOij$E{*emBjj`uFmYUpmw_Oj zudS~FmN2OsfKK2txdlif@BHlmAt62cfJ)-xEZ_*KI|fWLm%~#aNa1S*7l0+q@)e*{ zxg^~JlEyn@_drO<_XD5;E*;N+BjolKFzH;XK7b&DukC*UmM}TLfX?KyD1nD$@lIS6 z5E5cy0F}+x6`+f`Y}5c!!aHg8KuE|`BcMvTNDJTy>1_p!;_}cA z1Z8}!z6-E~IqCs)IhVqIKq`1=We^AnNj8A0S=P!V?@lNSG5E8Qa38;21 z!4KdFnf?Pz2bYM*bdR8uuk}R(mN1X8fbQbbkN`+G?;IxqAt6Pnfa>A08UT)vluW?% za`~4H1buw%Hy5yk=_&wpKbQL=Kn8fHrW6PXv1Nc7~=0JJ)SMNJwQTphmgub_0%(>^{JZahV?gf^ohUI}BLD42=N#AD6c= zKqh#nWdaBZxtIdfBo{RcI6}7O0W-xVJp_VjzBaQASi(fE0eXf@{{|qlyz{gLgoHHi z0&0%S$v)r+DLw?uJeRd&AXwmQsb_#C%)|wt7rFdh0TS{~_bm_-;_d;p#HIETaD*H_ z17?{^{u>ai@U`U+z!E0$8_=s<#(n`=Xb{50URL@qkuW%QuhxC&iUHWBwz_sFazidE-Q0@T=GuRA`lWXz67W%F5fGF zBcyX3FxOmeH-X@WuT}2=mN5H!fWGCDYXQ0Aoy8*{BqaV6Q1@KSIp7HSxCD&j(s}~~ z4}9(F4zPqNe*p9&mz^g-o_Hto6$lBLdk54rm*_9R5ikPkPwc7#EMe{%03E@lrWue(z#M2GBqXmL zP*GgMPQVe8&;yugE+c(F5X09#2LMZ$wqZcWa=9J>B#w6~#(|KK-3dU&bIF-9%^qm1Z;PP|OXEMa~>0G-XH z`x}rP-nst;LPBaIvOH8S7aIjQLh@q)lgDK#9tiUJT4EAl2{W1k=mIWZX@C^+PJ0Fr z5^|FTs3I;tNXTf}HJ&S;9;Nq45M@Zc&U>dm`t^+|6Un|%GEMb;+ z0Nu%}R(8kyHZvab}oO?jGb6I=>E#1RNnF(}0=b@-YhpvwW>>0kDL*3IRRGrD6q; zdEVJs142TwHUYK3Wo{dAgv9ItW|7O_0T6_I?e!3_glRqj^b(izGeDMkr}PpC3E8{` z)C!m24se7_JHV`RiFg8nHNMvO0$9R4z5#ljOT#B18@zM;4TOXg{Q+u|%W6c9ha)5< z8ZcX2{>1{pHedUV2P|Q_k^sHKk1$wq_P@NM_hJm0Y^x717ME1%r^nS315rVfF;aO8=z0Q zymbI_#yc(DKuE|%FQCr3r~$wcvNZ&l3ohv+Kyb;|X2t+Zn8*o0UvcT50_2)^o@RiM zkj8mH-EcWs1RNp7%YeD%vbG8YcYG~%1F(dd*aGxDm%klA9Pf1R10f;K0_uTF?J?j8 zIXnf-BbWROAb8?y%U6IUOyVt|pSg_P1M0Y^xa0!###({dn)1f--2u!LE!0dy3Xw0c0I zd1tZ_2nmrEK*ezBX$2f15AA@7Hx5WDL(>q2A7=^Kr(qJ^Bf2XnY#p37MJK7z!5Td2bgRwFAfND_*(N5 zUWrSE{`@y^C45E2sn04kr$)F0poiHOYeFa=zCqk*81uRX>BmN4}RfG*;4 zoCHWQ?-Zs2At9>)ph~zTX9A9pf7yU3X#j#MzV_J!Si-bvKv#3QZUdx-cPcu8kdWPOK-F@|>H{1h z^8B;sUUQ`Mm;k zCztM9K)QJ6{vHSkseJ@gHy8U1I70H@0Mo-|=>rIQ`C8&PUfNJx(X)GQY_3OGXQ{sCr=%i$yt%=5K^8Nd=| zc@EGET#^<6S>&CuB_Jf^dj(J-myUJ75puf;m?bV%J3z3^*Y@`SOPCxB=oK!DM}VyI zPTVOF5@OB)wZ`TB5^#jH-T-Eu%jF#qZ1A=62fz|$`w7sSTryt)+2WnqcOWDr`U_Co zTn2ssN65<`V0O4PMHYAjyL{~|8nA>Zi39W=myHBK_IW2Q83+lPN(Iyb7fA;kA-$P^ zv0NUqf#8s@)#m}0Fh>P|KH^eX49GF>tds&FA<5-{I^i;22{=N2ssVG#rLztQ&iLA0 z17HbL-3;h+E(aQr3*O0X2SP%^PC#99iSGd%AtQZ&x#IFM00h^3t!)^vgt;04^bMDa zaX@Z)XJ-Nk3CWrU)E$?(S-=qzvjCWTE`uQuIKK9}3|PW6uL1gj%lQT%kGxa54TOYj z?gHwGOK<=n&<_0DDW5paa0 zya48d%fB}u_~dK9AAluH*EgWQxZM8&^36Ln5rrNhAvOw7KV0%+0Y}JEJYasgBqRaB zA72|y0W4uY(*Q016aGQAX8;lbnCmPcB&0GIP?21A^8rUlb`fBrxXhOTK{Q{BEdwlJ zhAIFZ!{x0CkXYVnsRcqpF6sdl$3-;(j*zVuz{GP&Zv%n^zBbbVSi(eh13HmQe=i_O zyz|r#goHE>0VJ(rJGcg0`G%kO0fCRkLy$FPaxFtZP zbE#bg93hA6fXU#JzXb%Dd~JCLu!Kq62Xq#fF$+jG?|dBrAt4>7fXd-=a}GE{s;&T& z%VqBd2=e$^&OKlWv+w}ud@ga%fE4h~@GB4!^8OB}LN59XaD-g`0H%mbnG|^h#e8i$ z60n5Hhyip7m)ST#N_i(L5eNwxNCuSR@|+4dLYmS6Q^w^q69~%rT1gII3A3ID=n5`r zg@9D@&SWtV5+VvvRa|4R4; zF2~=14DwFl9}p6<8d2<_hPWh01CEe?v49!o@)HjPhOc!c0hTa#DS#f~QWF3&$~y-c zKuAblHlW71gt>qtB%uH><6K6HfZ!ir`z!%0VcN<7J;CL=0+31Gsi+1*LUwBbHN_>X z0dR!OHvwjvON<7B8NN2u23W$pb^v;oOG`H(bG&oj3xtHI0YJ@j*&G5KA?YK4S>Q4~ z1_XLQWO|walex8E}NGtpa9+OUecitn#&qEx;1y zcL&gGT)OuGS?8U53xtH!9s_EFi#-J#A^8`8+2pcx1q54sE%6qxgc-dD^fs5T2S9ds zr~MfS3AuR%)Gn9G55N(!_XU_eF4@08u+P^Pq{PD#CN>Jt2V90@0I|ID76*ic=tMvr za=Azb93f?CfH~r_l@0{Qd@Umju!Nb(0rUx%sC+<9d8fY+2nl&E2GkjsMg=%RPRjvv z&ZW2t2rl^AdJSL+lUfhxOD>a*fL!s;Uo#LA($fm4Yc8%GaD>!#0p^CwVGj`8^0k6~ zz!GM85YTsAk_;gCyfZcmgoJ#L1Ilsfm;@Xlx6^=m;8HaQ1dn`ee*v(B$yoyQ6PLvm zK%RLgZXE~-F`IyT;qtx>I6_+Y0Q1V_@&E|l_*(f9UzZOWaR3;nGz$Ly8aDlk1O(>wv_bS~#pfMoDa z=`0WuvN;c^OfEqPI6|hE0h7fgVhsqg`C8uwU!2&giF7B6<`SySqtcTF8%d@ zH1N(-BM=hO*aE0VE+?&kBc!+kFil+6x`3dWuch_^mM|0jfNtUPHwcL4oo)k!gt$>a zwQ{Nb2RK3wCjryOC4UA8+WFe@9AF8PxCrPDE@Ml8bn?#E3J?;~u@0y%E;pNiBcy5v zFx_1C_JE*=ujN?45@z8D(7jyZP66rTo#AsJB;@@PQ2ku=4d4j5yaUVtm$C;S802f) zPk<#%#w(zQxXiu-GR!+sUqDF6zz?7dm*+pg5z-V{=3z#-oJIq|C|@gy11w?I697HN zB`q0{ao(9s1wuk39Z>(c^kf2#kcVu*OmL~o1Am)v$h7I=E)WtD8~|#I%hVy@2#GiW%r=+aGa%UEYmXOzB~1M_ zpm(_(-vYA7JB1Di30Zvv)IOKw7r+tn?+q{qTz)=)!1A@OZ@?1f?ibL9Txuf9J>-aY z4x)gNki1wx9dile0Y^we5@1fajHCd;DPQ|c11w?MG5~$XXyr93*ZPzZv)I7 zm+1~5xaVt;-GC)bUoW5?m#2O}9(bo=2nY!|F@SpHQZxoQLe~BP=7~$n6c9Y~wTT(P z66SXf&@WuN7Xf+Yo%(|L_dxK=*A^ZCOPJVaK>u+Weg#DS7yP$;dj~>7 z^cSEaxLo`Ij*v2`@Gy~FwjzNb3XqH#z!GLA4$#qDq7ng#;hp|uASC2D6;QEU8q)zs z$Y~~E;AAOHwN!X}mMm4uph!cLFNl($NDrLT>v2lg_1T5C}5(+Ws(L36nDl z=u9q)NKo@h_xCW$zchc^FkdP?{s8TNS z1UN!^UjU=HJiGxx8DFda1T0~Wz5!j%rSK1s3f@_XsPqU4Nsb0oC71D7z!CBj511-0 zok>7Y&DZWy085za0MIpD4l)3#<(=GYAS5Kr1ymiE_yWKYGExMXdM+O&K+wR~+R6Y+ zn5zmvH*%?{2Be91c4~o;kgNtkHFKG30vsVR8Za$f2HSu@^R?Fwz!Ijp8_=y>&U*oA zA5T1sowM8-VHM@^1?W`uN)K4qyq>wGZfiF83CY0p6)O20}vYDWC?qMJ|8MfP}o$-3o++ zxOPA-ajESB93h82fLZ2}-wy;Ud~JCUu!Ko8fL`S?HVViZ?|h8|At4=;fLiBrGYvRG zs^$Q*!DVj&2sZgz&Jth=v#!~*7wOI-pGob$D#B)}4;AQjLTTvh@=E_o*@ z69@?z&j!>Lm+xG_5z<)zm}@S#ML=-F*Q!eaOPKvKK;Lr7tpwzbcNVLGkdXK~K;3gO z4S*x$qX{sMORENg2flXI23W$BcLMs6%T6~SPrQ@a2ZV&o4FKwyOY|_{2pJp!%nO&7 zF(7#T|2eYHZf3$T-1{Ht+0X9v%j{2b>Ma-%Jz{hsVnnnEE)hfoBZ!DLqPvyRdu+$? z^0)r$IN#2-*6^Bt5wHk#9s`UwmbMi@-nrRX1x!LT6VN|cVoAUzBySr~pDdF*fcC{} zkGp_HDDOUCe6yS!0`kL6%Q0XQvULLJITpt`U=y-%38-I|pKCyq{}1oPv0K0*)ZIN` zZ8&HKTb2&gO;2TTkZx|KegKn@=U+hYVe#en z{lO;WtPoJWEUhJg*2in9GQc8Kl>;#PS(d5*@pAL40VW|6wSYdrV%GsSA^nYj^06E@ z1KJ?3HMas5p^|NY;b*Dn0Az@p`7XdDZa<&}c^gg9ORJ<1Y&18hQmJ^;1AGWrc@ zi@bK111v)IGD~R%U=uP|38*;BM>U|W@LIqLScJNE z0Y-wQvjLD*ZuXi0lMt61(AQWJ9>6A~s2x!2EYqEUX7bu=H((LU-v<~QEEirtlH9Zn z0wy8pAwb_`(E@-?$Z`-+TP%6wfVRzRp-I3Z)Z-Ljq*w-K0NLT@L;)rt?s-5@vurH@ zHX)TUK<%;dYKrR)&U zj(9C%0T!V?vw&f-jGO^-%+0?Gz$B#W8ql*W`!|41NZmc4PFPkS0PU35ik|_CP&2Q9 zamMoY4#+t-LtlVN$mI{9U$E%n{ew-&Za$zcS!#*^?TXjpC4fb!{Bpp!W|?vT@{gOR zD!?SfR|DubET^@AO-O4!pl(^V8v*T(*Q#0oi%^TLfN{_A+Xjfu&3Ff35@L4&`U6W} zFJKdL+z+TnmL?ydJ@H!74_JgM9|nwPmiZAtUby)h1x!MM6M+88auWhN`LR zxmmISlMs0V^dgpt7r-Xu;SEs5Ed8H=R>Eu9Z@?l{^DkhOvTWuJ{6WgNsVD?YLZZch zUe5Ae3fP2gWLsC(B+RAhq1o4ge-0i9tYju@oABO~`ZrP<1RXK|rhLHUBtZ z5$b#rFdA6e!hke#voiyjglKbs-oz4{2W&#}76H}FG8qH37G8Ud0~VpYtAOEVIavp! zm7A6%U=p&m1?V0Y#|~f=3XBb!P!aJ4^2gARXKsodG5xjhBGl z$zomuHX&uVfa+qIy9cyxUi*9iEJ6jI0i%cI-zy-!+;n~bCL#M@fZoUA$^kYZtK$2E z>Srk`05mVJ%@hF^pV}NC-91tHj7Y@KAM6U+)L6&q4U=vc~0+gR+xgOAlcrCvP zum}}u0StrXsTGi6ZU%I~B;>RM&;u;)ZonpFyBAO+ER|kB3-a2c53mT8^8>~x%lI%L zW8B=2045=QV}L%+Vod-xAx%?&nqb+O2DC|DD_4L;s7M4bLM&fVK&H4ESprN#Zk7Q( z%+i$rY(fs!05#20w*hD~ytcLpScEE00mdxLY#I>7&07XA2{8@;eU9bw2(Srh&jKpK zvU>_>^StJ~04zequK**;Qg8#v0yk54fJw-+4d{z3z9+yYY9elaPr*Kwn|8ivgRE{xU!%SdJ?IZI#!Ws{o5oNewX8SSo4( zS?6ZH4loJ%ZUA(XCD;tugxtCTwZYQe253oMJ8TCmLe+Nx#wN>p4bhgt{&Qj2o8DQb2CG*((Q3LR^)A ze#eri25dr#oPfG#nRWr1&1O6JR{BT(ki4$W5CEFbPTPfd0gybpkda%iVx_ zX36UVv=?3rc>#-1k3PV7Wf>R(0N8}oS%4~FS@K{wO9oh)hxdnAR0I0wSY;8T?gnjEPaiDO~`RGpqwmCt$U`fmSHEL4fEQy3$O^)(Eu0$ zmc1rGM!2bU112E}51@ zGYeRR3Y-DPBFn!EK$f`ayar4{_HO_^#^SmMY(iEa0JY3g^bBZmUYmIZEJD4$1I7x= z&=(*HZZ3WRlMr2o|Ddn3r1JrrkeVVut+6bZ0NOgQ<(C5%p+XM8Fj=0e0NLPXpaw7r zIjsfsB#XNqunF021k@%=WecEf@!Dc5U=b?U1{m8c;~jvcxVi5FOhWp40ey$X>IZB> zntXstvuyYQZI{=|hXIRFkrBYiuzZaIvd7KH1Yi#U0~n_) z1qXnfaWi!Un1nnZ1Nu3O?-Z~JIXef`1xxD{pk4A>>K|Ycs_G6fu2`0AK(4v@eFRKG zCSCyjAB+74*o5?d0_uk4_#4n}d9C>uun3jR3;e;jW2q7 z6LMP#s0WsA4bUC|IdlRRq3Y`Z$p4=~{Ewrap6BwYjOwJe$ounAeN2b7B?uL;oVcrDZdScH0P z1&n%@0UeMAZcaJ?lMr_|pf|E?^#V2_m0mzKu`Kuit(n(ye!wEs*f3zUu-uOT;^wAz z3@{0?CIG#arEv1=qw{kfV6Y-Zy7KN=}G{42h08% zU=vce0jN%v)lER_;TTUjY`O@^1jc%QAHb$N)D_HeeFsdjfPH%jpYX6Vmz)s6m$PPeAkYTGbC=5o+-l zFosxu^MZd6gPZX}z$C;j2J~T;zB0fj@TLO>HG$5|XF}^evV` zCtwpY?E=&`%S%0=rFhNX1XzSRZvl)QmNpL{X>N9Oz$8TL1oT~&ST|r3lGg{Q49lb! z(Dr!k(Fa(B@(uyUKFi54AP3yE1Obzftua79WN}OaHX#dBfI4FNnFchA*TxiJ5$Y}i z7{@HV3xH&~Ia&ftLK@?Me!^lV0Gp7qbwHi6%xwVL8LxeA0v4eHDZn^q`IiRdf}74g zz$9e<0MIX4TozyxvYG|d6-&_>pk4FY%mrW(>h%gR{;>?*0CL04#T{T0qCWuoElc_d zunDPo1=JnO@;jj2^IHBFU=b?x0~j{T(=Q+o+zjN8{lO&Uv=GoAS==RnO~`f`pq^MN z9f0=CYl~HYMW~zx7%wd2wSc^Gb6*FTg!DB6`WuVY4A_J;wF2s$WupzyK6tIX1F#4c z=>m*TmaiT_zPK6b2TVe41_1q=rOOZ4gd7-v`eCUX0kj;it&IW}p^7H}}6 zye;3tfJulk3+Q<)mvewkNP84e`7FDOfK~vAa~ZG*6<-02LY9IxK#I7TG69p2=OmyP zv-q|Fn~<{|K$WnxW&o{}*HZg{MX0JHz$jx`ItHYio8J?_BxK?o&?{K%OTZ?i{~w?n zEXTKiR>^D4HeeAd`3M+QEEO++RC6=`2AG6=e*n7168r{iLT+<_s$uEQ8~=mmB zz#>$AF<{iPtd|1f;-;hmFbPqWfL_P)UJclU3_Af;&vNAgv<6=5XaFoiWtsq^k)_rR zNE0_J9>64|upQ8wS;C!wO~^|(pjuc4`vA?&Yv*3TB9vzkFj`r5h5+$!QylPtJ!xW&k^O|=Cun3h^fYHIyG7m^6H=7H9Nr)o`=v^$)IA9a zZrWtx4<;e$d_W&z(TV_@kmV9U1zGaS0d17mLJq(p)MFK3jIj*V05Z%s0bs;grj7tv z;pXWWFbVOU0(yew^c=7WX}to}D$DjiKwIOrsyo0U)S?X->ny*IfSBBjzW^p7_8XvY zu=ITbHX+B~fJ(A7{Q}x1uO;&)|6mcSybv(9SmuiX+2-b}6fg-1Rsed6<)#v_3F+1V zwZn4Y1hh1-)z<+Qq1GAzW0$3*8ITM&vu?m7aZ_LbCL!Sfpj#}@LBJ+ta2!y_EN7E|mgO~17_bPHngNUxmg+e` zPPti{2TVd_5zx%7oRC5wAE?G9W0J-9(Vh1n@iS7dW zHOu!NU=uQW2&jK7w-%t?@LJCaU=ix@3@~n48ZH63<7WLDFbOHW1@wCsbr0Btd^`Zk zW*L44vOmg$5y~1lvf9gZ|+fwt4|Q$Kvn;HX#c>K>f1(_yJA+Gv0|~!+=GoyAi<1W9c0OBp*;m6M#ub z;}oD5u$a?;O-PvnR3Xb;1kj3j?K28kgbFMHMls93Wk5=}=}Z77A^U59UdrOy0Bk~5 zHvv_~Qj`L;a$cKB0~VoPGk{USGIRilgPV&Zz$8S^0(vD&`V_DUsks1D70dD!pjGo) z{taLeDs%@J8q1RnNDVgwPk>3t=?kDcS={e{P002qplVqve*n$JYm2{tMW|ff)E|sG zmhnPB>bbcu224Ww$^gBA#i{^oLYk@o)yT4;0a_EUmDd6mp(1sF(aiGI07wfrBh7$G z$c-D&-7H;gfKA9jJD^%w>bd~U!)t3jfJLa{e!ysBnH>N`=jLq?FbOdXKyPQc3;;GE z?W2I|VA&l9v`$`gh5(CD@i1U?u@uY#(#_4(9AFaiJP+tSEWSm+Cgdyzs9u)V6+r9b zwbUwL5vs}rjDD7-Bp_aHezyRVkck~YA7HU}0h^HieL(qGjt>ECkk^`z0gF({6TtAZ zRGb4c#LfIAU=s3u4d@0-@D{KMxxEL}FiZC%pappC@ENcORsRMUBP{D5fCRZI`36iv zR1VNbS>7f52b+-L0zi$iTonP@IIne-0v4e%<$y84Qd%HsiyX_g%wkQr{OI{}lBST~^0vPd6b6Ef)qlwx`C0ooj| zd4~XtP}yO?h_JK-0h#A!a||#EaZCbwlqEU^*o6E{18RX~Q~}x|uiZrei%>lafU(4K zv;;_un}#@G5@IF*eVL_n9k290KDLMnxI?MD0pqae(dIea7^4|c)2Ft}AAW3f89srY&^b??OvS_b>O~~>) zpte}@z5s2T*FrylMX1MLz(}zSGdvw%s+ ze3_WqTjc?s%>02(SpXcnlc#EWam!*xZbt112H%C7?gB^!)>DLXK|%^~lm>1KJa> zB_9EcP~|Ux@ys&+2FMFHUmt);NbnobUs-N)fK5nu-pn7=8_PifpuO{2eKBAWYONG7 zK3GaB0Qux*wh}N2d9McaFBZcI*o0iU0QJq%-T-Jnyq0MKEJ8WmfRSTa@c{D6O+h*fL_k>od9e?M%MvV!E(C+ zXbxWM*#ay=9i{-IlBHo6kScE0_W<(`|M%|~{vi;k29&Y@m&Wpu1zrg?drtlf)s%M#g1&SM3UfzLALi}GqypiSn2ksgv?3tc!V)V1f^Kf|%7OY;mPrTjPRL^wknym1Yk+VYHz&1#M`vlN z2c(^4s}aZ&%FzNuJ6IN4fu2roe%gT2E|#$l;F?f(T|lCnrMDOG_i%I64`{tCjXvO! zP^KT)>SHMz2FCiinHvEby)2)jz$c*s6Ttoe%fApX_Gf{7KZ`2@oD#Ac z1r~=`ik5&rgPWOUpnRC+bp^N~WM~ao3$R?6fHA_2z6m&kEa`3FnUI<^kQ!xK&Hxi* z+~gks&EqVgBjB5mr(@u7f@R%`OpFluyV|@dzIhLkh;FVAtdFl`82upb(5Sr&EQVh7GEMKKS zj!+{NfVIGKQwc;Cx#`k?t|gWOCr}(?sjCAn30Z3Z;>#?>&44e?&8!=!T48zf05%~; zJCIDUTy_G%Rc_jQfciC--9F%*5a$4pS!an40%4Pz0t4`DuuKI2NwPc#fh?hX<3Mzi z0brLGTa=m z1C4tu%}L;sP{}P|f1jmd2bekFW_}mY4_UtVfcztt;305I$gKq|S}ffsK;JPphi5=} zmZkm@xFKZy8dy7FDY*rVQ*P8f;5=h_e*m5d8GZ&*=PXyRz{CYN9UnmRB}?WD_$H(_ z2OM6ptjOFS)HOGS1wh9?mT(bJc*F8i0-O_SupEfpvYa~r?;SUuYQS;NvQq=x5vtk+ zm^MqS9tb>eBTazok!7+4cqP~Fy+Uy0aR~CmCh`ezV z^#NV)EI)pr_=9D17`P%xQ04ZQ`Z2(z9B{qR*AxlvT=qci6It`Q- zv%F@2YeM-CfJ6z)#S!2y<)$qQXk{$vQ{a(M+6Ayx&a!+3j8$-xcLOv!SVDKeC!ro~ zV84=O;0c(i;^yQ9(5qS8??ArBvh@j^5>ojCEY`3r`~rPWZgP2%Ka|(9j1>YmgxnVc zYc7`FGQg85z%wBm8jxyWDXRr08o7zo0nJS;pAEn_AtTMeVKd7=H=tU$ z>1qQy+${U;Kw&FOT^DdpsMQ`I=3y!B2fS_E%nSeyo#ky1xFgh%0hsM9mjNKq!Hqr& zxH?&O$AMQu)r5d_7fU=0gu1!Op9S1KEK_qpj!;kYfYr<5TLdC~+?>XMu6~x*6`jOX@d1{_B;fOL^ScF94YG{y05&1^E|B!I^z8$|A#RQjfqH|b=@@t?BzXd4 zhFQwbfpCDE`Afhv!t!+uNRTCX3uFm(a}PvES-KyAo-u9?o`KSFmijl~now&WKw^TW zUt$P(}f;HN|pO1dN5bX)gsDr&%)Pz$c-cmB9WC%StscGs{hZ z6VMe)*ahUzu{_rUr-TeP0gDlqvlgIlo*Rz`D37wFbl`@N>P}#7fn}*1Fc!IyKES!e zGT{ZD33>1VsTfQD5HPXKO?DV)j-_tVNN=-T zr-4w4n~puey~DD10OSZ&YXMf8C6NUpyWA9>0bLoE=?kEEkLBeGxFp1X1H|`P&hG%< z0XJC-EuS61vK8VeD(mJgbMTn`}ZvW27noxn@&HVKd|f@K>j0(YXmqY zWOWo+d}1k@0Q#P}nF#^qFD$QN;D(T)SzzszEz*WT3WCO2++IR%g#VqA7K&XV9$Q$4; zW%>F5a)cWB2COocn;Z}+=cX%f;SXIEEC&TZv4f?q7`P;4trUn?vJ_VUzAA2JD}kzN zmbYraCd6<8NsZ;w1q5rjX>S1Poh-Xez&jyMH;}1iiF<&si<^RWz*EOE)d@&F%X2r7 zC6uoZh&HgCd4ZlrZdwO{(k7PF5O7VXssNB^W?2dX{uXY2#{kXEGBF7}63U(ewpv;G zXMiyeH^&NSY-4Gj2R;dvTmbfUmWmiK)6UI&9MC&hz7s%xCrfZ0I3?tE16b^0>D~hR zy16+_0p&d`^}E0gA?tgQEsFhaE-A{I)GO~JyZedaTae45Srj7TMM`+Sz79W9HBNF0V~Af zXaOQq+(cV}t}x3_8&EvWGTH%L5^~oC#AjG~dI8@oH%I+Im11e|0X89~A4txzlnw*I z2sd*hK>a++$0+blNMHiUL|LvwKzM)hlW0F5R~=m_{E z)Z;O*zriwa3d|(AIXMUPO&0eRkiW&U^$$2Dr1B0}+-6y@fxZ+sxksRUhh^*qxFO{J z4OmOF^nL=yE;rUU;LNZz{sPa0Y~(HdA+^U+RtQY&a}y~Bnh#h$OM!1fMk;{ALzaJ) zfI8x)O9MJAmVGBsc+65)2b>dXwE>7_S&Ewh?+G_EZoqNM^5y~V2sP9Wm}e}Poj~B6 z8@&f`U9jx-0k4Fr8359kEb&1gbj3}+0l2SOrUF2YP)|X?`p4oM2O>AzoK6B=w=At; zp!kkudj_~9q-qX`-?J>v13sIZ-vyxRfn_`f*o4?|AoYrJflE6D5 z$t@uB!cx8igkQOt-vvBxEMI$oyt4!kfh?hJEFk*9(tQH-d~$Pe29$oW)L#PEgj%}> z65lK(w}AhLo7sCn%dxya0FQ(+o`J1jmaA7_O#VAw8|@!JV;&%xFW{3<&K$6x&$1%1 zKg<+xQ&0ftg)HGBAis#^xdb>RWUw4qEM_@#0DUFgc&dT&QkGN=a6?G73s@^-S*iz& za&Dvva8|HPv;fb9JhTES2TQ*WOjL4{?EspqSem-wflxCy9SOkQ!m_sp%S#5hB*cFJ z#C4YQBf!_rOR{P91#CjJ3n1Ca61xI|UEJi|0QKE0lXt*7A&)kY>0$9c0pVV5 zPF?^{A4|(SApI;`pFoySjvpZEWm)(IdIq@p$y@$IsgGr>5V$7PT``atWa%vf{C;kZ zDgbSWrLhWlB$TNETLw#6Eig9B&0HPO7-0Eq06qy7Xa@F2SpKZkh$kRMKKr$@P z^+1+Tz9t~L$8y#J^z3uf>H$g*SW-H0O{l6)AaTgD)D8HLxcTh`G>c`z3p^6a_5oYR zEd4{kSeBdPVW9DZr8x+E5-K?c?4Pn!Oae1!+{{k_`Z>$@G?0J65>&t`A-54=@sg!` z0qDEp=5PrpzhDU9B zZI;Xd@J&dq1sp!GtYiW8$W7rH(DB3)z5oiJSzfMyb3zT?0I?UA^E<%%%8lm%aJ;eX zJOOuvs(uB`cb3>Y5cuFmz5v%J%j6I6N~nilApOPS&0qOL=$o5tA>jUDX(<76gxV|v ztQ?EO0YrYeiBF0PlnZ`hiR-%k=;dF5{-t4|vL1_6$HOSX?7OmQaaNAnIT# zngDt#xtR_DrBy7iVc?oj{#hVV&2ljZ_%&|YqJUPzl3oNJ38gIqTTYhc6=1BEo4hrk z(Zv!nflop`CV~ArmVs?xrk`b3(0_0`SON7D zEKT#kJ0ZyhAT!BQ9s|N5Zsy~FXNu)30Z5o7xDI3qb+Z9Pr&+qUfSws{4pKnrEKB_^ za80PSJs_c2N)7@495=HTphZ~Tv%n*vj5Ajb#ehoA(uw-t4PeM8Gf&E36 zl?PyEiJO9FK##G6UxEB(mgjfil#szMU@^{e_5<{-aO08HKa?j}Qu)9QA=O2|+A7OZ z31F;oBjtc|on^uSJQMOz1*A-t{u*FngPUwE(41sxt_Qve*=z(3H(4rL0JX(Uv=!*s zX8CRd3R5hj9l$xEZo7ck4ogoj;7xOL*bg{%SsHx69ii6!fSF+_9R>n>+^7-2wa@Y~ z3cM0(cmhZtuv~|L&>=S+(}4SkWp5V95vn!`&vMkffK=BF7%L;Hw zh<^=;pR$~rfbWc(woRbwoMmSlunE!9K=OhmmH~p7+~geq^;ayDN5DHFkHzSpj5Ey&mX08}$d}aA81wIKCr~vlgSpHQ4Gw_sz|W8z}!_dG!D{gbcLukbDDZE?@}-fNw&cg1})R%fL9GinuwQ1UiaY++m=wgk^gM zI44x)91tsIS)2#FW!&Tz07p5?cnr8B)O{QYaMV^vNR=uS3+%U0qH81 z@*N;l%}rz%aBD1Idq9p*BZq)h!*XK*5hpiYCqP#%%fT5?>|&|A1TG0#y9VNQEXB8g zub!LPd!VX;Lep{iU!qJw3r9`JW^^V3&AboLy)55;Ab)@*I1HQ; zaytSn`dGThfWAR)4kv(eKTG`-a6`!YG_W?rQlbFE;6_CN=P=8A6nG|NcnL@aSgw|V zi4kr(5bY9FDTAYyxVGo5B>(G0qZB1BDYTFB#yRP=g0RY?9^t2=Iou z@niwV6wA&ja7U=>3&0Gs#IAtAG&gbsxMo-;?|@fAJ=j2cmc{!7gcLW~7r;Ho(((@E z2(|eMSP>S-4-lE>Ci)9>MOl9G%s&(_u#6T0mxSCE1Mx+co-)9<#LZC!P!(for~+(4 zObtjbvy|2X!8kW_bwK?J%SQw7PDr2`$Rt><-9UJio6a`Cv&OR54#+x-s|&~yD$xT( zO_rj5pl5@d=>ecL$?`e~TocM~0EtbOivZx?;-+mB(6(99@Kv=FeBVp$FYV>{gB z%>s>Sme3sVNvOwpV1JioU=f(faB~s^^gR~$3Xs3gvb74F5>jabiw7(VNuckLo7@&q ze#A1i1KbdDzYDBcEWP`HamV&229GE!eCUOZhpRs&i1K)&< z+yaN^EdTBSb-_*7BhYclvi}SeUa{1@0q2BT{QzRuEXCh|_a8SiIlytl@+KR9xFgh1 z0bt&;TowU=J8tw+z;(~ETMoPus-_Z1+br>FAoRdZz7udivP`*v9HE}-0qcpy*91hK zxjAhCx?WgXJwWj*%eD?&5>nL(#NSvJy8+)jH^03=)d$PC7qAJjeL(V)rEdrbesOa= z4Ag(KGzEcoLXu-Z=7*(x5(wwGnV$kYzbs$VfXM&Cnv((;0>@?#d3BB^i^}?c>u~amedn)LrC>2 zuvWve^bQzKZsZGa*0N0e0MCRx`~oQ#OMiay4-<9VWD9}jdY0xA;G2-mGT^X*rNRNI zMsA{2Kt~hHw+0k8vy9dP=Y+bg17a;KJ&l0Z&COvm;AmxOXa(*FwcZAp9+uJ$AkfB* z>H=Ik%SR9JN~qy}Al=S#JphC{xasf%?oO6H1IQ7ob_B4xSQ4W^q??<<381ToWjX{D z_p-c%flEUCvp~F$<$MnC^>fn}1**I(JBxr#h_(zQ2Uub&K+wlc-WpIp$TDdH?}R)i zfsCKUyA6bgxH;JYJO)ck29RNvt$iR%D8~^H4X`X613e?${G0%#L6)&|;F?f(mq22a zrS~7;ALHie7SP678g1Z_Q061BHNjH$0*p;^Gxr8GhFCs7fKNgNzJdKImVY^5Cd^G| z-sT_lX_oy0Ab*C%RScXGvRVo(&axC$0DX#^nM$C1j^(u)xFKZ739Lm}E?j^y&yC&y zIHN4-Cg7Qn8aI$yU|IG66N}vBw*$>fETK-|n~GN1Vky2o$cc zY!3nFgsKbxu>{Lv5b&;YlN$pZYb@iFz#XCPrvP)ErEdlZnA}(jaBZ+O%>%E5+E@V6 zNtW^$5ZdG>5(nH{EMEyAN2rl?z}jZH*#IIbZo0OBt{s+x6i}RIsoMoE30d0%;=3%x zhk!4`&8!7f?XkRN0hU z%rf-~NS5XK9mo>O_XR{xSk8Wco>OjGW$O>6XDq3F;F?fXML^=5WvK-4UvTqV256To z6As{!P<9otb;Z(O1B_jBb6g8F{$pvb2R;dvYy|diSSnh8nOkn=TLJx!<+}~Yzh?<{ z0H=i9b^(hvOLs5O_rT3zKT!V2Qttz92wC?7YfmgC!+`P3jT!-*FD&n)z%wDk6F};f zwSlfumY+wUxQu1=1-K;S?hS~Sv-Erdz6x%RzJV$S zOT#Z<6Jq9V{~=k)Qd$TEtGJmf2I{L>K1zXiLIM>)Mq|0I1j04kbZUUd$+G7Jq?W~1 z2V@DAXaJ%vmZD~$r;eLxH&9y7^6CMu3FU7G5)CXDoq)fQo3(Oajk@Y-|CkUY4>QV4{zk$S%;_ z&+@qkd=oNq2poD@{#k$;;HK*Y=R6{pnQuZbqd@N zQhfoeZL=(00Y-`&xdEIzEE9LYGa(N)kV>=kKLHcF++<&X<_t^oJMc}&<|lBt$5QbF zsC{mtzd*+U%Xi++9|{jyMhk&+LfsYvu_KnAGQewbb65d5j#(P2fIC91Yk-+$DXj$p zC)}tyz;(*<(Ez*>YPcClpRruKfzUZO9c_U7f@QBA$Pucx3$QL(5wU;a?>^nRNb-ci~}|yS_nwqv&6zc(B>v@7N~z=y~JjXA3BOXBpc8t_gLw z3nV^RdiMeUCpSljfcC}Gcnmxe$~*zKzFEr7fw3QM<}QK89Lwi5@JXn^EwKN~^6ws) zk^h6&M&~1-=K-?+4CLptxZZ$MLRLS3#R8V1Z=kP`o0%L?Uc~Y$=|9{MGE@Mp6|-Cv z0Y(WodMV&6Wl5I<&xF)e0;w{V(FQH-M@(mbXp7Cd5bqNuA{~4Fub{Y2O3tJ6LuPfOkTi7Le&=iD!Xu z7dHiGfTx>f>H?4+mgg%VODNwB5bb3-y90XqxM_U=O8Z$-Prx;ys$PMFmu2Z4@DFhF z`w3`1mWdzWkx=$8ur)dX zA;bMZD#UU%08C7A)8Pl2!z>vC_$H)w1UQ^#Ss4Y?3^#=nK*uagI0O_bmX|PaPN>0I zAU4NxJ_mRs+<2mZW1eMa5x65%^)g^aSz;?dV1XN116+$NlP2&=sD~twUSjcX1ECl< z*&V>W%+itpa)jF42dp@Y;|PeXa1%WSx)LluCqVHk%jh|9NyyzL5MN{I`3LycxjDK8 zs!Wy!8?Xs6AA#fsOX&*`OmZ{#2Gnn|e0%`!gap2U%ofXa4hU~^)0vm~gD1tZR{+Qk zi>nyO5-L#&MAIxq6+q7}H`A3sX@=#s8n`Bu-w7o4ST0%8`7x+MuoNP3cxFdIm)95j3W z=yU?x_bgj3U?R&+!ws}Qu((%%XF|NL0tY#kaX&Es$W2-TJ$V-6CQv5C(H0PYVrdHj zo@Z_XVZijlQi}j}LM$EvsRGN>5g=c=vBdyuktG=ily5A3=RlT_`zqnD90dtLIwE{E< z`CbLm-z<)AAW-M#?gy~_undaJKd2fkrwSlP2yF`x{$<&21svi(;M@3Z1GG(mT-E?Z zLKL;Yv4Uly8*nvqbJGhLlq~-YK$Q^3M&P1_rF{VKskqr31T3vA-$Q^y?|TC z&Bz*H>}I)G2Wo`u2>{6+mc1b0@8za>2N>3~_;vwBA4_=;xFf{O0kCagxsL)9{oGhi zfOaEG{1kX5L{|bh_{Xwy0nD4Y5m!LZ0E_1uC==rC28f$kChq{xAUByTV6w279)UU` zPM(0&5KBh^ki*;ri-2{6rCtJ*R+gm?AWO)XParhPVy^*HHf~aNKr_bD{|giddDxWv zL)6ZqRszm(Zlns(Pq2KcfeIlPI)KC^OTH8EPH|(^0p@9z%O0RX$lg96?O=)Y1A!TC zlqSG7%d%z$RC6pJ79dB6*%2T-&ypPl91GkG*#WJS z2*?w%5C$S^EaUsYte>0fLqNCAVmt;)gp9_3m}F@?1Kbs1eec0+O37 zPiep(yLu-1uLC_Fb0B8}LksxiWANW_hRp z^LyM3e*rxamc%zuCPa4wi0`xPimN|(4!BV?1Exb3ZwpW-#Ct1{in2_%1M-NQI}Knx zW-)gG$`h8;ZXip@PCXEcv1}WFsZ(wmMnDs1SsnlignS(YqGv1J0YH1p@)-n*gqYs}j_+7{u;19b2FF$)GsXYJ0MR;?E?@guCmT2w5xwu{V~%1zO$IBK#h>cH6U4L>G%QsAKYyI0>c%SdQ<8T zicc1o61XG8iwfATvW&F>6JOkNNBXj)m$mVp8xySzYD%@Xnf&NgnuI-qZ7S=j(8gnZis5*;j4+kjWY zO(q1GJ6Q(yfCeE?_JOpPMH2-AUEBnZ0h^BHCkCjxS(eU#93cwlK)8p+o&+4d++3#s zt)8X-1}G9DdJ7!)v8c0v%fQV>4lwkyeC2^EA)GJ3g^?xy3i$qUGg<;HCYH-GAO=|U zpFoC?kuM-TnS=4ZG`F91EWEJhblCd82&h|jUKtpJ{RZUU=- zX@RA-2Gj|$D1nrd4V? z0u%^27YCv%EV%^W^l~$N3Fuc@&aZ$9A-mH+!p9QM0Nyoj6!(DH&$9XeGzj_r2&C6p z9M3>Na&uPzY#S_tZ-6Sma{3PB2%)V2;Z2t9D&Pom^IHS7TP({zK#>r|U*LF~WkQku zgKLMI8zo=}vHWWVs)RUh11@%1+BJYL%+012ui`j9arFWjLO%Ba!F`r7BVa$^ z=E?-94_OR@K%S5XLqH_T(rN`}kGNU40lH(BsvRg1a()tsov=Jk1MV0%BeQ_fQlwmPl0Chr~B!Sc|OUE@J z@3;xx0M>h!`YoW$vMgnREFoWVKHM1Buff6C3TR=?7(iQ^TE!=E`0i%kg76EF6bRGi9R+gtDz^~?J zGzJW}u_WVwqMb#54%`u9{}R~lU};GK6B=&(X`sE6rIG=j2{CsM9B5e{9)S5SZie$f zkB%kr43r7c{R+gpS$5w5PY*YWGGOXu@m7F3A>OM%O3yO=4ah!j?tTEPfyFE`e^BWGSLk<&D^AW0sSD$KLb!9z)|A&;Mcq=%)W0Qgt9*(?IXUY2?ZP^_}JK7czy zynF)NK9;c>FtNr>st&aKSq#6xGa(L}ZvSww&eEy`<|Q{$1@vsNRMkM45DOhZJizkU z33xWSvFZR*kmaHWs1u^M4@hmX?DYe3n;WGGu} zjl~XV_E^p)fC3@Arh#aLB{Ty#_qh@Cfc}7G#R*gh`Q`!=hb&XefH%rbW(6=Gu?+Zt z1|d)UK>C0<2OMYITpt431WW%hP$Wb& z1{|NWsLue`1veY#fZ>wm>k_CE!kGdtk}UZ&;Je~x^cJwBST64Yam}L70U1I@@<1@n z(((e>Z@5`|1=Jaq${Ua;(JQLz|5I88Zj1L3zZ``Dm1kOw*-qGxIC1!nuWS?>dM29|0+P$J~K35fNxJemQwk(-er!1#~lVg#rWvd0D_ zO)PtMz(2rE^CU2AX7Nn}ib0n03~)z?nR#H_!gB8fCWg4NECKDqEb(RFnGjuG;9!Jh z#|O+?xe@C?&nSy$11J;XZ4-#wSSGgt&loqE5MZ*inD&4=Ax`#z)Hq8=6p$0#1djph zBuhO8D5qGK&VVc-U(SKhG>bh6OgXqor2x$gOaBc}AmrgK5S?XFX94FNH!=t4=UKk; zK!uPCFF;~}CI1R|o!nSUfO(PSvJ5l`+4~8kT`Z9=Ah5)ZvJTkXENcxwwaoG%vVX`C zVzwCwdswn9fMbQ5AvK`&vYfR8MMCI0f#X${P#56xansZT7}i);^gxvmB?EBbXPNp3 z_}00(9RMtnWxxW64VKt2kRhaI6bJ@bw#ESaCO1FhfI7(Ho&xfOEI5G37R&e?FuTpo z^#Y*VVKKUZ5+S2*AQob2TLIj=+-$4@#xP554X6>)DS_l3%ToaGN4Oc?0*3clk~@In zfJGk$?g+6T0k#iWS`LAUC^!Bip#6xYasoUPVlECG9J4$mfcX<{hA)Ah7)#;`C=;SP z4a84bb~At{&W++8FrBe@AAmX`-XDQff@S&{kmuao6#(l6i}?*uUb39N16e|LR)A2F zWxEPYU2)T>0h$!c@()lTNZ+$O_W^+{H#Q?+dtkXT0jeBJ-yo19#K8~{eq>Qu0Y{#jbsM04 zV)?WKMMBI^0>{rRxoN=l!p+DmU?{Mh&jVFL^eh4wuPosuz*pp^*#lVKSXRA&D6y1% zK!%Wxbs+f8a=!uC%iIhG0rdwhCUQo9R8Dl9vD!0abCzx#l$%HoLvB|;XDf!G(z z;SACEWS=asbTr(0bBWELlA;rRB!b4`{kr z&i(-fLUx&fsE#FM0i503h!H^F!?H38R0#Pt1|)h}rX~Qdo}0`RVD4iXm;oAuJedR1 z1{RGI2=sFkbOAOa%a0pS{bN~L0dj;WtO8*Zi`@@62DrJFfY!{?zX=ox5#0ig2U*l1 zz-8fPBMcaZSiT}al@QKD;9{61e+2kOxEYNB7AwnT91x={`g0&d$jBuSw6V0L0Q(p> zYiU4jXQ^a>JRxWAfyg+^!vioo!Oc(}&`q)=o`Dh}b+15die>i=a8Gm7R0fO=7HJTdlx^lbc2x(7wpx)&S3h zc+~<2E|&3bV19|4bT823W-%IoG9iwPKzx~{Z2<6ixCsmbrWKaj5Kt$?q7_JaS)Ocw zT;;|#4p@CG$w@%D#?t2ivV`291wwun)dDcJ&W(Q&&`6fgC7?jaIS&xsV99v_XMmgG zH9)_~a=s2!2-zI~5b8{ z!P2S(X3x1UmZ};k5puo*h+VQgb^`7sHzPX0c*Szj1JnrF(+4C|EPMTc|C*a- z6EK`+@tFa|4NKVq+!11C1lZ27+>Zhix7=9lK>Hm_d;)kTMAtNMaL=+c1I%Z+5%WOL z1B=HAlnL?X0^&KA$z{Ov$W3MiFy&cHKA=vB6F-o8V(Hib={|NPMv5 z(}1_ajrA5Vf3jTO0}Vp<=74mSC6WgMU)(5P09%b^?G;dcvwXY(IYP{qfpDEATLBzD z+zfpI+6K$nH&7&mt^pkXvV_FbA6();jpA})aZerie<|H*ju^zF#>8ei+cdb6S6P}MA}%!hk@C4Zmz9>u7kxm z29yXH9S33>mbNLt-O0^{12Ae?YO_F%kj@1l*~Rj-2>5l}jJkp0ZkD76Q1r0qSAjc1 z?5_day(}#fn9y_M4*>0bER`VeOo+K1;K0E0unWxhb2A(PdWT3E~(Ksm&6dIw|)+4%s3hFP{BfvFL0 z8c%@6%CcMl3WR(u0?|>HiFd$h<0kz9=*L+8Re=g2k7_`|&eHw^c*nU3`~v0)mhYzL zKQsurs07lJEYB(+FvX3n4X{nKTy+2{2TPw8$PwZ|2ZU!>RK0*>mYelHKs(3s*$)&6 zF>eBn=UH-Qz_q~5$Pi#~vYd|qRYLUGfQvZ#i<>}fjb(BhaQnHr z2?54+7SkS3BjoWukd!PPQNX{!&E_#M9AK%(0L3PY>kPOf#LGFb9b_3x0ux)@q*6fp zHjCi~cqYW*EpV{I(wYV4L)^$5(6h@@%>!jZEW7~mFw5gB;MwEGS^`WFmWwh_Cq(Zj zklJV2`vT+vH_AF-J!J7U0A-ZrL%jSUOURjKAaul%Z2_i^xv{7L%?Zm{J5V5GS0@mS zv4pw+=P5U$2hhh^R`fuHkZ%ScamF(B5AY_q$qWGIbCv-M&>-Z=Fp$1r(ToCtOKyT= zfGx@LGY+V(SeB-M93cu0Ae>^c&jF5WZmt&qZJMRu1r!MpbpyvYEb0}&mEmS%6)@bg ze60ahLO3OGamSJm0KR)}Mz;V5DzT+Fpwc+Bmx9;EG>tC{gIosBS4*Jshj|L zLe9p4$P>#$0+@Z~X6O>oy|5&%fD$2fX&_c$+06j%S8kf_0b`NH`vBAkS$YJLZ!FW# zfWO4eZ2=g5XEDD4iZV;=9k?SzX9d{)VA-ky6BTY6HK6^I#r*?36XNw3IH{>+0X%hX0$RZI!&1`$bwVum0;vYeQy(CIxv?1mtN1VY zHj*Yl*#wBbK_E-W{UIQvU{P6tsb+5cHbA3f`LqKCLe5PB(H556G~iTmGdv6ETUpNM zfeInJ7lDMDCA$%Pz3UXJSYJXGfV3SFgwW2 z`X`{XuvEW*5+Ud7Kx~NRu>rV;xfyAC{ey9Y<)Rs=5wb@GB&{raYQR6rO>+k@Y-90t z0*Wz~au;w%h?yQ>+s<;Y2PVe3vGfD&6D;w6z%wDb%)r4U%Z>$@pW;T006o(zo>8Do zh_^8y?qHdm06a6?WTpVqEQ@Ids1xF34oJUN1M+ z7+_vyxr_r1LiU~mX&+1E5(unuqf7xdKg(JgP_46kWPltYX77QpWXV1Njty>x@_;tL za`p@q388xhjyG9CZ-6VvO;Z^#Y_Y6VfGQzMRp4TqW$GL7?QnDZ16V>V1ETl`vC9%u z02xART7Y1fWvdmi?{V|f2B;$}ZViwpWI+o=_F2Zef!PCYu6qI9A&b!fln5C$0XMycAmX-xz zBEgM+5okYWsVo7{gqZUH2Nx_4USR%`o8dK}C&`jn2g-!#4gm2hmfaxWNpYjt0Zi8{ z-d&(hi1$4pm1dbf0OSofcTvEaVKJWo%3GGxQy@#o&IAy;W7)m{rtZ0EBmqs9W%(K? z5c2f~h(54P+yTxUH|Z>(e`NXh2vi7p^aLdGEbRrr`@~J42$-K)zDq!Zkc%Hc`i15B z69^QzvDEkCV+06(v)qpY_BuC% zc0m2Z5}yF_gw#$0kp|1o3^4o4&F>tb6aO#14UZEj5whq4VofZQ%Ya(}$eR_w*vw+` z0X0G%`+=mArDFr|w{Ww$2@I=P>RW)KmBkeT?g;S`2Da5KWBb5F8#k##puL^Na11;X z;xGmrbg;CZ0rMJe}mi}BTGn>{^0z_ zjc5k+CYF^JphC#ERv-(fO(K*pbKaa@}wI`TUa!DATY#D&;ZzmS$>Rw zYJ_EJ0LT%dFbIULEcRi*G0M%g70}vP`p1AGA)@2J@feGG3UJxE*>C`cah9)Hph^hm z0&p?Gl3xUTliZBD0m~H2r3VnxEc#U-L&(S)5OlD#NWebB%~}9Z&$3j4K%S7ZJ3wTP zl0eB|FYY{l`v5db1^K0CsKY$)T zi?Irn32{^d;_EDJKY&Ma6Zi#88!WY^_dnDLv8V)60hT8fAUCII zupRL3bEB99%m*y1(?Ele?=wL9ki{_%1ftyBIRV=d%it2AI%YXt26BYZdV%l>%eD`2 z#JKtO1KLxT23%*{+=Ku_g5}>HP$k6iK5%i)(jEnT7u;+f1C~pc z?-(GGEUq&kL&)cIAb7*`kKXX1LO&La0^7zEUj5!_J*7F9H7gvRP#WI zkn=A#5iGg1PK_beA>phn1^Pav6P+4}~;$#>|y|Z+T030DILLRz-XpKd^0yw|9k*k2d&hoVeR0z2sfy56>J^*+d+*r2&^DoQg4$vTE zZx~36|B62(5&;5DfK(m=HU-Pt5uj>j`8WY`gqV#3VI@m80XSN?8M*|tDweY=phyT^ z8aQrc31tA6nwzG3z|h9B@&Hr`QF;U}+F7Qa0bd6L?=t^9mo(;Qvrfn zmaQsa@8agC2B>u`?jIme$igoW>1G*MeEeayhns69pzCEZwgM$WM%#dxo~2C#xcj)- z&;mvSOHBvV2PFu0SXg~eh|1L#QqSlJ;2gp1t!eg_-#P@ zAWOv#JQHGW5;(B1JWK=gL);9{0zJbliFu$*i0(xoKEkrQ1bD36C_I2^l*Q`>>V$ar z0Vx~H^g1BNxVhT^tacW25KxY@oNfbILU!%~p$V4lJz#2*o5nt%nPORv0tG_89s|*7 zmWflq>EI@P2Iyy4{#^hSLLMc7#4JnuHQ=4&CU66o=UKjQfd(NLvp{-*`8sgyVaYWB*9tcyO_e_w zye#L8Dg$l@3f+hUoV0NmT$+)M$+9Tw9JP$T5=9FPpL zbT|S3E;pMlU^vWDcLRz&7S{@JM~IhIU^~Jx<_9MBxk*W&{eZ==2|N?xa0@s%WN8fn z^HFYO80a}-sYZY@Ar=mS_%X}l5#TxD#u@`mF_w!sP$xw1IgmPK*}DW}oEv2du%5B_ z(tt9-@{s|ugq*ntLgy^m2Vm-g8%rM0T(X=!0|i2My#moBOXv-7UU4JJfIh{tQUNN2 ze5(S9YnG{Rz?3t#+;S6a1#EXLKW%{Oo@Ge`j3Q!IOVz(37R^9wNSVDY^IiW!#j8*oR6nKH0F%W_`< zCg!-Yd;#tAEb(vPnGjtK;9!AeM|}NZ-pP$<26`4*JS{+(5O1wO+{H544tSQh$!GwR zo5j=x)CqCY4WyP?I`n|_a1%5D))khz5m0(rmIi<Lld=MuHJ1J{ zpg_pOaUkkvQBMKRb#9~s&`XxDS)fA5g#{q7!IEDDya8^kZos_Ba_IpYgzQ}f(m|HU z8W7mxMkxW?Hp^N7Q0=gM1c4kOW_N&ah$XuVICi-iiU8U$%h>@?B!un=INoCkodB)~ zH%)QCu+OrR0IGy2T>uvcEK^s2?~t3@Yrqm^8OQ+Qh$VIhWC*Ew0D{LXTaSSKgqxoy zKpkUo7l1q=3q>Gu$};{A%*MI7{s44mEXFEOB4o4%#1brRKY;t3n~h(fBS4uD-8LY8&$4R=JXvlOlYr@g#XAku3GqGyq;f3N^MHKh z=FSON^DO2iK>5USx(s9q+35vB&n(+MVCsdNh9A%rSe7?{0wG^Ff#@sC#5Uk8a+3}L z`Ztz;dq9PdNBcmc#L^xGyzkrujsbI-mL@>1HUObuDU0tvWC-bK27=8j_bq^3$<3e|P`9we+kreGwVgmj z#j?`{%(im#+YRW{EFL{jB4p73#M)RU{{ilHZf*tuV+V`L0@Mh3JPaf?EFGhOzmuEI zF<@BBQXdBtT`aCC;EoV44q#iyGByWHbaRth0NQ(43@+fA5QlEypqHg}1(?@!BUgc* zK9=elP$tBJ1mXsk#{l5z=f=7Pn2anJJ3yTfy`X5kQil| zdIr2UZZZYHJjOEc1~dqH@(!f!ESd@s80RKf1#A;6KQ%x#$+GkV$rbZq{so+Rak219?KuP6Cl-mWOFz*2B%vETCIqNz4N! zLh2TQn3rXD32?7+)8qk+J{GSRs1dT{1Cnbj)9ZlW&&}-yFucxU4gw0v659su2+_F< zY;Uk^?Ew=3ZW{YQ`zDJ!3Op0y^%yt^vW%Ys^IP1c&w!q77UKm_Cd5$^i0`npT?3vF zH-Q_#w98Vv1?q%Y%mS$}%To@Jd)(Nb0BeLL`2r~SS^A1VmXP};AauZ@`T(X5x$%Dj znkdWX7f>MNTpfrWvE&+n^O&3Ark_9PPgu^IfeInJRX`%f5>^A=Q*IO;fH}^x+6goW z`Q8Pj&sZEiKp?@*ogT2AvkdkFstcCWe?X29S~C#7WZAX=jwCm~!+`dRWqA}R5~4T; z9H&?&CIHtpH#bv&A{;F2Hij^6du19gAxP$Pn^*6$sw5 zjQIh3mYXXHs2^Ain?RnB2U|cS$I==CW*@m(4+FY9OEm(N2swWU#GY6lj{x^GHzP5? z_`-4#2Wo`uIR}yjmc2{B|H@5s3K%Z3_|kyljisCc?g%k+4{VoM?jL}OcWx|spuNlz ze+Hfj(e(-(e6Z}i0rM4ZL>cJ$WbsshG9lioK)lK_`3-o!xXJtgrW%V$H2zR0#EAk( zeY1450J6?auobZWu+-ZCWrJl&17r#Lq6I>~EcR|-O8gIe8>wDE(*%fq15hC3p%I8G zSkwc6vzZ$?2E(>ODG7qblf!U0ETXs zm0h4ph|(T#(Ze!z0Qh>jxs3uAJHGkaLo839 zfPa{q(Hby3!jh~53M-5L7q}zDe$($Cwntf7l)!|I8@~!@A7iPgfoDR@bpQu;mWNJY zew>?O9ndqulIQ`-gy`-A;*%`9{eWkR8-)olO|y8-K%Efp79izdnH~Y;3^#Y9fOVF| zYzLHcEToS7o{-uG5DByF2=RaY-OnC3 zzY0JXVezy8B|;Wkf!IFFWINzK;O0gH7!O%YT|kYH$K60O%F>|+{72ku8i3(rmbwv8 zoUphCfIC9G3WY-Z8y<-WT0M2`EL>$m(SymE2g^+I-K;nUA>I(4YxXD}t z=0}!+4A3Ct$sLf+vuGZGz!NvYN5J;X^790!URah2K#mZFA`mXH*xv!iD>v64fVRle zUj>SUh}MAPHx~5|;3{#m@e3HWl!xUnSxtBxg^0+ih>eK$asko&hlsE0+B1*Uqr@#g@Ip5-$S z6bL!@0z~^*a<72Xz|C+8(D$>Pmw^f)yFY=1ktO^Ec>i&ur~_sb%W4B?5b|9p{*WGE zaWn$~GdFiFfNhXvPz|UoET`>2ju6^TAUwpf-32&?x%uq|v?DCbdZ0*%q5(LzvP}E~ zT%+9F3;+fj%RdWHCB*SCa52WxJ_`8k+-!~kmT{KvaX?J4xTb&%A)g&UaFS(g4zN#g zbF~1dr&$axAWz5xHxO~Kw5|ZNGu*7N0=ik2>KafY#UIg$ja?^YW47*r-M}T69rF;V15n?6|Y`aU0-g!c zl?D!0SaveNyq6ns5A>|EcpiWJN=xQ}J>8St!elPLfuKa1%Ns1xGk9Z0RSbW{K- zxd~PQ>jq1`1}Fn8OFuxCkT1VLXp_aRX#T@gkeiee&}^~vw*m!19<~9|Z5Fi#aPDv; zwSYdv@}&bRgk0za61yz1* zEFX3tM~K-;ARJ}MP6Li3ZiZ$7?J>*QJWwQrZV@;>VF@h(t{68>9>8$Qvf>4*gedud zi#W^FI^a9w=5_<|PsqYC5V>L* zKLut}++3dlx@#8W1yCYnGzrAgEN$0-`-Yp18^D-hsoerKLOQcR@|NW(2l(%}8GQnV z?^%*BfFjGHF9LUj*e?Ow4=gPoz(kH4|0mG?$Wr+No(VBm2M+Qq4-H`ciJRdj6OLX3DgPk-UXzJEYm%JeB z$P%*C41~%o+ZJHzgPX=MpsBDdj{*fkzK#LWPnL-Zz**%cJq75$SpLlb6+#}(0f`z* zyA$w!a}#g@<~qx_8)y)6aRo^Kusp8a8xQRs- z0vrlJtcL+@Gs|ZLC=z1+5I9z{IR#kS zSXR@3XlE&BfD9oW_du|N<^BP%Yq%NA1L{te_%o0vr1lkvXjyjNfY~l?e%}F|j>S^} zN`x#{fmk=ofr)?Iq=;+%`$ZWcpcniqJVjZW#9y8 z5c1>{NYAop5H^?$bF)zd49hHEC7?}%YtsQ|T~rJ@G%gq-aFBI_&ZscivuLM(=W z)G^Cb7?3C2*!BTyj3s#pC{J1Xj)5#8_hUdP&Z0U4rp~zWp97i%%jYFfAmm&Mh@P|L z(tz`Vo8enPf5~!w4^#-*odXg{mT(^MUU8#%0n90u)mNZF$oDrOea+%11A#O*cNM^P z!!q~r|R^T|xGSLpW9=N&D0EQgP zzb>Fkh~sYH;*q6Y5BT!jY#IQ|6U(;|5YH^G0U$%j=RqL&!ZJ1t*bCfTSpoGci(w4N z6Y^jjh!j~`r-0cvZq^-uuEbKE1xkdRUjSn7ERTzTyUfjq8!&#bTzG&QA$wMVWQApK z4e)<*(=37EDvK`wD85+ALEw%MGdsX`jpcq9nE2+#5&_!lEb#;2nGjt^z`+m8&IvHz z;6}uOo?jME0w@#W?E;94|BXLn@(S=Y0U~n^m=r9g3{WS;$sLetX6bkUq>`K9BVcV| zsXqZq70Xfq$P)6U2!vW$?C-#onw!)IplM_2uL1=^9@cLw z2+)nP825k@A*1_1%*N6d1>9rYY#akdJ4-DF)ClQ31Crw`Pv?Mtf}7DKFg(eUOaY21 z7X1xyM~MAfV0)USB@0YAxbf$J_8FE+9(X3i+zW6p%kuCF%+GN%TmpLLSrTQSOo;AJ zAilt|`vrKM+$idRX_3X-0P2Kz7j1t?xmc!~0lCD@T?=4!vzXO@a+&3{9mo>0vl9q; zShl->sTFP--GIi+vaAORgnTss(N&g-e}L1+O?m*(ud)2I02M+W4Fd^3OZzC`UFRk+ z2ACzw_c+iXN-e7rl0D%BEwmHDI$#S&-sDdngE+9vU12+)fVo|LCj%{w%R{`w~ z%jX(UB*eS~jzcWD0N~o?W@HO6gjvpafGQz+!obBIOE?1fBHT0|0+xN2)gwS0u#``L z3?UtHAb7}fp8)JpZU!#_^$|<_3dj>un+77sEIS!s_Jo_?J3tp>@jL(}LKYu^*eT28 zGvJPMb5j6}XDp^Sphn2!cOaQy>8Jqyb8a@P!0-i2y#^>QSzJHB9U)$Rf$b#An465xE{MtA^yfn~)DR0#Rz0}`(+Q|o}Y$W3MgFu$=31c3%2Pqu+{iAA#u z1m3v`?g6$k%g;Wb`e0d#0y#nyj)8E6#eNDnKDoI*1GH6^{tKW;h-eZx{$f#I1Fjl3 z8#jRAo8{{ks1m}N1up6=`5fT;;b!y+urydMUjXsTqAvm&LPknJQ2ZZ!8!aDzy$O(O zpMYAyQuzY%gq*Ddk!F^M1~9ASW~iy-54skXL^Dt#q)r9IR4lt{z}?DCQwLyFvv@m! z8X-$vK(dWxx(D#Lb91W)hC5iy{eVKl68i_-5u(!!YSL+7fjS`;SAdj(g`w#@Qq1Kb{NM#cf-3d_YLP$OiI14w#V_GSV9DmTpw zz_5?Sw+JZKSjtPl9U*2sz_y>|-V02ub7NTp+9gYT9e5^0R{%KJVA%-*^8s$e4$!m7 z;@Jhtgm~Kn;z5?l1HiMzO(qJMwpmOkK%EdLr$B0lr6U2z5I4aKz`Dy)PXfv?%hENF zCFIKu5ZYt0-vLt*ZcAm|@wP0qnQj{LBIBI~KPS$P=>Q0wVV;_=P2T2q+3H`eWdZ5c@G; z`<12T445c#<39)5-&iV_z%wD{QouopSsCL10D#eIRN5%maRj;HNeg95ui1&_|w2EA z0`LuSbA1IEhFOMkK#h>eJdiQ6blw4>5pLG+0kesv{s=UWvUp2Co)AxEVAISp@dnt( zxVd}>IxH;uDo`ZEeht_gXKDKZ+*WSnFVH)|Qf=z~Lxm7?3gFnra<2sZliXO^fgwA~ zStn2@M4tw@oMPG00@A^aq7Sf4vjqDAg_Grd0JtUO%n%TrVYxK|4i`73Q9wP*axw;# z2-$4~5^k2L4e-oyBc=eIhh=FR_#ouV43M5@akznimz$giFfOnRF8~cf9xeh|AB#Ew zgcrGq1Ocm`(>p(2PV%r3q%iLT=0Zou)U+y35azT-8GOSWc&t*L|9sH0ow*QD+NHc z$@1|4JP^|L1jM&k?w$cxl$+64K)cP7t^luu)P4e~7|Zq-;M?J*sSX(8EP)14Bjl^l z{vor=GTj1%_PDue13||3p7p?9 zie+*Ta367#9R_-jSqvtiLWl!1aD2kjJ`VWP+=M59p;MOHBv2>ByaTvAV|jD}@|+v% zEMUp7Waa?H1xvpdxFzJS4~Sl}D3<`o6*r+}K%HgzTmedioRvW0nxzm1JUMR68-VVH z<#Y@9AY@MrNatB%aUk%Io92DMc+0YU05k~sb_iteSe(Z|xWG+54Os74M$Q4{1Iy6` zP#}aR3&b8-HgkZp$jxsa&^)m$-T`Gol<$FLiN#(7ywBWRmjHd4W$*>45+eBqoV~Dg zd;q~$Zq}=S>5b*P28ara_Xo%kviJ)`-dQG^d;Vbi;O0UBs6JWrZNLK|_u7GYm8DG$ zxW2eq)d1QWOH~WJ64KoVq`q11`vG5_n=u1m_+dF40&0Zp9RV^8mYq=`^vg|)1u%>M zh_?~60?kc;tk{4&A!eq4&1RN=(}2B&8i}df!;P2 ze*mZu;w1ElLT13Dec*A4JN$hlh}-Out+ z00Me$ERTS3faUxNXb`fm3}g)~@mC-`$c^G1unw`Td;-d0miI59KnPbIh#6UK8-R0! zo6)A;KWI!WCoMpk5LzXW9A$~B0I!*wrcOXV#e0lSx*Pz>l;VEKpxMMBK(1A9J}y92jj*7r7A-kS|sASoE0UT@GG~NJp zm}T(;C=v3x3MAH9?B9SV!cF!E&~2~`ioQR55b~fINN=)qv;u)GZo+MVG0O7Y4m1ck zuLiQ)EJY0vj&Wn{0jxVL7kz*-&eE?33WV4*0I^*b#uo)Ctk|3tT?2 z>@?~AAWPgR6oBQKC8z`xWtMjpa7)OUP9XZia@z$sUb!)80reZpNiR?$WOqN1sIWu_ z0M9!&VhGTEuq+vY4?@0}fb=JeV+;sXxyg+K#xIs(8_*!+p&iK9Sk%)%_?w%^3}CIZ z{LBK%AC?6VP#{Ff3&a{Mwnf1C%gxmipb`HGZ(|?`lnIdt0m&v7)f(V!2E^Jrpl@OM z+5oDA@I-+#12jJuK4| zAk@pv%?DuaV=;aK%{rFUH;^YpR|DAWXW0<_f3WMhX*2^J11!E)ph$@4Hek=dGT8yR z2f4|rf!-k&LpM+%#6b^mJj~Lr1N=sA!g^q6gr#Nx>V%jd1};r3k0XE_<;H3TEM}IB z1yGE!^iKe{gxsA3q81jV18|IU6LJD-E6b+~C=qgY4oFO}6y^bsjT^HM&`q+O`hgEZ z_ACQwJ4BKPvePWiEgXIMt!fYQZsvHUnm1S~ z13;b-Gef}UCd)q~VBg}#GzxS?S&qkmA|bl1z}_~?mJM*nxDiu8?+%N98mJKBWd=Bo zvrM@G|1LK<4=}XHGPD5H330dxT<)`U1^}7hCK3cJ2Q2jvph&VTtO2)#d|C&hhb*>D zz>(tSG76}VSO#{05+V0@fy6P3Dgk&-xRFUfmuC4&0Uv~%I|0(CEDxtZ;EWqf1~8tp zoL>SBLiSw)*$hkk1_)npqqqgEmn99jy`JTC2&fXGcLX>aV2O>_v`vTyf<;L;|47pj(o`5rkv*rc#TP$Bbph^hO5^xq} zc~}O5+uV$Y08@3=}zn46nBzeh=PIz5VVV2}+!x$re}LXg7K0f2 zLxm6r&A{;$OM5He&vFxP1BR|yYVAOs5c6u_GRN|$0ptxg)*ir;XUX&dihnHqdf=9j zy9OY7%c2|x9CzG=MgVnz<#QA$5pvc7B<@)XR>1SXjd>E#J+hom0Uw0yaRTWgOUwlX zp15hA1B@k><$0h%$hQR``^@6>1K~0^`2b*jVHsHgl&>sDt3ZJenlKQ1W7&)V&I&ib zn}Ft>WpNuQ6QaBWBtKZ}dw}K;(yIA`jRa++5rOs$UlUJ@7!ty+0B#L8VhHH%X7L+=3L#!hz_FHPY7FrA zaFZJchI(0sY(Sk5hj!qykEL@OkUDN6Gk~R^r9KNN^ehV=;Fge2ULZQaVp{|p25v5w z0QDftKoBSqaz6wlhFDZh=mJjRXU1h86IR!#xsILrGvP#}cs5{Ow@ZnJ=Mf}7DBKx1P$`3IB5KU&6*xC z_*iNNphif~Fpybfc^m;kes0FifO&}}V*#22EV>CGPl(+~U~`$J)dARp+=QG!#|q1b z3n&s|b`IDJvE0oA?p1EgKA=~!r2RmJ5IxJl@fyqa3g8cO(=36Zb(TOFs1xEX0$fH| zrndmO!A*V}uxzpz2`I>xe?218Ov!W zP$fjK1~@xsiD`jghMSf?z;wZ~+z*ILmdXH-BcyW(h+MJ!GXl0OHzT8f>YC+v40s@< z#tOu9EL%3fb;HfC9nj`k{L{cIAL1IL8}Qw7bL{~PcPv8-K#h>eMIcjP=?nm& zdv4Z)fcb%?9s-&lS-fjNo)Ay#z-EzUViT}GadR03I!Y}19iT{v{as-1nWZfOxXawg zB+&c9QcVFBLd=~2$FD5+r-1*B8%qWls<5100(C<4T?3czEIT)V{NP4$3s^o`f(1ZP zWqE%9ZV5T_1Vq1BZl3{1jT_S|p#Ek#sQ@KHc7FniI!p8m@ceKi>VU4nveW=R2>BvR ze@OqbI9h;!_%C=HxmLi~1c+f3&>-Yt2as+4FaP0x{U1Wz1w>i^8R-V>3YMQ9K;6o+ zpaaT;DCvQelEpR%_}jR-8U~CimH`tW+F252AWuluI1ue%S(^Zyo!op)0$MeT#{pCb z`QQZ7T`c3XKv2WY`5a*GX3=>8g_b4m0}6y_T>|1gEGx@^yO*1f6+qX=;*vm>5O-l9 zqhlG}0AxQm=`FyjXVJz0)d0(O94Hd9X&*=!SON!tcaWRcL%=Y^GJOox33-zSvcoLK zb0A{mCUpVWM_9VDfZD{ekps$vXyk#^D2wk7@SC}Lz6XqBER#h*SXiYe^5-ZWE4Pw5dCdH+`+Qj z4!EbeQK|u*lO?19s)YE|0+|_>**-wJxGD4l)>#&_0Z_SFPKSUZA$vxE#2ia(6!3bu zX|@1{d6s1>P$%S@4aj;~oKrw#ft&m^VE3_%xB&Gc%aI!>6GAf&r2H(K3xI!#o8Lvi z7+_fp0AiV?90c-&w66luAj|a{;9TKmFal^pEXhrvLdcG7Aic`6z5@g$H{ZK}d5y)J z02E=CViG72V&Vvhud`g70PYAk`ZGYc!LpYDs)T5}0y3K{tJi?s;--27SfecNTR^qV za$f+7gdBSW5;2yuC&0VIO>Y@6#94M;fjS{u-hu2cOYjqj>~T~10_^)NGj%|nVENYo z%7idAng5VFU^#99{7G)Qm4NY(WlIH!6pQEt@`Utv0nsCtmu|p$%*|9Upgm#9>3|9$ zhX#Ojn&og12%d7&X#~t?ED;l+IA^JwfdU~G#({W-SFJgj{O_5_OhZJK+7{ z#-j!d4VFg@P$%Sg50L$3$@Bpc@qghT==6ZS2@tymK;6vJIt-Kv5gGwfEi50SfM3DQ ztOYQ(vfNn#p=2>n0(nBFr+{c1OOF$9s<_#90orz!<~g83$iO_1?qGRa0D_&|O#1<| znk63q6kROF6`(+fqg5cTVd)A3?rv^2BY;lJ(%1y5gjn1LGCeHMJAmxv#=Zww`&hCG zK&4|DJOqk_JU9Xp{VW}6z^msbdSiUnrosjcaK-R!gyapnJ+*tE~eTe1a7Elkf z^xp$zLhL;PDI<%r1o%g|SuF!b6U*l-AVyi-??9fAg-;-AW*Msi&M|II>wwn6()$Zk z2pMa#{2@Kg(xL!@R&JJ+fO&$Yq5>2)7H20=AjH2eAU?@5q6OS`ZjO5a-4u(aAE**y zYXHbNSbhfqIn9mV2w0sgWfP#9VVN2OiiEr#2NEupAsgVGL z$5NjKA|7tM9>6}&^5g~7UY3bPpiGF%B_OrHq7MRoA2<6Uz_`fLwgw14i(Citgsg4= z(Iu9-DBuima~}h=%Pf{%phC#AeIOlV=}Q8^6>fG?fH}mXH~|!^EWuNtK#2EqATC*E zE&=x%H@8_p7iKZt098Vq`~x!UEZqe_M!1PS0M-o_@dT(gS(cuGA|YR1fW#JyqXKxN z+~ht0!#2zC7f>hU;WvUvK);8WkP7IKuY{qe2PsQ;BNxNuN^Qpvn);nqJ^bA1LO&5 zcLPxc%e4n^wsJGL0BDsg$wi<-$c_MzZev*w0znlw-yy)<&f;AI6df$Zb)Z0qiA^Bh z$#M||+-h$0JAkf>Wp5X#5~3{uWHcOh{5{ss^oVR;b~e{h<(nQ8&Fqb#{rphCzY6_7Tw9CiS~F>X4$ z0JDW9(hVraS?WDNfe;HiAZ}%O(gW@ZZft{q&c<>%3{(j*U;;9eEc<3a+PP7U1J)@P zIRU5~EMJpAk&tr^ATiDI-~_x*ZY;BaVTR>=4yY5d&kJN-EO8$YndL^Y1lZjyE6ad- zj^%v?C=Zqh(7z|HVEU|wcPT>y$8i#iJw2(gg^;wvmadB7dw#&-wkR#{5-K$Q@a zMIa+tu1bJhxdH$b({(*6My3At7U5)qbK4e)Mo}7ydo<(^D`2TUU zdJPzFSw3$7amV7m1@eR}6o6=fW$Y1f-g9&M1ZW>vddonCkg-=F{m9bt4g`zbEPn## zCzi?=peV68>p+1J{~AF2nPsGD@(=DZH^(i2?uA981geDCQURG)mfsFQzH#I40<0C5 zayOuQXPN2+iiEt@0f`Tmp#i}A$xU()FjQGOjX<4{>n0%k#Zor|ks3GNalrn~@-zXc z>nsy?piGEM2ax(<(a!+>1~>b&fbo~5%>xMW-|!ry7swN`>I0%pEOSeMvl)>0%Ye3p z#S#K4gglc#TEWt{4g_1d+1UWhN)|;FP_(fGV?cor?{OfmVwu?o-0j@l9ss%y7E=nS z65`|-$aJ!Fp8`_NP4pbFcCm;{K&4?>$^u0~zT|*JH;dyR;MH=Iy8{e8EW-~#osfq` zAlu8Leg-0a+(cdgyN>1O4N&*9EPMcELX@gNO3z~Z2K)ouT>St>1IvK0|3M70B$|Od zAyutFbckiG4R8)~^VJS$jVvBDP$A@l21t*vjQ0RR6F295fO(Wfrw0^fmbd{Z5TbP$ zh>x+Xi~w#6Hy@*bZk)ws0jh+!vjQ0_%jhH^C%8#Z0ahD})(NO4S+-q3k&sPuK*G)v zm#-hmdq_sAVmK?5D&2IJ_7D#Zj>cJ7i0;Qfhr+BUxCaD%j`QKL);WT0qZJ@ zxdx~t%V`}b60+wPNUX8Mnx_8X4Rh1101WFa%Sxb5$Tt;`jj%X7fyf3o`7Xe|$ugn^ z)LSe^y+D}|ntmV^W!W46{M+384g$s)%c2nwJ1k`rkSC;l42Z^AuEzoAE;oZVK)c71 zv;!4Fc1#25eU|kZAei9hdloPsuy{RyBFR$p0tG@$ECTUEmWw68o#I9x1awC%dm*4o zh_*E#bIh{34#*R3svCec&Ek#%s#BKx7*HhS*e;MbV>#Ogyyx8XCILf+WhVvH3E6T2 zWG`5Pr$FSAo60$0zharW1k_oUe_5bR2-6Lax@I~42l#W`bQb{Q4a?R8Ao48Y3CI)D z{|rR`vAnzh&RcG#DuDKmCHDbT2s!ixqzf#E-$3x5o6ZJceqf0R#~&1rEcIrfK!}A_ zAYNp7Y6IL)+}Jt*U5VvV4O9s+&<$jsS@wGXS>{Hi1FSDBQV*zJS-uQFk&tu4K;n(% zVFd72xUrZ4!#m5l1*j9UZvx1Eu*4^U$R{@n2Vk$VtT+Mn7t6Z~C=eHY#ajdR+gV5z^&xQmj-lgETuD`N{Gn|AfsZrx&mZ7H-;Qw z?O;jd0aYhU`yEgu0!eSf+3WPWs1L7u@E-T<3tX4=1}cP%-2mx%mX=!}=;da)0GJn8Di46d$Kref3WWIg z48#{%MqUB8pPS;MdsrLzmD6LP&9$gZ)}dw@uo8?O$qud_Vq0d<6BVh|`3;&K>BZLsJ~ zfPa&leKTO(Vrd%(M3hBN0C_@ICxPfT%bWvn#<;n60@@uG%PdeKR)&4(7yJ+ip^ zfGQ#G`hiT5Wz+!3CvMV1fVIS;9RXC&EZd_%k&sOmAW>!sSOM<~H?KCp@X9hh1=I<7 zGYw?lSd1omE7{6F1R{>FD$*uu; zLK-4K^qb{i6L8kKY2OC4KP=%LphC#nE|6}p%qM{0FE@`#z%2eBdn*;_y=xhWI?Ya5IC5m2dEPM?4xA$!U| zqMar73V1uXX?_O`oh-|rK%J0pUqDvP;;aLaE^hJ-z^-8#X>$ER-OX~;0+b1%Q35F~ z%cctO_i*#u0T_E(7P|n^$5QSF@`SYa0#O~ywGMFhb2B&qX!R_~L7+m&4kM5rU|BZ- zK?676X23ki;vEMRLoCG!pg@QTI}jgcxo`k(BRBdPKsUm&Hw#n=(dGd%CYDt%AV;~W z`T(n$#k~Zm##ru`fg&NtLO{a8awY-qI5)lPfWgYLvjNlz*%Ae^6D+|P5V3Jni39dY zmYID(ZD;v+0F((~N&%@Umg8f<@8G8U6fjP+Y@Gwb$s#U+JR$vAAUebHk^`JBZl?YL z+F6#|9Z(_U&;yWmvm6$I;2bxd&w$y(5_tg>^DOl@pg@R)4Hq z`2$o5F(79DkXdBeZw92F8&xY{U1E`KfGWW9)eaO1Ij05^%PbEXz#HVo(gPS)SkC)^ zIwAY?KsLk@Hvo}UZWP0SU9zl<0O~cC_feos2$uy&g;{Q`fPbBv(MiA$mZmwtxy8-WJfMxTye;KAjo}cmCRq|kfa;K?Jq;8IxpoF5 zQY^I$;638Ta|IZVSst%}Iw8mNK=yYU}{ z74T=cnSBS07c6(5fVgBa*MK}B({&(v#nST&IJ4YrH@W|yy=G}v02M+8lt4Pi@}>fU zH{48j0_HqRz6((NV=-!h0wIoif%q*;S3lss<7RUJ&=pu3gFuxKi$)-G&+=>nETe$H*16Bn~`2|q5vP@NgA|bCofP|7|=nL?+ag+Q83@Vn+ z22dyDx|sVzwwXA&^$SriUHF~AaZ z0tG_6yMVZXWo8a=4{~!m59o$iOg^AWh!a1M8D{BT2BeXj=n7yRVG$BgnOK&>K#`Cy z5g;+j;@ASbW^QuZfMJYfI1bbadAJ8;EiCE-ATrKPWH^#sd88YmN@bOxks zEVc{4KgrG26=1Zp4CDYY#gfPac|xl0fT)9I?H+JWbMy5GXq_yc5>O%JLm5cVu#CR} zK^HgY?|^xhMOOtBZkBirC=jCc2Z+zHto#CQ4>uo8oi}Q~uuKmDbwb`4f$TDiaTJIIxk-%y z_7#>cE1(XsY}kM@AsTidwaVg~2KDP$gJSFM%Q5uaFIkcuK!uPUT|oMZWxX2+X1V$90nFDdULBywu@v<{ zfe;geK>UW~Vi<7exzU>d-9MH+Gf*W&+c=Q9Wm%m7oXr8=M{at(fT75;;{)o1Y*_-bPb|S@AX4I{vI5wjS!N`lF0=d#17$*(Hh|O% z%kdWAf90k-1{mL1w&H-Ou!wyiPe}g(5PfHPIRu;^+)Nz<+E12T8mJI*=p0B_Sq?9N z;1@TYS-@OliR1vqH%mPa6bP|!2gK_vPxpZPhZ|cF&^1^tOF)$n11~`4mu3GAkmCQv zr%-(W)+Rv8Dxhj+`Kkd$LeBjFi58ZJU%;#2#?tKlgQ1n>TmjSx+1CbSl`QdgAkxN- zLJineEGrs7-Olo^17-^tCW0WhjrPKE%{#iAVn@`Q|z0#OZ1lLc^g zbF*Xxv|5%|8&DynV+u(3u-r@o!Cr2LU4XfdCFKSbIu`XjP$0y{0ub+K`B?_x(=9~EcphYm|-zC zE&QQCh@%!D?qcau0`6IEHdTPm&C=)qs)ShV0y1+f&)tCZaAWTUtn(~c9iZ~E3=RNA zLLLkPi3OGpBjELM6E*>cMV4 zkP5OWJ%E3On^iAh46%Ir0I|yAUIOxjEGz?2$ubrKoNL^iN0JjZgp6$f>2;Qt zC=iTrvm67=8!VMLpx9(_?gIrv{5t^RTP!0fz#Zl0_!!V_vuI9%Dj~McflQ3$_X3bR z-1xJAHO^Ab0jgb=seeF`kk@xWVvl9$0r2i~lPm&;1WV^LP$%U23y?ivslNe{Bsbm< zz<$W`R0Y&2mWgkmOo+=LAa%r|7rsCEkGa`z28<^xZLNSvv&c3ePsnOJ5Itp?Qv=R3 zZtgXJ_MFAi15^lk)(50BEPZ+)c)`t%0We>(D24&W6-#ggC=lX(6o_Y8W-NgFnwwiI zpv$qCCV?s;PNsm&4NJEZka=#RF2MSaMa%)JTb8AHph(D<1t4+9;_w6B0ynt;V7O-) zUIFTaJX{5`4=n015P9S#5&`T*mY+>P{lv1c4U`E{+5u7}7TX@+f9B>Y0T|0H1BZZk zVM!bTc|xkvK=hSm?F?|far2b{v=tW56;L7M!!?k8XBp1}!4GcEZvpcsi|!s!R9WJW zK!FggB_RIAvQh@zHEup$0o^x?>m8^P;_eg3)LBMrfc)VmT?ecU7VR&f`eoT}TKq$i zkWC68A^tynihvUEHUaWg1sIxHraOT;A#b{XYzvD~3q%y$qThEY?Lp(ZiBi0t$ra4+8OCmfaBG?&C(e2IzDwp>?22h|dim)6X&+ z1*D#vLJY7Du$XrNm4W4SA1D&CCkZ45Sz;-`JH$=%31ApzSw02ognT;(vPKr?B@h|m zCZ7fDCYF&KKt0NG^baT#LQ?=zW|qwdz(2;#ZxJwBSQeiFG0swc0rG^jSAeLM<@y6~ zPH;2$1!!$7$#0-S$c_e(o@7}U{yzlm+bGY6>sEdS<#G9gSpAhpDD><9b-Zn~EN<1)+E z3Lt_kLIQb0`olnUh2>dTNK2N(Js`NoP3Hk%4zolK0mV8? z{Rk)!Vj&I0BP>s6fO~@*+XbN8WVyTos)QKG0huk9{X8I}+^FsV>o$wL2UIbZuScLr z$hi`b*kO4n1Kv0{mN&q#%X0n>)Ct*F1+sfA@fr}>=SJ}Z*b^)(zkvFH<-KX?4`o8Q z6hJD;a;pUVhun;|1I84~NhctVShN}-Pspeih#s>v^#RTkZkGB1ZJOnE0H_esF$APf zS#FF#@Qj<`QNVo8k{SaP85Xq_C=g=92E;E|e(Zqzk{jPNpu1uz%>Y$GOuB(gmgUL= z$ZKv43xGApl2`;(H!SS|ph(EIAdtwj)IxywA2*&gz;Mg*xDM0_Ilc*G?^rTXAX4B) zw*%PkS$1~;^#e<50w@zAlmt?bEFUSrU*u-?1Ta3a+?@iV#A40>c|xWyf#@?!&o$sI zbF+N|XkS>GZ-ELS0|g-c%JTLA1mC!sege!Dmi#lIcxN%b0tG@GRe<;hOV=mh{^Vx! z3(!?r8s9*b5Q_~U^TqNk0)LP-ZtN|9^_wNz3aIKVgDRj%$b$|b@x#*51$Z0Wgu4O5 zFUxliP$%TP4#0tG_+3j^_PmXQs>t>xx;3()njXktK>5LA>Oq>+|Dx72e_xWx$OsZ4i=LEs1o922*^yc zbdLbi$xU<=u+Fdu3!rkbELnjfAzy4jVwS}*1$f=uUkhC z&rM_juzOj476J7F%R&Gs6QUFZQa%>jD&Sw_=4uTv`dJ1dfLLNlYyx>gsfMsn5 za4vK6wF_v2ES>~VA>>06NUyMr9|6G-H|HmSd6h+X1}G#;JOdO6(Ru~M*H~7r0e6_2 zj~hU@&f>ZSs)V>J0GSBO=p!IExJf?&)=d^|8BlGpY`+3ULN>hvi6~3p6Yy?x^ZErC zVl2~jpian}29VuhF*XJN5Q%e>Y60xKEL}=Ky~nbl0?LGFbO5P+7GD?OPjK_x4Hyqt zCVK&qWXbA)JRuDOK=hF1U=VPoxM?>6+9Q^*38)aVW(Ly7Ec4?)@PwPk3Ba6YvDyK} zDNDuy6bR8j1H{i*c4q#$ACH^-{L?v&*Iz%BLBF_9{~1SmXQ>ozGFE$2Firc zoC2u=%jP-Yzvt%n0x&+XEM@`m$WqP$c|zL%0nsAM^&Q}R;%4vx(3V(|MW8~+j%OhK z%(DIh1k2ofzX9eK7Vih3cx5S8fdU~WzJd4~%f%1iu5hClD}T_vv+OkkRYJ720+|n% z)iywWa#L*wtW_4b8c=<)+-ra$A;)@vM2+RF5Ac3-)2jyzb(S3iP$y)|Fp&LW3621f z1~-*a!2ZiJV*%9S|HpIqX9dcHFiirfCYIwVz~2mrZYN-DVcBv4Lct>DfIK1n^FXwf zEa+v_CgcvvkGTkiuM}X9Fqe=tT9u|29sCrqxGC+}#b5}s3kLBSS@anj+ z(2kl0}9XaL?#Zo*>q4~8w4?`EJ*$oW{~8fSXebpiQ#$P5>1`#wLOEAxnz`2&TAMb^_)jmWm5d9J4s*fC3@@%>(fhmJuJ| zPIGhY2Xv<_nq{C$h^-YMbH?(!3dnPA{9(YFVJSxd)dkDc7EmPQ^)`^WWEqMB-YagB zdw?O!(s=;X3Auga;sPiW;_?be{bSMR0RJsF`+2~4 z$I^BOhyshe2l9lhJ_6Bumbnt(eBkE33}_!&EN?)CkZ12ey2#R31%gl9?9>2riAC`P zD4tn@zd(Ty?@jU#@iNPd0&u@@bE^b&uPml^ph}37P9XEf(yakxg`21ru)ed1K0x)s zveXY03HdSrBtBUjLx8u+P0k1yzF3AwfjS`%$AD~&MQsHl-`qrOfW6N0V+YhfEDO^> znGmHJAk|>8xdHz#H&-6ODE=ot#lQj}ngEel1oDJb1%PNX%UTd{ws7+m0<;Pi&l*r6 zZxJgw2dp}FpC!p4|Y#=-CBPlzM*jlnR#^7lfGQ!{K7dS!Wwi>(Rc@-a|IK0Dwj&S* z+8+O?XP^E2c+OY25kd$dgb+dqA%q))Fc=KNU=Rj_K?q@6>*eqIU+et>LXsu=4Y=o6 zZhirqkZt+%f6(Sx4qZTOft%`Lz^|~Zl>j+H7I*-2ktI<9C`;UAyg+!FrN0L7Bv~$N z0f!JRen4Mg*=q#itK3vJ1A#S`b3m35^$K9@ zvurK_GR2L11qdCm$Th%y$nvoc*n}L?0PTq7b{mKp+_dik{$rM-eIQ53nnS=mVNnf0 zIpxN63WQCT=`+A{#`1arID{Cu0`zm1D+`EUaMOAR1kx<~_kg%$@j5`7kjZBtdBu|V z3Pi5CnRo}h7R&PokRfDm4oKawT>JouTW*^27XA>tW6=u$m(AiS0xUu-x&if`<+~J! zK5#Qu4)`8f9x8z>A-byp!(ln~0rH6(e?1U-X3-h|_X|r&0I&%;*8*r6md_v%d*x=h z1Mt7G+;svuLUx1!^PR=$1(Yl|zJ4J5!LmLGcs^MQhXIEWGciE_VtE?_;yG>xCxE~= z%k>l>eprGtK$?_%16suqBV_&@j*D~o_P4-olDAVbK>RUnnm@}dHX0&e;?fuM^e zy#=@mSpqs>5n^`_P>Wc~4}fSfH%mu=&&~343}gv0ZUROL%i}p9OSuVO0--V%^BQn_ zSQ>5tn~+;Jpp~q^$QxjKS?nyJRB;pf0>afS$KQabhNV_0 ze{cw~kq_uTmZCx+UdzpFF%YO@c`pG(JxkOBqzP$N07*YfTNMy#;O3wP@HVnk*8v$q zuKIyg6H7r8kO*)y)eHoiSu(AFtA!=f4p@Y^3;}8@OLI364RW*B1NholDk4CZ5X%FA z(a!Q41!M;|@ev>tVsSA=DVqm!gj`Snvxgd7)#9!kQn7=?G6ZzvA7-p*EmbU0W3niJ_Bl;rT-O( zPH=Pe4)`WnT0Vg+A@*~CF~w5(3&;dF$-Kosgr-@93vkb{OcVh&A)nlUHp|jm2E-&c z=jDKZj-|;9?fgNUgJUj01@cZjL5^;3kVN z0k||4bq25qQ8)*vTP)KHKy;g%w?)9W!!nQrvV^!^1q_{~bsdnq+@v;v&>o9-8*uNl ztmuGE$h>_(OR-EI0I>sZo{s?kAxqy0kR#-U37AJLfeSz}xX~|x@G(od1$a(a7H2{eW@L zGS~#j2X3rpAoR!*3<7S4<)9s~30c(%XiqGw-9YS_oBUqD|H3jA0dj5<}twY&axW^970q~0eY5YX&Q)saPvD01U^~D=K=A>;wV6xkl|$@nPWLy z0V3brG^l|0hh=*M$Plu03rPL4EbIUY@elYEU%Nms4-l~w;L2yQ4*`o1oyUM$z;b*F zL|xp}oddo?mW?!!B}DNxU=*>;-T<30DhFUw^S;1Hsv1kkHk_R4^GH8+(N zK%jLw8gpAh$$y%1DdLUBAO;01>t!Ft802x9ywgM?X%T61RXyB$S1Oyvd zlrF&4#F7gG79mFa05!mJ-w#Bax#@}mz802~VIWJ0`cc4WW!W4DWRM&8BoJz2kqN-v z&hjw>*n}LK1GEm7+XWyN;--BG@OQEtC4n3vYt{g>i$z@rlx}WZ8W0Y%Om7389+p=f za0oH559qxtR|i16kDFEl2t-)+PXN)+;ynY>giKxl$pMzUDC=Y40F@;3@>Yn>&q2v~&JT?W(*mhx2~ zy2;Iw3ivdZpA8^Oi197J*kXCy0pvC};XNR&-NZyO-*xQT~=kj>(B0q%R2 zo*uv^6`@| zLY&M2`YTJF0>t0A*<1nw?<~bDfXK4QH6TsM>^hMAU>VYY$R{^9+kp3rrF|F35b|&z zNaa{+4uQlsH){qE{9$pO0v<#Qr^C z6tGk}fOK(_dlL~6MC2m)Ro%WwyfA>?f*kg8?r2m^^aZjO3^U_FbkA8`3u)Iq=^MBy-?Hn2>` zfM_E(Z)1S3iDh5{$P(gu3NQjJtuugZ<|ZY9Pz#H90dTjntSkaHA@h=e7G#-R1!8U7 zJgb1eouzLR$Pw~l3otua0y?0CxY753a3@Ro0pRIkSv&$9Li`*9dN<3M3B<$PJe~uA z9+vJ)K=iVh*Fc(({#zi~$FgMukq9>>kASzIW&R1s5c2Z{NDZ)zya9nS! z&T>!#*o3UA1GEX2RX-4$Y$n~-g3K-*?HyaHl7+*IEHew}6Q4#*L*-~ljqSrQJQ z>~WKM2EzL+{jY#0#d7%$ID}~V1oQ)zy&Mof!x^7X1=% zR7Er%gzHK1-!_C+u;QM8HcmlG7=*|F!_(yz-(>Flo0mAE zQt|`Xgq#!Wf6!bkpZP$nkelH`z+c32R}AC`*-;9Z#Vm#gP~6=3DuHkb%X$^yDP<}2 z0S+N%>Hxir<;@SoJ=_d70fBOs>t;YyumpoZnvkh>AX&*$)d@tr+^lp1-YS;-ULZrr z$q10DW_cL^5;ffP4FN$POL_!w)v^S}0E-a2aX_tODW3wO_1r8?13o{?&n%E7#P~d5 zG_X7>fNbO@ybOezSj-i`9bjot0h^Fp8-Ui#Qo04iTDY0t0sO5jU%Nn#kRvI;46@jV zfYQcI=okpMvmBoSo(`7UbHE|QMjFsVEJfEqypx;R8z9id@_q-1ZkFf+kS3(%0Ld^* z+Y1os;pX5K@bAUwr#-w${aES*unA;ifrpii^ZjRNr*ZZ^k( zz${DgBp@VQ!+$1l6&^n8_0^A!c z6Bb|-^63`PHd%V_ftbe4`6J-pVrhB?a)jK;0OmG}=N(XXxKTcUu+H-R1$cH@Mt=Z@ z5D#MW5BeTUR{;>;=jOB!2&7o*-GDe?(Mo|dA>HLb@{nb&5{Mje^HBwO4VGaakRjx4 z9gsR^>1Y5FC)^x00l`xiUkl(eS=1n45u&gIP|sMVJAvpqH*ej5?}BBZ7swLgIszDJ zmexT)UUHKf0zy|T-WcG%W?2~nY(nNu0Gh=zIR(USxOtui{I@K9637wqVjeK>SOSZH zVsoP}1L1p?@>Rg|z_O?U4k3Ov0R53=Yzv4x+&u08fhU&kJwQCOm?L<(k3lRO{ zrsxgu zHVL?@S?mO05u$SzP-|F@=YXh>n>q#X)v|0X0a-#6uK-3J%j_B;>$%CU10g?4R0G@% zEH~SLO~|%gKx_kmb?> z9743*0eTzD-aQa+=cdvD0v#;N&wvQAh*uy@$oM;u>|}ZR03u!7^yC0e3)B$KCEVrFNEXGZH81Rp>9Q6V@Le}&H<`|1Q z2q@#+xQ2mnoMk!&cqUk0#{h>A0~3Hg$#OLX#HYAvodE&~mVF6`X%_DSkS1hu5lGIk z;eR zC2qz{z_-lua1LY%(R~RRNtV-VK(27(zXd|8ESe3t*H}s(0h^F>Pk^Sfe7*p&b#8{= z0RINdT^7g@vf~RdH(88tK+(AIiS0jxw^-Km0nau|VIklUVx}0-cUay^fVj@hpa%%- zvRqdHVvi+Q1*8d?ssWPwELC+tB*o2&AMhTqZ2OIS-zHl93e+m0MlZz*8t^)o6rUjzGXSq0M8vu?GE4& zVq+K3ZI+@G5WnYU_7Dg>u)G_9cw~v50%<~8XF$?nX-flKU;HyZMOiJ7BjiFoVCJ#p z8UQ69kg)&|E?~KD0X#02&Njdy#7PIB7qZlK0r4VkHp4)mn5DQ65N;OP52Oj19R!jk zEJMRUq?DVR7~m~qX&(nNggl%8QXZC?1du4_W^D!tRREbEfS8|~ za})44uryr&IYRDS0%jwN#{!flZj@Uf9ANpj0Z%i_=p*0|;^7IK)jWk(>EXx zWU2oEL>r6t1*8e-{sEHhEOSEtL!^V7k9@!zVi_(3GK9P>22!0Y9i>2`i<=`45bS30 zRRXRsi&_O(gedd@Y7fhF9T4s1=FJcI`d9{zA^j?loMPG903r!)O11#+G|T)BkRjyf zE|8jG8A$<&S#ItQfuLjw9Rsd87ULAK2vK_usPioAX&}15P0=;rQ&?thfGi>2?*LAqzCXyktpi1IiUQ865~;v-Ix+9*gDj0B{J=VgULL%iakPzvZU#3<%t@ zEMEY^W)W9Fnvih|NZzwN-2#yZZhGzk?<33kBak6v<1>(QSavc%;)$EGcOdx8qI>|Z z7na-?U=d>U2T(IC_hRo4(N}J|3IN|5%Sj=SB}Ba&Fy2`i{-Wsh~>CxZvgz?EJsa1j*vAifce9s1_9-l8&?Mq7XN}zG2IDx@&NhT z4LF1t=mqqAma7O5FW{zi5D2(f_J;sb$l{FwX+kE)fMgL%-UJXS=4N6F@VZ%^r-2M1 zdnJ%6VY!$G5~bWUEds$Z7JV6Td00HFfJKN!6;R7rzBhnq1vg_`fUlC}VF$<(qI(Z8 zyey|FKvr?%KLSG4EZQ;Pu3;%L0h^F>=YZy8`Ah?`T5g7~0e>CK-3^clQm znI8rGgDhX;K#q_jlYkjzu@itY#7$@x2oJLy&jFqhmRbdH2(hsQ=rNX}6(By!&FmTw z7-M-~2gEo_R0Gn4w6=j{oTY6Sh)i&Eun%}AS*j0#3?WwyAT`BOa0(<6+)SMT!D*Jv z1>l-tiCh5|AucUIon>jh1EP|fy?ekn$5P<{Swbv71I9edZw8PH+{E93kiz180PaPW zo*ZBk^6UrDmRK6|Qh$gobF*Cl_>(MUML>>_3vR$%Vab&O$|^Upav;3Ma$gB}RF=+a zz#+tm575_H>gs{`1~;1xKwy)lH~M2N*D&DPXGz2Wix974fSO|Ip8%o<++0lozC)Il86ZoDeF+#x zER_p@G`LAF0-<9Tkp$c)EEB7MO~@w|&`w!;H-VVR&G{DKKVxaqfgB-s_5kyo#d82C z7u+aEKse3veGGUmSw>C3A;iNupkJ|cT>|lIZceX(fW=aO3y2#Q%?8qhbUy;gTb8*e zAacje#|z-KS%%+$3?Xl`K$X9MsH9+W%#ajos-&t1tfKAA}CP2%wOg00t4{n}Y z0skjUUptT^7NFY1uR>$K*YsO$vogKWSLih3?V<4fK(C7$O@1s=H_k<2)bE9 z8-S~X#n1qY5VbpiTFSD%3q;GfDM|r856jFUkR`;s0T|^hgQtM3;Kn)wLX|ARG~o8K z99#i5A**fxt%_y!4v1BAlm7ttYgncnAVt@(%wHZ{X&)00=a)j28i-iN$dPX+nm}fMkH>tQ?3mbJO4jye%x-)j)=j zrL{n+m1UtGNCdh0Y5;<5EU^IKYG<)q0E-ZvZGhUra@+w#L)_GL0lrR_jWCcUL~$Qr zbg|6#1G1Z&>>v;dvqXmhcMr==46q5=HV$aLEQb?7tdE=O1mKUbtjz#9LKe&cWZQJ0Pi@*$pH&Sn_&-$R;-v5x}dlJP!aFLiP>;sV$a^5g@V6P16_<++oq< zfJOP~a>2(eoRs1GdV6+rZnnHX)y;0j-;*R|2sxH|O(!zlWu15y%m8XBjYiSv;$N(#MUW0^tbD_XgnUXBpiB z96~(o0Qvw+*B%fbAgB9A=q22O=Zfe53(yjAi&5$Pn`O z21t#vbl5;*jGLneAUMwAdjec>7WD{soK# zOKbkgALKMQDHjl$Veu9N?pc|PaB{wvy6p+c#@mPE+DYN(%l1yRTi@kNE6aO03_E~ zwxU2p<)&l=@UFAWj{+G&evSjF4VIBfAhF5KT>=PdETLJzwZ&r00Tv-@6+qo)SziL8 zJKPkl06v{%W(~*^;(Z-3c3B2BK<;s4Z3Cfwmf$YnPO%*912!S64gu|eWz_&;huq|! z0{$bGsWTu)$d?PiG*}{6fO5=D+5*BSEX{X-=agml9&iXz;Q+eHvh)nZ&$#)`0D*Is z@pnL6us9z;nvmffkW90j{Q!|mZW{7V|KPo1*)9Mwge)xrQr9dCZXjWC^HmB2Z&+gG zfa{jUt^_PXbXEiE9m}x~h}zuL)dRkJmW>7=ONinCU_7wQwgB>xn`{sWIV{l*!2QH> z(+SvwYzqV0Gs|Hw5PRXKx*zaoSk?xC93cya0rQn55d)MrZZcy)_?@MH0`O#6E~fy8 z5G^x+{=u>*f%qpkl?y=Ni)DEc5IGi+1k!|zuL8+$mM0a6{BYB=33z{5&bNRJAscld zCH@_sVrLIXnm00`!@C`W*+fF*YfScDig0oBEFe-1K$?(AKai|r z$!h{4_1sJ}171JNb1RS`WN$l=YGAnt0f|O#n!15t6N}yhxB@Jm2w)LnaR5-8S-zt{ zw1u0o5y02V@-PZy3DF$~j3CSDBp} zmg_S>46p>#K$?)LDI347+|2d^fhCsrK|n0CM2CSiA*~pYOtQ3%1CbSO4kiHaDob?&$PjXM z21u>36wCn$m7A#rAh^zwSp-}gERiH&5#n+cP&Zkc*MX?U&E6*9+hVEM2C{@$)&XOi z<#!K|JKV$%fRN7O90BfKmYx&9Cghn3XnQP;7eH*Eo9#=$pJFMqfE*zgZUOUvC1(T5 zAvdu{AbiAf{{(mpmd*^|5aQ$w(2rT_K7jZMH=AET;FP8K2M{KU6z6|P6Ed3*B+poe z3W3NuH#fzA_kyLp6vzEH#xt;*y)SDj;~p;_?BmYnDVEU=iZg52zMPe-jYB z;pVCt@ZGYs1c59e_S*sDj-|2_kTy5TZXk5eB6Z+@tp%M z@gH~&Y8tQzQFslgc`Va6Kr|l^Z+C#NfMwtT$P(h(0Sp&Q>kA+YxkBR0Db?%UCTC_j2=C4+N@Mx*Gve&0+?CG$H-1K(dBqs||?wxG4z%-ddLVE+9k5 z&oGdxV;SiK67}5N^#egaODGDs8d!{Bz#>HLD4;g7td9fHCT@x*0bhV+CIMs#@je3> z%`Ag+fNbH$S^z?=EWstf9b`F30yZJ5)&Q-IWpy2hwR4lN0san_scj%f$QK#A8fLLY`VST%Itu`Ggypyp zh{m|7a|6CnmW@&%ONiofz!+nhtpwyaH`yv66laP00QUsTO&wqpvaJEoCRq-ffY=l_ z)h&QO!Lk+va)d1C0L*EYL?@ukaFgiy?{ruTt)zg5G{j%KF6{*1jOgLsf+=E z1(xM8KqxF?0!R}wJ_RHfS)QhW$Pza_67VjwoX-OpLN+b}sU*wJGLTr|rfd}muCgd9 z;96tJZ2%S_Mz;V}Wx3x0qU+pr?E$_GmXj2aB}DxZU~IB%9s^S2#%%(jEf#qWxVKq8 z(tu6Kp=&_fVY$5lVmdeNHsIf7IeGwcgsgc2%sm$M1yJ_6alHZI6w7oL@EowbegY05 z2EGCPkmc$Zh#zs&nt%C+fWflw0>m+kw-`thGFbv7PgwFiK;)F0i3-4LvOIf%3?X}K zfYcewMJEk2i&?%(fgB-6$^p~OVpjr62{)l?AY96F>;pVyEVcE3Lx_zAK=-f|1%P-tH?u83 zpn~N+2#89SXa|rcq}2%|y)12EAX3H6K`-E~W~uH6GK5?m1X49D1;aqX$IVm>2-dP> z#sF6xOJoAD2yr*IAk%)fixlgULbkIvQ-U43~owl0q-%(d_9mM?plDL$r5S< zTxTps2VfDRwhK_tS=Pfq^n#nBKERh|ndt|zgm@nWj7yfmVL)DSW5s~bHA`?Da9b<~ z6M#*~ssy0ju&mAiv0HBP=K%j5%hUppBjn2>VA?E^B%s`LlU@bF4=l~=faj5AcN1_3 zQLzo^4$G1b#GknN-2(#8EaL}&cwuplfHWb)CqOd8a%KXNS8f_E0Ph>i_9c)ZWT^$D z-dPrIfkc*@FB=Gcu*4n#*C&hp1XzUV%mC^a%kdi!&2dxr0ry#ehx7wo*XLXF2o$u>x+YD*?ZYWvvRx5wgGsn1w8f zIzTDnCgTUf#Vq|zfXB^p*$g;@XbA#(3CmtP5HIDXvJ(iDu`G83!ownZfixlG5g=L4 z@-zTMD!Azx0=$(h=OaLdkd0$N%FD772NG4>luZG_Y8GW0aMiHnW&w*3qw|33W4Tv= zXe~Eg%Yd(rk1{S#kxEonMb^)7^Ln%OOV!1s8VgYX2 zj{$!(%h4&2BV^4vV79QRX+UY^#&r#ZgDlfGfTxY+^$u_dG4KHB?JQRg5bxlo^#uro zSoU85(aGY?0%<}fKY?TyOWrpS>E>qQ7x0Ezp7U=1kRfER3rO{_ToeI`UT&I7fM6es zUIw@#ES?I$BE+H>Q2SZFtAXeMH)FMcZ;<7o9>@}+yAd#=ET;iL4sqje1wz9tS{vXV zVJQg#HX-M_04>Jy83tmb+zj^t{xO!jejrE4jwoP`vlzpG66eM@3WO(E*2e+QBun8W z;1FUa0q9dKZ! zZ6Gzz@}dKY1#bHGfuO>YJ^);cECB z-+|Z$H}fBWf0O0w3&;_2Xvv0N)u)MGVLiVtEWO&RKrrfV|))J_Ur*EY39GzGUfm8rdxPMqC zDgc|1PhLR#W$CQ}V&cE?Db8yFe;y#3{6LP7JB@&u&*EtYlmc#)Rv_$R`ECO|g)E~X zz#+s#7oZohboBu7Vs1|RfPkB&egF_9ELs#u6Vg2bBuiQ5MuA8fHy`7G*TXVA31kR) zn*dVfEFH5zqJo>FIUrcc;!^;ZmqlFyEJ75n0BRM>^coPY=H_i3@YS#kXh4<_*V};M zV`<$5WGy$TeIQiF;ync1^(-p}U=uR$6wv%ElV?Dzft%+Gz~9KycLn4Kd0_!&6HDL@ zPy*cO_dvLrrQ88LEi8-AfJ2C%44}8NjJ*T#AUBU6K%kAKI|qn%7V`&46Vjh||A%A; z%T@sp32{?W1b91H=G{Pske{VMs*7c$97uF?b5{uj!z`g{z}3TI_yCI#we^77%d*}8 zMEkfY3IM(c%S;Q9CB%CWF#1^rI{-PrjnxT+23dk(z#U~d=ml&-R`mng5X`K5PM8yK2$61yZf%pVBzso>i zl4X1q5K}CU3Zw}c-UN~fma{D&GR;kc4tQr+w)cPxAxjT{)GW)w5s;AFd>sS9IhL3S zxaL{xbHE}*=Ov&nupD0lQH7hjTfn!-vS9;RLKHs&#uCfy6Cjtl$-V%gBun%SaIdi3 zWC5FyZC`-4%5wM(#MZc}77u^$t1N5zK#q_Fg@C!vk|+k04Q?_eKzNg--vf9wmdgsj zAw)|Rpl`A4)d2BrZYt}5zz)l@9}qf=XadrNj5h;v zY|aAGg@Tc{Kos5Cf-xe#>%o2E^~UX-xwGn`QqB5ce$J8z4={C;gZj$yT>wPi zxEU(~eD5p|ZXipD?lQp0vYeI!@`D?{7YKc_Xw`uGi>0I%un9R=4`?}-&juj&&CPHC z@c*#fwE#IncC-QJFN@IuDB^$cDSTZ(I1do(VZf8mQrHJLgqZ0E^a7T*K_KqpW^fn? z6tY~$08zve90$^bOicjEVwS1|5OH&}G6Q%^Sn}t93?U~MfK(~V%Oa2{sh{RAVhn_1!fA<)9|o)3stmS`c6CZtsiB!et%r9h;On*$HvZD*;j1TutN ztpZXVECoIw5#na54hVL#Wc+}uizU(oScJH22Gnkr<{%IabFI)N-9mb(F? zm*uwykbT_5BS0v^;tT-pewLmgz$WC`2%rtHG>!qWL2kC=fIrGoHU;DexiAfwLoB&j zKpEyHHV=eHSnd_T6JzOI1{^}1tN{8bOPvbD$GF+t00QGI#an=gv&bDFO~~vnkepx{ zN&%5cZf*_%?-Wb>F_0nT;VF zF9Q54EKMaqj*vTLfVs-zsQ{EUZWJ#NR$0EQ0na+iXf5Cn;-MbUH(0tFf%qmjrvV_K zvDCK$Vv9v<1JZGc|OMpD)CY1z2CoJAIzwy29rEed|5%S^yFfUjF2B4(5(NBQzB}@4k;JIR1yZ{_R z{9FS1HOrU<#4TZduGnAWcaBGmyMv*~$PBo12n%z zH#g}B5dL9l9t1qUEW1O1Lx_qPpo{;9Pq8!x#Pb008wUdUEaOvvC}44>fixk*5=gpO z&gOwgAvX<+fVYTcdl|?OvUC+l6|*d;K*G(<*9H(QVTo-4u2L3z2e1gyxd*6aEXOGz z>fxsD2=JA&Y#akwLKK^TQNc2M4#-MwvS}dXWr8^9)Hn+<5yEQb$3tcIKF zC&2GxS$hF;ge-Ui%vzR27EtQA$$SFgdY1lgz~g7R`~@6BwB*11L2qE$a{=*2ZYqm` zKoiSy2_OP2!ULoU8Lt46%`8t|AkxB3PYvL0WjU_}GK6gO1F0a(P9u!#7fGi>E2LPj+WitxMFgNZIAk@PmM*(*) z%f~oi6LM%0(E3VfbI%X$OgS!F2<01hE$S^#~GIWhkH!FjHcZ(%|7|0NEG6tl!Szg9~#11!o6F^XBNlyW;U6#NMU=dZTs;b#5_ z@Sn1L9RoQ+j+lUHve@Th;Q5(uBO9A5*T3zphjz#+tj4d`i>qDLTp$<6E&5V&G_ ze*wfbOY{v$6Vl28NsFcJ3y9osbMOs#Z&|9v>mM?NT+IhkcPs^kK*Hu`su&2~vt&vD z*8@w$16YK(tN_$UmgXuT>Tt7H1Nfd;D(ZkNA(s7s@yznu2*?+1;>|!P!{W38?pKzc zcEBd&SqRYHSQ@*5*gH4dJ%B&UQWgPngj^T^%nz1a6i`07iH!i^FP8gJz>{O?i~|lK zP9_2Uo270Vi2rc2IST}SS&HWYA^ty}gH(VtA+t+BGLL0w1&HJW@@5V27O=E$02x9a zYCy`xQnLdj3b|R^1%gE^t`y)ZW=R|Z79m~@Ky|b9p90YmZm!ON|0S`$y6FewXy1=L z&TH7`HJ;18_bI!b^61fngOszi;U+N$ziEE z142S=^+?>1vS`katCy*rM zdJ^!8S@M2?LXAg`3AKNAQDajyUyWh_m(fK|@TQ9cmWSjtsERIoV3 zz!@QxQoyccnJ5R^tGM~70MsZ;PZW3|=vA+-$4>Mh8p&8j#n?GQ9x=g!tJ4Ho8~_ zb^y!d=3x(rcC$1e0-}e-^MEr#YNvqR%d&b7wD)n7?E`8*%j7liLdefsz#U-eeE{Nv z+}u0@y2TQE0n&$9_TK=X5Tzf$;xNn77hsNXllcQg;w)q0{SP4_-cy17QI@U@K#pzC0NNDGNEwhM~FD*uLE+Mn{X3|xh$p&C_5}yyTC0W>ka^Cmu2?|=-cC__yo}QSr*QK zBq7r-0q=k%egz~Bxp}z(j3btgdmztaIS+t<5H-)h#xcv*D`1^)Qy2o#Q*d3fn}u*kO4PJ42V6l$RCH;W+#8p4uSM^mb_!YC&ctAu$aN}eGZtJ+zj|YB#Y(#8VCu|dGk}E3 zO+*2VB9^5bAg`DuBOeF|F;)m{l(4)N0ak>Yt_TnY@-0h|%itpe2h<9d)CS;%ki(6DTgmd=48*IrX=?@aD9c$pkY3Fa?E-v4Y<2^SH7xmkfLY7U z%m5J4S$-@aB*fqduwTdWFbc?eZVU;;8d$tZK#8$bOar%sT%84+MwXm;ps$IW$wfeG zX8BqMl7#GC1w4c0b{$Bxa1+}Cj8>Kd7szX4Dcu7CLM$Hu8|^Ha9$^`fRsj%;vji%jjIy+p z0JnrZE(M%1mP!ri8|P-N63`?|ZZ(i3eKEk8V!3Mq@)9hKEkHns z!!}@Jnx(80ux7YfF@fkTOI9x+Y!=xMoDnh{1nfDM?qQ&Po|~&Upf0e~j{`4++?@d2 zMV68T5MSbEVFu8bS<>f#^c9x)0^k$kbqQE>SUMcQT;=9s4T!9<)NBGFA-1=H{dJbY zT|jPdWA6j8O%`zkC|fK;$G|NiA5Q^io2B&v=ySO_@d0gzrRoMq5_0_x@OD}90wA%+ zP2veK_F2APfV=~i{&yfC#Qg`bamdn?1gs-&j(&is$5Nj1^#^gx;-mp*gj6yC`-Eje z0oqTw`N#p(GnSrw;DwOag@Aj`(ohVQVDXA&v<5J* zxp~up$PG(p0}vA8vJu$7WvMj)dB=_03dHVN)DA#-V43RzZV8#v12_T8a39e3$j#FL zpgpm)4FO3)o{j+CGfQ*~NW5^fApzr+C4UOYdt;fN1_DC-%mN$lECcg^6>{^i2t+?v znpXhv$>Oa7XN1%?0Q-w&bqi=ua+AFSsNXD;d%z1JKMw%+ho#p8;=kP7oB+D`&$uaK z=RkT2Aoni;pAe&)Kv&^RgeL38uW&m0)%SaZGB;<28;N`Kj=K+a)Zq5n-qkyHl2*@jB z*(?D9LKKt%8!F3;23SSh{8j?dVwS;bK$NfqwZIu6je5Y2upGyL_EK&tngO+pWvvBx zA!JTF;FhyYbpmmXn=cd4D_Ht^f%Hn2+kU_&M56^PR3tbCdfE=shfnlhFLx;fLlU#M*(Mq<+=vw zi*r+72WX=#dksL6kR?rkH^#DP0EuyK(%S$-vW#{Bc@r$JT|hvH&K_W6lI5Zgu%@`F z9R#8YmhB-xO#kQq_`m(HLX87HA?-21nPEwp0IXR+4ov~N&GIx2D03{WHV_cvbRKZ$ zS*n%*xxmfF3Sca<MqOD z7Z4IM;|K8eSjNQ9AMAZ@-ckYcfTb$~&<BN{@mPuq0qdBXxgtP6 zVM&bu$|=im84wWSSp(cNmbNNDo^x|n4Hy?JQ5{e(SvKo|kdXO}fakN!Gz0b(H$N?a zdCfA|4rn(l51oLxWih$|pO9WJ;M}oP3;@(z)Kd|JC07}3zISK@X_!87R@xfxcfKNz& z7jQmV>J9+wi<_MzKu@w1p8(1?%lsJ-5F+goaDP}vt^oPV&C3m7i2s7WvHc!UQvh)u z03ji(p8+qGW$P8N)3_-J0W+Ot_7l)DSbmd$$Yim80iTdT%I_bXES8pZz*4w5&II&q zmdb2E$zfT`1p-3k765K8%ajVpJZ_T3fRWGAR|=>FEO+HVNXW)Yz$;`qi~_dGO<65q z7O||<0a`IjRtyj&EV2pk2^kuI6JhCY1FTYRt~vm{jHTWLlya8c9v~n@Nk8CfEDM8x ztl%bn7%(bX;&DK&VtE|{LPB;-0A7^kVhXUUxv7}}%o>($8_;T53KsyOv)D_3Pe|ba zP94k88er9P^SA-%4J@tOfD&Ul*#QDVRP6(9Bg^_BAe*?!I|htqmc%Kb8Z6)EKuE}b zAMjdO?ymv6m7AtJz-(hVdH}R`mhvY+bg(!tfKN!}9dJ5XCO!bGi<^%xKsQ->egLJL z8J1=Vh*=hI67UJBO#_b2 zvN{V`bKGRl1NuD6%<0e6w*W*v}A+{Cs3W0_^&1=JOm(mfy~}YTdC&%U*DPh7fPKTwvI&^CELpvPcE>W_4~Tn~a1ihbX$}L<1Itw$ zumW!C#sU42Wp@Hlo>+<#KtPCv8NhvJNt*-Y3peovz<6bOSpw8ImJSC933WhJZ5PK28&1Weo2V?;^ixq%T z$dVofRF!441_%lHssmmTOJ@UM7jtvb2$&@-wFaO?ShiaMQOcrr06rn@F2E^cN$CNs za&CtD09|8w8UU0EmewI4AjIhi;8wC!jRCTXn+*vVQI@kh zVApbUzX+H*OY;h#)v+9{0-~Np+W>q*I$MC#z>>WKSTSxU_5i(+!ZXZiq9w7U}?FHl{H@E$OF~t(I05!pKFbsr*EFA^BX_n=2z@FhI za}qFTS;iB9X0yD{0Ah~CoCAD9`U`+F&r-JxSPR_jIDo#$QoIf*ODyx7KtPBz7jTzZ zMs@+Y!p+M*U^p!8M}WG@a()bigseUTyfv1s3&39Irr-)NH&|wG0Bw`y_YM$SELH&c zgbbblXPc$v6|h`xj^6=&ho$lpP5gh~+RJusv?dRKPrDSt$mz6PBz}K%BD3a=<5KSOGX^EZtGSI_Ku92GB29 z>gxdIl4Z962nbQq1h_uSf&s`YZqnNTmO{@mP6q?iDlN0y-}z-EZWbAVdQQo0C)gj`w%Je?(T6|n2L8CwU;dY1Q1Kx<&>asd%z@pl2Akop1O zG_vd*0aggA>)8_@e$)^Y)* zpCzXN2naEy0`36IS1}+5x#=qf42$Kq98iZ?8Y_X2kOxt~8)hl11?&-Smg@jB&XN@a zv{9DvCP0j_ga+Uf(rg2qah9tNz>?h5nSegQvfBeFlPtykKtPCvLBO42NgD=af}408 zFs4~v#sGDOrDFmJ33)yRc(W`uGk|S#vtpv|+)E&*bJML2*@NNWvn7FixQ z0BebxmTf>^W;xjblogiBeIOvj`XS&tEV;*kT;(Ql3K(lF$#XzmXX*EWkdSxRfVaWY zbO+d*+#Egt<`zr&6QFIgth@liWl`P%pOEqcaCTTez5r{No9-V#-($HJS$|OWS?be( zfDn5bfP27Fq5$%co5dW!IATf92UL$`v=9gh`C0^c$1I%@z&_#Tq6{!kS!yc)?Tlr+ z3J~WkY7O8M($)dzf+eK^ur9e7Y6Ntj<*6A^u2@=Ifq)RF?SOmDQq={>8*Vnb0ppe> zuMbe~Sf&SnkdWUN;N7zfi~#lnH}|7}8L%`7Sp@<@+^hrcJ4?eBAVY5UUBLKYiR=ODC(F_S5E3%O1H3Pmu@k^f za`ScunBOd2mw@)ea(M-aUl#oq@CoVO1CIDVaRn4T0#*tj=biyQl_m8JP|{e2Lm(i; z^C#e@v$TB!GJ~75U%<#@iKZ%lP_tMz(}9qX`B{Lcu*_rwb~ZOZxqz9&GFSj;xhxMV zAo5s@62K>zg!3Fw6^In{uovP{+j0U^HX0Jn&xHwMUJZf=_Zql6{a z0;my|gEk-}WN9bhm9i|GfL+E-W-nltvyArxn#S@z2#5+6a~SXm>Bj-5lBI4Ou&TJ( znE>=COK}2Fs#)e|fPfHbbAVgJGO_^3T5evJ07GYKcL24H<$MhY30b`fc=aq>+koA` zO~Ec;##mmL6EZjjoMx7m3&1kCIrag)g{ATaP+D2m?tp+0xdGs| zu}nPyvYngc3t)7x^t}UWC(GRj5E8O633y#Bhd+RAa#NO){Rgv~WhD*JdRVeD0ny7M z6~HHCm;*R{EZzBl)z8gUA)pVi)E5KFAj@t92nbP94!9P}LIog)xJi!!#xP5~22e*> zUUeWOWJd$w#aS*I0eh628UrxLShiaMZJedB0}ztM?gD&5iXOn3U>WKItVwPj2LOGF zrF95U5-cYpKtPD9F~FT>S(kvE;U;eiFlJd2(|~HTe9r@PE{ppO_=HqN z&L5mTmbp~G+UF)E1JDmxhO+?WkmV^G2nf-Z2e?Nprv-rYxQP}4#xcuA380>^< zA*VIKJ7xK)1ne_z2C4z`oaLbw&@Nb->j81e;>7@;kXkd~_$;d}fOW-9b~~V7vrKjZ z$_>k>2?T`b?FHOhmYaS+-fSh7k1EuUq)91sO8VFln5(u@L5AKz#8Ocs0`38mM0BRhFDsw zfPfIE)qp$9Ql$fOgqw|ez=*TtH3I4=%XBjk67stR@WxmM+5vl2*co@Hqn2nm_73U~`FW9xvu$j#d(U@ozAxq!CJa=8nL6&C#f@CoT20gl5`bOKnb z+{~Q;`Wj2>C7`Uc3||2OA)aplcY~$v9*~>doCScf#S(o6)NPi{S0E&0eh7Fj%giTW z?{M>z1k7EQ!CyezV|hr)|AW|PG138_kX|O>9I#Ym1J)rotGR%F#FA40C?3nC3Iv4s zDhAwRmfli8o^W$p4j88_u}VNaV>yTdAt6g^0q>k;xel-|xXFwG<|WH`6QKDl?*<^Q zSj;xSC#2s2IM*z7CScugv(p3Uw=Bi|fO5w&KL`YbNE-&+dzO(nARoAS83T-frF{ZW zA6d?)fRK>YGl2KRvSkDIGdBebfce5Qy98*jEWay&cw@2F0H2V-2H?E2v}^-b$j$K% zpntGb?gPpv%i19j5F+;&aKBilP63(ZCV37R-z~K@8}9(`m*wyQu*HAH z6;SpBm??l*c>%OkmaKO`q_M~kz$aw*1vu#}-9Lbp!OfK@_=BFwQlADWSuDF5KtPBR z1#lIXg&aU;bCaGA7&$EQLO{)Bc`X7$LUu#|FOTJ-46yULsi^?W0+#J6Kr3V^tO10| zV(WlUNYMZ|MJz*&fK|-RV>6(au(Y-UN`&R49S8_f)djevEbHBXEaN7x4=~DE5(9v$ zv3y%VNXY&Xz^h=n9|i16Zki-uRRndEY2+86H=K6oEnyiMZl`%=3^Pq zb(Wr0K&fN7UIzj~G;9HGJyRSv&yLMwSc@2njiQ0(eaYo!SAf>SQhN)CRu=aj@Cm6t0!|yt+%sUcbCdE0=p8J>A)s`!JbeNIA=44h9lAi^Hgq+R>yk3@{T)^(*W}pBt`&k}TKpS9bE&;?Ki&qNx zgw!;^u~=3s0c(hx>}o(CW|^!7lo6KCIv^lKZwzqbEH_Pn9OWj~0vKZ~`)zmf)9Q%N^&P~M)pl`6O-2uubOHKd;gqV5)+%1-`7eH=v z)AtS-F3ar)pzg3VCV`NU2S0$f%Tku2{=wekW;qQo_gS(s0quZgTmi%(OPB-rgf#O3 z=ZNL15U@OM>WTsVm}NHtC?_n%7)g$lquWl4(y@{F5!4PczJyy$>>!P3zHgoHeA z1iVX@8UwI>Znjzh^NOXg1JJHnX1f4!!yRvv>;_ zzbxqwfGYkQ-ulr;ASC4LGvKALbiM(0Dj+XHz)WMQ{Q|UfmhEpqWU#2B=np<2?Nq?Y zWJ$>YtSoMZvH)FSdCCTqY?jtMARxqP0pRAaR22a-mz#|ez{q3CD+APgmT3(L3HeXzGkx@XcWLX*qLPBOt0$vr%SOT!4+`P>IW;ILK9H7;( zTrL2jmPKC%d_uYo;OHzx>ws0q&D?rnYjV%7H)p-0JD{4FaWeRmWL-mw6hqmfKN#89dJ5WDn0?L zlbh8fpm(w4`~r%}GMQ5Rhky`YX@J|!(whm$9&Tglix~xcLi#m;Gt5#~2UsKA>@)y+oTa!4P)1qi4Im&y zS{vYwv5a&8a-5r&F2Im1?LB}x!E)XQgoLaf1iVR>ts%gk;-(-DmwFpOC>c;LNhL*nnkob36~|b1aohfHKdrwgLo%$Xx^61(vA|KrV8V+yaaxmcAW8 zU1qu4142SJ9s=G9%b^F@4mV||fVs-Dat>%~ELlDv)>-5=;1e>u1)L3*?gzlyvwXJzAtC$Q0q=_Cz7w#oxoPSK%o~=YUO>BLDIWmD9gAZD zJ|UG6z`19c7zL~cZa&5VJz(jX1e8aX>jV%GqG1+rpIG+h0Qt;KWDzi4SQeK7^_3-K z6$lAAx(;}6EN`2D{mxCN3z#9xfcwjG8URxKcU%F{XTV4S!m^qRSlQfU7XW$=%cKe@xh$W>KtPDzQozk) zxhV%^J~y#Sz$jqZj{<5TOKB|-5^|{y@Klz}7+@E1Gu8x{#Vqdzpp~$6wE-f+;&%W( zA$1dQN?CS#0IQ6fqJBUxXPF-a6pbZy7zhY45(nH0mgg}*R&vum0T@**XH$S0WvQM4 zLPBoZfLF~@umIRK+{`QiW-ZI_3ZUsMgKL1OV+l3@pOD5j;MB7m?*LWJU9%ETM z1e8XWoMRv$#MCL^HnDu21G1T$J|8d)mfLGUZDDD=142R`JOExROW6}(w{f%l0+{VA zS?_?>!7}~wFEV~(i(!)}$00AKuasaoNB`qJ2 zecZ$g0i&Pgr3g?5SUMs=NXYXtz#C+#sQ_$?o2@Fq9AYV~0kmP3Ssf4~ETRGM328L~ zPMqbj8L&pVX=w%YF_x2dKpAJL>;eKptak&hWXbIVAVE&HEu4h0CSzC_7>1K zShnv0vB{!70zM(_XTaHFNqGaTZEl7_KzCW5J^^KirS%&K2yyxgxVtP>sbznVd)#cK z1I9i}UKXGpuuNwIAtAqW0q>Aypa8IsxVcvW(_?8a0kmV5qf$VeuxJ|K6Vj;!oKu$U zYQQ?31OO}QfK>FP5w*kf#OQaJ}uUVE%AS7f)FW}v< zjP(QdEjMq2fO*H#H4JF?ESGUWJh15FfKNzw0&oJBq6A<)ax*sr=ua%EbAa;9GQ0o; zgm_*8+!vNM2avDaoUH-I8%uN(P~TZLw}Ft5`MZD@vdruQ_6IjVhk*IXGI$JVUn~!& zfJm|!7l2Pl&j*}umWms|`r&5v4$yyDasogR{{wIR^y#Y{*Sk4=PkdV~|;1#oM zwE}htHw7Jl8DW|20<=<=-)=yZu~>b8Psm^ZaLQR)h5$?B=6D3qD_APW0Hu;;O#%TS za;E^die+jVkWp@uvw%^}(l-yNH7s|FKuE~O6~L=yIa~#7otv@^z^r3g*#fkBmaH8> zG_c4$z$avQ05~z0ZV#{;xw$$4^d^@2b3kck*}Vh;LX=zsuEDZ!3&<93(jNe$l_mZN zsBJ8-&p=4XjyJ$-XSoOgyMvpWFTm_%+5QH!E|x-}{Xv*4b}HZ#Qe*&5H_K2KVD)hG zm<{N?EUkHf(#LXA00e}nDgxYomh}=q4serK1{i}Z2@Oy!mhVa+BxHXz;0>|d*8=u1 zH%;|`Il^)j1GG3xc{3nJS)3NYC#2F2IAbgmoq#pY&4&r-lBK5?P$pQe`+-mY+Mo-r!~+0L)F6hbKVWVrhN_#5Rlf4)}!BJ^{yNSxo}g4ma7qfWFHz znNslwWsl`E4G0L)n+dr4EH?@u54eft0>&ZBem_fa0^HwgCYlMmhlZisiWr zkk{O__W;HX%UK_w-m+8=0wEzchXC)6r63O2_uR~k0pDvK}kmYs{P(N524}p-7 z2Oi*kvXq?y_7^wH=YW}H$?^g1n`QhO5I-#8E#MQun2e1`xw(m`&Ax;CpZDpx?24ovI8?S)T&XN}bY6r{oClC_yI|+E5ECauQ-Nns)O4T1slchNw z(7IWUG6B)UqGban>1D|-0IWW4CR9N0XZb7!lmV8WQXnA2O*!BWvNTiz(&A=6 z3K&Byky=0B=+`JipIm*)2254g}mmPo@XVFc-C#2g0IFhBP zAFw94nHvQ3NtV=MK$&70jspQ9p2q+;!O}JX$Z2lQrT}AxB{~DBvn-o75E3$f0q|^= znI*uUK56CrcZm$7joh5b$s2eN?4?sxB(kH;%WLbUz>@99G-vM)*W&8usT$cAQ zKql`fd0l(zXd4oEW0ic5Taxc za6^`b13-RolkNe=CrkVUP`_AS&w!AS9hZQYWVyHk>~C&rZUOU$W&0k`epw110U`cx zy!G}o;1g230Zt0bPzYG5fPDM}^fZ>%Z$L?BIr#+wLR6(z|G~{*Sx*OKCO3ImfRV+L z$Ocq}bvCLkb0Lkr**v+T72vV@yRCtyTa7EM4cWy$CTLPC!A16~=++aO?< zbJIBtm>SDv9MCFQYR3Ul$>L4`J|Wcv;8d~9%>Y)Eo0K^~uVxuu0F)Y*rzIdDM4JP+ zwJfJ=fYiB(ZUROf%f>dK*0bdA0wE!%_W`eg<>wHvW84fJ17;)3!zrLOu{2)*qM60> z0iTfC4d56ot9O9a!cBGn=&dZ1Pk_?K^7#S;gy?+-+;*0m4?uQs6H5X{C(Hg1pmwp8 zrqui)B;-;W;F&C$nSkBR&6on1JuL4zfY!^>l@EwM7QYbi38@zYPCv^|1h59UDJlo_ zL6-RnK(Sa-qd-82ks81qVtLj9Im}Ia17M7>oHYV!oTb_TLPBn~0^TS~K?h)uaWm5e znBy$J-GC-p2KxXp!4eDrJ|T@Ez?oz@9s#T=ZYstAJ;Aaj0cDycX9@@iF*OaiGb~@T zfSl!~ZyqpgmfJ-@onvWS0YX9^tODLVOW6isFL1NG1(=I0Sv!EX#4^4Ih-H@W0PqQE zdVsUSa&-b&4mWk@fWFGIdkH9OEXCJAK!}A~z+GoadjRAHH}OZn*kpNm2GlK585uvY$_o3SVRWk6Vl28oPCzZY`{9;rX>&14_QtM z0Og3KvIqzWv0ehW9!qW+Adk68Xn=9TlB@*OQ`VGr`+9 zi{pUtz>+=*r~%7p0tgBDIsjR83mgpd$ma}XQ0U;st@E(+^ntudM!)N8ldPblN&%lh_5Zctz+rk0c1Tlw|jumz!EzI)ELWw2ZV$yJq5f* zmgRH6ZsI1>2h3)c@oPXcSl({|(ZXUr06ro8N5E-ise1veHg0y_0KJ{1_ybTnSmwWg zfDmavfZNG3BI^DiySRBt1q_p=Jp)j?SfW;~Ud_o2q;0&^~Q~{R7&2crL53yA0fHKUoRu2S($ZZ7N5tgZDK*qUAwgAQ` zOJ6&njFATmsH2%g_~It#R{s1L*54t@nVk!EzD+0U@fM z0e6#S{S}Z~+~kFTvCWeB1XP#hI|+n@?EeM49hUo)`ajsa+%%;F<{ryYCZO%JlxG9t zfW^rLd_pP(fOE()p#s(sHy_1-?y>Zg0?IMVbvY0aqM;ISPgwS%fIQ_UQVSSoEQ@u3 zdd`v&142TMHUZuR%bNk%m)vx=0jAG#*#T%*EVU*eu36k3z$c{I4>&h0bAy0&%T3BK zpx?0!#{uP@X8BnG>=$kZ z)&TRBw}x@V?h68nLGuQFP6`9ARt7q54cH| zn`=ORa}&D*j31W$2SEL0DSZM$LN2`kp7?)p1!TShb_yWJJ^(Y7<^2oL(pb8F0Fln( zi-te=gw)djCxc}t1F$l=DN+DEi)B6sP!yKbd>|miNFm^6vpg38GKZV?2w>#0oRtA; z9!qrv5E62;3h?q-3TgnmfSVZ|Fbi3J>j6z=8EgbZ5lhev_=GfC0jHSdxE-)cxT)v@ z^a#sZH=vZV0%SQieItOOvD}UVY6VN91VTa{OafjdOW8DFS8=mE z3z$)sta(7IW*J`uL=8*04ETgJR{^J%n?Q-2-L|OW`A+wX)1U1EP&Zya7HT ztq^e9Ssp(DtAm@CZ$R&4Ir#;YE|$vF*dGEytfvF6$&#A|$Zl>D*?`f*lFS9vUY7m> zASC3S3V3}iO(lTc&&^>eU=Fa9Yk)S$vQh~Mi$$phd_u}vz!_rsr~|BFZn|TDKEiU{ z1SoNq`W7G{#9kZVjLtr&D-aSgzXR}mmYFWVzT)Pm8!)d~2KxZ* zhUH-Z5VtJG5a1Ki8v&d&J>^oER)kfK!~qdz7M}3CrjNq zV1043a|!54mf~wb`DU5F1p-2(Jpk?x%g7@jf4O;i1`P54;jM3f1Jo2ioQFV2$m%b^ zOJ&*m2JAF$3PjT%%ygF7R6xsM`Ar8zCX1B?_=F6y0Vj*4B@eI^ZjK88J)5Pn2vBlZ z)=GeY5V>W5o69n#0Wy!9WF=tav-DL1Y5~h#Ef5m2u^#XWSq@`>t#VV=446eMD=mOl z%#zg(h!Pgr3HXEzO~8q;boT;QDK}UBfL_K@Zvjd<%kD4`5Taxha5a{NaX?mZlRgO; zl`QcDpjNTG&Hy1HJLUi{%5t#)*wx(BECXf@%eDh(wJe3}fY4d&O~5ClZ~>=|WoQ?$ z>bZH`2lNJ()+0cPv78(O0U@f+0Jo84{Q{6p+~i#WMl(y|22c%_?>itQWPbp7EiCs> zfZfVX(<@-Mu^hbvT02YmCm=dloFw2AQuzg(PL_$3=08|n+dsz1J0olt&K0C$n)W)F}{+{6w6W0_^&1Jo6k(o-NLH+1PC8rSx2r<;^(Y9`pjVONR@DggoB`ymywG1HcZs**XHu z50=6cK>K8wJp;rSi?{@QLRwdVlVo|k0jzIsTJ8b;hvg&ylwX$0XCNTN`YYfH@gHyo znGE)DwWD-)-{f4jp>$jtoL%CjLY>t9>R zg@p32DFu-KPg1Ioll`xK6hl_dzg>4Jr04!?*X59s_pjAgLcu?Yy(r}7|7#_+kSzGO zTdaeO!hbD22C3@5HrfP*|0G`x$SeBSI@=(-_}}iL12RkgwOSL>BLCWU4`M=#z9MZLa?P&~BD*myTT`-Q6Z+n`})xkWs_-XBSenY{mf;QsNy!UL9NdG33;9H*pG?4Q%h{ zkk-i79YPVMuCE}!iLLGy5{tWod&p{LD}03X7PjSQD5fOk6$)C}#@-;ejk|vzkloJK z`3)HzY=2eDuT&?SnFNKD_$iQQv*l@!)5YC<24r@#C1yif58F^K6jACa5Au82S_>iR zyzH1?Sh;5(&GKSgin;eB|#hH zjk1+?Le3a>Yh92z&X&;&X~H(y4@H#va3Ftzt!EgLN$ze&A!~}QULbv%?Qjx`DJhzP zf*H1zImn&mE_DI2=h()VAj4&QU4hg*o4p2wl!P0Qx4>4j4LOV4?fQ_p#FoDgY0GR2 zhfqYRqyX|)*oIFaS>^8e46;17w!e_R#&&)Q#gtUuK*2iO_8sJIaF_D{*_&*0Pmr<2 z_VWU%Z8j%{LQ0}{$lGCS{(>BzyW=0o++{0EZ2gtC$F`mfMU=`+gZzEAsdPvVxckh6 ztV6cm97sQ6yVaqXl7<2(_``No1i1ls#Rg;_v#pjv#tB4WrZwwD1Yrlfrc3U1gg zMj-c=yQ*=>zGK^&fQ);#+-XQfZ0;-+QljP|?}2S_5po{6dsv3dC$<(3(w^B))}e?} zh$o^pa6GFx(Tgf$~ zzSz85D5OM-An%)P;t_IwxO;nsOqGDEM0LMHTEc<2Zd3RNW`!C$jB-Lz2W@VM5!l ztYo&OBuGzTOG$-dN=7v(NM-w%0l8`1b!0=f#`ZTCGSbh?qA4h=CxKHpm!d8}ESB7+Y*ZAtiPX1)VZXZyT`^bNMY z2#P7WeT0Hdw#FC8-Qw=(6|%S4O5P!3hi&x}Qa+pZ1BH}GLdUPXUADI*$l2qrD-|;L z*{(E5J7BBLgd$4qXG8uWTY(PA5qC@ZkoAWxxd_q&w$TzOrsSm*3Xa)2Dj@fSyNfEw zK4q(}fs8Y@ojORJv+0dcNQrAf-UVAiE9Cs;Zm=CPL$=3GNV{Zf>4qXoo%TZh6)wSTN?q?m{wrJE5hO8p2LWWgu@#;|`a9e5ITTZp z5<C#Qn))ygRE4xiVR3kW82DtVoI`ep+IAs&4b)@?!F5k zJA-YY7&0>1?hQy~v03F%NJ&r$dD(2GCgkLBw^j?8xojB?kfyUuHbD`kKAIsvkFBQ- zl6>xNIv}fnt-cG=3)v2PpqP@PekdqnTX7(_n7hDY*|z^cZXI_yCy-swHg^UY4QxLbkZNReE}@W;=o<2x*qZMk$Kvidg3M;NvL{Gu zVOxKJB1&b(kl)HS^$tlJcb}h-)y~%Y1L+-Xw+Z&IVoDm4p`erPC>3&T?uyePyNhi# z6EeEl(sCfx!zMZuQWEDwUN2i$5#;o7cUc0N{cN>mkT$@!R{=$oDyW8hhi$P2l0oj0 z>mh51ZKM&>huK~%D5j*n6$(b!F4`e?l)EY$vd7qVx*=nnEw>L+!sZS@Ath=E@+Q~@ zM<8dCyN5BzoMLO4fV640lPM^oRQW9A&#-N{kj!$IwFp^rY}3n-?y`NYLNO(M>rgPy zcDD(+3*0sCK=vZrpIyjUVkLf_*vH2-bNQtgN-agxW2IL%YmyivahipT+kaooOm_6H^n13V87QV?c@7F9w$ug4 zec*0v39=vAURNOFiLG-DQqOGR1{6|aZbRM++pZ5e|G3NBhs;;Dg+oY-*%AXNqSWvS zUE5#C`d~Y|g!E6g${Q%AWa|zJzSwdeAorWQ*(b>UVf%T343+fXd(J=% zsf7QeqIW2y#QK7~M7HA}$VuX^G_m_vW-{A)GNh%jWu!q7rKZv$Kb7qx6OuIUdUGI4 zW4qBIJ)Nzg0E#I&EP{dzwqgTvGr3zSgX}D}v`WaxW*e`DR1RBQ1BH~>^^lj#cG(Cy zI(IeAkeSD}*9vL*Z228fM5#p^@(b9KdLSv}Zln*girAh9AibEaeF%yvIUj+761J*w z$Thgzo`CF9w%lpRC}W$Og;Y74nukJ4oJGj1V0&1GoJ#JRJ;`nxcN9W)BU{NeWHhm@-a^V^ z(;_IOL>?ipneFWva$30SdWFnZwyQTtYh$bZgd$4qe?xvdTR}q4ucU*!r6kDeWJ^wk zbenBdgJMcvGN7P~ts@(9yScl_h3p=->U_xPW!ou)R3DpO0)>>grI6RpmQVpX1KbT( zL8im@XhPZ`TT2}jQR=h-@`u>UEl7sB+iZcX5w@&$NFQaJ>4aiRzPg}bjIF;Ha>u#5 z>xXP%YZ`=%3AR7OkeXyO#-NZAPatoKEqw}drn#G#fy^1U_c=(LW$RvmB1&B^LH-L)A@_p2)HcZe z%QoHt86n%N4XH~uy9Wv>3Hu=Limk?hoNMlOhamHYEq@f!ZrK*bp@>pRlaPPMHarc< zJ$KKukQK4D%|rSF+xa3CQ&PDC1&?go9^^i8m$L!c&unvBknzIyvjeGrY|b7OQW70N z-YZ-4AIOQhJ3fZYH@31fNPA~nzkni2WnM!52iw#&B%j=U-a^(FTWN zYBM1%jcqR*iYQf}L%zngm=8%hcgaPNmBBVr0_mA-FQrgSNqYqpWU*aTL2fp8RW*>E z!?sfg8M$n^jgZpWTnh>*QLT`d$2QmwIr-c@bV6nUTT3^j6|$Z5LJ_6P2Oz(QZDSCU zV(zj=AghFJdJNJHwl9HVO8Tatpp@-y26D@|Yjh#IobAs7WK^(~EJLc2&0B>+O0;#z zt74njgq&*b-nJprWb59Av>LXneJG+--4W#1vh4?u)NxmM3R(4ROXrZ@z?Kq1F(so{ zP|(Qs?*?+4xa+uwY>Vyh17tL_RX;i?D2 z!S~?YaY!QSR!y zAZv{6pa;^&*^2t1n381&3WP0n7;-1L8ykh}Nw(K<$e3d5oP^XgTR07cl$djnH^a6& z4>_~k2XM5U$WP!UjAF>wN&h{XEiLLSwiYeLp0|m=$ zIVX_2!rkl{WUsRQTtJ4$HgE~4HMZy)3MsMfAa9-RID(uF?n<8^bCYfT1=6VDyP)y0o2NWc*b$ml^;(wA0<^0M{VyjMqjAXW* z6iB78=^7ML;$}czDqBJ}BNGoRR9)==HU5`S330s{&VsLjb30b9Vg)@*|#2YE2o3IwPeWbWXni{G@ETQ9f~ORkqP--Y&|)UbaQv3Lsk!4eF3ER zvK0aDHK!EZ~+CYY)2vFdfXLXL-rcm>Mdlfv!z9l+F+AMD5NBQ zhP+L-u2;y};_mVdGPl`kKOt?0ZSNb3C{>U!^ef+ITTFsvm%HRt$l7BY(I9=F?Ii<> zDQVA!f&;dTT*y7-t|}k0kJxq!A>$8QZV99UHn$WCDNz-Wcg!|e1vw|&J(!Sr%GOc` zX=iLF4Nyd>atrd$**01rx!^9V9kTwiO?N_i$oAC*#gz2*Lct~5T|eYrao0Eq+1G4; zh9TpItz-;Rw``t3Atl-r-M7m6tv-G_pIZ2t}+_m#Vj0J3AYzbBCK##Vg}sdqO2FBDRuUqRjn z+x!jWd~%m?51C(VLl2Pl&Gz^NMU-m&2l+p2r!gcd^}idX;sdf0{*!EeL3$!vwi^Cb zOvy|l6eO{ICqr&Bcl~LQox*mX4jHLzO<9miV+(SikP;&g@-()!0?0|{F1;8sGuS2# zNXuk)7$u_VDsT#KE017Fw{y<(W z+wn2v)NxmO2ATD2>lcvLz?N|dMU%h8HNN= zTG)!;A-9#gl~2fSV@vyijCQv1gppsV4z@T63MsKuA+M9|QiB|uyP8bM>|)!?hO}x7?wU`h2J#0xuko0mlQUY0hY|o{T-p|%v0mYP@S3$u5TU8C@I^1p7LG~b9ZX;w2 zvCUbK8fH_iP)Lc>4tXPN51o)R%3X6eWR9_&^g`M=TiE~FXBwni6nXSh3Bfb3bel4Z!4V_RK?l*^{ALm?%y33>BuZ`+Ww zz+KlaWG=E@?L*oUTkR1PQEESc{AIR+Q%F|0TRMlVRkq|1(ml4(D=4PqE{l$>`hr2rivIDlJa>zJl`%?+2 z6E?$yLQ1?^$U9|AZ-AUL?k1Wb^PKIy8PYD;y4#?LQr8`j|Cg<<3zCq#gC5AbWGn24 z^eeVy2Z||48HR#uwy{yjz2WZPIAq_lbxuOY9oye&NZqrUb5KZ$KM#2kTiz1nJa9L^ z0-29&iEEJd#5S}6MU;Blg8XN;Rv(fV?#}ig>mOUiA*8>uZT*2_O0rL&AZD9AgWNan zzAqsAoo(O}GCtVuuOaoxX5B#{B|!vvUu>mMkn_#m+6!djQz?=VyjPv^klZfR4AsTC>;t?*j6$jH%8`mL2V|&eq zR63ho1cj7@C6JfFR#OH!ncVGGKxP(Oel?_Jvn|v>5v7vqAwP$0xDk?E?w&2k(%ITt zAw7@nyd8=uskEUWpKZGvatpZ2>4WS-wz&bwC}R5=gj6w`GXjN_L}QRw!qz+iIRinz>7Ug{&5~kvB+hWqbL6VoKV-p`eZJLXH2*ZRf5k39>ubc2XdtlPy<+l+EU5 zKp`b68}ho?26G{&o4bcR$n0TjDTK6Mwv%EgqEvY)6l}14zd-ILcl|MBZ?WCKL&i2+(-)+6*n%G@q{K*+ zUwJ;;S~BG9a+jV4nR{%L>5#V1_MQnvlI)$2i0z;V(*Lj(8Bk2g zav2l^Y^jxyd(7QfHDsT#z1Bd+DO+bfq|VsFMku7jY=*pZw%u09x!^9Z12X@zE!dD2 zvL*IF5v7LvAper>X#kQd?%IYR>zeIs1k!KVD#xLilC23SxMjuW@jP$p6$nl zjEHSu5mFCq(J~ZLVtJ7F$acIAIZxb`Zb9ZV+xiZqy|88MK@p{<4j}&@+s6?kuiW(> zLsraoa|-EiYz-GsOvzyg1@COd*O2?c-O4Rwf3l@TknzPf{s^gWw)hzeDY0K6?}zR3 z4RVzB-;Gl937HB1N$q_@S|VG1!o;s4N-ZWqeiB8Wh(*-%W$ zc`g*Bu~p?muEyPVA!Mhs<(5E32HRXIq%zr51r$=^R6$-A+k*)?+1xePL1qryNdu(i zvXxm-M5&Dy$k*92+aby0Zn_h)^4UJSAiaRCuNR6bx$TF7Lbk?1$SvaTXc)4K*-FMB zql9f$AZ4&=Q&32W%s^f#+uI!ElyTR!0GZ`%S4)sq!B)EpMU>iKgZxUif=x)OxLewW ztZKI8T}U_CM)#qZl9xj$sA1~}Ah(vgixbGMW2-)gjC!`6zmRHR)32bA688r38rc%= zA*YGE!3W5+*dCuCt(mRm9~4pQG=}^Zw(<{1TDjZ&f~+>SEH(Kny`60)5sE4KN``_C zw*EB8?d0w*9kOk5-9bHM4YCzBLHZEeax)ZDlF|kR!)#+6kUPTNKO3?~**be5V~p)@ zAEd_FOa}@n@rNK!*z!gpXM(%=ambuxOPqwXDYl_$D5BKUEaXqKwa!B_!`;~;WX-Zw ztU&r4+m;8#lw@x}fy*|#1-bLweeXc_0^7hIWGu4XA3$n}&H4j{lmy3+x6D?0201I- ztzAIoDqF@SqMk6XxJW9x|^S?BKN5wbSe>R%vzlkM;oiYY02hk`A(l~2gs z<}URIvUk|V6Q+J;_-wC9klJOlQ=yQOP=mZZwwg@H+2?LI8!`{r@^wf%WLwCGB1$C{ zLH-fja0w)TxO*;ztbnbp0@9D!&a0r9lFAw=IAPnagWOZ@avCA~jBU<>jB~c17D!#N zIqgtLNz@5>f7zP5At&VSxEC@n*~$hW?TT%E5Q-?3IRg3DY*S;9+;I0Pkaf$}I|b=? zY_~H|Oi6ReSG)|_4{WQeknzZtwhpN$Hra$iO5$zEduHp}g`5}eF83kx zA6xAaq`k831yDq(f>X$k*%r?sdE+iQgsgYAkt<05V0*cNVoKWYq2QD4;sJ8MxT|`G z>~FT6e~|IRmiq=NmHyv*j{5a&ozQYlcjnt-B4<^4P9Apomg+U67y8w%-Fu0e6M{kX6XGN{&H`{AIWc09g7D1|)Ei8dT zO3X6I>toxkfSi8r@~RDae`Pu5=bMr`gtB zNSk5HScD=!46wh66E^aZKpu?E?ce!8GCGV8Iam%Q`t~RiIWR? z2W$^{kaNggb0K6Nv7Hn{+8?&EQYfO-MmgjMY?)P%9CJ5qLe>e}XDy_kvh_7UF(tQ6 zP;kc9*aErd+#R(+_61u>CuIC(TkV2W$for|Atlled6#T&4&+>M*EI~8*KAj#kaojX zD^NtK{Yl8bWhjetF+4^J1{o(HJ9kNx%e>Y0g7i1*-C;9UOsYEs-aqd?kC0;V* zC9$QaK~6Gv6X}qd!uFmCX{l`8IZ#BYYaQ~_*y;)((YQM(f~<75LIct>*p|znn39xA zD9B_RtA^Yx?*7$4b~amQJ!Ist{cVI)E}Pj5g_QWMkf*cdbwExYck?!6=CdXCKw1IY zP#+Xg>S+M-3)xzSASvSRYy`53*($~%y@YLR0*Wcgo`wR0ZFUxNOS$`YA-jxiU=cFP z+3uGiRl#O?P)JFz4tbSqrCX3w#ogKtWLC3f>_M8zHhBO=l=?V={2I2NV@PVbyE%od zI=1=?NUvu*4565kqH8EmbMB?zjOm2ieLjNE>2XZ-F98 zWwt~9FxyloBqQ8?c0txCTW>FwgWKXcI3S>;OrAW&R@t_Wy`&Sl*i`YKp`dS9`e@M1|J}2ox6u8 z$lPFS`3Gs6Y$q`kQL6j{^0(MFz98A=E=$e-%GzO@PK0!y?JF6IDd|gtf?c+|bjaP~ zt}zR;_u2mBK*j-ENgkvQ*}MWMq(m!*yd$;=19JXw_f`g(0b6$^q#d(eRYMV_>S`hX zgl)ebl2h&qn;`3qZK)a3&)HJipqP@;4k)-_`)5P$U+y}3AUkCH+Xoq!Y}F2=uGst` zD5OLmg}iIF`Ekg(;VxkkGH=<2rXlT)?Qs^0DAhU-`S)z6i;zUzRjfeP1KXwt>5pvL z8&FKi%oY?pv3>7A?lX7&dyxIYc7FgF|Ja)TKj5ElK+14%~=Z(AcOUQg@ zo4kg!54QJPD56wP1o@wA*N>2VaaaEWS>J32uaN%3R`d?Vlq`Qjfy(^vMoImF+=Txm zV+jktvJ=@}lOQ9Btuqx;$!wtpg_M|?ke9-?n+-Xs+~w(znZ~w|4`~`(Vi6QkYPbaQ z)7hR%A<5vbtpc(#+0LpUJ&Ucf28t=!s)K@Twwy-D&Eam=g6v$jpBBi_*#_DnmB$u! zLLntqH{|8B9rr>`0e7VXkXgvKJ_u<=Y#Ad^M5(DU$S-F55J*b6>z#rugY9Mp(o5MI zTqvgGZ~+R+*ov1Sx176`RmiSjOIwGGO1AM$NL8`L+fYb}y$gBOY?u3xV{%t>1erB# zdjX`?vgMyb5v3N-A-|3-DTJh+yOAr%YG8Z5f%HbU_IoI%S2&V#IOw$B1c?_ujJhGI%?4JhbkYb=M{KJJbxA-kWg#Dt6iw$)llIc!=36jCBh zkT=No)(kmA+;z1<<}lk;2c(U#)pkJtz!~$C%LN&`mX4{#E)C`-x1cj8iE08zKmaqmnbKDJXK&H#~xCLqR zY%M+%QR;LL@)y|34tK>9M<%o!9@@^t|PD{TFjkh{v=-8E!;Y)yBN zvBvf%g48;j@dSmGcrTE*!ImCF&L(#g?~u90_WlWJ+icxGP(-Qggr#5kJ8X5ykoepk zq(as%TVXn+@3AdsLNO&NIZ&|AHl{=F0eAoMA^VW6vj{Sd*#4G4>JOV)28ERP6_6LO zTOm2;?yMcMF4!t;NdL>W)eXgzWcNWq z$Tm9wxtH914?^}8+rS89T(jMeLF$Ihnt(z|f+@(mWha(A-{Sx;>BJCOd&cDM`0loTC6!3*2U5#;{kF7+6)U)jb_AtPpc zJ%`jAn;k+SCE*q1y|dNaLe2+wyZ4a!$(H{JXKqd|5u+gt`@q_F*DK`NEa$%R5nqCCh; zV{0yi9F4ooiGmvkvEzUtw%3bmTWRl)OHEe??kWYZux;Ey(#T!b17tO^ zO+P`p#rE|A#gz2LP|(bF_YS!&+%B1+X2Kz=vdei0-++!Y#-)yuY22I+llDV0!6$!Ikc^t1h| zf!qP^I_e?YVf)(%8G~%q&5#;m^IM^i61@ZRhS}zA$Qj`-p$9TY*@pTcZH(=40E#Hp zIt2OSY^NiT2zM3ZkTt=!IRWXDY}wOLOv%hF6il&wyO2B0UH>9v&#>JuL&hvylLx6e zwqP9!DKWMn&t+TNft-2n()S>9fo<{t(iYj?kD!QBJ;#v0#CClO$uf8K7m&5Wb`V1P zDqGPt6jQQ%3k4oqY6Q7!+>Jd#_Bz|^Gh}SAb-qGslP!FMLQ2d}$lGGu{f3-v?(z~= ze`W5lEhIsj&z6`9MU)!WAb*$bDFc!{?%J{;YoF~b7t#;dD)XV3lC450IAqHyf!rhR zW=kRa58F>UWCUyjRggMni%ck_#Hxe56Sm_9$T{V%)Pl@2w)GZBJ7>#ihayT%bwd6H z+ea59f4S@Jg{+Y6rXSKT*%}6+n3BU`D7a!P9)sL#?p6e{Z`jhNAmf&8d3bZ$c5J7Ple)i7jatl4tHl_95$q?fDSW|FN|P zP)y1B2^74tRh>g_%-!~1$bMtXy@HH)wz(TfeXyx}D5S)BfV@w(hbPGS;;#80WPY=q z#E|yGR`vl!l-l@$e3kv*jgqOnUrEA$Qqzf$mB{v)4CzU1eQ8ik$!$6mB(pVUL2e3n zM>&w4%2tvG8EI^*1(4F%v|=cvL=4DFXL~DyoDA-|Dj_qI?W!8mve;^Cp@>rZ^^l*< zR?q}V4tGnyDs^QrC}=-^5n;0*S@l!7F4nvlYHWdJEg~ClphX@&g5}Y-0&)zjE8S`ErIK3bOjyDrz8ofNiS|iYdu%gaU_c)`HwY?!H?fdx&kI9WsX5?mHnh!e(_tAtgaC zwHzC7id)g_MN*khj2Aa|Ahy-0cRCxx|)#3Tex1 z3+GTosiY9{SJ;NHAX(+^`3AB)wzhjnUt>FefMQB2pP^u#ZTlbOZg7|L2HBfzb03hg z#rE?Bsckk#t^X>dBua$59k%8a$nm*5PJ_%{wz3RJ+hbeLf+9*~=0g5H+f*JT2i$!Y zK-M8!Z!x4FvE3R_Oi4pI6#QX3s)XEtyJ8cvkJ(mhA>)KCtpQS}Y|;dUl*G-DcgEJ$ z207>4U3Ng`1zT+wr2S>v>wzLl74$=X$hPP}a>-rtFl1e^jf_J2HQUQL6jRbZ2?aN7 z7t@e?%U#tRWZ$vv%tOXKTkaC1A~tsg3Mo-*koUkgxB)qj+&yeT<`Y|s4{6VACwovt zsq#a}e_`AB1Ia(`vQ8lDm2LVA(qp!-3n-?f?-B~$*zT?&_no`OJIMZE`x8ONCtJxA zq`ug^7bv7eiy`luZQ>nrez<%4giMw5-+NB?52PjhC%H=4_*F!yx@5>tWZO@LB#FDi zbjV6(Tgrs=6t=d$V*jOgM5!I^8u1I?xvq0Yn|=$1=2Uz z`eG=i!^p^6Yee= zA^ViAx*0Og*mhbWbbe>H*8tckbcWHGYiF(e7R6?$JW0Hx%b@NEkkz1*5pCP1KXch zB`FaUe6Wo@LhdJb|DGZHi>>n&GQQdVzCr4T&HRKyO8jrgQ@Q`$D0vCnzj6}(lguYU zW+Gc+Dx@W`4QWtBsizFcPiAY)h9rf%vs}nZWvj@C^fb1uLMWyry95d}w%Jn1P3P{r z9I`Xm2C5(iHdh6l+kcUc>dHODr+1?eu^*A5g@(zgc% z^K5qqkh{QL;~&UgWczar8B1&>XOLQE^Ddx}673T5R@f%4A!n7lw_C{c*t#P~TVuO= zgd$4Sy+Hmt+x{yg8{8GXL)IqS(kG;Ev8DV#F(so3{;z^GyTe^aDrEa?e>KS1 zWvkAF)E=9k4TY5GI^^xM&F4eT0e1;Ska@^9R03&7Y>%Z-M5)#a$p6E3S_MhKT}2IK z9kXrLLHY?>b|Vy1GGjr(Dcg4oY3i5B*u4f>*<*wd^tUI=Y1xUYVD_VwPN|sllAYw~h zhujD5#x^1Qk?nOGGM?BvcOmu67VbkKCFT+2y|C>Dkn@kbyi>@0Wm`Chw3sb1gd$1} zUqSvG+tUps@7%TBL)Hh|*#o41vQ<7qF(q67px}!w=M8edxtslf>>swDFUU~(fA2X1 zYWG(v;XkP;5eg}>QXnsp?Kll`lDI3)fXrmJ^(;tBVav#cB1%o=L4GRRM*$>h-1Qbi zmd18tKzce`Lpc;va##rk8EnNSq84&)cGB@IJT$lb^&WEHVJk3)JfTl*vwQ*u5H1tn}% zbC7Frw>=NprEIxNkWt1qw*skhHnj$YlsFrZSHbqM1v!=6HT#fR#dfj>Y1M3HhfqYR zjX#iYvSpq?Qp4T!8D!P6eO^F%9b4Ze6jO404F&aVjdzgSz}-;<*^O)^Pms~Xw)z4o zi%pB6kP>-^yk@qyPsnNEuImRfTiLD>_I{p+oJ_(gKRCWP(-QIcE}%ME4LvT=5DhavPRgl`XGIjZDs(9Dft?Nf-$!K5y&0q z?rsdSg{^4Larf^WvX|L9L&#WR`+EhcRW|b$ z3MujLAcNS#tu?^%x#y;D99;6P~tU@THBq)ZwL$=aV$T{L}tsFA{ zuw_(1TEI4GLJ_4tY9arat)~H!6Yg%BAnTN^z6H|H*bdvEn3AGSC^%4Mw~?oxXp z`!CygKV*b#uMVUx+3aB`q$C`LyeqaEft+jZb|)e8hAn>v(r(!n=AejDNehsF$2PnK z$vt<^E07hjwXH$=1Karq6jM^U4F!*E+dkwzahJ0X+0Sfqhmi5Y_VWi)|Ja-pD5NAh zgS=O^=D&~=b9a0RnQv@mH<0$uwtfdil*)X7{13LNCrCcI`+R|{FSgzo(!bem-=Ub2 zhA$}iVLSSPT$T6VjZ&O=@GCpvKdIGZ$Vg;MOM_Gro1{Y_C2=O?C9`$qKu!vGmpWvo zvegzqS{mD45fo9Xz<_*>ZLtiJbncQXAuEGzq#DvQ*FXoTEs?y8z0 zJBMwj6*6+!ayuZUv$-}DQlfewFOO}o4|4Lkdl-Ps0=AYRNGoJJ8G#~7m5)Py5!=QD zB*oliO+!`*+w?4?8*E=L6jRc-2nD5Vcgv7l#$BTa+2w41)*+*Utz-*Qm2BP)6jGw? zL0%Qx!~x_~bN6-xnI>EJF{IV7U7bP^rRpvqzm{!3grx3Q|NH%||25Q7|DR;(7AkCD zONpRwN=6@{u#xTGGqm2sUB@dlVzK>wgBqLJsy`vUh0Xtl-YL-&4u5sl$~K<_t+a8M zkP12NY(pAU+rjpj0cCZvwPr*AC^^lA0-LQOA6n?*ZnF^T?Pkj^fvS4gW=bKgm+iY8 zdZeVk3OeXxyEmcPe(sv;AbWrc zBuSDaAxV;+BuPRtk|arzkc1>jk|as^>~|kt>$mpdtnM}UnxOM_w%{TZ+~BUg1UdX{ z%N|I#$@b)hgaDgm4JuP&b^}Unv5f?vfXLmU2-&vTWIK>1$QIazI+XCS2NieN7Nbxi z#9b){t?#m#jvz~z?cx|x?XeA|pcW-W(@;LbCb@uO``m3@Lf$Cb+ZALwU~}9+iWplV z4>c%Z@*c_^vYkFakvMm%GUPsD3s<0t1lw;Fk{z?HzCtxhsJ%g%B%7@Xg-*D;Z9&cy z+t?SRKV?gPLy|O`LdgB8LJ2z(D1FBEC56Ou?mWGa{erF12Wc~Gvr6cP60cNH>5@%1 z1SPZF#fKsP6tzf{@{v%@=~CRkqhKRHelEedzp!Eq?$7Yup*)kmHr@BmwE_Z2c#Y@WvK8h02uJ zIfGISwv`MNc<1gp3)z}%)@w-f!Iry)I+QS4fQl`)qau{}1yXqq3*RajG zA?q;PwFgpb+4QSWn-UXiP+^3v&kx0Q+-(OS-zZyKgv@$2_YR~SV|&_#nv^&bfo=?J z*(ekp=T3VFc_!Enk09eDTTc>_8`%OWs7{H^G?blUTfBh6Chi_CA=fmUDF+$MZ09$S zbcSu{4ysb({ylVVVUv`g;4F9RWyoP=d#gaYIktZlD4o7_z=LCyf%#WJMdV$*mbiO3f9K^02q-hk5EY#W=9 z804<81=)Al96?AMV!IDPKa@DR2bFf&()&;{%$+I*`S;lN;*d4M_M3pz`)s}ws7;Br zQ>YMSn?Hx*2i)CeAYY8la0QtU*^<|gGS1eYhnkeQQ-E%c*g8chn&8g!2zid#o}VCN zl5MsM$xqmFHK9GkGqzR-lAg0I{X$hreC)dY)A2d1qmbs7&1ir+ zlyEiy74vL^Q&8fLyT~-OUSJa}kma6j%?hcCY_B$`MG5wQQ2v1}zW~Kb+>N^+?<3oZ z8#0yIlq-fCJ~Le4j~&m&0RU~?xS$vay)1yv|v<_tvkVDP3 zJPYZD*q-JfLBnRTLuE?LI-t}r+sGmm&~kUU1ldN|WFAPPV+(kp4kdi}pyDXo;s%t^ zb647g*2maPB4jbJT?8T3INQ)J)S`sw9+aP8lSHA|BzGGz$ZKSKi$kU|N7gxoW1;VWpu!uER&$!6JB^H7ZvY6U1`WwSj%p*ikuA0elW zZL9+6=h>1~NMdJGyh0U9*m;A}|Jc6XA<@B|rv=#;*eY#E>tvh#hJGmV>K7_4vgssu ze@eQzi%TK@5?gOCWOcJ`^+D=owpKsXrbL$tDtOo))lhteyXj%b=Vi-`K;~68jUH0^ z*bWR(lM=fpp_?@}{}dEm=dLjgc{bP}+hmbAGW=%kv1GZce>QKVyDO8NH9i2gmL+<1mXg$s*W+BTF+h-0^ zCD@j3p%x{S@1XoKo4E+ZlH6UEAnys=@DpT8vBjPt#VMP#1~n*Qvkv9bZ0`*ya>kwW z19G3U6+fYg3%02aB+Ia!{XjKJP~bdBwIShx9qNx&o41v)Kor z3MJePLg^c}aSbHia+lIV_B@+%6w=)Cfm1h`VVs8kF#DhjJp@$^sPG=B~O3xr1z0H#D)scI|;=A-2&~s748iH7K*o zCig?3Fn8Ml$hpV%DMI=Pn|lY6?6Z}3p$a9;M4)t(?J^392i$27A$yE1b_8h;*?N-D z4<&A64l10m?cYQ36n9-E$al)N zUWUwRwt59pp0WLVftr+f_X^#dvrROh=mmGFCgjPm4SYhzOSauFNSzeJR7c$(i=@pRlmhD&xRVlG=5IWDZ1&5&E9e3?v$WdTh)^ zwkJI#6xl4}P?-|5lThk`ZNvlxO57cqA=@LHY!=d#*#dJ=hY~*Kq2d$Uq611)xGOoK z^=CHI5@e~eT`WVY7q%fU)S`r_56airBpXodmAj2i$XjQ7+k#AQY>ptLXs{JRP=gXC z_n_Q6+vz?OX>zBELGBN>a2%RwvHd0>*(clT2~?wm+9{N2v)Rs}&=+^N8OYgT8@qz^ z-)zZiNbC%0g?=dUss@$1*>rDE zQp#QY9rE|E^|m0ZjBTq8se9R49jHx-u3xAiXM5}_{VCqZ-Lw?)DcCYH$lTAS>4TI? zwu64CN&S6T4BZT{`PEQV#a%-Kc?Q`QMj)e_?S2%J53v~yP@NLf6Hr#eHaG=^hq>FE zhFn@U!2%gZ*nCz~J$8_q%|8(S;~DdyRvw@`x;Ht(REo$dV|iu~iw zS%TaSwqhBYSYVrahGb5*vlpmF3FD_GgFG#Y?X8(aI zlyE0J{weKY8}Ek174A|!klo9sltbE8wp|7ELy6x?sN`ew4noN_?y5tOf1Pbk3t2bV zu62;w&!!)P+LV|WhYFi)eMTrA;BMOl`L@{FX2>kExo08eHrvx2)TG22J9HCd%Q~Rw z4tLr`$P;2aT!M_dY&{-G9%c)8p*kfteNc9fZE*t%N4R^~gk1Y z7phX?{vLE5W0ORo;30SGF~|{Tdy7N5Bes9XkdR<2oIqttoJd2dW461GIj|cJl~X&e_H)km`c%xC*r>LGcRZGi<>( zD0a!+*E{6RvMskD(-m8#4JmSLmT#y*30J>R?wV~xQvOrqhP${Fa^JGadZCFt+g2YW zyJP$4hia7IQbCylTS*Ot?zx*DhMYyVixEiwz^2hdk`i0g097cVdlE`NvTaO3Vwt$ZuhLh(Xp_wy7gXZDl(@hT4>c42{3J8VO9P>T|x^H4s-CUHQqUG6rVkT=Zs=7LOnY>s6}5n(H=Kn+Tm z^g+3Ow$pVe66H>{3Aqp0!duWpjO}+Dk{zJ}=UvFQp>@|?SP z5%OQK^*%z@4BOTdq`qWpJwt6ubk(3jmhG_)#jm)VeusQHw#)})zGl<3A>|F*K?iD5 zV)rj}bIa!Mdj3;1&t0P%^4zg4$RJ~Z?OqPa@7av~P@NLf15mceHmHWe58Ul(AXkY^ z7=a9rY`#%QT4sA4gQ}D`KLMRTvE_|Wu)>{T8ge|doyhl-+T2}6A@3L4@F8UCu*Hrb#W$NY2{kBTGX>>-*xu7n_b+ z|GQr)UP2RH|4Eq2K{5&3*$q^q1oa)1>1K=ELm?@5T_wob!?sq2^fI=31(Ni#*GGgaiv)Aq15vabgcj&9a^DLjfyy12M=p z#}IqbwXL~t?5_axv=g|5;wwnxOaj=bDL8=9|<7=oz35q;omJ@OfUUCzMPuA~{E+96?KuD$<7~6r zko<@(w*%EFQ5S}?3AUpM6h7vz_W*Jw*~CN0aKhF)f}|<7r6g3P#K#nLe#&M(gMw-9 zG8d5JjBPjz>CV{>a*%MrCcTBql<2>MQW>`QdnjSp*AI|ccH=;+x{LD?{L=@g?!&^>oLgu z!&Z+&%3rpB$54|J?@pi_LI2-7kBKxC?fOq*>KyV&*aj{kV>jFG6(pCk{aizJO7!NT zY!BN@0Se2wn|pv1WxJ+mOV{rU*e5O4tcQ={dHq2qfCL^Bh3-dA7$mnCc|AgdgY{m{$r^NIRlwD^V?5h7MyusaGH{|lO z2{OpA$>x(o(g5450;*Et`~Y;m#g-q0f+BYY4dmEnJJCYAAY1<^B~K ztr&wQPT8i8AX%F2>=>$1f;t6d&e$SpD0I$U*9GLfU|YL{^clAL6(qT2v)@1!O1R5I z=`7p$JtSUnmwJHgIW}b((q6OeR-hkB{H{W!8#eDNl)UAx`Ud&)Y;#S>ddGI%g46{z z{TI}x#KbpLxM%AV-u@IXay;UcweU zg<{>@eVsvGDcf=eGWD=kvXDZ?X1RtMlyG$m<$Bph3Q$DOUAzdn``BcU(1e0*>j{$e zvwb{6HA-;Rpp25OREI(X+)ckjP8Hk52c#cl)3hOpnl0LaDwNRu3#EtHHo6*r5;fd4 zx*_{8n?nX^wQTot=!X&~`=QbZTY3OW>bO&>A^#}bo(8h&*?zT7JyfBDnQ&YGmuxo~$dF~zUqR98SL$m1Y5kGS zB!w(xwhI}gdSV;ugIbgj?T7LeHi-&~J#)9AhP+j_Hw|QZVRMW?iW*yC6lzeyqyfsk zvYk#qkvey(DaifC7M_MC8f?EaknEjp)e6-pp=N_JO*Y#U%v5mPP{U=+} z4N2N;iWR6r2|KG$`it#r4H7%tdHj(5o2?Rnv_EXK+t3dsUhP1oUp8GBN(zSm?pNXw z$lvv!gx&+lDq-6?gw)+^ts|&SiLNA6kg`3dpm+~=(`S%R#+JE&%)M-yETojP9ps=U zC3fFJH+^jWJ1DB)u5k}}`q>srkWtBYUxwrZY{qA(PKoIkD63){tV7{J?)DmxOU)*H zK!zbU-zOy1u)TgkRZ5)yfzF57^1{cTf?Dnj-H>C1?W70N>Dc<^kTA*?Qb1)&>?om> zo^53i3XF00JOtScY*sC#8E4DspbjOBjzPr}F{Jw@t9yV_tO0IBMEkJ%R+uQ?WU1hs|gw#GZ zeFbV$VxkHa*4X-9q4+v?+i#F>gRT7znf+|;7Np!{dul^XN}TzIZUSuCUnsi8omSHN zlSgDbltRXBww_){9%KvjL3K)O_CwhnwnY^b4srLOhFrUBreVksW;-8&qL8?=><87!#35pPuPqPKXQ0$Dm zuL$HlXInmiOc!jGLr9Tfvm~GfC0r$;+$Gz{DHO?a7e9mCS8TEjG?8Q5%0jYhwvQZC zqXgG2l(}Im-9e#S?xu^7GtYKWg7kN6nkPt7V2eIO6-wx?LFs$8jXES2xob2a`vaTf z1Jah*?mwX)N}TLKrAM~(50or(r|SCrlmCfruN$&f*nWE;^)s7K4z($kkIv?1nCY`rbPcPl#;N$??HiX?iQkuP0Ch`L7E;m;}O)MgtKF) zC}SH;L5W`OB57z{&L&(ymOi$%OGu?)d%c2MlwiMs^8IZ2JQP!MH+~O!2iQ&?Ad`ws zS%wsYY@rI&poH%#lvA^f$dTSiR0X9hamd|TWlE8PO|mrpdU)y z)I%jBThjm~r?_)YLVgq5!xUtlW}7lYYBSrp1!_~GdJZbguv`vkh!Q#(!+PTaetr_OlJuDbX8(vI}f4 zVJPh6Zf+lPEwbGlKn53^J`PEj*p3rWl@j|-pmR4{@DvIzbJsqD93Hmi45V9Od&)wB zm(6kwl_@cM3#C@sMhZ~C$K7EOvaPYn9wE&-Ti^-mP{PMERNP=&tU(DsccnVCzR6~K zhb#fMiw{V(#WvK2T9gp&KzWf(@(abbx!dUa`ja=v_SOxVcGw&;ND*Qy$e{)$O!h;$ zUAEH!C=%vQrH0&lY+(&F5n=n)Lb83f)lsNM3AHgO6J@haK%oQfZjF#L#x^z$=?~eG zGms?Crm#X4O4zYM=_9r;JM{PI&wuZXJqwWin60u1X_IWTZs>;+uRKucgiW^!B~#qR z*C78XTdyCorrEXvkot_RB|>dVbnQTebGFA_D1O1+bOiEc*fLSbe95Lcgp^sfgCnR( ziQP%)=8Daqf}%O@8fnOL&9-m>8E@F`FCqCYn=uE~DKUKmW%F!!JKRn|U0HHMzT-guEYY!zReoVvCs}#V4C|7HUw!<{Xr3v%SwlkuUC?4#?eM zD>|WxZ?>r=NcO{awhYxMLG6Vyzibg76cQ%>dsezOAZOQq64y2%y@aj41xdQu>_MnP z33nkVEoB?ugTx;0Qu~ly#-@xx+FrKZIP^n_-wCKBXY-yw$v*C?r;uO4Hg^tL``NBD zkXp&6zk=G7n7D=t18jYHD6Zmey8!tH+1f?OtY&jRLdqewrzfaMi8EE`M#Gk^LD6CE zv~Q3{%Xat<8AsT9T991F7HC6tN^EwZ>?qsfFBH~u_t5qIC)XI8NeUSZZ09maI?guK z2URI?zaKiEV3Vkz;3RkJYRF+^d(%L=DYkzjkYHjfj6!8foG?JCX|~e|C}8GpU<$I$ zu!W}~jfL%J2I^44sue2Evc1@#gq1tnKWKf9?PdY8*x1HgkZPXo*bTKPL9qhm?QFqS zDE5!LuQkZ)U|aS>rUkZ208%*FEZb0n60UZj+#=gZ7>c;Ki$@^$5}WJ*nsBpi9YV5Y zwvQvIMhUJYl<}~YQc!4xyXiB?>1DgPfb^?unk*#ou|;!Gg%Y}Nq4XNt#vLTCbJw_s z>>F&35~TIB-It*sN}PO#N}Fuy7bqFvPF08eTWos`$SSh^HX-#ko9`29Q)2B4Dg@c) zf1vmdceldNpL`)ULpNmJWlQ!z$}n5M9BNYHjsm*bW9ulPXoNe@AmrI+dme&}QMOqv zBtKxw>7Y6#>c*gKjO}O~3LkRUYlK{JHqitbj@VjeNSa_#KCykiL;4Dvy9G&}*~)FGLJ2e9P`b)?`3s3J+-W7hf3nxuVp2%^%GT2h{ZQg& zA5^NdHT$9D8+T3>!HF2+r9ycx47$?gnXZD>r;@q z%~qd=lwWNBEKrjY@2t>GhizgWihgsK`UiP_*an=C@t1Aa1<8fU|IQjeZm3R)-W4d@ z#rCoag(d$AY3`4J)v>)bAk!$D;{#IY*$SUfgAyh? zP;QLv^aqLKm`ZeqZf)Va5udM`J8N-4amI6rU^hw7u$gd zH7T)s2fA5e^Y21YH+PLa$g|A05QU5$w)+?)Utu#IL3K(@A46F$+h7U`uX4AShFm^2 z;Q}(OvH30`={no%6;!3f`5WkbgDszjf`0A{_mE?g?c@Q{1=#w_kg&xTsz7B*>{OwY z$hPtd1-7|+euHd5Hfs~o?6BopP=^vmzo24>?dTgy>~bd;y8g5tW)mflWsmJs3aKJ& zOTAEw63Ts0exJ>(gkn+dE>)2CfNgjPGR4?p!;s>TO{#+$l(4CXa&fkI0~9&p&N&IW z6KusPXyTY{$_&YpY-bjzMhWUUD09LVnTJ9t?z$Y1^OSAP3F*^pbr&Q#W3w+q6-u~U zfzsz}<332d;4ZZe*)wd)O-Osmwz~!WP~!JCRLZh>Ls0UHyJ{Hn=h)`5 zVbjNQaHayOucY#(f4 z4WwzY{b->MC9IA@#ZR`EF(}dI&Ncz9f3e*dAxnpCY#LI1vmMVsElN;Wq5Kb9&<4eR zx%;w1Ucva^{mSwJWa|1)LS+$BNZ2fHs6h!=9w^t%HnIvuq};{VAa@U&%nwb-*tPf>%Y0y!0I7g0#x&!#zqBucjE5mcds?j)2RVB1JRqKdml z8nO?vIW8crn(h7)`k}MJKrP2;G13OtL*oAfu6O zwg-|=vE_Q9Iwk59P}anDq=drL-1QDZE;E}r1Q}-7TEmdk!nUM?s+9Psht6l&%;Qkd z%3Wp>a?G&}n;@Ny?Z6BP^K8;ts7#6eIVfdkd!L5_|F~OlKsE_a9m zn=%F|R@p*vs6h$e2`J}dTRDLuYur^&A@@3)^&FbmV7tyhGC$ks6;z{y#5I)JWRvHi zP=LGb0_5Cc`z%6wk_HP7gQsUhxbdzA4FhJ2`?otzwC&@N21sPA+cBdhEitT3xs#BuZ3T02( zUTjb}&E4ET$aThcvj7>++4L?*dck(=hN_gL*mAgxVLBsk7OBpwJt4w?faKoDH_IZb<*m zmh6EfO*Vxbs!+m?0!n|deJLTa#hqsmvVXEwh9GU5ZB`5YP~w#iDt)o(#-L<}yZAWd z|7PnoLe?L)Efb{vWowzCHYK`dp@J~=-#d@TIVj%spTuc9|8@B zE!#>S3XE{~T!3skHtPeV8D+~oLLEvNtw2RR+ffxtjBzJ_h1Lyh;u~ZcXZw7IR1<7V zEvQ8a+vf4<&xPp^}Tuy8xo7OGQXGY4h2*cNZ0u*luR9pu_(GZi62knOw#Nq5+Wo}elv z?mt84AvQ@33hr{ZUWXiEwzmeP+hhCp0SOVd!Y5Rw#EA}++GjicfdWzP2D*CxWIJFB zcSD*O+fNVFp@daAR6JyRQ9y|}ceVj&{fO;m5V9oL#x#)XnC(~#wJ1R`3gwe*!7(Uy z!rj+6EFYyq0cux%|uvP-s) zC8$OTE)SH+vX#70=!(1PHOQG`yV!vA*KC>qB)MUWicp0Tx_6-TE!)N}B<8tm>_PTB zHb)fF7TE4%&<`a}9zmshw)8QSEOMtxLH-A}y)PI%;CDf+G+7(nNv(4W? z@h9$X^N_E?X1IsU&uqyDNLgj;FGEdA+^Ik}FKnGE6s>XRd4)W$Y|n3yvCcNzgye5* zxfWEXMBNvZZLl4EL*aMsdIkBPTunAn0vSHoT2e^bVq5Bks+9QH2c3VinUzqm&0R(X zIlkD2hag>t?O+%ZzS*QYs7#4|J(T)kdpAIVU+xwrA)8?O@2pXrf;3(KNi>?F4kesf zprV9ra1Kgza~GM1)}?HM1G4n6tvMl;jP2D0wJ5>94CQ;-@+(kG&fT~V^7gTvtV1RR zn{pFU^s|Mwpavy;Z$mjH+e!$E3~*NsLv9tDbsw4-WV=3qWNNn2I8>vAL;}hTvB^)M zkcPYMQ^+~Y_IU>BwQTMTBpG2VXQ2uu%v?ih9oywCB#v^YEkJfXTdW9a$Jlxvp&v@z ze1b{_w&pXG9OuqigZvY04|T{o$u{*4sf}#sA5fbT)orLS#kSvp;wJ98ej(p9+j>{u zpUh^qdN-t;Vf!b8nv{4ahi)uv6a7$hmb=sdfEplPze2f{Wa>?U2L8w!8r8me`&a zA;HaNaYJQF%zB{IGTX>16!36&xCYr)*kpc4<7Ep3pbjN`h){8rZE*)m__!h!Zy}`^!sed zCM1cnDL$bJCG31b=>xW}Z%B-B=MfZtvLCWlB#<`FHroUJP~ufDR61hQDWGJ6ySNha zAG7riLe?bP))1sVVQURTZAx_MphAl6Q4hsWxtkt`d}+4KBxF8g)0iOTIop95YEokN zEOc|h=AVP28SWbMkmr(Z!2ubwZ1+w`e#K^7g6fo*UWT$cwm~lxzUFSv2f1$8gbm1W z%jVmJq?EL6iEZTs z3OsW6dHXHA+zTL76YMNIw+naMz`RoZoC~YDoXXR@Xq1UpD&)RH1~sQ7A1; z|98JKZh*wD|0JX)AiIQ3IR$CE*>e5fl5*~uN6x6a96cKei_@`Kgin4cD(?p z?!dmVgQjlwe&2$DCbZqArkaU!7C<|37 zaX$y0>)9l?P;iX9^*hL6V0*iVbmMIQN{}$YRwzSdN}PCxQj=__FHpe9-9R0(O|gX= zkjBLJ(}X&du=)uVr`cY)fe!A^!&3-X3K2v;FQv>Pn=N?`DT8eNmr#=ucdnqD9k$Lj6b*6b$wQu9w&wz5471HXK=M7d z+#^({L|p~SM%a$3Pp`5~M68w$p`%ltx) zBer45z@KypwgV|79J5J#p)w`<`=C^k?Y$ogoN%|Gf@~?aq8ieivKfb=4ket8K*coM zpdLz`aThT_>*s93BxJc@TbqJZ8MfDHs6`2O3zWZP%UhvXmb>wJ$a}?h@((iQ*pyC4 zam^NTK@Cdyc0;)vwv`nqa?4$H6>{grSx1bs&B(|ZdkWbrY_T&)`^?spfqp1)GYgff zY|R{$eBsV{3;Ano4|kCDm2IjBsq1X#C8$k_>L;l1#3Z0n zOpqXBv&=wcO3cnesb01b8x)XpcW8%feQdG?NTXm2EJ7Vh_*jC9{cMXKD52!8E5x7%B}^Vc zxe>P0V<@8IPL+b(qio?cG@)nvJ%?mtY^#@0jS^~CP{zP!yMaRE+}-9O=LFl>J*1yx zOFlpnBb%ZORVZPn0;Q+ezN(OD;?DC5*{9hmZ;;l^Hrs@LDDkQVm1fvV*n6w#PmwKF{5>67t#EGAhXYk4-ZKDIIJF!%&kF zyLHgb0-IkCMV;I=43KA$ZDA5Jy4dcgAo&uT(G1loF>QgeZnnWWD7?(w-aO>;un7*x zu)^kZLQ*f=s|%`9;`}mnzRH$gfr38n3_i%Q#&)s}>DJl$Hz8qzEwlxdDY3H+rTlCw zAtC|u(1VHa{evY8@~q0Dw3g``hxLx)h668Ddw z^9q|J2?d|ITTelbD%)Ec(!H?#yMTlmTj3HaQ{qGpO1-k3-avsmcLR5j?Tsyb4`~`~ zKMzoc5?0Gl@ty6Z0wtQ<*Izo6I`cVFL- zx5Kt94E@RU%~p{>iXS#h57eN9t6nJg%Qm8bBErmn_bYKFU7J^W0 zl)KvyK zL-9rKA_I`m&L#{&<|VcbB{bwra>D{N85kmi!@c?1$NY-Tl7 zro^lUN?x&zO+bMxcgK^E^_s2M0I70p0VC9=gnuTebi=kh2gUQ;JJPZ#h^@;&3X!jUb(wHgB)*c6Ddgh&X!0+k{X-* z3aU`T?lqM9VEev-#5#BG0%ZGSs}v!1gKh2s>QLhK6ZH6xO;dpqP3~e<$p6LG_YPTF zY}+4@@|&&s3AHHE*@TL1wx<>p`{8cp2l95IWCx0maVHBwJ`I}~hRow^UlC|X%eHa^H7TKd z3>7BWOmQfx}ikjuzcx`XsH zY||yE*Ti=52)$8)@)^p^vPE8?keR!#H^?!^wo!w$7PeX)lFYN&{y`N=xch=qR<_AD z^!M%m|7SPZfowK5Mc3G$)QfC;-B5=Te|n%tJDW!aC6>6W$|1jlZGI54EVJDVLrN!` zRt2>vF|LM+D{TGaP|U^M&IIILWou1BCO4bQ04dhko{dn05@%e3y9W>C++s6sK)L{1dJ~dvvyB9xDkUC> zP+DY@>_WjE?tFWYJ;?U44{3JU77rmI#8!+#WlGeaK*>F}^HV4g=58!hJJC9GdTr31FtYbbumoiz{nj@a@A$Q)&xxQB+0*-jszCMC$rP$9+^tU%Ed?!K## zC(gF|4jE6`Dj$$M!DenibxOEyLb)@xv2Q4no1c^y!T65pIqvBb8p3dJ6{ zyIq64k8CaY#B zp!hF$kv8NL=KjCGBK$(;uKy)$bdCRMNW%8s4K*pj)(aK7*$Of!D&=l+0P^&(oee_9 zUN(gilJ~KNR8XA~e$-G-#n?(=l2iR^5P_Ke*dg!n9!8Dx`L zpwJL^J66as%=Tr2v`RMD5+oU6D=$M8N|<#)DHYq58xlvkQ?EldHCuE8Qjf9q_@NFZ zZUvx64O>Hm664%Cb|Jr(?O_kHOt4KyAf=8ieF(KEQF#m%C)o~8pqQS!t_0+rV)G>- zlYy<4f)vwiiGR(5=-9s`nTjv3)QKF{|W#`yl zD^S?N-TW)$oM+3wLpm#)whl=b*iIWzl@j~EptOxG_zeXYxoiDEc01dup#76(iS1be z2@W=M4^*baY#)?dW*d`30Vj9I3dp*`);kQTTx@|6s7(p~MxoLw+p-3VySaPRLcTS& zzo+#5$?RdfoPvhd*+z^|lM)V1P{GS4nS-Jm+-=T79v|Ds0%Y7|v)duLpRMSC>Xb0G z0_C>Y&R3yGfV&|NQLf!26{YV(_BM|D0i_N z$bZb%SAZ-rw(TOMJYj2=pcW-MpP*u#?dchcopLu*g}e#2%o}7nV^e)ViX_|7C)A+C z?k1E!XY;q9!xVS*HsrowTl$3zX}0^Wi9g9M*$h&sMv19jD4Ss$?uWux-0crQ&MccS z1nI8Xyh=!#V|!OYRZ3hKgVHx_g>fjD=T4`C?6+)ZdPq}X8<>WKJGRgaRHnrCER-y= zty!SJJ$Elw$Xa5vEJCUWw%iibri5`PRC;7PaY6AX?qq9_ugoT{L*{3;uMKFZ!nWdv znv_ruK!q1J(+(7^a(A^0d0yE@!;tZfEgFI3?`+Z|s7?u6$55`u_IUzDKDcuvAXlBO zl!Wx3Y||G|Z-edP5_+QqWfsc(V~gaVP?Ni^TgdUnws8k(TWqy^Nb=2QdxR>Ka94&> zZMMl5Nc`b0`3l)OY>FDB{$<;%Lmf)|X+V#H<^R`>o-ZiT^}odGH{_SF&37P6H`|S% z`;$`2rtOAWlo;=Uial)oGAP!|-Hsgc_OZ1TkV(eo8io}8Y|kT5gA!-eP+rcK)j)>> z+^HuZw}S0>5;6?3^%x-85L>_q)hMxHg0jPG%X3gz$=$;|XkX^&};e|BgY>S(apk*s=L1jwRi%@cc?K}ttbleT@LDosO z@IIu{vvm%jHYKb_q0$uFYYd7TxU-%@zG=4n8DuuHO{Acq8Mf0j)T9LY6;v>>1+Ss# zEO*~GkjKonT7ZmmY=6&C_>uMODp17AU91YZ7T9{0jV(0>B|O{>X(9hQ+rAF6c-emSkaB~~I}Noc@ofew z`q&oCP;8UCTMOj%v*{Kf(-vD|5mE%$1{_d>5_g?Yew(fBf(}LQ+-s0~hwWt@G6dP? ze2{FHE$4@7l&IN;vLUvU9VooVU0(=thS|h0q}yj}Mj&Z~ZRH56QsUDwls;fH#i8IK zcbNoaKVlm_hcr>PqYFqlW|L;1G9~)6P%_5$nS%l++%4Tg);L?~4pN=68A?!_5-uL0 zQi5&x8H%5A7kPnvNjBjPGM}?;)S#gh+j|{qQiAOtRJdR(d_mDPcav?%bIEqrfs7e8 zMVJ0h@+-DbH&myDpB^ZeWm}U$k!$X%a>$iqvkXG|8@8Kas5j3xu7chuA+Clpw`{U; zC{*BXX99BEv3*TK+9I3F07>rI{yq=+PZdg-orO{*wyQZveBe%Pg=~*(Q5&RwV(VFg zI+VDz3_X_F8Y@uZnLCFY@>kd%JdovuZF&PzR@u^

T|k0jT)Ob|6BrH}1N2A@4hz zZx1rn*lPQb;)8AR5Nc53T@=dK+4LvS;U{;=Q^?(58%#ome{6dxNY-TQq@fxmdaj`C z7u)MK6mD@hpNE{^Z21DDYqM$ZA?Xj>=>t@y#Qrjr?yvl$dQo$!@l>Zzv$;?)V3?_OSH|Q-4zRvIQhin-cy>p;8~)avv0z zarf8{`TE(63dk&HyBvas2G~YMpe7|8jzR?mn?wUe2f5qSLY^VE4;^G2X0uO0awS`F z8md#mlnKg>u$`Nsh>E+RdB`=&7G8k#YPR1+sCSHQ-2uH(!kZJyXxOZ)P-vXH+cn6c zWt;Fq+6lIV50dEEsSRv%2T+F+uaBU|X*Nv^ zN*K9|#UcL;Ti+REF|lo*L&{mU<^|NEL}vynn%SPRP;8F7nH$J!Vawb?rg=715mH#$ zj!IC261$(E`~sW*89KCaSAT)ri)>48kipJ&UxQ>zY=%#$Mv19^P}adV+=9Z(-0im^ zr;|)-y-Y&zRq%OAiZm3F$3%yW!m8~Fyf^P0~1CV`O1T`t4ybKj~*i0@c8szTE4S9CiM%N)@h%LGS$@kc#eyC0fTLCB+X8RPO$Ub+D zUC0$-EA2u01Gec1)O*NwaR|Lpg7O&39I-`Cpiq>%t_0*bX4^;Hmc8SZwzA@3Di>jyGr*<8Z(pA^?@&l0FXiL*UWKF5~r zgAQ-FQ_CTDp6yrx8E)Bnh9OyjEieMrD6ugLW$)OQHBh+7-Gdf#-m@7eAzg_rJq1Z0 z*hY*{l@bq3Q2LQgG6w~pxbw|J_A=YY0;G9nTeL$$g{|m-%9N;Ifs!w5=c`bl%H5y` zvc9r~y^!jSt>c5*l(4=9mEPH2x1o5AJ8KZ~eX!+2kh#t_u@4P>vYj44O-hhQp+bW# z7=xn!xciPno+jJs8D#uotDHmf7MnQ@)hXdR1LeNi#;&1Ao4eQz`2=MI>;Lyx9-pC5*Z&e`s*pp%cKHTryV+D9kVMLM_z6`gp}PsCde}Bw zkl4#zy$#v=*zCWMTE=$YW&Bfz5~rllV?SG}7fQ&v8|sJr18n;PkVV1vI|wNU*}O`q zMTu`Js5r#7Fb2hjxw{>Qyh=8m4l<3fCG?O&#WpYvH7Id+2Fj1JwP&G2HFtlXJn<*@ z7~6{#GHBT579rUS zGh9GzO1QX$N*1=^EEJ#TE|PIRJ*4Sh+DCFU8M-DmG*}fEz*30G^h9nzoIul^XNyik z>Mgb&1Jt3!EhF?8U~8D5#5Q-1Imj=vJbq=5! zC3>Pz_L%K828CnX&7VTf6Sn*rq>Hm@Q;_tO?KBNlDY5?wN+;NY*HG|`yVeb4PqM8R zAk8`3a}g3!Y~}~3Oo`biD0#s)R)GR(?vAUF^^&dk9a3f30v}MD68?Qcr7O1OCKS(d z_t=7b*KEcg$ed%l{Dp>Y*hVC#KQ$@gPzn|DY?3}GdduBrKjbN}eGEXxJ2v|eBrmcR zl~A1$rbeOMJ=^&h6e)2xq=j4$Y+)Uwe`NdBL%mOI>(kI1CA`f*nKGNz427P#yR|@$ z3fsg2qa#yHxq}vEw)SoGJUhD&LKsc z?dSq(P-1rm%KxzWv(RCOyLt|C|FSLJLIz>s|8tG|J4n{`zXU@Gs!?L<5z0!~hM%Eu zH+TClkWzQwW*LN3!)&=>s7(puDyXDnJ5fXN5$%&ztz{^u zWBXizB9q)X+>lGpR`NjlDYoeisMo-Du?f9Vf-(SQrr9DQ6f$zxwF^0B*f#bct%~D}d%XVLbI+Qr|2|Zr3rT#&Q9Ct%4$bZAO--axCw%-nZ9oY%C~;Q>fDC1}IXxtMX3H6%8YOCGplpThWEKj)aMx#noK-f_3h7?inl?!K#qawyV*kbP@NKf9-y3*ZLJJNdbq1rAXhJ&f@dlLXXOn$FAvt$D-;iT~?du29D%f0tVw2#?$mO~ zreupMAoU1a&oI=X#H|tNQN`96g%YFOIW&-8&Gw*$EMsiblaNxwmY#xIl&Ca9#c{R+ z6BN^O*EI)uC)j-RkV(f@TYwakY>RfNL5X(`D6ePJuRw=W+$C2bw}EZY0~w~-_PmhH z$ky>eHA?hsLD?C$*KH_l;%+_&IcM4OAxLLt)9ypkIkwXSs7i_bQ7CO;3&x<}Ja?@) zWVf=do@BpOuv;7W2y<2SSO6ZLe-c(Q~z-AqT zLfhQkjzbQSZ9)fWci0koND^d|PeT<-*qwn=yKLXHkQn05ZGmiiY!xe{4ztZILLExH zUVF%Jvk1V#nOg>_FZaTV@wB zov^9GkRr}@6oDF)*nI@$Pucv(&|!kR`U&JdV_Ql87`n2C8jQ+Y>I6- z3xzMZ+s{GHG@Eb>=`Pv4caSv0_I?jlDRJQuN?)-R%1|)No$dv)U$dRPLYf@gKn)UZ z*g|!vOo{CVl+3fOeL;a+?q0qjYk|$ufmC;FIl=m;HYJRAL!~0yNe>ji=T0Vrd?hwf z4w)a=z7){VBiqU_)TD&+2vm4tGpV6ynY$|u_ znirD%W3z2S6-v0KSkNf8pixJw2hTZ>Jx2dTf=_V%F;CH@>hk8L(j6iWPXSB*jb z4%_@GWcg*gIfInK;{WFw+7#5H#CRGicCqzeK{3hy5_hg4Z#P@(1~N(6Tm?wc!}eT+ z8k9Ku0OfnxvQN-qA9v~sZ-9~2zn&espwRcs#vkY5%HnR_^Q^K_$%FVKkZ9@?=cd;GF zHOJN)g7g-)?J(3k&-O0@y-|Ym2+CO59*?2W0(UcU$YEoNpvM)q)FYH|aX0h~`B&NYUm%N{?e`T@ zuCaM*P>T}Z>QK?cw(t*%t#fz#1$n(}x;A9mU`uo$g^z8Z%l4-RCGK`Z`AxQV4|M3~ z&Mkx7TWl|K$Pi$g8-!%rY`I~mMu{2~loi=d)KGYbyS{PA8DtYDAl)uoa}tt<*j5Zs zl@gzfPWR0?w zJdo;`&9DKrDdA!hD#h4_15o^gyNC$+;%ve$WIkov*n@@=Z14L}lM-x)P~nWN5QU;i z?j}zl&pF%KDP&BsDUy)Xh)4hH`1PwJRuc$zAmta%I>oc}Rc7c2j_Qvuxw{ z&>JPhAE3-No2(3la@_4yAjb{cR~6Fc*D|H$^x4_ThrrWKH~%$6R4T9l|9fr`&; z2cuA|!d;gJ^1iV7w2-OFR?|U>SGL6|s6mN$(@_46O>cq@-?>YgA$N^!a2_&zu>ZTd@Y!DPigp$_=ue|AQh!+zquL*DzbS4e6C^za6M|gl)ac{--xecD_c&ymV5>|(Y8~60 z9_moywE=pZWYf$*2|ahQS;#-d)@Ok%2DWW0q?~4J+MpICI+viLk?m<2ip_90tmz z*2Xq`422iD+dqMvb~Yga>6X~MNl5Bodrv`CN?f>v(#vdxD=6sXPM3r1D{NS43ILVsV!|9>}g?@*f(#_LeY%XZR$;v3w_ zz965EP5g$;n`~b{(2$>PMOgY%lM>1jsIbLm>VcvG?ymYE&oJNvjZkKvEno9jZ{modZfm*(O&Y@tC{hDrAeXDLjz+gl*3Ybtv)22R+8wJX=uW zl)LIST}d2T<{xtv?FIQrzvtAnyfRD-M~`Y_2m%amn_4 z4mBuoHVx%7Y}pKSc*ULi8gggZj&C5tHCs;slI7R}MW{xJjS`f-VOxHJ!g=lG#V*I6%9N;= zLdhq#^Ij-W=5DYbvOcqg2Ow32tuqLis#C)C5|sPLHs*vPP3~eY z$o0k6y9Vi7Y}@Nl?>F1O4d{&$oPH?NW_t`kp&#yMb|6QG?Q$2={<5jUkVIJe|9Ey7 zfhv^HeFUYt*fx(LQS!fp`Uzy~X0sH>P~VM|>?iC*r8vXH-zZ9fND zWNg2ANZHTky@Ohm_;wE!o6;ceb4b-3pCGOUt{4iU) z0Uav2bALhZ5w@3a$e?1I>p-$mww$p1ry3<{x}mI^?W6|^k8#&0gPa;RQ4Zm*WZNmc%S=ipaP?HjDn^0k% zt*`|}t=vtDkY|DIEC?BGY>GWdzQ`8Zhw7B@a{%S+Y->>{vcz3A2Duz;mQzT-%yx4I z^*Y(cQ_veF#M4k_g-vz^gt=HmAjukAxd>G#VfFz^dDyO=AaR{L zbp^6{*`ifQy}{P=4s|GT>jQf9u{A!S#3px+Cgk_CJ+vUp7Tfd>qztg7f1ws7DkaW8 z6}Q!O+hqb8x?&qShnkddcmWl%Y?2HVz232${MQ)09J zgF+A7-F`uiN4AMJqLyxa)8Wogy<1VI#{O@diT|s2B=tPdon_?Pwr-BA#Z~% zGY6Uev8k+(qRDn-gBq0By#(dI*!;`TVT-%^3grG~TXI8&Hru@ilKrq5HlP|MrZ%B$ zhiy0jg@3u*7a^zM`2YTjunXzB{+H<8gQOC+_kE~Ji3^8Nx|^*Kg@RJ_?&TV?%GoS=NHxHgD?n{Z7{7-~3bvC6C_c!Y ztPJ^v*u)BC9%lQhLPJWnm3OE~3FQx{Fv4bPKv5NUS53$>$~O898P#mjA4opNCKX(N zs#C(21j=dHKBZ7(oI6J!EY>`nYr01?n139MH zHnfn|z*f^il4&;E6jY&vyJ;w8WScZW;tY35Gh{QdDdr*dEZg1!)S<+mMd;DY=5auY zIqs@X$ZuhrUxh65Y&UC=(#odwLM=*+`=H_iTmKdmvvIex4S5&YT04-*&gKe1iY2z^ zFw~&L*#ju=V9Opshs)flW02d)b{vNcD{MVykj%vvIEQMK*tmeQt8B{|DD39$AqzRz z*o-%j&cl|zg{13jBSol6i3cSp?PZfZLBS2~e9w^G$M*38X*Ssw-yp%yR;)o~O4NTs z$t||?e^4O6-Czr{ZnK5kkV<6hbf7jRtaq*cskFoP+6~2n+*x}e-!5BT2AM-_69drD z9^2_4)T9Kt5-Nn*f+{Gw&)v5g@0AF`SCP@NL44N&fgZEOaLM7fL2 zLat-BUJIm;v29zS-V?TeHt3BKoJ&w9&i1$rg-*GfaY2p*+oc=Qp0TOcAxV<$a09AP zLbo4EowIEQATh;VU4(2GZ1!D9oo2h=gF2Kr6@eZv*;0p4BE#L#G338u+dqLUS+?I( zNO{fXO+qb7d`m&a9NWSr6uaT>_6qXm*>pL`bjy~=Ly7|1z#Y_}#NB%+f5+B-fDVh? zxyz9Ip6#Uq8A@z(uaNA4E%y%9C{a^~vX5*h4JiD?UEdevEVGHgY2(tqjE^|#&)EDgm*USFjS^Q{|J<8~hfU#u+Z^eCb16x*u`3LChaAA_9JZ256WXJpgrAn6R-sUE6QV*fOhHn9a~ zpx`Wbty#!!W?QvDnmM*-D2?ea&9lIdw0$cAIq_VLE)}b~f z{M&#^i)_n&C~oKOF#!3N*o-@n*}-riT&?YjYqB6sdD$hN~)`G(X%wz&?} zp~P#!^QXsMHcdB_2yqwdf&6=HeKN=rX4{rS%6+z`0%}pBa~LW{*q%n9*a3GlYRG%Y zmeD|_BR16pq=>Q|O+pPy>^4C8V>Z7LI*f5wH$m3++Q?N^BoM$t>Gi6bf8(_Y#AwIX25Fq`G0tok49%7*9c^Jljbcir;c4yMlZL zHt`xV-?4q&Ktn~gl>*eHgmMuo+_RY;plFG^t0&0wz&2WejE`*5DkOhmlfFZBO4#~< za%Hy9Pbl)toudi4Dr}_|q<>+X{(*X{Y!|=K8zm?u>wn6;vPGm&=#9IsKFIOTw$Tr1 zYizXvNbQebgF2M>qlX@wY@TT-@x@(r z2J*Mq=FO1ho9)H|Dcfw?1*k=d@kOZk!`APBVjb>woRIgIt>uDDg7g1#4c8i^==xv6 z^E%X^#91Gdm#}61&|x=s>TSp^Wjo%13_WZ;AxPHC76?N%N^C@+Y#-b55fqkj_izk3 z``L_fNGE4YCm`tn+sHXorNn~^D6L?VWT4<6cfKrSA7cB+L7HK<#al>FvK8;3G9~Iu zP;!Lr{1FPMxEp+itfOq<7f7XM>%2m3N?5N!r7^bGIuzG%XZ;8H#@X^;kXg$%(T0X5 z*iJi8lM>`z-ai#|Y{70QI?3I459HCat;!(d6kA0O$qj7gL8wj%*TYb5nr%!4MU33V z)R1e2t#=&Ko7lD|px#-wf0NJ~B{&UG#?1C;ghF%N&CEg$3)|%!q@8C|Ss{s)?a&5Q zD4}}^N-eN$E<>V?yZQ=bTV%7lA+??D-UD?gacTp4Tw+UYLJ0?VLjlOY%(gE=7AM

XnLw{HI|G&n0@&JXmxa%uJ&H$TOfppt!%_<}n z*;d}6DkVOBK3%%9sNMU9-CCy_*0n@{Sqh{X8V*vfqm|l z`XFnBt<(>x4%iF|s7(nMLs03EZFmHVA8{8Mg?v#qK?9kO**3J$P>k(e2Q?|dHU$+< z*b387G|t_m3G$q>otYtHf=w|G$px7gKT~CnriOu&6naXUn7fA8Uw)h4$DDkcaiMN9fYcs*sp}r4Yr^P z3jX7+rH1THw$*V+^TqZ&0SPTOvmPo_V%7j9zuCrSpg^0u<5|f1!`5qoR2{Z}6>3w$ zKO0o~Wm{f?;=;=R$DYS!$k+A1M57BbOV}>m&`>wq$U4-dgu@M}AZ3&Ip=b|xn*qqv z%l08c#y&RtE+m(+75AVzB}_%2TtD0SArz5wH*^fS2H3(UkY2&|dkXaqvaKhfH%fR* zL75>o>m?K#=I-_iawyp*a*%d}Es=*LDmM8YRH203dnh%^_Wb~fYVO=+$Tr4SsX%HC z+uSSEp~UNV=y9A)Q-=~-?qUtdKf%`b1zB`#+ux9KlCAjzwJ6akZ2qaJXM2)Bu_^9m zdLXZXEz<{?rrA_-NMU3>h^lGi?45=+MMneH3!fvMp&KgPHAK3(4l#43kie z5>r!9*1|Sygu?UO?VBK{l}(s~bPH_Wc}QwwdtZR6l(=Ar(u-^b2Nbk(r(1#SOKfMW zkjB9_;DLl?wvZPpQ)1f(C7o<*TTozyyO(Xq>SD75A=N5dE(EnHVSFDdx!Fz*p!gbh zvMA*9u!%9qyw3I&hladtD`!xX63XXLVS~+-hN3?1t}>8klWp`GGWyw~H;{abOQLg(AoLhz z^C+RjF?UrJ{ z#W~xv6>3o8>>`v;v1ON_!wc@zPRN~RJ9a^aOSYahNS0v>tV1K8_Yu12exnyQa!SD@=%)+*6*Ow6Wi-O6fbjUeT00^Z22-|uCPtK zKtnHVr>{_x667_gP-P3&q3A1j-wnv~#_t+Y%2w*N{LTJC_TnzdVqo&?lMo1eVlEy0%^2tM^#9eV3WQ>WlHpaKuI0j=O+}H z*}hzm*2(5tgCr|#<#nh+39~*ZTSs8W{d7X>NU2W5Y(Z>tuXZHVQWO7#5#A5BgpS%dpL$H8*I~YNa%2lWO7zsA>>k@|9SVoJoBszn_u2AakS@ZeZ9~!nw$lz&rNsWO?LVat z*@E3r@QAxs4`h$Bt;!(HG262o5@KxTL8wfL*IS z+s6uIykfJvA$gXq=z;2#Ftq{YuG!8vp-7Iqp#bE%VGE0pKF{_WgnDn;*7u+{N_g9c zG6gp4Ar!jf?luZJifj`nkoKM}aSBOFZ1N;jp@iKOlzL$MPDA1&ckV05_QY1XhSX)Y zxjfXN#Ong|_{^rchY}U;Vh@o2g{`j)S*mQ?6-fEY)~rG;N_4(M#W%L64=DD|-An`W z*4Q#l$n?Rc`i2yBwxb`YL5bah_^13Qn_mJQHn^)xA@@JFr9Q~eWV`Q&WM6Cs1yrNN z)DV$v;cxEtMw1D56KosLkXFZ5dx0dAY_>P3LJ4;@D5Ync{Dj0Q?vno?n}JQyg4EM&du^ygi9a3a z(a7fM+WAvrhP!Gv0%}m=tRBkS*s=!baFILp4CJ=69nV6BCAJ<5By+F@tWb>-8#XAr%(lD) zg`M0zEJMx}HlqvDx!BTfNV>{4vJO=#@n8c=yV)duD7eO*F96v+Y#$<|S!Y|^g#<5K zaStj}qCNs8H`vY(p@5IO!DGm}$re6=RDQP3Db%Kf^(0iu9dkF+136-Bmwk}-giR%f zByqMw1yrGg?qMi(%C4&8~seNw#|})S<+wN$ByMEj0xtQrrz0A^!#2 zz6r9V*?!HC@{-Lv549-qZ2>A~*cR+i?25Zv2jtDN=~f`qHCtj8QsmeMJWzuYcfC;l zhOO;`4)ff(w;=Z|+sig&D6q{1A=w>UE(FymQL_(ai)<$cQ23s^z9{4@v57HA_rTVS zL()gKl{2VHiBIQH`iaeyhJt18G8xGJ%r<%rX)0_-H<0kcCM`f^O7s_@WR>l+1O;BX zTY7@5Z)~M!NcGNUs6uT@xOjs~HMZdoDE`4+9;_cqj|1luoE z_{Uc0+Wk|s$=#$B@_ezK^+LuLo1!0*f3t-KpgJY|3_`g!+nN%J{BT!QL9Py)Wen2) zvfYeBy@LDybB%Ew^hODBJ(THUlTAY*$^Q~|W*|p5+t)0lm9n`kkfevLY=tV6FuMq) zdfBd)AhC}-wG*<**rF~--Otvu26ZTLYaM!&vo$uL!~k~=Kjc@iJp>@jAlvi~q#R;P z??Np~REDAAFxx={iYdA4I)c0-Y`$a2q++X`K#Ebe#RSx##JePvSF`Capu;ill9!NM z!#0?O4C8EjIY_2u>*S#tC3^0l>;&8EJrve)H~$DZC)x64NT+AhzCh9`w$oRrN{Rh7 zC~aU1)}i1ucdZ6wH?pmML7Ex1=Wj?bv6(winG&-?=ugR6wy|y~VCL?)2eQtw^~xZX zg)Jb5+LZ850hQ+2mWQFZmAl6g$hW{|R6}MP+oc8?T4WoUfSQzWI0+T(Y!U+$UE*%j z2zeZAA127S%x0g1zZ8l8`N{HOW(vW|Lt?vr@d(Zj*$CvGENV&__yn$Mj=qy0R5ZhA` zitTYX^8k6nY?&v>w9lrhK#B<4Q59-XV)r|gKVb8JK!=Ci)juKk5!+G|GDO+#TafIS z&F}-&C^7X5Wn*l^lD$8LPq_P!q&sRyW(}Y){MSd4BuPS&Bz;LpLXu=8Ns^ExNs=T9 zNs?q+dmY}_bFBjh<*lh{UkW+nY(g)jKVkFqLDEyUw|=NfiE}C_b;g!gLxBW$`eA6{ zob7A`8cnkG>mcEREvSdel-M>vi4@z~6y(3;?s*!rrP-_&NOQ%OwL)!57@LEN8Mfnj zD1ObI+yQyBY@!pg+^~JQAXScSc@1h(LU|p^-?Ev#P&Ciol@D^?u?=rSrUF}Z2U6U# zNrO5QiA#%NWc_5j`GV96Q zaQsRLfO3B$Ii%_7CJMR*-pkRAnhDQ6@=2sy> z$yRVfWlA(`K#2ji^G(RF;%>kX*#_A{BBW8XbplYE64v*i;t<=*J`~q*XNy4IVYXZp zvS``Hk0I3v+sO&kqy$9*%8#-Il2CMvyYCd_*0HTzL8fuG$~C0WvsrRbof59|P%o8}vm zSlA*zP=yk@h5bJz|FLaKAkoTQT?*M}*%o>st&Q!z59(0jq!KF4u_aZ|shvC35agR@ z+aHFk3v9n5kb05Lqk~$M_^O8r4z{^TD7M7i?G)s3vgyr`d716h0x4Z={j*Sm5_jjI z+zMNJ9*V4Tx9Wga*Vvw&kkQTdZv~RCvt`$y8YPZ;pp1v@*b9X=xa-}5oL)9@8`5vG zHFqGXk8L>!RVnc?1f{mv%m+}w&t3WuTG(bAjzOa$+hH6McG#q6P?-{a=TIWR_HhCE zcez_kL$)AWF#~D#*o-$&n-VT=p<;+_umHvPxeFH|Z2Vv(xup3W02&Mtvn7@C}G9`CDUwI zMo7HkPCE_RGi=crNPErJV}&}DxMhP%S+<59I=$h}u?YEcY!6G2^_FeQ1*!9Fsa2>& ziRyKzaL0DA0mTa3b@?FAJ)74LnTu>S5mG*|&F?}DO1#^HawRrH7>Yb{mxw^CWwwDM z$oRyzcMQoZY@HLRMv3kOlzC=*NkXA2ce9s}^Mx&U1?gYebXiFH#&(i}s+8Dw2c>Fk zfqN+M&Ry#PTBx(Fl%dfNwxSD1oHl{nR*~ghwZW#QvI?GDWE1LM3hioaR2|BL^23PyZ)E3 zIRv>SZ12O6she$K6jDgp3OcAx36m30wukL}5(>+>Q<)%FFI&hA8RTrg7D(2|wmu8J zQo`#Tlvc3W7NB51cef76p=29hhI9jLrz?;|#inpW6-wCkK*>S2Z!aXOxm(?W>_cpo zZAhzO`xk&Zlz1J4O2cfU`_QSDyVwEb8)55>Le^2X?HHsUV{68t79~2*pn{I=@f?bc zb2puWJbJct8Zu9?X|5rqf$i`HYEWW#9?DI!`3g|P$X&e%txmBmK0-zl+x-(HpJp>w zp&BJ7U!jbdZSWlm&2YE>0XZ#fLKD*eWAn5isg>=m4OJ;|?iWhUvgNxXe+t;R(@UX+ zIkqzyG-_w-?}LPSwqQS0ro{FDlvrR}Q$zkm?w&P}&B10Jfiz2O*)gb131fPw=wv%K zK=Ebn#U@TemKC^ zHt8Bvr-ZF_DC=eW*nq;D+&O%Z%g0vqLxwH3sU1k>XS>*iUMWExg3{Y;;V={wx$8QF z96M|qN02VSRy&3yyKMGTs6q*M2`Cw4o4A0)J?;{hkUhkv%s|?Gw!JLWp~RmYR0^}X z@1WBI?yC2YFTyrkf~<#ZH)Tj2Wz#)FElP~PK!qc=z8VyZako>4JjZOU24s%2xxOIf z3ER^*)S$$f4wO4(%Ls>mikxw$?S@tpY)3ti@tmzk4#|^jeg#yc#D)^eT(B(-LZK9Q z4?~dilFg)r^l7%#C?vgN8ybhIlz1=!r7~<1BNVvi&TE1evTW~WX!M3{{vRad*b1{y znGy|lC~?bnz5x02+zl*2wmY`aGNdW6bylD@C9J!l;yv4o2Z|TDvu#4&2e#Z6WGS(Y zi;(J(?IZv-DM7IZ<;!e=eJJ|G-S+|HuCT2{A=5KkB?c*~Y?c$KP6^j%Q1*pwBngFI zxr?PB*BhJc3NqB#wyz=CJKN_C^hycNJe01pl?qVsgS+Vm$kAZCe1vqLY?=xrX|hGC zP=yk@-=O3d+vYnYwz#W*K=yC8g(jqJv)#9#4kb?hK&2nHk^twl9UO z!utPbjb9n0?)qPXrw?jT;%h%tkg(0EpjbC|w`#~EWz!Es<{q}w5lAUx>(@aIO5D{$ zxn8!m0gA}ETb+Vd``DhRA)|urp9PZlvt_N&-?y6o{~Jcg<-bgZD>@>cDMrxBW%(jRHj5<2uh5y zeS{(Z7wwo_VW@8&`L$8z&??CA}HhI_4pMrMocDf|+AI~4P9*R=?FHrTvNklD*tTZWXIZ1by7gA(uDP|n9@*nlEi+$A=l zRX^K+A2M#U?TL_FWa|W=8YQ~-pv(^2%RUqea5o!)oV#qfD5MXv>5d`k9^1(YRHeke z1e6M~1(Hx;pSxBHS_rePTtTA;Y){va5Mi_ApfV+9@=)TCZR8&EN4YzCfNV!>vNEKJ zvH2@dn-V^&Q1O^;=?#jn=`5RV5(?gMcRK|+a%|&fNO#M2YJntqHpMJdp@iKzD0#>BJr9Wm?p7U;{hqDj zgtSGre=AUj60g^w(gWM52Rbcr7xO~CN4DNA$XaIG-iFjqY|S00MTyQJRH(2$hM?Fp zchd)ur^=Q-gv>8&ni!;fWjl;R4NB}jgK}?dzH=y2m9kw@{|RHduf{pWN*iA!n0Kc!cy{Y@R1b+G2ZqhN_e}_X?%H+440g(B@A60WJKn zoqa;19k%`!B>b`k+fbPj+Z`w&c>ce>vetF{Cx6%f5}&&vn}p3OgEZZ2Svl0Egt308 zC}leyfZ{#e$<>fo#wKbYOE24(7E;OCmdBtbC6vdZd>@$o*A~*17xd(C7-={3s-> zvK4etnGy{XP-2bkd=m1zxf?J+wsp3U8Pa&zIu@u+3G1^^af9t;4vKrZvn@d0O}3l^ zviR7>mm$>_+sO*lqy&W<%KO;@9w@ra-M1HVi)<@fkZFglvJEK$Y?c62r-bVul-*?; z*@wbG?qUa!YmZG9g$yCK?HDB6XZwspe|JLp|2fNf2BpJnrE@5Fz&4$N91*t5G^9IZ z(_BN6C|l$Ps!&3A9!eguZ5ALg#$CM#*^k*49wBXgpiRHj7VI+Q50 zeQZGfC+-$~kgdX2^h26wHscP|ri6=Ks90qi3_ znv`Hah4Sxg`2-ZLb2o7Txj)#>E+JEcO__ldpKQS_RHuZW9F%Rct=&Q4FYc=MkgLUJ zEkTBFwwp2}YqO0#L$8z&e}U3JZ1Nfu>~ObJhaA6bUkyknZ2bSYb$vmSuKy*Lzo7~x z%ygingzZW={gc?uowggYOWC45khX`dM-Fu;aZ3S}WNZy3blS_EV-WJm*&c=oP)~0XDA*GOO5XW=J{6HvbQ5P~zPzlvA@A>`-Kg zyTk&ts$m;gf{ep#d&`hq%hp+eYLw`9Lzxk_7Y`H~J%;vv_+LZ8l0~Kf3mhw>C!dCV7LR zbKGsdLvB0U`v+v2XIp4OiUqbp3#wDX zWZz(`OhZ~P+dm7`p~Pz|RN7=4or6w&+{NZ0-xgc11G4(rww;iAo2}`BT9oKqg9;+s z<2n@E;cnUsc>-){A7tKT(`-Y^Alu;%)S$%fAe7r<^M#;Dh`V|iTHR+`JcNv4w)-PU ze!ymoLp4fFor)>RCkZ{Hpe1^)D*nWW$3AVKwI!{56|-0E%Y0yBdVtH*CWi$dqG? zY9Ylfn{*7SQ^M9bl+Ck!OhDl~?i@zQRbVTcAj3V|)C?pmvR(XxUMWFsgVGOdVLKEo zao4p7IUd4&s0YUyW@x0$JbLZlaL7&ZawtT9g<+feIgNeF-Sm;BF@gc|O@%DahPp zb6r8oFSe&^s6mM{IVjg+%jBWRH+R~5Xtm9D^Z*%u*m}y4yu;?NKs8EiRH4i-+tM2p z61@Lkn|OGKoL&D*Ff|~(ge}#Cq}^;o-%ynj4}PGOluaU>|0&SJomT=a$k^Vc&}c8) zd@m%(*$RD7nGy|3DAC7uu7dmu?goY+TR&T97}6-&IwMe<64rH4ae(bb55-m7*(M?H zAX{zG(F|kGBP=yk@&!FTq+vYhWnz^fAK=v87g*2qKu-#{%4kb?BK&5|d$y?~u%AKkJ z`DWSni;&gE_FIC~b8Ma`s6~md&rre6Hunm}=DEAAL7oLR{Rd=TWIO$Yln%E37Sy1` z-8Ph4VrzGxh?Be3uH>Iqm)V}XA)|}!pA3?(uw~^?jS@%uq0B1V@c2{If1fKwzUKlKH{#Lgj_K; z>m_73X1lq9WO25!Ec8kV@f?&sVUyoM!Bg&b?jgq++t&l6OR%}hkmQ`LT!AW-F!KT> zlWbRSka)qJwhq}-Y|#d!y=3e8f;yDA^$nHMY>gl2^ol!&aPcQ!hV4NDS+CiqdLVU{ zE!7LPC{e9|3O8&AN+_1&u4@qT+_HIxAakCrHVi56*ycx}1|{C^rTfhSaUbt&{p@mnrl`UxWjqPb05^8Lg092;LOb|-EvyJRS{yKL@2axT9O%{bT z4K{xaYE!~z94daYEuBH}CU>QC$os`+N8e!fP8!%h+tcP_UP~+pg4~9CEgCDWvOTJC#8a1)HJ|s!+miKa}ie`yPNqC3mZ8 z$UeYU(Lh=i+rJU0Ly6a8P-&2DR1cl1xr-T~zk6-||6Wk<6lB$~ZBIk$VYcQB)S^VE z6)I@i9&J!;guCf^$TP~8UWCkJY#Jw|)Uh49pavy&uR*zSHs3lF(Q{YdfL15i7JZP> zz;^G4<; z6tHlozknA0v7KE)qgJ;53?$651+!3@65BZ_VPjjngZy*cJ>Nq%JDarxY3A9oWvERF zW6w}=f$jJOiZ5~}uR&f1n^=b|OKe{aNabW({(_p6Q2vJU%WUQj6m@ZTC0zc=y}~x! z4VhNiqCJpejZG?t>XfjhfU<734ARUbc%-=#>)G<4}5& zEj$4QecW{!A;%Wmh6&R7*=lA;vdw1y2URHHZWc<4Y!h}!+~F><0NDd<$|Xp<%eJ=+ zbtv&?1u6yE+-~S}kGrY|@`c!DHzDgj+sziF4zuY*s6~nK08}_&>)V535$<;OAi{xG*<4XbdBpYn@RW_$mDMz7fB zn~;!UE3}|8B^rL9#5LRbFXYd1Hy}y>$#%mQl0upsTSo@9DPg@2D&Dfa^h5DHcQzH| zy<^L%AxnX6d>B&Qvz?4UO-fMcpnQ=npogLl+>XdM8 zg|bgPw(W#uFKnMK=#>(jYf$=?t+WmW-?*FhLXH~Sr4Q1* zvuU;=Nu4dS163%YI|wB|*fv9u*x;@nhU}kg3x|-l$##DPbtrK%4wb&xlBdvVi#ydh zZilsMXkGIF-#4ixI+uD9#z zPfi7!*bV9X*_u6&RLQn1hpLqLsDM%fY~}$dpyDn)2rUe<4QrrLHQS*U5{B5MV^EnA zedACh%G(t8lThRn*M%auqP@57i{z1i2wm};dALA};hrBvAVG**7vu!Lv zDm~lVGSs95`zn;5V9UFqsDZnQ4ahyocD4zbjBH9jq?lq0icp;regaU|#J08vg{Qfz z?n5p!n>7L%X4r0`kj%n1b_~5zLi_|u|6`LUprDnzoh0O#W&27&Ivbno3X;sRm9L=+ zCCucYq@C?54~g^KY40KX0$cO}(k`;~l%WnKZdIU?gRN18PM5fIyg@!E+rv9#U1pnV zKx!9TstL6yQT+`SR@e@Hpx7#RT|(who;5bF1TwqXYEnqK&Nkl*H7N0}56XGi3`!`n z!CgWHt$Nu8h9Ki6+ukrF_pxxD?L`lT{M^k>Le6cr+!Ul2*>q+|y2Ey2 zfvS|)Hw&c#Y=Jo_u*+R*9$E;ptvI03J+>z&B!t*3D^QsdGiy*{pKZhg`NP~Dc_G^Y zn`{fxMA-b>P@57ycc9`S+fopUN4YD7puhWb|NnZ-bO2dmY?p_S>X>aP1~n-m5{L3} zHpv+jJ>hQi9CDwsy8{yMKOsq$P0@lXl(5@|k~eJM9Z1Y^ zx7v06C;Kg1r5n=b+5X9(4kcd8q0$}OXg_pX;4U@*`R>_z)sVHwwylBG4{S{>)S^V^ z7*r^+J&r@MNA9K#kf+R+HbUknHqA7otgs!Y z%67j5$=}$FE~rL{$yF#*V;fwDLhsz|Z$QpEo8W`=A8Z~!ByF(0iBOdi=XRmgCtH3G z3N*RXhoOZpwzCK{+G6WJf`o6j;4xID#P$i4XtS*)ApZ|{&q>JEVY6OBnqRi;71XAL zu`E;+w*G&N9p|8U*Z&gacaT@YCf-AqZnm!nNF`-kE<;U9C|96-51aV~ipsdVdV}1( zY{PZPBxj2@AVnXW^b4v}!qzvGRj_^hK;eGw976U_E+t!00vQI_rg|WmitVBodZh%l z0!k0Eg_TfH&0W_Z>|? zmy3{YpH1V0Bw@CQ3#w2;_ZpNuVB1`W#0Yow4ak1Tw%~)bQMP+O)S<-59jJ7~mfVF- zW8A4iknfmnKMYypY`+mmeZuBBf?AaLdJGj#+2&56*co@X3CNRR(_cX5bGFk36fv2Wy?^F5=Wn*Oor|F1qxkr*IR>} zSvIi_>2KJY4M>_}TmFKol=%1!rEb~G9Vn3JE-mE#v~b5Z+zpKu*baLj;hs$@hsu=b zQ$UF#+lLbJKXA7=2-!+(#UV)Z$Y#_+ZA!Qpg^Fdi!Eq@5#9ep-@>bXcBV>7I+b}_@ zD%+bGYEpvzAC!M#%g;j5SMDb4ko%48YymRW*py3<;+-wH4Am*&X9ddE+1A`p_=CHu z2XZyotecSGlkH{;k~P`JMCg?g;sGfA#U|f_f-UZL_94eN+t&f4YqPnckmQH09D^#9 zFmnPWJ8V~Hkoe1;HVN4U|Nm!=XbRGH{V$>C3hGee)-_a;ur+R=({An@dB`VadniEG z9=53mNG)SaJwh!?R9B!vFWW&Cipja_dV@TDY~FXstYE8sK+1l$`6kq$#Jd)hQ?eO; zpvVAsiC<_{#Woli({08oh z<{{f8o6G@ejBI`<)TV?_7gU^LTUvwSChkh>kawERl1i7tj?_tO^%eHU`DQs+oBdAUZlW{0J$98@Sh3(v_&LP)4Tj&BZEU^7v zLb64+^$he%39nfw?O?OrLct~OZtoz6lWn{R>6Y0}OOV9Hrg(xXl(72@C0E$KUm$Un zyVV+GUt_D(A+4M3-zU_e#Op7pw9Yo#hE6@)#X690gRQqK|0k=LZMz#%Z?ZLepcW-M zSLJdmn9)of_Y`$?Q65y^r0j=(`EgB(X zknP?C$@kcdGf<5ZlmDPhh;7gYh4#7Iw?ob_o3IG!57<0QkTk;fwhUD%ac&h#9kS)! zP$0^kegj%KVmsS}Mq_OKen>cG3yM&g659bN5ocT5gZwAlJ?}%dQ#NY^(wwnnqfna? z#*U$4g6;SOil1{QPe9%zo0x_hfE3qk(lS)1gslpc&9Z$|q3{iNjyK4aV=KNxhFi9&1|-X~T{NLrN>G18={vUY z4-_nL*CpKj$#KuNA%S#7wwe@@Jh0h&p$aA3^+CxJ+k_GlAGu4YAbXijIRt5+*!G5@ z4kiAKK&1+sTL+y!b63?vzAD@7BxHSIyP1O2uWULq)S|?=1uDF;_02-D8h1N$kmsGP zH4mBVY%T|+{9t=>LJdlsS%Gp5w#*t7`Q%RPfmWMrM_$PI#n!V0$y;pxZKy_xjU6cS z&9)STLT&CILXh)^&2#|iJ8Y>#Nczh*6oaaico2tD!uJ2yS0rappzD8$-g9U{!uEav zjdrumry)VgR>(kQN;KR+i5|A|TgWftZlD0!df7rnNF!(Kl%O^xtUp1;KDL)W|=qGTKYfK&r)C!bK05)>^cuVM?dq39rY-yO)UW?Shh{K+)LR_TTm8a9gz zs#C(X9Lf%}jr2odEqAd2$Th+yQ$vPPwrvgc_X|+}pZPwu&?_Z4$Dp*1tuzh=$GMv} zKn^|Ir4iCiuxX|tiGeLL163%Y+X^Ko**0yEXymSLhwM{q3yYA}#CE>~btrMt1(l}R zlB>|EnLE`wsY^gF?&P z_1-~F7n^tw=~vjA50G?~ZMh6pDehZ{dcmFJ4DzMe9?l`_ zCEHXAQm5HcX{bes>T9TQ#ddH5#WLJ=S@lW~^n@$Qz zAK6Z1P?Zw<`k+*qEzl1Ip15lbKnoSN6*V;a%=V;#gese51S(TvW(-Qau#M;;|0{P# z2FUitCYypZH8%e=)TV^b8L0Tqwq%9kb?!*~2!DXmSiR~3AVPRWyL;iodX=d57 zTTq)4#zd%SV>=E&@j33~dyv=8ChkL)dA6?uNVUMW9EF;cP>wtg%3fx;`?Ir5Nem91ES3~Ovt50K2wcJT8?E{i*ve}zZg%a*sP}0XX@dJrl+$DY?yPr)dDg8;i z&9*0nI+XY$gGwTsyAL|u;jY>b`2uXSD#*IacB6*WK{nkm)S|@r2vpc(>(fE85O+Iz z$g|JZGC<}qn`;VE9e~g+wO*BPi&t(&?_Z4_dO7*dsb5KCRT{;gf^s^1$L!(Nz!v{zh zV3U@iG9~&dP(sD_QHA`2+%3LAHZ@!E9nuW385>ZW5-yrhQNuR)4aJAK3;#e~Et?>e z|700q+mJx2QMNZJ)T9J^FO(l+%lAQ19d{E-$UV+>rh-g*Hsuhcm|zPILv>2{8G*6} zwly6Tp5(5ohg?QB>m+2DV!N4wWG1#TGxSOcaSN25W|PlCK{IzdbC6?(?Q0&=S=d|- zNb--Z?1Ugf0be=m$5b`as zJ%k|ZBHPpfq;{~S4xtt$s$)=LiR~Z`#hl!Aok5;uHt#uPcCpniAms|%d>U#{;#~&H zt+E+zpvW3`iCbvZ%{EYgjO%QBMM&;p>y)4xCAy!W%m&-bGZgZ2H~R`XH`#JENbh6Q zeL&JJwv$h&N{M|fDCK7hw4uN@cdZVzAhNA=J^g8PhwZ5w5&~=%8C0gkj2uervW@gZ z{vdZp1CVWxO{Ru4AvV7TYE#0e7Ao$uEsa6(Fn6VK$a}zMGC-CH+ocgw9kLBgLrqGE z%s}}lo5Tu5kGR{kLGBpayB#tevn?z_ia1+g393`VqzlTPu$`|$;ZyEZ>yYb=Ewlj{ z5^TSlknEgo-4DG|!m9|SlWewKD0so$?H=Svv5kiz-6h*;1d^oL6h}~n5_XTF=rqS&EC=~+*?R9FYo2ZU9#Y@2H6NfB zB|6Jcp}_W7fnxXEO}{{%B3t?mGC#0s>X5R;cG!R#l-T_R34@USjmprDewt~}%zVB07_Iu%>32uTLn?2k}|67HU$q?&D_3W-D9C0-%B zhE4enX@}YNKA;XI{(M3uEt|UqosMuvU>_53Hbj!h?pT9g=>IF#~00*p{qN$im%&4RZctGtEPKD_d$2lFqUXIiV^g9=M>CjZLx!1?IT( zu0soUw)YKabe?VA2MG&o1wT}#M8gi0SY$ijh5Qcg211Z+i7ga{G)}fo1Zq>l`VmxI zW_vk?;x6uNr;vArEth~St8C*JkZO(X`)ok3l;He=(jr^w8w&1l zH{F380k%t_`X}8ko2DC*1lb}zP=yk@A@1r*$iC0EFbHYGZ1+P@hY}~X zQ0agzISQRdxKoWozC*VC3CJ2{`<;Z;M{FJw)S|>!GgOGN&HaO7$K2h{LY_FA-VT{h z*iIK9=sm`#8DB-q}YxF zQ0S7o-aW{fW)t@z{S{mD0Fq|dmZMOW5+7qw>YB}b0tK?%rO%*+8@AyjG@4^OOhLje zoAe4QQ=;!0O61u-ZXo|1cZ+$*R$wa@Ak96S@d0X6!o?$0EV2z&p!frK;VR@Uu?cUG z<&kaU9a5Fq-aeovCD@x#{)sK$f}$1fCVn9IGuzoOWU8_$B`<$cys!nOP@NKfWKi~% zZLJRqzj0UXhg>x_s|qr_v)!m6S)FZc7<#3I_z0B#V3X^hV1v6IJ>>Xg`!YbfCYx&t zl6VFAJn2m^)^)KW;@t{Vp8t9f{>?&%^QNuGPYV6QueaVA3_aE zygPz&ayCO8iu7@pIE7XfYy;l-TzKr8I1TXDBetUF!u}(6X)6pwSVwr#d8zvROW%G9_ldpu`y4 zNE`C&xI5}VwsAID*Xy4&dNzMI)TV^b9;i6Mwj_t*2JT7<$UDhq8h|WDw#z|CHN`fh zftr*M(L#9>n`8`%PII?84!O;2?-P(|hHb$JDJ*OS6I7>!$r&j7kL~;)6t;4wvO%s{ zwvZh%*w}s-AlV$-`V#a?39rjg+RkQMg@W_k-MS&i0^9foq+4V=-Gn3#HiaLmP{OVV zC70N~1CZ$CZgmf`FSAwlA+3w;Uj*t<;&l`%t+0(AL#M0U#ZDmK8e4Ayvbx!}laP9y zt(k&al<2&I3Ldt{YbdtC-ExrdZKw!;UgL5baED7VGtt3VMycl9c? zy3Mxu1{p=R`*%pb!)9ziHA+l2p-g~n@EZ#4a<~5jIfHD1@b)MD9-Bu3NkeRJQm9Ia zbG=Y%pDo`91;X6vmC(Wg+nEX)jj;6(LBb(ha2P66VtWKiMA_DKkpGCgXFX(#u~{b} z%`sbc3Tjirm>DX@*^Vtx{DeFCEaW|96Xzhy8Qa%9q)MIn3^bK1$4h3`Eb)7+uTegjJNS9};T|kmMHhUVXP{LgXN*35AZXofVyTmPIFS02M zkoJLXuLyN0@uvipN^I^Y==71h>NDgkv(3Ij)+e@`8ltf7F)^)Nx#{KrlBe&9?U?gHk-r>1%9~m+MtCF z+q)ea{bie9gaje@|5>B31eGb#;DQoeZ0D zVS5pwxQsj7F68ZH%k4oHIoo&`QuVQ&M4%=mD2|}Kf-P_iMfrvwnzi2P(t??lpJT<{Dwq5cl960KEbviy#GmSV7r$<9ZH<+ zfl8BX$zJHx$el_7`KH+Rm5|lM_N#)_(`=q0s6~md!%)G@Ha7~zX1Kf6K^_a6egZQ8 zV>_LMlvcKW6V#x@T{D!MWouiYh>g3|S!i{R?RgF|+S&dsK=OIEtOKf1;^;DzSztR} zfkKPi^|~RagH7~6`X#oe7m_;JmbaiPB|dILsbw~E01CLcO9!Ea6}I7hXmpkB@Bk9l z*rZXYOo_f2lyI|s#3BDWcZ+9`&BInahcp{(#uU`1go`v(^s)_JL-9@S!Z(oD$0p<< z%NE;40aE$d-ilC@66}vqew!`-1Vu&eCaREohwbbYG6mR_?~r1bE%*V|DdFc6$_Cli zT2Oe8yJ{P9h1jgWkYS(grmOxZS(t513cXT7Tn42N*yMdsFv8tVKjb)M`x=0BQ8t$v zk{q#>HBf~TW=5c7jO}U+5|6pl>LGiaEoy+YCu}`aP=^w?rlHa)TVn<~J>$+{g?tIN z2ODHPXPcUb)Je9~BGjTpwG%2_upPLdScDcN<6WRJVX8p zcSkRf?U_whgEUn(e;sO5!e;|2zOXHQLGf4aO5c$8jm^}7EH$=E;p0!LcebH!s7VQt z9w=XDlgOdy2X~ta$lYLjS3;&wwuM1R(PS$OL3K)))I!-Ww)0Ua+~Q6(4!OSBLKBdo z&GtJ9$$r?@P0%YPyqckOht2j63jT6;I}164z5lPTjN2hy*Z&eu7a)m*O|b-3C}DRQ zN_Mk-uRx-dyHz)2?_sNWAgzq;-zLVgbn4$JV+%AD|lb_n{UjGs-qtfkI>4?N=eEj!k%j^y6%vcSx#dd;5T@ zlsMOfQWI?X78EdWr~iQ#CfUw@p;04SzohXeVTvs%h02uJmO%*<+gcywpXTnlAF`R* ztSU$|!z@>`!YZ(8{6^})TD&+G?brXGh3jj zox3Y5f~F&yZ)At@Q$#gKVxEq}*eBszVJ*ocV-uA-2pH6xrua+lE%dY)2i)c)-@v_4y}x zgw5X#)hMyi17!}`mgG<<%H4wkavrgn1|WTmEj0*9kJ*MaP?Zu7v`{L}CK-bQC)|0* zp@mbn_X%k9jBVZs2?@4>2`W>fVFpT^vz`Bg{7LQxY>@4OEo6r@DYni6)TV^>C8&7G z_OcAc)7;rsA@3Di&J9^IY~ve{>YD9j6KYa|!Vl%MYylC9-f;IFfZRE@l|9IG%U0Qk z6nQpF1gcZQbrj0pv5g!<;R1KD6UcSXCQCquBHMNnl0C3}rl40!a9%;_5?kpS3O;f- zor4@@w#z)Edt%evLy`(xNsLU$QTKC^9BAhF6_y$ac1*cRR(?JL{;JJg}X$p%z< zV@o!n(;9cGZ^-w~w*LcJ>ukTjkotqoBWeDrMTxIcsL)`W>xE*U+}-v;o+g`K37Nmx zPF0Yy#nwLrH7Id+7|MOKwMU>xo4ZvVwEDyLtcQ#pwtth5{Ff~|1=T2V)C^^W(Esmw z99y7J*Z&fGXCbGA?eE{(pY+{q&3Q;FWm|SYRZ4txLa81$^9mG@ahG0$7JAu+JAj&9bWUBby#!HzGv4s+f5yk8QDhPp&BJ5KA_79HhBvQO>(!>hMZGuUmZwq zVsm$W{Yf&-R_=x>lrSTM(q^_RIV8?-r|pOA7Pi;`q@88!Q9~U{+|ocLD_c_wCFi(v zjzXI@w#PBZI?py`gw%Gn%p}yJMD;XOSYSJtf#MGCx~!0Ik*NfBc!NT3EKO!PpxBet_%C=%yRrGVVW zY+)s2OtAf`AlV7q+7MKugxWB4nPju+pwKCIw|dB#VjCNW^k;0z2}qJ=Q<$I%CG47^ z^f}wN1rjsddFCMd1zTkv(q6L7I-m|EUOS=E6`O7uN@lr>uRxpEY`tE{nq%AcLFyZ} zPe0V6MAsHn$g@3dL-AYgrh}00j_oo8neW*&`;fB0c6b0aDX}{Skmu=VpnS}lS&KfZzr0DujqI3#sP{P(Ul#{T%&p?rG?wnT0EoCd(AY%{P)B+@v zv7IkMHA+yspvzvih#LyYx$9bmoPBKTYmi>SR$qrC{cQG4s6q+%0Vu6x8{dJ%0q#<} zkX^;53`5#Mw!H|{p~RmkR8q5fkD%lbchzHPQ^Piwgsj7CHz`Q1Wz(NSElNyWK!p*u zzAO~iakrC$e4}iwJY?3hx$hz67~As$)TG3j5|lTvWy?@>oIC9clHgWg(4Y{V-OdZHzX3GfeKS^iUhPt6DB_8xZ84H_44h3hq z^D7{SmF-Ol>E_rL1|h-5Rv3cHlxWmKsd=_D9Tc#0H!ud-7TCh$kjBB*nSk1qux5ga zi)^oEDBNs`(A;(9<~)PWLjmb z_#lOs&9VVCDB*ew%B``Dh)~4GT|5Z6*V$xykkQY!y${JY*gg)R8YQ@5(B&puDGr6U zxSKwKoB_6rQ%Jwfrb$B*ku92mDwNQD1*Lb`Hm)Ht$X(+GvhT7v?jUW5?V$j5C~@Ku zD($hQpP*!zJ5>eR+-KXbLe>b|Zw*o(u=(Dg7A4l+p+b~x{u7EHa(CN;d@(k|4`e=K zOa4O2I9tEu=TA*a+?7K4W45*oiYB=8^g*5zwwHcrGRZcpg5;-cIW<(LMBOlyO|c!1 zK;bj)di9Vi%_bTk!#Ufh5t3%umZzX9B|c3Gbl*+NayP?1hcZ;i#Ezee5gEY5nlYXd83Fn(o@s4e98%o@B7ukXQ1vViB zSsvKd!;q@T_Adf`QiA;ux_e~1JAz^*?#2_4_lfN^37N`l$}>pu%oaL_8kF#J0p%)e zt63=W!d*27xvOl}Tgdpzc5@HOYHXuLs748i5_I{GP5ump>fG(TK+ZR|uUAOlU~|_Y z$va!Q0aYkr<^xJM*{;4I@q;^U8?t}0#X6Aoi>;@t^QR6aZgoSY7F)9iN`7L}<)v)K87SJrU1}Ec$k+yK&}1*$o*k0Q**XrWPKn+nDBH*Ox(tOC+|79)S3g_c z3mKGb`gKS;z;?0$RVlGA0A*BcK@kcLa@Pt%4mI1#9;6#$d)|ix4Vxtjl_@bBgHpq6 zBgarc%iYllWE)|Vr67%tEs%!Vl<<*(ilb~xS5QLFUFjO~kFlBZkj21uaR;f!*@hmV zPfCbBLU%?sNg0YwaJNx`ypwEiRme2O=J*FGOl*ZWs6h!6O(-|bcJ>KH%-pHIA@>Yh z_y;mt*nWQ@*(}?dv6P{Qso zlwM%_9)Uy$cOE@tUu3HoAgz;ab^_{9;`J0%T4K|gp`?quxCPo=X6v1UtZugLc}Ts& z_PGGHDADDF3Lds67ZhLRZh8gsdD$*kA@drW#s?{VY=?fRNr~NCP=1|la~q2Kxohk| zo(;Ce5Hz{T_7H~TTWpgDP@NLfhfp@aHW-J(+uZFZAeYD{oI-{jHs2W}4YK_^hpLn~ ze+gxF+3vDXFvOkV26F7No!&yaFk62C6870bMW{@P?GluVu&q8rfdlSdULaeP&02#r zhithz)TV^dcc>U+JN|$YN8HITWkPQoUuvOP=gY-G*B+h_O68@=iE6*A$NwYI0hLn*rtq-?2_$# z5~@*xdK$XCVvEc`p)7Y@R>*nHwr+#;IkvhTlH9P_7oiFz+%G}tJlnV%5^uRnc_8~8 zn{o}(-m~qkLmf)|*?>v~Hg5n*K5$nRq0J)O+%9B&WV_je)Fn231Zq)YA_^6r*!qs3 zc$vGMW61Z+);fXA6*hMYQogV~r=cb#&RjtGDqHpnioSBE%|V_T+fg2x{KwXF56SCn zfd{BgiOokS`^L6ZhQbZ*9xIURoz3(L8JcXFe~|QpZKwfNDe<5QWj@&?Ur_LiJO4N2 zXtBNhK)P?X1)=LtLYu81fy$I4(`g`;bJ-7CnF}l+Yc6(j#meaY)p0*GNG2Q8vdZq}8)MoIxE* zoX9|>F}Cz2lr(Usx`sB#+4gTBtC8*Z7E({J`3g{r5^F`MFv&Lm1jVPgyM2ayCN@JA zGEcK5Ymm~+*8c`IDRK85%FnR1KcJ|EJ5LMp%(A_-p-C&->@OsrW6O0({#2(#ofOL2 z*p6jTc%Hl7KFDQf6Z;{<0^8>RBz3SYtD!0-K53xLBAa;x3Oc#F9EBW9Y{Ldf=VCiF zLc%hebP6g{;^s7zaqbL zCagl1b++|2Nabhyw+?+$f_)RZ+hDs3K(S5k#&;m^7Tf7AWD2k;!;oT|Efj$ol<*UU zaw6O65fs_su6hi)gKX9$WZY%DNkOs@+vqt|qlCl-bh*bS&qARvcRM-AxzF~Mhx8FP z_dO&zU@JdB6-t;XLFp*lRT&ZwxzoNt_843271AED_0*vbC2lpKQk<>Xgp$YHIlrLI z1l!{`WIbV<>OkrwTSn;qQ;QPS-B96_?VtyWr?~5qL%uUMzXCF++3HG2dCs;l2sJ73 zehA8E*o;~zdcj>v2YD{p2F9StE4ICHNSsQj#UcL-o9P6yRM{?0A=N9}P#XHA zglGo3tFcM0px8g|Hm)IWo$c)gGQF`m?jS{jtx$j(lrZrK<=)xOo}fsRJ5>d8f3StC zknxl4w+6|+*w)^l8YR@;q01JV?Gp-pb9dW`I~Z zFWa{a5{2l0_bZ-0$lmpz#7aM;m9WjKpbjNotD#ain{F6NO1X=VK$|^my?V$hW7{@B z>Rz@_Bh;cq*A!Hcvpr2i@jmXREs#&ac4>vo{cM_fNU3ByT!5OC*zJV!18kcvD5~PF z;f6eeY>TVVq?+wv4U!MBP5PlaC8jr_tcGoH8ww9|x4#3qv}{5MGK{eK!jM$Q_Adfe zDRKS~%8atz9YH}ocZLMy7-Kt4LOKIm{~08VvxUx~G9|Vzpp=nqH46nMxO>S#wn;YY zEu@)Z%iTk5N*FCdMHAa`2}(?JCx3?gW;XE!vdpl3y+SGr+j1THq=a$Q^kj%+; zJ_yw)L9Kx`;die;iQB$L3vvlIz@6m!VBR+nfioZm`{WA@wGkejRF2Vqya-Y_atP zpm>109TD^|Gl6%>we_jnDt4%ke2$Pi`A+(FVqwxI{8N{I)LP$tGEDMP^{?)(+V z5odd=Lb_wNg@2HcU@N>qWlA(Qq0|Z6*(Vf8ayRe|*-qKQKaeKH*7=3nl&~g|{i%4y z_9}%EY3^*jkpG-5-v?PTY-37Db-{L`f<7rhF$CRRvIU2s*cEr*Bak=CwxWkj*K8F7 zq{y*ZCZGl-Tu(u{8@3TM6v=ZJw?OV&HrX6xykpy*hh+C`9}7^85?oH`vcOhyL7@lk zrdJ?mk?mp?(m%3ke2}EX7WG3FO6cB#(obv~+mKl1uCW8zpV=HCNLyih2tyr8oH&3= zFKp>UC|TuB6^Axo+4d8VwZ`_Fgw+4od}mOL5^LvBq0Tmc3B}*IyUjwr2AkmqGQYDW zZy{xqt-k;@DRH+5 z`IT&f6|xMlt=k}#itV2r`lJN=B6K&%cDDq@)ZC4`A@2~|sRuG?*pzFKVwf$o4mBv@ zX9LP<*;WHkWQ4n_2)T7^)?LUr%678{$@FZa5vWG}eXT*4V{GyxC}iMn=NNL1vwfXF zdLx@V1xY5@%4w)V2{RW^dXnwx3KFNd)8-(%i7l3gw9{-o_fUrtw;rI9nXUNO;P@NLJLr`{^?R6LmySbawL9P|H zydE-m*!1I&bd~L70;*DCp9#u%*@9*$xW-+}0y%taD|3);o$YxZ68vly2UMoStP@IY zu#GH3flcm?Rv_CJo6HMo0&D>v)TV?FKUCaiTiSvWB6p>2$iKs83PP43+eHXc?XnH+ zL!XooJ%H{)Y?2rh+v9E{4tc|DZwbh>&*nIV6cM(<8PuSJi42rGU^}~nB2n&C*O2>= zEqntRV{E^-knD(UtpL?1p;m-0<7~DkD0Ix-?K9*|u#HtA{RvyL21$}^iZ`f23A^u5 z`jqYa0}@l*d0LSDjIGj!v}v~4U#LTg*Ij*oDxI_Gq);-$U0eojUawJ6c0h6-7>Ck+(8=5Bff^5xhrMY) zaefoZJhR;epkReN!w%$lVLRQ0bXB(gFeJRPg(6Uy65CNIRbyK{f&%}zdpU+|bvA1f z(!8HOSp0^JUH?g#>Oe9H z+qt0lQ;ibT-Oyz>TcigHNxAEiL(U$ybp@oCvDKB3q?gS;2vsQIeh5m-*~Yby*vDN; z2iX;D$}vdW&$c%Xbtv&?0xBulye23)z+Kf0ZK~MjW+Ce!+sz!LRT{14yZ81 z*0%)3HQeniL%w0Q)(T|SvbnvGa)j;K2Q?{iW&_IW*s@ztbd)=-2zm5uM?q+EjIC!6 zk{j3p`%s+{n+H&KoNXxvg^k=j#v#`Po9P5HOtNK8A?Xy`P#UUI;z0(=nAjv&P;i<% z|25<=v%TFwx*4{GJ4mpw6$((95{-{gYL@Nn2?|)b8>m3GIks>W(%9HKHKTVwk$LN!WoO+lAFw$d~d zTIX)s0y+I`7gk8W!KRsqB%5r}1*k#^-A*XI#kS#s!~k~QLf@ zA1dvzr8l8ukUP~jw7JW+zXMrAY`?pZdXLQ)hFX+Zi$H}i+x#IE-{(q-5VJCJa} zChb!GsZ5ER-B9Y1?Y##ITyeK3hiqB4q5{%fvrP^_ZAv&Fgo-(~K@F6+;VzC!xE0wmTCPD{wbH19>0VPG=!gkxglX6pw5nJJg_r9|x2x zv8^saktgn|%aFUwX7xbEXSN$JB&)EEu0u6SNNhltFKqGv6smH!BSOwswyz+hud%uJ zAjv}XI!at}aof5s0fj?z?*j}YjSjOF4FXZZF z%lAPBIh$SyN&DDNR8W-?`-Y&5f-N`<1^c;cjX(}1+ln624X`~MAVI}unSjcan4N-B zgKQ&aD4^!<$O74h*kp5%M#C1EhuW0zu>ci^*_ND8Ld#vr1^GwVOe>H@$9Ay_sYcm` ze9$K)ME%g6o=vg^#m2bX*oM3YwznO~G|uJ-K?);VAq+JrVd4PFO|YFELXk=CRB^~X z#THILMibj_5|T}`t(`$NN~oPfmu5EGB@~+B?lucKEo@^qkbagec?(IbY>EO@p@iKc zl%8YzE0vt@fSQ!pt%mZeY?~S=>gBGX zg*9j@Zl>P%O^f z)fMDDW*g2yrUY9o4=GOAr1wyR61EOYX=ip?$z{;5I<_Yx?bWgG8-#B1(Sy^uY} zrc^-M8@4?q)S<*56;#Tzd556nEqB#nX!DM3P6t`<*>3cZy1=F%hgy`Fn1BioY<(su zUgU1a4EY|}S{BG$VspE8Ef*6s~dixDC1fv6+I9q0W{GLDDz2p?#=Ii3bNzrokqOLBV(K{Bg+9 zWP3|Mx(~L6Q%Lw^E1W@PN;GDm)EC>?B@}3JH*gKvzS+VzkfzPnxrN%4uvUPIKWwi> zDAD20_5}HV+49ejMTq}*))=cos;>Vep46aEN>IE(cM`VXI~42Y?)wAsO4(Lgkg12Q z(uNc=Hp?&6poHr#^`CORY$H-ABIhnHgWP>=vOdVDVB7A8Wc_R(15k|;Tx#f2$yU-p zp#kouM&-$gE1Q^u40CLsc}QwwTfT>?l=$=jW#-w;B`9d;?y?Ly7TAVgAf1El@D&mk z*`#%-Oo^KfDCJ~(Z$g13?iRlwn~SaZ4QZCyCOc4@63&I8KNa0%mEwyuCwUbcTq=#vucgV5a?+uaZp^Kmz>g}m!*r#i^wXH$+riVe2VIMkqop9v_p z$+l{OB3s;5&5%35W}StM+iW*;kW6G7wL>*ZNI0O&9X9zA6bf>;vkWAVZc- z{{u;{*-n0;Dkb(wG=IwE*n(0hc*9*w208LA%>LNl4OSQ=CB+O4vPz(%)>~ z7m(QI&Xa}gKWvp8r0uZH-a;KpyuOD@zihf9loXEtyI+Zypv|uTB=kN*Rteko3#9I5 z`+S93l<2BM1u5H81B&->H~j(mWNepTkhzyl(}t9Cw!;q8q{Qy7;Xmd3*fzVNsDisj z59H}*Ta-hSO11|DBp+a#9DwSSm>z_(Dz-rl6dvSmUkka^Y=1wV`6t5=n{Nz~YS{jb zLsd$gpM)~QYbDc+7-6FAE-l#Kfh4P!{(J}|CC(it}2B#y=-&6kadmirVmp4*z`)MMTrR&R9I*0 z8-n6~?skSD-v(Q21Tt^3x%H58i|yF}H7Rjs0?G&2vQtoWn>(!;@`!9l7HD#Zt!ECB z2iXGiP@NK+3s82KZOI9RL)<;OAlDw7X$3Nb*)pq;bf0a=2URKYzz=02Y?3V~c)*>1 z8*)V1-gY3}A=^R-5@KwHFjS^Q;{lX9Vmmv80&(sJ;*jl_Eu4Tf3ARoWYE#168B{!B zdp(B|N$zZykpGk|pM@+bwy_&Xb;fpb3w=_8q5$2c*@8tVcFx^*3G!yxR-Pf#1zY6> zQe3iGYEXj`uIo_lif!Z_ie$Nqe?abQHdzZY=GeB|knD!-qXX3_!PPbLr^`HBsT&I2 zayKo5oOf&&a!7yArs;s9?O_b+P~wCU zDm}5KC!u7SJJmF_`OLOI16eCrnoUt-S$78{ByUkmsH4MT91sY_q$N{DUpG2h}N27lE>$Y{yY3 z{KZ}G5#(yIiN}!Po9*)ilD65FQ&5!>pVCm~hs}He1v}hbUO|puw&5J46B7TOH4gKT z(Dk1L={;1Y#LWjNC1HDigaX~%EtVmhl&x5SG(BvSuTYy3&i_G08QWk3O7wCUX+nNE zoA3o$`qQLgA4=PQvHT_U>iaX~P zv}s~{+=i^vY*Rr6n1hq_XN3?*z(Vi z!NsPpLegcnlNwZ|#J)Eu<7NxKL%|j9S|5w`3Ywm?7Bri70HsJOwlq=pik+?6zte~ZmD0$BoV7o(7Bn{CJd zeNsZy2;GTnk|`**!`;R-gt=3>Aoo67 z*bNyYY`-2zcEGl_2GuB`whmoJ*=(Co=#abH0OX9ZjqO1CBevu&B#E;r!cc`0b|X;w znC&|Xi3#pJN09x5t#S-$lWem|s6&a@DX4VHraOm{DemGI(B>IiZx*tq*|u|#`kd`E z549-Kbq^IXY)=nR{DQmb66CvNyDUTID>lswq|CA%zCuk(?5;!kYqre>6wPthXhNPF zw#6@KGSBw#4aslWCOc4_64QeIPuV-R!EPvg&)t3x($&=2Jk@BHKSDRHelE zK`8Ucb~gkCOWYZ>kmHH%R0rwGZ2e=9@XQt(hsu=Lo`6ynwp9}pc;W8F4B4t|)>%mN z%9fjh+LSPAhl(|}V+WM@$DMo$^4HnKWytcz_O$}38f?p6=#vu4KIrb9&Ab7{n%rG& zLEaCxVG%NYvc-as;)_kX2Q?^RYahzB*xnDI$TxS+802oV72}Zchi&Qvl6Bb5PoWwm zsMFBpFIyx7g@lv;?pM05AZOQq64$RGy@aiP14+8s>~~Ow67CC7TFN&52#G!1rJf+W zj7?dAw7qP5Rj5OWKQ*W%XY;;6$v*C?@6e`#ZSE7Y_OsozAhnWB{{yusG4Tr(2H5%} zV}FXPxZ9CJzCpH@3^J?P+|bhB;0K(ZCKk5{Nh39dSH>0v81pwKFJ z(;tx2%XaYv>DSmaZAju{i*}$2C3JTg{*+#4+vtWwKX;8D$iBhmkVD!{wg&~&p~Q&+ zsIZ4Ie|g6x-nmB%HBH zQ&5=_H`7om&Gw#w0_WT5fhL4Jq!~LO)Q05`KQ6T!C#> zGXAH?19w#^^H}0HHXtTlg z=z^^8Y*QPHv$pCH57djEpTH657Um!ujW~o7CO3c=wR6pCu zI}}iIck}_-2H0dRNTXs4w4pX7d~~4VAlp)x@lOdgccpH~Kg4E|K^6_$g&a~1vkmn_ zpOg?CfbO(x5;YVX;ci0%d39`WTF5lY<`{((dbYwC)S!e3Ba|CsJDY?e2JTeTkb9gh zJOdexY`?RRY=Ujg2GuB`W``~(*=&nYXo|brCCF)F8*@YYX||*Xl9<^PYfyy}cGscw z4BPhxBwD!h1R(n?TSbJlR<_w)s6&a@dr)bPO&5WZHtyn4Xmg&e_Xx7u*|v`%^#a@H z3DlxQR|+aP*q+i*e38593&`hWyS#$TOKh4Pq;#V4?)lr3@qg;Lyg#USSy+j<<*r`hTWNOI0*KZPokaDN7+Gi>7-NW9=KbqU!o*_79i z_KI!q2I^4a&n;BSvUv+o@|wG95!%eL%{@WZ8@8KgNS$ZXSD_XqCTdXOmaXp%ir;a! z^A7p$*;*fvxxnUbLCOcV=Qh-&#F<|xUu4U6P5vqR$emUSc}i?YGHCLNt)~x?m)QdS zP@NK+15oyvZAlGVjPG~}qW zz0E+nH?{>UBsACxHmFRA#sw(#&UUs41)AIqxFFjHTi6Y0KG`}Rs7(oLYf$lv?R6bW zw79cvLjG^Id;qev*~WGt)eqarF7!zWiZFE7VGBl}*e`eAQOGNt{&&B!as-*W{*zES zh7=MuOA=~O!gUJDb+e6}LlG%=@e9b^!zRl@Mj6|74wCh>S(B&4JTw?lDD8Sv06mo8}eaRrb$mZ^Y zBs*;7eyBnTGb$(@WV=#B;x2dEVaOh0i;Y0qJ+>Y_)S<*J15^sLHH}bmpF8Iiv>9Q0 zoQA9iY*Q9U9c9Z{p%x{o=b^$O+ra`9k8#)KgnUPAeivkpv(?>@@|bO56>3uA{Th@{ zuo?YO^n|mTe^q>8{zHbC8f@v)n>uO3dCvsT;PDA{5AT zcT|FGw`{U!NOQ*)c!AoK@bL;2@7b2>P@=$HsR8*P*i0XgrO0;i1*sm{hT704B}6;W zU5QQ7HT|d96L%ZkkhjeC)&rTI*&K36QDG}6pavyO3_!UTwzEMfQsqvif!wcbVJ&2= zvHj{G**~_mF{nlfwQ=aO&Ssm0LT}vNnjmL`ZEOb8zq2K0AxV=>VS_4^uxp3XA8g+a zNc`l^vjo|{*ec7Aw#7EL8wKE zu05zAr2c#6@w5-cyZ)0n9ff=nw#yh~?q<^*LrN*z;R)2F#O@T7?_t|aLs1!bjSS@J zWm~+0Cgp4o*O0uAZ88tlDKUKqWfg3L4^X(DyZuMVrDPMzkYRw$SAnD|wtrQqN{RFT zpv)lK-5V5Cb7yEmjv=>;Hy?VYbi@RHpv^jsZ$(*;XayKLtj(dyzsm9hhD1*bT`3fo`7PL z++9sU-YK?WGh{Nc#Vn9wnoT+fH7H?g9?F^7-WQZE6LQS=r84 zp&BKqebD6`Tf`5AY}|EiLC$%$^=(LRXRGf(k_9$<2&z!ReHcnR*v1baagn>!A!K*5 zDdUiKiES?dbtv&C36)%I-ZLn<%w6>y+H|wcT|(9swwo-Z_OR)1pcW-2ZlS^|TVDZ+ zd%4>wLcTS&RtYlu*xb*Ma-Hq@1!_{_ObyEW*|K#gy1||H9rA3l9eqHPTWmcoNFHDd zw4pjBHak#un{BCU=1*afyT@+GwZmqTL53h(Mh;1L*@pU|DkUBaK$#GmL=6S^xbtfu zN0{wR3+eXR7Dgclrpt~ena2<-Ba`(Lfc~fjF0myX5RuLgZ zn$5BcH7MbF56YdhjYObGhP!waa$m5?jv(VD+x9UeyJGt|fohcCNoCeiO2m*nU4D^%I-#8){Kv?FTB9+2#eypW@Hl-AW)|h0V|dnP1qF zy^ylX)~|q?l(?&e@~>=d6%?&;=NW=L|JYuJp~*VitPYaDvE}qoof389P`1H#JOPE@ zx$8ASt|pskh72EUpB6~^$+kQPRVndl9?E>NnH^BD#oeV7a(uH5FGIRE+u;f%{IE&A zP?-`peNd{y_U?xQzuYZuK{ny+zq3Yh8`516#Zm4EKetd$ z$+lX6A_Lr2i;!EzW_^N;gKRg?kW9@sT7_zqkf=eILu~RlD5T+T=N)nmvweL)dM%r~ z1xZHO%5A7Z2{XS?TE}+PHTx%Vlsl~yvg_GmGDthd*3$=dC~>PFDjC?C15k3DJEt1j zG_pNvAnOF%)Ci=WWXp^~ElN}ypu!Z}ff0(Exa*pNeA8_HX~=A5tIt5n8MXy0)TG3F z81;1YMODCBUltsFtRWwz&INN}@Rl2DlvvneRG!Zvaa z1w7mxT|l-~Hdz+Zc-aCus7(nUd8oL?wsa3AeB6~DApbg>sRUX4Y!_unwZS&@0)0|K z^cA|>WRujP*cNvi4agf{duu|bZ8pajq!8H(-%x`RCOS}VhwV(T{wWgVPSp*$ciF-{ zkTJye+Y8C|*wz$KjS^}~=rYV^8-zmp+}#dA&IsF>7SbQEC3TP_%BC2DDwME04y6y- zz9%3t#+}Cm*^k&NW=I=no1KL^lz2S{m5$kTb|{(PF7ALfPuO~wAZwCsdl^!nvVE>V zElPBGp+bu7$p^*HxSQU9d}+4JEy#S%rV$}!hV3v2H7T)s56WM#ZSF(SOYRy6kmrhR zF$PU$*&gDM{F-g@1gcYF`V`9M*ap*3_=dav4CKnQ30IKemd$q!N$=SH-9S}JoWFxI z_iT3sC|Kam@CZ2`*iN4yU6HN70tt_7p(<3S#C8ozmDpC_puiJ%FYl18%x3+BG|z0g z7SyJM(I2Q-VLSeX5-;4zC3AoBSJ^}8wb;(BP>mAQ z^U&otTVw$WwYlqZLe3wybr+=Xu+`m=HqFm#{H1k^`C^)CS;ee zDYqeQH{0G0)S<+mU8p2w^M;{h4|mlFv?*hoJA|ygY&S=cTF$0VKrKp4B%wkdTi+QJ zS8%s;4*B}oS{IO6$>z>N$^o|L9Mq)5nOi8YV$0q`(LwICMaZLOJ1RkwLu@_IkX*wS zc!BDa*nEYu!)!}+D6Hk~u>rY8*i0XgLC2Q)f~2EtLv5%^i3c4hqi2(J+5Qw9Aon7hEC3muY}+CvTVneNLN!Wo?Ln6=w$eTnTIOy# z3OU_u7coe`!lpTfBp$Zt2~?qk?i7?>W!p$YqL;fy2C}cQIj$hBkL}?a>QLfD9xAP~ zrSG7mpF7n9w7J2y{|H$(*?ylO^%k420<|cyR)q=yw)uZhe4D%5H^?Wl8JduJhb{RD zDT8eN-%yhhcYmP#E?fH-iiWuJNap|K*<*W=LX%;(*XfKcLfHu0u?h+w zaMwEoxuR_1Fl0Dn`y7F!F}7trRHeix1C%*pGfzOlICqy*kmHzb*bM0sY=;&|IAN2{ zL1jwZoQG0Lw)X`naLV1H6SAe)iY`cV#x}VEwJG6z6)L9L27OTCoV$o0@@Lro9zp)e za>2H~4XG~K{_Q}Ylwc1*cUNq8VJMd6Zu|i9UbCGZLZ%#>G7c$j*g^@YK?y%eD3@nj zJ%b{*+*QvZ_Z^${5;ESi-DDwIfo=2#s!>AX7P@?3lNX>+k-MEDA$x@__5o>M*m_z}hZ48iP^rq+>_Ev^?wnorKW*07 z9=jpyKej0uq^`4NA z@2sJBL(;DQB%FAlDkb)}zph*|o!!{&eW}6H`bxKT!psbs1a32bPuSL{A$^jqJ_AWk+3Z%RLJ9XaD4k*(Ux36j?ox}8J$#2*h-%CLFYpyUO2)pcm|l5K7ivR<*>1R!;mO}_)RC^4}M6|UL(!caWN z-A)AZ-LSQykU7uhK7y3DY|qC~lM-i=Q2vfBn}VYE+-c7tPl4^|0-Aha>&Zg$;{Qpy zqi*E301Cr@eM>@;Bz;Megd|ClB-ug|k|arzBq2$XBsu1pm(O^|yy|;+TiFcB zJ8YI2s7?vjb5Qn|ZPW&Zh1CC_OVkdzy8f3aTY?M{wykAI*3I_00#zx&>4DNxwvrdx z?cr`}19HgNE;k`vFPmC~BzOy;0rG0t^be4E zlr8=UDYR?@WvD@kI~6GRkF8yWB0BEe?~r?p?d1b9>e*(RP~SLPwguHFQTqdB3~VRA zP-ud?UWwyRP9vKrh4hnbO&KILu`TyQ6-s;@fKpRzW+fCfbC*^@_Gz|}5lCxcI~;|C z88)d7dZI+X9!kuzeHfs?9CwS8kk!gooPyNzY(@*zri6=GsAyvwnulTw+=Uk)pPfx` zK$bkjuk1dk7iU*lvy?nV0Qf460H>EDojD+4|0*T_1Pb7m#Cv?duZK`Pp0< zNV3WHl!eNaFntRp18i4!khsO2rU=ihazQdiP2Kjf{ z9_rBS9^2$6qzbX6zMvK*s@hOtpY5OnMZ?^6buInLd%))FhRhMRS`VZ+WLxNi8kBe^ zhjK@3hCwKD%w1v#a!1())sXRoZBGOB#n?Jps78sNF(`A&_BsxQ;@r&{A?F!e&IIWb zY`SSkdd_w_163%o-wLIYY(X0oyx^{7hwLe~l_f}f$@aVq328Qq8+xL|j0Z|wv5l@n zfed%Y8<6#yO%{OES+;-(wJG5<2o-PGmiC}nj=R!6?u{+pgd{aK`8QOigq`7ga+3FSg!cXtu?+H3F%=*_xwJ zixQnWsL*D6)I-r9?xrRnZ-*^C37LP{)MiK_T>gL7IJ7_wO6;D4a$Rixc_&#^ z{4SIlV9SS~pn^O70c0O!J3EB5O16P0Bn+|b#-JxkY{#L5if#2A3Ji1iasgS@Y_n-d zJ;IjFKy6C+cLNnQY$vx+Y?Qmc0_4-OiABirkL{}jDRpei&rp*Ro?f8*7@N5Y9qGBd zszIJ{wvi9WWMDh`gya)!(iT*wgv~aTHL`tlpztJjj;`fDxlC-uZpbjjHYtN-X10qy zs7eW{0VqAq79NClE!=gfAjb^bx*F2Wveh(@WRA`D4=Pi_-58X#vW*)cah|(`5wh9X z6jP99fo*RZ>QLg(3{>hrM$ zPM{hkHcp|;2HR2s3i-KvNJ7p{Hq#}f53r@KAn6v{a2Bdi;z15diENTPD7ej??;f%T z+1?)@?GD?*6C~`i70S>PB^q9##2(xE8x#m}H&}UHh%_1AGo_chrA^=eF`!^vc=Po z;)!kG8fsAD&JC1%W^3OeG&H5q30NYIfk}25!Z9`Q`i0we>LAJgSw5#N9I}AC7*uEl=PQ~Ur zh9tvmPbW~B5~kx&Qq6XifW#5*G#8Lf!*+BDX-3(4GEj#SH?vSl%ht$2@qgSo?jXO8 z?cpAp9b=m;K`K34>IrI5qUr@IjI$lQLQw;ET{Xx%!RD(&W+Pjz0VyWg7QUbcCEk5Q zITM?q14X8|O9<{ixy@{Y-H>sbZLbIFv#@n~p&BK6t*Oq@d5O}6thC>-EUnS@+hY@rln z5ZQjykZhZ6?Ha05LiGkp2idH7Xm^LZ+XCd+WgB~dbbD;^M@SN4lb4|~CG1q7#)S$#}9h5s~^Xs8VlDoPAa$m45 zPC~{M+x-;Ocgbe7Ks8EC&O(_q+t54|y5eqs0di*81P7$QX7f5BX_oEH1yv|O@9EHH`(Hc zkRre~5QQ3)xD$hNTWsw(6cM>|pF{3#wwDXY7-XACLw!4J*$h;pMC}cf*=0Mqg+hDW z^%fv!h)pa)`hB)$36h4{mY<;tB|g4DsRK516$(bUOV=R#A=}6Yq&;Fg{Dg#KHfalb zqC|fiN<`T{I#A$*yTvZ=pR6&qVmG8dWi!g4HYHs2LB%-R&;S%W<1Rc1`4Vh@e;5Cg z<(zF@4JngsZyKmc3ATSw{(>z(1|6li8#h3nOSUs3WJT{(S*Y;Jc94UjRqnd(AnzNS?;bMO*lG`u;+<{b32IQ{T^Y*N z*$l5xT9xfnox}rJ>O8~i|zFX3bnYK6W0Ia{ASBZAbp!n z*8@p^*iL()3MKZ-p;U)0sDOgM+_jXDUC8`@)>s*av|ay8d>(-W37bU=JyBvt2PL}M zM#rIml)K{z$lAjuGeK$@TfhvpDdE!s6?@s1=Ac*~ccppA*Ux6MLl!yPr2|q9unjLm zO-hKYKzRk5!~-1-a<}1yJW95AA7mP0vu{Fj6jin&n7+X9IN%U;;Yp6^KJ2y~r zobCG-5)ItB3y^Jstz3jOMz)zps6&a@&roTSOT*|t6))ihi4 z6KYYSvjr6_Y>#ayI>X)6FXWwNOLzJHWS(PFOCg1o?NA0aD6zXA%FVO+2cU?JyZRvH zUSL~PK}I{fMw0p!|R}FF;v#r-5U6ifXfFvhuwlAnm33uO6GR8LEfy7hp62iuxY;iV4 zH>5dZ+v|Zkl=#yNl@e?oITSzVuA+eaNw&EmX!e5bW*AbX*mN4GMTs#jRJdg8AA_Q4 z?zYDv?-g5X0y1aVTqa0y&Gu}D8k9Ia1Ld-8nK>wO!=1(kxpQpCcF1_k*0Til<=Fzu zP>m8BD^TW+ZOH?L3fw(-A?H1tX#>(1*;1R3^nqTV&t^ir;Y-10Q@`LU45o%I`ybR?VY{3e2^vT_K74kIMR^B1g7hCxQlDF6_ zO{h)@*DWaf%{KZ2h1=Xke<9Zon@r;WlcB@5C52?aY@afyN(s(>C@oz7f4@>1fOfn7 zmoTM-91^xm6{PECQ;$FrDO+R|DpNwY4odd0ZRjCU#$DY2*?QURlaQv5?S2aCP~wCI zD)qA^XQ8;9JLNp&A7I;GfMykJzl)G+kj?9aT9jCIK?Nn-{3;Y3;_h}0@~YVMKFB=G z7WYF6HQT@z)S$$jZ74Uw*4}|48t&X7$UVyT5{8UgwwXhy?;l(C7^+dCHU?#MY$tIj zG{#-;Ipox{i5HN5oUM5YNeyhv8K^>uk69=+!DhaNf=2GrcaVLOZKMckO>BoHNSI=i zK0{BG=zoC{X10%4C@{_4Vhyrd*ot*XJ;P@FgxZvF@dXuU*@oIsY>vBd2l83jgs#m$ zS?1Z+yCJ2G?X3rDQi81y$}h0x<KGJW=B_dhxm;|sM#!+jc4LBMZnl5ZP?Zv5Gf;Y!tT{(Cs1LB?cfxO?sC_afV_Kbz9eK0vDH$LVxMi{3TjZ|-8Ga8 zvl((w)W46}{6pC^;_Xasn*mCcXKE|eN zK+;pT(XWTmf?Aa5 ze1-~5w#OGJ`o-N;74o*&(lyBZ&8Gf<6m7P{PpCnO-7P5h!{%>8kq&qD4&?r2TkP8U zlTpb2f7ZC~hWfhxmtd4ZHA+nOK^Y0#&;S(b=5Buwa!T0*6{PQB^Qs}KjO|SWRVZ=( zAC&54%a1|9KJN4e$llL(W`wkIwt*=~7+~9-hMp*~Jp&~aY^zo%Fv#7D4YDfPW)~s# z5LpWnixOkKP+^g+Uk*hb z+-)l$?-E-}37MU2u3<>A%=SD2H7Idf3*}sF866Z^;Z8FSx!r8X6OeJ0t;YoQdDsGG zs78qm3zS)7TbhGHUhW>|A?G@q$qwm#Y$*pM-C!GDhANbJumYw0Y!VL?+~m&Zh3o;g zcORtPVq4gR1d*+<1wB!sAqXY5+0J*NK#;q^eaO1Q7CL~`yKJ39s7(oLQK-1b_8Nm? zA?~bakZ+$YcMe&?Y-1@%dBAp>hMJThzlQP=w%`qPbjaQJE#x_3TPZ-MW43Y;l1JGr zk5HWwuAiao3EOA|3dgvMRw36Zo9rDj#M!n!AlVt)=O$K9^#%7K~!5Vkz1Z00_8@YhAb+*GxNcdorW}qiZ z^k<<&gY6>+1wOf3yo0Pww&Fdc{$ewhpf)93JVC`4+t3RX`{pkE3i;Y>LJhL~u&vi2 zWryvp0W~SX_66mC+4A4ek#O_>{mOU;@^t+#@k|K*$s}P@bVKrPw%s16P67+*K5iOU5=k1Q~kSZiXRQAKO0-RHcNN7E1TC^^HNha_+XrA;$pQ*94?fu(?c- zWRUI243#NidIm}=*{QLh5GE`EtHCCYb2zL$-bUFLgS=yGzJ187XR94RigC7uBd9@%cTp&3 zU^ARTkqPb+XOP>-HkgErlWcn_sL#aKNkcVC^jt%kDYn-eC}ietE)O}U*>VL)Z(-9t zK++ku(?_U6iT!0LHOm&PK*2fgT2;tyWm|cNwDWAwACO>UvoxV6O3bvN!~)ys4-~L- zcl--k7ujTzoj<7^Yym0Mri4!!R9s?P>W5-Z?n(oYZ<)=cge)$$OBJMCVH+NSnv@V3 zh4OATi4HniVnb%HtQ<1yT#q@8sre!#(a=&n=S5#BtbU$7F4E$ooy(&!}h%c ziM!mnLy&EctsI6lA-0)As6&a@$53gXO&f#aVeX=F$bZ1rdk)P;*tRYp)gfE+5^7PR zGXoWl*dDV`^q9M;TgV$_OW#4}6E<}bQpDH}OHhLnyPu)lDVzTVip06Aze4Uaw#6D` zOt9V8p}uoA<0n+3#N-#0NwN*Kq0j|)`yI%cViUS{|D?ZU^L9hhG}~JbRH4NAJ}7m? zmX|}p40rlL$bQXsHUw$2Yy)aYxMADXKu?s|)O zced~;v|Hz{D*-t^*w&MfuEADIL6T24+Z9x%gu81f*<>5fLE;y8i9BR$u_^8$%{SZL z1Jt3!pGTDlS<2>;?EOj6!}ctN8k9KQ3*}^NnSLnJ%bi96x%=3Tm5{NYt!EhOld}az zpc*AMMxo3A+ma3nDY$#kL(V}q(*&efvZW>==@8qn8LCj?fdxvb*d%jMaF{#aJY-k1 zy)Qu85w-;fBxu+QPUwje4J%M$l9nS!4P-pJ;=V#HWG%kVYb5vBpk3wkD(_@ z^q)Y92;0Xg6gcE=F#%bR*osL=eavROgxZvFaRn8lY(rTncEVja2l--bf3NoWljW3c z{T@=r+1?(YCMDROp!^wIz6>2DxEp_kJm+j@Z;&a;rl>>m3%1<`RHuZWCX`LFt$st{ zOYSN^kSonLEA0QtaK&~bfn*uBe?3r@5@Nkj`kJjz4((>S+g3o18@4Ycq|33nh9Su< z+tUbCri5uNl+3eT=^*irJIy#`E3h3+K$?5D9uw4|#7#3)DzY^!Q2c>A#~kD@u|3Q~ zvyW_(c1ZQamU2KXN>nXFg=e;d6)0NfuFC^?U)X$J$XsEo`5?tB+rlQ)pv1c^C|6}O z1fj?qcZprdU1J;Ehm7xRdk0Woovm{S)hN*ug)$#(uQ4dp;BM{=a(=Sq&LMr1O_zeC zUu>sos6vVT*HEg(7QBIi-`ursA$yx`r2uJv*q)1!&|$MYLQj;Kd4>|dY@-z@Al&|c zzj9oKtX=<0kiA1{30vR;YE#1JCsgcaTWUcuDR-qd|cl@hAcP+HAqorQKsxVyDN4h`Gb0;C&di!VYFEt}j4l__Dz1ttHn zeY+u1$DMl(vW>Bo*CCCbZN?9EDDgS~mB!h$+fdxVU33TXPq6idpjji^Rv1!EvNa=6 zixQp3P{G9ZcmhSIxSNVYUNc)d0hy=S)EAJ#!ghEGH7Kz=1LbDe{8=b6%UwMOx#!px z?;xXn`bkYpc*A6pP-D5ZRiCGEpWI03OVg;LJiU{vU%%})WP=FfGU(Y{{^L% z*z(^{(8-;?1KF3^&V++MX@d`(gnt^S=w&<6La}x3`oRK45ZlEjRHX!! z2&MPg!a-;^%w5+W+G|Am| z1@d07wW^Rg#pZg46qjtzA5eo5r<+hN&6a6Fkt^;rKae}acKiz&ui1Jekw5ii*#c6i zMu`m>l(}JB>W4x(?j8mp=PjE_3F-4}DHSBWV;dfUDwKFI3Z)8c5*-x0=gy~x>_xVB z1EhUmTbP7|5?f&kdZI*w1xh@!ozFsnC+-I4A?q_+XaQ1}**c3*n-bQXQ1ONB)dj^W z+*wy4-z!^g4YE|(#(a?SjqTJAH7P;91?6jO!ENa1oxATH$Wv!q2|=b0wsIJfH`pwP zP@NL4kD=@*+h`05H@S<(A=ek1>>M(**tRYp**DwgB~+yZX9h~Q*-BYx_lLWwTgcI2 zyS#&RzijFvBoXrepEV*Ss7wjn&rq_9ZQ}(JCI3sPze2WdHhT@yNZIb|P=^vHKA}<% zTk;Ev%eYgvA%8F1eg~TEWBU~j|D@_?^L9fmO04!k1v%S%9~2$n?p6+Y6>R!J$UMjv zAA%H0wgENNpu`;wlpA7eYoUmWJNFpm9%g$Phm2~r86(s;!j?5bHA>V@Lm3U*$qW=4 z<*wHXIkjw}4buN(YuX{Hj%|4fs!-zNGL#x)GrOUnp1ZUMvX8TktV3D@+u;TzOt47< z&=V#4MJQop`v^jTN$wW+AghV3xDTnP*o+aVO$irAP|?gbbOObuxeK2{J`0q$sC%l4Lnnv`I>g7R~0`D^IN%H4Pl^31cHqy&7ua?mpgJY|JVIGJ+iDpK zFLGC@KrRQ{>>FfQV!L^VWKOn!4X8>9u_lyWX6yTgc3s?U|3Hovwy$4E=Vo(Bj{YQB zWqXoBWlEUtg_0h&tA0pa<4&W1Y+kk_C8Swr>lubRl(;zpm3(ZCQ7FE_okIut{cI0< zXm*oraspBX*iw^FixO34sIbL$V1c3{cU^Okcbm;O51E5(wFOAA!?xgn8kBhFgmSxV zh7~BX$6aC-a);Oky^wLAZO;ewh1oiOs78sNEhuxq_PPy)BHYdGLe4|BTnN%1vFQ#V z=`q{sAylEn{wS1+vIS#M@PxZo9J0sQR?Z>qDckb}B*fV)Y3PX(GZ`px#x{Bb1rppH z-$K@NHdz5uC)ol;s7(o9sLWTm{ z?+_%rXIoQ4RZ6I8pmdSV`VZQD;O=$|a+KJ{43O@TEpCJ)Pi*ojs7whv(@^r6?Ry3i z%iOuGknM%7Y=blvwwXn!Ly6Z*Q0bLT>w@A{?xJqU|Hjt42F=#kw$>rlJ6m%DYEhyy z02S(Nk0KQP;BIOM@;2Dgdyx5)O&x|5O}4`b)S$%fV<`8<=0AZVE$-^4ko%i$F##Fd zZ1+j1?}yEJ3Dqbuc?D%UY(rTn^vm6T4sr^2|KG0&caXm8e~I3ENGf4_dw?pGIR6Bt zy4mt&C@AGl{|ecA*v{S{t&DA;4hg+%yA9}x65CBE(Z{y>4F&qSd-;K^a<*9^`X}`O zTUG+KDdAraR8+8?^g^*g?)v1APst`KAj=Tjml9H{*p`Q(CM7(LK>1-dvlcp1b9beK zJR@u);Uaf806dAeWAcUw@>#5Nv;#3}9)yO7Pyrr3ux z(`-o;HnspM_t;Jsp(Z8Bolri+7IZ;J``mrIAy1fXWeqYNu$9*#d4$d4 zhw7AY9e}cjY@^#y_=vmc4&*v!lZ7Bdlx-^v$xhflBT$tRoX1c)##TCkc2BvRibIY# z+hqdMow2DeAW4EPatW0wp*sU5&)GJzkeKAIo`Y-`Z1y`ylVZEShdPuvQG!aBY{@4m zp5{*Z0{O4l_Ftjd4BPJ;q`GGF)}a<9RvSCP9%T7qTi=J2 zEw;ATPV)oV6l_PokY8FA%~xBY!K3Ivc-oWNq|kRhRT$%qk)oJY~NZ) z6uEPcLAGtS@;IakvdtKw4kccjpwbSTb{dNBau=O}{CjM@R%kZFwq=7<`)o}+)S^V^ z5>yDYJuXAh1Ma5WkT=4X_CV%CHuXBBIAS~8fEtw89e{GjY<>}nM7gU6A@>Q};vQs- zvEA=OeWz^32vnoQCGPa$W5O-Mlcb2e`hk|x>SQc#5w=dYmD1zY|a z3Z}Tz=OFtf+gTpcrr8GWA>oQ`_W^pM#P%bU$gr)Jp};kFFBQm|Wt)A2)HiI|cc@JX z{~Ay+$9B?$Vz=D&eM7!HoA?7+?%2M5A!UJWS#tKLCM7&cq5M6YxfePra(C4ac^=qC z6p*RJcBF*lk8ILms7?u+BT)8<}vau=PiwXotJ)807tB zYsDe6aR2{V!*vcRy8f5&d;v8maXJm1A7bhC+SZJ-k59el}AT(#zRWHAp(ZHv9oqDDmJEN-5YREhsq1ov#hqm2B@F zNIS%~(3SX;pkga@Lr;`wkU@!Iw(~wHpyqCH0J4s-g$5zDhOILMwJBju4HZY(UNulm z%boQfw9Ai5*LQP7LPeFM-TW}gW8t3kN2J#r#R;-X|f~{m?{V$u{bO!Y1ybZpbyoCR>9HX11+$NH)#(xdBxv!5M(k7PgWI?apvFwF5b3 z*)I1W-5i@b3`wkPkqA_#gzjS~InTCn0*N;6>Zg!xfz6(PGx9!XBH{1U*rr z-wY)}Y#$aVu+QD%9Api%73U%K0h`edwJG7k0Tm-`L(5R?kh|~-ZQTng zqik*)C*?vnlo=`5D{p0aT}ipF=2{U|Wqs;dAaP zG02r~XJ7i>4@kSxXaF9lU8A(n>Hmu!94&~BQ$?HkB(#rAaz=`w7t0wlR+dn!U@ zN|=6xl3BK^XGpx^PE&zwIkuxJq`775d51cbxcLE<@@$PyD1OJCqXqd3Y!7W{_MUC> z7g813QeDYEwJ1>~g$fUB2QnyH;;yS7@;iAJ zU4$x>*zbf=4Yr^Q3Vw3eazpke+sYcG{bGAwhlCcJ#ScAEVkQ73zS%~%p+K9v;~mKQ z!zK$s>JD2V47DlYGXfQV*_Mu>m{9!xex-B*`MUm>Xo^D?3EO1?Qg*WqUqDSth+IN> zDVrn%9rbXxk%c@mw)Y%l>SeRvLGnJf!aY={gozTA?PoiGg2Hm{lrNBLfGzY285C^4 zZ;)(|ZLJPfDWTec(n>b#7qmOX-R(EzP_d15Al)!qT)6m?M9n7ehRT$%(*q?(*uHxq zQNx{E4%tT8$_hxMWt$m-I+S=l43++|X*Ez>$6Ztl`N!CL$Dmm~+txUw8fR-xKrKpi znxKM#?a>THC%BuMfxJex^c-ZKWK-K9g^BIZ4mBvTdkM--vH6#wh?%?k3gn(YHIRZa_6kOm0G%S+*e&3e9o1AB3D%HenCa&$D^=A*qe+?EtD!;`|YmT42jZ zp`e{R{V8N$WIH>9v<|j`BqS`c?WUk7N^Ga0gp+Oc8VW3P_i_VSU2L;?NWH?AEkJEb z`1b%6-E1e1P;8aEzB1(Vu!$AOvc~pRg_K^l<#(t_2~QtTex1$SgpPdNU9}+32HVIF zWb(5e{X+6hHmM}_r#dBUN}+6k?L!8Ix43ikLoSi6H~<;8*(Q~cEXa1Df~u6D8iCR~ zY~fL8cbB^^9putt&h{LJ8k9JF2<4J&nPVt&!JQ@sxl?S%amaYd)^iT^rP%@(P>m8B zmr&-4Z7BnVGTc36A?G!l=@!yw*;03q^oDJ?2vsQYpai9IY?5avc*~vd1+wSa-d`c@ z9os?;5(;dEI`l+|hEFJQ&vyO=1&Z7awjt{STc`u6OKctC@=t9_SnGz0k8H0!Q0$32 zYair$X3NPTOPOtK5K_Ldoen`wN|39ee1$EjfsS6e`_@99D%;8!WO`#Ok3;ero5cv# zDdE}#W#8FGr=f72yXXw$`e2h;Awz?0%Ld6l**@)1l@gpwP`b%hT84JNxSMiAjuzXc z2hx4Bsn;P%n=P^dl_{Y+040CeHbhA5a90mPwqG{;9;6W-{y%Hn??W9*oQOcBF1F+m z6qo!jQF#LSyV>?np;;;0?-``(Ve=-T7A00wP(j8ve+5N*xx2lFynSr?9Axfi`+M); zpA>SofqSSyi8~KaZh)=*2t^d!xyz7yknN=c8I^1^Z&2S5TlO8QQKGg1WmIe@O(-})m2Gtk z3eR&_*@j#;w%J|Cu)uZ`f@F5Ke+N*N5@Lr?dXcR!3hg?$+m1nwCAP0Pq;s;l&LPP% z+tUS9riAGJyOVywjC8|E4!Uo&HClvK_*VTf&n{2)|WDc;^I*?+EZK3PxPYp`E>xOb7 zn?VLewz*66LGB>i-~eRYVcQ#o`gYkmLr{$pJ!&Yk$M&j$LLu(v{z1-tw%iz`53}hE zko17<)Cg55v409mMc9JVQ1Fnu)(m7nVq397+GDn78ze;8EQ`<+C1#eO#0lG|3kt-z zJ9b0XQ#RQeq>i%%)}b~fd~QI+Gq$Aw6iaYd5+UC?n`sBKB-t+aAms(ya2RS*LL>s^ zQ*4rB=;)HWjT6X|W_v${Ojm671SHR}6_QY$5+*L8>^0l@6%@{Lr_4gG8@5mmGUV8P z^N{S8ZS5YaQbP3sO6S?EPtfiiceiE8QD7T;g>?69@i$0PWRusSG9~OZpyUJFcM}pz z+_}FY+ap`~2hu#T%?O!4btv�+pWGv^`L~%w4n>^1rb4%Awf`+m-@Sy|OixP>T|s z!%(5h_BaAX-?*F7Lf#r%S_hfm+0^5ZqRw_W0W~PG+XUr4*!*TF(%`Ocf!v>Li*t~% z$#y>v^?kAZy%pk5HA+l6piGNxXc-E9bGN?&IooW42h#trdA*Rd!}jKbDwH_C38jA7 z@>@_)DE)uGq7Op$uKy*R?Lt}!+rT~~bhGUqKu?s|K7m8B5h%07wsZ`IoZLN}K+a`0QykK}*is2dy23Vm0aYmR;1Wu?*(4b#xXPU` z3)wwv?>R`j#%2j2N?5By z#Z9)?1{4c$XZ?bFTWq;+$Re_hbs*(7+o_QKQ}#D{{!R z$5vKA@(`P42&z-U^)Qs(XB*W(;V^emE#x|2lZ`=!2;0^;Bs*mLoPes7;50$$Bes$m z+CAoOY6fyd*)HcG-3go821#OU5j#|-gzhCMdCIo242f~>>MM}#jLq(WGzqqQFVvyL zi4CZ9&X(MS;z{n5BILhd+YdssDYoBTNOj5P-G^F~SUrFWX}0+zD0;=+Z4~lm*z~86 z`I;?$1}U;^14*bsi90DMcf;0BLy;VJ?rX?>%l2{u8S`v2d8qG>En9$Ul&F1xG6lAi zM<{g9U2hq37TLrKq<>&*Rv~GLZTTIlP~zhUlzL<{H=*DYcj*>ne`Xu`fwX0|!(T{v zVUtR3{`5qNekqiwuzkp&z$-)TV?B6;!OT4UIstckaTYkgv`r z=pf4n+qxc7HrU<_P?HjDlTiMXEk6YvHMtwNK%OtQvsuX0VpGgR@^7}?1*lF5KZ{Vd z&9>@+Tgx`shK&E%_Bv3Xj;$l){#2txPdAhqV|(p^LVE7z`XJ{x zTTTw?4Q#qWNIJoGIs{cHv0n|PjBG&-6rAL)rG@M!wv{nRJH_@q4hd#9ixGOF#Ec0_ zOtXznLjenS$1{+1hD~OL)U#{>8`P$RPdikcV_RB+Vpi_{p1<`c-#nYi4Owh#mmWyD zz&5-NH7Oyo0p;y%k^pqH$lZnrc^qu-LCCbkX5WM4PPW26^dCugw2fT2Kw9@ikocxH7F*yi_{_C7F+u_G`Y>T z`2%Tp*y_K~7bTps>p$iFZ1-{~vdi6AJLCzlr8*$X9-B@Lse^0>8mLBz)*dLk&*tle zLLu&|eUSTrZPoyp!fdxgkn)htGy=U+A~6c3BW&FkC~(Bxo)vON*`z6G=$Oqj4as9{ zFEdb?64P@~@`Nqtgtp_{4J|^BQ?`>Os6WBh>4Bs(Hoq5ophUw5#glAHo6yENcTZc8 zEyXtJhjbTgnE=$Fq=9{?kY+nNfTEY&DI<_K!zQAT^@{B?25GWv3#U+>k{-^W+%=me z1%-3mU8JGq8@Apn$ed>jUqh-}HhCVZQqp<>$`sh%i%{^6ySYcmRb>17ER8=I@7YEx zkm7;u>=mj|lJ*@+J+cKqAb*LwmM>`TiEXt38Om(0O-T04W^c*;sYFRPtx%%EHmrce z3wLoPWPfE-cS8C%w%snMNr|6ss90rN)|NZ!uY6NJi?*bhNT zC7bLJ+V0@adjvUDY;VUfHpMTb!8x1H(MYJ>9lOk z9Mqtsm0PIL!}fd!MRnZS9w2WoTebvQ^=yOBkfx9A_ywv{lBx>j`q{Q?Pe+> zU|al#%!6#DA4oOCW|eOKRHdX#8I&1j>uZC8M(!f*kZXiZp@NJiwoNsp7-jp=Kov@I z_CP5!Td^1Nk8wBF56xNF&JB=ZoJ}_j$*gRl5vW8-t!5}O!M0|B#3XlBD`cNyb4)>c z8{6$P)TG3S11e6lCFY=*ojc6}WIy6 zgsPPEV}deKwxuyBc+6dS9CF3jCMO}|3EPzoQpDK?W}pfsMQ5SZDVx#>`4il2xuCf- zwof-?NV5IC)9FvLbGC<7s6+!Za5_lM0q0a=@D z=Tng8m#xPR)hQ|DfO68~ziS?{c_`fSPtw`~v@BzLTZGK5Y>s6}C1=ZfpeiMetU;MJ zw$pVesNhZ`AXhtEU>h=F1FYS zB-5~|5>Sbfc9Kw{o9#OV2`zW-OUT~CR=R@pI<|=%)TG2^9xC>-_1{4;J$I3N$k)f# zUV2X`ok1B(zZ4!q(jj1;)AC>w}zDHpu`D zO|W@}Ao(QQixDbQVtN!xPO;_2plus>LsrN!&2};g_1oDxry*&E%|8P@P@*vl#T{%* zPH1D6yC)Z9n`4_?f^_q2nH8u(NdsP};AA`ULD2>7l$(&(#U{2O>mu9d4y19jEd-!C zB|QY8+!CAR017X2cX0?Uudww-A+v`q9D`J=Z1Pj6N=fTyP{zylehvlKxSLBuE+1PV z0~y!ZMz0~o2HV*URG}no0ZMJM1&fehaM$t(&26!*K0$_Uw%2D!w!>zBg-VokQ-u~a_XgzNz}bpz7xvF$dYCMABPyFV3!Y|E`sY@fSw8{`YIO(~(t1GXy_qz$tf zx}YygM7yECFIZP=k#O4J-s`iyOU9ttG6yIX*q=WJ#-G?ZdXE<^GQww_g}Oo{z9 zD4AxHZ9v2bZ$B-`1 z);xh4l(dq73b$;}Nhn(2&UOKL@7S`JkhRD*n1wX=Y{xmMPD!d;DEGj&eFudfx%<9{ zmP>4lCCL25Rw_fPGMn`Us#4PB8#IS*3U`q@( z2Sr-ByLLhzIor@8WNBlIEkSApTc-!AQDVmnW!u^Q?kWE%q~y-M3AsDip0*&9ifzIV zDLdIR0qB(y{rgZ_&31GE1-iIvk3dchn}|X~-E8$3B-gSnoI+(v+@C?o9yUt~+SYNG zN<)rbw%#kKU(a@M4M}}$@;vlF310z<_p`khp^X9VW*;G&fvxZa=?2+M6{tZ;XRlCU zh^_k_iVkxZ{D8bhHt7qpj!zhWu=XXGp%wcKiaBDbZ1dk^#2u8nnH~-QV@$KRJSIi{DWHKHK9D zB!$?l(!-w~D3O*y@dLKLHfSTv-C;XqJ7iO+AYFuQLk%@3=|ck*j@agVplFo4VlU)9 zW;6Fg))?El0n(hX^$bIGN(zlYxj385424g*TeCpR3AQ&YWIkhaOhKw7TYeg#BPeQRGcaaq2 zyJu^^geD)@Hm@M=$IJvfVZyBDP>7eZfcSC)U0(Y#AHWprnBrs32!MnuVfm+$o)qSHUJ+khPud(+z2q zYzr$;osu3_p#Rr8|++A!y%bje!Taa1J7T$qWU2O6IRHdZ#Ae7Oty@#M+H+OS~ zkW0%}ID(8lY@;zqp<_FXLlsKWoE3fo-@5i9zn-50HI`P5lJvhuLJyqAWxM)@ zv}QI#6Z)b=R4V-`KgQP43Pmj3ZM8w3akj5^$YN!4sUYIcM3-6VT8cTXG7L&$IQ| zp)w`*9Z=HACYy)07r65-Kn@q%+alDz$TqVKNp7~h2YR4{aSe(uv7N3%8_V2v3COm> z_V?`lC!L3_>4zGWw6X^kR@t8Sp{SQTTNv`Lv1KEW)yFn?3~AQcj!&REC8-inZi8(* z357Sg`%Xd2f^G27bjDd{o~Wp>#5?x3KbyU0D{+GSIeAY*`SvkWQr z*gh&yg_4|aP%6k)e24t|+>OPN##R%mS zY}2Dq4%CJo= zLCPz(%nI~MiGDAX&axf(pujbE?VFG@$0oL*p&PdP4kXXBEd-!4CGLYz@|MkV0Bskz zOC3UvJGR~^)L&#fh(XdloBR}dpoH%Xia)TupFQI zbpuj;u=$%%m6Cp>@}Dwwwxw1m_{m+l4RU?4O)4SdH`|p8QZ(2Gx}XXrMZ2NY51Ud4 z{r%K`=ZP&nH22H)*$)|{^1tR9gk&xMBp!yL5+#kBpoEO=!VHO4?)2l3UCtJsfb?x_ zZ8oS$iFG?vRIt??P^_K1xp~N^WV>5{COg0)R5Eta7E z5w^!NB$?Q(FVF)e(r-|Fl&!A@ZJ4<`tV6ajHpMrjv#@RaKn+Uz_=O7NZ1b|`KSizF z73Gk3g3a6xStr@fJ0Q&zTaOy5Q&LC+LKW}Acji`-p1p*c6( z;38yLVv8+7vSl`v2P#q0ju%R-uzmX=;o;7`3E5ZKN?VZL%QoSMnv}Q1-PpgA@?5J>?33fvfVyG%6&Fd1$w1K;uT7V*t*}LzyWuAACNQ5 zCVfFehisk(B#*GYG@&vjrduk1N*=N0TA}SIcS8!uam;q2g!*G_ot=<$!shRS9w^c1 zhT?IyB^|VJ%H5M5vL)Ci2O!-UTV@byP||=ADkRyCOi=WkJLMSUO|gk_$a=x{IRR=J zkSott5RmbfZFC1x6xhyop$a8wgHY;@Ef|9QMebS-p}Bju)g#F8!1j6!$sXD4ai~N| zHwh?FVjDh(#1nV%3&>t(Q)eLkGuv(!YEt4S2Nf%9%ePSMg}d?{BYvR`az85H{F zPTvN(8*GQ|km-l5O$8~NY#VCml@c`#l>TL#?|}l+^S}EmcfF9a<)0*TKQts`OBx_~ zD_hSnRHnrK2$Ynw$;{Ap8+Tp{P;u9_ z0NFa(0*jDN&DLCk8kDr+feKx0&t531;m)=WdAr%Nn~+t@Hn2~?$|%Tp*bz}A<9f(Gs)DabX*rnrQRLu{K@ zkYbqa;~J_^k~0sbjBLdM>ipkv7J9chEX8)(H4X8>*?mY=t98x5Z|PK@Cbe zi$jHNw(c`1y2D-Y9P;|vq%>sRWn0ZangH8N7OGQ{{RYbIvE^=|aFDy>)QvF&a{$|76S54}=ic@Ik8 zvpw%afd}rU!jSWkEgOM`N^FK>NdCljd;*mz(UE|XWwz}kwEfK8R|;}e*cLCL{uj2# zD@b}}v*w@&N~H5p{Ee;e4%(=4cX$uk-q{o-NLORqC_@cO`lvvK54QO?C|c*P_zrnL z+01pw`o(ts1!=z7dVZifC53*WT!T#}tNtnc!`+%3T5htvDIoJNo1+6#NtJ(RjeI9m zrKAxJlxbl*)j~npKS`Qi$koag=!1-Mw%-9r(Z;qi1XU=hVuVr(HrpuVZ|ClM44PB2 z4O$^X2U~0slBw8K(@=?$c4nYNC)@WdB-GrwoshkYt>l9A8n%fgs7Z;-6{y(F*6)R4 zTJ9n~$k)Txz6nk0*fzHyZ7*AW2l}FfGXUlFZ1+JZ(#PG{0p#gtOC3U%0XAI}QXAL~ zVo;3|t*20Skj-}ng@(ARo~=^A>a{=O^HSY}8{1MVv@y-yQyXNnvrQ@?-3(hs1vMyXpbIKE*p9lP=qz_i9ps&36MD!x&-U35 zX`E~egHWB49)_XZ0-MDIg}Cs3K&mA+xecmP(z+eWEVI2kpx_F3 zbMuhP!&X>;jH_&;Zb;!}J6nb-l%!pSQfq9%HOTMdu4Mz7TW4DpkYR)EbsLgxve|c` z5+&X2K?%V&9D>9ackwV}-)2)ELHZrG-D9XpiJudw=x19_K(Sr!%1Ou}#N8l19TCiNvJ_dE7MTnn(cW8isrbp%|YH9wyYDf=Gg`pAYXwNu zxT_W+`v;rj5z^P$Zl9ngB}OVx@slm_3dO#-)4W5zZ??S;XtKff`w3}(*gOsBixQP4 zly9<4w|x95^2^qTdLmm25{QDA2)O`xxX@v59eLsFSTe0m;>D3pS`siF-Sg z>|(RbLfabdQuC0bo2}Oc^=sJ<+>q46CSQRbDB)X$;ySkXHE5%kyV(uMre`Y%NY}?^ z+JPFBbhZl>`q{dJP;`L1U@$q1ZjrYUXGzUCE4RpZkR2XfWk)ZhR>np z5w?>H$ZTR$XCT!mn?DOxDd{H%Wz1|#w@`45yYe06van4)K*n*ls}iKJvJE^#6-tV} zK&c5fWfk&Ia<^53=BC&_>yW|5=K6+Y(`*kvP>GVprTU)|cD4%{BxbnNw?TFXTeuz4 z&$6|tpe7~O)lhMct)_uu^W4q#Kt3niT`x4bz&6?sXQsaWf5NJ#0n?6k6pjJ_otIY+VbGX^m}n5mNftnoH0t zC6+xqdyGO-fX#XWJy0Tj3dQ%>`jXH_kh{YaWZP#`Ttd1K+r|~tprns$sBpkGpNFDh z?urG-d&p+KhpZ8{^G8T?#MV=W>Xa0!K)EQJ>o#o6)= zs7gsAzfk6s?X>0dPr(Ft8ad=TV+$xCW0LJx2`SFmRyv^yB~`khREo`}h5Q%XUF)E^ zG}~YwWVmFD4M4ICn`#IuQPPePN?fshn;?vb8&*$pYJ^3)0@P)!on+C7dfzzQ}gJ3PtX@8}mV)2e#A(WO-!MZ9(c1 z+rbW0qeN=}%099Af>5Z;T{Q%`pV?*)Ayb9z_6SnGu$f}eDIxZ692 zoK-d{4Gq1sc`}f^#`cnh%9NPCfs!9=xm#$v&fQQEa(uF#JV5B6i@4u3{ z4jJwLB$8i{a)zzv2dYwH{};+S*krQrKZR$x^U5L59NU`$vd**3bU<1sTfP%&P{OEz ziVJL~S}5+~uB#Uki)?{D$l+#d4nT$_wv{1BzRdP)gesI|8->yHr zD`ZQtJ+(pl3$_U*^mj+=-{UBwf=ZO=?}Cz-Y)9RYKf_(S4su?x2|Z-Yveo+`=WddDH_E!)8aq%E+?ZBT;}K08#rV|#Z%@gjG#^N_e_ zD=a{c2R4%%GCZ=KEkp7WTlXqdp`_p%lzw89HlSdcyHx?XpV?lvA#;Vzz6+^e*m8SN zosxz_Q2v$eBn(C0xKkfNzABsl7_z;y{hUDh8rxC=l0MkVNvK4L$qOi1XS=$D{GZ$n zWFhAlTQmn5zuAh7X(-vt_B;dm_1sO(LC!w5tP?W!vl$j4%x`hX=|U*tWe; zc#yj>XfurfburBw;~js z=Fag5`Rr`@C&)I#Hd2A~4z|-*NSbBSyh9~Q1U{hT9NX_F0vGYl&^x0coGu;&V`g66yu0SZ3Q@gyPTK{VYME!nW*z94~BT zFJyRSn_7qDZ){haP=%5V+fcg77WG5HckVj&Aa{*zYacRyuzejs>N=Y%0@W$$F$(2B z*~U+x=ofeCQ^@ztrcXk)2HRl@(*LlvT|!cmZQ}|mQKEJYC4brG^N?Sv{yR_H6(DEJ zKS}0$$S7k=K0?Y?ww^LnrNn*(%F5YfZ&0|6JMTN>QLw#zK-PA)nJ-AIWXm_81|^KY zP_cvUv_<;Q-|tVwU6&jZoooRG zAa@Vj@c?Ajv8je2buZht5vo(tw+YJY*%rs3Xdic_amd%tW}Sp=18kQzNN-^4n}MW3 zw#Y10qJ+W;C5PBHU66m6yAL40J2)x)`F0BoUIyy8kBGxLPaav?GY58;BF)aiAlCZ9CA#tY0e;njcxB7l25b! zUO*K}@?@a2ovo6Ef-~Gr-#~5$+x0DEo@E;Na*hCv-^sv?2A>}ID zf(oiq;$979y=<0lD7?m9st5A;*n0Jlb)D^?AJT5H$p@hZC49qBag*(R1d0ppX3db; zVk=l6$2Oa30y6BdolQYzo2xMP5T1{uesa* zh1@x|7Mbi%<{LJz98%}mUKLQCl4d%f{4Lu}CloDkXVgHxJGQtMvK867dLjKi+io8u zJ+L(gpb{mPhoIym+p`h!m$;i6g`7`p*)hmiW;0kJSkzRp{?`(ZuC|=|4&F5hy}Q1>5ffR1L_}#kSLgtQxj&sr66VZZ>x-)SyJE4JvBcCX`UT zhr3G^By?>3U67-fEz%7c^la@qNZ!Y`sfQ|*RPTq<{cO%bC^*2~{V?P7M^VuQa2wT+w*-UJ+^N@a&?REi@%xoq%RH8&;8A^_^ zb+1Bx3wL{KkaL_(+JKB!HjjXm6KpTrP?ZwXyHIwLEw=}Sr??vmK^`01Nf@$DvvnRp zT05Kn7-~?WaRL=**p?Dd+`-*b5)!j)lNXRfCr=*KFD8J0sTZ5u2+=c6q&%-AFhHR^B>pzg* z%l7^YNo#C#GWnlMlqkrdq>pX19rCYpch&(pH`ugl$hgTC)IdtX*3tu2DY4oMWw+Q~ z`=IbPcXk8h*Pk{_^LIiU(A85W^*m@T>l1rNFF@IdYe+m;tHAF+M;Aa#_@wF%WJ>2V9nAG3}7 zp=gY|bO7?5u<7?9Tb%9i0MehbwM8H)!L|{FN|dO@pyV0b{3+y5a(8zIInUY5Dae>& zOQs>^1zXP*RHelJHIz-W$?{P6k~?n!@?_ZFijeh+ZRQcuX4&#jP=gZ23RJviJAH-X zIqtgNA#uYN_<$UFw&o{fxMf>uK=K0Ha}%milC7ofPw6|hY%3HjayO`e-1lt9O33`c zrs{;$k8ImrP@R&#yP!6{FC?%LlsK$ z96@OrTjdxEwsJQehum_u>jY$OV;ee$)C#uP1yrY`&J2`qXWPj_Q6+bc9OUa@bKgQX z72DGtr0-;#cz`4|Tc!k+DAE56CA-*;ULe1QyY?#N>}C@+$f#wj*CAyO+rl?grNsRY zl-03WB*mY?z1*c_kVntf+Xh+t*bdquZ9kh_1vMz)Q$xi8ws#E_H*h!G1BpSlLNDYP zVl(wahGDid10*-Hbq_-oN(zoZ=@B-`3R$EjJC-DQVaN zSVh(g|rK7`Xtn#L^uT% zU2JWaP<)ZQ^(#oY*=pC2V~K4p4;hx(?h256g>Cd6s!&q$5lVa5v}Gu`%H4hia(mfY z-XQZDoA(`3``BJTpgJYZd_nnjwwneN-Qdpn3;8zL;w|levI({>Ii%lW+f_i)Hd|8( zl_;^?2_<*fp1UBwpSvk7L6o)&Cmxa_t=gHpeiLghM;VaZQBTi_qqErL7ouX z;uvH-V0#>gv|%>uB-Efp+6EO5+4^Rnc!aydSx6kQDV&fa%C_Nx499FAZb%+un_qz{ zlvG@W(kE z@BI@0UNc7|<(~{Cw)R#?{=~M~2307j-VUY9Y)%yveCF<64Y@09W8IMXg)P+ssbAT2 zdZB7twnP3e z?)Dsz^P5eYhl~w2&jO_UVS8DGs+5>shO$kzoCga3ayPUFd8Chj=ZTYb$lCHxqEkRx z8Jm9_YEYu#hl;IiOM6gU&fU{KB-+>}!;nM4mWe=ycD8|INUmf%I)N&bq)b5R4mOd5 zf-3GlQ;@rpZQ&9!tJxl|Aaxg;B?r|h=^_v1HEg|iP_&!7@IBNE%*y54Qws4jz3i?u_}kMgKVz~ zC_KcSy#w+Lv)yz;RwLW62GWkO#kEj_66#*4Xky##gW{vy{R}|D%(grPImXz^M#x}c zn;M1W<7`)BP=%5VRw!*{i%vqp3GOu<+r}1DqC{;6O3t&*2Oz(bySpIdTwpUFKt>l^ z@(@xkvh_rvDkb(~P}a>RJB7ka+a7n@K}j4_iJ1H7H@chKj3fr#DdC z%UxFi5^HRMBINL~H6I|uI@`(x?*{w#d>acxbxP_yg7Q~vJI7Eo%U$CH@?EpJ6Ob*( z_LPM5H*6CZkd$Z3TtX#E^k<>uE!$BJ@)x*kzlEH4Y~l_w7TN0ekn)~wp#)VaabJeA z4{VkfDE!D>>J9Rg*m`S_^@;7E4r$A5@^7d?3EvM?d}e$9h2j@Rznp^3TmMAJDb!41#8@`_CoFtwwFH0TxYWzAoV9(ZV0MV((nkB z|6)5Cg`(fwsV$JN!REI@wjZ{iNl4#hTbhQXU$*iLRHDS>9F&ym|IQOvPRQT#PhwyZ za?04EOOUaZP3eJ@a<(lmRHekH56ZT&xi+D&g1d(;$kWa??uV>Owu=Cy?O@aILk&uV z51^uott|q@JGonrLPE_}i$RVqwz*Ttpkcc^gXG<8qbaCDNy#*n*0O1@pkNPo``3_L z$JUaE%)M;h0;JZny%wQ5_4id=P`;1t<_U`Sb7!nTz5%xQD`YdUb-hFSLAKowNE%{m zenKTmEH|L!Fxztz@*BCEYEl2mIl`80g^VUPg91{HvK=d-DkVBPp{$v0y9)}Bare~? zc`R&;I>QQqc~jXV}cMP|(5M`8?#FW$ST4<~g>I8&c1+$yT5`C9SPOc_-W38WdgN&anac zTx@v(*%sMGb|Af*?Q|ECme@2ws6>fC2ud!q{e~g`3U@0S0^s-$i zpeiK>&!OxZTkHY~`?ynOAkRA6P8PCmuzlws?IxT17HUwUbO#j$+r$GD-{S7F1c`07 z{%6Rs!xnjg41TuuDkR@!+pIwqN~+hPbb!tI4F&hOyZ?dQLAEie>rdu=wv-G~huCy& zP@R$v+M)abTdN9+hPm^pA>SceRRh@~Y_mO({)p|i7m}iErhceIiG%@49&;vYxYbEkkR!vE*@O&NYy;bnJj-_Ehbok$+=J5BY+@e@=D7Pj zfZR813lYeiXM2c3>RUF;2~?+~i&H3HVCzjn(L3(KDaco3lV3u%d$#o}NdLh0eho>F zY;$?2M2SKHN|xA0?;-yacV~}~v&^O~L&j&eUjxBm<<9;E zdEVG=8j!WhHv9`|-`V0Vnm;utp_W6%8r!Y{ihpqTql84AZMhS2e6p3hAj22ilopbI zvt8++3MCo(pmc*RIsgTKxa$~#+)cJEBV_(%`!Ydl>GR+D(=`UwDd}+>%D1qMPeM`I zKS^mDY}QZ6Hp+JS4e8BneN9LjV~a@Le=1Qz(F!Fk zY@2P6f1JCIcF1XEbE+WY1Y1!JDJR*+x}hp1&U>Kj6q`;Dg>Br0`XSFWTk9ZXwX>}a zL)sa(>Il@Jgu@IK9c;H2C_c;G$OI(j*b-BaW1dZ8hYU`(JqILTVEdheDwO0|fYL6u z$|4k865uCpDTK>iKx+7pm-lT9QcqhPD2AmtX@!X;Ft#QhbN-Db1o zpzscNsXXNIv-RFV)?K!Pdq^8#lb4_dC46P5xX1Qhf#N~#X5S#O&sKPc93eJS9WorS zoqa*_FkANzRH3BcFO)uHlVsXI1tZ+8$|3g=+lvA+N7?KhkouS{*9p}rX;=g0V{9i{ zD0;%3x)<`r+5CNw?Ue0j0MaMemWCkdjIC^hN|cx!g_23Ot1-xb&fS0&a;Dg#laTR( zO*suI(`;KaP?Zv&vrzVu&EXfYj}5G86jyoi0OC7n^1kDp4Y^1|>CYzw3~{o4XYOIkjw+ZOGWeX4{37I=1UQs7i^! z5R~m@i-n=Eo;%eMT35g-L{w(Ae zW{cz?gORQM7Lt#!ZQel@N~+&OX%m~X1O-RAyDvj-Guzk;WFBKny+LXVo2~}cDe0gN z<;U4tzoDp=JKqoFn_#Q{Lbge^S()xn`YE+&9w%FFAkbax(JqAfTY;&hj zi4uh~DCuV#O+o%$?#|MXGr*?3f{c4?!D~nvWNXPoRZ6TDpzJ=|YY_^ExU)Y(o&&a< zC&(IR8?HdwL$>%U)S!g=9V$lHc0Zu_5qCeIkchG^Hz3C`Te%4tVr)|_y?>IQuwAu6 z6-qKFpmdxqs)T~4+;wz9?gZOb7i2zT`|5_&Nj8@bs#DUV9?GAyjSoQ46nE)C$ale} zH$t{F+o1{4U$V80K~jcoV;m|`qBa2~uh`~okUz`aogH#svzcchV~#C34=HchdR$PI z68ml_n`e`)K;c{NysMC>!1lHVS?}0pHXv=0Eia%3C5$^z@t*B;7m7b{*A;}sBU>N@ zIZABJFl2aQTRDQ{Wwz&Is6t7$IFx>7%O;>;g}cFX$o;~0d;ytX*;E-w{l>PPh3b^_ zorCgKw#8d0`p#YH4)WF5tPhawgYB{e>FaEL&ye)V7I}e6lu%Tmzfa@&cmH9t2Km3a z`=~?C2AlI6GXAg?e;{R(ZA{YtsY;1+8I=8H)3rfi>HFV#BGe9fTK-9DRY6u6+nO5E zwz5?!=LnfEWh zScDKlScDK3VG$M=i$w@w5kd%I5kh#K=luAY@0=N8?42pQqmWU>riY>*#_4VYZ_n$LJX_upQ$v>><1 z_Ph#ZDbcqMB^TK)HlgSecj`7|EVIR8P|#v4--o;_Y>NkwSY`V;g3^>6JAv#qw#Niy zM!4&~fI{nRiAzY`VDqIQ*CyN6EtH{T$vu>avduq2kuB~%pP=wI+eiiq?6BRvL7o^} z-~-B0GX4dncG+rwpx7RF>!R*Ya-XfJ0Mcx>nL@~S!1i7YDTi!>rI163lpBg4vH2?@ zE6&|P6{H`tRe2%*3EOHNkbTAWI{=y2+>H-Gp%j}l0;xA_!7<2n%XSuqGL&qXf)aOZ+tW}a z&0VR4{_ZvQ?|Qp14+S3BzD&sT$fhqtIZCEipp?Vb8G&L?+#PQ~@|mq}3({WLHg+Ii zhOKxHQeN3+ZOEZS_7IA{v4xHy>z%usQ%KLUwVgx$54OW3f7tq8Anz~R--3fM$` z{hw5YEv!JULbj(OC_~Af5-3r`c3uWWin(j7fWj`e9S;;JVJoYFJf&=AEtI2Vt{zI2 zv5huDF*kSjDkRI_Nvg6*UY@>R0c2O!16w%G+alyLPx@hUdi2U*qJee^?m4ckx% z@_X5Chaq<@Tl*-KrNq%Vl&oW`nS`SC+(it?@UbbgP_Th*dJgh7vb`=qqKR!_2})D) z%7Sc_t#u7Do4MOxhe9oEo+zZYvRT`Z%g>g#3uP!du@5EM*q#rdNIQ4EaVXruc5wm) z0&GoZkf)O^b^+xm>Ar$eU2KafDAvv0_bnuYY-9J3*2DJj2>E*1x}PDXkL@%AIh63d zLvf8Q`T<$}+?C`YeSmH52l5ZHeG1>7+#$B%0w_y~yFw^A#Ma@0qQl(9OCe)~&07uy zb++|N$UDkbSPh9Wwiz#!rsP{4WRJ5AHb7>WyXz(>G{NR?fz(Mh+Yh;>*s4093?)}O zp@hMfAA};)+)egE;Tg6U4GPS%^$kKE$(9^~a+FkcC^g5nI|jw(xhtQ5pP>d~< zhSXiQ)C1(&V{3bYGL$@cffD;{)o)P5=593$g%8*YzM#M%+tfGYIbzHFLODum`3-+c z#n~~ z2>CDB8k-^a729?zl%+&jJCwX;TL?hW6nD8U$hcwCd!XPg+kGG8y<_VffJB<@Bm|`? zSvLaN_iUS^komw}aTp3cvdKwEb=a~7cAlYV0e9|K$WYkK zcPLoM_Wc2Qi`YhUkSJz*_<_=t?8)o?f60ZA|0 zTPLK|vJC_wUme?ZFQnA7we~{}CG0^c?ql-|LskQKD>|e%vgM6K{wB7`3COLoJx@Vd zO7zV@$!4|-2}N7DQx_nkl`Xai1^sO07UXSXTU>=iJKIkLN>g%d6S6zl9=9Mfz+HC? z3U#t2_8_&3&36E~y4kjlpbRBTPM}1PZ9V}-dbs;Mhr+#VBbQL1kL~Un@@Q;tPZpCCENR+NFX5ZlZfr*y6+-?ow$)DV*1sRiU$r>m)#nxO0c@4H*A0(#P zDw?1)C6}5Zdxq_|6*6bJ8*hg~lFbP~>Kt3J8*qA~7_Q^?q5 z%bY>MJ+}TNbUf2f4p)@70CLlY*)@nfJD|h=d zQ0R@#GY6^fY}Nwg%ChAxK^aO;SWx1F?Rgc7d~(;j4u!wiE;gY+j;(1M@_e(!Vo;8f z?tLir!?t(;#eTW_K7yq9`FEZeJAt&kf07Rg$d}L7eE}&2Y^RryLkV9BiYsi(1R0k)B4DB8(g zdId7N*a8tK*v)pl0eORLwOf$rVcXb&(v&RPgX~_mSsOC@xO+c@LK<7>7*hM$Qm2q> zfUWHu%24tk2_**Es;{9)h`ZGrC_KbgkcI-oY*P=AXM`=|KsickFHlNnyL^RWque!T zAvwmj_X%m^Y!%;-FU+?53n>$9zj-Zxawrj2p!g)4Qv_L4-1U?|y1{l<2KlGi8Y>|8 z4BNH`%2J}N21?GdE!09$$z84_XNa zcN6=NzR&h_0QqgU-Z`-tAn(Awt)u7_rP}D z1SyYftu2s43EL0F9X3w~WIb`W(h2F$Yg%7LiQKi<2+>Mxa(eoLf>qOWk~&D^Q}Uz zU$(6Xl%ZtFCX^7r|IQQhTTmqLpXBEb6wYTG*@FTFYz!EV<=U~R-1rg zMcl2QL$a8y=n~RgY%|x8uY~RW2Ku|#JH9eGn1&omq#mGn8JqtJvfSJqyg+(6Th$xn zuV7ovLheeo{4Xd=iK%ZW>0x{Mg`!p5Y5D#?8P#k_1q#-%H5Wr(FWYVjBx>0z+)$d5 zOBIk^$M)-i%zEy|YoL&i&8dad2DYFNay7D@H9{FmHZ((tCbsQXD57##+75-A*%ks& zpoQ(L3-YwG={-=6lIcDu=9X&r1EqmVDaRvdJhP?i#FYf!R}tzZL+YTOx7$mnOwY(v2T zw*FnnJIHpq4~Y<4%ORAeXr8XYJcX4`p&0uyXyuaIYw&3uP)l+1lVsVTP6927ISyZ?dYG+S3*+n=-< zwvz(LH_KLE1Syhj(*-${aFs#vIW}1iS@Ya|R6_a!+fX&+H`#8zkb9A>y&lR^;-~>i zF0s|9P;{BQNDE|GY)TsxTw$B;fV``0ubq%sV;cxUX-Z!8LUx3$wI4Fqx!WIvLK|$J zVMyI%vvkN6Wy>3fGL)Q{fD&75&r?ujo4ei_D7?dVA)!Ewt!V-B?6Socp&TXM7L?j! zTU>==``mp;AZfFWZ9>`s+rt**J7nvQLCO)^=^o@z!gm10<809*$U5e(27*Xi1MNaX#K*f60qC5x6JJD+WK1u_e`dtZY>3R`FcQVZEqQOH%q z*0uv>D0#38C5qXqZ7Aa6ZuJlfm#`HaLxEDZsZ+>P#+EsQa+K7PP|D4Ac?HGFxof_G zWCh#a9i&yVRXjjG58JW>DOGI0&yYij@GBIrW^>*ltA@LtPe}K&o#h~ZEnDL+C}Ax?@qRY31X%;z zO<0gV$o8}f`9o~I>yUeh?R*o;Qle=aN)EH_#GvR1ckX@2(AmrbC^*XYeFS;O*hWtv zG0ygofYOxgx`6C3+vz1_PHXg zvdq?62|1LotDv~W=J7(-3U@1YkiN>6*8us~*e07GcZBV^8Ol>akp5y*^j*F6S>cG(hPNZn)eO+l`G zwykL>L&*{eC2Y3&c_?zg-KPnK57|bRp}-N_-3sK1vjrkhj*{^WD0R$My9LEgxLe@NL!Ngw zy%EY$GOa?XEL&$Q6#L-rxDAq@Y;^%h`(oSZf_yo);vPu(W}EGU97<&Sq4*D5CR)0VNCA3TC0G!ksY(8HH?_1t?g=*1rUK zi`gzMNVwQq)}S;c_tqi1gsn0PnWfws)lX!9*TLnyMKgaEnC+!q}8#VWFTKX zTm3tv_}Df-AcqpJ929S0lRuEv$lZtN`jg(oHdFxlRkqtg$lc7=?t-$EI4XsbEo?RA zP_&i1NF`+W*_3K1*v2;Pg}m)-uXT{j(F(v-Yvg6sfWYYSv{a<}h?LS1a04oK~0 zvpOMHkS#9=Whgn(3nhBko;4`a%U$mv6z*fY7=i*CTaylX`q^S*P>z!B2`Dwdwm1dF z2D$s5hGd9sOhVca+rvEM8)oZXgp?7s(`Cq^gl`p!>uk{oWQ}rHvI*&9Y;#+Xf1K@e z2XcqmhWDT>CGKn}Il!OT4CEQfnuxNmAfIi#QS|R^7TWLGw-eH>$Kv_zBbwSA(+ei---Q_Ob2N`>8fdMGE&vqPw zyf$0y2qX^JHb$W|C5yt4eaJRD37JRSy&F&{&K8=5)MK{P9OOD-Ycrt?B@dRM#3@_# z3KU6jx4H&}&)5n!pujoXR21@Duw}NP93{0~D3xTp+=pV9+%+FU@``OQ4r$kH6{nCd z#kPC~DK~7t7m!1V@D&ulWph%Hb;n)L9i*q(&h8=qJzJv#xgXfJpP?)z%3h)5Biq6| z6m_`EeL%(&o1TM$&usTUkoSeHGcWijkzqS2fYOw#D}wA-woMmgzHwJv28G_)WI3c} z*|L?8>w|5m8p=@e#tS7r+1l%&$QO5q4Ny48R-;0JZ??4-$n(Rdv_Uya8XZvTm+iF^ zii!Mx=ZXFxB=i1BT=ha)K3hva$rsi zKDMJY_i|i}|2n zkgdE4^7gPTHbbJ9?WYw=Q*x{wvisN`1CXh4*WC?;`q>gakUGHT(;(L%+tvV-p=8Mr zlnAlSk3f+j?mkDM@G#p*7z&KA-AzIsoh>j87c3$tFA`$LN6)3#Uwon8GHrT#gkY|%kFN1QFOqWBc zC|jooifwUsTn))>wz^tK+hNWt&wYhZ5NqD89!QYJ;qO?ru6D-DYd+ zg8T<;he61F$X4A4Wht@N4<(P-3PMmc&Ydv~8OLlH9SWYX^^Zf|Q?|-=3@A;> zy&1?pW2>Bl%yaIR7ogAun^=O>BwN^mT$gN5t5AlLJ?l{7itT(8id=KoxDACwXfnsUy?h}x_XY0CvvFDUuOR`UZz-?@v3zCRgRHl+Xx zey~j!Lf%id*J4O~u?>_$X-ZzXAv?#`S_zro-0fFEp&vGn7gB%OtUAag3UGoDc@0p8 zk`ql(B9HC48H(iplkD|F;R3dcb||2*HFZLsLbg~pl%u4(7fKbeEoxA#n7i)*NV?d@ zh9Iqk?O_D+m9lk@K}s3hX&7=S;hTcuZno$&WR-JQA|btkZEha&SF(MYklVvHybNV2 zakl~`tJpdsP_&x6_y%Ouuz9zjpqFiZ2lCdk74AWzj%~(<(v*BVgzS2@!DGnuad&+R z{XI$a-_OW@4ylc7b`o+mu~l6|8A`6)KnayCKMh5ixtn}|!Yyns4iso*>wAGbezxQ* zl%u4Yg;H&7yPr_3oxAdHNOrI-{X$xR?I%zBldqF)T!EA>w#OpKp+v9*ig&Xm${;Jq zT|)(=_pohwAb&4gX$|D=W1FvqvXuC$hmsoGNFx;O=Ps>6#sFKO6$%cr9k)T=5L;~k z5<_eoT~L~mMLm!`%r@HxnIqi2_d_9_Efj*(QMS}DbJyPh z$pf~lCP+JEYiWUeM{N6kNQtvmc0dj#tWGF?%qD`6b;8|5FQlKcJ!z0X!PYwnxzE_n zhoCGansg|6&bBiKMK8E>Pe4YJ&76XQmu%nDkoSshR6^pK?O`5DQ?hFjvQuoQ%aD1) zUHvK)x@C(-AoY&TwF$Y>Y;#*shLRsUP~x6#cn^v^aCd7%;YYTPBPigo9UViSCpK>a z%26_M4yB&i3NNA93wP7kkj${X-9Xwa+dvxfy|Gl5Tq!hV6`SvJobWPNbA zl7;k7w!AOM|HU@>4Y_k{&%aQX5`Fmte@cF{T_{lWhdZ?xGJe@&B~VZ(|E^W#ZpfSW zPjayW68UUD9w<%8u^PxOV0)~EOoh8{9~3HNOEf}i5u2|Wauu^}wL%$6mb60&7u$RQ zij;8o*#(75*+zPxKpERzAN2Q}Q}6iWKo zUbdlV19#dkWHhoR_n}}DTk|30RoQmqkZ5MBIEB)bTsnj77Pj9D$ZX|q{0a*B*_;%l zwy_28AXhuv**%n@WP<}GI@q?Kp-6zc(pMn_pVkUR@Taa92>xx0z65GihfueVjx38r#q%^ zpx_qU^c&>eW_!&-Vux+u3rbV+>Kn3SY^@^nCv%s({d_32$L1-7)O|Lq7;@Qcd8JT> zk`rzyalrOm0YwhE>#c&qM{E~0P$16MR0nyE*4*|$`&eq)xDHm*~J&;2Qp9aN~Y|#P8y5z282-2_E=0+g@HQVPXz!7 z4V3y|+r5KgpWKx{K=O-i$$_*S+s`xP`(_(|g_Iw*$9Kq~{%%hU#edlnImi-)|Ndq* z{6c!(Kgq4Up+EWa*-912UBEV91Z64l<${t5+ejG{E#xj;4jDyk0S^=`W;?Eiye_ud zT1b?zZPY_)N)|Ojb}8Ge3Ylfxy|+LiH(RI;Qp?#=9gwSnt*r~nQ1T!MB`Vpf`=E%2 zyVZUuT*X!pf&$fSQ^Sy_hApE*IZA5dP|C}8IRV9Lxob8cS;w|F18Mba6?2f!$F{rx zDGhADi;zQ!um!~%*_>6#YT~YE9nw{{vrWj~%+|OKxm(z_V^Ee7W&2RFm2Kewiu$?B z9YIDLn|=ZX+u80Dkhg=a^8yk9wv$UJP06|xWOuS{-a=*l!)z^;kZ*)-zY0=xwn{JLP{OK%;-hS$0kX!pn`nacaki&s$RB3w^+WCn zw)1uCsU~@^xWwOo9Lm5hbm{4MoZFm`qEOB?c0)>~^IwDZOVmsP^JS%M8 zEhtCH$PScRWh>l+Vr$$@+mMW~y&Xc@I@`c8S+^Jc}*k_AZ2*d0akt(D$!oTv9!N{E&GbRO8@Bg; zNV#Pj3_%VhQo~UEj?F&`S!wPL#v%Qlt!fhTKd`MDko%D>e-_G8VrmXbI&3csQ1ppA zZ3!}-*^(9%d|_)|gS;8G-E~O3vQP%;p)8wo0I45r!8qjl zWIH>7GL&pMgA!kC+ZRwI$6e_a6#iygNI`)gwy#^r^UJ2+Lpe&OAEA^e`gfk_e1>9q z|0Itykj!VRdxx|Fwv7+Sr?3_0Af=FP_6Kq(krn!%;zews0>~=n?xqmZU2JVG$X~*C zSPHpI*{aK-EG5<|p=23bK{XV0b7y!Vqns^M2L&tG`Wql`CEH~aBs^>_El`@0dw$5S zVyo<}Pv$89_GX5(@UPO=Z3&A&rgN_>4l$vE3c4vHRgm;Ql_6ShF!*q?%@Y{vzVH^Ek01c@`Y4HuNAWKkJp zpR>)DL*@l{@0Cy}$rh@H)JwLM7jj*(wberzN***oiEFlM6^f*|TWx{DH*5uMP~etr zssr-ev1K};93{0NluEN*_Cm3H?wb1{`M|a}2x*UO6~mCvVO!QA<%#Wg400$Do`B-d zY|a#9y>Qnv1L+yIGYR=$*%}uh_Z!>xB9x^>nFS@^*%nrzXqLNN1TsF@^i3%E$#%a5 zdB4~?W01(Po$NtrO4c1f_BY$+5oG>wS9}76e%WLKQbjRN5HfoXx$^!=3|&GQO5R*U ziF~&9TPRY%-C-IED{M87P@s@)?FsS}u_+lSM@i!iN)@xcW}%piyZ$domatuYLs}_Y zix~ftuZ(R!A5z?Gm4%Q)39A^2m$Qjd$g1FO!VT$_Y)=)C-^12h1-Yx(&TF77C7SA> zWHsB44~o`s=Wc=wFPqs61#8*9TOn^9+h{u^>e(IwP@0lm-H`2LJMDqY2JY%LDAdRn z9e~s(HrEj3QrYH4pbRBHMxjJA+i)0)v~YJj358qPI;Nq3pY3QC^0cve=b;=WBPNt; zXDeKWVjbK~uRt=u_O=FToooXekgtpFItnS>Y^^(xLkW8qiU-*|He~g1w{i&Sy=-~M zkiU;@@)UAwY|m#GnA&} z*ehg@usyy*rp{gWClnfGOXMJRjLr88xyISH^1^@0P_jgU5@ELaA}BJ!-KPr*PqK}a zL4hf@yK=~5umwC&j*{_eC^gMiTMNZzxLdD>yT@k?Q9duP_khg zO6;(0$Dl}zyV89qyvw$500s8gzK$TzKAV05Kq;H8^8$(;aCdwO$wRif6r>%o zZQMe>I9u^Oq#U!&K0*#9vQJR_ge{bTtW)l8-XJ}}*7gDU&)5#XAon?2^$(P##G07+ zQ}TkXpa6;{xibnOXKvUQC?+8f(R81lWd)lWf6mThwyawy@FQ2c{U&O_EGcONFCf3XcM zL;f7w?F!`nW^0c?SxOvjK*=Arnk^{$%Uxs#GKA~jc|zHPf_eWWrftZZ&-Qu=VjS()|skD%ci(p;#q%-+7aNk{-4(1=6b69*Q7eHCuNHq|~sT zmO&0Bd=*gK%NF%ORxNiWHIQD%HdhPz>)Af*A-9ihxDm=y;!cH<4Qw5)P_&V|cpGFi zv3UbfP-R>1g1pUag*}jHVVmiL(v*DbhwN6i!4PEnxw{^QLTzmRQAllPv&SJ<2V2!7 zl%eE`0VM)#`Lj@@le@_|DBQ*NvH%6T+4`0sPmnEXK{-mQYf!3(ZFe1t^>SApg=8Pw z(l(@NY(Fu`*UvV-4=Dp|j|Y%LiC`Rx53(grAS=XO!x^Lxv29&I{$aM#E66>Y@q^3on}iFLarIMHW!qkKA%_y-UMRlG=4g<$#$C@K zq(|7!h9LhsTcZxSH`unvpe!ZICZOad+rktSjdGWphKwyXT|&Waw)=U=yTjJG2#FZm z$ug9tWZf!c@3L)1Aajqq;!P;D&nCAZ)n?1?K&}I}p*<)=$r~F=9I~|^L6IZw4v(R5 zoUJAS1&-O)&LPhUn{o-|C}~_nsZ+Ms8z`3Gu0IXQGq$S-NIPe1d4hZwZ2K>el4PrV zgB(g&Stx$VCcYr+io1z#NWW%#`i1-{w%&Z>PwpGGa|Oy$qNx~4-m>kKK+!wy+-}H7 zvzZl8@Sg4419>0VMr$DP$o5bRr779vgKUTGv=K6&xT|l5LeFf`R!DtebG1XR4BK1) z%24v73rf7Q4fjBiH|}oxpzu3e#{d+_vK@sW&j*`#1jZJ#Rxq>8)G4=7yCHj;w^HEef3 zkjKjw$eaFCj*{^LC{@c=TLi`GxLbEYvYxG|4AOjTGv$!4f$hB#QX1I?t09LHDK8Xn zV)NHSmdf2h1Ee>zRjH7_g>AJ3a<{VOw?SD-Om#p>Kif+u6m8>93qnRaTe24lcCa<~ zL*4+}?jR&O*(!#iG$ogG$nIkM9fQnn?#3seP>{`;g47I&2&KMLM;_i44lEZ9u2aqxQ@Ttj-8t?d@_Pp}=PA@?L(^&^y}#M%>-oMJ1; zKv9D`;|(&V*)mxuIK$Te1$k%LF25lm*;>TRpVE}v%ZKbaw#q`toab)27z!=0iBd>4 z*}`tfwaE5V0c9xJQw1fK*v@O9$TD}0bx_!1+wnnx6}GY_$g|33HbXf|=31fD8rx_) z6pL_oAAsaKTUR%vZLpp6K)y}3dJR&dY?}j+LkZUq6yIW#BapSt-Nz`T@30MpA%BeR zb`o;$vb9e`SxOwuLdiY0nt3R?&t1fX44X|^hJpud(<_koknMF15=U$U8&H~(S5e50 zv$gI(<}r8syHMza&0|CADVuc&xe{!7$54in6Q@w(jP3ahikx%Tn}os_Y!_EhAj#Hr z19>jlVs}uElI{m6b;Y*mK(TA?zMmnPVjFvfv>UdEcgT0k*8K@7cWkFQ$f1Pq7mBCZ zqIt7_vhKMnQ6T++ZLSFNKeBzgAh*LdTn1$+aaRr{pV&G)Q1qF*cr|3auz72tV1{kI z9`e4j6*fZRjcrDS(v*B_f$Vp-!8XXua(CSUg+AE)U6A_8W(OhH7h6>yl%eEGKa|L^ z<%giiH+Pf6Q22-KMTY{vY<=U9N0k0Mha@MU93@o)O69Tb&OovJf0E^MkSt(ZT7Wc# z?Pn446|#+6kW$3 zl!4S*w$vNss$*;WfHIUk_<|DkY}G$d#K+yLkbep{uoVA12(puRnypYe&wp<4(ZEU|j$e~2I35vI~In9vO!Cj9Z(gSQ~ z?U28dt+5kwcd>1ELs?3c^+L&RwgnA}2D!@(Kt>OnJ_H4O+3rUmZy#Id7$h{dlQ5K~ zWZe{G_p@zIL*@W?#S#h)vdMW!4Y6fS$Th?^v^F^vXp4bLdkiyolhvbz@7UWGE6q}7YZ)2 zedo>n$-Bfhsz73y?V$)tQ?jcBvMsjLGRR!vuD${at+GWukh;d^s)1Y)wz*m;L&=YN zD6!5q+z3TBxVu%M@FrVFD-?*b9koH8EjDie%26`X1*Nvx3VWc~4tLXikc_dt^+Vb& z+dv5N?Xg`CL&`o|>nP+Hk)S>vJSXgF(CbrEpHa`AF)l&LGC!)^8%EmMBfsW zJZ8JFpy&yA>KbI6vc=Y+V1lhY3VF}i7PldB&h`_7(v%$AhwKZs#{TJ8}2@Dq3|u+$UPLeW4n8VJZZMTGnAubJOicf z*=paR*aLU#ACP=xE6PEd!#48+`JUL`#r&U?XSTrt$e~245Q@LB`CX8e;qIUm(qGxC z$|3(7+iE4`erLnlzD4CvzQZBa6MJQIn-SILcOWEpHA+3yUBLex{Y{i?9QqDHJ z1v!++?m+Phw$L7ARdRP@L%N5p?FjN$u^k>m?rOH`1eB%3+BuZ0VJopmK0Vgc^%Ya!Xm z*5!k=F1C|K$k)wQ-wY{1w#`<^p@gd)iubU|0A%%Y_t6FEeQZNLkY8iF?StI?Z0!S3 zmJ&xHC^^7ZGXg~ixr>ZKMu<%bL%|`o=}E{t%=T(PVuWpA7D`j{Y7Vk>wpJ4|N4eWy zf-EG6!KpyUc$N1pko=qh*d0?1fn^AkC32n=RQ3sbCzMxcw?eZIny>izq7XKvQ z*!J=v?VYWn5b|Z&mWv_fgYCBjawrjYL-9{GrvkFRxa+Bc^c>q+4dnl3YpjFZKWy7R zC`*a5CMfyKw$Kbkh5O%mBG(ETdH*E!b|{$7b{~Me1#F$&kWko8dZ08V>omwNWZN8o z%p&fJhoDd~n;d~u7h84|a+RwA6e;8Ga25)?*=pvYKsno*33)2m zlw~MKNn-^{RkFRVK`{?^{Tq<1V!Mh$S~XkC4&zGTXST|Aj*BY|*@>KdBvTE(LN0*yf6$3?)BYP@zz3dMMS)R@ex|`na1`A*r#wwLn@w+dv!S8(_Qc zfRsVD)-K4QgdK$9AvRARWDRk*(hupwYw{2OemV);+* zO}6|3C`*Z{LMRz!dntyZTij`-kg?5{bVI=%w&qI68)Mt8g2XOcg%?Uwa;Xlo_t<`Y zkh#y@coP(|*_>ubJzxv^A=e?>Sv!=WWJ4#EIAYuGh9Yt9N_(O3G24O$1y0z$1|ZKV zn?3~PD48CCQVF)sF(`J%-EkO_=WKOTkaoegF%9{WY{e2%F4<=1A%_xK6N+E4g_a@f zn!B47NKdh~MIiqT+u;V}zGbW4g0hrY+kujIYz2EzG|io1L&iN@<`4=#u=O89-bc2} zQ%E>$E$2|0l6y(WeqyV5(`-VK9Z00YNqhv16`cvwQZB&6`IqvR@Ao0Y$s)q@0YE< z0#Zczzt2c)dLV}qt{Nzw$0lnbEB~LwM?It$unjdreueE;h1`W~?f;W>N8QM7VHAe{ z@%{}jLP$cABq2$XBuT@^^2ym`vyi2WZG8?>ce8!WLsAc$-2oLSk##~b1>5*CSrtOK%oKdtb34WknKJInbd5fhmdNBEqVksD4_^LISt!x1d0xG z_Y;MLmd$+$*+$rkSCCf6Hj{v4qinZHs6QBgGVmtbR)YEM34M;MxZ8V_*C92ZFpJFp? z3vH0!!rfy#g29>3KC0fd()83#nzmGw99NOR!HV%d$mC&O0q0Ki50fg zA{6j&H{ya^t8Axk$mnJ3UWF8EY};#4jgr3Cp^T4hNkHLs?h4zGZ-dRW3t2YVZuTLy zU>gcT(iYos2r5vb;{=Lrvu&P2{vGZ<&mf1N&2a(gciD0=NWRB5aSc@{adiWw_Sptf zP$EOEA%AxM4AHm!xE1Y1%E6)2$@gJL&qC*zPm$zA6pQ*J$G>jvem1UE8pavy;UMTm-_Tht~ zIqv2+A@RhP-GXd+HiIA1KC@l!L9zl{-vLyjq`)DRcwv){p+J$l)iC6GWqXT2#uA(L z98$cor7xixC5^_ROquO00fpbWQ`|zn3Y-59vV5@p+(YUr+j16?KG}*ns6YwxGZd?_ z-M&EnFYbm*kfY8PE<^fnHu(o6Z?J8BLKRAUeL<-oHfIA0HMz?-Atq`rhDwwa z8-WsXHkBRw=^Kwx$~@P{O?m#Rl14*C4-|yO|BhF~pV zD5YiF4nm<3?&=}Pqhnh-flQ-p&!><|&t{534N4>~pxhYS&=nLlaCdYKiE*}$BxEzP zZKNRW1lwmCl1;KLJU}H%$~{7fDK=vs3YfUNDnPDjw*FVhXl4t(L5dl+)(TXkq_rxP zv9P_@pzthrwr|L1Wy}0PmN~XDDfB0`jqSV@lIGcz?NEUdd!10s&i2~{`4_lbQ9up{ zTS*D&7uhWRkle|3HwaZIF`|J|OKcG>6moIbJqmf2*>=Vtlbh{(98#^YxhA0oB?=}e z=V6ZdWw)L!sahQ4N5f1Q0|Ov`2&hZxqJD9#5tR}4%sf)k_||E$u|58$zp6LEk}PU zQIcE+C9c>+2NZ~N_a%p1*KCVDkTJoQ?}ZdMY?CUeMoHHLP$tPXI0S`nxeEaYLxVx zf-+xhQ}4 zkfY_FB;yyPZ)J;pL-IDZ{w7qR#DR4Dr<9DXwG9fjbLVY`JRNNBosg-MZLS+q$=Nar zs6h#RAC&82JMV|0-Q4x6A<@IOr-5t=wx$-+_Oh*vLNX=W>ljp`B#RMB^s%KTp@53J zk!i@)&vrTk83)+9t&n1nZQBObDCye{Wz=j-i%@uoyTTIW)3BM`kY$+d#sjIfY(s01 zG{Sbg4izZTA)uI!ZF3v)k8=0vha7q~$3CPVW6K>tas%5$2&z!x>KICmvkja=AtQIe zGsrW+)^-7zCfU|vkZOvp5{DX;u-!m86WhZr6rJX7EDZ@W+eHSl&9EsSA+3dN{|S=K zvi&|oB}(!Xp@fyKRDuF?+|9m2E*sn32V|UQ)72n_oh?#_YLwLT17#N2c7CC-gS$q{ z$)9|SY%UpOak9O1K<)S!fK4$5t^eau5q!QH$A z5?gFpCuG}ZGb}^e9k$CANaknj^Fk#`3izPJE}OIo1@^dG-GW^EY;QY|F~DZsgA@mB z=>Sxtq|rkt6J$F(g2IQ~DZ-F1#O9AcmLs;GD5O4STfT&(6Sm?NRG@@80mZ^>w@Ju< z%H8lC-82qikC_s6vUaJd`?TbG|^K3-0o-kmr(ZstlQ8Y}XY?b;YLs zgc_6xeL=Z6TYCeFUURqJghYa^Duw@KyJ54pLE0qSV>=|fWgC}6B}$5QLx~idsuv2} zad*%MxzcPc1Ca5a&8vnK8Md+ps!`J12$XqXyB~$ZS?=@($oI$=HA0pgTkjO4eq!65 zhNL`Oa|S99?8{60$q?WOrze7?xo3aWOD6v}8C0RfNGFu)VT*J@Aq98c3dqyTwxfhhO15tmr0QdH4MGh{6o#Ojifvj8 zMfO*01B2HB3sA+4INa}tsbv2B{55+&8lP(s7zn1uqv+&#@fE-l-H9Wsuv z#T}4B$2PD8)hOw38On^ZwRxbho;#lx@{O@oe2~S!Hopm}$JrjXAj!yP@IwVkT`~WoAoXLZKP%baBXIVLMAerdhV0 zTS#SP^WQ-YN;K}F+#K6-7K+-qd&xm!p3VFW+3ak|7f8FnHe7;a4z`mrRH7vL2b5T3 z6Q5AP$=%l%sw4aIzHDPLV+Xhx&n~vm~ATv8Bf^iAxIHsbDlsoN_swpGN)`) zQ79bYE^z_*&e+sfkR{4?bPcJ`+1itkbiuZff(n$VrlHs++rk6nk8$_-2sy6UjCn{O zXNwge`88YrD^#Jx!5frHu(ej8&<%IqD&$GBz1JYqE!*5Tq)M@6exL>=^wQa%a(8U! ztxz=0U2i)i?%DP_AzOy6*#&7I*j5yfEX($)gi4fT>4y@JY^gygkmGJd1G%2qPPLFR z&(=K(DW2K3$DkS|eUC$#0^8Cg6n^2ZV1j%_Hq#7bd1bqqh14asAsZyUu^ro?0wp>Y zp;(!1a|!albN9InIVx-p52XKK%XuMrm2F}js!-x;6H0xu4QxZ98h1fI<^9S%b(4Vz2{H7McJL%CtL4+9j{ayLH#i4nH! z6lBw}8O)G&lm@HzCpHSHhTrqy4fD9kZgr*{0k~kQtTT_c-T};D6q=ifpq>S zmzS-j4Kl8=dD|g{kFDGZ)hKDM8_KM+-7BE*26y^C$hXNB?T0Latyc}Hx7cUPKz zWLsKIfceDGUq2Tic7ZE3#dj(YcVJjV|$N7;VbTJH;^yR zmbry2*KA{HNS$Ci&p^@*oAMDVP-5>1iY3{8pCSJ(cPmB6kzy;AApISi<&jZ`eFJ#KHeYafvN%hF)l0gkh6gr?>j%~UNiav37(*ub- zn??!Qp4pC7khZ|qIS9#K*fxit5+&7!p+u3*p@Rai+&$?bSBY(695TMK#U~&|nQg!X z)hOxE3}xQg+Ge3}g*)FIFq*`!TK-(d4>LGmBAw;iZLiP=3U)nrQtpwKUOxGV{St!xPCUQ`q zo4c<(1^HBLvIb=7XIpPV>H)S7 z>GDs~Ae+4nDo`TZ4#m`L<8sJ9#NA~#*j|635+zw$V}D9)vZZ8DKyWwG3Awh|PP-uEHe0s> zQtYs8E1?=CeXF31pKWOn3h#1P7=nCzY$h#a*=M`aLFxe8&=@2gupN&>1xj>GLa`v* zrU~*Na`$P593eKxETlhT%gsUZW3~x9RH4L`14^B+4J<*SFn7Ua$aBip=7CHRwly!L zI%BK&pavyun@}#w_OJy-&$%1(L*jz%Vh^%ivMCQBZH#UI5RzT7{T@LjO7euEM4YV@ zfdbdu&7MQ91l!#uWV~V1#UVwKEs}t0l+<$zWp3Ga?x1jryT(1_yJK@@AxoO=B?qbR z*`}W%DZ`e0feMt+l%UuH+esPnXSwVAfE0Z0^48*WO`vcltZc_o2&4~kgd#S7>2a( zY?mXDtisl(hf0(bFhGe9HfaJ1RJmK7f?S_$Z_|*m#%8rZiZ8ab6{=Cv=sc9Evz;wK z;cxB~PRQ3_^SdC+58IC$Qa9O_S0U+_t+)miC}G}!V$$Hh_r$G${4M_^hIb%GD_eLM z(zmh61CU(CwiSdbl=uoksdhH!2^8w!E`JJnI@zY8kV(#VeF3Su*wj~0gA$=@DA&!_ zo`j-3+^we|p$#ihha6*UsX<6@VAEZpKX4pCwA-9EXV+k_NvVAT?16H;L z40HSJ6q5XnHShv_aU`|ZS4SRU1WPdgi4fTJBE@@ zwoDibEpaz?26pmCA7Fdxg}M*eCR9+3lHvnUHpn(G z1VsYB~J z12rkp*o6uSw&eg6zv1pB2nCXC<|D{`%a%NWOewbE2sCiVb`pi8G@JYqDpEpRL5X{| zuWKln;chVrc^=sEDaf2)vwgfl zmI9l-0%=~@vQU79V${H^#LVxY$G)&ILh5=9rEbe zx_=<^7~A$Qq&Bd9x7_@xb)0QU29+qO&;cclY^E+KG{N0X59FO>8&X1+DYj!3q%pB| z3_`MLw#^}^Oo`87C}n1I=%C{n?s9s_XJMNdhpe+~R}+xd$~ItvI_B7dW~f3*ZL?6? z#wll0#J>TbcayZ!xlM$qO08XgrRLO+fD>>tg$tskip01x`Y(# zY%f<(of6XtD7V3uOhOl%+-dG0zhFDLhn!n%omt4Z%_ee?a)+&!hZ>Z$_yXnqY)`LH zY?r&qGPJkH7Oz0AeYU|*XfnWd_ywsB*klc;NeN#QDg@a+q~xFChuqD#L4gokwjFXG zu^Hr$>6q=Z8yYxa>+6N2Fk7GxDpEokfD)%{t7<41;qFZXdCu6ZBak`DmL7%F=WL?} zsP%&F%m|ezNihW_FWLOlP$JkZ zVK;P~^)*x$&?Q0#<-my6a)RAV(Z$lMIn%afZ_iWeuP&mV#ItXn%u!Ta9 zEz8z^0_h&v)=wdMj;(qIRViV=fHF^Pk1;5c=WhHO63=Y08^~T@Q>7sN3)?{&>MpXi zJU}%{@;*Y@SGMvK6fJQ#SAe$P*zSvvqs*p%gADI%(RWBuVe73zbxQ2kpxg&rvkqNU zxpV(O{!g~oU&vWwn`yoMlktl!C4-c8Hf<-=prq3-DF4mYrGR1$?zWZC-Va+{1-Y7R zOM}qlFWd7Fq>_gC_n#JOQX-*)3N36yV^F;HpTyBP6li1Xn1tLiwha?xYG?a2LjxUb z3$u{a$(Eagij*+gp@f|6$^iwtxa(hnJl$-;Wysva*6M-O3br*b)Y{AT?t@B{WZQ(2 zO18`v6zbz{%nx~0Z0CEBrJqfC0BHu;_6{N0AlvT|RHnpA7)q(xN)hOIh&#(Uoz`H?ci6q3&t6iI&u#YLpajgR*9} zfet7-!`-1A+P1K@^+1kUHeWAfu(DP9AjKTp`~Xy^#Df~j+1L!j(8WA=7bB40&eo@g zoC|FG2FU1OlO`bLBAaIlYEaVKG?aI;&03(?5_f4UwC7^e%|otbwzCCj(#_W6gj6eR zeiziFM8gdgJZ#IWP<)lUmo+HhWixL;?lrcgfJ{EN;T>pTo$X{7k~Y}n0jNj`5rh(( zY+oTLD7afZfjnDm`BTWe%{Cc@)H`g~7f`F8ZSV>zQBvp{O761Bl2B-myY&?0-Dmqq zLzV!W{Q=S(uw@@1S&(f!50xo#S%6Z9Y^qo2IK*Ax4e}kawNxPMG23bt(w?xDYfwj+ z&H4>hDCzzON}sZgN_T$>N4Sf&LK|mnigw5rW!vq9bmwe8U6A~O&8>i{lqf2p%q81Q zKNN{^cRL7)D>khLvd7uNT1bD*)-?)sC)l>epc*CB$D!;En{yJ1Cb@eyLEE=%Q!|hw z#g>?b40mj58>C3H9oeBeCE6FE+&$aI5_FN_uDT5QAJ`T=kTc8n=!J}rY{qp+nPZD> zLJdml--hx}YzKZQmglZ@AKH6n^BzF10^9o`H2K0dcMPeDY?(0Bq=fzqD!j6tpF{By zcfBzv@W!?mhumehW&$$3v#s1h0~NN{J4pIqvt*zmB~n=^QDqx>f`XsiojyaJ8e4Y} zGJmmcmmqbW?Yj)MezProKqX2ld_u_vo2d?kez?17K;9n! z(sH&nJrwTZu3~^Ty4h?KkgbR9VG7bI*v8C|yqE350#zxYoP#n-w*7f1(#PHJ0wh#y z9w%h)XDhiN{Q%qS3e-KwcDD-EC`spovTC--1{59Qu4fC{*0AmDK#pOy#x7*gvbh3~ zVubA_2-PVueFWunY{?VoVw5{g1oG?IPNI-=jIHw$G8)*#6{H+zt6f74N?J@pc_Z6X z3W`l|H+c{3O|r!wAlDSzU=Erzu^r|i)ij&z1!_{l_X-uvY#(n>e1^OE3KX!gWvh^T zmd)@5nXGJ=-_XDuTVE5BY-|DP{!c|pNNrGJo^7=q3fj4Q>x4WDY}Rhb>|jePAoU{K zXdl$-WIOAJN|dBfL&+sJzXl4qxckvU-etDsQOM$ED~>^$6*jXGl6lx}C!sPWhNq#_ zDqDC4I`(oWw?e)(wk;cE^|5`~A?-Swa}ny;V9PH-6-t_NL+MSnYY!9_+^N@~jV-p& zI%L~sYZs7ihi!culKa`JeyB%n)>H|_hgOX0`Q2vsw>j#R(xZD1P_O96KEtx;L z;%rMYX!4rvxdT!q*i2nelM;y@sBptJq=e#0?v7MY;Fhgp5OSy3HijV69oy$HG>~Rn z&_U8YTTTxZDPbIk5*fCu2`Ko$UB3zPWZ8md$o$CGIt!_DY-@8+>l54iJXE42n*&Pb z*)mQj^vvDZGUP3=ov%Qa7dE9A(iGYDe30yw?RNtzQ(|QcN|o43JJ9hPca}ZKS7y5l zK-PD*kwZvZVT&9=9UpAnVW>h$I}s>dW&4go;ZN>dm(WIyt#Ad|zSyP{kgm>llZ52o zY??c$N{QopDAQo;%tDbL?lyCfXtLGvko}j<@dD|k;eWS;r&p-E<)6ev8LClIyaHug z*#CVC$KJ zRD*2(X{bqw#tc+Yvn^Yp_z-t5HYlKBGcQ2yVYcKVWYV$?yP$y)wi7ob>Dc6}P>~X1 z4N8o%eXT=5J$H)&@{F!*-+ zitXbJvY6QH7m#L}EgOSmX14Kbs7#5=8z?oyrbHf<+lce91NApHtkmjdecux%-!8YR_LP!voh zC?4Xj_W%kUvF#l~?qjy*5o9`HTM0u0VYb%@B%QKZ&Y>bDQkPI7!Zs3zf@j>FCLm9g zt@{=-pR;Y>LFx;(?|Z2Al5Hsql_;r@gOV{e(=!yh;_l`J^2XVQN|5E6?YInS5^NnG zknDzS^Ajpl;`0kiCD|Me==heqTodx8*d|&Y|75*myJ~~9X|{n5sN+rzXew?fO1u~q<}6yxzp@G{ukL4~I-3YW%5Sz>2x?H$;t7;*usxkZ zu^;XxqtITHEq(#He%S`Eph-#l@0M_Q4XIlGNs=X@CMA3+sL;yxk%r=J+|55g0U2BN z5puV)8S;>+gYB{a4Ro^gy+V?lE${{vDIry$L>Jp?6$*B9_f~^EJ#5x*$gE&X|3K|^tHLLn7*KV6WwpKVzISq9jON=P%vX6}b%YPQ=!s7#4r z4U`&U3u~cc4R`WU$T!TkH3nI=Y+vJ$c7)A233ce$@+PQ4NmDaWdX(*Y77FXRQ`?}8 zF}9E$vKiRg7a`p^+xikDH?mcip(-Wp9w;-x_UMHoliZE3Lt=_8wh7ryY^rTYKh1XF zhq}#dE&EW7lDr2{c80Bd2t_U2%^gGAvuyWa$YEvEpFxH>w&*#eu(9>VpgJXX<4|s% zt(kx>?A*C;A^!r~>mB5Du+3y3<04xs3n`s!+9#+%NvF?Heu=HC2*q67ZI__EWwv@5 za=F=-KA_1Jw&zbs35_9=%3 zHrN(=AZe2=*9#RXVN^j0!FDwO1-H2CAA&sFY{6m3yu;S2gVcVuH9ge2%l2-7N|a=q zfRcM`nJFl=&)t|A@&?$>Es*7aO*sc?f^2*9knE7{cL6F>V#NuiLTn`$bbQ2}Wd-sb zv)!#i))Tf7AEXVlMK++0Q?~9cs6t6QJ5V~p_Pq;*&$x31pp7V7Aqd&d*`|*m-38mt z2_(N{(?p;uC61#|CdSrz2}Q2B+q{BAoUL{Z*{|6gNl2eydrCpwH*6F4P>qt}4^TGA zHjsm&x7;1(q3sk~+Y98lWAnX2hBRB{4N}~*%~zm0B_66!F2iQ{f-WAoyZDCuS+>3= zrq3h zS2n)}YEq)1g$gCMGLEwu z$04PWEj9r)D5>8BpS`c#XT26 zC@t95noxL)yNXo!)5bQNtqrp6usyUxIzQW(9Fp&{U35cLN+^4w%pTi*9~9Z=?zbNj z0XB~svLCROG>|^XHah}!AF|zzLN!X#8K7*4EnZw5xO|zPUC|7QMMB|3aL?A4f}{*vAPp5MAw58e2e#EmD46B$?FsTcvRMm|ImebRLh2{B z(Ko0y&vy0>l_*J3g_6%~{u&f2aQ9P(yf19aKai!!R{VuDuWaVlmp{o$Y_~F~Oo`!6 zDD}n`?t+fX+{qP?@11Q+30W&_Un)rZ!R8!A*e!0Q(7qf$#$)S!Zq&HW6;JI zTWB1z)!Eu7A>B9Ix(Sju*s5lzN(uWcl=)$MoP#1w?#Atq_+^VZAiJddcS}$$LHd?| zk`9)k?pC%I4^*QhuNTU;v6X#LRL0%hCbZqocE1HVI@t7n$k53a-GdZzw%!A%PKn(^ zDA&c-Jc2H|xpRjhe-GPh1ad0aX3imFFI(yoQYzWBai~E_rwJ(E$JTWV#Z=sF-$8r* zZ1sD{HNdu%g(e5to^y~&&1QOrnv_VqK!qW;p%N6=aCcOO0>f+_ACOziw($v>M%X^T zpaC7*LIaXU*>X*&NC{(0@lOdo+f^GB9OJIP1M(Qyf^x__&eqxksf}!Fy-@1}+j}2W zq9oe@l$>PCsG-mlcVokl*Ti-{0$HZnlzK>GX4^ABvKh8tBUGlu$`q8cu$88v<5})3 z7RYC1yR$;pIku5`NNZz@EI=LeY~4<%LP?p&+ThJ&rJ2H6(brZ*s+ zlkG-8@+CIS4pgPY@h+5cv2_NZ$TD}EK}fjSY9YwJ!sa-E^d7dSQ>c5DZ6XTQC@Fpc zWxZ?zS5S0~yTfZ}+sD?HgdFQ^z7%BGV5_7d#U|VQ15~HP!y}XvY=%5^vBlj*0rGFN z^}RyQ9k%^9$mnO2Dv)xQ%~ORMl=N1E@_THv-%xCyyYvsV7huy#uYYnKu${F+lR>tg zc1U%|=I?}>lxTE8g%I1a0*W7T_o9RX$86?)$bG_=9E40^wqXr4aLRU~g`^0Zd=x5D zLX1I)Gq$gBC>Z5#aT4;Jv*k^Y`GRe722x+LUC%txN;&Ky8-Lk2+q2m;H0YBusV{6%mtZBB@14w(% zRz8F}GHljks6t8iVJQ8;HhKnyv)o0`p^Zm2MGUg#*mmQP?uqRu0m<`h?pvrziQ*lU zd1jl*K#>A>w^>NMuxXzldyy^t4C!Clx{6SDiEXO{)hMZ6hO%#L&JQSB=I;3u+J0x7 zszZ(nTcQCOKG@X1kfO?V)KdCWof7RbDEG;>(E(l5xU0$`{}h<^=K%unk2Z%OKlv z6w;{KIxZpE5ZmSzRHnq|HI&k@Ig-%vFn75W!P!8}x< zq_!6*t!G<%g~DUpRo~ zZ!08BY@T+=KFwC@g!E>%*>0$NhV4!P)hJ2V2W2g6k$xyT%UzEe+P1RoXduTNTSE&O zY;3MkNHNd$G6vNtF>Qo$cDCdsbg{slW*YK4*iL33=OSCD6*4;6gbh+IvDNHQgOU~( zp}dRjX$gugb2sUR_S|f759C^58(f1XJ#2^TkZP4pCZHxIeA`gL%l6@i;%nT^??V9} zTlN5Qud^9KkZFVM@)#P}Wa~SHB*7LqgNl@pE}+B~+iDC7Zgck*hdeuM)*Hy|XG`Bg z>Rqqtjb5Qn{tvnA!Q{2rtpzS-hdne>bv+0*1!#!Jc1yW?#dc9Dc61zSq z_rTWNfG)D!xwjzyBiri^_NsSwp0L8=GnA|P=k_AkD&ZBTUQv06}a1uKzlE2 z^(f>jvMpUgldo*gSCFd2W=cR!N+gm{;f-zR4vLq#JGzGg?`$1e$X#LE$U&wLw$D5? zP-R-15!8G)|yc358J!+{-+Wp z+1j9FlP%K@g?_mklS5uf|L>M?-VIq={z+2yLYh{#y*^0R#`fC}l_{~JhEg)Nk_I|% z=gu+$`8wF{Mj>k_+lT?u%Gn}DsH2OmdkU&h(#|xL?q>U*fxLIEqAxwY~q_Z(YN2AOPZ!=2E;JljbZB-z>I z3aCg4p@b3(Y+ouU=-_T~5b`Xt<%b}%lWkH9sh8NUbx^B|ZEy@KQBr6eN-nd>CZUj< zyLA)fU19q$LlzI4eHPNJvSsHWnU`(c4wWf!>3~veY^o*b*vDO98S<^OwRj-w2HUC^ z(r&VqeNcyBvu;8aO1j^I(pzk!eki=nU33rH*kMx~KsGP zF#=`w*=Ek6NPxTBOGq5BY2%PR$QDjO`a`y^Tc|t4wsi;9D5-uAWslgLStxqU-E$7w zK4F`Bh8$tG#0z9NWmA_RMTG6B4Am*o{sHCA*fu_)izs*1FUWt+w$Ol_7i^DB$au+S zZ29<;GR78bgBq07-vQ;X*bd}SEY4kP543m9=Iw=C3AXn>X!3?_ZU9mx*)nRVNeTTh zRJdh3AA#a2?t1l5;Erw20J+m_O(SHwXIq(q1~P1~(~$JQX0bp;N~Ek%BFi>14+S5& zJ6(W0Iks*mWPW1Xc0uYq+qWBPeP&x)g-VoESc8%UHq!^3+YCZwN_>W(RGH0j0v*3|mpg@g6}E{eWc^^fx`4D*wt*|Cg-NttJ1Jx)=CsqHHm9a%yp=djIJ?+qT2is03C)h^Mpw>yYvva6KNs1VhoMQ9Gp^%BYp9JKcW?R06EM~Ui9i*9IGiM-~h3z&A zl_@d&1f^!#!q3pLl{VeP=}o@{|QwnX{rvT7uc>FP}sqp z`WM<*WDB)?{>kQKYnMT~CARerNbX{*%AqPH>^)FsneDL`inzHOS3zQhEj9qzJ#4BW zNWaQZ98{;o z?mU#+Vrwoy7u(#qosfTr?bQW2{cJNUka3qSwF)Ws*t9;VK}n|@P=24SYYU16xZB== z_72$UyO1l$wiJLS580lBkSfGxI)a*%NSr{0BetOk6hG$fC<+Bm*g7sDcbIMC3NoFt zeO^NY5w?XSB%QJ4Qc#f+#(O9cWxINSg6G`z=OE7oTQCoqFWFjOAa#sw?GS9VT<%b9S>~XYN$d zsY6mXoBRhVQbPPfi5|AEmajhr72GY#AWtt_z5_BV*(SRnbsyVx57eq+8&pCiN(!l< zWIvm15DE=&w>|`U2iZP`A&Z*Lu7fl~Y*{@d)3A+?LuE=_PC%()HkAoF)^ZmxL%tEV zmRZQEV_Thrw4-e0d8k9rW_3UnO1gJK=`ps^WhiXmF1iA3jI$}ckj==p>w|O?Y(E>2 ze3H$*1yw0g+<`JvY%_aM#Khfg020$|+C#{0W(yxd`Wd#aFw|{f+loLnN~%Yp>@1t} z5{g>6d%l9U=h&taki*87NJ55rHuW8(u(KWALv>2DXQA8z+eQw$aBx@6L;gj!g%`-_ zWP5ytj7x0BGNg2|#VSyPlKMZP{4(3Y7Zh`I*V=&gR@l5v$mL;sm+F6-TxFYUgH&F& zOgq%1gkBC6*4WOwp}3E`-d-rM&bHSFxi{FF{g7#sZAA?Y2)0)ZByF)-MxY`kQln5} zn{C7Z1$VeRH9{UgTlW-X-eudKhSYm(-!o9_KHHKNDp69w1|F2teMlQ&8wf%j zXKcX`RH3A{6DS>JTRVlq=iF7!pp6SQ+XZC1WP6A~x)|HoH6*`cySRa>lu)LiOq^{$ z4MncG`^`Wi!RC2{>^E$sCrF=Un=L@yw`_Mss76V;Hz=E8i@Zb8JMMa_&~}<_rv^Fh z*&21okYRKEK#B*pmtUw(iRsqwKjpG)Nf~tU$epGW^5@u2x*+EhTc-jt=GlZ2Qa-cQ zR8WJG76+kxf$eDsioI|*sfG56Y;hgrdSx3NgCee?oE>s^%u{SR2c)vF=@+0jC7dooH>-`b2oGat-INx*N|<6O_GJQ9=6RK zBw1y9&qFmza6Uj;FI(Xeimh=sQHFf$Z08lozQLw?g>*i)@EauEWcz!E>Xf+FfO3Ae zS`#|n;?Dj9`2%cszmQ{_ZA|$6C%wp)kU+8>HboEgK?%FPP(H}^(+9Q%Q1O^egiV?n}1)5M)lV3Hy-hif#P>YE#17AyiJY%^gFjYwq%KD4bz4oI#!& zwzG4{l4a{pLF!w!P#Wq`Vmkv>a%`)&Q2LI$mpdquXR{R`?>$?#2w4kkqfe0Lf$iiO z5{hi{DpaLJu?A%x*}m&gw8Y)=C$#>=R{nx)Wj0F-(mu0Yw;@S|ZTJtWQ9`V%^H13q zHmMYfRk_=eLB3bEPdQ|-vAOyo-5XnR0Fu_(Ce=`#5-&AS?ww6N0v&&F7a4{8pKM(Q z$kAZiFhcq-wz>(DHQ5|f&<7>lPeb`{ws9L2Z*iBjLxCSQ>oQy-f)0Vx%1M@gtfi9MH4sh=%y1zjk)Yg|L20k*{~ zF)-?Vt>$G~7vEpztu8?-lZB+1}qE%Lv>22c*`q z6&g^75{=(bWt8pw2TJR?8|XlhF}ASq_fK8}TSo#}$Jy3;AdQjjwHFd5*z5|ZN{Klo zlrgc54MNdL?oNlGbu*hn3)!aFf;vcRVf)cTl4-V;ai~TKl?fuCP5EKn4%nL=2LzvRxcQ4N6pVq64%+*Q;=(et(Jz2KDN0VNU_OwcMCNs!H|cFezrsbN^Wu2{|E&GY`ag; z(l%TB88V4%o+_l=VSA}TElQkuhe|=V>?d@w%bl(Xg+gp6Ey%sc*82;Y!))Roq}pd| zb_xIc_x+=UWhqpSuszA3)B$&vJ}4YzOZP*bL$+ZRWQnmIsv-3en{*iJP@;bXsvNU@ zjzZ}Z?iLJCB+gbeLf%t0(iNYL|IcM9LhoTqUy)8iN zmu!wD$d+QuyCLlr+xRLZNwb};K{ZNH`k?GJTgVT^GTi+JAm0t!>JDVjvQ>8>-7T9f z3`uirw-KmLiKB;5?v5>f1RdwOlgA>hHK*?NkQ;hAl#1j#FGjWX1rMAr*ccwu{dg%VZnChL&+%9i?o zTs1cJ7i4^6JNSkab+)cH)T9Jo2P(d^)rGEqN`7!R-wg#n+3tIwr3RZ(4w=5#k_t%K zWE&WOT9mjq2$jCsIz!M!i#x9t3jMIX>L7QUZFUSY|FY%AAytP>Z-UyCaB7Cie{6lz zP)eBk@0Jjpfx=z?No>tR9tqou1G03pJv$+_l+C&bbto~j3{`sAMm$hj#@&$@iuAI{ zHXyH@EwBk$``Es=AdP};QG|qkwo(wPQlfbe$|%{c_MzwicSBKVUBwoSLAF6Q$qA%Y zvu&P2k|DPDGpI%h&I>53VJoDd*f4h!*N{)kc76ldN7z(3NT*{9=OO7R+g|~yQ{vhq zl+&}-o}l9~?(7xFZ(zHtLXL5^u{TI>WJ|n5vI#at1NxwZ-6oVbvHi55_#}6pUnpQ^ ztNcOEDYhAjP@!LA=IJ7_7PO^v#rLV^cHt72`Ca^vzQ(Xnl{ZT!L(2Hp?@l-DkUgfg};O;Tlw8T(iyEp+tteTL&cGu;~{dSC%cl2pMnL`rMEr$F}2vnv~G;Ld82a z_Xd>AbN9Rn1@GCW1JF`|Eh9pv2R7|4q%5)>?LjR{?1@07N47u|x+rnih(Vzzw#5_3 zU1obch0M=v<|L%5u%#}bHYE&QLFE^=gKH>N;B4oQ31wkM!EC4NmpIT_oM1v>8Ku4IM$a<-{i$kE4kH3#Vx zY#Jvd>t~C)pbtvuUWW2Yw#^kNKEU0_Dilz$IoBcQAlrivGN{=mwjlWs+r>82phQ&= zDrngDLr`LvyT33bYT4EgAlC?6?GQ5R*yfHQ#VFfd9BNX6;S4J3*%Ieaa*VtF6cjYD z?WUonakh2_G8x%Cw~%sz?d1+?QQ}MiDw)`_Md)IZJKYl$GP9jLL+&ZI-YR6au!%KD zHOSDXpK$=Ci!4XJUVvCGIRZ0{LP-dBJ!w5y)+`XBg z^%XY96lC+T<)`;vol=Dz_jV-hQ#n!p|U4(oaY^!d_?qjQZAl)XL zZ4Hw8*={$WIwg+!q1+Z*JOCXBxRdWd{%y9MUC1G_eeXf~9X59ak_FkyQRssbrjMZf zF5C496c2HyO+bM?wpbE!hS_>9A;Uh~))gd=ur;os1|_<(P~m{>F$X20+)ds?;vrk= z0dmFI)FsGx#CA}I6vu2`FHn;be6LXPgsuJtCF9)9e?Y-gw)+OOlwdP{L#8vfCl_Sy@%Q*5&e$b7|?Q$ngVn|=^#Q^M&GRK8~G z(?Y2XcR?K#zF}+WAy1ZVWgN2HvOP~g>KvQZ40R|mV}UAnY$G#JI?vtFEEKtClQ|%7 zfi2*KtPgBoE=W^kTU>^ON4C-mRHa0-7s`~_uGXRG6L&+K(0Z9Ix&_&u*(4&Qt*~td zA;}BddkCsgf^#3rR@n*%Q0$evi5TRov7H}7_BS@wDWt2jh0h@AJKNtmRHwwX6qNg5 ztEHjiPwwnDkiWrpcMCbb*v9gZzR8v-K(cQ(#Uu1V3A;~FzQy+Q48?!A^HiZgo2^oV zoWE={?~tLxcJm3z|JZa*s6mOxEvO(&|94C1{e==;|4H2bgG32iv#a-?T-|I-QphM} zdy+wl9=54Is7VRweyAv8)2N_iFL#G(C@5#^9)_0s*!&}qNx}9p3Mu>977S2}5+96E zNy%oKgf0fSyO@GPDz-r@mmg4Wo=yD)9Xq*;d_(>P zwyrkhaItN4ApIg+U6B8iY>Ca$4Si6;eGimhW*e77aW{8K1r%6eQw~5*58K`#WLRbU z9fIUuHm??HP-0aF71r2h$DqVIcemq^xWT43K`tL#+zc5v+4`m-g`aI_25M45YZfYQ zvAG>kGQi!l6AEs#O)o-AB3ot|GVQQwJ&-cUcI1Uxl-RQYm3G+zo6tpwyT%q2+GAT3 zA$OSVF$kIW+01*8D#Di9huV}d6otwMYzHwY73EHH0)-FRe5a5n#`b;&S&rD|FCg_X zTOkE?DA9NgRZiH>Z=iIXyMY`OIb{pyA#Z}MQ-G{zY-^8@Cdu~t1PSMC_6k&`#9S51 zT(FJ3LD5U@PT!&R6q}*}*{;}vO-P$&`)NUvYqphNs748uKPa1Fvr78@DR#r%jTG`_ z*+zOH`z_mXAEe8%$&`@vj%`~7)hY2y4dwD|OT*CdJ$I!M$X{Ta(nF31wkrdqFS2PS zAlV~ZbQ1cYgl-FzFR^V}q4*PbA2ujZW^>L#&S$oVdB{*rSZABth7|8?cRNs%5)2`z_`#M4L&;C>`VXLBgKhT^ zTKZyZA3>%jnT|0&Y@C^Eqe)F{BWmBL!ma?Nd|KNvi06V<_?>92dVzp zn)gtf5|)clS+M?hOL!_lsjmMdSe~J*fs{BsEWI{ zL1=xD&7py8YPP%<(hjkWk3td++u0aYqXeZ9$_}%IOi)bA-LDz)jj*jwLv|fobq3On zvf1pARL^$nfa;Vux&YQrr2V8kki7}6M+oVY+F%CZe?r4pavznPN2dJ+v6#euyHq;gv434 z)CJ_Sv#GBj;~d+;HKcH`b!DL@CHQhsah|Q7hmub2<{zNo0^9u~wB%wlmLbz3Te1Qv zm)Hhgp%x|Xy+NgAw$3|r;pWcUfI=&5uT9A9VVnJd%&Tm+6A1>)Zu!3V_-xtrL6d@;83ZODGarV2v3W43SzlAf^rg`qknt{p(R zI9u%yIzHvjehm2&YD|%GHkt7C~?Exb`27B$J+S#@kg3S_A%~QYYzzHRixM9Opi+s= zq=qh@xVz9mp)%Xx2;_cd+aHC@6*j>Dsb1LDjZm8s-b_%r$~HFzrCzzqPeb7vo52Qo z-q_CUkfqMnKM$$j*+L6YhZ5V1Q00Sd)eWUTxqI4f1}mWj7#elWo)wX};M` z0+7&RlkY%PN)&gY%n#f59u#eJw;X}if7!}W$kt)A96{PYw(Ao}BFy~vTriw~YLpO5 zLfI}h=_M4C{3l`S3i5TceO^O$DVr+`>3Z0TIY=sFo4kkWlz907<$BrFCFoerU8D^8 z``Ef(Acul&;}z2Pv(?`qnUc-%0ew)yeFMr5u#JC1aTRyTA1E-$rtCmYHQSyr@K1&z zwqFS(*RXkepavyYd!fQG+pGdgXt}#pLgEOUeh_l$*y2Nwag?o33n}z$J36RI2`xQT z9Ak5jLrDX7&l6B^oNd|+Eg9J|7RWTgrk#P5Cbpwls6~lA4yZKA7H~orX6_m;C^W^k zxD2^1Y>z9Dd791ag;Z9y)H>9rgrQBSJi~Ud1*L4QJIF236+S&X1wAle>XaD6+s7K7+h2w$3?ZU1VEJL7FAD*EA$7v)ON;DkbJ_ zp^TetEDuFjxH~OC>mD}6BV=1;3qCd1*)JsBV%z?M>Xi7^rTV8_fNe<%9dC12l0klvZK@A)?66(+L;4__ zMg_@s*`jLbgA%%jp?rvKa|DX-arZF_1;T7j1LWLidoV(V2;0OYBtKxgn1ULVsIo$Z zDBHdbN*r?cXNSZX+xk4@I%2CWK*nRXxg|((!gl9|nv`Hzg^F>u#2S=5<*wfc1ruz$ zerV~8tsQ_&NjA?8q&#PP*@aq^I1`3S7i`%GbaBa@?hp#4*iMch_Z3@j95Scb!~~?e zW@{#)HYF@yLgftG(-oAu;m(qQ!dbR-7V_M(4c|eQ9NXbNq`qU57NHI$`b$tH&-Ph{ z()Zjgyg-oxTk#e0KCqeUkhREm`2lGj*#^HLp~M#XhN_e(w4uxs+eQb9mbrTq2LEaO zna$A+*(z-L9!UGbHZF%GRkkw)RHFpt0F-@Y3k^cC8h5`#knfFcRSVhcY*ihkduOwa zLDCPl+i|E)iK8Yc_sJGFL&pv7fU>RzHK1a_;6YpkN={eF|Dquo1}Hkk-OvQIZefc~Lbho(i3QSH z**2|^WQOhC2GuCRIR|BJY=wC!Hp|_F3-a07&X*wj9Ghwd(mB||tB`b_?QadLQ{tKr z$~oC;e&~3CJNq`|cd^~=K#oPWu@IzRVoQV}*)p5r0Q#VW-9sqvX8SpU;w#*F;!wcD zR!Kn4RkoRP$lztWxrF3vY`Qempv2=0R9I*0y@e7R+-=`MqK~b454kqkmWq(k&-PS; z6kBXl&rp*R(l1akz^17|$!+cq>rhZ+>;8n6cG&!1kSWOa@eL_=*%sPRixM9?P$|S_ z>Kgi|i#_fxx}i{*ZBPce_u2O4kU7F8^h2rxw)Fw1O$l#9k_U(nDDejgxp!F-Z@+M?UvsnU=_L}Wlgd`cZ;a#Xk39&sW zd&4G;K(Q=$TT#e&%k~+A>^U~q38cGYE1p8qJlkXvs#D_S1(dsIQ(r;H1@0o(kpF?L zD+@V_Y#TX9|HxL)L$VT^;{p1hg!@M*|HL+4hT>)Jk`*ZM%%*&WoE5gcH^}h9_WKUW zt8Cr|)S$#_6Dqv2&Hg}%8h5w9kod-?7c~FmsLB&rt zw-QP=xO-MX!7sMyA!w<|mKlaj-)vePq-?Pr>7f=S_KZWNAGW{*bkXLnF$sl!*%mF3 zyTkTqh0K3!=2=K3%>H*vNXPdf4XI zA+?OH;Db7pXxxG-y=>>(P+HF2KoE-bv4umBSHac^L)Lz_wF5|_WP3e?gaJ1DF;u0* zTpY@%*v8JF=pc8e=g_*EO_73ZLu|n`q}8zfWFW~f+sZ9eqlC&Gl-06X3s7u?yPG29 z)3J>_LH1F$<7Y^xXOmSS=@{E~4XRV(R~^b3*p@z_<8kgvUy$F(Hr0Y06Kq#)NN-}( z{6VrwwrJPzKYdU_w-m~o**0ZRe2TjdITWz4Ir|~!G~2@fWU#VLs3G|b+l2;dP@-xC zD%jZeN1?wwigf7qQsdssN`nLZa^0+-0A#K$isFLfZVHWy*rTE%O>tZ zsx`Le9@M6UXM2i5sSWNdM^M?6GaULeVgHZ*S20 zKAYnMvPIbP4M=;yHvSDsqHJeBP>m9l9VmOq7811o6pL~9D}j7RY^yzx{g|!V3+YbS zYzjykXS-EGbxIr^gmR~B@ge9q!JS+S`OnyPbdV#-_N|BX=WOnANOr+io`60mVcHDk zFWIgwP&~z*b_NPuvBhQ~XPT|Y0U55@ww#bW!`5&?4N7z^Lxmf*#}z1%U&z~J z3;aRWZ?>X~8TD*W zSCC?iZ7KsbDIuMOiUu~#9h4mB?(iN88riyw(9#5(zXX{~Y#(JvImx#00<|dd;T0;G z*-UllVv4(q4=7||8~lRY(`@_SklD&6v?0|D+j<9TQ^K2|`=_#vZLS+i&2pFTfx>n+ zgB={VCD}4k$aK!8?S+&VY)5@iixPX3Q0bB_pn@(^+%?ot=!$J| z7;>lC9!DVaHJe!vsWNOS1JtI3p$Vvb!*(zUrLx>fEKvBC&1Z!?IktBjWVvITpM%tS zw!%Erp+ut#s@$`kFG1-7cLOU>S-nhHDgnV_j zku+p~XFJY7x(_zlEhPP9+rES9l=yWIHkUI^`C@`Zm2yM7}S zRIu%uprwAcwiz-h**w#ba)9k+25M2_j2$Ye*s>1jVvsxC0u)lSoh(A`A+}yOWY(~W z9!NFJ*7QPcN?6{2%38LkO(-?Ooh1N;b!=%7@{F<#??M(m+uKrKoKKb@f7k-u$hvO)x>sr0cj@L2CpE&%oe$Zs+1^Xq0AK9Mh=QvxO>Y( z>(gwG2gqh+%RfTe8Mg5BNi z?FW)N*=~QKIwg(@WB-&}V2ewjV;6Vw9>~AQw$lqame{`gApJ6%TM5bBY-JVnK?&1C zP=1B&dKij(xYO#Oz$#lz4>`STJ>!sJjcscJlCQHhCZPr;x-3v(gYD4@C4AgX&O+iQ zTWSt+`PtM?$hgIJ;DQtZwytHUNeR9csJP8mUxkt)ck}B|aEI;Q2Q3BJj9ZXtmo2#s zDMM@nL8wKEdm*T_$JPl$7h&$a2T*9A?e!3HN7!bMA@c!SE)J=pZ2B{(O$n#xQ2CIp zF9oGy+y&E6_=v5Qfjq}-E4Ps4gzfnbQpee>1*k)bnIcp??A?ZEa-xyS<#5E(7 zE3nl}(D4Iz_9@6;WV@S&9FJ^cHb`G$OV}aV6Pscl`k;i}1t?!;`&oqI&)j+3P@uwA z@j%WOwwX1^P-VN>faI@iIzQB)#Nz-|sIm3#K#4c*ws#@1&eq(6T<>g45y<$#_7sH_ zpKMb{P?HkUCs47$rb$4_FYXSLP_W6?eF-gnv-z(eQ;Y568dCnSEo7ks#U|H6RZ0|fP-c+rTMtFm+%1no>qBhi3CN~lvzQ_6Fx#~Sl4#k6XP_D- z#Ac!F2%FRa#dO?lIU(OD+oub%>)Bk(kZz2vxB^KHY?EH7PKlT6P;Q(}y$Kx~xr=N; z{t31&5ptN=HiD3TlC2(sWM($UKJ-Be_XkjZifudw#Vy<=kDQv5v3XNagA%K0s4&Ymdjlox+}++n;vAbk54jv{@d9L=XX|@}6i&9CC#XpYt!JpX zz~-((Nf&p|H7K~qHvJAQEwN=jA=5IOwh1ZSY)37qMTtGXP-%rN@CRLZxNCG7|0%S} zwkU<%UbaUWWL{%4_d%+4wp2gVri39CRNi1aP(vvncamW!yvgPpfjoY;_fg2Q#Wrt% z)B(1F5$aH)aT2O*vz<>tX_31DD-_vb3)>)Xkga2fth;P$^N=RQ_PPKGdu;Y4s7i@B zH^`4rn`35s8F_fdudX*TBzDr_fa$o<0BI}Mqu zY~l>0dSz?QLTyS|c0lDC+mjPYy>Vw*gu->U^fKgmXB+lFmJhZ=FQootlWssAO7w3+ zl?L1A7L@+tZb5`1O}1hX@_w_K_8@DE?Q$Q|{ICs1A)(C{i9uCL6i%SbFWbf`6zy>L zb_T8gu{kavn=t?1Eg_$Rv|ay67{7)j61KA&s749O9F*;53+17hl)K*ot(Z5AgP?~whGlLar6z!^|8g@p<@Mi@&@GZXWMB)4kg=n3(^m;xql&Iuj=#&$3XDGY2~7N|)HJ}XolXRF(wq>;P%IVd>6c0Ug-nb?dj z$TZ28T!NHlwt*F>MTvW>P-%*-vj$yQxbym;&@|htA97pSX15{p3|npoQrXz_A*f9W zr(vi(%hq=QrR>}V525fJTk8n&IM`O=kY%3jIRUAiY}RwALy4J7sItH|l7`YQ?v650 zWRXpF3wf8=0(X#gneFQy(zw|ci;%FwRw_YNN;E%184ugl3lv@DZm0&Wd)cCO$hO8N z`GmCVY@1(@WP|Pf8>&%)vkhf^Y=sUK+vIMdYx19bezx;&$iBs4jV;Y)c!E zG0yh12`Nt5rUFot64D}6Ot5Koq2w8NhkH;k$<`f#md@GyQOI<`_7Q`Wmuw3sP>T{D zPN7nY&6I>LuDH9nfI?}u!7Ipp&9;9HnKNuc7E;}?t>>UNCA{UKa+YoG0ZQF+mw$x9 zIW|KX^4zhVRUk{At^XBL-?N3@pbjOr-=Ru@ZM6ZVAGmvILXjey?FaHcvSoiEYl&@C zF#nV0iS0xJ31v2U4^*W@u@}lbvwinL(F%9VN@)Fst*nAsv4gA(q2Q2v)~d<%+qxJzzBfj>575ONC6|DF~1 zLXe^BKZ(C#NG@UX9zYFBtR6yzZnoKDC?VzUHV%nBZ2B|EC1Z=9L&jdVz7(X8v+bmz zCMC2oP_d8AeG4TO+&$kx!G5;s0<@%L%M>Bg0Gsv+QmWXFo}m^c_Ee$LAX}gYU8uQh z)S=K2+u|qW*04Q(LFQpLa|=>w*-~w&O$kGPP`JiMeSgV__S!LD6aMPVLaTl}#}Z*=E>+3y{{v_Ol2{X4zKUP>m8Q9w=*Pv#vq0 zIqq&YAfJP6#1Gl$*^UE{&dDa*fusv;+q+Pm62JDKoQrKK0v#`MSBgUZCAO&}$g#|J zbpq+#Y?=flTVabPp$|&vzJ&4~w#_RjzRKOlH5Bl&IkS*+jqM=^8P?e*?jiXG+r6Kzxx-z5 z2MPw+b_L5nE$yT|06i_M5mQ_L*``qaUp-_bFWC(H}u=Q#o zbCgZgL8?QxrXFfj!tyv&j=; z$WSH0_UVGsXWT6;Ly;s~aRu_8vzfe*^@8nk9nxH~4Q@h0iY>AQRVh&rq0ANAMi7do zxqAyi>(^|KeaMz!%O6168@BNnB+0U!9YZxrP@Y2BTei>{6w7hT*j7`JJ+n`g~AI%3aV8g?rdq0mvg`TiJmuy=>3BkXp`W4MQDD%tW9{AKS=z|h=2cf)$?Pmyz zPjlzdLIEpVMF%-&*k;BcgN^NG9Fotn=}b_A5|7PL!Oqq@4JGEd+n#|$2U~L%a?P_X zIUu8x?a2u#7TBg1p(Z7ym!YDIP2+)*i`*T0q2Ll*_Xf1I%;w*OOm4Q1El9b-wje?+ zN_+@HB@dfv54u?8?qVMbdD#Y|kb8}7KL(lC*@P2FwZXQ23biTW?F=gW*yb*v)FyZN z6cqNe8LlDE7TehkWC^hK=OFbqTPP28D6w6DDk9tJBb46Z?&S%J1lepA$h*sytwPoi z+vppl*<(9-hlDVjya81yQEWn)eYWow6pe7V{0prgu$BKHTa?WrnfWK}A=|YSlEm1C zd!ZU7#QLD@5t~#A#g4h#QbE2Gwof%=kF&XkA>AokaRibk*e3N*of0n%Q0|OPJpmmj zxr^79had*1|iFa)J5ai0U#lw*Co~`cyQWV&B4xuI` zw2q+S1DiVzC5zlWC!pXX+w?iKRAS3qLZ&A+Z5mRR*^V+$ixPWoq0%#3;10T|aM!qp zLN9EKMaW%cdn`fbS2puAq^hx{UZ6H54Ar3W8{0u0O4YfOd_v)OHs2TI`CxnhhAf|K z^KD4oU@LT>4ka49Z2wgGVmt4K(oOCLWKiUrEi8w;Ew+vVvi`8G4M3VU+v^}C{Ic0K zP?ZvMS}4)^m_+jIEZ3j0U#32S_o_cJ~N1DZx;Nibl3X1xikE z*Z&FyO>Dbw(9$GZ`yDcw**pzMImPzUgj$q1^8=MEY}sGvVwyXhF#As-E8B?#a?h~! z_CRJEo7fAfX4#s3P@58#l~CEv_N0PRbKF^mps<52Jq&r~*@ktH#mRQ4htvyf(s8Im ziT(+w;$r)pgwl)LEm)w)5?j#Mtm2G1M zih8+wTZPuw*c|JSZJjOegR~oL<6Dr#$9A?2)hIz3gtD7#p%4`FbN3sDd|Pa*2ar9$ zRy~At+ibRDNGh`3#-TbTj-Ele9k%#6bR6VPo`U?lY&&Vl5n}t!K>9s4_bns~vz70l z4@#IWK>2;P>mn47aHoBO0taldXUG|4>#0J9L$<9NB#*H*>QI9cU7t|li0$zUN*r@H z*@DCqwp1H(#o5$l~ zWV&QaY9M8bZD0gyQR3bxRJvm8j6oM^?z~1Qbj|i^g4`Ll*(u0;!?!-dA1cdWVvU1_CV?an{^H9P-12Ssywib_@Q)>yQ2UU zd1RCAK;9BtU>CAJv3>19nljsB1QMRvN>QjviRL3HQ(?P0fub+m4JDxUDqA!O*(oIckU+cA>Rkv`2%GCWK)$OU4t!LhNNF?e-)@s ziEFP=uE|z=gO0zsvwuMT7Ta9|a{RE3eM9;-TjB?j{jw=K&<7>#3UmLI@38$yp!gqm zo*pP5EdF;(sPsb4uKy&=C?JD`?M4a7yV-PuP=gYWhoFL#tyc>rdbr!xL86SUsfS#> zY)j*iQO@=>0V(>}rp!>264Dl^s9@8~K*@gY4rigDlC9eTEe){wosdbz_Thq*gKP`S zP>T{DR-lra&E$nHhPb;}he8^*!A;0L%(lM;nYCNMdIj zZbCImh_#^X9GmnPiaEI3`h$G)Y@c0@f3iE-TvAB4z*dw&QWx7~A5^Er%YG=g$fj07 z$4lHr)R2Fft!o%^xY;&FApHtkeH4;;*c=AvgA(qIP=1wdd=iR#xl2w#fi*Uz6>_e# z?b#s12HUS4lKa@a^H75ls|!$JlWlehO8B|EbwlD7n|>8?1=!+ika3%>&j%?)wjDpz zq=Z%gD(I7Ti73xr;u?|(v*v>zobdtM)FDP=(7XF637i^s$$a=}P)`2uBwpU^PpM)zm zdpA_2#9R-QNwbZ~q3AVtrwV92!=@O3Y&UGdK}efr`x$~Hw`?m~s747D9hA+nS;wH* z9d|e5kT1_RVuI}VY{zCuS74J(L(&Jf?HQ;}iC?o&uE@6JfQ}!zD>)&5iEU~Tay+qJ zEkpV;o5lmlp4p;a=z|ivH=ullZF3Wfzi{`l1qG^XP7!jxvONSLLyc`>50byJUF<^* zN>oLmLY-|t1|{CP`#Xlj54QDF$o0uqJA;f3wz&&P@x^wRf|`_IxQ2>Nw!{sT{N}Dd z2L)SfyLo8ohpkyLBc%LgdwGIdlsHp?N*%Ur6}tH2PWJ|dgr)y(2`BH6yX!xR zy$#4LVH2B>s++CZg4&d@{0o((Y)^kss)sv^#Q9HQ8CzNkd3xD~dm)RQ?XVA0_pwQp zP=^xzDyX7h`&2{ee(n~Ap@@>LI0AVG*i3rJs$#n|K$=0e!3juEvqdJMDkTaQC^N*i zVTGa^?%r(B`Y@Yg4zg+4^7D{(gl*gfNpx&yOHhpxlq*nnlr6Lh#q`|$u0g&rwpAZw zH?URxkZzpKwhc**Y_~g5of1bwP;P=P9)^xh+{q6h|0LVaA>=T#eIG&kDK>W;l3Ccw z3Fw0orq7}LG~4wh6t{AxO+$ehwpa#o+Sqz-A;T=&)*U3bvo-Fa1|_>U? z+)X}1;yhdG1#&sr)HTSsz;;lF6fU-|PpC-=zAvb_$X5S`l1tppx1r!N+kFRGaeGr-f8LHvK5n zri9ZmsJzM6XM|FI?t&&Lyv5csL!JQJ$~0uzW_zB2)FPYJ4s|Fo_D8 zA{5zWler;ph%MlOtb1%s)P?m@Og zHc151#@IHakmQK%JqFb%!Fd8@kJ$>RQ0#=ei6rEUvz=c+_ER?16{JhBg|8v$8Qb3t zRHwwX9F$A4)$-8sId}F4$bZ3h_Xs&I*~ZF{KE;-(K(Z?~#Vhnd3A=AlKF#*?4#lsz z^E9A9hON?soHuMUKae5IcJm9#Z`pK$>z^8wcr1YmIkw&&C~?Q#b}uC6*_wTj>z-{% z2^kA)Pbx_9z&14mH7OxI3>AxP8Xc5;}jMf(2?( z;)4|`RoG0k(8UXP7jsak$~NeP+^=l=F34PC6P6*>8{7H{)TV^DRj6ENn_Guc@7(2m zQ22w*umyQO+0M2hOM|UH2&uo=LLsO_iR~~{X|k;zKbrF(C z*@mB>8YRS@p==MEvWT%VAxkFEFxNfm69EvQb3mu)E5&!+x^ zj+NX+x)%S*Kfu-{g&Zoj4H={#WUI>|nVQYf4}DO={Q#67VjEXOaSeA#4HOt=Q;tAR zE!*BGWEf%l9fRaLHm?zCP-4{t6-L=+r=WzMyW43<9AnelAeVtHZikHHY<=^P!pOF> z05vJ0wFng_*xYU?Y2xnL0|h79rq`e)Gh1c@GEK2*{gBeab`*eGl-RQam8RJOyU>M| zyT%?AnqgavKyDk`V-zyavYC${m7Ohh0<|e&C;^q{*bb6V%E6uF5(>|=`K}<3lkNQ) zvMjL8XCbwVt&oE{lxVz%DvNCA4^VoEyMYoES!N5DA+MXQQ-S_Z(jB!Uvw%?;{_7)2 zk|aq;LXsp&k|ZNZk|aqIl92QzNfNTHy$(7SE2D3+oT7QAF-vqP?Hi> zn@};%w!Z}>kGcEZhQtJ$HwZaS*lHojkYrofhZLu5w+B#*62@cDLy9dKhf-(U^`Ah& zG~4be#n$@>nXcKy zCrEX})_R7%DPg4wmGf**H7Ik--OL*lF0f_aA@?2Ia0{9#vK_V|^*x*P2kKCw{}-w} zu)TM!{3-j$-I5fFl-M3*kmrfb*aw-*Z0G%u=9z6!1ql_lh#IO=qA(2Q{;_S0K+!69 zucMInh0Qh&S!!$r1EhUro0x(mb+*%Ks7C#Lb`ZLLV+&cJScAJCE984;TU&&zO}6S1 zr2AmAI3a0^?ZySwDRFcS%73yY)}eTtJGl??f3fZOA=@|GX8_Xwu(@_1S%3pFTV zHVhSh*)AhcLU8?mOVA!dfv*20#*QGngsmq5jd!zcB_X+#t(k(Fl;}8zial(P7f@2h z-Bb<|d)YGAkVDR z2`37uJj&KL0A=*t1qY$<7+ZS?a*wmEYN444wr3rrHn5q;pbjPGCZNhB+lUd$8o4_% zL6Ipo*(~HSu?6NJ^EBJXJfxXnTed;MEL+JARVmT54CTyh7b{S7j=Lc@k^wP3h5kd;TR-cX8Vmp zbxK@6f$~na+9?!Y;m(?d{4TcJ3}jnn8@q(`ZnoqVBwJ%s&7>=vMchwZBfCDyrf zKSBX7TjdF|Z?Mf(pm87DRTYwNvgux-CMCw-prW6xw+SV;xZ7?)Vt}pHh8){$&L7Ai zvOWDmiXFCT$?Bh4l#rD|4?#9fFO=Hl?ywIEhS<85kaLgCuY!zWwuTx~?z1foLv2dD zAAw2{HlrR&A8?l*heA=d!AZz<$hJQPnPP0h45T_@^O~V=N_e$EV#&JZ2hZ{`jjoS26ZU$dmXBz*w%bd_KdrKekhV=vus12bGGXp$edvt z4MCaYAnz4hISE;=*=Eil?G4-IIV8!m4QHVm zCB$;j^(~w928tE9+q#8(cWm!>khRF>cz|^GY!4+!`oK2z4Am*|{2!EmWK-9mc!|46 z9r8c1b-hEjGTX)nquN~{h-#W%Kj4U}wfccX>GcQ*YfT}U zCg|al%{2?9+T1n;09*JH@~GH4SCDy-Z9NZZ)NC&WNEl+X z-a}PN%s)ao4ck~5iVkykQh~f$HpL5M8DR^)LRuZ$*Bc}mWm|1RHA<+opldyw`3s7T zad-6t`Nr8sgtb3eC)nZ=NM~S^^+3`|w(VZ1PKh6VP~OPqR6_A7?n)}iZ(^Gsf^5@l z7sHT#hE1b`WV38hJ=CCt?g^-1X4{;E5_8-&rl5d@%{~L!=h^Pf(72UtavqW|u%#EE zCMBxuP|?P=?|_nv-2FNs(az>wg&a$4wKd4#U|aA)ieKZ2u0SHYLu*p^}H~Isv8Exzn9OAurqU8RXhv>&-wW zADftkRGVzA9P~{ID>qQt&-Qc+WwyARDMH}@TlN8RZ?g?QK{F!T;WMP(VUt#&4kh|) zP$kIrUWc-~+%3IBkr3O%2jtmfGqxdfnC<)<((JPh{z5{8Ez;%rQ^4+mJ!g zD0i=N$a~0U>xV2cw!#3UJz|?sLy|b#sRpW1f^r19K4uGzLa_vQKVy*Zgl)|LS(9v4 zBcwZJvrI$M6x+=#RHwvI3zR=&OIV?JnmhR-6IrLh2PoQ63E?TTkU~nKG>dnA$5z*tbjU{m{UTPPqvXkDBI@lXb6gY zvB|WM=bJ5{gUml{A9_gBVOySngkQGOBvhqDlL^WR?*DHI7c)?_>wk$ubC6fU7M+JI z-E0yYq?NL5+962~+nWQbQG$I1x|XpOSD{!hcat8-CuckJLe@Su)h47gqJ>C}H;$Drnfg&Y;9F zckT=n(6UvskbQ)0?g|>$v0dFj@=-Qj0cuiWya*NbY`u?Aa*Vs}CrBJ;Ydu4b2{vaH zG8ovNYLH@*ZTbyrQ9|||dN8tST2N|=yTdjVG_iI6K+b74|1V^mVQX}G|D>E{TarR; zO1zgrB{Q3`4@%E*m+pr`7PdhZ&!@ zfWmgR@hQl?#CAFj%{bWl&5(MTEo6Z@l=y9hDo(bwMJT(%-M=L$;$pKnAiB)0aPsL-l`EDqg;O?de5>MFla>$WnODG`2DO=wFq)4&t3_>kR zXb(XTXKXGlluC2=tb>B*Y_nsKGsBjffQ%PxS|g;)vK^VAHYN7VLZwT#z#Np$ao3!O zLRW0dHpq3&_GpJpH*BV5NR?;HtU%wCFyw~Hw`>O&rXSVedNK;{ZIfaCOY}PbX zrNn#&%2nCME}`fPcPCenx5lQ(LzY*zU;)zB*}jU9tBU}Dz@D<$T`UNy$%`GY;GT<9Af+DhuV}lw+)pvY}Y$bdYC(12nuQ0j>C{^gst}g zGU?dFLr68s);fZ|DPbi6mGx{-NhmYM-OL#j9%sv*L+%N-;Vd*`U^~n~>Pa@~4b-7T z|1DH8vc2Cy*(vUp9-xSc?V$vDrrC_oka>pf{2!#5WgDzPf|)H+hpLn)yhFJ;wv7)c zYT@qn6Y|cp*}fr*m95Z$vrUzLu6C$LWUi-gFQ$QWb2ARElTi3p@&_zdJIa1xLZ7if_rRtCy+DDW=KKC zeYR8@QbyPYE}%9g?p;Eq1Gdf;l#X)e$wQ$-wwD6rim}b#L#89P{3E1_v+2vwHzk}@ zpz<+W-wTvUa2I@q!Y6F)H^`l2TWvx!r)6)>Gqw@I|EFx4yCVq{ zIcJmgK%NX+pcgVThR=~?zo$rhkQl0vjxa{&!)0Nx(Bwf1ClArg?%=U)&u&L&0yh?keQ`Ve{7@V~4F#hm^l;OYcyd67N5t zlHmFOmSAi{>8}4JroW+(gl+H_a&@!qcWwR2BxMt%kgA8xD}%l%;Z+WmWo!%mP^Oo= z!T=POvyH1EcOToS2AWZ@^^ZX6ezwpk)S<-hF{q+sTQfk}0q*`8p@@pjG7WhK*{)|H zvzl$x0%?ZWj;)ZOVUsUHRZ0|>pxiLq=Q0%4a<}4wyd!L7H)PSV&8$P(QMStsNTO#O z_CqyFhy|eQF*fNA6dUJmYZvlOu)Xg=Rs)+O0_i5%9-@%c$ToEZ)hY4(7|KtvsgqFL z#9br>`KQ^s&LP_j+r|Z?pJl6GLNYU(?HX!O!d)IJ%&|?}K?w_Yse33e&!#Lvb}QRn z85&<;`>8;38=L0^YEokL6)G;W%{QQ=ox7VRBrdV(KOu*ME%5~zmf89`kiyBfBLx1` zqJ*{tdRSp|^*|{Xch9|0aFuOV0Xf}lIVEIVW77^oN)Ox75Y(o`9xYT_XA9_{w3oZ4 z9tv%+El)r$AKT+3WZGminIM&)Ei(gsQ^L?3RNi7cn1?a}?j$xSyv^pbLvE4n%>m8q zur01Y>L6Qj73xr;!2?xx+0MLBHpJb)CKTCY3vWT5Fk5FEGVimk2O&*_?Ii>W2W-}T zs7i_X11J||8;e2FL+(!EkT=GrIDsrjY{64V8)y4EgCxgns~M<936(5#onSLxL9r9= zu5KV-l5L~_Sx?#GMM#%olRZMxGq&v~s7{F=&rm+i=Bz^TbM8tt$e&@GeuHcmY!~m4 zKFg+QL9$D>Xd7x!LiZ0;$gyqyLWwKx8eQ9e3S6_}`|e@jrC66c&ysmgZkg3>SC>DHi7jqP|Ha=o(k`XE!C zP4q*mH?~#)`lf`H9jM%3d)kFE@7&FVp>UHe8-d&(Y{Q4pOpERC2vUEtNfS_q68%Z2 z(q?;4LD?_vmd>HbH`~Jn9aN=6;Q`8Zv2B!~ zsN{c%uVu*F&1U-tS)^=*7f9Q~Hc^KpGPctORHFpt2Xx)b7W#x@a_)Y{3gj@bsojualI_3)DU57g8&Hc9e4Eh26kB}@N}0G@6rtcW+g%WH&afHw zAmc1sY9CUX*#@Fen-ce8P-%{>6Nl0k?mQ<@XrArm6mnVF=F^aAfi0hbR5mvKCG<@R zCs$B;k*zNeW$fGq3s888tzCrN4z|@tXl9x1`3X`x*~}HFLy5U6R9Rsgd4;kr?vCD| z$SRwx33=RXffi(5WBX`B8V}p@4Bf+0Mox>kgZ064C|P!c&lRm+f~N zs#D^+8On#)Y8EKI$DMTn@`u@O7a`j|+n58=N7#~1NOr)cScMvtu)78oqHJI5P~wm~ zw+{-$*eZU=e#ADn4UNayu67{#F`F&~H7PM3hKdQc-UBFk!rk^EBqrHfN08%`&6$7< zDYmC1q&Q=nK7(46kUfVU(rlV6lsf0`Fb4%QY~44I^McKP3mLO)jXOwr$+q+WwJGtw z1eJ1Z#%Cyf#a;Rz6uM>`tU<0Dw*5L}%CiaYkm{Dr`vHAZ!s{ngF0d_pLzz473LPk1 zWE<}a{>gpMcG?ZiJh1i4AoU|#NDg%<@mm2^N^ENbQ1*$te}hn@%x2L*o@cgeEo82+ zjgCT^e{9ENkWgil8=xvBibg2+!uDx`qBZVTW+Cq@TX_z$)Y)dNkoJx3(gsNyY{N@X zjS^zZ(Dgf;)CI+w+-lD65V0#Kb2&qXNz#irhc;@{jw z_8|WcTUP|Kb=Wqdkp7pg9)o0p_y2bcwqvM433n$@p^I%I1tlc^OGu@mKsTH60zYO3Jyrd4$A1Hhmd#DA*De$k5N$_W~)DY&)+| zixS#z(8Bm8QhtRd1%^Zhf zOWa*0AfJP6uk|ys6h$cRjA-)+pIx}4elCsDBxqWzeDy-w)+og+|M@IhU8mp>2Ih> ziK<_y7+~A)3jHa$&E0P|B#LZa8RXbutH~ilkZqwKQtYzb4nQqR7*|6NA-1FjO6_si zKLQ2AY`de7bD!;d3^GR8+y+Q_!1m7wwJC9K8Y)HEu4ke2A$K|p6pFDOTOrpGTkj%d zinEDJkm{JNwG4ez!iozjC)l3cQ09cYnRO_fWXouagz3)NUbMBTRP$a|l5QRJ!Y{nzVoMk&dhBTLKgGorpu|-l)l@f(>D0juSaREiI zxqH2YyfVE!)H$Bq^|+-a|D?P?n(UJGM|6iWRy0sX)GawzU_?`oLCw zg>;W>mIfp(vE4MGIwg*NLis1Q#1|AVb0_aW{%5uwVee123frdy(*I*~^+2*JTe%l% zP{OPND!i~=DxpM;JMAD8cx8(XLH0UZj}{t#W82a}@&;Q|4>c*#F##3d*&ZjMWRtro z6C{4HWo95pi%mTT89vz#<{?F!t;+_rD8Xlk9=_P>4k-1_-Qo%q{9(IWg`6EWg9kGH zvZcI`QrP(amN2jhwJC9L3o3Q7b+)0j!)Jit<71W``+znJ2U>hkwSrvCj zMJO`JCVPZDYPP@=WFBJsc!o3@w&f}$46~JLP?Zu*Z%|IlcJU5HN4OhmL0%nOv<+EC z*(5)ZR?oKi3rWV<-nznns!@Vn3SE!06=hItg1gB+$Y)?X>xZn9Y$_F`GqQ!%kaUXe zR|C~4aeV~Jo7ifjP<)y@>p0|}VY@Xzwpq5ZDM)W-OHMYmLU5A+nf^`x3OKhAo(JjZVhTuVtgGc+Sz)2P;!a8Z9nw)ivIuKd@BGs zmf4&;kip6Jv0&=dg`IC^*!`4Va%5}D- zbEr*;_ZLvf%Vx|$=?(7E*HFmEHh2rUHre*?Ad{a>cz{$}Y~B*|O$o1Ms2pHh_y=XS zxhuRtVUcaT4!L*OP8-lnkgfj%Qtz^bKA{dJet$ug5ZhV@%I%hCKUh z*FBIq!Zs?0GzV*vc%bD#vtu6+vNl# zNw5tYp&BK`OwjcSn{*b6CAr(0gM6oK@AHr~#pbX*o5?#Z1pWjmSeMtP=gZgf>7a#ZDJ2fTyvM&hXOZj$|z*d zv+c#8@msc^I3zEyc}}1vC00+N;vL(38cG(qyU9S}J)8a#ay+mlt{}rBTVEbhl-PC( zP>T}UMd;y)&GiVS%G^CaLBVIX*$U*Wu;r?d@gJM^6;f8&j^3a)CH6F-(hFOl1*L1; zHQP|=m2LS4a@E-$e<9Nwn@JM+ld8d%kwV{;Fw_f`-`NiOpiGlHi4qEbu=!MwyT$gV zhGst57Kb5qo2@tkbtuuGhbmueXX8-zo4bKYDDuM=o`O6bw$3zU{$*P?LmI*N|1IIg z0tsFJORz3LRZ7e+LOBWBm;;J-b9dr|yizvBDrD(l3$8(08Qa%7Ba^+7dCsQ964 zIh%PKiuG}KwFCJSY$GAa+RqjbLpmj!>;RGuux%egbxQm=g7PXhX99{3a#u=1el^?l z8Dtw`yEupR8a7Q9k`1#(b5Mg4x^JL@mTmJEN{n#VxPt;ZHv0o)A7#5QLF0P1$!ADD z#+LpEH7QY5gNoy9`*kQe!QF2I5)EwL56CgeR{MkuMz)1-NHN8B+ksk?Fy3|Wrw0>T zvKvZGbJs6}f-`Kpa>zN$_N{=7W;XW#q?}{>Hwd*UaZUr3ENs_WC_T@eZWIbx*^b8` z*8*Fw0W#UxL?fhHWNVqAZ%SC1h01oer#UFI#NCV)3Om@cHpsoqHoOGQIN1)DA@vHI z)CF}Y(eH*TF1B|MlwIX+X#f`P;26;ExY{!tr&sI2rv|DTwDM%7vJ5571N>E-v*V}BNODHCC_j3jLcG%YP zkTu9wEkL?mHp@LE4YA!kLUl?UEkpS|wnPPrhq;r#K>mHUoma>fVf%c8^apINCM1ip zm0M7Q5@x@k!Xew`50r>;rxl`q3LLS;B#=GM*3$!xAG2-sLh=M#vkz)gqC*K4PuLz+ zP%_Eg)DR?|vSo%LM~Y3YgA8YE2YN`6X6u@OT9n|MgdWb>>QhiE!`P{D5K~+)cJ2UyJSR8?t_~seU0{n=RaR_$TQX z+iy2ir^Iy`l>cU{$)WfUch-K$-(kBQfNZ~PV`@k*Z2tdTmDE78uKy(}MxX{I?2bYO z3ES5gl<4NpZGZw&wu%w5_pr@PL*p{Gt650i%ciqHO-hVgp`x6vcM(eVaksq$i3+yX zGUVuIbGjgdlI_V2DF)c4*P#|AWH+D(6`RHnr3Se>3_w9OTlWs+9AfkDLPiZ+V-Hde zvn@rSHYMIip^}!(cm$LYg_Y<1a|Cu*o}6l@diE_NUxD+ouGITDe>4fxHWBSCKRL3K(zpMmnL zZ0b2E?&dBs5Bb;Fx@?fm!?t0E^y_SO2PE^d*;b$iCETq-g$=d|50vn6m-0e^O*Z8w zWcRb}Z9(H(Y(LwOJiz7&LQP7nhM?j$+x$M16uG-OfW#d(eGGC0*%EQcu*=qW0x3dl zJEu^K6540b!ycO}1Es>;J!hfdKHKaS6p#*2B{KknRn=$5{6n(`GoDD4P}ztNq(U4DVy&Xa;Mnd zx{m%dbH=tPh16-bq73R#qM;9}oU@(vL)i>>11czT!4_6So-A8O1DP+`)<+;sj_qX> z60X>+<4~0n^9Cq)%{Dd#MQ^w}nTEW1Hia3o+_D8NkhZ|~WrZYnY^#e^*rG>JgA%$EP@%@QnS>Ir+%-~Apw4DLhwN``_ZQH3gKaVg$=})1 z*HDuZRku*F$+mw7B|o_Py@$jWo3{ixKG|wz$k1k6_y;Mz*lu5-7A1_=p@(m_WCKe5 zaM%9<1v_lJpOEvH?fVNd3jY6Z3GNQ0?D}8AKOz37HYLt=LnR5@bq|#8=1wPvLQ=M4 z1?1{s>m7hhGB$A#QuVU6hM;dsSkXddIop#C%JgwJGX{kfY}pCO-On~`gl3d%hbBlp zz$TrAI+W<2gDNVv_jxEg$la0+im2Hh?2u=O&A1GiHEicAkY<=|&xHle7FyVotqJIZDgA&Z`^5QMa2Y!iEsWSs4EAF5G;G74Q!u!Uk!%)s4G9P&-F zt(`ztBU|+p(oL~h(vZ}|c9VhXlsI|`<)_&aS5SP0J9!@R&$8_lAe))(vk2+u*j$g0 z%)(ZFf*O=CTY(DmY?oCiVdYNy3I!I}VsDV$#@5q>#uwSPT9Dk%)@(yfN_6}{#U-}K zUnuF|Zc1|eCvllABZVAJHgzv#SYbQpgA^{dE+y2W1fL3eSY@lLp_H4u#bGG8#&$OX zIX!F!J!D*GON~QHFWbN*)TYF}DX6r;)|rOVKJGkbD74A;Vu4(Kw)q9fw8fTRgj4}G zy#xBDgcB!J-e&7tg)$;{!8Itn!`5Dh+(EWgA2hSe_Uwn$AvW_i)S<-O4piA=8wo+# zFn32`D6-EcJAgb9w!k4|K4AMef;3UKISn*kV7t;n@;f%&DAc6H_!v|yvh^CEN_dSy zu5bc{TWsSg$o<@SU3Q(lOX1Rwvziii! zkXhLJ|CTUXhBRIOOFXVXf`m=}0#zwde1&q|Y@csXRLb2-6Y}=3m0OTS#y0Z>X?xi& ze;|pRZCE(@Q;ia066m^*P1*y+6x?m~LcV^s_ddv~WOFDX-2mHz3X-bWriP$8C7usM z`9U_d4vMR}i|8T$5L?#-WYe&1OhWo$w)zw#)3VuSpavz}nW4f6+r&JS&~cYqfC8gz zN;_oNv+X&c@iDd^CnO(d^Q=NmO02Fy#R;}~FO)QJcjJS^NjCi!T}UVd!C+&2<2!X1IGkgo3kdvvJ62X3HfY;~bmz6jEB)j?SPqCH7>X(mY!r z3#F~xHFHpCfo=H)a@p7(Zz0nno2dw?>};6_=$jITo}ls)+rcxGad0Q8Lg8gLUk!3Q z+1~2V%nIA$JEV586+fU3B^ugLWtHvh8_K%58~BAHYi!}JUT0gEK^iaH ziyRU**sT3fl@jvgw(TXTPKh7OP(IA&bV2cb?n-XRA7Ptb zhinII7aNd1%BJx{vO~6L0BTS|_YPEuv2E@`i6ibBdr%wtufsn-b?5Q0bcOx(TIkxYK+L|M0-GqD{z-Mm){;Qql(5nRm5Xdoy-?)gFMAnzNSZ3VJ4*b1wV_ML6Q14)`}r(UQ=3Cd0A`hzXB1;tw2{cJJUX8z>>=PFsKiJ#4WeWS6n^JVN8WY+Fx|T+Y^fhMJV-uY;P&3MhW(F=-Rii!FQy zNmto^@1Z&+u9u*^o2^!c;%nSl|3Q8a+wBWvTW1@qLwYY;vH{69*c2a7gA#T>p@NU? z>kCS3a_8I6-Bn*0Vuh{-S!|P2H9Fe zkYkt4sf7$7wkI8=*khX>gIbi3oq!&~Y#Jk!+UM@j1O+2(-LsJMfXzP#8KZ2Cc}RK4 zwq%3alz4B4N-;L$GL$~zF1-SU;%tL%$aT!N?}1DSHemx&ov?W~p>Ilf-Ga(VwgnN& zoN`wPLg5tK_#WgwV>{i4X3}i^QAmBx7K%X~O8kyPl?>b336#Cy?%ydC$+B6}kmr)^ zIs=(=Y@?Tu=8En33KFi_ppD_r&(_3rWjtQXdjch4RmA z>Ru>b;V#k#`TwzXDIr^xZ9@g=U)btuNLFLB4MPn|xEp~AuWS=~C{gDwH4X*d*p!oy zy}`CO1&zP6{Y*phCY#3$H7T)bfr=k&^9xY2#of&!B!05#9gw5VmT*FbFSfo_Nb$|K zvj(*&p}h`0{II!vP^!b-vmXlnvdwNoPGS51TS9IJGIsqhQ5%Ak61Jl-)TYFq1E|!^ z7C3~`Qtp~ZP^gD(IRUw3Y>!FE)XQc%gH&?1%sKQ;2}4<^+{bp1gE9*4BsWmFpUrm* zxs_~hchJlL+u{SHRO zUOpjVgw6U5RVgvwfpR*wv99w!MMt?i>4v;|HiZnbjIjmfkanEyO94qH*j5Li8YNT) zp=$%1Sp&r;xx3OrJ|o-6C}f>ti;qD%6PwHcNvGMijZmEuKTJ@5hRrz(#b>!I%|U)M z+q4z3&9PnBAiaf6vjoZJ*`mu(gA%%3P{GQ!>4p*u+%-HV=+wBR|qJ;4j^x$So zrlHgtcl{So(8IQS2|3rDAC^wRl;oVeNc9vyCo$QiLgDWAkP7taR@R;+0KU{%^};M4iaK)5j|9;L}3ET z9kFdpLeV&PuTzlsn9Vi=SrTjoGo(FXo0x|rNw(7ks747&J9K@@7IHwb6n8&P$aluJ zwhCF(Y}GYLcg|+S8GY2&((QyM6@7W%2 zq2vR1Q$FaH4_AA8dUiP^QIQa1;uEvbDz`cbje10L^@{JsTnQH=B7H>QG{C7OMQPjaZ;; zhr1&y6!~S7EkYha{Qs5^Sc1%5|4aB-hBOkkWfvrLvz6RXl@d+sP)^Eru>nPUxEu0A zUKv|709kt3Bs-8+&bGM=N&48{_MjRi*dx%jf~^>ZV*T7r9zi}O+u1Q>9bi)>A)SgX zoPwl-Y`@!P|?8F+kujk+-(clKZ!=R zmIQK4u{nDngNf~_7g9{KO)H=lC1jP*!wj2d5K7H*cQ^zE&1~IT$T`R6*Fi=LTSE^i z=h>Dfpf)AmPeLUto6!WN7r0B$Kp`93;2h*yWZR#IOm;TG2C0_VymsiD5?&op*}=B3 z0%exDE386cC)>COa<8zRdZ8H?TmL4cUS$hyK^;o`-i9h}wzVLXUE}Uw2#R>vEc=jW zo$dMnGJDxZV~}Qp?KlnzJ~sIYRHa1m6v}O~eV##4KX)q`$h*Z>&O(*|+sqZD-DbPI zfg~c^Z~>}OLaYc~@32W9p;(Z+ttZI0%l7^ZSwn1&Dx}+Ed#FLuFx%7{RHwxAcPPKl zrfxy;2zQY-i9RS1=PuO`1&-O2 zD#)H-+fzg1Cu~0&NS7yu@c{}Ru-%m) zXOzwG3>gpEQvV=jjBTI>wJCA04wa7BIt?ft=g#v1g^t-?J|S0vZT=fFov`IQkSfWh z@4EieHzk~OL*-MpJ{goraTkRtKP&bGGL}NS$FbYoHD#=Cn}df^B3J z%4WGc8iOL2Y%&An$*~2Dkok)3!vtxr*_LM^;fAd=2URK2WQB5hwhJ2+z2$Cb3Gx=$ zqRWuwj!oi%v_-Z}Hzc`dd-FgwO0aK0*AHyPO(^!r-DCjrmDtWi$oj;l+J$sww(uS# zeP;XJhw7BL9)5Va>1%AsG$eavQ(Qm|O4z-G3U#)x zD=6{CojVT&8f=vUWPfLyyNAY`Y*&wv{DVzbhMJTZuRz5XTki{${N!%?6%yNQtvAT= z#pY~6hHti~7Nq!LoBo1Yl#u;_9y)9q;pR`NU+xYiP*4c|e@p1@ft+3cOYrwXMhRP^ z4^nosEh(WkCElx`l9bIj1f_epOAkXK8QY)^a`m$9>miezO_+dGeQe%I=$jH=r=YTe zZD9t=^mA7*Lt!P`_&nquU^`uaW>jqbc1S(Q7IHuxO8j<06*b%1DwG}K?%x^|(Xd&( zkY||f+6S4nY@=I{W`yl{8xnME@*q^DL@@;AM%g~YP*l&|$^qmZV=Es*mT|V3IHa9m zyG%e51KaQ^RHKC08FW3#Ce1)GBX?U_$T!9Io`b9=HpdO5n`V2sg`_iVQ$?swiRTYc zewI!B1jWtVMV=x599vfvvRT+RYLI@OtzL&@RyNx^)S!gB52&!fHqnL>Httg2P+*Zw z`3u?YYn*x*4SJIDCObq*$4&K*=DC9r_}!ZA$D}gi4!i zfh8#I=dQU7g|^t1U63ol_UML8+ia$FNF}mmHlS}x81h5q9kzo2lnHVt*@42lY`$H{ z9b$XigJ$;F79)^4%vOv-9ZEDDL6v>BvtuY5;cg%aMGn})DaaFL>!cy`A=~-|q=~V; zTtdPToAny1Qer+2<>G8(cTn`0yOVp!n_yFvAj=6`uncLFY+n^fa>};)0@Wy?@(Nw2 z*vt(mcE;UR6Y{0mMm{0yIa~Y-(q-6W9Y}h?wk_QLsZNO>5-6W#bM`>-OYTa&kUz&Z zt$=J-Y!^yMf6b;Dgk(2t(IKco3Ef(#kZ0S}L5W-L8hR*DV6#s^_B*!wNoc&tHfe(7 z_iX7Is7Z;cIjH!+wm%OgAG!NofW#7;*A6+J*lG^QP-a_LffUbdx2sT#62?8yLxnBr zg;M{x>)(WeRkqzN$oazdy$u;_Z0;bWd}aF=g4&cgw-1%-Y}W@+`i(nX3<@>aj^mK) zovrr-GBw%6Q%Lo});fc}DPbi8m0N62St#?#-OLpfZnI@?AomyBZ~>b6W;-lG>K``g zBh;Zp{}WW{u)RM+*O|4TH!L1qct`8%ZPW*cllf|M=NhN_e( z{6M)LwvAsXD&y|8tMDgpFPlvYS>$X58KmuFo9Kfi3bxaJs747&6?EOt7E(hoC3im> z$Tz^YHUe2yY}HXnH^^oghoow@8v|6Q#L+1zKg5=phT-&H*PVR!APwJWERqjY@2hCWQXl-9;#7--3DC;*@|{3w#(h*GUN-fovlFDJvNma(uLW= z9!R>+_Unb}l(@bLq$%_bpTob5UV$`(LP-V>3t!e@fnRm+XOpdA8v`$a%-M-wzoJY{DR+x#e$ zdgLxY28D`jIxXaSVms48rV`u0B&2+13r;~DN^DO<_|;e%K!3P)~<#>Kv+3;>87&`(;z6 zp{Nl2@0M_ofqY&6N$9$P79?yNw~)4*t#$`Vd)O8ppgJYo6`_1D+rMWhCgm>i0{Q#c z2CI-w#bALh&O00ZAg#os?HWVM^?xq8YLu}fv`#;$gY_V=gKg`xI zh2%=M9U0W5gq9q77-4e`K?xOi&%;n)lx;=@In-_>awKg%oEcH)NV-Tik$@Cbq&R)S*PZA1cqVor_S~%-zr~bTG>n+JoE{w$46eo?}}NLn)2DK=0)&muzY*{aqJmF5W1qEYlC)<$o zl&voS8RKkX5K^46HA7IF5>^hOQiAR22uhuEXNp3hBwIQLxh~j7&mdEZ?dTj*Ub6M1 zpbjPa(oi|g_K|_oSKKY#KnEGNhg-;f&1NV-<}BOA1EjiP8-9X>9NWP&RG~zn0%dO5 zHmXoK&)r)M^4zg4d_a~0TmBPL-?RN|L6Qfyvo=(vgu!1Z`^XmTdi+zQ$lXsjn}p7LGA)2C^W>@Dnl*>+v+Q18fJTbgOo}(a~JV3DqfKcMHld zuzhbsu|@7&0myG-D+eLl65H%Pq_eYKA40NaHcbR-P+~L+6&!4Rr%-%_yX`Ybbh0(i zA^R$uBL(SQY)@%OzQ#6v4K*nteFHtX*;IKbvCiF50Sb87x*s9O2Al5*GI-hQ&(Po| z+foH;QR00SD*D(A?@)4!yW|HH^s@~&A?G&RehV^+Y{Czu*kSYhLTyTT>w5ZADZn=0 z1EqGk%S)k9kWJSQx%Sx31|U<2Z9oAj_t}CQ09c~%L;{K+^sA^o>R8c5@d<9nH-S%jP1$^ zNfKrrr7K|kmi!@VHfI2vrUDd8YNyFK)EY6 zBVju?^lr zwgTJU9i+Qw`?-f?4{Yut)S$#l2`W6Y&AmYJB6l~hkod%=twHt@TdWT0pV|69A$gf? z=L>35LhBoPcwuvPphShcXQA|`z$@EKH{__YWqKjQ8=G1N4c6F>HNZDa4*q{z2>X)JNFWdPFlomq&-4cde&_UOK5<+gsEn(|;Aagg{`X;36 zVSC+zgkCnQ2vsOC7l1NSwuwC`+{fMNKIDMWykad_XdJSomY|;gdCKQ|CuHJ(DIyT!6Wc$Z<{|o8#Y?G4b zKglN9l08s^5*2+=!N9iP55=dr`yGJ({;J@=XGM!wQ&9?9WSwyyc5mN84{dG;RJr4O#*&-8=EzZ`fhjeFbTLws$U~3qm1|`~O zpu#!Z<17?UayMm##0$360%T9IDVHGqCEMXLBu}$-IiV&ccwNxL6B{JMCZa{%+ zw!2Npk!92SA;S$@LWBl$Y(u+HixT(tpyDlCXCF%DxpRl1;2qm*1acPG=1w5vJzMS+ zQarF}6HuEHPLojSk*)s{N)@>aTtT5Hw$?S|DzUBRAk#D3a~@Kb+06G)hY~Z7Q2B*z ztOTVi+#Q#pgI6}`E99=S`QIS(8{6kQq^hwkHz47it=NPrlxX~hGIh4gA1M66-H7n= zCr^VdEP*VaY?59`-DKPBgCt*U@BL7f5^RG|w#8OZK#_0mCPyG|o9%oQvi`6sG?1pl z7ScjJzihucs78tFlTc3B|L^;uIt4|${*!1mK|TrFtr=SAW}BFUv^{L`c}UvJCbvO# zO4zkSc`4hs1B&%==URpQGPd#>Wb0>}^*}l~+qD;x4X|mppavyIx1quyTVDW*4{^60 zghT~fGX&X(*&K(EUdi@!1j$F(rlU}k64Ei~LB*yzgA$|M9i2k~HCuNIa*VP0(vU&J zR?k3#<7`VeP>T}pZ=s@=%}{`n6Wk>qprDRz_z80UW7~g*jCwYq0x2fhJXNSo32!y1 zWMG^BfKpT3jxqE(t0!M5!b;uE6%QPUvF`N1e8jP|Xe?u)w?Cn6s z6E?r_`ln=!yM_b`p0X|XLe4nbV;^KZV>8MjMS?9g2(>9;WEd))vmK5=sU&w2H59sF z^J*YhitSwsnJ(ED|3S($TVWFFP@>)lm9N;&O;9?+-Owy_aLpE)gWOrR&OBtkVOzIB zsvO&^9TINYtSeB35_79iCeJqEhQfE;oq8Zofla;%S?<{aTafyJ?Ry)NJhH6@peiMl zgHX1}X5NP)PuyJ}Lf#VFSOl^@vqhtjrpzWig?e7tw$Gp%C4QbmxeA*j1w~)EE2be| zm2LVOT6kl-yn(beHdP*yzO#i3P@NLGAEA7mZSx6=eQ;NQhWrgSTLrRxvfWo9U6XC{ z9g=;qB|o4BB`TUwp~be}g5uxY{k9>o&F1-q>_2SPuIiuk9k%%%NdC)qD}|brpzDVo zgoFQX3Go3a(ektr1sQtSTw~B+FWbvF)S|>$9aNOEW%W?9k2}p2 z6qK=@Ohe9owmvgtl(UHzNHM_Hv_fr4SXqQhgKSSrP-=)flLHDV*wRkOHOw};2APy> zN9&MsgssO5btuv2gUTwl4?mP1_YA_HbV$9YuGLhAk{eA@DU_v*$$4O z3MC3LC^Nyf5r@J$?%ooR=O5d`1!U2)6anHCR<_!8uYOZxu6y$?zy4j7F)*yCH>sFH=*D*+v^tO6xrrP$hgCn z3qXnhn|2RsQ^M&!RN7_h4@0RScYz2L+GA@)Ay>p*dz@|onYH+LXvZ~_ZC#81ltdkO|lh!p~wYylakt>yeYQx9>{vhrs#t- zX|_;5)N{r5I{?)vaa{rBGHg{P6usuoItux+Y`0_3!VTMm7SiU};yOrr%O;XfiM z1?BT>-_ua+jysnb@)y|37RYwbHaidL9@wrIA=x9F#ttuiQ|DEYx%@&XDr*oM=P^OJ2q0~woa!VRSOV)NWWZAy5%gGw#7 z`3ETV&0W3-h1zVoXUO%#cJ>09I&1?~NcqbatU(=0Y}cW(aQNRXVeJ!2cl{^vC8)NHoLJJzU4Hu*xXREm(sg`YF1FBQP-6oWuVEgBXVmj^; zBIN(aHn3kZzLgXCIOo*xX^LL5Ye`71X4J);07n%jV2M2@7}6c_=W)HggX-tZbP_$S}{QEt?}tolY>R`C(#=*-Kpjfdk3i*hw)0Ua?cr`n108Izg|v{{%hu6B=1sQs zNl4{md!2%WEjFtOs!(Fi3}yUm6LU~_o4eC_$Ro1JZIES$EntV#0k&@kB-v$KU4^QY zP+o(wK{m4oitKTB?S;G{wy`b9y3ZEfhBOCk(g4(R$hIAXYLxgHf^uOt#~~Cw;;wiE z`66u7QE1_q?J@>wqim`(NP5B+K8Na*(4B(vF}BS#6g%avo`L*vHrowkJ7c@Qg>(tF z$pR!hXG=an4N6ozL4_pS{xcN6;O_SY5>srRDrCQ8tJWZWnr;3Al3%gienL%3(6yk4 z3|qVnC9b&}_=N&lw%x9eKRIsL+TD;L$L5kkgSTuiGN?s~vjb2u&z2p6l6Tx`lu)q1 zcA|ou_iTM*knw>{9ETK-Y|RO%O$jS{s8nQoGC-*(?o88AsKl0@fn3jQqZY_iW;?P% z$``htMW{oGz9pz!Vf$Ey(y!btIiZ6p+k*>ozp)wCA#;uGVgpjWvkm(oq0V;ThbojP z>_C|hwvAmV+~DqQ5AuAnEgV3WCR;uXslV9%9Yc~9+t~?JrG&val>KH4CZI^0yPqWF z{b5_XgsdI5$`z#fWwT_V9wGeSbHPmxs!`(j9hB>0i`_#}$$t`MMab99wo`%@df2|o zkhYi2`3gy;Y^67-P6;!0DBs6+)qr9$?$lq9zn?Ag4cX*uy&Xt5z_ukc{v;b@Ye=95 zCE9zT!VueI9~4(`HzkL}VYbvDWLL5&havq4+u;Z#SFv@ep(Z7GHPFK-TTKfk)Z8up zg92l0caxAq!=^VvhHbCp!oa3=Lv2bp^+2U5w*E~hW#lff1%;;BTHBDz#I_oMOfziHK}cz4Gw(wkO3WNW zFnN zakGV5P|rHsZyTyn;`%R?^RQLBKK~Tm;Lh3u`MhkmQfOh5ZK5C2`q<(FkaUYpu7K*4 zu&adfeztEF6x-&`H3s=bw(>Y++hLp4LAn6jwH}h~vT3HE1|>$Pp+bxl62&hlbI11e4TTHbt#lyIJzGg={>k#dX6lC2 zk8D@Hkfg{qDub$&5RpUKC$^p;C{p5XYZ&rAvwe&})-s!24QXE39yCx-g>7mAs!`&_ zKPdOgrZhm&Dt8A)$oIz9H3Kcw*fwS%?K@j-4wBZ{78am7CEVGd{0G~=WhmC*F0lgn zKiLLdkgdtK=Z17&Y(E}I)?#ySLJdl+Y(a%@wmA`sx4F9sK;jRZb`P?5*kb#T{+F#k z49SI~|K2g|M4%=mw4%^M7n}1GN=W{b@O%aZy4hxukfVn!lY$JrZ0aj$P|9|E4Yeq- zHwP8_*!+1YDdVnDfP(#O%a4##&i42O83)*mWk@l|ma0H)N*H;AN<(ai?@&s?oumPU zhS|JL$faa^Z$YLJw#6SvsbVYqLLExfOTPY89%Vc4fzoR3hWem`F}6@YYvKI<^Td6#mEEsSfh!+2oUuWs)s01*r{e-_wv}ifz>l zRVkrtfwD$6^E?!p=I(kC@|xJj?2vVaE$V zh8{d@@fRqu!QDU=3V7LeYmj4;tzCxF zCLrY@TaO;NOmBVZwMksy6-O>zn5Mg_mh1|z%1}kKavRy1dsuQ;1B}j;|9V|l? zN)((>=9F#21%>0>y}2RJ8Qa1JWJ$2)HzD;o+dn@fNwS@ZP?Zt}ccJVBTW}AGq`3Rp zhrE|;YhlQmW~)RX%@v#F1nSAK-JC)-N*qr>xofsq5{hQIlU+i-8@8P*Xd%b;bq#56 z*_=5@nrAEJp*khZ+(Y?0wyQ@dR^U!ug8cVvkuqd^VC#K_bdPLXZ;-6W)_8{+lxS~2 zg(tShCKNAmH}wsP&upn5$X;es3g3Uyzpx!jAbEwYs~2igg0~NPcx9{gLy0PPi-S<$ zjqOeWIcjYB5yuf_Bs6~l;TB!KJ*3m)926ygBDEP_tIt4kKY;z{a_{Ekp zLy8ugb`EM&!s$Fz`ey65L8&%(0Xr1>VQV=cSBGtN6*B#@J+DDZA@biX!R&!Ll$h~C zNe$hMh=BtvZP8K_DLwi_s`U@P20kzwv83y@dIcK!faN7xik zkVeH8dWL#N*?wQ38YQk*p`4nnT7#lv+*v;$pN8%B6IvK&n`l8=EnB<|NhjFkzfheL zcDvet%Iny^yP?=W?p#vHuV*XEAloF{>;R-Quw4&9vMDx=5^7LlR0S1`Y<*);e44xM zaY!_=H76kZ44Xp_>CJ3U21q{3Ha!hBDIq-rJy_UO7AP^t-H{avSlPN4A;&zMZwWFi zu+^8L!9}(uC)A?Edlyu+u^HB(oYWww1kWOT3zJCI_9&9e)&DdBAo zDmmHa51`a4clj_BaIt(=$)Or0 zUJOFHBR1tQ6pe6qFar6G*}BxwLX>So18Gm#YFbDdV_WzK)hXd_63U;l{WC(bIClvX zch2@R56O~jZX48~#EKm%T(HfpK=BlJH>;3%$)R56u4oV*@qlCw#*@9xMfpEpus%baTID%V(%$b zykqm9LCFGljdLh?&$gU`oDXb|X~_7jN8#2GMt^Y!* zI@@bk=TE{1o3#h3P-0FBWg2V~{ZROmyVC*4(`1t?Aj=n9KnbZ^Y~LzK^3Aq7230Ac zJPu{sY-Sx4`Qh$b4|zLmV^fgzmn}LCX@ukdZV6H|)YJ8!gl!8{qr^`ul#{SI7NKZ2 zcf}>h*TXjLfEIe$E}f88%BEU_qH zNH@qf8G>X(Y{>(tL5YeZsGwlmKZfGN-2I+FqLR%MhwLM4)dZwhvCUsV@=>UWPD;=n0WP1{R|CE~M&eRQsOl;|1$Th<@DuYaB zwj(*DoMr17f;yDw8-~gjwvQ1gJ;&XW8alADJ!l~JJey$xGB2=Q{DV}BY{Ldfu(2H& zp$a7mGf-xUZDSS++qrw2gFMS@3k#6N!Irl{>J_$s%aFv$cD4dlDPhnBWmnmPZYbj7 z?#Bap*VxuJA*-9MvIS|@*(@T|<6*lAKs8Dn--B`+Y_WYP>g7%rhJ2fBI}vEX$MzM4 zv|DV>Q%LG(E1f}gN|;GP`E9nV6ciJ=Q(r;;9k$3dWDBtM<{;fJ+g2Wu1=$(}s6mPL zN2svJ_V@(FL)=Z3A#tBARe|gWY|1xCf5>+D4#~r8T@9#73En33aKu(?L5T==i$74{ znC<(E6uiQhD=v%&lX6TVKdJ|9ZJkB zLgj0=F*}sba(C>24sO__tB^a#=3j%%w``y5kSfo%?1h9owxSQJP@-`g$`sfxccAb+ zcOyZ_^S~AkL6%20$swdJvTYtgk|(zJW2j0AwiuKxu@&M_q-(KVcOcm}o2IMlPYp_pc0+|WTb~q)|8Tc0 zgTxM7Qx4gG*&IWVUWop8OL!WFM97=R^cQgS7df2-4kfWE) zXMhY+wz?4->|{$Jw%{kXg$%o`6&nY$r)b(6Pxbp$a96S5W33+t)P|)^oR#gFKULr95OY zu$k^5^%UFHBP21hjh3J)B}B?lcABl{6^fX++j@h%Gi)F4kk!m)Z$O$^wudIvV_}>6 zhH8{}@dM@N*p!0gPf;s(2NK9P&(_roEiABY^g-H1wpu?VwXrP>LUl^GQ$YD8wtpi~ z%+6h66!I^#4Qe2pgKbX>=~mc&bdb!+=AMKalvtU93ae~$CMfRW?#2v>Yi!y%$nIu~ z%|rTiwtgEV_pt5Qp(Z7?9MHoCn{yRPc)5FCg94juGaks{W6O9U!xo!*3mWvZ9dAP| zO6(0l#cei!5K4;NH9}Buhi&-~at7EQk09eNn=uM0f^4Z6)TV@yGpMx3c6bh@LflDG zP-vgcn}%EmZ0{M!bjY@N11ZC7gakhyT6h7nbv<-O@Z1P{oa?Td$>i&~D$@bk1NiNt{ zrBIa;$}%XMVlxjwkxTBbhahj7ZA=MSuh^n0NRwfcjzK-wY}?~djS@d6pj?*Cp@*V3 z+!YOwFUK}L4K3WVUCuz-Je$e_N$=RgR;W%1-HTAZz_z&r#qPPQFGKzZHk%W&J+j@q zAYGAdavhRAu_ZU41|=$dP@%-O?}y^g-2IA>SZ4F=LiQK7>K>%8u+1Mp@>jOoFw~?3 z-7)k~Ws9Fci8t;B;!vQ*wwr()?`-WPWT>;bE}_8>wwEiYMTxUnsMui3=Ah&!cbYpW z*kn7ohn!z*eMQLFViQY{;+w5mhT4>{@(PvOY)@}c>W4d19SU{W(hbP<%QpH2nS_)7 zZV5-ZJ@i5DUN(aqGE3Pm1|d}++wd?X z$k+}>pb8}lYADmswxNN-a_-)=kY|8x;U8ofWXn%N>LIp&Mo6MyJ2OF5N*J7lvcqh_ zIVhs!?q?qIjNq@7@M1|X@9trUdnlrXao<^Qo=9YQfZcj^e_pJaw1Kml;C}W9_H9;&rrh3 z-C_j_%(LB9A;$un{v9$bvL!yCK^xmp6KYZ7UJEKNv31%|($1ax7YZ)3y>|8f$?0I5 z>w%0bY&j{UaI$Ipp*AI)4nU<LZLObmI`vY*;dCO(>mMpIHdHjnRQTy z5;JA$5>#(+5fR*xvn6l@e?_P&UL?*o7kd+)aic?*ZHS0c1U7 zQyf8>Fk9#t>N#TjJ%MVJxE_ac5w>aqiXL-ky?}gCw%bc+;e>4>18HMy@hl`gWs~1R zbxPR1gYt2w&pivzhZNAAbp1INs#_Ye$6)B4K*nt-3vWr*;Fzpal_q_917&vx`!agEt_u` zGUVCnBhcU-+mae;QR2M@Di+ub6HxM=yW~G8_`o)7fSiwP`$ouEWD{l}#S@!n7HU(% z+Z7>VxfM9}*gDvM^MkL@@$oKH0vaP`Jt6$|>aeVk@0NmKK{S z38}x?u2PVs%{F=kRVg8I4P}4WdU8;t!`)UM^8T`Y6d;{MXhPaPwpt64%GefupgJYo{X+SEwtteo zKgHzSC3+zL0NY?6WE*7L>xXnhY(E2#Ou^<>Kn+T)D51hI+uSG=S8{hV28knVS}kN( zvBh+dew3|$5|XRgcBY^vCA6lYhcPy%8A@old$vGP2SX<;kGp$;YL&!O@h+xZ2QwsJR=h7RW0LK(=tz}Cq^ z=0&#kTS#SNd%c5%B{u5=RH4LN5z5%vCZ3`2GIyshkjKF$uR@j;wm=P1JK4VLkYtr@ z^%JU6Lir2Iy4cKZD6+=gbqDgg*~YqLf3mK#MY|!5hfOMldN$a$Wl)V0Kjl!)%jOt@ zqMO_mhasPjZCV8_Y_VOcA+4WHH4aI)*}@Z0of5kBP+nx)G(fQ(?&?O!A7Hc1K(<}B z`&mdAWSg`?vOTus0@R>H#S&BqvF$HI@qO-oS0M3#&EtaXhip|hqz|*rZ$R=Rw%bjp zNeMbX^blc-i%{a2yMbLO5M|rlgB&Mp?S04)V{?U}!Be)E2-Kp)*%PQ3XUm>K$usUW z2`HFgJ4r&$bGE)q$e3gkuOP(*Tk{%fQ^HCPDy7(-@=)rMJJUTBO0%UOA=ee#XbCc9 z*pAAO@|vyZ73xr;?+q$v**@N(^bL1Q4d@`p_Rxgfw`_)Q$ed@p_<>Y+Y{NqTpM(P2 zfdr~hqR0UDIC4Lw0EO5A&fiv4Vz7bq#`&RvCq18lD~$T`S1 z_W>D)*m9qcLcykOL2XJnZ9}DDw*FrzrQ|NqCI3@sgss&Lxm0YcQphyQ_AG;xYBuu# z)S<-85L6yx8&g7Q4R^;X=wO^pItID5Z2ob`Ji+!k0jYFs%X&!o$5u2z6-qQtLm55W z!( zk3)4z*iArrJKJ{>iY;^Jx`g}=w(=EZTVb2cLOLhgbq%Cp@NI8uL#B0 zxZ5s4qMNN*hV1KXj#o(UVS9Rmow+-6(qg<6z&-v<>%HiH~W?r@hJgn|LK;bF+R%eFrP8G~$s8dB`Bc{EU) z65g~>Da1Da4@&KGm!E_}2W&bc>+rTWOJYoyZK^;nL&qL)1+nNnZA9MF& zhYq4_mKDf-!j@fy%rUlcH>5gcJMln5oK3a~RVY#1f-+}pU)xYP!QDy#@|?4kf{-Q2 zX4;3;7i?FDkR-)68iA^m5Q#$BOSYakRT=m}y z0Hl9q>sLVXD%*|{YEnW=1wFj6Ime(xjl1V@DDciUqk|lEwu~M!e6Xpfpuq;)@if$; z#9lL0{ABZ6pk$M~h7}5au`Mq`&KBF_5@h^lGddten=R#p+LSP|29KS9({(@?h`1uXxG;EFz z6dmWTC=C6{r)8V&h88B+E_)%Zj!h+lr2p8$a;Q!T-9u1b&$c-X#U{C{k3fC{n@tVb zrr7Q^kj}_9IRVM0*^>XD1|=#CP{G8uZ-nAA-2Iv$(ah$Vh3vCz)j3FSVVhro*xS+rS+pZgOEV8vdkio|0+Jpv|*j~1v7A4M#P|?no4M53d z?lgN)(7|@H4>?!Z`ofUW$tFf1#VT7f3biR=j_D`btlO)7;)&o^3VXzO%?z08^ zp~wMuKLe2WkZnx?S;K4RY-TiwzUSyQf!TN zs6mN#FI2c>d-Or^Gkw*Eg7*k|xM8auLx~)B zi!ms0%XSxs9Cl>tvzi19$FQDEP?sdIvd+Y;zBg z@rf;0gcKz@g;Hhi0yQY~!q%!ot_s`gCuDkMd;WryRW@@Q>QG{) z1C`&{#=3_8l&*1i+zlPPvq`0pyUym9LFNy(PdTJ&uq_Wk!Y5mC7^+aBQ3Yk1Y?o>% z{KehKIOJ)ug(o1(H=9HcsoQLu21xS5_HKl#lwg~IvK_XQg|q)|3Fixt zwd+3#iX})RVGAuoJ>6`-D^QIR*IiJqhpp;{qP^T%Hz1#s?RFDd=wqAkLs}VIT!f_k zZ1P>GP6@kvP+rdVy${6(xO0Ue{~%jA0@;SxW=|lUg6;Ygk`1$I5>SH@qe-ZsWb3xW7vw)sIQHN#zA0fo$Lx)I1V%XT&jnJjDr8b~?E7Suu=N^I+(vXyOZ5=zf= z_c8??EU;Njkb99WYlh4=w(&VgwZwKZ4+(ZQnGLE?qG*RQ%WPi`DD2>FWfk(Qu$9&z zi<8adfz+#PS6)csVjJCps+17fhO%pHJpm}<=58wpdDq!KLXg$NW+{w zDhkyo@gfH0HrbSCP}IlW!8zpHV(Usl3x2kZG^E{Tt7Ram$hL3;)hXfb7Rv9i{VPDR z0C$N8$iK@r_ypO4Y+*PPSiIo~u*k_yjfZ_+--F!mgA)B@Z*~4tH zHl#md>;Hx15w@MKkv}ylq16pN9J4v4P$J6RvkVHHu+0oWju=~J2r`_qsg=-Rob6Zz zwJ5Q73@V=NnKu$K8+*I=E#E`5||nts_F_ zJGS**NL65a-GhXCHtPXYp~PGm$~>@597Ewp?oLl2PmxU?hb&KQfdr&3v3(~Y$uryP zB~+z^@)eXVvzfC{&-6cL&Nh*)|2$pJHF!)g_R>#b)b;Y~O76 zeUPrrHYtZ>KWxcCs6mN}VW`ky+aH1Azuf(fLZXoP@0Q@vK=!WxBv!SMUcxs250ZDY z-A+PHO3)dhhaR@L2}<;GH!uqYq-?u$kfV>SJr5aVY%Uu#*w6N2hgy_4y8;#EY}r*P zIl!I94Fw0;PCSrvh^=oEGAh`_El4rU*4&2Ll&}(jN=mk;Ae0*6&a@AORBY)($Ti9~ z8i7n|wxcMd9AoP_g*uezJA=v^wvTftJfqJZLH&UoZiR1lHZk{bR07VzLlPMtIBHNA< zTClNwsUYnVn{y13+Sy9uP@NKHbWnbo?Me^D9Nej=ApZ(mWE!$L*?P^8Zk27z0?Ax# z4J*{3MEfFCSYvx!g5qxOrW}yC&X#gQb`P6!4bpG09j-%iFI$%vYEpvN2R&@E)%;Mx z$KB!%6xd?B+l3r{Hhl;(Y_lZ}ph1ys=m=_2;@&Y-++pjSK*<1i?l=_OWqVCP&LG>| z1!UY~%Uwc>5SumwwJG5=3zhcS`fs7s0e68rD0Ilyx`$k0w$&nJI%0b+LCOf5`334w zV&)YpAG3|spmdbG<2rP3!Y2KM+%Y!)7i2zV`}~Gfakk|SB%HAo1@)gQlxXaRG6}ZJ zUMPId-G~hGB-z4p$a29Z8G_U)w#{Kka>@2S0#zx&riQXw< zHJic!X|ik~Bh+)l_G^M_l(;?%<#KG*IVgI|opk~7<=Jj+(83+t#4@BUu*Fv(={=j= z1=T5G*A3+#*uFhb?2$XyCgd-&mA4?<6Wgo^=}K(Z0Z8`DrrCoUlo;KI3T3vwFcg2` zZaV^r6}Dy+vcIx9P9c4j?dc4Xzp+gxp(Z7yQ_w?=O?3q&-nl!vh5~iA?i}R!VDsf6 zLxZhefCfL=mL8!NCEhg^`C^Z9>^qN8|Z_S-E6^rs6&bE0jS)=wx)p6 zz1+Php#v$KWfXGvv1P{~vy5$A3#t0qPIQnUXOm4r6-pGRpv(Z<*EAF!{4 z^n|-&9`eQ5rthJJQ?|=TNE>HUl_2RETeu9>DWUrn$|u-1-=NqzclCG3pJcN&Aln7o zeG}59*e1Ur*(F=@2Wn8FLKy#3A6Kv9)!Op}^*vga+@~UZ$WHCC-|l;saaO3?(1A z)67A^BHPJ4Eds>51FWi|tP^iL|_Cl^#w$cBS zWJm4DEMOFd|N2OhBne59BuSExB*}b9l7u8lk|aq;LXvE2ufzL#u5~;!?`%Ql8r#7( zq<&?S2B8im`a@8)&h{RLGH={1?n99VTj>Duyt5hOkmZBz{20=FvJIX>LX$0W20c@v zkcP5fY#SLU+T!jt3wgiU?AMUB%~rU9Mt;~Pijbtkc3OgJl%RZoa=&b$M<^zw{=X&s zJVCy$|0S-~Ae)5kxejT&*{tu7RLXY!3DqfatOezJ*pAy!T*jUJ7xMSA{p-^G$u4L6 z>V|ZEY%4NIreLecp#~+)^h1Syw#xzNSjpXp8VU@s#WavZ#nv+l=?B@i#vr+x?PDDJ zphTwuDh{#T8=-`TyQyhN9A-<;K+X|1wG}dqvhCX-g_f;r0cui$ZxJeuvDKHLq>j6V z6(~5)cI$>*dN#v4WSn42Za_){+khWxQQ~d@Do?U?L?~tC&a(rBrr2I~p%oL`Tm&*r zv*n|Z%FL!agxZvFas*Xo*!mJs+QMBh3593bS}DkFWm~;~%yVpymyp`VX30SvO3db= z>O9-%EtIiycX$Uy7T9EE$m3uOR3OVD+h-NhIN6q7AYqBE{0co&qNxF8U2GR0P;{BQ zp)bh0!WR98tgCF24m9Fs+Z4wCBw1s7lRz~}aP&Yq4_mPpimh`usepW5wlgJU+h9`- zLRueNcnFeivi%N2bxK^;O4^W#DULT=K zhHd^CN?&qUs6pW@oBj=QU$LFOL*^V?e-lz)vxQnvhZ5UusG4V6`-L($+&y*a{}d^( zS*4KYmMtfPEJe1lK1g%NcGM3EB{sPVdZt8C4Q21yzBEv@%-!-R*on;m+l#JmH_8rjB|py(8LC(Dr6#HMgV)@in&2O2T6eS0Cv4BP4^ zRHKCI7L>EFSwtu{%iUEF@>$tNcOlyxTYL}F+Sp`KNIK899fRtW_!)=tb~e`u6kp)3 zdw%OYKkWplFt03h+wkI{zqQu!@s2pU=jY6p%?zB26 z6ka#g*P+Z2cZ)tKa?DopL!J{h<2GbTu$}*dG^cEXAxKEFMZ(ZC zB?|jc_Ka=g0E(u#dp(4_=WO<4$eLy=B%qNCwuv)Hl3_bNhia6d%s{zIwon#|Wx4yw zLB1=twHwHmV|%`ZwAXCb5+u#DU6-LcC5}Bp`5U(5Cn#RvPF{ojw`~9FkiE$E)qr$& zY%8CTti)FNf*O=C(}oK7Y?mGAxXj&1*W{l94{Wh+$WdYIkwN-LwkPcsDd9B> zRXl9-*HC($yTT0=_Oj`Vkb8sev;>)bZ2b?AdXp{m2z4m2{RCD0Y-=?rv&G$09f}0l ztnZL#n=SVVSwyz67Nq&dcGQN1Ae;ObdZt9NYwAzg9k#D-C>r8!Sq6D`*(!3#8fG*1 zLnC`^mjjR_!ZxgiYLpPuK)HQ3=_nM7aPwI;$!1@H8kBJBh6-nF6YJ1%io4_n z6gX#7`XNV}Z8revFW7!WNS_8us_`C}hFWKfIP$J9Sbrce>*mQ@GGskv(1R1W` z`Vx>L&-O0~H7TK$f=V}RD;H3*z}@2|6uf1d$w96nTQ(0F@7P9eA!Ui}@D6HGVow<= z-?IfOP^!$`M->V^ur0knD;2i;SIG3pW@_vlV-x4ka2CQ1y-NObKNg+zkvuk$1N65ajt_>kLDdPquX} zq-nCf=pf;X%{Bo&Q(}G+%C^|XO;GfkyAw0yZL=w6A?puYa1I*juzk-%l3%t}2UMek zsuRiy7ysW9EXz==>wgJXtB_B^HtK|UP1C9w$wHBL5ZpYRMfET6`{m1cfTb_9AWc5K+aLN+9PDpvduq3iZQmE8q}l& z{To!$u_fN2{m+i5|r3AT0{G8)+2zmRg0?Wt?}Pc2HEl|p4BTTTY0rnuAg zK_L^{Q9raY&DN`eOlCGw4XI|>ni{B03Cp8U#lrS52Bl}YGwY$Sl`Ug{+;eQhQ;^xl zb}$X8=h>tds6&ZgHKgjB4E9^icYitu?NaA5TjX*U@P#!?Jb+*tU6!UWT za|HP|*wzw|&BykfgtVJ%)^kYeXS=?D>XbN^h4Nc$$2lk-;7)!6`M25r-9mPe?duNG z{bO4xL$V-Sr2;i5Vde=c?66(FK*u5OM(R*tmo3(S9AUPePe{MVw)F+cBWxev&<7n=5ApG z3ZAgtY9UvG%`grbPuY?akTS_OV1!zfxNCySXKWoaluB{unT0~}THdz$%6xjkX$a2T_8HY3_wxttDxMwS$LeG?FNDW=1;mNTS5xSzStDKP=gY7 z`k+FK?Ykd3{^riDf&y)}sv2_qu+0ub`VQOGC?x-7)9RoPN{s8FqLBIjme4y1CA$8X zxIG1l61L_v93qvL~o3Ib5hSNZT=Wa4|7*YK;aQK{TbvQWjj5G%v!eo45S`o3uU1WCAM==RmZk=17*g!d%A@p zdNykb@=UPh%8BcuR+h0DAu9u6x&w=iki4v{)D{KY?Uv_YGyOH zp^+K3%MK*5unl)v{#2ucST~fLWs}OFn3cONIpmvTdsjd<8=G?g($2G$1|g}PZAt^x zDe-&+$}g~~$Dp`_yT~}?Uu5etKz1kFh7r;&vDHnG%*AG(ff|%>I|~(-*(Pkz@d|fI zI}})DQ!YXdH{0$Kq+etES%%~uHjf+npu}enR9t79+kg^Y?yff>af3}4fSf+IV-YfJ zvi0pi3P0PwU8qS3tv#r;#kLZKk^$}>V^DCLZRQAaiEP;u$oP+KBnc^lY=8Ym@1Zs&3{{}YKHGj3N=Lbqyg=at zHs34cjggNws!_4Uby=`hr}A2Hv>6e*=kwHP-mOJh7@mXH#bm|67)r= z)L=`LpyWGu{SQ#^gKg&#a(%M3pCDtC&0T|(Uu;ixs6~mh?@+nLmivTK-`r_iP^isz z)P`1m*m{2K?yTCs4&HLnTL)|+>P8qfoZnb9po^x^^_s~4BJ)(l3Un5s?Y}| zI$xmTEZhAnl(2F))qun~w)6+&w6UqbAj3S{{x_tsvvqZ#CMEcUxj&T_*y<7}>ELdm z2MR8--S$E*C!0Y58JF0SN=WHq8yJLIl(;(tm6zE%!%%94JC7C$t+Ku7pcOaU+yrD= zW6MuMDi52^1hpyQ#0*u|+4^Row3oZ!92DMQYt2J$AKR(}GH>9EivZ)G? zHqI6Nv}ixTe^p>mnc=z>xY+@)5aP=#%94O)3*+gpcBRW`u~sh-%peyB|euK}p? z%r^fIO22Sd*nz?|Hhma!zp|Z1Aak9q{{T|Iv4swy4kfmaplX9{EdgcTxqC`NkqfO1`I(oZNR`Cr1;7v$?^d;f-PQZ{D?()O^Gg!w;7Wo%R3 zP@NLbd!T$Tn_3RV<=jOSkiU?2Tv5^l9nVSsI796DBU zmz;nCgKSD8 z6NSoCY=Ib*GI942heFeAODE8ZneF}*GR?4=Qjp5RmQF)$N*KC?Dzj|+S5Vr@og@#1 z=h%D&$ZcbLD?;XZwuO60ZD%V!KpjdnRH5nu+u1Xead0>A3Pl##!f%kr$<}#?EK6+b zO-SQnduc(!GMnuOdZxtuFO*$j8<*Js6kX-+L<)J`Y>HmUy2cjlgGM}T-~Eteoo!VG z)hMB=hH_pu%POem2=8B;8`$o`UL>_&E*b18gn}6yN5q zY=!(H+w?qS|HpQ*0O^8k8Yd*%VT-z;1|@W_LWL09<{EUo%Uxp~3WV7lKFG1hcISum z5w^)~NWRaO`UibbqACOxqilO&C~?5uZv+x!Y~BONdB|2fgbZ=E`C~|N#CDT_nv|eF zgG$G2iE}7S+8nj5m!*`7U+_KnTD0ZAKd*PBqC62}5i{+;bugyJ9E$#)?CC)>YW$lhf8+Jkgo zY%5Vn)?%x~pavz(96^O|w#yUfxXs;25(@mV#Zr)?!`5>F>3`X_E+M&a_5VMHk1Obd z5}kRd*u{2VfD)4bB~INzVmDj*9&$?A)D_6k!?s_A6f(B17pO@IzE`N!%T|Abl5*}A zKA>P9+ieqaDcB6(kg=aF`2#7HYy*PhPc2H^l|bbIwvH4^skrm>LZLynmp*7k%{Hfm zOhasW6{OOz>4u;-C7cXHm0`9%EtDSNE~taTqiiibdF1!;_IORJDD#a3Q}o+;7fg|a5L3m+7n z=5A;U@|xMA+mLmJO%jAgENq)0NHWXz7KUn+;Mj+9R<_~+6r1C2G7kA{Y-h)iZJtea z3Tf?Z;WJ3O!1j9%)hTg31LYlTwJa20?6K|HAX9`*Sb$XfY~DqvO$o0{ zP$kMXzXGKXxGT7!aEwjA4!IB6PB$QPoUPvvsgKw~0jNWXZ4s&-v#sqwnG^1wcA-du z%^HC`r);?>WJ$7(9YUHjwxc6RNU_Ni&@&~9Nho{H_LYL7Y3`OUAnyfRLenKDMFI58F}@TIsOehal50n`sYH3Az7o3F&>PO$kFWsM5u@ABWPC|0PIH zpl~;v?-X)N+1}0|a}V1>8dA&HiW#UwiH0kv+RJu!4Q1rq4HTeAA6vKxc@%7&5@hLT zTYrEwO176rNEl$VJwwlwn6E)u72Eh56dmO5Z)iK_eQr?=~bEW?TJ* zYLrmzTKZFNgv}y_Vx!z$$snJWZL|-vjj_f1A+3&0rh=s7Y};z6PKloyD6eO8jY9DW z?#g43-@rDlhwPJV7Y0aYWYbJRvMILcG}NGkZVOZ}v29wR<7w_1HYi|bb1XoP8MeDc zNN-`AbV2f2w$uvrL5Zq0sAy%|TZa;J-2HAqqK(b#hn(|lwE$$Wv(5j56bo!OJ5ZAn z^kJyvU`s@xxL}V)ou0MJHu^GD|OOWln2h!}Y4ay-Q#1>IN z&y*+(K-pckjX@|H=I(U}^6s(OM<8p2t)PWQ_Sq)JAxV_&bONeTg3<`(4%k8_C>G=H z#|-%n+16$uTb%8A4$>a6S?!SYnC;pD)hTgo3Cf?a9WO)i1b1>bg9q2n}nBfC)Gf-SZOIWlZLQAmHuwiScqS+nQh<|YEk0u8&rN^>%2p$3U{6+6nbQPX+bMhwz(h3^u(6` zg;dXMI?3{%+LUl2g(@#>eZ5e+#$B)v3cs?o`XP6nZB+%C-`F12kh;NU8HPHPm>q?x z?`)$wDD%PHp&p8SvdJbPPm?V$1zEn>KBpl~i*3mQ3EymGEA&i>rgBU z)dW?nZ1XcvdXBroEEKk}>1~jEp6%2QneA--i;#MOEwlu6D6zc^RUK?=ZYZhAZdVY>JF+? z;`u$4-)2)+pt#6gqzd`}v30#b_8{BFE2P_DtG_|A5S#r2YEZ&$6DsVoO?*SgVeXPY zP+*TuDXjj<5nB^QA1Ne{vUz%;4@!LQgNg@ib4n-?ePX{$Ap{0jPCu}Q|P%^>Y;}jG;Wt%ZWt|VL50vXTPM&=-8itTV7YEfd3 z11g`h1)NYS&E1C!3SF=*twJjqw)-{6bjfD&LaHoV+6T2MVQ34gT(RwML+KoMk{}em zX7hz0cb@Gn44H4(7WN@^fvtD|btusghpM-1XU9;c$lbsx6uDyypFy4yTjw0I+_SA` zAWfO=B?}1;Y_@CYnG*9iP`1K0UWB5L+?|vlZuP{FnnfL#4-Z4ojm+1xvja)9k=7iv-BYy>K+*m6-QHOQUz z5DKZ;j*g&}A-3KGWYVySNk}!!)=WWdN?5*tDkE$Umr#0?J97>SYuPe+$UVk3d<&U% zYzKFcdYnyKhB}n!uRv8j+j|wtOmMgO0!0jLrB}!^$!2Uo79-pF2c(%|8~lO<6I8+{E?1#rH+Lf*D6qyB^Fj^}ThAt>UuWCeg5+Mdk8S9K z5}iS)xWRTGf)YONruHCllP$duIsI(v7-ZOD+mAzv09)4y)T9L8DOB2KtDiwhk-LR7 z6#U0_n}J+GHp3NU++j;zL&^}_Kmlq|;%*Tt@3M7DP%6xw=K%`svAsM(D-pK2XUMeA zmajpoD4XsLYE#0=J5)Jf>uW;k71=Nt`jO|kcX;N%UqmXdURvv?%Dbb{dvT3#p0~EdB zZfFYfX4se#$gKCuESb%alw&EfbyXJ1v1^M!9XDg8HhE25w zX$x%Obx3;4_PYVqDRJEo<%?{!02IID&h`)Tm)LH0Ap1Srco@=^*%A>*_Q0k%fEtvr za|jhGY~M%F@gsNc1Qe*URg;k8iEZ{A(m%6ZT|n{|Hf#xFE;-dWNfiD zz9Ho|+hPZ5QR1Dj{-<)A&Dag4ez;5ZK%oxXpd4EHW!qCgCgJA)TY@kEsk;7`=pBUG zl<+zPRU~ZlBT%}VyMh)9OWE|}kh_QNbOJKV*!qo-x|c0vf;yDgHbYf8+uAIY>ErHc z4vHw)taixL&z5sQ7A4!*5~LYmJ6eVW6`R}*JyW9SfwF^aUtTDx=5Bct@(!_8wjis9 z%`8GA!)%v9NHW4UybIMRA+`tQM%koMD5m9ZD+c+-*xuuiO~>XufwbdnrBg_%XPZhv zbxJ%>L-`3d^(7QHa2L6P{F7{5dB|>L+bBS~DYkkMl9|}-_fUfpZXck+G}}ZKIyQ5c ze1-xuY|2;2VPV^SgY>g(Kkty-%I0Z8AC&mqf{Jr&b3ahR#@+QVB+j$xB;G$c?QF+V z$gsfH*9$2eZ2$V8CMC4`q0%DTiV8|PxqDPY!6mktVaVlT%Z@_EWwsF=q+DS;)I%*w z?3skht89TODCOqvV;Tyru`OAk6%X6J6*8@}ndTvtmo2>jwJBl92~{@O_FYig$DL#q z3U9Lc)*!c^?QI=0Z?P@-Aa#JP=!ZI#XxN6T+iYk5pp3}fKnRNbV+)5NPmrw>fh;?0 z>j#h~#P)Is3A=2zW9XR@^9d*$W*a|)qI=w(oI~CSn<4{Q_t}D3Xe7$^or5F?Y^yg= zjS{N2P%g%1DM7JA?yky^FU~gl2-%L<;!lwFm`zrLq$h0Kb*N5>pA9IVU~_#!@l)=~ zUywh^Hr9cH;8c2S{mKuRRC{Z;A6?1HR<51$7yWa`u?^FH%=Y!V>Id9l%Cdg1=o1cLcw`@1F zP?Hk$HmFo&OW2|09e4eUP_V?dvjn;B+1ksHvCQUnL&^uXClA!3#Mup~Tw%*?La9gY zv;inoWjhj~l_$2|9mw>|ChkJ27q;de)TV^xC{(GjJ;b2&D|hB2C|qaDoIvh3w&5gX zZm=DsAoV+&^aAQoqW==Aez3h?L77kP7V}V~$yO>ro-a1z9b{>-o!>*6Z??e-B(&Kg zRp^-#g%>FM!?y7XMLXQRzCqq!Hv0!;6$<~~5(-Udr0ahP6W@?T!gl%t)hIzJ`2LjZ zW(!H6n3TI8Ddg*6TkC~vGPdVFNZZS1RYFoZ+qDX+Q{vbVl<#9Z9){uy?&Mm?-_Q0> z2icWuUwTM4z_v08$y98WDX2jSGiInT$aZOgj@8_a%t3)6w%9!6(6IG5ApJ1gmJ^bX zuzk3o4@z{dLd8+G`!y(`B{Yy+oIixPLwpz<_Z=Nw9zx$|V8&BmwzI9)AoBv-V;xdE*evf* zhZ3`&P<4@Qv;}3H+#R-|$P%0E7xK8+0$rPbvMjTGc0-yKwj~)Ptg@Bm&@&~P`k}0w z?P36mu5mY{hP)oOs0OmGvq?sw5ii^37$n(XdmD#pl;AKxIUifm2*ozJo1BJxezvn2 z$hO6%vO?MbTi6Ckx7mK}P@NLj7ooh!R$GGN|G2ZQK>i@xjT^G>u#K-nx)57p1Cs5s zDg01_5_SSmA2W;9y=z|jDM^G`w)|-G5 zhum!^Au-O@OhL{gHrEAYIA(jegcK)i(>bU~37I@pO0a2eq2wud2X{~~$<|$lTxV?l z3S>;NHL8&EoNe(1YEk0dD^yOi85>aQg1giQ6w0s-enBgjY|iW9Ov zvzeEnkr%ehRY+1}8}>joN{D%(+$)=O6N=Tj+uDMBZ*1?|kgdVy3_{v>wo(X^ey~mL zL3K(z--q&_Z0Z;kZ*mujL;f$et`o@KV%s={bl+_CGf38Ev!|g3CERA9!VlZT6?EL; zE_n?Fe%X`-$RXVR|E{=Og!EniOZ+K8atWK~0s5fC=SQg6%{KQ8C8XS4*C4TnP4@;l zWo*aqkfE2YuL&vSZ2wwNlM-5OsMN={@(U#u+&y+}{VCYbHY0^xO17*FG7hkf^g&7$ z+hIS{qQo8*R32mtsG*dayAKT%8e&@-g;q3d_hXQ0n9ZbzR3mI@1JtI3p(&^`%CnuKyTHp@8_o9FK80`l3}MzfG@fi0ecv<^1e4J2J;+rEYBl=yiE<(+J< zG8A9pu3UlqF1G0>$iB>W@dD{q*fe!Ww#pW5Kn+Uh{)7r{w#_f-c#XTpHx%%&IXaMI zo$XEt{7LU+o9u?<8*Hf_=z|hfa;WHI+fzV^P40e`kmzUg4noc?w%QP62(ZnMK#Fa) z8!gnN1pPQv64??HQ1Tyl{YEGlWZN-8t{t|v88U|0+_R8!m+ff|YEj~>9V&;}atwq7@6in56wNOi#0^g?Y)Sl)yxF}8;-D1FGCS%kuIwoDLmAF&PZ zLgr((gFQ%n!X}ME9ZK}aplX8cJq~3~xm!GeB1yK=Ddah0Go~O*itRiNY0lXOFCih# z7P*3+DN)En*$cLf0u;?~_gaL!mu&WX$eLvZxAKajM@cKr+0DRE4){ipmL+p!dim$;MnLjHTUe|?a> z%=Xm}=^oftRFJH~R#8I@N|+gj3Xg1;qtJ1cyAd4}cw&p`A;&XY&m^ROVcVL5#LzQl}z6_L>au>`(;U2bD4sy%bR&OA4FWciSq?WT; zN>GOqvt_8-$2R&1Wfa^UK0%RwHdzhwDA@va$TGn8*?=@Ewxv%<7-TDdLC=(EYC~By z+eHV84skcsCH~2)VT*P{)?qe@3>q0>+mu6+QMNY)RHFpP0F=|R6$hc%7{D#xHb`e;OV}aV zG@D`(YEZ(?5>zm=eJ?}DGu*k|P{6`g^+1kUw%H9xZ)LmMgyeH<+5q%HiE$As+Sqz` zpu{|P+q;lxXKU_3&ILAC6f!v29%7JUk!|`2YEnYx1S&b%G)XAA#N9y(3cA?3FCf=4 zoBt9ruCO((Amu9CVjgNy;#~nMyV;C)P->04)IAjPunktAm36kgDrEAq2``XpgU$O2 zwJG8C4XXIq=0BkHCU=D<6!x>}zajS)+vyKv4zTqL|Nf-jW(!H64kfmwP*r4G>xDA^ zxO?h@B0)B*67uY@>k^f9*Rb|Tb_ix`)rjd z$Qorco1u{dwo3~niLnjOK{ZN<%|p3EHmL)O#kt#ZLcSxmcNb(kW^=AW+7q_Y8YE4y zO?ja@C7%1B{3)Ay3yLSXi)=&wGq$cEWKXeegdp8HTRjZP(ros9s6h$02T+;`L?XV5C zD6!`kDnGIXx`Ka7Rk{18_h#n!Py zmT$K8MM%?Tds%{nA2!Ay0q?~4Z(m^dsoSlHmX13fU zl$zmAYl1=+wj(pNGRxLG3z@8J;vA%!V{6VsZAw^nKouL?gA+>6b7x+L!gjXID&$^Z z8}>kE2it)cQZKSeH=zzC`nRB}lkI&Q$}DlW7=$7&wo(Z4EVCK+Aj=Bd`97psWgCn^ zf}1T8hn^`>IDxWjY#XOg)WhBD8RT7Ov!@}em#vV2MmE?ct{{nz?erR|QG&7n+9Hkbe_B5Ar!u0YaK!E0^4c=GT*X2CLwi^&2kQP zC^35hRqxnFvrwkQ-C+)j+_TAUAWxYsa0^);*go$dO@(c#3<-~HswgKec1SN_yK+GCZZ_=_^g)U7WvD1+ z>vcnk9`3e1kSJqodLd^on`;v?$k`sYAVnYBvHhmRxkF%XVLuNf&|0|@PU<^`VL3AcWzu*NpA z4IO*9Oa6lb>ukyp{;pgr;0f}2|x--Zb zU^_mC4BKpd8Au_r{mVj4N@(Sv(m%G98z>p%?(r51?y$|2AXkVjTZW9gY$K15GR$`P z1hpu!rv{bx*aCGZ72)oq0fqM2mOi1CDBJxPWIA9owINlEE!~0IlrYq__ovDs+kQ8c zj&moGLE$4dpB!=@v%M)G^9kF+0HjW^6$haXB^oqP^_1;w1j;128yJHkXKdkd$dh90 zOhA@%wsj+lIsY2^z_EyEE`blhP$gx$X8$+4M4VAwzvpsi)^wTNP5S%y$jVT@pBK#m)KlU zD1OgfIR^R5Y|}@O{ekV`1kzR5G)YMI$QDgO4NB;~fC^Q%%}eO`iMz%X6nJKH>K^)_L{$YU*4g%|P~wfd-)Bf{uz6o0=Q~^N4KjSN&3{0OPqv#T z)T9LcH&kk}C4Qjf7kB+a;Y)^eqixOv*P`SgFQ$eX; z?zBTtNVxz1mT)u-t#th_u~!S3By6G%QgyR6^-!A(O80PQHbY?;KUu}K|JhZ6lxsH$LlcR`tc?iN>}h?1?e26+b9j9$p1VmtRinnAX~ zEl5zaMYf@5N)&=nc8F~w1VuI6y@nz0Fq?fJvW~D74xo`ywuv|-(XyQ$Lp4fJo!v=eOBYe;HfyS{PW}M-r`Y~I zLUt3|*At|hW?QL2GBaDH4mBuY<{c``uw8ya#}@8JT2Nq?E!Kt{R<@pBNI%E6)wTa8 zxsC0k8~UI`rwl62v)#*~gq^#oen?zkOAkO!2b)?A85Y_0HITx|)-?(>DZw`em6q7* z<51GY-GTuMF0*gAG7<>Ah=2!+&H;T4b|C zpbjNwqfqr9+vp*b337LM1Vwh(WC_RiF zwi7nh7o<(Fg}))`DckQ4RHwvsA^N9$lC36z;%D61dLVy_?WPy9pRGjsK?yrUP$9$iJq#URa_81Ufh=292RW|TW+xzhj_qm^l3%lFP0$A=#?4SM&(=E& zC2qLeo`b{!TXPF{ys+)XAybV_IDu5JY~E9- zO$o1OP^Hc`pN7(J+!ZoVxWT5sg52+Hr`M4AgRQ>+sXy65MW{oG?GjXNvaLNpnJ@01 z9-&B!&H4;^zS(j$$kJvTdxJDTY)9{q&|#A|p=U}ITTu3w?W+w%h4TNmgymnz+x5SM zO4q@ktP(b}6dLJfyOco^Dcf)#RHKAgKa}fXld7PYjJqv0_OX@5 zAgO|FN)Od3@!SCA``Oe}P+ZAfWE%1huyy@M(j7J<)4)*}|LQAA(w!tpNRlK;l7wWD zBuSDa2}zP9Ns{n5&w2TqYtHL-{>|)KAiIif%?jxU*y=V&re?FxLk&u}TY&O|Y-27c zuHi1}h5|!u$`#0=W!qbY48v^SYmi*W=Ji8ON~{K;!U)^!Hk8nFcfAXVqip&RB^Sx9MSJIX zfI{z|Ov!S?nAnHSjRe;~D!tsq4I^h1f`-B4wb?W_mNxVZZ#hr&y2AqC`hv;8O` z%QD;Q0HpD-y$nLa3Y$#}RVndC2W7o%qoYt{mAjKM$me5I7$NH#+l~p+uCslaA;|{Y z@(fg?gvujb9Y+G!xB}gZ-$vlvBn{CSr)hY4A2i@$jxi+BK zE_bC($RA{z5+VB@+r*c>O2vS+LSo+4V5x%xnC%C$(>Ge z^rv8!?N|!Auh@EfAybY`?1NO-Y^{E%LkUYNsC>irsD{$F+?j`&oo+)lN>B!&+#6dk1VtO% zeTSj-cea&7$kt@59znVfHftP`w%D!{P@NJ-&Y+u5w)i;|YjY>hK>jbb?JQ*Puzltr z{WqKY7Lxt2mG7ViB}^Bg{4d*O35pAk|KAd{Pf(!ie~Hm&$RT0tsX>Nrw#_;um$Ee* zP?Hj!A5fu(?cozj$he#AKw>Xj`Ui5#+0-2miLv5jjW;{aPy3n|rX|3;uTCGL$vr9rlzF({?s&TE8%Lu@Z5$gO3Y zorX-qY&SEIO2?+RK^;msu|wq%w!Q@@t>r6^RTMAI>pwXj{B zK#>{l29uD_$`(mM)>$^m1*EmHZCpZ;bYZ zu&K(B&dC<4K+;9F-zrq6#MM{m#>G~9gJMhE*_x2w&34;@?8|JUUy$CzmiUHbD{Knk z_)iT=*p)zeFWZ+Cim!6#>4gG5wn`u5SYw+}LWXs=D-|T)VABmkO-hUnK?OfsuMSFV za<`?2!~k2%06DkVTocf^$o4o1DYn_B%utIGG8U+~!={;qlDpg;&Otjtwr&UH+GF!O zA!CTG;ewR=Yzxa!n-cF=pi-F4=z~%R+@;o`V1#X86LKH2?QcP*D4VbYsgBruL8wCs zuOX-$W1BmG(#PE851~+;%@Bh;Cv2y2$edv7KZVq%Y{4_=hY~yIP$kK>l7TX3+&yQZ zaEi@(4SCPma<`Br%{EeiG#6~gMM%i7$seIAC5lf__LA-M8H!}NTdG06E4FeSvgX*# z?~wMI?eYVX+^`L`p&BJbJ5cVHP5KK(^W1HA#s9Q^$M)V0*$Qk<8Kk>sE6O2hk!`Xc zs#D_mKj`LxO|6DvCGNr+$p6UJH4NFyY-=Ns{)w$V3dt&L_Hn2|33o;)|I9Wv1;wk} zC8wdl3!BmkIcjWsHpuYG_HBpcbvEw;)TG4fB2;)|n{`8p26xvUNPK70uR_iyTYL=~ z|6uF$Ly8vLb^vNoLR*B2pKR`3DB0%jX%E`@Vw(;_t`1u^0vW&Av`3KghwbPXYExoQ z0xJEo1(HxoDF1&;Xr`cG*Z&e0FCe#s?cowKb+egrkV?vyzJWTFFn9-*d)N-{p|p%U zNeK$|vaOdPkDTqT0-5{R=3gMSg01ih{ZQg~1FH11oi(A1lDmJOQ1~BP=nL|y*nYkt z%K+P|aPlXOn(ajb{e8Ut|K}xJ4^*YZn_eiZVH;IIksW|8f%A5w3#NmbAfCHmD+Wryuu17&u(TNsAILAK%uv7Nf18YL)KpxiNA za21Nix%*y&)=$`0{E#idRt-S9Q#R{1Bu%ni??QD-90@@;XKe8>6iaa@KZN|}Y}-eW zJ|NL-|X#%L^!;+YqrfhNPffC zyoZ{U=qy2nTegQXl*n^8`3#A7Z0Q%sSzuGwq49gRg9fB1vUPnxElODbgo+Pr^)Dz{ z;%@#2+IeKV6Hfo+DzlAuL&hhzWDlgQu>F%mZA#ozK&5B4A0?Eka_1d@f-h_@gOIz% zHmik9uWUCuNL6Rkk3t0eQc^rNT*;6T|v@*w%=>0PKm2|=tjv_D?qV-+}R!=zl!bl5wZ`kjaDGNnk`X< zWP@yqSExY=yKhil!}j$K#fP}_w4i{NtI*mRQQpPG~ylR^bO zTW>Fv80BuO4-yS*t$xTk#^zE%o>b2nxH|tZ~S@%$7?)77yFV8KhZZJ3fa5FPl6ARVh)-LfKWe&m0u-akq2}`PSIV zcaU|R&0K`E8*G;)NaAN3dV*?{5PgPnn{3h=6b*2;6$Z1#UpgA(orp!@;b zmCF0y&&p_e{o8AUF z6KruiG=9p~w*V=UY}<=aixS#PQ1Oh-?SYah?w-8R&N1)DYiDKl(G zBGjhDo?WPP$rjjyQd#br`%v(TZ7~A5b8HV$$aKwSI)+p?Z0QrILkWXPsC>(Ikb=^A z?j#pb=#Fju67m$--mW0?J=^>Zq%N`*^3V?@j^9I-2ez{ZC{yC@Ul|HNvV|&;x6Jla zg)C2OtFMrz!uIk83D0b{CRC-wn--Ln?(sl-?_U|LF-Mn;X%mu!4?~WbS*ZS4w8PdZRw#pC4Ly7n>L$k z0*ZZcSDJ+U9kwYmWdCNnut54BHq9&~`(=yFK@Cdic0hTd^8YPi!wJQ^{+G~jK>-Py zV;OREv)!*i1}WQw50dw=rPiS)C8{=|f{bl{3rh5I_qz>=ayDNOa`v&+LeRK^ZSDY4 z^t0U_LM=)##Gs;*EfI&3|G4Wvg?3bIyJwJVfUR>58P#l_45S=nd(J{_N}RcdN*cD@ zEtDGKPFH|}TDIdNlO<^%GK zu?@8$^ElgK2U1V4Nq?aqO7wS~|EXeRd+&xaliV%Hps9NX9wB(bxdPD3?HP+Fnf zJX_EPMIGFI+oAOZwv`3Q=47ibLb^pZs~eKK*seWLof1b@p_?VP_!<;*b0_yh{$;l9 z0A%;DeTtBNh0VPS$-HdkJ*Ytm(_tvT%61un;y&)QM^IpmEqV+&*4cUzkYR&uGYQH4 zY|RwZq(tWhRM=#DxP%e`?k01PxW$&fft(_n`VJc3W;?it6gzBPC8$LS>t(38%T}*I z$sl+0FVM~&+ubYV3bBnhAmct;vI!}}Z2vx?HYM(TL8Sw>pKmA?;m#|h{}eoAdyzoy zDBElgWIAHI>4j7=HoXGsP{N54Dj&1;4M6EQcRPbn=!C631bGr{%R0z>%J!s()JZnW z81zGl*$Jp}#x`t%GAZtk%ux88O*R90(`N&^xMV9ip(-VsmY{5w z?P3{1AJ%w&cY_&5e_Q;(r4f)G#w;9O(#5Q^b=__oBYe@FY zrpQALO4uzx`6}C25sJTX=XrzzHMYtVMy)q$KJY_4Buyv6p|b@3;~C)<=1YEeQ)1{K?Enm#D`#ob{)w9{ehRza?B zHoqD&{;)MPkn)#pVHj#t;{6Cz5}yCRB^V7*s_TD=sc|SMVH=o)+}&*ZQ;lbtvK029;%ObMsKTm%ID|6q2(UT#%=a?bHpK6>R-0kh-5OxC;GHVrLDiDA`v0 zQ05m5;Sb`L#RrL;t`Y`V*5OXB3ka25|D40 zt(=6cIyUn;q#a?qynrNnwxKLkql9P<%8jx~Z=tAxyUjajeT?n>9Dp-P}FRLOaWB({{+^Vaqxo;|iO05mI{D zj+US{CH8or(kffP3#EMAHGNQUjcsuQa<8*JY(l0DHj@ac{A}qRs6z>Zdr*0k?O-2D z2e^|&pwJfEdKB`AY;Q5hyv;U$0;zY{3a8KyC61?{$}Zbk8p;H@`*#V2_t-*LkT=Bk za}8Pc*;ezACd~FyfP@1!+XGak#G6Mb8(|x*K#@c4PO6YE%BFaQtVe7+Z;&>|_Vo@) zj@g!5P>m8QZ73IKvwTC*6Yj2lq4fmYu;lVjwo|s46w)QxWWA8|jBTqAs#D@eKXj8~ zbE%-%Id>&B;8Q1X$x{vfndX4?%xt|zum7&2DaJcp3-neF)qYE$A&94b}WatSE) z!kz963f9<;&ms3KTWqGsp-P+Wy#ZyuxLf#u!X38aC*=KRGj<@$58L?{ z5UT$_7lgZ^DkTasDBH!hCWj)D|0TXEAYV6|{U2nNvgHRLZ4cX+29n6wPPI^t5|krQ zu9q!13Pt7IeUCxweQYa6$fjVcnjl?2n{^tJD%q}QpgJXv*r1z#Y;ijjQ*kF>fcyh& z+l!E0&Gxwj=?B@|9!REPD|?{^B}}hD`60H;4JfYVP8)y%!)#Fza_HE4b|J$E+vXl5 z*RwVEp(Z6dBT!+K?I8*!4BSl~L*f`)`UG;0v#FEN_ypTQ3Q`!^x-Os=C9GdU#YwjM z6_hk_H-7`|OtIbNA(xqL{2nq+vn3xOrG@QZ8ERAFUIi-6u>Dk_l$AU0D-@h%dwGN0 zHn!O&WSV2UX+bJGoBj*xP{PSKRGw$+6R!T0c5t^NfkF#xZ7Jk&vMu*Q=0&!rK1l6i zvnZh-O3bRD$`ae~Ae3=)cQgcrm)T@G$m?MX=poAr+lK+tc-a;wAYqlQGznEH(PW0Q zKDG-B6j|eLa2E2dvqk10>js;|0crhg8%{{F$@b=gYLwtuhH?S6!U`1K;%>qRt&41D z>yT}mO|=Q>cGyB&kaU;rcN?lx;%X4O39{8fP;8Go+X3VcvE3d*_I#MrKGA^9CN)!W7_J!?Jgd#QWmUbcED_eOF zvewzmVMzPNb{T;r4Yr{ps749VV<`8|CQU%mCU=`jX#IojJq6iXY|aZv_sLeggrse@ z$sAOt#Pb{I=8H{z2gN$vh3_H%H(OT;vj4EHl_C8vTfG9wgqQ#SW3azK4NACsh4Ni& zV+|-S`Cme^2?e^@l%J48%C`3f8G6{hzahDd%`060sY!`d2~_B1o9%%Ta_+8sA+e85 zuYjBiwzv`+?`P{9fD}r$?LnwT3GE@M_>ax4gOV!lp7hYp0NeB!o7My= zHEc&_s7;AIGf-)WEienEwA?l4px`juq62d4*dCmaX@t$R1gZ3F>1C)x34>m!Jj!<9 zgVF}>BpXm@jBR}r@{F^+Z9(Pd--jxbY-a~h#>CyfC={Mz3&kL> zne8VIS*F=mPa%zk?d1#-X4q_Ls7i@987OOI8@+-ev)r9rLp~duA`e;T*meq#*3R}- zge3E9%a2fv5-Lwn&cSA>LeT~8u4>S_lWq77vMsX3-XWcfP1b^>OKe+hs7{F=9q7i* z=K6(V%iNW^ZvN!=uuVxJ`wH8I4AOhqG<}e4l`YZ_H7KE51?7Eg8)_)N#$7`L1=iUd z!;oWx?S2F@_}L~5kbILZH4Zf?Q8ft_0&M$JP-2U_-)Tq`*?d;Wxy@FyLE}4YbMug5 zm+f`|YEgp01r>vA2{)A7U$0U?K;m&*xg%WI;3&?ZI zHk5_TNw&irq&{Pl-a>%Qn^xN%Cx`Jy4AjlyWF{ z#}-sT(E@khN@)F_ZDjzm71^qTknVxas)eK_wrd?!r^Jy_=;o0vJ_g0g+{ulQ|A}qe z1lcQWpJqt^%;uhfWL38EEYzTcX*-mEVY_rd@fvs9MJVvf7F~iIb+#T4WO!rS^g{9m zThj+MDbcwB72eq%Hlak5yGaoeKiJYckh8_6-h;+J*$(z0MVqZF0<|b%Jqi`S*y=GT z+2L;f1lsv#yE}zkKWyVE$oR{aOhZbc_Wv#6-zC(h#Jwx1)W!C54W%UiOZ4WUU^m-K z0dh;(W*;C^58KTnq>{1eD^Q0LPO4D3m#yy=O3S(1d4ocIZ0&c*qhMQZLFRt8r#7Tk zvRS^NA4<&rLY04P!;<`;GAizlq)>Q(P1Xx})og)2$TGO&L7-B1_p(-Vs zhM=sL?P3^;40AWAhkQD=hyk*Wut_E$t)6XT5|WIvy-h(iN^n@9oPn)ig`#8JP0T^- z<7{X1kZpoZ<%D!bwvY>wPO|;Fp*kh5u0S^?w%RHbo8r#44*AV&w|>Yz%{ICP=`C!D zZAdo5rU*g}O4toSc`Msj7>dtw=Q)G|Hnz$Uype7~8&Y^;X ztv3TD7P#BWLZXwcm4lp%Y_3~q+{N~I2Pu}=rixID5;7&I=w{PALCIzA4xgbN4_kK) za;>oW>yXjQ)@VS=Rkno>s7;CYpHRuiX6!(zHSSVBP;i}XpzH2W?hUs6Zph?k6J(HT zlg%fGI+XCLfXV^3xqnc4i@W>)6cX7C8pyNFcB+NUJ8b<{@2C{_NMr@GgfbG~02@y8=0#v0$aS_TMvVAT=ktlad9>{mZ zR`x>H7@K(w(jK#2Za|Va+fV?iQ9@LNawlxkT_~F1ZgUS>KV^I0hipkUX9Uunu@$3` zG{rV~4Am*|`~3sq^$jZAvCTH2M1i~O79`%Y>AxUn zkuClWjX$vU357o?N^IK_s6`2FDO7x9bN52YGIvjX(9RRvv=VYv*s?0f_{^pqgp^gb zqamnGi9I@~^uiX8sWm}wp+;z5xNyzlZW->#n23y(!btqwQ7An899n3-L zCU+7C6#8IWcS4>P+nWnAf3nRlL+UnLVFmi3#Bm=~`C>a;hcX@R{%u0xZ?@1D1A8aKs8FJWTBj#&2kMz`?$Nhh1L~p!v)CJ&lW2}IwhOz5t9C6+j@fPl=$%s z-Kf}HH7GW~U8xTF)ofGmkbRKt;sera*fecOHpCX`Kn+Uh{)O^dwvDd)KgEZ+Yji^a z9h*Z2IY!v-<&Z(oHqj5sN7+*Upe7}%)KI~|wy%K_W8D2}A#t3|Hv%~)*lMHDxRGsc z98yfO-5Q}5B^ailqKPdr4JD_z>$gHXX0}}$p*AJXxS^7j zE$4w!v)t)cp`eZJcnxyTvGw{PlbuZrK&pARmI!qyVQCjCJJ=rgp!5QF<}eg;vSlKW zXOV5_2r|3a4v!)A5}Pyu{ZOJm302%|?mHU z?Lo*MX8Rn1^apHi9VCmemGw}A5~jzX{2|-r1Qd^Qr!_%=Betj+a>UqrW+1~c+vY4J zkFz!Bpe7|c9Z=zf?ZF8p65LHLLED%R}h`cRK|rbkEil>Ry1!)^>8)``M&i1B(YLws@hH_1|!Uz=o;BLYIt+&|D z#v$7$n`#o$wb?>bko1e~cN(fw;;I$8>9EyoQ0$vK+dSm|VY^*`?7wWIE=Vub|Gy<9 z+>os6e~F3}s6h$4t59CT_O%AZySelDp@5XF5`Y{%Y%|-CLB@8q3(0%gbRnooiLo$L zkhAq3LWw@^wvHfC!PYv4oc(OB1T?N>drU%#e{56dP>T{W7f?~frpZFd0qzcS(2km| z`xbHyvia{IqlT?<4=IP(7D`Z?67S1UNy}z@hEl`arCy+*j%}b0xkuRc8<0uQCVW7u zQ8wQv)S-meFQ{x_oBM&%W8CG1(w{=(Y=&;gGr@M+1DTC%{c=b>$re;VKa|)}LKPF+ z$^ev^;_i763Y*!iTF5)imeWBN3){#jq?utm9)koco7@OhDN!^**;%$vGZe9Lw=@I! z=Ge-!kk!s+wnN%^wo3;jaj*?7LN!W=E@-pVsmalx+S*a zCM0#UO^Q&R63=&_n`Jij9u)I%7v6{bD{Ng6$nIrZi$eNUwt5Vb`Pl3yP=gZgPNDo7 z+gJ*UuXC47LxBx82JstX3PFU#sfC3@gdKl_Z!k`{1pRgSm zpmc&e$pjQSWm}(wJV~~LAlVCBV6 zh;1MV30k&r3aV0~Z~pA0E66v(X1{@~dbWHX(vGr?-9r)s+vx*TqXcCc z%8juFD^PTtyYDKrKEbx~3fYWo)i+2t$!2XrQWM*C3#wD%$QN`o#TNgDVrK5-!qcDp z(`?%k$Zlc#ltTI$Hg_*1v$B=@pavyOE1~=>+ocMM+qlyXLV-E9=n&+vv-Rj8!#vxj z9+ErQng*yziOvbAu)y{(2_>A|O`0Kbku7b3oGv!?EHu8vb}$Di+-zMAs6`3uPN=xd zR(C;34|ns+(9R0m-3sLLvW@#7<0@Nn9a8$({%t~SO5EFmN^5LC+fZtqJ8uvQZm_+C zAh(}w_5d<%vfUg)ssNil26ZUmBo39g*!oVPw8-7g85G)PYo9}&9k%5RWZq?a%0lWO zo8=n%p~UPhRM}%2E4+= zHp+JK0Y#3u8*D?q7+a(RS&!KyzmPW0w$WAjljMZ$tsAOQfzU1;gp5_TMha5Cuq|9bZA!eqgi19wV-8Baa+kV+f_1imJIMXUwto+q8f-!dQoXbJ z%20+iV(2jB|ILfy4ujSgD6+xbi5v3y*%T{~b(3vp719RS zzSbbg7TdBPs!>8E0OdqB%Qh6<=I&}2THj$C4nekEwpbX_1=(bWkaUl2>jR&` z6Jm2Epx8clr6lAJvrU~t_5-$y3rHVf(_|spAzLH|H7KF`7RpE2HtwMK5qFJyC=g?F zlpx13+kF`_#MvgEA^8bg>IG_2qN)xR5^VboC~?Z&Zxa%eY`#y(dB#@zg2q#9b3c&c zob6V4`BRG$4Bb#M&6enak{8_d%b}eN+pYp~U9xqQkTJ{V8Gw{mY|n#Gn-XWVP$|cj z(?O|g?sTJ2@P_Sp407MH^%@~lo=r4Csynup8R}5N(hO8CuszN~>3i%(q#ohNcwEoGql80<j_J%eMUn z*@gH2Zwa68kiP4G3GNmoldzTBP=gYtzoC3L+vP75mvX0-)czFcVT(#3hm5VK7c%s+ zZT3NOIa{+IYEq(81r_?(9@J1m!QJE#B=)nVhasnuO|6H<|FInyAccypYXWLf!uljs z9AK+YK}j`t^A>1lknPS2xioC!bC7X}EjbSKP@59>Tu^D4?Z*wJbliDYpx_AG z%PQp7v(2tUrct&VKcq6S>9?Q`C7f(S9HgFyo+0Zzo1_M59c&wQNV358)_`i1;P`-YPPW1)6kX(Qq64kF z*v@_++Y+0q>-A4MH(RJ1k}k9T_CR$?T$MvN9=4hSimh;G`v>{GY_|iDeU)ug1L=Ki z2`waBV^fSk4NBM@h4SlcUt>^wgFBBA3i#P7CdjeLHZu(w0&G_^kbH|xXM>uQ7_&nK zk*#+DN^EntwFrqjY^^28xy$DAK;uESM=zwh(K*hypKYqD4X#ZN*!^RI)Q>Qwt*z%K4#lbL8dsHZ~>`K z*nF2zhZ0_|pmKt3?gmPqa+l9Tp(LB(9`c;AojyS36kC58QlGO0E6@)mcB)V%&9?Fi zWiGgTeuKgpHfs~|Ub5v{kR{7D@&#$G*p9y;A;%^c>VK+Iq9}o~*KD6sD00KyQZMAY zWh?hV);yb832E=xE>)1Ez&11p)hHo41m*78q&g^CMlrDW3w+q4NACMf%30x zV?HQe=PtPp1>V?{n~ShZ(K`AMB&1Wds!?sw1+%mR@I%Mi)GrdD9Ia~Sz>QKU98!Gp) z9dw|yf;-7C6zXSN?|S=_N6Gfq4VnM3&C4LQimf1rekgIgAF2$no&AF{YVQ82q3|GE zNCSB_Y(HAaGQ_qz0%^2tFQbq!%w`*hs+4$Rgt9ue(J3f0!rjR<|%Z!jOH2?IHr{t!$bjNH)tBIffdP(4BztHnxo<6rba+k%9tt zHpd0zm}k4cgbWV0i5w(fU`yRVO-fYVK?Nt<{ymgf?w1Zq>_Ob=9A zXUp|MsSWOQ3MlAjJ61yOO}5?v$P{1`2O-rKTWbjFP{NW9DvNB7dMLfkop}rj?XYDg zAkQw_kO?ve*$&N+dXG&y1N~5@Mx6#~X6+|3U|JI`!)Bao}gHg157FKo$iNLge1Hwm>Vac>GLy|VpGL#aAQKUo8!CUW^{qhZ7I!GK^366Jf-*nc9fhIrFPrQT@(LgS-x2~xkfrN?2_MIhM#8q3fP`+gQWC0C zqUjvUO4%+hphypQgIUNYV~gY~$r(z2=An5?xZxgCh;_4@Kqh_mpL9s#ZY(J1+!*(k)|70Ix8|{Yl zTDC+FBpYT^$e{)$>?)wVj_pec#YedF3_t-rTV)V(jIzyWA%lVKN(afu*mR>%lM-WN zP+^>{*9avhxZ5&8qLHm-hMbdZt{G_D#P&D~DW=$_>`;pmG7hL{X45P}$!YEmm!KUB zTek;t&9M2skkQK4@IlI1wuKF-O^NrLP|3z-6rt1{cc~pHXlEPPgWU6M`}>f|!6rl? z)dHI@3Uw&qH3pTPY;z}2dXc;QDHL+C8B&mEiS0BEncZytmymjyEqDd}P-5pAs(9E| z@=#`lyXOKF_Oe+YAnz($?h&&1*hVUlW{vH*3JL3M@>i%ziQ*fS-C+BCha!INmRgW+ zldar_tN}LjH>BNSyZnVDBHNJU<4-k8h)SW{Hk-5;itcc?*$1ufvc301wji5R1?l$K zifTw2Vw)U->Xdjs4BhOrsr67S%w5<3`48B-CLnu+ZEX_LAF|b_AX${nZh;zxz9rdflwoo_Z zt+M^}K$aJ_RXL=ovArlD;g!wy52{k)%>b0GvyEz?$QySjTFBR6Q;a~?ceb5TNZVxl z8iOPsY|BQdMhO)YlxwkBrlII3cULpedYf(72HC#YVs=Q^VUsOD(r>n{MW{}RA4|~9 z51Y#a#eTUfc_F{h`u~M0qG@dngAs0W{ZeWgA%%Tp}drBV-Je=aM##} z0x~v71akDU-A5sVoNeM5lJ~KtPM{_ws*+Ga!M2}*68+r$rXf+u=DUQP|JZ6*(71|i z?gml}u-)dN79|+&p`w~C@c<r^45md*1DDTmpf-=H=n&NQKt zjxE=MQX|~yzM!C35ASo z85QK2WE&cU%qF(OAxJ&NCe=Ydl<3z(6*Jqr0m@8sw=e;PEo{X}$UDPkG(#3E+qnhO z%(4y4LV}GgJO@=NQE))nIkq(?6tQ#n>VkapZ1!cy>R`*SK-vYiF&`vxvYoC&HA+x! zLb*k@;1(2hareCqtuL{y1R<3T(1Pi;yG0*7FD%w%9hG zAi2oae1@9T-^(SS!ZzDO9ZKwQH~9{UyKLzX$Qfi)x1sSpwu26&2(fkjLM=*I?`r?4 zxX)JahLU0K=4H^%0o$D%az)t2`yt~YTk;>IjI#YxLv2dj(?F#owjV8&igD*1fr7_u zFQbq<&Ne#^nNHYljF2k9rk{d3lyEW)l~39FtWY}1-Hr_kow2p;kSE2qya1Wc*`5|5 zb(+oMhJGk9>wzj4Y{RQiCd1v)8Wg@{lldWUmMsu~ELUtFBBaT&E$%|XHCt&9s#2mU z3}tWFE+SCmmb<|t$d_k}97EPSHc0}~7T7kDkmR22Ed|vm!EpiQifn~TDEh$NL=IXn zv7OyOwnsMA9i%I>h3+Bg6Wi|tRHww%GIUd6t5u-bGk3NZ$X{i!& zA=xXN;uC66!tNK8ud{u9L-9B6Ji^zX0u8o`1aiEy&GbNqCfijnB>!O3DWE1L#*|Q@ z#nw9jB|f>^8id3)TWbh%ezCc9(0GUKQ4cA;*`~&z7A0gRpyCgk#snpQxjQsNJHqGx zw}kE)$kp|~1ph2#l(04CAZ0h(f&*$(;=L0pN!g4`P^yQ!)G`#5u?=`3cQ4z%4>HNw zgbhg5$L8CFI+XCb1(g+Sb30JFpSyez3MttP`;g}!+vx#hRfU9*glI;M9If0CNnCZ$lF63=DO%`}_34~kj13-?3*8MZDJWVf=dsUiI=TU`UmY;5*ns6h#L zBT#;hZOj10?c62Dp};(wauRYl*!HF%!vfp)G$ePjd96^B600_-u*fz$4<%gOT`xf5 z5}V!yIo)h=H#EM?*0%yFJZ#&mP>T}UYfy28&FzPhUhbX((9SB`^fu)3v1NB5;~JYb z1S!|qj>1rz5_=Az(gs`L2uk_6YaTZS*X0t zc94V8JKRZbq0lbd`W@s6vc26y<~_Fg5~L2X70S>LC5}Hsm3_9e7bp|v?q3}WAFzcQ zkT=5i(}XOCY^$G;Cd&5m1qnxNwjZcUi8sRcpRzHw(QYVm%-u;3Xi6l zhHf%!t{EtH$z5p{@@Ltm?2!G6?ZN@+b8MPLNOsK@S%Mmr(CvZpH*6bTD1OUb!v_WO zY>o}camRMQ2^k7(6CxzPXG`rsO-fYlL4_jQ{yvm=;O_ST5=(5pDCB%(tHq%4GTYn< zqxHdz4H;`}o;;*{WqU3_ZAzSZfJ$|? z+#{5F<4#wBf(^FgD&&4=>wSexO*ZijQhl(s-k}a9EVZC=i|w%ur9Zhde?y@*Tjm$? ze6bBle*R?cupLSv^*5We7y6+@e;-u&VSDd~GQZp{sGzXW{{NOxR72jb|0NoSAd7_U zd>GPnvkmAWLCO|3KvhZ=wu>XUkh5Z6Dj%93)Y&oz6ow zN>DnXTt8dT1x1zIeY>Ice{3r&kWIx_U4?W5Y}R#1s%E?PLv>0V*@A8c+2Y$!Ov9Z# z2>FNDwnLCz%k~+D^uuiKLrA7$D<44(N|=sA`4P6u1QgeEr#*uLqioT0$YEgX$v}oN zw#_UgA7^Xkpe7|cZ=u2j+ru4{Fmg9pgv3d z;xt>m0VOTm&3`~UGi-OCkju(8-hqs>Y{?%;X=D4>_4}tbCGK@Yr8%~r9w=q!&MSw4 z^K35)$nE%#gga_R_m8_hez1@32e9|UydT3QAxV-XNs=TXNs=U&Bq0e&k|YU9k|as? zZ~fQdYu>XCYrgZDCbqdj$h5?k8-kP$Hk}%3Q^KhRDlN11k3lIXcfoNeyu#L+fZQ&& zRU>3xWqUC}DmR;D2I^2^W)>>1v5nfGw1>N6I}}-G>s^98Ubet8WZ7W*T7lF)wq-XY zY_dIhpb90LHlU23?P?P`+~RIH0C@v!hazO%W|Qnf8j)>t50dP#ee6S3N?1IEvO%^& z6pHS0H+cg2LTneOkZq4mnSiumwr~=X?z8=+pc*BvUqiVFTlEHt9dKvMLHUv0V%eFKLwJ7mn3VO=18O>1gj=Q7<3gy{`<{;NS z+x|RcDzFKQkn(}e>wwyn@a}|4MYj1>DD}u)ehmsgvFW{#`lg)OuV zbttjD1C`5cYauB8%H3-iid5LF2axBDEqer6s%&F1Nd3-s5{HBuo9rB_P@;GNWj@%x zFQLOacPnYg`^i?yK-LDE`4-ZAv0dLmk|x_o0jg3$vAKl! z-B7QT&E5;uDdAoQ<$KsB2B3H^cZoqL(8s1wL5niBJvF57XZzJaGC7-P3~EqfWgIFD zu+15uGX;0IMo1iF(@jGTC0l$3G7PcxTOqlMZN~;RDWPSDioaj^aTLXu^+RYCGkRZ1vJpsbV4(gQ_TxV!0td@i<8Ib>U9izy(jo2_>U zlCH6B4?{Id{2GCB9yX^Iimh|^q=Wokw&@ATzQJ}i3F&-nY7^AE$#!Ul>Xgtu3+4T6 zn{!Zni@W+f6bP^_E<%gjY!41dFS1RpK(ZaS9;%o|fim_$WQ1XO3?F|%)vz^>Ru2Z(YJY+g!6AO^?oUK`e+LW;J43!dW&o5Bw zf;)2s3MbjpRmgqGHu3?PQ*1|{km`y}+Jrik=x;&gG}~tzN?&ug^an*UY{jnbfAZY0 z8KsaV%XZldsc+ea`XM3578!sllqe{n%pKc?3OdYl_pXM#_iXl2$XZ~_k3pITwh28X zDYBg#peiLOrl9O2TWA`JK5_Ru1NolW)~t}N#8$CE+7~wK0wgW7-7Y~jN*r@SxmUKh z3yM{^ldVDiH@2O1$X;dp-hg!PY%V|4TVpE)pgJYY>_GVsw(DIeUgu5|h60~#(FnBI zVCy-8^j~aS$B?Ya);NJ0l;}8v3g2vx3FxfF-P9!{{;;L4AV-@`m4OVuYzJ9L-eK#y zgPN4!yN8N@Y_$g{A+-OyB`iEa!LI)#-j^V!gw5~@8N1mMZ;(RDHdupNl(<)io_g3i z4Jg^mo#z`0^|8JEKrR{ETn94sv*iToKPlyGx^AdV38y_!X@IR?2Bj3-1?5n9kgcVF z+)B39A;>($_A(5qRBRRv)S<+T7Ag<3jgCWUHFw7oP-KLy*9dtuYylHw8D;x2LuxJC z@+>5bu|3T}6-qSOp^T30Y7sgd=Wciz^6J?RS0L*Io5T%i3~ZYoNHWRx;f1P{u(%0j zjckQ2C_2U6qzL&;Y!^YuHqEBogS2M0@IEA+Vf#CPYLvJhg>n|QY7C0aa%Ve*{8qNR zbI3l&HlBoZHny`A)H}~6zlQ3RuzLgL?QB1{P<(+qcOD8XvXu+a;u72JBcyk*-8@6G zWj1XYYEWXV0u`KWeeclO3U}KdkmzD-enO5_HfIwuxY?dtkbI48`WI?aLi!IXdf3#G zo_|WLb9W?#f?l@nKFGPj=I@7$KDPP*q}XIzQbH|Cd{9A8em3I>l-%MjISPdWY(qN8 zwavD#hfE@yFbOGl*t}Cvn-boqp;C}--U6j|xyxIjaEMJm54rc)&KDqam~FrTsrK1I zPN+kPZ5LFIu&u2@=>zUw*P+NEo7D$-j@Yt($P#56+lJJ~Y$rRA5Mz^tpb905VJLIL z_8ox^8Dg%{klk86-)tja)!gN{C)U*$Xyl8j2>l+sZ(`OSaD} zWJ|F*?jY?ITk#%}rrD;7P>m8VpP<||o9YFMWw?vHLjD`Jt}0~DvTf8L-7Q|aow67IjD{2kjw8;a+-OLU;XJ)5Gd_n#IEY{HM#GWPS>4Pn>3?=K_HCCX|C)=_cay8ft3I_%@_^bzvP z*yJV1+Rqj&LmD~TPX&?;u&usBRZ1v-Kv@Nwr2$0;xw~mXJ|)}e4`dr+i~T}c6+ z_fOJcwrvSiqr@*MlvA@g`=HnecTfG0U&A)7fb64eS4v2yWm6ABy<=>LBT$_Zy0uVV z$F`}1;^W-a^-w_1wm1nbPOv>pL3#t*q#2S;vL!80gA$cyzI|M1_ z*pO=`4mR-uQZBPKFQGOitfZlmlkGVJ zrB=8z-$G#*Tlx-iudbpRirKp?I7-%{mk~Ws7b= zi)U;-en@}LwiSS63ATm^H7L=s3l%Qd9`~TLBzIF0NW5fA9YT&2o9Y-cT(KRTK=L$O z*BR8L1YZIwUbEGbP$I+K!W9&}VY|PEoLM$Q7Bb$lC327=$2NEmwJ34#0eZS)>pViq zJa?WF6uM`7D?_dV+uR#udSJ`FL&_qXt`4;+;j{sj9@+Z8q0|$1!5=96%+~sa+$FYE zq5q%EFKjOoNL6OD^gta-%=AI!SGG|(l&)}htbihKY`sH}r^*%>hAi)FUn7vZ#ho9UHn;>t4?a&Nazt|+RkfzDDIR{C;**@l>DkUr~LfIBu z!2v~oxSL#od~LRiRmk?sru0DC4qMm@N&ndXd{B)N*SDaY(E0C{P~C=NUH?h61tGtL z?Jfk_yV=I~A)S=%>;UTRVUtIpIwkDJpnNaePaKN(apyjV0y4Jp1+>`DHk*R46!+%A%lwT`2~^>vrSi^CMBe+P*KgM z{(urA+#P*FK@D4X6LOBS`CE`t%T{kgiZQmOKd42C4_)$qdeX5OrBHI5yJRmE(z6Zq zL#_$7{Q<~iU=x&(a+1xfg4&euu7*lRw)s&gHN{5U4kq&wlOE9o@YC8L4uu4wgy!wQCx>I z3vAyT(BUF?D}Knk#8wJGRtKAT2huFFUGG8?C)-FEs!~EU0%cd&q(@NH#og91Jq9^;^h^TTW3>cpqQ7tNEY&Muyx%*b|2fuJ*3-Yt35!y zem46PRHuad5|rO!n|Oud0qzoSP+*%)QG*smw!J!}-(mZ0K(Zj4=NoEJV&w-a?6S>u zptBHnx5B_biF<6iZpaa4i}yf=eYSoXB#*G|$e|`Bv=mVBfXy`oB@VfJ8HR#KY%?0j z8D-08A>%QdW*kz)*p4Tl7A5u=p{EnJfC);*xoeoA&?(#UEaW<4dz^zz=WHfBq)f1- z7NIsJ3@<~a3$}w5D3#<+;)cSPY(5XA@dd6!X~6jvlX_W4ka2ysC>sdH>^F-W*$vzl(6kOA!j#R+XWe=Z050+h-O^k8rni2Sqe&#e2vz z%4RG=7A@Q56Qmwv8+w5R9b4oTs!*a(g)-x88#U-q&)s_+@=mbXzaXoDE&mN^CfO$1 zki^J#-hrx=py(R>r|cA4s2hr!xclvaeA8@eGRS6TtH>ek44ZWjl3LhqhoBlIj;W#C zEL&Uy#jM=P#vuP3+s-&-x3PUsK)QK0ml5ic^(+)$sf;I%WPZAkj%-}Sb-Xp=x{@Y6}CqYbmro2Y6B8i*;1R3!_B4&K!!E80}+yY z*t&M1CMEdxpyE1PZ68W_xm!4df*Wl2QON0IGn_!iO}4}-r0}y1CZHB2?j@n8Ew)Yy zN(Q*|TtlI4wznI|C9=)sAkz+8E)OY#Y`O=iO$n!uP-&N~zXYX1+y%=}c#o}Bf!txX z)py9e&-U^GsUmEa2GpU%OcN>}u#Nse=|k?0f1$_`Td$z}Cr^|uAb~8$Y+q7H9b;SW zgM<^dr+%nHi6#Y!f&vA$au`~CV4FRF^hLItBS`kh zrj0=jN{q#!!V_EHIdt~S-S!0}me`t?kmH5TnT8Byw&x5ae`TA#g_@L*zJrPtHgy3? zym5C_go0JJ?q|sP&gOrCj5W6UE2Q{fTdG1WN_?n6PjxoqCzSl;F8Kw88f-%?$o0jx z--b+0HsKFaezSSIhW@Ed3Gdxdsl_(m3#ERz%gdl}n@v9exqsQt2O)EZZ9oO7{@6ll zs6&Zu4OA9{|MP$U_x}lNV^F&5KZ&p7P(;FJH9($jwyY7dNZH1wA$1Si$qXd)vdOGa zg%U*@l<8ypwnK+9?pBr{Z$DdU8M4aR%q~bXz;^A1Bnq~Xb*M@S(G4g&$R_neQ6+a< z0mwJR_9;R(6`Nxh(hjo~_aLd7Z7KrQDDmy+LOt?rz^9ahgq6ha6_Mcmp!bu=RgKatqte57eZD)-P0?WpfFte@a-ndyzoF zIkuS|$Z2EC^g+gXHjNxo*x8O1P>T|KhM=bfw!koyT;#4X0)>{?mbH+}!S<+wOv`Mh z2}tQ=OHD#;N*Fdlr4_aVGn8_1Cz*x9t8BhG$n9qPn1{@3YzvE!%EMN0KpjdntU%>; zwu@CL?d5LJ14TC2!d}SZW9#@J%O>0U7NqvGy=_Cn7Mm>yRVXnRf-(WN@qOrUo4eBk z$SbnRqmXrnEf|9|LAIYbB-v$KJ%_54P`-e&AvQ}2itcfDlZJd@w$U5Nw$B#3g|rd2 z-aI5dVB0Q0HA?&{Lb*dW=Q9*L;_m4M@<-XGE0Fz|?WzjtVr=RUsP}~J@Dr+2LU$9& z$JsVpQ2dm;dK(Iyu`T{Vi|1?)UBmyRPq0l&A=w36vKMMlqOu<LCJgWv`#2gU^{U^t_QZhHON$C6W1Z-BU^I=YE#0BA1Xbu zJqMuFGk4}4C|qJo??UbuwvjMoF0&m)Ak{0I^a$!uqW>5wSJ*yJp!6GeOJ`7|%2rH3 zo_99mC1k0wU0y-z54NEUB-Gg=S*SvZ!X1?PWZSrh4jbIPKS16uHv1D~ZL;M{kmj3h z;uVs#*v{XeDkUgtQ1*u{REMH%?tUAP@0V@u8?trSDnF3+kImYFq(aw!?}A%F{ZBPY z9P5U1U2O3lC?@$&qD%()yV-W+kX_35t$=hrY_1`wx0kIn4Am)NMg!&h*siruT*jSd z918TaMJJ#|Ia`kr(hsn0nIM^htzm{5l<1g+3WIEqbI_TRyD2*)4zZ;cA%}`hwG0`C z*$!49xtguZ4K*pj=YfhNY&9>G&~Ue~2?a;l?zbSPmdzkS#xb@;5K`#a2KS&ACGPD* zPvdNz11PEI&J%?~6KroW$Yo%gJB3V>Y`JqtX=KwSp*AI)rl8UkTmLnbGI1BYfx^>l zty{=#W?Rie<{7q^0;ICASstMdC1#$X@+{kE8A@BZJFY;HIkw(+$YWy*d_b0Ywy#e} zZD(6(Bcl;tPj!$ z*>3!hY?n>D4K*k+wgVMHY<(f&I`FeHZAni0sc&*nUW3=y{HV@Q6$HXVnWl#o7y ziid3K3n+2K-O(i!jIwp7A?GohKLZ(KZ1pUpIAL45gIbjMa1TAj*^EUfdCFb#2@0LD z4ZT3FbGH3g$dq6cs*v)6&0B-ol<;1MN=df)FDP}%UH%&ir`Ysu$bH3j-hs?%wt+6q zKdG+SLfue@65BmcIm5OlgVHzLy~?3Tmd!c{d2ZRVLy#rMHl~KucWfsbNXWCv#-Iu% zisMk`p6z=AIxKLvVuZX8Y$X$9EwY(sAk8D&^(-WLVjHnRRZ583q3ko8bP0--uCt!o#u*Vs1pAl(OB zZ6E5bv)K=!Iwjmkq5LP?#0eB{aF;lR0$*&31hm*>+efHVblCTie9$kU#LZiJ;LZeJ@v5#Bv4YuT|)|m`q`HIAeWr&u^%!G zu$dH)Qo)u|LTySI9)?PTYzHGyO39r>3x$W+d^*UjV*AiT=3%ylNl2w;D@;KhN;H_E z@(A071xjnU8=Qk8qio@M$fITJEI^hqwsi-j*0H@gAz_@&whC1!F}DU~^lal^=x~C& zQy=6tu*tU|>m*xn8`2ores&sL>f{Sgx4V|rW_t$|$ zH=DOh`%jKFwrV$I@UYGILh^OCI~mlZ1pNS1^s=1|LWvFT22@bc$F{46oSSTI4P^AQ zxyK;I7TfDM)S|>$1N0PN%Nn8NHh0=-C?v9-%s{RkwmvIl3bKhdNV&__v_ox5SXqKf zA-3mbD7D9(*#(8eY-u;--e((Ghs+VSqYX%Pz$W!W9ZK{EpzQyZ0pIJ!i9DLDmFY{uR~ zZmY>8D!QD+Gq*EC7d2Wr60EbD3oe*7mPvSU$#~p za(CEP&mr?4+sg%{61xApC0J5WhY~YssNBUidIO~;|4BT)g(BT-y?MwZWeXG_OAp&u z5mNWEEk8p-AKTLlRG~yu1HaA?#NDJ6@~PM^dLi2|o3bC$s@cK=kaUFYZxE_c;<^gTY1pc2C^pKS zZ4~lr+3v<5`xx7}9@6R9&J0lRIGcP5s#C)5G?dq~{mek|3GUohC}3bK+n~isw%G+p zZ)Ce!f@D){S|`+?#Fz^znArN(ptEW2w$~xi%+}n195ZZAKV-15JqIB9EZg)B)TD&; zE>yI#sl!lWj=Q4>6tuB*A3@G}HvchXw6oPuAjJaP(izmE#D@g*w8&<>gpy0#C9j~6 zgKa1Sxt7`XvyjQjCfq^F6*liZ)TV^@2dLy?n}32*tK8*FP}t3;e}&v@Z0B!~*~2za zgH-Emp*qx|#C8KJd)d~$q4WlKuRl=4$7bz7o=vu_F#b;#KigP0q~2mX>4Ag*n@k2( zC{dI{nQgXj1#~EKw=x8Ici2k9kTu9=)S)$8240$R1DRh?N?)DrKuh?`+$dP7?ry#>MTmLmA&#>*>Kut<$-9p71Hdh`>WVw4OK*3wK znMcT(W6L~4#yd7m8B*lgjw?`$5_{gEr+c=*2b3&u*Z71&4{Xa#$W>%}Y(b_+Hq$Sp zd}2%eL2XJHmgxVf^vrf3g;FK%Bz;i$h0WIwxyx)H1CaTZZ9xgCDr^N6)S*Pf2vmM! zyBLMiRqh6LP~@F0tcN@`wvGX^e6X!gLFzi&+cYG6ve_(9g%WdCDAQmYpN9^=xI0~d zyiGQ_1G0Xz1)Y$l#rETZBtLAcYfzOE%Ii?J&1UgI(O>Rv{E)B1Ho6Vj{@7wWkXDfX zcT4CELDH`OBy5MF8YO;3pqzxwc?89}xqCW>{8F~*IArf(yE=n(y=>|WsJD;p@Di$1 zLU$U<%h)zEP`sbJdKL=E*%t4h#R0a5dq}Tfn=C@ILAK-*)SyJ=3sg|D?Y}~2L)`tn zL86MyTZ0_KY}GnsP_xZ{LGlr{yKksT3Hmlv)Ucg(pu{M516>pU6x6cqc0 z)Uml`kYb$eRSvZ%adr@T(z9iUpyUL1S~V0hu$^ci*Cbot7-TZCiQ|xRimf>TwJBl6 z2$f81&n74}&7FA$3Y*!|vygj+ZNvtdEo?`2NHxnQU4lB4=wF7)R<_R-C_TsBk{gQH z*oq#=GtXw+fGl>l%S}kVz%~?sghjT92vsOi*o87nY#V#fp@X~keaO4aWb zJ8aQkXfepvBN+Zkzst5Ifn*`Jh7@X0qN5Kg?6E!eLuX;`rWBC4&z4d`jtHA-7&08N z9gINoL$)q0)T9KT4k{k8)$~vz%H6^w6g+0TpMsn*HiH>5p0Fh>kRr}DI0v;Tac>@a zI%VrDK*=-iJPs&y&i3YnTnV0C`6nDXGD161% z+JW3@w$%`1zGiy~L#hm${9wpmdhI<2V$#W$QhMJUO<&1!TEn`?`eG zdA8*=B;2z-WuOWrnr@*?f$i!JI(*=6xBz*JY==e2`p71EhBQxXn=g>$neF2hs#3yY z70Q;_3NLR4*V_VvVT9o*(2R+Hyj1ef=&t38m z3dz}qjv?0o+x`h;Qm_eUkaCdCn}FJs@ScQ9O1AkcC^f`g{u&Ca*z{S*Jb zwuT|!4BO`jWV5h2w2*d|t*C>fR<@}Ls78sGlTdDsO=W^&Htr&3$Uo24H4EA8Y#Vcs zZh@^f5A`mx*%zTYCEPop{1V&53KVy6mso`Y%WMh{wCH5p^FsO+wqGA4bFq20pavyY zwxPl*+guPjb8~kag2Xj8-9F^-u*DA`!#Z1k6q0+{c4API5?XPnxWVQ+hY~*SUM`^E zCfiI3a{AdaX~?+6rn!L>0k-2?s6~lAdFW}IEl_}xB6p1<6xv~1eui8@w#OI9w996y zK*|tXstUC!VfX_o?XewvLa8u!k|q@1XY;incZBVu4Ve$v7XBdBAzPtq>YqB4XplnX zBesiPC>`Z)upf#XvxNsBPmHZI2w6_p)>V)?&i1B;{{4B|f8UX|QK&+RxiKhn#x|~p z4$rweH9+13n|ungUa$qHAx)C)X9kj7vaMR7DkYR{P&UP8S%9Kf+}$iezBJpY6S7^i z#axg!!`8b7NpIM;*P$9Eer-UxESu90#csKK3PAoG+w=}(zhk@Fg>-o~br|ZsXFH5Q zbxPkrhT#Mut?)L_dBrhiI)ai{HuLQS@l9?130)+d8ZEjCdODSz0S3aCv9 zD??DJ&GtMDrGB|HYoKt4Ev<#ze{3V;kXh*c@0M^h0jawFlOQ!h9ZK|@pt6MR(+s7% zxm%irB2u>E9OUU?Guk0bFWcoJr0!!IT80D}TVw^QP@>?5GW~2D9_Ucc-Mbg^4zSrb zA*+HdzXfRq*(O9tqGUS{LRCso>_OQfw$MHlRdM%w0QrX5)}oM2%~pv)+7UMEDJ0df z-JU}=N*qf$gX4izJ+w-Y_2@it7j_}pgJYYJVN;iw(Dmo zZs1N+h60mp(F(L^Wb1i{^iynGACSz%*7$@Pl;~(eg=x0O7IbFjZt52jXV_AIki){J zl1%@TVV3Pc3dyZ(U42lK5`6toagMDv03~ePEhwSjJlnkra@yGpBam@{Einoy7TE@M zP>T}x^w851TgLz;9o%`QpwKeg+ce~Ivdvi_(+XS83MpM|x_PKg38xECX_c+t0j1pB z1)WfMjjiQ^+#a^oHORcq_OcGCylfU9)S<+TA1ZIKjc!9}A9u$)P-K&>Hw1b7Y=JOj z*<$;OKoa5sDbd4p_+mymUrO_GK*A-2s7B-vy8 z$U;?0SiFO>VYb3O6y4`;vIzMiY!^?E?SM`B0%;G~!mp6@i0$tUs!`&44a!B?s&y!K z%$@BE^2gZjz9IVw+jtw&#o5j}Q12<5yvzJgbxPRnhVo}@KRrAlW6GRt+^MF{XhEDYm{b=yd3=giKFt!Y-tIX7lbrZAy6Ghe{>3`9mo6!d*TJh0ARE6UhC_c76(( zD{KP^NcF}RN;Y1LvYk9aLW50K zf-00KmZ8iS+jj*zY;w2q4tc-XN*|E5#b$0mnjg07CM0RIjr>4WN{IeK*`lX{Wuts4@S*lfK}g%WczDC1xo zAAk;*xjP+%yiPW`3bL-S1=Wzo#rC6tB&%$zV^Eb6%HvSh&1Nw`(KYUFjF8X6HaZR2 z*4bh+kk-rAYlWm6Y}+=dMu}f`DCc8yE@;;&YiXng%WHh4ajxD*7psWl5FA+q`YKn{z7d^ zSP^Fbsgz=SmO!a1?#w+09oW zv`{3+R@6bBJ2vA4WXZE#PD1K?wjmQF6xbqWs6vUtER=a*+n9q6i`>1>L*7R=`yyn0 zV#_-q%`@A?3M479ov%VwN>F&9>7eYt8BL+ zs78rn`%vzkEq(ySYTU`9kpF{iCkEN;Y~OK6_sQluhk6@qr3H{)#{U_nz6Ov2Vx|&dv5_~PF*v(dJLkTH&3x80shwZ-0`cKYYHiHy0_OT^;A%%=> zupeqs;@$xC)X&x#gpzXZJSr$O!1ktwTne_iQOGpNmK%eVN;aJyYE#0g0V)l#^-n=5 z6?egDC_K#8nt|MEwpA-+9$|a2K`ITKWdZ6?VrB^{kFt$Ap|qB}V;2+|W9wamJUX_( zI%FAV``UoidbVXhBuub91)vHgns%U!f$eG+I-KNgI1G7>Y=;rZI>jbAf;1+!&0|P1 z&GvBuRViWd49c3>3JEAW!`6{TtT*3Hf09VTG_%`NIJ*%mxF4QxPA}iY;4sB zC^pZX?FsVR+3re^eSvNK71AxToxMT5OKkEQRHua9I+S;?{WPHXGI#E8DBxr(|3Hf? zY_lCm?_#?V=Ke{x%BJmx8k898feLQ6J{fej#@)6Y5{&~a}GfUFWd7lB;R10 z)<8{4NNb^@k4-%eB{sP`nt+0Swr(Tj++y>aAY*{7ZiW=wY)i9HixMB^peK>dXor$J z+$9&GP>^kC8FKBi?XN(l5S!qJlzVJm57efFcP~^5v(0ZpseSJ9TTnQ{rWYai0o!>H zG9R)H>_Ms{w$MJ*p~Ut9RF1N(MWOUDcds!h5@WNTLY@=0>^WqKvyCMo^(os)3KGuP zWYqH#3&SaWK-y%#V58sJ*0nT`!zta5}RiV zYEWWj8Y;Z7%~_zcGIzIDNPK0}%|ng~TYLdBys`B=AbFK-#|bqlq2+>#?`*C$C{g3? zWgQBBu+8`&XPqtMhm4r5N06(<_IM1L ze%MTLNZDpfok49%7`}i?zibDWP^!b7Bn^fC*nAntE%g6)OZdn_=C1!FF5E#X30vVF z>QJJg2$j3pE}o#Yl)J$fDAL0geuX@}Y@IjA(#N)5gVZv%w>l*Bv)R6&3MJ;gp^Thu zybT==aCh2)yb3mX*ZekF#CPLOMO0+6MJb zupQc=Iwf>3L3sn)<}wtYC1JkZhVQxd}BWQ5k>=X10A1 zI-B9{F9?YiHt!zfm}RT(Lk26`{2?TtW4nt&O-j(8Kt&td*(sEm=WZYY1?_CRNyxdt z)=ojjMK<>}q*!8my@6VkIGcl>9BkPT2VM{L$rNE&6k zU4v?rIOc_N$82#Q6pL{u+k*TjY&+YKJXa~Z0Oilwu8*L2 zf;&wN3S6*7+xlU$SjoK(Z8D;}U96q9Y9zuGk(k&{>+hsar_AW=q{cjtrZs z02yxB4vLUG%hvS_H7UXO0u^uBYOhct$K65|3f{5Z*C1z}&F~2s@7WSxkfOjg*n(P= zxYvfB9@sh^C|Tsr)3xwVp+~m2Zpih-HrET8p4oCTNLgaj4M1&5I30vaFKqoPC{^Yz zsD{F?Y%LAsuCT3+LFPBMmvKl{WwRKd4kc!cQ2Cv0bQ((6xI3PKA|GtMR>)Il3)mpb zC)<}DQa9L^mmuMb?P(dRP@>5NWtwbPZs_ovyWw@n+hRN1fUG}k5vw|J@QU4k26De-f0(kXFJLK7pj&Y=5UvjS|-rP)^EL zO+v9A?rc|(znAUq8nXAXjb|a9jO{E3_4c#L@1Z&+>^?wwIor=86d&NuU4jA%wsIL- z9Auk)gY-(an|DYy#HOu74N8nPpn{66?;AQB=5G5364h+YU&t}S<`fqH$)I6-mO%1R zw&@~)hO|@3gtH0R3A{x&t2pb@^7(qH6eR| zZKDP0w%KZJs8?jO|3P(1xbIr}r~D4vgcOPgxl8myfn7F5KeQNP+Z%xNdu+dhkSxsR zQ9%t#tf-;FKHJ{wYWvW!ssCnv~F*fr`g$E-RFX zara__f+uV<3y?F;mRW*~r)(N0q&Q{)}J&e;O%P%^A^?OKNV0(Ljga=v{klV86ul2+$o|Q8rG<12HuX5v`^9!R z0o5s?+X&^GY?~%1{>@$83>$ z?*3LGQ5g8|mf-b3j;{YCR=tox!ZyDN$-CL^wxA{@=tZa~WjhN(i5~6-_Ml)d+wMN( z>|<*mKt>sxI|?cK*3UUpy^<6_IC7XByDTmma zw@{lBR`O6u#r9l)Qp4PtAEB_CE&U9+N7zQnkXgfaRDo2ZY|?kALy7(msH|oC{DjhD z+$}Ysh>oq;f;{7F#$U*yXS@7^)Dvt&lI4FA3~Ui8RG~zn56VokZS+HjM(*ARAnz2L zT?tuDY1QpiU9*?0j4|h{>NL*)2ok0#S zo9Y5GY_J_%LUJElR~l+kf-eIVH`!`gDBJpqWE*^e zT9ml=3O((xb>5(4kULKe3hlDJ)gf1iZSD&)?Xl&)A!V3N*M{1ZaN2=N`)vJP&VNcp zxC?ee;RCi-59B^%Ta`iPBeoYgq>8dx2B8imW`>~hG25sbO2@c6)0_iW=QkgmXXb_(@Au*nlpof3AFP`=3alY-)p+_|rzz!O{f23mY( zo6SM`65CB4lD)8LAD{*$#vY+UnXRt`oxO6mU53O8TeAW=-q@V)kfF-<`~k_|*`^y% zlM>QRs90lD|3HZk?v8$;V4ba7SotUCC!1da85?YMDWv#fTk3;al=#pOJvG^k3Ml!_ zT~Y~!T5LnZkn4wSe*`kM*#s@5{AKg%pf)AE>!DJIZGI9;{c)F{g2IC0zgvRd47t1h zlW=Z<%o4VNIY`yb7Mh1Tl-OQ?%2Kv92bAvN?$rrJdfBY2kf)C=y9QZgY-3(X-OqO7 zg9JI7YzwMTqPPuZ2H3uLphE?BD$I8W*SmB*p6qQ7A5vrp{HfGfDKAIxog;=&{5 z42fAb&phO~W_w+L^f|T#JEXW_yLCWcl%QLIN_n=#DwMqCuHOv>3T%5G$a%-s_Ckgt z+uAmyyk~pffm)O}7l6towpti5`-($tN?19A zDpj_}b141Hohc24YHXPdwD!U_at)bY*^X`?)f=0%0CgzQSA?o{wvQ5&dFO8F5ehfh zN>7mcgUwKb%uTk7S4jQIHuMe&Uu@wIs78sx7nJ>G+iXFR7I$xL$n(Qy`-3cPwnEq1 zKWTp1CZ&+1!*(WvUMWG@2j%|Qg8fib82JB|@H+r`yZ)EBu7a!*wwf9m>t?fzLQ*N) z%{Wx2#Bm*z?_rDUp_q(2`84F~W!s&BY;v~mSxDQ*=CVLC1zW`mH7H?r5i0bvT`fUz zC3hMpd>0L-1VN35p4h@?s z1nEcF4#SXQjIHYk`l1ByF;p67tDitgEq99vC@{fxmxP=;HvI);m}E;{LP|Z`Ko)9I z;(iV)PqB6KP|Coa`wj|Dv%TCyE+gA~88XhWVFUxI6BF!i#J&Ipnsp`4y0P ziS1JfsU2*~Ly)k{Rvw0GlxWmIStr}&7!+CIZg>LnxY#0-kY$xkVt_PjY+FW1vd;Ey zf?g@XJ_qI8Y{hvfy20I)4f1-}&h3zOlWlMr8uPM+Rv_sX+utfwr^F36l=rc{dZ5@g zch)V)=V!a!hHN`*6C$J)*%ARrw#%m2hZ>ZycK{UvY(Ixke2=@eDC7^aRb!BSpKb0G z(uLTr&mj2$+gJ)}QerF(6~k=3S5V@RyPa!DjIe#(K#n6eX93bj*&d6K;+Spb0s5kZ z%p+8av8k(2@`Ssi8Wf1Lb-zK*Q#Ri_WJs_zJ|N{8+tL@*qQv_aR8F!PexcMkcd0)p zm|`1}xc|v@!FC{pjA=HZ7aF`|^YlS&N_gvsDjBwgK`4F2T|otfvTV8$XziNqY!ou( z*!s1Q>V_?-gF2Mh)_D%S5Z#4x&ur2l6s>W$ z9fG_sY#(9B`pV`wg2vw1O2?42&Ndx~>Xdkqfb#Ebs&gpT;4XXt`99dXGLWswwwZ;r zpKSFUB>Q5s-9imYxVwW2-)xg5DBj{OS%&;SY|1Cd-e%i>hIGGdzb}xy!{)9-O-iga zpyD6f{3nzU2LHb$+Ei+3P>H(OuV#y=^fY`fji7bUcMpi&Q;OAaMv z+&w9vKrh?u0OXXjWrrX`ADc!EDHUwT8mL8yJ>yWhpUpo3rIg$?C!yc~+p+<24YEBL zA>$C6aTXd>v8CsrHYE&Op~^7Zp$$r_xsxnGp%FIkGPI^)dtZS}qil<7kZO#r=!QC! zsNaOD<80?!P)5t$fFBA^u!Tg(tz+v1AoC>K#y+Ihv%MTZ!W5e|0@WxnABD08wuuuc zGR@uTDdaJ-DUy(7hAohSG$yv6G$fg2TfKr_DWQ4|<;-m6JQSVd?z#YZEo`IrkaeCd z_5h7p*<=++y1=$mh3b^}S%dO6Hs>1@TjZ|%4*BeCGfl|0#CG`wX&r3pA4s;$7Wsu5 zl+Z1B{;A+(+mb->748~R$nRpa_d@nnw);Lvx5hT5gyidNsX?eoiG#yX(am-+0wp%M z`x}Ks51U5|IX2l|b&%f6wlD=Lw%Bf`p)X3%nV^!7En$X|+uZffLjgbA-U8&@VQVi! z29a&e0V#Lco}EyO66aQ-a)2$j4yE?E8}mTHAlr!-a_zJAZbQZpo45lF9I!*=!wy;6d*1?8^Tf^8_8zr9W9z$v(!$XHw}ikw z6zck4V(S4~ld!E;AX7KnQx#H4*~~9chZ3`IP_>6`v;k#g+#NTea4(ze8*2rn1}LOw$}wH zHo=|M4*7I!w+_fQ$u_Y9Y4vQ0RY*3)rf@?IO4##21q0iU7m81Fx3&%WjcnB&$Ueh1 z7l3pow(B4ypJf|6fSQyTJA{g6w%#a|nB#6I28kB7uQ=qGXLFuGdMn%GIiy%%n@K}o zl#t0lB^#Ui8cHs5cXR^<>}=fy$hpMkD?$bbTcZRim)Vvcp%x|HKS5WjpwQjB9Md7c{ue=4nA~N_cBS6*t?$AC%tUuF$phPazMRP71AUvYp8w zlb5Z(4^nNh1^b~6CAJ5ks*i151!cCmdsahXKbvI~a__L^#v!xFHm-xzyKEdW zr=c1piZf7lkL`OFiUhe^u|S@Ewu%+9gxE}rkmi8xY6+5r*+!huD=ft*FQtl;}6!#$g(8&a0oj(eaMCHBan@&lV+0j0{^HI-2Ck!^Vha#h$Kh9Tn< zn^6M|R@u^HP@58lC!oqR+uO0%H9m+Jg8(4$*=8OfTOZryBcxTZsjHBzpDj{@ z8kEre1{IWSTklYOfV;*AWGgLOPD~kb0SI=o}K9Y~c&2Mu|cO%C4|&W}%3SySE(VS!J``LY6hQ z!X2bpXPYcR5;xmf8G5Az&FOYYWZM_azy==7xG`7WN`GllCwwrIL zPKo1fD8J1X??5p>ck(X(KlygpcDo^)$oAa>X?NLNa!3|nt0bAt)Z? zPNRnW`)pASWDl|Rj6=Euw(SW>9%gG!LQP6^7@*=I+k+8GM7W!tg~TJa^c>`fvZ<_) z{+R911}S1}T}#jxC3u&i(g|CA1xm)bTU>(zr)+m_$eCc%Z$gGMw&WJ1OtKC5p%x|X zi%|KTtrLJ!Dem0+Q1F87d^V4NBNsfC??PpG7GC!`+$#^0(QlPRRbtHn$4t zI&9bLko=Er%mXzkG3JGe!tnpMgx+l^(e=N?ogGM&uzl@9j&3$*5YkK89z&3#hi&E% z`l5u)5mb_~sbf&Gm%F1l6p*uZpFz$(Hs3j9P_Q*FAZ0(>QU+>K;(Zn>E7=S;P-=j? z)GZVoWE(0%t|7LA5@b}d36Id=Fq`KIYE#17GgMKtExbbM5$+0gD5PQ2eL!oYY-gX4 zX^gGE1*yi_f^DcniR}(l)v~R3iT{+D;O@B_3hUS`GRQs2mXkwfJ==Icq@H3s8Gr-> zn_LCeC{a{H*=e?K4HPkQw=xcSX4ondkj2Di(nFeAwkrc9F|&=#K(CY#orQ99Y*Gsp zwQ#p>g}n1@A2!HpWpgY+V+(AhWk_mcn|48UO1xNu@{4S$4Jc;kF1!i(me{&{kj=ri z>4&t-Y;_TmIoWJ`P=gZg_MyTG+hiDuySPh6Apa_x@))wOvF)Egx^=eSQ%LS+b0?uD zCDu|-7&|sJ?-3zrTVORlG4%rTsP&&e$WC#i!v3ZA~wJ6*B2xK~DTO5N_F}9)>>QJJ7 z5~`lColilTICld^D16EmGC}SHTgMEU&)7ERA$5}NWdRb-*{pV`Mu~X`lufZstU!?q z?oL-BPnu2PhAfwC0S~0fu>E);$ran`HuOpf)g368Witn$=rwoOLCBk98$E!mH*B#( zXe`eri$c;{ww)MMr^L@VlrOM3&!E^Hcja@)S7e(>L$-Ui%M7F~v8k^i*#leT25L}3 zcL6Gt*|v&M{E@px3G!Fi?2nNBiS7Og(pA}}YLNVyE%gdDDRJ-}D%RKzKA^-4cYmLd z_{!#KL5?@J*EXcDvn~8Vig&i#uE0NiQG!kil^Sdb8I=6suD=foG}-q0A?GJsdjK+g zv8|~f0cp>)^(5=@IwNWzv`g4Vj(Mx2mI%68;}R6T6cb*Mv$z741lN?eQbtZ$gE(y*n`ylY(pVPP_l)?P>m9WBPct-ws{Oi2Dy7XfjmQOwghBR zu@#b#W|(dA0+OiN&Mu)>N>FB@+z4AR2Sqj9{pKO>DBJoSWF2Fx-9uyJY?d-4)w11G zpgJXvKSTKmw)hJa({U%SL%vD2-3DaSvwb%q?G&5q8!^)ct?%WO$Iq;#?kEJH0y z++TspD{P%rDCOeL?S_J@Y%dFx`UAOp3N763?;Tk7*am4EgeBEO1wXY%4Ie~97;WM zmr6jv3fs^*>HD zraD{y6Qp`)3qC^~N^HMC)dt&o9m;%g_uPQOO*YFX4nFb)uAKTRsBoRjb|1KC&L$8z&)j+u} zHt9GNmHaPZdjj%yvwciLRwe2G}Myq4*$o$t}n~#HRE^b`{&c2Rnv_^OfQln*^ARYa;qE31iKA@V6UZ^f7C(jb<7|CNNTFrhO+jCj&`Lw42{zXi zl+yRe-%m@xog&-;0)XH8{{&v zJ-kE4SvF%68Z@(|zo0fH4F5otIkv-JC~e_RB82`aG|%RhKxQ8^8`7L*XPn(br&a$U0Zsvu*AO;kgJS8QJz zs7(ng<4`5b_Ba8huemenp-_%3V}RCf*hXd`Q=aW;7E;}^Ni9%^5`9*vT44LIL76-5 zmX@G!k*%~0x$oHwF34PByI6zN4{SplkWgj|Z$dRn6ns$jk!{ltMJn9AiIC@s&9( zA%~PrrGWH3Y==rnA!F+rg1#uhI}DY2+3F)uQqJAt7!>GZyVF8W1)F{nGW4@0ry!-0 zZNLb%C~@Bel?T{5W+*ktoqHY%4zax~KrR*Ayd5$Qv*jJopqfp)0<|gObQP+Mu=Tm2 zw1&HY2MUd{wY<>U7~ASLWEy9C+JRJBHgf>#P+~R+RVUa+51@>WyW>MBJjo`DLT){q zKL(kn*goTs+Q7Da1_{$_<#VV;iN-XPHL_i1pvVk&!`G0<#1^@MEVFEq0;DmsZ51KO z9NT*ddZh&WBb2kS6`!E!Ja?Rp%NZ?KK&pe7~8^ia{m);kR)Ho4oGfkZFc*DU1NVsly`y^rnD3MsbP zW)`6@O2{lhB|n?m2_<*9J90q*k*#|ja_+MEHXuWQt+5Fy_t=(vP>T}p{ZKi`X4r*N z``o4WpkRn?CTrDQij}TY`F?#PO^jdVk=lo0KKa@TB9 zITX!tx2=G@H*6nD$eL$!3_)YJY^7mHT40;jKy^yI7=!Y6Y^n(;R^%=`3Hk2Xx(tx5 z#I|XKv=3}`6C^9M+2)`ICEU$Jg-5nY8x*f_m$XCvCpP6WWUsRAuRywIw%=7qUSo5+ zp(Z8PJW%n4ZGH<%ymEK54T*1TS`l*8+2R36|IXI84=EaKy9dx0CA1Eq(g&L>3MHG| zJ;k8FC)?~PT|KuAuS{oBtY0wYh8FK*3+OXiduY{sEbK*cQJam5i;} zf;yC_|Anf(Z0CPaM$X-U)4#@P;8RB@&@G7v(0!R+Z5ZS57HXg)H{%D znk}*mH7KDw2o;QMTOlYu!(Af``Auy0Bgj6>c7F`%%xqI}NIu7wNOCu{IQ3V>{_Uu64HFuIN7*-E3kvG`PX` z)dRIDVMPvAJZz5&D80#@X#fg&*)l`W+7{c08Z!CVjx>;Jn@u_nbtus{0ag8MACpjK zhr1;M6c*V^M##O(W|)P{0k(@dNWI54WQBwvTi6EGC{b8~vioeC%TOf5-P;P}IbgG` zL6$IE!3}8+*(Nt3Nrdfe3woslr60;2u?0ma8s+Xc0C|tu*7qT6jIDM6jh(PrB9JuB zb`yo_lsJ9@G?eG^;6xh0ix@FF3U_WT6ntWP(Lt^%+x!${d}hl}LxVLotqE#V!l@amys-7nL+MxU z0t-;+jjgo^t<~999gyjr?a2wL8f@lOs6&a_b*TEmHtK;gP414pQ23Kgwhg(z*!(+? z`J3%?7gD#_mV=P+!&VMKHA*xdLfJOkH1&d`vvq$3HA(>ldu)DP_&!7sT;^EWjnuxtUYXlMQBXM7Air~Ubeq7RHwv^Cnzsx zdwqstecV}JA)kWnwhr0)*(N?9t&%PA3CRZ76fLMh343j*Fv#}Pf#O5lt#!rz$**Fo zc0=}IwmBK3Q?p&mA^8Z~SU=RH#Ml5-)Ufrcpu{M5J8DQAWBbxTj&U~UIHcFIJx)N1 z3APzM^hF67160znsb`?%BzH%%P(aVtZGoIqY(6VwFt9ajkaC)BX$fjk;{7sIHnJI9 zP-=#|)EX2tu?=lNu35H&O~`0w6MWF%9Gk}vwJG6Egen%cg*_-e&s||33R&57VQ6iE z?JNSBY;66PUAC?v z$QEGR9EP-eZ1oXH7G$%HK@Cc{(?W%Pw#i8-9^x)J1^Exylt#!NX4^MGxUr;tC0SLtKtH6ZvHc7{k{7mB74%98RW+1*WiyXL(Kqg{$02W>ZBz$Y z-`Qe%Xsp2|n}(zxY&$bhof1E1p?s6gX@O#&+?B16?~83_5wd-=T`ob|7Mt1$$$r=( zE~r5X-Rn@H&9=1x#ecbLY(oAHo81T5|Jd&RkWLu?|CTVd3(33wmyp_nnv^&gf{GHh zgD{ln=I$>7iBdMtG34lBdp&{lGPZ>Tr08Y4O+sIkpu2!da<;@Jl8iK-lw$d=&{1f;_Wq zwmHaRW-H7?nmM*f8zix?o!OyRN>DCCxp}tW3KX?+_qz&t7ueR_kk!Uk^FU*ZY?duZ zYG=FIhU%0!E<*Vwws-)FIk=PWL%wCU-2=$xWcxmZv@2|`C?s>SRbo(s5@t`K!YbR< z85CdRPLqQC>uk|9WOuXmTtT`Gw(V<3?qO@*Kut(fH% zD0cxJ6gp;W>7lh4+v+rAI$?X7fmCrevl;48V%7pxPuWHnpiF|h<3%WZ#wK$>?j)Pv z37OB?K3$MH#kRZ-2^VbT4X8$mMlY02vt9b2$R&5fJCG;C7TJX?S8S3Xq{*^vg&@f_ z+j|&#r3CvCl*_RdkD=%dcT;i5n`b*uK-OEf!EFqSK?!@$P@%&1^8&@6xLd13{wiCw0ok9~ z<~|`^jqUmylE1KxwV@^@#yU{(m94ky?4J^E-0gHjVx8@)2Xef#IpvVP!S<+t6d!Ce z1JD;GWQL$plTEFLlAqiiX`sLtTlYBR{ATk_K!z4uV-iySuq_#&7A4*rp>mteFbk!A zxl7GK!4BJy6>|Nt9oQhFp#A@rAS^+HUH?n;EJJNdcw2!g61IgkDBaCn!3~9^Y`RTo zt%vPw3o^;r`u&iqmn|ql9ZGBmpsJj0eILs7arb-xg%xa;2;}Z(%S9oxl5PA1QV+15 zoI=7Nn>-2CC{av7*&(*?G!#*Bw{it}hS@6DkVVaA%0rqFwyOdp(Xfr&L$8z&eSmVK zY|;u89pi4h3VFxbK5CFv%jS54#wOTG?~qi-Hr<5klz8z4qU&c76+GMD7NPPLFype%QGbGvsquE8YSlIP&ULi z@c~5+xI6uXJYhCP3$h%t1=^4%!uHdFBu8wkUFZMwN(t3&C>Lcj%b@5nch_>r8)F;o zhpZ=Tu>oi-&L&eq(o?n_HB_g>PYsk$usO$}*co@_3CNdZo6$qIbGAzZq)oA@XCT=H zTVxh$P(rr_Dx}%AtWf-tyM_(&XV~mZko}77ei_nb*`{2O{F*Jb1~n;ha04pl*bX+K z#0__UTacJ%^Y|ghE!(RI=?iQNdywLe?RFpfq6A$SDizrh5h!`jUH>r@D6#FGK+XrY z_9o#Z{$eXhAon+$p$9Uz*e-e@^$**S0utJ6VI@?fL}3WZ{<3WjLy-=5ZzGWBkIgm) zS%iuIZwUo0r0M!!!sH|*k+7XjL9djcG(x#?^?ArDW2-GdW4&w^ zJ0z8}-8i5+C62E^`98MzDil+2CwD`>ezsi?WK*(zdm-%rn`;}A4YE~spavz(2B5+a z+f@*XtGLq~K>lI2=pkfRv-LzF-3Z%u43cZunsKN}iH2Y1JMD7?%jn}*y@ zHvbG{USa#3h14##WeX&%vX!k+jS`KEPVD~{eFI&+MMYp({+J(G6w(~v6y3IBig2w!8p)e%fVf%|fbxPbghVmlY z>j@Ow<<6Rbd;zxGBxKuTo4A0qLAJytB->|GWT6Hn?B$?Bi0vm2#Sge!yMz2;w(31( zKV+LLL%Imtbp?_iv5h@LO-hWtK*cCqZyib;bGOrg#2DLG6LOreIlm!&obB-kQk=5Q zbf7Ov$O!3wDka#|-B9w3yQ3Z`kYwwYL(X$Hp8_(Z*cwVmdBL_c1hpvfei$mJ*$f&e zb;(_73<_r0h9)4_72CliWX!S&259h_&0~bxl<;PPDmk`=IVgR@U11&y<=J#LXziBm z%nq3fZ2ilS>W(eA0(B^{y$V%}Z0l|)bI;wg2MU+iEL)KKfi1TUnagbBBBXv~I|)ES zg-yN>)hJOsfU-|)--l47%H2v7@;tLuVvwcAW;%s5FKkz5kmQwZBn7=vLNpEK-q@s9 zP_)k7_BG^vXZyH;tPM6t0UG;YD-|JWlWqC|s#D^{Bb5JSQ&pkZ7kA+rfMfMjhp+ZWWJgu510_+^{?h2kCVl7Eo@k4-7L{3pAh`~Q}(FNJhn|4aOp zL2?P3yANto|9-9s6}#Ey2cd+NyBifG_ONM3Acu@CJ__l3+4{7QLe93UgT5%CrH4v= zY_4f2so?Hu1`71E&6**nk}Ye23umKrI*on&8H#QYtU zHL*>UpvWwDr)9`vW>Y*tmN~Y-Go-Pw{k%YudA8L$^hycU29&e1nLnZE0(aNnkk`gG z+J>x)Y_SeBW@nRiW&TOJ#J1B7)hY3_2g*CxoN_3(%w1Ul`J8Ms1CVWn?Q#gxy4cie zNVdur(LfDK=pKg(YiwH+P<)-c#w6r-v)K)heS_`Z23KhL< z2R10N#oeDB5`Ao*WyrD3_PPS;{cH7cPYHp>(w zEwbHALv>0VH$nM(wzwIJmAI47L%s*L-37>2X8T@*w2y2q2PCVoRh&?R5@uJS!V}xo zIux&Rr}04kXSS#pve(#pwjtdM+x8A5e`Ra#LQP6^1fk*^+d~LS)VZ5Jgv58Y^bzD} zu&H8@{)6o>4k?;!U1!i2C3w%F(kEN}0!n^yx0rzf-)whT$k}4k-#~^Rw&X3OY_km% zp%x|Xm!R@5Tc-@AI^4ORpx__d%QNH>CjY-B%)dg$uKy+E>(HQtP5S}0DdF@Js&upU zwV zuV;G|q1Y66);-8)V7uLiY}0HLVMuFaOGF^q44dK@YEZ)72~;q#{hUJaS?<=7kl)N! zO+ofKwz*44XJNa(g5>jTV>zfviLpFXw6gWyL5T(KcJ3k3#`g69ITqQR6-aMqd#pl= zCAOIt=!+6EZ&1m>rfxvVW$uoeP{7I7{S7%+*nB^b!Nu12g_NsoOTzU(wJ7mk0+rX; z3_Vb4ox4;o6m+u^XmFFwGYqvU;cWz}c-a=lp!61Z1uYcvvFRqE zwQaVuDahn!>o-EG9k!qe>QG|a3{^$8^?4|>%iZ$=6b`Ui?2vnpE$4vDLALP~NWIT? zvI+?yHn|(BQKIO9vIlJ6UMLdgZe<(t9I{n*AWMYJ6o52GY*#@@5@j1XfLVZa?%z39SLBbkF8e zLCF$#PiiRez&1MyIm>L>amet7Up=^h3 z;sT2Nad&zNc?A9cw**BNvUL3~F_42a61JZ_Bex&nD+ zY@^SRwU;gS0*%SpWOYc|$F|de>Xi7|gz^eD=QkAV=dSz%`IKxk9mqDob}8KalXj3z z-3`fx*djepgA%&sP(j7ErGVnY+%=StU(IG8g6tz~_rs7*!#1UX;xa(hr0tU9d z705Zw)?S4SMz%FKq?}=U_CPI4oZEuRCbrx*l$zykOoW1Fwvzzlnq%wThm00B@c}*GANVUW!y@EQF=(~oh4z`aQ zD6`DnQUMA(*-Ayoy~1XAfXptoi$_Sk$~IJmgf+Hs4XRP1@CIeq**4#yh?~2&56H8@ zX8VFH9=1XY(rmI#{z4Kj+u0xVN(o9y{-1JNY(Xg$^>OzrgS^{p>wS>b&sOV)#&*~& zgOF5YyHP=PN*o`7^1E#DQ79JRPOgP~du+Qp$QES#)kCv=ptl~u=O|~-4WZi6Ou>Snl7kGiH>!sc+B>&0VQJGO?x5nge~oZ z9C0?)4x~S2JKTj73AU~v^hF8Y5L7y2tB0XvlDowtC~(epcMLgGZ2CB4xL`{rAZ40u z;2dgE;{F9xzGUlMLa7XQ?kp6%VtdI!t}NU9Eo8iA%ilqRIW}zxYE!~#8LHf{^*ur9 zJa>U-D0Iu#dV$soY^!z1bjSA8fK){`^C#4y#Oyazy=NP3LzxnH#~mpAz$WXu{U>*s z&EE}~AK5;8Aa#XpSq=$LY-I&hqeSBXl&!K|4ndJ;?uONnr^Xi1K$aIa$vC8WW!sv7 zByVi*lh7+A*bPvw&Q>%+(Rc2qW+88b?R*Zhey|N%p|K`g$OcJ2+5YTMof0>eq5K!y z>k1V6=FYkX`C4qZZpik-Hn9n5+iZy~NcPL7@Iwts*b|{bhwUc-#s9cl+lTzZ)c?1H z>H%c$`d`9a1ky>^uA`8=n{Dg_YEokC6e>#DdXrG1hr68=B+A&n(vYK<&3Ogsz@!J=?k#%1m+htb@V^Hp>*`o@UEULuMn}xCv6v zu$`D8!New?hia54E-7uh~`A*-Fu5roE;*h(Qt>R_8bgzA)daRlX;*;Fwo=HxCMhkPq+ zU1yNZ#kP44X;<0m7m#d?&6a^0lyH}Y3hQi>H&EQoUGf(4Z?Gwgkln+!UxIX-Y` zaT)4RqTU5n&)Ck_piGjxfek2p&KBB)+$pxs7G%C)+weo`G~0^^372fvJ*Y;B`F$vx zVVej;kt^;_BakP{rZ|Qy*KC0kNRwmxIfW!QY^zD=l@h8cD3@n5UqaDa?yj#OZ-H$z z2U+jfVtHt+$R@jkr1xw)_fVY@KOdlciOpGoVh`MvtB|kEHuD169@#G6AZ>+B-GF3I zY>_6^poH#ks8D6w`hnul+%~%v%37hDF z2D{n5yil7GR<@yvlknJJ^sfXBxt|39i7QTUMlqeLS>@eGA5sIj}dn-Yn z5jNW+WYMq{o*>OA+hh%rjIo`)La&sde1~%5Y{3sGs^#wY6Y@^5t+ybnj;+>)#wOV; ze~?tqcGGqLPjyNhmqPg|wzv$68Mu@8LB46W-G0bsWcwa~v@>ij6(lpURn$;}5@tuC z!Ytd>I21Q?r_n+FIku=CvRl}CrXk%t+x843x3V>7p(Z6dEKqTQ?ZFBqY}`#RLgFG@ zdI@sa*;GzQzr=Rvf)oz6u65{(61*EwX_>9Q2_>D}E&8Cq3fr9@a=O^`yO3d(Ex89N z*VqO^P>T}x!%%shtrLM#ZtmR2P;i6ov(G6iWXTIpl7*8}dZhB0Z4hh)p7gG*PxK z1td9UdsjlQlwco%axu2zFcdxEZb}1r<80?+koA;pZ~_`ju!SZe=^5MK6jZ0g4I`9K zvb~z1*g1FBImnk{yPbz@7i<$YNSkI$*df^^n_?MiP{Q5{RLHRXtU~cC?$+FpKg(A2 zK=y04xh+VSW4qplqQr+hR4lViJwmA$ z?ov-su);Q2hFq^~2NlRzWfR^Y)f=1l9coj;+Xqytu`M*A^gDNVEhtoH)Biy354O`^ z$kbr#mpuMS{mB-TLLEwM%b;?TZLJTw{NnDV9}2hFEGo$J&6ZU|<~G~dFr@imJ05|A z4x3yDRVY!^Lz!Q;F9Q@2rvJMotV~1RuKy&IW+01%&18nO-E3DDNFrq$UVy5U5M6|_ zJ#1136qRwe?Sy>2Y#%PjDra-7L8E^`(V4LK!Pi z1tFV?Z6gHf2H9$1NTz19{e$Y1aDN2d4Y5tcp}2;-WCHRJvnkIYyOwSL9MX@l{airu zQ8rHoYEWV&3+2by=5L{dj=P(CNE~O=6(EP6Enb8S6Ks9Ykix*W`vNs7q4f$CCfQsy zC^^O5a~%pyv(0`&P9t093!0i?)3za{iS4KZwJ5QttN5qlEStX@N}0K9^gzKmwq-fw zvamfWAmconaR5?T+0uhhn-YdJP-%hfPz$AP+)2iu&?1{}9CF*)-X|c_65HYwq;|07 zjZlXY4YN>rneA*2x^!|kV1>dfY#|%uaj|vmka?ADeHqfY*=v6j0!6pEyN*IWKikML zWZhwloj{`^n=A=QciDDQP>m8l(@-wJ=DdPpd)yVTp{*d>Ob)W`vt8Ulx)7V@0g@fC zMINC#C3KgdyD-~k8HyirSFb?+2%G&4vj1ayc!%^+w#f!0KVnNYp#~+YzM*`K?cfJW z9CP>k3yEm z)TV?L2UND0IhmxdyrK*@nH4DbM!L2dN*}q}x!361R4sa)Iq*7rK1p zZYcoqHqdjUf4Fyph$(gw{yt*%4WNS zELFC<45WQyo4A1_HMY}Rs7eXSJd}NB3l^Ygox7hRWyY;Cr$4y5~Gb9I&eB#D-4e9@kiY9ciO~VbE@A6YL;7yEZ4D%svNg0&gAyHMP`-!laU4p>xSKXWVlP{I z3UbKV)H9HwkL_?4QYhHEEKrjYd{(H?&sMWRNhNoSOHg2d?S2_@s@M!JXljrx>4ubQ zwt;o1MTz?xP;rQ@vk9d%+>F%zc1Z7IyLLeGO}5b$s6mObRVcs3*6V>1+uZGVA<@s)^g)gtHs>~E5ZRt~AjK}* zOaN+9e?O}S6#{IU11P!2-M>R95M=9)Le71*tr#>FVynj?Jkb@*aoj5*FUy{8^{=C6Ye0@5t}y;wW+^94WUwuZQ%(@A9Hv2429xs`U>Pe zVLPotrUYC6JET5k3w}TyN^E~Z`xb9a-3#1A&zIpk=t#V;VkCtF_zQZ(6ivrv-~S~;lj#pb$) zk}d9@AE3ZD+iVeXw%IZzXzGVe`vNIDY)7w9ixPWkQ1O?|Ux!k{%zw9pMgt0V{U>4h z3vx-=9={=DH=D5osibUa;pIe$SiP;{KT z>ut!VXB!bA>jYaY0F4^hWc!eGl5OVzs!`(SA(We9b4H=qGn?lgfLrp7phRA5QH*^Y#Sjc65;MG z40->t+5SP6DBIl;q&;Grh(nSX+i3!-QiAdf${w=?&!K3XyPpflcfz)ofvgF(N){SD zWwYEu(j?o>JyfH_u>zDkV~ZD|Sc*IOGqiQiw)+Cv(rjO^knV!bRfA-gY^6F>r-a#0 z=I;fzxYM>F|212*1KG1|JzcMV(%-OccSG_VTcZbRP@+Q)`Y_f`HOAj7`kk6cXR@Uzu9C- z$kS%?ry%nW+h-cmbl8@!AmNv-cnwu3(U^lWg6Y5K#)~^B()FLjp$Et-VT(LMmTopl z3DQd0Hp`HthwZ%rRVl!Vz`FAMV%iUxH^2yoGnvk`RP4x|pD%e6lkhGue_ZO;B z;<}{zr<{_lDurSL+*x~}Efw2sA7mS38&^U)HCsXj$%fbzLr|R(_J*N54cqq!6d&f! zt%Lkpwz3|wkFd>6Li$m*>uE?n#x`n#8k86_LwOxr?>v+k=Wb^K67_7&MaVJ1=5#;? z1KX1mQcSYVtU^snxLkt@Q*0V9l$_@7pAQNc*}AtO=M3A{4m4$AtM5X}S+=Dh)S|?P z5L7g?O&vn1Iqp*bprD0qFb28i*$(26(aI*ALaGHe?-|slgtv33WMf;ngwl)L-DRMV zolSoOxtG{ZZy}R|tv?T`m)U{^s6&bEB2;#=tvy4RE8M-jKw%e~r3!gg*|Igr>}DJL zfHZ4t$DfekVUxF@3MGndD6`J?)qx^j?pC_q{^Z?YD|JH_ADc-AX*b!f#{($5Zi_o(jBnXY>+I>W?O>llyJWc-5s(`xS)81yQCZP|6^0GL-r`!{syE! zV*A;IB5jB!4{7|hEuk_BS?{C+dYPwl+ZeX z3TJGtB$P~X_nd+P=WMeVkTcDexq_xH*tA(ldC7K^gIbi>a}O1-*!&MrD#Kml5eiO1Er>4Jdubo#YD&-Lv_=A$Okb{Rc8Vuq_I; zKdB3Bc?r~^L_-f$eq=lAg)WQS4Je@S6I)0Lc}i>@6=Z&9TOWcnWwzI0NO)nhjzSgc z?@J}2OoeTH0*btHcQOfit85A*WO-u?m>_M9?b{4V-q}{?p(-Vm7ocpN&1{FFAKYC# zAYX%RWCgN*vc*=R(I%VB14+NwcDzuH5+Rre%T_2P@NLGqtKl&``;~LGX}-G{*zFTLw*UH{S>lyvpt+adMVpv8j|<0r7occ zC91BWyo~MO21@jD_j?P8ayD-sa`drP3y?v0q*+W zp@52Q?*noUvb8^SFn6O;D5zySmO-u&w%$I- zILapWL#i>h<^a^DgcUVZ(y={hp!7I*rV%KlXS*DO+!JiWddOs8`)7dElWfvys6&Zc zGf;Vo?PC_YoaS!H0)>ri1uNv4VVhcn%qF(;B}g;NHt2)|Gh5gNRVYzdgEDh$8|zTS z!rj{j=^j+21Z4=yF0uu~P}I)dPXzKUv8^3JRtH<< z7#dw>vm_v?lkFx6)hKc79LlY*#V??ki#vG++FE7X%|bRe+gA?Kt+Bc8A(@A*^Z?Z< zVYUd}t+QQ~ptzSi?F;1JV2i#&b{|_$4bpG2ZPy|B7F(kMH7L>X1?9Kd9>1Z4pS$S} zB<`@Kh5DZyBAdD!GVHP)_CSgNTbCSaQi4wb753O_N+=oRZgCI_?6chuLCz4HK?_YC zuq8(!WteSX9BNVG{sdG!Wa~^qsR(x-BNY6{_G*G$QMUOx$aut-n}<{}Hk}P>Q^JWI zDjl=+Eko%zcYzfsbi&qJh1?0YRS#r3WqbBQ>Liv5kn(Ws19_02DrF zlkG#EG@JhbGGDNL9zvQ+w&f@!T(K2nP=ykWCr~ECc5w>XP^4@K{}n|y?PdA74B$ojyhDnp|MwonC zkiW`S?uYDeY;!6|Ut_ygL-KdF(P5}TiLntVUuWypL5UCUcJz?gU~3v6$0wU}8ZtE5 zo@OA$7u$>(YEr_b1uC@IGz(Djo4bFDP@v7$?SPy=Y+Fufs>4=yLCRmYr8THUi4W^g zQ852^OPKOOsjmMdrnaD_SEQnHpU^{EL&{?l9}0TQ&61}?v2ph9NWY!6t{4foP+%HY)UI+x3cZq zApHW{j~$ZR*gVTngAyw%P=1kZ-VG(}+}(H}afwa00XZCO@lD9E%+}|J6i&8X5o%IG zD*zQ%*j)Qi(#75L0Tften~gwDH(MqOO|7wMk0GUp?dSw*QDRRLDz3BnQ&7swT_X(z zH`tc1AeWEr@ftF2vKe!bYKtv>2em0-=m9EivmHJ{X+L+85)|5D^OYgD$o5`=OuKB0 zZ;(2`mVbvjlxS!`imp{$3paW?ZL6g}bY zdK&U2*hWl{^^`4UhDMWYvUx~)#bsCX%VrNk_8Yc`5TwtsO&&t>Tej3cs6mOU z7?i(bJBUMxd+vS{keFxloH{dp)* zV%sY~&S$oE5t=Hqxt}5B3){;J)S|?>DpahnWouCCmAlaoC|G4X{)Aj_Y`rbWSY!M9 z7nDD#-r1TRs7(ngUClq0>TFNlQ2K*AlMD(q*e>Oe`;%?BA2K!B{tZCtFE*(f>QLgA z1}eAMKD5x~H+M^8P`J%j7>7JRY*Plv++jPPf;7KugENpI%>DPeAUq3IC{eILnJ%^s zD-@CZC-Kb&dAr$cOOQp%cDD>^d)OvikVM9I>V~S6pj?Nty==h^C@SafXA|=Ev90+b ztAec}LZkg`mOV(SWV_jiYLqw@hH?XJ@dy-CaVI~5wg%aDk0G0y?dt^64Y9eBkW9l? zNn7x4RhS{#JptzPhZ5Hy6utjr_eUz=|9@3AoZ9hPA9b4lOYEYu11m(xs9?MWd z&)xJZBu=oU-yny9O#sDP6w%MY`O`k zO$jHHP-&5^&j_XM+yzWfXo;<5hTIOe)p^LY%=WwhshwYKwckPDXty+L$``lTdpsf(w?K5OMU>mPMx-eU!3ds)H6z@=-681izy9nF&Clvq3 zox273qip3iWItk?`-Su|w(G9epXA4Eqf)3ri7^?JkF)jmL5UOYcKRVP!PXpr9H(qf zHDpM#J!v4t8QaVV)TD&VF{qGY)99h(Id}gIP$13LJqkms2#n}p0| zwy|?a^TKv~0SOg0c?PObqL_s;uWVmAC{pEa25=cSEr@ zci|pr>xZpN4%s?v8wyDG%T`lDGQslSdknTgs7?v@L(p9p+k_U1Oa7CP9EJSdY|3%S zE@j)Ffb>0VKa-GL#^y0X4N9z-pnNae{2Y{!b9XZjiG6H38{|;1#qE%xpRI2hQYhJW zSD+>(v{s?Q0GrDLB~{!#d!fJ}+w3OfRI_Eap{XG@tq3VKY)1j8MTtH8P;r>ee*mSl z+%*oN;0W7t6mpHSJ;os87@P3~Qt8;zr%;;`hEh;zob50TrS;rNuAtBao9`NO8`$1& zAk!q<;vJ-(V$0{D4ka2Mq4G4_*%NeW^>r^50EZh1!q%pI-en7$; zo3#m5C^6T9G8VS+A1E@<-N`THwX!KB?LS!-*aA{WYh(MCL6Swb)jp_73FUq$YiBd7 zpy(2J*J{Y;U>g~Rtjlb%5opxOCeuOE6}BBcRHMXC1C(>IIj5o6DtE;hXv@tuV}@*N zY!?MqFdW3#V8_D!~jbx6O(HtB=p+ia;V zs6mOU9VqW-JJ^L1JKX*5L88d!4MC1wwrUtM1lSh-L5e-L+ast+3Hmrx2(l#-P;#HU z{xc{LV%s~1oCj>}3ur3L=FULML$;SJ)S|?>Tc{Xe%icq&f8320pkS2kxCpt9*m|EK zV~kCFfmFwA%~z;R2`e?I6lZ&?L+KOlOrKCF!FKrtxlh@K+mI>A_OAn}&)B40KY!{_ z;#N0QPO*LTK$qv-EyP^`qA+z)L%v+atIt<3fnfOIcxu6;;WVJjU# zbxN3xKzFZfS5YWl3y){}(v?`+#CNM2`aq@e~SIk zFI%z+DTVp}ZV3b5P>T}xf1qL)Tjv)_N&b`Qk#znP>}GqFLM|!Wd@p3|VaxSFDjAzj z3AHKVL33)GWD}P>mjw0%{&QpC^0h)l?T{HOwgr@ zyCX9c9%PfvLmoAoe*rQNv3)K=8V%dB0}_VWicY9PiN;kZqh-5TgCZl`4S6B&C|kq_ zS;p8T+mKerwz&gI#@XI?p(-WVgHTq_mJdPE3GOBjA)kTm>>p&EWK+eU(J8i29Fk76 z{U)FqC9a=AIU`&39E#0wXT5~BOl-Fq$TrJ1ego;uY>8V)HpiyOLv>2nD?oP^w(lYo zpXbi~4Ee2WG|-fXt*(WX>ugJ7 zP>T{D#-XB@ZOQYoK|vqe;0)y2WILFJj9YAi1yXIZd96^J65ec3$T-JvIlt%*s}YO zIm|W|hBSw4#}P<~u*r|03MGoiQ05=o*9jDfa<`I%yhm)M6l969nJyshG27J@B#E;P zXQ3)3L~~H~giU%6MHAd@KR~`ywvR{1nq+g7pwTn7LK%{#*rs2h8YP~;LAi4_bsdVO zxeGU-tqZoUFUWSuw($+=uGnfnkSxPy6NLZ$J^m=+UIN`+vrY6s@ho@AUdVsLrc^-o z9NWGU(%-WEs37?rn`a1WP-0~m%HOlik3xw&cQ-mnd|=Z}K#l@id=fG|vh^7uMUidS z1T`t4Wrhk*Y_54IS>o<_0SY{`&DtSnnJwdhre4^zE0D6ncC-q$D6z)_6<^u>UMN-N zuHl1%Z*0rkkgLY_xC0sA*^B{5RcA{Fp*AH99YCcIw!=dx-QZ3Vg+iZfz8K_gvc1P4 z(-+&~DWq<(<I2 zMj+b|+r=29)39mukZhPOVu0$D&^-;^Y1uYsp!f)P^;yV2%4WAf_A#~xE2P)4O)f(6 zakkVF)SyI_6Uyt^4qQ-Tg1cWgBpTSf>yTrTt-1jjrq~v?AjLG>tsiPqf_@h&7}*kg zP;!R5{ty%}vF(K+=PX-00!^9O+((ddj_u_bYEj}`0xDYAvPmd4&)w)b6tuD(UqG$} zw%!b6w6TdW3Y)YB zbtrMG4wYSO9}VbomAj=cDC}k{d_$f!wy6$e_OP7`l0RwI*#^5I!OIrzfhv?J$f3*z z+lB&)__%vhLf%a_+aP4wV!IoHwA*YGT1et&I~|3pl%O1kvO8?S2`DOZ_cICkcG=d9 zkTt+oF+rnyY?e7l8f3efhia5KW`lD3Y;ijj3vnl3hPDpac2^)zQ0 zk>GCn1QJi#(x;Fk$)-+0hBLOqG^9webzMPCO7LAng>$yr4U|lCw|EByF4*q#kn@tw z@CZ#^u_d1%Wrl5_47Dh6zXBDn**aAymF3R!4h3)6UOymLj%~gP8E@HgEl738ru%`~ zlyLG3mG0U4B;9{X=eY|=q0j?cO9r_MY^!~c>5=WZA5s_D%qplui5WFieqtLLhAvCo z9gRTYXEvD*@|4;9ddU33_Gy4L6}IJRNO)x{&OjAPG@7AImF>a;Mc%j@T7bMYw#Xu6 zd1sS2AZ?v((+NpF*xp@Gl@jc0P`1IAUx%Wf+)er*Uz6=@3$lK(sdk{z7F%c+l76%O z?m;z5Tn|CHHd{3e#eTT6{)4tUY_~^{?U!vl4(Wu2|85D11SISFPom-ss#C(=Idmss z`@VqU-Q2k|kYCDH&O-Jcwz*qKFJrsDhvdC%qXnoziLoM-m$UUgLy11_c3vP+!Pb0* z9Q|z08e~wiJ=Gz_0Nczb)TD&VFQ}km)3l-FAb0;dP(aPr-6j2#bBJxL8=BIv)q5c2 zFx!$GYEj~Y0xD|RrUszY2zRMLC^*VCsDWH#YzJD%sAChxAk{dVcN}U{!rKH?(z7j0 zLFozZ?u<~#z^0#t+>>mlbC79@t=|f%r`duws6&ZuJ5)BZtt~^BGu*wbKw%S`#SM98 z*|HwUY-StVfHZS#$D5E~VUzoz3MGmnl$mGy3P2GncPsmlcY&>R09kBorU;~6WV?z& z5lv`y} zKR_`zci~59YmKd|1lc@n8)Znh&Q_~HGB2C$4XRVZ{X2BG!8Xx=;y&(@O~}8=ru>HN zTWtG3kbax(=NFRu**ub-KQ$<^B8BohZ1cTPLgenI4-$9TbV|q(V2i6D!ya4T5TppQ z?G8gtN@$Hhg?%=c4oZf&d)7mN1Gd>o$QfqKOhZ$LY+4hfjIbSPoionp&}pbjM(4x#cn+u1+pGR@sU3<_Vch2oIslC6_~%vWsdXOJet_IeHp z*KF2Hs6vUk43x>Tjo(0#8}3eSA#aXNk%ug|Y=Hu#y<__>LXvy7)n}+m3FQ|kn`bju zq38p5*EPsjU>o^>tdDH5PiVBrCTl^`C$^n7RHMYt4wNggIlE+kiam2z?1r|=Y%?;* z_QG}{hjbM-O+O@iWs3|zbxP<~Lw8lSO$`)(!Mb zO+gJxRLwy72HU|bl=$TCcMcMpY+ftm_+qQtAVZ67VF^-vv)wL3O-j(aphBB1;f9hw z-1V5ZAw^4LM1udQwmD=ac8=KLJGFaE6Cl?Hk^e_O16JFNIk$N zy@xuKxb*;)Rcs%R(B&X^OC>0*W-F8-&k)f4X8qi!WWbo zVcYnIBBR{B{XpI^Hk;7yYQR3JHl$&9TPeL&hcXA`NHOsbZf^25CFEgZ@V{^?z zG7DR20jg8NtR1?WXS;GhaVvM)70AE97F~txHntuQq+ev)_Cj(yTf+x6DABPE<(JqV zcc6rWyXgQVF0-YBki*HQK7b4>Y=?)C!o}7Vg_@M$i$R4|wptuYy183Cg#v4A_h*pP z!)8cBQ|oNWOGxQu8@PsAl(>Hb6*t&Aw@}K*ohJ_kH`!hbkZX%={s}T}v*n&4m7h&l zf!dUCQiV!8Y<=%gTI4S90flziTAz?Rz_!|gOnYq4ZAcwtGyg&zO3ZZ0|5V;*8<9en zA?}W3Q22mN)(3gQZ2o@8e8~1W0BIs@%W6pY$5zxp6-qRYK$$4p#TXPh;%-O}d1Gu5 z17ta7lT1U}INRn7BspPwpM|QFV7EZo1Y6z;MNhe#T!ef{wzDP3dd8-5LZd0RkPDKY zv;DfE8YQl;L%B3tbpwiBaA)0uwl3Lj{gCa7ZG0EfW!MsXknEaG5rXQJuos5zvTWZG zD1O78`v~&q*viL{{g!Pm0qO79u9J}bo^A9TYEWYA0?OyvdNWYsfxDe7Bo^43Imq$I z=DddtMYg90Nb$rrQ-qq7a9M&1B{t0qlzisy-zyX-vvt=X=L_3b9h$1J)f$O?9Bu8+R$8?@z%R+h8~3dS^T6fsA!FK@O=t*t`m;O$l#GsMKIv7=+TF z+}#a9p(dMN3%S48PDdeAi>-egQh&1rC!h``wkM%-n{CYqUH)+QVuHdQHp?93`DM$_ zLuO&|zgxnX4bpV|C-K-02@*E>GE|{NaRthBvwf{X5h-^o9?09nR`Nm?8JlSn()O}l zZ9@_{+pq{#DIpqwvVCmQeJHBnZuwq#I_d-9R!eo9zy&Q^I{7x*K7ec!c7k+$EnN{}`LH z4B2&T`xQt(&h}G<DYp18WSC~_lPLbA zFtY7Rp(Z7?WKdy-&D94bP24^ALxEYgSrz0ovt`uK)Et|37*bl;jz*vsCHCl`;yjyQ z52dW!H4IR2fo*vja@p7(XCUJuo6!uZ>}+WZ)TV@?1*o*dcDM+o9o$JAP-vOW=Y-r& zws#j~T47sUgVZjz{5sU3M1v11udgs26mwE8e3==@_5)fdysjZZ9N2Oylk&w zNZ4Sr{(~x%m^*?pKDO~V6xrnNBmsH1*c4}wWt%N<4r%>t-xrW%hix?jRVksIg|Z@> z`4)=qa(8_X`2uVs1<1O`7Ar!dK{nYlB;9A*d4Xz_`1uOuLTt_&6g%LqSckU4Y%`yb z?U3!_3(`f{G;K)sk1f)H>Xgvk)&HlvDBET?6hGpw-UIn#Y<4+hKW2MSK>9e_Kx%(Z3#3Y+{9CDnoRVN@rifv&EQk=8h8lfg7=x3oqnk_L0 zB`>(^w?ctSwmloM4JzFyXwJBj`A1dY9o(`b&19zqf6e_S?Mj`hj+wd`DDzg1Mfz(fI z(j?TO#H|!mF0p;2q049Pmad?1nXPaQd0yD2a*(;gc76wGUfBj8Afd_@euOHND3qYg z8{0-1iqyD!t3cj&HrpFysk7a^L)s6vi3TKTu$?xcDkUhtq3kDH@CS-Ex%>Hrd|zy9 z66K$)Ew+jj8vSOo^g_}$+f5%-qr@>Kl>1?etDsniJNXc_^~<(94A}(xf479M5lGkd zp9Gf2~_5viAvo#i>1|>Qi zP`;1t(Fr9K+)b}SVn17Y4RR>i)LzIiz;@_^6e_l^ZKz2Jz8$DA$X45hl4|Z2gHT|I z?LGuKHEf1MXlj@(`43WR*#=@zixT(aP;rEP>mATtx(R*R<%K~HSVlS(3Xeob{Vp*vyHnTotG`)hGZLTiglpg-JyWH&@Lt=of zc>+21*qlko5M+BwL5h90nG2{%371z;A;hN1LdgT}{^g)Rn63LBavrj6JwQ_tw)!Ka z{KvLbf?AaLP=<<8wy9Srb;Moj4GPBC2J4XPnC+ke8RKli7o<92^L|5ZN_hK$N(r_F zLG`EfDR*}gD3oN=_dxD5w$on7lw#{wK?-18=HVM*KEg=kdS4Q8=(p%iY6#?!}eu{B026><{|GbTWJBZ+_9PLkoKPK z$^l98Y{M&1l@g+>Q1*dM>VcvK?zX*%Vr#bRC2cT7}TbOAw5*;V>>iJX$5zZX(-gs z=9_`sO1Ae|$TYyVXo1u!w!9VUP@-WGDi5-qEkT!R?gpGtc!(|Jf;<|wjvF!$v#qZ~ z8ZF!F1|*EIS+}4HCFcB4W|VDw7mAE=cd`e0b!>_dWEp1*gdwe-?K=WVCfHVwpeiMl zkD;uA&76RuliXb=A>S0+$T?)4W{X`wqeeDa29nOO?PQ@EC4T0hoQchO55;D=D?UJ5 zX119kWSe8VC_y?4o8|?Q&9g;bp*kgW*PuHq+h!e#FK}0HKz z2a+$br3Ce#8kDH&hVl-!gB~cc%-wG&-h#30Wh+tdkUjVsriwo*S-r-WG*ba%scrH0};?zF>@|CTK}0@?4_dUTNfo^4wX$@6Rt1Js~I$263G zV0)Z_5(Vz2&5-!WmbO5SBAa>vGCZ*zE<%bDTbBcBQi9J36`t8@E+|>%ZgC9?ys+J` zL(U4D!3RyfvL&}5WtDAU2WnB`{w`E}W9#fesTy~l5EOi8dksUbI@|m|$oRpQJAzaV zHeDQQQ^H9CDt)r`ok8g)cY$*#^u^Y?fZQ#%)eL0%W_!*;>NcDC7V1!9<{m2lu#FU; z%MN!(MJW8sCVPfFg5$qig8v0Fcl{^f^A*xa*p_S1--o#Wdp%LCLlsIienJ^3+r<|Y z>EUju4S8j3kq%_(Ws`Ji{-l+&ZFWPFKDPHBs7eWTIh0kfjs$vUiA?YC7?I(;P#|S?>OwKmjvbcM@{Wv2CTG zDGOUY4Jqf@mad={B|cn3MJwA>4oWR>m%4+3HnzbB$hF9J@CX_0Y(fcAEwOpaP@59o zDp1M6w(tg}m$|!pheA#^eFJi@u$?v`lZ&nY8&a>b1%IFpCANQ|vYTy9GW@5@HSS)d zP}sv}>4iM&Y}r1@>}4BMLYfV>V-+O$*yKY{g%ZVKD6`4-H3CJpxLeUd-fgy$9Ca6jYQ8Sd?Ws}ZB(ExYb3y^P*?PC$L2H6}AXmp>g;Dn?hw&_)< zMv3QZQ0{yWK$8?r^%Hg+K0KepN~B#W}yf>50j?nBVs5!=Ke6pwM2 z{0I4u*_1KJ9%tK+L;4f8p9CaNuzAj)1|?R`q5LV^{3Vn~a(9z~#4|SC4dh6%#cv_Q zIa^;IQl!~-3s931T1BXE!RC5~l9${)zd(U2w%ID=%&=u@(9|`X_5)I8*^WM;7A5wy zpyCaizYV2w+%-B-@Rn`4OZz9+9ou6!WV~lH${Ukh{qCu7ylbY>Q)%y2O?rhdPvKFhJ#JwzDbdvdrDU3>1E03(Z2F3R`CmGQYB| zTOm!A?bQYeZ*0~js6vUkWhhf)8+SpGckWKykhjjJScfbhY=I3(+hF_Nge0GAtA40T z31tz=HrdR3Q1pwt>wU=AVjBrV)^E001R8C#$&MiD58KW$RHMYt6DZeVb0(qKFL%Wh zv?VP6_uM~o0ol6#lX!6j=_G8LEF|k@i{zj>C3N3IcT%>^2PoddUHuXA%h>EC$llBL zP=@q!w#ipW-p7`DgBp~mszZ4N+d%_L^mF&yghVBq_ZxByuvLE`gNkiI82OW8knL6i zH7P;g0~OS4iC!o<#9hAv3TW8&l#p|nt*wHlv~2DnNIAmxG7PmSac&eUj_ONT;puLM#!jV6HSn6f~{$W+LW*|50wmTPYY0bk~@Q@^K63$kYHsC zA3_yM6rxaOfo&rOMQq%?#Ubw^o9z^`*xByRAng*{L>iJf*iJ8@DkUhdq3kkS@CJ%H zx%;_=d@F2gdC2Nws}!KoRW{2LBz3dhJVP}~9IHUNHMV#aig~z`ze8K=Y`Y(j&CB-n z3F$W2TrEiEV=J|xIwj2hLU)^NS6!ojif?hJl|ue)wx|rU``LQ>ApH*8c0VK+*%|{- zgAyHTD8I}0sDTmz?xsf|agQxM204OkYCUAwXFD`NiV$1ZG}NR7-waeZV5`kS$uM_| z7ASDYc5j885jMjjH1&@yxdbVrYy(cHMTz?^sCdNIaYLyXcb;`9c+B>?0lDIA^IMSd zge~WXR0%fSF4U%klRc<(%GMWx(n;uR{w!p^X8X)Rnk?J$JtW+)6(67qB^rxRCdYPBf+Dxv z4ZT3#JGRIxWVvUP)F5r1ZL--_7AqXIY{4NyPk*SpKPNxs6mM_JCtv-^)5q+FYb0$ zAhE^PT!kFpY)%hkXtO}FevLM=*sh(Sdu+tdk^>ftVR3I%0sgDJ?>%XW~4jB+;N3R3m4d9R^1 zCA{50B?a5U9hC0p?k*37lx+G($UVSz`UII&Z2e_OJ;)ZUKppDu=lGzqnr-bJx*X!} zJkj zAF}A#Oe#n_!FHvFBnGzOVW>*|J;6fRNj9kticWF2t%rQmY##>5YGiXvL!&cng&9a{ zVw*NYHA+0UK)G2q^#T+#a~EEOw&vKn9FWbzw&8?y^K3O2B(t*F)}T5i+^<7-3v3fU zC~o5}xdr(b*_1nw-OjeZ3+b2Ge)b@_gUu6y8kATGL-}R4`F~Kt$=%HnB(AXO;*i6| z7EeHiRkpq}Na1GNJ%^f<(7J#MYizCzl=N`-oP`4GY_qqJ)615*ho&~zv;|1%V>>EB zElTWphKieP{ud~<#a-hS3U0G4*C3al?XeCSci4=dkV<4re?e_Z7-~bMUADsxln!tw z=^Fo2Xpha;4Y`AC?>&%dpKVbNsY7ge1=OKL!vIu1U^^RxF2mdnXrS;RTSyCeB5a*e z$o!9OeH_w6*Be+k{5bY%)J2yDzB2_;{+>;HxV6}G(}$ob0F{)MKhY;KAEPs%s87b(=D#JOImSYylfL8*7{ zMwL*o&UUPVTpw(`Ly)n-CJsZYPqyX=)TV?L9aL(vJ?WwJ7k8#fDAZ!RoQB-rY{MqV z)MootZ@ZAUm(3Q0EONHH(Emxe!)|02FbczeeIy}Ck|aqI zk|arzWP~J1k|aq;LXsrCNsfK?$7|hdf1mZttbsKBY!io&M9y}01XU?P5rc9AY{m*3KcTlC5$9X@}V6GLTfoc9Vr_lsI++<%ikgw@^&YU0(t6kFf0&A)AKn zUkTEUvbkO$nU?MO6{=IhOclBtW4o$BaUFM>Pbe_X7X5rz2YO7N+n;tX3&10^lo zEsjCKS+@Ie$T`PmFhIt6wxkhKSlI@rp%x|X&p@RGw$3b+vT^6JLVJsBZ#KwfXPaMw zOiOI}Wk~5@)48BFC7imU=ViA3btvuRF1P`OR@hpbklV$!8i35JY%e0Dap$;Wx z_o4C{+h`cdc(^-`K;d;Z*)intvIR~c%Ld!mDWvwXEhiyildY73DwJrtgtC6N%PT0d z#ocfY@&?!z%uM7GTbNV3EB@d#BZ!Tt>8f^3B{6y4=+@(uFsv7Ns|)_pc* z9nyx_LJdfI!1mjOYLvMC4dugZ)gLH!$emR%{K+3-yOlt;BewA#NEc;G^g^;@Hn|+C zQ^KwSx{I-WE1~!ackW>*5N9ioK=xC%SuLbbuwCn*zB4xM1k|9!*d$a)vh|vv#5s4{ zW=Kr2H7$_ig3UP(8PaS|3y}PhZQ2esDIw#4iWxTb3Y5I!?r0SXX4$$ukn@_&?}dyx zwz>~e+^{WeK`lyr*oI1ZHe(P<-Ex=OgZA#&h7KUtJ=?({WGb);QAqi~=8ZvZN_dY$ z&qcO{GbsJY-Q779DzWL)ko$@4ECZRJ*#@p5)eGC+4b-8;_AOK{v#k}N%qw@VMJQZh zn|p#hZ)~|2$Wmn+t3c{^wv#F()Y$qypb905pHTLL?cWy^sdKl|g1nz>&uwU~!Djx2 zG+%62U6X&3G}%U^P?Zv*GAQ?tP1+AdTik68K)!Fb&q2uAW^Lvu@P!3jv?VEdkgB+G28 zCa6jYWiym>vRP)K=n8k&^N`QQHfn>ct86hlq;<2&mLcgH+x7}nqr|UODDPo&dZ5@k zcO@_6_p(iILbeUI%PmOfV^fQeY?Cb#gzA*gy${{_*)|WL_!f8dLnshnvqvHOHrqoC z(u-`9r%>MxTj~sIP@*yg6@qLBX(+MF-ERgG_t?DGkYk^%dIK3kYzudg{DAGY05vH= z{|FVsY>6i*dC1*B845<&b}Nwch^<|Pj8QiC2c$S=d;Nr3lsMajN-?%v3rd}Er~QHU z;%q0skn5DKS2Fb{Q-V#DLdr9?rVMIR!b(5%oMd|%fYRsOnUzo|#gIS17#VdfUP`(V4ehvIeaG({-z$rdd^_6A$e3#9*I+j@oinrw|Xs6mO2 z8dUhl_E?7!E$*hiAn}_m{SR`q*;H-F@WXc4f#e;wt}fG`nv~${hKj#zwH_!btp0yX zSnPv>UH?nGmqShon_&<#cC#giAcd4|Pz|*xabE+Kde}NzC?(_0GY;+bvb{|}u0FPT zBV_7l%bOsjoJ}_awJG6r7J43F>$gH_1$RLk6dGh}*&(-*ZFLzk53#+hKq?iR#SL{R zG3$ZK!)&7)P)5z&@g@`=VUqvMujJ!WdiW0IE=;DFS76Y?o0e zGS1!b3FOtYMNXl)2{uU*(iqq_Q;=km?IR6UDZzdP<&12FYbZL!-DDo}nb^+nAnP=n z@&VGC*+P$ybcXHs393=zdKt=F*s2vMHp`v$9rDky-F`r}dA9Keq_eUmnviUPP5uql zDPi{qy0fu;|3dLa?%a~;KLzY;WhrD|Vw>%S^bWS`eyDGmO{;(!lo(S&1t(kYFqBx~ zZhHh0U2M%!$g#@i)IkO}+mjxWudz)}LQP7@OhH8to7xN|*SR~gKtV5C_dMj>VDm3P zMju;!5mIcjEjgeTB|bQzlAq1E3Z=HVORYhB0k$D8H{fUSQjwf%Fe-KeJF@kw5xoez4d!KB}Nyyb;drU#5FE-O9q-?UKub?(14CkQde{6?&DBa>tau0>R z*?bR>yUq6T2$_G_7M~$ihpkYCI+SR5gUY{b=kHKPaQ}Zx7_39#uKy*38jwfA)@eeP zZnpJrNG)Z1`+mA5EKpv}=A4IO zBixl1Aisuf+78)9*)AQBPRpiVfn;NBkyWTp3Edv(PRF+Ch2rDf)qPMv&t~6(>=SGc z+mPPCHW`HaCfQPZP=gYc2T;Mtc5nzKrnvh(fqCgp|u{%`d1;2`eq=*~#|QhSDqCnSY^>i!IY-`ICEPxaU@=${k z9rsY-obB-eN~E}(Dna4}TlyJtq}f!jkl~W;@C}k@*t%*^lM;M&sCdOzYe2~?cZ>g^ z;5FO*H{{H*89I>hhAk<~{z;K%8|;Q!l(^pmm2TNOy-@0oJC7XNyJve-K&}GY{19Y% zV9O6f$|9Ri1GOpPR0};nvh|Nc=@NIr2`Kc$)|!Oe&uptE$o#_gVun;@Hp?v3p~UPw zRDNX}wLzH*cgJ=p{Kh6*hCEfazzSq}XZu=()HSwc41;>w$66B1w}r& z8x|pNgDnz-=Dye@`;ex|ws`y9${@!Oo3kG>sMwwcAo(!cv=VAkLPiA@)okh!C^^F2(I^zuuyyMo=O~+B z4;i&=bpxarV_TYnT9o)O4V83kMhlc0=Poq|?djQu79iII+rc7aGO!5_NIA*obwX`Q zcy~e1Mz)1DC_Tm9-8vL9vFUw~dz$Uc51Gwu1KW^lhHY;L>QG{P7b;uW)Z6Ew+h4 zC?4Q0IRpi^*%WHXF0$=wApH*8j~40+vU$d#1|?P|pu#TOyb((5ad%^a#CM%l6+$au`A*?<%=w&P8x zMTtEDsC2>>5TR6@yG9V&J7rtmhg=D^#{O9+91rl!AtnW~T67wHW_Kt150Y&b)J8eSV0-O9BntNai{y>@{+xIUdd1PCa zSpQU|gt8RMmDnu3Q1pqr>wd`h%r>fktS@XaC8RC0$%Y~6E8F%6RHMYNQ7B(wbLybj z8+Ro=tFvt`K=DuR>WfgI!De?r_Aj;v zC!}w(O|C+H|JYJ%P=gYcUZ~JwJMck?Z|;8mkl1GPZbObAw(1UK=&&v9LGoX=+Yr>G z1pOgY6g>am5)wyHvg>~d12HHlVcU&E&Th7L0y0Y3+~<&@hwb$OYEj~B1}e$ea#<+V z%boTH+UsLGxrJQ)Y`q1@Bxe(gkaB>nS%TV>u<`;uE7+c1q4Xek<|-6YvSn(Jdx&l1 z6EdsVj=mt(Fq^ambtuu_hRSNT&kmFs;clsG;ZI==Td^DRjItSJkVVUO(Fdu=*oFom zLB|#zgesIMsG#gP+lCs7=(&5>K;8*9+ZZ%wV7nWKG?Q!-21sILJ2OI6N>EHgxhb~2 z87OMv?q?SAO|z|8A*-3KVuQ3ZY;#MH)WUYN4Am%c%mw9V+2U>}HpgAxI^>^c+u49@ zRpo?}SA%(is`^*PxZ zr%;0u9Z9IL!uFVg5-#qhE+KK1Eqw(!+-#~GWLRT6%tLYyTh~3*qy*msR9t7PJwiz@ zcZ<(ZaD(l>3^{#lhBwH#$(DSF6n?hBI@F@X{RULpV(T=aRDe6rH?+6S_Vxq0M7DXs z_9xR0TV4VwgKWAUs7(o{z0mV6TfZDi?{ODYK%srMmJ)J@*j9%j^8wq-2&4+LS+r1x z609ziuqT#rHdEL$}W#jd%to*AAA}SiY)dMrMTrk;s8nY&jzXzV?owmWUW08&54pbB4h)c~$tFxe%71L$X{b#J z?=#SIi)~>JN`G^AXN5v-HvJ;x{$V>?g3KMZ0VkyTW!rN>9ZGDwp|Y_4|1Du{9m;h5 zFY$E)3QO4L{E(-cEf;_+Qns-jNZrGBvI_|^w!RQlp+qqZWqaBFMW9F@cPqz`x1a6# z1e%kxnG=v^fbA*?Nfc}&7f_WFqL)x^kWHF}qDtYvQ9y+$w)r6_VdCy)7!s%1bQ;KE zW{Ybf!wg&hI3%~Q?My&TN@z_&#aT9&2};g!_hN>E^K3J-kkiVRorjDIY#JM+u(2K6 zp%x|fEJLM5w!jLMvUAs1h4z-%mOYTm!S?8dOv`MhO-SivOK(AKN*ETQ=M}cYAe44- zC)tNWt8Bgl$n9qPIE2h=Y>QDyNEc&Mk3g~$w#X<{r-W`DbQfpa)I;%8?&=08kYKYZCw%c{6 zNeOx%RLrs^{7~|myMb*em}A@Bft)vN?On*2XLE-j#Vy-w7-~`C>=9JDW6K>wseA6U zacHl=c9MWx4{W{Xkg3QfUO>u6w&o?&ri7I&^judtyqUV?`*~|$Wmjw_y?&!*oNAWP-hEw zpb8}lU5-CxKiM|Ap-6+f_a4am#b)b+=9+AGa!B)!ZDJ6TwAjvupeiLO)KKo5ZBGM5 z+uZ$VA>R+%+BjtGuvI1??JwJ$5t0ht|DOwPOi+yy$7Y~>7h8N5ib?*L*k^_O-E2EH z$R=g`XNPn>Y_4TUCS!YEf$Een3Mu4d~>LQP8WrJ&*nTP+PG zHQX&;LBUbB`)kOlWi#X<;}~1=4pQjY1|OgnCGI~$rE#{-6O__(=P5&b6Kro4$Yo%g ze}_zyZ21pJX=Kwipf)9(HlgP!w*GG@ZQ?HY1BIs9TECFn%(g07{*!rz?L`WyENqru zs6&a_eyBXlHmZO!bKD&(q3}GLY#8!b*#aYwWr6K$6jIyRmUWP@$X3!r6-qQsLRmZ8 z(> z$DleT?8c$H0NZy0if?o0K8FG#TloUA@375gAbpVSIt%sfvT1Li1|`OBp~4w*7?i%??rt0krP=fb$bHFnW`xWcwt;C#b;Y(f19d2|JqwkyY-?61bIskW z4GQPj=9VDO4O?y*vgFyuT#)*f?ZgcUcWiy@P=yl34Jdoh_HPr46u4UnK;8$oXAzn! zvYB@w%_G~@J|rozjf9~pB}5}o?uku$3`L*0+d6@KFKnNukhRR_NJ83IwqgpBR@kO4 zp&BJ#TtWFaHdPLaRk;i2A^$sD*F9vbv28p+x(~M6BP6S{*`A>~CES;xyHB=>Hz?lV zF8K}xzStCX$lhezZ$SEgY(Gt?uf^v1h8mPu`GE@GZ1cj(pAv2EZX}TS!=~$j938fJ zFJ$;->z6}vVdMXI3_A*_NeL|_RP17N4MR!E{}NtCpkO!Kj23cA*|Ivw*u$oofD|&e z<4LGRi9IH$)XNqyL#aOQ8Ww1;pKW;_a>>~q7a-FBo5>C-6>Mn-)TV^t73g`8?Qj)J zE4h<+pwJMT&kMO#Y#%9%qiH0Cl9$`D*gEAWK1`nX{C|l?d@@UyQ zN04QVZ9N95b!=~ONEm0coI`J?ur16%@^!Xb zE7YU}{UTKKvL%+FyToL?R5icQR1v0Dh1eb0VuW2 zopuM>6WLC7A=eIDZwN94*~BoU++}MapdY?OU0t$uLGD*mNz&3IL znZs;Hmyqg^O`3%|l<3bv5ddo9IB26x*4w`ll)-D7vBC1>0T^ z6isvY(+l}7+1BKcHN#d>K-w#|xgkiJWxE-MYLqypf%4aEaV-?fao0Bv`ES^ECLmj$ z?cXG%yJd5kAlV(;vl*&W!ptmmch7b;55)`IX>3s7fh}r>>_xVoWk~_VNj-1mFL+1WOa@ zP-3R097c_q=d44 zY?mr1($C%S2;`NsMMj~y0XB&a(kR$A^^jzc?ZW_7DZxGk<&XX zCf|nYl(4%4-RarBccJ(MckU1rFtC-wkbRPE_6X7&*{+YFz9}|s9BNQvECCfvY`y1D zVw$_{3rIAxH7_B@44X3x87yp1IY>UsHhl{hp$?VoY{oArwZvWOAGGIS8)`$YWwwJ3WOA|zU2A_*uCRH# zp*AJF_dw4swuL?@y~^F4916MF^n;LljqPj*GJDtt)R1bOZBGMrD6y@D%3ij$aVWFF z-RlGt_OZ~)aZb-AscIAO2du$^cP?Zv*n^10_O&WlrA?~(B$alc@8HB81Hpf1sJ!C5$K+*`? zR0OI~;zbn7AF-)Upjec<@G0a!X6s5qwiw$+3eugh)zXkG&StxU>XdMQ4c(oxP2{0? zg1h7$6gXp3JV5p&+x{b@KWF=Sg8EWyo-)*+#7YG!T(Hf*Ly0tZHy@CA$);;SjtpD8 z2^p@~`oAH0mTl(;YEnY$7b;$}xg?%HC3D=pNTJ{j+e|Oy%(G?tA>%EZMgb}A*p8J@ zixPW=q0&8DU<67PxND3;dk<{OI>=RId(=awM>f+Wq%5(er=T_^44a|nC$>Wilz!$; zG7p7b*nA6+yUg~n2$^5m79EhP!d7rX9ZEE;LghEM^ED__-Ztd z2iy8Kq^`5Q?LfjOn{^MWP+~p=WgBechfw5;yVE1c+hmi+pt*l+!8oL8v3(~X$v4~T zIaH;D@&%M@vsp4w^oP6aEadC3jov`kU$)pSq!l**za_{DkhJT63EM@eMu}e~C@*1i zzCf{V?n2Gtb>p5(-<{iYdsmz-GLJEH<`_D@eV_Hk5+| zJ6kvpRVeZA*Uo>+F0pMqKoJLb?~jmona%bL%{kfb%8+J-ZQ>1*xY*9#p(-UP>QHW# zZLa}E-Q4{&A>SI?+BanNuvLB_?K<0>;Qf=-%XTAyYLqzE1LZf^;=NGJ$6cQs@^7;3 zC?K1k?Vl3TZLztAAz6U!c?7Cc!i*NW+h)7cL2;2g%>)$KVT(>e_8?o23DWPfZJD9I zJ+_7gYEYtM9xCj!JuX0r5O-5{NIYOmJ0M4xO|=3U4%rS@A$f$Y%L6qj!RLjFM{G48 zl#Ft>xCI4|+3vR?XN=7dgp4O_$vsFBXB#|#T9mkd2$fFRI!90{!JQ`t?VYi`#UWRc zZT<{0owMc7A!Uk9mxkJuaGHUhFWCC8p>&$N;0+YIWNY0*?hM;%0Wx2)y%Zr;md)}6 zbtp0W0+p}XMk`Pz$K7!i3g58FJ|It?E$|6hZrQ%RAoU&Fatjjf*-CAwLW!ndC|h8= z?ArKK?(V*!kcv&;f!xDvXM*ofW;NSDH>4V2+v|Zkl-TZt${Myc zIg}aY?o|PWwQO@kkY|i7Hw;;HY-1WoJaP?Zv*b|^Q^CS8W2X709DAm0qz=PG2iusJ-Cc9yN^ zg`{(AQ=3qY5-+x({5+dVgko0i!a>Ntz}B@7*=%eZ2as-&t#$~>>}<9uRHuad7<9M9 zHgO8Y9o!|)pujSlA_duvQtq*(RZyD}hDV_1eYV3< zC>`QXqJu&QY(72Y4zqn2AoC&H;uNHcuob4E4ka2aQ2B`Md=AP)xf@)7!pCf(MaUCl z>nuT*6Sj3Hq>i(_xgg<`&AJ9vC^5edWfN@UJ}7d=-Kih)CfVfM(A+s&a0k+)*uHlm z$pzbL2&z&-ISl2}Y?dP^ddc1OG33jzjm9DC6K^LxpXj50byJ z-O8aRCFlpCVwEj11SQ|O8&E^R8r!Z0a(=M2wUDvS<{pO>pKPxaP>T|0jZmq|fMhc3^xqDAT-T^k-6*Q+{ySs)ogKQIdNTOsryMwBfpm>0C zLu`AGP*la;&lBVuW?L&mRyA9t0%=Fs=H4NxhVAA9s!`%t1ImxG#hXw}%U$0$Xb0k3*AkyUG+n819ut)6qsa-Dj~a(t!Eh0PqA%{ zKz$~*#wgUFM28M4OtU@ep@f;csYytjVM|Xz4hx&g3>jwG4lR&;j;(7RYEpu40V>Y3 z)fSyJTcH+R7}6k20zB_OwlZS@>7ud}^eKq@br zB?EORF`I?T8*HODP{zmI@hueIWRn#jkDo12ge+TZUnNK#U|W8Hgl)FcD^#IGQx(dJ zY?n1Cvcui*C*%#XMZTc9T{cMz((JKqwjs$r+eZhgQi8oJ@TXjet*KE_fP?Hie`%p2*rVc~N8}5!GP%zKdeGEBo z+59Jv@s6#23MuZ{mXc755+71fslaBugi;UOrLLg8BHK_7ay_yg*VyKS?LT=w*m4rcQfC|Mfz+RDC%ur+VC$1Z6-pEpQ1*-MpAw2Rxmy{Ay#Lsq zN1(YDn^_BKzS*vHkfhBvG67X7Avy`=e%PcYDB9s}%MAH`**-0hRS5ilOK{9X+OGd4 z78f9?gl)G@)I;F~wvYkx z*w{Ko$g;?`J`JhuY;QA=u*7DagDRAmw?bJ5+xQ|BS?2C^3GzDGpw$Y#HX z?7M6a50HM3ZL$RQ?X#txp#~)?U!g*X?cfbc9B}vh4vAqlZyjRROgqadrqQCD?Mq zQ0k03tp?glvYlul*Ew77IAlt(i4&0Wf~`3TwJBl61U;wOp3G4Ck~{M(6w0t=<{|eL z+lUP^XW5SIkm{OEx(szF(Z2$fb8MfhQ09iaB@YzNvlYFN=a$X530dygF18@`J=>57 z2?e%r5UNn3un%P)*ftKJNRhkuL&*EcW{W~|CAPa5q7XVh`1DY* zo2_Pml2Yy#r=VaD+x;};l(88skg=C7IR`2F*ajD%7A5X4LZyDT&JvW8bLVkFdjo85 zF36=|n_q)WgKYVANU3Dg`JguScS3}ohuHeJp|py-;0_cTW^3(2ZZ+F#2r`eby@Vl^ zhRt#Wbto}=43$UOM&nRM%iVDT3XieL&LNMEEpP!@#@W6uA+?@uISUCBY^5Ajp+wUy zlr^wj-b0Z|?uLty*T@zrL32}Vk{3v0V%vO$B-3mkZ%~yI>@_H7W-HX8=nQw0Uy#qj zcK#2t&ax@nkamtO)Pbb)Y`?I)PlfYzL>1X^%}vLdtzM zZwhKt!h0He4zVpS zB+ao+O+qzFyqJRWH*6|16w7lLwm|+{wyt@|cE`4{0O{`8YKxGpz-DtmbxOE*LU#{r z6RS|X$X#*`3OuqYypX-bw(o=VPi#MasPCE0vkf&Uv9bddUfAaMphTIwn-C?uH{FSbAtN;SD_l%TzTY|Af@tHt*C3YosyOjStPW=q$gHYE&yLeD>JhhI>- z!=0oBg?`z5ZOAQ%|KAcmI*_^Re~F7-`+rhN*b3cHhY}4ksNBtV-Unr*+zk#u;U2co zAmov;b%r2IFWb5rQundFX&|AW%{m5EC^0_{W#w$+1}HMX-Ki1sD%j-H(A*$fa0b#S z*}i8X$q?JB6{=D~*#_lQY?dV`I?UbmGUQXUjk+M~2wThzX*F!Jbx1nOw!Hz>DDi6( z%4^x20Vp=cT}g!eI=1Ou$TrS)xew{|Z0ay1n_!DXpgJXVA47Krw#^eLKFMAE6bcyG z>`BNz#rBYb^d`2+OQ>&}Ep-JoC{dY%3TC#0Jd~K>?)MH7Eo|Ng$T7=SeS{2iYzxnj ze4g#L3^gf1{{|JUY>9U$xxn2(9SYjmb{mj$k*(c?jCMBnH>6l%d;Ni0lsGGd{#0_X z4jV?Y`t>G%i52{jv;sDAW zu7=|txu>g z&DQvW8kFd0L4`}U$2OG6a5wb}iC1jtu7f{0vTQ0TWVmKKltJuU z4MNF0cZ(`0c*}OLhMae7hEd3P&z2m66a}_HJ=CJaeFIc_VCxv6RFON+G_?1~_BI2# zN^J9Ukm-pnZ-tc4Y`R6LO$nz<(DMshzY|KAxeL0W&?{TZ4Y@09tLu>YjqPOvQdQY3 zeyBr<*#K02XB*vtGBxgwccJhHn=Ax*>TH29Wcg(Kia_cH+ww6ae6f{Ipb90L5>U3u zcA11C|F|2zfV?fX$R#xQ%_hl0nl{^J4wC$^edM7kCD`wwT!*dj07ZYfn=C;-Vdwu_ z!uc~~?fPGW@)gob*g|iRw43es9jZ~{dL7D3*{TgF*2A6kALN&@-F`#1UbgWLr0Zi# z2;o1;`q|{&P@NKXd!Rcx+jlP%AK=a{hXM+=vI4RXvds=bdL`TSFw{51rqw_VN{nft zf{Lwo97+sxw><%gYPRMiKjcW859tp`ebf zdl_<$v-wvbqn@q43MnSomON045+A%!$-rjZgi@2-rM93wBioP&xu)0-f{@9?ChSAX zX*TZx)TV^@L+IJewh)EVGu++9ppb=4e+s#0+0M=&^Bmhi3R2Cp?WLg(CAKqA*~+$d z4P_R%d%c0eHnzDt$g{|nD?k=I+t?$dUSd0Wf&>R!Um2=UqF8~l%WVIuP{hgI$_M0K zVSE0B=3H#%CZt(qyJ|raH`~Y$RHcOIFO*wjlS&T%6!mboC53$JY@ag7>Sc5EL)s0t z;s7M|u}vwV8YNz+p!_DAY6Ob;xeJd%{w=mH9b^lzZRjE0He1aA$wW5W6jY~#`)TNI zhi$?F#e>`>=b*qYn_>a7@3HMKLi&BSpCza-#O85A4N9!Ipuz#${2G)9b9b{2iHB@D zALNLz#r=@sh^>Dcl1JHgcAzFDw05E5F`Fv{C1c#ZgrVRG+sqN%2VCJreQ zY{v)~+%+zty>qtZEaXbDJ?0?O1)J#>Ql{C`_fVS>hKtbiCEH;M zN@uu}yg;EVHs34c&a!>HLFQ|=#Tul_u@&l2hY}56Q2B=K{2!Fbb2r$A!nbUp4&=FG z>j;rQS?<}^yCHRf?X3qA9@wmXP=ylnawuD58y|!skKCOOLEaLZTn)`Vu?01d=9%qV z3rSwsR>z?#C6p(iT$#;cgrcw9U7H|Zg>7^Ovc9p!W+82rO=g9p?`+#Ps78rjb|_zC zb1p-%5AI4UkiX70?S^chY?mHL*I-j`K(a5k$R<>$gzf-z*JRriq4+=U>Om;bVzci< z_HVX_14!Ryn~XqxKWwQe)SyJ=2~_B?9h^dmU+#X-AW;bZe@pPDAV=5#602#*AYog$ zg5=$7x7SdU67+efC}m6BLCGHO1|FcGjBWQ3a`v*dpCDr&o4X7t`q^G9P>T|0-=UJ6 zE%yPX2DsBUpgjfKNfUAnvh{vLCMBEr11X2tn!iw+5>_Nfe|lE2JxQVTFn8u&D5Pe~ z^h53uwh;wn*03EZA=N0GbQtPTqJIP`YuP?Wq0AU}OFAg5V=L+*&p4ZL60+#oE~X&$ z1ly1q5)5o%3sj**VIIm(vTZCt5hHi+i;#DU&E|mSOl)^fNHfhgu?k7dY-ejwl@b(Q zC^y5l=Yyga?tc7`ZXa~(f$o;suCh?v$(`l~3aqe2Zy~#jt)~F#SJ}3T zP@kKvQGyzj=y-t&Yiy6NP{PCAR234}+0r%0;bl{OLWT{t!!JngW9w=`O-k^!q2eZ6 ztpg?f+%0xR{}kL}yYGga0XBmSGH$aa`yhqLHaGyaC~#!p~UPmR6b@KbwQaJcgJohe8MJMhdgn%zy@SFW&7HM)Csob03@8Tl|-mQiKbmB zn`FD(ha%_P4Tm9biY*d><}TPI$B-t?ws`_cF4;a#p(-WVlTa?hR!Bk7EAA#QAzzm5 z{0g#Ovng|sHpdpqL(&_z-#e&AiR%wgKF?NtgkrbcS)U>Q9ouagvfZm?YjxZAGvdXLxB=o`3JH;vCRs{f6_m*T}z<87dCAV)S$#zFH|VA z^~#~dD|g!pNUX3mm5}3&%{dGis%%dqko=u(S_?HPA)|weH8%AGl>FfCXc7w6*}6@T z^OMbQhKvoix&=~vu`SI*ElPY?fJ#j^qa8~9<1XcZ_F8O1E0F7(?O+u$wb=v@r2Jv? zdZ9KYy!)W%4%@;Ol>X)JZW{^-yZ_%3^g+no^}mF(J;*F!8#sVe-E4b@P=^xRM^IVH zwibgjJ>0#NB%>rlAHUR%W2W4BPx1l(2AjV}-<7 zHr*oRm}84CL5BJNNU+0hjl7u8A$&w^Vk|arzBuPS&B>S1~ z{CJFW%#Uk~bzf|K4oG2T+jBxqN@%&D;sTp<6H3^)d)|Wli)^!A$YE#8_#nd)n`RGE zF0-BNLoG_|2|}e6HeU!zI=E|up};EJ$_eCjvOS(c#x*u$98$U1QVFO{3B#9Ad7bU( z3QBEoC&@s;O*T&!a=F<)a*%0@ZSfu&@~{;ipbjPKOHgH-?V=2&z1$7FK%pJB;48G@ zW9z&@=3TbUI;0laUK^0G$7cP8s+5@ffiix!i4GLr=k826{U>*TP0vrk4?>Y6?rw%5Pnc~~16hyRqFP8BVUtZj(i67bN$8ytzowvE zl+9s;qNm)IOprImHZup=&e*Qzq47AI+6Kwa*}`_HP6^#BP(H!7y$Z!HxT~*0z9gG{ z1F~PTJ-8uVifw8el3%eUcc2C(szj)eW;^sl@oVn>0+5(tb00#>H*B>dNS|d}h(L;4 zw!0|Qqy*g=RLrr(&!NN}cl}AopJ&@oL5_R2b{aAi*jzV|@`3H;7HU!ATplVF*|G&F z`N*C25ek&pPM;v>6I*WuGM3rIDx`X5Yu2DPC9Hlx5+Ni!tru$^0=8YL(fpzI%8 zU=fN4f&Xp^ze|v(>pzL>4#+BDt2!ZVH=AW0l1kZbH=%b*9P>cA9=4blipsc???T>Q zw!J;bCTIKJhsOKZoIyyYU@M29Iwi~=L-~HT>k}xZhz79EbZ2B+AFv*tqhLn1?fi~2l#Dfl0nqun+XaAHmaJSJ71*X|v zdmyKgZC(x;XV`KINM&LhAAs7Fa5e~)XW9DHP|D1mUjqf_*jif1Wno*JfK2mj&y&!Q zmCbB`I+U0(LX`!!(OD>MAFVaVfV zyNE#6EjHCDr1h``&mieG+uu3#PKldIDCcFXrJ(2zch+mj>tnmSfo!{M6FF#HWQ*q^ z*&dtX0jg8N{v(w4v;90lv3>4b704H0t5hNT0o&Xgqzkg$d_eL;Hf;lHP-3(R6+&#i zKT!OLyWL+%46`-=pygvWha~<_`Uu;T6jGe9&GbS|N=WxX#VDIv2_;UsJ61vd7+d!+ zWIHrN#uS?{52>!$ z+zU{f65bY}a++;n8A@Gqmv=zH44ZBZa^0|NTGuw3rl2q76ZlM|_MDC#M3!Ah6MXKEG6d}(m+h+-~*4UPxA?+Jm@dc8; zvrX5acS^i`hjJfmL!VHz&RysW@_w>)wIExAZL1B9f3dxHAX$^m)^+|*bxOGJhVtKR zlQJmQ;w~YFd_Qc;e#qWtI~ag;zihvQki5gTp@tfiSk*vA; z4INu251A*~HXk6hp6&G!5~kR!WvEJtxeAmquuZ%|;c4#9-XOP;O;Lv|Gi?3_q%pDm zG$F|>+u9FQqlC&Ylr^)Ng~UHa=D53&KpqR*Xb)taXN&eiS}U7O0ZA9wc9qaOC4Q-( zoQ=&f3`G~YD~&*2JKKyFvMsS)jYH$hZ0bo!w!#*kg6fpeZG`d;wrvv>Tjj29hI~#o z`#fY{V|!SDbS}0jJ0xFcOD;nVN>r^vg$=gDH7LHx-QPMSy4l=rXnBjR=7IDcwuK!? zvCVe33pFW0=ZA`3ws-(a>~PnA2>E?%`$v#tm#uva8ALW$6jJW7y~LmvCC;5gB|lsC z0!r?4r%geD0NZIAavret-ay76n|KSU4%wP_P@57~3s5=4_EdyYN8Fj7pkSCS{S3K| z*+#06DZ+MKgN9Dnq#sa+61|^LCCc{s1*K29TWUd}7+bLoZJeO0aK0Sv6b14Mj${o7#pv z8n%la$U4fV5+SXYE$D}&V{Cr`=$#Tb524&RTkQynPH<<9Kwcf&T@p2|Z)JP>gcJ*GGfk*T3F#J8w6Upwp~ND0$A6ID&ekot z{3pi}n^y`Mmf7kuNV&qc)CaXF@u43oIoJ#;D7ngAatI1I*#<`-=NjALC}ec83FDAz zoz1O-+LZ97hsqml3)4_)le_#36m+xc%#dq~?c4&HJZ$|7(9kwpU=iw2VtWayc-ht+ zPxvNaSzJuvwiPF;Q)85LCAf; zRt`ayAe-qJ(j2l~pFolj+ei$mQ9>jRWslgT7f>Y3-OeTCIcEF3f~*m?He^q+9dsbw72B_n`X_msZKE4%P-3+QDqOS8%b|FNyITb$-mr}i zK+9RS*dV08W$RNziX7XX25M45OA8h6*qjqkBG29PB;>zmn>9d=0$aui86Ma)vyigL zb}|RGD6z*1l^)rAHYi!*u3?7)Pi!kIkh9G8xC$Ac*^DkoRbfkQKy6AG-h#?6Y)9Kr zs>+?j2L)f*JR;<(v3>X<(;M630W|c^Ryc$@l&B9wl@GRy2$ZgKH*gAtKG}k2&_;u; za}Jrm*fx`py2oruR#M}*(X|YY@pzsfOXL-ooW>Y*smR~miBc$oD{X9XE zKen|BRHKAS70L>S|J@SIZ&0M`KM6MVqDm%EY_^2*s}dLdgM+f^Slu3%FuAz43LSOwK7p?et0E7`V3px6L+^-;*D zVzZA!_CdA>9i$s#o0@{;!)(cEs6mM;6I4*M9h#x|2zP%LNYt>o7og=)w%Q`3*Rm}v zLy9rBI|tOH1l<}`9A}HKLx~CQ`rVLU$F}c*9FuHqFJ#cOxppDt6x+)l)S|?>08}!t zWrI+1nmg?g6fm-#9z)I>NHxpWj6-coSiOMCX11qGC^g5ODGdcJZ0QW- znr9oig-lkq<2z_*flXR~I+W-wLKPd^X9-F#a<}vhh3ss_7ieRN%}|5P%WRkLka~q} z@DmaoY@sixN{K=X%B-?&wV|++ySEPHUSqR$rT@v|V#{|!nsv5G86??YJC{Q>N>KJg z*-f^<02Fa^_d5uAw%FFykk!Lh)j-;9Hp>_!^|IYgK<|_|riXGnY%v2A^>HVkfxNqH zd$W*DWc!|j#`oBqR!HV&E8CztCCn~C`F*zQ6(|O7NUQ#Z$KT zGbj<`ZZQG*&)DvhkR#5fzk&?sY>8_~nP3~pLM=)>$U&tGwoV>OCb`>qfC86nuaA&3 z#Wr7tj8|;A3ZzQ2jlV)|N;rFi%GYdtbtskL&fkE7H*BpY>o*LW5 zGGu*YQ>{YUcedafB>iCfTZi5$anlXu>TER+6#eARx&wI|YXfj51m#<7KgUq)hdWmk^0nD2G06VQHg^u`I&3!=ko=ELn}QmY7)?V3A@tuZ zq4x%gcl{@E_ZAW*Y|T4pxtq;Vfb>$fry`{2VVik^nv{@!hKe#abrnkVa(7&V{BpML z56IES=KX{W3by(er0i!~YC$bZd}u=@C7a<7N)B+B?8^L8K*csFg`9(Ihcd`G#3uAX zs$n*FKh&m#w*jcEW?LA7QX|~u)lg8wrW=J^qipA6kV(tduY-oh*aCW}Ly2tzR2gSm zpMla7+`Y^~Asw5=0&PsPWv!4|&o;IQsi)XZmmtBwCU-zpN)(+?W}5BW1%-{=t!_f@ z8Mg8kWHGUsypU#=?b-)P%xoijP>m8I`%rd{O&WwE7VdUJkY}FlGYna+Y|AH*c7d&U z3Q28j({bpX5-$@_Zjo*15{lZn3td6pCAO{%WLsw2%0lBSZ0|Wp=3ukkLv>2He}M9< zY?CD@=HxC>hJ0&m$`{D)Vmo++bn9%tZ;*V0ZKDn~D6!gr3Y%>6-%#Am-R%z~Zn2Gb zpk)tROt|?c{We=)H>B{g?e#!SN@(>$#T_=M0!sL}dsagJUAEal$RV<2h9Scqn??gE z{cI;%s6~lA6HsZN%{K`p1Kc&Hpuhp!iV<=K*&a=h@sQ0p2dP4Asd=bP3Bxw1e8hHS zhf-ngBr8zxn9Z{axgu;IYmn)LZE*t{in0~lP=^xr+fe0{?P3Q?$G961q0kvy&<}0I z**XEpe9pFc2&ofnuSbw@!Dfv>RZ7f7p-hr(;tUF3a(8wPxl?S4BxJc_^QRz9n(ZeI zNv_$}ZlD?^RBoYchRvLZA~)RK6d+HQZS)bc-m*oXAZ?CKR)M5EGEFrXj@# z+uaP*qy(KAD%RQJ7AWz_UH<~)Z?NqzLXI!C_7Y@hvbh|P@|*3&3AHG3ZXGJM*s_~Y z@`pRE2MV;=PQ8%xm#udfGIrR+JxKM(*4&5Ml&~6v%EHlqw}ht1>H6|>MrADiJ0GAr0F?;&+R z+h7qAlx(3ARHa1W8OjW>ZM{HY6?bp1kb98L_6}Kw*z$EqGt4&m1xeIw=ig9`5|nKy zJHi&|KoJdhzryW5c}CgRyCJKVt=a==$Ji`#NIK4TtAO4qacls}O|ZoVp{R~Kxf=3L zvh8Ugo1X1k3yn{)IVT{Qfvr3V)hS`t0OhCIu8mO4$em^u^3AYC<{-O?t;Y)KX4!UZ zklf7HutN<>TwZ|+b8L^RP~5`ZvwADwZtnb#P;iT_^#r**Y-<(Bw9WQhg@(Lr z<~OKAiJ1?mvcopofYLthPMT0?mreEqZHR2XU&y@2_Vov;{cJ0e+&}$$#QgWRC`q9z zB^rC7On~jG4+`x*LAJ08vK+EWh9OOeZF>Zg9I<_jLN!XTk3-ooTR{g!j=7tf zf;2n_dxk1+m9EDU2^Bzg?uTt${u9DVw(#2BUy!lFCbS^c z3!A$QwJG7P1C^_63te~rlzQbZ-wg$8Y&sd_dSg46L#B7O{(flagDo%sbttht2vzEA z>uM&q3ifcdJ&& z{mWLiL6#1iX$jK&v0bk~5+VHGf5C_os!>A31!cR~q?=Gg@}GpAEy&Z&_PGsNrEJSS zNZZ3!6d|dMZF(Pir^L$xDA&t26oR61?m}V6+sD>*0@)O7Tc^-?Kim5mBvZ245>TBI z?vqe{fNk;$imA9uTtmJ=Hf0vF53wEOAl)$AZyu7X*)|@a1|?P>p~48;d>M*sxVx=D z;wanrE3~X-i@ibmF}A)sq!?%0Yd}p(Xf>hY1e@~*O6a(I{)POLY_mfCpB#F&j07@F zv1xiBrGf3F7iv*rj{+)9v-y-z(#Tyy1qEiV^Ikd3Xd0Cgx)Z-*+2Y!}N=+Roj;Dim5` z3$8&M%WR!>$h^X~>4ww}wpR}%tg=~mpeiNicA<=uZNd+Q*SI?iKyDYC;t;Z|v-yu8 z%?8`gF(lb!TZ=+9N~px3teeez4n?-OySaco9=6dGWZh1 zK{k5}vLCWNv>{!HZR!t_AF(C7?*FMli7F{n2(ul^p!hL&e{x8Su(|u8B-mUtkn)12nisao5+tdzotL2+B`9B@ z>?>Q~6^hij`+b8vZ*1#z$okG!Z9v)&Hp@38t+U!KUAa4E=109Z0EU8xWxuB_8;p(g0g003}u2Z5%>@ zLAKW;$T`F|AAyX+Y`G|;QnQVpL2XJnJBP|6Y<)>6rQyz>f`X%Ltu*A)vaQ`frZKkX zTWDyU&76lil$a?%l?k@dM<}i1?&JvyO|r=<(1xDPSB1<|Y+p45`PRiy1=FyfwVTZ;3yjR6VSf?IJK27gpx7FBE(hduu~nRqeVuJ?9nx*E-E2bgO*X9uYEWX- z3l-dKy}M9+i@V)DNc6BZ_o3x&Hb)TBd)b~skYa~z<``;HLiz+M`qy2^sus^(#oZ&$g6-T9o*Zg-QW7!yS}7;4XO&1%hmYMaX%`c36UpAvWO| zQXR3mU!XQ6yuCu@Fx$dAlse`vUx$JbHr*HGI$=BihD=em{x&pp$`T|KexTA1o9`D&wz+HkL4jYk70Kg2IXi5RQpotnX6%JjLgc?&LaGmHQ^K$k zDtECRsi2hPKM9gyDA>*B8G&3sVO!gUYLrmfgR&Yna{!8ra(5GiJX*HVBgi_&7CnZv<7~1hB%NT}jY02} z_!Wn8IyT1z6rJR*bP0L&Y%^)dHpO<8fyNDN>RU)Q%@)3c>Xgu3fbvGR?IIML;jUhS zd?q&gGi0A-dw79#X11vsB%fnTzC#U4RDD7P3)|rr6rbnr?;8@WZ0rG}KNY%dz9MTv7` zP|3-boq&>S+-dbtz{PfIfSl`Wy)%$;gH4=;RGVzgIjBtut5&G&W_z+hsV(kIOHk0m zmR^Be+iW9F$mC@^c0ofsY|>4rLy6uksN!S$+=kM-+%5T_kjPdPp^ZH@!#-s8vt1rQ z>V3As5F`ZHLSd*%iNXn#Ibhp5g~CDZ-p(NRA)74$Swd|2B&0cFo4kS~VYc&Ys749O zER;QF3*?|kguCB74AJ#Y>{5bo@VP&K)P$T z9VH~sur*XrgA$jAp~4N@;|LVbayPAo#9Ox1IJBH&8=8dlcWg&fkRs34WrUiP;4wkP zd$xBolqhhwI1l+B*zOk~N0Ci$hYXKwiDgJxVjEb6T9kOO29=)JI_prf%-x0?3OuvD zdLUEY`$B_{LS`t2dP_ZD+NgSVJj7(DkU19 zpiGKO|EayMLs+#R-X4YK^PNj@NraPr?RVfz!3bp0ps;|r=$g1rT0C2WN@6zS$} z>JRcr*)F=u|77i9Q%NDMj4ddGq`ho^a_F5BH~XQSoUJwhMf-y7oi3v zMwg(1hOO5D#Yef@bwZ++t?7c6$JiX3kba!)X$w+Ju+4a(CMBeOP*KOG-h&d8+#T;j zemz@v5OPehc|(xFz*Y}K%4xQx6R1Ur52sMc$YzK`$r!1&Dd8;#l`U)w_fTq{yZi$bw6f_+kZXbMybPIaZ2d3L&>~yl73xr8`wgnt z+1Be&dWpN21{7LmvwTAvD{R>x$n0Pn>p<#Nwo~Exp9CkHyc?=gqSymv*4Vy#p|Fd) zRR!c;XDcfq%Lbci5YlY2T@OPNH`|B?s!>8j3uU+1q!UoY!`;p#4Gd3YrhAq{C+LSQ-3zcu! zj{cxjmOF{0@=w8AHjfl?<=8%Ckm-(Xu@4%`vlaTG4khYUQ01QOVhBnXxEmOOLJw@g zQD~#c))|A$k8GPdNL^xk)kDG)n{^thQetif%9PnA%ux85yE6;quCOTIZL-ZAL$+_Ws}pFv#iouyvLCi^9I8`7_XU)1vu$5Ov0v`$SCFs6 zX3s$OKemS~q!Xh5-4dqmAbHn+5|Z~&gA!Fms32iGEJ5*Z?*7Vsq{cLUF<(~{nHdi;K9AJCtfm)O} zCx=QZwyXk54sxd*fC58or-P7ln5|b08P#l}22zc%HMLNi5>_XmvWD$x5=xD7XEH!R zEnC_MxyIN=W+Br!+wmMUG{GjdLLEx<+MtS#?b8mWC%Ic%fkJw=;wrQ;#b$6pW&_*h z2Be;58{C2fBU@-2s#2ohgEBL0TOt%Tarfqj+_P-91IS`#%O66TIkw3#B(boaN1z%d zC{Lm6JX_!lideb(J%>CCZ0kwLYGbRWAnhWXd* zyen*bkC4s5_WcBnud+ESkj%+eu0nN6n01L-!| zc77qbo2~H&H7IdeQvIjG7Tco~ihH=5?uEo{wp1Uq>}4BLLi!!HBNe3Zv2_hYO-k^L zK*e3Q_faSza<@1R`S;lFb&$i)rk{cg`)rA6NEu)oFhMO!JTOD01GbI@N(Q;xSbzeD zY_E%uGsHH(3>lBuat=rpW*c9F+LUm%4waAD`rJ?|!kym(1y9&oUdR`hS3gUlCfUvWsCWLvp_giE&4B~+zE zV;ah&*sd~A_=>yXTgaVe3*SMOYc@#%(q!1Si;(1o?V|+MD8c>=WwUIB7btSe-BbJ!rD*@9n?^q%eS8+xb2%{G)Pu+=(H^np8T*Xuudi)?q@knNFeLI#bO z*y3_X_Qa;>hw7BDKLF*+Y(Ilg?3p{48uC@xDjLZC!ZtSs>8flu6OjCsO{<3*lo&NY zg&JG$3>1IkZg&{w@yRx`1T`rky#f^*Y-%Tz_~P!^1^Jt7 z-J6i(o6Wlg8Cq=hZAkgUw&a6al=vV*r8b*kA4>jmmpp(19k#&`%Qc^#@!qS$~kgKXbTC_Kd7 z>JQ`|W-I?f7B!nmsQr^>gzZ`aNi=LDJy4AjBE3*{lufFDB3kZtl#pkP?NbF=$Jv&L zA?*ZPaRid;*rv76J0)I@L%B(|p-CvJ=Pon_d8gRAjF8R1wq=6Gr`g`kkj%(tn}_O@ zaK8ZMXV@m~P|U<#Vj1$yvME;~yP54^4bsiA{jNiD3)_YpYEWX;0~O}k=69gDmAl(r zNL*kW_e0AzwpaktFS7L=LJB+E-VxNKgw`=sTw-%Zp~NzG&oRiq!Zv#jIUH=63&^m_ zrb$6cC)-IHYEfd(4OCiV^W8#87k7<2D6r19Qh=NrY>!3AxXEUGf>ds{)HBqkgyAYw z-eNndK`9S+k`E}j&F1-pTwb=1FUYjRw%CG(d~Ag_)S*QEA5__8yXboRr?kl3fD{Vt zu?1z&hM%n?hs^tIoBfbF!1g)-2?uP}A*f1;IW?3CvQ3Oa;Y04u#vpfyO`(G2Pm8c(vR_aNCNTX-L;Q$lwT%BR@2Ls0CByLuS% zrP=H!ko}tN;S|zk*rwu;{Dv)=fEtvjx`YZ@w!-H@@$CiXz8SGHy^)TV@01yruFJt?8o8+WEbDEQ8n z9)?^WY$F=TRA)QZLPMWy(g~W3M)|Nmu+hm3U|1BTZ7zxY_<)^BE#0Dx z5w@KwB-gMtYEXj`mp`DwDBI&F6xVV$-Gsz3wp0sR9%mc+h4d3_M}Lq)$JQnJ_@^c% zc%)EqlI>jvCG^}a_CfwBw)=j_VPMm%Aj33UVhBQ z&rWE_&SqYRI+U2%gepsHqaG-|%-x9>3azlocA*Uin{N*?ud;pZLux16N)Qs(*h(R& zN{Pl}DC1(gI)TFL+zrPd_Xb-y4p}zYBo~mz&9;3BNw(NNuAmwv*fUVp!&b;bk!|j# z?jVns?cyG??y#wfkk-c*EJ4y;w!bp;PKlc@P)=m4y+YAF?yT>S*UxrWhiv<76JOAH zfGz$F$qv{QZKzHO`yD7BWcv~7{}emq&eaY1LTr^D$bQ5&Cx>)lwi^W`KW5VoKn+Ta z4nllM8%^092C8Ui|G0vu*g%ao79nV4j z1Y5Tia$K-^ZIB_!R<}dSOSYvIs6~knt57M$W^h5tEAEmTP$11VxCJ?{*$%fMV}?!e zL8==zw+OW<;mr?~vuq0oQ0kVu{2>&~vFXB)>yGU_0-5q`{io2-JzL-m>QG|)9I6!9 z){{{BfxDL!6e_Y=uAz-bw(JdLF0qZ}AoUa5X&w^FZ1M-FN{QkllzC?ReuBak?p7<1 z`-QDsg)CJz(;K9DWxM`>BsI2?22`VjNE6Dwu}OcR$UApCzmVsH?eh<^*4dUNpZ}!& zWGhM`X@hOL7ka0}%RVUg#WtjbqD}5XD#-iI)-?>-T5MY*(D)DA`zR!9v)RU>Iwjoe zp!_e}{de1B|86J!_8{<|d{m?2%)e-eK!kX*vHu>dtFvAPHqy4mKJp}3U0 zTL&ceu#K-l%QCjuI;8Jq>vKa2IoqBGYEnYW3l;m=oV!p$!QJy7+iXK>Bim~S5@y(}U5$ULQev(f%9z+DWKejPyE8fDHnS=EAG zQ{vYglv`$VSfS_&cO@I-b+FAWLAF)4s}*S6$)_a>Bgv2AZbv32h1 z+mLUA&F+Kjn`{pvq;s=P?L+b{w&VfSphQ&&DtOoq!%%#iyT1q|dfD8k(DDvj?F`cU z*cK9yVwdeM2{kD}cLfziw)i!a*yFB03;F$Q`#H$5&(_XEh5(!E0a6~Yy*xrKN}MZ0 zr660j0woW*)4oE15Zmb+r<1DRrM$Gyr^GQolzU){1)yk=JNY5x zePr7^f@~$W?_+5EiOm^>WM#H;460MY>^YQwX1l(CVioQ*DaiN27D+?)DqGJDqj?xvq0@q;b(3@z8$hN_VMlkKPmDH?2DA5fDLJfBeU zi|zdjN;J7!Y(f5Sw)-~ZXtC-4Aj1z^qO18&$~N186lzi8feb4BvUTK8vcug*KNR?5 zdmVtBLj1p5!u$|q?D|haP7SFfY~!O)n-b2(pmH}`pAJe%x%2CxU=Lf%0J&srYcr6k zm+g5L8j`b_El`INGghe5$2PhMr4`(rEJ2}uHkkw3P_p@)ka>XZ%LS=bY%80PFvwQg zf~u5g^g@{-wksbL9_DU%4|1#7!uybAgiR8JG#a+;5F{C8`v^ldO0b_mSuI=P6pD;- zHx-9G<7^iR$U4EMx`ebkw%`>con-sFhTbW0GYjSPY_%K|o#M`V4|xr2cMp(lnr)&4 zjT_nGWk@!|rg(wsl(7E_=*v@w#laH-mgobw60)D7NiR}PX z5!u!cq4XYiFGojh#X21Gdw1NC>jYlTeis#T1k|WcyA-;ShJL zH<0^?t$Yhv!fd8Iq&a50EcegW; zc*izwhL-beF$<)>XX{&l6a}`uMW{&$ttF`Vz~*#7i6VE;PRRerHoFcvN^F@;$neCb z@j%Kl+ld!yQDV<7RC;Ff?Lo;3ca420@WQqdgq&5j#}H(EWiuW_sv2AB1Zq>la11KH zu^q*s)H`>Q3n=)(=DCDib+(Ty$n?p!n1O~GY=tb;p+x;1RQY1NxQEhB?golb=$kEA zf;L)goib$pVcUFx)NQucS4j9}v%W)BO3c-vOowga3kv^nclHgrh4cSz35qsk>H1Hi zzXNF`Y(K)!KS{dT*1Dk@B~*H#tdz|xhax@P-6$ZBjBRuPvi7n?2O+JTO{RvVeQdiL z=$#V3v`|jL=9qw@{oIu%A+M5c#sJv{*shGwxQb0Z3&{rA!gEla61uHWeu!<`2E~TC ztJ@);n$5lf*+#u{9{S#}-?MqJHk= zZpgdOw&#It0k&^1G=9M5+=XO8w(=fSr-a!6ls{y<4nnaIcbX%}cf=MshU{Uso+zX{ zX4{ED@(5ca4mBuo`2s4OusvQv@hErGX-GU}OJ$(t7~9Y-q(5Ujx`PyPwypxyqy$e9 zDxR~wm!L#~yTxb7f5CSD0y&aw`Wj@oWJ|n5$`sqcC)A?EgDu+zyD;sVas(xsw~^M3~E!tnH(zLvi0>tsT_Cy0VsIK)*6IddA2n*WV&a2 z)<8oAHuD(Np~TDtRC!<<)kEnbcP9oY^vEWgfi_BPzFElp#P&4@smp9DR!DeeE7_nb zB^sBYOoi=g1q#1#H|&JmRkpATvb?fMHX%)oZF>unys>?3Lp4gU`=IPQTS0^(AKXpt zL!LU@#Q|jfWK)G8ZG$ZshNNF?e-Y@N5;sqwT$8PK21UQQvnC*Ki|sB6*?!n2uAuQY zTl^Z5{jw>tP@NL?b5Op+_LGNVf84nqAfJ%<@0L(`gzR1aNti1`ItklN1(J8OXuxAL#odbs3K`feJJ7~7TXq*R8`;MEka~vgGyn-EHu)h`r9|-v%FMETA46d? zcdJpzJ;zp#K^6;}=^WC`vt3_65-ZzC3aU{;Bn@R3*rYd5#KzsuE#z5b`@Dm!cDCgL zq+Mbw79r^}+w>FkPKlS#P;P~7s0u|L+=Xh8ca^Q{1F|{UwmzZpHMaLJNakX*wV*mB z+_$0pI@{zQ6x-k~(e?LFzD+iz6tcV74rGvSi|tnq$vtcv{ZNAvs{>GBn{9pwihH@c zRYT$q+xRH7>|=|KLHb>`J{_bG+4l5MlM-46sJO@GoPiR4?w)5M|32HS1#$$~GFHfN zz@}M*ltH$WC8$M-Jr1aJ$mVlG$q;u97Zf;RTiJx1VYbID$au_V^g^l#TgnHuDPedI zDxa_&?L(<3cak6!JZ1BQAXkj-BMh0&*cMNqp*UON6zWi-J`PpR*)9@LI>Ft*B^0_~ z3tmARNw&^4WWHqE%tGoE+iMOIuGp;iP?ZvM4^Sq}Hc^7Y*W8_zA$NvN@d8Bo21#z&*6L7=5-JTSn`1M7LypqUhXRT|a#vD9-V)o)AY^-DyBdbZ%WP^5BztBHYoR(NbWcF}3fuN16no*W zJ_Y%zY<44Le`R|xLAn~-)Ep##V@u9M4N6qmpu#)bp&g2UaQC+iiFG#jDzyB`R$GJg z4Yq|1Nb$vX=Z2b;pxcIuO}6+Bl=$YZUxfTEwtYY3_+e`YAVZtYbqFbc*idR&SuPl7W0O{(4kdc4P({J^S%cF3+%0`TAthV!6WSPHGc+Ny zitVxmsR!8xe<5LrE%XOfDN&Gg{ZnR`ZA%J;)!eLKkUn`Ii3>e+5*pm$0fGefy4wwMKq z8n}}$K;CJ#y+z1oWcyx%#%I`^4oGHVD?6b&CCsiv`B}E>O(_WPEww*mlZe?riLk&t?4nl8jpahlH**aw?xxwAW z3l!L7dwqqRZnpV%$hgIpt3xUe+xQpMri8O^sJzYA*M?GF?))7nxWm>GB>&{{v8{DO zrd_t@9%x8pGs~e4C1w;*Wshxi080D0I~jyR`)o2bv=LzQX(02#|4F#RX5`v13d4VW zOOhl>LK2cBNs=UCOOhl>l8_`xk|YVq`>c0;Jm#GB{hn*jY_?AwqzSPtk3+%%TWJES zQsT4`x(>5lPD7DH?uKR|Z-gx}2U(8TBvwcpW!tntl4G`aJ5-|t`!bY`u@zRJ=m~d| zZpasBJNH1=Q#REGq)V`cHX-R5+us&cr^Iy;$|c!q0VsCPopm4br`YZeAln7ocmx_t zvn8UC?2=7!0yQXM?-a^s*nZBS_!W2V6tsQKR!Kwl8@9PCNS|f9xq;-jY`PrOq{LVr zD&*LDAE3k?cRP=em}hG}L5_PiX9Y48*q*A8;(=}E4Qf$B<{c^)*)&Zk`N-W-3ksCj zx_=<&6Py1RGM3pIe~|K-ZAsGoPv4aIAcaa5He)Z8df_hB2L-EagG$Kt%66cFrfY1% z5Ttrz^A1C8N_ZQA%5}DdF)01cU0x4`8f^MW$o;`~HU*iQZ2cxk{mB+ILmf(NTcApd zZEXR{d~x@(2!+4dEDp%?!kJ6VH-4x8KyRVh*QLDzq5Uw$Yer2fAp ztn5JEuKy*JcOi>}%@l;R-E3DONFrq$K7?wN5IusjJ#5k#6qRwe6^DGiY##~8Dra+? zL%Ke;;sqpCuuWy4Iwf9QL%Du7^(_=rau>dX`~z%V1<0mi+bBX~gKYH@BvZ55o}mUM z+`mBiA-0Jc6xVQ#( zGi+J|q%^S|8=-GX?3sZ|vuxY5P|D0*a}Em5u`OF6mxb-o22IbiO)o(zD_eRQYE#0H z3o0+L9lD{kjXTLY6k25SZ9r~2+xsSDT4Gz=hSUzWf(Uge(XanD)L&Gvc<32SWDBvhruTnf7Ou#I0rk#+7)uOP3NO_7Bx8*G6b zr1i1=+v)>UqlC&Ml=ZWj%TRQSyPFE++h!Yig{(Vlu{TI3vdJ2dbeCY|bAjw#QxR7xD+$W`v%9vhA~7N}#b2o2Cbn9k4}up#~*%E1-OsZBq%w54mfo zpzR2oeF(B2u{{h!`Y79^4w4_UrN*EpC8{Q%LX7QT5=xwK_csNJaW=0Ba-6c&%#b0$ zwlEJV&e-l2pcWa5HQPxLa^0}?9zfGsHt`Tr-Lkcgpf)9}#GrDH?I{kW@3=FaL7_Zb z<{Wb0vkj*qQ-SR$1F0X_q&HB968*POrO5Vi2W1|)TPi@|5?iqdd7jvePmsCHcJT~p zp4kSgkWgU@*PtpT3h&VM3){vA6sdCe_6d1k*=*mCrN)+TL)tgCi9bkEXFKbX{Zow+ zlu{`B&K8tG(FS+Fa>)0=w$=|>n{3qqNcYKRQA5%e+pPwwQ{w0dl>1_fk3z9;?&NyN z|HHOxfNX8HFC#Se%jTMaWF5BhEYzTcSqqf^W4p3KapB_sTY`2G+V1*aVsr_zOW1mx zkiMI3%LU1$Y)v=Rq(sL$ROn%Q+<+1??xy^Z*vppQh8%J>^)6)SV>{e~6biPk5Y(as zUl=O(v(+O|Qpw%oF%%eJyFY=PDmFs`G7hpOlaNx)HgExbQ{w(5R2pLITtO)fcb+U1 z9A>1!j^l0R64e?64a)I(=t>ZW$Sx^(qr5OUZK!9+xHvf*0ZfPAkzff za}!b<*vwy0hZ3)UpvolMNC(OoxjPnm|0z7hChLYg(`?&4ka>pfvlr5s*p?NLFw0g_ zLRCtf9)zyVY?nh&WRANbE#$SZMRbs5o=q|iX{~IV6Od$q?R^reQG$IM%G%fpCMded z-Q*nPv$LJgL)IlWl?~E4*g|$ly3F?Hfa;XEz5?Z(Y_(M=w!)p&1NmKScV5W0$~L|U zjk(zpTaavxO(8-JO4tiPc@Nu95Q?vJ=RSb8y=;|3$iBfg7lrgbwwo9v-(=IBLQP7H zok0aZTW< zPRM)1R(3&_ESqT!(%!ONtwWL=+prI+Q9{%YW$)OeJ5V&w-PSJTyJ!2@gRBKMM+nkA zuoc6Qw8%Df1l1|=;uy+3vZ>=xti)Y70r{WUy3Qe6nQh|&8hd7|UqZ49o9!BEP{Mr{ z%D=Ep+(GdwcgcHb`;|>ugzPo8{Su^qWBV;b@;aO61!_`a^%W|-v(49`M1#BA1|)v4 zjeSCnCR_XqGJLZ2wIM}|ZMOrpDB)Y^`={a;o2wg2eslNS0|kEAX62Bx&33JTjK6H! z0Z7?lJ066-DX~WbmHybawNOe(|9?wp>Y!lP{}PtRA(w>haRQp|W}7xbDk)of8fsI* z&@5E$VLP0I(lYKORw&fV=CeU=IorD(GWD@7E<poPa#M}XNtzjFFK#^hYPNR@l%ceMiEF)}z zQ%I|0`#FOoqim}ws748uG?X1#`|%{KD}*=E=--=Q%Ro2CiLX4xVws6h$cKTzJxw)qRi=eTS9 zLE9EKyF~F%_Ib7kDWtcuP4+_a1-4Wl)TBg}5-Ql(4pdNLk-I-NB-+`$!;oW%tu_J~ z9Bd0?kYbtbP7k#xK|cuqEAUA{2>m_f~?uM{Krd$P#7Czd+h!wuu@fiLss4p&BJ9 zKcMUhTksQ##<~0bf_$fJYi-DyV5@c@-5HyutN)**Nw(W=s7{HaGAMV>7MDY@6nFA| z$bZ4MI{?|zY+r-W*d?1w1IaRMWi8a8gxOIjf5mn+4#lsz(;A@d8@8wsvS-O?a>A$^4v`=LE=4IdKq#Q*wik_@W6KHh7?7%u63wI3BC=e z_{dh@gpwuh7Pq0m6WhHAIm>K@J;?aXmfVMw6}Eve^i7HT5vcUS)`>!?DtDd}DEP|u zdJ4H}Z1YKI`i(7@f>d?3u}i2;38z<3`JJsV3#A*}1#(d6gY7#HxtnaO50L4T?fDT> zx7f^Os6&a@6{zyXHu4H(zPUSogTg;-vIgX7vu!sa^Do{5b(#LxO+j@^TsJ{EC0or5#Rj;u&O?3`+uZ_W8)O@| zLt|>TgaeWdu_;!d1|{sRLU|3_&l(gT=FaVfwzX^(A7me4o7;l)I<}h~NIuG@3qVau zj0K^>7+dcFlo;o3=MWP0Y^@{6F~R1HK?VccQyfxEvdx@9ElS9oLq#K-CJiO0xI4-~ zfoZnx8^}4s=D&rECbq^Mq?~12DnQ?q_)vsOW;Wv!l$zr%^$Z0qY=c$EHP3cXgQl%) z!aJl|VDo-JZAy6igvvIyg>NXm$X&h-h3stlKghkrcGflUPbLRjzZ6n0vjt^PhZ5Uz zsN!T>>xVKc+`SAyVHcZ44S81CvKq+jW*Z%WG;3@pqmbZXlk1@>C5i^m9zOHg);P3nZA+uUurAm0w#ha0krY>stE zx64-CfTRJoDL+)F#EWexx5uX5g z6DWVkHj#kh5$=*nX#0pwc>&p@Z2Omx{+R9e3X;dzJXxqoiPaobIANQ=hZ1q_ZXY1= zlx?g8ITCE~GGsVo>wAF|Nw(crs6`3iZ&2}^&DDUCDej(|P~d`X_6u^R*{**e<0YH6 z11U3X$AapgzA3S%8!BC~ZTCQ_Ywnu8Q1FIrSpm7SY>!H4`j%~a5K`sX(nC<25{9%; z`Ht;S2c`4eNyeejJ)3U=au?X%Cn3`V+u}5&F0vI&P=^u?b5P}x?R*}}l(-wPLE$I1 zkR9@r**Xr${LHq#0%o?P_gZeK+Zw7_9$djv$^$lL+%N-VIO2NupRj!^(32g2kKCwe;2A4**^B5%oKM^At*e}Rt!U)88+h)WHzx~ z97CE}w!t_gnAyS!s7i^#Idnb8ws8SPEZn_aLf&~c+cjjdvgNaoc7bi;4wBf|&hDWa zB`Awfc9AVuf}(cre#?+=iEZr#vO3tRuaIt;%~FS?PPW?yRHwwzPbjy-7XN}`F7D)Q z$iK?A+ktFuwl6{bPh)Fru5L)?VJr7Q4N91mL-}>KD+Ltya;F`Dwl~YyeiI>w>G7Te zP;rl~ZikXV?iQD!z&_jk3gis28QhTZfGz2Plwr1k4d|N^_cx)^AzNn)N=3Nyh*0o| z?KJ?oqHOc~(DX4|?f_E7*v2AIn-Wf=Q2B(d?*vN6xeJ^^p;NZ+GsvA_TTMZxGq&e6 zq)xJ#ub>VkUf)2KbGDHjlu2=SoQJ{}Y_bQ)lV;m~gv^(0pHGk`!?s+3ge$gE6{=F= z^c!@2&35??MQ*qoYC_&DTcib9ZrLP1kT%D*`3p(z*xvu38YS2zL;sY`vlXOJ^q#xP zUdUHqJMV+64{RzWq${$8RFL$M?N1HWDRF%m%9Yq^BT(##JL?$aFSFh0A=@+C_#`w| zVM|OwvKKam32IQno*Bwl*?ufg{FOWR0<>LYt1LqHH?}zkq_4BxI3f8vn{E|qQetck zDm2)7y-?zVyB!}SHrZN!$nnYM+<^=&wx?Z4@x?Y1gj$r42|>kgHq9ZF{Ne8C2nw{> zx?_;@m(3rCj2*T{0#g35EuBN(l=yG~m4vJRZwbZ>lh#H~nIGc0^it4%BnuUB5Y#(!w)xhSk zLb^$|q79N7*`}7DIwf8#L%At7wF`<(a~F0){u#Eeb;xF7+t`4{X4&eSkj%_x+lCsH za4$mnIkt&CC~n~{xesm6vnj)n-O9Eff%FS(zfnkTWAmIqO-ihuLWM=P`6QIEb9bA9 z#3i<|OUU71i(f&8WwyR7q;RtB=AafOeCMI!3Y+TzO1ijveuM(6Y_ny^>1MmGK*lvT z?JK18upPfa-;~(XfJ*CZ+f69t<*wO+f*WkhKak7E_V^1;Z?a7b!~dl6v!x|an-Ye4 zpz;>mVK0>4=1!u3LOX0eCFB;_-c^ukmu+zfQU}-y!%&A34LYc@$96sjWrExdOhDm% zw$LQx39)siAoBs+x(U*R*U)K1M)`M6f2PBm@Tjh zX=7|ZYmnrGZPg3aD52tmvT-)^78E_@?q&z_CD=v+koAl$7KC(3HrW9rJ!jiFgzA*| zc?9KBY|a=IyWp-Ahx}=_nKQ_C$#!`Tjb+$0X-Ia(7Rf*jO6b0U^4Dyew^00syT%=~ zon^BZAp0%bLlM&F*e0JK`5jy88ER6ZstOhIYzH+canIdf9TE#{-Vey}z*hT&3`Mqu zZ%FaTcGrell%W5EiY2y0m-e5MPu%rOp+K2!PX;-k+1hf*SYdPbL&_JnmjURT66e%V zsmhksK&e;mbR$r(#&$9ax!%}%_0V*kO*BBNcea)hYE#0>3{-BgJ482Kl=j;*H~ z(vPxj^+575wq`HXq(p}TDvYx|Dxrj)yQx7)oM1~2K@J0(S_>H_*$#D(!pPP&4z(!3 zHvtu=*y@u|a+ zK7`axHggo}P~vqAs;sb$oI)8FcgJT?c$H0-f;?`v?KEUwWBbfN8V}p@4J53ym2ROb zB~Is|YcJbn0g7yJH}nX3eQc2@$g;^MsX$sk+h!G#Y_Yx9pc*CE-=XX_Tj2wW?r=BR zf_x&|`8QtI43)K6lnW$RA?6>xXOyY~w0u zEXfrOtEQvQ1XJiBR>>K zvvu!4&Pz7`E@aHGHTEFo728q>`liH(FjTr`Gaf;y8}3raP%z6j7>8W9YzGNwI>#oQ zL#jJA?*-JRgttqmoM&6OhSK-k<+D(zz^1>0+z)JL_mHW`)?b9wk8Hsb)S<+78LE`n z)?T2@6L&AKP`J!ysY9M;wrm43SJ*~BA5x~!0`!?t0A#{Ssqc1R}N{Qr%?whT2W z;eG|mcd<>lp}6FK2}uvM-OZ-lfb3GX{Y^;U!}hxc$z^OF5o%ImH2@WQ+2;45gq*wE z14!&+8;d{=1zS7{8T#4!P9TMnZTA#vQNs5bR2*P)rJ$sWyXQ0%7-XBhf}Co$>l?^8 z#HP(bN)6j_9{Q%lo(HHj%(ndqrL^2NpP=9f+j0eR>DV5t(DW$V^c$oaV@tn7ZAutw zLgjI`!xog*b0_(MLKAGhU&w7>d;fz>lWdEUv42t<*$PsqLy3l7s4~TN-Unr-xf@VI z;Tg7&3i6oPI%>!~%eFoYY0PY|Bakr1W*vj7l$g^)*A}+%NhmVU-RTtMwX!KpkY#}_ zV1~3dwjT>5S!7#XfNGRbS%k87HnRhYE^&9`gnSORkyXgL%obaNbWS#z7m}{9?f9TN zC4Ty$oQuu51I1RkEA2vlH``1QvaPXQhM+MIo8}Obt+Pdrpavy$$Dq8IZ8HwVH@Isg zplu(U{T#AyvOQctdOzD_29j^FrLLhSC8}UiskwdOyw%&edI>sgrK&lhA)*#fTgcS``j!q4X(trco%A zV9Sg{?lZPw17u3F9T_3@Ih%9_>QJJ87OJG!KIWjz1$RqUD4b?1+91y*n{f#;XV@;5 zAnQ0+`Vl=-di@?He|`M`9PmK=kWLiYb#!oURdO^N%HP^pWpGXV-`G zY|lPOtzi zV`jEQ*Tg@`=GYWcs6h#PGAM6h`;kNOdG6f((6*JWG62~Z*yhxb-o|#Lf#i#9x)G>J ziLp_rU}x*qLy0Bsb_|f{U~3s6$1|Y8wjr*amkY*CyM+9yIM| z6GD(`i_IH`+LZ7Xfy&!#3&&7;hr9d<6cX9=3CO+6c9w)p0k-}NNWI4vyo5TG*uH`) zLAJFll-cL*B?pB=Y?gb-bHJ8;fXrdG(GsLNWIHKCLWE8J0#zwde1)!$*uLJNNR+#k z2IM_vD>orajLq}~X;0X$ejrJlZMXx~C?P5s{waIPChdlz3GTLfAm16=M=xYevN;rx z?wqZtgrq69sX?eti5Ejq?t)FNg<@&$!aB%*$<{Rv*)nV!6VTWdTYVCeU9;Jyp#~+~ zo1pv++r%6c&vKWXhqiCols3qoW81ew`a8B?2PDt4c~+n%C019V!adu(2TByUyY)ii z1KZdp$7j$tCHADC(kt6`8cNl;Yi6L}8{6^?6-i}^B>!~6w(N{|KAc`WsuPIzXWR^RHejRKXff&8&^S*ZthOikXOp47=|o8 zY=IF-D`Wc^g(SUft9qzL2^9mBm9v?rplBa=H#3k=!8T%sto>{;3#3!B$rd2#0Nc(Y zRHwwxB`Bw2b2_2eAa^Afaj2kYJ4ir@3GV)qkZ54@UO<(LS6N-x5{cb_NUA8q5vIf|y0Z6yUX4!|NLAKiis7{Ha5h%CM7LP))5O?wu z$bZ1LdkWdYY+q;4*dd!M1<4|8-u_Fo{c`9t%KFIvS_UVT-Rkr0FNO)x{?Lt*b zoDM?QHMYwT6nW!r=n(SO*&;`f<(*9ugR~8{%{V0aV0%wMHA=9bL)j)<;R1?&ayOZQ zd@Z)~YsmV=rn-f6-)x~fNczL}cMsJmalHuT+HAEF6#M1Q`V9FyYA|28{{1 z|8EJ2Iwb4*U!vjzYEZ)7CzO}4{d__3ZtmP|Xj{rw=|J`#wz;n9f6~j?Zn`0PFPlyV zH7PMBhYE7G-hL?2$KB2VBr4cigOH=2&8dM5O1390q!?hE8HHMukQs-HDmIM)N)B>& zWP}20w(c3oImG6lg^U`u#vG&^W?QmC-<0@ZgGyR9;}Vn_;V!id1$AtLF32^?cHoAl z$Jm5*NHxyp-GJJZ@U{t+^=u2q51EW?{b5Kw#TJY}9ZGCR zp~^Jd+6k1I;qK)W3Y*w0NyszHmQ6usGu!ATq?uzoxq<`>n>-6uDN)Qp*Yj*&c_?D# zZsh^;F0hp!A&ZU8RED&RY*!UXVrLtEg=&-#eS@+~Y|;i4b#S-UgnY|vA1%o0WOMvL zx)rwKFC=xbO$jsqRHwuX36xu9Q};kIH+SJ)$iK$crGRW6whbjTw$4^pK{79!Z3t>m z!u>Fm-(Z{2L2(~<$uVeqlTA4R+5K$$laPLk?RN^2Z?kz!P?Hj?W~i{kHa`y~MDA`E zAaR#%%nmsMY;gx<*kkKkffPZu-BqYX3Eyi_ai7iQg_0reo_$c@fNgdQa)#NicOc^- zn>GL`BW%Y(=$jII4xrKz+x8)pigMRHf`Z3v%Q46mV|$E4(5;Elu2_pP=&&m zY@r(D$*^_mkok&j{R7fmv%P*o!VR1C8>&)bt_@vh*~b5%$SrrLU8aBX=GYWc$a2RP zkU`o!+m9TQ+_SCrLp4gM3_#fen^_G-AGo{GK)xc|$OvS8WQ&bLx)PgA4@sZcb_`IR z5A+H*olw5Uw&{Z6Z`?K9 z&~}~8z7E;n*&a3^eS>Y%56M5+Qrl3I5>>lUp~-fz2PHna``d@a7MnK=IlkCx5y)sDwO2Y{o&z ztYNztf;7WygIY+?vW0a}l@f(<=z4^0V*-lkxOnahWY$fE-RX^&@0hVLN<+6fU-|3e=(m zUll5@vej!)(#_rCI}})ByZ?Zk9yUV@GOn{FzagcUZQvLBro{a}sI=`#OQ%Ez5Y z3I#XWUS*KW&oeQ~`XSXe+n5S!Q^KhlD(|rM4MS;>yTAw(+GYD5h1>zQRXt?d zV|zA0>L8nW3hGee^$b+mXB#m?nGko!7ASncCR>0!VYcl>$b885xddqb^8 zH+cm4l5FS4koBBR6^C>wwon3+Ua#Z@G8Dh%&iw*y=h!N*ko}Hrt`6z*Y&Q)^e$S@+gqoBX z`+^Duw%#_Bc;IfQ1BpepmN55Ejz>0UH)JTWJ@r6}C$VvklHdu6MSBIcU1U zCRicW2b?aTw2zS;UWAoUMha1-iKVtWg! zwAt1~DD%tRO8^RY*ev^y=Z`IW0GWmS|F?wE2&C!yU*bs=5+rQ$6R1jw;wg09&GvN$ zMWo!Vq#$n(TR9C`WNfA@NZZSHbpuJ{Y{NOIMhVe8l5L z92H16z*ek6QWe|O8&s#ni+3nD$fj;WF*SGL7UUmd>-vFg8n%sJXl$6R{s+mlY&MDI zpBj{KFNN|WY!kguT*qCq5857OQz{|*7~8%I(vP$Ksv)_a%`*%&DX}^N6(-o`$Do9P zyIVaZPO^&?z9eh{cMU4$g;&2 z_=L3EY(HO+WQT3F4b>>2(t)xfo4ITLpQ5|m-E>2~0NaQRvhK0PpwcHo8*XWadHATi12-GChDY_(0u zkYZcdh7=cUcOulH1pOXVOtU5Sq2wiZ{b4AOVcUy9&MUTd6f$13xlbVF4cp5p^i7F# zNvM=%%ch{zEqA(0D41hAxq@7GY`s}%I?pENAk{rvD-X3PVdViT7ucR2q4WcHrZN;N zvSlie`;l$<6*86Bj@}^k6PvUFbtuu_geqmWj~0}9=5FZ+3Rl>QzmVsJ%_vy^$y{Z- zkU*MOw!t1qsIi56p(-T`3h4TcZ9@q~>fF7lAn!YyZ3wb7*z&`W_JeIg2T7W2XJb%} z5|k5A_LD6*2}N7n{Z2u?FSa!kWc_BVnjzf}n`Iu7w%KkMpgJXv+M(PpTigM~I^4-u zApalR?kZ#x?*G3fe62xaUH?mPc_EpEt?YvulrXym<-6IgcA&VFJ8b~k?qQ1tA-jyN z=K#|8vTYqgayeV`2x?NIBL)@v*dF6hLc!hC86@_zrOzRUl1-h43$-tj zl;FFCii2$RJ1D8Y{_Rxsbw3eLf@3QUxP{`Y@Irk(sAec zfP$lJub+@>jBWlKnjUA%wIP+BZR`(fQ^IN2!atQK*!rYU+Q3~v28AZszU7eH$hO)K znWosD2O#w{n^_HYDDheYRc6>mMxcy|yW>$PJj*82Lmo5RwgEEFv3(jLjfHJ_1`_7k zO0!Uv5~nTDwUzDC3Pl#U8(M_CHnzwTWLadBI3cZ_ZPNuwme}6iP>mAo>rmFgR@i`| z%iK-+A)k}&d>gW^u&H(-or^8B2T51i{`R3dC9a2|oSUr{fnsaiS&t#VhwbhJvaPd? zC!jGeTOtX`HrNywP=gZoE}^`S?dJ-LZ*u3(Lfd|}N)EDbvCZ8>`fawG2S~odrYk{B zN{p4Eg2>kU0ws32+j)h=09)$~a_q4=8;~K$_SA$F`)o5`P>T{WKTt8mrs+V*1MZFl z+dl=uY~9_E^N`Ko0~sT1jb2E3#I~e>zA5oR36-L3#z81`%w1{-3dYz5wUFzC?LY@j z$JvB&NOj8Qoq*bu@HPpR6Ko69Q2LC!ya@^=+4OUe`<(4;9x|oa`fZT{aWmi0pBYqrr%NOQw>vIPlQHn|8@DNzhS*SBn6K`4^r zZsh>--m#SrAxoak6os_+Y*#TzQeYcCg=&-#J%h3jY|<1IEpoS&hJ24~9~sD6VsqR; zx+k{cEhH_oP356FC0-Pu+%uc{5sFo~3qL{r7q+enWUI1mRH3m~wt5Yc)!1zBP=gZg zKcM^@+e8bB*SSl6L)-6c%3sLdVB7zL^dD@$U5o!DZ?bu$P?Hj?GN|y$Hs1#&THM|C zL*f_PmqEtG&f8n#XnG7qz@UqBiy+v_DHjIdd+p(-Wjve31TZTt?3jBnb@4&P;8dFQV--ev(3mM+Z@}a0vfZhX$BzKJX>TCYEVMA2FhF6HnmWE zfxCtd+P1OT$07S7+rtE;x3f(eA^8$pY8q-%qG}c@IM@#6pu{qFfAf&&Wb@h}#|m4` z4jEi*3(JsVmF;c?YEgpT4HeyN2@jN9JpY$s93wZ+zZ0!?qTiKmcihplx6wJBjG1(ij%r!oY#YB&B*xv_ALKn@vq_fz$r5MFOCjwk+e9xUNwA&u zK{ZNHDxvHdTTlf>lidBPA>TRM+Aw5Iu~kPP-36Ov43ehVZuL-|5=SSY+$CFl3W{a8 zlbay_72B>EvR$)%S)j2SHrE0q%d(Xhp#~+)I-vY5+m#cF=eX0ZLfdz2(KX1PXY27o z`g^u5A0#iZHT_VN5*<5G;eqXO7fKYln+ihWBU?HIIZABmL&)&Nc6bCS%4}URs6`3B zI8=OQt0$mjg}cRbDDc8|e*rnGY=#VEd}T{sL&_T4z%BGmiTig@>5Z*(52fndd5Tc* zo$a**xf*Qq&(QP-TkZu?HQB~$P@57?>rnZVt?vU$x3~*@LZL6V?=Q&x&9>TxOh0VT z9Z20~Gj}=usY8j^-B9J1ZA1oTI@}%0q3|D@tRM0S5C7j1wg(_{*Z&ee2O*7wZCL{e z-E1W-RHek}QRrIAb~z43dbk@hKwcSJ#0Xh>*(5WNR?fCL3rYIe-shkiCD^S{R>4-V zLD7EhCYK&-%mN$z%XkZ5FUbLW&u-nKIO(giHl0n%Fe2P;!>L zqc-eGQm?WFC!r1{ zwx^(qn{CYmW!AWRF+*Vwn`IvItg~eoAhVZk)DCGj*iIaf;A4}oKvhZ@J)10Ez~<+d71Ndu$&^kTuBW zh(Wr2wqhKThS;XgpgJX9oI|+-Hgy_`g}Do7Apap-*9~Ngux;EzV@GWDJ4hB~vlXBQ zCEORG{4v|a6BLhemwbk{PuP@I$R20guR;1#w%VvBb<|H*K{)+dD&X|`P%)S`rMIaIu4bM-^X40q21P~eJfRt-6?*{(H^ z@rF%10x7d>$D`0UCHCl{(k&O+1oY||D_g^Pw)HTisj8E2W4Aq=6fjm#of&Vr=M}0` z;^!Nb>##W+Q0$MpQWNqE#s6;!GhdLc>wk%tKhT(jP1Av7-E0wI<)0dq(A^E?rEHr$ zP`rn`MlZB2W3wwDdoSCA64J}rCI=yTA6se&YEq&~3l$V>2RbOx&)we`Br4gw6Odzo ztu_f6RBQ{=kYbSS&IGk6K|co-)oh7*C^^JkzYPj#*!Jv@bC|8|fQ(u;_X?yOVS8DH zzA16e1C?}aSud0t)FHrq?%xB1)(-2tQjk!rDipDC_f~_vi)^-c$YN*9e?ZzLwuu%baj>0z zLp4fJ{zBPhw%{KWb#nLH<@zVz3fr0#vbxx+GDx?|X6b{ZZnoQgs7{HaDk!(c7FRm;F6j62T- z6g**jy@XtGw)tyl`jjn~g;WW)u{)?u38(i^`HZcv2&I$U1xirpob9^|xl?SbFOcbi z?fDf_r`gPPs6&a@4XAR-Hu4E>a@1LE$SlSsU_Pvu$@E^9|dlu=-D$EZcH7B;2x< zdY~#LPRpU|9NVP=irjHGGyr+?Y>`38a?d8wK-vP^rWTSsu)XV`8YS4rp=^<@Fabp$ zxtlaXz7pH{G-Q2ZQ_VuUGFxa4l0LKj%|mrcT(?2F3R}$%#a_6xE<^q*+uaIedu1DU zLt{0zga?wnu_-p71|{rmLisw|&lVJa=guuc+YPo#0J49u&Fw?_Cfm&cB>!a7MW7}n z#-dQ6#nyWQCBC@ZIfcY;w$>Ts_+fLVAVZt&DGe!p*=DYw7A0hEpkjwjlY^3f+#Thi zfbjVLEus4Xa(4YM!T$&uC2Wl+NZHM{RDr%J@u3Qpq-@4FDAmJV>KzKo*an-BtC#Jd z1x?G@gda%N$L9To+LZA22bC3U3ljG~rTe+dOQDdGP2UT-2iVT~Ad`x%UkRxP*@7yl zLy2uQR8h074MUkB?p{Wqu!hYt26=|rvU+m#cN7}$nap&BJb*P!eqo74+MjofYdAm0?*haa*| zvpIGk-3(iC7m}LTrh-tN5-&nfZkA1b2*u3Yg^wWr99vfmvRT+R;?USUTRj2EtZcS( zs6h$$7f^nIZ6X83ZQLcVq3uOBbC00vNVVQ0I8A>?0 zyM2Mg6}GV&yTlUt?vU;xY>3;p%x{4e?i4HHdh-;dboS;K!J6(*{-#Ja(daW zyCLHSn^p!XeQd{a=$jII`k~S$+x7sI@^jZ5go0aa%NoeF&Gx8;rgzw;Mc76K^;moEJ2k6w)15u6XtHf z1%(gULT<3FnF`IQ8s#0Q3gsx+3<9kr#guByy$Qx%hgklWgWB6g}teCI$IYY$KPD^@1&S1?kdkvMeOMWZTI> zbxQoqL%9r_^8t!oaaVeT{MT$VWyp5Jc3FYOvTT}HNOsE>d4n31(A|LYIkwFv6u;xH z(So+~Z1x|>e$V#sABlFC0>j zBhbJT+wCaSqy(J?Dn7HtwNT=PyTJ)4P+{AhgdDGI?J3AmWpkMz`5W7d8ER4D>>O08 zv1J#a8?RA-1&-XkNiq`GhpXY?c-zQL^2%p&BKQ|3bMDwpiEtpQ0-6 zWD>|X%C;kgtZKF|8KfO!a}GjM4O@8#s#C&@63UOWU8$g$mOJ$rtYEwFX=prn;McL)kD zvb}~Or;Tmy7&0!g?<+ud$7lptPI2<1!RlXOq4_ZV#LP6*6zIeZD~|FWYh*5;oaN4X8qi z#xE%2W4ru@!du*pbRdtPEi8EcWZ7ox?t#=I+h#A+y~Fn22URJ-HUMP z81e?$&PSm6JvN0J((JSCYaq!1+piX?QR4aplnb#{C!y#eck@Qb7iPOPLDnO-f3uJ_ z!WN%{q{nOnR;W%1yEZ5vW&5^6u@mlGE090N_Ph$&PT6MNkS@-4?SW)xY?@7|L5a~V zsE}am6QTGyciRC-OtLkDko|(qaRBL4Y>$V~z$M#s1Zq-3ItmrjY^qZzamC%y85GE{ z^&}z3HJdL58M18kG$g-aTe^l?lz4vwm2zx`Je0iUE?IzrdA8vP$a%+h@CX?TY{E07 zxM%ZJpf)AERiSc`ZQ&hCJ#d%*fc8slx+dg$WIJm?rZU^$52Soz3;seKN^Ey+{OS3b zZA}8DU$}dbLZJ$qr5|#?vSkM$bCqpe0jb{DPLz;PW0Q?S6-pGxpv*hl*EkfebGM>{ zJRfXjJ!EOHnWiB1C)?FD)ZJtoHA7WOh*+TP7n@`OinO@fT7%>wgJd+mN-JZDR-0_OR7E%!i99os`MWSn3#${>ZFEj0kODPd#? zDo?T<4nrvecik#ze~QhkhFnIrcMW8kW?TFRDNSsJ38+JfdIR)4!**_j(q`_4W}wh4 z+x{%%wy<^PAoCpCx)oB*v%T6NVS#Oa8LCiXZUxF%+5Wkp@FI7oZpdR}8`yv>OKgEn zNNs2P-h#T9*;YlUN(s*aDC=M|??I6j?ye6Yuaj*o49%~yMI(^L#U?$0Bx`Kjr%;U& zKhL0?o6V7gqU+q1QjpKXHhl$IH`p$(A+489m4l?4Y~egqr~ZB&g7QAL%?Bv9#a;ao z^849r&ya1K?Y;u(M7GH{NVdb4e1{s8sAxci0NX(mitlpw+k(U(o973z@3B>XA^kqv zLigsM1`gP6B~X(RbbU}U#1`*|5{KLk%Ar7*ZC3#~j@a5t$Pi(3jY9HcwwE!eMTxUo zs1#+(>Y(HacbZ8k7-Kt`f}E#peJ02lXA{kk;*70nf!dU?vH+D6Y>$gj>YO{19okQ_ zr5%v#f^BpaGNss#)*$62o5TZkDADJIp3`g}J}7;~-O@G`%CHr8Aon$!AqbhXY!~~G z>V|Fj5E621p(Ch5i9!_0+_G)Npm3hMw>advW3!$^mI7P;0#e_zO{AgjBHLL8s#1dd z2FgCL1#h89iMyXW$ot5)R)pruY?TtEd1A9XL6T>-n-{1?iQ`o$_reydLD33#vJc4j z%C_?fS*vVcUy$~V&Dn;eHMVjGs#C&Dm+w#ccebk@C|2iAErt9aY!Mk`Yq0eWLb^}3 ztszL(WNQpV4NA1Dpu!j1gBpsrxSJY>#Ba9LKgix@Q|cl858I&u8tAZfO+!se@XkQR zU$)vTln~1Q-x3z*p+MLF67Q^#qnk~?1Q~kR63dWW!ZzfDT9mlwf=az?9XFJea_8QF zf_-eSn~+n+Hs^%!ad&(Lg+|$=S;(zs^XDM*7~5waQfb(h?;&BFt@Hp@ zDA8DkGFrCFXDIxSyOCGOqhkxdL6!-&?mDE_vu!q@?n$=yCRC*a+c%UouoZrw$P{;z z!q%U>Mz-^AXnvYa(FR~P|?k%3P6c` zz{A#a068|;e20+1%T_;v6sV$t~`Z=TOkkHk^W-+iVAE$SAT2 z*N|d|&2s~_DdFuFDhJpW3Q%g7yL=Ja53=bVA=e(;*%M^iXB(_Q$^*7w73xr8y9Pan z*w#Lv^dWaIpHL{wW@$n0BerZCGDq0Pe<9T|+ew%IPePPUCV?uHC`zHs3EP(p3dgux z8H7BiY~>-y5@$0hA@v#Cl?v)ku#JvERZ57AL)mjSi4KY+x!cl1-V3%512msvvrj{s zOSa++BuTSPS)dvvUd%(eD>mgK6wPoKT7rDnY+Vk>nq}K?LfRX)nhTQV*sSYNof7Ug zp!_Y{gb#}4xl8yV{~eor2eK8|_I4rNJ=@P7BrCGHLr{YfD`BYcz&3Xb#Y^1XoIv6u zn>G&F%WSa(q<>=Tzkmjw*>*0WCMC44pyCUgGYcgu+&$%>z$@F#9ptF8W$q!v8=JZW z$!l!KWvE4oy)RJdoz4FWCF|TZ-k{(I+j1RpHrO5-knxkv_ysANY^iUkO$j3%sQkrt zC~W^J)#9$Z2ipH;^Y%inHrsn2Wcp!S9DtM^wt^h$P@;Yqdj4fQAA!=s)Bm@GAvF~0 z`d`An26A_^b+nMVhi!cVQc2ieCn2GiZQck~C^2V(GE%mGvrxE?yVE(yBV!w|LY98E zfDKX)uzlO1?m@QI6{t!H&#O>Y&SrK)ks>qda zH00N@*{&hm1l#=$q|>ua<{{Z6Te1K(C{ghM6%1?#k5GJyyWb~BG_rXrkbRo1T7~o` zwuN_SV217X18P!&t_c;*Z1EP9nB{Kp2MSo&c7Gwq99z3f{F7mx%_V{43v4e^s6~mh z{ZPrumK}tWi`;1xP|(J9qJ*4FY<;7U(at80L5gLz<~Y=*gcTiBcCbC_q0|a@rYUIO z$(EjmT&rxOX2|4XJF-B^H8#lt)S*P*BJ}KL`&fd~>)b6lppb{H=!D!GY=$++>}9)H zhg6$v!(K@6v4wn4g%X8rD6_@3u>*zu+`a8Wo^3YkK4cNu@*zmQ!!~gQbqCnaj-e_g z$YW4;mn|5FB0=tc5|DR~ZS4Y@-)E~_LYf0MO9qmJ*lw~=jS|Oiq1+)`><)^Cxsw$k z-x1qR39?4mzRHmHn9cbDNuzA#SEx=2Gc_oG!gf`MVlnR2pOF8QE%F7~;%vQbNO#7z z)q!LQwuZ3trv@e3d!WKO+e0rDPjWXUgTxEA)Bt2pu_=cj{UzJsFf@>6>rz2YO7N10^!tE&hW7*KBtakR!{cH$a9PwuBLq=h%j3pcW zR~zIku+1$)#(TEh3Zy8qXDMv`GKu}14@;+3v5FBk8G_i$W>-r6(Q3T+fx8i zKC_wkpbjNw4xr~3wy`jju5fo8fkLlr(i6yCW%Hjx<~O#_Ge}iqTTVj4J6kCQRVdMT z1!d}Nm)B7EgS(L& z&366{&Hu0|8jz;Lw%>##zihuPs78tFKTuA1{{NOx{e_}k|4W?j4*bd2&2}q+tUYZ1 z`XH@@E#41Fd)Wr$P@NKX6;NKv_N|0secZW5A-|06c?`1kv(0KD-2mIQ4w4PBX(pit zB}S*9f}E|-1jUEA+cra@f~{$R?89u11xT-Cdt8JDM%bq9P?HkU4ydSNQ>{XYQSOe` zpn#gK#{)UW*nD2dpkb@~Ao)1k(l*qh#QPnnq-8S%q2xdAlKW6l$2NQjIVac-jv%9+ zO^8B@Nj6UmYE!~n94Z^w7S5s66nFUxXy3@DOGB<{wzCXmGO-QbK*|}m;4Rdl#P%KZ zY-U?4Lg`uVUP@5N!e)7b+;eQ%7sx!%HeQ8P3v4GfNU*ZWKA;LEil0zsk?rdX3fs6_ zX+xeRwsHru*x5{7yMIzIvt9K--43=<%kbR#mwg>4C*!n}zK!|N83^ggC6@iL}Y|axX5$5jc6bc-%%_Ja4ge{YV499Hh zOGqANJHCQil-QevN+)do9F&Z4*T_S`Q?})M$QfsQcz}#&Y{oLANU)`zp*AIqyh7!3 zw!=3lmE^9w4((sCc^i-`#rEEWOqXnn-;grRR``KBl&BYie|o-RJMV_l8SaL9q0lwk zejnt{vUU0)^9|d&98%@jUKNmV%Qim(RVXnx3T5(a|1?nej=NJW83!a=A`2|Wi;zRtFJ2*p0Qs~Dv2~v!AU`0vQL`L@A^gWNXTxHYKbKLS;GI;}Db@;?AUm z_7!Yt733Ob8y$m8O17hMNIAkL(Lo(b^y#5z72AgaN{@24G!2E+Y{ePKJ;r9RKxPfw z#XO`MXB%FG1T9-=393+{;D9p!*fyL{SjXL)3-U~`S=S+po-MxtsVCVcd{DQ6?aU8V zDM7viWvAGJyHLc)-OnE6on~7LLGvcIN*L12uvv~FiJ9%@1gcTucpS>jvc(cm)WV(Y z0`kqV?Oa0EdA6@BNV~x1%tBHtTR8{SDPiUg$}h5A-9s@Ocj^-4Ut)`tA)B48_XW}| zvu(XXG6!4Z4Qf!Ly$%&t*d7{C+{xY47bLE-rM@A%i%r>q^lNN~!v3EI+-zMvP?Hk8 zy-;zTt=0!6Jlrh~K!FXmJ2~X=vgwB*!zNo|1d{vMhSX4t68AJvX^X9+g_3^m+!Ii6 zo9%THa*AwoM##9smNP+$0GoCeYE#1L98}(A>$gIwAa?;9w7D7khfswQ zjS(mlW4nw(;ZyELP9aa6Eqn%9&e*z>kUGJ(nS#2{+1}Gol@e^%P&Ua{xPc-U+)d^o zZ;I`_0L@>rDIOqAnr;6Pl3cOXfki3+3}{-(3fPirsPNl0g0f+p`q1-LuX1L%Jf{^&ljBVACj|1|>$7 zP@%-uHwwicx!WFt#4=lR9I`*LIdqWzne9;z4ZN^TPeDyeNKZq>3Y*FdC0@BZvOs|< zTh9XIcw_S|LWUY!eF>7kvn@HG7A4+0p;DdAum&YRxJ#}>!3Nu~7jk~G9rz$)lTFx$ z6klwf9jHwSZ@W;r#kQ~yrM|h#hoJp7o9+m5{ji-KL#7VfU<^|JvIXN%hZ5Tf=vk=z ze@j@qfYM$6OMJP6LfveZ4CL-%%Vr_7gl+s5QuVT(+(CkrO;&^|lqi;{`3~X*I)S$$Q4Ju5r%`HQ5BX>6| zkT}hzbwPF$Tg(mVXW05TpaC=6&L-5Pgw_^RoMm&0P{P99QveFgvCZs3j(N7s0c2QU zQ->kBmF+kJwJ5Ro1S&1E`A?ywjl0Gf6kK9kPC`yQ+d~R6F0&c0Accc1bq%#CVI&8Y zSJ)2oP|C?&_dT?~%I1B5TrRfvN656sw)hMw-E4&l)S*QE8}z)+cK!~fJ=_g7pwI@} zeiL$g**Y!Ayvery1F3v$ufLG6#Wvr4_@@db<|I(Y&-Skm3U70F+7Ed|wgEY0**K=~NkrWcBxa##03 z{y3X$8?v3T-S0rU1lwc~lAW_9_n`(QDh{DSlI`FKieGT|dkl#wHct$)U$Rx>kUq_} za1ISzvE5!kO-j(Ep<;$Do`DkA+zsA9fh^nZE#$aiYu`bJ9Gj~M$#2GauXP=-_ zo-O+VCGWV?RH0yj?W6`d@7el3AY+kD{Dc$_Y|SsIO$jS)s9a)u>_DkU?o3_bKkb*< z(mjytiEUI0nV#8>WRUWOO)>~|DA6|rJy+O1hN1K;cS|ZLRAnowA@>`bVH`5o*e?D- zs&}?wJtWlGLI$WpiNZ9L`C!|afx->$-ew`sC!2L1vNYN9R!IHDHn9YCx7f~>p(-WF zoly3hE$D(GZSH>DkoSjeZ3CL`uvIo8%`cn94@rdA|K9~SB2=Tq@m(m_#TMIxqTT;X zl!YK)58F-{vP#&#B9OM1&3OVzrEKL>s7?tp2`Jykc9n!;GVauukiVZTas}B2*m|>& zZjfy&2g&4YjXcz#MEgBd7-D;PfZ_`7rpk~w%$9nF>`FG}E2JM`JA8u%RBT;!s7VRl z22>nnt2Lp7n!CksC@{u$_X9aJYm$DLaa z1t-{E6_8WUHa7wpC)skNkix*G)j(}ZIMqVsDYpIzC}rd>FbVBXv$dummx*oF1es>o zp3IQa%x0d0I+U1MfSzaB#%xg9!rieQ3eB-eS0ML1n|~EDFR*>CK`JZTvIi0t*-Boh zLW#yLC}U&0+=jwS+>HbvkDV%~P=V@{uv>-lezxx#6x-&`^#S=sw&zdCw!=2tf^-44>oz3YWz+mZ4N8o5 zMgCL>vh_)z_#SuLQb^oqYsw(|0h?nG(udd{hoFH&wrM5Qq=d8zDu&rqV^HFVyQ6U^ z5Mk@lL5^cKpB^$q+3E&Je!{ji4Yeroeg-PV*bEjZdCFaK9ty_Uh8H2{8QZ}UWK6IL z4oGp%=5a!8N_cZYQ0wuctm+-U-1$x+K+>k@UmhnJ_ zUN-e6B$u)sZ$T|e>=mI>ADce_C1u<-f>5xZZTSFl4zN8OLdHQhV+2yj*-}xcO$j5X zP$@C-^Rxa&?r`@?MB6y#E}y{9452;1T{q*SpLZlDe&>hsX^DBF1fN~^gWdVoS> zZ2OOpTf^3Qg3RM=>lH|)WqYkc!auh8cc?;%xeq9#WBb>H!V}z`wjht5ZQuv8OtJ-j zA+>?+yDR#q?kToa2~?$oXDO65vYGp#$TWA?gOJz6Hl~2)XV{`jNMmM`jzW@Iw(T*f zMv0%}P|m{U&_U5T?n-*dH_tXb1z8u^E~g=_l}%-aq>F4}3sk3so&_jxW7}MWVoTiB zmmt5L&E|k?%WU^fNatXiT!UmQY{_+~L5T`4RB*B#_@MYIcfWo}bg_AMAp06ybr;gR z*%tPpfpxaq5Y(gu-4Rsuu*Hv|#0GbRF(}|=+l@nxO}2IdGWghB7m$35?d1|`QQ~X{ zD*4&6Stz;9o#qw_ifku$kaLHvuLv0fY+?yg?6Nh>P@57~UZ8T2?eP^#?Qv(SLHqk` z={n>(U>p5}Od+HP$pJuDrAk7thLhL5cPxRCr{2NI~&3cT-o8_{5gFhV0L5${eJBVLQx20~NNed#FhX-Uq1o z%2s=X5>@UNpP|4T+g$~6)Y$ZIkl~#z@eaxBY(ovGMTvV&sPw_sX+g;bckUl3_{sMA z3ptx?bKS8&8Nb+a5=hZv)Am7aN;vI@%HM4Lawyg2E}($+f7n_|$kkz69feH4Y)@m5 zQh59SmSEOG9ZJmTpyw{Ou}LW1{lCQHDJay#CN)8B37g*xnS0qjEs#pew!8odeQc#g zs6vTGJCu>JT{@s}KX)UmkY|7`yarhY*}6TDTF$oVg}R5>-hEJ&5^UR0R>4--fg;1) zO$H&alI?sSnjc|P96}lu+x`(G8D;xDhH8|!9)ogfwrU)Tj&V1C4*4`}w-=Cgob6v4 z(rVe_8A$q%ZQur~Q^M{ol-IF+-$Ahn?p#I4uV;HMLAFV@*(XS6V7q>SWK(RKD%7CF zXbmbD+4?@9_%wIhpO9!`YkooY88$~7(wo^HJJ7%^+jQ6IpPH1A?tzLHHkA}g%yD-l zg97txJ%f;Afz3Ar8LVveVMxBnwxohulz6X(N;WpbIFwxCF8L1%+S!KnkaL;szyKK? zY{E38SYh+bKy6BRn}y0wwuN~pwaQ)I3hletbW4zHjqPk1GP&6Xose>!E$D(el-PDd z&mOk54Jf_A-ODBv^0Ha{kb9FYD?(-;+xRY|+G0D|g9JaDECf|3Q4B+wZMLrn6c)K# zIe|PoY~@qP5@0hWAoVWWRTAnBvW;FsRZ56lLD@YvNfwIibGMa)ya#L_d1yYwX1|9t zhit_MND^k7Dnm6&ym*FkM{LShC>r4|^alBk*}CeGHOjWpfV3xUwI(Euv01;NIwjow zK>1U)2_gQcSe&~=H{?HKllMZl1lwL8q&sK(>4#)VHn$vVP+~;^6)xE3Mxc0#yPHu+ zykygAAbXlEriJuZZ2c3^K!$B+5^7RHYYHk}vpG#rBFo*A84BF6&CEfL99w1qGTgGM zZIC?Ac5H`Ql-RohmG0R5t5CAQU1JRj-m@)xAZL;7!3!B5*o<3{qQsWkhT4=c5`fB& zY==Q8Rpzey0NQ_I^BzL3XSVkv$n?Ut7=@G-wn7Z*P@?_}dVXa)KZnv)?uJrO=#6bZ z4Y_Mq)F+QVIG2=YnTrj?Mjm+evoX{BtcF-Y3S79NM{l+dGt@-ntfJrwKb zu5N(*18lZw$TrA!KLhFHY?Bs9HpG^khZ>ZqScD1+wu2=oKFrF6V$Th<@T7pbwwxcqnoMn@|Kpjf-y+Y3xwvRU`J;&Wr9SY5}6&sLyfz9v*nXPOW z-;ipNZMXvoHnxzE_)~=vg&rug#J153h3(wE^+BFxHtPUnaj@m(ka~q}Vi@XnvYm}U zRZ5Vnq3kMKPy5ISkFgCsKrKq#dxT1-Y@H`48RyPjfr4jj zuT{vIV4Hh~jOT2*4@i+@(>9?tC7iaP@&#M}50pxA7x;ztFWFjM=YMjg*;XZx>5A=1 z3Mn&e=6D^0j0Ct9V?;G4V!cna_89mW03il?Q6U5o8H3(3COG`CQL5~Fue zq0QD;gyKKkZI>Xi!`3WA_Fp!~3#1p`|Gy%K}||X*P&uJo9Yuv^l*3d z1qCE*J#EO*%jWAq1}R%zNd8IQ$F|f1wJ7nv7b?lv3^FL$&s}l=3J$Oh4?)gBwu52F zC}$H?kYb3XF`~QG{P z4tiFzty!V;7(8kqDHXW|N#i5fgV?r;vAs?c)rZH?!H3 zkY<*xn1Uo0wy7(qMu`{KP;QP*nS-M9+=cRxZ-K4r9DSr%<e7_Ad>E&$v6yK%NBKzzt+MXA9gy z>LlCu9n^inwpxU$l<-`FvMDz66BN1R?)nAtrrE}-(EJr!v<7K1Y|;-%a?Q5=3Dqd^ z^9#yl*&J;sdc$3*1Nm}n(_N`QS#Q}cdmwF|O(lh-cWhx9RHuZVK`38f+Z=*o_uSQo zA%Bt0rh;q_Z1-wNS7MtShh&dz$$wCT5*2!=P-Z(YK=CK;evOd$%;uSa>@RH9Sx8@D zTbPFiUfFJ~P?HjLOHi@O7GH)EZ`=(!p+Jpo*9AG=+1hT%P-k;(K=Kc^mrbZeiL-vF z)L_esQ1X*I%`OydvYqTf&M&sU5M*qziD5|b&DM-SZAw@z*-OZQMn!d85M+`VjuGGvyr zT|7goKDOakNRY9G-k=I43Uw&c&$iKk!UNpBH6hO+oAn#A$l3Bgka~!1Lb&`>w}S1g z8>&)*ycfz2vjzL0h?2XXe#kq*wkC(>RcsXnq#0$ij6f1K+s!Ccqr`CylpAAHpoB87O_J%16%J3 zq?=;fT7_grw#FLNphUX|DonFIc%it7yQwWmoMB6CLv}NpG63mk*$#uyfQ7B=0BTZ# z_Yf-1vDJ>C#5{M4Q7Ev$b{B&jRyO?^WLRWNoI`RO+fWK>QQ}@2DlM^fGEmaao%;p~ zF0;MfLQV(UTmdq!u;q%7!pWw6gxZvF`UI6%+4?I`%Eeuv3hl44wQ7*d&9?dhnbz5! zJ|U%t&D?@Il$dEl&l_xGzfjuC-Emj?PoYgVsRVNS*!)t+yv6n@gH(RDMMuo&Z~T9J1`Pb?YE?kZn^Bb?>pg8=xvB*ruWEK3ic1iX3n^ zX@R^Uw)1&t{*X`%94Ii0yY7s!`&)6Us%{sxByc%-#Gtq6X&#*n_pn+?)={u-N3F&*Nm}OIypu`P#M`b9G zW9xZ=9Jg$~SICfOtG_|=JGP}d)S|@u22?7r8NQ(8J$K1(C|G10?m*55wgchnPsS3P z&;uzR**v{an-bpopmLdQVE{@!ahI1v`_F8;VaWBub~XZ;Dr|#lNcqYZ)Ic3dY-^$C zD%;uwlz!vxWfBV2*epiK{mzy(LFPK!_$;LQU^|(Ega(_;3RNgkv_Y9qwl6yrZgRJ> z0(rjJ%Bzs2#b$Ct>Tk9y57gae8{LGeln~j1vOjDR5sGxU+X_J5U$&1RG%tMoe@n0* zK$@=qB^D1MNjKY61gcTuMHI^QuqjWWsD!)F8RY9_>qx<53%_)Hr8t$H?P+**GrXO->*)oHW z;UAk?0m*f2$4aP0iM^vxX@bo^1|{{}HO8UfB-^qMavInk^pJ6i%{T=qjBKfCs7(nY zW~e;Pc4&c8Chocyp#2#(?;_+fv%N1trdhT{2c)#H6`W9q67_4)^BmjxI+UL0ZpaIT z7TETEklV`E@k8cCw)GuIWn+8Yg@h%x`F*HDiMbGzv9tXhnT*0{UQLS8r9*ex`_&KA9cG#)l-5t3}MZI_@L zC4QEnoR`h<0!25uE4@NKAKP>dvTm_m)*-E*P4x*$x7os9P@NKb+E89(+w4HG9q#JF z^`HC!Hd_y5+hx1&g>*r-Nf{*DV@nP|4N6oDL4|#`gJCFsz}@c%B!<{LYRG=bR@FfI zFx$dEXyAzLb^>Zrg3bUHBW!UalsM*Ya0UuQ*>-0k#|c||4l=~pTvkYa%JyP|T9i1u z43*++*%c^x#+}9m1ruy1ZpeAg*0%u}lWgK9q_|*fZb5BISP`LeitRA~r7pQM?Lqr# zw)6qyx?&p*L#7PdQ3O(6vq?^%4kh|dq30~y#~GBq;ch7jg>r1g6y&~TGh9LDJln-J zq`G4p&Ot(fEtH2UlqlRonR~X42PjT}xj-XOMTjv-`4shp= zLBT<`*Er;qv(24D#v!)c1*A~0Y12@f5>7Kvd6=#L21+Tp3*18cBW$fZ$faUiEkdSI zwx<%LRI{0%pbjNwUZCeOwy`Rd)^K-RgF@qM(htb3W%GYR=6`ITUyw@2w%mq<3AR!P zs!*b_>*h}xJ=nPTf6gw#g1%^|3Jn(ciUs#1bY1!YZa1vM0z z;cjvq@|xMs|3UM!YzjT3v9RqMAjurtuMw(I;`$7fn`f)eLeT~8=I0@wmF?CFSr^&< zEkRlvTYMRkF0lNWzz+L_a+7GemgxsH8hiqrvkSWYI*b6C-*n)jfhZ5WU z&~t=sO%A1xxqDGSp(vYW1ahCSWk(@%jBQ*4sZQBWw2%;IlTAPsN)#ud%o*F)6ckQy zw_<`k=WJy&WJ$7_<{IFnO1#*HayM+s02Iw}7YahYTehwP$eL%{IE1u!Y_%guT41wAp*khp z#i0B>+r$|ZD{_}Ohx`w0@)Tq%vF)WH-6Pvi29lN8+&55z5-Yb*;fZao0L7oVyD38A z3!C;4vRBw*PmunVt-k^dRM~c_P?Hi`HK_Q;=KO#XHSV51p};%aObc?<*)na&@WH13 zh2#ykc0LcKCEN`y zLZM!^{UyjPW$P?M=03J{C!~_Gy}BTwpKX2}s!(EX1Ii4r{qsTLLGDidkVnooumf3! z*aEwdTEX_c2Xzm#t%jf~B|L|rtdh-q3`It`yFP)uDz>pWG(XA~O+XqooAd&bjInKB zLN!YKyn=EXHb)kUj&oPaK|U?p^c`gV$98!SX?1L>5+t2q3zwlfCG@;Nc|F_aD-@gL zuKou34Q#eLWSe5UZ$LUD+vFD{n`TRXLk&t)bfAKX?Lf%?DL%v9Z#N{G**v|FeU`1- z2k9+r3j@%=9NVoNYEpu37%I-Q#YdpT0(XOIC}3sV)j*C#wzd{B*w|bXkbH^lWfE#p z;;a!W+1autD7nm?W)=!M*iPmk=L%b&6*4;6L>r`7Woz1@HYKdAKxG%(<0_O|;YTw5sHMk`+0)Ahiq#VXgp?%6tiC|Tsry#oaw z*j{%bXNhfYA2L3&&PA<2xwy&L%BF?mC;l1erhBKFg4*!M6MY37>4GSExdX z#u}7qvR&4p@E3O@pOB};7XE@P-)!A&NZn@J>_FW=Z0|zhPgP2=^+4GUTcH<<{Bk!b zgSD)j-i+?&kkNJ}KMn1Z3@F z`)7c(GPbx8lJ>I=%s_QY*qw#)18m=OP;8Jpmlg8M*`95XZHR4l8PX}(u2&%0Fq_5& zH7GIah6+lyz6~fo!rk^JB&yh&TabN}%^^a1HQQqV8W>}n-h-NykUoHl8a7oJN{n-N z6oCR-ww@En@sG`S3K?{4^)pC5!M2oyT9kO7f=YTe!xfaAv7Piof}Kq!hbojPDxl0V+m{jw zJGfgJg*+>4o!!U zgu5Lmzr!{Wgkk~i68n&UmrZ^M*@A3)N04rh?dKSh?X$UKP=gXHaj0;>Hg^uiL)_h5 zK;j{rHVxUsY_SZaKVs{@fd(RMJGW4i5?Xgq@tDn7gc4Eio=Q;Qgl*;ta>Up&FOcDs zOxeYm!Y!4mCc)@1uD*j24VoUWvZAuuC zLgh=gLm8AxbJslx?O(BZhagvm?R^+BU9&B!AZ3=VpoTh>s2_)(Z`jWNLFpWKLwYE5 z%eHTT+B;2#j&qEbT%vqsKk?r3S6n^0DbQ$uL*an=C<&iDm zg4AWUZ#UHa#J0KtRVm?l6Usibnf*}Yg}ZAJ@>bZ!cA@!Kw&)(Dsj^8!kmQYRI}Ft* z@iPMDYHW@ZDEiJ_=@jzS*`^ba^@Hs)327T_s!K@v$riqX>Xguvh4M|d%^Vc_;;x>D z{4F-yJ!Jc4yMKUmZMMlWB>Q1YK0^&kRJ=lk4%@*S6#wP!_ZQ=pq!?yvPC;!-STR9mCEKGJN{w)5 znuGRLZ0QBaHOe+>gG_3+BRixVW0R~v9ZK}ALeCnuk2NSg&fSs+3TfGjUda8A&9DWT zb!->gkZOW$H~9VvN_p$ zC6I2FZA%KtTx<;))SyKBAXHdmdl-V^ZtkX(khsp4QbBePn{o`&Z?GMXLjzv6E*;dQ z1g{<{ZnD)3P{POE;xrW4V!NAx9DX*v1u|^2CFUWy$TqYHwJ33K2`cTdb(W!IfIGJn z3huJKx*%teZEhVh?y==IAjLkL)(5pI;nWY657_#5pj3#vz%I0Z$ky6}Tw%7=5M(-H zdkRC!2%GsB>QG|l1bRMZ8;e8fD0jyRD0IRmy@1>?Hvc7LK4ts7f>d#~}#o+Mki3|TJNx?do5if!{1>b_)qe}k%&V5>vfG+UtoMXtD; z{DQn0w)1aj{+dnEfizjRec|y>k{h<)Zm33y>%CAe$5!owqPN`54?w;=+pQe3-m(1~ zhO`B?_y{DuXB$vMbxPRPK=~ruw-$;$aOawU{3W*MNyzrdHfw}*WwvV*Bzt1h%t8%H zjLt!YXSO~o6o27v+Xjghwx%7jzp^=2AbpkXaTOYPW1DtEO-e|6pkj?pwFxEOxjWi| z0(G_?5psO6`2vuk!B!7K@=vy<1E@ub_lHoa$!3T^$uI7bQ7G7A8$N}c-)skGkg?4s zBq7BQn#WN!{#Zf8IndQsRVyb@StMKHIi#c5#x9{>O1`h5 zn`pL zlDKP$hPsp4u41986t=Q>C^MC9ClQL&Y#Aw#r_`K=_S4v+GoZP2?gp};-VC;v9H=If zttlVM&SE<&grc+AN=l$dN;XO%m&2wjpoLuSrmCR+Jhq5hs4kzaryj~JV0&zYVhh>o z0_d4i$F0y&5nDkAWQ)05?SckN*phmo2E#Vq59OD#{R~2(GPcfP=#`SYQRt+ct$G4l zuHf!q3JTxe>)-Rw+*zopip|bLh1G2F7D}jL8(xOqDfPJuoz=3nZa}Mb++A-$BlT?M zyHL=;w!04%H?n0OLP6FX zN;});GxSZVv^VIggKhc)+U(>m@*A4yV(a~d+Pm4F)OA>8J#6(+knUwWiGhA8DGWh3 zeQaw9&~`s}$;r^v0NX?=)H%rZn+8=3v2|ra>B4rO4XI(anmp)^5?282j=M%rUBZYVykF|=BByZYk_)a*s|K8npw7m zPAGehEw%@Wo@X2CgB~e)8-UybTOiQFB6k-fP`_m>9f#_c*tRC2+-0`(87OvzZDtO7 zrc~4-bhOIWw*=WW?w(hm!F9HVb*N#3?Q|2$-()M=fkIns>wC~EB`FR%*=CzOf|hr< z`#XVTm#zC8YT9G-mr&t8TkQ>$aKLtW2fb4&{{cF4Y|Bs3>LGWDuh7U5+vqzK9J76Y zLd7R+?LSb`DckKI^hrr&TX=1>0gg)OyJlmjszBw(xy^!%Dek zd)Lr6rCKtes~fhU?1&_xm3uYuN!x$7*4#!A@kE1_1yR$T*`QZ`oyrIfMd zHbUQ&T55)_%Gu&uq0I{JM%tl?O195VsJ)7E7JkZNYTSc2{-F{{vSz_zsx&9rcrwgq*! zvd!#3Rc&mM`%q>(Tc3j>JJ_C%AWy0Az3{@??_@hUhvvGtE4qSuyV=%mpqd`GlzS+< zmu=z!itc0kdx9P*>3W4+KbwDt76!Ph`GWcf*$#i8x*@habr)8yuq{VIvBPW$G0-!m zM&qEP5w@>*$c}Q?o&*h!vE8IV4dZN;X;A(I+g=6~nqnr8hvUi`>mNK_iwess##`*!tU` z;$^ny4k&4bt+5;Wq~x?0x>#i^9)Q-@xLY5B#@5+VN1)aXw#hNbY_h3IC}oSSdm8$t z)Wa-vwar$$0B!DY7rr}HSQEQ!`72QS9^1+qRJPBSxC!Y4wy|yKmy+*Y=*F>i96;NL z+}$2RQ%7u7Cs5}x+x{6;al)2!38kO1EnY+Fj4gBr-BBVQ+C69cc!XvyxNCWVx-Z$T z-k_>0wz5ws^O|kv8;ZPP%lLylrRE~;!`i=Pi;jlo?zkI>g?jJVUP4ffXKPA?vLD#a zlA-8Fwh|3JQnHZ_xhFQA1uZ;tHbM*_ z`d}-lg6t=Et2NNz7h6(2)bPzV-U#LYu>CYcp zfh6LeRQP@|VKqgv*+Hl*F>$@cUBm1VKjKSMg3?c^2urKIo!y2)W%`+~M} zxl8_qrt;V(ln<*jpY1miswiOVihYRm4`40NqjIlAzsUw!Bnmri8ntG^pFK zC1gTXrEDYFP-Yq1S1uG;&em1{c}m?BLHiYK6$YBCaAkSs)TB)*}`{l2rIjW zEw&Ddu4Nl)fF3D%Yl2)IThIb6)N^;y2K6_vm3BgPjci-pP;L`jdLIY}flTNnD6==DOyT3I^ zy4kunp{5=-zYP`kveoWE34Lsb2hcmE@{gdiezxTkXmx0^3Utw7tk( zV?H!x+0F`~&Ly_u5~yOCZKD)QUtvqFfYd76R26hbiK>Nm*VuaMp_z5=9vY$U4Ys-f zs@h~bYK1bl*a|wJ$ZfWjF33|VsTbPcVH@j*=61RJ8H9TG*gA%xntisrQ7HR>t!e^_ zc5DYz&?6-|vyeMvv-8lx5qBXA^&hhhFGF=FY#*yo?kQXA1{8b7cC`gPQ>uIyIyz_D z*@x@}cbSLK;3eDKG1PFy7IOyWU$YHdK%pD9*DL6ilBQeemJ|ZTbUBcw~$ChTbU^zRGb}XHRU8>M^X#!RtJ?uv0XPnI-0Gb8TzGUw*|V1VasZVwqv=Q z?}Vn}*kXI2&Jf#RA5;;~_BH^eC$KdOq!QUKMxZ-N%s8~0#I`jF%_MV|HUo91u+7Xt zRjF)|i%_O!>sx{%)7YL?AWx}=b!b1G?PL?0%iykP2kOmaTib(bve;4_l%35saRf!@ zu>GAtkCb$sLoSyseBIHo7V@~OxqnqDIOvs}>$#gPfkqnGqROD4k*&W1DsEzX zu7Z-9*&1u1PfAYfp^Jd6xCvTs;ch*E##-4@+o0Aqw#g32w6m#hD5Zm~yBGSV)I&dX z)yY;n1Z{S4cQg!5bhG7;LG3+kD-%#zFI(a?r2E*$W}#n7zUQHvezpz^Z4YpFy9`YY zvW2e%7*^*H+x`YrA#6F@Q2H?2;x42{*g^-;9VK!I?T)g2978i>+_juR-Q#Rm7f{s% zTiG>~Imxzj3q?+`Wq8O_YVHx*pJt1Gf#zno8+e0yXW3pppqe?hrf(>Fp6%=xie6wV ziFgj{k&=xl$StzzSZKj=Hx+{Vm)If_p}J+Zo@6L@h3zpFid|)^ONX8*b({$ut+5s4 zKz5zG)jVi$gDt5LYS?5OFNX5B*nSKY+Ggu4hh8bUtAtK=*s5!wLA%;%WZ_3 z_StMRRCvG^-wGu-w&8Z@ol>8j(AgndYY((~#NBltG;+*VJ_rRTY~gEjg;jjYmN^O~ zow3c2L!XqyOhFgtY=bk<`UQ8dbI{l&Tk|5+dc}6W1et3#vkIl$ux+kG-;_$*g061a zrgxytJMJR)p^1C8UI(>%wx=Vg?18QR6w;4uC+E;FC52be%@f<&4Yd8tUGhCN^};ss z0Cm2y{XRhzZ){z!Q2IOD{X3*S*lNC@J4)OSwEM}Hr(VLE`QmOV66*eDONfE0e%MCh zpv+&kuXrf(kF6~U@|3zsf%et!zxNU<(xADBe^PrHP;Vq#RyI@<#kPNhVwO zG}M&E=4YY8Y_{43C?SXK&_eH&%3p!ba@m&Gpw&F?5;vide74bTC@5h2-i3+_+1d}F zq$0N4L+F!|$`k0Km~HU@WYbjgY9c0SF`d|1oIS=Xbf0Fk{NL8@4 zyg;6k%QwhXvXy;8wu-y$Z%C@yGX9{ThHW~#YGvywgghk=#gJ=bt1E?UJ9kIr zkaVyWR6#)}+e!^&y4aHHA>GY3)(EK{wx4FmQ_|52xn8!rcF6W|SJefDZ;$xz|LC9x z3I^D6`XMvOW(OfX#1PXZD$cO<7}DB zke*A(f>;`w8F_3Js-N!+}7F%@!WVYE{5~O$7a#JC-%eIsTc}n6lA-Bgik`38??mlxN zIbdrmfC9&MT?Cm!wh9C3BevZ#NFB3fRYIPU`D)0Wu*KFv_LRH921w4>-kPA`oUOS9 zG8b$YZIHfXGo6sSV%zG5JSAy;kh^A^8G!5!caZ|gEnD9R6x^{rjX~y~tzi<h7Q1H$6bq$#xw(vQgVd-DC8xN^Jw#p~SQ?mC0IraDNF(msP zvJw9z3!jigvc>&CK@{82A7rB0-Xq?_(lKm7G^AqLE@L52NohRf;@GwmAsgZ@Jq40@ zwpk4Y32adrkV$0g&w_Lk+j9=2lGz&bAy3I^A>>lnic27y%H4V?B$_R?0t(XDCaWNm z&ZcT1ox#>!52;MHhepU#QX4=ni|wctvf13_cR-TEw$cR!xonBOkjY~k>xXnc+xH-( z3fMY^Ay3KeDC7#+swN;?#NGZBB*knwvrtgNwm1(N!xplTE@hKtNR_dDtU{iWmJP_2 zvxU#P3d>e-SGEgDCELzE6jZTg973j=ZSEM-HEhvmkg8=HxPUw*FISMOV{5vFY&~~p z_mDKOl{`X0BiqI^WSZFY8>E}rramAQutj`Bo|2wl$hELNs*kX2D|dBKkhHNK$3Q_l zTR{ji9c-%!knUtlN`_Py+juJEDfvl*TsK>1CS-fKyUT{8m#sPv3i{X%3Lw+ZmRk(z z0XAzOHOLlU4tYw3DmtVA<-|A+yAGvI*&Bw!$4qt+1`_L7tLi2f0%K4#+E+@*>~=i zCm{J?OPq#+Pqxun$b7MV&qMl~t=&TEhwXM5@|0ArLGG7re*?0A++}Y=qSSxz8gFqI z3L^eV;tn7a$tH)8j$(U1hEz0L%NgV;xx9c}3|rYXWMjG8zJ(-?EyF`Wh;8-}GVyHD zFOW`P>wkk(BHPOcty?Vf@~Leov)B|v)#W#K@VH?7i4^OIWMUYIey_G=0BwKSCWTx0IDj+@0W~w1I!?slmc}mh6AUDf4(*)T$?jl_BR0qYiwQ9kXdK* zvyk3kt66~5CflKfJSBN6klSKgUW4p5cL|%2?68e)L%}ZF*DhrC*xC;uz0Y=Y2&n_M z$`i;_vUdhK$CiBw*+cFYt|2*Mi@Sq@W40j=nG?3~=jmbTQ?}p*QfF+JZ;+>?^b>OD zY}?drY=_j`CLP$NcJrqNplG;+py|5jXL-v)s{3=M^*j8$w;GHe8 z9x@+nV~vphWczN0)E8SvE95D;ZHL@9TU8fif4JN4f#jDhrymOb*cJyNqaywt`$NN! zj`$~$QAkCyeT+k%l9nmRMX_DYKsK7YvUx~i*mf47AeJp-88UHfbE}XJu|;n{DxPg% z3-Xk_>_9GoE&K^rST>QnGY3f$TgfpLB(rUtLMDYxUqCvQZR!eAnl0iM@|5)4LoSW& z@d2{w+|@lplEHTT3I&;L1s{;fVq5)!bT(VkFQjtV#?^0Fo|2zP$mOzi#y~cYySq3@ z^4Y2rprC;5APF*sY`LkBE@HE3kSb=2&xAZB!`YB4Vf)O5tl_S;0FqL+>mn#9V=Fh1 zDQDX)gLDO3W+kL5+2*SuPf1K2fem+t1y| zB_soEU)NAD$kuiTnIX0t4{2emc!Jb0+ujT0Dam?=+z8vkCuB#ti~WIQjBV%-3dY&q zBL2cM6Kp{=q$k-fVj(rfRvHg^O12UqH_euw0@)evW;7(TY*86dFvr%H1(|ua=Nw2c zur=gEYLV@<5b~51l|at2t(QV}iMx~vNS4_qtDsB;4u+28w51Add??FiKvb7IGYLD%96!Mf*PC#y-ZGQ@~ z2i#@PLgLsK=b_+`EzUybh)tFuea!a03aJyemJP^La=8V$Q?{~Q$ewYxy${JbTgD+2 zT(Hd^L*|k#`V7)nZ2cFIx@LR1f;=USw~)JGJG+PMEqBF_kle9tJVU`fTk0ERJloU< zq#xMSH>4ifdVV2K$%9h=4gb6+wz??DK67^z1IY_pK?n+7*;W!D^Tw8x4C!~au~bNX zu>GV#o|2AC$bGWiWkdFhyQ(}$zS#~6px}osrx-H7Y}P>fk1bRVDHZwe*gsqec}hO2 zAs4~cS_j$4f0C;PNTS%vo1q|@ZKnk?F>IObkd9@W>x5JsTTBn+DH-U4T!`&;0J8Di zH3=jMZ093TkjPdt4w)pj%}Gcnv*{T~rLaxUL7tL`MaZSH^)5kHbN9FcNg7-IIuxX{ z9dAM=gRO7}(wS_ldyvXvOLmZ_Wc&zn*=)Zjkj>$)^Bj^~w);yc$YZO%flNMI`1xyC zx_~YB0aAr*OHYueB>ok0MQkJQkS*r!^AnO1wzeNAFl^UQqJm$B_eL#muD zD-QCM%*R8nf-N=)vX$Hora)4~_NJkrnyonlGBs=$S&*(}Gr5qeW82DyJSAyGkgI2# zDS>PQcadd~G_v(oKtU7RQx#;I*&1pg9k89$L#l2U}M!WIEY=Kcu_ZYK9=y&2~5pc}nueAlJjTJOSBW?h>XU>0=w6g@S&zuX)G} zu(exA53=1XLu!bvat-p7>}^0!*s`}FJIvj}E+ivtaR*Q^$~JTenK8EaV@Qv)1!s_& zV7t74JSC;qkeg)NzJ=@*cj+FIX|~x%D41c3dV$O=TmKuR=h&VJVsYFPxuyrRxYL)FF74nqSrbBLx?I;tn>)hq% zK(fKMk_QEwY>9=C*nL+%8*H4P^JY+pmLUpDm{m3J%y7 zn<3-aLamTKWDEaBhoz3#J~|;!NlOppj@hpIAbY}H*&rmRY&!x4XKWdxkU3|Y8;A4- zTl5s9F4+cVAWzB59OSOpnie5@&E44&BsXj&t59&uwy_SGJ2t%q>3g=R9Y}e$h<(UY z(&Hfa!1j0q*+=f`P9b?>J3fbkXSRYX$h@$v-az`5E$JRoZ*1cakf-G533Bgjov)Dn z;O_1nl25kkFDUq8JNSXjH(RcX3QPa6*+@wJvc<$o;W>#zR&`;hBK6CP5PM zPwF}a3L@Fc(;ySYwwnRzXtvC3NX4+t=R%&6m;%VfvJDnNHjca35=cU9&1Fy!&vsq` znFKad4e3O-&00t$v86RYo|5S%$R)Eywm>$8yWTcPQrVt5pg^1>4q zkjh|N8-hF~$s>@y%kaeY`5)@ zr=+qAa)WI9J&+yZF1sHRVOt!8f?>9}VaSZI$ta{p+1|$?HOAI51$jy?XCODuRyGgW z3GTKRA(>>$ScZZrw%Jw4OtVFAKzfF)e+yExY%e>Ir=)Qoa&v5F4zlyy6(2*gz_xJ; z1&eH{7m%@RQ&*5)VpF$}T4w9Hhdd<@50G17t9yp*DtAY(kgTy4d_ciE+sYSYHrSGW zA-%~qreeZUTWmj(kf)?026EeMcX5#2;jStHl3ligBq-Qp%SnaIKATO0^Z{Ea6H<$R&Jo+gl(q`GN){rm5@GTo2!P@Ia^E}03742dO)@=>f=75+RVgXX_n-tmp1=43Y=7 z`bj8wWILXQ%oAJT9HgJwRu>@k!j`-Qc}m7tAot4ly9U`e?m9Oid1t%dhJp{a>OIJO zvbh6Df3f8rLF$`r=>+nW#Gga%hi&8%vcKGYUPJQ7)^-O4D*E5C|Jp+);-6H-6Qm>A zc3&VB#g_FBc}nI#As5XS`vch+?gsxLiDi3>hz%=JB-W22vTlOquOSxN^hop=x&O$*s+t4y(D%jpv zAzjH9Y(T1t?Q#q9l$7p5u9|IoAF?&vr5{34%Qkxq1$AsuXOO99>%V|>1KaZzq#D^8 zZy`^~={@9{*oq$^+sxhiGb909>KhcauuXnIrj<>7L%NNv`xjE}Y!50fEKf;o6y!SC zj$$C&$z6U3k}kHD1SsfcOH77L58GHOq=t)-mym3;Ro_6t4%@*UWOmtdA0WNQW}hIn&ldj*c}j-gA$P#``3YIaUF#1d zhiun>P;kUn9vKSD9JB34L;8d*GY(RxZ1eGurz9o`a%XIVDUdzq?o~r_!PcAs1($5+ zS&+G6Gr5qyX4}k%)D2r&5#%YEE`i)FTVxqz@3`x&faIR-sR{}_TYW8L9@tLmA^pf! z*aWF3wzUBAlq9!7?wM_(1F|pN{dPg}%GT8j1#fKk{g8QQs~Ljy2b&v))F)fs800Bg zntC-3(g=N$98c6sSsP~HRLJTx`kXkTe^pA0(UcykR-B2 zy+A<{Ti+XGlG&a=Af3Y2@C~U{w$oq8Q&JQWAC}W>>rqhn+48@Cmy}pY(%B|MP>{j) zmjIbew(ev|XR-NINM*CtrbC{R!%WEKu;u4KHkZ5QJV^4`5(}XqpKY`lG6ihk2GWIW z?d6avV!N$`JSCMikSk`}uY+s}ciD}Q7`DY`C@5u%YlTc1o3ul^obA06QWb11J&>p5 zvJY~VY-NLxt>SK5AgN}{7=?lww%KvW)UriSLAs8ue+E+ZY%g<=r=)Qaat&-}OOS2k zu6PxaCbo@rC}?I&-GWTOHnjuk7B;mHsaCcg2YE^!jv&{@R(A^7cJ7YOq3}yK{Pw7V zD=6q>Te*Qu7hBRjq`TS19w61j_VWaJN;+O4*UNVI4%t5Ls=grUXFK?Tf&sQ1l@OL0 zWV4Zw9%2i{KuXw#;~-DTM?B<)*;%-wDWWaMw8v z$tK(VEEH_9RWCqho6T8B@37^rKx&t5X$|s}#BV}wk8NZdvisb9?m}|F)^-2|j_vvo zGKXvxCy+j3+dYHSF>XrJxf}G5oUy$26eaU7% zA$7&J^$mGS(*7WK%{CK}7?!=^E;1UDTeiMfD7a&L3PI+ctsxQ8p6w(VQV(oJ8uFB^ zr9_$OHygd~zJ zZWs!p*oH1JlpIl6eO@kZ9pcG zt$z#BNo>zMkVxh9oCAV>q zD`cxmfNT+W`$>=#v*n~hK?&Pp8e|MxC==49Y?2MBGPaLg$Wzi%0J(Cut0Kr&a93s^ zsbt$JgMuoyj7rEy zhD;Ni?t^qQ+tdK00=5W&JS9CNkZWOk9D{5tcXgAHw6PseLqR)R!5m~d*j5)H-N}}; z1gS2z@fFBZ^0NlHZnn-%$o6n|w+%@zTlF3k^syZrK&GE9_XyGhZ1x0FgKY8Vkf&t$ z5^_UqpVyET?pp648D_ioP%y$){sfs(w%r#Rf^F~*vXk7s zMkI$NQ*6!AP%zDQ9t)WnHWLr&S+>nYNX@aOr9hsNX$`q~w#W>~E^ybI1<4}YQw|hZ zw)%X?EU}#wLVB64umn;oY-^>ErzE)oa;t0;Rghid?zaY#b+)d0DE!{ozt;8rG3!7IdVX0TP+9=3Vau@@-H@5r`WZ$`4PJrZtEioAiKG{Z7 zA@jxdod)S|w)RX&{jl9;L!Oe#JjngB?H54ykGt$*NL1XvWB;Oof{1^TxN^uuvPmVR zquAc7Ar;NmQU`fTE*l^h!&cS|*;wwjTOf&J%V>v!5Zi1gWa8PPdmx>_*53!IM7Ea! z$WziNkV|4a8-Z*xcg5q7q_AyFLP089>I`Hw+teJS)7aD^q|(`XmLN~b!wTdw*y`3H zo5|hLCKNtpmaqSIpdg!VWe+krY)KB%xol%ckji8GIe|PS9p{kCXS=(EYyo#wH;@#v z9o#`d5nIj!WQy7B6QoPnLa&fAY{T!6r{v=ka;0poKaefs?&=Sca<=lw)UbjIww-9m zRI+8pLAr`D1%%xTW+FT3o9(_I3VPV8hal6-=7u5N$Cf(=seZPl3CL3tKMlD7wvk!L4s!Q7 z56KW)n}q^lyIzLOFk8hMq(|6xH=yth2mbwDXKh2ClKEZ8jj_cZKz5wF!9z$U*xrtz zV3Mu*3^G$}7Z;G8W;552nqk|zg*+u`9&)p6Gmns+<1X?Al6khiHz-(Od-{OPB3r{Z zq%GUYFQk^(iXwDao(g}M54mNwlvv2Fa5oWxWR>kN0SeaGx{@KY&gN4gy}?$K4yjGH z!%WCil9vOyEw<%6$Zm6&PzcEm+h{Qq?6Q3s$n3GTmqU7=?WPh^2W*u!kf&s?4swny zyAiU7+$}Uia>N$b3I)e(L+y|`VSDd{^eJ1=1F18%%Rb0cQaT8^bGB`P>;-q}qmW#( z&5lFC6Zi z6S+I9fFy~nq#6p6**0n+lftGOAf3uK)dVTc7SRHEN_yHLm&W$k0oio!>bfDxU_0)G zf=sr80mx*rtqwstn=NSsQaNnnW00rhX99A$Y@O4P&ExKF7Lt6n>IEn$U^}poDP+rC zfpig@U4v9HTl^;EDH+~|TnXFfE@TaNtp|{lvRxlSK^a^531rIIcF!PP!IpUmsYJuT;!gi7j=~lKv4XHM^wRFf+lAHy(cD9Ke$aZk|n+HiJ zTUQ|zbg|tRL#CUprWDdWY_1$qy=-|^kf&s+26BCD3H6Ze=We7Ck^#1_W+)hBYiotf z5Zg^Vq=l`b3sS>udp(e+BCBWw$UkR9bNb{LW|wxLld7-xGMhs*?9Fa_yJwu>1^ zO|g~EL!OeYMaWIFr7uHvhP#zb>43cxUj7liDV4JOm%q3fN9i*?= z`Wqm1&Gym+c}f~vAa}!d)&|*I?ut7hxntYthJt&x)IP{~wy6O~Kd>o*)FWHZ2;?bw z7=zpsTiqmNpSe4lhUA5Gt-o|2AD$bGWiZA12p zyQ)1%zS#~Apx}os=Lj;tZ1x1we{7+1NU8XL$Nu3<$W!uh4Y>%m);q{X{*zpJNTS%v zpP(R`ZRZ6tF>IOdkd9@W`-D^+Tg(sSDH-^KT!`&8A~P%-&s|eABnfQiu~3l6RuT`H zB(}{&NGG%D6iB78O>4+g5|IJ9RJPtM$ZGB$b0A4$tIvmmbhhI{$Yih;mOwg_ZM76q zS!~G_kf&t43Ub+OzcrA};jXhDl3cd?MkvT*s}3NO&*oYoUBH&x0jWZ^r7p-*65k8C zBDRr!$QE<=IS5G!TiY-c7`E$C$ds~GOhCGfZFdS%6 zQpNVR3I)|{%^Q%ZVY}FZbS<0Ng;X8e);{DZNjrpGJ=@GNWE;4PJcFc>t?vQ~n%JJM zAk)m&a0}^x?c^R(Eo?=Pkf&tr8FH;`DQ}Q%<8I;u3V%1^-`~;S7Zh}`b^Su7lg+EF zuyhw&O%$ZM*$!hMPf1<~ay@Lz36SmOE+H9`KDNR}KZEY(tfh8Do2|hV(dFPzR|Aw#x>{Q&QRtxkWSw#hljEVHRaNUyMUFF|UR?O_G-l+><6ZjJ3|6SC{vj;!k)8IVk}Wn@Fa6x(br zWTx4o3m`qi)?WmvS+uf7SklA2M8iDjC+t?VSw%C3qAWuoh zG~~9~?q(so!(G(^B)e<}77F&*a#kR-&t}&keZUslgp^|&-iACSAG?q{WNSTu>=AcY zhmahzm7hSt3ER#YWKP*KFCl%#Hg^rFbGDc}$Wt=lA$P&{`Uu%e?wVd8xnetigMw?e zl26Foux)-r`j$=qLF$feIwCJDPf0{HQM`)r2fkFBj03RL31WB+wKWFr1aRdhi*l5MvKQc-MK{g9_* zeh_leY_Y?Tjp1%^6p~oBw{a+lV{4v*Oo;7b2Ga3tW*$-rY+H+vrzC9|a*1p+tB_6N zE^-5sWVXI7C`e&@+JQ_eTf;u2HQR}UR2o~+G2|&(JB3_2TgnAwGq{_$f+Ume?*G*QI^Z;viMRR+#{KO7@ZiZ6JzIYRq#M|tn;_N5*4P4hN>1A#*Th!b z3E5`u*1I7I*i!qTpoML605Yv?N+8|F);$8LcD9Ex$Wu}~3AqloqiM)?a+f~`Nf+D7 z0u*$!B`!gxhiz;H(!FfoYmn+=>)3=mCAZs<>u0OlgX{oz`v;H=vgI5>!4TWx31oyV zbPnlZHo1h<2;0Xs$gHugmO^@+EvW)h8*JlMkf-FQ26CHho%N92;_j{y zl5Mu?019^44q73z%a+>#={+{v1*v_u_+H3UGTaZj1GdjW$U5#?haov+yB>vtBewDh z$Q-lnPC@#FEprx9r)=}{kf$WZLhg)ha2c}a+`X3TT0R@+A=Ub4uVl%stzGmCp zhtv&Q+9BjAnLdWxEnDOnWbe4^y@2GN?db{%JX`%OWFFW~?jiliR`>|1C$_a`$WxO1 z2DxXpi4VxWaQFKK$tzpeFBH77-K)Z|%sX366r?}cTnwZ>+44e=r(`Jsa$jr-$&mf# zZX^|wAGWVFDEMV-%Y@7y+f6p4Rnos>e?=aoBK}G36+oVntYXMTvMm_MMsXKg4oNiI zP$d+^u)S47CYCLzgLE9*MFXTlY^BYRr(~-Ia`9~G?T}62Zl)8GM7F3NC`e-K>w`=( z+w%aVQ`i~=QmJgGBao-0XdH5yZG955Y22mEK$6ZjIR^z9Y<~-o$z}p( zw*5QEmT;H-0EuB+e1d{fwzyZwl(ES>q|4dfKOt4Y*75^+N-qB(SIJfuSrnG7;%++{ zl4`b$I4G!Ln~jG|En9RFr0dxFQy^8(_M#zANn-}&8raUVAlt}YaV{iHY#aGd(9D)v z1et(sssz$4Y^n@Wt!zCNkf-FK3UY01b+wRf=kBN;k`A_lCMf7+TL~c3#g^0t>29{M z4oLN|{d7T|l8#=;^|IadL$;5*sv$`F*$#%GV1O-W3^Idkb^_8vY@umL3ES{2 zhukn*tA*?ccUQ}hjIx!lLBSZ?&i_fetFFj)FbKncJ-7@K90m){purtJ+#$F_aEIUy zhdOontasH7%yjmi>F&J-nNhaPO-PTi&1^$zoGoS#3Y7HiLwq zQS>Xl%(ff_sTH>5SSV038V~tZw(ms9t#Q|$49Pm%Z7O6p*s9VYv&rT&A-%P?Asp`5m^QBFOD>_iiBBV{0jctY^EdfXqHyc{QXD*tTmRb;y=g4+TnQ8zFzh z7TXNDWA6G}Avs}tX@~47TVoew&e+a+Abrke`XP0}wjofUByAY-muyp`kh|h8asrZT zww@`--mpE)K<1XMZXVKs?Pvi~cWi~rP@rUG74r9NDI1V`;BIUSl1H|m9mqbhb$G}; zvxNglzpzywL+X`n{}c+8bfuL+QjVY5k)ie)=b zfdVBZX^@X&+suGmJa_5YkR-58=R!7-Eh--}f7p5pA)UnbSPZFTw)#>iP;y)j`4qOI zD#)dBxB3qf&6ZjR*)+EC2FRqdsU}EguywXTDwFN54GNU}>x6t3+d((vvbp=)2T2aw z(g0+0+5QYcCXa1o1k!)mKF1)H&(<~x1xl``Az#2&IS08y?sgp{MQk}skS%7LUxAEa zi(iLy37c#}s+8?*8w!*(?Loeb?P4Et<=mAXK~llCbpqK+wv2PgRI$xmLb{qQ`UX<} z*!lt#D0#Yvd<|Q}6Xa^SJ9&Ymj;;6|vh{3hpO9%_(?5`IWSfXEzfzVhA{q*mbj3ox ziS0ffa?RY;{DGu}?Jya#t!()kGHq5y(`OUi;&2is^46e#)1gM24jdjaISxVtHW zq?@g(1hPGBdu5R6Wy`IEbRV0mhEzXWLM;?18LWr=0NZ;b`e;9J(+&zy%GQrk30oh5m(<#VIv6)#&PqVGh zLu!UCZ4nBTOfExymMwA(a&z2uZ$L87_OJz6$5y)wnFY2Z59vj=fvT|jPyyYDMVR@pjkq2Eu!{eRDOdk2|yw(3VnZ?O4iNNuv^y+MJJg%8MYu_b;( zZkxLyRq`v@Vf%=L>@HhN3}p7$uHqo=*~$|kwa>Pb1O-a6QXzl9HkSsuL+)ZTAvt0j z$cF4O+eCNeUN=%`x$`DBU|SXq@UQr2&A6b z{*6O{lKn}@zp(wCf!r&1i*t~?vHe+q>^s}=5@bHuK35?9$=13KsV}zcO(;-Ou><*U zw%t9*{cx9k0EtTZ|LmVXf^5Y9lDJdIM6$^_q@&ngFCi7p)^q~}O3nl1W7tX`AQ#Ks z<`X1wY#FbRjc1#FhfD%n^cSQP*?NB<^@r^#qV!jRlKN=KC$XKxLN1xRq6A1%*w+3) zHkBNn0)C|FYfGLoT1YN()H=+g>wd3)ynoAXCKVIv`!l7T*mi!#3Cp z1xnufAz#AQJP5f`?ktU*%4cDe!CTDIbC$kef|??SqsP47dhfo<{-3Y0{gK)#W!`wViHyZZ}Bn%HWu zA=}J$cng^pwt{;|x3VoiLaL1|`2`A;jJ`p>o$dPravj{Ye?!vAcB{&MWxLp_q9D`F z=3^k;!IGtX8hkalcGLy%fvD;$LaB`f2QUt~*}g4_~!V>6H} zv;E9Lc7?5D0Wzy>VF}V}Y}KofT4&o|hXN&eTae#iTik)%CU=P*k}bC31ITW(eH=k% zhpqJ#(z|R|=aAZCtGI#!B|A5e_iWjBklW{O?g5emwzy}=9lyQy_iC_NXCs%~qcQ z1xk*yAb-PFlnc3A?pFUo64+7;A$!L*UJRLgHdPAg2e!^~NIkOMRYHN1fBzu=#CA{% zxo7VFHbC;iwqzmu%J!!PGH+}nZIFIv`|NS$yR*~=`J>Z3aM_k zybCB$vTz0Y9=60=$n|nJbO%Wv+s6ZB``KEaAv3^s^$KZWEB}DhAluFt6sX^aSy%qb z53$WfLT;G5*ceDg*aqStJIeNw0GTm1n*`}`wzCvSO|X@uL4lHu49HKirDsEKio2;? zNT%7M@*z9J)>8VwNYAm=l|pKs?YJBYloVD$-m$IzgWLjlDRq!6vW+)Dc8TrB zLS~t*vjx&CY@rQOt8D)|p+L!gH{{pY{`Ns`ox8;WNH*C13_*62ZFmGSTWp_Wklto% zorKg5+x0XQD5;o({4U$BgWMi>*-Makw)qvv?z6?ML*{@@HX(h;_PPzJBeteJC{S{~ z5BX!Z(j&;7aJP8^$thdLIb_e+rY|9L&K7+G=?k{r0I5s1r+X++QvU?`E4Gst$X#<+ z^bW}l+uA2&Z`o3RAQRXoBC39+@7Po{r0&_eVxd6ET|DF;*lPYj?vcBLWJsRa@-<|i z*_P5F^TL*t1?g9|ksL_9v3=!1fs(cY$iK7Q6hZETyUG$sKH2ulAp6CZQwf=GHdhVl zAGY{fNGbjQ**{nh1xnr;As@lk+zh$M|0Nf#kVLVSbwDIN=kd9@W>4#Jt zTg)I7DCrx9d_3FpDC82jYnXr}k?nK}vVYi$XCafswmuK(WH!ABsT8)!WhhV*u?G26 zw(bqcY3}Z~AW37Z-GyvA+o6X{23x@)q%+x;k0F)CmV5>UN=7drpUw7t1-TsV+HWDr zWxKtDY#v+HBV_)v`DaMyv*o@)s(@|b0}7NRd_%sFZAexB$`x_<9tlY?TT2XN4clcL zWJ=h|6Cqv7ww(m2GPbN#C{Qw+2KjQf*i6V(aMzy=NhRA$E@Z3N8uKAj&30A@>3?j- zK&pmqqZA61q*Xw^mTjsEa&_EA)<9Cv)>8-B2DXO=$TYImH9^|49koEJiLJ053Y4sL zLcWP}?(#rNT0NFOSjv>gjvxN~zcd%8DL#mT)e-a9m$bu0y(??P?QJ18fyLP@rUI5Awp6eE_*Z?&gjl8Dfh&h3qif zz&T__*j_IoJ<4WpAT`Ez9-u%;$phra**2dbH^E){EA;yne!TyChwKzv)E8u?*?NB< zJ;U}G@$Xk^maRS-3X~kjLVk{|C;@Ww+^znB#IdEOKz4y`TtjA&O=UoOiLEmWQp;?2 zIZ&YF-(SeDupJaYZk4;g#gMGAEtNoao$XIKWH#7FDj~he_E`<7Ew;8=C{S`;5BY7j zN(;Fi?sl6Y*=5UVgX|vLdIp?Ly{; zP47ecmTlq?Qh_bv1PYXNok9MN?fwFC_uSQ7L-N3OcnjG_w)}g@Jh3f5Li(94=><|R zY@=^bpycZV@~>>|-;jIb?nc%8O5WM3q9FUhwig4LPqy56NPn@pL`Z$JB_u7X8RTQx`YRw8 z$K7)kB=KyGHIPkUJFSCEBAaQ1^dGkMCP*c*rL{tVlF4?+C$mL%K`w>6?jA@|*&g~J ztJ!J=GHGl_Ly%5qD;R}T2HVOw6evlaf_x_1*bL;dxci=iB%7^c0kS!4w@Z-8WvgC= zbRL^uhtyxTye%kDvakdBe6~anxdQHn4j?IH`#6GZ5nIbCWQy6Y&LM5s%C8_*!nSh* z1xm8+AYaNh_W-#v?qZ)IDQ6pag=_`e%R6K$+3XjjtJuzdAXUv)5?TAJK*>flF)krj@O;9MWxUp%PN)ZGc=ScZ(L1F19}{knLt0Zi7q@ z+h+%)d)ZpMA=Srr-3tXuDh43m&$c@VxdHC7M<5Zl`7y{2vc*k8W{6FuAwA6YIt!^0 zwk8J!O3oJ{Kgw3R0=Y5nHrF5-XUo`x>;&8NHe@E*qW2&@#n!tIscE*SLnu&Ee**a# zwv#i+&2m?C3CSGW+BIb7*-`^!9NWY_q!-xK6QmZ|x?Z3_$=w^|m)L4PA-Bxk!8ar; zZ21v&zp|@rOHq(nV@ryK^g7!}Jft?*z7nB8Nn0}HH`#7dA-Ba{WjZ9=Y1&1j8QXI^e?X(B7mu$uTkhx-87f4^T>0wCSuuYCafs%*` z$ltPcPeCqlcRvHk9b4@@WbfGy7a;S%RP(Jjb7vwiPC?uEN{ z56LUr?Ez%p*s6}9-|s#9|616eLi&R(_X1L%YztRVpd{fI@?UI2caZz$?)?FhAGVff z$g1@J&;H9-$VB`vRsI3#NVe@SNJX(_srp|9N@gP=AI%mU1GyOP`r{ypWqV10Y#du- z5@h1p&Qc(qz-H1QmB_Y{0R>9ZvLXM6Z7LUXN!&%|Lz2waQwZ4o%k-*z6vpD%sBWp+HH=5#+1bHcuc|&0YFA^!o^P-v3=f zwuUY01~Rp5y#dm7Y>)Sls%NWzf&wMSFOY9wD|&}qBX_HxkXW|VAILVbjYl;6$~3d7 zXh^rPb;d%fmF+Gb3e@k17a`xqc90CYcJBUaNIKY-(jnW)_9qK6U2G#cknU#t%!5=9 zTU!AXD7h|zd@oyN3FP{?+bx5npDm{nvIA`M)sPXk_*zI0vPnIphS=U3p+HGfGvtTa zE?OZs!d+@3^bJY?qB^dh9^*(R1D<=7(Dpg>922ILpm?zbSf$X(4YBui|E9OEW}8ie0wpn- zkUwGT&xYJ7ch9+yoUt|LL-w5Qv=A~EY{o$Pl5M>dQdewg6;PmLvI_FoY>_pPyWy_8 z4w758hX%+7w%R7h+_4?CK>D7opdC^VY%85mpd`5m@{ep|eUN+N?t1`|XSR+Z$iA@M zjzH#>t$G~NZ)|=NQtxbeGf<#pVGi;iY>5ky`{ZtD36d|ij}^#%v$d>4=7;TS6VfW< z|Fge*2T~FLOYQ7Ifs(8P$Valx9YHRNyVz4mqS*${AsfT?atWDOHhTlFFDwXXx77COUCO}@Z zt^R>r8h0rvkfgJXYshA>{iH)CldUrg(phXF2U6K=|NcUOlKle6=dk@PhFmUpizSfc zvHdBB>|eIwO3381eO5!dfUUI_QiW{S^-!Rs!a}}?ZMPY6#oT4LL1NhEJ0M%a7S|1# zQa0&@bQ#-gKcvdpng*dj$@wtkE7(fMAXmxV<^&{FY#Gy#t!A5^h0H&;Xb0&Uw%$cZ z)v`S;LxGa|HOSYoooqm^p1Y!LNE+DIb|Krymbwoa%QkTc=_WRH0;y)Ut}`f5a(4mw z7Pgve$hC5Ja0^KrTmC&{+u4>LA=AN@^aANlwvjhTb+LVYK!K9BZ^(DE-KeHtxgPE+ zqaf*J+lzs0A6rg5Wct}$BBTe{;*+7@Z#n({I%_Z$3Y5I1L4J^}ITLb2++ActGR#(% z2iXy}t$fIgvSk)QdW>zxKx&*VrVI*{^i@E9g6+8qa+BOO)Ic)Dc3KD7X}01<$jq>< zH$i%qO}9d7j%~6X3Y0{2L4KaCy9aWPyZb&!7T9V9vWskoLy%cwD;S0JGTZVvq*mCH zr=UQ|=nUjn*}mr>x5i!j0wn8fw@Z-SV5?e%%qE*(hx8U(?iQrB*%o%7KuLm!{0`gD z0pxbMdq09?kFDhtvYzeo95VZCi)#Ib%Eff%G|>iERFrx?tOgh5{vNage`cn@WJ(6?c(IkX*C%q(JtD?LkB4 zmaQ%W(t+(L3sQG%g}G3mWaTg9@7YocA@{)DSTQ7zY(FKCePZh2J2HUP%40RSZCZlAS@w ztIYpj3$jNb7xBMjZVZx0wzx^iMzIY{LnfN-br#YwY}P?4mhF5I3Y3(rKt7Iba}9Fw z+@)_qlE5~-4cSDts6EL1Ve8$8bQ0U+A*7Po>QA6R$?+NFQ`m|wA(zVC>NO;qEj2(k zjcxoMGU;sU3DOyCoiC8eWV?HV0ww=GA)m!|@C~_a?*2x!{7Q1zmZBh=%l0Q0GI?wx z@sR$@_L&H&e73e^C{S{p3i$%I%5=yTa<`iaNfBF44rGhj=JOz9*y0NyUBV_skSb++ zGf<$UsSNUEY!?-fE9b7X8j=dOts2NyvSrjmriyK*5z^Ic(an(h$JW;h1xlXUAz#DR z&;_|#?oN6jsbeedhipCDnn0$3O%Fr5k!@lWQkE@Z0t%FLO+mhi?S2Mw&D_<@L(;-_ zxB%H!w)|ztw6QI(Lb{zTX#-LnY@=IHpyX=@@||q$9&%mW-5fyD%~o{`*&epNQ^@qP z$-B!g_1uaF&LEBk=VFx&PQq(|5?RqLBKuOjTrCUFkbYtd@sN6E`}YS5l^s|V4rD&qKJy^`$<|r`sV}zcA}CN&Q3CmIw%szw{cx9E35m-3|LmWy zhHS+DlDJyPM6yXeq@&ng8zB|V*3=9IO3qs$AH!DK0l8T2HoG8+W6S7;Y&_d^KV%Zv zq6ZLcWBpIRi`Or%pO60ge~?Ia--b! zpF=Xn_Hqf?akj=A$V{-E1xQb_nFmNsv28p-fs(XW$WOCPy+dw>yT~s{X4!gvAUntQ z5Yh1~GtX8R4Qa=A6bq>Zw!#D`P_ps|@{4RKDUe&@ZcIb6%=VKG*%h{qEXb^~g&at) zu~q+t)H>UK0Td|7D~9|A+hPgiHn~eIhh&RwxDvA4Y#-H-*h5{ueV~{^(+nj*h z8F%T^kestk&qDTsEy_XWlC5_U(pPMc%aFQet6zfxCC3|(zhNudhTJW8tGkc{w$y#d z-m#4zLgt=Lok045t@8|0k8F1rP@v@BHRPYz4sId$%-!F6NM6{M9wGb6_U8pMZ)_uP zkbY+%_l-eW&eNn$0tKN;(v*x zLMoE&Ee#5kG-X0QitQpBa?#wC=0Os}wv`XrShkEJ$i%VD7)ZynMVCP;fvv9s3Y0um zK|YbKp$2k)xI3wXB#EuK5wgi_YfX?zVbiUUPGy^Dhm>ZE=z;_4{CAIQ|OnaHkR=~}k+Xh_wu zrNu#k`h9o|%9W*lsH!)5%u-57J$1z7|s5Y9zK1Qb1#-RI4YfhP zmsJ1%o&M;6Y(HB|H)ICbu6iLYY~=%x8f4oUgaRd5Bak0rn;V1NFn6(&kc_YmOha~* z?PV4+V{FzzdYtWS5mFOuB`Z*%WMd8TlWggmkelLeY8#Siwx~VG&an0DLuQuk@etB; zY;`A)nrAycg90UmmymaCtJjcQ;4URVvdA`m57{NQpGU|nvvs~edW9{#L28xl-zO9( z+5d+88r$E9?q9ie?iQmU*n}_^6+sy&wKDetqhUAlN?-a6MY&jQ@`DSxhkp5wdzlD^_`TyrRcn1Ya z-X0(y!Pfi?xyb(|7q5^+v6X#5Hkxhg3ojz&F)>h}q%RKg@odit zkW1jMAqkR1w$l{I{$VRlgG>_JdIqGE*>pCfQrISQp+HGQKIBu`x(gwvxw|ihB#o`M z6td}ThvksTU@NGCbSB&KKS*VL+1#AlgP@p7X2=aw&LnDwY;_iJ6l47=&Nyr+u%W24zu$9k2x|D6(L8^=` zYY7UJ%&tJboGo@8auwY5Z$eVZ_OcDxDz?Tw$W*hP?L+z>n>m714co>E6evkMhkPyD z)FtHVxQo1jq@JxOK(>MH;T|%LY;{kNwroc)kZNKpe1`%hE1!^WW=r{jTnl$&5xu{X zR<@rg$hNU{#6qT>EyP2*gRS}xq&nI5lc7LKo`!rE+hRK8y17ftf~1FSI0v%5Y#(`$ z>0@gxfOJ3GRS~2H*eXh(K*>%SZz*~z+n+PYY_N@7Kzfty^9oX1Y;Cttpyc`v^4n~ckC5BpZuc3IUACMz$nLSt ze?Z2w#eYM3pG{QXuhaqCTO{=ROsoIzEt+B=f5>(b2e~7*(nLs(*|w4(d%~8H3Yk;3 znKVeBu|;P>>YS}F8w!*>07pmDo6#k zh#DwR(p3lfJGT1<$lY^S(*(%_+hGf2AKCKTA@jtx+zIJtwxk|Ny|9h;L4lI50m#3y zwGTn=jk}u>NZ#41#v%K`wl@ixPqy3{NPn@pIY@o8B`iRJlEEd&|FFHUKu+cU|GRBo zha}>Esmo2sMzWRdKqiW9dk@mlY?%j;ieZ~Qf&wKmr;v|j>pzEF9Cy!`ki@e!-as~i z?KD6pkclali4D_AeX{j_YWkgY!4Crzp|RGHX1T%Y)7$> zPG>7ffK&$C${#3DlAHqhOtvu%xh(F!(;>-b>&Sv^4%=-GWOCW6|3W&C%@;uGFI!$Q z6ew9JfqXt&VmagrxErd3em~Oo|8?C*HDrs}T52Iv%yv}|X~R}-AyvY*(+mYlvf3bD z$~M;lxiaozyCErO8|Z~>1=~wMWGdP0Af&6<&W0gX%~mo71xhw1ApegoeHwB#+)d3w zQp**>0 z8&Z92*J|Kbfs%?S$oI4D#z1a>yX<&Kgl#?%vV&}K$&eXhlT=6#v%RK4YJ{yR6AF}^ zXG4CJtuzmEW87`#Lo&{mQ3TluwrK;INw(-RNKdi#RzPZ+?Wqb1l+@QieunL&4sx^H z6*WRK$F|l4*?G3qR>(NEiFQaYu&FLcEwXj>K!K9GKFBYz)d=L4xjPtwWQ8q%6tb&q zOXHAPV@sNX^g7$f45T*LzUH7nN!tSCH`#8MAh*R`jpB%Y?*hEK4F`AfYd2l%rg`y z>3fCz8Qb$aT_aw%?coqIscdy8kk)KRXOK!`E4+jPB`epE zPiIRBkjvn1>>iR#wx37HX0dg=Kqi|lyg@pLt@;yExorF2P@p6)V(?c!k8Lpua(}r? zjD;khZ8#pX1#BOQkSSzqO@?$4+f^#0irFgCp+NnfVT-?o^dOtuL28KY?EwmuG(AIpnC;>fawFW8en2wHw)F+s zF}4gf^eZ#YHWLZy3AX4MNKLZ!#X*6Rrv%7Pu{9(?ZkoH36i8;+iqjxF%eIyQnK?F{ z4e5EdiCjoIwupQvP|{Th`31K7V#qCWS5pef65C-pWS80Ut01$&w)_v$t87VikXmCK zZGZwLUl#J~Z0#+O+u-h|4U$c^s!qsmvF&w3W}7Xy57IkqZU9odYzae9pk#0a@_TIW zW03RQHBUma&vrQt*#ow+ImjHcZ97OGv1Kkn>X>bI1qzhJtV8~Ut$!18r`$bnLvqH} zxChyDw$pvcT(FrVNMEw8pFrx0E$ti%luTYi{+ccF268vtbq7dp*&gm88`x@}Aalod z^aAO7wt{y^J+Q5OLV=RxAILwljYSOq$~|%S9RLDA$_RfR4Uu?6ci{aoQ1q*Tb+ko8h0s+kfgJX zFGDti?PnD-nQWaKkj`QYTae0T`?m`PO7=bEbJ+eKLN1rP#bZeF*#4YB_AlG;1!VHs zKCd8Mz}9*TsY15vJ19_6@d)`Mw%upQ6?2#U28m&t|A1@>TiiEfO4&q>{7RRxy+%T+ zoUJJa3Y473LB4{mG!b%@+-)X7QpJ{$3fXG5=`_gvV~fs&bPZc?Hl%9Vo^qi;Nqs)# z>)1{TAy>~`k%6RvZLJitjclnEkg;qNRgi9CQ#FukX6veh0ws41kZ)nDX@XoUcLyzy zw6W#4L$;l5sS`3CY)L(k?qnP3gH#vW*8mhKX&ZukH`~n!%`y^6exLHf&3s_^E%{)xVzYdWSFgN2eKn~LkeOjye}eQZn|_7V9NXkO z6ex-Kg8V#N_YdS8clQyazmf&E+GxlwvK_`kW{IsJ0n*EC%YPuX!j_x@1xiLWIUU3(TJ>uk3+9uf6tb(54i{K#ttEQWcxXW>=RqZ8DyT>!Ud#X*s8A~^~$z?3k6E@?jiri zw)hCSckU8jAo*Y$euL~M+s6lFzSvs7A^pvErN(}xe%LCapg_q^4CGb*|E~qv@sNx7 zUow{nNhDibGGwFJ22vpt&GwoG=@>Sf38`4N^K2+kQj!PxIJV7v$i;J)UIa-3+q8jf zB3o1$Wd5-ARzNz5?Xe0{$!zsCP@v?v4)Q5%MU9Y47{1eD$u^kLSE}Of*qmbmVEsaAqm+j9KWb)WXW+45S?Q;%N`D|?q zP@v>`3GxMOm8*~|*DUF5|VDVs(+B}VcV;POfOq*1El-doP|_B zTS5yIC>d;n`~cf~2jqmi=59y^*)Dq_JH%Et0GVO7?LkP7uw{-wYLsnu3<{LQOhSH) zt$!MF*ZLNdYD=pZ}EcDe|eDK@hL>1nq0HAu~{rENlilF4nz&$31CL2iz_?tMt+ z*&Yrd>)2{fAhW=BbOz}~wt`DYEwQa!LxGay0QqIMv3tm^aQFQP$tqjN3uM>WZr>oY z&Q|>i=?ymj4XI7GyoiZk1xgm8Aiu?y7z?>=?uOza*f8 zjC>mhf+-BcqamuykZkiBB-X@$%++haSVZ`kU(Aa%=j+yezl3i}}+*j5E{cig26 zLvqhHJ_^|fwx4mxJhF98LHda;%s}dy?cY2UDA`|t{0rOPWyrm9x3~()8{3}^$iA}; zZ$ajR?Q;jxpKPrjQeSM>2T-7-;u!MZY`dqB`{6G80uoj5|Jgr(1=)!IC2_ZqiDZ*I zNJp`~K0qp(t?3yGl$^gpK8CII19Gw4ZGJ%#$CjZce`Vv@rXwMfz!n_?=|r~PI7t0r zdrE);CG|;=PhvYsfm||oMQM1o=}@2~Aq(<-Y(qJa>*wx0 z50U}4mIBBM+hq}C2HDC>AU(vkT?VOPwya7hP%>K$`4P6*TF8xZ*Iy6G7~4xDWXIVW zn;|p7cGe2%NjB2~sVTONE+|lv)(iP*wyA!|&2Sev2+1s4&oE@?*d9h9GtX8x0cpo} zGzF;zw!&E`P_i-)`9-#rMaV62H?|DPGTYB8WLMZaHXyUg7PcV0##X%xsdct}4+To{ z4k5q6ws;J=P3{uUAlYIYzJTmD+s74TcGy~PA-&6XbqA?Ewu(n6P_pw3dC!*p2DyFi z<~|@fV2k^P>>=BLn*NnJVtb8*^f8-_fz%1xc^nidDM^I1Z8{CI z7i>|Pkhx^*&4%<9+hZ=IuG#AIp+L!TA>?n^iVWm#xmzuTB(SAcK=zJpyb3b+Y^ny* z4{V)vka}dhYk&eJ|C%8G#CFgExo7VFwnOs5w$usPSGGSrka=Sp>4Wq;+vfnJKG@oZ zpg_s>2;@K6D#s!B#og{CB;RZ~Gm!mZo1cS>D*XTKk6(av#Qzdmf>b2i+X@sYXhr5#i zNfKM}17wrg)}A1f!lqv#oys=x4k^tR@dX7+x_%&^#&#bu^DCFmT}?D38El8Kkj-Sv zPk>Ao+wvbsXR{@xKq`lAR6~LK{jx^L=d!hDK`xKGn;b~~vQ_A}OW;>uj zNlZ87E7|&cAy>uSb3Y{2Y>k7E{l|7X44E1>GY08ew)F`})v={bLxGaXS;*J3MLNhe zaM!&ENh90CGGr}V?HXj7*p4 zJJ@b7Ak)cKeGTa@Hh&ALZnnI8C{VKS2>Bki#23i*ayRq_NgvzC2W0!%TD~DOz;>l( zf2D=3JPJ~SY&$Vfpd>3E@%*(J80PRJ~?b@o7dg)Q_!YL)GuK!K9|A;_<>{T+qeI(LiXkZiF1 znS$&l+wcrzw%9)BAid4jx&Wyiw(BJ*P*Sl9`CYc%b;#{;m%RmvXPe)F>^@tZhs*(+ z96&E_nmf7s$%Af<}<|9@>zpyaIs@)2y!-H?m?Uvkk4NfcYz0A!=t zwgw>+!RIPF8ljR`m+3Yi<#@NnZp+HH=2js`uHoqV@!Cks?zmiF| z=}5>r|W}2-x4$?Dhj|q^PWvfqu0wu>Oke_2KN`u@ycdHqYIJVSm$S$yr=R#(Y zP31#+iLJ8`Qp;?2#ZaK+Un%5Q*bd4ex60k$DoEDYmi|F@o$XH@WH#7F8X&#N_Guxt z#n#pW1xl{lAivF4*$KHF?smH&*=5V=gX|vL`~YM;Tl^5D_t|6wQU`2rV^E-^X%g~> zY!}mzJL0Z%4w7TGEeF{Xwu~jnoU+ZVK>Ca=dL2^dY<-(hpyX*A@)v9kdyu>2?qna5 zE4Jbz$X>Ipoj~S>O`k*hmTlq^Qh_bv1`3pP1<2pA-QPp*p1YbSNFLY@Um*L)mj4c! zC${BJNI$bB{Xpu4Z8T!xSAmkRD9FFEwZ}s4jk}w8NZ#41{y_GFZ7&%zpKQ4r(qC*Y z9a7(H30Y8}WH1NvKWy)LkW zkW1mNdl-^bwue#3YPQ-5$oxmrT`k4Bt5F!ff_=}yZ34j^f;$9fq95)M+#%S+A-Kb4 zta13%Tci4cXN|Ki#pUF190=X>izSkhhX6xF7Y!2J) zHe_i=SCB1cYrBO^3ER~jqzzle zBcw{%cAud@N!A^CG8Y=dg)SGJPvB@!}KY&Hhc)of>Rkg8!TO@sm^ zn|~l*%a)!BxjOEq(;%s5i^_y-16yx4WE$BXb0OWt)=&VcX10?eC{R*lAaB{$${^Rm zT}mY+t!xw3kZoi8sfA2CTXzGbJJ>=Kq&nGZTcAM6K^x?|*z!9e*UjBhHza@Al6oQA z!#2_nnO?TfK}h$pwGTt8pY3`S3Y1h%Kz@L2ZwhjQ+-1)~B5VuukR4)+TZGIon=C_m zgza?|Qlo4w8&II+d<*hpY-PKU8|Q9oACd{Sj6=vyvdtVrW{NHP6w=ddedmyxVSBoS z0ws+%ke_Ay7a%vsUGW1X^K9!+kX>L)eT9r;n|z1#BAfbx)Dqj@A1F|A7qR>+zsyz_ z4Y?KW4r3u%Wh+R4>>As05@god{-i*9gKbnpYLo3N9SW3mWI=w5?Is6u+uT*65q@vie;-NsvTq5M7*c2vLq3UZ+Cc6Pcai0gB(wEaLN{^Tn2aJoseX*{d7Y%i><2{GTCgQAJRE&H3F$z zwu50PP?9$W`8>9z3CQJhmpBbc0o%wdWDD6o<{?wW*1iboVz#SgNR_Zvu0es4-3`ba zw(M=lm2x+~3rQJUoQG^V+u$K&D%f6+AzjI4Pa##sc76^8N=mOFU(L341GyUR((fRt zWt(|`Y#m$FGi2)7`d%U3!1nkKsYbTOFDOuQ@&oxMw&KXuU%6)P)}kS?Y^iaOZDE^8 zfJ`f!`UB}Uw(b;2wX@x6C{R+H0r?KL!z{>ka+jYANf+C4K4iPul8PYnmu<8J(miaS zrI6}n>!^SNCD&Du?_;Z~gJ0tHGI*C7AFmbeMIPws}dA^Bqa*oEvjTbqZ>58Krt zq*dvEXMe>Bq$2*4+C7B=C0Q4ck7S#_f?O1LvA2*!vkl%sHiqrx0Wz^{_8HP~Y-g{K zif1eRfC43(Uyx5=OIPc^a*5ncM?#Xs78L{8KWx2mkV$5HOn`I>Tf-knrLvu*K!K8? zG{|eVwG7CmahH+}Njlp^E@U&*J9Jc&w$mMdk zR0~NSTT%mL^Vvq4AXC8hX(3(6*4_rGBDU)eC{R+_4f$fWy&lMwaF^W=iD6q9gls8W z+%ROy*klyamcDY$aWgnPuDf z3+XvF-3O_8wy6OqP!cf&`31I~5y(02?#H0tpV;E{-y~$0*p8+lv&>dF2k8~Il?6zx zvL!D;fs(Nm$gi<|uR(5|yUtBWHrQ^rA-l;|y$6{sHt!+5&6axvsU5b(6DUxUa0dBZ zw&4rN?Q!>h1<5{J+bv{0+vOc(4%jLlA$`cU^9-pYwyZZOP%`%c`D3=&Z^)f+H=s6u zCI8r7A|ZRq)*J(wGq$rhNT0KrL`Yq*ZT^7*C26UUzhs+EgWMH&k(rQOv-M^}_J-{t z7c#eO4F!-6Y{x~Ax??LcP@rVB4D$DEDV30W;BLGcl1H|mTF5@Jbu~cdnJqLy`h~5g z1yZkU2W?QGB(D?lZ){85kbCDYu@{mLwvm3wezJWGLgtIDeHhZ;Y*(X@`eCb_fC44E zQ;=6>|J~=aXCW8ypJaX>l1R3=MaV|64K70_n(cKJ(lKmy15&YU=UY&qq;wbZaco=r zkc;Oo{ScA_wwYtdCbC7HLMDl=?;O&9*d8w-mCV+70|iP>0_0QJiXR}C%H7%%B$_St z6|!k;6Yr2oXH#F0&S2~Qfm9~jUBuR}0wuN4kk4W}jD=h_climB0KEy6*8lD|ukZ(+M%fm|zhb?cC{ zu^nwfwwTAgMvhCkO zrjIT69@716?h#T0YzZ$=pk(L`@`G&eACMF7TE8I~V!Kq^zp}$@WI5#Kx$CKfWP$CW8nTY9z78^rY{w0dUScb3hSW0KY6}!7Np6Sy3fp)muk6EklA3X5lC;c`C&+HvE_|Hfs(}u$ZxYHPD5^oyWv?#cG*7WA-l)c zwg{PhwyR}Gd$x)-NFA{4Za{&OtZm32vd!;8?uffs56Lmx;2~sB*j|ny^N-D*Li&{L z>>N^OY^7IFpk(s~^5<;ncaXc_Zu$X|OSY(I$X>DazCz}j?eQJbH*5`Gkh*0%`GEo@ zMUgwd@_}tF8gh5srNlvU&o+?&*$1|tB*;9nb*Di3i7jYIJ+swjK!K8jEXcpG<>x}~ zmAj>UNZ#0ziXi*WHc|qa54O)zNPn`mS3v5E?YasIlvLJ2{+n&D9&$h2Wj8^h%Ktn2 z7c68W{*%PDK_-$-Iv^dz_SyxhXttIfC{S|V2l*JbvO&njahwV>*bS~TIJ*4v3z8;}KNyiK1 z^Vx3RAXmU$)h8r{Z2RAkEn>@w*!`6$W^+-HE@6w0g_L0%iiZLvZ;6mEWou1_Tp4#4 zsgRVjm8V0tf^9n!GL>wZIgqYmo6Un%HCs#}6e#I0hI|d%vw>VKcTMGx)UlmbLbjf* zqy{n#Y#ViuZe-JqkZNL^YK8(O5v`DKX6tE(oaOGm6Z-v&V_yILg={O^Q7>fL*a`Yq7ybZYl?%sDH8DwkokQKJeL&yxVRh&S2m~H12QX_0x7f_&N?h5jwY_YeH z8{=-^4w7-Umj}pBur)tJW|Hmf71C2|<^xjGY@1(Dpd?N0{mRdIsic9pHG2r_GI zp#;+FY&B(&+F(1VfC43X)sWw0TdIZJ7I%pakZiM!G(mQU?ZZN5m#w`G(tB)I9gx~* ztL%mXCA&S4_iWkykUQXReh`vFwzy%)99KJmgQ= zwjAWnxJzG#Q5%rCWb504^cCCV4y3Nx8uy_<$;koaZ`g{DA$QB&+CNAF zTk1Jv@7N|TA#=~BZXo@@)*T@A$aZ%R1xjk4ApgX6_yW0S?(*Lud0|`rgzPI@(hp?b z*hVAvf2H5qKBFM@!PXHA1xl{tA^*u%l?1sj?)H))`DV+}ko{p>NQaE7`0ueUJ`2(j z|4AeVQju(Lc~GE!zZ?zuD7K4Y$VGEkRtiZB+jco*W7#sQAQQ(nTLbBMw&;3DC9w53 zLV=Q}X2>V9HMK%6iMxO8ko;jQ>4I!B+xlO~q_F8eNT;$*4nRt?MGQfKlD{L6Ph-0u zgIqdyb(4@}upLc9Hj}Mj4l-G6D+`d$X8W@QsT{Vk6(~^hwFdcIw$4q+<#Bhj4M{#* z^&Vsk*!DeS3fXdxAYH`fP9RmxmT(3IN`@{VU&8i&1v$fA>n$XuY?pVCEn_QxgiJZx z&NHMd*fQTBRmnE@0R>88z9C=5HlX~kTs3#kk&x7|HOD}awX*3 z*~Y6O*TLO)EhL?6T@8@!V!LgEOgCFi3#9+D`8G)Pu;q0^fs(~;$oH}(_Cl_YyWxIF z`q@4PAv?gBDXG7GtX+-|y`0ogOQ znJ&njvqkqn`hu;m4^o$GPXkb(q;UxHS8V@AAa~7O@i-(mZ0nPdy=6Kc+yw*6bkezE1;L*|>!Jwp12E&c^ks`9_Hf9MSgl)QaFK7y_F z8*-8VNiNjkuOy1CJPNYWY}+xAiDAo(hjc94Y$BxM*kY2QKuLcp8=i$+F?a9tkd&~sEkf3?T`ohWl&xY7(q(Kr8;~kz%i4wlC3Cxw zuV9PykgMcw;1H53wwGhbR>SdyY~~75b!?kAP@p934)XPE(+`kq;4bnR zl18@PSI9Q8J-kDvnXTaq(w6P`2U0C;MUh9p3Y4ryL%x+QB@S|J+>Iwd($4mi1lbO@ zt`x|0vIPz4F1DHsNOiLvWI=(Fyj;ltWn0RJTn~4NMUeEejg&yPkL{xrGW~4r6_6fa zyQ+fJAX{ZE6e!uPhrF<5H$iTQyLk)AFk4(3WJlNrJ0LU4_SyyMF*e%+sd2XRJ}6L9 zItcj*wyhz^O>&n$3dt1P%s6DH*`lT(GsD(51L;||$2myNu{Am>}I54rG?t)IOw_*}4xPwZe9H1O-ZJ|3QA0?eGk8Yux2uLbA@bd=1$R zwxj@=O}5c{NN=%yK0<1nt>Xm>lw7|-euu5<6LP!U?R`VC$CeXu{42Z9wh#pw&lVpG z=>s;2htwh4TOt%FX-S6s5!*#73hD|p@`j&078B&2Qq7@31{B4K)9ou~; z0r^+9&S}WKad$Hd$va#1 z0%Skf_7@@Z$(FkU=`S|72B~khgiR<=GPDi(AGY^h$f+tk6OdL9NyL9rmxqvzWGg>` zOcdMBDWs#>GA|$%!!~yX1xjLWAs@>&a0j_K?w%hYiDzqmhHL`c=__Os*~|x|lh`)C zAoYhWO`ZHIP%;$>`DC`p7|5k?*AoXxD%(Q>WHnp;AIPM!9j8D#ovknpQWNs+sNN01A{89Yfx-t^I>s3wJ5!khHQ*Ttc>u?dKXY?QGov(j9E!9#WlbwNFr> zL9Eu=$sh%GJ)GQ(_=1L+aA*E~p#vb7XKfs*rL$d9p=l|pWuyRC9aCfG8n zAUnx6Qv;bPw&;3DPqXzkLTZNXsTm5CG`2#1mhE3VL`^0jX2Am~SXh(yvZ`<5x`DeE8TFAX{*VzEcE8A@oWZ&4TTOjk!=G!3s!Is+z zsZX}WZYWTa&H~tPuCfm;^ zWV6`1ejt<079!4mrE}P7q9K*bb`T2%O7apQpU1Y81i5_f5>p^4U>nhpEoA#hhfEP$ zdlsaN*{*URRl-)84+Tng3n6dVvP&RW%H4b^BxP)I6_71w8?1s%1>0*4q$}BMJ*2AG z&KsdXNvVZ=HQQDzw|Oy+v5PF8rd3$pg_sV2;`gC zipL?>%-z}~B$h382C^+|6LXMhWm68)ZEW33kZNbUTY&;4wd;`YU_0D|Tqk$=JCJm- zE$>0Ln=R=8GJn}dk09N{_IU!SUbc=iC{S{J0r@_*s%yygbGLU3$pBlNDZ^SeL#VdmT$<9uwAJ0U%654%A%m(hrIt@SKN+)>^NISJY*)=W)mSj z$rhaqsVTPpR47pLlm_`}wx&$T&2aZG8A$P-F&nqOiY!B~{4Q%yakhxdnRFMB* zODuxiCwIdokbJRyltT8Kt*rtwKWtZ3kXAMSo&6QHkc#+EYPTK=lw>tQK9X(TLN1EC z*fvO_*#4NrH?``k-O<}NRrs1rXc%= zt#<}8$!w2vkWOK1aF9x6J6VDPB}J={*KBL+kW1q(WebvYwuv3cX0ZM2K_-)}`vB5e zY~ct}*=)7{pg_sN8RT==@-HEm%iYp7BzbH}0kZjQBlnOgVEcT8bRk>&3#5wJuHT?Q zN#!Txi`n+RAy>j(cEshc#IP+yLAI1FE*3InY!VOYa<0Okndr;nT1?0cU23J^s((PLbjhRX9Y3?Y;FzGgKY7ekP^0` zZ75LkwhQ?owpI_hVeT#tAsJySKY{Eh+x97B#@I41AU)1Ddj+WpwwPNeP||+~`AN3t z2gprv*Ypg@G~4MbWM|k)J|Hv8w($k&IX11Xex>HwrXrz0Nkk0f7ub5@Am_NdPk>~R zt^N;Wm)MR{AhXO?mt+FL&LxGa9T*$Anedj}Nox9E=NH*AROCY<+R$T^} zEjC{P>20>$YDn#{E!IMTl7t4x@3IXyL2i$`cMHirTU#4sJ=hYw$lTt zBetx5C{Qvt2>D~S*kQ<>a5pdt$v?K2amb#sHBUk2jO}a&(&ubu9#R)_a-R9UnmIj;-hz3Y4t=gZw>P$~oj7xEsHO z&!BY~dc#FKjhWka}f1c!2^XdGC;aV_W)!+&g!PKahN|jYM4k%6_ta zL_y|@tvwdf-)vX$kosY(Oo9R>yUCDOwf{Y5&(@HO_)jvQ4oM_iToz=b*amYT6V3LT z2k96#TL`IGw)0{rP*Pe7`8c+%a>&JVmtF-)0^3XtWE0t<>LHWF*4GH>KWvZ9kVIxufV>>E>Y&%0`_7gmgcf>xR?-TS6}sC>iR9{2<%=AmoI* z)?rA7*e*vQJIq!-0htlDohe9_cjnE$t8rluR8%evU2j6ms+2^_)Yp!1iznS;tm?1DQp(;{fR; zw!#NUEwimYL4lIwSIDohjlV-~mAmgxNY>c8ejvNfb{lc~E3?5?6AkH2HXjSAEw;P_ zC{VJP1o>^Y#1zQwa5t}b zAbZ8u+XtCzw#NZT->@|dLF$(6WCRM76pceZu&qr(?vA^Z8A$HgCgvdf!1l8MnMbzn zB}hN9g%wCWv(>Icfs%tw$iJ}V??CRAyQMuy-q?~3Ap6cXas-(Vw$Bqtf3mfoLF$X` z`T`1+R9-{=n{DqFazET<-$SD6{(HP$c!X@kf0DQt$V9Tq8>FMyUOyle&DQb_1xn6U z_?3@gD~p0$EO%Qmki@ZN#6vcoZ6*;i32f2HkWOUlONCSt+fy18C~3@u{2#V|*^o=- zt~d{p6t?vO$fmNT7DGm}O&Un2v8i%MrL+C5gaRdZ)sW9%tE+=tCU=JokYuqHG($F< zZMg+9Ic$I0A)U)M+6k#Vwy$m|P}0#0`FysUe#jMYS0#`Xvh5E;wumif3^K)RZUWLJ zZ1K~OGHgS$P@v>(9`dDZt&5N=9#SP@p8@8S>3+J+F|n z+}*!J(!y5%1=&`%qaVn$u@y$%{Ytm9twckrgDp7@3Y3f`K)#dhI|*`K+;yfv(#>|O zA^VrDIs-C2Y(5Lpy==L;km_Sw%!dLc2}O|aXB#eo+yHm)rH~A=wN*e?*ecZ z1-WJJ#&;lDVf)#G>?&K=0c6(L!V#p`*=qhlYJ=_I3<{LwT|$17ZRr|vTihiENVeHV z?jgIw_VEarUAFcYNbj*-y+LZ9t@0BJlx&~ z*KCdDP@sN4j0pJ~w&EJd-Ey~92T5Q{ZG`L{+e9;D?%7l;q#xM2+adMHcGn37N^1W? z{)z3d7jnlEiA_fYS{EdTr8ryvWnws?9(Ok-kmR#fmqE6GZNCCCg>1RikS=0#wU8=iOK5-sB|}Y+ zFJXJPkTcx1wn0+LcG&^hGPd$=$dt3~^gz0TEwdj|m27i^P@p7c81hwY1EY|u=I(hM zk{Y(=Dah8coz6g}j?K(Nx}I&rL8^f*Z5axbOszt`ku7oqa!uUzY(dh@_OJt4%T~V+ znHIL=14y^B6&^#XjcxTG6evkPhkQHR_$A~zxck0_q?4^HK(>qR_8v0bY&B1i{>$cH zAl1W`_YMV07C#~1%a-^9xjycOBOZSx{cImmkR4!ai-pV}+f_WIg{>k9QbTOJ$xxsq zOGAE`Z9W}xBizMiK{CoVm;>1{wwFA}jI-H7NKdex6+>#0t+W&hlx&tmeu^!<3Ubrj zP1is&!xmK!*;%&UM##*uJvKvno~@x3QVVP+?NFeks0;FrZS61Q7P(95gJg+qVgRzs zY(D~-6}IjXNUySmF-Wbk)lNczl7nf;ue0UPL2iS)r3FYf*^-tZyTvxL0-0^L&oxNz zu(fYOzi-_B_qcn#4FyUn_aMK=w&x+Y&t3KrB%W>I1hNNgac7V@WRnX>AF;h&LF$;T z7{q;(H;b8veV#4D~~SlD9#~N3gXHLoV_^$;BunQEcTCkd0>Bo`OsaTjngJ zW7%fsAr;3Kvj_!B`j;Ue&-T0uxdiT-HXuo4JKch85?jeGWd5*i>_a-4O&>xkg>C8> z3Y0{gLOzwP=NxjHyZcK>(%9;6Ae+v16d;qqR`>wvOtzILNM*4lze0hMv3JO4vweR; zE{D6$A4qc9ZX=$5W%Jmoqal;e=3^mUz?Pc;sY15JBq&glkOKK4wqXsqV(#A4At_;N z%Yv+7yUc-1DO*K8q|4ZL3L#a_mQ?};O6E!-U%?hz0l7-<2C5*bVtc89Y&BbRJ!ERw z&Keex10p+HGm2juJ7rn?~5z+GeyB#msneUNQpdl-OBGh4$Dq%GU=2&7ur zipHTp$?7EJTiH@(AlJs-_#7ndY(EQ-?O^L#f=nk{Sb=mGTg^J8y4emkp+HIA4&?u` zE$uj$LA*z7l?#@Wu*%dY|@rBRTdVB3m;+$4AD@sLch%_Ks0nk_0B zGBa#_sgRy!drX7W99v^16eu~#hWtERaUSFrxLYfL#IdCoLw1pE!a!z;O_f7>nXS7L zQY&nC)li_Mwhr>EY=;eyTjMUj8IpCjJ#Q9$U^BWcS$?CLrV4;-?{fz$UYhI%IpBhXN%ni;zEJyI6+YF?VHa zkesk>Z$S1RTgEnIPT6L6A$`Uc?ICr})_({EN}i4(f5Fyt3b{+}{+&Z|#a40!*=x4- z8_3+S={rc@vQ0ifDzHU7LxGaNuaLiEyMKq=J$H3qkUX#*{Xq7StswICSLTUrB^uJt zY=7b)^};rm00m0Ek|6)e)|mpiH|}mUB=2n18Ib*8+s}f`CtGeVq`%l)KBT_c5{jTe z$xsR8f7sqjA*UMgOh8&IAc^=->aq&5k!h z*=qhlI*-ktK`Ngu?-B}>EM7ytfGsgVu8_Oodq|4dJ{}=k%+~e-nG&|EH%J?{icd(D zvh99Dfs(9EWEA>F|iIw94`R{IwUlpORzzKbn?0CL^jEeRxl*^)*e+ru_8 z2AN*A&k0EPv9(V_s-Nw877CP9EOZEq2BgWP4WKq71lYmgmci`#_EFq>>cdW7wD z7gD2aEglM#oF77djIHbha^u`>okB9fmT>{uNw%3Q$V{~u^nZ|>fsAo*cy zi-oLe`tR(&jE79be^M1mkd9>ANrqGuTb70bC3ESJk7kR_f?N!D138eyvc2R%Hjb^i z5Hj&>XT^|CU^AtVN@UwChXN&ORgh0&o34S}AMPUSAxUQIZG>zJ+e0&CQrQ|>A+6bt z+aZ<4R@4OrN>=|uKAkP44{{mYjSoPQ$@U|V&0^~sflM}A7=v^UTg@b-a@h{1p+HIA z9OU!ZmKGqF&t2jYBn50EE08T@`&ffa5nKBvq>I_Ewjou*R=EcSN_IWu4O{jRw1G05& zQQwfMXX{g+ztRnCkCBjSWNVCp0wpJLkZ)ouPJ~=DcWZwjv23ZSkZoa`NP|o(o63ZA z8(Viaq}ti;a-l#;Z2{yv*ba*z*U4SJfuxIVxeT)1Y)O@n`O7w14e1`X&ss?JvUM~- zfs*Sc$oH{TwLq?)yS+9@2H0{sAv?&n&W>5S#QvYMAY95DJvE3`2f|?P3&i zquiBEKr+U*Jq6iuwv1WGOt8()Lwb@edJ$4nZ2iknpyX*4^3!Zh8<3mf?%x(9vuq{1 zkey>&--paRn?8i}0^8&{kQV(palTe@}c?R;2Y~yo~d*beU0g`97t|iF6u-&de z=9R5x9nx=XeiKsfY(rtb52uvduq2E{eO@7f7Pn2HzkX!}jt4nOHXa4e2j(_826FZD9hkrEGE2kSSx6 zSxA?&z0O0bf~{o{3Y45LL%x!&Yz=Z%+-+?@Qq7jJ4cQvDnO(@#vPFAH*Rk~-LaLta z=@<%>G@e4ff$iTpYPU z_XW9j?hbz->0m2}{P~sbWLu7gOc&dqI7oN1jV3_qFWXlV6e#IPfqW0!jfPw=cU2jX z^s(({LAIYQCl@jUY%U+tgKY6dkP^0`5-3peRtotcw$=*B4Rd!<1<43oc`al|*|zH; zGsc$L1nF_MSqrHNwwN|3P}1K4`AN3tF33%B*VF^aG}~z(WM|k)1|c)cwlM_hIW|2C zsd=`kaVSs{F$MVrww@WtIqvS~AX#LqcaU9TJ6eLwGF#y)q*vHh)*-dZmb?W8O2&2| zzsB~x2f20bIu9V(V7onn>?T|FKgevc`7=mwv*lhwYKLv{8VZyo1jz5Q4c|jxq?jj2zxn}DvhU^X7gMrK~TSGad z1KV*Wr0&>?YM?;LY8~Y7*-{!I_rTqFGbE2}KP`}bV(V&$%rjf)g!Btr&0k2pvK{n7 zfs(ud$iJ~I3FO|nOB{jZgKcCCvY%`p6Oj30YoCVnH`~=Lq<+{c7ob4N?jq!s{qM5? z*(;EX_)jvw21z7a+$LnB*ao*D6V3Lz3+Wg(>me1(c76y2N=i>4AIG+J3b}ah(k~!M zV4JyuY$99KEo73|`tBh8hwbqJQps$M&rqP`@6)7T~= zA(PIgVj!Kt)*T0_Ot!lOC{R-S2l82LhbfTD<}N=Ck{q_>49MoPC1pbnMT(CD$d8FJ!AKgIp1Jdlirrv*lDnwuEh=7BYq{z5&vuY|;d&GPXAh1xi}l zAYaaQ(E+&%?#j9$sbt&kfov69Mn7b#*=7eJUBebV45?bS{!u7U@-zK(?K& z;21I;Y%BjD-O2Xn98z6uW0z2%+09(Qj6et;ri2RiwWP6W-oN(6~3&{}MWjthu*~*h3Gs3o$4Czs}Obw|qwz+gD zP!f{``Ej;^9LP;@_nZgGBwKSKWT)6piy<@3W=bJF!?sZlsadwPDkxAgRRj4sw#a(O z&2!h&2+0E5Lo;L@TYW2J7TJ#5A-%*_*afL&w$;B-pd`5u@+)lP1CU$g?pq*PW9u4$ z>^j@+7-TlsY9=AQ$>ygawZ)b<2L(zN7a+gQmbe7D9qxu#AlYU6ScB{yTiYgN_Svqs zA??{J_8@h@w(FrlN!AhM5838VAa}%F>=`7-Y=ak&Jz;yfg3LcQdkg7PwzE4(ow1cZ zLV=RaXULzkrN2S$g1hMtNG{o;z9DHOa z_6Krz+@+*Ka?duA2H6L;pA5)6vUO)e`iU*%Lh6~Vwg3v0927zRg)QGe?v=ZxGU)d$ znE%e=q)N!XvyD_k=7a6C7Sf+=?G2FnV!LjF0wt9#kpE`eYlGYmciEkgsFwe3Z=oBq z5&uczdLa|ZCjF3(VtXBgR5V-5Fcc^`ABB7jTiFEUV!7Lzf+UVDV-~XUY%}waNnnd! zgmfZX-!i0<*q&CQKuO~UX>95gQt51e z&!Irc-6iBR*y?T|m&x5>fFz5p-~qDPY|BrO$zl8R3h7+7(RWDYv3-3)fs&3N$mg@& zL`46}6>wJ-4M`!}ek^2**m4peQ_SX)AYH;1p8_evHl(3I$y++)OW9hpAXmoSMGhq8 zY~}fotzg?OgiIw{W(lOL*k(&1Rm~Pt0R>9>s~}&)_FMzGTJDz+TZx5C z1KUO`q#N0E2c(+Vrn;a&Nkk9io7sB$AZNL|AAqEVt$qlyt!zglkZEHp9EWr}+sY)Q zI@pqDpg_sk9OOIMz84_Z#a-tTB;9PcE0F!mR=p0H9yY%T>0Y+n9Z2=DE$%^ql7s`u z_p=QjL2iJ%_Y+74+1kz^D{Pk+kQrjDxQ6sF+s-YdM%c3Mp+L#pBjiWfVqYLP#@)ah zB;#x^ACR43YyO7JB-@#a`IVkxGf|M5X4{N`0wrnjke^|jPK4Yncah1E%(3;RLUx|* zAq_GMYz>)^c5KJlkXmFb%7X$Ws|Ao>VoNE8+%k9L29g!FpEAg9fCVMxI=JaDKg0w%MDskOQwv&+`j3+X+!tA0rRW2+P>P_jD=dC!(T z2DyFi<|iOIV2hoG>>=CWEM$(@Ugsfw%w`uMb;5SO3k05o;)_4j9N{-JVf5TRM1-V=9)@~pPY^isUyyfB!%GRtg15 zS}GtP#dc8zxoGanY9aZE}gqN2T2Co;Syvs z*$P%6lf|~O4(V*Rq%BD0u#N3Nfs(I1$mg_aY(yPHEu^4Y3SAX~up?+h}9Y`K?^ zE@E@nkSb=24^W_F=pOPVZ10bdGu-`sfuxk}@(r?OY~`PjDQDaHhI9p6W<>0-R3+P7 z6ci}=69f4wwt+avRde^807(s7b24OW*-ld-Q^#h~Azjb5kqM~=wzM25P%@PV`9`+L zLdZ37*Ha8hGuwlKtYxb&hfE9GQ6;4RvK7`qs+DcE4hobcH$uLRZM+$B?c9C0K+?h1 z*#_B8w%ZQKbg|WRL%N&I_d=?NEpGq{lq?G5d)X33AlJv;@E9chY#$Sl9bjvnhRh(_ z)hwiitzrRELu|W?P@p7h1@gme^J|bB;Vxzql2Nw7ZOD$Xz3f6}oXvVjPq3XGKx&e$ z^cV`1Y@R}XiY@&Da?{*RUqLd%7Ih2RS+?Fg$jq@lK0tb&t>GC`3v9=)P@tsf1M-e- z?F({?+@+|vU&#{NL?mRF*?yuSv%=OD3+Yw15D%#}w%Q~pP_mx_`E|DZG{|jmx0C_> z-U5f$f7y`TVjIbY%r@I+KBRZp+KM2x%XVD?1xhN*Aiu}9R{^Z-@LbTUi(6PPp6Zf#j4eqaU(oY%_z9IcJL=hV%tn z-zcOm*`CIsKuO~ixA$ejeID+gm+wuuyUf7b(A^plWdI_mFwy$d_P|_YC|IT)E54jKS zs-7VEWc&95*)O)7cgTFRxlc&{u*LmAO11v~do~mi|EoaBTNLCY*#5>qF7kiLMI0ni zY~_iNjb__UhRh$fObzK6w%K$@#j^d$f&wM|IgpQId(MMgJafnVJDkSpfyeGQTlw$@F^8n(-A$ds~G>_NJWZO22ZoGt4R3Y5$pL%xD7<_vO`+znhn zQpNUi1=(u0=3B_ru$|pOx|Yp6LaL5!^BD@1q`g7Do^ARAat+)?enZm8)~gbJWt-R@ zA|cbv*6;_?mhC7OQY~yn2~eP9H3{;6*-}y=*UH^^8YFFOKN*m1XY0&{Ob1)Yg>)xd zO#!63*!GK{KuMl~d^g)t8RUAnOQ?jTmu;jPvVCkHwUFs&Yioe?0NYg)qz2h4TcAM6 zZY$)4ExQA9L)^`GK{Cu1+Y8wdw!wbLjIzBBLVAqN4nu03?R*ppl$1_Deu8al3UZU& zrO!e##Wph!*=e?@Maays^({ksmhEvBQgdvL8&II+cnk9LY{k2fTi|Z(A0&<~^#HPq zY!gS2Sz=SCkX~l%I)~H>+ubD;D5<@H{3_c)fZQ5)`45n+vn@YCc7rYP6*8M_qwkR3 zV*C7r)HYlD4-_c5j!68K-(jnYhTJZ9dohsgvE{@=_8;3qB4j*UTneQ3*+fI?fbA_E z3Y4^DLH>~KA_sCu+?C}+a?G|}2-y?1j1tJ4vdxx4`iw2Q0#fH}{Z&w)-S zkh|pWq!E%Uwh{~3Yqs^jkhx*g?U24@o9u*CV2kL10wvvjkiTQQAAsCFcXdOMJg^;( zK=zTXU>q_}Y%7zHer8LWfz%7z*c=on`C5SdD_h4BBnj_hZPZHoOy%zh{s{{4aHR0oh2l@@vRMvF+SKI+`u> z9#VhU<{qIy$)6X<$FL2&K`xfN=MPBY*qXl~8_#yCl73|p*i00p6WKQYKq`qXEe;Bl zOeH`*nJqFIaw**Pq(YL)_K*fy%~qcYnKZVeY)GfG73M)IgKf0{3X~)lLq3yj+(0gi zyYDhcve`N-A)CW?TMe09wwgLf=dt+)NaeHTHA8`t#TLjHuqCuXu8_Oo4oHgFKDr=V z%+}fqnG&|Een=a(3V~E9+wL$FD9IXwd>PyP1mw!OibRTUg`}P>%0sq+t@i*jjckuckZxjYIE7R*+wnOR zC@H#vyk%Rvfm{oBDR+?kWt(__Y%AN(6J*-hx?UmO&KBMw)xlQ#1qDj>e<0tly$R4q6e?#V&Ei)qJSNeo)HVRUwY=2^)KuLcbfi6CiiaT~jh77i_1gkiBFp zNr%i8+eRj&ui11Cq;A-z@}NLTL?PsF*?Nj07r47OkleA=mqYfR?XVIu4{U`skbY!a zse{xLTXG{5C>d*p{4?8k3*=t7>u7`ImF>0zvTtnF-H>@_^SzM%V9OnV)F<1bK!K9@ z5y*eB4Ua+Yo4fZ3NPgH_ry;A_|3CXLXCV{uzf{Enq$AmO79kbImbC%}O6JxeAI%oC z3AsPq4QxXa!}hWZ*;uw_51BZ&vja%SvzcQ^C9rLtLV=RB3&7X&2*co$ksRo1xk)*Am7ARJP)~M?$#V6 zmMwJ|vMp>AtC0E2rZynm%GR|7sW!H|9Vk#z`w#N%YzO<0>)0k)iX$PThCd_qRp;(j1K#3m8? zS8AB;EeZ;hw8TJugzX{@a--apB|WC}v*{{GFR)G4K+3U2)I)*#z4-#<7uoKcA-BX` z-CszS*$&$vyTVq`37J*4m2OC{u_g6EYMpIt01A|R3FJ4}Iz}M3$=%HuBwK9NlaSqJ z`!@}l9k$##Nbj<_1xW3&#VGsvH?4O~F(l)L9ENY2=rZy|fmc6tYy3pVoz=}WeaXGmSKrM*Fc zlBo~KU$aGiL+*yV9+mbhxn+BZglu4|{{xvjwxd``-?J4aKpdLfBs8|;VdAGVi4 z$i%SOVMxcaosB{&j;(Y83Y2V4K|Y=>eHL;F+)d9zlE@ad2-zgI-et%nvpudtI)$xa z15&AM$6HXKq-YoNnr-bLP|{cr`3AO=M#wdCS8O3^Vq5WekS%Kr3Y5%EK>mm= zW*TzG+zre^a>Dj957|?;=0(Vyv7IeL`kc+ILF$5Sa{~&Lq-{g~l5Khya#!3%dPuI> zdJiCb!}f3lnOnAoQ%DE4qjN~zu@zlGfs)l5$ltT2+(GVvyYUA|9@&1LAp6AD`3jk5 zw(t(=7q*%&NWHS{|3HC~yvWR7`8T$uXvn>DmkF1-noc(xe}*#x$zR>&l>^|eDfiS4lyQps$MJy4+JxDWCvY{i3+ zOXY5D2olYfIttk|wuy1bq_e3hNN2Eh%|I%X?QRYVl+-%NXR#eDK`xuS{8dPD*p}BJ zo6DBC1(`gy(H%(VvwiMCs(`J19}1LQA40y6t?C4FMcnP3K~l_?a|ziJwuNiR7`C_o z=~6bihg2Ec+anYxX?cNsIorh>%04nVq}%?YFi*y2Z^K*`V;{hfwni0yI~vcqiU3y>LM z+gXJ4C|l+Vq{i6h)}TPipH0Y*vkh!RZi2h#T}UR`nmuHv*iH{1GtFj>Aw9#kaSEwf zwzLZhwwZLu#Ir?bK{|o0F9%YIY)^Smpro-7@=0tb#gI$puDBGE6t?wp$fmNTRzXIy zP1ZmHC0phpq^sCwk0Djf_U8-=l=NRfzJ~4j3UamFHQho| z$98%L*?P8;N60j=Z9GG|kxjors)=pt0}7Nxd_%sOtw-hl%31F2BOz&FtN#PpzifxG zkZENrOn`J7+e#9o+S!s*p+L!48st0JzB3@#$z4Y_BwcK`xsdHYA;w9txBkH$wh~ zt=K~Dmb{!$o=x>Y`hl&h4^oe8cLPwMq;?4MPizMxkbCAXe;kq* zw&h94zOp6GK<15YbPm$*Y@Z8|`e18cf&wMiE0F(Wt6GQL7k7J`kbJY{>_GO1ZD9{G zs{8-HmAHLKNBl35Lr6uky&Xe=l9n^bN3mU8KrWiQvTI2Gux;N$Hij+Z9x}0PvyYID zV~c))R6JY%8x$yc`ha`_Thlk>61h83`M;7Rwvs5wCbO;oflLaUj)Qb6+hhWyG+RV6 z6e#IVg?t*@eH!G_xvR^BB!lfR8?u>f1$mIkVp}PIbT(U3F{E z*H=X9+G~xk3`50u(hT@W{~YlLt5A>G9Wd?wwnb7O0sexKg>3t54jQUVu~Oc zWg9Gk>=@fiDP+dkYz3qz*v_gTHOW?53kB-;;dhXqVoPs=+%$L77LpmZs8-0%vh}t@ zW{&N#6Vmf+4Ly)rU_0)E0wqO*kauirLy%kKE@c#wCANui$S$+}OhRUbt!oC-t88Ho zQfq9r4hod)FF}5tEq@hq8{93eL$b-1xCPlQwvipkY_omtL3)R+Z68v*Y}bcSprrBy z@_TH1XOR2HUG^m;o^9b8vioeY0Wt?{au4Z4w%12U9kI2%K!K9;H^?8em3>0)guAV8 zNKV-@A_{+H&)8<7Aal+Z9RukLw!Sz>U9vqTK!K9RWXNB!ouoqUn!Dn3NN(8HGa-A+ zmYM^Zz&4o&={q)62&sFv?qVoVa%Uj_z*bidxkv5}Dj|7dE2x3&Guv_KBIox%`LXyjN8xPq$w(2Cv zra`KZZ7~B1l*DI4zKCr&7jnhiz2`$x!q!>@S;KZ&0+~{_iZV!-vF%hq zs+=vW8VZ!m)k404Ev5l-mD~+9K~lx`Vj)}2*4zr28n&}`NY}ENE=bj}ZT3KclC*xv z*RxFzLau?k$YJRBE8)EU8-;8W+rv0yn%NqrAZ^)>W+2tVRx}R&}@|FWelL#~y( z@l{CL*nZX_+s@Xx1(^=EumkB%wwiyC>SEj9hXN&eN09GkTRMSU4|fUYko2;RTtc>w z?c*9U{cLRk(gSQ)_mCQ7t9*h2CA%+>7q;wo$PIBf{|U)3TkH>HN7x1KQH6&APGwG0>W{b*#%nVy!4y0$<9`hhI$JST~ z1xk*KAwSPnTnf1b?$*j7acrqokX>Y(sDaE9o2rNOGFw+8q*mDOnxQ~R?O({RvK_QR zZjHPAPDs|-mb)Rl!IszunN7CQ0Z4DLeF~(u+1f{-K*{wO8Ij<_q^gXEZP+e7w*E#nX}r);yw zkUnFJK7-UbTmJ}MBwKk4WTM!1S|J_Hme~QRKWuYdP@v>bFXUs`2Kpfv%iZ%JBynub!;p<< zI~|2g0-KqDbRyfv6r_^a(q^GR$<#dLli4B{A(z5k&oU&bY!9oD)ok?}kV#`Z+JbaC zTj4IGGT2uCL4lIw1ITBxjUPcSi@Wa=NV3^F&mo(`c6$k#T(+7UNawNn0I7Voyay;y zviJn~0=9%#$Q5!o{0>PG+s7wli`iO#AXCD26=8m*4O>Mtq)OR#W1v7uRy^d(*ya-< zSI%8b3M3V5gBr4xY%l4MsbaHPkgjGs%Yjr4TWLNNDA_E8d@Wmg3FPXyn=XZ7RxC z8Qb$bkuOAVQb|D`H2 zARWoJlLe_LwyazzP%@Vf`DnJ7BFO#WZlDB`7`B&E$i}iYS3o9??W_vY@oc6RQVDFE z^-!QBtqJmpY||EUN!&%YLch1<=Jj7YWK-B4Iw6zF*3biK&34oWsWi5tK`2nNIt2N2 zwv4oN24&m?5C*g9t*lg$?9Af3Zj;~n(iY$4mn9%PEx+V&w`%yxAMsS>uz6DUx!dj@&KmVF7iQtsxjAt_^v4UjEo8@z{1 z1>5T*q$}C%3#6*p&flOwN$Dr#tJ$``Ay>m)dPMoJq?T8ckon7|3L)Lf)>RCtHnuwh1xjkm zA>Yn+PzkvX?(%CO>1123gKQUDVk2a_*+!co-NW|T0;yiM_BJR`zh8ESd>>m?H{|-c z+v|m7fGuYLvV&|30vTb88-er?n~XtfnC)!>3Y4@=LwhW96&P5R&os4Ikxpv$jr0p3rH`p zO@Bx`swv{hPudyYmieIU9wy{Vk zQ1TTG`3<&?SjcU1cM}iE7F%@^WVhM=r9ftfEjJC)yKF84QhRLi*-)TlC>Qeo*xvIY z=ehe^1j#t*viWwbI7(+0qG;Q%xXv-v(42)fs#KBkUwD?XoB1+ch44*Gq&bd z$ey#EwnOHE&2&Nfl5L|0Qdewg{ZOD}Y7p|*Y>~r|yWy^96p~xEhjGXTw)!c^+_4?a zK>D7oa2`?*Y^x3mlq4@h{*i5b6>?A9eXm3E%+|RD*%!9k9mu@0)%=6>8=K#U)H_?= z5fms{Jc0ZNTf#ZyKDisdgyf6u;~KKxY^?z@KWta`kXHTwpZyh2kc#+UYWD>Slw`d_ zK9X(z6LL}9#r!}L%{CZO`78T}?Ij8_F>E#l(y?r3agd5*D@}v~C7a2Rk7rBQkW1ih zIvtWkwx}$~Cb9MAKqi^(F%QxyYz>8wN@Y7Ph5{u;rI6QbYvqtj<1VENl61C-8pvj_ z{nSAwldY=}(phYw8B*D7wSS>N$$lH;bJ+4bA(zYDQa2=dY>9o4&1V}KfJ_0~r$D-p zt!)HSMQqn&P@tr867t1td()6B;Vyd)62rEz0NGNu*d@r6vB?Ug%h_JnAXUNEvIzxB z&bJ|7$yT-pxhn3qJS5d@8HbRqVVgOIOf6gV8Kmpj`Ys?<&-Qc$1xgxkA>Y7uatFCa z?us8FX<}P{hHNui>KkM%+vEqNTiDb$r2evXtEyiGO70>d-^y0^2XbxP9mGP?&Q_2B z*$%elB*=8KC8a{Ti)}OwQr&D{8Bn04Jsa{pY&W@(>*cPh0Fpkoe?^e(XUj2=8DMi| zkRD`x&W49PUx=_q7p*h(fKGt0Iy1?f39JqxLMwyAk2P!h2S`31I~ zWym@1?pGmMWUJqR>=N7I7G##$3U?vB!nX1cQmbsq2T-77>L1-P@tsr8uGDhTepyl<1YOkl6bb6N603yMZG{Kk*)6y z(n)NOACO9BYy5@+CC94fS3ZTUI0|y9+^zkAM6;#FK{kzTA^|e#Y$_Sj8EjptkjiAc zOM?O>wV9C5Vmru&TsC+4d649=Ef+vGmo2dvGI?yH2GaR#pJk9LU~8|00wvegkS}Db zs)Jk+cY6(x6tm?tL$-u%p#?IAEv^mHrEJmxsWP^=E+|ma(hK==wu^qqRd81(kW{j5 z4@0(!En^Ha)oim9kgj2io`zH{TmLK+D0!NPd>vcUBIN41J6VRLfvsc>vW;x(8<1&Y z)7y}4W}DoFlx2(XP@tsy0P-zt_eYTX%U#_mB&}?R=a6k?jY64 zHueAoO1_>T-^JGP3b}6XZr&m3VXOXvY%kltAIS8v10|4~;O@B; zl1aAa3dl~eomN3+n$6TgdWLPI9#XSxX-!a|WXeK*jxDkka`W8vv_rDM_RtAg$5!71 znMJmvK1eUI6%ImbnQe6l3X~*|LVks9d>nGC+`>FZ7maWcig4qKyuGEkq6la zwx0sXJhF8aL;8s=7)U*{)s{nnlKo1^zp&-kK<<^hr8-F7*b*Bd`_4Af44DtM&lX63 zvbD8A>Wl5V0}7N>c0>M~ZLb${Kip*xK%xf!Kl>L1vJwAFVn-ko$tGivj$(VAfK)VF z%QO@yIiH36AGWdu$i;BCwFpTpTgD1xlb60!}NebKgDP&XGQZFE**(R?boyMkaA(hV7eFp_f?j9hY!B+PSxlHa3ULnb1 zEBJtHHrw(SWOCS&RQ<1XF574%r1IFlqM<-Zdo1Mh*>2(?SHN9W5+sFe|56}Z#Fmo= znPN7V0qGL9xNJxnwxL`oQ1X@!`BJvOMUX4w?xF;ea<=j^$X2jzS3stcEwdWZRcy1h zkg8_;(*OlZ`kNqM!}e?;SIb>fDapyB~+7g{^)HvVYkQXCTwcRyYsoHntT9sdl#HWhhWGwhH+Uw(oVwb#m9S z1xXj%?G9wS*{c6Rriab%L%NqO_XtvbY>OvQpd|hr^8IYXmyjFa?)@5)LAKTaSz)`p zhs+RL#S^55*>+waHNuwl4h2f)J|REK7V`tSG42K;8h$0?Y%fueonUK@fy^Y^SsbLN z*i0g%rr9==p+HHRhWrfMbUNf_xr@w#WR9&j2eR{Q4|$MTU~4FZv|~FehSVZkQ7IHC zSuKbB5?e|Yq!gjt51xiZyAb-lXhV%nlmumc#dSttc zgaReCe<1(Fb`T4>XYTS7AbDY1PJ-+!TVg6?-q=RdApOqvnE|N}w)Si&P;#9M`A@d0 z0?2)Fw^szoH(QQ@><`;Q8Dy0FfA+^!LOSAqiBv->lI^V)3Y4@oKt77?q6u=*+?BOJ z@`r7^6|ymG86A*`Wt;7SbR1iBFQnqx`um|k$RVq5tK>1?*7 z14!ktjU7ROlCKlU=dyL2LoScIn@dRY*{W|KTfp`&K&Fr__W{yHZ0-qC#cc7fP@rV! z9r7h?@1Kw}-2MH5q?GM4qUl$*jIBHxGUaSLF_5lc%Z!IqCEHvg6e#(V0{JSo0S&oo z?w->jsbOo*f^03@X%1xS*i1g8>)AF6A=SW^Rssb|rb;2-$QD@vxhC#oxAUDNIKX$`ykuNb~^x>F1DH> zNO!aO5lHp0<&8sulEq2L_p&9-K(3Fw;WuHOr`Xc( zAvew4^dlrQY*8$F|o1xqsYcH$&pt7Fr;?&lcMTnFBWIfb=2TYZs)B z*jjp_K*@PO1lDi|wKd{xELhg~fgL6op z*b1&7`^>g{1DO}Lq&rBzvW-4K>W%H|2?~_7ze4_+i4YK6WK~?A(O9d9kWFVh?1M}OTj3z2Guc*#AeF_IJPHL$#>OF^&GtPB zxg72~W+2IByPbn<9$U48Og@`mf^-2}?kc1T*%sHKKuP=-N zgLEyM`Giy*+vYbEC`pTG`IWC{n~s8919y=zkTkOO#zD4;?I8g&&1?W~U)F&UQWv1xiX6AV0yjwFtRM?$TEvnPQt+gX}b0)FxzR z*!s31Juk$kklkQQRDXYEHrYlaA-%=+84amzw)R*kP;wm) z`5m^ZB*^V@x0eFR9$QWtWdE@(WI)EV#brZ!pG|Thb-?zP4+TnEiXeZ;c2NSkBksz| zAUS5+u7K4Vk_x_>^0kZ z4`goGbU&nT*(L`e71$z%p+HIZDCF2*Y}NlD`@!~aA2Oe8xkr%xV*7pG@~_l4Tl_f` zC>gqh{14mvHRRMV-U-Ox0R4VM`v3ca%X`R1vXwtUCW>w61=7)MneUML!#4K`1xo(> zKt6_TAfokGE|$CJC`jVinqwdv&vqIInFKbI2EaBDN2KY%yEw2xLmwuErp3 z*eWI=Rm!$I4FyWF<{)3jHopM5a_(Z5AgN#*T!CyQ+shhcs@Uu%q^sG^wjou+R=NiT zN;W;@YuVBdAy>!U^f4s$Y*A;BZD8xYfJ`IX;}xWv*cxsj)y#H$2L(!s9wBep)}A5P z!d=Q6B!Af^J|Nr5_VWdqHnuL+_AA}a79t_l!B+bR3Y6@}LcWtNKLK)G+$|+R(#@8b z3fUgEku=EkvVCSix{s|b8&dsj*SS!jq_P0=18jRmkQ?MK+dv|03uTZUVvDVW%rKi& zLwbbmwH8vNY%L8?pya#>@?&geEsz`MZmSiN3AT(5$WF4&bU|i{ExH%d(`9@2|!Y7tUPY~9OHpyX~9^2=;> z8<1P!?qCa&Rkng%$gZ(1|AWjrThalfH`qpxAhpT%bpi!S+Rq`s#ddQExoz&MZXnrV z`xhX)%a-#1nLRf51nGZlaj%f_Y(wu*pycfn^80Lme;{|j-9<$EujG)eJQ}h`Y}+xA zIcCd@hx7^CY$Bvi+5V(Jfs%d=`7^fXbjY1^*OUdx1>0#3WG~rD@*#7@wowS_Yc^d1 zsT;PbQYcUoQ33f|ww@};1@7)^Ah~0!uZQeC+hHSQ9@q*kq#xN<{zB@BEx8>El#F#k z{+aE&8*(q)b@V~<%62;d**CW8A;`S5{XTE&SNel3cN|imY>Sgnpd@|<@?UJjbCCPy z?tKB0AGX#d$f}Y5&;H95$VB`vRk04~NVc6#NJX(_?LdK&xjo27v&HN~?hkhZhmgdu zy&OX}maX{=GI4BY7m$u;GuMzxVB5Tf0wrnpkWXZreuP{ScablUB(wFtK{kc$;R7%8&7~FlkF!7vRQ1MsgTKL3u%zfVXMi6 zR4&_oHWVny%Y%F#+fo7K^0`YWhNOUP#6Y%??V}7bMQm-AkS=Dss)kevTV)*-DA{d* zykX03hFmFk^DU5+vBkDQww!IS12PqCuU(L?WV5}Hs$x6uhXVEc&jrX=vuzDSu7RqSx7gqJ zu?v~MY|2Brm96UlQf+K^M^K=o_7w8%YzOC%>)mf7GrkfzWz&2?i<=7%xp+HG@JLDJH?mHp3#9dtvB+G1v zeUM#YD;R{#D%;8sq}SMzMj^G%HZ~3gO1>r`zrofq1G!D^Zss7_VykwL-Ddl@1eqPS z+*L^Lvbl9g?XksgL4lH?9mxM?QP^gUZ)I;0-hRx_bMNpcS4AKAw9Aos-G zcL5~NY@NlBePO#bka=aRDTnkMo3Dh_J6m226ewA&gZu|uLL=lpxf^bVN^Y?62sCRK)*MyS-4LBx?Zjk!pMj(l18yth|AGVhX$i%SO zX-LPioy|fjj;(Y73Y2UvLOz}?eFbs}+)b}RlE@ad3E3pJ-fhSvvpw!YI)$ylLn@W+ z_y7u&6dglev#p&%E{(gC3rN!0Caxfx!S-_lnM}5>J4k1-g$GDwv(-LBfs*}K$mg)- ze?TsmyQMEk^4Jnp_pfX|+ejp23fMlQAzjGU77M8&w(EE(P*Rx$`C_)c6v&lumz@TQ zVOz+6Y$;o8He|}!Bp1@uZEm9ow^o0woQtkgsPuZHHU~cST*0G_tMtK(>i3wI4E;ZE_IO&1`BIQY~!Vqfnsa zejM_xY_(I6Yvb;429kERf_cbxuq`{tbh0HcL%NG?bQMzFY~Slpprm~Z@;z*~JCN(; zu5ureKDPY>$o8}497AS+?e`N{f29Z6;x8a2Y(rO2pyd4q@&bzfg7U{2JT8TF9+)*U&6e8Q`m|oAeYMB+7u+3 zEp-;MX>1eokV$7#i;&J>>sp3XCfofg6ey|LfP5C);TGhwxy#>$B!_KzAF{b@Nr#Zh zV;en&bUxeHDWnS6+Ap9$$;}ny|FTuyLavazy#PrOTh1e7i`f>QA!FF$-ymJWCLfS0 zWqbdE0wv8qkS}AqjOhE7E9b5>8j=dO?O4cGvSlPdriyJg3DVVU(J7FsVe8jWpyW9n z^0jP@S&*ya?lcFIdbZ+x$TqO8|AkB=n=XcQ6We47q%2!RITR@Au7rFu+e030sALP2ZyB&a}hplP|vb}8k zBarE1%N>VwKbxC`)Bsz;3=}9CnuGix+s6Xrgu9j{==UoEdH=Tp*U%rgO+nv7KE)W}401KzfF4;|@}@Y-tZr zpk(R^@^fsFuaKMPuIC++1-8de$U3&Ve~?*ZJ68R_(o1ZAqad}+wi*KkO8&${euZs3 z5pt{C{Y!>qjjb~kvg>SjX^`1qtImY07q?MM&MT zoh(CvlEO8}2e!2h$lY_7vJJ@t+r%zpAK8BPA@ju6bqMKawr~uo7q*%+C{S{60r^+9 z{A45i*)> z(n30oO|?QQovpha3Y6S;LOz47wg+;V+#U8olEqdq2-$45(n0cyR#Hi{FBjVH?_k0wwQzkS}3t zIe=U#cb7+yl(ChaLbjZ3`y4VAY?)V(u4J3Nfm9V+On?F<{ST0@W_x*pTn%@PuaMNT zoxMZ0j;;6$GWBd5{~+DKrXvP_r5f3$qM$%YL@eZ+*m~k2XSsVwgru3R?hj;J*p5;m z)5`WY9nx)VE18gLXZw=_1xm*9Am73EuK;qL+;tQ|(#3XXAluDWRR)vkf;vZh*UwW=ICvTH7EiY*!tS8DcB%hV(GoPA{ZJ*s=zoK*^jy zev~bC1af2C4U9oD&h|P1*$K9$X~;~nozFsgitYDZdcRWBY@3Tvpd@Vt@-uAHYml4e zE^-r+Ikw(y$j-Ap?m}jPt=>c0u^k^mYLTt*1PYX_o)Tcj;A-T(Hg5K=zU?sva^|Y<-Q8zGi!}kh)=OXoUhLC+(2GWh?4} z+#PpoJ&**p)PBg`vrP;_=7CKOL;8`eYZOvXZ1>|(prmFB^3QCCGmv}XE`J`9SGHvb z**CVNWyrjxq?(A+xra^C}|Fmk7B!gfLt_prO%MWux-CWHkK{p12S=JvtN*oXN&%U zR03On#L%w-CC^ciPh@M1g@x_CEvA>&t>aqfLtDT zw@r}bvsJY~wt#KF4Kjb(ayubi$mY5sRm7Ii2L%=RIWGu*X|KvKeXH3r#I zwz5gcl(FqhL%N(Ta}H7!Y;y}xpd@Aq@|A1@E0C+=?qv;s&|+TX_MbhS+usp+HHNf&4Jrd@1BcxQnfTWRz{N z3bJEtuQiYvXZwAX=CAYw+j%3TCfQ1wp+L!IE99rx(mNnG&E0etBr|MLy^x({>+Ofk z9NW_%r03b{hat7Vb}|YDN(v_+@7UI+Ah*a}$}A*HY!ma4U1s}nkXd2tT88v0TUdqE z8e7c<6eu~^g8VvL{x0M;xLewXWRorF5VBirBgc^0X8SsY^bT9w1*CS_Zmys}NyRPX z_t^FVVd7h5(<>uS3~}ht+ozwPuv|gK=RC1U?Ka$w%h`lSGMGKNWZa- zc0%f%?YkQal(hFj{)6py0CJz)RSrS&#kM~J*>AR-amf5*bCZz%VT+%El$!ki>>rwg z0wwPYkdI(%S%O^T|B}lUNTS%v)*&0sw!I0N7`DtENXN3x?m;S!E#?3Ul=L4#KA!F6 z1ab-7HJ(F~$aZ!K*(A2&8^|QHZQMcn58LnSHh!g2*ruMKKuN?a)ab8F2HW2#NN2LG#6T*G?N2-uC>cwHd^X#^WXR=k*O3ZIF56uiWb@dn zG9i=C=CdJPz?Pc_slRNC1yG4uS^-(Zc2xzL61MVMNSCth z)I+L_EvpF%l*~0lzML(#4RRIS4Rk*($cCUdU9lo%ch!hRq11YS}i2p+HI6 z8072NrY9g*&t2p+Bn@o6vyg3Mdz^<%6I=Zvq%GU=GNhW>3fG`O$?68=Ti8;zA=k>? z_%0-EY(M*uZD;E|giHrpIEHj5TlE>Fy4Vgbpg>99HRQY5mTn=}!(HM%B)x1SkC5$S z`+SB>KU>=yqzBlpKOi;8R`Cr5N_KxBFKpS7W504k+|5TrGRzhi2iXy}!34;Rvb`li zdW_AcKx&-rLPLR)k_^aCux({QZj!t7Tu7$aX7V9B%@$P%nHjdeVo1-jJ(WOej;)~_ z3Y45wLVljDs0MNi+^y9?;@DCfA-l*nVIi}`rdlDr%+}QosTH>SPAE`P(*yZcw!=Qi zt#Ow>2+2Cz@(^S<*pfyev&l9(4(TnnuSrO4v$fAafs&g!$nUUKI>_yEx3>hz9$U^T zWcS$?)*<8B;kesk>pF{SPE#nF@XKb@K zkUnRN4v@NF>wka(CC^WgzhrBCh1?Z)r|*zlvlV|q_J(c!A7pOXbj0|t^c~w|6r=)M zL@X32>5hl|J=;Sf3i8Du}%_A4Os#g2Eey3#or>2@O!7WT*-9KWrb(kW*8*5|EZQ zNFx51y6S*zBwJZGWTM!1dLbRnmN@{a7`8cq0wpmckdI{>7=v6KcP|r=#IrR`LpFi! zY!)(!Y`@Rj{YodXZ7f16nJsMv3Y1K(LH-Y01=;bAeF(kdIkkb{#-&nlWqJOa#`H{yM-j1t@9qTIc#^2kjZ7Met~ozn}35;K3m== z6ewBzhI|2AqMG=X`^(*MBqW7wpV5#lVrz|qOflPa0;COFc`~F**mhH(KuJ~_W;x_**wU*YSIgaW4J37J zQT33mXX|Z*Oat4Kg>)lZeJiAz*iPD^KuKX27OGvaM`CLy&1> z>l%f0J6jlsR0mtl6ci{qn1OsJTmC%cy0}|%kaV*pEkm}4ZDbWPy=-6WknUq^+k#X- z+szIXD5=*t&C|K*@a`gAyBve$h^=fAvd3)O(~vn~%bbJsDckG=W`YZ2!I?_sU&|n);QzvE4;N z_MNRN1~MOPJ`U2KY`KY$`eIv5h5{uCsgVC>8%~4VKkhyADTGuMTb6+WC3B^ak7kRlfLsiB167d3vc1+oHjb^S9y0N4=Z%m~U^C5- zN@Uw?g#smM9gt6Ao9=>KGIx=^ko;lm?T2g%+v6Z)QrYTi5A6 z`E<6FS;%E@H$D$ZCfkpLY!+MRGGwyZ!YZV5*s3=mmCJUp1qDj-b|Ig~wzLnqeC`qt zAt_)RIfm?Cw$D?@6tcBlK)Q(S`U+CTY!$aqpky~d-mqmqLav0n`DaK<+2Ydy$W?Qfo&ZS=+e{K z_LL5(2DXMQC{S{e1Nla_qI}3Takust63doa4B2M3i4w@Pu&HuLx3YCrLaL4Jz8VUY z)YL(~o$aszavj{|TSz+DmRlg(#g^0#nQpexPDuB#eRV^sm#w`I3Y6RoK)#QyatLz$ z-0h7(GQgHI4%tDrg-OT=Tl@^9huCBeQp0TT3s9h>c?t3(Y?mvL8|AKa9g;D&?M=vz zvt{f+W`b>Y57Lut(Fc&4V(UMG0wvEUke_C2Jcry2cc+(-%(4~VKz5F8{SGqoZ2AGx z3v825kaBDhuTY?*`yKL&Y!9E1TjH+vA0*3cM{4F*c7?4V3Nou~D>0B>V@r;Q)H>T( zA`~e3PKNvjTSqG7Ho3b^gJg@XDigBXZ2Q@e*b)4-)H+M zg`DTEr2>)zwyP@093-h-2K~!SvnM}5>c1UNjg-%Fiv(@xKf%^S&O~~i4Vb|78G_O=J9 za<=9JC{S{71o;ZK(o@J)a<_F3Nfleh6=bW~W^N!;!xkMNUCY+@0I52*=O-vo((nrT zdbZPd$Te_R^aV*H+xkDqHnF8f%>Bw(w#g_+H?yf&NVTwa$3uaV`$WjMveo{9TpM?X zsnG9BHu0!Z1?iCOU|Y_FOeb4%4y3!-M)M%m&GuaY1xnhBAm77wYarLlU1b?0eQf&` zknLy7sfNq|o2!NNAX|I`q=ap#2?~_FH$#4it)&fe!`xkVKr+Hs)(zQFw(VZXjIm`7 zKzf{QRvgO>Gf1tn{ken!C1cl+Ut{}s z3%PafI_@FaV7q&S>?T{)3uLy~{2QdV*>XQ2wZpdf4FyUP)cmjfF57S<4)4ScjJSQJhA-@LH3!ga}+W!Y+)SIuWZ#*ka}Y~n1KQ%dGnBe zXIpZR``|8d8In)7kyXfkv3;&X=9{f;3)27Cu6H2y!&b2m1xj`gAg^Zs|354H7;+K+ zOXg1@iDZkrfNT`o;1y(|+1_p-9m8e=q+;1F9-u%;$us2R*tT9F7tdY#2P6q>GhdKR zWQ+QNOcGmP#KNz1GTT!Wr2eoq#6p3RlX%FduoWdiE|t5rKagm)R1MiQwuyAeq_e3k zNN2Ehhi$nGvbk(Ym5|9}8?A=8Ak)sa z@&f4&w&Zt6b+V0pLciCD|6hB5e?z{DtwXtAxo)=GNJx6vs$wA9%eEf}nLf7ML`e6u zxnxKUuqC8Ifs&y#$Pco8WI#^1YsrRWi0vvDvcqg;1&|qG+bM+fC|jn1)EL`bDHJG) zset@A+dviMCb)a4fn<`csUEUZY-f#-nPxN1ke*@NXob`)TUrMcD4FVl{2W_kFXZOA z>*Qfb=Gt--6T@Tiz}dC|TTx{5D(SA>?+r8$O0)m+kWuvU_Z;7m(R!yS{?7 zXDh#j)B)RWfC431kB~oPn}3Gf5qGg~kQ}oOen9qw?ez;Xr)>5I(r0Yv5sSZ4=WHd> zP@rTp7V;Nt=?Rd#ka=S3s)Y12Td0QA3tLSc6eu}pfcz_4zJ=TycS|jhyt5^> zL-vDhq!TiqY+v1w{$gwEgVZWcAsThL%KeoLQ$o+7aJr0SQ`~U1;n1pP^|B|>F z$V9Tq9HgVz-WDJg&DOjG1xhYfARoh4x(>Nm?zT1|iDS#yfowe6%pPPC*rE?0oygXA z1gRvp=MyMU(r^y>WVX{w$o=82=mwG$w)H#6rn03zKt{7oK0!K-O}#=Yovr&F3Y6S` zLOz47_8;UjxjR%#zxw^iT;BghK{lIhIR-L0Y{~JE&Se`-gj62ecQOodm*W1JL`vR9b2(Lrk-tM z7}5=FdJIyHY*Q0Zpd?}%@=a_#vyijgJISITqk!Ohmdr!-5o==o2}{$GCgem0@A%~xz~{DV_Uq10woFeknd+3 zeuUfrcOTD?46?PpK~~tVJ|Hv1R{jm?VYZzgNR6;%MK1p;P%;+{`BApmILM80H;@3y zINNIyWGC2~QXn(QcCH~k#bz=fHO;n}1qDjdav?v%Hk}W-S?(eWA(>=M2zr>c(2)SkM#w{c(Y(FiKU1jTRhs+vV=!En-TXhel zHrNjOpg>99AmlgMmWCj=#a-ekB-?Bw~QkDci*n6euYBU@KIq@LLB6QMv!%^%1=vmK^F?uEPjbVy#=mNOyy#+H-=nRm9)JV<}A zeHB3JldZi73Y6R!$bYd_mO<{DyS)lX{;}m$L-vPlp%yY~{{OQoz5&t^|4XC^Qju)$ z%}}7ExefABY?mF7i{`Gh8nvSkcFCXQ`ZARW&ZJp!o&w*E0FP`_`cfP5lb z<22-wxI3MNB$=&v0kVJC))yg@!lqXsoys=31}V)Ju?YoAy0;;p#`draxpeMoJtP@y zM~9HjWGgsk}n~Z!!~ve1xmheA)m|EaSypX?rt9;$!Dv2fouWW{u^Ze zvgLk4x{%F%L#l`^L9PBOP%;z=`C_(@Xvi7vTH+unVY^CzY$;n=GGxlwc2Xc+&X$=5 zsS38a3@A_%lMVSwwt-y8RdM%{4@os!Qz2w)*v^U}Q_E&bAzjC|Q4XnkwzMiJP%>2m z`3APgddM|$*V70|6WgPOtYxcfg-kQsaXX}2*#34ws+DcE2MUz@>4$t9+xQ^l+PV8T z1W5;5=O|=5+3v<6)5TUj1?g@!KLe>Aw!C>LP_pPC-^-S`47on;hF2l!XZu`->;PNq z7Gwt5u6H0UY~}lq8e-c$fC431$B-Xpn?Hrz2zRj;kc_eoUO{$@?ezvS<7_rSdV=lz z0aBA}CC^ZxWb+mBQ*7xUkelXi`U{d7wx}P-&a(AJto_Q&u{}jWdY-L57E%jrC-G3A zq%aBcj&1D^1sP8C%92WY5`V79exM7QF=N zOSZlhNL{f#uR(#5hE2#{vz=~3?uNUfJxFfZ);(nJ*iw%m6WAtCAbroK&LQ={)_n;D zO75>A|HxK*2e~Ki4(}m(W-EAt>uSmkjA2w)j*?sfGW~{-HD|Q1YGu`3Sa_Y{*6aFS*Qx zB#N!90J70++l7#cVaqg-j%AxIg;X3{Oa&Au>92x(JljhRW< zkV$6SXod72Hr)ZK6t<}@C{Pm73;9&Go_@$_?j8mqNn@)UhHN_9(I{jx*#1sHI+JZ> z3i|yW!T&$cKeJGvWNaSt*=+wDC*`;bX`ZarbcwNikdN1!N7|)fHq)*vfAqUCOo-AXUbe^#}z@=AI#6&KCOy zxeD$EJ|L-Nd;NlJ6(-XX=1C-g0y8j&Vf`jTVXyFC|Ugr`4+a6V#u{}H(mls8{1DAWZT&~D4a<_+h;dq`q|q0AU(i#Jpide zwu&JrP_jD$d11>Qhujc%^OKMav&GFoc7$zk4l<)`ZwrtfW3x+;8fUv$fdVBZ>yV#d z+uDTOBzNgMkW8`7>_K*#E$RR=Gi-fFke+3GI)T(2Tf;dNC^@-={5)IH4dfQMTf2kA zv86shc9Cu32{KD;>J`$-R&_XA8b`;ko{!a zzktjaTkbWazuDX^r2erb+(Utqp-0I7uzfs3PC1-h(((pL#Q#!PACQe?EBl5_6x+@Z zq@&p~Be#B~V%X-Qp+HGY9OPr!1`;3_$K6X3B=KxbDUeNIJJXO!WHT9%PGZ~0f>bhF zS}qhQnaYR!AGXLs$fa=CQw&Ke+hYl2HCtUdWYXAbBjhvL z#x3Nsxck=vNj6(&J7ja%?m8ip%U0b3={z>y2dR9vyg?{XvN#0!0=C3a$o=JRcpQ>K zw$Dk(7O}O?K&F`OdJfWtt=vJXgl%^T3Y286LcWx3ejRdU+{JD|QqDHG1KA3;*FDHo zve^SjSFxQRL8_XqArVp?Y&CzNK*>QW z?GTa zKxT?9dIZwbY<**pnqhmMfC42A(~zHKJDr8x9Ct+vkj%5KFG6;KEp-Jlj%{)c(u-_r z6H-fT-P=&0&`?fgo4wxLKUQ1TuP`2)6= zILIAxcbNdm5nEX@WRKalQy_D~mYD|WQ?}U*NS(38WJ7_H{#?kPv%Ta)?t;6#a3JjnQOL=_*LwvQ5=Mfs%-N$ltN`G(s+L_h2EpXRB+4>;v0TJ7gZ& z{&qq7iEX6^QqOFE`k_F{*dXLz*!~Sc?v=ZaQApm{?#3be&Q>)AnGZHU1L;q;+<8cS zu`N0%P?E3=`ER!2RmlD0?qeO2AGX#l$g0Kv&;F|&$VB`vRlX1DNVc5=NJX(_9YcYV zxl_nTv&CLOE{3~-D@bD5UT+{9$JP`e6VG=30O4P?Gim`6RaKFUTcx z7x@FpAGY3z-Cx-hw#O*Qq_Wk=LRzyO$3rTOtuP4+l&t=Nd^%f-hFk`B82uwx%Ih&0YE&BsFX^3y`g4i&}zA9b4ZDr0dz9)*#iu*02c$N=~*R-^f<9 z2e~Hh);uJZE%gYp&1@4VkZEC4=a6n?>$-$g8{7Rg6ey{=gM2&N;XUL!xXXWnq?2v= z1+raiN$-&9W*hy4bPwCtH>7&m+ST5#0wp(*kndxwjDcJ~cYAS=46x-SLUxdCAsI5l z7M}{~AvQ^a)G*t71{5f1&W8L5+hs1~M!733fMkqqyAZPDY#9bJ6Ku1kke+0Vu7K1O zTYnW4D0!}d{4`r*J>+J%J8gtymaVuMvU6A5mF~?bI(wqB<2nBr)&cskUQh<%%^gEkxfYb+Do`nJ>i!G4!%Ngi8LJY@6PMiL=Y!1k34>A!4ksgNpUyGerrB^8;F zFJjxvhFmds*?Eu{wuJ)7maxSYL8g>V45Z80-bx`=&emK31xhZeAYZ{&S_`>K?zZY7 zsbb4$f^0S0OfzI^*rMAYUCY+j0jWB+=PoEv($EX}dbZPk$Te_RB#<<+tq()Ci7j;u zGL~&}0@BTFY8p~4Y~8a^pyYlY@~v#Oi;!#M?r<5BcD8~w$ab(TZ$PG#EqNQ#U2LPf zkm_dp-iHDu?T3)>VY@wsTrYQ(XOQ%chcliOy2wT}VWJlSye;_l)mKk~QD?QFO8x5%mwwO35P|}|O`AN2yB*;y1 z*O&syG~1bm>8~Q$K68-B#Ufy<&a%s zJF0}tGTYx8NUyN1)In;M?N1{VC>gVmUt{~%0=aeWI@%%GV7u#t>?T`P4`jC3d>^E@ z*>VRVwZpbJ1O-YGMj^k;HarfwJ?=gxA=zhZoq?=pyPAW{0b99)^dZ~M5~PmUvR0u$ z$=o{RkJ)0kAa}yuzz!s*Y_EHeJ!5M+fXq4D`4OZq*vu)UF4;EEp+HI6738njrf(p3 z&0S=GpWQ!VsOcGn)7^IWgo+cpmhpk~6 z3Y478LOzA9XaRDm+^sD_qS;bcAe+WEu?CrRHnj=q47RRqNM*9!??Qo+8V~s_w!=fn zWpkH*0!a?r@)=}v*^(|HlgBoC4e5NguUkkJu(jVqfs&g?$p2-le1TjccYAM;6tU%e zLbjM~;TtlBEnXe{N|&%nB&15&-lL&FNpl?J%h)axAXm;^X)+`gY}+Z2tz^qcgG?3M zYzCyO*`l){Rm0Yw3k6D^^C4f$)>sI+I_^%3A*p98E`@9Z+j==<8rgIeq?_0#YanIW zBI=<)Np~aUo7o;L49_yTXH|7I@!hsq2E`8|9|cM zJp}nKwvJKAb+g@$L(;=mH3ivTw*48%^s(j6L%N^MIY?I!)#^ykQrgyIe_#iTjnvO#@Oagp+HH@1?0!s2Cg7C!QIOZB$I4S0kTtU zXAh8>W;4%_o?+W~h14ut+6NRUnfikK99!fM{LzqKVH;0}+$wkfG9g)G>&$`dI@?_yWH#8U|3Z3`%@;vxi!HAN z3Y09CL4KPpu@Z7S+znSlvdi{a3)wxk)&|Jzvt2hq+Ow6nKPw+#xEWOYLRkZryj za!1_7_Ca#YHaGy;6Sh}@%qg24f%F;M`52_m*-9p%K*{DbU4hIE+tV7PZ`tZMA$7-gvJC}F3ilu%*w#Gc?zu}jg5-g1;smmfY(Hm^d1C9j zg!D68xQ5gVTg@F5C^@)?{3~1j6Xf2wTY7=yoh|7dvL9?CpOE=v`}&6T7h9V;`IY)+ zyNQGXB^5D{|Hrl$2e}{avJ)XuEB~Lj3(1g;_+Jv23YkbYNrQA0+gk>tqS>0Wp+Lz+ zF63j_N(&$t%iUHXBynsR2D0&NGo_G8V2iGRbRt_{6{M2bo@<~$Nkcv4li5xiA@_&7 zqGm`^*w$Mio645j0U6CU*#+q|Hq{HMbhhq(C{S`g2>A@Q+F{6La(6fiNfukd1Z1<> zmZu<-!0Gwac}V54eLE;n(!LD&e74(F$Q5u`xdF*vw*4*07P95+LZ*n#?L)el zE&dQvhHdB=3Y5H`LcWBp)D}!7ocO8|Gbg|u4 zL$;f(stz(eY`y`~y==J_QhjWTEl{8&p&j!5Y{Q+98{qDv8MWLMcb-yyTc7Cs@p&Q|>oQX6as>g-p6lDsI$Z?Y}LKyHh>#CS-y z*+vo}yTkUG44GZFwp2*(v0bM@YM-ql6AF~WefV=quNDkTJiXeN$HfSJo z%=T6a=@T|v0jX2Aiz+BkQc?@~Gq$aI$enYS-UR)=g`D?)&5*rhi)w?+62JAf6G=Rkh|k|H?ek1gjKvOjDKw~$e*|NpGx?;#!WzeFA(70LGg3-yk2wcKHFhXzohC zA&Ftz{()>PTSnyhuS^`W8Ib+M zww?u<6gHg;=~TAKd`M}wh(ah((p?PsG`5El$fa{vTMkJE+fgNCGuaAiAd|(mQU~d5 zw&X@g<*))691K4(S%Qzh97QWn29R1xo%zT>Q$nv5iMTuARGoF_3hyb;d)slkF}MGF@!d ze<0n>=2Icn!YfEm;<>!?uPRq>1X>afb0NUYY}7y*{%(wg{`~{QbTOJ z6;Plgs~YmdZ1c5{8{sat0g_R+!6wL#vAs4!W}MBoL3)DiyaQ5`Y$e@Lpk%Wb@>6W- z1CX2MZdxFjVT&4p>?~XF7-Z(yo+cnY&sIMTsRg!^Stw9axBz*_wzdelMeb5oAX#FY zScB{`+s_7MR@l0>A-&2Lb|JOKR^y>S$-yDy*V*zlL46vw&-j~U$XV(Lh6d`IUfp?G!#Pq zn(ee0ayQ%+l|pjMwq6d|JGRs+$ON{@8c5%>sd`8~uyr>=fs%U*`A4?eR>(bZci0Ze zGh0CyWM9~pdm!`5mfR2NH@4A1NWHUt4?%&F_EE@xu-%SB?vuO9DM-H9_Gcjb&6YC{ znSX4~LHdU+ei>3~?fSystw(f~{o>a*_W_E_Wb_Vk_H+Y&6^U0c2v>GLIo0 z%QkxosW`Tn3n)<1e+Bt?wwD{oC2-dmAW399dw^^bTk$hwlG!$1A^nF4u z+5lO@cGU!#61MUdNSCthv_YzjEvpj>l+1NQzML(#4{{aU4Gch1$@VIctzv5$flM{q z`52^Y*vurPYS}iYp+HI69OUcRrWYVr&t2paBn@o6E0Aqudt8G|6I=Zzq%GU=Hl&)_ z3iqHu$*PBZ3tP$&M92+sH=hj2 zFk4(IWJlNr(;zd-_Lc$ZF*chGsd2W8TqsadQULi0wyi?QO>&oRpx>|X;{9JKWT)Ap zDj+k%)>j4TS+=JdNX@Y|)I))ilSatTvlTT%Zh^bCR!AIMY6oN&*(SOmv&5!)A-&Aj z)eor^w);USP*O7t`Bk>VQOK=vmp=i?I@|ITWH;E7W+Ai5HaZXKEw(QQscp9QWhhW` zvkLhgw#p62?Q*xb1<4*;&Msv4*%tO8Max~Nx6r77Td%lDxZE8=b>5t3rIk7US}u(hQ@rj+d}9nyyFPZp%g z*miQDKuLB!8``8u}pB%yv8g1xktq@|JCF1ad9hrH(_= z$~HL(**3PHX~?v*b@Zv09b`t>V8HN*Cl1O-YO zQz1XgcA_CS$6ZM#B=cHuE}I*M^d4LM7^FPg@B|bnd7Fa#K3nSy9WkBkaZ7~Z9lqBRr{)=rSA9CN^y%$3A z!`4;;S+()s*?(yu6Y-zap9)AvvTavFDvB+;1`3qS)j>X*Ew&MIG29I{LlVpO(gN8y zw&r%o#Iv1sLOOxX^gt?+?Oz`hC`liLd=lGVfm||ok)x2Lu=R~YHkIvR5;AFQ4Syl6 z*^Xu*mCjbY00l}`7a^a)mbwDDOztMuAjx9;*??>|Th|t3a@fKSq;uJ7Jf!m2_79*y zN&Ye9^Vyb8Ay>d%;sqpyY@=6@En@q)flM)5`yHf9*sdNRRmxWR3h4#zf^8@gGJn`!qaj_%X5%1L#de+m1xm`2Az#h5nF_fY?lRILsb!nV zglrvKR1Re7+4}P!-N5!(0I5c{#$qT?a$E}eCbp7t$Tf4f_6HKnmR1ef7PiS+$h5Ml z21vKDbvHq(o$by-fs)!b$ak6(&Ju zm2D*j(rawV8dB?Q;~7w(Y}G}O-D2A-fy_2rUKyl!*jxpq zcG(iDpg_rR4dnOO-s>Rexod5NWS{M_8L|g#6|ImtWZQ0s^buQD7o?8a=6aw&NlZWF zPuK7>BIG04=93{8#a(O~B++a`>5z?Kd&z`MESt@NbR64R9;D*g z$_k-C$-iRAC$ME0$R%?3w;Ylrwx~+TCbRWbLnejou@=&)Yz+;NN@F{2f&wMQEs)o2 zYi*EA=PtDqk_@)VZpdb`{q#a6i>-SA(%Ecb2vRw0wIfiVWPc3uxoicKkjvw4X&RDz zwxn6e7O;)ZL#B}J(?Pn3t$i6%#cbEBP@tr81M($oyPJ?Je-$?p+HIF59AxzP9h$El<@W$V#U zpyVzC@@;H&*^q1J?jRSE4z|Jq$ab=@hDC}hUjvL+xs!8SVusY$k&87NRPFbDZ5w&w-NO>@_@1j%2v(-p|hu$8VuW|nQ^ zAEf8l^fsjC*`{})KuN?txAqL+e0^G zZrK|8ARX9_1|W6ERxD7UWOW4c_iU--kbB^6ViJ-^wx4OpKCyMpLgtw*%tQKxt!5EY zuWb9vP@p7#4f1bnOB;}T=Pq#zk`K1g9msyNee6Nzi>>_t(%)=XN09nqt2~7QB|GPk zSDXKx0XbKYi}+76e*;M*TihLFqu7QXAQR2@`UL41Hv0;xShn+bC{R-N1^GC(%^%3c zbC(hM{3}Udn~8>OB3o1(WRlqW6Cj<;_Lu~z6t>1xC{S{&A)m@tk_ovq?$)v)(QIjX zkWFWsEPzY~n<|EMCR=wYq_WuV%Ai0=?H|ZzvmI1HE{D58_J>*;1?jIr7%3a+HByDVm zZ;)+gEBu5^2iwXwq&wM?BVK-`y4c2}pg_r24CK4nI^!YN!`)3HB)x3aDUj`B+e?E? zKU-b~qzBks7NiE*5^|wH$#6d8huGc=At&6mmOwJhc4;6x!d6iMnNhawN=T2fWz|4x zoNcZS3Y5e&LVkj6uo-fb+&#BIGR4;14%un8(@x0zWivgHo?+YQgVZcr`XCf2nHI>; zu|MGKHxW?Nl^0wpOckY8b&ScBXuci$V3 ztg&@%L3W+(b_X&WY&9Oz|JeKiq&C^|kD)-x;wj{}*b*-wx6R$i6(l=sA2*QQWox^G z%pTj-1Ef9MpJzzzv+cY>fs*VG$RDuHe?jh$yIA%5D>-5tiiGSj+eGoq;A=cOQAqX zaXI7z+u9$<-Eo&%4aq&*WG!SL*na9E^T^iS1nDQXU?KI)R@(*zO7=S-|H4+#4Y^nD zmU^s}&5M(~sK8GRw$<{swsV}zc2`ErfISu)5w%r-X{cx8v4~g31^`C=m z#D9{wWynOb$tt9y*k0Em70uSN2?a{dw;>5)CR<%B=1hU0!xn+BYeXMtQTcTJ;^)Uln8L$;o+bP6&J zY#VfrAl1Y+y#NJDB9!%$%O1ATXPO%rr6H%AU(}y3L*8E?O!nzC`mVvpJDr34!K$GA}b-8W9zGi>^$2; zEo2th8X6$&*p8YYwa8Z70tHG|+aSNhmf8uqW$q@rAz5Mj>4oenTh{<&*4V-jq}SPM zMj*Apwm$|1O7bTm|Br2H8giT5CC);!#Wp$**=@ED2bmqV_GL)#vR$o0YLBgQ0}9md ziv`GgwwxWv?Q=K32gw0j+yP_{*@liFbHw&~0_kHmdk(1+w)0CUP*QdS`BS#d0J$^n zG9Dl~XPbF~>;+rYD`YO&`rjda#rF6KscW{zA1F|A9P#lhf5TQ14Y^zH)?y(EY-tIQ zy?TTepVPBimgD6ey|9hWr!TK`!K;xhp7u%jTOEM{B`M>OZ)cmB zgj@%A-_wwEvUSZuwu|j{9x~l*HH(n$Ve`w7>SfDcg90Us8<6i~OWcB7KX)TL(C^R%Y z6J#dX>?@=v+0Nb}HN{r;1qDj}{Xl-2EhF;lSMD!&f1@FpVT+1`>?~Vf0%Ydc9+MzF z&(@F%sRg!U4FyVyGa>KT*0LeD$X#k4Bui|Q1(024`zeCV3R`z6q*vKO8Klkud@}@LT-b*rFuyIu_ZM@c9U(?LS~EYvlY_YZ0#M8+F`ryf&wL#y^!B!+wF(k z9(OrIka)I*VaV>Y#f?GcfK4VKeaQAY1*s#pmKi8ezYp(){4ra(gWL&sn@f zkiKJ60aEvDJ@-(c-0wocnkWXXl9fzFe?tT)Ibhi4xkj-E_ zoP|s#ThRifv)EP^A(hRRvH}H4#@8U9!}h%axm@l#w;;)5yWN3oK3lbiOaYrefOH{S z-Z7+#*cMNrKuN*{z1$@ZK+?xHIt1B%wvS=R46wD2 zL3)twY64P2Y?aecpk!wT^1_xg54mCP<{cy>Y;nub@3S5LyS5lwh0GY+>pG;z+3Y5y zCfLrmp+HI59^@z4HuoVn#a+e`B-3m&Cy@Qi7IhAp8MgjQNYAo8UPEe*tua7>lH+^G z&$E?0L2iM&wHHVnTiQEh7uhC1A+yA$ejvTf)*Ye#_xtr$*zTgBKuK*Zuk#@klkQQ(vbPbHkJYDO}5W0NNurowL&>vsD#AZilavL|erU646to9%)0 z8C!Heq|Vs}2BAR7lR*B0t!Wf;m)xC!gWNrLb^joFU_0D`>?2#@E@YnARy?Gi*^&<-^};rO33t7!p-vF6(wxcFUXRsBu zKq`}MwG9fCq;x_)i*2GCa@pK{_d=4x)-?dxT(;XG$mFrrj6gb{&5uEOjbzqhpH^`C=m30vDTWJ=ktRv~TJ{%k<1jBRHV3Y28;K)#%9eh+dL z+{GS1@`r8c2(p!IFDHFh!&1}aJk-rL*6h}kevaQ8Ju7$hQ1V~!hCX*oB#`co}nRd2r z4e1WHkO8Sqw%TkcP_my3`7XAC0?2i9w^Rg44_i_xWP900%OKOo_E`bxezx{1NDZ)E z*Fb@i%6iBTvh6lPZiu@a3yH8Tv_f{6Ev^GHBW%(I=~1@V9!QO`we&-QlJh~xkF%8z zLvDh*%~41u*)k^}JH<9L1(|8K=ov`=W$T}V)C}9x0u(4|T!Q>8+sO*#=C~_ahh&~@ z{U2l(*wVHkBWh=af>>AthEo9c& zlJ6nC!8Y~?sef!=&rqPG;|=ngY&Rc}+v2Y38_7xN;nY)2}G+XQ)?99#zB(8)|UX;RJMmC$fU6~q(WM=9cf6VvlVATfs)m1$Y-#n z=0Pr#yNLowvea+vXDFYPic-g`}2kW*xG1Y*Cw#sb}lohI9kl<1VBc z*&6quK*{kTmM!fPvMp?r*N|ytQvuR#Y~A;eYG=E9gaReCFOctG zJ9vX!CwB#(kaV#ve?zvLEh!@ASEh$;EDF-SY@acZ>SODOhXN(niIDGSt4e{~0C&4- zkPNcrW0vg>htvq$TOkxEX(@sHDBFdB+!%M|6_AXxZB;^cf-SQK zGLvkxb&#H7i*AI}G}}Nk6exLWf&5>#rgq59aCg!P$t+uG4`k=q*83nc&!z_dK(fhJ?IF9xws!!TZMM8)Nbj(@Q%LQyC0sy(lHn`J@3Fn#K+bd5 zdI!lq+vNjf57;W6A#=#K{R-(LwyY0G9kb1SL4lGO75giH!ZsKQxl`_*qaitCYmS5L zIooLhWG>iDGNdorHc}yV#g?891xltfA%D#lnFF~S?t1eexn+AOfNWr^FNVw=+fgZ` z@7apVA@#tv`UeV>q*O!xk!_+Da!=fS*F*Bm*3|^r7q(jqnOC-&Hb}p*`3^|Ev*mX~ zfs(~u$bYaU4nXdcyOAMCzSus7A^XkNHU^m=wyOz9EC1iw|7RLf5&uc;%s_#X?0LvX zvduflMR6Cq3`sQG&?;nO*k0Bl6U$~dAsxqdwhgIxwz54aQ1Wjd@(FAiN03Y8?(YdC zNo-N)kWFUmyM#;%+v7E)Q`s5izhI#PvSn;>RgfuXlNv}@u)Wqn>JM8> zBNQk(Z-#s&TX`$ws<_*1hoqV#J2tyvdwI1bC9uYQwxx8VN*+xYGvzLfdVCWYmjeatNRDJcJ2+p>pD7hCcnq`TS1jv>{<_H_ycN;)ne-^+G$1-U-%s%|0aXWP4j>;PNtBV-2I z+%u$y*y7(HC2YeVP@v@P3-ZHktt$RkZiKswNJvK6DqvSsf;fs(mB$RDxA9zgDxyTKz!PS{>fAbZNzd=8m2wzEq}pR<`8 zNL{e~3s9gW{Q>fqY=561cg0=gDTK52;wT^Fb(3QZ@|vIJV7E$i;J)F#$;e+sqVX6WO9>Ad|$_ zKL_b#w#NlXrLZ+FL4lIv709Qum8?T9jk~pfkZ88FZOEpxP3}S_gH7#2I+LyY5K>ue zcgIkmr1lK**=z?Fkjvq&;2M%#w&h#M=CLK+LnfbX>=DuhY@g4NDrD<;g90VjACNC% ztNMmqF?YKv@mEs9mKz1xQnrN{$QZWxcu1GANg|}m+1`?&KuJp)Gu{BjfuAaM-YDgN`O6wro$hO`9 znI<;f4C!XJsTN3Cwup8pP}0*0`4+bOZpgKASJwwg8{6RkWZT&a1u`9MD1WGZgY*EK+kn&{Tf!C;C>h>? z{1Dsw9^{0()&od}*)ESDJHl3R3Yk&1?Q=+vv1MIBYMgEE1`3qK+(CYVZSVndliWQ& zK{CbG{0iA=w$pdW{ADv=ke*@N_<__cTY6;DuL330(U6~Gi;RQZJa@ebkSwr0Bth1( z)u%#ck?lxBdWo$l6H?1;tJzSXBqa~>D{K=5kXz;My9kmswysjhuCv{iL1u%k<`1O* zvH2=UZL;OpLV=RSddP3FB{o5Bo4XMU$qw5`D`a=s+BzV!$9B~PY0vhj7gGCdJN;0g zBzp+*2W<1hkUQiqb_|juwxJ2g9<#knLBCJ({qJ0{Gmt)IJDY>l8C#iy0ww>JAb-x5 zu?o2h?*6Vra>*983E3;QzHP`{vpw!Y`i8AxA5yn$$A?g$r1%8#fo<&!a(CRNUP5xu zHhB%%2ezME$UL%j-$VL|Ej&W%nXUE(3Y6@>LH>oU;1hDM+%0`W^2U}Fk^C$B&Ndna znGd$l7)XDzwZ}v1i|slQ3Y1i)K>nL;Hw|(>+~s6Iq7Hcdmj&5~|0Hp_kcniId`L&J zy%s_$nysY-3Y451$j7jiS3oY7yUj{S;@C24AREs%QwNy@w&+GkC$jZ7Ln?{wsRas@ zG`2%NneC(#aw*)E^gxozw%!NXG`6%s$Y{1Hfpj{X8iiB_ThBNYD7l-2d?s7nU&v)~ zcQ6Y{He2BWWOLY-7a^0&mb?P#JhrhlNaeGAZ9svNjxER+u-)uHu8_Mb4@nW*-T`Eb z*>aB|Q^Mv>AzjKAe*r1OHhcvIO5ScDU&hvY2f1?YE*>DMV5@kB>>swRSIAVdWqm-p zif#4_Qq^oRD&<#!l7UFb*RVZDL#~#)rZ`CI*iI85ThCUS44DSDjZ{cCvgve4HL*=+ zLV=Qq9LP7b_2xm&a(7<2|i2KalEROR0teCF8Y_?_~R~ zhg=tTolTH*v)x+A_OMmAL8h0@cR;$2Ew39={cMZ9P@p7X0P+KDBSVlItiC{gi7oXVa?9LJd_uCq_VW$dRkp5()L)r3wh#sBb+(#VNNup~$3uaV{3OW# zV_Qmr+$ML48j>xx(G19LvwdViW{0gk7t*_ISNV|IW2-EJ0wp^okoRmkWsuwFZoUGN z1Gcy-$R4r{)j;Nm?X?cl$85F{QYUQZ%}}7EtQGR7Y@6+nJL4{+3zBoTnI6bqutoJl z=8~;{5Yks{j{>P{w#HE?P;xvD`5U&9DahS&xAqs3z?L=#**mt$1<2g9sU=81uywCM z>XGek4GNUh{)7Az+rbv(p1CX7h2({8*+ce~E$I+4Z){`7kbYE0o@&Uav)$K1E`z(e21qj54x1pG#a7q?nQXR| zHc02NC3iw9mu}^!YvQi=9g=3Y zhfl~_w)!8)w6GmT=wInpwxVcAwXvXW9DZAT!7IxB%&SwuU80EwCN0K!N&wSO(-B+uA?KEpnH- z4apMQ04Y>{OmTn>W$Ch*t*-f_5 zN62ikeLh2ao2~r~QafzdA5fsA@*DEIY`ZG`S8k8HoG3^<+d>Rv_u1m&A#=bciI6^I zdrgMa5nD?d6eu}Qhx{>Hc^2eOxZBKux0}Q zcLxKIJh2rDWS`lVM>(&nGB*tQKWwpMkgMcwZ~~GlwwEc$ zRZnph5 zC{U9B3HctjrEkdfa+es9`77yT8;yc&KifwPWCqyU;~_oBc9jUJA-2jCC{VJK26m1xk)PAwSPn(gV2#?$-JsacpUWkX>Y(6v!;G zsZmHTvvrR{YK84?5(<>m{)PN1+rcd4*0?KJfMlI*c@eT3Y)LDS`NuZ42I)<<&kaaz zv2|=gfs*ST$ZxY%dC2W>w|fA|E?e#~WcSz>P9fvj;x8b*&n8!pI$(RdfdVBhcaT41 zyLf=y5qIU!kQ}pZy+ZbcE%O61r);xdkUnFJR$0GN=WGL!P@v=~8uAxxO>vOB*~6I-4N(#@&sDdmWJZWXtP@^cS1! zh154&!T=N~86JZC58L}Nu)2GY@NS@V#JVViSM zpd@A)^091#tB{N1?s*-Oc(&$E$R@CzZbK%K&Fn!siEU#aQps%TM^K<-`ULVRY?0@X zOXaTj5|T8whik}cw)y~>bhe{=NN2DWJwYmyZS@5Tl%%{vK8tPQ6LQ(yeSbre!`2m% z{VSWxb{hqmJhqxxNawTpct{nnWTl)p12H38zpg>9GE#wE;cJClJ#9huKB*M1v4B26}xHrg* zu*nCcN7-J#AT`F;qH=x}C^?UW{5V^A4CE%b+l+%`k}We4vQum`$&i_5i%x^|U$*{q zNX@W4WkP|H#vI7cvYq5XZjQT>LP+M>){7y#z?Nnp91yO73bQ zzsy$G0J#>As08)Vkmk~<;2!8XXa?!01A{096|n!?fC?9=iD`&Lvq1(dI{M}w$dBOT(NBgNMEz* z2T0wpO+P__l89Hx-?H_-LoRT4{|Wv6izl!Dejt0#b{LWSEAzls6bU&gOF={lS)30I5&5#UdzBl28iyFSd~~ z$bED7UIEDuTU!-m)yaQn|78tiBL0*5QxEA#w(UkpMX_aDC{Qxj3i)WZ*bd0Wa5vZm zNi5q-4`k!mn)@LW&vrHl=>#@245>u6f1^;KBz*$%No;?oAeYQtvsNV3>|JY=)kx(*?e!xoMqoy%5p z2B|!@{R=2il79{Pe72=q$Q5vxcn?V-+vp=?i`YJ%AydrO{s!q1wyO_Fm9kZSLxGYV zmG>)e*m9yESH|6Z3?$`jaq*C?U>iz=%pbPbWJp)C*)&L1v7M(wfs(Q;$XBy%=0L87 zyNrBDYT0HAAzQ~5RRWoMwtfTY2DZm?NHwxGRziW2<7&t^v6a+8u9>^F21qPhS~Fx@ z*d|*b)5@mWA>GE--3h67w!3a9P*U3m`3|;&0myZ7S0Ipdu`Q24wwoXHNFT9f4MFOd zZEhF}l*Eid{)BCC0&=I^Jx@V$#@0Lo*>kqjImleF83*Z0wv8o7U9qLFLV=R$b;w_{ zMQ%dwhP&QvNN(94b|D+s>h~dY$98lG>3g=K6G%O@*x|;_EHF$ST8bhkOa!?g8XVxyw0*#IP-# zLbi-8?gBF9Y;pzZ3bxl9==UZO|GiFbxq|{F=MRvtWGjD$ToreluaH!;Wqv@mhHd5x zGPP{cs_<93j;%iuQuS<4(NLhIF%I$#Y$pkjYvisZ8ImTp^;F0Y+eM zM-$|G*={W4`naoVgQTBruLH6JY`NW#8Dw+4kRD=-AAppw4G%$qlDA>V53{w7L2iV* ziwQ_Z*(#7f7wLrMyFdlJQT-ud#i9LvEeB&WNI4$p+hP6lDLgRmVbRlg-CNdW$VD2~yi^ ziz!f`Btb)dhixPSa=YBUXF;;Z)|Lxd&vuy)nSHiDMUXyV+b)6BAzOAC6eyXifcz0# zY!&2=xf`s3^|4zj0g&5e*bV>@ex^f{Yph13Pxzji24lHLXROSZo~kh|h8vLBLb zw!T5g-mpCgWNz6SMj;*8j>aK%$5uQA1xi-`LjImDbq;b5+)XS%^2qkH2-zpLt`*2U zvxPNCzp&N(gVZbA{uUG{$=`+i8{3kH+&g!Phmd@*jUGexlkMXaGGA=%7m)sDySjqZ z4_oCe6e!ubgSs+iDZj=gKQMr&q?m9OUA;%SePIfo&!kvWaX_X^=@`>raPtGTUP&q*B-#bD%)UaUSGT*-8o_ zm&V;%F(jHT%|JGtZL%CP8EmQ&(wS`C)sV_!yQ_r)CAAHZ&t^Mlf?N)F1uc-|vMskk zHjgc-6EgX1W8IK0VEgQaR3Tf(02C;>9)f%kTh$2Uin-ezgQSElcM`ItYzxznF>LX( zkS=4Bc}SJBy*VgQ(y|Qs3buYwv|{&cd{iXK&p#vJP8Vve5F9Xo2^qru7|su3`ly} zs6$F`RXnSQps0!R45wM+h7;uCb@g=fn<5z8+n9e zhwbATvb$_;Z;;tzyZV5%XZ!OFseQH`Rr;$yNp=+E57_2oAa}@JY&;}KY(t5VJ!X4J zhRg|@O@s6)+gUoK&e+Pbpg_sL9LS%uW#mKdg1f(kkX*7wl|c52tKO?v_R%d1Ff&hwMAs=pv*!M zvrR=qI-O0$K`Mi-CjknS+$BLildUcla#`FRXh^cz3Ns;_!?v6anOwHyJV@uUjTJyD zpY5v%3Y2t|LcW0QrVMh0+*SR7q=;>=3bMs)xwVifVRQA6E@g{vf|Ow!wostttrhZR zY^@!TE9dT_3z7=9ieAY6VcY75OeI^^5TvWvW``kF%@#8T1xf}cAYa4wJO#O0?wV#G zsbf2xgKRxpse?=d+r|>48`<8_e$+rc0V|MlR$ z3AS+vvT+IS@Zk=@9fCUqcR2Lv%ctJznj4C&1v9fCJ;b)13aMeXtaK<)GMfqc5w_TD z$c=K>p9{$t+edi+OZwALu!GounP*5tn@&Bk?l`EWY=4g-x6j?&DI^DMaTkz1WE;4G%n{q`4Wy6R>>Z>|*v=oIKuO6n>b;924wEp)L%$Huyy7@>XGd(4+@l27efAt?VuQP&)nseLh{14R1Vo!wxmkP zys?c`L;9WVvldbxY;6rtpyb*@{*$eu1#(~9?Y2Sk&6eE>*&nv~Zpf&M|IhyTK1fIW zFOdOAMY6pOLV=Q|5y(feU5r64n!D0TNMhKwrXd^4mN5sJIJOxF>3Fv2B}gT(^{qgG zlBYGuC$cqcLN1BBlWjI%}rR(=bqLAIScC{U912>Btlxo5}?a~Jyt$q3uP2V_UtUcMkR z#%5Lduk<+EStO(;*h*rcK*>fNPM|gYXP#iY^e{B32YNjkiKJ6 zuaLTD>w1R*C3l~Ye_*Tmf!rf^2NC~%B~NVm(U5&+TZ)Cu3tMsmq+i)ak|6cQ_LTw! zO4>B!-`Q@`A@{*uMHVEVY_?Cmt0gq62(?l3)yJ4t$N7Buw`0E$Fj{dLn@9frVR>|^mRZ!p6$5{atYiu^g@!z zcG?fwB(~x~$Rx9^4?{YIO^-q958LDf6ex+9hI}ep_blWzclYy@eHF4rE5yUiTn9%4QEBHO6*+1O-Y; zP9Zt0B9`HeU-F&lcYR z>3ue_kUC&{YlZ?PO>K}rWV`5q+!1%B-H;r!ZS_L-ge_wLGN)`agOEOBiynd0Ia}Wt z6exL`fcynp!!+bBxjUJKLH3$$Z4ojzYz@kP2)On^2&nYa8-+Z1=m6 zyXUTEACd>Q!$ZhEvgMyZ=80|j4AReR$(N9NVH>@M0wrI!kbh-szlYo#cQ=oayt7rl zK=y-e?+r4aY&oBh{$g|Akosmzh^YEipky!#@;_|vF_2SNxDt@&ct|4tm%2=ZY$RJ* z3S^?#wo@S;&6b%CsTj7|OejzilMVS;w*Fkm#c}tX4@o>*V-aK%*iH>(64^`{q?6dz z|3NC5Ev*U)luXt@K7}o^9&&%U>u!W3mF=MkvYM^76*6gTN9~YKXDjG}R0i8h4-_a# z>4$tK+n7Kui@Wb3NdB^Qj6yb>?RFe8Ic!x^kj`cEGmy$-%bkY;B?}9X&u2?qhFk%6 zL#vP!vVE*Wwur4|3o^xQS38h4Y~>zOC2TtfP@p9181kiTbEl9i<1Y3Bl5)0zE6Dz1 zd%1y31)IHtbS2x_1Ei|hN}i!W$;K<>tJ%^&AXmfP)E6YRY*DKESGJC=ClWICY>&~9 zZeXj6gH$8iaRL-5DNKgEWn29NxhC%Zq(RcmHl6|57Pg-($h5L`=0LiQE#yI}ovpeM z3Y6>@L%xG8uM~2f+%1+v(#4ik3E6J8;cCeAuzl7-x|glB0aATz*A@zt{A+=HKih5_ zPC{;syUl4x#@RCF zAUnY}?I1JB7QF=NDYo7fNKLantwDj3`c25su$^o}ZkD^EJxJ!**7hMg&z5=w8OJtp z0_g=dbq=XTwysMkP;z$-`6afR0J&xE4(=gYVatDl>?+&R3uM;VlHVb{&NlK1sSUQT zZzxdG7E$vnzsYtJ1-UKmDq<(LY5@dGSTneQ3*y1&$JlkM86exMig#12R zb2j7-xVy-OxH zY^RNoy<{tHhRhY)dMl)_*>ne_ZrCQfpg>7PFXV68y89s)xVslf?$~OFA$!kuI0~5u zwt@*rKe8=PLF$PuWflsQjLt*;neBT4axdJqFGKRmcDoAMH@3G^66}U4CFGn8!Lq*lkKM* zvRP~$m5}+%7OEkg%~n+hsT{Wb1}IRH+XVSsw#636<#CtT4oN=Sa3^F7*gm=;Q^?lZ z2k9cVs{u$Av;7-_0wp^mkT-09$01k3-P|N3rEGCCkS${yn1f6?+pB~0KQ_ArsS399 z6(~?rvJUx5w#`k*RdJWT14%X8^d4ku*rE;~Q_I$S1nD}q#}i1^v(=wNf%?4}fqVm7 z(GBDpxmyj8Shmy$$TqQ!KS8FMO}#?8g{|`)Qmt%vpHQHr`UmoDYzGl_zjE!|=;{yh0HkHOf#e> z*rMAYHObc30R>8)x*$Kr*3b*NY3@$?A(>$-9)#>H+uAT>=GgQYr03ZtCLrb5BBr50 zN!Kjo7ufFSA-Bj~%_1aAY=_H`U1rN)gUkxs@&=?=*^;**wZ=BO3k6EPJmlBe+7BVO z!QIU1a8_t@T_A?LYkeuHG6?eYV% z2W(~EkU3=AR`tKqM{Jowh72#g>)}1xhCKA%D#lSp>No?z#;mw`>okkPU3L{~&Y6c2o)Jd$xiaNIkHv z)Iouglt#!uvW+!C?uona7D%4iI@%%o!gkvUnOC-|9!S5j`94U!v*ijDC|MYS{0Cd& zDC9o58ybh?i|u0)vfpeiGm!aVyPAWvy7~X?FJFLE#Q#z|OHiOBYZda5Y;)_7i{dVJ z3zBHIfgQ-ku)XX-CYH?}Kst`?>*{(~V zK*_&y$QQBgRzR+pyT8?t7`FLZ$d<6hH9)46O)R9#*j}3rw&_90RI^2oK)QylcMMXsY)=zVprn2p@^x${vyiLjuE;^sz_zvs z*+#b1706h&i8V+!v8hc+HM4bXLxK8zI5^~6*lPA6*UH_&AtY^V`6rNVXInahOb1)? zC8RsqMy?^%#rAaz1xniPA>Yk*^9Z>f?kZj&>1Eq{gKQsL_9tZe+1xjz2iW2x8h@pP zZ7>Q7l)S}2evqv>9&$t6T_i#>%vP2H*%7v_RLG37Wu`-VjBO?pQsZnf*-)UQFBkF? zY|r_So8+#c2$CtbQv=y)w&F6#%&@KhgY+z$u7cDY+hh$CD2b?t{5)HCBjg-+_f3#2 zu+_Fgc9HF{9WqO71znI{W?Sxo)CyZlKNKh#709o$eGfryjl1?yNY>eI$057HRyhTk zO*TIR=`FULc}Q)uEi6ERl7wZ*@30N6LT;D4_jO42*jlzA>)9@MAhXX_?je1^wtWDp zL$<7AC{Qwc3i%_p*bB%VbJu?b$qC!b4P;N*8t));#&-4q>2o&o45));@xNrQ8cH3SFl9xn}Nah9PNTI~;{N24s8K_O>9?%a*eX={`2+A=S^8a0mrT29F^>!1jI$IpMDP z0+KoRnU9bfWt)A50wpnTkRM~~|A5>$ch6stOt3Yo=3m)K zw$n(+OtG05NKdn^$3bd_EiDlWluRZ=ewHmV6>@Xjb*Dix&-Rc3S;toU7cvWMM>&vQ zWGl#r)DqiDArvS{F_2$o8!Ls}3U}Y-kgT$GR6=%*?Y0^+>ugnZkltYP4UpPo%WZ-J zB?~Q(-(pK_huk)IL!FT9uzhqxc9*TC4>EgfR|AmtY~@3c+GpDtfdVC2||FR(e&9<8jxgYNS=0T#u|G%&23m_Zuza*|0GLdXj0_iBW*D^>&vo%#ffs*qo z$j7ji)M3i;&CY?qC^`EVleL$o^$p z+JH*4n z{5_-$+u$PZ^*8f7fo=xXMs)21X9}1L26hXd`t=mA( za(7<}NfTS`Kgc$-9aci7g{`0l(yeUEb&zUfOKF4xC8JG{Z)f{%fm{c7?d_0svfXw< zwu`N@2QuAkz7NtpY&im{UbclHC{U6x3i&>^p>fFdbN4<8$pBl+3}l7vat<4( z9%9>Gg48fu)+!VznO%qc2wUtHmHl6x+r* z6evl%g8Vew)D7fjxQo1lWR|Vx0kU&!4^NPpXRCXKv|~GZhtvXF;TIGrS^0tdBHN$H zwqLm=?#7}aS!Vl*h3pDjM*?J4*+LSe*VwB5Kx&TMfB=?&fMCIbe%xfb1dLfQ8Hv z+iNqVkJ)S+q)yn*JD@;GNjKzA**1G2cg9`%03_#Z(}R$`V2c`o%q3gz7^JV*9w#7m z%~n4R1xk)*A%DYG3Y)O}pd1D*7hV(nz=Pjf@*xK%)K*{wZJC z-)z~Rko{qs|AvgZ`~U2Zk7)muj`&|9QILvcdy9brB~9^=k7Bz>gj_Utr76(wBfok7 zmkQZfwv2Sh#Iem}LOPx;IvY|6Y<;;MUYG4?!-Wn%vM|m*%Y?5e~|gZ zrmG;G$~I90Da{s94+ToP8X=#?cHacKbna?eA<1AnY=>+nTYeX0ve=e;ApMsuxgS#5 zY@-4NO1_35pTpKZ3b|bFZpIBkY(CrG3}gz}a^@jj$mSLxRm7IC3GC1 ziy_s`mRkx1N*2l?-@}$z3AtYGhN>ayWBaIuY(HB|17rr+t}LX5t-J+NgKRr(P@p8M z6Y@iBbKQ^|<}S7mk`cCn0mzQBy$nKTjLnWfdYtWS3{n$pC6iE~WMdlglWgg8kelLe z%0V*C7PSP~8MdAk$jq`mu0eW^t!@)i^K8f4P@tr65Au#}bsusI-2FL%WRY$B1hPwP zKWC6xX6w9!^a@+JhSVxsb$|jT`}dGvW6OJj+&XuQFOY1oCA~v-lWq7DGFxn)-;myB zYmMmqmD*vuj)DRu|6(D(%eEU2xjpXwCPCua=2IZM&laa4bHFC)kUnI4&4knuTT?a^ zC^^rC{4rZ;0pw1&+bn|Qlr5tKvS)15Wso^%i>`q51zT?wq%PT>YM?+#eLdu_*iITD zcgV?!jTUS36D7h2JKd{vdL++8ggHcGH*zzYJ z`^>g91(_GNW%Gd0Sc6~EkpjD?Pe8nAKX=JK=R49w*}cRw(MQVe6u+Z z=^wWELrAIn|Ig{cV<=Gab_)3jw&n}SMgFhfH)28(#a4C;*=V+{JIKVaWj;bWmTl%4 zQgLiCZ&0A5?*sDjY|meiOW>|Sb^S^b*-j%No5WTe1DRyD^*BhUu<1ld{b8F-h5{uK zsgO@)>rR85=I%ZNk~Fs3zmQF5JIsMh23tWsq%+x;3n7)omSUhl$!IC$|FV6TLoS=U z_DV={*lw#Ko6A;N2bnxJ-vH@+wwxwN6|gO|K!K8kcE}g94Ru1Uh`aZ0NQ&87`XFoA zE(ah}!d5;6=~A}s5lEG>WsO6DlG#bfm$SvrK<*!R{d161u)R3QRicUg*Q;3 zWF(|9acj2kuARlGEZ#Fb&!5$OKybJ z3)^TD6e#&>f&435dpqRbxV!0uSDiTGdYat^YQY-J0OiDKJcf^;-n<|?FO*k;$EKuOFNmGa65IMYq>|avuAo54`H#ND42NSfKk z*C5-%_Ok(*R<_PDUA(iF&zakrTY$v9g^I%Fr^4zlKLXZ z&#;{s$jx$BR0hc$+uA?K&ayCt6;O;&ek~_BAILO|!9VS5Lfvq4J(vNJ*e<1b5mXZbqN=7pv|IGHC1-TdQ z+H)XzWxLIT>>FEUA!Od!d@-ay*m6oC^~ttS4h2dQDk1;HHdGC{Z|>e}A^BlzX@IPH z{QvB~w2+DTU#h$X(vfW2ZIFs$%j$#zC9~a-k7kSQgIo-E{R5E1vb_vKHjb@v1Tyh# zXJe2~U^A1DN@UxZh5{vNbC6GBn{tp#<}Pvxk`%U{70CW!dsu@^DqGzqq&3^oHl)(n z3iqHu$;v+D)7kzUK`w*4u@gu#*?!I-o5j{~37Nla;TqD}Y*hhLIc)p)P@p9D3G%sY zi!YGN<1XLhHM$zKssc~*!`8M81xl`0 zA>YeZu>rY0?sm5z>1WH{h3o*^yoZdi#UDa?kWG#uHN^II3I$4NV;e1n0wrJNkY8tOuY}wNcQ@6LY_e6>L3WF6uK_aKY&lJk-eGesklJNSXomtN zgPoAyV|(w0oae5&50ZVh%K^w9u$2u#=8$cB1ky)rnd6W;W}BUa0wpmskUwGTpM%^f zch3%zGq%Ph$ey#Eu0ZC3&8$QEl5KqxQdewgJ5Zoxau4#?Y>@|$yWy_;2$Ea2hZD#K zw%T*Z+_4>9Li(Pq;096;Y%2i@l%zaB{*i6$335-|eZN5R%+~P^*%!9kPsqHoRsBHv zjm<|4{z|>GbR64R6{OC?1JYjPY>w=w$?*P6|!9)LxK8z7!Tx&*mf@+{4Hcl*y8RXQ_3cf zkS=3;eTGyyThkj9C^`Rt{6DtRZ^%_}x2cAHC6#O$QIM@-n~s4@HCuE%q-)rE6CqX0 z_LK|-O6pS~U&nTm2Dy6fiZUT-U|ahO*+#b1T*z3qiF`;mv8f_RHM4aYC{S`&3i%ec zntzaMDz()1l#i-vY=RqZHe{aJ!Y-s=*sAs+^~$z?2n9-V zPayxsws;1)ckU7|A^Bh%zJ}~4+s7?rzSvstA^pvE^$4jSwtp{Bpk(I_^6L5jGw1Io z$x*_L`Bo5PmW51CxH5rK3b+vgCZ z^4Z!(p+L#?IOGf1DyAS;$ldM?Bt>l5^N=lOn_qy8VT)gebP1cRLaLPQZ5;}fG;Kk? zjO}6va^>8WdPx4UZ5=?if-U11GL>vIr;x5iol8M5_kYp;-LVACIvZe*MIf|O;8P@}&JlypTxzKQKV8gk9t)x<&4!giPd*;cmv zWXQC!E&qXZJ6m!Zq&nC}GoV1pR~F(DLNdYDI0D&8w$m}lOtG0sNKdn^PeW>kEo}}8luSCv&$2}>L2iz_ z?iEPp*&fy)>)2{HA+x}Cv<>M+wt_uKEwQcaLxGZ%Bgikajh#Slg}d)FNLJZ8E+MA+^u86AuMSvXUTwz&4ixxkK(^H6%xD1L=@GW_!tm%n6&#hV&`hSuUi`*h&hZ zK*>fCjCOCWc_-BcMQmuyiLkiBB-se;Tk+hYx+Z`kVUA$7}k+z16q3Y#Gx*j8I1 zcgNkI4oL3V#=9WyD1%cEvTlFv$DA^x{{0m#&1ms@1TbzP^KZ3>k zzgft>vklKf=7a5X0n(prt;>-5V!K|20ww=8Apgy_y9K!)?*8sVqF(-gy?GDWi2o&V zhmeV6lVeCnvAv!`Dw?h70t%FzUqL>Gt@IXhvD|ImK@!K7@d(*?w&`ccB(O!lK{}DG z_XAQ%Y)@ZMprl@n|H>z`okT(|g}b5{NdB;`#X&ZeEj1A`nr$K((rIie6;kPJU1?CD z0-9{c1Rhv!A>Yp^41Oc61L_($dz(;F#t&!TiFm~ z%h|R@AoGtca~#qYY%`OPs$`3qfdVCcbC9oMdv=hk=B{B0k{Y(t70A}I6|Y04j%|Gt z()Db52T~1elY3C0B;o+_jcnaVkh9#~pFq;YR(lTFX12pi$h5E(+(5dOZ8<=yjVgF=nP5Asg7hStsfE-O+eSSUC`q%BpJtnChTIHyk!_I7vh{R8c8=|#3o`R; zb-j>wY)Ac&T3{<2gaRci!;oKO`!fc)CGN&1AX#SnnS$&JTgNP9R@uTlq}SN079q9H zw!aJoN^;jAzrnV+0l7`?61O4QVjJFt>^9qnhs+LJ>mj6f*{+Tuwa51F3<{L&TtMEl z{k?|VK6i7skQ}hZ-9z?}ZQv0yM{KXpkUnO!Z;(1+JO6+JB_-dGKV{oglfQCj+@(iB za?Um#1KA6^)DoWEj1UicWmSN zkhy15MUZ}A>okyhWVhlWE@hFY;TiL zprmOA@=CH|GJn|g6{J(yCT<|5*&^fb?3i4@e_fL>Z=dR`zk_@)P zcgSY4<$pmYi*5M_(tp{KBd30)ve`zXp+L!3EaY?8+7lp`%iT>9BzbI=e;}LBwx=Oe zz?PE%=|VP_1*syogd8YPGMESXVz&1J$QkaMiyu&pdYfs&LJ$hWeMtwFAhyYCH1 z+Sxj`A=|-ry9=34wyJ$dcd_|HNOiO2o`3s~5*>>KbKuOjoWt-2Kol04VOx8J>@8dB2V?@<#22LR*p!<2mAYr^ zii83schQi4V5^CP+#`1f36MOo@&Ua92?X$tT-hF=W5kvP&WJ&F0D>{lgYt2`TmV|IgWAH54d$tA%_7TXO^CBLA0M zSV*GS%32^B&9>DBnHaXrPDsbH&2&R5jxDAS3Y7E>Kt7)Bc@T05+%=3qlE`*C2H7OG z;z`IPv#n1HzdVuEfKT7vWD$43Nj^Z<*|@1W!sL2R2f@V5)>$zO@VwlTdaoM zKkoX|A*o<{$%JerTVpn4s@TqQAzjU83LsU(wowEHO43RoU&}UC2Dv)!A}b)NXX~kg zYy;av4P+YG>gpkF*^U|^)x=iV3TDLDI_h(*xNywvK+tw6g_) zbO&42Fr+%!_D7*WN$v#XyVw?|AlJ=Z;w&URY{T=A?PdE|fJ`4->oTPK*{)V0HNf_7 z0}7PvY(ZYw{_a9&9c?2xnBiJjw2yI$5s>rxq0qZ;~;Tt zsfmzXU>i?{%p#jgh4d0zXBwoI+3qr+KuPsq$gi*+uShvvsKhVZil4ek) z+gmpjC~4}0{2|-L0OXFiD;Uf!sZJHRq5# zupM4P_K_|B1~N}<%K_5QY{?IhdSM%Vf&wL9FOYv_Yk!B_8+SLKki4^1{y_GFZ7*W} zSLTx~CmPaUY%Ug3-)sp9P@rTm3GzQ|?W(`Ti|5BIfkd0(3%YsZ4+jcgj zquDa^AQi(lTL1-0Vu~Rj%hq24xj62g%OHtoYpj550^4a7WD?m-Eu@p!*6Sga%$8=M zK*?k?vW-nZ zE{nVGDMl*TkK*@%3zw*^==~0lY;chAhl3KQ?c*xeV^&~>3p6xLi(hY2NsgP=9 zJ5GZFC54%gw`{9_A=kv+pIk_q*~arB+rsu!2$@#4P6O#SwonSGcDCw&P@rVL67n5v zc{Pyh0!{WG~r@ zyCHMMw%!ZrYc@RqsT;P*K`2lXF#`Epw(c>=1@7)AAh~0!ordf^+u>FFf2A^BlzxreO!`2Xy`e1uHI|5D{Ikd9>AeuGpLTh=EOD4G3+d^B5Z z#Nw}940rueki@dR#6UKVtuY=l@oZ;_kWOGTDUeEJ+en21C28r9Phy+Ogj_Ork=c-> zu=V6Z_7B@bK4ena>WUz(*^UgP(%1^ipg_sWKgg%E{i%Xn26tmMkYuv`)Im0jt)me# zf7wD4q_f$oS|OFgw%-l~N^-j(pUbw`1Gzlz68j;^XB!sC7O;H`L8g$cbrjM?Y**uu zDrWmP1qDiWW*~3a{?0?LguA%~NJ`n_mLXflHn0kra<zjPs%NXeg90VT50GzQD|&`pBX_H> zkXW|V56CvLjekL=nN6vsU+EUM&PYhLvfV{Pfs*Pt$hWZ_BtWj6yS!vbI@p%}K(>=D zDGf4RY$F+v?q>VUf>aM%TMiT`xz2-pFIz<+rBuk%lnXCWlKJS)Ee992^1*#I)nT=Tl*#CHn_XFhGdhiGC+2V zZSNj3+iW>ckltZ)FOb@0OL&I@C4--k-(!3KhMecFIb!)&vd?xI1=$0(vRKF*vTesh z`iLzv2~x*wvnf!ZBt}F2gsndva;MxqXF_tu)|d_1bGFl5$Xu|Q0!Ux7trtP+iY=`K z3Y1KiLH?R8vI259+;vw$a?AEm1KGeWeKK*_=s zko{(BS%%CH+tn(h)#v|bfB6QaBL0`!*@6NkS-X&rWSjGli{dW!5Rz!Nfn&(Vu)Ul@ zCYH@!Kst`?>5q_0AydfqSqbSPw)PrG6|-H}L4lI0M#z`2?KVNKl)Ic3 zNDSLT8)VDa;yNKy&L-WEu3&rZg;XV5%K#K8IUj<26HRMLP zyU2iKl&vBgvSVypxsVxW%PN5M1lw#8q$b&7N})i>KpEtx*q$pOH_csBH6$}^r?rs% z%U0R|nOU}tKaied(-u7XNQmbq!6HuUJd<{z71f%Fzz-a4eV*%mjU zKuN+jl8&W@P zReeyPWM=^K>g&JH0^|tfBL0)ik3bU17B>#rD7K+V$V9WfPD46|&CWt9mhF5V3Y3&B zLOzb|-!kOlxyx9CB!TVk24oZ2qP8HD#MZw9>14LYJxHan{W*XFCC5jQPh~4PgG3Gzj3)v1sx=5AL*Qo@#-3E5J%g>1+ew)i|q zm$69!q{`Xeil9JAODW_l*e=Q-SIJ#@B_vgBTh)-QX3MODOby#?1Eg!&qMIO9$2MT0 zK*>`pdGXeP)w)-i_ zwQ^Vg7m_x%!#T*dvlTkXbg-=~LAsMIc@4^4)Bm+mP$w?q(N~UbdQj z$o8@A9YUs`E$;-<18nXLQiE&>mr$T&_!{yTg3}xM%lLC zAU(#G^$DqQwz+R8P!bcd`71xcHW&rDN$#FwAemxoj)&|t+i4c8njyQ+cH07(4Yt~LNN=+FPDuS@%kO~#C5wHK-(pJ~gxofFBLc|| z+s6oGciGy;A+yJJH3@0YRyhNyeYTxhC{U8U0Qm#9`9;Vbau>S-$r0Po8f1^zUN#_e z!e+N1ead#W1F18%G7kkxHV+_w&X#cuxeM-QP9eEui@Jd96?ovM>xo4aFg6sp^&ktlC*}5bD{YpQvg=k1Uv(?2xfs*|M z$iJ`^Bt!0%yQNe}-q@1TA^XlYnhBW?w$E%xf3mgbLF$X`x&R84R24)1n{BrgazEVV zltZGv|2z8^Dj^&3pCqmZGLdXj2k9ub*9J&Mv$Zrqfs%6z`53nHHps4ipkkM>Y z4$|ptY8g@)Y(1+`pyX~H@|kS){~(ve-N7~_*=&V-kj-IR-iJ&sTk;X4^Vr5tAeGPd zbp{1WIxZn!z;<&DxkB!$10+Rkd-sqnX3KqoObMHNfpjTb{5zx!+wdn8D0%yad>LD7 z#MZA|Id>OPkW{c$#6q@`Z7Uu!Rcu*FkgjH%O@UMmTa1PRB?B3duVs7Af?OSUO}UWN zvz_Kcwt=m*2r`Xq8zqqb!=}q1)x7Tsv+OZ)>{iX%iVoFBrR+We<0h+cGwJ= zHnyTxNVl`Cv_qs|pg>8&9OMVsMiwA9$ld!QBtvX%E07ho%QeUhvsG?FdW3Cz3sR$O*}G7nWX?l= zj4k#Ma^u_$9z!z0_Hqi@Nw($-$V{=FT|s)9&D=t2hHdi>3Y4TjLjEt?%roR>xr=;* ze*Y!;-{V=|2W02j9=;&6z}Bd?f2AGUQ6!`m*@|PJK*?$xc#N{7rETgZg;I$LcHq&C?0^PoUUej(&H*_MhS_m8_o1IZTKXgOrJ**+>Ev%}V2 z1L$H{S-y0b5)rWDnVfx*>DK_Sy^SV>UYgsS~#I zAt+E%HUjxmwtr)gJL4{65|VSaztfPtV2hfC%q3g@JfyGK9v!5v+5Rj;fs*4@$ltJ) zY(VaoyS0Ci1h%vt$lkF{?m_0BO&vh`fvx)pQjctRCs3fI?i}(@YzLQ+d*-g-29g)H z5ND`zX+1^s1KuL>+d=%S72IQi-E6;``hHWbsvaxKL1(1njn=OKL zJX>@rq!QQ$%Ai2WQw8J`*_x^$m&DykEhNcor45ixVO#$LnN&7yA)UrH)e0%i7SRC( zN_x5=pU!sQ1GxiZ$dWIG&$Y!+MLFl4gXRz@M6!`kK>mej%Y?sH7 zEoZAZgG>e6_64LX*|M%7RmC=U3k6DI?jc{zHuwm+8t$H-A*p3+euHcs+vx{n>eYTA z*bKRT?nYW58DRTpgX|z%TPI|O*si)EEo_y2kQ!#&8Gr&M*#h|yw)qjrjdB+|4#^nX z&?IEX*&Owz5SiP_nrU`DwO{HOS3yH?slBU$&?%$j-9$?LcOZ z?Qsv%^K6X=kXm3nK7s-z#ix*WY-{I`TjVbF3X&zZ$s5Qnv;73ftgv-IKzfxeJV9!W zt?m^Hl3Lt&R_F4p~Bes@OC{S`<2Ki&Q@=C~^aQCko zl2f+KI>?@}{cV8EIa_oSq%YX|Eu=2lo?4+m$)66$U$LEZLGGHnl3qw|*w*_Yd&`zK z1ew4#H4N!HHZ=yRd$yhlC{S`Y1^EZI`oEBSUz<>%q+=WM?`$`_ko(}SdLNQcw!K5hezE1AK<1mxok9ACE&dWxO8wvOr~cLO zH54d$yM=rNTkAdKBL9Pt zs$k1rfC43Ri;%Bmi(P?S6?cPckW{n1Y(Tb#t$7PFwQOfQkgj7h9#Zven+H&!B>foj z4Qw-~kZa^F@&fw3B-DSm>%J?74ofYsUMJQ z<8I;$l6JPAAINsFbw%#~%5<`YXh?Ul)y6@pn{7V<3Y6q0L%xS?DHU?P+$E+%(#JNM z3E6(Ok8H>cu(jtwdXVj^08&G2RmD)CWTzDJ!j@AGxnb_+Dra`DBDmSWX9ND z8z4Q-W}6^2!FFz;KuK8}M` zKVwj!elJ`G`FXaIX~->bxAqqj$Cfq^*+sTV2bm={wG8QHw(eC(t+3s#LxK8zqZIP1 zYzNzrTjQ=^50Z7Z<$cI*uq7QqW|M8~1k(T5KF=Vv#ny2N1xl{3A-~O59U!;E-R?ak zyKK2nklkZjc!7*(i+_joKAU_(>VWO-8w!-PL>&CeAF^FULGFmV@>ocY*|y>#d%~8P z1esH|*%U~hu|;c0owE&OK!K8{EXZH5HRVF?lDm_9NUqpQiy(W=wq6368#Y}A>07p` z3P=UEh-xTM(o+lhJGT3J$lY^S{|Ax>w!>z~KC%_ILgtBWr5)1GY{^}adSM&yfdVC8 zeUN`;>l}pK8+SJX$va!kC}cm__QoOe$(A<-=`S`n1F3JeggGcsGQ0r!AGY^J$f*cC zVoK`@BoY5fU9LeklC5GBGEr>XTab=s%i4uh4BMQC0wpnrkdI{>Jce8xch9Gg#IrSD zKsJHx^a?VGY~~iyNo*T;kVETf){>1DR5`t2#&nQ2HG z*rH}3+sM{851Bt~j}FpJY>ms1YGylLg#snT8<4kbYyTkE!d>bPB&}?ddys8o``L#~ zJ6rb=q&wKc38XsN>dv7+$^IqeyVweDAlJ>^Qh=m~E$IQWy=s*jh9cC^^r7 z{5V^AHsmI_`Y&~62pyaLx^2==X{g7MX?qCp-Rkp%m$gZ(1 zk3wdhEqMac8*F1!klJMXnt=i(9dnTX$9A&-xh?Lhmmt|@+gpL`4qNUzWOmuyCZzY+ z;)I{uZuX46rSx?!7+fdVBF@sPh|>rI4Q z;O;&dk~_ABG|1kw9i~I(fvqSD(vNH_Igol{OUZ`k{`CVPROdr|IYr)ZpcLZ zCso-8=}5Ni0Z2u$WeXH2nHzz8G+XRA7)n^}ZZ zBHQLN6evkwgM1R(%m(C=xr^L_et(jeum5%+o67dE2bnas#sf%ewxc6RrLz^ELV=Ri zbI50~rCvcUle>u|0%WthocOeNcE5u~fwY$>Fw+0M(LKuK98e&7^K(?MOstGa;Z2cC} zjckvtkov>+rvnO<9Cty!iLIm;a?RYW^+RIW(uN@0!ZtY!nN~J62I)4o?g>b>v)xTW zfs(qvkndnSn1fs=cLffTF1F<*$ab?OtwN@UZEPLVy=95o@+dvc)D0zy3{0v)DJmmgzcajLnEL&*`Warq{(;zd? zrZXVDz&4cyDaRI(3k6Dg@*%&-c3%j&CGP4=AX#QRG>~0kE3AObD%(mGq}SMzYazAH zHeL?}O1>H)zroho47pA2ZdxGu$5zu0*)6ucPRMMt<@G>%ht2gtYL_ix5DJtG3*`6M z-bWzkxoaJVWS{MF60!$u6*G`IWZRyF^buRu0;G=F<`$tqNz4l5PuK?6Aa}~$^9Cem zY|UGcJ!d=Jfy@P)@sPe`+c<#K62C{*z;SMr) zY)21}zGo|XhSUSw>MIne-!Gd({*i6s3vy4~eg8o6%+?io_AC3sb{h?uSGL+XNWZc9 z1W3KJ8mry;M|*8V~+ox9X|NHW+a z9b_}vewHAU#n!zF>1?*J4yhcrx_?lhWPcm-xoib{kjvw4X&;h&wxlD-7O;(;K&FuG z^9<5OZ0(njDrURBh5{v30rDkmyZ4YQM6hYF&wq63+X1264$XK?i3P`uGscJ~Ivh~zLfs(s=$hWc8|AAaPcL&Xo zbg&h+Lbj7_xg9cHY{^}a?q(b7fm9FMS05B8=@^83FWZeku8+IwQAqmP_QoMQz?M4& znL#!;1L+~Q_&G=k+wcMuD0y3i{4iVV3gkw(yI6x{l&xYDvSVypTaX!N%i4wX1lz2K z)FfNXArvSXIEMTb+w&>prnzgnfMka4^a`?n*-CFAGt0Jd2kALB{RpXfw&`amP!jP5 z`31J#56C(0?!O>eWNT0tzp_hghmnw3W-E$;^a|Tb9Hds+QWBv+$#^p4*Vw*OA-B$5 zXF4PsY`2+^-DIoDfy_TPp9kqJw!A_}ZL=*FLxGY61Nj}ck#flGa`#>d$sSu<4P-sr zWgTSp*(w_$eZaQe1gS%|>=r0cGS>$ABevL1$Q^Sx*bT`E+e_PI#_OlP!C$_F5$UL)!6G*?X)t*D@m2LkL3Y6sEK>m$wDM0R> zyTk`bKG;T|Ap6Po@dBAIw)S^Of3sbELh6UD>IVvx>_lAt%B$%AUNh%JLoVV!$$Ts% zk!*1Zkd0y+N`g!@+iMD>W7w>QR4m(h1{5eM%Z7X$+rM1M#dDWY07(Md-y+B+vPG3b zCW)=T4ARMLj}?$gVf#}J1xk))<#1Op3`s8A@+f5U*pen7lg~Cb1?d8|&lyM+vUSWsfs*S5 z$QQ9yFF~%DyWJH?O4xGOAzR9}un8H%7QYSYGB(+TR5{z5hXN%nhmfyeyEukiC3od# zkW{g4T|l;)E%O>OHEgrDkgjEmzK2vD+rT3fD0zB@d_7y!8{`_eJNba5k*)L_vVYjt z)zz;|6Pu2LbTivj45TbuL_8EI=}Clq3)_7%AQAzjE8$jYLDT!}bvi*NaENsKOq~>_V*hy32f03H^0(}Z2eJ?N@9D8 zfdVCe;vt{Rc9ICW6z)n=AW3CgPlIe4TUrKWG}}}bq|@0{E~GNpdh($_$z37jGui4( zAeY76fq^8Ot*`>JIc&>SkjZ6Bu7z|S+gLrM^4Y!`p+HGTGvo`{ZdxE$$X#_iBt>j{ zoscbN%k6Afyc2ut0&5w-Lyfv9*pvuAIAzNk}T#DrO*C$+k5MnJTud z1xQ!3%`QT!hAn0V3X}}2LB5vlc>{8F+%;`MQqOj}1K9?)QV*F%wv7Ww|6$X|kZNL^ zK7|4$5f_keX6wCzoaOHR29g%GhC9f%vK>A^rj4!W8Pe@+E3c61U`zRc0wv>Lknd#s z{()Q9zmjgY+i1x4u+_vtrkBkpK)R1DFBww(Y>TN-pd=w3@&jxmnUEXg?mZil zA-1+W$O_wK0c3{RDvKdK!nR!ssZqAMdm+k5rQhRJw0Sc7t+(X{8msCpc!({+!eG!^1`;<4%t_>q%O$3v5oaW`kn2w4^kg&9fMGy z5$O5Dy+1?hRKuOCAG5~kd0-_+=WaW+pLFlJX`c3q!QQ$j-f!w(<$T=*_tjOm&D!46(q@Q zrMHkxVOzh0Oe&jxgmfC))H9?sTf`d_DCzltd^+3x7vwUyt5Cn;d4-TJW^=`m zDq%}7P@rVE9P*`X@0E}<+_ly~zjx99ZWEVvkS%AcXoO4!+jbMAE7`JIAXUXS*9HYj zVmcvT%{JH#xf<@Cdm*W1YaW1X9oy*;Wa`<>2&5a>HpU>;$d*0{1xlu;A^(Rhau#w; z-1W{w(#-bYAZytgmLb!^cC-rVR<@!INVT!8{(}M~DLasFXPelATnBgG`;c_9bsa&r zi|zIVGTm&o=aBAU^Oun7Wy`;T0ws$9@_lTH50LBUZsZA)0k)49$PTi#y+dY*?dlWK z!dCeMsbRLAi2Gj!O0uINKf*R23%OD5ViO=4V;f3>>^R#?3S=hOtcLU?+gS#rrr64| zp+L!IF65`#G72C!!`(~~B!AhWN+CPT)>j6ZIkv|NNYAr1Rzqrm?YI^SloU5W-m$Iy zf!rc@sTPtYw#in=F0=i#LuQ4oy9?5*Y@r8IYixD>P@rUg5c2D61;dcr;BILYl1;Xx z3CR9q8=Zp87Tf0xq_^4H=ODGicD(=vN~)G1zst6}0=Yfza@Ha7Yzv!^-Div2hRgw* z>_YmG?bSo-h^^%i3Y455L;jep{0wp@-2Je{Ugk&K7+S=?k|0M@U_= zJv~E#`u(vl$X~IYd_eA+yOM86ZrIk{ZuTUr!k0^3v!r0>{NJf!Z~dJ>^P$z3w! zAK2>CAos}KK{_N)Y=v2neP&zEfy@hAaz3PA*~SVX^~Uy93JpRi;9^oNZe}s)8*$6AF~fWkbG_EjABwRoo30KvK>2 zQUuu=w&qgE)UutGLAs93R6?qrZL=B*l%&@|zJYC~0dkGpMK(e5hpo>-wu$Yb6*A3i zjUAA-Y)4&?YGEtxg#sn3{g7{EOC5q-8+Q}KkhHV?j6$}9t!n}@oorzW(p_w|e<9V) zwm%02O7b1#d)StiAlJ)X;wmJ4Y@_Rt?PvSggvePlIs@8Z?jdmLvDw=-A+h$*>ZazyT`WB2N};6KM3i4HW5f2u)U2yfs&SS z$RDy@OhWF6yYd-Gj@hxd53{w%J8UpRq-+K_BqGR_Y;p&9;63nHx5J4C!08sZ&S=wulQTP||Y+`8&4z8_3;rSAPe|1KZ&PWFOfI zpCR+aw(<(;XSU=INWHL)e?ftguOG<2vUNs2|H{2_cM}cCJ6la0WIx#U5+L)*mX{3a zFE*D7sc*J~bSO|VoC*0Kw)bqvsdzjSkk<_bU>7{Z|0lNVbY%$V9PimqI$4Evp<- zF>G^{P@p8H2J*3NgLROL>go9i*ci&5p zvax66ET+n@NGBfh|fywvnwb12TWu9F|i>LJz1R`&-Al{C& zEww|^!Evp66Cv3BAkUC|H>4X9$1Kp56V|(s}+&Onm1CU&>oen|vlC5+EGFNOH zW01aP)02?8VVjUKJk!|H4q@LJP zcA!AX_#Wh+*}nH7_rhK05hSl{w0>~$^%@jc{nY+kRNK)AP${?G{ z_D}(tG`7ZSNNcvET1cg{6*oYElGQ(u&tOZnkjvz5q7{-Xwx4#$X0vs5K_-VS^gue7 zt+pRhd2IWGP@p7#81ngSOQVo0;4X0jl0vr8DaaPFeat|nn5}&d(j{zH3y>;ht6G8r zB|9sSH*7iUkSpVEeiM>%wzzG`Rk4f|!+eJ6z#<(l*gJhg-YXGtnY?%U?Nw(P$NKdgvk3(vjZD0}#lsrvCeuk}S z7IJ^NJDG=MmaTLVvU62lw*t7f&wKyJCI*wyWfM{5_k0nkSwzu z9zk}6t?(2wt86RhkX~a;zJk;`+xQI>DESJI-(c%}fZQf`H&2lKW2=xVJJ7l)m z^1dLw!{&YWuvMf&=8$b$L;8p< zD-%-3Y;)OApd=;_@+WMA1&}-C?zsq(Gq&bZ$ey#EmOoskPU1N9gw+WJL-b;JzG&Pq#oE-`=LNd$`Irq*(Qb| z_r%@zC?wBpT@#RfVY{7z%qv^%Ur4{P`8i0vv*kM|P_noL`46_lRmgpEH?j`N7u&}s zWWU+kwjuMwcC`y>mH6M;U%3ydi2tN^4xvCv_6g)8+2+q67sXxdB_z>oL)VavVSBlS zOe~wdhjbj<*(0Rl*~(s^K*{DC>&}OCHd`o!R1RBR2^1*V zH;~U|E2w~69(PMskmR!^)k3y_ZL}UTg>0XVkS=0tZ-!Jc+jR>RD5+|PdnKdbqeVg zw%=bK`;}^C>$!pgC3iQFZ)2;!gIqgz2M>^RuoXT-wv%o76*66H$sdsJW*hs0R1e$N z4-_cri2VAM?`6A*hFl+a)p3yYv+X57c7QE688U-xE)~*4Z1L%k61L$?C{Xg24f$cV z);!3KaCcDv$tYVzF=WTswn`y0&X!dU=?S*kN=Qwz#neE7l7TwNPq95WKyI45rY1;c z*iJ2E|FV^~L1vb1qXW`&Y`Pm#^K8?-P@p7Y0P+iLy+e?5+}#gDvdGpj2H7RH!wJYN zvlUH4dWCJ}FQiu4Qs$vR$+(038r%00v{aenN7>_VNwcQ?}-a?_ZfSwzDWmpR<`*NL{dP#zTRU^d!h%vdyGG?uxre z4aqfIUj}4v*dDSVbIaD43+cdiln<#pw&EfvP_kMA`Fpn1GRQq}H&Fq}Bim0EWS`i& zY9aH?7V07W!dCkSQm<_L%}}5uzZLRtY)kEsd*?2(3z84E(H_WtvVHVH=8LU;5Ypdl zR|2UYwyIGmP_i=)d6o3vbH1D@$VL1onV*3qk}YlyvQcb93y_IsdtHQd44YknR4m*1 z8Wbog+k|`^+rKTy#dDXj3rPaoUk}+twx~nMB(e1$Lpqu5@f1=iY=17GK*{kHn($1f$gCJvW~5x8ZwJ)N41b%Vk>HZ)H2)Z zA1F|gVj;i6Hqi>XRqnpqAz5SV>VoV#+ieeIHrQ(WA-&1w2O;&3Eq@palq`-yev2(} z0&?5jjZ8tZ!}c))*T`fS`vsErZYM*Uq1qzg8uS5QTZGIDShup<(LvqA6 zvlFbXqpR;9LL+*mRnOjIM*`n?td&SoG2$^fP z$7e|2urXz;J0}7NBe?va7t*MA#xjXJsqae9wn~Z_%1KUp=WFFbN6CwS?_IoYb zU#Vxdx-=+IvY!t57q)^d$h~s6lmp2dTT(t`-`PeBA@jlZSq$k|bbvY{Y+(xMs*ivi)B6`ByrM?X?Y3(QGZ9P@v?z8}c!1 z<$aKgM> zGcl(hsQ)wv-_#P%=IY`A)X)QOI?1*Es=6H{0zLWP8|Z z{z9gg&Cfx)k1fwZs-JCf2?~@XtU`W(ZDbvCgWSDuLNdhGwhdWfyWEA$Fk9t5q(|7c z4?B+BJ!GcX&K@B>&1PO8HN&>~1_ese zKOz5@ZRQ(tv)n~SMEy$U*!rR%JJ0qI1DOT3#&}3OwxdKyEwUA-K!K9gG{`TprDi~G znY)QBNLJW>av;0P)|C&LHMUR)>2Z?Hedil3xM&O}3>f$o=Cku@;gow$XaX zZnJ$fLS~1py&2NGY*#Ii+GDF~hXN%#osjozIX#fu=Wf0ak^{E5LC7Am4GCn9*j`5< zeavRZA$7ubJ_!X%%4Q&c%Jy#-a%bFSEI@M3_IDAo7i>{0khx^*UxV}&+v5hLuG#)< zL4lIv9mwCXm3YYAa<_H>NnlGmhU^{N36ozA4q+$bwo!0Do}DA4f#*D>Nv=KakrZQ z$v0bWGGu?)7E&RjQvN&p@t;I8Ar;B?mJJ0;TJj(t#dc8uxoGaniy?_&+bV@@ zEL&zdWa8LnDpFg z)7YkZA*IZyPd2Yj_d-Y;!MApd{uU^3`mEpOCBJ?)e*%TDInh zm|xjCw$muc)U%mbNH?%;#6zl)Ej>yj)AY_Kvt_0G;Ryhi(VYZ!dC{U6;1^E%S z`5DNKau+)X$r#(v0%XV8UKSxU!Dd$=J;`>q2B|5wvP~#ZvbhEMX|{}A$jxv!<01LW z7Ig^OS+>4o$jq@lod<3v9<%P@tsv7V?g5?GAE_+@(H3vcxv|4B2J2pI6AN zuyubxdX+7FL28YyPR0HzP`?+~h5R~OK@8+JxLb;YWRoo^5wic-Mw214#rByB>20?5 zbV%*6U1vgplByiY@3QUYL2i$`oI*%E+d?s9_u1kMWDeM*9MXquua%HGVr!{^0ww2l zkUwTCZ-m?lcmJ9oIc3Xif$SOE-!{mcvqg78`hu;$8&a2SPrXo}q8Aa~7O z$p|DjZ0lo?y=6dqon4UR zvEBASHlMAgA2J1Oeh|`yYq4g#slB6Ob=v8<~P!33u-^kd(5u%|X_%T`oYT zjIDAB(&cR1E0C&S%U*{9C3BmQuVjnehFleQgS(Jav%Pr8*041nLZ+7O>=@E@Y~~D7 z^=z9LP@p9J8uATnGq;dyuq%GUg2c%lqioc;i$*PL~ zm2YKBje=YocM~y?w6p!hLAHafD-klCY`@=5`IYWst4)JcH`{(X6e!8hf_x9#QV!&L zxl7E4q>pX15VHMjAH|RvU~4y!9%Q>Jhtv>TRTUH{*{OlNu;tW4ZkW6IMo323;+i2l z$~M#jnK8E4Hb{@N*-l7Ju$^~9fs(R5$WOBU8-UytcNqf7G~3@1$j-1ujYH-yTmK}a zXW1U7AvMSLXBG;S9M403o~>jNatqw8Ekoki($*lm$TqnFnI$&01?gqB?j1<2u-)xJ zfs(od$gi>;96@f4yMj|l*4dWNA-lnrbOo7Bwy~T4k#v_mvF>0LhW~nThvE*!9g6FQ z2iUk3Hm=1TiaQ+ESs$<5S;>SDI`1&^4x~5PJ_Dq-*g7AeK*{wJ@{-R?Ui zyKK2%klkZj_<@XPi;qnBmELEQXhEEU*|s!fPuMau zA#=($n+@qRw&*-aowNNZfC42?MUcN>Yc7S{C3h!fkX*5qRzmigZLJzIH*C5N(zk4r z4Uh_K5zSDbq}M|Jj_tk;a`)WTcS7>OcGwNsN4CN~$ULzv4?y~vEqMr1FKlDOP@v>% z6!NcZUE`2@l6SV68OVOH?ae{vlP%9d`it%N{i?rG-)sphP@rUZ4e~#1?;DU) zX?P|eZQGDU{4aI63)x7vihamLv27nhI+`u(1X3|baX< zgrtEjY8kSPZ2hZ{X<~a^hjcSr;})b^*p7FgKuNKOyk%QGfLtqgsmGACu}z#pww>+g z95NkjJy($KWD7Tt>SC+Ag90V{50LL>D|m)n4|hwiko2-8eL%L4ZS)H={cN8rS(9@u}$_tdXY{2fz%RP?+_Fyxf_Q3Keqb6kXz>NU>y4Wm%;zn zRfSWKU1j?>1DQ3pg$lkLZ=0oOzt*8jnk8H~&ka}WEDT4wfV-=8pX8W#!+zWSI zwUE5B-PS|)jjg5$GVg4^|F8U&{$R^%gVZP6Vh0o`N$7_B7u!fL)p__86q1*s>>~K*`)Rs`Yz;?*rq+?lDUgKgd~Nn{}{5VY!9c9Nn>lgfV5^ix`I?X zTk$OvC|S9KdoJnvb|P9x{A%#K&qPUybcPKlr=)WhHbMMa<$xLv_ewHHq#E-dbX%8 z$TYAG^gz0i?XeG1O>9knpg_s-5agTLN=6{p!rkg$NGx001Y}#;CZ-_M#-?T=-OkoC z52+5eI|l_y>i$8#lkH#ya$VdNtV7bx_HPriJ#0xkkm+Uny9en$w$FV?^|N&zL4o@H zsy@gMuvMQ!?hkjnmyisy<=#Mch;1Q2M%dyXAU(|X`wHf-)Ck+#3lyl|j|M`1lhWWI!^@ zR+I=F*}4-U`@(jc z44GH9+B8VNvH5gJy|d+KL4lIR9LRsLCFVo!le>{ZNWR!UiXr>W)@~s4!**2;X_fx} z*RHf>X%lakq31Nj_WB z6=Vz8MsFZf$o3f^UBuS$0I6cO>nA8sQuPY?61LrU$dz)J^96}vTlj%&8CzVW{*@_b zlW0g+u)W4Ys*zLD*u4027}l~h8~%(hky*%r36I>=bI$p%QbvZ-cBwXyYDC{S|O2Kjcj z`cBAoaCguR{ob;L*MEJG?PB{k0GV#KBo|R4?1tC=@8^9EW@#+s!28`nju~ zfn2bE%Ge}La#au#xl0VmwpJaQ!h1?W(&G(Q@vzu->r zW7D6InrEB(h5{uK5$V733v7K+kaOJK$3U{k)({WbCAPyv$oyj~N`dq;+j1JDR@hQ9 zpg_r37UWmizH=bA#$8uFB>%Whxf>KnPS{>X zAbZNzG6tD5wzCOHpR<{1NL{dP%tC>Z^aaRYvP~~S?uxs}Wk{~s`d1-)!}hQanOnBT zEl3Bpqa8@yu@!qLP_l9W`Fpn1W5_*lH+~AqBiqk8WS`i&uORcx7H%N@!d80+saLlB z2PjaI{|xyzwxw6dy>plN0m%p3=oe%^**<@ z@&DhfoMgyF{4beLg(Q+KE*-K_Y(trliDrAvhI9;@&4W}d+j#*LC@Cw3d>q?mDdgh0 z%P5B=fo-M|vWaX_HIPYS8>oYHGTUPVq*BJyRkgMdbJU~*#wsjBLYPQTL$kedSzCgN`E&3f&b!>k=p+L#gH{|Qt znjArNkZESqNsw+~n@oX}WsA^Iprkhg@~v$5S&(bvu09u% zcDBQO$ab(57D1+yZMg)}U2Mr^km_a|tAGL}UsaIrVe6`eTrYPw^^o+j)igo2pKY%N zG6QURZIJ%M<~krX$d=Fz1xki{AwR_S-VZt9u5A#KVYW+w>x!W)AY>Y=aAso8azw5t2!^mSxCJv7N3$W}3}xKzfF4eG5{vZ0WmDpk&HJ zevU2j5OVX}^&La9!1izoS;y9J0hvX%qbo=+u@&7y>L1(69TX@@d4&8j+xRo&R=E3q zg=Cej`vbCTY`0&KS!b(NS-;X7Y(5fFn{4?pP@rTn4)R-UiHVTg=58bzk{z~>RLJhK zwWmX7kL@ZG(w?m{2U7cNJ9$u`B)bsu2W<1jkUQiq)*9e1=%aM{vOC&vpx1f`i8CX52S9{j)$N? zN%08e1Ka9f$lY<5IswT&+r$)PAJ~3oAoIx9GY{z}w%{Q3%vSdg3Y6@xK>mfTU>$O= z+%0WF^2V051KD@B(LKm~uzl`B`jf5W2vT2c*C$Y*r0N{<-)y^=ko(~-=LQm$`TyC! z5Fi`zza;JfGLdZZ1nDTY*B3}dv$eiMfs*r2$j7ji|3EI5yUmE~Ur8KWW;A5u*=AxP zlfV|80O>@wfh0&Ju|1_gfs!T-`DC_}49KN$SCS1$D%)BvWYgHv3LvA|CW|1Q&ZbHs zmBH3q1_es)Dj=W9R$mRdEbb0!q2I5}v*lv0uSIAxU03=0hdxMZIX3HIhObMGCg>)%f{5Yfx+wde5D0!QPd>LEY9OTNm zyI6pvf~{f+vXyLG%aEyJ%UXkUHQVe4q-xk=wxK}DpIyk;vORmq)p6H+2uVHL=`my* z*h7NKIH$fjTAy|kh}L{NQT(j4P=GwvK%tQY?W1z9%0+Afz&8lc0Cj*nQMgnU$)p5 z$c=F~*b2!w+eya)wKR{lYLi7j;%a{ss+Ux#Fw?Pn9RD{S36kXdC5 zdyrmZt380!I@|sc6e!6*h5QEF(mCWdxl6o)WQ%R|2D0019|1BuY#k4f-etRbg47;c z)hiSz*?EV&XUqA5+&*{nKad=-#YN`+${w-}MMLI@?KKwC$80tMQYUQZNl>7qEEV#n zY?~T#XWV6ELUPVFlMUGmwx~SFT(S)mK>CX9u?SMvY)z$5pyap=@;7WHm5{sTZnYYc zz?N1A**mt02FTpAsb)w&u=Q9-J+j@kL4lIGPRKv89dtwPnY)5MNM6|f4M6slEolfc zZ)|^uA^pzwISQ!{w$5=VP;xy9`A@d$8OVKcw>t;PH(RcQ><`<*5@b}?|7TVF3Zx_c zm&h8VBH7+Hpg>9MHsqt&E_NXo&0YCEBr$AThmehB%RGTh9NX*}q~qD5FCmq{_U9T3 zlsw%+K9Q~Y9&$9}h_;+hHPPv)Bq#Ad}6uoCfI}w&V;*<+6=sL4lI59LVRfb>%}YpSznvNDA0$ zN+4Uvwr3zy#Fke9>0&ll1*sCYgjy(2GF%V&QnvR-$Qkb1S|BN7yKIGQIa@^sWGdLU zyC7Z3memWXDz>?PC{PkJ2>EKZL4jNich4h`)UvgVLAH+VbOJK6L`8{lr_3z9!< zA3u;CWNVMi|CJeHyNZUiuvNxEYM5;&0SeUb%`6~4!Zx1@xl!(7(;@lGHk1k3F}9a% z$c(euJV;NlofSZ8lC7*53Y2V=LVk)Zqa1S6+)YC1KC-&{yNCau{}0GdY-MZ z8Bz;u#}*2d6t_d(v8{GOZjrmx9!QqhCi)=zkL_mwGRtf|Ly%r!3&W6FWvlxO1xohE zA-~2}Fa^1F?v`dC*xjpW3 z_8{?W3;U4WXNx<6%mJI6K>CpF^$b!+Y^|42pyd1-^2coD0dgnYZQet2%9i;A*)z78 z7s#BmMZZJ(f^FavQkQH`-%y~WDWc$4{)+7+3Ub%nmBd1F!?qR=*;}@>B*+A|$rMQ6 zu_+Cyd$!&TC{S{j1^EZI`dr98a(9pq{a!2c`mYGG&usrnAoIeOTn6b^w!amSdSm;l zf&wL-wUB>jyQzoV2Y1y?kbJW3wLtcZEw>Fa-)yb}(m!nR-H=k*|3A**UMNuV)(`mz zwzfgYMgA|j5J;lfDn=n2&9*fLnHaXLNl3@C%}zrqjxA;m3Y7d=fP6gL^CILDxNBa9 zB$4fO6|zZer5lh*W?SEabPAi^g;Xlrl!pQ(5r>dZW9vJHoaXNS6q0ndh6~7MupM4O zCX=n`7SdU4%Xg5QlHJ zWVB++{>VGQ~C%3)yM5s07H&uni~nI$$=3h94rJ!OzuX1l9^0ws0TkY8atsD<1rcLfcQtg-!T zg6uk5l7-9$+ut@wZ?b)MKx&Jvvl|MOT=znLo2_~Pay#7Z4nnfamOBjDJ+_5W$auE+ zaY*m8$t0u>*xsh0KuPNy<8OkI%GcC^0FZP#pZG#_05)$4+Tnw3nBl*_FfD*m4jyj(q9I}yY z6;+UlV%x5PbTnI5J)~mT<{F_uNlXjmW7!54$U@ z+xj0!C9|aq6eyV*fqV*EbaZFhNOWlDi5-aZ2bk0X<~aU zf^;)mV=1Is*pADfKuK{W#>mTWD9MO>SC+w zgaRe|-H`8QE9iq<4|hugko2-84MDb#ZFCs=eP#Ur=j@-OkRD*`7>Cp!w(ChKP*ODm z`9ZebImiuhm*XH2wuL3g4ztCrKxTwZ)*wB~_PPP7zih4BP@v>|7xH6l<@=Bu=Wg>5 zk_ooV6Ua`o&747IiY@vQ($j1M*N~cFd%A@JB~AB`pJh9Fgxnl=B`=W7v#q^Bc7ZMJ z6Eco%@*C2NY$~GkS89o^Hwp@r+{HltA6tDq7E+#VxE>0W zyfs38pRKJ0atGX9v_f*oR?z|3BetzB$Q-j}^+NiDZMGj$r))8UP@v?GK>m#Fc?5Fj z+%=Cua=~^w0ohBo(rL(Cv8~TS`kGBIKjB^$DFY%O_^iDx@2fOG!epg{{91vZ-tj4UkD=Yix$JW;?QwN@pu>hXN%losiF9OYMPNCU@g~ zkYus_3_v!Ut$PSEIc#AV(z$H4e<78}wm%L9O7f>5pU<{51Gxh366YZ)WE*vmEn@pv zf=n@6#|orN*sj(fRmxVi2?a`awjpoWa`qrs#@+lrB;{;zN06;x8#;kZCEM#6q^sEM zC8Vm^&aa_BNm+n=4cq2Db|SXDt*cxvq!&09$nvEHY=4HsuaCfo_$t+vx24v^h*0vxs&!%@Fy}&lT+3rsvx5Qoj1tkC24zD1)%vN{{nH9F>J4mmxB|k!Hjcx213Y2`kLVlgC z>jQEd+}(UZvdLDX%710I*!ChJv(1(l1L++$7YC_bwuD3|P%@kh`8~GxRLFVm+R`D} zXS>XV>;YRv4rC75w(}r;#FkYEsbjXeVkl4&V<3ORHdqe1Q|_KCAvt4fse$Y{+i4wS zF4#;Xq%Yaln;~_@mfi{lN~YQ&f6W%z1-TpU`g$O_Wqas@Y+!5n1DQLvqajG&vlWd% z>Va+LFBB+AnSlHw+xQgZp1Av-f#jL3dmgecY_|?FuWYsdApOSXS0MGymcI@KN)|UE z|G}2H1G!J`M)n~2V*A*K>^EEc5oCVYu1+AW^8P>jE6*Vn@xRp0B@`&hzJYus+kAjr z6nC)?kVLZ$JwY~x?d1hBv26Ao(s68OpOA`YEBk>0B^wbHzw!xe8PSkSolY)*;?nIK*{+6$PTl6-h>)8f&A=SY4);Mx`$L7Tkj(jD7kxvd^=nH8{|5;JNSU4ldbR@vR!QdROPQs zH(PQPqjc|8S4#_B6MHOWKvTfBsW{fSX9@67%vyG6NV2f#i0wsT1 zAwS9X+zz=Z?wY$GnPxlff$R)hX+LCU+1CF+dX7yCq~_VCMxa1R#2Dlk*!m_Q=eWC{ zf@G1cVHUDWY=`rZ`NvkY2th3$jL3V?! z<^VFAZ2kz+TWoo!klJQjJcj}$30IKcVH>%D+%9+T0g^qo_6Nv%w#z5T?6XzALi&Jh z`yEn;Y}sE>pk(d`@<(j3kyXEP$J`A@Lvq6Q5)0W=ww46QoUxrHLHeA{q(bU~Z9_wW zlJrc-U$RYSL+*;Z$UI1{+4>70d&Bln1esg5#!^TJwxcpg-LVx{LV=Q%YRKQSrPe|2 zfxGbrNFLdKnjrhc)@>p4%of@p{lZq;38`1M{cb2wlHUjUH@2k#$h~uyI0VTD+vqT4 zKiNJ;A@jx7F%Ic(wyQ}<{jgQdK!K8-ImoO0|6fS<4XIeR^Ia%VQnnBIIJV70$i;J)aRU9mh57%-GIIvmM7F3)$Rx21TthmU z?eP{;DQr#mP@v@a5%Q^QB`=Um<8Jj063v$O3E6bEiEqecu&IdZU+GM?o+wCVvE9W$ zfs(p-$Y--1BtkBSyMh!*a@qc+LBH>e`u{mtQU+x5+5Tohx`6F72U3M>o%v9p>+1?tVKuK#0bkgaCR?1fAX+iX9iYuTa)AyvorN1#B-(+K42*_y{7*TCJ$1SE}YrPGjYVq2Sq zOf#EafOHGn&wuBETP%``l z`60IVAIJ%JZILyKh8Fo3AqXG zp0gpDWNXQT>=fH+0c57xOfjTq*w#xSHOrP>4h2f4Dj`3|7Fh$idG7k^AX#90Xn?F^ zYiNedBHNLL^b%W9JEZ=xt#m?xl9V3EFSCvJL2iY+?*T|w*}8`yyT*1q44HMd+P{$A zVDsaU+GNY0f&wLrGmzh6OPq(?Hg_Wqk{z~>CCKiwwXZ;CkL_v=(w?ny6H@zZJKIp8 zBzq6?2W<2EkUQiq_6U+AwxJWq9<#liLFR*9;4%sWV{!hqUvps%8`i8ABqV`wnmhCtS3X~MbLO!sq#zXFoyVN8|?%5_% zAp5}flLnbbww??~Ke2@@NIkRF$Zx=cs8}Yv+t{XCuY|;zqD7M#r zNJX=?4nl#FbAfyeTlpyDV!7KKgCveEa}u)gY%|l4NnnefgLER>zyhR_*q#=lKuOaw zRw0+dUC9O{scdUokWFJt+l7o~oAi)QXH$oe%3$j~h5{vbr;yKNtG|F;7Iz0% zkYuwJ-ajaV>;vS+xEp+eWSs5g1+o)tE$@(-WIOwW z^c0)gVyj_oJ|Qj2WG z*-)TlB^UBbY^ep1`^VjQ5hTlOKP8Y|Ve2k~%qm-`fb<$$Z8fCU+4gIpKuLZB*$7l-v|BwwZ&C0r1sdV2B1L6&LHGHTh1`#_PLuM zh2(%OZXB|QY(tZfIbwUAhV(I;orBZ~+xY?%C@EWl{3+Y!GUU#<%UFYcZz0X=zYWM< zutjY{=8|n-7t&X3j~-IjY)ywypyc=%@;7WHXOO$)ZuJ6^z?OCm**mt0TgcqAse4F2 zu=PAb>XGg4848rty+Qto?cf7)&)gM!L-NA*Pc{6?zOp4nLFSF^Zw#d0**@bS^}*Je z2n9;6lOg}fR-Fd9FYb2JA^B#@&4TO?+d>XxRN?-c;~NBl35LP$ljy%j@&l2!xx zD7K4o$VGEkUIj@E+g1%^W7#t6Arr?o+X(4+w&)f}C9wTzg#sl{?T}AoYwm(v5_cy( zkR-E}_Cq#>ZS4I_y zIiyP160V>?$?y&2OWEE7l#vxZ0VU$pnl&x4*4dw$UMk3 zbJte@NekOU5o9e}Ln&lh*^bH}-Nsf_38{9rm1-zZl2Qlx4z}?I$aQk}-2_P&TepR5 zH`{F+WO~?YJ0ac6=DQ))$Clp*1xgkNAm7iHI0U%??nZ_o`NQ@x3fV!n_HoDzv0Y6< zTG%RQAT`XkGY17qvK{0{*yfiYH_Bb?3M7BohSnfE#`dxSnQ=C|4e1HCvt39{vX$*a zfs&0w$WO6loIq}xyXiAXX4s-GAv?>~e+`*Aw#QpY&$Bh&Lu!HT_z?<}6u&^;v8}#A zZjrmxPe_*7CcYv2kL^b_{mLw}^+Z8>g)PKDYL%@n9txD~CqjOWtsn(*>)b7+L9)S? zlmXdIw$UueY_WajKzf_4BOg*bY}bWQpron<^1E!i26B7c2>cea}w$bE2EeFw=W z+uj3Yzu0o0A@j}VULpO%7XJY$RrLSaKl}v+O5T1TAHmiZ+59UP`M=~M8j>isia5wd zvu!0nCWb958Pc(Av#F4ZV~a_L0wsSkAs^57oDI1I?wa!;Nn|@MfNT<5X)$Dy+15)T zox-NeA(hHDRS5-3B5EL?#@1H{InCXD10?Be4b70vU^}#s$z&^PhjbR(awnv+*;0C- zK*?AiHa`yOLbkjqNENXy&Om{Zgn7spvyC{& zm2mgI1W74d`wC8H+TX`HQUP> zWNX-3E+JFPc6JTvIyMs^RnN9@4+TonpCI4BHvIy*M(!ftA!%ak|AcHa+ru|xTG$#R zT7IQ1+ffvxTG@(Yp+Lz>JmlNhQj;Lp&fRzlBpqx&X^`z?>&}2o7hA}JbT?aVE~I+c z_Vb}YNq!OJd)bysAlJuTVi_d;Y@-#B9bo&Yg3KSbj#@|$vR&0fYKX0>2?~_#v_M|i za@rs_%-wtkBqMBb-H;t+8|sD3U$)nNNRP4EK}e0WoeLBwDI10S1l#5q(`-?5keOi{Sb+2_+v6gn=GdB+p+L#;D&*(cN;V+3z}@N=B#td@7qW|N6CN^4 zZ0Zow|JZttA+^kQcM1hc>MkI^!gg>4xmE58ZXsD?`*#P~b+)8O$ZW9veTMWV+vh8! zw%9s9pg_s>7v#6us+Ikf+u?3E5|UlE+!)C2u`R?w#)`3ArQg%5xw&X4}ex>sTa1fDJW3#H3Ruqwyt@|y>WNrAbDr2`3Kn#w!Ia|e6r=OL;8!& zZ9?jsEnx==lnn1d{)g>-A9AYr|JRAyjv$HnU+VG%vXN{R=a7kF+rET!G+WjUq+;0S z0u(5Td4PN@+u#%A;<$T$fh3--=Ca*ZKqil^wi?p;Y`zv!1#I~ZP@rV73G#((i57B2+>NwBQq1yCv23 zE9qrRih^t(+h`2*`#ISEpR<3)L3)6#BN0-6*shbIKuJ{^$Rdl~+M-oV(2$NG8}a>mfVIHq!{1DYobq zNKdm3v_fiz?Wr9Klr(igewOW|2Xb@VmGnb0&$jjlvI}f!0vX3PIRfcLHZ=yRCAQuP zC{S`Y1^It$^|O#$=I&q~k`=bXMaZtQ{rd-*HMZncNUyW~U5C^L+t(%(DCyjR{3hGY z9^|&Tt3H5an{Dq1vO8?Kr;yoYbLWuWV~f9nlxG{hfdVCO0rLB7Z4Zz;;O^oHl0&wN zSI8c*ZM{S0m@VrI(kE=QKae_Qi-~OiRiNZgH000No?{_*&Ruf?Bo}O_Nszr{D@}#W z72CRo^fjB#gwze&R5lbSiO7TeEn8m!9Do8P2}6+o zVjCHT+&6dcqmcZtwU0wqmHdD9Urs_M;(w{i8AwO6ZO=g}iY?ngfs(l;$Vaoqu0Sq^ zyTLU`V%c6cAREWlvJIJdwzFMGC$O1)NF}mu972JT^b^P@u}z;rE}6T?OGr}K`mZ6I z%Jy&znKZV>dq``xqen=kvlYKUfs&Os$Y-#nenKvjyYX*Gvea@ax) zq;uJ7;~|yDwx0+EO7c@6pU<|G2Dt+65;GtvWE;(bY!TZ>4rGeiI`Sc1!gf^%sZzG8 z5-3ozV<2zXaw;HK#@&1sB;{;zwUDh~8>)v)CEIHwq^sC$3#6*q&Rd~CNm&QvYuGls zAXm#>MlU3FY%~3kt!IlGgiHh5fIzyD?QsNBO>9kLP@v>^0`kplCDV{=;cj&n63dph z0NGZyiABh?v8iQ9x3l%ELaKx9ZXF7g)NMh&lkH#!a$VdNcu2b0{vANJhb`$CGQDhn zPa)mM_IVDeezwjlC{S{J1Ni~A>O08&;coWuLV=Q} zY{*ZuHRnNYhP#sjNM_keiy=G5wpI$6c{W`R=>@jQN=P}jh#DwR(pv}lMYj6}$SrYK z-weq=wnGcqWwye0$gHp}cS3rVEx89$YiwhEP@v>%0P^c>T|He22lq<7fd45W7166T>m$*_a`9^3m8+zoeqpOD7B?Iz?LxyxkT=!ry)sVi<*OMGF$%wWK!537a^U> z*0>C*G`8baC{R+o0eQ`~x&^s(?oxLl$zYrCkj-TKIe<(SThB41v)RHaq;lBmE}%fk z{uShN*$QqUm&e`G9VGc|Nso{%U>kjgOd;FnE2N9qIzAv(%y#_+1xl(^_pf{j+ioP} zO1aC4fyA&a#6h-jR7h2_wWdRX`n`EKVi}oTW=2(D7ou{d^=nHAINoZcQ6D=CtKkNWV_h@{e?_7Tk-^?d)WR?L8_PS zYX%CGbk0M*kL|`muAjT=e~=8Y?X5ue4_od!WCq#XCZvbh;&&h=Y{Pp{pyX{I^22Ox zN01xg?&1WJQMQV6$o^&9x`fOaTh6+cSxq$ zPCp?#!&dqOnOU~=h@M~RIW`>)sd=`kSSU~ukpTGxw!S3DIqvRLAX#K<(2!kXJIsL0 zKenQ5NH4Q3=R#_QEu{bol#CTYewFRJ1afQKb(KM~&URY?*$uXuYRGJ|`C3SCvE?;D zYMX7b2?~@XSjg|NjkG~-m%H~4NcPy;yCLh@E_)%f&sI4A=>xXyK}a34We-DvlDSdH zAF;)bL++Tn!AVF?*j}a~d&<@_2bnXrvjs?>vzaAGU9fE|LxGa?HOODGO>aQ%io3{d zNUquXcOiSj_TV9N%hq@Z>A-e$45>S|;xi~vvT_0Wd$!bT$USg3ehbMX+s_?jpV+z| zA@j@@o+16hR{I92SGN5RC{U9B4f!{=CDr>Y_s(5n6eJ&PqcM>EWc!GN%okfnBBa0B zu96}3!&a3B1xj|(A+OB;XMau>b?pFUmqS?{}vgvFSBaq2pQ)7_MWb2uLR2JLa6ci|_n}vKf+rd2Ka=0s4gd~^k z-#^Iau_diSCZFx^I;0EOJ~ts%$kw?71xl{>AYa5*eE_*)?sktLDPhY!g={I?!Z~CN zTl^KI%h==wQsr!K0Sc6~K0v;L?cxb?mE4uTLQ=)H^$yu;w#+Zc)UeI|K)RMKIB8n`=2f~1kHG!?QWcCZyzLZ*{#xf;@4Y{_+y>Sh~jfC437O_1+l>#~sRz5w?mM z$c(aW&q4YxTb6^=7~9+u6ex*Vf&4hz;2Pv6xO?7!WRk6A8?sYur@N4uW;6Sco?%-* zgw!ls`Uw;$nL2~~99!fi<$6w)rZ^9dZ|23&|1N zP(5Ui*7qHz7L#~j!>c5Z_vF(jRwwNt<3Nj^ZZU)k&Z1MAuGHk;R z3Y5GpLB5QwZ3S}W++D0eQo&ZS3E4`vt!>Cuv1RQ+x|(fvA5t}JF-OpUB;9E@I(-1f z@ppW`h7q!7B>Og!Bw-{Z9xY=$VAa#swzA)BqWjSZjjxS+;UBq|I?RYlVcL?a>CQ^K2s> zkP=|iyC8Xit+N-B7TI?DA%hZigOI+&78-)IAa`HGkXUA$c0sCOyLUs%3Y%jbl0$4K zlaRE^*5-u_N^JWeJN?wG5K^LSIRVKVY|1Jm#n_J4AcGPu zQApopi^U*qi#yXcB(~WWb|5v*_Oc5pJ8YhPNY>agNl4md>pOxBN*o+RdVH*u_HKZijj2lQkWOLm?Qi|;&4;hr`dW7^Nw!{;prMatrg~T!2Y7tUT z*uFm?<&@3$4gKBq5Jz^A|AC})wqYswCxa5FCP>%W+N&Y$g1dMPBre%%>LK-tZMhLr zGHf3fNWNxMTOcXRcH0UWlo)J>^c%KxC#2=Lv-UvZmTj{SQt#L*6-c>fTeL%Rp6%5E zNe^t}BalIf>rqHAu=RT&?UB3W1SAZ$<|#;hVq2evlxH?+7Ls4s{C-G!WqS%h1|>$9 zApMQ)av9Q!+;xW_@y@mvhSU$XhIL5!WD9RV@)uia6Oz8!X5x@Ri3bhROKi?PNc-XL z>;MvFwvH5}{<7_)Aw{zPJNMU~LUP4F32_cdCbrKD$e_ej2GT3p?y`_p#hv{Y64h+S z_mFC4Yb`)Z4cnFh$+c`%FOXEnw)6%Wlz4lG^m?|5FGy?PE?a^`Biq0)q&Be~RxJNX zv9QTiklf4`HA9ljR#68Tl$dXT^cJ?~CP=e#=awPS%64UiR2y4Q8>Fq_hSvDce2e6LRuGhg&|0EvyC_*wTJE81u4C3onw&P$EJ-#Qa@Xr3K^7G z@k06l+m{d06z;rpkQijUpNCXC+t4DU46&UAA=$xZTY;ouw(V8OphR^9(w%I-knUrPJVM$GcjYHY%(BhBLh2mbV-ZsPY_3m8 zo@dj)At}JtRfY^o>`LNK`T|>hC8RBK7pjK-ZmGcUe>IRAWSg#slx4PjBP0tpM>8a? zu${I*1|`~TkRD=-w?o=0cjhiggxQvRAa#xHy$@0%Y^nmu>ufnYBt_Wb$ zZ)V7##CRQ~=h?CikoLe`zXcKnwnG_GAK98)A;n;ewn6d}o74$O&usJEkU@#3UPymo za}Pk;D|eTJka%P3aX@O3ZO;iQ?`(~uko>_G9)qM$wx4mxpu~&{>0fLGFQk2Q=bV8= ziS2w2Qh(Sw0+3Q>(-tB5m#uaglBA)3ujk4NWKiOB71Ar%yb(w<{gb$hLZXswXcJPa z*p9a$rJBvAL9&@`D*;J0Y}E&lL5Zb9NUvop9zj|icM~U&sAs!5gVYALfeT1!WJ_H^ zaub`Jg(M5xMh-G4QE?CH&1``ONRzpHeuP8|+t@RtTG_5%A*GeA_Z^aLZ2O;()W+6S zf(%Nml_9;Itt^H9q;+sNTM3Cyw#RBn?P43Lg_LeKy&jT#*gBgash4fH88RqQXNB}W zwvY|d`nmgRhr|HebQh#5Z1+8oGRWrWhh#h3i2_MOY;8l(-#w1;?h)I=knUhJyC7|t zE$D`XlkMFDsUvKYlaS(K%S}P@D4XJgBsbg9EM!okWggPU*kTKi=Hbp1gv2=8f`HTs zwwDm3OtN{_AX#P0tV7ZiTVD(^C~>d_>0UO=4x~+U7ukh`kL`C4QfJual8`dXW~3l_ zj?HxpNq)AAQ^=r1mk#OkY>7)q3vgF|4T%M|)f-4%Wc$8_lqEJ_9+HD>`2r*@vkgB% z1|?2kAYHJv7a?thyZ8qrLTokPkh;pY`~xXrwvS&(USm^DtACOrY`0aAL5aZ{NMC15 z*Fjp8J8L5(HrO^TkQ!sFY=M+bw#8OR-eP-ggQRV?@lME~#C12M$JzS(AZ>@cQqS3THX%i4tBphQ1)I5}a;0U4B-I)L;mw!1?}%W!8;L*kn4_ykh3Y^~>za>KTD0m(VGsthFEvMpsHgA#8! zNWWv7xQDcR?y?V%$g>R?kov%O_zWopHu(*bAK9YskYun`d_e{!=1Y+N#P(cWx8ybg{ zO16_pNUmbDc_FEqZQBPKl&GGAbTeCU9@1*KD=t8ymTfW!sda2O0#fSPlvPM>U`wq* zQX^YS6f!8W5rgz5Hq$nwS-1=AK%$xLWfxLqwy}LkX<^GGA=%2-djv_XYzN1XL5Zd_ zNVl;?bVzICu6zlJcDA`|NbO*Iyn&QXHrE{_cd_YtNa|+mdV~y0>^?zy4_o~!r1f$a zDnfscwEOqD?)w8$``MmlJ_ zTW*BZVYYV*q&V5s7Dygp%e6w1i*2wSGAMD>3F)J3)*eW6a~JD_#28zp0;wLh1v{jS zv%NYXd4kO|0!fo>*Q1a@i9Qdct8B>$NSor$G6e}Q+xj%5PP6^aK#Gse?}y|WHX{H@ zvuvYFkU@!yWk{c6>kdJhpSwgD67y^g>yR2?Tit+^1-8;ABrmf0;*hk&_MkxqC5HDP zJ;-);0BOtIwWlB<*mly8y24g-3MnBraSq9=Y#$ep6lR;sKn5jlvyi^VX1|5B2zTjw zNUXE979cgswrN1h23yq&B*)km-yms|?d=^hC^7y8>04~s5~OW&*Z&KNINM=GY(wIX?Pdp3@7V?tkdkLh?L+bdn|ugK1-6YN$e={U38X)=1 zoBj>S-)x;_NGh@IO6z|zC{b4l=|60tYDg<{_f-RlU$*IbNR@{Fo%`<_A*JG4Wtcn7r%vsovkJhsY7hb1xRtQ zeHf5D%%;9Tl9TQB4KgS(_yOr7Z0RpZb8%<=fy5}=<}akW*(y!ZKPh8ui&c>9VS6=0 z(m30A9b{1Ax&hKB*!nGyHpyL5hJ?!2+zP2vZ0l{1;$@RMA$gk3-wjDVwx?dmpv33^ zq|dNj4no>2cij$1%(3k`A=S^;FbXO2Y~e9T4zQKRA!&haMuiMYJa{2}k+X-nLl z%|Rl_))9czWwxC~ND*wc%aFXnCRQLR#P+!g8I+icK>8}%T@=#7+}SrFvBq}14XF{f zRt-|t*|rjp9A&FIfTRt!r9;S|#M=?1$Ji!LAZ?So>=`7s*aj{jb(`(*3R2>1au$+z z*rGW|(%35QA%hb050Jjg_WTHG3GUp_kl16pdWF<|ww`xLIbhrWgybY!V+oQD+1ARC zL5Uw}<4<~uZMG89j<_pSLn6&KQVXfaZ0Gfma>CZx1j(mtS~Db_vDH~2gAyw?NIz%$ zYKJtPJ8u^xF4*pSAoY@Ms2@_U*iIBk&al~rAnBTIdl)h(QSE~CEL+eGX*b*zJ&?$; zO-@4UE!)i$q};J7K1jZ2OU*)3o~>mbGAOaJ0O=2GrXZvhxC;nKJhHunAk|y=HvJTm zzSz2S$e_gTC8U3|)n7weiM!AZB!1YwZy~kJHl2r*U$%S!k|pQAbHC#W`g_LBzweEw zFOWfrwj!jP*y10MR>__D8xmD)%Ri7>&G!BaDP}g+6#J80!1riNxl`W9k$hOc5DNSszZIEnX^K?Q|Guw4HWKe%M%Yk&6Eja*bE!<*AsoiX= z0Z8d#D=k8DFPm=}lKR*lRv?2C!(mA8XFH2P+5mU$8<0@gb~Yh(kgX;TDRwrYLGlpW zM*@-@Y*Po2L5bT#NFQdiryocqdXFG^pKbpbk`CCK&LD#lYdWMS z*~*uYcF5iAH6&7Ok2jEd#5QsVDQPx656Q=DosW=o!nXSa8I-7dh4fRlP!ZD3xcmBm z#5vpaH>B!p_dk$w!RC;*{v=vW6QNd@-3Uv4oP=xN1c#CiIyHnzh{f}L0XBZO#uVMK&V<$?t5gB}n>UyI6({ zN_2%F{gW*bhO{s4>enIh&9=G$sU^1WO-T7+^Ti>#%$C<6>6dMI4>BlmdI0It$iH)c zdkWGj{z=5skT9{;oI+|P+wwW2RIz!U+ieyyC^2{o={0QWdq}J0&RT#( z9owb>sr77?FObr}w)h6gjcl*)kkrIB{skG7xGq7ug{}V=(wezTR&4)C$ZXA3klMnw zZiW;qn^XtMt!(}VNV2g#H9-a?MrBBEW4p9MT03{$ZII|-+v|YTPPT?_NaYA^azqvwu%$Tpv3$cq))Lu>yYN<&V2=mX|}6t zNcFMx7RlN4mDtAq?ntW-n#GTT=TqzUf4^^jO$yKjWl5Zh2Qq^z=?v_Nv0&1Qq7 zHMZ?`$e=`Z7xec;yMM=*U=O6Nb64zxM3ik(fz%DQ8#|=L*py*N-egOSK++aliyJa1 zvEhOAZ8p;+q{X=lOhICY?PVHLHMX%?NZDn}_#rvL*1G^ndu#_wkU@zi0qOf}kr1RE za90jPBFQ$l4ylK1j~kGZVsmXl@)4UJhom%H*DhpGVs{VHkJ;*zkaogdCIAcGQZH;{hG7QcnGEAGsBNMzWS3y^xv_HIB*mQ8(u zV4fwVjBV!x2MXR9>r{7KETEmT3u1KX<^k_&8}I!JnC zyKaCCO7vME-C#?~koLr#r45aosmL}u02!3H7=-k9 zwr&TceQ=j>LgJIHVH8ro*jC3N<(sWE4#_1pp9)DoY!6<@pv3SDq?g&w<{<5tyY>Ji zB-g*c&7DO^t@tNVvkWOFHn9T9m24lYkW|Gs6@d&&+(se2n$5lmX=d)y+mNVXYtwk%PPT9vlDgP_ zB<)WIC1xriy_>C24QV~xIcp)&%XVH5seNo6O_0*hrZq$I09&mUk`%TT8)Q)8vmMe0 z*}PqlX6Np%2NFYUL;aBIU^`YIWth!21j$aetzk$SVXJmQ1|^ocvAs0c#(BwIj0n#$dC2oh6lV{4G=WxHC3 zlxeo!7$p1H_O~EuhOKD_GAOaO3+c0LHSU5xkchCo|3d0I+oWmtPfC<6R|Ux%Y)TCz#n_JOAcGPujgY>{7PCOw z7I&r=NNlq$v_fi}?WGM;cGx_fkgTy~x*=(ot*;Lnuhcvw!{pirMav3L*kfiH2|q6Y~PEJa?0jg zhU7E0{0bzUvkiwKgA%6^NY~lgHz4hTyZ9y~F4=10kb1?otU*eK?IQun*KFzmBxTud z4=VN0JtT8=yGIV5h`HZLIcj;%5SDfeuPSxC;az2+e4fo=RAGAMEV0O;{lW|O`k`Gw72f}~frr!r(vVzeUhC;g4>vJ%pY+;y8F z@y@na3#lJ$4Goa;$rf&c=FR?j0Ank{{vo1)K**bb5^_Oj@ zA5x^zf9L+%K}fFnCn1I)$;9?K3>nnl!4c9c+3wtsR>hrt91_)R$CHq1W^46AN)6kV z50Y!ys^%c6j%{fkGAQx30O|E?6G2F8;4Uj5(a1Ki3aL$Ohii~xVUwef+{_k@L6Xc? zu?-oNnBRf)7PjYINV9V1-iJgh+f@=$ZEQV9kka<2|Nj0p_!zRZ|C4AugG!XNrbBlf zY(JM!s*}6fYbesiR=9zD-E1RwkiCcPJP*meY@Lr#nG)I)l<#A!dxg^d+^rO$=m6W- z2Q;g&dA}jYAlv;9WVN#mNqc{ihS*L_P=OM*YUsqlwp|0ohPkV*hx|^qU?b!lVJli7 zn~QC-1u~7Y-LygmB`NLDnVT)u32lvW*U|$8JZu|%kZYXHq(JQxYymq|HOcnkfSxHa zHUjA?TV@oBPjT1lffl`N2NRHcnyqOH>h!TirXlkTTX_b0rKCAObUDlR7=W}n?p#Yy z(9fnXL!NoIt`O86VA~BtwF_+Z>rjyrp$#ar$o9PnC6>6Gjzc2Ime-((Wj4ni)GOFd z51{%Lwzd@XK}qp6lnt?&Poe!)?v~G?P?+ug0#eu5)C|-gVasKq#&x#ATj+}tNB2-J z%4RJ<$qnve1{995RlY#pO}2$MNZDe0eTOXDY@RQuL`l~r=q}FI_Y0+VxJy>-|0$xe zS*jr4F59{pvM1PnYaw}$&EEi(DPc50`F*xg8A=~;cVUI1Nw)4bX!ejT(E&M9Yz^Iz z^@we?7n0I!rGBVD3Ev=ea?JKH1jSCc8+JneQ?@e~9VY4qn?bmGSAXJrQYh8h!DY3Z<={Iau5h$MHZZQfi z-m<;LAom^H_%_se&z9YR%z3u{1oTQthx^dw16%VUq!qY}9znrJHt7WN7;N)rQ1=tt zlMdBBv$?OJA|)=bq09?gPYz1Fa<_K}i8r>!2WXHTbuODI0T-AM*oRM^^XAon2K z_AS(DXEWy^^AKCG0KHPuy8&G~*d|{fZJ4{<8x(Z1DIbt$gze}H>UOcU{6Mv%Y_VUc zNC}fE`KOGVZJ`QEjB)p3hJ=UBQwL3qvt=5f-U+ro3sgVJb|6C^lw@gzvMO7o4cedL z?zaO9dD-T=A$6L~=!N=yY_0*Qafa<;5c;A-mjlYpvL&2Qa*n(DQ7G(ZTOEVE^K9Sa zkP=|?sgPxXE$@X&lr%g8-7T`6&Oxaq?%D%TB*+$DgnY|vHOr7)ur04Z@(SC>DpaO~ z8iDd5w%aI_Ugd6Z6N-k}(%aDN8k<#v91*t71Y})jt2}_DDBI#8RG`G`5p=S_Hhu!d zV%%MyLHF`gBhio&IkUPcpPz`k+u{moYbDHg} z9(tvujwa~xm~E#S(oVRmwL-yDHerK2XKbJCQ1?08R2NjMv)%PTMe6TCAyDRm?O1^l zm)x}uLE?&SYZ#iyuvNLB-fOlcH&mZxd-Fgalr%93WpCKBQ_y~ny8#~*x@9|@h15GX zc^>M&XNxXCjd`|;AoN9vc>(1f*q%dBvcR2t4GKT9U9Cf2gRLh9DNk(sTae|Mt#Jn` zQPSEjboavcvj?SKxtmQwkvFzN3i1`%Mvfu-JKOmwB!94V>QI>y+9j0#WUIS|(qG)I z+(6N9wy#@gw#4SmLyjM|`vPPwvkg5#(l6V|3sj(ltq7e+WBqJ0+sk$uf$IC%+BTpMN{Vkn*?u;29NHh?Zdrpu3fp@EQU}@81E}B5mOF$R zhu8+w&=)0+PN1BF&3X)ht2W^`NrAS-y!=1+wUhNPqO(-P?-`&8Op0{qZLPgN>6cjQ3*x8Y~5yP zcA70w3pspj4GoZWhHbS8l4jXT%}{|7J}Y!G$M#@@Vt($1J0Sl&+gTUn46wELLbe6A zoqou)$W}858I&Z3ptB{mk6~yl$la6+3M{kTx*?ZfvyVgVD{SdWs4B$P>V=*uvFU^K zRko@*C?4i+aUNP+V|!bG+!40%Ak?|emKBgW%GSRMy;9QQ8g#kA)*OYj7FRv%}VN1SK@?_KqR3%hq@XO(fXDI@G(z_Hzl< z@3YNZLm!k>xPh_U?VjP2M2Dd%jqYRICqZPh>}N~*4h?k?Dt8lluBcSQ>nxni4W zfqWUZn^wqv%{I^u$yv5kCsd|{+ymur*f#p0bdI|U1&ZFX1?5s z+tnx}J+Sq9paLcKC!mu8ThkO2d*p6y8uA-#O*XAeb5IbO(;-yl`U(B_QTu_ z3`3zcw!;xfjj+jXsDGU;>VX=gY!#Ex7bWJWpxg%A^E8xGVR&!9UqTi*qgs^Knq1x0GvELq4`$F`n>?DcHFcaYq`=6`_7lrSEld?VZFGn8)P z?&1}STG+bZq1k4(#3$sC*&0fawS{f93`thDl63Z`0wsKv&`B%XLp2n$aW`BG`PLF)4TYD2^>tNeyhD@DoHCD)=Bw>Tjy4XJ2p{;K2rn;a&58G`ISt>mf}Sa{ISlCoY*j8Ou5h>Lh873e-aL@o&Ne;?bq=v*ry#R~t=|W|QqtipbUDn{ zJP&D3?xG7&aD+_?LLL{}ynwn#*`7jBt((og1{EoBxejH<*m`16!o%I(79_^m8h4V~8Zw%$IdK#BbU=p@G0WQSs#+^snv{}x->2|2gfW=A1gob7Q8 zGVQR9Oh5)D=_+)lv2{*ETf5xt&Om_#Tb&7i_T$s7MJ@2FhHrEo7m@6?ZQ=NMzVN_t3;OTjl}k&9e0wQ2hgrYG1EVZYXSKy+4@%@XBAs=4YF0UHAf+nnQc7=8I&Y# zLuWN?{vBwmmb<52C{V{Xx(~VP*)Ee%djnhd5meR4ws#CYQ=;Jv(wo@AIuy5XSGt52 zo7rZrA-Bx-a07L=usQD_vz6^E54}=S$0Kyv%C_?aX*TX^U!hKm%)sX06+p2*kdfBS#q24~WrADZ}pY6>8eNfUw z3zQvT%eF%M3U>qTP-u|tuoF`4Y;q6OKg1U8gBl%d6$jM2?2~Ng z%aE+Hb%vlaCA2V6t0upLxf{>i=0)>H*` zM%f}}$h^T;u7zGHX|4gfjIlj7LE0vFE*T1LvFTRGv(47k26e~Tb~~Wj9k%*zs7Q%W zFO<>PzWbrXE_c&|kVvrQhoFf)Hir}H-Df*>LG=f0ZDY^}CB?^~Y?95aLi>l@EqkF* zitXJ8sYh(;9Mqp?%gsZL$83X(&=)0+f>7>+&AI|5Pq~Y&Lg6#E$_V5=XIqFuiq7^L zgDe+po^7Z^N!L5j-6dOJ0!m$Rm)wUU88*uyDOZ1!fT z{gW--0#$vnwc4O(N^G`6`Zrru7ZfjXx7Y(M{;<9ELGCi!xB_+lvSsa%S(^O!8$O%MM_-yp-c^1&jOUF z=W!_1%+|3BsWO|k z2lcnG)h3}vE89v6`l7^V8p^e@c~7CFjk~*ZDBQ+2bP0Le*^V=i(!plCfh?VDTenb& zlB)C2T^HL@0ZMgqS2UnV58K2G@q?4z{ZXNE&AAwLk?*?90%JldY*0ij8o$)&}`qY~>Ef zIm$NM4cXjmkG+s-jBR89GAKzOgw8x{oepSgoV#5o6qsPE8--kxY@sozU1j?khpML7 zrd8;f68Bz6_p&)=p!hU*Cv(uEkF6~Lxo6n67opBsHuExMo?{EHK(Cbaz6xFX*(M{9 zHqTuy3Izjf$|mGlU_07|x)<46G^looEtY_alrSAYnIPN3A(U9=?&Sy)g3WURO{}nG z&Y<29Ti*p#zsh!S1$|JGB@1Q4Y>^zazsB9~9TbYN%{@TsI-Btb^+(xU&rstA+r=yN zMTxF=C>LW(d_u`h?&?cWc#Cbd40*TNzNPCwDRDMmC1lxQ%U44sN*b<(?liX3dMLHa zU3(K0NwCG6A>STbjTN%*vn|^o`GD=C9V$~o?Sk@2w%Z;keaPKlKNL-|r4?xQh|M|# zInr#K!;tlut3L zTVH?-N|J)m*%g~#KwBB^oU>~x9z*5=+u14fN=Y3$bot1(a|vk%ceU40@QF>_K%Qr| z&s(Vbg>5Pi)xNUb6`&#|>`zeUjqUgaN))+kEkfd*ZR-P?_+YF0hI&8QmVTi6FSfT| z=!23bOxZtWzuB@?(0+-#ff^|E!**B)sbw~~5$gYCi&~&YN&R;%p`r!)qQrbFl&fHS zZiAAhe-iFaC|t>Q)eU*8*n0XPrJ8Mj0J50b8tqVtlGYs1T@Blh6H3)`H#-VN>evcn zkguL?WCF4`u$`-r+{o5B4V5XO%|Q7kwmLtQws5x+fTGQ8UyIPJ%;sH&94&13E0ER7 zHWY@WR<@G}RG@@y13Iy>ZEr%cHtwq9kiVTRs6oySwqgRZb+Sz!K&CFXn?uN;Bqa@< zb+e^TpsgP6TF#+BFWbfiYruHxuM26w!v}eixNkZP|nY0^+L&c?qWVD9AK-Q zgS-oD3-ge&$o9GbS(ey5L8wGY*8;i=vh}S(sb%hxYfwb6S)!0{g>5|s*+Xom)(A0gWg+s+eY(%5QVA%l`c5jxvt`}lyi65LIFLxDZE z+aJia&t{i$e`-HqOPipoBwK4W^h}A(8c086tEz|MDee{*w9*53}jQqo~3ba~3w+yiN6+(rAK;5nP5KpvfK-VSwNusu1T+DkU~2vnrRe6Q3&~$>se7nQ3Aq5}zu7hn zC|%;N;suKSum#?r*)rSnJLLFf8~cK+($v3e30EaZs`w|V_ZKQqV!z_{PbVg}rYa~_ z$=#Y6@>j8yYawSf+iU}5GqXK5L8cnE5g9TlNw-30wQQYj&{iFHyB$!Vo~^DMay77p zdZG44wy%Dus)=oS5PGJ>{Sc&E*c?tM-pt*J3tE)f+QuMv3)}WM)M;fitB|>sE$D?_ zDe2t@UE0_t=OC?(yWBh!Y-dvzAx{U}Q4s3xWNTT0YP;BCt5A^=rU;bjW?P6ti5~7= zVvy)%^K3&CeQcQ>sJEZ3F9FpLupR6}ACzP{gt7`-n~$U4VX*#${{w#6Q(K#A8r=wzO4T!CT%?yl{Se}S!k7;-MM zB}X9J5?iwyG6mVzJ&-|3(j;`Y%;ukhwgh)i(@ez)YH`#XfAZ?4g z+9VX*W)msM6KDHOL)|-UQ>Rd^#&&lO6)9oAgfhEq#~CP*;I8!s5_@c0x6s5sTU8$F zJz!fZK=nzsHv{^hq=^?Od&ri3gZ5M04SYbMBeugYNKLcJKT!WMTl5!dJYlOa-T&!} z67yA1?v(A>3?GV(+Mv5Dwx13tmEmr- z8;V@B6?!3GmThDJvfr?s4?=Q|tq*Zd~dWM2kZ2BwY zsb=eXhq}#dyPr^P4O@K)DpDd;hBCEm-%|chi8}74DnY(2h6q4EA+aa}uP3?mEt!%j-sIir8upj!O#E}B!Y;4vc zDA~qcY#0i+vsJnvZwK3g8&W#iUOkYdi_J3$l_=?Y3cBlN>+?aW9`2H}P^6d5G7tIs z*wz;ydq3On5+o0>`2|#_gb{-B3ft%!lpf^nVjYUw*}7xU>=0XG3vxKv8g?M-Fx%=b zBstkidr*NAz9e)q!uF7YVlM86k0Jjk+u14PbhEYVkZp`@=Mpk`*lMmJgObDzbT-cR zaSLrta5t5Q0+VdF1<0kc*`J{HDYoX|}3wDDLBK@dsL*VSD?9 z+_P-sriVXu&aq{yAhVyXzXp1xq{BMsa-OZZ5z+$OMJ-TpflX?GJd14etx)$8+fy4< z8)S2LLPbhkc0-wEww^vHA-LNcfW!)0qaB(Ev4tH_?<(7m6RHoh&5S}HlvEglvTJP4 z31~mU-MI>d*4a9yAvMaT%|QJdY_)!Vg_Shy4Am2XQ%^_qzU>it7a*{1| z0+lHtpF{aWwv7uYo#L({14WP60$FG_&Gwvw9LH>9_mK63?dk!NPT6`5s6dJRXXxaN zt?3PlopZPL4*7Mq@+aiHV4E#LwoA6hGGw}98>uM#$)F^?5<1JUb(*2AYwmVyp+J_c zt^so0u!WkS_8i++GgNiUHf@ETDRFOu^gA|32Nb{O?xYJ^%(J!iLhc8)?S81Uz-AtV z%#UorA?THo-iM(}gKg3UX;0kc+)(hDO&NzgFKkDXQ1>faix;YWV~hEqA|*_7P^QSX zFb^f(xqDfF#0Q%v2u*ylWdzjw#n!h9)qk@ctU(`?WQjuA5?dq&?f-E1y9I^HY;!x1 z`pahQLjBV8ziSDueWe=E?kgtKQ<`uFxvMm=Oxry!L11eKO{f6=u zw%Z>l-OSyf^!TT!%$7Dmvn_1aYRF+_+pK}Ct!$O`kYr<9Y=jDwc(p($ZEWK$P^_K1 z>sH9$!PegnIXl^sosg}It+@v>b+fJaK?WsB3Ut=P=C?yzz1%%HpgMWaFfcody zq6XCHXRCODz9=#O2Ic12p5LKlfIIgW6kcGvDnZ^www_-|Sz_C-F#co-vNcvgB}!T| zLwCz;KebRwa5vikMON4fO^`3dHX=jzRkm|0B!}5L+n_Qfv<@i0##Yx2r6b&}^g_{f zwy%C@Hp=E5gd7`e_d}31#x~@Hq)oOH7gV5xZ45fuV%r{vV%ywRtB^m=7W6{S9k!wm zvT1CSbC7A5?PeY_C`nm_&Jt{?Ahfl|UCRm-*k{{VgtrT00Coj^9RU)ju|A+Lr2GDbk`Vt!kz8E{{Hz{#R*#Y|=DMQ~&BZ zRU{!<)SINe_oh=-RXV(kF~+>j%e>#l7-I~W_rVzRHekRQ1I8F*j4{T5F~%4(|M+Fn zqzDQ0@%Q|WkB|B5K-MkpR5obT$+oc$skhk*HlfNLHp>-Os*)dyr5{glRRX9({7hscQLF*^nP2@nPAlr2=q&sEP$)LIrn_mve&)6CZ zp*SUO6+`|oTS*yYKj&^<37Ie0vZ^5cCEG|1)DU5d)Iy3YHcbPRpoGIFC=g|pSh731sGa9HV&UUYbif-AAy-<=8!+lUN!PYqdIq$gJH$t;Xw(1dR;GWGk z2DJ!=hi3`-B2<#`AaTkB37KpOGZdmk!wj^O#dbOe&1ZAhx(FF_*j!eqO~R&Jh01c- zme-(6DO;``3R8mUfLwWOF(iY)u|WS-|G>LfHzo(j(}C5*Lmk zcOhGL5V96=HyVOQi`lNikh+Ae`x2@wW%FG@k}|fs7!;v|%{b&KXDhseR+QY$-b3RR zY(nNmN}5Wxp=_wSitStiNvqk^c~F!RJ^7HghOI^c*;L%E6+vPxo4gd#*0Gt(Ayqxw zT?Hg-VC%1jVw4b4LB2+|wt8s2iMzc<$kfbM(E{mO*jC!0x>h!6CnRrUo9u$(lz7t( z`P_F^T|f_+JK381A-$Sydk|{qVk;Yl6dJb0Q7AzPITKKzn{8|oa%j1WPC=F) zHmwCR=-7^Cp{8E8`UR*+&*oTyl9X7q0tNfn=4_DDz+J`$G~3TMya^2quw86HErV=b zJ5Y&{?Z5>ILu{&jC`5_t2hh$iTY(RnAK`A=4;e?xHe^VdM*Rkk7$p8G+WfxVs;NOis2z5z=k5g-uZ14qL}GB;RG*pMm0(STzs%U2L{R$iBy& zY#B1U*``(@{XX068r0xnGuR=;0b9@kB`BeF8wz;Yc6TAiA$LkQWbv_CJ&@suE%y*= z^0Q4GK}E-Gv12GniMk*Z46q%CAm<5pjpxv8kj;4s4VQ~Ib+MZg@iEM z=p7WI#K=9gbI#VC8A)mWg1f_P$au+CmkYH;*z9>w*%e!%9LkKc%_yKSB?!fk>zd72 z3N6OCJ1>WZZ`e93p^iA4ry5e;vQ^hY*$K9_dgy`@^P3>|9hRh(o z2}mks3rs>$N^CMi-aNK#3uKdVS2_oY`D}{|kXFu?ZG}_?Y-1~sOu=?#gJP7R-GF?B zY`#rsy@^>wfW6SVDaY`KWLH=^K3qNF6a;H9l%oS`0 zr;xsqttJdLRI#mJK#FQM`4yC)gz0N2P{Ve2136UO4J05-En6rF8S2>DGp9u#qha+lKx8P#m#%}`qxTeKA_)3Eh)K$+caM`|cc2@TzlOUvfyffjqX zE7n89I<`3j)X~e9IS48BY{NrPb|2g22y{V-UE`43z~&VpYd?3YDQI+nZDSfz53&`^ zLX}1~%RD3*VoNSU5lR?bhCIV;XRFZ42zMRp(D*2u+YV{Q*s8Xm>T$N!ZAd!7CUZei zN;J74ugG@mfozl9^&LW@i7j{pX{Xp)0+7ngwi|?G(`@BuP>d3+=aA3BCW%1nGu%x? zA=50|bqvzYvFUE1x_LJL9VB01YZRg>#VK(s6Y?*zmE=J7CGO^PA+wb&O9tte*+%40 z!wOqO0V!75G{sPY5)MnDfQ_wI2|3odvsXfvbv8u}WY}PvsfC*CZ1?q0(I%U*2})98 zxCIJ2*gD%G=N5PSozSe4ty%*OY_r+4P|FTmelJwA%Qn>q2`;uoKNO-wgAv-=y& z=H1-2jzPwKHrE8y=3!Hspt1wDWiyoNWy_s`!jvG+L9Ro#*aEcZ!_fF8n{WwfB5XrfQ1unt`86bsvZ>=xloCA&$a~FJa}U{K z+^uC?rzGC6$+IDCoXsqORJUw*Qb?9y>(7T`ln^R_e0OYZMbLVZyS);~bk9~%4(Wt} zhi3^Z6;NHqgG6aHB+q1Q=TTA(m2Q8=E8xlC-mp=Rgrk zxXy(<9c(=^Xr+@ozZ@D@vo#b#nl84jVyIffR$K;2yV>TIP?Qogt01qIZKMXW^>BAt z3yC^5O#`IuW%D*cDm`0mDg%$t;4Yi!1BXmOppa|txO!Pc1vb=cWF`H*sxty%$PJJ{BW zpbJXOFNNG&Z02&v>f|m_0gZ06^;bjc9kx>yRJqI6Ru4&BY_3KqLJ1WukY|r=xeZ!z zb0_VD#`oF8E=c2HyXl6i57>Hjkkrc-&_hv5Z0d)+hiuz}kj=+k=`bW7u`P~5T0dL% z1f)7<8=Hh=0k*3tC`Jid3*QbgJ0J}5y6(|#y$&2|@n95L<& zP9e(;Tj&fj#M#;}pr%{4y$Dp4V5_`_l9ae|0|oEc@)D3U$=zfUn!RU>3-Oc&gu#br z3HmIkCF4QDNe)z!$<{1|geL8bl&CviY=5trv3=PZK=31eS0=CQ!NU2~ORzul^Y?m77f)cxW zAa@a)w->S&bEh&uqa|z`1CY9utzZbMEMv2bK$3E{mYQZ=yc z?n1IgwsJQVqXer5@-?wZ4x#mC?k0{PQw!VmF{Eo{(*>crHa33XCD910rPIx8V(KX?1p(Ch$PbuBb7$Y!gDT8wP@ zO;E`Y+f)lA46`NLpb#Y*I-#8rw$m+&#+}%A?qx6 zqbtzp9NU!*QqQw>Z$Om`Y`#rMvdC8Fgd&u%xdVBY*b4Wc6)ShM`_T9@o8W~sD{MnP zsCt#{+z&}@Z0ZvzN{OCR$h*c?6NYT-+^ttS!n)}yXkqz7-35;LTy)U1ItiZl1q7+d8Q zw0Oha>NYeSXUlUz9k*;IH>6Ck#XV5=9b4ZabU}$HN02+o))IiM_uTCSp;5v3@GPPH z3{q!2NL)IHDl^$65lE87HXel{lyDt`JlSkLx6n!scm6wQT*B5M+@+++W!uVxs-AR>-zd3B@QuQ3Lsk z*etcsdNFtR^^mEAZLkT_m9m9fpt>@)j&?|1&bHqP#VN5$1NoI~HZ5eY;7-;HnJd|* z`XGH3+igG8P|apALW&x;;4qY+gw`=ApkmvdfE=~lDNT^2j?HR@4E1ceGf-0l+r%7H z)W{ZFfRdD`vqHfpw&NAZ+00$z8Z_I&=G=e=TG>h*P)i%zf)grfXUp1!gbudRJt#zp zk$q^Vldane&8xXP^g+fhwz^}eO~YnCfy%ns3PVt)mTe{sg(*R}gj_vr#w%!1$KClg zG~CP98HYObY@P(9>|?9Ghq4W9YZ=LuE+{cS8*=xvnI({QfV+eg8XaWo&xh1Tw$lQr za)_<12$Br5xk{i2B~+9{o)Na?3TS1NJ83mEKE@`hAk8@2O&wG{!PeUdNkz6mGZdx7 zrZ&hs$+q1A*-YG(c0uA4+hRAQHM3>wAk{S6m>!Z@*scsvj1shikZ*>~Hw3NEa#ueJ zndaCw$06N3ThSy`x45{G6X{}S89JY=_Wr(S~0%WMbBkbZ@&#s)R4 zvaPQ}3LBez6G~9R^cECYW4qgi9P8W-xFE|0TgVL=>}>4^P}3&c-XT=vV5{^)NlIJ^ zK*24xyi>^O{J0If|jhBkqdJpkY7ToD%9d zX3MOClmWKk8YugO?XniSpv0~Q$Q@+!HbK@??o_SNXozj29a5jM6{w-gFq=gKNzU1l zS|~yZgT0XFg6*siTDjz|V*nbDu(^$p=8COq1geg*t&Ty`Yc`n(MJdr_g1j-dTQg+4 z;jV86660*aIY@iU*0Knx5^TFxNOs3oz6!-C!MX3FXYc=D>;JfIo!=3LuLtERuIzXvW+x4(J$ z(SN6Zp7zFTsTcqLwa4H7L)!DN(o_G>f*}0WlQ*Bd`F;1d-b@|-lka}#^(SA~H(q1@ zp09ZGufL~I41GIw(D%Rl=Hu7jc=YJe7E!obut#zyIBbQ~&LwM^AqJ_{oCe+&K6?D-HwMqsKTZ4S zkN@&-zx?2*k5gCq_!|fOA^pY6&%XTP@weZ9_LFDtJbL`YZyfRR_s^fD?%D6ZN`Lk2 z#h3r{?DOrh(W|F#KmFvh^yeSHeEReM{L|Af z(mqFexU;G9^^u=_@hbh(^cSfYzWD0-^Ry42r;bUD)8Bsjf2V!=^FRIh&;Rx3N00yR z8&|>)^G|87(tnw{5BNO4`uf6t^!baozWyX{z5DFl_ny7~BV6K#say2hmp^*vop;`Q z>jxix_1W`}-+J%I|M=sy4}S9Thw1PBG%fY|`|0nefB4=9AHDa^yPtgU!6%=5ke>Fl cpS_!&{=+A4zM1+)fBf2`NB{nd|9JO*0gVB!NdN!< literal 0 HcmV?d00001 diff --git a/core/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala b/core/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala index 4088081761f..c18787474ad 100644 --- a/core/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala +++ b/core/src/test/scala/io/delta/sql/parser/DeltaSqlParserSuite.scala @@ -19,11 +19,10 @@ package io.delta.sql.parser import io.delta.tables.execution.VacuumTableCommand import org.apache.spark.sql.delta.CloneTableSQLTestUtils -import org.apache.spark.sql.delta.commands.OptimizeTableCommand - +import org.apache.spark.sql.delta.commands.{OptimizeTableCommand, DeltaReorgTable} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.{TableIdentifier, TimeTravel} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedTable} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -120,6 +119,60 @@ class DeltaSqlParserSuite extends SparkFunSuite with SQLHelper { Seq(unresolvedAttr("optimize"), unresolvedAttr("zorder")))) } + private def targetPlanForTable(tableParts: String*): UnresolvedTable = + UnresolvedTable(tableParts.toSeq, "REORG", relationTypeMismatchHint = None) + + test("REORG command is parsed as expected") { + val parser = new DeltaSqlParser(null) + + assert(parser.parsePlan("REORG TABLE tbl APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE tbl_${system:spark.testing} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl_true"))(Seq.empty)) + + withSQLConf("tbl_var" -> "tbl") { + assert(parser.parsePlan("REORG TABLE ${tbl_var} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE ${spark:tbl_var} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE ${sparkconf:tbl_var} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE ${hiveconf:tbl_var} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE ${hivevar:tbl_var} APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq.empty)) + } + + assert(parser.parsePlan("REORG TABLE delta.`/path/to/tbl` APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("delta", "/path/to/tbl"))(Seq.empty)) + + assert(parser.parsePlan("REORG TABLE tbl WHERE part = 1 APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq("part = 1"))) + } + + test("REORG command new tokens are non-reserved keywords") { + // new keywords: REORG, APPLY, PURGE + val parser = new DeltaSqlParser(null) + + // Use the new keywords in table name + assert(parser.parsePlan("REORG TABLE reorg APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("reorg"))(Seq.empty)) + assert(parser.parsePlan("REORG TABLE apply APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("apply"))(Seq.empty)) + assert(parser.parsePlan("REORG TABLE purge APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("purge"))(Seq.empty)) + + // Use the new keywords in column name + assert(parser.parsePlan( + "REORG TABLE tbl WHERE reorg = 1 AND apply = 2 AND purge = 3 APPLY (PURGE)") === + DeltaReorgTable(targetPlanForTable("tbl"))(Seq("reorg = 1 AND apply =2 AND purge = 3"))) + } + // scalastyle:off argcount private def checkCloneStmt( parser: DeltaSqlParser, diff --git a/core/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala b/core/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala index d7cc09fc828..1982546c512 100644 --- a/core/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala +++ b/core/src/test/scala/io/delta/tables/DeltaTableBuilderSuite.scala @@ -366,7 +366,8 @@ class DeltaTableBuilderSuite extends QueryTest with SharedSparkSession with Delt } assert(e.getMessage == "Database 'main.parquet' not found" || e.getMessage == "Database 'parquet' not found" || - e.getMessage.contains("is not a valid name") + e.getMessage.contains("is not a valid name") || + e.getMessage.contains("schema `parquet` cannot be found") ) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSQLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSQLSuite.scala index c0113a529a2..950cda8686e 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSQLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSQLSuite.scala @@ -16,6 +16,13 @@ package org.apache.spark.sql.delta +import scala.collection.immutable.NumericRange + +import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN +import org.apache.spark.sql.delta.actions.{AddFile, FileAction, RemoveFile} +import org.apache.spark.sql.delta.test.{DeltaExcludedTestMixin, DeltaSQLCommandTest} +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType @@ -119,12 +126,14 @@ class CloneTableSQLSuite extends CloneTableSuiteBase val ex = intercept[AnalysisException] { sql(s"CREATE TABLE delta.`$tblExt` SHALLOW CLONE not_exists") } - assert(ex.getMessage.contains("Table not found")) + assert(ex.getMessage.contains("Table not found") || + ex.getMessage.contains("The table or view `not_exists` cannot be found")) val ex2 = intercept[AnalysisException] { sql(s"CREATE TABLE delta.`$tblExt` SHALLOW CLONE not_exists VERSION AS OF 0") } - assert(ex2.getMessage.contains("Table not found")) + assert(ex2.getMessage.contains("Table not found") || + ex2.getMessage.contains("The table or view `not_exists` cannot be found")) } cloneTest("cloning a view") { (tblExt, _) => @@ -274,3 +283,183 @@ object CloneTableSQLTestUtils { } // scalastyle:on argcount } + +class CloneTableScalaDeletionVectorSuite + extends CloneTableSQLSuite + with DeltaSQLCommandTest + with DeltaExcludedTestMixin + with DeletionVectorsTestUtils { + + override def excluded: Seq[String] = super.excluded ++ + Seq( + // These require the initial table protocol version to be low to work properly. + "Cloning a table with new table properties that force protocol version upgrade -" + + " delta.enableChangeDataFeed" + , "Cloning a table with new table properties that force protocol version upgrade -" + + " delta.enableDeletionVectors" + , "Cloning a table without DV property should not upgrade protocol version" + , "CLONE respects table features set by table property override, targetExists=true" + , "CLONE ignores reader/writer session defaults") + + override def beforeAll(): Unit = { + super.beforeAll() + enableDeletionVectors(spark.conf) + } + + override protected def uniqueFileActionGroupBy(action: FileAction): String = { + val filePath = action.pathAsUri.toString + val dvId = action match { + case add: AddFile => Option(add.deletionVector).map(_.uniqueId).getOrElse("") + case remove: RemoveFile => Option(remove.deletionVector).map(_.uniqueId).getOrElse("") + case _ => "" + } + filePath + dvId + } + + testAllClones("Cloning table with persistent DVs") { (source, target, isShallow) => + // Create source table + writeMultiFileSourceTable( + source, + fileRanges = Seq(0L until 30L, 30L until 60L, 60L until 90L)) + // Add DVs to 2 files, leave 1 file without DVs. + spark.sql(s"DELETE FROM delta.`$source` WHERE id IN (24, 42)") + runAndValidateCloneWithDVs( + source, + target, + expectedNumFilesWithDVs = 2) + } + + testAllClones("Cloning table with persistent DVs and absolute parquet paths" + ) { (source, target, isShallow) => + withTempDir { originalSourceDir => + val originalSource = originalSourceDir.getCanonicalPath + // Create source table, by writing to an upstream table and then shallow cloning before + // adding DVs. + writeMultiFileSourceTable( + source = originalSource, + fileRanges = Seq(0L until 30L, 30L until 60L, 60L until 90L)) + spark.sql(s"CREATE OR REPLACE TABLE delta.`$source` SHALLOW CLONE delta.`$originalSource`") + // Add DVs to 2 files, leave 1 file without DVs. + spark.sql(s"DELETE FROM delta.`$source` WHERE id IN (24, 42)") + runAndValidateCloneWithDVs( + source, + target, + expectedNumFilesWithDVs = 2) + } + } + + testAllClones("Cloning table with persistent DVs and absolute DV file paths" + ) { (source, target, isShallow) => + withTempDir { originalSourceDir => + val originalSource = originalSourceDir.getCanonicalPath + // Create source table, by writing to an upstream table, adding DVs and then shallow cloning. + writeMultiFileSourceTable( + source = originalSource, + fileRanges = Seq(0L until 30L, 30L until 60L, 60L until 90L)) + // Add DVs to 2 files, leave 1 file without DVs. + spark.sql(s"DELETE FROM delta.`$originalSource` WHERE id IN (24, 42)") + val originalSourceTable = io.delta.tables.DeltaTable.forPath(spark, originalSource) + spark.sql(s"CREATE OR REPLACE TABLE delta.`$source` SHALLOW CLONE delta.`$originalSource`") + // Double check this clone was correct. + checkAnswer( + spark.read.format("delta").load(source), expectedAnswer = originalSourceTable.toDF) + runAndValidateCloneWithDVs( + source, + target, + expectedNumFilesWithDVs = 2) + } + } + + cloneTest("Shallow clone round-trip with DVs") { (source, target) => + // Create source table. + writeMultiFileSourceTable( + source = source, + fileRanges = Seq( + 0L until 30L, // file 1 + 30L until 60L, // file 2 + 60L until 90L, // file 3 + 90L until 120L)) // file 4 + // Add DVs to files 1 and 2 and then shallow clone. + spark.sql(s"DELETE FROM delta.`$source` WHERE id IN (24, 42)") + runAndValidateCloneWithDVs( + source = source, + target = target, + expectedNumFilesWithDVs = 2) + + // Add a new DV to file 3 and update the DV file 2, + // leaving file 4 without a DV and file 1 with the existing DV. + // Then shallow clone back into source. + spark.sql(s"DELETE FROM delta.`$target` WHERE id IN (43, 69)") + runAndValidateCloneWithDVs( + source = target, + target = source, + expectedNumFilesWithDVs = 3, + isReplaceOperation = true) + } + + /** Write one file per range in `fileRanges`. */ + private def writeMultiFileSourceTable( + source: String, + fileRanges: Seq[NumericRange.Exclusive[Long]]): Unit = { + for (range <- fileRanges) { + spark.range(start = range.start, end = range.end, step = 1L, numPartitions = 1).toDF("id") + .write.format("delta").mode("append").save(source) + } + } + + private def runAndValidateCloneWithDVs( + source: String, + target: String, + expectedNumFilesWithDVs: Int, + isReplaceOperation: Boolean = false): Unit = { + val sourceDeltaLog = DeltaLog.forTable(spark, source) + val targetDeltaLog = DeltaLog.forTable(spark, source) + val filesWithDVsInSource = getFilesWithDeletionVectors(sourceDeltaLog) + assert(filesWithDVsInSource.size === expectedNumFilesWithDVs) + val numberOfUniqueDVFilesInSource = filesWithDVsInSource + .map(_.deletionVector.pathOrInlineDv) + .toSet + .size + + runAndValidateClone( + source, + target, + isReplaceOperation = isReplaceOperation)() + val filesWithDVsInTarget = getFilesWithDeletionVectors(targetDeltaLog) + val numberOfUniqueDVFilesInTarget = filesWithDVsInTarget + .map(_.deletionVector.pathOrInlineDv) + .toSet + .size + // Make sure we didn't accidentally copy some file multiple times. + assert(numberOfUniqueDVFilesInSource === numberOfUniqueDVFilesInTarget) + // Check contents of the copied DV files. + val filesWithDVsInTargetByPath = filesWithDVsInTarget + .map(addFile => addFile.path -> addFile) + .toMap + // scalastyle:off deltahadoopconfiguration + val hadoopConf = spark.sessionState.newHadoopConf() + // scalastyle:on deltahadoopconfiguration + for (sourceFile <- filesWithDVsInSource) { + val targetFile = filesWithDVsInTargetByPath(sourceFile.path) + if (sourceFile.deletionVector.isInline) { + assert(targetFile.deletionVector.isInline) + assert(sourceFile.deletionVector.inlineData === targetFile.deletionVector.inlineData) + } else { + def readDVData(path: Path): Array[Byte] = { + val fs = path.getFileSystem(hadoopConf) + val size = fs.getFileStatus(path).getLen + val data = new Array[Byte](size.toInt) + Utils.tryWithResource(fs.open(path)) { reader => + reader.readFully(data) + } + data + } + val sourceDVPath = sourceFile.deletionVector.absolutePath(sourceDeltaLog.dataPath) + val targetDVPath = targetFile.deletionVector.absolutePath(targetDeltaLog.dataPath) + val sourceData = readDVData(sourceDVPath) + val targetData = readDVData(targetDVPath) + assert(sourceData === targetData) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSuiteBase.scala index 5194518e120..a108241074b 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/CloneTableSuiteBase.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, FileSystemBasedCheckpointFileManager, MemoryStream} import org.apache.spark.sql.functions.{col, floor, from_json} @@ -214,7 +215,8 @@ trait CloneTableSuiteBase extends QueryTest // scalastyle:on deltahadoopconfiguration val qualifiedSourcePath = fs.makeQualified(sourcePath) val logSource = if (sourceIsTable) { - s"default.$source".toLowerCase(Locale.ROOT) + val catalog = CatalogManager.SESSION_CATALOG_NAME + s"$catalog.default.$source".toLowerCase(Locale.ROOT) } else { s"$sourceFormat.`$qualifiedSourcePath`" } @@ -942,6 +944,77 @@ trait CloneTableSuiteBase extends QueryTest val targetLog = DeltaLog.forTable(spark, target) assert(targetLog.update().protocol.isFeatureSupported(TestWriterFeature)) } + + // Delta properties that automatically cause a version upgrade when enabled via ALTER TABLE. + final val featuresWithAutomaticProtocolUpgrade: Seq[DeltaConfig[Boolean]] = Seq( + DeltaConfigs.CHANGE_DATA_FEED, + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION + ) + // This test ensures this upgrade also happens when enabled during a CLONE. + for (feature <- featuresWithAutomaticProtocolUpgrade) + testAllClones("Cloning a table with new table properties" + + s" that force protocol version upgrade - ${feature.key}" + ) { (source, target, isShallow) => + import DeltaTestUtils.StrictProtocolOrdering + + spark.range(5).write.format("delta").save(source) + val sourceDeltaLog = DeltaLog.forTable(spark, source) + val sourceSnapshot = sourceDeltaLog.update() + // This only works if the feature is not enabled by default. + assert(!feature.fromMetaData(sourceSnapshot.metadata)) + // Check that the original version is not already sufficient for the feature. + assert(!StrictProtocolOrdering.fulfillsVersionRequirements( + actual = sourceSnapshot.protocol, + requirement = feature.minimumProtocolVersion.get + )) + + // Clone the table, enabling the feature in an override. + val tblProperties = Map(feature.key -> "true") + cloneTable( + source, + target, + isReplace = true, + tableProperties = tblProperties) + + val targetDeltaLog = DeltaLog.forTable(spark, target) + val targetSnapshot = targetDeltaLog.update() + assert(targetSnapshot.metadata.configuration === + tblProperties ++ sourceSnapshot.metadata.configuration) + // Check that the protocol has been upgraded. + assert(StrictProtocolOrdering.fulfillsVersionRequirements( + actual = targetSnapshot.protocol, + requirement = feature.minimumProtocolVersion.get + )) + } + + testAllClones("Cloning a table without DV property should not upgrade protocol version" + ) { (source, target, isShallow) => + import DeltaTestUtils.StrictProtocolOrdering + + spark.range(5).write.format("delta").save(source) + withSQLConf(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true") { + val sourceDeltaLog = DeltaLog.forTable(spark, source) + val sourceSnapshot = sourceDeltaLog.update() + // Should not be enabled, just because it's allowed. + assert(!DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.fromMetaData(sourceSnapshot.metadata)) + // Check that the original version is not already sufficient for the feature. + assert(!StrictProtocolOrdering.fulfillsVersionRequirements( + actual = sourceSnapshot.protocol, + requirement = DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.minimumProtocolVersion.get + )) + + // Clone the table. + cloneTable( + source, + target, + isReplace = true) + + val targetDeltaLog = DeltaLog.forTable(spark, target) + val targetSnapshot = targetDeltaLog.update() + // Protocol should not have been upgraded. + assert(sourceSnapshot.protocol === targetSnapshot.protocol) + } + } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala index d7309832d0a..fe9b6125e36 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -122,4 +122,5 @@ class DeleteSQLWithDeletionVectorsSuite extends DeleteSQLSuite text = "SELECT key, value, 1 FROM tab", expectResult = Row(0, 3, 1) :: Nil) } + } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala index 2fa789a883d..6cb03d7721e 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala @@ -47,34 +47,18 @@ class DeleteScalaSuite extends DeleteSuiteBase with DeltaSQLCommandTest { } override protected def executeDelete(target: String, where: String = null): Unit = { - - def parse(tableNameWithAlias: String): (String, Option[String]) = { - tableNameWithAlias.split(" ").toList match { - case tableName :: Nil => tableName -> None // just table name - case tableName :: alias :: Nil => // tablename SPACE alias OR tab SPACE lename - val ordinary = (('a' to 'z') ++ ('A' to 'Z') ++ ('0' to '9')).toSet - if (!alias.forall(ordinary.contains(_))) { - (tableName + " " + alias) -> None - } else { - tableName -> Some(alias) - } - case _ => - fail(s"Could not build parse '$tableNameWithAlias' for table and optional alias") + import DeltaTestUtils.TableIdentifierOrPath + val deltaTable: io.delta.tables.DeltaTable = + DeltaTestUtils.getTableIdentifierOrPath(target) match { + case TableIdentifierOrPath.Identifier(id, optionalAlias) => + val table = DeltaTableTestUtils.createTable( + spark.table(id.unquotedString), + DeltaLog.forTable(spark, id.unquotedString)) + optionalAlias.map(table.as(_)).getOrElse(table) + case TableIdentifierOrPath.Path(path, optionalAlias) => + val table = io.delta.tables.DeltaTable.forPath(spark, path) + optionalAlias.map(table.as(_)).getOrElse(table) } - } - - val deltaTable: io.delta.tables.DeltaTable = { - val (tableNameOrPath, optionalAlias) = parse(target) - val isPath: Boolean = tableNameOrPath.startsWith("delta.") - val table = if (isPath) { - val path = tableNameOrPath.stripPrefix("delta.`").stripSuffix("`") - io.delta.tables.DeltaTable.forPath(spark, path) - } else { - DeltaTableTestUtils.createTable(spark.table(tableNameOrPath), - DeltaLog.forTable(spark, tableNameOrPath)) - } - optionalAlias.map(table.as(_)).getOrElse(table) - } if (where != null) { deltaTable.delete(where) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala index 1802a782f50..c3b0c16efa9 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala @@ -394,11 +394,7 @@ abstract class DeleteSuiteBase extends QueryTest case f: FileSourceScanExec => f }) - // Currently nested schemas can't be pruned, but Spark 3.4 loosens some of the restrictions - // on non-determinstic expressions, and this should be pruned to just "nested STRUCT" - // after upgrading - // TODO: should be "nested STRUCT" after Spark 3.4. - assert(scans.head.schema == StructType.fromDDL("nested STRUCT")) + assert(scans.head.schema == StructType.fromDDL("nested STRUCT")) } /** @@ -528,8 +524,8 @@ abstract class DeleteSuiteBase extends QueryTest } testInvalidTempViews("subset cols")( text = "SELECT key FROM tab", - expectedErrorClassForSQLTempView = "MISSING_COLUMN", - expectedErrorClassForDataSetTempView = "MISSING_COLUMN" + expectedErrorClassForSQLTempView = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorClassForDataSetTempView = "UNRESOLVED_COLUMN.WITH_SUGGESTION" ) // Need to be able to override this, because it works in some configurations. diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeletionVectorsTestUtils.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeletionVectorsTestUtils.scala index 51265ad119c..ba01c377289 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeletionVectorsTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeletionVectorsTestUtils.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.delta.storage.dv.DeletionVectorStore import org.apache.spark.sql.delta.util.PathWithFileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{DataFrame, QueryTest, SparkSession} +import org.apache.spark.sql.{DataFrame, QueryTest, RuntimeConfig, SparkSession} +import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.sql.test.SharedSparkSession /** Collection of test utilities related with persistent Deletion Vectors. */ @@ -124,6 +125,10 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { } } + /** Enable persistent deletion vectors in new Delta tables. */ + def enableDeletionVectorsInNewTables(conf: RuntimeConfig): Unit = + conf.set(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, "true") + /** Enable persistent Deletion Vectors in a Delta table. */ def enableDeletionVectorsInTable(tablePath: Path, enable: Boolean): Unit = spark.sql( @@ -135,6 +140,12 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { def enableDeletionVectorsInTable(deltaLog: DeltaLog, enable: Boolean = true): Unit = enableDeletionVectorsInTable(deltaLog.dataPath, enable) + /** Enable persistent deletion vectors in new tables and DELETE DML commands. */ + def enableDeletionVectors(conf: RuntimeConfig): Unit = { + enableDeletionVectorsInNewTables(conf) + conf.set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") + } + // ======== HELPER METHODS TO WRITE DVs ========== /** Helper method to remove the specified rows in the given file using DVs */ protected def removeRowsFromFileUsingDV( @@ -161,6 +172,14 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { txn.commit(actions, Truncate()) } + protected def getFileActionsInLastVersion(log: DeltaLog): (Seq[AddFile], Seq[RemoveFile]) = { + val version = log.update().version + val allFiles = log.getChanges(version).toSeq.head._2 + val add = allFiles.collect { case a: AddFile => a } + val remove = allFiles.collect { case r: RemoveFile => r } + (add, remove) + } + protected def serializeRoaringBitmapArrayWithDefaultFormat( dv: RoaringBitmapArray): Array[Byte] = { val serializationFormat = RoaringBitmapArrayFormat.Portable @@ -182,6 +201,22 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { writeFileWithDVOnDisk(log, currentFile, dv) } + /** Name of the partition column used by [[createTestDF()]]. */ + val PARTITION_COL = "partitionColumn" + + def createTestDF( + start: Long, + end: Long, + numFiles: Int, + partitionColumn: Option[Int] = None): DataFrame = { + val df = spark.range(start, end, 1, numFiles).withColumn("v", col("id")) + if (partitionColumn.isEmpty) { + df + } else { + df.withColumn(PARTITION_COL, lit(partitionColumn.get)) + } + } + /** * Produce a new [[AddFile]] that will reference the `dv` in the log while storing it on-disk. * @@ -229,7 +264,8 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { cardinality = dv.cardinality, offset = Some(range.offset)) val (add, remove) = currentFile.removeRows( - dvData + dvData, + updateStats = true ) Seq(add, remove) } @@ -276,7 +312,8 @@ trait DeletionVectorsTestUtils extends QueryTest with SharedSparkSession { addFile: AddFile, dvDescriptor: DeletionVectorDescriptor): (AddFile, RemoveFile) = { addFile.removeRows( - dvDescriptor + dvDescriptor, + updateStats = true ) } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala index 63bb19feaeb..92572c650c2 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaCDCSQLSuite.scala @@ -275,16 +275,20 @@ class DeltaCDCSQLSuite extends DeltaCDCSuiteBase with DeltaColumnMappingTestUtil } } - test("resolution of complex expression should throw an error") { val tbl = "tbl" withTable(tbl) { spark.range(10).write.format("delta").saveAsTable(tbl) - val e = intercept[AnalysisException] { - sql(s"SELECT * FROM table_changes('$tbl', 0, id)") - } - assert(e.getErrorClass == "MISSING_COLUMN") - assert(e.getMessage.contains("Column 'id' does not exist")) + checkError( + exception = intercept[AnalysisException] { + sql(s"SELECT * FROM table_changes('$tbl', 0, id)") + }, + errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + parameters = Map("objectName" -> "`id`"), + queryContext = Array(ExpectedContext( + fragment = "id", + start = 38, + stop = 39))) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaColumnRenameSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaColumnRenameSuite.scala index 5fda09c508e..5cc50dede1f 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaColumnRenameSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaColumnRenameSuite.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.scalatest.GivenWhenThen -import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.types._ class DeltaColumnRenameSuite extends QueryTest @@ -415,4 +415,56 @@ class DeltaColumnRenameSuite extends QueryTest } } } + + /** + * Covers renaming a nested field using the ALTER TABLE command. + * @param initialColumnType Type of the single column used to create the initial test table. + * @param fieldToRename Old and new name of the field to rename. + * @param updatedColumnType Expected type of the single column after renaming the nested field. + */ + def testRenameNestedField(testName: String)( + initialColumnType: String, + fieldToRename: (String, String), + updatedColumnType: String): Unit = + testColumnMapping(s"ALTER TABLE RENAME COLUMN - nested $testName") { mode => + withTempDir { dir => + withTable("delta_test") { + sql( + s""" + |CREATE TABLE delta_test (data $initialColumnType) + |USING delta + |TBLPROPERTIES (${DeltaConfigs.COLUMN_MAPPING_MODE.key} = '${mode}') + |OPTIONS('path'='${dir.getCanonicalPath}')""".stripMargin) + + val expectedInitialType = initialColumnType.filterNot(_.isWhitespace) + val expectedUpdatedType = updatedColumnType.filterNot(_.isWhitespace) + val fieldName = s"data.${fieldToRename._1}" + + def columnType: DataFrame = + sql("DESCRIBE TABLE delta_test") + .filter("col_name = 'data'") + .select("data_type") + checkAnswer(columnType, Row(expectedInitialType)) + + sql(s"ALTER TABLE delta_test RENAME COLUMN $fieldName TO ${fieldToRename._2}") + checkAnswer(columnType, Row(expectedUpdatedType)) + } + } + } + + testRenameNestedField("struct in map key")( + initialColumnType = "map, int>", + fieldToRename = "key.b" -> "c", + updatedColumnType = "map, int>") + + testRenameNestedField("struct in map value")( + initialColumnType = "map>", + fieldToRename = "value.b" -> "c", + updatedColumnType = "map>") + + testRenameNestedField("struct in array")( + initialColumnType = "array>", + fieldToRename = "element.b" -> "c", + updatedColumnType = "array>") + } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala index 5ab2fbc38c1..875e99b4dd2 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv -import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException import org.apache.spark.sql.catalyst.catalog.CatalogUtils @@ -35,11 +35,6 @@ import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType class DeltaDDLSuite extends DeltaDDLTestBase with SharedSparkSession with DeltaSQLCommandTest { - override protected def verifyDescribeTable(tblName: String): Unit = { - val res = sql(s"DESCRIBE TABLE $tblName").collect() - assert(res.takeRight(2).map(_.getString(1)) === Seq("name", "dept")) - } - override protected def verifyNullabilityFailure(exception: AnalysisException): Unit = { exception.getMessage.contains("Cannot change nullable column to non-nullable") } @@ -93,7 +88,10 @@ class DeltaDDLNameColumnMappingSuite extends DeltaDDLSuite abstract class DeltaDDLTestBase extends QueryTest with SQLTestUtils { import testImplicits._ - protected def verifyDescribeTable(tblName: String): Unit + protected def verifyDescribeTable(tblName: String): Unit = { + val res = sql(s"DESCRIBE TABLE $tblName").collect() + assert(res.takeRight(2).map(_.getString(0)) === Seq("name", "dept")) + } protected def verifyNullabilityFailure(exception: AnalysisException): Unit @@ -333,6 +331,63 @@ abstract class DeltaDDLTestBase extends QueryTest with SQLTestUtils { } } + /** + * Covers adding and changing a nested field using the ALTER TABLE command. + * @param initialColumnType Type of the single column used to create the initial test table. + * @param fieldToAdd Tuple (name, type) of the nested field to add and change. + * @param updatedColumnType Expected type of the single column after adding the nested field. + */ + def testAlterTableNestedFields(testName: String)( + initialColumnType: String, + fieldToAdd: (String, String), + updatedColumnType: String): Unit = { + test(s"ALTER TABLE ADD/CHANGE COLUMNS - nested $testName") { + withTempDir { dir => + withTable("delta_test") { + sql( + s""" + |CREATE TABLE delta_test (data $initialColumnType) + |USING delta + |TBLPROPERTIES (${DeltaConfigs.COLUMN_MAPPING_MODE.key} = 'name') + |OPTIONS('path'='${dir.getCanonicalPath}')""".stripMargin) + + val expectedInitialType = initialColumnType.filterNot(_.isWhitespace) + val expectedUpdatedType = updatedColumnType.filterNot(_.isWhitespace) + val fieldName = s"data.${fieldToAdd._1}" + val fieldType = fieldToAdd._2 + + def columnType: DataFrame = + sql("DESCRIBE TABLE delta_test") + .where("col_name = 'data'") + .select("data_type") + checkAnswer(columnType, Row(expectedInitialType)) + + sql(s"ALTER TABLE delta_test ADD COLUMNS ($fieldName $fieldType)") + checkAnswer(columnType, Row(expectedUpdatedType)) + + sql(s"ALTER TABLE delta_test CHANGE COLUMN $fieldName TYPE $fieldType") + checkAnswer(columnType, Row(expectedUpdatedType)) + } + } + } + } + + testAlterTableNestedFields("struct in map key")( + initialColumnType = "map, int>", + fieldToAdd = "key.b" -> "string", + updatedColumnType = "map, int>") + + testAlterTableNestedFields("struct in map value")( + initialColumnType = "map>", + fieldToAdd = "value.b" -> "string", + updatedColumnType = "map>") + + testAlterTableNestedFields("struct in array")( + initialColumnType = "array>", + fieldToAdd = "element.b" -> "string", + updatedColumnType = "array>") + + test("ALTER TABLE CHANGE COLUMN with nullability change in struct type - not supported") { withTempDir { dir => withTable("delta_test") { diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala index f7336d5b7be..2b3943124d8 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDataFrameWriterV2Suite.scala @@ -62,7 +62,7 @@ trait OpenSourceDataFrameWriterV2Tests } protected def catalogPrefix: String = { - "" + s"${CatalogManager.SESSION_CATALOG_NAME}." } protected def getProperties(table: Table): Map[String, String] = { @@ -538,6 +538,12 @@ class DeltaDataFrameWriterV2Suite checkAnswer( spark.table(s"delta.`$location`"), Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + // allows missing columns + Seq(4L).toDF("id").writeTo(s"delta.`$location`").append() + checkAnswer( + spark.table(s"delta.`$location`"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"), Row(4L, null))) } test("Create: basic behavior by path") { diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala index 352ebdaab3e..752f493d769 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaDropColumnSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaTestImplicits._ import org.apache.spark.SparkConf -import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.types.{ArrayType, IntegerType, MapType, StringType, StructType} @@ -356,4 +356,56 @@ class DeltaDropColumnSuite extends QueryTest } } + + /** + * Covers dropping a nested field using the ALTER TABLE command. + * @param initialColumnType Type of the single column used to create the initial test table. + * @param fieldToDrop Name of the field to drop from the initial column type. + * @param updatedColumnType Expected type of the single column after dropping the nested field. + */ + def testDropNestedField(testName: String)( + initialColumnType: String, + fieldToDrop: String, + updatedColumnType: String): Unit = + testColumnMapping(s"ALTER TABLE DROP COLUMNS - nested $testName") { mode => + withTempDir { dir => + withTable("delta_test") { + sql( + s""" + |CREATE TABLE delta_test (data $initialColumnType) + |USING delta + |TBLPROPERTIES (${DeltaConfigs.COLUMN_MAPPING_MODE.key} = '$mode') + |OPTIONS('path'='${dir.getCanonicalPath}')""".stripMargin) + + val expectedInitialType = initialColumnType.filterNot(_.isWhitespace) + val expectedUpdatedType = updatedColumnType.filterNot(_.isWhitespace) + val fieldName = s"data.${fieldToDrop}" + + def columnType: DataFrame = + sql("DESCRIBE TABLE delta_test") + .filter("col_name = 'data'") + .select("data_type") + checkAnswer(columnType, Row(expectedInitialType)) + + sql(s"ALTER TABLE delta_test DROP COLUMNS ($fieldName)") + checkAnswer(columnType, Row(expectedUpdatedType)) + } + } + } + + testDropNestedField("struct in map key")( + initialColumnType = "map, int>", + fieldToDrop = "key.b", + updatedColumnType = "map, int>") + + testDropNestedField("struct in map value")( + initialColumnType = "map>", + fieldToDrop = "value.b", + updatedColumnType = "map>") + + testDropNestedField("struct in array")( + initialColumnType = "array>", + fieldToDrop = "element.b", + updatedColumnType = "array>") + } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala index 1b969f10104..0031a524f6a 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala @@ -2660,14 +2660,12 @@ trait DeltaErrorsSuiteBase assert(e.getMessage == "operation1 is only supported for Delta tables.") } { - val invalidStruct = StructField("invalid1", StringType) val e = intercept[DeltaAnalysisException] { - throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(invalidStruct) + throw DeltaErrors.dropNestedColumnsFromNonStructTypeException(StringType) } assert(e.getErrorClass == "DELTA_UNSUPPORTED_DROP_NESTED_COLUMN_FROM_NON_STRUCT_TYPE") assert(e.getSqlState == "0AKDC") - assert(e.getMessage == - "Can only drop nested columns from StructType. Found StructField(invalid1,StringType,true)") + assert(e.getMessage == s"Can only drop nested columns from StructType. Found $StringType") } { val columnsThatNeedRename = Set("c0", "c1") diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala index c3cab478af0..6710a18096e 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala @@ -600,10 +600,8 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest subClass = ExistingDeletionVectorsWithIncrementalManifestGeneration) { setEnabledIncrementalManifest(tablePath, enabled = true) } - // Run optimize to delete the DVs and rewrite the data files - withSQLConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO.key -> "0.00001") { - spark.sql(s"OPTIMIZE delta.`$tablePath`") - } + // Purge + spark.sql(s"REORG TABLE delta.`$tablePath` APPLY (PURGE)") assert(getFilesWithDeletionVectors(deltaLog).isEmpty) // Now it should work. setEnabledIncrementalManifest(tablePath, enabled = true) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaImplicitsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaImplicitsSuite.scala index feb7f626884..c9df8c9fd37 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaImplicitsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaImplicitsSuite.scala @@ -45,6 +45,7 @@ class DeltaImplicitsSuite extends SparkFunSuite with SharedSparkSession { testImplict("serializableFileStatus", serializableFileStatusEncoder) testImplict("indexedFile", indexedFileEncoder) testImplict("addFileWithIndex", addFileWithIndexEncoder) + testImplict("addFileWithSourcePath", addFileWithSourcePathEncoder) testImplict("deltaHistoryEncoder", deltaHistoryEncoder) testImplict("historyCommitEncoder", historyCommitEncoder) testImplict("snapshotStateEncoder", snapshotStateEncoder) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala index 2ee16f67ccb..3c7159d19aa 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala @@ -76,6 +76,105 @@ class DeltaInsertIntoSQLSuite } } + test("insertInto: append by name") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + sql(s"INSERT INTO $t1(id, data) VALUES(1L, 'a')") + // Can be in a different order + sql(s"INSERT INTO $t1(data, id) VALUES('b', 2L)") + // Can be casted automatically + sql(s"INSERT INTO $t1(data, id) VALUES('c', 3)") + verifyTable(t1, df) + withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") { + // Missing columns + assert(intercept[AnalysisException] { + sql(s"INSERT INTO $t1(data) VALUES(4)") + }.getMessage.contains("Column id is not specified in INSERT")) + // Missing columns with matching dataType + assert(intercept[AnalysisException] { + sql(s"INSERT INTO $t1(data) VALUES('b')") + }.getMessage.contains("Column id is not specified in INSERT")) + } + // Duplicate columns + assert(intercept[AnalysisException]( + sql(s"INSERT INTO $t1(data, data) VALUES(5)")).getMessage.nonEmpty) + } + } + + test("insertInto: overwrite by name") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + sql(s"INSERT OVERWRITE $t1(id, data) VALUES(1L, 'a')") + verifyTable(t1, Seq((1L, "a")).toDF("id", "data")) + // Can be in a different order + sql(s"INSERT OVERWRITE $t1(data, id) VALUES('b', 2L)") + verifyTable(t1, Seq((2L, "b")).toDF("id", "data")) + // Can be casted automatically + sql(s"INSERT OVERWRITE $t1(data, id) VALUES('c', 3)") + verifyTable(t1, Seq((3L, "c")).toDF("id", "data")) + withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") { + // Missing columns + assert(intercept[AnalysisException] { + sql(s"INSERT OVERWRITE $t1(data) VALUES(4)") + }.getMessage.contains("Column id is not specified in INSERT")) + // Missing columns with matching datatype + assert(intercept[AnalysisException] { + sql(s"INSERT OVERWRITE $t1(data) VALUES(4L)") + }.getMessage.contains("Column id is not specified in INSERT")) + } + // Duplicate columns + assert(intercept[AnalysisException]( + sql(s"INSERT OVERWRITE $t1(data, data) VALUES(5)")).getMessage.nonEmpty) + } + } + + dynamicOverwriteTest("insertInto: dynamic overwrite by name") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string, data2 string) " + + s"USING $v2Format PARTITIONED BY (id)") + sql(s"INSERT OVERWRITE $t1(id, data, data2) VALUES(1L, 'a', 'b')") + verifyTable(t1, Seq((1L, "a", "b")).toDF("id", "data", "data2")) + // Can be in a different order + sql(s"INSERT OVERWRITE $t1(data, data2, id) VALUES('b', 'd', 2L)") + verifyTable(t1, Seq((1L, "a", "b"), (2L, "b", "d")).toDF("id", "data", "data2")) + // Can be casted automatically + sql(s"INSERT OVERWRITE $t1(data, data2, id) VALUES('c', 'e', 1)") + verifyTable(t1, Seq((1L, "c", "e"), (2L, "b", "d")).toDF("id", "data", "data2")) + withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") { + // Missing columns + assert(intercept[AnalysisException] { + sql(s"INSERT OVERWRITE $t1(data, id) VALUES('c', 1)") + }.getMessage.contains("Column data2 is not specified in INSERT")) + // Missing columns with matching datatype + assert(intercept[AnalysisException] { + sql(s"INSERT OVERWRITE $t1(data, id) VALUES('c', 1L)") + }.getMessage.contains("Column data2 is not specified in INSERT")) + } + // Duplicate columns + assert(intercept[AnalysisException]( + sql(s"INSERT OVERWRITE $t1(data, data) VALUES(5)")).getMessage.nonEmpty) + } + } + + test("insertInto: static partition column name should not be used in the column list") { + withTable("t") { + sql(s"CREATE TABLE t(i STRING, c string) USING $v2Format PARTITIONED BY (c)") + checkError( + exception = intercept[AnalysisException] { + sql("INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')") + }, + errorClass = "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST", + parameters = Map("staticName" -> "c")) + } + } + Seq(("ordinal", ""), ("name", "(id, col2, col)")).foreach { case (testName, values) => test(s"INSERT OVERWRITE schema evolution works for array struct types - $testName") { @@ -309,7 +408,8 @@ abstract class DeltaInsertIntoTestsWithTempViews( assert(e.getMessage.contains("Inserting into a view is not allowed") || e.getMessage.contains("Inserting into an RDD-based table is not allowed") || e.getMessage.contains("Table default.v not found") || - e.getMessage.contains("Table or view 'v' not found in database 'default'")) + e.getMessage.contains("Table or view 'v' not found in database 'default'") || + e.getMessage.contains("The table or view `default`.`v` cannot be found")) } } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala index 0b13bd8d6f8..26a4a6e9cdd 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala @@ -340,7 +340,7 @@ class DeltaLogSuite extends QueryTest val ex = intercept[IllegalStateException] { staleLog.update() } - assert(ex.getMessage === "Versions (Vector(0, 2)) are not contiguous.") + assert(ex.getMessage.contains("Versions (Vector(0, 2)) are not contiguous.")) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala index 93fb87aeaf0..03c0e575070 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala @@ -19,11 +19,10 @@ package org.apache.spark.sql.delta import java.util.Locale // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.DeltaOptions.PARTITION_OVERWRITE_MODE_OPTION +import org.apache.spark.sql.delta.DeltaOptions.{OVERWRITE_SCHEMA_OPTION, PARTITION_OVERWRITE_MODE_OPTION} import org.apache.spark.sql.delta.actions.{Action, FileAction} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import org.apache.spark.sql.delta.test.DeltaTestImplicits._ import org.apache.spark.sql.delta.util.FileNames import org.apache.commons.io.FileUtils import org.apache.parquet.format.CompressionCodec @@ -283,4 +282,23 @@ class DeltaOptionSuite extends QueryTest } } } + + test("overwriteSchema=true should be invalid with partitionOverwriteMode=dynamic") { + withTempDir { tempDir => + val e = intercept[DeltaIllegalArgumentException] { + withSQLConf(DeltaSQLConf.DYNAMIC_PARTITION_OVERWRITE_ENABLED.key -> "true") { + Seq(1, 2, 3).toDF + .withColumn("part", $"value" % 2) + .write + .mode("overwrite") + .format("delta") + .partitionBy("part") + .option(OVERWRITE_SCHEMA_OPTION, "true") + .option(PARTITION_OVERWRITE_MODE_OPTION, "dynamic") + .save(tempDir.getAbsolutePath) + } + } + assert(e.getErrorClass == "DELTA_OVERWRITE_SCHEMA_WITH_DYNAMIC_PARTITION_OVERWRITE") + } + } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala index dd14d2b3ced..7fd4a804a52 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -3077,4 +3077,172 @@ class DeltaNameColumnMappingSuite extends DeltaSuite insertedDF.filter(col("id") >= 6).union(otherDF)) } } + + test("replaceWhere SQL - partition column - dynamic filter") { + withTempDir { dir => + // create partitioned table + spark.range(100).withColumn("part", 'id % 10) + .write + .format("delta") + .partitionBy("part") + .save(dir.toString) + + // ans will be used to replace the entire contents of the table + val ans = spark.range(10) + .withColumn("part", lit(0)) + + ans.createOrReplaceTempView("replace") + sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE part >=0 SELECT * FROM replace") + checkAnswer(spark.read.format("delta").load(dir.toString), ans) + } + } + + test("replaceWhere SQL - partition column - static filter") { + withTable("tbl") { + // create partitioned table + spark.range(100).withColumn("part", lit(0)) + .write + .format("delta") + .partitionBy("part") + .saveAsTable("tbl") + + val partEq1DF = spark.range(10, 20) + .withColumn("part", lit(1)) + partEq1DF.write.format("delta").mode("append").saveAsTable("tbl") + + + val replacer = spark.range(10) + .withColumn("part", lit(0)) + + replacer.createOrReplaceTempView("replace") + sql(s"INSERT INTO tbl REPLACE WHERE part=0 SELECT * FROM replace") + checkAnswer(spark.read.format("delta").table("tbl"), replacer.union(partEq1DF)) + } + } + + test("replaceWhere SQL - data column - dynamic") { + withTable("tbl") { + // write table + spark.range(100).withColumn("col", lit(1)) + .write + .format("delta") + .saveAsTable("tbl") + + val colGt2DF = spark.range(100, 200) + .withColumn("col", lit(3)) + + colGt2DF.write + .format("delta") + .mode("append") + .saveAsTable("tbl") + + val replacer = spark.range(10) + .withColumn("col", lit(1)) + + replacer.createOrReplaceTempView("replace") + sql(s"INSERT INTO tbl REPLACE WHERE col < 2 SELECT * FROM replace") + checkAnswer( + spark.read.format("delta").table("tbl"), + replacer.union(colGt2DF) + ) + } + } + + test("replaceWhere SQL - data column - static") { + withTempDir { dir => + // write table + spark.range(100).withColumn("col", lit(2)) + .write + .format("delta") + .save(dir.toString) + + val colEq2DF = spark.range(100, 200) + .withColumn("col", lit(1)) + + colEq2DF.write + .format("delta") + .mode("append") + .save(dir.toString) + + val replacer = spark.range(10) + .withColumn("col", lit(2)) + + replacer.createOrReplaceTempView("replace") + sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE col = 2 SELECT * FROM replace") + checkAnswer( + spark.read.format("delta").load(dir.toString), + replacer.union(colEq2DF) + ) + } + } + + test("replaceWhere SQL - multiple predicates - static") { + withTempDir { dir => + // write table + spark.range(100).withColumn("col", lit(2)) + .write + .format("delta") + .save(dir.toString) + + spark.range(100, 200).withColumn("col", lit(5)) + .write + .format("delta") + .mode("append") + .save(dir.toString) + + val colEq2DF = spark.range(100, 200) + .withColumn("col", lit(1)) + + colEq2DF.write + .format("delta") + .mode("append") + .save(dir.toString) + + val replacer = spark.range(10) + .withColumn("col", lit(2)) + + replacer.createOrReplaceTempView("replace") + sql(s"INSERT INTO delta.`${dir.toString}` REPLACE WHERE col = 2 OR col = 5 " + + s"SELECT * FROM replace") + checkAnswer( + spark.read.format("delta").load(dir.toString), + replacer.union(colEq2DF) + ) + } + } + + test("replaceWhere with less than predicate") { + withTempDir { dir => + val insertedDF = spark.range(10).toDF() + + insertedDF.write.format("delta").save(dir.toString) + + val otherDF = spark.range(start = 0, end = 4).toDF() + otherDF.write.format("delta").mode("overwrite") + .option(DeltaOptions.REPLACE_WHERE_OPTION, "id < 6") + .save(dir.toString) + checkAnswer(spark.read.load(dir.toString), + insertedDF.filter(col("id") >= 6).union(otherDF)) + } + } + + test("replaceWhere SQL with less than predicate") { + withTempDir { dir => + val insertedDF = spark.range(10).toDF() + + insertedDF.write.format("delta").save(dir.toString) + + val otherDF = spark.range(start = 0, end = 4).toDF() + otherDF.createOrReplaceTempView("replace") + + sql( + s""" + |INSERT INTO delta.`${dir.getAbsolutePath}` + |REPLACE WHERE id < 6 + |SELECT * FROM replace + |""".stripMargin) + checkAnswer(spark.read.load(dir.toString), + insertedDF.filter(col("id") >= 6).union(otherDF)) + } + } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala index d95295f57db..9c659853eee 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaTableCreationTests.scala @@ -2022,8 +2022,7 @@ class DeltaTableCreationSuite checkAnswer( sql(s"COMMENT ON TABLE $emptyTableName IS 'My Empty Cool Table'"), Nil) - var answer = 3 - assert(sql(s"DESCRIBE TABLE $emptyTableName").collect().length == answer) + assert(sql(s"DESCRIBE TABLE $emptyTableName").collect().length == 0) // create table, alter tbl property, tbl comment assert(sql(s"DESCRIBE HISTORY $emptyTableName").collect().length == 3) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala index 5060464422f..eccd1af42a7 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala @@ -16,22 +16,27 @@ package org.apache.spark.sql.delta +import java.util.Locale import java.util.concurrent.atomic.AtomicInteger +import scala.util.matching.Regex + import org.apache.spark.sql.delta.DeltaTestUtils.Plans +import org.apache.spark.sql.delta.actions.Protocol import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.spark.SparkContext import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution, RDDScanExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.util.QueryExecutionListener -import org.apache.spark.util.Utils trait DeltaTestUtilsBase { + import DeltaTestUtils.TableIdentifierOrPath final val BOOLEAN_DOMAIN: Seq[Boolean] = Seq(true, false) @@ -169,9 +174,37 @@ trait DeltaTestUtilsBase { case _ => false }.head } + + /** + * Separate name- from path-based SQL table identifiers. + */ + def getTableIdentifierOrPath(sqlIdentifier: String): TableIdentifierOrPath = { + // Match: delta.`path`[ alias] or tahoe.`path`[ alias] + val pathMatcher: Regex = raw"(?:delta|tahoe)\.`([^`]+)`(?: (.+))?".r + // Match: db.table[ alias] + val qualifiedDbMatcher: Regex = raw"`?([^\.` ]+)`?\.`?([^\.` ]+)`?(?: (.+))?".r + // Match: table[ alias] + val unqualifiedNameMatcher: Regex = raw"([^ ]+)(?: (.+))?".r + sqlIdentifier match { + case pathMatcher(path, alias) => + TableIdentifierOrPath.Path(path, Option(alias)) + case qualifiedDbMatcher(dbName, tableName, alias) => + TableIdentifierOrPath.Identifier(TableIdentifier(tableName, Some(dbName)), Option(alias)) + case unqualifiedNameMatcher(tableName, alias) => + TableIdentifierOrPath.Identifier(TableIdentifier(tableName), Option(alias)) + } + } } object DeltaTestUtils extends DeltaTestUtilsBase { + + sealed trait TableIdentifierOrPath + object TableIdentifierOrPath { + case class Identifier(id: TableIdentifier, alias: Option[String]) + extends TableIdentifierOrPath + case class Path(path: String, alias: Option[String]) extends TableIdentifierOrPath + } + case class Plans( analyzed: LogicalPlan, optimized: LogicalPlan, @@ -203,6 +236,37 @@ object DeltaTestUtils extends DeltaTestUtilsBase { case tableName => tableName -> None } } + + /** + * Implements an ordering where `x < y` iff both reader and writer versions of + * `x` are strictly less than those of `y`. + * + * Can be used to conveniently check that this relationship holds in tests/assertions + * without having to write out the conjunction of the two subconditions every time. + */ + case object StrictProtocolOrdering extends PartialOrdering[Protocol] { + override def tryCompare(x: Protocol, y: Protocol): Option[Int] = { + if (x.minReaderVersion == y.minReaderVersion && + x.minWriterVersion == y.minWriterVersion) { + Some(0) + } else if (x.minReaderVersion < y.minReaderVersion && + x.minWriterVersion < y.minWriterVersion) { + Some(-1) + } else if (x.minReaderVersion > y.minReaderVersion && + x.minWriterVersion > y.minWriterVersion) { + Some(1) + } else { + None + } + } + + override def lteq(x: Protocol, y: Protocol): Boolean = + x.minReaderVersion <= y.minReaderVersion && x.minWriterVersion <= y.minWriterVersion + + // Just a more readable version of `lteq`. + def fulfillsVersionRequirements(actual: Protocol, requirement: Protocol): Boolean = + lteq(requirement, actual) + } } trait DeltaTestUtilsForTempViews @@ -260,6 +324,10 @@ trait DeltaTestUtilsForTempViews } } + protected def errorContains(errMsg: String, str: String): Unit = { + assert(errMsg.toLowerCase(Locale.ROOT).contains(str.toLowerCase(Locale.ROOT))) + } + def testErrorMessageAndClass( isSQLTempView: Boolean, ex: AnalysisException, @@ -269,14 +337,14 @@ trait DeltaTestUtilsForTempViews expectedErrorClassForDataSetTempView: String = null): Unit = { if (isSQLTempView) { if (expectedErrorMsgForSQLTempView != null) { - assert(ex.getMessage.contains(expectedErrorMsgForSQLTempView)) + errorContains(ex.getMessage, expectedErrorMsgForSQLTempView) } if (expectedErrorClassForSQLTempView != null) { assert(ex.getErrorClass == expectedErrorClassForSQLTempView) } } else { if (expectedErrorMsgForDataSetTempView != null) { - assert(ex.getMessage.contains(expectedErrorMsgForDataSetTempView)) + errorContains(ex.getMessage, expectedErrorMsgForDataSetTempView) } if (expectedErrorClassForDataSetTempView != null) { assert(ex.getErrorClass == expectedErrorClassForDataSetTempView, ex.getMessage) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaThrowableSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaThrowableSuite.scala index bf112e53eaa..6f30d4a0435 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaThrowableSuite.scala @@ -22,7 +22,7 @@ import java.nio.file.Files import scala.collection.immutable.SortedMap -import org.apache.spark.sql.delta.DeltaThrowableHelper.{deltaErrorClassSource, deltaErrorClassToInfoMap, sparkErrorClassesMap} +import org.apache.spark.sql.delta.DeltaThrowableHelper.{deltaErrorClassSource, sparkErrorClassSource} import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.core.JsonParser.Feature.STRICT_DUPLICATE_DETECTION import com.fasterxml.jackson.core.`type`.TypeReference @@ -32,11 +32,19 @@ import com.fasterxml.jackson.databind.json.JsonMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.commons.io.{FileUtils, IOUtils} -import org.apache.spark.{ErrorInfo, SparkFunSuite} +import org.apache.spark.{ErrorClassesJsonReader, ErrorInfo, SparkFunSuite} /** Test suite for Delta Throwables. */ class DeltaThrowableSuite extends SparkFunSuite { + private lazy val sparkErrorClassesMap = { + new ErrorClassesJsonReader(Seq(sparkErrorClassSource)).errorInfoMap + } + + private lazy val deltaErrorClassToInfoMap = { + new ErrorClassesJsonReader(Seq(deltaErrorClassSource)).errorInfoMap + } + /* Used to regenerate the error class file. Run: {{{ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt \ @@ -44,8 +52,6 @@ class DeltaThrowableSuite extends SparkFunSuite { }}} */ - private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - def checkIfUnique(ss: Seq[Any]): Unit = { val duplicatedKeys = ss.groupBy(identity).mapValues(_.size).filter(_._2 > 1).keys.toSeq assert(duplicatedKeys.isEmpty) @@ -108,8 +114,15 @@ class DeltaThrowableSuite extends SparkFunSuite { } test("Delta message format invariants") { - val messageFormats = - deltaErrorClassToInfoMap.values.toSeq.map(_.messageFormat) + val messageFormats = deltaErrorClassToInfoMap.values.toSeq.flatMap { i => + i.subClass match { + // Has sub error class: the message template should be: base + sub + case Some(subs) => + subs.values.toSeq.map(sub => s"${i.messageTemplate} ${sub.messageTemplate}") + // Does not have any sub error class: the message template is itself + case None => Seq(i.messageTemplate) + } + } checkCondition(messageFormats, s => s != null) checkIfUnique(messageFormats) } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala index 2843153fc20..e32a0e73139 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala @@ -38,6 +38,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric @@ -198,7 +199,8 @@ trait DeltaVacuumSuiteBase extends QueryTest "numCopiedRows" -> createMetric(sparkContext, "total number of rows.") ) txn.registerSQLMetrics(spark, metrics) - txn.commit(Seq(RemoveFile(path, Option(clock.getTimeMillis()))), Delete("true" :: Nil)) + txn.commit(Seq(RemoveFile(path, Option(clock.getTimeMillis()))), + Delete(Seq(Literal.TrueLiteral))) // scalastyle:on case e: ExecuteVacuumInSQL => Given(s"*** Executing SQL: ${e.sql}") @@ -831,12 +833,10 @@ class DeltaVacuumSuite // Helper method to remove the DVs in Delta table and rewrite the data files def purgeDVs(tableName: String): Unit = { withSQLConf( - DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO.key -> "0.0001", - DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key -> "2", // Set the max file size to low so that we always rewrite the single file without DVs // and not combining with other data files. DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE.key -> "2") { - spark.sql(s"OPTIMIZE $tableName") + spark.sql(s"REORG TABLE $tableName APPLY (PURGE)") } } @@ -861,12 +861,14 @@ class DeltaVacuumSuite assertNumFiles(deltaLog, addFiles = 5, addFilesWithDVs = 5, dvFiles = 1, dataFiles = 5) // Delete all rows from the first file. An ephemeral DV will still be created. + Thread.sleep(1000) // Ensure it's been at least 1000 ms since V1 spark.sql(s"DELETE FROM $tableName WHERE id < 10") val timestampV2 = deltaLog.update().timestamp assertNumFiles(deltaLog, addFiles = 4, addFilesWithDVs = 4, dvFiles = 2, dataFiles = 5) val expectedAnswerV2 = Seq.range(0, 50).filterNot(deletedRows1.contains).filterNot(_ < 10) // Delete 1 more row from each file. + Thread.sleep(1000) // Ensure it's been at least 1000 ms since V2 val deletedRows2 = Seq(11, 21, 31, 41) val deletedRowsStr2 = deletedRows2.mkString("(", ",", ")") spark.sql(s"DELETE FROM $tableName WHERE id IN $deletedRowsStr2") @@ -875,6 +877,7 @@ class DeltaVacuumSuite val expectedAnswerV3 = expectedAnswerV2.filterNot(deletedRows2.contains) // Delete DVs by rewriting the data files with DVs. + Thread.sleep(1000) // Ensure it's been at least 1000 ms since V3 purgeDVs(tableName) val numFilesAfterPurge = 4 @@ -889,13 +892,14 @@ class DeltaVacuumSuite dataFiles = 9) // Run VACUUM @ V1. - clock.setTime(timestampV1 + TimeUnit.HOURS.toMillis(1)) + // We need to add 1000 ms for local filesystems that only write modificationTimes to the s + clock.setTime(timestampV1 + TimeUnit.HOURS.toMillis(1) + 1000) VacuumCommand.gc(spark, deltaLog, retentionHours = Some(1), clock = clock, dryRun = false) assertNumFiles(deltaLog, addFiles = numFilesAfterPurge, addFilesWithDVs = 0, dvFiles = 3, dataFiles = 9) // Run VACUUM @ V2. It should delete the ephemeral DV and the removed Parquet file. - clock.setTime(timestampV2 + TimeUnit.HOURS.toMillis(1)) + clock.setTime(timestampV2 + TimeUnit.HOURS.toMillis(1) + 1000) VacuumCommand.gc(spark, deltaLog, retentionHours = Some(1), clock = clock, dryRun = false) assertNumFiles(deltaLog, addFiles = numFilesAfterPurge, addFilesWithDVs = 0, dvFiles = 2, dataFiles = 8) @@ -903,7 +907,7 @@ class DeltaVacuumSuite spark.sql(s"SELECT * FROM $tableName VERSION AS OF 2"), expectedAnswerV2.toDF) // Run VACUUM @ V3. It should delete the persistent DVs from V1. - clock.setTime(timestampV3 + TimeUnit.HOURS.toMillis(1)) + clock.setTime(timestampV3 + TimeUnit.HOURS.toMillis(1) + 1000) VacuumCommand.gc(spark, deltaLog, retentionHours = Some(1), clock = clock, dryRun = false) assertNumFiles(deltaLog, addFiles = numFilesAfterPurge, addFilesWithDVs = 0, dvFiles = 1, dataFiles = 8) @@ -911,7 +915,7 @@ class DeltaVacuumSuite spark.sql(s"SELECT * FROM $tableName VERSION AS OF 3"), expectedAnswerV3.toDF) // Run VACUUM @ V4. It should delete the Parquet files and DVs of V3. - clock.setTime(timestampV4 + TimeUnit.HOURS.toMillis(1)) + clock.setTime(timestampV4 + TimeUnit.HOURS.toMillis(1) + 1000) VacuumCommand.gc(spark, deltaLog, retentionHours = Some(1), clock = clock, dryRun = false) assertNumFiles(deltaLog, addFiles = numFilesAfterPurge, addFilesWithDVs = 0, dvFiles = 0, dataFiles = 4) @@ -919,7 +923,7 @@ class DeltaVacuumSuite spark.sql(s"SELECT * FROM $tableName VERSION AS OF 4"), expectedAnswerV3.toDF) // Run VACUUM with zero retention period. It should not delete anything. - clock.setTime(timestampV4 + TimeUnit.HOURS.toMillis(1)) + clock.setTime(timestampV4 + TimeUnit.HOURS.toMillis(1) + 1000) VacuumCommand.gc(spark, deltaLog, retentionHours = Some(0), clock = clock, dryRun = false) assertNumFiles(deltaLog, addFiles = numFilesAfterPurge, addFilesWithDVs = 0, dvFiles = 0, dataFiles = 4) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala index 11110b98830..158164dc851 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala @@ -37,8 +37,7 @@ trait DescribeDeltaDetailSuiteBase extends QueryTest import testImplicits._ val catalogAndSchema = { - var res = "default." - res + s"$SESSION_CATALOG_NAME.default." } protected def checkResult( diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala index bd079ba5d6b..4e8f1c0e6e8 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala @@ -65,11 +65,23 @@ trait DescribeDeltaHistorySuiteBase protected def checkLastOperation( basePath: String, expected: Seq[String], - columns: Seq[Column] = Seq($"operation", $"operationParameters.mode")): Unit = { - val df = io.delta.tables.DeltaTable.forPath(spark, basePath).history(1) - checkAnswer(df.select(columns: _*), Seq(Row(expected: _*))) - val df2 = spark.sql(s"DESCRIBE HISTORY delta.`$basePath` LIMIT 1") - checkAnswer(df2.select(columns: _*), Seq(Row(expected: _*))) + columns: Seq[Column] = Seq($"operation", $"operationParameters.mode"), + removeExpressionId: Boolean = false): Unit = { + var df = io.delta.tables.DeltaTable.forPath(spark, basePath).history(1) + df = df.select(columns: _*) + if (removeExpressionId) { + // As the expression ID is written as part of the column predicate (in the form of col#expId) + // but it is non-deterministic, we remove it here so that any comparison can just go against + // the column name + df = df.withColumn("predicate", regexp_replace(col("predicate"), "#[0-9]+", "")) + } + checkAnswer(df, Seq(Row(expected: _*))) + df = spark.sql(s"DESCRIBE HISTORY delta.`$basePath` LIMIT 1") + df = df.select(columns: _*) + if (removeExpressionId) { + df = df.withColumn("predicate", regexp_replace(col("predicate"), "#[0-9]+", "")) + } + checkAnswer(df, Seq(Row(expected: _*))) } protected def checkOperationMetrics( @@ -487,7 +499,7 @@ trait DescribeDeltaHistorySuiteBase checkLastOperation( tempDir, Seq("DELETE", """["(id = 1)"]"""), - Seq($"operation", $"operationParameters.predicate")) + Seq($"operation", $"operationParameters.predicate"), removeExpressionId = true) } testWithFlag("old and new writers") { diff --git a/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala index 9a4c31c1bee..d5f33c47b7e 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala @@ -209,6 +209,14 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest { assert(errMsg.contains(str)) } + protected def testTableUpdateDPO( + testName: String)(updateFunc: (String, String) => Seq[Row]): Unit = { + withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> + SQLConf.PartitionOverwriteMode.DYNAMIC.toString) { + testTableUpdate("dpo_" + testName)(updateFunc) + } + } + testTableUpdate("append_data") { (table, path) => Seq( Tuple5(1L, "foo", "2020-10-11 12:30:30", 100, "2020-11-12") @@ -269,6 +277,26 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest { 100, 1000, sqlDate("2020-11-12")) :: Nil } + testTableUpdate("insert_into_by_name_provide_all_columns") { (table, _) => + sql(s"INSERT INTO $table (c5, c6, c7_g_p, c8, c1, c2_g, c3_p, c4_g_p) VALUES" + + s"('2020-10-11 12:30:30', 100, 1000, '2020-11-12', 1, 11, 'foo', '2020-10-11')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdate("insert_into_by_name_not_provide_generated_columns") { (table, _) => + sql(s"INSERT INTO $table (c6, c8, c1, c3_p, c5) VALUES" + + s"(100, '2020-11-12', 1L, 'foo', '2020-10-11 12:30:30')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdate("insert_into_by_name_with_some_generated_columns") { (table, _) => + sql(s"INSERT INTO $table (c5, c6, c8, c1, c3_p, c4_g_p) VALUES" + + s"('2020-10-11 12:30:30', 100, '2020-11-12', 1L, 'foo', '2020-10-11')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } testTableUpdate("insert_into_select_provide_all_columns") { (table, path) => sql(s"INSERT INTO $table SELECT " + @@ -277,6 +305,17 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest { 100, 1000, sqlDate("2020-11-12")) :: Nil } + testTableUpdate("insert_into_by_name_not_provide_normal_columns") { (table, _) => + val e = intercept[AnalysisException] { + withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") { + sql(s"INSERT INTO $table (c6, c8, c1, c3_p) VALUES" + + s"(100, '2020-11-12', 1L, 'foo')") + } + } + errorContains(e.getMessage, "Column c5 is not specified in INSERT") + Nil + } + testTableUpdate("insert_overwrite_values_provide_all_columns") { (table, path) => sql(s"INSERT OVERWRITE TABLE $table VALUES" + s"(1, 11, 'foo', '2020-10-11', '2020-10-11 12:30:30', 100, 1000, '2020-11-12')") @@ -291,27 +330,84 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest { 100, 1000, sqlDate("2020-11-12")) :: Nil } + testTableUpdate("insert_overwrite_by_name_provide_all_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c5, c6, c7_g_p, c8, c1, c2_g, c3_p, c4_g_p) VALUES" + + s"('2020-10-11 12:30:30', 100, 1000, '2020-11-12', 1, 11, 'foo', '2020-10-11')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } - testTableUpdate("dpo_insert_overwrite_values_provide_all_columns") { (table, path) => - withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> - SQLConf.PartitionOverwriteMode.DYNAMIC.toString) { - sql(s"INSERT OVERWRITE TABLE $table VALUES" + - s"(1, 11, 'foo', '2020-10-11', '2020-10-11 12:30:30', 100, 1000, '2020-11-12')") - } + testTableUpdate("insert_overwrite_by_name_not_provide_generated_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c6, c8, c1, c3_p, c5) VALUES" + + s"(100, '2020-11-12', 1L, 'foo', '2020-10-11 12:30:30')") Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), 100, 1000, sqlDate("2020-11-12")) :: Nil } - testTableUpdate("dpo_insert_overwrite_select_provide_all_columns") { (table, path) => - withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> - SQLConf.PartitionOverwriteMode.DYNAMIC.toString) { - sql(s"INSERT OVERWRITE TABLE $table SELECT " + - s"1, 11, 'foo', '2020-10-11', '2020-10-11 12:30:30', 100, 1000, '2020-11-12'") + testTableUpdate("insert_overwrite_by_name_with_some_generated_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c5, c6, c8, c1, c3_p, c4_g_p) VALUES" + + s"('2020-10-11 12:30:30', 100, '2020-11-12', 1L, 'foo', '2020-10-11')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdate("insert_overwrite_by_name_not_provide_normal_columns") { (table, _) => + val e = intercept[AnalysisException] { + withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") { + sql(s"INSERT OVERWRITE $table (c6, c8, c1, c3_p) VALUES" + + s"(100, '2020-11-12', 1L, 'foo')") + } } + errorContains(e.getMessage, "Column c5 is not specified in INSERT") + Nil + } + + testTableUpdateDPO("insert_overwrite_values_provide_all_columns") { (table, path) => + sql(s"INSERT OVERWRITE TABLE $table VALUES" + + s"(1, 11, 'foo', '2020-10-11', '2020-10-11 12:30:30', 100, 1000, '2020-11-12')") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdateDPO("insert_overwrite_select_provide_all_columns") { (table, path) => + sql(s"INSERT OVERWRITE TABLE $table SELECT " + + s"1, 11, 'foo', '2020-10-11', '2020-10-11 12:30:30', 100, 1000, '2020-11-12'") Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), 100, 1000, sqlDate("2020-11-12")) :: Nil } + testTableUpdateDPO("insert_overwrite_by_name_values_provide_all_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c5, c6, c7_g_p, c8, c1, c2_g, c3_p, c4_g_p) VALUES" + + s"(CAST('2020-10-11 12:30:30' AS TIMESTAMP), 100, 1000, CAST('2020-11-12' AS DATE), " + + s"1L, 11L, 'foo', CAST('2020-10-11' AS DATE))") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdateDPO( + "insert_overwrite_by_name_not_provide_generated_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c6, c8, c1, c3_p, c5) VALUES" + + s"(100, CAST('2020-11-12' AS DATE), 1L, 'foo', CAST('2020-10-11 12:30:30' AS TIMESTAMP))") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdateDPO("insert_overwrite_by_name_with_some_generated_columns") { (table, _) => + sql(s"INSERT OVERWRITE $table (c5, c6, c8, c1, c3_p, c4_g_p) VALUES" + + s"(CAST('2020-10-11 12:30:30' AS TIMESTAMP), 100, CAST('2020-11-12' AS DATE), 1L, " + + s"'foo', CAST('2020-10-11' AS DATE))") + Row(1L, 11L, "foo", sqlDate("2020-10-11"), sqlTimestamp("2020-10-11 12:30:30"), + 100, 1000, sqlDate("2020-11-12")) :: Nil + } + + testTableUpdateDPO("insert_overwrite_by_name_not_provide_normal_columns") { (table, _) => + val e = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE $table (c6, c8, c1, c3_p) VALUES" + + s"(100, '2020-11-12', 1L, 'foo')") + } + assert(e.getMessage.contains("Column c5 is not specified in INSERT")) + Nil + } testTableUpdate("delete") { (table, path) => Seq( diff --git a/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala index 480b9c1f9e1..89ca0e842d4 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala @@ -26,11 +26,6 @@ abstract class HiveDeltaDDLSuiteBase extends DeltaDDLTestBase { import testImplicits._ - override protected def verifyDescribeTable(tblName: String): Unit = { - val res = sql(s"DESCRIBE TABLE $tblName").collect() - assert(res.takeRight(2).map(_.getString(1)) === Seq("name", "dept")) - } - override protected def verifyNullabilityFailure(exception: AnalysisException): Unit = { exception.getMessage.contains("not supported for changing column") } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala index 1c84cb9726c..dcc1c0502d7 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala @@ -78,27 +78,11 @@ class MergeIntoAccumulatorSuite } test("accumulators used by MERGE should not fail Spark event log generation") { - // SPARK-39489: must use reflection to shim over API changes in JsonProtocol between - // Spark 3.3 and 3.4: - val writeEvent: SparkListenerEvent => Unit = { - val jsonProtocolClass = JsonProtocol.getClass - val method = try { - // Spark 3.4+: - jsonProtocolClass.getDeclaredMethod("sparkEventToJsonString", classOf[SparkListenerEvent]) - } catch { - case _: NoSuchMethodException => - // Spark 3.3 and earlier: - jsonProtocolClass.getDeclaredMethod("sparkEventToJson", classOf[SparkListenerEvent]) - } - (e: SparkListenerEvent) => method.invoke(JsonProtocol, e) - } - // Check that the reflection-based invocation works properly: - writeEvent(SparkListenerNodeExcluded(0, "host", 1)) // Register a listener to convert `SparkListenerTaskEnd` to json and catch failures. val failure = new AtomicReference[Throwable]() val listener = new SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - try writeEvent(taskEnd) catch { + try JsonProtocol.sparkEventToJsonString(taskEnd) catch { case t: Throwable => failure.compareAndSet(null, t) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoMaterializeSourceSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoMaterializeSourceSuite.scala index 7b9ee00d662..b7627dafdb7 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoMaterializeSourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoMaterializeSourceSuite.scala @@ -76,6 +76,7 @@ trait MergeIntoMaterializeSourceTests } + test("merge logs out of disk errors") { val injectEx = new java.io.IOException("No space left on device") testWithCustomErrorInjected[SparkException](injectEx) { (thrownEx, errorOpt) => @@ -407,99 +408,6 @@ trait MergeIntoMaterializeSourceTests } } - // FIXME: Tests can be removed once Delta adopts Spark 3.4 as constraints and statistics are - // automatically propagated when materializing - // The following test should fail as soon as statistics are correctly propagated, and acts as a - // reminder to remove the manually added filter and broadcast hint once Spark 3.4 is adopted - test("Source in materialized merge has missing stats") { - // AQE has to be disabled as we might not find the Join in the adaptive plan - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - withTable("A", "T") { - sql("select id, id as v from range(50)").write.format("delta").saveAsTable("T") - sql("select id, id+2 as v from range(10)").write.format("csv").saveAsTable("A") - val plans = DeltaTestUtils.withAllPlansCaptured(spark) { - sql("MERGE INTO T USING A as s ON T.id = s.id" + - " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *") - } - plans.map(_.optimized).foreach { p => - p.foreach { - case j: Join => - // The source is very small, the only way we'd be above the broadcast join threshold - // is if we lost statistics on the size of the source. - val sourceStats = j.left.stats.sizeInBytes - val broadcastJoinThreshold = spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD) - assert(sourceStats >= broadcastJoinThreshold) - case _ => - } - } - } - } - } - - test("Filter gets added if there is a constraint") { - // AQE has to be disabled as we might not find the filter in the adaptive plan - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - withTable("A", "T") { - spark.range(50).toDF("tgtid").write.format("delta").saveAsTable("T") - spark.range(50).toDF("srcid").write.format("delta").saveAsTable("A") - - val plans = DeltaTestUtils.withAllPlansCaptured(spark) { - sql("MERGE INTO T USING (SELECT * FROM A WHERE srcid = 10) as s ON T.tgtid = s.srcid" + - " WHEN MATCHED THEN UPDATE SET tgtid = s.srcid" + - " WHEN NOT MATCHED THEN INSERT (tgtid) values (s.srcid)") - } - // Check whether the executed plan contains a filter that filters by tgtId that could be - // used to infer constraints lost during materialization - val hastgtIdCondition = (condition: Expression) => { - condition.find { - case EqualTo(AttributeReference("tgtid", _, _, _), Literal(10, _)) => true - case _ => false - }.isDefined - } - val touchedFilesPlan = getfindTouchedFilesJobPlans(plans) - val filter = touchedFilesPlan.find { - case f: FilterExec => hastgtIdCondition(f.condition) - case _ => false - } - assert(filter.isDefined, - s"Didn't find Filter on tgtid=10 in touched files plan:\n$touchedFilesPlan") - } - } - } - - test("Broadcast hint gets added when there is a small source table") { - withTable("A", "T") { - sql("select id, id as v from range(50000)").write.format("delta").saveAsTable("T") - sql("select id, id+2 as v from range(10000)").write.format("csv").saveAsTable("A") - val hints = getHints( - sql("MERGE INTO T USING A as s ON T.id = s.id" + - " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *") - ) - hints.foreach { case (hints, joinHint) => - assert(hints.length == 1) - assert(hints.head.hints == HintInfo(strategy = Some(BROADCAST))) - assert(joinHint == JoinHint(Some(HintInfo(strategy = Some(BROADCAST))), None)) - } - } - } - - test("Broadcast hint does not get added when there is a large table") { - withTable("A", "T") { - sql("select id, id as v from range(50000)").write.format("delta").saveAsTable("T") - sql("select id, id+2 as v from range(10000)").write.format("csv").saveAsTable("A") - withSQLConf((SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1KB")) { - val hints = getHints( - sql("MERGE INTO T USING A as s ON T.id = s.id" + - " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *") - ) - hints.foreach { case (hints, joinHint) => - assert(hints.length == 0) - assert(joinHint == JoinHint(None, None)) - } - } - } - } - test("materialize source for non-deterministic source formats") { val targetSchema = StructType(Array( StructField("id", IntegerType, nullable = false), diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala index 4ee4caca3d1..e4ee68b0f03 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala @@ -30,7 +30,8 @@ import org.apache.spark.sql.functions.udf import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest +class MergeIntoSQLSuite extends MergeIntoSuiteBase with MergeIntoNotMatchedBySourceSuite + with DeltaSQLCommandTest with DeltaTestUtilsForTempViews { import testImplicits._ @@ -204,7 +205,8 @@ class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest |WHEN NOT MATCHED THEN | INSERT (trgValue, trgKey) VALUES (srcValue, srcKey) """.stripMargin)) - assert(e.getMessage.contains("only the last NOT MATCHED clause can omit the condition")) + assert(e.getMessage.contains( + "only the last NOT MATCHED [BY TARGET] clause can omit the condition")) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala index efadd80e388..60561b461ea 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala @@ -590,10 +590,6 @@ abstract class MergeIntoSuiteBase } } - protected def errorContains(errMsg: String, str: String): Unit = { - assert(errMsg.toLowerCase(Locale.ROOT).contains(str.toLowerCase(Locale.ROOT))) - } - def errorNotContains(errMsg: String, str: String): Unit = { assert(!errMsg.toLowerCase(Locale.ROOT).contains(str.toLowerCase(Locale.ROOT))) } @@ -2553,7 +2549,7 @@ abstract class MergeIntoSuiteBase executeMerge(s"delta.`$tempPath` t", s"source s", "s.key = t.key", clauses.toSeq: _*) } - assert(ex.getMessage.contains(expectErrorContains)) + errorContains(ex.getMessage, expectErrorContains) } else { executeMerge(s"delta.`$tempPath` t", s"source s", "s.key = t.key", clauses.toSeq: _*) @@ -4702,8 +4698,9 @@ abstract class MergeIntoSuiteBase update(condition = "s.key == 3", set = "key = s.key, value = 2 * srcValue"), insert(condition = null, values = "(key, value) VALUES (s.key, srcValue)"), insert(condition = null, values = "(key, value) VALUES (s.key, 1 + srcValue)"))( - errorStrs = "when there are more than one not matched clauses in a merge statement, " + - "only the last not matched clause can omit the condition" :: Nil) + errorStrs = "when there are more than one not matched" :: + "clauses in a merge statement, only the last not matched" :: + "clause can omit the condition" :: Nil) testAnalysisErrorsInUnlimitedClauses("error on multiple update clauses without condition")( mergeOn = "s.key = t.key", @@ -4738,8 +4735,9 @@ abstract class MergeIntoSuiteBase update(condition = null, set = "key = s.key, value = srcValue"), insert(condition = null, values = "(key, value) VALUES (s.key, srcValue)"), insert(condition = "s.key < 3", values = "(key, value) VALUES (s.key, 1 + srcValue)"))( - errorStrs = "when there are more than one not matched clauses in a merge statement, " + - "only the last not matched clause can omit the condition" :: Nil) + errorStrs = "when there are more than one not matched" :: + "clauses in a merge statement, only the last not matched" :: + "clause can omit the condition" :: Nil) /* end unlimited number of merge clauses tests */ @@ -5171,8 +5169,8 @@ abstract class MergeIntoSuiteBase testInvalidTempViews("subset cols")( text = "SELECT key FROM tab", - expectedErrorMsgForSQLTempView = "cannot resolve", - expectedErrorMsgForDataSetTempView = "cannot resolve" + expectedErrorMsgForSQLTempView = "cannot resolve v.value", + expectedErrorMsgForDataSetTempView = "cannot resolve v.value" ) testInvalidTempViews("superset cols")( diff --git a/core/src/test/scala/org/apache/spark/sql/delta/RestoreTableScalaSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/RestoreTableScalaSuite.scala index 13ad18fb304..624b7f1098e 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/RestoreTableScalaSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/RestoreTableScalaSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine +import com.databricks.spark.util.Log4jUsageLogger +import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN +import org.apache.spark.sql.delta.commands.DeletionVectorUtils import org.apache.spark.sql.delta.commands.cdc.CDCReader -import org.apache.spark.sql.delta.test.DeltaTestImplicits._ +import org.apache.spark.sql.delta.test.DeltaExcludedTestMixin import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.util.Utils @@ -55,3 +58,215 @@ class RestoreTableScalaSuite extends RestoreTableSuiteBase { } } +class RestoreTableScalaDeletionVectorSuite + extends RestoreTableScalaSuite + with DeletionVectorsTestUtils + with DeltaExcludedTestMixin { + + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + enableDeletionVectors(spark.conf) + } + override def excluded: Seq[String] = super.excluded ++ + Seq( + // These tests perform a delete to produce a file to vacuum, but with persistent DVs enabled, + // we actually just add a DV to the file instead, so there's no unreferenced file for vacuum. + "restore after vacuum", + "restore after vacuum - cloned table", + // These rely on the new-table protocol version to be lower than the latest, + // but this isn't true for DVs. + "restore downgrade protocol (allowed=true)", + "restore downgrade protocol (allowed=false)", + "restore downgrade protocol with table features (allowed=true)", + "restore downgrade protocol with table features (allowed=false)", + "cdf + RESTORE with write amplification reduction", + "RESTORE doesn't account for session defaults" + ) + + case class RestoreAndCheckArgs(versionToRestore: Int, expectedResult: DataFrame) + type RestoreAndCheckFunction = RestoreAndCheckArgs => Unit + + /** + * Tests `testFun` once by restoring to version and once to timestamp. + * + * `testFun` is expected to perform setup before executing the `RestoreAndTestFunction` and + * cleanup afterwards. + */ + protected def testRestoreByTimestampAndVersion + (testName: String) + (testFun: (String, RestoreAndCheckFunction) => Unit): Unit = { + for (restoreToVersion <- BOOLEAN_DOMAIN) { + val restoringTo = if (restoreToVersion) "version" else "timestamp" + test(testName + s" - restoring to $restoringTo") { + withTempDir{ dir => + val path = dir.toString + val restoreAndCheck: RestoreAndCheckFunction = (args: RestoreAndCheckArgs) => { + val deltaLog = DeltaLog.forTable(spark, path) + if (restoreToVersion) { + restoreTableToVersion(path, args.versionToRestore, isTable = false) + } else { + // Set a custom timestamp for the commit + val desiredDateS = "1996-01-12" + setTimestampToCommitFileAtVersion( + deltaLog, + version = args.versionToRestore, + date = desiredDateS) + // Set all previous versions to something lower, so we don't error out. + for (version <- 0 until args.versionToRestore) { + val previousDateS = "1996-01-11" + setTimestampToCommitFileAtVersion( + deltaLog, + version = version, + date = previousDateS) + } + + restoreTableToTimestamp(path, desiredDateS, isTable = false) + } + checkAnswer(spark.read.format("delta").load(path), args.expectedResult) + } + testFun(path, restoreAndCheck) + } + } + } + } + + testRestoreByTimestampAndVersion( + "Restoring table with persistent DVs to version without DVs") { (path, restoreAndCheck) => + val deltaLog = DeltaLog.forTable(spark, path) + val df1 = Seq(1, 2, 3, 4, 5).toDF("id") + val values2 = Seq(6, 7, 8, 9, 10) + val df2 = values2.toDF("id") + + // Write all values into version 0. + df1.union(df2).coalesce(1).write.format("delta").save(path) // version 0 + checkAnswer(spark.read.format("delta").load(path), expectedAnswer = df1.union(df2)) + val snapshotV0 = deltaLog.update() + assert(snapshotV0.version === 0) + + // Delete values 2 so that version 1 is `df1`. + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") // version 1 + assert(getFilesWithDeletionVectors(deltaLog).size > 0) + checkAnswer(spark.read.format("delta").load(path), expectedAnswer = df1) + val snapshotV1 = deltaLog.snapshot + assert(snapshotV1.version === 1) + + restoreAndCheck(RestoreAndCheckArgs(versionToRestore = 0, expectedResult = df1.union(df2))) + assert(getFilesWithDeletionVectors(deltaLog).size === 0) + } + + testRestoreByTimestampAndVersion( + "Restoring table with persistent DVs to version with DVs") { (path, restoreAndCheck) => + val deltaLog = DeltaLog.forTable(spark, path) + val df1 = Seq(1, 2, 3, 4, 5).toDF("id") + val values2 = Seq(6, 7) + val df2 = values2.toDF("id") + val values3 = Seq(8, 9, 10) + val df3 = values3.toDF("id") + + // Write all values into version 0. + df1.union(df2).union(df3).coalesce(1).write.format("delta").save(path) // version 0 + + // Delete values 2 and 3 in reverse order, so that version 1 is `df1.union(df2)`. + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values3.mkString(", ")})") // version 1 + assert(getFilesWithDeletionVectors(deltaLog).size > 0) + checkAnswer(spark.read.format("delta").load(path), expectedAnswer = df1.union(df2)) + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") // version 2 + assert(getFilesWithDeletionVectors(deltaLog).size > 0) + + restoreAndCheck(RestoreAndCheckArgs(versionToRestore = 1, expectedResult = df1.union(df2))) + assert(getFilesWithDeletionVectors(deltaLog).size > 0) + } + + testRestoreByTimestampAndVersion("Restoring table with persistent DVs to version " + + "without persistent DVs enabled") { (path, restoreAndCheck) => + withSQLConf( + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "false", + // Disable the log clean up. Tests sets the timestamp on commit files to long back + // in time that triggers the commit file clean up as part of the [[MetadataCleanup]] + DeltaConfigs.ENABLE_EXPIRED_LOG_CLEANUP.defaultTablePropertyKey -> "false") { + val deltaLog = DeltaLog.forTable(spark, path) + val df1 = Seq(1, 2, 3, 4, 5).toDF("id") + val values2 = Seq(6, 7, 8, 9, 10) + val df2 = values2.toDF("id") + + // Write all values into version 0. + df1.union(df2).coalesce(1).write.format("delta").save(path) // version 0 + checkAnswer(spark.read.format("delta").load(path), expectedAnswer = df1.union(df2)) + val snapshotV0 = deltaLog.update() + assert(snapshotV0.version === 0) + assert(!DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.fromMetaData(snapshotV0.metadata)) + + // Upgrade to us DVs + spark.sql(s"ALTER TABLE delta.`$path` SET TBLPROPERTIES " + + s"(${DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key} = true)") + val snapshotV1 = deltaLog.update() + assert(snapshotV1.version === 1) + assert(DeletionVectorUtils.deletionVectorsReadable(snapshotV1)) + assert(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.fromMetaData(snapshotV1.metadata)) + + // Delete values 2 so that version 1 is `df1`. + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") // version 2 + assert(getFilesWithDeletionVectors(deltaLog).size > 0) + checkAnswer(spark.read.format("delta").load(path), expectedAnswer = df1) + val snapshotV2 = deltaLog.update() + assert(snapshotV2.version === 2) + + // Restore to before the version upgrade. Protocol version should be retained (to make the + // history readable), but DV creation should be disabled again. + restoreAndCheck(RestoreAndCheckArgs(versionToRestore = 0, expectedResult = df1.union(df2))) + val snapshotV3 = deltaLog.update() + assert(getFilesWithDeletionVectors(deltaLog).size === 0) + assert(DeletionVectorUtils.deletionVectorsReadable(snapshotV3)) + assert(!DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.fromMetaData(snapshotV3.metadata)) + // Check that we can still read versions that did have DVs. + checkAnswer( + spark.read.format("delta").option("versionAsOf", "2").load(path), + expectedAnswer = df1) + } + } + test("CDF + DV + RESTORE") { + withSQLConf(DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true") { + withTempDir { tempDir => + val df0 = Seq(0, 1).toDF("id") // version 0 = [0, 1] + df0.write.format("delta").save(tempDir.getAbsolutePath) + + val df1 = Seq(2).toDF("id") // version 1: append to df0 = [0, 1, 2] + df1.write.mode("append").format("delta").save(tempDir.getAbsolutePath) + + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tempDir.getAbsolutePath) + deltaTable.delete("id < 1") // version 2: delete (0) = [1, 2] + + deltaTable.updateExpr( + "id > 1", + Map("id" -> "4") + ) // version 3: update 2 --> 4 = [1, 4] + + // version 4: restore to version 2 (delete 4, insert 2) = [1, 2] + restoreTableToVersion(tempDir.getAbsolutePath, 2, false) + checkAnswer( + CDCReader.changesToBatchDF(DeltaLog.forTable(spark, tempDir), 4, 4, spark) + .drop(CDCReader.CDC_COMMIT_TIMESTAMP), + Row(4, "delete", 4) :: Row(2, "insert", 4) :: Nil + ) + + // version 5: restore to version 1 (insert 0) = [0, 1, 2] + restoreTableToVersion(tempDir.getAbsolutePath, 1, false) + checkAnswer( + CDCReader.changesToBatchDF(DeltaLog.forTable(spark, tempDir), 5, 5, spark) + .drop(CDCReader.CDC_COMMIT_TIMESTAMP), + Row(0, "insert", 5) :: Nil + ) + + // version 6: restore to version 0 (delete 2) = [0, 1] + restoreTableToVersion(tempDir.getAbsolutePath, 0, false) + checkAnswer( + CDCReader.changesToBatchDF(DeltaLog.forTable(spark, tempDir), 6, 6, spark) + .drop(CDCReader.CDC_COMMIT_TIMESTAMP), + Row(2, "delete", 6) :: Nil + ) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/TightBoundsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/TightBoundsSuite.scala new file mode 100644 index 00000000000..7a9bca8d6b0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/sql/delta/TightBoundsSuite.scala @@ -0,0 +1,246 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import scala.collection.mutable.ArrayBuffer + +// scalastyle:off import.ordering.noEmptyLine +import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaStatistics.{MIN, NULL_COUNT, NUM_RECORDS, TIGHT_BOUNDS} +import org.apache.spark.sql.delta.stats.StatisticsCollection +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest +import org.apache.spark.sql.delta.test.DeltaTestImplicits._ + +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.functions.{col, lit, map_values, when} +import org.apache.spark.sql.test.SharedSparkSession + +class TightBoundsSuite + extends QueryTest + with SharedSparkSession + with DeletionVectorsTestUtils + with DeltaSQLCommandTest { + + override def beforeAll(): Unit = { + super.beforeAll() + enableDeletionVectors(spark.conf) + } + + test("Validate TIGHT_BOUND column") { + val targetDF = createTestDF(0, 100, 2) + val sourceDF = targetDF + + def runDelete(target: io.delta.tables.DeltaTable): Int = { + target.delete("id >= 75") + 2 // Expected number of files. + } + + val operations = ArrayBuffer[io.delta.tables.DeltaTable => Int](runDelete) + for { + // Make sure it works for all operations that add DVs + runOperation <- operations + // Make sure tightBounds update is backwards compatible + tightBoundDisabled <- BOOLEAN_DOMAIN + } { + val conf = Seq( + DeltaSQLConf.TIGHT_BOUND_COLUMN_ON_FILE_INIT_DISABLED.key -> tightBoundDisabled.toString) + + withSQLConf(conf: _*) { + withTempDeltaTable(targetDF) { (targetTable, targetLog) => + val snapshotBeforeOperation = targetLog.update() + val statsColumnName = snapshotBeforeOperation.getBaseStatsColumnName + val tightBoundsValuesBeforeOperation = snapshotBeforeOperation.withStatsDeduplicated + .select(col(s"${statsColumnName}.$TIGHT_BOUNDS")) + .collect() + + assert(tightBoundsValuesBeforeOperation.length === 2) + val expectedTightBoundsValue = if (tightBoundDisabled) "[null]" else "[true]" + tightBoundsValuesBeforeOperation + .foreach(r => assert(r.toString == expectedTightBoundsValue)) + + val expectedNumberOfFiles = runOperation(targetTable()) + // All operations only touch the second file. + assert(getFilesWithDeletionVectors(targetLog).size == 1) + + val snapshotAfterOperation = targetLog.update() + val tightBoundsValuesAfterOperation = snapshotAfterOperation.withStatsDeduplicated + // Order by returns non-null DVs last. Thus, the file with the wide bounds + // should be the last one. + .orderBy(col("deletionVector").asc_nulls_first) + .select(col(s"${statsColumnName}.$TIGHT_BOUNDS")) + .collect() + + // Make sure tightsBounds is generated even for files that initially + // did not contain the column. Note, we expect 2 files each from merge and delete + // operations and three from update. This is because update creates a new file for the + // updated rows. + assert(tightBoundsValuesAfterOperation.length === expectedNumberOfFiles) + assert(tightBoundsValuesAfterOperation.head.toString === expectedTightBoundsValue) + assert(tightBoundsValuesAfterOperation.last.toString === "[false]") + } + } + } + } + + test("Verify exception is thrown if we commit files with DVs and tight bounds") { + val targetDF = createTestDF(0, 100, 2) + withTempDeltaTable(targetDF, enableDVs = true) { (targetTable, targetLog) => + // Remove one record from each file. + targetTable().delete("id in (0, 50)") + verifyDVsExist(targetLog, 2) + + // Commit actions with DVs and tight bounds. + val txn = targetLog.startTransaction() + val addFiles = txn.snapshot.allFiles.collect().toSeq.map { action => + action.copy(stats = + s"""{"${NUM_RECORDS}":${action.numPhysicalRecords.get}, + | "${TIGHT_BOUNDS}":true}""".stripMargin) + } + + val exception = intercept[DeltaIllegalStateException] { + txn.commitManually(addFiles: _*) + } + assert(exception.getErrorClass === + "DELTA_ADDING_DELETION_VECTORS_WITH_TIGHT_BOUNDS_DISALLOWED") + } + } + + protected def getStatFromLastFile(snapshot: Snapshot, statName: String): Row = { + val statsColumnName = snapshot.getBaseStatsColumnName + snapshot + .withStatsDeduplicated + .select(s"$statsColumnName.$statName") + .orderBy(s"$statsColumnName.$MIN") + .collect() + .last + } + + protected def getStatFromLastFileWithDVs(snapshot: Snapshot, statName: String): Row = { + val statsColumnName = snapshot.getBaseStatsColumnName + snapshot + .withStatsDeduplicated + .filter("isNotNull(deletionVector)") + .select(s"$statsColumnName.$statName") + .collect() + .last + } + + /** + * Helper method that returns stats for every file in the snapshot as row objects. + * + * Return value schema is { + * numRecords: Int, + * RminValues: Row(Int, Int, ...), // Min value for each column + * maxValues: Row(Int, Int, ...), // Max value for each column + * nullCount: Row(Int, Int, ...), // Null count for each column + * tightBounds: boolean + * } + */ + protected def getStatsInPartitionOrder(snapshot: Snapshot): Array[Row] = { + val statsColumnName = snapshot.getBaseStatsColumnName + snapshot + .withStatsDeduplicated + .orderBy(map_values(col("partitionValues"))) + .select(s"$statsColumnName.*") + .collect() + } + + protected def getNullCountFromFirstFileWithDVs(snapshot: Snapshot): Row = { + // Note, struct columns in Spark are returned with datatype Row. + getStatFromLastFile(snapshot, NULL_COUNT) + .getAs[Row](NULL_COUNT) + } + + test("NULL COUNT is updated correctly when all values are nulls" + ) { + val targetDF = spark.range(0, 100, 1, 2) + .withColumn("value", when(col("id") < 25, col("id")) + .otherwise(null)) + + withTempDeltaTable(targetDF, enableDVs = true) { (targetTable, targetLog) => + targetTable().delete("id >= 80") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === Row(0, 50)) + + targetTable().delete("id >= 70") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === Row(0, 50)) + } + } + + test("NULL COUNT is updated correctly where there are no nulls" + ) { + val targetDF = spark.range(0, 100, 1, 2) + .withColumn("value", col("id")) + + withTempDeltaTable(targetDF, enableDVs = true) { (targetTable, targetLog) => + val expectedResult = Row(0, 0) + targetTable().delete("id >= 80") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === expectedResult) + + targetTable().delete("id >= 70") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === expectedResult) + } + } + + test("NULL COUNT is updated correctly when some values are nulls" + ) { + val targetDF = spark.range(0, 100, 1, 2) + .withColumn("value", when(col("id") < 75, col("id")) + .otherwise(null)) + + withTempDeltaTable(targetDF, enableDVs = true) { (targetTable, targetLog) => + targetTable().delete("id >= 80") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === Row(0, 25)) + + targetTable().delete("id >= 70") + assert(getNullCountFromFirstFileWithDVs(targetLog.update()) === Row(0, 25)) + } + } + + test("DML operations fetch stats on tables with partial stats") { + val targetDF = createTestDF(0, 200, 4) + .withColumn("v", col("id")) + .withColumn("partCol", (col("id") / lit(50)).cast("Int")) + + val conf = Seq(DeltaSQLConf.DELTA_COLLECT_STATS.key -> false.toString) + withTempDeltaTable(targetDF, Seq("partCol"), conf = conf) { (targetTable, targetLog) => + val statsBeforeFirstDelete = getStatsInPartitionOrder(targetLog.update()) + val expectedStatsBeforeFirstDelete = Seq( + Row(null, null, null, null, null), // File 1. + Row(null, null, null, null, null), // File 2. + Row(null, null, null, null, null), // File 3. + Row(null, null, null, null, null) // File 4. + ) + assert(statsBeforeFirstDelete === expectedStatsBeforeFirstDelete) + + // This operation touches files 2 and 3. Files 1 and 4 should still have not stats. + targetTable().delete("id in (50, 100)") + + // Expect the stats for every file that got a DV added to it with tightBounds = false + val statsAfterFirstDelete = getStatsInPartitionOrder(targetLog.update()) + val expectedStatsAfterFirstDelete = Seq( + Row(null, null, null, null, null), // File 1. + Row(50, Row(50, 50), Row(99, 99), Row(0, 0), false), // File 2. + Row(50, Row(100, 100), Row(149, 149), Row(0, 0), false), // File 3. + Row(null, null, null, null, null) // File 4. + ) + assert(statsAfterFirstDelete === expectedStatsAfterFirstDelete) + } + } +} + +class TightBoundsColumnMappingSuite extends TightBoundsSuite with DeltaColumnMappingEnableIdMode diff --git a/core/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala index 8d38676c35d..ef51dc14c82 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala @@ -702,10 +702,8 @@ abstract class UpdateSuiteBase val scans = executedPlans.flatMap(_.collect { case f: FileSourceScanExec => f }) - // Currently nested schemas can't be pruned, but Spark 3.4 loosens some of the restrictions - // on non-determinstic expressions, and this should be pruned to just "nested STRUCT" - // after upgrading - assert(scans.head.schema == StructType.fromDDL("nested STRUCT")) + + assert(scans.head.schema == StructType.fromDDL("nested STRUCT")) } /** @@ -893,8 +891,8 @@ abstract class UpdateSuiteBase testInvalidTempViews("subset cols")( text = "SELECT key FROM tab", - expectedErrorClassForSQLTempView = "MISSING_COLUMN", - expectedErrorClassForDataSetTempView = "MISSING_COLUMN" + expectedErrorClassForSQLTempView = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + expectedErrorClassForDataSetTempView = "UNRESOLVED_COLUMN.WITH_SUGGESTION" ) testInvalidTempViews("superset cols")( diff --git a/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala index b1c46e9f124..36b9c037706 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala @@ -551,12 +551,15 @@ class DeletionVectorsSuite extends QueryTest } } - private def getFileActionsInLastVersion(log: DeltaLog): (Seq[AddFile], Seq[RemoveFile]) = { - val version = log.update().version - val allFiles = log.getChanges(version).toSeq.head._2 - val add = allFiles.collect { case a: AddFile => a } - val remove = allFiles.collect { case r: RemoveFile => r } - (add, remove) + test("huge table: read from tables of 2B rows with existing DV of many zeros") { + val canonicalTable5Path = new File(table5Path).getCanonicalPath + checkCountAndSum("value", table5Count, table5Sum, canonicalTable5Path) + } + + private def checkCountAndSum(column: String, count: Long, sum: Long, tableDir: String): Unit = { + checkAnswer( + sql(s"SELECT count($column), sum($column) FROM delta.`$tableDir`"), + Seq((count, sum)).toDF()) } private def assertPlanContains(queryDf: DataFrame, expected: String): Unit = { @@ -607,4 +610,19 @@ object DeletionVectorsSuite { // Table with DV table feature as supported but no DVs val table4Path = "src/test/resources/delta/table-with-dv-feature-enabled" val expectedTable4DataV0 = Seq(1L) + + // Table with DV, (1<<31)+10=2147483658 rows in total including 2147484 rows deleted. Parquet is + // generated by: + // spark.range(0, (1L << 31) + 10, 1, numPartitions = 1) + // .withColumn( + // "value", + // when($"id" % 1000 === 0, 1).otherwise(($"id" / 100000000).cast(IntegerType))) + // All "id % 1000 = 0" rows are marked as deleted. + // Column "value" ranges from 0 to 21. + // 99900000 rows with values 0 to 20 each, and 47436174 rows with value 21. + val table5Path = "src/test/resources/delta/table-with-dv-gigantic" + val table5Count = 2145336174L + val table5Sum = 21975159654L + val table5CountByValues = (0 to 20).map(_ -> 99900000L).toMap + (21 -> 47436174L) + val table5SumByValues = (0 to 20).map(v => v -> v * 99900000L).toMap + (21 -> 21 * 47436174L) } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DisableUpdatesToDvEnabledTablesSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DisableUpdatesToDvEnabledTablesSuite.scala deleted file mode 100644 index b871bf661bd..00000000000 --- a/core/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DisableUpdatesToDvEnabledTablesSuite.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.delta.deletionvectors - -import java.io.File -import java.lang - -import org.apache.hadoop.fs.Path - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.delta.{DeletionVectorsTestUtils, DeltaTestUtilsForTempViews} -import org.apache.spark.sql.delta.deletionvectors.DeletionVectorsSuite._ -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN -import org.apache.spark.sql.test.SharedSparkSession - -/** - * Test suite for testing all write commands are disabled on tables with deletion vectors. - * This is a temporary behavior until we properly implement and test write support on - * tables with deletion vectors. - */ -class DisableUpdatesToDvEnabledTablesSuite extends QueryTest - with SharedSparkSession - with DeletionVectorsTestUtils - with DeltaSQLCommandTest - with DeltaTestUtilsForTempViews { - - import io.delta.implicits._ - - test("DELETE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"DELETE FROM $table2WithDVs WHERE value in (2, 5, 7)") - } - } - - test("MERGE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"MERGE INTO $table1WithDVs t USING (SELECT * FROM $table2WithDVs) s " + - s"ON t.value = s.value WHEN MATCHED THEN DELETE ") - } - } - - test("UPDATE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"UPDATE $table2WithDVs SET value = 3 WHERE value > 0") - } - } - - test("INSERT INTO is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"INSERT INTO $table2WithDVs SELECT 200") - } - } - - test("INSERT INTO is blocked on table with DV feature supported, but no DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table4WithDVFeatureSupported)) { _ => - spark.sql(s"INSERT INTO $table4WithDVFeatureSupported SELECT 200L") - } - } - - test("REPLACE OVERWRITE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"INSERT OVERWRITE $table1WithDVs SELECT * FROM $table2WithDVs") - } - } - - test("OPTIMIZE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"OPTIMIZE $table2WithDVs") - } - } - - test("RESTORE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - spark.sql(s"RESTORE $table2WithDVs TO VERSION AS OF 0") - } - } - - for (enableLogging <- BOOLEAN_DOMAIN) - test(s"VACUUM is blocked on table with DVs with logging enabled=$enableLogging") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table1WithDVs)) { _ => - withSQLConf( - DeltaSQLConf.DELTA_VACUUM_RETENTION_CHECK_ENABLED.key -> "false", - // Logging influencing whether a transaction is committed to DeltaLog or not - DeltaSQLConf.DELTA_VACUUM_LOGGING_ENABLED.key -> enableLogging.toString) { - spark.sql(s"VACUUM $table1WithDVs RETAIN 0 HOURS") - } - } - } - - test("CLONE is blocked on table with DVs") { - assertDVTableUpdatesAreDisabled(testTablePath = Some(table2WithDVs)) { tablePath => - spark.sql(s"CREATE TABLE delta.`$tablePath` SHALLOW CLONE $table2WithDVs") - } - } - - test("CREATE TABLE with DVs is blocked") { - assertDVTableUpdatesAreDisabled(testTablePath = None) { tablePath => - withDeletionVectorsEnabled() { - createTempTable(tablePath) - } - } - } - - test("CREATE TABLE with DV feature enabled is blocked") { - assertDVTableUpdatesAreDisabled(testTablePath = None) { tablePath => - withTable("tab") { - spark.sql(s"CREATE TABLE tab (c1 int) USING DELTA " + - "TBLPROPERTIES ('delta.feature.deletionVectors' = 'supported');") - } - } - } - - test("ALTER TABLE to add DV feature is blocked") { - assertDVTableUpdatesAreDisabled(testTablePath = None) { tablePath => - withTable("tab") { - spark.sql("CREATE TABLE tab (c1 int) USING DELTA;") - spark.sql("ALTER TABLE tab SET " + - "TBLPROPERTIES ('delta.feature.deletionVectors' = 'supported');") - } - } - } - - test("Enabling DV feature on a table is blocked") { - assertDVTableUpdatesAreDisabled(testTablePath = None) { tablePath => - createTempTable(tablePath) - enableDeletionVectorsInTable(new Path(tablePath), enable = true) - } - } - - def assertDVTableUpdatesAreDisabled(testTablePath: Option[String])(f: String => Unit): Unit = { - val dataBefore = testTablePath.map(path => spark.sql(s"SELECT * FROM $path")) - val ex = intercept[UnsupportedOperationException] { - withTempPath { path => - f(path.getAbsolutePath) - } - } - assert(ex.isInstanceOf[UnsupportedOperationException]) - val msg = ex.getMessage - assert( - msg.contains("Updates to tables with Deletion Vectors feature enabled are " + - "not supported in this version of Delta Lake.") | - msg.contains("Enabling Deletion Vectors on the table is not supported in this " + - "version of Delta Lake.")) - - val dataAfter = testTablePath.map(path => spark.sql(s"SELECT * FROM $path")) - if (testTablePath.isDefined) { - checkAnswer(dataAfter.get, dataBefore.get) - } - } - - private def createTempTable(path: String): Unit = { - spark.range(end = 100L).toDF("id").coalesce(1) - .write.format("delta").mode("overwrite").save(path) - } - - protected override def beforeAll(): Unit = { - super.beforeAll() - spark.sessionState.conf.setConf( - DeltaSQLConf.DELTA_ENABLE_BLOCKING_UPDATES_ON_DV_TABLES, false) - } - - protected override def afterAll(): Unit = { - spark.sessionState.conf.setConf( - DeltaSQLConf.DELTA_ENABLE_BLOCKING_UPDATES_ON_DV_TABLES, true) - super.afterAll() - } - - private val table2WithDVs = s"delta.`${new File(table2Path).getAbsolutePath}`" - private val table1WithDVs = s"delta.`${new File(table1Path).getAbsolutePath}`" - private val table4WithDVFeatureSupported = s"delta.`${new File(table4Path).getAbsolutePath}`" -} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/optimize/DeltaReorgSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/optimize/DeltaReorgSuite.scala new file mode 100644 index 00000000000..715d952a86c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/sql/delta/optimize/DeltaReorgSuite.scala @@ -0,0 +1,113 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.optimize + +import org.apache.spark.sql.delta.DeletionVectorsTestUtils +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.test.SharedSparkSession + +class DeltaReorgSuite extends QueryTest + with SharedSparkSession + with DeltaSQLCommandTest + with DeletionVectorsTestUtils { + + import testImplicits._ + + def executePurge(table: String, condition: Option[String] = None): Unit = { + condition match { + case Some(cond) => sql(s"REORG TABLE delta.`$table` WHERE $cond APPLY (PURGE)") + case None => sql(s"REORG TABLE delta.`$table` APPLY (PURGE)") + } + } + + test("Purge DVs will combine small files") { + val targetDf = spark.range(0, 100, 1, numPartitions = 5).toDF + withTempDeltaTable(targetDf) { (_, log) => + val path = log.dataPath.toString + + sql(s"DELETE FROM delta.`$path` WHERE id IN (0, 99)") + assert(log.update().allFiles.filter(_.deletionVector != null).count() === 2) + withSQLConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE.key -> "1073741824") { // 1gb + executePurge(path) + } + val (addFiles, _) = getFileActionsInLastVersion(log) + assert(addFiles.size === 1, "files should be combined") + assert(addFiles.forall(_.deletionVector === null)) + checkAnswer( + sql(s"SELECT * FROM delta.`$path`"), + (1 to 98).toDF()) + } + } + + test("Purge DVs") { + val targetDf = spark.range(0, 100, 1, numPartitions = 5).toDF() + withTempDeltaTable(targetDf) { (_, log) => + val path = log.dataPath.toString + + sql(s"DELETE FROM delta.`$path` WHERE id IN (0, 99)") + assert(log.update().allFiles.filter(_.deletionVector != null).count() === 2) + + // First purge + executePurge(path) + val (addFiles, _) = getFileActionsInLastVersion(log) + assert(addFiles.size === 1) // two files are combined + assert(addFiles.forall(_.deletionVector === null)) + checkAnswer( + sql(s"SELECT * FROM delta.`$path`"), + (1 to 98).toDF()) + + // Second purge is a noop + val versionBefore = log.update().version + executePurge(path) + val versionAfter = log.update().version + assert(versionBefore === versionAfter) + } + } + + test("Purge a non-DV table is a noop") { + val targetDf = spark.range(0, 100, 1, numPartitions = 5).toDF() + withTempDeltaTable(targetDf, enableDVs = false) { (_, log) => + val versionBefore = log.update().version + executePurge(log.dataPath.toString) + val versionAfter = log.update().version + assert(versionBefore === versionAfter) + } + } + + test("Purge some partitions of a table with DV") { + val targetDf = spark.range(0, 100, 1, numPartitions = 1) + .withColumn("part", col("id") % 4) + .toDF() + withTempDeltaTable(targetDf, partitionBy = Seq("part")) { (_, log) => + val path = log.dataPath + // Delete one row from each partition + sql(s"DELETE FROM delta.`$path` WHERE id IN (48, 49, 50, 51)") + val (addFiles1, _) = getFileActionsInLastVersion(log) + assert(addFiles1.size === 4) + assert(addFiles1.forall(_.deletionVector !== null)) + // PURGE two partitions + sql(s"REORG TABLE delta.`$path` WHERE part IN (0, 2) APPLY (PURGE)") + val (addFiles2, _) = getFileActionsInLastVersion(log) + assert(addFiles2.size === 2) + assert(addFiles2.forall(_.deletionVector === null)) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala index 109bf46054a..f100239da0d 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala @@ -196,9 +196,9 @@ class CaseSensitivitySuite extends QueryTest .option("replaceWhere", "key = 2") // note the different case .save(path) } - // The error class is renamed in Spark 3.4 assert(e.getErrorClass == "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION" - || e.getErrorClass == "MISSING_COLUMN" ) + || e.getErrorClass == "MISSING_COLUMN" + || e.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") } checkAnswer( diff --git a/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala index 818a5a02ca4..8baf0820f68 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala @@ -805,52 +805,78 @@ class SchemaUtilsSuite extends QueryTest test("findColumnPosition") { val schema = new StructType() - .add("a", new StructType() - .add("b", IntegerType) - .add("c", IntegerType)) - .add("d", ArrayType(new StructType() - .add("b", IntegerType) - .add("c", IntegerType))) - .add("e", StringType) - .add("f", MapType( + .add("struct", new StructType() + .add("a", IntegerType) + .add("b", IntegerType)) + .add("array", ArrayType(new StructType() + .add("c", IntegerType) + .add("d", IntegerType))) + .add("field", StringType) + .add("map", MapType( new StructType() - .add("g", IntegerType), + .add("e", IntegerType), new StructType() - .add("h", IntegerType))) - .add("i", MapType( + .add("f", IntegerType))) + .add("mapStruct", MapType( IntegerType, new StructType() - .add("k", new StructType() - .add("l", IntegerType)))) - .add("m", ArrayType( - MapType(StringType, StringType))) - assert(SchemaUtils.findColumnPosition(Seq("a"), schema) === ((Seq(0), 2))) - assert(SchemaUtils.findColumnPosition(Seq("A"), schema) === ((Seq(0), 2))) + .add("g", new StructType() + .add("h", IntegerType)))) + .add("arrayMap", ArrayType( + MapType( + new StructType() + .add("i", IntegerType), + new StructType() + .add("j", IntegerType)))) + + val List(structIdx, arrayIdx, fieldIdx, mapIdx, mapStructIdx, arrayMapIdx) = (0 to 5).toList + val ARRAY_ELEMENT_INDEX = 0 + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + def checkPosition(column: Seq[String], position: Seq[Int]): Unit = + assert(SchemaUtils.findColumnPosition(column, schema) === position) + + checkPosition(Seq("struct"), Seq(structIdx)) + checkPosition(Seq("STRucT"), Seq(structIdx)) expectFailure("Couldn't find", schema.treeString) { - SchemaUtils.findColumnPosition(Seq("a", "d"), schema) - } - assert(SchemaUtils.findColumnPosition(Seq("a", "b"), schema) === ((Seq(0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("A", "b"), schema) === ((Seq(0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("a", "B"), schema) === ((Seq(0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("A", "B"), schema) === ((Seq(0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("a", "c"), schema) === ((Seq(0, 1), 0))) - assert(SchemaUtils.findColumnPosition(Seq("d"), schema) === ((Seq(1), 2))) - assert(SchemaUtils.findColumnPosition(Seq("d", "element", "B"), schema) === ((Seq(1, 0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("d", "element", "c"), schema) === ((Seq(1, 0, 1), 0))) - assert(SchemaUtils.findColumnPosition(Seq("e"), schema) === ((Seq(2), 0))) - assert(SchemaUtils.findColumnPosition(Seq("f"), schema) === ((Seq(3), 2))) - assert(SchemaUtils.findColumnPosition(Seq("f", "key", "g"), schema) === ((Seq(3, 0, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("f", "value", "h"), schema) === ((Seq(3, 1, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("f", "value", "H"), schema) === ((Seq(3, 1, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("i", "key"), schema) === ((Seq(4, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("i", "value", "k"), schema) === ((Seq(4, 1, 0), 1))) - assert(SchemaUtils.findColumnPosition(Seq("i", "key"), schema) === ((Seq(4, 0), 0))) - assert(SchemaUtils.findColumnPosition(Seq("i", "value"), schema) === ((Seq(4, 1), 1))) - assert(SchemaUtils.findColumnPosition(Seq("m"), schema) === ((Seq(5), 0))) + SchemaUtils.findColumnPosition(Seq("struct", "array"), schema) + } + checkPosition(Seq("struct", "a"), Seq(structIdx, 0)) + checkPosition(Seq("STRucT", "a"), Seq(structIdx, 0)) + checkPosition(Seq("struct", "A"), Seq(structIdx, 0)) + checkPosition(Seq("STRucT", "A"), Seq(structIdx, 0)) + checkPosition(Seq("struct", "b"), Seq(structIdx, 1)) + checkPosition(Seq("array"), Seq(arrayIdx)) + checkPosition(Seq("array", "element", "C"), Seq(arrayIdx, ARRAY_ELEMENT_INDEX, 0)) + checkPosition(Seq("array", "element", "d"), Seq(arrayIdx, ARRAY_ELEMENT_INDEX, 1)) + checkPosition(Seq("field"), Seq(fieldIdx)) + checkPosition(Seq("map"), Seq(mapIdx)) + checkPosition(Seq("map", "key", "e"), Seq(mapIdx, MAP_KEY_INDEX, 0)) + checkPosition(Seq("map", "value", "f"), Seq(mapIdx, MAP_VALUE_INDEX, 0)) + checkPosition(Seq("map", "value", "F"), Seq(mapIdx, MAP_VALUE_INDEX, 0)) + checkPosition(Seq("mapStruct", "key"), Seq(mapStructIdx, MAP_KEY_INDEX)) + checkPosition(Seq("mapStruct", "value", "g"), Seq(mapStructIdx, MAP_VALUE_INDEX, 0)) + checkPosition(Seq("mapStruct", "key"), Seq(mapStructIdx, MAP_KEY_INDEX)) + checkPosition(Seq("mapStruct", "value"), Seq(mapStructIdx, MAP_VALUE_INDEX)) + checkPosition(Seq("arrayMap"), Seq(arrayMapIdx)) + checkPosition(Seq("arrayMap", "element"), Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX)) + checkPosition( + Seq("arrayMap", "element", "key"), + Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_KEY_INDEX)) + checkPosition( + Seq("arrayMap", "element", "value"), + Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_VALUE_INDEX)) + checkPosition( + Seq("arrayMap", "element", "key", "i"), + Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_KEY_INDEX, 0)) + checkPosition( + Seq("arrayMap", "element", "value", "j"), + Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_VALUE_INDEX, 0)) val resolver = org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution - Seq(Seq("A", "b"), Seq("a", "B"), Seq("d", "element", "B"), Seq("f", "key", "H")) - .foreach { column => + Seq(Seq("STRucT", "b"), Seq("struct", "B"), Seq("array", "element", "C"), + Seq("map", "key", "E")).foreach { column => expectFailure("Couldn't find", schema.treeString) { SchemaUtils.findColumnPosition(column, schema, resolver) } @@ -869,7 +895,98 @@ class SchemaUtilsSuite extends QueryTest SchemaUtils.findColumnPosition(Seq("b", "c"), schema) } expectFailure("An ArrayType was found", "arrayType", schema.treeString) { - SchemaUtils.findColumnPosition(Seq("c", "element"), schema) + SchemaUtils.findColumnPosition(Seq("c", "b"), schema) + } + } + + //////////////////////////// + // getNestedFieldFromPosition + //////////////////////////// + + test("getNestedFieldFromPosition") { + val a = StructField("a", IntegerType) + val b = StructField("b", IntegerType) + val c = StructField("c", IntegerType) + val d = StructField("d", IntegerType) + val e = StructField("e", IntegerType) + val f = StructField("f", IntegerType) + val g = StructField("g", IntegerType) + + val field = StructField("field", StringType) + val struct = StructField("struct", new StructType().add(a).add(b)) + val arrayElement = StructField("element", new StructType().add(c)) + val array = StructField("array", ArrayType(arrayElement.dataType)) + val mapKey = StructField("key", new StructType().add(d)) + val mapValue = StructField("value", new StructType().add(e)) + val map = StructField("map", MapType( + keyType = mapKey.dataType, + valueType = mapValue.dataType)) + val arrayMapKey = StructField("key", new StructType().add(f)) + val arrayMapValue = StructField("value", new StructType().add(g)) + val arrayMapElement = StructField("element", MapType( + keyType = arrayMapKey.dataType, + valueType = arrayMapValue.dataType)) + val arrayMap = StructField("arrayMap", ArrayType(arrayMapElement.dataType)) + + val root = StructField("root", StructType(Seq(field, struct, array, map, arrayMap))) + + val List(fieldIdx, structIdx, arrayIdx, mapIdx, arrayMapIdx) = (0 to 4).toList + val ARRAY_ELEMENT_INDEX = 0 + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + def checkField(position: Seq[Int], expected: StructField): Unit = + assert(getNestedFieldFromPosition(root, position) === expected) + + checkField(Seq.empty, root) + checkField(Seq(fieldIdx), field) + checkField(Seq(structIdx), struct) + checkField(Seq(structIdx, 0), a) + checkField(Seq(structIdx, 1), b) + checkField(Seq(arrayIdx), array) + checkField(Seq(arrayIdx, ARRAY_ELEMENT_INDEX), arrayElement) + checkField(Seq(arrayIdx, ARRAY_ELEMENT_INDEX, 0), c) + checkField(Seq(mapIdx), map) + checkField(Seq(mapIdx, MAP_KEY_INDEX), mapKey) + checkField(Seq(mapIdx, MAP_VALUE_INDEX), mapValue) + checkField(Seq(mapIdx, MAP_KEY_INDEX, 0), d) + checkField(Seq(mapIdx, MAP_VALUE_INDEX, 0), e) + checkField(Seq(arrayMapIdx), arrayMap) + checkField(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX), arrayMapElement) + checkField(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_KEY_INDEX), arrayMapKey) + checkField(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_VALUE_INDEX), arrayMapValue) + checkField(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_KEY_INDEX, 0), f) + checkField(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_VALUE_INDEX, 0), g) + + def checkError(position: Seq[Int]): Unit = + assertThrows[IllegalArgumentException] { + getNestedFieldFromPosition(root, position) + } + + checkError(Seq(-1)) + checkError(Seq(fieldIdx, 0)) + checkError(Seq(structIdx, -1)) + checkError(Seq(structIdx, 2)) + checkError(Seq(arrayIdx, ARRAY_ELEMENT_INDEX - 1)) + checkError(Seq(arrayIdx, ARRAY_ELEMENT_INDEX + 1)) + checkError(Seq(mapIdx, MAP_KEY_INDEX - 1)) + checkError(Seq(mapIdx, MAP_VALUE_INDEX + 1)) + checkError(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX - 1)) + checkError(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX + 1)) + checkError(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_KEY_INDEX - 1)) + checkError(Seq(arrayMapIdx, ARRAY_ELEMENT_INDEX, MAP_VALUE_INDEX + 1)) + checkError(Seq(arrayMapIdx + 1)) + } + + test("getNestedTypeFromPosition") { + val schema = new StructType().add("a", IntegerType) + assert(getNestedTypeFromPosition(schema, Seq.empty) === schema) + assert(getNestedTypeFromPosition(schema, Seq(0)) === IntegerType) + assertThrows[IllegalArgumentException] { + getNestedTypeFromPosition(schema, Seq(-1)) + } + assertThrows[IllegalArgumentException] { + getNestedTypeFromPosition(schema, Seq(1)) } } @@ -901,16 +1018,31 @@ class SchemaUtilsSuite extends QueryTest test("addColumn - nested struct") { val a = StructField("a", IntegerType) val b = StructField("b", StringType) - val s = StructField("s", new StructType().add(a).add(b)) - val schema = new StructType().add(s) + val first = StructField("first", new StructType().add(a).add(b)) + val middle = StructField("middle", new StructType().add(a).add(b)) + val last = StructField("last", new StructType().add(a).add(b)) + val schema = new StructType().add(first).add(middle).add(last) val x = StructField("x", LongType) - assert(SchemaUtils.addColumn(schema, x, Seq(0)) === new StructType().add(x).add(s)) - assert(SchemaUtils.addColumn(schema, x, Seq(0, 0)) === - new StructType().add("s", new StructType().add(x).add(a).add(b))) + assert(SchemaUtils.addColumn(schema, x, Seq(0)) === + new StructType().add(x).add(first).add(middle).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(1)) === + new StructType().add(first).add(x).add(middle).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(2)) === + new StructType().add(first).add(middle).add(x).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(3)) === + new StructType().add(first).add(middle).add(last).add(x)) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 2)) === - new StructType().add("s", new StructType().add(a).add(b).add(x))) - assert(SchemaUtils.addColumn(schema, x, Seq(1)) === new StructType().add(s).add(x)) + new StructType().add("first", new StructType().add(a).add(b).add(x)).add(middle).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 1)) === + new StructType().add("first", new StructType().add(a).add(x).add(b)).add(middle).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 0)) === + new StructType().add("first", new StructType().add(x).add(a).add(b)).add(middle).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(1, 0)) === + new StructType().add(first).add("middle", new StructType().add(x).add(a).add(b)).add(last)) + assert(SchemaUtils.addColumn(schema, x, Seq(2, 0)) === + new StructType().add(first).add(middle).add("last", new StructType().add(x).add(a).add(b))) expectFailure("Index -1", "lower than 0") { SchemaUtils.addColumn(schema, x, Seq(0, -1)) @@ -926,6 +1058,152 @@ class SchemaUtilsSuite extends QueryTest } } + test("addColumn - nested map") { + val k = StructField("k", IntegerType) + val v = StructField("v", StringType) + val schema = new StructType().add("m", MapType( + keyType = new StructType().add(k), + valueType = new StructType().add(v))) + + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + val x = StructField("x", LongType) + assert(SchemaUtils.addColumn(schema, x, Seq(0, MAP_KEY_INDEX, 0)) === + new StructType().add("m", MapType( + keyType = new StructType().add(x).add(k), + valueType = new StructType().add(v)))) + + assert(SchemaUtils.addColumn(schema, x, Seq(0, MAP_KEY_INDEX, 1)) === + new StructType().add("m", MapType( + keyType = new StructType().add(k).add(x), + valueType = new StructType().add(v)))) + + assert(SchemaUtils.addColumn(schema, x, Seq(0, MAP_VALUE_INDEX, 0)) === + new StructType().add("m", MapType( + keyType = new StructType().add(k), + valueType = new StructType().add(x).add(v)))) + + assert(SchemaUtils.addColumn(schema, x, Seq(0, MAP_VALUE_INDEX, 1)) === + new StructType().add("m", MapType( + keyType = new StructType().add(k), + valueType = new StructType().add(v).add(x)))) + + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema, x, Seq(0, MAP_KEY_INDEX - 1, 0)) + } + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema, x, Seq(0, MAP_VALUE_INDEX + 1, 0)) + } + } + + test("addColumn - nested maps") { + // Helper method to create a 2-level deep nested map of structs. The tests below each cover + // adding a field to one of the leaf struct. + def schema( + kk: StructType = new StructType().add("kk", IntegerType), + kv: StructType = new StructType().add("kv", IntegerType), + vk: StructType = new StructType().add("vk", IntegerType), + vv: StructType = new StructType().add("vv", IntegerType)) + : StructType = new StructType().add("m", MapType( + keyType = MapType( + keyType = kk, + valueType = kv), + valueType = MapType( + keyType = vk, + valueType = vv))) + + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + val x = StructField("x", LongType) + // Add field `x` at the front of each leaf struct. + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX, 0)) === + schema(kk = new StructType().add(x).add("kk", IntegerType))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_VALUE_INDEX, MAP_KEY_INDEX, 0)) === + schema(vk = new StructType().add(x).add("vk", IntegerType))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX, 0)) === + schema(kv = new StructType().add(x).add("kv", IntegerType))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_VALUE_INDEX, MAP_VALUE_INDEX, 0)) === + schema(vv = new StructType().add(x).add("vv", IntegerType))) + + // Add field `x` at the back of each leaf struct. + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX, 1)) === + schema(kk = new StructType().add("kk", IntegerType).add(x))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_VALUE_INDEX, MAP_KEY_INDEX, 1)) === + schema(vk = new StructType().add("vk", IntegerType).add(x))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX, 1)) === + schema(kv = new StructType().add("kv", IntegerType).add(x))) + assert(SchemaUtils.addColumn(schema(), x, Seq(0, MAP_VALUE_INDEX, MAP_VALUE_INDEX, 1)) === + schema(vv = new StructType().add("vv", IntegerType).add(x))) + + // Invalid map access. + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX - 1, 0)) + } + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX - 1, MAP_KEY_INDEX, 0)) + } + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema(), x, Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX + 1, 0)) + } + expectFailure("parent is not a structtype") { + SchemaUtils.addColumn(schema(), x, Seq(0, MAP_VALUE_INDEX + 1, MAP_KEY_INDEX, 0)) + } + } + + test("addColumn - nested array") { + val e = StructField("e", IntegerType) + val schema = new StructType().add("a", ArrayType(new StructType().add(e))) + val x = StructField("x", LongType) + + val ARRAY_ELEMENT_INDEX = 0 + + // Add field `x` at the front of the leaf struct. + assert(SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, 0)) === + new StructType().add("a", ArrayType(new StructType().add(x).add(e)))) + // Add field `x` at the back of the leaf struct. + assert(SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, 1)) === + new StructType().add("a", ArrayType(new StructType().add(e).add(x)))) + + // Invalid array access. + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX - 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX + 1, 0)) + } + } + + test("addColumn - nested arrays") { + val e = StructField("e", IntegerType) + val schema = new StructType().add("a", ArrayType(ArrayType(new StructType().add(e)))) + val x = StructField("x", LongType) + + val ARRAY_ELEMENT_INDEX = 0 + + // Add field `x` at the front of the leaf struct. + assert(SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX, 0)) === + new StructType().add("a", ArrayType(ArrayType(new StructType().add(x).add(e))))) + // Add field `x` at the back of the leaf struct. + assert(SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX, 1)) === + new StructType().add("a", ArrayType(ArrayType(new StructType().add(e).add(x))))) + + // Invalid array access. + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX - 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX - 1, ARRAY_ELEMENT_INDEX, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX + 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.addColumn(schema, x, Seq(0, ARRAY_ELEMENT_INDEX + 1, ARRAY_ELEMENT_INDEX, 0)) + } + } + //////////////////////////// // dropColumn //////////////////////////// @@ -953,26 +1231,209 @@ class SchemaUtilsSuite extends QueryTest val a = StructField("a", IntegerType) val b = StructField("b", StringType) val c = StructField("c", StringType) - val s = StructField("s", new StructType().add(a).add(b)) - val schema = new StructType().add(s).add(c) - - assert(SchemaUtils.dropColumn(schema, Seq(0)) === ((new StructType().add(c), s))) - assert(SchemaUtils.dropColumn(schema, Seq(0, 0)) === - ((new StructType().add("s", new StructType().add(b)).add(c), a))) + val first = StructField("first", new StructType().add(a).add(b).add(c)) + val middle = StructField("middle", new StructType().add(a).add(b).add(c)) + val last = StructField("last", new StructType().add(a).add(b).add(c)) + val schema = new StructType().add(first).add(middle).add(last) + + assert(SchemaUtils.dropColumn(schema, Seq(0)) === + new StructType().add(middle).add(last) -> first) + assert(SchemaUtils.dropColumn(schema, Seq(1)) === + new StructType().add(first).add(last) -> middle) + assert(SchemaUtils.dropColumn(schema, Seq(2)) === + new StructType().add(first).add(middle) -> last) + + assert(SchemaUtils.dropColumn(schema, Seq(0, 2)) === + new StructType().add("first", new StructType().add(a).add(b)).add(middle).add(last) -> c) assert(SchemaUtils.dropColumn(schema, Seq(0, 1)) === - ((new StructType().add("s", new StructType().add(a)).add(c), b))) + new StructType().add("first", new StructType().add(a).add(c)).add(middle).add(last) -> b) + assert(SchemaUtils.dropColumn(schema, Seq(0, 0)) === + new StructType().add("first", new StructType().add(b).add(c)).add(middle).add(last) -> a) + assert(SchemaUtils.dropColumn(schema, Seq(1, 0)) === + new StructType().add(first).add("middle", new StructType().add(b).add(c)).add(last) -> a) + assert(SchemaUtils.dropColumn(schema, Seq(2, 0)) === + new StructType().add(first).add(middle).add("last", new StructType().add(b).add(c)) -> a) expectFailure("Index -1", "lower than 0") { SchemaUtils.dropColumn(schema, Seq(0, -1)) } - expectFailure("Index 2", "equals to or is larger than struct length: 2") { - SchemaUtils.dropColumn(schema, Seq(0, 2)) + expectFailure("Index 3", "equals to or is larger than struct length: 3") { + SchemaUtils.dropColumn(schema, Seq(0, 3)) } expectFailure("Can only drop nested columns from StructType") { SchemaUtils.dropColumn(schema, Seq(0, 0, 0)) } } + test("dropColumn - nested map") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val c = StructField("c", LongType) + val d = StructField("d", DateType) + val schema = new StructType().add("m", MapType( + keyType = new StructType().add(a).add(b), + valueType = new StructType().add(c).add(d))) + + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + assert(SchemaUtils.dropColumn(schema, Seq(0, MAP_KEY_INDEX, 0)) === + (new StructType().add("m", MapType( + keyType = new StructType().add(b), + valueType = new StructType().add(c).add(d))), + a)) + + assert(SchemaUtils.dropColumn(schema, Seq(0, MAP_KEY_INDEX, 1)) === + (new StructType().add("m", MapType( + keyType = new StructType().add(a), + valueType = new StructType().add(c).add(d))), + b)) + + assert(SchemaUtils.dropColumn(schema, Seq(0, MAP_VALUE_INDEX, 0)) === + (new StructType().add("m", MapType( + keyType = new StructType().add(a).add(b), + valueType = new StructType().add(d))), + c)) + + assert(SchemaUtils.dropColumn(schema, Seq(0, MAP_VALUE_INDEX, 1)) === + (new StructType().add("m", MapType( + keyType = new StructType().add(a).add(b), + valueType = new StructType().add(c))), + d)) + + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema, Seq(0, MAP_KEY_INDEX - 1, 0)) + } + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema, Seq(0, MAP_VALUE_INDEX + 1, 0)) + } + } + + test("dropColumn - nested maps") { + // Helper method to create a 2-level deep nested map of structs. The tests below each cover + // dropping a field to one of the leaf struct. Each test adds an extra field `a` at a specific + // position then drops it to end up with the default schema returned by `schema()` + def schema( + kk: StructType = new StructType().add("kk", IntegerType), + kv: StructType = new StructType().add("kv", IntegerType), + vk: StructType = new StructType().add("vk", IntegerType), + vv: StructType = new StructType().add("vv", IntegerType)) + : StructType = new StructType().add("m", MapType( + keyType = MapType( + keyType = kk, + valueType = kv), + valueType = MapType( + keyType = vk, + valueType = vv))) + + val a = StructField("a", LongType) + + val MAP_KEY_INDEX = 0 + val MAP_VALUE_INDEX = 1 + + def checkDrop(initialSchema: StructType, position: Seq[Int]): Unit = + assert(SchemaUtils.dropColumn(initialSchema, position) === (schema(), a)) + // Drop field `a` from the front of each leaf struct. + checkDrop( + initialSchema = schema(kk = new StructType().add(a).add("kk", IntegerType)), + position = Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX, 0)) + + checkDrop( + initialSchema = schema(kv = new StructType().add(a).add("kv", IntegerType)), + position = Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX, 0)) + + checkDrop( + initialSchema = schema(vk = new StructType().add(a).add("vk", IntegerType)), + position = Seq(0, MAP_VALUE_INDEX, MAP_KEY_INDEX, 0)) + + checkDrop( + initialSchema = schema(vv = new StructType().add(a).add("vv", IntegerType)), + position = Seq(0, MAP_VALUE_INDEX, MAP_VALUE_INDEX, 0)) + + // Drop field `a` from the back of each leaf struct. + checkDrop( + initialSchema = schema(kk = new StructType().add("kk", IntegerType).add(a)), + position = Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX, 1)) + + checkDrop( + initialSchema = schema(kv = new StructType().add("kv", IntegerType).add(a)), + position = Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX, 1)) + + checkDrop( + initialSchema = schema(vk = new StructType().add("vk", IntegerType).add(a)), + position = Seq(0, MAP_VALUE_INDEX, MAP_KEY_INDEX, 1)) + + checkDrop( + initialSchema = schema(vv = new StructType().add("vv", IntegerType).add(a)), + position = Seq(0, MAP_VALUE_INDEX, MAP_VALUE_INDEX, 1)) + + // Invalid map access. + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema(), Seq(0, MAP_KEY_INDEX, MAP_KEY_INDEX - 1, 0)) + } + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema(), Seq(0, MAP_KEY_INDEX - 1, MAP_KEY_INDEX, 0)) + } + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema(), Seq(0, MAP_KEY_INDEX, MAP_VALUE_INDEX + 1, 0)) + } + expectFailure("can only drop nested columns from structtype") { + SchemaUtils.dropColumn(schema(), Seq(0, MAP_VALUE_INDEX + 1, MAP_KEY_INDEX, 0)) + } + } + + test("dropColumn - nested array") { + val e = StructField("e", IntegerType) + val f = StructField("f", IntegerType) + val schema = new StructType().add("a", ArrayType(new StructType().add(e).add(f))) + + val ARRAY_ELEMENT_INDEX = 0 + + // Drop field from the front of the leaf struct. + assert(SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, 0)) === + (new StructType().add("a", ArrayType(new StructType().add(f))), e)) + // Drop field from the back of the leaf struct. + assert(SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, 1)) === + (new StructType().add("a", ArrayType(new StructType().add(e))), f)) + + // Invalid array access. + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX - 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX + 1, 0)) + } + } + + test("dropColumn - nested arrays") { + val e = StructField("e", IntegerType) + val f = StructField("f", IntegerType) + val schema = new StructType().add("a", ArrayType(ArrayType(new StructType().add(e).add(f)))) + + val ARRAY_ELEMENT_INDEX = 0 + + // Drop field `x` from the front of the leaf struct. + assert(SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX, 0)) === + (new StructType().add("a", ArrayType(ArrayType(new StructType().add(f)))), e)) + // Drop field `x` from the back of the leaf struct. + assert(SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX, 1)) === + (new StructType().add("a", ArrayType(ArrayType(new StructType().add(e)))), f)) + + // Invalid array access. + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX - 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX - 1, ARRAY_ELEMENT_INDEX, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX, ARRAY_ELEMENT_INDEX + 1, 0)) + } + expectFailure("Incorrectly accessing an ArrayType") { + SchemaUtils.dropColumn(schema, Seq(0, ARRAY_ELEMENT_INDEX + 1, ARRAY_ELEMENT_INDEX, 0)) + } + } + //////////////////////////// // normalizeColumnNames //////////////////////////// @@ -1442,10 +1903,13 @@ class SchemaUtilsSuite extends QueryTest badCharacters.foreach { char => Seq(s"a${char}b", s"${char}ab", s"ab${char}", char.toString).foreach { name => - val e = intercept[AnalysisException] { - SchemaUtils.checkFieldNames(Seq(name)) - } - assert(e.getMessage.contains("invalid character")) + checkError( + exception = intercept[AnalysisException] { + SchemaUtils.checkFieldNames(Seq(name)) + }, + errorClass = "INVALID_COLUMN_NAME_AS_PATH", + parameters = Map("datasource" -> "delta", "columnName" -> s"`$name`") + ) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/stats/StatsCollectionSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/stats/StatsCollectionSuite.scala index 0e9163fcb3b..1be619fcb8d 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/stats/StatsCollectionSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/stats/StatsCollectionSuite.scala @@ -33,10 +33,11 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} class StatsCollectionSuite - extends QueryTest - with SharedSparkSession with DeltaColumnMappingTestUtils - with TestsStatistics - with DeltaSQLCommandTest { + extends QueryTest + with SharedSparkSession with DeltaColumnMappingTestUtils + with TestsStatistics + with DeltaSQLCommandTest + with DeletionVectorsTestUtils { import testImplicits._ @@ -88,6 +89,25 @@ class StatsCollectionSuite } } + test("statistics re-computation throws error on Delta tables with DVs") { + withDeletionVectorsEnabled() { + withTempDir { dir => + val df = spark.range(start = 0, end = 20).toDF().repartition(numPartitions = 4) + df.write.format("delta").save(dir.toString()) + + spark.sql(s"DELETE FROM delta.`${dir.toString}` WHERE id in (2, 15)") + val e = intercept[DeltaCommandUnsupportedWithDeletionVectorsException] { + val deltaLog = DeltaLog.forTable(spark, dir) + StatisticsCollection.recompute(spark, deltaLog) + } + assert(e.getErrorClass == "DELTA_UNSUPPORTED_STATS_RECOMPUTE_WITH_DELETION_VECTORS") + assert(e.getSqlState == "0AKDD") + assert(e.getMessage == + "Statistics re-computation on a Delta table with deletion vectors is not yet supported.") + } + } + } + statsTest("recompute stats basic") { withTempDir { tempDir => withSQLConf(DeltaSQLConf.DELTA_COLLECT_STATS.key -> "false") { @@ -345,7 +365,6 @@ class StatsCollectionSuite } } - class StatsCollectionNameColumnMappingSuite extends StatsCollectionSuite with DeltaColumnMappingEnableNameMode { diff --git a/core/src/test/scala/org/apache/spark/sql/delta/storage/dv/DeletionVectorStoreSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/storage/dv/DeletionVectorStoreSuite.scala index dd96f68b463..28c0b4d2ee7 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/storage/dv/DeletionVectorStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/storage/dv/DeletionVectorStoreSuite.scala @@ -169,7 +169,8 @@ trait DeletionVectorStoreSuiteBase val e = intercept[DeltaChecksumException] { file.removeRows( - dvDescriptorWithInvalidRowIndex + dvDescriptorWithInvalidRowIndex, + updateStats = false ) } assert(e.getErrorClass == "DELTA_DELETION_VECTOR_INVALID_ROW_INDEX") diff --git a/examples/scala/build.sbt b/examples/scala/build.sbt index 20c7a278a0f..3669159e527 100644 --- a/examples/scala/build.sbt +++ b/examples/scala/build.sbt @@ -35,8 +35,10 @@ def getMajorMinor(version: String): (Int, Int) = { } } val lookupSparkVersion: PartialFunction[(Int, Int), String] = { - // versions 2.3.0 and above - case (major, minor) if (major == 2 && minor >= 3) || major >= 3 => "3.3.2" + // versions 2.4.0 and above + case (major, minor) if (major == 2 && minor >= 4) || major >= 3 => "3.4.0" + // versions 2.3.x + case (major, minor) if major == 2 && minor == 3 => "3.3.2" // versions 2.2.x case (major, minor) if major == 2 && minor == 2 => "3.3.1" // versions 2.1.x diff --git a/python/delta/exceptions.py b/python/delta/exceptions.py index cfdfb24702c..b64e06d14ec 100644 --- a/python/delta/exceptions.py +++ b/python/delta/exceptions.py @@ -17,10 +17,10 @@ from typing import TYPE_CHECKING, Optional from pyspark import SparkContext -from pyspark.sql import utils +from pyspark.errors.exceptions import captured +from pyspark.errors.exceptions.captured import CapturedException from pyspark.sql.utils import ( AnalysisException, - CapturedException, IllegalArgumentException, ParseException ) @@ -151,7 +151,7 @@ def _patch_convert_exception() -> None: Patch PySpark's exception convert method to convert Delta's Scala concurrent exceptions to the corresponding Python exceptions. """ - original_convert_sql_exception = utils.convert_exception + original_convert_sql_exception = captured.convert_exception def convert_delta_exception(e: "JavaObject") -> CapturedException: delta_exception = _convert_delta_exception(e) @@ -159,7 +159,7 @@ def convert_delta_exception(e: "JavaObject") -> CapturedException: return delta_exception return original_convert_sql_exception(e) - utils.convert_exception = convert_delta_exception + captured.convert_exception = convert_delta_exception if not _delta_exception_patched: diff --git a/python/delta/tests/test_deltatable.py b/python/delta/tests/test_deltatable.py index 5bb233f72b0..0341eb0f84f 100644 --- a/python/delta/tests/test_deltatable.py +++ b/python/delta/tests/test_deltatable.py @@ -1073,7 +1073,7 @@ def test_optimize_w_partition_filter(self) -> None: # assertions self.assertEqual(1, res.first().metrics.numFilesAdded) self.assertEqual(2, res.first().metrics.numFilesRemoved) - self.assertEqual('["(key = \'a\')"]', op_params['predicate']) + self.assertEqual('''["('key = a)"]''', op_params['predicate']) # test non-partition column def optimize() -> None: diff --git a/python/delta/tests/test_pip_utils.py b/python/delta/tests/test_pip_utils.py index b7b49a72850..ec5ffde1e9a 100644 --- a/python/delta/tests/test_pip_utils.py +++ b/python/delta/tests/test_pip_utils.py @@ -18,7 +18,7 @@ import shutil import tempfile import unittest -from typing import List +from typing import List, Optional from pyspark.sql import SparkSession import delta @@ -73,10 +73,12 @@ def tearDown(self) -> None: shutil.rmtree(self.tempPath) def test_maven_jar_loaded(self) -> None: - packages: List[str] = self.spark.conf.get("spark.jars.packages").split(",") - + packagesConf: Optional[str] = self.spark.conf.get("spark.jars.packages") + assert packagesConf is not None # mypi needs this to assign type str from Optional[str] + packages: str = packagesConf + packagesList: List[str] = packages.split(",") # Check `spark.jars.packages` contains `extra_packages` - self.assertTrue(len(packages) == 2, "There should only be 2 packages") + self.assertTrue(len(packagesList) == 2, "There should only be 2 packages") # Read and write Delta table to check that the maven jars are loaded and Delta works. self.spark.range(0, 5).write.format("delta").save(self.tempFile) diff --git a/python/delta/tests/test_sql.py b/python/delta/tests/test_sql.py index 2b9f5e7ebf8..9ca0e3a76ab 100644 --- a/python/delta/tests/test_sql.py +++ b/python/delta/tests/test_sql.py @@ -105,14 +105,23 @@ def read_table() -> DataFrame: self.spark.sql(f"CREATE TABLE {table}(a LONG, b String NOT NULL) USING delta") self.assertEqual(read_table().count(), 0) + self.spark.sql(f"CREATE TABLE {table}_part(a LONG, b String NOT NULL)" + " USING delta PARTITIONED BY (a)") - answer = [("a", "bigint"), ("b", "string"), ("", ""), ("# Partitioning", ""), - ("Not partitioned", "")] + # Unpartitioned table does not include partitioning information in Spark 3.4+ + answer = [("a", "bigint"), ("b", "string")] self.__checkAnswer( self.spark.sql(f"DESCRIBE TABLE {table}").select("col_name", "data_type"), answer, schema=["col_name", "data_type"]) + answer_part = [("a", "bigint"), ("b", "string"), ("# Partition Information", ""), + ("# col_name", "data_type"), ("a", "bigint")] + self.__checkAnswer( + self.spark.sql(f"DESCRIBE TABLE {table}_part").select("col_name", "data_type"), + answer_part, + schema=["col_name", "data_type"]) + self.spark.sql(f"ALTER TABLE {table} CHANGE COLUMN a a LONG AFTER b") self.assertSequenceEqual(["b", "a"], [f.name for f in read_table().schema.fields]) diff --git a/setup.py b/setup.py index 84df4707d1e..6b2ecf4afe1 100644 --- a/setup.py +++ b/setup.py @@ -65,7 +65,7 @@ def run(self): 'delta': ['py.typed'], }, install_requires=[ - 'pyspark>=3.3.0,<3.4.0', + 'pyspark>=3.4.0,<3.5.0', 'importlib_metadata>=1.0.0', ], python_requires='>=3.6',